From 474fc989613d1467fd11b30190ef2bef70568bdf Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Wed, 9 Oct 2024 22:44:29 +0800 Subject: [PATCH 1/4] PCIP-1: Distributed RPC framework implemented by the Pulsar client --- pcip/TEMPLATE.md | 142 +++++++++++++++ pcip/pcip-1.md | 464 +++++++++++++++++++++++++++++++++++++++++++++++ pom.xml | 1 + 3 files changed, 607 insertions(+) create mode 100644 pcip/TEMPLATE.md create mode 100644 pcip/pcip-1.md diff --git a/pcip/TEMPLATE.md b/pcip/TEMPLATE.md new file mode 100644 index 0000000..d21753c --- /dev/null +++ b/pcip/TEMPLATE.md @@ -0,0 +1,142 @@ + + +# PCIP-XXX: Proposal title + +# Background knowledge + + + +# Motivation + + + +# Goals + +## In Scope + + + +## Out of Scope + + + + +# High Level Design + + + +# Detailed Design + +## Design & Implementation Details + + + +## Public-facing Changes + + + +### Public API + + +### Binary protocol + +### Configuration + +### CLI + +### Metrics + + + + +# Monitoring + + + +# Alternatives + + + +# General Notes + +# Links + + +* Mailing List discussion thread: +* Mailing List voting thread: diff --git a/pcip/pcip-1.md b/pcip/pcip-1.md new file mode 100644 index 0000000..c7e0bb7 --- /dev/null +++ b/pcip/pcip-1.md @@ -0,0 +1,464 @@ +# PCIP-1: Distributed RPC framework implemented by the Pulsar client + +# Background knowledge + +## Request-Reply Synchronize Model + +In this model, the client sends a request and waits for a response from the server. The server receives the request, processes it, and sends back a response. This interaction pattern is fundamental in client-server communication and is crucial for synchronous operations where the client needs a response before proceeding. + +## Remote Procedure Call (RPC) + +RPC allows functions or procedures to be executed on a different machine from the client making the call, as if they were local. This method abstracts the complexities of network communication, allowing developers to focus on the business logic rather than the underlying network details. + +The implementation of RPC is usually based on the Request-Reply model. In this case: + +- The RPC client plays the role of the requester, calling a remote procedure as if it were sending a request message. +- The RPC server plays the role of the responder, receiving the request, executing the procedure, and returning the result as a reply message. + +## Current behavior of sending messages in Pulsar + +The current sending behavior of Pulsar is when the message is successfully published, that is, successfully persisted to the storage layer. The MessageId assigned to the published message by the broker is returned. + +## Analogies of message flow in RPC and Pulsar + +- In Pulsar, producer is equivalent to RPC Client. +- The RPC Client initiates a request like sending a message producer. +- The RPC Server side receives this request as if the consumer receives the message and then carries out customized processing, and finally ACKs the message. +- If this ACK request contains the result returned by the "server side" and is sent to the original producer(RPC Client). +- After receiving the results returned by the consumer(RPC Server), the producer(RPC Client) directly returns the content of the results. + +# Motivation + +As we known,Pulsar's current asynchronous publish-subscribe model serves well for decoupled message distribution, but it lacks a native mechanism for handling synchronous interactions typical of Remote Procedure Calls (RPC). +This request-reply model can greatly enhance the utility of Pulsar. We can then use Pulsar as a distributed RPC Framework. This also improves the fault tolerance of RPC calls. + +PIP-371 is also committed to building such a model. + +(https://github.com/apache/pulsar/pull/23143 and https://github.com/apache/pulsar/pull/23194) + +But we need to implement this distributed RPC framework in a way that does not intrude into the pulsar core library. +Therefore, we need to use two topics, one is the request topic and the other is the reply topic. The client side sends RPC requests to the request topic, the server side receives request message and performs customized processing, and finally sends them to the reply topic. The client receives the reply message and returns. + +### Why would we use Pulsar for this RPC call? + +Implement RPC using Apache Pulsar. Requests can be sent through a client, received by one or more servers and processed in parallel. Finally, the server returns all processing results after processing, and the client can perform summary and other operations after receiving them. +This proposal to achieve the function is request. Request and existing send function of pulsar can be mixed to same topic. This means that the user can choose, and the call to the server side (consumer) can be asynchronous or synchronous, which is controlled by the user flexibly. +You can directly use Pulsar's own delaying messages, that is, you can execute RPC regularly. +You can directly use Pulsar's own load balancing mechanism. +You can directly use Pulsar's own message consumption throttling mechanism. +You can directly use Pulsar's own expansion and contraction mechanism. +You can directly use Pulsar's own message call tracking, monitoring, and logging mechanisms. + +# Goals + +## In Scope + +- To implement a pulsar-rpc-client similar to pulsar-client, we can encapsulate request request as a `request message` + and send it to `request topic`. This is a pulsar topic dedicated to receiving requests. +- Implement a pulsar-rpc-server. Internally, the pulsar consumer is used to monitor and receive the message in the + `request topic`, and custom logic processing is supported. Finally, the processed results (including possible + error messages) are encapsulated as `reply message` sent to `reply topic`. This is a pulsar topic dedicated to + storing request processed results. + +## Out of Scope + +None + +# High Level Design + +## Implementing RPC in Pulsar RPC Framework + +1. **pulsar-rpc-client side** + - **Sending request message to request-topic**: The `pulsar-rpc-client` publishes a message to a `request-topic`. + This message includes all necessary data for the `pulsar-rpc-server` to process the request. + - **Waiting for reply message**: The `pulsar-rpc-client` listens to reply topic. And return to the user for + processing, the user can even decide how many reply messages a request has to process successfully. + +2. **pulsar-rpc-server side** + - **Receive and process the request message**: The server side receives the message and performs custom processing. + - **Send reply message to reply-topic**: Send reply message(it could also be an error message) to reply-topic. + +### Handling Timeouts + +- The timeout period is set through the pulsar-rpc-client api, which can set a different timeout period for each + message. If the timeout period is exceeded, a TimeoutException is returned. And can be returned to the user for + the corresponding custom processing. +- There are two cases of timeout. One is that the request task has timed out after the message is normally send + to the `request topic`. This situation will allow the consumer to continue processing, but the `pulsar-rpc-client` + will be directly returned to the application TimeoutException. When processing each message, the `pulsar-rpc-server` + side will obtain the time of initiating the request (or use receivedMsg.getPublishTime()) plus the timeout in the + message properties. If it is less than or equal to the current time, then do not process and just ACK this message. +- The other is to time out when doing application customization processing on the consumer side. If, after processing, + it is checked whether the current is greater than the time to initiate the request plus the timeout in the message + properties. If it is greater than then just ACK message, although it has timed out, the message is "discarded", and + more importantly, the `pulsar-rpc-server` needs to execution rollback logic(This is defined by the user when they + first create the `pulsar-rpc-server`). Because if the status of some data on the business is changed, + but the request times out, it is considered a call failure and needs to be rollback. + +# Detailed Design + +## Design & Implementation Details + +**Let's take a simple example to see how we can use the new SDK:** +```java +@Test +public void testRpcCall() throws Exception { + setupTopic("testRpcCall"); + Map requestProducerConfigMap = new HashMap<>(); + requestProducerConfigMap.put("producerName", "requestProducer"); + requestProducerConfigMap.put("messageRoutingMode", MessageRoutingMode.RoundRobinPartition); + + // 1.Create PulsarRpcClient + rpcClient = createPulsarRpcClient(pulsarClient, requestProducerConfigMap, null, null); + + // 2.Create PulsarRpcServerImpl + final int defaultEpoch = 1; + AtomicInteger epoch = new AtomicInteger(defaultEpoch); + // What do we do when we receive the request message + requestFunction = request -> { + epoch.getAndIncrement(); + return CompletableFuture.completedFuture(new TestReply(request.value() + "-----------done")); + }; + // If the server side is stateful, an error occurs after the server side executes 3-1, and a mechanism for + // checking and rolling back needs to be provided. + rollbackFunction = (id, request) -> { + if (epoch.get() != defaultEpoch) { + epoch.set(defaultEpoch); + } + }; + rpcServer = createPulsarRpcServer(pulsarClient, requestSubBase, requestFunction, rollbackFunction, null); + ConcurrentHashMap resultMap = new ConcurrentHashMap<>(); + + Map requestMessageConfigMap = new HashMap<>(); + requestMessageConfigMap.put(TypedMessageBuilder.CONF_DISABLE_REPLICATION, true); + + // 3-1.Synchronous Send + for (int i = 0; i < messageNum; i++) { + String correlationId = correlationIdSupplier.get(); + TestRequest message = new TestRequest(synchronousMessage + i); + requestMessageConfigMap.put(TypedMessageBuilder.CONF_EVENT_TIME, System.currentTimeMillis()); + + TestReply reply = rpcClient.request(correlationId, message, requestMessageConfigMap); + resultMap.put(correlationId, reply); + log.info("[Synchronous] Reply message: {}, KEY: {}", reply.value(), correlationId); + rpcClient.removeRequest(correlationId); + } + + // 3-2.Asynchronous Send + for (int i = 0; i < messageNum; i++) { + String asyncCorrelationId = correlationIdSupplier.get(); + TestRequest message = new TestRequest(asynchronousMessage + i); + requestMessageConfigMap.put(TypedMessageBuilder.CONF_EVENT_TIME, System.currentTimeMillis()); + + rpcClient.requestAsync(asyncCorrelationId, message).whenComplete((replyMessage, e) -> { + if (e != null) { + log.error("error", e); + } else { + resultMap.put(asyncCorrelationId, replyMessage); + log.info("[Asynchronous] Reply message: {}, KEY: {}", replyMessage.value(), asyncCorrelationId); + } + rpcClient.removeRequest(asyncCorrelationId); + }); + } + Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> resultMap.size() == messageNum * 2); +} +``` + +## Public-facing Changes + +### Public API + +**@param \ The type of the request messages.** + +**@param \ The type of the reply messages.** + +**1.PulsarRpcClientBuilder:** Builder class for constructing a {@link PulsarRpcClient} instance. This builder allows for +the customization of various components required to establish a Pulsar RPC client, including schemas for serialization, +topic details, and timeout configurations. + +```java +public interface PulsarRpcClientBuilder { + + /** + * Specifies the Pulsar topic that this client will send to for requests. + * + * @param requestTopic the Pulsar topic name + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder requestTopic(String requestTopic); + + /** + * Specifies the producer configuration map for request messages. + * + * @param requestProducerConfig Configuration map for creating a request message + * producer, will call {@link org.apache.pulsar.client.api.ProducerBuilder#loadConf(java.util.Map)} + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder requestProducerConfig(@NonNull Map requestProducerConfig); + + /** + * Sets the topic on which reply messages will be sent. + * + * @param replyTopic the topic for reply messages + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder replyTopic(@NonNull String replyTopic); + + /** + * Sets the pattern to subscribe to multiple reply topics dynamically. + * + * @param replyTopicsPattern the pattern matching reply topics + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder replyTopicsPattern(@NonNull Pattern replyTopicsPattern); + + /** + * Specifies the subscription name to use for reply messages. + * + * @param replySubscription the subscription name for reply messages + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder replySubscription(@NonNull String replySubscription); + + /** + * Sets the timeout for reply messages. + * + * @param replyTimeout the duration to wait for a reply before timing out + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder replyTimeout(@NonNull Duration replyTimeout); + + /** + * Sets the interval for auto-discovery of topics matching the pattern. + * + * @param patternAutoDiscoveryInterval the interval for auto-discovering topics + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder patternAutoDiscoveryInterval(@NonNull Duration patternAutoDiscoveryInterval); + + /** + * Sets the {@link RequestCallBack} handler for various request and reply events. + * + * @param callBack the callback handler to manage events + * @return this builder instance for chaining + */ + PulsarRpcClientBuilder requestCallBack(@NonNull RequestCallBack callBack); + + /** + * Builds and returns a {@link PulsarRpcClient} configured with the current builder settings. + * + * @param pulsarClient the client to use for connecting to server + * @return a new instance of {@link PulsarRpcClient} + * @throws PulsarRpcClientException if an error occurs during the building of the {@link PulsarRpcClient} + */ + PulsarRpcClient build(PulsarClient pulsarClient) throws PulsarRpcClientException; +} +``` + +**2.PulsarRpcClient:** Provides the functionality to send asynchronous requests and handle replies using Apache +Pulsar as the +messaging system. This client manages request-response interactions ensuring that messages are sent +to the correct topics and handling responses through callbacks. + +```java +public interface PulsarRpcClient extends AutoCloseable { + + /** + * Creates a builder for configuring a new {@link PulsarRpcClient}. + * + * @return A new instance of {@link PulsarRpcClientBuilder}. + */ + static PulsarRpcClientBuilder builder(@NonNull Schema requestSchema, + @NonNull Schema replySchema) { + return new PulsarRpcClientBuilderImpl<>(requestSchema, replySchema); + } + + /** + * Synchronously sends a request and waits for the replies. + * + * @param correlationId A unique identifier for the request. + * @param value The value used to generate the request message + * @return The reply value. + * @throws PulsarRpcClientException if an error occurs during the request or while waiting for the reply. + */ + default V request(String correlationId, T value) throws PulsarRpcClientException { + return request(correlationId, value, Collections.emptyMap()); + } + + /** + * Synchronously sends a request and waits for the replies. + * + * @param correlationId A unique identifier for the request. + * @param value The value used to generate the request message + * @param config Configuration map for creating a request producer, + * will call {@link TypedMessageBuilder#loadConf(Map)} + * @return The reply value. + * @throws PulsarRpcClientException if an error occurs during the request or while waiting for the reply. + */ + V request(String correlationId, T value, Map config) throws PulsarRpcClientException; + + /** + * Asynchronously sends a request and returns a future that completes with the reply. + * + * @param correlationId A unique identifier for the request. + * @param value The value used to generate the request message + * @return A CompletableFuture that will complete with the reply value. + */ + default CompletableFuture requestAsync(String correlationId, T value) { + return requestAsync(correlationId, value, Collections.emptyMap()); + } + + /** + * Asynchronously sends a request and returns a future that completes with the reply. + * + * @param correlationId A unique identifier for the request. + * @param value The value used to generate the request message + * @param config Configuration map for creating a request producer, + * will call {@link TypedMessageBuilder#loadConf(Map)} + * @return A CompletableFuture that will complete with the reply value. + */ + CompletableFuture requestAsync(String correlationId, T value, Map config); + + /** + * Removes a request from the tracking map based on its correlation ID. + * + *

When this method is executed, ReplyListener the received message will not be processed again. + * You need to make sure that this request has been processed through the callback, or you need to resend it. + * + * @param correlationId The correlation ID of the request to remove. + */ + void removeRequest(String correlationId); + + @VisibleForTesting + int pendingRequestSize(); + + /** + * Closes this client and releases any resources associated with it. This includes closing any active + * producers and consumers and clearing pending requests. + * + * @throws PulsarRpcClientException if there is an error during the closing process. + */ + @Override + void close() throws PulsarRpcClientException; +} +``` + +**3.PulsarRpcServerBuilder:** Builder class for creating instances of {@link PulsarRpcServer}. This class provides +a fluent API to configure the Pulsar RPC server with necessary schemas, topics, subscriptions, +and other configuration parameters related to Pulsar clients. + +Instances of {@link PulsarRpcServer} are configured to handle RPC requests and replies using Apache Pulsar +as the messaging system. This builder allows you to specify the request and reply topics, +schemas for serialization and deserialization, and other relevant settings. + +```java +public interface PulsarRpcServerBuilder { + + /** + * Specifies the Pulsar topic that this server will listen to for receiving requests. + * + * @param requestTopic the Pulsar topic name + * @return this builder instance + */ + PulsarRpcServerBuilder requestTopic(@NonNull String requestTopic); + + /** + * Specifies a pattern for topics that this server will listen to. This is useful for subscribing + * to multiple topics that match the given pattern. + * + * @param requestTopicsPattern the pattern to match topics against + * @return this builder instance + */ + PulsarRpcServerBuilder requestTopicsPattern(@NonNull Pattern requestTopicsPattern); + + /** + * Sets the subscription name for this server to use when subscribing to the request topic. + * + * @param requestSubscription the subscription name + * @return this builder instance + */ + PulsarRpcServerBuilder requestSubscription(@NonNull String requestSubscription); + + /** + * Sets the auto-discovery interval for topics. This setting helps in automatically discovering + * topics that match the set pattern at the specified interval. + * + * @param patternAutoDiscoveryInterval the duration to set for auto-discovery + * @return this builder instance + */ + PulsarRpcServerBuilder patternAutoDiscoveryInterval(@NonNull Duration patternAutoDiscoveryInterval); + + /** + * Builds and returns a {@link PulsarRpcServer} instance configured with the current settings of this builder. + * The server uses provided functional parameters to handle requests and manage rollbacks. + * + * @param pulsarClient the client to connect to Pulsar + * @param requestFunction a function to process incoming requests and generate replies + * @param rollBackFunction a consumer to handle rollback operations in case of errors + * @return a new {@link PulsarRpcServer} instance + * @throws PulsarRpcServerException if an error occurs during server initialization + */ + PulsarRpcServer build(PulsarClient pulsarClient, Function> requestFunction, + BiConsumer rollBackFunction) throws PulsarRpcServerException; + +} +``` + +**4.PulsarRpcServer:** Represents an RPC server that utilizes Apache Pulsar as the messaging layer to handle +request-response cycles in a distributed environment. This server is responsible for +receiving RPC requests, processing them, and sending the corresponding responses back +to the client. + +This class integrates tightly with Apache Pulsar's consumer and producer APIs to +receive messages and send replies. It uses a {@link GenericKeyedObjectPool} to manage +a pool of Pulsar producers optimized for sending replies efficiently across different topics. + +```java +public interface PulsarRpcServer extends AutoCloseable { + + /** + * Provides a builder to configure and create instances of {@link PulsarRpcServer}. + * + * @param requestSchema the schema for serializing and deserializing request messages + * @param replySchema the schema for serializing and deserializing reply messages + * @return a builder to configure and instantiate a {@link PulsarRpcServer} + */ + static PulsarRpcServerBuilder builder(@NonNull Schema requestSchema, + @NonNull Schema replySchema) { + return new PulsarRpcServerBuilderImpl<>(requestSchema, replySchema); + } + + /** + * Closes the RPC server, releasing all resources such as the request consumer and reply producer pool. + * This method ensures that all underlying Pulsar clients are properly closed to free up network resources and + * prevent memory leaks. + * + * @throws PulsarRpcServerException if an error occurs during the closing of server resources + */ + @Override + void close() throws PulsarRpcServerException; +} +``` + +### Binary protocol + +### Configuration + +### CLI + +### Metrics + +# Monitoring + +No new metrics are added in this proposal. + +# Alternatives + +None + +# General Notes + +# Links + +* Mailing List discussion thread: +* Mailing List voting thread: diff --git a/pom.xml b/pom.xml index ba39698..9819259 100644 --- a/pom.xml +++ b/pom.xml @@ -212,6 +212,7 @@ **/*.pyc **/.pydevproject .github/** + **/**.md SCRIPT_STYLE From 46e4831454c595426d04271dac468f672e1ba674 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Fri, 11 Oct 2024 21:10:06 +0800 Subject: [PATCH 2/4] add RequestCallBack api doc --- pcip/pcip-1.md | 88 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 88 insertions(+) diff --git a/pcip/pcip-1.md b/pcip/pcip-1.md index c7e0bb7..92ace31 100644 --- a/pcip/pcip-1.md +++ b/pcip/pcip-1.md @@ -440,6 +440,94 @@ public interface PulsarRpcServer extends AutoCloseable { } ``` +**5.This is the most important interface `RequestCallBack`.** +```java +/** + * Provides callback methods for various asynchronous events in Pulsar RPC communications. + * This interface is used to define custom behaviors that occur in response to different stages + * of message handling, such as request message successful send, send error, successful reply from server, + * reply error from server, timeouts, and errors in reply message acknowledgment. + * + *

Implementations of this interface can be used to handle callbacks in a way that integrates + * seamlessly with business logic, including error handling, logging, or retry mechanisms.

+ * + * @param the type of reply message + */ +public interface RequestCallBack { + + /** + * Invoked after successfully sending a request to the server. + * + * @param correlationId A unique identifier for the request to correlate the response. + * @param messageId The message ID of the request message sent to server. + */ + void onSendRequestSuccess(String correlationId, MessageId messageId); + + /** + * Invoked when an error occurs during the sending of a request message. + * + *

Please note that {@code replyFuture.completeExceptionally(t)} must be executed at the end. + * + * @param correlationId The correlation ID of the request. + * @param t The throwable error that occurred during sending. + * @param replyFuture The future where the error will be reported. + */ + void onSendRequestError(String correlationId, Throwable t, CompletableFuture replyFuture); + + /** + * Invoked after receiving a reply from the server successfully. + * + *

Please note that {@code replyFuture.complete(value)} must be executed at the end. + * + * @param correlationId The correlation ID associated with the reply. + * @param subscription The subscription name the reply was received on. + * @param value The value of the reply. + * @param replyFuture The future to be completed with the received value. + */ + void onReplySuccess(String correlationId, String subscription, V value, CompletableFuture replyFuture); + + /** + * Invoked when an error occurs upon receiving a reply from the server. + * + *

Please note that {@code replyFuture.completeExceptionally(new Exception(errorMessage))} must be executed + * at the end. + * + * @param correlationId The correlation ID of the request. + * @param subscription The subscription name the error occurred on. + * @param errorMessage The error message associated with the reply. + * @param replyFuture The future to be completed exceptionally due to the error. + */ + void onReplyError(String correlationId, String subscription, String errorMessage, CompletableFuture replyFuture); + + /** + * Invoked when receive reply message times out. + * + * @param correlationId The correlation ID associated with the request that timed out. + * @param t The timeout exception or relevant throwable. + */ + void onTimeout(String correlationId, Throwable t); + + /** + * Invoked when acknowledging reply message fails. + * + *

You can retry or record the messageId of the reply message for subsequent processing separately. + *

+ * This piece does not affect the current function. Because the reply message has been processed by + * onReplySuccess or onReplyError. When the user-defined request success condition is met, + * the user removes the request through the removeRequest method of rpc client. + * Even if you receive the reply message corresponding to this request in the future. + * But if there is no request in the pendingRequestMap, it will not be processed. + *

+ * + * @param correlationId The correlation ID of the message. + * @param consumer The consumer that is acknowledging the message. + * @param msg The message that failed to be acknowledged. + * @param t The throwable error encountered during acknowledgment. + */ + void onReplyMessageAckFailed(String correlationId, Consumer consumer, Message msg, Throwable t); +} +``` + ### Binary protocol ### Configuration From 91a03466ff05e424f8d3cb6948ca08bea23e4362 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sat, 12 Oct 2024 13:27:39 +0800 Subject: [PATCH 3/4] add architecture diagram --- pcip/TEMPLATE.md | 2 +- pcip/pcip-1.md | 4 ++++ pcip/static/img/pcip-1/pulsar_RPC.png | Bin 0 -> 247980 bytes 3 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 pcip/static/img/pcip-1/pulsar_RPC.png diff --git a/pcip/TEMPLATE.md b/pcip/TEMPLATE.md index d21753c..543c9d1 100644 --- a/pcip/TEMPLATE.md +++ b/pcip/TEMPLATE.md @@ -7,7 +7,7 @@ PROPOSAL HEALTH CHECK I can read the design document and understand the problem statement and what you plan to change *without* resorting to a couple of hours of code reading just to start having a high level understanding of the change. IMAGES -If you need diagrams, avoid attaching large files. You can use [MermaidJS]([url](https://mermaid.js.org/)) as a simple language to describe many types of diagrams. +If you need the diagrams, please create a folder named pcip-XXX under the pcip/static/img path and put the images in it. THIS COMMENTS Please remove them when done. diff --git a/pcip/pcip-1.md b/pcip/pcip-1.md index 92ace31..24a017b 100644 --- a/pcip/pcip-1.md +++ b/pcip/pcip-1.md @@ -66,6 +66,10 @@ None # High Level Design +### Architecture Diagram + +![RPC.drawio](static/img/pcip-1/pulsar_RPC.png) + ## Implementing RPC in Pulsar RPC Framework 1. **pulsar-rpc-client side** diff --git a/pcip/static/img/pcip-1/pulsar_RPC.png b/pcip/static/img/pcip-1/pulsar_RPC.png new file mode 100644 index 0000000000000000000000000000000000000000..60a829689030e23adba36df2e5f234b344d49e88 GIT binary patch literal 247980 zcmeEP2_RH!8@5zRN;ioT+AJl@%rGds(t;>k$Zle?3}fH9+EXfulBHy6krt6%8>J#! z*%e7f$R5J~9nR5&F89ABs%|&t%sJ;f=Xbhs%&Js({bA-OKUKFUk`u`W8t`&XkcZkNH89pVq?dl zQ=kP=gVzM+$!TbdC|Y!A6cZvc4tyif1{J(Ba(D1?CU$sZoXya&@Q$_wg1If;>g$b0 z1WQYx7hmmYKqL|zzTRyQ!5qFj_+(Z zhc_`rMn+-b`?D}Wc82f7#?-)=;4pOj;3jfG4b3Amn?wQuoR3V7{WL*R=0Py6X6wx&uDL8ZU@h~ik%_Q zCu#7*#)ycwvIRP?>SP6dDZfcmPi2er&P`ytvF2blWDJSmHxuYLMT0>bw6P;XricT^ z;sj|i8rUiZBMNrdiLwdfb;yyNN|pL+g!z&%(F6wv^D*}m92x@h?cnmjhrt8?5A!DC%nfYu_T*<7+gDEH+=v5_CK3&xV2Bt! z95Kd3Z9~U=>tD#HDns!GoDFgKf{5xv(+zGNjcEAz&V2MR6B+;oW72;&00tmixuL0; z$m9*~4iGF7pZ@t6c<;e@Gv1yP3rixS!;&(QU~FfEBT5??;cRRmSPf=;lcSNjoeiWi zV_ET}BaJc51NJbJ0Y3!zFc=te{196jrT>VNhqa#<4f_Jb*a&;}*Z|)#D4dBZ?9bM{ z(IFWI&xi(&(zUPS!4v>UyjWq_OCW9u@fu_p2?aSIDE5&@XvBRX0{r<16;2*UBoUJo zpphu73B}!s252aw0f`LJiR`HVCqR!Icrma5r)cCI^7%i7$ruw~zz%Yf@C6D5$4A(NdEt+fgEs(W40#23Y2lg20km;eb_4Q;hD z_$egg$S5iT2e|R(dpI{m4Cd#9y9vzq7lns#)i~f96v@nG3FZXAA%kojBO)s&Cn^gL z8!fLfUFop=M^eTAoYf9RSO|O!C(LMpX($ae>|Vc8|Dl4w9>O5)@dbrrt?i(Sj12Sz zFW}@sHV&28Py$wf8iDEv@b^d=VdthDYMWG4c~uEkcq3kA1HkV?wGDv9h6Vs&V`z+_ zs>3L=`&Vl46T}S@LWA{A@(=}sO>%7zfbEq$??Py#sbiW*DnE(J0OI*Ctqe$VJq?us zR@q5Z24F0w)&}(S z^$nQs5Gh2yfgus88-ia3dsCF{%xw&aJFSdl&GCb6D#KbaybE&7c=!^uhCpzZtdqe8 zwA7j5kZi(|09suD+$Bpe;7t>6UKuNS`GRX9u6Z0sj0EZD>AlhNb|@$BF3z+4-5Z0RM=tj0E_{A?c~-VB+4fW zlSAb4nR*l!0)xj94=|7sHZs-416vMK*(MjxSe7#dB?BX^aU{bB{h4;s8D%_(qZ^}P zVfxt8Sf!{DW;5kN3mT#0$Fq#5%f=rEr|60#1{DZa=|gnBcsGKNwMWS zif7u#76ibLV>pu-TfU=oe#zJ(1X~H2#6|`EDHmIYAGn@|u|;TfH!8?mFh3`@3`_s} z#uh6LQ}!B zfV$b#T2u-4y5q(nB>R~}ZJ9_*G16uBV_V{o`Tz{j(Enc{9=X=h>rjk1KL)9;jvt4I z&WW5E>_2_hR6>^F{9~45ti(BCvxIm2W0qj8vmb1hg2XdVu7Zz?}y4b16_ z*We84PhgmYz#s%q_OBfnM%UJdJP8?q`{@NIv?yFRMBEz!t;uQ_$n!1aHH#_O%ODIx zJF-BG^rud^5%ycFpQzd3h`h$QJz%|b6n6DyEiTT~qL-Il%;Pgp3R{ zd}#1*EQ^o|}bdSkMlhhxV@O?oV;%dhU5>pSPLGO_R3>NuH&Qm`okMTU&Sde%shU8y5 zO(tL82`nHhH~44Fg_*|e1sK-F-!fE>ZYKPNLiH5VjAM6?f4ec=QC1bpQ zT4idgH4-6@b&uOgBr^Fe3s9|e@;h1)0QE;(##mhjuJMkqEt6lz{7Kp}7O?>35tFMv zL#Yb+x(P@V|9G!w6ol=1e38Q8b`nd|KS`wUj7+Wsr^m}Yk>?u5*B0QrAv5^(wFNed zKiV=zxPXDz_}T({t6yJR;68^x+A>;OkS80)*A^Is{QBDRCvo}a^D;;SCnB#&9kZg}F%g&TpVDup9lOE@P+*oWG9mU4ECW2!%qb$K@W~?^yFZ1q+cVfw2m|Lyfinxiv&uRvxKcjn{%pG$mMo_TBH+g8M~( zMvD9Ui*@Q%^HIp#5r0H~$FCyQZx;Z^vfZgDM?*tAf{ zYkj6;TpSDU{R(jr(Wl=sE+X~bUoI|wN2yNxxQH67DEY^9zk0DcEad4M7f~>M^OYq} zEFb@Q`5`>(XoO{(hH(+~i{^{N`t*CoMVKG{%9?%wm-#)VI{o9~4{R;|nfc-m5c2em zilhR(E@l#eSn6HtrF-7lu!Ps<)97?svo(ckpB)9co4Om z1U*ECn8u1X7H{^)AKDU~AaDN9*8MHT8xbZECHe9G5P?C+h|^ask@C%-FpbF41V^yI zeGj!9h01=sUXB`a+A1bmVA?cL|64RVqz3;4QjlfN{yT2{!GIKoi$cg6Ot3@73hez2{vM4(^e}{h-ApzCH%TmD|07hBEi_s2uFl7O|k~MuhWG?*2;)yK0A$+_hG7Uc|KXTx zSj)aK;`H?cNWMBj%=Kf6Q@@y6{$XmHl|V95v*$RZQ>}upwe;f{HmQiRF)o z+mlV&l%1jl;L0;)Zc74?4ldJ`jxRTv?-nSqQz8crt2 z8KhCjjU0ylBnl5%337rE?1!YRh|vE~cw=}SNG}Hek|)J*1z}jr;1G%sUkK2f#(rR| zhc-s|fgjDJ{Ay~6XvO!BNyB=G3^9!rZ>-fN{wUt}=$QON;UTX4E1y32{KI$;HVFQ^ z68xVqJ!+~)2T?d^jzVQW&_8}_$Z6{@(U^Z%@|cXo@`uFjpG@H)>+m4~_WSGT7!=;f zL8q~n{<{juA1xgt5g8-@5Q3a41djetc*7wbi8G)f2YzVrp9@UTv`z#9pfMAmM#xg4 zKRW#X>98ow_D`y(9XaT<^|UCV3E+V#K;V7tw~)+)jKD*vlqt{OjV_@o6O3UYkdda~ zA#1#q4S4QBW}N3!3~a2xqOjokh&^~m=vjT4ArDXfmYK1iy&jhQ$b}UC+5aK_lI*eZ zubLsp5(KghvC*l(qmC{T0y)62W7^7L1g|)beM~{IAWfTO^fnL+97(6USSIMUrlMFT zfXAm?lo+d38=2dw9wv|%XMAl3W4!a~$A<|-^uJ}85E{MW81P4Mv14-m(RY-?^bZrG z%S}KmV?3iit;_=!%QOxX6DY<^xiEndg1P!}(9$U0c?zId<6?&`6)~Z|NssJDaZ9LFm`<6Y!n< zqn{Yvko9A7y6^8IkWP~c~bMH6Q#X ze|8>nQ`4BEN7QJ1j$Q=r@tZb&dBmaufTI^AU(Hw)MnQ;8PhSEKk<8bA8uL?fJVc*< zUydFA%pYH0km~qvpfA&gmG}wz0(*@K=*#5SrhXP!fbVp2W5oZ{js6q#1=Qb);4t7z<1cQ#hHceWGZ6`pW1f-w2NK|C7c5y+`YcTK*3L7tnL zfPa~M;_PR|qiNht3kQL5ORT~Z&T|lY<2UdV(~snW9=aQJ8b6xk8h+k=64hWLOHV;A z0?>D|(nO+x6ZjP=pIH&`mbNw!4>0kwAm_4hRECZQXG2zMM~!IN7JxsG&#euellp%;_uwkM`FrGgfaeg7Br|~a5o7xMGZbuyWF2K-^(I(B$y5CAxOx+;?%$(v42B52SPaZ!!8SGA zQcBi-iqwF~*PBDOA%gt*?Y?L42EO0rg<6gDk>~DoPJMzJQV0gEu#q zC72V4gMAzrQ9(HwInY1x?e>sSF$67OM6k33t1ufF4lbqxh8hfJ{|x`XX!>Rx!2&n< z#vUa7O3E@&$!qt?dZN7rZujXRsu> zG{_@R0#*PA0V6@a2fvv>`|aGMtf0xOO0dEk@hTe-ZSl5vf+aAAq0<9rJv745&i_pf zhK?q0;QI06hQXgO=q@AArx0@`H&0~AEpl!6so3uuvx7YQ_N5CGk=Bn<27onxZDl}Q z@HA8gghZP(&j2`yUt1Z*$_i1_#5Eufu}zvbfK2Nb*M{lr8{o#2Nz(?<*zjv>19~dK zhC$|%NPxiJCJq4z<9+#ylt73ER;G#sW1RU{`wV_-j5jbL8d$(zzy|=J4)7v$I#->x z3<`?n6k9h+t2*j_sF;(Is!}m9a2oADD{teIMQ2rZtUMdx-y|n%y6BWlFhKPJ&w`d!R?mrudr#Ox5Ap8JGOeur4QJ2=+&i}yM(&w^nP&5sqA%qS!2yscRn@N zOvfxW)-J{@=2hKoi3S6Dv5y(lYj$!kpxsAxL5lpxs_2XdY@Ij$T4%RNbkXdYGG5Bd z2Y-}AWQeSIe{iO9sLnCry(}Y+^YNqd>=99Xky&54QD^uVaA;V@-D9t)qBkec|8gq$ zj3?Ff%Q;O~xvaYQn*0)MmG#Rr%_s}QFCQFnX%{jcuNAOlh&!43*N%_yNd|v_^Rtj$ zD)N>7UB=mA66IZ7+nC5M7rLPPvf`z-+;OSzvoJkc;-_>_Ge?Qq|3s~0N`KauRWhI-oq1#z?HRG=03%O3YjrYi}_B|qI z74nH2Qk-vyc?LNDXzja@Va2mR%I$8ml0P5$$#y>1!5>9pSVFWd?n@=-c*or%I35xbgg=ltmo9RGer7j4Bn}~H6Z;`Xn%WD zeSDdv{9L}f`dSA28S}GwV#UIH1Y=~Ev4i2mV}|U_*WSe^f8@2@)`FDvIVot#=}-|->1q|p1}wo+1aS}&XJItz)OJm1K9*y1oV zHtQkHQ32KtnRQg@%IeRf#2APg6r+($=6+c(yt;lCtAj#Rh%@PuMuk&dFvIe4LEhB% z{Q7|O&p0ic=RyT{Ug?6K>i?ZY+_S8p3StL~0XTNIjzh@%RcGf&FaGj3zBgT5C zT$jp>>;+2(XF|#gXwgh%U;A;?`T(u!g*KO}-pXYM-V!`|wUgeSlFw&K|Mci^x#itA z2jWOqu41#@0zG@m6Kx6#LOSDi_e*|$Ltp>lcJB57U1xlMXhKh{M_+@Sof+|?U+Cwz z?Lk$aIYV2k2KuT>So$hmniU~27PMrshe}qtwmmP{&!k(tx=aK+sPUP=fE{@~oohH3 zaE)}%7cy=DHDZV}$P>!`i1);_TxPorDe&zP03No&1oBC8^V1h)_YXWj~-~y8^~U9^IBBG>;f0yWjd@KJ~w$_Xu39ZGO}BnR7!-fNmd2BWmX2UCZ@qjuBjq zdcaxEVM~nt;_|og)Vkgy+2o1~t2K0nv-few+{#|dP|KKAen-BiDAxV7XLI%C{!;-d ztHoOmD=C-u(8fsRAP=`*{DPh7L*SSeO*(zb$k?VJ?5l$A_ru2+Xo|$sca91{HHjv>a|f?vN7R*kuxIr;g>`5=`P7@SeY( zo86{x$7YUm^6JaXfz@<3)4RT=t`~21QaVSv^M2VHpoMaqhirvCIG%}P9$73L^1F*6 zV_hsPvM7+3R@EzM0XLSEGSJ^!kdWTj7}wWXrX*F{>{_>>WP|fsslKlC0XHqzmPfba z8e-GkXT4i4@>)BiQ@pF(ufMO^!=%|tiiNw1klvU7_gua}@i>kfK}<>RQBt+U^nHXk zI{m=cG)YUb?s_+mZ%tNrU1(_9=UV={f!e3HS&jKS+7ivY>ooM-?0PbV(@y%HPDgvc zO?l~*CZF=i>8?ksU+DQ4A8tM6lLzW;>0HsE8<5r>!7m=YwdHNXlC_`uZUv;88y37O zBczM--hPH==!iR6FWKMX+kVs6&3t8*WM91!K5r9K<1PmQw?~W`&zw}}u@@<76?st_ zTzPfT<`NWuScf8wn{BDCzaV+c8gz16AXLxMDu1o4AJ469QnOuOvzT#(LL^n8N9XO( z+7sLh!=A}&aWk|$B_!2He(t`i+i~RsFJJlL8Q8fU#DjQ^yTsd zBUP8hYhTP;ADj5@TuGVs?xwX?O+s(y(mS$M70oDSx5}WzA7r(E{6d$igt8U#nDs!@ zn`H0D#!=Z(l$#V4##kP=t9-z+b?TTDBnW;&SqsKj@8Ls6s(hD8>WCTc0ab zYVYZm&Sk82a{pk&9wd9|S`z6B!)tqVSd(LAZiQj=+>=xx9vxTp+zuJ%I_22p17ql; zJLj}7An^VCa^U*?=1Dy@iIHMm_fAw-F&N7+Gu~IM^>O=UCAYQg*TvXBzoua~7I>Y= zXX(0C=6bQeYHN4sz~{QdB6a-^uE}pjf{t$sv!CO%>e}<#0KJfV_9as7?FC9oOuMb5 zvHDx2oRU|^_w`l{^dIiczuqKMC-!EJ(&MPMz`B`w*cErEUIVpqO|zH0E_B60GMkn5 znH;Rj*!b#bn-N}5fO8O7Dp<;^u3Gf|N``UJsrU_v`<76a56ITKuG?;;4fQk20Yn(7}=` ztSQ+nHWeFl=cQKn(nl<*dj+_y(oL5yQaqd}L8wgGt-C_&QderT8^etrC@9R2See@2 z5u01P*l<&+eVGM=sji}7ax;i*-K#=s(m|NX_o8779MF*CnXx{C$X{AiAsX>afBoH@ zz!f3AOH9t6yuFARrT29Cw@P^Qe~x>&Q2tVdeHrh;%O@Y|>k*%D6x6VF-92nW*?w!z8t;cq$;}dP?#&OY zvd&_ykKlL1OR&99(7f_owhQ2m)%Hoo-l{C;?5RmIsVS|eV_lVRi*=RT6sT)koR(C{ zan~_hROaq4x+w4%9QVx4AML0cfztF9AYWD*Sbip2j{Ak0SEQ6%s9WdO&FpTrS*~2* z)S(~E`77U}--zo}ds7!+xoRnMf|6G|ow>nXiwP-La>`b5o<+L;L|%6-&T=NHajj-` zK(A$hj{R0_n|Z($RIz0D6Qw;?5}FIoY?{3kcjE~@`GiPL>;Z@RS%uB%1O3+sc|mCo zRc;PirWf`Tcl0>LT%X4~A41R<;FR&;Ma@ksMrAo+Oprf%d32Wbc5ZP?pRU=c2c93oMwqPwX#xxB7^H90Toht24?o!|n%t zJmp?m5OU^0>}-pjielIHduZZLt8p$T`ZWlO$CECkfjA!pL92Qwz*o|#Em^E%n-{OS zUalN+a?SW-LNDL%Qtwc&)B9Y_v?Q~@@+|Jm6YG~{95=Mna#-SK6*$F2c5xhs62|^D zL%C6(8OuqP5q@U@t-@_6Ef3n$T}_0b*$mn^YLcw-HEx-~J>67>@`~Pa6tleHjD)64cV*Y8Bbmbn$$&7`lXhSny32sy0NKu& zhZ{3%^S1!DTZ$4c=m_@anZ+MdG6fYccWw0Pn_TEnY}_ zon0DR5`O6~XK zPxc2V5fhW4;3W3kHnpaQJZSqIq4HiD+t=kW;IJN~G6mp*PQ(`*9*qOzMhU&zjp0rI z+**HF$<fX$PLG)Kv)sL+pKIN@XYPc>BrF2Oj@6RpRdo|{aeJMVw9e9bvn?96< zEb@!T&d{Rgl-?-h2N~Lz{4FDz=S6c%DU|aa7pXo(mrL0KrE7cFq)l#U!TMIb@hzzmW0$E62ujX4kj6G~UtBGCsK} zUzU-#2}rtO;NAS7P@@d3S3SCxmy`gegw`L{t1blqu|a`W09%f3>X!FhaeMJk=s z^nq47w+6X#hF_PG%rr}R;!x>SXIpNa6#;>Lw}7`G!8I;5@)ghD$`xEFM5jQ}k@l2* z$$=p@2>>LlQsV3CxdkcRZ3XH1;JXT)3C=x5GKoKr5qZTwkX5u4$sCE47r zX~vrU&Z#K(YUt@>ou#b8bk1}tORfaT(W@k0Tz6ZRvRRL1r<`*%70$bo#2MrL5`U%U z4&f?!I!?oKv8An`-)HEVk9m-Jy%N$>OvBok;x#=e?N5hmQGvmyh5RdK71LY42 zs`9G}@>zHT&Xie29KINPBM-AVV(BWioS?b^!3x(FrGf|Gzs5GgcHw8~e2q4b-RmEV zvM;^tywLzi6S$v1(hQ3iLXMvvu!r<_=Xyj^$a|}=ZzV)!+>m5fY|^59>*>^8)~tR* zr=S0IG6Qq7ZR2*&v}Jm!@wwR@S6M zcVUV$m6;7uPv3b6$He7b!54^XM|jp@9HJ;X%lOkcf*l?YTuP-c;bIl9dwiBh zVK+n!%WdfprV%Ro?nlfd7!VGn{ge* z(&EeaZP@(tx$nPUYs!B@NBpzDvtsuwdZ$_8`TYlWXL9S()6P&Hkjx`dwIyYBSZ!!s zliE&$U9JO+t^mMqi5CL=!KBrnp8}^5lm_6cxN&$s1t~Rtp9t(m^;a_%6Po(#AInOd zw+~7nI9{S}VNJ1W(#YR`(3RtU23?Ezk&FAO9$b{-XNap`=R(m@ue#xS(UlAVNp`hE zy4ey!xmq^>LLm=x`f?o73c7-x*0da2b^NilQyE9?F1sQ%%$7tzF6I{i?yUWD zon(3)F|JeXk+<9VH$6AHa>c8jiCDj9416w?s&PU$^|H4oNwL?v##d6JE4MDxqu(bX z0nl6MEEOmd=E`0u<(5g1Uj)<0`s)|HKHGn%yDm!jxl5hxZocbzn`cUO1enL+%y*S; zR$O&eS^xYZ)AiCpO-VG$p_Y$ooYiI?sPE?XjNB)Nuj3V6qC8N|lpZMY=^lL~L>esF zm)?0NR#CFIBsagTvNIk)2H}WT06Pvk;`Hw4g!~Ov-8t>io5k10+{|2xuX5`OKJgZK z|LmJqfXWJehx-h2*ig(3Q2-W+=XbNrPH%Q=mf!Hko>swh2oYf<2m(`lZ`6S<8xOrpSn<2uAEL%_>w=0gRK zmdt_BfHvw-woiAesrocB4n%~OetLZ9a-|1;RhF{;x}jOx2G7!iS%nRr?Q^8^?#cGC zRDF_jh8P)l;Qb=?26|qB%-dNd>CFL_d^6p)bM1dKr7X9IFWG~y3wGGv?3lHobyk?v zGfJ9eC+;~5Da_5_Ej(J~(v)a2s}|z3IuFr&fU<(P=h(7KS7X!Gdpl)43lipMwS6Hi zDoFR5FBfpPWlOGjyj{g!zw3CGc$0NKMEF6}D`pYVoGVN>)ueC6E=->woy&k&LC?bu zny3tOYEs|FZJ{45+r#zt=Cckl+xbkq_KbHAQBPV;>F1jclHFIXOBWO+pB9-r1c9N z798oIaumx2v@(DGAc?$sg`T6AM}u}%_8aw(R{y9=P(adjAAnrE%LNaTO6az^ym88% z%b4nxZK*pu|9o#tI-u}3;NJI34zx+)JBl=NDJtmc?WKP4@G?Rb_>ywPg1#A;?!lU>8d5>p>qwILuoX- z9=q&eWR?rlt*}beBemz}b~-m+mg(T^^I}w>J4exZW3}f1V0hneT-N~#Bt(;o%V69^ zy+c1Vu{V;Xzykk3RD1sFfycHzivlIPKHKJg-caCB*JmOd3$d&VXqCgB%&_2G)W8@` zZKdR#@5hNX`Jmnjm{j&T89_0J>pnujkgfai=E#UMT5EZ4WKi8!J3>?B5*gx4rR(VM zs$#a%z0i(pCJ_wVMOGX%I_bCuP_0$Zbt}a(b46-=0N3ZX#^rL^TZT*Tx^5eNUqS?YxgK8+oUxpoV?m zla|^{O^!a|B^sVEdfcU@0fIhKsa3f)>1|hCnq5*YEBZ6`ya3#XeGf2W?Sw7Ie0zZ- z5B@A@(FVz{jdTfjMw3&lkmV{H0yTa7sdURLWKzq;2XbsH%WoIZ*A3`jB4M}h*Rct| zlYfgPCQ#TXcwJ-LjRb6n&vC!)I|4x9bBw04<1)oW=%|F{Y*4?3d$Ha55s&W4noZRE zgBOM+wCBr5L8&bsa5H!p=GQ7Jg9;H3=5JWXa+^e9m%CV2%1qSpxl5#! zi*5_)#Y5RtE&EK*g)V7N`a zh=Oo!ZQTc6f|GoitY@kEY%bdqy$@W}1`6c!)#Jo7FK&ObmhR25D_aN5XYR+MPrprM z$ljtIa=cKn?(!J}W-sTY=Ou%6>-{PYMs`G(u?zgYx1qA5$k?Z#XZ~*Bw_1xc zmHdn!6o7PIAlR+5gdwU3@7GDvx9XTbhyJpyn=8@_(7|~m>n3bat0FSeY`Wwnw+Tln zzDtR-NFIE{G6P31DAT7eG4RCvNTt zN$;&_9t0fH+>U!?aw#aWWq0X(cD{tTSf}u9y8Yqj9HoLjgu1pqS+Wc&CvB-F3UBRm zmLSF1)vgr?HgV>!&)LxYOkY{&9z)k6pZHb^IvOd*SrQ>?Url zjEiGX25H;V0+z~;2R`WJ>{^|+{@A1SG-*z8)c%GcNAq-))zL?GxskM6|ZUxakDPkPjMTqR0UP>OuXM4AE${a z)$nrm!{Sp~yxBi54|c3ROj+35C$-TXR+|Umo&lYfZJmv?O`J9e>l_giyTg)uYMGeB z_Vpi5$QI<$yK;)3U~}BUvFc2ZV`W#g8n*!BOw#rjk;G$9a&$oQ-MX(0Wcv7qj^Y{7 zZB}j%#?u)lQ!`$xTHC=7HZ{ETP$PAZ6e+;a?(8TQb@X=!JbuJd^B(>P=th48- z_P1W9E~Mn4xV_pB&!EQeIJqf_C$&MRIUBXn3H|r3*O4>!QoCnF3NXEn>N1ZadaL`@ zauGuIX5W6wm;9FbRgn3ut!bSguRSZwLw#eJY{IE!N~SoJn0)Ws`v75dgJxm^BWFI93y!OPr+PS)V;H||tjcG=jHJ8>?J zvFtrBV^k7qlI|RqZqKE|@m#-ac>Jnf%B>}O6jpuv`irY@mjjw$KEYGO>>!h{rhm=9Dy_n zJB!`vVIFJU8UQzVGNi@FHR(B0hq^H;uk&(wrsDz6pFnbs?x~?Dg~NKJmTw%UiBs5Z`2%LPl_&gKx-A| zC2E&7bh-f!Ji)iYzCt%GmO7$gS%I^~fk!VL4ld9tO>lUH3GIGn3o`23d7=BS8DISp^fC84f#=Q7*p%tHs+gn80k221zDW84O@`4mKr0h$L zOWwbrO3dCGqn|oQnMPMb3m@^B+LfY2f8Ps+QzvFYg!8QUS`U_0=ftQ;%lS;_YI%xg z&_3R^kbWa7I@u}O+s|`uGe3(!8sOCd06n+6jTYl1bm8scIVlur`p344p0GgE?eRX7 zQoiAI+3I)|cm3PS>?yevXgNxXqjl(FX%Z~%L7b=UTl)=Ff!-9@nS9I0k5;|${GhPErO5d8}ne7ito1mD7%O` z{sn(U>BHqcCdy2+7wIK>O1t4uX~mD#J$7VZy_vjUJ{PV!e~&Lbysi@Sq2&5~i@zJ{ z_E`W{p0Wp`_qwd8Yn)Vq4&J#n*#h zDplJhWbfIe;HvrCggdtJztRu1R$_MAKj&7fA*w;#!)jWzW zi1|?4`y|7;qTiG9ly!8`T}{vRtGuia3@k~%qtUFNYZ%p)degVYKiBcm-G{}IOIsth z2erGt@Xr(WbgoeUD+_L&0>G^KY;UElP7glIivO>pE_Vra>scfvV=yKU;;+u-JLjUS zg%QWFqUyXK=p5$O0r*$pD1^$q*D18*U(*fu?BJBD=)SM;@$M`er}PMcuA2k( z?6ea)Y(Ae|wh}9-*L`B~!<`7SzOc`A$9y*P6vca=?dQCBljl-`3#g6tV)vYDzF;Yy~o>u!nUcTpGE?kM>pRmbf#yRwF)8(yf$D?kObUQl#cx|OSD z7rZT}ig*|iK>9wep8oHQExe}nrmHaaxH4;o2__woVNnz3?TM?hRTQ- zmKdfmU0y%?x{|JCN}XvYrsxA7`%h=nrbS9s?8~7 z*GI-ZI)5;;v_G>vHqtJCY5fh+fhD`;mP*XT7Ju4=KPK7`Ka+GnZ8xTCL;WqMD%P~W zHy;LsOW0K^PSx9MF2~N<74<1xjW@$0EYf@(t=z{;f{8Ovtb20ZIr=E8MNKA%!=GQl`&J-Y@1dgy zxnCAl;eL??EW6Q%;~yqE-tE2yGIJHq%7upr#5o`0GL>m6og~i)%;2nfePV`x;N7YHlWv)Ht^>wxA8qPJKIS6OWoT}s1e&n+_1hbnk3ZEI2HXRFC@p?sA+<9ticCDyI1v)Qg`B#`v(2{08w=&*qTmM~-TX+I?sV zR86f1_^PP$WcV4wnm#Q|-)*|?M9j6FEjm2!JA8Uhw&%_?PJr%c31{DXo_n3uPhuq3 zEKm`>dexOe-by?A2r2T?>qrT<=sWe%H?T9KHH@M)w3T_3yA-Gn$!MN-rhe-~b-y5* z$#Kp_+6!u)&Rl)|((LPOkIs;zI@gnI-@#a>BUx$~Y@{M)QW0^f0%3NqB>CJ`8OB@e}GserFOr1OEKXYZ8 znbViZpvmW!KF6*}EPt^6pnZUJlgiGwcU&p_ictFEkCG5Ha zsbRfOj5=?N`WgOUX2w(=|4jqoxi-1$>$9xZJUbrPC_}uK)RYw&Y6 zt5OEtIG|aylmdJ54SHq%mnpW=Z63!(|!TT8p}Ofo+sO&;6h{-wSEzY&;#w-miFF3^& zC3ba=xlL$uYFDNCBK)bIZdY5g2AphqXN4zhSN@5h-Z}+J^)$;MJ z#pF7p_((@H>}E;krkEVP5KCc~iONm9q$fJ?xsAb7wtS8uqk&n-5`dYOR5(u ziZ79;>o{wg8!Y8)*4OZN$HK*rl2v+7USl(oGB94aA#_Lah8@-O_!gb*vS5tv-R@#o zO!=yD9$NE?N?KB%mqUz0Qmd-UoRXJwq9H6|ZlY{>!jCytWSM`FnZ(rsPMxy0y&YQn zT+fnJme#?{HC+UiHx-HV&MZHUC7vwHmoC?SXR*Xl$Y}@5qAg-dDXy#u)kk>VS-va+aPM6FYISa=o@QIMgf!n-X7E5oa;M8{2@55!HQ{bYkVHA(%#uGvxi!Hk zsbUwKyV%@$*SEO&+`Wwk2IIHvGY`)QU+s7a~=&u5QM&wZ zbY8?Ag&gWA&;^zD0k0=~{kp(xseKOiLX>5|k3B7=JCP@4dXBXA@1(AP5ZoSHL!j-yLc!s{c9*x=C zwEL)UXdhdpdwizflGhXuUOqIc=*lsB<-1*trOw->XF(tHq5Ab4*d+Izl7`EkKYqn` zcCKfbScZ0Q7-uc@nT}(&6m3^KgwmDg&)TQX%2}W!E=HqfQC5_&RkYU)xDg2uBKWot ziX!IUy&fLn5F{GY^+`3v%Tl+#=oTP@w8TF&6h=8KM0JT9u{=0uEi*^^l^|h0y*lyy zAo~+^geeOtOdNp3d*;BoPi-G5XdEARTU@@(w9Z#VcOQpWu~?qt4CZB?@3FNvHCfEk zIsB?$o|d+6S$*S~&a#c_R;Z@k-VRz@&$Jv?Oa@epu}DtA{;qi^?$?BwJMO_Gn1H%g z33b+Tt%-ZFvXHH}Xh5-h2%J$##Rd-3Do+$wrZE&(F)Ea*olvB3{Z(uE$ zaVbEIL^6@E5Rz+{RiUzmke>61>Y)+S+BIT@y%BrgB7j_#mTqCR_BPAgY){bkmAq$6 zj!2tqHKHuqw1ABT`=Q0nF3UdoXxqAkeW@(U3Cq&V*;HC z0iK{b;%sfnEl^Hd7_3|rvk9sNR3v3RR}agPRS*eC?JT`|UDdjrVo~7by6`z4cgR`E z9AbU?aev!uR-VHB3uI5-%{@QUu@mZd6x}Yw<+bX#h@qj_{RXCeH@*0+));#p>%PV| z;{nuAEXH}uI=G1TzM9pc8!`+0g>}vP6y6^{MXQ`}_Bxt7swHO**ZC92HlODZS-;Ee zaf#=v)YAvk$1M-?gUcYB5 zbIJ*z+kdcHoUdIqHYD!YT;V-jbgPsJ`TQ>vS^IKgbA(t68tfW6Y1Hj^sl}wT#~Z61 zc2Ll*Xw49q`Fbff;-VPBa9_<>?v=z*X~?N{AK%K=!uHqPm&+5ED&UX$$9_@ZW4ME`P+SfV$oAdF#0Z zRW?&5kK=Wzigk`}g5DpN=E`BC@87$wOn2!HkesO=wUP14Qx0p|cg50TFcQ8Lw~VLi zNa}fL4SK^kLw%D%in1Ro=W0r)xZlt^M3*xoBhi>{hP!ux`uinEnPoLkb6rnunH|Q* z8Rt0Qc~GR|{ynQL6we>!LeH!eY^ZEGE~b9-R_lk|aax)whfT zP~9mhZ+K6{DtM+89RO{hxJu7cT;Zs5#7~vQPp(K;zB6=kpk*&O+Eu5oy_1lbYuwuq z^U(U`U2Xv%DL3WPPTIRcXH2g2?ey!HVt?{Jn!tX>IE{cYUcUvvyxN?NqHvME=L$5j zj>qoD9TGQrWy_G}>-NEDNpxFg89pkQlSJj*wp}=J;oKeTvQ9a!JsW+vHZnyjUh(wm zk4ydIZw4F=3k<&h%>QA-*0c}R87aiT;=@;;UcLGdG^E*u@Ns&cLbq}TT*>-)X?OR^7P^Dw`v)FI(L`eNHtG+`F0kKGU1~7 zG8S5&GgOM4s&;!{Hh(7VzWoG6Sr`<}nANH6W#A~0bdL=ecky1Dpn4|d^Du`EVSJV` zEm=?pgf1X@>sn5uwo#HY*?oG&ypBox$om|2&0NL&sZ;c^g%9-$7Mjz~*Jw*#t@NcT zJeKU^;us_kb;)BMYOon`J#jey7p<}|C)z%4p)$I&h9Yn2!F2XF%X=ygM4qY=h)cih zFPX!tGjk7vvO_>p$8D8FKb4vFwWW$Z)Xh{BW!stzwpKD$3cX9&Nzq}Dh>_hj|Ad^k z?;+VB>lF6?zJp4|lzN!cm{#q<3~#)7_fm-M81SV~X98c4|(@6Qd4hzbypX9lI3vb#xsHuJu=8m|kUo;|9CM{Q)i-l&Jw@4KQ6Ihuq9p*{&uXO&0QpWN39_~dRb ziCeUBF*?V$Wx?Z|6&ZK4QdEi)H#FT{73+Fi!^e7dE})^iiJ5bK2pXVJOe{%7r{n9!)Ypu*j*kad1OXIh(Q0afyrDtcse>3 zJ1IpqOUmj^BJMmd>iRM9PdWSF&X_-OK4sxi-=w*T)ou7XP{3Wr6QukA?=5*9K9!1I zdh%TPiL0XTD^twv*th*$qr-F~bN%9wUX9X|k4?XGTF14n(E?P=AMrgA-)UZr6khsA z3A(UW-C~Zet!FY$4}}!zAHw1MZ&1{Dtj6s#YbN}8bX|3-CfsZ0`OMrv_X*30PtT{+ z7a;7dekOrP0_8I3IJthqz_DDr^OVDR;JpejrGUc(%`fFgW-|PFG-Igfl z*yr=nJ*i2$$MXY_SHIRWA1t%@C&r8Jgfvrgd~?E2)BCl-2FtEoVpppW&=}izUC@h1 zaJ3NG!s>4%Y&Mm_cty?aK!NmRsN%bgRM1@xI_-!L+1)VjbjD!z>)T~rcGM4(d;Y#L z2psmvIHom1BXouaIsXoxea;0v*E=T+%xkJ_TGQn7|6-h?XH}qs6aYRDOwx~Ep2oi0 z5zhOds8hO?ti-+$unp7u2K%omO{w1#{}Ob+6^TiRkp;!#75w|@wleSCGUtJb_#l5Z zBjLQwNQ9vlT~`5wQq{%Hxy9NyELvlr<+8Y>d>{~^)d2DoPM+oK0in)uz>af~W3>YK zhy?^VlH0&igDxh9?KR|nFsJ_u*Lf}mfIh`!(U^n?2E`^y2kVDQaP7Efu6fcM1pz}e z0dArjMi!v~tdU^ryKLpju!0?hEf4B-*3*#n!-l7{FAbU8x6A5hh5RSk##J%T!*k!E z!~Iwcqu>(`db8&9UJSkO;t0UVt3CJx@s{WNtvbfzF;1~(Z- z(84J6To7Xt(!6gmo=8xrlRV~{0kb#MXQNym@wD?(P+c7TED~8b z0<5L<_8W96Lh*=n-FIE6hY3NPrmZ(Z-MIC>d6rOrbKej8g&2L@6uj$g8ucgQG2j@} zBA7Bl&bSa`C0{qNfm-gN{{O=Qbf-+Am<_b0#J_0Q-M^daT;0#_Tlb`8gZ=v9^J0qk zeP^uMe18CC_7^ls-H5<)`7>S(>+FbCO^dRMAXqL3r1Q&pOL}Dv(ix@bN^hJxG|KaO-glsbl-m`#%ptX9(_<#3P{Fhb=z-4Ss=PyAyi7!mXah z?MIZqTyW(+<8msf2S3TugxdVSTscY)S}Q&k6q~1b>1{Aeu!U1!9?dkBiWCuknQu)S zS+pULo}|k1z(HUfMH}4Hnqr@nRvy>GHEIHLogZ$1R{!3M5U>sYKYj&NF=StQ^JlFHZp_8#B$6r{ zRZAS(chJ>JzZ<2>{GAH;;$8o@ank|9k9_Ij`;!DE+)XTsRF>Q)tO0xF}{92eX z7hIg;yN+Sk@me+L6q+}uZW!OU$6Hj-n0XBtg?(pAnfeZcax!3t%zOE3bsCy zFP@2&Pc~C*GiM~d%;q6WjejSre)<;Eq0PxB^WNx;l7@xXlxy-LH;ipQ^h_Y|v>tud z@9yI-0`hiCzr3<0hJFKxhr1tu*`?rL;n!z-Sf18_+i5^6 z&`JjZAZ9_}>AV!;;GIjUzaRQNWj4xNji(D{@e%;?-xN1*&T~hye_Z>KlCWhW`RmX- zAV_tR*MsaPU_@MU|7%2SlyOEKs07YCg3Kgc>y15TrhVhnY=Qw=?Oz$3HfTTW7|^I? z=K(Nn6VMA3JGUv3}qR!e)MDL(v2B)QktFs34!I zZ9iNB-iK%X@lK^G;PuYMCi79?@%|c(&?g`aFu*Ap2V}Wqmmg0a-%KnAL`4rkX7~#9 z&*bnRl3mkklEz~WUmHK*&&|RYLiqqGD`onxsEST%oDaUj`Bom5dn2~EvwUem%fOi5 z)(tL@{_4T7HNdwHwNp!H0@zl*XwX4Se{n8=sff<}I0u+kbq$dwo29Z#fOfT4Vbum< zSw6^%u;PcGVIWLcdbU=KpzDYKjP!vY%=X0R@G8 z^(d`I#`_cmuL_&9r207_u#+zvUK zyr91xZ;TfsBj)i$lZoHv*BU+?0M1y{Hy(*pYwiF5p8u{jFnfBexKe|kI?L#2EZ~o6 z$8{BfcL2BP(PqC+keeTF%RK7@C>OhRFSEmF2GA~eWOdG|VaOnW+uXVBY^_z#nIBl* zQeg7sEEJmW32>P(tX%u%6Yg@le`kUC4<+vlG*&9#tKq!Nyc?D^i;Jo;WvGm`8j(NU zzQuf%-A(jc5r|Mwxs6pKXFtQ+6sE)gHjM`4dpo>GH8`^EUj-l@tIYwxQmBJBpi`w> z;1RQzq)0r+S1@5{E^p?$n)ui2j#%!j24V0bfU*L>`3?e>e%U8Lh;LX42=!RB>VBKB zjz>_BQKxNES^oWe$t!n1!S2N&fCeDlh9W&s!{WJBfEj!IpV#@g@S5R&DJW0=eVt8l zV|)K&p`e0et&J3O-J{A~e=kO^PoRNaRg-3L_6|_VJ5#(HJ+(sp-Je;yvd8ee&dTH1 zdH#MC5D>d?dRO}Gzh@~AK`{y_0+_eI>}0zMvSnYjbY6V|oS-}REh;FHcoKm9Y5um4 zNPC)d^DR6_y(yr0rsfBbURl=Eih%5@ivQI|*k`8zZftnDi;oqEb<$>q2FhlB1U3&b z35sKf2m0#hy+1Eh5*zJ-@6UgX9jY1van$G8`OV0nwI1L*;EvMxp}rF&UqnaS;M(ORb0eUXXgj|NO6AoMdc%Bejw#?kk+u(CYwy&I zD6kCFYCNEfnqjjvPc6LJ3w*F0dJN3^*&;WZ+l~YG{LTG<6(Pg$6j0ojN^8Ct$|Fq` z$93kuJ=;DW6m5sl%Hki|b`gpK$pXQjYStAlHO$YSp7%Xg_;_)e0Sv8WbIFJJ zTs)^ZsTzHTH4Vk^zrYR49a0iw1jTm!shEZJ6R7vUOpeF}P8M6vKq}6Cu{G5{;VsLU zU+-r;G>?O8Hl1nB&B7~J1d$c@wwK@WV{fyK8%W)9b_){i)z(+^56o~IxOGN4&aSuA zT@NgzYgvS}kjmxfK$r}M<_EM5FInUHP3$${A-rE%yOA5S)`*d;r;okDODRC_%KseD zkJAl+u!GsJcc8f7%k^No5p|26FhQzs*eUZYWd;cF(gv#rl|sB`?))!b%e@3nWjdUAp5V5OZ8H)PlV!)?u06mf`;+nNPKPw@F&<#$ z&ME^=jc1U-rJ%Dx^EIFCImeozu#jdYhLAmJwyaCwy&q`F30`JH_wB|0Ur#b$Hpl*s zW&6=q13?$OEoe&WkWGXb&~O-`G@p~(an8hZt5*bVCEj0Xi1vs!G!!NrE=pI75zZM~ zRE8QXns&fHrB=!5^Mp}|@xCnuT#IiMTkSt;c!d+O1#4mB;K_n9&XCPcHS3hlL>gIq z^JtT?!sk(@t@mpT7)ct)lrMe-tox!aHdCyMg&@z&*oC*ej)PV)cao$U2q(XJ9ry$v z#F&54lAPi7zxB3ks}WDcW&o}l%Ps^kE3|-KqzgcB<&9AYm&W6zkv}qxgna?P=IYGW z-dA@DxHUu!k0|~=TUiEXuWW@Bl5?knxf=l2yy6n}4e)d}lnVX;F%v%mJ+Mb!Su*`0 z&@6PbNs$>#E9slWDzsqH4g#SoM5B_@#e(k z%jVVz{~g+g%N?@q_>3A#P0^hpc2>^scjq3>0WaZTg3Ooo+f6GUWo|=D>heLJsEF*A zt9BqILV#lja@}~>5hmcg3uNY`ftwqf7JlRX@ApR`I3RpNhm`#l^dy~V5qAl&+R`}@ z(=C8p>qw(GntHE$UE#E8(-5%9DSj*n{5 zx6}o%g>WwE8-@7Q_2H5E!}3N@Q#|+xNJ5E96b|SRDk=Tw=w{#hjO{H);PKD1V?zW+ z|Lhd7Y!3g{hj{;ieuiMJRMNuVf$oLMGz(1O-yE>ZbEO^3rxhh|CkXXlWxOMs(j-LBmlKd=(Jq+I| ziF+P~0xrMk8v0|K;_2Mx+>r=N!fc%R{gX6*XtM~;#wMh>(hil3SAN|3?u992d8)@w zrMa~E($r0ggrZ%1y4b3JuH$hG_+0k9Qe;9EYq?Qk_miq*6JkyDq{~6Y4-hS+|{N|uwOQ9NcH7@q@pI!5A3GwKnmRgwzEK^gJcTyq?0qAZwt6&!YQD)sxiHs#hZ7&x^V4YaikC3DlQ8SXi z;MD82F<=%-mQFBrL3k=01BkTK66}cH%7QQPV9={N6zytC!!TwoB`fGu6{ab<8pc(6#J{Jj2G!GME zM4AO%UOUgc5FNb#&5E99oh^+ySQWy&wM{LS7y&27EJzG}f5jj|+jwLAz`^`tHcaH; zb1#IS?V7;K1Ev z=6itq=DpVD2t)*^0>^~CZmkvoQa&uU7cRTAJS9aa2^e5-Nij?d8qf>A;+w#G)N5uX z=*fz(E&P%2Q10Ck%fjx>%D=K3X@S(yi!%x5L$~eP=5mUu0&DkO28HbhtxeS`8Uby;BcnRctiF_&mOjO&uNigm6S-(P@oWdLm>d*&p?M&7Vuq)~ z2By6Fq<48hl z;fME{f6rcqs`rE20hMk5-qrDPW_*QM)JG|z5--L`P)U9Rfq^lUw@p}+?RNzsfOhwq z1*)FI7PHa^7p40kDSpt)32;a}sdBSce|WeAhgjS~RTZ%Et_Ju#J~A=9Q3+OEGb8X4 zyBE?973s6sUOcCwO6tN(95aP`K3Pp1c?cp4&>^v^k&JDP$3?qpHtBE5#*0>;dmMm- zCgYefdx~!-_yMcNn0KO zms)pIIPtEgmy3?f>Zfw(P4@Li{gp?6VJRJ{QTfdkjKLpq1RxWeE0-*kW6CYyNVLIW`KFv5y&{sscqs;M@eo(_^L8~nyG6ZY?`Np25aOj_tUGRWRlJAwDxBr#Vw=#30 zwVON;p}e|EiI5fuiv%K}baYqme+|PS64kwHy_4jIQ#v8{$ICpKXVIDt*QT;qh+nY5 z_96|>ZM=&)@pHS5KLe0&Mt(+gU4_2NGuih-&MSYVD17RQ$m1_7n@pqY{7MfyeToKJ zq(`96kWQlcH_?U}S`VqUHH-P^uO8hn=B1UTx()K`_a*LYD0%Bp^V^i;CO*9j=ffCMg4)sTJ^xO@%hFNhu?rUWgHY4BQGpU;?iG( z_aJOP560*ihrmYXRyd4)Z`&%Sk(Ra*1|b?I!cRG%IRO^JpLb+QCM;T9NnziV%>$Xi zUVv&90GXyS3fm4*DBLyTHg~jgW>F*zm3_?tQ3yBzj2hYG@4lkG{TkJwId7`%h%tP> z$Y0(L?dnVj`DH+}&nuObbKS-7yeR9OBA5_nd+*Yj-e)s;m4KRqU4d-3H;vHe|~aj_=dP zq=m@_Yi19+>{ZjBID|^fT&if52)YF0WzF4`r1Pd*uA(@{3UwXEVyR79>8;O49_|5~ zbs~s#(tYWQuVXU^?PD8Pxmo|nc;fk&$zSbxQf^EKt=1SL&dtCoCi710cL5EE5T{>=bi?U?`^!%&L z0tj?BMRf4eH=J;4p{6^Y`yL21yaDT^BBSOC2szM{%hskpyGtU-*^%oEeaEYanxP_{ z=oW>-1@aD(I68y`cZ`_@YTR?|Fd9nuNsm%bn zR0`VZRi_|{rBj~2Kky=CVBm>zCNLQCGbQhUj`7}4flc%`04P+`iv(-GJ6V?@mgRnm z{4M3{74}qiPkUUY4HQdtj$+dqZUfruGOX;@RL<0i)C48der}c>)9hE>%h*1uU@Bia z(MNl#a1=`Ud|otgQ)7ngrG5gV%yUt(UP@6CEB^`~!_B-EZt-aesQ+VsPH_*4uiq;J z|MgqTy>ldyG{7*C@JwS`x1F<9=YWH}klQ@6Lgw;4KZFi1Lsp+4$z495DujeY&Z! z29I~n2(~_7{NvE89r>hal$EV4Q8KMY-w&fQI3aW>2FBt;EZJ%i>*@q=DWggWJhMV; zL%)#x^pp04u&dvk$cNTU z#Eb4Gz{nK{t<&BFd{A`Jf~Qe2Ydyt-RS;nAG`}?|Uv78i)9a>kOZXv6DYI5cr#AU$ zjc)~`gh4{R#@9e`i03Ri^x(V16Rp{B{eNa(k+~-rC+*F1~!5GmJ1W4=lJCl9<0x$^hxH z1(^2pvL~NbN_Ko+N+&>?4&I{OGe3H|ccbUw!rYm`YhO}~a+#{tk77ckyeVp{Y<2(g zSkw1(|E*zUJ{0i@?O)xUxz{V7S`H!kUo*Fq#-_3Ut#fs^v@_M+l*meb=kpuZ@m~Ek z(-e!$KL=mTpJ%~8*5~ddifgq%V%wR2l&D>8o0Y!PX%F?blb`A{u>s88D4bcVWneZ>WAN`}KMMp|kVlot%TH7x) z2h3s~9&jsq$(BkZ)M0f?$ElCg{|%6)Cr$-n`OkW)x^g;z6^;9Ti2dN0R$+$|y)XPH zpg_1-RlPNI`bg}P951?Lchqt1#?Z=-mC7%Z(bOXlxMwzmc}?nbMH%5iYt*WMBU{L@ z!`#@z%o*asBMMR~ANLfG)BqEu7h@Gvt+nCL7Y~SQXJO+jafau3BVt>8n!NCcjOgV= zq=3{YH~@sH1^7zDImKzV zc$m$sH}M^jK$wKqfMH0zZRq4|W+EsvI3h#4hz+(LT_W#rk*VR7YbG=$RWTazP4Ri< zY1n19UF#oylS%pd?n_a&B?5Y?YqJF>G%B|o#QE(`^HP6F1=cBi7QZG&l7*~rwjaCX zAFRpsV13YhKon2tsdL#BS!Ma;O; z_{E%a!oegYR_eGHHPPc{AZy}O5IiLyiLEbNQ#6Aj-e})7_|o($^m%)7eyWyN06=~M zF!(Ja7mBe*3E55)AHLKYme^wx*~JI^KQ2=gq#sflWBGXV{(8amF4$(AQo#n>>P$oR zN1petrEOE{21jZA?6f9x>Wqa_b_o+3Y`|3hteZv&o_EdmC7I3JfADb9Z5Y6U`=D1W z0r0n%AI`tB^n!AVd#(Nz+kqjiq#cAz`Ew*gJ9uM5dbbxfK-`D$%P4TtP%V1Wj6}Z#$yC{nx5L)3 zHzMB>K5I7BqAq72o);^RN}2UhGq~k5Vn^xu04EJM-{3GvIBoKuJ@=I|YfkLK5$$e$ za;kPn4195(<1#PnDOYZqwqbiA{JaV?ASAn-wv;&Zn;>LM;JEivB#p&S)UW4QCq%ZR z5N-L!7INU|VEf#hNp?#INEvw5(kxrW{9OIF!ha0sp5VWR^NgI?VI0XaC2Xn(OL<-} z)-c;Q&-_!G0!A96;ZOHg+kD%stiAYz8{klg566;y48{63(eI>=SE;Id?hfeC?|srf z_w_nB{PwKJDz519l3h39On9BM<$(=4Am1xKTv* z?roFrmXK2Pr_vY5HbhIT&bu9%c4oAMg}tajF?jbR@DSDTwA}PBeXmr2_=bMT9J&x652A^ZyXi&c93IP}aXN{IDmxJlQ9c|B63?A2iWz-Q^9^a-fr& zlPNd8BR4{mP$%@z{dUys5aCP{NCuRvU^Scc<&xyE4EB)4cX;1KH0)1m!xeXA3XE{I z^?;9FrGtq`V$pGk)5!MM=UXHCY_p8gW@S_s#@Sv4{&OWNT^?|JZE^Q6w0)Bn2tz*bSkD(k?SxxY_GRDBr6PS%)YZg=?oI)vR0$wF}UGz1S1u| z5@+h}B)Sf9Q1MT-c51@Q>=ywWI^>VVyC9lANfwgF$ARY|+N&g$va6($5|iRnJ_L=cR`5F+K9qTIgkoYy* zel@q)XCQghgvsaMsW8o)pFQW2{Iz9}KXZbs3=i6zEbD21z`4Cy?NT?EL?|wfy=syZ{_GkA{Hwraw zBy6)pM&{EyygWAkHPsIm=YH?>wtZb#kNwZ!RtRXx%T8ooI#Bb#vVs^R*548r zN*$zbKi>T2AFr-Tk!kQ;xrYE*!cl^2gts4xg-Tv`(jwW&$hK%Nm!~?zgYkD9@U_G| z#`Swi*|{-zOKVWz2ay2Fs2&^XKOwdWi_Xw_-f`Y0AIF2Z(jG7nD9c4+#1ndlKe0?4 z8AvK=nz|=bqy8ntFTwB?Gn4?W#JZZ?EoQjZA0pd`yrt*~s#pJy7#&(w>64Upv@hvo zTdPL{jeJ~@c9V9L?4!D8g=t&?eOROaPQj;!{wvZSUXk9E&*3zXTL)`;Qi~; z1o1bg6?z8y;={>$1zN0IM%O6@6n_Lg@?^DUmyo4k7EFt++W6sdSDc}wmI`Ux3~|L> z-WuJCki`irtuRn%;!(E)XYLR`d^9@{QV4|HEc08x$jW9(&@U| z9sTC;rZgcD-jpU+r~KAg*a0Q+FqP2_k%duflYc`tv*h?px1z!PAt7E*XONn{cc?}E zwSUv>FMV)%?KC!&_mrTCSwQnw33MJIrX8Rzf?Ns`+FWM&v5dq0K)Tk= zXNs(LE8_Qw2NsZS-&53ORl}~V^wWymx;-x{pc!$MSBCfm$rr%R>ykfJc=IL)CeN!k zw>78Z>elF<>qB5}y$Y`*e1`BTG*uXppsW<^SJ&al28o3bP9L5m+#Zqiy$v;s3Zvub z=AETm$MClP|B<8(PAq|EYu2R1flAmp)FnZA20?kb^}2T%pMC{lm&9pW4NsK?+3t_i zXc9!^P?&!ouWh65mUJF4MK#Hz^>b)*)q| zC~!l6M6H~+QzAE8TU~k{Eq`lCWIlxiIJAf;rVZ7exwy3AyqyVZ!5Jq?0WqJf4L$|#J2bd^446@P7 zQNUpP7?twp_5CE6hHi8Y4MVi}V9cn^WH#p2@p9du zIL64&0;~%UX%j3}NUJ`Y)o7mBfrj^w8AJfb6ofhI@Gc)R0tl>u$nG8Pct- zM3SnPOD|W+D{fnGgxFZ^W&$Uq)>-E3J~B6=Xq@}H5WrZ8K8V{MbomjA<_?5_H`tX#2OWr$`LEe%xY}OW1kNI?~>jKq?!*#X`SJu<;x!44qe}#Sx5B ze%ZuZ)c^ijOYvbx5A4KWCu(K0aLNxF>~F|{J#q1EvnYmhVH}IgF5sHn#X1@EkjX61 zxKJPy)dI~Tt0s#1Py>fw>GN{E!>c!fk9~KI1#rwF;WO83%v9s>T${f*W&3z}8_HNe z8{UC9!lL<#theBD*vzQm$3QdKHdj8GRZpqeTU$J0YG%7epLoX!=yMO88jm?M>zArp ztW-rtowB%io(>n^N_%c7#dl}1uQ=_>t3_ab2{E-p+x#82M}!pcM<>@0{_Ge~5LU;7 zsgAL>n#FVcoD9l9>_E%?=$|;a?eN$pc*C>c5N=-)YL`xOEW56{TmYSqd-cbj<31}N z9s(Ylw}(#xii?(;=82DBiesWp_A@KT)O+51@*#*G$G}<6!13dNJQEQ&P63Mm)g?DgPe3sJ6bcjLloF+lpM)9JZ-=1OFTBf%K9vgH1 z&#R1YBNBTvRvW_54A&p-BAbT8idTP#77}?KiIh{X>Fzuw=OaKFE;|tpofDzf4XXE7 zdaQVRByexvJp^ON6oohmGu!BS^>z;fmW2$sSgsPQKgHk4fFMmTuE7teeE5a~q7BQ@ zi%olhJiZmuDlD-Aa+|h!&(ln{<`OfWR5>31DsPi!$ljgLQ6#(*W9y-By7xNhvEc@B zA#iluB^ooGgPT>tJ9B-!?G-;xSW^kcYFe#JQ1}a|_cTBF;EOg0ZQiRDIHFo!#b^8{ zDCBw$7@h{Mm~CaCnni8?TizD34m!F;Lkc-dt5usiX*EII;5JV^p5^f=tgvubCmH!}kSguM&?_kjm#6z zW$a--(LZ=_fz<2*NUt5b(&4Kw9RjgX`=l8fUPmHG0k`8Tu5Pde=={M&xz!wzx4WSg z{c@w~r17ufqpQ078~QzKHTD=sI`!zSHKPk46Rqd2K^c-iD^8}$TWcp2K`j%~ORGj= zDd6%9*Yw`zy$mVAu3GVMg@=kHR?XVc@5-}~?_V~=2JDRC^Od;;Y4;};3j1;YYi=_h zK9FI)X5#r-QxUt*u2<(|h~{72KCRLUJ@;r>Z{|YiC(AB0$0ebhsCZG!y1pdMzF!Ro zpEfAI#8y`5W?{sEfR~;TDu*6&K1Z|hRvCyM?F2hDUfmX=*{L!A~`jEUF?$ssjxp{nFRT6=NbFG1iS03Yd z%vx*X>RZd?eJC?08E<$#b%Um&vYf#mw_Ci@9i zQ3I6m1ag;I{MuJK|4)wvrTgY4@7hn@`*0+6ZJ6iAXCags$?5S&X8Jenue8;mNe2`a z_n`Aw_}PG=c7P5o+dpq-eihNQ`ue;BTY_5A_>e_Yr%&SxZ5gNsCd?(^FXasY5VE$q z*`uG%Agv;(V;oTypAPxb7bKpuznkeKm;EEH(Fyr~pOn(4GJj7BKPZhZrSf_>@e|&6)D;29@ zwPwx;m0pSJb%xV(V07)2cs$BU?5tiI>$E42#fK)Uuq%aylO5fCK%Li{JGn#V|3g8W zlK4CqdNVs{@DJPgSNHZ092p#mG+FMy!z7)zG`Rmsn5AI;WcKPHd~@qaRCQ};=3ZIl z&y_TWA(d>A6;e??w(-zD{u7J+O#j^!dE_@Yi}{x6u@`J)htAclH}vY$ua^>kP$G4G z%(sw{9Ihm+Q!KQ9V+~yBO>ObZ{W<$$P4&t%Bl4+++n)qJi;+Xy+Y@QUM4)D$L0m1o znYBA{iywBREg5)y#kuw$5?(4fclZ`$cnv_$-q36JMGy=VX19HvOX4XhFv^i6)gAfx zDN?1)J20eMoZtNLD>v;c`WdfP-Nzs^X|iF8+BK}-c2#dZ(EgtB@)MTZlc2X1rrH>K znf<3DI=X?p;jVthKgHbUv3n+aAN<9C6#ZLCx$OLljP<|2uJ%`kcYq~F1vm(b>F!kE zWq5bhz~bk>JAL=#p4T#1sg5Q4qs5{yBj1ba`!l^09K;_Uo)cScFMqu{hF7MeFUI8( z4w$GcAJ&e;5R>OAqc`!6?wL-py@UMJG~bSO>qGJ2Y}XMYV4jXukoYmV4L`eq)k% z<{ED*#zN}H92;z3t%{2y zUDH5s$0u0d7bXiI5lL`~3ohK0d|43!A7EZTHVBEI4_c6;z_SCqb1&PhjV`#6&ut7FWaq2TXAUeV|$!|d75mLX}ZN>AJnylmXAipzI z4%(qH%u6$UXVI#|cmIaP+D&SsFhaiQ1K}XKC>lLN5=ap+*WgNpupEFXKo5i#*L|KP zA~!$dQ`AM37+sK0H>*F;DZl>u^LZY>^)=T=-&{4?R|UW~hS307^eXssIZy{xnPZU2Q5+RT6e%z?5*y{8(;ZaOtG4z#ID_KswHP)^-PS@Yi_ z{)0;3UIbs?=bybp?)swru8bbp5UOX!f?C~v!~X=hwjg>-k1-4i`AmBwi}t=(w}v$9 z+bxki1NQ;D0Y4oJAcvb zGx!Pw>}QwFN>|~1ut3G{)A}e?p5opnneBU_C*zj70Il=i;gy!c=cS_Wl@^VU+S5_e zF7M+^egK~>SJ(vP#K&I3(P4`p0RKeY_iU=f5a<$U#uU4#EPNLv%QceF#7fBAxONuWA4V;yDoL>1Zb>+Lt^HE1D8gZdH;J2{+mO!Z+=#yvVLnQYS0&JJ(2#|;h7EP9iU#lM!-KPl# z8w9;CHuTf4v3wcag?*@Emk;=}L`b{YvaF1^yZ=Y)|5X)RSMc;QC}YcP(M&hj4Pa!F zt9d@ZCmASqrgN4_{hdWZ$KR49KK*Xy%k*7J4u^#%=3@Q;{wzq*KdpmgsNV_&Hx`^+xiH*Od>KY4vc+oZMWs>nOu75$H|W zg75lcvg^PmQ%HG6AnSs+J0H;A2F(G2f)@NPEh}YJXUolnJN@FF4XqnjRoHqT{_q2H_?9`0(RDO4l^A zxzrxM7JU0UH|{!;C2-Fa)*G0%tI#qo)6!xXsIcxnvQyLIHPLQAReoNkUovSX6&?p3 z(klK&hB|0%9^!ydV_CRLR0+ZU4Lz^#Vi$2{P|1>0bH`2mgzQTJ8Lxs;RP$cw@#IWui z$$2B8eo~+^xQ_Ox7t@T)QrVevC1?vo)|uzIf}2K47STomXg||iSK;Cp?Zxqf=R{#EVOA@W;8IeBq+2c`{8a zv8dv{q5#LP*UC&MTI5nNdpH5nYE655%|zv<=|t$1%=_SLQKvFPEO%4wfL5Q$y>Zi- zZ=4*PwS>>{mzH=yZ|8+{nX&1z6t=V(vtc)%u&YYqG9Lq~eUM@E?&_9zAWysBEm-em zGVPrq_%x&{BLQ43@q#t1OzvG6%5$;=B{5kPo9?CQd({+Y`MK9mI9o)dd0xv4RMxok zcC*p}&6qbY*CVGjEy9H|Lj6u@U;Px?d`hvgchiD;2eC5x3e8kadCWwi|G9z+8UFD= zM$kIewi4?sx{j`B9Pq@1dTs}MjtLChl_u_CEVO;(PrQgbR2OY{Rj>>uWpx0pD{6T! z*{sOx#tmQRz>Loe-EV8}kEG$e#qd%O_yENi07p(a)C-D>NHJtAVG_HzV87bg><{7wAga3m>R5x&O*ViAZpztv_qAIT*PB^0n5T{a9Xsnh6h92{l;I zt3!E}W&}&VONrJsQgHLt)I;wOPm)lBDqAy3pBOE81f#xZsfmJL*g|J5XfC7~sh#%1 zuY^`?gkC_PhlC@kZ1(H39s?(&Qy?i05j28t7_@JhpW-X!59SDlLjSA5(MWs90KyQE zPdRycip=C_>Yuk_Ip~HTv3rV7esi__mL6`st}Bd-){$Rd#R30l1KZ7>8vekl~{1Kz0 z^fVwD4yhgSO*PcAI0Fc?RI20|=^roqvu_0$d+|D)7?=3}-fcY^PFraD^WB?tc7k{T z4VYLTZ4ol>zQ7+8&S>!=2MZq@SHru;u8L3c8l8O@$g(d->We8ROW;#Zq8l$D7LCt< zyY$AKvkzI+=aX{A_X5}D&}wq|Z)|IiUM3v?@dO2KJy>8oJURqrE)FNq#a|m=WYF4v zN|{|qMne8SEC9+}zW0ulsj^mccv;%pZhBOT1mq@(Tt+l(Ewi z#!}b9=z{b2ZAc0)Z|o;-f9Eha_LEJ=^>^h7sE=R_nnv4c>WE zQX6n>{nk#l2JA&p#KpVdLDX^ihX&Za2j$Uw97U>`SCA7P{9_LZ5`b`3qAKoD;F=Cz zZM25H5E_29@knv8wp*8}(?KX(_Or;dPI>d`=evb~O5OgEXsd`WU47gHCNJwk%{`nWqm_%1? z+@#(qDI3s#)pnb`BiPd*wHWIh2u-vJ>I#S}n}(vNJpFqheTgv&z;4#+ANkC7}- z!i|#paX~oKGFq@6vyW7}lF->QEWQZHOYozSZPlo$J}%Cxt2z?-Lv{C0jF>1*-^n;d zeu+5~Os)fQyJLAO}K2s2rkRzpLU*=iiZvmaS_Wnq}!cTt@)&Y=P1&zRs z(gIM=@`CVfAUY=fvJJSfPe<MWX&UV9EE792$b?)p5flXA>Tdli^K}&egh%9u|&Vd9w#RAjIIfLfwXnZ-oKMwb{ zlY|rtbAB))aeMH7N1sbVdX&JCQi5t$dLc|?7@4w=qQ5fhyzLT{zH)2(wF^!9{&tFc zvUD=Ae(IJ31*}W*qnVcR|9rFFL4X+WuW$0^aE$qZNukY%GVe<)&21jsQE9Kik-GQ81lR( z-njML&f^qx_3!Q8L=Asch3S(eq~M#LrrN`lS88r*y6t5r4!uK+(hYyrMQwLv(E$IUH*?j5tB&-PP6 zP`8{1CN3A;ECZ$_9-R0Ldub;|B`BUZEj5|yI?xJkI6D634zzBZ#y)th+`rhe2gq#L>nOD3gO*)q^%Xpa zdhBN~v(sb!I^4i?xGB*Gt(wRiEz?yV^E}UexGV%_UB41PHgz~Co5(W}sP2&R1 zfl}Hy6j;ZbXJ!1_E^%{aP^sc?-L&_Kn>_O z(f;$a>5q@^hIo=J12BO3it7J<_t1QqWED{ITW^zp^B3L6rV+~Nq9ycnV zZa9CS3Uk~z`dh_Qi!RQaoTzkIExarAOE~UuyPg=yaNhs1)nFc?$5D{9kP! zuveI~YX^oL&cxFa+NY28ssh@@$g2x+KX2!b12b9)v!;W^~Z*c<91kJmm z%vNwca;ggk`&mP+MY|$v^#W~ zAQzUlc@Mm>eS`XEWD#W}b7pH395k8Y0-bQNqQ6W6KWwobUEfMMX!2nZEf9b)KPERJ zhu&NVMdVA8yp10yAk<#R1nNP=^`Jg~)rCmMbHqIfEbX%xi#wW0Yo9Xitw#FoW?AP! z9$$WMdn)!S=zP08A zqy59iH$(GhBJ+RP@O3)e{4XFdz4~h`J{|y7KIxEP-GmlpSI|@k#OF+&%*X9K_##hU zb0;i|Z-8h>jOjC#_-ueVXH*^=n#Tkc=(E(mwFSI;{=+9YM$&ZaWr^~__OETQh6)jZ z?>ca-l8B<<_&R7&erfSmw7Y+8|2;t;XLBMIrTy+T`#ilyA*%eClQ$=mA3nD2)PDab zlr!*6w+ziQT15&H5k)G+J7#3e^Z#84SsQl~9(t;lBpDzdv)U z+%+<)VFL%oPkv*y%)Lc2{V~OneS<;LRk_tvwYTOzwd-}BCMkqTG~5q85@VmS20!-~ zQ%7I$a|4c#tPlu^TvI3g;z_bYNb=1yhR}Rzk&DV4NKI3_3+YD}u2G;ZtVlPuqn7%~ z-y=;-{)se)u@sT4J887S-)f>aV0MdJs zG}jC+t?}~ zNsQMu{a|kOnM^e3iNB=IzH1L8A&(%uYf9LZrY7B@$YlLXGX+AJUy>bh65HIA3i!x4Fx{)md}qTybKV zk>(yK4oj$&)E$_8G;+hfj^pPO3Dt(LzJpL`AmaH12C#s3H}Nz7QPzGJ2(}m2q}Q0= zm4_oD-=wBRtVu6DWz^*)Vc7HjN&$~@aZwD+j+k^AMS5)* zgo(AicyiRr-_nJ-h`o1-3ABuqzcv@>J(b<`G!LXg+{X+C~#xy_-O!G3#fYP-tQeKa~OLsW`7*JAq zGpG!=53ix;zzxHm&^f$Y2CTkj%LcfP7H zGy`#Ey^e2Vzxit747lM|#z1FDn;Epr^)AB=*5t#p0w6j1WK5j*LN68HUH+!P8F+j; z0gAYBZq{4k;|NtB0xS4f*n{RQe~XL(k8fHmR<&k<+_j&Ol4d{65Bgy;YRg|OJGqZ6 zGB)@i=(%HwV(U#bmefVWP?*nCK)MvwmAumW9*kxBCp*m39J84RtW#abo^Y6qhK|=e z3AHYzvWD4BWTm?|GpuNPC9{m0?VFB^g#%qM#w z#ZIqoVKNua7bkjHZ1k<9_U0RT(zkZRqhC#=C_FUdX#*+q*GUuhes8_Ua8T{P(n@2N zuqbtZ$C-xdhvD!{1eg#s^!&S4D#mN#mpS4c)F+7RN7tw1kBW^Ve-2*?RouzqRgWJu zKPUzUgus{Idf@!huht+}N)08>3JrIYz{|N-U9vs0!TBn_*-C^)zLug0a6cFyK>Y^d zzj|EHofwXRddg<#;?9Eu!@F(BHPn<$)_lv!Z^o?1KVl$_FMRhA>;HXN0n;}T=}Pg^ z2_68nSu8U@g^vSYCg1gDPkG1tjlTT8@jWb`Fxm`wT7J4tq%l?zEJ?7?2B=jr9qs$cYn!hwo&KYB#t9uL479Q ztch;x+?7?MMROW+BXv}SP1QiwusUYb-Px}6_)?=4cS$<#NMZU@&b0g5fE!6-I{23A zmi~T1pZ|QqCNL~u*LYq|0-nvK@;7(Y-$Jl5?-`6)ZWF&vbhIaMIMR6X@5}tEt5bFd=WNcJvcL1}{V9QDE6&V#}ym=^7cDd`K+rrg8iZ-%M?fujH4L zGPsoTnS<_>iD7Sll+^)v5!Le!aO{;-xhgyCL0+Inth;F;4H-#2;`}}Tyzj$0t1n;x zCAht>!P9rTTwOkR1ry5f4E4f(wc9NeX2O)fhuwT!GHJvNYjS&fm6)9PIQKsjmr{po zzM?l_M$+e;BXmKD%9>RlMGD13(z;Ki(DnP6L;Xgr*iRZcqukB$4S|K z3**yw#sutVzzOaL-UN~>u4Kn|WqrYEZ@Nj-EH42%jmI`{@VF|Qcx{|z9OJcz^LTdI z$d(|%sLH~_%Hr2&IL>a*gF7wHf@`&>PIC*TdmI92XSA5dK-2y%3(pYW{Z|P2f84%s z=3>_>2hm@c9epFoU8HFCJ*E!5c!oX#f~Hh-p&J9KdZ|&&L*xTwGe|@Joe3B`O7G_h z@M^ghek!SfKb#U3ri}PG32J3)#+NQkRfz8;tbR^SlGeR1QEs}&GeWO!SlE|>krj~E zUQ5tvDw%2?@AsI630tP8BXds0~?EG1xSjXNn-kSnoyO9qy zz)YwnSX;1b6#N;l0qy|vY{}j|Jo_(%8jx-~3;^@&!(g&CH+&d?-BL=9+8CPE0S`em zc4KPg5y>n-?}_>Le>Kx8FU}sh`)qt~n)2<)i?O=RLr^u!z+*xf)MSs$U$MWFcbwH{ zqMxds$E)P;DaKNZ*zW87Ok?LGXkT(+GiQV~y$bDc&1~KJ?{`@H8Gm6~EslR6boGHm;M9{B<`@(@SOqgwSXU>mVw*{%8C1DiAymwkLBC6fCH(V96-(8-la#B;0T6d z?KKfiYN4c0@>RB;cCCqbwA-vYm!c zc6Z(sxI(Ld&ULIL;Mq^yYa1631R^bN?^I&nQ;y4$?kUP?Yr}RCkChBA2GP;-; z^sDh$w=qtxxhLv(YqQ|?jVpuFJAHIC7W9XhD_YhRM^=rM|NXFM*#U2|P|9OrS0XH3lFL{)mq z@yo*~)8JkFbZM+&I=8dtJL_5&<@oBaiw3)vL-gz0LIXf369u=QlZWj+>w?-CEbQQi zm+PqxfOq(GdlZky|4>1s?yntk*JXbgkK#)|Z@Ye8Kj9BlE6br*K#hbIry0IZ3e5~@A}hCH|2gjk&WgoF4k00h%-8yHyy z*IV@furL(w5Vqc~0gJ7?bxY}~b!lAbquCKPxzx@!yhC&F$acMYTmb)iPuJ<@i6@5L zjs&bU_A_q)$mA^LYeX^%hj-Uh9`OYno6GQ>vx3##H1EH+g&KHqGXuz5hba1`@~G!w zZbFloaoLHr=298<%}%2lO4snt)e+`Dc+GbbZ4(g03b1Ph|eQ zR%7ro=-RL$PhJ(3LEws}ZT*B3i4 ziOc_f-G14+nFuJ)ZN$7Poyd${J>$Lry6A@4rC+>sHcTaI9diMxyP5EkVy%nS9M*;ay<-UOhYRmVQ!iDD~vOAb)QlJSR72 zR$$`m#1VvHIfU&Xe|{AokFDCO6r!W*t+R6MynzMEbmceGZ9z@IiK&|Qhdx?Odi&ud zuMarea@QTuq|ShI=`34$UoQ>s_0{L`TTwAq4s9i-*JbCI3_p@yqLx^U`mm*ken}6W z3thPK56Z!WTA%-Y%Fg@)9>93KG&lL^+)+N^uyd+D*f9l~(|%*Hhn9AadJ@b-D5UoL z%-}#X&j8+XUzP*T5W~q@rJBB0vRNWjFEXyZN;ivQZyt{rYP$5v*r}VV`zAUiG^>qD zYE9h)vWzIH-|oCIV)&t*0LbL=5L)>?FsdoBkbobyG0=|%HA0 zXLu4k;g4Iyrqjrk{^(*CN(vO8-Wiuk!`jD1R704bN4a19 z)I0Ya-U*?-PQBEK-YeO~Snd&yJnd60z_~7=0*-CJC2RzHN^Z_{!^BgU!n`8^BoQ*o z(sx>*)5Ocfvp(C6u>B1NiYqcE1PGmo3n;d8cPZYuxeiRihNZ}u5dpiV2x+_1KJk8< zlr9kp4YQPk5b1DF{Pc0eogbD1!7iNrez>lH{Gy)mZwlZtSic+mz%N#FcaHv$Z1I%d zFOyJ)4?5kOy=U>Xd%M+ZYX1ZSCaB#MXZR_#etZ&x1%fbbW$e9QfNQseVMe2C!x3G; zMsQ_93yaUzS6XL|^!j)041?c*vc;b;f68b*ZS4ET0Y>hk-dZB%W4o{lE7lz0dJSy) zk9#G)TZ$4-t@33lNgX)X2|iY0%2h*367J!Uy=R)a$&NKs-UsAArI@J27w%f=W5&1$ zP7|zUF983dl3$Ain`~F3hx;DaN(DzxXeBb#GrR(=6!&`+)!#&>dM9Nd#vc$F0}IAu z&X;H=%wwKjU0bT>nGo%4M^TPCBX)4h;rvt{B-onjrBg~FNL7Qm3Dviq8o_Zjzgf{< zS+#v1yLX!p>LBvbA0lyE?B7#W6Fa0Qf#Up@%OckkOt_Rp_I| z8n)3=t^f4b#UdM`gy^GRAc6i(bW!@sN48DpP*ud6KCu}e>yMV0r=mS1>ZNw^5pRMB zZ)W}X`hB27fTe=ddG*#AawT@D;8@2wrEjWPkRS%cKPKG~G8$AO5ln#=u)fVBaa&lT zaXoinH9rs%bYNX@Gd1vk0+ZG)l(dJqxYq+pgEtQm&uI4#ybcRm|6tKBgV){!XFR5XceI7%-2wJCBmNhr2*i^PuU;qzJ(E_|2V%+I0NhP7uAnh zjJbQ>r9Vp#!s{@s-LK{W`1N29d~f#ItUe_Uj+88vba3ldnH=*LykNFXXSjFBzt;Vi z_B=miEpl4Dq)PZUC=kR}W@4x#ZNI#l)-~`*Qdv4)@;+q#Lr|C#E;`8*bc1F*emwh= z*=7Q6{F5=GGIo27mHed(?z4G$2fYeB4UCEmWV<2} z3E#_?Ja1UVyw5~8xE4^ojj47xm7!zICTu4kUB60L*Rg5tbKhPRiVe%6L|IVpbq?~O zznxbJRU16^V0>BcQ9!EBX1*DU0&m_KssG(tt%Us7pGquQZBphKk-3;9Q?2l9MP%kQ?W4bh-LqxWw>MUsYU1%pGnHadCNKn74iC z)2^K#p@?eF@aj~d?6Kytq?!rn$YknHjHDhWvG=t{#ex@j9ILATE|CIvnI?@Zo_`i< z)}~Ss>tsGB&91eW*!IAoqOn<^h^-b>fSE~n^d}c0_fhjdGqp0(dIuS!%m&Bj>WlQ> z!B5zfsBQL8P`1)m-imgUN++S$LRAw4a)9x6kH5gJ%s~G1jQK&$_5(Zax3&_(DYG1I zGw!Qa!83ns@YHA(Cg;<0ALNObD2lD2sT8#V=G}|BBX)8{lN+veB@r;0)nWdqfthsW z(p@*Z32T{x2-3P#7NzjH0-2HJ`%9doRfq{S?hlnhX}6PzEnlJAGbhDuis(g)^xEvp zpKeCs7w6B3CE37;=@Pv1`2j#JSqUC@ejAz6u4H!qJ!7;QB^h_r8Fi~aK=SoGLmWHX zf3pA`QChJL_1dJTeSi+xtX*PEH8HkFc)TJ<7u<#|={0IBMwD}j{ zfZBotJ871rcnB|Yb~f<)dTPvl1;Y=!;dD>##UK^YB+!qWmV80l);%c(R(_P4$Oe$5 zY7&00^IU>y)pr2PZyxUsqf+Q#v_>s-|8a0b>#E{lVsj3seTPbVCt6I9c}!>%@Xn2LrJ$KVO&fF$c*;;=k7Ea_kG~!*l8YL;9$=H-M|L=IupUr3woBxDxCVVzQKt-7%CSp1JNZZw-heC6tid zt^Y9cOQ*S!!umeGRjb}`Wz|L_FQ-X5^k(K1gR^QKza>{yLVSC@twvzfJ-~r=J>WJ* z>?N*%aL;mSD{Nw=Kk;jmc;Y3tDZ9@qsckLKS7@7dTPUl-A9eC`p%HD7oo6RG;Fr9HwBFbAYh48cRBar};zrsOCl)4A$STW>dZB zf=b7H9QmD}f88smzjNwXTGKU5F;dTOF(+6caN5p-TUn)8NIvegAkdEmY*~^ zC7$J91K(mSBoa|uL{Wymx9SDqy*OHhh>3*+wj+#I;n}P2JwIkd0Sx$VZYLKZ`}AZPvpna$R<6`DvUvm(kcxO4 zXl_nC%Kx6t|8%>mtkPa6xSDLg6iUmFN{zY4q>kYloZ`)+r0-AVg4b@?%w47;)JLnB zWI3G!MB(Frt3IYQWse$P)K4vS(WdHD80k@VV82rAe!m&1Dl|M`gb39A_J+v9!G+>g zK+oj&&y)Wc%8tG68EBB^XvQ+JlM3|Q}5DwcQ0xKaBjav zi?d1UbgM0Ef-z|n5k&I+xHm-SiZv|VUUZ`#Ev0bQ6dLxllRc_A){nBBk`$+NYpK=x zp+>^Swr$0^?zy)ZD>iU#aJ>gktgKU?yUj`&daSkVb3WkN`x|CF#1=%iWZRJ#!l7h5 zmrq>h3>g`8<_v|es=G?ltiP#5;y!@|+42qaYV_V~zfPp9^R2;fi7-hohh&AiXL8Gn zy*#UZ$_o$-l!Wv(`gf;cuWx^O52bi~dsDfxBnUNV$GQbai>}@yNnh(n;NllXCf~%g zzX$YoSdDxi#Aa)R2c@9u$EB^7#&wG#e7nCFJ=!185Ru-by7-ekALHd&zL>Cm>&vZJ z@U-?yrtT5>VRk!O+Pp~g);kjYZX!+x`FU&Vd5@#A`A@xegm>JGs8kjjZ`I+s)jeIa z>ryWQBn6WwKY5fa9^(2ZiEY~H13TNp3}KAaMI zMMGYt`R_{Fy4T`+Y|A*=m;r&S4J1?#A}MI!KPLiP+#)zNFS8Tiuliv0aJgqff)p~F zZ~9Eo<>fQ_wiL%(P^<~Xppp%38Tk;wul|n8kxN7CDO^ich1c>4b7Rv-=ZJ3k&)sk1@fzrMhMl z#J2?3iC*gu253@Jz~YAzA(HK)&`WCf;hE9ZY1MjE2axFQVC`c9yH697G!I8e0_r=O ziR<=)_aQ3;ch=XpUXz+w<`q3VnK2(f*a5xgQg3&abvIj8ZS(Zu+?W$qzlp)Q5 z>5PFo0CJzZz@D7N}fj{3!O5jK37E!VCN`+takW^s;#9k5bE}w%zWA6E;UZ>sOwH?yI#7&Pu zv#kDT396?|zECjxPzT*ra~^kC+?7D8Rg%J&B(uQDOA&?aW4D@0>`p-5gIWq+X5sNR zsDvN4d`Nv}@07spD(NHf&MOaWW9nM%tAll{g|PJ=;r0X5^4l{~YJYrl7jXL9L!mO< z6G`p^5}53P;*r8qPGeSXK82#SiJkVDyT3{aM`eg4T_UpnAaR=6fefF%J!T)YM@p&> zM*mzv5~os1x4YPZCqM$0&Voe-pN_s^34c)pFJh!EhpI?(SwailBRzY~1i6>dbnYDtRu&4+T@ERURzXI5m+DCfKQ1SU@ zsJOHylD)N=85dN`QbrvurotZQ3L|-5LeSp|T_Vaa`kAu5sDHhlKEGB`0{?z(`rE7P zm}2%7>FCbn6F&2@e`a)KIxMv5NeeBa5Hg(mP5Ynkz{sxOC8Xp9#eJ*s@m2b0bM>TW zh-Rh|*BhD$8ViSUZm)HJw>Cz1#;LbXUzN(J2dB(oGQo-ctVM2xpW1Wo5xNcpG81tr zN|-sBQM&rK+n15fY3a+%{JNA8F(U*K3cK#U{#t`-i+YjaRa;%4HKTXfBTBz;;n8UZ zcY+0LNWR&<&R(~Lv=b4JWfq~t8`a>BCP`CRM48Yj2=CDycZo!#?#|c;G@O!7q!-4? zDXjQpIE!Vby?r!tu&d)tzPaAV+{?PUba%!5Fa^^Oc8;p-HSgrZU8QG*3NP-bg=$jC zr}i!xy+WHyn>-d1U9P{vP2fz@NsRg-4QY(UMO@=(;cVWfQa}J7fo2tu# zH|XM_+v)%-`!m}Je1UdZFLU_Z-#GnYU{8FM!<^0>u>8~L85s3AOkSS=)TkrMifB?& zX#wb-wySLlthQm`E#2e6)K8?UqLTMcB7!5wL!V#EucQlHF4pu*bOYxhdyP+A@1)hw z4a?U|L7UH6$docmG=k?|WU#1TtUQ(DwNLj{aqJE6H_N*0P^^@<$d=-%?&$954=h#I z!}(8mgfd}N4B`UO`K83VW~Uk%f$@@ccl3-Q<0)Eih03G&pnJ?&65jdAdCRXqc!pM- z;(V_oy*#;#gG3{(LmxGoR|ZR}^)V}bs+G!^t~-Y~LtY-#yo-k1Ij{qk#3lRK^hbW4 zwY-83p?VySJ)k>M(cd5}O)gG+k2kRT3RyfY3_`>Y5`Nviu~!tLsk?YgF6CG5f4*7j zw|bKTt7yWlEhbpAmYMca{4$d|&exhcf`G}-LA7eneQ%8`*DV%dRghlmD85z}vJr@~ zESRbqZ*ZC7ZnZOeE~`7b)A2n>&m$rzpXUY&w*%^jy^R>A1@VX151d{RVJ*02;PX9~ z^8+2FN5e}WNFu|;gSK;j?xZTHFP8I0euS}=C7R_2V#0z=*xHk}U}349{ODPUclWFk zQu$-sPvlumI09c5I!0>cRtRY;Cnn#6V)f)WqO6m<4vOL(?Hjr(pB?{`@hs2&#JJFV zB@jJw)joEn{p>i!EdM-O;cLBG-pe$f^p8mjd3rGuPvdxp1u zKN(+qradz%q(b-CpS}=}-pBp6Y3T$F0gPZ*5F)<^R3>EtVX_o_Y^J=Ts0zTZcDZ^p z&=W?2-B_BqN}~KeSy5@pNl*HuO7{=h}l>=D$tA;;B$fc?K~xrk$(LNF z5pg3oC;}5Rk(mR10@uB2Gi^^1${4|+ug>-b0bX)Q&BkU{utsDfUj37aE>YXPD{HXh zT@oHEAz{!aLK@|{ktCDOlQt~D@&28jiCmF0s_?_gIJ%e&1 z=EgmSyEh}nCbY&5FOyhiL_A^C3G>v|KzjyY?scjUUN_H3qkegBqxSlNI6V&4@6($Q zQfr=pR+Vf0`0{?&r<%$SmHF<&_&cZc%W1EC>H;|(es90dv%;3++Mi7P#kk$`PIhM= zt4+&IIgU5d6zCi#zEz3=qAlq;lQ1bt^^N{511hd@YZgP}{gP5iR@LQ9ctj>#Acl+) z8=|B9pir2`vnHrwL3&M<_Cq^iATwF1KW6BMC%W~qg}_Kc8)%&f=D%hszl^AbN1rW$_g~YyH{!AFH-0+bH3(k+yHq zvG0|h6vhNhM{%^NK;DwBcrIW(tfoIrxKk#fzU$4XwyjLqB+v`0uV)EN<{PCp@E?@< zmSSPzwtFGCh&aL+f{DQrf6ekeF9UkEM8`#|7d}L2wJjo*$-TVYU3Dnm-w_%fILgn|pUAQ)1gc6z`i*Qo9Hx#?y7Nm`r^Zg1VYK z{6&za$HW;pC-EXY6HPW6q(y-7)W)^-k{;-h>IUFOpDR(khCJ4#eMts$r_fvfQSiGM z2TkKc`_4495~d(>PQTtc0O*vxsFR*_4`_KWu)$a-iF0G zLhNpUQnL2=eO7qdiM(8TWcP7H3&P=>tPJ|&(?CMkP%XK?FIWUCX?6>l+XUa|KUw*d zRVq9Vu^dE@U{OCjFH=k(54bmrc)me6xL0WZJ;rc`UqL5TUd>2pSWInU`P8qqi6z5) zDyDoh9@M*VskVam^#ZPhqbtL&ev9P3S)dH+RHBOOF zvq`W#C(S4S(0-JCMvKJ z(W6$lh6;57c}9SSlIT;zHWkn*a13O_-Jj9irjEJ!}B zkIu{G;}n|*d7`MHlFMv^%pjgetRBgVSGl_;#+xg}Xm7^236F{h^|;Pum=V}sA*%{l zoh41M+a?`02hCZGQIUy|;F{B8Wp<_jSF~m&k!Zl7Ni52_r%_(w3)!_2W&lN~`<%JcRePlq zLPD1w4r9Lav)8sg7}p;ZJPcio>?SBHS0mZb!t&xm!Z6;OJ?6eLE$vd5)zp{Jj{S0& z$7jDChYCw#_CCOm(}hDg$PfFUgqa0=d|B{wClTqjzp-c3N~6w~R{(3e^6iEVy%nb#gWpqhDg2VZKSYS;3koy5&Z zyfU+!boMkpj8WFjFMu*-%Sxg2=vUXxOt&w}N8p@5pFJ>-zv)otPBk}7p1(RCrifkk z9UuspYfZfMkN5jDq=Pc$B&F^0z2gR1!9+1(oYY#ao|8kTq?GPIiz;Gj%ZxsGKNO5? zOCD#!v@@_U#*iSwkuR`o8Rvh%qO4hTC&{#ZPn4(T^q?OxK_Z!I!Pe!StjcW;%-Qs&-$6qn|*k>h@S1nqD8l!Ue)Zc8S9E za6>_sho(=Kg09}C`%Vs)&kE&0+zO@wqs~Y6X-Bd1ACp;mHtJVOR`_tbGu}aa+I*rZ z<&U;Q_UMbV6(D=ccRLi!M?!$#wHn3`=?W8yZ3y2n_8)*V%6BdCAIj? ztab;~@uu~;dq9#EMCv>~Qp(@YdaqCdjw-E;v`=Dr>=vzzEpPu zd2*Loz3gKi{ALz+s}j@KOC0{T?F_kHRHSK*M;UcsLryekoo0N@%}4k=Xsf((vgz9w z5*~TRm&i4B$eoUIKY-{Ti8~Kw_2LPlaC*(M`6zfqqPBG|Is=tT9iDzk4nMhSv*ZbM zn&fcAw-Ts#9l*xb?Xec4E3-VM-i`-Bq_btqo2DmIdKH_^C%J*X+$s`b17+`cQcBLn z5=#Ql8_TYmcvUJ~-f8k|4_W22uT&TPcnm1_NH6XM(`)xZT{PCaA$RITn}vE!oAyno zhjFdB*e7?@$;5+;@Jko@T9hRXAFeIoMUWU?bys%EbzmNO4su8?1#*YpVA}}Q=@_hT zqDz;5f-*M@I4;|bcf5mfk!5=!Qy9D53Z+)v((rjzrtiT~{srm$q{wRPMhY5iKxo<; zrOittS)^Ao9)mF5tLX^iIleOLnZjqY(kw}W_PYoR6^-&c=r@O9A zjZ9o%^OZhXW9jn$A{`ouf2YRfjXQpc*=}X*{$piZs&Whly9rd$d4ZtmB*0I*L`MAX zG4dHn8%KdQo$EKy@02yYJgJ@vOuf_18jb21w8U6hK`^#|V$)fD2G9+IUFLvx(1CKz zFDX8UiTs(S%%@NxH{Glrhj&*pph)IL)LTGhZG^*1dWc2xHr2gI$8?Ec$l<=k_w_Qx z06*XddKNYHwK+UJcIqYw1a}32$SXUEH59P5B6{@KV~CvxI%X&Ty3hw=VTf`1Iio~+%r+zLmzN?=1j|g+|S>#zRUNmZ-)S}mD{K{fBrYK2l6dG_rtD6z5$^~rOayv z;~W9UR<#bJ0(5^K{^xkzhm#!bTCU|U?0lr=C;r$$a^QTnH=b8^2Y(ULdeF!C08Q&{ zRxrpE19oItf>PBCk zFt_b~Pv$fr*$mp;W&}HGGuM#~LqZuXzgFfyUts*t5%@UheP81gw;^N7y@RGrUs{+E z4^w*JT?#?eYkaniUWv`u1wim#Q17q0voV?bfo^+Xk;9@98@}6qKv8tid#CC5O-UB& zXEJ-E=DStiRt+&iaI^@Bx+5TaEB0Y^{yai`0R()ggg^=?lJtgQ6B6KB7fl> zAPhH6pASUOOPB0bKz2>Zai?D_58o}P*lROj&b~;$I7-L(Xv^|N2TSHx3`nkWmR*2@ z6pRf$kSH7MHBxm$@1q8TVqS877ald@es>I%$*c2tGm+#F-NX2bT0f%`joPIG?~BzM z_;%Cr9ia`F5ryl5kt|tMnzxGB0{4}iE@ z2MOYfbH#R_d~~mY%e#nQr}#I)XyFQ-P7vWj25cGwo_JlE3}dwsuKuzWPSV{-r|l7W z3Yv~VeCE&;cm&gk#i7Hp7;E@_*%`~aJ;ws|qrHk7HyS#EfxDOvYw?`JORx0hivNof4MQ<$5JA2` z7`z}}eNnnv!{7qMU4|uqsc$Vd1j4bC>&1;A3)H-mT4mNFKkmlbf47-feN%t@(3D50 zhTKJjCZbqZW7}E_+^Vad$Z?4Qo+a1auBQ)>nd%uEJJArV!jEvpnkaB&`j@C>{Xr;G z#UHrrd!l6g^O%EuC2t6M^Aj%Nt)g#M)nxXbzroC&<$Gt=j`-=G%((35)w%oQJMV@p z1_V3uUO9t=85c{&(O=6)%nu?fh2M^0zeP>1_+BqX-1BX}fsS&!u@v|i@kI@?2j_~| z3?>8!Mb)s*(*EG`PJ&T95HcQYIoP6}`=+DFA zyXJ#5QWwchO*fjFYb`u!Sxkw0m2 zt0Y6?l{%_VzqCb(-P&4*42rEc5O5vNZfOn@d#Zbe@=jvYgj?f-w-Livn+64Ebi3# z(d;N4&$aa`|%D<&chXg91D7YTl?J(G2SCxrD-kO#N=*`awWU z{kMJTs>!~5qp4a^ZkP7l0_3CxpGmG{BblFLA;bSa3e8}7C_iz-0VNDO{#2uSlFTCN ztmyd7p9Ri*5-AeuPgiWV1~3mAS&uXxAg#68m$^oMLmW^3Tc$Rlq{Fu&!hZ{%F+Xnr zt#O9GH)6(!TOa7)A$<)cAl<~J@N;gT$!y2 zJTyDe!q$KCE&qNE_zF)rzLp}keNsFF;^+GKe(fFjPSQB1gVXka)YJKj9j_KJpbiDi z;rw492YCl^1=dvY-0>y&2h9KWQ7iC$)C9_W(kPa<*t1^epL;rj+Ns-{{0c1(PxjPS zzuyLmUSd=D-8Fv3CF=h+rm>~4J~1zwEcq4E_vxTN-+@o2yB0I>VeUWGwUb@3%SdEkIM|KlaSv$8()diHxOobklW_{pA@I zY<{uGEqB`<6q@J3HS6ZM*>pf3DUEx)4SvRkLwv}#jbIzEnGb!$U{s|Go+6?5$8@tm z>GpC6UUzFW7z=0+{6;>)bd1Je$r9->MXl98^}l^1aRQ@vU*HRxWB(s8OKiJ>uc6n2 zQ4zWXLu5j{YKey76)<4<_Y7~kO|ug|H%bjeVhUb`9da$8d2Xz%C<~hBw`l~Ug73IjN1HurctGyK}G%zl}%MlWAVSLyP^Ry!C`l- zIV%@5Q_#l>K0u+YLab=V&fKfW9H6aov^VID-tl@BC3m)> z*1>jh*ky_&rcbz+4u0~@pmg}A#7g-5n`RzhliFW;cQfHhpNe9cMwarwhmr~F|9iz; zzWeGJJQs@PzSSxq%b%N!lC+iEeiP2$%?N1lux-H`l<9ncD;v&NJ*wFO9$wm~Aehb1 z3IdaQqp4;Q5+Q$49{b5Nr@4oq@}5{?01_O-n2UXfi@5V!1HZw~bv%z>j~%yoIN){V zp|Lt}36>PxU%JO;FMPkk6xbf0rb`$uEhjq&z)vQ}bJOwv;|_fH5L1P_IFh@_)5XJ5 zFyKKmnhg@vPnl2WGvec$cXMKlCbwr+$w%dOGpTc8X7`X_>6m{wcyT(|hA18eNls(i zsvnA9`ftRvNF<(vET`vou2|C-kDcThL)2D~V9HX(%=v!pO4Sj*@7jS+oSK5DzOrys z?PP%`53{i>;4p-4`e39w5VAXo5v~qet=^r_)PX_(Pe!Ck#2gJCoR94G2eoY3To$f%FYxHdj=>su zW>6F6OD!VyClC&of@8yPQ4tQOw}7>F4kCMmh+azk`vD8xH!?2adlm~?d7j!$&6J~G z5lQ$vD?j)?>Iv({6pfFE<1M@G(>#}-kPwmF9b5%{jkr!y$kf*aN`Yd{#+sk+!eM9N zPv6Ejn)CvWbCd0ps+|6efIHhK`&i^L=!>LGJ~Wm9v8E94C>7gN*drQcn(s-H1sDlI zaMd&evV=SEMn8Qg{Vwcgq;y5GV@5$|V1vLiSD>aS4zH#C_ct!WKOt5r{vS6Q>Z|!X zi@z=6jiN*F%T@9-0-`kV93d`Upe(5Dqk5nRPPCDkaS-*tv==F{oUo|phfigak9uYj zt}g9@$|VE%;|QF|VA+{-`iUX#{QzqLN}Gwm4Yau8is8-6AGp(aJfL^RCv8eoDTy*} zwO;-m*ai3&Cn2D(AmmH32q@k9Ds}Za%wK{ovtR{y2n^Ch%uGO}EW6%W|dx~QR@>k#yF{A!Tch1!pe^y{`U(t{w_k3eeriSILG z4K?udb0Ax56ft!ePLLrMLz^4{r&drNWv3ZaB68p6K68S(gnwD7C3 z?+AucZTKrGo1rbhwphn^0!~ZPya<1?-mDxBU$VvP=sHF(n@H;_ZuIR?E%#5@&!(x?wOCk#)4CFFJ_)MeNYbkmiN zRa*$olkwhM-|v2bdPoL15tUooju$^)iF5^R9TP_&qgP@i|w_$l}& z(^`4RdUsf}iDut&0Q`&!c_uH&z*fgUSIQ_kvl@WvHZfgYy9(h!x(0}Q3k5bjsN0%l zNFpWkTQ!(TGLgLODH8az3fhgY&DtP}aGH=}$n`L0vj{m#V0GFcHQLDDlly4RESBXrlnw(ZT z;G{Ntfb4i$3F*|HLKMpo=})6xii>ZwYN}`dhqAX0in@K@zn5BSX^?ISl@z37u@I?6 zq@__*M5LrkB$O5qkXl5fySta}t_A6k?tZTIJD<<@`OW<1nR))Z$NQdp@cq7cU*~z8 z$N4(^eY^wiod`rl^c{G-+^y{qo8ps|mW)#6MD+bpgPRjF-AMXu^by3`UV~E=Av&+B zE`O+L{Kuxjie>8KR~dkuDRxk`E&x3j&!x-?$hZ|<#n(})ilFGcfobfYoY~~ErAbf()QeG6{zqw*0Z;zy1)ih$0ZWDtt zDRg5heV5-jflK5J4Jqy}Vc7c<3$)L1VZ|$N=rf3C!_7_Za7x^X&Zj88yx8wk-${zO z%geWjgM=$Co4CjqpN5;*df@z_@A6v$(len?7x!!XW&>wHi-5) zb$HNY9~fM6xVdi_pNSXn-}_I}te&Zm^J+@KHyw!&By!3y`rxNY4<6&Q98#u6Id4&ZaEkw4gtKx|H7iI86ZNZN=g6)k2_I*D(=Xiwjl_>&@+J3#StNfc z#U%zfk)BBQkumV_3R4U#^}&Ye-#B0RUYvy`Qp4yZ{j1y-MHr)^Q&f_G+t76i1kOHd zqu&1}RrWOMe*&VxjX=l$k3i)X`oj}z*lmMscN1a4M?+WY^MQ}i7Is_q3DSu4bY4gU z&aj9*9Pj6j#ot=J+(R0AF5l2$MGadq?Tn05yhWy=+c(|wfqirpBlVk+JsPiTQN z*3j`+$-sZfJJy{`=$>2uvejNn>>l2u5j-L4t54!m$UD2AXo?LdupA*QEeALfIBxfq zAv+CnHfL&@%sywJ$~6cyRIg6(o`-RC!DlSlazElZy5K3-q?K}ol;)Civ2kr4PS_15 z)dad+TRsSC=1`c*&a1i&=rP0ecN>QOrjVZ93B!FM%@;FP>^xO7bjbMUNeMIDrrb|R zY;V9bZ@(6{)CFgb>N6S)zQtB~y+XJC5=okK$4U$;-0~MJt7hhBc^$3dpJN&SPznJ* zfkV60pzlZ6GuVU5Fv*gkFuKLF*AoPITW+Kyg$d4Y+-|JqtDlqlQf=F8rje=N*yCDh zBj*xxm18T>s!z_bmX+Q#%V1m>TQ7W z`YEz`EdcN5K#g29VpIMP^RgObmEJL(rW*mUWnLC%?=WOJ5dmS;Xuugj2HnNp@8~ z9pV@^_-U-3nZ3kZWk*o(-M{$LB-PU>suS4Ro%EydfV)yL4f~N!Nw0DGtRx4DnNXR4 z0HRo3_Okyq{xCiHbhW#YuP28C>GtJg9!+A1q!_{{t7b}&cijmPv0;XA6R3`|qg zaVJSZVfGbY=KsGpw-~)lBe3XCH=8^79LteqLgs~^#2>Br2ij3xt&B7W3)bDr%;C7# zpMObQ>@-I@<$j>3aq^KtGBZ^l3@Q)ZtBJACIZDVEVyp>v30v`H@GA~)1MD723PyVJ z#az;|vZqT{&a1b1c=_ZDGhGg_*G(82pk|+PgCX8kMMU!0d)N;58!etpTkC-D=87MG z3BgQj&yBnX-PZoN`dA#Yu<}H6c3l?Vy(s`Tm;oQoDZckOZ}&yTev` zzTf>Gcb!=E7am-3Ep&J&_u1Zkia?pQ@!)hx~# zD=Ps`5+=#OY#=p2d61T}m{juoyWLe%`ZSM+084Gf_U1ESLJHl*eXh;J?)I1OwF(SM zs?w85g+%9BZ*srJU6%}%F()87AY2hyya|g?<;#d}sfz0y6k#qY$T@o8L@?Q~+|9>r_eV$6U_y6(tHW4Y1<3*yg z9guwEZN&+Sl6~)4S|ojtGDghY<4TlchDl2gpwg$F3#IeOeNm`?qZUhVO*oe0?sR+_ zOxLZ_=IT8d!U-_a^=Q7RpgCK_z4{a@Q_4tHkwBKXL^%Cs(8gJR<9MlMe|+{w>6h_| zx>V+_mn)UMbB(%@kyZ-B9Wvn_v}|MQBZ^Gdc9(}fsJiUnGoE}$sGkv~F9jlrHV^JWm(~ZDZ5vAZDm6@>q5kM{ z!!-$0StBl_ee_X(b}pU2DL~OKOEa~bK8+tQ4x=RH^|)R z?mP+82=RaVm$(0(ywzuoAwh3fn@e_p8Up*rsbb`zFG2mBRvhD6DMgN zN9&(}m-g>-%Y1+z!uqC!=5_Fh)MV*q-A6}Of(5END)D9VPLe@{}ui>?&4 zja)akeN$p{_$sej+fmTiF>RrWqK!%+KNfz8@isM1|J3`v3mFek6kAC0{J^5C^b51_TA+2LgnEu)4X3;KI`K1{8IrihTA-(Wi;7n(>&7B zkQtp^>89u924seJ>lpsG<#>?wPtvUT_r~!rO@|rkeY3(^(+28B;w!iGM{3D?NS*V8^{Ks7v?2fOl0Bq!zdm;9}S*8q6Ye)5e9(!u0{sP1v&3%`cLRCV|A36^z4T6Hl=Uw0;PiKUR(axQxacu+x1v+6zbz>J9Hg zC6`w#23+{+7r6v)839E>&AoiWQcL7&)5a^l#^U971))aKX69dlMz>S6* zBukuE57@L{$LfwsTM2m*5PU$&DK23wIBz8LEI0vraYRko?0(WQHAaa1+9N(&Q*n7T zHB`E1mstO=bJjMn)_MVCiB|CMUL-h<>b17X9Tnnw&jRGd4c4>%fUlTU!|!gQ)ytT@ zs4*s>60FSvv7;sy%pZfmUS+k!LH~mx*+c7koId~!;ofL9g_YL%aY$9129qp|m;~7xt>y>s#fZuTr^4WC5&FHw=IEmlW?gfHR9JGYfzqtrDw-eZq$GW8ge^ zmla}n1x}_-)^QBb8|OJ$PoM09ew|~+2GknBmoyd0?W_-IjD$^udIKkMHQB$xq_+f!EXjqg!Vl{o`{@r{e*?q5L?FvNHD+Xam6s>;L@*1Yy_-n25vqTpJIotYzUgrbrV8G6Oam=l!PZ79Kf` z9AItfU>prfwv?V*H=f&HOHWpdq%mp^Nygxf2?fBnH}008&|P(w3k<`B8bcJ?dj}=( zyCK~89l$nv^XL|{189ukSu8!tkI~yb9vQFx+7=^ z1~QqmtHzS0RsIx#F^524`36&FD|TLi5uTYOx$pDIuVvsHFF{12uD)WXHGIsFkmLEJ7bjvmME~O%BKKq?f8*`_>W(=KXqPLs?@%Yyl z5CO1dA9?u76~iZ1(nMjIc&(+ysIgoxg$ss|set9+UYmERYiOmxLmc#qAj+-2@6+Rt z|G6^ZygN+)y5Mwx6TLpZzM7@JJtXzT{s(g!h8i`Pr`c=v(bs@#qVg??MbA5(Jj1+E z{N?3@H&UwLQ;v9t2$@x&m>b-bsgBbE2}k&h{4J&3R9Ng;q%^FW9~ ziSJ;Y>^9T_-tpIb_oX|8A92zexYZpH6c!7H7L*Y0Phm|KqMoP)tt4(b9b*qiA_{Ei z9?=aQhPj2FZYo1v!XLXRJSr@O0vqzFg{+c|3LHlWeayWtxwSz4qW;H_U6ebS^VJ;fge?-M zHBC)l8>FGa1-I}~BKm=ad8FK9$jV=&>&i9(wNJMisp=^*EUilnz-IQ&C)!``uVH96 z77XRI6F$@UE5zDDCQ$S+pu>Ope_sX|!{S=5=JuTDvhvVbz22FgZ2gfjbaRn|jjYH( z+`w21JoW6`hu7t8ht<2%6^<{Y@Q5CsYt#O3G2o^F7!_64uWLeG=85(QZgS7n{wPZlYKwT3f>M}<$eM0}FJwA>$ANLndQNvFeS`ZULUDWL4 z!ha2>EvgMy_w$?^$k?@x7(`AdTZ_EpQRQDoy{_p&f46 zPTQ|Hwd;N36YltpUP}y&y3_Tc&d1n);@=ECwLvC%&R%Xg1pye8 zV5=@ItMMu=!cg{yOMHy%usDdCSt-#7wV54i7Q;3UJ-#9kHG9tfpMo{xe<~QCv$W(7 zA(*{6JKP1jS2`c_zwA@C(W%LO^wTCxWPsP=jGT|0Fi+nVNW72t(<3{`&?y(>wX8pF zi>~Drg28LSdg@rTxKu%#GHrpma39>znXI7jHi8y)zy0gG+!)sUec+U0XZ}F=cQ+hMLrL5j=#v`FwaclPeF7W8{-RFX;5X>%YykxQXTJV!!XbTr2R72q_>6 z6SS+QXjxmf<G^m*_8X^qAu+p1WIQKX7VoHxesHg=g{>$ zo6*NZrwOA==v{QavhqX8|Bh$|wEp=htmO}i#me578kK6VI#G%Z$)NLwJDM5*_6UCI zDe8$|9;BSE>+^bc{nMCXHpsH58nk^WNCdkB8gIrDX;3!>VW$5b$`$ko+HZbbwSQC4 zCleTF8g&ykY?1xlAo~qbbznT}HK6bAOW}7NsAHXF)KxrpAVUUac@$Z z6Zr=3zjWbxY` zy{z4KVOV5;k=pOC5zu&>{L-f2?C}=(HVCm04SUpS9BZ#?$%hKx*F3uw;P=<5Ra|&N$=z5czLEgAr-q^ohTkDa~>_&AP*^bMzBF<0ECWV_i0} zESc0)W8l%tNEf@f*5wvER`r3xL?Up-rpCd?mZf}NS=^Jm<@5ILYv@xKa^g!3==yw~ zZxOP#qIc=$A~bAi?Fr~99~}Ov=27=xIX$zHKU=rhCc^}b{Ly?Tsw|^dt?HEPw{IG* z(fcH&;rSiJ@k%PI>jybrp6cRXs6GRaQC)v`wvVU#dzPfWF+U8)!p?%9u?TO4JhLJ- zHl!;_q#Di_eY%iY`rQ%9D*Zj3uSE_d)F~oLg+5aJYlmM$)*x zxgdQ{6{8l34Vogq{QtI&M${S72pb)7bk=H#KAV7< zn5uAm02dyD_51f9XgeeOo_ie+mM}9dW1_ng=e|4_`JlL_rsRI+K^>bj^Sc`GS`!I- z<`>nSCg!hF^WiheVQTtpICc#(*IToX3lguB8=X#BwAh^Qz?1iH%}!ZL_U(COPp3y# zk!_Tw4qyA1lAubbnF4ZCurvB5ae|FU^(e#N%Ou_|KyNy;bq5uEF~$hnxW6r1b5ar) z4lu38n-k!YVh$FV5d()lP&r)#bLi*)k3%nl*iJ*I?#%%6?cld&tuSXVn2y`h;Arg( zaHU$-GekcdyF7iJX74{re-YIvscXn1X#29uXm@?#y7B@Z{i|kyU?QOOI;3-7zV{F^ zD9M6{@cKu$NPu{GC}xN3j)(lZaaH>kFY@_hGEUV3paSh&Zvg}UKnAV@xdM9e0{hN|$ zN3i;l%KWW$rjPee{w#^CXU?3bZb|71-S(2ay38oofgIYt`IBJ@e!IJBDq#IxJ2C+r zbdTu_p_}wY*#XI!M1}pzvukZm0cqEVsd*8Kwg1v)jxvb4F=;c(P9iLHf__h#Td;m< zZTvwF1_@NHN}9G{XXt1Z@D6ft&G$I5jfWdDY#P)y{=fSWuwoT~jn)bnCB}OYj-my${~CU5eHLD;g3{rWA%Z_pUnBJ)g;7V3{Sc zI8~Uw$;mJj)A};s5ZryiieF*0ay91o07V;cAEbD1)(%eHn@2XdrY|&NhAZ>Gouu2g zEEPdAf~uy>A!dHcF$E4D@AR7mr*Ov?`Z$>TO#J?9EyL^zyEAGa5VO{;1++R@qF+pZ zTqsY^_qJUr!|Y8O!^jSo_cno9+97=bv~Zr-e>TokZv9gWa$2p-6~e=N^YWT zfApBbI>trutf!`$>G3hf53+5||GD8`gZ=IY*Ahmr89KQD#?cgA@E(FdW_Im{ooLyk zBNNt_6-I5~LEwM~@ze+OXPZsc&#hbj@hyEZBXtRk&j3u%tKSyle*|VofL)A&)kp%H z5PVCGvEDalVUU3rRJ*kl&`hnF@?jmFH|zO}OBisR`elffYg>o+U&}H1%yHLR{-CgK ziK5^4=w{8B#r6y4Q|1TEmgqN1s7;5xrMx#A7?Z5Fo!g|pVk|X;+|#Zbxvl?NQ-yL+ zqR?Vmekl{cbuH%yA!UB^82;-Yp-CVhl?g#>FfD^oZ}R!{%2yKcX`wZqZ6MfWJFjP^ z|0BJ==~~1{QIJoBSr-__s9uSYb?FF^Fpdk+7MP5z?feAx3Q=Zg%L5#wQ~M?e!iYBc ze79ylaa-k7+@DE9jAfdbuK->MprtfGD)GRk@?1H*`j1wE_?MRwDtERz4ItvI>1a**%R=WYJ3iQ4xfD zPr&0ebj}$2unI#OH9w|LJPv`nCPa_C25o-Y@gH#964A|QOVsPR z7U{upvXefMJ{iJ1#y%Thsi0P;{osT^0}|or?VxO!B&Pp%ylVb)-Q^ zsF{8>_Qoq@Bf!AN7H|C$`scDcAM9c%{6eX1cvszAs5q^53Iv-NzTek=?Mk=0-AU=w(X$j*z~jS+3bi`SGxfE=Uw} z+M}*tKVgk{H=^LH!q*b1)^8L^GPZ%W;f)G*xdQg(G0+xY|Czwe#Il7$qU1n%8I<;2 z|MsOS?{|P-l?{nQ?qf7cY#2Uekx$R3PxuFbSgDf$@?+LqS-dbjN82v`@?R@?!qe>- zEd*t|5W&9==O?2G*nI~&hBZlyE8Bn%GcW;BcCzZ7;X^<(i#I`mG$_3bplqo>##&%z z>k~ofVUU=(4DU8D9X93g_w?ikq>?uqPsRWh_ZDkI3zlZ&OiY`^iHeBPdHlN~B?fbE zhnJQw!t-l}!(PF=fnzTYr4jiO`OnsiiA6Y3?X3Syh9>7gVZYFHk)j@5@_w2mzh@+O@n8_(}^62j2M;vL@^O-4Re5G?2erG zoEI?tZk?X(<+yfT!Ct~FmE1LB%ue<^TeE05WWilFv`;W)=gqJb;ZZAux>vG(9xZpa?4pQMURC)VxD~Fl&+_a>3H?-zUQ2QTb4!AHGNWDf4vPZ&rSl=&0{@L%fSj zbP$RelJS&Vxmo|;_8hCla1V3)%@ZP&=GaBZZe2Z>(aMoDZ#FAD{QfQ*m4W3L^v?X& zwk-nEji*OH-Jt&kU*y9T(DX%prZ5Q6j%)@CF;CrJI)tZ9b9k_wn(y8QyKl>ZDofbI z2w-ilmHD`_cUXNHg?dF0RTq0*Ln1Jy#hVtvMZyHJxLc&GUG9KEeO@>g!$V5VCkA{DX_25(i*$l4n_r})Zf8x7OLp; zMt0YXUlNy`&^{l)B#yl8{7>S@1mjTkw-_#KJ_IIl1S#t+xU06}(Gm2G8^-soQ2(-p z_V)rtAd}Nkv1zcBiL5nkckkbN1Q3J!SoAE>G6wGf z$yhYP1BmE}-c=nU`SQp|*ay)A#|uY-GvhR+#f@Zi0G~x-3i@X3)(}E#s;DNmD91Vr^=}04u!=Q zEGbDX0y0?Qnx}(vk%6{YFfpUuqNAi8<_1ZeLz&-@O<0gjCD>-Oz62LMeTL-+Q@%nE zV^5$qJ`trC8y%9evYE!I3Q2>IAm(eAm`7Fc=kj2~@ufkwUDd)?d2+BYl=I=dV;)oR z4|Eu1*#6&vZQ0jM(V-Iafd`>4Vau7xST*4|>%dJvNPahs){$tqQ6`1GW`#?dIv8hj z{$ORsnh7tMD&$Lfu8bj%{PxG7CKN7{7(*+4T6gMaMc_fK{QbQ8mH7TU26>r2Htz6V zxFZ-4^xVBcuiP{Cm|b{XIF_M&xH3}I&?*6DD8$-Ih z32q%39+vPQ8d5xcgn`!YyV>5b2!>tMiS*i8U@g~1(rJbypQ)c-L|~ZV^e=y4Iree4 zZ(q*E?YwYNW9TwfjPqP~!8tMp-a|*+;pj>o?-3`)t)6)$8d7C zCyNA0ipvf0#pS!n>xN+p)ZHu~$9}hc6G9ybWOX`V+d-PKiJ0P*sv^7zKa7+q348JP z!kN%aIUzL{u!l9m0O`4f>+kFXVSz{ZkXVo>L4>f^RUi9AzcScu41C27fm287u<7$M zL^?#;HMHS&i*X=ZOh^`etG04sbyjA47Cx=sN`-#Gct7wgixIG3+@2=J3a8=^q$Q#e z=@j{%qq?&zUz+v^%Q6I!Dk5wXH7-Zhk=zG)tu83G_$%Zy)(|R+! z==`z;BZbS8|9;;<(vbXsTnIt1gyfG--2D_F<^5YVT66wQCP|5{hzH#*XBX_Zb4b^H z=#r9EW0V4AxS)#fkzM^qv}GdXaJ^POFDD2)6*?TEgd!yNNV2bq2oXW4nz&#h==$Uz zB(0=^6s&4YjH0Mf6qULiVk&H@NfWmSMb#G{41sLc6F_>`KE_v|6tPu|y}pS;_D6MQ1#prF>n z)xnK^xbUg*_%iDHY{KdNvC6SZpF^L@$o}*ST3S=n&@t=m`N(mC6iqGKI?p+Y*2khM zVXJmME4zu1s)_)TqG^UL4VO9V;8v|&(<0ujI7zejdzMKw94)e13D|cY_XXps`H)zw zMZXnCB8{F2?%h=%C~wAgoWpA{WTPkl%KpNZ=YF3Al)1pQFp+|mP{+e z7R(O&aK>aJ`ZzcqcbF$T&1`zXb2oGYa!62OeRB=x4}Xh2lrnLWSNYPriu^uIKKNBb zO6iZv>WrEbcY8jeZ4U(gW7}<9*^!TCh$5rn(SCum&q;-kgEpX!jz9eb7MA352@BFV z*N8*eYpbfBR{xk>cuv1jD3|K2EyARmY&V`VwJM_~2K<8#QtI)L=T^^p8I~pNS$W^9 zoT%ci*WP^w^&s=_k3mJ;tDBt#ovA77gJTgRDduN*8E)#`X|;zYQHN@J8}U=6DWL(b zbz7sC!bk4Wp4(58exrk7}h$Zi(IctlCNvuTrQ=#(mq|RDy^iBXBt?IxGyQeT;tOA`Iq{=gM=S?U@@>B8kJMO> z+YwIVu1(dtQ90rgQf8Ed8Dq}&*&jVaI5bB|mDmn#4trNY>ZEst?~NM`x0i6bA?s$> z*nhaau0!L+5sWH@rtCR2jBg<1AX*|iEli*kMY|IfZC8Z#zUVJ zX3VEN_->1{G+y6YONaw0XD`cqZW;MB_|kw{zriNRyQ&8lNjmL2BYYan{#oOpflvAB zOAvg<-R7_L%#ty-0ua5{#Uvi>eGwkEj-`9-T+!zF1-*g%E{A(EOz-TZWTy8NUVA1j z8zN~*l3`FWG3_Q_vmN{njnBd$=7BcufDXF7^=+%*ITFW->4`s{{XN4${UMOY<{~uo zP|huEwPcl_V9*EZ@Iuaoh513bh)fnf2p%fIA6k3n8bv{dNo^7+%M9+gKL!o+3D$@h z=;eqFNSQrXF<4b&ynpJ#DD|Z|BO}j+%n{LXD@mbm6K~z3BT~*aBa-Svk6Uz2sg48# z0?!+Vyit)Y^d0}`+9HQ!zw(U{+7}-b-qgpcza#3F6z0yaIIS6ozJs+snIm`VS?D5k z9nPNWOC{3)GkmRSy~So<6{#U*(;tg+a*LHJuk{z9tqefEd+-83!=TH#4Zg5y`=+Of z7w^WN(aIXbte)w7cN{A0Sc%1oDux#40c=tYTT1E1ESEWw-aEP?km)Hg_ zMlJIh{iYi02l-4WNhr@3fz;v#=MDJ{2CIM~R|BWRJL(f?>BHSRf{sSEr3nX84-V}z znZ+BC7Z$j^DEHBnU@cGe_cC8|i*xi=2UcO&E&J%O7?SGaMurZX1Fl4r#G9FghpBtc zQgcl!13INunFR{inEEoMx!UzW-@tBD#lqhd<4$j>#I-#e^AHn~lK+=Mff zV=|)+qj-<@Yu%4zn8bS6k0cH}4iI5Z;&E;=)vye=GO7b-ZS3`X=<%#XC_r162_y~o1+F@6)9;@M})TKLR-!EYqLRB2jCb>npT3apKS?h8}ME4&yJ?Zy1$ll*c7V-+jc5*1yZLf9;@Q)})pcahxY%J*N)CX$hoW$cSg9A6hWPyRu1`l)+vX>p|w84o!BQ%yALl z6&_WFDY97)FyE!fa1(^6Z2yiFhN%Qwj;fDpi{*<&jf#!lcYriUY2JgE-HR;6UI^bc z(6$M$%8{{RM+qSZ{gVp&UHhxa?i<|E=hWS&rpGt)YPm;QVZCW$od&aF@WA_pjotKl zJowx^WPPUmaawhbj)HDd)FeyDS?6wTX~+u1Q0z~zIAjp0ofIi$_zqvI;UfPqF- z;uZd<+<*kwWGfSs>EOn8+*>I8z3!t@Qyb-9L?D2EAWf;@PP^}7?rb& z;QCm1arXOoi{%m`MB32~o=qo3YVGi~8*8bQ0CS$=M;<~#4T>NkuRE2ehs-NJS}3lz~)8Q`Vb zPDt^U9g8*7wVezUI6{kjl&a*nWx-zHM2Mr-Akfs;Illzy;G4&*aTVIT=dW+RtT2Ui zGcGv~P1@fM3Kh%m7depT5kBX)Qm@ju?6@f4@_yf4IVrhh)50;jptx{pbD{Jb=^`N>q~1q=QN%n< zHK>HJJ^~tXUad@HEf2UcLsxB;1h%7fQSXjd0_v=RDaWRU^U=PnCClmXP_>DnnYB5; zsoY$_V&akKJn2jLj6TcCT}3XW-0SCyb@`L0_r4}n=QKmFl0PG#U2Dhe3BhGuiGR#l zb2zUdrp5TE(?ZDau%|(Z5bqkYLW2Av9^vlvRA;U7%2|185I|VIxF7|df9i=;)mO>s z5({q%M4-i#*3I_BQ+A|^=*#{T7C>hC=f2(-&Um2VjH5W|+AJA+lLL_+Za!^F%pb(! z_{=!+6CP<=FHX=gOiXN8>AhF%ek!?6@&DNc~}T0o0~Na_-ei`HU}ykaf?}xLE|%=EBPm0r+3V9 zcn5rQJ*dv$$2wf!J`i0_c!ZP-xfHT;d~eTB?&FH?`Z&5Mop)3+VNtR(^J0D-i_Lr{3s1B4Jji9mha~h*>14R7CDtCSR|70zAy$bZnq z#@QutAd+Fldh?+&uCL7N)Lp;yTOV#TmenTIrhELl{Gusyq=vd{PG4bd{NTFPjrL~} zl9Rp|^UL9Vg<1XWx80L=ffft}tj}5G9mmyasHYdN)x5qqP;p~(g)Ta|a=2zp1$BP{dC;;47hc-TlPnnL3;u@h@2wHb2xK1?D6?$KYts#U-Er$qSsSgzLg zFQJ$ap~ySvplAdZptGFj$#@hwy{qUec{ ze!*Wx%$irI>srZ?TjSY}@K-nN8A*xRzi*;bPvF+z_lH;jJ~JzJ7p+K~==>#K?TfZY zkN~505M^JmFum+pH4#&`_(pZlD-B*K<4;iWuD_t}_2ILI`bj{d`1$y zPD9d2~~i%^swYoQaG)5+!B@I_L@h>u7OM}AIf1$QGu20^>tq!hNA zvUR*WkFa9dpiNqAz*koxg(0njMc$#I&RiR$8w6HNZaDkZhmnVP+fN~n7<05-sSw*4 z6|*hG9gXZ(J{(Av4H8&XG`<{`?EL~=PV8r5XKG;&k)#4XnrbJ9D79tphVw*mZpAOo zrWVs18G?$0CR0B&i=c}UHq38xaUe>gGNaE%U$`Li1anAI1Eh3m7FZ3+SGy{p3%^Wc z+v9s(i^YAGYj$%JT?Ne;X)?rRf1gK>ac!?DmAO6Z`^wJhPm(DT>Utzn6l@i?DN6-a zd(Gk|*3f$Inp+{QZv=61%4=;JvT@SYY~tFF-q_5D$s zu+4RdC9~$}COS7<*X8k2?Hq=^-kEB(z}0_3>U*b>wp8a*L1S~{@E{o%as1cWM&ar} z_j8@a-M=fo!9ktTbaV^#*)lq;Q}!z<9y4?sNkQUw;cZvKN5T8<5))|Q;Hg-_TuHjt zp3CN6cQcG0v91c*Th_oOt}h6Na}~|#7`H8Ow}Y`??#n{|$S(Q_&==F;$Tjx+SMs)1 zzuHv57m}divSQyz#T~W^4!xad{G7>|2kD9H%*P*9rZCBvT=3YDq310{r!rp7ySx2f zHe6H)o02L0JMYGw6#NsmDpjqUQnfCWh-7r#hFy5`Uadmb4Q>RWp+#l6MmRK;qvM0kC3D&k^2y13+%uX@e^o98@UL@eej28iut}v@Kw)QrMH^OzT@) zSWW2_;`t0E!2i5&=UC`0HxP%msk({wU2-#OImmN;h-4A`@!JY+XZmwv`~K_Z5OC>FiXL5NX8q ze7*vmX4>W<`r|Zyz=duh&Sj?J!{D^Ow|JRw=Rm*iZRwGhu)7)YT?vnc{JZWy(2igX z%w-AhMeyrT1&v`DIo;39zFCYX$JmkzTp_Zd{U(ZT zee|LU>dhiHDE1ysitG}yA?X48(uH~WRepUzaOTxQrw`w{Iwa8Jp(6Wrl<#k{TGfv| z$=`Yz^9}pBMnJaB2>dzR7WbNYM0Tjes8?f; zZggh2c|?ua9KKW0#VF-T_Hbx`BBP6SAx7Gz+e&MkM<~H4ML8-0|FuIQ?!lw*8O8_9 z≷>P2F*)QhCHZnAv~3E;qj#xaExR@x4AOs_v)annTiF)8s=W=5@1Xf+kYBgHzl{ zH*W?~S&5zg_1M8DWBD@}>n)~}#%g^2rnbLB{+ldla%282{1Px1h|&}H-%!F0y=Yvd z$j(2mj4`T)oy$B4+X%dE+YvNr29xSx7VH_xXRFlIS$=GKJKoV8J^AUxa!(sECe2AJy?U(Z>v;>1d?&8Ph;3_;}_i?t&1r#5i*bT%*0 zX1urAWV~EhyS_Z4Ph&AxrZrTd3OinP7cSxp_N$sZ)wod~W@$a~NIdi&qKP_JnXQ&D z$6uyR_3n*aBW`#tU%&jFP}(61c5MUi+V^wUo`*WDC>+ITrb1*6JvozgmYa`yBD>JL z>rod}&Z273kLaX?&5t)R$1EP?_8r?8uB1U1dC;F4i!xuh$MCf8-(zQ#KlWROkcZ66Vdv>Fcph4LNw! z1jcVJzEF`7lV#{1M|(O>$DUtsN_+eQ;Ivq-CMIvr`&jf&I#t84O1|mkqAKtrq+(EjlHB?)lWE6 zc>JBa%{WS_<~3ct3E!e$w{sg3Mb}|VPHkR7xrM#DG{OR+?4CMM^%MUb%AMmxc0;~c zw=7=%t)(mf`|O!f-T~>%SL6O@n9BjFjc!$dFeM^gWR24G3(Ick0-FnxCB$!iZn3cP zmjFVCXY-KhES!_e=H*;e!YWg+k|EuJw~mK4cRc}DhteWmZ(L2R23XuyT`&72K0z+@mY~F#_NNCMZ@CWFX?0RB2X}lDI2wQ<@a-^7v-><#T+;e zEJxo0aXksW;2#8?0>5|n5H-6Q$c5mX_)BlmN7~v+sjBnzUmv~xSwLPoesA}{-@Rij zhBD0Os@C1xRc2j61#`&C=XCO&CBtK8ITCc#W{BP2I@qKJ^+@$4qH3D!h$G)xjcthE zlI%XW2e;@ZMfiVDn&SVQv?BwW3N8E$MJ3&B=?@iNtn>|#J>G`j}PO7!6S}0c;zr^_r@tGhm0l`kE zaRlW;#t0jyvgSg5`PQze`UMkTXu2caaf2i2ClfLA=tvQY4EuK$5|MQ45)7Zr2h7E< z`5uFJxUfP22-}^ZUykocsFB%vF8*PV3?JfcVCGSR%Q7&Hghg_~@rN^=f>+cm2MU=) zW?0!-GVDX~Q^m8%rOGl`;r@)IyqU^@ zr-s{_IXqEVZ2NFKj)Ou1i+XDG;N)hitxJZu@D{$$fsxB|yba;cBIYH#oN;Y*2)Z~4 zl$modA0>U|lrNSv*|3MDjPPIYjZ>=kk36)}JhcuL@)bZ&!5hE526b(y=ngo})szds z_vc|hphV=eSAwh0z#=$>MRr(^#jvP3Qo(Xh1zdjqMc6~;!MC*;;5D>QRs>s8!aF-t z#}(Wge(Yg)v~u!&O)herTbQ(@CwlV(bv9za&dd0PqlRMg%R5ijd@jb{Xeod%yc1g#5vjZ5qsPDsI z@t=1nq}o~R5<1qfcF#9t@TakF*Al04r!ywp|H5C~@KkznaI{?i5 z--pB|WmvlJ76hKHCvfTd zr9KMfBdCA&QJ92`=tjLWXHmmRjTT3^KoxlaUS^I==lB4klm7Fo2#3YSM_#oo>o41Q zU471ke6G(&X7T9xlA30o8rUH5Gg|FRJr1R0G8Qc-5Ce%5D;b$FZSVJFH% zQ>Z4>?(tf-6q~GEdX}6?;JonmO`S1H&RFGnU*WAj|6+N^+|a}v^^xA~Tz3itet;oI z3&(1^#zd>=JWWv*c1P$v{hJb@oUgPd{3PaWqN7*nBF$CJTqXC~oALy1)bR1uEbC>t z3Uc+7XVPYnQJvHqUyig6WYYB|mSb-_wB5x#JxexB<(4FT$E8{zJ8#TJ8>KKdrGb~> zTePNYSMItOn0mvqjmgE~C`sPq=DDVJotzQ~5Wf_=Q>wJz=5S;&c5ZFtbLJ5Gdm`J> zU|W%SE;aP%r`f19A%6n?wk}VNpGb6z$44SY6Wg#Di4lPbY2dDFX2zMf*gsnuA@~T1 zQF96k=^-S6tvp0khd+p|E%-olG$9a?%Pp=or>X@5ig73yuy&&yE8Q=0;Ab+$Yf zxDU)DR@^U`R}^#F4N05*f3If2xV7>Hun&Jv87cOOo&GgVHc(_0VjGMQSl)bUHceUd z|4{bc@l?P6|G0yUh@u>2RaR0~WgZ-4XD7)HQD$+BV`mm3GP29wq3pdgD=XyK>)7KM z$NF7Qyxo6tEs#XJ@NiQL6XvFBW~As!NLfeLp7C`-(!roGSyU5n z?$GFw*T#k0cW=E&1Du(Ax6+#F@<2iZVhS_%p5MLYD)s3LtmZ4|Be@s)_?PD7px7N^ zM(LJ5+xfb@&?2wHbH_X(g!tC4o{Hr=!cF^}r&`xn>+(P(3g2 zaeuSjI?BT8=D!14+m^yn-JdA)=&s9TX-??VcwIgu$=n6(pYuyGndL`cJ=hn3Ik1=m z0nvcJ>LscGt;?3H(xRVq{1KPRjPz|XTf`Z{;Z>EpcL*dh$ogA=eu%-`{7gkKr;j^| zaK>NVK~XR?;cdDbsGpHSW5sKmz?bmlDK zmR{8LJgFgp;fT$u#-4&=Fl^)G5(i;;Clw zH4YOUQHBWBmxBoJ`eI~}W3u*FcVMK9=NSv@t*%NCqKXduk+bB7?eY==v=;FtvAr(D zMB@^q6(TXP?W>B_3B|i5hr&U~BY;^UuTQ3p-QH`Hlt8-UUb-gFRH+CKo58Xil=J}9 zyh-+JL{)!t+iJ~kr~bVT1BNmKdM|@W#hhLOtnB;;7oMokEDy6>FBcll|wAMPOobtIp-}vCOZ+BWn}Z=*G-@5(s`F&Xd;o_JEws@5?n=H*gf$YyZVfw>;9|&HqSA(Lli!kf@p9+evfi*$ zu=DM_$D8VFkG#5-YulMFcm5QzC$DEC22%skyz20ze*g5wHwAUa_~8rk zj2Aq)<*N4Ps{7P{OkmfVOUSP0TvhCKb>}IDZMfLj@gb_v&A5~~nUJ#I zEdtahGxwg(4Zs+9RSLk{&wwjJRa1#6&2_;mmw)ZWUm{O)`pWL(dIqcFgUhJ?<+eMI zT3b(}O=SFRF{*UkpD(TT_hJz8ekE-9Vti=^AOrIChAIVI=+r-aBc`GBxz|#w&oV zP1@$`C;+Tu;|UZmsmyZ;Fak{#tE{EZ{`uj5sp^+|*MK~-HITwIYII-- z-w{#U$|1j12T1sL_+4XkpM6}vV`7qU>{LcC66Nt|C&Z=qvP*&c#+Z+j%_KkcW2C)k zt1F6CrNEmS$kb3#d@_7?=`$}Z0i?U4_qA#R*g3AqaTpiaqO?P%DhU$I&6fhi2VEd za&LU75GHmE=)zHGJGeUoDZ{hzcU~eO;46DhH|-0+o})-U0zLv{yTbFy;I&MZmT{xGR~P|sAslFdQJhX>p8L_|Lc_)J8#k7bUp?Y*OkH& zOvxROj&3YnS>NC6%=mh1>48<$jW?o&;sCHveQx5sh7SyfD2m8OUNNd?PXt9qNqw z&{Mb(`{5fqdD?M4&)$E({Sb<*x~WTr?Eqjo?#F6DjaWwpeOk1W4{y;il^O>u2f7@p zr(zf@t%WSUARdk^Q!Ac~S8t*^i~5I(COLsR=VY1DUL(A*$fQWHg$ze&sLC&btp*Y! zX3!S-{Z%C5mIUYHHH8gQiFKdw0a?9wnIgXq(>TSs+yK0lNy52$ALuAx<1NAuTUgJm zyM@)A!y81cfqeLS`iIQtJ@l{!`=QTG#1SEN=ZzSnTvTvP(YpR|$eqJrKHk->^-Ri)NLP^f%VDw-jixoY ztSBIugu#MAjP}ryj=?Xk`5QF!oa*CXM-&5?zu3dxjmb>JEZ0YhoF3unkYykPYDh3p zbWG>Sld^d<2;>}ox)35)2+LwYxXW z=Uq#&`8=0u(M(k@Lap~){gxE-REl814#d`5Umw%0HkQPZw$8e_JGHjcAt_(z)(ft>m)IcK zYnaLqTO0hn&NZFlC!qobjJ6Nx7Dv_qQ4cD^Szkmy%J!Hc? zsW3t%wj5pqBv-ya0#FZsca%?MOB7zXv`foJNE})Bog<#O(DkL9&kiId+pvY<7dG5` zX9{U=4|8QE&>;3&Dj3*7Tz|X$%PueqRIve3`c-wxM}q~2eSML{5&wMVqzJW#7__&# zeY-Yotd)3pZi)kveN!#^=Vw_HJJA=fZvIJ%YZ?K97LON+ouKLv^KgF1#jHbA>hzE; z|BpjPn-#3f^Pm&b++7A_m>!p~x;2kA3gi)>)%6(Sa&&G#WZIrfz7XcxWlC2| zyW_vRmtTgt;w>5uAuj{g8m!du0Qr6{X_332e2PYXRR2f1K4z`H8olm)0&dL+NYBF{ zs5^-iJN2dRXvL0tHkZ}z6<4x+*G@@~D0jWvwdasZ%`@mJeC2Cn7isJWl6D{k0Rrqu z(OjGr33n|;Ef6SXAy#vup#1{4zaDlg1rLxlrR~3{uY`lOx^n(0g>0IqmDIxGdqu*3 zKMou|IZL5A`*RQ6Lj%1D*(I~>fZ+q1lg!5 zuYvOe51*+h6{5>f29FF1jZli238@Ok2}p64&6b}g$T(7w6vK;~vda?uhb|?))5l&Q zpa7H|-(h6(p;?gZF(9JyjRO3oOo_@e58s)Rql_Cf6z5MC)x1RAfVk)tPoe@l5UOFG z!TTJuCi~zDfOkY0f>;&MA}>Mh(sN)~!NV{>$GrVCeM=5bA~#}@>>=IdgetR#O}Mrm zOkXd6h_Ay!fGI{WVNlr-Uuz3Mg9Xi+#~f_ob7355!utZ6bt;B)i*Np22t~rJf0iQR zFz&tg5s1%jYN{5E??>hfN=IIps9CYthT#Ec4lzv+42MWpJ_qx3Bq2^XFaL`L*a)#8 zmcN5M{Z#?Yzmzp(9&5#FpJdz>qfv+%?@~3Q8jVndp~BJrmmZP9)&EKh_Hst&9W&TT399*J*zuG!uNrU*Pblz`2H7-w-=YfMBinWjUO%pQI~aji5s&&WrDzfM9il zz%3^AMcWN9P1Z<@@f6D={`JJ}@%L{BRfktp_qZl=(&$-y&9Nj|cPu!*GYUSAAM zBaL8IKpP8-tD|ZrnVMMVbbAtdrV?paOM`aRLn8a`TikWCJ+L`&Jh18QcV1Z!z6_nouB#oR-7%a*%yFh)w<-qcT%DhqfeZzp42);M zGP0a}?99#TexUU*b{^A`<_RQ=NvT&MKJ2xSvun1rb&u@<(%iPa$pO*&BFujc$Xhmu zCGPG4@(Qg4-D|naEc@;9(uvpA=kuxF#<2eopPNJ#B=KC=#sK9Mn#OVh?5#sU)+&^V zy<(zI4Pf1z&fA+ccX3S!(}OSJ`3E8j@00fGX@7Kxdm}K10i41 z6rLnEji;0o`Q!J&ekSM25Wqr@hhiTLJ3`uCMgFw+drSKMB)YStZbh}N|NuL5#q0F_4haQ?d+Y1Hl1c2p1| zFdRo(Y)2NjgX;#O9t&-P#zsGHI-8FmJ716T#E7*H+9?7FKJ{$u;5ca98)sX>_*;bD zi~-ab9x=d7Y)S^1tBbYDE78%Tr^k$*dz0zCpl>Ay4*_5==BVP*)8fWI$xPee5toG< zjRLYpd{}5>hPXdI{OQfq0^(nXk93}nwU6UGm(WL*KZCRBrX%VRA$6TRRo3JikQ&hB zNlkK$#~}AXJ_t_PtL2^%^I zJZ^EGEF&KGN|tpvv<}Q0+;om^HHw*u!wNhmRGqs8bjHBEKj`goac_jTBi>Qd*yFLs z)viv-qdrMN!`I_@`og;9ef$sW=+Dy$=@xm~U*?fr*347S09D~gV z10bnxdTvWoP{7}77~FX!kp7mk!NWMv+uL=G1^p)Xj*(*M3D>;;jS|QqA3R`ThmQdP zFDWG8t?EC%TAp2Pi!VIE7cT#=B4sjDHd*^Nq_<|ivmh^7`C_MN7PJMdKdYSCg^q28 zgNtL$I4G$WmvIl5a%f_sH)2Q^TwR!DE<9ij>G9nm8~)amtmywZOH{Qr%I-<)c=V+|nl5bV%zk<}7QpH(HzgnXcf2`Jo#U z<~F)epmRq4yT~a^$fLF6)wS35kV@AwP*3r20@{0lqaC1mdf+2X)CJj3ghP^nmOH?3 zK+MH>+cRT1XAxxHJ%$@^blxq_C@C2*0K zot4HKM;Gks4J{2d@Th3NP$x8C_Np;lh`ar41+>h(eTvyiQlQyY`BS+8r4!<7#m z$-z_i?E$kAcl&@;a!+apJi-kQ<332J08 zE3W#C8`*7uiO+zbh3wQg-2k7LanP(O15+Esyj(&#R3&^Iuu`{OwFp8P5za&kT^ra2 z8(6~nW=i0)hoKj3vGdi2)8TyJ9TH3|NE&`}pZ3nUtVr3yLE6B{_HUrhfMw~iS=ZnO zhe3Q|5_E)gcuF`CljM`XF2$EY4r%qPy+E0AI5KHFZQ{)_lo5eK%##ig55zh?+aHF0 zJG>C)(yW=hk)}C!*&^Y6qBF*Icz-@Q*#m6!4rY*OklJknL!lefTy>T|KU;^1?~oAF88T%7B=t4c{le6?)k003lGxJezkw!0$@|8glItUpArcgi6f`{qubg zqu9RU59IqXp}!6k067RyOKkAoqm3}HE)38DCAKG=TE9Vh2qMCXXgwZD-ql7R-MD861c z&2b)*Anl=4K>w*7WxNjcovg5qo}o{*2Hz<_grau}#W_5=A4tk;h5cZ5)W&vuTO#!xG>_6DE?bPE+yL{rYT)dV`OUXavziFCYS`_asbS9IG)5WGk3dVVlJt$|$qu z4@wlhwNVKn+(vLHq(*nFloD5x^>0n;#6=s@8ZI7T`~3`aQmN_n7mW`76tH=p7#+_V z(LK!u9j8=_|NC|C4Rn7KF%?SXt%qFavq%g>8Ys{+FKheYA&t?AdzawXJEj|2^pv|C zW5PDQL5Uqj_+{KIQiV|e1b7=>s-?Zte=@2-o>g`Yx7~HOPGz1R^hwX@)d2$-Zhc`U zQ$=cQB4}Jo>c0{%BwsMgdX^^Wd>iPLZryy#J|e!nXC>{@M8J%@w{!W3y@$BEo)@@A zoVRTeZRZl*um*|j@C1Q9pmY0GnF(KM?sFX!t*?-*uiOSIA2*TC?}!*L5;%Ifd^uPJ zk_OcKkP*%LN5s^uDAo&z*YXnJ3V{OT7`}7KF`l! zpME0x^Nu0d)b{;D;(pgh!xOXO{ce&aZyX=$lEGg}`v+ctCkE>9s+0CgF~pZb&5CP1 zct~V%8I0=-pDcC}_v>9a_#=6@+pbT97<3I$OQdt;=cM4uJkKv;p00cw;#s-+iIZLq z#I5mX)UNF^(H(80k3Md4h3OQknAiyv0>xlDWp_$zNC{<9H4Jhr%%`TgCIl2;!~BCw zq#j%~pbuw#C;ar2ZEk+)6Q{Y47C`*9RCfu-^o#GIm3rPb0Wx>w1dy;LhAw8;w&iJ-+2&)@?)4XZ4U2V+{(W;-v`tN87b3``xSW*s4Js2=7?UAZ?lmV zpMqc%x1V{Q-vO7cD2zx3G@R zrNlLNi**7kZjgaQ&&(fGKym-ru4MclnrlBVp$&uLgqjNl$)l@37t@OSsk0#4HFDtAcPjk&&*@l#Ip&SIRs_~JRo}w%2sUcB5 z7L8xb4Aj!CKFquas|9(mGMQTOWB>!n0>zIPMgtEm4sqNi9goB6SXxTrQV9yzAdR|f zh9X3-%fF4-7)(jf+YqYf`#1rEmSrfC}^P4+hkYLAW~ zz+TD#c_Z*3;}E_AtPC-<0p1w&V>Q=I@5_c8BYb&8a8qQ6xY4(@s_??>jiFJx){AoI zK6$fQ8FF5z<~{lajVMF7WJTkJZcQ`J)8ci!AoC&ROQiifuPr|LZim^Qr$jfgnYR2+ zZR&`qka9_Nf4c7Yx{>}eyX%^tfIxKH$D9t=eo@EES@WoLuc0$bk=sgZL)54SaHx&d zlsFwoIsqN_qR!YGcFrvcaGyUTWp*+CF@D5IjUIhZirwjKRmPANQptWtPlIksuLS7K zg5nXY9Os$L#urADl|weKA(-7g_8z3yXh}ugTS1fGBfr<(95t3B zEkBZn{%LOlok- zwk`a62rJ%?6bHJt=T~`6*o+~{W{W)sury+MeaPk&{C$kN99P?dY8}6Nj?Xghz{775 z?-6U(E#2DKyCDa(BvV7>`Sc}Uit^2Ev&1$|T)pfJ`0=s}F#;*8%^1nUCP_bjbIPE? z9+~(0K6cD21nIH!PrvG%!OG$YEcI>A{y;~+TptJG#7#l$hXMMP7mw55zEA(Y5#v>t zNJ-ES6T<(hk}{e>S-ej`qNoPS@tNp$)=4 zL8ahd^jxBYGx<4@0hjxuy9fu&6HwH_2U+G=ml&q2mmkVAWf^Mo)R*qYA462zd~N!l zYzLF&bFjgDinnua`}9h^z}SW_uw2NuHuFTh&F$gr-6)Hy`Z!wsO?4LBZIwDTnphd~ zhuVG)H{$wds1%Lke9(c-zn^G>o`lM_!_fq|tu1!T#_R;kU3=>>*cv54 zcn>Eo@2LXV?5VReJIdQI4sy?EZB=?t}w@uWOsHD#k+D4MG0i3L`Oo<@j)LNMhE|OM~VcOrfU>*YV*W5sF=1eM)B|!?T0nz(i?SP)4 zop$E!Ml;OZa1^V3Z{V_*ZwKSu!6VET7BCXgJjP_n{ zqt8t(#FHR4phhwyj~U5nZ&cZ>;mVr?@9$pU^<9o_Rn|;`&h_>4%JC;_ht>3vufew~!Z(x`4N&%0hm0Uh8g$ zOE0_VZ--E>fJ>R73s2X?O*4fLe*w+*B!d5qoVX}}mSdW7c`K(0B6dBXlnC9P&M987 z+&U{RQ#TGgn;V3&ezap*$7Y1TVjy-R^S~In&Su(D$4Kr;x5;LreIvQa@y5x`R|TRZ zotvQ)S@C_X_f`*laDbiI;b-!bi^}0r4Hv9|@<&L6s&RR0=WlrN*~fy03GO6+y7ws; zrCZJ^X=8Q3O&Zhss*vX$(jY$X&SCP#=qW|9W6o`|aYjpR?KSGnx6p5Xu8&+XOlzOr zWiz$fy?@4Nv@=F|<5M)=*Q(Jd&^%Tm)V^dwV-SIir&%EFuXF65FPVR~w2QlCS+cq~ ztXsO5)iOL(Y-c`t8}m(_(a!aUgbYi(>ZS_KjmXnRgtgT*_A13p>MX&6((^>HYjFy>Vu~ueT^Ydn#o2Pxcof0TS~ex>%Zb=|3%e$uX5ThoNhsAkd+b(Cw#8Hzj$yA zV9^UP02ZCm# zV&WG7BF#d53?VHaPr4voNvv+6aHjvew?(ZMBa-EOl28Y~ge7aDTpJPA@rpq=?MIW> z1!$li1$U%KDM%35hixJDAfp{pnVYOEM5#OQZ!xOo_aAy?zG@F@NPU|iKX3;By(Cp) z$|J9it@8KpD6D&d0N=cW+lXDR<8-t2mTmi4&pzW!8N zw~Ernai$WE;k3+@4XUP>xm%+=1zS@!GGBOY6D?`uVDbbG&kKJ_65`wqT!tKIzDQt& z#@%@Hu65N#>>3thK8q#DJvHO&g#<56j#`3igmjmLuMu^C+6eV9%^m7*Q$TllWP1eh z$Ooikisz7dnfbUdwsSa!_h^emG<{WZ!bZMEyy{w21+V&=VZK)ANO=914a;Dxl3m&d zHa;ryhfJCEZ`4^fo@&@%*@F%F?@vfY@2ZpPy55qfM%9JE12-R(n9M0sxvw|Re77l~ zpQaqOB&w!a&rR~K3>Bu9n|KSq0myP^E|OFL_$Ne`^Om6zCO&%!hI?!m{wWW!PQC!6 ztEWeH$QdboCF^fsx5rAJY)QtvO}yn!y5@|D6>QmPYQJQtq+^52H0TdpK&NS4*oTM&&{NdXiMOfEqZceWY@owsj z-{;d;w?z4$iktpI=PP6Srd?u2dF}+b(+j5c2xG_7q>5e~Ra;mjyp$vBCh}8Gxx#D7 zm&c+`8k*{J?vP|^)8vs$l3PsT-UTK9Nh-5w{nwpVrOTVD1PossD+6kSGT$J@@MkV& zmLR-4Z!)##H=-f#k(j`aROr@(Qomha>~J z_H3a3;5NLP;N?k-9c1MX33V2D?JVb0yq^SXc3bcZGV9tLv8#?iyhC50>~S*PxF9p{vc?6a~o3qxL; zoYc6xwhAeW`&o;dn!GAnnvkc<=H=$~>(XOZoHW3g zb6r!YX2@b)&3%J9wA8MtSI;{VQ5zUj+h;PcCEwA+CrG>lzG4p5 zdOj{!6G)FBdl&Y&`xfZBMVeM&^n)qc2WvU<#Mq6`&nrpTh90$c(94X?xB8%e)bW3h zAG)?p#Y8XT)U2!m1h~04&C(W7M#TMFn_5Zxi@<>UJBWdEDY+WUjDuL?#^ zw6|naFxz{%_(PLq29vMz-V06iy|2Hhq9p>*wW@<;=36y=tgZP)oIfvMWQ-F{rHEJf|InHyqSWk^ zn6W5%sg7i}aJDagr*$V7jet23S-Yp>Qe^R1=?J@+?M9K!qdVeNNJ4o=)cu{K<(_4c z;IbQ0YJ}M53*pGVEseg}bi=q1;*nLMc%zrauI`Gd&FZ8#8Q+kKTc_Fl@n%HESX}07 z1pG9ZyN~TgEuaXYUA|U1DqF%ZzpD%VMC|+ZZ$!kp)VKUz-Y6+nR`hW*j7e%@W%z%g zsNJw8{0~J9F?y#(=?tK#$use>>%T={39zF#4pu-EHMiNkc7K^v-; z#kUnb#M7M+LK3&ERk`p<;yG)ubC1|>HxXX-+tuX%MO#}cqOa3Wa4^hml<=9{K5GyW zmi&PFfEDI(!ECoxofNaE?nU2Nzq9aN#YjFVJ8A!muH)HqmE>7o_VmI94bjBR>Mw#9 zvgS>*=H-of)pu11{Z|8mR!G{rFA=0V`D|Xz5xGW)6WGrtN`3Z!=B+&uHqhdj&lpd8 z*{Tk*6pSdfsY00fMb|W)H_C==LZ<7WWxwB;UV*>F=8Wz5W7+eDV3=`bhL%XJ7`gi| z7T|qF|KYq}ya-=U5$YfVVzF)gOaYFF$eMBY-UX@%hr9O0VX&^2j6u$%`6PN(0!HeP zdHUM?vK6iX^w`hvO-?Xly>-ne=Plox+lnbF5EvUv(y{Y8OmHb7De(+1N$?5n@%B8$ zZign=Mqit|yS^|ZWoaIO3iVxX>A~&BwBr3V)guA|^$`+Y4(2(*f%<#YcNmbEV8lQT4F@XBhp{Sa{Oz)bVRQeh%J$*=KXYKX}mv+{r)d=aO#u zaGEQb8z%3sU-ZR{h)x$O>27`qMNQvDvr^sU_K-{hE-!B!|HcNN0O4SbQ%p zVz$5T^S=1vwIYLvelGxc!SVrBN`P=a#r>)rOjv0_NTl z8{~{b`9v`I2$_YIqhq}%|>l; zjn*;n<*)W858`o2Ll%18w?$uQ?8GkviPGS*?nk39x@$ixAUzD_J0ezW(Go8ztNWEB z-y`FX9-Xm0rzMFAX^bL3)bO=w*fPkxbPq)R~(**lsrE`j9x8v;Lh1S*7ce! zpYN()2?A=~v{kMh6}0mW`?=Hq7c~rK86YwpJ&)F?1VdXQ;#nBRsy|mt()jWWCn=xg zxFLVqfQ)7C|KNrtq6HJ3^^xL+f=@_7zom|s7gGFko7KFOM;*c0-h*tkywf4Z*Ul|& z6qbXpSEO!DJv=k!J8~y`E%C(#p6}n^0S*g0Zaaa^=W+y6TUJ;`MOEdaG4-TNI~vtu|||5=v;!>MZ=37x=N98n;j_KI5}C3HjQAA*)%&|9G2 znezI6KyRZ}L>aHzpB(ZpG(<*^KbvR~KQef|s68t1V*HUu&(|5iHE(umu+XK6|I_lx z`&-pj%^Y5ze@R;6sj5?`sZXk z#w$nQYUN1=2J5|t%~!r z^Innvg$LX%eWpif6Iv%)FVEz9@J5%nAm5(3+SI@De2gy5u5N;n_blX|6{NKiQ3Ig^ z|L|W`6)ZF|3^gXT=OQWm{TrG|2L&EL6V2u>3E&wZv&S-dvl-*$!*(Mqc-xc1QU1Y& ztTmuu{fP>@$V{~1q&|O(&?c~Zfrwds^x0^-LrQa>65B`Ya zy&HVqIr7`-&`d{HF-#fUBhPY9x6l-iAkI)AL|vS`c#m5f)yf7C>mevhwNq;CasEi< z>Eh&$y!&8x_n*c2nU?H1h9U<;ryTU7+-b-_egz?YY6>P_X%tr|H? zlJcopgi$dOLcK?#-+L zoy0TxwO*cgErM-E@Y(m+yCURWMCxI`80qYXRzfv!noJ&7mnpng1U3uOz@+nb*FDO5 zkS7={eta0l=ED_%WRew@?N-uGT8c|Ou21U!O*p~8%E ztsSmNFjXDD`aAWVG;#j_1vM#r1&pa+hC~||vVzCfXvCB1rC$Vb(#~NK471(6_psBO65pMHRQh*9b?4#f5vxKbp3hPG zA)FFUYb~W09`Cz7srj`fu++F0U#ugjU3QJUi&mYpSPJcQvFP2|U(+Q^SR!~y)p87f z@yh+cHDI4 zm>&44^CIy8FtL6fZ~p(0fHVrm%nbifkg68=(OFw*I-bO5-m0n zel4g|%l7PJ^>HP_7FooREU856I7>jtUEg=L6R#*t=_#ejw3{UGC_J3^|i=) zjV&3*7`*z5fm`m)|U3iZyBZ_~p;bn}QA$l==7|fl2!?UV@8Co>+W_X|(H2NT` zxSmm|&Rg9gg@^iY${wX)oX4|GL5(te3v6O#2@U|})gTML=3ty0Qx&^hz2cfo!P%s6 z-ZZO|GXM`65{OTdtC7LuUN2~F53+P#wJiEk5LQPg8_6vGjJ!)sofBhH-Ps2F1t-|{ zWxmMcRS>x_49&yoc_G^lQAry%=zK6xRr;UEcCO!7UEg#&xe5V?MEhn{kYcOnJ2x??%j8uApb>j259{qLGjusP3x3lwA{|% zB=I!wVn?T9iYVIaH@bEF=LWD{Q{j8p8mtiW!wvvmv#zT`qZLVV%3QK}HO=f68zq1) z6-<{rwgBodo|n~{XycN?;&|+?mJcK(U=xsRA-IZSBJb2FJW%Dj`d-)mkkiTBtBAT1tRx%<) zh8Y_>;N=l0&UYNJfdmUrftG_e7 zT%9VB=y6cIlv!HSXSiVjGPa%m{K588{c4%?47L7+#9Skp^2hB%}OFoy(*2IG~q+Z+bmRR(fZMQUPX+Gy`amN%I^tnkJ*I_%%+ zS?6u6F7Ith7Kl}y1H`?{u3UByC^o!43zV$uLEps@U!sYwQ8G~WB-f6gJd2ujcKAT$v?tP?)Lb?Ex!-PV zIp$$5n~$Q>)z_zmVo>tXl?#mcxpiiFc!el?;$*FnNMd!B0$LW8AAiC-CV+dNSw) zRuRXk`)%~v; z0E7X*PkuL_EYd8Jxb>Qk6jB^E{8FK&LO=+EuNZC(Hd4QWXU7>i6erC^{5@cC=3*)+ zwIe~5nka1Cf1V@ppXU%w=X1|*V}~OTL+219S^31Mgb5Cz`wUXl2w)fvlaVSzLQZ3df;IA$EK&%LQRqBTZb}nuDZ$*1wIT^`T zJ2Z~b@i$|IdAW`O zApTY7?U*qyU~eenfg#a<0S+>posUQMcS~u8s7m4gj>D>u@?2I} zTKkt;B(V}i9(AM!OAHW;&4< z(kY>bsG+e`y1?-rRLok0VEzwoNmHlYF=yU~9$<6z3_h{Sw}wFU5E5${+)TQkbvf|# z@hkAUJz0*@cz3J3BDuXo6Kc1|?YupIfPr_l_5N%O^+_ElU`qhyoYk$zA&Hhp6=ZjA z(KmFK%YgBYko(-LD8K}}-u!xN{uCJva>U#0m{a^Qz>Hs%fUDU8M(WY@6dYcISbZ7B z@ZsCeE(o)9!}`a6rw?<*T2Cw;v1N3ANz5PrbDk{zIZr*!@Tu6aD>4QyF@QZKB(|6q zSqvFqhX$`3o#2d0AZ)AH%GW*;LkNgIJ*9RA9F|%r^p)_Y_cxY;_n&h7R;T*<0YC~z z+Ej0}4ui;g-j-~47g%q`e)Lw^1bC!Q=`E>ch-LM$^ReN<$Ph`;X>JJl?e2`Nc?yXE z+qV*sZiwCD^91MMnVS0)ZIfNyx<9WO0MO{SJ~?*w+I6114EX67=?bVrR{Agf)O#9) z9+Ms5$1YZnhyLC0DGPOXyHAap&Pgaya#2yO!w|5%0(*g&uTJRSk0%-;}WV+&zAlEu^IfQ3ppN{vHdP zZ>jn7?{31~rXjZcVG*1~r*db+ssxJClf4i%7avR9-!?#${RcD+qQQk{A0QhUH+YRa zgf`2rY|LpG~1#0`LApQ*WaXceCT0!Gh&u}U=<1&bl;@1|XV^~JsZpej7+H=%o6 z(d%J*Ro!d2(~zznEg=#ySNe*1`9i4|hAT5=3iI>GI<^Q+(w)YiIy*zdX(OpTqLg zGdq{T`oz|9=wJl`E*3*6sZ=i@>he8@xXqT*%&Pmdw45Z_yIaM*=%p`E7}m}|v>24o z)Trecxua~(RwJZbBjRCW^5Yl+aJSwx)J}fXJ@mLG;6}UQ67Zp?^rGW6MBFFGc?rDL zuirk|3p;5)#co0VJV${;ho|P$*5JVP9?#hwXI#tytt_o9E;lH647O_|vbC}Kyi^9< z!Xu3^^utmR8~uLVgbS`XGCTd#G^U4sY5-P{I2bjT-Uv9YH}W5{%K{U!@AW&cfa zo0hhB+d^XN8&$?5T9{f!LL1)*WH z$ccM(cx*1|k7I!P<8;C3nXNaL(Tjm{uUJb}gxzKRqz~{<6bur)@d{J%@hoA*e7rQ8 zeY4#v@?Zb_fEdrm0JM2+%UMa0FftlBG#)0p8RX5#7k0Aa%=`P@N!WzgNyyFf)s#Ql zUTya7=-#?>^55wLzL@_M4rUN zgt^_R?+9^*{Cgkl&gOq;FNGuQ-^3J+iJ^p7-o*_6JOXJ0?MsNmN`ZW}(E_)qZW{;l zw=*f8fR15Nh#ttL!&0EPRN)=6`r8m{lVY6wpwaCR!fzv)+6_?y+D*F?-~5|^sp1tx zA7pUC{;t#pV8B1=2g@28io4W9D97)f_`YPG=4W7V{qi?3Vk-;W9C$BO@u8RX*#U3^ zyfjf0ewP=Z>r1Ese5`@hsLN5YTJ(QVSOff9p_B4Va<75vt&y>R3ROi69t zch6G)O(b~4(*5O5wPx&0)YIg76jGMfM3!+uZeGgG zZ+T?0H%iPjF<0aOf@mjey4?Hg3$X-bnbDef z4FKS4lS3sLgvOwkyTGx*PX6EtVI(2;D()$iofC(jK7$3~W^ z^kA(XF?}-nq}j%7?}_RP$mldsT>xA~S_C+V$wH(~n%l4v`?Xj4r)(0dy<^LqP5~xwbKfhk$c@FD$#rU28MnWLr7iW#^bR+81 zyhX#7XQTCa{Dfa#T&^dg)>jId)1B}DN#i*~EPy84lC4ci|4a>&5OuK+{>cGxVN(ne zHUUdaXvQQetZ2k^5)kd>*H!1mJ-hhZ{KEWGG+LpoLYUh}zeU+U@lB5q zdz(63#LFE%9)_6tDPH}JbwOhsqNiVbn8Ze$)VbjI-b$4Gn*HHM8>z9@hyJFxP>ZRP z!`>4=b+=xRRfoU=IXk2K)H}KXO9qTyGS0^M$%l%@Nj~^5SH;I0roFK!T!R*f^ULE9 z{?o8z#f)m|OKUxY)~32$mCmoNNa74o=U$q;D9EqvO2P)x6^R^6zW`XTYxLHQ`QaKs zmArM4o1`*WndN9feh7V$n%i71+;FMw%tzP%gR!rSs=Dpoq`N^<8U&Q??h*tJ0!nvC zgMg%jq=Xv0kI%7u3 zpgb%bFscEKcn_O2vF4aT4AN2-x(hHR+4--khw|D476?jOWKu}%_21s+g``!dl)!F) zyH+f(Klf8im>6oK&%1&!+=UQQqDU37#k-lHn;GN^qZWh!2}hGGxaG>l6sSxnrb88a z86)qL3~sbCILW39SkYCW?$nZy6oV>Z>eboW;13Y5Nru*Mhj8HSp0lM{($bQ+={{<` zP6R$p#qpCe6@Ya;&U?p{_0hJdY%~q683$IPJq&X1WHtX0*J_=4{ByK67fDFf10fS& zO-otXlRM6apB3P!#@FxwbE4~Iiduxsgt0tp+!vu6=r-Z%T9yK08r&s#R+azFwi!J) z(%R1fDkS}}NPFbQaKD4I%=zPDq`@&@(6p$C^y6;VyJHA}4Z2a_@$$suE>mzBlF4@? z?G9cargq%VYbN!#IdS8OON!d>@h&y!A~jqe{{KSy{XA$Lj@Wl%e&i>qfpcALV)7>IpOe95OH=C)aNN@i zZWjHHfUDLfAIpXm?)YlRCn0~*w3AvUj!%2ehv(#-x!2Ddtnxa%*9g=_g}JbRC3J4I zY_~^RK3$~p>=Fk~*&kQJ0|)1Oyx#tY3jhQ$5~h^5pY?)0bGCpY=9Z_49>bI3>0bI+ zfGyJIZR4-s-n$X+jrsQke0zqxV0R#H7T7v;UHcaE^5)pZ>FN@eVEEQQQpgO8Ek%J@ zl^Ei_w~%HT?IhUKj{pmRe#Q4?0v6bwVE=(!+!ew`nSH7Dfn_PVQlH2@Bzg_#5@VvWZ(D7mw!s_6V^zX33i`K2`|=-WUZ# zZggrd{;malH$%DWs)5#u5ZY2m#JT$Vku%6TM7~{92)Xy|M^Rj-rM5m64a9=4{p7Q2 zIBpffA;rcKhyb4U2IIHUJf3fE?eZ~4=#@=2=l@dSPs{D39vOi>kiu)@TYTst!@IZr z`#=jhuK%M}Q?NB(t?ndHalfmj1+L_iag7j(8Xt1sk6HcEvG3E`>KmI2Xp&2&RDEQ7 zgsLM)62*u~Lv{y%kkVmZ?vZi)EN8<$Y=COC7k;LamO7~^!4>8twj9O$q0wpdH(#tV zFju2Io=4e2v?L0f6c|A8&^Xz5)gMFmBWT!)P6rqI=?IGSDc?fSlTRZ~2IFnN)UG>j z>J&lQI54QmJ(hc$AUX&Oj|yG=-MiO<1uMwP@T<&xSmJVrfAy>EGI-RM;pRubbc5p5 zFM&RvbS)a!R(IayOC7eR{bOJsf#aR{odQv=y0WS!)E?O&%AY#VCvwB#Yla>5Lr5j_ zL9)nWhQ|!5d1XvbmeojJ=zu!&h2!RXZg%kHCVy5W$v^Kc-DMuGv(tZsD8MQ?%)K)! zCfnEeDE2iF5zyZL-s6r7l$DAoTC~6wM?g$egg(4Hxl9mDHaP@E-mEe9al{2&wp zCE=c-b}*}=0GjW@#i%z%+7`d_{NTF?+{*V8i6k%JH&FiRqpBDBV-6HKN0*4~ZTjVu zy}C(F##>R|&kMgdK5ERz16v!hIZ#Nu(f=T(y5MC*yZ<+XZAFd>NrsY1a`g;| z!?pzLN2p=lmac%6k{LGiNL)nIfBUOrj<>{%u}7vfa3AQ#_9!Kh)tJ^KCB4`6_qUc~ z0+y-(&-0JU9dmp7fl@Tf7Oyu^T-|&TQ_=!*&N4~D8aREcFo1fJWchM*QbC(*aA%3RMn+5WB+pzx% zQG)&oQ5H?{#DC7R=PoxtHuhrs5WN3aF%O`A%K6ejzx*$>zImFr^D03LAbk64!j=oK zMT;NlZ+Ec%z(fL}7Dle#|Lh*X9nD&PP+T?^jM&HQYCe&D-F)TxwBGgatM>St#eDAM zs}3-+X$^rr;kyZZcPfbY+huCUa$g!7G;##kp8w?T9mUJ0`^5Jt_NXA39VS&T ztT6Wj7E$T3GS%IS<>bwYAxLyzAM-u{H+y4kXzt+y2sP3w@xzd?oNXpm3Zdqz_ud7K z-{D*Qe}c{Wm&$!dkl5YPj+;X)bq68mIEB(4q zZ*+zte+jzo55miI$aGu#Vf;yAENJlt)C8@+cwAllmrs|6Ve3*(%v{%)dx9ryIty0> zedq4hmO(60VmO-2MXUwXO}q*P99o^xp`BgDdbJ&^T4ez54AnV}n2n zf>D+oCA4|p!B(&MUX*_d!qe#zc%8Po{Av$1ZRs@<`PUaUXx+YK;tpC7l0Q${d^SmZ z{Ce*_534rBfFYI+yrv;kD{28=bkXj=j$X{1UG`8`F6R?yl23I@tY?4!lCy@QmpV!K z-@q_ssvzPQ0=vEoJ!2(7S-ShusYQXZ+3g=`5Z0A${9h%8AkTG81KRgV`#uCdi~Q3t z$KBcj2FM=}ldUhudX;C+?Yt|dnN&DZGxoUNsG_9@88fZ6(2Yy&zo_2zxKHsMHxQ?{ zQ((@}p6Pw{G)Zu&sC%~9o2~UXFidVXfF0XrSx{ea6R73?Sa4Z#eh5$dWO&Bj@vtuo z48fdPD2H^bI%x2DjiY83ry%QQc|@x|9sKlIvnqR^KR104Ssdpeh1iJiu3dAn$Qsq$)ZE+&X$W*(hc4~zlt zdqhNJ{$T?(ZAE>cnSIsqb5WT=ldLVr7DO3U`{(N;y`bQnJ^xwP7gzq{dutujk5^;{ zFG%Ngg2ne95)rKi07-Dx4S05(k;X-SxhdERJUm!5vCH;C87CEHMI1t#A=H`17iVC) zLHglqQHttnmS-de-_jTI!F%#aW|3VnwsldO!SLB$%^{K--N(P9)m1&_@h`A5>OQ(L zqRX_QF(WExKX`!{Lg^%Vzg8eb;E3rlmCrN*mk6VfLtO@Y3%L`AqPq&o)5Ps7I&xl1 z7D+4GoGj;CZ)oGs?-F`1h}RNQ++~H~!wg8H9mhpH_?^FO7clNqqHW9hnQI1B0-_e9 zsYp{iv(QB#2+4xehv;PU@+kemkd_;sPoiFYP$|bB_pQYf06!5)_bxpliS)_n&yKKn zFw*>F=ym7w{CM4v&-CEadj{IKUfr9zpYM1tFsKZ`!xro7sd?K&K&zxRE@y+m6TwXf zKO9dYZutHl&)d~^$mk7S$|iaQcOOABGvb0BPqjZHh|4`hno&ZyeU(TZ`{LgZ@USU)@J6B!MGC z{Z}F$>wCe@<)Z1xjhWrC9&BP5MGsFtl@V^LD<1%6Xh^fL!Nz%q_iRS!`@zNrVT-7j zOaqNmuToY0T~~F+fTWesLi!8cxappN!W9tV6CZ~(wR(?`&AG;{u;ht7L_U!#V0xVk zu+6IEztr01ep!`sxc$T3KdAQSh|aK6c4 zK~q!gJcg}yfFB-C&ha|q{5RpNiv*rUq$M=$N-~f__LBX@T738h z{Sf6uG8N%QT1DkejkIG~1xjv#cSpf8cYW;G)F^^R59{r9uqHg6-HG}Axntm7+9MQS zjrKS|&c5kuZZZPOlhSVuBUT0`xHtR*1N?*MM=rdg7YeDqX2GkC(nUwn^IAk1fgi7o z1#v|~kLDFDd^Ra4M%B7*#+xKx9hH~{P=ttE;e+=+0oPuxQ4Ry72x3a|s)+12*g&*X zv0zj<-avwn-X^yo8nV>X`QUZ$4-CAc9IuyM95lSTCR?I?$jTbQl>-T?@e@+9)raiy z^TjcPs~c3u)%#}0e5g#X(M21a8Oo1F<4K|M9={x|=rjd zUq#$U9b^OEWFt(<1rpTE)tdu4em?_`12;QshR}~~yi(@5c>gci2vC3?^H-}bY4SYY>e^Kf7jH1H6#*y1 zI<(kFG*$B}6Owtp_ujOSjL9q-s-iA2JFy&rJ}Wtj4lbXx-=2tMj->1>`<93~@3VfJ zaBpYR$OB65i>&^-v0g|#9lVp6Po;LJDs~3lm5s~MUlTAV zCb47vB}e?NKy>&GW?xds#n;EQoh`YH-;c5xqQ#ST>VTpPb0S3F%Zn+B@l7hf)NDgY z)#&+D|0m5FSk-q<*Qyz>KfQ!}WCS9tyt);twDOvaRmmWS`&U<6+x z_gjBA{%V3e(pX>9omqN?HEi_mD81xzq4F>Pcd-aBzU=R!=j6cJEnCRgScK-)fg~5q zY1TJX|3~zZgS#v0o@;*}S29;~XVUx_Cr&WgyvX2kTppgv#SQ+ieZxhY?$G$6-+c>R zKOZn7pAg`!>V~%iLn3Wz2A1(~cL*96rZ04Iq_vc3F|O@f?b89$eJakk3j!Zem_fXY z)^qjF*tn1Tta*InRtn{@X-ITIEXw*`4fcQH@(G?EBpgSIK*f|d(H=ud7U``q^DKY+ zygVR@k!>rVw^=*L(`{&&TSOtcbymp4gtu-6yLh=NNX2ueM%Q=RUSZBo1|70Da@83q!&*WaNf5RVsFfhUY__zBw2@0$Z2 zfC?=RL}=Mi#|SdpVxQtEZJ`5$=e+-!C| zd)W$B{&5m3zm4sU)3;-|*UKN$6B$W-%&&*v=NnLvqF0bIqwucKkK8X$4ddp|`IGu5 z$i&5EjMH5|Pezna^nLN~#a8P-im$$ZlDUfo=Fs(L2s>$(5P5|1_b~udoz={NX=Jc@ z{?aC^D1oIY&aOUmyX(GsV*bRH6AsiR z;X240Aq+GcNAE9ChpUm%Kh$XtYab<5J!pL}LL*#rS%7%9_-;`!1IHBZLonYY6TbAV zz=ile@vAr-(VMl&A@kM`G%b>pMs_u|lhpMs$38IEF~fk36bS!@v$4eT50GCki;Zm} zb(O~Wet8JlaE3=UT{E;@274p5i?4+=CpmQOD+aT1m)l{Je*L6NNmau!gegl_^f#ua z%`w$>^yBIF$H>dZSWfPIp$fUZG~}9REunHzbr;GSizQi zF6Ndytu`yS@B1f6cH77ni7XzUF2*=`_fS?mLH-eWEc0HHSbuv0@?e$|EiwL?m$lg| zUk4yYg5aI-=ef*3b9E%&BQohEVvCJQV(i-fj;fhKwW3*Spnw&M+92@57qe{vZwHU4 zjRS-t36MOUM?773@oh)JNgq`gg1&K>6L{xP-Q#77CRXY+Q9>h=IvM^++N(2+jBfHw zTXvXuGC%SHk1d+vBPW#Id=%&7%M!{lPQRSXyP%cGbLck)3S=fkq5BJ=sB~n6Hqv@(#8#Zjzq%5Q=8naU-!jQ zy$b2Hq`k+0>n^->JH~*rFTv4oWFT7&HFsCcaN>uk9CG4P==vt?{W`4(^UG*^e)LL7 zc7yqOVVQ*K?2WAOl!ZzS?gb>0{m@kTTjOxVhlBeQXfJ8y(fBh*=p7z0!l?Siekumz zDTm`Dvt(Mkeya5)FDeM&4ioplwpTar#(v*>KfLDsq;7&V+B^M%Yx&+@4mL7l%J7S< znksv{ci4|alq0K-(=dOIDhfq4zOY zx(Q`|B`#Zpi63!MMxi9vKxD{XJ2kos$<4`j$<4M9!G~vti=5`&*_e^NCF*#>kVM}O zQ3&5%o12$tFx+(F$`gOVB|@X*=~@mi#aaoFi9Ycfpx0)8b_Pu{{Zq4E(w_FdDC?GO zg?SH^g$j+w_O;R%C_31+$kwiVDkH z7#9cFhRyr*@$X|3*|$w<#&Yh!u2VGFSEKM-iB?z7vO z1&fwHStxI#w<{Q~8R+;npK4yWdRXG@h-w&dpyn?m=nvAQvgFg*br3R5w4HE;>-J;) zF4ZYn_szW^MCAcNV5x4dhX7^Nmm~L=iEY^7X-iEcq4=16vEYB;m*0TJJh4VKa}eA_ zA40WufZ@JGq70p^+*(Ok9p2cfE@ku1aC^&s^l*eKlrua$|R?r6rCUDT0}KjreH_%guSxXsYGVH`w@_ls04-l;*byasT{ zb}GWeaVW`;IIpn_`;|O6q1uCa&#ZO2KBkXg6k_4NJPe6a>3vh#BlzYLqJ?^Ly;gdp zeLKtfmZan-q4B_H`7m*xyucnWtW25M^4reuPFSf7=={-Y?Hf&nzhtZA*4PE3xz{mV zrHRC#^(;XHG_aMhs+SaM@d0i!5qqAlvLpBwoPnEOF6+ao{%jWZIJsJ47y0S&1 zvPW-O)bxf2o@(BO60KQd_Ee>sg6q{j|W~VT3U*eT_x3z{*$kTfXgy!DM^M`@}Dl z2p%i2{3>FPZ(dHd=0o2&g*9*JcoL~pc_rV#l042!ngVM0OFR@b4$@sV{YX+{EU!a~ zFQyg|ihVqIVU#3T2@wg~$N9Y1s<_OoYi#Dcy`L%BiXPf99Mh1fZ?!xofx91HSt>Sn zj8^nCpHtZpSdTLDR@N+ZC-g|Z%g>NjLvqwqtll^xfd{CVDdiJ1nL2TMapZ3MEQ(+$ zItNX&Caer^%%31*a4KRCZ&=Ev)_sktr(CGEBfyPSDfi+H=E$e3w-E&0FqU=K^HSkZ zCJ0_gD+&DE%j#=)wFem^XTuvTuV1xQ<7W?HH@WVT*|^jDAqbL)Q3?tTMlw85o_2!P zG6E@I%lKW(JtN6`j*(A0p3ArCt~zXs&4l|p=RftYAMul6SB5@UkNHkuD&J-ZPdW3` zy}V)XWyR*+G3K@*bi~zM$j@5yR7gZ7A?p%QlTI-Ds z{kB1@33T8Fqtp_E$GBoY*7!CH%>_(V18Ua^9)`NVV0q01IM5kJByF5jQXDE04pe32 zFtK?EmdxdDs#&1DR<)l(21XbAIg1i>2)!VDzFYoA`r-TC%356aKW|9U`1|No&z{Eh zuIfBV+2xAa_6g35P2z}ZsUa6-jEvFMY-7*Qsh|FS$c`c=+v&wHET>l&r57tZjqL?v zI_FANe_X^ND&4mMy@C8*S`ROHk?`xjMb z8geZxFU^z)bS$@DhKmz8&*LATEtFd%d-^6a!Avxx!cMLl2lg8$I7;w85^W{MZ~JuG z3y~ce`{dLIAAFE%l1jJ3E0GHPW=K^p86C`&V1YOq@Se-XJVV8@&W7;7(Aec6F&;g+ z3u0yKGVrnYLb|N;W1-T_^twFf1sdwkrfS|g;eH&@sd)i$`YF>)%qh|Bo4jXOUVq|# zDl&XNW5!uhpwhltQlRf>I3Mk^AsRF>b|3;m7sBlZ)HmTnbYkw2S^)vS6Db1A7}>Rj z(iJ_5hSJ~{Oj*nS#-+Rs(m6=*4K^-lXdwOPn(owe^ zmuoxVshJpo{_Dw7WvlB#`)Q3X{sB{hlDyPBHMPVkpRlf90K3ov`wB9KVxOwJ3AM

4JXEI7?!j%gXo-R<4Kk#WIC>beDe+IX0?eP6{ z4dUWq+s|QswwVuf&?hh0bw=JBTGY%FCPTD7JHF`aD}sy3l3&$%faPnV6dxc3BWdm@!(M zL;|OB>51sW5cu8t;+4~83xs3A#cy)hDa+DUGbd+>`o;H$A;F_6DY9Z$zSeE@;B~x_ z)*WFT?z+;B-N}#5D(-$>Yurb$MXLz)AimFX3S0x7Q`!<{UgqM3T^?^u{$cfFA8*os zmlVj}cM+@enrI<_;9K046qX+#3+!T(KHxk~|Kf_3|L%~oi@shASwtn_JUMdc3M=3X z;ARPt!hiYK0*|lEcg>|NUSr0mQTs~}Lt@{~lJTeqJzIu#O$w((L+ z)qD4YTz!-TC3p6HNguabdD`bRjea4jWZ=9TgqKfpNaeEw+*z3`u-NB7{)g3X(mX}BJU`dD24{i%cZoU2;cj406?OaUv*FM4 zDdoacwLF`gNJ|ic}s@>^e4$ z^`qmDHM7Ojfi7iEIwtO-5Zr%7JQN+wn^CZ;N)ERMj>3jO`(1$JWJUE+` zhs03o)u8=K<^4r!XRivL_MzfcW<9XX(s*PHmb%lykJ=}T?yEsdrkFA+nG&qf)TppL zj7_-wG>Of-BpMEUJbtXMpg6ke+}yLBX*L=5#aHk!?1pol{*1HB{ApY9vo+tyoK1}lMmP}?8ythcA{;oOdc zE2Q~}SH=FF1jRLU#ftW2){qpNR*us-x=EetY`!2&`1+^U%G3vK2Ub zy(o)RYRi?(ktz9GdD;}qFD}11tM9;YJ@+zSI`W6g+B>aRuv6%mJ6q^`*#IahAw3+d zpwkU}*HZq37bk3@X>Z%LD=2ZAPhXk!z+W8SXM}8riN9IePx}uSKtnIOSrVJ6Ej`%O zn%MusBe_$wrI*}De+969T&UK*>o1RHiA_@bG^L^{l!~zD)8Y1HL@_~(ur|A(4YLws z^<18O94F#7@0lM4KJv>sC|CSx&@(kNlZhMD#K;9cpAU;n(-+*ZGF1I9GlX3xXc1ccJF|_ z9Q24i0JOO_p?8Byg}AY_MqU=Ti&YSF|a!|b(V2pENRYSbVhIJ1W832(m-|K5KhtAGSV>lz$H|{Z? zo2rtZp34|WAForMn^x~#U0)n=`4ae={dssS|6`;6uIOj3;GPfoc&{z)cViu~=xxnE zO%|YYyTlBFXNi`fl@g?nZZX9cgks?1>?D^#;x|mDbjd<86}cJCT)-3=!&0hZaW>avrB|#+A*=3p>S&M9%fhpg1e1lg5^@;*dKQOr}nX|U~Wo_cZ?k^M0J z^Lh#rO4frHuG4}PAT-M#4*k}6?ldAa}qrCc!y8OMLyjyB$?ailZ8A~l2vtMY%jGX;-foGr%z;%#F zAiJ!H%_GZvLKT})_AX{J?@#l*TU;unK08rkwoM(vxZH_;#&dIdB7o4}F;S$vMBWNU zMjb}55jWR+jd)mL+07ZsPtnLi&@l+~s9!`5P@(h&lx~!Z3mN=+8)r4r!)QgSEtc)x zM|H|n9WQhH!!)3e>o94b1l=hzOu2-?-4_h+l>aIpojIUceth!d`Pobz&!gS^{*vN#=( z?xzq6CtMpLrysm?6bv@D!)X=}3ew%C4Z2UFJ^LiDV0Dl=Q;!j$bs)qRAZ62Z8a)Cb zI8`^Rz-ry$HWv4sokGRX`?uHgs|&!aw!8(wf!vVk2Io1$+;=LpUdddfuV?5tOIG_? zgN}u;i4;;z{Yn8?qN)+(d3Bu8n;|N<~(j*2ax) z+FRhKz2Hfwlk}dZjdB(& zB58+#hKt_FmD_rS>B~#`>Wcw&sDgB;7zMT0=J;1xV~?-j5q_Pr#2-xk>>XwG!QLBB zMLk}087p-8K1$|F6{T^Pe11a_{!Q2=csE^mK7(Po&F`c}|9d3RoN#BR5W!tLZW{iS z9H)Swl-^UmOU9EhNUnh0C`wv`)6a?1$5>wQ7?}mINQ6=I{}3eOCrF;;P6=$#mw|fv z3d$(S5<}3)kilP2X*A;*nu>!m!u&3wXqr^*0_F6j?#N6aX8aMIMue<{e{Q3qe*%Jv z4BiueWShwgYku^US_^!6262=>gYJuaA!-b6hAXm1kjcNb`{DcSbz)|ef^8m=*Kogd zTx#W>GR@U<%P>7Y1!)q%T_O0S<^yWPaHIRm&U^VTpue+D*#fk%4_m*QYFXtk zZpRe}g-UV*n^}boVAZ_L_kr@$7H~J}3c1k4doBRW#f3on+`JZOkTV@C+ot=jKq`)7 z5Sw(FMU?ggixY@}`w;dT1jlev;qk@IUV&~#h<%tUo^c=CJ=4ho6~a%`Kfp~K248Fo z2`=c0$SWcMtzxmp3V2&;E1cQqz&G^w|3(wj4+vy)#i8OE=&UyRJ4D||A3Wj3yzdq) zBq)0;zxq^*jBzzS#A2fG@jCd57QI(Qxnl#aAPlk2%NH0hb29R@>+-bj%O8P9ECXMs7vj$C&n4nv0DFqcEcQCq^B4>teDLBv8RYzol zY?E&RfE{iNqJ>^OfXN^p)gp1x?}1}dXIVF~fv5-c)`h{462iBiP-cTkF^?I!zYqD3 zdp({vf(%t6URK0AYfXPx2rf_e$)PVaNYRkQ0bv`e1EG*d2(8Ol(Pp z_ApB82Thg83K<&{&Ga78p>`+(8oE!V*tpJLO=d>m zT6vdk+jBcmrmL22jODui4yo#tpXx-xOC*0oVwv5W&Ys5Tg$QcceqaSs7V8xE0qw=^ z+wrSEvj&#Hb&s;(X++jjwC3! zV@)0#hWuwhd{SoGJX+da7{ar?0ZXUDSs+;I7r@A}rw>I3;IV zyj?@-<=t&o*kHZCu?zW-U#5j~v52r?LK4R^MaC?{GU%?M^e4SH_%UWEF`Gd4>lSj0 z91;My+4|QE#l9a-msx_*X0ZaGwIab|fIW?QMa*r=OJpwU3zQnpE%$K3XUS7}%v*rr zz6cw#+~~3y8T^<^Q$6oV3H_oUR(LBrjIY()Gg!1g?*Lel=${D{D$wZA3*(bOX>K~T zphr^5G;%!F?G)nDo1%Fu!S7Iu;!rEL@#pz^ zaO-0Q&h(-ZP(`9&5-KI8lGU0mT60}RJva63jkf$@S2f=o&g_qW%D!`>Pw(+^`sB7e zufB!^?zO`5!(;1IPUQqkoABlD9n#+Hm6{Ekbu=>#b>hRMoOdV5XU zmxfG(p0083d&qplmV_rqKni7pJ^L+*je-THgpZfsG$wr|*b*G#qm=#{P^b;{Bv{I1 z7GH%PYd;~SIAbCjZ6w8JhB2NSu#iByYE{E1Rk-3UX_wQ(8Do^ObX5(1rS8i}f7OQS z(A}~{5qdGgde`vE@MCe7O~WJ^fAmKBA)0%>xWjDOc1jd|28F|zWtuK>5_-$bMK&1Y zhCz>ff^UgI>6TF|G?GKM;ZOI2LwpGT^>6{*X7_Kju%APw=2HhA;Rq+Yes) z?EOTwm5LGFghF=QQYQ^BPB0B}mmjD=waTagpIXAI8rJk5Dr32dbDr9(&thHAesOix zl6IR#GTUn;M$uw~oh<|feL9a^Ex9u<{~F9F4f}@rT%1HyPBud>O#Eb>1qmau71Jbj zwGF-g`oBf?l=z<_8yL63WntJ@)pts*uw0e`fLX@UEp`;4Fj+%X3w*h9Jq=3agVOfx zgf>KfMl!0A6d&CQI_UMAV(*cvh$G9h^j)4Ikoj&c?gh7g`vRM&S6U)2$n6%pc$4?Wu+InCp1dCVt_T#m>k>w3h@da~ z4npK*zv*%Vj46R=AJ|r-qugtEAlo%~Lr1~USj|W~s3e&~`g9)D%fT2mYCTM^Mc0G; zT~-+5G{>iNoyy8e5kURRVhuYO;ff9%2*fhuWj@FDl19b+N-XKN_r{S*_sY28@#2JL z;J(}_dWEUXZo*09m;870!D=*5b1tG|lzcIiB;c;IxL??NjoVfI8l%37G#e99Pv^$e zbEzH6bDQxFkDoCdV;4qNqcfw1iIzNQGqMg|jv6iatbR-ZJkI0a7;iGr7c zuxWfp(pqcMwxz+(!an*2P3f|*4|AlvcxFV{%VfCjo7S^J!A4jxw9f>)%zD6iz*_e^ z(FS{_``nujhvZSBd2j#)&7MEns8Q!84R@1TO;Es7xsM3W_*A7#MSwke$G*V0*0hs+ z$9P1grTwOW>4DU>u|i(KY+n)Ppc5lZJA6gF7|6}1y>QF6C`^66CaxzVrwF?kh|Xet z9}Le(sD)0X8ai93KIVH2`2kcji19#A>QsxSs0s1cw3RmzG#u9FmNkuq8U#DF% zsR@4N6jqo$YCwlSs!xN<@8%2U=knQ@IQ3i@abVMefp$oqdsC<)^VN$0{#quV;H_o= zT=NRRe@y+`3bd%W%2Zevpk&WcKLosUnorRsx5v)0tiFWJ+~53h9WKB$^0(Cq*_*_L6gZOYXlqq4NX~hD2R$L}GwH5pIkBrP3kGK{gdimW*wnNB)Q4%P{VQe$LF!C$Gf&n#MNTPOq+?Dn||?~3m?#kJsb z!Vt8b zaiy0+l11pEVEB|2PIN~!ocvx~UR>uv&VqR;K4+7LbiyCw+0aaKEMxbUH~F#wNui9l z!;Cf9=eNrF+}vi6NQEC)ui4(P7V=P%43G***c=qcU)?iaHuqWdxc=_p?!agkYbu{z z$Co4fP>0R<)UBqIo|n?iqAZL(46N?f1VeHm6f%dD3Z22y-;2~!*IN(d2)77tiQup~ z^clkBih91WZy{kjfsE{-@huN5N(I zd!L>=?mwg9R)fwM5>8(}MB!tAfSoyB!V40;H%MrcgzFva5=U+6)5l@yKLrca96o$@ zaC-J7+}^@4O=)D0ldom+l~gA7Y@HeoRov72-j9ejNEjR$`LrFbP*?e-TeV#a&^{o^ z5f~Ufxxh`2pP*)%r-9HN(Nb)jqqf)jMo6+kUL|ise=#1i5>b%U>x=n(coke;A`KcV zgS=a!IpeS`o|P;yk8%gvPy3>CiTb76UC2l&*M$4u! z<&i3;2$w%o2OYNq2{I^IslG+B!L2Gb@6^eYf1gX8k-!lm)$bn*`RFyWM85}sOhDWe z{M=PaAwgZb5W!2iHOEr1hyQy5%fM=^5HVgUGDsCXtPnN^SbYJ7eNMFOC^57lgm+^k zzxVHPmdA+(nOsqCG5X?SQ$%`V5#YOWh!*bm>DlKD`bcw??0AU`OSz2#ORTBBvspri zg6#Sa|E)j&zdynj;GIz0%)pRnN15@!zXRewbK>KF-&RlG+7E&COCQ7hrv1`Bs1Oez zBi`0^q?cSD|D7k10E%q?1qt~N<}Bggf#&aA!(jig0Ugp+@FV0p6` zUH4@xlsJ(?sAN=vHk&v!Z2OWQ9PNNVF*kO@k2G`&*b24Cl=7@hyeI8j#A~4jJ-7GW zUnzXIyC!aMAij97OK-mO3(Oe92sG80;5rI1k=p`6o)BQ)`@gtoJzJ#Nvf{8x`WhzK4i!a(?2KL?~DgJHGfYu>&v&rWmvjGh->uB z%WBGPjR9sSo(Ol5f!g_FlqiRsOlNOG@%na*D&fB zM9uYH15JKy7)U0`b-C#z1)oyFW8yJeIs$_L*F-{yoj~dsQhPFOlzDq|mC0w-_v4ie z@BzpiIs(_{XNci!8BYiJP(TMd<0BUFvGC(_#EFm!!#`$J z%4o{3jJ^a951XBU`00hcjN>)I{X0j zsG*YbtD}+*XuJT@R67FDv3vAU*`<+lktgq(d45+~2R;x2k3QEWih?D^nUuFU%=Pw!840LY$rkIOPC%YN(^BfWj6)!NmZ@SWi^vuD03zUJ$u8_odQz z^}R}Y%~dce0WTQKGP%quP1}a^Q+0w1MAcB%7jKbsJ38{4(I-l3%+5 znq%ws>qGS;p!S@4k|FqQ2Pk&d0pQSZqHsbNhHrz@m3dq`1`MVgx80WCvSolkw0_xh z9xyzu10}eSaVJiGAjpgsN`3z-|F?Z;p0L`6#;{!lsJ(VA_ND)=y^{Z&ZmA{B{o2!i zpHxx5@Zx+d80FvEtA8~0#0!@gURx}f9y#jF`)9jBs4XJPR3L4&=p!k$ioWH|`QxM! z@wq890O+kf9YW7<9VNDz7n;Ern#UH8KuY=|JNwq~d=u9}u(ah0 zf5lj~#LUBW#sh8LJ#d4cU1*2`Lp3fALa`@JZG|`Bk+OLEV!(FLA0$;7*@(J00WUKw z4{$3CQ`rcmjuV7%5_o#2Yt43tE}E@#YwobS!n+Kn}=9)bDA^{ow- zsqW~}#+V3VGvamcP5eHf0DL3)8V0PU7Mcm0fZtgL%_0CCYtSTDh02^%ZLxL`g63pM zzYUIkL94P{9LfC@v4`j2b51`T0^zrR)+JRbKDJcsK3~CvpILNA7|c8ZPiG11C+$v! z`7h3M)6+pPsJ8Gf0^q^N;AuNf1T)0VH(dGl*H@?OF+A;67+7Sx39GlA0cfDBBn;6C zZfysOuyQ#RZ*hyY0zCFUzWV@C<*@~An~+ZN6KwGHXQBHTcca-7MoxaGJN5vR!mRBf zQP*EyH2k;f>c!U`e*Rl^Ke5@jN6WtZ zTXp+2@|AdkWx?o}F6{`np8T}r$vWvV;a^d{=QzD(+L7RT0thY%--+51`UxfXgT=rC zimMdR*TOh0D%(!^fhZwp80<7BpFme`(bj~mMY)w3LJvq0p_3wC9)t-!^mg?DaowLM zXjmo23XV@7!Wi2TF{!rUm;6JVRwhu{Uw2D{K~*+kP9KomH7-~eaWE1CK**@mEuIp> z+uIiVq9n&}kYuD$SB_ZI*IkD>-)npN^r`klSP$X|{PWzTq-2dnbi~jA83^@+3bpQ4Pfma_t7!#}Fhd9t+>7YIp&?G~a$>+AdtWKg= z6*SY33`BdQdLl^*hkgz}ju2Gw1u?ZX0i{JB_$xbZo4st_yrJSZB!jb0Gj?xpZ{w?Z z|7u(;$rNRu)>bQsl3q`<%Ak=m&dB=_0W}47EW-dJc6w`J+`3qXRzr(oX55aIugR3` z0Uqb-z&T$Gkq6L0qk4y_EKC=@eZqIkG$eHN2Oy?{!o%B=C^~D;g8F!xfcLeeRpS5HbJ(9 zN#f=o_BBfOZprmcV1?+5pz$;Leq>rq&->%tw^x2L9SV2k@W|*S@9)?#yF{r)s z#-~FR*sFz)AH*S1d@$AS7($VrIs$jpz$R1JnR5$gDF%K4qO={JL03MaS>CgxMHg~0 zhZwFn-uDtylvovB6{)ajL?WeMBAPNbjmF7V*JRB*j|lEB8U*4-i`PCY+gFGsis|Y{ z&i;)rL~!W$11rs7#gZq$DWcvMs-7U)klz&$a$5-F5P4VHIVd4>@#f^3C~bB2Ir@#rK)t1`oh;V=pt@jh%kG`Y1E zYa}!=3<8uAOm;RoVI)&0$S=$#jT`$%zgi)Iu>?+q`>FBoQP;#N>Ml{k#JT<4N=aeL zA7NUTH)RpWUM5fRKSRC#McDx|5cdx<(BMcDw3XP0j!aPTfOya?p6rTSksHW;l75?2 zRdzbF?g|q#E){?G@6d)(Kgy>=t}djJk=GVuF64yAAT!Bl*f z5K{cSyYm>QP%JeZsqiw4ks@t{O{02i^*zKJ9VtbCg%^#B!a%;<@7~g_=f1!@wb|{N zU-Xzt2u-c4Nuz48Q-8mB%#bB=?k-q+)UaN5qxA6Gc)9Dz&lCMPk04}zcfdyYHjas| zONw1FJ-%x&vKk4sANXI{#NiX&g;!yB(J;{-(sk_3$LJhv5ZGF>u!ng-Xzn>u+}u7# z`*@7n`{X~Z9JmxAPygBeJSxKT4Sc|1OtkGLPX0W;F@1HHJxsZ=3)slq`}5dz`ZPD- z@a^v)hE~M-?9c0GwBnvs-h8wmGy8qMp1c%_^<$yHFs2tUG~)47FU;rFFZ*Dj7>ko>Pv9k)u5je8*eMrX|p?d1mt-3H-{gX*1#*D zWgQXY1Tw-r2+T(A$&3ac?a`x@We+Y+--HeP^JWOl&Sf<{i$vI^KI61 z8N#dK1@7hTnX9HV0e4(-GM>-EqYmwGVUbTJ_Jr1*WbJVl9_D|-vOJsWJ|QZHR(_J3XvHAw3u(MAbXw?PVx-gjvaS! zJ2-_L(Er{JqknG)Qkb8WRp>=gZ+!7{tjAZTlS%8PuRLrg zB?1Sn1*%KGdAf>@)P2G=r8LoAm^l`6j1tRp@vqg3neA0~ynTci^RxyOWIERfT|$!e zCvGz^P^JVmapVXrxI2zIa({g22tPkv-j#n7Nqn66PQ+C@NwDprc{NEn24o5$PR~rHPv*PEYD3SGM5&$%`)FgQWvkdV8_FhrUye6VhDbm(gtd`c&_o%)jMSxDWcxCVU_G(WA5h7fwUrpt1RN>Fc2a&YsKU1Pa5>i=88?Scc@0vnN9?9ud zuSaEle|9pL{+UJ-SJH$x-@Vb`BK6t2>s`rZbe0bRJz_)f6M+zb1$;eZ>$#yl>(ORG zdV=%%m3ZC;AB{ zHgbbp)`!z|c%7RjQ43BGP2MvDo0P<%uiOmSEJK3ti7!!IRVM?q$V1xazO zc~oBeE!p$f6o&|f?5&JM_TDQQ8QJ3~$t)zo!P4)?1TZZvSsmif&ghL8&pN3 z%P}0mV4bl~W-}Oby*hdJ(?GdqQbTMcN*X(WRS5Vaft}1(f_VU|QlyjebHqH*cfTbY z0>Xrx)h`vfpv;}h3*7w(n*A=1gg49p$SD_diQ%{u#gYMsK+Sdk{`-U5-q@+z!2!^F zx##JMydU5H{o{L0XlWIYbTnaA4CFfNSU4nZEaKSA3Q5TdKzhr6)fune&V6977S$DQE}aA=ikl3OOXAiq#0yOAKKTi%eYX*XLv#GkTWMQBH8(Wf>^%$$ z5U0I1FgzLtqEjKo(~bHRA%Q(Q=UIT=d@}@6zSlD-D>W=yj@-Sjm)Z3h0L=AZTN0Tl zs?M-|R2izb@KKTFTyQ4|@5BjKT4HI}0Em9skN1`*b<;#2S+#~fBW4*8_n1@G9RLsu zmSr&elmL_H@X3I!_wGN!_OO-Fk7)lfVvQu`GNN`o5x;NWd34Z@J`bytyzF<*x6|Ys z@Lb#x{(KM4Uhv7aK%@4$d*>Z=zqB|c|0w@X zTDCI%NxFMTLWzJ`4L+XBEFvm$GvCmkh~9q}E6-Y6#FqAHJ|e32=HJ-Rc7^qwdX(tEVHEh z9xIj5t&6pJB7Pr88W>z-gNb zG#?Z8JA9`I6lCAZ2C@BzolcK%7WQDLTiem8VNNT)Yso6s6E)V~V#nHo0)E`yCyw0* zF{z&!B{}o(mhKx>iO-V$#aGVChEG}mrID1PDd#2}&&*#bvXDB9SHKgx0QT4dpZD~=f3e8^r8X( zcU(#U&p5bI`-QKf1_klVOy9nPuHsq2iY`DrnK-63F(2~H6SfJ)j587$FC&)amj#4z zEP#|}!x=)?c^ z9mXxwnEUA0*Oyr{A3WC@9=Dxb@>T_VGL3%`4rJI(nL0A#tIzCU136pI02^*nHZcL1 z1gHu`Wz|B9Ms7Xc4Fmm=tsgz`co#YkC@vuLmhjlT*JOO>v+h8D->ubvpVnhE+%Q|+ zTnzaLs+9Z)Q~KGEjG4ClqoI@auCf9ZJG=QgrqD z5}9o2L(r$+%)M{wezH}YlQY^tqIE4(Nyq68fvL(POprfQB(kN+bzY$_iCo7c9J-eJ zo$s?jTZvvILKXS1>!=$vBP3SBK2fM7=$w5p*3ogAX=&scYTlrZAZWF0^n_WDey!~7 zD@8L!Jm)r!GN^nyV+l8J4rJ*0f?QT@x^AP)@2=L^CuyEfn$Eag`g_6tbo3-hT!mh`Eq7iIOj4CpZI7m2OMSgh+b{DV zG&0&l*@F1CjK_fqBP-IIc?44E;^P+R8jQQ0#^kzSpux8iRNqaU zi<#r+;=A|58FjorSa(FB`sQ$Zn|_-CrPHe)46zLTfgbNbpYqiLD>BLxmq(tPV(W@M zcgT1`)A4*K58XN6q57~s%eF$_Wl5@NM)18aBO>eTX@(>|5NSPkzn|}rrnY(7wyk>a z&Ur$5p|g&9mma&_>%~_E>~EB?I=|;Ywia$adJmxrG`;kYimq~mu`uapvlt{|auQFK zMLWg&kcQV@>$E!!JWH6TiMV2`;0*Ps=*jCj8!fuwA9JZI`Ps6BMCX&YWM*#fdNuuwCZ>z1&Q9;;Ypd-ssTlPG*f zsX83B$%IR{aYNH3v!e_{7yVzv#o~SXGSRejKKlY% z-p$HJIGPkA>ILWq1D*P!ijesVmb#Ah@+e0*)wUE_(V)Sj@ued8+zTT&%S+oPUZ$Hyh~4|tCmQ&8%Qp20W|+;}id&p|iYKa}!d}jX z#_tuMr8eqFWNR8)2+9nl&hgFJK{MlF@Hbmv*U;YSslBHL$#psYV^U&TW8a3@Qeh2J zy%x*DYXo5f{;`c?H>GCmD0=pB6Zz%2B3n2F#zN;DFFF;-EBfJWkMnnO4(Yjgdz})^ zK&KpYEgC081~t5=C$?qdS58@$yi~>0eHfVjM_=x~;dx3kLkUrJC?s!Vfx8o^2Mucm zrl}*uq8|U?5N-D`lZ}?CDSwaZKfzyrsI)WGlHll__|kvUH`jYa1!BQ7A7rb)6PkS3#Gjbm>^<#TZ%R&2fJ5uHLng!YBvShucRCD(B3oK<^AxT2s5;gu_6churG*%R zft$d!aEC>DBJa!n;*oGamAK(6qBn}-Y*HS^^EbdJC;g(CjDHpE3&kovu=HLTX)Go3 zQO$@EXLTyzXca|ff$OgnA>oxF5bT8&OU`?gxUwIkbQ8iXNw`RI z8;0K+_9pyO(fd{%yQ%a?ov^|pQ9S2r-9^q0xR-_wTNH{BnY+WvBXm*R41aQXXHye` zECC1kMAGMsTLMG8d46;I5m}*`5V-7aPAy~R@Ab9uK>Xc=X7Qw`C)(F&MFE=);0 z!PLH9;9ZmJy1>nQ76bZ7IPY9Bnp%b>4iJ!HfiHk{RnEDqmx+ zG9B`#lO1Ga722sP2TIzaA58KkvMWrby>=^(0ET>|maLJe>ELA5aYlc5TNa8}NYWSm z2maC109U)~3V|2@P?bI!IPS1!E&6dZR0=^vc9-+8xW)QO4q=N-W9@4k#iBpY( zMK2Y~uO9Ayklyxrb*t8|=R}PLQ=f;%>}R6u#M+dH#Pvkk_ueHwp1#>B*Lws;rXBrF zd!1w=rWlef$%->S$0bYJeCy#G=3D1tbBIXU#W#8EOWtAwq9ah)=*KL@o+%a#pRJrN zm+dt?20X^b`f3K`bH$OS_9Jdvfr`Vwy4P|f`WDfuKEw-_!1RUbb*nAJ)vq^jY=N82 zn~+Ge*Z12|;u6fx0COu?)LM_ED@AccT61!;B9xa`p>s|5)Z1BfM;J)$8R}Lrkpe;a7oFPI zA-;O`Vq4yZ)utZ&1p>lDt1u+=(c_3|sm1feLxx(Ik^!D8&vMXah|S0!V!iNf%Om_P z=$1l79{3tUoBqFFg8;GQ4uPZrx(H=k11&d){+gxWWV9}dw^8|697-rod!1I&>AGPpMC!br7M!kzE|#xauJZtyw8 zQyVUD4A<({y3FrkJX)!a!BTArZ?$nlYWFz zUus8SQUbrxFf5}|?u8PBS%$6KTwf7t(d0HqfWu9Du;M2jc7~YR_8$Gn0TGYA{*3?> zVb+M2_LLKAH)vhzEj$>}m=XF`j7<<5D~Ttc3ILYA5Gbo^4(9*Hn+7XD_?SHvFDDV} zH|2MAovskGgw!T=kVP<*%B)lBYdzxRBWfpKt@7rLihxl!-rm#<42PCd8f5?haj?<2;R*B)MeP!er zq4+z!Oar~axa6YpUhxrM*b@6@#RxF?K9Sr@#g@zTy$)-A{v2Iv~z#t2SZ_NBg9NSFxY?6m?o`3r&~%c_EwWEyNd&sX{hHt{xQfYS}nHHW$k+U!*HzRSBPil@*)=7iM(n(_rzc0~!^zj+x&ahvvZ0=dJU)IhO zAIXYa?K%*j-awcK+0fdg+AMjofyaGNyLKGuLpQ4-{gjV zK-W`$tMHD+_63A$B_2QbUsxc+v`V2KB_8ch7Ans|NKq#OMVn(R7b403cW~uFK97=( zguGB0=(#clB;ipc2@bERd_=qiub4J@N$7_k1pxD$wG&y<2oa7r4k4-Lj9Gu({G68- z!@A%YLKl9CV2vI8+2pU{*@cAJlj*S2`DqjJj6YR->pxchz!i+{lX1@J>Y68>6OQA1 z9mTW*G%43zw?~=)G2R)&YYD{XUYxG!u!)rkc;uM^W-Ib38TL zk<;_nwCw5DU?_wo%2}oJ7V$D=h!tg51>dMLk`{HhxQ?82ylgs^(0a9Kxf>Qr#;}Z{ z%XfR${2e5IlKn#w=0h94>b)OD#<#|^lIrt1)3`9{SxHjfb#8fxR@7pBNA$jifp+L( zxM_swF3nm4A@>Kq@ocUoIvDB0sk9YH9=mOIPX@VNwvuvANkf;%oYJzTaFOLl-_mhi zDFL(I)|HI6%_5tM9kBKZ?oC=TvGwJXuk7BP zHn$?c`1TUv+49<6-I83OF+g8Sj=l8hRi)dndi_4kGx0E2ETJ~SGch#&nP=8P8+6S} zgs9>HtP3q*`|=32;(Zck-YR?y{Vo?Rx~ZHPKSDE{a9!%Di@d$DkrXz7lammB*tpx}~`|6o~M4=p8d#!Y=$Q|)X zoNurupVPuuWxhtXuypaWJiy~tj2DPy`#|L*nS$o%p}@`cT;{~fHI>|KVzRpbq)vss z-i#j9Fw?2>Xx@mi5_0tVQJ5N1D1OV5yakC)uwh-Wh;b=z;?tl8sY4HV{Cn7*L;(;!x2y zR;|4jRKoBy_&A1Jg6zP@7|pA6#nGV160OfIrg%-!E2ld|ie!rq!@4NU2|dpDk*v;& zL{(&ThsaddqEbrSdfZ8Sb@8^y)i_-qklVLb5cE1C>dqC%5x-2g_PR_-CWUqG_tGN7 z_$v<6sg9NJs}lH&@YtV^!cq(}6bSCSN1_f-P=|kHK`kv8=gPmvdk_up;|n#_2<%lY z_FB_|9EhIr;oFK<--V<;LH}Xe3SUcj;M>~)_hrq1k2{OhaCU#zw>YhLZ7rl$o%Ouv z`!#fF?b{HUB)#7O=9sUK$IPcPj!C%0T=C6YKZ%sS_ST6%Agrj&9u(_;Divfno55Fd z_gOBpiMf+h;>AAO10bY3Ws1J9SFwFTFZ)E}y=bVdAg`5Os-oa(Tmq|D6XDlvU!r-u zbvB?;*eCtoo03|pF$~32ru*|Q7M^p?PQE;`)lrQ!ft=m zoL}J~jp!k3NuLo?LDr%1C~`!zabk_^yUg$4c0WS1E|@l7UyQ68b+DfSbzQbShUg<% zQ38VA>)9~8l$3x%O^U?|JU?ytJ?XJTIr{Mlw={m*7mmrnB@2s&L-3#N)>&!DwYlEV zz=>4k*}_v;iQs?-#5&y~kjQ#C`ne-K8Bq=|ju@bLWW8#jX`WM*7~_FQW%LBbww-Y! zziyHkKUP$-3l!X9kf5E;KU|~Gj7LhQ{8IX{&#$T zL~Zn8K`kwzteWr^%$8kEWZfJP1Id^e;p^~0rY^WI69%T)gr>u1#l$wL?YwaSD7_;@ zOr9pZOIr6%q8yCFpw}xrrWWvJJY#Ef-tqUx?2uo73y3xSc5V@(YV--=idT#Dve6>P zHJdBrG-{ly;j|j+%cKLHM_tb}bED>C=4qtBkAcgJjTGBx)X*B;Tdop>u6Z8Z`hKy& zJo@8xtD1kRtRiXU=Anw-hYoaS96_S|G@>&w`p)w=m~pywysfg*h2)`t;|H|{ubzGI zWi&Uvwyl?d=*|TnTNN2tb_;tL>8A;0c*IG8q1mc{(Rkd7>@cTW6R-A=jg%lIoK;`; z0y2XG$mpSP5(h#iDM#$@>{ac*;wqZ8%?Z%mR7uSMa*W$|IDagCI_CTF`&Q*x*$kE1 z0mGZgqy^RI6x^i}1PuhjGj=y3_via1gH1JOz% zpE7ngu-^X%@T$l9dCGMirk=UAdlC~lhhO@TK*bkFmuZmq(ihTM#^Q%PO`8#E2A_$` zV&)ky*A_I{l{!-ASk1DY-c9|KdNZ#TInR&F(dCxc5qpMpOzQHfxT(0PaC=nA#e#hn z{Spw#MCyfQ!X*XGAw>S-*v(i`2=JM)+4JT48#NT0^9J?fPOFFV=;)2@6ZK0@F@?P5 zn<6-{0P9#B(QWq93MkjJB zUx-zEaMB|!dNl`Pa8qiZ?f_64zme9%ScUYrP475k2Qx*ZT0+QG=tqx4f|vP;U3`1l z(oct^#6_Sn;#QH6u9Sq#fWuAqzDEPe1_r|ZlrKbt<=}GJa?&z$6sZMsf<6LoMTi=o z@{MEGDzGUVyFDe>*!cnf!^sM%>eLZS8<-e95nbYqqY=l^zC0})uIX~opZ+91Z6!+9 z6Jh35wC38g!b{#gWzUc^>#8$%eZNUs^NvYB2{W+NGudv#_=86{tCGK@SBF%gU*l=3 ztWLUTvwLDX@V7t+{ZD}aL?$z}G13TSaXTI$IM2xJeDj zq7ejr79^V62DSnC!go;Cszwm)xpVd`E_tqtsN-Zuc;jU0Nl;qmm>wT3A_*RFwxn7) z8tn~i5yM%V+Dx4TY?*RA=(*Pyfy2jHiqs~TWLE_y#1)k{79I_7z@Lh>i)|j`$NRLc z$ZScIT9X$I2bkS&-|m2QN)yFtD=x4+_e}GQ3}$(AaRH{qf+X>H(@I3@{49&0Ng}Ph z9I#GSq$LA?bH1mII71U}g)#mpJkj8Hoc64gTB3a`7d6ZBJ($^1-h^`|B>{^9&<6M|7EFd>Kts{;ms(CmM9oG(G%fOwlOjS<;N$ zsRf-Yk zm<^}k_w0`W?wi8KUV8nA=3|#vivu|a!Kjy!vR8%Et;`w95efuj{z}DA#dK<=k*0~T z#rEgKYo_KF{X9>Of_jsv2ZGH#=PGh^|1=#vZ{2>Hw8oePGoK^#Btdlb1e$Y_F)0T8 zaa+H}@)jrW_S`R}wztj7fcq!jdzpQ>2}6uG-^c>oziDC5BKh7DtcHozXWZc1b-YEz z7Xh0xp@AK*sD65f39mT?Ze#60>T=OdkK-1Unw^U26H{+rzOP(ClIij2U=DXZ_mB%d$D~GsScZA~@ z!2#imY6@Q9zFWCPza>&_jTL6`jF(6xx?&i0Zk@S5#(YXO&22V{D3L59sNyyqIG3_G z{nE)CaZe-}snz~~f%5oV>Q|;=vb++l=IL_tg~33zsk(|TraTI0@|+PI`fY#@7Gc|1 zQM_l+__pTmIbbfiWT1Rq6_PR}u=D8Z0ha+k;3dlikd*P0^_Z$&b)vQdEH>~S78?ju zZZogEo}7yWhwiwJ@-#u$Hr#g(sVb8T{HHrSNZ3kc*1Ca`GaDkmnI?sxIDoDh&g`ZpBN>M4>KE= zJ4HA+cLdw!j3M$zNz&wXH>V74G@Az)IoN#k8@pM?{vw^4R5czmdz1iI9OzQ+Hw^8lKw2-+E_r^NufRo+_=g61vX{3knds069vexz|wO#)wwc> z&+iJrSB~9OzmKB51=b z+(nF2x4!~7wP8GIqqL*GKd194U2JMO=fefQ03?{X{i)7~@-IlR5epLB#exLi0xahY zzIoa}Txk{Irb4@w>(;%a%V121!G}lpZ!S(9MH$i}g8)9|kyi#zcG83E!ql7|i{ur@ zKLhuYWVYM!hrzlUdyg;SKF=}t1tAciS$V>Pi9%Ii=D#P|IxeUtW9-adg!McpgfRO0 zef`DCi);=C=Ddl!OP<}MoY5o&;&bQ%b^OQk=mIYrZ7lO{$trDrX~!7@rh)+6LX+c#1dyo)&wI3Ck1VXZDqsJ=?O*G&0D-ZxA2noTj(~=`RUj z`mQ8eJ|LYNRoxdD=1AS^D(A~zv^gsXrkH}ph;cK5n%L07AWV`JxaU{yp6vkV-7G*_ z_L&3igF^&`WKG+=bWZORyuf_b7LUNhAD_TN2-24xeH^)b<1!;em6RVT5LDK~oj7UF zwOJNq`8>V|bqDk%vkhal6e;5tz$*LgJ8MD~*51kfI>Lk3y+0EOW$yTj!65La+O_(^ z4iQ$VGbDp&Hc^J3rf6AugkK@+m_YiCD zz?Te&Jh-7}0~8m34ro@>!?I)qq+Vp*1xU^mj0N9Y@{?99a=40i&FuvQM#2=bb8YZh z@`quYu;D4RL5W*nb1d1{jLUjH@SH7yc=E#GvpVB-m?P6&gTOL+V7Mkb4qSSEb$FUf5&V6dDcdK3l+0(-I7r|ywvRasa@N0hmIf(O~jxzT>C@;`%QZ+d<4Qc|VosUYj zIj6j^am!ntl0;&d45CIX-+@Q;p6K`XVH&KU96PucoP=|%(PVvm3zUbC3^j3u)g;*B zJ~AMgkmo=%Sw}|BKn$IOE~ejv_@vMAO(4vFg%WcF))>E49joUH_SeWV*FlVYPUg@KPlo?q2fEYM7qaJ45(|EeK)7*Jm6|V7nH}O27*g!P zX_W_@{N>GHUDpQD25=)cGFn(ib~p9IMkZ3ip4O&Ziiz8_+u;v5vtK$Mt&z7#kl_nE zi;dQJcE^647=WdCrr*4ht2n*}v-tu6T+1;3i{0^Kz7Y3JwYa}zQ~lzv*Ke9&W&M7K zma)|nAzm)bCVWS#njWE@tYtEr=Fp(pH({)f6cUej?cQ++0Sqi>ZXcY^Eb^mz491YT z3BYXvG*-6a+b1Z-s1u-6ew9mWUZ@=DJwE_oJdMg9!B>@|idHqm8n52|RIL#$`qBX! z`i8vhkz};81kzVE&W3IR6)@tEuXIqi%Z9_amGu@^&{)sz2Vf7K-2GTqQ5qot({k9w zNVBWS_Dwa_G8g#rjlBnBE|}4?Vicn|YsfB|hWlRnDs<2E_wfuzd+b)iabn7+2Fa_x zbNf<>K6CWFOVzAu96CnAslY#rJUQ{-+v(%D>y_vEl!8GsLq>0?{K@cltXQwlhwgS) zG)29w80HFZ21>m)&aMjZPRk{JjR^(@iI;ol<%s1!RY$*@P!rF;R_#F=H|$WEWuNuq zE&w4jc1-zMu9QqUt=KdU`75E2?@h_p4z)Q`4-Qq=MyBwKS-+QT+(bM(w!Z9NRS=mc zM+~p<4O`&b$(-K*X1VZ<@~!C6ea-PJ0Ud={N~W2fxN81W#Z6!L^&VU6TxBrM){35Yf@+rYa^Th5*cq&o3zBf(4#M^8u?`c z-Hot$)+D1x^jXMR``Cj&;w4T${Tc8}l7%qfmFln9TdT;Tlxy~j)Rt`?5g5eFPR zxZDcyK3W`}VycguBs9Pm)4XR6D-y1D<$KMytj)M`+4NC6991!*Lh8d{IhWrE)hrM! z({o z2{+3wujxq~iwt-AeautI;cJ^1-w|{X8h}?jJ@ZC|i(QmagV6cEdHjxl5vFP{U301- zcIW4im&A*Ee?grPA_%f>@S3I4MztR|wf@;^UAfHz@@B=5i@`h&! z{o>7Wu&2g?I$Qq*b%H}}vtCnOE%Wvqx0H<((>fMctLgN_5P&@bVMWO-49T&c%~u#x z$4s#}H>R2%?$8XOn^rxVG&;OSkohI2=I!B~)IG94 zpAcR2A#T^?mjv>12(G8d@5U;$XiPoK6GgHW=aqP+v-skc;!+bl=~6FmdK6aL_X@`z z^krmPVqP*WsE*xYhg^ax(5d$msD}<{*P(aKZc%rC=GE;Geif+8|5EYRJIG=f>aZp7 zNoFTx0506q6{LBK>!zrCHdfpqJ?tkd7y z`VOCY-(jz`!h6%tvOnn3x44|Klfy&;Uruq5J>b5l3nx@4qm@HD&%@k6FO<$+}S@1~FrDj8fF zw(HUZzSHS!z6v1S1k2|6;Xb^VWFm;d;s~-%39ea zlsH|L%*;m92mZsbI3!2G<0g(&4}e#>8xj7YjXFn6!o~D-AoQtvhCgdi9o@#*z}xh? zwQG$D%9cQ`Rix@+TiQK(g{2V=X!v&$qb#FZF0W~MU6z@_?i&=GlK7noh1V-t#kV(vvo6jvdb}2bK${aN`b_-DlPeb*i`77?xzr$)Zr(X9M!T zBGvavUJ873!GuVng}}d!6H;gIyRvE0a z`gqT+Lc~X0-7KXD7auqNV=1!skA3^cfm&oop=tJ)&a0Jy*DciHq}U8uOmPz+Lx!`` zbY?s%dsd*+iJzMucc9~4*h={9q;2JYdN#mp@}456E9NiBOB_CxnE@z~?f4L5cb8~_ ztI#sB9-$|b%N-SxgND>mlQ!q!h+|~siSEnu!jm7SuIT^Lg}9?a|K1Yque>i;TNW6N zf;VHj3uNz?3#hzME%z>xINeZ*ARre;HOZwm|clDmZR|$JF`p>~AhQO1Wf9N&< zdHMx#1GpBFOh;X+(^tTL?<(S< z0(R;!roT{1{u8nG!1h(qf3G5y4p3NPn=m}a1y$wi@%M^$F&Aa`J_P@Vr zTa_`QDHfiMDQ)0bY}#q~_3z~j{PTRWycz}a!R1sFvRrpb$D(2;L0EZ}!y2+;fPcTv z;QfdFFFI?7hlL>p5Hp+R2z9ca| zV|rRqYOXV^NFIj#-FNK;>U$eL)(WeX8VuL+_*|E#*y%$TLc8zXqjK8yx?-1(xJRW= zm5s`WmCv2kYG$==c8)DVuF5_yHz#KZuuvT5bHwfAD__2>|G?W(iHyBG-wKfe%Wz+; zPS#gUg1;d2arTY--M`+o)BYycji>sGV&2O`wTs8#PtI5L@|Kp?b-w>9*9j8C5D_9$8=8$bagoP%N9C9T~cRmE}((#Rb!!f>oo|z+qfnraL zCOqYL?(OU^15y{aztg>B0@o<(6O(@M;^7OXl-BXzpUba&DQTNEzZhg(&ccfV?q(68 z95)#sMIDBpt)0Lt>?W8x^Z53R zUiUdD+Ke|MTIiWMVnU}ebVMk#|B!kmUXk(oktsIj3P-k-N|o*4U3<6cxo7wVV^H_0 zNsP0`6A-T|tP9P`Mt)B`r&$vVFTj+74!tL)$8;aC^#>|csI|z#7=-RTD+oT}f3+cy zCZh59LHj+Cj?2drju*)I*Y~zoW=Fe4JW#FiM$($uzmBp>dS7%FwY|wx{LB%K)fi<@>4X1x^@%($O2eU48++_TlS~t)hGs$AWS2K>A`P zQt+pEO&#Y$t$FWlAaaH-LYH^tE>Gj!kMG8AiQ33lEp><-${1zd5bhYaW;3}FwEQLtSg z8x{zd(GtXL4^z*XB%DxAir7>)+zaTtZZlseB-ov3bIy=?6?p3xLymniNR)w5X$AF= zXPiy>MHnC0ilfXWXC$_M4qspL4mB-Vqyvw@aLRa}=9lFzDA=T#+n|>KC3MQS?*DWiT9`%Uu1@-R(a(2P;|LM~{2D~ad zofkk3L|(ye5|3jnT#~uxRbf2eFpzh<&3V4+c$pT=nD!>`SC`#J&d^t`C0Fbj_`H4X z`up!4RW!-FZg~NXMe>yy$bBGvF0O3F3Oc0a#xHNb(ZZ66uqE&?pwF`hP%B-Yd4oHX z7w-H49Q@0m{~`rUuv^3b9ye5yaSVo?1Zuelm?@8gU*hahd(fA4#?vEYdx z!q~qk<|*eli=DkJ3MmmrbC0mc8?}8`k>$DY(LOkL{s7Yk@v4r5P@2KvsoQ9F#@=DP0;fG82+jrfs-!0w2^s%JJnZrPf6!g`Hx97WFt%>oWlCiC@X&`Q1 z297`A+-(p8EluHO*hQ@}-Nx4&)$2zlUcR4+2i4#MM=MkMm}C0v z;|t!hnk%;e@C>YXoqMJ=GA!;i>($%YDS42ylUW#H^C)(69iR}mEGbX?Cj>DL9sbLc)#P~9Rap0QQ%|R7X##byI)>A zg*0o;PUYQXW^Rf)tvERzcS`9w94xD?%W|;^lU#rLB$-Vv=a#-gV>34( z#uphg1FC3lCigL@n}muK?@vp=MA``j0Hi`TK9&5Zhrg%3UL8Q30p`84ZuRAM7u7N_ z?lvxxM#{bT2x={n8rG_}`m(FsxffuH;?lH{JQ^OJW&XN*DKx|B!S1J>*M?fIioySW z4B8EMrq%rNRza@b6ZO~zu*&ro@eEDQnBIw1orqLnUuu=-GsD?CJfHA#%0We4snC%IVq-~O-9UHj|bM_B>f;1AxX#&359$yud8(X3la z2|=fu{O2!e^`tv!mH?6Btp%6r1XzH0=l}wQ*oLcVE{)^v;zx(=xsGqw$s^^Q6#+^w z(+J$u1l7S>V64r-vSp^hf}CX#UTZzOAU+}&o;0S~Z(iz7$zC)UmU^Zs|MzQRbbOCN zl`&q`XT_um1REv;_P&`V=T#G0-s-g50O41q^taQ)`#RE}ptJ-GFiWqy5JCSrLuvvM zz3+gTP}}G1W)acx=DV#S(C>mxb!tP`1RJn~G*py4Y38uxk0F{4L9|)ADpHdKM78>3 zK>qp7g~jfMnKPvMfyt8uV2C89nU^*J<-#8IQWC)~wlVOE8sB~x3*i2M-Tol-gyJKZ zu(c3{vu+^XB5LpXukyTn!=Hg#fiNhFAciM5u=e9}ux-o%?5uJC+{C&yI!QLGWDfBQ zM|@T3oo?D;5bfN1li6IgkG{RSn+vFMLAW0~h5PnP877k0Lo96C z6w4yJ0}~@!xB6C>*BYs%nLQ#~z;SVfHE_El^Wg_D>lo5taknFR7(7wGJp&+#Ip6;P zy=^~jH%FBT(8>kr=2w+`OgYy>$(X*Y+q^=t7C8uGL|RPY;gONQ7xqkZs^I=Iz9XXj z@cM5gM2+dv{M31RXThn%uy@jMPbdrMT=fTHtm$mIE273Ihw!)gr&kf6GX2}Dgt5odQS2rujYOEWh#E2unIjmV zeS;emPCSor^@B_syanR|%_O|((gW{biaPQn9^_UyZDo{HK=~MNu9$xdslf3&{UQIC z9O9?dS>+rQVO56_hJ(bohkZ&HwZ~q)Hs~`z=y;Me+A}uv<&ZDNEI=3lD<8QRZ4x9| zIMxbdE_H~34%e5s`4Nc#mFqk>627(AB5Hc`B;9k2AD6&1+jmn!&bq1{n-`xRNDe$8 ziM4~*1_0pSOba{|L4gbkdQX1|m$m*Go|^^E&Dl+h5ip4%OplSO*r)W}t6sC@`NUxp!DGhY z!*@TM)m~RhCl9D=ECIHgE zh8yW1Z>Z83hTuJRyEl-R(Y}xxCHx}A{q4**VN(On`q1(`P#f3Z*Ft3~=hxS$WsvhH zdohJ)C2_r2#_}<)rhhlJr3e_J;fO2kXs1}Elp)tnct_1p4K~6wB;1L8NF*ctvVDoH zPZ`mVIQYmxsNh^wSZ*#DY!>UC9xDN3{Qu^$F1pQXS)ps6?}iIwdi4hA20uF_|Jml#fTWpx1il&1OHG~zkJlGaR^&kbGYhZvB! z|9I25-ih8^7^}XEx=8J7;tpdCKeC^R{M^n1jxh5v@1$GFb<7CLp#MkLTSi40wPD-D z0D~aR5CbTUgwmkWF@&Uq0YfOIgh)#dHPlE;cL*v9A|+DNohl&`4&5En@a=h?=Y7|D ze|&4rPuAjIckJuh*FKN)q%EUxxbJ6#8~R)9ZWOzxgUsa&TLJbY^daoyxzy&`GfW)A z=g)br{v(8OR)-@CiFW$;GUr2Vr+|}PKd>{gkH!KIZ!VV8R+BI%o7Bo$$_ zw8}zXj1(bM^ygLEucplUle{zjY58vZivc0S#bLtTFRjkWeNt`%eVKdxd)owVVrlaU z+C0aLDQ2b@Osz7wy@rFhFfaX61)QvSxYb2CKRb~dbyQL+8{$e|9ZI{K+qRsb)f5{l zg?ea)>Ix&?dW`b29%z@F)V+MDnfv+2Ka@ZZgUD=)^DV~ZoIgn9^_#rge=QX6nUp*; zZ&1(q{;Zp(u8CwxNyt*GUn}uq@LN{$^+EF0%mlpwBl3|-P2o0$PoEf0GT%dFHS}Wj#Px3h+BPDchUk;oWdvW_m7Qd&>{vNO542O8h=P3eXp>dOqqv_S z*bL2S`8cLsoQOY|G{kc9Q(X4mT=a7Z@4SbtN)R?2WJ#IjXL57VUQIiM{x=Ww_rLawv~Hb|B%p&e-+-x?vs4E|D2y^xyK`tOnbbILm+nP|+6uS@ zPdgpbc*k>Lp8 z%ofYDnQTYJl))_Nyq^r?wDi(N2dvBa)Wo{tlls1kGfDVrSjw<9)<3-Y& zNP1dCacx%N{TD9>+l&K)xG?1vdN%A$LAe7;ERl}i9KwR2#JZ42^8S?R@WNx3yv*Cx z_srLB5H@EM2GYI`#z+skHuRSjE6rPP(<6q{upH7d%^8HU2D;D|yo=Z85e6X>D6|c` zPdQI+fy@e$G#Js=OoL3D!Pk6?_omp>n`MdpbADco2L#b!q{^CZ=9!%HzSO$5K8dzSJf$Ed~Uw7-(vNv zv;U%_byNS>Mj(rYw&WiEJ!V6cFd`tT9Ed-Ey2Dc=Y}<~)o%jhEUZ4-*fs944a1zfB z4n_#GOB>uHv*p`>L&l=?J!&j^W*XuSDaxh4onR+0|3{;944dT`#ULW2*yd`*sp6nQ z5po~V$yBkkjy~?td#mSD5M4*L+dB59j1|K$_!le%FVkS9SYinVEHzhg))QGOO&im; zb7w1oQ1(b@yH`}bm^>0|V>)k2?}bj?!U60*X9P9SYT*p=GT5)!XT zBRF6WjD9#82HyI=c|t5e%P zF70Bw>ita}}0qDsL`67l`@w zZAjvmgiJ8!qJCdcR<6$V4`#Umu73P7Iii^@u&c-{+3J5Gc?dsIKk4(wrPLNNS~Qz)UE^OS1ut0EPQJ8xQ*7?oNL6(BK=-#nSCIMgcSLj{#IydH&-4WtU>k zaDvzeRDWGO3VxV%GGWfmOgDzV$(?pYau%z>OIXzQNj+R7^N}U_6x-kDCqf-D{HFU? zJubv~KYTY244)owyxVZPp2&|t?=JO*I!PPD>V9IH?QS>Q*MD1|T+&UPuBR)_?EzK< z@M81re71AGrA@kDODwJ@Nst3*n;$>mDkIW4@#Uzet39HblTfl z)os2e(R=TbkcHR?v>%o31ht-oy#|q_twGHQ<}}E!9WuW{@$T02>-y=f@m>gGPWePf zn(!nhTCLOvx|GGG^)pWLDAcKV-hA*Zk&-Mi4MMPV)V!%ZB)GoREs*32&qqokDai=I zSOHEw57NlQI|7Au@w%& zW-}DL7(x=zmmx~Ny7H-_z%}2=^TGVk!DPWi5N!=}QfVnGcL4MYGtU59hf-89<5o3` zUG;5q4(8_Mrpx2ST^8%&mbQ2&K8&y>mDMdz7-#ZRsBn}xW31>RIas;5bT|R>Df-^L zuW2l>`wl4)HZ9Y=m639WUDGenwJ5)`I5h&{_h5@j)RALQCZ#ZcoVe-OXF*Fo9Wzxk25CBx?*-jTCUCUfv@>Gymngc z81*uccTTv&53(BL;=V1AspMoeiSa3`V*M5{{p#TMO4t2y(?&XbTKy9RvzES@RNRjb zQWkLyzXTgZ*zy{kTkg84FDmuP!9CJWcmD2C?Kz5@A-xv;X3kq)r2AtUl4{H)C6QS* zn?Yo(wx^Pd?!1}K82WMbNLi3DmX}(?d#tkEs@``n!ssiy)xsWKbK6bACQqkPS3!|2 zlzs3Fdx3~FK4IGJI3p(&kQMu8gS)DWyijkM-CF4!^g*G{C}e*c{$Yuvq94W`4jO+mh<|M}OG^uDr>v_-w<7D{i-I3H=6|Dejb<1K(9=`Dh?&n1?j6V;$T}#?l2f zGhTF$ku*JvE;pBAr7TuYFzo46>zL{^k7>S6e1iuwOq^hXFi2e&hlU7_s#Y=y@TBh3 zlonfxb(2X{FJ#8PuNZ|c2KlsNZfNrH6l%pqD^;AhoKi~v*teG)T#xTOh+O@0;vV0@ zyE>88Vf@iQ`_~Vu%mb6seH~P6y27M(X=$u&&sgA=K#O?7j3q{T^s#n4{0XDIh`*8m z$M*eu>b%t?t}2vm*N7}BFfz$@Gl64$l9(NmFhb}7_RprC4^OK0OSoN^LQolrx6qA3 z@zLHC*5`^%o|{gv;xc9%m*yHY2OFbdE21Eb1Z2N)TgK$(0um|$sAs(M#s zpye}pBbU$dNm#6j92 zWwis{M_G#gsR0&KgY|SXi8UGgo}$RCeUVSDdlD)d4b;wqkDu&1NB^Ry#=HQUivWl~-x*N&Yr(zwU69I#2R@U6Qo(zVDE7Q41NIYs-KQyr*)NDFDa?R5`FE=K>;pQy{{Ir}+cv<9W zHF%hsN01OZ9&T+B3KD_7GI>iPjgw}oH&jVfQke|9So8iHqRTmxIRlNBVxp~Af;6tz z!-nA{Y3%wuq*gOc1W~Vzh2-^f%mm~Up7KO(TR&L6KHV{&6xMghfV!9$!-&N=>`FlF^Aq=r=+GnQb>%r44D%WQic9?^} z82$~~HCmJiv5Ni;f13J;t3@1yiBge;u%RPdW8E&pzj7NS*FfA;oI36u|FU8Z<%_03 zbf8gQVit?*^t^@QLdeV$U}jt0Z4nYhTxOnMubpY3=4=8OaayMkhc;Fs3%UERBTC(IKcX6O(C^R?|T< z#J}@V(+kqFWxuu}olL&Gl7As}UhH=|>qxs#xPRYqpbc|PNmS^uS9oeTsbFzdfUiwn zOyD}idTBg#KQq?$qF=}|Dgp(W!|h*%oamf9`s;u$l%a4xfiDGQUBHNi2NJH3o{0W+ zIuSUbk|1kkjPPgJ`9Z$AX-esV`w^~haAL5eoV8X-yc;CBOym;m2R)~1DRlcYLv<&V z0mDt<1Nm(@Pcs~RrXCymef!wpxTlws-~gE|a)YahA@~kLeEK!_EAGYU#Md^bRSL8E zIGwh|Q>*>#PHty;%-e+iA1b?ESV zSp2vr>Qx}PU=s5F>hAfHZzR2pgXD7Z-4~5Loc#5f8J{erfK0X#Qx@o99=T$LQEFOM4syppD`Kr;!?w&<)eq4Uf1HDx{Te2ER$peAmw7wat ztIaJWRh?Ew;RI2Et*93Ark8{F1UxX($%zAU&;(naIBZ9oi_FI4%(OsUMciOpMj$q} zoxVk9EL=K4t(YzAIuXy@KxvSM0IlaFrHF>Rx3a8t`6}eV^!VOku<#JXaourmV@!{n<`9HJeHM;P+a zptI|QQ1E)Dx+K5U(Nha?cH{vcuUlr^7Q{V-M8G!7Huw#3@+Hcyd`O%xOy5vMjqe$p z6PM?F>=UGkDr9)ER<8F29gumC;-DI|J3jld>pbLxMP6zvOEAx}Fpowql?n>5vykoF zz_#B(jLxB|%J)WV9=^pXske=1go^!rNOWF7Tiz-^3}1%L9JTsJ#;pp}tmtN9HROZq z&|Xxtca_Yy^&Yg{Kk-65E}ORgT|qk>EI%AP*@BVYx>53uFNa@(NGg)3`h$!799&`?T&xp(nWPF{+c>bhhw>&??Ut~FkID6l_^nJ-K`pn!-DlT^-_ibPU_l-F^aGB_8;8f8|JtMf#X-)UFt;_bR-UYY3Qi~pkqxLQ6p#geY|eb>+OO==mT z56K2bvBs$IIl9@=rC|ZuC!26L&Hm7rp2+{PpzS9Xny%3E>R=B)iYDveW5pETqJ;@Wh&mY&UQzZ@2ZH)@U!|H_=aJdrzpT=w4T8y8U{1$OoLtUVmJUrlmg>5zrXx?|~Z zDBYbTBoEyb>uM$%x7~E(#PD%-qL$;7RNO(ry1#=T)FI-%CIX`5ggt?i_G%JHe55z zlnuY2wBO9l-8{JqQ7*RhKFq(eI{$iiml81%{{c`rjhlY(m5>HFSq&c#j zU^v!sQQS6P@~Q>feC2h-ix`dM#^vD)qz-9e+<0&Px(Lro8eXL52)x%`_7gj5;Uc_OBj4`iINe*9U`bA7y>~PF3}9n zgrtTwby$YD?O_(=O#0y2neF)#tegCWR!ltwto+6{H999);_Api?&M6zUT=AcD+4&U zGFEm+vGkWM5#Gv@T%F5-L24h{EYU4c7R;`I?fv}RPO>?Yg zwkL&4ANi_4OwC1{e`rOevuF@>1&n9ZTj{&O*-5=dRc3CCm+rU_`|TlKbH4!{gg^~^ z35pHa=dTu3)$=^(I<_K=ZAiFCLDwB#hqJiAdQFOL$OoQl$x|FSvtb^~o;U7^Z=EF# zZJE~$9(MRKV3k=>AzhuZZXJ@HQ4Lx36@|-GJ_C=RntHH%>~7>LD64y7tYHE;%r;@7 z{dm~<*hhQXPg@BsStXwrJbx;)w?(qqS5K< z`!l}R>4uM^|FyMH&HB`NY+NyTt$Ts4p{vlNKGCDLjwrj|z+#7;7wD;*qOaeoerg%? z`ta`Zzt<$*5YLkEV1eOYKXLa)=&%$aE~a!q^y3jZt-7Y34J;UkVi@}rLlU+pHK-Uy z_*7*2VO*@FN!Jtecaz5k&wdjV`>H^QF8)dB-St|P4(7RVZh{9PC4+xEo14fZB;|ve zuUy!)PHdnz1tnr22d?C!o#!X4BWiLC{LkYznmUau6VEs>b0r}TIEx}V(@OKuXuUV~ zWP(C!)x2Ij6o>3D#5Jy--Vp4N_`QYV;k+hlM7>0<<;Rl=L>E4LdAxp0JZml5b|0^zB{FJfmu}8kR(b050Sf(yUgRk0dw03l&-QYK4dxN--!^rQ?T*S?Icy<}boXCsFjt5$kILPsqbSgOYIKxK zoyVr!HZL_GA$@-19J-OL+F)ui^69Ywq@751lI&QFll7dykBb?$7rdaVcZ01@yakt_ zbjC-vg^Y73*}d0oGwJf=a%pn2I`oI>;EuxwmrgF>_KlWwq*2#?H0PzEt(ts z3;OT)tZr=PW5Zd*vECqWA;b8aMGwPLfVX#BGLn)CphR)ce`d1q3mBK#wjD|tnhCS5 zHV7*3s3<~QX|^|@p}LUx8*z!bG1zFufh$r$&GHELBgSVL+{Z10Cfj;Xs@f|SDD9;U zx|s;XYUZtCMp#CPD6>&Uywx_<8=I6@N?$e8{50|`oubo{l&Ee#yhR%@NhQj$M~)KS z{&07`^gW*4#*Qf?KNhu&_bJ5rC^ca6~j*wj`eK0s3h$~IC01wyl zqpG53aiKokE2BE&;8W6hlKe2f@>_7_(ii4`p_>aPorm1G zg31Sjk85*iv7CJEAPs!FRR+UmW4g}nIz2$VhEloeseXf$tXj0CoXwD#kxJxlnxxCy zJJROs?Tle%Fqu=B5d4{BYAj1&RQuRWI0@%H>U9J2H9+J`Dbnb(it?cQH$*~WX@_?z zv{<`%9a9(?%DmdB+7oo7yKC<$PJ+Rx6d6`H>d#SD>n5Y&OzzanC)=6^i?<0!o^ z=VtP4l`^J2KZO^|Gd&M2=zW$ue zL2kzzSCAWHW>DNu{_@Wf0q0pmc~%q}97MZv6*=*s+ibS`RH%wPssdzM>uI3bRUx0}I;>!}YRj z3`|F(HfZIhOR-sr+5MStHpk9;sE9c-#>Ri^|CIGQd2e4_z(;l-#>z0F?{cMEWnuN3 z@maFekJ`T6y+_77Dt-|PNvH_pTOJU!YG0dX7M=d+*E>i z?KW>64R$(7SV^M2m)zJMMjY61ua-{U9=S}?(mPDiWdF}??~PvaAZKukruo|U?#iTo z&O^t>`QBJ>QNLSZPwXhrw9CJZAjzRV<1f^X;X#xzuxg0w(LFBR&w9m}+H0>MvovP<(!@8OC4Y84>c9uL zUTt78x?%2m%`+~SRpRHkwk|Gzy!dzFJ5661b>EA(y9Q_LF#LTlf7ITVJ-sQ{!2|8b z{srTMVj_4M(;<3E23IzF$4UqsOSZ;ld7y~ISzFi%B3()>;8E(W?kbMO0r2I<)2S`v zHyN!rTYqCr!CmGcniGzqgfg;=IC3g5|5clopcKf=9XMJqGCO~<$4JSqLCM?Vf0Sk3 zjx+kINiFv9(y&`X%Z`B)21*8g@s6w@jXbDWkS#Lhn39`FVY8ujR=*_^{|hnZK7jcw z7cg^q*5MTQ&TjmZI=IEJXu&NWdU9ol<(J(8f-xfbjW^EjqC6U@og zVWwh8m+>rF0f`R>bulD#v}#q%WMsn7Mds_KN^n-iRWB~uE}iZ02jwc7_srL05{dDdRIwMA^NW2^@iagA*2V_=#&Kki$xekb zdqnP|=m#VuY#o{?J6bO_N%mOfA7$#Mm9}p{3KJORHM>NI_sXLGmhVw}iZHjxI8Ihu zBUzLQDH#~Qpl2w|N=bn6xU^B{KD26?P z?0TGTd^2GXa*PTck*R}b<;HN!t;jTIi89UIq58r*oF2!@9IWZ|sbk`uO$a&Y_Rx(( zE}caxpW6kCJJYg-6{@%Wa&_I$Uvi^V?$jU1l4M&%BIVfZ!8nKg?>j99 z8U5$3ujvs9%4hAOissINs?xIxMA9zD3MEfdVU~npBeMP=C7|oE=JmMTr&qq7)T!a} zPm1tLd(ULc8gqJ&sr5>Iu0q{0uyL_^ew4?U%|_VK*PXM@a~SXwUF?(f~EE zgIpw?EQEOa&K8bwagTtG+l_KH*HdLrNPq(n32fBrB2TB zKMIc@{m~mg^LE)cai%9ybj{9*%x=&u>TzDdgL~lJOnfw&wcNT_#%Gh+<}FmM)XXML zHem?~#M9*4`7yEXZ7x)YrlsaVcyalaE329M!PWO|hu%kf1A*as^Q)srog);Zv)KLh zCJM6&paIVK$zC`|>eLiSXfU=^>aiPoAT`GWVMmvXcUWqN$}l0wGpXHYXOvk|ycq?Z z|6S#jZZ=xueN3I?Q*zR%`Q~wLI~0ExCx8+%A}MbYrv)G3=P0OiJ=%5K!K+mLPbYB3 zhaWV^7I!n1JC1Gp7v6;W{2F z3JNSpvY<4Dwb9Sw%3+j$)Sd6eElMXZHV~F&vS6UM8AV~v6nj9_eSIKiMy}se5GP)p z;sT%J`PHdNn3q*<-j3Ch6{SODf+%UQ<8bRz zY>(?zSX}O=*=zjqO-eF5Qcm#Bn@u!$+M6dI4L1KHO@g=a`~*(tlwvoJO;}z7sB1iM!3P4Uzf*K zfU4Y!&ygrcM>-+F7xsrdYKhr6Hx|@MJ~u4B19BH7xUVireH^7Y@8r7AR&{r2B|ezYFtIj7zd zL~{eW#+(sh_M8Y@C#CFSClH)5*+%nx*wT+Sc-dz8kwpdaewmY<=)b>y$bbG#(YIUM z3n`HlWb$*mS<3H=I6iT`cgwx)KLn(@Et!wGDGq$>n{R!AwkhUj#)yP<@{_~2IoXj@ z6c1l;Y3S)Xj1=6mI$?eHNt&iNY8+{~d|(cT5#oPcvV=GnfQX4v3t1APj~BEM`sV<3 zsS3EYazP>?%yloSG+_7j%Dg_tr;Rihe_gwwAMVCo1y^68uY2rz6Dc=wVEB15MMmU{ zf&4*c4~?Eop@7*tq5oa~SM4TGX1Hg)v2}XU%Z9TnF3sn?zzd}h@c6e}^=^{IWxI4R zErO`HE)e*ZDnY(_Fp7En6pjs2WM;9T{G&(1sTZd>PI3VB*#q?exv8SWCmk2911ROS zX%UE=f4eeSQ}M?-!y$dE-@RXA2H4=V6`M8wt+f9=aF`s?mXP}tUVJB|utDvP;X7`6 z^^M#cFKaa*V9$sD5olQrwz>g;#cA8`n0~GW`3|2gm?aYnETHzcaE)VvssRBYy1g%i zoVE{yo%Vz1xSDRlf*Kl7*yxHlv4PMK^0pCpL^1SHJpMYlVr73Ja)w%de>SmktXV+R z{gQvy755ee>-(US^=QM8NALGJVgq;0$L28e#fGIgXu;g zuZDum;OHTrjQiv6%ug(j;!A!Z{`1ISLE}8rGBH@0rw$LNDql66p=(6gvcV<>pD|c` zN9tlApY*KU>L8|U0`OdYV5+m86#{X0dKAGoOZoBGeyViXaO0}zm&#&)qCUUigW z=3monHUS#Uet=#$-~;A{^3|xtf%nMJHzzZ zC2Bfeoa5D{s;tcD{|_DUA3zGcQ;tAyZMir6Q1_2L%`pGZTS?ZF zkbSv`4BNXy`7G5U{%2l8KYo~B_O>uz@-Ej(7RoJu*Hg&#S3D0q335Fl#O;s{f^a(J zC$Y-u>FMi0(BV9zBo1;6Po1jUSYT}cE3pufZk2QZR0a@q>Bww6^K|_Nyp&2QPMrcy zGrZ3aes5%Y|9q=z4MIoW+YDwe0hO26CJR&wXU-;T zfNzO5s(P6QIk@IUwweb8`&(r}2I2g1q|3bsj{kj6oN*a}Tatj^KQLfxIMZC%1@WQA z%H;~@E4m&H_168@_5f*Q1kbUV`;~aSgC~1Q4)N5=ItYMyC$AG8pRBDT5uwOJTlh*1 z)oJ^9pOO3-QqQabU9&R!Im2$E>a*0za(+Lkse(axpE*3TW!;P+S^WC@X~m_QUe5oz zYU8AJhKt-QK6!X2EmZ-&JlUh@1KA=RECG%G&DzJdKr6 zp&Cii5m_v3~{G79W%yD#;aB1n15|7NzEg^usp& z>Cd0^NCvnTgdgB(7{{l;#RTN1s;ynid-y?T)HsVBteyV$<%S-9vE_y~NnK^*NbbU0 z_y4y(7E%*60C(595wfX4t(D%vlt^^pY5W6OI4o!whOOh$zTceqCY>|xF_Oh>89Ir# zA(O(Vb6zZg&GeDc((T>cP@kjm)sdpl0g~ts!1`itRIV?cTdVjTI5SRvvFUEY2!=sb zs}i@o#wQ4E9zANYDSTlDL6Z;roLUvrQG&eYkS|Mhh{a{}Iq4U&FH((7HM7T~8sFO} zPBKeU#ELRkIGPR*0X8F;&hek6Ji@vd1E2IJLgwt!qciW1r)1wB6x1Ht*XLrVxU;XR7F#kysijEZt{XBdY$bQ{=HLq z1u-NIxat^wMk<>2{$2nOSbrXs5}^}CgHhf;iOH|*QC30n=1qb3OI+9~4n>!});@nL zR`@s58tw#?7+9qy>RNPje;>V}sebqB&Gfs$^}Kz?ZtAV6nJ#Khqe!C1`Vn8*kYES< z#VUv9{dbm^W$lQ6A_C-E@?Sqny3nehp2FYN>#(bdcwR1Um6mQ?Y=vx1i=$Px(L~?1 zc#mF|)Z7jpJn+^@EVsGiDEPIpcog}4F(XukDnBu|!G$9C(lT4s{Dau}&AsLq&^4ER zvh}~L0Lf1S*!vwsq>L$u&-7UO2f#F@uUldMp@KPIIHs=H2`F{tpdm z-MhNILtxkk8^%k%k4%t|Pxsk3w$$WVbt*ktboC_Figgl_B)#1Xb`rxc>(00yz`LB} zBG!!@0!I9G59*J|*$c!2;&U&At1LL=#@HVB8h@X>n2y&H62-^eo)3X$`sjLp5nLVk^G}OOCDcKcjcg^9NJ2TjY%8Qj*IPywy78JxB=bhijlm*l{w`Us zpz<3r`6~CLPmZsn71MFCrS#IF_nIUZ5+7=VUS5!Ra`OCmFyJnD%XMFBn<{1A*#pxs z2JKS-*og2O6pP?Zs^kRx>o%(RrP!!tw*%!S!U7k3+wsy?9p`aWB3rub(MqWV6qJ07%BYWl!1ITZTU)~23(yX=z&bjt0bkG zkM2aFkb~dptfB!g>L&$EOLIp>pH-n5ZpTPyf!Cnw;!~1~U+gDU>8U{tUzxbv?9G1< z*J`xpPT{ZQ2m*?~W|2rAFHEMec#Jh0T(#*oPwNVDaUr96nFGrDSD!xmuFl_$7w&H0 zn@R`lI}Q!9itqZJ{u7kCK2z(ASpOZasq2}I?qj{{UE;l6 zn_FJkXbtY&&?~%9R|=2!!4&V+wQvt4>7sx`ZGMcHR*KLa-fXY8_~ItDm}>Q(W(p8G_7{5!mqXbs6=0vA0ZQYy zv<|(ZY#_w_Ymhda zkEiRe$g<4hc2IEt;I#{PRHBj(RK(eL1pV^lE?SmqAS+uKVLc3ukTkXMH3Xv_%!$DOLCJWTfRUI$71hUqzH^Sq0|X6ZKIR11z{ zFOR6*z|bNCpr2`AX`f;x82|3;i4Sk*FD9F+WOtrh4Q(8h-P7hz#DRJ{%E{WC-9_li z_wI9i%k4|v^JoATTfQI05TN=$W(kD?4w4IbGW-Wr)72$fBa|A-`cT$ z=ic4~?qal~pSO_AUR;N;_e;_vA%kn*kGt%4Ju@@cp+92HB27_M13N# z=l0!K3K6%w`~f@S^@H8*AnXp$x2wM!HE$xnBVN>~} zjAbTM;Vj1)#D^RG#JgNceD^shCz1{*HD8kO+@8Gi+tlg{oNNc;DGnNcgNodXHd@^= zn_1GZB|os#OYQwEf39UeqH`^gJVjK$k^v&XNcv><*=KH^KNsa4n#a*8iq}XErh|H2 zlByE@tA=e-_&RV@J8M?P%JLZkLWqd$e%l|N;w~KplRPI~)P{h=t53UL<)p@U zHXX0#TA3946f_eK&w^Fj4He5Eh{Au$fo5;C=X&IhiDOOs89vY2eUKvuJB;$L+aCGi zKN2wp#w2BqPcCqmrvZ1bwg4`+4?p+-4ESUIzPZH(uyZRFGQpFS(E`-e(#!0pI*T;J zcnp|-m#ZKEKk_UwLlHoi=J2~4g`;NbVtToj3t&G$uc5~~xNrrfdR;w~t8L^06*q?8 z#ZeatCU4*;0j0Tlr8fyZj7k|FuW>|!ne3L^jyd4-zntcI;77*JNK{OpnYzp zmV)1XG7Jvt7jq3OJJDA)Fx*7>zB7DXuGFNa=lR@dOtM?R-e*kq_r9}1grVCp5VH|3 zmWGL1*9Dz=C*8-?GQ3ZZE!fiUH!L$G1eNd#IPk|n-Ks0t*MUvvX!<|!Wq34~=eYLV zG=R0a-eHEp@p19#3%w~O9JEMk{x8moFQJjQsvD)8_ zesAEPJ~j=-Hy7VZV(N_NsA+zB%u4aY?I?I=)cYcEZWG(XQfwe2irFtTWGq;X48{J&qn>n6Z zeIvMt^xh)Jo&G&6zC*V|qR z3~7-&HJ(>}y{;^e*<*#g_{2@e>aI}>T|Cy*caBLP~vX_i*iCa`5#uaKWWyra!uXE~#x$6h_qr2FX5=qXwQh#`?1f*7A6MqK`aK9Y{>rS*BaLMlh zrkd>VOyaLsYn7mi?e2EJlm8GN=C%ed!Q3Ow#I< zI@toy8iNb(edvbm6et*k`f3O^6i@KeLsj~M{3|O!icouAa=1C=n}4_UPU;WC-ScOM zWI=8msCth%YQ&<8i;_v-H!dMfCiF3o(15DH2sYy;-45T!qVe7;GrGf5*MPf7Rpt?u zhMw>}qsGWU=u4%hZ(MZJaP@dFx4W@oc<~1e3T5;!W)cQeuP%a5os8dLH)cVBXj0gJhhnJ0MwdU6lReJNG;%s7_@p(*15Arv zURd;OorUGQXP@eX^khb2N@1 zndR!S>}RFKPF3!0A6Tx`;Pw`X`MpAXbL#1R5DwS|&i{*X@qIY|dFXuf(mCkO6Y_6= zeTw{WyWex_Q&5Gp;mTL-_<`riN2ysX*UJC`h2LLXvkv8dpDsb%5oTT~Po}3nIOPfV z9Qt$3mgjcAWm}mG!`7o0_|3||WPrX!-j8X4T^2ZjE{4$Rl*%NBQL((Z{PCeCaE+bF zF;)_rvPB@{R+|29V`23WJ17zCx-0nB=`FQV!C>bW=%c<-kQL}=cah251@_M=ZKoBM z8=_~?Rw+LhHJ)+ulPm)P3k-O}^=`C61_$r?gz#X8V2t((|%+C?Nt@u@z%o&jyf zNSg`fn4^*$hPeS2eQiR#3sxSMIA6YsY{Qgz;2Be>l2E>diq^sH|jxJ;sJb801XO>hWvam>+6&RI{3YH4bC{>vsW1BFj(A?H%Zz{WpH^pJKjsL!W`GXd- zQ!P|V<+(&{my@JdV^2|xOTC^A`!G!Di4tG_St8jRR*vwcJ*U}Bk8wK>v1sQkemzGO zHq&e7az1!xh@z6(o>H$Mk&x#r_*43!x7jesGTVEJmKKD0Yj|^Hq7yJ3{jADmu!9Pv zasEc2yR=F8ChC>Bs!ZTjZ)+q0{HiEXGcyvHObt{J5n9S|%3@GL+r)>pSr0RN^Y+s&x4w zfDxaS9j)3lN1Z%5fq2N3ilo|mM$!%&6(jX*^5Ti@R+J;YJ_=wT=^%5%JZNB(wWOwL_KM7%BM*+kJ^vpDjBKR@%-sJWkVL|ot}@_O z&7EaVQLuc$U7hmWfw92$eWjAzmgmqbt5MIrSR)OpEiK`(B%!7T$LKAlwbCEpm`Jl% z&lC)-wD8NO(EtCk`96pbkYEJ8D!io}U-Cpe-Fxe6qaOhux_)vz#lG0Ym) zQntzIFN%~i^FjUKnAT1cYri6?Z4Jky&Xa;8dx!ASSoLgvcFUQE6XcKDnkB?$36E8B zm0(M-hqFLvnbIv;qPShur$m^1j*8`U2CPMvy8^9q+m%Dpob&L}F#*2C!+-J=&GlpCrPh_Ra42=<=?*{5hQsT5 z)X1z*DFnfCz>4Re8728rJ+=AHE6M(3Z?YT08=!De@aIeAaS@Ev#9)m>tO!#a_{!#x z=-LEP%M?2Ss{rTX#+*@-p_S`VVC_+?AjaQt>oI3 zX)w;hIpry%LaWy+XJ5=f|6}mO|6s&WLMbXWn7yxJXKjtAQdR=!mioYQlsa!k|N1L8 zA-|Ifujwp=#2v!TQ)Y2UiG2u-o+q^TQITXwjpKqGtd8^=*OX(C>?}O? z&i=Qg6mW}XDzhn6Ghj-1!&>7==nhRoeA+}qC@)y3N*6qPjAC>ha;nTW{L5LXWcFvC zc1wE$IVj7d;tB{pDaK1^PiO zub?7(Pu4S>g)9AyQ;Ee&WT>>?EukzJ6MSaHY4Sm zzZ+$q?z?Fm+USpSVx%+C6Kp$~us%0TUH{vs)D8LAnF7^qjc_L>ZW|3W<}r z`xR{#IrAWoD7Bq8XI!dMbg^TW4Ts|bJVR6O5JhLCe8qR(hepl0`rXPwbXu5qnb8>R z-c{^L9UB(yw8aCT*%MgvYQFiNTzkHG;;u)K0!n$E%69;&O)q|FV!qO>Vm)YpCSqYR zR4Qs+6Y}drlz>GG-Osb8bk~EtB>coNMFoV`?-Y3LXQq3?@%ZX1H?08>H zpV=?RN44tXc-5!;*Ff=yeM(}%V~VQE4_WFtL%-j(+<$gtUD}KokgR)y%cU zr&`^t_X?u87>sEjX2^X)S5X+myR?Emi9KJ`SY=|x=QunX{Ni`$A(K=-K^x1cj3aMk zm<@m$w7)JXhBA{erjYX;kt#@G!+qAjk)OWq(LxZ`HAgDo8eugYLmE>IiC9!6hVWJD7X;BbUT^&0VEBjfpVWp%iY@~H+ zZkg}%D7e6+$|145qP$}8&Y78NoXdaV8CSO-1%2F;6Xi>VilH-P%kSD0BlZ@&b)aGhv|&+{lIexvUzpx52S_Yd3} zN=tt?;(b6#uF6E2X?}6~ZE)#F&(IZWBwlrQJ%QS3tgY)g=G_*=swhq|O?Q@g$mEyp zwP9w4%WdLG&ca!W&+cfr4A6hYGF#OBpc4N&Kdp_G?BrdjqzJ7QMEfWlKhGGLPN|92 ztJp2N>I2dBNDf|u`xH8cGeIur6|g>zAHzR6_rufXbXaIO_U%PlXO7ujhYp-U-v_iE z#fJzM$vv)Lte3mbL=CSq?-_7mL6Ekm-`=+}+@;z(y8T#MR3It*M z#75NdPy1*`+(Nt*#M~e9=-2H(M61dJiZdDqXddsAznSK9?S8{S8j16`1L8LH89o5f zw4#@w`~1~w8Z|9|vwo)_C!Zim!MgVClRg2x$n@*Da6@!quV1pHRkg#ib2Zodo(jb! z$!zZ+aiXl&Xvb`%YlCX}$W{Qoa7J96XRRWjHW-?r7*sEQ0a=-f1aEp8dM4BTZyS4v zPhJm#&SOE4FHrkA&a{?40jVk4ApXw$g;V7NPRuKPD1Kl5b#g_kpLRcY6-d~r24GlC zx!?KHzLBdti5*)uiDaB3dYklpNjZOBx0Y=GcLfJwE|vDDdIFnygLW)T)N~z$wm0im zB2!0&aSa{1zP-gxN{w-PntKzW?8SKl<|B!6$^G_mL6jCiIFk9-8&%AMyhd0j zD0%PgLuo~w^2ybJuubaHT!bOh1&KV&o4}78%y-Mh?@6m)e(p{^OCSh^n@iVQbX^pzdQJoVD z&rGNi7^#TfKLRy&_Vu9Ec>G>M(*x$Kw|G8oh*QC_s+eF4%>@VDp}C+nkFv!ju5Udd zjDu$NR6YsX>!oC?J{y*kzYMJX-uvLj81dbvC%8VMjz;Vf&ztqF?O;@tl-hspOz134 zgt<7b@Mr7(rYXTXUz%ZR_YWgfb7|9GDws|JDpe9VI9Lh|ZY=<~5j~PPrCr%1rQ2kc zA76~0Sn2krVLw+vJL$B?QEo8e7TQfxC!8PUrpi4v9CIPa>xBUWk&X#(pJ8Dtc@k2q z23?XPqx2J|%r3us?DNBoSePSCWQ&{L$a0&UpK5Z9(!a$x(9ZvSH9n)Sg6ON;^BvB5 zS+->qYAIdo!Q>wH+qK%S245b^zTx-g7@k@g6vU9ugFMO)H-SgtLP-6GMd86Vv-_#1 zlpKo$90tt2&fRZTL++y_2-gSO;p$Q;)!HiAcaZ*ic|9*<=OnvTW9t#rg=3X-kKO#s z*p&`==$A#@Z%S}6yEnZrPytC7bc%59Qxrh^poy|c%!%X&B#%rrUmA_MGb{hf`+2&Y zbPDZmTUQndsd>4dt%m4J^(ZN0dFh}V)Qi=>BO z^k+^o)rBa1k0(!131v5-DSErJ$_P1CX;tjF;mdO^#;2~c| zb|C~6MB|s6AOVeq7LgoV1@)VQJEjA^uC$etu}g#rNAG8>zO%}MfbV;0<}k;)c@8=B z9JLPwkVCY&_2y(ahW$N(vt~6^a%pYE!)LX)_{ZHNtp(((%+#$drpAFC?rC!WOGELnc-bvlW3RFn8@q7V+0p4V>(RCNMFmhUm+uYHkAk#b0t$h=8I{C%PNk0Gr+;p*51F6oUI z^c~LSb?*Lq($tPNDBM@$g7n}G@7jZYuJYIfJH8Km*YD)>)KjdML#yGL5 z851oR&2vjq-Ew`Z=~>R1Y3mtbA#?i5C?Y>jt;9sde9y8=@QuZ5gBtm4&^sZ?mnGr?Ej9?C_S~9n4b?VOAEwbK79q+oH+$|72 z!O~&vdAfQV%mDNic7VQJmPxpOUfVjcg>lVUb}LTf#smtMa!pfIaR+7 ztgRajV+TWkNqMmnR;K@idF>mBqR+`znvCx%*_9a#w|j}c#(F61jz=K4wSxIZ8q9M;Sj z+n-Tg9{2H%%@JqB$*T5u81}vKdOWVN%sIP-^k?hg9gMKvm`f(@i#d9{ZQJXS$)6K<7f z$ve&;;;gQBI@`%^K^k++=cgCv1d|z?P8D< z?X(p?6qAgQViJ^)IL_CHKcDQR3p^gQ@)~IAuddiUEE7(DfZ48{pbFa|S9U}j9m9R^ zw{sZOBFuu5$0^d+*ROX*|6qx5$*r>XTeh8`BJ={k-9uvOOhbdhce#)c z&djY)3pB!F5kbb>|Dg7{P5XSZhuhfgoz=beOTKFs2y)g9T_mwa)1_Gtj^G`&CWJm>xfe2@Mn_ZCX;GsxX@S;kfd++2?r; z?px@iQjO;_I?l&K{Ie*dd(;<-Zxx18JAB^c0&JRKra!9A&9P-1$x-h5`Q?Dg`bHo- z<#O9)EtBsjx755H0>$`nkLQ4?BcG-#$0^sr0Fw7Ly@ z+^XyI6?6MKeTU&+uQK=TxQ|uY^|W=?1Szyo zQKff~-`5B4rq{-KV{4#xX=Jep_0U|;C{ODNqo2WF?Ml`Kx|0_sJt;J?gRwTp>R!hf zjn`SkwsU=&aWy%bR}w=7g4-{Yi{BE75?n=Cz3dYU7W5ya0`(Q%(6AVrwh0HOL!|%Y zSLQg1SyS2j_e*9Sv|tF$ub`b=X*$*M5|%wato|toc4=KjD9p8>1Ip&Mo6-KQ2nJ|n z>_*_hhPApe;hi~IP$RZ|&@-&=B3|f&GVfjaJ!QX_GRGZ40*CKj$rKk8kT<0uEX=l| zWGd6+=EWHhKwa<^>_v|^cUF5%y+dA9)A!%1YYbJQ+%C7lT7 zXsoNsdofR;=zRoB@3Qp#>`>pMyMTuAa{C#`d(Ria9Cb^UXy21YX;BhFzf$}_ZXqA& zzGmtqUO)iI6A2GUZKXydtk*6a(JzfAp%oTK=})qsC-<8lZ8;Lc>4htYm3c8Jw_6br z`mSnF-X$jd&Z|^;c$ksvsUy+vV~oJ=lr;2i&80o>xkUU_$Z6Sr7(OvEo6Q9Ll-2C; zU68-jP0~Jvh8!Sij-V=O>R=qCop2RJ%;n)2D z@KGNFXL+bidGT=~Agrre>&yq&MrB)&((!4un_tabqgV^{FUY0StI(kvYfBlNWV!Jy zdi~NP(+MI1DpP{8^(8#+%%$bIkx@ht`*g^znt~H!ucYleAA)mgT@h18^zdi=78SjJ zcIo-eF?ug`zN%Swcyv8LU=MGqvz9|grZ(RnJaPyB78G5&>ERta(2$Qa z9<5wD+i)(I?Ww}N+n(6^U$nbkSwMWnW)?Tcjbi1)>YjX9_*Ar#RS=k|0}vnEJLnGa z|N8hq=n)i$dn@#KtN{m}+XWc)GZSdXEsb3a! z{(7K>S2U-xi~r;v?nb`k(@b~M$;eY z_NZg6tGr5(##oVRWwZ4MrfRf?K@WK=P-xv8YfiMWY*pG^n|J5gQ8Ps`W0Gydu^FPW zR#)*%^%IxEjmwiSfPpuu8J9|d7f05x^s1rMKWiwrwT7Zr1+mJt0E50Zd?)tE$DWs0 zZ0aKq*hRKA((cc2-p!V$>m0^*Ea0f|3Io0=BX3nBbq~Vn=;Tq@S00XT=f_2}9;xR% z-WY#qj)?FWeyDLo+*pf8k`QMOX<{Zvu}mzF_s*xLChyHJ#-5~`Y0ro-1xo06Pe~4f zrl;NLw+6h_gv@T0Nt1NG4Zc*ygw0_BS;WoU>B6M3Aw1A93}4C!;Eb8_8CU|7kUe(V zFq7(^J!UAdwZ{yp6N5sQ=jK}gdv|tf&kO=BGJ7yQqjdG$p1U22^Vdfw%VdbOfLhcN z0n{SqNMM=b@w9z`+H}B2Yz;{^oarvMGxrcjS`VMlxYPt&)bA4@yXz?6N;D+F-D<<3 z8_J7w6&5+xP5&D~DzE1cLX!)GS^i3F)}*us@UQL>2jVANLslTrWKk>|lC`KuSPfY! zu}rGT0Vg zkoJHK-pxNEjk&Uv62K)d?fmJ77(6g^c2P~O6Fh2n9s*8sFnqGvP0?fe8WMh6#&1kH z`5!@0C;QqG&5J85fyc3XG3R|><&VZ@Ds0W3Nz@;HlK8amq8Hk87<8*4b5 zj6Fg=nZ;GKDw9Z4xHn#T1;Cd00{!5!nF11R zC&UdZf(VVtD+g`um*H{~T2~_t#-^5SC!r;iPr35^1i&<<5%x|uutQFRSIW)7BZL`_ zuMiZ85uFucUtV;nID;lU#AHCd?L^Mg22DWFs^|#eqB#^hH=k2A+!e)<2j3uYHr+7I zG%kyYeNB;mJD^@bB#XjU?ax3CYi^B$=BcPyQ}tS-muA$XLq$~65}{(grnm+T{qax? z2fb!CUoPJ}(W^8FRZndVf|hGPN*#Jv1`WPFtPb+RYVN(H)j7T|k6S7>uTVE+I> zKG(W(qEk7KO*X`pL6_iq(|#ch)%A<|KZt!^{HcpCIPEQM{~EZ|B zk~O)pnn7MhSGAX!0-5M>B~elum*e90Lq0f~{t`-SnTjA`f4)nSnpynvvD3!_C(TuS zArK3CoopJ7Ua0P{d6ConXRvg8x4gyokG}5uvm*&{3|iPW?b9MDy!UvKkv^w*QN+Lp zG322)I^0>fATJXWy&i4L^^2&;6VbWmD6c_aaTs3z<9mev&~$9=lx!%M=9;t%No;^v zJ$g<0K(h&ov$u^*!|wmZYZz@iq*XL_`s%xE8yo@jOnmH|2v{T5 zp5uJ@CsDCO48ZI;208cnw_F>a;#Z(OjV$EqWstsPa6>9BAC6Mgfo<0O>d3a(jK+{jw@{{AlP67;s%eQNUavy)CPFQCEVD7Y$V+5Q_3PTRvFAQO3O zkHUuyqz!$N@9*x0HE3Nei=Z%(h?n-b%oW%&ow+83Z?4s>|31_i?yt{)UpEzbPJvg1 zgN9H=hFkl`&K0&yN4w2gW@Z6Qr~Q93oj`F4LVu-)?5*9jhi9r+^Y%3~+-0??xG{b~ z@O08e$10+cD?n{bk(MIemyqsC@jnqTm1((42CItS>qJ?nWO*Q#wLtqWUj*BGI`3OW z@KJ>J2B*Evbom(P5olyw3=PYZY@+1r5kS)JQ#Zvw@0I^S=Y3Dm(Qdk8LPCsyB=W3I z@j}5Qs3KVe)y)t4UuNQT`9J&XDBafnr?t($#ObqactmeR=~R%H{KAcV3cLtnyAvNuup9V9&47h;`l6xPV5!rVtn82k5rVW$LsZFPvfs-1 zrS%n}QNiYWNx$>cujh4-c>WrC*8>;wcb_Kec<;%B-?^CA7<>ND=F_CzS`YnQsoJcw zBgPq%vFrOO3ATI&C|;~T`sxe5+#0srXa+k~EU=QCOM;{oP)mt+$yfH0uOp~u$l@^U zrgpbh_3g|x?aTnweBeTX0xV8{3jhf(D7a5vW@BU10gPTVa9MM4(UR^y#ZIsK-NR&< zC<$TTyWVPWpG6;)_-NoJQvAn8?ul;s!VIe8`wkB`Z#SpUNo^T zF;>ZF$xe-uyn&uZ`t|_OOa6IQmPjUuo5TY#;36M2@mEPlj`1V7f0+)B15CQiZErDj zy0FiJL<`_d9W0yrV3=8#`9WOo3~l3Wii<%Q{k*KhJ5MG8@dk15h>wq@LztA>F8Y2l zP`Yo!iiBeN>EX=uFiif4$^_0A8;}I{IvtC}tNJZ`J~`!PQFr~Gu%ne@<&=zl$p&yZ zP5&3-*A&DRzpIh-#cN`xc_-H@^plfiz|X#q3W%nDIQ}Km$)F4ADcTL4-YF$f+6 zp4k_^QN757N5Rz0p~l*j7N6t>#NjpDd(M>p`yi6HmH9KRv|dDWz_t_37e&h{Vv+aM zpWAig8A2Psms*`WPCxHC4BZQZf)6QaX&D4z(pH9Vf`%sAEdvhhe<Z#w+J}UhC+Cl+>Pe#GSc+qM90*C8TZQZCH?eP0Uw&-6e8`;ZW z7aZhdomwFmm)jo8@sCB4?D7NmPHSJjX-`)53~_SEsTjNd&(#1~>3L!GZfINBl;Cx@ zuj%yFZ6uOQYJu;-nopT6{W4E!!fR_Np&+oN z?~ikiVLg03ReqsX73s(OJ%KLp`Ex-JY}m_2QTwiY2NckSI7nJ|e^h=K0xEq;xAPI03WUrl}G4 z@HCL>g1$IxUt9RT1^u15(f7;QecOx=`6u^6z!rNP*%A$U|@_)?g&EH6DHlbiIOp<@~x#NiYn^J^9}bLN-w(W2m|C;_3YQ!cL!Aeq5K& zzMKH$!>}n3c6PZ}0T^v+#c=#@gJwhfko^J$i)YuoPK?0I-H`4ee?%7(HM!wBw7zIe*vdZqYr-uZJ-6?oF!Y|${yisKa& z7JYI9eccPG^5^Tg<}c|mL)GdCRIRq#NRj5&JTjY#w57n#&+8=3Ausj;QUJTca?!p> zpO-1SEp^xFnmhB&5;rYs(yKOn$ouH;31|PcjG>t3EoU52zF#}95xESpmT?M^#*XDc zTs%3OHxP3_iFM(ke&2{p)IzmT&%_1BvoR^NKlJ9HvB@`&3kM$Y)1^f)bKY~BZi zlByA<-5GY>mD{Fbs>&TD>ClN{+bd#9Dc>E8U45S5aJ8`tsE&WkK8YV0TZicO0seEX zVBT^qO{fDShg6W0k7LV3rM0+4RM+3`RXGZ=iRY0oNJqzHW$z?}5^WDR{G-qydi+E) z0!mLrlsd#MD_$42C0>L#M$GWqh0E~!)GrZo^oZ-Sgd3$(55VBduPqKDVU zyYf@2ej_w%QqH?#3S>Bqq<78i z`5*)1?#@#gu_DbT_g*9!uYL_ZN69Go^)XL@9$=UuO~)}hbZf87j|qf4Z@u=N<&oiwJ?|uYoA8%8BPxX4gk3JFu z+{-JX2mi;mw%BrlSu#bZ!Jm{)=>2mlsBvqluGj9`dsNIg?^T-l>Ic`n&!PV2~3yl}D4vdcWE&VPOG3is3fxu2IET9Twk5h^CI2M>jC zxIbL(pYgNn%(Jxjn|qM*5D5`|K>Dfl<~5Mx(CAG@6T`n7e*brkw!W$Vy4-z-_CR3GZ-}Wu`^#Xqf2Cj))4ad10g*bq&2Hs%JJ$GQS9aSI~F1T ze$7X`@nkfo!;2oM8d^VsW|E1lM=t$<1T8%x9x_2S+NwD7l6hR&YvDtfm@v*}$`2Yb zCqX&3E&K=44s~CxvDTWkxu(vi>mWZU1GI$~pzR8vl9HQm8rR{vC@YlWQ?##B z?4LyW))xBbYXGM2c)63;(-a3JX=3T$ShaO}aH6!R2xUI1M}?4U=8Azoa&JPK;*ww6 zN))#)BkF;UW#4umasI9jEPp3^+bHh@;Eh_aZ?@XMf?%D$aV(x_uq;pBFNsA%O|la05)hwR9}p%=l0D5 zqmrb;WC1OHFM+Uw`~4jei_pW3SQaL#{6DNuYU{ku4*M^zA_cGd!iY-WP%%RNF4}1E zwX8-*?E9U@x#j?O+M}*xt#OJK1L2t8 zgXyzC=6VQekgrx_n~12Rgk7@##?4B)fI;`$9^xH^QJat$IM@G~`{!N`g`>aSg;kZ^ z)3vdGxkhzv^*^zbdMQ{>oBO)R;+FNW(&3TY_gX`tT85+4EFlmzAJAOa%wV59{vDjf6=>tt-$7f)KYYS zuEj=l4vY9`VMW8$Whw$0xnzSp7}M;nX8MPWlX9&K4e1ttIZ zoWQj-P>!=GY))40t*}9ypKOpUw1Ax$B~$r1jcqSTjiQ(a>A22s(CAJfBjXl+2382ty~0jEva(dm4}1f`T%fiA{nkM(BLE6jHHLDkzQ94;kTsNh zjwVv=GMd|3p?|*?VT}(6he0G1Q!p+_dUD`PIMAo9)sQyMk$t5O=9`a1+0O=f1@9EQ zx$UEoVZhO}O8~jrGRXAvlaB0r?yw2JSg8fi8&iYVw8#AMn*R=2IJg$vj&}L!#f!lF zC(;vnC?+MNpwciEHfJ zKC3(FVMg+?ATF1#^YeK#ykK&<=-E44ms=kSkZtjYgite_ICc?KsP7^>c5tR?sjC{a z1DzdlBVV4MK9KU%foTkp902;IOz*Yg)AMj^_F=SwdJMuScUlNXBIH9awN3usQ$vGj z9|&tbQRZAognm*vCl*;NtjE7;L|wq!NftZyDr7th1RtOJ6ee zWWA@$Gp}@HcL;WA_R7A7@+Ig(L>*F`d0o7WTO;tysl-5uDp5 zqN8P6>Hb3gUvaC>8h?LRn9QU{jujuDO9Q~|QN1cT$HmQRC|9+BWSouP=IUUtKa&5E zAz@??0U$);hru4QAB<}GaR$Y4WY!MqM<)76#Hlk}ZWLrRUXmx5UhQaH1Mc!GKl}qKL~tm747t($`v+)fkyKR#C5PlH~F8 zQ{9S&B=Z15%`QU@^Y>4U7Cbp~-JuVsM=u_1JNoiTyz&w6OPvF&Dq%aH+vl(64OU7I54+H!C_+O!Qu_#U*eV`O z>U89>Vr?`Xrq&guy9ck_mAju>5IZJ<*|8J;a_dW=0ksKjeHNZaS()8To@`fo~p?={nvMxUc!H`TOwK@e7kEVVeJ* zxN|QB>520@SWg*jRT1bI-Pc@*Z_HzQz#-CA?_PhH@oQl7q%l-yv}&DiHQ_5 z3Pn6ehivPhu(Qk=NFtD>LCu<>}1h-RGn}m=c_Op1UlqTsOJ1v_7Z(qYUqHTp`d)w6eBu>oLd_PCvmwobHUVroS&9{PP3bGNsunz&X zXJe+Ca$a3Sxe48fVkj361r^vAV! z`d#HDOr!!UsFOZ0{9{1yn;!}S5SuHbkqqh0)C#~TpZ?Uh8+jsk70lH*QFpPQ0{`V; z>-x6ext>D$9l1ry!lx$iuasCwQxiWv zS&6+IhbDwII&*AaYjv{R-6KW~JQK7#QEpBPiKK;2W`uEhG*ROF`>l;aNNYy=f>A(| z{&mK)OVjsn!Z?~21E@%^sn3l`O-32qdd`oP#j(Z){r5WoYFnQm9~)RS4DTDwc=sFW z*D{|)@Z`CqkLdR6EIaVvNGI;hhUgyESdiRcd z+a%KAPLj-v{p$b>aPdTnsQ^4{NPeeS;7atj$hn)B<6qv}J?FzMd@QBSRrk5L9Q&T8 zq`AYUEPI-2YtOpGN#+l=++8kaif35GWO`GPny!~h$_W<_HU ziiz=Q%Hnd;u$n=2sm)>RLNJe#f6e28t$8f}94g(x%#9t(BKfrI8MW_G#Di(wF2P6J z6~Ej}6u>f3HHLg^8>wkJmRBt0j(gN`BwO{r$*m1T8QpTp8Gd{j(=rQ&)!3Cx z@R3gRWF#NK&x^HA-7vizzYr@phpPO-n0C5J`leq>-~NYYW2OW&;mVa=vfIbgu{9ox z$oJo~VPzkykL&$#kL7TV}`Ee~K zP=|~lq?a7q@<`S=?5IPJf!FUz5mQssm8!Wpmt{~Q&k3`HfY4>SPre&M;Eq**Cd5NY zoS@<~SZ$g2dQSTaD2H#j+S+D3mUBK1;zVbqH$d?C=>}o2rp9juL`I6>Uz$BZghegf zIt=3#jLw|lk(ZZmFLSmzH9W=*qKwio~~!P}o~70MmcK_fj0ZUBDRn@IG7) zojKVL1OO>xywm&2l`L`VYKuEGti0KIAT4)XqH3W%JNYh~w6=^x4?l#N_Y|kg)X1kH z?e+lBRUbhZl<(%vn@>@r0K%NGf4aB?m?k$9|CNO+<=^fc_yMA+abb)?ghL>t7!O)X zKL>M)zXJ}HcXOqy<|0Hq-xpU8D({7x3k%V>DgNNW$XBlS4xro=l^;@Xn{D)AF}g7I z!{$3YWKX~gJT$^mkoM3MR3Z(2etv!gDx!u>H$mAg0}_>UAX+kkv7F+Z0X$d9NHyG= zitO4rkwU+pKYR#)0bhOK@?Se6hBa0}l$~tH+cOkGjpy&YJ#Sj$S9SE-*8=8e6`4I% zpeL2k1i97`Xv=$V7x|tKPy@~!L-h7={XYqI4bfZ)<*s9muq3uizrGcD&R&prBgFPW z3|C~|CBZ81lAc!xQgiO5Ih_`c6!_)ZC+#P1f82JWX_^H*8Pj6)azUZo^h%#-@WG-@ zvwXOiI*@OzQq^lw1s4eNL}l+%A1D{EEHN2&0sJdxYDGds>TNBNz$i6 z<^kLJV&HSNO~an?PiePsmcI}bSf19DON!L_ASEbRE12kwL|~NTDw-qz^X@xP+5wm+ zW8Cy${Ob5J(m7b`8K|?HK={D2FDa)>vS#gRo55hYYvx5eyNSKmL@`c)*oylGXcjT2 zDq=hn3(G0Mmk=df-Cj49IS;1;0{_r1H@89s*D6RXD*W;Tedvqd-UNGPmRF2sBv@r9 zsj%T>odztpPBQi@=O15#fb;}ld2&4pPUfGn%XRu%pqlbI4@C?LLYWp_46|@;k#_m< zfnMJGcP?@hc3Jp&W}S-?ALUVLjV8E99Ol~Y0`PD`jU{TT!?8{;okwz0QdZm#0=-)J ztnrB>h{Dvd!9K!$NfuO>lG^&46t}4!Lx3^xLDsE(EN`a2(%9_fWgdi)@&dE|wb14; zB$|~^RE@T!8WZ-a`HTfH_uXc#~{%8h7Ova{KfWj+FvYD%y6+IlA8>=%D z!(xr6+rA5~QqbY%HTikrqzh&1BAqY$CC}-nTA7&OYHl0&*5R@z zKwLQ6{Z2*C{k+zLHNR+xNr4qql>p8*Ai6Km%v0|oRwOa>r{$q$qc{{B0j`NC{T&AV z!!u)*{s3xS`MTe2PbK{J8~=T&IfaBa0*irF zc>j%MJ733&l|+TZmjJ%yBQmbR>F-|gaJ`K@ds{exq5M_;znCZ)q5f^?PbteM&+^=} z#Dv`3S0H_|AU_ERxsE!o+%71C3y6t{9Xxe0#LMhqIpsDpNrv-x@y05nl~SPBn;RCP z$bL9i1SbY@T74H0pcv*g!tePg@n>n87f1)lpD(q0+>!%t!y(6bVVIY6n0|qy^M%Jx z*lX>#9?3d6u0Lgp=Ev$ClKO)9fPa4%B4^BTIMXN#ok@3#fN}Vr=N`C?6$<>5ewA!e zn74Jusij1*FMq!+k!Z{pSaDN1ZR>Lmz=t|ununOs1lD(Bo&P=nqcTM|3*~{4d8NoSSK< zMmsc99j)|mq`13w_s%c7h(O2q0GWvA#yxOajrYFFKWg78J{^bbpPpAvk_ndXrX3}P z0}*!L0U~ROlvb`l9YrajTOl6PZ=@@@ZFzSD?uXGTF^9pn5PDiKO?tO_>W7>-gb~(L z0HW|Na1bHxIyOEY@zRo4b5Qf3)utA04Rn-+>3;L1qgV7Nl%+jh=OYIKDC{aAS*jnS zqxIh94#Sjqhi4bJ=J8Jigo6Y0$w|Jyz1~hCG~wCF=y(9yo124VV?>cduMFaXK(9w5 zUea0u;Ashf5k94>8wI&|3lLhBylS_)vA#;ofB*iy3jr_kv8^ouoOzd*q@<+YbKNJR zi0ECiJt(SsMIlG4HINS^VdT$VgQ#nJwxJm0o$lUcJ#u0XyBs$p{{^sm-$ll5-DL2z z2>;J~FiGg8zGT763H^64{kF8W@}w!+q>$smB0{C_W)_=3W&DO5I1phDMfcd47zRYP z58yt>KA&F%vS0>eq`ibzp!&uQ!l&`)q=b=^A5Qc}0BEH{>ZW7>NE}J7kiIVhw`K}C zPG}NfrEudX_i%mWg9c7-9lY)`)TN9ZYu4R4e&iHbPWecRit@s6odl0P6bfv}*k+RH zS^;fSSAyKT9VqaYo!IzW$g2!JhO-`f;I6GU zP00xMow@fR%lZ)xY+JAy1$Hig_wB(0#@S+`3XUUv(6-|NF3hL-Y;H>q7DOFR*$%Rk z7a_)4>46H?$MAh(H6A=t5FcWgl#)-bR_f#Jl|xmRpBNy`cYE|F&_52F6Op^*{cNc-Yc zuMps+<1yUa0HgYHvIX~K&Y`!o<769-+YkC~xPX+4?GNYgG{n0Fu4lJ}-%0|I@5plR zEj5AjK{#=1XYub8Vqa!}aK!g42oN^X@YPFFdllXz+$V^IC0;CHMWzDTHk6qnDRI^YX#2Zlh3?-JWPT3)?OE z>MMr`0qm>Dm@rbfuEQXu3?T=&^d@lWY$d%fuRQRx<{*1kt!cn9KL^F8pZB~$v0s5k z#cKh#k*JWTK{hN(!zFbN^J3Aldij!XKhQaUK!CfV^cx0fM2+t-ofNH2fsJXn3iZaU zvDTKB7L!wy_#-FUBDbx^ue;!P?aUk|j>9$`xpL9)YpeClBVw+KjzbI=u-`p|ql9WMowD{zzy}XPXE4V~=#vaLg&0-Hk-t zo%kPjH>BWSy#%+TQpxLGqWPFruM`raNKNnj%2fNpi zvpD4wD|srP@AhXHJX3 zSf;;_oq&*(v|wG%hWYyh6_Wv)B=6nFxSFe07K$AI+XzB%w)boGAI>yFxw(3&3Xncn zhU%KKq)S_>j|fHm#sz)1oR73c{QGIz9Oyj z+cB#$dGBaevF)Y*J_9`o^HOOUGmsR5={H}$RD#Tl9oitZT5hd8cOxZ0$*A=3Xhnx9 z!TPrLMO>TE+qw1c^UF%ZHa*AR{+G{EsMay+crA1E6x(Ab@jwhwYgsLr&j=yV@65?6bqM!aov6wFGfnOxPe^(-yprOh=KjyiX_jDG zFi)_`9p8VqS+nNRxjA~ZNLluWVfy`?E^V99T6@?hRug5)6W|9M?wgV}5)EVMKHNz) zV#g7?#)5NcOlT_FoFtd0C{Z0rVsz{`4UG=>=O_JjQ(KF-rO0-mK7w}<68qUQL5Z*G z_Ao*&SY=kJc^J!sN?y-b3Wkn~P$*4#QD+{+PHE?h_u_6X{7f8|?t(MmWuCs|HX9lR zck`%=QOH0&b^5l%89wAkm_8(l{P`okoUl1*lp@%@EIS{Z;Vjnc5!iFTTF^k8(3LYk zZ#!rA%}YoK`>HKDahp&0l@0FISTUuqf)f8=ewegqJ0YcqCP?2R=J@X7@AWA5Q&d!m z4p7#pGEBRCgdl>74Em;sCJtn?o%ExJ^xpV4Z^2}{~12*!!@iXsT|=S5Q;hA~vHCdolbznP2olSFBInSVw{PMAN``7t?w8K@rQ0OCfwc2JJ+~=w> z{0xU`G)0fX&9ooqj{PgOy>0OW0RQvz{ literal 0 HcmV?d00001 From c04182e4188e8669cab40b69c71da2df50792c6e Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Tue, 15 Oct 2024 20:05:13 +0800 Subject: [PATCH 4/4] fix typo --- pcip/pcip-1.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pcip/pcip-1.md b/pcip/pcip-1.md index 24a017b..8a4b1c3 100644 --- a/pcip/pcip-1.md +++ b/pcip/pcip-1.md @@ -53,7 +53,7 @@ You can directly use Pulsar's own message call tracking, monitoring, and logging ## In Scope -- To implement a pulsar-rpc-client similar to pulsar-client, we can encapsulate request request as a `request message` +- To implement a pulsar-rpc-client similar to pulsar-client, we can encapsulate request as a `request message` and send it to `request topic`. This is a pulsar topic dedicated to receiving requests. - Implement a pulsar-rpc-server. Internally, the pulsar consumer is used to monitor and receive the message in the `request topic`, and custom logic processing is supported. Finally, the processed results (including possible