From 5589ebbbbab66d9b163f81db1076574d963ef5a5 Mon Sep 17 00:00:00 2001 From: Sammi Chen Date: Tue, 12 Nov 2024 15:06:31 +0800 Subject: [PATCH] HDDS-11627. Support getBlock operation on short-circuit channel. --- .../hadoop/hdds/scm/XceiverClientCreator.java | 35 +- .../hadoop/hdds/scm/XceiverClientFactory.java | 18 +- .../hadoop/hdds/scm/XceiverClientGrpc.java | 7 +- .../hadoop/hdds/scm/XceiverClientManager.java | 100 ++- .../hdds/scm/XceiverClientShortCircuit.java | 623 +++++++++++++++ .../hdds/scm/storage/BlockInputStream.java | 136 +++- .../hdds/scm/storage/ChunkInputStream.java | 26 +- .../hdds/scm/storage/DomainSocketFactory.java | 7 +- .../storage/ShortCircuitChunkInputStream.java | 138 ++++ .../apache/hadoop/hdds/DatanodeVersion.java | 2 + .../hdds/client/RatisReplicationConfig.java | 4 + .../hadoop/hdds/client/ReplicationConfig.java | 3 + .../hadoop/hdds/client/ReplicationType.java | 7 +- .../client/ShortCircuitReplicationConfig.java | 129 ++++ .../hadoop/hdds/scm/XceiverClientSpi.java | 15 + .../hadoop/hdds/scm/pipeline/Pipeline.java | 6 +- .../ContainerCommandResponseBuilders.java | 20 +- .../scm/storage/ContainerProtocolCalls.java | 30 +- .../hdds/upgrade/HDDSLayoutFeature.java | 3 +- .../apache/hadoop/ozone/OzoneConfigKeys.java | 6 +- .../apache/hadoop/ozone/common/Checksum.java | 14 + .../hadoop/ozone/util/OzoneNetUtils.java | 24 +- .../hadoop/hdds/utils/TestProtobufUtils.java | 35 + .../ozone/container/ContainerTestHelper.java | 2 +- .../common/helpers/ContainerMetrics.java | 49 +- .../container/common/interfaces/Handler.java | 27 +- .../common/transport/server/Receiver.java | 370 +++++++++ .../server/XceiverServerDomainSocket.java | 317 ++++++++ .../transport/server/XceiverServerGrpc.java | 49 +- .../transport/server/XceiverServerSpi.java | 2 + .../server/ratis/XceiverServerRatis.java | 5 + .../container/keyvalue/KeyValueHandler.java | 62 +- .../keyvalue/impl/ChunkManagerDispatcher.java | 8 + .../keyvalue/impl/FilePerBlockStrategy.java | 21 +- .../keyvalue/interfaces/ChunkManager.java | 17 + .../container/ozoneimpl/OzoneContainer.java | 60 +- .../main/proto/DatanodeClientProtocol.proto | 11 + .../src/main/proto/hdds.proto | 1 + .../client/MockXceiverClientFactory.java | 4 +- hadoop-ozone/integration-test/libhadoop.dylib | Bin 0 -> 156696 bytes hadoop-ozone/integration-test/libhadoop.so | Bin 0 -> 482992 bytes .../hdds/scm/TestXceiverClientManager.java | 13 +- .../hdds/scm/TestXceiverClientManagerSC.java | 101 +++ .../scm/TestXceiverServerDomainSocket.java | 712 ++++++++++++++++++ .../apache/hadoop/ozone/MiniOzoneCluster.java | 2 +- .../client/rpc/read/TestChunkInputStream.java | 16 +- .../rpc/read/TestDomainSocketFactory.java | 76 ++ .../client/rpc/read/TestInputStreamBase.java | 20 +- .../TestShortCircuitChunkInputStream.java | 160 ++++ 49 files changed, 3372 insertions(+), 121 deletions(-) create mode 100644 hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java create mode 100644 hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java create mode 100644 hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ShortCircuitReplicationConfig.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java create mode 100644 hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java create mode 100755 hadoop-ozone/integration-test/libhadoop.dylib create mode 100755 hadoop-ozone/integration-test/libhadoop.so create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManagerSC.java create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverServerDomainSocket.java create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestDomainSocketFactory.java create mode 100644 hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java index 75ae01c10058..4d901e465651 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java @@ -19,8 +19,10 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.client.ClientTrustManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneSecurityUtil; @@ -41,6 +43,8 @@ public static void enableErrorInjection(ErrorInjector injector) { private final boolean topologyAwareRead; private final ClientTrustManager trustManager; private final boolean securityEnabled; + private boolean shortCircuitEnabled; + private DomainSocketFactory domainSocketFactory; public XceiverClientCreator(ConfigurationSource conf) { this(conf, null); @@ -56,13 +60,25 @@ public XceiverClientCreator(ConfigurationSource conf, ClientTrustManager trustMa if (securityEnabled) { Preconditions.checkNotNull(trustManager); } + shortCircuitEnabled = conf.getObject(OzoneClientConfig.class).isShortCircuitEnabled(); + if (shortCircuitEnabled) { + domainSocketFactory = DomainSocketFactory.getInstance(conf); + } } public boolean isSecurityEnabled() { return securityEnabled; } + public boolean isShortCircuitEnabled() { + return shortCircuitEnabled && domainSocketFactory.isServiceReady(); + } + protected XceiverClientSpi newClient(Pipeline pipeline) throws IOException { + return newClient(pipeline, null); + } + + protected XceiverClientSpi newClient(Pipeline pipeline, DatanodeDetails dn) throws IOException { XceiverClientSpi client; switch (pipeline.getType()) { case RATIS: @@ -74,6 +90,9 @@ protected XceiverClientSpi newClient(Pipeline pipeline) throws IOException { case EC: client = new ECXceiverClientGrpc(pipeline, conf, trustManager); break; + case SHORT_CIRCUIT: + client = new XceiverClientShortCircuit(pipeline, conf, dn); + break; case CHAINED: default: throw new IOException("not implemented " + pipeline.getType()); @@ -97,7 +116,14 @@ public void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClie } @Override - public XceiverClientSpi acquireClientForReadData(Pipeline pipeline) throws IOException { + public XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) + throws IOException { + return acquireClient(pipeline); + } + + @Override + public XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) + throws IOException { return acquireClient(pipeline); } @@ -117,7 +143,10 @@ public void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClie } @Override - public void close() throws Exception { - // clients are not tracked, closing each client is the responsibility of users of this class + public void close() { + // clients are not tracked, closing each client is the responsibility of users of this classclass + if (domainSocketFactory != null) { + domainSocketFactory.close(); + } } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java index b7276d645b44..c47321b815d8 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java @@ -53,8 +53,10 @@ public interface XceiverClientFactory extends AutoCloseable { * @return XceiverClientSpi connected to a container * @throws IOException if a XceiverClientSpi cannot be acquired */ - XceiverClientSpi acquireClientForReadData(Pipeline pipeline) - throws IOException; + default XceiverClientSpi acquireClientForReadData(Pipeline pipeline) + throws IOException { + return acquireClientForReadData(pipeline, false); + } /** * Releases a read XceiverClientSpi after use. @@ -73,10 +75,20 @@ void releaseClientForReadData(XceiverClientSpi client, * @return XceiverClientSpi connected to a container * @throws IOException if a XceiverClientSpi cannot be acquired */ - XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware) + default XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware) throws IOException { + return acquireClient(pipeline, topologyAware, false); + } + + XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) + throws IOException; + + XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) throws IOException; void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClient, boolean topologyAware); + default boolean isShortCircuitEnabled() { + return false; + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index c02306f8af8b..4cfc75c5a921 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -86,7 +86,7 @@ * how it works, and how it is integrated with the Ozone client. */ public class XceiverClientGrpc extends XceiverClientSpi { - private static final Logger LOG = + public static final Logger LOG = LoggerFactory.getLogger(XceiverClientGrpc.class); private final Pipeline pipeline; private final ConfigurationSource config; @@ -133,6 +133,7 @@ public XceiverClientGrpc(Pipeline pipeline, ConfigurationSource config, OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_DEFAULT); this.trustManager = trustManager; this.getBlockDNcache = new ConcurrentHashMap<>(); + LOG.info("{} is created for pipeline {}", XceiverClientGrpc.class.getSimpleName(), pipeline); } /** @@ -246,6 +247,10 @@ public synchronized void close() { } } + public boolean isClosed() { + return closed; + } + @Override public Pipeline getPipeline() { return pipeline; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java index 07b704417216..8dd2e0f07049 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java @@ -19,8 +19,13 @@ package org.apache.hadoop.hdds.scm; import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.Config; import org.apache.hadoop.hdds.conf.ConfigGroup; import org.apache.hadoop.hdds.conf.ConfigType; @@ -29,6 +34,9 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.client.ClientTrustManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.ozone.util.OzoneNetUtils; import org.apache.hadoop.security.UserGroupInformation; import com.google.common.annotations.VisibleForTesting; @@ -38,6 +46,7 @@ import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.hadoop.hdds.DatanodeVersion.SHORT_CIRCUIT_READS; import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE; import static org.apache.hadoop.hdds.conf.ConfigTag.PERFORMANCE; import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.NO_REPLICA_FOUND; @@ -64,8 +73,8 @@ public class XceiverClientManager extends XceiverClientCreator { private final Cache clientCache; private final CacheMetrics cacheMetrics; - private static XceiverClientMetrics metrics; + private final ConcurrentHashMap localDNCache; /** * Creates a new XceiverClientManager for non secured ozone cluster. @@ -105,6 +114,7 @@ public void onRemoval( }).build(); cacheMetrics = CacheMetrics.create(clientCache, this); + this.localDNCache = new ConcurrentHashMap<>(); } @VisibleForTesting @@ -117,17 +127,54 @@ public Cache getClientCache() { * * If there is already a cached XceiverClientSpi, simply return * the cached otherwise create a new one. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired + */ + @Override + public XceiverClientSpi acquireClient(Pipeline pipeline) + throws IOException { + return acquireClient(pipeline, false, false); + } + + /** + * Acquires a XceiverClientSpi connected to a container for read. + * + * If there is already a cached XceiverClientSpi, simply return + * the cached otherwise create a new one. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired + */ + @Override + public XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) + throws IOException { + return acquireClient(pipeline, false, allowShortCircuit); + } + + /** + * Acquires a XceiverClientSpi connected to a container capable of + * storing the specified key. + * + * If there is already a cached XceiverClientSpi, simply return + * the cached otherwise create a new one. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired */ @Override public XceiverClientSpi acquireClient(Pipeline pipeline, - boolean topologyAware) throws IOException { + boolean topologyAware, boolean allowShortCircuit) throws IOException { Preconditions.checkNotNull(pipeline); Preconditions.checkArgument(pipeline.getNodes() != null); Preconditions.checkArgument(!pipeline.getNodes().isEmpty(), NO_REPLICA_FOUND); synchronized (clientCache) { - XceiverClientSpi info = getClient(pipeline, topologyAware); + XceiverClientSpi info = getClient(pipeline, topologyAware, allowShortCircuit); info.incrementReference(); return info; } @@ -141,7 +188,7 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, client.decrementReference(); if (invalidateClient) { Pipeline pipeline = client.getPipeline(); - String key = getPipelineCacheKey(pipeline, topologyAware); + String key = getPipelineCacheKey(pipeline, topologyAware, client instanceof XceiverClientShortCircuit); XceiverClientSpi cachedClient = clientCache.getIfPresent(key); if (cachedClient == client) { clientCache.invalidate(key); @@ -150,24 +197,46 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, } } - protected XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware) + protected XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) throws IOException { try { // create different client different pipeline node based on // network topology - String key = getPipelineCacheKey(pipeline, topologyAware); - return clientCache.get(key, () -> newClient(pipeline)); + String key = getPipelineCacheKey(pipeline, topologyAware, allowShortCircuit); + if (key.endsWith(DomainSocketFactory.FEATURE_FLAG)) { + final Pipeline newPipeline = Pipeline.newBuilder(pipeline).setReplicationConfig( + ReplicationConfig.fromTypeAndFactor(ReplicationType.SHORT_CIRCUIT, + ReplicationFactor.valueOf(pipeline.getReplicationConfig().getReplication()))).build(); + return clientCache.get(key, () -> newClient(newPipeline, localDNCache.get(key))); + } else { + return clientCache.get(key, () -> newClient(pipeline)); + } } catch (Exception e) { throw new IOException( "Exception getting XceiverClient: " + e, e); } } - private String getPipelineCacheKey(Pipeline pipeline, - boolean topologyAware) { - String key = pipeline.getId().getId().toString() + pipeline.getType(); + private String getPipelineCacheKey(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) { + String key = pipeline.getId().getId().toString() + "-" + pipeline.getType(); boolean isEC = pipeline.getType() == HddsProtos.ReplicationType.EC; - if (topologyAware || isEC) { + if ((!isEC) && allowShortCircuit && isShortCircuitEnabled()) { + int port = 0; + for (DatanodeDetails dn : pipeline.getNodes()) { + // read port from the data node, on failure use default configured port. + port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue(); + InetSocketAddress addr = NetUtils.createSocketAddr(dn.getIpAddress(), port); + if (OzoneNetUtils.isAddressLocal(addr) && + dn.getCurrentVersion() >= SHORT_CIRCUIT_READS.toProtoValue()) { + // Find a local DN and short circuit read is enabled + key += "@" + addr.getHostName() + ":" + port + "/" + DomainSocketFactory.FEATURE_FLAG; + localDNCache.put(key, dn); + break; + } + } + } + + if ((!allowShortCircuit && topologyAware) || isEC) { try { DatanodeDetails closestNode = pipeline.getClosestNode(); // Pipeline cache key uses host:port suffix to handle @@ -185,7 +254,7 @@ private String getPipelineCacheKey(Pipeline pipeline, // Standalone port is chosen since all datanodes should have a // standalone port regardless of version and this port should not // have any collisions. - key += closestNode.getHostName() + closestNode.getPort( + key += closestNode.getHostName() + ":" + closestNode.getPort( DatanodeDetails.Port.Name.STANDALONE); } catch (IOException e) { LOG.error("Failed to get closest node to create pipeline cache key:" + @@ -197,12 +266,13 @@ private String getPipelineCacheKey(Pipeline pipeline, // Append user short name to key to prevent a different user // from using same instance of xceiverClient. try { - key += UserGroupInformation.getCurrentUser().getShortUserName(); + key = UserGroupInformation.getCurrentUser().getShortUserName() + "@" + key; } catch (IOException e) { LOG.error("Failed to get current user to create pipeline cache key:" + e.getMessage()); } } + LOG.info("cache key {} for pipeline {}", key, pipeline); return key; } @@ -211,12 +281,14 @@ private String getPipelineCacheKey(Pipeline pipeline, */ @Override public void close() { + super.close(); //closing is done through RemovalListener clientCache.invalidateAll(); clientCache.cleanUp(); if (LOG.isDebugEnabled()) { LOG.debug("XceiverClient cache stats: {}", clientCache.stats()); } + localDNCache.clear(); cacheMetrics.unregister(); if (metrics != null) { @@ -263,7 +335,7 @@ public static class ScmClientConfig { @Config(key = "idle.threshold", type = ConfigType.TIME, timeUnit = MILLISECONDS, - defaultValue = "10s", + defaultValue = "300s", tags = {OZONE, PERFORMANCE}, description = "In the standalone pipelines, the SCM clients use netty to " diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java new file mode 100644 index 000000000000..8f2b5c538e27 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java @@ -0,0 +1,623 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.hadoop.hdds.scm; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; +import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.thirdparty.protobuf.CodedInputStream; +import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.LimitInputStream; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.Status; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InterruptedIOException; +import java.net.InetSocketAddress; +import java.nio.channels.ClosedChannelException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Timer; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantLock; + +import static org.apache.hadoop.hdds.HddsUtils.processForDebug; +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_MAGIC_CODE; +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_VERSION; + +/** + * {@link XceiverClientSpi} implementation, the client to read local replica through short circuit. + */ +public class XceiverClientShortCircuit extends XceiverClientSpi { + public static final Logger LOG = + LoggerFactory.getLogger(XceiverClientShortCircuit.class); + private final Pipeline pipeline; + private final ConfigurationSource config; + private final XceiverClientMetrics metrics; + private int readTimeoutMs; + private int writeTimeoutMs; + // Cache the stream of blocks + private final Map blockStreamCache; + private final Map sentRequests; + private final Daemon readDaemon; + private Timer timer; + + private boolean closed = false; + private final DatanodeDetails dn; + private final InetSocketAddress dnAddr; + private final DomainSocketFactory domainSocketFactory; + private DomainSocket domainSocket; + private AtomicBoolean isDomainSocketOpen = new AtomicBoolean(false); + private Lock lock = new ReentrantLock(); + private final int bufferSize; + private final ByteString clientId = ByteString.copyFrom(UUID.randomUUID().toString().getBytes()); + private final AtomicLong callId = new AtomicLong(0); + private String prefix; + + /** + * Constructs a client that can communicate with the Container framework on local datanode through DomainSocket + */ + public XceiverClientShortCircuit(Pipeline pipeline, ConfigurationSource config, DatanodeDetails dn) { + super(); + Preconditions.checkNotNull(config); + this.readTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + this.writeTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + + this.pipeline = pipeline; + this.dn = dn; + this.domainSocketFactory = DomainSocketFactory.getInstance(config); + this.config = config; + this.metrics = XceiverClientManager.getXceiverClientMetrics(); + this.blockStreamCache = new ConcurrentHashMap<>(); + this.sentRequests = new ConcurrentHashMap<>(); + int port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue(); + this.dnAddr = NetUtils.createSocketAddr(dn.getIpAddress(), port); + this.bufferSize = config.getObject(OzoneClientConfig.class).getShortCircuitBufferSize(); + this.readDaemon = new Daemon(new ReceiveResponseTask()); + LOG.info("{} is created for pipeline {}", XceiverClientShortCircuit.class.getSimpleName(), pipeline); + } + + /** + * Create the DomainSocket to connect to the local DataNode. + */ + @Override + public void connect() throws IOException { + // Even the in & out stream has returned EOFException, domainSocket.isOpen() is still true. + if (domainSocket != null && domainSocket.isOpen() && isDomainSocketOpen.get()) { + return; + } + domainSocket = domainSocketFactory.createSocket(readTimeoutMs, writeTimeoutMs, dnAddr); + isDomainSocketOpen.set(true); + prefix = XceiverClientShortCircuit.class.getSimpleName() + "-" + domainSocket.toString(); + timer = new Timer(prefix + "-Timer"); + readDaemon.start(); + LOG.info("{} is started", prefix); + } + + /** + * Close the DomainSocket. + */ + @Override + public synchronized void close() { + closed = true; + timer.cancel(); + if (domainSocket != null) { + try { + isDomainSocketOpen.set(false); + domainSocket.close(); + LOG.info("{} is closed for {}", domainSocket.toString(), dn); + } catch (IOException e) { + LOG.warn("Failed to close domain socket for datanode {}", dn, e); + } + } + readDaemon.interrupt(); + try { + readDaemon.join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + public boolean isClosed() { + return closed; + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + public DatanodeDetails getDn() { + return this.dn; + } + + public ByteString getClientId() { + return clientId; + } + + public long getCallId() { + return callId.incrementAndGet(); + } + + @Override + public ContainerCommandResponseProto sendCommand(ContainerCommandRequestProto request) throws IOException { + try { + return sendCommandWithTraceID(request, null).getResponse().get(); + } catch (ExecutionException e) { + throw getIOExceptionForSendCommand(request, e); + } catch (InterruptedException e) { + LOG.error("Command execution was interrupted."); + Thread.currentThread().interrupt(); + throw (IOException) new InterruptedIOException( + "Command " + processForDebug(request) + " was interrupted.") + .initCause(e); + } + } + + @Override + public Map + sendCommandOnAllNodes( + ContainerCommandRequestProto request) throws IOException { + throw new UnsupportedOperationException("Operation Not supported for " + + DomainSocketFactory.FEATURE + " client"); + } + + @Override + public ContainerCommandResponseProto sendCommand( + ContainerCommandRequestProto request, List validators) + throws IOException { + try { + XceiverClientReply reply; + reply = sendCommandWithTraceID(request, validators); + return reply.getResponse().get(); + } catch (ExecutionException e) { + throw getIOExceptionForSendCommand(request, e); + } catch (InterruptedException e) { + LOG.error("Command execution was interrupted."); + Thread.currentThread().interrupt(); + throw (IOException) new InterruptedIOException( + "Command " + processForDebug(request) + " was interrupted.") + .initCause(e); + } + } + + private XceiverClientReply sendCommandWithTraceID( + ContainerCommandRequestProto request, List validators) + throws IOException { + String spanName = "XceiverClientGrpc." + request.getCmdType().name(); + return TracingUtil.executeInNewSpan(spanName, + () -> { + ContainerCommandRequestProto finalPayload = + ContainerCommandRequestProto.newBuilder(request) + .setTraceID(TracingUtil.exportCurrentSpan()).build(); + ContainerCommandResponseProto responseProto = null; + IOException ioException = null; + + // In case of an exception or an error, we will try to read from the + // datanodes in the pipeline in a round-robin fashion. + XceiverClientReply reply = new XceiverClientReply(null); + + if (request.getCmdType() != ContainerProtos.Type.GetBlock && + request.getCmdType() != ContainerProtos.Type.Echo) { + throw new UnsupportedOperationException("Command " + request.getCmdType() + + " is not supported for " + DomainSocketFactory.FEATURE + " client"); + } + + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Executing command {} on datanode {}", request, dn); + } + reply.addDatanode(dn); + responseProto = sendCommandInternal(finalPayload).getResponse().get(); + if (validators != null && !validators.isEmpty()) { + for (Validator validator : validators) { + validator.accept(request, responseProto); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("request {} {} {} finished", request.getCmdType(), + request.getClientId().toStringUtf8(), request.getCallId()); + } + } catch (IOException e) { + ioException = e; + responseProto = null; + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to execute command {} on datanode {}", request, dn, e); + } + } catch (ExecutionException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to execute command {} on datanode {}", request, dn, e); + } + if (Status.fromThrowable(e.getCause()).getCode() + == Status.UNAUTHENTICATED.getCode()) { + throw new SCMSecurityException("Failed to authenticate with " + + "datanode DomainSocket XceiverServer with Ozone block token."); + } + ioException = new IOException(e); + } catch (InterruptedException e) { + LOG.error("Command execution was interrupted ", e); + Thread.currentThread().interrupt(); + } + + if (responseProto != null) { + reply.setResponse(CompletableFuture.completedFuture(responseProto)); + return reply; + } else { + Objects.requireNonNull(ioException); + String message = "Failed to execute command {}"; + if (LOG.isDebugEnabled()) { + LOG.debug(message + " on the datanode {}.", request, dn, ioException); + } else { + LOG.error(message + " on the datanode {}.", request, dn, ioException); + } + throw ioException; + } + }); + } + + @VisibleForTesting + public XceiverClientReply sendCommandInternal(ContainerCommandRequestProto request) + throws IOException, InterruptedException { + checkOpen(); + final CompletableFuture replyFuture = + new CompletableFuture<>(); + RequestEntry entry = new RequestEntry(request, replyFuture); + sendRequest(entry); + return new XceiverClientReply(replyFuture); + } + + @Override + public XceiverClientReply sendCommandAsync( + ContainerCommandRequestProto request) + throws IOException, ExecutionException, InterruptedException { + throw new UnsupportedOperationException("Operation Not supported for " + DomainSocketFactory.FEATURE + " client"); + } + + public synchronized void checkOpen() throws IOException { + if (closed) { + throw new IOException("DomainSocket is not connected."); + } + + if (!isDomainSocketOpen.get()) { + throw new IOException("DomainSocket stream is not open."); + } + } + + @Override + public CompletableFuture watchForCommit(long index) { + // there is no notion of watch for commit index in short-circuit local reads + return null; + } + + @Override + public long getReplicatedMinCommitIndex() { + return 0; + } + + public FileInputStream getFileInputStream(long callId, DatanodeBlockID blockID) { + String mapKey = callId + blockID.toString(); + return blockStreamCache.remove(mapKey); + } + + @Override + public HddsProtos.ReplicationType getPipelineType() { + return HddsProtos.ReplicationType.STAND_ALONE; + } + + public ConfigurationSource getConfig() { + return config; + } + + @VisibleForTesting + public static Logger getLogger() { + return LOG; + } + + public void setReadTimeout(int timeout) { + this.readTimeoutMs = timeout; + } + + public int getReadTimeout() { + return this.readTimeoutMs; + } + + String getRequestUniqueID(ContainerCommandRequestProto request) { + return request.getClientId().toStringUtf8() + request.getCallId(); + } + + String getRequestUniqueID(ContainerCommandResponseProto response) { + return response.getClientId().toStringUtf8() + response.getCallId(); + } + + void requestTimeout(String requestId) { + final RequestEntry entry = sentRequests.remove(requestId); + if (entry != null) { + LOG.warn("Timeout to receive response for command {}", entry.getRequest());; + ContainerProtos.Type type = entry.getRequest().getCmdType(); + metrics.decrPendingContainerOpsMetrics(type); + entry.getFuture().completeExceptionally(new TimeoutException("Timeout to receive response")); + } + } + + public void sendRequest(RequestEntry entry) { + ContainerCommandRequestProto request = entry.getRequest(); + try { + String key = getRequestUniqueID(request); + TimerTask task = new TimerTask() { + @Override + public void run() { + requestTimeout(key); + } + }; + entry.setTimerTask(task); + timer.schedule(task, readTimeoutMs); + sentRequests.put(key, entry); + ContainerProtos.Type type = request.getCmdType(); + metrics.incrPendingContainerOpsMetrics(type); + byte[] bytes = request.toByteArray(); + if (bytes.length != request.getSerializedSize()) { + throw new IOException("Serialized request " + request.getCmdType() + + " size mismatch, byte array size " + bytes.length + + ", serialized size " + request.getSerializedSize()); + } + + DataOutputStream dataOut = + new DataOutputStream(new BufferedOutputStream(domainSocket.getOutputStream(), bufferSize)); + lock.lock(); + try { + // send version number + dataOut.writeShort(DATA_TRANSFER_VERSION); + // send command type + dataOut.writeShort(type.getNumber()); + // send request body + request.writeDelimitedTo(dataOut); + dataOut.flush(); + } finally { + lock.unlock(); + entry.setSentTimeNs(); + } + } catch (IOException e) { + LOG.error("Failed to send command {}", request, e); + entry.getFuture().completeExceptionally(e); + metrics.decrPendingContainerOpsMetrics(request.getCmdType()); + metrics.addContainerOpsLatency(request.getCmdType(), System.nanoTime() - entry.getCreateTimeNs()); + } + } + + @Override + public String toString() { + final StringBuilder b = + new StringBuilder(getClass().getSimpleName()).append("["); + b.append(" DomainSocket: ").append(domainSocket.toString()); + b.append(" Pipeline: ").append(pipeline.toString()); + return b.toString(); + } + + public class ReceiveResponseTask implements Runnable { + @Override + public void run() { + long timerTaskCancelledCount = 0; + do { + Thread.currentThread().setName(prefix + "-ReceiveResponse"); + RequestEntry entry = null; + try { + DataInputStream dataIn = new DataInputStream(domainSocket.getInputStream()); + final short version = dataIn.readShort(); + if (version != DATA_TRANSFER_VERSION) { + throw new IOException("Version Mismatch (Expected: " + + DATA_TRANSFER_VERSION + ", Received: " + version + ")"); + } + long receiveStartTime = System.nanoTime(); + final short typeNumber = dataIn.readShort(); + ContainerProtos.Type type = ContainerProtos.Type.forNumber(typeNumber); + ContainerCommandResponseProto responseProto = + ContainerCommandResponseProto.parseFrom(vintPrefixed(dataIn)); + if (LOG.isDebugEnabled()) { + LOG.debug("received response {} callId {}", type, responseProto.getCallId()); + } + String key = getRequestUniqueID(responseProto); + entry = sentRequests.remove(key); + if (entry == null) { + // This could be two cases + // 1. there is bug in the code + // 2. the response is too late, the request is removed from sentRequests after it is timeout. + throw new IOException("Failed to find request for response, type " + type + + ", clientId " + responseProto.getClientId().toStringUtf8() + ", callId " + responseProto.getCallId()); + } + + // cancel timeout timer task + if (entry.getTimerTask().cancel()) { + timerTaskCancelledCount++; + // purge timer every 1000 cancels + if (timerTaskCancelledCount == 1000) { + timer.purge(); + timerTaskCancelledCount = 0; + } + } + + long processStartTime = System.nanoTime(); + ContainerProtos.Result result = responseProto.getResult(); + if (result == ContainerProtos.Result.SUCCESS) { + if (type == ContainerProtos.Type.GetBlock) { + try { + ContainerProtos.GetBlockResponseProto getBlockResponse = responseProto.getGetBlock(); + if (!getBlockResponse.getShortCircuitAccessGranted()) { + throw new IOException("Short-circuit access is denied on " + dn); + } + // read FS from domainSocket + FileInputStream[] fis = new FileInputStream[1]; + byte buf[] = new byte[1]; + int ret = domainSocket.recvFileInputStreams(fis, buf, 0, buf.length); + if (ret == -1) { + throw new IOException("failed to get a file descriptor from datanode " + dn + + " for peer is shutdown."); + } + if (fis[0] == null) { + throw new IOException("the datanode " + dn + " failed to " + + "pass a file descriptor (might have reached open file limit)."); + } + if (buf[0] != DATA_TRANSFER_MAGIC_CODE) { + throw new IOException("Magic Code Mismatch (Expected: " + + DATA_TRANSFER_MAGIC_CODE + ", Received: " + buf[0] + ")"); + } + DatanodeBlockID blockID = getBlockResponse.getBlockData().getBlockID(); + String mapKey = responseProto.getCallId() + blockID.toString(); + blockStreamCache.put(mapKey, fis[0]); + if (LOG.isDebugEnabled()) { + LOG.debug("received fd {} ", fis[0]); + } + } catch (IOException e) { + LOG.warn("Failed to handle short-circuit information exchange", e); + // disable docket socket for a while + domainSocketFactory.disableShortCircuit(); + entry.getFuture().completeExceptionally(e); + continue; + } + } + entry.getFuture().complete(responseProto); + } else { + // response result is not SUCCESS + entry.getFuture().complete(responseProto); + } + long currentTime = System.nanoTime(); + long endToEndCost = currentTime - entry.getCreateTimeNs(); + long sentCost = entry.getSentTimeNs() - entry.getCreateTimeNs(); + long receiveCost = processStartTime - receiveStartTime; + long processCost = currentTime - processStartTime; + if (LOG.isDebugEnabled()) { + LOG.debug("Executed command {} {} on datanode {}, end-to-end {} ns, sent {} ns, receive {} ns, " + + " process {} ns, cmdType = {}, {}, {}", + entry.getRequest().getClientId().toStringUtf8(), entry.getRequest().getCallId(), + dn, endToEndCost, sentCost, receiveCost, processCost, type, entry.getSentTimeNs(), entry.getCreateTimeNs()); + } + metrics.decrPendingContainerOpsMetrics(type); + metrics.addContainerOpsLatency(type, endToEndCost); + } catch (EOFException | ClosedChannelException e) { + isDomainSocketOpen.set(false); + LOG.info("ReceiveResponseTask is closed due to {}", e.getClass().getName()); + // fail all requests pending responses + sentRequests.values().forEach(i -> i.fail(e)); + } catch (Throwable e) { + isDomainSocketOpen.set(false); + LOG.error("Failed to receive response", e); + if (entry != null) { + entry.getFuture().completeExceptionally(e); + } + sentRequests.values().forEach(i -> i.fail(e)); + break; + } + } while (!isClosed()); + } + } + + public static InputStream vintPrefixed(final InputStream input) + throws IOException { + final int firstByte = input.read(); + if (firstByte == -1) { + throw new EOFException( + "Unexpected EOF while trying to read response from server"); + } + + int size = CodedInputStream.readRawVarint32(firstByte, input); + assert size >= 0; + return new LimitInputStream(input, size); + } + + public class RequestEntry { + private ContainerCommandRequestProto request; + private CompletableFuture future; + private long createTimeNs; + private long sentTimeNs; + private TimerTask timerTask; + + RequestEntry(ContainerCommandRequestProto requestProto, + CompletableFuture future) { + this.request = requestProto; + this.future = future; + this.createTimeNs = System.nanoTime(); + } + + public ContainerCommandRequestProto getRequest() { + return request; + } + + public CompletableFuture getFuture() { + return future; + } + + public long getCreateTimeNs() { + return createTimeNs; + } + + public long getSentTimeNs() { + return sentTimeNs; + } + + public void setSentTimeNs() { + sentTimeNs = System.nanoTime(); + } + + public void setTimerTask(TimerTask task) { + timerTask = task; + } + + public TimerTask getTimerTask() { + return timerTask; + } + + public void fail(Throwable e) { + timerTask.cancel(); + future.completeExceptionally(e); + } + } +} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index d6353be9d22d..5fa6828484e9 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.storage; import java.io.EOFException; +import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.time.Instant; @@ -26,12 +27,14 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; @@ -40,6 +43,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientShortCircuit; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; @@ -77,7 +81,10 @@ public class BlockInputStream extends BlockExtendedInputStream { new AtomicReference<>(); private final boolean verifyChecksum; private XceiverClientFactory xceiverClientFactory; - private XceiverClientSpi xceiverClient; + private XceiverClientSpi xceiverClientGrpc; + private XceiverClientShortCircuit xceiverClientShortCircuit; + private final AtomicBoolean fallbackToGrpc = new AtomicBoolean(false); + private FileInputStream blockInputStream; private boolean initialized = false; // TODO: do we need to change retrypolicy based on exception. private final RetryPolicy retryPolicy; @@ -269,14 +276,75 @@ protected BlockData getBlockData() throws IOException { * @return BlockData. */ protected BlockData getBlockDataUsingClient() throws IOException { - Pipeline pipeline = pipelineRef.get(); + if (xceiverClientShortCircuit != null) { + try { + return getBlockDataUsingSCClient(); + } catch (IOException e) { + LOG.warn("Failed to get blockData using short-circuit client", e); + // acquire client again if xceiverClientGrpc is not acquired. + acquireClient(); + } + } + return getBlockDataUsingGRPCClient(); + } + + @VisibleForTesting + protected BlockData getBlockDataUsingSCClient() throws IOException { + final Pipeline pipeline = xceiverClientShortCircuit.getPipeline(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Initializing BlockInputStream for get key to access {}", + blockID.getContainerID()); + } + + DatanodeBlockID.Builder blkIDBuilder = + DatanodeBlockID.newBuilder().setContainerID(blockID.getContainerID()) + .setLocalID(blockID.getLocalID()) + .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId()); + + int replicaIndex = pipeline.getReplicaIndex(xceiverClientShortCircuit.getDn()); + if (replicaIndex > 0) { + blkIDBuilder.setReplicaIndex(replicaIndex); + } + DatanodeBlockID datanodeBlockID = blkIDBuilder.build(); + ContainerProtos.GetBlockRequestProto.Builder readBlockRequest = + ContainerProtos.GetBlockRequestProto.newBuilder().setBlockID(datanodeBlockID) + .setRequestShortCircuitAccess(true); + ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.GetBlock) + .setContainerID(datanodeBlockID.getContainerID()) + .setGetBlock(readBlockRequest) + .setClientId(xceiverClientShortCircuit.getClientId()) + .setCallId(xceiverClientShortCircuit.getCallId()); + if (tokenRef.get() != null) { + builder.setEncodedToken(tokenRef.get().encodeToUrlString()); + } + GetBlockResponseProto response = ContainerProtocolCalls.getBlock(xceiverClientShortCircuit, + VALIDATORS, builder, xceiverClientShortCircuit.getDn()); + + blockInputStream = xceiverClientShortCircuit.getFileInputStream(builder.getCallId(), datanodeBlockID); + if (blockInputStream == null) { + throw new IOException("Failed to get file InputStream for block " + datanodeBlockID); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Get the FileInputStream of block {}", datanodeBlockID); + } + } + return response.getBlockData(); + } + + @VisibleForTesting + protected BlockData getBlockDataUsingGRPCClient() throws IOException { + final Pipeline pipeline = xceiverClientGrpc.getPipeline(); + if (LOG.isDebugEnabled()) { LOG.debug("Initializing BlockInputStream for get key to access block {}", blockID); } GetBlockResponseProto response = ContainerProtocolCalls.getBlock( - xceiverClient, VALIDATORS, blockID, tokenRef.get(), pipeline.getReplicaIndexes()); + xceiverClientGrpc, VALIDATORS, blockID, tokenRef.get(), pipeline.getReplicaIndexes()); return response.getBlockData(); } @@ -332,13 +400,35 @@ private static void validate(ContainerCommandResponseProto response) } private void acquireClient() throws IOException { - if (xceiverClientFactory != null && xceiverClient == null) { - final Pipeline pipeline = pipelineRef.get(); + final Pipeline pipeline = pipelineRef.get(); + // xceiverClientGrpc not-null indicates there is fall back to GRPC reads + if (xceiverClientFactory != null && xceiverClientFactory.isShortCircuitEnabled() && !fallbackToGrpc.get() + && xceiverClientShortCircuit == null) { try { - xceiverClient = xceiverClientFactory.acquireClientForReadData(pipeline); + XceiverClientSpi newClient = xceiverClientFactory.acquireClientForReadData(pipeline, true); + if (newClient instanceof XceiverClientShortCircuit) { + xceiverClientShortCircuit = (XceiverClientShortCircuit) newClient; + if (LOG.isDebugEnabled()) { + LOG.debug("acquired short-circuit client {} for block {}", xceiverClientShortCircuit.toString(), blockID); + } + } else { + xceiverClientGrpc = newClient; + fallbackToGrpc.set(true); + } + return; + } catch (Exception e) { + LOG.warn("Failed to acquire {} client for pipeline {}, block {}. Fallback to Grpc client.", + DomainSocketFactory.FEATURE, pipeline, blockID, e); + fallbackToGrpc.set(true); + } + } + + // fall back to acquire GRPC client + if (xceiverClientFactory != null && xceiverClientGrpc == null) { + try { + xceiverClientGrpc = xceiverClientFactory.acquireClientForReadData(pipeline, false); } catch (IOException ioe) { - LOG.warn("Failed to acquire client for pipeline {}, block {}", - pipeline, blockID); + LOG.warn("Failed to acquire client for pipeline {}, block {}", pipeline, blockID); throw ioe; } } @@ -354,8 +444,13 @@ protected synchronized void addStream(ChunkInfo chunkInfo) { } protected ChunkInputStream createChunkInputStream(ChunkInfo chunkInfo) { - return new ChunkInputStream(chunkInfo, blockID, - xceiverClientFactory, pipelineRef::get, verifyChecksum, tokenRef::get); + if (blockInputStream != null) { + return new ShortCircuitChunkInputStream(chunkInfo, blockID, xceiverClientFactory, + pipelineRef::get, verifyChecksum, tokenRef::get, xceiverClientShortCircuit, blockInputStream); + } else { + return new ChunkInputStream(chunkInfo, blockID, + xceiverClientFactory, pipelineRef::get, verifyChecksum, tokenRef::get); + } } @Override @@ -527,12 +622,23 @@ public synchronized void close() { is.close(); } } + if (blockInputStream != null) { + try { + blockInputStream.close(); + } catch (IOException e) { + LOG.error("Failed to close file InputStream for block " + blockID, e); + } + } } private void releaseClient() { - if (xceiverClientFactory != null && xceiverClient != null) { - xceiverClientFactory.releaseClientForReadData(xceiverClient, false); - xceiverClient = null; + if (xceiverClientFactory != null && xceiverClientGrpc != null) { + xceiverClientFactory.releaseClientForReadData(xceiverClientGrpc, false); + xceiverClientGrpc = null; + } + if (xceiverClientFactory != null && xceiverClientShortCircuit != null) { + xceiverClientFactory.releaseClientForReadData(xceiverClientShortCircuit, false); + xceiverClientShortCircuit = null; } } @@ -567,6 +673,10 @@ synchronized long getBlockPosition() { return blockPosition; } + public synchronized FileInputStream getBlockInputStream() { + return blockInputStream; + } + @Override public synchronized void unbuffer() { storePosition(); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index 983bb74989ad..615c582c3857 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -64,15 +64,15 @@ public class ChunkInputStream extends InputStream private final BlockID blockID; private ContainerProtos.DatanodeBlockID datanodeBlockID; private final XceiverClientFactory xceiverClientFactory; - private XceiverClientSpi xceiverClient; + protected XceiverClientSpi xceiverClient; private final Supplier pipelineSupplier; - private final boolean verifyChecksum; - private boolean allocated = false; + protected final boolean verifyChecksum; + protected boolean allocated = false; // Buffers to store the chunk data read from the DN container - private ByteBuffer[] buffers; + protected ByteBuffer[] buffers; // Index of the buffers corresponding to the current position of the buffers - private int bufferIndex; + protected int bufferIndex; // bufferOffsets[i] stores the index of the first data byte in buffer i // (buffers.get(i)) w.r.t first byte in the buffers. // Let's say each buffer has a capacity of 40 bytes. The bufferOffset for @@ -80,27 +80,27 @@ public class ChunkInputStream extends InputStream // in buffers. BufferOffset for the 2nd buffer would be 40 as bytes 0-39 // would be stored in buffer 0. Hence, bufferOffsets[0] = 0, // bufferOffsets[1] = 40, bufferOffsets[2] = 80, etc. - private long[] bufferOffsets = null; + protected long[] bufferOffsets = null; // The offset of the current data residing in the buffers w.r.t the start // of chunk data - private long bufferOffsetWrtChunkData; + protected long bufferOffsetWrtChunkData; // Index of the first buffer which has not been released - private int firstUnreleasedBufferIndex = 0; + protected int firstUnreleasedBufferIndex = 0; // The number of bytes of chunk data residing in the buffers currently - private long buffersSize; + protected long buffersSize; // Position of the ChunkInputStream is maintained by this variable (if a // seek is performed. This position is w.r.t to the chunk only and not the // block or key. This variable is also set before attempting a read to enable // retry. Once the chunk is read, this variable is reset. - private long chunkPosition = -1; + protected long chunkPosition = -1; - private final Supplier> tokenSupplier; + protected final Supplier> tokenSupplier; - private static final int EOF = -1; + protected static final int EOF = -1; private final List validators; ChunkInputStream(ChunkInfo chunkInfo, BlockID blockId, @@ -414,7 +414,7 @@ private synchronized void readChunkFromContainer(int len) throws IOException { adjustBufferPosition(startByteIndex - bufferOffsetWrtChunkData); } - private void readChunkDataIntoBuffers(ChunkInfo readChunkInfo) + protected void readChunkDataIntoBuffers(ChunkInfo readChunkInfo) throws IOException { buffers = readChunk(readChunkInfo); buffersSize = readChunkInfo.getLen(); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java index e62e2a6bfd2a..8a8f3e265748 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java @@ -138,14 +138,14 @@ public static DomainSocketFactory getInstance(ConfigurationSource conf) { private DomainSocketFactory(ConfigurationSource conf) { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); boolean shortCircuitEnabled = clientConfig.isShortCircuitEnabled(); + domainSocketPath = conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH_DEFAULT); PathInfo pathInfo; long startTime = System.nanoTime(); if (!shortCircuitEnabled) { LOG.info(FEATURE + " is disabled."); pathInfo = PathInfo.NOT_CONFIGURED; } else { - domainSocketPath = conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, - OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH_DEFAULT); if (domainSocketPath.isEmpty()) { throw new IllegalArgumentException(FEATURE + " is enabled but " + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH + " is not set."); @@ -268,5 +268,8 @@ public void close() { if (timer != null) { timer.cancel(); } + synchronized (DomainSocketFactory.class) { + instance = null; + } } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java new file mode 100644 index 000000000000..0210d8b943dc --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.hadoop.hdds.scm.storage; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.fs.ByteBufferReadable; +import org.apache.hadoop.fs.CanUnbuffer; +import org.apache.hadoop.fs.Seekable; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientShortCircuit; +import org.apache.hadoop.hdds.scm.XceiverClientSpi.ShortCircuitValidator; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChecksumData; +import org.apache.hadoop.ozone.common.OzoneChecksumException; +import org.apache.hadoop.ozone.common.utils.BufferUtils; +import org.apache.hadoop.security.token.Token; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; + +/** + * An {@link InputStream} called from BlockInputStream to read a chunk from the + * container. Each chunk may contain multiple underlying {@link ByteBuffer} + * instances. + */ +public class ShortCircuitChunkInputStream extends ChunkInputStream + implements Seekable, CanUnbuffer, ByteBufferReadable { + + private final ChunkInfo chunkInfo; + private final FileInputStream blockInputStream; + private final FileChannel dataIn; + private final ShortCircuitValidator validator; + private final XceiverClientShortCircuit xceiverClientShortCircuit; + public static final Logger LOG = + LoggerFactory.getLogger(ShortCircuitChunkInputStream.class); + + ShortCircuitChunkInputStream(ChunkInfo chunkInfo, BlockID blockId, XceiverClientFactory xceiverClientFactory, + Supplier pipelineSupplier, boolean verifyChecksum, Supplier> tokenSupplier, + XceiverClientShortCircuit xceiverClientShortCircuit, FileInputStream blockInputStream) { + super(chunkInfo, blockId, xceiverClientFactory, pipelineSupplier, verifyChecksum, tokenSupplier); + this.chunkInfo = chunkInfo; + this.blockInputStream = blockInputStream; + this.dataIn = blockInputStream.getChannel(); + this.xceiverClientShortCircuit = xceiverClientShortCircuit; + this.validator = this::validateChunk; + if (LOG.isDebugEnabled()) { + LOG.debug("{} is created for {}", ShortCircuitChunkInputStream.class.getSimpleName(), blockId); + } + } + + @Override + protected void readChunkDataIntoBuffers(ChunkInfo readChunkInfo) + throws IOException { + buffers = readChunk(readChunkInfo); + buffersSize = readChunkInfo.getLen(); + + bufferOffsets = new long[buffers.length]; + int tempOffset = 0; + for (int i = 0; i < buffers.length; i++) { + bufferOffsets[i] = tempOffset; + tempOffset += buffers[i].limit(); + } + + bufferIndex = 0; + firstUnreleasedBufferIndex = 0; + allocated = true; + } + + /** + * Send RPC call to get the chunk from the container. + */ + @VisibleForTesting + @Override + protected ByteBuffer[] readChunk(ChunkInfo readChunkInfo) + throws IOException { + int bytesPerChecksum = chunkInfo.getChecksumData().getBytesPerChecksum(); + final ByteBuffer[] buffers = BufferUtils.assignByteBuffers(readChunkInfo.getLen(), + bytesPerChecksum); + dataIn.position(readChunkInfo.getOffset()).read(buffers); + Arrays.stream(buffers).forEach(ByteBuffer::flip); + validator.accept(Arrays.asList(buffers), readChunkInfo); + return buffers; + } + + private void validateChunk(List bufferList, ChunkInfo readChunkInfo) + throws OzoneChecksumException { + if (verifyChecksum) { + ChecksumData checksumData = ChecksumData.getFromProtoBuf( + chunkInfo.getChecksumData()); + + // ChecksumData stores checksum for each 'numBytesPerChecksum' + // number of bytes in a list. Compute the index of the first + // checksum to match with the read data + + long relativeOffset = readChunkInfo.getOffset() - + chunkInfo.getOffset(); + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + int startIndex = (int) (relativeOffset / bytesPerChecksum); + Checksum.verifyChecksum(bufferList, checksumData, startIndex); + } + } + + + /** + * Acquire short-circuit local read client. + */ + @Override + protected synchronized void acquireClient() throws IOException { + // do nothing, read data doesn't need short-circuit client + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java index e35d20d53e15..0b1d538fb184 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java @@ -34,6 +34,8 @@ public enum DatanodeVersion implements ComponentVersion { COMBINED_PUTBLOCK_WRITECHUNK_RPC(2, "WriteChunk can optionally support " + "a PutBlock request"), + SHORT_CIRCUIT_READS(3, "Support short-circuit reads."), + FUTURE_VERSION(-1, "Used internally in the client when the server side is " + " newer and an unknown server version has arrived to the client."); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java index 36d4d90e1afb..d0e2e6e8d09a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java @@ -23,6 +23,8 @@ import net.jcip.annotations.Immutable; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Objects; @@ -45,6 +47,8 @@ public final class RatisReplicationConfig private static final RatisReplicationConfig RATIS_THREE_CONFIG = new RatisReplicationConfig(THREE); + private static final Logger LOG = + LoggerFactory.getLogger(RatisReplicationConfig.class); /** * Get an instance of Ratis Replication Config with the requested factor. * The same static instance will be returned for all requests for the same diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java index 7542409679b0..57db149047e4 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java @@ -49,6 +49,8 @@ static ReplicationConfig fromProtoTypeAndFactor( return RatisReplicationConfig.getInstance(factor); case STAND_ALONE: return StandaloneReplicationConfig.getInstance(factor); + case SHORT_CIRCUIT: + return ShortCircuitReplicationConfig.getInstance(factor); default: throw new UnsupportedOperationException( "Not supported replication: " + type); @@ -102,6 +104,7 @@ static ReplicationConfig fromProto( return new ECReplicationConfig(ecConfig); case RATIS: case STAND_ALONE: + case SHORT_CIRCUIT: return fromProtoTypeAndFactor(type, factor); default: throw new UnsupportedOperationException( diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java index 64969eac4226..f4dc891f47a7 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java @@ -27,7 +27,8 @@ public enum ReplicationType { RATIS, STAND_ALONE, CHAINED, - EC; + EC, + SHORT_CIRCUIT; public static ReplicationType fromProto( HddsProtos.ReplicationType replicationType) { @@ -43,6 +44,8 @@ public static ReplicationType fromProto( return ReplicationType.CHAINED; case EC: return ReplicationType.EC; + case SHORT_CIRCUIT: + return ReplicationType.SHORT_CIRCUIT; default: throw new IllegalArgumentException( "Unsupported ProtoBuf replication type: " + replicationType); @@ -63,6 +66,8 @@ public static HddsProtos.ReplicationType toProto( return HddsProtos.ReplicationType.CHAINED; case EC: return HddsProtos.ReplicationType.EC; + case SHORT_CIRCUIT: + return HddsProtos.ReplicationType.SHORT_CIRCUIT; default: throw new IllegalArgumentException( "Unsupported replication type: " + replicationType); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ShortCircuitReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ShortCircuitReplicationConfig.java new file mode 100644 index 000000000000..aa72e8cacf3e --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ShortCircuitReplicationConfig.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.client; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import net.jcip.annotations.Immutable; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; + +import java.util.Objects; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; + +/** + * Replication configuration for SHORT_CIRCUIT replication. + */ +@Immutable +public final class ShortCircuitReplicationConfig implements + ReplicatedReplicationConfig { + + private final ReplicationFactor replicationFactor; + private static final String REPLICATION_TYPE = "SHORT_CIRCUIT"; + + private static final ShortCircuitReplicationConfig SHORT_CIRCUIT_ONE_CONFIG = + new ShortCircuitReplicationConfig(ONE); + + private static final ShortCircuitReplicationConfig SHORT_CIRCUIT_THREE_CONFIG = + new ShortCircuitReplicationConfig(THREE); + + /** + * Get an instance of Short-circuit Replication Config with the requested factor. + * The same static instance will be returned for all requests for the same + * factor. + * @param factor Replication Factor requested + * @return ShortCircuitReplicationConfig object of the requested factor + */ + public static ShortCircuitReplicationConfig getInstance( + ReplicationFactor factor) { + if (factor == ONE) { + return SHORT_CIRCUIT_ONE_CONFIG; + } else if (factor == THREE) { + return SHORT_CIRCUIT_THREE_CONFIG; + } + return new ShortCircuitReplicationConfig(factor); + } + + /** + * Use the static getInstance method instead of the private constructor. + * @param replicationFactor + */ + private ShortCircuitReplicationConfig(ReplicationFactor replicationFactor) { + this.replicationFactor = replicationFactor; + } + + @Override + public ReplicationFactor getReplicationFactor() { + return replicationFactor; + } + + @Override + public int getRequiredNodes() { + return replicationFactor.getNumber(); + } + + @Override + @JsonIgnore + public String getReplication() { + return String.valueOf(this.replicationFactor); + } + + @Override + public ReplicationType getReplicationType() { + return ReplicationType.SHORT_CIRCUIT; + } + + /** + * This method is here only to allow the string value for replicationType to + * be output in JSON. + */ + @JsonProperty("replicationType") + public String replicationType() { + return REPLICATION_TYPE; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ShortCircuitReplicationConfig that = (ShortCircuitReplicationConfig) o; + return replicationFactor == that.replicationFactor; + } + + @Override + public int hashCode() { + return Objects.hash(replicationFactor); + } + + @Override + public String toString() { + return REPLICATION_TYPE + "/" + replicationFactor; + } + + @Override + public String configFormat() { + return toString(); + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index 9ac32c469ca4..f15df161a8b3 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -28,6 +29,7 @@ import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -50,6 +52,15 @@ public interface Validator extends // just a shortcut to avoid having to repeat long list of generic parameters } + /** + * Validator for container read chunk through short-circuit local reads + */ + public interface ShortCircuitValidator extends + CheckedBiConsumer, ContainerProtos.ChunkInfo, IOException> { + // just a shortcut to avoid having to repeat long list of generic parameters + } + + private final AtomicInteger referenceCount; private boolean isEvicted; @@ -94,6 +105,10 @@ public int getRefcount() { @Override public abstract void close(); + public boolean isClosed() { + return false; + } + /** * Returns the pipeline of machines that host the container used by this * client. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java index 6c5b4aff57f6..ae5fdcb62d14 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java @@ -524,9 +524,9 @@ public String toString() { b.append(" ReplicaIndex: ").append(this.getReplicaIndex(datanodeDetails)); } b.append(", ReplicationConfig: ").append(replicationConfig); - b.append(", State:").append(getPipelineState()); - b.append(", leaderId:").append(leaderId != null ? leaderId.toString() : ""); - b.append(", CreationTimestamp").append(getCreationTimestamp() + b.append(", State: ").append(getPipelineState()); + b.append(", leaderId: ").append(leaderId != null ? leaderId.toString() : ""); + b.append(", CreationTimestamp: ").append(getCreationTimestamp() .atZone(ZoneId.systemDefault())); b.append("]"); return b.toString(); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index d3f39c023b73..5dbfdc8fd7bb 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -65,11 +65,15 @@ public final class ContainerCommandResponseBuilders { public static Builder getContainerCommandResponse( ContainerCommandRequestProto request, Result result, String message) { - return ContainerCommandResponseProto.newBuilder() + ContainerCommandResponseProto.Builder builder = ContainerCommandResponseProto.newBuilder() .setCmdType(request.getCmdType()) .setTraceID(request.getTraceID()) .setResult(result) .setMessage(message); + if (request.hasClientId() && request.hasCallId()) { + builder.setClientId(request.getClientId()).setCallId(request.getCallId()); + } + return builder; } /** @@ -82,10 +86,14 @@ public static Builder getContainerCommandResponse( public static Builder getSuccessResponseBuilder( ContainerCommandRequestProto request) { - return ContainerCommandResponseProto.newBuilder() + ContainerCommandResponseProto.Builder builder = ContainerCommandResponseProto.newBuilder() .setCmdType(request.getCmdType()) .setTraceID(request.getTraceID()) .setResult(Result.SUCCESS); + if (request.hasClientId() && request.hasCallId()) { + builder.setClientId(request.getClientId()).setCallId(request.getCallId()); + } + return builder; } /** @@ -149,10 +157,10 @@ public static ContainerCommandResponseProto putBlockResponseSuccess( } public static ContainerCommandResponseProto getBlockDataResponse( - ContainerCommandRequestProto msg, BlockData data) { + ContainerCommandRequestProto msg, BlockData data, boolean shortCircuitGranted) { GetBlockResponseProto.Builder getBlock = GetBlockResponseProto.newBuilder() - .setBlockData(data); + .setBlockData(data).setShortCircuitAccessGranted(shortCircuitGranted); return getSuccessResponseBuilder(msg) .setGetBlock(getBlock) @@ -365,9 +373,7 @@ public static ContainerCommandResponseProto getEchoResponse( .newBuilder() .setPayload(UnsafeByteOperations.unsafeWrap(RandomUtils.nextBytes(responsePayload))); - return getSuccessResponseBuilder(msg) - .setEcho(echo) - .build(); + return getSuccessResponseBuilder(msg).setEcho(echo).build(); } private ContainerCommandResponseBuilders() { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index cb41479b5f3e..a8b0c49a11df 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -239,6 +239,19 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, return response.getGetBlock(); } + public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, + List validators, ContainerCommandRequestProto.Builder builder, + DatanodeDetails datanode) throws IOException { + String traceId = TracingUtil.exportCurrentSpan(); + if (traceId != null) { + builder.setTraceID(traceId); + } + final ContainerCommandRequestProto request = builder.setDatanodeUuid(datanode.getUuidString()).build(); + ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators); + return response.getGetBlock(); + } + + /** * Calls the container protocol to get the length of a committed block. * @@ -738,6 +751,18 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client, public static EchoResponseProto echo(XceiverClientSpi client, String encodedContainerID, long containerID, ByteString payloadReqBytes, int payloadRespSizeKB, int sleepTimeMs, boolean readOnly) throws IOException { + return echo(client, encodedContainerID, containerID, payloadReqBytes, payloadRespSizeKB, + sleepTimeMs, readOnly, null, 0, false); + } + + /** + * Send an echo to DataNode with clientId and callId in request + * + * @return EchoResponseProto + */ + public static EchoResponseProto echo(XceiverClientSpi client, String encodedContainerID, + long containerID, ByteString payloadReqBytes, int payloadRespSizeKB, int sleepTimeMs, boolean readOnly, + ByteString clientId, long callID, boolean noValidation) throws IOException { ContainerProtos.EchoRequestProto getEcho = EchoRequestProto .newBuilder() @@ -754,6 +779,9 @@ public static EchoResponseProto echo(XceiverClientSpi client, String encodedCont .setContainerID(containerID) .setDatanodeUuid(id) .setEcho(getEcho); + if (clientId != null) { + builder.setClientId(clientId).setCallId(callID); + } if (!encodedContainerID.isEmpty()) { builder.setEncodedToken(encodedContainerID); } @@ -763,7 +791,7 @@ public static EchoResponseProto echo(XceiverClientSpi client, String encodedCont } ContainerCommandRequestProto request = builder.build(); ContainerCommandResponseProto response = - client.sendCommand(request, getValidatorList()); + client.sendCommand(request, noValidation? new ArrayList<>() : getValidatorList()); return response.getEcho(); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java index f4efe1790bf7..d598aea4ca98 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java @@ -43,7 +43,8 @@ public enum HDDSLayoutFeature implements LayoutFeature { HADOOP_PRC_PORTS_IN_DATANODEDETAILS(7, "Adding Hadoop RPC ports " + "to DatanodeDetails."), HBASE_SUPPORT(8, "Datanode RocksDB Schema Version 3 has an extra table " + - "for the last chunk of blocks to support HBase.)"); + "for the last chunk of blocks to support HBase.)"), + SHORT_CIRCUIT_READS(9, "Short-circuit read support"); ////////////////////////////// ////////////////////////////// diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java index df0fdc59a4ae..9d5484173b8a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java @@ -474,8 +474,12 @@ public final class OzoneConfigKeys { public static final int OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE = 8 * 1024; public static final String OZONE_CLIENT_READ_TIMEOUT - = "ozone.client.read.timeout"; + = "ozone.client.read.timeout"; public static final String OZONE_CLIENT_READ_TIMEOUT_DEFAULT = "30s"; + public static final String OZONE_CLIENT_WRITE_TIMEOUT + = "ozone.client.write.timeout"; + public static final String OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT = "30s"; + public static final String OZONE_ACL_AUTHORIZER_CLASS = "ozone.acl.authorizer.class"; public static final String OZONE_ACL_AUTHORIZER_CLASS_DEFAULT = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java index f8b3febfeca8..190b7a6cfaa4 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java @@ -296,6 +296,20 @@ public static boolean verifyChecksum(List byteStrings, return checksumData.verifyChecksumDataMatches(computed, startIndex); } + public static boolean verifyChecksum(List bufferList, ChecksumData checksumData, + int startIndex) throws OzoneChecksumException { + ChecksumType checksumType = checksumData.getChecksumType(); + if (checksumType == ChecksumType.NONE) { + // Checksum is set to NONE. No further verification is required. + return true; + } + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + Checksum checksum = new Checksum(checksumType, bytesPerChecksum); + final ChecksumData computed = checksum.computeChecksum( + ChunkBuffer.wrap(bufferList)); + return checksumData.verifyChecksumDataMatches(computed, startIndex); + } + /** * Returns a ChecksumData with type NONE for testing. */ diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java index 855153f39ed9..95fd356486ea 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java @@ -21,9 +21,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.security.Security; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_JVM_NETWORK_ADDRESS_CACHE_ENABLED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_JVM_NETWORK_ADDRESS_CACHE_ENABLED_DEFAULT; @@ -89,16 +93,32 @@ private static String getHostNameWithoutDomain(final String fqdn) { } + private static final Map localAddrMap = Collections + .synchronizedMap(new HashMap()); /** * Match input address to local address. * Return true if it matches, false otherwsie. */ public static boolean isAddressLocal(InetSocketAddress addr) { InetAddress inetAddress = addr.getAddress(); - return inetAddress != null && NetUtils.isLocalAddress(inetAddress); + if (inetAddress == null) { + return false; + } + Boolean cached = localAddrMap.get(inetAddress.getHostAddress()); + if (cached != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Address {} is {} local", addr, (cached ? "" : "not")); + } + return cached; + } + boolean local = NetUtils.isLocalAddress(inetAddress); + if (LOG.isDebugEnabled()) { + LOG.debug("Address {} is {} local", addr, (local ? "" : "not")); + } + localAddrMap.put(inetAddress.getHostAddress(), local); + return local; } - public static boolean isUnresolved(boolean flexibleFqdnResolutionEnabled, InetSocketAddress addr) { return !flexibleFqdnResolutionEnabled && addr.isUnresolved() diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java index c1b65e10bc9e..538b565d284d 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hdds.utils; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.util.ProtobufUtils; +import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.junit.jupiter.api.Test; import java.util.UUID; @@ -26,6 +28,7 @@ import static org.apache.hadoop.ozone.util.ProtobufUtils.fromProtobuf; import static org.apache.hadoop.ozone.util.ProtobufUtils.toProtobuf; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test-cases for {@link ProtobufUtils}. @@ -46,4 +49,36 @@ public void testUuidConversion() { UUID deserialized = fromProtobuf(protobuf); assertEquals(original, deserialized); } + + @Test + public void testContainerCommandRequestProtoConversion() throws InvalidProtocolBufferException { + long containerID = 1L; + long localBlockID = 2L; + long bcsid = 3L; + String datanodeID = UUID.randomUUID().toString(); + ContainerProtos.DatanodeBlockID.Builder blkIDBuilder = + ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(containerID) + .setLocalID(localBlockID) + .setBlockCommitSequenceId(bcsid); + ContainerProtos.GetBlockRequestProto.Builder readBlockRequest = + ContainerProtos.GetBlockRequestProto.newBuilder().setBlockID(blkIDBuilder.build()); + + ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.GetBlock) + .setContainerID(containerID) + .setDatanodeUuid(datanodeID) + .setGetBlock(readBlockRequest.build()); + + ContainerProtos.ContainerCommandRequestProto request = builder.build(); + byte[] requestInBytes = request.toByteArray(); + + request = ContainerProtos.ContainerCommandRequestProto.parseFrom(requestInBytes); + assertTrue(request.hasGetBlock()); + assertEquals(ContainerProtos.Type.GetBlock, request.getCmdType()); + assertEquals(containerID, request.getContainerID()); + assertEquals(datanodeID, request.getDatanodeUuid()); + assertEquals(localBlockID, request.getGetBlock().getBlockID().getLocalID()); + assertEquals(bcsid, request.getGetBlock().getBlockID().getBlockCommitSequenceId()); + } } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 2b7592e1c356..952396084aae 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -632,7 +632,7 @@ public static ContainerCommandRequestProto getDummyCommandRequestProto( break; case GetBlock: builder.setGetBlock(ContainerProtos.GetBlockRequestProto.newBuilder() - .setBlockID(fakeBlockId).build()); + .setBlockID(fakeBlockId).setRequestShortCircuitAccess(true).build()); break; case GetCommittedBlockLength: builder.setGetCommittedBlockLength( diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java index 883f6cd851e3..ef425f56c728 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java @@ -31,7 +31,9 @@ import org.apache.hadoop.metrics2.lib.MutableQuantiles; import org.apache.hadoop.metrics2.lib.MutableRate; +import java.util.ArrayList; import java.util.EnumMap; +import java.util.List; /** * @@ -57,16 +59,20 @@ public class ContainerMetrics { @Metric private MutableCounterLong numReadStateMachine; @Metric private MutableCounterLong bytesReadStateMachine; - + /** for remote requests */ private final EnumMap numOpsArray; private final EnumMap opsBytesArray; private final EnumMap opsLatency; private final EnumMap opsLatQuantiles; + /** for local short-circuit requests */ + private final EnumMap numLocalOpsArray; + private final EnumMap opsLocalBytesArray; + private final EnumMap opsLocalLatencyNs; + private final EnumMap opsLocalInQueueLatencyNs; private MetricsRegistry registry = null; public ContainerMetrics(int[] intervals) { final int len = intervals.length; - MutableQuantiles[] latQuantiles = new MutableQuantiles[len]; this.numOpsArray = new EnumMap<>(ContainerProtos.Type.class); this.opsBytesArray = new EnumMap<>(ContainerProtos.Type.class); this.opsLatency = new EnumMap<>(ContainerProtos.Type.class); @@ -80,6 +86,7 @@ public ContainerMetrics(int[] intervals) { "bytes" + type, "bytes used by " + type + "op", (long) 0)); opsLatency.put(type, registry.newRate("latencyNs" + type, type + " op")); + MutableQuantiles[] latQuantiles = new MutableQuantiles[len]; for (int j = 0; j < len; j++) { int interval = intervals[j]; String quantileName = type + "Nanos" + interval + "s"; @@ -88,6 +95,23 @@ public ContainerMetrics(int[] intervals) { } opsLatQuantiles.put(type, latQuantiles); } + + this.numLocalOpsArray = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalBytesArray = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalLatencyNs = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalInQueueLatencyNs = new EnumMap<>(ContainerProtos.Type.class); + + List localTypeList = new ArrayList<>(); + localTypeList.add(ContainerProtos.Type.GetBlock); + localTypeList.add(ContainerProtos.Type.Echo); + for (ContainerProtos.Type type : localTypeList) { + numLocalOpsArray.put(type, registry.newCounter( + "numLocal" + type, "number of " + type + " ops", (long) 0)); + opsLocalBytesArray.put(type, registry.newCounter( + "localBytes" + type, "bytes used by " + type + "op", (long) 0)); + opsLocalLatencyNs.put(type, registry.newRate("localLatencyNs" + type, type + " op")); + opsLocalInQueueLatencyNs.put(type, registry.newRate("localInQueueLatencyNs" + type, type + " op")); + } } public static ContainerMetrics create(ConfigurationSource conf) { @@ -121,6 +145,27 @@ public void incContainerBytesStats(ContainerProtos.Type type, long bytes) { opsBytesArray.get(type).incr(bytes); } + public void incContainerLocalOpsMetrics(ContainerProtos.Type type) { + numOps.incr(); + numLocalOpsArray.get(type).incr(); + } + + public long getContainerLocalOpsMetrics(ContainerProtos.Type type) { + return numLocalOpsArray.get(type).value(); + } + + public void incContainerLocalOpsLatencies(ContainerProtos.Type type, long nanoSeconds) { + opsLocalLatencyNs.get(type).add(nanoSeconds); + } + + public void incContainerLocalOpsInQueueLatencies(ContainerProtos.Type type, long nanoSeconds) { + opsLocalInQueueLatencyNs.get(type).add(nanoSeconds); + } + + public void incContainerLocalBytesStats(ContainerProtos.Type type, long bytes) { + opsLocalBytesArray.get(type).incr(bytes); + } + public void incContainerDeleteFailedBlockCountNotZero() { containerDeleteFailedBlockCountNotZero.incr(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java index bfdff69be46f..c08495344c24 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.container.common.interfaces; +import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -33,10 +34,12 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerData; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.ratis.statemachine.StateMachine; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; @@ -55,6 +58,7 @@ public abstract class Handler { protected final ContainerMetrics metrics; protected String datanodeId; private IncrementalReportSender icrSender; + private OzoneContainer ozoneContainer; protected Handler(ConfigurationSource config, String datanodeId, ContainerSet contSet, VolumeSet volumeSet, @@ -73,14 +77,21 @@ public static Handler getHandlerForContainerType( final String datanodeId, final ContainerSet contSet, final VolumeSet volumeSet, final ContainerMetrics metrics, IncrementalReportSender icrSender) { + return getHandlerForContainerType(containerType, config, datanodeId, contSet, volumeSet, metrics, icrSender, null); + } + + public static Handler getHandlerForContainerType( + final ContainerType containerType, final ConfigurationSource config, + final String datanodeId, final ContainerSet contSet, + final VolumeSet volumeSet, final ContainerMetrics metrics, + IncrementalReportSender icrSender, OzoneContainer ozoneContainer) { switch (containerType) { - case KeyValueContainer: - return new KeyValueHandler(config, - datanodeId, contSet, volumeSet, metrics, - icrSender); - default: - throw new IllegalArgumentException("Handler for ContainerType: " + - containerType + "doesn't exist."); + case KeyValueContainer: + return new KeyValueHandler(config, datanodeId, contSet, volumeSet, metrics, + icrSender, ozoneContainer); + default: + throw new IllegalArgumentException("Handler for ContainerType: " + + containerType + "doesn't exist."); } } @@ -221,4 +232,6 @@ public void setClusterID(String clusterID) { this.clusterId = clusterID; } + public abstract FileInputStream getBlockInputStream(ContainerCommandRequestProto request) + throws IOException; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java new file mode 100644 index 000000000000..3df17fc384f2 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java @@ -0,0 +1,370 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.hadoop.ozone.container.common.transport.server; + +import io.opentracing.Scope; +import io.opentracing.Span; +import io.opentracing.util.GlobalTracer; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.storage.DomainPeer; +import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; +import org.apache.hadoop.ozone.container.common.interfaces.Handler; +import com.google.common.base.Preconditions; +import org.apache.hadoop.thirdparty.protobuf.CodedInputStream; +import org.apache.hadoop.util.LimitInputStream; +import org.slf4j.Logger; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.FileDescriptor; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InterruptedIOException; +import java.io.OutputStream; +import java.nio.channels.ClosedChannelException; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_MAGIC_CODE; +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_VERSION; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getContainerCommandResponse; + +/** + * Class for processing incoming/outgoing requests. + */ +class Receiver implements Runnable { + public static final Logger LOG = LoggerFactory.getLogger(Receiver.class); + + private DomainPeer peer; + private final String remoteAddress; // address of remote side + private final String remoteAddressWithoutPort; // only the address, no port + private final String localAddress; // local address of this daemon + private final XceiverServerDomainSocket domainSocketServer; + private final ContainerDispatcher dispatcher; + private final ContainerMetrics metrics; + private final InputStream socketIn; + private OutputStream socketOut; + private final int bufferSize; + private final ThreadPoolExecutor readExecutors; + private DataInputStream input; + private DataOutputStream output; + + public static Receiver create(DomainPeer peer, ConfigurationSource conf, XceiverServerDomainSocket server, + ContainerDispatcher dispatcher, ThreadPoolExecutor executor, ContainerMetrics metrics) throws IOException { + return new Receiver(peer, conf, server, dispatcher, executor, metrics); + } + + private Receiver(DomainPeer peer, ConfigurationSource conf, XceiverServerDomainSocket server, + ContainerDispatcher dispatcher, ThreadPoolExecutor executor, ContainerMetrics metrics) throws IOException { + this.peer = peer; + this.socketIn = peer.getInputStream(); + this.socketOut = peer.getOutputStream(); + this.domainSocketServer = server; + this.dispatcher = dispatcher; + this.readExecutors = executor; + this.metrics = metrics; + this.bufferSize = conf.getObject(OzoneClientConfig.class).getShortCircuitBufferSize(); + remoteAddress = peer.getRemoteAddressString(); + localAddress = peer.getLocalAddressString(); + final int colonIdx = remoteAddress.indexOf(':'); + remoteAddressWithoutPort = (colonIdx < 0) ? remoteAddress : remoteAddress.substring(0, colonIdx); + } + + @Override + public void run() { + long opsReceived = 0; + final AtomicLong opsHandled = new AtomicLong(0); + TaskEntry entry = null; + try { + domainSocketServer.addPeer(peer, Thread.currentThread(), this); + input = new DataInputStream(new BufferedInputStream(socketIn, bufferSize)); + output = new DataOutputStream(new BufferedOutputStream(socketOut, bufferSize)); + + // We process requests in a loop, and stay around for a short timeout. + // This optimistic behaviour allows the other end to reuse connections. + // Setting keepalive timeout to 0 disable this behavior. + do { + try { + entry = readRequest(input); + } catch (InterruptedIOException ignored) { + // Time out while we wait for client rpc + throw ignored; + } catch (EOFException | ClosedChannelException e) { + // Since we optimistically expect the next request, it's quite normal to + // get EOF here. + LOG.warn("{} is closed with {} after received {} ops and handled {} ops.", + peer, e.getClass().getName(), opsReceived, opsHandled.get()); + throw e; + } + + readExecutors.submit(new processRequestTask(entry, opsHandled)); + ++opsReceived; + // reset request variable + entry = null; + } while (peer != null && !peer.isClosed()); + } catch (Throwable t) { + String s = "Receiver error" + + ((entry == null) ? ", " : "processing " + entry.getRequest().getCmdType() + " operation, ") + + "src: " + remoteAddress + " dst: " + localAddress; + LOG.warn(s, t); + } finally { + if (peer != null) { + try { + domainSocketServer.closePeer(peer); + } catch (IOException e) { + LOG.warn("Failed to close peer {}", peer, e); + } + } + if (input != null) { + IOUtils.closeStream(input); + } + if (output != null) { + IOUtils.closeStream(output); + } + } + } + + /** Read the request **/ + private final TaskEntry readRequest(DataInputStream in) throws IOException { + // first short is DATA_TRANSFER_VERSION + final short version = in.readShort(); + if (version != DATA_TRANSFER_VERSION) { + throw new IOException("Version Mismatch (Expected: " + + DATA_TRANSFER_VERSION + ", Received: " + version + " )"); + } + long startTime = System.nanoTime(); + // second short is ContainerProtos#Type + final short typeNumber = in.readShort(); + ContainerProtos.Type type = ContainerProtos.Type.forNumber(typeNumber); + + ContainerCommandRequestProto requestProto = + ContainerCommandRequestProto.parseFrom(vintPrefixed(in)); + if (requestProto.getCmdType() != type) { + throw new IOException("Type mismatch, " + type + " in header while " + requestProto. getCmdType() + + " in request body"); + } + TaskEntry entry = new TaskEntry(requestProto, startTime); + return entry; + } + + public static InputStream vintPrefixed(final InputStream input) + throws IOException { + final int firstByte = input.read(); + if (firstByte == -1) { + throw new EOFException( + "Unexpected EOF while trying to read response from server"); + } + + int size = CodedInputStream.readRawVarint32(firstByte, input); + assert size >= 0; + return new LimitInputStream(input, size); + } + + /** Process the request **/ + public class processRequestTask implements Runnable { + private final TaskEntry entry; + private final ContainerCommandRequestProto request; + private final AtomicLong counter; + + public processRequestTask(TaskEntry entry, AtomicLong counter) { + this.entry = entry; + this.request = entry.getRequest(); + this.counter = counter; + this.entry.setInQueueStartTimeNs(); + } + + @Override + public void run() { + entry.setOutQueueStartTimeNs(); + ContainerProtos.Type type = request.getCmdType(); + if (isSupportedCmdType(type)) { + metrics.incContainerLocalOpsMetrics(type); + metrics.incContainerLocalOpsInQueueLatencies(type, entry.getInQueueTimeNs()); + } + Span span = TracingUtil.importAndCreateSpan("XceiverServerDomainSocket." + type.name(), + request.getTraceID()); + try (Scope scope = GlobalTracer.get().activateSpan(span)) { + ContainerCommandResponseProto responseProto; + if (isSupportedCmdType(type)) { + responseProto = dispatcher.dispatch(request, null); + } else { + responseProto = getContainerCommandResponse(request, ContainerProtos.Result.UNSUPPORTED_REQUEST, + "This command is not supported through DomainSocket channel.") + .build(); + } + if (responseProto.getResult() == ContainerProtos.Result.SUCCESS && type == ContainerProtos.Type.GetBlock) { + // get FileDescriptor + Handler handler = dispatcher.getHandler(ContainerProtos.ContainerType.KeyValueContainer); + FileInputStream fis = handler.getBlockInputStream(request); + Preconditions.checkNotNull(fis, + "Failed to get block InputStream for block " + request.getGetBlock().getBlockID()); + entry.setFis(fis); + } + entry.setResponse(responseProto); + sendResponse(entry); + } catch (Throwable e) { + LOG.error("Failed to processRequest {} {} {}", type, request.getClientId(), request.getCallId(), e); + } finally { + span.finish(); + LOG.info("before counter is incremented {}", counter); + counter.incrementAndGet(); + LOG.info("counter is incremented {}", counter); + } + } + } + + void sendResponse(TaskEntry entry) { + entry.setSendStartTimeNs(); + byte buf[] = new byte[1]; + buf[0] = DATA_TRANSFER_MAGIC_CODE; + ContainerCommandResponseProto responseProto = entry.getResponse(); + ContainerProtos.Type type = responseProto.getCmdType(); + synchronized (output) { + FileInputStream fis = entry.getFis(); + try { + output.writeShort(DATA_TRANSFER_VERSION); + output.writeShort(type.getNumber()); + responseProto.writeDelimitedTo(output); + output.flush(); + if (fis != null) { + // send FileDescriptor + FileDescriptor[] fds = new FileDescriptor[1]; + fds[0] = fis.getFD(); + DomainSocket sock = peer.getDomainSocket(); + // this API requires send at least one byte buf. + sock.sendFileDescriptors(fds, buf, 0, buf.length); + if (LOG.isDebugEnabled()) { + LOG.info("send fd {}", fis.getFD()); + } + } + } catch (Throwable e) { + LOG.error("Failed to send response {}", responseProto.getCmdType(), e); + } finally { + entry.setSendFinishTimeNs(); + if (fis != null) { + try { + fis.close(); + LOG.info("fis {} for {} is closed", fis.getFD(), + responseProto.getClientId().toStringUtf8() + "-" + responseProto.getCallId()); + } catch (IOException e) { + LOG.warn("Failed to close {}", fis, e); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("handle request {} callId {} clientId {}, receive {} ns, in queue {} ns, " + + " handle {} ns, send out {} ns, total {} ns", type, responseProto.getCallId(), + responseProto.getClientId().toStringUtf8(), entry.getReceiveTimeNs(), entry.getInQueueTimeNs(), + entry.getProcessTimeNs(), entry.getSendTimeNs(), entry.getTotalTimeNs()); + } + if (isSupportedCmdType(type)) { + metrics.incContainerLocalOpsLatencies(type, entry.getTotalTimeNs()); + } + } + } + } + + private boolean isSupportedCmdType(ContainerProtos.Type type) { + return type == ContainerProtos.Type.GetBlock || type == ContainerProtos.Type.Echo; + } + + class TaskEntry { + ContainerCommandRequestProto request; + ContainerCommandResponseProto response; + FileInputStream fis; + long receiveStartTimeNs; + long inQueueStartTimeNs; + long outQueueStartTimeNs; + long sendStartTimeNs; + long sendFinishTimeNs; + + TaskEntry(ContainerCommandRequestProto requestProto, long startTimeNs) { + this.request = requestProto; + this.receiveStartTimeNs = startTimeNs; + } + + public ContainerCommandResponseProto getResponse() { + return response; + } + + public FileInputStream getFis() { + return fis; + } + + public ContainerCommandRequestProto getRequest() { + return request; + } + + public void setInQueueStartTimeNs() { + inQueueStartTimeNs = System.nanoTime(); + } + + public void setOutQueueStartTimeNs() { + outQueueStartTimeNs = System.nanoTime(); + } + + public long getReceiveTimeNs() { + return inQueueStartTimeNs - receiveStartTimeNs; + } + + public long getInQueueTimeNs() { + return outQueueStartTimeNs - inQueueStartTimeNs; + } + + public long getProcessTimeNs() { + return sendStartTimeNs - outQueueStartTimeNs; + } + + public long getSendTimeNs() { + return sendFinishTimeNs - sendStartTimeNs; + } + + public void setResponse(ContainerCommandResponseProto responseProto) { + this.response = responseProto; + } + + public void setFis(FileInputStream is) { + this.fis = is; + } + + public void setSendStartTimeNs() { + this.sendStartTimeNs = System.nanoTime(); + } + + public void setSendFinishTimeNs() { + this.sendFinishTimeNs = System.nanoTime(); + } + + public long getTotalTimeNs() { + return this.sendFinishTimeNs - this.receiveStartTimeNs; + } + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java new file mode 100644 index 000000000000..536ed018046f --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.hadoop.ozone.container.common.transport.server; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.storage.DomainPeer; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.hdds.utils.HddsServerUtil; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; +import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; +import org.apache.hadoop.util.Daemon; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.nio.channels.AsynchronousCloseException; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Creates a DomainSocket server endpoint that acts as the communication layer for Ozone containers. + */ +public final class XceiverServerDomainSocket implements XceiverServerSpi, Runnable { + public static final Logger LOG = LoggerFactory.getLogger(XceiverServerDomainSocket.class); + private int port; + private Daemon server; + private ContainerDispatcher dispatcher; + private ContainerMetrics metrics; + private final AtomicBoolean isRunning = new AtomicBoolean(false); + + /** + * Maximal number of concurrent readers per node. + * Enforcing the limit is required in order to avoid data-node + * running out of memory. + */ + int maxXceiverCount; + private final AtomicInteger xceriverCount; + private DomainSocket domainSocket; + private final ConfigurationSource config; + private final String threadPrefix; + private final HashMap peers = new HashMap<>(); + private final HashMap peersReceiver = new HashMap<>(); + protected int readTimeoutMs; + protected int writeTimeoutMs; + private final ThreadPoolExecutor readExecutors; + + /** + * Constructs a DomainSocket server class, used to listen for requests from local clients. + */ + public XceiverServerDomainSocket(DatanodeDetails datanodeDetails, ConfigurationSource conf, + ContainerDispatcher dispatcher, ThreadPoolExecutor executor, + ContainerMetrics metrics, DomainSocketFactory domainSocketFactory) { + Preconditions.checkNotNull(conf); + this.port = conf.getInt(OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT, + OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT); + if (conf.getBoolean(OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT, + OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT_DEFAULT)) { + this.port = 0; + } + this.config = conf; + final int threadCountPerDisk = + conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); + final int numberOfDisks = HddsServerUtil.getDatanodeStorageDirs(conf).size(); + this.maxXceiverCount = threadCountPerDisk * numberOfDisks * 5; + this.xceriverCount = new AtomicInteger(0); + this.dispatcher = dispatcher; + this.readExecutors = executor; + this.metrics = metrics; + LOG.info("Max allowed {} xceiver", maxXceiverCount); + this.threadPrefix = datanodeDetails.threadNamePrefix() + XceiverServerDomainSocket.class.getSimpleName(); + + if (domainSocketFactory.isServiceEnabled() && domainSocketFactory.isServiceReady()) { + int port = conf.getInt(OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT, OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT); + this.readTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + this.writeTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + try { + domainSocket = DomainSocket.bindAndListen( + DomainSocket.getEffectivePath(conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH_DEFAULT), port)); + OzoneClientConfig ozoneClientConfig = conf.getObject(OzoneClientConfig.class); + domainSocket.setAttribute(DomainSocket.RECEIVE_TIMEOUT, readTimeoutMs); + domainSocket.setAttribute(DomainSocket.SEND_TIMEOUT, writeTimeoutMs); + LOG.info("UNIX domain socket {} is created: {}, timeout for read {} ms, timeout for write {} ms, " + + "send/receive buffer {} bytes", domainSocket, domainSocket.getPath(), readTimeoutMs, writeTimeoutMs, + ozoneClientConfig.getShortCircuitBufferSize()); + } catch (IOException e) { + LOG.warn("Although short-circuit local reads are configured, we cannot " + + "enable the short circuit read because DomainSocket operation failed", e); + domainSocket = null; + throw new IllegalArgumentException(e); + } + } + } + + @Override + public int getIPCPort() { + return this.port; + } + + /** + * Returns the Replication type supported by this end-point. + * + * @return enum -- {Stand_Alone, Ratis, Grpc, Chained} + */ + @Override + public HddsProtos.ReplicationType getServerType() { + return HddsProtos.ReplicationType.STAND_ALONE; + } + + @Override + public void start() throws IOException { + if (isRunning.compareAndSet(false, true)) { + if (domainSocket != null) { + this.server = new Daemon(this); + this.server.setName(threadPrefix); + this.server.start(); + LOG.info("Listening on UNIX domain socket: {}", domainSocket.getPath()); + isRunning.set(true); + } else { + LOG.warn("Cannot start XceiverServerDomainSocket because domainSocket is null"); + } + } else { + LOG.info("UNIX domain socket server listening on {} is already stopped", domainSocket.getPath()); + } + } + + @Override + public void stop() { + if (isRunning.compareAndSet(true, false)) { + if (server != null) { + try { + if (domainSocket != null) { + // TODO: once HADOOP-19261 is merged, change it to domainSocket.close(true); + domainSocket.close(true); + LOG.info("UNIX domain socket server listening on {} is stopped", domainSocket.getPath()); + } + } catch (IOException e) { + LOG.error("Failed to force close DomainSocket", e); + } + server.interrupt(); + try { + server.join(); + } catch (InterruptedException e) { + LOG.error("Failed to shutdown XceiverServerDomainSocket", e); + Thread.currentThread().interrupt(); + } + } + } else { + LOG.info("UNIX domain socket server listening on {} is already stopped", domainSocket.getPath()); + } + } + + @Override + public boolean isStarted() { + return isRunning.get(); + } + + @Override + public void submitRequest(ContainerCommandRequestProto request, + HddsProtos.PipelineID pipelineID) throws IOException { + throw new UnsupportedOperationException("Operation is not supported for " + this.getClass().getSimpleName()); + } + + @Override + public boolean isExist(HddsProtos.PipelineID pipelineId) { + throw new UnsupportedOperationException("Operation is not supported for " + this.getClass().getSimpleName()); + } + + @Override + public List getPipelineReport() { + throw new UnsupportedOperationException("Operation is not supported for " + this.getClass().getSimpleName()); + } + + @Override + public void run() { + DomainPeer peer = null; + while (isRunning.get()) { + try { + DomainSocket connSock = domainSocket.accept(); + peer = new DomainPeer(connSock); + peer.setReadTimeout(readTimeoutMs); + peer.setWriteTimeout(writeTimeoutMs); + LOG.info("Accepted a new connection. xceriverCount {}", xceriverCount.get()); + + // Make sure the xceiver count is not exceeded + if (xceriverCount.get() >= maxXceiverCount) { + throw new IOException("Xceiver count exceeds the limit " + maxXceiverCount); + } + Daemon daemon = new Daemon(Receiver.create(peer, config, this, dispatcher, readExecutors, metrics)); + daemon.setName(threadPrefix + "@" + peer.getDomainSocket().toString()); + daemon.start(); + } catch (SocketTimeoutException ignored) { + // wake up to see if should continue to run + } catch (AsynchronousCloseException ace) { + // another thread closed our listener socket - that's expected during shutdown, but not in other circumstances + LOG.info("XceiverServerDomainSocket is closed", ace); + } catch (IOException ie) { + // usually when the xceiver count limit is hit. + LOG.warn("Got an exception", ie); + IOUtils.closeQuietly(peer); + } catch (OutOfMemoryError ie) { + IOUtils.closeQuietly(peer); + // DataNode can run out of memory if there is too many transfers. + // Log the event, Sleep for 30 seconds, other transfers may complete by + // then. + LOG.error("DataNode is out of memory. Will retry in 30 seconds.", ie); + try { + Thread.sleep(TimeUnit.SECONDS.toMillis(30L)); + } catch (InterruptedException e) { + // ignore + } + } catch (Throwable te) { + LOG.error("XceiverServerDomainSocket: Exiting.", te); + } + } + + close(); + } + + void close() { + try { + // Close the server to accept more requests. + if (domainSocket != null) { + domainSocket.getChannel().close(); + LOG.info("DomainSocket {} is closed", domainSocket.toString()); + } + } catch (IOException ie) { + LOG.warn("Failed to close domainSocket {}", domainSocket.toString(), ie); + } + + closeAllPeers(); + } + + /** + * Notify all Receiver thread of the shutdown. + */ + void closeAllPeers() { + // interrupt each and every Receiver thread. + peers.values().forEach(t -> t.interrupt()); + + // wait 3s for peers to close + long mills = 3000; + try { + while (!peers.isEmpty() && mills > 0) { + Thread.sleep(1000); + mills -= 1000; + } + } catch (InterruptedException e) { + LOG.info("Interrupted waiting for peers to close"); + Thread.currentThread().interrupt(); + } + + peers.keySet().forEach(org.apache.hadoop.io.IOUtils::closeStream); + peers.clear(); + peersReceiver.clear(); + } + + void addPeer(DomainPeer peer, Thread t, Receiver receiver) throws IOException { + if (!isRunning.get()) { + throw new IOException("XceiverServerDomainSocket is closed."); + } + peers.put(peer, t); + peersReceiver.put(peer, receiver); + xceriverCount.incrementAndGet(); + LOG.info("Peer {} is added", peer.getDomainSocket()); + } + + void closePeer(DomainPeer peer) throws IOException { + if (!isRunning.get()) { + throw new IOException("XceiverServerDomainSocket is closed."); + } + peers.remove(peer); + peersReceiver.remove(peer); + org.apache.hadoop.io.IOUtils.closeStream(peer); + xceriverCount.decrementAndGet(); + LOG.info("Peer {} is closed", peer.getDomainSocket()); + } + + @VisibleForTesting + public void setContainerDispatcher(ContainerDispatcher dispatcher) { + this.dispatcher = dispatcher; + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java index 42daaa94be39..3452642f58d7 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java @@ -66,6 +66,7 @@ import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED_DEFAULT; @@ -84,7 +85,6 @@ public final class XceiverServerGrpc implements XceiverServerSpi { private final ContainerDispatcher storageContainer; private boolean isStarted; private DatanodeDetails datanodeDetails; - private ThreadPoolExecutor readExecutors; private EventLoopGroup eventLoopGroup; private Class channelType; @@ -94,7 +94,7 @@ public final class XceiverServerGrpc implements XceiverServerSpi { * @param conf - Configuration */ public XceiverServerGrpc(DatanodeDetails datanodeDetails, - ConfigurationSource conf, + ConfigurationSource conf, ThreadPoolExecutor executor, ContainerDispatcher dispatcher, CertificateClient caClient) { Preconditions.checkNotNull(conf); @@ -108,19 +108,23 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails, this.port = 0; } - final int threadCountPerDisk = - conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); - final int numberOfDisks = - HddsServerUtil.getDatanodeStorageDirs(conf).size(); - final int poolSize = threadCountPerDisk * numberOfDisks; + ThreadPoolExecutor readExecutors = executor; + if (readExecutors == null) { + // this branch is to avoid updating all existing related tests + final int threadCountPerDisk = + conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); + final int numberOfDisks = + HddsServerUtil.getDatanodeStorageDirs(conf).size(); + final int poolSize = threadCountPerDisk * numberOfDisks; - readExecutors = new ThreadPoolExecutor(poolSize, poolSize, - 60, TimeUnit.SECONDS, - new LinkedBlockingQueue<>(), - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat(datanodeDetails.threadNamePrefix() + - "ChunkReader-%d") - .build()); + readExecutors = new ThreadPoolExecutor(poolSize, poolSize, + 60, TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat(datanodeDetails.threadNamePrefix() + + "ChunkReader-%d") + .build()); + } ThreadFactory factory = new ThreadFactoryBuilder() .setDaemon(true) @@ -129,10 +133,10 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails, .build(); if (Epoll.isAvailable()) { - eventLoopGroup = new EpollEventLoopGroup(poolSize / 10, factory); + eventLoopGroup = new EpollEventLoopGroup(readExecutors.getPoolSize() / 10, factory); channelType = EpollServerSocketChannel.class; } else { - eventLoopGroup = new NioEventLoopGroup(poolSize / 10, factory); + eventLoopGroup = new NioEventLoopGroup(readExecutors.getPoolSize() / 10, factory); channelType = NioServerSocketChannel.class; } final boolean zeroCopyEnabled = conf.getBoolean( @@ -168,6 +172,12 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails, storageContainer = dispatcher; } + @VisibleForTesting + public XceiverServerGrpc(DatanodeDetails datanodeDetails, ConfigurationSource conf, + ContainerDispatcher dispatcher, CertificateClient caClient) { + this(datanodeDetails, conf, null, dispatcher, caClient); + } + @Override public int getIPCPort() { return this.port; @@ -217,8 +227,6 @@ public void start() throws IOException { public void stop() { if (isStarted) { try { - readExecutors.shutdown(); - readExecutors.awaitTermination(5L, TimeUnit.SECONDS); server.shutdown(); server.awaitTermination(5, TimeUnit.SECONDS); eventLoopGroup.shutdownGracefully().sync(); @@ -230,6 +238,11 @@ public void stop() { } } + @Override + public boolean isStarted() { + return isStarted; + } + @Override public void submitRequest(ContainerCommandRequestProto request, HddsProtos.PipelineID pipelineID) throws IOException { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java index 480561270ec4..7cafeaecc53f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java @@ -99,4 +99,6 @@ default List getStorageReport() throws IOException { return null; } + + boolean isStarted(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index a4c143439852..3762cb3ead26 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -607,6 +607,11 @@ public void stop() { } } + @Override + public boolean isStarted() { + return isStarted; + } + @Override public int getIPCPort() { return clientPort; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index aa9c4bd953c5..4147e9cddd0c 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.keyvalue; import java.io.File; +import java.io.FileInputStream; import java.io.FilenameFilter; import java.io.IOException; import java.io.InputStream; @@ -32,6 +33,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.function.Function; @@ -88,6 +90,7 @@ import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerFactory; import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures; import com.google.common.annotations.VisibleForTesting; @@ -147,6 +150,9 @@ public class KeyValueHandler extends Handler { // A striped lock that is held during container creation. private final Striped containerCreationLocks; private static FaultInjector injector; + // map temporarily carries FileInputStreams for short-circuit read requests + private final Map streamMap = new ConcurrentHashMap<>(); + private OzoneContainer ozoneContainer; public KeyValueHandler(ConfigurationSource config, String datanodeId, @@ -154,7 +160,18 @@ public KeyValueHandler(ConfigurationSource config, VolumeSet volSet, ContainerMetrics metrics, IncrementalReportSender icrSender) { + this(config, datanodeId, contSet, volSet, metrics, icrSender, null); + } + + public KeyValueHandler(ConfigurationSource config, + String datanodeId, + ContainerSet contSet, + VolumeSet volSet, + ContainerMetrics metrics, + IncrementalReportSender icrSender, + OzoneContainer ozoneContainer) { super(config, datanodeId, contSet, volSet, metrics, icrSender); + this.ozoneContainer = ozoneContainer; blockManager = new BlockManagerImpl(config); validateChunkChecksumData = conf.getObject( DatanodeConfiguration.class).isChunkDataValidationCheck(); @@ -648,14 +665,34 @@ ContainerCommandResponseProto handleGetBlock( } ContainerProtos.BlockData responseData; + boolean shortCircuitGranted = false; try { - BlockID blockID = BlockID.getFromProtobuf( - request.getGetBlock().getBlockID()); + ContainerProtos.GetBlockRequestProto getBlock = request.getGetBlock(); + BlockID blockID = BlockID.getFromProtobuf(getBlock.getBlockID()); BlockUtils.verifyReplicaIdx(kvContainer, blockID); responseData = blockManager.getBlock(kvContainer, blockID).getProtoBufMessage(); + if (getBlock.hasRequestShortCircuitAccess() && getBlock.getRequestShortCircuitAccess()) { + if (!VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.SHORT_CIRCUIT_READS)) { + throw new StorageContainerException("DataNode has not finalized " + + "upgrading to support short-circuit read.", UNSUPPORTED_REQUEST); + } + boolean domainSocketServerEnabled = ozoneContainer != null + && ozoneContainer.getReadDomainSocketChannel() != null + && ozoneContainer.getReadDomainSocketChannel().isStarted(); + if (domainSocketServerEnabled) { + FileInputStream fis = chunkManager.getShortCircuitFd(kvContainer, blockID); + Preconditions.checkState(fis != null); + String mapKey = getMapKey(request); + streamMap.put(mapKey, fis); + shortCircuitGranted = true; + } + } final long numBytes = responseData.getSerializedSize(); - metrics.incContainerBytesStats(Type.GetBlock, numBytes); - + if (shortCircuitGranted) { + metrics.incContainerLocalBytesStats(Type.GetBlock, numBytes); + } else { + metrics.incContainerBytesStats(Type.GetBlock, numBytes); + } } catch (StorageContainerException ex) { return ContainerUtils.logAndReturnError(LOG, ex, request); } catch (IOException ex) { @@ -664,7 +701,22 @@ ContainerCommandResponseProto handleGetBlock( request); } - return getBlockDataResponse(request, responseData); + return getBlockDataResponse(request, responseData, shortCircuitGranted); + } + + public FileInputStream getBlockInputStream(ContainerCommandRequestProto request) throws IOException { + if (request.getCmdType() != Type.GetBlock) { + throw new StorageContainerException("Request type mismatch, expected " + Type.GetBlock + + ", received " + request.getCmdType(), ContainerProtos.Result.MALFORMED_REQUEST); + } + String mapKey = getMapKey(request); + FileInputStream fis = streamMap.remove(mapKey); + LOG.info("streamMap remove stream {} for {}", fis.getFD(), mapKey); + return fis; + } + + private String getMapKey(ContainerCommandRequestProto request) { + return request.getClientId().toStringUtf8() + "-" + request.getCallId(); } /** diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java index 6a1d5533cf2c..0df62ad8e0f1 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java @@ -39,6 +39,8 @@ import org.slf4j.LoggerFactory; import jakarta.annotation.Nonnull; + +import java.io.FileInputStream; import java.io.IOException; import java.util.EnumMap; import java.util.Map; @@ -144,6 +146,12 @@ public void deleteChunks(Container container, BlockData blockData) selectHandler(container).deleteChunks(container, blockData); } + @Override + public FileInputStream getShortCircuitFd(Container container, BlockID blockID) + throws StorageContainerException { + return selectHandler(container).getShortCircuitFd(container, blockID); + } + @Override public void shutdown() { handlers.values().forEach(ChunkManager::shutdown); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java index 4ca578d7717a..d83657cda096 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java @@ -27,6 +27,8 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; @@ -46,6 +48,7 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.RandomAccessFile; @@ -55,6 +58,7 @@ import java.time.Duration; import java.util.concurrent.ExecutionException; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.GET_SHORT_CIRCUIT_FD_FAILED; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST; import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK; import static org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.WriteChunkStage.COMMIT_DATA; @@ -208,6 +212,22 @@ public ChunkBuffer readChunk(Container container, BlockID blockID, readMappedBufferThreshold, readMappedBufferMaxCount > 0, mappedBufferManager); } + @Override + public FileInputStream getShortCircuitFd(Container container, BlockID blockID) throws StorageContainerException { + checkLayoutVersion(container); + final File chunkFile = getChunkFile(container, blockID); + FileInputStream fis = null; + try { + fis = new FileInputStream(NativeIO.getShareDeleteFileDescriptor(chunkFile, 0)); + return fis; + } catch (Exception e) { + IOUtils.closeQuietly(fis); + LOG.warn("getShortCircuitFds failed", e); + throw new StorageContainerException("getShortCircuitFds " + + "for short-circuit local reads failed", GET_SHORT_CIRCUIT_FD_FAILED); + } + } + @Override public void deleteChunk(Container container, BlockID blockID, ChunkInfo info) throws StorageContainerException { @@ -374,5 +394,4 @@ public void close() { } } } - } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java index 7751dba429de..39862918ecb5 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java @@ -31,9 +31,12 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; import org.apache.ratis.statemachine.StateMachine; +import java.io.FileInputStream; import java.io.IOException; import java.nio.ByteBuffer; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST; + /** * Chunk Manager allows read, write, delete and listing of chunks in * a container. @@ -78,6 +81,20 @@ default void writeChunk(Container container, BlockID blockID, ChunkInfo info, ChunkBuffer readChunk(Container container, BlockID blockID, ChunkInfo info, DispatcherContext dispatcherContext) throws StorageContainerException; + /** + * Get the FileInputStream of a given chunk, to share with client for short circuit read + * + * @param container - Container for the chunk + * @param blockID - ID of the block. + * @return FileInputStream - input stream of block file + * @throws StorageContainerException + */ + default FileInputStream getShortCircuitFd(Container container, BlockID blockID) + throws StorageContainerException { + throw new StorageContainerException("Operation is not supported for " + this.getClass().getSimpleName(), + UNSUPPORTED_REQUEST); + } + /** * Deletes a given chunk. * diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java index e40fa635c121..73d4f27ba743 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java @@ -29,10 +29,12 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.IncrementalContainerReportProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.security.symmetric.SecretKeyVerifierClient; import org.apache.hadoop.hdds.security.token.TokenVerifier; import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient; +import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; @@ -46,6 +48,7 @@ import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerDomainSocket; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis; @@ -60,6 +63,7 @@ import org.apache.hadoop.ozone.container.replication.ContainerImporter; import org.apache.hadoop.ozone.container.replication.ReplicationServer; import org.apache.hadoop.ozone.container.replication.ReplicationServer.ReplicationConfig; +import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures; import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures.SchemaV3; import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException; import org.apache.hadoop.util.Timer; @@ -77,7 +81,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -113,6 +119,9 @@ public class OzoneContainer { private final ContainerSet containerSet; private final XceiverServerSpi writeChannel; private final XceiverServerSpi readChannel; + private XceiverServerSpi readDomainSocketChannel; + private final ThreadPoolExecutor readExecutors; + private DomainSocketFactory domainSocketFactory; private final ContainerController controller; private BackgroundContainerMetadataScanner metadataScanner; private List dataScanners; @@ -207,7 +216,7 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, Handler.getHandlerForContainerType( containerType, conf, context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, icrSender)); + containerSet, volumeSet, metrics, icrSender, this)); } SecurityConfig secConf = new SecurityConfig(conf); @@ -235,15 +244,39 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, volumeSet), datanodeDetails.threadNamePrefix()); + final int threadCountPerDisk = + conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); + final int numberOfDisks = + HddsServerUtil.getDatanodeStorageDirs(conf).size(); + final int poolSize = threadCountPerDisk * numberOfDisks; + + readExecutors = new ThreadPoolExecutor(poolSize, poolSize, + 60, TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat(datanodeDetails.threadNamePrefix() + + "ChunkReader-%d") + .build()); + readChannel = new XceiverServerGrpc( - datanodeDetails, config, hddsDispatcher, certClient); - Duration blockDeletingSvcInterval = dnConf.getBlockDeletionInterval(); + datanodeDetails, config, readExecutors, hddsDispatcher, certClient); + + if (VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.SHORT_CIRCUIT_READS)) { + domainSocketFactory = DomainSocketFactory.getInstance(config); + if (domainSocketFactory.isServiceEnabled() && domainSocketFactory.isServiceReady()) { + readDomainSocketChannel = new XceiverServerDomainSocket(datanodeDetails, config, + hddsDispatcher, readExecutors, metrics, domainSocketFactory); + } else { + readDomainSocketChannel = null; + } + } + Duration blockDeletingSvcInterval = conf.getObject( + DatanodeConfiguration.class).getBlockDeletionInterval(); long blockDeletingServiceTimeout = config .getTimeDuration(OZONE_BLOCK_DELETING_SERVICE_TIMEOUT, OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); - int blockDeletingServiceWorkerSize = config .getInt(OZONE_BLOCK_DELETING_SERVICE_WORKERS, OZONE_BLOCK_DELETING_SERVICE_WORKERS_DEFAULT); @@ -482,6 +515,9 @@ public void start(String clusterId) throws IOException { hddsDispatcher.setClusterId(clusterId); writeChannel.start(); readChannel.start(); + if (readDomainSocketChannel != null) { + readDomainSocketChannel.start(); + } blockDeletingService.start(); recoveringContainerScrubbingService.start(); @@ -498,7 +534,19 @@ public void stop() { stopContainerScrub(); replicationServer.stop(); writeChannel.stop(); + readExecutors.shutdown(); + try { + readExecutors.awaitTermination(5L, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } readChannel.stop(); + if (readDomainSocketChannel != null) { + readDomainSocketChannel.stop(); + } + if (domainSocketFactory != null) { + domainSocketFactory.close(); + } this.handlers.values().forEach(Handler::stop); hddsDispatcher.shutdown(); volumeChecker.shutdownAndWait(0, TimeUnit.SECONDS); @@ -547,6 +595,10 @@ public XceiverServerSpi getReadChannel() { return readChannel; } + public XceiverServerSpi getReadDomainSocketChannel() { + return readDomainSocketChannel; + } + public ContainerController getController() { return controller; } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 6cfae24d41e7..f6f00aff8e9a 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -158,6 +158,7 @@ enum Result { EXPORT_CONTAINER_METADATA_FAILED = 45; IMPORT_CONTAINER_METADATA_FAILED = 46; BLOCK_ALREADY_FINALIZED = 47; + GET_SHORT_CIRCUIT_FD_FAILED = 48; } /** @@ -217,6 +218,10 @@ message ContainerCommandRequestProto { optional FinalizeBlockRequestProto finalizeBlock = 25; optional EchoRequestProto echo = 26; + + // clientId and callId are used to distinguish different requests from different local clients for shortCircuitRead + optional bytes clientId = 27; + optional uint64 callId = 28; } message ContainerCommandResponseProto { @@ -250,6 +255,10 @@ message ContainerCommandResponseProto { optional FinalizeBlockResponseProto finalizeBlock = 22; optional EchoResponseProto echo = 23; + + // clientId and callId are used to distinguish different requests from different local clients for shortCircuitRead + optional bytes clientId = 24; + optional uint64 callId = 25; } message ContainerDataProto { @@ -360,10 +369,12 @@ message FinalizeBlockResponseProto { message GetBlockRequestProto { required DatanodeBlockID blockID = 1; + optional bool requestShortCircuitAccess = 2 [default = false]; } message GetBlockResponseProto { required BlockData blockData = 1; + optional bool shortCircuitAccessGranted = 2 [default = false]; } diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto index 1fc5884e24fa..63a0a7d7982f 100644 --- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto +++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto @@ -287,6 +287,7 @@ enum ReplicationType { STAND_ALONE = 2; CHAINED = 3; EC = 4; + SHORT_CIRCUIT = 5; NONE = -1; // Invalid Type } diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java index 6edcca65f926..251871451891 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java @@ -93,7 +93,7 @@ public void releaseClient(XceiverClientSpi xceiverClient, } @Override - public XceiverClientSpi acquireClientForReadData(Pipeline pipeline) + public XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) throws IOException { return new MockXceiverClientSpi(pipeline, storage .computeIfAbsent(pipeline.getFirstNode(), @@ -108,7 +108,7 @@ public void releaseClientForReadData(XceiverClientSpi xceiverClient, @Override public XceiverClientSpi acquireClient(Pipeline pipeline, - boolean topologyAware) throws IOException { + boolean topologyAware, boolean allowShortCircuit) throws IOException { MockXceiverClientSpi mockXceiverClientSpi = new MockXceiverClientSpi(pipeline, storage .computeIfAbsent(topologyAware ? pipeline.getClosestNode() : diff --git a/hadoop-ozone/integration-test/libhadoop.dylib b/hadoop-ozone/integration-test/libhadoop.dylib new file mode 100755 index 0000000000000000000000000000000000000000..573a2b9ebb8e0c59e6e7fedb3e0d5fb5c654673f GIT binary patch literal 156696 zcmeEv33yaRwtsaxODBP_B_V94!=edG0NGWfvq8d=U=jozouo-R3km5Yod5xpM$o}Q zeQ8C-WhBTrk{)%UCQ950qCOpGX#^Q}6F?`3K2cfHEcX9*s_yN+ebWhnH#6V+e;@e8 zTYanQ)Twh$ovrHL`}eco|I&*w4a2`c{Pn|MYXD<+vA%SR4Z`2YsLjdAPFa%8Z+ZXE zz`J_xNxuPt$$wOtlT&3{UFB(}hWFhtQCIq!xYBPwPjklnnN|5SC&yA%z0BgirG~fU z6H(PmE3Wig2~Q`%!`%75oSZyMRb_Emq5HS@eDgmS6oAfM+hVcqXjI zZ}$b!GuI%l^jktz+)({LC#Sk>RdHEE&?z4F0NLIPzLFq>bE>k8Fo%izOl;a>E9K+g>nD^{4xq*^6zz43cG{-V6@YnY+3+*nmKzO;B*kul$FE{~rW zKLP*c*WeDCvdQ2O*Fb(AIE>38hg8ul?3EwP_qp4hKpIjHX#y3bK>=PS*`0QHT)8kLfH4OSG zjF&l0dkym+jXI7$uJn)ej7$f@-7h}@>QTeSS6eEH3iOv%W2rK&h)?uTLG+t|_Jo5F zc|lq3Ccm+VOEgmlG(*Nxz9s^pDgm)jNLe$5Kf_U-@^10#_(-g#uS7aD@U_ zC~$=WS153W0#_(-g#uS7aD@U_C~$=WS153W0#_(-g#uS7aD@U_DDeL^1za07`#W;{ z>Q67$)}PGqufMQ1p#DOV#&%NcXa8Bt>__@A=aC@h)G>ooH^|^*npua6{0vU)2REKZ zz3wi96Pq)&iM11LdAR01JJHsv9bj+MM%t+@Hhde;a5(6b!)!6knt50oZeKH=Z7o83 zUBI=rXvQicUfBAW&d-_|)npyqs*SQop#CKF3oeJkdBVQmU%m<(Db)pz71)`PaT7H!TU*G9Jg!deaJ%51yv*A9B_ z#C3%jl`2OloL=FQ$86o5n(UKZ%$WSbAwnTr!QoRw^?^Rmm8vGi-t84 z?|J+^#{pmr>8*$++|-e1#n71I1i;J#Z0n7Xi8Q;Nj=SBLXsz1R5nm2EbRX#%5sA zhX}7o;B`pL8u?sb7xWm5afu!yakZk(3x|u*gTv?nJ;pJo8Yki>#|d>)T`qxBuPK~I zK@v``ao8fgRGc`B9^f>QIX`o`267xwjyxyfutebS4aO$Ae0e!Ia2P$nVG47;D&v6i z>C+MpPY4{|k#Kn9a&X`qja*MO5DwHo!Vdekm1LHsF=y`AQvFd+{jcQu1*kvj66?7pLFlis(`<{NgPP_} zli-Ehk4UzI!@P+(i^#Ug?H5k7-Hi5!%9QPeP37$usoGa{(Qc`#-8KpCFI}|1P1W9i zx&mLt_(oN`qAuE*RPCNswiC9K&!F=W&uF+^e6$K2X0p*ehPI z=;sIjf@~STudokvk9-fZhZoi+Vh#zg75jFPPCl9OCgvmj1e(t-$eM1V@VQ#xyL5#^ zt1o-0o8jhLXFP=M!tt38$^ST+@fIInP182X2jqB6A={_rz7E+`elJbn2D(k|3O6fw zK{TTJNYID(TR^Y`4u?@EudfzAK`(E6>Y)+YQZ>!gIJ@b_dExhe4qw+szy0UU{`F@I z1K=YDg7-n7zYcs4hJWa1b7}+a=d=O#4z0iaEa-g-Jdph??t?#+;H-yS48O_ai`NAU zA6qBa70Z5#pm8hwP0k;=euZ~E_sK~2Y>;3t@~%HG@lW??lH320Ws3Yb;;*M}2&cJT z2VZ=>;lWco{pxSQ^=veA#^TyKLQ~%i{9OZJcQkA=mxr0~@!{JZ-pL@d!e1wtvGc?n zSumUOdJvm@Y(!Fhv?lm?LJ-U5{Srh!WgmTzv`(Iu=Caf?ml23P2=574&lJlD-&Cv< zy59nvm42s?%WylbWt{o)^iiyTjOW#d+l=pK*l@sfo@c6sgT+`7uGBQh_nO@oZ4LRRNMW@kfo zQ|ix~lM%Q0@wlZu+Tc8fI4w>y_=6D|-In$N24{pOatoKaW{D3fne?ht$fIl%d3}@A z7dRcc-q0-bLzX?ATyI4EB=7px5r*au;Zwi|->-~WWc|Rn?qZE?T4qZa!?xDpcp)DX zv1Oaq;KVUF!9V=wa)S7V#;37mf13Pzwcg%EIy+9ttvr@)ocO)okCo)UWgV>4xrAR& z;;O5Fqa0U7q0DjWpqMGbJ{Z4g4fn^D)+3*XEZ+(n=QFP2FnZu0RY-Bw?bghAj6?E! zdUQ8&724&ZU01mL%i=2Fbx4k@Qg7h&I0U?i9)H92lH)24qX+b;?k286ojb1DDsXy1 z!l@qDOOC5Jj2_@rqll|ecE?q%0*6`&hs~FcgCmi{xF`+{(YL$fs)k6ErMSuwGjb;9 zjakB>urm&hnC@lOk;q~60EZ3kxT+x%Wht(5#1sh}(j^>b<5~wDsgO}G9N3R8wVrk) zav0t?c>I-c`^_SbX^6ZF^#6g!F^-r9G42Fu+}O*RJBQ&tu3A?^_cTP`!<@IsIG}u5 z#8SXP#8Q1E9D*(j2RYUewt>@4WiL1m_c7-T83&XPiCD^!sECyoh*+tC*_g9M#8Qq# zcdQm+$GL2ZwHnw%sP~Sg9EpgflAvevQav#Y>1E#jQLi|{k%+j6w=MP2_IIkbx*I7* zb2NFyM(!A;f$dPW&%cq|1d20cyTx&MTGh@gHd4T=SG9Y)3wX~l=hv4I^E@0UVxIeP zRmD7B@=ZQg4`UujjKb%ke*eapr$JXE?4>H^@xsAJ%;QM>&O<-9kD!vzi^V*W&2G@$ zPCk}b{LrAgPvE7Bcf9cO5$`nUYy?ZJ8IR8u;r6*a`olXWG_W4WI}N%G|2Oec|H=2O0I$4fez&Ei<+p^BXiT1hL4uKWgg#e23w|ZwE4G-0g-3+pl5uu24oXYJ=_` z+IK)ql1IL`ShM6)yn?#+`vo3`U@za>OAZJJ*~gEV!Q);@4nz*cBM0I3I>fzkp7_EM zQ|^gL^r~1v_JLI~iPzZV^T=cCFgD@~McM*vAHWwsHpbAb)dbHY{z!g%!|>-wW<54-rEH6ZU*9l%2mE^UQ?~Ky|1|0W zTUp;az|j0TY!7cIV!(#szw?YE$AFy2e2oj+D&O*Kj-S#q^66cHm{SYk845L-2bMWTOecv#@?zUc|V#B9?H(d_2phLp{k2 z!xggKrC-#7KAq$X{bfGHLY7fa`z(&cO}L-LY#PJ^w*sygT^0VG0l7IP%){s-!|O_y zN60anFFvVqVlph>c{!n*_q>d?zwy=%?0q%Ffv(sOb7DX2c+9md8#K4caUbpNoHxJC zb7qa`Q}v6B(~{^4nbt~kVPAU!=87Ub2`x;*Msosy=ec0 z#@2DQkoOa~UNY^`|KEi-r~UiP87$Kt<%6v%{QdoG?@9Q#Tn_%||8K?rugv)gVgN2X zC?EV(g@37^ZJUID?d9N){$1eTp!EI8?^o(+xlea`6*$*M?`O_>fq!i{$_Ed+8uw!D zavRYo#>TU-6!X=xj}Y@6P}FybdgN_I{U@mZ(1kso0d^zq6_HOdQ`mU26|MVSl-CpI z2X6D+=k0{8Xi(-FavdIB%N?a3z@hHI|39`nMT+T;QTpBsEVfrmJspt2iYe58K=su<4Ky!67)sIBwp zXGhEpp|&4J2_5lIT*950Nz{hpem2fYc+F*9 zsO@!WE{(WayWtV77n+aEBQEpk-;=pe-_CpzdlIr-s`Ef}zG)ZMQFt5nN}=B4Bx6!#Mt@l2jBj$uZ$C1Wqxr}wsRQrpBlkN)x?O)Z#wHchz8usJJL58y$ zKbEbK14j(-A==Qm48E08-*DPl1KhN06FR+C*Cc#s>v@-QpVSd^G{E+Ev^j5n6>*6M z@k!^iZl@80oug>D={^?)o~^(S96< zx!>SigBVl;IxkWC!V3f4G^8N#c^FUk8Mfw=}sQQczvX!Ebw0GvEPiTym-m3Pi zgKVHlIH#9J-a8zKvr~iYT4^ti*U36rkz*owhENr}jv(878J;Y=;IAwz)Q9#$IZU@M z6l(=%)WV53t;8en%6Qt9eO9Xzbkqdd=q#Z7cUz#1Vk+VX*^zebV0&~BtN420cAQNV z@Qz>(N3}z_Omv{lMd#?AKb=V&>YjfT_CO>2*_2(hHujSK#+fvUHiLl^zyF4~FUv=l z^v4>N=g!wjmCn6R+_MD= zwAT0-?)y>v{wyu~(Lc&?b}#I^0!J^ox2p0pobD%rZM1(oV&Akau0Bl6|09D_(=v6- z;Yk!@ghO^jz5wfEa%&$$^QG{*Ri%shn74*pl-JP5$kogS?^3|SWbiTxJWT|jaaM7k zbXGNq&I4;u?}v6c=eZVVg0ate2D5PDg` zXGN~e?SFyK@t~nMUxcpB#r!obQ#`c&#Nd?aN^(Q8MKrxe?k{A^i*F3}x|3{yZ+O>L zkJp?}KD^gcJ|u&_NuYBg_yGDR3Hm?9SR>I@(=67Do<$!bKctI&>{#v^-i8V{CW_s7-HjUP(mcA_zd;Z0+;KS}Z!{XONzI@3KSO3saW z=q>xr5$ID5|Bd6iMBw_va3K%>#FfcDu@5@<#FfM70j@uSMl!unUUXK%VY9&DZxRmg zUJec%Mh|c}h5U|;1Ikmx`C>=RL4m__5)Mya4h|ef4{&fY=Yuj1C`X9%#g3R>y}8`n zE8(!IGY+-N_}B+88WK5-?s3?R^Q+Mpn6nZ%P&oqSQ{Qu5G{oE>a43;*$h#aIIE)_P zpwT#Ql5s%!qhtK+ZbM=b;UN0s{a0HFM;4%QwzRt%N!E+;TN&r?h3_chdylM7!%CUJXihpaS6&}8$QBTuKI8gYWqyYG)yeur(a(2&M))8- zIgd)T(S2`l!k=njw}#osepoRM&6C&tDEwsu{8|3{l)zboeF)wbevb0o6K~@bZK$4p zYc7c2ayvcrwlBI;O!gLNLw<-SM)GsVNJX#{qz8!>I(M8Va{T1a^Re-~i*jF&`smy~ zdN_Ra2)B>^RUqO9`2I2Awd8x(D);D!Pb$6IkXYB-hWER9+ACt4*3NpkA@RIN7Kt7l zAC<4pxc}ynJ<_vk`p9QRIiFUC@n>@WE5>tn>>iY%j~imnhe7ru>|5}g_BA)++UlK; z@sh_DpEwhY9<0A$jgxF6pOet_Ua;vbSSR{D+ZncJpPb?q4;>7U6XIzk#yJFD@cB*d z!{>*0bMpDk!_~c>eBj`7ejG**=GR-(DR0E_!F`+u68K~be8x-oT!X91{&~^FCq5iT z5Af-yanc?r#{uQ3;`y3}m|X&gFbRji%fW%e=m8E98s}>=4k$;Sk?8Qbz~Otn@B_Za z^^)npVe|lpK^o^{vffAe6zm_DLGg@5t#pp4R(+0$=*EU=l+R<-CgOPv_`S7FUe99) z{mI+MsM>oykD+Kc%Bx*NqOLE&ZD?B7*S(hHvqT(joLBqWMDhGa?IEu-N+K@e?Z>Iw z`*?nXx0~qIt|4((7voRyYFC@+b(Tpn?hK7{7HHj((|N!8G~&inh#Sv9Uz5GiLyu{~ zteM&1eI2)vl1>gza3Z#Ec($h#*+ipvoY_G6zYuTT(-4E)nEQOMtP2$SnRHQICmVY4oU6#Ac%J|3$Ps(Q zzG))P_!C{^c$()LYm$&_)VggML(D|a&l1mc{SuHvn7)O^IV?S2OS-#=Y}f5_evN#D zfo|Is8Neog1^Tq9>7vLFwnAqTJvp2i9$$le234Nj%h%9@cglWu4)*kwZJ0-Adp{dV zv?0AM$Dp*QF4L>KT<#gV$(6P&y*j_f??3b6*=xgMKT zKhC?}O;6N60oZp=@~(I1`egW1yz40s#px)+pW$6kyyW$G)`H^~<6ZC0FS~P5B1eU1 zAb3ug_jT8K&NapEo4RGQA9Gs6m~&eQYkp)nYu-#U(>uYrUdt+u1;Kx*zAB4khjj>0yqmTL*aSysmH&dQaeU;VKp`G)32>YM$o7VkXxL&dk%whDv7n%7V z@_{jq+Xr4P=J$X!zu)3|i9T>`B8PF&`8lE!eEPtSNZglv;M$mN0-rJop8{N!z8~VN z6Bo%l!2c0)An#vrc~!+pkS|AcirWWvM4~MDz_l^o2^3*uie|wC8YfPJb%!=)wnfFr%kEhhI+g z7PKONl5_~k2s5eL!588FC$1*_{lD^E z;a5t&t3#J^>AvfGVXR+tFsryzZMWdNlK&vnL_L2d>3Lr7wq+6Ar|yG(k&1H;>c0O+ zd|Eu;>+xw@;rmfcBKfqA;kS8wGm>Sc&k3K^ac7=)z0#+B9QDfzz3bgRsqkqX!;8J^ zRX($0_zLfOw@<9{ORdQ3xN&pW34fN(F{CPd)<@v8ZWBJMLzh7CafTUjBF#@14mlXc zDmLRh1^Ieuh!Nx3rrLgs&lZd+E}A|!ICP6a6D@LPvY&^wL3{d?uj4%a5BZpUEWCF| z8DFYptmVR``JJusJGm?i-`p{LsfYeb-~5=Lw{NbN0}cazIq~0@U(U3;dH<~*Ytg@c ztml0A(eSIAN#7`aSt+;d(4`R$o&9MwUH{Ggv_rSRgNNFmR^#%o@~7j5di&F73tvN= z+38uy^E>(=!ZQR&JNiCwX$= z{Ju@(8fa~{MzJaUo(wnq>Pvzv%dZ-)K2imD`lZ25l5~d}ZtegT-0Vw(t2GFnq=sA9 zR|U7~65+B9T-T}Lw)R%Rt-Un3NvzH%Tz#ktZsVoF)mnYiAy=n@d+-wBO8Z4>I>26d z%UQ>z!QI$}JhbAx3Li_B-;vi`lKg7>`ldtOASv!wtd+-LdmqK$HvHA~k;awRs`VP~ zd(xUR;OqhIQ*pc&dG^Q*77c&4&0Tkw!O8tmyn7GNxYp+BM!R8b#u%(x!{5`#;h=4T zCBeJ>NLBj{fUn+@e9zOys>~Cr$L=*o@|}eqAwSCPcgVCII!4e|^(?O2|4`7XC8RU` zSfGk#4P)K&)vuL&bOj9HAb1NJ#;g@~jOa`BbGKK|qkNnjCR^VbCTz?UiV;o+NtZs|e=pAWaDFWH@HGc~-3oqfP3&#kfLyMxHrtWgqdEAhn}xYY16RT; zFbFveTraHkK3{aU@SRq+UkH$0Cf;vmW;i+Bn6H4uJQ z;KA`%%)8|$mz&OnKlLR#`)a!dw%-?SWcxW>$O@Nf17C-ZJh!a)uum)GIjHQD?DHws zRNWDU&n1tM8cXu%$m_8UeA1^bZE2O}FT>5hRJeEggv-8E!QFJJa5wpcYxqnB_xG0y zce78p){j+i-@a71_xgm}@{tPe(MyF}>k}?Jpn~fkcZu?_)hAqoLj`x_rNZ6j6R!2I zD!8ea3fH%f(eh^%+@ec_+l4Q{-c!Nde2H+o@U;x@sNn9pM7Vn2Hrx7^3hsNC2)7Gg zpkycK#zI88Y5A65hyjv@T}#a$D-(0=rCp zfo_V&e$g{Ay81-tVF|9BbC%AAc|9WxyPU0jPnP^{tb2(+8?~5wDChzHtnH2WylL^i zLVB+o#=}`A-P81ZiRb)ME@Cjs{Sj}XFke0P=&Ia&3AZDDek2ln!I`yix`yMrnf8F? zd3MVmv1dlU8SjJnQLd#8G^I07Jf?%qzp!=z@STtI$>8Be!9#UyF2|ZN$i>F|r9Y1~ znUd}^B>%K@faDvBv5DSY?IkJqjxKw=qn9_HI&XUt`|$Ew;&Vj~i|FItGgSCMb^Lsr zckYz-mPlqy;GN#(nj8VX`nmZ?KANXJJy%O~ap$W*Pgy<<$QLQkNAWeWF8m@i^}sf; z=tDSP-o?3oai+gJd2Es7@n)7f6}~#;h2|A$pN_fEn!*0@MCcAY0~v0A9BYzfODD-D zoq>C2psZ8E?Z5mYWVd_S^Y&fkv+>=?1Kw|RZP)S_8*vd@U3_1aa#0F-caKN3>BO7z z$WG&V_L1Ft)?V-4?8)4Gj`jP2mpqnQ&gq$s?)Y!OnkJdwdn4<=9K0l1U#Q^eJ>V(v zQ>)S?O8L%(3@U$Q<$y_r>L{7I%G?5X&D=Ir_d)YptM(ZcYH=Q%^J|B!E%*DKZQ5yKbJ zdX{xyZIr|(p<{WSd*4#rqjR`~`_@3#MCT=WoQfE>9q(4$41A9PS9<>?U}s~E3ENmT z2)0qfZQ~~3#^K8G&-B4IvJr%rURrmJTGj-7!`=PIdiqy+_vbP$adrgbgP=`hBm zI{7{S-t!SLBhBTY9Pbt zbxc#*x!V=+WLqG$MfA_CkAN|2|_4;@j__)_c9@YosPPsnrL%ni+ zSn*q4AJ3rdV}0=FaAn^6SRanW|IPY<4Dj^%dL1J{&QM^|1te zBzaY>54<;Nu)F`W-u)ZM=E!sEv_3HAzhZsFVN7X#cs=j5!M#3&pM84qT&#~Yx4-RP zAAP*n2fwfITP9<@(7C3=5s-1!`==H5j?a~RA0IZzk>~)8C7+zy@G;!F?UL#pT$!74DNaP@R1>BS9KbzCqA;T{dw|R|Pr&*fkAjLd0|HJci?PG}Lz29pOKfu-kJ9Q)Qf}cJ4$8ZlHMgu>x zS$^Ek8{E$!01l74yIIqYZelh^VoUU8$zxB(^Ql9UV>*rJ!)Na4T653M=U69tG*-|3 zb1C-qIacHFf6Kgl>X0QPI*rv69nuq?`5dp~KRn(OBQI-yv_8~684_Eyo9|9=z6W1h zrKcS+x5o2W(t3C2{U3w84~adc4d^f1fAZDc?cd@1Y>vcj@gkqRJD%fBwsAAYW%qQ3 zYnAt^gzxT%sU7d`L%3S+@7xEvz_agizXum;+nr4!c!c|2**9P6@e#IT&b%J*eAWXt z%^@0dP>!M2TkX<0{b103Ioi+Np~JVo@SeffC)nl{7w=w=?@Ym_WXB~i+pG=2KN?-G zY&tVTW!R_eR88;}O@Kk{HNnr9>FkcR6+TQZbRzMXat&lpbno*#Uv11?;yU~uagDnV z`N}L$-cOE4WnH;ufVU4sblCg=WF)|cjIb#}Mjq(Q7X#ks!SNs&L4O%{E+cY#$U~jsTCu;#;bPB{;G%y6eRqxTVM+V1?*5N@^GCVHAH=+j=#N;Y zk^2YWKlNJ=+&FJBHotZQLtZl)#D!X00&nm4keI717rC(8Qa-Ce`Mji0Jy|rR)7ti- z|NG0ZX+5jsyPqL*Bp-ZEA{Sa41Kg0at{wiUI8R5h$7YfdEHB=n4L^T&HroP8?ZyWq;Ey5Hj7al{G)_ka4ctwW{tM!w#zjkiz??;WGy zI~~He6n-7#HT*5~i{kqWQTAUSbv@Ftk=0X9k?RKOn^p60pVmqd@kx=7=f8dUn~z|A zRM!>?!?7d! zDgNk8CjCB$GQHdA^iIJ8p^xy6cleGZ2PaT(!1n>^nWUcwGH3MpJ{4(*JENGje=K-8 z2y2GQ6Hulrc-RnQrFX=K+dn-bc*ybNJQQn<`iw;T0n@FSH=z9!IaBG|O->p!!LT87 zC&nast_$jzIt}F^kd^gwSVjJj`RXi z{uwawzN*n}XhS$01|N439azWIWBsspLAU6Btb)O=(mQP|er(?Q5JN@Li;oY+dxk5< z{^H7xza0}p&Bk=A5G44x186Y38F=3u}ajJ>E=-weh_wEpT zZ)E>Sc9O8p+t*t&-Sa;o-hq8Ja3Hztz<0RlTR`skxjM;tX6FFgQT)j|hUP9mXW*JU zb$SuW*G!L(MeCAa@%t&B`~0`0@93MSgww8#zv?7Y#(vO6`$j;Py!T1yj1S4tQ=kjs zcam_zUtjQn^uc2Au04?5wE53P)1%+O`{`(|sCUmb_j=nqpeyI6?t0r>#3R&S0K8TU zzG){tr{5j``W(>Iz}7U(cx9$-8m?zvok?))I*nmQY9PirkWfQokdN28llG+XJTG+0 zJc7l458}V);P)=PUjh00>{ECj8_!o$eUirZ3F`Fz9n-%3qoF1>hV?&IXlM=z&`nEa z`l-j@hwoz3r+%AfXoi0|IY$#&vH{O9wp#EmsEuO0kI^4`=|g%KJIP!->U8H^O~5x? zkGgO!U*Y!G54#>Ysg$qW8*Cdfcj`M9Z6nY&g13#Zzks%Q2DP5(b_{qP8zISm%S_u< zXiNAUr~Bjm>f6VNcE{y5co#OUIl0a8Kl#;v`wZIPo$~|_ba%wWYHie>@+HT$k)yu@ ztm7|f-LSqx-SL*i%?^aKztxocR+Bag4T(kbg zR_H7457SuztQ}2nYo-D4TccTX9pFdTX)2CDf427$Yn$7;x3TvDf37VEdjJ}?^*5mT zW1v?gczqiE$VZw2x=OGeiM?nKN8k~H_C)U`eOc3keyn*5`f^XnVv$wj6mi*Y?ar|2s1` z;XcDW>#(*5J@bZypo$`EIC<-j_%Cw8B3jJoZktW+o9V=$<%5|99m6uMe|mFA#OY)wmuyY#!!MyctP&NOar*ItJjk zOh=-LFP+}T^Z8MTExh-;&KAyf?|Bhy4bB*)68xWCQ-A2=q0<&$_}>t1b3oRPqaE%4 ztKaEIYfrgG+xswl=iB-K=_BCl)cuvdeJ0Kxx#Mny9cxuSXQj4ds{B8Fmx1=Zz5p&5 zqksDVtgnIM-pBI2fvmZR*1OvViZw&;sBFhRhT6B%U_FbyH;)~Z&k7t#Vm76&;j#f) z;qVtEIgg^AuOsM7(nVg+3n=EuoUSH%$Lv<%DBFMuptV~kOs4mFiTpQYo$r0I4p-Ab z^tq4dzOnPSzhm!3Tvfv=c7f+{7&|WJy}=>YXESGMLY5<*&K`|>A%?n2`rQio;qvdV zw~~spSaJ@Chrp{ zvq0^KJ6ISO`okE`n-X2 zOI_I-I#Z?QZyV+&_&X0c&&56{_P4ZR|92|UwhNhVgKZlD8ZLsb>1({}K&O+iZMr^~ zBV^q*1bUUr17axX)u-5^T!xE03(=xC`jW2Q0owe4{<2=d`J(JD)=vMf)`+inN)}Pf z-lZvE7~dq(-5xxo-OYb7ADRE7F&>^f znf#Dqyot*3z<-_t`CpDF%S9N?4>q|*FO`b2|Ofne3aS6A7^Oai< zG$bDF#oyaY_2H_mt!2_qN3X zmTSGHUWYZ!=Yh4ZoCooHIn4vS9_#~NqY~axoIkn z%5>9!z6r&c75UE2wyibP(0ms51>ae1J~qVKyd3g37xEr=`{>6)(B(HeclF33xJlAle7J)HJ=%*p46@2L^Z zTQiuGVzgG=--CS90@xFh70>$^#Tl_^#wvQFj&hRG$*f`{&fL*DjX~a$-qYnmyvyUd z+#ULqGlarUmAn*9`C!v}Z9l5QUZUp&Y<^V|^TJaav3K9j_nW0BKg zU(deKn&f}psU19e4fH<8y(y9B8|d60(Sh)%`EpwGwLZ|k(&>7{uy*=x-rgMth`B}u zVZTosb$kbMv^a0ozfJPH{4{l$si2Q;o#BCKP2}+e$iyqaS=L1q8x92jI7}72>rT3w zG;{_X&-f!w6Fj}QzwHj_J<4D2q}(UujARhcD>#Sa8`ZL2>Ia{FCE7iKv1FfRI^>{L zhQmC0z1~=NC1^{$+=up*H{FUW*;ilBWUt5h!Yvq+bcFogGMu#?E#h$6i-@#qG4CME zoAS160c#+1s;fWVH;VZNVZQr-`_@GL7Q&x$wj`IqxKC$M!<2ZAl=sF5+K=E&h|+E- z-!I;+ZW!!CHzaBI)3-5X-9UR;XE7fscJ$~5>{Ga0D&3GL=?3Tm(haT94cjrt)+~I# z2m2SKBV0apgV-w|f132e_lSx1B(jS4`*fij3=wi{)DV*``1RZuu19njQ`QZDCFJ5` zQU5sVDMva4@{hg02U@cX&UE1RGQK&m9=Y}pf*It^nt67F20os(e_z1jvezujzt~IC zf*-U7yk%Xii!*_&pUZDZ$#U$2rG7AN3+ruYHu!Z+8;Wsgy^x*QfVo`=^~iY}#XMPt z<^gC!_2k!6TR)5seEJidp}nOz*}WD8E_^MXck$e^kZB{~}F)oes9^|> z=dz0P!F$2KdlYmfy+tv}slbj`79kJaftVUPyZP9h@)z&qP-$+le^ zdE5xy%j*+h*FY~JE3j+G$I$tAw>&HKEY2Qy&I~H-niVp$0Q04s=~stX#SYA0UQcUq ze*@`D=(!`GD|G7WNXajZwC_IWS%=YR%kff-OR~oADaMRIyQzSE-t7Bb7{P%)^w>U} zt5bZ#LVg!%s+WH=1pbYtewUYj!-Ri>xRLa6JA7P*IC2HXZ$-Y$0G_m>ZX0x)IEzl- zw~4ZEL_PVf+XF=|)gQmrw(KBuLALxZBH{(kF$-SE&msstsHO>W5sKafA`Pj!*NL63U!)7LBc;gIwNOImb5*a{~W_?-?aOV-sQT z1^W2PDbbpte6-T;0#AMh^HSxMzC%Ut%sm5JE6=MZa_SF1{HP?Sb&$umF&~mqE8ZVP z&o+>pa-P7S6>>(hN;*U2S=DkzejUjIzpq$-BpVw6o8MRQSK@1f(wB{a{ks@n_fY;E zzS3!|fm85jNk)Z_34IM8=}nABKGL^%X$XdZ$qw^^5qmGXvm*D>W&fc9z+o%+y_|H$DeEu{yskAYWgGk z^hf`${6Nw>^j#!+4n)qkLjIieeQGW@SkDiGUjtjF!7rt~1;Lj}z;x>ntc{<8z2%nn zF!-9R74lgQ+$b)j^^Sd(rbyH`Lax6?Kk{u!9~dG3R8Q|gBO2eXq>VxkWP&yXYln~g z67^HcxzfK>o--pKk=WArsA%D<}ArYvMFoGK$-rYa|+x{8OHZc&I#6))f!w`8e|7 zi64AUI2*Pa`1x%Y=gP=V1?vrQaj*?|2Bo9~vP|DTi1~iqOJx57z)QUpC!v3{t|mH* z@Z3x>adU!`;0T!)&!Ex#+XJx=30ZjmZ?Ri+0j%G9f$%{P8&8VS#C1h~f=Bt7_xuda zh;M11xHe|w)tGy@z5P=;ZvcOh=U_a!C7Fi?);Sj=>mYgF#hCZ---&ta{ILfRfU_x( z&tN^XeTzN(F&doN(S(;6K!dm#W}6$p`1#Hv2{+QK9IsOG&YQQ~c;S4FkK9lw;D&&% zfV01(cgU7#yc6(`kmDkZ6RUwQ9Dr*8!~Hvq;_6_3@cV9-G2EYJ#A-G_rp3K>KivoS z4EE!6>5Jb%`18kKKl}yYFA#rZlQZwHOuzXLea9cVYIu6)ANrazan0NGVmjOOa(d#tSrm2uL|xyzUrK-A{;E!O6YqX8ebc6w(qFv4x>Mb|=ohE)w>^#exf|Gz)(Dn8 zrcJZIGQcoD^QxHitjBWGEu&+?N^jJJJsZNNmE5Qe`+#2q!jAK6P*`Xv-46~M!>^%X z^Z2!Q*gVFj&e||!^_Vuczx2jFVc27yI!iNjwFR&(fL#UHTLJrCz@|Fzt7Z$}^M0CP ztH(lC9!1+RF&f)_HwJ`Nqx`*&O}p>Lps?P-bPWz0$FHGb3;4Bn*b08_6Sjbl#cC!; z$D}`sab}E(2^%*{6Gr{V&eDeULc6iE0>Z}fYf#vHehm&Q<=4=#d-=6@*t7g9;l6>@ zGy(6I0q+p-{tNIPi~0`;XMPO`E8*9mur2%=9QH?k4GsH%Uwelg=hr@Ap+5#q%>bWi z&P3BKr-DG6L90tJ&k`zcpmI>yJbr&L%Aha5Pvy|CP=0?1%7A$?XxcNdIkPqlTTQfC zaE4%Ag)-u}=2N>EDuX^$o=D}O zuwGP7pmK27chvtTDu;&sg~~=MgI-jwpmLwEdjXfec|>z08ii8-9aM(wP`ekX92E8e zmG@E^^rG^9Du;$`p>i9Qdxw=!`9~`E2^;fIlr@@~X@Je{u1jy*v@<>C?j7mYO?ByY z_}jZ_2i6JxcE6Aw!LGjQo-o~(xEaVvV(pEFzN9k<_#Rh3(nZjtKf-^@hK{OPkNwpI z#J#hCuM@Dj4}uu1?v6VS;Q3H~hKk|2K&vnd3`?6yIZ!G$5*Mu&w1Dvrta(4ga z)7aglp9xNmX5jIsa9`4UcpfpLWD9%}x*BmlX&PYAJ}v1FiuFh*(muQJ$Alk7ei!MT zf5`6+$9}b_7kky@uN{R=;Wm=`E8-RQC7P zHx%v2pXNGyT`WIadqnLAd zK1H|ZyrH}CEOEnAPYQcPaq8z7e>3Rf54%eDg@XFoyX^oToDy5Ia`Tnfvf8b+*GtJFU;;G!$K)!_558n>; z!*?dNI5XytJ^TRB7T*nB8wA>F_t_)C2w%@PeffR92E={pLvtXw z&;uWS1K3aCJeho!pK`lPfvXk!o0PYUo&a3?O1SzMKexX)-{?I)&Q}69jpaT5AUpIQ zpUYX~0|#-QMcLPakD!ZK(+lYxc;WV2CEik5ubp)W`v7(sZ=8Sbk{(1~J-)-?jZah; z_=vo~x6r@5J$TW!%+SpJF^L~CFWPaRd`*gJ;e%w;nQK^tW*u-5do6+=GqA_dzRcP@ zORGCR8hPCIc<`*I26?n7y9sn|LC%Hl4Z%GcbLj69o!2YoYQ7GGcGjp`M5+iIGt%2)tkH#4U{?Ic$d<}3NBIY5UDG_ubev!PK z1C6L}EA{~?zv;v{V_;vAPtWA@X(GEveP~Zj%!%Zk*YUijfJb%oy=9qR9@-AF6HL1Q zL=fw@UOTiT0l7`y-%YDQ{Mlo7AE`afllCOM+DdiML$#!bK7!uk^>}wh41afpnCofz zK0gacIMQgBz$`}5qjN^ z=ZP++D`gv$3jE|fS#f3$dJ=k2`3|X8tUt2lEAcK$#DO*(>b6BC(3u^wXCg-#gF3{d zlgSUkTFT^mG>ZB`sE7WXd=~P03%+B<;QNfnJ$fF5=CcN6<`2J!+QC1dHDN}1We{^> z4fZ>&9b*3u@=35|UuR$^`4P`m3HhP30{q+HAGRVkC%tNgF9dzw2!3ToAwLlH&j8yw z(7Fh+sKZ(^D&8qVvI9FSd=ALZt-!&P=Njymp|zOLS;*A63$7;8!StPHC%%tE`CYAYjDGLJ@kq9j`vlOyu)9tU2hv9^%UZEVRj)aQbK591#BxrOvzIK{gj^q8Lp z`yb`FPcboLn=+eFu7mvQP%gqdKXW6Qlk@}4Z4>GS07ojzww`<^oTtSzLaaVIiurt( zH^Vn`s?k^3uOa|vbGnSrg6+ntff zNkS&XyL^bo5%xTcMeiv|h0ehm82$b)uD;(cFPa_~9I{2PiRL;!T9f)?E_^ecbNvA- zM^4=Km=&(AQ|Y{ufri z9<*Eki?)9KV8DXy8-V*tKJ@7+AAYgc?+F19?$Fln0bS}wL;nKSo;w2SIZyrup0HZM z6V$1B5`?xFb247n-zEhX?bi_oSR^dF7fRSuqkZi9VgkeexhPx}+Cki|Ouo`~P;k>}cnGtQ-_ z?l`ao_gbKT#}$|C-h+OvxL?u_`+1ts<3_}=qtKrC+TO?7tU>!@G(PMu;Yj#GpEMJm zxp?+a#`8(EsXOoT!jW?DY8=@)S2p322i$z&V#R$WF0>X1S2?xltkj_Gajul*r}3zSA(H{)dG6W6(j3 zIFCm;eHyO~`F`?C$QS0gx$g_VF&Z|*H-_z=IQssT1fsIpIm@OM43&===q~1=pU$ zYythYKjw$;Z;(AX3SD_4;*HJFkcREKJh%G0*4bZ zjT78Bv|{ZjZ5_uUY9gfAxAM z9XhP@IIqXsNari&>qT?;Je~Wbj<<%W&wQRdo+kPAe1ozT zu`T&l@cZ~V2j0%@3(>Qu!Z)Wp8`k9sws4Z|gw*ylzEyDw?_A*X%%6nk5s@pgS~EXE zJ%@ue+lG6x9P6R8D1KMa{DkuR*6>$(JkRUzLe2qvq<0{|S38mS>|{yS#2E&nHR(Uz zw!+{%Exr3eK6lpw-R{nNNVyAIv&bhk(mvc=;3(~9dih}*=sOplyH(qI`T5H}=ts8Y z!dm2zHZt2T=!N<}Ce)9?Iy|1oW*qw?p0U-kW}NZDIW74A`-Fa^_r#tz>ktZ0!AO3#PI#<;L)*Hbxhl}Z|QDY$18v*@%}aF zs9nz_)UU^RH2Cw`>yi7|5QejGJ6U}!aO&I5`eTE9$@m0YIr>olRP=Apf&H5Ve+V{i z%Rq64Mdw$SX$Ky<;6D$}(Ii*s9@;**d7a_Gw?E$g44#_=EP597=Ml_#ewK9BsYceTo^KLTY{_}+uJhzvNblxII+j7lCTa(5 zX@~z1jD4i9cP7{Cw7M+}XK%J^BT6>E^w3}p?ziGu%p*C5=HCPFvsiN^ClrJ5F{JOS zo<~e`4(pQMH9_Y*5bI`=%oM@yh;Oh`{ZOnQI+HcnT~GQGdrA`xz=h^cI`&({9CZGT zzRfvT6C8s5t!dz42y)3&+J_jNGl1jyS%&5&zoGNm@f`IbKiw9h%~yW9<7Wtm5s0VH zxUzG);~#Phm##lO@BZz`nG1PLL!6q*-vdng^Bl(I^Ofxow*f=Ay+h7Qj7Ri9yo2@W zNaQwP2&ajgR$N~Xv5x@mQF6@Y{)VlHPi5NTne#>^57e~fbpjXuy8zg}iH6R$ zL*xcTTt@g&zMI|waqPbg_1_M_b7zo~Z=suUrk?BO9idfmt)Rh9zJ9x=73pOKZrJ}8 z&yv#^27G%|#TQ3n<3t;MuUgTEf2&xY4}7)GePBE8Lmyx7N88mG&+_zR6OSSOA)keG z0X+{-d)Z_QXpJ!JMU$>SjJj_j^9*|Fs}NSP6S|`nd&oQd!naUP`v~PQ{6rn}W_B*s z1&X@QQ3t((eb>(4KK>IuC+p*z(3gta)qv9#ZqRjI;s)9361N9X=h~-%&ecH1{h)KT z(7FD|(F7nrjU4qx%1_hT4bL;pq&sQ;q)T_eFH42p(P=dN**jjw-sQv4aZiLCdTg-9 z;N0E^`^Z@H=Y}H(gL4-h&<*`acj6h2W7@dUaltLO6kRhbu04qM>tjdj=r?$<%P(?^ zP7}E$0)K=0u$$H*u81DUDmq|m@534SGheZ|Gl)4_aldT@p3xh?Z1TR42|9`LeB|@X zFdu~M6mQJgJqxiz0`{TWF}_?^yieS3qvxX)cob_5&D6dQ{{Qa4$SrZed%Yj~5jLZL zy`QzeUbA^{7VfPY!S)YAt{VGf{o8l4IEsaQ;q^6eSL2n1HF^xPMB`|XtKE%xpX-k^ z;_!QSV4hxh%J{&BQLbt&#_fQehV2huJAmz{zQ|QHLpL=S1+wPV$Q2-l=sz0u=b+1r z=CIw|pN4EJ;S#-xKSc8=(3|3Wf<^v}6Z~50bWNfDG}h(=k&mY$Cx-W0^*{Cui#vw7 z`jhSO;zN{uln)*pfL|fgbnjVymYeSdQk+9LTv$6BvX=rGOol8bK_(NScd_q-eG<|o zgu~ld>nAhz)TiP7wXu{3G)}kW=<(bHcsF;QHfs#@ZHY0~HWt@=jZwC7xISmp+2Y|R z{S9r$YC*az5S(FWNX2Zl~I@|DGdt(iZ&Gj3~`Y&6$Y`6anx~xr0^LGc_+BfUz zrKa6%T~t=f(%jtzLk$mnQ&X_J@P^a_L2C+kTW(!=VA2}P?!5CiAIKYe#{=5pyxn&) zzqyGlU@pO9ED(P|_#1#f?OfQniXG=&*}-^Ddo(l5n5$uP=MFPG5K^;zw;{$bcNXgP zn9FE?A9aX94Umb^Iv;g-W-%9fZgejnb$Abo6*4oruaCM2u{Sq5+(+FYvDV|F3^U?@ zXB_a<)fDe$!1H9rUjD8PnHM_H*#|FbC0^7@yr?B!;G60vh!=?GP7p8Pf1My+KrfuQ z!VB=}GI?=f?H+vpg4q~kn(Z2cv)zdCV#kthZph0XYkLacRz+NkSW3}OKS8uxGC{Pv zcY^Isw2Sa==U}=i)DG_koI>p!d9_n)eo@v;_zFsR4z_w~r*>}*Lw*(Qu;xzmMUImB zMIYkj5JfpmQSPrO4_1_~Qj~`)%2zAOl#i3)BOl~0U#lojRg|w&ly6j&6BXszit=1V zIbBiCRFrR4ly6a#f2}AlQiWUnW9{&D6dqM*C@(&D9U##%6BWuzg3j)SCk)6 zlpj`eY8+ZE;K73CKd<(C!ZR~6+q73H@S<#!e3KP$?AQdR87VeX=Pc(YfMaeE2Fx~b1x5#s!UR2(Try8b)%rt z=vnD z%4gMO@=&Jybd-&Dr(<#q+8Nr05F!9hey}Gn?p&48@ReD&rs4U+ER89H%6{Z#D${M{fKcA?fFECf? zePYbI?c!sK3TawYW-(S3TMCLz`SXj*N)$v)7UcC}r*u!rb-`h&!oy*&8^mF+8<>x& z%Ii+PNyby+s{B*M8BXK0vLs1FO!?p#IKe7SD~#Z8S)rQn644>td6hNgRc2`gCmlv5T)3B0qOshp(erHj6K9-x}#_VD&TWTZC7lM_qPe{oc zmz15Uw^Wyxn=7lv>WeM<8gq4}ex<3>0*#_K7w9v}O=T8KsU9@Yo65}9g+;T3)L2Y; z)e!NT@#*HQ>b#w~Ej1MyOA{&!t5=xHsytxR%2pani}Up*rW$>zsjRT7D4xmpidVXC zNyLDf&~tgx8!Ib~HGF}gXS_@-U-iW+<0T0zF4J=+DOn-#cTYa62;>z4>cJ87DzC=C zTyBZRh(Ch$#{=08c7npK7{j-w6}Gwz)>dGCJ@cWb*u?lr z@e`qyu@H(&jmt_+thh{xkn~Hw>y409DG-u=5JF`pQiXX{zD6X&0)J@^0AC7 zq2JAA9{DRU=C3Taz#54*T>LzUT58Jjatew|O*x=+VOOo@rl;wRd3j`&s?2&}5+#Wi z^6UYa4Z~LeViFSi#ZW)yRhD?kXmC>s6So)&WPv%~G$A(5vmm%`f)zx!sv<0ta&QzT za0NEsx`a3zi2qB0b%D^LvSO%Ib9HGxDKWlkAa1h3CPQ7mN}>m=hCZStUSO^+%fC)P zDxa0F%GZ|*v#&3P3sPn&w!mem(ysy?^pz&qGw6F$ydGV72M=o{&6ib$;XEuYy!;{1 zl@Bq>;+w8msu`=wOqJac!iynTM#Kflt%;>_vdNNHSq$4->6^WrnqEMJY)yzQH&zvm zl_agFeNC%D%lyUW$`V*hEG=@Yg9)|H{l16?-O` zQdYf!!P%=eWfXW3DXA2byS_7x;uN>2^Ys_On~%xhrIeX-s?0ei8c+P9BB+7DLLW0z zpT21Re6}ckUV6sjbe6I(C368hpNy3BY?hLhnUu;>($cdt@&63mUPAxBHZg4uZ?YsM ziKQeaB+q6kNvUb`lUYi_oP;z0NzYi20w@VdNhw(@Wp=`%`2d!lkxlZIlDH^qDN9LN zl9q*=OOjJ=rYh-@O_G?BmL)FfHz&+T_h@HTf!x4AyCO$ zn6?ltGc#sm)CKd?AiXK#R&oZ=fe$wa)$=nl76NL*f|TrxjQJVqd@k833l7iG^LH(f+D@yvoy67h`;%(SAs3X#n+ zwhr8Y4a1lgEGq<4a!P{Y1nyUI(p6WQ^i`ELu(#w6 zlKGUe{Z(^&t44%<-dP0;i5eN%Dd4oxT z*GqU5p??+TBhMWERY7qfs`7zZB_(-EK?s^LZ_&jV`XUGsoaYtAuo&>r^b)s<@;$URcz4 z(w&l3&Eia7h((Ykh3WtCi59}#l)ER&=|tWyl96D*vKoD**<982^b5uG^_V;eLK#eI z|Nl2d+*3D_-&m<%RRq6Kgh_mIBuYdg5?T|ISqPO@au>6b#2)emY49a3O6n*G#>Mkr zDnddtlpCR!ua(p?*U99imsfe;L%cxE&A6Wh-zz;ianWpLAMg6iq?@~~NB``!1t}Tb zLg>^&&iELskWOC)yP&VCDJO*!&+shCPll>DM*XuP~2ALBx2kv{_M53E*L3cHvS`GMU`2|CgQ5%f8$t`Kz7s~ql^ zH|zo+D-r8*Ap4j)z^s&JCmUus2Q(}12^X88>|;mekJZmk%ScD;kgOD>Ig|$?dmvId z338g9nCHj0bWtnAD$&DL1_c>YF0KR}P3fVM~LqZ18M?bPVYG0$r z-MG<`T?5HhZ218l^nJ(k&>_#0`u`8Z0RGhL3-EwQjp#)0HKqD%*a?EC#^_CEr`sKQWKu9NQgtDU?;&C&$#;Y zBAk!7iRe^~RLJ;bzCOg17aOY(?^lkTou5USRX}4P6zM9J6A27_^cx}isk-nD6*v`U z7|Tk10cU=fov}Jrnc&F5nnFn$OG!c0C|phK8JSTBD5x%|G_T+$n2)PhcyFXt%S_lC z(bKLZPtkkaRd?>ssIP*LZLUO4*-M3Ech9026DFY)8!A;$Jji-iR(jgfwhs`w4$G|+ zdl6`7;)qFl-onbwrKQ9PX>hK6WVONlX|8`t&4npEwH}+Mhx z9KlF?3apDJsP;}@^AO2gP(Z7Qyebj7SkPZ=E%Eh{gUBEU(w$dgL0W*Xb-sU}V1a|j z)9qq+17;Q?gBS*J0FPFmz4b20^%@p6BULi3T%nTu3S zd2zW3;j<~9ra0-^yd2B2=}eyTq-!aiRJ|gH?|54DH_p^g)L(ryc7NQrZkRcV?hKdd z?x`BChMltp`#u~#i(=6uN6wm-$)aMD*;Nx$TrS3N?uZQ=K72SEf&by3bm2u$Ijjdw z00{pco)ENZtTF6=UcD*!(Lryo2>P#+%gd)cop)^KL-|i$yZN)Q-|zms=Bi^KwQm08 z!$-!wP#buZizG;a4lZNv7|=J$PN*iZU~<*60TKYy9<=BC>ZemeO0(Jxlq)O7IV zf8TXr5TY zqsofHr}iJ!-Z1B&|GW!#_5QG=Ddg$1yPBqyZhJBD(EG1_W^lg#?2*!sudTlItCYX@ zKJ@w0nfw3v?1WFQzK(r)ll7*5yccP7>}fjL7&GGSmu`Oap;!Oj=jk_gKe^j4R<}7I z=fzKYy*u()=nKDDXCB{ge$kRY4m0kZIj-R4+Jw(n23CBgd0qeEw$Xh*`r)3tUZ{v} zdO7gvqkFbbI=H+2srQ3cPj<3C_wNdRIBHv1+iSg-uOByah3=8ke4FOhyl+n4R8j(^ zG4$gJrn}PFvY)U?~iIK zTKv*oW_|F%g68c8maJ zjk~{p`K3LN^*a2r_OTCNsG2e7qi_GT>cihfT>Y8<#>met{X_GPY`Q*w>y67-^xk#L z@-?A<4?FrhSMY<^JkLU2crxhrzD)g+_L7=zJHnfMWOktW~cLqH}6k+^Ob=g-QSSgzVpzbJ09G>3SB1NN zxW8;-&6~xu?|sSm!QhPruh)HO9yRrFQRYLddi_N^C-k-ENWbCZuMSxB=Jl_>6C3)* zix1q=IL3eZOSeAb`o~)n|L$no^5iFDhdux0uNvOUfBfp7^1gdw=kn^12Uqyj2L?Ye zYiii>y9$FU2P|d*ch24OQ23hNho2ww!rJLWUhehy{U1Ki`^}HOe&vVH*TsDGS?IwJ z58R!2_~4HnFMUx|w(-;Gzec{f;a68TpKe>#U|u!nm8frS(QPhY-pBRl>jMjJ4b}8> zJ!$%{;^&uj9B`EwQ~q8Wx%zNO-T!>p`ybgG{nOuiNjt-Mb>TB-B6A%J<`k!WwyNyL zxy$!I|M@M4J}C(O%5dWPkEdll|9bs9Prml$`oF(8=a{Q$!qA`Ic;mNkz53?~55AQ3 zhntPD+MA(`JMQnbZ^%~x zx7_!GAC3|;mN0S1pgWgMy>?&LxH10kl;)?LTy^{1&)jnJe?Od%q5JJ8@1?!=*8bnN z|8Vw$Pfi}`mC^Lw%?1QQGx81hto1vwpuCHb#9DFkDmcKqVa??Xc zBDUYFpZ(%9##Qxi{-Le?$A3D0ZXdbv%YCL7-}%)3{B5@#-#$Az_Ae#o;tS;~pD&p> z&FHMg|7I2?7d}B9BtIC-MG7JaEIXTuE8z1ySoH;2@nV#Toc>_1h*i;AxLmd zaEBmgc*6I-b^e~e-*naLHLGp9uikeR3`1{TV$5@H>D4+og8>%}=R;NtyNl~{hPI|2 zwgQhN;%Lr1xWvEhxp=MdVW>VU<qx!OJ^HQ;s}fLZLK)=d4+Q#@lk20sYxp7^D7wTqk~g->j%jrcbEF}rmiGAK6WdO8lt&j%DOt3 zsJH}_IB1K`yi{3YltQI;4;DLbcl@alMzqa&C*BlRB|&WD#^95WmU0+Bv@=w%Q&k>K zh%_*VV&>Y{Aw+zklKuW0M}2}`I^eIhm&L}FkI3t0St@RKA_NjH22AL)c6EjNcv&^) z6$>A>;{fBw^PSz!o`*;Oq={Lfyphp6?wHUvxTH*PjrLkTi_$WM^iToB@^p&tcZ~>9 zBUR|i2FFII_7~D)A`1?2s6Flym~YP^DaC$wJKJ{$iFy5$ndeS}-C=&;-v|_kg%PzA z#)!G-wY{}Y-}91KSg?Ofs?3aI6u^y~%}hl5ix?2b1k%5%vxVj*npVY5syBYX?Z5b? zeR15EQrdGoo4Rnd-wsz*7Ej!4x5AdqaLf>X=wmlrA?rNtL}YNo!XUlA&b?xp2iG5H zrm?3%VR5dseO?Dy)KgAml@!KGlb1{1Qq{k8o4fMBKYBjr_pmqKZdV@k@U|{P7$LVV zkT>j04HYrdDGnSLyEFr-n-?2g!@9HV<%FFocc`?lSDk^`oB6vDw4L>-(Z$zKkKKxwN%1!QWv^?i5L!-hG4nY_MPghZsFI(@KaXt>gyT#Ylx~MKAT z+@GEi*gF4s=x|(Rm=;wm)C7S^w7Kv!l=WjxWb<1>1oh;f@QBPgh(a>@NX~^c#4_Kv zNJza3UeS{nlU5+TCdx87B|)Ml%n{mA;;ZkqiU1O`XqwjEVzLOJy#+cE8TJ!Tq3B~wa2+B^!1j2$wZb1HJW z25*K}E3J&u#`p9th{ALi#4gOep7l(bLzgU-1w602;xnDEcXeEQM>N{$9eCM7sZS^j zlw=Mwn~XitD;jZ~M%zwt?-hG8@pLy2rQi zc2%8BFXGTrw>Gj!hcoGq0n}f5{kKw1h7PYQje>}{)|*|f*N(<(Hx`QBHudWTckC7( zciF94wqa0ww|M1gj?u`rjboF+G|bb^ zep7tx4_ZsP8EnrTuMEgZ#p2D*_qqE-<=<=-^P~pZt^^nl`A?P&PXB7a2@5n zP3(BW!Ka{@rEBrn5IN}RM#_PxDD%wFPE?(cof^`h$;IVhjnU*Vvl7*Cetf)06xZ2^ zyortSv{tgRAJCScZyk%i_$dX=#No`e`mh-%?TnBV!owwPX5@r4(z zX3x_bdXL{|C4e{qMAq5xs=P)vI!N0%(G8DG+ZLLPjId}}9Vgee3t8 zmyR0B&qGn`PSuPW5V^T+CZDqk4WVcDP2LL!e3b&~S2FvnD`AW|rXK$(~%m6hS(2ad)46DCGTn z=b4;Cy4qXSL?f-O5Pmit{!oO3(j7;RSMc2tC=Mn3jYaFC-?}62VP4bA;mNtwIQNbXMLBLDtSsdhHIq(x5!q@n*YTPyJ zyW23uX_2-}D*kafU**H7A%+y}k&>rb^y!ay8p-87u^V$$pVzmXq{(87wKVFTbD4kk z2X5=*@}-bWQBMfqY!vf5RI>z^xM{<V5!NOhs zMJ@(IYDWZ$EI72yGihxT>q=kmL3dv+U0Cy!95%#W(nNUX^N-kp@p7I3Ua`PA1DamP z`e;x8xufAer#TW`#M=DzEmG|BV4pIDd7BZeI#jPO`uROy`33?Yf*#AWw=;04kht@> z9aV|nm*QGR^tdbvGp#r=$$gzpISGbF__)(j>nDd&NXk-6Br7~6!K~}}xA$skza+^@ z^;LPzsP7MC_Y{+_mZz{VE6?M5yb`+VK4-@NwJLuqofn8zg+hZLg;riTr>@sBE>Mbc@&Y)g_v zL<@O7cam84&Y$%qt)Ac=Y0`X+E1qr}8YL5!RC5R{nRC^o>qWK-+PtJcY0;;1k(!g{ zJpQ%1yRaCf+OJ(G<#f(xNDLj=~DH=H=hi z+!K|+)Uly_E!zn7=_cDOk8Y=TH+{Ttabz5e*KfxS=1capI8kGLAd*e;Sexkii5*8at@-W5bgD+BO*Jl^XNJ1TnRwXa?j0Z!tB4(MS8Tu;IwF6CB4Y|{aoy`g7gXgN?nF8vUg6|z6Jr|BQru8e6knew>PzVN4 zi9=KwZb*}Qh4_7_*{P+-h=@Deq!|$(mcCNMwUpctf%RdRDiIE`AAnN-n;N`1b=A&M zru8>8n0p}E0N0(;-Ba!ybq-a6wuu9s7IrcZ)$+jEec(%%K}qUFa0Hcc*$zR?l(Zr6 zm&R<;-gDhV3U|3yq$vvmcdi6u_zF9JGnL*YgWIvYMoT?4m)4o4Nk-{vbQe*2JCa9> zuQYrIeXxbw*6Jxj0wF`vUUSqnIo`Moyw@2v{2+P} zc`38I9-M8TtCjXlmCUwzB<;_Jhsa#$ncad}iDohj+j){kbg#ECe0Wwk+dQ+yquP5O zP1n`^xoIDk_Hg#?x;da)q-3 z>73zT=}5N2ZmgtwW#3Rg>U@Av+-tLN;qC6{j@m0mS{5nUDqZJ0KLq{mdi4LEwG*p$a&ju=-vWGqz z9zswT?C}gKXrHbczA>-P1U(eH4=TlB-(?d?dNZ+8N!`naK`NSTiRgdpi$6#ysu4Vb zzK$C-AATPH#nbvDNclrbRjw%!xZ+7!^+{tYB%8W(Evzsk4k79XL#k-BmXulfy(fdV zeqB9YD_A-0&L)WbvvpdhS&(y3!jmyWiG`$ZxSYkq<3(lC>5_@mBi0jUDcPq|pnbj~nr)_Snrr7&bW|ClZFy{QR0ehsiuZVrM*GTsPg%|SXymdFu`t7LfC&+Wv{|I5O@~O|TUC9RYk@bM z>0L)KP)}^9#`U!Y_3;}1J+j);)Zl%&w3x0hVa&Sxt2D_B3H{%H<`Df-b9q1Qy(gmF`}{(N|gb^-siT-K{vB`9_A&iK8el11`<4xf(9 zLFLMZ$SDp_eCgH)!@sWunViZqo;Xa&$GHQ;h;17f93R;ez7^bdqfj83uLN`rSNxWC z3T%p>SFTTV%|6y!KJxqJBSec^k>Ik6pwG-y%JTIy)4wAE7JDnT6;P+>`wQ) zpV&2|?*@M;OEe3O%Az+ch&9qGnOKH|p^r20@c@=Vwym zTI2-Kp~=dZG@@9-QS6Z|%VX>;VHEp6eqfWy+d;v5{U|hS%^76C)G?4CL zV2m{^c6O5K6WaTy73llKz$H+Nl|R%DBC{MKFJ!kp_$%*nPYJozcpbg=8((3tv_E-{co2dO!VOwKcD%&VbU(g+$A9RfYUzwV+FJ2txmHYY%-c2 z)cem-MpDpsBfa_bY4Gk@-M5pnpV$NW5gHcvH3Ll-R7?jEk;`qo(_fE%>Dzi9{|!i9 zC7@^aqt#ZJ^CxV3(@BqiI_?{Hm|qe6=o_!N{+v1yPjNS_7uR`qfY zEm*W(1>~VVAiyT?uJh0yq$XwaeH*7^AGT?vqtJn%)O<`JITH}T4gXYv>`;LTM~L{> zOYL6SpZe{T<(QqNL+jX{ItjeKj%`)9cu1~?+$En{v02$p6*j70!@bCOXKaR8Fq8ge zjNYAfV*SW+R-XgcpSY~xwZB5dcUZ_}r{{BGfiUP2ape8)IpeUQ8KN*=QN+OKIi=ojw=Eq>gS zN(aVvOG5F{IK5&{rZo3JeX zjeg~H^7wWAhpxY7Kg;6K>cAiWnOdcfMYnC_)$d8#8!egOJ1}A*f|``sW4rgOljwgE zWf}$udJY{&z3Y_2vC{U=w6j|KTW=dp*6yaB-psG558ji9QyvV}M+B{_sGHgl6h-ky#%|e==FzW62XW)#L zTr+)(C0Jahwg(|HvD^?38;{Uo!rl_OQp`}#?i#Zch&^!1JIjd(&Bf-DjEbb#+7`SI z(ZdbDd`NFo<7KL$&Z+%8Rt=Niif}M__N22IuG`<`VC^zO_~YsF8(M&3`^wpD z%in2I~!-D!wop^DYN!% z@}}zEPLv*=l|q<}0)6v?UZ(QDa@8N__AVFr1I!)R5PD7+*Uu%%QZb+s(~LyQuZD=blC+3z4U}=JwI^|4ve+?Wa4Im#78u#u z{2SRYYP7g>gon5}&n87qZ2q5fgdCwjLYp6$f`7CY2_%u>A3 z`O4_gcP6$YXU7*KI+QvK-gg5FQ{2=m1}$wX*X{-L>;Z!F>2<3Uc!FXR-%bY3X*6}t z<1*SVpzx?K%9jKm>?aEz21vy2oMjQU4HYV}t!Uh>ol)$t zJvjcNQ8LoD@kk-M&ZHl&ZUiAQNu>fc>CX5t)#J!5b(O+4myqrvw`L13UBosz{bLDr zOk-eM%&U5x_(Y+Bc>dGXu&2LbVVXPzp^IMyL!CVq!v)kO!;^ALLr5|BLK=RN#k$OQ z#4ZsT#A$r&jeFE}%2^3I&2d-6%=p_Am!XRKF);-iDN%T6FRfPEGmR+bv?1uVQv<_h zTz#uItD-6)p~#8=8&yspm(ftEMtyg z!NiSy{q8i(Eps}Y9lO`78|>M8uJp0bzZa=*4vj~q#J5}qxk+9$ly^py{$fn-;89&p zn!V=D_jZ;yX3in{66M7D`xZ}nmVZq2Mrpm(5SVw=3fEyYnVhmTBMONTny%H&)d{7o|Cuo*mh4d*xejFSZ}_SV{N(A5R&_`5XLr-kwEjQlT=+kp}l7^ zL8Dp=f&b!23=dHN1JPHO1>tD!4ZHq(2UdaRdHh3#+MQiH3#?z9t6;rK%N!)eKIPim z&8PWeK0Gt7vk)`7ROii0w_zvnB|2kbTtaDDk*<@r)4PI75X;bM&=B2Tl4~Qq+2S;d z6BWu?rElJCx|%wY?1?+9Fr_aqH!`!3_`q8GxZO{~em=Srp2VO%*0WmHo2FgiuhN^9 z_ut5Vq0CLuKSp#jygTIRCZOHz$yCJQwjqY)`t5arF=zf6Qxi>Rr^HBMo0{`0s@S6g zO3%mKTekkVySl1?TDeWuy5XoEX}+J6QmyrrRVzN%l?IqFx>e%1Z<*=00x?DWv*LT0=|k zAwH_qnL7Ss##}%s%!{|)6-c0#_k^EuWkd6voWrhCrVXRoF|JzlLj0T-Gj1 zHWJioHhT@2+<$k)sT)2G5EX`GQ5Oo9%Aea#@R+sSiX{KokV1jGY6;6Q$_rCoI(+)} zdf{v#n~~s~))S$y5X#Y*1L@%#*n>Wr_};c%+Rg| zPdV)7ONt!{Sl3I@Z~{2gmcGYnksqYTH zExp*E#`T)sYn`jMlG~k^n-}CvWAO`i#e7)0ctG^o4Lf)9bGa^bwRGOI z{J8tnY)Mky_1M`ebTMEcZ*Tr)x-ZnXNT1wh{i7llX!rwY#R#08`#nad3Lqu@gn|J%9mKWp0e0_Kg$0X(XgN&Z|7pxMmsm zzpuBgB4>0vPZa-F*lZD}llphW>NQTEGBLYsG=`_n^^kD(i{L3y$R-@KP13@{lyrW^ z-Six^PdTy(M3D|r@LeI(xxMTC``*-K4{tbP*;tKgoH|RcQ4Q)($;cPv5A7UP!D4*# zmfZOs9@PjR#C|5Rs(sQip!?nRB;aO^j|P3xO6-jU+lQ81G@mLUvAJHjeo|x`7c+gv zk|F>0d!JxB^KO$ECCJfuX!s|Gn+ck zE7O=SX{eaWde>ccux-}np%j(?IZKlr zz>+D6(8Q~qNV;ma8}ZJ$Kp7ozcD{~4F;J6PVE0;h;PCxItq0RE-LH{HBPp9h!Z_$O z2kerti*~V^P^!(s=kXu#L97@>%}LpmLZ}Syn-W51s9Bz8({~{T7#*xZrjxlZL;oA93vY}F!Wb}<}ULbqldklPNR4R4K0))_Fu_$TN@Zjf^x}haUT@%T>$NnOQ!`v4YyJMYL+70#Ax3P&Bl4c^ zgW?-lZ}w%GUkgDNgQb{MrZZEBQJ+N>n0u+(biXy--T&~M!L8Z)SWn!+`DDuE@fPDi zuf{2W5C@j7K=D5S&Xwwgoa=)TEnMO%fDlO+>Z z(sM4aaX`R1P}y!!U@%0P8X5JLP!)Ie8?(_8a->&3#yUHuA|1s*>i% zA7-6@^eG4>yl7*Ewk=uSP|NnQHY1qmPT%6tinY{e1gE}bz?y%Gkn^m+S>nNsB_g=L zjMmZ3J}hJ22;7fK>KrsZW{w#Aju`&8Ila%YPv>_7=tqBy=_);oNlqj@RfrmOHtg+N4^RzW@`0 zP>z(t*l&TSIAvK8(2p&Zn*e&unxG?oeg?_U1UEDJA9IK-!UgW_fpBpCs)s&D=(4j7 zwa{N^PV#^5tSFSY=N^rtJ7v~}u!!`C1GDWDV$Sc8VJlB^WZ**$Z8jc8>10yMescWg z&q&Rrc~c>3qY`ACV6G}9fHwHlu~Z>w`B^B znq_?D4W&wo3j{mUlH{>at@(a95jki32PiFQUg_8$LDC;Q8K9Jv3(1+U5zJ4PrpR8I zd0X_nJ(bDZ(1UzOw+U&UC@1r^yvt$w$|x61A#Bha;a^aSH08p3FM zZN62lBf&ry&5gGgjf4J9jQJ+F*ZGPoqS9U=)PH7uApg*Mt5>rppETDuAC7inYmroU zprzzI)FOs%For!OL>mbykE=d^d(Vep`?R}0kLiX#r< z{_BE~{v$$@AP&mC0@W<>-MHqALNhduoeu8SzUBAm!4Q&V!T9%(ef|BhJEiZ)3w=et z=cz@SZ6|d69z28<4PhO<388ma8@y0j+D^T6$x~SPyj5fZJ#O&FYn1NF9F#sjNpDaNjCY9z zScGS*b#%M^#ty5R{!}t*$Fqr%pRfr}KwUCRRuR^m4%=-TZNF~MHc(UxuN*O5v)zm8 zYtW0=a~(1KH=w9YVrJdL39UGDzIaXMh^dEKDiBBLIMV@H+JA5y6@p49lcXJVBoiRxLFgk;q(WbXO!eMYL$qD>>7ype)BevyO3#}?3> zg~4!`HlaueMCE9X?Es!=Cou-iR;`@IYdp0D0I64LP8zi0wo>v(FdD9ejaK|ki=L2Y zGr(uVHjEFSkth~n%{ex-8_S6PE`Fs%vyHJUgvnChMLPVeV1D1*IWr!M)wWVU&@sMZ zJGruxV{%xfciE1p(CbSbu#O=%2t&J5=Ip5s?km6 z?V@QQ=?wOKT^r1?G!l2|Z*$sQ5fkpSWK`N=s}Agtkx7`=gjF|8${JIKc|g_F&q?&{ zJ0!Q3NpsU6>KWCc=r+-lu|{qklSNwNkG9x!d@wNR2LSdvQ70IwhE;A9{2E#={SMVn zCzGPZ&?at&WvHgJx2v~Inl*da`M{Dsiqo*UoJsrWHfrSsMn_Kjx5zSPWzP5x( zR;2cfZ1c+epSDW~?l8k_L5;UQOpAMB3>`~_5tGANt|qtBE2w*aGJzZkZl?slp4D~^ z$C^r{w#If#yv0hxP1_vMw>cA7kdSU8O3n8=)Mu8ip4x_g=+727p0w6$K_do{mRTgx z9N6u?7lpoRYG~+x#9x{j7+dN8Vp2Pk{1IYziO=zB^PO>!!X#Rf8+B)aK=edCR6;}0 zI^|N*!n|EUk{@(^xd>uEM6$&UpVjQH%Sh{$9BJn+Ro%oDv9xi&#&5J4`znZnK1j#< z`>U0phPSmzaY)999;IkRO6r|-Fi#VFLTxL`AMt7pD>oq=wv`?Ie`VS^)Nn)lNA)x) zcR4N@-Z0G4sYJB$S-T=4lC4+};s1o7yK{Hs>j_#xjA5!JsFbeWX)xhR%J-()@KdvflK2~%S)<(<5dU(!g@~=JZe8v``tuI*`^uC2G$%hZj3NqWaGUMbRk{YmEpjqr1 zlfo;Yp|N&$3VLHrFpFxh=Xy+`=^*vkMiVEDQR*##5`UJe&vm*2*1CZO(|Nho-mc&#~(g+x_ytE zk)Vmky$DZv*Jt~t$=gu-mu&4FIRww9Jljvc;P?Zw-`}70)hfh43SF zqvnrcrySQu&r5NPB?P3j9#a8t{PIXeh@R%cKYGxpXRMs-bbq5}*%Dj$v+YmY!{NHs zjYi*DNtY@`T}VJW9@4P~^K8CvWQpo{8|r?&ocU^yLv+RuPPbEGZ`Ka}$0)!{jyn>D zgO=BDSD^pk-^MeRY;x~X>YXMgJM4y9HH|MUhAt^WsB)mG%h5P7GfdK6nhSxv9nIjgi`~C_b83e~g6(pj$6}SqpRIS5Zx& zbny^4n#Ot(LT1W49dzE^D~REa7G?c%P7G-h4SX$+aLY~3hkNiloZ>RLaZJYIj?oZx zxG${V^W;;N^ea|1v7m? zTs-^~#4^8Eqxw(u2`3|>g_L5M*U5II|NT*VK0^ME*9*s7UFL2t5_xx-Jc3>sfae!aUjm>R~7B=Drjag`q9c#o#Q zuq{$_!V;ReFI69Ck#wT#f7jJi!sZ{+`Bu>ZvjpZ3(ij)?By2S6ki*wi-{>novyFkd zuk^kfdE)XYC>>C-*s~nwKS&c)<~=$)g$|H0sUig*9uXIsn>j0j`s8L zOGKASp}0VF70BiPw5}0Rek0N(t=aq3rC2upI@QdLC;?vL0%{qUB5Yhh06Sio!F zasP8^>=4wS(5&FoRz)VKLEHyh2UE>$uxcJ-Gl13_0UG{*YkKc-nR?(-Ew3_X&D~8} zdD4)#+go=W#MPRs9WQL^=p(}0jQyM2JzUQ1;I{c$B;n3DV3WYW|bPt@0*pK7iF z2MPplpmw(5!k^yr*iuWy8Va(s)jp5^Atos6L|xbk9S@k6$^Nxsg8j`g&^PKlqHDHE z)E}7NWKFWr^cFGZAVtw#h47C3k;ncKifJ+;u&6YoUUH|G?Kr1%!Ch#e_?Tcw-AUg{1bfZ!8APS&$E^I<_8xd(sox4u^*ipTa z2y68t*^$Tx$e-4P`W{OL>mq>+LonRF1`V0GY5#V6XXeBA?RMEP=Hs9;mdKyK7)8)e z>9jM%#^hBUeo!Fbk7Rap4xRbJHT%{vHemiXto=;kTG#WXM|;)mNTXRGOBLk(%YTC2|ywfyr$u?B0_I{`hu!m-TysOT5C|Jyvvc1Y> z-jw!bWwiCtaeL*z*2WPSb%E-5DM;b)$z0o-y28&+iF)L{%HeK2Z{vrkpm!ayd!^?R zZBDDA(>$N0?4BZ|p**<@NjnJhJa?Rmf3QfDl&D>B*<}9YhUlr}_=*_JIvL6~JE}l3 zSA=CfO}r#I;SPa-Py3ykNI6M`^jg=SBzFSz3kLq$-`}DiAJwA1KJd%Ma}5RObLTD0 zaNuLyusWy9&N`@=&%wZBOn>VVnwTYY#*diNCpwCuC)E;%ArT&FyrG23_^Y6I@OYlw z^^kS!<)}fc=O7r!Vjb2DX?Gx8_x|)MCqARF5uZm|;=e+k@sgoOlY>cw4Bl~toT|79MdQ9g#WHGc8&k0N()Uj;w>y13}Z z`FUNgRC$IG9(?M9%!X~=M}mi(ZH?KWA&Id}cOHVZwi@i$@;R*6IU@4QEw^bC8==XH zjgz@$zJwL3A8NHhJw*lAAMNsZO25zBpGTB$Om`I%`9H|fops2Ia>Rtx{VfSzESwHA zwR?(0+)y^fV=yxbbj39vY~{DAHukE%FVm|?IbkVh;)VPySzD*9>yx9%M*c@GdVf%U z{iE{&i-G>4Gz|Uf;~30xs$?TyY+MHa=FEYJ9&wi-$3XmrF>3xbyw=QFP0r2fPdr)d z{#A3l%18`M9AzPlchBRW_MM>In>+6hHj$D`4k0Au@=xT}l-?Hg&@j4pi_ik%4?b{6 zDUT8p^`U_o%@H(aiGK+hZv72US0GGV8W23glHKLU?#@whf4yfnT8OnPe!S8?NOQsB zsX=}n;E19$Am{9oKz7Zg0T+Wn0+pb$OLb;nrQ|{9VGEBfm|+2+bl?k0&gXjPo@_|Y zUF?!DO=pr@w31kKd@|O(mM{@eHYbLIMah!*#jq2Y!m>%@h>$}_YfNBxFJo@n$ZYQ! zjff%NWtNIcB>9J3P4vjF%qm%X9TgjkpWf!R+qJdQ$OMi{$kz{Cx;e)P=$XG&et%1| zFPuTA^FDXLW@8@#2yvPO)&1GLqx2LdyN%>X1iU%PT`}q{%DI2&uI-Hr=#1CFVU%!5 z3{Iwzf8Y??1s0AAd*py9TLcX$uIMN#z^-9wkp0-9R$aKs`i5`*~6f$xdwXxs%f35C?A40k-LO{-s(dM5KO z$}g%Kph9?(vCFzr+tq36X#W*l#Udjrcx}^Ps5H5;;*uV#%Oz1kg@9N_u5yx~Z~t{? ziO#gU1e?)36@pXZkI*{?d!G^%45?!TY+@W18!~;FBlAe(WSt)!ANV~pj$uVkti`wH zaRMr*(P_0W9EeCi{Wj`vO*4y|$n~nZ-cY#Bu|T8viqB#9jl?!7(^%{0`7S2ZOeTvw zVwU0=^Nw?yqzR|788JM)6@lussJTNX{U*H_Y7P$k!#M=s;(HG<)kM%M zrv1f%(K3KwaUm13!Y}n{7g?0L6j&ujBQ7A5_MkTZq^6@Y6Q|2>|Ah*cx~EY5aeF18 zd3}jiX`qA%x|-Z*HA>$sbwk7Jb3eVpWgf*n;B6j`S&V2<-bQ0_?y*#`Ic)RuHg-4wJELxvBM zDIAEJCyof|EZ_+8o3M+)(&-I|pZm83w0d{YiW>D2A#h6?O=0kvW%;prmFg=h9C3}H zabY%b_~X{wMzHVfSt~eg&^ct#z7ana^?fWzA5FesMuT{*7 z18TDc0K_1HdffnmAU#jd4(jpt{G0)3D3CzClb}I8RzR(y|62?V>Tv@NYWNC$Kmq{9 zpoP!USpdyIlktD^|DA#M2ao}Mz#roIhybKH1ORgZ0YKA10uXVaCYZ>G0E9CV0JR2c zpc8}!K!Ex(0{|WXVT=zznG*p(9uWX$uy2^LALyLl0Wb&e0EqiQ(DuQgV}}C3OgI1~84W<5L<3Om z?*X`mL;%7(34l{f1|U^Z008|X0EwLn+CCM4EXoET8a{y-<^WJP#Q>mK0)V~^0P5Ub z0=kY;(D{}DkoRSv{272Q`V0W~p8=?$N&r@;8i0wb2EDEa-JeDPs-YP`7;Ob$ut4_+ z0{y>x_RJnMp$9$R?`Qw-Zh-vx{ygXBoBQmb{#gGxs2A6N4!ZaM`Ez|duLs?-{~Ynf zLA`SR%g8T|^5US|37XJBxAdQ*zc|K=W4<`(96=L$&~uZ~X9xA@`Olxr;5mcp>^}$f zZ~xB;UYzj7K_vp3(1V_9_}M`v`Ja=%IH+#_%VaN3{^ArbPWj?gFAgd*(1afJ-xp&* zPWzJo`=AVzLH*MHThH*~px#{nWl(Rf{~T17|2fNxzkYF0Zy?Zw9>n(I>@Uvo;-E4H zP3S?Op3(m~sK@Vr&hz5DFV6Sk{4Xx>;{V1jdXUgd7JhNiPykKnL831%_Tu6%F7e`$ z|2f%9eSrQa_~3Z~+5Z%e&`bVTF90C{#QQ$pz@Oc)e!(_yySoN1ORVe z^1r$Q)%;8TS6`qp`jY?E82~uG42jROF< zd&&RC0|?sTzxzw}!Vm5x{~I3wfbx?6jS~RCe98an3iO%LOa5110PyN1|EntiV0y_K zFURNpF9X06XnI)$HZTD&0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD z0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD0WbkD z0WbkD0WbkD0WbkD0WbkD0WbkD0Wg98-x4VO^7-Gtoc})(XqbH75gkyO`hPqCcsDQs zFaanEK*`M7&duG< z$lS!$%jUlY7Ot*NuFq>6EF9b{+@CjiPHyfdW_HiP$kN2d{{LQLkW@2yeWCn6;2Pe={GkYgDix)Gu0lll4lcS@B8E7wa zdlPeW&;oP&m-8{VcXGA>ouIkBn~wuvWZ`J;XyI<HlBnfC^qIZ&>z=Zf0eI*UTJKLDNzsw6yCHIR~K*9O8hY z(}hwKiz#ZXtgLXxii&B-v9fZ^`DaQE7L^s16;n<_lVfH#O3UB-UF&@h>ue7yIM3uf z|KH}>oBi4E{;vDpd+oK>*-Ia$jBJiEK`1)G?4m+%U>?OGn3rFWlUJBUdlY&fMb-Sz z&p^hQv<(&(WaiHY6?ls%ZuH-e_|yNq%v^JzFmN@JFE~Hwp{z3r4#*?YhbfQA6N;Bk z!nw`<&MU|%^vOl+Q8)IX{b;p(T{N~E^|n@Tx9aUSz1^<2RNU#DZ|L?BlcZt?scwT+ z&mrpAq3VQT>eS)t%(K)GvhR(;S&Q_!WqP|zZ|UNpn| z;6nAHYf?&3y-bpFx~r&mnSNhJp8A8!8<>!jm*H1`a#3-doHM&1qhNk2GV(cBP97D( zS6xN<-V92e29h>uAIcsk2i&kuOIP>1^T_#hDkJ0O7y5#9y$%_tR=SmYDwVfj(4U-> zpY1E~sLxy|M}b@qAD`ySEGqCpN5|Bo?r2r=G=g9K0=6rj$95HMl{+0JZ&B0?$|Dr{ zTcV;>?K&8=hpIfv4)yQJTJ#AKk%74KT?OfezQnmN`viuYuEl$ay;#;Vi+$gg@ zyD0_KV^KQVo1+SRR9e+rQM66T$t=tX4Vz3b8>5wbzkq!bO*ycW6nnL2Wi{kkJA`#* zG{RaNO*hUTqfJPcwF?jFeeI~U1akAq8Vpbdd>obH%e1(nU6g7{rJ}Meoi2lAVmb1X8_K|AY7&@(Ks!rXN?Wrh{l?uog?II-YE*;!&+ab7% z+Uul!qaXpBgm2K(};#ncNwZmg~Km(km^aFk3B;xriQd+VCJfIMr-#U;4b}cjJaEXNV2V8O68mA}D$13lmAGQW+g810e+ zbUQ4KO(~#qrEZIzl94|l$L~|iVyDf|%T)KpUX(|NR7EV^UUs2~(lb>(5*rBS6nh3` zWq9Z2(BrR;taVhMpcqmRl^aj7ow1~+Q^+yTAi8Ad_|!MCDn>sxC*|h*)y*;HaaR_D zJY5x&6HFtwle2yF;Q4DzuKF}aKW}b|%~$Wl2Gx)7V==Nr!ad; z!0Ss=A9Yl2b14Q2<~JX%@2I1+y(1;$&2XV5OupYI3{0GoC(4>h8|wfjK>pq*ipM7(NL1tM4QC-6lpUzF=lS zPJUsaAW41K+3WLD^@571-#6Wtn;*b^8h%p}?Cq@F#`Q^4J?eLzr)6hQP+)9!VnRkH zIW*s+Zt6nr^KHPhKz-i9l){{VM?KwzqWpIqW!fsY*Q_0At{Him*#Ww4^ab&LS8pJVDk({-fkgGsLR9rIwiO>jN;#iT$$EP?TXa{q zxl}uZw@K>$ZWPWZAlUj6w%=?-bdpqUx9~)v(nDufJ_oC=57u`xwZJ5`3r@TUapim1 zuKffjjV&n1m`^^(;l4gj=ea4jN$Re+XtnKioJ3En`{OcwbZLkCMmhC7owQ*y+-rOR zr{+0wiB9=Au5fH&VL{Gps+T3Hzms6aCJ46vi3Cw}myP3kQ<5q_N>`UqO``I26qO#G zNOd34nK2n^Wj9oSlhmeend-Z4$e(6+m&O(N)P}gAT5;6e41W$D@YA$oxQppWb50>e z^!cN7&4{DD;V9*vkx`gwp66XXY7_Zc@jXIU+oWS|XULOBlb8BF8JrL(z->{jIhwBF zo1!SLm3MX-n@wHL65!ca>wmiMA?wQjjVz z_jTu*g{tU2EsxxvuO8`sDc#srRd*`9BvS6#MLB-&w4!{vd0;njMyigAu9YigOvchwI zUe36TpwD{|on#VKZ9lrGFvssP7frI-+=FT(4x4CDu zTC)tteR)Mx?b_0l0)buiFq+UR7WdLl-r9@q z$V=eu+EQ$H>1_>d$rCCoYJD%fBW>)ZqRbn^>RzaN7O1tovec_2-byc@lozVCjI>_QLNpxdx@?IC2pepaJ+k0rhj$58JJbf+Dz|3=Dh^pi z^q5^1pJ{gPi>GA28P=S9AKl=6UQZ_STvf!+(=S(^!0f9~)m$4Nr5es9x3=!U6W6?A z{w+ScATWyf;_4nTSYJGg6+7V9)vEAK&=+L3%UADSPTx(p_ zu3hcDqpCWCs_Ac0`%$&s>RlC4i`?tl?}#diiubPTxYf1DRpBm=UFNQKuV@z&v&gl= zk`b=A1b+s7DQ7Qf21*Hz=n&)-IE6#0ICsjDIS+LT?1FQSGX!&t0-VquG-k9sA?A-bI+Q! z$W8W6{#C9jS5?Oy(bMDlZ+Cge>~fVxQ4nHg*0w8;8oDlK1%+)*yRxXVsQ&%?E7j=) zrKovDt?Fmq>v}RTA7c1;=9yy+pT?Xt&hRkip%)rHmpO*obg-l|_r2TjCCroWH=N6y z^r+zl%xnH?c$u*NuB1|rGt2iSH!@eeZO*S_9`LT=2IjT(hL1nd#J7QY7<0^f#-7fc z#2f^7P^$kpmp)-3IEHq)J*ylXt5h2E2hc;-3G3CtzT{h60DCow<6Jc9Wh z<}u9Cbe$l-Co&IVPGi1|c_#C<%(IvmGiNhD#GKFkJaaMgJIp1_P0XdteNHvuDQ6zZ zyqx(m<_hLQ=1S%==4$3gnAbA@nRx^AR^}S!&zb9(_cGTr$I(3o>D9n|5_2Q-ub6i+ zr!zM(`Bj#O=3|&knTIo%GoR1AocVI*3g#l_O6GFr zYUU@H*D}Asyn*>g<{IXc&M@JtW4?g7o;i=Xfq4OQBlBYBUCehdH!-hZRtru3S2D*i zuVId3{(sEz%&#*iFu%v#pSh7aiMfe+1as^F&VT0Pm?ttLR9+J_T|UnwyI1d-&8-f; zhd4{Q)F0zkXWxEm=+ji(1Sbs6~2_gQ%E$MB$o&G1D8J_;Xckpw>kyX!OFOr!FnMh~BnCUrhH+EXz9>y|Cy@wAGXs z_;1^xZ(@BkhhJFqYaDvp4t*6DkO48qe__#AJM^|4`pfCQg=HIuUs&|{QN|8N_-#A% zwRBxz$z}hAMK9Ns=xsanNtCZxo@2eR=tmi|zTV)!*inD$M`@PE^$pbDz%!Xqe*@2A zM*R((&5ZgRIG-8yH*hgC>Tlo@X4K!nrOc?mfyTlq+%&5PCH!!3A2CiX7{S92ljQSh6o*C(9Uthnb^6llc#J{}`?vWVz`to!BG<7uP zOIY$}faChI?Z{twAHn+x?hnGEcV1t%9eO|Cz;30uK`$)&5{?rX<-@i^zk~H_IQ+t* zcU~X19r_i+j0bP9|H3WlZ9DXnha3HD_Fq`^(NV?@M)++z^l4`sdr#I2i{5#Ch#mEJ zl!pc`52(L`8<|mm2k&A={ThIt9Xyj6>0_7w*Qk7ZO;YOZcp3y@^F_pX zzEnG&?`53+?~#Kjf5MXfwRGRa0!IGXcI2PDf8%`|dSTkt^#gvL4e4z=^zy!r_jl-p zMPGp(TJR74+ji*dlC@dkJm`gKSNk809a``YdfN{D))Z}4d9)9`FzssnC`b9V?a;?f zH2SV?qZbxEnu)OBAN;rN(C?k3&FUxm2mgg}J{4J2f*o4$4|>}U{k0cpvwD^OK`)H+ zp^rD_klwaKzl`;RX&-uF(MuTu!+){k{SoDH7MDl7KZ3KF@%{+TXU6*@xR@F5kKht! zyg!0VneqMzE@#I3BX~J8-XFmg%y@qUS2E-M5nRoT_ebzrX1qUwH!vf9UTk>z$#MPs zdQkZFbO&*y`dSmG=Qu78!f+b-F{+*M0F3ms?Z^*luR{A3@52Dma2+$=zrpp)c>e}B zFys9j+{ldgZ}2W=ynln6nDPD%R{TDO_iu0vGs0)*=Y+xG`M24@^~90xk5$viAzHGd zO?n8!edK339mgW+VcU_Oqh=cW8LSr;eGlx=B6`~necC0){u%3qMW5u*+ji(nXgC0t zI?hjF(bI4gYp^AjP@UJ z0&^{Me`d7*z@Eg6_8;&FX0-o+$1tP)2RxA(?LXi&X0-o+XELMx2Rw@z?LXjbX0-o+ z^O+G}`+B;S+TkVcNP^B(54HJt;z-w1HK*rm?M-uE)fNl*XLraTEtdiLmS?BG<|*mmScb&;{(*vaUHMZe0JL*>!7LqC14IpIof9}^b+ zUWeYcL!UO!oRHbg_%AH_8i(GtLmyvkPUy>eVbSM1^tK)PQr1_nUbrQ_ZHK;z^>0x+ z<9Zer{WgdHwjKJd^UVo&buoHj(QkCU*z};i$32Ge%lUx(~ZW1jH9^xF^k@r ze{w$T(0gw(`c#oTQ9G~Kvftp~GU7<} zlX^~{m;0Ld2}}H%*iCRMZEU+v-=*e+)vT{$7QKW?^tK)Pp|=?QJFFKLeT~C^+YY^q z$HDjuV|-3Osw2zz9P|$(z3u*Cu-!ilmi}S5Z>Rt8v%=GVwSzYjM@s*_ z9RH}}O!$Q*{Pm9b+jhi%>#gR5U7UWxqIW*uY&-PvWk$b;^}?brcldAHp|529-&rp# zdM;j}`@7iDzmD{@``5vC|2kOu*WsR>-oX*!>HVdHyHLA%r1Vbujfv2#?k2s3CA^*8 z#tu%UjcrGKN8N7hU0E+IdS`#7*fIXAUkqtWaOVEkZw-Z4t`*qFjtu8C;z;qSx)WzB z8enSTBP`*GHs(FK5KOHX7JV^xXu-c!>wGZuGM*6Q3*o;o>?hO8#XF?8?apghf9FJG9^*^tK)P#s{=n6?6QBaX!M|h#gw+ z4|>}UeceOatcLeC@fXJV&__G22ip$))|E!Tgu^c^`U+zX`ET2yFZ`W3VIAv*MPKRA z+ji*t{@$FhhT|_R`c)3SZHImX>xZ#kSoF^8*|tNU^r-Ow<55w5ghgNF@ZYvWzm4@@ za{3F4zBbx808X{e2SZ=U9u%-%71{jods+Vi>xD({jK6J%Uair=Q=j%Q z_|U9jI@BEx9!mHWxb!% zUs&|3w7OX@c8s?}{XoXs;e5M1j2IQ3|4%u13vs0KQ1OEC@Nq8B!V(_m^=#V_pULZt zemCodMUQq>EMTOE*fCxZ@fl-{7X-I7UU1>B!sEBs!3m=y$FKM=+Iuzn7*k$^C47Fz z{oc0Q>H876?=?>8?TmB7`D)@w>AU?UBkIicU19NmI=cx*`q*})?}&}&gi}~AEc!Ty z-nK(Ok@YiJFD!cJ`+{wUKA-iYSuZU5?GFEKyPbZ0#)PNWoeqAOI8ypef7y5tVE={1 zf3Goz%9m|N_}8-jA=V3vp1joyIMq5I41E*pm#|(K_LFJdMJpCC^tK)P#!cF+o?*SP z=u5Ff3;scG+wJsA9~+*2n;rZ}c7(r}^;fW7 zxFx-9hrWdM^?dyax1_i2(3i4)IqQX6(%W|E%US;bhhJFqGj;5n<8RyT^j|nGJpG#- z+&(#S`Zu!w@3a5H;(x8<{$tw_e))Y5e(!U##iB3A4lVeX>aatfvPGNKoW4dcjPp@` zhZ=K8Z`+|SW_@4Q3yXeKv~d6o|82XSevec7$h8W^@hRavfH+e5UGs)Vp3?$`|F#`^`F$6D?*;#b zaX!+|sTccIWJ{n`mbtW(AVB|ba`0^qzR$srI2hH)1X^WTPdf<$^dH$`Gj#`qF=b~M zpXT7<4yNK8c7B?JuW)dpqy4gngI#pK7aLv>Uiyu=ivJ&T5aHE)qG5zrb3elfuV(tz z6BdM5Gct`9gjX{L($RwOY94MF;nh6SF!D?D&$L4e@=LSZ*pXkF+Z#rHY3^tk`K7t5 zVdR(QqYWd!H1{%${L*}k;a9=5E8_uwVwUlRhz9I3-Y<_?#y?gt%XmtRAI14HJ{032 z!7|AU0 zSbkrC-(!H~_ZAbFdn6e5F&-UuKR>WAJ{&CLzcF4LEaR~;{u?agyD=UeEaS~Fz8);& z=h5E_mi}CfKLE@40`!N1rN0yXD`4qAL4Pk;`g76$1(yCT^v8pxza9O%VCla_e=S)0 zW6}Q(mi~40M}eik3H?i8=|4h$8Cd$m(0>G${vq^-fu+9-;}gK~#U_0*-T^G*88H3; zEaMB%p9+@#QuObGrT-rN?O^FoNB=!o`sdM~3YPv-^bdoj{}=teVCm0A|07uX7tx;y zmi|hNF9FN=kyXsnABz50*rk6J?bl#wpGJE)SlYYM9s!p22DER0SFASagZF)~yw9Wk z04(hXXpaC(djr~6z@u1?_kFOu&!hbmEbXIc&jw3-HQJ}Z(*A_@X0Ws;qx}&q?Tctn z1WS7%+E>BSev0-^t>^Ynw4Z5vm5CqPqrlSMg!VbGw9lcv5G?J1XnzGu`zqQa!P4G{ z_Gz%RPoupZEbZxNKLbnq7}_7e(!PN91hBLh;Qbve@8@_Q084!z?-yWsf57_=Sl(yw z{-X6YCcbzd084!z^>eV)zfs@S=Wj9Rqy7w*`Z4N*V5#q+KM*Yaedu2VOaCGI`@qtl zhyEk5^besw3oQLr=$`^h{}cMlz|tRv_5`rB7odFrEbsq#UkA(kINtxk@_vu^X|TL6 znUJ@+7heUm-8$AOi zme4_3P+tO%VMhONH@Zisar+Q{?+Cs8o)P1_z%qU-hnUbfM()jXaH)guaPUJ8e#*fc z9sH()-*<3>ga6^+FCF}ygMV^xhvUQJ7w6!32cO{J(;S@S;9og7*}>;Kc$$Mf4$gJ( z0tesh;5!}sfP)`%aJ7SG4&LhE_Z|F+gBu4fxelJ_;4}wc>|l?B{SGd6@J$XbbMP_;S2*|)2S4fHXB@oV!J8fYhJ!zHaHE62 za_|ohjy@qge~zL0cVAkMqxFAiO`!DzT2G|)BwA0V^%PqB(RwPa{b@ao*3)T4U4Hshppp!IB8N78x@t!P*I6|JLbJ(t!ow2q~9 z9IeT;rqDW`)(NzpN9#mdC((L7tryTbnbs+^rqVi<)-+l#q&1z^X|ztK^&(m?rga9b zGikky*2`(Vg4Qc(MH(DStDO$V6aW8^7MD;wFQwI)CJD-;zo(=>oSJ0AARFjgtIekk zv0};`_I)67h#`52881XKfKd`b2Kq9fESl6f$_`S@>JQy~14!L4#7?0dScE)rpC$>F7TupE$7q$)s#j0 zGr9Ux>}?GKIbwg@{V<#feHFf~ZrER0Z>t;no8+O-&bQf>T>7qjo5egONgH!je_6h* ze&^#e_4sQ3;W%Kvw|_V;(08u!UGT#Sh5p9>;RM3W4$x*7!lw~vt21V1mo^rK;C|B- zv_2T}$$yzkpzSX37yI>eB4KkF9IlgXEGiN+6D~N?Jgq|8gA{Rsg|<6Kw^uVUL))Ej zOz6-mha%11&?*Ng{NWQlv`USqw0Uy2?p9!?k7$)w%`;6LitIqsM{qf}W~Mk?7Y{J6 zMN3Xc{8+wBOkm9`LGyUEn6!uNhME83aD&3rjkR#EJJi5Xfr~W9#^DBzXXt2?Ghq{V zw8;gT>!Vd-kD23*8d_VEq@K2i)h(%eyAo?oQ(!u_XVX{{TP=V@u3E12;vJl$y?m)1Jb_7s$f^jhtwHG53! zLVKVoV_N5uJ-tjz$^*=SB6G$Z(sg|Z?odHCldH&#HEjq}>Y>eN(~`^foG#5%$b?N? zqo=z$486>ga~MwW1UrY}f}WYDb$ZI3LuszJQ_z}s2hE3$se2B^`vbbrJVj6|o!WoO zAWvvc7ffDBQ@ZFGdU(bi`;ckfG3UtaJsz6;MViNGS5&l1X-4ytdTB-+v>x;9FxHGY zJd5Z41~W-d1Vxsf)hCic&xOh^HBZA6?nCqOaCfZZk)5H4=wMSH9?fjiJUdSqU964v zd_FMmH=~ayKYW%RYepZZiRbt6gPF4iRJfGGn=_KXy&TStAjM#m@hQt*8Ue+aP|nAk~q>;sV#agtxQkzzPVyGx9|c$ z-6Z=r5I4Vb56lesy#r}*M%axsbkq?P?R87WP*_4Ij89Kbo$8q~HZ5&Z>Uq;>4rr5l zW;$g(jWkU*Iz7dcY{F!P2PlArWTn=ENzpy&$Eh0eb6+(hvrtbI`M*fR$0|N@2X|XqPPUJ4`lPv zyL}{AX<9a(-O}eZb2HLBjZ?E|-seL;dom`Lj&!zHx9^#GA5E4;I%_uOK{j34Et*4J z)K-e6n3Ro_=WL2lP`E)RQl?O8U|m?&qBKN0uDf?58N8usgCqHJ(5$y&7*>8lX%p7( zv|phJvzlTNmi1xwgJyl$**c>p#gc}>n*c^`KA0dha=JZetixt=|I^OW=-e{q^8 zc~aWM@#&uA=`(QyZNXwXDNV{bMBED>(MC6BPd>GJVV-HeWjNGSG7CL4C}9p&qar#n zTTe|K#LT&oPw?gG@n%jh^eo?#sL0W6K1`_%7nv!~#0L)>?#a~8 zA%Adwt_P98;X#8(*oTwTFG-s|)sxFLOcNhI7?y&m=S@l-@0mT{lS4NRj z5|PFRWEAfxnO&gJ~lOlyrD!6xtX4hDmb1p0X$Z9ba>;9LfNonp|4^L@6C@IYtF{RlSmePDIl+t0>zMa-#hCCl0M>GMb&Ydt*c#4OaT8#U_ z&I+aY!A=aNc-V;%b1_2D_fL29>3K5!84DIDkJ+bbr>U+;bDPRxv#$?_&KOJ;G0aFy z${<}ldY@XEihNV)!A%zry$`bqZ@T6r{h}*VWb2pQ%s?JxKoMrT)~qk(jKU007B2ib zG!g3@b0vCmeTCTpeT)Q{h}Z-<5&iVR#9&5lZcgGnnn*lIGq3s*%{%RxW*Sx@%A4Fk z-kJFY0Zi-~el#&ykeP_5K%%LGYF42N@W4#1z|i(k(G3kSN7EaRCitNmQYlKr+|d0tLTx3LQeQBZiuiv-#cW z-$=kP(EGRSx& zV|b8%k%W|{v_xfwtW27EL>9qNFq=d93n!PUyP`gOWG9#%&R;mmvS_p#y@5;3{>VdKJP_T`pjSy9_4c6C1NPh2MF_#m6VRbIsa8jx?k9T$)w$H zz{nAtlo7&u1SDN(#LJPNa#m3P^9hIHT=LqH92yaK2JM*=c1$ z_)8^|jJi8wQ^^h>B~obl@uSWglxjDQ4#+pOdeAZY3TC^3%1s|aanqktd9==?6-z(m zzOpXCt?0{!hqAojm$Rr1!rVg#4jPz5f8P0YL^THbU-#i9eRhw?tbgX7=!N$N&RqVl zr@z}*d2HGG9g}Jne02NUWjUvv;<_pA$vN(4UmRFef7@~I^oV(<_NLg!?(hxiKI!Jv zhH*PUU%9xUv#^CYSh?eBl^aVzJA)W7w*67iEnOOe9gYwr(d~uZ1D9H z&-$_78C&lC_@8}ZPb`j$>ilzHO5^HRSDtbGuiu-q>GkTtD<|CbM9QV}cYd^U@C{9O z`Tpma$6wkx`sGi1-}uV5m!_Z5Valfed%HZiXG6zRFU~3ZVai~i@9xJN z#=QM=N|Ac#x}w#`cf4!Mcojj(RXxxYto57KV8)$ z`s$%~zrS&Fx3@=EHsm)BZn)-)4>s=^d&|nY;D<{`eSYhVm98KDR`u+cYVFpK@89#r zx9LxualyGOXFt&M#uG!{K%v}{d*DO8# z(Y{}vIcrMYfX^qLcgxttQ+C|j{|`4*t)3a!^m*2X=y%_E<7)TRneFynyXM*ji{JnA ztrM3Hc=T}X7eASG!XK-W*Y>#W`TFx7__}Ccl_zi6XJ38x^-W)n+I!>W+Yi_MgXu3! zsoMGMKfjngHP&_a37b4`rq)z{@mgBtw#(cQ3sbF;0QCUYXd4E=I?wt#F zx}U#n@5tB2-+SQ;JNm46@vGl``1#~hA0M^2bnN=kYkIA^E_l_K%kDh?AD`^|bo596 z>iXp3qFp&%e)H_^Ge5|RS$OG#+y3$KBNC`*bOc-=m}KPI zHbzGsT)q(h$EmV{lD+2y4|;%&M5eO|Ns2YW9RAI zTc=K)I(6!tQ>T`Ddscz(;!Y_k4(rp|aiN2$jiVJLUuV>gK1AO+9QlrcjzjUkuOmxv zQ=>Wi-*~+WW?dZ_D$P2&@U~c`6R%abDY$jDmy`N-=yC~HvqizJtG!%TWG%Q>wROSG z@z>%sRc`uWny%)v6Ot@fe1pO-4#i)0sHDTXYBCP;$nQWBNOH|mU`SAXh|5p$r-5?b zmyb|{6R+oMMC)pmyA0*X=ghI59d)w3_59$Bdv`LF!?Al0Pn()0CU)D#_9+7tHLtQ#M2KREjxWk!s*f_`Fw$#3v zP^0`yAiB)kQqsmb^1I_Ut^W~O88>A*dO$>-XG7mkJls7lCC8oF$#H14qp|y(jPChs zGBUCb%b1vYqN8))Azg#$CrgJV_)V zzv21_zK`nk-}UuzTo>W{giiDFr0ssOzJFR@|B34oe4oYlIeeGlyByyY_^!lvExzmU zwLa@{^MaMY^+kODjqgkNmM-|yaYJgC+;i7cQ*QqK`~5b4d(qzFJCC}5=2h2!b9*Zy#Ep@nq3$XB~Faw?AyX4C*{k=G$-};!`tD;d-gH{pX@|Y^`oyiL z-SX!69bcS#6~3={X8*Ya*ZZ#Z7{9}H&cI0-yPvzDb^`9t*ta4*Q2p@wQru6(*Ku)E z;qMzixS($FTSt~5Jp+Db>6j|Zz(u>?uxqK?{@T?>&$XB{`>0ThYeoZd(P&W+4I+4 zebKPq@Apr+>G`7{c=8#?C;7*1{if?CYmy>#~pH}1TyZ(i{6i%-4r(5kI()ua?Z_0Y*>mt`;f z>cO5r_RF~F$9)%ma>-j$K3n|mN?+%D8@|4~W#2^&-;JAoPUc0$tG9oB;?4=4!~A3S zjA;7jiN>8PM!Y=s+-`<j-q>PmjPIkwDDX&tsZZW2B0=2)Zyshf1h zZ1IzzQ>pMjQWpo~Skh@@Ad63prr%Z9J7Kkgf1j$Z5#$p;lR+##R_wd+5Cz{cPr)xz z7snj&KTxiJDZu+0UZLr1ELDJYy52`cKf5_LzM|mgYy6i)KLHOZK(>a@7rW@@m~SY+ z!x}zI^S_0`A3lB5#gV7$ZTp=990q^Kk6Zj9=sc#;g z{#D@zEImOv(l4%1@Z&X|E7BFbu~oq>eonIVr&htO_D<4r%h7TJ8}jLYJmuWoF~3N6 z2+H#_(t;Pp!}}kn>Ye|(!aGG>990&6K|KDCTF+}a5yMBTwc|GyzB?ZOB_yT4w7o+t z@-Z#`H>tFxw_c0R1@Ux#5I+pRF-PIAR2Ro@AP}KnuK-tSc^+@k85B?FcNUz6%#W7O zAWQ!>;VV_H;|}rX-5fg>D!_vpex2^$9Sn)f4++y1jqZ@>z6~LafyIcWS*Yeo+BnR`My=^c|YM zrEec-`fUmNHd)(CW`ezR*Y!^LgQ9zzrhhY3U-Uer05dhbSj%D2BMOkO#WYjPKYx{i z|BuH15`rb2%xMbnFWufs-7cA!Bg?1h)MN=Yx?~({$sjbTV7Cr zAsXIQ<4<@@0s3qB-DtPyNtpuZWkAO#n$Jw#Ay)rhs>jnU&nf&9wA|Qy(wT6%g5Reu zj&7QcW3B>Ne($fkUox&yaBE5UbuG92XBGT&O~1aomb2z(wWfca9er9U;5tjZp zD}dE6{dE6kzNg?bv_L;TOv~YJ1)rknRBApaG$?>JKgUbZBepl^Y6Z9aMxE}j+5~+V zt?BQWqwpWn^k?HrI&GRkOKz8FyU5Y}_%;4q=wT1XQH~wAE5V?d^0`{$TZ?SEn;ij- zZzkYZ!EdqNE!s{iH65Cd!1eg}p0<~C7(U^dT>QePSX~^SYx!qvQ~)hbM?}lN@lOi= zfX4r;uDAI23NT&6U$peW>*!w@zs}H#gZlVc zj{%PS%c3RH8tm9(;rENj&k(s`9PAqpUvJ@`7!U8S`#mF}-?v%#z2othS@2`x;iqXo z+4e_a+hOVL;kvyUy1iC^eWdN8R^wl)iaAm({$26(*Yr|yD1KQX&d}K-bpO`s{(Vl< z*<$h8Bc9GbG@rEz{=A3o#~e-I0Zr$TV^zKR33{?f^FRM~g=pDzIuwch)v7!EOHJn; zEr(34SEDuj39WC%EecSi;f|7$NmFa8OX>rqb%Bx+M@iwRqLOldoqtkQeZXHgs%XfR znri>3(uq_2Dld6XN!j$$l8UP8(kWHf`W@B&X$8~E{I!9qn(8r*YwD|O>#C{)6^^N; zQ>N6E#bq-wqML!(x z)|Qr4`b#QH%WGQS}^aL0ib9a2*r z@J|mMaJd62&`?`m8t@-bk@A|0Df$B{Q#Qq4dSDWm|K zeSOKbQ>rGGT!DW>G^8e`6&uNCBk2$W>BGrncWhur1D_2q~2cuqzghu$MmckoVPda!xAGn&vG4bCO zwYoG=b&VgEGF)y7FO_jjB^+Ag({wqqWIvhk${r-I9Z zItRU0)8SCtv0N2#q200j=|F<2Y;8?_)%23wiqi6Hs_Oj?mDDM)c#u@@E4^1`XLQyg-4uw93KDMTAGDeq? z{=i7ijt)f5b^dC^+M{Z69O~|X3Qwu3o_t`;L-a*{F+5+@)T%%Z^|NjYZx2wr%4%w7 zunR_4Pi&~Dz;v?wz!;H%MLDsm`r@hnsY5FLWs?u6^)?ciz!7;H@4Y zL;-N_)eOfbA~teN;Zh13;tw#lrZ#}vj%Oy-{y<4XHO6gE4a3rrLa?N?x~#GWV+SL_ zjvAc2Q=07~t0&1hLX` zcU@g6tbsOvU~R(EW7QO{jpjJ2YRUo~PM(stLp+DB`+yXHjBBdeh>9Y(D`gTNK_feyXu&nSMW+Q2!m*2lG=u)gE15jz8QWoP%-Yh2tM z!00}tvY~piAEKCA@0i#yWir+gIt-sXpftu`S5+~?PN)Ob>!AGpoY3*hpy3!YV#wM3 zhm?#OSdwSEKihVnW4r&Y?S6pmKG$}Cj_tm`y0?+%t)|XfO`NxyHg7d)-fGIc)r5K5 z-%(pt>qp@3=ei7KE_*T+4VaoFDubFygeGTu* za6R)J-YKhMGWX1!Gjkk6eT9REl=MfYb$9mJ(cAuK=0)$%9^lBjbVT9M!r^COZ?qGh zq_Y0_RMq}}ov=sV8PCk^pF;?b1tibeA8N0QG~-LAIDY-JOF(U0r$G8^u!oFn60(@#l$Cr78=ih%X}A-2iovuKREF44~= zTJTy8Z%cq%X)^)-r+tdfo&@;A8lDzZ=LRhLk7; z;o}nE>ovSG0lrbgrzgPQ)bKe8@b@)*UIP3R4PTG|Z`bf83Gkg7zCHo|orZ5tfTu*1 zoXrIIAsW6X0e-lKr`-}S|DGD&D*=9-hG!?hPuB2(3Gh=j+?xQ;)$nl%@PQg$nE<~? z!>1>}FV^rm3Gm?>J}&`2O2ZcZG&5}2eX&h;5}^kZ8rE=8+^SDj-$=d z&qf;@M?0gR%{DlW07pMtY;YVcjDAcT97h+UpB*;%(NUs))2EWw?ueHHXw!x>{;QwQTH`?Gg+Te3+ z@P#&bs|}uIgU_?U``Y01ZSYfU@C7#bZ*1^IHu%Ff_!1jD+Xip5!TZ_Z>uvB;ZSajY z_-QuyW*hu;8+?lmeufQh+ThlibY}0c!Oyhe@3Fx-|BHU~xp3-dP85jOVc6hnZ14;l zJl6*AWrJJK7BD;029KV}1w7jZKiftp#|AIA!3Wyl18neo8$8bj_uAm+*xm#_#zwJZG$hd!Kc~aZ8o?)9$0UK53%8Iw81?#_+}gYavOY$4PIb_ z|9bnY1b&skuM+rG0>4V&R|)+8M*`nCPx#*3v@OjW>@wq`6o-DIi=d>D!s+5}(l;{ADStRq($_N0seU*^(sfL8${uz|`f8>*RS)mj2SDG6Omiw8-XiHMnC28b zyiw94nC28a+$QNunC8?vyhzf6ndX!_JYUl1GR>)UxK+~qndTHa+$iZ&nda0vTr265 znC6r@JVDaOGR>)PxLDFhFwH4%IA79-G0iD$I7iZ*nC4VAoGIz=FGQMC*l>oVcQVbX zYuF*_FPP?(HN4|Tw*MohGnw8Z>35jsR5ZL%(wmq*iRm^;zrgg#OfQo3DyFlTo-gU8 zOmm7DZk69g@D9X-?6?JARP4B!a0)e#5AWi;Y>+?k2f<^a!M1e@`TyhNW*em_dbUxEtB>fK4oSK9;N_rF1oO*=YB>e)@oN|O0NqQC2oN9#UOL{5O zoJxdSCH*AR7c<=`>4i)WWx7_<4>Ij#dV-|yVY-m%VoBf5^d(H^OZsM}hcTTa=^L5$ zF`X&tYnkR$IGiErI;Mv+?U3} zWO|XL2Qxj2>G_gAm+8?=w@SJ{(_@%!l=P`gk7c@6(kC%Jj_C=KK9*@t(Za<@k6c1m z?0j(cR_;{>dcd!T@yZNuDDO!Ft58PHm8&bxboOZm643e}u5M_w72PkI$(mYPXp;7edd4^hZ%<3G0P$9wOz&oFJ(N+#+46G zL#F1(J=Q&@(0J25+C8e!`1fe9@#9DgXW8F4Yf_v&JXsjm+!#EQfj5 zcaaDv&6$ftQ(KC!Y>UtM+@!q##T)WvrTL7{yvDjhW1V?5a9NelfULGhS_fjRD>T*$ zahCUy4P&L(cn5U@Be%^RY2AX1LSqg4V5-lEm=Eu_C#%@m&zCj9*>6-TwIi(DbKb?C8POn`ERH{{961SCC2LwcFRDa+Yw$iZsOui~c= zVmC%*<@_J<8#*o~RC3vFy^YdF$pn4)|(m!ga+uM4z~idvjQ zLxUD)DiffB`;3nw#kH|?q;(18T9ErCXn~?z>|N7&_Rbg`v4+j69E@vw?k1FpQX)iA zigHEyyT>f&t4!RZeW5d&K}@yI2Ya8rZBJ*I@^ib<7^BM%zZH+?Y(5X6D=fo2`CGt4 zt~!(vTi$MNgmuRdz71ozsjyB}SSM+$lNFZF!dj%TQWe%w8tX@d zJZR$y78ZA>QR4^aNE;8+SZ^t;9fZ=MmMg3!3TxLL(r4xq3Tu^xg}8&XZdX`aHC9kz z-C<$%QdmWlC$6EQZGsZY`<-f# zs43OU*^D7T%Bwy{zg`KHI3@E28>z1_a_WzEzR_{gY~M*E`OIg8m-Agkg+?2Q9Q{XX z(eB(f^Q!+qz~m&9rzH@r+_DjBGKP`3N|k2E*9mtI{nqx{vZ>+vmJJO#7*X^2Dy+av z+?^?Rfrx7+Zq#t%Y`z)HW7trJ+lpZktbC?~7-OXmdCc9Uq>zM@g4|EWg9t|+uz~Si z(YXe6+Mf}>r`qUGB)nEN-sOTs+UHlGevw5VBx%4=fAME#=mBw4lmWUAC5rGMi2)oO z{~-h8C650;z;nGJ<3HH(1p`?Ve8KCpDm_MOmIuRBy3g1ZY2};DF&1OMHdYq~Q?nE# zG;&ANN86n(Qy{yhJsE+q%z|Y8BR%&y81pJemZdUW_uWp8zXBO>Coq*z>V=^>x6L*( zW3W;om$&6hXEUtJ!6Fp!E>dW115=`Z?@|3Z`dRs8ROS1ES7A_WX@g2Rn-M5E+)Y2G z1bb%rg070QQygyR9nWKQcRuqabm8o#y_wGD6>QAjjE25G14VsC&nyXrBCex=?+YR- z5}G~`fu;D2x5*IdE-=DA<9NWfyCa)&H+Wfzu}g_d*x@Vjg~1JsYJ#Ik99$CI@~*S_ zL#P<2r(=xhmMZ~^>y!SlOGw3Qn0B$Wk_yBef;36O60@`US@FRwOLYY>i5#K~y zPm+@Kg|p?KQKY>05oE+un5HvxH-$&yR>cbz*9f+hFiN< zA$l(Cj`HOmyBy~?G3*r@FS|z+ON@5OIW2(Eu3A2)7eUBvYl+!GOY=WvHSeIkQA zU@ue(bJxCN1Efju(^p7LkfkzQn?(gA5U}ElSX6*m0s7a=lc0Z^=4GJ-nj1CEPtZ~A z3u6J2JLN;qpzF!eE0B4(O1u#KpS0r;@U98+1^RX-oz2(j7DPvuhd?^&*OuJ~3N>GAg19NHK|sF#5DVqU>2}ycIQSf^%#HCGdPrWw_YG;iD+pSZBh%yvAGJ zmQ8`QNZ%14Qqj6PE<4{LmeaH?(;K{?A8SEaobEL~Ge>RX46p%k&fYM4%sYiO2OBxP zLyI%VGbG4YYy}6#*YOVGx`s;EgiUJ#Q`z0}yhZF9w70bh~N7vXpZnO2~Cq zEbApJ>j9OO8OwT-Sw@%Dr?Z!!;IkA)w-^S85MbPGVT@N8--_&{=4ypeV_}@9Fg}Z6 z_!LHgg>j6+*c8J!Rbd=wVeA%H=vo%TNK+VJpQbr|Lt#7|!}tu&UmCGgvM!j1tcc5G z0X6b9Kfnz7%NGHJsU5YPOXBQ0*AoJ9w#>8u^H@%Wt3a?K*1j_UZZ*#{b-cRO7Ozgi zl7Na=A4BgHikryGMNiV)tT8%}DH#lVbi5i%|75(%C=H7HuAPG5OQUJRQqDPdx^#8$?Q9`UD@}N&!o5i+kE=CAaucSdJXiaSq)_pz5Nts!CCbCp8^na4HMO& zT+Mc+{2sjDQ7|7-2K3t)#s-CPtc7ut!ss2t_?yDmk*#U?6-Gu3qeWq?wJ?S#jJ;R` zRh?~?DUACp48A{7^<(!fg2f>atiQoTs4p*()*YCp%Q~QB8eNvNv>HS=&}2e+OOoPM z#PH@SdUm|8)jY>*bU7!6H#;ezf*8W|qzK2y5Nd4*u`0g#O_b$K8v@ogb(>y@Aqd4=acdGCqsQSAaTM2UVDkmWTF&Ya>R?v4 zq85;-zT-x|i_3$N1VK?TNVFJtf6qtR9DX z#}zH_He^_n4_I>}5NTAV_v1*UY3~n#vZlTJ8iwnDes2rATF*#w+c0kn$aGQ7`EofY z6-?W#@5=+J#u^MXs7OV_b39o`D>5=+YH5=hwzK&Y_LG=^3Z-?f zv-w5oKrBXZ+T1I5SD`Vqm)j`G#45ix<%P!UvyOHIeOVd#&O6r@8b`1EtdrZgG%eyv z2M<*IHWY8#lj3Z?7dME5oA&I*QV%ME;BRSEf{!kuOpI|*7#y1u?O4SUX8`SkEjv&o z{JQU=b1PX0QeqaJR}^C-Bv^n7WP1|#A&B=ZEm`}3pzDg0*}q=tB5@69CPL}$=&(u1 z>WoIaPeW5%D`P0gb_Xv-oZvcNN;AuST1lqcc&~kqo|g>Hu*dA);NWa~IyzUPDMegQ zpCr*zp_bScvtkmH;m~FV3C)f7Skd)9?GfQlls(v|+p5ZQQDxJfeGLV=&d(5+(LACe zK0hYYpR7|nFx(RSO!&4BxF4fDXtaLYd}S+!wCH^0p$`-1D>pV#L;pADE5#qFc*cDH zC*zqe=8+|I?XpPyOO$)h3z=JBqw&I!h zGAV2rsE%hYjA7__=1B`f$1_hAa6Kt_c_w_y{&@TPMDdmb0rc#6v0&rn7~bNfgx-%K zEKG{IPN07hhiYaV7AE69tGw*tb zvocvZ{q#Dn@%|9tGJfz{c64kNHP9y(q9l!2NI!mlbD;6aW8^!Uf?s30HF$m6fk9JjRu ztEn|db99iT$|y5OY?G?!*n*O@KpA1yF$A3x815}gN2U0|RIf1vi^(5iO*z$L)TfEw zA|&w|M;b#iBd(ojVN+X|ragx^Z^9-Uwf0Z6lCsnry(t^5a~_D4vD{~qywxA58Xeq) z9c~}|wA__o;!H{}A%L2tq56yij2J{fFFwAQ2>w)j5oCj+vyY&w7CD}juTL9h3oe4g z)DaDaA6_ZR*n(|8ZAj*RO~}_?G0!zEo_}X^H!LZ7gCiwy25Z(()GikD6W+i~6Ze-JO#p-Owa^h`rSod+8)mA;WAUXr~-oC3;*>tyVPSqZG$ z_%YJjn2LJQ_P$g<#9nasdywsxMPAWInfR@)XjT3KDJgSwP;*J8VISmZvy@gG39?XCz(xny= zdtH~M+&#y~7wo(ENbXHx;Td?YIK0AjsaZqw_*O{De1Iz~7K6H3E&bS5;C3PdA=gbBm)g1??vDflH(%jS)VOlq zA(8H@8n;p5UZ`<N-Ff<3cs&nLqzvT&(6A=f62 zdv`M26D?dUR{-~MjXO0N?zaR&4vvfr;?C8$?qs;!MwCW`TvHVuN!ch1)2& z4{O|U$#Az2NTZl9xU)6x>B(@PvT)l3cf7`RB*UF!;VQ#CSL42}PjRv{EO*>;m4&Me z{wR&RAQ^6sg{vINPK;`l+4aeAQ!QNOpkCIv#mR6t69_p#WK`IN8aFE$?qeD^U*Y~i z^K)d4$Hvus&j`=PB5;VG8bOd7*l`k_?!AIt}<8-b>I7Ya_6d-nq<)D;4BM|R!=v~C~Ao~n}=#9eO!4}YiO?ea z+#!YWZSCuWd{R21qymOT-2oias4h?3U`fyGy|cJM#o~X6S-r z>fy<*cnP`}J)-U&h3xw$WBs3pQq`Uz8;lWb0GF>VNx0b4=z+K?MEtCrGjeOjA6+jK z`WGdo|C>1a&n2NBouXh8jkbX(;yv)Y{pw`=9vnTK?{llQa$`4F%g33IK)i*0$miCwoCydgglyRP+W}{>! z+8l#EMj4z5B7IAP8ZN}i3J=1}X=owNI}Gs!N8_-Nk7u0FI1grLnMfTEPG9OyhVx#= z#T~hQ3uv;lX0IXj=;21LmzdXWkg9??=4GJT{%+#@o=*>8%LPXicImm#>}~Tv1yMU@ zlWYeVN)nfjniusEAu+g*^C3$(o(f-6B%42}`5_XUyg#PWKq!iGTdcg9jGQ z0$H@~J(M%nMiIl=09*4_t_LNp?~l;51Fi2;^ablPKU)`x1aq49og#V_tN9;jgw5}T z^4_3xwt4x$d@79P-d!IcJI>-kN*ld;Bm&D4F<&C`8zqO?^^+p(V_68H|I|jJTwS0m z)qHiURP~`*0EWSNySL@t!13m9kh&6VjSuEv9sWB=0!%lfFR3;%I`vNE`-wQd$He%) zQZ**VcpS@^PZk*K);9oLs5;jP5UB&Az;<4 zhwSyjQ-U7iAg>vHghUm`-XbVCgo4sL^X#=Yee8-Rw?9RBQeT(bWQ^0eJa~vxYTU#p4;aO4u-CrcT3@oS zpFxchxRJst1Bsk7^l#J|92L3E1PqiVU#4lkj%LCIw)t>YvX>l%xe%-r4*JsB@;1;5 zgI#$zGZTk&tS5?|#~NNbPSYIa4R(K~GbjIDk?RfSMp`$bReXx5gq6=;Ljj=AZHsg< zf$`*)9N+Ik3{ zvf`+xj-?l=$I=+PxEuGRxUsQoj)5_`o7Sel_6u@1xQ#YBu<%80#JrEK#8B3>ClWZ4 z%k$P~Cc^{9vqx7&^&cvffKky3b97;m5Va5b2b@e#Vjp3OSKG%IRK5M|<4>ZHVjs&^vUwkxPuU7R zjqLsG1NrgxG2F6`FwoIuqWmxsWgkev`vI&*Mx~Dp=g=w&S!HxAwpj)JThtzoMEkTB zzW^;Bg~cPQ5^Z7aPePBQ_VQWOUWQDJSE@jqyy zarUwh1Zgij(JX3iC~vwz4$NNO)%Fsl4SV^^BxNu2qLjN%h;N#+`7b|hFO6~b!gzT_ z$xp}2-J}um@qI_}cg@eBw-?3_^NWifI-=L-XVClR0O-B)yJYQ+pU-@pgx=4Y&x|eP zQStZ@I3ZZTc}ocrdFE0*lgcncO^^fC2>P*yDED|ix_5J7Q6X{=?lz#kt_TwbDqb;e>rq=RjA?UOjA< zI2$Ow5VjV^j4ppbdVJh|DA=0~plBR0I>lqbhqb``5_03@O{t>4o@c4V{(KBT ze$cgd$8P^%-9jR(TdCeCmjP4PM@s9EAK5Y*0?fV=WyP&upOqGy9-ZRFL#Q}ifyHLm zqwH9|iJ~?Nq(UuWt#Ii}H~_@)>mH3sP|(!{mAH*>;q8NYhvFV)=7ip;b!JFNkLZ?p zsrBXPjx`AU`+G`-XB?~vg|b&H%-J5~&BuAQgrEdU z&oy-kRA#Rs)3%Tv=aa9;v$cFq@$2P!TWmPIz1Kh&nJdTyJFCe|xnQ3YEZ%f2I>a0c zcI53-e16kbvlad%RukD`rlLqN8|>~ypMohK9@r@lgXcXD?~K)zI<$k_8f!@tYpdZ? zP?xN?$s^6OtfxgkFh`3X$8;!VoR#(9kdN*Mc0Ly$oH9$({Qyx}{Xm&Jn-4=SHe=fN z@Nz*3n82LKyam!V|Ac=?_(FlK94u&LgEOyzbqAcPFEX~^QFg3Lh_rwN&gY?R@-*k1 zH_?q3L@20R;l9wSrEREX`6}F#_zDn0)yC&Yn>!anB56(Er3Mx>eb*!K7cW;fj91W> zUgn!XlyzS4CsS|{m8@tF6UCsIXz*NuRz1Mw4;U5RX!40YD*XZ=W@SV*fU<+m+mwLw zx{j!{zhtdHf;-ss`*@QNNTSuN&}tHoG7_#se@QEV*9&a7zgdCx!rhSE>8f#_tZef* zG~0veTsGQwI)pK2EPE+*Uset@Gq>$o{ALI$`0yL=qnKT0wY1F~hNE5u##>7rnaudW zV?^A8-gG;Mzv(U8WEO#1OWX7w?tW_vT6WEPr@;8W{aCig{Ns}lAr8)@2A&5HkJ7?k zuFc()yUSy|82F zc0FoAPnbV~zwk|%;2G-_*&k>JzO1YQV-MD53h|zsP+l!2#7d^sPf@1QvjCuT6@Tbq zDXm!{SO2AtrA+ubbcXQ?i)D#QHK~g6NGkbOmiJ1jSl%8mDNVFgiWIWi=a`p1K?&t} zpt>Fu$D)kAbXb*M$|iE^U1+=)z7IzhVvA=SNB^(hkAsB;c{PqN)TI@S$U6e|fB#I= zzUsj7v-&sf>lSb|?JEKDlbA;$ytw5GSb@{ire999*6a*@!K%k*^d0vdn!c(5@)%MM zb(x7WP5V0goI~2$U$#xhfM=a=lI2~Q%%p$NU_+a&QoRkZP7dh{kGa+nvDj zh03!qX?eCg5IOby9BaK$ds?ELL#d!!t1!l<=5AVd2ymz)mJ3sR$~-ykg9ASTdsoY` zc?tW$m>R_Foa;C#=sniO6WC3ACOdE4j2jqLDn^pJ_9K*Co0A`hLJn+-q&8$N#X2HN zeum|yPtmDda@zs|#@_)phX@{?k3>$9u?O{BRw^vXXD}b=n*2n$3|R)yyef}AIS_w4uCC_ z2y8@E;CQiaZ8c!9Jmt?Y{BikQ8B0;iCk@lQ5I`F8jb7+%c2T@5Y&EY0Kb~MF*PM${ z4L1)%u4;%>EMz6x*bh3chM0!u+8?*#L-ZPJ+W$>bs5W5WQ7dPA2>N2~{kxJ!BY6H$ zi{p!M4eUqB>s{pQ`Zv=2u=I=3oV3MeUB`hHF!2!1#8A*ZrC`3}Su6)RmS-tUQG-)~ zXx^aFzJv=h8$miaQ1^R1h^u~FDUD+%Zjg|Z&fNtmko?mqr}!Eoh0XI(%x8QDnJfk} zSZZHqW~;0dqZIE~6x$!edW8(zSP92#7Mi~Ate*FoRoL`B9@&|Iu4?+ele2~I|3DPw zUIO2bmTr|N*Tig0A850yn}LF6;q)uFp<*tk>HAb?%Wf7cdv)b^ecVehia69!-Ktfu z6-~^*i`Y)-w;n{j^SlNY5~|vI@ik7;>$Kpo!fo~Cz zadr$XQv1kIcgUwNPUeKGA^QWb)|FUol~n6&?hSJ0dyht|eihn*s$W1DZ?AeG4#R}M zvHFD)p=O5iE<`9T+wNg6I?r0qjuR`rh!X3!s0LzfeuR2a8LM-=s`IFXI?rL9vw#cx zk0IwM)7vjUy zL>klk^jqznSD3Q4<=t8DMQ3KV_qv&j7gA6I-SAw2e<3I^A4IE(>5+HYU^{Zt(T$Ayxp8R8{3QBV~den zSqy`Jo$YU0!&j_`-Y}fcT!4knKFXgir8c1G6wY}iK7J4-fDQNWY|eo-gioOT=w3zf zSjUFn`Wv;f5zQuvzo>A~EL8;B|HXKgfmbMROq4qN1Jv8+n+s7m*bAe{1CY7AafZ_q z6rSvJ%brzxWtV7)MV69TM zI#`3z8MG#YRhk+yVs?iFf(5noz$tG_WX9RJrnf|3>2tW#KL zR#vx~-4v?S-@d$+K^~^4;&oR*9nq0n&1t~FtYrvEVL~Z~zv(;5 zbOc{d)KkX|=Sh9ycN_Y9>9xOt?lquldB@H44|H4={{$=wH>2?!$s*<+G#_3Xp4r5( z3R6&gzWNP#!{i4&^qksCgml{%*|dZ4_9%&s<=tioDU^~-<4fe&yqTy0v!nE9+g?v` zEXBYM>Ytm_cayNJw458uWvs#6D^#wiRLhqQy&xVHV<`C!dq~A=hf^8MGybY#QZx1- z8JLu1s`j7d!}=aYSj0*6rhBx;-9Ge^ugaNA$d3 z+~OD8kAoW=mGH777)dre3hi(-eTUGh82UA9Ow)G=t-ci)T;fx>;T$L3hj(+P#R4mu z2qj?Ni~gkO9z|j9vZDFfSB^I}@&wFEvro##SNF(#=)2U0$raA+Z6ubVh%tZ?#U?R2 zZNBm}!z|Pk^FWa^=w4MQAyL-SS9Xv5_~CokWXel&_g&=O{e^es_lJ1mH=x z$iY*%DqnCyrCEaBg;r+6ML{Lo$3or-O=j?xq{(A6;qD}S%>IPCD#A`hSiOn?t@&_= z)f#u?b!mzmD>2`CU_aK@u;C%sx0fr{mYIw9$Cmp#EVpRxkoS8N5Twsk+8ymgGjK zs_#EI>>(Qj7*);dAR6=(hF8Wl#wuWjCxKBt-tEWVyYnsnP881$CKrGsP6R)WG5NG$ z*_eC?)mThEE2AEnoVk5JCT+$j{Ng;B>< zARqPeMUs|%;?I}>5%K}(ISZ9u@}50)nu_gxBDg~et1jB>uiGTyLYP+lOQjd*h<|J4#>dV7ynbJ)G; z6&y_l`Mw}8VJdv`9a=}J^6wH^t)kCjQ=12fTs)xCWMTov?1cOb3akgRF-gruiPS8< zYekDM%$L3IbFcH{i;#HW zJncbPA2ue?C}SN(8`Xl`4Ja6CO~(BoyrR@OSN8wPRtJ|gH-ZaHFK#j4f{dXzkY&Ru z*1G6xcT3#qY`GTX5qn;5J`u$VYz>V|DJ)yFu32cnZX)4A{1boDoV5{jEo}`K!C18M z{}8)<6SuJLe^|RY?qwum9)2GcpM#B-(FNeb+jM@Gqv1Hj^|Y(_xW1Kdv{aAXVsU`N zLdgDx4;s{~Y2T#4nb`fo-jmrC(n2JNsf4S16q!S}aD4vYb2tDCE~#b#}F^F5RV6kGM$x!t&chwAZ%1WRP*A4zT;ST*m)9Zt1gui4(e=1QV^ku9PRpNu=*Ac*d{ zOGUhY9w-tt(;*(w2xxk93RXR>3f)cr#(F*}?L!-)^E=KbE>P-$J`(IG_pL`w2P0+8bq*X5NCridn9@Rf25z za{P<-kD!_rzzUB79ESANIT$psWc0bj56g;RR!y%&%p=)C9U62dnLD5+%n57bQS*6y zCPF<+fC)MJS0<#4oYLF7V?6{_srkkLRA9Wb{5UuO#!RbF0ek9wtb{nxFY;boC9&1! z8+Wi2`WqUt0lmtH0PqSMy$9k&bcm+U|JY(NutJ?fp!-Q6l_cJm=ST2RhP;HP!1|Sx z+1s+wYgEuQr~q!%TLIj&zoUX)GqbSlJM&EFyU3YqS=5$$i3!(38wQ6GpsSYx$4BJq zz=nQrM9dZNki|QH1WoY}60*v|FFA=*9>f9ATpwy`zeC4wJ@F2ARLH^6yvrnCawcRU zdpON(2G#bbfdP{?hmlDY&#fJRk?*&Vt`8QWia1;9*}sFRCavxWB+b?`jsAI2c~{phF@*&zFo@5 zc*|kpD*D}zXxMlRCgd7BocgVYi08KIPTs{;ALud%b z+V-_7E&Ji%*LEHcweV09JTzWuq4wNE-Twn?fULm9A~-_zrYEn_<;ypDZi?lX(|uu% zD2sVqgGn!1-nAp=1z=x0VNs;07BC(oF589$jNG>N*VWtl-@u8a zn^A>+;|6D-+-s1TV_9*kw=4%;HE8DzCx!;67(-IMq14wrMrGQ%{PZ5E&(aj~=Uw?3 zrdsQ~O>5K4MW{=~%z?hIX38+bIPx=8)*F?94WIE1LKOsMg=J$>+aDxK@JMNXUOm|6 zq6qA454!kDK*xh?Y?rED)aROI&dl>vZ>U3oF~)<_Q(Gc4OT1;f@fsm4J}7_TZCZs0 zZf9Vqx9kPapz@7r4PW{~`Dxxz&mB#Xlt8aw|H4rEdL!5Q&$`qKODh%%)z#6s{V3+w+ z3R1o3Usj;`oEg0Z&zRu(HuWPh>iNdPvLC!FKksC|4wKOHn`ZnNfkP&}xf@z{c~*Dr z5$G8luo^p!O)FE}#(=wWH&qy;R(Ev|dOqKI|62E;e>?BT!RWS>{N^_sUSYBY?URsH z&Cb*^xWxq5NXZ|3Ly@uLCzhS_i3vHX*oB)Or`-y{0%)yPFVW=G`VLLg)<8M`WA2#? z{joudjBjn@k)prY9cNvr16M}# z+{gn3<}Hra7qu{2E{qJ?(Bm~4y4V^Pm=34)Qp42v%aeKo!GdZsm#|Rc!%Y}}HsG`y zO-JKf_JD=2mu(A4(OMow8C47UEUSePPfs(af=8J#&gfxo4UykZFk$8*LxYQiqtP}L zz7dkvJ*O7t7NM}Lb#VLl-Aj-uL3O`YOZ8=qG>%5h|8;+Z1wyV*aY660Ax4*@z$86c zf9ZB?`3O-sT=*YZn8yP+S|1jW8KcX4uc{JD=$-I0NGSa> z@4bbFul8^!NiS?SZ-V3y&7ICxH>p+!8@yaIK$Wg4%sr-H6|uQ045H<`f2bL{+CREs0^7&21MmR4!0JYNZB^p;(L7hK({R;b|T zc9|jqNYmG>u50Qse9BlB}N^_H0J3ax*}2I2xoN};((+q7mNT)7o;;MEmMkENM&z+HGVr-RY` zD@6Jnk?F}(U<&Ykz~rbZr^a?&EP?!=f|hnfUJKv;gS|Z1`)-}Z9_zPEPEMQh{4wN+ zu8^w7qiXFZrT?QsMnNs)0ie#fQ^{;eKTBqoS@c9c@?=;f;o{o%XNli*eb(ve$Ovrk zAil(X+)vCacB+7C@;Jkog@hadg0}mI4e2rcb>Uvo7eJrR<~PmKoc~P zv0SM4nWP-*`?ueS<(t1XICV;KwlLg;TEI-iPw^~ZCTCtbTm#H*&*1M#1+YLUi^oC@ z>uk`vh}jI0W3<~M8b+TU%KHRnBjcN89w?eBE6ZXZ)+4K&kBdi}jT^p$BBRLowEZkC zcLGxB@73Cw8w*t)SeLAa6$PiJ;nD3tHclxJr7+llD1`V7Fz6?!72s+qew?m1be%|| zemE;GI6Qo77TzSnY-Xdeeg0Vy7LYJozmV_t?P3Q1AaXaGaAn-^fjlgJKTs7{vY5!% zT*`QMvI4@}^N?@!;YUv4@X)f-8K^uCOYE?zJmTyXT)g0*)yq5vy%C*(XPC!=J&f%s zW{SXp!9Jv%yv9KDe6S^nCbS#AVnP}mN4Ok5s>s;aen&!!AqN-wTv?x4teWI(IfV1I zGi1JorPO#Kp)+t0SC=>eCBl0_JUSk@N6Exxc@Ee&9|x-|9GNgLV~sgt4mg8HI8xq+ zCqTFeiT78-$-`#Nvsy5z{L(g~hJLLZng&vsE)S(d9P5Va<(;9KOseT{p5(mgZ>Z6`G&7?yg%$x_dhLs*k5afw?@-SUVrN@6m}B+pdfM;R`it=g zrryh;elsKc$`thqd)s`Yn|Jni4hSu9F}!-F!<&LY@Vad8pp{dkUkqXPtQ6U9P(xA0l$Ht5e1G9^Ekw1n#=;SvGoV+JTs`rBrghoKMUl?&{o_d#~4 z9FZTi(Tn36&Z5nHI{Yqb6~xf95I<8^2$LGf;E_jL5y~Aiq6pM2Slg_8bKi}iy>3Xp0(8TzSKW>rs5%uD z$9gprtzfU7$6l>Ka|TPVK0s+#YvLDSxwHi4th%V_w; zS{&`e)jE#QUmy6VMZf*e0$5p6Ib|h%XcHCHyBpFRz5pms{xm*XIy2 z@q1NouYlO_gz+P@Md2~x-P#v1FhkD*M0{;tX8|9Sl5q&<**KxehyK_ff2?RtOv^mo zb{;wLc~H!E5X#~uFmm*j+>G=g(8SYYa8Z!8vl&0#1FbzIcN2cE8SX^GC7mN6f|(#}Dp5NA&Y)H1vVk$Z154{I$(mjs(84DOuAc$U)8=7 z60JJX3-|MIPlE@KMeKUMXGeZm@G9K+jJ3uag=KH66-J~n5<@Qju8QYHB1j&$s(BZPFeYzLye*Ts}y_1tzG8wKtffttd-s4^Bl zWHLK5q58il!uMgm_eEa{$B?`YNI?6Lb%-LcXj_H(z3eh_V*c$_VraKktd=g+R<hv(4s&N&Y;^%_R9TX(XkDBJqp>^UBpW^7WY z9m%JIC_}EKMJ-G%t66$B-L7ZaExm|BQo^Dp_6keA{@kS0qDo>TCjMM;c4i!BG8qA_ z0jNY?y2N#KM%AG_KRP_F-p6z+PDj1+dUHL_z|?yc@kfZz?dCLe8>~5tJ>5JF-3R$* zIPTuPSDt5weGgxSrxl%ZuR<35`#f*is4V;vXCN!z`S@P%$}OGn^R%aXjBn^Fa=c|Y zwZSzp+2O;fs#kc9V}ks$I|AZNh#M?=vc`GlT%UECo0`N&UM(-rUrMqTdFj$+&>*nT zSS=prFP7ur@llzYcn0mox`=c5H?ZN;9EG8tT?y{xE!$NT>WOov^HHM@>N<{-z0-Zh zTRw0{{lyk5-Y8{O&cGCmOdv8Il~oMhJj%1W)o(DVJbCRNdtO69%i{w+U-jW^z5{hY zP%S*O;7Q@M7Srw=Prtn+InWD8hBpN=L8;91aG1q;OBV<;e3)f_!kS_U4`O2F?5ZrU z`O0*~7-?WR?En~)r=3-NIV!7CcL`Qu)3ac!6MX$Pc{_7x4yASi>2kB{9`G^~tYd=( z9)NGjJ3_hzjELsTZ&4{#$~*g*cpY;#dr(qky0-lmdUl+q1S^vF=CM7T$Wt?gQu{rY zKZCvRG2X&kw?02)fdmcW~654(Br;x{)4o zcF(17A$y$9Y`||XZ$NSxw&b19tRbS&Hu&sO|3b|5p*Psu4a$Sh?)e38KMaknb+(I5#NJj5rxcuM9qZE{Xjy!V7 z$id6Q9dg?Pmb9ilou>H_!~`$mTl$SR5QF*<)EO_rofQ}_n+u_QZsQPa@TLULbO&!h z-5<+x$p_}Gl2w2M1N%BR9D%G$LIX}lo_P&23^-6sLT&wmz_Ir^I5^n`O@wcC=kapE zLR`#-YdFke+(f*5fczZ`lkb4PNJsqf_e7KOHy+F^{Yvro0Jh4(-(lqMjfN?WL!8!TzxiSWuBJeRLi=v7!|;#~#1A!F<%$T5{p z&bfb)x;$r~CXC=*zsE8W9X=kTI-f5fz*}&n1mga!;)V+i6K?-}c)N)ES!@nqFy%Tj zU;_!ml~dA4WFx%a04`?WCHZhlX}e!_;*luMGmUP@8tCg>o|Q?UF)S77^VKajH@A1j z8h?6W$~K%m=`^c}E2$ZBxJ5<^vgyS-;{fne0Q*Gxd^yizF(I4xG*SE{Tt3_&e%d;WLD`cD{ zGB~eW!$__Qtnu1ZGa3(`2L>3JQ$k>ZoDCX=5SGy;r4)swIh*+#3nCrJE6r^bIf9*w z3_tD)Qb~RZd%;$Rn%`po=82e2gg=KSN_+|RIYzi@+OqA+@l9qYwatzNb}1+IwBOTe z)tA~nmn@RM^T^*D;E!y8Kftp&u{;{pLsC{hh{Qy#)My^<$V7Q5DJuOTeMWmNYtdGt z8-7P+wT!UPA=Jd~2@K~@n~l1$d5)P2f91mJAF0UJN&l^XSd-DM{eM7g;O>pi^*H7b zJ`F9vwoG`w)Y*t8V%xru?+tkk@vO&q9=7*l)2D}U>Ki;f*dKlifhWd2{BDVxX4s5( z4m)ODhsXW`egLpy&bLkAXc|7d=<5JcD8IqX;e8t zRf*}kt}@@D>f3n40y}l@TR4V&R|)(o zfnO!?s|0?Pz^@YcRRX_C;8zL!DuG`m@c)$ruBorCt*fdIRAl<=>T2p7*}khwuPHri zN@?|^vqlDxGwHm3V;tvKRaXTrbi}eoG*kzwruqw}m-%Z0RW;QrtE%R#!b=liLaDGi z;IFGLol+omOsSe!RyU(IP;+K|jblh@b#+Z3b4pEVd1l}G%Z`8ZuMU5;upXsWRa@z=D};1P{L>W`fsC##FAeCz0=T#e&9`t${q-gN^Zp;+-U2MD zuIn2gq(MPa+Mz+ZyBWG0$$=S~8DMCT4haQCK}1O@13^F#kPZcr1_31{R8Sg0;yW|o ztv=6vKhOJK|L^~u>tfD1JJ(u!#c!`YfViEbuMegV`dvs^67#N#grg9gzF?G}mlx8< z7pw#^2CJAD@&6`#eLr7)CtbKF(kJLdpqdDOsD~>I><$kCd%zLSzApTrpJlH8e-&XG zgIy6|jCg;iod4%CIKzFx4t`Efa33%rGeDpye$d}^J7Mo{t9~*Hd}#rF{ZbiI1Pbcw zigI#=!*pB`?kAN0oY%<}0YkIO2eA5+bzZP53LJ#=^8x$AeNcdsfssyNeJ?lyh4KIc zDglNgkbcfC3cuz-!5#g4Tz!KC^pM7WjxI_b&Pd?Z#q*a+V(1AJI8ndTW(FtP@mr%m z=+^Y`fICAylzg21JmCo6Kb!reA0xLQK%u};A0KEC*a_)_UX4HH`Y-ja{(y|o^7)G% zV;5k>l`-P~muW!2ntS>EwKjlY&_;s-nz}l=0#pGL8Ce>b=-YWhJpkSP`F3I@dg_{b z5Ict;J69M8a`J+<=bz7L3xUC%pne{I6#>iPigNkc05gh_lAbCs2EZZ+Vq|0oFl-0M zShEkz2a14!PRIs3o|v-(9E|Yu@c1YGf7fO`pq&8{a6xdNUrYloA{g)v{6aqkjPVbr zr67P>55T<8W+!a-^X=!|81Ad;>WFqqP@kZmg+>_HaM}+7kgEkk`TD@2o}kO16K3Hs z(246q0AB2b85xit!V&4|g*F2?Ox4u~?ucF;kmlc)>pMC9eT5Dj0fPPP4VacO+OG)l z3-b#JflLvA0=RlW9X#M5SH$mZoxc4n{gbWJ$`iIuU(jqJ;Alb}eL;c1HxVO%?+h6L zKX*=l4uG;5;HLz5*S9lL)z{Op1nJwE8=XEFshS%B?fp0%iwzQV$r&5AgeFU*Lo;lLoWHK+gYG0{AnekFx;O z3+m_s7jS{XkVr29K+O?=L-vOQ4|CC-eT#TK{rff5-oS9s>x32qYS0VJ;IFIM@krJ81s} zHV2|2G62Qz2#CcOe&T?Dpl=SuG+iVNF381=4v{C30D~6bFfh=~*99o`0z4!Tg*;D& z1^tI{)m%N`|LYYgg zlyQawLDdHV_xK4BFuebApuo^tdYWLUqa)z4!M;fFiPt?vG$)e!qcsSmg7ou%q45?b zZURyUdt!hUCW3nfoOT6)yaHfguahVRASg#90_BSG1p+TP09XvL4;*mo05pX2g8_TQ zSmCMMfAM*WBha?;8*cs|j|*7!@8kZh{DGjXAAo@S{8#CN{?rEm0ic!OGzxQ}pH<-~ zM;})Jqx<~5j{RK8Kj=6Sydaks)YpagbQ%70_dn@?2Ld96nFE#%;AN@ElT04IJxKu4Pjnhk!iDGClow^wli&v_-`Q?&$t+hIrcJ5BYeZTVrfl7wUzc@Sl49 z%RK0QrU;f4;H$Uz5>-NjP+aZt$J70{4 zM*$w*&Jp9>f7u5F;pYhg_A!2N;NML_3?N3jAX7bUJ$>M>BS7@@0bl@v7#jhEp^*;+gHJ#Z9esTRU{obt zh>5kM<9#*-3D?&>X_=4_#iQ|~qa{@O2 z6!53rgPdTXKTE(!3>v|#J^z9i#sCK31)t(ha47JEMxK}lG$Q2&I{^S04vdW1Zvb;(k_4wJ z=7;bAHj@|<{!=qhFx&@e=LAq4gqe*U_Me9Zd7$8M^su~OL0+&i1fp$c3^Dl+eK2~A zfcpv<1DON3uL+PGLHhl+S^TSB4`~Ecz)g@ENYrojC%j<-J>dJmrh1wdU>FjOUBD>J zh$n#tjVkSoO#p!&o|IiUF^?wuI|F|D!&Hh?1Fc|GX&`Ss8PX3NcOmnZ( z9l&2;#0B6Ltv2kzFfe8h`IkQhA}IpEZCt{qDum_=9N+>6P>fFynjs*HpSD3iaAq?a$3F|f_iQ&l!q`;F9psW(zF z`;YZNe-lkzi2m;*plA8ld4O^Lp&drPfFyse2s0RL4hO>ny#OHs76v`BF~o^^0a+Yh zAf#c$Zfc^&C;3bG4nVr?RIxBaq6N!;vbw()d-?$jhX$B{r30}L7#{^VA0WL1L;?Vb z{zOe^u!=T6KOguX>js3NQ;IM!2{7H$^wg;>I|3;7r$O_B(Q-I3V)P=Q0HE}4jEn zOv8Ve3hV;7EdZW+x&oV90R4dl*#$+AXtW1r2Vez|0Rf};UBELxSQDe`XsH3a65o@B zaezBQ0nh<<#nb^&(F4X803@f-8-ISlsDLH)bai&|1-rmKyZ|m8T|AKhLg5Pl>;YI9 zH6Wh^2ps5-K!bV=NR|e>`1*RGqy+>5T-{yyPj+|w=*`CI4$m0lI1B*uH$F{_o#2Z9 zcYHbnJ5{g0_{2yFEfypIQqhh8?1M!5{xkQ^C)|Sp-T`?4oiOeH?+WDq?YjTPI24UQ zeE>%XAk33}30f3?2?(vWa3Bi{Yy zfAoic*8jUd{ImX_{o$YO{@EYu>+AmIgn(#)$q-=nZh)`BL>hF`7|6v#Pc7XN=?BI* zHMH@;!GMeKfI|Tv2Y44u`=1U0oq7Ot6b1ObKS2->ozOczz{5JB0X*!leG-r<2d@9t z5Aen(8DfAefW)8f5EE8Cp@FWRex8^>`E$qd7rp4r$Ul?&b6&u!!2y-|Tay22>c6J& z0$}BzQ#k!3E$cc-K|3Yv8K>c|{0`&zBx*Pyd2KEi|LdPS15Jo%xKntMX!y;g_a8EB^ zAbbd%mU}?~ApARs0iid;0BZQ7%+Ujhg8x~Ba`E#8l01k%OW_`H^kD;9MF2%{bAkbC z2LvEJQym?C0CONkqbEVd9O?_C5`EazV8AvVt@i+?MN|4OumG^gzv;#LSFeBThdHMM z+#nM0fIeU+41)W|5dgdr00#my;35GzK|RpS2BC=ou3tF92S6%73t%juk&_S769Wn` zB!GX5%4qQ-;J`@}5VV2A0`!r}pX2TvDW7# zBP{)MegKboLfrv>33m2CIzT;sWxUW(4IQq~v+$!2d-+dh0Xw31sYIB`fDgc*W556! zWdGcZcp#DJL=V`{3lk0b(F=s$$)WrFm-d*ke(wPIV?a?c#b`amq(Bi+PfUiz9f1yz z0E$2%fr1m&I*}#P5jZde0O23o>r)YG0V&y^9s!irAKC$J{>+`8P7hKB zcn9nQK;r4&5 zUsWup0`qe9f&&=}I1DWo5n;Pv;0)6aou-$l)d*RWJ`AUX2g+%#TfiT25-A+v}?-cN^mBVCIG|36A2}2taEPt@Dh7Y^@GgAxmAF|UFVETLk*Ny_+ zfw?K~J4u#eDv#L+iAtCa)>5oJUMx;>IkXf}^1#xuL{+}XUAg2DRCJCE@{N#^fr{T1 zY_0wk7H^+s@wV}%w14I|JcTtE<&R@jVNZt6bDD2?(lg!vY}Zkj;h5Z?6HvZ8r@faA zCuMm-BjWy&(e1`vd$V_&OXS$Yclou5vUE217QO0Kp`!MV3iFm@U=afI`d4=U_ z`Eg$MB%r6jNwIsyy5Sh9xgYQn5wE?441CFTYeh>lMQK;e6IF@Hdc^tP+#t$l7GaD>^Z?K#*O>t0z!USYnW}1PNAs?*f z#}2!1WMMI!Y$@&Whz*auwLtl3VJfUZh zd7mT}>oP>nEwVMJqNoY-J-?3OOl%iedqokJ+9PK%P9-Z16;{IIvg%ac46P5V3vA?6 zv}>a$6j!wyB$IN?3K0q@w${=9D%L|PM*dtRl)BO_PUx*!pM@qlWw+7Z{BX`Sh{|_&M4h@MIHT<=Tmo?ERue`8Jvy+9NZe`^vaz-|8M>)M87#z((gN z#g;8VpE$N@t;Wc0eE&f?9=|%ct=E|+J}5V~*7SgtueUE&Wu+E})}-~l zjJA^Vdun0IH(|57uwxtRp;mwAfm`j`!Nr#vq9TppuVMZu4kJJMFix+6=ftQUh5Gc- z-Hh9fPv=tcoZqG)=z~CcL#EiTlfZc1Dra$NyBlV3&ZN$JTwYilzWI8gK_iCmTSN(e z{JRf43g_3k%rljGwG2JqvJler+-(tkwGoV86a0*)iu|)}#X}aKXRVJ~&8Ho|n$*Z- z8Io?+7&&wA-xZx5&9r_|aPOmcN>1{3h2R^46hV6~PN7PaLg-b<&b=0_Gn+XXLDZI;K`SJ2&M?nixkzB&;V&c==XP3!$}XD-;LZHU8#odj+I{ z`SG<>miCba&QQkJY?RQjn#oe5s>*fF3SueZXO89d=3#*uCe+|L!?NYKMxDaZcW>`M z%{Ez<3l`*Y3$j3644tVL3BS5Ug4mX2@w^W+cVT}V;;zfybKPKd>6&Eq z&8rbt8k3fbH&yD9(~3h94HBjLP(z4RI)r4`(j>tjWy)q!owc@?+!~ksnRz!xi%WL z6h#uFp8Y6_oG2KIad1D5Cz{iDxNyl48ty|4>#CD>erg}^^q|D*@vSxfQeMe34^=Ck z>01V6>j*J_(MnGv#dzxwJm#+ zyj=pb;f?a_@9Xa_Xi#=c!!A6DCnKlQmm|A^wtrf8!EbH3$&(X34Wa=O+s_cz9X3@N zL9R~@G~+JP;MwD$#8ckaCf0Wh)tR^mYz}&>w#hw82`7B^QGZch@N6@um8`&FY6Shq zALUz`_CSxBDSjjt0LXl9y?sqV2K#5qq`i z+iqmG&yoMOGT9^{aYn0yOdw5Zybe}tO~Dr{9Fh$|(OIq^S4V_WYo+gbtT)8sw;0>K zx#Fx=tb27!sHaVL2D1|-Q$-4jDa znp?G>Z+#eAFVUL@J@76L%uv$kMv;^a(a0+i4n_H$uYGp+rRooc2Le5tVIKp!oKw`f zoOIWc!a15@g{z?k4RJb6WWl&n1Y}~PgA)mJJrys`1)1)Av6t_}soM?~EXdZHCUL;F zyg(@Oytg$aYPht?)8fj3wi5N52U#DyQlp+Lh6`vB?uj!?QLlWCW!a9b?p25H^VpJf zx5sq&uiY*Mt^7H&^2ulo0Yl$=$xW&YF+3PH%oWk+pk ztGt6u;h#g9`e-n>g+p(#>sQO}m&t|ZjI-b^_CEG`>SL8aW z_N4`d2W$M%sh=Q~;qV)4mZAI2&x?dYqna-3c#;?4ej)A^!x_L!5PaqNVmdfsN6K-o zbMysw@LaC4*12_XmUC!L?&TVnzt&F( z@e4^k54pv%MdTxsBYl6?B(C(k!gDp#w&P}5gjQt+ClYYU`-Q-cD!e}PsMGfVTrDtq;+xbo0}k?J8j=ejKe1@B=tqh z_rGV9-gp&=y6mgtCZt->)q9S8sz|OI$20PX5wC*m>DHXrf&oI?^U+>;B&&X@9)pZL@Ho?+x+&jctm`N8RlD4@f&no_M;KtyF#MKwA z?~HuHR{ytc|kG|C$xGYkVJ@0Ov? zsj4LYB{ztjJLF*ghp&er)$n?gYs}Rw?AInl7=zqErX#pyo!UW@8xi(6*lJ=0jqZ2|Sp{oOyK# z{f~(ktfX#=Ew1{k$M1_o)H@qB3iE$au{?uNdG9XCYka-XGB=4QuB%v>Ew@^g& zUJ&oZveKo-$%43n?kpeZ-p~}u@|U}yY!z{2N=WN;g9xJh32}XOgVTkHC${r$+0^>D zDg4C5dvV3E`G~c;N;4dy2&NMF4UQG}rR$0~JI|}CI*w>2hdX$Xs_>fkJQ#klSp4-0d9*=HsY!OvMw}W zuZn2ebdr84&z85VIeqIr~?pSr} zUf*Qw0~z<{D;YS&!$U*^q2cgm)ufTlvz&EvqRdkRqt_~$zXoA#7L6e0vTxYm5|<;5 zC3)TSTn*k|99BN^oPwdQm}i;jmZK^>)(QkTfX6s#@|8kHF-=i?KW4C_Qf;>Idk>9*uA&6y5L2s zCE)KK)H{LWvdv#*^)hNwE4Fh)md~w@k1Csp$wnAX3SY8kBG4u!6>Msf4W#JjQ#7Bl zTK-X?)lZ5Qbf*B}l8d(g+J}%2k*v;5QVA>$n?fuh+F8y;KID-0?phG-qND1L?Q1L^ zXGK=c>CaBLsEnMhJ^9XRaLXG@n0He3n_LC#Q!G8RXv0lxGaN~^q_+`jH~lYR>+2UV z^X(GC)+PtR7aarPt(Vu;bLXtiT;ZBO154FZe^J^DpCoew<0VnDkBKze6A?)$yF9y( zUmeJf|F}6t*?SMqoBSKd z7h~rntD6-z&NXvr703QS@fTBP>s*A>McB+cb)-rPtQo*@GJaSau6SP z-mKM1_F7z33}>6`%d%NwT$~4^eGL!#0;p%n^>W(0cEsYIcxXq54$pSxt-5?{Gi}Xd z$_dWH=j{JD!DQK4TeKMsPvq)K6po5ACnnAntIS(i4x(LHZ>h}X{VpA)rD%79+T#yDK7fMqf-DlE z>1S9wJ1=E*PTv%YKJ%zH&z^$(i_Mzz1&m&9k>RhJ z+w(utN9bBtbfm&1mo`4!%on(DDPR0)#L|~^!j1&~f(VnU53#Xl1DY+wj^?DVnG`Rn zuM{60tIe@?_BAVo!DH_kHMXEWQ^u4H-7k9ME-_D*moiU4%U(1BejHPC2d~8~!DH4# zsPjPufqvXj`9d2>uua>A*4?=7A7KwlREB5$kfeoo#|mfEzw)-So5jU{zb79*%XX_( zmXUGBwy>r!`dvESp1rIJn`JdrZT2Et7Edu-RN6VH-NP6aIdXoy8ToBx(l|T(!KP96 za{N$x&)#->Uq1wUsm>C9&z7z-^#Z5Ed7EpTy)Ycyr|-)YeWJ2wyES=m+ zEK?d<)vL^@PM;? z8M<%h!GY#=@WT6N;0^30U?>Cg8SDp1>Qh$@u!RKo)m8{mu<8VhKc9MT-d1?aD}b;cE}q_);~Uf|CNV)!1JT< z!RfmxMR^2ou?Jtxe0qF2Pbcw8zSq{b`H^^mUWL7v$#+x7L%)92meanUM3Fa7YlNe( zvZ*07@N89?>>j>s^oM-YCB9J}KM#~ewQ?p0C$=o*57k*WS6o%vQl2tr88>1QySw3D zy^|Bz=e7l}V4#R^)S05^B^R=v~NT`=jPgyKTG0=c>ByM`S}~R`4gXbj?!-qL*?Jeko@~ zhueILZaZN))hCBJ764|(%(f&@~3MYpfu+^9+JGyZ*vGiS@I?K+cqZ^^i^>Rz3 z?!Ztnc(7i6IgRP&Ghw3@Flskv#)fg$hoSZ=^~#%_m?y5Ws?pIuMq4>Pthdk=I7T$} z7nGTn34L4ZGQAk{qRW=K8{ey^>b4MqrZ1Jn$z~v)>i)5(J89#qGwyl~p?jpXq*BbA z-jD3f?(QnuV_F7m7cb2c1uxkwwttWiq}?K{;x@9VOzI&-ZNz$HxyfTn+kqUSp)H z0_PFOTHd?GDeWTPoUeW7k#{Si_!U}}K_dCE)Z{hH7vmz{BKVe{K!xzvJc_L38l8$scQ=?TOFIIqgA+ZRCGMC7#U9tosB|MO? z_YhSvcIH-W?p^Wp1!Jp~n>~BS#Ewiop+6elotaMx_Y2|nSQ&b2k)`dSJr#u|R7zLC zBKS;|TK&=@ZXGyhuD4ch-Q@g42kX_qQ70v;$7JI*O;^1Ql8!%Z+=(XFQ`|~iG4^C3 zcBZQ<^|eE_lW=`;VysY!JATFzLyp*%rJ2Qb{b^e&7M$aUEt}SKemr*l z$Z&@#v@yx5dT5=O&wH*qHhr{}QM5zhf-Z|j$~U0_P6FJbriaw?{#_oozQV)LsYPiA zPk}A=_S=43cYrQm=#G1DZ^YKeo~2oBg#D$c=h4}l4I*WcN>XD^;(304PYG`-y}Kc3 zbl<{;>vQw549bOu6lblDA&?laoiy#iSh|eL{A|JO+5@-b#GwuTOs&hWpUAbkH#WZ_ zx2!QJ*W&+f9%gC3?a5Gj=YqSCG|oom2f81rm;ODLRdg6RlNN=G>tIF5TCkaZ z{M?rHq3keD;sgH+6lZU+!fuHOGTYc{nROgk_Ic8K)^5KDgB&+!3OhR_KI)5Yd~eSA zBr{uU>?_`Kdq5fK8VGOzp}k}>#HN*8HxEr17T`OixsS8_#zdu@^zKa=N#TSlwUgi$ zRi+nLUy@ghv?-dqNrOyTM(*`m$a#j7%h3p%hwNpT?K5OHMmtxebHEnaDC{A*DXGO`k)8GeyXXrNr6bRCt+#XbggTP%Q8mCiQLr zQdh}ZHWjq{#D*N#vP_x!G9UNHFf!@7D>n@_5};UCiF0nH(74+}je_L&cy+#(iV@4a6#snVoyVZ$ zS)`%yQC2?o>K$g6i$fFahW(a$1#AzQ$~n3Hl;7w!rT4uZhRMjq1V}c1i)Dy9n7oLU z!Tn{UOM2P+@hfa9Ck7DU0Jh*Zhr5g7qz-AFQ0Vz5X}9MOBhu~DeXihG7lp-3m?Zi- zt0(N9t$xyT@k^P8FYV=eHX=(&{qN7LV|gp4zfM7T8++BBjwqZnx#LYTwi&(8R*C4OqZkHs1 zK$)GWklOoRDcZe`$*+oTI6wA)zy$DXQs0|DNVAhQt8o+{sG3pTSEQi)q>>s=U$Rx* z_b6aW&MxLUzcv2C`wIspHz~v0>}QhGQ1hhUBMIDvO^KH+i;w9Es}AlssJ(mOO*6?! z_gP?AYme8QnYc^j%XE8r1fQTvzr09u{hOj{Z6J_>4GcTHRE*TJWv zA$g+fx=l zb6jidDqIv2F?-G;IP#%?vf={HJK@kB-WJ{k0cG_I#9=0y1cwUs2PDXn<7+sVDL=iv ze!;CUbUt$%IYXCy|IzHwP>Dk=v-OQ-VLS5Z<|=-`5Ik`;5;pl71`=L;xk{Z?ALG=9voA5S_Q`b=6_v#jF?f|3`$cfKGpg+MM;dFn3mW?Jw^(2LArYJYE4Wxu7fs0{ z@5#@&ecuvw4aw=ef3miYy6VYy^xBu)3Um1yi&)ikjQXkV42`d# z{|qB28}#ta6?}H(dHj^?`y^U1Fp?RKctT_+Enx>yHT}X6554}yR}9TJ%oyBk^r@bj z4^ZJ|36m#MJthZ_^9a?4Rtu4pn~7)3zY-VS7*JUMuCHJv`dD(fPFOO`Kl_sHQ_!U? z$9cYx)hm1h%rG`eo_)5)C0bS?*Lc>fLNf>dy;lzJ#dzSx_0{mbfX5!PSA;#%D+b(Y z74+RJH|K-WE?o(dTFef6?g9#9ct9I4UKSsKa)5aqtn7Q4fv*_%#?Bi%Yl2LcyRuD` ziQ{eJv1o1hdiO8qtH3T3q`cDLGBeYNx?8P(j*>^e_?@tt!{B4JS4{d!>Zt=ttJbTn z_qkE65(}vvweBPxj6F$sa|PMb1N zRQXrS85L^&8T5(kRS^Qns!Q|9m4%+nN~LcPx$=HpPtMSe`WEw z{q<_D)Imv5@xhsD)gRYnCVp`4wr$KE+H7bDHEwq`fww(T#iKi;Qlr*%69YrHR0o1A zY~D2(wY{Ux22W?6Z=4o+Pu@#`o80^2HL_b=eZBi$(sG++q<`C{?(O!_HsbbCQVI2< z;8OLo&$SezC5IKcw|X^Kj*K~f<248O^`rd zs)S*AF#|36oO`8G=gB3c>ZbSw!-vZSM`(;CE~ocOtX04pCCtGy%(3jqtnoBh0qr_F)3|0?2u zB5tyrjQe!ULKJNJTg3vBM$%&aj@RXjJDl~ti3~yR+}5-6da!STf5z7>x2qXC2DJ%M zF!ByxFI7$9ecm%tmk8Ld^(~Z z0Y(S6G>N`Vk)6AU|T&-muU) z1K6#hkH~pHH$&C(smmhNDvFbwyV{QVD9#M1(T%@T;xWPfK!_Ei&Tq}_4D`<_)d6V7iUZ)K}GJ zq5P9S5-uXwj$X{ZwG1`w;GcLk-T5VSxI*{#gS~p@r`H)9s-1WV8p#slBA)9DC$)Tf zdV~8N{axEIS^rnxu3l-=Xc0q?)Fe%hWJM~7AyyfiK6%Tu#;^x&6t_*A(TTH}Sb~Jmsc=yn~J>GsvtbNq| zWb`A;J%?|m(h%&nN9zPt=RC-*W#7^A+%@A&wwvSqu@^2*@v27-hG5au^)E_T6_u|oM8gn~p2D8_Pp18N`Z3UJW+kYju_y1YFu9R~9hB5YmbncY+cr<-;& z38jWnsfo{%mf@S@jOR7_-Y_Q$UtG_ClQgp;f}EZhOcl^tmEy)IDTxYdiQh!qf8mgI z4{_cr-7=qBan#B;vMO)%8ZgdVj@Am|GI~ZCue>rO<#b}r%4=Keatmp04Va?HZt#K< zNsl};ThBT(iY-5o_$sx4Nl(m6Irf^O`W>z`dgHj+lo3|pA*P*(*l)(h<7|h_y7?9x zWUe!&N7g|qiE!HHSng%GnICHp+iW3X_a$Pqs}_Z#{8;n7&uBX<8QhJ; zUeuBt?Gju+Ycx67bx^9NWEo7CIIDFrLp4*y`tz5u34O^Vvb%{NW)v(Vd2aAfJrVt` z>6H9RCxn~+e0jWUj5tf(yTi+*TLj1NOWF*}WFcEu?62$S!qYxwOIfI|JeitPSf`S> zR{Zg%?GFK2USw(Jv4{h6pLlo-gIm=l#F22y%ly5Tb1cNLsj%km$Dh@Z;m@y)=~t%u z-1}zr{%gpGAURIz$0^=Rgw}k##r!WGo;h5A*7eSWUSw*hsqo+uqHOZre@9u)bfs%$ zWT-mO*x!lm&Ra9)Sohc~dL6r;UfHu%V^^ooZ=8oTNi|CFG|_(y@GThjamB;u&oA+- zV>?&0_)Ue{iE*(*dQ_vP%wQF^k!5PK8@Cq!NY{2-q2~vOBZ&W&*m{z>t2DdVVMD}X zUV!B21GoFQ-}6KQJUG)I2(El3W1!V;RUBL>A2v~Ryqfdf_*#@r#DM1J2Bgo7nHDEH zO@Et1kwX5eaC|M_>?G>;z!XC8OjKdE7d5L)f!4AgMK@i4bNWEovoq$CP}_TshE|cC zQTSReYiVzKS^My5Ze;GfRPKq{`WzM@zawVwwttl`AWI zfhO+{;OnHhWp-CYx%SIk=A-CUsX1TKd`PHEw&Hc|PrYS1a!n{PK`rMyW3x1m67jdp z3;SAH+Cm>Cr)!kHM2GH8#zCzKCLV4C>+zbGk}uKH-lS^iemg~8d)cDK9WRa43j6xQ zt9B5a{hqcw?hFFotd_k2jr+pk(u7qLpG?CY#tj_0h|ZR}2Hp{P^QiqJ+WyPA?d9x^ zn}hpRLxdd+K*+qhQlO zhCx(R#*DpiV}#9IJDARBRgqI!$o}>Q4XA|0A@|!&?dCz1_o)V(+iRYzDF&C)iUco2 z2cIt#G4{GM&)q+53@V?)5ol?t!pl2=g<{nYn?mE-6O=@KbD0SrOhoEYZqA}QU={0r zx&vvwhG}?e=(GT-&hYDslA~RSIe9yegpWn;+NS;v)0H(HKsh19f=lK z4qYgJIt+_Kebey9?u5`|&fFVcG7{ED<4p%(5}_5AOL$~V<^tZVniiWDd){eqZekIZ zOcvvd2}h_WQezaUpPp2XS;h=~zDg18A}Kv@x~!-|ZsE!*pizx||NA}1T&~5or8|8E z6Fq$tk|B#I+(-9@l36ft z9|jXgH$U`dji<9vsnZP*&OEd=dceoFF@OKu&8cex&phP54;rpU`B2+DBIik#RW6mK zV!5?~j2g05oBOJBx7GTfG9SwO`V7at83+2lnZdpA*U#FYR+2|wyzA3Qs%CVy2`Mau zq`K)pExVyj@+kjxV^r%F;=4oT)WDhRZ_lyFhOsGQmD$dzF$F}0+$>+LV9AV2-k9g; zQSMgTF`{--om0RI5{4AXM|pH&8~;d;oFCk25GgVH5gjSsnPx-n%j)Z|xk)lgqQd!H zn)`3KC!e7;urZtXtK=EG&`3$S?3cU6b+ zIXJ^h`6DVdV7S{DWA*uf%eYb(oT(NhMhf4UFZ(^;ecF+eadUE+>XUnm-*A2XNE-3Y z>N0x0+YFDKx0yp2O`Uo~#jNNG;NuSyaX$vH6&7@lcVtiCF7UeX#*)>Uv6~TvL)C6EtgxLu zY#Qe9b_d69>Jr;30$a7eG3a9ki9yUm8v{{|)gTi;IqWUrYnN`3-ruQu$u~V5aaicv zrJGE-e~(@@kXM)ua-b1&MhE&)!TM$UwUt{B%4NUe>?U`?y3zIzcW$UndPVr~>(gp- z^9+X+;k7A=_jc~=FC@qAf8(m^NYp2|0-Je>#EQont8E(9`tZ(eLnL5&#_YVKMUFYG z!MqRTmH;)o(`PG~+ zWSN}t;#{fMSo59|(0MHwu{UYD1aZ8VtKh!Z6+LvpMZCp_G%=`0!scU{!K}67v56l0 zo#h?W)ub}&ZxYns6{B$ObILccUHrtX6lj|HbZjQ6yWCnt=DFpn?ZAR*d62Me8)+!$=rcY~xZ z>1U+p*let`F5uF->)MBnZC@X7So-pO%FgIIVXMTa>fmuTkV^w|L;q)?>8mMbj)Ks= zQ5KwRx4SQkUgE!N!Dbb`$Y>>5 z{R<=foI2l~#|pMTD(myfdX4I4S$|-zo=)?vRO-p$8(T66-dA0J$AtB_*Em$)FDreA z83qaLi87@rO;m6gH;vx?@~LyLGC2=I>_Dd^9&<{f#{b!NOM4^^oA z@WZ!BoqeSeVUkMWqc$9Eh{^>2(9y5m?Z{C^OQMRr*^f+=n~^~!+GDOg6wgQk9$DwL zzL4q+!-_a6y|<7_#YQZiD~>Q^xL!={6xZrKA;)P*`+Ars35zcgE~)nS(b`b@Q5<_ zdM&l?$KVSyYPynRyeU%GZ@!j!@ts&&_N%G_aZH@Nk#w^x+~ACy#RXe&Ecu!kDC1fw>ObI*A{~}Uz>j=@7-71{7>>F1e&ki_1ays@Gi4tLS>4)_hDd9 z3*gZ=^!Hi6;0wtb@e6#P5T^fJJDi%}+!bbrzR(B05kYB8%7NXvIKhYAogwA?4Ic5T zVV+a_lE_b~`v{M?YIm!Z0Jk$nLM~#Go2~@mPaPDfJRO}L@jwk*KG_eC6~X6U!eDjS zEKb*4XPm)3V;UFos5JF18fypn-OxI4oYL>u3Lmmx@7Pk|?P5%glSHQu{3U;8TN#tTvN)|Il0 z?&C*G!p^k|xaWdDN@{U@BL#o_YL=b90^PD-{WvGEJal#SQ=<5zFE6cKKd;qN9Ci&3 z9AtZB9F^jk9`l~3+ounH_d~fod(W@T`n$#_&}Mja>jsBh%o>@w+WIXe!mR?Hp6&hH z*LHi4^>^%T0>{lI>fVtg985^f%TG3Jdru|hR89{vZ+}?OmU#c}toztEr_#~q@4gL3 zKI0p~dEn45@-}zC&1-SUhK_CU{E=-3=dF9~VV5R4H5F;P-k6!U{*b-hmLE0La(kJq zx#X#C&;McXO`xNw*0tdp(nHdr3D6mUfMGN;Xb=vDK}i_H9OglkfXoDm%7KUsK?V^4 z!2zS9LK#ht5f((j?ih_cO3iSUxd+*w5I(_}m{qDE^wf?nktyR72 z+3&N5diPLOZ}q!t#qNjEQXiE3QpxwhBv1Rj?Iz6ou-a!kK5qQB^-;#M*OyMYG-1j0 zZ5uvPHTC;Pm*4W@vRC?!c;eR|YCpca;Mijq-`~38oreoPFej?)To#LxO3$z z)AD~@+qLp_uUCF`;%oKZd+pVvS--q=!;ej0oZRo9D>K`?wQARO7mLodYx&&Em#4k( z%oE$5FTH2i#_y`SH*9+;W7C5l-v3tYp%2%$dZXf-!z*@vquB>m$whz{!_EUZ5;oZO{!PDe|Ym7-|f@5Vbt#25gpD?nLh8O z!TsM~mf6$0;9yDTEjvG1aO%vTdk&ueCbs9sSG&(xyK!;j+0TBQT=+n*$~(Kn)j8Pb zv%K^9k7S-1)^GpX>jNh@-n4&XVe+y$vt5G%?Vwyaa*rdQvc*}B1zfv%qS zo^3y4`1cPq?z!{53iFj&GETKFMQEv(flX;%a&#r?mtp#&EEGv{qy+gNB8{s z+mQXk@2+w3-tk8^_RUx^r{|!7mHi1->%>?5GB3UE)0uT{erf#WktYYAoK`P-&wvr> zr9EC)yVU%8#e$?-dtYjrb9rln*N@kI?dQ@q8ye59ydrmf^t@Ntc~k=+nJ&M-18kj zY;$nJLjTT5?J5sGXVuC4;a_>V|0>9wa-h@x=1;g!HeS4Yqp@nvoa$SvEy%p<)-8j_ z6rP&W?%=`P!8Lm}jvqRsdF=AW)vx_3*|?>$QTTSBCbNr{3}5@uCw(?PlCb;yR|C$U z+4a~uIcs3l*%lfT!e6QxSJ5HUl{|-d4x<j(yZJY ziu(NXRP)L$*Lb4qjKdcO8;)#K>BPM+EnAhf_2E&~m#)0F!Gd>=ZQb+hbBjuU9Dd~T zjyh*f9(?Ec*1)CGmnyWGy`cV#^-J5lUAHvvV*H-5tva4;J9WzCTQ`m$dHDR`8M|+a z?(_5H^sL|CSQ}k@c17jqn(l40;O@(HALaO8mgbq#Xqf-!&Ap=^Q}jF2-k9NOT>RbA8rzqQhfgArh#teL=#;||g+<4Tc zC7FjOyYD(=ZcX{LutD7CXC9pU#$DHJ+~i($V*84w`-)FJ|A8mJ*&jy5#jB&2B^COQ zZ>cbDX5*VDc%R4}vD6%L$I0`(#*fE(1<41}M$YYF4FC0~$KAR2tum|s>%8~=0lO;x z_TIsDBeTDMtMU)U+a8)W|LsdhYkW1lbK@`L&rSPm;mB_L{u(%JefN)gZEQKo*!sfP zXrb%_cV@b|7{Js>CkR5 zFMlwt!dI>Da%I1qX!Ru-l zmaf~ddjI>0H@`V(R)r1Mw;J-!Td(DA-jOo9_@_;8ZhtEF>rM7kpT04vMW@eyczw#J z&&1b16g4~H@I94%cYHIa)riM#88@NA>cQit`abrbcqSPA=(TH&m^YsFOmF{a#j~Hj z;C*1suV&3FJKbY%8!-I$J)=h!+|@j{>UZw(cTTRp@5-@;pFKEu>X&hQyL>fz;?cMN z_~XpB#qGAemH7GMbvdVA_~MHhkACvpuAK+cd;fa0=P#qTd@_2#))&8WZ*0@G`TDFc z8&){bt$NJs7rNNZbEZaRzI!I;XuqSQSD#sYXOnxkjqSMb&0%}0%^tP2=+lYyntwf} z_v2Gy_u8F&Ti#BvYTr=b)o)v?&AU^5@2r1h@P>v_eCXMOjcTkt za_hRCBc87R>m8?;zcB8fF^^8L*0`gWw`uPGaqei(-Bkt{uKT-ee>^a?`0KR|-?%Tm z`li^Y7k~Olg>9dI`_`Gm(`p?(mH^I^iaS+~}I`@-=l z>n2X=^j6Zz!QT0K<0=OK-O4>`h|jDPe0Jo+W7iJ9_*roLunr&RCggt|Q}pwv754X? z9o1v!n|6y~^+&(-dqPfW_fBIwADVLKEnUWavi9)cFAk3K9nCn~>cCBX)^1(1``Il= zW`4ZB)AzxRY01C7^Ui{uoBy5m=$n0Bd|^XfSM$=%kGl7Nc-?>_eYcN3Sm*kwquwm; zGHjPseavl-HJsSsrft3r&n=F9xB5}n;6-Pw#{VC#cQ7<#0q3gl^$ZwiR<~QFM;XRE9B;jRCD<3oxppu(qK;t< zi3$d%0q@1tCm2jd_$tC?2&Z|2!F+^6eBd*T1rJ~M&aD^> zjzQQOUyt8_aCv3Kdr%(20K()d!C-5Irx6w)j7`87V-a3LSl5g4@%7L#2pd&HdW6Z< zgTc6H@REbUNeGA33^4#MRrs2{>hwNbtg z^{*2Q<{;c!HyGT7aN4!_&UdU~EJQdE;c|qz2wz25fG{}~^_Y<*KOxDMfIgl7>> z!^d=6#{stt1{WgCZ50e2K)4*?MTE(>puUw5k1!YER)misjBOJPZe={evj|&f1cMos z5uX_hE&@;k3gslhTJQ3#N+xBM> z9!8jo51hx|hIT->5aE1;-G-sxAUr$@{jD16F$U=oP8$pTAzVH#81z;L9*=&6a4W)m zgt587;7Wvh5uT-d0`gBrI1%lH@G!z1#Q5T9Yz@>4VIzk4fb%qj)ADiN2=^j9h43`O zO9*4}&HAL8IM1nQPlVGD79w1Za09~A_n|*w^?}^!s1L$d5l%z672y(udl9Zgco<n?&Vdlq~)nh&0C8lB2MJ!I+$ob<+a2@6W!ynhtpVXzIZ;H3TXj83K zgQj)Gw3G!=d+Y~-SAr9f&S>k8d)R90Pg-nu@TV+_YU@v3=xXmzo9AxtPoM2+@6WML z#rV?^lZl&2e+rmL(bn&6?=#T6#uDt8j9_p9LZg%am}%Yb$8;fNuOiGdgTdum_L!EH zAntqw8PD$o_IxWC?6>Bd(4?@@Ckn z`-PaRFXK48PXE+(YU{fX#~p;?I{HR{T7q2pDrca{;c|GO}1~Em7js2HxQiQkxjLG!J%|qLy-4)}H%k+CY`ud45 z1EMJC>;QQGlYWqeJVxMJnT#^oZmJ((3@nVwVn67@elQc-O>H024?0Ea2ieG?5cxIp z2ZLYZIJ_UU_ovS0n6hq{y(-h`lbw815!23}qGLL;-MD37KaD&~k>~&Ed5ZRr!W@Hp zfm`)?qE9cfyXgMj!=Gk<%hL?)?l?_M1feklQ6Juy{!Bm@_(r<(uWRy2~@xBdd$g$U$}okatvNToYB^4_jHfZ*}5$5PeI`NuB%| z)`RF^*rj|>Vzo)y&Q9CXJ32gur+umkfsD_@^u=BWtuFVrUTD4 zslA#~bb5}Xy13u-#{a*Mqi*13fcHP?UrUh3UgYsV?O*JxhmqH@#9;6hb>53C^sj}S zo9A(E7Gr1(x(^1V9xxqzR`08a5UznppNf0g_DR8@spqP{?N5(J>^Eqai1FyN(@@4j zlyMeiyz`&*r);}6`&0Q&o8_B^`0|rG#>%jW2|93&V1JACVt&VcIl7D37?*;$MqzQy z5SNNL3^B2>UuGe$F5<>82J_#4(l4hWO%BrhpY%&!E4Lzj5z_ZiWiGP1v3=2;^P+~M zGou=vIT&=~f|rf`Jkk{-T}M1$u&$xwwwsvMI4IG-duTdhQ1OLblyv~Iij*O|Qbq+phS`Y_>i z@TG&-4Qb-)2ZOJwG@;i(aWB-xzr(b~$`Pk0?Djs^eLm98LE14YZRmV0?kmK6eOs9N zJNjNWTwxtwvt|`fuhnGA#G5zI>5;XXiALUk3JOX!ImJN3K-)FXTPnJodZU zQBaGTgxmTC;}{cad%w4b?{0rmC)uL7_eQ#W9!*q(6`Jb5#1pOuvZf7e?X!V4e%NI{C7tb>b}FkgG4kNZSp2RIFj7rv`(g zuby|p?%y-zJ>4zk~0}vB0|ZMScSt1cNhh%(1Z8>Z-5nPTeMj^@0xUh0vg` zERrT;FG5}mkyk4768CU@Vt5~pfi23~0r_&sr~D^%FEy>-BUZhGkJq_aJm;4-`ulN4 zUNi76{s+7v;H5PF`}{a}%mFV8y#I9W=!P`KNHYm(-o-I>too0*-xv2c;+e||@6w%o znPHv0lTY*kFKig+|1af79SuZ&ldi{i@K`_5Zr%JTqvXAJd$e_2JHMCbUI5;5@cz?t zXWko-rUYraGjE*xLf+enx%TRL^Dua4{{vnW>@!I>{C$3`Ya{S7!23_@$}~Ahll$LH z^BB?;{x{OZp_~$=*@ZN?#TQ!=eh(vV2jccIrWc=otet#drtBhQbJBvrew1YiSvSgf zT@4_^63CvTOtf8hf6{C_8~2zwn9^`%5)V*)okP-?XE&tFXc7!=WP~j51?Cw590S=2 z$l6ebNpUI!9Nz_qkA;nGB;yCM9$oz@Q)z7Q8tI5Cf_w<%xs;==7g;@c9m?W0eg-Oo zC|rL!@`hJF4wHv%1op#7y98+q^gUbHv#PkJhLm%CY2vH7R7UbC`UYNbn&0fv0F!;KgE3*^+>y466}Vb;#k^1UK4X3%UTE7VaRIg zvM^&Vw#&x7D66xat9O|83z5sh6Ab2j<`0|Ut1V$8QaV!iZJqIS*->B0y87%W`(9Xu0O5)dzk;dW`fxnVGrOA$R!iG z^gEY4Mv%Aw}!b6|76kz6rH3`HN!#=W*z+=C5-d?nGQZn$s4xptOJ=AQ%`_6G3o(!8?#$Nv}kvpuqr?h?|yb9KLrWQXg5ZepE@ z*e^TzIM$aVue?sd-~;UVe?PBuMu)Wr+7TO9jCI!SH1b>5Eg1Yx&xd7wrOb9G<|&Ms z4C}{;*>x88>$L6n?V+v_sM{jmf2wCM^=zFnGpsE$eQ2bso;~FK;vD2vjJ#^q=%v1FB`$T`@n91Nf3&>cul;Qd`o6iFMWG}&h;7vRAp9^jdxLe2-b%XvF z*|;$Zt=j~tHHbh#$8GPk28qhzCP&7I{*S!&A+J|dUJH4Tt)4-wJ9G!_>>E`!0CK=R zi*((31%tmUT`%SsU&QN&vR6y}}-^}oHp z;J>^k!;&rUqaH(ks|N>z?N|Wnx6rx=)z1`m!gYMd1z8GZ*l;SwF2tQe9J;ceEtn08K=>nu(d`LWM8!~3+=x8i*( zj%EGO_e7uPTM0RpvoEl2Ip7`!7q6~V-3E!eQE$^AI|f;|2z3)}&bmDpR<{n3>(&?f z?PY#=KX|XoPu!Nr*Dy0d*s9S$LmJ^&+^GaKD&@l z{wQ@HF6J{)pRoBywMB-7u^r*mF<_Zj9Ntau{lAdcG~_h|d8LfTc+_oD)&>!Mzr zI4`cpGe(4(p_g=ppkp~jcJ#r@t2#OchrVL+%FX%ZFv}kYd#IjYa0yj&pO{~|U{b)% z#8q=2URBKvi(-RuE^EX|?AVoH9R!>QW?REV|0De0?SD1!zZ&>o4gCLA1MB)uS49@6~^to*xD$JTVSzu zR2a9CV(XzWZj;2;S7F>bi0xK|Ybrck;cFBgqi~AC6BVwd@IMuNN2k%nYX}^yYcG|CkI#H+n#fxC%=(PWkd`Ddn)`wg(p>bL51$#s(cmJS7CD% zc2r@16^>HjWEIX+;W8DzsKQMud{2d6sPLo;FR0MnN0qO_`YLR$!j3BJufkC(oUFom zDqN<*7ge}Ph3~2G3l*MJ;RO}CX_&y4pu+kpY_7tND(tVqQ7W9Q!g(rOrotChxJiZY zsqhOGo>bul6}tPW@>N)0h0RsiQHA|gI7)?+RX9(D%T)NH3OA|nJrx!llI0xxQih${ zwQHG@+Nnps>r$FDxxU%;O;T=5yRm7L8=Iu3r1r+=ShMgIPAO>Il(u-zOL0t!;nFFy zuGw^06XTr9+X*9aoOwJ!_4h~|XKo*%=J-e)L*4JZt-?s0!79BI{-1K3x>o<2Iy-e8 ztIBKy)yTP@rQ*Ghs%Hm%fe_L9NAi0$k(z~W2TRdt=K`ad@<8&pPjdJLBW3)jf1CfE zuQ>div6BD3tX^2|cy>mNWgPv%;iug#DJ6f)PrS|)Jp0DuTjFoivI1#^l6ClR`Ef9tOMcu_lHb(1z=stg^H0U!r6oUmg(NI;Zm=Tq zQ2C zso&D!7wnb%&VSo~26uG$r5{WFJu)cwE9=t^uX~j~OC^OqMUghi-?ZN0?>Hb?2b~+R zutcqaE#umE9e&YKNf5Rp9G0rNs7_cq2X@Cua~Y;@wsIVQ!K0YaV?n0*SR{r-GQ4xK{ zGYoZbXT-n3h!jS=Tm?TZs=mO8CX9F%rFmLbeVGw$8NvT>dsPF0%o;FoQg$|H7vf8TB#3vY$OeM`v{*JMy8Cx9@wJ)QGd7{|wJP-MkG0dwB;#^hb zXQ;@mayz6}l~qukEuTR=OYC2ewoZ~~Lv(+zqm8zPvC;P-k{Ho-(3LicV{ddJj^FY< zhv+!NbB1~?wo{S7$5fs-*=oBR-h*Q+1!dGtfX1>to-e#a(f_&KQ%#{;9IFbsZ4+=Anr=sOU9KClnR zW13!>k^xnB~;^P?Yw#;*h#B7R9s3i_96 z2HFD@##F?wZ6@A^peQB=sofH+j!DNUc+J4wkgSWThJF|$L2=AMoKUQpMEAiRF`wcr z{4z~R%rj6^oXom3W-6MqikY|uv;#5xzfHWESc2eKOk%+tut*mx@SK#p)XLeX`evN z2ABy`fOeSn{ZPmtGvO(q64Opco_({BKh2~h*c4JM`#R_)-%QE?O111SQTjA9egsgO zWe)>$h8e#QDBZF@N3Ca>@f(0LEPDge%{Ai>0A*QrW9a$;Gv0uGCfl-SqectNcsha( zv~2ztda;?%4JgO5lQEo@m_I@Q%=kG#MV37swO(T;tOiI&0bEQQrGzVmz!; z7cILAbp3&u)RZ~k~WJD@b%z8`hjZzj56yh^w2Y_!;yW}q%mhHcl!IUF_v z?SQgudlg#ch#43Tl#M*0`D12a7SKT3Hle|<%|H=Qj%_ERgcD|9JJ1;0PC*GL&A<_$ zJlnnpXM4&FTn5Uw?PI9*4`v{_Cfd)od3pFr&M)(AJ0InpmGjjS+m6TC{wgP|Lfbxr zbm!%aRb<;Ok>~GbQY+-L+P3GSZT~QnZUww^+xB@+-iWT@6$QxzvK`!xD;8&oyD7!t{ zJ<$rpr||aBt^iccif;y#5@l~jJ(I2Y0YIry_6(Fz(~6%8lon+_4PDo=5}pJ~kFuXb zJJhifwgP2D+51tmdRD?wpsXl+7TPw|O1uJ;9c6bxpJ-qu)~bc}i?Vm3pEa@)+XLl9 z*)^cE8?3-kpfOSQMKGIKff+z~s0&b2EAR|Zew2L#=hxg4eYYUW9)r@;t;8*m&5yD@ z$fc!~bP#Asl-=LNzpSkI-+>CF>|LnYEmnMdZH&JtI~Kamuo9XAt&Xx=p)PH$Krf(m zQFaUTiS}0fM4;j*`zSQg5!waX5oO}_CnwF3VJDvh#D^pNgW;CrA0 zQTBM~tfv+5)f}XQS*!IKO^YU?k8*=o)=ufEAbn zWVq}%&?h1Z*17C?QTR8XRqaQhVwXJ~nqO=s`s<_pT=op; zWr>y81gOMiZ$^tfVkPwjDs|c4Lc5Py2~&U$xa>N}bD5R+6wonf0#tQHsCl_6Imdh|!Sb>_?B8A)j6X-cBa5IqCZ68MIFIa&gKyhxn zGjdsF1?~e%a@)PpUtY2TPXVR4?N87SYplTAK&fu~U7W)!R^R|on%f?Lb6;x(E&-*x z?I~!LbygrH6&>1bKLX}@E6^S&%WWTn&Nf(qF+kaFyFc1!6Iv5!pxgGK_Y_-!bwD|8 z`wyJ^W-G7{XpGygfnL7V3S0!rbK5S|Y`YbxbsgFd={$zF-~>>S+g^%X_E>=m4bXmWdo1+wp%rKj zw9aieM6FA$!0kZAZu?m`t|V4q3eXO>{RZm!g%wx=RN}Tvk?w#*rEdEfX#9{BcoVV% zZo409cEk$o2Ri1qi_s3ptiWGDr=S<0Z>)r>4bgsX8|pN^wF2ot7v1(cly}Oi!I5Kl z?BQNqE#%1YdhEuC`^k#$1!kPbJ_qI*D=-Qu$zv};Hosa)Q-M-E_I5CTvl0t|Qa$!6 z=;e1S{tciskNpy4e^~MV21@tXE1;JvR?TmKGCX!ivTI;p5A63A{0UewiUbT+F-R7I*@`qnxCeWdQek z0@WI!vN01-<{T+WjY&kOoaYIoLy{KLIR?!tY3YWqITA%(L1q4h1Cu|7H5x@&>nYfT zw)oi*m_I>i79q+MagX81uo~gl+T)Pb2K zsGtD_h8;DcI7TB^jb4pK@YAKir1uzp9Ji;r&#`*7&uW1CDZ6IKTnfrBT4T@^MZIhzzSUu!~P}p^?MM=6Ul$u1p}>%CM$T%GB(DQYsblxFg`RT$wnvT3Cn6}ST%q}_4#~QCQ!}R zoKm(vpBF>27rjxcu`ek+Lq0@?WWZE7ktvRtPnBU}bt20g3T;8*6nPGBlR_o)`7?}y zLJm7(K9GiqRUv;l6h5VfQ)D4EA&E*sA$)WVqo9yXN6cs0FtI8m$D!~6H=H61nN14q zM11NEqo9x%9Wft+!^EnP-42D%#^DrM$SG22gyI8p7zKs+5G%xdf({d_LK-*}K30cQ zWFg&2;fUe0b{GYP0uNU^0_1Cb9{F8 z@L@ijA`3}`B#O2N^$0_ z+Z+y!6OlNos96q=W{ZeCc`|Frq%{7sDD6_G{p%Et%N0r&j;nBPkktfT%mpo1wH2g&3q0HwT4$VQ4II7Iu4v&VS zh&);5Pvmj_pb04w4`s$fBserIMdGM3TR1$LoFejMnZwBAbV6fPBp%9q(BaTT6^WzD ze9hs}comT+%lw=?&Mq`-MdG2%%MOReu1FkJW-W+CUNnP6AB6>l#M>k_4$NN9sAC#{A^ zG305JD<`Fgg&3T)(Up_-r<503+U&|nCsE1{Pa9r2=_8b~Ytp7yPP&d#b|TvN%1QaZ zI?+V5`IVENhSWv-UzkMQlg2>eqCqfROcfc{brjP=7%t|mkzr*~Op{@_m@^ojUnr&x zFz1 z6j{h4q+CJk(4ZPdK_MF*F)gfNVpYgKhZ3Jso+1nRofL5qC{IBl)etMh1Imk4AuSxr z)Cd$=$PiN451%YgK_N38@z(NURmigr@hl}MbJ&s};*u%wgmVS_8THM3Ma+Y2VNz_NE*~u_AmI+Kk!=mCW8;361?L)*&VOaGMDGH_?UrMEg zBZC4S(ug0DOy)WWkuDnaLnJcY6sBV}Y2cSq{$s_kmQv8sg2hz}NE0>})%q`!QiMpp z0FY^!z?53}5Ropv3<#5`#6L3eNhIbQ0!fkSVt}K5g@7*$LIrgwxB>y+8-xm4QxJoa z`3fOa(2s%y2>3Q3RB$H+wISe(g;2o*6kHDh-!X&=iYRCa0be(S3f`uGTIQPvDd1VD z=KPe>XuKofE~Pv*CFL-%&XUpyV(yZ{lT;KA1*-}qQ9bd)T~c_uio)SwH7Dh6{BV~P zp17iLNLYhNc^p67C50+b6b=h(8Y#v2;VvmuhN5t2SSv_5gdgsbLbWIghljNj5*J^L zNU>IjDM#kF6!T4q6bpR_^+<6PBrd)>kz%0>p&BW!Lowf_NU_j^P>U3|rI;^Qq*&-c zs6>i~QOvh3QY_A2s6&biDCX-IDHi80R3XJHDCYYZDHi7~)F8#%DCP?qDHi7~#*Fxp zBk?P8_}WHt#L1U&s0PC_A$IX)j^qdplyRsF!>UIPU-3weP(vAq3c-bt9KQIG9HEOc z4z*%f6UgD~AjuI5DdSKzmEs#K9gcwS(WHQrH%7f9Sfd3zf`K2uRz0ZOdK^bCzEzXy zcqJ3ly!??}vX~scYm3PF5E;S<4sq0TIO1D4$>Zfl%^n6KU3?=KA|Y8#95o?{>W#^U zui_*{p1{pa$2W2^k`%-0Pc|>7d^slshBcmXOv6`nk}UF<_XIO3CFU!;(EOjFfE>QP z3l(glfPZM@i@Z?5#}u$p_~y-|9u4c16Bm!yY$DDSM=a5>_-kEqD0~waLXpjT4I=Tk z+bdoMo?0#}gTnQMy^orHYIiu(Y-eWmUx7s_x`9>UpNtau2X2OaA-cNdCtAqp>&Xm! z-U%5zU{)l0fzjf&An-hl^R9d|mf3y4Xci!n$d`QLK0W?Eyswy8L8KTAIX2WuZ07PR zjq%D6i-a2!lb8v|Y%KO`U5eQixw_4MSaqx)?uZwII3jkZiPzGDLf!@fDE{q%Xvbg zN{ZvR5~bpl zl2U8n{kK_7YNoI@M~kI_D8_8oQ`YwWcgP9=uA}MRlszXH*(sT}Gi9 zt2bpI%LTd}dqut{uHj)%dKqvf_DaM5Im)Q%J%T8E=}R-On{dx)!>n29L*P9S|BIAi zc{du@IHxtUBLC5y;^X9X3Hw>>qKniLXWAUfO~qc(kDggOhI1zMJ%&;HF&yet;M+p} z)nWnm2Hj(Eik-|C*sHL`ol&QM_6E_ft;y_O>`_KsRmG{B79z5yv&%)>_0P%-(D9<} zI0cM^mMRU$WDl8+XNf$`CP-(^K-MU9q0YfF>6~*EV^t~?1h})v#?Ixiojudehl_XJ1 zj5x3AVGTp#s;Rg(WpND~s5s0X&@#4h?Ktt6-ic_J8-+u~Dl{J3BixnA1j9>*iYVz& zQAIjbRPhR5hHF#ofpd|{R}rt{?7*N#OXvvp5Fv^4)lCApO-Paq zYZ(ZsOj1R9L--%YaZ_)qNLWPb?RFN$(#gW2xS8^tzZ_y?kE3Qvog!qcLr@U*BYJuPq(0n!qi zUjexmo)&i@^zmZh+34FynU9_po=pOHeRQ?(yycTU(5rkcGLgp5gx)vNZt%6>W!&n4 zo|>+GEr@*lw~}YSNNxD|ZzazG0lYr`Tgh`!=If)oh3Aj}Nj~~pcn$~7BYldG_iUac z3AND4Q+@Qf@ElKQ36SQa%Z2Ae)jj~}zBfoanZ&OmW%%fH;rTxCVSp^@SadpJH9)qH zZWo@PlimXu=%e3-=aFZgeq!dT^g@en{!ta85~SRJd}FNy%x#VYp; zg5p@^ez6T;N33$cV49Lx<$l4eOJkM$#eRSTvC9480>H6Y<$l3GZJ&x&?iU>a&c-VD zi=hD8{bD+Rq1`WvfV|rMVmnZrcE9)cE1=0l&;+` z9t6tJ?iZXevb6if$3WTI{o-e!f!h6|66WU|?S9b=XpDBh=mV6e-7h8sX52xy6RzqkTasNFAW;1W=z-7hkLR%`c*JAl?{_lx_1inaU2Dxe+O{h|b@ zM7v*n2UM!vFJcnWblUx*G0-vXevt)qO1oc-20E+VFXjPV)b1BA1HpQz+%G-_@@n^s zKY`-3`$bJ$kCL?eMMt0%?S645P^xyncmya-yI*VuO4sfeUjt=m_X~Gbw4ZjrxE?55 zyI=GN8mQebrUK2Xs-pUkn4nA3D~3(i+dwEG1otU~R6!5OPayI+h& zE~~Zs#X_KU+Wq2npknQQ@e$At?SAnKP>FWG2wa2q)9x2dfevW*i~c~zwEM-qK&Q0( zMIq2x?S8QZ=%RMNI09s7_lwIwUhRI7l7jZr?icNVlC=B9aG(_JelZs)Rl8rT21?WJ z7w-e5Yxj#EfHJiEMTJ^uKka_e2q;^-U-SeTsNFB_2FlUy7mI+#X!na(f%3Hb#U7x1 z?S8?&TcF)9zK3kScE9k{M*C^^i+VtX+Wn#(P?2`O7zVUjyI;%%TBqGFo&hS>?iX8t zc4+sDgFq$P{o*1}sdm3etb_K`?ibC0j%oLcen6+R`^6-nv)cXQQJ{<3{bD_kq1`V& z0m6fUa=$nW6sO%Us=(l%q}?wX0i|g7i=IHK+WlfIP?~nXxF0B8yI(L}hIYRw0?N|v z7n^~ywfn_qKm)b=#RZ@o?S2uchxXI%7fpfkwEIOjpnUCqF%qaiyI;%zny=k2o&Z{+ z-7nSw6>9g34}gla`^DEltF`;ZUqI`$`$g6IXg}?K(HdxncE1=3RHEH4?guK>?iZ_o z4rup_5};$+{o+TUQ`-H)4+HvH?S9c1=%RMN=munH_lxmBUhRId04PqoU#tO2((V@} zKq=b&;ya*J?SA1-Mf+*@iv~dH+Wn#jP=2IAG) z{bB^rI_-Y35U5zYU#tS!q1`XG0hMU?3!+l(esLV=fOfxd!HjuKyI))fbV|ElbOJi7 z-7jdgy{O$UIC2c_e!-Db=6*rrY?=GTV?br@7ps9%wEM+-KxOV1M}f-RFD?R=xnER* zy&^-qU-0%B?icUK>{er+7tK2UL;$oHi zMLm?06sz1XvH((ImHP!HsjuTA0klDs8y$LB^Yv9l=yegzpTM+ZHO{SI!g);@q6Al%k z35X2qb`V8f`7|a|NcGT{!p(=b&`5e+i1X%yT$w<$m0_)-l&RSPrBo{9T}QxsqcVX| zbu`|kIO3|fJqi<>j5Dmw{Q1k_Wk%pJ4sYh5Vyg}gMYHjlu&kDb)s+G^8=nu$1ghCa zP|DWl{ai>jL~oR8r2n}%Lq0@?WWZE7kyVbEPnBU}bt1bQ3LiDYDe@e?BZal(^Jf?Z zg~T9Mi1|PoCRT;e{ajG^lp0Qvg>)mO1q$J#YZwKEOmxJ2mJJiDLKZs|KH!E^WFc>m zLgNvidc!CvWS=AEV{n*Q6>`p@@Yy(=A`3}?#6`;#ADF`^D5QlW<`Z<7SQRqRq42Rf zoFWUkj}(p=K5K_jP{^~6m=E7!VpYgH4uwzT;S^cOF;Zw#<70Xl1%=p%6=FWOhly1o zwH*o{=EEtnkPf8KV8{XRal)YCClU{3e(Z2)4vNH4Wu9|* zG!#YT$ug@$EG7t=kRtI=W;=&N!%`%UDs!~Mqsb{EPnP*Gd7MsYjEclVnQu89ny4ai zRGEhy9*tKKd9q9!VllhWtQCoeG8;M^8oMHKRGB>;9?f78d9utY5W8s{i^M^BD=4SA zERtN6wVfOq(IRqWIp2UIZdYldkUTN%gxrL9AQ96OZEhi1@%GZN8c@oKgf_Tx(kx0j zm=?ltF?D2ElPRXjFkDO};m<@A(}ox> zrj`tA1I08dhKuE`_h%H-$`~%Do($_(ifMKX7gJGiD~H5QJ7l<+nlh|3NJKZFkx^1Y zFJCXE`8G`u_og=9J6x#h*GkkJlhbp(nm zWC1DchwqoCppe%b@ek$2s*t@7r9!RpCo2m%LkfE{Ex_f~RT;5DOp|b!SQXO5q0mko zPLYN5g~Yw6yclOOg(6y!!&Ayf(nl!X6G1F9eghIQOMibgMb6TtlyVB9wK*hzDgBvJ zjw@Q9%SmGqDSAAu(dDEKC}mHkb-J81i&A#MnUSQUC}rQF^}1ZzIh3-|Y0WMteV$Ub z7OmUmq&pDlrlmVfqGstM6w~w_E|#IvEl-jpN~BIZL;qnAY-eF|}k^ zgDIx@JX|bi>3b=rJw043XX!$UX0ruDHboyio2MXw7{FzxtKDlHs$QoutR z@k5fyTpvKBn+E+5iA+bY3sFrP_@(rmsOEMGhFVxmLJN2{!sen{{{Sg(Ba$xwWLhRL zr565z)XkRxVG@1O+$Du7P!tXe>t#|-;)lDWP#KECp<#VUN;KY3aF-OSMNv3Bth1oF`C>$hwK_~W zGOIx1=9>~J7WxqCk>VRE=BpDa7P=6sk>Xwy^IeJ*3q1(6NO2y;e7PdULI*-6Qv5K* zeA^<$;{1g=q zd~G8+;^fOXRD)sNKn`E#NRH4z8Hc(stUl!M6_4ZyHI#9v5L^h!;fo*15xOYjP%DP@ zG&y`7BsoGMWgM!;u(p%Kmqd~yv{J^QesB{+4qqKfj!;b*hf0DK9AYkCrn8c7*;+DTuvg_2$RGxO;G^r zG>N-O1`fq7zyq2r zo(u$(9dNo0-dKmnmuIqc^58||2>9kq3OIRV)H?#h$flaZtt)*P~VIpxbaAuy~b8OM|6D>_LQ`OABP*C-|CE4$GA_fS9% z-`<4^PEf#qIr2qbsNgaMY!tpcYXh7f|%hDUk8lage+2( zBwb0wdBk6@PT1U!qD%pI@ksp0N~~ceWgky8y3SgCC3iBkp}P(G7|33Qm&4SqT`& zRjAUGBu+`skzqYgaa+_pL!@a)@iT}s z7vRTe^#a}uMNcwC((aUPqzr`{;`MQ~(K-X=*}eUYzvd29HC^O$54m$m6WUGr8La2b z>YRyEMj{&902#pXqedFuchh6%dDG9SWEW`JXww!flsKS3S^&xNJgM$BC*K1%)lzxI%_2znEN7i~{Et14sMd@WRE z6`D!HO9!#Md~q|EKkHQUNmZOVTt2y}Qq_I34)c)5 z4D6x~_sKfE4E!Q?QHT3v9X&pLqR>K~c-=A^qSYj4%TgLHg=hMs> z#*GCYhF#QuhOGb7zz;HREO>Lw${Y|s0e^?R);`2NWO~^0_R_60D<8W6;0;>3VPrp_*sjT;uIc{W%EDT%~*CNm1go) zSx<+gmXa(9m&AM-UQV>F>uS_tlfSGKBmBf>n%*?TB`n2HSCrWad%|3R+i@@$`z_A^ ze5#W8)!8H=uI{t=nFz)N>@kNzw2cZHF?+6xfcoti18wL@SM}?H)MdJu0m%R*8L1@G zsY1575%WRFNip{ZM=_O*7t_HSu#-6$#+xlT7?Ro9a?NOVn1+pp8+XY`!$_HgKGInZ zh0msf#yM1@Wb;2HKj=8I=913>=kHTJ$!>9%ak+syQHwaTk117PSdh79AZjm5P zjF|3n#4Ja$#n>x62X-GhVvYjuXB_%O%EJjU0NZe|345FC0Y)R) zJxEjc27u#;J%ru!pwi56NT@StE9UW#Cg7YA^Oq`G63T{F4Mkj~5(@>wThH*^s3bc= zCH7LoZFt5i(GR*rHR__w-mV##BWg#P1Fd^dtfPHZFQdV`3H-efgTupcpx6X-+xQ#p0-=UvG zULCD-;_8@=xMtX^@#<)u6IaI}z+JE>R=tJ1rW`t(fmf3^EO#{G#BfTA#+@tn#DaM8 zvdGH=&c%NH_>uP-=i~U~iSc;-*vZEE6bZZ)zajmCxEMVID!-ml@gH zVyK#OsGbDg$G9U%>oa5coSznhyH4ykjO#oZjx8c?6!2ii-F*i>S|YOJviFZ;uUwEL z^kU$iL2R$2qg-C|tUtsxZzD+b1d3UjBMG`^MB7Q{WE8TgI#eDygZ@{Y&1I2I39{J5 zY#JzHzvm7B;9a#+Kmw%B_s7>B5d5F|vD{!)@ zAP4i$$l?cPGfNReY7;UixRex^0kOUBWK%_EWBTy&hS?yW;AwgYQ#mh5LA-*?2ngNH_?=wJ5U=_JofB zTH>HN_6#(ZQTJQK^aARP9mQWzBj<^$isy*;s?6WGN-`6Yhm@rBD#;8~zn7V2cwSNB zGgpacI^xOcwdH1K-ughy zTLoyQJnV^l_^>_@4<8=`Z^tg@qYvtH?n#NotpRp9AAOLYa`|+CAlirsBEZcoRQdmM?yKVl!`px zLs|UDOHABV=`nf-s)$sT@)n%AO!>J;c@U{SXUefxr4*AuiF185+7~D(Qj1My@s-Hp zG?Jgf?pdlxbHLIk5N+$Z#IsrvUI-LQyg9ec1^?R;G<&4CLJ3r%RY=5pBo;| zGM#1|Cz{by0mgsSL^k6%(TuGT$FI|Lr0>6#>tz~y>9HxotKZu0KZGXi4S5gjl_%XM zv&Q@(wij=4w3RPTl^>lNhTSt)5hsEWa;iCw#yrm`!o#5idE>$>eavf0t`mzpma6C_ zMvXR3p*uyPQ&Hd~7V1@LPK4$eZI>0Ast7*h5SlUci`zk`ZnZm?l`WUn==3_2^7s&W z&X#k4OvGkwvzOsGVJk?FpdL%GS9(RUOAt9Jq&O6GmvilC@?i+csSgOV9D-5Fb*n66 z5FXw)s%+RF@DyU2M*SK}Oe;jC8x&xXKZO*@kt5DdlQ1}{6x-W!u z)77MRH0ikz(vYi3pK8+kA*5MXlYY{qpF&74UQO~i3aL~jL}PodCe_lU79pgwSCd+4 z($El6JTw-f|L&SJH-yykYSQfv$#X=JdT4^-`CY{gFN>?+PjxFcRug9=(YC~*Ime?> zPuZfSW+|=UawNuh*24zw$dFfOT+NK{Y@^FJobd-&Fxl8?yu%!vLMxHcwGAc%v!~~2 z96KvF?8PxRbGf(|HG3wJx*FZ|DeQ?~V^Enr>oCO$;4c|h6*qZi&zt5VE)fsQ(by~c z1|Y6)^hU(>1n$f@u{3=@v3$cA;2i8#F~Jzy+o~~&>m%=_Wq~?R*s66!H z$n&lutPUlpiD5b>hR>8-Cl(V!Q300=rV<=^16(A7NBETK-+aNepEKQe;rW4K#XWmSBa7{T>1+%2oV#)t~&EaFR~?p9T| zFrp?SxITuvb=3?U`)=GQjkcm+c-{FGXCjQl z`WRGL9M;F6CgZR^2K5z(^)aZXIINGsQ;x&>7*u^6*2kc>;;=pjwG)T+F?hoAOQi1m z?dy=Kk2#Akk-FzrUyftzIsC*wQ5msZAHzLAc|Bv_Vl3CkXe`&qV7GBU<9TA>>P>?j<%);`$`_hG_bMdbvP`d!sKG85mwJkm24WfY-|fGTd+ZWbYFn zBXt+s=aAUPgyLhQ?%fT!Vuod+m);;3$Z&t*CGygr%)MWvHoWvFa~}}E>!m-L`=HF% z%LOvrhXhFSa)Au@;lN8spW-coZrn!__5q}Nxj=^dc*0KrX~3?R?T1v1_&f|^b-L6 zebPSw^!G_CW5nqNGSUDHy+B4EAg^8^gI}MC(+gxg4wR%9$S4L%(F2!sjBY>$dVvgnR%O0kAY(Dm61_mi8$gA6fed~drARN3aSmv;ULYd@7xi^| zfee1&q*yPIF%W2nULfN>pc1`62ES5Lsu#$32k3xaAmbR&F}*;>UqGky0vY_W$XUHW z20tcpQ7@3euYbTJNG*^t7s#s@$aoDXPA`zbZ+0Z<1v35sO3@2s)WZEss$L+2U)o61 z3uH_HO4kcyJPwqh7s%lEF|zam8OMRL^#U2uRndNWfee1tB1bQf(GO^hULa#SP@Y~O zgWsaa*9&BP092qC$oK_lzFr`MUyxX$7s$97s8BDEF%qaqFOb3SKCIRYWV`{iPA`yg z5U5x$kioAt?9dBj)T@s6(+gyD2P)MIWJ~}$pclyC7Zr}_1v2>Ugj0Hfi~~St^#U0e zfG+9cK*mC#6um&k%Rs4mfs7KMG`&E^NuYGS zKnA~Dkf9gI;MWSW^a2^}fU@-h8T=N(K)pc5ET9~{KnA}wFvfPiPs$GqEh3<9(n~y+8)n`9Gi+$cVlM?WY&WXasahFOb2-`p@bGGVTPrs29js z3S{U7GPvTtS1*w98Bm;FAmaj1l3pMqJ_YTk7syBlO4SQw+zOPY7s!|jl&%-Zcm^m# zFOabVC`&JpaTF+9FOXr?Li_0jGU@^4=mjz|fX3(rGKK=>=>;-o0OjiiGT3(u^a2^n zA)Buk$k+_DL@$tW0H{zekZ}R1NH35PsEzj13uN2~v`#OOkquO=7s$91Xop@PV~^Byf_PK)_Ih zD4+pB0tf^fd2JL$f(2p$qN1WGMgTXtO%!SlyUy$){NNx578TUh4W?zu;8l)BW1sNM5wc8hDbnK1(voFXP z3~9A}K}G|lHTDG=XF^(MUyyMVr1ka%8P7u6WM7c64$@Zpf(*Wt+p#am;5)dJ_5~Sy z<#w@sK?dKjov|;-;0v{@>Uz8g z0$&QTKPlf%JbuX3o`g_=_dPv48BErmVH46e;3(oUy#9HK(^Qy zWbn6+3+xLr_=Cnp_5~UI$>C!Af(-tyu+_dGgFhZ@voFZtF9es_7i5frw8Fk1qXAO8 zeL==akXG3jWKdddUyyM%q&4;h8GKdpI{ShQ{yuFz?Bd})!lLYVNSo{nGWee4t@Z^O zd~&ic$l#NceL==RFtabn;1B$=FUU9)QuYNIr$frVAmd6%*%xHo11bB0j29qfUy#AC z&-K`!l)ryTwDL}Rr*B8hF~sGZXs8!t#IZjqC4Ya?DPTGN{-hM_{-hM_{-kT5$L>%1 z2L!u6DG|FrX+=Ni@%JYs4f~UpLz{dBg)PN}%abGclR$A5f%&fEyy*jOJwTVUNLQsz z#TAwfdAippbIznnQDjGaNZ<~Ew&JeUfSQJip9m}hz`0Zq$cHS+;k-%%tcA-$+&`hr z5xq!ZB4=A!iX+AlAcr%rEZ~ysh(@9;nNzU<%37F`&mx7^-~>%mE>n4L@c~ZO0(of? zyeEPviJZMD)DdfN8RrDf$Qvkg+NN{sjDI3g>Ek?01JsTQ_V0%x8XWcSWC`7Llju+DIey9Gz1%KsVz)d0%xgVidOOu zOZgrYwS{R;OL`!qf;pEBm!MmswlHN0oZyBjTFKEYh2A5zg=vnG3w@Z=;BX0AiQ2-H zC2(#Wrf4OvuoSwcl%F_9$p#JL<*l6&b31n^vQT1=HxpB zn?6xnn6d=U$ioz^cgD3hfB~cQCpa@1Wx9|6s=?{OQFY3ZDE?D zqzf`Cm{b0630lc8pF%%Cn4*;&P72+DbPj|lFjtEY(_0XNjk#|2DRd!(DO$;MK;rZ% zgkUJ(@3-T0FoYmexC10kk3$&N1#|mS^_0^M5un(Q2WU`xnDW`Db0UgkR2}J;MYlyH zPwTvxJU+5?XhiX#^C6!@H%Ao5=zQPj(E$?4(>nhqk54dNBT+o)>7=o!9vsx?Z90oQgU(}p4qa4H9HaAcpGW6aBv0#nlst|ubZbTN zpz|}ILuXeM$LJJ*Rata{Me?-H9RbJb9E;*W-vr`xmqp=57vI$prxPubqxD<^Sbbfk zi$e3%uoL(a!q;@D;fX%CKvDd9>4;B>av-4(E(iUCC?9$Hco^2B^qYj$_NCcpN{#V)l>U)0z2)KDw36PwG*0(<7}ldS zw=a#;pB{$wD9!Cl<7)fTkZwInA4`}X_i%1KN?!y-E3z)RbD0A%=15eE>BrZo_HcA3z)L#u1KD(`!;l!K zLqCA%d=1RUYSO{4QT~p_5!~lgmI%k8uU`YCDW8k!{YyyUT#5?-n!*gOw!-~LjB_y{ zgqX~)F*AQI!4-mR`HoNYILYA)p;JQH&=t|%{09--{0=p9U8~|4ng1}e; zKLFsuLJ(*o@EZWGF$9761U$5k%MC%`Is&vZR~|ILR+;X6nCN#n@z5xnW>7u`;w@4N zv2egcQ`ksD;iDk7kWvGfho-Q(hQh}|lmn3m;>1H!*lLq;M1BT*q=1FzqS8(v-EaEV?n1ic0SJiaB zB~+a4I(dh4BgGLzkf^?@adA$u9l=*@s81{|(diOY`T9FSBT>p+vI~}f76Ed& zdKUz)CBUCKav?7WEF-{9;rg0OJ0dj3Py5ABbJZ0!I)Z2MsZZfboXVytx_ccYn*B+; zzu1|pkpqg=OEdO`kl3Xsubt$M6b30F=?K1{Bhkd9!7*NUKMSS5+T?omi0VFet z=U9W}oj~`hoC2_7KU8S%q)OcrmtzV>vo<{@;8X_cz+t%P z<0S4!j#Pc@V`LG1kRBh1ldEi(pxZd_vCborQmS72aR>59w(2mF_TWjiQPks7Ovg!~ zrxM@^uV!W%OS_6`BoonjC(8jb$eD5PWbym z|3fImhye)cRiU}}6>&s)yaR;Y@c3wb_CtlZ-&X(7Z*aCBXZD%8|~uL56}c^tTx0=NlBho4M6 zJw}W-Z!xuXx_*bNn{B2?RHjFe;XxdQMaO8~3=ljO%ZTiZIw|ejuvc z>CFyZZmQz`j?y15`~pQ}en5el^}iabkk&Z9%q7cOd++%B46mr4X*i#kDv(tly?2BrMA& zq++|hvgTS4(XUP_Kxh?qI3c1*lI(Q3v_d?L2dd@Z@ql z(x3P=N8K<;h+`XV!qxcH`Jqp*nGSSsP~$Vg)i^${z;QQ@P9|K9Ik>OscT6+kYV}C# zQ-bdsI7&>o8b7I~LY~Yt6RyU*&5&o|=(-$&v#WWo5w6BT;tBwl;V7&yz5NR6Y#)ih zP+7Q(A^bN;up^e!VGo1b%*qaZ+SS(pe120o{^r{XOi~rN4a!?tfwv5C9}sLA!GYz! zhHz|2tC3m+IH^`jQ)2vbkun=bx8l06FG@_A4WsX6!fs4pKwP}8^hze|&jhn!^m$B} z!UVHn^d(FDxsSTr3X2a;$k&rSQMyqSHVYIq78%C>Z2E-+fEl%LWH#H3>Y#6Ps z&4$tHngMZ%<8`{(Fj}QEATDvDPG>+IHjGy3X2WQ8Z8nTn*9?eD%qgMXomD_wqPfH8 z(Ag0IaWq&*1jOyn{|Jbqy*eTwj%MnJfH*d}BLd=R`i=;QqqRCBAdc4Qh=4d2>ZDj| z;+zknr3#3vQ~`07 zDj=>>2gJc1AT5e@1F07_j6NU#RXiLvjDDDyiig97(G+w%95#&Bk3-6A7(Eer3YjtW zFgxKa#K*z%+-cM3e@o?BMnz?m695#&Bw=(15uwk?+SH;6&!)OXR9u6Bu zC-KzicsOhrJs5(Hhf923b^rt&5106=Vg>{q5105R(*i-q!(qedTOsIpxWtCCCn4x~ zIBXdG5d<9%hYh3uf}rE!uwisrN5{m&VZ-P_5Oh2oHjJ)^pyT1NVKjfqsN>k@ zf>dkc;ckRfXXD}8A=TS>xGy1%v+-~OV_}1hhua=fqm75Fhty=_;rMFHSvDT-Vo1$4 z9*!?$Y_aihuR&U1uapIvWpn7^L+!9_~y?n`}JXb&$5&c(|t_ zVT>^GaNj^m+IYAkd@vN-c({R(GBzIW5J*)v9_}1S)ixgP4oEdN9_|fDwKg8^7f5wB z9zZg)uIY&;z2&oj;^0Z8xQv=q$V2=_Zg&FHXcr3e${N_;pl5>vGH(p zB`mP?L0W9%;a-B&YUAO)gVbi@;X1;DvdqTA4TH48#=}j4)NbS9E`YSk z#>3qYX|;`qdjrxM8xQvrq;)nPuB;OMXXD|9LfT~G;U+=aYUANfh2+?HxT_!~Z9LqA zkcw?Q+?$XxHXiN=NL4l-F25`K&&I>;0I9~t!;OJdYvbXjL#ng!aOXp+xAAbdK^kY{ z;hu-oVB_JwfYfN?;oNTMKN}BM1!ziaPv?~n~jHSg|y7Z!#xjag^h>%0#dt;hx4{W|JiuBYDlYXJlq&aYivB+ zv5?l;c(}_Tt+(-T_d?oaT8xMCBqy`%gcLAhE8xMC2 zq$V2=*A8ixjfYzcsoBQE@!oB*@o;|uTVUhiy7WN**?732kQUo`xJi&&Z9LqmklJiK z+*Od4*?71IA+50Sa9pu(xAAa4Kw4$v;R<@9|7<*5Ur1|gJlsBz*4cQtV<4@!@o)d9&R3_ z8XFIH4WwEd5666UHXd#{q=)yBj90;$c$!*%G5{$Mmf3i?R!A#sJlx}u+HE}C+mKe- zcsNR{Z9Lr1kk;6EIIgL!v+-~vAgza8JcsdcTprtG2J}ED^zuM!)She zMm${Mop|6=t*psAeP?6#A~xbQ&~6w_Dc@`uJr67=-)tC7A(_t!%~=UOp~*KJMh}Ia zOupGL`Y;Gp`DVjtBGvf}DY_t$J_ABczS%IEv|6XI8QLT#jxo$nT%NG_4Hbe8C;26S z8Xs_90O)cO>8iA6afMYEoJAoxip)8aDn*g;BMkycl*jp#2GlH6a06-ilfeBJ0=R*+ zB!}}VorAS-S%}LSup`bVg^8SPWhsu}2GWuo&b+dKORgiHCCZXH6$_xOg(-P0DYOPB zXqvK7<>dy_3gBcdke4QLL>HnYa`vWBN2m>=IcGES{>gr=Y-i>Vts=EtIzpi7BJn`Oq9LP?{k4# zs=HB}@fuRtLQWzB1uz#H^0*IkRvCg#L+B_{C2*=4rsy{OK?-Zh`Dcg%CEbv!U`|Lw zuu(F?r*M`Urf4ONr0fqRoVtc6P;#yhb1oZ#jgqB4g%jK`MJsuc6grPM>kUz$WW5h_ z8XSU+k^~S{0_VnIidM2CDJ@XK33G@7CHwm@XV4+oD4FR~IIRv-w33CS@QLAEJ4At! zhkTfm?+|R1yya6kBM(!wk}agrr^ab|hyo?$NL4WB?IGAG+100TG9RXBC6h^^!;Ulj z5Cux+`Y@;bA=oIn-lxzH5TT}V+p z=$zzp=va#47@cSOJi44Bd0OYKFx~QT!M(0MKN9R=}PwOlPti}eq zwW4^?Im+kI*%iexI*;;sbc03mw9ZQa$LSo4;y~ZS#OW@J!i}zX$e|N0lB4zf3Xb}^ zN*9IZsbMGZC8P@wH9XPh7AT5eFC8(GCNgD4+P`si}d#YFiS z(Px)~K1!4~JbidM=(|LDYtpBegZ@O6HxYe&IcPBwRd>_pmxJzv#5nzbA;f$q-H$LG zf?=2@a>Q|j=^+fmw2>n&CQO%M7^aaNaTj6w5W_I7$KdDf%Zv6g(5Fe3%Z)5Nwou>r?2(3{$j{JRs_(q-!%ofs+0{OkZaRHcIyM zDRh8_DO$;iq_EfM6%A3Kj%&*!Lw>ITR@Q)rSY=!bV9M z5LLpY2#QuRloa(LkV}D*!+dy2E^L&X=~JGJplBsGk;41%<6H`qJnO@M=fXzGI-kNOQ={^s`dX&D5F#YLaSdY^85~hPa4C_&v zL2Ge(+{3UQrN09#|3Ec*ScnLc)13Ntu@8%RsCr)&eb6!Eio=G_aI zae5!K`CR63L=(6euL!)e(LHQVnfux*(6Aw*cBMpU*f|y9k z6*%$G6gJmT_&A7WQkLVyLsQsrL*XMKt|#RKoOoyoO<*W|EX32K{Eib3O`$Oig^z~# zoRqFuq2Qq@G>f6|@enFxjSC|hw&rm4li3GIoGT?7R`yWl(eU1cxpbmoWea5*4NoJ? zwG<62J1Dbg_&mZ~T+y(yfij7PZzast77eTRD|2Z0Il^3i(XeW}GKGdeC(QL24Xbu5 zGicZW66XSqhE==OGow!WN!*ScF5761YJ8SMGdN-oa=4hIIm!ZA4sC&V7Ldaw9?el^ z$Z}{9d=Qeug&)mPw#ag56-PWk4wr*8M;Ro`p=mG_ki$hH%~4j#a%dlX2_lC}N1CHd zljYDz@PY%5PqCb^eE{GcEWCk6k;t_sU0BFSTwB0MNDfz^G$&*vJz8E$&YPAKG7V4pQ3-AgKew@V& ztn&zNAgv?S)TD-=WNMh#Cw)sY#4XOXHeGgz^Dqj86CALdhbtANrpiiWM~~fs#E2+lI=MCg2Uy7d#!(RQrkgQYC=bWC?k=; zDPE3~W|8Xb@^@D(8>bE~{ut@Wa!iR*lhV9)k~=VKLlFy0CLed-?a1QSP+DDb1!|Bw zs+iTUPjZTzI=Yy38<&*KGrh1C1vDi0WzkKAtU+V)Tzqh+j!{Ar21BHEISyp84#lTR z$UHX2mZr|+^~`dC-++tnkWiF@g6&H9y+0LmuI6)u#6hS+l}X%c!qAHNhNSK$ag(38 z2+STPalM~-8_d?zB(C!l&qMZJCUK3Q$d?ywZxUDgi8I0MZ4y_RMA?dZeU4hYzo6DG zrO49@KHwDXg_iA5^b7K&V(q3F@c=|sRtLnjWXp1jZiCVti+Uk76ZBZa4wn*1N1FAOyyhV=i33I(lX1>=M)V@b@~;ZuZV3v@d$`T4vf&>Agu=J%AKs` zkp*n%N*3}78^$X~@JH=~?}76+e@9XHHo#h3WfAw1mKZw zj2WH)@Gy?VT%+$&5NymwAmSu(l_A^{BiKMB^Ss+BVcsoP5F-XX^c&LGGi;Fj#7TA_mO-45y;IbE1 z`SB$zo`0PQT5vt zRd0*x)4j@ZHt=(9)(N^c`|?v9f9?%$?+SCO@YEmFO*RddIvhvI)3__e59RzozY6mC zI0`zWN@8On)3|2;2GdRIf$mWcnUwEjrzgQK95d4E!Q$3q^&w7ilTb|=qqeoeKa zU4am#((up_{ z6^8gc2!Wf(hR`Nz4B?9)!QMp0b$(w4<7b_U7f;lB_t5{GQwE!xc%-X-wQ&J-pF{oc z`$P}|`t1gt@2Kgih zNf`CNu$PlK#q`fD+vH}H=md9wlepHvPlRBn_@T&-t$q|Y+Z!#%>u%j(de^;S zF>W;d*uAM9yIV`~njjn{91_HvYHVK%`D3Q38}>~#w*LkBe>h5ybj0cuzr^!RygT72 zJ#qSUq?PbNJPGn~Oq)6F1f+Fg8h_ewF%CW?Z>xvoO-L^@g<Un;V3@L^s()$;I7H({XpFb&C`$1S01zSHw~3I5`P&>2oM69 zA?WELW9(_8@bM0Hgp;!X^GW>LfQK4Dhw!tH$Gf$R1Ll3uz{|pZ_y&vpY%#z%vH~A~ zNK1EzfkqAY4RnGseicCez#}o<)I1J?z2#Eb82-fZhR{Dq&>tbG*ezRBCGkL9VrVEs zKM`LMO^czg)smQBUFW%J)5+XntwDmm5!F|yo(8v> zc`rB>t)|?es&q^h(ONY&x|37|YM{IyEAXHpx**sxf{*?yhR{7o&?cv<;tvgOGqYMw zrxugU8T2Hb6+?m_WxvA$1M`V@M*XJ?e)x|AuC+mIr{gA1D}q}>1>CaNkb{-21Wi?0rP! zGuB?s|HVoq^n8Y?NiPsMI&%B!SP83$78qH=}e!< zbR8U@9EcAX>=vsk#Q76;i#;E(xEQC-+~_08;CSp7dpZ)5435WcvFh6F7OSq!Zn5f` z!SUEF_F?iF9FN^%)wS6zR$VhV9=pY=YX-+-w^(&;c8gWl435WcvFh6F7OSq!Zn5gx z>=sMC{Z(*0c8k3XIvE^KgE2UsmSk`|?Zx1Dnu)>jY%+u6X?h07(^?FUr*#+{&qAFf zOT})nTuLZn$;rhXk#K&OyKyZZz^OBLi%pu{V%wSeB2yV0zf(3i9=pZ_9vTDYIMb{RC5Hw^$0v)Z#ekesp}2(ec=yeugjp%GTkOvenp3lJ#%{45dZH!uAdT^-5*DP) zZn2$#*y#9OAlT^m{UF%rcnUT;{s;&*I-Y`!jz1fMjgDs?8y(N0ZFD^E6dN7?7z7&~ zzZQaxj!&SkY;=4T1REVc5(1VVreU1u?p7QDDQTnQTObwN==fV9Wo&f(OOUE;bo|$l zs%><95>H!=jgIdNsn$lv?*pmMM#nQ?2(FyjsKc&q=^{wuY;^nskQ!`s{M(QkZFKz4 zkeY0Ce7Y39WuxPVKx($p@rOWavC;9TLt0>?<8Oep$VSIM18K31j{gi&tBsC#F*j_p z(eb??Ewj<_dq7%YqvNMTYPZqx7eQKOqvP8kt+vteuR>a5qvL;sw9ZDybGo_SM#on} z+GL~S$3WU@qvMZ**LifwfKT1Xii9iPD0^(q@3zcZw28y$Zjq#7F? zeuhxV^N`lt==g6SZL-nvvJ(AgqvLx( za%^;b9i*glhW zo!<@pXQSgezAUiO@f@!f+30u;}=3| zx6$zrL0V;_4 z{s&0oY;=5acl4i)j^71Rqm7O~08*2Ujz1pKEE^qvDWql_9nX8W#YV^94Qzppj$Z|7 zk&TZ32GU|19WQ&J|7>)84@hk`I=&9lG8-K~1=0!|9X}URyN!;&4$>+c9sf9_)iUPd z*4SU9*FaihqvL;tw9ZDym-R&d+35JekT%)q_=6yAwbAjjAvrcW{z^zm8y){3q+%N# z|1PAAjgJ2TQk9L4FYbl@v(fQ8LaMRR@q0k3wbAj+S7)Q+j{sJ0qvOwnG|ooHUk$0j zM#nz}snJHqzXz$wM#pb~G|NWE7vjfb%{Dr|H>4ID9X|@v0vjFQ0BMnpjz0;~VjCTQ zF{D--9e*pNHX9xPB&20FI{qU_D{OT9Uy#~ubbJ+l$Fs^t$L|4YwT+IS4rz^zj=u=f zIvX8-FQoN0I{r;an{0IakC3+7==ehZ3dlys_k)zQ(ee92Dz?$_$3n{3==h5uRoUqH zHb~VrI{sxyH8wi_dq}l5I=(|S`p-tk?*ggbM#oQrG|ooHp9!hKM#tX_snJHqKM$$N zM#p~(X_k$SFWv$DXQSf>L29wl@ePm`*y#BAkQUkK_`4x3w$btLKx(zo@qa>Uv(fR} zV>-LcM#t|DX@!lBpAM|?2G=h z(eZsCZMD(yd~&kU@qBW!(eZU)W~1XLLdr(R9|I{H9e*LDY;=4pq-=Ejx_Ajl|Ih1G(he6r3V4lm9sGoXuH)yj&t%BOdL-(+s%nw7EpcV zi0MR`n>V0FwL#AF0e&~i0?O0~L%7k0Kgxw&ra59XPfz+hfzQfNF)q^_u>>X-{GoyB zHqHsNwRA)QWL16653_*jw(W_s_xXJ;P`tVuwHfyyg)QVHGEe|>p&`?Kn6t_dY#PFq zcU1zXnqi7=LmMeHGUuNm3Y5I+!<>+YV58(apTb#cn4*;w0+GMLWSqK&C{R-4!<@^8 zV54NLPvHbNOwmeCCWYQ3&U!->DEYS!a~d3ijgsX)g>&ODMJxG$6uPE3VGdEC9OMsKSY6&9ekKm{t#@G@Rw7n1o{EO6s?5I)spT& zItM}&D7oB+=`9GsM#+6Xg)W3JMJrhiBu<|~2!;Z#e8=fv2tlUsA|P>k9Kx_JxIYl} zjM5DepxBQGXxPu^&^ZysF{)eue#9Ljs*FQ9S5u^Eq_AL~)GH*L)tG zGLby3a|?MKA?U7&;z4I6AjP2*CyHZq*7-cTcOrROXA^lJ;8~{QCyEE1i+m2qbQQ6b$&t~#}B%YqIl5Br8UK&V=0PbbXNO3x|||;TIV?OIGoTK6~%+jW}ibB zRTRhQyus(uc@@diI$t1F-LROK&FCp}~1&ZR=OGkW9lmiKUa5-oS ziRzK3Pc8@TLzE9EeRMe}e=DyZ5&G^)665s$g%I&r$PpPJae4^D@NYm7$wHVe!!S%EIpSc#^dW{} z4mOTBl`x%(VVGue#MOl9Wemf#6W+H#m~O`~OhY;1ZNl_JhGAL?zt;q;ZUZ_QH6{4* zRg6>x)9V<5&E2}QPoXO^Owl*%1X5@z`X@saJQJt-Fddd5*eJQ$r_hTTrf4OPkwSaZ zwHcy7$wxj+UuOt5N`CVxbby8_T1h1kb#K!v8lphSC?BSqGz1$Z{DrzIfxgo)MJt(4 z3j3Z8)er?rZu4P!SVOQ;^1M%xRk;+cWHw@ZPzwQBv(wj)|aXCHs=X z`*1-n1xilv;d^pnqvSH5LT7Kdv0BM8Qg}E2kV}D*cYK&G;cy9h*7%c8p`SQR(Mn2z z#1F`Yp`ex!y~yF5y6^)D)2$qab-^bCQKK}y%K?fWrLQ2$A&B1Q9P|OAe6Hwy&Ou)% z$_Ikp=p6KWq8xwdoz6j1NL05Vz12BrAELZ>=)KNC_ae$pr#CwXJ&GuMi{9-VbUqT} z^mK<1GfLk~n6B?ItVijm3DXxIhV>}@DPcOt!>}Hu|0GOrc^KBCbQd6Ty3fO~9;Js7 zrawIl(@u_RjE*C>By;)oUk+X?)R%L1e+pNr}JC8Y2R4Hp1(US@E$6`n+5oQnY=#AN=6 znV&&st`KO7&i5OER{?ObAP97XEI$Cib%P+#mjHia#wCOxuqT0y0JxeE1P&+g2LLWC z1c5mO_=9e)F$95y1Q^fD<%S^8Mu2_Cl?M&5RmSko65Wau4~?>E2IXTQ){@c%%LqI) zg^e^6J_9p(*Nj!upos;~=&tPa~dCmxzY6Br5~3vnDN zcjClDQ&fnmR>Vg`Tu#b=aN?mUD$H3^_;`p1fyB8mqG4+eS3jBW66Q*YhLt^(c{IG4 zFqcj=tZbo7qv4J~;#^D7u(E?Pi-rdf=HiNml?{|hG(3(lS6ei!+ON!^;S&jS`9;I3 z?aCAyzML@EV>GPVt<0d|`v`M^M#HMz>X}g|ZJ^i4;j)e9sK#eGG=n2Hk;BCt%~2M} za%c-jqyfjd#G^UN3|S5hf)7G+xbUMn$`)A;t%A4iki+F5%~1x)a%dVyw2;F^BF#}& z$#Q5Pd4CGRAMYi*Gn^^-pFDqt0W&Q&pC+-grqWz> zMU5sp{(GOomAC*!cQ5xfOfouNF6?|K@{|XnGKQ!JjGfhsyad2sQ9 z{`iF@=XA+QXOGFSOP2eE&5sI%lP@8^Yf*4HnoFM zp`c$;HzcHD#XgbSkQUYf>2HvvK?X6>Jr9xY+znGlx>uc#eoBqtBXz8ws2hz!5F5`x z_dFBmKA!}YT*W~5{5s~ko{4AULchIzSH`(x2i*ISQpMo0{93Y}C5vC6RoPB2zVT{* zx8)3^;ICQ#L%_vDHHf%lT<-yH9+%Gfj?*zXDvz2u8);=6m~Vu<1jqIaFg(EJF!Kb& z2XJ)y2KWM$(UrkrTbc4BhatRLA;JtK%CqO;` zhidR%s=-YVkH*pIV>Iw|l%X12uNzGF>lru_wMJb%2$2mQYX~!fglvP4F|ciLanEg1 z#^X7Ct2?69^yiJiq;1^$C{~5fjMR;DCzi4wN9ty|2lZeNj#Ldo=y>-#aLewhq)n&m zGCW&U_kpIUA=?x+WSgRf*rMLTW4g1F*5wp6(=V!Ko1$vADXPXVs`vRQ>N$R%RGp@4 z^OVs4-5Wj|jLuR~=+CRW^$r;Q8XRRR3jO(VesKN?c|8stN&aF1KfMpdGdC7T!5oah z;-x&6ekJ4uOjFV2FRQedAg^SaieO)*^8N#PGt*SG`YVb@w{JfjI=cKn1^j%V%a(6U z)lulLsde+>-oJ_T`xKDWwlv)+eS?MO!H?YH;PVF$mCIITb?XJ`L0@X?!#itu3 z`kuvI&M&kR1{rp5&E5q}x^5GT47(2q_AY1GQKHo#qr*t?Q2(w#Y~~y^C9$XJ+?io& zVunF3$U){Cq%{Y*-XJS;kVg#iZVvLML4M3ZzBEV@w_CP>^HA%*4ANs81fy|cC)4!d z2APeVDggCe101qV78@zP#lVc@&+4Kvi;T%~1MY7y$bF$a3`0i9>yrk4IRrb!?{Cu! z44{P~r_XrYi?(&1oRu1u>wx#N68;8!{9Y+br6tPr&He3pM0gf_t?=J1m z4~M%U-$WjQ&u>lf`QxiUJ}}ozHs(c9L28bHEZ9%XOPqBZd7nH zvZ|+Qx#vGHk05y*ETBtVsnXY>w9jxPt~IKAfuKuu63a~5NSkI1-0dxcfN4I*hMSZV zx8d55^vMQZY~cT6Ncya7$tLVlm7zXS)aK;9XTmyj*(;5wDkC}f=zsX^A2JlySEK1X z)xUiE?s^=$f9I+Gt*XKhful@?n$N5H0J(RAJP-$a>1@?YCqSBxLyurBY6QCm@})S2 z9#G3Y(#tX2ou|6R$vkbUl5q6jNnj}JUzt?aKiKJX9jw0w$=dqYs`O`2+7mbuADH}0 zL7?^dwESq&p0#PTz6~A!+rXNxA^foCY?It?B!2m3a@x@G>kQZfcG8{9d?pU}^q+Oz z8A6V?f(&12$mUy!8sz%)h4Jg0*_xS+8$Wk9-|(YT{L=Li8ubQ_vW25zrO~A{LMgK5 zuejP=9%u-(uP>Y8L``py^1)rJ_r|4Smxee|AKW=B z_p^{6V_HQi(}p~Q+*=@jgTs#-=ParpzJK<{5sVw>3e^Z4oz8O|=d2Ot$wS;YpG2-{ z9fBkA%E`JyV?eMC3VhZizBGgxL4tnr7@%Fe+2A%a?*%6_%OvA@Re%533QSTJI0?#+ zX9eaP;;LZz!AF0IA*>G)^pHkX#dqaY0M&BR<4iIqq}^a@)GrV>Ufp0|KJm_oK-~C8 zCxYv^$=y0);8Jnp2V;hDFdK0uPk3ix7ybWm{Lg3Hcua59p3xh%XOe37^w?yuy4T~B z(P7=?6L4LZ%;+uIW0bnjz-hGF*j+ex;JTW(S|bjpnsMg5lGThepMtAo^*G!nyedYS z$>Aj^T5shJ4I`A~aVz)Tb!2(|nrh^}q5=bmlTQoCD=Qj#eIoqE@+y6u-UZiJYYyLl zD6i?PIG5Mq`r0lk@3Huhk&7!vGjC_4$R%CW^`n@d%bO|{=QQ-QZ0)SlZ-zbyi#2sx;?I+(|Moji|ZG=tMr-t5F6d8tBdlvhUD$vX)>=F=^h_i_#E_>U&I8n z4LmcI^TRgZYy-cX3HLC8k?x716*Oh}lT0w%z~9G&*O|ab_r&muCz$XV6U;X7jzGdj zBxLk9@cCvN_zctOXwK+J_k2dWCq`AM3Ng|h+rTrbJGL86HH>u6H`~C|VVZBYfmhdN z8+dhXwt-jIjC98~@CIfv4U*7?YmFHt?fj_`ZZlcN(k`k?yo)B_iEvuS!I^(@d3!bZ3(*5$R6T zS0d7#)~ZCLJFQcRNOuu6H`~CU%+%AE z%1HOYf8hpiL@C$={&plv73VHhoV!$U?o!9Ot7#)Pfu9S~p4bFF^#v{p80U^%;Q3HF z1&njYF7OnR1&njYF7W!<$v3;eKZrc#%$UzOckBZH53Xa+pbC50UEnDdFwPyjz%#c~ zz&Lm80#6}Xz&Lm80@IxRp6fn*m zyTBg;p|OB*?$`x>HiV`E#<^n`_$wgHDqx&Dc7eYgLURG*+_4M%BM@2&80U^%;3+I9 zV4OR4f&UMLMI}`!eCD|6FCi>0F}uM338A$l4+X^CatduFW*7Jp*l<~i*#(}$ijoh} zxB@rBR~WXJyp8r0={&1S%r5XOdUeSexQDR|d_8DuN>(5VyTH$Yu&%`H0>2Q#`jW>` zw+grXF$kMV%r5YJc(;02!8+aC%#RQp*E4bM>a8BG_YTr}x#j!`e=#KVS8uocKu8(a z>j}EcORO@;pQRYCmls~4g zb3Kk*gWS@t_=K)^z1hGJ=UxYCoa-GA3k-Ej`HT1l*ZT+9aJQ5%yK8j4qaclNOK*qN zRh1xN0n!in7;Cl1XN&j-w z`$1addSg(^zHa(NNQ+?!bnO9dDSu$z>Uy`K)5f`_4?=2ly?;TP;Fi7%X_@QAakox% z%Qiz=;d)=8W|Q2qGJFqdcfE69*2!+!P)Ms>uNGw<=9V`=TJ3rVqqnBGoze3vTdUIg;W8LyD8R!*W z57>I9TfPURr0_UEoamMx2dP+igYY=cb~6hhWrWucH9OVKJO-&scssx@&2DBbq-x=P z1nqO((pUxhPk0PsIm<2W3#nFk@1fT7-O>Xg)d`PL73aI9r$MS0UVqs7Lbvp0NaKWe zE^2hKTlzAj2H~xMbuV|zzK7H(yh=P$ctdih&gehktwtNJa?6H8nkBsHu;;aIdMc!5 z;oXf|FLBcsLTVA-H@GKmaMN@}FA&}+Fk9WycOWf7U0}hbZs{M877K4Oq}$!np7@^8 zD!h59^<8e+7)WixI|KLUJ#N{Fkd_H=Eo^qbTXrp^6~cQO8Xj`XpMcaZyv?{nmb>L& zLRuxf;lLhq%kwMIf5LkX&(srcx-X6t&<}6A zW%NU4V%|vD>@7FF52UJ?_dTR{+|njU)iLjL+-L8(WfwrIiFxOsE+4q%+_|YX=8Z#- zedMP3a@o3=*Bfp4#7%Qwruvw7J9_0aH|=&q|DjA+ew~}9)371tnK<`RkQ!rNIW&Cb zrn#q5Q_QP^hHu<7w@;cC^Jb&9>)rGXkeXwjiF1DfQcKJ`7v>q z(u$ZjAGQA5E#>bl+hg9vC=)C5M?zW^^KQcZB1Gx=kXFaM2`JMOrTnqvnwa-7M{iNe zSJ1ADdH+T~q(td=kk-e%Jy6dAQI@Jgbz|PHU>1q8evr1tyerW5Vo^30k|VvFaffsg z=@TF&rS~dIDHZ7}AQek*PxO1aNH2$!kzOIB&Z6`kNLA8XiF#Iw(qAA|OYbab=q5@l zx}*Q3cRg&~U6j>As+Ha?xRZNgM`cKL()$24+g_BN1*u+od|7I>D8CWXIO*+!dtygX z-VUijde7lL>nF;;gw!a#uCUq8BKyR4 zyDiddg7#Wb&e_ib>G8|UFp+77v`BgfyKohW(rX|smfj1f*+@~k98#P0#Cvsop*r@`D;q`7^~ zYU#PSLlEcA-D=iI?>N|OtVnZnnsw5v6x`lcwS$W*(geHgjD5u zW6<^^MP)lAOgXya9z0s4`Gc|=&sz=+Ger4jNVT4Klf?SJ&R6GoCxUsL=u)yhn(2A_ z!!E~*G=Dud&hu8o5+{jL{$#Ad^ZvnIaEhp$0jbgRHlkPNi1PCxHF@4b*t%JyZ-X?; z^L~TAGey^@AT@j521xTn=l3DCc-}E+``M!NMo0@huRrw77hRIQ(SM#d1f`rSD!M~j z?0H-8jGZq!4};VS?P%?VqRWAh+C1-V)a4>kITO+{&$|FEx>R(&7}5&Qn+MBZF3RtN z)b4p_!7htL`D#e3JZ~j>>`Ia0@2OUM-q$hw#!r-$^+Er6UQd*HjVK=qX&o$q-dHR$ zhe2BJd3@3Eb)teBlx*_6v%$Pkq`4o-R?n;V9Oq_{=Jq29Ui}@?Eh5d`Mv`&w6KKCx zq`A3BaoqbCO1WL6xtB;L?oGh`a;HcSg;W*y-a`4t0H|bq_((s z4D9l{NWTYZS={T3TCW!AzaXuMdpE`LNg~o^ebIk$k6-iO7wMfLt%`fAk#CKbR>!>? zVeyYedIGRDac?4O_L)e}fwV5}tw2Al6Y1+9t%qG8VHfxnkT%6VnA7=Mr1|5Mt#NM| z^sN_N`Q$hW@5m(HilU#KWWwu@v>!z2CNPT=-exd2inP}c{g?0-p_rdUrUFt`!g~hH zpGEl)NYx4NcG%@tQOX}w)Fiw+f&C^*kAYO1@NO=^=Y!~W4y3w-w>NqPZ*Ae%XT-Ue zOWex&t>9x9_|>>2V*9~1d)r;$DV3OA;HQA)lyE{bFQF$iC7jS;7kDDYC1w}+YoI4n zVs?Rl970tI17Xigq={6QltVLif&TzPO^MkBp0rx0{5@14&Us@a)Ve%5vCU8s8;{HQ zXteP%A8_jdx|~D0DvdvSbz+YpZ|qo~%sG=PMUkD@wFIW4JkFmqpeCWQWdz0n;9M#Q zyh1>6aKCDRwQyO8%L%X(`+^iEa<-MFI5CEm$I0Q$D+{>fIx)V&UX{$LSO8@$Ov!X9 zDQj?orYZZYyiMJK!MwGdE18P(oWRMSVE|&$AsXI8a z{eAf2T-arr6FZ5gnLh8K2p-d%*uF4v>}v9;nRCKyEuGkt1lVnyA7%m5ZSN6f@ALay zpqA=x)MngF3R}oYWS{`%LPN@ss$kA4L$GPcV4uRNW|*Se&_D`n$@yo90wrhqFejuT z*eJQqr*M`Urf4O6!+f0X9Zp?C6ewBe!<@^8V58(8pTY@ln4*>R01~I~h_l`h1xohv zVNQcXuu*cfPvP7+Owmd%B!!MCPMAX!D7o8*IfD+tM#(Eah12RVMJw4r3ZEFxwL=sr z>3~!PbMhU6jgp;w3TNbDidHh76uQ(nO%G9^jTs=&$1p zKSY6&4}F+Z{t#@GZ1pMh1B5ABNf#h-exIguAVh(Zkv>dsK?pWV4)-Z^A%rPfNehrX z`V>Mi6x>RP4u%k93V)g~Jq}@57tGaO^%&C)5uk8L@yDz`eGZ)yQ5>VH3a~1RZi`5s z)>%&;A9^}8qIhW4ET2O+M-<2CyxQl{0TRj6I-ewuV*p(vQ9S7U%IDBw62&n(lYmuO zbfrY{w9XpxI3&=?62*hg2A@OMOBBcGoa^)Gl!@eNowt$45rXcTC?0ga=5y%8iQ*WY zn|vPKJCQuCGXq$S2z2~J@u0KL=g=J##W6aM_IY#^Me?-HOUdK-K^IaK4?36o96FYw zI7a6hpGTKdBv0#f0IT7I&ZsCJbXNNux~QT!M(2J$kIt(|p4NE^c^qBn){5dm=M6rG z&aNnq(b?|v=mv}AX`TE{ULKueQ5@(i29!s4Srl$`)sRCcS|ms7;f62j>nh!Anx}@H zz?YC_Qurl=KDR(o{Cerct|!WYgg&?&^l_qmkK|dhMhm$_K9Q0?Re2nO`%Rw1x zu5Nhx@N&=rM0sn{rchJN4H}brlFkJIfUtl48ydP6T1b7x((=L)Rf@ISGy0>>llK~-MZGN z&=nb`=$lmlQFjyllOYP8iS2!u4$Ba1lSu@BSN8G?N*EhK(Mkr8!oEKtmjWdR`S2CFuu(GG zr!0@4XeHN>qCNz2DNypb5C4)28zpOe3O%;b=K6ya-iLJKhA2?d4XFw~G#55XM)(vu zdqX9-x+RUI@NT{_mjWf{`Y>I>;S%($aj8$CpEyj>N?rt#N5^pphJy8k=tT}erUfw| zd2}m>VO=o8%+)AO?{a{mN3GFBIRw$$oP$m!$`Od(=Nxn%Q9cm#M(3c{6Xl&u?{p5z zP;+$)(p#N_en6D>4!zeo=+8vi>GWpjpq-GYdW+ue9CQE@^XTagA!d{wOIU5xu0~2V zQqfSxTf&Xn^X`nutw-t02&;|SL%H=R&5hdg)JE+gSdY@&s69_@)Er$A(iCsj1 z>?gJXT#9gYFz-@a=FxAT&F3zvjcO0aS^~VJ6F-nmm%0^b9v%7tMCU6* zqN*kx{2Jx2Oq^H^fwyo;U%v*}8s&2_z5g##_>s;908L>AS6g8w67#qi5JF65MwrJ> z0-Y-a+47y(T?FXA=VCz+c#gnD0Jv@t1U@G4Zvb3E2m)IO+ya2B2|=I`a{N93Tv!MK zy$L)HfNKmvU=)ED0dTn?2s99&mAUeu0k+C?=Sf7{aN?m+HqD@X3}OpOc@-xfn!-jJ z3Lk~oL!>ZpTm_G-+nvod6h02I_ed$h5(5ui3L9=Hd?aF9Nf`u~ho;a3hQh}p)&)p> z5>7lcg~l)xJ{qx+q?~~h4^5$2426$J>oI|LD>G>LaKc=m(XeW_dS=u~KZ$e6;j)e9sK#eGG=mdsC5MYS znxia`<;-bT#G^UN3|S5hf)7G+xbUMn$`)A;t>VNA0OxTzNORN;k>$`d7z)VY zB9Z2(J0i=WeeflS94;Mcj=CkX92yB;aB`looUnZW;EO%)u^UL_T9Ym;WF)RFU?e1m zD^QvfGLjxG|4U9k%Ly5YfD`+U3JY5;Jd_JZ|74HwBuD~4)VV}}Z(3<*!(=bw{_Ekbd0l)X8_ESPr!I=~$ zm$L=U&Hy&+OI!{d2ED@RJ?KExC2z-GXh6YNb?E+n=0~IFA|cS)&xw%|?}Hh+*3k_G=p>+^Tb zXO7~6bto}^NdD&#(giOeV}7loWeVoN6!}AyP*uQ3Jb#!Hs!Uhz5gUi7&ExrD6F*tEZi<$w=;gtGjk{edBgzR5b|;Kq>v*}LQLj?C*{4Uf+sh^ zDhYWdOrDVE?h3ofEwF1`u0r0VJQSUjCs)^@7V^u1&@PVvvrx7prAW?)RXfOm;1o;l z!d~p;&qYNF&rk0FoA=&8pzxB}Knh+3wea!+R?8_ERSRK}5|U2-F@sQ6?<&T?*C%8n z2o?3Q+mI-i<4$zs+vo{bev7k^V^H&$EQ5)p9D>-~SjvqhC8&Q5Qc`x|e%afqDx9o*F_mNZ4JJoU%+PH&!9-JNJER@q%u11;t- zB4wyN8q8s`9eIb#5*T|oxf9ZNlY`Le5%OK6kCbCj^HFj$YEmZ|?m1e1j~4ANe?e(` zNP+t7Dc8dCd&#{~pS|VoC})g(2f6Cye^LIw9P^_JVQQ&($19I zqZM=I0$66AoD02Y$wQHTwtO0GY>_vhZu8|lSmPYIH_&tCJZL#j-V43wO9|!$@@iP- zLOC9FSRhZp^+obaTwg5jL*7ed8h6g6G7tRA8~=^?yp<1OC(UKHT}w$O~|8 zmoFmMv+`2h8_&tLu*~!FZ7^SuzoXY)l)s@}FUgl+nU`e>dcT+7ptKG0f2ijV@+xqC zlt-e3jj|p#+$4vi=KqtQ!VW*lCAhCP%WKezEpj?)^8Zoy-eFc0S@>{w*A0Cq-pPS9 zK?_-1U`$Oyo=O?G%GIYy2)jLJqRoZFR2~UO zk}v@~CR(RUA?hAqt|1O3VW@WlhT$q*;8le5;}`RAx|~~rv3W*5^SP6d(XPbNl5@Gv z>s4xb6?Na>-dygX7LLm=FF1w(>3b~b#bXLT zL|dsHf}YI4y715sL;szgSEAnvo4q1llqZm{tJrf@DVlqRt4o@e{Iq-}N|G%HPO)5}B|?{Rr!nlu-Q` zj*{Y;y88PNdkjP=u?Ngou~^zpj7eOY%lu4-=p3{SCw}=5H?KQR1rnBLVN&n(|=EE;W2H^agO%xS90Xn7A@Ah7w&ZmP-ddC1RE zcQDS=sfqr4Wce@Tcd5Gxc}-1>ppG8l9oS0tMNRc9@FynXtWRN2+<|fXpOk+hO7--| zGd%b3lv?A6J@O6JjirMm((e#`9Y@U7el!iTk>Mbw+nL1TJu#x*e6T~YQFu~ zpyvM+TBzoWF`^GjRHi%B{CEJm=7UYm*PtSIVAC}pY-&zN_6OJs_(_Sui2-zK{0w;~ zdF-#ji9Ogi{3cXBHoqL_=HSExKEd{dOjowST34tZnTT2|m-*+eJQx>HZH+QxjdM_$ zGq7oElo@MW5BVDE*z&SOAGYYPkng6BM`~GOcj}&j{3N!L+@O^`O)KAqlAl*iz9r@7 zqtvXe{8k9V!-SxfD}q+0Ri>5rQ~-{4Q1Dke71zW!;+=RQZR*g}gD@=Xrf!VuL-s4w zXe&0|)Qxd{$mZe>fX@y7Q5e`Z#%p<^cZb}Cx{r~;#&|uOx+mme)Gfh*xiLPG#+?Y6 zUiOlokw~AYDbxHDp*#*-@?xn|9x}?>{8d627A6GEpZ+V`h`;n&q1im7`DbGQdnT@X zx~4<(_rR&FoByn7{zBAf0XE(IXHE03g}jV9_VlxH-P89%zKc5c^s{l@(@#O>R6xn+ zK~tZP>oMsqC|}2x+*8Wm8E#2y*vi;2=ow}jaabDq2 zY$b07&ActH^vDL~3~b4_r2O~8tq3>sTOq_ztq#px|BF<+YksDgGW3?xy?`=aRo1bOoZb7g7)C_KORTR$#J*!YE!>wY&GdP z(VcN}d=#$|-=nbqVbj%|8TV<+A!z47Y`QaM#w%#}Lm`i+j-4?x-kZA1Ava?yJ{=91 z6EEaV0Y78Q$%*2VPCo;>&eZLK`~|jN6Z0^X#gpr#R*M^g8na>Np*JSu19HiC1iqym zyjbAWmjcs`^ADl$!g;v22;CnOYrNMkvCf;PeRXf<)vx~rdSjj_c zm_x>%!v)<<5Hq%C?0J~rV-0TXIZu?@o@a~D&YpJh^X^iV&*3=D5V$q(ozFP5@)hFg+S z6SkW!b&{)u&=pn4m{IP@n(e^wPoaSl3=C};&3=!WQC`r@^}O>w(jq6G#}@sF%6yAW zx5$ZCvPI>%(=Nv5{|{a5#QXB>;`~Z2b@jYYj}N00dmQAwu$4R*v{IQ?HbQwgw&b2t z{^oEa!mT_^2r*QvLn|ACR;EuBT9|@rls3cU@5S^OxS_>ZF4NRb`$|(c$8=NYp++;Y z>85Uu>84%^`6B9`!tu5_)|Yn%t&mqzcM3+j%`tsFe-!dV)cuaD#OBx`?Btgqug6w$ zThRP1F+KW!2<7|OlK+!By%%m>xcN%jrBKyy+w?_l*UzR!PP*j(q4@(bRIZA-?4RZi z&HpB7{vD?IJ5i$@*mU#nFwHN;b72~rZvGvn`TZgH!B(;?Xx3e(S^OaJXl%*rrL48# zT8EppRtWEe3Bhejn{C!JLJPMXO`e2d*Ku27sq#5aLi{KwXAt@ zKEFjiF_ueX-UIx0Z22$a%$OX@D|2tNenxsBfGzy}L{jK!Vwp`r_yf$H!>Le7vW__+jRJxg<_Q+o4oSjrr2 z&XuODv(a|`xhsD?NK7A#>5(!Q=LsFW`e=*A^hh}dx)In4X3jndvjsglh8zv~P;B`p zz-N#&=D!y5CDe`Nz!cMSEpJ19hPst7ODdLp8l{dzlK$9weSn(n8RLyhu|Y0@b}_d6 zI1Z(fSPpImoyQ^5?OHQ)K@;W)(sV?rWg8X^I1SlW$I7)&YT5cldoG!P$5p<5P!{88 z=yW>vD*WOPxAOWg=K$nPLeE%|jIGG&V{FNJLVO81(4>pl!zp#o4oR3?Dui#t1pS19 z2W`im7}pEU=Al0Y*Qv9dFGovu@1q_y{BCq@EXN$i+ZT1J7gA8!H9=+j$MhZeţ zR(66A{|JIDD0~*5D1;lr1XEcaQ0WDN+r+w-9y5pe>pM-|J@bt+r_(xV z>fzF+hYplB4GOjCeIX%7{dWOKx&!J=n+njs?BnEDLYxSKtyW4+4=js!JWUSs!USm( z4*GOy_BI6-2-?){)NPjBgEsBPk$1eDA)a#)j*yaA`FAL>g!|oMIRCq&wq3B5PZS;< zo2;=9&`Ayxt&g+Hw8Kfw6dnDGHKIp1q?U_*HT3nOUkv@q5PwpL{}0jAd)q8}bBD#d zsmUI1#ir<9YveeP+$#L#!sUJK&Y;YUE|nD9vmWb2&w30LJqPkl8GiC0(SMQAr5_QU zwAm@!G8%)cayMLIx(!3L{&QD8eXxu-pD8`wj2VOx3!5HqK2v(USpa!Hb#189XG-6h zErommb$nX=Oby|M{1(VJV#_}lL*ADv^#_#kFl0`pr2Z>Ke*|;gib7E}+-Ob4F^zvy zoa!$H=iTMNT_VWcrMV+3?K2RU52f!?CJ6yXl!jCvA`N-PG-Lw`ci_%>0lhL*nD>{+)}DK4yOGf_o42`27K2;<NQkAYl)v%?` zk=D_x`@%$X=;~W^S`0_3!1VF&GO%06!9m~E>PN0Addxf$cL3Fo2wUH_KPf`^w!O5n zH=cl%zWv#9sWvA*ieUb`E6+h+>PO4Vja_@8f|b~`T`xCwoeFshwt_QpLAqS&3(^^o zXJN}vjm1;3()RU3NFGwv)Uy$B}ypB3vkd_&H4I6^z2yFU-^iRbL(hUY#0PQSn z`hs+$(if!LAzz2Brs?dHoZD0pCl%DPb&CcJYrs>ss?hTb%i0$0S#qdckXGv3OQ+Mo z%S|hTG5#hH^murW(if!Hkn79Xl4FE;EC}YDid>Kyg>XTbV8+An1!<1R1Ex zNiw!vAf4^5Jo_*iTjnbLWZ7pZ?jW)0v1P8(PnJhRo<>~~d(Wj7dI$`~h_|iEB*qGi1gDX_L6d zB#a2k$EHo<8k6wr`8~1i!Q;tQ2k?0EmN_M(0t=r7E^@l&kQ>}8UWE^Q%P8I8H_Ox{xQJaIYY3pqE^ciyn7qKNDlkyKosac!(x)9C{6M|;8 z1kLRFrO=Rvxs`KDm-WF@q^DndXuZT(<4*Gr55Yxfus$vB_w-HaE>vbaHr>kmJ$+N! z8Mh0S*mNuJ_xjM^um@yLjQhOLdcgD9k0(K%N*!DNfLBW0U660ZR!o1$L!R+VK;~Ea z_4x6yr^k;SkUznu??ayRc&qjpzD$dH)?lRQj4k<$RQ}9qArtAFFM7RG?+cv5o!m7- zF(V5uN~tdeW$?niHUlTAWFUFb>Uej@r{Kvvg;F(wGuN*pxtd20r+T1p^vE<)gz(5z z_6&yQD?E4mQe*c!tohG5Qp|s)G5=UpZ!k7(etbZR&+1n~zLdJVakq7qG2tDMTd@^z z8G)<3DvmbKLVgrmF=u#}dHR_kiE952-H20=Q4R{zqtJDpzK!V*ke~8T9xuju4F^N$ z;LFUmpC^P}VM1{5Z3sr8uFHjH558nVG`&%@%~{`CC(8vIVIe0qKy(ZgnI?L3?}NUm z9wDfHZrNW1v9meY%NXN)qUY9#p`WsbuEq_YKF&`vhMs~F7=5V?eUdTst&ne`j-J&S z#?X&IevrDQQ>p7rLvMuqZ*1E5GdvyU(6t_83^si}%rIyDOvp!Ki)S}g!cV2A14BypVnM#I%byb zp97DP{uyHW=WE#EBW${VhM4~8H3FjnHr+o%P5+F9ya#pcpP{CIrb0d#o9>^XrhgbG zLf57TD4Zl#JY(@{p}k3^s_w$#f~YZ~td98b1ujPW)LoP(wq z$Vs_kg*iwvT=ZFuH?T6By-FMVB6`L(udGS-_gR}=eyrH+Jy(z2HK=4IHf^)_Ts?R& z%A|ohUZLJ|_1OIf$WyWDG5No)9+NMEya-!<9q#EqaP^pcKjgn)(_``ux8`(w-m;K^ zQhem&lj8qeJwm?<&WqTRJ*0{isF*cQW`y2dUc8pBM4xIea$^!BcDgYsd>6@ei)XhYs$_%p#h z5nJjUsVR@N>8PVMra96s75EGbIjI{&_X?6+BYORaMTapy(l78`nIzSph)x^tYZ1c6 zD{F%-j&t2f=NgOcmoop|l~2wS!%a7ay8yL53!66FbYr;ZAU}?6WbQP6WcQ(SP2%15 zpO5=<5j^Kt45%l#YUFIh!%qhCBT{^et%N|gW1uM?jXQO0hYdd!bNlC+Ts*r6uKsgZ ztAgpsc0_#m*_%lHW)X-y9b* zd@#5k{DUbCV_r(R-y6Y67cL6a!4nS;Y_3b>B~UdI zQ0OixhdE+?|61XnZn;kCxIdZPn8zxs=gGo~z*X;NDvnwu=L-KZ%XLzhh4T2H)ra!P z3FR@=Nv;t7&z6hXq);Ba9Up3NP>-Ta9-iOeB%hT84VLVrJ_==Wn$?FYQWegm*-37f z1Q%MelPZ*Mq(vUH`cNh{nM_oU<2uRClHx_nc2c84x%_JNp&JE>pv(<<4Xb$CpqjstAAGBO2byp~l53D|v$917RhB?WHh5xbmF7U7SuTqpH=C=d8|Gs8u2_B;{FV|OQ+hubdxooBhYhZY_C<*!yB zYRhY(JYds$;Xh@$PHI{xj~!MY%HzvW9g=S>59P7D)ray>G5~T^gFRY= ze}LuUMp$%ntPbf@s&bcUwv&1(RLIIODKJ>qOd&YQw>!zVCB+)cc2c`SxqN8#p{n+7 zmrEL-Xl*t%_`$OAQlsc-_}Y^LLxge}*e;hECpkn?jJ9kiH7S(Ke5()TvU|H+FoGT{ zDVA6^9G9V79=7^WE@Rr|GTljDA}Q8ewv$>N%H=1k59P9NyIh)`pIy z3&es_q6_jJ&fC?W1Et*?EZHt{-(a>cfOILcWx6h=W%V^A{)ho6mG(C;OG z*Xe|m?w85kPC=H}0O<`{cWhNlA=G1MI5xbegIsQb<^V{1&$8Ry5T;>gGPY#N`KAqj zpA+s2J`~`7uvx~?_BXQ0o|5KOo5o3w7v0xZ#{)y>!{`3;>nf5Yi&N;gz#&aV9s^p{z$36Sn@Gqtu8n=Y=bNrcXomgu{Z`_(e0bj zrEe5na0EKT-@*a~$FJS##)r0%Jqe==OY>C`WyKVrP-`G(Xu zqt}kSAj4m}wbbLW-Nk*TXAo}Yvx>Td+_;%I(8GU+&=mSHK`d5FP zTQ03VnlC(+xZOEB;*8SYJaRciX}#-q)A~U#kAgpl<3n%Ke=zJp{D{L#eAd<4mo#_q zvb~`kK(B5O2JgR|qj$IMg?I?F`@kNERVd&an9d;j16S~Rr*l}L^VP|2&|GI~{s4>; zm*zWS@k5M};nnAp@)%a;(jKhh`8XEN#GoF642OZVG)|3=f`=yN(q$ znCxO4hxm6mNccAejolknTg*XFH+FF!)_XAPyLc$;JDBxcyf^FFoAqnfks^({UVqm0 za*mwQ8|Ij=-zId|2RY3jZgZO7psoF7EzS^~yhj?DHs!HZM=kG*SRW)xDLXn1ITEE& zr^5-mITg&*89Nxk#E$E~Re={2_3D0HoM0-DhU@TyzYkpVpD|t@=ESgqRbn4BCztz` zW@)`Vv(jHxV*ym2!vUs{w&IRcxCvVB#R}tb%rIfkFPwx2G47@D6@ugdsH^=vvxHwB z{FKGI6fWVH=f$~%U!GY*(1QmDi>4w!H6kAeA66=(g6rU^Q3ZEV@hlZw2TwJW-9W`g zD()#k@gu4(qvBI4xDH-yWYyJFaIi~o9lY46s#Yp|G&#Y~rpHEC^~Zj06@K!}I(P}L zgBKf9HIDir)SGqi5<@-c_pZ_v;yQS-8P&%F_D;r6@mo+NxDH>NN^pz*i5s} zb?{=d%>D_mer&ec=Q?-^Zl!rAy=fOYU_umY@uM@tr99X#5r0PEn< zOa)j6k4-MXI(RgF0oK8zwF*vhnrWQ^tb@k}uOfQi@oVn^0{zC>@A%d?__}#+oT2G=fV-Ow@to2SHg?EtA$FRE8)f7 zD_w>r)%dNjP3-;hCm_`N6*z2TAC$iZq0Z+@c(ISGwnM1*A0+M5$`p>;2EQN|!nTUt zAWZNJiF{GMFN8_{Y&Pbr$`c_p`dkSwwxi-A2-AJ8gctj+^i~M7{3Cf_{ZRT(2u(g$ z!i(*yehtDxpDW?Tek%O{|C;>=X^daAu+(49e)+BR2OupeSqYD8;H^l>N_e#pT2rzT z9);DZdbG!POUFQ1laiJ2D72;CM&qy&-q8@&rDP>MrdgkomGGG9hSWJYDzOsY84xz5 z+VG2&@UFqXEh$+E@9z+{rXEAxuo50ub=aPgmGHQd!!9K&;Z>s-F+VRW;f;ahyXtl5 zuoB)ekkXLQUswrmF{DaY$x3*)L#lCAHBL&bg!dGrT35+RcppQmbJcUGB38oVQZV(d z!j*(8;q`&k;Hoa@8?1zP0Hg`7Ivp5R!kY_elB;ATyvrapy6SgeSPAcbNYh<)A|$MY z_Zp;Gu3C-mZg9)DLuzu>;W#w05?)Cey319v65bw=nq4I;;T-{KsjFlqyoHcjTqP^v z-2`a`EP<}YN_c;R)aoi(32!r`)rew17h@$nryRZJssxT!tc2GMQk$zjM$NDi-Z)6> zTqP^v&49GtRk9M^1&}tlN>;+V3(_W6$x3)HK-%IeSqbkGNLyWX4~|o;gcrjVXuGRq zCA^-HcDd>-SRO0k?GMTER1er1E8(3A$@i43gm*opw5MbxyuU-L^pvcG_YtHTPsvJn zxs~WYPsvJngCW&<>UeY`R>GSGsoqnv65e7+4W5#f@K!;Z;HiP=O{|3XBBV*4Iv+K{ zN_gKwYV=ea`UorGRaBw>JS8jP?G0&`r#7GsSP5??q$W?D40~cFyvrah^pvcGw-!>f zr(`9(cOfnHl&pmJJERuW1-*im@cLGx|2!ot;T;O8)l=u;SjS3ur$bupDOm~cCP-^M zB`e`Q1F6kZvJ&3^Ag%M1tb|wC3H|3OSqX1XNEm`>+z;xsbNN z5|GxprME)b>Z!rF1Ysq-ry*_kl&ple1=220$x3*?LBdxFWF@@LozZ_v$x3)5A*Gd) zmGBxNRVpPb;hhDkMk!edZ#kq|r9Qy1ifu}64V7N;cb9auhb-*Bv=XWYe)@B z$x3+qj`swm?!-WcmGA~anuI(tu5Q6Pk&qgdl9ljIfixX!2cZP4gm)>VSxU)Dc>IQT zlTxx0-g-z2mD+-%=QFpIU&wA&YAVJMtc1t$WvNmNQ65&p<9OAgR2hzPtc1q_YlTv> z5*|maR;6SmyhK;@pHi|CUOz}{l#-S34u;gG)Sppntb}(8q;*Q+W*MhN*_DviDVh#6E8+ErG%KcLCAHdjQcFz9N_bC0S`kyS65bX_tuZAl;qe2?t7A%5!t2}v{TEZR65dEiZ80S) z;Wa{97gMql-dT{=$J7%zdax4SwU9Q%l#7E1E8#s1X;VzeN_ekB+7eT;65cnEw#HNn z*HWy6SJV^z7gM{TeOL*v4$>~z8lw$X!aEd_6IZel9zP_E`)OGTkKYYW$Ca#vw+2#W zTn#{;)4j6iA=Si{tc3Rkq}sUZj$;rj;U#*Z|KjRllz^4+xfY-;d#B$e{m%%;dO$vG_D4sJgkH_98ybM$x3+q2EdBA`VONh zR>C_OQY%VFYq1jEVo0mwN>;*K0clNKT@=Hc&0h7RklNyk-P|u(ZSP5?xq`HKfhPGoRJie%1pHQ+A-sg}S66$I&u@YWkE&4B^w!mgs z36HPoO-iV#ICHQP-ZV&!3B}tVtb}(ir0EGIE8*P=X;wmY!C8fs@ScIxlu)t~-WQM- zB3)9igqQAz{!1uX32!*0r3tkYrDG+$VFG#QforVN_d+f ztxl-funShgbNi$J5=vIW>j$YVp>9dwJqoM^329wIJ*f0bct=B8pHQ+A9;FQlbu%oE zmGBlp+LTa7;Aq84c(+5^l2C2vhb>r%64F-K1rk=m`xw&pgn~J_5}q1>{!6GeC=V;) z@ytPV-U&Xg7Us6-CNvbEo#7cN4LaI%wJ7E{B zg!d;%bxCy>Fsy`kEu{LSl9ll8ht!Z%`=M8GHRJ6wR>I4zLvs_Y;KyDc@C)kcEyS7J ztpeTJ5IZ;NQ%cE7c$a{Mk4GW_R>Gs;r}#jFmGCH}Q?e4?(z$N0N}G!82Fw5Iee}%0Bhkg6ZcaT z=6D4_bZLCr%1|6{2mx~V?dF4A2GD)wc;kpNH4i|88iPy^0NxvA0B!1|JrqXL)@-w{{} zIVzy=i9AX%g`7vqvoIGQ(<2lV(i*^gZjZoH$ddtu5A#uqDda;^=&|DyeuRQT;?QcC zkNgo>3h5P4=m&^WOdx$ z7+{?jT~1kfrp(>Q<8VS}R5l*UJUrmgMU{;sWiAYObY5lUnKE0+0u{y z6LJSBya}PtEtC~+FCFjSL^+Vq2iF1pgeXsW`s6yG3QB!)(nr?;?M{@Z5q)+Y(7lQB zfTs_y19~h`9-8#&bwJN0%7chLz7FUOMA_Z+`E@`agECJ4Uj&hR(v5`a5RAe!k>h$9?OdB~Km-ma)Wf+Ayf;rw$!t^0VVOq)YCKIMpF$&X6j@Lw(UdAX)J2~D}gz0vS z!ZeiQ-A|Z)$S6!pIo``a^kG0JqoIT^Uta_;y^axBj@DctItRKUqZD(n4kCq?qJJ_% z!8tKGfa$P|z*0z4K%o~iN->38LkjIp*Jgx*LLLoZ`Z^=96tXd(&;c5ym_l}u!d|0S zG(tfkouD;`en(ge85vOMJB{SvnnI2xg?&$lYJ`G9ngf^~)(9+x+!|2mc8yX@Ay1Q{ zF9IDYDCCm>?orb*2lMn01EO=-Hw(oS(w7t-hbMKUppXLt_~MSRv}ATbxhD(76mmH! zJer^HNI@YF1n?IfVJYO5fI>fUc603jCEmXyi~>rbjUUhvlFSDardv6C#)kmWqcpwC zA&MEL=Mm))L~nBk^fID6ujqa5fZk1%Cj`CG9ngOh<&mtH_tOu*`ouToenOOoAidQc z(sFS>-ShNbcR;%nWvA1d-2vU3D0_?E?GEU%P{!%$jvz8hpG%mo?l;&hHjVKYi^B1~_26sDCNZzo~8&!eyzrOScD=}(WsW|ZEYFdghs*o@ML6Q;*K z3Y$^-bRe;CPP^Ap3vq0GFcU(Qw`?*p1U~pGoI3 zh2!nOuP&H=e1jSfM=sgAF&u4y42QOGyzS(0ipOxY88RFi1Q$ZU zaZdagjjayWHlINCHB4vhpa zI63@Os^LWK1HkdV;m>*!IoD)}5hHPK0V5$foPjc&h>^@_Sq3=XXgLuh5rC0+#Eu~G z&z2am6amM3l0PSqc#kDU%tgTQHt^>%5?{B(h|LIKd}GlMllY4zMhr*5@y3w28N2!x zpcf{vOruYM-qoTx0IL;v3Ba1i$+Ha4@dgC|XU+`3!5e3N0Gwd~?!m#2UvE2Spr&Il z&RI2+?tB)U>0Wb4o=**B!FHRpaEG1{`k>>X?R8bcU?+=irCiU!s&bWf#tW76EcN zdlv@&On_mIoX86UTo6!q0_WFU+R^dW2D&YQ&eeM?(ee1}@w=%y8E4{jGD9)l`x(%h zgT`WB&#g??f#LUs;8rE~W)A)m3e_nZ+IJEalsYLNv{G^p##Xi;us%-FXkgtcxxQaY zZMoECYNu!jlm`^@kiy#&vUE<-Y$y*b<}^u4wFb(wpgc&JD@56ytB%CS=Rbu^%{(CQ zFWe>k(RUg>dN*KqL5kLIz8-4)-DdtFbbKu)SxXo?zMva#eGlFXG2+;I@_V%If?784 zO2`*r>vj4W_^#~fx}AT6+=i_;Nb&~1|sB`qErJVc?K?5?n0r3^4HQ+GR9A+O0 z1uLb^o{t!4?C*BuSd=}RLE4VH9g2aviL-mKutBr2D2nOL!Ca)pJtk2jB_Tik=dRoj zNwuktsiCRPK*IUh@<#}hU&oVrmLDrRk)1j&?BY&L1To`p*j#mv6T33UYmnO&%<=f^ z47}Xtn8TN3ntzj2`DY4YwIw+D%S6Xz{Q`46D!R|CE{87zT_qXi@U8aMzX!R#AZ&ij zG05S+qU$y|(CMmdv5CGF!Z1s4%5x*Fcm9JM8AN{zoQLH)i2hGW6GUGpoyS-W{!Ol9 zz<9Dx9L{G6mY)N4`g{Cv$4N6-o_n!0gXK@do}Xs0Jojv{eC7W&SiUL<%8#oQPX61G zGKksdzIQi%f*|^qR*Eg`g{qZ&zB(jd?e9dPoNPHf^$HtBs6Zt}LtG{x}wclCZ{iqZ4$m7a?; zq9)afL+t(X)!0X<3C@-e%Adl1U5cUeA6LDD{rc2{;Cx#7E%qBy44mIqQH~IY38_Np zzbGG!{Yfc?&3{!n5&MlPM$PZ2Xu|&V)QQl4S9%HdXQdc3|3m3r*l$WPWPVrm-?6_i z#fbTzN?*ipbBY1;zv%s?DaOnHR?3*EmYf=#hOS%sBZL(>5-wki*n`#_374m^I!D6g z`$AZgBjNHC+HxdZeqRXdawJ@yY1ZdRxIDAokR##pe*m&6N5bV98@we)!sYLTur)`* zw&C)#AywLN`O6{I*l_uKA=TP&`Bxy- z*>L$Ckm_x?d>Rw(4K`eU7^Dd{Tz)E~Nj6;mOh}D3T%IqyOt<0ke}^>7hReSLsmX@R z{|afL4VSMeLvPt|`Mn`6wc+w7Kx(n!@)tr{VZ-I`fYfTk<^K(7wGEg54AL4KE}tq# zx7l#{-5{;A;qr$6 z@+-XF^(N!{u*=)NI4$ zpM$j2hRbh*)MCTsi*Q9=VZ-G|Kx(z&^2b72ZNud+gtW$n%l{Qpn+=zL4bnOrF8?E> z^)_6-J7&-}*l_u=kT%(H`5BP5*l_tvAZ@kb^7ld7Zo}nYfV9ho%Ws9`*l_s-=A?WZ zF5e4M+J?)IgH&n5<)=fcvElNIAl2G%`IV6BY`Fa2Al2J&`L`i8*l_t>kS5r0`3lSo zO|s$gdq8Tm;qpg7nr_48&ww<`hRa_AsmX@RKLlx^4VQlnQnL-0=lHVJhRbujYO&$+ z9I#f{aCwectu|bKJ91fV!{tl5qW^5T{2q|nY`FXpkk;97`Gt_y+i>}tAZ@VW@_&Q0 z$%e~shP1_o%RAlBe>Pmc8>Hkr|m%k6v3L7r}0;E2EbQ>;z6Qo%-T>fv6 znryiI>yQ@OaQQDFHQR9cI2?aVZMb|_NGeosg%Y`FX+NUb(pem11lHe7xQq%}5N zz7cwKz73b>nxttPE?)f%MTWq-e8c17hxcn=Sw%c&|FCp!+;qp9lGU4((b28!b z2{@}V;qsjzWy0kL(~)Jv7O%N{6 zH}kv`@DS0}hRajRk#PCbz;bdVT%LlTBjNHC(m4_?e;>-J%#m>US0U8oNVq(a+8ha& z{}Mu7j)cpThH!blZLB?bXQ5b^KaSTK74&9eR|hw*3V=|!yw=g5D&g|Y0h!N9Iu{9- z|0@ByECaA~gijD)UHLR-0Ash`0L1z57FJe3Hm>7#fXrO(MwF>} z0Gebpp5q-D0K7NK071C?i2>Z&5q7C_yxsY;B;c*j!lO>YbwL_!K-o zA7%jQw$F*O_jx}TswFKbM_CGpZXq8cLm4m?0xbB_u7>$k8G)rC2L%*9YDOuh4f9E1 zE&2Qzp`egu0n7)|2rPxH2`GF@jZ#b@uamM7h49feLO~%v1TddvBd`=w21Mt;2iz#d z6jD#hP87nY-UtPS938-X435B3$T6ygEV=MSH?BNPx$69OKcS6O+c%uC7R=t8$vHXh1+DB#f9m5n21z8Uc72FuDbW&Q?UoX)Xq9F*4w zP@L|vY`B!QA31cQW#yQ1<^a~Wt8`Hqo*s5WHz8M&!rLtR+(KFL_R{elB+7w=KDZ9( zt3-Lq(wxZ0l!qpLdL7UiM0ri3kFNuI zK2dfzeSRI#+n|io{}(~zp7cq=bO=Uang{`8gy|uS!nBd&{YscF!zfH6;rRtfoIb=T z%)tgP(GjLoF$&X6j&~GcdKsfI?SvQ;!gM=EVH1eCf-wD%QJ9u;yvKm(!+=gkLkVBL z-V9)R9V4(Dt=|U}x+0?#bFfwd(MJ>glMxEeiBSPehh+qoLXHY3^kPOSrjYYUp}pza zj8ITWYXH;N8G)scCj$x{pizn`-}YxIgnC@3TjtvU2N!cs`DfI{DCBnQ_NvOg*8 zdpcAj6cjQifazh4z*5L10flbYD8&?VA1V4G(2;^dUJYP6XQMfoXRWUS%HS*%Q%DgI zeZta>8_5NQ3=iPbI>OSD!vf0kEEH47S)}l2UfYp^LT(7)H#@>oNLxVpF$={M@-C2g zY4>cc_>B;~$dLw0=2bxAbSp<;lW{!|JxVvVrRsOQJk>=)LZM=0d4Eo!;yYXg{LtEqb>*p!-7^ zr>8rD$S6I7FkRnK*o@K_5T-9Y3Y$@S6=6EZqcDwx?+Fn8ISXt?=?@6geI8A1Mrj8~ zoc{DEY)0vBgy~?9!e*4-i!eRzQP_;q#{$t&>(^R}-ZLIA1#5bLB84)2>?Rk+Z#mwr z1jv4^9pF-waQQau#_4^`q;r|V@!rL+E|`9NgBnNC_XN00Cw?fI$+ZB=I34;S#H8B| zN?lDl_zlYVMaSEZz|YvFuipS{O*mZsI8t~+!wCRGVFC%4Ujk*ElK~M#4xLseX2dLK z2r~I2ER_HwWI0(71~w63SS;ra!oUs!jD_VCLKw(_ti$Cwn-B(i5@4(=Cll{xcZ0JciH^J1b;;D?(*b?~q07M=#)ouqt#A8v-iMoI=e z6}8_^i3Fz^@FpQSoXjyCZGlWaw1wln zLk_2S3?~u^PJ=k!Z{%>|$8fYQGWpOdj@JWloYO&uqYaYb&@>nd$l)ZB;b^O5IJ6IL zg2>_2k>P05WH>Yuyx`=#W;s#&0FZF`he+gHlgTV%B+e~hBqWD3P=*sRk{K<3Cuf}H zM2tj0!sUBnH-3sGMl3}@!sQPpak(W%%tgR4;qqsa_)klW*o=UL%ilucrJFo%Yqs3Op%Iw#jbvYg4DcEW*#))tp=-oG0AmxNOPn4AauESPvjO08O1XMyT zo0UKUsw=w^j{rlj>>8qpKe@Hs|)9~pVO>>9E&Hn?3V*c(9 zoL%a*ijhzinh5to-vpq;&mM==*LGrslGXT)-Oy<) z_m9SvJNC5MKNkD(i6w7Q&(F#yM&!3)KmThg_*wbHr~)R+r-hUJtbC%O>>etLsNiSi zlOw8F_52=GFd{xVvT79-!>C|Hd~#IPgH-H81wSjF99_k)!sqj3DKwvzPx7dj~6ll-iFa_=f#A%0dqIivb!z~1fnsboZalAo1No?NsHiX=ZPpPXs-3v;kP z%k1;B^2ym|zre@-oHVbl$r;T36tmCI$|vWV{h2B3&old+UQEt6`#-Tcr<#3!RzBHe z_W4=)5|5fM*&r4bQNGo=v` z&nBl45l_>n5fM*or4bQN>!cA8&jve5=9*lh4#mHG=A7hL;gT12dK&xQzwuMai1;Kw zE1z82c@wppsbxg`ZYCn0hgtHnf_#it1v(dq!8 zKKUjE{Z;woTRO4%s(f;j7R*=WlW&`B^;hMS?`pw(RX+J%=}SmoqrWPje82oh22{ugrE=ZGX znEbyWHQF%wZIGthF!{VPbe0X19{{PzhRGiSX`v01KMhi|4U=zywA6;luZ7fN!{pzD zw8DnTe-EkEhRIiAI(W4WlOF|XjSZ7O3R0U5lRppAIvXb63TeF!lYbJ@1{)^-A*4+< zOg@ggi!C-xz6R1(8z#Rur0q6Lp3}s;Y?%BKNRADY=d`bH!{j-+o3>%{KR~LqVe*`; zt+8S9oRY1zVe)ez)!8t4PO;Y8FnLa-HrO!v_aRNNVe*_9on*u0IsMsa!{nzxnr_47 zIh{GnhRJi%vdMm>ui|(^N`ltF!?VaZLneTx!urzHcWmHq%Af~eln!3HcY+=(smmre+{HvHcb9e zNRADY-w4UKVe-2mrEQperyBI14U-=Usm6xM9}B70hRHWWspjHcb8kNKH0OeifvJHcXz!ZnF)Oe+t-A z8z%o>NG&!@{#QsVY?yqf9_T+CCO-nwY8xg$4bmDLCVwWRHX9~?4WxB8O#VSg>us3) z21pxhnEcm}HrX)wf}ZF<8zw&x(pDQLe=wx&Hcb8$NV{y9{H2f_8zz4@B;4@JSLN43 zO4~5`&mmRXF!@+7^q&orr>DHuhRKhGRA36Rn@O#aW1Ds7m2E2J75 zCjTU)S{o++KBPJuChx)XS8v1Q`#@^2Ve$t;nqb4^=R=xg!{o1q)M&%xAB8mChRMGJ zX_gI>{|!=;4U_K+r`tjsCVvp5W*a7dDx{@0OnwEV78@r26r>e4O#U-Stu{=aq42A1 znEYr+YiyYONs!uXnEZK=*4Z%m>maSSVe*tV*f9CWA#Ji@@|z)Tv0?Hq+)G<+n0z-# z+ijTqUXXU#FnOLinJ{^tIhio|X<%l;juMgN04;H!a$sY-plPh8J6#QHXlc$i* zl`#2VQBGyT)0DM>rrKL$6@4rMz_AB6y|z^5?-81ozoIl+gwuyPjJY&Rd|GJx(YtPenxsd)ez)EMNZ0N}k* z25@QW6h4R(z&kp^E_IGKlRvKqyeiyAMY3_JgMSz~dOwrLlKD)SsioufgsiL2=feyj z-8PyitHk@cQ1<$0G-f=C6t<8Lk)aHvA?F1!pDH7;G-PE!;iG1hV%qQ|DKs*lKO+=a z>%#!%18D@7LR=s^2R@}nDW;HKq!ghLKDtIIC}jTt=Cf=BmO@SrD15+;QcNM2kV5AX zpL!z{6hgrY| zM~b>Q;L&Z7m1oL)fIOb`bZBJbp-ip>t2uOYWaCJgKL6am6DZb%DkRD4heL!WaFXCCj$;$FWER!=0^dKPMNGcQ)Vt;Jp|BQ zlZ}TmhXouuak6ow%xM9S?wzbWQ|4mwIAGB6lZ}Tm?+Q3{2W8_(nJ)%BI*PLLOqt)3 z$MJ(Mq-;Et*$I&5(6N+_BW3O#@aS^N$}?roB#-`oI-|1jQ07$uhc2pY94T{cz@zgj zE6}%8Iv_j^{zC2MPM%I-oT~dD*8=t^>Lk zQJ$Ri(RDzNA<7FaeRdttbBOYQrw^|Kx`HSVP5SgYppO#eK|~*42lNf1>~8w}I-uVG zP0;@rLFAq^4J1K_U=-$9>v)3+(?b}AX(PuwgfLx(QJ6+@ym^G_LyW?-lH*-Ym`=qg zY;L{pB}^}46sDcxkg?&$lYJ`G9{vN>e zuts1h)r5~3G5 zd**WpugU_Oj4uJAN9m{9Q_Lv6nka`LdYeP}8}wg9d0x@`+yVWVC{GA_qdTA;l=?`f zce(>wLzIW$v~1{JM0xDcd)*=JF+|zv^k#QJ&mqd*qIbIkx&q1sJ>3yRM(IZh)Ab#N z%_zN*Fn!@s*o@NO5vFrI3Y$^71W1D3@+fRZ>7j(_K99m?l%7nO{`4qpM(HNPbg)NZ zGfH1Ym>%~iY)0w(f#|^OPc2378IRZAu%`DiDU|7BH@PrO)A1Po9wWO)JHVxguvMmI z418j2dzdNkLO+zcU>()2U2?|5p`+TlODBG&ILVa(?Fl;cLx@SoAb4F(I`|FBFc!zV zp1__SVy-N})@YxL^!^%BmO-fl+I3}^z%^E2_^sgp7(fTln(o|0v=B1}+zhHCW(~^I!25xep@6v=ijI{v6rKuR6_D5w z_~B+KY_4=JPX})_DGTw#%~05Iq41ROjw9tJ{BScAnm{N#ExaY9Jb@o>hC*Wqg{Ovh zCn+D`hnu0$EJESw;jIUf;KYc5tvOtCW`0hXGbIMr_R!`ra1M|Jr%nv4ZJ|wL;C_TT zmttUT2W=JuPaw?66$5JSuPj(0jaoDMP^ZIBFyrom7^4kw8WM_VPsp?z=@L=LBp3`d(L z!=aJj1t+Hmrh*J7Y99dj^f?b8h1~?_nhY^wB+e~hBqWD3P=*sRk{K;0k#mjZM2tkh z@g}jrGf4c0B}Obo0N!c-+)UypmKZS?0msW@j{hQ2N3iR0F=8_Uj(0qXUz4~ef-eZZ5$6wC_qLXnZ zE<`ci+Z)QxP2(_?=lbPWL-4;DkDtzkPcjGp3kqEdU*x|3)mZ#?t6T*k?eC;eQ?e35 zvHu%|?m0|f;#4tu_;&1e8Ht_Wox%GYFvTfa-1&Z}D$7w+*XQ80NvDRS-(`6RK}Zej z$$iJ~Phq$ge9Tcomz+h_?uz?e4}-g-HmBS_VIa6q0LZCKwjrJ4Us(mADt$EeWBzp% zsx{*KeBlRHCl=^F!rl%Il{iBHVD&2xID;X!GU>bVLSFbr-jl$X z>Q!e<4+u{2l?uIZOx{{pawE0&&Kp^ti;2DB8TqWi z=ft#ucRvf&EbX1Y6B>`2;vQ_#Mfq%FaUE#7#(VTd1M%u9yQt+R(D1hwW%18lxo&sI zso{Gob6H*jZQkAhs}>;ZxfLALo1mJDt>kg;^x?3)6njgk;_@wX_oV7xh__L7QzKO4 zzee8dosHN^|AwFWg{;fbsNX@@N{8+ZRX3_uLA(}Qr~ddk)n|2L%huhxxNO0UGn^)U zX6^v;7Pd0HOVgyyItMM9imhzHe5lIU_Lm?&i>-9`0>?S6fL8nw;x?*w;b%cE+uyJc zP7rL}C+6XJ-19Y*cGk^EjXyWo`Gb)i_tw-)L+}XoB}nHY&Dq#`dpJBn z5d4}xPWj7{mc8ucJuBVymE>|3bBQloaqHp)GIHyjybpzuKh%_?8KxY-IWdmM_}hi5 z^cKn@eElya`p==ysT6_d-eg0{3>;2Qy}&#NmfAc}5tV~rF?X_V8(%?+yN%wT@Ovc6 zcXE!GWIXHt8cycShx-c`t}=wAERYI(h#U@VNG1tFbrVvdT+K2mPr%gIcIhuZI^wJ`mz)@(n&)g53M<>o&;r}! z>=KL>J~Es|O0DZyVRH!P#cwqxpJx-`E1II?>6+771eSQUkQO;k&pkxX(rjf@s?yCI zx7su~dgWe$<5I7YkbW<8Mh?Ny=Qiqb&rM^T8HHicZS2HBZ!||+x3SmP*zbM9nI}6( z`DFB7FynORXmcl|?>)LhY4YpljJb;W4RnLo!J17ia4FE2fkC_sv|$*#2b;bO4APf@ z9T2~yikE>w`Z7?kFN(#cF9U=0W#CYVCJ0We;nijN$#V^TELG%%0AR(_1|beGK#fnByQ5}59Bau$i6U3Qk}*++QOadhYa(Zda2 zTiok1Nx%%wLvek!-)O4QVE!@A%~WUKTvq2qROL8qIYp9?qw6+PQfyhN=TXt~KJ6bFectn;e@}GU zGvY9^3g;f zljQWgqALXawcun)@x4gTycbz*-iu6KlE58!ZKsK#jQ_gWr0se#ByK%M;s^JT`nAZI z!>+0Q9zVK|YWZ3u_hPktE%J8k`L%p4l6z^V+Pqh}ub-w|ZC;YBHZMt5o0lY0t8wVZ z`&8Ubf*0otllz$$CQ}1Zq4;i99~1HQtDC7!xGjq>t74T>UH8KN)m1D#b!s2%Ut{)5 zd1+a0I6vTKFMeIM=3Lo;{p@6^4sH&_(ax#f$jEo%KHbr17Ua>HUA=DK-9FckZj(kZ1aO{Gg=_!z`vEbr5b1 z6d{(jeT`KqvXmV5m3n>1A3cdrH zY$$t-ieIVVJD~oED%!KSz=ML(n*PYDHB``(lHxm{{-~-KsHmfY?|}NFtEOPTcr1RZ z%{!nehN}5vs^(CCB=zPU(9}@u`+HaE3h^CKe@68sP^l~Nqu&8d@f}eAd!a(dBcegm{bzX4jJ-vBK!Z-BxY zAhoDhK$=kLKb#tlcI7(bpnN2khnTtRpm+ujk#PQ$=^}&B!ua?CldLtbQ6SGxm+X3|Dlv`BQ@o6 zktBatbq?COFqf+&`9GCbLuk(BGD-d~T3DLPb&~wwN(TdJDXQTihn12hKw42$hZeT!8;bZQs9)k%{1wbiMSK<1FLNtig0!WG?}GXjZpC+ywifYaP`}Ep=!}xK7x8US zzq4DuJEUE)YhbFbZslPR95*%t>AJgBXR%6d>~-jRxfRz!N<%{FecXyRNR@7^7*efU z@h+qqH&%_)bAVg<6Qo)uJhAwks&!L`!-O2_?^==H~urt&xI}TEV8)L-! zaJOs;qzP{9bYOe9Wp_fF~K5%D5PfFISi>B zX{Y%E!3$vt)bj|tn7;?yVmo)E{_5@Gl2W)+*bcw+KH4tc7t(UuanW?g+9l&4t+bub zk+Wm%k{OWJ+0Lb~|8aK7Vo2+4rv_=BXqT>r)M`7&pdL=POE*K>WIJyn%?7(vmZAD= z=OVPsQ|&ar#kkdW9zd;}W~YZh+F?6~Kswz{p9pD}?M#E^&#+6IAX&oM6Skggmo9}A z6V7?4qjT)iR!Aw~48&M5)h^o!sZ2Qhz48m}vi4Zts}xQT*k!t1b}*zW;k*y+GwtF= zNY%n&3!Y^cH$$os&b!F2w9^PdLNCY_W@LAT=W|u;86`aRa1GF0F&KUO4*$d&(}I z0;yFv&!bm8Yp0tbZ4yo+%D381uYj}}mVmU*PH%#=RXBZ-%8Pc{{~+xU&Y95nid|aX z0re-GBG~#>yKE37OFGMtv)Alm{;p_DIyI>2H|*l6kW$j2XXIOU$$Us<(isYyy=|u- zfmA7-Zy{~8i{F4$C7ms3XYbl2-$JUE&ZWr9dv<9Y?{ziOsYi{yZ>Osu)k>!u%J89` zJ`7TwbnZc|d~ByDLaIlau>58_Jr~kA>DaK~r*`^oNDa~{g@(`V^b3$0rBewFU)pK@ zkmeNWOhsw8+G%?a)Sq-VBiG;9=}JgV(&4trcY1u8C!Oigw?hwE&C)4GX@As1)oI9`eRlg_hO}8aH=!QlqIe{vt0cnF9Oq!v zd#OnCw{9p^>lY^W$M?~MNEI61I&ttc4;X{qDvkGvcx(qkblcbt9E zCJq+GQz5N%oKIkhLt$M=>l~*)^wo*dMJIIL00HWk{)r^9HPYvMBu;Qdz|5g)|#PvFM8Wi#Ug&?5BwKT_IIPoK9$i6GVC# zr0R&X0vb*irQ;ygM4ZJAei%^4tBpA4fO)2Apo{&h$V5obGUWtu4c22x|h;S_GVNb@wJDG}#4=$j!r>h{~Xeeh%+0^ z+eKPdq5dLHT?AjT6X`u6S+4Utq&q~KKUf!Yoe!b?E|G44lyaTJk;*+HJsVP)>x@Qw zxlg1QL#lKgI?dfL(yJj=xy}nH!!nWn5K^`49E-9)EYemF)Sv5Ih+0`L(p@0cy3P$? zJ|@z2km_8AC$Fp!>B*4lUFR6o(UYi6NaI{5iq^AIq^kcb zc&>=_T1X3B=Le+nsz`5z)Z#kxk;-c#UC;~l=Q@*Mmp4RuKS;}6hwgx_B7F*^m9Dec z#gjy&XFyu#I*%fs{}Jg0kk-3SE8=a^QmgCS4vT*v(%iw_-k zvq=99X)Ejk=`&H%p*QN!bzn~G3z6nY+q+z6IrME69oTcMsB=mTPYd01Vo|3L!oCy5 zW5G;Cou9zmF4AW}DvLVJNajaTHVaZ^)L9GW&!Th@q^hXHpVIkN6h8r}I_lg9>^D*T zCZw9Evm^)42hs5>NVQStP}B;3+<>ppa0e|s22DBM0)A{`uf1Rv`6xbmD|gV?j3|Wy zdV|JhM)mBb;3(K9cCnUX1@s4v&C*h;fDWOtrfB+qP*qkykI)!gLOY^Rl?8MOja{a( zssj3i#x9Sh2Lr1vpi^jUo@Ulqg#(~9!l~p-xUu1^2#~|MmIjy$n~B(Y zN+qWlDr=m9WhjWrMvTHsVp$JP<|fUNX!R;K~hg(U|PU|#uptO0Ge;gDmT z*#!e9lFf2+T9*M-U0IUbnkp@}Lyc;K-0A_G)nx!>>U>K+=D~bf3c)tREIE-MZ+pB_ zG`ld5VV307QvOCBHFIjHdAdxUAgkd1(kXO8)jJoUDc^TFIV3B3$v| z3^qi8l0!V0Q`!)0luYm_oacrqTFDipa5;&S-w*{#9`In!hC{GX@~TJS6gf=MO16>0 zwJOe=Llh{y**IGdQK00%9?a=`2sTP~c@)m+!xXKg0!V~Qc%0~mC{QxggE{jL!A8kR9)-&R zVTxALL<-jjxkeD8K*?Pm%teC`Y?M6jQMi&2rf4N!fD+@y4Y^C9xs>A9|vjR&3YdK|8H zWaAi}zj-{agJk7tot*%yF@P&1*?7=-xX0nTNj8qrd6viHib__V)_D_o91^(Jl8pzQ zt33`^V6t(H&Mh8~YcpAST4x-v8Una#lZ^+R{X7oWa&alqjE zPc|NO-s5q&DwK_5biV5GxL%Z%r*-~79>))^BxU14XE`9n;rdcGj?p>9<8kFFD^Kg> zc|mG8QHTAgcRM~G)g#q=zF+HcxKfohE&Y(^xj!l`u3=^6X`TNikE07$x3ck2P8*Qo zaLp?l$CR_H$Kz^PR-V>59B_N9%bSuzFqPDx2o1 zVJGks@)0RCGMC;0N%8g4k`@BhK*FWCHfTqpeC(?ue=^mj;|?au&dH^^HgVNqKYY+~ z>8=eropIUVxs=xiy_qPRCYScwpeu;-nZ%{OHs}VTtZpv-wLy6}P>jofA;i3s<^hRu zT`&xDthHnx!dxf}!?cklk0Q*K!!S%ES@KN6TuKbXw2~#SBFweLFsxs_?k#BfNPDK5`27h^H&MC9ji0d&eu=P@rVH2ls9Z8zp%_R0<=rP_&YM zq_Eb`ZcBlZF&;d>Eo_ub^(ZT{P_&YpNnzb@Y)gTXr#<+Wwy;sM$)j-fHM_a~M~Zp~ zw532vM}#VvYqa4M^e{Zcqj2FiOwlE2AcgIatGOWxl+5v9F7Jk5Q<6J93fFwY6s_bX zQrMcg5FDaF$yN{M%5VraN}@pYLm&%9E9nU&#`WV+3Q#bd5Eqj}kV(9eFjtquuuk|2 zAZnE6B6EPEN7{RdatPw0a~t$UqU={(gl>a=N|YU32k-H6gGXi5=@D3qnlwLuYE8t;RkJ1|nb4ff5 z>rwhg!dxQ{!+Ml1021S(c^KBCbTwhFqK9EUN{=MWW%V$uN9nT(bDcd5>rwhz!d!R{ z!+Ml{5J+ShjP|)tQFnU9s}}(I8Yz^yM6Xj}1WSHTfb7cM0GpyEX*e0hZH&wJnRqs1 zSh6Pqm0~XEYgDgr987?>T+0tc)2Z@nIx(*62M`@^2IDcCT<6y)zu02Qn+SB4_)51A zkfvV{^|H*vr0|ye06H!s*xCwjBQVB2fe>Pv&UQxTN;fwOG)2eb89$M?0C3MB2=KI* z$OizpeGmj{348@WopYmJj&xGT5%>uJHx+_0&Li+Q0PZgYfvX85Q3!4|1cAE=@Ng3D zIs}2$1ZZV$KxlxaGS&GXqPf^O;H6O(&7kZCawjP~?nfQ@qjJwe8Vb9DECm`l8XvrL zDlD#{usg{8NSTHYUYf#!8w$IGJc*QB@xe<|XaYlFw~$SwtilH`O`$OigRX?8+1xzKbxoWi+hHt<0d| z=LvJ4M#HMys?Vqo-4nNv!`&OrQH9TNXa-A00LQqeqdBTIWH_{iC3}#=ogU3mX2@`8 z5IhLU;r@^2C|hJWw2CFqC5O90nxhPo;m|Y~3drGJk>)6?WH_`BUV_Ns4wB|5(_}a_ z5>~;@9QfNb2xxaFjYAtP~b0V5$f+=$YgkdgFgIf5LX*P}ThBN4FV zU@AP1#AkeB$WjFG;?532@5tC^J~3o20+!rG;)^8aVE;+04%v(V>W8X%R#0qjpBOS6 z0ZZ~*hOz{=>J{K5pL`?2Z6)dkSnGf{%;RwnO=~C5lBatBx6?GB$8ow%s+1q{0p4MO zAD?nhm|Nb8yBIgubUZ%E)G)6vw`V zL@9IUE|~ru1jykgUJ!VOz^_op{k$NskpL@&yKgq_Xi2&zs<3kGv#Bs!-BF_@`6(?R zio&h907X}CX9RZr@Nn!##@jXGJ~?MNW>@W#DuHAYK)gfFUPvcqb^US>Xo=+Yq%DQ| zL@I~h8@7@ihd?MUz8V5(*P~-LC)@o4v67!4N^+0fZ^5#X=hWh(ORoARWAaqOz4N;x zPAXY46hfc83J7J%4QRc|zPZ05O8ewlV<1%LN+6ZV>-L7wKmIn5F3EPtL2{rWG)>WPAXGAqA%+PEGPpyyGN?rKUotP1*-QIHH&ZtxNJezT}Zg7?-@0w4;>J zn4Cf3xKfs2y48`zItRDv`_aOR_l1RWDvCBC7x4!U!DG0rXbtZ16>YkE#w$^lypaXW ze%Gysp>+AN=U7%+EI-d!?AkgA`2{B+>6lgcFW~qtdZ!4gYXj~Nj}Hf)>Qwv?@PXrz z$Zbfipv>jB?TcnZwu@do0I$mYLbF@yL#;OV`k&9H3t(s)DRY-%#1NvZt{X#}cmD=x9^QY`z z(MQ08h6E#(w^P0gL`PK=haI^K`+e2v>9_G8J8j5^BjXWBB-50?J{ z$=XFFNI>W+De7r=As_)*MgKrf?V?+uO<0oEm?BtqEdmPn!ky$ts)k|pNTcvThVdhH z6vG~a`3sL>7(Y@C47(boDm;T>{79Y8un!qFi(&jo&12XahTXt0exw#L>=dSVFT?ne zTF$T`$Y|j+4C6=YMTR|v`YwEvVf;vKVpx0fK4lm`Qrqy+^#kzEPIu_gA5C^A?$dji z_FDvUx-#)G0JPs<7>ggNJrPh=$XxWso#aPqAAH1bLg|a@6G!2G&`8M+ks3k90LB`} zyTf>Q3hz$f-DKXK!@Fs`o5Q;%-rd5xt9ZAJcMEv8mUoMJw~=>CdG{6X9^)OepIXJc ze!P1bAMtxoaYbViAL4${(W>u#k6YFE&V&y1#S{bfj-c_CRoegT!|`q}l-j;1q2BhA z=c1JM3?;;pv)K-2#>qmH;{0GJZfn@2WTkafC7N7)%w8*EjR@Yial(1{mT2Juv^s0w zi4&&cn>tYfi262yrsG!SB;P7I8TyNR<}jEpkv4B^CYx_n%*3{H$y7V_7SuiAgM6xJ zr}PqfmzTDW^TF9lT)mhx1*>E*a#XZQMcv^?_3!XLhrE>>f$|nzrAX}tddlBGXVwne zDy0hUT$ELaJnHR77I4Juk^`QYS;r`c4&@Nm&&qVh}6pucv{EDy(R-%G(GZ&m_e#e6s={B^~mS zqiU6->13(`GDrIRIBIdQg1;M3g5M2bjO)Ghtcr8ke5-WuGA~$V)~XnXH-&Z$8heG9 zN|VohtK<<@`xKShSAMis=3W<%Ui&5R+NZj|ytr1$6j-f%n9AxN`?Sd_n&~U7?O7_? zFcMUO1L@N3--X*h!^e}r9;RE7u7eW(fI`I()ru}5fz3%LL%Ulg-8g_uR0-VSYfx(V zVGUn~&B?N!B6o!s*{WCsBUPMi=TXT~YxwaKifoPGTU5Kxa^et}$WC$;{2T5p z$(OJc150K?dBt3nP&wYVbncmPtK?ucKf5Kq9j5C#M(14q8r@&sJUjrB*F$BsG|G9U7AM4f8=B%?Io8<3yH)tSFR;5rLUc@mZ8 zh7OTJxseLxy*S#6?rk7ef_sz^IGTQExHA!hE|&sskkjye!b!mhvy8ynEks6{_$(xJ zlL`O%RQQyb@QYPKvzSm^ev(8{`IorQP)GSRtMm;M_8bNyKfgE$G+`oHat0cHmkBWO z+j8@>I;D;Vv=?pI-_+U$U27F+h6T7(t!>b?b_C=Dak17m=vuoF;#mwj1tGmTgweQ>WAr;Y?VG^kWm<7GJ4oR zbtyX)AB?I|V?y<_+8TG5in15w4l%83wQgO*P^i6esn)exx304wPr}94wOY5XDi+sPS*Kggdw@ADQC<2~`MAkWbga2i{q}4l@a5>< z!MaSU-^xVA7#eRXOnr4Q#w|u*$9>JjK<{>!n{amAKTNppxNIHhxIdVVYfRPsc@u$V zycjm>bQK!ijmmVjPivcTk%@oyqHV*B=@w|yS+M^vD4a6g0&Tj!2eF-jjKWSk2{)%xVEn1LITu%HuCce*X;n4IaDy$nTl*QTKGVo>-dAXw3Rj6&5qpEClVDnpPy zn~MKDN&INgQw^e@H*9lsyJs_-jRE!#!@AICS&34k^6Eg?ekN>5CMv^ z7Y%7PO!R~aXTxU(bv8KY1fNz`r&rKZJFDis{BhkTu88RW_M!{;$miQa7u~#9!G6ne zspj29H}4-Ie}jw7yNhn#eeoJwj4SUgbe%4`f!_=HUkt0qBu1@bqzI!rt5RI9s8Fd zxYYCVpK`CMXrV7003LsN<5(Gg8_M0|pVG!>b>0FHRp-v%v6?^ZQ+l4kyed`CH-oBb zebsvjvG!_2;!_Va1I1ja9R80z8ky4N3n$?-9;?3$88nc(-?nPoK~PVbRU+A}RmF zOg#qGdq8ag)o(WxH8>R+t0|#5D4cBys*u$|VQWz6Zwf-1l{moUZwItyY8PWg;sS&4 zEg^_-$Wkv6sNHC>$(kBBnVzFV(-Hipr zU1bn1cHHixR^m+){%9b~v+PTOuyWHHzwpDDJ+H}H(SI78%`kec5m>nBk^tfgV|Tu~ z_ruJ}w@+NQ#L@ftmpI6fMg$3w`I}ecm#*BE24r)(I!M6p8vUdbP`$Ux%m%b+ggwr# zwcIU%$gBS!k=Ofq8Ejg>wjet)e=G6=HSYYK9891?H`~OahHha{)n&u)k(sb`AZ(!t zJID`%Isflu?=*D&mXJv{`icpok-L9j1Or+fxym~iY_%AMXM}?h_9e075pAK>QvbIX z@f}w^lwKBU3VR{?4L3E^L+NFqrm*)yUW|(mrI&@8!fu54DrtNuy)4ueHZ>N0APnO} z>1DA$NA4zsoq}1^~I$g0b_78f&xeVS-6>oOFckl8_k&6 zn7c}?)0%xDli~y9c97W)RJzM>b3X+p^(1aq;3{2|y#W8$MAUnL9BQ)ve?CCGijmeH zsK547e#T7k0rDc$uE7=k)}*1H42{OPN>~^Wti)Wy`x7bXI@a}RC7v}%SM<*GGRyiVN4Y*AlkubS5J@qjL;8_Wv-#>_fz#6jUP+I$T!cB*a{v9 zrk5EuEw?p5`Z7C7HJ5rbJYw9ltIg25zo{h-C=RYnVP_)4qYSDt++YO0mGv-Gw7Sv@ zP55qaWcm>pmn;0Yu_0GrEN&58<*L6&H>lf8yRsK?ky(vrEuuS@JtN66@&a>{+!^t|xs~E;*wL67c zyW9x*EnMzglbfFhU~EA?GA7V#6zbuxsw4W4p(=Z7hv?E&i;S zGJmOWiY`s4r7Oi0GpwKMn_&#+5S(jVrHA$Jk-txIsbT#pJ*;;=9R~dn&D@C@nRV;Ud31@8| zXu@@~HERi%c*U&VVyMOlc$_VU%1cD83Gd;T`)jYXwmO}PA`KCoC$&7N%hMYfv~h2Lv?~bBCDU` zQpLh=K5(JA?gXqM;9_+S)e4@0xENPnyplf+(UXbSo`}|utHb%z@cR&IZ(q!>?J)VY zv#kR}F1Iq|{FVAga1Yd3TMYKixXK?wp%2t;Ud~@NyM1C5egfR&)wVv7_+>hgV~@lj z+78HCM0pmMHHNt#`4CHjviNtsbbw`_A#(O%q~Ac>fC#OOKYWvRrGtqRGr3tSeKbyVNaBnU4bv>`6W#W=qIrCXOSSzjy6ty33C$B1+kO|__V!xr0Tm>G0>g=jYvrmi5zVrJqONeFVZI5Y+xjJ zfziSZ5>$Dkz$@}VCkS)ttIhYLSf9N}xVe+Q>&Ih)dwEl)B_5&_8 zBz$12$6<#C3<j`6;!<7XBi%I)hkOXG4(N9uc|GGqH5zmDNCx~SmvLo2Lp@=qwRi{yU8#)Zg6VrF-O@0z=-6mdk0O<{UyK} zwi|0;h7~O~!wM@P+Sv$PW=ZUAgsFkJV=z8h-G>`Q>xDHrJ6)x$x#Iy#{oSem!J4hd z8f&iB)_eeEy%U$R=4x%tHzEHU7e}Ynwps$%3UMdDMm5I$ zjSzz^W}*x%ta=62QwAEsZ8RJX3JVNpHo|9{a1IJ#eOtqnPMqv9`VMs`<^gR>K_7NfFX*Z*%+YOO0V?G>T&gY1(QV;T$PY1$qtqNdO1%R41za4F=IW*(=1*2te7UaT zPk{axm%G3y<%l#F>M~iKKRGZWRa@QfHB3{%R+py^#&g@Y?$y~nUwSRLHaBDG>jqnc29l<$(?{jC`L-c-uZTahB@<>V&~{k zEaBzb`!Zk@NJDUSz_)5rc0Rv?gkR{!t`(R53bY~b0De2Jboo&15$HbkB;@4`8;=YW z*lHtZ6Xe%%6<;uEf>mh8ufroLei|}{OVvb?u8G4Tcg3YT+eLQOQfWMOr-73*&sKb4#mAKf1jSqxfWx{6oVeFVQ(c|th2)_#DK57Jh zU8==Up~l^0!jD1t%0PJg`*dNpp?sW^vFXePzrP9pYacvy{cxy^a4!!_^hX*3zfV?x z_g9@86i`)}<=zlTpohYB!grZ)e&OtJKOC`DHCk?aW8G$C^df_+Q7kU4C>j9%pu~I? zcAZlNNCOO2`x7-E9^Z9{085+e+JTUuMb4rv3tax-k{(<7$8-g z9H{@e+?7dP4V+IJsv3*d8i8X`t%)}kx&1JZ-i5oT*X*<{HpfFoi<%x!Yc1V9S}f8w zn&s{ji16RtBkaf)A$pjJpmIOj2+aLoCN}DzbaEh7e|U<^nn2F^4yE?5^f&p3U;t2G z|AqOx7H^qqlhryL2o*JeidYF0(jr28KYM6@XVd@LWTX6OL`5qmQU-v^E z_;QV3?7yN8?*yEjRkvukqtyTHMf|Q(PcGmr=)}3%-$>+FTs_ahki4K5zkk?yA{NhZ z_2)@M3u?HAG#KJQ1}#1WK?m`*wgKXaxCVTCIMz_|`E|xQ5YNSR+zd>!>|6ZTp8?!w zSg2k&KL`9Wu3BijRn6ZTPRH5`u38Rdx2Bl%oe(d_bv(Z>X79A!@hv*{m!PrFYfuFZ zK2ULzjUn8pLHz=t-<9lY%YDnh{1WOWL*WYb7!l*MBrZo|w#PqK)+`JzKes4ReL>Sr z-k}A4$2C4)H+uatre;|pb~aY64TtR|+SLebFYZA`;Og`VCbsF>%QH(dIB`eVN)6jM zhN9?L6Hm3kQ;fiG=Qf!{NV*_My50!ox1>;M&PR*Bg}+7x*HN-KTh5)A#cuV+d7%V zfrfgXPqPxEP1xm`u;^(fOquFjBk+Z7nIU1!h|Ud?nvKBBwHgvL_n--9<}6%(Deb*g zJ6UJ)#&$yf#sI%xkS}6gRD@lD2(dG*=w=f^wUh6RpxR0DE?sK2lPSTZ;_~07ouHW+ zl8WBN2#g-B2_nM+h;;%tXYK2A?KP+3>ni-7=JnOQg=-^Lq7JwVL&fYOjukBQi5wa1 z*=m15ac3W-qH!N<_9(viSi^XR%p|q)vqWckgZ_u1>tVcCsKI5@UL0I<8<8_y{pm0{ zX%PE%ABbIX^_+;^h%@?g*cc0OB(D6E#!s>A$;lTGCg(4mrG|~O)Ua_5;InY`p@O%J zS{uLK3|l1@A}nV$J}w61LR?nqpN4WYjETBpYAEXDI=o4xsq$5@dofd^1~60B$z1eq zeVDiSri=cE;_&q#=T|^?Ak~|2<;W5E2~*r`ppXyYN8D`1^^bNCDkotLi5HWLJz~CG4k}8WZRTN?%+mfm+-Qra(R8a5EZL1EY**l5VLgR(M{o-MrIn`x;G6K8I$0jLu8P~)k$vTDKKYtCEV_CO? z$a7!L?uI9YI``$BI7V@>Hm}y{{s|WUen1hTclShv>T_T6^tmsU{~vY0Y5T;XPZR!E5DopeD#U)Y?H|TZ4#N+eknsAVrT^dy6|k1sKdq;xjOG71ed7OO-jCpiR=<}x=D#X-30ansaZY- zQrEJ`ig-Kp#W+tli9C_KpJ2SR3*kv6#Ns^NB=S^Jx6vq1H^I-BevQaEj2PwVCXttV z4Z^+n6?N84pKcQQPmEHWzW^M0PsO(4`~l#|CMCq;{Qcj^e|5TX9&Zx)KnZ1W9&Zx) zFnv3UR2gr9Z6Y6+{2M}5oX49)wv>DZp*qgvO(LI{{{f*U&f`rYUzQc4)z-#&yh-Hi z(i#YLaXJM@wv~*6P#@>g1*j+zsdXmvQ5rz{@8X=`1K`%};xf)WLEej#xM3aXhRob#08Wm17 zc>_|FEzOB0J0Mlt@+D*uCz=$YbTzgdiZpSeNqqDiV0wP#CnqDfy!yKHGrG&vfQC8Rmggukj96Y@ON5l%GW@1UlHG$)$8 z0;x>M-l!a$XtEtrCC)X6>2RV+X&LHINOPje{*bDL91kW=G&vPgjgaO zc?43Okmf{__aM~^IR`nyi6(zT8YiSV(WG-Z>QBh_=u$Y*SNAu;3y9SbL#41lyyNOPje36NTZyckU$Cz{NL zv{XoQq6v>+UM{3L(c~>iD}^*Cn*0Q5osd7Fh2TV!3T$?+7t)+)auB3eA)iN|!igrQ zLfRyxInm?_NSk2^bl7!vdKsjxLYfmz-h{M6NOPjeHb}dKG$)!Q+QV8>F2neS6HTfh z#iTSRnj8u#C8ashWCElzDTkuR;Y5>5AyrCgPBdwOR3)W3(PRyzYAMZ$CLcnok+L2& zjT24yyN|U}niEahbwK?|X-+iR7g9aa#JIZIPLG2$PD&dV#EB*sLTbQoz@vBIL=*nr zVxyGiM3W~VO_6dcY=#p}-hnh-N^_#gPmr3VJO(WiCz^14nJ1+=(S+kwvy{bX)lX5O<8Yi0U4QaWQ=0uZmkXA}*PBgg?(mE-xLYkH+ z{uiY6QkoM@Rzqr)(wu1WA*4-Get_D*;A*DId zWCEmJQkoM@u7G4YatTIloM>_{q?jXLLv7+jlUE_79BEE8*#@c1k+~QZaiU4G67}au zbE3&!kg6PMPBa+>soIgZ!PYp@WD2AjN8SO0;6#%fA=Nt4oM`eSq&i2M6HVTQRPRW0 zqRB2u;~e=s9&b3&q@ok*&ygMQ_{NDQ`#@@R7NZAJTM3evFok z6HVB5n;dCQG`RuTJV!>63Qjax4yoCZ=0uY>AuV*IIniVrq!vf!piR_@lEj{Bh85>mqJ?aNOPje9gtcbX`_YUM3Z%pHaXIqXz~f9 z&5krDn*0T6t0T>cCLKDX{v6pG<->_4gCXsLt?{mb6HQKlWJTnosFmYH`XWep!8RwF z+zKfbk>*5`RglUe(wu1WE~LtcJj_-nn(Ts96_Msdlfo{jzlby^n)HHH6OoId0VkR; zUTs906HN{WRu_@xM3Yk?)kmZ`(PSp1aS>@wG+6?vAtKF*CaWPeMx;5>ePBGQ~_ z@-w9A5ot~|$?J;xi^xeRJ5Ds|4ryLQniEZGAvH&&Inm@eNDCv_$q!wsLyT*wo zS3_DFk>*5`yCE%)NOPje^N?0XBv;dLqRAJK)d%!QLOV`082~BeN)9?W(PT8FGFO@tP0oc>=}L2=Ni(D>SMsNxaH7d0kg8pI zEXs}(O*TTRaiuxYsZPBf{gLjAdNGsY#HXfg~^y(`U$CZ|Fg=Sp*;$rX?q zT=|<+Cz>pS)ac3%=+ijSAIMI=3h4&y~H9Yn*7Z zH>8!WG$)$yx2V>+@=@dyCz_lNX}v335f3MtP-=DM?XWmbG`R}WCRdKdNQ)Cq9)`5p zl`Byzn?;(xx3d*?frJxHc0k(UN|=);n&kIF{khVdXu_UjMdc|mI5>$kdrmAW`ydP_ znpA{>7#@G(GJJF|g@I}#Gomp{2~KEmqRGWz z!5at>aG(i=Sb`H8oM%EImEd_MP0{q5&{LM+aV9v;EZ9Qa*!-|WFQ^b#61Wg`@sbDF&j7SJi?mhPSln6SJAT~h zkvV5lsVK4~-XgFFmg4+L18N#7wi37;0OwLcz=rIQ!+DhkmKEU}y@iJZMD z)DnEvb~%AF@~4zJZPT%J#2*o;^l_f00cy8I6tc=IXJZ=BcB_Q!a`F}o+?VVh5%2Wf z0999(7(tY=*#LDkZIH>NG%vog!HU&A$ zqj0Jjrsy&>lfqnb{u!b`30Ei-%n4}-HcB>l6wXq^6s_cEQno+|r>-Fil$0S&ODMJriL3YVrhVGdEC-4a5W*C#WF-)nYYHJ43O*pj1%?o0690uTR~*8yPPiC|>g8O9 z2v9hrcw^Q;kE4!!QO`7v3u-9Qj}Sd@@xU0B7MCrurDbcKJp4uVYA$GGWf~MV4$l z=xp{lTzbjIF*;XxJTA&)~ln zI%~+|aKgo?Y&_^}@Hkwe%EmD|FY|a@yvoYcIv*sDqYIa{vhkqv9goAsu528mbC<{C zGFVog)>#SI<>FX24)oO#=W9=HbA2K|BP7NA^{YlG$^P<2kO(X~PQ5M?*wnq3?8NTO`;T*GUF zP9n;t$u+$;=oLiSh`7er2EB(UtD9?nZO|7G=&EyHGJ~*s(WZqgv4t=f1Va(cdpHk% zak)YmhPNQHCH5f9CBrbx5zG=p33ClG4AV-MIEgS96~nN8^`1$XtBhfocCy56gt_b( zhG{5EJVTi4kYSjXvc!LZsAj-LMokGmzIJ&qR~>*3{!Mt-H((lP{;Mj z5CwhWBoF3-We7G(W_lE^VumSN$r4g%Z!T?yC{Xge2Xn161REt=JPH>;!xXK=0ixRW z0c|Nz(%pl(Od3u>Kj;tjC|o-YQ?!yvq^yOxxS$%MK*=>8%oWxUY)Z1sqj1?ZOwmf- zBt<<0D%((?ng zND5o?yKN~@@~j8%Y6}}B?|T%k6Nd|{mHY{cyH8sf3Mvumj%^E>#P=u6W##M%9}h&0 z(p+5*Q1mGMPof;9x!T+Yy@lu&pj>@!gFZo&9fGUTZP1NGIsR~Ux()gR(X~jKtJQ7L zd<3eR=jwGEv=32MI#;vXphpsAZE}#BAQY9Ucy`p55rqf z5=*>9n2Y0Km`1Y1=Y+Xh9)|TOErGaPJ`ck*6OPd#%=PpztVihs2y=lw4C_(4fiPFx z!>}HuX9IDrM=qc8De6wIcyX&=EGC6A*VuI`j9`i92$22LZh%eE5^P)IecZZSeaytO z8N(8P;!`Q+I=)8vhGB_P$PRD0h#!ciQ>{jz%LV-aqT`KVJZ6&%{2Jx=SS-;(>BjdZMJC(U*|ITW$bo3M1Is3U?#WLrOQmyflTyH57IS zkqg8*93Q+ig#|Yhb_r2U%4B@-(iED&P}nWR(WG3D4_=x=V;Bm%hL}dmWBA~uDKv|r zuzQGGfw4G_1<4 z`i%O}J#hs&+_upiRrm~tX0XI2a=4kJIm!YV4sBtHKgi)0kLD;dWH>Yk9)y5hZv1GD zvPFhNt61V-a=0C&Im#dz4o!ohfE;cTX^yf=hC}<{C5Rku9chj-O@>1wVHKR5pL|Z( zJ^(E77eC%1k$X)#v5=9tw}6q59PU7APRK}lwCn`fo#1mqMj`+svBe%j;?Xl01gv=6JkyDiXNmnhfIDXz(Bt^g9?;|iyu$)NKE)r{DHD@%=Wm!;25Hmxn&nj|9k@EaQ7|QKN208qD)s}t z!%rh{-(@toRdDohyTxx~Qt`*JZ|!!^Ux0fj{uqTSCAd~};ND2$*rGcD;k&zk#rRJh z%hBT@q-%FA;A?JN#%^MZfrd|^!S3+{G{pZzLUy01R`EB(`zyhTGaL4x05#K}Mk7dVB%$b7}w;)(si7y{9tB60+9Xl_RA$eq6;8B%g zjhEd6Lv&M-=>MluGBLRo8XK-kvW$>gxX>jj`!`f|RIy(2ROwjdW03i8S%J0BQ$M^1 z^*=00TYZfS(zZC$x?&QxadD;jV=vo^*=wJG_$Y&3M6R}VVlr<)e1$=WBjIiR*_ywB z_$99VsS~X4OPT6lkp5r@-F>#JdPEj}dKG#_RyG99QyGleNoWqKNYBjhNr> zPE9pX48>Zc;8FC+XgAx0v))q6jlc?i&ZGeYr(QAPY!BOlX~*TlT;0SX>NgFoj&B(% zJkN9vQ2Cc4T1E2+Y@2R=P8;n6-*$^V+kLETSA$~o#+7@{DQ8$8m(VH~K%9h&R>9fp z;4g)AFRtXDCb_8y^vkP?Rc5l$2gPl_zY##;H~O_r#zgp=RWRJhk=PWcqRpy%q6y=k zfA*q*{Y}N*sxr3@I+x>8nY&eG&OH<5!&NX6#k(!Ww!Rm{G6ublgcl{5ax=tPxLAbS zRS`acw2>tCz{RR!@zrhoJ~e_$>k(AQUa~*LzPR!yS&O?c-~tGzGvEwsad!s%2f~}U z3g|SpB#*r_c@|#A5ww@`2is_0^&EKkOa9N4?@pB3w-yXC#!8M|rzqC5K?v4K%`^ae z;|Cf{ct7Rczl)W8-{3z7BeDjPe;K5*UqBmr78;6O4cmhhf%DcZP-7;C1AVHM>|ny` zGGVE%CXAKZV8R)HZxeoz&qWop6DJQe$b#JvWG^)?kk*5Nv@SB?>--p2(zD<@nXq;@ z1>&b3Hv&hR9JCqVx^Y<_=pWGA_pJamK&#xHW>vh6#&cn{sj+3ccF-aR4=^oqx@wVU zpsr8GrCQ{4)gqTbyp2J4luWN=0?$Hv6c=0HMXL3E1?dZ11sw8c6mlH?8RB*Zeb1mh z8Pxu4j1Ra}8=a}zXg7#maG^oYR4sHr2z%j59$~Wjcd+=Xaiv~25UtY_`41F(Q;w~Q z#?b@TY5#$tf}(wIDxMh*&&DZhv;m&=?NC3^*x>|ahiS;d8Mu@kPEdAu6ykjhf*np! zcK8C)2e@d5law9G&tVjVoQ;t799oWobTBSunUj@e&V)D#S8~3|#gDT*q`O~w@ z;mR)8Am5kaQjem;)uZTsi1*;46^>L^SOsY%u2hXlj!(rGb$0B2=t_2-m6~8MR^r#Y zXSL4DYRAcj6d3%gV6!BF1U!HwMsY%Qa40^r4K=Y0EWETNMnT)0S(Twt6#w zPFwEajB{45cG`;3Y0Di_J`tQ8&M$Jc(^ia5Tkf!Ol_5H9xu><83s_u*k1{%K#ptx< zPAvE*0%CO9a!=Ry`KT4QQQy;P%bld}>9pmZk>cDkb{bPZQ{U5R%bl$6>9pmZrSBIq zXJ_ksI&Hb<=zBVCxl{B#ownR_^*x=o-1GE3ownR*N^ce2t77i-{A=+)1*a_<48PNI z5iOa5(-!TOg3}hwl!DV1i=2Yf7EPam(-v1)QgGU$by9HJVu7s~Q*|$ugYYkhDaYuv z<<6~m0Qce%e3a2?D@La+cV7E-41I~AblU2XaoTdP&ifg`MapTbNI7j4DW|O>?X(4( zA%SN3Z;*PHxhvxR@p2!f)0X>0@;QQ0I&Ha6Dj^o7)0XQwZN;|Jv@4~JI5~_M<5!~G zmwJuFy~t1Tt1U2EeboI=j8c@}adeH-R+QgybdA$il;3f5jnh_?PFt>V+KSR?%l$C@ z1maglTVNaarlT zG@r<}l1UKiqjcJGzc0H2!ni1%w%i{|AAryhrPG$XGyMXD#weY(++WgLAWVtUY0KT! z?q>+oqjcJGe@{nH6HU=aX^cOVFfU4{E%&c#%XIdgmp>dw8c2#p5wt7LT@twAgfK=-{ZSi}ob-vS9Go*UoY3mV4<9w&B4Uigqr>&nMHTq6lX?(kC zitn^F2-0-lY3q1MO}^9CKOxQYowgQ1YWAJB)%n^t@NF?CPG^0J8fMFX}#~XwG>jT@3i$Aq)ool*7uM$`%YW&QdFMrv{eOZhwrp? z7^Gdk)7F`gEZ=GCMo2N=Y3o@?Dc@=96G&yg(^eE;&#CmCw)#S<@}0JhhE(l4ZOwpG z<2!BL1*z6|+Ij_2o$s`@9a6pTv{i~J{W#xgYkx=$zSGvJkQ#lbt*aqT@tw9Ffi&HB z+IkOClkc?kH>7#K(^ltps6XFn>qtloeW$IdkXn4Fty>^1^_{lXLR#)SZG8o4rSG(r ztU&$wPFn*Yt@oX_PJqh}{ZLNe<kUX#e5b8#kf!@iTgi^7Ki_GK z*@UZS|eDj)SzrciNf>X_xP`wFDA&fR^Bq1gEVv zkYc{m))q)9-)YOKME&_rTb&_Q`c7MiLaOqewkAQU_MNuoL8|ecww6Jv^_{lfgjDA{ zZG8u+-gnwcbwd65PFn*YHTX_jBOo>UPFrU|n&LZcT?1*l@3h6X+vGcKJpgQ;@3gfZ zQnT;0^(CZ*zSEYoC+g34+S(J+Qr~H-7SeLxY3pQ2D}AS}nUL1`PFw$iwBC2xdKywI zHsR4U;Iy>~(k9<&>sLsdeW$IG&Zs}%X=`stJA9|DF_3ooPFqtUS-#WO4Uq8SZJf3q zg_QE0w%&nM<~wbD2dUC`+Dc(}v&whc>ItdZciK7#QjPDl#dx*8)7E%kb-vTq3`q69 z)7An=<9w&Bryw=>PFwFnYV@79c0!usJ8k85Mg93sTiqZv`A%EIAkFigw#Gqf_MNuQ zg|yIj+PVr-i|@2`7o?@W)7C0T%YCP<_aUwHowojjw9a?hs_cgP^PRR1g4F6eZB2x< z$#>ei64GYhY3o5qTYaakw;=8CowmM*w99weQYXs#PFuaPu@>{4wvL39@}0KMfK=u? zZCwee(s$Zg3aQF>+Ij_2wePg`EuN{=S32Bq>wDmHi&A!vt zH;}gaPFt~Fs6XFnt2?A!zS9(Bg=Ddf@EXR$_4@(?`42nAl zaQE&b53n}?v^k5kRoGd$vqS`+;u(+3Ig?67kuC8RffWMlE|}41Kutr11=(2)fODxJ zP)>jx&Z{(__7Itft$ue&^e2UZoNZ+&mKaNb9L~HlfK9F?&L+x~ITZ__%!Nt$8d7Kt zPS7;vHWl{~58z}i5SJ#g#CoD6a`vWBOMHo2mlHT6|4y0HHXU0>Oh8upIM31mwc~54 z1ejOO#x$VqRs-4PdF!mi83}DphmSpF7p7s8)X2SrjA?Ub`O5P zEo?K)5(n^OmB$l!R)&(X8HVq9AVu*xdDP4~VJ4TB$bqb~&-q~nFxA$TD6_=(xj^z8 zkg#cG2a&=OauOLxz!YSn2Xj^#f=xl@dK6AI!xUYHrKHfvoPUNWu-0oH%n4}-HcGzn zD4eB+DOyP`5a&0Pj8oST1xl(tnB6-B8zmz>3MaT>idJ$SDO^3`tT#k~lAAo3)8G(n zl<;RXR0^CMhbdagd!%q_iWBA#1xkMRV9uaJuu)PDM5VxKb(o@+3?_v=hI8!@1xo(m z!JK@DV54M~N8yY-Owmg2AZ06*aGD;XK*@_9%z1kVHcCG8D4fiPDO!mOM18rEGyD(* zN_u!Or~Dz)(Tn7kKw30JP;c_4s2SOAmxz2;RS`dPbl7~DBmk`1ft)vx*%Qb}% z3fQD z$4<`$jch!WYMsa7GDkL!(fPT@;{r%lp4OQISd9T(8p+0k&Ve3>3ntk(M&~$>$EB34 zJgu{dJPrw5WXZ;Z&ZQoQOE1|tM(67ukBc%{d0OXA@;E|pxh5MAI@<$M94_Ky;~1T_ z9*@gAS$SG#BYE$kFLU818xK00Jr0+HvT=;g6&{ZZMOk@T=ZEBR{NNH&HXd|3fE0%d zOW8O^XO+j}l2cZm)>%&;HvqU8m5m3T(>)HCsIqa4&P5)Ni&t5BTIY-8adhFbRyH1V zZu2-??8?S5I#YmET3iOp%F{Y)0J~fq%f^Ae2I5>U%Z3|WP2_M9Eh|Usxf`&0UF8ym z=BZ&P@Dj3~6uyLT%`K1=UoS23Em006T!U+a#u2ExJlEvfpxueGb8?NY4SEPsb|bFY zwLwoM$_CFhyf)}1MA2tLX?e&YkY0c)kIm{T=Q##etwh7{yp#S$m;!xXKg zJrLElxhfi>K*=x<<}zsrHcC$RC|o-YQ?!yxNMYS`K{Z5ylDj>aE36^dD0#u7IF)TF zTFIBBsE0sX3Y5eVs^CN0!bVAzN12p`qLmy;3ftklwiGBi+k=<2g^iMHJqj0l!-dsK zmXpHP{9RiLlx*~1E(wQI(6h!LJqp)}!xXKg7>GNfEer)UgwARUnGzgLn9Iu96Fv`! z8l}0q9H8h?dOlGOL0oNagFZr({fev4ZO}J}vO{n+x()g*QI0=coo<815vZCVSF78g z-HEd8aP_(kdI(WgI#;vXpr;aLZEkJ1%DTrQu7VLeLkN0{sBVVHKZ#BqeVz#fM6C_SApSKPy} z9;I&qqCAe?@+s<0uXuHbHN}&pQ05xDPK90yaLXk?b`r~r8NjA!2^voP2-M~3V*q9h z{O}(Fm13^rYm{#omgq}>x8r?)>P)J22z0riA4pKgo5*;~CKvcM%I~pQqKUvB0)A;e zK$==}F|~gyDSV;f27r#s2)4GuDg?UR3(e}Y>GLEvBl+W~MlAqbpA;C}$Pu@D5N5uoQ8_ZWh}^#te*!0m=0u#^Dn zjyn$;P!1%zI-euD3m?2Rs$54j%5ES&A*BM_2)s1qJ><;P9J_)LK%AlY;H4=nuA#6y zh_0lJ#|JM>VZjZBT|(56av45&X$nnXDC`#EOj7Q{2QN)gZg@Jo>>A=aQvQt(UYbI) zm{iz3#G^o5Zj5NyH;1iz<~xMBQ=(yI4`m(=|3sKuCmL3^P^QsvArO~)DH>LGP-f9^ zKf>Hx(Xg_CGKq%k33Ina!>ats92!1{Ft=YctjexTq2cQYb3aDIs@%#98h(f{H)u4h z%B}j0`p`Cfog8l4XpSm;hC?%0Vh1_g%+VZWfeeSXutXZL%Pk(wQD(?+Xb?OI$>GM2 z<|tcaIJAl-P9%rhL7JlslHt%amS`e}n?#zUtdil-K6nWthg(OQqfC?G&`4MXCnt%` zAI%Bd2Y@9?*nqZ?$h{^_3>k@g3m6FjyWD}&oRE?9Xt^&rGks3TNCaRcDm;nAdwgQZ zQUomV89y#3@lBr?G8X|$?BvHIB>w6XLpCE|XMW5_JO{D#7dN@wj=W6D7|QqdkB-XByz(jb85o3w?lhSm4K}NWG{GzxEr+Iwbn9*Z9Q{awV0J#{^%)pTg#OZ~>hFfu( zq6^R)vdbMg9Y~5L4k4S5Q*O>Fz!JwZjB&U{r^zb)ttxY86Q#^8yI}g)5Fm%UcR}DD z0{ogIH}ZnO8Um~o?yuRjqb1(+!W`_ZsW4mJQKNC%`qraxCoVwI)k|NV7=KSRT3hf0 z;^5oAog)0Fysrc;oE~^IwohU`uK_jsc`f+8lMJ__bM}JJM+vd$W<0NAd*w2bRP=Q8 zwpibsLlLFSN?#6Mb-dqBCXhE2`#b~UI)VF!Kp3cmSoEg7A?%}mPd^pij0EC?l1wcf zeHroNHHuaiorcC8-&YBh(Rr{!d_N^rS?Npap(jzwFZoBm83nhKvIJs6U9umIZZ36qW3`70&O0g+*T*ln@ zcncbAeqaDVS{=JE`QBeJlpmI0Neg};cb~{@2#BD#DuPYjR>2NNI7~qmgjB3OZbWEe z&k817Ko_^1({lOZHiQhOVii2kLhY;Kjq~E^ZLd96G0Cp&m{I{f&T@w5+=d)Ck*2nD znN-}`s{u*a4K`oi8eU+cIN;9F|LsK;sOm}{GBZBGIOS8&;&4^4HI6T#U5;f;FK%GZi@(ug45ZBIvXZEaY?qHQ>q{fuM;= zru7xXjkr2ZvQAgkb@+MsJ!f2<=#h4Mf@XLE;_bNdjzhH5^BL4`Dnwq3P=H1iaSX&d z2K^VTNhy|h3B)THv@e4y8I*fI4%tFb-{Cnh%W8Yi6)$UBARYDBu^Rl3O5q$6Ve#w0 zKNwfyMZ??L7e#~X+bn$I)Z7PtF!2)a8$uKb_yin%rTmes;;#)HgfJi6`ihQ^!Cn0w zfY|M*{tBhuKFR7_iDBShUaqc1Na#4=Hp|R{qGB zjA9#F>|3LY`dL;<9e~M5{tR3tLm|w>&G{7iL%0VwODI%9cm+4l;p)fu{YN9@Ye*mA zDm}p@-5o8m{RAVxYpMfO=y~o4Y@hZgs(yPWhQ(ks^S6@&ao%|N+WQLx<3;+vHbsI?@gesD6am|u0E&xjCbPB z+zZ^xc$qJgfS}+EIN^YTLnJuij5Cg)Mn#PgoT8CLH1P{D8i~dPhbYDvNt|$qqDCcZ zB1w$m7@|hs@3(8{K7Ecn-?!fT-ui#bvsU-1J=D;Rn~<563dz+g}7R{n=1n za1<|JFrv5qSkfMF?4pPXvFL%vYU^qETF1l$KuXPIL-{)jJIYW|*!({Ye(!jUT0bV6i?s8gmp{O8Jy*iT zBA#g|8y&^VUv6-zv3yX}f21)h4T&0)&loN2FMM*+Z@!noO>?+d3Z~8vF+6Nea}6KL zQgepE9|rt|@%Xva{Aw$*^9&P2w%qY}`JWi{su=DygEJ0VbfB!+%Jd@4`%i`!iczz> z!87j++P>>$;J0h?rbxj$W|m<^32gft+r1b1oIR@4A&A>3<-@nz4j5bqVAKv(_X95_ZHvERFf!Ckhfm$xFwUIfd*l{WbFSz8j zHnNck&j7eHuIhRuHmbmP{lNg|;Oh79G@fKD(CG8Jt(>QHEb#Mj)$D27JD>Q2ar%j+ zy~F9tD(D3Jp2dS|FT?z&rg&Q;Sw6l*q4y%hlS{p1VnXq9sfTSc`4*Gcf$>POhR`_N zFz6KV&cQ{PmcFQFYAO%$x+)8FMyk`UrezE$Hc; zvssgKHfx#*!J-*^?j*3jl}`c-SMG%2^1x;&0nP{u=y(?1wKq^*?H2*nPdlN7`F(&I z(!~7iX<^SWtR2q^4ErzQ78=L1yAUqCL4tiTV{!kFV(fpgF>=6F^Z zM1*lXTfsCqX$+0y*#QU_xUGgcEi5#SXD1QQnVUH+EHsX1QXu1awhUzdTHLDWcorJR zGrFUN#_>$T#_>$T#_>$T#_^03>CiZyN!U1^N!U1^N!U1^NtgpXIG#z^IG#z^IG#z^ zIG#z^IG!=z{?_q~Uv1Ixi~_5L<5|FeIG#~nbUdS&=y=9QrsEk!PscM#i;icM4js>! zs23W?vmKCVfhmXl91V_VzeL!-54S2ho`uHo>~UhBCYFw8144&p2xPM*j%T59JgY*S0wadT@$5i^{VwSH zAm?~SC>wV?%kn!->v)!pJDz3Zj%Qipc=ktjOW75W4IIz>I((;@HI8R(0J^ir@oX%B z30dQKwmX2SS>t%N0Kl|t4HI~~=@I}lv&Ql4#{gz$(`;w&HU9y?+^lgtdjr7ytZ_V3 zkoSVDaXf1TurO;J&jtWkoHdST1eRotAcXM0yNusp}h}iuJb}`Y(&jDFSN0M7C0}o{Q)g>UT9|mTI{^g zt_QTld7<48XsPo;dkxTa&I|2xKr5UVT4xig&v~Kk2xyh_LOT@DYUhP^5ui2B3+*R> z);ce=wSd++FSLIETJOBj__@~x=Y=*H&_?HlHVx1w=Y@78pv}$;?J_`~^Fq4|P}X^& zy$q<7uv&sraCXQw*XCZUT8I~ zs6Xe0HXhJy=Y@74pt;Tq?E*maofjInpjqI&(4GRc(0QT#3(#Wch1LgO%q($UXgdH} z>b%g70d$@7Lc0pk3g?CPD?lrq7usupRyi-UOdIOYd7%vjw8nX%?E`47^FrgNHS3%g z+6{o#J1?{c0c~(zXm0@8=)BOvcGREqLhBD`v-3jR9+2m}(B=ZlIxn+W zoEO@?fI6HP+Im1;&I|2hK;6y@t)Um{&v~JZ0W{Tlq3sQ5n)5p|89-~D7urXF);TY1tEI|5Lb z^Fq4>P`C3!TM1}_^Fn(J&{XGz_70$F&I_%o6ZPl3(7FN5c3x=w5@4?LLOUMNeCLIB zIiLm33+*;Q3!N7lb$7AzLi+<~OPm+lMnFrQ7h2dC_2<0M`U6_wywJ7>w9}f| zd7+&PXtnb~TMlTA^Fq59&|2q(_9CEl&I@fbp!LoRt+^lS&v~J31!$x5LYoC>lk-A5 z1<+>ag?1Gn&v~KU2`KBl(4GcV>%7oD1k~cZ(5m{Q{+t&YHwx`?UT8A_bvrLK?jJhA zd7&}hROf|u9-wK?3+)C#Go2UOF96MUUT7}>n(MsKJ_0n~d7(87K>ax{wBdjjIxn=H z0WEf3Xomn=;=Irn0$S?4(3SzZ&UvBT3TTD%LVFm{O6P_4CqS#57uqL)Ry!}WUIS5o z&I@fSpta5m?NC7LoEO?ffYv)Nw4VUl;JnZt2ei?7p}h-elk-BW9)$XHUTDJsVZ3FI z4%-b-)_I{F1E|({p40WBFSHu~&30aB4+5I&ywKhNG~aolWw3Ivz^!Haagf?mM{2d7>cbLi;hG$O~-^pvVjDML>}k+WUYaFEkFG;e}S1n}QOppe8*#bRwoA{{6VM zTQ4*MMVZ{#qr-|ayRm176=iy34-YHS4=r4rX$WC;XerVWEyVF*gK2{((p4qI`C+7W z73qo=;sCK-0CgAXix%PpF>+48j0+LdZ=g);5Kz2%zHIsN5D8q{;5<+W>QxXGXO>EE zdZn-qu>X=}<~%B5#z|O!(`$UsZK{lD>L6HSWn`g=kT!w@3o`kcCDUk-Ae&5{uq?DBBw2JOo4^v#w2&YpgL+^C z+8GigllTadX?RGIb;7%YBCVX(h!~4|Js^i;Ee~xJm3T~6OD!L*7Zv$*o~y~nMo+s& zB|hYd?MpQetsRwkOrB|wB`w-PD)Q+(hmntc04*ey_>kwmmWOteN<1ddg_e&Nm5O{i z&lTijmq1%fB|hZ2*7DE-Q;Emq`L^YwZKfih&a(kz=>llAslKb829=PJuXt578#ljn1mkM^R9d^*q1$jAPJmZVC2$g@94!bAH~ zB_5OKG|NZJQ$;?V=K}JvJE2Xg5+Cxs&hpSwRf)&sdB5eO4XYxb&ht(3v3H?$s}djb zEP^CFw0TwHF?n`dK5;aYu_@p0G9=JLL7nGp@CCGyRZ0u_(a|iR)vOZTt7cV*NINTtHllvhT*iBzg4O?ze32T7$8(bQK)eT7t3H%)(KR307{&tm8kLBY|0WSS$BWJ(Grfr27xKwG0`iNC(OEt$s01lefoo|c6c z$s~(b)|1FWNzqK1U_qO>#*%5bOpr|`zpyMcW+qv5CNGeM@}`9|!GcUav1FP&6J(P~ zD=3lz?Vw2(oylafu-0fCO|T%7LoAur(gfLLa;{~e2{p;0Gr5T@tb5v36D-K&x0b9r z%E%^@H!Ms43M@L45ET6qC}%+?gDv@padr6-Y0D|(Xr1_a^x2ars&dXlVr z=`%>Cojpm`z4Ub?)9{`o>t31%)W|7U?>ZIt0~#Ttc#1JRe;kmKu^a~`!=6`xhe?hE25HNWMboMBIIw_r{TM~ZyNK~v zOxpQ1mEWFt{t6QKSwBtwnm{%^z`9Ua?jZ|{ zUNf47O~LOAO8OhP@zN}OT%$5<4*pJLIS)5pnuQN;Sl(cb&L_){aO0&}#79PF#Ae}N zMwUl$B8*GDwji9@j%_*S1}E|_St$V0@V z$@QQFTtU%fk%fpxlSh-xB^6B;Ifz&^c`uT=#-hn00}+WPFCdwVE}AUQFXGVTD@f)l zj3&#oizqbtXC!l3Mw8{aMGTt!1j$^d(PVjUX)|)uGJKmnT)oje^6(K4#o+n1AO~F2 z(L5r7h=;Q9{ITTWN{{9dF+@BR2wsHb;rfr}5m`h$l#1t{O&+caX&wE3b+kcVq9mkju7YUy4F~z6I*o@Vv ziZUYsTjP;&Ga08k#)RNV@ciS**omMF0p>gA8-P~=Y6YB;0^bbF$2ByaJNZ2SBun4| znkMLe{4z^e?FbCPz>m8>9OCx*O=`f!H64#{GU?{^O{?Ui8`FF6@UVajb1}*}D2VAeoQm2_EeLW`l4ylB>au-kkHWJ9gMZ7rS5fb?6BiHldgjY#mrErB!QI4MfFN^DM zaY{nW(Kw?OK#^!%ij!!XMOW_#AkBea=J+q4Qq3O#e0b|lSio$lzZ!_}?!6IiZTK<6 z#eER&*>EqzU(CR|X1jdgTYuWFSk=t^@;=1P?!F_IK~@J#5cabBgOIx?zXpKs6|Tl# z!!y9IL!igj2vlJ^7KHOeR9%jkg_jWWdCR{^+|1p9dxtpQ@|%gf6QwKo!-3;1U&O8I zvR!~{B96Cw2X6I$0xADU7b)SYWx^dMkMoOcr z?m}4)0XU1OG2j|k!-u&F;3Y&I2GmxyEaXD~?;>gspeAM*@e6<-5p@I7nw+DA#+;7Q z;Hvo#P}>OVOn?h;jo4|A9X<6eHS#FDi6s8*z}|$bZG3>lMxoRTJhfk;2TGoT+gIz` zYmp*;ucQ8M1vvwck43oi%&-jnVNQmp<@i5n9@fJ{nnz9bT6Z>RR^7`QEuKVP@0H2~>q7O!nPlxNS;c#2JUinFRH-8WIrS8>%H^n0D*wxF{>r%8cWti6I|--4r^ z*XLZrqREi_Dud{yqk7jfZPv-9HLRx3k?tqB8XqxyzrzFfej)}irE?LvZ8wz68-v*N zZM-tzIH7(av>aF6Mib$wl9E9i>pb(A@0L)`Wtn*{R-bx9w*{{ImXUlXvT_fZUbbXW zh$!$3YyF8~pq@`OOekPo(UjqUc*K~m&FG$n0hPBX&ZA|Qk+c5lZZ=Fe#W`&D6hz~X z4F{Wc-3_Mr>%NqIh2aqPCk*h#m$E+-WB2@fk;fP+xAa_S3NK>(K=+{iK&Tb>pz5M_ z587ZNE=RraciDmHfA!H(zCJq2*C$H(`cNugA0*}b0sVREv=5J;)dxwPfHAm#RqJG= z(<2+O`g5n?hB!P!${umG15o1do*qzl1i}UJZch(+72))oQ*j$sU^Fj%0s5o#Z~?Mu zdbg)XR53VE%^(9_tH#0Xs!J|!r6Mdo(Dap zSY3F{i>p0~|9KYO`7L{*r}0k$-xMUG7*LW#!*Zx$`rF0Xz#K&q3!#o@=nTuG76SAF z=8_{%z+3KQ+#1=dhv?Op0+3X3yeJlG+9a!6h?HFN`iM|wD+5*{HKmCT<~UxG`7d}3 zH~fl-$Ddk$Gb*@9U+Y;6?SzIk4p%cB=+vwomx1SMz)NtoP!wwaIu^}eHm!=Db+a?G z!SMvBkKhU^xj7=ZO#uH+)V~qw0FhWVp0_vg9 zu@ji2niW!TTZ!DZLE4jWdA;exull!`DR~2<{kO=Y!fb(!= z-ZRaQT5vbAMcLFS$}8LLWj;4V>dljyn96${UQK80Y4|-I#D~T`Z4?S6jr(HLxEC8y zPe)|_W*v;`)QvmTjXTthJJgLk)QvmzrE&K$4tlzA7kEmXf2B0;Lad>8WHXl{>EdGC z5T6ablZb-xF{DTe+}RM6zK`V))H_liu6PQI)Wub0BkKlm)AFwWgCNS@pfGkbyS&E`UX;KS9z4Gm+CtV6{_w;-_u9Eh#WiA`Ji^H%mC~kpdOs$dHvM) zk$!)58~6vP6Or#g)gNUZB%ZZhY7?GuuzDCgL(~GKGgNU(HB4QPrx>mdMRJMhBDC2;e8Hv3Tlj6kXM~RlA_%Q`842$y9X;^4d=Q15dQQ`WUJ0pnQ~PNA(&c zzmwV-<=I(nk92lX&mz_|^*5wHUCjb!hI$*EyQ+S8-rdv-xbLpMk80jSZ3Uh^)oF-3 zQ|*be?WGpszPCCB&$5rY9J%bPp2M@uQh%O+H=i1al=oNn<33wGfqdtv3*668b0N<&)tyM~EHwa6akg3l z$(*CkM!x5&1A$+p9>6m$R?AVg^VB(z#`$VzP%lvDAeV2c+mP>tDhE4W$x6#PF_Yasa_slmwi7Bv(6|E1dTTt8MXf`6rY4p03P zbvmSUs~U%A`KcO@c5$0pi8OClwBY?r?TR$-Q2l{fr49w>&(*u&yi@HD&bw3(r0@&1 z5|X)Fy$I?r)gsjLuT*bHV6{3EJol&_AcuQZ19H4iJ%wkvU#&vf6wb-zZM3A6CzR^S9~~)Y>EJ-*~Rysoy~|kE$H<{fGK1QhQsyhjRW? zT?L+Z)I6l{u9^lJZc>v_^7qs$ki+}xdg#>$>RLR-hw2EF@sR@3hwa48d#z-&;L%ixHj>aOa(%Ezby3 z1Ma*Nw95cuTELxmf?fjb=4EaMcV-!G#p}U}_e*EMKN)B>?zk_)_s$0Ok0AQwkguLh+1M8;rLQMj{|t~nAbBuD7g`>F3_vf#Cn=IA z!&`9#Zv8ke%V1o#D*#CpR+ToOFe7Fb^vTxQRDA~$%`W!q4gvZMqM4&t#Q|HUx8}aa za_cBf-1uh|*=9aTUbayEua-~A;k_{gvRgHg!As6SGsRWQ;n%)bG6FRuPQ@qFq| z84&r)4$3tEyi0`t2B@#$YTgkk_)nRnPlk}+%AJtKyJc_SYk2z*T<_y*B{6+4FyWav z`yH6{5daPZus^Q!i2#m6;1~jD12_wTQwdxQ;93N(#)UT(bNq(Q(FUNv@>^GUt0eao zi1Py`!i?`kU?l-&oH+~cUtE%LD+0{~nDHP4Itei2?Gc!Y%gbJC1o?ARasUbkog7^9 zz;7@A2Sa@g)a<4h>kwfTrnZ|+v03Aco?VP`aD34I`9A2USY20vy%#Izs!6Pzx}&^n zq)K{F*{{iyrN_~zhHGS9~?3Bd2;8o)-V4(}nVy=2c zrM310suNf9!38jwf`o@c4USG7iBi-Y4g5$j48_$-eEL*iW&qk2SAK63Wdfu{{Wl=n z;AMzzB5}hO}r+XWzO&gc=tnr)|n=PIP&{&E{45&dQi|af%*=A zHhK>)ThoYw9#F%ump!Bjz=63-8P0yz0N@~@)Mop)0617EExKba=y^h(y=+T1TtS9! zq$UF@7-It|t+T5^PI)gPT=fxd*WvQ2)&t8>x+;LiN`6%{Yjvm=BlPT~I;5kgH^^uk zfdX19dYTChHxfxtlqB;%^)lkmBwn9jEgOjh zQ(C1!+(;xiyyvYT`*+}0<2Dirj;Q+q5VDa-aHI}%BavXf4pSY1qjb0$J#lb!?RxSb zj)`b+j1F@nk>FSz=0+mHaXQS6M1tdWm>Y=%C+ILY5(yUQFgFqjPSjy;BodsY!`w(D zI8E|xlZ`}zg*A5~&muMwp}>mRNQ9CsVj~gCtB8$6D5fGd65%5kv5^QxU&KZtlvWWN ziBLL4Y$U=5)*Fcg=cf)rqIsq)8;JxLwmpTg{~T^L-|Yb`Hxdb!wEu9_urkL@M6id*g#cFN%mLN}R_DwC);|QWCT9+?W}LM-bAUCIUYFzkA=o~I zCzY+w(en(uhrEox4LNgwbqeilW6m64JruyE9QO{v)*(9ra9fAW1K_!>LoNiAbz6u0 z2vDutI^<&TZgOx zwA5`K!XvG&b6bb_=<`>&twRO?TIse9nFeT;+dAY(K&#!>A(sML3b6bb}3Q*Q<9l{fy zYTec$DZGkW+}0sO0d=^oL-qmG<+ctv9ZoLbZ5^^RAkS?bau}ej+dAZYK(%h` zkQIPh+}0rv0P1jChx`dpm)knz6F}W=>kytiGr?^g!h>d}x~)TauFN#IbqJ4yU2%+UT|pIS0@tw{^%0K%3pxAv`F>b6bbJ4k+uk4hcF? ze{SoLEyTXlb-1lVjt11_whp-*P`BGUkyulvDs}M@?Ail+d712U|=R-4zT_spjx+e z$Yww-ZtIY`{-{5$VQzkrVUX)*&qeP=9XgkWqjZx~)TI09x#}4mlLi61R28*?^Y1twXK> zbe-Eey_9kKz?Dz|mW=YUqbtwTBoqW;|0Av*$E>$VO#63{xgb;zZF z*1N4kZU?l%Z5^@>&_=g)2oG4;|c%H#Dw{-|lE|}@I4!IG~ zY`1j?Pa~M?whnm<(0sRb2u~PT;I7OV zI=6KQH|$^GwhrO0{43qoA=3e^a$AR-2xzt2I^yTFfMO%k_3@F+3LLZrh#k|y!>?Jmi`QhJ%@ne&GU~$7XBk7{6YbVxrz|<8a*eGN^n;q ze6nU}cN-9(4R4hhiDNM?%$U5JjM;rlI^5Z;96PUZ_??3{}%GsDEXz?5bIN;T|BxkrUQSlT;ag+gHdwxaP_slb zia5`I04e%e1f|+ICyYwz`Q0S2_?#a`1XFF(NoA2Zo{J?<35rTWVadWL?knn@P@43CqArR4lG!2)T$Yss9DCdej}YEUEv&Qg;sI+Iak>5WV{ zbxp7!lYK3jbJ+yhWOA});RH9yqBFUcEK`vQXT1p)WO9!sa~hl=n@rYQ7S4^6EIN}< z$wJE%C(H>JWYQa0A#(SBV?gXjnni53o_Yg$(*+*$R?8vD3Ssv^GOz+$q=&8V8uf^L!NRlHF;<-sl;RQJlm#4Q%XfX zo#&6p$1Z_JmP&lcQx2vk4^1zXcuby~Y+5wRROHinwty^M0L?X(_>iX@OwC?|Mx07K zCeOocS~Txez~w0@e9D)Avt zIhdM#3=K<_cubzV*|cbKs>rAFJcWGhPH2p(#D_fPU~2NvL{*8$<13^7nN(IKZGL6c zFF*}w|4UHJn6wut0S$slGW$`_pG-0>gh?`Ggym6^X);WbDI_@jkW3q5l1wS#GZ~U; zR7{dT1J(0alT0gPl1w?_oHCMWc1)5fD7fa3Ogm(fOi6iuBPgN z1g+6!)EXc~lWCnUqYfvPDoAT}8Fg1u+54PdiFy>NtaMtl%i>;4Dr<|@?K0{uKnApQ zCn%?xhtd0gdBHvhJljNT#(sN!Gpe6q0E^Pm*;n{SA_7 zPfwC{FMSTlG_WVhx|d!~GA-^&vhJnt110@9g!X}Bk&w1{@e?%vC9)8vja{e02%i5R z63ERnrz3(Q==l_!--KX5>tl?}7@j{7NXeLXd`;yD!}Iqcfk7JaW6^Y~#{wD9pdX{? zc$YFBi%A2&rt&iuSZ+wz(T9(mBalsOE~fT>O_m=3$pru%ml2ef0^NuLE(RnhCi3Tu z{3arEg+R0Dc)b9pH-W&#f;eF-5jrVco+MifWXy+IN>}JIzZsU zLY(kD5{7}mHHJ9hE)phyz~zQG;c*fuWv)DE0-wrM=U+%|$BF?jP35B*Dw~0y0wp~K zH(r{B4{2D~6#Ri?nTH!M&BDhuENl+`bh2EC8!yem2RAHi68^DdS&17j%|a0v7B&n2 z3bH(g8!yd5VHg%R4gXHE@N{ZknuTI9ENmYBA3+JYFrvv$97?y$%_MWBM3Y4xA|6d{ z1|{Iqi6)CIL^PVb70Fyn(PWW>h((iUk<7&vO%@r5NHqBrlDXQV$@2Un4o$v_WG=sG zvOK$pLX+<#Sx%@HW4JuGh(VK|CYcK~I<`Ewv>CbSmbi<(;3X^6(K4#egG&K@Q|_ zYRN_<5b;nJo*Ygc$l=t&BVvenC=k2|nGP3zbY3Ehh=)?~?cT;-QdW1t-r)$CH!~2%bNQ3RH_=z_liwSVBl#TR=}p z9_DNeKK2bjrtT#?gAvUonvt(Ku0 z7w3ck-vic`m0DF4_EKvgx{0EuN=9M7bm<<0xN~}K(YO*P(KL&$-qAqz>3}uapI7_{?8lHvZYLrH+a~TE{IOm9XDwPkC-TjA;_2I<`k zOw?aCvp+p2)E*h;z8lEXa1D6Ne8@Z#(YaMct(uif*=%d!V!3RlH!xhs%!LSF0(>C& z*}iz=akMiDQR-NnsVqzn7GkO>Y{Thzh;i-Y={B)LNrMk@24W48huBIULe1)u;%_Cz zSNnR6zeWZ_7@1yJlEF}y!BELy@Y!Hd9g>6kjpgceT%E!-gz;-l0Z`TQgEK50wA8Go zU-1v=oUko)&{NsHwnD+=L+|IR^oQOvP2Ble9rwm>LQckMb8Db_=#ReP$eqeT%18@N zjpE;#-yPwuY!CCTHle}tt+wwiN4TBUa3C(Po!@FRly2v@+H(-fw)0zUhH86< zf2)nexrUlCr$#cR9=?3How;joB+=FWL!gG}Z@4r0eSjL$#Kbe^U`Res{tY6`!I0Y# z@gWi4MDD@V##@O9Ao~oz=ME>fGLt+#Q8VUXNM@1ehG-dn&mB%`{WTF2h=?5wnV$|s z5B)uN=C%efZqs@aPkqndFr=uxJs48bwFg5I&>wV%e`@$Y_+ZEaKj?M`Ll#O7hAfmE3|Zhe z-R@w>Ldn69g_46I3nd3b7D^6=Ebzl_cQ9muUv|5LAqyo3Ll#O7hAe!2Fk}NysffPr z{`z3Z)L``A3suY42ScVFsDX>7YX16Q$kabDM%kbmzCIW-wFAZ+MZf9mgCSG%@s^q3 zH~*m%{ipl7dn2Ie>+Y~`{Oj)ifTFLvw+9q`-Tn2!kf{^U#*XpZ&V!9|n)|x@20%02 z*WF(q44JwL6NT^jt?$B86@A^!jgO}CZuDRwYq|M1$!{0%x((yU`UzX6zsu}GQTO5 z91L0D2j1>r$O6Ccb_YWi_=&eW7_z``yhC>|WPV#jl)Y+Y7kb{jFFhD?8i=y*`&T&_ zlFZn^R6c+9z5lHTL+Yp8$fx`P6A;STvdD&1HC*g4h6&(zzBmVaeh78ZM$(mjG zPXsKRB>ztz40$B$_)pB87M8wHG`)SYY-zm))ouv-15PY;g%1E47YqRWVq#nA;T>n3>kiXFyz+WDczHR=Bz`g|;cXSjy7X6q zB9m1bnPc^!d##_7%8wOjbS|U*?>rbXe6erYvl_bs4~7gkS0GbFp8qPz&Hc(_^ZO5c z?o9HQ4~7i4d@y9V<%1!^t19HpHU@WpP{J)A3>p6adN5@8RS$;L7LGk(v`;fgTfSDz zyC7t9;ua5v3~ASoQFOfjZx4nHzdjfe)zX56172GAa?G7RC6LYFf9b)HA=gLr1Iqz` zeBju@kRcaLV)792ydf=<2SbKjLDA$&2SbKiJ{U6G^1+bdmJfyuw|p>UxaET(!>@2K zWH_?RKAk8-bbunofNz{Z4!^>|kl|N27&80{2SbJry7FXvlq@PwbTDN26%K|Bzrw+g z;n2bM*+nm+yh!k(gCWDOa4=-}6%K|Bm%9w#WrrALNP_3@$Tr9`1;R%hU-e+f@T(pS8U7z}Fl6`@4u%ZB^kB$&5TB5I z3@PdBF@YQmN%(*EU`XcZ`MmoDeBr$1gCX%L@aI{6;3}K>XUuVsKSyLaA2M_I-dI9x zZTKw^8FMD&3k-iT1548F^7(FyITJGbE97Pmg}eY^cPQlj0DLch1^$}&W%1((wDNmo z912Nfk%+p>cSNSoBjoc|+&~-;h0On*INpjM630U!^PdyPTQNHvI35a_?}3odTd^It z7IP?Mc0K@mDx@yf19&K$3i;^>@Y_=%Hvn*_LY^}cQM~*>CdgayINbW%Qz0J%P_U;$ zN{WR?nIcYwlq3uGRLIj&zI4H!3OSF77VN2zq_ueY)0qWt#dWyV&EvDZL>zC$H;LP3 zI&kk0$6IkTaXb|=@3Z{86^ppl@l?os6LGv1JBZ_{koh6R@m3s598ZPJZ%Z6+#c2$& z>WX_a^fE&SG4w7&M=+FPz6%&?X6Q_YDAnS(7@|0e-(l!AhOT4i1BPy9D1%$^Hip_5 z`Xxgn8Ct{84h;PPw~RRzQp93Th0Ic#_E^a05C)q$7Lu+ABY7<3Xb82VaUbj&ke8r7 z7Lq6Wbm(It_eRibIsp7ciq_S5O^i>9z!|TJ*|L5FPI)cnSjem>!Wtxxb0ITK$!p^G z<(FI540cA`AK$$f2Gsrrxa{luK{W;znVgqB0F=TY0n*tQN$Kh#T%Laf7&0SCADm)h z-8{FV8s}F06@Zc!JnDy7SSy~QitCtGH_+_zCa2`KH>glP6vclOJe^0PJilG{923EWG^PL5avs~%ll~8n6uDe~ z7924S#Z^Nmh(}tu^7<0M=ZUIA3cr)@_0#9Vaha(8MD^oP>@1*;!d3Hk6!6g^&qI3+ z;G;y{1OJOhTgf$R5lj}iYPUhewZi2u^J*_ZoVCK>FU!{YgocvcUshYYBRGFA=Og;d zT58vU_WM5Uf&FD2wSOmVG^daLvaYI6L3^yo2f7Pc{18{wdq6!dsLuhuP1M^!JyAtr z?XehV7~!h=3HB3tQn;1@yqKuj;Cd=UL9PY(8(e*Vj?C88QI@G?s}}X92g}pK@eZi3 zlo zZjhK`5vvcn1}u#Lmk@% z8ScR1o@4l88Q%JyiQ_eN7%DQ9-#nx;dcg@Nd&*bE?SPt%H1@;QHp6g!fESTXYwz^3 zb?1hVoh{xrCYe`}%ox^y$|)(%*+^m`uIii2gWL`dn+)pVh#Hiczv^EbhDYNJltKLy zkqqFdgz6*K8#~?9>U}6!^&bt89~(;^e1V*RS^alI<}pbwcN{dK$<5PVkTOW`aFJ$_ z(?yY2PaJ*eo$O^h`Y+;bSeNpe>Ji9x3`FG2x!$r3=Z1X)wRf=W!*j330{`q_*=G`% zlU}wG$NNJnu9PP7M)Pb$iuk8n$Oj*yIUtK#yi>{&TFL*>UnLxn#YN5>jais;&B+QGw}XQ_T8bCUs9l%}j7;2D-z} z!hUdXU-6ulIwxQJspUW3%5DSLUrYLsZB+n=le~ha4=KpGJLUA zqPIiC8k2`jWt~msyHb`@kir6{vaY-=Q7Zjz32@}#MDrBOCFaG5bpfvGzZw0h0=-fh zsQ_dOq3e+k z=T+Zq1k-`+onY7iz3N{W($;YjPeW_o-lWAgcwn3wptr@T(alF22977MG7Ric_bn{D z8Vv3Qfumv3-lec}(EYU?Xe7pXERA%+aVCz4;A6Wh+JZVp2B(VB-AWv5j~T*|<#-OM z)G=e!I}@%WGqqGki!gD4Iihe}hw2;kYcdQu&RN_+nmz@OskI{`zeVOi#sTDIo& zD(chqOj+>~XQGk(o4qbW!)e~3MBY24JDt;d|3gB>D zy-%2Tv^T1npK|U9a4N1ze;$iRJzO37G*g4VowiN~rszNKcnk}x_C$u`>p0-wA941< zReiDH8wWb8zz!+QSm67H%&Eklng!?do*sJjcNd20LgZQt`8;;^$D~s^2xSdH7*G1Ft%79`LO=C6`*|Mo!&^p$&qMqH@fDJO^&l z;y|9cRK~^G%f#8PEDleLqspF)LRs0)a$Mfz^GwWV@tAG58Gyn<*16Pp&uE;sU|fNS z+dzH2FH+vNY)orRgdMGMC%!xDTMw(0Y zw$iKbW!NHVpQB54TrCoY%lT$>ReiKR<)aX&Ul`4=AMnY zoM*MmdAK|A+TlE_-Nv)pZ5#}A8_yZH@ww#l10MI4Gj8L^#OtTs#&gDPe2%1+e!8BS zrEhNlGp$)cOZ};!gm>eXqctmNZCJuEtyw|Oh8sxaW24YK&k_ZtK`ciDSJ>XpjOxX} zmHB0M0Cb?R`;M5puMXY=!7J>K8F=QNylgz~h5rGnhEKIM7;E|1FmMr>CZb3KS19&j z-qL#zM*~+l8zG-J**mL916Oz~alEBZ!>vdISGbrs-qM%iRxbvw?EBg00Tkxp(Xtf6!;6r5%{=a27q+o6$1aRe+xj?3$JE^yro~pO&hqfN@d5Pi)x^SE2VUb z4K#4Uz9lI((7qL@>Sy?54K#0ss_}8A*g)%6MpIh6@GnfBxAfLXut?)p$ZeZ^-qL#z zN8?sFn>gOm^N6EyD?FAs-qNQLN8?twm^j|jml8+gR(LgWyrq9Y9F1GyO5%7+^H4FL z)t07fy3bD)(!XPfd$FX~F?1qBFEMl>LvJ!fiKgFSh>}QeW@tJ?J|%iELqWT=Uu z3mNKQh<@+sAq?Hl&{%|WG;U>k_n^S)#kiI2lcgvNpW`Kw?X002I@oC3%AVEZcL)!t z&At>Y2fA4swz+ z#;xpYxf2maQgd|zCh8YnNLtnHvr*ROTH?;<#oyOKZNo*fw(%r^jUnKUM7CX^k-z4^j7CX*h-zD}h7CTB~t3ClX z@5(#eVD~_Wk$2@CY_JCq>&i>b;ZqzztSfJ)#@3xgtSfIXgFTm6SKd~Gy_8s2-bRDH zmRMKbI*qNj7R8&;9Tkj4k#*=5j75<^x?n7dhazazm^8(v7%+ zZlY!qwGY7QM9oE_{~X7{UkvaZqP!-cb{K~Y?gRL9qC!Oc=RSO}cLDwdSKHNt@CruO zxzt=E?H<20Wm^LDW;}+19R}tiwHK{1J)VwHo4`>@{dnPAZT`%E|-%B9v%6pQ-sz%bVIzrp-2F42bvCKEOMGQ9F}i9S5A8&Tf`xIa^UI%m*S80upictra zQD?-cyJ%{uKAdG3zJfli5q+@5GWx)>ias=WAw|}!8c%*BrEbSzkv`1H8-2KO02#)7 z8@mPJ5`DOFB8Am_Ir@JaSN5mU&F zr=++E@b7Tt$041WE$os9UIn@18nO;C)n#fT-_iJGSl^EsF)8m+uzoZ5hXQgw3WVv1 zu?sFQvtMn*miAr{w)e<(Co*cE3K5;!Wi(H3VqSauRN7RRz>yAVP*7le()v21DYEMfbBEzU-;yeVuf&KWMXi-hepTO9uylof1JoB;qjaY=C| zATSn}F3ula0@osei*jMLDbDU-*+tk`9LtvW?iIG-WMgHW29~3Rjm3H0rN)`zMY-e2 zM(NxOmS5nK;=F^vMgo*h{#p!caOvXw)abSH-a!Iq7Ot_y=?|6;VPkPDTiQ$E{di{K zep{UFz%o(TSe(0DYAwR{A6uLSU^y0-6z3`gt{@<)jKIyfye8YKWKm&U4QdSLm2+|5 z7gK1wywl5;EW%bvv@j?!iYYZmjq#dX&2cGXfsSW*_f0Qb$~b#TIwdh- z618HK0@h-zS&q4m@|p^-q0IP6Y5%EnWzU4-vP(jq?2BA+%TpM7YCMHR8Oz!66%!hF z$Dh$1O{%9lrdThn>~z(n;p}y#Q=c@M-C!&GPc^xVeBQ8C=+*pD>IOz?ohCpF0qi1y zex3%0TPLG$6@$ZO)EJWVrl9`pZ|(xb*OrA8jSc?O&D((HYx_f$HW2vQw%}{qf^T-X z(^^rP~@VLO6XZZq+gr-$&KHNDLLR#>c1)E0QI*M?paS;RZVa81KNXR>tD{ zs$P!+sN-Ne*rAHE9TnYJPyxO4^$7NO7znkW>c4IlApH92KfE&UM#TEV4_)Y~{Z*04 zKLYiB7+1qEh}-h}BOzgM^k|$2>~kPLgR6(}>v`&{$$PwmX=zISSAm zV!S5k2L*K(pgV}+$nGpk?G->z;3^zC$~*EtAiA%^I09GU(9zy}A`S&GOAuqcqlmZ@ zz>jc+97`Ns#ghIT;6I4^E7CnC!-TiL9%X-9@OcY8VaT%nn>6*Xg2q+nuUdOde|OE;=Pt<&ojhyjBPgws90<=hrLP!rh;lM3WU z#z1*Fn}i@IV2F{o7@a6Eg@KZ9vAougQ+I#7{@rJ^uBbk6BXoyd?jl;aI!pEqe1IL zh_6#UGE-a56lxQ5S|qZ-W*4bydX2WAoRFB!RF|_=Gh*JOl$Ko^OE9t%@@cPA1FGu_ zsXlFen0755s~5${hwfdkQq_e5CCUftgL*+(JB*kx(d%q>3=@T=ZyJbg3kAqu*8}7A zt5-S^&C_2}4I=sgmkl1C5eAk@L(5_iAL4w~x(EzijsdkA7YoL%B6Xt!rk;l6DHOqR zel?USeMNP%swK*%Wrqy`wccfz4$GS@)w>+^F0V5wNR^x7u*!>N+E=Bu$k$44id?N# zQfpZ0)6`mi(P$r;)iE!v2t5jl)yPuSN9x?5wW2f&9Q<6W<4u7#s@~+^EHO$mrfxx@ zOH*krs9DX-uBT8-W3*YdY%eJQR7^`Da4n&?Z3%s>%BR@WqNdPc@sfSK&Y5|qm5H=z z%A_x40(F7<^-`ni_g1U7FU)G5-RaK>_7^4VKQ?;W0k(LZkNNC7n83hk9)*gI&Z%!_ zj(K&Bm)>GB%OK@Jt8iY87i&rM&UJZQBAW&1&)#0$R(aQr%!25SfdziNxiRwwGEo z8i?<-L?;}$qiz|PV!2$UDDhl(7Fk;zh7#=JbcbxS3v&uud8XNpbh>n;crne0m#9O^ zvuhj`b=_O&dBVysI@<|FJx^(elTCIqx(Cv2%%+LHr^0)VdPAkIj=GQ<2Ju?g-9zYJ zfqb92x2w^^9)g2Cf(^Y8jnM%z=Ep#{C>98Qa3uQsdM_R)IVzc z(pj0dKfo4A%SAWZflirpM`UzqP}F62mABr(y7dl;ipGM$@koX$cn1$lw%*~DTJH$g ziOGYFOg83jm%eGtqnb-z+M`9!p+jSAw>Q=`qmqU`u2Mta%CrRG)Qw}j(G#H9Elyx( zIngC4Ei-B|eB`Z5iMrX6FiB(`YqXPfAF)lm3)wa<`+|K3OtIbORNIUb9gFQ8w$Nt3 zWN=#EW^Gd|Z>^=RIgftF)8hs=^5wumT8Mp2tx;8alu#|Ccq^=l-u*Hj#TcPNyX1JZ zWRzPU^#TpH!8BHAO-)kb&aCdPFb|W&Z3{H5zJ-a&8yI5*3X+x8GuahHLj>Cu^eU}r z+tPZM<^dt~snp0y#tEG=S+JGJ{5$1o5+&_xiN=O`wThzUl&bseC3CmQ|X z7;I}pl}{U%;JF`wA+wuf9>Q z(b=h6q;1&Ai1CK(CIm>6(aZ}xic+a9mo@cxie0uX4=!)Z$(Ors%fq6!JUrQqM^tLY zc&W$d+4_|pn`i50&gr~q)loAyt$IwdRVQBZx>YBdy6vF1s?@5-#~N{Y3)hXfq;aEZ z+eSS3%Ny}FU)qRm>r6q2+K804x)|tpPwa&)18P0Oizf40V^qm>JQuZ^yq*3RN?W_0 z*vDGCnb(&lNl-<3melF=IxbV z9{t6bBM;0^VAxV4D;uWmu{Tv`mL0xrmQppx$HwkmTGYOqme6(-GoGPnnA^K)YiBqJ z?+vCn)bi;X0pBn?ybL(BS9W|kJf{64^h|+|*I6~F)l3vdT5Xel9@bskgO0LOk7(p= z5@QL*y8KwDI>*iwDdVJP=*$#4({UTzRyH43TlX+IT0izsWFOYqR4xtcAqm!uBOHK7bE1y?(4n@k`D6Y5gEyYSu69Uo)r> zuYB&L+ptXva?9H{z?Q5Xx(7?kSb|EA#dWH7)D>X23T>I!Ru5-TM@Xp$q{dc9 z&FT~t5oVCs>Xc0s3SB{ulj(C6%NluSB`)taA5YH-MD_tI-WoJ8@-?4Ixm( zGo2chKge)DyGxX(v;C@J!vS_Ms!B{Ri>8* zN{O)rg349S;={b>-v4-dt0-I0hS=ybzE)`~{Qsc|m-V9;b}ATzNMs?c70A|V-1wt4 zC0fU>w5a>Du60j z6QQx~x^e2jXkdDFz1?;S)=|V(aSg^0EUsJ z*4t6#!=2F_290smj%4Cu!WJFTAV;eSO*|IqlmxDGE{O{L!+6)T%P`gT?HF`oK7e=W z#8{6G1FwBGFh@O-m?ZV(w1R_8_xgx0AW7y?^W0WgpLy65CuVNIU9C+HH-OPK0RjiOpe= zRBzkx`{*tz>&9Xo(vP7U?5G62N=4I-ZTm?T_Dz?{l5v^jda-y#d(AZ^yI8ZVS zF;mo9W9Eg?XBvY`stmJ23q$K%6%MskC`e2RV@*`&Gdv+1v;H%}zNJPoduS^Sp4}#l zrCpn@)FD=QEga8$jIF9Ta%|_di%_*MNL|d}D<9O2B>+bx7=eGFML;zDX2zU!Z8dlG#6~JT0{^jxQa- zp*-emGgHFYaNefL;XEZqVHI@`?%oG+m936CT7CjlHp>)ap&l3}Ca8KiX6*lg>s0Gq z1)D~o)ysSsWX3XP62}jNO6CjED!m=HwV32ObL^U9Hnmnu5Ia}da#v(dO!CEBe!J`Z z^td105JjqcQCTN|_F<4$-hg8>TZ*_-x7)r^yX}{3xBV-Pw(yRKThg^nF*f_aCqbpd zFTOI|@Qd}XO^Gp89b(^iL)j^e2g@+mSY4Gwp9e|Hw2gH{)J*t#llO$YBuXpQiqL!x zw1r8EU3^MRS<6*iG$E$WU;3pRrER7tHgP?z*s(&1EohYP%16g~ z_c5$g=?1VyW#jZ=tK5l}V`l_O zV~=f7<#tQHE%JJ7Zw)-AvEk*ZO+i(*w@thaV$)*X+m|;QJ_u#o!;X>b%UVT)?T{L6 zTWFF3#EhWL)}mm$d($h~Mrgh(W=3ZEp63{gijU;( z`&h>3$JFtmj?Z=HEG4XyaSKxE5SLa;_u4RTUv~BehqfW@-@Zn4ZFiU8phsvO)M4f} z+O!IW$FPE9FM+5NPFb}~C;Run>3b+`#&v%>Ygr`K9d+_epjA0WSkmgSq5F>xk?(5x6)6phHHJOa@sGA$95df+#$V%QGD30 z-TRr}xRsp=P)O$s`l*D@+Zs?E7vS!2^}q!g(xwhvt(M0I%wF?Ur}1s(eDQRrBx}eUsC5Pt`A0C%s>XAdGUUb zs*1Hht7Cd79=()JudT;@P(1B5Kzl&!)#ti1<52b`qc~g5UpmDXRWXmqE>nMJZ*EG|6#A$KXOtxAtpXsgp=aMUTYp|3ADU6B|b2)7$tkGFn%T4I@WYem!EI z1A!XjoRO!t@Rs7n+L!ORcpq#>6a3^SHkz;>+?!X7{@^~|KIId8+ssPqWKuLmqXA+v z^EEcTZA5#}Yr|X{Z+lZpUxu>u<9SoHCl4HWVj~ex*@hSM{EVqjS~sMiv{h&|3w7hm z*xbyrgL(00qt=pKtEto~%hzpev(w-5nU=3V4#F6Q9e#~%VzsszX@eYVsxELPTYVZ}GtxT4) zXKd(!kGJ9@SNhMmuXSY5DI?BUgu3L5E7s&YGt=waJJH$p(H#*SM5CD6-Y$=6zPvXN zh3Q-Q7$-iJi#2tLKA^NY4YV`vlCS$>pRU*s>s`+fe_~qaT+=WOtdf(c*cZyaFNwWa zBK=C0t_7yuE61sA=U;W*7{#rmfEn8mvQj`+=n!4QIj!f|fSUu(f+ceh>$==xB+*$I zInb52DjFA5n*z%F33_V9~PHry^*M-ZY0`Z8>P18 z!0%F2+YR!Y^spJOUk(LBgH_s69c`t!>=$J{?RdJ)Ha;jI{F#wHK1GcZU{8p_bZJBI z{(~MYz-DDfupX8Aw7QCaKzg_k=o9XT@b_@NiHnBNLbt)B-Fp=3iVy zje7nRG8p6`7t~9{G&%1>f$Eozm@#5TkV_qP#3?}#rp~-A=z)}~W(0-QiIao6t%Ism z-Q)lXR80bXo1g${I0L~WQ+4NC-tfqBa-dR$BUr}J->bg90grJ(klH`YzF;0`PCc^^ zU(_sf9Lsq@{`W!ZWC}<>xncbES7wBSi;9tFqlUSi85Q7o)J8=G$XW$x!QP2f04jo2 zg0q0rif{`k-co=nOWcoeOHn1#q777pfQk_Q61yX0L^*;fP=s1+%)wvkNmv| zZZ^bte$a3*t8r3&(13DE4Y9gH>}7uN~dy6g{($2NIaF?D#IRDUc(wBSm6*(7_aZXEe zF2S{2SIKpezEZAbdS%A7+&-7Z5ysrV8DXbHQA8-4Zz}WLN(L z^qqy8P4&*mM1`G;8(jF4aQfoF8fwn zPL^3@sl0=FJ_ef0@)uZm5I5UAI8ao2D+)Y4HKGd$mD<#(3lsn^R&7Ucx2#H6xT*zx zl$(yVyHQ6_v%tNJ^Ej+?QBdM`lMayzzEF!D9gghK1@A}Evb6F$l)jBD^?=OZXHfuE z^;xPXo!_9H)s>Y(%GQG(VghV-+*NZNjy)jik-+4-hi0gFj0A;V@>w=gHjQT^m5@s3 zo@wRJyJ`fCW7I)#aAI0Gq7kF2n;e;YWG+BgFNcnR1;_&>Q1AU!&@Ad~hqsX`r~upp zx3201GAjfZE@yBX(Mfhz7dZM7Oy>MXeZk}Ud_$Z|QE&k%IA}2zxd;m5Jd9AhMMqk6 zE}2WZ&?Q|ua{Yi|ECsw$mjeNh$cZ91sp2Yn);JhHEf6i!SXxYB|gz33G~k*q^k!2B7z# zY`^>OX2Qif_keck9>5f1YSlWImOw#&Sg*rT1z@^4D%DL?j}OqO>0+n@wdhzkKCR?9 z8(|h{hHr%}oY4&Dd1jd3g$kPBd+|&&Ow%l;xbO&`j477yAd3N}DGs<}52BI#CH{P! zKVp<=vLm$25wXngiP0|qUtyW|fH=l3mx*02+ohdbblHWp%eYr=)G`ZZg%os`Fu0)So!u*M;KVD))^tt%YmA zKKJM%5e!d2TC7%hfn2o@6&C5j1@t6z*&zzrH{J~~Qh({Bz)0VCcXcHs7f)3wR~>w} z<956FpmfmJ0Yr}LIY^GGc8tH)XAU zKoyDP&lfm79cFAOOo&LQnzS}l<%@?PeN7w=x`8=)5TI4ot4)D{XB4MgPn@=fr+c*` zMg>2FWEe4OKphj+q}|%INBbn+s(<89AjqL_d$axW>i>M-mcDUOO6OGIrdSv6(%~*$ zJfy?7=;CIOvPXyq54iNGRy}%s7HiXO4vI`y)^mN4kd7)io9#o zGNWVKne3*&q1t@kDY zyG44z3LWm$3!cSq6(4B)jQ##n1)Vk8y_^dwq^v+KsMHa)gQkCBcOyjIb=A;5c#Gva z@6+y)`R+mJ3N=6nVUcn11*GE^&i_~E2~<y+o?n%UT z&@yWNNdgU+OLW;X9llPNJ*~r6=mp=^Wz~RjP$0~|8@B_ZSFghbk^7*|zZGRm(~jp| zQqy&taIMhpUc9~eIEOW;l}HH_F{q7*x{@~nADO_5qb}n;!j1U@L`4K}Rt5TL1gNSQ zYTf2Pa3RK3%FzqpFA(N`7f5oD__zR+Kc`WjA6cf)`I^pw6Jnju9CwF4-*vvGFKMDL zn&S>h9C-d$b!D?Yt5Kh~L!V_DT!b#pgHJo8&-*IsEjRfUdV8ixL7#0BMge^)?fyWY zi|ax{`nOUPc0Z^uLs9Ohbj4+Qev{5^)|aV0@OakhicX!|!Y5W=U55JebOoK*jZ`e` z5U{$K7v@;rJ(DALjQDaR`^eBWzCUlUE_ij1NR5*^pI2Be^_5UHygj(DN8y43F4!wK zvK#b;S%qhgoXkdj!F?F=HUI)H0k*``CX9%Bu5=wDU=7%(1dLq97*#tP@$&DcKZV`W zgE|iZ6|LjO>82HwHWMi41uh^jSMUeSQr?YDKgUkdjm|ryaj4DftLHR*NpNoax?K8o z>FShgwLv>;Dz&;oJ3G4dcK5Rsvh860-2F2+qir*~qJ3#cwAebL8)juWk3(B{xgg&G zEqFeRV#x*7VlBj2&MXI%M$@IybZInQW(HjvS66(`W#i}o|Nif=oP)?E7|U5Ama{}G zXGt^oy+^nXmUHgW9^-4$Ma%&m;d@@kOW?UP>)Cf9ZT%a(_r`7(I>rcqW2t#^!0x<6 zR-4D9d7Ng~E@dUOk;;vp^^~V^z8ci$@sc)#Sq-f1lm@=-%xdyB+0LZ|Y?|%7(oRU; zHW~~4jHcWL47I$(tSLgkl+4LmPpG4o(_7CK)2(M1{Wz+{SR>lbn&+)QUWO+0O=d(unGywACA{Uh3#QpkF;2d*AEDI|7*As=?*VLY$y6$sN)2tV)@gl% zRuAGq<#z{6-9CbtWikg?W`P!CFV+&8(pXF2VCZm`kpRHzY@{pF7E=u71P$h#41@WE zUJSHe2=fPnd0~eAqrp6n2J@mdG=~>kb9lYJ94#(}iQSl7>5fyFROh30B%z zQZ2{*q}YoEEc#npbdwln#Ams!y8PM(n#4=go^I3_1jZTn=eXBfYdGXqN1g}qmsCl5 zpc6dP^VQ>jgzGg)D`UW!QXD(k)2m|< zH6G7@5+%kP$^|Z%ZZeg&>Oa7}awi#Ep64)BgWH0x#=6}jNYt3T(m|o6Q1pmf=tffR ze7a!BAEkSm_^rbs{dgn&w=XcCt)33i167OY44u{li+Ns+x?i`J^na>B#$Tm!oQLO{ zq|P%?4Kqfe`HVuJf{i?mD-$`v1vdUcga-WQ;+sUIXKhB!5dV6R(E{(&#PgP-t^@T2~f zcW?0SP2OR%WL{3_4*?7QsXs;v08}f4S5wiW3zK@0EOu3_hCp~+cALi=&RvKP;IT}~ z?_Ep=oa?IRkTQewQmDRw1OgHgD0&jTmZ`_yeK4iZm4^j*sFwR`dBF^X&4%CT9ytoX z^Bs{!VPExUj-%#*r3=-sO@CamGx@o+lYdAYBIW*t$hLa~(Dnr1LQ=&Ky%m5Ist=M) z%Vat4)E$BUky3WmPeRUqh|n73W8kD+?y8Tz8@Zn{xQ&+sD!Ezg-CX9~_$7WF|CL{I zMMh;JYzuH-t`}i`74%~xk4wOii>Vl4MAi`iU4k5|Ug?O zzcLfVAeJo7;?8|tk^{=U8@2sO_^{9F#-HeHT0sNp3tbZ3@X$|$=8UDLgXk@%6mfB7 z0u@TRWDD4_Y=X4`uuOz+CcP(8+AG8rzdndmzFo`h0jya>?LKKxJXqU&U0^PdkCm%O2iej~-D@K!3;2nGh=&ARF( z03ul#b_k81!w5X*w#+idIVgj*fVh#{U3KzZTt|5|oO2n*$1(+9j>>Hnvy|p`pDV`S zcm?7D_?e1G1M zW*Iz{XtfN`;Ie-PNNmfsUQ4yyLEBfjc3BRta@<0#KBt|_w7Ll?*1wQty9K!PWBzMA zhX9V#B{wo4SGa@ijdT;OYoK8kg3MDa-wHE^nLtTmPD^2Kaj!a|h-B$*0N8rRLD*nd z-M$A*SQY?CF; z{Kmr=Vrk^$5o!+A)Gb`ByWRUy{P!Vh4l|}+xJ^5k>xC<{bB#XtS-p^|*DP=q+=SZ? zU@Ym^qzRRq?jHj2@0^ca({t3y<3|Rt4)Pd07EE?twF$uG1dg?1`tqWCoB`EAYoo;Z`ojYxajb_0PUK|!39QOtpE%$Qw!RaBsmz?`FI>Gh?_T{ME zRmQpdBR+Y~oP8rj2JByWV4n-HC4=9s&%R9;gEUyM_SgdJZ-*-@J+N;9&d~*PF}#3e zf*D%AjfR7|P0|u>s;$6$Mh;wgQ?4c}1vr#|MDi+K0-Tx>0owfz9?Z&~q7$oRRbN># z2kEgDhxL`3Fnua)hT*eG{GpiLC`1D!(#PIsiL?l_wM;qx6J*Ny!jxwZlQrif^ss!X z-2LAA_YhfZA7iTJRyE&564AC_pTU~>$14wF60X`&$tFjb*Br4j0j^AvD+|e$b7Tb% zbC_?~7t;GF2aBRcv&{dPcCOI#MM}@!uTjSXDt9Yx4}vj(c*4^ntb)}Mkm+PM*ek4V zApm>5K^-+WQiUw4`y4t)b`Z!Y)bk=cH6)#@sN9$UN|X=j$X@h!2evM$f2SFQmXh61 z?$cr|O4WPceN>;V{?n-@GQga{Az}cmVsf8V#{Z#_nA z?!>Anc6GQtSjDe))xu+Lb#P;kEMeJ2+T#SG901HP$}1=>!PshYUzvN{JrNO3G$EAf zklv#N= zO@7;XY?}vc8y)r*yrWvb?fkbsuFYE@0y3*nMjxnABdCy}GN&6ka!XkW z-DUB8|BZ-s(-HMUB^tC)|QLakS(^lDsgKeea53cLg-QMu(Qz159FgI($1Mm0fp z)Rpk_x7yp&`1#d=K2}Q~MGgVMmLaW*(8p?fdzwD34D=C|K9FLMx+=*&3eiW@-p4r1-4hIDbb)}j*nbOM^>Z{l%FQ#@8(A#tKb5BFTx|E59Kfuxi6d1h8`=I1Bqs~ipNfUT z*(@ZvP5nn?pW`#I4@S+);ol78Kkm8&Cg>74ztrW1acBm>I1&wKQ^|{Dggj7voWUC= zHb&+{Wd=>gd{UKm~`Fwfh?0z`MO5r7yvyXFcm)Caf+)$Yk!mkY*!ao2q&8R!%Ai=4U-F}k0;@`;#@5EPte@l}4 zk5EN%AEV+vM#X)MO0wEqP!6Xv*auKA%*tVAx^pGTZPM6etP&24f2xX%#ed=?Pa)}x z_+3p^>B6gct;Pm%dsDaxbn`a&J}zP^2TOb>MD~v`z%==3?fy){v9oX{rLkLM#V6lw zkY@RkE@pTQ>$g7;jUpWwcyiu2Cec!&N|*)9Ij=K_&@d|q|MJ|2T}du+qj@0L@*n%E zvHhkVoTNso#8t|p1Atw7XzE5@(FIH&G9!gYx<9LN!w?Umr797i2C>nN>2ET4@bU5h z{6#;CG3oRyJb-Z|4#nkG)Ln}B3QEbOg8;%8CGadTdkz7>=Qt~dK{(oCb{O3z(0Opr zlUWpe5;eLxivdKik&jKE2eN|;1E~1ywHSMcH!h|M=dPTOj!Vm@QrK{k#@ifKa|9^6 zo-Wjm3ci5KZP*9DvYWdzcY}aEv{AX3y1(GH8m~5cRT#{Yut$wAirk$WSpYxlF-ggm zmAm`gr*tU*d_mJ|nX^S#59&Ed%xb@#c8k!c6bc<1Yran$fYKnQTEn~nHjN9V+Svb5 zlS}xKFa-7sc+kvig(+g<3@Td;(hm|Au@a+S9{0shkkw1Kl zJjtJD`6D`r>L*WB4V6y5EUJlKN^73B`vDdHsh?uCmy&0WfdQ4nC&9ty^z>F*myR=Yxh1Kd8gdy zNE`XcJJs<%68~dbU8hUe=&}Z_?sGTVl^r8!x)QC=p2$$vQJbIY_oRnFt$p;8jShXAtRE;x7i4mgW-e1>r^t+<<#gw&M#FLOR3LmSvDv zb|crT$-|k7<9phcKF5_3LO2uKBD(S0sfwWwQhppI&m{DJq$9MmXOeHo)*Md&Qm&r#0iDE-q%f!V!M#r#ol z;M5;w{|uuP$;uU-03L3_63J4egO_mpZO3N7L@rXDS)hWIFJ72k2>kw;lY_HXOwJ-S zb=0hZL)(b<+TcP=XwZQ7d8&ym=&_bNM=nA1iL4f<|2(qG&y&Gsj&gm z8Q~SF^|khTC9hap*n=L--KSjjoj(GZk@ZCX%ocvyD5M2Oi&#$xiOiB!p!V)+T@i`v z5_>ej$p1G4SlQqoc^EDYtXZR;N8i#MJ@ot}-}R*k9Rk;UKZHDL+UPoWA@vSZ{HeX5 zB&JaUkAc1LtQrpdQd@Z^-|mt1wmI0iD=p?cFB&D6?iR&J7Ie8wm3XFWU9we|j$7bK$Z5n~MpyR*v2=&V=b z@X_TQ>Y=@|WtOS~Apw;a`uQBRo9zdRpBs_j&2+#CM&BZhOJ(+8@k{4z<897`a{)@N zTWJ|ESLM+T6{%eUBFb6g>xrBQIO|u>=B*50!uV_qZfFM^B8%%wHSX5!K0}`iKz^(9ytOO(dQ!f}PoSYmk*cOOhaE4UXMnU&aumLHb>Y%*2kZvJ*lMueMe z@RZ~Bi`_(0rBEKRdrq#e19nx?c%+&hvbyCcQCm^yL_Q?=BgQ}A0vwdx2Wj)p zECtOwmob4*HVi|%@Su>j`cp?d5>f4tu`$Ijhp>eZ@iC4)0d-Jo+yP>;r2$rR)@b)! z?Yw~#rxkE{;8&qR@nKzXBWmvlSI-g4Sk`@b3nU2N89?Wl5xt8-g{;3?AQJAVHH|nC zv6jpEYv78hwZ!fuA;AGGP_k3x{@iYKa2LC}mCJ8|&I(ks7`!EDf?7-V>2o88VG-2T z*pQ&zciRa<#IV?g-GhuGxIMf8>6)z!rqt%c=r0@2EW1VdhI?6vB`krb7|VS#0CbvG zD*reY)mEz?{w%FB{5Z+<4#l+-0`F^-tk)nAHnGr0A+tJ$e{rYWasD!aW1PK|J|Fks z2G)bKCtN_hwsMsW#*b$0GOZVycoW)V4nioH#+-}bcddslq|v+Kv&QHxvK#>s#WE9C zYJtA}$UeeHm3m{g9+o9uzq2O1C@pMe7Am=0|J$~>aYKh%Mp;s zGJWYox(xY2-%B5+z1hd?01w3)Lw%2|=;E12Zod|`o@c>?c9W=ed-}T?#?{Y9OiltR z_wR)ljk0wHcItU; zy5efR5R_Z*9xB6IIRrQS%UlmkXq9Q^wF6@WH@fe4M(M#cMLXpLJ0A2Sv|g z5rZB`4DXCuE>%ecw{XQ!f_?Q|Q7FZUO2esYF7=gFT0F^O*c7^k`}n7G1JP~*45Rx2 z6TE_HI8%Rz5Cxixk{wwC#$GcWv0~`~SnY9NVP9CmNfzLsKjf|8uuLBZj&bx}2F-Ql ztSLqI6-)qavG$2kxVakc0DTh&J`{HT5>1nSx3B|c>KE9~tmU%?XaL>ls;9%&9=Y%% zj5U|+k@X=b5TJl(zqbd8X2Y|N`jkGRFI7K4(d?M_)7TDXnfhPSYbZJPmR+r2Hzb4tKgX7sF(zdeHajSg`?vh%NKe^s*Ya5iH zHq|J0Po<8UR5v5(92Z*+!3P{1qAC2q*pX8M07y(=k5MjGTv!1LiTlZwDj{cn&=>ii zLgcA^O4Tqdw_3;?+D>nyB}~Qn!ijn47Aq6<`%O~lavIZeZ*4b^YrAHh_o&%?CSYE@rlr7!YvjU#&6y<6=P$Uvf9C}9W9LQ}y0 zREP{_d5sZ@;Wc7LaepA3QAFODveD?Tdm24ooyC252$RWNq?)JmU@>Lx6=~R@&Iwu> zRE&buyru3GChkwpbl|G3qFvPaCNX&0ksu8CFc6f;FQ$-a^pp=0Wv=>+pU0e_5H=c>RLHJ=imsrKF2$QNMoOL@{t^u+{6gcw&Nne7omd(rNH}fi z2VBaDVPjw%5)h7W?q51*VDqv%=zvWYS)_wrN#4nfpoI^(>ZvY+tH2h)kJ2?iEU`HQv zaZnf81ltRSQFkFcR_e8>0n?+ahLGn$l3PGa^b-1$AiW= zACZUBF90^dECVd))zT3$=KEc!kpgvcs*ypGx*K)ac!4?x2V~sHCDuLD1+U{ZycnWA zpjMe@wDz`Auf)Sl_Ey0^L3nl)-oHx0J=E@2->_YXs4AU~%M3SOn(fVDANBCDW0T{j9F~F!pg+ z;$j#g@!KG7UWz)1>*!Fgh8z^c%>Eo4bH=3Z!D^RJlCb)gi%(YI6Ax2dGsYneOA^dr z0pt$$o%kq3V(nG!T&?qKP-zh6(7nbOy=O@xaf!H#4$Bs463x~8WWaP1IS4hu2C~q1t@H4!Ag*8=RK^7QX zEDdmhVK@B#85bC2nbKQe@U}Vd91hRs_=^e!-$t%KbS=Jr7g(m`o(IJ}58+n4zy=ST zXZ05y%|Tb@|6N(q{_)?M$V0+F!DX>taG`v%}q5orcZAsZfx>&3gZgQK0F z(2EfT?%=ue#n%NOf&*M8Ow06M5!5zph8fG5WR8-gNDN>@0yC&|F4qVdnNh!3DCeKG zU-WOtdtvKdb&sX|JD@NPo zbV4atwa?jvx7_;0Xox$!4YR3v-7+JwKJpnd09bL5h*4e81~SJJ7M;)j`MNS-@`KcecWs7+%2jFXzE6U@6N}bKjM8S)3%R$oq`i* zTB|8=+yVHhz!9^t+;=Ma2=N$>)#SQs)EG1Mh;j-x>!LL*0otxWgl=VY$uovxPYarz zZQbOaTl`d{6WIy^T;Zi8zHHcep)Zg~`+OfAP%G>Z%TWR+u95DEr_-CIIA5~aUa7N_ z+37EM^JmH&0+n;oyn~IhVFQ29R5anO7;Z*QD4(2q0h-J^op?%ZShyLrLXT%^Ls?y= zN;F>p<)ga?>Mxh)5p%2YF;b_EKh%&BqY~}RaT>-)-MOL?7haD|%amj9Xcj_ya1uUG zls<-aqjwH&E}c1~Hc-Kd$_D8@1CoPxf1>A8-CR$c02eU$=O8i1JkZVmV0Fm(p48dc zbr+KUlA(6gt`u*PwOkGAVpRs|0}6(Ts6{#7ZXzTCYQINXekbnl;CMlD-9vJvH-R&M zI2YOyUp>yn8UZ)0o}+ePzH`+AvAutf^+`^L6?si>toK-5hFnI@N91a%LKe7miTSwd zaV)IN1OuHd@$b?|<6J~=uo1zHe9&N6?xEXJo{N!GWilj#Wsw=t>gjNd&j2Vluliv6V@gKK`&=J(*#mET&JlKO0HB3v5b6)4Dfg`wER1NEdHcx~gFKJFLQJ(=-UG+Rj z46dudP5^c1UK3hpFx}XA3-VV z!&B%O6Ea=BGRcB2~Ori zt;|&=$Mq652I){G-SQ~`&n#ehXuhC%uEP18;v}Zpj%5OcHE|pkDO9}&-$Q~eMjnKm z1$8zDdwsC^1O@z>Y~}O%ELxq&4bsb{k}#Dt684sOX(ib#3V8v~ns0X<=G{SwdZ7T2 z54u!dp^JCwv##bB6ey4Q!YX;%BpXX{paV>cWi2F;{BN%!^>ZGe!oCrt*XDx0boxpB z+-mtI7`}0bfY;@ETai{bJ9bdKuWqhqzenJc4vr z1?Pg#s1S61L|3vD{KP@J=6Mh6QYtN5OCpo6f#{4(I!?uSviX{fcP~sSZL-g7xkccb zP!c~5&1{Z*Y=^csEM`_VA{h(&wB;mhrSRSTIV*JD3GPFq5@lfO=@Nc`)@)C`O&lJU zBl-ZI6)b82YV)+1jUav|CY!d4%MK<9dYf&|qt-CSx4GD`YZl&+QEimou=sYKx5n~} z1+GGXvj!OaGe=>kBup->eVd-M4FmkO*4UNomfZ?i%vJ*g^VkG(F4H_(ZAX;usPm8D z_$*OW5(f@jUk)hBsUI?bRdnM!E09 zp=<623c|LUuRcoLbs`LApAofv1 z>=M<3E!78f@z-=Ie!)z)k+<+s7%t-!WAT5Ex@;eEN!^I1)MCoxddn9&4_g-B0%xRt zf1JVTt-$F4n@7@f-%EeI@Po4D&OI#qHT2vMVG?9PL@*EG)*L5>W`u7>bICOUEE17X zw*g5dNTA~7xHxwiNuImW2a@RHwhyrEAJA9!PFnc=2Jdz6v&ExBjWkzU_^@N=});N z3o}2*BF7ZmHz}SsYQoM(C;{ zbTTVvq?RrIh|WmBBe@73N!xRFj@XW5k`!1CxJdqd19%lXBQCmIpNIYD^q!Fkpyf8- zg2D2IOdeaHOR#?v3@I)x+;H%5fs=FjeE$my$Cor(?K40hlibju_nB0YK3jU^i#E@s&u`O(shX+x;_&XMKbg^Fx%eC$wAd|(9 z5TpA+T79#MSVCWShn+y=Vtezdg><9mtp|KrX1UX!(Nc>}8jnT|JkJ+g-kA^WJx@n& zVW6|Rf~uSf!rSS~FD_BABLYN~|xD2Rz56@5ydow~^~{`PjEYbpx$!PCDtHZoTX=tyjac zLY)*-b>apIO`K(n{H^;lnF`i(%s)wzVWF9Eg_@JYb3(3<)c>$Zy$yeBAb4;_zCDtK zRr{$_1Pnx~YLit(*>-suv`_G0da$)n2AqJ!6pX(Sf0tnTK*#a9teO5!0klTVlx^Ee%5bB%`1Nxhnm;(#xpc-VH;Fu=yTMcU&4>^ z$i+8j(3iu5133@yhe2QXDc1WJ{)ooq_qH^4#o_PdeAnvP!~A*zyq;#a1Sci8>E>&| zGau(>7B^>=`J$w9?+5%464C}2o=wBSKan}bbH7sV%HDadpPRi@a5SV|gP;PR;lM|J zS-?#cQgx%LaJBc|7At&|Mp~}1^>^WKwyzG4SH+a0c9A)&!v$LqbYM%@I(AtaLZXZt8$nNJmz?fY(Oz*eD5TRV7S<#0-@W zE;2SYcQX605S1{_zDpy?&9}-AjWs$t3h<@pBFZk_4(43R870mGmkjP4S9Oc53D>bp*W#TfQbIyfLKN#Dt~B! z_&z?O!KeubdGSpt_nok0xb^-97PVO90;XKyI&zicEaQ6|g91|})voN-iVPxDSnx}& z5gk3I5gQ;yybS!jit}@wFrq_O{V6^B3hdEB!{f~IsR!@@rhN5-&=LG$BmPaHD!eR1 z#0J;XLpv#fg)OdFU*rI=QnlhVeKx8#M=k4~-knWQ!zQt(#il38W1RxJ#^2V*6^JWb zS!!z#RZer4k4zZyR^`~SCtx2UR&p*c)%Y7+X{5;E;ZqD7K+3V{5SU(v(RYL&C*USq zS`iLSmx{)e$wVlqJS^);$x|*qJHFsnP^O!?o-`C(aXTex^3=_wOyx|NWN;NxUj+dH zEI*-j3AS|L37Oa`^Y=|Yi!X?>P9>}CXNn%sHpfL|pwf z`H~t-)W^wJ_!=LiEpn@+>3$4#I+_lgLd3kh6+}#5B7_85KEK!O`L~jH4P{mkZenfl z*8pZYNe#&*d!O6EYEKGSZD%m0Y12}D_A*^SgT9noHE}p-1MXKt>BOHS*6J&X*7AzKmX_R_!FXEpf_v2=igbXq_ z6{!sjg0;p%EQA-iH@n=WH-WpzC~!H|=+@#~3Jn7w`*Nt8Cl;==F%Z(_!u6+x&z{iIoSQUN(O^yN6N(e9!_i@j`gM5 zV*{O2W3fc5R9E?p!~T*xoyWCZXEi zk&(4isaSI?wIk6VS|3ZT8%gx0q*eV$Y%~^6C8bjBj^1%osV1wDd2UI>Cx^yU6TKtB zn&_{wcd~bLGkdNZ8V8_jCk6&$kYfjr1lBoM3<1>&%ianZe-Ui@A0vr%G9TRb;cynT?-(qt( zIkYFn>6kZ!-_!~I7#wmAv>T1YC^*!n4@9J z#AIw7GPEn&J3fg2vEIT={&cgcw zM+Su1$|L}qd6kwfZ%RaxuTJpbV`duld@=~>Poc~$#MAXpnivMVoON#sj7V*pmYOt^ zV>naOcIg9Vlws$N@x-oBL$W3*JwXLhhY3moY9$&Sj8F7MCwrqKL&;QVAl?RL*A*X! zqD+F-TcAxRV^*_@qVq7?AX%FA{0Re9{{75w zK>793h7xx^s6>VOnx zK^f?4GiQT|8|g^{5~vaqs-v23g^YQePPOdWn`i0T>T%*E&H281Z0+lt7$1*`+mV)* z?CAvrt0n|QiZ=CBQd+o&Ak!jKwAwNc(^1<*a)&foH{LrMV-+cfMo&*WlS|U&q_^cc zqmB%|JPM~F1u9KF$ZJbiW$m@sRxV#&b!FA^$`#92tX#fg`8Ab`+o521)GLyQmNj)9krxLJO$#%di!gVb-l4sxH&`#S01ZPv`D@{)hz0Z*(FzNA&H!exO^VSyGS~(K7)@e9LFE9yg*p?y zYh^0{5t%}hHig+@=TlKcEek55V5X*`p$)ugelW8RNRyH4KtUD}gKsb;kH^y-?J+H| z5twgKz5;@85g8pG6}KL|P63?RF$|2nIn? zQZKHg53eV&!^SgD(=7`VX`;xBv|cdu+{~%?pxKEAsGd!OpmGweX9|RB6z_EX^uR)X zV*~8~n1vZ0s1&z@gqg|Sr19f{o1_~f9tUxi9wg`_q;V&33D&K*gmFxt0T}=Wd`n~j zWZyv{^Th;Q4A-c)FBT=pn`7fi`t8H19pjS&opu} z!Oh_N&7PcqUj$h`*JU;G)!60`G^>{_2~15$7QnjSk>{I}H-oTc4rg*fK4)`!K{^o{ z!)b;No=U}heU`9vHW9AxN@UF=$fp8wbE1C&8VTcNQ`lEGkkn~DqR5SbD9A8kT?~Ux zCgWq!7^#7fH4RysfNI1lPcXL8Q=!=IzFMTgLVeVt(f$PD`QDM}Xz%W5Vj?v*k&2GQ z;-UVLp=dHrw|-zE-WMH1z{#QkDN0U_ZcmJ$pcmPu5S2?b!R!DMk2_+~9liaD#8|X% zd}=I}h_;T!;>qMl?aJBfRA_i(Y)XD$vI4CGFl--5 z06~({o6Q`DkY#MB14N0aK#tC%L-Bz`D07M^20pjL{@zq9o5X^>P<_k#hL-wiqjgD>0LZfV zxF&WXs`m!4gOZt|(HQb$biz+R7xtML^)ypQ45^Y>yD5u9@_0+Tn2biFE7#g_+G|?s ztft8T%_y{Jlnrgw+V-Bd&eo{#cQl$D8;U0Z+Sqt(66l6P>Eoj$@}kk5$ffcnf(9f7 zq!jXk?k12fPlhsv%FuIW+7>a8?p}zS>0@*P>O(r=Xo1dQCt~AN^)mJuqlja6v}dA! z`&4wOKXdw{u%?MV439MTU>sM`qv@#{Fq5H4te%Vwgtm?hZLdv?0!ztcV%#zj+|aD@ z{=h<42iIB*OM#C@py5KP1fpZGO9;K~!I+*y(`SI$V=h^J!oJyoWBvY8O6$NtX5aN7 z)E;Ok&}lRjY!WpvVXT4BXl%48HW?eSfIu;aCg?kadZA{hk4&G)G|e<(e2bPtdd&FT*_DlFd) zEQ(=~ClG!+zv2W_}JtBIk9Ye`8ok}b~>}WE3 zV9Nv?B}(ro1P5qH*e-(`+r`buC~AQ0tcac|R34GGAW@vcJV`OAh~8VU&}TsS9XJ3? z_7=!MvnIR-89Rjr`E9I5HlcfThRw4~PZqzW#VgHiXE>1Y5~*6FDK7s9+0DxC8TM?o zl?a>x6HbHG%}J~kf|}HwhPODux|laFs7CKLpsa*Je_JJLv}v8iW9+7Hc6uXYB!i~e zU?QXWnQ+6j4yMm5NUSsCgp5JJbnUUxUH}VSJ`IUhA51Hsfl^dYMk631pvkftVOm?G z;9&rN2h7RG;Ud=&CE}0-{lKOv@zOQyD=0WO2mx2jjkpBg|mwH9S!R@b=7y(OS@Cg6>H&GiG~F*3rP*F z(Phh)t+=wSwWFb%aX=SlBM}4KLHdJ$aaUq|XYV*X`~GM=5sw-tHrYE6i_&ScqxW=l z);CA%YC3Dy)^yYhM4cPj>uc(w?X7FOIy$3m?HH?}1=)j!&eL$jDd5{8K3Fo^+t&y8 zbD8un$iC7dyQZU~wN{E7Th~TwTU$EYTbty$c0*13nIXj+6liIcfCe-l>IX)E`B9Ik zo$WQX^=CdYgLp;}0&I6{TYXD(bA5Y9Lu-qatgr7}-yUrjZjxz4qwP%%%?+K=mbP{~ z<&K)oV4#M&j?h5A$Dgxd0ZgztHp*b6i96?(&7MAH8Z?dI?zZ;&&d#3bx~`VmPD~>j zWkgGsA4;vdQYKZ?zTV6QjN94JT;JN&>9IbDwH6#gmaYNEG&OXfk29m6&(v+B{g~PI zwqESO;Tmyt6in1RhQg48? zTuxflcXUKMz-{$)Qq^!qOHH$pNuO@*9cHfR%AckTS6AQESz~G3ADaZDjqMWoA08v* z&@_z11?KwZ*7hDCTHy0oCCE0yN1|+qVlyXm>5vi7Gs7WvMkz&3Zwmwym!{U*jRwAs zPP}REAh(;y5OvoztZOAyP`uK1MYJ}9TW64Y4~ZOVZEdu!q1nfEd;R+QZoj?Jv5D=& z(zETUrn3{g-&s?;v8JxBUEXZII@-|GwsJ+Zb=|s^D`wBDo>=!GP)x;)?~xdv+A@^d zfi!T;D3>l@Sp)3NtI{K`ZCGz--nO=b*sAHQCq|MxVk0A9e(b!AjU)eqcmq${M|N5* z%IS;7S{|)4_Mr(`J+Tp@(ydtmvL~#@H~a&O0*R+I2M|;PhG<*djeF$wDjDB1Z?P{wxRuPNqdyrGWAQsXF16NzkdV^@bqM40$ zh`9>+7_+M(z*=w07+0c9^ab0j@OgA_d~6IjG^`4R32S=lWEQOKDa@Hwm!!>dA>etuQ_6ST?s%zW2q&mCnsBfZ_BUKN3^x2sfA#JcSELk0llNGp~Y%hntd=g zwLE*{@@Yiez2ZLc%tRHQlDVufBbP9K+; z*mq4wPfP8F_STlxu8t^VW;O^6@XHu%WOMHr(J9>$g|$s!*=X7ZVzkT&rv0{#4fRdN zW6NN9UrQ2FZ>F>d(PpqVa2qwIjCl`i-eR(wH4W!=wKTxp#v`<3XMK04r!+J8%h)+2 z0L{K^R);g)(Ydi>>#R^gE-~B_v{1Acvqos9iSjYpQIA+*Luw%0S z>e~T@(l9y(i-pMpum1e4tGVuX>3uAb9NG;%*V{iigr&4Jbzmv3oIP#~$l|R=mmB>W z8ld8aB%R^>$+;a+#sXVc2fU#5?X6vHMut|1k#DN6t6wMY$&kKwHf#nzsVS|`8rHYe zGGOw&SWS*T`$oN)Val7vFW(gz+Q*S#!kSTM&XGv4enKmK4ZB1Rg=Fz|>{5HA8E> z=`!&GW-5Isidr;*;o%cq(gQr@4^XBOpIek znL%((-R1@;`yRvuaEv*MwH$zD)^qmbmG!i@5{1j+)x=oH$dD&OXV|ky2@{!E_XYe` zw#B&)F&A%kzi<-bn77G%YFnF{qFd@4)^F&vsK9%R{3sj%JTV9mr-S9Ml}qjq$KDA`&MMQAB(6QxT|9W{2foJH?(bD zCB++Bk#F#8vm)XSXnFVweq~Ew#$AYedQfRYeO(u1d%bufzz4SdN0Doh9@~HfxiMnW zUEAe*gcgf6p0eoJ;yYlTTA(=*m4K-t8w9I6EbGyPOeCnO4OW8TOlELVQ(N2G)zay? zI2Po(dQ7FOwzIW8%7CqgA+{Mj%Rd)@4FbX8@&x7=9W!k5C{l)~v!;jI%VbFi)QYG$ zlI>l_#fCX+YD2mT0NVIH=;k2TVicoK4Dq7V-h~a{g$_0C?F~q7c)Z7akpLm4416Ss zGvDMxt1*V*mcHbejUP;aoh`<*kqF}@bS9P=pA*|^1O%C779HR=AXQ-$0M@r4G{ed9 zzB85Vp=28jO|5K=BsMxgSp*{f3C6$1K z7FMu~4%Z!4fg}MoW`lO|6g{<$$ctW0TSAsq(q_hEP(cq8iY;(?(rTv@-f6&m8QDda z6}hig;3A1^aDRJK0X$L$(z|3g{Hb?t%}^K}Z8cjA^IM4EfVZqSc7)2L zwg!YFR)&(C2M#PkB{7^$#7$Dg$Q=}(KSDtNf~HyfH+AKM1>zC5v-WgeT`Ntei;2^dM7ZZr@D!$nc~Hu&$zVs|c-H*&_AlHLq6i`+*rZDCJJdWEG%}l~`p-9@oqIT4qT!a@bwRaNov>k-Uh=rg!aw-x` zepZ8?_S%LPoWl`A-p{xXDacWirkg8ONEijT0-FhBo!>?JZnb zX+u70eTS`M+t?@I467wvQI~ddU>&FWO?RdjbB|h*pcA4qQ%6^3d_sIcJi;{Nr~vjF zk7JL9B-0?b_%y)oP@BbLGosjTIL_;85CVjDVf#!hG-$RKGKdHvJ&N6HU}OYDSf4lh zJ+L0hbWh(%A{pa?1oHXVj?8cyzZjqUz?foVSqg`vk%!w88&6=@#>mK2*3eW+by!l% z+N1|?Is8a67TXE;2|eL_AO-*%h;xBa;(eoIrovmXGaYom@a2P%m^nW|q^#>DqNac* zp33gP2Dw;gY;-Ka-J7h5_0CK@wb$2nVTEvWJps0}26UN~K;PDO@#TJ3xa>lSPsD6+ z(jyU`Oek7f+nZ~evI=YJYT7z2_?ep`1>2cDtgB%(*9M|*r1(6RmOFY$WX_IE_hCLTiQc?qsc++m*tkjeKSX#24eS$t7eA@@)5V9V0K7^*U8QbAjO*$ zjv$PWnQc#~AtQtCCL<;CYQ>7Ut3`kcQ{JDzKIymw_prqA)HP5=et-QVKK^PDT+0O7 zV2qH!$Chw2S^umNS8CWYF_r?qk0ziUq(NP6ZCeOm6WKik3L3!W5x48O-Rq$&Y1b!=FqTj(RV2kbvA#zsXLMm|9!YS-W9&LqG?{ zu!SR2g+N8jYKE}nI2mNa3gon~U`c!pZwG0I+hcu`>-v-2358uCKs)3_HrNs3`c}i! zJ8-VVOh`hBG+|>{5bK@T&DN~n+Pan`vpcg`!u7)1*0!E>lc6{gH~v;&=u|3spHtvr zG2Ep2B0Z_|`JVT%GdMhF%7+q@EanJQ8MM73d;m1M3v0X4fqr~+1>)O0G=lxr0IF{i zeZYi{Q-hnCq6aYPL~5YPc$@|n3Z1cforx^YY@r1LcOpjzcN-SC&tRpcwPmfOTF9MX zNA4mEZRcL{+0hBn>=-s=U~4_+E8EC9L->d0(F zr>uTpPO#zB7TmHKE9h%#t?#a{HCRQkDi0h5>?CRg zpcyM})j)FR&{!?D{NwOHKljDCVK)@`iYZc%eO4fWUCvY{#O7qO<}9VkO3|_qg`^g{ zkg##2p$02aW=_x{u)*TB!oaS>H@(DXsB1PdCJNr8Y_yl|1Mz7rIqxVFSQW!=7m$@) z0KExMhk8ej$Pvlnk4G~p&9>OBR393r2FDp2!Sig;x}`UTQ;Xwfr{cQ)4!Ujq{dNns zY)v|o-ZD(!0DBGf^^TwNl+}!$f(KcDqAf8pGCO*CZ6z^gW+gnpKnl9mMrrmz#1YXI zR=Y9K09IFU2|;V2E)Ut92?7EG%Wn4H8GLd-KRksmn@~^EtA-l^l1umAykx)r02F*2 zng@zY4!JhdG@l{z1M{G7h~xx=T-q4<%78uW3S5mt00t2L(bG8v!e9m_5PVKW8{1vV z`7j|Ck&}kZ)8+z{%SbSra)1?|FOYzp+=IPP!yHb868<;Nx1#HMIkr6v6<`}8g2$N! zNN;FZN~F5^_Sx@%B>=XbtC-1kiSaG~;Z0YD2+(2Lj0R#5b_+^UD9Pz_15Gr!XQp1H z{_M%wVln5D6#x@X!t1@lQT0#~hwVLUi(6T@L)j{GNAxzqD9WldoAW_F=KetX!$B|KKn+LFE^{q9WAUqGcT{1ufU3}jD{9MPtm-{p34Z$-cDBO z7MmiOotQA-U4yhuKlIk1BrObeLw!O4`hrp_zkagb{GpNlHa+9Jbi#SiXnqqrZ2^_2oS1>ET=*X6KxPkxVN#r>iznq6A-8LV zc(fdc>+$8NDcP7kz^qRQ2bIAuahaxglViKU((ps3v07jZa4J#H0ZsMn0i|!*jB7_c zm26C;aL~*IR<%e%H{tj$6c3L%_T$@}BzLkS4plPsJ9MxWAI16|nl~h2h)*FgGlLJb zVE?p#ns`=iGp62Zk62CPR&H0SygI)$W*x?~H*iqe;df_>N*KI*bSrI}Q6x#JhcB1jm7~JCSZ#pVwt;0a*-VZ8en&n8fl%EXK?EZc)c>zkdzavR%?1$TU-RR(_x^givpm{yW_&*&^OE{t?7 zm%=%(w%B+Bj@Q9ez?T~10KtC2kc2+&8D^NT2>D?;NF zah^eAEY2mX+V)xyr}LU9KAq~_E=%7oPv5Rg-(H!%y()dXDt&vkxy7>oHPK~aB<)Ec zk~$5ed>8~C)_C7?{;c57O8#Wkf!^Mp+#TrCf+hW1z~$b;DEAgfxwlZty#-V5uTIO2 z#1|ZASMOL9=7CT!AmH+Lv~O$z{4Ls1@^zsYzIB4U&vuak`KrJSltxZ~i2`F$C9qk_ zrQakzauQo=w7tD7!9hQK%5Mo)3%S`Svveu8bS!0pYN^o;^6cxPs=m;f^=eieY@cV? z#|)5#mQEzcmttSt(!svIrTAWA6yHn4lERMO)Dnon#O|fbmoFWP_l-7EhYeJ|nA!k6paY*vcL~v1h*olxs|(?@Pvaw_{HV| zd4rE)vC!Krj9aWcLP6pN&V)Tl%m>fo#XXMOP%R8#!8fpA&Jye#AMvh6$K%PIGs6ve zg)SRzrEgr{W>N787MRPKTFJMjW5FE$pxFre{f&H@wR^Cv*&6`tTfOx*(*e=+QZ8pN z4K8QD&64Ad=}Chu(O{l5{qYpMfD}2{K5@m@Jo=IYe~<4a#qQropk<#J&){}~?PT>8 z*SC|Iy$NL?(I#erOD0+0W(d55@r(x0_Vy~I?17nolYHvi%qL8wSPPxbYyBR^d|Tx| zI4LF%Fdr}DgNL_Q_sH0R(J(Yum=PXhUyPSmr>OIu@UO#Y{%gM8>b*A2??GL! z`7d&QsnGnF>OWPwd00w2y;5m~E~RvnSGq|+If;h0FTxWJRiU@4{W;F&@cx`TbEMDk zojE^8fq@qlI>S{z=NsFvl-stO!c~ts&O6cKz8qZc%lQN=Zn{7GiJZS=!C&S)Z3?_b z_R4YF!&P@U&bII!j*Q!WKYF=02L<<OBV>r!9QIxx-Oo zd)W?uWveZu{6WH@k2^Wb?hk+5d4LrkaQ-C6iK5!mIV^pe(~IJ7+@}A9Lyt*=$DD7o z!MC07JAm)|&L49U8SUPgZX|c%7oC4*lYe&JOHkhnB;x5yIbY`G%NP-bKh8PMr{f%V zo8LaX^m-ItkJdMt8Qf$byD42`+Z1X{p+<+P(P3(Ignm9JB!8pVuWE0?2j|^C4-~zB z-qZ6y(x>M=!>4EFJv+}(_St!_u>6&IKQRT;42}ML9;p21rqpMjiq|X9`1Oig19iz( z6~C?^r&j!?LYN}_8`N+*JqYCWChs*^L0)ePLAvnIw4UL$Z5|3AEPJ>N!#-U0!*W3S z!}1g52J{od9dt5WdPwrWiJ~LyvBhm`Z-$WYziMOJ6;9^A1}L`{Q#W1%fD5Q zf^V6QQE;OCfeI8nQ1KadhJw#je4PbfuaMqFZcN9f&vx|rayj_oWzz-tRABQs!_?|V ztsj(wa6bUt=(pQrUf@7L>mg|6V9VpCn;zRu)M6?y7u({0(`4{R?Eu4r;GP=TTY6+oKdWgzN#7CjGq znOaWxBSb!D_*@wz4T9;U>475j<%FN%OEVC=4nI-+qhb*LN5v;fLHHA;uaz3YzgBvC znMe31ik~b7;h!wNy$mn26q5Au(_#?)r^SzyLhc?heSoB|mF}O5g8g&vooi@(@7(XO z;5&1rqX3O%mIMYJwWFgCmw?hAF8S9Iv;(F8wd9FX6g*KX9iiafxsT38!K0?5w2G1r zy3xT8OTbG%ERk`7Odu6{P~mtfxa7D26-*%0?Z{#R1J9XG%uxKpB_AyTnLb+bSDeyc zmHe=z;9ZvR?E$4Y@8%NP6S-Z>XVcLM2N(Rb#)%%YcpEngFcPZa;UIOk9aaQo3xUWrm%4JGg! z{wETHsQPs=P{nGv2;w_@rJu$SDs3~BU`j|GPvwT+EZI*O_m`rBDBgW?E|@d;FZ@W} zi9GP&iM;3X(aCf9FXS5@d?EiQ1t|DQ!D|Jk;I)FI7)IKn(U0?hjUVS7Eyl}fOvy^7 z<4x%JzI-tEeWqhD<_r0+vf$Nx>6n~au)h!m`wQfk^_OR(4%=tq8m*$}0OLL@m6uegOi6RtyqUgmU^p1iTi|#B&!JWm@yU)HdG5cd+ z@0Q-xp^xqL`tN4{FU^snm&^z#;s{b?$3W4GMSof>MSp5Wu~K0A?Logs@%qT#z?k9J z@=oS~QYSf~4h(Z&0WJc32f735JN(+=U*+AIkGg`%HdK0}08jT7K2QkOq^isOhk7Rp zfX@>krPQ<7{Q|0ozfkx!;`;lA?<>OP6GdO;<;9{mcp-|c8ZteG|11B_IcRj}oF4!Q zkL*P=P8~Nd79B6*i{cMrVvhtT22S|&oLAZE)j9u#R)51*$2s)z!Z!+?ZnXJiF`hn& z!T8p~9=@;Od9qSj&njB<$04}o7iC3#=|r-_hH=L31xi~CyPw^-|2h2QsU zy-{>4N4m8b3}9=$ujpRFeQyz%#1?(E=$kD1CPzdO@EQIJAO{eF4?_kNVNRP2@M7r& z?^2Tv*MG4^CtiQPn2(<)Cew=Fqfhve-0$T=_`a9>(_G;Dr@24RHL~{e++VZ&*SQZx zQ2tQl(TFL3H1d1|<cpUGZwV>qK|bPv`yNpXdH(F2MNDTk4C;6LBW?J zUty0Z_)6qkEch0Bl!9P?w!=+chY-!5=f0VXX}@VYM8Ttx??zDYUDG~GRt-SyUV8|i zv~Ty?qd?lXd+i~OY5ooWDEF0IP~w%`pXEXm{S33h(`%78BDlOk6j*I^TjWnz^e2(` zN4%mRMozHkMC3KUXn)?F6tFw<&@k*KAN?XCm+*`F6>S~F@UPrk3HYtK_hA|qzv2JV zcOXCt-@({v8)7F6i9f&Vgb(Pi=xL-4he`lY$#MX(e3jvZRqw*XyAC+!>45X3gXbrm zXH7wXis8@#K)K*Nba0+j2hk5WcN6rxP4#K&I~+O>Bt35>09(o7K;~6bc$kv*csk&G z*g=gCoAw~OAzJvX#boxy%P_H3bpd`4UwA2sF1-qB_o`KV+3ERKAN9hOrw|tzymsD% z8aGX$MghDi9f|}de_nRp2 zn!_j8cQSOn(=-H1UUq-Zf}gwJWQ*x2`Gx*t7gPRY_a)NnCDRV5kO2{Kx-t?99?E@! z-1-Ct6x7)R6zs~Bfhkp}xx(Mi_;g`ZH z@6G!KC*iM#pJ%NX!auHk0F;SPwxO!5%^Mai2~5O0=z zhH=DaN*^l4(?g|4Y&dbG^lL2tTIrLfz~Jx~CAXJ?<8L>mKCRQ34*#O~zZstVckxpt zX#P~mvmE5vlCLu`=(<1rb-Kix!{04=$rij+a)JdXO8&(Z*l~&v&~rXuO0&kn7-HVLa?-zCT@iv$n> zWx4NLRj2!OCz)}6|J~=_&hzA)^HzP9`qcKRI#qQlURHooI<4YFj=Cl)R4WZuKod;V zqm5WnaU~s#l@*WD5kE>1=HvhrNDp2S43R5>uhG-G7IuejE+pRxJ83!{QmB$FV|PXD zV!GeOp|cqq4b<$g4PgK_C~^RI!j4w}aJ<4vI<`074kIcBzwI4IqzB)Wv2O+sg@d+quqWd33acvE5;0|_nT`t~`M8i-%KOv&EYnI5 zyjsRt9kQM--};dIp=f1%1ixiCB|puxNX9P;ImGyfLZE0DOZ+3s>k}tJ&O_$v!Is_t zvVN4bKZc~yQ%r-YqsFYa9o4 z{#7r>wUVyeO4oikL}I^UiK=%r=n8==iX{Na(pNDd)k=g~2ZKG4q?&qR?V>jtvhF68rVrI=-oz?(2oRuRJ;L5-rm_GD0SP# zOu6m4OYiTlYfliQ$XMp+N$hrAX4qv{4*W-iJ@&YfkAp@r>qZ4bVfrE>hpwWp*Di9M zJLzETbY`)a&kCHT9gJy#w+P${ysLq`fwMfWXRdQKor~4ZS<35{CBgg$miy^i>~~HM zgxM!6mN5IYz@G^Gq*wxwEX{mSWQiaf8HhxVggE4}-*8q%>V`;D1Cc9J6>khsrv?5@ z;Ah1ffaLwQwGGJyV;Xa0;3&EqqXN?@XIfxBFLwjSBLmGOsM(eS&I@!^E;teK4~V#c zWD5cp@@XN+$c%KjN<#5XU?HOv29{WTGlqHHmVL%JD^17GS0gMmZI}hd7!|b{UH4CAvOu!u65NQAw6nTE~gdb(JrgTxK&Z2T4^EC1QYdWBgTej(TT|lU(H0UrU?J?1&vFBGU-5M1~23^ z&ce{;dPZ(}=sg1WLWgLNVo2Ckx)@h6v7&m1pTd@zQyt1|-v5d#27$rT>1a$>Tv1P! zhwdS;M{xx(BIZTw>+MRB-<~^_6MI0mNQEx#m2C$VVIsCQE@D0skKX@i$Y1zuaf51 z;Kvm8n4;dch`LGVM~Zse;;TqulcWbRDP$VGk7*$wbXy2-vk#YqZtb+dYT1$`J7f_Y zl9hwa*hZkow1~4p##s@vnQS+Mt#nMxqzpl-rDI|aZ9%bDQtS=6NQ#Rg4@2xJH9fqw z15*<8NXB{;GK3C_X6JQaN`mG{<1P=qAbpezoM7uL6JWNv+G1J3^JuAgv{V-`+CY9o z!miNylIhj6Tzf(fGvsjSQ4|SDi(H?@>9JXyF*9c~%%qFrz%g`I#<-@?DVY+4KK2cC zqqDVdGCSyB#QznvS_7+t&H_-3ivurkuIfVISqA!7_hNR23Fr)G&^MW(SfaYm4*HtF z*NPQ-mGMQJL zcjzcQaE@m8KRWO*JPK1sxDw*5^9P3i;JnMe{w_Gn*kkBOj0rqMM?$H$IB*3ytOz{v z|2q)>Z=2lvI}L!10Is@u(Pvz_?%HiZgMI+nD)-x~bxx0a}-GKrCGGATE+&H6Tw zyPoX2ihNM}%S~_nWotLf*q1+7=KX1#2EvNh6kl4^DprybY@k#+PK7JDMN%2#y8WqIt zjAMSpNz~LxvO<0+)YM3#$Pa~TjVN3kjXM{|Jc%0%t;AT0#tb|pc}j(hRuX$Z7CPOJ zofij9=EcpAQ-yne+(rT$<2ETkCP}t&Yry(0q3bnC~Dc@Fb)AnXjn%_!cA>-(n`C%xtl; z1?)uvqTjc;&&T^nx4-#BNwX0&-iYRFo5M(Jn{VLLhUWRrmGSbMZzsOJd7%PSp(|;2 zzZtA{znMhMGA>!6X;N`@^&^28**uf!$ZWp8InuG7LeQNco~JH|oMx+Sc<3iUKV&9*j!23wn*YvyTyL{Za=O(FAQ z)9I8sz1b|Q%%Y~pn?mOCrV=$}N)mM73N>X)6dkxiO_>tyW};^Ed#UN|rqJYe(@cvd zX-y|nlgUkYHH9X-n(l3CNqt(=(bQzLLQPF1Ndib@TvL-NP3MyP+@@QbdeSMgAg9_g zt|VMmEsu*2^Qy0_t}=LC^}_@XS3jWuX>ep&PPJ8BVYsSVT6JVunhMsk@#f4BH>*vq z4oQ=%&#aD7@XYF8G2XiBM{zNET&i}p8lqgSwz4`#59_KQ!o>{BsXmY7^Q!-g%O2NN z|Hiv~QAH7H4Zm&(jlXU6hG8m$?r|YFG2D1)GjrTnAY&eMyy9aHoDmeI^S$GvN3A^ z%*M+a6J17q&^ka7)-LY~SS?sypxc#(jD4lyU;)|2CUa&#+A+)@)p zJ>9stF#?MlPica{DNPER5G`ne1iO9a8z=z+wn_n88;)%3$k^~HxaFyvyvCczXA|_4 z0koXrpVveJWXT(;srm;6GpX5zMkjcg*?1{g6g2s^37?)^{iLqB=HdwPeZz}P*Tset z8X>Q<8r|iED&TUuab6Q#3YyG*QWpSG()b=3-eZXF$#y?^6kNs8Cy_o4-E4B7^6od8 zNeFL!Y+`G}BALu0Ts^Ia*14#`i3TVVCmNh@fFgaq!G#8@#9U}_lfcaecNu)Q!99gb zDpAznPy-a2Lkcx(S5brS8nES8s9C!tnrNbCJwH)aTvMg`_v=IE{rdB%!TbhGwMt7H z>>#kC!9E{d>YuC+T~5}Qs7Dv_NB|FsdUUCOxIQZ0VTGEy6ftf$Gp^@7hLCu7;8Pl) z@J~@<;URm5!bJ#Sp_tNOWdoReWrLlJxDye5tSpKtAQ6lfH9(9ixwwFPBLFN}k!jTO!eH8#!Tz6tL zY&tRevw9%>tlo%vDw{{t%O;RrZ&^JAFRQnszS%Go*BufKyAFxoQNN6aLGr^()ab`g zAjyxPko?dvNPc()OnwQ9pV=@#huu%mT)UsR`2-R@0>bcEUga;YyDOTpcSWCxhH=kC zKaNI?ejGjH2|Udp3%8Pn76H;2by>aKdI-#|cc~r%FV#zrA(|e8MEO>Js4`bFpBsJi z3A`-*RC1CxUS7jvBtUZBcrpgoJQIDJI?a4y9hDhTZzjd2$7ILgaxG?LegBHSCHfo2 z|0eoIH0tzE(Yv3(MVhJ$lpY>~r}UVcxJV(Qpk5&v7BWQF>KXMHfU8(gA8FCh=9rz7 zw=?D%W66p>H#%1)D;HOvA_$#}>fEe@EWTN1PF+-@IdvCtG0~#Bi|eX_u(nfF zLYswkQT!Gv8u)dK>mDRNmOSF76fBRdp}d_?!BrRUJ+JMl00RPm;iO zQK+e(L=#Qa)K8+_Ow`m*qTNl@)Ne}FxzulN)vZ-MPKdU5PL);M(Xgt@4_L=8eyDPd zPuHqk=Mr|jw7y>D4)Hr!WN*?mc(?5K?dHb#BJ;BTUKXnaT1=-R+X*UqH;%-UJCWqipM%HBnM(GYlCn8XOPbfxs6LK`$R!qr-1 znOkEcrZDfOL`+qgrZOS7R$eV+LS8M2nwcP3C4iMgeKJ9bNkrJiS}^ZLB?k7sT5AGx zWP;-FV{b*%4K$Z)!IGC14NQBrRtAlqp=f+et7y7|=1MIX?TVs-6|dIHq@6PrjmL`O zV6CIIV4$P5&ewv0uGSh4Q%aW<_d_DsbPS;dEbU}k&Dk|!k=Zpz&|o8Kjn@VnUu!7< zWuLT~FJHAmJ&J2|rEIHqZwqDDI@sl97WL|VAuSlMc}c3iupv4xxwm+V(3{Ai=M;#oU1FpmNhIOs@ppwZ>iu?*diw)jn3l zTUmkmu(HDD3Shgr!p;h)@#gA61b4-)l1op2{vpnZDwRUogDz|%H~I-DeUvJ*Hx`x+9Mg5;I81Q|>V z?7y(^{$k(afa+Tu+Z=e>=Gd-wZHw)W{RH+qiWyw&IBVI>tr7?hC+tYk3HxI^w0dlp zq)3`1#Y#6tBaPgC}i1lw{5RHrXqtD7G?_ z5eMPLc9xf7Tq`CUj*>A*EPhd~8xXubeMrIWef~VSZnCm%+h{SpXi?8h8*dgc}dm)1hxuIM) zBL%i&wBRw@W1rnFLYZzQO}EmPr^Cjfu_(L5nmUOz#|oNpg=0LoD~uO+gveN!Js8*X z0n+}s4&_pWp;+3Wf=&b$a=9g1GNOO|=Q0F8U9t&Mxa1~H~&=TwhH5E~{*L;#!u~=*p;Ic^+ zGpmcmeSxOnzL*26Fvo{Fr#TUv>)go8M&};p`X1*lY|@Y$%m#0EjCJvOtZM>UOh8hg z1apTdmBy70h#@Xw^^Yn}qXTzgEwUFyLM7y8w7G#BU~(nS^IsTzebW>tR$!fx?To7F z;*4!GeW1;3H!$Y3@6avL4tp9}Hc8B$ud4`6W3%X~eI{FBaD`dmcQ$0|cZw^V)ibv3 zJV|3axppPV8`PIC?lUGmG4$}VKk$0nstQ1 zI_{;*b;h=dXJ~Aq$OI+GS}la}DhXbF3j}X{$+Jk#*uJpiBAKiC2l12r9$U|Q?3BCu zta^moW%M9}9v>jW2Vn3A#Q^i;_T41fZQsYP=~vi|+2#F-5B`qezkhCXr1ZlYa&zw< zSagvr2qU)GM~mymdWd~Ad5%VV??|>~8~5v#-5!hF4&Qb_zJ1#vqa$)LqhnS_)xc(T zEGAIg@tg+EbsYbUnV-2G@;V?}@;Z!vrgWAnmI+`vp(E@$L9s;MWp(_Hz;}uz0LfDI zW{M~gM3SY~6O#IM1Aedrl6z1Edj@2RtUJj%!gk}KBNQ^H%8ELi?f@g4>-YeQZcKn`~?_mLe=Q<93#vBIAZZd5Z%;E0eG~yaz^Avght^vZ0X$Yokf;c(6K{$D4&c+<+N1JrQv}^eu(v%N z!@bnrT6q;gcMxoBkCL%b5nwDWC3II3u-Q^(owZ{Tm!7`#H1lUn`^@$*WoG*o?NL1T zBlScNw*P_X5AClLz25#7pKi5(Bm;9gKnp!+08f!i8&Zc$PeX_8?f19G)1&q|9q4G; z*gzu6rKbZjayH=zAcpH(#!#u5e zPF~NoyT)kO+TCnt88g6~C9QI~f-bl95RRvYv>l}v8IEe3%~gllZI3=>b#hBut>j9! zm91qw6!KARCvl13B*ZIK$Yo?~uWJ3M4YK}G8%cvCN!|&Hw>dVU+E51E6G8AQKD9y;bOXWgwx|`u6#)vlBB0T$>zo?$$Kw$LlU!ZX`78?y%DV84sRL^?kBi1@nokv1pWm{Qv2ngVaM{mHx?`P6h% zwl!a{P+?^gNL<sx?huu{EHQ zB~}q)NP>9&Mc6~7ZxnF?h^I6|k)F~_@<5T6!ZAhPa#u^>yIS6%aNu`ZPLBgHJ#H>n z69SkUUmOphIR2IfZpF`P1z=XI1(c2$3tB;6Yqfy!q|S*_XOtc(4DDe zN@J^9ZsF=b#M=@(Jx;}&9w$=(j46TZP!**r-Az>hC{-9qs{%}^ zLQJayOsPUls{%}^0?gDwRx()P=9BgeL(1ZnB6uZRUdf7AlH-aW=uGWPjV%!@G0bm` zdh*_rXu8H=X%}nVtY-UIPWClB)69{A$&xe8Mm5Lt(&nppS=D?Tm)VVLF}a1~9|)b? zVqOb|w%E`Db$COIT?{?Y;sl>gv_K)piz;LW_@8Ncp`|359=9V7@Q%1I;-!EuP`oJk z5lR)W-!lPlmaOOAnWm5>VvN*tOtVQ`tv0FI!sa5Q1QxvBVpmJjv;6Y9g9TWI%#B~b zm9-04fF*D%{*e}m{FjKd8ibnjkZdk2jazZ!pPG>8PfbQ~YHie$+n+=dnC(Pm7T~%i zc1tWmx5VD$5=?P3_AwtH$Btuq$Hk?`p`$PbnY)NoKKZ0?Q&ZUrZ)&=Y%ha|tEozF6 z;!M*=e0tP$VRJlPZ?USSjC7|Z7OB3CMXJ-|row zn0LtTc{JlZWH&PDMzec-y4P$^b3E;7ev(TPPc}zBstm-1?E)^W{P5?4O}E%$Et9wP zisE}fT5weJapXC!Inn{&N?ULh3Ng@CXp2NfF*0)r7vJL8H&~RWyatovaZ((bCRj9X z6jL`Uo*B&8C)q+_6&Ny@u^+_`ZH337t!5#287`IW0y`AS5?0viEW^*rI*)LvC=!OX z%g`aNz`OidoN0cE>Rw{8e;ZK{hR#{_CHI*m)VS*{(qs*k>{;`?!McxN41LQ;x>NGR zykT|<2e8mfvO}l9i0A|0(fN?5M9BJWMzNz$;-BuDCN|G*u7#~?xz&`rEVdw4qZ#pE z#OvE)EKJw8V_Qwtii!QLzPIS8vr*)54GtERJh40mS}u=S7lY;$oz=RS!!f`Q$6&3Y zrO(QF;?5K3v+g`0MU1{h5@YpO%+?0Lw>Fs75ORUfYPh~3fb|WJXy8c0{alH+ztLA*K8YA# zaowbEf97otl)P?`HzWoXVTh84K19mv26;>B17A{K$^$Oty=BS+aD?jy-x8Th9snuN z$4iuH>JAxeW6)o&RWi^yNEzLgK|I~2Zm6KxkQf%oburWH^ZHrCu?=Au%ID*ehMza$ zb$_EPjQ~?99=$(UROHp$TMy-ZZ@nQgs$o1F^RvF2)o?*W1Ra6d0oE&DN-Z$&h8ENS2YW8iTpFv0|`p2-d|E#pqi`Qn$2> zYS!}1W?(`+Gy{x`%Fl?ZvGO62xGH026o+i4P)lc+LdG_n+EB||->^{M`sD=93CRhC zoSe{o+~&LwJ6usy(fAz3hGzgn!X}0xcw$(l-b0-kHUoeU7sUhbEVxw~6U2%K8m7ac z4+%UBmBaufvE?Wvyvk6LU)BC0}qFu!o|W9{!UKx624qo5`8!t2|XNrBwD5XNc03Q z#hma&I$P!RCo-OpxDp|Wx`Ifeer+uoCLrtp7bhN24AIseiM~YOlA_V0c0^l=rG=s} z`UEZ>%|uDvVbp3^G?tnE0n0+C@@Z=HG^Lz1-aRf7yjM4i+GjCdS2LbgOYuqoFG=m0 zF7PnHT1Kux!J_4e)m(P7I&!;ShqFC0mkXA1qw@4hrM#$%1TIEh)xg!L2jkX{us}eBT~@yUN)?DK9@iN$gh2jz_Hp7{MMWK zOXD*nW=6vFnUV9jZfKqo4AXDtvX=xbB>I{7&q_Y{B+FX})PfCEY{fU8k= z2;7O1=?0Krdneb&?NpLH^Q}nZY$$~S|4Z$aT+gmVE6Vw-If%ObZ$VkP}x zu?;sUQ|NUbM=p+XWOT~*F`7t^$Y!o*N91yyPi{mWpYkGB>BU*AA~qA>9I?f^@FzV2 z%V=;wfn2Y85jTVH@|1|X!T0nDuJ?i;gjn1t zAB*S*@Z7|gJeyow$$G2eiNbiq^$YP|Twieq|5rHfRTa|Wpqo7S^=1&x+r&Q9;FIcL zA#pPJ$6(+;D&b&oFL*K!z?>X1Jp>V_hdc<849Zf%fEMd`645#(9)<9T>l%S;E-4p4 zanQLS0Ox|FKmaG1+5|JT0PY26@IcNCCCQT=WeJ45JQuPk52>~f#EX9=7iIB?>jndE zD7gTNdAt%<@diz%w8<*Hz@_*^Q#^os!Q(;@aU9Z1mO8=K(!7CZ=3>7-c;K=ml7p5&D2q6LfVECIR{-S+VX!0^GZ^{}!7&-284^4v*h}ND z_8exDRhc3zxkA2NuDO;=!PeGVrx$jut2K;^!ldA$n&-K&>wL{&tQBSxD=8B|DN$`_ zv98uOF2*`o>ljyF`MX1+sOAqf>3{rCb0njUtu>#Q`Lz~tS=pjm%lNdc*4KPmSL+%t z*J?c=C-WzAYV4{3g?H5$RuetuFsv+tPUy8q*4zR>DJuQ2%UCm9fszr(@t~!)^#fDb9!X@Wds@+ga&c%&tGq_%D zMvWY`UQH#kxauXY@xD}5qDZ7v*=CMmz-@FjiXB~TJa>bQua?J?-Sco#Inf){9?R?F zYS~<*mJQypvK{1PfN1$LIKSF~YWBEnae!p;9x`YkE>|A|6X?QeNroXFR!x`0>D6Xd zgXEc@g>@(qm&etnRtLDP`d(i4RzJ!8Yh!9mryZx)m_?TACvv%-9Bon6GhE?*rfRWu zg?nz*O-%hJg_>njk|cmc#zmP_1rV&ZRb{HTRW0I5^`feWL2T6-FI2(~(cL=~wiS!u z#MTOXDyVX`r@~CGyqj5hkzRSXsPa7MBZ)zj2Y*ZCh0UXGN(+ZB^yXROl5qkUFROq8 zyG$`gk=s)ty&{10ir@3BBLLr58p7k$hbTz^7F9k=;4B2n7?erox@;s=V#oau^ zXm`bn6-7q+#=+uI#m~ZFzt6%q^FXEvl^0fq8IfR>R!g0MZp$jr;(KraHm}pb8kDP$ zS|o`|5}iM?x7hph))&neG`C=8?}lo}-O3lqwwJ(;*&?jSa<>kW*+Q zE5Q~jBbe$Tm6828e5djRYBd3+fasRW+lZcFQNTKiDzE~YjwDg_0&`)6 zSQVyG4W-Blo5#XD4<~`kQmkw!Q7rOMsI^%vw*Ab7R6pCk3_w}`GT#E#^Y*dLMNk^P=PS@=j8actm5`M5D~D<8K8KH{UN2VnPX%e2e>BU65_1z&fW%EdWf1z<50 z7BbV!ese=!#pNAVL1t0w>0BU&tasp$I;LS5m-@I zyIk)%TVD4Hfh%>dGWcrUYu2r2lBg?)Bx)(}P(g+zAZ%G(WW+MX5K1nuyM@3OMPs@( zW9w#d-`L!`dE6p~;l&ycPw9TGDT@kH)|8{(IJ^`ny%Ly^U!iXJmHrR!8^m2fyojs*7Aazsr^T-01dfv!FL@QQOrDq}561Fg!v~QjrsHU7{2Rdr zY(&6#-GGe`xWtuvm#_# z&-w3+bN(UQd;TtY^M~SZ0`Q0?nX{w+#{)$?%|XO2%aq zPu9YsU>*zmag%;2H2((cn23%->DO({g6n~D9UXj zIN3-sqW-U_R#;-aO%&?q7v=L_v}@MzW8#!gcy7S%0QoVj-2tayEv8&`DY*d!0q8>t z0=5M>24bbmwt)M5-oyjb6TR={q^Zn*Fuj?%OPthhiE2Ps@+>lV{LUPjSL6iFqqXM+ zZeseFOrpn`y?DEY``9d`-bsVh%Cil3`Faeig4#!{d z7I&Cqw8JqFH{%>LKn^ZnI$(6`Po&#&+0*3Oj&SYa5!-3Kfb+C1pX(0u`3W%$^{^Px zYYW9dk#+?sR)AyltVq#zoVG3JlF8+Y#@ABmwlCj=4^-LsARYFR0z{`5(Imc@ zPD3$q&{1^j1^yol@DwzC5G_|9cI4Vab5pQ^O);W^WqG8iR2!7Fal)1{u$;y5BF(mn zS+vTQ&2}prU%tT8(!g(Bc>30LKsWaXT)&uSJA*=Y=Q*}tK`d%E=oG&@R zvJ3cb7ji|$UU7g9@0a+#3C*+y-_3xTzQ;k3C<0fUzc?KqqJjPeMh1Mqh4f*)2qk(e zmNg~3$5P@fR;?`ToPoY$9MVC~f(h}o%&~?RTmyCF#PB7xYv7dlg$cgs8b#M=RM1pT zrL7Dupy3LTGRE~unk7Nl6oM_P=(QXf$cv5QdH|lKs&1vK+d4$F!PuK*aoS|NZ{r*a z^H?7Lnyrqmlx0Q&vb4lyTh6qHK~-{TW&^A&#RThWu;+$WJm@&WL>@sRk$smO*8^k; zogRqFia8xiX_2_Myj(}sKv8~-D#5(8EF7U-O=)8svt(w?;zwyXgaBLYiq_@gfa|&} z7iNK=zT%SlHqQ-+He0M2!3${J1;H!TmJD%2<^iM2pL$LNcns>pvNWl@*@xpxdE<@9 z82HGPih&yy!}FObusRlihDm11+cH#732GJw+><$UsOALZ24Kn{HvsY9Qt{tHe4JsA zaXC*-HRl+#jt1WoMEnV5^G|@%S(Jm;R#wVZ?xE%mz9G8!VgV@YxpK1sL3^5pY-NE5sjoa1`vfc-iilNUv=o{b$S}i1mCb5C?cUAg&7%+`{K#SWLXL zxth~Aw&U!5kK2~92R({yc`Dx|xo-66LYsn*+Z-9g21DlOP!TtjDsRwk${UMkDU3#x z-RLE#^%8pMw?PIS(FZ{r;?l}>-YtZVty2*(=OkVdzpAQL8w*L_qBU-Uw{u;@F` zV;W(^GB|dNo%NJ*c$HMR?xPv}YA4gQOt#$1xK;NpA5p=M#W$r^h7w8M2l#Otd_|9< z-!jxW&Ws}r2t~LQimO#sMF#@kHc-&)3_EVStqgjbQ!`z}B>E%W@Fm)1XlGzvCKvIP zDh8tC3qV+c&aJkIa7d78Y6y<9aExW)7>k6OLWg2w&GvMe16E(=_}YOI`#jns{}QYD zWBH(M0eyu9wlr#!X8+9Y_{T@t;xqdQy#Z&0J(nT5Xs~5So_&oDS!18+-Jq!$Dt7oZ zt8ECo;US6(T)YwX6$Dl&VgQoZvP@_fQ;0Kl)KtqF`+oKk!yV(%R_bO|?6DuR2Mjdg z>Sv2X_N$y|x@sSe(knkZhs~VWo#*x2Me;{T)<;9=qzfG_|HqDH>^_%?<*ep- zy|EnAIsn!L?9(ILeF6CXgtuUb$zmbP_Hwb4mlA>brOsksz)yNWWQe`uGD9w-k%Tu7 zwTA{Q2TK~;?7YWuJj;@gYiP2OiXE>}@eR1WH9KIQ6|rZFc>#+w(vER}ECN3tD%oH_ zOcn=SS(q2J|AW%RC<<2dNgM@&H7B9S(4PDcut7vjMme>HQrGVSGIeTAT`vdxDs}x8 zS0AbPqS`R01%YP6oTD{j>N?t)X^n&JNNn8jwYC!KT7oU@TG#Uw?9r91sVf;$*U?Ty zRJuwHt!p=!RIp65VJjE5&)BfBA8TFhYgN%-i~V3KXE$J%F+z&5?NwHa3--%8fw^WJ; z7`y38F^q$@L=4*UJe49F_b9n+cfE;1i_*o%eLzsUjnx6w7sDdFQiO^)m+TD;aUlEx zA*RrTq?+E>5aONpVXXIM-}));J{>HLg#9+6--qY`RTm`sed72K{66_qO8RuL=0*#t zDp;o;QdjRK2D%1tKaFF}wTvjlrdH8@!761fLZ0uK&@5N^!x&w$uR4?`(}O}N2$#f@*wDh(2_XCn9^{Ne|niidc$ z4IKbU9TKDt3djp73o9@U^>zXmH_fXwi4K?WB`ydun-8Tl`+cTt{|1PZb+K6Dpp4p1Rvs%>5!5JK&Ggh znIZr(McvF40g%$VL0T?PkH`(*Nuj`Z@>65p66{q%N$n1)(rj31DTQ=5g#eI3x|>1( zNFm)Jm&mH_Y~DaW`y(`i$*uzM=dx-catrS1K@I-&651ZZM>N zToJ->Y+U$ZUJmo~EPwWDX281VgI%8*8O5JEF_rWU*XeU##th=GCm0%SG_9!>lQ>z4b2a&f$$Wg zj6ha^e?tuI(Qe7EO;)!`G##>CeRmrDD^5oUy&X!Y?{?_63S|Uu!{^7EmaCmVJLJmk z3YWu6^O0s0dqNI}Fk{)MsFE#m@R<`pi`eRDXhiS?4(ukdz0go@cs@%+KFgil{Imy$ zsnkM2HfOuwU^(l3Vnk&eWA-U&hk{R&&uJ7IlTVSWgzpVXTt8dh8wiPZHBqyiIhr7d^S+pjW&ow3~_QRaXk_ZlYz>H8k_Zj{SVWuwNYEsQ*X+zA5zvzA1G~dt=7} zE;@l!r<#7o&-Em z{cn^a*fJ|-@=eD~r2(4cV*$%Kqq1CS0N@g*anNxDN*MsAaPkiC;{vx)D1dF8t^NaM zt8Z)IHr2HF3^LK!Z3|M_07wpr;IIP&bWf!-t%F^*Dm`cvWz23eCS-Y*w=p#;1=!t+ zx1VU7z6y}~q3x42-Ax)Z8&rROQhZ0}^*hpFpb~J5^J3D*c#lu5y$!H9FUHaLJr;12 zmrKsuoV?hESv$J)irsZGz-Q+IzvMjAm#!VIFi&m!2d2MeRm21jV!ee};IH3f{}dq4 zg}BYn+o%BoKIA%^0>-*#d!XFFHJln;152CT@JUV+LL1oHgg!H09;2k_&U$9RLh8H_ zE-)bKinn9=t}s@cf1=hbMD4*sN!MnVTIgUyey^qUwN9w&%{f(Sgu$JNZRgQ3oCo(0 zIl+;H9j7B;jUy)46|l>Gl+)}-y$7o(a{duIVZE-Vy9vOV3++Q z*9M#<3vV$ec~g|QrujUZAm1hhqTI>Y2`aWZ+*HEa!Vzejgse@HanZ-ixQQ|@w6&~U zGYPthwsYw}&sEx?>@Bw!5GYXE0oX;gRL3Xg*yeIo43@^#>v54bE@*J;QGmjgNB)n#(-X<#NCZwo5A%4RU`2zikXxQPEi5w@5j6IR3g5FoM-{ zgrY$y*x=mi1in?#_>=<30S&g!1mv+z$Wt`P^$pGwPT(gLjc*eqX|NS00F~!$R32#q zG(nQ4yGdg0I|u&m9o*wH=Nz;Viz-B@D0OmgXo!#w*N<_)svKVuo&V9 zNH#jba~k46e&MKmToj_^^CEJ@2KMV4a1ah^!;FBLxIS9oiu%+J-R55vDtQWXsj5)T6I5{B8`Axr3$X zwsAIZ!50F(vh^p5`Mp^*YjlR*gfsM&%lQxELoNbX<;vwKd3A6WoPLoNGC9QlPnVbx zvY3+$i$hitT^VwaPX|Meo1ZaO%)xdeUD}O-GdO-n^_UT|kwb`$B*1wvAdontJNT1y z4!B;rAyRzwe-8$}nRH__P2X)5zKw@gYL#mvZZIjd6Yne0dF;aiCrEo3GzXIxqXr`p zg`CzX3_cYsk=j6uq&<;B~3SFZ-0l+l!3Y&cvQVZJL+1^i6qqK)l4wouiaO?07+gr z(*~)MAy<_msOfCKtv{)|>tiF9pAW$HHn0x{FQ*Bx-~MD z@J2S3@Z>+zcmXA=a;mYg#YcM0%EA_r3-PXHBR(Kl1x2k?z~R8{JfLm6Ymdt@*nvgK zT%G2Y7Ga~0a#=aK40cenEHP9VUao4PO#P%%p{a(cQf;LUh80~c5Ob5zhgxyOV$R|N z3|%owQlouf)Ce=!7D84UU+ye4zN}1De#2FKfyH(xd5iHqOM8le?@eOgO(OQ)Y{a7w z8)=_JEcm<3nYqjO>TDx?;WSG`zZ6@|{!%qZ6kVHY_WDm$uN>Q?vD0W5RlVhf$5k+o zD&_=ZLr@}2yN8!O!3TpIero(C%7ekhBq_!h>l^;_w~<_e?Lcxc7Km)*jWQ*b{koBv4 zbUBDHOLe3!DiSy_%t|L)IfA@V`m6#wx!~el@HroaR=GB~0&bJ2hg683*L=*~TmU;O2s!Nv< zG>KSXtEAdc*f#}vn;mf>^h&7rQRN7uKqiB%RLBttb%8woiAlke(9+_Qz`L<+SKj1d zpNLlrW{}LJNx?b6fvRR3Sx^!U;0FnKFGm`%*F!aE@Gp6o=k_v2B|T&)bwZ02{d8vOA_rq&WHq z4zTWSvbGEWoLfAmyF=gk0pHPM=bQZ}UhZREDYDGBgS3$FaI@BfFmxJa19hUJP88Eg zjYC-NI5RR)q&cQILOwwQrpCwSpiw0WnX}WQL4Rb<5ZjM*%I|Pi2EL&kvpZeImuiX2 zExyg?S-Tln80N2)mrX1z9s9qa9>OUiOeooQDqf422a9MMe7M~XCcejZY9o6YH9))u zu>l_$T+2MfHkDkaf~Q6@&9U+#yGCwtt4jj5x}-Zak`7%l|KAN9WECqVqdR=U?nB6u zpMV--)5k<-k|mK$>IwIdc_3XUXVqfGgi~@|#HU!Pfx~NTi*bS{*4-_(ZQ=7{+X7CM z%|lGHS|N{?*|5xZE4qP80gIfN{3*bj`B7N#FOLbvUXI{zg2i%do5CKtkQE{D?+Zf2 zwvb;#u;*)CMI0hkaYsdQqT=a_;zGrT6-7z-%t|7s@|wzq^|qkW@k*kkQV!6GRi;!C z)2mFcA{JL!Rz>7hDXLG7P5bFf0=T;X*)z4NpICofjRle$NkaoaljW& zvB9JY~xAIyRmKA5GP$3 zLD&_%5Q4ViGwotwg#u*B(a@4m@mYn16+}UWB0S%%m>VuuhZjKb)$n5j}g0KC^5%Z94}*({%#Gd#-jn!{W9 zjfR6biymt~Zricq{+Poz0#{JsRCFAqJ`G2sV+#L--FVfSRXyGp*gGZGVB?kA9*%wb zBZIJQd^$_Q5|jk)|HdZ$M?u&Zo)L_f`g2eumqjj*#EBPcBSk)rOW-D@i*>Ps>RMeS z7iYGJ0vy01&sH(SkeJV7L`nT+4aD*Ws~d=^jgB=kUdO9h_g;{qUp2_VpfNo4m{G-bITJ=as29o_C|Yamb~+^_#_RsJdY_i z@q@U=W&{ha4)878F}#x$MHmvH_6DyB7Wr6AU^KQ;CugC;VDTW#ZhK%8!?D5+-?OkU zc3?(+g~K;)nk~&Hvak~YdkQW+{^vL{#$DdeC*2Xh#qK#m_cXk6uzLjnVnFSwN8A5ko zf%NkYc0C>1 z8SXK>Yx>AK4n1iVO#fL0(`i=DG7hWwGFbYz(z~VW3cr^AtMY59KV`+of>!TDmnVY52sWb2#6t|zGM>n| zaTuY-n`fhZ_s^$2_bg4x^H>rcAuDQBWpc z#ppb~EdEuEP7}()s~JXmS$K8h^2D<68itWs7GBe;w&&hq)H0&;%8Fk`VE~oGi1g4} zZyZLH_0e;$`X$e!=f+{w@!VT)97bL1qvsx_)^l&YaTrfnALTt$BaEAm{R1ikup1$U zr;_PpvxeDFPCD!IP)QTA7?;K8dCkZ0n?!w4xciW@pt_2NbDq~ zeY*MK&uP4~AO3;HU-H8zYP^#lz7lv@cHRV>{eoH!(fHS{3XRCMz z)Wuk*@z@*%)ImMuuLv%bj6YXBKTyqtf!Qd9&sX>ibuspoBlmmY)YCP|x1G6FPW%e+ zvl+jpU;HOD9`1+70x!${UCY7WF9-j7Ir!)0;4^_s`xGeya3siIp2iFItLNV8V(c!5 z&-Z5hV=7{p<`aennf#0Bc;IK^$8`MS6ACx&e;qjMMVel`fb5e$hvt)Z(l@@R9_jeC zbo{4rEz6!!;A4ZIe}0fwP{-$`toTn#xuwS!v64?|KmQqtm-h2t1a26m#~s~e{OZO% zKYu<+;-&rnfvB%`ZUpojAN?KiE5pv|eJRHcf_%$7WgvOJz6!5CXcS>A8ST3i^!LI|agnZA@`H0Mrzx$fM@rioY zu<<|GBSSqq(_zN%Ma1LpLybr4@~oQyBM-Rb-$duTX0+MFpYX-^YCa{peL%A;e}U*K zIF|7L)h`Ec3tY;r<;(wf<-~tO;vq)y79}A~3!16%&chX8+PSzKK0g7E*0k13cxc)B z$4ETHh(4uaf2iN%F|Ccmgyi4KF zsf+OraGCFVb*5RKKQQq&zUARVlaHT0H<)<7V*HHe|4`$x`sBJX8V{~mw%zKKgSV16 z+UIn|;8o4%?;1D0R6wdO58E`}S$7Cg+72RI*(Z6oiXW$@z}{KqE_Fij{!dOu^?IDX1RS|;+m=DGNE4C z_WY!r_+vHyKC8>Y_iO&)y1g~^46mZ?rsp39==`mt@#rZ^kvFtHu^KmYxjn7%6p8Bx%VlIa z_%zL@Sm(PrL-bWS@izmP_3mZg`nAWzpY_F$n*28_W&W&nimF<6zQ>h=zgQ0bM~R0R zY1*D2Y5rN|#9yN0hZm`cX1TpqPW;C@evw~!ei5z%%T=1rm)TlwSB)oUD&TdEXGmN> zST1vQ{CAcrFkXj!r*UJi0$$N(y<_5^EBtS|eR#Th+5OQ=z)|E%{oE3V{^=^nY?o$h z{GIL!pQ$NxfmedRo$NR6*kr~pR1r;mYGUe*_USZP;fFMz#u|^^qyXJK8T~bG==!V8 zVJy~ofo_LQeKu-5dZXggS@Rb)b@{C3TMom4mt}{>W_-Q&*z8AIn>Z)d_|xrPClhbr zi@zfIhZy0LRP=7@VvLixez07!%fSoE!4FCt{fHm`Dz%iIbEYZgeYK3*8t>#+&fk={ zMG<{FK*x_xQotWIzF6Z$-zuP^#xH8TK$r7w8gCe(^L>i~W@-FgjhpWmVl@7d#*1}% zxU0>&PvSa~2lO5yZls_c!=@NAQj(i$GV#M z2!(4i85tT6|4`vJoiE3LS2C&_C3<}7Qo+WLI)09BI2&rb1v+T9tH~P`uwLVTmAHPe zT!!lS(SK85zRvf38ZTL_@Hv{s5nWDtB_3kL{zXMJ{qr|8p6u5yO*V0D0JD8KFLA}g z`v0zu-&yC^M>;V6360d=j`8?z#RT21 z{H@V=?h5rhQdf+J8ZXl2=dyNmevgK?tbVYk#NiL<_AOTPpA4Mk(|jLlmd{+^(K?Rx zvP<*H8LrTEI`ELjllv>+f9>j8r@peYq2rtN?gNb{YX|vz&1bd7(`P6E3pnJ@)j;tn z>Y<)@=xpeq@lJZY&_o3r|I~Pi#`86wO%m6UESCd1e$Frjn&r7ZDkke=_;iJ*X}OnNQolYjeks@(RSrH)^Dol% zbgL%)R^zd{JnRPW&GY9)b|5-&w)K##k)Uh*S1AZBe_p40rE7{rk9k z3>?_wV|P;j)PWxxy$AN_m*nm>s9(R2L1KAwBW9{k)TEDkCMBo#8PFd}y`G%ZKP9E_ zbA6KEPZ~%i2e^CnP5HQ=F+c?v^2z)OrTKk08}2lO78lvH-e zz@(I<)UtyR;7;z*t5=`?@0A@PH7O<4-Mep(_fpCZf4A>|o*%eV`g~H3?4XGWhPQowt>5q)^iQgkB#U0bQ8YAAV`Do1dsY_Do^L=`YD+e!oFvFH@J^NdvpEFfu*QCz+HMd-v&I+TI$|M%7Pc*;@x$ z?X4lx-d34bi~HdHlW4u3sim~^wYv|EnG}D!|EEZ%-TzZO)9ya;%FM~qT9>l^Yduo? ze3!@p}IbBlm*z@K7sYVL!(ERrJ>FeBqycx9M~rr{J^9)x(mqy zH{JjV59p7&F{mfyc6q%VuLlZ1FXR92-{*Hd zP{H2l-{+&>4;(NkIpx(J$;oI!r3QTmyyu2>1`KrfOG?gX9=!$(NOnsUomB4u;HkOVBmZ19?3nBNHCRV>odS@)<&g@ zU*jpek5VQ^_9XQf_-}ajmz4(0&D!IRjg5_O-TCz{FL!hI8`QsFkL2>GWgk1h?c-6v zyOCXHc{Kki6)2LH{-%6W{9Cn?`z7^5v15Hg>ne+Fr{BNw^6#E=$G2$J8lF>9lIJD#NzpycEMa5?{!lG@9XjyTJGtMb_PNA(|s+U~YCD()Wrd%izlpjODuN{=QHZ4b)H zfc`#a>&F(!2k{!RjWMV{yk+0ua;%a&N2S%#F^Syx>j`-l^Yg{_%&v{%^H{3gn7i7Qn8`+gMfTihT8}LSd zl%MkKQGRjJZQs9X`O0j?e6o?1`sL?S%>K>W=ec|IMYU1YlPykb?`_+EqiSi>`G`X{ z%2Jb<)B`a{H<>)px@w>%)PY@`5_|NPS<9y3m@GTM=}(MW1RwKeP9;j@9GH>E$uFtG-5E%|SQ`t)-9TG!tv(nY2m`~HU36LOSC_u_zoZ@|+k zuY{DJRjNvWPv?HUK6s_iyEOE_;ZnM^S}M9vS?$#q^8nuDqjzR&>QmUtNq5<{HA`l> zx&NQim^>f_Jz`w%9=$&7lalmr()e#%AK5j!<5V0Y<>UT6;U;2ifI|14*Y_y5!k3*k zRO>!{*&D^u$w!>hZLiyn;R;>g-iZ2JtoWZ>GSx0yn}k1D#!2WerQ`WuWfHpKUTm%~ zipR)Vjp$`W9nW`5T{;%p1()wzWtVOT%>DLEsWv{;Mh&)Q7p(rQaW!D{47>l1%x)8~R zy&R7JsmXw@vUzc3k{BTy3CFTy0@e9G);`B~x2OB?2Z#_sBqTzCC`dz* zl9CFgpol2=8))bdvIIpaK!^hKd+*KLo0*&0yGvF&&(7@pX5P%aHy>}_jQ2a}!Xc2$ zS4N|Bcc1kGmArNxLrbh~O2ce!3cMl@9PF=Xhcw^x>J$)8#70|M2{+WFo?_p+qe zp^8`=b_P4`Y*ZVO>J73+fGU=Ud20!Eo@B4rB2GO2YQcj=H^WOuGm+zW8+$I+7?aqF zX0VGnX@u}3i!%AP4@Vws0OXF zyt?MSHr9v8+9FoIAbk~*pe|;d;ua*83{ydxtZ6Z|J}HCZZCJ{bGy=cbgJya51hiK` zZ-^DdwY;|r?KD<0yjw{+&FWMFKB<7TI*Al6+BZBxg-|aV%g^T7VhhRDm0tlICJ-aV z+iU~5tO94mP%nRptT(Lj?Hp}bktWFDu~t9JcD>Pny8tU}VIicPB2$b=D}*^6BY1`# zQ<~L`_8l>HL=v&9*6I$3(J>9R`U4U@E-Xf1(xw19?sjL1pxJb; z7}E)R^Oq;f%dks}an;oQ0hA*31F|xJEfvgUobJwD$&RgKiT$1Uvoxd3;I8I88-A22 z7%OnkZjlD}EUR#hUdN#@7szf?-#YmqOSe12ls9c(fw=|ynpIeFrH#W5 z8Q`%Cn=<|aJBB{?TOCgY-+EcAN)gRl7nWfumn`KKPfCVW_?DAdN0MeuV;IuifZV8l6l5RM`6t;fd9x41`3 z@6&=Auk$?S_!7R!>+j>^T=ARcMcp3^z?kh8_mXHbuj2)#+BnNCkKW6r?kdiu(L>3Q zX|CEezd(U_F5*XND_-yKQkTvt*pD&qErOQthJQa&(69G=sr!uqDE*p`x*Nd%XW{ie zFm;RbPKamIuLLykLS@tR^xiOaH=mH0n!jIuGSepfWr<&*SIg4-iIl znQTVq=?&q;ANE6e*-sxH8?X13sr%jIcI+fMWxVR%@bG%>NZrqGhw&PE%fsvaXX-wq z5smB9PiEXizt(?oPuc?kpj&gEImeeIU%_wV^*%LqFL)Vx-sG-_*L&C0UBdYf`_cB( zFgg};5QBKkMH)QAS}D~{fScluL_Ii~;<$LHTu`0hpFm`k@L^lniwNx+Z$Vg!HmnZ*41 zCpQ5feiKA0LaIa-(yiu`n{(G vxcQRg^fH_~_$mME`27^Z+!mJj_s$4@;k5Ymub=;O5&ZA#g1_kjJa6(p6qCE9 literal 0 HcmV?d00001 diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java index 95a0b0e17fd9..5c19bd630d54 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java @@ -99,8 +99,11 @@ public void testCaching(boolean securityEnabled) throws IOException { HddsProtos.ReplicationFactor.ONE, OzoneConsts.OZONE); XceiverClientSpi client1 = clientManager - .acquireClient(container1.getPipeline()); + .acquireClientForReadData(container1.getPipeline(), true); assertEquals(1, client1.getRefcount()); + // although allowShortCircuit true when calling acquireClientForReadData, + // XceiverClientGrpc client will be allocated since short-circuit is by default disabled. + assertThat(client1 instanceof XceiverClientGrpc); ContainerWithPipeline container2 = storageContainerLocationClient .allocateContainer( @@ -108,11 +111,13 @@ public void testCaching(boolean securityEnabled) throws IOException { HddsProtos.ReplicationFactor.THREE, OzoneConsts.OZONE); XceiverClientSpi client2 = clientManager - .acquireClient(container2.getPipeline()); + .acquireClientForReadData(container2.getPipeline(), true); assertEquals(1, client2.getRefcount()); + assertThat(client2 instanceof XceiverClientGrpc); XceiverClientSpi client3 = clientManager - .acquireClient(container1.getPipeline()); + .acquireClientForReadData(container1.getPipeline(), true); + assertThat(client3 instanceof XceiverClientGrpc); assertEquals(2, client3.getRefcount()); assertEquals(2, client1.getRefcount()); assertEquals(client1, client3); @@ -270,7 +275,7 @@ public void testFreeByRetryFailure() throws IOException { clientManager.releaseClient(client1, true); assertEquals(0, client1.getRefcount()); assertNotNull(cache.getIfPresent( - container1.getContainerInfo().getPipelineID().getId().toString() + container1.getContainerInfo().getPipelineID().getId().toString() + "-" + container1.getContainerInfo().getReplicationType())); // cleanup diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManagerSC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManagerSC.java new file mode 100644 index 000000000000..24ab521d8f82 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManagerSC.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.scm; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.XceiverClientManager.ScmClientConfig; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.container.common.SCMTestUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test for short-circuit enabled XceiverClientManager. + */ +@Timeout(300) +public class TestXceiverClientManagerSC { + + private static OzoneConfiguration config; + private static MiniOzoneCluster cluster; + private static StorageContainerLocationProtocolClientSideTranslatorPB + storageContainerLocationClient; + @TempDir + private static File dir; + + @BeforeAll + public static void init() throws Exception { + config = new OzoneConfiguration(); + OzoneClientConfig clientConfig = config.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + config.setFromObject(clientConfig); + config.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + cluster = MiniOzoneCluster.newBuilder(config) + .setNumDatanodes(3) + .build(); + cluster.waitForClusterToBeReady(); + storageContainerLocationClient = cluster + .getStorageContainerLocationClient(); + } + + @AfterAll + public static void shutdown() { + if (cluster != null) { + cluster.shutdown(); + } + IOUtils.cleanupWithLogger(null, storageContainerLocationClient); + } + + @Test + public void testAllocateShortCircuitClient() throws IOException { + try (XceiverClientManager clientManager = new XceiverClientManager(config, + config.getObject(ScmClientConfig.class), null)) { + + ContainerWithPipeline container1 = storageContainerLocationClient + .allocateContainer( + SCMTestUtils.getReplicationType(config), + HddsProtos.ReplicationFactor.THREE, + OzoneConsts.OZONE); + XceiverClientSpi client1 = clientManager.acquireClientForReadData(container1.getPipeline(), true); + assertEquals(1, client1.getRefcount()); + assertTrue(client1 instanceof XceiverClientShortCircuit); + XceiverClientSpi client2 = clientManager.acquireClientForReadData(container1.getPipeline(), true); + assertTrue(client2 instanceof XceiverClientShortCircuit); + assertEquals(2, client2.getRefcount()); + assertEquals(2, client1.getRefcount()); + assertEquals(client1, client2); + clientManager.releaseClient(client1, true); + clientManager.releaseClient(client2, true); + // client is still kept in the cache, for create a domain socket connection is expensive. + assertEquals(1, clientManager.getClientCache().size()); + } + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverServerDomainSocket.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverServerDomainSocket.java new file mode 100644 index 000000000000..684314f2f1ce --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverServerDomainSocket.java @@ -0,0 +1,712 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES 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.hadoop.hdds.scm; + +import com.google.common.collect.Maps; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChunkBuffer; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.common.ContainerTestUtils; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.impl.ContainerSet; +import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; +import org.apache.hadoop.ozone.container.common.interfaces.Handler; +import org.apache.hadoop.ozone.container.common.statemachine.StateContext; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerDomainSocket; +import org.apache.hadoop.ozone.container.common.volume.HddsVolume; +import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet; +import org.apache.hadoop.ozone.container.common.volume.StorageVolume; +import org.apache.hadoop.ozone.container.common.volume.VolumeSet; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type.GetBlock; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type.ReadChunk; +import static org.apache.hadoop.hdds.scm.XceiverClientShortCircuit.vintPrefixed; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests the XceiverServerDomainSocket class. + */ +public class TestXceiverServerDomainSocket { + InetSocketAddress localhost = InetSocketAddress.createUnresolved("localhost", 10000); + @TempDir + private File dir; + + private final ThreadPoolExecutor readExecutors = new ThreadPoolExecutor(1, 1, + 60, TimeUnit.SECONDS, + new LinkedBlockingQueue<>()); + + private static OzoneConfiguration conf; + private static ContainerMetrics metrics; + private static int readTimeout; + private static int writeTimeout; + + @BeforeAll + public static void setup() { + // enable short-circuit read + conf = new OzoneConfiguration(); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + clientConfig.setShortCircuitReadDisableInterval(1000); + conf.setFromObject(clientConfig); + metrics = ContainerMetrics.create(conf); + readTimeout = 5 * 1000; + writeTimeout = 5 * 1000; + } + + @Test + @Timeout(30) + public void testDomainPathConfiguration() { + // empty domain path + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, ""); + try { + DomainSocketFactory.getInstance(conf); + fail("Domain path is empty."); + } catch (Throwable e) { + assertTrue(e instanceof IllegalArgumentException); + assertTrue(e.getMessage().contains("ozone.domain.socket.path is not set")); + } + + // Domain path too long + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-datanode-socket-" + System.nanoTime()).getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("Domain path is too long."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof SocketException); + assertTrue(e.getMessage().contains("path too long")); + } finally { + factory.close(); + } + + // non-existing domain parent path + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir.getAbsolutePath() + System.nanoTime(), "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("non-existing domain parent path."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("failed to stat a path component")); + } finally { + factory.close(); + } + + // an existing domain path, the existing file is override and changed from a normal file to a socket file + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + File file = new File(dir, "ozone-socket"); + assertTrue(file.createNewFile()); + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + } catch (Throwable e) { + fail("an existing domain path is supported by not recommended."); + } finally { + factory.close(); + } + } + + @Test + @Timeout(30) + public void testDomainPathPermission() { + // write from everyone is not allowed + assertTrue(dir.setWritable(true, false)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("write from everyone is not allowed."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("It is not protected because it is world-writable")); + } finally { + factory.close(); + } + + // write from owner is required + assertTrue(dir.setWritable(false, false)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("write from owner is required."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("Permission denied")); + } finally { + factory.close(); + } + + // write from owner is required + assertTrue(dir.setWritable(true, true)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = null; + try { + server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + } catch (Throwable e) { + fail("write from owner is required."); + } finally { + factory.close(); + if (server != null) { + server.stop(); + } + } + + // execute from owner is required + assertTrue(dir.setExecutable(false, true)); + assertTrue(dir.setWritable(true, true)); + assertTrue(dir.setReadable(true, true)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("execute from owner is required."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("Permission denied")); + } finally { + factory.close(); + dir.setExecutable(true, true); + } + + // read from owner is not required? + assertTrue(dir.setExecutable(true, true)); + assertTrue(dir.setWritable(true, true)); + assertTrue(dir.setReadable(false, true)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + } catch (Throwable e) { + fail("read from owner is not required."); + } finally { + factory.close(); + dir.setReadable(true, true); + if (server != null) { + server.stop(); + } + } + } + + /** + * Test a successful connection and then read/write. + */ + @Test + public void testReadWrite1() throws IOException { + testReadWrite(false, false); + } + + /** + * On Linux, when there is still open file handle of a deleted file, the file handle remains open and can still + * be used to read and write the file. + */ + @Test + @Timeout(30) + public void testReadWrite2() throws IOException { + testReadWrite(true, false); + } + + @Test + @Timeout(30) + public void testReadWrite3() throws IOException { + testReadWrite(false, true); + } + + private void testReadWrite(boolean deleteFileBeforeRead, boolean deleteFileDuringRead) throws IOException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + ContainerMetrics containerMetrics = ContainerMetrics.create(conf); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, containerMetrics, factory); + DomainSocket sock = null; + try { + File volume = new File(dir, "dn-volume"); + server.setContainerDispatcher(createDispatcherAndPrepareData(conf, volume, server, containerMetrics)); + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + assertTrue(sock.isOpen()); + + // send request + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(GetBlock.getNumber()); + getBlockRequest().writeDelimitedTo(outputStream); + outputStream.flush(); + + // receive response + final DataInputStream inputStream = new DataInputStream(sock.getInputStream()); + short ret = inputStream.readShort(); + assertEquals(OzoneClientConfig.DATA_TRANSFER_VERSION, ret); + ret = inputStream.readShort(); + assertEquals(ContainerProtos.Type.GetBlock.getNumber(), ret); + ContainerProtos.ContainerCommandResponseProto responseProto = + ContainerProtos.ContainerCommandResponseProto.parseFrom(vintPrefixed(inputStream)); + + assertEquals(ContainerProtos.Type.GetBlock.getNumber(), responseProto.getCmdType().getNumber()); + ContainerProtos.GetBlockResponseProto getBlockResponseProto = responseProto.getGetBlock(); + assertEquals(ContainerProtos.Result.SUCCESS, responseProto.getResult()); + assertTrue(getBlockResponseProto.getShortCircuitAccessGranted()); + + // read FSD from domainSocket + FileInputStream[] fis = new FileInputStream[1]; + byte buf[] = new byte[1]; + sock.recvFileInputStreams(fis, buf, 0, buf.length); + assertNotNull(fis[0]); + + if (deleteFileBeforeRead) { + FileUtils.deleteDirectory(volume); + } + // read file content + FileChannel dataIn = fis[0].getChannel(); + int chunkSize = 1024 * 1024; + dataIn.position(0); + ByteBuffer dataBuf = ByteBuffer.allocate(chunkSize / 2); + // a closed socket doesn't impact file stream + sock.close(); + int readSize = dataIn.read(dataBuf); + assertEquals(chunkSize / 2, readSize); + if (deleteFileDuringRead) { + FileUtils.deleteDirectory(volume); + } + dataBuf.flip(); + readSize = dataIn.read(dataBuf); + assertEquals(chunkSize / 2, readSize); + dataBuf.flip(); + readSize = dataIn.read(dataBuf); + assertEquals(-1, readSize); + + // check metrics + assertEquals(1, containerMetrics.getContainerLocalOpsMetrics(ContainerProtos.Type.GetBlock)); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + /** + * Test server is not listening. + */ + @Test + @Timeout(30) + public void testServerNotListening() { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + DomainSocket sock = null; + try { + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + } catch (IOException e) { + assertTrue(e instanceof ConnectException); + assertTrue(e.getMessage().contains("connect(2) error: No such file or directory")); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + } + } + + /** + * Test server is not started to accept new connection. + * Although socket can be created, read will fail, write can succeed. + */ + @Test + @Timeout(30) + public void testServerNotStart() { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + try { + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + assertTrue(sock.isOpen()); + // send request + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(GetBlock.getNumber()); + getBlockRequest().writeDelimitedTo(outputStream); + outputStream.flush(); + + final DataInputStream inputStream = new DataInputStream(sock.getInputStream()); + inputStream.readShort(); + } catch (IOException e) { + assertTrue(e instanceof SocketTimeoutException); + assertTrue(e.getMessage().contains("read(2) error: Resource temporarily unavailable")); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + @Test + @Timeout(30) + public void testReadTimeout() throws InterruptedException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + conf.set(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT, "5s"); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + try { + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + assertTrue(sock.isOpen()); + + // server will close the DomainSocket if there is no message from client in OZONE_CLIENT_READ_TIMEOUT + Thread.sleep(5 * 1000); + // send request + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(GetBlock.getNumber()); + getBlockRequest().writeDelimitedTo(outputStream); + outputStream.flush(); + } catch (IOException e) { + assertTrue(e instanceof SocketException); + assertTrue(e.getMessage().contains("write(2) error: Broken pipe")); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + @Test + @Timeout(30) + public void testMaxXceiverCount() throws IOException, InterruptedException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + List list = new ArrayList<>(); + GenericTestUtils.LogCapturer logCapturer = + GenericTestUtils.LogCapturer.captureLogs(XceiverServerDomainSocket.LOG); + try { + server.start(); + // test max allowed xceiver count(default 10 * 5) + int count = 51; + for (int i = 1; i <= count; i++) { + DomainSocket sock = factory.createSocket(readTimeout, writeTimeout, localhost); + list.add(sock); + } + + Thread.sleep(5000); + logCapturer.getOutput().contains("Xceiver count exceeds the limit" + (count - 1)); + DomainSocket lastSock = list.get(list.size() - 1); + // although remote peer is already closed due to limit exhausted, sock.isOpen() is still true. + // Only when client read/write socket stream, there will be exception or -1 returned. + assertTrue(lastSock.isOpen()); + + // write to first 50 sockets should be OK + for (int i = 0; i < count - 2; i++) { + DomainSocket sock = list.get(i); + assertTrue(sock.isOpen()); + sock.getOutputStream().write(1); + sock.getOutputStream().flush(); + sock.close(); + assertFalse(sock.isOpen()); + } + + // write the last socket should fail + try { + lastSock.getOutputStream().write(1); + lastSock.getOutputStream().flush(); + fail("Write to a peer closed socket should fail"); + } catch (Exception e) { + assertTrue(e instanceof SocketException); + assertTrue(e.getMessage().contains("write(2) error: Broken pipe")); + } + + // read a broken pipe will return -1 + int data = lastSock.getInputStream().read(); + assertEquals(-1, data); + lastSock.close(); + assertFalse(lastSock.isOpen()); + } finally { + factory.close(); + server.stop(); + } + } + + /** + * When server receives any message which doesn't follow the version, request type, request body sequence, server + * will treat it as a critical error, close the connection. + */ + @Test + @Timeout(30) + public void testSendIrrelevantMessage() { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + String data = "hello world"; + try { + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.write(data.getBytes()); + outputStream.flush(); + sock.getInputStream().read(); + } catch (IOException e) { + assertTrue(e instanceof EOFException); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + @Test + @Timeout(30) + public void testSendUnsupportedRequest() throws IOException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + try { + File volume = new File(dir, "dn-volume"); + server.setContainerDispatcher(createDispatcherAndPrepareData(conf, volume, server, metrics)); + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(ReadChunk.getNumber()); + ContainerTestHelper.getDummyCommandRequestProto(ReadChunk).writeDelimitedTo(outputStream); + outputStream.flush(); + + // receive response + final DataInputStream inputStream = new DataInputStream(sock.getInputStream()); + short ret = inputStream.readShort(); + assertEquals(OzoneClientConfig.DATA_TRANSFER_VERSION, ret); + ret = inputStream.readShort(); + assertEquals(ContainerProtos.Type.ReadChunk.getNumber(), ret); + ContainerProtos.ContainerCommandResponseProto responseProto = + ContainerProtos.ContainerCommandResponseProto.parseFrom(vintPrefixed(inputStream)); + assertTrue(responseProto.getResult() == ContainerProtos.Result.UNSUPPORTED_REQUEST); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + private ContainerProtos.ContainerCommandRequestProto getBlockRequest() { + long value = 1; + String datanodeUUID = UUID.randomUUID().toString(); + ContainerProtos.GetBlockRequestProto.Builder getBlock = + ContainerProtos.GetBlockRequestProto.newBuilder() + .setBlockID(new BlockID(value, value).getDatanodeBlockIDProtobuf()) + .setRequestShortCircuitAccess(true); + return ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(GetBlock) + .setContainerID(value) + .setGetBlock(getBlock) + .setDatanodeUuid(datanodeUUID) + .build(); + } + + private ContainerProtos.ContainerCommandRequestProto getReadChunkRequest() { + long value = 1; + String datanodeUUID = UUID.randomUUID().toString(); + ContainerProtos.GetBlockRequestProto.Builder getBlock = + ContainerProtos.GetBlockRequestProto.newBuilder() + .setBlockID(new BlockID(value, value).getDatanodeBlockIDProtobuf()) + .setRequestShortCircuitAccess(true); + return ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(GetBlock) + .setContainerID(value) + .setGetBlock(getBlock) + .setDatanodeUuid(datanodeUUID) + .build(); + } + + private ContainerDispatcher createDispatcherAndPrepareData(OzoneConfiguration conf, File volume, + XceiverServerDomainSocket domainSocketServer, ContainerMetrics metrics) throws IOException { + DatanodeDetails datanodeDetails = randomDatanodeDetails(); + conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, volume.getAbsolutePath()); + conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, volume.getAbsolutePath()); + VolumeSet volumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf, + null, StorageVolume.VolumeType.DATA_VOLUME, null); + String cID = UUID.randomUUID().toString(); + HddsVolume dataVolume = (HddsVolume) volumeSet.getVolumesList().get(0); + dataVolume.format(cID); + dataVolume.setDbParentDir(volume); + assertTrue(dataVolume.getDbParentDir() != null); + ContainerSet containerSet = new ContainerSet(1000); + + // create HddsDispatcher + StateContext context = ContainerTestUtils.getMockContext(datanodeDetails, conf); + Map handlers = Maps.newHashMap(); + OzoneContainer ozoneContainer = mock(OzoneContainer.class); + when(ozoneContainer.getReadDomainSocketChannel()).thenReturn(domainSocketServer); + for (ContainerProtos.ContainerType containerType : + ContainerProtos.ContainerType.values()) { + handlers.put(containerType, + Handler.getHandlerForContainerType(containerType, conf, + context.getParent().getDatanodeDetails().getUuidString(), + containerSet, volumeSet, metrics, + c -> { }, ozoneContainer)); + } + HddsDispatcher dispatcher = new HddsDispatcher(conf, containerSet, volumeSet, handlers, context, metrics, null); + dispatcher.setClusterId(cID); + // create container + long value = 1L; + String pipelineID = UUID.randomUUID().toString(); + final ContainerProtos.ContainerCommandRequestProto createContainer = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.CreateContainer) + .setDatanodeUuid(datanodeDetails.getUuidString()).setCreateContainer( + ContainerProtos.CreateContainerRequestProto.newBuilder() + .setContainerType(ContainerProtos.ContainerType.KeyValueContainer).build()) + .setContainerID(value).setPipelineID(pipelineID) + .build(); + dispatcher.dispatch(createContainer, null); + + // write chunk + long id = 1; + int chunkSize = 1024 * 1024; + byte[] rawData = RandomStringUtils.randomAscii(chunkSize).getBytes(StandardCharsets.UTF_8); + Checksum checksum = new Checksum(ContainerProtos.ChecksumType.CRC32, chunkSize); + ContainerProtos.ChecksumData checksumProtobuf = checksum.computeChecksum(rawData).getProtoBufMessage(); + ContainerProtos.DatanodeBlockID blockId = ContainerProtos.DatanodeBlockID.newBuilder() + .setContainerID(id).setLocalID(id).setBlockCommitSequenceId(id).build(); + ContainerProtos.BlockData.Builder blockData = ContainerProtos.BlockData.newBuilder().setBlockID(blockId); + ContainerProtos.ChunkInfo.Builder chunkInfo = ContainerProtos.ChunkInfo.newBuilder() + .setChunkName("chunk_" + value).setOffset(0).setLen(chunkSize).setChecksumData(checksumProtobuf); + blockData.addChunks(chunkInfo); + Pipeline pipeline = MockPipeline.createSingleNodePipeline(); + ContainerProtos.WriteChunkRequestProto.Builder writeChunk = + ContainerProtos.WriteChunkRequestProto.newBuilder() + .setBlockID(blockId).setChunkData(chunkInfo) + .setData(ChunkBuffer.wrap(ByteBuffer.wrap(rawData)).toByteString()); + + ContainerProtos.ContainerCommandRequestProto writeChunkRequest = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.WriteChunk) + .setContainerID(blockId.getContainerID()) + .setWriteChunk(writeChunk) + .setDatanodeUuid(pipeline.getFirstNode().getUuidString()).build(); + dispatcher.dispatch(writeChunkRequest, null); + + ContainerProtos.PutBlockRequestProto.Builder putBlock = ContainerProtos.PutBlockRequestProto + .newBuilder().setBlockData(blockData); + ContainerProtos.ContainerCommandRequestProto putBlockRequest = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.PutBlock) + .setContainerID(blockId.getContainerID()) + .setDatanodeUuid(datanodeDetails.getUuidString()) + .setPutBlock(putBlock) + .build(); + + dispatcher.dispatch(putBlockRequest, null); + return dispatcher; + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java index ff55ee83c176..6c690f987e27 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java @@ -306,7 +306,7 @@ abstract class Builder { protected boolean includeS3G = false; protected int dnInitialVersion = DatanodeVersion.FUTURE_VERSION.toProtoValue(); - protected int dnCurrentVersion = DatanodeVersion.COMBINED_PUTBLOCK_WRITECHUNK_RPC.toProtoValue(); + protected int dnCurrentVersion = DatanodeVersion.SHORT_CIRCUIT_READS.toProtoValue(); protected int numOfDatanodes = 3; protected boolean startDataNodes = true; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java index c5301ba4194f..0c323b61bb5d 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java @@ -62,10 +62,10 @@ void testAll(ContainerLayoutVersion layout) throws Exception { * Test to verify that data read from chunks is stored in a list of buffers * with max capacity equal to the bytes per checksum. */ - private void testChunkReadBuffers(TestBucket bucket) throws Exception { + protected void testChunkReadBuffers(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); - byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); + byte[] inputData = bucket.writeRandomBytes(keyName, getRepConfig(), dataLength); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { @@ -124,9 +124,9 @@ private void testChunkReadBuffers(TestBucket bucket) throws Exception { } } - private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { + protected void testCloseReleasesBuffers(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); - bucket.writeRandomBytes(keyName, CHUNK_SIZE); + bucket.writeRandomBytes(keyName, getRepConfig(), CHUNK_SIZE); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { BlockInputStream block0Stream = @@ -147,9 +147,9 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { * Test that ChunkInputStream buffers are released as soon as the last byte * of the buffer is read. */ - private void testBufferRelease(TestBucket bucket) throws Exception { + protected void testBufferRelease(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); - byte[] inputData = bucket.writeRandomBytes(keyName, CHUNK_SIZE); + byte[] inputData = bucket.writeRandomBytes(keyName, getRepConfig(), CHUNK_SIZE); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { @@ -205,7 +205,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { } } - private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, + protected byte[] readDataFromChunk(ChunkInputStream chunkInputStream, int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; chunkInputStream.seek(offset); @@ -229,7 +229,7 @@ private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, * @param expectedBufferCapacity expected buffer capacity of unreleased * buffers */ - private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, + protected void checkBufferSizeAndCapacity(ByteBuffer[] buffers, int expectedNumBuffers, int numReleasedBuffers, long expectedBufferCapacity) { assertEquals(expectedNumBuffers, buffers.length, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestDomainSocketFactory.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestDomainSocketFactory.java new file mode 100644 index 000000000000..16cf1eb67d40 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestDomainSocketFactory.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.hadoop.ozone.client.rpc.read; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.net.InetSocketAddress; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests for {@link DomainSocketFactory}'s functionality. + */ +public class TestDomainSocketFactory { + + InetSocketAddress localhost = InetSocketAddress.createUnresolved("localhost", 10000); + @TempDir + private File dir; + + private DomainSocketFactory getDomainSocketFactory() { + // enable short-circuit read + OzoneConfiguration conf = new OzoneConfiguration(); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + clientConfig.setShortCircuitReadDisableInterval(1); + conf.setFromObject(clientConfig); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + + // create DomainSocketFactory + DomainSocketFactory domainSocketFactory = DomainSocketFactory.getInstance(conf); + assertTrue(domainSocketFactory.isServiceEnabled()); + assertTrue(domainSocketFactory.isServiceReady()); + return domainSocketFactory; + } + + @Test + public void testShortCircuitDisableTemporary() { + DomainSocketFactory factory = getDomainSocketFactory(); + try { + // temporary disable short-circuit read + long pathExpireDuration = factory.getPathExpireMills(); + factory.disableShortCircuit(); + DomainSocketFactory.PathInfo pathInfo = factory.getPathInfo(localhost); + assertEquals(DomainSocketFactory.PathState.DISABLED, pathInfo.getPathState()); + try { + Thread.sleep(pathExpireDuration + 100); + } catch (InterruptedException e) { + } + pathInfo = factory.getPathInfo(localhost); + assertEquals(DomainSocketFactory.PathState.VALID, pathInfo.getPathState()); + } finally { + factory.close(); + } + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java index 256148dfb8de..65490a678af1 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java @@ -20,6 +20,8 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -30,6 +32,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; @@ -42,7 +45,7 @@ abstract class TestInputStreamBase { static final int BLOCK_SIZE = 2 * MAX_FLUSH_SIZE; // 8MB static final int BYTES_PER_CHECKSUM = 256 * 1024; // 256KB - protected static MiniOzoneCluster newCluster( + protected MiniOzoneCluster newCluster( ContainerLayoutVersion containerLayout) throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); @@ -64,6 +67,7 @@ protected static MiniOzoneCluster newCluster( conf.getObject(ReplicationManagerConfiguration.class); repConf.setInterval(Duration.ofSeconds(1)); conf.setFromObject(repConf); + setCustomizedProperties(conf); ClientConfigForTesting.newBuilder(StorageUnit.BYTES) .setBlockSize(BLOCK_SIZE) @@ -73,12 +77,22 @@ protected static MiniOzoneCluster newCluster( .applyTo(conf); return MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(5) + .setNumDatanodes(getDatanodeCount()) .build(); } - static String getNewKeyName() { + String getNewKeyName() { return UUID.randomUUID().toString(); } + int getDatanodeCount() { + return 5; + } + + void setCustomizedProperties(OzoneConfiguration configuration) { + } + + ReplicationConfig getRepConfig() { + return RatisReplicationConfig.getInstance(THREE); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java new file mode 100644 index 000000000000..e8f2129d5989 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java @@ -0,0 +1,160 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES 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.hadoop.ozone.client.rpc.read; + +import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientGrpc; +import org.apache.hadoop.hdds.scm.XceiverClientShortCircuit; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.hdds.scm.storage.ShortCircuitChunkInputStream; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.io.KeyInputStream; +import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; +import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; +import org.apache.hadoop.ozone.om.TestBucket; +import org.apache.ozone.test.GenericTestUtils; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.event.Level; + +import java.io.File; +import java.io.IOException; + +/** + * Tests {@link ShortCircuitChunkInputStream}. + */ +public class TestShortCircuitChunkInputStream extends TestChunkInputStream { + + @TempDir + private File dir; + + @Override + int getDatanodeCount() { + return 1; + } + + @Override + void setCustomizedProperties(OzoneConfiguration configuration) { + OzoneClientConfig clientConfig = configuration.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + configuration.setFromObject(clientConfig); + configuration.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + GenericTestUtils.setLogLevel(XceiverClientShortCircuit.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(XceiverClientGrpc.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(ShortCircuitChunkInputStream.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(BlockInputStream.LOG, Level.DEBUG); + } + + @Override + ReplicationConfig getRepConfig() { + return RatisReplicationConfig.getInstance(ONE); + } + + + /** + * Run the tests as a single test method to avoid needing a new mini-cluster + * for each test. + */ + @ContainerLayoutTestInfo.ContainerTest + @Override + void testAll(ContainerLayoutVersion layout) throws Exception { + try (MiniOzoneCluster cluster = newCluster(layout)) { + cluster.waitForClusterToBeReady(); + assumeTrue(DomainSocketFactory.getInstance(cluster.getConf()).isServiceReady()); + + try (OzoneClient client = cluster.newClient()) { + TestBucket bucket = TestBucket.newBuilder(client).build(); + GenericTestUtils.LogCapturer logCapturer1 = + GenericTestUtils.LogCapturer.captureLogs(ShortCircuitChunkInputStream.LOG); + GenericTestUtils.LogCapturer logCapturer2 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientShortCircuit.LOG); + GenericTestUtils.LogCapturer logCapturer3 = + GenericTestUtils.LogCapturer.captureLogs(BlockInputStream.LOG); + GenericTestUtils.LogCapturer logCapturer4 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientGrpc.LOG); + testChunkReadBuffers(bucket); + testBufferRelease(bucket); + testCloseReleasesBuffers(bucket); + assertTrue(logCapturer1.getOutput().contains("ShortCircuitChunkInputStream is created")); + assertTrue(logCapturer2.getOutput().contains("XceiverClientShortCircuit is created")); + assertTrue((logCapturer3.getOutput().contains("Get the FileInputStream of block"))); + assertFalse(logCapturer4.getOutput().contains("XceiverClientGrpc is created")); + } + } + } + + @ContainerLayoutTestInfo.ContainerTest + void testFallbackToGrpc(ContainerLayoutVersion layout) throws Exception { + try (MiniOzoneCluster cluster = newCluster(layout)) { + cluster.waitForClusterToBeReady(); + assumeTrue(DomainSocketFactory.getInstance(cluster.getConf()).isServiceReady()); + + try (OzoneClient client = cluster.newClient()) { + TestBucket bucket = TestBucket.newBuilder(client).build(); + GenericTestUtils.LogCapturer logCapturer1 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientShortCircuit.LOG); + GenericTestUtils.LogCapturer logCapturer2 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientGrpc.LOG); + + // create key + String keyName = getNewKeyName(); + int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); + byte[] inputData = bucket.writeRandomBytes(keyName, getRepConfig(), dataLength); + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + BlockInputStream block0Stream = + (BlockInputStream)keyInputStream.getPartStreams().get(0); + block0Stream.initialize(); + assertNotNull(block0Stream.getBlockInputStream()); + assertTrue(logCapturer1.getOutput().contains("XceiverClientShortCircuit is created")); + // stop XceiverServerDomainSocket + XceiverServerSpi server = cluster.getHddsDatanodes().get(0) + .getDatanodeStateMachine().getContainer().getReadDomainSocketChannel(); + server.stop(); + BlockInputStream block1Stream = (BlockInputStream)keyInputStream.getPartStreams().get(1); + try { + block1Stream.initialize(); + } catch (IOException e) { + assertTrue(e.getMessage().contains("DomainSocket stream is not open")); + assertTrue(logCapturer1.getOutput().contains("ReceiveResponseTask is closed due to java.io.EOFException")); + } + // read whole key + byte[] data = new byte[dataLength]; + int readLen = keyInputStream.read(data); + assertEquals(dataLength, readLen); + assertArrayEquals(inputData, data); + assertTrue(logCapturer2.getOutput().contains("XceiverClientGrpc is created")); + } + } + } + } +}