From bd1c95cb0de4dd5b5d5fe34ad9059dc463460841 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Mon, 2 Oct 2023 16:49:36 -0700 Subject: [PATCH 01/32] Better flag documentation. (#28753) --- .../runners/dataflow/dataflow_runner.py | 20 +++++++------------ 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 950bff768ef4..7ad6ab04be68 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -97,10 +97,6 @@ def __init__(self, cache=None): def is_fnapi_compatible(self): return False - def apply(self, transform, input, options): - _check_and_add_missing_options(options) - return super().apply(transform, input, options) - @staticmethod def poll_for_job_completion( runner, result, duration, state_update_callback=None): @@ -496,10 +492,6 @@ def _get_coder(typehint, window_coder): coders.registry.get_coder(typehint), window_coder=window_coder) return coders.registry.get_coder(typehint) - # TODO(srohde): Remove this after internal usages have been removed. - def apply_GroupByKey(self, transform, pcoll, options): - return transform.expand(pcoll) - def _verify_gbk_coders(self, transform, pcoll): # Infer coder of parent. # @@ -589,12 +581,14 @@ def _check_and_add_missing_options(options): sdk_location = options.view_as(SetupOptions).sdk_location if 'dev' in beam.version.__version__ and sdk_location == 'default': raise ValueError( - "When launching Dataflow Jobs with an unreleased SDK, " + "You are submitting a pipeline with Apache Beam Python SDK " + f"{beam.version.__version__}. " + "When launching Dataflow jobs with an unreleased (dev) SDK, " "please provide an SDK distribution in the --sdk_location option " - "to use consistent SDK version at " - "pipeline submission and runtime. To ignore this error and use the " - "SDK installed in Dataflow dev containers, use " - "--sdk_location=container.") + "to use a consistent SDK version at " + "pipeline submission and runtime. To ignore this error and use " + "an SDK preinstalled in the default Dataflow dev runtime environment " + "or in a custom container image, use --sdk_location=container.") # Streaming only supports using runner v2 (aka unified worker). # Runner v2 only supports using streaming engine (aka windmill service) From 71c8459633ec86e576eca080a26be9f42474ecb2 Mon Sep 17 00:00:00 2001 From: pablo rodriguez defino Date: Mon, 2 Oct 2023 17:07:58 -0700 Subject: [PATCH 02/32] [Blog Post] Apache Beam for a content discovery platform (#28734) Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> Co-authored-by: nams113 <39890215+nams113@users.noreply.github.com> --- ...i-content-discovery-platform-genai-beam.md | 338 ++++++++++++++++++ website/www/site/data/authors.yml | 6 + .../blog/dyi-cdp-genai-beam/cdp-arch.png | Bin 0 -> 271543 bytes .../blog/dyi-cdp-genai-beam/cdp-highlevel.png | Bin 0 -> 31242 bytes .../blog/dyi-cdp-genai-beam/pipeline-1.png | Bin 0 -> 146525 bytes .../pipeline-2-extractcontent.png | Bin 0 -> 130427 bytes .../pipeline-3-errorhandling.png | Bin 0 -> 112800 bytes .../pipeline-4-processembeddings1.png | Bin 0 -> 49246 bytes .../pipeline-4-processembeddings2.png | Bin 0 -> 58035 bytes .../pipeline-5-storecontent.png | Bin 0 -> 74751 bytes .../pipeline-6-refresh1.png | Bin 0 -> 74889 bytes .../pipeline-6-refresh2.png | Bin 0 -> 72757 bytes .../pipeline-6-refresh3.png | Bin 0 -> 53972 bytes 13 files changed, 344 insertions(+) create mode 100644 website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-arch.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-highlevel.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-1.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-2-extractcontent.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-3-errorhandling.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings1.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings2.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-5-storecontent.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh1.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh2.png create mode 100644 website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh3.png diff --git a/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md new file mode 100644 index 000000000000..8057374591d7 --- /dev/null +++ b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md @@ -0,0 +1,338 @@ +--- +layout: post +title: "DIY GenAI Content Discovery Platform with Apache Beam" +date: 2023-09-27 00:00:01 -0800 +categories: + - blog +authors: + - pabs + - namitasharma +--- + + +# DIY GenAI Content Discovery Platform with Apache Beam + +Your digital assets, such as documents, PDFs, spreadsheets, and presentations, contain a wealth of valuable information, but sometimes it's hard to find what you're looking for. This blog post explains how to build a DIY starter architecture, based on near real-time ingestion processing and large language models (LLMs), to extract meaningful information from your assets. The model makes the information available and discoverable through a simple natural language query. + +Building a near real-time processing pipeline for content ingestion might seem like a complex task, and it can be. To make pipeline building easier, the Apache Beam framework exposes a set of powerful constructs. These constructs remove the following complexities: interacting with multiple types of content sources and destinations, error handling, and modularity. They also maintain resiliency and scalability with minimal effort. You can use an Apache Beam streaming pipeline to complete the following tasks: + +- Connect to the many components of a solution. +- Quickly process content ingestion requests of documents. +- Make the information in the documents available a few seconds after ingestion. + +LLMs are often used to extract content and summarize information stored in many different places. Organizations can use LLMs to quickly find relevant information disseminated in multiple documents written across the years. The information might be in different formats, or the documents might be too long and complex to read and understand quickly. Use LLMs to process this content to make it easier for people to find the information that they need. + +Follow the steps in this guide to create a custom scalable solution for data extraction, content ingestion, and storage. Learn how to kickstart the development of a LLM-based solution using Google Cloud products and generative AI offerings. Google Cloud is designed to be simple to use, scalable, and flexible, so you can use it as a starting point for further expansion or experimentation. + +### High-level Flow + +In this workflow, content uptake and query interactions are completely separated. An external content owner can send documents stored in Google Docs or in a binary text format and receive a tracking ID for the ingestion request. The ingestion process gets the content of the document and creates chunks that are configurable in size. Each document chunk is used to generate embeddings. These embeddings represent the content semantics, in the form of a vector of 768 dimensions. Given the document identifier and the chunk identifier, you can store the embeddings in a Vector database for semantic matching. This process is central to contextualizing user inquiries. + +Content Discovery Platform Overview + +The query resolution process doesn't depend directly on information ingestion. The user receives relevant answers based on the content ingested until the moment of the query request. Even if the platform doesn't have any relevant content stored, the platform returns an answer stating that it doesn't have relevant content. Therefore, the query resolution process first generates embeddings from the query content and from the previously existing context, like previous exchanges with the platform, then matches these embeddings with the existing embedding vectors stored from the content. When the platform has positive matches, it retrieves the plain-text content represented by the content embeddings. Finally, by using the textual representation of the query and the textual representation of the matched content, the platform formulates a request to the LLM to provide a final answer to the original user inquiry. + +## Components of the solution + +Use the low-ops capabilities of the Google Cloud services to create a set of highly scalable features. You can separate the solution into two main components: the service layer and the content ingestion pipeline. The service layer acts as the entry point for document ingestion and user queries. It’s a simple set of REST resources exposed through Cloud Run and implemented by using [Quarkus](https://quarkus.io/) and the client libraries to access other services (Vertex AI models, Cloud Bigtable and Pub/Sub). The content ingestion pipeline includes the following components: + +* A streaming pipeline that captures user content from wherever it resides. +* A process that extracts meaning from this content as a set of multi-dimensional vectors (text embeddings). +* A storage system that simplifies context matching between knowledge content and user inquiries (a Vector Database). +* Another storage system that maps knowledge representation with the actual content, forming the aggregated context of the inquiry. +* A model capable of understanding the aggregated context and, through prompt engineering, delivering meaningful answers. +* HTTP and gRPC-based services. + +Together, these components provide a comprehensive and simple implementation for a content discovery platform. + +## Workflow Architecture + +This section explains how the different components interact. + +### Dependencies of the components + +The following diagram shows all of the components that the platform integrates with. It also shows all of the dependencies that exist between the components of the solution and the Google Cloud services. + +Content Discovery Platform Interactions + +As seen in the diagram, the context-extraction component is the central aspect in charge of retrieving the document’s content, also their semantic meaning from the embedding’s model and storing the relevant data (chunks text content, chunks embeddings, JSON-L content) in the persistent storage systems for later use. PubSub resources are the glue between the streaming pipeline and the asynchronous processing, capturing the user ingestion requests, retries from potential errors from the ingestion pipeline (like the cases on where documents have been sent for ingestion but the permission has not been granted yet, triggering a retry after some minutes) and content refresh events (periodically the pipeline will scan the ingested documents, review the latest editions and define if a content refresh should be triggered). + +The context-extraction component retrieves the content of the documents, diving it in chunks. It also computes embeddings, using the LLM interaction, from the extracted content. Then it stores the relevant data (chunks text content, chunks embeddings, JSON-L content) in the persistent storage systems for later use. Pub/Sub resources connect the streaming pipeline and the asynchronous processing, capturing the following actions: +- user ingestion requests +- retries from errors from the ingestion pipeline, such as when documents are sent for ingestion but access permissions are missing +- content refresh events (periodically the pipeline scans the ingested documents, reviews the latest editions, and decides whether to trigger a content refresh) + +Also, CloudRun plays an important role exposing the services, interacting with many Google Cloud services to resolve the user query or ingestion requests. For example, while resolving a query request the service will: +- Request the computation of embeddings from the user’s query by interacting with the embeddings model +- Find near neighbor matches from the Vertex AI Vector Search (formerly Matching Engine) using the query embeddings representation +- Retrieve the text content from BigTable for those matched vectors, using their identifier, in order contextualize a LLM prompt +- And finally create a request to the VertexAI Chat-Bison model, generating the response the system will delivery to the user’s query. + +### Google Cloud products + +This section describes the Google Cloud products and services used in the solution and what purpose they serve. + +**Cloud Build:** All container images, including services and pipelines, are built directly from source code by using Cloud Build. Using Cloud Build simplifies code distribution during the deployment of the solution. + +**CloudRun:** The solution's service entry points are deployed and automatically scaled by CloudRun. + +**Pub/Sub:** A Pub/Sub topic and subscription queue all of the ingestion requests for Google Drive or self-contained content and deliver the requests to the pipeline. + +**Dataflow:** A multi-language, streaming Apache Beam pipeline processes the ingestion requests. These requests are sent to the pipeline from the Pub/Sub subscription. The pipeline extracts content from Google Docs, Google Drive URLs, and self-contained binary encoded text content. It then produces content chunks. These chunks are sent to one of the Vertex AI foundational models for the embedding representation. The embeddings and chunks from the documents are sent to Vertex AI Vector Search and to Cloud Bigtable for indexing and rapid access. Finally, the ingested documentation is stored in Google Cloud Storage in JSON-L format, which can be used to fine-tune the Vertex AI models. By using Dataflow to run the Apache Beam streaming pipeline, you minimize the ops needed to scale resources. If you have a burst on ingestion requests, Dataflow can keep the latency less than a minute. + +**Vertex AI - Vector Search:** [Vector Search](https://cloud.google.com/vertex-ai/docs/matching-engine/overview) is a high-performance, low-latency vector database. These vector databases are often called vector similarity search or approximate nearest neighbor (ANN) services. We use a Vector Search Index to store all the ingested documents embeddings as a meaning representation. These embeddings are indexed by chunk and document id. Later on, these identifiers can be used to contextualize the user queries and enrich the requests made to a LLM by providing knowledge extracted directly from the document’s content mappings stored on BigTable (using the same chunk-document identifiers). + +**Cloud BigTable:** This storage system provides a low latency search by identifier at a predictable scale. Is a perfect fit, given the low latency of the requests resolution, for online exchanges between user queries and the platform component interactions. It used to store the content extracted from the documents since it's indexed by chunk and document identifier. Every time a user makes a request to the query service, and after the query text embeddings are resolved and matched with the existing context, the document and chunk ids are used to retrieve the document’s content that will be used as context to request an answer to the LLM in use. Also, BigTable is used to keep track of the conversational exchanges between users and the platform, furthermore enriching the context included on the requests sent to the LLMs (embeddings, summarization, chat Q&A). + +**Vertex AI - Text Embedding Model:** [Text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/model-reference/text-embeddings) are a condensed vector (numeric) representation of a piece of text. If two pieces of text are semantically similar, their corresponding embeddings will be located close together in the embedding vector space. For more details please see [get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings). These embeddings are directly used by the ingestion pipeline when processing the document’s content and the query service as an input to match the users query semantic with existing content indexed in Vector Search. + +**Vertex AI - Text Summarization Model:** [Text-bison](https://cloud.google.com/vertex-ai/docs/generative-ai/model-reference/text) is the name of the PaLM 2 LLM that understands, summarizes and generates text. The types of content that text-bison can create include document summaries, answers to questions, and labels that classify the provided input content. We used this LLM to summarize the previously maintained conversation with the goal of enriching the user’s queries and better embedding matching. In summary, the user does not have to include all the context of his question, we extract and summarize it from the conversation history. + +**Vertex AI - Text Chat Model:** [Chat-bison](https://cloud.google.com/vertex-ai/docs/generative-ai/model-reference/text-chat) is the PaLM 2 LLM that excels at language understanding, language generation, and conversations. This chat model is fine-tuned to conduct natural multi-turn conversations, and is ideal for text tasks about code that require back-and-forth interactions. We use this LLM to provide answers to the queries made by users of the solution, including the conversation history between both parties and enriching the model’s context with the content stored in the solution. + +### Extraction Pipeline + +The content extraction pipeline is the platform's centerpiece. It takes care of handling content ingestion requests, extracting documents content and computing embeddings from that content, to finally store the data in specialized storage systems that will be used in the query service components for rapid access. + +#### High Level View + +As previously mentioned the pipeline is implemented using Apache Beam framework and runs in streaming fashion on GCP's [Dataflow](https://cloud.google.com/dataflow) service. + +By using Apache Beam and Dataflow we can ensure minimal latency (sub minute processing times), low ops (no need to manually scale up or down the pipeline when traffic spikes occur with time, worker recycle, updates, etc.) and with high level of observability (clear and abundant performance metrics are available). + +Apache Beam Pipeline + +On a high level, the pipeline separates the extraction, computing, error handling and storage responsibilities on different components or PTransforms. As seen in the diagram, the messages are read from a PubSub subscription and immediately afterwards are included in the window definition before the content extraction. + +Each of those PTransforms can be expanded to reveal more details regarding the underlying stages for the implementation. We will dive into each in the following sections. + +The pipeline was implemented using a multi-language approach, with the main components written in the Java language (JDK version 17) and those related with the embeddings computations implemented in Python (version 3.11) since the Vertex AI API clients are available for this language. + +#### Content Extraction + +The content extraction component is in charge of reviewing the ingestion request payload and deciding (given the event properties) if it will need to retrieve the content from the event itself (self-contained content, text based document binary encoded) or retrieve it from Google Drive. + +Pipeline's Content Extraction + +In case of a self-contained document, the pipeline will extract the document id and format the document in paragraphs for later embedding processing. + +When in need of retrieval from Google Drive, the pipeline will inspect if the provided URL in the event refers to a Google Drive folder or a single file format (supported formats are Documents, Spreadsheets and Presentations). In the case of a folder, the pipeline will crawl the folder’s content recursively extracting all the files for the supported formats, in case of a single document will just return that one. + +Finally, with all the file references retrieved from the ingestion request, textual content is extracted from the files (no image support implemented for this PoC). That content will also be passed to the embedding processing stages including the document’s identifier and the content as paragraphs. + +#### Error Handling + +On every stage of the content extraction process multiple errors can be encountered, malformed ingestion requests, non-conformant URLs, lack of permissions for Drive resources, lack of permissions for File data retrieval. + +In all those cases a dedicated component will capture those potential errors and define, given the nature of the error, if the event should be retried or sent to a dead letter GCS bucket for later inspection. + +Pipeline's Error Handling + +The final errors, or those which won’t be retried, are those errors related with bad request formats (the event itself or the properties content, like malformed or wrong URLs, etc.). + +The retryable errors are those related with content access and lack of permissions. A request may have been resolved faster than the manual process of providing the right permissions to the Service Account that runs the pipeline to access the resources included in the ingestion request (Google Drive folders or files). In case of detecting a retryable error, the pipeline will hold the retry for 10 minutes before re-sending the message to the upstream PubSub topic; each error is retried at most 5 times before being sent to the dead letter GCS bucket. + +In all cases of events ending on the dead letter destination, the inspection and re-processing must be done in a manual process. + +#### Process Embeddings + +Once the content has been extracted from the request, or captured from Google Drive files, the pipeline will trigger the embeddings computation process. As previously mentioned the interactions with the Vertex AI Foundational Models API is implemented in Python language. For this reason we need to format the extracted content in Java types that have a direct translation to those existing in the Python world. Those are key-values (in Python those are 2-element tuples), Strings (available in both languages), and iterables (also available in both languages). We could have implemented coders in both languages to support custom transport types, but we opted out of that in favor of clarity and simplicity. + +Before computing the content’s embeddings we decided to introduce a Reshuffle step, making the output consistent to downstream stages, with the idea of avoiding the content extraction step being repeated in case of errors. This should avoid putting pressure on existing access quotas on Google Drive related APIs. + +The pipeline will then chunk the content in configurable sizes and also configurable overlapping, good parameters are hard to get for generic effective data extraction, so we opted to use smaller chunks with small overlapping factor as the default settings to favor diversity on the document results (at least that’s what we see from the empirical results obtained). + +

+ Embeddings Processing + Embeddings Processing +

+ +Once the embeddings vectors are retrieved from the embeddings Vertex AI LLM, we will consolidate them again avoiding repetition of this step in case of downstream errors. + +Worth to notice that this pipeline is interacting directly with Vertex AI models using the client SDKs, Apache Beam already provides supports for this interactions through the RunInference PTransform (see an example [here](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/inference/vertex_ai_llm_text_classification.py)). + +#### Content Storage + +Once the embeddings are computed for the content chunks extracted from the ingested documents, we need to store the vectors in a searchable storage and also the textual content that correlates with those embeddings. We will be using the embeddings vectors as a semantic match later from the query service, and the textual content that corresponds to those embeddings for LLM context as a way to improve and guide the response expectations. + +Content Storage + +With that in mind is that in mind we split the consolidated embeddings into 3 paths, one that stores the vectors into Vertex AI Vector Search (using simple REST calls), another storing the textual content into BigTable (for low latency retrieval after semantic matching) and the final one as a potential clean up of content refresh or re ingestion (more on that later). The three paths are using the ingested document identifier as the correlating data on the actions, this key is formed by the document name (in case of available), the document identifier and the chunk sequence number. The reason for using identifiers for the chunk comes behind the idea of subsequent updates. An increase in the content will generate a larger number of chunks, and upserting all the chunks will enable always fresh data; on the contrary, a decrease in content will generate a smaller chunk count for the document’s content, this number difference can be used to delete the remaining orphan indexed chunks (from content no longer existing in the latest version of the document). + +#### Content Refresh + +The last pipeline component is the simplest, at least conceptually. After the documents from Google Drive gets ingested, an external user can produce updates in them, causing the indexed content to become out of date. We implemented a simple periodic process, inside the same streaming pipeline, that will take care of the review of already ingested documents and see if there are content updates needed. We use a GenerateSequence transform to produce a periodic impulse (every 6 hours by default), that will trigger a scan on BigTable retrieving all the ingested document identifiers. Given those identifiers we can then query Google Drive for the latest update timestamp of each document and use that marker to decide if an update is needed. + +In case of needing to update the document’s content, we can simply send an ingestion request to the upstream PubSub topic and let the pipeline run its course for this new event. Since we are taking care of upserting embeddings and cleaning up those that no longer exist, we should be capable of taking care of the majority of the additions (as long those are text updates, image based content is not being processed as of now). + +

+ Content Refresh + Content Refresh + Content Refresh +

+ +This task could be performed as a separate job, possibly one that is periodically scheduled in batch form. This would result in lower costs, a separate error domain, and more predictable auto scaling behavior. However, for the purposes of this demonstration, it is simpler to have a single job. + +Next, we will be focusing on how the solution interacts with external clients for ingestion and content discovery use cases. + +## Interaction Design + +The solution aims to make the interactions for ingesting and querying the platform as simple as possible. Also, since the ingestion part may imply interacting with several services and imply retries or content refresh, we decided to make both separated and asynchronous, freeing the external users of blocking themselves while waiting for requests resolutions. + +### Example Interactions + +Once the platform is deployed in a GCP project, a simple way to interact with the services is through the use of a web client, curl is a good example. Also, since the endpoints are authenticated, a client needs to include its credentials in the request header to have its access granted. + +Here is an example of an interaction for content ingestion: + +``` +$ > curl -X POST -H "Content-Type: application/json" -H "Authorization: Bearer $(gcloud auth print-identity-token)" https:///ingest/content/gdrive -d $'{"url":"https://drive.google.com/drive/folders/somefolderid"}' | jq . + +# response from service +{ + "status": "Ingestion trace id: " +} +``` + +In this case, after the ingestion request has been sent to the PubSub topic for processing, the service will return the tracking identifier, which maps with the PubSub message identifier. Note the provided URL can be one of a Google Doc or a Google Drive folder, in the later case the ingestion process will crawl the folder’s content recursively to retrieve all the contained documents and their contents. + +Next, an example of a content query interaction, very similar to the previous one: + +``` +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"summarize the benefits of using VertexAI foundational models for Generative AI applications", "sessionId": ""}' \ + | jq . + +# response from service +{ + "content": "VertexAI Foundation Models are a set of pre-trained models that can be used to accelerate the development of machine learning applications. They are available for a variety of tasks, including natural language processing, computer vision, and recommendation systems.\n\nVertexAI Foundation Models can be used to improve the performance of Generative AI applications by providing a starting point for model development. They can also be used to reduce the amount of time and effort required to train a model.\n\nIn addition, VertexAI Foundation Models can be used to improve the accuracy and robustness of Generative AI applications. This is because they are trained on large datasets and are subject to rigorous quality control.\n\nOverall, VertexAI Foundation Models can be a valuable resource for developers who are building Generative AI applications. They can help to accelerate the development process, reduce the cost of development, and improve the performance and accuracy of applications.", + "previousConversationSummary": "", + "sourceLinks": [ + { + "link": "", + "distance": 0.7233397960662842 + } + ], + "citationMetadata": [ + { + "citations": [] + } + ], + "safetyAttributes": [ + { + "categories": [], + "scores": [], + "blocked": false + } + ] +} +``` + +The platform will answer the request with a textual response from the LLM and include as well information about the categorization, citation metadata and source links (if available) of the content used to generate the response (this are for example, Google Docs links of the documents previously ingested by the platform). + +When interacting with the services, a good query will generally return good results, the clearer the query the easier it will be to contextualize its meaning and more accurate information will be sent to the LLMs to retrieve answers. But having to include all the details of the query context in a phrase on every exchange with the service can be very cumbersome and difficult. For that case the platform can use a provided session identifier that will be used to store all the previous exchanges between a user and the platform. This should help the implementation to better contextualize the initial query embeddings matching and even provide more concise contextual information in the model requests. Here is an example of a contextual exchange: + +``` +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"summarize the benefits of using VertexAI foundational models for Generative AI applications?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "VertexAI Foundational Models are a suite of pre-trained models that can be used to accelerate the development of Generative AI applications. These models are available in a variety of languages and domains, and they can be used to generate text, images, audio, and other types of content.\n\nUsing VertexAI Foundational Models can help you to:\n\n* Reduce the time and effort required to develop Generative AI applications\n* Improve the accuracy and quality of your models\n* Access the latest research and development in Generative AI\n\nVertexAI Foundational Models are a powerful tool for developers who want to create innovative and engaging Generative AI applications.", + … +} + +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"describe the available LLM models?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "The VertexAI Foundational Models suite includes a variety of LLM models, including:\n\n* Text-to-text LLMs: These models can generate text based on a given prompt. They can be used for tasks such as summarization, translation, and question answering.\n* Image-to-text LLMs: These models can generate text based on an image. They can be used for tasks such as image captioning and description generation.\n* Audio-to-text LLMs: These models can generate text based on an audio clip. They can be used for tasks such as speech recognition and transcription.\n\nThese models are available in a variety of languages, including English, Spanish, French, German, and Japanese. They can be used to create a wide range of Generative AI applications, such as chatbots, customer service applications, and creative writing tools.", + … +} + +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"do rate limit apply for those LLMs?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "Yes, there are rate limits for the VertexAI Foundational Models. The rate limits are based on the number of requests per second and the total number of requests per day. For more information, please see the [VertexAI Foundational Models documentation](https://cloud.google.com/vertex-ai/docs/foundational-models#rate-limits).", + … +} + +$ > curl -X POST \ + -H "Content-Type: application/json" \ + -H "Authorization: Bearer $(gcloud auth print-identity-token)" \ + https:///query/content \ + -d $'{"text":"care to share the price?", "sessionId": "some-session-id"}' \ + | jq . + +# response from service +{ + "content": "The VertexAI Foundational Models are priced based on the number of requests per second and the total number of requests per day. For more information, please see the [VertexAI Foundational Models pricing page](https://cloud.google.com/vertex-ai/pricing#foundational-models).", + … +} +``` + +**Usage Tip:** in case of abruptly changing topics, sometimes is better to use a new session identifier. + +### Deployment + +As part of the platform solution, there are a set of scripts that help with the deployment of all the different components. By running the `start.sh` and setting the right parameters (GCP project, terraform state bucket and name for the platform instance) the script will take care of building the code, deploying the needed containers (service endpoint container and Dataflow python custom container), deploying all the GCP resources using Terraform and finally deploying the pipeline. There is also the possibility of modifying the pipeline’s execution by passing an extra parameter to the startup script, for example: `start.sh "--update"` will update the content extraction pipeline in-place. + +Also, in case of wanting to focus only on the deployment of specific components other scripts have been included to help with those specific tasks (build the solution, deploy the infrastructure, deploy the pipeline, deploy the services, etc.). + +### Solution's Notes + +This solution is designed to serve as an example for learning purposes. Many of the configuration values for the extraction pipeline and security restrictions are provided only as examples. The solution doesn't propagate the existing access control lists (ACLs) of the ingested content. As a result, all users that have access to the service endpoints have access to summarizations of the ingested content from those original documents. + +### Notes about the source code + +The source code for the content discovery platform is available in [Github](https://github.com/prodriguezdefino/content-dicovery-platform-gcp). You can run it in any Google Cloud project. The repository includes the source code for the integration services, the multi-language ingestion pipeline, and the deployment automation through Terraform. If you deploy this example, it might take up to 90 minutes to create and configure all the needed resources. The README file contains additional documentation about the deployment prerequisites and example REST interactions. diff --git a/website/www/site/data/authors.yml b/website/www/site/data/authors.yml index 0458bda2c963..2776132cf586 100644 --- a/website/www/site/data/authors.yml +++ b/website/www/site/data/authors.yml @@ -269,3 +269,9 @@ riteshghorse: yhu: name: Yi Hu email: yhu@apache.org +pabs: + name: Pablo Rodriguez Defino + email: prodriguezdefino@gmail.com +namitasharma: + name: Namita Sharma + email: namitasharma@google.com diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-arch.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/cdp-arch.png new file mode 100644 index 0000000000000000000000000000000000000000..ef42340e905f5bcab8f6972e42600f3f48690c40 GIT binary patch literal 271543 zcmeFZcT`kq(>;n2P(c|pQ6&io2$Hi2Y!Fa_WDrquP7O^|QB;x=6(lG*i%4pcf`EX~ zN(GEDacS9pgurCLPCMM zaZQPY5)v*(94;r|MVQVVYiFw z!jZlELV8cCm*q8fBy#gIh=Br{(pXV=Gl=W$cp{@^N8$!FZ=g;ko@n0>{t$x|6P#(-@A<0J}k*6 zx+=wXhjlOUh6g+M*Hhr0)Ry|2I>YOloDaF;uTve7&SrBcRK4YNbE+gyCPKi9!d`gi zS7Q7|X-_9RAFjLeV_N^kvx}tIw}vN`bai#}SbG{7S!^|@#%8eN zWKD<))d6}@%e2ioNvn`L!-MuFJ8u-@XH!TJaq9IHFju$odSX=fHi0xDaUb@R;Z>IC{zr*pWVSH!O!0?u=7hK zDrRlt9UnCr9kTM(_dTiFt)mdcc*d6{YjouTs>%2Cr9Cg|<#&@MsM4Q?1BF>C z7JtZo6xln<1hC$W$ z5B%2@NOC@Qn8S#eQD9zsU_9r}_)3e~e!3OdQizV03fsN&?zfEgkWvO8bKTYVLF%Mu zZ)3I^PAbfk<%ey-Co7@^s_BD%`F(+iJvx7xX1Od`qLj?d)84;-AHIRH%+or_rG0Tx z>qq#GJ2tf3GcX7V3!@q44X53u&-Z4M;KFC>di(aP0|yT-RA(7rv-C<&92D6f@#xX+ zNKthCR91UCV(l6iLaB$WMItp<>=T&S%m@efdPTfJpDFxkNx&w{g>eO%Qubk@@Q-9J zg>Sg@+l%BZ&s_i5HBo6K+?AU*owCCOr22K5la*uSj8E+KqPnG!sGPaJg6qsS3ZV7> z;lc22Woa;G*(p9=URhgPJEpdE^RT3fw?Ew;T_4t4qukeu-MVr$JtDpsW}AF}Ejk~R zW$=La&29|$G{t@VSNLV6?;F^thH+foQ9eZ!D%p1Yuo-^U>oI9XRo?WA9RndTy&`pL z7mSu4T~Et+O64x5%%b;0cQP|RX57Ag z+un=f<>i@~kJ@IpVV_Ete;gE!R6s!-$QXAl4CT~fPvuwg_+De$R$q_Xa?gQZ5Pn|_#1y~cvq+NxNBl@v@hf_**Joi-Qls2*WrtfQ9LL(T3`EeAZza}y_biOJDiaEq zUe+1Tk<+;xJ9cciyTGo}CagC8D67J-mCIt1OaMK#!`g(zLcTWFL|2|7gZuPDrjeH9 zpw42KJSLO&5BjS0;+UV$>Z8O{%gU})R93PlSm{-K{kmLSRWm0PsX!;@bPM^CwC|~R zU&zUM-`uQRxqkgR`-KZFT2>pXfxh8PD--jh0TuTems?aS<|oY3#vncg=S=Qnd_7uV zCT&X?m@5ltDDDC<_Yw zND@Y}N->JT1qBA4OKeX5@#96tcLLU-xxjA7v{(=)+MceJCqkNCXP6E9+q~woRAZTE zRvV!;di;o^NaX5#kfMF>=UofKjaN5F4^6Law8MN60El)&0Ya$i>%YTgahI-_2~BQn z77#4y#H>Fr)2l3vvuDbHfngmdQVyQv8lF#99&N~q{qf_6FaJ_n*pSQOh@_J;Z>@54Q|LNAz6*@n1`97bzx?YB{`?8Nk#+xkOpIjar8PLVro|CM@9~hvKmy?+~=*nmvt&9AyG1ECOO%?s;Jn zx7(F`$@MRXbM(tGV9Nso1Eyb}QBm{SCtWz~Y5S6y0+kPDly;mG{fcfPH1+}w{&RN;ir^{lKc ziOY3yarY^T2SWEt`sC*eX{5gbBMJ%1Zk-Zw?t;K#P}kJVpt7}V4--lzfR^VV8lS9wy< z{G|$&sjKCJX|zH-z);idKxx~%b=Ts>(bO9C{0}w7LbMf{)5cG6MUtmQ?;M?adj=)q zG##U)q;%}~@%W^q^NeT2=0mkc3Bvgwzka>OD5j+)(@EEbzLC$k{N1nnSZS+$VbM!eVdLMOD|oeh}*Ql(f5Z_j2MqO?|=NG-)y2}g1k`fH~}j@ zC99$m43nDom@L%;1YMlyGNPSx#}_Y&_rG}Y!uQtXw^w;=?Cdo{lXTLHlx}M7j$I^Xn3VEH3K@-yllMdM+@gZ(fmY|y_zbjPe^1}t z!x43cdp*S8_0#|kq}qGYiyQjJy||c)BrSuEg?<+uH9J3H#1r4YKlealA$VrvH2>(L zX4N$r84e(qHJVG0h-X$lKitEN%Cy3y>*lCb&5559vW_7d2?D5~j&BKmE)4pZ$F|h1 zY%uLyBfhjOmYShcNZobEERKqDaoS!eV{Ga4P8Zrm=hbNH{3vkG{zLOF-G$jBtX2Li z6gL*<4pd?087_Wq2h`1RG+CgeTO1{HO$kdD%}7Vjad zBiqOfm>&|DkofF8H@KE&_u$EsCzZLSbKqX1M~CNin+fKTw!ZQQd0h@azv-7&i9b}cin^Nx+=95ZS)%CKe$+Tu zlX0WOCGeAJYqmt6^ANtC9Vf}}a3w^lKoXp2Xv3jETVmHjR53Wu=YH;xJXeo=4<9T(s#(U)!I_kxTD>sjE1b_i-@sU*XaLPAHv?`WMu_J`~-s0 z-)1UJAB6_`#w3(x6oW~-KYm9=MYWJpA-Y%8rEO16l*ejIGP{1qDE12m8!eEixV4g; zH9_!ejRyZm3>O?4Vq4_~!VLnB9!=dVA-SR!*4}v~n!X3e1|+R?$jLjk_%TsZ;id`% ztytwWR11VAW>x!8c{<6AN^Hu(1`MfgF+tq<#T|)A#Bs1gQ7(Q<-beqyYRgXsY~IsPr; z&OFOl4pPYqXKd);zyP~JU)lb;K7vUuz(s2m9>OEbh<|6Wn2uHE2?n?MS8*?H8Ois_ zCBN4iTcx0}A8+TeAm-MQVTOJOUI01_WnRtVY;}mG6EJ_B{f1j&< zfnwCvEU#<9r^3h2vI`>CgMzmAKu92w3PQZP8msKF(5hDHsB>je&voW{ab_PuW2gDw z5=VGKkB*Jcix(;O%?ninwj*DkAC>mjoKnbR^=HUzDrk$%ihMCvCK^*TTF|;MI!SG8 znb$@3td0Mx8Sk58Cr>5(}xNPmYjNNQIjL}hA9q4&Cgm@6qu1Hj3&yL6yei*=C2v!L=5ky>DNCKn_2$QBb zWY2xi^k3cLuUZ!4cX$*xTDUh_+@-DEf21p~dQ{S3G!EyE&(L|0BP|=axM2`YKT`HpqiD7YI<(WC~7gM4>KLl zkFuutI>Q6`!1posUv~FjQhd48GNSCBX}8)^Dw#>U?*HNiZzY4c?N_n^U^2z3gQt1! zUGvL9HNj0Mwx-MX@6)mnh`3!_kM7}V!&qOrQaofzHq~I4XeOd6?7libB2uT^jBbwT zFc+9yiVcvMb@4~PCaw{7ECvZ5=k7AHX9&emDdZ8=k_%l|+P80?^pWPHNs)qRe*r@y zxxl6bL_K)RQw1nS-38lz`|{<7aWC%vgpq9Ao4JQNep!CX#t*`-pBd`7j=fP`6mX8| zw<|5m(xf_y;&wyd81aJD<~-rksIi>&tA^T=V3((W`X^mEC7iogegzZ33R7P^$GbZ(rUmdR(fJb)^tJ zwWv``C`6l0bn+vYOKh#r?H5lhWL12{l&(`KtyCd8PT0T-gJQ#YZL3YogMYFv+aKo7 zGpQGd6YWaZ5;SnZMTt9Xci3pLZlc zLql_6JiDg0wiS34euq|L8)n==kNyxic4)~ zKO&ExZAf$AUfyeX7>NC18mZ}9Rxofyp*$8yFc2d3@qwIQ%m>!>F+xR0VQsbwEb??; zIV2f5U=*UcQ!kkyU{qq6SXiisswyhj;M+)Y2PrmWHPpu#rsmw$(_cx@`4szMHymN{ zD%F=jo^dZvVhKW(>XvA4ODG87IA~~41Np#h#3f1&AHBFliECSZ`;=m?V+*%clrwp_ zfP7tur%}BM80xe}72HKLcU%qgvu_0?uyFCQUChkP90q+?K$AFSm*y&>%n@DB=ikZ% zY!w`@5{-fRjV_GuG;{gKlx1Y6!0COT9L%|2tpxX2v%j=(>KH>NYjjPTz-u7~({RYd zo5`jy-j6|JSm+*YmH5(A`JuyHHbS6oiU%n3Ky7pbXmY-5Lc!78V12$WmJ6f%y&rQB z%M~mA`LSepbQI)ioKAdKfp(`yCnVxi6V6`srpCzyQauf2FRO&5PH#%@xR*6t(3wk{|QYyfvg{rhPcK&q;ln3eSE>3od-VF~IxeVDuO75QY4)aBg2w0Ic8*z60 zU_mtU9F31E{mRO2mG&b|@%3W^IzDxVW*^a68K6$g|5Sb>O#xb)-;&MD54~^qc}*ZU zmX(!lgM_}n2UHT6Z1!dRcO)5E`pgLX*=E%)wRCsKvKbqmg`9TaA=mEot}aouS;vc; zu@ROAARU3&_NHzS<4s1e%;VC|Z-~i~uTz$T;205SFx^@+VqY_0yJF9kQTuv&+8VAD ziJa1q^Qll8;^B0b%xRUdt2_h-B|p1id{#P}W{5+WOX0$UFZj@mp!_n(R;=Tpy%Bz*5$ zX8K*1lv#~kO-*BJcfcgcr?Y##cEvo0Si3Mukb#tawYoj)As6Y6$2C+!3DlD%INQP{ zMsCO7kbJU1H2eR2q?*s3`Hyr@vX^Ri*?%uP@;|%`Sv!&jO#s~F46)2N808tc|2 z0Or=_bLrR@tGZJfnQY0H?M%fl<2X4}F(E}*~ z$%>c-gogAmeJ~RRjlfOFhw>M{>>-`68>p4QkF=(SDH_`?O%hgrw(~@9)Yv%>iaL(J zR-l#4*eMjhRdbF4HVsadoRo9}(TWjq>GA%GF+g;|$?8i>#KrMOs0*kTIO+hlg1T}Q z$S*U>s&pY#QN2qG*hPJ$XyReXWhsCs!NoCFK*6Qc&h-;ua71+3fbD5qP2rwwqqjyB z+FW6f0KA9F&L3%;zb6qAzD7nF6)?tkdEE%JcbMqRX>$O^AmXb| z&XRME6P+W72n_0 zh0&)UyGoS-0RXhahYK?2FWaLy3@ znMM$pEzoY=&+L7OE(HOLHLfr@1ky&|i161<2`N$_Nw@+56~yxSXlIm*kw}`28^Eliv~^%Fr)!&Z29d}@)8gZtnI*y$>>1`E8#8o z{jCH$d0|97z1C#Lln(PdprnEH*XlnZnsS)Tig*@< zSO638t_^$vDSSX>YYa*OowI`{AV0x#pWtqT^z}aP8*suyNC1+E3}QY_Fof)o&uWsk zAA^iMTp`k4zRDjn_{QibBzg#meP>`574wl*Y@#64NL-;)Q8tBaFM6t{eE4e-U=g_f z9p>+v5eB-1moolE=`}MI0o=U1KSUW#>LeP`@t)vH!#kMSE>G(l^nj)`U(T@JKo&pr zTZNYBn+9sT1{L50`R16*Gnmb#R3!46=C9}#MZ3WanVFc9#vJe?6L~#K$jVPR78g8_ z%h3tZ>;CXXohG6#Va#QbuZNh|vziLUv#tpN5rKfycHTUWE3>RBDk^b0g*~mHQ$xr` z0uYp^E*XAr>GkqrC+-2Z1XTb81Vtc5L~U)D!tD`@fkNJGc(xz?rG(1=8_!`pbKt-M z-;4};xEmt4mqzM~gNuW1s20y*g}9Qa<6EkP1tZWVPT@sCF_NK=%7kLUEfBw3Qq?l4 z%gbV`d~d^ifM2kmKcA!&ORy9iJq=%DG6As<641<^NbA`5@A>`_N zbk^}N8j@rN6uyiJnm24|6(~VH1*z2D5zoWch;IwGJXW%@FQEL3LCqH_7r!KedYq12n@b*6-f0cx2U5r zWFbgOFn{{eD4oJ$i?Zu?3Qvu*+kQXqkZf)?qx%tA!rx^szR8WV9r6Qs9Fs!1j^X=3Q z&pRTt*Q@czU|pnv*VXtJl&NTix9VnOpJHO90yw{4Tpa5(X)lMcS)Itc5OJIXoPdVz zwm68Xkf9uH_4w?`J_qiP{DFvOfs10_)mOT;;b!1pb5;&2o>l&IbEge>-^?e5>88wu zLy?YA=xx+y43V-Lf|B6wIEBP!D|M{xu#kvIXM2gm@-#c{b90X4P(3=uH{`y3Y`Vh0 znd~ZW2G?5_AmXAxI43v>6gn1bS>Ol5p|n{#RWZzCkm4#aE>rI7{~5~ua23E!4>R{j-$oMBZ z48*cWOAZ@mYXAVt-n=9EHrb>=s@p>qy*tY7Cq}v9nFdQUcTvVH^lQW z4t~4bcC4>d?Qb|HGc#^T%&2PjY%E708oS<=Xy8}b1U%m{OQTwDtq?Kl-kOp?X( zfx0z4N%QlhZdXDP(=qQ*f;w6rVo6+KwMOHFpu`)2NGdnsND~b?%|v_2)xh9Tfx2Ze z9-i9(Jxn7Is5&7K}p}cc8CDA!KdQ>%~SZQ}xiBC}FpR%lHh1Ox@y?3>{)gW$u!uiwb2wwb==V zsiiq6d%$X+e%Jx3P`~6<&hp^wgHp(FSeM;0p(Lk^YC-~X(c@oa0k%BTfucxsk5RhI z7bnHBk5CrTKx}v8DTS<}$1SDnWF_yW1a_=n%t+(qQ@esd%pbxlKu^S6*MnRZoYhX4!i$To7A|3+23(58oo%214~4cmgs%SO%c4-xbBiw5h! zk9_Mh!8C)AYy%;0HvS;<1L7cw3##dt19{0;`4(ISdn>N*8_X{ zYQia@u18(vMCvP!MNgeYNqc_!2)p~>g0}jA9gy6}siIglUFYja3q*;>c=POlqKYuL zfIx01MvUMSsV6%2N!sBR>_L2NCX)lCf;C2oijlmzjOWg+*W(M-rcBG5*Cf|((Fmk1 zjfCo`31W(6dc-0_IMDGD5S!y(45{F!0%5YiW#?BD@V-lV8Bm*<9}MM`KEIQBTRT9T z#ECE{G*5IcWB>^ogZdu;b?tAA=<8~3_zoaqgRF_GZyqb29*_0>jZ-LA*sR%p9-=~%$5Fy2Lcg(z4I^_ zo}LxZR8fIoE?Q%>NMu;Tm9MR-(Lp1NL6!f=3Ndq$T->F^!(n7MD_QfnsC|*?v6h z+i`+PK^A0yhURJ~qa^HBZ-(F?dQed!Wk3+%)`|jPE&Ni=poE># zX#*KkXFg9?V!w?}*+LH?Mm+LqdfPW?hv}Uu4Iek~HbkRAAT%y631DNJgEpRaf}(U& zQ$n&{sk`d#Qt1Cd(l5ZpcW!HoCQw4A;?)I&2BC6}oEzX(>sQ-Ij^p?S(0uByu#$qG zcHf&#+Ci@x5}i5^#ghIxto^MV^$W5n)GNU5q5?an4N)O2HE+s+yl~IyC8EdFk{ynk zUf8yM$|kRCqB%bIW4ZT6z?{9E<$+(NAjN2rfu$A9qk)(>Fw(q>$t*(OY{=R72nd0m8?iYe|$5!c)WtqurljY`RF`vnS&(!NO_C#;Xv4%n)m z9-=d}xG*(64&gJEb_wl_s4N%uo?knB>ehe+(q08j*L0@eC&dc|#NIdS7eQT8zICfNTheUN zydMi)0RzgV5K4bkokVuzLvy~qPr^lw{2>Z&bNW|Iz;S8b zA(iR`q<$Mo{2j{`(W==U+5<(=-0CsVZ&3>lSo57o*S-i{Hpl3s9%Hss*Y{5ye2oRIOiLAt~f?U(>fmwZAX=iZ&^qMMl zQCv9m{vl?UBfVNg+S(M8vBAS32Ms!nVDP6R)r!lrf2TQb zHOs>xYvFnH4*q|}1tAc2scHz;VLYaE0qCOdPQT_spXXt#Ho6I!m)}`DLK&e!1OaSU zY*i;n3L>ugmHu=?f@$4>Vwa75D&C8-Dx~GknZq(7;vF=DbQ)Pey>(8pxp~9UTDNHV zWM`gLbk=@oaD*Btzsus5>a}avg7$ksSqUMHNU}L~wH`iT;q zWipFqU+{22lNI=V!>=w3FAVOjL@ey_7_=NN93Ycx%Ae`?L!$wX!pEa_cqmD_-VJB? z#P~>!EiTp*G;_~258HQX<`XP+@~tm}?qqc&hlRVe=r*uqu$9s7f%XAgkn8}x`B2aB z${xyfCyfDe7Tixjc7@lr^CM0c$k%1}Kqpf=egW9asCWX`<_E`g2!m~7bbDbhcE`_h z?*q}6PY_J>*|TScD;?WSL62(xuQKzy#}_*<1XZh8pu3RF0BPK0@;hxO6IlgYz-TF_cdLSkHw zD4$RkjahYO|7|0;Xm*!L(LptnPTcmIR!{f2e+#4&r3Huobx7bn-GD7}@}4*!%xWX^3a zF`$@c+*r9?4wZ@lHG1@O^Of#$@T^(ZJJX3H4fl{mYvfl?aU#9Mdt#On7OXdAcD9zr zuQb0&m{%!ik~Z}x12Lt9Cs!&A;Jfq4l;ob3fsS(<0~6>ROE;u!1#Pmi(5K77^jo|3 zjDdKZDk;uT!4ayz>&7#LJpZpXj@5ii);W5rz{95|!S7=B` z$UaOq-p1U;09`tjIFda$e9wmGtp4v6`$WFi>!HYvUqM1PrdEn3G|T!O=pl#f5yI-J zhZJS4r{{2nrgyhMvxfceWo$BS98?oD{$fztvbkra=DVwGZI~q0@WVE9DdB$k`IXFe}!(IdgYq>zn)8W5+=SP{&m8G2}p9zdplw4u@=K*kb;P(TL zgw-;mMrjO`UsoUdrS*@F_f#KcMuC<6HLUdO8WK;#%YUZuZA3Yd2_@LRx)*y4))74h>aj1VsU`pcGD2fPD?J^AnJq_Y2%0I1`Q0_Y!d z!D__lmfcK{3fsOBAIJauLdO5F-toVeZLjtAp8QYiME~Ee+pntA_CR9#GF1G*n~1 zZ2yMXuO=IR=_K6Zy_VATyFyl1HSW-zp~$}Kpc2vyo_A)A|u zc^F5o=3WUFEctdcyzIMe*D%ubZ!_o6IXaKWetfU6joiYjNRFB*g(=t{-Og7YQAw`+ z=_!c`v3A$$NwWS>6{0l4P(@&O2qRq$-JUMro_~#E_k)3f&NY-tP#rp6MZ~FzZn*1r zydh~gL`L@Kehmj`g>@1*VcnC>sKra6yBOeotfHI zs1-jsD@8o_`>~!ZP{i9FfoE#>6S+G4z>=)Vb&~C5UTXTxziu=6V_=YCTIBB$Tz>M; zLZ}s7IvFAOxvP8E{q6g4`uyu}vp9%?9!_}$d%kV|t>%e;&U0qmi+{jr7a2-+ZhLA- z3jBu#cy>P7huxk%iTuBxnDuP2&cn})F|Og|ODWu9I7@?za=vYYmQ-~Tk--}_wIa!H-u`y@QIMng+h6SVc*jtMcIN}x^-Pv z?qD@x$Jr|d3cA*Efe>TRhYl&<3}$O!)^kueRb-&2uFmHyiUNVti*Y*x2~+WY&tYdT#&=k(g4AD4Zhhrx_?aSS{x*)E60wEn?h zfbuX4(o(B=S_|Rc&UVin8D+8U)`Xb!e=ho5DpbmeuRbV$bll=5DWNSbd2$VgFI;L7 zan|hjIAXH?4iqHQ+KcQKURRE0nU#5C8vg zfu39Qr-v<8nV-@<&6J5&%97i1NKE=afr=zUmYGQP#~G>5{mb61bB1l@Zv@rj&lZ^s zrTXmEO1ePDo)O{k@n%Js7T5i*mZh%jOKu9kC*b+?pL>2Ty`~k;)r(OM2qn+c6!AL0 zR(kIK$dUKY7B|Px+FWXh&rYz(2riJdf;sS4Ys;@7$M@mmnd_xj{HL{}R}X7%ie-q< zW9ZZ9YKS8Sgil;z4h6aW3y=(ImkaGE=m@~nK*N}fS}OQ~Ad zI+g2|?z_=bJl%5AehB{_BZ={;f4t4f#QgI@0{y!e&w2N)3hZWQscA`4_--M5ciU3<_PKA+ zZ-38I_TKxO)Pm}r?jJts_Wxzh;`hv~!+JVo3IXdkT)JFg2rkWR>C&9lp`m)esDF0! z^yzbQ-bbFi(d4oh1qYz(EOhuhnL*6|qHiJb_apuV4eQSedziHFw=6`LixqUwXU=r{ z_ceKK87D?XYE2$qigXAbiNz6Cv0t}lGUehjseoxzlSXR$`=E-=9ayLwwUUF`&j1|S0&Pjm>YG=?5o z82>9}QJ?duxr4h_#zu6=9hyTK)hluXn6(?p7)Q#+&$=v6^2|2FT^uP}$EhY922*(> z9q!nRF7C~+8l_4ZW4zj0_gB;VKGWZeeWv|yiw=K{G~MVFRJt4NE|Za9=NrUWFCIcc znjN@C<`^csFKec_~vjHN|m+s-|!1Us`mI=^@caCw_dBqb{4X$|4@8eYXdSzT>n zCM=hijJD3LdN{15>tvO)A;}YV5}R@0(i;_}A5BrkYfPxdMgrdl5AB>Gj0Y%kWA=o= zzrtu060?ZnA}O7nT)G$9W{x^WuKYMiu1toIM=04*Y2?1pD@0bE?8_ z12>*HKf@Z`R$z$}S92E>vx^dQ)+9Yl_lilUD>>*X<>_&>grc7}rN@2O#Wx$lSr$3! zDc#(fcfJg99;FP8I$7JAdQfK@_dVPE|3Kxf!+KGiO^QPoEyOrIi|4Xv>U3cHXL#vvIz|&bL2SnuTUpU(JwXfWH!K)OmwOrT%c?%uM16?y%q)m(BHk@(M1W+lhr{g>b?j=|e2#ckuUZ zIa+(-G)^XPah#C&fP>eqzRTlGX-!dxMd#aCVuq%&1p&m7g&)|~*8NX7N@?&qc-7L) zx*Ajb52HEymY=9(iD+m1aGx_Ae64#9B5CAuGi>oewf3>?k#6_$*F-P+N|x^=tBGYAf>_D z=30VELFNoPzEuQ=UFgu9Zs%4VwG8)9;4aF)9n?&pw1|Zh>>*jk$+}K9vBZAwfD^+} zYes71Umg8@=R1fVXY`tl>KglE_vN&yKp2BJ9JIC}=5e?VH$6hBtru^ZNAEsALtXwQ z%FM5nBVED4W6k3E39-dm>c>r`Iyur-Q{9AKY~$>vhRTVTW9}oy9k+OYPsB4phE)Ie zk1_uAdLPT$C*l1H#0Okz(GtbS40)5|zQ#ZH_D423nldd%BPi!GjI~v;(r_<2Hny-; zhy^5w+l)be7mX*0J!b~#3a2I-6Y8}5<~G+m zPi&a$Vwp_lla9eAzlX?on$B6q?`DRhpYC_}>!6a+tX=)<#{w~+KCJcL?Q32dQ&VfV zN~>Kf^Ygq1jZP6_Q~O7p#7yshq<<@Zl@J>Xn_xQEHqnXfZiXt($mHj7)YHB=Nja_v>nxZH!@qOQtE1gx zB}dS;-fgqvCPBnJiRy^ovx{c1su&)#oW?ikp$u&Z`4qaf_q>}E&1Mlr3>E?2whkUP2l|xjp=PvfacO+IgqNnE}#J8w-!J>D;5=w|btG2TePiVX`dQ6yp^vuFS~Y{93ViDVO`i$+2Z}7YNFv&x#*cgp{Ye z;RwA$qc90k=v(kt4_twR%VT_#%lJ#LWz)=+-tp5COUnR<1|F?*8kLK_Yg6fq&R%bB z0_NTcBMF~$$~M^w5X@|@*dv_pNa}- zXRk4vqI;Av?(Jsax~P*8zNc{Wt(Ze)d&yP_8~^@t#`F`hX4y=1V)k*JcA_%U)MLsT zpnHP8Sh@LzyCex_D|ovEYTf`s-gjsAyaN0FTP7>KB=nHjePfq<^P>;wES3KIViD~^W=)C{WIgNPN+a- zx)+-I#l5in(zB}%9x8o4*UC3tmH_-OZO&nx%~8m2vN%Mv|IQ*P+{&FrxxLu9y`J(PovXL8CZzm&zHdCsU1htqSd!(o<))nC zOJ{ADMio9mV=(8U;5YUcz3vodtl1_{1h3uP*uMGQ{iA|G5pIs2cxqwf7$*v_w$;d^ z?Yqgs=ovygdSANSkpEc`s?TG3G*o{LGygFgvcLJcrV8aAIBbq;CG`r3{(yI$499k~ z<>W$?iq)D1Ct%FSuGG<0IzKP>Rz2-vk*H9)NRcbyH>yyG#xIIQk0s~5v;43ikUJ(4 zu;kp+PZL`b`M&%nOY}yxZ%I08ck=>&?_m3hQkK!o$mAJ2K52E&%s*+S&6PcW+i_=0 z+u3C>0?kiRAj|XRit9WUf2jZKy*yM@gY5l9qD>eP@;t9EkM~n|!M?XTS)!h&))<9| zimZuNX(e|N?k(M+naU`a@jtOYZvWI>p$y$ubbIh$KqwkLrSLb@@-zO{;G{SPOI}>D zlKL=5>4>?K-`auQ$mG=Q`sK3(XP49#r=_nm?o|t;&c$H=1SdJdq-J@(!%XZu^2$Yy zh};&ZVwcJ2^ghNaljUzV7!RU%AUl=?Vi#Y%sMGKbKRm~IvB7LTWqi*d@X~*OTGSRsLj4UOaj*db?!>Xkqm1FY9f1;kG_g6^vG@WzcLd7fPK&_uex4a~Cf<@C#42`8xZ+mc?v@xCi12$2Mm6Bt+j>eUOMi6OLh zTT_*$s81B7OQRY^e{ zw0!85u1#z;Z!{;2bQrlOqKR&5M7JfK7`Z)I@-QsaZa+z-M&WoRXyE-%R8)or9^G_#G+nLYhf$cm3ct<*4tzVbn`8`P z_VPy5Nvv^~E=q^kVP#}iygqt}7(FCPIcJq8Ly_!8AlHeLwf9DT+HJ|#6*A5>mueBr z@t0L)6bo})eYU32(HZX!95(B~Wcae9#nW3N5uUwR<#kfV~*jy*0ESBmh~h z&Qol7>E)!6eu7TOseRaS4PLA@nwQ%4$=Iz&=TY&V#A?ZsoJt!YQm&I`@UExMPtK{* zz-OX+#2tpGWe`qh^nHpqjHAiaM5S0cu{$1I0p1^`12LO_GlLHZ)5v};hG?lw6)Ih~ z`<9rHp{)D2pN6J?n|G#JvVOsy zFhIp{2^W7r20o_-DO6Jip(HX0$jlg^Ro$IDCcm2~zTLe55&W({g;2-8j6d)>DKM7P zm6Gkx`(R>IVd}Wb-Y%oWV7AUtn(q{so)}AU2Q7NE*?OEkQrH0tWCFE{aeDlY!2=6(4t+6C!pnSPTEQgVcJE4A58e9JsR@ni>NY*31I4Ea?7 zZ8NFwymsyNLRqH~KhxQs<#FIiA-Ajmd^bN};)znT!?)QGvPi@h4~+oxKbd{U4Bmk@ zsE%RS3YxJ_AG)T=*||QtKU$#-*1^x(alm@IhSfuL?5!XF+*bEcw!g6iGcjbWATQIn zb4t$F`U8{2YOhP_kRDMut^!N8G)}8zvD90Jt5tC==x&#^$`kOnXJIucpbdaEnjQ}d zjze#+KI~IVUDgI)j7J(c1xU9C)P^RX_w&291JT{JVPIAiwN!9Nu1?2=6^hAFCO@^d zKt4-3&XB04SJU+6stl4>Q`0$#(*=4TF1!vH!*4>`b1?|T5xpLA;6T9Xu>x>tFwxM~ zdyN54S%%y6ubRD$weqdAA^?Fl`&^eb+9cS!!$>uQw=+8$g=E`P>EH{`UfkOk*B ztZ99W7Gq9reJ@e$a>uz|pCu@iQ#WbEWNCPrybvkY1C*rXwL379fi2(45_`t-wu2D;juzNs>rF7xb1P$33C`&4*Euje*y(c^cZS$QE$9EB~tf^(nn6 z;YwZ>+gbj*zI?cdUVNghNibvCzIVZ>>E*>bI=acFcD1mYVD$I&!6Wo61V_8qWnC+W z$B+xlJ#1S=6z^9ZdEjf7$*}P`PY|WyT)-7_LoY0NEG_%yP$`#SSv_9*hK}fy6uYV; z4kawt=*qRYE{Y*OtyG^o!RaXf8xpkL9xV`y{x-xVIHtWi{*~OKJ3s4-zCOH-vJqs^ zg+B7Udc9eO@nR)*q(XP;B*&w|KK-U?n)3T2x2SBz$Ids`(5cyf*1eNBX3?XU65M`q zVM;Y8$SO?X9rfvx?~9}PELi)X#Kk~VcN5D0S!m1Y)-M}5$X=qIaXnM__RYMh`LSxl z>snU_)^m$0!%Mkdm2j~&KlhjbUwZ$=?dHO0{T~X>zMUg|z9oFYRk_q%Hqtr~voLym zxCy!oZYP$qH*exb@mWi<$)8eVR5q~Qrd#BB0^=(sJ$Rkf%Hta&@5ruy%A|goV^7v- zA?Q6~6&*~i<4Y53G;{wm7RtKxV&@AvaO z_x-wGuh;!N=?4t=?_!Ub)ehp+D^2U`gA6_XFYG{pU;X^qyqDm*-Kk~Cg7-f&%$nWtgQaoOC=9L{j*A$C(^@q;32M-eN%{#F#CMVjCE4hC3Gvyp>jA+Zy znFY75Jq0aoI(TzaZ9+0Q5M_vC=*PrYsaKw&#i$bWeW=L6kfa{#qyJd%Awc%t7dvY z9QZByH|-7(8L{lD@&q)Y({QQmTExbra8YUI_U9jd7d0K#3pZ9xUL^K`_-gl0W>hWNlWn(kdiLNth+j`}3fasW7O(}a@P(l;2TmWb~ZQ(n3w}qnEUejB*KHZB^cUxP| z-+nCnmYcv=_-rB&AyLa_G zQS+$teeF&&o7KtFDgT~K?n|AYc_Z*~bbyZ2v?f*j{8YnPsxK37r+F{NL_|7`y&GUd zw9(Lbzo7{3 z3(;1-S8$Tx`1N5+XR`zj6{{iq+GzexPOAk+n`M6SZ;MjrytsFq74X&pNO=-7UlkM6)t@Q!RDyBQg%mU}g)Q8v znfBnMnVRl-b20+c{tf?PJ{&~lM9=-9JGb`;4lGtvTN*7=XM%g&63@Sb`}mTR`6?;B zm^S}Fmp^hv-oh6}>xU|Eg$G?i8d8fY%fa3~*}>Foe`0r1=aTJT%I4RawTv>rW>8F& zE{Hi79wnoSyB}+zzPU1eNC!$V^%B#H@Exk5Y2;GQZjW1$o0XCiVcxwGgU0@S)+a`} zvdR=+eC=+f4-c)m)5RhGE|i_J0aE_5$880T4AN+Cg%PM)f?*&Ga}bYBL{Zl_9WONM z#0)QZ;*eL^>D?G0QRcikZ6E_8gl)8(ah~g1CH>?wKi?U+?kKIGW)jfrOt5_`;h`o2 zgR}XaMZL@ae&L7_vT5m0!;atmTh=!24dcO?gm8-!w#U;c%O4TFM5MP$0x|C{JUf!+ zH$H9wJKYL4*S%C@tv~x^?%IT`@uJSz?u0ANpmQ_b#kKuA(`iE|=Xn%ZcL5wgxmdpK%AGSsPXszDfP-P?y;-tXS2MCxA; zh|z6XS=b~bkKeY7UjK3AtP z3q;kC54)xSTu`*-AyTUctD_{xn7cf7LSn!K!SnklVE<8z;C0JM1w^dZc7a`R{Ah&} zZhJ4}>2H!jL*h$LP;qlFLKh(PwaebU+dSxXm}(*VuQ5V`dhqz_OkWV2jON#?u?oZR zF?(|CqtryB;@?+M>~xiw&EMJ^WvWF7j|vc@QoKxYxYd}_wS_c4XO*imR6x^YOt;Hi zmS$v4Yb8$?U6gGFh|h`=SRUH;tlp~h?*9^V`>6WyAvHSnhsnL^ADI<_J4MfjV(_!# z|7Jol-T~|w_Y6jTc&1PXNDra(4n(vl5YqyVocinysYjr3M!K6eHC}v|pYLmBn#ma> zI`Fy%nZ-4FzP9ER9%NbvD8<=nc54 zgE&APh17!O-P}wvF!PalG28_zDiqA>2(ew>1?t@P<~d9!2o7R4PPeMK2 zvNBh{!Q(^335H=+JbXt11_JAR?b$teG{H+ibfkMl1(+7Dpot{GcJ z<#!Yr82QKulLhg0J7eum+|T4Rzl_M^8LrT22pbT#)F$o_Ft4CmHEE5|584{m>VU2k zxBIYCdC zr2I4{Hv&oRvWiHS+h^T+&2-RQjO9CzPWJ_+%7+)2K+!KHu@M||Ty+uz3#8l*6PwnJ zQp#t}oq8A*cB{Hjh{vF8S=vSZ1b1FK?61`QRuG(4E1vljSunBG3&1!lg&hQHI&4R8 zM;$8QKfdVP>p-4J-ucuaDkKZ6sPRIaN>r3*V+6M}vIh-cI7f5*rLw;a=*?eqEgq~x z1ISf?Z&xfV_zI6Xe`lFCw#};s);*7Lg#d;&OGMmjWxPAqTXAi9uI1q6dW6m}`Q?=@ zm9$co)`4(0pu;tTM&dFzbO2Nsqaf2In3X?AFg&9!P+AWR6!jkKrxUULJ}Y23aB+A? z_ZlwA;n9wO;AaKofk15KeSYVQ;&=h*E=XMnZ_opZ=QSUu=r$sSfiT!|(QGmhopLx3 zI88*ug6&Xj*Nk53;=alwS`iE_$gb&#rnB*brmAXXk@}s~;Hv_XmadzRH|D(1<$sK0 zzq_v4{bQ*Ga=fn2AGhU-t8xk(X}#udWznNsMg?)JMAy|J7WVzyX1Wb#m(+5Iv5w2} z?898e$UXz36nEnCnM#w}7Wl_1}oq!(qaI(AH*zj1Rw1f;cC*-xtArKs~w->qFA#i~X-9ET1xp>0z z+!y*)lD3J=+R6Uo{9xugx^zO-=x#4rK3ILm{cb)MdR}~wxiP52LHot0K%|JFq8itN zTyYKdO*Ya;Rp}WePq-m=g~Q6A2qH|tm45Eq$g+c8I&pbLa@&W;7+fp%g7n957TE(0C5UAkYd-iEE&J0S-9&RF2B)VvUBMeX>gqYqUmyNr<4To!J) zgOH5OLc5fqrO+l{^VE2gbHDGVR*kP{>G;wCS%3(G$o}r5@8YY+fR8Kc6uOj7iO6?F zvu3@XOh6aCryz<9|sGbK52&N85d1x|ZO>FD_aiJv)I%s$E z>9I?oo3gnKcpSAmpqqc`fzzr>G!Pmp)Q#zEisSr`=gLf)Cfd-6ag&u_BZQ?l2z@5$ z$2K9kr2>i8Sx8Bjcf=k43v2)zi>6w+#*3YfzHD}Amtyi*yW=o+U36EfR<0$3HH8T$ za}L82!id<7{Dzc2y_K+%XQ#cz!;|nU>M}4E#m5gH1V(Zi5}c@m165QQ)p)>3hQA1xA>PHothGzD|!o2P9vDh;GJm94g-Go zDAud&IBD{+!+O3i)y4`&Hog_!t9f5*yODZBr}PrkAmbi={H!Wi>DRTb7w7D}LvKBu zh)(@N3Cpy24R$bMRNb<4Pgtu$)@`uW?C-ZgrRrIhEUO=^Fh9Q!^w?dutYumkm?Mut z(?RnjgiBrr<&-OmvK@edkTehDYPuIt_Ch00!W0NYS3WKsaBAE)|M8;L%lchzC&wo? z)ve)mNVr0_Yq2Ix-RN0Qsxbk#Z4ca|oMAmTN=r~QeTOamrW57E*(z$|E$mA?8kv|l zOSXEbJ6=1Po;V6Bcr>=atPQRqX1Xo%xnv+Z$E=klAS)ekAjrA7HYB!L znbUrLSuL<*7pl8ivs&yNZ!tDY_@1r^x$1hldDh#mX ziVOmXAW0>)<<pmZuTVw(y(ME0Y}2a$OU?OFAQF$ z@iCmA4tk#6lB}9cqiuiYd11QbM3t=d@*qLmKl_5tp8ljls%8njiB9W*rbz2U+pS6z&Gk zO;}Heu-o>msRha5@J875BNvfZ8Y6^8I59REkSRYVHjV>qYzLsQ@AVLH?R^ucIqL*C zQ;PUE@&?F7lkF2!6ldw$UWe#@V1%0EvF!sG{Wg%m=6fF(@sUns{ln9q?}_7oF*U;0 zs^u*g6xPhyTmY4Gv&r0hH)vYOD-z6);xi|+r8vg;7oSnLf{`GqK`wJuVD{ha0`Sk| z0l|!Tn?S`2hyxX$t^R2%*qbH*ULj5&z`CZ3;lhkR$BEcw{+JuSFMB!J&(!i(R1ef# zN)Oh=M*@PP725i+LjMB`%Sp=qkv~XqRDzdUy4X{La#|>7J4r z=0Rb^6cZY*tN#PrX8po`Ko(T^J&Qo}+rJQ|`4ZKcTJsbMOYl2n9TQ6UvJC2k@fiAd06XnL7xh0628v~>{TTY& ztao?bY6dmEuM2Y)lOw*hxce@NEM$|8y`T8|*Z-b~rTG?rIeevNLtYee74q8ax83~@ zkw0L&Ka6=DQHEwjz542&Wr!$)Y~P3u(<Z8g^MRhNslja?EO-sqx*Rc zIABTTUty#EhXxi7J&3m4yKf2Z|N2NI*iPG*$hZ{qWh`1oWn6rRj@-8do`f@S{!EF3 zSM0U~;{E8)jU$#Z&4_u=-XpgAR_IjfzvD%u84CW{Fh}q*{*#F-1W#{$9BT69Kq1-p zy*taW_@7Vq3Q))&lk5ZkVnUL5xnZkkK_1o)yP&u?5hb!nkPYkhUeS+xykA}Ajfptg zTWdRcE3{FYz3nH-NcFs!&h7Rbn%`TKop@Z;&)O^u}o9zi}u{G+xH z_pW%*3-$Nvhc8r=?|xaL_kW!Bd+$7U>G9s~e9b6Q*zNL9#M=8F!}wo5=1)r9w`s$- zPul)xEZBQDjCI0aQ|;fc9lP}3bJ_c%JHubL;{Vs%PF_Q{Qx@A8kCh1ddsjq6H<|X0 z@;w5MY4kqSa_)ERTTWMAZHSkofjy%et#SLSBJG56O&s)=`;Dj9Xj)MTVJoAJhSx`R7IX zTb9W<{r+^|aeL2>pAPR^*)loULK_$5Piku?(Z4sw@Ij&})lwxvxrXfz>1g9WTt2=0 zWQ7euO#!)Pk_k}tKe%(4)&RT;C--Mb*d6aVXlNs9zpx)Y?nPbaZbm#XJZ}MEgT8G5 zn|b^kvZXpO1O+I#S(#<1n#J>bvgTntY=0ZYFm_N~9{X{(q|hP{>XE47DzZe^)jmxqYPAeXL55!LRW(J5vT(ir=xc459iH(5T=3y< zC}c#5+$G){alLxodh>ib@qa-i>XCdxM36QZ`wygWg;XR}_rwmeRk&Q&UsD(bXqyz|mohn=%C(=A6C zj@9C+wi`d$5|<~=BVL)f!mZ~V1HJ;&v|S0Fp&7cp$fWT9tvmNCO5pkid|2(OZH6%{ zY6SlWCIZE*a6a{jKcV&51|)!R$_2gSW_@s@NR_BaWDmSU$=oj~q$G!@z-!ok z450FC_`*;w&Hd)W91_l^He7r7|lYIMOwt?$o`}f*0|+==y=Lz$6z7&n9o{NyKF2W69IYF;JLN4F%t(k_KTaZ z{%OiW=-75xEBR)1yXGc*Z#lW&K_&KFJADjseuFzC73BA^MF<)4YI5+qCGV*0WGW+}2ymgD>u6q`x zdLMX0%cEHs_oc%EZBLG4w_SR2|!R7wdxPsO}2*!S&hFr&F2!=p8JnX ztKP43Mg+9wdcVw?N&D_9QwqQzawb{S%*>3jq-7%Z!)qKTjx|X!{x9`2;5JspuFik@ z9yeVz)702FKcRVm^8KZX%Cr10qA9sPC!P$K4+u0UV4E0@pip*Ncv)FBqu3;%I_b3A z0>&Wo5EnwT5fmpU=HDX1;zr9;TO671ZpAjAOYLenT|4TJpIW};$O`b?m zVd?zA=u5q{e%jM=Vj9200s_ZuC&!{(gFC1+ev72?D!FO#C|X~xmJLuVEvZgw3F&Dv zj9g4!VMmvA_2}sMkYIQ3F(q<&)0J^6_wII?MH)(rN3>m$#*TM!L2MJ|iu|%qdQ@8U$4P3qhJ9CO;f|T5>kc z=_lO5B~U0>)u~09uJS2b?m>dJ%Xl=K(9ZP86MsS59+t6@fSMDHtRj#G- zHm9@AY`ubW7K*$d_!%#)z11vs4!UK=7geo=7Q0!emZlcjW^5+>`T4yE2_=uxR%fxj z8Z*}R>N4xJ&QUJglJN-!g%Kb%v~aiWn57l?nHjV9dU*QuyHz?RysvtmqkhhYvo z^HITN-FNKp989kGsMU+K>^~h#y15U?a+OW(f(V#@g_I(M-*-ee*3QbTAHe0|&-AU- zEKOx(=m?pPPOUwuc{|aq)7>qIe9B->z4$kC*Dhz_Z&=jdeVG2TdSm=UrS16KeHgAJ zm!HJEj$3#1(<*3vh4cM8qr05sQ(YF~r2FV}t#EB&SUXv;Q zRU_~c8!CDxdTGyRa;>4xW^{5y(58VC(AHpz(U}fe?gV+3ucF@#+X}B=Uf!aQS}^{c zP%!Yt!Dc(Husl0Jh4B$JtS(<#(F$`d_o_c%ztEO&6^=;KcN(&qgP-EHY>c9Kl&x}i zI{mU%u!d9L;7f|-i+`3N5%&C;!Gw2IoF(o0&MGsFS&44S{n*#t4~S%a_v-GpJ5o|% z2sUt6_q|1JqTZVDoHj@C9=RR9N3vT`P_{`RJ($DdBZt5bbGN8ZbYdMh99j>{NAlc! z#@#egQDv!u(beHSVLrax154@pcKyTyJHExwt=tN70b#HDQ#O|I6vvP1rT2vQJl6-5NI^q5``G3~( z?v;pws+JD~#9yKHx}WzS3f=qOnt%8~2mzCOYSZ3V;jcM*{>~fdgV9q5>w#DLKVB*d z@{^jmFVZ@uZe*mC(`Is`4@cMk9loudZOew8*&ROgo*0k(Y?9hu^>puJV6Li+jMksg z-M?xI#ea9X_caOL|A5E-+mfXIY8CrmBI>y7-o01H?SIX^c4D)yu4lfeZMc-dl$1AF zi?9)xvA(LPyGp>8yfh3UUi11drtbdK+*SN0TThmF({QO&IdA)l+WT&%V6;4O$>A@S z|8C=Tb5ajEKOlVB-2WYWe~(PP*5s|{JTSN4Jf7g-(5RgQ!tDOb`AG>33(WVg$~vx@gBx zs#knGXvWchRPIMq)bU3a@^dJPKh;uARzXi)+$7~EgPr@Bq<{ARmlpr8ab=X@PE_PYA6B__JX}l} zkV7$QLOcFo1u+@z{U@y`nM(|29{-=Y?Qar~HrZThW%@G-=iXO-Fh%{T!Icbt3r5Jw z*Ih&VD0df`7IZU}9e#+@X<4`AI_Fd$)21NE6(XY6OzP!%-)=uj7(D+yZoPXUIh2Cm z?gPKd*fhbu>b;)QTlsQCl-?=d5jjh*arX-#goQSjefNf==v zd?TNDB-4p+h&-<3cwm2$rzB)$U6WA;{P1U942F=L!+c=ucd@t6K_Dd&Ys7_|QK~w{ z?Z+Nw?6ssI*N_-_1fK={(KZxS`v`$@h&aPuHy3uIidAY-vnfNbD0@ ztA=zr7|r0j-ftB(qlo+DWcw^cL3dX1&WlfU7sRFPQ4_!Us(Y5H?chgexneD!7F4&(XUL=lx$$Dsy85E1OXQ6 zns3VET93}3?lS0m4={|}k1FA2_9|=H6P6pRQwMMU`hCc|U#tL*{?dF&@1kw}rxPFEx*_J;YB;qYcS1Tllb6UUx)Ps1FUn;_qY zAqXlzw-drD16xqD4&R0UODuoN0P@qIW|ibX+^qdbg_vBV25ANFiMPttTLgYW4^;4o*ql7Z>$hxRS=zBSNhwt zC&XFlBeUqQ>&==2%5686eg{s_nSO`^IMNP4d$eIaM=PtNYUl^-hKSMlMR4?KU44Gf!Xl%W7oJwit4uCy5+3x^?&&bhocwfOUhlLnWh5HkRWbalXYd<4 zVeY|1W1cQoN7DVuDHfvmeM&5_r|e}m?B#j0SA0DXNhwJ;1~k|G7?WWe&fRFI!cybs zHbY4^(3FN51{D0~u^SpbOO1X>#4GLE+g4hD`i+ANYHO=Bsv!>_(SIwx2#58&7_={d<*g z;f9erM4Gf9%gpPOD5%$BVo2fo`kmSZCoNzehrQxOVp+g%R0%xA<&IEpDq+Vv*{e3) z?>aOAwqE25cU&1y_z_%S+9tj8oJ4UJ0AZZJJ=JQoor+w;Cq7BN2tj@4QYkNQ{c4zC> z*X7}e>gRof8i23L&D{XVSsdubeCKLLvxO8MdCt%1JV+_)GMx7!@2!bJTESbB_?FAw z6L6zXy(G?+X5SH=ofuu7_0Jft_PsDcDd0;jqLgfWflAc%OZGIq>Gs1XSMzck4I^zx zh|kUZ&?<}sV4HVw=um-vfY`vpDMC?>95W1<`p9-JT$=BGdamgz-DQTpJh^B7kDq#1 zmMmr_lKmuXE_a5@4csU`z@<|v>#4t4N4X^zGC6o_cIm6_ddemJ<+l^}<4Be(_=C^e z0WwUIiXlZH;i;1rl%2UtIpXa2v`6%uKswvWxT~Hnz3;p&z^+5dl0C+4FS%rhuPeZGHI^*}puP^tiV1r!}?y zb_Kuha>bB(2+@1V#nG*YFoKLMbr#sjlAl$bNO4;;XbA)V#1X~csR$d=RLr#Wn)02m zv-Z^Q_rMHH%5*J#$o@QLYcfPF=d--l&2`qvO?!4S7|OzAi5I;-n~M;tZP98&(Eb3G zLYRoL2Rv?LGp(qsQ#)`Xiayr8dDg+Ei;xqx$logQ{mC5S(f}#3mx=} zer%rH*bN-LdZI21=7F{KEZ?8UxA=_~&%(nl(@! z89bd1<^*pDe%JC76AS`3wgb;jO*=|I%iPqCcq3OAIW4xTC+T(0$$5wXm5YODe&HjV z@;O1J0gE&P)Dem9BcJxbU(!axy@(X8u)Z%HscObBn}V_?E{j4XnP=kQ^w#Y=m_U+_ zkx6He(FlF9@3d%!TM3ARxvTq`SMKJ`n*hP#UQ}d;t9^Lj#N3736?ojHgs78kTc&WH zdL3oKPs%C6*fek)*Mq|c9Od3((L+a5VWe^_9$)0s%EodC_=&?JZTk>7Ttzp0VokSj z&9`@@bxUWd7aCSa9?`p6&fDoCZ++a;IzS)N*QZ^u95Pm-mN0*Um9tB`>(zVQNvaHn zKXr^*x0qr|*nSYS)8KSb`MG#zeYI{%9wVRxkIqb{*vHt*zQ zB_LVc&IHScz6Weh1Fh(G(G7(N6bZ-OR>%!hgh9{U9j0od0d-V0x!!bptM}+IDhC#^ zPE<5xI7S)TUTM?5Pao*Qi9ZVwg0h|EhO1rma6>fB#V9CTq#inFNRo@1U14|UKr|Dvfn zu6-bybzkGEZE$~ih@uT$0NXaE$IAs@74FRdwg`xYKs@~`Kwi$LC@B+Q^KWG|BgPhdPII%WAY(PU%mN!1^|FONq ziM%82VcV24k;--c{ui;Xc5vpS2YfL16KDUw$*%Q?S5e{DwPrK4pLjWXl5F{GPshu- z>a{(v?|auSloAi?`Bz&uEJxm8o&F#~$5N-3&j>=I&97}q%yHG9=!Wc6ls+qaV@&6e zmUJG^NHLMp!+ju3kpO2nTkH@MxqTz}cbrI5m9s}po1j?W;XuE@dx)Z^h918=M&1Ur z$8tNjTPrf*7d*d0o>v1xARrFOjnf-ji+(hs8L$i+I58T)By7+Wb2w{9wj~q2ugb*t zw-#n+J5NHRMhd|M#Zx+T)HHN##<%i=mN8oDkc5FWeqSRBQb3{dOSd;nT{gNC{{RgJ z8Aq3Lwn-a%>4wPl&)xAb7mcB|a$K((qj-{Yp%(zBKKr?U(yu^#`1JxUwmH%V385iU z_w5N(tEk4D2fSbNU0o(jbcywt>y`UeQ>42R94+uw z4gGq+dZ?Sz5h?wr1Sw50n)kEH^k6J8$gd~*C`dzAQZN&t91$Mrs4TOykyOtsu(*)> zxWCsB_b%12=~>j(MYErlrn(xad>tpT%{ye?0Is6L);GV#Bh#YWfC&3d(SEk$D@7_V zV3F>e5^uirP6<%~lTS4+(>?io!LW*8-kx=8@RG+k)6e$A=X4iD-^nIyUa0F8pWkk_ zX{`}PSXjeGKVtNl#VIK{i?iQG+nxw#Zsq{&R+On6&8 zoP}mC!@!u0&dXoN^qY>Qx`(+e1`c~D=b^|Ld zM(k=sOD69&D$|cG2-nq{d#CJ2!{d~abY*+J$<1V|Fxzo!US^|aQq}9#fsb2GmR}Jr z4hUMuUlYc)HbSCwV^+US{^az|4;;P*K$RkqwNuZD;P%Zg$E?caG6VIKoa8al{#paN zBWUW70!V+}d1+w}24f)lB}rA*%_C4=juaRLQj zCNOTl__!2f2BfRSjo6v5n6C=hW4H7^@v1`wH4fNwe8eGBRipn149Sc1<3uT`M0?mp z8!aU+P>rx2LCw%soEaFXcS~SrfDkHVJN^g-`tu;`kH&mFfz&k5x>B<{r*wW@CH}qR%Delpsd@t^Y2lUcE3>IcDj4zRs8M*K4HGySk~b1hpfW_* zMaZF6!kKv5nFu%f^Ntu8ekK{DXi^kiWF~!29cTEsfK}k?-Di!4tdH9ahcyBFwaiPs z$xk!OkA~tP!#&f~)n80Z5~hX9Dtaa`5U2Wr;6#75&Vs-3!XkPI>QzA5EyGt#T4RSh zAY5WMxpzrk7R(v=1HF*$Yo}WpwRE+Zuky6Ru8g#rl#xMk6 zgIB&KZZzL@utBN z_`;@YV5e?2vhkaU{CVzgKe8u`nS&xx%51?LYW}(|yfC%_af_j$Qqi4Nkzfonhw+Wy z@zVXcz5D@=PqN+0PY+*iOc@DVoTRq6*ZF*mnKL>WwaMW~w|c-KCUMCY(Qb zw@1lO^9XPA7Gz~|+0gMkEu?e$Wvs7vFA9guyWX1EM}zi*(E_CB&#E)dQJ*zy8_zzI z`=t9x!8(`b$xKO-UmY2@pOLOlAF~^8uhM6@Fz20^HK)@*1IB;(dQQ@$d=N*Ls!it| zm*!al02dt*KH{_h|ChY=3CQ2h4BZ58X0dp7w9aM#xryGw%}o7!kxO@r?8s3aIv=F` zX%aqvmXmU~hceexZcs`j;Zx$r5Ug|S~lgmVds90h{~%)nX;<@@uAVW7VDaT3(;ISUvz zMbs6)rz0pOPvHu1=Azu(4_KuAU?rI5$N^d%xhlFGC+6(tSyPtEn1^qEh@sxJ`Y@AE z=6yTL<*UwYnx?Koq{br6gjHR~pY*tzx@4%*O zWu{)vVmt!PY73sVuPV#)|1F+$RkSQvk57zf=T;O;sj3rgdKggiZ+7a6hd%PlTxNYc z-s-bduN>N39j9m8$+)lYs`u+NNn`<}pvkVH_d+=95XR(1T6^RV3w%`6^j6{BLHnw& zRhrpm)*s3;J4`x~u4Ax=V(D79OVIDVO39~i`nGCxq9~7I6k59BQkeRWv2z(5M=C2w zsd$vU$!K}5t%y}rNE|^_3sZHjHmI6jkh;n$-)ow21lmOR&?i_pll?(*OQQh=-~7%1 zph1+1OCt4C#+ytaA3_gB`|sdtN7E5ui+&G6kvJabB1fX9r_3^=IMn=H#%9MZamCT* z=kv_RQ$SdmSeflyf<$!<^bG`fu=0745p}5g!V#j{*U`sR%(YGz7>`c95M5q6yFA`f zIraPxWFb;|<+4Lq1!sAIWd7<@x7^1aJf@x#b^rTS47280z~;ym)F zy2h-X06@%IYT)zgY=&m4+?nBwMy?KrkyDf?9VhkHD>;^jtU7F`*XuAvd8r`|q}ii_ z5zvYI1@sp@dbTv=PdF+olfZaJLH0K==w2p}izm636nCWLq*i`No%&1z#GP7BmKFug zSw^V8Y6qfnLdSeY5Z$KtK;hun-O8iOc9o=^$_*G1M9&NlXOl_jaCb&DzW&#%OftY? zR;V6-7g7M6rfh(wbiNAk7KS0i6h&e<^WJ>o6(*#Cpy4KLs)Ww?i!WYnYWncC z7ULE)Os8h&ppfWCyQ8foK9CI0nTcM2J)Gk%sPsrWE18){P~)&%rJ`%97s&*?6A%a3 zBPnIg+PZ@)S((`Dz+e8H)wj}{%%RV9d&|LPeI(@DzvQqnXgkcT{+-@aC(PTsW^I`7 zeNGdenbdtEpixUcnU8Po=J>p!ccvy4vWFeMMXsQKK$%v^ZocbgfxwDo-|esFy>{%+ z?-4dg`#Kfl6CG)}R<>g#^J(P-r}9hJJr7TB+a_YKmu}7o%d8LedD!GXb9pc-NKD+z z^R(-#OzS&?_by^hr%y&Fb+&BucX>1YoE|39CItkCJ3B>2LiCDOK`>#=uwE^=pKvh~ zB08{80SM*ECV1p_Y#R+h}Aj+X1bA3ouly9EgFa%ir+HS~Oc@QXh zNh~uSU#?lBSqoY~6Umj@GenBm6-T~Ux>(Ry0J*SVh{&H#_~z}`z6(XKJ;r2}YZ`Ul z(dZvP_Ag{p>;8(dvnrXohl1kbtfoUVF-s1O5MHSSt}2lGI3{&PXK`~i=@UQF6QCh8 zUuU?Px?>-zsN?Cn;f~rL@HUc zm>peah0o1!+=fblg-(GB&cz^bYlpOcgI=KeJ-$b50H%T~euO-j~&>Ux=lc;;g5* zs<1XjW!_udtsjx$pOff@5hpCcrj@j;mSXF_&$ul5j%8gRQ=7G^(<;xhDsfzHPmtGw z!<4n|r)%j2;SU8>`8>1ob?!;uE8VdBUD26dOq5M{6 zgrkAAWs*SNJwpr}szAu?(tk&??)o_}?hLa+;A9_#CC6%Urp=h%IqC)8;l51Fwx-uGVZN0HR%;Loi(~WkKW#3zUnAKU z0SbI+PQ3z6fY2y8-eA8RnK!}4~+LB>A%;G6$o}G@}*jILjGjh z3=}?2zy1u$jFs8`bi3J%l^%{4OI^n&d!{2VU9`|URCPpH60==RO}9g!JjXx~V3BcJ z+3Ll+Hl`1ZsRh12$HmHGoEi|2KMB3i?R{(N8IGjx4^Mx5Cw6$rWQ#qg?x51~v;Pp= zgRTT9DhR9r!`bOD|o(F{PBs@u&#A{rST zO##>}a_uGS+rM&k8yth|veG?s+F!0GEQweO7L7KM#F0Y63Z;}p;caiS6xGyVO-ZQ> zo0!cR+Zd0&Rh){A#vG^Njl)+H~g6fBqBig*X5l=|<_nWZ8l`MTPxdWsgL zB=%aXXKyu%I7z)!OZ6b}owgDP{0MdBSX<%^pPAB?G)vnJmDgdCJlM}6hA*Zz@Eo=} zS+83vpS(9Kc-7ufcGLRZM6X_qFV(s3*$Lb%%ukgm(W1m>ih0Jpox@^{u+uo@s?{wA zu>E8h08dJNaR~e(@1}9;K-U|~3^;9uZ1hywWvBU!9VI5lL#|0x9^kT9H>M*SA{Zr?Tko+pAsVud;~PxlkD%}k=7<&13MXrImYv7^+4hk`7o6(JNH_P(WPYFN?W1~9btRIf`l93W>>od zk^Ey#uJ-*6HbVR4nyx>xfMj}6CRMGHVy}Yg@Edy-QOUffbCK?uq`QB;lAmVEUOtWT zlT(yygLB&3vto8EWsVUg^<`kT9VHQC&r#L`auV(}Wtm(ap5{~#(;cDu69SZTd`DY< zirNZy(q-c(iWcTR%*dtQt^p)0d3cyh7LO==66oe=rn=BkJ^g7XOh7kaX=*VCI0bRh zmG5m9n+4|mX}}i%8&Ggp$)Wk*Av*MU@YB`@O;p&tXD+cJdY?0@-jANU;0BS$gtDb? zR#1Goe$nDjW$DWO=-=zXnc6RZiLF-GY9^s?C_Ow0n?aQEqft8xWdr}-JhUZ*#eu?t z$U&zXr;%z~j8J$=Hvid}kxRuD4v?N6qR^2q9K+PuAF5<5ilVk`hBrX#(e;fouzgSf z0^qBMHp?JB)GmifNdDdVzRY%WWoB``D=z&lvi;~F)zFD8KM0zb#+<{45uT%E1e-Vg zs(X=V^yN*dq|OK4zYyq8h&*3l?lR-sF}b6o#^|G!)mY``rp1L3=uiKcnK(BC8JD%A zz$-vqg+hAHVVI^K;ItF@2{#2a78oT>f?CQ!ueTD(3!_X1Z?N_Df8?{eeo zhIpZKNk#|@Jf{an2>l{J=M;_{-j~&Z!wO)O^zx>dkT4@5Ywc>e9Qe^j1{kTUz*PhO zhFqBXJas5Y-R(HdxfvzoJ>LY~I=hbhv+;7o-kLq4r2E%Ybgd&1jhfh;#VoF`HKAs& zAUkl$RsnQg?vY1kx?em^=ZvELWb;cGlirNj;=XSAX<-7;OMIDQ<~sr{=1!1_s-Ne) z8Fh5|DLairYw0<=Y9}fhAlyJ0-+QGHVRW*p3blWPraY!%^W#{>FBpklNVCUY~f5m5#1`TxUcDgb(S_codulu3< zisysBtG>@nQ%iMRnHjU6jt}KlqCj>kiGFfeWp4I4qW7}fm&E2^hyS=emwX6)L2N8{w*Rm{hgW8MXIBMfvLo zb@P!S@ZkB32<4eNiE>`c`ZaSvX<+|1FT$2xpp6xNr)plbl_og>m@r;0A$0 z#iDnrThO4`)o|1y$}i{0CZ+CC*8dK_xtF6~v!WbP@T>{5OjyO?0YUIM!Okf3NKG!} zCbn|~s}SL4juIqtXMKZgr=qTo<&5xUuyCw4<~@;KIE%?2d=%y1z^Q+}ke$X)yDfus z?Z1O)+GqrUI&G3Ooc#wjJEDXq z$Brp_S~N4&`g6<%ye~*RacxTsQ7r#8crd~NT()d0pF!|>Z?>41W9e8uae7Si^Aycu z#ZPyP$zMMh99B{L;c{(#J^S)HTD;eM1va>dZw5^oA%0csZ|m2bg0oi24$Fu0UH8a= zYDehv^ZGFxy{$SCBvOH~*JOIbmx&GsQo8=SH$9E6KU-HhKah<6HH`EEQ_EWoZ|R-- zTq{Y{^w-Ddn@S?xhJNwKmqe7Q!ij3Tw1ztiYN|^&VUe8XZYQ9Drz<=}M6M}%nWekd zr7BIdYi#k#R3?<~^_)kpse)NFG3v8XnmE6HzTyriV&(cCw}l}b zHo<`I^7VN=!@XuM6(TpGAj;oXzk>z4oDh#yq8ng_-22z#~mne0iZ)!TKXNo3<^pWsUPoXky+K`r5YM2LZRl&m?0**>iHjEKj1VA z`z5=q_CG^vg4C%xE=)U=`E??VL$|$cXG;hTk-J{&tHf|&)J#wMD@*qKKq2eMggc9^ z0v;+UG4$-rP;3TPqy1`b40f0?8NjUW?e_yhK8`8;V1fTW|1JC3*bhpY-_}v73YLp= ze7cX)HNR>eBBaC)io8P)jL)B;0Derm+W-ET!n8}|EU3Lw=k=tCn05awqc-;SC=3|({vpDC@jX5pR40H&O}E0F zjKgPQ2BugJ5XmYfD$wQ`wcN79zy73yY>dPj>M^8yPMx{s7g+Ura`+!waz32t?C0aj zec5Y`7<_`%&rwXcQ-k|Z%6~20Ivv=g;v+P}Qa-$X;eswH*@QOG8$#BONGY{2{ZA=4 zy+;C8L2yC{6#kHJ2HJ^_j%~o(^91Mh3#dTGB`uxRwKJpFd8&;S)ySTtO?R6;6dSqc zutS^!dC#^zJHUy39|TQ{F~@M}nw1vE#~baJ${S%lz+cfOwqq9C`PNt6Q)Gke2WYSo2+b5mt#NOE>4iQ+U4Unw#JKQTfmV3i z;lrTCP;&+|OzZ9J(I|_Uu05mOHo*~XN1>DdB=6KUpJU_?K>C+rZiiSeD`uquvhyWS z-N0L3Y$t;NvGVepU<_y-evxrKev9D=tPHRW{Nnv~^(P07b~C6tQde8-=z)P`f!jGI zbHzeaJx(7sl;8T99-hobBM7y_2ADWASF;91U~uv*u~N#etG^)#^A2vGas+!*fzkrB zn%D`9d_u%P_=ayWa4uLtg(62k#cwqm{7SIvS!%&7fmtkY*Rt@$%Lh@9a60~v{XoQN zPd7bB&@%}uUBjr*3OZ{4Bqoy!=qWIbMU+0?`kB*IJrYnf$psud^xn0v7slpY{QF#p zCeZ3E4d%zE1!GArXue61T{Wfi*z-x2J=Aquq4M$VG0SJzGiFln=;~m(wywQoM4Dd! zhsI4uIj7wA_GjfF1n#-UQUMddy61Ihy)5*6sjwG&H2Ox0^+;_Ho8h^aDGkd{e>_lB zyzfsZS`YiHfz^lNhLwctn1}D|<~E--k$cK4Fcnnd7G~YV8~y_&PWXA|NFFCmBqO9; z#Nskfpw0pcKx|_BwdtPU%sXZmDPdlfH;IrpBg`9u~`ig986~m zS`Wy4+{N!RbV}+u-e_1w;fpgEeuH2tg?%7n@mNHSf-?S=C-gS}kHD1#`CJb}e7p9~hpO7oGi#tfnw1 zbE-?&UX)biokbkdVtrrp?rJ!KWp2^|yh#x+>rVeeJV^iCr4&lX@X?afv)R3ls?e{7 z+$y}(JLLKM<3h&jIaWTUrLi|eEPl<_?VHat>A2w}`{y0ehk;l?Jf)HqCH_A2v5gTu zPNadd>lX{a`)5rOSPcd4T#V_Z{zF?>-KZnAx-UUTgjGJkMorV_$PX$F}VcuG7HEQjpfv+3BDy0RI>p zM0o(p)!EyMC-I;OfN;?ZZ+mhk4awVO1gECr0cnz-K1Z8rp6Qg=sKR<`zzF^Tpl7gw z6Z&+Jl^gdx#AP4W3{2XyfQ3B6}(AyBwq_K6)k|9L-9R_ZUxX( zY&@ws8gzR3iyl=LDoaw1{_A~YAXh*gN+AXm+r)SuA}IQI%Q8$nwMm}r$B!S&=**`& zdBq#LtgUfgv>lQH+j~DL7~3z=vX>NZT6u(m*O7CI=_9&BBERL7?=i_a(Ev^jLR+-5 z5cm*Q#(S@PpT~zu=a}9GnT@4Iu0KqQ5iu`L@um^fyl{|N9E#aWkAs`35DH6>j&0+( zaqagGJm$|`s@h}HOVMtlrtKiGLwOa9G2TXKR2ULS7h-Ue^| zhAD^$IpFk~8U_;@%L1f+;W> zY>ko!4OXk#?w7S;O;}}|;pJ=|oQ89#@tWdmFc~^h1TNqIA)tqx|>5 zRT{;Ec095h@q(%+lF1tKiIiv;6{FxwdxP4%3U&kCOn3(sKv}`&N`50kb_x z#fSdf{nj)DPrrOsKlk{j4C)(QVM`YBE@g59olvOKTlfh$YkZJ^09-X%%h5w@4vjG% z%)mjzZQg>VZnRSdhYh7( zGz$i>%u;R`y^|?InXhAll2kh)uLjX?D>+cm8}0sLsf|GjeeLgu?Qw z`gbQ77N7{-oMDvqG=ISdk8CZv0LB2}M(Iv(g&$+LB(&6p>!M;ONmfAsd#I-|V9)mZ z?2U$^DuR2$szP@NtUU=I0mfpz+)Lj1c!-iw?f^Dm_QsOIJ=7N^A z-RbtyDZ^g?a#w+HqF25r_2yUBR>}gu(#c_Aj{j)kb_i>l%XYDAO(3R`rFZi9Hqv30 zBK82ma`b<>Bfagi2}&$@u7t?Nhv^3xjb}uR)HCHw@+c92;^Ls2B~hpFO^&Cw&P065 zKY7d=`4G$$d^^PbdR1vBIhQlZ#=t@Jw?Ya4nSmXRPI=lDpGX+GCv~+~QidR=Jyl^Z zH^W!#lxHZ`kg@J!x@Ki0FvZrYpLqr+C@Eyx+P?_~_oRJTrrAZ?Jo<c6>u>>orRoEfbf+F;Xg&n9gD#UC_?>W zCbrQHXG{yQ=`K>xYg*2dUjqx~2N**iPJm}Y0W{&zr!YA{@(!#=R0s$7(o?c_rl|OM zMP?B_v+ruG0fLTGM8{`v#NFz1{wYEb!+#kTUM{e|NI|m7N0W`vXB`(+**G-0D8K&h7G*WJ|hq*`@aS^yaC{EXh}u!aSOkudi;JqQ*@r3`%&Vt zH!RH2x=ZK2;!u18+zv(<5J1uS%s2N*BEow|k}-f#=x ze>Gtu?|+BV=JGqXh3WiCV&eEDSb)v(gOkL`D()dCfX0mc%@{60?XeaV0HB0F22PVv zus==8Vb(cbJ~xAkn^+N0$hH$c9ox`Rxo4peq%@~W3u5-Ud5!!yxuqOgfWZH7Oa04$ z0}!twyKMr?9bnT};6zl{W3kcqgD(T117dwmEDE7OQG!ay;zNf3jIB`!89{9OZVd{y zih#}4p~G3jN=P76sh(#e+h5_y)xup+SUKT?zcUevr%<^J)!s_%#{rViN+sy|al4B|hc&w}Xp=I$$`3v8X1T^4vM&w;y*iwGrhl*ol^^#{rm)Peq<&g1Ym%aaLQ@Hy?yM z)HV5zSXfZXhxw%p0S7TDsSiLhp2c-IQOG`GFe32#kERa(VZ^?q%)6$B-Ly=BnFfhB zK$y0wQBj5sw7oiyPfIb`|L-;Sts0aMNiqyS!R&f-QLxi(HMc(a`b>F86;;FXA zy}f&OebaUHz@b{TeZ+nT;#m;n)nV(U`)P$|U7`PCFD|F2w+~S6xlmV4mArlNIRrpr z-e2?*!Q~r3xdriP;O1$0ENkN@+_o>yk*)D3pcKQ-0L&_Z$w2R6E_2s(y7; zH6nV`JfXhpe})bz?8uzS+z7FS zn)>pVVL)l$hh_t=+;!r4=oY;nJn$8ukLHSge|6N_>n;+^Q~(SL_;^4kfMPuC1R(q3 z#S3EMKccc|*t3F&-#7s9S#a^!m$Y8LKlJT&WNH3WOWevMRE|>)LCF`L&3(T;qzx#eL4Kx? zB}n%f+25J92^dW@gUf0;z}R=&uyl_Lyl|JvPls&ti$m+SQBo9P>0^5FQ7_>=V97owO_3oz!_w%GP)-8 z31XwEdO9N~UM+vUgDog`{wQnoD@+I=!aQbwUYC1QCoE@!Gyoxb6c}+R1$qubgp>{# z=faB5uvV=u8X>S)0p%>sp)nhCR$%FmzmWwQZsZ#@XTsDKrh!iLH-_%1#`$Feof|9Hyb)$w8G#>W)pM7xTE zL?;ED%X~z^uAn+V)-fH={_|?xQ&t6~AIB+4=^#2rkj)?W$9=Q(LpubW;iF*vD=b%> zHvPJB6y1aGM*xqnElsuLSSybgZ3;St28SBq)~H(y`ro8~2T0Ll=c4XBjaC-Tc%kn}v^1>< z&}yL?mib>eK;6cjsE})dHv>dZS+76XV{&yoPDz@cEC70&h8-<6-?y9bRP2{)V{8Gx zT@T==bU?i#Sd!U^&2ZAtoZk;l#g?mURcuFHLj&I-R-z2rB*x8yfhYF3S(q$3) z9GZ-4XZ+ORW>$xD6A;#|iTPW$7AV+k3L7|>pYZM}+!G$302qhuf_@Q~{;%mbO!^&C zPuk{#KQcof!Htw&F%zTFgfTMYGA3sP-}no+i*H|rG<~ztljqBJ8ome=5tK+ru8W|Y ziLOEtY*%}8b!X0*TnvY~RbI;GM58x#MHZ%h(W_Rh2)2*xLWJ%~2i)Q&KYj6y55hmMb#9q^UjlF(H2Elaoq(62*Bx@Q>sI@?(~1qV z=BCxhqga@ujnM*{7sxe}TS|u8ZjMpmX2_R~XuFDIj;h$TizDuWcmu!_B`E~<$=E&# z6O8y#6Z{^_P%yVpR0WOBIh(}NJ{hSDjT^-oa<0;VhpMvVxllPxB#Mdtm8yF|8~`9% zWP#)b%3)I<$BU1TlW>#UlK~t^4WY51yKFgKR6J&Gk|@2jz7Th$-#i?QgL}SntZK&l zKgIZ!1ZydXh377|ysiP*3jp;%HBr(WIVbk(5XUc0<8qs{`{ZK^IKha}!0KcjfgTC< zo-XKI%u*foI!MNJoYX3iV_e!N5|UpVflE9i0fr|-VlF^cI6I{?^$xf9{3N8+Og)+r zO=<7Xtq~Kw&nI+-x~LpLr6DXwbPAxZl0QuG9qZn^*^8}23%r-et@JAk{e_WNp5ys% z>=7iQVn$m0#UDT@ja%!`D^>f;YGSua(qaUxr~pG4fnWsFM%@q2Y8>Ep{?MDwKL@L+ zd3op(-K)&a_D!=#A5BeDXNJo2(Tf@A9PcLJjOVC6v~P^op3}*yb%a?V%6D~YW#Yv7 zXXEb#$Ei22XJjhned?to=s@qbm&{stPoOV_8H z{K<}|tvUx}JCgcdw>c30+>D22b>h>coo@4U?;b7pPeA9%wQ`z}59b;Dd4nZn>3atk z!tO@Kgt5%3LnPDfY!bRwdTy&8FmW4Z<&sXWrfZ#D^(s^K3E1cwo4B8)ms5xG#!~jp zjUU!DTXR}9E-8F(sH@wt7!X)zyI#NrFgz*%U1Uz54b-jETS(UG9x^H8KZNYtvPzbL zxnoF6eySXQrV-1W*%+Mk(dv4+Va3ow+9#miXnIioSfgELFG`sMF()P%*PqxT0__k~Iq?)pVF22~VwPahqgg5% zRvQv!p3LN5zvhLSPj?i4fuc+5&%cLHvQkw~cBEg)@Cp;m(Faf5pXW1nplaWCY&mlS zY#IVS<33ElGO@I-JLL`gYgxr?c>md37Dm%rSaDGM;`Bx)C0o$+^(hzvi=GeKgF;&X zUOz=^18{*W&)MK_T0<^?#+3FrS;O7(#nH+CYBcQb@yS~#5^f&6MMRm0X0wj8z@!P z_>Q4p&RqwC?GLZ#2yv^^O9QlP7pDg!K6zKd;&27UkuU_Z{}dyIZPF0jGa$+PJo#?s z@=Ns|E#mpb7k+@WHGU=uMh>QP0peq zF=OeOl;2v<_Viq*n~T*deDSo#HvMPr0dmsy@`XvQ0mHw*-xzC&)^5+zHp;g6vJ*BE zBaaQi@}C1_zMwm$<+PZNx*&-8Kaw^+#wq~h7i^cA;0p9$@L-cNI%@Us++;!i3Z2Zp z7FiY#+VfB>1iCb0^P^A^>0ll3+DJ-;qN?|>de(CFI6w4AP-O(G6R8_Ax3als&~-cB zU+z_@()R9pzL8!*|-$4OaPWz<6eqD!sa2$nd`F-8mF?L9Z>pQ%rCouN@E zmwH8*`?nSVlwpDEpUtEv9`szEp9s80DCbQtZI&yoTd>)nlA)S}I4rAwmSwxBC^i46 zRBxpeFMqZuWDqMHZ7XZdoa$;8YkjuW-DQk(7y~ti4-X(32wG@L75i*lnv=QsyY$49 zbXCbj%OR~zwHS^D_oVd5A}m-D7S*HEv;C)=`BWj4LoXAwq7h7X_6AT40A&>9fS;9L zsGX!XlX}LaM+-t-08o|1h@`|!v$@{FVvf$%`<9|m9!I+5v#@DD-dRxl9w5nA9oNP! z#DZpzzDJho$&h&v$JU};PcUNqTaG+)aBhS5gxMNLt9@Qto+ZyTFTcP13ddU;G#?N6 zLFGR~_r=@>VUg&;Eot&^*b&Q0u84&>9Y@5s1-^-fBxSOu;%Nr9Nm)G9LaxLU*|`f1ept*Xp=nI*`!5HdEZm zxP(z$6zIG%E$7O1$2%14ftX)TqgEfQk83U`&2jD}~p zzO76ej{~2nWmNt&a(u@64GIW(q>U*_rwmEBc&3QunNso)XqI4pu7KUNq2*`jUC;* z;W7;oL{oZ)pysWxldK_2r!*G_U-*H)?ko`EgnLl#SxC+Z~+%$r@EKz3m2EJRs;peK!=P8|4ah z=HnB6kFU#5aUwKZc@y;(J=Zw(%xxUqf4+wD{Eic1w=6D$VoU6!HqeZ z7Ne@s$6UEI)xe6ugu9kLDwv_9+$Y181~wQyQI=<834pL9#IqBGo(-JJ#7dWWe*84K zspJ4eD9Z#cKysnDP3x7<(q;!s?--kMFK)1RGv9bm9{@V5RS=o&s&}exfffI>*UcWX zcJfo)020O3EVZZ58_#?j66M z0ARwh15SsnWnrHcZ{Urdhk%)chCJqT!G`11^>N8N<&r%bjPk=`(yD3hjNg+q^A8N# zk%-$3We2*yH|TGPYl!L<)S8ugiL+PAm8y^Vsg~yM)1r~D+Sn*XWn!+So)p$?ToC@W zFKnNE!G8B@{vwj_M~^#F7obTr3QQdT<~o;|OkUoo0!$2+&6VE4l!pdZ7+>heA*;@6 zHYx6ca@*%a<80Ly(!|2AQ-JM?!06B*8xzmO1`zU()3cDjEWny%%4rlD<$BQ-ezBgk z@>XJ2dzKgz6UxcQur$)FeAUsZ-}*N{uJ5vta>QavgQA*Kw`C+VAX4|v-_D40&=2o5u%H9i&?F%?vgyyUhAT8 zo=lf12tkt8ocX`qJ=z3dez^WbifT$kwDAm{aXbD>G%SZUOhVdHZ3)6~2Ox6#Y<=!~ z{zpaPqwxpVTv>l2fa6Y4U%R(&iuxNs!P6j1ZO=AJ=-BX&L}^8`%7e~!&R<6PS?ngZ z&11ktL+m^UItos^iT&Qivp#(HS39y#xvUTi$*}AFG~IYwbmj}+UdciW)}^S@RUic# zkGD`8g91<=_8o7V)q^A@sJmM`)r>C0y}#BI(7NUZ%H}Yw%9oNg_?=f?AAKo9&T1Cns!#A!N-l`N@vOEe@`^a9ie+pe)?l{ zLrZx85;So(`_L#IaA<5xAsZC0Ba1cgiFfU`gYSiq|b1`v8UQ0hUXU}&)5dKaN; z;}eP4Q~U=>d?|OL(2vaqiaTC#?$qxqG|P^Qo`uvmk^p7P1&QKfQLLMQmD7D#;-hr} zSFQ-38JaYz@{XcmJ^pl#OWk7jk8P&;#-U|8zGUprj1t;&`IfmXBrsS2{8k3oEApi~ zJVDP5vocPk|w;nDjI;{*y7q!mOm3j?#JKE{4O5m=o#S@$0~>jz*AF6TvSo z+c67KvGjVt5D!t#F<+TM#G6g~`r{CGHUY7ky2EkQz7ntSg@P4HRPLb6bg66p|G`_#L&}dA5j?vfB6rFXqySf>F0(jSv{6e>6Eta zY}Pib5~`C|wn@`S z+(qE^v{daCqV8@>M*PsUg~XGYYD=U4MaquXLf)9)w-R*m-~?dT{v? zG^+vQ(>W#pMLi%?$F@>IyQw=1TPL*j4Y*b29|>l53U)q`(D#r$qPuT5?)1IcS7ZW} zg)GGd;!l5@jh&8a7#_-?r$P$TrQy#;^7#IEQcyqHy1QwJz0x^Q&3^# z?5$at=P1`#G=*sSn>_s&35hy>qzJ zOGZIOAnZRM&!MKllcL|7Cb2{s2Vw>r7~?%VR-roO$yy864B@B`p1dQLes!-Fh(_E+ zBV(Negx>;-4d1z38%k1Fij$WQ$YjH!?c#p?8O+2l9la*`UR?7PZnzcToQ!{UV_0o! z?p}1c*%$uIuD$YHsJo!xk#iIz4c+aWiFcJ(=AIrph{%+lYjefriv}mUidkd?!(~LW zOo;}xdfJk1(uE}d8y7)U3WbcSe=|jmg^8X418#}r^iu~mTS$npWVY{tC2Kj|9P^W( zAd60nvE~)@w9z4=4N_Cli0rdB6_o~Hr6T(=T49BG^KxXpc}|MCqQFsncSHfRkmfVN*6bmC9rkVY#K6A?s0c+}NU z>5VgC(~tw6zXf%n&YGU!X6rQ4W(7`S^K?A8T!bb<;vl9??5JfatGg54s8tLCmV#<2 zsW(q!$r4FBv6kpC2w*@5DIOd!kYHO1mo72st8E(IM=S!TS=&@$3R99kwhDB^G6yh4 zCO|;FAI8_E(~ArC+51Rtvm@jrDcu%SAT9}A75(D$y({T(itm>H1S{B2phe&g$|GI{ z;Qp06tY`(T1ehyFv|;n;1njwpc0E{*nT}A{TFef{a&vcDe3Dg$VpB6l>~T zoF~A)$o_@J7A8Q>!CmIct(>-T6)9$5{Qe9IoJk1_^mj`@ zGWH#lEL|;!t-GWlpM^}_;3!qR-6B=2)5ZZIBEwKy+g)}6kT07cn!Z*`0=_8?u(aMA#~=?q@#XWQUp0+18n|&M z>3f0L;knim(&Iq2IQG-ra*ElZMmH>YRLQWIwYZ>#4^AcK_Q-iZVMp5Xw8N&Q3ua(R1B;L`BLT^^p42%*qYxaJR%b#Va2z`eNT_GBBZ ziPWkQMGK4_of;cF7P@G zg^L1Az_pT+D2U&+Q_eoRA}r9cjS!qMeCBMBs=7}y?+pvIBR4SFoo!L_A04&Uwc=Ac z`Ks(qvwwA8gzc;~eMW51~;N^xXf z3R=SW8m=v3UE1>7i>v0X8}ZVR4nFSia5S`%9$x(}?SrZbP{Y!1e;+W{Gq)LRLf*do z6x(;%B%N}=D6Fgt7yeM~-HXJR=}*H1&sUHMldCZiELT%3F9sweI3n*FEg4X-cq$9& zM)(YHoq3k7n!Zj?y?~13bUMIcJXT1pHb!(d@7-ue;obaKZ;mXjT4q3>(ExOTP`yWI zX%Kn_Tp;@>x%f~LsywUHqLK@L=Id-GCpuv)_JQdaaxb0pxEF>?%UZxIPXO)Y<>?)hi<( z`aLmoV{?Y-s#sNm>pZPW#+Ga5_noH_ByFvJ?RRm%1`#$zV`d?uk%6ICfDW6xo>MN6 zs6d+t3iJ<^<%>Uriw@G!=>%f{A=?ff8s-gEU0yw| z1A?0>snvw*C0dEwcuW_=yYPJJ=nZBT_Z-Eo6j&T)o$5QJMW1-NX5T4)7Xs}3f>olh zI$5%BbS?+}^aL(d{a{|$_ZFd`dE#|NGf3-S*cSDj@A=9BEm$Dg25YeP2K-M+Y*WCi zzCsN*;1!7bKn&#jIZXozshXOCP20@S3#*4tBjf)i*^GL{P~brz45TOuv@D%91X;~+ z);ce6M1gCYC;+G!=$@)V`k$f+BZ1c++P-3f_)tLmG+m1KOFmui|G?AivDHhk-vAiO=|-@OP37}Y`lhkUC_Aa#3J z3WBUl|LA8&-0>b8sgHR2-+hdliFW`v(YAjB<9s4Lr%kSbdcr?P?DZXN5fB?JLZo62 zbe%LzoM%6@`+|c1plnlAnDO#@_`hAgxP!fDynUOKI~VmTcaB)AUAWTx*h@kL9YVTY zX8*NIkAbmL`OO)>?siq^y13JRn z|G7H+&*+Q7`G2?^3Vhkw+3f&;hrOhqZWovEui*VhTAF~sb#P!nubXMMd=)R?_z$E< z@xbnLp%aZY~lNQ@OdpJCh&IrAWH_D*V$OA^zvNfu_;B-M#qWg|Vjp13U$n zYX6Ud3*4rw;eS~s@E+X$fA_*b*Nu+mgWLHzeCsFm@L~7`2txe-l}i`#`w#Bz1UdNy zFuKe4w`H=?+rz8=BLhG4cw5aG{Q(Y_UGab-&Io!Mooy}f?LWy(?$o0{Kiv`E`u|`= z!y91AAbG#%&!6wE1$31R|87bVo($>=NcHaj=jg#hR<-Ww>iQ2>>-p~NL}0q42dU)~ z*S7BxeGOOLeLzL&hGlBi7sWv-feSA;lCD!>^{-xkc-1aClJOUC#nY_o=xBwVI`LYR zQZbE2f=*MQ?!Q;YwX

lFOh;MO$0j00x+u67~zYA@@J3o-nj@J3}jjsSV|iiHuBI zeb`6@MUr3*QWt}lQY8cuC=3L7$t%&;CV;XE(x~N16906`-T6rvNFd@5^%FaH=kmPI zf^V^ph+YqbNV_sBNW2suS9c)B8?|wAkj5pLl0#mR1BK?rm@a!?PzM3wFBZ-9Xi7FG zdGn!i08S`94ZwRhc!WGVOFjG-Scr0gWMZ8kU-F3`@IUMc8yq`fG$Qei~|PIxJ@vE-^a4=*qi8g@82stejM5MB>afb^21LT zPyq)?OXF=_8QjJ~!!LcGNV4mW53P7{G%<~FAYgNafuUj-rWY@``<$DUu&Niu5v~Xi zv|Nq^PTWOt2(_L(e*77ReelP-6To9i{4+eqdT#qxDE<*b_Jh-&uC^e9ho>kCi5}Yc zB+~c%onNfOK?=Ljb9Ci@cORS_H16>=AUs^Ehz%Ghu=8o&%K3jR2jl+T$;pVx93Uly zhk~3)A#gL48`y;Fwy%MpnPm56mw@56y%CBQey0WS4>Dz-c!f9JMQk98d^LCf=l8pe zhsH1VQt+=99={k~BX`@MxDX$sdG_f7C~e`dEL?$#Q}3bGA@)PtALF7=O3N1uD>MiX zD20pYt_Pb!TH6V|__8CdE8-g4t{lEz>?Jik(p$oC{|3#`4GVHjn>XWvaPlmTP zMY@2{Jzc#x*36Mh^ZCNRMh9+Who2%B5#SSWus)Gr3pbM528(Wkd+7ZX5<)}E zV_Jl)R1w!DX6AdK;Q?3GU#SGAK}Ynd>ke5+5gjUp?SNKwd}Q}o100#l9Vjj$O^3*J zoN8%F$uHJJmw;wkl<@xj5y(}()dUO6iHW^IBnlV9&bJ@WbJ@&n5Fa^$)Rndm+=uY+ zGe{7w#{QxvP^iYEgN>U2S{$f>HcqC7ZkQ&BdO{Hz0AczqlBqTD3E+{#-vWQ1#bJ^Q z*f?EA_B+&|F#uXwy@Mg5N(LcxaXBdY)LlqWPHGraOyJZmIZwrI8s0Xz?br#j-dK#c z+}Cp0%tklntIu{B3Jr+hk*Gw4*1b7w%YbrQprCMPY*^NxBP1jQi*OJ0_FaO(I*hWV z>N4?n0FalOCIOhZ@(KQ#OfH#=uqKp0jvK8$1ucKr>0zNN!2`h<%RG9T0y%U!jn{1+ zEU!dXbu4dQ`p4EwJWf$rn0b9DULz(0)@2m1pwmD@0qIsj-itIaFTi#Ww^Aj+s5B#u zUO;#tjCl8h+#3+LQmwU%-2vZTafs;FahSJ+0|NsifpPK|P@EYAD5CYfAl%OAia_EJ zt)f%{Z39Nled#g-!yWl(DXv^jG#oD8U?*8M^I}1OPBRt+uO5&Iu-`Q55HnmHQ{=Ol z2}fZ1>6dVa5B*`o43c-$@gSWHd|}38U{)@OLoBpf`r{pbouf^TMo)og1!E|Gt_7Y+ z3v_3_3a`nW;V}EA6nO@aB|;y%+zXb*4kAYbSRAREc9lpL;BvZBlB#}7Vt@s(;U>FM z?^Jlx<3<}}kamETJ{sYM>wKzLujWJjji2s^>m9I=cYRv0TQOif0Z(zB27Pim+%QN? zoktrfu&5-9RQK3PQaa(c7jt2ZfKNb?qe!+M2umF}LK6#;H`@`R%xx4hJ`@7`36kRC zK4DJJ&-FcoRXIxHp0B)Zp+P^nt_Dv5MK=M5iv&Vc(B1{nNJs~e>{V(tOts5Ua`*;( z2`2H?$HP+iUv0l0A^|zD79y2fhYnW!okrw#0s^6NCtN6m(9|aeEcpJf)1%EmA(Xu9 zgBSzbMw|JZz&*H}3!}h^=gg(?ZJ@s-6fJzu;n)8eaG;Ojw|W+Ep&58?cvrCAP@q9A zj$fNm;l5I#d_abi+Q!W9T%7_FJknFpecmyryT%qw#GU^&&_oe$_*}9o=t;wE7qR_o zV_m%e4^TcbGB5rUxNd*o|LZ-L{tZw6d5%>7iKzaMfD>L3|DWyN2*OJM#2m_REd~B} zH>8!B+!fY_5edq67<4b#y1G?Zt&I7sL3&1v0d1eHS+< zH1aGS^zd~q;Zh_P4R8uSJ-IB*?~*hp!d4wVW+eY&MNwqF|zYxeYt-^y`z4 z*Wi`B8~4+VnG0vPUqqtgEts%rjsE3KZ89GeHi zLw;LXJ$#Y#yvpQM>LAbkjq3kjJ|iRg*ip+&!)a1@CwqnG%(@efpobeacLeAKjl?-Np zOTssZ<(~$)FuH6X{b57PKUxf6w$&q0eb!A`*qZIoW5O`dFHd?*JfZ=n)^%#82K6Q zoMd%Ed^deyd>D0p4u=Il=G_<_hfg@(N^C!i_{cBwkx6%)mBufDLMY}df#gd5hmFyb zQt5|NN6J<6ct{n}pxPnfgXf5X-wl_DwbUT7eD<`yz(P0Mt79>PfX){#>hc@l%)USE zd@bhyq(IH*`{{j~tJG}w!PBq9YmK7aKTxL~vFJ6O2D#7|BN@PawWJskK7ogQ6lFPnV$df+to||JviU6hYPtf% z6s}z6od`KTu{=Y|DMZk0=(`Y)Z%n?v?dp(RTetC2OAeRkRbf8gm=OAdhtRekvFbUT znXyW|d3PQr5yV-Z&}4*PUan2isYo-^c^SEGY8rewXBNi3iUBBRAaI2W{1^2546%V4 z$M2PEZNSTAo^X+}uJ%tTKTuBVH;>7n+jyYmWS*@+>td3qc*`sJk0!tL2HUVN5gv7#Q7@oL;ZkZ+r*rpB`QTmT-v zk15&(sR_tYHPF)*ZvZ~M-=|MHR^sg2t5e}YdQ$h2f%hLrghPC)6v}S}&gI^=ZB|sOq2eO%4j-V` zmk!BJBqJTXvKZ1hoI<^y!tvvJg}`X)C#lK%)${zOu)1Kc(2JZ|(W_~sG7nuA${tJ? zE;*RB#HIeN9&6+i`C6QZHh=H+s1sUK0F91^u*oM1rwAP8#-%1NiU)H$gRmz1M|we+ z&@ezT+M!eu{L>vNU%uogzI{Xa#mmywyRr7<t8&j( zNXz{u8N$J;mI1xRbqOB}*i!`We0x)&M=V+8%>5b?5PWof{NA=cfTC)CdqLsnb)7KZ z`BI86S}vps!@OrEzQdNCY}Z1^8U#eRgJ|$x$kcM(*IM+7mSzj?Z#AD;`NonfrKcg| zH|_iBdJ6JfhCyrmH>c|4o>~ps4u7f+z+Ubzc|yCf`n_O%=zGD|Y=Gna@n)I%H?X^4 zqWD8tISPC+(q9ne&p>}WAcM{Y!ek?TgBSaVnj-FyrAod$g!d!i5{HwUWM~r>l1elC z>mF;thIIv&$4GO&4x|)X-c|D&h`awO2qtN?K}&Dhvm*K(&OJnVlKjI>ciiSXOOgmf zc|Orobz9w|S{)x(C?uWcIX7tiz-gKuo%C05l;*VVrz z3up_rh+DQ*D_`d;`vvIQhdV6JNKa2~iVF|U=29SY@T@qGg&V$6J*U=f45{`GozQUi$gSNF7@~i}q3FgQwvlMFy>A6r6M)IvWWF#iW_zjPLRcyTOL- z+;!9Qz%~0;#dj0Uh zwY78elb%(7Bj&-A2(1W63vW6YxgwwH51g5qbJ86G+xg?6I%6xV6<~+;UCtLL^@Z)N)Oqg69ujL9c+w*-sY5y&73NY50 z8rZ(^WXhd8a%55?!+6MpgFuQ{A!EaG|_{Ss%I-W%0s_{Mv@f_HqG<7F9i;v#n;a63ArW?d6xh&{} znN&EBe{fn~aW~Ky?(o261g}%~oJ2wLcw+Ucv7f7;?3g! zTd2ETl~rKkc56~-x9!m?lFl`Kz(T$Dcdu;?)e2j3fU-!@V1rL`}av(_uz3uc;g(Ido8& z;eHUO>^jr@FSMI69KAtBS}pLuOj75c;f}x2YsW76N6xHLES+CKj(PBI*tOJZZWrWs z!Yy7J|0K)tS46?1V^R9znO>M>G4iuD*EP%0x!{m^Rfzg<;i@TJ13ey7;K5a zKDO-(C1~F9*i2l8O*3moE_KUoDco~IbEB^nLkCsFxIcl+}+1V2?)pRmYE?{U+PD)GqNk);yrMZx7eA7whTY=Hug+XrN(KY%F zeyn~zmAkQ><(kF%@;t?$hX3~hbLZde1JP8(v4LBd%gT~Aw>;Y2G-RK@nJ)Yy-r$)& zm&=Mvteh6vz912Od@3T6-Wcw+#dfZvF!x)IkQE)~LBtzXP7F-C^>G1pRmgx??#wks zi{Ixm?6v=JI+l1x#e^XIHX8Bs`50V9MTn4_hP~*4SjW_a#y#AS_d#7psl2RuwMBM4 z#WGP&B|peO6&t;mnPMwf6dOyxm7sNtp0;-W%EPX2X7?A3PK1x|Th8Bx77|?EPK;u$ z3^bH)LEu9_zxgHK-rFO-c8h3}>M6OH{HeUJPyWbUIT*`)O8m{aI3@|7&#o*vKbI1o zsycZZRLWJ4ymZP>((e&<{t<8J+9#Ls6x~8O;ahYsQqiez(iT+;@^6$FjiN zb(YUCS+Qd1TCYi25d2dg#+;V5v^{)ock0dF17~(L+Wo3jI493ft z6nzdHvN7c@?C2XyiCb$OCwp14RmlpAGpyg|{{0Z1N#m(BY@lyRb!IK(K-H$yDKqO1 zh4pjj_{P7wXjO8exN8pq7q{i|V$Nnxgfv^sb;l3bp=eUx9>WPc_U~WkmB-9x4LSr~ zx0XJjPYxSlDy`w8jW-^d308H>rH~pK{6_Xre%KoJ7sy+0@HoZ$oV+D(B|0Mb*6r=1 zDemf-d+6IxIeEr)&7}I`(we_gqeKFcKXdguW=dF+1`Gt9;^ObP#E{b&OOfjDgFCX% zcwUqHvr6cjbLeLm@xK1xw6O1U#J4Mc9GW)&zQk{bd&}oryJ_+>-EBWcu)W}qk*%d* z!A8mxjOY=vvqBbH9U5v2rvBk>9Q%qj0zvIQB3NYiaz?L;Gvf^SZ$)ZEud@OtE9E$a~!?63@%vVFvI zeBjO5gTIrcC&{@doG{<9^LdG?ZpDtz$S8XvK7H)hWB2s^y?r^`Fn5n9|pb)2^68PEW1>HWHE}Gzt@uC~g zq`O>al4X|HR(W!Gn@IK3_y!!X`iqnBwfKVY>Bn-K@<-8(Lx<&KFE(p2bWu$bA7#$&7%@K>+kD*&s&#Ar`MrMA|1x2 zb#Ya!FfC>|yPQZkh+nVxJ9&5n|ATCd;)0|g2G^eXtN+H|joX-luZGrry}AsRcOJVO zkz=qLI&n;PII`ZrZ2h)St@G9^8B0s`yX<1#3#A{C!)mcO=2~$=dqE6uAcE=f> zq6qxB$yQJjm?}!(GW6!85nZPetZWtB4v@Sx|VHr&)#k8(B z?vfI+>T>R>q9W|r=$nc$lz)O3=HmLyUGp|_=`%+#6lUw;j| z)yrK%eS$&Y(Q}$371tS-vStEj4J9ko;DRd~bxorZX>@BT4|N=Z6R}0bG|yLd+CE-h zcl&dkM!lziDB>@#ondZG-RYaAs$VDTfVtuq<12oT@CW8aeO(_{jCG(F!)upU-nU zd)d??A#|8F@I2${ri?7v^bLpJ52{Mj-T7A?mQ5E%`=>^tcW0p`h{K_Al#TJzichx> zbaQK4IhJnjbw$-T0_NUQy!}iL;|tYrvaM3S#C5d-7@?tD1?&C3hFO$f~6O5lt&Fc04teXq zL^6Vmn5h!D%wx1%T@wY=btY3Gsmc#*2{WP`9?s@+g&6wRYW3>`71+BqyCl!%F`YO; zMRHJW)v1wV0h?(oah-Z(stOa^E_B90w9JJ)J+Jr`(RtfRKCl|Q=1B74a~JeIU(zp! zwUS)-u;Z84I+jk@D%Tp_nZ87D=4!)fXOHP(Q}9lf22@FuW{3UVU%E~a6Ml<0cZ`mb z7@oZ%BgLa(w(_=bhp#7bFs&UVCa!x|Qk?ObS0`wOPw@N+s-#7tPqkcn-rh^>ed($i z9$AEv_e+Xls%aMzUbdsK@MK(dGRI-XE`#kCN;2^ArE318bsVK;^WT z7QAa+%*La|?94d(qyCe@uV0mx-W2Ufkc(rVoIOQO9+uiZ1ydHjIZ;v!_XTt3{*YRs zZLmzz>EFt9q(Hc&Wh(vtW{b;JEJ4)bUdTx9qMRM^meTtu-N%1arrupLin(GHIY~*l zNF-^^)){T7}ib#>_JpKyq@`75iNwdl)} zMhlkJs879~P*(4^x?lav5_6R_&f;emH6=A41YS_6TAXht5?@;!H?%z6?}Ghm)WR4> zCt#sqBk0^s^E1*q{`qEqzx3_3=41)e(y7>k`^vF2Pubc6k z70k`GYfFb;;3ng8kDg+eNb_!U^SZw3cq#)P_95;5H1cYSFp{-Xr`UOW1wuR@ELHU6i}ake?c`~NKeDwK zdRs-moifD0YrF=-L~j0JBtTyr{#F{N`uV7}|83HE>YY8Cy_X=kdyV=!#xb?Iqw$#f zMvWPcM%lSh+mv3@aM%1Cv%IZw?u@#U<_lQC-@eitHJduY`=n%(zSel4gEtnvy0PbGxcPjG?^D9p9556&et~2EWW?C1e(WUmt5U7ugJ&Q1J2vU1 zx8`w)es+HH=w;i6a#r>^qczHV_UeV3FwuPdlvM;?}ix zq4^C(5iKL`{s^+8nGdfZhZQBqMYWgc;dj>TPC}hmuh^GH+e&x?{Tw6P_VkuJd@hi- z*$8+&{EJfX6gAC-b+uVJyFa(PE1!nSXMDH#MIFWjGbubC)3>3#JL}JVyEK-jzxS5n z$M?=o#@7_igiq_4nwyK>ojKLV!4}0}x9}lit{r=st&o^td2-b-Zcog`~fg4>*`>`OQksTM@pFh&VS%OhgSHPwC)NwC>|al^nBA znSO;a<($d;D`!_CIeG<}FlEXm<6w#|-O$h}*KTF>BL>G#MUog4QEwpnbs&%<`a#H& z>+ne(`!Uw6?Cftj(PGQ%%kLcO1v8Tm^)q8`^$oehK=s{N7o?vq1uM>6E6jSK^-o&; z`3=|k@P!8Hn(if%N<-!YGEd~LKvURXrpf0VV)HuvztII7nDtN?V`jiA^_Z!-`gi2^jMFIJX#Myf zyE8Ouv8NANh+J)t*wTZ)m2aYOKv-*9jNbu*mZO%KQ^r1qX{zcu7N;4czZ65@Gcfq& zCipg)?27`-&8@WfZ57Ohz3ADdd|4sCra+9WQki5j_5oGYCz($*7K zlX?I4?E$^?^z;%#YuA#ojes|2OTS>%Q*x$=UhNHPx&&hmzVz&6!?==(;#+rIaUYM# z-)^i8VL7hpy+B_-a^3S)#^9iGF!@m-r*m2h-4*tq%WO9s9q+}j>@Auu>^YiVsm|Ws zIxutSX0GR}@}buPoaKMg;M}i?{vWR10xHVyiy9t_4v{Va15hNDE+s@!Qt1v+>68v> zB^3olLQ3h7W(WZZK|s1=Ktj4}i1$4B`+x7Z-dW3~3z?Z`?sM-s`|Q2XnVpSwrNOF| z_wV~$-mT=M)%Y#& z^x!f}U4ev-<4dqS)bxw+FfV?$Lblt4a9;dA?j`{eirT zSd~qknDtU}?qB1-@#Y1mQV!|ky#bD4Bs$X)y{t{g+&_)vmMe8oWl}Ohkhqec)oC8^ znH44T(Dm`Hs)~I@&25;wK6Sg#=u@c`L&D=M3wq))x@9%mAoo+Z{#FlarDQfYUUt+n z_6RNf7Gb#cn!2sMUvn2NJWh!ksm!3Nl8lYA=wI)An5#uH%#5)g0(dao2agH zrlsZ8avZaQI->jBQ!?IZl}`lhsPm^sIdck>x5fQ%@bl@ny-EJrhR~l?WquyuIY@A%F0iomJQAALmtn>Rkx!` z;c#nfbW*3t7dmr|3yl1XTG$<9$PIH0`~wLUgq^&0!C?lL_G2Z~SE;#$EAW@3B9`?W zhI`6|MMXs`@c(zo@%lm8Cu#9mzmfqU8LB8tzUquPj^W_jb1wFML`mP)=2J}w833G% zUAdBb8pW_T^Wl6WY++eS9nZ=jExm}O#K7%4*Z5t2GuQuq=n-fwxpIE zADJt_It}o9*LZm{b}3R*$K0(knA)hl&E;I1y1IrGW|{MJam}Ky4h=l6ig1TKkSC+B zp@EiIdICX>BDNeom5LPGB~01EX*ugp$O@l4V!zg(^CvZBqzXK8&|uGJaSaWFZu4pTdr(!?PhFcj$U(<&wpm@{BU#0 zRcxQEItKj)q?al~irM3CAW)0bx6xti@HJbZml=3yJRpHg$J-3iC!7Z0k{(w&$ zuyG0o7kK;w`2Uqg?Cf#-JL1R*7<^(g_E<;H`RM2APs2&-{ojIp<4slZ_qX1DUa#;Z zl@idX(ZA4~`mg*tIj&BR8O^ed!_-3#mlF(u0^(I}XL2dM1eo3u=9uw}Wo@UDwsNrj@EXR8O^ zZuKcEZ;tUUf!3@A6@9(&nI#q^3Y|Q#j=?_pNy!ZWEm1!2U*T(3O*Xu^NnvCH_58rv z5vz=x>}k}63mQgHww$a>aC*1OcvsS&%q6vA3h}F})x+C~Kc;)PUgV7T30NLP5vu>P zeOW1*<)G5)f=2eb=zcsja(4He2Zuv#q?cXm~EsRrKp`PA8 zpR26t(d=ozdls_=*P`y%;Zr?n^C;_x?&jIpQ=jJ!cBp1sB^)X6Uu8k`3Yz;MgmIXX zuLNvHM?7S#gCtlT)ub57(hQjL8zJXdsj;z zUT+Be!(WFDWElsx!M`8q1TEc;T4{sf-s;Qa422@2Lg`}9KFbYX6vOT6Td>Vn_Iqw& zle{1JmtW3X1K+K6I9I%CdfzIxaCt-Im#b2kUjW^+h2fG+-v8wWvccR@Tz`5J=TfIf ze5?Tx&W?qPtWok1X-@-0#`USL4`MNb^>6BWRxEY();t4&SJx1~_X(9VEM1P<{!F{5 z3J#%1nGDHzE#p+JApEl2+g3fKU~#GWTUGOsxZAu)8^f|u@@Go?;mIMCrPP>}5Fehj zH}O3!^Rl-{k4ZbO6j;21wGbpwuo#iG5#%&Bim_CBak!=oGH2Y)YM9BnR)?KSzlV8w z2wa;C%apgr!K(%>^5k|zYCq!zgI*jqBMN?W_4(HoEqhSHiE49 zCosSq1m44!rV;>G+2dg|E+cwM> zoOUPXtUk?_-W@dXt!ZdoTfFiVeYyp*fnGc&f#7qVHhQm$0%ODthZiY*t5Qp@M9Bwi zt7SaCcTe}~!%spOvaD8GTy^4|kkB*lAa+AdUhmDV&*f198Mk?wOTjuXrt49Flz*1>V)ue0Bl{Ra-5R$%}jroc8Y{bH_i-MrB|b$J@;K51STi8ce#-!NSj#A zkzRs=M!+b})(Z&J;P@KN0cUA6aB%rv8(1lxKE0%81%@|2=b2jma^F3ZyfdsSaeio2 zQ|)iyr7vRFMQ~aT+=!9IPckX4paQZt#JA@1p+41EW6jq_#x0WgGU?(z+Enw|0J~b&RJE1Iv}_miS&4f?50|*U zHIrWCir3?yWBKw|>+|~p623iT@WrU~^+BX^i4x1<)&&3VTCq=SP2W^VHywjeO08r~ z_e4?rx%i>1d3upl#;Sn1Fu^x3HmO$Jm4lwRU#dGb=3)`AEbJn^PKUO>Rr=JffOl>* zi@MKdED%b1o`aWIUIC%z&NetQ^Y#cA3RM&5jmsU|Eo%ShprcCNPHs+z&Yi{7?XaW^8t>hJg<%eJS?gb4bl=XEb76Gq<23HMM)(<0}HdNh= z2j2#00vnS)+K3qrY`XL!eA>K)Gs!5ucb9=jO!FE92^Lr41SQRv*Wb%CX&!b|yI&KR z?F}dHDzSOqnQ#7NnxANshPe{z;n&p`Qx2k@J0zWzVQ*UX^7(l04D0}>TvtOo6o6!8D>D`zLZKj!} zt1+LyI!x7U^p-mA?v9soO8BH#BuVSPq9Pr8>V`ccl}<}X=g3ZP^<^}7=f#}$rZKNp zEhr9ReULlPGb_%|x_hs{BlG5;(Hy3rieZrh{jB;n9UrXTg=TcUWrOj&2M+1Q&_Bfs z45LJ_?C>x@RimFLHzsSxTn4Jkh`T&bnG!lz$NW81Wj6)KD8{^PAA2)B?Z0X7Qsy*r zn4Ft2nouCx^@30^5g5VcTNpz>Sam72kN!W@*xAKq=U8D&ISB{VwA(q7r=NSTTg?4k zlr8m>-iEjBrh4HJ^~7I)y^dxw%7HM9Auw31Z~x1v@^!@La6;>{ytS;Gv~|K`y1G;C z*B0k!1(y1u*3EZ%^zG;ROkM4RgjY6>@g>S-<&!D#*I>^)vAYPo3aly5HIK@{znv=G zZ=%1+_E<~L^$oQAIBThkWi?g3rv2J(OUDmlEZm2>yD>nyT|y{|i&WS5D~*(Uff4}9 zmY&$P)E_En?dw=j_38?_0Qg0+zFEyDjpJ`4e`siRN=%dVjJBX04<#nK za7b_M^6Si*yBN44EBjxcl7Xy~)hODX@x`HaEw!5n#Th?JZ$N~iiO{1$8?I-{jTXEhtNAQgunZTK+`Xm znPX1P+>8-@L5ble_(h#2V%xZFtR}N0iv}pJ?4p?d_HA4d0U*S1Yt^jakgz)nZBD7c zMgp6*`|PRwS0;f7K%LQRJmth3@)?;H$~sw0QYBE-Nj=R2{*~$VgUA?f0)GB}-a2<0 zF|qyOgB`$Ceyx(nzAN7Bjh|m$qyeP5xzk#v&^TP&a}<&$iILY5h5}s*I2+IOIz1zY zr}>Z$%aaL#wYk1O($PO``JNL(aZ8H@>IY-zcG0IHHus>cj}ul{B@D!+KQRJTzP84x zH(B>fS`)`0D=>yXye_M2NtAr)vj2D*YW4e}e%bhgYX<_1)E7db)kE9FLFdF2~WY`p^ZapKS4o4bcQ&`bu! z$%}Yt;{KTEZ|DB2JAm@2g)+X6DS~VaiMWjcXLZEC5;d_4%-~bRyqlcm(D;qk;0E^M zu4C1c1ICK-*Ll;S5m6Xy7~IpvxWV-7%0OTvg>)QGftaFI8E(PNb!@Ip*%a5ZAo3Vg zxyfF)vo7)&%#1(O#4a3#R97{u2x|~&SKW+J{TI)nR~-62&y!T5rfQpV`MN{5`#jsH zS}_ye$LY064kZQ-hg;t{7hB51WXaK&m9NRR@6h$|cLi;KqT5x_i-6Z$Wm@!4I*iv; zJJCmxLKFQg$>)*D4r&P9q4d!^AM2M=N+z0%PWdEf8uKDXRavcn0=6M)lTxJ-2Uob_ zevUbCYeKFSD4<1L_Id^L?Up`;TLb!VVulj5&Z6G79EbjBzgk&^148rWLO(U&r1sPn zb+<uT&cYh)Ga?j&(vF##V$4 zid1r`Xb9N0*xenlzoEVFl`h^?_qMUFOJQ%vY}ey+ zlaE+j5+lHe-zmHvGQ_w8UnA3v*uF>Yb>?KD3zB1^_pr7571C!&lSCQUdO2TRdNh1B}YO;jN^>=h1bwfl0woQLlN_5u|5Y z$aJ?kou;NpkX}4!FbgW-kA{~}QToNN^|iKzf5o`7>#WKrVEk0RIw1sj=))t=rPG-< zbC$V3&Y_fWhIdS#npJyhlmgtiB8~~Ays1k=!fO5_R&X5t?0voh8LC)GaNbeKw(-a| zpSSd);%;~SD6>V~ZT_EFn>(JZ>&5L85{e)Gu+^{L*&d%~lDJosT?}?z3XL9Vr~2y1 zJekwKNTQAYM1F~A-}t00={Or!5uNe&#MA(x$TZ}>*NjDO`;#rhd7mM*a~LQt{quhV zF3~f4W&<1IZIn6-iyb0IU1ql}YOBu#UT+ZjSg8H$wHrs9S765|XIa<*t0cPPfk>dJ z?@*PwnDA zxcoIGE~pbY|HSl3L!Gd@Hi{7~voNL2aCk0A=?~P!nEV(X3RnxH7{cao$kmxV*1HSX z6RV7qRHw=tgwR+ORR2fvNL_XJgKQ?V0$VWv#fv%^k9JDFUd5L|0m4OOTv1DEovk#; zRj+3TE3RqX^2mjhv1xYz*^O}m1_laG>VR0{`t{t056?@uKW|tF zKO#t{^i+H1{?*gxYm^>l)N_2a( zWObMU2hH6pdpaJZE@gwxgyY3Q5;qu9-(fIDd@|==NIy0d6cc+G$eczr=9l^+z>gRf zCX(TDQ=?@0?auD~bkKy znWW9GvZCvrS|=7tcyfJE9O`aIoHFabDIkQmOc2m3U^y0hsZvo<`y3R3Ay%(LKU}ID zN3Y!M+yn`|Zh>WI%*4(;s;=oWL~Vw>>)ojYyiON~+!#sX3#$k;Ha}PkbYxXO8Lmi- z?_p%(dLunpGkQ>YvRKJxWh!5#Yp0`0(*naH@0eW`wJ0ljmKU!Gn`zhf6$9{^W`1=S z@XzlBMccU7PP>hy7kZjfo7c1_N=EGN+1>Nn(UTJl_zx_p+#r%=)I;CnN$)L2cumym z%)Crpy+MM5aK9fAMnV7`!x~XdT@ScvU?Zm!@I&H`X)$$4mL) zjv#J=YzXdR|cnooxB|h6u`>M+GHc3(vNAeJP!0+r76oNGLAyqd1`F zzVEAQKH$X|9GYG+eXlHyI4d5n^Yt)iD?ZXh=>MT$(d^Qz5fPH7P(VL(s$W>Vo)Ehw z1k(A1z(*_dO(5oL-*wSU?~Swscoua&_vP9wz#R2b6NY)KxfzAEvTV>|*g5iwU6~+~ zKqZax`TV^z`8k*jlD^D{uqx67VXNNNdGzpmINUMWsDyZd78wSmVtBIX$GxYaJ8w#` z?d~TD<=c4iLJsv^#)<2-JJb99rxFx3VO01<>!sxAAiY1MSA!Y*8KVwN?+JSlf@jXo zCb4gr^w7Bu*XKXGm53j`lL*FW0*fP5JancBg_Oo;p5EG3$dN|&%xF+25JxUK1WF@&B51#KdiIY*Xi9h z^O`zk1gCn&-?5&;d)WIZS~o}Tr&{;?5BPvC)4bGWTYt_mBel26)5m$;7(u6vev_Z0 z%V3sxusUM;beCj#_oW7@AY1kaj4!SC?@6?(RhHGPVDS;!?(M2}=>LE{NABJeyuUPG2uiz5$-)qSxAA9=e`FTqA`ogFi5tb>{Y4t_Lf&n{0?SEIC7d>-v zvB4Sjb}m}JQC~YyhFgXvN^NcoKGnu4nH7zlZ9q_(fBUxnT3u&nV1gNmZwPM&=lB`; zQ?4PU{%Y_?&GH$&C|f?Ki@k!5%A>!v;&KP;J_&IS_eJKdb1yIFO>qM8Ttw!zPe!jWE86okYM_;(o zcQxoQ7KH*GkYV)Q3u!LrJK*R51+orU#>*n+6)B&P1dmt<; z6cOM#Vc|ZeAn}gBOmG%X?H%=PvecEamyjA6sup_cfD1aBwDgUj;59)*D0<#ep8*i+ z%~!Gv72?sZd%trWru~MmRO)P@bJcO| z8p8P9*y)T`)2UvuU#cLvXwB(7YT%+}Q84hhR!LjTC2?=gw8dycP?&FnAC5#-!;oAL z6$2e&{a}XcL@`6u374Q+Y5{f%sl}KU4*l`aV?b`9SiaX9Rls zg{C{I{7>edazXJMt5-uDK{)=T7x(E{`70$t7Ckv7PUItTTTNZR_BF)2KRi%q zC0$x+=t|5!QV;=kqv5%@TW}L~=ddso{)F@f4jVdq9kD>(wNtwbWJV*YS2H_){vwnQ+d+qX*GVPT5=}b45G7*fmGk)Ba z=)7_4roEI+IaLjRx_F;XZerit5m-CmIS#tqI%ivsF6@$+S^HGr1Vu}U#zqmMJ}`>c zcQz%BzPX=a7U`7ZA z!DPKv;7$YmUTF`)muG^Cf!sjy97e~b{;UoCrz2Bi71%y17E^OQKU~tg>qAa1W@})) z{2w=n0sCj6cjk)Ktc)kB;0L@ z{xckAWkvE#H&xuH^9v8V(qIl1Fi-@7{bsp24t!_75=$ooo)`87e_AINoMk{PU$4{v zI1e8Xq?}ryE87+(FL|vmXJ|zb<^k3$I7sV%c)BTY3)Jze9#j-U+AiX(?6;94Yqa+u z?<4U}W^b_zs0OQJ5B}nR+-#3}Nz6gFvgeYRLds93661iW^eTFX1uFxxwwF*G{gol$ zk&?ZG5tb8du>MP%XgnS3{3c=Td6xB8a1`6dr8^+|zApW>`uwUn>4>)7xb}u9JQaB) z&t#x;3f*ZxKJGG0IrkI&V509esQ`JgM$V&8!#IJ4D&Lq0Q0W^ zzXLc~NARl2NgbnXZFyjf+W(z{v8g~yy#!Xx|2gX6#nT|7$Skyg>iImP={t}fpa9tx zY`$gpAVS&XDpKUgvA(|~{G1Zj34b9t{FNU(H-BwPS?`hGLDV_c-cW?eEovw`L~6&5 zW{R133W>4KpahV`IF38cT$JbLKEZOInRw5lqebZQR?rGITPbV`d|mg@iR`gtsUG(1?@_#FH z$y{>YNkliN6SK&zeek7ke22!a^j%8vh2qO3*nXFixzi|5@NiRskr9<%xt(=lF4g;C z)1UXrV+I_gh%Hn?(BA^MLKbjSZO*aSI37YY-d&6u1wQC?U(|cgA6FRSpwh79F|xEh zUt%p!rFJ7%H6t?N%`0$#B-`f0g22nj4&QXa>@|R$p~>|yOfY}$|D z+TwVL!3n_Q&{#yJm6)VzTTiBR;89h;$6Nd7l+3&bsI{lX(%KAE&eZPwoVfD05-cis zswD?@gsbnQSeHhc!u|{BDJqA#^`(;Y^|h!~AaQ_P@T$@upZf+H5Bwuyecr8S!P5J8 z84#XLA^c@#^#Vi$+!kvF*8~g93^tS~y^nupJy> zi29Yr|!4cd|THx@|s$;mIeWS(Z!qmKf7=oX9e?t0Ael%ryDSjq>x`4oWOHE z(Oo5q_@{pz8oa)a8x#~!*Vao(&K6p@zteqQUBYF~B2GAqX=cpE-#MiTG-&Pio(wXu zh?A*gsbpE?9DOO40a3Z8;HLoGGfuY^%a!U5u-~2qH`D2|o5?@nB#H2IxZiuXv|te)O-(QZHa83a|GJ|ebB$94jxKM=9rPLxcYu; zt{Vu5gS`HQwH4zW9{Z?5qE-4E4HFVDKjaa~c-^bFqC50~r1o_;$#7Tdj( zzl9I^u4)tuIVy&LKA!!dvj}S;!x&b7x-KvYzIAhHYR=?$?&!^D=TXdTJ3>JsY>)!_ zjsym}?^r9bl@_~pP*CCL)i*{IR$Tn}m;fAAK;yLS)6b@DBEII>%1TJKz^&^z(HJ9V zVO|6r)8UEDO%S8S4*%Rhdo5&UT137nXM!Sth0VPt#W@YWK?`jeF{WRN-`mR%l~ed= z(%f&k%e)ZVWn}N(#>|-af79!oV^Mv4^o2|$J2umR8$WP+MTDil)-%rbueSL0JV9>V zr5oMexIM1VinVpoFNnFgxNM!Kz5iP@5xxightN!MKJ`}3!zx7ph-Yrc;|Muug`h{T zQpxG>7f5ly&%T4PU+6H3%B1p_9CPuO2n&PdA`oMfP5n9^%vzm~PWKY4(|&2MH9BSF z2R3UFFt5axV}3)4G4vL)7-&9_qU+!ko80~LIE3;mu(;>-C21%o=DwU5x{7J1Z1buB zGa5k!0VwmENXM9GBF6adRiI_-^pbA(m^>n*;c2;Yr2?8*P@5^QKM+RJ^_@dyX;eP5 z^Hx2s#s%N}=dx4^Ks=!r|3*h7#G?mkU|9L2RXOKuR;zbcwUz)1A>m}mS>UY z?Y#0ZWs_$s+z~kiBfd-D0XD2Bh?`u9^!h#D5(lA+va+EK%|Z zYy6a}KLv~n(i(Id|9IdVFKnIENlBwH>v>xWxmU6mSI*Yd>cO{VzxaPVPa+kg;&y+rth#+uu>3Q31O(JO_oEq;W?$&&Qh2|%%$1B1RN5dg7H`(s9PEj?OCVM4j z?5YZ;3PCTUj>wizR)2~YN~aM=OhnWB%t-@4n4E)sfvW`pfELUK)Q=NLI$r_KX_D%O zEU12~A0bwQjELuk5I_OC#8zOEG$|nlD`J4_uK?W+Dm|X(8cB3wR#=Z(RqJubX>y+hKe|@Zy1d) z6i2TJKJK}dL$FHW{c+I87ApKUaZ?$qc9v5{YEEG(I}i^w-S7b$x9LZOgb)UiUb%D$ z4g3UlHL6@5g@V%({;0RY?+lK(Rw-XpVq^qDnY;Iu@uyY1SlceeK zW6dM0(NY_Odz@m;&#Z(sG&IV}K9Kg_vi!trA-&Mo*O&aIaeXT)&GQz~@Nie_pu1(R zK}|S$({UIK3^-WXC!QOt5(OvI8R;_T*@Ot$+EHYaL`1P_6F-Z7uQ6eOOY-`3F&ud$ zfch)YJY8oV0|l&30O(=-UYFi^?E`Ss=Zia0|09YWLk@A2I-1X-{DT6Fp$7#VmyN*K zLX1h4Pm{Vk2DlFlt;O^ucxtZFzD>+PPAg!G!1ln7=YES#1Fz2-+?$}q zBWhJIhyIVc-SRr=w=+YA?UE8lGI+S4m9N!pgdoi_(8(5%@8ouu^&%0$`Eb9rW^?eW zOynvDhs(|!alZjL1FSYs{r)Q&tnZBh>Ohc5g6EFq@Cj9Iz*Gev&J`+3W9}(S_MTfJp95{HpDRt9KIhFR;7a`#r8!H(!bZrD}DR zhZcb;^PfZ>kN*Dnn0lTQcYoX7*Ifx8r{X?y)Oy)J*iTmM?CB=V@=MNlrL}msTt3Ks zWa{@gm=*rP1rl7d*D5NIg_O*Uo;mHW}fey>zCc zHivCRL%C@x(kDMH-0oX8e3r*m1sQOHWl!Zs7g<~J_2np7FGvG5U}Xq-#1u?Zc&s#m zh5T$hhVPUMLW0wCK42O5|31ZXwlhhNuiGq?j=NG{z8jkFUY;tc zK=6sSwJu0l4wQ8+7AZ*{Qaoo$)gJlszn#dxHGulw#N_QS?2v#%rXpv?HDd8#OHEqy z1=zupZ`=Lz9Gew)t=3BKtB63Rm&&L3E!j*B_W#(}1sacz&&MM+7Ue;kaQXqHUvsMg zLkqy|A0R>?_X2=EXa;06y#9WtJcE0IQULwk_6W-)vmo)tSrrfW>C!mIjVZtmt*tQx zJGnsUGoB7+MEpE+;)CV2Xf_FT98k_^9TQ|Dzk({}qV4^~&)I{X2>m@J|QH zzX9tImym6G&-}VHRv2R)aJD9WSL)yvl$y+t6+IeJ`7*RgwFWS^em5@nv&kr;!ZKE%IP_j@6Z+76hygs{TL4|@z)&lc6PHqZw(!*FO_y zJC6GLf{f0KV#>X5 zYd#Li!_8r-EaKxm1zQ^gQL7g1-ia^2`c+$VTsPWSIR!2HZv2YnZ(W|g8%ooeS|Z?7 zKEpr{ZC>m=h7@C7Q@yO>!BJq<0EeMvZVR1&5rcq@!X3~+!FXj0U#f+GwaOq#!$;rC zGO$m>kyeHM83lgS;Tg3+vMj_6WnHU^WE*h$>Qvrsmel%eFpg(a*u8~);x!JfA4930 zbkI<0Ch2VP(nJ%QPF=)z6VPq#H=lhDS{>3Yo`utD7DGQTEc4i@g7=>;#?!N*Us*Ki zgpO8&CZQMYmPFnYUOJLF{y6@;vy}W2GO)4i?jq^ zw(mlF)AcX>e{z+X=L<6MJYBK}GhKS@){OXpsv`_E&jo8LSR={7Miw0An%p@~+4$a; z{Dgh^KJxYHWgtw$0z;uuW=uCe7&0D$X8|qw2YrNas7WK)k^8-X6Sn#@M@fKZ0f$~k zgXhw7bIGp%$yJocK^0;%Ak|)eOA};f<<#-{j=~VzlzdYoFb=oei*m+u4XE{X2(0LB zYM>E`wO^HmH0-%LJ?hW~;~-1zPw*W_db#Rr=MzZrh|}F8e?z)6n0F>q=n>X&(Vf`7 zx_{`_N|=b@3O;clX{Ou{c3MCb1`7X~CW1x2Al5EjHAJ4LNwxL;mPj6lb~HlP@6O+T@NN$kyQ=?_VuS%N2kd6$$10ZNp%66af5q ziO=|3Z)8DDUPS&9$ba>iJO^dKiEz4MA4mc9ZmBz5WWL3D2U?iIEXJXO3J3A1rU;ie zL-7C*d7vZv&|RVKcg?X^y^+<6)C6(2qPhJW0q(OMpZwCB@v?UGfhGgl=(5Vaielf{ z9=Gh|%*zuRX z2>n|>5kEtpbRBRQk82iebqF_|-+#Yvtdp*(lULu+;zsnsri$5qiz=bJQB!nr zFRgfBmyJ5m9~6^;cuDC7%9FJW48Ndr-4ki}b5bL&M;cI(qvhhVDOXeNsqDI>0U_{0 z-7OSC$b);KK~D5h>@6`N7leTbv{GOyZdQKlL17A5UG#<=&hn_ z@BXH3`hz%r|Msrs>eTjc>#$mbE*ZQNVSXS;tgUo>AD#f|} zPtp#m!!%+pJa7AR9UO1|M|FHdQ~DENz+O91EjH66q}m~RPT5^-0jIKM=8 zKQnYA!V0bZ9g82RxJC;wOykqL%Rc$ZpvNU6^eWNzP zyajL#^w)$UWh$D8lu#CY9I0uRwV=+0U?w23lfiW%hYj*9Tqo?URms`O(}8}XpF0%C zthYswAl%cxz);2U?^MJq0y)gkED{PpopX%MyaPBk?d-$AE%+Hxm^#DtrMadI4s3bL z!69M>Q;%syyRgC7S|_@RUr#vaouXHi4ph}5#LJ!QfE$Ei7F_DuV|{uG*<|lO+8h}z zO3yAb)xK)(WS}~~O`nvpmMVrW400C*4>NkA7qHC;CP~9Z;SNH=uy91y&vUr5nSG zM!|mZ)J`8>+}slgx`DE7h5sp+D4^+5J8$Q(n8vMj;z~7_$){b&Tn17dSyN~a034&fj>|afDekb{4r$(qqiumGk{m|ge31-9;uLwuxjJo+Pcb5V4oLF^?|(TFLe%6&_!S=t2p zu2GMB`PoH|tCTU@(7H?5y6ZwpK4dtcX!(mgy-fR+)wK-NW=%%uPDVMP2r|GU@QLm` zHa&~9z}F<93@82N{QD74s(#*R{<)QGSA^kYtyu}Jsks_533rkfGC{M12gL)b2E;4& z^95C7ht+m9bon;|%QG=0{>wPc@BtCl=D9(-Nk;jgy_&bTN^gBJZcsCjA19AtS5kYa z>2uwa_glQ8>r2(trM?T=obtT!5Up%ht-4D2ewv^TN?Mt7Es<~7NXR&_4<;~XCsVGI7=>W9BCX)poL**(4 z963mz%i)*j&%o|41U;$WZhSsI z9(u4i-D+$XzL+Ac2_#OLmwNTLrrd4dv_q2cCujnjDaf`B^OW4$y50l+cDs8mlu1%~ z_*EWGZw1dIdu8)vDUJl^-~a53wC>8hQzmRtEpG-w&mB|cl+FA~Sc4?%tA4(z(n zU*F@3s5LCLCMa9~t?;{cF=t;&r|nkj!a(-hHo~huK@`AE@Ow`aKU|A3!a@^vVh~B` zMejJtYEh_M>T=_T!F$1ST{-ZpbcK%I-7!61u-7A=b0&tWFpvU%gf5XhIgTFZPVSol zRkp58EuF&tZ`U*xsgffO(*K1DcC$3U*$?Nd=Li~)q`|}m7=}6kH~0qcMXOV{vl0bi zP{*gzFL*29_@}bE2}udc9T)jyeZH| zp(~LHF?z$DVVR?+5E@$an}(DO9sPHJv0^w9!JT5lN8A~3a_WKof~DoxYFr(hd*g+W z*x@sOE`eSF5`P^GuR%CSsC^##q9MePt^cM0US0|Y*ZfJm;bT8{K9mr@5uRtZg1t7P zyVluGVMM>jMtEMpivcW2PfPf6ldLXYJZ!so8CoBck8PTv(nMNgXNRX?j}WRk@i$6d z@I_ipti~&x_2q_gN$mjICoj3VYzn$BLWvHIOXv4jHKDbX2FzYJ*OjQ44T~gU9tiZ_ z%Kh?EY8&OAAA}p**#M0$ za7+;I?f_IklKx`GE_6MzoJWfHD-4zcUX7BHHjd4<5Ig2I;X}jG0&VgVBT6oe%~bhyi*L9qJ|pxHfG5rB}sm+ zR4@E97dX)n%RMe~9)vXobTU-bHd}L?1SCFecxYF@rEL|G(-G}NTQaQ6xi$(g^A?&j zb3XR&u!>%{izxA|&I3=T5+2J(Ik{F{>*h{YpGsn4<>s%{(s0x2huGH^b>+;F3E% zA<1E01iuVSm;w0#I@?T&Z(jz=ei2vCRy+Bs<7JPMeC&Hd7wQ79P1sNmDXa35W^xLXy%zg`8qLu}DF$Zjy&MGSPY zEKg-0iQg5F<>SQk3@vC5Kr22wIK^;b&bhzCiMA>1w?Ey8mXL`WpvK(Neg{O-Bn*9eCDlkg{1$rY-;|72d)Zn5u0!nf zle`F;hb&|~H6+Nnj0L%kY8YGFPHJVo@-n*~Cgh}bu7Jdw|950A9ok-PCqnPK96P+H za;1I=4x*(m6)TXTkxo{O=cJqyGE(L>ZFCK}RTW90U{+E^Qq8RWTpZQ?kKe~;-M;5M z#SU;^(hsa}RDCxhq@1PK{m- zmz#Ni*8%VdDtdZn1O+t^dIV?4987CuaJ zOFPqwmLhREc!(aXBY;D010rDY0pjFGw^BQwsCT_>eo{y4cNcMl5`;3@)MN8ydY%<% zo>EVSAhs;F9*m^YWkLTLQlsp^h$M>F6*T*tVZ%lYw~7U%lKJU!`mRvWfviS${Ill( zI=?_nWPPYpbStUTya5K=VQQ$ksl$7LSDY$c=$Ab#MGhwT+wxs4UO)4uZ`Xh^V$T$F zdzY3*Hh2L9V}P}U{4GWUlh}oh4@srACgHw<$L$VK4XlR>Grl=+6*4f!3#R$OIDw=k zxJ(f4LXU^PwWkskY3p%lM;MLEfu}rrw37cEydA&xEjFwEpyZS33amj&$S?#u#504H zkpK>FZG`ZZ!u}Y2=8@99N+^D>Zvv>H-ikl7zk$xKtTP>lT3Z&o3`D)@4+^j#1ij9B zFXe7B%YAoX*NzIA^6l5m7HLYT;kr6+>`1T67fA^X zXqp{a{Rm$e(+-MCR}&IrUQFAh@MS#S-6j9g&?r=KpQQAK-_iYdxqDIqG1!V zQnLptXYhlVz3sh5DKW5@WQ$q&M+ZZWxbf@hQie*stZsb>=>w%|sx>4W3y1@ct=Md@rbV+oAF|eTL#TLY z?GaO_2eSAP?QboFK!?qL5=Q55Det&jo%5^r5#a);U0k>!_$(&%ZL|&;b8{_e@gTg~ zVYr2b7#j;{&rukEv$khoML{JX_r%CeM{T=GbN|sNOT^TYUImj^N(R}&qidNdl?TBJL$M+3&yIwX;Df$+$F+fp;e&gEM{wSW#1GUj(I2KsSP;3#Mq zUJ$@!qIbaTk(|MQ0TyX2Iq#HLWpzeqTNeOAJE_%6{*_Xg2|YnwU%e_Ev)l z&pi@)o}4bV1|uVR1cdGtRkA!V{hcqr4mCn>=!Z}!9i(`+Kmm_s_zLX7lPxir z`*n%k_7*VGC7LiQFVP{ev$M^U4H*psucoW#KB#xA^@6lJXoTz{D=>VG9PT{T4J|H@ zJj5geSvWYlW8^sP+4%T)rQ)u{KQ_W2*Y7k~A><#kpp(Gq9vrSwZ$1L#5Tatj?7?JU z{UIvAz)jn)43xVElpCioAsRXo%-K2pm@UuZ-6cuClB49VNtstvd2P0hqsueW;4SIV z(H#dz^oKbM36YQX0|)g3ANS~R5PI1B=z8o*!0guaK9D!Q+IkDnUho*0HcW)tk9Tq> zxYPU;TsR^qfu9xeN@D7sKz}h}0Yq>`2<6lN1_*#E`fBS_X_9=JFA%^WutoxPg_ObP zYz3=!1d|93VFx!4)li=pFfepn5~1QdQpxGa7#n8jq5%$2S(Ej}k+G(684Z|L0wCW7 zpKfiWmTM%rxUcPEa4DPi{xmDghXw?xgg_7AXPS59J1Q^N@oX z(H}Pvu`z6R-UIqH4CyTpD#SxUq&ly!L(kV52Umh{m_R>TZlkT5^^#DiEJ111=aTLA z6)nMjQNhQd9N425hZPOeVK%>Lpo_+&+v^Y-Plb~BAKkMNLvzq(cQy_-gHtzS8D zeFrp}`*^{(?Q-Z-y#5KZ9S~;GX*AG@Hb`{Zo;s>h5!z$!*JOjz`2*iM^H(g#M9*)S zX<}9rgPhE2EfhqC>#^2xS5H5zuf+kfAF=FI3dYp$!|aZ_g_x4FOV;QSFy|ex>7R_H02PRe6Iyx`_HX>ljf_*b^D_a=s z(FTo}G34ZacY0!c8c~Dbk_)MI0mdVKDFH8DWC#`Ed*EMKzKg&Rh(A&=$Bfc91f0G8 zpp!y7HT#GzOozaO-HaIj0WSnR3FT%brv4uI^$pl_k?t>4ANOaqOgd!SKu5wgT^f4d zGGx>8!NlU*lA965HO`Iog|NHN1C%@q-DE)Y2Xn24ala+5xk@A`2|0>pj(0+m`iIRW zWa`z?WWO$Z&;OP4kgWtvyI2$IX_`!(Dd{2 zCVD-D9ED^>%EpGDfVUhT+*uOj$rc{T@>A7IcoU?W-~DR9mVx31Rb3qp+}dpGuW3NKX;D;LS(@5UMb1I4(u(iTk09nJa?rj$ z$p>Ps>`D$LvAqU+HqMi+3TjnY{S982z zNW|D|%!YR(o3b|heyfbelXnR^{7W-aKn}i#1ci%BLr?^VU~}U0kbp}7HYBtNP27cK z3Yvt_KNNe{TLYv+?TgN|aL}R8NJ^0Dup&sl9vD7%|BRu}H2ONUikbH{X;RP>@h`!% ziCn8jx+@s`pjB-7!O4+%z)rWx5R9)cdp|-AQogBcZ}PjQ+k}cF;*Ve{dTllUnYFm~ zC_puY8eseDSu&Ok>_5a}qTOcs_BqOQmGv~q5r)1;%-=qRzgEk+|4>e*p9T_UFce`Z z87aEa7f-DSsG3BLTA{-DH#f!d^gL?`XBx8+w>0%$RKc%td)HHs@zV(y94z*A^oJ1S zE81DOJIGo+{1ax`cJ8Za^dE6U_fV}D`Z3=u2_D_Q z$V_PWJ5S~=T7$Kd4|oA>4iiTz6BFRGVUUX{%M`GiSdIj)8pyg;Ip?kOi_gI4sOX#d zF*)WM+B(v;$M4xD5*s}U>}KH=rx(c$-;?Q#ye4JC?%1esUALbbxDWrmD#(;y0s%Lo>kEX({l!*O=$?soepI9ykGli; z?aCH)djA2}LlFG}s6hyJkYOT13k|r4!8AhOjJAto&+E!$D+3w35_BA)RQB_>i3kGn z{@k0m_Eb7tz!fY8D;-2K+QH;!qkuwvMVZQ27rKsPPF1bOn&fYQ#E$ZU!46x=f^4*fNbOu5WWLtRC521d6R8Aq-LZ<$Y zk^O40Z0zGQAW#S{Z@@}SCPN*PH_)d1Z5^p95b2&{-kUSo@9CLenP28Xy)`5{my9i2 zeA{;A_0@FM=^34qKKD%mud|3oCCSu4s!nXa;+4<{`zpA%A`nZFJ#u-%!EU!*C$|uI zOdzvR3SAm&Q=f6&g$lV$F*Qq#EV@%&FV&V3X@AL5nIlHb^PcQ?M=p;-7Z zG_FJ+w#I{P8PWg)I_@IjPXEQd*uc!CupY!ie2f-*o7Bo@(3n;_x@Q zS0Hma=#Ij7IGuT*G*-Ue!gl*rEq)6-Ss+(vOih#qu)QrJ!qKaPs^B%=YwLvjUleZe zg}<6owGxT?i?h&DQWrPoo0?gW?oxDf8aQIilt#H^Fd<$zpAIRZ?N+EmUOE8k?qtR* z9-i)ahe7j$1#5k)S^Ua7L%(>| z9{J#lE8lp;BNtj6SUNLvliy~5uop4a10ys*(i$62Fq}^Hzht2Ke#tp9^)5*XynkO^ zgfg&euh~+&|4(HD#i3I`eFh6%GH{y8-uX2Ou0+0&Rk;Zznc?0>ZbKJpz=41kkRrOT zqs4iIdI;VZ2vsP-0g9iu3vurdKFIqD(s-!gh{7I##<1`=xvJ_na$2Iv&NvW;sGNBA zpx&YsitsaWO{kK}lwKfiJb#P^!x5~W#3I|!=GmU0x$#60fQRwM#}y#P4#w6GL%cxp z1z|Bg3$AD|G0*(-!#4Z42C=j~RpOCn52m7K=!>74T}*Np0oda+XbRChLrmm>zV%z?T_O1Iy-9g&x_y}t=0#Xp>iR2toCxmH%J z4yOHBKm~#<>@9GN&DJ(39lw{gTn#munMUfWaO+xH27FBPGEl(M5TZ}4ujBWH%A&1+ z2DU)|kM@OYe*M`HhN6U3AlJ^WN`SEmwfDik+Em6A%r2nGgz&Kb_OL1cwo2XVT=s#c zRdW^x5k$cQ{!(}>5%4*)(4f5LUuyyyQV=JFpHrfMXnGl%G$obl_%?EHHw9wskR<)>f{a2(#>m7Zd0?I*rW2C&k9(yk zC(ICZE}HPpadneVjGHHM!GE_jC6N>Ao9iT$gHS8X^UnI5RTP$(!Z{@Y#xr=`Ebrl$m^`Tv-ZRX{YC%YqS@i@=Y<>dGIHL zbRtrmt9)Z@v&JJvQM{*NGx0RE!*n#$lj9>|}sCqgp! z3kVLsYS_=BfSywcW~avH7x zRY6nF*&WkWh2~; z;r5wPXexlr!gQ%S=edcQI~jJ+y!$qj!hF=ivfG#KJjao}3Sk2OyUTK!&A&S?L*rQY z+5;|qC#!rbL?cDcGH+H&_I(wHa&PO+==c6~Ydh=u*cY+rkyCI*K&>PL1Qd~N2DC#d zWO2~b0w|XV3=Bx-K#E;>qN_(E6kbT(KDG$-`9L5;wF=(Rb2NMH3u_Er!5zzGDKLZd zhF%V8qspJAsSZ05^F$p;S5O#0Thn?hQidS29`<0QNOnW&$bkSr)kl!D4DC~2qGVXm zgykkK16MDjjX+AhkQ}L0Fc-Q_9p_H+7fLlgpY8 z+VO^4ThJ{QJzNJoLd&yny{q7aJq&~_M5a*Ur9{J*TmeeKMYW(u$t2+X@)V39Vn8cA zkWW{lz4q$F9%0L5J}AjPT5U)U}`>pHRP3pVkch7E+ z=Ky^c!O~FL@=$WpO(cW72gvMagy=~*`B-O{=NXM0Am@ffW>vujv0w%xnhU@VLh|my zS2En!SHzfgIVwkVcjCk_Vxf>r0Mv58dPGL$AnPallbsKvs90mm9i4b>>eKsjIH|}R zgkE+9G@CA>S)dt-Tz6w?$^$b~b4ZSQ#qd}SW(k$D#!96Xio zok+sMXOSe1^)gqBqhJ> z{jtX-r!b&invoq$y?S3cFhZ5#z3Pp(*GN?f);vya5@+j2*0~55hXUUvJ4s{ z_xC*at|UOS2o>3~1}W{JUIb8Iye;H(uqyCrpc5TlYyu^rUC90mJPJL$?)-po8e67Hx zSiYqYTBP^)dpUccBNM$H#HO0Zy^K^2Lgk^nuo*J)7${FjbhipYGMu*Jp*L)*_Xsw+ z_1~tm>&bvup?%n*#z2-q!@64Wa2D1WSi~k8e^=M7klvakCJO5q-E^|rX?pf>fpMCH zwcj{8u+#W-IIda>;!QuvX0o>~htlC~f z4OGH49#MKQJfOT@)QM>v&NET~1shxF8kfTy7dMUP;8Ihd#}AdRbL8)#-iUBDEAM*L z7r>fx*?ovihzP|VZN7c0=A{?<&a$*Lo-{vLcOWxh5F&VTzy{_50vNV8FHF(T+!3<1 zFMJwT39Npk3IvQkP@9t+olOV*-xyXHl<_ivgcrTpYXHXNAAEzGG1Dj6IQ-(n?Q+C7 zK3&VOJCyg6^Ks`FIfdkvmQwmRTi$g$5!I`|`K>#rwDDU)D;=nN#xv_C;OwxN=9Ub` zPu4f22_@_A-gcFbd1%*nsW`~Yl7e%3hM{;$A-T(ENp~qT26=_={F+&+??#kzsq}e= zYe~NNjmK%XjPB%*5Ec98;x$NW&K)`d z5=Bf}s~wxDO;Snl&Y9h;#S1K4#}45MWJf7XaHbEvb`eax z^F!_vFwfk#5yDSQAg3H`5wI3RKrB$MyciRC>IV$DUeRD#Aq*;qiSZ|Ik8G(P$zX3_ z8B2^!Z+J_?fhI@B@UXB1nC=G_Wa)Qm+LX04Da$r`_&{mftxQqE`!q6*r`_5~&sso) zc&H6>RhhWOX%hMGL)(Z&_?4Gag9UaZ&?G!Qhi8(lgx*1lwCPaNr!BJ8iu*RYgeZEq zO3AG9(fHe2GsbpGTr8e?JD1n*)D~a5=+!VdVf=uq4+>D`HFL)E8Ao0`3w60FOgmg0 zk>JoWS1Zc;aBAl4{TkwnP^7{7cdLoM3-|p+Ob9zwYM?BxP6NqrEI>u;2@|^zT`h1q z-Mmu&Q4oM7a_3wY%uM+LRe5l+V}OS0CAJ-c2Bg7ad$YH^Dh4xir85{1ceEY&&;JN( z0j=)LvClw0q-tV|UKJz}Ib5Jq*_(uAScbyg0|VU~?eM{opUH0&q%!$Wh>5 zX(Sw&^MAP+(bf->+eV8?3VZk4j6)Ba%f>2`lV9!_sq+X71sP|~pG0ExEvkILBn5uXJg&Ys+_IL% z@_Wx*(3cJjX_y2*^%L`1hw7Eq$i2i|LD%j-8CVtYxVc<)fZZ}q^squ_8a?yJrq+Gp zp5|1*&O{C+#hg_{YQob5%gwo$T3U|py}hMJmzI3XBQM`l%pe{DEx=Lz$Nne=k+WXI z2)z;p2H``m8u93S2Ru03HY(vTxU>M&UqobiD?ZVy-M3xY~dc%Hu4jU!B%IE+(PEXkDFoz2oH^LBD z5#uCC_FB_T=n)k;Fu{P%IgJ$pd8y=0!cHZq=D_9>?j*6u)RHNgaoB|pLG*6pW&#_i z+|DS#{E@Mb2@{B+nDFy$um=S__Y+cM4uzcCH}48xzxXol1q1YW5H<)>+1g*`T3II9 ztu#9iz%Z1Zfb!9Mj!>i>Ic8b$3UVDpVcxc@gXTx;&)z$BhwvKhL_{q}2=IRlX0};QF_+cxxWV+Oo;9E)VMqChf3jQlJxpu(6ZDDYGK>AwQQm^p zl5+N!o4qS>yGa!_OY%=HTV=7DKI`o6cM_r~Ai3xaRE?F|wd1^$d*@hkb&(DOcqXC_ z{o;HK1HjM`2wWgEJq-m|=n>;eM{<&Rd=rlYOqc0Eo%b0GnIV4rG?V{TLMX>Nm4_vj z|0je4p`fmMoc@c&264M}iHY^zjG%Xgtn_i;9}DGE)tP4a#KaqsefkQ{h2GowDO~4B zsZS*y7c*5~y=wH?Vd1*DA1%kZcB-xE$w{xKv)>5j%hnpS`S_PHq8&$t+agAN*VCjD zllCs{Z!J~CwZ|(s!hkt$^Xn2h>3}1|(8p`*&#p3=WZj4~jFe9nduOQWN#y*o3I`(! z@jp6a4c^ErTsi#OGU%qv=JH8RZKn{PveoQ=*|neX>G zV18anU2W36#H6IkckGy7@Ka0iI}hs*xtn;3WIEqPB1Pxm`L`xOUrkvIKhD{e}5wJn!dYj7{Tf`0>kpWV|i72;LP>t#ZoT`LY6Yy>- z8q&)cXmCWnsaQbqy(4xpC!R~;+7+mSLFCWcDW&*C5IjEs?JvLTWmgA5z^w%1EH0&XLjX232)dc<%U*XjO6-zn&a^=*U8 zXRPkv575ezLKew$a=mSqVC=$XR2soPD0dD?4R39^+K|TZ$APiuo*q!!N9XaC&eO+M zWxSIz^;aZp^x(S;v;-gGl0{aWM?;4E9k+8LNMPr!c+E*Y-v!|&BlRm|Fz_k~R`c>t z-kSs$DO+Rn38GXwo`g&#j$TF30khgWLk(X(-BrH0B} z?}6HY`J@vCmvgkZkllJw{;G4*E;>};hi>MJ zRS2iVlFA1vr%LE3n3K+j=1n+x?}>aS7LrfyVjg?#?m+)6;)0L#cHDrfOkXdrNgAp9 zJA&G+UarS(o8bpb+`}VTSS@_pr>@XPxQ}K|xP7W95BpSJv*lbMYl}`iHJFaU!IN^j zcE*l~{@`MTX6?C40iO0lo;KZ>YmUc(r!XFzRHra6ew_Df3iYYgX?}#<6Fj^U8;9z* zb}ztfpxIEP6@ocVd#j-_*LeP&m?48G3Q5la27ZQtB4Fh!Bv;Y-8fk$~#C~KRYXiel zd`Lk~P)hY8Of@rlqQ)qvsR}$S#6&S^ob&t}s)9n!($C?LP(s1!7ed5>=~_Cj$%w4V zJHqqML#))zmx_>|W-8vwP#;0;&K?4rM)X@K0dzbNL*^7RY8n866qMbz_g`s3%^ev& zlPpF~;7+x@wBPtA6c0GwP)b4GDv(X&w-K2-aGEvYtuarBKt~?oEz4=SiT@t@0vX!p zG6x*zidu_2^BGf=sZ)W`ztSmnN1&Z_BbWSr;#x zlYW#+=-_bSKtG0K@4SJ~^BdT2p@X{Ipa_OcJFLu@0`}3uinQBz3y|MH*IsiM{~U#& za|zH&CBeP#40yXcR1ttn;4)y;bTt%@%o+Tr~ps}|f9pPNFAB~<| z=*zaQ=yLOZ&mSuk+`o-NE-g7_yk5O~{^c-^ScCwa8Uz>Pc9~*1aqndQ z$addeMys%#QLdE^y(QO!R^+#>X}p&CdUgGCy;iHieP#yr32o}UCfj}p|CysL(s&(@ zjy2B9Go_LwEik7#i{%WvINQm6*C}ZZQdebkbHs9r+yw}KORD8)4&QOtj#*`hw7}*= zf_Sc$=F}-n4WVP0E;zC&`IulHyeG1W3nuu%WhASsIn!eF>7nb{ik_DAIt`7jxEBtx z@@iP*AZm1txnY4RZBa*9JS5l%LHmvvZfOL7&z@OAKNhU@a* zzy(B<2{8>pE+AgZ_c|^>()F?QWqbx5C<0OY_uxuD{g+Pv{f|t~eK|XNofep~N=JC9 z@@JmGt>=Rj6QfBXA*W;$9)EmaG>wPv_SRG+y=>LD&U)Q<`qQwqgHimUjJ>F?6Pp+61{yd;=={n3EgmqhjNq! z@k-hqHqR9lSnQ7>ziJ(+65*!P@RQ1X5IR54g;(Yrvod|GX2h{F@L7e8JkKI?=kXsJ zeytbS!Hvb{lXz(=h3`+RJNGSQ+?H{8E&TzE0(khiS%QjlloaLY`;F`iy4#e-i}l&( zQA*pz#^S))DY9DDxuXfS%L{5iit#y{&Rg%Ce+CSlyq$zT;cIXD$oHmppumbBP9&`d zf#v%E(Ki$FG!v~42)6zV=jjhaYH}-N+*@sJt5jN42nT4phyenyf`y4~z%WPaVBG@2 zap1%`$Z$Ny3Yw<8uSBAg+XJ$uKaj%W-yb}A$oR{phc-Dkt!WDvLFcajXGRFlgRTv? zA_Bb~567%VGMLde2fyl=BkcHJ?-w4_w(W0dTIs*S|~^^K#0Lg8#ICAfOC z=lPuOri<5>b9eH{*+tY@I#hL>^}+TFnP6ub`6bc9e)rrTI)R&U~{wQ1tI}<#%#J?Vcr}AJd z+6v~^Nqy&e@!q7Y2P*QLb}Z@hrYk9DiR{JO5`ts{@^|~{2Dr_Snz!Qp0>}IqwIvDX zk3h*upx({V@9e5ezetFwxOMRWM| zt2hB=(9b}dmWeA`*Sh6e@xKTg2CkjH1LS|WlS@_(-oPty7rhc)tRJm#{Ze8b)@4UX z%i~6J&39sZ#ALLVzRWYABdocns7T2%w5(@5>-dr&y@6F~pS}<4erG^7!?)mN&h?d* z@_wuN{;sTAKl<8uA8OV#n*+CVAN>4G6dK}VV@Fq_M3^HCbEytATRh2nb}QE=ud}mr z@utSyD&8oZUco58Sh2DWyxyxXF`BQAIf4NK96P28N!Onl;Ktn?+<(32t@kYC0zv+x zq$Rc8$-7kd25Wm3rXiUPJ{+`2B!|2dZ&}^hN-}0ZFOi95l#p_cNhv}p)!4kZfPsq( zdsw3HuqK3^q_33}fJ%hLDCqZ3LpD4|iX~1r4(Bj$-Uig$5YHVG6bj}f6SJgP8mA{B!58N7KZ_p3o&(q}rz440JkI#N1Kb13 zaB6}zaGeJ|fyiX8`syK3HRz{U4x(;DofwK5ot(*v|FOb{?ckdUwu1}PFmmnvi4GkK z*)Uib3m_Nby&etr=1`eje;eZH3R9lg_b= zw(5#fa9$zN^UI-K*4&`4c|ApPXGi3DMqg6pVVOa$y){5`Ij~-fV!gEr`T0a;ml^7N zqbeF$!_J!Sw_6hS+gCd~Ts-8MMXYl19uv})FJYF6v40XEdE3H+^_AADDCWfsj^_*W zTuE)#`9UuD=v*TU%F0T0`Ume>m(f;Fs3AyMA%hTiYHK5KdzMj9SNNE8_OuWX1;ERT zx2TX7ZkhLAwx8aK%s&Gr*J#V5b%nXrc`I6&9K&ss&2PG&e#)n><=&d{z$N0e;?yb! z9FsAKxgk?T>tC+eKg(xUxWR0mBXxQy{V6O%1C!hb1!Tv1tw(rZ!{~J+@K0xz?RmuSd6~LD8f+8*ojG7`sIb7YDJ{WDh zHofiZe8>?R8rllJTIXNde~dqNKIBSn!11AURnBizcxzzP*M{Yip~%!;RTVSDIzK(_ zhvQ10#m%p^AhKxuN}_`b-`y|bcEy{Of|y(Ix9lZ>4U^38Mj*ezACnOo-A<5*gN2CK zE2TS)Gd62??2aJGPoMBsw`q?(14d}c<0ByHk#e@7VaF7m1-x)azo*)tUh;hbK{DR9 z$)8YkqQ>Rz>4qTD`SdjTDEU^?Cz@_3H(7r0cG651~o_iK!9mV>ja3q9{ANO;JG zk2R-{Nw8CnZ6rSRy+tD_yrv#5*|D6`d*?&`+nw_QW4xU8BK` z5S$@Q!Q}KQ`yn`p7K}M)f_{Sn^yC!4Mp~geLcu-iJIs$Hx>>SM6y2KE&;C{RLQT{o zZ1?19&>{g#1n{`f1ip{^6;gOb(H>H08yWLjNwo-K&R@9Dar@1$56rB` z<6i6>b_3~{BYRIJ>))Rl6f@t5!S957MW5k@=xv*?uC7|McgO^yZ><=9lw)>HAGuzV zeZ-&GWk9pT*J0$Yq;T(>BYC64MVMN0$vNie?dEdY1$PBVLq-=D4SUL@f~h{TQU|zi zo0RNbB)g^N(RyxdXCcBttOGgb4mWrK)UG^H-ruwgd|m&9 z5;(xsEib!Wi~nzJB)ygTQZIqaVf_Y_D#!8;(|G=KLFDJt*rGk~?dx7~eCH{Xq9r!g z1DYGco!;{+hUrgWiSO00@(3BKBGdEruQ>@~3j1E-Q!beseh;c3^;ZFad zBOeOz_5FA{lKmv-&!d)`HaTDIXxVnNVHPaV>$((Hpq-uB&ZJ z9>`2ueOs;8bAv^I(*Bs?r1@8ZsQHJPKE_kZ7Hq#uno~fkXxa0h zc4b3{DqaGF?Jd9-UXh2R2!R@V*ja?;szSkb1uu0T-u(8~)}?+;#pVLlnvMnjK#4%i zt)#MIZOyE{aydVLwbIYjqXdmH<>x0)_6lpl)zi9ub)W27#DlKWUL zq&;2{5<*bcVSX7v-4QBuhr>7OHL#7k!RP4mlN+v{hWu-6{}=w=!ZCMRvKBUZ@GR+= zlkC5vb?=oYH>xA(XV3m5sjBFFV7be3u5i#@dkkS66h#6zUvwE}vf^JV1Je-fe_SLK zZrD^HPx)VBg14P*j^yvFpb!!+=Y$&C;Gl~8d=Stwz(hOb3>vD~7pPsX5l;eLVdl$G zV%pCOfyiiDP~`#h%A|kzyS&~$_}ADThkq!d5gEnDG$F`>#GekkeV@T?+wd40rMwsM>v{UV+je&u zks;q%!b;xx5ySQ@ElLptcA|An>8}-xF#Z3>Je}z;JmM=`M&v3fDYau)SPyV`$6%PA zylV+Yu1m8j$cj1Yjl`6vbrnTA8+}5ol!xwpB(=2?uMs(*0uGRS4@eKWKU*m#?tKGx zVCEg^4^w~rvhPlV1haA;=wV7|UZgDvr7T3IA)pL$4!-S579^d9Ck^Ucu?{~+3z!y) zSa`X*0@Fj+5&EzBh`13rX~-xU&)h>W?70#wMH?ZXT%Bv!GdezgNQ2aA`=qaY`PSH%Y$xU8~DryTF|dwM4WgoxP%B zeiZ*U;p(r*Vw{O6PZ7x^meYIBjpqcn#s*9_h*BWe#If_Eo9U6Jzxu6SYsYjXC^1cb z3@%O6xnhz^dyyK+a=^W>X7g#^TnE#t-)1ly=r;H+4btZ(mZW9#n82W_z4v5QIJ38n zB1uB8##g9=XnULB^LQDSkKdmCOz2q$bCZG2=3(>EIQ`}sLx#|2onKb*PKLDnMhF^+ z2bHk)7_`P^gcMWL#H=I)y(qrkfR z#ItyEG7=A=WDDI*(qTWk*&c%dJeVlRDO3!J>xDhNtEkH`B9#$5&Bp8*;diH71uyF_ z@xTNL9z=|aLdLCm=7(X8HBTPwt|LXRF#s&F`NN2xe^Dq4M&)cIQKbN}p&*DAWO zH$8ihVgCm(a2u(Z8Yl}afe!q+r zOue73PCqVsl`0C)@6P2DRIXJGL{u1&ijCriL2z&t%doz$N1@)L#6=&<;dg-674MP< zLSa1+jtreb zT*5&6xbg$?1@l6QWb{4oThkE?I-%wuE*(HBff5+AO8Ob^t;g{H|9HvSSc7@Elg+;2 z8nOe>E54;>C{t@S^$crkq0jYNgkw9{+t0mlE|Qy@VJ6O{@^}wR*w2-Y^-r&j=mb9(;Uzy9R)3^5lX4hCId^xL~9)ver8Xp>QZ=bf#|LGh3xcvS$4{!MaT zHB2cS&w(!#9xxs*>A077+c|Az7m5(5w?O#=n#{ly>ZlfUm}moDD>d@w2phzCEzxzp zBZ0_+@a0`nSdPwz1zVqAkKL>HPFz%{dWiac?}pI3MRSS zLqG*qzZhNxFfTwKRIVCms9v8!0Skke|8v3LM*e$vLC}W~1MlQ}&-bLFc(H*$QwGYH zcb{ooV8OMyl9Y6F%xWX$)!&&~LQZ=d?^-@wu$}r*T`-?k4AowE&RGE#T<2;8E?2u} z*@t3YjLaNNr*}^D%=4VN5ofv-PjpUUq_rY7;mGRm?rn&-!YW^RD5c2rlQ&lCV#w6B zxfu%L6S>S?P!ye*eZIW;?ZhP{wWtYNPl$_)EqlOIL|pXO69IM@aEEg2wAoKqx=0ho zoz>|g1WYiz6oVn}U`ox(ru@n-RcxTWb9CV!#digY^U#kyg@S*NgatY^m8|P(8*ksB z(P*zcRW=*_woRGZ=W>=(w)}5?;XdZ_Oq8Gi(6(almJTIxkxs1%Iy<+MyI!gih{@ZR zZ|rURkB(F;b?T?IsN1P}VrYxplQFZO*AK%y(;p8mx>4s_eildL?+Z{45|B7j1jVMg z>+=1*d|+;#a%f{#7!>~(5nPU@6 zxT@*{+%Aa23e3h>)$Wes?V-Ji44bQJ=_&3?LS{*Ds%pZ(=oa}tfd_%2l}{x_(M)EO zFh}u}jF;_dbr6`8fQc4N1}E#{;P4Splu(s`@xtFBb^>(_X1Dzao6dXWxnQAm@Z4C5 zVP}DTuVu-+P2+9yuklj`c>JaaytXa6dqe5-f3T&%2os&}EIV%)K}@zRN!2zMVmD?{UX845kYZ(qC8yXnl;Sz0@^u)tbUD3Y$7D-SOCHkO8gsAyJ ztD~2TBlA0^k8?UUik5Hi+389|XY=FZ(Ac04 zQPPW^Uy3K|@&(a6BJE`|6FI2d_ZB{C0NZn3dz-&E3K)DxWN|No;9GLOKyfdiT3^yG zQU=;b6a80D+~S}N5((~+9p|Zh3*OuU?j|QX8R+n236p>N$?nGc){3#=ptv7oIsh{k z-0fiYfV(HbBFx1>#?wDuK9w7NPuHu_+4HA0OHQk97*)kA+UMFMA?WN`woVi(%DfzFz}7JrH>ztW=v(SD7yO!g^HZ*=g%7LVFWvJH zxICy#Si)IFN#dyGJIr*&a{_bfBqlMkv#Z}6j79dOFdbFL0-?U42+qkw0hyk=_OTe^ z=pk@kMC?#t+_7*>*A>vk!5FqUrO31mD#eE1+Raxffb)#zlO_xx}hfDK*`Z%gUt z58GVnt}07cl(yh0L^(l7BsI3FtgJyxAMS_C>+d7Hu-q^na?EkdaH*%*tLbBUmkRHC zx%%_ALXu(`uAe^DcJjr=q60m1?xUMJo7(I50{eo|g+G*E4d|y1MilAj=gk@f(>{~Q zah=~4GAMp;i(7H}^iP88r!j(xOB6reJuIhQ*)2#bwO*>4w!kbq%oK$ds;`$R=XOhm z6<@z~t8w843Wa)(*Jqr+wRkL^Uix5Q;_>@bB~Ethl!k9V{R9t#0$sM;+e4+q?IQT` zjxOIB3C^Xy?=*GG&q@S`s5OS9#*S|?Ht}$sU(hORuV!OSF@LtWDOkWF@QwX;=Y*ek zPi#Uw*8f9mFY&jjkFJy znP1Pm10T~%A}Pmq&EseTKD)KAW+lsS6v&I1cPE^C^8L(A7Mn0ltG0Mee}1x4SV=}~ zD&`$R*{@%J^5w$V(`A0RBi-V=L6FNH*9SMO2Tkj(h z>a(on2qc5=x8>+}G_swBZyxk}vc7XJy{rv~-f4W*WP5*^Iv!qEX$I#5Hq%vR{jqTu zdpEK5s>swBCEJCNMeF%6n`wJK$~c?!rJCu_y#yf}+kNQ{_igQ*Ym(_KrUmcZbs2(Z zxY6+A9u?0{vv6))f-c3_==WiMtLw8Ft9^R)If8~l#{=Fidb{ts?+!(S9X4&U9vbIIU}=CTQAc(t+pIIh@nj z#KdRe5wo>(#Rti|Z2FjWI5zbYs9^^6yRr&bCu@gy4%IN9cTI$&`x20~h&=U^UOxT# zhrCfgyb%Y7Kq}RN*7Ub4iC?O699sNpbc5at(-5+Tauv7wboQw4X3M7>R2D=EIuue1u_#Zl#x6Rd;O@JF6CKJp2^i;mXz zO)mYvA4>>q?${i+R-C?nu=G$MV)Q#bVOfF1GMsk(#+_PZPen1-P(<}i-1zwjeW%o6 zF3N11+CuRDlC$KGEnRr{vpzP%+f>{`d74327%O3m8!v=;1Z zXJgM63JUxrKVB<>m4djTTrw=8L`|LW`-3U}=8ot@b{9mOjb;anZ&Q|Yu6_cmxVRX> z?bG7!yd-{94;pRXNCbEMWba8??{~7DUHeAGWBYbBHt1lRRq#xS$+0wD#Oj)0&!mfW zBYnVexMaiwi2>_{_V67iHP<{t&54a)F+`L->mMK2Qq#J_$}$D9$Z{%w_8=^i<+7R8 za*K$to68hlE^Z==y%Y0Duk*J?f0mW!a71x=!P0KeaL1=RS~ zJ2h*9E_LWOzo;cdh&yy9wvXKQ<4oWXHG0ar}MjbxR8a@>Ym^iq-wKauqka4cY5{gh$2HIwPUL6772S zlQ*tH0JBfYaBRNPEjWu(1hkF`wFOm=eP^ed!z`OP6v*vdN0b+l&p(*hWL@mkeN^u) zXxAyz-j49~jfjDON?TTl)k0ivDfqwjr$B4LW2h z-Ljus{fI>LPqRa^y3L{As_0UfBGPVe{Sa8x_bZ(9?CMZXZ|bDAnRGW0JlarKJDo@5 zL%nDwg&xRR>pl7vGr>PsP-t;2hIiS*#j$-}_q&45K^I*JyhtBUSz$L=%1l9yt8gRc$QpgIs+$YXZ2?s<#e)ja`kw&wK|s>dmIu=P?IJm2hMQlZu}Z+9etO- ze?HzmfS#~l z$fKz3D3M~l^d2s_f>V8b*14CzekD6g2$|8 zLObL0yiWZc_EUAW8>s)9B}scbQy zup)q5_oq__{wIC?uK0+-y(5gV-yu#8Bi=a`@8g@uGoU|9P%vb>u!MY9oyD_$A)|y2 z8LQP#AwJZ}s(=2v9Vd3go?h!q_Y?N|1=5q}rm{L{oG%r|6BikI8C$(mg9={fk^=k- zZ|{3*u0N~Ic^d-U6YMc}gI_1O=r{Y%pr%Jtt)^*!?jB#o7Pg;y!W|^@}hdouwARKNr$nK`Y*Z+q&lmzi0>J% zlTDOFM64$DIF+bg^D}IZll?M)e>PYGy!u2aDRsY2SqAE<$c23Zy&q+^Yj$O zKynf1f=z=RTaRM?;d9?SdLDH>b=YN8b`~7j5@&DHoT`Km@s>#5OU$G>I7()M`Dwkw zA6bYj{AE{>NB%WDJPIx_%K0_S?~iaSV;&K9c)8dOEyyCr8&z(JLrBwlPc~dWgtIMp zwOt}Ef*AgIGU=t91*2lITc7$`d43`G!IvTd0*$w< zM--L`{F!c-FZjR1^91h-6e^8Eg1m)&^*dZ3er69j3~Y_^!8Hfs;0=a5bvyo7?p{#= zsB^cB!r|b{s=eOeq5D*tY(m~08yU>|WoHXrO$?8fL5LIO$-^kPI^8mDWoKSI6c>69 zUOY(TFvC6=Vi*6>Z(CONLZt@5ec^K-#gPz)XU1)L>{QixoTpssp|)SM@nYn=+COlR zmYZo7od5a@1j$&y6dcwYzu$<|uqo&OZ=}{gz-uT$q>WVAI>91Sc^_<<6%)F?;=5u}5zEjw*7)cU)w$<>EYX{D__401pw7_nK z%)a)V+dRzBz8c17S8sT@>r1!aXD9@fJ?9Clu>PpHq|}!UQ#G5ocz8uAL+?t(0fSdo zAKx-7;2KzzU)Yn})2mCM3-dUfmGh~ucD8z})JzO#9B9F+EKVkm_#M|HSr0Krl*cvv{zpUjQP2fF*^bH6(t%+dO#%pI04B zT3)~pKZ$&mUH4wX+>z-EpGf zw68$=D?3oR_=MPskEy}6wp3u~mI;?r+P=p~Dn!)yS*c4p7#^XmWY$jxTGmWLlNa=? z|L5bKe9W7EaKQkqnaH)?yo+RH01o4HwMco{1hq;4gaL$5htK!y0Pk z?YzrP2+ZQEIe}Jo?^+KoFRm2hP!5dP>iVjrJ=5HU`(yZEfJSq7xOn+y&wTs+l8>u< zkW96$ewFi6Qgg7#y9#&DZQ%@vxsX%cZ{FQ+Quet6r@1?2C=#|SgE&Alp`RkfVLW-V z9~jxc9@Mp8qQg=ZH!r-)VXz=NWY&XQy4oBn!Tmnr{ktgy(e(z>pK@`Y(@Iq>eWv-y zXC-;vL`P@Xfu^H4VHj{QS#wxw*_~2cm@6nY14fPi^&|3^QQqZEpV>J0d%ntyqu)f7 z2r~6ET^+?b%st3vhsbKM*Z2zAL~~seI+|fu5wPiJ_b(kp_JTxEdS`DNqt(vuJ8-ex z+j1Ch^DH_0gnj9S3K?WpoIEVko-X*^Yco|7d|NIMM-bEsu`nQ^ED~Z5!Hi3Yz|IWG zM5qW2A^-iQqZd6ofXCW5ej>4=ACAZ)pAbFE(L=Li=}$u$;`&kvNWKPm<=dVuJVa#K z9)uw=gOL6Kvp5oSV(=}+`&?a-g*{b)dVZeg1U~-H$G(2ag$HA=nT>7dZPn|VYdUNc z(}0E}hr}F6sAf1F%n~j4auR%+tNdaAUGWoIT9>_MI5WyyN{4yfGm_N~k!$myv(8`Z zjQcv0={jlJUn;VWI5y0@ATI?65RSZwn=3E8%}P5k{U#oOB$KRpv!n*%e|!5Ri;w9F zen)s<*P9yw6PM15l9$j0T_g-#JJNwQ5?cMQHF|4Ku9YqBf9mwCIw0V;(GM0XlyVms zX{E)xqGB0O0PQ78JlcB(ZcC;6Y)tghGVnRB{)8I{K7Xn2JcMavA3uCJJgrh>TNYbl zoJ*bzK|HfmDy(0v77#?-aK2^FSk>cikf=aWp@{8n<@5u=EJZ|pGI(FT`>X=;fU1Rk%SMuHNz8vIZ2ef>Wi6bmg6_LC9-TUsr&%K!NB z;}l9o;J@O^^Cg<&c`2Q2Rc-4VupEh30oSs~=)Qp@AIb-ht__cGvEDRQ=`>3v^HR5Z z#Y%>mFZ^o3W-RE*ulORPWpQ_UY`Ks$UtMOvD;XR0eCuysaI&zGgB&Kq!c8Gu3NbZ4 zO>9US0O6~i@qhO_JWEQ-p9;FAI@XjYe~VIZ6UxT_``drNzD*hrx~JO+!_sqa5ePMpfZ!D-OJziDm+Sg^{|?jEH$e?OjYV)q&O$Ce8jOCwj7kaovUP2P#kz*JtxB^#*Q z0gUpG709<_Hdo93+gMcXBC5ZR$toP zzw0e@4vr-OEuRn)_OD4O>@3XePX=4n%#$^lw8FD;Y~F=T^zIO6&RMHxS2Viv= zKl8$3!JDQPFmIe!fp3**@}WxRI?XL{xINT@1-ca3Co_~PPN!6GaT<%%tO(KV|DH{(_olS1V(Zl2J=n!q5fDjd z=tAiqZRzy3J`^y#&OK5+HtGo3NvVTX&{d>RWc$?Mftp67(S`p#y??)bbrB@>{;}VZ zF*kG2eVjIJO@*MF2?&9+Y0XP^8~m53~P)x`Gq)#Z%g3^gG!F;X;imQp9^?gvuk*fo0hK#riaJHe67&{{TOYDZN$yiddku&`<98`!xvchj({A`~1Qzu(94@uSR z)bQaUmBQ`b^KzE|2}W_|03os^;MK2xX>5EZ?mx72`|qYh-Mr+<-;0Y~>$G2*={&tu zS~HInSr_<%=7)vDcyig7u}|G3{ui;|O0?&pmBrhkTFt**drdo6v5^f@l5+N4ztiZU zqTIeX6`B7^W^r&rR0)w!G-%4Dhp}AE&6t!WQfHVomy zf2<*REQWmfZ82Jo#eZ*7KUbe5m6E+w%lZNG2Hr|^=1H=MzX-m!d{l1roJVMU1TT4wE#afki+w===2$V19>JyKy6K%|=ic#knl(UsvUvmOnz%Fdm;T*lrgcS@vp2;d zzQmpE6aX#HQ+kgNGj8zl{XeX|cRbbo{|9_ZrBb;XlwBb!R93bIu9NJQU8!u!%%*vj zP$wgska9AgJWcG?$_s_x_;m5zW=&EkE<>n=X}O{yvFnOe7=r8ukOzs zk7|ac*T3LyxXuizW-1`&4HcjkrB|R(jg$Vl*1Z!G_mRRTxaH=gg`3Zy2A%hMTJi# zT&#Kn2%l=0$WCGyz_(-12->DmLKY&XU}2F#kbG_z}#xBeGA;wb> z?4d81o>2^p%U)4PpeQ+{xX^I>*l|2A3|UEV1a<;)WdUa-JJA;E6z#W@^F8jt!j>ya zn+DURM@$AxdosG;A{(%8WF=)x9QK!D3*-H44BzxS*MLAt@GzTOkp|oc8&U;`E9CpQ zC}jBc4pk$>ybRK(JP=G0Qqpv05z^0B#+8hV#Q7y2PvK<@1^dNKKcT|?@#9CAsiZHM zII#0(jq3H)1J#1K({Qh7HSomoHQHuF|D3&IC)RRGOt&dvU58d|jr=Go+)yMEF(il- z{!kvtfW_!p*M+#M3~RU-1Soy<5V8W^aB;m$lms19HvkoM&)eiQd^zvw=J6h|wECN2 zp)(N0KDIM)wZ9YzS%CccJOJ9DR&lj}GPW`1Ap&)PMaecSa!Inr7lcmD+_LJ*P&m8# zlbTFMZU~Co2NI*ztvPX$iznuSBs7fvxOhC?WJOIE*yU(d`?D zV7`B4?BmB%C3-S9HiH$J5YvXYk9!uoEQ{7UPqVYLn?Z2J>>Uo#mC$~hn~ou{%b#~; z*!e?Q(%-=mcj@&hMYX4XOtpLy&uKSO3+GaHkTfmMxtvgt8V8qU9- zEJZs#l9^H3klK%h9}VXf!m*(Ozqw8nr)2b3Cb`UxfPss~?Ag0l?Tr6i>Rl-J{Ecaj zzHn*NuAma9wyd%(oN3+v$96Bbr_>x~6Frgzt8;6JdE}(9dK;TPGTm|~m~4wscV%}A z{)(*gu9=%BBcSKQ_0{yP8wC3?yiu5fK~w4dl%@v&W$#m#AC6#3D9OtNR?;u~gYw!- zd|DgEj)oAu1(!F48W@k1F`H;S1^@+3?@jD7X2bz+(`-8_ENd)uG7?af{}s)a>&HX$Q<(`I}dg= zg6IlJmV-zNg?f(IZwS=BSUV-U&W~?Ci+=Z$lerV!`Ax_k*fZL1SkQrWl*SxKu!xOkeHmABJQ+P?H?8mz9Tw z8tf&HhD@LdlFZU5c>KLEVK->nUwIzuTe}>^hU8{u_YVu1xxse}eJdSR+lG>%nZpnA zVs5R7vl)Q_iF7mnP*z@E%XbjgbcQ6(X*rwl>@bHt1Q$Z)&7Vgx+feMyg|bvdE7d8< znkhm5T0ci1R7?G$ER9v2C#2VFrG87+bdFepg$Omnm6jt02Y!r6p`SL-`)92ezgJO5TdsX@}#pQ1hfA-~Z${|g(?cVrN$4E!RYm$RZrtcws>hDq_*{h@;m?d{zixiPGq zHf|J1!5WV5AG-4GOhRgDU<-^kZ9^@5?7jOp#=JT4zi^;n&6S>_D`+%V=3*<>z+{dK z8f$*bGJC|9xbnuExBZg;~SY zH1YSU-^#R5DmP|_8o(f-P`5u>=URL(^&V-(hW&oyugBeZ3b3u*kG3YqYcB`TeK1+~ zjKjA+fcV5;5iJZPE$l4KsB~f24>EMH5XBru0v0<2rE+=nY^jD;S+@s2PU8Ll6$+|SzbR)0)AAGGj z;0U@gz2wkl>da~L$0H);jva3dHYQId@YiK!9``%q0z*TSU%c42wdlYvG-hliTTtX5 zZBA0B7jb-%QAmy=cEFXt?{x(D0>Y>e1F()>1caJ1dg%JOUx;Gjd-w`yGsit(8V_y6 zVEK|eQHc_xj!)T!ef}yhaw+UiM*e5?Whg1Jqd%-pP~!5Yeyx6RBl#Acf)_E(h0avS!<5=U6c)&agt_DZ zV+C%y<-P&V(JHgXdy{cNzFc&7OR}bdNYDM)i&G(jK_T{ZaRvd+4C=O6P$@zt3|p+*(Z7A1YU>oi`K_5al-;8TmU}c7-5u7BA7O$ z?C8eQ3lAYkI(70latpWdz5M$}C-AGBfap;jb%#s9i&COZ0yEz~aj>H$U1l`MmUPeW z&qbGgS?_ZJZ?Z9UcOQc&ys`aWKN}hxY~#vfpIQVKh<*vlAkd=TLVygB!z$sma;U+f zt9O3mt3U6w5K|ENL*>c;!UpV(Dcbx@(vhjatu=#`Ow8@a8)kZ^l98tkfz&9}n;J+H z2Y9r}*%}+*0wuMPyvPWdouT{J2tmv?6vCyPq8Mgcdw-Zcvt}DA`qMag(2+Kq=r*IU zc&LkRvVHI(E7oW=06WPMN!ni}Ya?pixde+v6C=*rVZqk;lA=6q(S}b zI+vavVn;uF_6%8KrR=1Zl;BVio3!?J2i-!`SK#{)?%sxn%7e*_w#MU9WPQN)JlOkd z|HHuZg_V}dAV^KL1_(Y9Tql@fK0cgi9IOp_C-qKMoEB4D4ZJ#L)yi1DXL69bxRNBAm#<9qPMxlPD%`+7?_B66-qc<}_2*^|9{UbL~ zEclpA{G~r(JigC&UbF4oW$U^TokSN-?*CmftHx)y83IhQfBz=@ja+=oFv7X24e$kk z&cN0rN5mU9_pje$b+en3niS2u-)z1t?${x|%b{~xKxZ{yXdE0-+{L+=&$ABz^bmBhwh1%un11NT3)dKBrl5x(h1 z6xdI@hKq~qEzehx$`?J`cx(zS=FBv>D4yG(7L=XHVz#1HRyMjS-kQ@MTNx#lyo&OyVxr+Jw#f1Xf(< zMs;|@rRAxJ^{P$(+@Tj=J6J+dBA#p0P&cN>hK{X#(N>GMQEmEsC%zyDr9$uR?Y*`9 z{)oW%R9AWoI*;MBpjSCp!};Kvc$Q+{CJrCkW8OE78k-`P+$z+pI^U;#D=u1^Y4XC2 zYiXpI`p*g^NT)zQH{rREr9z@{x4IvPMf3Y@l~V%swJH;OHq$znMn#Ow>PAI=Tot!U zwV-dfnqIgV*(O6WsgY`!CsfkKG`m@KIjOZ-OFC4Qo{)No<*Ir;TbUy8A_IB(JwB$+ z0CzO`>|9v-%#=^l3uNsK<8uwwyO(T`Q-*7QB)uw$;vXNitPNSc6Ekn${ARMiW`@Ph zFzSKBkwUqq^(pQ2ZT&0P7m{fdB-2t@*)$|=@-<6v#xLvtnuz)IFjRQp9=b95M7dA* z{LP)0Dsu^<*W7JbR7Q>qqZVq0zb+qqTr;p)e)x00sceT?sKDyQRNY zPb=73lFG`5fK%XEpG}g#mJN`=lN9-hWBv{k#j~OHt0Jk(x(u!ESkd)seFDb|ToWeF zq)NLvh)BVL>6fQ&d*H8qf5+8aSc!K^glb6oixHOf;fwJq?X3g0!@k-kN5tAzqJKVd zFd~uowaf=obKRf#z`L!Um3JO>9f&6u8BRQ&CJue>jk|b*#HD0D!1F*RAfr&K&r~r+@^#>nCMXk5ul*Flp93XHY z2j>%5*hmA0qr^Oen5Ie#2L)D^dE<*4})+_=hr7X^WA;m?q-9B>*>8s!)o7oUwUHp{pAg@ zfV=NizPE)qcJ5xufzy_-Oh+GFu%hp{dia!NJSB7ZP_~_e$G7f-hmyr9_iJ7)Su?~j zZ}}T;>MgS?bGENFy`_djUX>*X_SPJS=41jxt{Eqb4$V*`$6{NvZ6U%i!RYa(qc4s^tHc4rQ~$?zS-hgVYFHTAKrdJ<-tKygOy%4 z{Mozz)RQFdHQvHEXX8eilU&jGyOjIs(k2hH1Xq~1fMK2K_@FEycG9!>W>UlI0u14O zec}VW?@@jOEMa#1G#7s%Ly1#xgKXaQe+m@44pb6kz&^U%7$mv$x6pm)Vk*dx4XA^% zZ?0DkBK*N$ z(OiIxM{tYyuKhw{UzTd3WbR$qNtA{atF%)Hp{n~MKE z+LV@lG3-iPf}BjdwbL?P*B#}*_5?y3;1bd82|mkxn% z9cNgj#JdtGDjchJd3`uEQ^N9L$F5!fk;rg*uUt2ISHm87=J;s@5uEL$X$>Rb&;AXD z;B1hSHQ}cH#8^v0)Xk-}x{3Fb@#f#&tHwAema$5d#Vz7}!bJSJ?Xm9^o%J!zIqn<| zW3B37D8FYn-Mesi%%>Ww|{P6 zW=et!z4cx#sDZ^EncJPrTx9&>%suo9=9OySSt7sww~kDc!11d*v)L40efq9r*_gO* z-x?iqH_%O)$zLT{m5Z>`8xn6&2zt!it?-x*)aMH}>Oa8F+zdOUsni2MJFb_``oClmr^_afibyu5M0DK3!_HI;#~kMVrHSUn;osw$pr_RV67h_me5OoyT+c-bJ*6 z@tpBITxHfC?NPDjo~O^}o-=xQKSQ%}Kg+T4gQxiGVjewona8#XzBh70LL{au64yp7 zC{YzjdbyEoNvg4xzjN(|3!4>QPPux$n;IgguOd!GICZ)6S`^DdU>vv>*RhqQPfbnD z=aRE&JGUpo;4(9_Jw4rGN-7>qt<&;*5Mat$-7?gSQgz3L{t92uZO>FT%H$XI;^yPwsX-6| zKL8RkNO+^O4m6}+)cIgeET@3eGzQ+cS;5YW@T{+0g>Yx-33CStF5Pvqo|4N2I+;9 zX>o;V9~X&zSH*dQYX%NFDv3s>x=*?ylL!3?G+|>DIWF;1LGaK)6|P)8dCPxHF;vb) zl6W0gPMYPv0>-bBfcQ*^2A%Dv9o|J~L1gyaoCZ!8e>=)8PLn5ycCWsI^T&Z3tn#k= zpI1%b2ehNx^{z9Zs?}oY_{^LD%+iOr#5U_#Q)_aM3z~0 z%e2^zJVl7SfQEH(xo@6xkr-PrUrhNz#d1wzCq)#jB!()BZnp0JJXl9s;5WeDz4Fyt z_azXQXOHqo)|E#@j5Un49z0~eLk&Vl(%))t{W#WB!wMWl#M(Q@4(w+<_mx${;I3bg zDps=YHiXu4MVJ{RfH_74=L@0IXDy{|z3Zdv4gKv$o%HHTa)iSa{a&iDrsBY}hc2xs zFt?d4x7Lce&ZX*J&ej6l8izjt0~zom$^j-C<0+NAEVq6^%`mY5mb~QB4}qnUXZF5ZpzRJhNDIs6ecNzUGswqQrhT_VIpZ-}3;;I_{ z;%|eVHm&0ZIC7yyKStOZ2LUW~X=Z95Cg#E9C_?Un>=8dU_K{g_!K*-4`5_3brgp3P zM=e~=-=&)X9stQIUf!5D>)>Fa**j1mGviTRv@?@A!DWJlUg$Lc@d5~d zvDb1wkr*N0ncnRt>}xsLfEV^G3>qk67QB^LpU*c4OAt1~3o$-Nd2g?u0VWBgpD{bY zo^wKw5M|7GBBsFj`(MT#klyX>LPoRgo}|q)t}u~WpLYjq!FDvmRx5NQS#=e^u5uW6 zR(LV^)uTMG?&%?Fm$6oH^RM@1v!4eA%zM<|A5iRHabr@zL^@kPIrt=vf%e_A*QZio zs;)4;MDE6A#*gbB3|Vz(*H~+kdcu(-N8-MJzrF$FNUhz^fG_4_L--XlRbO5sJdX}b zD0V^Gnsx?|LNo^SppBrLdxywK5h5xc=Quox_BX3k5Lg`Dzw;MFpc2@=u`72!3c3sl zpyHdEfDA^6LDs}v)8K)nsy2dVgFK#}5E^SiKKi)TA2v@vht#-}q8;sNRWQbTu}+U3 zjYRR3yaJ9*t4^v`(bS-;p?_F9#I%y41Nj#(wDjcmyrWAm)cKK@+p{d6ze_(%(6?D{ zVzr#;5GhKs@^_$3(ew#UYlXbcy-pNa`>=_xm0ujhi@XhJtA|fJfqbF7Ju{yX52(k9HvZ7YHVbNBFHTudGWv7w$ zhVssA_Z0Ki)cDqhv6eja{fwCUfc`H_^YdXEB^NGbKex8oaTQD-!}5+Urb-vsiZ8bN zSJzIw3A2^Z0h>DXm7fm2Jjn3IetKFuEr)#lD}W)mI$E{WSNSDFRw*lti)r_=Sj)Tg zj52U#T2;uj{crHfh*e2nrcKROiYPElmv<>p58=~}u`9T*w_yRD9CQ2_m0;hrrd%kO zGe6kUX?#_6Lb=S-1H%S^ylaN>RMo1FgZgkya3&%Z#nKXTzxn=lcZAT|fnqgWtLk=I5kKgAN^ z6k2z(V{fK{_p@Kti5%bD&v2-rMXRlMS#KbGCA0bQPua}o(+t3u3&NHD#6iJ69;GX} z^*3hnjWzsSU3!CBcx@H)=8ft9K;Xy26fsAReIDZi&0296E4GO9 z_sT}qt{*D)-b$JPvB#|~k&puW2Ap^$69OR{!)S`e>ZC{opcqm<^vpRnQP z`vRD@Qe}97)5$tlFDWUh`t#>4Cg1KQI;)LtT!c=$zo_A{{|g9nveFF? zW8$ceZmo#`kw2bh<5Ds-zru&jYXjnWaq9!f?REc)z6)r!$p6Qj!Uz8EkO;Lk2e3VU z^5j4AN14Zmj~rn`cjld~+d@C#>h`I3HlZG9N=OBDdh%oM0uqUkWxGo5{6zXG#n3nZ zTatxq57b%bl6s$Vy!roiQtKkbgVKFo3CG+ zp?;ZVLy+3~ID+8+S#~?k%6ciiM7ZV`%pahpr4__B3|+1MBYBf~+yX={q+k*7Dyu1h z9etEaI9t2I_sb5FUy0im8VaX=zCC4?t4%U0fz%VbwI<-f{X0RjD(xGceh%^Gp_=loRt zs4hpQdtnBKA-Wl>587sZig7AsAzn4HBeOp@s|G5=|JPuU2gb`ME@GDHueFNXjt;ju zKr`1mG0lA6F|tOp?-`?PbG#7`H|^TVO>3=j4Qh~rP!^G)`vO44h8P3IO|{&z?Rg@@ zcRN>iSD8I`DENLMiEs%{Cl2NCmu$Ch1Al3w+QcqgXy=elqI7y+trg*QhMW19`Qc8r zNVhVDS>HIK)7tc!`ML#c=~P8zE{SUj-~>x~w$_MOGA|bw#O-KcwMAnVXWNX}(LP7{ zgV?Ov-+u%~k;X{}QmZB~1I*jf$kj+P{TLO!_q`_dx735e@o_}c+8)bns$9*SBO@?N2$s!XJGcxSR9>crWOG0idf(zg=+@vUNBlE?* zs(HRLC#vF8!VGq)oZft_=&x+-?8EtHO%RF;wt_O)ye%hNH`_8zSI8pN;3Sx~J}92c zVi2t%xhe*`LLJiPJB5PkK!~{$e7~ZA>>+1KZBQlOn6+O8InKkGhVGgSB7-~cfkn=B9|(Jm?QQf>!dekD6Qwrzl()-j^KC1JfdI#B!rvI&-P+D%@LEw@Sv7K|%d97qopiP@}N@&_F# z1e1*R3)vjIk3L}>UHB_?NbHs}%D&=rlo!KRG(UG*a&qW?*+^w6?nIZNCh)f zCvFpF&`6V{h1DHt9zG>x-QB+FX=r3xAM0Z~!mytXi#peIH0E5gUR|(L%4eRRpU9v2 z_>RgpvW9eg1%99xY116dTZpw*TsosJj+q)!ka(4XX1QZtE|vR5`_NCn&0sC>7D&WG z1&HqGM&KsLwW{)NF!gyrs&Mq3Brm$N$O2f+?bjXA+Tp+6!SoM5Zr!J`KV=?zkAv1D*3fb*`v}vNZ(_e>@wJ*$1_rTi z+sh2nbM+gAqtE>sIP{%6@7TCNAFf4Z6bb~tYGY8WVB)eZO3uQ*um19ygAF}Ayg@ed zN`5DojS{lCuA0k`x)(caboy-8ooH*z@A)>oVa@sf@Rh{hPGOD?nQwmCKSJ34*)L_f zwIexLnUn5}!s2T1Mq-kjwYog7fPpyqEV9eJ4w=#SK$#rE=*{@x`*Ro{S&?Sy=4UVT zJCZ3m7m}x_hkKg8Cda={eJ(vxmlAU$`n30>hpMw%emuSrbjX{*G!U$jiZ*iml=j}X zZEvR0nwPZ(Yoy;l$p~Q;IXDABV@9SNcpb{n&D&=Br2h*L-a$>V8CgZYxQ(-$B{VZe z)CZMWP#Oj)m7<<+@uQJ51S`s*M4lo3BAOfhw}VA}1EaAhtXcDpI4Rzg5Q9+V@o;@B z1^W~wtc*|+({Y94QJTN+7dZp@U=JFro83)GT&`e`ka`{aE!11qufE`A%BR<&VlTsw za$9$Ndt2~v+uoV}s(4Cuyl%FD9wo-E`E1!+LUvB_jlZvF^sl*}(aY3|3ZB+a%NT!E z+#-~Bsbaq%Mey5&b)>Pw#^I&5_s=K?q>=Ys((0|M!t-VQYTMgA@r5dr)Rg#6 z68l}ZjM?R>i^Yq^xu*(di8MM(qS&E~VWO?jY?Kaj?{)N2SLn2D1b=~H^oL#`t8N_u zxK^yL0>(P8S8j7M171ycwv zBYxt?<}W{)@cJ3^T6C#mz+G=tD)$550`szB;RI^(RZ}r3W5=$cwegn1C=)#DamTZp zZ~J=FHv#cqcmJm4P_)IRy-sb|8-a{%NAM20LACR9p|X}K+BHt288NIfLdi7l-3l$a zfi0w^dqv5?HIrVAAA&qK96oZMQfVC>olQaj=OrfA)B>r~+UNvZ`;n`DZyEG{l07bSB}MfZ;kP-R><$1-V?p9j~M5#3$PEn`Km$ELHw+z(d+7ZP~?WYsn8 za&B1>J`cYOW7sQ>gF_}Juv#j2d9SC$th6Unn%;eCPTw7)o6k(iN%hC+$wO0mo95U{ zWG$Xid8$0{*+bLoqQ$Nyp&Z9rnN_p9FFllM8a&l(o0CC2~9YcK^W` z^x9iyzDMO&kpgL@>kOO9{!6=eQS|&Y+Kk}fW7YT^t1cv!loX%!-wGv-i63mUm;cct zc@#!Bnb-!r4zqAX51fQpj@DW%T}q(dtn&F)@Uy^2D&_cBw2L6>80Y}9$j$HF=@m;Ygi<0s=;@aNcYS(4FIuiQTA%K=ECY=jvo3OK3l?uMoqkc(5i1qoi z=TA$nkcZ^gUn(~{#`V(;>@z>a!#y3*6?p%7GNJwl=j1Iv=*p1%MAKYXM5V2TDxgKp zO=W?Y#ps^T0X=f?1({gQh%g-u+?)Qu%Fv`SxuEFa}xE?{V$@*67czQQEi6EN=D`r#Rjrbtq@f*y$=| z81p|C=g?-cEcouA*Y`|6J}$Wao_8Zq=C@AnWlOXdV@LA^rvwNf*JUR{w+)c_`6a}W zg+GjFpv1?Pwd@FJ?l?_2@m(P8Ek&8Uk}Gu!vqFt#4sIwT%6gKyb#+dq8E-zoLgWcu)L}#VQi4 zAQwS5hIxjm_nL3NGE(4D%=%9)!x1KBiI<}|(s+_UF+~~J!KI2S8)f79{DHbjNk`KJ zbE(`}tJ|~rwTF&HZyPJLu{p}jx!khjk?Q0hOL60Y+3#fU80!3MqW4;rP%v|7k;*3< z1yaitldI*>?w9P}=I@n{dYd0LyuZTBdwOr?QKp2#<#*>iwAd!<@>kjbtJ3 z#@UYWZV=2GOOUl2Yy|T9ikWen!Gb*h>$ZA_b#)as%1E26VoKlBGEgmUy4MVM6M6V>z*1^?YNVb`GEi)PG;uWM+ zaAi^+kK{ZMKO}2YDZKK&ReJT0eF|nE*BtBVG4<@1?JcE21ni*tx#391Oy=Cf?Htw> zP>B)ZCs1FU7}ij<@rZ~w(l5bU4DD8FBc1l}rOu=@*O(s(wjWKDqm>n!nmF$j@~>21 zCq?r$TgO@(nr!pE6k1^WW{zhMUr#3crTh@tY<^=Iy2O2de2P3&URM%yBz;0#V#D}1 z8?tPSpuX^yz4YSw25mLQdp$pj2#pXf3hSvn8zDf~ zy7fhy2XkwAM&gd+ugVXmLL0G^+tUotC~!PG@&S<@4Nb~F@4 zXuig}O!lJ${#b(m0*)7uW~O-3lOZN28#Z|Co`qGif^m2;>9yqcvSB*9${YY$TcFVd zB7_7Pp7YlX5PNUPU?+zzAu(=x74#69{c6Wgzf4qsHWko=?Fh)jL099^5&_k-ijqR4 z{|+;$DO?G16OfV>4j{b~p_?1Cw;?oX0Qn~xbl(ApG{a&oJh$kH2Pmv1&rhPE1qUWB z<{@Pw^nS?f;nO&`{2C9@i1BF_1-q0dEm$y^5a4K2*E(@cUQWx9u)7@;WO?A*I)jtu ztNzCN0rPJC;BewpRlzLe4@;i?md_LQ?ol-4*-mP2rMzGVw2NFEvMoE*01(~agOw(L z0!gy$BNV21Kn!?sxd@sAh(%&_P2E9v=sM0oJa_j67>OESpszvzAUr%jJu&1+kgv%g3Nu)YWCU?PB1z87R|ckkXE=0L*I>I=};1K7Dp z6EZ`ONl~aBVd6lk9*1;*D*5VCv%8UR(f)Ddefz?y1ztU;V_odSMY$$Pu$lec=RQp| zmHR6Si!Corfi!>Wjm4JAJ{;7Jy>FFYoS6m{)e)S>Yk8s(q+TWU9tM}>U%U!9YB_GF zKrK32vKqF6^tC}G`ip^do>QH-kNq6{$wi3igrIXTR!2(ms&d4ahRmon+OPP~q1FUF zKGSL@Ar0I=#vU1@i?6zvhxN)y!cIzQQtdz7M*$WBh=KE~Qi+zv=@r?nfbIHQbrV}b zr*dcCsEH|DSM5L$CNkDxvcona{eE`L)IUBxO?er{GoH#4*5m|GCIgsf^>XL0Ra133 zj&-YoqTV{y2me0lTOM@9I%6m;?O+>o_dOffvltF$_r%ZrIQoani*!DT4EbdXLa}>@#P7uZW{g7wX+GNe`38 z)#?m|0cI04mVF1oLgUv!ouyS*KCDmrn-h05+~6x;lrRB-4#gX8bpTnBQB3wWg==E{ z4j{U+oVST0fRlx`7GJK7x5r@Xjc4-XStpea`iy9McH!{4MSr`ZsfGqL;T<%qZa&l-G-)}6G;7~cR?CP^^EP1ip5jJzZRZuAkPrxs?|5LEdo?7nlCnNT)XAf5O zIJ1yJEs$+)(win+k{73wQcv6}5ZdcJL3TpW9?piLvv__lHi`llj&H-`m0NmBZE#lQ2Xu*!a3Ow)&kNZ$0zE1IJu(t0O3UayP%T|oZxDh6 zrX`MAz(Ik^r*bw~DHzlapiZf-IPv|#^S!I%zU#crUC{OIrhgOkijuGDeQXDF^1gN~M+} zM+@iy=b7lCiF4a2wl^o-7P@WlS+nm8;IMmD@ZbBY0$Z~#T%d2`kypsoS6NEWr*FtE zI5|SQj+c8QsR9{)e4zMLv3P#`XCace#eft|lCOdSqGwnZqHqviQZ6P`sKa^Qr8#GTg zm~LdbK?iVnZgHJj-!ld3T@;)lP=4+_T2ggw90*{R!dRI;NT*=Uw~e$=Yrb(&IQxEQcr480rFBSQC6AAd|jjyS;pu`AGNI zx6~!LXo@eQ@u3qkwtkVTf(k`?*8~&Yg<28^J*#>GW*8-N1v9Tc{OCV}P1`n>YJWiP z)q}O8kM{4*vt3u3tC(m20-r$8QI8K(A9g;XV$Ri$@*szf)gOi0^bPLVA&wG41v35- z_$3^=svuo$G9;h?jligwnP458;V72cz*<^+y7Xhm0UB|TkknpB-%APNg#;4Nglk~MdQl)~DR zoV)QZhwWBPgu4BA5q3#=0j*o+KX7U1s3-}9qEIQ%5$^rtD$qELQw!%f#*Vh?%GoHV zMC%Lgx3MfVgdSv;xn0C!&>kN|@gXyOE@@(*ji43U#D#QV^QwG79|Hl+-xv@E6|F3F z-7yf;IE18=<5g6;^Zh%=r21Ni1>%>t*+Uu#>61m*iCm9+=&DBBY{(JkIB*I=HusoA zCmLbzBe*O?00aqfv8&}*UftwOaXERH089fTU-3>a`oV@F33D|g@?J3Es!&^15hqt z$i_34WT{jF*Rhhshj%dt8Hn#XbSh{YRZnTWvRs_6A5XO##VKyLPFQau7e*;ig~`wx zY_gM?aWrG>XK$=j)mYcM`m5=l?)=x2E}(9bed5awYJN0ynUI=a_L$aUeF^=Ah8p%j zZQft$Si)aVF9DCs8Jx^1BUKHJJxQQE4a35g;OB#|&rf*mN9#DN^}r9qPU=_GQB%y( zk}t#UzSz-nz%ScalsF>6R6SFmnu)wOFcM$?A?w!kPAf%w2I9}r1f5RTEVuGH8P(q= z>H2ns>t15Ay0dc=4*}j%8(D<$2Utr)tU0faXx{s7x}ZPXDBDfXF{IfItS_zNd$Z2y z;~nXzj=e~(jflW56yXF1p6#J;PshQMpLDeZmPQos^5ybfvrP+>PZ&r*a4s7WIkRSx zp<63e%gxJ*Pf{)4Wx6ud`+RjU<9rvif#-?ZP0|g6!sNacZKMhuFBJ zMA-e4@}&9OJfw?XD2j>)1K5ZiQ+sxMW{@7#18k--HK9Dm>_AHL7nsd#axGf+ICXsA z^iM2eZpg=}WK~|7ei!&sfjLnZr!nU!k$@G4FEB&Y)14n1*iw zKB>f9$5eSPLZdmr!Oo2jc_(ZoJ_KB2gner%9M(opW%xR87QREzEUv(jl(az-%4@T* z19GCF$GuQCvu`hZM;gQ{CoDc}Fw!mqz0r4J_1NraQ(M0-DzC7yu`z3ql%5TLTB0vd z$Yz_zPxiB@fq<=+Z$E0EvAlmX1lVk7q@y-8kTr;v>yAC8NIzV*NVNpr%MrR4^kIRv zkf5v|c3_4OT%LtF0o0M`YSKwJXo>`F5j~)G#DRv?Cb?*Gc^2Ay``&|oW5&gorJup%_vbRh;@1c!G>$HOJ&_3>1ozT&Z z4Rx!t$X-~x0ll_m*G==Ew9L=x3PVw)b7lQ(7QjGAm|%HU*@ByU$NoRr5E>}NC@VjF zn8kiVj~o~9^?sRa+kqY{<1I*b@#>p1NioBtB39V7O#~FAf-j#rcAHc{N^7b~FHF|W zHV-klasmV*(rye1%Cx-SKnCRO*O9Xia#)%kwh z!q^#96Y=Yh75RIF2Rer7-6xKVHYqWX$b@)A>9zN<6{3G?ET_Id>Iw5lTNf2;5*Zfr zHZL<%$Yd@|H^cH-)e@qXGG8K%=F27K+b#OA0cHc^msA=N(F~Mew@40rMIGA_}Zv{<}0z!+<851}AEXn$Op9KY{=a(;E zt|%#;I=wHt%&cK$m;d`QkitTLC7vt-(pnR`Kgprm!ypbsz83)Lx#bg{@;%%@+-c@o zuKgl6>-$YJT0 zUq_O$o$MuG9{G8H81&R<*|Hr0IHgxwh`VyinU%J8RD5Z#9w&=W%N9;j)jxt#WKy{_;h%BvfLutDVNn5sNJE7$#Nhb}j|slo@U_%kbELFjqi`)w z#_kj-IXlehyMO4j@b}AvRg*z*i=T{rYOz95PWAX9(}@^*&@`>(PcaltRpNp*x_I~{ zCl1c>nu6+0A_(S_UdU1=W)>kxb!NBvpo`GlY~9<|c;KO>M9o);Nqr7Ny)Cj36wcPBvwIX!-psatk5mWR>roH}5-5KOr8EAF>b6i)c?< zNJmTje66*$NVt!I#~wG;Qv?`0>!xc?&M6m9{o4*H_QtC+k2@%ykA5|VTHsHy#*V98 zLa{>(_Ue?)D(SSeI@^cuKKwA?h=W2M*Up_gxz3+&Z`K7(Br+9kC~HJ?DQufnU{4x8 zXkI(t@L-*osdpEoB7)!3aFS)8f)UlL5YTz|(>X>RI+cC0JHyQDO*9yAay? z!W?|)SZ{wsS1u>WS(;BKXIIVA!FsHkMKrknx^ljBHeW>tKR%N?dJAaDR)hi_P{O`u ztH2srgnPoRyEu)@%gy(I1}zb!F0Re0@5^mLgMbJ*>?ho*P2tuMJu9=&Xm@eN`KkvI z7+$TEO$`K>BtEHL`*aH;tz5H4>>cQ@5`o1_CiLPCQsfm5o!(c9RGwSmYh~Ecb3SX5 zx(P~F#$nQX$MvuU^FIl1R9WpO@J?Kd_n(v7@IXb}yfN<8!A9O|>MANP>-!P3ujPaP zkE``6(AoY?eOLHSXfG>ij)hK8pQoWcXPL1l?#sHhd`06KHkqf&JBNOR^gS$r{GZ>^ zBt=2Mr);#?GlTKiUy#714C`y9`Gwat2nlW1ATS?KiC>stgCMTu+rdQs`5*2!wkCQh zVj`J|j z`knM~Cw2-^2&HK|%$kq4_D%5Lmk`U;DUy+G-v5*IDgtxlufL*0g@;Gz8=o23I<#hf z2^^m=RCFR}D3W7mslL9xnMxxgZG3@)kjjoT`qq+LLNtm7|$! zDt12SZ-H&^``r?V*hRBF(t>0~A8>jZveOKv0})FWX#Ws*hjxzerP_6?D`z4O`JM4vEocC) z)k({9@qcy&C4pS&{hsg=x^2A_%YDA9i0Z0}Wwrd6MP|oLeIOxG&Nuobfx>fp35S|m zoYvziP;;=}D)jU$lnP&W^5^>uBzNwn0Sv^`G<$E{Kmlp-!iSE+r+2Zm3K_i}4_5GP zcc|2$eE3-V^;p`HcB7%zV)o3Q1dql}jdiv#9;!-;JCIFq5^u;uG`E3XQC}TG~93)jMAQ zQm&(-A6RJBzcr!UWv=i*Pnyv_Mb*?RHMqT3vxL}M2NYbrZhSt!FxX5E4N~Aw9@Nk{+n;fz)yt~FYhA&b#z|CrqhUUz>*$Tg9Jjbvs_zt%j)vr&k zyE$Gfep6wti7XTFP>?+E#6CRd^*sIhwey048oatW&5!mR203Hf8v`i72g+TSF?^Y8 zU5kRU-B{ZjAl!&KoRSA`nk<(+wKyIl5%s|SObb>tgE;F`Q?kk+wteQ4m5@eL<^1V0 ze}Es{dEwpq?BaF^Tj-rb^}59ScRZX}_?}hv9%QZ|t9rg&jPPf#KaK@4tBOj__2~JVAdmDP!0a-FOv%v0 zJ*QIoOtu!;;K=1rBpj~;JBgxOSl zU`K*Tc#@_urvqAqF3JD}?HiuklX@=Zvp1wDmz>Ov5_5Q~f@;&7$6M(Mr$9C$TQ{(K zk7ig7)AMfP*A4*?)FPn<82zP{%-d4s`AA#<_B;Fjuz5QE(T46w4Xc~cOj-byO0(eG zpRVJ|(f1(@5wrdA&sx!Kix{i3XU`TajieO$iNi~d&ZPmx<$pr`rMQX^IQEYd$+ z6Y|%BT3avYB6u@60FSoHE4d5-`9_|r(DY|*#W_>q6=)2hVMGc9<9)4It|RApxqo&g zDTp9ZA>yxnGc2+pKQrz|ItqXsu-3yO2volb+nn&&M6HGa52>BvB<3p;ruO*SV34{K z+rRs4uYj~v^X6%&pSmC^_$y;ELw*$X=>d4@P&(%QFce1x~J&F_#IbWm_*&$iHufn zyN3M4$8TBgWDZ-GI!g1`+Efic^bt=}Pz}w9S3Pr@5rjs;M$WSz+JHK7`w%!;VPD>4 zZ5|nzpJky_fUGD*d{NCMVUtzI8cSRkM0w?*hI`x1<`96mPyA4l_w@F+WCh|2E4X}& zw-0RfO@)w+=I_e%{7%QuCGz$`Pc_9eX`GJhIXNZ2Q+m}cC)4p|!rZ0z!Q!@FH%DCg z?r`Q!-KW4$ap)i*S(c`XeM{5AtxGL&Ue(pfivNfV9$cE$wfSriDJ|WqXd7^0MpT!K z%RkWdd?g7=5kEz9AV~tpm%HT0BsuCN(+&^~aecwF(iUu!d81(Cn+b@vo5Oc1NDPud zB%OhWpc+UBm+?WcX^?Hz#Wd0!ABKcRAZbQ4z;w%n(@Ql-Bn}f;U{Z=TB$|QN%R_qb zf{+dKKz7$poK4Z-WbVuQ4c-4NJ!hJBZlst)LDIK3kfkn4M1A|Fl5V_0=Q{T_W0SCEm(DJx?wuQ+C{cgKC^{ZPVzM~@#bb`Lt51e=Zy?4#-$`+vCl4yY#6=WEvnqN1)M0s^{NsfzTj ztRN!2gpP_xZ$jt+6_Hg0lp;#+9U>+4VuL71FQElQq$NP41_*&~Uew*+_vf5FXLk)T zd7pV^?%cUE8l@`Be=YtX4C1pRKxof6Y+s!5U#H29vPLh~&)TDH?rOcFcO=|d zY!zT-;v|FDD&@RKet9%{8Pfm=-Kc2w;V_Nrc&oily?f<_C@+6)-)~AMX}biJp)vAR zXnq00wDw}KDpV~lo6mgi=}qYFzVN-nvFDE{6&Oqsb7AUy;)7viHZZ4Eao@8fTeuSW zbZ?w8-q0!Ds;R&J%fHW?xBEwv`6a7_SzUZ&vAxLkk zidS+!;AXMyIM=&%Q!XLM3=U@lZ#=rPn@*R^ycP!l{~ItmI7jyAWthf+b4@nVz!8{R5` z;5Q*^%4E*rhaD~2;{|adC zKLMF)zgPL~odH72#g|b2W-ro_AXJsj9@g}Cw#1byr&VC))}^Y*zgYIIXJkq*tv*dR z!rjg`w}2l1t)@u;gbKgL=!jp*AIk=3!_|b0&j5^)4Kp0LZkd>ivQCq#;2emTzUqge zS7D0Idq#W%^eVUC^7xkaSxK3AHd7|kMu$s$oMzv57IAZ@vL^&C#$2N9=lN0v&J3fM{q_%-xR^J-;I;7N!V+b*R^L;6df4ZUp3-k0|` z0S8G4(Qo2r-S+yslbxAiFnBb|s5wRy{b9d237T;kckO_Qp-7P`qwwmI?XJn70LQw9 zo^Rw*D4M|QLUgICTHA{0YTars^pt99BSm!YWqZ4xU$l5J#tI+Gl^TTK3 z;bz!t7&`#{-T9~?Pn2MEqQ`61cXx}YAg`MoW6fkEgRc*{Px+`4frGYOxhgh!V1+Ex zq-D2PvIYgfs0~NA8FYn-_guyKhEj;wGFK{)QAdo&S|J2ElKM?ycEu+MB`pt6$IP#M zP@;~kJAT;J3@xQFaLmFBBFx%Fzp#rFFj2;5zdW?~K00(8HZ2v@J|j*Sk2-0cx#8V7 zy0R^2_my`raE=Rfe#D_O3Lt|VzYM>$j5HA?~u)zt6}8b|8HZ#HWMYuii$N!0n`BVDnTXlFGKJrwY`|j zsg-lb)JzPa-QPa?pm5apcCS)LgI5qzt@s&MD+J8n%?PB*1oYH6)u<6nTtj*aRy!D5 zmPwr&7cu@;JAM404UeMpeg$oq){={}tL>VNA3*RXa9`PWWeLz$Ih0D9JuuvWm~6v* z4OLe2+#_ii{=wC>xi)dzA|6_Vt6b$fU|!4uBYoUsmm^{B40dGje5*PE8Gr=ck{!qh zqX*;2I}Xq6K(g=ldWlU;mF}Z}2RS?duGM0d4l?cD_Ek6+$nCO{H@1{^P}ZN@z_%Or zbu=Unf}X0tF^d~0u@uGfGF+*xnaPN=)4-W}fAbgUlPF4ow0R1*Ll~4gRa; z&wSKHImk=VwtYGbff3@PYPG;H(BZuIe4sPG3$P592L1LokhV)ZUzcB_Rg7QIdS^oa zB7=!4sny4$-d$WJoZmD7rahf$`;M#_7|Pe3VyUY3>Vg(>42Wpmxt?hsqh@Tw(e7N9 zByQ$4s^S#X0-=`%#mU2;qt)zjtz0Kf4^*DG)g)sVhNA8TrCi65qN9>Z;eZv|`4X*G z>BMUSu_9xk?7o{1u0PRe`t_Be$oLbmRq4JPdh zsgO3#b$=Q-5@0OXypL~VtZ+fIoe+3({ew$H#(qx>N#iv zx2gPnSr`5y*|yXe{s+Dj4;r&={crzPhra#Wl5^k7w4RXd-6f9ilpV&siLGO!e$l5d zD0r0QXS6i)ul=and)hlLLmYIL{h(m^(bC;DslyzY<|AF7~N=|I%K+1KRh% zebFsOp_;Xb4K9qN9sv;bkx#2)j`g>wQh`l+D^k-uJ~WvldvMW9+bwaCYTL>{GK*j| z&Kqbu_32gZpjXlMwMjzm=4~rO-Q9T!>0eqn^n(6t^V0l$2+Tcc9x(wtWA6Kht|c?i z{cjPsEMR=I*n{e#)xEm8nRp@fO2!4xKg^PL#_W08n6mo$#;JP)H-bCzyXC&~CThgw z8q0q$a@85Q$44FrB;TdA>(kSWZBz|Q^AAnMU~U+S2=TFg3_j9vtj$(|ZH4}>#ioI+ z;QP1xGP7mIWwNe^knLXb4HU7MN(%p^V5hF3uh4K~4^tE!}Q{ zyN#Is^F|`iJNB~l`&8cwcqgm4euTr%#{6NoQle&XLM5NV=F0nQf=AcJ#n{?7^fguJ zFcT%xmlApUe^|21NW`nhOS&rk0a5`bCSe#Vsj#;rS&?Bsm+&Jbj7LBDp}xUX(V<~{ zAS$~gZ=9}?nA@hz(CFq6c{QHojJ;*0{#xHLpI#H1{NgBm{;`-0=bq^Qag$v-uZyn) z?QLAD|2)5&qga(ee*cp)jqmD{V^JR54Sky})LKMm@yRLfUF>+Ono}-bCf}ko_*kL!^q9J~uh?J;oOI0Ly z2v2dFOQF@MOiSc(A^*A@il{^r-{ORS*UUykiI#&1|5J(6fn`e~+=SNX@HN zEOR-P-*m<+*f?V=(I0Kj&GfIF7fsIhJ8E1IW!y@(^Px+fwBmB(T}aaDRKm4u6cn{R zUnQlL}X8t2WkYdbci*HA-n$U7)a@ZCaYC zV5+~mLe5i^a!#2~_!9MpHl<~w$S5syn6KMVdB%3~*g08WJL!ankDC7UxcS?GWTnnO zWx|*mBxCULIPW8O9ZHI?yBsMnP>~xFl#yH84P2uO7s%noXbskbuh_Aku>b|Jx|f7XsTM@ zUJ~mXx`sZ3cpT@|{;JKSqcU^P9BVAJ<3n8~u9rw!1UC@9!eH{47M8upgOa~gYpI^u z7>6VK8lgf&XVVk-Y^LMA{igV&qRcfj3(FjalOwxc_s11pWmZ;+Xx<~BN^|$C^m?u* ztC(|uyn%6gJv3lCehEPEL{nO-74IU(W{Ju*%Qd8QU}!MZ`GTO`zr%jFy*&kA5jCvo zZzvGGVHw$N&Sjop>UJyTCnGQK%cy^Bus@yr21fN}!BG6EH8NJb>iF9aNtXi<1+4ql9A|dD*PYp4!pf#_ z1One#ajZH%p;4%9Sj-=m)AZ<9wa&BTM!l%$HelrzH{em;M+ixEx*)~kbj=?{V?M5( zOcYe{CiO)hIet9t!9}|0Nv}A9XI~YSo%Xob{wc?wxeR{jMAXD+4ebCazihbk?W+xT zp9Qu4#dksd9$%je3zymYjGqZ}Dcc-|#kjSKhJ8M#R8*d%E_SB1ll+^z_1twg-0*^Q z*-;D2JC1t)G-rH#^nzV}IrHh$XKfG9CcHRB^rGcI=b8wRr+ZzV6S4St;; zrA%Zdzt9id6~#yX$HSW5q@&K+f+b>}=qxkA)HSU7d+>n_e*NmQS7HcZB@Y_j62*Vt zk{E+KDF6D!Eb|M!>87!bLal?Qgt8I4*}-?_aJ=f9yw0SkFAGv+>zv%crur$T+83vy zUW#EuZ$Ue)R-zn+@n(vl_qWeyk+HS+mB#o6Iat_|eP$D~KDP7Z$`pK=>UyVWPU>TQ z>9zA|#GlINEjwCSSlDjm4Lsi{NHDJ)v{;WDl(Dp=Xz4F=ISej7FvEK7R0f{pRQ zp=GGOHX%R1XcPt>pLA?8bH+T1O~dJ49JVcpR;~Eqkm#0r7J|9tHy(q2K<-8J<>` zx0!pW3*+|gc6wb!Ki{JFDM}VtKks>$$lVQ#gy)xcgg6MTkFZ$C72I@$8yR0!v_v~C zl+)%&0HiE(=KMQ}ab3R$E-eRR)r#+?+1}88xI8PLrOe9T^Q_ONp>{UehU!(Gs1Yt^ zW#8Trjmm;w#B#@Eit@c=!vpVOePv#PD7SGm?EdnCYBYH@y$Q3sgxFdKA6r(~s&u1h z1AEksnBo@vgF^O+o!-`8%*c~>0r{3?AsIYL6u2tMfv)R6goFGBJxl8*RC>0L&Z4BmVr9bA{Fq-@OM zA9dBq*Zp$Uz4#^WBhj+P*UqYF$BH@poELHC2g_^}uVU%vQM|WXPK^PWl9=OKl~|gS zNYB-)O4i<_Bl|hnQI^y`oz-0p720R$gBEpJpBQW2eBU#(;hQMsKB<{2xo$}}4;iCw zRJNp!!}*Ai*2t0JQaXX|Rl@Q^viMTxu<^;j{lF`m!iNZ z;1hA3v_Y8`?nuGitrw05S4t)i_kbT#DjOvY!4&3gYHF|3B0qcBOcdA)m=`wk+>try zRXv;FSw)a+mogOatE^4blia0Fhk$`ECDRO_IlpO2G6Sy^kMeT~ENp4j@Sa|FCms6c zwCr~O%)g`0!E3;CzB&@mzNwleY!%4N8c7~-Ws$-Z%m=?oIO)fMK#9 zn3Sk}xSwQIIA&C_R9#z&>hjk#DP5ydgjCoiF7#~m58r$U(_|;XP)_STcBNGKr2MHI zmM1(&7Cep4C^RvR?hi&!br_x$i)ET|mwhfa%(o9*mR7EdiQKS{g0|(G9#m2vCJ&|n zd+&O5da7VgnNNQ4E(Wi_$He+YO6LBls6Q-xvH^-HGWzF7qva=JcL z?yg^$0y#I14si%98Af$Nhb6nHVH%X*n~Y>E3I*!Ww^ znPi5$yp{fSMu7pu)2Bl_;3L`*O?zw$t$Ut9ov8Y6usH3EW?gV3=fL3Bl&_!lRwF6?9>-p$g@^yC z5|wO5@G5vZIpbPwS1@bG&;42Yu8BAbHJ0-2wnIy=_KZvplax6>)ZNZUP_*&PoX<{p zI@E*JMz_T4>gw8?SbV+x!jv9KYYvmYG31|Of_Y!PDgQmTI6>ZQw)-_a5TXAbNXgCB ztAHNA<+&#oGoN3$`Pz}~j$V~m$l*UrS7_Oa3Zp^E2$C(iaqE_8$QWC3=aJjRQ$s$p z9lfzmITy1y05Cu}kNg>HSm3`P-s(@^pJZ>1mGDkuKE9?6r-V4TMm;;k`>J$2`aXxm zb~NcPeifd_2XgNXJilBBvo3U$#`P0TALKG!^{=#u_N)7Tm8W{(;F1YTdy~P&Y!&6S zihJ3cLb7*HbWN7Q6o)t)K`&>z==ecR*KIzOjR2zx+kE8<5^|2tHFodEtTcs!@71l& zMYnTL3v|xRC73>nA7e4vd_959q&@!gpc47_Q~=0QkLG+aOrDmp%BKzk7G0vR*dmTi zp);8CV17QgqwY99C>8uYMDWvkyX_K9L^MpLJcLUT1F*;1$+GUXCbSr+Uq7zr&}%rq&z+VtfGYRs zkNn*2{*ODL*{Og7xHX)(%=UKV5aw43i*_+|d&^nNu6dN5GnIere_$e-Oyy8@qkJkQ zIW7?D=HtDmC=;-dmJNpIE$jJxDaLVe);lmcl?EkJ6Ldu-P2*l$%f930qd z4}b6A8YIsA@j4a5k9j0{t z^79dN8k2=-gI}ibvhF(L}Ne*)OMx-h{Dc+1w(CrYXT6v~EAT8xUYSyh|K$t$OZ@ODy5dgGKp`cIW8FuGjX0m@`+l zAjeC041ivw$g=t`#|56L;rREMzyOCCx(H8pexK!zjNc;&;9GLyim%V13~iS*s+4?R4zCtf24RFG z$i9qiCt!6IPyo6nCaynH&SPjZW&ITkmmov=Ic~=&9SNJ)xH1>TQRK1cmxv#Qpp$Qc zD{ZjIWn!z6nvknoWC^);JXi97{P@C3D5uo5oaudo$seN3283qc`b^A}u{Qw%9>Z+{ zntC?I0ikS3^xBQALF%<3Evy5Wl&2es4okrzo{!>(_*D>_^4Po_HVPoiz|wgoq3W#n zw8upVSW2ftyRqEfT|5#q-WnwKSz_7d^LzIe5a0LCursP~Nj@)+UhGPvp%z747H}=)ai5X3MF6~ z&(!h1GOO@3D|3Sk$Us!0{^;-hqa9bU+0f2|JBdk!M^kc=ZW^)@cB>zr)Ju3H2#A39 z^AR*+2i4&Fq} zdqmb}vW?YzimGZRvW(lc*}G>Ip1kPtcR@w zGyYb4qrk+{>cvdN_YW!FLCF(^%R?IQhA`&tKO@aIgX|M;adi)4+3|O%E_zeJ@(epR zf%$M*|U=ybg-*z2$2w2IFik?h3#cTEyKOtDFs)9qmM20zM0iS12jTXGFxZtSzh5Q zt+MoTC8CH~xQMfYf0$%}Va1nML1b=^f>9#eAP*IamFfEq1&5ubOP$yT-J!3}FHw_= zeN5=7N#%3L${BytKKIXzJ4a?^l^Xy#$7N~|Qze|Klc|yTLU;8lajXHdL_j0E7l7P zzJ~!21f3D9klU1AEvtK-ns_Fp2NrcvaLq@Lr%fdInY`w9>UhyVubDt*Gktg+F$6Btj zK^C}4T`zTljQeb3534z+*!acUD;GvRlTN=`p`GV{<->Vj=y$92{WGP$^|r_nrhU{B z$k8D~>dkIyU25ehTot!FTG$~2ypO=E=(}#aDov&bqa561=9h*LG|bUcW9$A}pof~y zA~liKwkxyiVw=XyK(>lvxgw8%_vS(oJC%IwoN0IW@E6ss@n9Kmb}@X0Z;V_T(hYl@ zf$we%v5YV_bh+$Jo6#=E)W3L+0IZ&Ht!!;2zNofnxCa+9H_i&}x@%jB$^U#}+Ct6U zXlIpLsW=PwJs}Mv=d_ZEj_zzB1Z?LEfCOQ`r{y(5*hO zm}I!@NRR+5ZdBx%tlU=iWlCh0;neBS@NQTZW3o-*@V^5AM3TXJ;DvXiT~wAiC146- z$oThc-(pp3T>;U^p*T*dsV?9ra|aV>Y8ADD{DhE;`Z&8zk&`39#+Y*ICl`k@jj=}O z=67qHoS}E&Qq3c}zG{!>waupL1x1Zwi_TRaj(=vHs&iK3dU!2u7TIzo(6Cpic5C`p z5*wK)ZQ|}zcIE0JATTtq{i9a;HHkNj&s7S&w#cTUt)OrpU%Q-gn{(|O zA7-xm*3$HlDWGB&VqGE%bu=jT@2mCoTw(6~yu3Zx>kru{zDcyB*AId&4 zZRZQG3Ua*{6;-1QYw>_`E=6tH6)aaW&$_mL8YH<6f`cSf?F}KzD9?wo-6H1Fl+IUM zh4Bt^Q2T9lg0S?~c$zv)D(YYRoY}B;_^bx|f~$WWQQoJ(708VKQVgtg=g-R}q{^5g z#KHQy)2Yv%2XZUNaU*Ltbj3ZWtCc|=m>LgO^H|bGfuJ82pl4GmQ}7VS)yESE`71#sDafeo@U` z4rYeX>PK&r!M4HD1%vhcIHNm?R1&-#-OboQ(@m?re&+wBYuR$ z1<&d^9UWDJKZXj2D?i_BHtuFS#f`F7Tx;MZhlX+FfnZmkO}NI>7vBG<+}yLGBI?SZ z|GR1tz_vdmsW}ZGoavU@g%ety3N?jH^-bdpq{#^;BCS>{%Ok5{9F$ygO?D&2Z!_-D zJ>(4$cZZNaVPA_+0jkmGL**~AcVdS3lg*YYnSlL(v$Ht*vxSIg>u)D!~lqC&rKVxmv9jm{Z9%Xw3|fSOr=ru;DX) zJW4V?TE+t&dd86W0N-)nLWr8zUyF6x`PG6=X*QO;)_WXq+NS;6X+5*1$NJ-Lq}6n+ zE(h~VUhD)!)_$N$@!rU!vP4t+>XuJ~Mq}t+;Qf^kdXB7DEY04a{aDB65;sUOXrmHO z72GW&6p`ATv#~G_{|6ASZ4|{601Md1B%oxHXlBfKM7UI^og8b>0t&cHz@BG(v|Mnb zoDz%a2n`288|4)O)-g#q&}>&CY5Ekd0gmZyC7*=xE%!dPZ{+t(@#rqU%{-pyjm;O< zueS)8ry-nowz}otTGrbZPF9X6ik>-t8A*JC$6LD}eRp{j@JCXVotb?5uVU$7o+0h= zp?}=aFJ?Gcvc((*Cch>NE%gjg*pgczD=z^aD(p?l2d!_1L-(o zOz{K3(p(w<$mBU(Q(=oZe{TJ^i|X)pkRUYnF6!gV*UP!YC4mAx9mdUP?ED-;@T60$ zys*^x^^Rg8{!0ZA<1`&w2h+gI;z$>aSs~;vFe+3HC4Nmh0i5L|BxA}!Uy;PR8lX2wbH_%7$eXtOctDb_8uE@i+0z?mdBJZQ72kW^uGPz;l=qbHTJ_yA zVI;3c$%2P@h7rg77&+Za|K|LoD^i7Y^r-Dlb38zONJ*cDaoA6nInY9F@dA(+-I==r z9Oj*!cd7ydZEYJ*2^dzl^$Gz9gSuBhUFK|KoVozUE3lP{XSp{E_!wpGg_K&vmVK6Y zQGP5#k~&(++CO8ohG*d#h~97sY)=+D@1h$rGTi^EaVT}A{2k&}&YTMCgqMbGBP(-n zez4d?Jon0Ug`CiVmKevpf%&i*vk4c8>mnBJF>}zP{?Zy4@wp9zABK+Z8{xr`a|Y+h;Hym=qt3We%QJ{m;7sKE6Lz6ayJVlP&++Rp^Pddy!H;eO#Nc)6`^ zOpZG2mr7iFV?^S*>tN-;`A08yZxh+~i=;Anvf!5rKt%L6UuOMs$eSi>A-P&={7m6{ zJ(u^*jKVmGB8;U{$4_%{YGkUto^IFci5HN#_PLCByLy@B3L0u^<&EA-`?$xoG%yo# zzyr4y6lq(Y>y$l4;wsN?gsK3hppG{s8c$C7_Kh8%69#30kT!%eN$^>7Nu-2z-_~mU zUiFLs=n9youef1I95^0yKG@4Bk0@FL^FPas`I%06 zwZ<+$XX<)6{CQZ3y~#^COtjaRl#8jV+p&TAO_`~~NyUgg)4x5EB=g3JUD3ihCk$ES~5$8t` zb!COm0@Q|xsHHZJGgzo#;u{sDh6Ii#;0H`*Bp{;d?Rp{ zZdvz~eoQWHAk>8xThsLhWAVmly(>NR!SL+6{84@1Ycz!PIPDzrk}}=bKtIIfEU6MN zrkCBBK?)YC{WWr1VXwu%%Pha%W)1SG^4Oc}?r)yl%Ht*Peq0S<`L=ZvBpc&ec*EG( z<npzj?xD;|}U#4Or! zF~viO=ae^~xI`L~p8@lV(B^O(W@AP1jAiQAxL~@$0f#p+#HxXwmq4omv|JluP!2ii z#!96Z%yQ_>)b3~TtL^P`op?6(h(oEXyv|3Q1SNeV5r4-hn%O$_n~PVwY|Mzpv}JAs)_4S=FU}@Y%LM=y&6NOpWRuIX^0qx+I04>a@zK)-p9^=&l?( zUe{YcG6{q!*=-#XC%qnG?}Mg8Pb_kYwGczv)d;fxR0Hdbut+@<*#&$vP&LLu76p0u zrS~{6g1f}T{v_8B*qn`P%9M7W3fb=@evM&J5JqKB9=Ej{e!wj)Mx7q2dDoDy=3G}{ z3pIK1#p220 zdU|@&8SGDNt`Tl=N;qo*1nz+8zfM1w?zT)8WM~Z#qXKGHb zrC5Ka6@blcOSlJjG+!lGT>3eA>AwEh+hn0;!FNZ{`<2Wc0ZY$9`37GJmDt?R0a?t- zPY|g?$GAYA*zx1s;=PY@yGo|7)-Q3Vv5~sonOp2OLaTZBC-d0BRK^dPp6IOr~@;3+P6#epdu?&#?L_=3dJG-4=h%J?aA#dgubZ@FVCI|6PJyw zf&^Ucl}LB0;MaRs8E*Xyf+jC-uTeW6yA|hV)$Lm7AeH!Ac+mNPGFRD2UaVRW#;&u^s0kW@c%r0r+=$ zQ6JuZ`l=a>IFH%)n{{cMcH6sm$#cdzQpDS84^)sx(az5$lN;`DYxQD2-Zo737C{YM z67TjNh9IeQ-~Ho{TkH{$Z6`$eeZR3;NCN!>aUeX+TwX*2TRX2E;r6GXs}u~|Qaviv zh5it|y1cDWCg*W1rUw7Jc?I_v(uk+`xV7R7{(1=|gFRaPx%~j3zIZK=-ih2Z| z;}u-?Bx*kRUb#C&Xu>gf_91b+Pcyl*na%iF>kW86MBLEqwQ>>BJ3t~aG_+C+aKUb| zf>UNO9Z$ZcIGQs^4dOoD9V&Vuqwdhb4U%&x(4nXbEL7J3Ad7f1R$5VK8|G_w1ld+U z&TRFyL$H!#o6%Q+x~ytQW2;*4#j#&%)6iG#+>IWLQ|BpUepqTa;-Z=&isc)cGou?yVaVX=xb!D&}nHC({yqMQs^<#X(-+~|3 zedu!ZnikJC+u|!LogxdXT{;=M%`$0sn=eWw@xO}hxoP_P!~071CY;x!@5+juq3tA0tyLL3egSn|U2YX^d?@G7MFjaK0P5ySa^nW8 z*(qf&CF2nl!|2kdLmy;I5hxEL6SB(~RnQG}tq7s~R0ZMV{I z_!bc18fvbw5{{oja)NN)fGY5m3Es1(v2e|%){m`8U?2BOkanumLj-qtiP?!i zVy<+`sQ$u(S*eL373~J<@FDtwpJHaI(M$fK@N~s^)@}AOYlFJPVxU3IA3JXX=`g4o z-g?T~DcMGRx#&YRi;}GQfYA_)vEs)92LNC?%3>OL4HXMYSGhE$J=d;B)W89CRhN5O z1{7!5IsA}sM4^dw8M%|QDrb-YXWU9r>CTF6uzPp7)f00RaJa0@rpxG3KyikTFEfC> zOQi^*B8RPzMom0_{2b1Ij8+C^4y+LSU=`)MHW21YH)R3IL$uAi$4|xcFo$q!&n)Ey z13k%QzHC2Go7#w(xmpicq1_Wh{W!q!T`GpY@_9UG@k-$JCX)M??38{q$8Poj`mns; z>IRMSMX>$@%&Ix9lAfLp%%P{(NImP)`(Pnn8JWSKW6BW$SHpadhFre;a6nM4NX&^F zvBG_$3=?%Ux+~YnR|YFJ0^n6h#G1=9jjhYjAk>}< zH&W3Umsx)W-yR5RuV(^?oa~uvrTnhhT0_@oKJEY!5%yRt&qogufxnS-qU)uY)2%2f z=@i?@S8&+y3s7TgOwHcacI5Z)nj}L=iAXM&JV)T!?&RPM{SyPM1wQ2a_W{SbW2r?v zis=BlF}tlzULg3}g!l>)V zpQ&8GP+$d?o$hZkf=(E!@n&y=iYobd3R)cD{t2;iD{VZHCcs`Pu2li;q6U}-?}Kr{ zABIHzkbysFk;0taL2HDF^FUd~Mg~7EQ}BYudOX}JtG0A2g{iil`!kUe1px4Tr{ACu zL&qMu<3l{VCif$D1m4)sZk~+tSrCrQy6*W^>iA%l0R~m<%PXuXb zgZ#wtol(ZA>kk_@H$hJZy2Ar~n_t3D)z{Z=6+pLb6R`uuObB-|Sbk|g_qVFyGPKif z7v8$Jqsj}&DIE^S+dzT`wI^#Ny$8nGg^QZ`SS5#9r&ofV$@>Amm75jdrz&iL zoza}Q=znpJ8L6<0W>oQA;1EBh|L*eT#=)Xyo@0RY3=s8N*hPdQg!FjJeSevtQ5O$~ z!DhznD1sy=wIPh2t$aZoDy;bEqu_c(UY1!o=5$sgQNtt`p<)nq-|hF;#yDl~=6PTq z6q5YS#S7jc8jdmU0a3r{=9uR)Fcc9a9%7gu$q=*VS-x)mkn5t z9ZNbG+g$##pe~@Y+5=8scKo|H_Rqt-9~Ng9?sd<-#m?p~Sf%h@)pI+jOhiw8C0JWT zztpovId+@xt90_Ymr^H{HD`Ny;X8lszAqlry|7(;UtnNhf1Q8bDVxbRx0a4?txFg5 zrw+MByU5SN6=ab<(QgM_&Bt;SJkgOko_PYH&VgSD2#krj|51JY%%*QXD|DbSkIE5% zHyXR-Mr+$~DTG!4k`~{D53JO;lxf;#(*#C|O1r~Ka4($7){&7qyF)GH_hx(RQu}w8 z5yD0JIg&*4b;ZKzL!?~t`r!VHHL)-S7zPPQk3}4!U!xMxxLQNsY{A($<0$wah&_Y1 zk{z#wJO!&c2}v4=BeE}(WuPR1MdPxo3JBl2QD*bU%AlwSZv`c*$wSP{g?-gSqfl}= zkk%}2CNE*<9VI>rTn&5Jw#YP4E&90a2?ZtWs1X6#TPZ771^^Y5CyR^>n>Kx}APli- z^dE9?`}+m&@B{|78)IR56+$#-M{NJ-7L=5G%H{3Q3i^YkTK+;mV%}aTPSmr5MW;(U?Q=E+2h zVTrZfiFn{7dVbOc+3Hj^VxsVbpA+S_hgvz&%7K+bEb?skTa(|qYK7vvP@&DmR;Phe z=t2d#mVj);k}{MF7>xuV_}l1i!ZA{f~<`%g+J8M3Nn5yH@T|xoI3;_~hazhy@-x^!8c37UI#9j29u5+&d~n#jUhBQK0k39pCNdlKv(zVK80`h@EN5^-FJ9e~N zz;Xt}Ye3mkXe2%#Zat6^;o zt!UbcaTm~)qi8Lc!uEF`Sl$zXh4gP->FveS3j<~enHiE*a%BZ`q2D>91SAYVtE;ku zVUfxxJn%4Jaomot1a1L*^3D|#bKiKx8%CVHXoAXl!+! zzHgmxSa^d&oK9T%0(D^={h0;`H#n4szs{KfB68FF+iBD(yRr@sWm%4iJ$uPUGZmyY z4}wwsW11ahZr+G8oO>9fc#|u3AnasJ1T17E$}gaPmK?STTmerGep=j5#8HTMxl+CY z5G4S$Qr4h5rYB9Zi#VM1)Z=8PT;X|t=9p(7%{K4+_V}ZI6-Dn*jqP!#EAuN#7;R2X zj(>BIY0yX``Z)O9Tjg;J*aD4E?r7qVWEhbBD(#UG#XJ^cx{$q}P>aq^7Q}U>pD}jU z4DjG{50O4|K3RPUw2Ape7H@>~gVX!xU=#-~@kM2|6^^Yno_tBabNy!l*F_~U5H60a z1i99>)tn=G-mIc)Og5K1Y-)$rL-~)zpC=e9H`5v()6BIAztUZgQT&E3NaU`E%dM?& zb5ZR9H9R}$49uJ7qdP>!^6(Ptxp1R+CrBnj{1QJcrLI*X){uw;eI+4lX*=$J<|rs> zdtc_oW<9SnAKmFnr?0!wyY*)p(WJ$Sr@YkQ{Zy3hIa9lBN$HZ#U+0WXm7aYcJ|fLbDy~93Cx)^4qlP&GN3B z7ykoJkO*H>hsGT|L(@+ku!#D#a#Id@)8W>f^^|*@Li~iOVisP32&vuI)$OehniR7KSl@5Up?><{dRj$8mj3Ma?b|=MMP3+mrT+a(rG&_DAV3Odc24pm zQLgmsi7<;nbA)3CE-Jni34?%)IDF+m%@FN2_|M1(_UfZa$^XslPC68izV@1-sq4~5 zjlz+mP<)oSMSOG9KX4aKe03cuNjK(CEa7TWHUg5b^78U_oK3XvlrP>u>ft%2Rqkt} zO-(h+!B3vFS5X{niz;h3H+;s%#-^HtYQcAy!15-wh_k~M%VP+7xZm_1(vy%eMESyC zMa)DvT{IBn%U&iJ>1_v|O*!p+1Q^@wE{m$W~i#%WrPh`CKl{ zfBwnY5)1rhcTNT^LxYfCwz1;bgXi+OQB}NYNS3VNrr7z3vi(Ut+#DyV1Jq>CszEfV zWuV6AK+Sxn{OaKHg#4zVKgLcm3ZF43ef98J^~DMSbiWA8k?$xkdj*9g$2wWVnSrvp z?_OJ~LEfMGbI)I*7*W|Jvzo#Y2bHLkUm>rM3v;DxtM=G@$4f=(=_BjAcJ20dU#{88 z82RkV0J(W``ta5%96qDBA?^Sopc>Fs;$n0FpTWsTLctvJG(v@`+?r=B{-T0w_hSCf z%`ot+t?O$g3=0m(3F-K0RrIBsghK_++XC+etzISdK>_V=*6DQU1IY8E7r>&L#7Yke zOdIU~wOh|b3_(MM>yX3gfpA5o1Au_u*u_pAis(zE;3u3(@!9Lj<#z*clJ0fVjXt8( zn$l0sC%j~Fj0K)FhCa|R&vT%8S#kq3b~Qb+i*IjOc&$F|mJ2IpWwBFSQ&NcSf_Wc~{H1PrpLuwzSG`ngrFYX?JRGs>0+|hj~fl zLnFv*Oa11v4}AqXI~_{N4TTVN#9v;5o(Sk|LD#c90ER>x{5=pid>`-XK%xT)7R!;Y z%Uk8fz(9lz{j9U-sCyqC0L zMeSll;`+y*`RdmqQCaj;U{6KN&H|Bk^QHn0F9R(UPoF+T#2bJQMQ(NICejcY7vKUj z@?n@J2|^}B;*2B;*K;UI1wLiBvxVcPnYO74hZ;kP5h@L*RP6Kxb)bliXb<7FAO@&{ z+!qC*saQx}^M91;U0T07%gkBz&yoKr9vU6<(o`0a!Z>#B737=9eM0)yfX*@l=-W^VIWVa2(KBrl z*g{jFkD(rlh{+KkkUH>26<7?U)g{&EO|Z|l-qT1q)gH(V=#0QEhVeyKSjOW03JP`3 z&`l7jhPii=((iZ+E7f+wgJ+_A&xB?^sUK*|w`H|Op%Ngt-|Fg=T8R}a^JkI3&35ix z>o?Qde7E6Xop!Zs^J7lSnU=lVjA9~vGPh9V&n++aHNe0IE(y&{$-onQ)>4x>zZZsu zoq!0P9|mXx3H%jFByU{yJ7{OH{8wY_eRp~l1?Jzx>3mP_d2m7SL z|Cy%sM~Tx!CNzFPzvey6%TZk$LwbX>mO=92K+DREs=77wLUf(^Ou+keR+4*xYbaCA zT;DE;S@8vbr_U!FFc8C-qcXX&~6JwoG7ao?60`#k*p@U#E% z?+Ob1dzKD^KT+nB#+F50qwgp)kK-r2aB392FqrZ~eg8mU84x^jgXS3?9)wHH(~oan zFbYSRWAvu{F{(Ru?`xwZdiQTFGJ`i}-}m1=WPeqcnD}0LHQV@2BD1e4Vx(kIGn?0u z=#Sah`FLp3%t_+29aigQwekwwm_~=qY&`#Mx7)*US7c- zq>1p*L8Vco4VXYJCJnfbt?}ctJBrpqJhnPVnvQVH3e_3iy+`E~B9sa>E#hkYPFi}5 zJ~mh?zO4W7opoVHQx{rB~pa~O-R?21Q1Su3Hq#``n4*}`ZkcS&;^!3Ff+%35JP3FX$L+1ODT=Ci<~+SF|HM} zJR_g>OV{L0C1q<7%rl2-K3L)mU`BEI)DD#n2r z4&AcP1tdQ84ufRY)UYaK0XWI?8UFGi$1j>d5n8XM9{xbl+&3M`G*j{F);DkXZ>eIX z+00GCbo=NlsbOU*;RT%RnI{8wvrkBa{(fe&Ob2>zI{?P|_I#5Y2bF{cef*g~c-k+i zE1;qh+5!e?umv?1HDw|7Q32py6KW2h4+uVJoiqZ6Pg~~hfV=l>2L3E$6!0Mk zm3w<@5eoRZm^rW+r))m@?~Bl7drpVm;q+$XNpvmrn{Cpd-_>`wSt97&5xxVFfaoDb z3kY}qM$2)^n&p+=ysc&JugsyXr+WO13UI>(dp?i)hE8!EbkB!4-OTfB#>@gBs4R7H zROzcx&OU}#s6&NOKcQzqZI=s0C6JCi3I9`>jd17tKO>090-mc~HOTM&{xAEDiWDIXF4RyL7-xGjjPu~hsd4*ebz^I8J-U@kK3WA_itPJ0JR26T zwr}6=HW`dv{MA~DNDz^X{sqrtyAPEpQ2;KW;O{|d7*?FYM};o7Bm!v#5wfA$WQ!Vy z@6W6fdz6PTb?5<2?Syf>iCi+SR(XLqXJ9%Lo%&otux?thyst#?AI z0V4r!J-T6c48S-g*AZftKOCAKn1!ix4Om{x~4OuRG;bAojizVT*w^vN=u#L1KtxMbQ0B z0AOHs&9CA!kdA5ib`2&(J4B2HLSUsDP&Y;C_T`JiBLa4W078Zk3+2#b1!;vI*TX}( zWX&u*h`>I83=Vn;s3a+0_<-yl6S8o809o;jjM0+u4K#$EMhN23br4-3idRqc4R@xp z>hb6Dth*M_Kg0P}gdJ|3p8APHOU{AQzZU-feWm`uk?q#k|BZRO??Op5-(&rlFB)CU zDS5jb8sLJVzj5m@v38v{E4Qa8)f5)S8IYIFyp9#Lk%QU>N&>M$aDJy$>Qrbr)Jhk~ zC!z0@gXnIWd`HEo`1Wh0SITXd`&36oL}9fns+8b$`aDqGs8?R#cT;`lLs$c|h}c!6 z7O(uuX(;N_u4p9D=T=pSv--f-xMz~o9p?Ufp(B7gIr>$Ifyoqd<*Atg|4*Z@Ku9z+fsB5j8WDg-w5V{>O(QJBw*%o;4uq^O?;&vIa#&Lk$M-)M z3~Ond#Z?Byg`h#NzE(X;_|(k?)bEBmBu2LAI1sG;;s~L}*ad2qZZnONeW=|lQ=g`o zFy=8dZR)6?vwAjdF6z|(0g{g1W@i_Zx-s~!^yWlB|YeN#5$emw_NXy z6`GPTPj<|$zg;bk11XvHua3g73-=s;d4~ct;kJl5bV@FcYzWs>Redp_j(xrtQr@4! z*WX<;>U8yLU@hMZ=&F2PY_c|%cs=sCQ+LUuQYL#+A4FOz z5vh-Ly_spTI()+=1J>Kc=P%w$d-^n3_ThV^?G}37p>Xzy`Tvpi9q?4||Np8RP2Eb# z{&q7$sF0OWZX_9zk#H*6JB~e5X&9xDttgx9O%k%_u{k6T4oCLp|9TtM{r}G6(e1`L z=kxiz$LswX&*$@%C&BwG4`^_w=gVbeGLT))*osHt$OXP(F#oB{&PX#jqxS758d(pj z|JNfhF34R2<2ec&Y%Oc?)Q95fBB*=|1gh)bUU3m?X_Wlf+hhXa+NFx>gE7OkNj>@jL*3}RK zc#g_V)pt9HYC?TJjWlhK(Jsg!wFYa))IE#e46ioa zwtoBlw`7;4o=0Sm6>eL*6Zu#el4mxgmFu?M_A%Rdw$zEwzVnknXpWoHehrLgP)3<) ztXNls^a&rI%T@nn7k~|jb@?Y1L{*^FC74dy^QTqMtuC`~RBM!>To1|+8-h0zy~f=N zvXCBwQjr5z&<31m;$!iGe`Ju4j<`jXj7v%=$^aPC4Ae`f^A735ii4_m&i20TxC+~XeegzuCiWHo^vIv?ZEb5#NMMa8KdJgRMHO@NtpY?H1C@pyso z4q0#|QIZ?jwtPD)VNw%pxd+w*R4S(t)4I< zElEx$%*Xqb@g^#o2Q=8}7pxNi%W!5H^^$UeEXD9NZO zu3#%!k=z=3029G|!t9?qiDD_H}{%_r7K z`v|zQ)M_-5VGekc2!;Pyt-84k#%DS#eqhnAJ$&ujH7{p?X@3O53FG{=n3!J9kqWD! zA44PV8Hx1Q1(S34A|vp zc>xa{ihqp(%m7r00K|vwi>#t++pN|nO_5J`3jDBmV@ID2o-VV>6U82LZ9u36+_cwO z{;m_R16~BgN4G`l9cnzg zUOLH$Xv9>}nb@Brmv zym~SA-TSe&ptOV7Lx2_85qL&B7UDEV5Cd+;L$|117aJ~ZH8wx{?3X1^tgFZaW0ZnX zpL&EyO0W=6fD8c%M9y==Rg(~wh?diUUz#dvXfhauOCaivaYOBO;IKU$yj2IX1F04bK4z z#~1!T))D;tv^JzIq~pQO18#|tXe;nwyd9sw3TRE#9BO0{ga<7woOBi~w$23~U=?(B zL`lv-azH`~0}H@q>t1-Z3OuDeAe%4sOhh0a_rNoKl^&}ufS7~9bI={J(za@cjPr}* zYP}KH!NLK+hPDelXd>xZHeHxIlh&tp!fGz$W`LD344^$P$6$odfHQT5=*fuE1PJRe z&2AwfIl#@hxyLMmo`Q8p#BcO#z`*#1kWdMY#Fe-$K45*jkHj zcJJ)k@j2r%{%MpWntQFzhUqQYX)NVgiEqfs} z)U2^N0CCioL?#d?u>37BQo-j_fg`hpUk$MHfpAzsM#+n5EJDyetuwa=>Z=hu2#_K( zKO+w1jkY;(0GD9c?^Jxqw z4`_SBwj_}hIuc3qBnAVV%6Obm3h;1XMmc2CsE?^Rip5P;?^YvddPaRE)Gk0#R4vBA zX!L77s8glbLxFu(=wphru_HH-$fRwL+IHm6U-N_1RtgSVHdy+f!VRbgkY8hO%f@~F z87xwZXznXryWRvVCd2Tm*l61q)`cz0Jzg_@j82OsoZL87cW(+i_bKjOq7E~Pqpqvj z!r0MG{vDZ{29UEyYTHx|kV8bigByw?h@}s%37m7t0lP>`mh(m^OiJ0kiY;8ldEzX% z;Kd;<)lm!nA{usPThSRBB1m)q3DMw$q_{0 zdGH2gyG*g~xXUNJoizKbTV<5ma-!AL04s{oV!YDcjUr)`m9(oTfwxqr1DZkitTkxP z8$@<|c)xcw;d2}|Lc5`D?m0P4icTb-r8?c}4~`dS*0B4Bp6p&Wu8fu#7bms6aOF`U z3Ct#p40y;pJN=~S5lQoA^)*l?I5aH%ML|K#>N4ZaKfnV8Fh6GO2D}Pwv&fFU2M(O2 z{R^HN2}wZY)Cfk(%cl_1ZYYBY=?NoH!0}~T2T{9nodUpY$6pLwK3WDb=6p&b)U`Rz zWFXx%Sa^J2K3}<#=}3DE+m8wCbLdt+nA#D4fER;geV|!xDNDqz*xzNZ1~LRVabkZk z*DVzF+5+;&zyjonOOw3@jdBQji-&*|4IjP#i_c`9ibd>rl!{26&Q*rK{ z4;tbE#f5p6=VuelXS4Q0iT7hhswvi#smMm6+G?XdvtdQJ9s+$m{0^&%O*1Ndf>a&t zd>P6XF6>wZxIAqI;l~hSm}Uwi2#mC+))og?ntDL?hRQ;l=*x@1Md+j?u$WynNt!$L zKd+CU!?s9#WCmouH9<)rQ6nn+QFCmBjS)P=bNM&<-nsW57saZ%tDSS(?k*#~Slh2cA5}@&|wq3^8G|9mD3zZAo{um1zQ&Ibxc(K^(4!p^Uus zE2T(w@P8m81?i~PE;>9?pu3nC1h8C0_0;kArT#it3wL$fGovxqIMHDseSfDvF%f48 z8V7l+Y^qpNAKoAn+%V;;Z&|cVZBniapKdZSC2E;+&-<7}Ri5hn_8o80ue&ke4Qc9J z$B}f~sjnfgL)ak9fsDy0O3>f|w37)0ZDHSHIIX=05fFHMziSTMt#ou%VWcn%;^TIny@_w2EoD* z%3W35!55?ep4RGFABJp+DwtzPu8wLc3jWnaI(MLI%c8n22$W)NH!AU;>m;&}g-TJk9%+cOfFFq%BEhH0|VsC2PkBlV9mEW_Clf$ zM3Xxl;}fTW?acR&Wg=3(0bhV_Ds9(6pY5=Zj7+Wumcrc|s)aKVJFa8aG1%m z89izgx88TKuTxC5@EoUNF>SM|9~E_fO6EL;(c1Wqefx(Jc54Pt6Y7bd-FFGQtmJG) zP-OA1Adk2KYqu?SP3~L;g72zJ<%@EnBPo11)fq(9Ht-WFK9c~Y<{$kyxil;3OG^ZN zBM0mNC0t1lgp5dDa#GlFzE?JQ9wBfC(YUnPz~X{riT_QShdvB{_eiMG8_;)`ZRR2Y z?7$4NyCcc{ru=8BI9|9;kA5?roL7F}L{~W9a7^Po`^!6LN1?}RDIzp!%o4;yw4Qqo zRQPV(KI-rG0Zpky`knohQ+ulQ?wjEdXn$H3(+J!#s9~J}Lje;Zo05DhCT_m=Lf*=? zCd5)Gm!%hN8vdsM;$)P}ftdhz-wbJe0iRsYtKY>;1~k$1HXa(zO4;}^?v<4$HN<-b zydo#Kz|mET(=ac_xW#rY$gQ37LbSFhd3ac)`H9#TbIzcCs#h#0;hm^Qjc#+}6`b-p ztCln80AGfR>smSrWdDP(A`5;D?iO?9@LL4Bhrj|nF7?54C;^Nx9I)Ou$n@e|j`HiG zP=?+RWNJMze-ElApe1JuP|gvg9$2Y=5I!^8-Rw)88R3;;>xaM+I%@)r$*`A4EFW;? zdaAeCdORAHe&eyW?vN3H2TeSzz3)nea+iRP3-tai#x_(#H3rooo8=EW<%s9egtx~> z4Ol>$;fUFZcR-I4^TT_lW(Ea`I1q3JSSA%c)eoFyJ2mV=1UCl@h4kS-*`wMh)oz?H>NJ*o zTiqZ`crOLO5KC^&gQV0v)VFWku(b5Xw(70qweD!4-p);JUtW<*El@kX!YG#3=oCl` zs_xK>6#A`b1n;VCSdnJ(XGTS&FQuim+#4cg4G;euZa2N*>H7Jrpf@LKydgdS(s$+f zZLm9hhs7eXJvAyhcZe1{N_B7ZEg;b?D!gZV|3HHFv8ShTWqN&()|0@9uj~JYUAZ!ksF{Hr5EU~T0UYuS`)+zRe-)~ z88V3%jVTy$86|Ks>6rhxw%zOS4_kP{Vu$dHFP7N4RIq| zMOY0X=Kluu_4P{$lgOxmm4{3KU)4e~&$EpZcmM>&wA|%43MWs9F9nuH1#vIe0HCh4RPy5W*hufei(oMwYp`$Xqx$YzI^((Pq(r0 zWMo%S3f|?ykd!YL=ywT7$paC$QoSH&h~$Y8=>w8dbRPn0xVmVu$sq1@Hed@<BKp6{2)B?8LE>;fAqxDco;1b}n|?_31%XP{zX@>7`J z=g29qZmP?dO=g*(K-hQwI1IGqK)I|R_@nsK>y&%Q$P_V9y=hH!-q5pd6g{zxK!^KK zg*3uO#CSc#@Gm35{O=1}eB1Z-w1n28SK?F<7Ue^)nKvb;310s^O7@_|GY#S)E`!OFxw9}=Wx{yUnUE9G= zpOU>>OK9M!?=>qaW5*WG0WHrLgr1v$p>+)oGD0y0>{-4{ID-+T2o*g2W@dP0hZzrS z$1@-W832%oQ_P6EhImmA<6(mWX0C;!;ufS?pm;t4f?zt2u0b&ap-@s*HG!;!+tUi{ zzF5(AvU6VyQGXL+aK9dq}u| z432{6vfnv9i1QJg#W*5L+Lvwsc!*M{TEs{50&)T`7~3B1_8plPbp&wr5hP=wS9=u) z$opepxu>NS!7O;Qz(XOuF(^True=o%Z2yR_36T~&ff_SznJ+~)I6zqNHZKlCpYiee zf`{?w(y5_PVTDZfYv#haC5KZjA+XMYDFBM0f-*{*Y=UM4D2W_LJ`3fSl1t*`&b;Ei zdR|8%hlG$NkVU@egW{4eY~hU?${twi8rkXi(#+%wur7NXDwqebP|)%&Tx3Pl8g|h? z#XCd{32p)fnFC0Mh0CqRe$fB|@zMdQumNl#tRCObmXM-}*wNn;F~J+qQ}LHuH7-T3 zmq*5-uVh`pd1>YxC}!iV9zV)9?fU4?sS0_7`K2AIaMl8k!PF)WZGxe^d8<^J*KC|Z ziSA7A<3j+5+Ts_J6qn%4Qfi)L49?5Cq}MkmCXCTYjRHA0cbK-7HM= z$n!TvpCQAC@(#Y^*}lbB5Iz^&SlIT)pz#7q@mw1&@CB%aqWZ%*{Xnz~re8&0&ewu< zSTIxsC`TRf`MPgp7P){@jb)C@$bdc~WGf+M_ikLi{7}k=J`P&QEdi=F({@7p*fABG zagdZxm|{nr4i1`j=S=2-;h!b8DFnFUnRaUau`Hxlp+B3tjp&!gK@dO#9=0DdLQ)dj zAx|KsuIdLy9J??7_%Uj_fptE^jKh>R5N#`>q|TixP~9qcm=*X19ZIYKyVvf&BLn;~ z4MErBM%@M2Y>4pY7riB1jUE}!yq59N1EFuvDnYgf%C(~^*^Ytzz*vl%cWRh{@$M+*?2~+HVQu zTLkSnn6&Pj2IKAa1oUaQcAc9r*m&d22*AHUb`Djqb)3OtU0^VSZ6~_yRL{;tHbkevOZF;=r61FO8pD zEi~E>6T4Y@M&%nX?rOl?vB;df&{ZtLyWTF>af~~_W$Zk%7^*MI-q>$h7OvfY%)qz8 z5VHwwQPHfzJiV|Dn2`@8Kp|>ItangB|DOyn$q6i;sQ$IZCj_!grlsUP9Gojr=@WoT z7Bw>1gB`gx7ECBZE(a=b^tfyFcpL>~m*p*?02s-uFf^ih2VhZJdjJl16Ju!lNJ~XYkYQ)AB`N zHna|p-EN+>6!apYKa?wEYg^SyF-h zzz*eGDA*hlx9UHT@J4;0Zv)854Zd+6LUu!bd(0M#KdrO)B8769-Wb>jj427kx#jFy zCi?=LiE>A%nAGc_tif;C>%&ocf#2`0Mfn^fi*nPm_T=x~cD zL{S*=nOI2FwV>M1Qn9{MBJhFi#`pDKYuYL$!$3U{yapPvcjzJ01Pb&Mv=yB%p3j21 z8@d7nsouhHN$|g=m~u*LXESbw4PX`Mo~4W7yCPU_7L6!O7t`ho2L%>IP1CB9@PF3h`BDtsx5Cw}m z`pMD4w(|R96!ZG=0m~<|3=bl@%5remqr~iAfbk!CRDz`mx8qm`z^&8xF%zhonVQ#?y9J9yM{;BU_-|F2uLykAfCeT&Gap=$71PnAmhuo;4{N&g{%QDO6jzJe6ZK55Ycv z>(-!s7WnP$@*M$(qn?y=kOhMCPWeLbTBFohi1L#>O32*u7I8JpEHZ~KF098XM2MmI z1zrZ1ejh3j0rOhV4BwDZLXg^xBomp&3D@*3BMk5UJeUF?4%!Q28ndIg>M^B@)}vJK zjgxbwKwoI1#Sgs4ja+hDMF;-Hmj`H%)w$G)8t&5SRbE%!4Kxw_;-Js(2o3&;UY~Uz z{m%7m{)o(i=69la>qpR(@Xy>5bVj{k&~WDup$@W!FTX=`xFNmaXKWFhWC&;vHTZ`O zCv&^y!z7*z81>>2a?V0X$*@9N6^{ZM~yzCf6-|{~&0;wyD;i5}U^^8~k{nwyQ((w7yr)Tpo z@*VweQp8A6Q!|l8HoTGGICs?#5Ys3IL5N(?2MhFMJ#RkA>h(!<3AicodWMp^t95~PbXScGVA_dY)O({szl=M8P zY+w-E@uB43wLcP^Ff|RDfJc{$nCOrcL@}Nt!UjXilzHSCUKElPXK8-BQ}Av#jzrDW z%YFc@;OM^tA09)kahTb8cogjILA`j_(ccaayqoN2e_x3Q2S*yBrvY$oWAiFTApwu8 zj65<+#p}y0V-N29^XFn|IB<}bPu+w3pz5ZAjX|c^x|1(6swD!idxP2u|mYHXxzBEEaUh5`54FP(~4*;SW5o- zoppc~1DOsuj#biR1zMFxL6I;5vxFqdnMWGlNy4N2k(z=j;snmC z$V5Sug4iO^;4adO*jyK}b@>S0hY=nVlfm z#-3MXgSbgA&`s~h*ex*>-9e-~^-G(t;XLfctA5ZMjz6CY7LTwHU_@n_4hndIiZS0V z*>LC0tHP&f>9^LIg^cEoeuy-ok_A?#hkL^|5)) zG{Uxdj2xsDFe_j-B(g({+Dvb?ZY;MJG54PMtw zBTDkC$}w$MbLKS&V8%cWtEEF1aU7xWM;Fs01kX8 zG0tO223jF;N4Y zD$?&k10f4XZqgfw84=$IIy>BiR!-h^CB=^4_k!&Y(qasLMOeqLD|bp-&7`=lZtfQN zYFe*t@9y(pKXvijXi7Dk2F{c-s4fib&^OEox)T4i7b#1C4!xO85XzI-_B8}ni8!E? z#Kl%$Ist9Sqv@u?&QHZHfm{;wiGN4x$wT@jdVx9%5UtLXQDb5^GY}h3^ow2BaL0^6 zKwd_mCmPOqgTd#=kR^h#fu*GvNDc?qyQfe0tj@#;82>mZ(7>NV^R2qN`T6?d>n8{E zvu^F)%6oE4Z_T@e2jr9$A-{3t$PsA7fdu{;gss1gr8Qh%{aJd*VbQnRtIdk>8ZWF_ zIk#eGzAVnJ>^3*|UB~|Z`_6Vfz>R%tx4KCkg)aAzk^g)A6aq#6F!=`7ke3cV zH{kNy?O3Vh-g^39e@T>XTc2M)`oA949JmS_w7ty|jYRrS?A=_k#rW%5B1^<4#%IzN zmV>m8G?@HxNLR5F8mFu6(1AP$C{Hb9u%*f{(*9botOjtdDocXS2Y zb=8tVZ88ism+P_c26VT0di7tyyZ=*nLSmY}7cZ(MtgYy8FAzm|ryPB2IU%dfR-PeCZ3by7YV-I{U^ft}vbOwoq z*#^wDnaPs#Le+csMPaw?=EhrV8@p%8^4%;ce4QZ$@i!AEZ~4 z2X9+BvbV{|R2(6D3pED+jO~#y<)MdO-pzoEtcYzN zCp4J54lR*dPJLBa6MI+&`X|JakS;@56|GJW&``Bal1S%GSKnVh>Y3U2>GVDSoQpqY zHf|T)^5#o!2v))BxdGJ2&n{71<>tHY=0RF2vm2hDNAnl;7%WEbm5*{Z1ES&gk zvC)^=b-CdB1c^JrrXZctaIkg%(jD_4=m;2_Uwdy(cjxHiKV}2gBU(ZmsxX_u7~fN* zqOlx7{l>oG)}R>P|7rxI4F}MI-#gx9Y7!;Bgk`{| z_|E&rDK%yMn1ku^?p5nZzE6S0^^|22vJ$^?xNsJ?V(<>KyYW=RJ|d)>$dw%=W8Bbv zLBX}bSrMx$)L$5c^9#(K?(3e-`I*0Ox7H1ssU&%lz~EBeC(T`@T%a&8VHHuwurjOu zKq6^$p}*jbW1c4A8V6s2Z<+qj69%MhxAHSzg)dWEJSds~(yggb`QuwHN;lRocrmx3 zAAn3j0b`Fv>FLcmZP-&6%9)}<(3U|=d?o2j6;FwJ2~swY|8Myyf6!H7x~;*gyD{Q1 zPv%qOokj5PdI1}Aa8x%g>-FR@Q_l7IncRJ*y=otCtR^rXGD!Y?8~XL(xVo?iSdDJ1 z<#@4;xrNYv=JJ)ylbW{Bc^3LP8r{h$Bh1M&tYD}AC(f|=QPa^kP)(%umS=`NCJ5j{ zoF6@l>q*61Ko;yO4*EOdL<+_j$+@{+XEAa#*LoyQ&7WfS3MegJliXTA@E1-5bbnAd zI?d#aY8~k1Z36wZhs;mMsR-HulN`EvM^;&Uf(D)u?=N$qZZV>gMwFvNpgd8#Tm@pl z@w>`Edt!FjbezxhW{6P*NcsRXJ$KjnhR|OiYxr8Zk5IbZ@UCGpnf2@ZiM}+w4ED~s zFraHz|I%SeirHc@@C2xR^((FIft~ozz&ZfK4q99rs=rYIfrc3nUNBHgo=}_yD$MMj z$*31+HU20!ZafWHpC7_ZcdZvFCis-VXg(|LG*Ew&S$?j1t!Jf8?Nj!)0zx=P8)`xN)}24e>|h1zl-SrVC@FYJvQXgXhMsckedV4DK%eT}M5#*p%|yoLWd@;|BR zKNmDv@H^x{zGjvSzIf~1yLX#Oo`nqFnCnDa?FOX##JUOhi`5E=m&^bO;&HLjmS z^8W@EX=Rg~GNPcXxcKaZ*^LzWwazdC^kl`^a)U^cCWbn|_)ZRV{dBVBCLJ|u437-< zsN)7~{wCHfnsP^zR%zFTGF=LzfB)QLs+;_5m8zPRtA2b#G2$#pe#T9Y@Y|HM?bTT@{TUMb+;p{>K|GOD6O0rJMeII2?u1MlBZ2*hdk^ ztoGk0*--^Ch$zu78s5LzoXE%;k4$2ki0REOc&m!-LGHD8mv2c|8I4^on77NFf_B-~ z88}YnGctEeHl-3SswBe^_y2F1!~mv>^o0b~HT$o(65ROikZ)V?vvLqkZ8)R4oBj&A z0qA(SX@efqdKCEafw=iPeWTQP=S_wK+*W`Uz7Zo&)yjRJxNmxCA4NViVvlpRec^ze zY9n0)MrVS{7>HnChct2T2I7I|2Da=mdC`~uIDl?aZK*oa$Psp!9Yrn|yxfF^t&%le zYXmU~X1NGLejz@|;nB^WC)t}5GrX=p(L3-yep=k8_^#srlus16KmVuiqnn*frP9$? zi5b_G(=+&$1vvB9s*G^l0l7h$J*`!$TItrt>9{jp=OtyhlnXFjKi&s?_om69ZKPRP z?SF9ZElu(lw}NnAY2UhA-Qb&<=^1TZ`V8e8wj_35t@v#Y= z)$7R>2M^BSuwhOa(~3pseFh<{%1sMF3xAGdw9KjG z*qA5o;T|P6((YM|7Q->2O>M8E=eV43$+;Jl5sA1uaoJMYQ@;AMr(Gw(23{#tMDcv&1nRe-X z*E+TGa}SGh1pkaC!uYXORHpbMIi4D)8>y+Oz1$QQUyM;|rFRHZUolMC9sMqPNH;`H zDG+1Znf()|dzVyN;5TEe?S5g5=XOJmVtK05YUt~UbJk<}Q#WE-S+5ApGZe&adaP%pIm zgUupW3&&UKb6j#peG>zgg-kv_jPd=?v5U3oviNb^Si@2$tS=wUUBY|Xh2>a`iRfS$ zr)bR>`{x5K|89GQef*H;14)($TK;`gu~t>bNnWskG}KU$a%%q1vx*_xj{{#K;V)u6{POt90FzNq zUMA@TZ+X|?#$=YX>O=_Pxo=!@XC!9JWzD%!OR_n-1Wi-dB_^;gVtIm>1M26RM-H?q%_cV9?~ z)3cwpybd4BJTUnx#g}05v)nrWg03JHt>#otykA;Ha%q6ms=cB_CKFxkFC_qUGy1b~ z*OZ)M%p>h_I|n;JL9Lp` zaMhPG({L=x=5Emx6p12#9+IZlSoe@D4qIK#%>bVyCuh3|;@0-a)00>k6FvpGgm{`? z3r)5+cjvf>U&^W0otftS39Re;$*7sq2VP2w4^kOi6~Oo!GYH;_Je5nCd1zW?e|idc z^wF5@kTSha|Ly}J9t0SnSf2u3lo zMrX0p@-$x848Cf~=B(kYwrBIS{&*9tx+=l&l;p##T1v$adLGqx?0=J1@s25dx`$N! zPgS(L)>Gy0-@kBfN$SA&t8ibqpc>(l^<~SVbANi@-}ikA3|^T{{?r(0KE%n{n&>}^ z<7s^-lu$p?AVYZUB6P|*z5d+=Z|4^78a0=q!TtLP1nS|6C_+AY{!!W=QEZ+I3C{ap zXY-u0DEVV6Z9EX!VAg76sScD;T?G9m&e3b$Y?fk`E3Ax`qp>M0UJI=Ywd!^4yYKGv zL%2n(MvbecNH>1`>&SFT-|ONWo}cb(v(}Pr4(I)Otq-&u-!=i8-wc~I7Y^>zT^fqt z-(u>h-dJW%=SeRiz*@Su({ysZo+K!+L88ggqg7>USZ45+yV3H|Gf^Ur;nU`HcO>gk0rUk-mQ_|_HH0% z3j5u^%WU%up6;E$V*3KUc}Q+fK5-8g z)_o3_Q}ni**f3jUk+Bh6yMw20go|}ssQNe&d`fywt$EJ(}7K`*=iMUleQ8(!{bJ4Ab#z#q8&E?w}D}=%u;k!gbpDTx7u1^lvB% zS+<9KdJHo*dt(PzR-H7%MRZ@U&EamL zJug0&NTB*@%6PvCBlVY2En|LwnOF=jL9?v@M!BcAT4DBjUCAA37xw`3)aG+1u;jl% ztYnIV@}Yw7>eZk*W=}pm5S=jl{2lHgFn>YgunU#XwJuv-s)0P#@}R+PV)s4d?8dw8dw7V0>YSmWkt3+ zhjns=Z^T(2tCOQy9d!o8xv;#MCUT?B=+xi~rb^#uyYmWd$U|-isiF13S(T^PD}Kc+*1X4$_^!noY^1dGB-^ z$zs@BfM8VFG{{(S0De4=7wI*Xn1iKxkWNU|?@efnbn{0sy_Pp=dP%Z<*|*#39P~zY zz4e0>W63}m22^ncFtA%0(D&T@4j9;PvrM~_uX$aAR5-T_jvtUq545O z?fAzp#9D&QTk|!m|4zSEuMm>BkaHrNKr-xMc|i!YTiszR{3L|WS=}wbR4nGrJ@;Pa z3x=)F(?=N%o1Sux#@+3kaPK{?_gdp^7-qs}WoJ^yK!ewyy_v7BSznaC*_@iDU5B`0 zLoT*bN7BRjhwT2lE&nU>$Dnf>z2XB(&D7Qt^5fQVJ#VIP)Zo#PDeky+E2x*dBn+ zEUGitIHlLlIQhYu;~(_zkCaHW8TWU0u(ZzB%uxI5q#CrpyfoUDogI9^a7;Q!8 zJ&QMO;WRHWr6)cKkv76T_dRyvL|-b2TH8U(@OD-lYy;YJCZ9T`GFm5(vhkaj{PV%I zL(j``1lLi{Z_t_XEZ|IdN=>3iFyA7LPK{sLwD zk)vH2jXSxc^Q~E%G+q73n_Bx`3c9+H>2bnLL*|+h%>%Kw#=mR>$zwt5l+)}tGS|oC z`Q-M+c=JGJt;6G*()W2W*aP$aDZWw4~xpE`->abtT$ct$nQVTD>-AYZ|ocONTV2F)omEW+aUz?MGq6xbLw;1t~5CX zh4A~2TwkHuXX!$X9;n$qP+}5YBuIN@!Uc($f#Gf0bQfrL^v!sk zVW<>~|I{!v#T(S!I!qFZqLQ{-qC&3x~It#Vw#sTB*h6)#tV6(6jh&S2rhyrx*Bt8bg1zM*a!PQx@7-42XDM9rnB zSMHLnlc@Tl*J+$(U@|hqGeO0Uj?2Y1Jj+~88GacgNt>X6H|Cr2vE6YdPCj#?M&>nl+5|oqO~U_jHO9IBg?b(^Xu>UF|O<~ zqYvyKKIuK5_SGUIfBHe&xsJQpoY+|o=Ntof=-;J8yl~j4@w}$#si?5g#UBHQG9B9} zqsV2JEE;#3F4!b78%{OXi!eA=f1Jqd@5+?tw;gM_62>!;Ov&)Qo7|0;caF{-Um|JF zH5p8mZLvnr6)G+b-^JfoesWa&&Px!6sGfD$;g)v~96G<~moLkExp%|j&TMHK%s;Se>knL-+kAfP``KFg4BuT*wbv4xM<*X&smEpBE>#Ve>S}= zS9YSPbg?sHT#Wu=bVx)wORL%)=QhgEFIpp-^Ns6s!`k993G>4~9ySbF$72;Jz6Zv3 z*gf7l!LFz$9sl4`sPqCK!XsNJ6SxXv;@`dQkZol<-u)5wr~Lscwz+)ObC#2Sb++%g zXdS%0U$j{%h>@(X!~40G7j}I}Ub|*>+^LW12)A%{g-T)O3Z6+wRPFTgx4WWt^YB&jo3Ze75#pxNrSfO$)6E$q;-zkJ(L7VKwp;oOxaEJ$|^9+>xG|i8=~g30c2~9uEO(`#)eHIkyZm zo{?Ey($@MuxiphkwPz2t&5zmp-{f{(BoR-vm5fYI4zN`*WN1y-I_NCV-S!MTan`Jv zwY$A8Ro$zunDppvE#rKUhX>WY3v_K~M3&W`X}xiOZg8THzFV$iKvCnPWa-h-k*O@! zASYKm{`^-pWwWWaq3fPQAHq2;d%o$3Elo#~UAZug?Z_s+l&oJ7Q3$|U>JCvd)hmeRGlV|IkqE7+qrY%@NR2qS#z0)_{7p};yrRH>O|xNx;b(sAc3lV@UD zSzmP`?Y_pI%S%&>I3iMA=VBTVJssvOf*l#6dY%bZ8}m`Fu|}=eS_Gg+YTb;t{leF# z*M-JI7W$a^O{jkG5!P9K+KK~Qb^m&4dLaa6>(jtbH0{ulg~KTc*@Ytupopdl%cUwd z#$vUrVnrr(eg94Bu;vTrHB39DgpwvFJtstP!gs%y`b9fC+R$9;h53!upC~G?KiglW z4L(F*mwaPD=xeFI?CQb2O9SosUZ2hH+=(PP|Gd#--}di#ch;@xp_N+p`vu_2X2Ks` zO2b4tibKj%nY)hBwq7PvY!h8^U5}xlKU8(IPTml9xEphaiGJvpLeo zX7TCPyx2l4t5+^Ame8y#Yuz<&JNoh6DeLBj!}8NoUbIos#>u>lT_zV|8m3;?*YXMf z!=_~wrjfNz;9=4FqDc^%P(p;&&5Xkft7cm9VwxGBNAix@oD7Ke$4ArD1DTliXV zKX5P(66~&Hw-nZ9T@DQ`f#6warp72E;{l1%MDpzufA_)xEUWdngiN=2%zkk$ZOc&M3|VGrwN zkTFxfLzApC*B=qaC_evVAUM5mm(KW?mzIWbD*9X3Vz1>534 z5EyxUct@wp=Xc4bTI<~B2GolfmkB6EVbMX|Q6ufjNEP`_k0e>x(duMc`QUVYvJyWe zdZnk{skfZj&QkU9qjI#b_*yKi2i#kUI&Tk8!{Mi0JvO6U&xE2=Nv=QkVu$}$duL$* zhl5tQabweBFx4SuYL$!hXJm3!Uo;N{)r)I=U;9E?b|ruI+Jn8<1Pq%Jy9or6UDB5> zK~XchV@Z_?f#QEPU*OOfdRZ=&tqTWa&-r0mZ0^NN3?kSYgzEgjWx^3KLjngFKPudKoJq%?i`KE5=h9v{gF<23 z=)*yzn8eR#-rCX?c*&2?tmbs4QRDlX0g-a7g#-P)3!D}A*r%N$^HAo1U0~7J>Z2rB zqujBGKnHou*e7Gnh$SPMDaY70NH!gz%nIXbsE!~QVpEg9J|+<{7>~Rqp~v3)p`g&< z1KHs>%8_Zqt*z0J;iir9F) zIOTAnaxhy&NE{*Dse8{}YDRP7kT=@jlUBK4#bj6Pg*QXViCWi~ZU(m`)fZy7v;K|X zzWx0sh33Za?H?F!j1qR7JJXL8>ohIONi(e3qLO8BSz_^V zQ-e7Gq1fwJ2}x*T#;9D8Gb1|vc_7#P>8dQxn( zshnB-wL(z&F5Gs_V-Qgx#yc}*J36GTM?M@3o6~X_`7qj+647O^Ul-aK6X6mMxU2jg ze2%3KqC*T=iZ2bHN~zJxHVcyvw`Q^yahywha)?QL+=>M;F7{DU+T@$BuQ6ph&WCdZ z(J?S2wYRHhFHlLqJE{2;Ulj<&YllXyc8lH ztj1sMGB1uh^YG$drY~%8{@u3h+#H|f+U_T?R16Av79POI5v01l4Ra7<=^zboe8nnA zZ0)CWlGb!)wdFonZZ3@{oFlAn^fP6Fhh2Y#}BfS7lM(Nq5MBSl{V46ks!P zBWO9G)2@CX3I19N=B{@JD;wI{E9!{-@$D{vPOsMn^EZpQMsO-0b16IDZPh*wIxcUS z;|!bijxDJ|V%IJBj8L5KVUyKyf!nzR0n?`TcKP~m9N2=Me){WYddqV17LMg-Cx6S` z#jgTo4+asHz9#qVyOF--Y^`edC_IOvLveBUc$Vd8DB=7!(2Q`%D@c-I#W3m8SGQpT zB5stQYpW|BA{Pth`&5+t)Mgzo!=^wWE?av}RsB?p@ZR~s%ygB?dT+^{7L%1ESl95< z{b3=QxvkR zMujY8%W|kBvhVv=l8L0TBs(FyL-u_uLbC6UeLs%<*q8tFqM7-=|97sdX)Zb2@AtmX z`#kr3KljsF#X_fza)U#MAeH!;{1LM-M?KiIgc@QE%vJY))j8}TeXisakdQ6dFyD`G zee86MCAQ!6koU#FSqV=4?o&+s)Jve;#0t=RK6 zLx_>yyq2jZ{Q*Keqi^?q1WY*4)x`eR`zl`6XZ5`-q0E6f+H>wL022Y$qdW=g=DMDI zr(4F7BL`P=)b>-Mma|=QlpA4Vw{}m#wR-CUQT$|ZZx}>!lwgj9I(GiZ7pr_{>`Wi{ zK9Jk*tb5zf7~y~AU1xo^XG7XtWovef+7NK#zRCdCTNy*ex|k?0S^ zq3LY5l!EDu^`$9u`yRf}j@Oe0nS6X$#73#9KX)3|WT|Ng9?gEnFv2)2vXP;XW>=re zG?ExCzy9Ic#84kg#I*s9da=H34~70ET*laiRBhcrE2?eBZzFl-7KaZVRh-@arP7Q^ zh!g(7s4sYw+7@&4e98QvWgMAR)EDP={f4@V)~K7I-KHk6Ht9tkzpGt1XBC2xk;F}C zISXJ1dx|9`CtKI`ao3->ET}UnQUger(NZ} z00>#3;kU@=!8s5AW_>on7Gba${~HExb&f-a1inKTePfh366^>6CT9&hwsMMiqtNkd zp#93$VP^3D{apWsXr_HjsaC+aGykTwN5x2CrT1x_Utf-KJ&5Uk)i=eRW|%&zznJ^l z)3E#2eq6yVNhgPbB3arlKvn@I)8uMD%&VSpYvfpc_-9E@n=YEp?<(nBP9wT&PLzH) zALDV5^nR4Fw`f(}G&G}J$fqE`&9~4PL4JV!a}ch!_#d|4qvoU1Up|8sGV?f<`8<@J zeb$0_4|Rv z!GipP`C(PYo=O5aFvYOq*19ctw0EkZMqRofX=0vrzo}JCO$sb z3aR@9MNs=#1cpli=085nfRJ_rYNe>}i<@ipy0xK#9q*r%e^BQoUUE@t{#3UJ*w(<+ zdd8*)Dcyhi6EsYUbz++7<+fM1ucF`3dq5HENOIS-HBL;&89mAD4P7sWGEHjQrAWba ziCD6!bWDt|^#i}|5+_amlFHKLSwJ<;UfXmzS;g@C!=%8-eAs`rcx9|)x2DI>271sf59&KyaHO60UT0zBGyBD_ zx%O&~@bV$$B#CW&fx1kHd!kjyAs-gazlSq>%aV^u@`ws6XgnI*0wzG8>9+|pjmC6$ zPnX;$Jr}y=i{lgF>9(v4g?CHSb2fDQ5Jket9v@ZtmU+TFd6fb@Dp)772l%zp7T9~MbM^T9 z;WZqVwjx)qjU9VgS@o428ssNVtNO;Ug*(dD-*vh>`fx03ipnEAXzL>|2e6Gi^_tZ- z{Z1wgkKGeKy0O$_!)r9Dtz>?&h3RzFJjp_~7|1B<-VTAg-7!(PaQYLG-V@e<_o@sE z0#?h(UCHW)Gl=A0&iN%)r^~8F*lF3s+cLsCGr_-fhIWAvrI5!KIx8`b&kz(fm0!dQ7%mtlZ z=e7TGChoXV6ksHrHhSl-CcB?l#~QrUl~!{ZO)@!6u$gkqYc5U+;kOS}=`|1lXhybd zxEy!=lJOMwWEiteo^V-0?L0|Rgf(n?ZLuouZZjZoI!~tp&yB^T%1uVW{i4f)8Y&mT ze55ISd%{g~)l2~Rp;)!l!HJGgQST?yfB){?aml8w%~+S?-yVUjNiSAyYsRO?I_i?7 zov7s#pn?E`ShMUr5!hFR#mF^e?LR1Bb-}v5(|K~}n51tlKMTN^n>t<8IenUEs);#F zjAG8;C+J+lL{v&P=e?@54FMB@l@Qv!{%YcimQr~@YLtCa>&LG%KNKb>aW$4dBvk*w z5M=ZAmjE_-HB~}lh9a@an?H2>ay>}X6b%iNb87eLzl7f8(~kz=BU=%Au_w>>5a-l? zCDU;9Jm`qnxOeK)^7kK`xaa6cjc^uU{w32FU_L_7LwDQ)wLFCIGGUBxJ`xd@TgSmAaqsZ`O6ON z6BDuXz2y_37rGA2Ux0sydO8!vk2R$)t#Oxd!n@4aD#dAZ^XERTR)z6y<``&(l=^yAk+)Eo19B9c_X=u9m4e=O#NNs9|hO* znvYHtTG&&gpr)&T-W%q@sFpiz7*t+~KbutS$1hL)Q*9Ob=E(k%5#Y4qUvfI+QC3g~ z12*VXfKh_0X?ghTQD*xjzF54q7=#nF>IU=$v8U)uTHAx>oo9IT1?-Nd%c-z zb!3B@k|*Fc1l2lK`RvpN$klPTeVCyfYIEJUU-&ILE|ZEi2KdrA!?yds0l}2#`0dkW z`HkKU8{vcIJV(~KV27W?)pJ&M0%1u-p8M7?JnMYV;G{{G5yP#8`z0(4lYSdV2GLXf^ zyy67tnI~ZAgUh2ro5B<~D+5-a?jvjEWMjL{S+a2-{`jU{miKrG#H8-FW^&`ud3vll zMMbfM`-eJv=dLG#R0Y0^vzsTWdlTI??S_mb$11WV_LoW21<$=noow8E5nd5=MVEC` zGcclhm|6`%&S^ya(9Gq^TVPR8J-BM`IGG_;|BdH|Q}z9JZNQ@K2lCj$@;M6@x=nQM z@VX$N4H2k0m#6ilLXFmx0?n_f|frtBSGg8 zSDa4jNwpe(QryJDq$6k?EuKeto)+n9devQi7*##AA;{cG#Ep;s`tn}UPaw0@020cC zIhTBCZwCG)YYxr!w0rNKez(U$M{g59N0Q|k&YOD(3d8jeA3d$hjZV=|nqh$!Kv4}c zpyb8vtw?-<^|<)ce0#Pfh5BA65oaYkkZ1%i_#DgB?tY3EB9s$fE58yvlk7Pm(rpPk zq`uYBEnpO5@0e0lzpGJL+l5Z^l^=lyRoQj7L;#ddfOG5|)Dq_yFNVP^f(_HWrKzrg zMaPEi7j@vyw#laBM#cQ?KJZ|OghG^9L~(n(`i)~iWE%#Rv@t6A0F)CPbKUKr{miBW zre0{u+Nd}4Izp+9sX zT^H+sH-Hl)3G^{G4q^u*pZP-SyeZ!~Eh{$sA(5C#hMI z8l!Ld)4gTl8@^>V(Q+?=P~7_5P3&}88wff8j7icqBqtFZvbtT@RVe)9P zK^oh?r*{j_m#Tzl6H5!fk0BvfMH@F~HnbLwWIie>y36GJQ_sD`Xlgx7J1*8uT(e9n zT0C$L-xVF6YHZ?L-5z&$=s87oTx4XV;pXRk$G}1*`q`DI6Mx@kJS@WDswy5jF3aNXo}Mh4Djwe)wQX5(UB&T^Jn#1ug%wSpS_tG8iOE()>NXzkHQI z;-p(=vLWNsbRw|G{yqr7Ow#+siWnIv>9~sLx6mxHsrM- zsYIgU%-}iBC-Vyy4qU*PDe?F5+?NUAyOMgdvFqIFeWN#g#2j&r_#8djfx)qj(Vpe7 zRALCODCT`ic`Iz!W`Tks%&dBR%sLmm-b`EUWQ7Dp&%tsPXTS3%xge4AjX`1p|bCZ115ZX6hb1Q?LdS{b7`naQOZo2|LRImA5af^b zzJ`NiX~*=DtnfQz@MpKLDLbu^F4^2(`~0h?mp=8PgGW9rdU}iQhefmVhS7xWqGO_J zQ5M&1&(u!L(WsED?HS=HgyNCDObw7=@b~y`MAr^4Wz);s7`tu;iAU&m7N}%V} zYcD|^zkgNJ!Uyer9#GU!Q(>h0SH4U*7UgV(E-_)c?V}x9RjN$s8%x*sBDMUN#^~Dt zetyv=C7a3>CZ~4?F}Yq8U_*4g#OA3IhY}%+={HhnM#a$YS&JM2aDIOfLeV0I4KDq{`m& zm9@jtwECccme9j@sZ*_S8e^eOE4rH-%~3q=ZeCFR3p!s^@6NH0)4uGQDDi-f%r()% zz^BHlmV_INp8ng}(=Zr%9++w}0)@B!ReSHswk=Be3)6AIycYy4r=A)A4idzz41B6w z&inh{w|jug0~RPr{tkWokv$%P{c(FL);H!LB}U28G85840N~_rCa6xt=!CmhGEAxxu?3EGyj@lth7NNi2l9V>2K zm-X?{(UoSxw?7DN)#tEbf(U*n>*;IjPk@lyxMBIoH3 zh8`==2BBp!_lVaYqhXxyTtwQ>7!AED&QJC8;qe(UsETS5TKcrpkg<$)6>)#7?!Lk< z+B3Yg;%?1st~ZdMA*jQ-si79^WNBi)DRm&`dE#Ejm+K#&Gkos&cIA|O2i02cw*wn9 z289&76qF?7fe`r@U^(cB)wmI@b9h5kR43SXT?+^LpHXT>E@o?)V(Z9ms-nXxOHD6a zFwKa*$~JSmui`RtA{2YgMK`JM@dJXs$Rd_<7YkM4_QDm+jT_Gz-&zdLwseRU~ zV2%-3M{r(U{#@I|34><6>3I{|hT3BCO19RLpoWcm^X4d%n6)DGk+;K=emZrGib{E5 zwCGgURZ}GDA6gERe;eNSY;pf z-^(`14}KC^c;mb=S4APpFD9mC)Rp;n^^+#Vg{`RBF8+PnKP)U|@&;O@o^!A~1+;6I ziRayWH`bE3I+~r@A8E0$NbvF}DE(#fn#q*{SY!@|DSev3XpGU_`6~kCE)+MbvNOh5 z_-B_HkBXWJYR#mG#W{IMA7g{whF5GY3OA^^@3!16+?^K0KG-D9Ams}YZV{GA+ylhavtzzvU!GjY5~w4y z)VMHFy^xmyKh_lExQ%^6^~^AI2Ciq<6~*O?0Rey)9hEZ&q0p3{Pu5D& zD(3oQNqvUXDZfQmj@M{9Y5ekaPb@UREiY_4o{}%fp$dRqV8XN?o1Feo<&4O5iQRa^ zk$R^`qh6lg-YH8%Z7=gYsF<2En!`DGS%X6iTA%M%tsGK_lRe!HQgx0wTP5C{*V&H#(xe*pNZjcTK~Zj_2aEuyw66U=B`u$P_^yECNG z!E-3*AbE(@7!>cnlndpUkRB|AV}(M0$p?B2pqp+Ef5VJUjbAckLf&6G(-SwtD@y_e zr(sJ!C`as?4{ic4?HltU*Vg@27CR}F#n~`X0UKvtZzCWIM9I?<1G+`w&p*|(W*V+TsRnC^x02VW z>CdP~k9=dmiAOM{F;<#3ZU}5U;?+h^iS-RyfHFg(y1pLQHFm14CtbU+DneqWSH-}k zV~awxeZX}5n9GXW#)j0}3#>~-0yWjPbA?SJLPRF%Sa96h)58@*zy+Qm2CrXOcu-tr z3L_tgQh43=aZOSych9i(UA@^=s!}n=>Prf`eVKPXwnDh93X5W*UuPy-UnobfcsC10N66d^e?PESl|69Ln3TdU{7$L1_^6Q`vyEM_0H?yDn(c^YP#h@|7DE zYPm1HP(6T>>~tHM7Pxyr`u{KxnH#nY+NfDBIkAH>ax)i>$%5_!)TFnB!U9=k98G-z ztsLA)XAaT}^@H8oQE=W~cr3)^^n`iO9$S0ggUW0qu@h1hek$fq!v^H<3%bqjH8v8d znecD(Xl;$bqI$6=2?sNK^6w7epFz`?T8=T6Xpyw8hW(c3Zd^nH`iSeQ^M-9)6@TyL zv?c@l8|>&j9)aGtDpNlw37M;)kaSs}Oh(OgEDmiJ48UaP94jI zQ=8rblGab~WOPyE&ddhmN-QU0IKyT>b;p@}p36-RdwH`$O>eeA-TPe{BrDRh4+L4p zxsH;DY;0WBEh9&`#JM)bjnMtS?B|^;10@ZvMA~M!=}YBT10WJ=nb>cU617U4W6R6L zB@Z0{Lztp+hKOSurzj7%Uy&U1)Ooec^rPM1XWr7l$38#52nfPmQOHot7Z~J)f%sh9Ni(;L&oz{du3K9w-;G@0EC*adWG)pU(@j9~z<6$fuwM^==9(uLH+Wi5TuM zZx-EcYzE9{@ZE-b?I>r~;Jl0+R?8bd3 z34Z=OrjztjNd&>wagVdBI+J9Dn-7a+yN_gUzPnix%AsDzRVPZ{9#^dPYURDrFY*<8 z?P!sIQOxm%qAw|b8qNvOp&lDf>$aK|f6sGeiLjWU@ztidhvV*)aWCX03aPD+g;fgI zO+a7FrTO^=7$`IYZ2eEPJMim4`>9G)IaONAb1*sh!GdyHJ5N}d)9%VJ@JM%_!&FWG z=zN;Nu|8Yhm2Q;5Isq6%;I(69QakHE_-jeHPpbEFb0XGSf{)bt_GY+SQJi5@GL3Gl zNK?#jDlG~X;_cH##iF7}YtZKHl*L;M{YVs2!p;@qETlEJ2rBjYto1lY0Vz7sR5=lB zZ<`aXc4YQXgQ}gIDGpg;AXP;yI4r{~t%nJOR}iFCX0r=_$9pnHr)|xLFMKuJk+(ue zLW1+=S}!K9=0S?`0djJOf}Uex4+h zpAG!$jB5`o@=jC%d6l6Y%9UP=!}<6KW?oLP($+GnGe>Q*0p}ybEpO6_wO(1ZUpFVC z{Rb1Zl|eDH{5x%H{Jpc#-x*0i*9%6=;u2SldmeD7r{^#CzJ{hB=^`u@bbgB8Ef$?* z7>a|GVQqFD>Al&_HsESQTj|x^yN(C~`P}CvKK9trI75Tt>?Z>Lx;EG`_<~MCD+#K{ z%N}Zk{tK&tj>P4jvl5T1@9x{$Ef%lo-zbBfzYkibu8O7?pAfosm2D(hwo{5(>=G%o zxZ_cKt|+OM=Zn)f{;M4-?N;-R$8OIDfQrR#X@SYDj?%H^Y<_r--0ArH|CB^PpQ!=0 zzC(c~1@0C_;D)!r>@a0!5(tgl>775m1&1U-x5BsjC-c?<;$bK8Ju$SmejN{SqH;8^|@^wYLK;8!~FM`C&<;o|Pim zaN&vFP=0JEZqn9);dhzD;ELcx< zMZ-URn>Mvwrsy?~D^fGX+fl&0N#6gfGg-)( z<=$iu+>`Mkorq_sc}ycGHLR68)orc==)WYFkfE!e;yq?c)GJ=~uuEcw&dWI8`q_(i ze!|gpIj^K}%B<+o&4UwL4pl-2eA-j{A_1H`UDoBm{Nc$IC;f3}`!z2Np?JQs`v`Xk zP%qs^54+&Cfrs%nP+1f#6kbq;4!5O^ENt#8YR`^qS@1m&0MgcK4x#SwM(YB-MCBlpA86gxv%K~#hvEog@I#pp?%E+zU<@F|_QcpRE7tY+ zt&DlX5%)LGKL%f2Y=u2+F!sC0sZ^mP1-^D}uM&~CxL32mH}GKN;(D9P5i!964BWM5O++_ZXh|ll<-L1xGhoLUIsH_w9 ze#|XOBKgw3gF}y7pz91Z`9aHIJN6_?V;oBf4jLYzYU;RlYp#^=7V8jtBg@98q~hAN zQH4ocw?$toTu9W#gnl^Uh?d|N!#3z1YX#Bc+2aNoVT}wXb-Zm{T^zR1Fh%N*RzPq2 z9=Fo;tgli={f+5tvme{(-@CV}U&GK??@x6fblw;-G4o0kKsqAu&KnLj=X~jL;5dZsxV_I9-$LqQV z#;M)hptv2I4~?is+fjB*XZH-lXoIZl(HqXA+vnLxW)8`yi;5etIg7mTB8R0}TV@!z z3;d-yL$b9^7=_S&9DLFuh{&+*w^UrJ@vcKdk26iD4N-61UTY=9-V6I4`K$ZADz;w{ zbO8sN-Aev^>~sbJqHH0SzMmJ;89H+d3#+~_uL1Ld_#RfQ36jrJb=Zwp4ZnHQ3+uVg z1t@#bZq8hqJ9HTSY3CtNC|sO0#5)sAVHaRi-x)vZuv+}4Uehx?g;XXBM}%5{cx9iP z45x>ArrlII;loI7B|DeSTt$7*_eus;E`Eli$I#|lrs3+CT7k2hLeB7Kz2^LY?lCXT zXlN3IW)w&>=<3mzv>)-#k=9r*EsLm8yxI8b(g@u4Sa+{&rRjx(J^wjdvrMipKWss8 z(j8w$Dvzgi_KkT7&IT^&DIZ57EaT{NCf`^ zW-?a?SP0(+O!azu@_b{~XHLr@^Y9j1IpKL%*pyKa26S|EIR5yfyc=BJFMvOI{XZTP zEkrL|vx3TRt-c0_8kVE4n8KHaH^4!(%Y7-q zd6Eb7go^6^_VAyi>CT%uJ))Wdr@T<7weQ*zCdv z)iap|ku%UhD;26mCxtQLGALHnN`>A(>s#9!<9%vt)nl;VSp!6R3bfmqoeHO}q)16x z<%bVdzPW$bqqe3MW_ro2;rnHYx8jYTyrvg*=h34~waPspURgW7dnoDx8Dy|dxLz;(R>!Um!Hq&Pz zMod&2eZ4?HNrWUpMwnPtcr>LeWo|iXfGOG5{ z-G0aYoX2azG>ElYKZ~7{(+g9c`_Ou(HyWY4L)(^tAEUcb8om37SAc+Z6_kERKwHC$ z1Cj&|b-06)XyJn}>mGa`tBSg_H1Nh>k{l)>d}rzqsCIO>1yy~IlC@!vzzPMuA^fJd z8F1n3kXan-bICzDHL@fKCwGQsB+AY$yCye)%IuqPGp*<_I49|pdOBt}MzqrG=J%&a zWr5o=dJFM*@=}JTLOW*8e!TdwCw!B9E25z#FD5xI?R}1>7|YxDF%1n*E9amk-2g6- zEH*G2f^Mur;r+H`-qc!kT)p{ZBHZ2iBfQ}RNJ-b&*fhTd9wv)CimsH9_0&7oEN?yH zPVrsyz5WT0;GrivI{#i1GShtlY!B6&df2u%{(41u`pK`l6gSgh^P>u}?TxNRey6CI z(dWjjoXZ#0!C!%a1SSn{s^IqBI>QeMYI$CUm6Ff&NMbu9V~pyRaPDG}bUi2Bk6;W` zwZA>a9K}R{s|7UNBY~%xvpb2EjnoLA`hG$Dogq4zkgL{$qmcRUyy^m+KFD0dd#ISS zb`jRVwYr4yock)Hx`)Z)uL1f0uGN0)Dc?2rcwbH+bQN)YDe89Ms=xP;cUuj*i^R~r ztRciV&UfqO-J~)1LinJ47zg5!wZ)-jk<`pJasnO>*h$>Vf3ihd^V^hpC3t zosh_k_vZD&(VptkfP3q2?<`S~D^WmC{%h_lON#o8aXac`;M%T#2_n%^$HxHGw(zB> zWzPPbVxsi4v=sCcFic}-X=09m6#)io##zwo{zJ`Qx30X@!g`4tB5Z|@<|FZIa2Sm~ ztd|)|925p0yPtOXJKsh>61q3q!8!+A*3660IMS?OUVfEj;meY~s0*P8D%fbkEEp6_ zKw2*4djdBN0Lwly&42tqH7%fFX>+}gCCM&hG~aGq-Xw5;WH1=PkdgUbCBz2qzNi|& z4Cok!16&2d`lLnO83+Y`w+1I=a&@zY%`gi?NoE4WMQ4A}GTHUe5tlIsFYP#&{qbsS zvY`!QoZvh;jfa>d&O2DNGpnG~eGY_gNVoyn**O#>ykRyvcl2VH^p!WM5&B?`n*$FBgOR+U!GC-_ciw=E4D$=9nk27-OB{ux1jJyiQdPK}{M7 zHY67yGanK}EnJEIibG~bN2*Vy6zTfBJaXWr0pb66Jy*eay|k>W5LmBT&QijTRn4-A z7X{Eo$cCs?l}m+$^!UKT@c}>4J@h%mDHp5RBJTM5`uVj+Uf&P50(z9Z9zZC8f+#FJ zL(Kx6uE2EeU=Nci?YeoEos<+tsjyTNlZ*EZNm3vfg4MVGe}-&974Up?Ym-S5K`xKA zwRa9qGR$Mt9M{!S)iW&(TdMr)HcvifJ!C|`9UG4%iyn7YVg&>Qpe1HrG3Z&!Sjsy$2 z3_O+z-gxsH+4~qcR^UrEjnAEgYT+DVJO2obPWLb113UucrTLWP(kM5dwtMIc@TSJ3 z8q&?9Pfbnr{)Y{BF;vH0=>~Fn`a-Z8ZyM)8wTrO<%L`36$ero=%#(}pX$d`Jl>RUG zcHhpYLH{X@M!8L?A$&iO1KL?FDMUfk@@jI8KD;Tz!`KTR7kfm2?G9 z<{WjNGPT51!*BqEo)OlU{a6pc0K1|0T-!Y##0) zhyOHpK}Y9AvN|&O0KAotIY+}PwO@A<#)M$03Jn1BRjec^^Qxe4GQjLPyuIMSN34gD-d3mH%) zb6ym43;xyo$cez)t|cYNa%hN6HnHqnU@mcr0g${7&5}pakPX;t%s{V1fs=DXse!j~ z3CpGR@5DUnW#p7Y4Ad1RWM{*Y-*Zbq*k`!y@N;vlcInCg3_3vZ3%C&y;zvMpvFm@S zxBeEt#w`MkM$z9blJoK^mjF<)Wo(aYSE5NxN@Fks5PYY-?yyPg=eq__9orrq%YJrp zYu!%OLK<5Or83?Pl9?WG&O%3Gho~cKL3#Tx?Em9}$nXDJS!rMJ@$A0mdWW6FWJ3;yX?25C zN8So@r!0U^0ojy>^h$I^FNnF=PrZ#QeokOf$PCn;iCa%#!s>wX#=wunM- z+d*Mr?;#ixwB_;-AC?{vleOEQix?BwN8HTdu+WHr;Aei7zA>(@WMFsFa^=~>1OwQ| z0l6g~_YK4H0Mulb$D2sk9>o)Q?D9X%ZGZp6T~6}n=5wXjg|Yxai_*Sr0LlXtdG76{ zDRGTph%cNP&0qsa*PZcHI6BGR5;waBecn9Y6x<=TO4{9j@x^L#_TO0e=S2GXnvtVI zi_HV(-X+7n;@S?&y)43Sg`5YUxEn~RX0&z4j`wEWC+I$9n$4|;1=S7bUSK|OCks3& z6pA$eG?f`c(>E#viRYRLhR4$41brl zmfO(Y@r98G)QpT%2sI}&{jD2R;Njt0AG@7^p$h#a+jntH@)#sdk#`M_(c7jp+Svl7 z1mj+F4rY-JmdcFF?BDR6kW0eQif^WAqsL@3;X*-{VoO?4JU@H{m=&6?IeO6s~O0vvrh<;kgx$#)UXqh|Kf%G9bu{{ zDQ{Y(DwYsBXalY@(YL)XmlqhF?YdUCd?g4vimT#rrW)Wa1_cfKrwSd!NNK0D!>(+8 z`rSMjostOSf-+Ql8gQYj0MDezsSaFSf=k0trelNkh9HSj=Nc=cv3ff!%1F@3uC5i8 zqZy$z+RrWj(?phiFxp4732vg^B9r!2NUb5!)K~-oBxzVd&aU7nS?inKB&(H!136Yu z(p`Ip97|mTp(^7rbAeIdz|MR^neoeM?1(N;9uqALSxRo@&UUNa0{+5jY36HJ!A!}T zbMq-k3apIGwU7}O1Pv&tIMUyuB2uu@y5R@j!F%k8&yX(9M6*wy3S=$!+nthtUE66T zfHF1Ta#GLoFalptiUCCt38f7aLA zxXH-_^4|+wyQ4xHPlZ@y)2Ulm$D0kK3r#y`ba|dZsOAMEnYrBuh*My1QRkuAT{zDG z-&LAst9!bSn5}f}2&AX|G}2yMGHi013hSaM`-LRO0zhb6NB>^oTzJ>?C0~ zle!1YGbD+L27$KF+%C0;m{xYE=*lcx=aovh?L1lEAH1VU-Krd56X<+C;(}N>{$3;) zK7*w;X(?e3Dwoy#u=Uu-%g}BvL56J_VNW=Aw+o}y^;$eFsKndzU2#Z9N*;kRFx8_<|ZFMhGgE*lifr_m7lzJ~loLBr8Ze zTQj3T<1wo=^h&9Dgg-l05-ivCMq~^AZKyx8-%4q>10DmZglz$Jq-;l#Lr%Eu2DjB@ zi)wR(OE1(S@#j(H+2YRwbl~YDw7vZhB~t~A{U3KLOkP=VUIaS` zer6UiY~+yGwc(jDdC<<4zi=AX%{SfGd!Euo3-sKMo2INHwL=)jiGL6#wa<-(&>B1vB@4PBRjhQhzs9~y^arD`q5n^UZbB2x`u zbwoRC>wKO?f^U7Z?OHtsSM3u5$ma4Td{`1L+Szy|VMg)Ln_5XV^jER_EPU`5{yv^A*(q2Ct z{q1RN-yi7C1qczP#3tpP_g#O~|2V@KFDSA+l>u_eFsME!U;ZD??662c=z0el&w;Kd z?S4wr#)OB%0Md}f+Z%h}%IdpP-k8yNoH88L+krsrM;I!pH|(?5IT!MIzMZN7E4 zBW63DtmQ77TzE^RcOPf50vjaOErw56#6LSwa0p$qLJJzgc=spCuf~3G z_xZ)HWpY$t3fzo$L7m;FmJ01ey2x8z#+q~LZ4u@d+Un4SSN@%5Msq_!_BmRQ4U+l) z5OnY~*U7OlB6@dM-wzCm!to(ie`wvSlc5P_=xk9D8@YDDacZ|W_W8+Ga0d2uDe{a! zOc^{TKgo%y59|~OfR$127*oqfgKzG==u*!*FyjPqlA@k5v|7xUkg>!GIp>QV?VUN~jVn0|dx46BBwwMQ{3FXe_~ z){y+y{gq##>i}FDS|QkCC>Fp0RxUteZ#I1-U#`ud6Cy<5fl=F;$13hkYbp8FEi=yW z{z1VXeupqY(4+VLGrIHBuUCH0ETBgQ_ktALrAWJnK@DD1?hJKay!4hwThF#Ql?z zMbD$&opYCUqv0IQu|s2>u|0hk+G5+=t8N={HWuY3lbmw;qO8O^0&L#?TwT{pCjB{( zk2@8U;P9TPENgcTUCR*&jJ!q<=HAT?NLh}18>)%q)K$sRjxP|cnFJ;IAC-nI&1~O` zaGZY)e>d*Q3np`k-8s+Jmy86hCU-F>2%9{+%iiGS0Ttmhp&+Y+WkUEt(88Gy49}x@ z454(d;~8r8bY%W-=STfi1pP4n-@g4*`&X^WsgcH;oMB`!J0%deq`&`oxM;%OS^mYpzBY|wNoR- zD|PWDt#BDurf(dv8A3wub^Yl-G}!++UZ$vC{C9IlKH0rINmmUX-wMro0`PHBcHX=JuC6l*62eog@t~3+4;0@i2gvZe6e zG~G=(nHcQn%iES6|O{r+v z)k(?|UV5xd#)W!%SjvTBIy{!(7|LSMP67upP{Lcgg59`d7KX;1EQ?b2VF9iSI= z%!RUh@{N|urc#n}%6+f2%T{uCW+6f+9^Mj~^$@H1F z?M3ye*~3078E(M~fv3>0ZsLQ6T+qA{g*4{gnJIPt_sEF~LC-=@%$}$ho7s(*{U4Pw z)FRO4*P(8VG0*wTCaY4|Y`lMaVc$sbxW9~4r*`|2o8`%W#y3DEow~P@6C^8Y@HNw7 zYI#(pE{AyS)=&7egYh3q6tc5Mtpxw`95N-Z-HaeMQrhtX?>6XhN5Yb9J@9G$HB2Rf z!=14+OG5w0Nzy{Iw-+V&l6r_!aso6nsRZ5mgvGg}TIk(D*KQ|=@OKA<8HNvlH_`6= z0Z_CSKtn5<`_Kg$-wvFNwszLpM)!JmY5)(FPwOup>g<61pD45@J zUSq#=sc<8I4wah#;2=)gHwFnM0hIK&FM5A*QxR7<2O?8s1>L{DAO;bFA+*>E?n~IN~81F0_>rv>+ z2Eb}p8L?lScL;G+tP}VH6qd97*Rj1sb4t3)4Mfl1+F)ku(=8`gucb&J{-aT%bKp zMX}WQT4A-f_$ZC}pufM_V)A3PwZt(U&_sbE9X%#LAZJhLuxpKZcJgN;!GyxQ#;?G| z$TC^66mJ_N=4|d7VBX+CfxUj6(%aN1sy{N4hb68M$(qxd9`hAg9x6_}ZG-JAi9nqH zZ*7}MH39^@kHHV%kN{l`kKj{aG=^_B&K=ful?r!5$ z)SVjpZ0SfnOH+z@&X;wnF@hTrqj4Etm_RYhEZpE~V5Mk`ID{_t&feAAN%Kq;dP;9I zsVQ$+Ta5vX#F@<3k?^${ETfU(LMfuM^Uw;1l8T9Ax zwj(dangA=XB8_K$4OQk0j#jgQ41*BKYJdGCJzL@z7M}MQ;q+bRgn3CQ$44|9w zrhwldSx*CPgm!@(E=h{9d!gp#iT&jO5n}GhwDt5AEH6#{EkC&eUz)xo!ew{UJg+76Z!3nB8|V6>5fK4uNgr80a-{0%t!^F@ZGx#f(u`VHKNLDw+#-Qi zR0#7?hH#?&HK;jYgM)gca%N(XrC${-J?oip5zM#Ma-lY5o-;j!&oy|eOKL7Zx&tG> zpFcLQQj+VnwZxKp1||g4APknrRdA?X%cV6@DXHa;i+AxSrM4fQcJu9kbfQlqrHaymY5k<2h zl|XP{7U3xc2SmU{=)afi{HGZc4o z&0E+ZzFo>6|D_zbTg}aG#<+(_p~m~Tu?Gu1F7TF!YIQ&9s^a z)bO7Gz)}nFO@yt56gS{OX16{M!!c-f0<(bgG3z%(`Gg_L(^ngEJHtSDcFTwUQO|zI zBKs6>W1!HAoSw-a7GpG1)ZdCs4Je?|%+7mibBvFFwlU^8$x32T!lrgd0|E5@mk&ZS zEp<0N5_i`tdn3%O&$Mm)L&nbj)%z_1HXn#_S9&A3a69cFSD$>V0}e^kvN7fWz(%5SHVpD&4n*9k=nIb0q@+fh_(cEV5{YnNmV3cm9kDIp}VX zhNx%ul=H{FX;VFv1N?9~E8h5{Jl^TOVX}sJPP-0G2@P@+o)ih7U{{I#9lv2IXYLf_ zI6=RM#02lm4?0JtV_wU3c5&6S9w2KpD`*m~=EC;oJ~DA{#PK68nsuU*ft=%vtBF!- z?S-wd8s0kq_rB^e_xOzp^$Z|p@|gllQnsn78Peu~{0gbOP%w~06f5uWdm&7RN-}_8 z@p=uUoIb;IZ$*IM32LwVE;w!$fgdoP?9kExg_LH|`Ju{Vcu<8jWmtAz8|-s z?-b7nb`aT5MpSH_pIkx9Z6#$kXFDZmp#K|K@UO3kCxj#dWcqk_xN+p!8n-huMYb&IbRWxiJ< zEsCV-wjWlgnrQt(i^6+O)j{P8D?e}72r^N&UyNt`^|yWhkF7U>hqC?tfVD{R;Auf* zsZb%5EZMcuQ1&frCF|Jtofau;$x`;R6O$PG7Fi;@G4>@Ph8a6q-*Z#H-|zpv?|eQz z@x;u1U-xyL^F8N$&-Y*gU%Y7h&R~iPLo(!`bJo_Vfj&$Lt^fn9#9#f?F@i8{1tA?k zRl{j|;9h5{B+v0-SLI2KO^=B3?=ThWb$xQH)Ehwg;8G*KGst4TmLN9rt44&gw-r(BvwDNZL< z$yCTD3HOI-6j$2L_P_s8_yrtbgx$xg{-~;dL0`O;no8O1(wF-q*MCFNEq`v1oXM;H zCl+a+JyOB}ck-%qvCS*EF#W@SYQd%Tizea#H0 z>)nl6f9t+-yDgtF4{6sx_M7CD<=kTj)WDdTj5y(cMTiVVasYME94dcI;_k_vZ>wqX z;i+3}eT@CUVCDi}j36E(got_2M)C--f)HF_`}^G#zx!sK2oOk&^bi!ij&b2{aAE}Z zy2>PcT+pOHO~vSwkF=zF6}1MV#jl$^PYhJ0RWC{pJGVfC^jnDr!>u0{S&#TZs24{b)ax5St!i$Kr_pDu*lu@-u#mCdZF~ zXVr${rrJrD|EVNizY&8(581y>%AqD0TUY$1tvqYbn}#JLM-jYt;jnwf(;vigX~f3u zHF}kP4f(p;RiimdDuYh5_tN=+sz3v52cM1w+SAw#X}Bo*mKd)uC4?BLbt8QoO5dUR z4JwoKeXW!cQC7`S7J4&to!Xjj4D!P#-w;ddDpjwU0%Qt+-rOm_ zcl?4k{_{wy4@NrB2O?+xmeEKI&+$8=>#AW*0&fD5r(cqqgu3WVnXn>u8$R+(FQdolm zDEmJ#uyRdWC`TwXDsBIM4VVk6)Wk~DGPU|a04!2sw47aZ2o;V)H6;eKs7abSp%@nS`9Hs^RTI`}&-V;JoiL zX&Y+Y#Ao)qCU%R0fH@>fra}0Zun6$E2lV#xVTcSW?40ygSz*(cTYJrF$#+kQOwYY$ zpYBKrghcx6ti3O#`A}JLw21U@*Tw&+XB`4Up|a+!nlG@s8E(Dp2L32)2Ob!^bNC3* zZ-)4&$dnZjMki2bm_$s%p~e@xf)G`9>9+AOeWGM>MM<#>DImV+omujPLLDnaI56j{ z@_VI!4G?CxWAVV(x&Xh_JNp(oAd9jr)JQhd5@%Tcr-l`K^J(9d{E=*AP4qA>m8r=U(B0O&b9Em zTJ#EPkeG78n#0o4SGH;e{0o8I(bD>8w-*iqF6cJ}IjZZR?v>F-ia_eG>+>6RVn9BC zxgNGmNJ|osJeu^h&OmYiD2#Tcsp3d`1LAMngnFUTA{QE!CIcr}KjeG{cOUt26GE{_ zWlXgLm;Xgn5^S4ie1d^fdx-T4aSG>@wyG+yv znIV2~lIo-)Vd{}xlDhes(UjIH3_`$)}&M)qn99Rel znwJI)4RUM`Q9KB$iONPNOciEH=e~GG`pvz0v!tNKC48{zB=mZ$TrYAG$fu7ZibOu%V;fvx^<2< zCb`(}kmLhlLjovy zYY<-CCMS<+Vh(cZne8iei30WL6xnOn7xB75m zuBR11uQ+{id=ST777Us}){__iNdCcSHa@Kp5s-KXtrrDEkoSMYvF~_ee_$RV=|1Fz z)eGL~{}llP9d|hyalQP8j-w;jfvQ%1Wy2Vm0|{(2T%O$ZcVYtj$&?L%>Irl#i#MO$ zvk`H?J|KZ>LMKO={xNQIHQCw&)i9=xPP_8qHC;euL;4;$eJxi$zYC2&MGs?OOAZ~t z@$&FA^;i#A09|RH@{RwQYt^(<62yQL?r=Em1YT(UoN+|G%zR3efrVuy2(K7zygZ+V z2Dxy>R=S*Akaz%I@H--lPpMjuD+tVA_%rZ^Z(nyfrovVys|}2C_+LGh>tW$&Y!Qp@ z+5il-bD<=Ge&ffqolPI>r>**YayVNUJ>g4+x_M1qsV(RpChBAW2qHyan$S!`m%4YF65rXoq_R%gJar$c$r-<(pe^&XJ{w30|IkXAQ5WEQfNJtt|YbHc38gNY3Acj zvSr)E-BW@#EyJ-ntMK-SFihybwmddgk#?;Me$yqMkZl_+IA4x5+We$=m%-& z3)AJR!+2Ae({b)@G|&K$6_?>vGdfTWW;s;JYj@shT8m@nEZ`K7r9|eMBKfb0R=2oI zVAqZFA5EpXuWr!>=X~)qX?8{UMninG5#e?hD^(ZAbhS3)Er`*18s6S;0))m>$;d@c zpZv#F7wCbw5+}4x)V?mp|DzH$K5Nh6Mik`vcG$^asN7#+1P7Mq-M7$iapf7s2Mtc^ z-?f%u@P!%7@d|on-flZh#oyZPBc(gh^?t3XHA1e0r1yKq@SXtQsk8Sq7{1-AtV z-R%nFo|L$Q2%0)BL7UR8_)GI=#&pth48}osu*6Zd9WL3oN(&(DuAGzDk$-%9P&3$ZLGFPF$9Uw9nG@Kj{m4LC;nK<}6kyjVV zrYk_7SNA(xK_EfszGcfB6YmFHh-AQ7>0l^<_mcZ4G9#09?Q zSRWXpcaD!qN7!FbxMX2yVSi=YU579yt6c#f^ijz5nY2@lLf(!25D+h z)`~*U5DE&Bilzn)N>-eb?C;b)=MkDGn8{Y;Hi$Nf$ z>}!4f=e(4VoW+lk%!pQhy%Q~K!@AE3QSxG=*v`%x)uD)Q;fxaRQ_uEXt1_C7_E+aX z3(+P=W5MF$>>COvVBj2LSQ{pQcS?BRE=2(_jOU&~kRdL)CUhiX|Ind>)>V^ZB_$=36{OZl@8N`ZsA_y!eGjXI<+Jv8J+HlG-)sxY8`UeR))Pvn zGwHK~LcUa7IJqVn=BXVjakX|5o7X$TB(_OI&EoBE@NTtCcM>8B-zJsHUuKBJnpI!; z)}+bqBQezwIK7FX-R4D6(o&UMmQK>Vn96b)+LHOkm9dVe*y&!&O={*l0@9W}R@b5> z9ZGtQ)n6RVeJXU;`BSfP;{HdQ(H=AC3O`7kmLvKV2t-2cYy&SU+4NYd|>!>YC zq&D^UdzI zRF8$~hqG8-zS+D0sw2;VRiu zgg^Ir;;|MkzJK*zbxw6{z1jJBIjB$UW3Ye#bILUH3^v|w)QPF3SWL1^ZW_&Z zNa>nvEaMHa6E(Q?sWrPTdtxl25#o_LfOfvOyHS{u8-ZixOj|Kfd92L*_HjRdzm8i_ z@7G(=wB+?*7_P8t+GZz(1v50xvQ{%~Ol2H>Oqm;z9dkKgwR&;hee}0$p_in`N*B9N z-01vfyU)oKhshSVIE>mPVHvtk-6e$Ax_=xi5 zN5kLc*pS+*JhS)|vDAr>O^oYO$P_Vb(#+1k|AnV+j8=M6nQGXxCN0+w--LJPoNDC3 zbX5*{DTO(LM$p~ZOG9aPet7iG_Llq4(GI5uY1{ji))l|$o833<5~f)^@K=MGML*vm zY<$ma)zIi%Fl!qAms-Px4eQk!wQTPq`n zbT%DJJCH(|e0Q9@CryrvPMvTe`nI-N+~O zTlE!<2A*vtNv=vhl1R&$v1E}bQrht(P|t2UXVts;Q?sQE4~yN&t{JTPQgJXqZpW^y z{as1&s^P72TKRR7MzI!8kZ1Lf`rCepN7>%FHc@G z-gfE5=kjlFk>u}eerOENPG44quyUJQ-YR2NSbrxiQ5I|vEkgRt z=rQPoKeD^Q7SvPrSCf}gNIUMF$ZN_t##-VNa84;ODG>6iMV7_(_E>hTWt``%(^gl3 zTkrm2#>&X862C$5#DeFJd>-eG$mqU?u5yr(K5AxtuqnCKE;9Hb|9E}GT(6E-zPsPi zrHjmKUMsc3Qjglj%7S(L3)|U9e?!W)g=k)Q@M$huBaA7^I)aKix zG|pmitm0G~B~q~Tb?!4m#pMm6Z|Z`>nprImdU#PUnwVTIGsruwgkSRGJT0R#E5@Cv zFE=PkY+bh<_t3dSDx_udDs%TEGOwD>f)P@ZT=`UQYeV=X9DQlZ084N!uMepwUG(nc zJ#Fx+PzK1AMH2dUxAgVBmT}YVdH%gAyY|WAWaYrXfL>l;L!n!faBZ{e_0b3KbxLd> z_l~PG8l6{Ev6s*MF)8()b+9yAU-2m6n6TN(fUD%1a_subEZSN0!qd$b>X>7AgCBRm z5?6Xj7dZ|~=QRqhCUuO~d@HGFT4uW1wi;4$DXM9_<2Ab+GP=&o9=(T4{2mN3)qYLm)dwt5rg7#5pDXQa324&cReV z6Rf@oGY_)~pTuhT#*ZMr>Wvj*$4-nu`&e%EBE3QB4 z#L9udyZh?0JSA+|M%7vmS^2BfFQ#CX$5Z9Egx9ftft)Cp5GFVC02(+m*_W)mY5kvbFsL)wy!- z6uwO1z}gSOR(n%jnJ+Al|wz2&WOK% zPE&aUj0r)S4bwdxLDF<>zAQp(Pc=`3(>~GS-F3K$erEqg6o%gwRX8Z6bLMUgmq`_A z@H)uKt$Y%5oc=(Rbmv)5)fLMX5_ zte}7VVfcj3te62wIEsBM=&~>@3{0U}UL~g*6eIVblvIjdkSTU*#`excFU}}L|DM~b zs{sE#NK^`A^_ad_s7xGKi$Z4Wh<3C#y!KALJ^9m9jm`6Gy3pCoa;Z(rq>q~D3lqSs2T}33bmYn2@uCU5SFF;p)7ah0{r$GdGvq=*W^^e;6nA+bkjkfmXdM-S2CHTPnj& zFye;NZKgcXf&|QtlCa08C)@nCF+O@>U^%-s&{BAw#dsIO49~dJSls#udw>N#SC5p| zPhB2_o^ZJynXHI+Q42Nj>Z%x9z7>C5LB`j>a5Kl*azj>E*v2sn3n*xr*l);!Ax&!% zQVx0^p@4$Yn#p{>4VutxBlyRs_U=|%WP(+VJE&p`EF(CmOUBRTDP)8l8nk~?K8j{A2e2J(r;WnnSL#VMLbnW>K@ve@dA5PQwn^U1xnb_Kr!Z|V;JD2)y%?H zV@9W&BPQ1zcr2Kv(pjSo-fUUz}w=i6UWCQ+&>;~1rF_kCm* zhtjSn#1vZcboLxKKR+mCrWV(0bme z*6SIw_G7Gir6>wX6j98T6Z9U6uwuNW^_e|<*uxYDgp|$9(sRAwTJoB%G3Hwj>V$=d z#aix>TX!$Y*F5v4%-Y&ZL6mA~L&5)CP-Y=q$8vYacmSiIp52xBIXAt6rWDBy8nc|E@4b~ahqrZ=@&w}9@HL`y>DNE!Jmozw(sbkUP# za;mR3E+#fC)9J{m0Db?U?|1IT)2Dr@A$0|4uzj2QzVQ3EcJaE=pnJdB;A&fvF%4r) zo)uRL?O}hRmCm3uPLHy53r2|3Hi4j?d@me8Dr$eCE%tt|{e5jW%iiUYpv9&TE2@LDJ4NHEB$KBfv5Ho3|RZ42kpePIa z=~(Hfg?FZf5`NYG#?UdcSr($Ff4*e;x3jPz7nZ|~nyWSXsMe$JASNg1VO{oQ)|pp7 zU&iSCoy!){m#l;6c@55?&Y^sMn(>W(eM;qyOE~=8_EJFAGTDvvt{ATHP;S0=>I_?le2QGBf;E_xtQFzR%ezgqh;`_4zfsU^elR+g4LF{s!yyggQx- zYJ!1e5EJrTXOqqb1c}?ui5qRK9$056bp{Ut`|lriWtQWEas`Q}v$e6Z{4G9Ece83g zekJYH<6+Avh1|kn+67sFg?~G$8|0gUbR0O)I$TCBJhO1}a+jyt>QO9V8hs&0G+)t0{km4d89w*#C zF7Tw-Pn;0KCa#k=A(8g%D~F`SA`0$BM%BPpRGN7J?zMbw=8f!Kpdoc_YSCNNW(py9 z{m}m(dqPPJ|C!6Ed4~jLLR+M){_jE8O6|&aBbl|p9FJZv}A7Uy5qHnN)BA4U;Z@&(lRbiRF3>QWp^NBvB>o6xr?FKWVwee*umgw z16?oni=Wq`4CRK6WB5uWLm*>u#0%Y|6O;3$YCIJR%_dR43in4_b&wTH7bY@^w9Fhs zzD}&m+--HVh(^rd-1qZ?)#aDly#$??W@W2zF<`}lX!2DJ(nd>(WjCH$7iaV+)=ai+ zJ+u@-GpYVuGxG6LMx016)7Fa9)Xw_IaGmoP+fU&P?q2t&IXJBws0MO$^16)}M)~5z zGv!uutT&$jQ*sO0X*`iLj`J>Q||{?7&2g-l1qwc(LDQdvw=cpH7# z`1P71&COE^_M(u-gd>c}-S?@Rfd2KQ;bOP`*7pv-CgDB2p)6q{$N zD6&xi5Dr>f6T{S0182nu&8fV)oEcsErn3d7YvDX!e;D?|MB@Egu~HGa%7KfKXCL$= z)q97BGb}7S2mh)l$36sQ)Pd@AL8sIn8SZ$CFvY#3W!`Mb4xD@ZbeOTf+{2k47RZfq zfu4p8Y_n|Q+QeIL*2y3=Y41ars{yqo{;l*7J0o&6;j#8 zW4Ns4d)4myt2rCigtWNQyr~E@Q_P94E3-7-$9ak{f2ZOY-VL^CF5p|fl4T}PG4K^_ zL2?gyT~Gvmx~iI*(~qxqTWd9;QBr5%=p89>CZzHZx-|kLdFf7oV6e=v09R3&dWP>M z!(#oiCxkbz-m3FCMB*jFmiCfFYYGX6$t+$u`rjyBozaSFVHv#sxx)6U02ZDowkWfB zhFt6?{Un?n#vQBBnE7YPWr`&t^-F5N%?sST=fSa@&cQX_pn_qO`X+5!-y+Ex%J9#? zL|*?B#7rSu!tmqpFJ}@w=_}LRT!Q60JJnw~o%CCxUcWQ9tSo$pl~T|)%oJKZ*FUZb zTOpS|#cqA66)~QL)*TmyT+ZuJqG!@xdZDuv-lkO7l(bM4_j!F!F<#ZCdzgg2L}vc8 z;Wd+}{m&GogPjtKmcKpI_nJ|wV!FE2y|$K3ygsiS2MqWfyegTVNyTP|z1y%?*YJxNn$`0JN(@fFT|dsH*s-|ZiUQup%bfc{_aT;StKoqV18H@ z^8VRqF}Dxyt1|{O2`)AMgPH6Za)|f46&PjS{=+!nB{6BV#+vu?qX94>k$ifILTKqu zu*DqTez^ix-(V4+pkcod%cgVJ<*NK<+ih>tq3SZPs>zU}HOTuTrl0ms90wPv^)pPW zWL&)$wpnFEXwyvbrdhzK#pX@6(0Od$>;qo?AcJiBy)ba=gHDqywcd-@z=t28sJs%t zjr!xv${G<6JL3_XWg*}lJRxA2<{fT{peA0FK6OEEeX;#(1CKNYPTB> zch|%u^~YF3CsDt2U?ubg%ZJ;vcY*Ir#$6p+A4>R@%NFeUZNa!H-?LXBnDvS*lsf${ z3ZnfowPmidUwPwedzsSt7Bc`4Siw-YZcfWK@P+Pk@2@V;>Pk7ix2LXfdezHGCKNJA zpFD3QN1$c5=>&AL)KbcJL6j4>cE1kH$UA#o$$ZDr{c*R&GhC##v+1(B zPo5c``=mT5YLWb5V|j(?J2e>i18V2ZrzMX&jf}W}Y(bQ;XYseg%!as+@8*Yh;D^Cm zbMM6@207@qj2q%c;-Z`ykkjca_P`{Uy|TK=<&B!i_OP@2lhujLSOLK4`p>H3Ns-i^ zR|TE#Qm_f$Ih>@jN&57708Ze37K)EAjkoBO+CMEh2vP4PEm%&eb=T}(W3u!|NaHb? zG7HO9TNzp<;4R|ern=I!Xt5T!CpLGWa+!)!MBzHdNjzX6nxc3+fxIB(vGV5GvojSp zfI&<72+Xa_9Ye$uVk($10K>wPk?r;M*L(@-y&Tqbc%N~p-mN-eYV!{isfw!AeG$<@ z?22EUOOFZ=9XkIl$2t}R3RrTVB@4U-izo<~uGzyo2et~TV^fQP3ExEeuB*B_@_A_E zMbDYAJc=hIck=PJb1(DXxZ7SHK@OVw!2|t8j(_Nw0NNlxkL_7UQHV)hY1!jjvy4 zeEv0zDdaH6`6o0?eFuVCC0pBY+B|HYI%n@@@@$2k#Q1j5S8;z8+j8+<_G8(9-D{#^ zBwF6h+M6RGlUQDm7ht>Ef0%M0Go`>n^n&F|7+CuZVp}p<+k}U%aXM@1Mmf z)Bfz58}Gh=@>)1De9<=(2o?&V&9PiuBoe{Ubp^%r6c8?VnwR=B-P^7k1y&|e2j6JC z^!+ru<88Zg!mQe60bgIYV@4)}HGz$^+0(AGynU>;)eA{v%DY_kWQt)E*j^iM3Vt=u zU%APRT7S3|1;q((g>Xgy&Ov!uUt#b){=c5O#C7xo zhxyH!`-r>{f()|@gEj9svMtXH-8F5E>m1smfFOwj<^F4%t4=}_oyhvuz0e?bCmKnM z98Ib}H70D;YZ1hFRQx9ULyD*=RwI9!t`D_MLB&L7q0j0L-=mMFamIm{jq=|QL{VFi53*pWH>ngFgTz#yj=6YaP6dMgItOb8< ztDWvsSah56?a#7bX)dqu?!jmT4F0hfg-#dBLQ<}QSgu7pt<;*#QyZ3K_`7OGRay25 zo0mHL>6#JPJuWiT2%`_>=3}U`M$(4=mH7eQ`^KTcv#528G1e$*eWo9o@L~i3@s`tgdw5RkQas1c!U#fr34*7{kb4CpiPcvxE)NJ zSeW~rumG0MQd5^~?d?b%WBA#xsClfisNcSAq|d^tIdS?D*~o%Qsk2p~PttmCxknWEMC9em^l)o^IZS?s z`Xzj4GTrq)j$f2gha$Pn2u-nWJ{9ByIYwpn{o%FYT(js!k1LlygjVWvH1%?SG^G&UcPf z;vC5V!C^*kiaP2yA*@mnS%bZoa$KCW&kzJf3DD!zwi_JDr|3V@%~SCGw*NcdX~Q&{ zSq{`*f_i%xA9N#(neiR$?TDp*&w&c^!MM1hbj9;%f)v zlOYd~CAE1|b!Z)ap{f+Z>+;RU7zFp=9l*WULPn$K%+s}QN-|ccT$uex*!j@V07rfA zAt8(FE0CeOZ}h*W`|I+lEaUoug-D2oA)NsDHUMIn-Mp{*J&zR@#5mnrncVSw#}AeY z>{tGDjnb0pX6!L>$^%84`Q{GTK2Y475wJb7i-7jFGV)1eshRLea0bBccY*1M-kH9> zzdCH9vP3yd&}reO7&q)hSCy;G5@^}jzFSz0&R6Hc7Ahh}f3>sa1Mvsh8xST%K;r>q z|8#OUdB;FkRrxoflxz)rBM_d1wCn##twmP$lAoX-#1yuVTP;R{XEe%$Oc*2R(rqjY zBN{oA6aI&IrWd+xNF+>q_dlaYi{6S!1aGWg%ojSost+V82KnG%wgS#_nLg;jCq;um{62y%tO78h}B)0&+hQx$-OD9_+{xrD&18c*yRuRJ+AL|xD?1p5o@vwWB{aVU$E^wX;WjT|@bW5+7gv+$k| zpFLf~4D)4XEADv8?rcfjL+~o#pNn|(pGBiH2I>L=HBh^a@nmG<--l=RmzlHCz2C*- z{-#nHmQhWxe79g_;5#l9xNx=95lT3h<5jO;Pp$M=Q*n>osel}R2QPD(WBYz6U1q^n zgX^*YSb-#C0kU%-5UFX)v~z{SHX1gHKG`_gU5`UlOCAU0sd3`=o;Q0?BIKIKdt$ndkN zcpoS30>`e=Y%cq$;9B=J$h^W{-+R4Hm8W=?-7QmoLRu~?A2KLbL!X;riMl4BRT>lI z&E*|}*h<9qrqk>A7i7+%lUcX2-i4%d=Of;|bDMgzFJ1VrG9sBwZDz;52V9s_0|(SdKuCg`Js7S_rOA}J z5Go{KvET03V_#JOSmG|NHq1h_QyW7yCp1+hG7}v+T}{ zB4vw({Gy#w{2BPCo|!OMq)drY-iDqQT9Rfs?4BN2Qm@=eVeoNd+nv%0#mP*b)Tasq zEp*?b!2{m{vFH%M2AB5zS9#(jNRC`Ysgd5LLk>?qj5SWEu{UL;f#iE0N(Q|>4ubn$AAqL<$}4Vy}U)nU-^(rO-0gMs8)%M*zGz0c0dEj z=L;!fj~rGYo_T;mA`u`}yw%ry_YvZ&Z=F9lULwtYGK|Y4e18v^uWD7Nqu$dt{D^I6 zj$Vuf*UuTBz&6Kwm#k2uDJy7}C+8DW|IqcSu>8@cyR55n1?Am6Qd^6Rn6cJ)ajoYO z2bJ6Ws2B#e5k($=LXh6!J;BJx71M08SEEuMQ_u4Kv;@%=M3Z635V{GWTO8=tBRa;p z060RD(R|l;LtGE#F4!6>5P(8b@ioUE!zgaw)&MBZNW_JvhMSsmWXwc7Mix{>p{adi0}GJy{tDNwgs4 z7iDuAC-`7}tJUk9`Ym-;#! z&-?pzeLRwx{H)K1WTBAchnr3;ejePv#xenqcf#wZ9io2H3}We=f5g(^g6VjwX7=-G zxN4^l`Ix?scN;Z^!nipUFXRV4sQ?&KcdQ&rb)JOkB~pSw@DRZHxk<3Dr z`|}GWhNQC_r1-V|LKH_@a*1EWV*Raxa?QNSI*(YF0X8`DaQVAmH<3t zI7Z|V;D<=@9&#FXHmUwU6}4>f!;^VB2aps#l9}2|CLo1HWbNR@p4g!4q&ovuosz2N z_){22D#MQ;1u38dC?a!D68Ap?&xhMkK#Vcn@xgkd^Meq|AbS%sH<@1QH#0B~044dE zCUk)zUpdJ~>?lq`vT~5*T3EY=MYe%W)ddOS;|v!xr2OcPbrkgw`1SLvesSo{RRBUa zoBBYG)Vl5|{nsghuQ{9PMKSsWf=d`nz+dV#8z(7Z0IPw%IrA?yqoYGc>DLHZc*@-$ zx9-3uxV!=&SmQ$sFvP0HPXi7CSB%(p)gg*P#f0s9ss>l_3{Ydh&%EpODv{mZgrfcD z{wJQ%(g5|@vOVP#O(O*at{78M+Ha8E1Jht7uSakygTM{$qXW{F-(J5qpW#U_f^CGn zWHjb$Xr+IFj*$g))maM1{D*Xrw7*)_;Jy*y)LW1b0HnUWmnqfw1F4XcLdvH`4>N#L<8zo$ZPkCf2VOTA2ndE~{kX?`%ouyw|&|j&tC9yE)i0Y7I!wBtl ziy&`p5pJoK!@*Vc!xU`4ye+8x2FGBvgFYb^b==W}i~UMx_7&TTCyz$~_CYyKv8Es~ zQDIxh4EbfCjJyL-f~erb)M9dS@^|_)zVd(O&XFa$fBV*qQ<^I7^Zv?-vQTeT%4S;6 zx^=)OPa(_JoKaM#P(;@>u9&GOH?o=uzv&^{Y@rE=k#(n|B3q&5oc(^|);mqep-(s_l_wHyxQ zIcS=iZ|#ekcZHI>ACMc+W7(hAI&U#*=r((q?m^zfzV!bW-0;mLZPw})+S*<#4Jnov zxhE7B`Ff0=rX|XrTdQBqg6Ner@xOy`cdt;welwb|+;cB&iSTr{?dkVDIX&cR)Ex@*Y3G>!Wp(!A&1*U!e^UnR6}-AG?4E>O z$}IAIVsu{jO6Nd7!EE$7mAq&K4+<$zsSKidfW0uY;Ct+^e5|iI37Xx9 zt;K!xFNRa6s6-!69L63|){zbImWooibP`AT=jO=X)|up)!x+i`V|^hN^Ken?Me_`| zZ^xcATvJ`iGwQHpXB8`~%kZ_(NieL_0X!c+laqh5pcX9sbEc6iCi7K=!E?w?t%DFe zMGUb~UN$osaEg?D{V)OlwUfRIcH0kgrn?rdjQ25CPBdy3TH>9@e@UJhx`J&ArUcp3PC5hHV>B&^TQK1e$Ne=-XpvLK(%gq}zWwOA$U zGhH=%bxx_GewSXS{J15yNu_a$dFyR_*1G0Y^uK}86dT`$F+UCzZg|_851CE^A3^Iu z@Rnevht+cMC+GlJz+Ulg#znI*U>}X$LaACtkp=(KR5&B-lLtLKdm}XTB853~mfbHx zfd^%1)s2`Y6u8!HNJwY9`y7dT_PD8Pdy67}u31EIzSZZQ$sh^|5DU5B z#FKyWSg!jL=TD+ zJB#T&=e24kw_2@+GiQ5su#&oL(vIYV&s4^b8X?O;c?9ZJU|5jMb}o`mfm-rAXdSPj z04n_rgiik_s|i0_bn#&LtLvNXuL*8G?m3_Ji!*O42ph~5`M?Plq(wMMy-v_`INct*HEFV93oCGNgvlR$pl!<(LMTw-! z9Rq+sS^yWIyeh*vilqJErR*h$bt8rMr^hG2Uhk)p5PJHM_U4K1o(<7#CQZF(?K=HRTW8B$&J$0mT#gHTocV+m4J z4+jp#n&EneuslAazVfmZ58x*BB2AjzTE@16g6=@#zRAiE0u<9p34_r%4_&OqsGSew zfFa+(je^Ad$J;MX%KQc@Kr#eP=6l_@FlTSRwetS1zG6BDECJTaM5W&(V#g|@pv0T0 zzf}1BeLd%0-N}*9;XsY}Aqwf3RK@u8U*6@C&!37|_3BtJ2fbkVA>_;4yP}h=9Z|Ri z{hQHw_sNulB^GBKqNM)g7%rSWFQUbemm-Fh$KN_BGyccsBI#Y$)zVB=A-sy*i8}J-UDtQ zery|16%s9>!VqHFZj)FFG;${)eGKjsy_4#d-d_JFKcUWVIMdqwduEt5!AaiGd$J=K z-#GcoR?#{~(Rd>BfuuCKw*2tpwb(0K=UXSz?;cmks6+ZcMtv0w*;=}(t@b;5p#v?Q zrhJ2Ui`4Zxb0)ai;4EZpG+^@M)cQYq&;O=M z)Y(j1#*6%(QuR4~PuYH5OYzLX-eFnIJ!fTmL}20wdIm#?Ue8j@4trW6oih1L+T$~p zw-&II^=c>Y?13)cG9WoMW-#B~(TW7(seD*BTSJUd-7B$awp*UZgidXhgFExx4=H@4 zp=@k^_{Su$Jh|-#cD5&%iiROl`WfD3pM_DWIb_0rba&a2|M)gFgofy0>RZH878x7@ zS$w#!X!VB{4P37S`@V$p${_>UPFf|6greb^mye<(reD^bB6c`jg~>WsZkS452}4O{S}=v{r}^41xi0F&ex+O z+FrQqD8)Z5GmY!OOa{_Uv958|6LLeH>DexUAGVOooSY8<7PVz(?Zf(d=Xr0Vt+N-U zvRUq*Lo@-&d?}Em5X4hqu$PnJ^wNo*=Sa@YEjMrKK3gBxa8lE32eAPj zzpi;LeAD@pN=%g1{Zn(^W`itwVn8W(kH`195|izP)x|~kO@M)MPVSN)YtENb_;OCg zJR<2op=D`p7foCBouA69^$!h=KxIJJv_W`Q!Nck1Xe+oj@)Vdy6ev+4#t;g(O zb0hq;?}?)b?~r2(s+_oDHIF-dbXS76gvjx?T=)C}31>)&226T~gfDGrI7|CD|WD9ULl8uwN*teCP7DNNz1pk(086?5RJa?ugA&K*u8q*L9j^8 zw@)x5*7lNe+#c3$d^t$Z!PW3z;DoHj{OY3GQ+MxaC#znux$eC6scuE=_X_Gnon82h z?4rM~fQUCSJoWs=VorPjH#arXB5hcFvr6-Wn6vw1vh!N>fP=5_Ah?O%{t)-6zafZO zTjJvqr;WHj*dBSWF0;wj1d-htL2DOy+2zDl^~Dn6R@d6*nf2*r(iF2VQhw?hNSuqs z-P7@@GL|p^a0vTxsn0d*;(YPjcoY?|T567xp1_WaNoXvoR(FEyIDFB9Xtu0h!43j;Ktz!`JUFg@?>;Wr0SpDUd?-vw1?AT8#U@MwRoDh!H327-sz6RGER%|&NM*{YUlqa5vT0+?BH)g zy<_Wx0kM6mgJMHZ3q*^H#T*Abs@87G<^VR5@?KmWx>1!t{H9rGOFG&hyS()XLpbjr zb(&Naf(DPi_lUZEIZQJGGhlpK3#)J3kw6U^MwE=zXMvr=~;?GI#eVJqQjARaz;k z+b9nvs`MAu9=P*2{`}8kxxzGxv{QZB>S@Y(L!aulEiAHn_)2DVT|=)Tr#2qDMi}mp zlG!>t@U#NiR`@fIISu*&VcIC>XF8-;_V*MQ;<+yC{i9C4FG^}J1{bOytr><6-2(wZ zLCP`0S~2>6lxM-2OkKC!X6k7-1b6+bt5{Ee@T=jz$N*$woqr@fib0a~)L@e4%du?i zmY{=nA}jOTETz>)gy+Kaem$PFtIuI)Y?@_t2-q(z7VF)LXuHgqF8?BT*G?+`jlf{? z?Y{rGpN;7(%o6lrSwPiouOz@Gls<_mXucS6j3zQ!47<6s_DRgbdS}*sIlCjBi0vwQ zO`4ZOTS?bCXJ}+XiUI<`>M!ZeFeP<&(JaEh6w-uocrDHU^>HVRydHiiu-=kAKta8m zV1l>+Je>vW45r@b!17`-v+*X3hd=eZ>4%}{KI23&1Pnh#0*GVqgYw2ZUr7IxF(8xS zG}CT1w2IblWTt6Z6T*WL=B9%&1q~*7JnS{^mmK|P3Qq6R0_N_?nha4O9o|R}jIY2t z+-LsUY$5KmvBIE`bu6U&(m8fXgJfh;F_5nIvSxZHF%+)7Vk4s)_30X9Diy*^nqPDR zecxo5mGAf@{{;8AV~F>+Mh$ItpI*$;n@U3*>1Qdl<4kxt8vtmbgI$$Wg4Y~AJp8oo zzp19_eqk8^-X_DxMven$fF>^)geC;BGO+K`chin*7a`IL>w=+{$g>Im|Ld>NvxY2a z@60h@h0hyr72wl?>ZMal)z8aJcM8im5!44?%~i1Jm5>td)s2O#6-Sp&l+m!#wt+JV zzba*2tR}ANQ%ch4&ohS3I3LA=MxD~>ArYAPY{EcR z4U0G6K`=phY?wSENTAHSNqFj`5GLk0lMd-VXl|mmAFU=Y^!e-m8|%?jQ3y!{IQzSP z*PV17ai_Ij4 zB7gipY`qCM)a(EMuR7YJRUw?Tk#K};SvrMKwk%~VA!Oh8txY=CC`(yJAt5Bo*r#k$ z*6cJG5weW2jcqW$*R49A&*%I9pX+jUonp+q-}n2zU(56Pc(rcB2#$ZlOuR9crbuf- zzAU}fXU=v#M`bO!Op!;n$F*f57tpfmlQ)viB*7}Zd6s1l=Y`9W`7X;eI9cJXazh~) zsVYD0;4{RHE5Pp8N0EjzPGH`fjlYWVdiB2rLxfR<#bhk3dCrBCE3pxV@)z2 z)hHg(^`9sYrYK6~xvsxje3D&Ih?NZV!r}>{(Iz?4x%Rd>1n?bx(kmS-IR!g7B-$sG zuDuXxNl|j201(6V-jLJt5ri=eDl^B3&Nt6|^RUa^G+gS{)fv|Q>gm${Y4ZLNX}JrpFj?oa zPO3kLRq@xV-+-L$vU#u@hE47TTL;f^;k+y5k-ZBb*3&7z&D*b6q7P4mGco2#>}?xw>nR-4anlypcErt@a~y-=OAVDyiTN$OtsFe?-p~YB7C@& zdqc#cscoeTA8oz--mI3{@<4593uW89F7R}^)Y4#=e3rJVmdpifZBgf~8;$W6JagtU z3q>zPEHxM4f@h#Y!^S}f^CUn2O2x~N27S#@ICxK?GCXH5!Te`%;<7c4(GooN$LFR& z)Z|4^2;kgO4t}Yo?l7=5K#C6%oX^U|&7f?6Bp#_c|8P957R=WsCKrws$@-+fAn-OCxo6%@PA?lQybc560wjA13_Viw|nB#CzrUkHfgAgj@VC!z|_ntH2Wd+4A}A>Y_V-nKVTib3Oe$K1$=#*KWs5sv4VmmlE40|c?R#P z`9re@WH;&A0kE@zgzf9I*u|HfpbHO~n_G;ESxuVE6eRGyA{Xpj!sBT?N1ynur!u0x zX!D%pxpRGg;{v+SbBk<6U066O3`$*`tdGQyoyJ;jI)@Tqt)ntKCT+0NAkm^ZRB#59 zK~ViP(Rmbv^$4H2i6t0PFeKvEn{D#0mCn&$6Wa9cOarM|&DH9;f&6gZ6793zijUB( zGb{RArg^xz|DL1g$~DUz&SAHoOol78A;^h&$;2Tw{xNWHzUlq&3wk)?I1wlK?bh!L zX7vVc`ayQPs1#TRsEKAM$kMU1h#pkpgy+8S5&r&lzp#dRk!kfilW=Ar1|0Xm{56k> zbc?VJrD0Gd#S&}@oXymOOtATeOy{4VQrt~*yMmE>ZfNj+d5f>NcS4#*@*~8X;ynJR zO(cB4wP&bTB-@dd?6hNcZ8zr!$J9d2PUD8Dzv$ahqnC?Yo^Ab=(yJ$5jsMFqa=wLNE$Da( z-FG>8B=jQpm#aX&(E9!JBAC1xKHP@bb0t3~=~ihUf5Z6aZXx$O;P`+DF$3I5JfN!> z>IS6M;#YFvs{$t5o0BuJyVbfWD7XPoP1=HC>{#xNW7m}GrT%<3cPdpQptiV<1t{BFaq?1c5!TGT6()yj%H&`d(=P|!-58o>kE zq7A}Z-N`Taf;i57GF|E-O0t}WeOHJMlKb*(>yO@lcRVH6X%F6;pM!yd&WJl z$vrP!o1%XncBJ35FytxbF5T|`e&Q8j4f2kt=!oRKm@%pK5+^6>Ld@X;{?s47<)y=~ z#ycZQ0_LPNV#ED9(Whi!_^a_-Rp6n;FAuzCDE?s3fA%y7uiOd%oFnvj8Q;3eNOi_v z4WWaD?xc?@Y}ok7aq10J)B=}%VGihUUiaI>|MJ;P!N;V|;A19WL*N6mhEy~Nl;M_^ zPzzg-JuCPv<|84XYquz7gzi+%DTU=f$@!_1*}_R#XowzK9-En0^S$Z1zrb321=Q>` z>Ub%ORI0HquX+6(tWW7#!)StoLxr@gRGwzJ1SJM>=ukcBOos>N$s(PgUq14b z`(ii3`)*+0(s0Wf_{l!_iCE$b0&O!l2`oRcTt}X={7IAxjt>B>h`^e&yrU8&A+Mg{#4w{h|0Xkwkoz!2V1x)14F<)nJ%~n_2-VP1Aq49ob?M^kJzMqVW1d88BpEy ze#x85erB=lNz*31_qli8bQyN3Y4H2d?%sulvc11&Zg=)aM%O*bo;xxgwaO1bXampY zHz+?QAtC(MJ#0{as{0YdH?ABBWO4u_++85|r55LLb>yivpXm@!YfGPOB>Kw>Kt@k&`vx^vszXeL`fD;ugkGlEH%J5uQY#J@JpB^ zTEBnqpKxUxb_d4OX;{6$QuoHDPY$0PM?!$)1LJv^cC={o>wVUFlW7m|tH>0fGKwy6 zs9cLU!Kjz@zGu=f2OIHCJ-W>@KE_Dx+uTkSDMx)-x{q>NYwHQ6tg5d(DL0*Qa*p4M z@7uVA-y5ndI3y!TU&?JixHela7&$A+;hmY~kg#om+%kjpQwKQZnX<+L!%NNZHm{;D z;Ha?y!_to9{HsrT-h)?UlT+=>v(Xo@f!d~tiW4i@ePd_QX}^jM3EACaa~>Ws^&P0Y z`(etz(^td9odhI;pZV&Az&}Ru@72e`%1Hdx(EbL)+{t0afFbh>$CQvpQKuYVJ#qBV zFz^>TpeZpia}86-M$FPP9G)u!6Xe?TkeZdFwsTMUAHmRHL4H)bax#u89pR=4$t%wX z6Gv~(iW~G)kX|l$m4lJl*;`-yR>*j&b*wyUV0v=N(kp)NFRn8eNesc*6~#{Il{vdE z-44rsPy6hbYJ`jn2WXu-q(T8(@gJ>VA!qB!iq5vTDvrOkV zHKR=*p(<3sgAh9->>cINY@NyWd>LE)s(#lI-%UH9C_pjns8H!aeFgoVo)tGDOO}s+ zv729MftIN>W016j@5-~XlAqxFsO19YB})q=4<`&VT{_=WR|h=PO;CtOB>plQlasm*J;27yWfCCQHwOE&&=6j?BlAWmGrK(5a ztrUBX5h{|0e;6JNcMsE;c=s2T+@{;IMmxZ=)#@&X)FKaY_{{z~U$8zRy|$*II|*z@ zWsmf`Hay>+c;+Ku^#dRKcee{a)flkmiDcwbsLh20A_ElVsGb43)<%CbMSZ$8-<9Xz+^)HB`tnUYd-86W=$2ZywY(lN>|mho?j7Nul& z$%L;fRNWT(8$fM->Z6Mcu_skKd=-K;a2x6-k*%zLcAa3|wg7?RjI%7!O4gxaEl!Wk= zX!XEbJ|y7|6w`^tmOcbF4c6y_73jOvqj0TQP!(2_9YH6@M;E;opcz}?9XOYQG*Rx_ zboWsB#QL^%Eb5E(q$aV6M@GH^K7fjUnRCeNYjGI}lm$aIJ0{;VJo_7*kfu_QGeEKG zgc6j4-WzxxD+A*$P;o_@*rkkvkaa2!6~2&jgdy_;lIcb`PleOSTN?aBOVvQ4O_We; zh_~D5j3R`Urt`Zsoc&fWNUrVXVz${Yin&!D974GW#sen~5+-NN&^9i=vQMj;hPZ+I z6Ne#xC3}DpyS5z6-jMim#h}trJk;Nja2!m$W|Oq4PsBPigIn3K)p!}OCoS;%gtH_z zM?)731#}Gp^hl^KkB>1o$DBFpV<7A=EpzsX!(SOkp?r*%$z8TorKhLZ#2udWm?b@H zN^C5fS31)I7p?6ySJv#;D7SeoLZ%vh6PyKh1YrK@Rosq%%iA?nhIpj1Vf5gyg`sD1 zi8ddfWC5p^JNc_hBpmF+ELu&J5=>F%G!O^J3yF&89`XCyhlZuTK4S17wM?~w|5DY# zI|W{4ejgp2fPiUrbid|ciSOz+W!x8AztcmN1_6f~g1ex}4rf@>j;B+unG*vaeHGjX zRz>-jF6`z!UwNYx*MDQcZEdxwl9iQ7+J}DIwAY%xREn_}yy?H@U%T+`Te3^DFEeY` z3UN}g0Ij|hEYONLMQU$u-h8klI~n$+L$JECXtzV7>2v%znnC%jL6+fi_6!><(2|uR zVZ7&;Z|-+~OiSh4Qv`gU#||v8Nklr`*fmmP{jto$74|mHSedpgK4({*K2#A*TBTAb z7{$v;NymA4y{J*T^W%IusHHr%BG8qohONlJ;IeyCH}TYVd^5HSpOflSZUv(s>#=SS z4rSx7dLp4c>^TTEtZDd|dSzS#;=Yo`15irAspp=Mo~x`!nrTNUDDZ2BgJS!p@L)*N zB-DY%*4$Z282?G8Eu1ge3kxR0!lj?W0R=?Vm5IscM4$##<}{i2fQ@b()X)T4oaphU zQ3>!vf%(4j1`(>s#aY-BxTb1N!|f~U-hHK~x7|4F1zv7xN$YHQTgab`>px%8EjctQ zqjBP4=s17bKk|DfmaW_TLDoye34to*S@iSd2@D4MfuqEvGq<6$u{JcfE0s+cI^%w% zV{$C?ad-Tie>1!fB2aT*`7u;r9;0EESTe`NE~6}_Kqv~(-9^kL$8mof7~EWQ^0|{y z{JxaN#b#W;L!7tWYCW7GZvl1tveD=y>!v~2ea4#{{TGg$w`FD1G1*RyAdGmB9rRmE z5YG_~G|-)QzWJi2uCpNEH8rD_XIAdmVpU6gE9Pq;p35kua$r~Vs^js*1 z?B^pEc|D>udw)bBy;3)EHQsHo==> zuf)UdBHRR?Lh7dnZDoempvKQyn|DSx3sw_ycQguox#{uCS8?1%KM|aocfdJ>BqMNF z!$0hsW*;M#nw|T8^XtmC4J9he7|meW0x?ys}asA*Y7T2t^S|2p)uQ$ zu3h1r)9sb3W$<7Eld9BGFvb>gCUirz1>2Fsj@P}1(>84h{4}20mMw;2gUNJ@;5`Qq zr$H1DzNzhg--An-zzGQa?Fg8FN35eTbuM6ED}NTmi+tMR;>WAj1X~fb6i~mx4VDjM zy4Do??0%dCzIdUm<=Uu%qLGr`^12gF^fPnQ`f47{SIoNq)hFuCI_AFfR^K^S3?MA} zTI!uPRk29th3K}&W6tDCwp18W;_9AUI##3Q2GX?S!A6>u4^_?s~>KLP=Ny>$^`gO$F7VE zamYrz-jKcO6_hD=88{b6y51ShH6=r(#mc1*NHYAnnPi}?SrZo$YE6L!bi@>@;ZW&B zCY=SYRaGEUi~|=1oXY&va^n5_K`vW3W!>`E|C&9IOyBT$j{za4BV$;5ytOzvn~8Z& z&%5+ZVXc0m6|#+n>r^CzU;CtQ!AL9dV*pFk`> zm#2Ef|Fh^@*(X_QZbt1lE!Go*$JxwVuy$`tZz%u){7T(uU=;ecXM| z%9n?h@JBWoE3U9|D9OC-{ALSS-M-B&@JbdoSY;LGxFNXdD=z%NEPb?xYgmT}&}gex ztb$VAu46F^5>mk3k2{~;)rvC|?08Z9K95Q+Dk=-!kF{I`@k1~;ai!a&Z*P>wr z{M#up7u~B3*D}Bb0%t=^`7aBu+*Q*4W?Qz_>g~Bup`C17t!ThJ0Y~*@ER>?5)F18D zUkPs2aOWFY_pIRlL3kq5tnFYB^c3G2!~MZTC|fsLl&3LP#lTeS?er-YC@JP#qc?Ug z+mLE~e7jTM)fQWIRT{1PneA#z#K&g3HMv8>a4#?bLtJMoj!Dnhr`=p z3lMLQgvUTw38AIeKi^vTa>`oCGZD|K6dYW;r!`81Bx^^*DkC(=!IONw{N0CvOn!5- zGTpk;i>lI9lj@*IiUj0#U$-6cd>y$v(SVfU>cLL#qh=A;*_B!K1n$(2geq7JC|#e@ zoy*kom|Vb2OL(u5-OC?UCa9)Wte;V$@M9*0daar51`49zZ$-|6jGaJ8W!BBn@mgU` zea+diJa;q5f1NR}zS8Wh?AcXxcw00lT9ICEF*XLd=%1h#riSnSQg{${ySc0(JCJ&= z`3NYMXYWu?&`V|Ruv_mRrgo=!zpp>I7fB*(I)kBva;Brm+dd=9jem*uhZff_n!*RO zp}b}ppj!mGfMaHDRF*#D(Ing~lLo!8aDw3~jT9BT6wtLC%uf5bjqA=>WKDNkK@=jRNC$CzF%Ba5 z?n@#yQZQ7VMaYX#`=Rka0wmr^r=#gO1AHH;I z6Y+&m{TlUmFjXMsL|PZK{$zVR_+g^&j)Tmc=l{kru^)?Tv@65$jr)510`*UHPfwrs zf{AX*T$n@s^i&Q$B`)8x&WYUT;W8OeBhfC-)t+fijjUz*=?oj1=|(z3@PccMnP!e- zQ)@z_GDf`9Q*t|I%3XM6F)u_5U-egY%;2Aj+S*ijr}ZMGX0&BkWbJ0{8ps9mE!$32 zf@bxZMuMK-xMQe1DdAJg%OMGwP>G3WGCc`dAaputGAGzUja2i**Mi^{qU?;sj>Ul) z35#sQ)%h|%rzIAL=#&vRS!oBA1Wki&Z+8BAZoub4?Q++cw5^JYjDL$v3UnGN$2{ht zRUJgSzoG;bRhqubyEU)~VgGd;yO&ZVV5|^T~GVBCgX|UWyj&#t$z$ zbAbemdMC*Qw#7hWg~cCeaEAAk_Z65#fWfaT!;b$9V=;EWVE56ZnQ%5A+#v+h1H?vnqY;-I?_0<0uLaXFc&xoKae;powA9XNq!Bl~W`Nt``ar9!3|>9MJ2dRw zoy$1A_e%$u6sSWnad8vg2RWy2frHpa&;h4!%Y>hmk{*}~K|LT#Z~yV?EUNLE{^5K&lzo`x)%tW^ zsKTxDN=B^=7R!ozE|cpm`J}V{dYX z1NGLnz*=K=o43A1@k;ZQjcatzcqdjQ6N8`_aOQS$q?-!@d@DAX= z*AhNU?+pFP+M?E-`pm0%x`mrW%v;#T3z+PaWqX%T{~ARR%6-pB?d+yaL}ib%!d_* z-kqZjoMiGP`_UHqh$N%e+_I!2cjGhYf|sAI&{1&d#d+slwaRt7cIUXr+QW6T2mviAwN2-h$%C+1$)L&2k>*1{63} z9j=vz0HDZ~-nclalEVg<@|*4aCsXH;IaFyPSSz<;*U!~-b2<^P^f{9*aZU)RK7qta z@WvZ76M%y!yUA*N0|7aF{MdD1U_51Izr@yLbjeE1pkShzv60f47e^bjfjS5#T8K2hF zt^R5d%v;YaUN2=~DX=!?byNk@ z$ahKoEel)mbJYCb*H?%U-v8qU*JXv+Ylw~X%!nl*uJ-(YvX8;ACvsd};umIOmn z`Y~(I0dap$_^8u5#owyuOg`Y%e~zzNjLhcpZR0=bc616J;pD>nA=%GfHHToPK|>mC zRtGBhhGWQxK+4L zjVhJ0^XeVHCF~4wjkQKho8D-kVGwD=jB2X_FY-EOU)|S62`E)lNGB?sbIM~^Xbf*!zo=9UF8Z(l)Sulmpf-x_qDJh7%tmy zrcny?A^lsN&mlXznC^YEQUT+a%Lk>YIdDLQODKJ&F(x*Z4Q z*f%f>ug6Wi@qCBx!7*!{*|0tV!fzV-d6uB>UXh3H-G;qnDAf17x*JdwCwfGi*ekM6yAcm=ee@E>Em zgtG#&8wp!9L7R2s?-)+NU&sV(a{8RhLpp_)pX|Dx(3501$0SWoV zLueiS=K>`M_Y{A1K1(3SXWm~ES?iRvBKAU>>E{GyZ7jRsI5tjzZB+kL6D>`ZAAE!_ z4jN#f!O>75{Xy%exAx~R1dXl0sZ$Eg0OC2-=Zk|4sOS&q(;+Vw$W*-22_x>&1~KnG zP7>u9zZQFk6V@IClG1U}0x@mRwKJBXLf?)UDnDVgpi8g22WwF>KmP}~0L&zvgOAdk z@p?gJb=>~=GSv^r?=W#4bL`?&!ac#;DqW&;T|-ryVdw~`vwpuH`17xe(3JsC1@ds# z-T4JO1Cy&#B+xyUM4F%@02n_J5J*9t7QC>f?tD&$;8q1T#z`evPT6dLc(-q$(MF25 z*up<_Sj)74U%0?@l(UC?(Td}%GW^VSDEsexR}{!NH(`IcM+a#g;8&rB6wrf}pjV)K zn`j;CxGb6jg+Ay*0o@TH^qnIlEE+vkBSn0=vk)W?1ZwT;4@d~7;yTrC2*TV9e>{76 z|0VFTA(lwGVmQLGfVV6e)W3-Te~^QE$YLBXtJ7AE4E4elON+`NX^WkNCmw-PK{Pr{G#O{3I7emJX z+P8omLX#w*&ZN{T+>AdP$N%ZZ+X?gxv{XQb_o z!|VM~x_H6XAr)JKn>p%y)ZLU{dai|WD3`ob_8Pl72JYdee}NAN$RxZ`_=O7wpxAGZ zPs1X-Uew0GaFCX%Ex+soeCdbnzRImY{WvIfTRqoQ4OR>^m0yBhjFpCZ%z?4Dc9wy&_2P(R3&M-m``h?JPd)-uOB4#%ttJa1USMfiCL7l(fL*XZTIxss;h4);#sZ-pjK zp^?YpbW~Ra{euhU>F>!GKt#O*!nR0cuP-r3zU7NnR?RFfr0eo}0mx-tN8dznUg}$h z^5h$U_;k$)@~gu~btI#a8MnVQ$^?dJh<3a$3iQq@IH?9T-^|@)D?iF@ z`Zwe(fh{x^Xcq^5I_U0>f4p7l<nZ(n*~fE>&L!TT4c)cbizCclwk`HaT`1c4BW9?J__)pLUVI3Z51(ViX0M5!n01Mg)5s0YHeb zUi^SM0?preAG3}%5S(gd86S-H8^*mUd2w*(+B&%Ks!lr}^we${oed~B+{@`LxvePv z-#RMP;554o+!M0PW|emi=OnPyc2tqck1b zQ|^E#x>j!SYTxuxpbr6ZSh%FIA2tpc(F0QiT8e6+wUrKC3sYpb%(PgvLq-!sZ^)y_ zMU>1{;g?@_0x@9z)1|hMy*9AUWY!m*l1HFo4n*dLKA2=;s{c)5m%F4P=xdWzrU8fx zATHQP8YVbb$Nz@z@d#mkCCRZFilJvpjO?YCriaiTX^1K6HkIpcD_6`f@7%uOXMO{Y ziF5b+87Wfqe%*5XWtdfx3cHTA>{@RTsWMW=$h5sD*;`lCviAxIz=b5m0s{kJu$U%{ z?x>B0yU}OWVDS@rj`Xw(FX}1xkHS_3c>VDi(f^*s;n4_{el~+d#8QSqp(8>+LNdlO zZqd1E{c9_))+MXeblaE7-5~CmIIWmX^wbrb5p?t$aH^Er$dT`EFkqHOjnpAb1GC_- zljmw4Ai%)rD_r>ziRztAY=*Hh|5rCa98lUP#oqch^=IeWHO~X+GwQp^}5(p(nWy&si^vRTa6-f(-tNDE? zNHV{6>BMkAAP4Jn&e2Jjh(jELqVw9W_V>U&Rcak;lV#sDn%Gx*sCX3q9wvQ!F6sva zv3&OMSU-qol1m-e*_6;p*xQ-+4!)zscWzuC)Zd&G+sl(rxLpgaI$nJ$kCKN4+~~ay zqKfrsQ0jui88%ti_VwUcpAZK{7TTW>kPEyosP!%>NZ8G0IO3(gY~#AtF3~RCS+wjO zvlTzkzUn$kb!+up`^H6YdmA;{*Oc2uy)M4V_`@Esy#3et&ENq{stMqnDs<4w(ynj$J=E~4vzaGQ|Hjk^nkk&t+kuKPSpvMuACCSo7D0ZB2F9C{!_qm>o~iqE zmd(M=QCnEaX<}Vp{bQ2zM^^zgP#2+HdAuz%5FIK7ENZU#0N4Z)-8@X1Qo!BT5@cq2 zn=F(g9z1DOE=`<>g?VjeS!4^hF}Wx(82tCgyQ>I2KhVU!0?rD8^X@O*rsDan4c0mk z!Zjz!?!3U`T7&h#c`1+a(C$31@vZvta1NDo)-%^lbr+|7$^{WP3%gx;#?3s;eNR~rQo(huot#RAv6#%%v_!|%7noxInP!!Doje-TRX0;QmgUJ^p1iJ6{pTh%4~w6l z;e$wHoLCn!jqPC*DVqPxpvAu7qWn{dpcelHgzd2gTYXs#AVs^&pmh17)?E{kp92$G z;jBA%KMtN>v|}PH-X6QHH_nc?lxKRmP{vt$&+^_ry1%u%TPDKoy_Ug4Ac16n=vSIk z3R-rN?Zww=$~VQ-5`s65!q#u@5mIj(^K-N>tk#=hNZ|AD{u{Gtz`@D%$3pGnczcA&0)5%?rJ#- z!*8@l)%~eOr{sP%_w3!C%SFlmUY4(Odu-SwUEcgBR|D=?E@1k8*;{@s5rhtu+~N+P zxVEX~JVMiliiaU|Xk3D&!m525#~cY(HJuY@LDI?VxTc`wd9eph4iqiItv9?>40b+@ zNB+CjX$N%ZJ|jCF+CefmGh-z#8~=dqJJc5z?EI0vh%RbZfLMckI>WYH(kqt&T$~p= z(0lk5hjovt6p6v-hN)$DK1)$p6CQze1|H@+Wzp)+H>_#t{#pzM_wSwiQ)YuEB)5r` zmGvv$AAj!LxrTqY>x_8l#UbzsI?&L37VyX*=zeNFNzhH-@6%Fp6y*m3(!=|Y-1yBv z@Yghq6_C5eMffC9C!~z>I|%8W8lu&J0Q+IN;-Oa!IfpVIfk({0$C6SSl|8*tKLeCh$o}C$sF+6@#2T=bTOJHGQJv{$q z?fMBNgtUZB>DMp{#Jfa~-U&;10}6XZ;8{L?%X`8$8BN@SM-$D`1Llq@#EVLi5TL8< zPK-|5@a{D^O9I9CjTr>Xk+b)!m#H9J*)ZBP51qc&2KN`11Tx{)^!vrM7%u`=ILPIs zK-Ow@edIixo-U+I;cORgL7J@dcI6791_BSd&WvN{))!cR%X_(;kpjHEQs8ny2Njx$ z0rxkiOn^cCK#Ti-S6CS2p8xUhbE-Fw$+244T2cdCv|%=Fho1hsJBMvOY&4(2SZfgac9aRKS}c`Xbu=A;3}y;S^N*`m9Jt z<*)!xTRa>*2R-{37=nHwqw94?Qo=#Ja<&S8FgM1j{)~#%W zEn9-!%Cn4e5))%342<5=HPH+?wCsSKSRuBcHTmQsa*g)R0f!Q@rhWtyNAvmMSc5y> zA(6mM{ z9zIqOpzQ$j6|_;Ub!W2}8*Lm#xR} z=ElEC(hJo5LkVgy$&dISAj*@J*w`b|KyPAOOq2cTgjOH=Bi7*e%|U+_t+r=jbU`@v zleOD^<=2byB&lI#tlD0O{u~zY>qqWd-2pbhhQZqmnDNfPzxjiC9=L}=ZT{=~l3qNA zJ!@k)D$Jd|5(#LhU;py&A9Bkt|EC-R|L`UPDlN}Wz|Dn#`3%kzNIqt_&6OicMU%^Mzix^4 zXI&%T)U-q&O(S1w)Nd?k`yDCwsrE`S(iiaTZRA!mH{gKV4Q_xD^0WSoFs^7EB44#pwNOyC_z5xwkdtYAi9LHC?))=dCJ0*wAgbR2k0 zvQ3(se_D-^nh1(RXm%PZ9DznY71H05>IrxgXAsvE2JNA;j|-`P#h|MR`NBbof}L`N z17(G>`1xxoJqh-VWbm%Ezw}MXE*-J@izxc*y{RePUuixH#m-P5q&^xob2ag}~?A^(8Dj+iUq2HcE$4_3vE59(ddL_H( zTI-dWeM-wFCVvJj-(^0*a*UhXs`LKRACD^qg7`b_R@alH6Y|t1D+Th78s<#QmNZES zVfF591kAK314Mt%#6>;e|y&a!pe=ymv>8yRr3qD z_skx0Y>Cay&3?l(M3rc6xF*f{CC6TWIawuR_t?7Sda%$)P^Pe9oQ$c?Rjd%7-pDB7 znfIyArS()ZGe=_1RM?oXxmj5S{P-c8sV@}1 z3`v_rKi3DFt}tJfw(om=t^6!5CFMlN^BcM*@U+3mzzTNgX#T~`9X8d$H|E4}m;VR6 zUVcM!xU0PU%zU1I$jeI=8pA$0_!;vvlSJHNORmjwkX1~v6?L2A$fuU{(Jd>~P(x|& z%Yh4I6Zl*+AMyiD;ZLE_=fyLfK{(U8zURgyvwGdz4<9tMIzq_VDay^gzOZQKSa0iI z{d_be?1jElkDBJif#~JsnFL|gVCgu~;Kz@Nc6h1ILx*bK5AzK+I4htFuyaS=rHyjK zT@mZ9?9j)Ln`LNc$24!Hsq5<>bslXB1)-2UrN8{C!0F6q&z@CdM;k+^>sS^L3JVeoWi)FMhr6ghRRpx|Dif*BKRRMMJ%SjI8Q%=d}3ho2$nN?T~vd%D6x-TMtSG96EzJUR2@ICIUL zeN`syH>*tQ%AZfy z_0p*R);EpI3f{i^YP)aW%&f{TlS#9~HC-#z5Tmuh*B+~j$6S&!NG%z!f6DGxoY?; zS8HqO?6>(M{T`;44J+iTncg-vEh!RQg`tMFb((DJ={_-jnKi|wAzZs73uI-&AUI0;A{>h zD;OPJ3zK?T$HwLR{blS@@C<8Cf=}JMBcdy>^m4j}b;B~9mNRv;LcA&Qv0oL?OKxGB z+{+>5`cDqlML8pxLhV{DYIQ67_BGeBs=)L(_uyw)fvcc$Xr41^dg}j1gl-(T__!&M zlZ7NR^wvs>vDxRgUITm`?XLJ!b+z`1_dH}*mKn(`&)^`@VKu|%c_u9?FdhS&UWM&Kx*IFOr$E=LAHttE zwVp49X8d7Z6T_d4AaYw4U-^+dl=PrtztKu-Or*fJBt5g#)S1@cxuHZQ{M&=CL%WZ@ zS*!hg$@IgAi&3w$3=7VUTo5Y#DmQbFylXICgA`uAXlF3&O($(x)Mn)o(8HZycc1?h z>q5#$PkNN8?6QaNnWOqG_hp-db7Ks&hKu0N4?VIb)YdJMLWLFL+aNO@`&^?Qe#&%Eu$EA56cS#=MA=7oUU27Q^jqvNbqy(X z{5uu@bTrYAJJEaPpLWvcE;UXz*vssxg_{}fmMqojR_q^5@b|m09w>VPx_Ih}Bz6hz zT75^?29Nn%@wxdnZg@~c$@eUo6)#_gYry9^l=b-@c_GKq3s+T=QoW?6< z(*~R63i|bQZt(KF$Sz5`qokx%R~3FpQ*@y{lPTh_zh zzgPcrnkndj+R=RSt}P6OnN>{Wqx<&3>K~+v^@j)~F#YrRv392H71thV+aI4X`c1p$ za8q<*l>4+nx!1L<5&PB@rKlYH-XbBY2Z|dSsYBKU7rG;U#;zZjZo|fWraYSTUVAqk zo0n`)AIcmWnFz&gSUbL$^pLqw77J~b!GuLZ)I2g%xjC#&ILOX z1V|@w)o@OxJ-$*s2_Ms0p9Y48Pmf66c0d6W=ksNnmfRAv znoPorG{U~VNPz>}f94I{#%S+cvchj_Nq6p&49&cjRCFqkqyPA!e%LJZr03sFv3~4& zRbZWIun?YIlHHmQCt-w!(aP*`?yZU9;)l!=mOtfd-Eg1EQ!{JmTI}AGP5{o{Oo!i= z!ZUwfpsO`oTo9zpVx}f3VXCfMW#0K-%!(=PGRzWA3@@L2b;Nz>Aqp*6`~0krOHb3# zRuw_Inx#$n9^cyw@Hy*VkNOxTXPJS(%e4x#uAqc|7(0ak* zzh-A&REH>XL8RoY5YJgGA%W3X7!BJ-h_@K&TrSU?*sh;2ullir6$O^U3xPz-iH{4v zR1Z>baJpy1A3ugCM$(lY`grO4({S91uN0Tpjnu9Rrz>fX71wF2uw_O4U9m5&3a`x= zSYJA$U^8{!dsVlz>&B&J$+SSdIZR^G3~Z+0{F?lW>Swjq(c^V!h%?uRJebc7cPQ z`jvq7(Ij{KuO_F|b+wTMeV3nHbGuf8Ub{P@Y~ic*jQ@46kjlzRTcaa;(UCFuh~1y% zc#|aGn5Nqmvr>`h~xivv~vrGCI(Pk3y6|%!BY?jpL7M&`|)_O%G3rVU{dv zwGMO{*UZcfVQJVk)5|tY%r3ibV?jyx7#DZf8aw0G8TtY)dNVJxbDXgG#tBgOC|Bs`h%f&=cta`#)0Nq6Cl-G7Wi$)g z_sz}$!F&%plzwLj6Q--z#KcN+!>WiDHm1_AJ4D*ZMw_-em^BJT_iyyH-CI4yvZGsFL0m2%gW01 z1Z7W)iAlOnv7yrvg_e5#kceGt6t*1XPrp0es(YC5mLwCO8`}EcCs;^Vy|RkwRG#zT z%WbE1*?BnEWi1p*>jcH%{4Z1~O4$Ma*|}s%;(IQI6L>+}&?r7qIDVa)Bh`L`@Qo~7 zn9JE_#a7Bcnz~2AO~tJ`H`GC(A7kuFlUtV_KL+o#m;jq_VeKNt-3D7kUMMv+(9l3fITRmR zE^TEFg5L*w)Y*PNS|3WG)0t|^y%zQ|nFqeF2b0D5lLmiC6`mfwaJWlfRz{%Mi zESK{oxivL4GwU>puGE@}(Te6|A<>TLJ9)TbI>5^RO4bx5S125rY*_ulZUK8$ZHZu1 zc-EBto3c!snhy|84v-J&YLko|#xjG4!kZh?-)Bxf3pC$26@R9PTP?17&;K)e{a!bm zB}FC;0jw8?Ney0S4v9;zXZGQ4H0Ss^URT&NOi3^tebJFQ^kIQ90!QFXx9l>S8-lR7 z#*z7I1z$4J>Ho3!-f>NC+uA7VQkP}PLZvqaD+mZkk**>LC>E+z1(gnl-oa%j8jGq6J z5I*oc_=+!ucw>RE*5ZNtS_DM$4TXaxa)Velp1Bj>9DYhPYVP7leEz=p9_Lzw?+vhPHX2ewwX_NB+s^BHoLNYalNnQOyUarR6fud8%Sq##@N;&7s1Ug$xRm+3i zY+^%Yu+I1g+(rt!fOWM)(wAMau1Ah0h+eyg^+W}yepW|t@N zGBMYivi0Wyeu(X5jE($T?>?ShoSp~@CxXzn~u{wCJ&;sVNAcgHW zbKme&sgKyA(UZZM5m}Z4%Scf-Ta3Y`Gp*hCi8y+cK9EUZn&0W&CHh+J+u&@ANHw-3 z-oL9@9v3XLf1i(V39bV@cxl1%J)j@Ilr}i^?tV(zU}l-eEl2Tmo9(@RRfZ2zUx2rD zF;Y%i>#}0`ZRY4`Qq(}A%qjjk@XZr!tD99O8$4IVnWH#v>;(nZQ$lD%^QzSi*Vi$G=qvXpYl2b^qDoaNVVIrYT%JtXoI7oGeuznYr^ zAS$}YGd)Q3mDT(RTx%7?~J^W0_3+ueuZ&B2%>f2Ix0R= zF#Tv3?qveFq3JY7gq*V#8m61uCG?xIsoooYe(yUemr%0E*L?z;*iPb4bA)bs3p}+}QcU&0#M4rY2w{gJj@cKiITD;&}?m1Jh zOllizuaB#Ie)ccv7*HQWFmi#~R`70}j3(}J?i^`)A}J&!v9KrX?qd zSJ#3x3SbGCqfHj<<3q?PKo_^#kO%5px)IuuCA2qmw`k`?t-t%mldr-|{rQw_jedV6 z1P;|?@wQBkA54S{OJAv`&rgY;i(^ew%(p};Ri}*w64u0VJQ9IRl+t~D{tH>G~AIo zp0OgiwFRPX=1-x28A!rE=OO<)x1tU4fd57U5xrMnqLo4(z#xC8;V|8fOvJ)KM|Ucq zi?(ZT3^r$(RxWf;njJOVmKpi?9BpvyqYrO94EhbD_?Jxk^?#w!-v{g=OY4gT9CW^VZopv9%M)u@=v-gq zMvkvOc<>)muVoHlL5_|bD@Da0-Yd!R)tl(rVjDhwM9>)`{4d`n9WmUqWAC3E>G=1! zsoo@I#c}~`H$$qn&ojr7VD&(Dbsu1w(fhsNH}0#=*xNdyQ8u34VSH6GLjT z?_Iqb4sW8<9PT-;L5%uN58;R4^gJGWd;1rHPk#HS3I16arv$KR5nZ}g#w=U?E1g2z zHCT~!w)sz*lCBHGNtJNz7mn8tosbw_Z|9_=WBc^0CcIMUsCvL;Fc&8TNKJ;OKA9?% zmSC#B8wU$UKCq+n9Y=AEqLr#tQW*GZGptZm7|b?Xys#uu^4!tw)MnZQ9s0G)c;Re- z%zR;eidJg8+NNAy$SBcC$D$kbFHZ(X6FePONWl)KwW8q^ehGk`@oK-(Iq$8g7&YyU ztzgZ7Guj1lMvEbBpz&S^1}o7L{J!PTEX1>(Sj`osUNa>kB7%in#*N#`Y{VjZXGJ_w zZsCTdp1Up!-3<>ABg5Ssu4iW!M|9%c$iGy=&hh@;c|><*JFv1pvGeh8^<3^Oi-Qo4 z9ON@}3!QB~T>o@rxPi_$@-vu54)_$q&P1huEWZBnqeqW|gCC8C*Z1t)9u<%mIzK2|R@o9hipD48#f2(S6w}i^SIh3*k{ex=3{u|OC*Y)WMS}b>OW9)f< z$>I2r7lK>bYBy}s|Hvf%cP8kv{OTz#ubFyR2u}SiOQ7z{XuIoxFu(Q(zrx7^Q7)eq z(!cotZ3_O^Ds*Ee|Dm>#KPri-i;?{4yXA%8*0!LHDcFD!1=^*q5xexhWw z)^+ZU*}oYk?c#JB2J64N7wvEVwJ?3B#e3u*NRAhKB+XLT|v%dl*dM*^8Zd=;RSI=Xzd;W#syEDK4{;5VP+m0D> zK(9`y6zt#l1lT)|!ALCFJBOtEA%Y1;JK%#=TfSCQoJX-6WZ*VlZLuFee$<@E0=7Ao zFpZ4g9walLd%Qac(o>Yh%gE-^(TyqeK=dMSE=fKR!qDpfH1C05s{NUOAH8~I<(pl- z%&`4PD2z9jlH~=#(5&|0^H7i9Fo13)?H4S+CdqVSL9K{fEB#M!GDbL$vIZs|^Tmx9 zM%uIEcMYLErx#*SRvU(k!FRhH{m1D-bP74%ltMzqQ$~Isp4iJ7Q84C7G-gQI2;W@E z1MAv7d8cADqcqAHP?7;eOT3rJBdt07k zUn!+~@`d2vXEV|r;NgK2%KB(qi7sT51o7L$vga!P0_Yt>odc2*5xwE8x$d& zYzTLNO2~xY(I{7$5^^L1Mi*=QJR~H<(e3YXd{(~$BPT(y>o`~VsxAogdBt*HZ!hAQ z=;Nb?diNRrOzeq-f|IsS*FRA!Uf|Q18ZjMkkqVv1O(?7MenJ!T6>TnrT$M01$Wxmq z!OvAx1b65$smlR>*lYO}{Q+>Y;EZUd{SS zp>;>Lg+{~qXOXI^;yAL#=Vo@>}RX_TkH_oeY^-M?!K)c6hJvZ!`&_V|zq( z8WpzaeY$gLH~t(L;tzv=LpuxbBcsarSQq&fLD8pc3|38`&@Gw#CIxI&&`-MCctC5C}Z&@Jmrq5lCU2$Osl47^Rst{e8f|$V9?pLL{Q2aG^2No)P^g*A;@P=t7gzugMoitA?YE1R;hFa{ zGA2anj6K!GxL2^HrljUpjd~0xgvA1!zIx@#pKvN&>c_I5aDP+qh(KY+A3Ju;4Wzlu-F zmxyQjG#nOtkETM*O8duaVO}SWlr%^hzdr47=VG{c8g8*F_zV~?P_2-8P+FcFAN4$; zW~`k*Jq)uh3@2(gjZT&<+wyVfugMMHY@fazRyqiB8w)jyp_az^_P3R~!Qy^!&UJujLl&_#w9po14k#4pFiEHdCMZ> zdUs?I7Qm#h{7Lek(|H(T--CVXettDyVj$zDRk&xETtZn}Ml#lvofPHHNAwcP?|%_6 z5Wp&WL|uA3+j7)>BsnQJuLmAA^p!ql)vkE9#qz#N=iZZV#clLpegyP#EE&NsapnLx zoU`**bJEw_;pg-srMQX=3CtIa;9L+B(u;(wl{73E&oeKLXY2@hbMK>YDw$3BtGIs1-WgtC-g z`cT2S*WKl8>J1E*%K&=wcad!bSk zbyyHMC7N~1_B{cQx!jM%ox7!2zT;@bpO77|{{4)y^~41k@0A6~i9BN83K<_<;m@pW zVKK~Pl|dhjU%KSEtWOAtlHEfPoZNnjiy%ArgTfDL7_WFIX6pwUtHIj4%++z6yULx-c?3Ffs!)JLb zlKN2l0Exk0>JgH{VqWm1!Z*J@2v}@9SBMq|$^h(|b6rs&>y^H3vV8|{u*@}6oqm8V zJaOqWvz(v>LwkSv)l_~3+OUKcY%ZkpN(8uqtGD0>Qlz=$N#*;Ji6ldpZmj{5D`d2U zVFUrK{9M6*JI? z=L-peaA1hm$YxzoboaOYQT4kRP#m48Jv|uVI-b+o2fp;BOZ>-eV9U)u1MnrfKC~*$0|w2-L1+2iU&I5piA8YC~m0XG7XxHP)v>@X-Kgtj}V# zu{L`N7rGz@$C^{1CH1*>0`lU)-$Rb-cmU6hqxo)5sq6+!HWY&+$W0$6pSRE}+S=NJc0LbE8YgVi z^!}0|hxyhg~&m-khQZKgp&;^~W(jzNbhEOqXEboM}K_?0ECCy96aA zhytL@9~M<4%5(ZVi@dj|k`2U>Z9f>~i@Xvtj>5>D1aUA28{9uIaNgV7JAp_78N2pw z=xJwGB9R2thx+4#L$1_9)VWuM6@I977kuwhXJ3z!l6U#;4U9ZddZAL+-NdtfpmD zl&ps>CLI4(EVaX8;N1YIw;=!kIqzb?9Ocb4s1X4gq=?+e1{IDBYB{7^ajFbN$BiA_GsKKIMC;YPEBJQnNkspUEeT40lcSb?Q5h`JnZkbL*BT$?7E6|By6 zW^@qk^KX_tsyNSV_%dq^Y*vk=Tr?>CWMb1)Z<*-kukfo&#dhw4M8BGMzosXO(G>oO zl$H>4UYZ%kv|KmSx^4_cQPe0JPzr7aY-A!AQVZoz_QgR48N@6Wl(<&9*_ktE_QUT% zuje#Z5@VryUCyh3hJt2k0k`+J!2SxmPW_bHeidm>38=scyw~Rl*7v)sH`sSG8F}`U>K1!pT-F4-4yd&u|#q+=C}-~RH>kJr!LJ#nmN@m~#u zN)}8U$2-JSRaHg4$}qMj@NMUGyn-Xc2`q^P!nwBHQjFp$flu!do^rBSHRRfKO3rsV z5I;cuQG!|+?%Q)-E3rMkDQNR61#=Z^-Kzz)Q;1W*6u_@E-E9V{LYr%iy+8Q30SJYu z)rjezznb1ycH6vzO6!QiW*&}ce>h0%6$L+*aTE{=;qzM zsg%@RaH6m;OIS$AF7<5tK;*r~;7YJ_YFQF@S;WWl%``MmP<|GcYu7)7rIj zr{(UIM8YF!6NdxX74cDIbT)0-rrCMbwf*C5Ry=mDrM)IxB4c_M)L3eRO(*t{goFfA z3?p;cTi;&<+|`MegZM$L4`uDDt}e^NhYznp11rOy6NFNqG=d!f4Pk&Ie1*FNbu$kz zozVRgLM2bzKbh&UtLgzB{1GKv*dXw|TB~)_NAXRGX{#5JdrR0ER$-Mv5nC&pJM9}pxV@2TtYS)W<9m}q4WTsk)~$%N36C@ za3xSceGs0J@miYMcA5(zeg#A7tSM}cij)>&64c|EEH34P{~K7iLk&5RS;A0fe8FgS zo0hbV70%mXDh8>o6NUw+xL~ga-2L%tb7`^1JoG{O$N?B-6utA~+t6lRVGbuK`7?W9 zTHrDnMKS_epS71J`_*tR@IAU;8OYA8%;k0q6mKgF*96$TlytQDmS#sa8mrD}D05&G zEmuI*vO@pQr5cBF2IZ!LTF9;Dfzd)XGY~3#1b*S8xsf7P$}~sd!e?z|2&i61avB;M zUW3M5nGP%ckW21+e>Dz4nb3lhcE>+!3h!ULuLg-RW`Q8PJzV?43>>{fCuhTzxdM$Z z7Jb4w$r3x0aL}R9o``2`J1uj{OX-Bs$5Tu~*V#0D$QMG%MN>b8bGOXl`WekqSL>IVyr==k9(I4tRVg3|zhe#9I}n zzrfhVf(y{xVvJdXAxlmck@;#Q0Jb3$`)zpMrDkyyE0CNy#hJ1(?`8c|jT<+--&8UJc_9}(@_H*&wS-&7B3b}4VE4=7#C8NUE?%iquZM+tik28i#c6nXc)@hAj&GI_UvV_8mpG)N!QgI5^waMl-Lu zD-%FD#my+-8!yWnjep)h4gkN_A~CWIO3 z9X~KzZM-IuQ*5PQb zds~(&xZK>DjV?6mGJ+L?>#!k{7-r<2e9b)D~-T`hDt>2h}BeAMh1^%`hw! zkNZTyL(T-D9P|XlRfwGECHa(1g7Obo0_^`>Dk>7pteA5KK`$X8fwl4nw53tase3fD zv1e9C)-dN97y@>1p;)^;0fREbnV#Gz&zD8;Ue@xMzyDTKhKm`H2%wj1jYmN!q@vqi z5lLV+QpTAKy0^-yARY_T-q;FMWNF^%9L*x)T9+$?qN%>y4pSqyJ0!i@j*}qhdmdGV zGj!9VzPi-Lv_`8D79DQW#`~(ErbuI-@OOT@(E_7EV-S(*c*RezZ0J~j9&Hj-ZLCCh z=gVJPvp~zCCr;3Rjl?8a->z7UF+Q2)j@R1apjB}cySVi?*Aj4Dk3q-;ghLm!T8`lw z4-fCNPtM~Vnk*JzHCQ{}$HW`}(;}LCDHsm!;#6{njc+>KkdOu6M1qT(8w07bTx8VR z&1*9}H#_j^#Pw?yZaMj0UY8=ptGz~FLLqiigytxId(0784-PK)SJF&mUy{r}404zJ z%tFi1qMlxDph-bE=h5Pb#{~z zy*kH2gma;?ac5XDBRJE}C%jezj~UEqdzxP9mUDmwrF%hZtJxu$?NBN8Q09du06T!- z&D%m#Lc+uKO(pqE9De7Mb^HYAF>mVYF4L)>U$PvB8v;&_!qbd2ZJnAb6TN?X7t;y9 z9s7T zrpmOq9`o@{j!{yLSNQ;9nIO;06c-R+dJd#tG)MQpDc;{Cr^mZOARKbVFgP{jUWw-U0AHUWt^`Xbs#?7YAZPgfP87dWQBu zm+5|}8&eBIfMqw!$pg0e1FxF$%i~o-Ojeg>>ytH-P;3Wx8V5T{q(`(H{h@Sq4tgoj zW^c(bj+WvQ0#l6(WDmp0s8fn;p>kMN%yyiVm&?3x=r~j}mrM`*wNMqdtIui+G=1(t z>*SQVQHlz0GEI+zzXQ-(I!^7Tjet>+q#o8ba%UeC^fth@7C9@+m}A+@M-(4KD)Gzx z?MaEj>=Nl!g7-icy>o8l+6x9%S+4n(mJUJ<<>UUOjLGbI}2;;$% z6xn=MKP4xTt$KE-p4WRe)i@yvA1)!}J~wuKu(Kos+^98Duus$J&%Y_G^1=Kf9ap8k zqJIsripN-1r8uWD6z_AJJuot>c^j%{1VK9FHzw2BrD~^{K*&9rNI>56?Ppj|Rxq5G zi^cE${0tVhidj+1DL|kV6eo(b;Ic%$7PUA%_)|u_9{<6veOm{-3hxS=YoXxgXg-rP zA(IxS%IGs3@>)rVXvlZ`=0>nn210=PjC+FGy2dkQ9%z<)00W5wpqU!sgF-G)i`sm8 zH#-BS+o_d0eMaD5`$5PGwfhoL2#ZVBUGsD<#2s1j2>&E+={>WZQ`5o#vJfMh#58OV z8-!g_L=K_NgMQj?Khj+|4=@K{{Z8IZr#6HBpbo&(`u#XWCL~258qkCxf_A|F{xt}~ zU#4y#iX?J0MP}9$Py~s{O_7{k0dFA1(K-#t%(s;Ten75Ga_dHv2dT-9TpMxd-&JX$FM|aBmo-%11i`j;bV3F6u|*+?(*52 zT3!CfNT`l6=#6vIE*ELZ%)hxH(Dgr7{r{UoLf1s2UN4e(7I^1}`w1?e>EX6Xe`CL_ zFp3>*QkS9~YN5f)m_SD5FYuKEv`IaKG67E$z^j0|`Y0&ArmOq;sual14x{TnyYoLO zC;ac{zt(f|-;|U&ymeQ8u1w;fBpLD%badu-QDZ8q)B=u>PyeJo(J$b2MSi`1i}sc- zX6^rUE&sKe@Hwp_&)0eeYP`zhN_}KHu$}THE)#wz(d@fz79oEIhSI#@f`6UAqEdS; z8VCkA(wZ5Z19e)OyJ74tt;CQvQiatGPGj`m_r%{_3h-|1LZaNvg%d{8dV%DXxPI=d z2HA4X1TQQKmIaczG6~G4D8TBT16jBiP}9Y-dY@Yf$_K0>Aa&XU%PQnef|KL0-m6Q{ z4jzsAbQ@BUSNp7ay*R}A6l$Fl#xxW=v0T+9HpKzc~fJtHXc;rNJ`2sO#`q zdkC{|bM|AzUwg*mt~cV37Q;D;SJIY>VrnWd)C)eQY8LBPA~{vvn`0aL2mGKI-3sy(yh@BmdE_`jn{$AV>#|uR(M1N>x(*9uDLZ@&IX{P;-=Ej4V z^_#BIZpuD0hH*Uw+Papw&1*a_UTr&_uhg>TWK`#Es($WNqFt^2sxp^R(qd<}1?M=l z)(n5#L>E)5&LMfDFaYiG8=bWI6CdeZ9+Ijdko^j_!of5#S#8)Fx%$LQqmYS%|Dty) zVW?I1&GZb`nz(8FYoq`BWd2n$!-5URk9_(Te zNJoTYK3t($ef5{hA0k5Vv5orOku!uImQ_5 zBsYZ+w{h$PHz3W~9fo_<5W@aV;arcobF~`LvzWV5)!UWJsDH@=E2yOo%X6KB+-#&0 z23hjiUw&+7KPyv*Mt4Y+x;!ISC*$0;7ngb;=NF4WB6nc@!L;1FX`A6YO?KQJyL?Sk zcB^PIb8MsYw>{cKcR;@% zC@_S*f>dSY6Y=%588ZKkHi{_PC}cJ85I($&NHc7#rT$HvtxhGPQ!{;dR8Q53uqe$U^q9Vu!GAGA-QH3rV`mJK1s}b&x-v?13Z&7H* zK4e&{o&@8#xHg-OdMMF{#`YJujIs>|NXiyCN*$si7F2Eoh^r-~nY;WCiSD=c0(qz_ zrlT%|_}O{6vd3dGp^y_q``GbzmWt6EM4`w-bML*E?GSX6~t7@?&Afvjri)W z<1b%@R=_z=+^FkqR?khh&~7p=?sm0j^j5fpUd|Fw9l1IUvy;BAY())L$rflVwQ>%w z4ToXgn%EI|VoLx571FHT+Pu##7Z4QMAZ>{+uQphi@YF| zZiP8szdre@^GHI$HWwiC5tN}AbaK+g+zWka9UwhXBJac{?)dFz-p`fiZ}-AVT|Wc6 z!&eDSO@v$RkgksA{_aAjz|_jME7()u0=$Qc-i>xqR6!-`?NItmaZ77RQAPr8AigX?y7+T0w1t z0Fauz5-?_KguuaMe%wr8zk%9v;+8Om!8k_faJ> zusOArT3e_EKzogWA?s!tfcY0%um;Z&$k(j>4?g5l4KtuQ3boJ=$>PXEs?vg=(Cfcbw?zBo|GEu>cA5WsLuAouum3xxRA@(= zjQ_H_ZrxWwpKN>na>YU81En%CzPjz57{N5`Tc=GJly}q7ebf8fAJfnDtSL3^2~QBV zx@xf>5f~YFV0#Yl>6SnGXl}tE+oE?{9(-f=?`p>ww#z~Sfum;7+{Jp`YVqx*EYC|8 zKXi53D!E?lBgos%o0HrkAHv$@QjHfHiNe;3Xxpd46rH`*twT&j#WKt8*H3sC9`as{ zJW{X&RI@cNAq9O9UJWl-^(7^bpNB6r`e^M<^{1e=Ur?{7sJ=V=I%p)MRsbLI<@~3Q{O}w$hB7`MnUZod z80mU}6kLanCw8In66r&FP9QFpkrMR6$5xg0?Q2jCi$3t|1Ij44U56;fE(*Ry^P8-2yJadwxPZ`m(dhWf_v9419$Ip`xRP zsyPttXGA6EjuTy{o!f82@&M5CFP`fug+`>EQF?lO2>50)Zg{pYP~29Db5D@p!cbgr z-*8fN5R`b(+)h~XgEWv?_kJ8wbR-BFRYHwa7@EM(&U0gXk{8K7|9KXoPg1U(qg+dCnCot@@%VD`p$5eW%=MQZx2vj}OIbeK~yfHcG?a0}q_r zQSa4JV`bZPNrVI)9pb4N=Fyy^lcU zAC#bbj6G+Dxzqtue#p=y?qqLRh)J!e1GGw(HJ8%&3C+Pbs>hSL zk8d4>Qdx?Ia-HObI4*olptcb#bOC3};XQ2>|9YTZq?SNeZ=<{L z1Cazy6veiXI#L{m7Qt?G5G%Nae(#^RV88y8xc5$d`I-uP}MWx zdbc5UH{BH#KptIlsfDI+jhweW8<{A;Z*bV~dBmO%kB`3cMvu1?4ynzE>l+nD2TQ8hK|BPOj9*8^F3Xe+VegvT97;AH@ z7WgAQvurv8TG8K`ii-{beorvc2ZT3&(fcGq5ycM;0ctCnA5j~J^gtE}>>i9HhrAN>qv%am>Uati6_#J}7rN_etUB+oy`i~*OBGk54c=w_KU|~t` z&%!DhJ1w@&jIsmY#5XAL0@AK9``Z50G1=Zm?sXG2^Nm+1*bJocD@UIj_E#!=;erKcX_ic8WU0G z4fMtWB(F?SeJ_;2N`phlmIDza$TvY3fYCtr{wn;FTA9`C*UYeXR-!KxuPFT~Obyh?k`qEE{VPq#mbXdiSSMamT(HdHcM zGNxEiLx*Ecw8!2I&5oq*`mp#Y7bvOs#x&0A7uDFpHqBeaMdJ>KtsdevkOIC+r;!pZ zXb6lg5y ztemz#goXf3-&3na!;ivk1C6^J4bupPfUDPL@jlJfZZ{eRL!7k0cVKwlw)G)&toOn4 zmDkG#`@t6do_ApyIv;USHE$2kQK^n^cu*Bp+m0U*v${%y+3c08@hxAf&^<2Xz0Z@} zMlW^HowGR(Ck!(qJn-dl-?&SUKcajs?pfdAQZ6!*>Rg^CY2LStL4jO+vlM;|h^=-W zSB4_G7BP52F0(xp?X=D5Aaj0U#%^Lr7a?*dq{I+8;>@_0Bgfs(-oPFJ6k#iZMqcAU z)|NN49GA=IsRxsLM}AGSXsLspb+iZeJA~H+G5rkPDdQbobtyXwky?9gKVKfO_zHMM zkye7R`#^t&a+vz#a$x!MM@<&9E;C<6Kk?Ue<`cgNJARWu?VuMI2I8Bh1xx$C3Ov$7 zB`beO5Z=146+b5=Hzj1i2Y$R3D{LTWq2iD_LC!dEPF>nE9UA23^YInHB(hIHKetKP zb^9|Y1VPD!5!F<=7 zYk%p^Ouq{Af3qW-7TFRGsi*Yhx^Q`l=3wY|oXBshY@d78tB}C{($N z%v=r}X-d!l1cA-yAs_4Lg=>%Jo`42`FPZSHiGjg2*!F(jm2)bG zZgc_@1!g&oD0=oQw#!+0w^VPQCI0d#x$q*|GQh2cT83A)Cb!aq&erPL8Xr52PwI1d z2QDEs1_3I%=-k6fP|CjJ)E2!PNAFU$&?j)w54B=zvC6eI-;w`FeEV-pds4mmk5DJP@3?p)GY+p-7`qV177WRF(eHh(-w)kzRs)rmJNH+FN1 zND7wR(4@u<3Q8rWr5UbbeeYfxHCppVG;(OhLgfn%J%6TH@oWm&l;#n75xK|UdBuPI zv)J8s&wXQ%3%;;0IXguvODZt%h<-h|WjjY<#g!SLXM3xs4s!@h3dJIcly%PHRE|Oj zzaaACF44s=lEA@Y%U;ZgHmQJK_Mdxt?u`M6fo^7p?x;;E&?3Drxe`%%5VE{RiUL02 zIgC=tPuK0#?cXFDL!*i4IxIe`&0;QT+AV)qAK6PJ@3KrLhmy5>q=(M6IkMf?N06Q`0fRIO5v@R?FJ*f`s$MkWR zH`hOL6Z)1ki$15vzO6OXfh?76-f)X0C9z7r($8;LV=|3AFO7`XAT^HmYydf;q_~+k zWu?tGCUgGgLLM-zLM;WSY?c-ms(~-|&0rMbQAUl>kM_tHc5r8BfGW}%+1m+}N#=!i z;kx;jPAU8DmjB{zC3p`OQ^{`xl1wBey-xeAb`psB@+E(wM}iV?t!1K@4c1|)KzRkL z-^C2o4cilCIgm>5t2y`_t6ju7uU;Vb{Mi)t^6DL>sU;vkG&EfpNZhf{>Qme9bLvvU zvt0C-qVp?!r_ME+-3hov5M-1uo0+ij5Vv~8P=sPFq^cLKBaXYKA`L^1Tr$CSJNt8B zYkGRekR=7Z3YJtM4l;tTc3xIe^c)OmWE+b}1X_=s&CUVH2+vQoX{Nf>`OK|yCikec)rPgDY zHF8>JB2zqj;!AX3k?8`%QItU86+h)>z5xgl+gR~L-tt-#5k>q4qMoqIzvk{H`>~MZ z%S8VA&85rBC*j5<#PfjRI(6iO7%0a+Xpz^DPr*ZI#?!9fMP&1K(EhC5A%zePlpriL zHU&c&u?!Eg#CiQ6qcuk~yW&;4ZJYJ8$|k_lm-yhapZ5NI z>Z7w7G#*lKW59MIgn48gZiZ%$>gf1 zgy%dWLQhj|mvvSIMNM1AR0$ervticsNOpbDbZF)&ghUlZm6~J#4Tz4UvZTlr45BW{ zxW4USYm!bq(qL!1&LhbEo=Udh>z)N#g7sa`(}T74ivn*eBlp<_p5cQXNaJn zb(T$`L3U5@?5MnolV2)}r0Z&+gy`8_j6VIreh`>`!k%6%vTe6i!nH`J>+>_*B>qQq zRh;raUn3DrX}qfyS)Chp<|y!RCGTg>$iVa<{fg-D8mH$W>!TEw3x+$VL-4Cm>9pkt z2Bb=4jcIMNmypP%gUTLOSfkdvC^cD3tC|RhnB>~)$Dpuv^H=FiJir}r^D-Oxj<>(Y zJa2=*W%@M7Nr?E{mXMN!`}GMbTW|Ak^Ju(}gWk~8=^`-`hofDjR%1@n^gJmZzQ=*E zneIe{TxO9^uOfGVo{g$+m*nY#fOD%wcjFeEmSLSvi6BH?eP(2B?DbJ$Su|dEex!i#Ye4J?tfyJm(*i*RWtym~%ewiP%jZqY30y)bl6%6d7`KRhJcCc? z17y{rEg(^>)(J{coBdC3^Q9k6MA1i)Z-oV9UG1U6*J`CumEP+RD_W0#xD=2q0+ky= zDk|_D$X#0gIKYbc;HXdQaC305Hvw+W#9ABVqwpSB0gD5VLof9XS18T_$;BojV%G0{ z%$?98y!tiO@9iYlbn$cEYiEcB?tmev2GMHFC_FqQ1T+7X;<11YjP0~pa!z%XGj5!l z1aSj){>xhnq5%Ysd@_I-e?Sa|Vp^&)1zCvQ(zE@1lt<+0 zw|NI_w@+0`Q-XAQ7G0?Re`W|3yMD8WJVUF++OViMb4JM=j%lzEkfX@gw=`ldRYEV)qtLtKpd#OvQlpcYXdIby>NO zk8WI09Plk4X|~!+|GVAyS91495N2vJD*}uF`3>Nf(g$$L8`A~W&@!wwzwwJ$iE#x! z)pgY!LJ@5{Y@m5D@|Q6PNw)gVjj@w`@B9P4}P6Qu6gN1Jk#K0W2#w7&&=c>fmbP}Scu!- zoJ1yJb}3bjvMied!ErI8t`>mFeE3iN-hu>9FO?h@6jGvlPYt>&yc&fZk*0D_U!TC^ zhsFT41NM|OFZO(!R4n)LiC3rpo@>$Q?oXVzZaFb4Df?|1;^7*>z-DfTIW4wTxZyst zeu~9pv0k(7eeHv?b)>qPHzWY0RuMjn`gY2u*EHx*LP)>3>uGN&(izv7duIe)AeUA1 z`%F4C(Bg`~;!v0fqQ&9tbeRq*uC`#3wTY*RFwd4wYn`cP!2!N^-7n1fu9oIikRZqr z<{sf)g7{ZVOO9O1^}MIyZ}~NL*oD6R+NDnWgIez2<_C0Mm1_~bwTUak1V%Q0rW#7& zAC2;q4o@MpSCcD2ER>Lst4c;Wvb2;@-z{_1l8=o5a2Zp6O(7k)o6GSm5^o@*Sj}a* zfP7)e$R5&f&9rg9aiDmgK9GzzWB}3^7Z5N54j;8D!W}SrSQltim zg`ZzF{fe!4^vwI9NqO$pc=l@>By=4V7YE!Tf*aRdfRzW~SQ0qs(gYOcN}<-*J{#b3 zzQRw>qd6Em8N77aeQ1B8hYi9_Ci2uDI92!T$%&`1M}_6cANQe$WBCqf6Gr?galxkuwpxG zk=JswJ$y;4z2B1%{f_vBNOt*U>(p~8!`*iL+X6;;;9ndzUMwqFI+1O;pnmon= zXttR#ECRk*v)^_@mZ#uL4?h^JM?m_A0IlDT($e-|`Ovp^pe(g_ifN9mNz5l-|BIgP zGZV@?gUhIKuP!OtOAf+0@g0kxz%<#L(vo-CxnFRxn%H^Oo9j(Ugr9(M9rJUyi4G8Y zzQN>6mz;)I!qb5m@LIP`1}MXn`xsMkVtwT`q<|03=E>oO8cVC2P+AIuAPCCQT_r8K z5;J`KON;W3IoWb%onG)nC#*oau-R81QDwC;P`e&(<%3g?qF^K2S11x|%qVix>QN^m zdngmp8AkeBS;4MO;Igy1@y@{g@(7x@+! zqq7!Cxm}F0CMD!jL^AC%Is&;E7 z`PjX$ng0voTNh;~x=5wW#}a+QlE*88azcSUn*+eCBp@hZ*SgP=4*f$-+2BD{Y#j`C z6gK2_yc3PDa9J9I)WDNrpVgM`Uf9~cOIbAPwz{bj@GO%n5uN=Tv*?WZ`S&b)+g{8} zWDOhQ(FWm5&;Mjs@h8T8ino}ESqfgzdIR*+`o$B&oX;pr*iyS$dp12p7!06a#5Q9k zPFUW2>cAO@JRpV+zcvmdNZD5pnRFqAyhWOEbZIIs=ew-}-Hpf8GgHV=4h2c)(jG#b zMI%0m3#`F+ra_-x{4QAYnbB}zfz`9Qpl39Q_NvuY*AD*l`yp*U*8bsohrQ*V_a#RD z4s#=pG7B+H#4zLb>4Kvh2gP!H|H#%ks;WY*uQNO{FG}f6n~GED1bJ z)hnH;Q!v!>j23V*EgZ;IA@xTL=CXLF+i7rl9y2XO8TsZK?7Kf8Qqn|3r4B>~S<|-U z<{&kCeAlU)CV`Vz5p?eoKfZQ~QUO*t&#+||op}V*x%3rSrC*SZCjaOHCJz;-k++3_ znlYW@E(v3q~(V=li$7dC;Xs>f$r8(l(z7VEJuDX zj{X}eg+N4O0*?-%l)YJS0re`l(MxY{^~~kLNiZUt3t+O|R}F+h^~8XVQMkJtFo3Ck z=}{9}njE*cgE)#VU=F^p9ON~~|4a(@_k*Gcvk?-Uu#*RM34EjZ;I&yd=3b&;yb({a z+tfQt>?qLG;vlTv2Mb&Y=Nj2g9T-IpE22wdPl~vD{5Si>1ZMFekEG=e(yB&qwD;U% zfbYenLCyMsuoWyKuWe$nE`=+Xqu~4xMjf-2z)f`0N$89S@PiYhE(0HOEtgX-lTZU6 zM6z`Pm#;^-^uC4)hnMcM#ikHO*wxt}^tEtS$EDeIk1iLV4aDP*(#r{(E}_X8AwlRr zRmM&rLtr&F8PUSHxlv?=2v85kI@qG4<51!=`7)W6IjFfAMWTQI+H1}!wMFvEE$Z$5EVFD%Od ziePgrMC&UZMXS}ku?$49dlCPyKQ7?X>N6P>2ZDd7lmR75ns>~j*gT_Msm5*d7*wv3 zZ)Vb;)cpwCmB73@ue^ZRr$<|T_M(Fnmc2;3AX+!!FKaP8boi1fCzn6r^!*1dSl$gqr}$V_0?*7Dfqu5M)R3ss!967;WgL zhXn<_iTZKfo0gMoXID@#5s&3VH$`#fYp>rO41G*3g2?&<3XAlJ;>YNFUiye5kvH-n z)@;NwuNFZLuwW(-l|v<$rXo~94wUb<9o=>nu%W<_1X8Y0i=%4Z3USn>YD`wa0f1)vFxTmGb9Rq z6aOj!|JD!?ei}M)jQDVImqCatm3JuPiA@MLu{~DB?8||31pmD~UdOJ3SlAu3rSkPlH{I3<96B zbJn6-MCga5cO|dgs`tWFx|BrA_?G(AH;B`J(dXxuPN}*0Da~t8AAA1XIaB+HpjwBe zCKcf{caw16yQ2wjJxmN1hy47+Zw91)Wm01j96EW+>x2j)$JuMXvqx$)@%0sqmiFe~ z7>OCy^U~I`qef;aM-90dCWY>IcN3bc2K=`kqVx6hyTHhZyFcHv58ot1ust6THB&lZ zs$9`ZnZT(p=l=EAif-YrcMBP!cK%z29kH@M(>~cg%`#PTUTCkMk$JPpnDdWpHXS)B zklV<9$sDKI@meqQ_GfvMj|u*hlvS_q9;-`U)9=%5DBfPC10O5C)0M+@?78cn4{IEC zVy%;9gYR6`*CC8g3iYjeJe=6}i?bc?((2G7YNwtwFEYP6<$>vxC=Gw`K)NvaiQhtH z#eB}NVQ`Vpa7vKhybsF2(A&vu+l6WWP`=uJO5dOLZR)3a?dFoA&$l z%Dx?+d(m5FGXC~)-}ZUIu~QuNhIKr#hpkCJi|5NbW-phQrE(2B9?85efxp1@%f-0_ z-JG=WD5)=0Xa3NjzTP#CI3Z4kbJdT^KW~b*!XBE=`Y|S*?&-Lj-Ti9#^%1FpDD28O z#l_%``e^cxG1sub{F%ImFM=x;_=W74gIE@NK2#bGM}{#srX0n+7c$;U{v|ZzczqSbUk*2F18IjzBUxW+Oh6Q5D$ZmjRJc!#dEUI@;TnXt;!y^%P`swe*cwD;XnO=jKt8J*EjQ5h@)NF5Y#=ta5|M-UMOQHmmEq$6FL zbQ~2BX^J#y3Q{b9^b#P7K%}ckFF`^PLPD=0w4c_;p59T(2eK_TG?EqJEUpbbdh7vHuQcxLcIU3c{GDb zJZ#v!nZ)*XS>dqp|+v+mpaJz`^lU$e_QT9}6` zi@%|84p0AVuWD{^g+;C{@zXf%camRBJH=X2NoTgY`sUoQ->95;^+D}~yst~MPjrP2 zT@oK))lAokjH5khqp#f<_#Yo$VqBIGT%S&%%tZdSGDf+_c-n zb!@m@Wtqm^yZD3C0;4&qH2WC>PIbWAHDbr!!w1L5#|1b2mh1%^!wQK|Mo7wz>=XM* zN5OBkbQ3o_q`Cz~s zvYRsMJ2}qJOa@D)f~9aL5Pvy1rg`$@mYag1CiADO9j!7OpIGqYgO@b!ZhI^EetrYO zyw3SNDgk6Y?jB2?0&a=Uob=o+Iq5SJ8P>sr^K6a=7h-sz`+26XvkQKjuTD_tc6hk% z*Bs}t8@kf}CL{O!{>oHHamD7*$qi%6^M`qnZtWMNe}ZbpM3cF@KWMpN{Trrl+KoS!P3M6& zr?S$HQp`(^`?!2g5O4D8L+9nRwL#Mecyos4Bd=y$A1xW1Om<1)0-whZ-P1p4L*4ny zV1P-y5fq3!A{Tn1disHUjE<(d_L;SECG*IPjJmL}-mQi9vEr~W@HPjbb^63O;MBoH z#`Cl87rJsw$9VCUCEygunUr4B+Qb5p{?E>$ ze_fu_b@N+k6Z#AW^1i=Ky5aoW`q&E3yvvxr9fx@@BV4|zVCcN=UWRJtMh;r<01olb z*ZY52XBn!)PBF}jx8-yM7tfK#sA${kX&qire&zbR@}))pPjm!QT}S_ZA5*Dh^!A8; zxmVjM78X7I%ZcQR@b{g=3Klp2`IW?zP!JRR%Ko|{8`zX(zc-A(kDomx269|YmS%h8 zHdaX-KmD}Mv*F#--GzTTF=06j6-Q}G-CI)^+}+!k%-};UI`x_ZoWO6DtePFrxfj-} zqa!SmqP0z@5ra9xr=@;U|8Og#bwi!e0B^BPiyRj=#1ASITSDySCVS^rDtOKMh$7z& zeT(mxVfJKu;RGk$8yWckrrJNxNNa2M6QP#GJKLSers+I8xArvm_3O^L`DU5u({0&p zk_cYBVhhrryr67p^CU3SdKGk3U%nZLcCh-n$$b<(sYI5Xv8 zxuFkiBss?jxE;;)UOma#_5r0qvCh?0_x3(zN>h||dPH{iZMO~foOFGfd3Jhqn9e&q z+?1Y-n=!hIT&;@qj(uw;(H&p1oW$|f>r55^PUaJcVl8LiBt@r9Kghv9dSGjXjo4=@ zOIIyotX3+R$fyD42|riiy&m3G=tPYgqZPL+77qj@Ki=2am=kn}xxn}6zGt0KPbL@_ zzPY(6JFvM_n2ujK^m1Tk0N!eL$PdTWQVJCJH07Jp)dz8Q_Sn@Hd{nrSGd2q*ZO?A$ zeRk}J9cf#m;Kb70SEx4~2Wr7~YLCk*(@9XBt>Ss{+{?7yFO}d@l$^n}KrypPnLGk- z^}X|=cnYsbDnyJ!^v&YS*uYZ@!vG=NdQ!{1*(T0|pEvcU=eEhTn}q%rTM+%C`Q<=$ z5XDvnmI`e#dKHB_7aPPID)c2OyGfMQC&pVPne zTDj1@hjMv!Emp7C-hII=+r^>gyv>M-gXyySH%~WFGLQqIl<=5VT4A;JT6T7Js9g1h zfDX)hY{T{cOrTS#7St9cqM9OCXU@6sapv-+VHu7dPetB5-^NyS;iuN8M&J6)-h+`7 z?X$=VWn^sef!P!5mzRt2{hxcgp44^Yl@=5^`=lpfeI;-iU?Iebt2W+#1s{xYf$0%p ztJDe0UF^s;`6n`lqx@4vrqz|Pll~BCrQSK6wq+j0Df*ngu?g48$jy})&JB`Ab_P8> zKXKKiJlT!xKv?tI|4dZ$x2VPNj>zvJ0QJ*=`7pb1f;dHDZeuoR|V-w)p% z`!k+)KfY18fMV~?gtlhuf%zM84Zh9yCLw^ zFQDs=Sit~zmCKbI53JspL$hCeUWd91x~RUJ)yz>Ycy>!eRcps5>-2`PW4{wtb2>c1=WlN1jG_q6A0GvWASU~>#$`<@NM%gPA|u~X9;fsd^lgVtOtdhB`B+=z2L zGG49k_faP_yH&VZ9<(+mBYzlsNW9i(a&`1+y2!Z|7v&abl35`jHxbhn*H@;vU*iid z582n;j~BTQEw?uMx=6lYP5L85ek+Z8aGv37YHNLyfvMHCH!ojF=$R{X36z`aa7p#N zbS}3-2ymK4)lU(5k|Id*|K=z_8_&whBd| z~V&wDCkG_GSI#7!$u79AlW9P0hay}y!({|5^6dPjrP1SnevHh*eNs$hV zxJ`k+twDEsMw=~Qh4b5f2-v8`JzK0{E~)mmO61MpmvHT z_nR~BS1n*S_Ci4HR;SBe+WLP6}htY^=$S_mN0lNz^?!pcs;c4+`HzXUb07?FnKR$UFXcywVC}$uQ-Ts)`teV zQpz*mgQTAF)P`5-D&Rov5N7n_P&}a=(vBrz868Lf)&Gzx~m zmo?fNJ&|Q%Q#D9K-gGt;a$xMCc>M4#lLcrGng-)tLm_ABVB9Pp_xIP3lL?V9;+#~( zVH`j#vbr(>e`=`#rA+WkMQIp?iQbds z`LT|k`laWw;1K6d8SUP^fB?@;i=h{2RPOs}(e z-WVj*m?irU_swgX!_=mX^Sza&=h6p^Nwtg1{Z+E*SsaIB)36qshfLEtlU=q{d;;40 z((P8)D;7Q$dUiJUR@qyA(zM-ykF51j52g8&uEbnrHk+AwY-nJ{g`jx6h)K*KR>Zte z5tcZQHl-j%V3AU-d|fvGKI)!t;nQ-6s0$1Oqb8fgK$!>l;)ED%T%H!u8SP8BFUHA< zh_{$6yF(t>IC>ODECf;mEKwccn489WV#U`wrvAIq#;Rk4wWR_J5|cQ$XC%-RPqq!V z<#BBb2(L~ya`)|a@!Q6$)z)7By&T_hM?>oAFlCdPzV)bj>ums&(qQ(}FSKB1P3daS zF-smLVHMhQUH%qLo=ErhEp-$5H74cVN4I1AKbyj@Tp?Gnm*9!gw?5~L!oe@`)WPGH zXZOd{tjEV6;yw*cdOBeJ;l7;{H4E2U{wMGK%!ooqkH4zG@?jxZ%StwaZ?i1J;JCH>s)Kz8E7K zk*saG;(mWOQ*4Y2Da*&@v80R`w_&%_<+Z2*=X>s|i&F?vi~?W+A0+tn9>nYn-%%r_ zJp!DwbJuOkMj=YwT&Xli0|_R-`E@un$+9vFlNJ!n*EK^#oIAipV2NdZ%kjFYy`AHX z6mJT^#bplJva4d$@CC^sZa?rgg+N4Qd%U16jtPm(c$s5@qaS9})g9$U4E1uYg(n&V zojE$;x3;VFSTp-Uhd*;klM6c8nvCN`<{qR#DRMqlgs2@q6YEiBBk)#;A^;N3{I0GX z5X!&4N=s5%zxfzIErPk-`GlW~x4cd-q;kk2ObQwdfh;MteWi+NzikQkT!O-CzQ-@T z?ed=oIkBtNzpSdfu4pR+05n||idY8%5lsPS)U^qNowowPhB1QS49Je}f-xcqK~k2v zSNa-iwbA}v=2Na$h$tGvcz44)l^)%!4>c+|tk{5_zbX8DR<*w$$Gwj#8sYGHmiA?oFg2KePLwpEM2! zP-CBvQCM7In+%))4x^k_DyK@Pk&>q5{Hg*RxRzflbB*-s5&VyD=S2Qp=i4l-C@cta zC_q=}o~Y@)nA@ZGYWkh!FVMO3M&s#iuZI~`ft1T=||u)253yX z7h=a_%!d|@v43c4)=G@H1LCHt^0X<)$Z$o7UNsNX(2nQwtU%m&z8KFP4dMCdtmiWI zC(L_|=NWa+LIdWr$b}w#;Rv}>Sw$n!&p%=0ib3A(TCTehwnHb5oQ`4}o>HsS*Sq#z zV8A1J8DP6KP`yICrlMy&Eqer%KxEOob}d0zi!%=zL=TikhaQ(O+w_6`2nd<0EIL9g zK28#{N}gZuNK3EQkvr~%)g^Th{7K|Y$|T@$%kjsW(Tan9z<7~tVn~}Ql)W~QABaPy z+;_v6p+fMMF9X9zw)r@cWC6mAp7F&Lk<1g*3*o($4&I^E(tYd&0PPV<l8K3 zE{zbr9gGkb(I0vcjt21Vg-?R!4*@g`yU45hiSYIB*qugS?$&Y8l{?PesgMI~WY!;i z3BCmv0D;uz2x+RQ$x<^KK5Ac&Wi>43Suuo!ES|4MVdG8d&2fJ~(g zJ4o_8qZ@3$r}lM8+nnyIx7U)d01XLQ4`Q;7anZl* z+kw$hK%p80?}ur*FN*m1_>gljIFO^U9FdetN#4covegxGS%RLAATpo`h^h<^dAkCnJa7Fh6mK zg{8;n{MV0L+g0d&W6L0jdZo_-$426fIkmAoHn0pF7t2S1F~4Wfgyl;3)*5vi92g8r zF`&CQk@CNsqde1YinaVaen`?~>rZ4^D9)rxE1@6@a@#RZq#>txFsSI-g@Xsj@F;xW z=o>}I;!!jg+y=u{1b{DE1;SRAuX6Y>M<(2|zvn_k?qhjT!~DQ!L$#rglQSuia%htH z-X8v0+iiFNT3jKIdh&z#Y8Q+Spq?pb>RM43*i(3o*g5~c9}mYikCFVU>DHFU+v<=A zSxQQ`JJ5~p68suM$UoiQ0n^(-e|Z>F{~}$vTWpUT7$aq^eKpg6vlND6`q5dJfG-VU z@@1|r0&H%iSIHSR#46-MmG`M_$Bb3Mh_3*II{VWZm$SaT6i#-j_NBI@S|{>`pmAMe zr>z&;{&s)F)OXaf?%@VDyReQNU=P8@cN*XWN@Q?$R@(S})KCo!G6=sP8n9F&R^yxb&q@QRz`}K!9mpqA2`Lricssk9b&;B+n0eG4fXb|V#z0Bs0uyb7 zm_AaB{H|w6mKNrmOA3)kL_*N9lED20Qr`7d!t5mw_qYR$WR^F-;e-LJ6_9T=E z7LNuxtn@@pJ*5Lg{Tb}%#~Xc#+R@X8QKABM0&+WucLFMBDWLe|wMEnQY4>bsVW&u#J(K?(Yq8JtlEWt&r0|@;m5H zBYNosu*dpXRr5|888it$I5vq9&aYg$5EIlY?OYm|OOR3;IFCXEQc@6Z!;*0_R0pSR zyTN@}j4w2}^?+*cG~lfc0Hig|-6%N#Jwd`A9vvf{D9HX!1t>j zIyhF|x4M%+*Fggn4JOnY5{{=qBS?Ph#v*#FKkSTe^O02&blmOLcGZcu=5dFJ1Dpkb z=rv1&;4qOZP-@GN^Z>_SPX?92OPZ##6nl4`hQo=2D|hY@*DSg7CAUA>yti**S?A6L_0 zvhS2`fDm|QBBIytTU=wR-Ttje2P`9Hk2BVMxm@4gZ zxzMM*k@M7Yx+p*ymdMDU@Rr7inIa=jNW(&;Gle{lit$B_yTF}Kytv{fUHo;$Dji^C=Cq(+k&|}~xiIC@L?XUw10{si z#1ust&IW#Gyzqb9XLao7?-QpPpZ$l~gYVzt4gGtX;J->&{B1kW8-H8A>|ZXZbJlXl zgf$jsO$SFGW!y&arOO3oHK29#1QQrLv0*-Z+Xogt5Q=N|+U1Y&@v*tI9V4H%MOa)i z5=<7SB@BN{!yM6K%*7r#Xb4LS-o1O*ZEfz0V?v#tqW_vJtc-61sng(;iyk%w$JU=2 z9M%!A9k)d_I>FHb??f3`@)|67i@pR~S_QIBc^F?*X}XlI0hEO7T)U$p7`;#*&m9F1T- z6OAjg@0Hpj+M!wTqu+CN@Vjf#S0fK&{e3bW7t6BRXAboQ%r7}6z>Up4__ii49P^>M zZQNFU`kZu+O=l7NPkOE)1UTSL;a~I3?f7R)%;r1fSk~=KH}3 zo|L(sOY~c3zG2zj{`HV*#jmQ_5hs)fvQngkLRX?RZ4T8x!M$%m2_g)SPfhVZvsG`J zj=_2Lloui@DED*lmh_Fuo8zl(g*97rE9|S3iwY&)5!@e?c5JHVXbjui>;>{JZ^pnE z=yFvTTu*CXkk4GFb4Gd7X|8208}CU*R&TGuEgkr49fy%)W2xQ38;6p}uzF2!Y{IWe z-)DXhZ4p>}v#(D~Fz1`ZND7+Q zFGGtRDWhlM;}ct}3r2nrGj|I`1J{es*@CRq)lU?y%%iu})R7(xyI=vEf~b~2zHy@& zz#wGXfFFVQVhgt#e97!m=Uc1g2p01y`GIkGDt}&7XPlxh#^Db8eZf0uu_p zE8~v0D^L$W`v2JRA24%2KG^%5taxASP@*(QqZ;5*mL0ciD(Jh z;8M=`;M?oJE(#i>Z^Q1ttD9`w59^L%_&qc(dstobuvWCZVGaLbq$H%IMI{tO zrR4ObBo&X#E6T_TOGqe6NPOtzwg1}}oLsEzZG8UoFFcJ(?1C>G`|AoG_D0Z4aADk0q^ z>Ad-TzQ1S9nwd58JhPr>{uZ34GfIi zAO;2&0WKC)LgemH0sWzSF0Y}2f#Ji9fe{#jfpHEM1+HOWcs$0y*f7Pw5Ppw=LHQ!B z5h;p+acf6i8HM=!_wUKcX~*}ThNiaPt7{jRmof3l!cvNg8oKVU1FT(L{enP8H}8%g zT}$UnKTm%)4Ypakx{Ys4M8_u;msO;ep#6g06x5gXFAmNe%vQA5#%3hznp-Mp=mx!w z+TPiHmz*{oeb@6!_5U!+Q z5EK7?cy#qz9Q5(=+1c4CEG(RvnTd>y%mbal)tERBeHx3$$46Q5XA`p!QntgbPw ztfoy_!@$7Iu77+%-^|X=#i!(BrO|V{?7~X#z;GuwpY-fJLvypG<%a0E*N2Dg-#T*B zG9%t5d?+gYgfcLfSJg?)%#~Bo{*aa(8Ji%lqAew_{w69;3{Vk~R?^Tll~vL54+51n z^!@#VOUf#oT)mZ%dL?;94V5*+-@AWL&-~rnb8_~a9Ui&Z-ZnC~jfsle`L$?dYLosU zy`!b=WNodXsZ~);r>>znAvs-ARt-?puygjTYiL0zp?m}1SlKv5MZ_AJTDN^^cXacK ze48K+sDwqt*48(tXXS|_R3&b@XnOmHgoMY|)HV49g&UgNrf21*XXdDD8yT2d9~>Tq z$0XD?G{5u>Lh6_pKev}t)%Ngto%A6sASi5LXhc>;8>MevRPsqf&$O+-FHWzWZeL#P z{f&-#f3SA?wWY_=*{8m#&B)xYwz6&W*IrRx<<#&(LGj1u(#e~UN?@8R$pIMP9U6J) zAG{axdTtmP_fv0vF%>+56QDvIcLfy$&MF=?1=U@>T%y(FJVUorb;CQAEr*@AX}=5;es=|UXRi8{@Ejri zP&yu`yBGA-;~vdMb8F{|-kGOmr^jayi~pbg5Dr!4O};y7xcY*Dp^x}Wy4a1$Q>g_G z9yML9LnTswtutp&5`?H8p3hpF!7&@rLglFb z3KJ9KVSyS!BUfsKXhX-g8ZO4qr!59^+A5aV;=3m-w=rI%&12SSX?}>y^tvFuh4D0I z62XrT!yG!31TZirUqe>!BNhx3!w?643#0xX0Ck`NAtJ)WFpSS0ba0Ho&Hsmn#}PIT z)dDho(g21#COLw?Z}dDWCdO%j6BpTU`Wy>Oyvl9Nb+RkaFI?(Bt(ZAIDM|_( zK~MZAA1QH4%{$+WjUu6e-{z!%os2Txsuf&lDSGC1K_x(>C@s$Ai(44PN8qjEa8_%^ z7Jfrcw3#vAQ*D0gu6s`&!#(xj4{9hpA>x7NTj6U(E)|;M_mDL9=BuBW(8wpk!t|3- zggfZtJL7dOPpXZ;?NIoGMFD7vSs1DCX~pcmpCMDi?1@zzEa)=xv<=6m_CWWsfv+++ zE6AKs_%w6-V`yPmzh;27Zm!lW))A<*90a2*w(T898WK|PH#>|*gzIA(6Qaepp(&^b za={3rt9--0*XIk+doBaH+74Pl4+d!2G98vej%!6>j>%s}ac?MgS{+`6*!`Ur7Zes2 zR;3+=KTyx0ZxKyYv3Vzf;;h!pE2-)~HJ>!i25U9#r*%X1%IP72zz~5?H`*s?Rm_nn zsscGZd@~UUe(qXDOQf{zD{X)ji(v-e-tM7h_~jC`fsL@#ZaYWH9ir-*MzGdWhHgHm zkUQzr9lUTNN|v|-;s$bOVZ$6LwYhEHfwGZ6wbC7cj3lv3l|c^T5UL_XMvDF|WQLD- z$KK&{(u4%eX5hmxp?k0!lzIsX2?7rY8G{{CJMJ?~75oSZH}9 zBUuL#{UpQhQx=1*_}|6Ou(ZeE(LSSqh9EoaXvX&PL{$Sq)Sm`cf9zna^ zJeZu}ebOuV<0>&=ork}#jT8S*sKnoLegz<>LHZdJe>MXnw7%tsM{eeWhb9#rJl7?Q z$^K_Nf_-hj6TmQ5wzB(U#?yBomB$vn8W-c?515R2sx3g>oBBe(W_DCm8e%PDwL#3p z%@nFgDy2TL8@UY=(6|T&g~MA~vciu9R_himVOmu6kOoS?1y_ZMoGij1(n8Gilz=X1<9F_OjY#ow_b`XM0S`d&Z_UfX+5*l^*L zur9;xeU88H%Pl)-AQ^@T`m!&Te#ALo2OfU&b~n4iYn)Zqo7ymzmflZ&_K&?`#x zc&ye%OmOM*-p{q$l#jVP-&m+K9srE7XfdxnWK*s4LiabJdxtwS-<6oE>Cvjce)`vy zPpeuO^Ixa#SAvz@8)g(RZ`-uhrMER7&8A-yQNxz1=Tj71)We=(=^H(-4ze?vxeM@j zRlFte^DGkc8qN@T32qjI)e*`7>AkPn>Ubb(Qq5C)&YivRo=h$mR6j0%*4US-+j!*7 zTUlJ%e*g)b%6A{4i=urcRRP1V#Ew(NM>WAG4-*m_My?X+&;w?Fn`bB;<#>P0{6l0q zPRIc5bTVS}8oOO^+N>)gsPiDNMvEv{tv5$_mG3PJ8tF&Y259Sve!gxWcmfu_gGD-- zRM9BBi7b39hhjxnY~9mfnECE{T%TQVv}~ou1|vj~!Gzizb(jcub041kp##PmB!QTr zZTR9#t5j5vXBin64TlK;Px9dfXiJdN5n;tEB;` z5sqdwjrG&X_>Q#?6hK>Y2ksiO0gnJwMx%bjNsAi$QH~NM{IGbRUt^PkEr#OZmaEwZ zJFzpTKVmaIpq=^XPcU-tAi#-vy&{`NTO#XA1+oEamvM6U{`ALttHB<4!Oz(beu!Hs zC^+?~8bS-k3bBhKkCQMX3Ez-HvRcPij?tkd>GZx$ek+4LsEdt>FY!b*2|R*tkkP^A z3fbXBF}kc)DMeq$uTdk(1gu53gDi?9^%a>7b`Nm8Qzht1JmSDv+6WdueSn^$89(oG zV%}c~=tgCLe_r^o$x?K4EUSJ&#SPAq#Ff@{r%c7*jT2$OH+=u0jy4tDIlZIISA6yP z&k`OVEcTu7<$hFD^0N5Dr|UG*>mkJh-B_Ar8caq0lvXgr*ZdFRUFjMc&f)GF$rKha zro4XM*AC;#+#Ohzm;C9tS5Yqj%RPJY!mgSpC0f`;8@dQ$WB~^3Sx}2MpZ5yRAtE83 zmTvOB-4n$OIDLmlDFgYt@Zl$z#pB&Hq*h_>09?_ci-A{F1INl&+&2Gqc+WEBV*fLk zW#9crROmUrtw-J#%!Vifcjvv!FGg5cO<~2`cm5*+BykBU@FMD}_pC+qMdL6f#eK$u zccT6g{_%KJU2hfQ6RaZM1qU6O{&~h7f8HJ9Ys=n6R}fOqPbGKe_r(|1O{t3Pc#L>+ z@thIOI=Q>}NP=!*m7EwS(iLh0t?(W{=M0dD&|LR zs_JK1j%24r9Z?@16WmKhK@mc$QCR$aW~_!JkRYBWW2|dun7wwY>8w%<+LvU5Av_F8^XHz8lIG+ECNdn5{{}f> zc{Wej<|y~%SXpQ}xAt)VcG_OB29CO*Kwaiwjs$7F*I}EEofx zDTRZ=1eS%`iRR_EwN~l*WQ&|l!0DaQJYe}9Pcp76=gKP9-Cc~=prm~u_%OQ|yhU-Q z#iy_4udrth)R{6JdhQj}d&~_A*@djx7%RJ~*_T**(0v>grPr&uTjW1o%?+%qWsEDB zoP^3e^cXg0MRB_F%}@yJftZv|M4`1(Im^1+#mrAe zXzG%HBM180gi@*3co`&KSo6BL`|JKc<=gD&YiDj6vEM2&nSUmPL?!^J_{OsCr>bu0 zGt9A|oqacRXTGuxcIPd)ZU}7)LiO%P%~rJNGdKy$Y^@4YOayI#|2Q`m?Aa|~?15|w z7h2q9*6|0*!ZuQ~%`W*h_z)9rM`$3iuqEW%?l_LJYWU$i#c^{Su)(Jfa%YN*y0UCM z-+AsLR%iNGDsM)g%Ln2OU6T|SxVI>mkopb z7gy2%Hzssud<&KjvO5KA?OyCuRc-*| zy2!;L?ed{P+L{Iy)YuW0J|D2tk(4$(k99%mkfDh;5IOhCz%_*@BXxrU=0Qpvr!+-A zjD9*(JY#r1IWW1MJzkUK@n@stXjlqx#0Qp1JP}88b7dk0^Scw9*|x)9V+ts>>CrJ* za96xkcnYknXz%fHq|J%pU5UmyV&p0&G^z?LyvlabzpLifd#wl^vo$O?EHr~Z@e!22gEh0nvtmAA)UE13hVA-_pO47lTKe3>%#>@AN&2o(TCsZj5 zw$e24C=kcA*iE5?h^c8?M{M|p=uJB&G=h5RtAHAi82_34SwObWFBQ^Z<6!}@k|sQ` zi^(C}W?Yt}-WFc`{`SW%uK{g|;Qa8y)0sroJ8(PAe+ldwm#Rw#JbI(F^}aRu$}X{| zVr%=I+QqWWU#anr9}#1tI`lYID>H%vg=BZhsdvu|6P}k=t*>i3zIGS%?RiyKNUjgO zx?ry+-^mCCg^B+eweTysKT^I>q{_Wds-fO9ZkAyzP^-xnqvbBzxo-BwPU#@Y@VtOa z#U&NZ`x+P7yr)Luh_Qgz16i?Z>vWm@6eir_IaP@T&8KOzFMc61nc!!h?75vlhk+YC zS`a$C@kHS40imb8k>`sY0xnr5#^d%Hf;hL0zTJUOb8&-nEeG(05}vDTw?)ZNzMHwX zpH+^F+qMb_*KkezFziZ?=HWoQU1|@URfEQ*YuVz+{Q*O;P~|nWwJ~51%j?02yRXjR z;;iL%+2$E{Q-OA@yCzxv?(4m4%hI6}W#Y))K)|}k_)_D^)zP9x;kkpZKnHvyTU^^In?!Pgcd3&NefvZK#{e$Mm!dQPYIiT7%>7!5M5yz{p$yfQRL@LQN z@-3O$i<`T1Bbg$pH$(0oJZxb&t7X@Dl&8ZHON;@lVY?9_;$i8tA`-w!v%K-^g{_t< zfV2LqbowLUUQzkRw`r3G%Oc-l*Ll-@jEy`k+MkkLF1NM{9LOly%D(PlgM)=eH%q|x&a~5c>21}^oZ)k5tnvLt9*_{wBaI`!@)C&> zr~1zlLA`sKU%$eKqx<2FS)0#(Y-<3N?33QcWXJag>)W=}0Rwz|SEjHSYhAK#Y$`#?@; z7M>I!D6Ao@E)BHI*6Y-^7h~Oq4DyWy7)PLNaf!XpXxG+v$?dQ1z~enCBgL+81`{ce zc_$awjc+iD_$33QoP7MmaMn5w2W3;%O#_HLx|iK4cT>n=Wo)U;#lrtJ6Xey`%;K>N zb>j|uX)@2AbxnnJ0T(kb1(?=`o}W5OB;6HNynAe@UeHjt=iqDS!6O1WF)Ti#1XWNT zff)a;B)Sn~C&CgEVvf%frhX&+{+X+-GQgqJ9w!P*7nL?Tij^1Q!t0?mA~z4MEw&~`ijM&q3JhSnnVx5)!EZ~>qLMH-jx zl9k$Nj1z|8fQQe6;*}L_L7t$ox_ArW4Zs-RI*N z5(9r+VwO)&{@F}^cJy<(DPBhOlJxlJm>vvy?nlX5l}a3pg&gd5LEu3V9347g{0r(O zGTMVjw@vWR(h)|kF>YEb;LA<)#TX|&Wa6NSvsa7QOtwvCav_Ja=SP4&M3{?R-D4t( zTU)O?O=bl>{}JVcFogRSnFq6e(#|vXO2diqTGH?Bl}E#Q*v2_Zi0bJc#=WEydD!-~ z(=llfIGxamaKDUy+5wM;4HuDjXxV03>M=F3Gcc^@Xc7-I7lVa~WymZ066m-XsjiHb z%1pm-g^XmL3k=EVt`W&vuCo;XR5osi&E-t-D*AMe1FbC!v|Mb`kZuoo8Qh1ZSB*~M zK_Zhv2d6rFsHfd+=kI^XdTiUwt=;ZXIo30xam6!S15YV>d>{F4o&4AnF6?eoYH0b0BS>(r?^MxAE~kjWB=9 zUvs699|146qslPf-E1<|&3=WiZ7|U%B_*Q<(Iu`lSE66VgTRO)W^}_Yu1nUW*}#r5 z=|a(|r;gS2%;GGfK<#kRiGE2Q4UtWT?wiov z>E>AcuP^k(kmhXtZRoyZk2t#At-os7xK9TaviScp;aLkM__v~Iu<^et|kbxT{R(G4{Y#8HjdV8d0-#={$O$a`mAWm z6W?Rox;E|f;8g^pzVPGN=gZ;XpSpX}K+I#db`zVo0T&AuRg*lI887eA+R-}Fp;ea` zLcuhth&3L;E|asQ!?l{wky z>8U7T`Iwo7VQ;C&c^Jo{t*y;Yz+^86TuXl5Jz3QyoLwA#@FG$!)m#SPe?HVx;b62a ztTXn}Ej@ac0fi?Dk0a)nP0%FVNn)QLG%{vJ zBP-^x@H|ouXs#|H#qKOGvpVx~k}dTn#}mdXLCXrp+1;pGO(6xYjIt7ml%594*l!v9ADZ-Omp@l3QB$=7i6*(x} z6pazk8s;8-8h!9uwo04tE4N!hMpjI-)mI7o$C`m z3VCL4BE2%D^JVm<+AAi_?)cEIJr)uT`{jGZ2~i*NStg#KKpr4lo7FO<$c{PaUnz#J zeh3wKe>YtQ|60NW;=da!B^_|#JkMu6azl|Nnlc9!GkVX$G7Wx zintGV-E-}g`J`IA)IXDH8893u%s$Ezurj1!%52Z;>w7GAcLcd^VMcFJLdNyMRw9vv zsADbEovl?qp@--#tUvcM6Zib7EMG~Q5ajoP6wO={NbZ7{MlD&Str))3i34NUVKh1t zPi6I)la^uHu-<1dq~mr^4SulxjDof^1#)Qlse-sls+BVL9orHzS!wn+?s4Hb_y}OU ziFlhnb{2)pl}Fl0My1+)1xH1|h}0{>W)pIydGTx;qj*?j6~C9H@&9LNC7@SBFuAON zY+|{;56Ot`O?tid9hRP8rCo_w4quxJXGrVerluH&!@uEf3Ltrd-PL<2h0y+%tOJZ^ z$7nMe%WzhD4zp(2FxhMedVh9C)?{A)h%&UHBG{hr`)Ai`~eRk>)V07Di8a**eXmmq*=Bn^uCtR%74h3H&q8W}Dp5}4M! zIqxlA^IRTgYSXiED+*GCsCch~OH4gMFr1E^F;6KJG{k~z(-ycU{7gRYy+_PiL?T0@ z?^cR#&qr>0w32rW8yEUHdBMYCB>*!N~sCQAgX3#{7n~e><*`xWr+(g8f6K*%u zHeg9>vbdB&1H1E7Cq3(N7y_tJmIBC@zt?L?MxC;<0~4e1iVDLTa7E4c30@S$aZS*z zOA!I<#vkt?-?XYCmF6^QZRT{Zx)`Y`;n)|IQw*N2Q7*wKB4zR$tX6K$sl>sT+e5Pe^buTUtR zn{A%P(9@~apsd->&mbPaZpjo@TCA5ltswKST6#M97`W?KQfA+F|iT7WjB<7XbwY{5iEmUQ@sF ztO*^OSc&sM@9~1sd8YgnE16+&QF1Z| zpC2zwzq^vnef|(dCHs^t>F-t1{b23S-$&s}Qn=g+`bvRiZG ze(##|I^OoReK0ooD5__)=$X?luCNrak8YDavz2L^{xTBVw&$!-`En4R04nc-c8?NnAP~7qWWQVR z%5Yt&BVa%S&lnX#aMq$>*?xtzcks>R4sTU_tqk9qgyWC+on5a0vbrDi)?b<;QP_8Q!K= z*zm&J==oQqo%+}q-e;MW{IRmrP!KNuPMBmOEJHN(7CglLhAI4XK!&Wq#%;QfG|rbP zI8hF3_%@#}ORg_&hg+V@#cZ}6O=xnaK7~X;LpeNK|N1m`l~m*3YJ4%e~$Wln;PbSDoFfMl3OSS1qSrt zNO7u0ycxJ~%iq~Z&%2Ib3(g78lxP7-8fDX|p#MF=m;19QKREvT9?1p@&>{$1$Fy5F zSn1BV$>R+CFLF;5!*tf1WtDN16wW(Xj)}iD7o6+6+=(s;z!P}+LNd01a;5)+FYL~r z23n8URvS?QmE78T{j0@upMbx4yYKXzO3T}tEB1T(Q-w^*B4*bUqob22C!L4l76R5! zUtQ=Q-sM!8iDadXrM6;r5(p2qS<5CS%z2dMB6(?cRfg~IKt;GCpqiIo_CZIdcR+;2 z6;~3$$eOJocIY@vm={t6&aCUlBf4itl!<>+&fXyz>`sI0IM4`}S}iH}ztd-pV%3ec zc?j$EO~|pPMGRvKieTS|oxdWvtN*PjcU%{Blp1@uKxHBw`y|K8QeIzX;C7-ad}2*o zyE27h^v9QA!kFAY9N$bRRu~y3-DP8?t~K8;FKHSom6%a1X0*<;bV-(%IXg7_A8LDC zB>bM}pA-_DFkyHkHas&qT5Mv0>Bs}$Y|6ePBSRG0U>WtdYsmSjzCQ5eiX~RAUFd2; zw{m0i!(xoJKWzunZvnr=3YK}E+O@CKIoR=GSe~%vRTKPtXNSU{s@0(UO+KPmbotOo zw@e8T=OFG5ev{U7@8$P*ZH2ZJN0gziQz0ak zKKS(~Ge8lP&%^k7*>QkFpr;T!^r&54I$(Fv@Nk+D?z8eTd)|J0Gjq0qYH5c#%%xX8#h{-=TSn>5 zc?Pk>%Ld!T3dQEL1EO|Pu96&&6mWTglYzKzt$RJ^sDL@11#b6i#IGc{5h1X|1G%e( z6c9(ZPm@4aXBweC5_tSVW|T=%5*}bK_|+~NVzazvBa&{tsOjCxatB4m@1(25fn0|e z@YV!@fbH@z13sc?@D58L!OqZqd6qu@tD;J2>y4#21#6C3ejjh<8E5D&Dd% zzXPl9k59Yoe1zi9YAS8G%gRpr0N+%AOkug_6Ofy)!4GX;$J{xO)3|L2=a$itCS?i5 z*yO8E=OsU;*MItp6rk{tEl0tgBw4kd!s^zCe(SUa-Dqs=wzeK5xwP@$e82ed*d>Dl zhKb9s-4WX~0jv`=3U?D%e>eX5w$T>PJcI{lzvRtDmHUlTJsGNmC!jgz_1HFuFh zK)vCdZb$yB^)$tlKSlA}U06!&TvJw<1T8%l^JUf?C4L^fuFL5#BO{2YZaXKR=WwXL z;c77ATuxX>Ajb>xm`wb6X+9nu}07cCnnimp)}f@0@Z zw}h$1S6U3l@VU?}iB5u3Vl!s&%Km^op;_2)z#k!lz2CwRt^`NGjnV*K{36sOS0vmhbiOy2Y2iBJl9o2Rl_F*;F5cA zSR~%L#Pt~IQ*(feX^BW!c$Ra!;X>Sw;eAVY1;GSiRjP6RiO!Z03r}^uOb-7eU~C1h z&-yC#;{T;%A06>rTZg@FK0mtnn>yvZ|NB$knl?099078k_*CB`^1IkR_8rrk8QI7t zh`cQC@_5{1G=2u(7#Hj@K2IR}=q^6~aqplnNk#PdcS~-}FU4@yzX_rD-uCRH3w?Qy zBUtsc?tY1Pd?XOT>fF2~j5yjT@!D;kNJ%v|Y05Fif^WFO`jb>{ost1SD#BfA(yekP z59CIDJZxs&pviCv_yGY0LBmfdX+SIwVEn7KL7?s!7}x{HkLSN-7xqok3H?p-tusXq zBH%IYa4{}$EA$T8OBtuUrYB>~1Z?^!K9;aC#)!(+q(!F_I~1q=H9`saB7X%pr)+VN znRRoz|Mt~lC$(2)elXZ*_&*+FC}c#xm8uin{55#=5+$Bu7wn{~&7ihV6O7k07ZQbj z(t{5qD3=VMRACVp;5%6eC^Og=W*;d4*Y7{5!!>xm$)IqkiD4J!OVO_e2%?BAl17ang9fG^J-gOIiD|@@A(ML;p85mhq{F;TFc2-t$>;G$cMxj{s(? z4YOcBJs1-SKPO0g{v@g8T@+%uAm_PNFnqB~{zHzpOh^2r;P@FGdIySg&`U0_yvizQ z(JP82VaSeF})i#YPQz~ zs`DLE3O6t?6M=+-qAvp8YZdp2&Xhd00F-l2_n2aqE(@U7lF5h7qPvmz*2i8i88 zER1_Ne!nVP8rH4LUL-dzG}6!&fyh<9?-&GDzV7QSjl;$s3{auf<*p6_84DV90L?_7 zM?EDkRlu*R&-~cgT9~=FxY^K(VkJZob_JR-ipMP(cQ^P?DKetf%62Sq97*B`-uL8( z7vCdRudqu>)n#9=!)|Apxm`m)Hk9T+qBeZL;z!)@_^7*Jd$_?btt@%qdUhz_cxx`*i4rgOd`I!$lf9Otv&yeJ+cqRk6Bf<_ z686pnqhIA;Y7L?CjhRQ>VdSV<#>&>jm6hbfjAm7XZJlL(IGRhzsU#o%9kIDeYD4&c z9p*7*fyeh&xO-aI2t>;QMC>#}e0GH(E7{&E`P)O;_jm8!i4TR3xFwT^v&QuZ{1SNp zdj_f27()yM+*V8@j&4IZYiI$UVNgU!ECvby@zE^Yugd& z_)YsXs))6r?k*0x)o5bJW0q1p^8|3Dn%C5cB<@!)Q+Vk^wvUZE=oVGUm^GGWQu+$bIni$-4!E zimdS{{D?T?0{|cmwXg}iRDmBrLv3SCd-<|Mv81bo%ZczMr@I9Na>=bfiuF_tRJC9= zC-~)F2~(qjvf+-j<2bmGx(A}(JeU_x+Xj|3ufRSsd-@Mavci=4o=e4m zMwF4s#IW%*RI5HwkSYbf3=n!0AOkWKXp4R?T4CBZpxMkWhTH9v8=taPaHWA00~nC4 zZx_o57PfuJowhgMH3~nWXj{6IZApGK<}C-1)s*7sz0)HI^5RD*20Jl?Apm=93FJ5z=m)^9jy31x#wpq-Pu8m3Ot>NZ>udczbzrQiZT`vdJ-@op$*X4~ ze(5Q6=!HKjES`?HAc&2)mK*SjI)L!iQG|~O3>=1u;oi9`6~3!;zd6#}Lt{|r_5-xC`mh82cxe3%UV@MBlF ze?z%t+w$4-0X1R2B+Zh7ZL5lZr!yy06{L6r*)4>wV6cD6Zt#jXP+q$mP#A~{L2My3 z*R6lX{Uk z%^Ws2E*$4kZE&=QkC5gppf=`JguC9P8c7HcLsv`3+M6NL2% z`DXCox_`_x**eo}k9y?r8=8Fl9M&RbOy4^@K{Ns^Gw9LhtU%5Qdsquw`zCuB$>pA~ zD=^o)P%JJ-S@@xWnZQ9W7|i&w^iwET?ebvZ+U(lBZ5%jitapjT$`NA2^*LkSPnmjo zd@oe`0mZP&d}zRYBD@jVpbHk}tcz)$N)Bgn9=44@@L_D&aX4k#Mc5JEzE}IBtF`ZX zlv>+*ezC>RsY_|%ESJV-yvxow(IV`CMq&dYvJ9f|ZvBVJZf;l>r`wwayGtQWrNw`M z=3CSw-)Ya9o>6L%!$d$w#?Zsgu*Lk+N=K4mLJ-&Hyq#ex+HGrI406%i~`| z)s~!+@-@R>b<6v@H)fBeVS~J3ZTIh=N2r$dPLCv$v+ez{y(#gn=*nE&?yEkjc@6l4GiA%fZ6m1`uQkLl~zVv>}D+=dk{} zVDs+z;Jjh^g$s93rwU)dbpziO6?xq^_9tyI@191R2$_ zpuC`vNh`~N?DIZ5D0wKW%$5h%qPWCsM>^-@x8%|K>-x8&i@}`H-{0vX&Q=Kb)5Pd4 z0xKaQ;|hx1`Pm_n7N5=E>Cux5AyCpldMzpJ!Tr%``+V>i|lCy<(&mu#`)mAT&%dCKf@w2rZN z^qbLD+g%}f;f?OjO`e6HKg5jlzwI>3_SQLdt>`#Wra~qs>$o}-G|v0cyEFfDwS13p z)NUp$WjL7Seq)h&Uh1hl%5Hf>NKdhOaok{YQ*;td>+qKX${uK{&IXNpxmvrc>OZcs zlq_WFPpiz;*JcRua>}V~F4h?QAkq4`j@^U@nl6a$)sP(lPd)N+z-Ixat(U@+Ai?jw_zURL~riwW&PlAMvO zpSbtCcSP5D=x|1e+zZMkna$e<+&2Xj)!Rz*o#(np6~-=u~p~oXX5Xo>=KsK+VU`=B5wDpbF1oHvB$_-%uhn}9y6pY|46XLx4_x?Pruz-JXUKajg#{z-m^)ubM<)E`=M#IEQP;jTG=SSnqZA*do zU8!}G&dn|bYgT;!p=w8UqjEU#n>-gNOAgv#C_@fXPbiHJQaUJw?tfwh`=TW@YSNM( z%(20x#HD0xpo}_T?L-78>te4IJa4NTuGhA+UV3%q)aHiKNHiGHHOa99`AEEg@|j)g zK<@S?_`wx@zsQ7!)5p80K8zQbM3G&Sr}4T0Eglbi{EV8%R!?4GXK_FY9JvuU-nFy- zP8!X0A(QLZFK<1#^_ixb3!YhS-Fw;*JySmGXehAK@ULIcz-QLK9K)T6lD)csobpTuarG6g z*9V=Q974|1g;m|@#y_TBdiz~r)0|x;`?u9OLvy~3VLI#yuUy*{nVPv6QR4Aavzh2M zb^Mz#+fa9E`iR{e<7G@OIg<9uFP`$1+okni@A-F>wj4V%7@vFHgFwOk)WLc`Y>Ytt zHF?+-<=?4oJ`9X8X_kn!H`14+b%U4~>R9Ql=)W45iJA3&_b>tPMO#kpT*Nz(AK)Z-^7^$ zOvyvs_N8v#)S&(Pn)|Uh>-0ZQeoV+f&C?olmXw?_an^j`VX)c?gRI79i}`v^EZ_Lc z5TpES^u{$$iAG%D=kcvITVAu`u%Ycgqz++rjBdb4o4hU3^&+6bWH-%9(E8r{B++R`eQD!8m7VD2Z zgX;JFi~kfQ6YQZ4!G^2ugk`Txj|+m7-q=d`x$~Sj`OtXC3b_d`B-J(eDdWNbK+D(9rBbR)!;46&5>B7{L&K0YSy{vQWS5I0p_fs#X1|s3)Zw@)0W}?bb^X=vA3VVamtI!d_q{Xq zgGl+qBFroRvVz)^G*IIqOd*~LAUk4fV`PQL-+rfPkC!LV@JgyuKb%d`mPKNmwBbym z<3rA}bN$si*}>&*#H#SH3~}6&=vCU(;iZnG_w_|QoVN?#KYjd(ySj5Yng5J4)jN@{GHzO-evO&xch`-P6RqO8)wrU(G3N!Y$L;tOd zTGUb=N(yI-uC9Sa8iL>EJK8pg)GfKWrZ6V_9P^40>{JZzQtP+F-jox%oLSo#ydUAh@?)hTr@)pN&o-k2^}MOaIRJ_o$E`il>-_P){1K0Q1eC zjOy_5o@9a-yzEiQKxY-TBrtFbhG~ig5rqyd z2k6~(7rNCD(!s5vL;n*dL=ZaU1Osmf-1>}799IQhEvGR4|IRlH)~E(_zZ6wBb9+q7 zWB7~cwy)q1_tIC(>58JX?iUWi7*G>oBw!YdmpuRAalc~YW96!!-ea8l$Zr*i`#ABQ zUW6e%jtWU^K)NFM&@M%Gf1=(D@TO;_ysXYK3t4kh+DsH^j^>6Q#Yw7U$WDtMLzSBF zF6R`g88cN!IDV?<-FqJ_``>bF;>rR3;+g%KA(`T|o(8k1%&<%KTkhvpl4Yg89NqDt zU2{i+%23olAsqfE<=>*MzV#znS93Ef&%>caHptHh6Jp17YK1QE6`-${$=)hK*w^yr^c-y;JCF6*%YjL1^2IOD@P$V~dpb7jkA zB;ztacR!0`ygYDK8L3G<_`bH7peBRIXEpvBg-69lx|dfde;_CPzPk3qK3!r@-k(I! zq7-(-lDi83--`S0s3@MUO;8a*1O@~IB@G!#0wN$FLJvvOkdY{vAt+HYNNkcgL>cms z1`v=OB&ay#C`qD9jv_&jyv^@@zx{U4+3%d)bN1~11q@x)HPzL(?tSikp4y=H&fr?4 zx0WMZ|D70PWAy`_QWC!mE1;TNi@8K!ndir(wnJZHr>J0{5Y(fy^)b7ukA^De0egrL zt%^=YR8EPFjW~Usg_N?&CPwf1B)y3FBV{ ziK)C{3(^dlkE|xw!4*?$B(PZMXm4wqc4fceLigd{gK}%Gg}4azH+8X-nHDyjOmV14 zw4lYd_*ATa{NN!Y6*|YVG)SO}3Yo~aG+bUrx`e0Xnymrz?7zAgzKU@H@)CYE$mQ8< zvx+>XfAQ5g2lON2agZq&ZRM0uJc~1cW3`p#fV7oBazbb|W`lVems+fX<*|H2j-#P$ z)E-3ZgI%gzZYezDwyvYMXpzRFmhPqe8FjIypLsbPH_eHxgnLyK-sx+P<`F0=%QV|X za(|MrVAjX1NWrU>4Zn0oaXZnxi&LeIQaTdK5Y8ohjgkw43wu@lA| zg`>C;Za}A(Ert%U6W^TuID;#IGtN|-K9+TUm8o;Az zwt(&D0r<-&g3~GYGU&$sijygIC57h*S_Uj!DUY*?FUZe(hFgD>5nW&)ru^En{((6O4+&W5Kj6*3z z)02k@Pc%tzm5Ct$lKk(tn(d~2U?r%fUG2=Sy;F|5z-ZICmzk&WQ{(m>!=kWLK~ z?t9E<4gO7}diz2RrVGcW^cZnwt?CTka(|_W+d`9HS8ciFk*t&?8@ZrVu+m+p5~tbY z$G1*ypCk&^Bd~G(}-*2N?4 zXLG@c;G~>hPQvHHWBhZ8E_zQI4>L<1uL)zBS(q3EPCQ{n2moc8XMmOM5 zw^z8}jjLRsN+~AQj(hAeU+hO*WcBmm?})0f&f%YllHU~yUSEH)6lKQeyOrUQMnCf+ zd896>ONHmVL;Hs-WYZ2;pQ=>>)x@Qw1VKLb@u1IO9~9<1P@{tV`a(!BLILTQ{GUq2 zK#s-#o~F$Iy$dB%GWnY`^9D94+I=B3UVAlAPDr# zcN!VF|F0?`Bq1eiMN?+C;aOdncIHp6!2ZpF4kM_oJWSqss(@+=h?`=5Qy2uw+J5D_ zI+Lu1$ef2H(AC1l8~o0aqlcL0suYuMmxq1{#7Y3%2&60SPdo2i;xd6*x%>UUTQ9L>fVZ z{{Pf?yH#d9(Sim3;~q4+Il6O0?Wk8^PzOXFixdQzALQQwuX)AzAVkF^ufS^3m-nS+ z5?wE@=8dxR3cAI}IMZTCiN$f#CAVv|@+Mr9bxH*l8OkU<|PpH7-=+1Jq;7RIVci#lNv81sSQ62@x~*%)qZ- zw8NocK>X0ccD(wbIN3na;4|=lJ(nZ;UmM!b6AGR$m=3!i$eic}5bh;xfV_$h&hDt|O|| zz@OyQ$n(sly?c&48Eja(bG;`hy!!j|kF<4=*IDl6+H#9qyDNgnup zYrLBC%jlz+Q<(yHl3v=NMU4 zt7lb*0dh5gPRzF%85z0a>|i&z2i}-wc5_yDG4s++7JVI^Sn{oKc&`ae72PitADGdD zAMHSa-9H^w*RIt*VX!oMPigL*F*o> zlF^ugHrys9dOQe}x&&Auqruv^nCNZvr$Pl{-e|kd#u|BLdtzYWJ_|gK0A@HME;MPb z=KFvU>?HZuC+4_sS6_kCP_YL@Tl&{;C?lV!4u&w0;R@W90>nA3Qn#Me1%Asq{}2i? zK-}toeKxlRJ~|qTs}czQ&kN^|ThE*!q7*XZWYWwPdMpI+oogln@UHLFfxQB2DzHC} zt4<7ZqUG>$J)l`*WPiDz^1}K5)<qQvr z=i|4EL|?dK37E2n3X zL?JQnjr_OwBdBKns^4eGeeb6F1i`Le@06x;hen$JkUvK%wE8Y zDZP953?2g?NP#aLW#{eaF1K-~=AC69LR$P2wZQA0@J&e9NZyCS`>iPx17iTn z(JIe^s^}2@)Dzk%B>qhC|AoE-(b#TSque=fF!CKhzqwtx-7VzemY3 z7TP`w1zE5RuRu&Nw7u!2J2Ppv--0z*j#P4wqQr#lBJEVII>Kjij5*_O7w)wGk%YkW z;>#e8$IAq0OJl<_dMA}LLRwIl=Q)5cu%VUuPAK89Ql;CsQ(TBGAfqmwEea!s)Sf&D zVE2j2qgQxxY+m$H4;knIZS&K*xJ2;kJ9GQ#$!dQtj)o4!A_g&Ygwe!fNInb1$*^Dx zet5&2bYaoLifGWYsJ+T$5h2>S%Hl9wi*_kqjoq=Tn1IFuy)_W4_hJt11fWXJW0ZP9uBit-a~nA9cD_T_U|STCph?}bMV$^v8Qt}#%V^TNjt{vOO-BJ z%%YFpbH>}ZZ!^sB*2l)y_UKMQx;SDvyJHXQC}P*|h(TGr`w+SNW|JVaC$)PRNlE0( z0#jap^wzkgRd~C^25r-}xD?)z$7*U^Zeb%ZQytrquREAt-W}6YR`Xk8dH4y<;S*~zN?KE&H?CqS& z^t8VRsll^1XdQkE4e3Q|{&oX01oPez3y?j(xHul2nye+0s@B5r;gw zeTdDq8-Yi)fJ=q@n`s;{Ro6a;XRh)38is$~hqZm?wf!0MO>t6lVBoHQZ||?E2l;II zsa;)NyM?QTF*%&edzr(_Tm!28u0^hkbBF%;S;GNh5O^@>{ZFe6OD1(Q0zEM|7u z1?yi`(@swN(Gf$Rguq)N-QS2BzB|K_{XHY;9A!nF7^USq3{ta{$2t<#nY)?`2d=riU4LHEua5LBx9w?U>=!r0b!}Dp( z$^sOgqGd>mpBla8HFj0|%twU`fAA#YaFO$neSUxr`DH;1uJfWd`_E%7rcFDA_=41( zFjb!ggsx7z23+MWCHnLM8O);*%zMmP1&{B3I~2Dp5i5O{e9LliqlXEtA46Gq9UfkCB+%ext6}1FrpE``k;vPtW6E^DI2d#S{?!+BGoFdbY z=p~8zb;62*6n2&<(U9M~p{)+={QMnEHH5AGa))RBeT-oNeniTCE#_@c6t|#pDj(HQ zu)65>K&tV05X9^s?{cBV%(4k~V|vwgS|nhzZ$O=1?VwK$X+fh{TC$}zXK%Rho{hJA z)VkgFB^c6QO?QMWfG~tMp-Gqli^?)P;;{@TvllMG-J*GeT6jqU^S%yt%2*?+UxcHy z+!wR@?ahA;Ccew+0}mw{TqINQMYmx5w05D5x7GRuZ$>F~DDNa446UWYVHsKJ>C;^O zozT=IGEkvwRb@8FiWTwoNd9cMVF?&w4tiHeHpt=;Oy=(s|N+#US5zaWt# zbmOB^vqt$5P<1I84hIoRQ%P2H!LC1ICW!;Yu+gJ%R;&OedaqBW1eNSWmGEq^MLvRJ zxIL`MSgZjz>Im6%e-$8%rg$~g-*~Er4(+^Im=$5f4R1p&@ha&JQ>>ZZ?0v&6ocOr^ zH~G#zOhJtBDi5`)wwHWr7D0rVu#T|Kca;4*U{FYxv7ZV(%|(5!^2g`tn5E^TTf}B) zTtBq2)igZ=akqi$7Y6T8u*E;C5^S_ZO!o!VnQ13_T}&Ur8`{{H6KD;MbP~TUylH4d zHFIEfO?oU@)oL|?f+YAPJJu?O`T-C9rY8Qb&C^TwB;Y%E>c&13>wx-noPBEO(}h@q z8vHT>Kqg%$#f_b^6Df>Hf23QYvIqciV;3)Q2#6~(DfFlx-|J1)rK#3ro5XT=Ubptu z)6U98j14tCZC0_FP@GcFN|hFE-uj&$mAK@=7=x&O`)5qm0g>1|!1uDv@Y|pYX@0D@ z?rKD)cK{eq>z^$>__`p|huFG>5M#OJWSo&d0SH=#LHa$>M zrcgM#UByc~XF6Qyv0b79H16)VtX9H#uLD+uuu(V(s{o^Zw48A`I9;&k1?c_0SLjQr zo@Huc(p#3M{gCG6YHBiUu*Eo{ZFMx&z62MB@NSwF^*?ITP38;md7M%u-YVIO8!sL= zu6~ex?Vmn5;VzKDCsSk0aj96TjY%Ga!xkG&eQ7})CyWs`XY~qvZPwy<2a<$ZBBy!C zL9S8SFKYUl7%_5H7Pud2w-_ZJ&xK7;c1<4oDVYGn7*2M46QmRug1Hks0XY(X#JSGGzwb zl4`vH$!lN?8c~sUsW;s>{#Mot&;aPKG%k@h-PMd=flmo+w(UsYs^b`;>#iAv1MbE+ zhHprEg098+N)=E0uqz%2gc>Ef4RWL)HHas$469Q<<)59HiJ_kOT&sLG*G*-YQ{l}@ z_A)tu|H3VUP>-mUZ#-+N)qFEPpR61I@35<|Jt{|izPzZ8UNfsdz$4gw4@8l}2B-Z=opSAmfINEed zq}p(s9H4d?UTgvg??_a>iyu8X`HNf|_A+?~u3Hr)<>w!nU4j?C@uu|=$R%>65J)lJLkKDx?J3mLc?AA&GqWW#dqm)Qx z7OW%yO(E6L=o2|Ue!-$lg#M!Bk$0#+gqs8%Aw*m2>%DwDcpvgOhQKl94`PBZ9+S`i zT8D!PUL1CW$ZJO5zUJvuVK*h0YCPorsZ&mp5(9aFkO)=c((LXU5$sSqRaI11v7$?M zW_NuQ(FT{S))+XGuS?BH?Q+UprNaN*mD+d{e?x;o^a009DYwkAcTrsu3}j9oPbuEF zJ{^F7Du{C4@jD0{G1h%Nm2YitzP8&4fv8qAPTv>k9X!eJOK-Gv`Aab{XsEg?^@*@E z_xGw7KeW`xx3pP&r=N@nt+sE3CRz-N4S%+P!3Fk%z>;om+O)A4syfZe^Y5P3g!FZC z^Am*c-%Ul`8_`Pmhta4L{Peq+>k-~~GPSOI`);f zUy}e926hs*aGxn3#*$z-IRSZlqIW+>I9S_yf4^NTpucCQf-&#yeKB$~hh)4>IrAjl z_9LmSD%)*_q4d>fBGf`^8$*EsM0`Jsb#b=H0x~eKOtCcj;573C#c|8q7E*k;vT8)PM4ol~D)pdE_ywdYsiC9(_d8@^I0_!&1ia~ zi6glkZ&@Nfm&{$`zKZ2t;;#*>!y*?6rg;5ZhHuRVh6*{Cv=N6&W`r`Cg#zlx-kZxe zy^9^O#)dyqzUXHj1|OM-C9`3uY~3|QiatCYFn~*1dbA|(=X8%diWNy~poguTGhf%z z4UO+hE${OPX-G+I)H-#G{D%~unU}f_L%iTeew&}~l;6G(6VnpOT~EHB=^_4*m^+#~ z@4R14V{=M$Jb3IT^YWW6* zGPVEzs3oX&Kndu?oXgDo0l-4@GKkG)}yU4F1 zV;fJ4aqQAOtl0W))fxpLV&4U1>#J;h^PQmfg@LNtuH8{OAju~ls|G{+$yCK@D-yva zbB_>X9N5+s&cHthO>A)G?-N-EELq>b&VTeRMF1V^seOM&T-oX{HJpYfF#4HW_Wc6O z93_^*a#42PZ$R$fIRyuzVPe~&ToilKMLz6y6__~o&ymtaGfU606X zx>e5jT2L=}mFbvGAJy#QDB&yrG%c!=xynaxIg%TTS(l$6Qy2fl1AkQ4ZX0~Hw7?O` zQuI|(52NUuM>B#}ist^T7@w+R7zTsY%ocX1r&{N5a+D-lD&5T4-gLc(y*7wI zLvS7?q{V{$I>eGx-c34;GR-?O7tT;met2(RlNz)2kH zjp7v~D?aCbHZ^^Am+?`ft2Kgx+V^C#yZb)<6%!K)c=Do3^Kw_xdSO|HPKR;wdAV8b zZX+c!f3vR<-}cZ;4#=#>=R&uJKhAC8RVT&dtJ($#l%yI9x^O&8l`C*~(&QJV@SaxG zlMj+XN*vVizL~pT?boH3R5}dYbijg8ZUPg`1I@VEC|aT%Iz!kP81pC==j@{2nqJIZl05MS{SKVxy~& z%2BB%ph${vS2l)ZeC%dC3`rh(Ku9*%-ZozrO^#`$Ijty&Bl&mVA0)8Tnj`vAB{mJm zWZcT`sIE1~L~69|jfZ8Nx#3_sXtWw4jPp|ax)$}?6k3p+}Q z)=s1u+Uw<0tG{Qqd{RP_S}o%8rZj^E>)gK7Y?BybD&cd8Iq?n+PRq5hUtFkp@}!Y@ zC_ibQt6V!}-N9mYW!o|(1;_hRg|Dj{Ga9$jJ(}Co$P5D($QA)n#e?@>1Cc{O;oUfR z^-~`ah-vd`=EGK;fFX%J4ZVx)T0S)&O>?X-wF%M}srK4g(`Zb(pmvW`l~J zF{`#dHd*3QCS8}35TcNF{_=Zivd--uFT&Ba^yjT$QbOq?VGysf%KFapws>|+l;I83(`$zc{<|9_$w&U7huiN`}?Y?#&fV=g&O5$nM*(Hd|cE?QAx>k zv89<8HY#wSv`GCuG=xw=^ri!K6638wCU*W8xS6?P|v!bjfQmRQ7JmCglu9wPU61@Fl8 zrRm$Qb}dw!ro~1s?GMR_Y`ooVdbec7cS}%Dmt?sLj>%zVEF4W|v882!zfn|^$Cq8J za2;84+NDJHKsXr>QrrML^695;{1Dj_mGo4JKm*k-O2rod`vZ->beY^jj11d;-@w=G zqv58B$_|Ttq19W5mI_B}dr+t%BlWN~Zuy}ltRjLuBL(#|v83?gCQg=m!<}}uS`R7F z*CC}Kdd&NpABQ*ll-uPO%8MuRHI~(GmUg5S7@FAcN)b!ilGD5fTQzzG-WwL!3n>)a z_7$IA2lAHxikGfGM|C><5to)Q#q>wnHQK4f5=qC+!As3_ zX;b_Ank#S}xib8K=aF8GK zcRjnzt8;W!98-P&Zl)5=9XL0(YCPkRQ*Cb-kLZ&#gk=bTAe7e1(F98PusrBK-M-Ra z)cLkD5G@x94&Au*WQs3d4%YW{ zx0BbVhMX0i9Rx~72uqg?fd0iyc{_Mp9WC3;Mmi_;HP!1g2QNJCnE?@`ZfeCGuZL9+ z;a|~T9Z6u@A%1xL=x$@DR^2B#z~nkGB9RCivd7eWxVYFvEr%EI4UX*Xz5CVlmJ6$V zuR%!Bjjvx_TQO$>nju)#b-kM+8V~zcrQFqlhy|pvyE2S_P^V%zg6Pj@net`JP%!Ay z#iY&bb?;WY5~EP>p1#%6z_Dq(f?k5foG|6)LrpZOzCQacv}V`>4eAW-<|BaPcdqx` zWke`pofrADZt}~OLV*E0y$gvRn**wwQxEW|^Yqe)(*<_jS-rx)V?vXDXM1v=Y|32v z%9eO-ceQN`>S3l7_eBf$Q`2T1Jj{*Kwl3cEDLm}zvNv2!lWKZiv)2bPjm9CheDYg4 z6+E(lGXeDLzbq!C1rMd{pb)bG5<8Umw_S!F$G;6aq<4Igz5qLKK;roSHpj9+#$^cB zXMsO}9sn_v5c$sz|9%gc+oyMD`={Bf+KUy$0#_Ew=*$bz^;Eff;(T5y%lgkh5B+!& zNOWfJK6N4E-K=i&f5`=jHD3Z?xG*FkD|klYz?>K zmh}|KM1JiSTG%LzP0rmBJtUiaBF#_$8@MGi9Mc_3zbdt_>S_|1= zBJDcq*rtdkykS&E-04EYdE5=WnL=w4$Ig&qGSqpLN3VHes9fXF-FyY%f}2P=20$SB zCNez60#k9zDQe~N4H=;}4yC@%Hr-wh{q~b*l0tp`*L!Zjbh*e9K&nM~Va==mOdAAi zw+LD0z+tIB&9#c*8D!eOseTk2z5;sFn94$%EMXF`{m2o!1>0zIEwNNqy&1Dk9OpE&2RrRHr$-mm0)^W zjpU#x7vbU^_WPER+)F)?;N-4aGiL$Ch}sK2Y`w-KSN!W8rF$Y!K^g2YSXhj7<#YaE zuOs)>jmsCy24)EVT%i##szeg%NNo#A{l2AzZQ?9g+o+Va6eNmtDnP#?ZryXa#4+_W z9@3cJEA&F`VZ?q)l%vq9fmT>v_O-${Pe5+GgFFfm8kVrI;VA*j@6zA4fQ7|xe5mez zrM+$#d`Z9=*p2XmDmexISsim9+2d6-7mHa88TF@PloAvxj#TJb1(IxTkCuXPFA*5OAlw;U zxL1pY#M1a|H{a?xxhCFT4i6LwgHmu%Y_3g-hDagIe=J3@JjJX;H+qO!+cY0!9YUfa zN{04c~B?mE1jG!(^DdzKf4(g|xca6cWLX4T1S#FhC$ zlfN6m*M75`8aRKbW@f6=x(kwFU#MSkJElfm&~j_{z+C5^0xGW1DQ3;vP{GXp;&_UB zqf^>W`pMXX=`+8{Yv8qN!T*`Z45?%eD?qh2v8(6dl#1%5Uaz%#p3K$#$YBTq8zawJ zLYm24-Vfxjmu+o4_kA<)eAVSRWyTbG(Q|HG9oSkwJ;G2PFpGcjXB}&qjkbYn{*(1P za+eciqMZ9VwI$_qAl>10F!b>GqnmRdqn%sz2VX;tKMljzydBh<&ije`CAd3?x0(dU zX5~t+jF3W>L5dIC{5&#(x%P|PB!XGy=JQRDW0MfE3s*Fg201{u{WoLGP-(m0(uDwd z!LNlFoEjHpIP z**$Z{y`^9R`JDqVWjjIkDi{J017+)J$yc+L?#M1%m_sE7JYE{WDc!FR@r>QeL#Nk; zVDE`82FL|p&54`YAN1YwC!;2WimcQ=Il$MsNUP_p2H6rFAbolBiPWu>%pJBgvXQ_q= z1s5XtWLmbu-q3huoK`uP_fe`OPm9P zLsNk2@y#=Nyx&4(HjNi_6kyk=M4i7D^L3Ggx$LPd=6Md->Bg(6C7z{E1Jh@!vbTTF z8MVSEY0Amk79qOjUQ6_WN8(b*as6f5Z#jkK(%Y)OF)x9Us-=o3`t2bG1tP+a-c4-S z%M>%U>O4%?MKXfr?HF+1rv%?S2iqAX5RbD3_QovhGPE%r&moQq!Qi160+7Zo&VQbq zXMDv6BT%2Gfi*97k-@epS^xD{ssBpti|*~8n5nEA1dMTtz#jIuF*7`xB2=-_k8rTT z3jl~`Urs$(^k(M&OCTPj2j%JyzOl&h-^a-s zv%c&Ak%pg+d&$Nz1;(#uk{CO%IY1ml-f^4A5L?c{vzF z5mW`mdZ%f>MD9|;9(l7f`T1p`P)+NK)!eh^UJ0NKRor)9Gr!M06kimd_cZ@%J$2p( zBbbeh!lKfcK#K3a2To;|cbqFtU$%;4eb}W$3@#C5bX5)1o!$2hFsR0AG&#rvb1h!S zo5yn%Pj~kpYB4{5PBkRYldOUF=f@t;{Zx#Tu-z-7%xYpwVQxLh(GAv=IWqXgCBteh z{bQLTY~MftE?>()VC!w80hNFJi$@*s#^v{!uP3ngFGotG;IJemkM9TVb{GP8KL0r)I#cZv z#%;w%M=oQCW(pN>g{?359XN|*B|H85*QDm;85%T@-14{3{b2Bd`DliHf*hE#?b`a$6qvE+7Js%;TY zBh*z{9bkT;BcJn?xM}@(B1px#;k>KxUAe{r;Z{_~R(v26sIhq7;n+l69XxZv_qDhn z=je3+{>hgAP_WT`;7ap5kV(4tX|XK8yxR854WlUWg!%BNe7{kk>}VbF&N)|Q0GB^| z{neDs%DqhP`!cs7n#nLk8~Qu59JtxTzo49xm$R~3HI?P18v57+2Dxgu*je9&8EX3uesL&E%v zCwrJ^qE{GMWpq`3s%oJ@aT8$uJ6#-8^h;}=O4kEc>Au@{s36wwllS`fOIWZJqdSUt z_*dne1MUZ>*|NY^%Wlr>JoQ)DAF=p9mBev8DSJ7Z&BQXNd(t|61Sw1)CG{rJ?eCH3BirRAJueDCaM$iZmY7&=bIn`PnNR&gxF z%5pC8S^)Y$Cg8ln>!fz=%3<~wwc3@zYqHDF=4mN3UKvRdtXw^6mT15dIr|W0-!<4> z$+ICPyL?Gj|GmArsK%>TLj)_NKGF}H;jf@`_*O`dnjV(YUmYOMcCg#`I_SGHOhP!w zVIGWt&V@AZwkHTWe&_$KUO4@yyy)!ZLH?%23D2qs$kyfhz=q7W<@ey8SY81q?yIETik5_A^C7 zI~bfGXw2=$Sx4+~uA!eFGrtXiI~%lb`P$q%eYs`!QFD~Qo#T{_$@dGeB@-S&u%dS~ z-t-LnlDYhoJVa})^jvx+HPnGiAsif4qHdV=+`q@9xX9lAALj2xJY9d|F#Lznl`sYsjE@vouo=!VHdmo@Y&MQ4##` zJ~}3cPTS2q^+`C?HC*{5<8FH8|K$OvlHfUWt`d_D#hJP!bYBj4gn_%&Gk0rAOIK^? z2Z6AVun5191i$bD9bsWf;ro*J?%fpD}f~ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-1.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-1.png new file mode 100644 index 0000000000000000000000000000000000000000..b3f6e926b032f2d79c7be1bc24cd4fb2c9caf6a7 GIT binary patch literal 146525 zcmeFZWmJ@J+c!#zf(RBUtw=~W(jY1V64H%ycekj3N-6@<-QCTIh)Czq-8J+K48y=a zNBuv~z4v?Hdw+R9?DejPwPxn>y3RiOIDVHOloe%&2`LD%u&{__pFdH-!on}W!ovG? z`4YH;RylZsg>_ZZN?KZ3R$7`)+1bI|%GL}E>-mQmO#&^|PO=o;&ry<=*mzm*X7N5z zNXlJie|^Nka_3HACt|cu8V{k?}p;vHq&Ptj}d&Bw$*ab zjBKetoD1kXTkEOkJ)XsuggugZ(Df9G{p%*#dy_Zl4cX3zu0yJlc&Cz->(~U>1PyFH8^kNuV^pQTq#>(yC!#-lynb5IZJks<59)t?RC!Hj_@z7kJRoN& zAg)QTGm1oBOTFU9J?jlMg1`{(a zD!$7lS337)C!Fte07I{h1psABQ)GO~5D zo=W@7kz<^pCF-t3R$IRkA#DOtBrPd(st6*CGb~bl-Iw(CNyo2_#al1!zkH_=Tv@v$ z@1^=)|A5Hkw#oiGoI1LdQ_tW$Iu=D5CW9}%SHeGcyye7Qc#V&PMg9Dh0ztHt0I4g2 zAJHn}r9?W9#r+G<_f_eu;AaoIyx;9n5PkfXO1dv~zV|3Vo0Im|-BOo>hO&EpebV9CFx8v4fnD*b3P`d_{?q*&|bXdrdW;sO+|RnrjM)c$L+Gf?Ysy+k3gPbAzw> zwfXefe)X`V@h*@DDp?TKU&~xqj8N>udJf?{7#mBpobbRM+&Jm$y}-rV9c^uGZJVnf z(7-~}x54Jlk}J)>DTVN0StQoqa7s^V%O=P1mCtKU&LIeUeYY(5&8_`LiBq`y2 zDXQm}ZWx8ta5a@k zJh4w(MfUDWk=}Ux)bc%zahMjJ`}>>UC4HkLA5eXe`uvIS!OiRUZd$$ph% zW>C9PbN$+N#n$E#X-}C4H{9N9Jn^U_kNz<6G%*Pm(7{)^}=tNgopox^N{(-~eWhu4WS;|I0)IPApjSYfw{WkU1J zvfD?yMp?!ti=p|@(&BtrMsk{%LZcii&%O3qj!AyoXeU&=pwe)yp)dD(F#T8h6MC}n zJX$BOA3igm8vGl4XLx4B8pd2DHU{B1aOb1gE&k1!;SRVRJQJR~=`bLXJtwGQlQs5y z=U2ril+3JvB(W{A@m2FXw%qGncewa?t_s|`Tf#4DHExk*vTX6iu4-ug>&sH^-ti8t zS4~58&=Vp3#J;Sxp|OFf4K(8=#(Ru6qI(&W8B-OCUo0tvDja{E<<0!c-NVzPkx(5E zPozwYE~i|c8poZ3QqU4ZVQbD4co%|g~El6qiQyWTMtM1 zM>Xx<3tUa4N`wo<*}l}(oMM||nF@8!OHNGwX5Xwo#{8W*^5yrJ{F(OSh2y+1BG}e8 z?W6Ta??i5i@Ci3j*@+9A9UTl>JPUAfb?b4P-t@ExTm3q~-EN}uv&p@}ZBbZVNZ&0- zm_VrC$==z2#dz(z$HctzaLlT3pIe|!;+CeMhhVvGvu~j?6!qK9tmoHw(VfC$ck9ML3edVx4 zZ7+ACDo?J3{oXY7A@ia0D%xOMOYh~GyZ_=@?2We1^$d~zSI)Cb)T-6=yiEef-h>nF z2fYbO4oxC2);}r^({yi{E^n$h>o(q5JcL9b-Fi#k%Jam!a=7U>7~OS>cbtH)v~+u@ zS*qPq7uDgmf7&K@+Eo$bpsAqaRx*_Byky;F-MP8LzhILvnoz1=gXnH3avLi+D@3SK zs&N+hLsDwb4=zE7;grOJu?}ZdvSk~qtMk3Y{IUEoy6<%fYegp#%HTZ?VRP2C8xE{< z2)9N@pW}$H)UfKE>S_^UcaCD|tp{$+jVd1clhkF6`LiwlhcwAzD?Y6TowIAGA%y-M zZ4Cs5wIqp_3VjO=u?*>=hzi~yMu%&M&nR>%==Y}g+OZq3hg=h(n)Uy&{2clcmMI_k zG9reG(w5zt`VoiltjGPsADK0>b{>k1$xV|qGqYYS+anXhQn^YgMk!WeTYiXNLj5ki zYj@WuIqUQ-*BxPxjvBXiLxg99DLEPpZ5x)I^0RGh$I81~;*|8R)H$EBpf-bSu8%d8 zbQnUs=;6yT#Z2#`iz%@Au^XP*RdmLO(vOSS{=w zi*+!X+qt$%eC^{SR?-K@K?PqbN>jYIkrpRNx#t*@`mZM}$5L{pCFdJ3+u1=^Sh&CM<6vQhSYhG)ZKDW&F@I6ugX#0<7x!~876JIz4e;?y!}+H*enA@UKi7D_ zz%wi&8b(aN~;Ob8QzaEHVbn2U}L^xvH7z&p1!aTuhv;99^v(?CCJ$8W}sdxr*Jt zkD2Jd{{F1f%+u<>XR>$syDhLmPRtWdZVoQa{~8-~6~){YR<`mqv((v^>&>tDP6%cK8iS9KRNXK4pJFsZBfe-rF)=YM_px1%U0 zX7B%U7Jm-<_g!$(;)J4{|3x)%!sQQY?SPO}R!@}Fz%O7k%pZ;^_+tL^i@Dx#W_|l^ z9}7zoOZJJBnkV-5%$2j7?dXer?CVmbxACi)E!FWzp4e>pRoAGGSXURri#}#Zlp7b5 z7r(Dn*Huy*krVqk+|Gqn^ zWLgOhUcpvZ|Ih3HX^P8ght&V+8EIZ2R^VpCL8a5bT^(lGI8a}L|Gaz_$+U)&Gh3?v zw7l1n{B!^58CGBeA!%L-s|2OYe>fdduyv3B^z47O{y$Xzw_N)_4F7MB<$q-T|6R?o zpHrU%T;|Hd!wdw*=8WHzvBtTElcyz9k=wzHLANw+rKuNYn@5*e3tBE6&~Jnj)B&bh4} z-PqFoYt5pP+VAB89F4EVcX1PnlF^MQau9wZZd0!Be-^Zl$~~L8;u{D*JmeIC@Ej{4 zx9&@0BP@}Ge&SZ1QwJ-%hV96DVzNF6SX5h1iht?!2&sIBzLUmM*U0 zuGEy?t`=lCZ(;2!eS(gDATe1w4T;?I-PE0{xG)$Ae0|svw6^qVmkAcvHZSp4Z1XAr zPqDle&~+cAu^RLn>$DJ3ts;=gKLUItT99nfe4wh`#qQCJD4#^dW%CrWX$x_z(5$h= z)K5S{@e17V{f|ak9Cf8%cPBSKIW7+vq$XShs8zE-Tdou zs9_AbN_|*S+!TE;S(m%yew*J(L`nO{hO*p^H}H|+wW*Rep;B?knXm7`y6=pEHe2fY z8NQe?xh-RZ|55XS2f4MsG=l+AID9=?=8(S#Q=Sd>rNnXXK=rIw9U}jnfH0M^0_P(QSt8K zXkqN(@iF^5790`;%w~Fymi%;-%{&8LA&Ee#<)s`M)-oH_2h0H$ywJ<^KliJT2#N^p zr}y?{>Evu^>E}Jwfg$c$`CIL#cBTB_w!f%KLUiBv%xlt|;o`%FYA1j$X6zqR?j-ppCrq}k)K(kyl&RPdZ zoUVRB9T!*0Ze5)$ty!~KN!e9z6dT!RJx1y8IF)Gn$M}+0E9UQ-)SGb}?`9{RSwdXZ z{%f@=z%`5MGBa&1lSyEYq?DeF*r1GHx9wOaBJ_1P5o?I55aUOfV2O}ic(p!s0ji?> zV@SZ%%D~E+QoWx)%~Yf;8a7`26j5sDPc&v&1|d2$sGhFc}uyV)oE`D~Cyl;*^o{64yn4YIinNO@j&D!45(rR>`SpFZ1y zAIqRtAc9Tz|LP3}jNW+JJs{jLusc?27%cbXC7^ELDtVfHghcS2cFIpUKrueoecd2r z=TsB&+u@e1!Mm(xvE|9gLCh5Wp;HUvYv%DSsY+ANJ{RffIC`mZ9dOQt#D zS*r^QDqfkm`H!y32D(aTQ$T7KDjGl1j-K0T99lM9mbr_V+7#SbI@jH$Oz!o;G!A?( z$-lMwye8gTSAl-A^o1Wk^HrS7BQMhu++3(bTXX)#FKLRHcm{8dIkEU>r>PM6335gK zn|HJNB}#9dweG#UfHqt?6mBk~)P+V*P%&_aD=X5N<1Tn9nA_ZMOY+z3QOJN3U!n?a z*p~HCV(ggl2trPwH8q-c3AKhB8;r#@o?r!r5mwdO_c6;WmQUJ%q1rkLFvrY-11nA} zthj0V=5I8nzzS5O6^Xu^dy^PbbT-EURMmU0Yb)WRNvL{RnU19$iAt94-`u29bd~~x zUarcu@4GBz%ROoG1a-Sx9tSh%{XoP>?`^xN7;q4n71F7KUGlm;iYCt)itaQfM2@k*&BS7M$D;Os}<>Ipu5gJf^Xb^EeXSCDNeM3SlIp(5hh51 zo$^v;Q|y>9FhvJ}<-OGb{6h{2na`GnzL} z%t-GBj9T~c?`2^HJ|m}7o>`a4`-#%*>@+BP}6KjHN()B;D{n zkM=?*SnB`A$C4Ttab0bpYDSyRbSO~V-~*8!>U3Y%@2n}iRF6BEncZ}a>;ml0*K zIbx^8ds|>1iIKCyxmwxYnGcPQ9z(R0b)~HJw^0u{lu|itN)3JYpALq$t;Fi~6*V1n ztrAF_1R~FmA%@i%5csIBd1kl(&jo7k;=8EeoUhB_{9BUX)GfIA>Cr|GaAEBa8(}qx z@mxla7xlgN+sS*g;|#1*{1DoQeImPGTl`Q7=V(-7^YQGI@2=0Bd=f1yT~sWfg-|it zIt%uIKLcHGwjNF^G5aFvQMtk;{RUSjc-0$-$WC46%e|VxoXkk^$t1T?ZTngp{}ZFi zjhw{Q?ZqS^mmY>VJ>wM5#VfpalYE&x`N>cSwflIwLVOEqW;a1_*0U{7KZV~$L(!o3 zRO-@9gPThuq*q|-tItk-M{~ete(p5M#Az9EwOZ;^|GiO)v<|xOCtWr9-6@Eppn4etV^^>-3dHW=rlF83-!C6bNI^N2Y#ZXG>5bqa} z;-`6Du${wC)ft}as zpWTvIi9=@NKs{#M;wBs*=xLJpAa{E(kih0ASOA}Tf(D^lVe+Kgm_BuQ3$8_~pZh&V z@-t1Jj15@roPgiyk<;#&u7lWN>IK{O`+4(Kw4md>Wac~~GT>~+F3ja{2R>oCP@01jL-T;jzxT z1Z@wZyKincDb_d7m`}Obz)IJlIqkRsr+8DJ&*bM-B~#aZQLh8j;cA;PM*{|jk;08( zB?&jNU$Z{DV#Cco$8r_74cusL-6z>6SC~>`)*W>{S#8EVpHOoSQ}yclqGK zjvdTOby?C6KCBkPsuoQdT6UcF}Exe%1(24kNJB!2$RYFQ_;^V-THqV-b2 zR=HQL!c?KCYZ*ul!er=ePg+nrg_eRW`keMAvYV@QO%)BIf6fa(gx23vOH&W-1nB zC#@;JU0dTvCH6~Ph_V$@Thp^eG%v9vIRq44UR#5dV_$G3!(LKz)bz()Ufc4ykvy_G zzF_=)MFKU^|Ggcf?jFq{@~*8N=9Ech!<4hPZqJ4WxkQij5{)fV|K&`&GRfC8P=55? zw0&Nsp{?}KG*&B@2Qs|_RjOpFgXcWYk!w?X$zH4Rt6xnboDSiRt%ap64GIZy2Hu_1 zP94-GaV6gV8>6~#icm!g2Q?B!wMb)y&3kJOW3&D!*v!f2+8Ou6eN>iL!&9B+E?bLRAfEQ$gNno}27`5c*w{(vll;3N4+s@Hz)th%gQ*$ZOH(Dpm&F zk{5%SW`zi^Dn*JRzEX~<9lE{XBdhVY-*R6wi_;%H+Y6ECxl4jN-757ELySo8qmSo2 z+M4woTk$!j49^eZ?7CDVU|XdS8ie_Y;Aw!N;u7%LRo3T6%r+QD*^2tXn@XWp(ZD<I%I>Hl+Lk`F&GQDTq#)y63*0$Ew zaf4Q~ON*vS=+w(;1tQ@&SLYZ>p(@#trJ+kNzdnV}OPX=+!nPyGIlQ#Z zcZA3$l)dJ198&ZP#IH^z&A@7~^O6Iu2#s0dkxqZaC#s$lF_9NEu$N+{zoI0Y3;fQ{ zp$2*c0%=qwnQ*)L$!uM#?3hX0Ii{*wR|0``pfKjkQeg&eR2j`}XEjIGAzhcNM1`$V zlrm>6+>=K4Dp#kzNjaWwpd#)Ef8YJPb{pTG4EpityLmy^J$>sYnII! zBDxvTo3};O)cfYb&h2fLt9$i1b^6bhDaPrn6ORh!pu<7|DkS!B?}&4S&E`o?%IRxs zGCpdBz55$55(T7oL%;iY@wf9kcgJa;x2?3uxb4n1nsKaGfcQ9n=8vumoIAXX>%)I5tpQE#^ z9gdb0iFC$+BGMHXK+5V>k)PO781Hwp2IWdIFI-ewi(?AxHZ_?n`8vf;o8gG{$(0%R z!hjmV&+==yQ1S49>gM`s=cy?+<-OtlG1x4j4lCEz+Ju4af`Hxb#y7-Q=Wbz-9V%O( zlv?kFP~GSo79LD<;!}nlZp}dgUilb=AZ^a22&iw_OH1LWj*5SMm}Iz<$k}popJ~s& zquZ6yU38Soa7vYZtp;J4rZW{=4~wB1H|Sju7N)^CUsK_Jh>4Dsjkf}XO`bke^fTGA z)|A9*Hng`Mm5a^cP91Vh6ZC2dD+qH;=Db19%FmJahF)^kTMWC%#flN!E6sibl}n|~ z8ui927{16D@Xxj|P)wVgb-QuS{*Z2y3EvJ)^8QqJ!ZI@)r&{QIS3&9t>Xmk?Xj!vo z8?ByQUp3b;`HXgi?^bNT(uUfF)%b@qT6-Avg9Uev`sL4#)KkNPcLeSA zUimPEbcZ(Y2UyWuTwYc!kYo-i6%L3o_*JjDCU3CmYV^ZIbDsv~7Nj}Cn#YCS|Z&i3;Q z`FGrSX*yH_t@TBfk{)p@r;QpGJ@X;RKTd$qX?`1wyq+%#b@hVHhCT94#8`oWhGftSLuiw$fimEfC0;+8{oIsS8*QerW^$Cp*_ z`u6Vva*`(?YStWS-O4u(RZ0aUX1vyt7tVlHnCs_2shb#``evGZW_e8w^IOBk?|MX$ zr@`rgkIOef;EaFD-MS`d-g8HA?n2W#Y%{B7P_p_Bi8ynKU4BUQ909&>#!VA7&VgUM zF{{l7KH6l2gv@qFC@Q#JKMI6YJ~hyyEymxm#Pc}dh?a;W#!cG1pTUA?GbgV(+`9Dm zcG76%hlCLqMAfh&(O7w#%R7AhMYD9MqAKz?xj} zJiAZ+Rjq95LP2Iq69021ymlzB_Y`u0f_%GHetq@R1tSHOMm;*KhV^vG@CbQXw)+ch zp2mz_$kyfUfjdTt1@m>BV-n~4kz8CQ2dKk9%iVQLF2gl^chBp!Th3)|8bMy!xAQba zXwc=TS_ry0*R{FTZ=#DjQ>Py>Abk31JXMCdjfk^}i%mD*sirMlaDsBOlu_i>HpgG8 z{{R)v2!<~AH(BJT$Xcb^T1_{92WuQCQSfG&$)5GcHRaK372gS85Q!z(w;f)T3fU(G(FZR>_pA`7 zb&&R*U$8h}^Kqo=QHLTid?EL@%$jY7{MbW2B;9&D7!-^hdaZM1N+zHPFo}iaNzruS zq@&@E4t{}~;OC3k6GXp*s#qgDGdMNF&Z0hWbKU7fO7wb%Nr3-#^z}9eRw9sA)PD5w z%Rjl4Cagef&cTdQAfoMG=2#U{S>x9gk_*XTFJYXs2ZnCk#VM-DXj)Zmu@1%;UrCu? zZC&X5z^Vg-CzTC0pRVLO<3h!}kM;`Rl{V}h)(%^5zksk`6wEsL&p|Uaqy3^(>S%$B zxD;;yf(aq=+=WZqi9jT+4w;B=F4iviL=}$Jrsf(r$>vzTlB~T-o^Xp)`JAxiMA99I zJNJ*w;Q$ciI|f{PH2-=%&!>|PNr|8kTA z{aGQcl3|G3h-LM*^Z**-o4_9@u3D|NdKrwx`ze2{YPc0A)j z5{H^y@PEWYO!8SGe`X&=4GRKLFA3jsWGN!>r;*R*ap7ml>K0mYhw!%IrJVhCzg_=S z4-G^4YSmeWQ#n)&)B$JZPu408iGNv5@PgY`^9oD-0OtrRyOq-V*{eWr4W`~qeSC*+ zF@U-B!?qTlou}@;EXK0(-!+F-3YR zC26^aG{>7h=LHJGB;nbU)s|_(lP0HwGPHJ;BrA9YnXJuc2i-dy3;_h;2$56VSNW1| zu3tZZR#x$gIM6}bw@Z3%nyOVDRS|~gU!JAuNQ9m4DjY>HqrW9dsNcvNRc34pCbl{B z8?NL&Li?>HdmIQx*Y{T$#*1X#y8Jt18W;j%ark<-)~(;7?&iH1HHUA*#d4r1uMTPtw$t4KX%gZ}JCN-FzWi0rOKyV()`{M) z)Ln=0D>>#pCd}gKgx&KvYA#MXB!XpGuc+CW;d6b7^Zt1|k&mMWhwwC)W-Y;0z6>%i zvH#;HQORb<@JR7+L)n-bOwx^^2CsmKrsSgq8RcqQfP^)5)@X#Lj-aOAI-_2>3dkcb z6NpgcjUHY)%bKqU{x&cdJbCW|H^s2Tg7E~BXjeN^W2(D=3ner^b>q`}M{}!)fXDj9 z63BTqm<>w0@jZ%@s0u-3DyQkRywiqzlv3MEPYvn67+e!PInwsYMt4nK&ldd?`B$ZYT-2z@st`lT z(PP8-TaDwI^V(uL)zcTOBX1j~=iU3RS)54cOIH6p7_s}=Fh107eolJx{LPEz=tCbp za(Wx3G#wM!)6f@UuH{3r+ZT%KlGZKrm=jQ@EVu;igbA>n(xgBi=@xyY5^9X6eA4qublhq zQ2c_J?$nF3wKB1Bmwqu99u3Fg+S@_F#a*(t63mX<0iMvZ=ibW3u0v7Rx;QEF&Zp+i z$?T=uWLK1e0JDam$S5~uTXiLZYl3GEREGeK%{~&TZC!h?tFyzUNg<)PwQkgbR4tdR z#%c5yIZdw_Uk)f*2iaW3&&I$=97vq8Ty%3ZvRO1EfN0#c(kfejd!Nk%7rRG5@@gKWps%iNDe%?# z4M-6Lb+Ztwp+C%7J7IDt$du z74T(5qQoWdN9IQ|AF=|itNTx{ZO(1vrqaydd)T6R`Hw^+HR3HHIxEjr75~J=6-*#w z=+nXX^24N0dTIEEvBU;G#9Pbe4rxvhjuUWux^zH9`km-8{u^{wN?XfX9ET2nN)Css zMPJeeO+{9yq`vL5RL{9t$!hNoB+-<*rapfiAlnQeyiJ5cQX_3l z0?TQ#$|`U-zv#?2a!(Ey+E5l@5XE{q0BFPpU4OTQ9GVe-2mvw07Z)erUjDIn-A7Ts zEK+3FyHg4T$uM(G5It|IR#GAi{WWXT$Sl~Nh9uC$EgyrlFu^5wg z;aLbXRV38l;W!GUxG!0oRKIBT-@QfLR4A7C$SFfaX5h}~Ag#<{$fXb0#Byp3VA0+> z-N$c;I4+;@^0|^n2U~tn1i5Cp<3d9;3!60GH!5N^lDj+)>OhU+<9T#xSKjP2OJ&M_ zy0GryveQoeTJY2_T4DBvQf)saV7uw%>;Gm(N`TJWrkSdc(+DT>gQPoioLo`%DXu96 zDzkm=ffivAG0@qL!FKS19i*-ejbsB{F-{NMxPSMOYk_-@UV{f>rxwmOz; zVR22@ghW7NnW5n6d^;e>WGsi?ZA%1_%v-%*6d!uX1anD0O2{?P-pnRT&bWk>u3(;D zI24|Ljhj)3KS)=ubT}l>8Dy1#JY5+a%L7tr4L68f%|ABjh#)==6olW*GoR6esEoLG zL-_z9{ffY8QXJ0_II2fJ*{$#be(ratqQfmpKaDhAiWi}?M$xI_lRumHz7s7dvOY?@ z=)r~9yZQrTD!IA~SW3Vx&9EQB^1fK0U8yAryhS+bE!(Z8n5NU5HwFT|3{>g2{RxXy zn8#G}vt7o6mEl4fs*Mo6P7*bns}5qXxf8@Mn!TTqd)G+orRmtETo^r`!$mWvKcvX( zo6|VAK-Z9k{k9u&hiwiY-#Agl|v1ZHZr_HR@cs+xCFQxQ)g=u z7aQJ+C|yst!TiuQRVA_N-ZIL`3ruB*_+nwWkg5T$BX%%CIjIEF_pBshb+XMHuH_WX zWzhCb?hdNqpKT7Br6Ap!@3Z&WP)V$cZ~CKm%8Jt9YiHmDMtGdJ2l&r7>0u;{B@~C5 z(DI~WRpN{X(&VcslNRgdBFvD7*08p^W!G4TV!8}SB0M?h{hZ_^RFSZ0(>~VHuZWg~ zU(tzgzQ}DvG0FF!b0O+xL86+2&6yS47U34N*|DCr5PItl+%7b@;ACdkpFE6KLR^oD zA`h`O3o1PJ60sn4$(3A_F7l~Q^zem;X1G$}m>drD53x`gw3ihhExe;^*k!5h!?kbN zU}WfwANffGjO>ETJvGId!7ND#?GCGia`9EZcd=(~acWh~%CCJux{oNs!}F=&9r8Sl zNlVHC#B)^!!1%e1$wydVCSfKdyflFNNx)?dot()lBNg@~GX~MTf5S_@kCFti z_hRuT)-SOsukr~zAJ53{w=cINVK#B-o@|6!=g4S=!;`N=$+cU^V-UT_$3 z83n58&F98k_nvDuX11KxVC6pD;DqswtX3uPC8`S3Ij zTMU052ul80?2X@c&;% z8m(Y$0(H<;5JOw3p7$k-B&LY@Bmm13ABank5J4?cBRibOi>Y|?8hyvB-NpEpnLCUq z=$qdHN{!)eHqtcoVj>2f0?;Yil?;Dnjv2^Ys0_bUJdnB9&_&r)pB*6=I8-xEP0Kvq zEDWaw_tI77_>z}oEk{H0=F@df=AWLDaWkd*R&0#qt7dLfpavH)lNe%QWUgHDH6qjj z2WR67&;%B}v41r|2G9g-G*CRVMN=MZjZh{y`6IbG2E_u%8sx9K#JXEKDbqY`eXWShaH9~}E2{~b; zi^AOU9<{ylC-Phav_HW*mqGBcH>=8@dS~YW)f9e<5m5fdAb&zeG$i^6aHaNXzl8k9qw{EH4HQ&<&N= zZLW;dch?@Sbuh$$Me49l2xxuh4-7!{Dcsv#CRc70d3M;}+sz5t_6}sN+hK})vu0o=DRVqb*Py206HleCc8Ct*PA)8cw`{RPbl1;?Knvmz-d=5p&RN`Jj2{@1@ zK@8YVi%t^t6xBZW`2U0S-Sq>S4x6EzE#u)xUr>S)DZk5W{uQjT-*s!W7X!r!U+fi} zv!(GndRYn8(2c(*9bC6U5wQfEXRfW{+7u$hEC^noVVg*}!6u>GmHpxzm$%o{DjySYZ5HQTb1b5(_wRxmIvGHUs^BT$i~+%LXtk54L`Q5I4@-3)HDj%{v+5 z_;{2{ox<&!J7|3Fz#|aR-ui~loeVom4O=B!u@7g!Sz4y}pIW7cNxCt&CaM4i%XTGW zYx{?B0qg2MR{{n|%%ISSbH-7LH0U?1)uIk_ba9HH@y$OJGhJsl^y*+q*e-yy!m*Uk^T6M+Uuyf+nRbt z0GNP3Hsu47gvXWoO3rZRIpBizmNf2A^o=EcxXFZx!t1?`^7_&jq^|=LdY|jl#GR*H zp1LZp5jNN)FwM@=D@274z^O7kuaNq4s!zdEYu5vh+gDA8!5`hfN_KrC#&<3293&E;}^>%kmBE?461EU9J zO^i}K7eCmO1#aeD-3jyk2=(e0JLa<~uGUce4ALP4SMR(+%mrK|ogFQJV9k+4 zaO4q`!a+5fM2$(Ci9;C;axtStRYP(d0T-z>SB31SA60_XVq>}t-vxZkP!NPUZrGcj z=w(i`7FYu}quvOb5dxU1v=M-*e#FNC_-ZP^2cgfU0Ur>X)nMEfpp4uxd~iG-tIngy z*lm3O)9pR-)0svO!NGf+N3XvDh0{oInQSv~nlNw^MAJWCeC4{A8|QW;FP$b_(28=# z18!3c7jT=}aj+*#@vng_QI+9Yj2Zyx)=y?E{{`vV00+vxcxfi5D(OYdJ)>vWv4Ng6 zVn6U9o^zkFdq$!%!gbgG(bZH1R=fg!^Xi}H+f%irLPrDAeuHhAIM93e4qdk}?&J z6G(EQDFc{AIAZw^Xjifj6U{>DFYM|;0I<&~`DU>L(6A7wzMg55G|b+z;;NDtbbCaH9*5mB~*=20R^+^IM!T7CEk#PSXe2VsT?O-X(7wRTYHYV~d+uKts}5CVgCzwB}Zu@$ec<3pVS z9E!;&7XnkZyvKB%7?^0d0=kE{zs{)<+N~pHjb9i~`X0DT5jTy)Ha+jD&?%>2!8E-T zvsl2(U4^1dD*wENWCWb3AblNk4nQiGyoZ+I_^UnfCg%Won>c3R(`zS7qLu`3f0Ts{ zKyjKU+SNE+ST|9credtp3GRpAF9R^kfHA2UhQ6=@@hQ2r@yE@SOp5}3=3^FizR2iK zGU4%C)vGuSe%_lS`Ky4b_#B$yv6QI|9=o&VZN!3nz#Gd)A|@o=F{(0>;rCk!X?ox+ zoU@;k|7N@&&VQjs%oy$UMUjYY$+QCh8qZUJbI-wSRl@Xt`-kV?sP@va%9p}Hj36cQLPmkfn=~$8(s743?DdB%hK++fAPk>i} zd0{5t9VT7!)D6F+VUxuLxK@eC!?iV2-@R6XMEXe4AqB1TGx9V_Pl>2gQ*d0Pi;jh-`LV>k(#*o0HhM z)6k7Us^_?@WjA|#{^$P!=vxHA>I-z?5S-^NIc+Zd9{SifqC%|04rF!oTyTlX? z|J;vocjJ}-@aDte)bo+;if)co028f@7k0C5rGak7=ji4_P8Nv-MSpn!^f9Rg#^ivo z#Jx2TL?RqX{gnT=6NBS{%8SI|H(Zjhz`sfaRf9)8T61m1wX>5&4Vwk4JB_;n)WWWp z2-zCBYh{?98md6+WernZ2YV$utN6(gIIWdGH83yk{h0bctK0!|J&2^wsA zELLFXy}`h1`inH~9K?P-52tcd{WpK+?~E2FdV!G3CH6Lo3n2S>0c4aHz_%;Go)Q3~ z&Dl0=-&81GV_qi6^Z@YsLvS7x@P?hKT8HgtHWP||Ckr8~ zpi0hvf$R?m3+|sZjaJax-5pQ9kBy@a#wxk-VB(LrGLV7spO3zwj9qv(7(9!!B{5~x zGV#;(%-(@?>0Vk;oEB&-2X7zLg=mNYm+l$%mdY@4E!BaeZvKtHQV5H(A1GwUbLclL z7o7vNL@UOLwMd~scmhPkSN59m`xfT__-}db|03_tqp57e_))y1P>GTTGF7HzNM>P6 zB1$r5R_0+7LTp1)X%J;5Qz7#_lc7||JkM0-VN-@J^SK`Oes{m`_jk_vp0&%Q;nzJ|~Bxx^-z7e+%3#_G477Qkg#hiF?SN3OV`VkKaI{BcFp@Ov>5 z0O%K3(UdQj&4nW}xbfl4RbM^?M{IR-k@|;$fYWMG)2aTgy?77#u&;K5KYg=${lG9b21opaB*7tL%q`XkG@b*UU zpTvQjq4hCgbcn?_*d4PW6>uW2&H!w-K=G~mq zZxWxoB<%V3_s{K|76Q^nQamK$N=}A8rzsEt@|EHG{N(7m2Yb92JjKH#FiwV;w7o(_ zv-2l}tZ#>f2$QQJ?t=bdB(kF>S2@@+JmtF9{N>Tf6!%;7i9+%1YIROikWfzS^8vf@ zLRML_o_NAWw#>cal~8~}CF>cTBvFSuEQ5~qVG4SKDJV@OIP(?ag0`~)AvI+jwDfbP^CLfP*sR7PkGNcox*N! z0}a>)`2nm3J6b|p=HWXj?QDAY`1=0nH{QLPg`}3v%=n%|B9*?UL|X@< z*TLWmM|9$=wPXRhIPEI?qi?Nk`5k7mt4r+SGC_alemSCeRM8U9Y@S8fqjWYSf2x^; zh4`@Fq8Wv4B3EE*=te8@6vtz_jXXYpPOk8!d{0mBE4;C~RA*3cMtTl8erEXN*WBIa zD*qdl%-qw?nx-Yd8fBy_S%>Ud&l8?+U^ef{3bsWxe zZVTJM^)*TSbp}m~=3nxfy4WaEwLTwOVV&FGl%y`6P+-xXbZhK)6_v0LgHzbhsZ8>A zS@1G=W~_IL>%Hp9A#zv$E-j5Y^18;@H|<(dNIiW=fAc*#27(iIHGk*o#@TFLoh~%A zwYl7|pDWFf?MB4W(p!7Omrz_w_8O9)4IN#u$Xx&!6MjH%ht3(Dp{v4H$+}g4`_a^>);Rr|A*T@&-zd*W{-dWyi~oG1f8|~jls)Wp ztU>0$QIzJr&wjGlPR{VKC**3^;G8fX!JiYqHEW@6!#zgT51pi*j*VL~C0T6(ob@eK z0CjYe&5iN$tQKoy+bMh053gBOQ%e zk?`{`pI^ZqO**TLW}^9F^DbJiYC&d=?h3uJu%SdWJ4{8F)thk6P{K zkHxl|&e%$%J|yxw+SYVi|BEghaEDW2;;BqI(|p!OYQ~$%LC76k;L;Ubb^HH+&BoZl zjjOuiVK=$-n)ma#HhEB6+$y+Z`~W&C2p2*}ghLSh)=kq_`N@4@FJGNu80bojcKx29 znR#L9oGMje;gCn)E3|}F=lP3wzmP9CWg;1aI6K)6)!u!r4PYPG=(~*4IzSMkMBia* zrJ%?3ZShEq$7*s9QkKKVOn2@Im4;d`Vj3cmZu-i5Wj$_#A5S;yBg-j`Urvb`XE@*Y zsQ#*BVi`LjT3UkP%k1SC(1UuE*Z}%86Y(XG$98>WH`9%Q0eKvm_VT@-K8&t4=K*Ox z2E~8!0_U&QKFXr@Is&?#_1^8B_~>36*;T(7a}G zo-$7l+U*`cYw`ec7z61IP^6fKdHudq4@%nxvQ8EF6ik9`qU1zZwfdMwi4O`Nge*$# zPyXT=5j^$mTRgycH6XzFXDk{sYbWDtPW}cGEG2f*f=>l$NIH;MSJ#(qT}Rt|@xY9p zZ!4Fz9LtUaF1j^6UT`D$2lm>4k=Eaj>R-&#OC8 z6_-}EB+5F5b#~tyZF-F2Mx0mmZM}siZMV`Is*rSF4a?9g)yR9JU7%W$EAM=CX}YUv zKv1paP|gt&T8KPql2=Tyh#|zuOMK(hW=uQhI6TDcMH1j9>{H1QT*w>i$t9o&XU~rSqi532xYoR z2-CFk;qG9QN^SDIEI1?wnA~3Y?F7SVVV`R5&JNFo>onY7-yOFB$Kit9DZ9rmKc5`8 zcS1=|1uBWv&SX^`5?$z9la~GiI(XnbR1lfM{cU6Gl;)38B}XD@Y}*Q|yb>~J#x~_E zrVbDaD#D?`C^4P%EtZy3xR&?M7Gb)(GfuS zWUfkzan8`Q2XZo-nbX90ZWbdtvH@^v4ZI$I4lF>`Z~r31_4)xlI5uC$>nxEz2VOi9 zCN8v0(D?lWGqe`11rgE)@N#bFyVa6=T?I}IKPfT9pE<3Mk}3gHWCf|h6w8I7{hB7?ZMGl(A$^$>&% z!-pPl1nYT9XYL|Z?hcTEJVDa&$0;5Ox;Z+q(2fz}d&NlHF!;N<7{m=-4XxD9?|sjH zBVxo=Wov|8s_wmIIs!P8ghC>E% zTA~n1@uCZQzFh(w^wpL188fMiis!ShG^Wsb7|)N?+UVNEY=Ju+8Tie!8QTxHV`*Kg zVknR4E5S~_K6GIeeaV-^pR`^gpch40jF60q?8izN+EgzkW!1ZDWsx0E0S)e#y2a!& za(Ds!0lEQ~Mc_dw9c)Kj9I4F|?iWf;mlk{vL~w1uCYX!flMQ7hR|6l<8sBwIo$j>6ADLdF0#kSWd5}f5(d9+W!?zi___hfbKlM22#6k89d9p(LQ4Ap$C>VygHYg|s)AS_nFwKmg}Tjk=4q_!{G^cSTZdLEFw zThdqwtk%ufcaa>!2uV4&B3%+s;-+R_5|iOVqIqdDOTqo|fT!)PBYlEpjKJ)WLHROc z3yl4@5T|dJ=ob<51X$b{xN444AE)GgPAn2)1!91_O)F5Nx@vlPI}m_vG1A*P0o2OM zKFLq^THlO1#v>t{3Ecg|7rPC;DIWnV16gJnPTRUbh?+Vr=leM?%3Wjy?Dq}5E4{Gk z3~7+pwb=xeAZ>T{Wp9X&uxC-(-e0ih>v8YAB&7!_dYhitEHaOvL|Y`Q?E51ySU9@j zcn_>YC1#UkGA6&!xpbvqJWMh1qcznZL)bx%3;8z-oVAMluy)6~A3A3Zx=!yEGNnHxgcy=lRh`u{M0ShbvTP%M;>!84Ti6NpZTRvFIcQmA+r>L#FGnpp=LXsoea1 zvfJmN6U`FBn}l^qJgmO@$Wr1k#HaY*b+K?!+g&+y@`` z;ee*#NLqfUwiiQZU4FGF*YDk4kHIFR_$0rho_CQTxdI0f?YTYdf$7fmsp@z48ewm@ zf1>jIE92af`Y@wRy*O>gomoqIYQngBzvHJta6=^{(|0{2seuTqi%3(2=xC7YqXDPB zhP|a5m&$teTWlD_edR>@1_Q7>Uo`a`)sI^rI^He3|L9+47Xf8=h!2#w&7gm@jC`sO zzo>`OK91Xe+B~PMO0O629OCThB-C$hN|L!kJB=>Ko`VN2|v`K{R^>%~A)mma4n#c7`Iwm6*H`ATWEMWgP$m*rMfriWX8 zm8EPTSwU#NqAf;!yncKS$u+nZ0$;pg-1hr+lOiFpI6CgX(O^+snovKNYOQh$QKk90^G~YFRkz^FP(zv$s$Ri|BOB2|+^_8R} z0oS>;E0sxkBfj)$`|QY8Sjy;BSvr(v`*vdq9ewZM{0w6OlJ-O3{@fL-Q>g=0)Jyj`Fm@SCKl;~A zLbSd~q}`enU`+frnpY@CPk**KEz$x>(@_~M8mV#dzS0{=*AWifdd-!dRKd_E(~H#C zU!ArTgR%iKD>;Q>+9>6kZ;*{px=H|2GTdM(wz8}ZoQBa*uE7f=kJ6KDKPOI zLFWcqoFw1G#jI%)=A$WJji@U%AT1GsU@% z)VvvkNTv_w!FxvKQ&Ee}h7$b`AR6L;cD;Xc?y&r602p3*nTI#1U-Lo&*C76@BxK_5 zl6`#owZl99sN10X@<2m!Vl-VTtN2l$Vkma6uh?ATHS#%V_sK*etoac8uSd0#-cYcL zq0e1`@~}yg5!VO5D!ukJ08h#&W7-4AmqJzBDq=*e(Yl)l4?Wdi*wf)JwONFD_8yn4 znW^6rFFNNh@{2rhO*E^)$IE*d2=e(Nj4I4?iEBd#V7;AyH`c}-e;I2X!q zxX4CM>vyR9-f<}F_?$1>LPja4saBed<6VYO*!qSek23zg^yb!g^%Q_TFH4yO||T~#_|$yo0X^MTbK zA%VCX67^W`ta=cd?M&Faj6%nm?`O?j5Cr~rJjwUJUxr%yR$m++hwRS{THp`|y_hr?=Kgjt&^=TzQjDGUi)A}}glK%qi`EW>&U993*{L+l_ z2_7sEcf>^INqY7zh(>M!+OL@GXa)d2MElfs-KOGcmn~%YaOQgEx(_A&*qBw+#3Euh zHxwG(ziLKS;1m8f^P^j=HieSZ#{KP>i5CNtfEj~0n{%w8H|}QO(DFLjEzZ~sAO3T5 z> {Pu)<=a3)WE;+Q`ccbc!>CKSc$H5I?K1##+$-pA`c%v06Z^Jy(h0WF!Ebqc2x zX9&ioeH~H&davB0MqNCKNga|j2@vS{D5k2Mh>~NN}p)lJb;3RJ}MN15LlL)dcLNxx)%#-Jn1Eqz< zkfgZ2DHAcDn_pH5<=2Q^D)RB^)7s|WObDw-_VEo<*c+6Zxn*_zRZ$*`aZJT55(1BX z7)vH>V>(4zOI~SZ(&@{{Jn!C>qhnxn!>U3+0+I(d>sAs13RKZkLlqS+pg9_#N^0g? zO{d!ypKL-`6^r1gj1DeWZ`Qy`xs-yXSjOW*@r8Zj-f`GKhCx@oAm!1H;@N zF5tT0(*#?e!bvnb_Zu?%vTicbLs-v(2)n+tgO*D_TSx_(yZV|n1pU1A^S_4GR%nGtWu)O^uQv8Ogt7F*w4Ig^hS7 zPui@`6I~HU-3qF_!Vr#${Myw!f?~-G?KlJXCH3Vq>Cxiv3fcw_o9v@vJaywchV%$T zP}z6r5s?_xGl<9g8Rq@c0VVz?Ni(q?*Ll!cI-OD9eGS(@(aqNL=+OX@cbk^@Kqem? z@!@8SP5S}yis}N_n^xhjN}==z3kpR`cNH^gj4~M}kQpW)k9bt~gbka>GXW^rp35&C z#yHdGr`r;5eiafuIsYyJiD*%6<5rXWQN>FYv%R-;;+dr4yv(rqB;EV9r+wPMMX5e^ z1)b9?ogY`ycFoIXZ)Jzh8elHDipmQz(FmPep6cEwVOr1OI%2g<+fiG)Nc10~@m4)WnkPrT8ldLh9H*p_w&n1J_Q%^ob4?;8_p|ad0H$)tknFHpxsW;?Paa>}r zPF(m5Msx(i>J|xj!xq_(a&=zE8Q~z8AcwAe1^5vt&%*DQd@wj_>S3OtO{9iA%Cl$gkQ!v*5{6xq6}yI zi)~I}ft`;vQ2=0r&chmW0DTf&Sb2ylLnJ)99W%u3y`T7+Q6!Cw?q{CaKZ;If&YcC{-dGK1 zAW-GvM~gw|Iq|51vm{I&f8T{e%3FQ804zqta80s|Hh%iwPadM;Kq=V-o%!T}$4$aN z8DB*qAhc9b;3FD92>d$g13vC#fwt+Q%um1=DIGYv591>Qu))&Ek>EX~D&tbV0s7Zl9Vj82?_b30&PubZE*tXi;;4;GY zQhobibDW6_kIIAJ2t0=e*hH0Q@Fxcmw!?l7<@AS+tbyO+JG@oEwL6OdizkCdzn@*oseItL!N;UmmZj)5Ld$ zKnivs$0zy`aiQB~K%7Jk36n#`0r9-e;+Zd-#2sp^0KfWhi2a{Um=pkmI42!INjOt5 zm{H`5a?b^bB1&fi5q0>9j1ouxKQ}J%+J}rQmh9;Op3D~6u^-^|){L48MA_nN1C8(~ zt?fA;0XUh`7umbSIUYmgb($>VE>X?`PvKqPL=Md$)qVIKXoP^$(}q6{{&To>VUpjd z_`d$Li{;@@ves#Yv>+lL6WF_44o2L>Pg;Y(8gYLT?i1d=i-Z!H_oqYTr39-04mmY) z$ba!=(GY)@jsWu1{RBqwSYDVo`1-M8&yQLBvscX%Vf;=cIby`&vmwC!`79L<;wY{n zLQIl-x>Y>2hRo&CBdwW@ML!coMj@u{yYe* zv9%@sJaP1tV270aq&h7AjHp$}ko}J`3ncz3eFruz!Ak5E!@&a&7EvnxqdAjGV5lj9 z0R&D`5cEFC$C&+KRv{%0VhmZ_OVuwVh}#J&TTqVw)nx6cADP$)-rfHRYZnO`q*W+* z=g>cY5(GKuUDW2k`rjEhzY9aZ;r6@R@t^M@(sHZzl_35ssSF0%OP{;~nZf_|GS!F5>!IB74a+ z&zKN91JR;%;EU)RVZ`0_5D1XU_YkE{9*>~>Wvzw&IX?n0g12MJ*>S|5X+fi&s3&pq zCyAp<;fBxsSG}>lvM37p+&|fI*vvWbE*)I6>b?KO;N=RZKf_0?kGPl3@56JiJRW!B z>mcAGUe}R*)c!M`k2nZ*#GU%D;%9q7Ae>6lG0O2~A5k(LNE*5KaNQa3&EXTFF=_`CbV1L7!XVcr=#HYZ_DN!MZhO>EMLOY|C1 z<2{n)L5B#xLB)Q8c?);%zT-?BKHX(_Q0CM=CxjAzXC#E`vk#HPPhNv3*;`I?|Fi9} z2w9L4N`rV7YvN%)UuV?VPaH*a5+cN@dGCDOG|@i$sYrkCBjeT!=AzN|dW|z%FUW zYu0K+I|3Ceu=~+eYkv|^M^M|sUmi5^#u1Gr@^E;rhVAhzQPPhu!vv4Eq1cHN3^i-^$B?0NtqQ*mrFR!KLII5PaFx{CR__C_zpK)-(IJ%R~eh_~eOwF8>@mOq` zsHn(GL4@6A&j_YBDx`92V`$4HYIvKB6g9f{uK}r=$Ds+lb#sdrgfdm%V!Jc$Bj;nK zm-_day^KHgSi3RiQl@5q$SdQQn9PF`iTA_{+|j(6?zTF()#mcXo=Y3^(G|a(Y_aQD z;7L_Tlqf@&$8~Q`J}mPq@W{Zuw5wM)XmaV}O^~qNsK@+8-`st1{A)pQPx+8bHrAxG z5Zl{zw=Z{T#wr81;nz2}jFCUQrEgs^V|8QkX2n*5*w|DIZon3EZEIQh-8e@I zK3*EJ=CF6(*CrA4UhuKz*wwgIC1vcX@J|67XndrH$)$onBCe1;8$F0`BHAlDb4X*UE z&l{@8az7V5j9Fr^cPjDB*qmGsbNMr%->`ghEAUBp=%u0K;`$qs;j{%GUOJUKH2H7g z?7N!QJyy{-E9Rso@>NY-tu2&o0|7we| zqjLy?j#C|CnEykeY!x#1YLQzVi*{XB;b||vi<{wY_gjt_lCa*e;kuQKn^CZw>z*&8 z>djWcPA;?0Ec`0;6w?g09~m+z_vGAMF7jK2u}`fJ?am-vR2?vQb^2Qb!{ss?3?40f z{deiSip-^keC(ZRGg#y3&V>y9hOX$4h5z5>_K-1*3QhT@Q;TaCpO|y*?rXcCIW?r% zycO+u{%e6&$8z?EzB5UkS|j)?+e-zGtYUB+zBv!M+I41k+57>S)zDqN1220g2Y3&p zZxtTTTX2y#z~<0kMKA|TpU35z<*aG(iXZprF%mBIFvYBjZbi2@%Bubh#R>9#SC`7I zu)w-Zz1_52-t5{6=v`=U67#+fS%H^A66SK7X zh94_txUI#;*GE^%N5W#bY*Txi(7bXR^|%e(&BQYkbA=QS57f``8c=U-=|7r{nNQfj zTu=?dcqptcs~CDHSZ1QO7Om75D?&*&xRT3ut;#8*saFo*H-Q48FAw_>AeYLbyXRh5 zQu?l+8dMK`@_h|`vyZ%_K_NfU=#TiU!qSC_&94^qVj)HKFNa+G+QNnx_0A@B_WbWl z4dp%6nXaQbv)RAxsxa$3G;=7RFvDy0%8Fgm`JP+~c#j(;Pf9Lxng8(pJutZ6xgv{RFBiv2v2dLH73g4f_>)*#$ z9`c<%OCv3eeAItelPT3c+y7mC!hA4!@!&mA(2D;T4%KUFzl>=^Oq5WbA6hKleQev% zL3x0OEi_(nMiVa#W@v-dVHa`Z8@{0VGU=m(G6Iq1m|ls1lPVZa%qjb)E8V$mF&`6M z{1F-aI7~j@Gh-1_h?mK0YS504uM1S|Tnq3%>BlS{fd9WpnI>-2A${8-inGcLriMEE z% z4J;FqdW3_vA685A0ZEYrez}hFDqEfi;EiuQ7_|_bD`6s=AN&Ja3{n48k8L+w$mIS_ z@Nfdo!@W4tV>Yuhkt=GkL40R5|E6&z3C1^GvJcKTAx};x-R>t;g5qo%TRirc5RJJ8 zn@MSY=AzTi?RWZ|vcidglaj~xM_#Pc{-Eo)D0gzjPL}HnG*UXN6&qyc5KwkP96!Ez z7;hrx_YRW_gqoR^txp7SJ@9a;$@_SwL0!^7-hFW3c$R^q{xu%(w*yFDxC^n@KFucV z@&8GWAHoOlDD4UsJ_OK3%Sy`^rc3yrdWjz)ph!-hM4Y2z?Hzj7kzKw;2#i zhUjKKZW8i<0h4WsZ!wG5K22zdl4@DT^r~7ew%g3X?P{YuVPBEM0y7`AKn@imbW63^ zZoZL9F7bq=%w|hV?g`42g(tmqDpt4A{wckAqYrKzM~H4mpoTM4R22FKC_jFWka>(i zdL#r6a~+i2w299Q2RdH|xMqM8wl|@pwP0;|&bHBUy7LAkOSAH(tvb`*00GdobHM>n z+^m~XvFYlt0g%XG0CnlO;$}Djf_^=fX2b`oldjl&{ByM}zOi9*ZH~!!A&=I`>uyZL zg%5`2gv0)5m$V~=xfpRNwpJ1vk*?y-?K@g-!tE%hM2+9?~>RLFr(zcZFYx`@Y zYd`mhDXRuNAglIGDm~G3uH>a6DEj*Pv>)g#qeV3tJP)gB`dIS!o*(OpJb3u(bNo2} z2fLa43F^Wfc~0s480jNwYq!qnj&&RP2Lr2#V!C#Iz0h&(RVa&W>fIlAkL!~Gw{TKt zx;UM%jkSi<93(aQF}O^fFXpz6O=fc;r(@japIetLbs9M8T84UY|Tcwt}*O_48U%Gyq{VrmcGzrDuum8Q0D=x zaL&_KOz|M5FPCG(_#VfKMXGK>6Ql1y70E(3HMM^jDfuv5hMK%!D7RovZHsZog?|K6 zsIm2*K1?>Bu>fRMLQ45@0g^!d7!UbjizS8RI7;Jv9SEA?`dgufy!6y^!5<5ZgVX5V zyu;X+l!vF=lZ^@{p(w;F`Tg$v4ZF+q&C~lW1+#!vVul5&*cY&Dmp{glz&zj`!z7CGo199I^c7^B-vS_JgB&&!*T=}@ z_S@?tEO_fj7YET_&(6$PEUZF>vFb4%T@FZ#g#zlF6HCYBakx24Yx#bm&rp_F?jSUX z_~lWE)*1bHY{8mmT(TV>5(WJ(BGvkNhu{S^$`yr5iZ!=o=dr|G6*67BXKUlUMor6l zh04OPxyIl6VnilFhAO5+v=z4;O9&yrP@M8AG?{#@G*HR-R_f4=i!w6B9cW?z$` z$hLkX4J-1__B+L_3V1jU(;+HyT;fuO=DkiH*IV*NNz0RwI^sx&%QJZx^WE(04r}%R zNQ@8zrB{rZMey|Fi<#L*)e#GjdAmBJjU6FJB#km;$A!D3k zljx^dObnEo#79b`4>WZ;1J3mZlI7LzmuR1a;+>jv(`hqGqMx!cgA`>0Vyjoj5qP~r zW*=t43eXL$V~rF)2J9irbzfT?!4MaZJ^sVVkSh^cR;1df-#-?C^FCAfkmh=6fzzQy z-FI<8Vo!SO9Mi+h95PlT3EqM6fr+j90D{(~einqvzVl32xpP_iR7Z;`^~MVYdhxlB zxfHzuCpwH;05jH~`dS}T#O0P3#KM@V)U{1=8_*tk$8&Z-K-9bi& zr8;^w9dAToM`ADcJ1u^ES~EBd7?@J{$x>7=6d$WLzL*eDOdn4xa7qq`%L%X#vl((bM zf-0$+nL?p(s%aheenueQ^!r_9G|l>`mInuEn;SzIW?l3otf|aOY4qJUIS5<;ZGQW{ zBCI%lW6+_>`h_o@;CG2}%6m(9Gdx_pEh)c8p;0sGDvg8PHac&ubDm=cdw9;~!_JSe zY7Ajch6}i~E0SAIrR00hzL>$-65FA2_{=Lb6=jyc_^ilsFM_G!O^8gFt9LOIKe`;5 z`{bpL#@z7w?RX3paOBRJqJ1Sr<1T?Zl{atesj}qbvfu)ry=6x*eFM&c0&EXA>&F{9 zx8eRi>#*i;uc>a9@qsOgXSVJ>l%$UwJcC)98jbFv?!r8f8Q0}PJDTKEYC3WqmH>$? z+3)=fg#$N4G3Vc6X5fO?u0KWhy?1R&3X-65#Z9Z-bQRkp(WY_YbqV%n%nSpKL zIsmcc9F+7#)+vtePqpHY?;Jd~W{nfRtU%Z=DsaCB^0)cYY+tZmCn$oBzLam=*T zPz_y{ZOzKDX#2qC+s@pDxgO5{HL{CZwd(rZPs%7qLHjW~=pb0^$bLP{kS7$hPw(&U z*<0v2$6kxX;^n^+asKrdd*rvC8_h-{ zs`out^fi4OIo|U8g2cp*wbd~17BEw!sQm{BHT?|ACoDkN>e-Dk}&7#}!P71Gd= zsq)Bj?!~00fqRG0TzM~TQ!f?08fMQpRH5{8PrDCquvEJ&jD;(f92wXY8G&;7z?{_Y zg0Y>yHvxNkakxXsk)F0JakCgoK|bDC81zA981}kOrRrzS#)0MSxcnT4_tu!jz~5*I z=W->$F}7$p1P`_-i#qfjlDgxN@w)G(1lxI?A5boIze?WrS;+JT)a*VsiLW`o;$G%Wc}S?y-InV)X$ zv}M>$dY4yOQEp;gDvA(dd5AJ5jl8E6i9V??HP>x1JTg1EJd)OzeS>VW*;I46cV@-5 zE^Q@uplMK8;*h9}tJ7-W^3l0e>U$dX3aJ|D#z0}E9<)B1lRKUxqN5r{g;2(<1EZ1f zTvV)fr1~cdw1{+;h?Ko~DX%UX*C zIyHtE`H5bIvi7r$DVBBx->A)~taR$#^U+vfL7WG%MJyoIW+Gfk1N`W|f?(kOxDtt$ z`qD#E73u78UZnttK?;4#v(b~zLpa5{9?t`folN|8zrN&&S$;U=Dx%{lUl+n{0*K$% zI5{eZSpahd+YFS1FXJ)^-oiqQ)-CR}xr&NE&1=&?t?O8nl57SsTlTow5K#LDk?z?h zooQOLtm49r{w5JtN)z1RZyYu2D4GoP0L-*@-j)(7MtunVOt_pha*L1I(wd4rJ_K|} zkAKIic4Ka_w)oXyy!Sy(0sfQxb($zbv^5h|d`JFWRxc*TZH^*&q4ubx?H)yLINW@8 z1(h{gKas9MBB^h__V8a8=pTf-wARaJl<`sMv#nENYkH;9zu}rW-$=C;tR_d_Kug~D z*5V%KhZk9||CoA}p_yV}o&(N1S1#`o5WC^xM5NL<5HoL0YiJ!PPmY;wSuRKJYo|k* zF&3|?D=14~aMpIXN}&dEuD7EAB9<@h06J0*q&J|bnYZOetXI*XEkY=j*{DBkBQZ{P z{;QUWjpX!$tNx5w{kNhHA7+3FXV%O;ZtE!=N+aMF41yMmZI8EBnJz@q4*arCznk$# z(}D!b#g3VcmbM$qr%P}z_vrSTCliADc925wtGmvBXve`ka+dnaiT8K%8CNtk-@ZKQ zIr(CmF{+W;eUw8VC8IqqNB*^k?lWm~qyPRfYvblvX||fqm+;A45vicCV`5hBdT&y> zGBh)EIqANO_F04#+z(!WJLb+HMpJDqefw(vAU)2z!F0w3KznSjbH@eRB{u&KW2^JJ zpeJg4`X;c&WcC9kzJ*C7H*dY=T}*4wbWhs(f*7n-hDY!bH7iWxVDmgM-$Z=q(A*M{ zy3YxvREZY~m+u$sJ3KCTq(M#(JF^}e?@|GjU*UYNU_R2C+1jl@NuaTY-1L;aUPJ27 zT#5EH*_X%W5qDEHoGT-0BiZCfrB^`_MWY~U{8==Lx@m0URRkfghEfLQC4H@x03~P- zL%qK9ET%HzVaa;3qeotoHOo8hQSGnOWmRC^umeSIps0z=4KN1#Y({z|}49&Og>pEip9p2iOsw z`8D;VilyG5EE1gP90seIFV!9W4p=QJ-s{9ow8Tp9(aJM_)1GNCq(3HjnM7Lh?HK{< z5`u2t4OtK}(rXt9F1!Ot$H}M0v)3A?CJo2(5tg(3i~=X9h`a{a^8-R@yO*Hi2mO7* zqZ%Q|SWa^CIH)YSv8d_-C&RcQLeSLHh#Nt;Kc5Y>rm|JW7W#D}XGdtsu`yEARLa8L z6CrDqh*@&akXcK-gPR|%YfzYfa^QGGQ@DwsZsD5hw8lv%t3$TMx*O*NlRBLoo2&<7 zw4(&icD*jT86%kko?1~~4tRswH#PO9JGq=xIw3akDCu|6kQw8j~C--s>)HogXKR@RbN zJ{d1?m~K~)Qdy5dudT%QubL1`bsG7_a zRY^<{S}NJ>9tOOB0$j*!Ke*ZYm)ayvsGp_mY8p#wJhicdRbf5_FZvlq=1TAcWyz_Z z9`&LPe4p^RF`>P-I-yT@V#;D5>It_7RcKhCw0y_qk7qUgKUXvkgQB{)1EP|{J?uY=lNG`qI z(78o`5afWPl6ymhfSO@B&U}z!g|r2j;pO8&cvqCfz45~1gUh0l!-8;^TPnb`Kg1U! z%^`mnJT%MwC)!;AQzOR0nMdIBEHTU_kWYPn^hZx*jfGkB&d^BNAl!6^1zZtAWU7qtMTTc$4w5F2ue^Xs#?nJo^nyiAEHAkZ#9#u7vg3 z6Bo~de=vy?wo637m?V7iq37=ChRt*iLb#|0Hbf6wBq99v`T+IQ$8L`%AUASW1@_?S z#3<>V)uIlADH#R{5aO(Ytf&(zs;&J<#_2b2Wo}OeEi1v8(*a|)?=e9?P}{@jU*Du2 zkl6)pAZ&q=yo=^LLnEPKPCDcp@D#tVV?V$_vQ&szLlOa-u%iEj`i&tr`%fa^G7sv+ zk$H;u7)bfc!<3#!I_=wmp`Z+5N?x~Gsqu1Z588D{{>C*21hxiAi6+x6jUfDW?_~`u zYE9Ef$V1iag&c*X^nI$`q(?X_b4A~@HOB7DiTpZDiBEwJA0GFLI0D%O&dG_Pap=+$ zs;Te7YQ+o@(qN#Kx&{UFCO|>*8d~a@1yf8vPf!f~{^dE%jUl+WyR)1EJy?}NvLx(L zEO^Prf@B|t2@tMa!80J)i;Z?gbV96y*@eMNWI)w4hq8kll`6IztEbi2CCRh zAQ~5#no*(vp(L7+j28dL$O-#<{NYdh3HtzV4u7bI2ljYHz}`-FSx6de+1G)@J`wV~ zSw4t0vI5C)nHG8jnv>Lm1HlW9T;MTo?IM%UtEQk2=izUPz)sp=`8x{-QR-dgJU{hyk+0O1HOHloA`i6z#e0OrOUZkMb&Kd^n zp?e<_tF5W(4>$|wWRSa z-TS~BbqJ1?sImxQS<69Bx~twI;2b`~Djx|6C-;Pxq?|5g zpX3Qtl~g@Q=;8YmVs?^NmyoPwdTyk4XLaxE_}ZQfO9}?yGW>$=6}|8=!DjX0lENjZ zxnwI5>t!sNaz#iJhrwyH%&;g%Y=0Q$6xW*+> z9c7V@R=x-wHC7b1snld|eySEl`V;Fc-S3@DZ~_CNVL)A3sC(RTFO7hFnT{a*}48(Ht&qOo3mVi=`X(Ylb<3egFZ`qy?$l!apWM$=a38>9|aS3LCTqkw1$Mr@w!`=(W zmjMS3Vl`UqY3IIpf7+LWtfOS37K;324b~PX&j6>JWABOM=WjC$4?c(!t@h{q(DIuF zlI>%c_#z1qXaf?`j`A=Lf^l#Mu1AVC>&KU%rrTBlN`SQSI$fmeMLC{FK_2|W01 zs8#M(e9Ec&PHVA)S+aU+NU;zQgXI=FoC|(&Upd{|1xQSdfk3GPl?sWYygc%EmFuqNiP%xWE^j?l8Ybr7151Mz2izL1cIOz zv@nuffmCRDntqwsyz~+%;8Ccy4hCPa7F4vMhI*z}?$xn|x9pJTxMR zz%uxLZYjy2qMRZTo6ljpacAGidO}|$lB$$MDqxiKM9Y!R7iEQlUpj~3NWTF;E^K-I z2}%-F;fk@yHf{jyCJ(rNHaPxEv0Vl`?MTYxGj}tApiRiaCP3;Iwpn1epQh$BtJ_0q zS{IXb3Nw0eGM>sRg?oyBz2LelZnN)LBrNcc^A`B7CT9Uv$8Oho65pxnFc=cg|B|+D zG?PzQ&O>WF`GO&jwKT~drU>i4g7n4N0r6=ucIq0$*5&++;1UpMEXP&YoF}-y6Vl}z z2qfeJ<^9w9NJ{dbZ;C?huAb@|Xy5jk8xoLN%(X4d$mx8oU1fQQH08*T9^GJS0svdqh$f${Pw*us0<){~EPnHl)AOa%qH>rC4=ka7T$d3wfRv z5h&5!4KgD_^uEZ0b$2zeR(WTf@-JY?R7@l6Iv+o@+4=(t8Oe)}f~6 zv9$oh;l_P0%$F-*3|(gnxPu3;uz*Rq|C56bVa|HsB-|IbMCRPVTA5p1Vr;R4F!eG9 zgd@9A^hwqoWtR8pu8BQ1Nqwxa5AuoaPO**wBR&0tQ zeBQCP74M7H0(wyC+og*NJDwG2rT{X<*71!sVahyF4KIY%bC!G+tB0-Um(J zicLcN0<}>1{8T^!p3x?N1#%R=E~Y^{pZ@}G&smMOT0`(V*kJ1$@xM8^lL_$ z@D|XLezjNl5AKNN+*y>e9?Jupm-w7{0(d<^HgXNEdlxanpoYTXZn6gQ{?>AL3}V12 z*uIV+|8gB9UC$Qu6Ti?^uqc8D8?pFnCDsi-H&1EtPQ(j>JemjtwpOVVTKg;HDr-R3 zo`vA5k~lDB2XKIA@jSONY-iNuN+1alMQws-L5Fx2Eao;wq)^bz10ld5&^L8Jx5bWR z6y4VURP&oRGv}7LLYEbL$Zx!Rg8GH{zMEXT@Ut<2RS5kmXM*2>NsmFctnGoACPoR@ z8sC!_M?edN0-r$FljgZQiAqt%z;yox+Jx&{cBG8z4^xs{LyqkiEwlQawIT(uEJGwK zH{q;NA}%VmK;awpRE=85b{lG_RzQka>&~KGH3K`e%ki5?nFgmqof8K}bPe=|7=eyL zK{teYu0VuExW{vAW1({mQk%v|1{`78+Q>qhShR)e(t^{mZT{9rOXz6lnX`a_5u>#c zm3n@hItj5yJKiba@!x0VKyIQA4f6OM^a^Ha{egdR57Mk=B-Z;-0Hjg%7@-C_Z4)=$ zbrUg#!=V@E4DcLjb(OK(6ptP|DQa)rDcMD*hICm~sqk>OwJQeN>+7Y1fj8a?1tV=> zBwd>Pq4+v4N+F@Yt>p6>a(e@=!EEo04a*=ibu65eT^B6yyv?MDmb}+`l7Ht!-$JzH z>cOjtTTs6k1PS|Ven69QQqWROqgs%(lS#Wh_AfA_)t1=Y{0QvL_F$3LK-&Q}2nyv> zl;@)7L>1R!;Kv}>1M-Naze zs<(d$?emP!h_f^bEgJ)nMqLsG=)RAq|CbJoSrVC%r~Drz3(pmDQ5*xEQ){F)|4`9L zLK-PKJanK581BkoA6?}snK%s%#2xw$&`fKP4a`&3!kp0ZeP{Y831Km_>?cv#Pz?XB%iwSN$>4I zX#RA8FCz!oEFx&TUx8;OJ`3C;%KHwUU@miAIzsuOtQw#gwn)>Sk=HVZ-XI`@j&2AC zrbXWSOkod>qRY#O<&uP&09M5-g&Zxx-KU_c0nvW#33zDzFJ9RFTcW0q@!C%bw4akI zUGMG&fVBW8-kpLUMZn*ayK14SHvM7}vQnbcB@)@zJ^YY8_D6aiwr1Ut0ms@s9^_&V z2Du_g%lg7X3QPXe4v0AF_3pPQD?;pm;a9@(kW`t69$a-0dgcWP7Jq80W3V6@NiZBq z-D~c%_z=e;l3bVa@kMS`h|Yb16M5F=JD1Q8NW*r`NVr2_^*ivM2X;`E()!qJ*#T*l zsorvQ^J^{72S^9fVCd>}O(X609SzZG%L!fm6cHCS3_`AC4`{*ckrBpuwyP5Q)wX=nZqAU0Hu@i>X6;()4vgU?6v zo;oQrh&vq}C>aKZH2(>zQ@t{p^583M=0FS}m%(Q_g4@VI&YzK6YHK|gNx%r{cy@26 zx3r$8rzV3zgUH znilqXO(BMrUQXW5U5J8Wt(3qEO};zPQLo#uv%Xl0$w-9(ldiU(WNtG&{k_PfwZZZKQE!BMlG`E<`ujJ+1#6dv6|21>5$G zB86nAsLV7GAt^EsC6zJ{nN!I;ghl3*p;2z3WF`?A!ZHsfrOdMoidHGl zvX2mT5zv103x6*;zd_Z*yyY~p#Cks$SU&eH9%yj`F}_j(bnI2g>%5$O0062X;XOven`QZNr9Nlp3FuMQDeqwt4emKQ+Z5?afq0W55TvfgI zdq=@mx+<2o1D>b-0$_D~-+gWY|6C?#g01AoeY=b31^7iT<_vh%*e9^fpp^8w9X+?)}!m`jhG#hc#Elg7Nzutj`0|K28KB^k2cw5C$z~ zI*AnY=pMLaxbhtQB4mp}egBXudxVEO`&`}*m&`$r34)+!3_y8Qa0gA|m!xWfya^SO zJYod*JDlhP(F9~0tKyLLtN)w;p}3a^Ct5u8tPyXf(JLYASJ{eZ+Z>pC2<&+sY$-Yg z?_qK>HMsJui5>9>KMgDPRe=NcaFl;Sv}Cu#S9x#e8-%?3{_&wHib(*1rV6Dl5hxg} zNQPGb-g4?aM}vSbLrw(eq|srX@OV`_ftBnZdKmh?MLeO@+gBTg&e1qsNf??l280J< zFMAKf#?1n`<9>}K<ZTVWM!pWuZR+>V%+X3FeDoAt>_dEsKNjcaM)*Jr_5v*=8 z*pR9Y$3HHm8i?4LQz?m5-C%q=Kr;L@5aDto?lb@_NE`<$P)kE-#u+8D)#F|`ob2;U zJRk7Mj#z&<1*?6LT|~Qf3FKvfrRmOR#2P%$Qb+@AS5>aS&wB`x>-erA0<(1Ek{91Ae{(rhO?0AxG`!M7grG03oX-r z+opgUYcx`JH-sPvfSD8kym|l)^P6iba4eU|55>>s!S{PT0|&7UWbX)`58d} zgn>I?77C?b|C#4nC-`%$1it4%(!+AAWp|83cVWKmKC3v zR)?M6SYI1}HUUC0bBb&luS1q@Rw8;)=q&l{`OJIg9YMw<&=hXhoUaQBJ|bqgIAH zQ9S(xV=D@{?_d+x3==T4^vY~x(H!&Yh6XM{0JVHB>jB~zw%6E0?L@H=z=Bbv2Cl^J zJkK6Z)i)qfS3mL)b(0Y(#8kIU5f`}&1nQsXuRde4LQv!%8m(yn*BeOzM3ZqsR`#gh zO*M+Tpk&G0j~w53n=eGH157`tgES zRO|b-kjx!u%w7%muFs1WOyqXeICBCW&H2>y)I7|F70$gYJ=qH)T$etlndEP4?m4|s zBaIqIopMUuMN58x5hNjnb&jABWZ@DnvEwiIZP|X9#C}oJe}?S=u}(Vvh|K$c&5!*m{?LUy_IO8{dxof~bvv0(+^J-!%j=;z?I zs-|*3fEZ)-e%|!k8R(sCXzEE^PoN*@C4v-i+H*IH2o90g9)} z-Q9M0yv;_+iEa!b=e_^{NUf&BDj#f|{#QqLu?cV|=Wa2`vyB2ST1CXW8GgbdxFcd2 zoG#~upt`-9V@7RoF`u2V?Y<0sNbcv@cTL`%ph=;iW4G2e0!GYQWO%!~&%EGEE**}4 zc;@aj(e&1irlTZ>h?N-6`A)pXelPE#Yjz#Bc<5|biHUd? znZ%vU^;~I6B%H^e=PJ$(Hzo`~oIgmoUQz=Hz_W6G0mmW_$nhrH<_n|^$%%rrP&;||ykrI?10Q%RmxZbbw$^Q(+p(Rge~6my zq4>~S@rh{ra zXGQIL43s5*p2>cK-PWKt1ceFD(DMW4BefVX()YvFK)LCUtIxkzh4LwQN2)C(=2k*4 zYNVNOa99~O;y{yS>&$~3^DS?=kBEzxtgBTbPpJu%tvWz|>A-@#bdn8t{!y$Z3!K_U zt#%4l``i3=d}In>d;V3n_YII@e--t`BLn=g>jOPObP$h}!j7Vb77uKDRohUsP2DS^zoLttYN1jd?W!jyV(})Zx>YGdv;Z@hS?GPk1~t4_F#~4gm7~f*O&} z7qrFd*LPhT|N1uG2Wz_!7vNuyOp{4mQN-F?#4f1cQaoyDQ&MKD6QBBatUbey%o#^p zVXG3X?L$Y&zy;?)dGrA>Oc@52(;g~fIXdS!sBM#&xc>Pm@H**nF@^9hPSWuVHeH6- zx})tHFb$uhj>f*(_LW&L&laZ#u5v`GTc;RJqztq{pE>=gFh%dKEzLV{yj)yC$2V7X z(z$t;C`O3bY@crq8)77tfxG=zi2fG@?1rB9N#d<&N-&y^9YjWxrK)V+)@B$f~Cqj)M09kSALpM#fh0? zaikz<9?Huww=1eI9=?1B)f-1Z?F-4g8)mz0%jAH=wBPq{XQgaGsF@cngGTrfDv)*Q zkvtA%C*t&03dD66R|yw!OU zidZBIiz<8v7>a(t!v|~fYR#*@5Vb@3#iK=R%$qBh<$MDVr%4pkG7(Dw8|3Q4IiJ!4 ze$z|PaR208At6{B$Rqo=6e1d_ieVA1%k`h%KOVxC)A58=oIUhs3w$Uc?yP9vG)#y) z!$`6@Hs2nD<;>{H^GA=@$L z=Y5;I9)hxSyI2qr>+fHKfAE06Sa?Fza`QtK{~DD-AC*T?$}FlliOmJZ!8y==Jkdb4 zxxkkw>uUOniF+e`(}A1FagJeg6Oc`NXEgSp1(Ci3cJE&q1AeO?H~3J4%1*Y;TL)|@ za+s|R1%wnbEL_$B=J`}&;Sv6+{CyDU4fkRpvTghxDj)vz?d6G0jz0P=F!;WU`%iCf z`n^E-kkjCU=*?Tx5($5*)pg|lv!A5ozW>In_pgXjG=bPo5RTNxVg8$vz8?zw+4{aD zjBTD(<8F}jp2)^)f5!G1-3tLluWOS?Clb`pZD7oqP@g6i5Tm0&=5*q}uw%7C&c<-< zQ#aAoh3#}BgZiQfVK-tqHwYt&tx>A58xmq`Fw*NqqRF2rAOebifd`{`fG38Q zMU+lUgt^)Fw&1TTa~&3ZKv>J^FD5Knn83~F3ZFLDLZ1Vh`6p5Edx^57Fafet2AdDe zxi|2kt4Yy51m^)hrU|uTg}X+(|GH}9Ft|w!*YYM`6J)>djN-H3#JN%;lmq9v7k|m! zyC*=>!EvZ+Q@0;Pkr@l&x?7uP#S7ZkP|eGBlZuO89R?3l-tiYT=e;DDr{qwm+U6Av zfxNL{GzWHbBzm3;Fm)b*?VF1^cm{rtYj^Fj%}L!}qu%=nIJuaFC7d zut3#|TcZ9PGChDo2EIPAY3Ey0>}El*r|TO77yk1ZU3`*BxFA5 zty8{mcXMzdm{j2VgUA0Ok5Y6ghy#r69H6X>PGeg~^H10Xw553;z;EDLAX)TBg70IKALcqoRYfo71Gshc62J3mvCD z+-$?`J%3F>bqzk$YFERyxt;L{{7@p6@E1ehPDpbEPWLww!KNT)mY}=RMf8|Aj}LLo z{0)Lf*B%TGwoQSO{U3t^-O&FrIMA*9|FyvZHg-x=A!e-q`!R=274!ef;te`9?Y*(! ztviHj10cucO1i$6G&E+58~)jp)r32nAnrIz=yps_ZLYeeww6u3-N4oTef8u=uAzK8 z>e8QG(IUH?B0?=ycD?quAt5=!eEy7_X61L6?$FU|4QmgZpJR^R4BuNe!9w6uG1%9=5lW@ zcI(Ri_zP!ZZaj41tha+_>3)@<=ngN$h*v+J)H~nuQNp%M8Bd?ro12#5hVxC$z2egz zyOK2X(kEHjxb%Z_WyZBZ$*$#a;N7dY+sIkFI{0y(+j!%GO?|GFamfS)PNrEAQZt_G z!)1Fj%A$5pr5Xti^>WBzb)GH|v>aq3LY& zhGmnT8YpgYkQUFV+enPusGf8E+@aS)JKVlJseY!pN09GK8r+>XfP6h~8LOTTBd4E*f-nTltvx%t&TtWh75(t#sSk+Ik`|^8M&F zULUFz+=xz{Pfee4vDh%Wu(AO3X}DWNq@g;J0<{EURi~!rJ6MR0=qQFjuO3R2^rEgT2dY}lHZ8XgP>{@B4-IS4+aoo+c$_&bjGE<1ec zx0wt700LH3!$5Wn!FGz>4?$N(lc852%d&D1;NR++PjpVSkgY;mLBumsq8#$#pf5YJ zG$kXJXbBIvtQc(Vz_1$87q@Ngd)6*eR;+zJwi`JQ_kw_kKjOd{Ssmg=?Eur} z_;fO{7B_S!x4?*hk zmKhhHoURK3;;VkCj|la_`IBrwI$}2+CE=K0n!JcQw~+gjCFdhawE}a{`WY{yX_&&{{7ouJEjGuTYfyr zg^i*@G@?GnTQ7KuY;=-M5$re4FCY@L)5P^f@AV=E~W1F2)J}FLiTO_VE*Hn zpG2Adx7Z8sZ$`QTQA~P6hs2|rSaBU!J0{3EvqR3A^EDYz@H`!@$R~&jCqjhS?yV!( z__Fu5KA<^W+~uW-eW4dHK@;cb*P?e=coo>~PDmLx2?46OQAi7e0(YMtv*>8Oy=)W% zwKR|R3GKJB9(@bp&}smk@gwkLj(OvLfV%M>I0k)qm}I<)0Y;?*$#Y&r2@&Poc^(1R zgpET-7+un+5aHX&M()Yq;s{r^5ittX88i}$sTX4qL)qIduVH>`#P~KJwq*RQMteW= zoo5bC<6qTo)JGedwgSXb1*+CGObZOQqIw8d=&oFo2qAZ!fE0kyT0~wBz#L${*o~@zS+HCUxKqw}7f z+LSIpF7QsQ12Yk1<`G(bcc(X`V2|7NNa{jT_(&O${XN*02zfmVJx}SE)X>8DgJIML z6opj*geAf#_Ol9#*&^5p;!m@*efx2b1As+eka$1=_%A?;7VukM_9&;~&X2JgeJm|H zTa-|VQr4)=v8!>jH{)`2yOZ&qQe&a|!Pu!w#26_3e*uN!m#?~V-Yl0zQ?=yEdotA3 znJlLH)YewuYYT-cqt=s%@9L%d!sM@VYQg7@Wh=O;e!!yK%mBVA&f(Z);3)`#L~&L3 zFu+fqp+>kiq4UsBt{*TUDDHe7g53R(vkRgT`N~;zt@YMnTR98@xI-P01ISC&xA%8F zt1UX1gl~L?3R(kADH&0o!H)4HDPjQRzoIo6cKcu7Rs~?=aCgo2>#Aw`w>*|dl5Pk< zLUt<0Ml$TE?*78%%XB!}(>hkr!FTKz^n9#FtWb4U=U%58Vi+d?7-OnkREp^8I^`(X z4q`1^cNuP70lb0|fNc^X`ak$$eGZb~4<`4fWgrv~G$&Iyv9GPQr-0VbB6R@(O?d-Q z`g?Mr<+crA^>02zkld`RAGa5LyF-nVyrOc0kkqZ61MqRjn_~;<$#pwYu(9dMZTy#@ z0piAUH(+?maT;m*SiG>FsGnrcWQa-o(X!3~HKrDEDS5@E?a}}w*$4)z3`Ok>BYvQH zyvvl`u24M^MhoRMdvS#a9Y1EK>v;hbCg(Y;{X5dhDkI1EM`cbY3v7SHDGt>|B|>r` z-u+A&sEHdM2o}NWIomkN<|HuN`^0PSYzaHC58c)Mn67V#}Ft2IVX?vTe=50`B78 zykZIT8GX@K>OUfG8)dg2Y`*oceq#sPU%MI`%^=ZRrw`U5@;6a>?llU|6_4Em(H zdAwr`csU+Z_DHHMxo~N_l|=3(PyQijE`7Rc$y>0NgZpA?))DR#m&CYIk4%x*9^rn8 z89VhptE&wMo zP8&nN_4bhLtpQyWE1#&cHI(TiwJNC^G!QbY%Z|IHhWA?621wgQM`ioru$yFO-5=)Y zm@#7Y2SYTJ&lkPo_!P(%VF_i1`k#6dAoG1BEn>_*SMR(H)EwwyA{&m+0SN7t)=e@4 z4O^gGx+sysfl4Xz9s>&ax}o5lZ?<}mO?yX7SB)RrGf&K&(H2n9wfrp~qw76dr+lO$ zZG;>GyQtzd>YjLaULBl+d*IRqY-AhHooiS6pkU;pm`UD}BZB4&CPF!VXp`&baScGP zY3Q}h_i8(on`^XD-O*5Ph<2E_Q%M~oZ#ZJ+S8`<1x7VV*c)>pR!P`qObk<6TMgzPk zW+jHDA~(MM$Q`pb3mzEo?S2hxn{8>|hukOkq}sek?Ih{!QlyE2 z7VR}5*~@J<8W|FNYM0!XpJih~EPhHZ6lQzhtkO-J*f!p|KHc}>lB-!4ZF9^v8s_J} zz*di$d$O(HZILP^sX_qit4%;bqf=rk$BuR#V->t~5Dw`A`%3IMsi|nN?f>3vHa^1G2BmU_qlr zjMb(54DqjDu4MVu3k5u{-mK3plx^T_#-LKwxqij-$D8dc(C7ZfE1HS2>`t+P@z!YP zvKssDZLQ8bmN{r6Lq~1P=89+T#vL)kx>PM0Z+rezjIR#oWw|fy+WfjAof&m_NS+~0 zYsp5O!p4svmr{UA_0;R0k8#jWOgeEeZ%tj%EHp1}dPZu&>eWDGYr~}H_}VW&WP+t} z%&LuNiw&q6Ow^K6x&$77kfu=A0ePnxCWE%6ucgfcpt~7NNelTb* zT-L3ZaOX{mOjGI>XLjon8uRE#Xp+PF&xkOzN{?)}>Qb319yvJ_a1v)#mwV8yO6o*0 zb}rI}t9zzUz9+hUW7>qy(9f>keW=qYV-%AV<$Qc@P9|&E`|{g>?8j%SI-)-rG{4iC zvZJ$$^#CYjVd~6WuV0kt#M8V$uNWH{;SDK^BIe^=V}*J>>EE5_eVjM8-1EoHPj72w z4jwC$gUf8K?!EYdqdPZpt;L3MMsL_XTDzf=LdFDh$*-jo2Nz1({~R%3eGV0E-Z>l* zhqp-S_xh-p(@%-8Etj@riVM#QxOCA)?Y2|3xtmp^L*pV$H~)v-b#_oCzy^AI1B@F?y@j_h9ow_q*Qt+eVs%FVMY;DUJ9B3z zztF72pcKTldPv$jHahkB6S+!kslVe6pUDe)TUsZLvlvKSj!liSF*Ljx={nrTH@|$F zbW3-&2t2uB#rnPGJ^(H3P5XgMPf%5z7|)pSGCS#1Ct189_o-v1(_mg)Y(+Oo>!+77 zR;DMRVWC^UZfZpy_RPPfvll!Ougddcgh1Lol)PIVdcKpeXA-byK?bCbZ~5XJv1*pz zCZZSs@2GFN3cb6(EdQVxDaHt&<~3Zi+M;JCRtA{EAF9LlUbh7keR8&4siSKK6?N3< zv>Z*^TtjQ6h(SwhdU!y3d>g&i26;(*XZpL=l`fmp#d0aQna-{x0O>VOOaZOmH33(h z17kf^@Y!hWwM;pS#JiTbCz3OnsqGlPeN$>Yu+970(`05>7J;QVJnee1&DVvVSQ*D& zTm8emsV*nWL|1bM=2qg|rN%r)o%!d1AYgEhpqpXdp|`zdsAsLV5Lb#+%o2xV@rtz- zNt)-+^~PVh>4t_=P?XPMJ@vD9W4)a}W8+ncVuKtuH)@Hsp>$AUXhNxBO377(0!PtF zq-F(dkftIY8-$GS`;5RN_=WMrmQ~NC|PI;jTDh>*1_F9)vji% zDg^~GEl>v%=`1+qIadNdtCdzV4}5R@IxY^;d<@@Qb&}l{80SS^@CETk z7u}_(T5@vsD~XEtYJ+WapOI6~aM)qz9nJ&GV3^<>m&i2jSeB)g<*pjO7WU@RSYf+?zs4Z-p+Mm|l>y1w-n4Y*HW^@{2tFbV_5XS4>y<$U0c&b?s1&t? z)k#&#X;?LnW0^w)nP&@6)}(#;leJk_4D));R$fejt-EdZRiv4ttYIw55q;a}C|aO@ zT_qFv!&vwFdS5%Ae@`)t(x(yScF zuDLY8>*4Kr>4*8+l=Z+-&7`kS-e)wmJut>TO=I;NRa~*+=@)F@GhtS6!Tb7H7nG`M z=&?;zHZ(NQv%i||4fq0h$g1T^61F#4yz8q_0-rylHPty)tu5mLq-=ddje7M1v?;)# z#b(6afaUl(qb0R8v7DOzK#m9GnCz$Rhqol}P#!18NzXMfC?sxqXn)XH2@vr(u_tWP*3 z@EtV4Y3;Oo^IcH)slGj;9xA9?psHZPB_N~p(SuUQMw#R0TeCz%Tn1mg=t+4x@uZ^= zlPYl?T>ZXcV5AY4$;q*o7#%F*>saRrNATn5!10KLSAC&wZbC0^^DccFIYa7?LN4^eOkPU&925CYD#V?${Yq z>0fD?^QLmV18g)Y#^BzxfU_3`g@}t+oES4lw^?Zf4c(RXGjHTy&dlda&)+`>&XXUV zVa;C6R91gG9iM(+*Lm7m>RFT;0HEukY2i?n=?Hl%?>a=ts?PbE@w1-pxQ;j9>?7Ol zylZ02y}-j#9wfo6IYt_IT)hla3&jGGk}7MBmlGn&KmKz-vGAH^xjZIBi`pQOKOURR z_*qfZFMRZge89m7N*6j;u%12u1bSU5a;sYvYY^`Tjb=8n&&Oxt4_U;0TXx>P~GPqvrCC%z1H0FRAl14Z=2UYu-rOd zD#P2(^$-Xt3!M|L&P-{_!N|SfTmp7NvdL^zt+}0}1Q$3fGISFo^ZDLm7>(o@BL>|j95oL6I997`h9@vT#LnUnVvGsM9kpn zrTD(rGRksEx#p@9GSwbh=1UNJSQ2I;pa)eeko( zd;7K#o7&pHwz5E%kt8;MvI_I&j=5ibc!Xv6&~o&mm3eD{wvD3bTI0J7uJ;^!J++b} z3*<|l$Lh5|@~&3!#r#^&+ zs51H2Tq5`<$FMyL65T898YUUW0Lcl+U?`k1Nm zjp4WJCF{qIfn|^rWZbT~6kPL8YlufxyJLrpQJO3;4c*|?dnm@NX@$v=zv4Hp_deAI zLLo<5i?n4B2t7~5Ctv$0-?SItP;t_0JkAQ(x>ig3+Jvc8ZIzWv*f=b&!P>8wF|A&o z8pg48){@bc*wB&2cSd!*D?cHQH1x)pzZ$AX5;b!_8QTE?=@Gy8jZFvF^`69wtN81g z_ujs5|Dbg4@1&Ghrj~zGF)OOfMl-6Quj>EdPUzP?zHxiVA54rI(Rb_7FY3`t7K&__ z662vUk;9p@fQhoT&paY&qNk+G$wPIc;|_gZ?X`zX$C8C??x3kNdL%$O&r>fnIeeM0g`*nWld*4jwavr=SArm!%TCHVwql+;YqypR>U*Cx&RA1VF5^kSg6?Vg$ z*t?F%?hF2Gkx zZf9ZcP)sBKqd-Ym9&|>TB?@vWynDa(7!2&Lna01S_{v)!3u2*il^~?4yfpNX6$iCZ zJq9V9L-;{dyC92zmD+fO|0r-LLY2Qm=kN)DfEAD|4D$YI&?oB$YKd>h#TjCL3so%r zifGyoR}C7QEext}6(iILbfOyFi6ygH{6~T0@z}wCn$ITfl#Q+0nlGxeL@q3eUlK3a zJBB7#UJ&0dKm+-Q^oHy~i57ME=ri8WOZY)r5Zk3qy-WiBqrgT96abMT;&8LI1m2Yt zRQ-W>;_F$rG!si5LX4q&bM;jYO7(^C!bdgb>hOca<|3QJeVvKWJLo>t^q=p~X|w>8 zo&)J5G~;={1D{02<9_!&A0%eTR^jIdL=zQY8L^u1(ZuZzgaTzOK#UB#X6lFW9|e9- zlXLr%9xFr6DU{mYF8vqP4MKBPe1-746U55C0ig0YQg(zNd0~m$?d=4#f@bi8B-?-) z-WlzEmLOgGK}L3o**F3JTZ+MLcgIkbV;}Io3BHth+bdW6{?hZ5!Ot&Uve*fkA7GUI zKO>@D8;Q+OY?TQdPRXrU6|n(^v5xGdxsm2de>x6ZD3nxBz4pEE2ZWb%fl|GF;fTho z3T(t|>Nb#o_HQqRuj3a}#REHg_Tmd7i%sB8&K)EBhAi`|fGotu4w?W(oJKTWN2b#< zGVEsZ^F0=U{`M)SFedRQZ0LQs1&7ESh*d34sH&wh zf_v_Pbn;|lLO^Y}FgNQbh_zjVa$CpLG~iU+JJyu=5MK3qC{=wk{XD;P3CIH{AUOUO z-oNIwZBMLZMsy~q;J{17%qs9Q9D;y*k((rzY>+)};;PZQxsyYauf;IOTrE{7QT>RO zwRO%!S6&yeahvA>*Lmu-uaMXBgRul^?k5n_j;LXy^WJ%4;_VTL#8pW#p)tMESUubt zgw5W(e*Hz0>^f96*pkDOb~5ywgbmmVm>4U+Ty3)jV9o$e zlPm7?<2OkcfxFDG3(8ZErHu#LevF)f9qxcK(qU~MK7)mbPj25UsP^Xb-R=cj_H|53 z87{-POhhSyRq|E?TZVM1?E+*$3}ZYa_8K~=#gE={-)$tJ4Z>IH=18C{!=vtQgb{7dnT7SK={m~jA{6&i(qi0AtzQE`hSUwGL=7NS5$jQuJ>HsR1 zaTqfP%JRAZR5T7}M<>%9K`oH~E^Xfw*k_&VgM&BRfm0$aG4W7$#%(VsO`Asy4plED zfh$~K9~~W_fpy9AMa|?)h;2X_kTyHUNI^Uk1E@6peth@ODMc>1IY=c9-!V)(Xar>~ zESgZv$PVn4@an&X1YkjED`LE-4#bkVudgl*ez?k^nKu;aA&2d3;+<+tQ0#)b4BO)k zDK}elqk*{C0+Z95GrFAk%*dfE5EyAZU;QZESa)9mR-#oC^42(VyUtzTdb-1CV+e0B z@VDI=^^o@<#+l#l2KEwMNJZ}Hr8|~eoA|WfWCHhRL1?2MR~mSfdR((m>v<_rdUdd? zd5Tvs{`i4T4^>>WBP6AP#Hq9v7%f8iZ|D85@8W{O75DtJ31fvR4^In>5FW@uvyQVL{1GBdWjUREgN(JBh5H40NAG%A#TU! zE!+tamL`Hm#a}1(CkIwdOFx4*&EWVr@o7TE2h?icfjR9rf{VZ=cAzclryM3-Ib{#X zO0PBQN+$6#B(|`$vM>^IrD{8m%O~xp?eSY@yUpKXnSkJ&eYHWXY;`qv0+W zaMfD?#Ze%iRQwtQ^|L>HI=fd@5NgRem`it|Zth1^;qv&XvJ#m{b>ZF0&dMZ$vjJ|g zZ`V-AeVtwpRnO7aNqRZ`e(dyLTUz?rIiK5Xb>Js&h163nWG@rbvI6Va%@fTKYot#kPQD#`PM4z9w%@UJ8t%zOI(5zP9mcdD=wkQWv21;# zJuynUW!qN>V*dzW|8GvUb9)M71#tB?&%@>G0;;DsadBMZtLcVC*xHFfTsP z-6h`kaQ4zu7EK2i)PzhWABZIxB+58X_1HiuC$?ExqLv-%wuh&?-S3QQ9~}RD;9ZFr z9X$teEfJvIZGKI25uAbw;Bz<_n7ylB8q27eTFD}6n*+_R^fQb~P%8s!j#v9=EP#sg z3aU3}iM?R&`e7}^YJ@`et_>fLo+z`0D^rIXGq^<3#3@5sJ{EyhAjqdQE{*j7(B{c&uk z(DZ>PAheRQ9>fX+8wgb9efbbTxNr1|V6&a7yCAs|BABe50oUc4KG4-b>lhs5CWfm# z&CAFPwia~e#1|TudA-VFfv$XcKv3`%nxpOF(Bu$vsDO&E8{qb478Z)z}cOv9*^gidEYuwG4-{tC^=Hx1MMgCAD1l+s`vmp7R}(I%UhD`alza0|mWDrdEjj z)y`PkXn%5QbV6vf?ewzA0wmN)zek3HJ8_rf>N(*mAI+SY+hKx@c95;b05U;0|ICAm zF$nm6UK>`Fad=_M|Bv3)liSEXP^ya39jI_64wpsAscF{O$4XrJf2zo7OhO$&ix{XF;z&!)#ULr6hA`} zS9sd*yf#6uJh@=VXTgVKesIPzwLn1SHSijYl|PZP^c)S<*85WOZL6R?L=8qy1_qkV++EX%=qI0h~XaN+Ms)P7jc>+c7JL2?vf=`Jrhzih+*3LL0gkDNYc%Y$@AA(dEUZ>hikP2v4(7MKB=75$0|9hALR;J9z9S%2AT>f@p!GW#k zMF?BCd;e`!DX!@=)I^DOCA-Hp(9yRWhLSZTPmUPduar;NF)t91Nky@zN`zfWqU7{< zN;&mq2MFR$*f6#u=dwug<2WWn zXSr|hJ1%}&p6Behirp$sz%Q8T4zCS`WiC z>}zC~PS}FcJ2|4_?RF|+8L65RLd$a*-X%cBQ5|{1P-f|i3j2j|= z+LukX=Jolr6awSt!PZ{TR2mBL3HM7&)t^cZ1wYh1TgaWPqk2@1%k4puyJQB6ep0E(l!|9ogXj#wlX z)~39@_1xe!P;o)6-%!&^DWETkaqh`m1S;}z;NQGwF=n8Yyx!SfI*8s22^TiYvJMXV10e&Vbmo6!^z35xZ%`2(({JuNp`Qc2(z^^{CesCOozUNg5 zqR+kyMBgOtsVPD9yWmpil#q8lT}O)02Dq0T3LQFm)|<~Khy(e z3BuPn)VqO%w4AB>XNA|GaLi5U72JOM=Te%Tr`4*H(`rGzXB}!7`CIa*Tjo!!X>?V*P$2p zAN$BgA^rAJ=_}!oCwIdkdoevdO*rHsq~Frdu|&k7>xP=6_3dIcUaHc0mDqek)9TYK zOYh<*i=o=9J;koJU>v%jj<7he5N_oG&Za0Wx3j7QQuZ$BNf)Ee+x7a8b_N{l7U7qe zU&yN2Yji8__lwMA7TH(2Hepw^h4vI@q+yzF+dj&R!6cZOgCn~|2~W-%=>INb|9-;N zZIFPgTf+D2zUd;ER#y4*vv8h+%BT8#Ek-gEPvxvuDjEF>wixoHzF)IE{aw&|Nok?0 zcfCytI!5&!HLLZP9*(`+y-VrnPVx3#`}Z2y`?!Pj58tSZ9HH1bVO2ZIvHFc-5+Im# z-n&u{C12w_b{u*Y{y%3dDtEocJ_?Y2aSJHN^4p6CPsV*ax}aCN93ZnAxAIA} zv=3U$Rp)TZuK*ugt0!PjXw9s=`wwt@ajHH~*>VZAnr*o+9bV%Oi-4xIeI{6N7q5_= zg5gQveu#Y=?pEg*{UgVz60RI~5*_P{#vLP|ttyax+2_EzF6MZxE5mCpbrM1nPE&5} zMdgVyHS3VGqBOf|B{B!jJ#$@VSQm6!n3-gox)lJqmbDs-?`_i0lM~OB&O}Y^$Pa(< z!a*;%AE36h&nR@S|D&+2&XL3?1&lzQOCi&QCw2{{I|(go?t$gs{^ z{r@ONhBaOWd|&w?@fn2BDHTzIYGS%sX6Ew*C+sWGJ= z%2cb@HkSvcdts&2!y0F}uNljX2hK8n1>5FM;vXHJ~&}5cMczU3&Ur;uf zFwXO1$kNKvbLXxRAph?rNAOBigXnVZSeidADChuq_-|gx7vM*nmqY zX9HlkEnr*H4@KnldKDF+-m1_CemqR~M!4R0PF2c_58&Z?`FXb>VQeVpPbpFh>?b4a zIUjnscqcoL^QkR^{>qX-4`Mql$|LQ4{qNL z1TSpzvQID@Ub@~rr{dIn!gQ+fk6A1Z{!$~;t~-uE(vHbr^$+U{YCJjwH0y;Ss%Zm~ zV1~pLAdoTO_^Z?4MW58la0}H3CZZ3Ss9SFlus+vC0fFTturkt_X&`#&^&3U_R9gVCIYDZM z?eomjnAPiGGgTHONe&$8TY`|D&9`FOeifN;6#9VpLFF3kD&3PgwaVA8Ur+xCL=R@S zqV%tS8hX}ow|+dHjGFRDS}EIDk;&ek2ZEL=0X$~7CMCg%b?7EU8;M@e?juQ&$FG2^ zf;Yk$DkzlgNqp|W-KQq|AVxe)AV5r^u(!v%#Go5kz_Q4R3+a6J>9{;rx+0mW{zbtHGS8vJT+H}pt?ze+Skc@&=IEWD@On3Oms)5E* zxZlT4c!@}3NaV39cRlgaM?$VA2mUVO$Ckn~VLMY!Mz{fp|0W{olK>{~1WBcQucd0^ z$w>%Uy8uDCztXMY>z&N4D3#ZI(cI>V_-L}dNY2*zj(V@Q^NVfcBj^#zX;a@XF`1vx z{KkfVjVrFB2CN8)JUCPu%2vI>esEP3c+M!*wEa4GV;;J~*>9U?cOUPQp7x=8eo~*3 z;V}5;` z?GrHR6x9=Uhqii@GjWL*j?wcdfu(m;Y?XL1IU%lLKtWWL2j%8~870avE%O z?oBy4vy>#WX0iBFu{LD>Qm|@T_}5*jb3f0dou6Q-{ay;Z;yCClG*Nfz$Bqq#6?&dC zFfDokR^p38f8e$%+m>=Jq5#*zOVPA?-16G?>g}ynsW$|aPPHB9&{NU7k?>f=L2vK! zeHD+Zx88W`%FoexFP>iLqx1d>PyG|>kGA|{aVOt^Gwba?rne%G4I^y~=IC9jbLsxX zV#n&25p^(b8GN+VBB%)}|E<+~C#^IFTa81QCvLCj3Q^wAN;44nNzae~H_jRhi$ z(|S7B)6Pxv$YV{X5nFx{azKF;^P3mIRRgRvVxTmrk-5o29^01X1vN!eVTR7S3rI7_ zcIRR;J>!;l$1I*XHM0zEflG#Kt<}+G`-p>@5FP1JmR}St{YjGtiitLV2p|C{(DmRT zbQS$Qo8|LhC&UaWrxRTLw8OANqu;HpzgL4mKGWmi^_WXFo`_y$w zD!_0Ldn4!K3soBq*_Pw%P?_h%5a^7!iSwY_y_j(_wEy{m1M_{nXFfNt;4ZDW`MZ)c zQ$vzNVq2^eec$FsvB_ii{RzO?p3L9wEW()p|({g@X?(saFbA&s@f5x3^%VrY=F~%#M?eZ&7 zGu4HvC~Pe)c^S&F^Eu&2uTQ%7uJqhiO|w53Ssx=KW00z6Wf4&T#T`+r#Z#VB)LU({ z&8U6iK}|l~_2G|MCQa&V8MAHW&uzE^w}J1$Ys~$l2Y$CYi3@szdD#wWS$o~b(2lku z3ZV6qP+ez_LPogr+)@2|z+(vtkPc1`Z-kdU9=3F%9P{RoY;`@nGusv*WKo-z({y3d zRM0&f1;o3j;B}k6_5igXZ?0s*3Sh$z1fD8qgRSWkZ+p=QeQd_<)5goz=Q5Nz&KW@U z=7(#J-c?7E@}bd1FSzAbO}bx@dbMil6rds}l)SfSn*)MY%eoHv*kkasay9%+Q?g{8aF&UcSVVYlu^|kGY5QAn7 zk>pnV3tR_&F5TWjYY%;#HS(qBIo78s9GPuT>g)fsD%e%~Df*<9HczIN7`=Y<^>YSsCW0^Jys;Y=R$n1K0)O9`fAPnx4Pte zAZ6iY@Du2Esr*Lv7Ju7n3&h!zHNy2y%U2S@W>5WiBwRiq^CHwiz1bABlc9j&uWfi6 zGZ`L1%-W;-qy%l{^8idZ%^e}%^v-_+iEx0M;=V|FR%VOdan#i1=B^AO`hDCWA~%~3 z3*cuW3!E65E*TfU?8M7R>C+olf*d8w6#JJ>He__J^#1&T9i%PlhQ^cn+ zIRtK9;wh>funge(gP7s_bDRVxrU%6gZ5p56Buv*GR+y7Fehv(s21D2l9KXH;VK??s zfB}9}^)M}Qk9lDZ>PM8xP)tLW3YLB0*e*)KGV@>#?)tOWS@B!fU<9Hy!7rgsOG@eV z41~a?It*I<{{EkzAi?0q8gd``Sjg7*xqx>0!P9qW{y;HSSg`96Il<~5=7%}-?2%$3 zIM6xIU}^5vfg$kB;SPb1h|4-;Mi}NMWCGYJOuzqmUDRO?q2CW4L-YTC++I#0a(o=V z_P{Z4SqBe20Gd3)Vt|LLHI3)+E<)Z2M&he zJKZ60h$1N&>e5O_-}>rehY5b2T~Yl(N3`}Tgm@9#f9 zD?58#Gv~~laU93#lmPZmoJ?H9e=|pngh()|nFA_43|!n(jGvO#)e~6?G}=_2l;UMP zC+g=ecJJI{?`2*FJxpq!|F(A@+6P6(A0rJLlwF#Zg(~Lp$p6|SSl2ob|JLnj`q zV#L#$y`Ox42ZvEBc4+o#hU;%;#o6esIcC-bSH@TFmQCpGescp1uj2 zG2X;k=s`(_40${!*?yr=j~lo?Itm?0x8?i+{+}aG>*-%R02><%96V$zE&m>iU`v?Y zzuO-=p4g}PfF?r2X;?f(%%U)52K5l=n)`c#Gdz;19+bDF&KB#1_JXW}(EF%%zFjrhH6=T3%qg2fnK$Lz9z}dntHl#4GD~{@6eo)Gz;f9nBlSh9pg; z(EfWn{ddST-{t_>Gk5>gKyGoY50###)k4B5(HF;aqzJnlp$9MrdJUKb-kYI~Rc|c> z`AK=pB*XuPyn`$O;se1a5YNBy!X4jIg2*#Qd(9aGiS4zM;T7Y|sS>Yb`v*|Z_x`+_ zuev~iEoS4DEwh=%Lt@bW>{Z2s7kmG?^&mX^ps4eI!m|&}uIN(CqkljadVp45YwLnn zi?{fWW#|RX3JJ-o4!kWC`BI40aXvNPl&pXqx9@kG?@XctSpNd;7 z<2}x&6_d#su=L#5^u{eYjA|qOJFlQluQ-RO)Bl}S&W|6^KTn4&@;@meYUlP{G*qX=*CEae0H@{iCTAu1Q$H#Pqz_=VGY!6Mab*R5dxJuph~ z*?fnp=){=tOw>$2(zr-}7_XA){1rwe3m2GvwnB1mwT;(op?Y#Z=X?I-5Zm)MWQ&ES z-EE)|Q3DL@Vxn?)M$o!9=ox!?xpg5rh24LK%m$k9j2nKCefamuE8>K;(Z=>`?r#_z z(I`tdZ?#I!hF05GEgN7Rq}p!DT0o$&?|Y+t26nc7aetK~byJDQ&o=g4gj5h6D4Zsl zB*z^?XwJUKCz++tm@(<|L*CH1{=KN~AaNlIwCKz&f8EF-StR@6!EmpA4F(>#$=Xt> zvb2-MHkQS5eqkw3Jzu;+)K@6$47p_Lp~J(ZMo&(AW&4+5Dqo1F+xUq3!XO_jk+gyHleMZstsp6DyzH=g>q6Nb*Qdsc z^sEr_DqA<@`97Pc=gdzeQ?!G>XUU&tV2pZCAN296C2VJJr~O?0v!37G$oh`j?lEQH zf2Z;Ra1;ha(Ruz(-d+Zy@s-0v^{z&AE_{2dg(~OvxJTdh0;W%U zhaho}I%x)i;vh2usjB=ZhZ#@MI* zoC6EXj(Xo3!^kIzeMpg?dm?$^S88;kV#$=Eo7V|X@A11#(z?Lw>+P!QK@ZjG^=E( z3tR@|pQ*fc>E)C)K`#u^Y8QdurOT);*yoe6Q3Uo>URic zqk7K9rK5O8L|i>-S2003>yzu8-Q~^CQ(gwt?(=itmXpQQeKUd=>eyYOVW&S+hyRg<*&Lyu*f z)uGIz@vnqTm@}FJs*l@YFo78xT&enr(XAPpQ)!pEG|}g`29pMa_VSY6;Aj5hxd-4& zJrSdyno66&Vf&fllX_)?w}(1SGnqu=eo zz?kBNW@or{5`Ab@&3-XMFVuGZPc^N=zSGu|(_%-RkZb$dtIUXw*`9BO<=?ydhgcPS z^fvdlQ_pJcv!l-vUs2@hmU}!;ZLOI(v!50G=av{^MSzPKZ5x7`hySxvA8_@ zgW{VBpV-}YeXwk^++~0IPZCbJ9I=Y^o>GV_$2SPfmhgqk{@rZ+Z79@iDai91c+B8y z3XVY-`CfH|Bsnp?8AwE5C3(L+|TL$7}CoX|)Jv$S=Wk z%c+y3WWBG^U(-+E>NaxEp=IWW#ezNyID=bS)`)|8*ED4+p_WB6F^Sp%Ko`J zj}a$%1PdkN&NoB$J^S4cF-e>JOl2WU6@_BF-=ZCrL{o2X)Kg?UGkt8;Wjvy9&|M6J zx0X+RMXzV@UNV02IPL2|P15(wxe>gkwacTDby_~Mp>cQ(reSQE3OXt5Vry>aKfdA{Z*%Ju_9yukY;(Q-ld$&Igev zK`c#o)Mq%W8%a*;<(i9o3sTFU&b*vS6uiA9yl8-L(DjCX5CqT}XlI`u&zz#ZG?~m1 zi;fpT9zh}7_YFppb4)VPdq#=vQn0NJsXqzPK3$WX8uflGwsBTzuvBjQ=TAhvS#*&k zL{NG?dZ#mGx3PVFK|y>s%YJXz8MjHbhP(4sDsQ3`#pbnaS$PE*@ z(K74FA1pljU87GsnJ+y|cV}ydH5P=*tbT|4T*Yq`7+suPd(UY|f#dOPd40LdIXhx6 zgkb1LCUj1#Lo{b1u!y++d&MA^GK-$br)D-JX_#eHN~^Sm%;@ZckhsHW%HNZ_C zG@SbH(q37aRQTA2rGB#z`K)J1CjLzvd<6oio~7wE+kQlSc4m6?hjjYgq9f-t_3h22 zR9d{#=!4(#4*QJ`^pQ`n{32|%{=aF zZN4&CJUb0-(&Mz66-~21QtAK3ZD0U8^QzE)`3J5-LGc@N^1AW2lg97wUW*6d2l4=X zMdy(;XdQ$6M2;e&WWo=ty`z-ouH0s5y`{*swCE}NN^JxHQ8${IM|>08aVJ*^t%A3; zmK$buEwCmE&5G*dBTL}r;ys{Kde?r7uI0U}H+FJ5^UP-SBi%z(r(*IUz{ng`m;c~eyXh>pQE`E=42s9}kpJR}50J*ixIN=0h zGmSk@d#qvRkcaDaQu@EHGBw7yJ-P;)b{}cAio(r1cUgA}IYoY*er-l%&$Ts7_n?>r z;kT{;#itKqmU=ZsqTSzr^gO#G58e$AJU`lf{@e2ki(tI$;In?Lc%Y*f4cm@)aL5>b zQec1LgqO*;iaAFwBbijnNO9qJEhST9SZC#E4*$f`c~pv7p9>ooKBZB1iBd(Bgm-?BO%D3&71O`#RkqmGlg%#s>=%g z7x&1=lD7^%n>SM92QP3rt;yec=WL_Q>nF?#7WTC4*H2F5I%)l1I|{3sq}0geHnEV} zyx8Uv;uL_zdCTkJo?sj+9UAY!f`=K4+bvG9O^J8!QX9?wA5g6!oqnq`*|27L-gv43 zVout@p#PxkE5IPWf=JGmA+eytX>$oGM6;gF9Hla<$7&G$A(4_xKyyi;7}hTT+c6_# z#3^cuj}u(wzZatyiKG2#P8PZ48C46>J9isIb;MDTkbjO2uMk+0Ap1))Q6JLKA`*^ zFsGpLIi4~aEDdhA@M8H#*JkmB5|1GO=3JSUiV>9{-l;6(e1+r5QG znXK|LUf5k&nB!$-QCgL`1i6x|R-C4H&B92x0gmq_a+#k+>~;4bUfe&k{t`)JnZs}E z=U2=ZqG^aTF-cb>6XXksUQ^@~&pP2FrW^NNt~U%X@SpKsA!!(L0#FShmXc`NA{dWESC|Hj*GWS7>RV8S1z^$xe#pK> zz%$y_42mL0JNhh4_9anXG2%H|8DGo?^pUpM)4hA{pqrYgujUNE-|;z4;k0u=pMsLB z0%A#u2YP z2}uW>zkb-ozM|n=uT}7TaF79=NS{!Z==GG!l-Jr7wW*mDyL->q0^JCEkA-emkh%ro zNPs1lpvWgRlyDw>9daa`29%!ilHL+#0vjX}p~x>MIPLi@DDSsigda&%m?dBoj4|3N z_N`(9%56WrLo09U#jCViGZ{(<9$7*ESAj)Gx1hSl;Ql3Y!CIs&yGO+zm!x zJVmk0=O2Xw(TtoOqROPZ=z}m^bSNk&H6a8sy%?kKzR<~UYo}_AIRpyRVGyB>NeZ8y z6;{vlaUxZE)R9+q{F@s{6sEAm>T)earxklb0WuG0kyLT1r@5p(^8f}Z08N*m*nY8)CyC@Ab zO+fPZf%w+hd4XYj5i?)%Cyd?Ej1_;+9yrud} zDM4gS$utam%>;=w8=c|zR4t7g1zrRJUT{7Ri+y93RfGMw1TJgu+NMxsEi}#u;fZDl zOXcjSxlzeKo_rmeQKcH%xU&+yHd+Oulm_pOd@7u_kDP~}&63ggfCJxZn?nCvIja~G zS$Q$_Km{AE>^6_j^`+E91&8gy_z5?Dd50w9!1%9-!^!M{ZyAj7)YY*kG6rD`8+-_( zieo$fBrFIFcYE=54(^7aZ`r2}W;?Vo4JY2XxDTwnmJ22f@DsPTypfbF(efl`wkG~h$a(qVt=Roz8UBXoZ`mcJD9@<>rfIx`IL1zKeFm)g#uGvMyDz%=QLJe z5O1W6sviOcc}=0WMkh>;xNJ(tesUlKK8KXFZ;#Ctb^H@Ba zOgxc0Qjx!Ml>1qmr7%-LD`2Hs(I2Hc-Cq>!3GU_+*_aJ6DowYq*CCtt;+#f((Z>YNZ3*{uv1ACo;?*mD?U*OKx2GEg zB6}q72a*&NIAARVV(jkurV`}5`<>(;`O2O!WjE#U{zF(2@an2*CmWv?a&GfVhfcgK zpY$G=IE`$bRD~c&IkreDl`|ECSB>G_2-twH5;*H3L~xpLH(0@-*Rg^x1BDsy5@2ns zkF-%4?^)8+Rab6zRap~$yBgUSHxFv%J292$wN80g0VSM+pp6RQolxHKr9ncQ_}~8d zTM7I>)!5f~B`pO4NzLkvX;=!MM~Uq;oqC#tDi$75PO~DdN>6SBh|=TW;p~#Cj)A{c&(@CL!>1k3RgPekdVhQFW8dy6WnY;)9bLKs% zF;RM_m?WN^uI-*zfw+iEqDB*AOfT-y0nYpVA@9Ih~c$UxF*#p z;v$FwlTNRMC}9vu0C$6kazHd~6CwL%bUoaiz<9`h7Rda-T;GQ52kj|3Y_nJVfOPuYgtGS;r(Z`JW4600Ecv#EX6f zZg}wggaybLzCQ_aEPf}%5};(78+Drg2`QTXlSl*$AvG6qO0Ig5E8-LssI&5GB>b6< z-xC|Ks?zv5c?-ff`cYX@w-EnC#7w+2e)XDShz?u@){HD87`~KX%8ls6moaUtpeqmR zx6tFy@~q?9&2IR5n8dm^gM;i}$lLQclWPev zC?GO!862~Kn(Cb2P9)0hflhP}Y@Qe3Dn)aO1h$BZ`IXr@PWaKi4qj4%1>EdqBMFGA z+fOs@#8A>eRF*T-$$%(}cZDZj9;9m4p`$tFXTjfl53)R7b%uuoe7e0Obm%A+U*TvL z7Cld#1a0OME-IgK&faE@D8Ww=5@BhN z0+5akXa7LX@~FdWrt&xDFh<8tNTB*a;66zlul2cT%yTu}Glb)s=3Z}=Qpl(N5zsXe zrdcOLVR`wQe`wBu-V~SUD}b!zAn)V&CZZd0?TLR=HNB%$gjQ-*-B`jH=aLMwyfl@I zROxFZg3UxgR4@&nif^bB_3R0*P97}8G~g(3-KH)%p~6${+RiU<`jYe-GZ7_S08GtHqfYm+;_e1jS zL5m^XI7esuQj{x_1)ZS#qowgIY#LfodSLa5D5br}b)sdY$gRUNIcnqsB5Mdh-7;yi zIuYtRwh6e1DldX@my?R@yyPbgN8H*?SU+C{qM6yf2PD4Oo-gRmpPjPy#8Z2)N~i}P z?Wmtus)WjmD5ActpML~>Arnk;8y(`tWBD@sQZa~s9{=~q1aJ)v*oTWh%vZznWwfif zIf(Nt#R;7R={}kf*NR3kr58Frn&ofe8yDS4DF^ukh)A^II+h;y_Yxo&tT(N&VCtUUtU0OE!gI>R=XA~Q&Ad>UjEpOI5N<7h^l81BNW*moD zknr=zQE)gw$(VS}@9;Jl7_tN51(M%Zz+)ECmS!6k?Ra|u@1AT?m9IDVh^l_hBa7Fser9Nh z-t{W{wtT+9=PLlaU!GU*{U{OwUVR(6Y|H)l1`m92AAAr492K2#l12(PcHe@486;DQ zFJFkPlLDPJvFN@Xlu(2X8J>s}Q7f93uP`I*M+lc_!e$6{t4&21VWiIqy7Sh-SlS1m zZkYilk}Qn8ky0^OST)1?F1s)Xeunj7NqE->UQ&AY%lPVh0UWLO0r)4i88iWy1G+0a zWB?sr4Ui83wiV0(OSgKj-IN*)%aT7{W$yDkDm=vGzQ*f*5=f~WClXALcJvXfKWlJ; zEO%IQNd!9O8@0FZG*U7$iQ+et^88J?|7rn<8Q18L4QP+Hy<7qi{djr(mXG&$f%u&B z;aY!^%bQ}XnNJ<&^rdF^eGOo!){;!YU&IjQNhz90l`d*!da5>jBwRFbI5`pv>mwnT znc%*7mR}@X#tScPp3PSB^dT}mBQy;L?2+bLpj#L~bqsLyiJwI|^3*iVX{*ncLg-Vw z%1A*lhdUkN5<{7bxY8m4(VGuMF(AnR+JlU@q1lrmJtA-gqUgyc%GtywpR>L&*g(0? zRLrnGgm8qxiChHRc;!QH-6dZH$Pxir5KS3K7344I4diQoWr%KsPFz75#=?z{01oIs z^DVeYAs!dqU%Z3#gBDy8GvDZ!qp2dt^_k=fy42-wb1w?uNEc31KnBF} zB0Fy9Q~{2me$m`1o3d%kp6V%M$t}duVT70FJ@~EdC2&vvt!~YgOyyAgn=HWneOH+K z{dBRFP%;8T&BlOKq1Vwpn)+zjA2>93kd~>0Jyglf{kATCRS6k6hZdzC@9NVYQZT)3 zVKsp7Qy&^1rk&EjBLb6n1UlS@JU;m_Y#zWM-Z9!ZJkb~m+J|fP7UAF@tD{E_Ka+%X zG@_SybGPaGI8OB}uHL?{O>XENZg6{%`8xXw;|^^_gxhvy1B?=;cH$o&XE?qZ#nVvo zytu|^;^>5}j{~Om?usoOo^ZlcKOn^?<3XsHE{NH+sB+3m96>v0j|npyL?T4LP#~S` z=beW7)AHA-RVAAbqP$e~@>A5|8h+rP9X_mrpLb(fkXp8U&k>^;I!5t~&HiS8xnYrk z*`N3n>XiJxggbX{>`zaUt;#nCTRMg&Ycz$;!6egqmlb-Jrf4h$d#xwsBWiSEa@d0h z0?b}~gIDyLKTTrb3E@iYZx>=uEZsenj<0?NoY2{7MW1?)amDh;T9T~L=Qk!RyS|Rg zdcox7pvL*%*Y%FC-Z;Bj|F|)0a(YC_QDItP>OA|W4kKg&b`Y)NBhM(16>=i`Fse6u z>m++?Z>y&9P&dR4?jyZ8oGR$jKYmP}UiZ9=6z%;5!DX6xB1T@6eg=IV$H%7J--AyI zGM5z^*I%5bxAGaL-*oLU3k;B?E@7HJ^S{e{1bukzEBQb{?_sJ%cjytr<-G_-T;1@a zpx<2)eQTzkZJqC>E@G}{7tOi9yxwhEO=vF;KVlG4IilR9oE&`hhcvZuWYIWAVg6{- zz4+oHytP*f8qD(N3g4e+- z4+-E17SvtzUnTW8Hl085Jz+{9hpD;T(A(ZrgU#*2rQYq}Ie4Gk6v804HoVw2Q$x*72nvAw> zze)WX*I+z{Fn913#2>=qUYEqD3u<*U{Sky#hrjJs37k6GtBWyRrM1=I`JsLAy;#Q0 z23{%U@g2Wl5Tty%s*Wjf1X^&Q>Vn+SqwFR)WjntXwzz`*73w}--CRW29@DV*x3{s@ zO&HGl2&8^n?6~S`1HkB3V(|!rpg_wm42YE{%ePQ#d`U-}MRTo8?Wjw*!KUZ@QSR(= zF?yCiG%n+bla8u+XX_w|?)(u$Bh}NutYRzI*rv?Ae0jb2&X1PvCRxR@uKS*yZG&0^ zIyE;NL>C#8A#n#J<>(=B2 zFz45}OudjHWglyq&kI&?sHY8rYBBas^UaU!;_>ZsjF`47fz{g zg^DP-ucsWO8AhFrEHJg>Q$v#Jqy}rbdAQP~@;qCibtJUl!U33#PHbnv(NCRWPHrxN z-ouikO#xV(8EsO-sZ8p205Em?SQePZ@hLq_W8zVDNnvDu!3D>lU@=<(ESFIL()a&< zH^GdSWV+nwxm<-fA|TtRmk?F9s?+Fuaa##hDZ)i4yWBFAC$L@xp-kxT@-79>&E8aUDk#+8-@cvU42V z{5;LWG+xXK)O+4QF&lFkhsi4F6;bHREzlHESZRDNr=VB6mnXdtn%0ol*hr4a%I{T& zRde&voqpB(rxwI%^BO+zT(b71T#}UVYghjOpsY27N7ZEN$)IB8N#;4>n%t)9s<(JFZU|&Z+g{+cC7=!L9XBFBW zU&y>`_X@^7*|qH-wC?r#R}bA7E@B7hH3*U%{W)ZsOjU_)^j|LBIg)d@Y`>xxeD=c= zJY}yxkNX8t1aX^t725c*9ZW1|i-D#lPphh-C3E<9cm;@4S#ES?TU~>(CRX?VG zP`=c+hY$=&vjP8qj4zQ>6la?+y1r)Vy#K`&e4GIQ)q#(hL;rDw?)cgR_UUu^rzU^P zptr11+E7Iu)&j+td&9_I^e1nUZY)k`K#UQAfkgLHzm;KfJu{)tRlK?Yi^ljk{5wdpjm|mr~2u zP!{(cCtR&aQj9>s2YmfIUQWJEDxp#c%ho$F`1n%D$4u}>1oVn(a{~LJn!uTE>j0S zOB4SQck2h)M(3>y|3in|8$P4D=3Yp zdjM-#ogpRoOsBHc4t)IzU*WILjwA>CyPQgVboT7y^^g^l7H;+m=I<@pOoJG`4BUGG9pO~q!uzJ`n~ROGlJ2Dv}srH zZbw(z2BTM_?}zmigi68IlYWW{VT1J8T{Hm+{joaY@F@_4l%F46>^$RvqMRoN5%Qco zNzi48mUdW%r#r6v%S`9UJJh~cq38B&5fPGu^l|7Lo(P_-{vhjzCg!&H%{^%|DswC4 zk4vT|5_ydbYt|wuX%Q<&C#fG(=jM=WqtJ&o4r9nK&31}OV$C&-q2aT2`5l{k{@MY$ z6;FWpQ+E;Yr?vE-B{a$WPx4vS(9eO34UGFFpLf5k@1V-K%|x5}&12SE1Dc~(tg_h^ zO0aPFtf+(CB_HtSWG&>qYuN;^3_%`VF29@f7Y491k&ukH*D$`>$l)3j4|`XB|1A9s zOq2-XQ$mxEKk*oF#@c8BJ(Ocy}e=Ye!9=b>v^!z>pR3Vn-d3SMr*_|B&%b-k5#ze#3t0Ey8gt%s} zlk7lMsveL?T3E`usdhoXEzjo;0DwL?YYS9PC)}wB2QORe zoKV2qBAzf+^D`=4S_nU7RJH*YUC;SN(^7+jOoloLm-b2Us_xZKj_%yx607u~qXxr81nQ!I;iuSueUm$Nr{}VPs%Y|^N zJX($-#C&!g$U&^@p4Lrv6lys>9o%Np-N!h+xn8*GmAwH>z3us%>r+V^KvgBJ2EDc< zyPS6sdJ#5RUsW+OTw#`a+ujG!&@ucupmet}e_ZsQHsA+PoT%rTP;W!X?7ScJm_Njy z)7QX8Ocd9PYWf)V(9094mnq6{F8QAQx`BwKs*&t3K9CS|ftb4K9{Wo4{mb@CtbXf2<6@x}p(Hd5S@a{AMHSk+dwP@&nM(mYF&oek zFzC9}iW~8J<&MfeTfyuh#d0Y$eDK_G%yH>mFCOV-E=WVsj?ba{6j)A2~{soga~gUm4eWv7l^%l%zIvXE3~&BG~L$k)5F zs0Mx9aB5C*$h?k}PTR<@hf>_Pa(4lhFxvy@MYMWf=Ib6ao+2cJ5VXz1lt8#Qyb6df ziANiKe~HD+X)a;U3|J2L3=eF<;~4WD#2KjpGE_TF(zn3z%x&zB>Tpv9GG<)#LymzQA! z2)=aqbb@eB_>SK7L7e0Xq9qEohs(5hn%UfJ(d>Bi^tZlC`%!oVym?6M+_131KDKqg zjr55b>eB{5m<-!WuBdupL*DT6jrl+|(EEFa?HezEdxB77$<>dNauQ071F-9>%Ha|!KZLQ|7fnQ z_@wWku|l7!aO0Y6SX@ytVO_lcH7pFskL1;5P+wvpnV+f!JeGe%(Hy#!=F&z(@a!U{ z8+uiMdnjN(iB-k?exEHm^?*K0m*e%V#7Z`2uOs02MOR)&&gpnZ*hirrw#zW2Y@DkN$}p(6fq!bh4JC zXv#5>t4Jf3Ju#~k&}swVuacHIdzSz?4kO~2b%UZFamblUM*6Ool@qz_h4(=9%1R4Ex;)jIArm2M@MA-q)d3+G zMHD#yaAF3E0|*Xqz~0J0Xw0-Nr;Alg|3 z4f8Qtdb(DBju2=GH7h}4{qX#FhDT!nArSqRwRR=y3&{)uhkHgRN1+@jBb;!B3OQ>r z3vi}Tu7_1G%*$ulKo67GYXD=HMxfY}u~3>0cOk_j{r%Jij7rCpJyEU+@L>Jnmn5K~ z?RjM(>^LeM4sftx8LyFZhAY+u91T;ZbshxHYGW6P636H@DYO4f|7 zTuknKBv^zQjyp62M4g;dYW&t$4JhNDt|`dh%q>h1+LDr!QMKgjZ?Ek8Qx8p*QQx)P z!RhCn+^J6Ecv^b4aC5bAmq#4uW4QW5RSM02S&na!osA-7dwA+ z@l1?6Yvx==sjW9Q}Q4Ll2(5}j!TK}<0=Qr2p)6w3+JI%sN(DQ zTD*RHQea80R{bONjfRT@z=wJ7q|1@;$vElTp*P%QIpxR6G!x&SKVw`Oe6G0L zlB4xk`bY}OLcWS~Ntbcf(LEQv_o}4klIaCzvGDaJoKeZR$G-Z5DP2<7&v%_B!{HcY z=Gotao^2Tl+)8X2V2JT#PmQ@>H1sg$^9mZozJyhbCig!b=QC_N>tJoo9@hNBbRDDZ zU7@-Jbf^i7Jg#Qfg?Kz)3$m~eij>Q;HlDqE543#hff%5mc9KlF{b*mx`}s{E2SM(q zP*cgBU=oN*8aH-KeB6E&$}xa8$v9e|CTKia*CDM?|7ac5DAzxROU9EGDtzba(gmtt ze1dmR<;cD_dcY8r+~IFcafY2dS0@0y+tU!5hQ8u80 zc)X{6VrPsi4M#TDlM}ADn1rso?g*Mcs=4M^+SRVcmcLz8oy;u4B>oBmDdMO7TSKK2 zN&G<6w@qUp5ii|Rt(^U#UQ`0a$J-TqgD;ju#}I6*^NMP$okm5X#PJgl^EqqUUFgU5 zs7hGQrhuNhqqE2<0d^Ast#5hjuCJy>5?j(QB+J(-STpRLn}YLA>^3yfblorqf9!|Z zVwpO94gsG`D?Mee>xy~2V76ABGRZsw_)B9DQrN9d67+I~zHOQ1qsPFd*aLQ#OAWRI z`Wux5aHgp2S1}}i3th7ArDy{dZS95~I@PcebTUWuLf zG8?*OT6mtTVXGOf4Er!)HK94_O+pS;(y5SByim4fNUd<69{eQakfY)AI(nM=*uQGt zecP_nVB*QR#kNXUewa$qb>d`0rxy^Jdaa$IS&sDSl>t%V#k*L&V?EA%#RYS;SXBF# zPs=3>=|Zt44v>1aje5wj$q(#Bs$u%EkO?tAFJ#EHB`r?>DpFEFTo&f2x6y7g-H&BU0*qF_ro4YUraa@aZx*&r?2_xJ?E;KKygAq%IEv0}d?rQ2%?yrSlJ z25#J$e4AGx%uJ!=YOgXtgH17I?V2CRWsCte7C-Yv)M911`{YGLcF?%oKhm4kPRQg^ zi5Pr@!=)ya@bboXIwD+&^~$yK1fXqT!-zYV;YesQ^fPhfiM0v|q&Sg5e}9v~W!7x% z;|L{Biv#`JSL;->5ra7s%>Md!gdRDCe%oW7)&C&)NPr^f;{)B0klA)vgDdxXZjXumgP{<@rHM+i;V zJh`gTb;v&G5(THUbua4reoA_G$`X!PD2?gp&dR_@eqn3w2you=L+e=#9vP2io>(S_ zUe$$HjrebS(N%sGpPp!ty>d92>G}O?p*e{`H7>_%_pTVc&47P0d{+0cR!SFE+7b23NOBS*t~ZK7)H{U!;j%TBS<)kJk8R~Zh< z0pXQ0g+$;<%l775O>*w}P86JoY@}g`gDpQ$Tboec`%?T&wu7MwA&8lAvNHF_yZR+g zwfKeXJu-SbVK~>_4KDG2Hq{eJ6@w*9Wt=2m9r0v2Eb76~Eq$uj@0$3|?(B>`2Dd>@ zaT2|cw+}mR9yMzVa$L?X269dJ{KD`|o95txo+aOzu+a*uI#t_yp^{>kp)G^{dG6(E zTVJqxq$@9Ff@pfjD&O9!gt_+)CaKv|LV=u=HQnovfn{%70(V3q2&Iy5B(8}6+OI#Qf5usEi>M)~z14k^6%u%tcm$9Y%b34V(R;jP!C{Jq6BsO40x z6{s9TR319?jbGBNy?30OXu|7IoJfNcYH>{F&R@yh=V;G()i}FRcf>EV^#O$=jvC_T zXW^yZv2;kjE2W6jC`^`uIZ^lE4>dvj>N>(-CW@=7%a%QweCgFE=R&7_CQ{jvUAQYs zjDBcB-Y1L(+kqlur;Iw&qL@ph_+YkGNl$(_kAu6KzP>(i%3%%_m;qN%DqH>9_h%V& zm3qs}m`z2!d6V#2RYGg6L-ym|m-DK5wrQy0?t3QA5mhb)jUT3Q2+*D6aT@w#tO3(QXDNaP^GgcO^HF(wk;RkU+RH2YG1OyR{ke?T4fn4ZVATd-(e}fI8qi3->XJs z8%VjErs=?~;EirlC21(;XuHa4{mS|aBzE#_P|rd<**f_kX+}qm$NQ`)b<{XW?5@H5hfLsswkZrYdk7U>g^m=^8qU5Eay+ppF19& zlO{0oNqU>gz}@5;D8t~RAH)?Z3-KcN%sU_Fjo-Nhn&KEuiBFji@%V=I=P%ERor%G1B}r1Q z<1AJ$hET~*W=$D6d$gUw#bPRgktI_DKqIW-b^KR;=Qkf$O}=HGE6j0?wuJVJy+xJe znCLr85h1YlQW7yAD`4e#XKt{my+^6U&y~eFUAYeqL!P?lpVVroTZBQ9+z5Sqi^B3p z<8PJCXRHEZSwmB5*-y&g7;*Kn=@hrR7On)Jjf z2gWCUYt~v`-;JkVk59HlJzO+SZajkwDH1rY zAoJGp%C3W`w^E1d-Hf4@If9q3(!`l|mWO3clW<@RK6WZl$UFpl`p4ZY4JUSY89A?e zGM_HEO$z2FY7r_(CZtvU?#;H|p6PXFPzDUAkNg#S1b%u3W8yO$O)O08F0W-ANO2rS zmn`b8z?SNKmlsFNoQNKa_W7bKFaPSIdzQu;EL`PX_fL~sjMn3Uswt0{jbk1vo6#?h zU7i`{)u2|=a(@w9R^;%8f9F$LmdhVhd2iWg@cntZ=5m=WkfR^9#f<|`*MT1#c98V0e(qHAi2N>Yl)(26#;($0I4EEhTaQcjFx8G?4Yus23h zf?v0)D^gW?elR)7l@~o-p7}ym_q{4ghE>Q!e@6SbWD`QJ|2WwVY=jPe$&3aELa3Y$ z4RaIaxGbrP<_YFRf0&p)*PSQA_qXlDm1T2cg`X)g*;xI~AkOE0#>s|AwGmv3mY>#dYB4YK5Q&Xv(Wv_!V?{oMP+JyE41twVR#2G%R!|x_8eUVyL<@|fcWpO{4FEzn+Cyi{j|^=&46sRFO1LwrwXuViMuqipf~HPng`6eR zGI{zjdC{h?b;Cls`>)n&)Li3fWS#fo+vYwAjDk-?NiYY&^>rOaHro|OZi1JtNflNZ zMEL!NF1klt0w-AR!|=T|UIjTUw$w&Nor*yt^Y|lxMi`-GbH5ayDOc)Lkj@RO4)fAE z5w!v=ji3tE_d6A%@}r6tOVKH(^op{M8ycDD;|||0qEM=(8lMOp<7j_m;fLtQ}YOfFfuTD9Y!JL!B%|GyE8~CrxEq5Oc=H5UPk>28I zI)a#xPz|n-mo^`5S;U_%*;`06$mBy($J>xd$SvMP+a3`XoTE=|jc)$_keM`FYh(y#%V>0mUnVG5}#6dKK& ze(mwL%ZJx`GU$c{J3qg-0^NAaJP6z>A4U^1_+qs+L?-8?OJF*J%4S zRb-g=g!&NAHG2#w3^ZN69X%)vG<-qS#4d`s3Hi6N>UIc zMi)TP ztNQ#nRYwmz&t7}2x#k>m%rRU?p23L8kG7UOS9 z&c@Mcrh~MqT1!rDmRy@xwq|8A-CJGLJCQ8m{f;;bmZQU@62gJIQYzcIjAKb!W~Bs2 zo3^1`uk1#DtfE+Z*LFuL<0#Yfr}#Hpf`5mPFHKPCR1PVT?altdOp&W3cK;x?cTzuY z+k8revKSawbS=KikcI1htG!mWSSK@Dq~3U=de_g!&oWiHS6dr%B;9;H{u_syI5|~t zd)(Hauf~G$HPyPv)gM-fPH;39Vk7)6uqlwLoSJp5wn4zsLi%RG$!>*nlkKo?LukZb z!>TH)!T%&dukA2U=AME;kI*D&{{I%lL>emw`p{G)VfP9JPd8tQ^R&^}HElqk& z#%g2}CEqLN73h3;djkh`8e&a|<=*6sgK|6ST3N3R$a)TeMnLENH@d${oK;3jOra*$ zcL&qxv3sQ2OyrH#Z%MVAesvsoAJG-BHhN*LbVO&mO1xU!A;b`_y6&GP$5vpvzHW$e zKEz5VIXhxK<+EC=w>v2>Z2W4*Q0y_-BC%?;+N%<(v`SFm@1dwh$yLQCz1yZ1RURlk zW6G5xLtYtov!FVr6I>r|R4WXN9O(P5G^_UvrgbV!BD?iP(#IPjpkXCnIa43vxcwFL z$PFz$*C~4FuKGhMB|V8G#uSzeJq62$EyfCWT3f1c+m@YadQ}IgbdI*-hab1KeAh4z zma(;!(^NQ0RJAjf{fr2OsVW! z4@pl6SjopXiNt3=DperKtuO6QoGL0#*I{L+$ugHK@;QLy$wQtii`AZ0jb^rK5AyS* z>=In5#8<*ts*G}!2ix(Ufw1J5x!jk=eQk>|R+5dys||lw);fE1S>@D@$tlWyYBX~o zw}*^u)u(ZDL^=GM`rv>7QM^Ty`B!@C&pmnK)vX?Z4ng{>-;cgH57@2{ISljKCE|7) zP0~~k@XAl7#C~8)z3tck9T`wrfr$z^R{{9!9p?!Xe5&mvD46GUNWV*APZ(z1l%#s) z>!50lu)=MPdHkZmV>_05Gs;tc<_*8Uzsm>0#(a{Ccm=cc3Bs;cWHi<4 znl<)mX}6#J$M^&r9Y{d;1RC| z-Cx;>u&r&yCor{k9!rZil6a5}%?zBxj)X92GE>YeJqylaoyyduly}gbjnAD|&>stL zK{x@l9!deGH>9KOo7y_Rm97@tDERbK+oJlP%}_`L$)+zSq~YUwppfmT*-m#WIw;%Ai{yQGC$F>WjNZC*?T z7c(7>ulrwq%DFQA@P6YoN!ry@>$sY<+9b7l6}N%tFH;~Vr(1db0|D`RXN5&$kKpot+&+q{%!_tw@=CZ z{-BrCyWN3fTmuWp3`?CySSTKCHCmBx9!fe zk**gcIW6N|YQ8J=c5jR#xI-!nck0pfWy?hnu${>3D>0R> zp1HRvV;zjMcACt$hxb7-y>n}@6=->c3iQHCSv8hfAz!(rqhE)*hKLP^*fdnzL@PY0 zpLg1x5*kv}BWbv1FQB&tsd*&k=8>`c;dC3)LL=iaCp zPpefUx8rK6-i;3<8*W@yQZs#Qo$jmJ0CMEUFc|IcwTm&PJQa`mfN0W2O-&TDYlmyRC;9_Cou5Q{eIc}(i|8mBORY)w zmFf3^{nmR~B6XC*5f-tFP)&N7l;F0=(G9*Qhrk1#0LGuykEmbHi3== zmw9Mkf{^I-lck$?^||yzV=cJ!X-nrBt4`z7aunvGUluzZe$Nc~F)@;e+PA@@5$xTG z<*@m+ufS<_fKfT(h;v21G+J)7(z)=YFZ7#csrShoqo~}8u8)9CR;kb^h){oz1%rfT z(ueY5H9H`dzIREg;N@2pBafQuV$|k1sFHtqYgGBm>=Z61KZP_aIigLT*+mp_fwN(% zO0bT;b5{>s_@_w;Ze68YY9cXxAPax*%C|GT0Ketn-Joyj^Y-?~ebGOxhHC_B34T}e zrWR}E+O}Glb(HqriJ-*3{4vhC7vX61$M^E!=+0AdDdr7JTq9(-kyGNtb-MN!gzaos z5V$qQBFkD>xOZlRhmrU6aQIY4>MzU-l>Jzz=w#L`JVr(8xUE@Ix|LUjR%q#lO=xQI zg$qv3hLkc%M_WPk>V)V;RPgD$cAPa}3`jzwdE{U-=FItI85i}+b;37uw`!d5_MgJ* zI!5BowTJjNI>mh`ecnd3SEh?YZSpSd;kZVo@k&lR5}MteRa%9-INLKN)zAlP`chyQ@MMU1 zWVDMx;6OpXV1u9Ip^lEJv=URfZgWBK+AIDznln3HR_tugVv&`SXQ6HKye?5wFDEv< z6dz?)pgXckt@XC0DRT(Kd540WP(}~$F(Z+C-$HDj zrGsp60%q}9QRgF4Co6ZG%7j>6B|1*7l>`)V+$=8cK-}qf0**Gl=%AtN*7YkYL;bF- zz2K_F@4yVqcN6(hrClpc6P7_<{j15{` zq?Sjqj9yxGo2wtI9$4xbcvzp;V;Y4o25N>Iui7%p;1>i}VKc45exrChKEc1~o-~aQ zmGaMjb1B4)c2T*4gf)g{vtB?Rcr%-7rae0ITD=z6w-N=LC;)P;QAMmYuQj#ARE*DV){~9(Ni=yNiu=!l|MC!+qa|sB(hxU(uA@WcAbEr}=|IN~7 zW1z+rbE<{ZL!neeJx^%D&2}a?ELlKpGk18)2{cjtDK{e?+jT!)vi{im%9(&?ItKqm zY_g;~VU_vsMbs9}D-!3=RX>xrN)kUe$`KaKOS_^9t+e;HN^uRD44X8VUp$*_xT~FJzc`*YDENs0n_?xKX&4Z9&Z>Fk3&Q2B)8XyrrA0WqCBe zn6=^2RHAVEc}{kQik*364a(AH_wo4fc)anYIk-YI?cQ{KvJz(f+sKjO8AQ>jFHBBc zaMRzSM$WyASIrd+v_x;!XtfmrzPP!nbl+oi)a!HUhq=bxVh5@9c#pvc8|Fbfi4@z- zXZO87t}MJ1dL&R1oH&)Hn>zUFICh=Zee)&v#`g*8Q#0Czv0VSxdu{o@WlYL` zrd7>X6rB#XUg@o<{yn|#`Q&U!L*-h#j3=exsTSqx>|5O_)BXV;H`{^Vrro7&**5J_ zAybX!yCmE#6I=K0X6P-C5ZpmTvFW;Oe-z_twzWGr3Ko%5@LI5fnSvqf#g(Se0$w>& z%Fv0BjB)iTtG6|6RW62l7cGv_R|=Bb)1xig?n`uDVRO6+EMhE4K4R{o!L{;Rba(+T z*dCfRBVIFXR%@6lIb;~*n8YkRztEcdFhVVY3LmUKuEQY?2x5E4WcEk$iyGB_-Lw`J zu@EctLfbL83JDFa>0xvm-=C+)n|eg`95?MrN95Mco+~e!35eL1E?DF+B|Ml$d*J$6 zsjTKj2i+}hnLU#NN5aBq3GZ5#-;8VR?DAWG=eU&xgfcz)>1DPMoIz#A{Ta!SvB6Y)wBYRfr zKF=I{x&O{z;IIc@0uFoVgcR|37pL87Ex+Ygohqd}>MG1Xr!y*WhQ*umu?t`P^Va;na>x8~Q`iliviGw0!*DEqEbJBRERznt%7&rylk)XBr@`?!_WWH9`$j&`h={&q8|{YHvY-?QePgFhJqbgdCgT+)+5DGdLx@DCNI z5n24^WW8UP9?!mW;e9#SiQ!P>r`^0|!&*0 zZMLc!pjci{$2Pb1p<9t|OgppWjtw?rnxs7JLhpWCJ1xPdq+s)FN0sl)Yp)N@K5XXn zjF5UGjbSdMR`AAO${Tk#SgRg>SLbGB^P|bO+BEU{aaXUX_9hP70|~56-A9n2RSzUK}MdM2=s@6d!w z7sZu+-^rblm!_8=1+j6Zql{dJPUalRu22V;_A6X3ELwqFO%=g?(cP&62=~$R$ty#3 z2kMVJ%tYqV=eQmuQb@mf9%C-aLHj9FrvU5NO=Lyf7cnPPOpn6ozc^)(Hym`Eqaffd zAzp{@#YE+mcRttE-8@GsScVN>*<~-9Xt|}0R4np86u#(QDJYmw$3=^n!Fs>Po=%R2 zsfSdSDb3uz-uRUj+U>iHRD)VqDMfzt@j8{1U?J7i52ue2k-PWqP51DhMlY70#BKOhl9gsa3aPIf!J8fsz?R5ykGH9XgTfFN1($R-?K6A0l z@Y=zJ@%sO?O5lgnaR(l)!Nl0i(idi3FL=8Ve&+i`=$|OaWr7Sd$R=HAkBeXBGv#4G8x#?lS=Z>msGkimBIo87iI@J0?zi0 znmUw;FmS;se_`rL>BC!NB*y16CqL2u^lr2oFZ6}nLRj)q@P(D*V7jBiT_Y8yI|>TJ zdterVgdUbmhvj_9;BBiqz+=9c;c&8)zKYPz<=_i5!YGM=ZezZ3YBD}3e|&@R9MbR{>o;*|1!-B3#U4tA$4tH7^CJ--Q2`YjfG^w$ zLo&*74h0&E3kuG=ksv-ol?8vi7vy)oh2c3?;OW<3MrNZY0C`L*Yqb3ZGi(^t?dC%v zuj}JshY-9jHRN@D{omL1|L%1a+Ab{iBxX^7Ym~*vlfj(OmHsUM6;xro9+ae-xyq?Y zW87Onje04jl*lQy4z&CH?pL%<^bXbdurUsr;gkgXp<6ZW(}Rc&vJZ=3AF`C*`{{XP z$79kOVDa_$kGjg0vC3Rhp_gTPKz&V0b(p&}<-Q&2cD&h+0-k0e1c`O&J|@?05+nfK z4EvXO0AJ6tCm{oja*Dp&x-5NvU~&#*!C>tl){ z!&6`@Pk3EGPL&Du#c_5|Y`i>5WZ&AtzU{UR0fTPYhQpP@jITyu#V`a$1_g(E>$h!2 z?l4lmTD!OV1&95X_X+r}Tg5zy#KpP;UrH3l{Qv6^8a?8oTw9`a?4T~t|6spx2*ScT z(TpnEAQzW*u;`iw0`m<+hz-HWvDT|{@VAmxO6KjY|LQRL(Yd-44;QT*IaJS(E&FfR z&OO5oW*F2F{XCpo;A5IE9Zpf4=5^}23rJYO>B&(c44&&DE;Ja)(vuOM2_ZL(z>IR} zv-hRjJjrv+{>bst7WkI{P+F3R>$0OFuuQk@s%w2Y&3#L5cdhbU(p~O8q3xc5!8M2v z_{f=k_ZR}$S8*8YJ|+N>DI`P;Lx_kQxz(u3s$sSURPSDnfTbg#R!amvZP@M;EJO?) z60c!G%(H^e?Z2;O3ldSRb!4rx3*8Us+cjJ7ke@6M<`x29aKc~{j55_GKfU0a;2hVJ zHY^1v0r?zE?8}Lihu}!TIkUIb&F-Mw8AHZ7Q0H;9Cog?fF;xIH+9B)B_UL26T>($9 zDMF7{L$yYtzs04r&pC})?bxO3FtCnhP~x<|?j-%6S^(+qI^7*AL>=d3x4$#e-q3#J ztc9wv(>K*WY|^c?$cNiPJd~72-g$jo>lz7X>+$wr%(B#R|5Em|G5^;p5O|{Vj)|pX z$DZaSTv#Vw*V$x$(?Ewr=jAUwsk(-yqrNu3Xl${FJLJ3fj7 zBioG29SlSRqeK+T(m9R4pbwtG5`-^zhmZ?udjPrKW~Mc;;KlEsIYv6;xtm9VoGLCu zA~Elez+NbMH;gWgQe=Pn63>ebgKFRm-#IY(3xspcDUZLYQS6V++3cI3*Nu55FRHjj z^QB>jBwp8ogzz_P4z?Ai%d=(IywxF8s?%(4&}z1Kb7|y1+cqx{AY@gH32PLROaWt4 z=@^l%4q7b@jp)un>)cce-#PfJh;Ry-R|L9$9&jz$<9*drYOwX>{4)qz}cIR2$MrxaIPy$^yYuZ~!k%BI3d0-mnMek&Cl#wJ7 zqo_j@e7DOPYzrwx?>d8XYeKU;Ci(LSHyD+P30Q9~gXy6{Xbr1W+n@@bJ{%^G;x9D< zCGju!#v+8;Mpu<8Hq25cZd451KUM{Z8mY|9pVB)mIRw zl0pLRv(g69YcgvFQ?br`sF#cP$H$MEa$O|nGO2@bA#D)%VV3|}yYykLI8Z>6(nxnN zI&*UB`C24OC_O9!Hm^knft+TGMIefzGix4v{ROzd=9NWLDW*2KYh z6AE{)v~2n3KyrK{QNnj8nI|(A5Dv%L|MO%+{fC%=xhImY6%Ns8DtO7B%6Z8yM8@HN z2Pca-{IGe2K|n4`sEEMFO@XqDK~q7`=khgm+mgB*{1o$u=(B3XTfR^mthB#;d(is) zfbpccmv^EkU}-CM4XEI@hh`=)gZKICTR-?NXnAdeSSTK}9!D4E2=RjEKbHeDi?lBL7NWRuB>U!OoycudJ}@q zZ}Gq491j0?&u212K1hbb2^q>f&s(KDqKgOM1arg^IKIWPYqZZ~MUH zcl%~ZNKc%TXGF6yVgpaI=Bv7{@vHRw{p8hbW1*@yyPrR?Z&cKQDg?)6$D9ertHc`L zXkKY|G2P2P=WoQfrnz5K_gST@#jATRd2G}z68#)F!idTjo9nH0|5CkPGm+GCzA`j# zu~8LCdqnzAei;oP$GVQO)lQ>DGj=A6>Qsm15M76%hYzl^MmOYscYI;sM7x;rG3M&g zJl1q_x7uBi%ULEAYg!C&@FK3Q>&#DKS2=(%$WAjDBKybBCSD{)Det;6 za2e5<%2#m_rdct#U-I5rbT(51cAPRVS}v zW3s^QNhHRT2D9V%kK?kOw$1BYW?A*ive>iUo*d-PGa+VmB&;l1$x=mZ*K~;saupLZ z5eS{c+wio@z$P1ci&#g-@N0~p6Q!M3g8w{#s&Zb=$qZZIc@}=>pFPg5*^On9NYRhW zdDEXo_Kt>qhbGx+eCg?d5EcqB@KU$!UcTOuGR~(3+~9~qU_2!a?|@cw!qs=$Lm0#+H6P zZWo6~zxZCIj^m2Z@&Eq)2mu~Ij1iV=Hq*H8X*z- zI1Z{m3c-0GwMzZ#_5U$yHXSV7b_V~X{NZB#Lb$;jWs7#D#=#>{3q(=fq)4Jws^&C` zr?2#&0{S3j3TdU+G1Q%D+VMP#D!vgZN^K#r{gk`Lk2|dZSC|=9?g-VmtX)RJS)NMjT|_ix-~;R|48ZS zuAKsaw~L$HG@ugcH^ZosW-z28m#lgM<4gH)XSBKhVqV3rh6T<`@pOIT(MNO1g-R9H z*Z_@g;gRo3g15z8ez+M>78T~@&O7rfoG1Q@m(&ky45byjJFF8`*1>Aj$90f>&$K;^ zXQrk|lib}ZPx^TYL)aWO8$h5#S@0JYyQ|Zglod?XMarxfGsYiJdlA72#-Z{c{jc02 zLV`XcX)joiUKu&tN&lQ}-VC>)JT-YD{+Opb{W66Qd@d*2k+8C6<26h6^(!{E1X2A&jm-?#^`tud|tFYa&_1CAN~H> zfaH=%+}2TtF#8o^^=N#N<+Jf6G;`!i5ewYJ6L${bsEoV8XEbcXUAF~kP&W!`$A(KTEb`JTNsbaNaSCG)=nA+ykX`Y##T8@cvQ@X zqRIbc?~?chjEV{G*gh`N(0dBs7uQ3#CJ|;eYU+ANgUuPf3%7NVnWtqUC!JNgbcS9R zT~=4ND3j5Ke1ynil5I(T+(Q#=XK(m3LW2uWhWi(%i` zqirWa1YyxE&0?A5FBAZO__ZZ0YThc!@k_;Q3}gqdKxr)3X{5U4OQ@`Ra=2Sa?VPy) zPcdfLjTEmmB2b4_-dxvlZ^~uK^>uJ5eS`on(!1zp9KEf9gj#<$CVNNHQMp5-_dxI4 zy3r<9YQy{*z1IUd61v@qsneQrE|m?^Hvm12*@*os+H!E2_Rqm$dNThMY#5CgF0TwZ z*DNo{c+Irmu`_WC@*u_iCQ{tvU`f)89o>w*;lc|mcspZZ(ReroTZT&Y+C>r`wqWtg zBUiQt>%ch-6@X7W1m4tHH9s5`&N#fNRU_eb&?G%n28HuC_mTsjZktyc-qkw6c84%$ zk!_c_Z3kAJnn*HHtXHj0qdoxq{2;}@%vRCZz3JNXb4-$U3 zCzhIAd1#4Nffwkm&5P8kmyH&(48dQqNB=~?(yu!^0oe?$|KNjc0J9ia8S4kax)LOI zwHs?Ef+^Fqyx2kxg@}9Ya_|Yb8C$T~9}GB0EAblBRWtXbOw1-npIijl*8j!K{uko> zzaY|$Ll&M}bDb+tpCiQ7{FUM2WW8r!Z=wLtD%Sg8pE}MN`3$R74zi(KYy{e!VT8ya zs-fOP1gGPb2ZUCVdp3-J;Qq*mTvaucQq+*OvWL?Kj0Dx;nCA%_s{=F)S3-2L?7zZk zOkC_}U<^!I!l0C-8FL1gn&u-Ys|#NJ-H>Vr*LeyAnX0~i{*g#9Vpmb`WGeOLdz6fW#j2dG)L!`k)*+_8cK; zm3h6#EE4xV#tuUB-$Ou0xJ)!<9)q5Wjn6e+Rn7vv)@6XrZV-Hg;awE?WWV2=d|`wD z8#lr#FM1=tiBTWw&j0F3G!B`2f{CjR2oi+39WLjhz&BGX#_MeT6oc5{-QK8ac&5|5 zOAgc>h@Qyq?ub$0b#j3`nE6&lJi~235{22)Uh)q@ziu;pVqOs0=y4QR9qWJpp!KMF z#83l7zI6(VYm)E>x=Eez(69m3naKD*uPRo@>vxb9{55YfNrkLK&oq=r6Wo4=m`m#k zmeB@uqC^Ja8bnPu`zU$IEqp4uA#$|Fbq;JsP9PEd;A73BPcG)Y=1CnoMjl$%psO?| zhrS2G#C1iinA8#Ikm?d>5@!dhHUP#%k6wEq|GF&sjyITKJ`AN0*6WmzpQy15E^&s7 zDnX?$jDO*lGh~ItsmznErw5BD82Du5u^X?-)R1_4RShZX^&C6tgTuNl?U2d#{_-^8 zR|S9uO4?O)GDx+9eutDZD1R1!-q{WEGgPT(c1h!*4GQj!%-JNN=DxJr!lK-9>+;WesUgQ9}mbuMxR?ktD6fSnhTRB?@ZqcndxLuZjB5^#&U zOm4cq?->%kJZ~5!(GeMrLVy!}rj4ry0Il(j<}&Y}bGJrCqF-KXtP4 zVg>B3H7k_<>Ca#j4)fi=2ui$QIm>4r4sS~i?;*@VHb5HR@MHk$bl<<-MyCUV=_#?o zzbS`sRnk0@H>~fCe*#cgxIw)-m(op$)joND;z?zwgUzWS{Om%lC)Kue7};`f!o8Lw zaO_=Q#-w$C0%muoJ`3XrW-v$JH-Pq#o0})E3eK-I&*L$=uv3TNxpf;hYX$-Bez&2IMOFnoyZ6|}SacA|I z!{Mip12hWxJoxGwcjkAnp8U=XX5*|mRM49mI&Ofba?d7^kNDCklo2VQJ%yR=;6 zyD?UI$@@JL$Y?%sJaQ$JLX@F@xSN+h<+zCE_U!uh_j166%3zvzvygWG`L`0+Dd@}XO*s}vTN z$|6yvd*i~xqW z{ABkHRqxtPip|20itRy_|^=%?~ zoj+~<9D>0$q@DLMQTPE0c_I*TltkqrH@`t} zj@x0Wq;%v&!2AIA7a?x zmJjuDq3(itMhl2TJzNK$EP2t_+=7%sR5)xan53v2rQ4O?GdyoQ5UCG2jQLnt%-iTT z>k+BM@aOwHz!|jI42Nj0C%ikPffT>0NHMJkQW_|zpD4J^Z=*Rg0*FNGkAO)3{_$%6 ze1UP;5reEgV8(JyIaxCaZq#NSLIxLGgsCau@cg^>p%UWfP0C{vvjV*X-EEmewTg!g zERvL0-O#I|ruLyHW_oDf?BsSunv1fg$LItkFi<P8=^~u-nu8g{OzM=Jlmt7Eia%8FbPZf|Z3@*|I3l!Z|^8CFu<~tN zE!jC9SV&-#Fei?_#^2epzNKR^+Z42Gr`C*fGs#*Nhfap;S=3QY-mtP4_0`3=! z>TeGlIQ3?_6O)u*^1g*+nH&g#L&_a0tr=4}`7s2QEH(O%qT6NMnsIQ-6>`F0f_~bB z-c97xFTGq2V=>Fi)#t9sK4ET3cWK3!xsiFWo}2cwC7Hum{8#Oh01l51YX6r5meEuu zRcVY9?~4~ZdE1ZAmf0`ApOwl&LbeC`XsssDyQ~A6Xo@^8ZYh~j9=_h-)c_vjXvHWv+;OKAv<@5BT1Su^?8&VFc1{S2VUsIz6h}!%?xSg9(^lT40uU3!PW3^Rq{T@5zD#*$44iu zdM{lclkhvf$B)et!Hezwlc$!|lS#Qn)Cj%DdBrR3lDFc&3L!XGVKECG%|SHZSvHO+ z5rNqyG;9PLYCccG%peV+Z;92cJWoTy&mZn3){?Vp$WyXx2hHUm&N7)pQvkNOXw#U- z4%*FsN!4vcP)i(3+&t@AdLs8JKflwRT)#hp5Ks0sgtd!#-gNi{mpDdQ7(Vyv^!NxP z#H3I$C^$c`LzuDF2m#-1LypN+L2nEkqCCi~bP=q( z!17z#9u~r|lY+x4b>C#_L0%ZSwF|-ALbvjx;YvCr61f5IoY`;-0BAL2M)B)*2_w`} z`4E`1JP*2!3To^oK1#P!pydM)U86&S5dH2jaM+D=??M`-i{w&w1^OYeb%xt0&$#97 zP4aM4|2^jGLGL3O5CCEP%{hR;@CRwsAf=}8&UXo$o`icVpk|HwXokeWVI{N3g`rbGCzm`5HbLR5#GAL)~j9%Mx%|8)c!SQRm~~;Sn@k z_qMAUlSY!U252u~WP9RUr1 z-BX~zOOCVvrpr8A0~rJy04=TC?oLDHJX1xN%dH^X(YRcDAPUm5^dl)XAa(FrQbJU z`~Dc-&)yYpB(V?13cza^vq;W&Ss%|IG4gtZ^!lD|ogV$xVN83OsRt6Iy?3g<1m5Dc zYBG5(OOQ{4WD{)X*@P1slG`5wxRLvQGBUnoYfCGqPCU=PFNhW|0D56$e(JltSSVCw zx4R-V5_F!?V8p0d0hqzForZGs2E<#b;b0*dp%K~qnQoSrd}xtqenn2?&*w`vpfbCH zD7=yix)ce32(2cx2|zW}&Rpc)_BBSOr1w8x&R*MX#-vP;KqUK+)|2MZ-nvJ0W)r2y zPGppA_u|v7yFcRe?eiHF6CMvlM7khOwUG^L0e8+2*4VKuS*H&cMfL*`uU7zvghotd zNMKg~aO}>971c&~Y1uhz67Ogr2bbdZBjjLu_m~##qBSKbk}_2z92(kip^7_gQ10gq zKQz5YYuE&kfz%??`gcz)nwHIaggSeM_8%`%_6TP3y9B7Jph4yb~h&Vg^&NVSX}8qh;1E)E}47%Nm6qc(EnY3 zKKkxNPa>C+j2BWS0nG29MVu)3qC^0cjeC4#IWCf<3O`@^{zWK*hP3;KW%e(~u2sQw zDUJX5c>7x+kZSsR!iNjAnCD&7_}MgkJ94eEam+dypCQ$M6o&LM6-^yW4*i$?snb2a zYWV#}xkEe4gD6Cc_YKxY`a|dx3;~*DO;LMn8WoSosLC%qxOUm~&Zo_`+bC!)k_h$i zqBB8FrX#6w|6&%}6aCUrx2KQG3ub6wrc%0B=m>jrZUmAQVa^8G!yB^K-FG9{9ogt3q6jUzMOiG|k=Jel;`|f7)3GnV znTvrP_Y;6CqrndYv?;=@1j$u9DMM}QhXw;OgM$`{vu_={M|)d+l`Otgnw2(iA9* z@dB@5-G?nAh~~r{kN+0B#`KVV5#eA;U@d+7bOrY+O;^z41_-%CVJEw#Ue^K}{=@`~ zZ+)hwapqX+?~`5=29-`9e_>(9nsQ1W8_oVrOiK6f6F`C$5_bH!REsr*u=MalcLSEb z@Z*OPg+>ZPCtmgHKF4s+ZpW; zQUAa_rJ@q<&GO3oGvf7t>;^Eqto(do*V4kSrR9)u^0kq9?8>>g(5+$DuAz;uKW{Al zZC7`P!Jw4ePTfm|ks25S`Jbu$QE(@sk&KekgQCZ3XO*HbrUUU@B^dKPUKO z&HEs?TI2ft$`w>QnNtY=9zd|@O1(9wJ3T3%07L9;Jmv0K>$bW;f+6-=j1_j1i`)72 zD0Gp8zj$$z+wRTtvRzH}v^1xW_0Kxh#!CsP*2LfWLW>Yx*7s3@)N9{BE^iy(6$qUZ zJC<|VYeV@O#Un*@n*LHlgDgq50i$b}UL28FGuaOg69%*~5ssFgwcDBm6%#C>raEfu zo{TfkcJ>_I_3e$h_nE7jJrJzF^-wcxtKi<|Et=6X_r9u|s23i=i}U;mz})h|{qo^G zY@aS<78w>}Ski9?bp04?rrP#91+{8zi21bxG#_;n)E3g?4tmY(t8>%bx>&;UZj)xq zJV=(3Zmr>7ikx17ktL?2EQExl;}{OqKY;?MOQh6x^6a=4lX{F;pV1qD*ttB;NSihh zgqKe7pu=AUci05+%8=>=>mF`FDjpFpT7_(>Cw~5pl})y2sJ5E`b9U1^>`>{Wn?e53 zcpTr9;Gc-6RAR}$FW?MzpjYj$)+c115)2j5(rrt18w^x1M6gh?pHu-}$n&Wt?MiDA z_InxOWQ>~%*WmPc-frTs|8bxGqy`AZiCzOEs6=kyk?^~n4y{Y}MSx6CBRvOiI6T?E>flt>#EhXlI>*q)e(%rme z3s>fw1yXCngZ{TZ0k%}1*x)7BYK6~t{7z;*F*Y9|z|t4$D@6D)nj1%XU&1`JIQA2R z2^Ob26ix)l@sgyaYSc4K*!w+YBz&}1IV@Xcq%C=C4rxa>Y@H*tF36#ZD!956=|_+q zp0ik_U1|A^*XOAV9K`G$-hT)EzOz`Y6+Tz6hI4m{9(r%J>Z^R0A#umrg|I;sYSC{J@uIbY~n|5?P} z%{`6ydY#@*H=*3%4&SbS4=d}S-#(5{Ot8*!0t*c%&`-X639EP~8)L1^l-<7Rj%=dU z+sHz&&`@cJ1hS80)e(_jeFDPG$~mFmQ=>ks9WkuOLlSs!p42%YN!{C2F&+)Cm>eb# z5nX@F_(v}H?B25wP+pA%mij?%_7r{a&3NUyrEfmuDTfk7sl=~@R|FF3$JqFGWmr9` zEQqHspfdq*pmTw6(0Cy)4BLz-`qx6YNil;ygX?qzW}u@}K&oTc-Gvm2RD*kj1Nlu* zVl)7AH#;$uCG3v%s!C~aAJ~`| zCXgIWlcAz;JHxU5)V!0SNO0`Df@-FEQ++Db`#ww55+}{%Hrh?Vq?j}Bro2|Wh;bVq z>j%IwZMc9A|LQH;y~Cz5OpNEn&d$S(F6@RQ*0Ol3h#a);8!uY{ed}XKo0l9@qX^4)Prv?OWavb2;eI+J zI8SoVkCGdk)u6T^)C^L@c;&BvXSQ{YP0s2Vx@MQhpBi)QpL~TmYQ~qNuK;lu7Jif! zxD6BOg-!QULj|IpZK3^tMy;5t%bZ2^Pe#x?dCsILv)^^ zeNE=F42$CLdY)^9l&MI+Tw{H_nhS^26F!5f>v#z1X$t)P;iNTE@(7?iPWD8A@P53D zY9#WAJ$+vo@k)12bs@qK62;y(fH_JnNA|A-GN!455bOIyK06Q(*FvW825`mX!^?QL zbLI5lTqGWXr-kXjK-Mos&-QT2(jA|;y0dK|8m`^|$w8+3ckxNc=S_f$)9vmV0aRl=OD6Mn|xSmC3k+`fj303BiY`S!kPZSm( zF&hM2!$ODJ^b!ZtlP`H_x325ZJ%#S~9cwSMFvUc6jiz_p>tQTE=^Ay1*&o91Vs48k ztTM`Yc^~^Ckw(!BAD9_paA!Wm*5s1eg_1ja=iP03k2|`uYKNv-(xIGo*{3vG*5#we z+w5#750>NQ&SY_BEgn$__ht^zmc^caMD3KC^-{{o97?#o3t#>qcc|A2c%>+7VyWKn zAql5hfNs6old9LnJKg4kGB|8Gcb-A9alCVYb5REhavc-@oXz#{Go|77Ib^QyWA6=O85w#nnQpoSW}3!8u~6kjZYHEcgsjS za5z0K(VtFtdKSszxA5qG!ORn|Pl3`0ogtOt4I6K(}%+^+g(f$_dutSKv9iwZ7I3)eGx>6G{^jywv~cg~OC4 zT+~X4%gDmGr>7)dBnIW~9eY%<{)acLxBr79SNVtRA8>+d-M@-N$KaV?l#7&jp?2{Q zyh?lbzofQBYGfH-DAsi_ILO|Sy+r_cQGM8SL~yUeE|(LcshhTL$?-`Jb;vdRM`^I-v=FT8P|KAol4=#(cJHz@FW={=b_|$qbIH1 zt^djuFUp^i1jU+_=95+DLqgzfI3q`<50yjXVpx3RU;_>)0t(>Tm;hOdpw_?m~ zw1?wE00*~k+6K-E!2mQw!mjZP{WlJ|5*{av5a9hZ|8Kn3qwOC@^@9Syb*#7^czOpZHU}QqwXr0`PR4}vZkkWF504FRnb94 zoGOC6&yMRrT38!0A69q+IEV2$&asE}1DMPfSLA?dfXlgLnH9>b&bc1oKKT)b6%KQu z@zDxT5-s)LXMn@d=e8@NRYb-VS!uTAx>Kxa)S=b)bp^)WIMXW+(hK<+Liy26kb%}& zUj&L7qZT`gP5s#8CI@6}o~y{LZU>P!40(MC_+D`uY4}|Y5&B+%>WX`n2uQl77Dm2L z4z9!jR1a4@qXOxFLlz6Efb+_ixBX4DiG#g{j+!>#{9<^tvOuT1iMCXQq_#d!yEB)LqoUv7Ucs2vt;_}Z{ZKle;5NxF^O9a z!Z7Pj7tBy;mk`ng7P*Pv$F)v zyJAL51p#0~6=)TX4$?p1L))5>HKh7V4gstuIG{EoiURp1m}rCyimK3Yd>UZP7fakT zGldPGBLe=}iAjKYbV*?ycL9{26M9?3^HmtVc9al z&`$q`C^5{^pD!5|8yb!Jw>05TCXO@)^6mclXiPLV<%H02)&g!;*zO@GtFPjwxRom! z!pVcHKZu7Gi==#p?jF*h(!!UWk$UgG%V5+5ug}|^KrFA%vO&WJ`8#;794twaepoR| z6a-SmSV09pIPxJ$T$ajDq%s>#B}f#+p+~JzY0(=*&nov`r_`_jI-=#-GUCWEKb0!E zH{3RCJGQ(dnwZzg^2V!e7fG+Zf_ZpacnU7{$Q#H=J^YUmI%sOr+s{{rm7p%57NNEcI#=xQAN6LOT^^*d7=s@r>HQF&c<(uENfH)39%99QBH(x9b+BFZU0EAu zrAMp+VJDfTeFoI4r+#SbgL=eWG&4gRB2=Md-m|cFo)zt!aLh zK3^Vq=tm14<`>W%6~%IXM_Jd~Tnou5iKW`4j_?HQ>*Z1jd1QaWqI9rc)P1KUZKHjK zM!CCw2v~uoW?wch(@;5)bl;#dW~X>EW^S)=J0ni;w`*Fyk?wxg_~#LPR1__;ef}TH z-a0PJplKUcL;(Q_r9%V}X(Xk)yQQTY6a)nck?!u0?vfNF6hvvHOGUaxQt59_yzlpY zp6~bl^Zj>mah-ea&d$!x&dza+&jNbqh(VQB(BDIwE39`)Xr%?v%|f9m;(1pmM z2jU`H$%zAs?I^nw0fW}P&g|6kQd-(HS~Moy7U|cVY4;gelZqkZ1ppQWpnNdkuP;!| zL~X5GOYlnE3~if^?n>iY(L=YHB{SBMD&ZG%!F5?DPLWUCSTz3{&}IAW;XSH+?1R7_ zV%$n{{NScImA5nb2KDu_gnB$Y@wMkid#lmxG)7XLYsSK&gmu(We8XqS%q2sXymlc(WS?oNHtT9; z=4N`XCdc$Sw)Uhi6NAPW!T1i#UdrA0H!U)>KEXHXbfrx~WcpUpG_)M66?*nqsI9s~ zG4MDfD+yH_?WgOcK5Y! zea%dZcOt&bF*EC(#R?`5g$ohNUAlm&UC0yEfs^Sni6{M=xPYsK!PTcDBJ_9`iD9nL+(ZNcDk&q&Jt{|J$O5 zBvp6F4tt2Ml={sfSrG~ALpm3Vdn(6y;)MsYxqr4M-kLIwaf!LOug9kW>>XF6aES*x zq9G0)BenU2*i}SLGs{uUy3vvoby0ieM1^SLtx?RpY{ceo{`R-><}7Qy&9xjfqAJT( zo<7omJ{phWq92U)1*w_IDdYp76Q8T$M&1j!D9n=3YF?+a!GEWf`eqP2t4oDpb8oh7 z^vw>orzkCdnY>REvCiJ=vP8Vn!EF8qO6*VWz?LUui-Sc~2Lqh&x}=VbPOi z5q4hr*UL+ANS8=XY(#825B`AUM>OhC5xy0JL;5r3&V`p{wh@^uY z^ELkxM4N+<1*3l#U@;KDP)V;i%CGq+KOYS|+>h${C%*+MOn_wR1&P~azG7+!cUq8C zrpDugKc|CVV?w-{bVIgvf}hY+bJ5t%n@*|GPfFNy5#@Uoftli zc(!?s0$p1Qewfo+;e#Wi+q8s8%Tmza?=rQbhE!3Mh*c?Kbb`Cp3#&rlgK_g-Ay`+x zgoem3{oMcM0+5cRw1~4WRK2E@aQ)@SQ_kRi)e@Aad<;c<6OtsrUllMo0=Q2^EHbR1 zOnYM#yn&F&CQ|4id!$eHZ|Pt1sX|3EW8k_${~*m*qoOXxV%pO zeyuS#T=P0TDgv)PU~1Hz{rv&2=g5-|!kI8=rQ=X`PF|)aG&bF4!Xs1 zIeN;Uunw6w(o~pE({Y>zUy&T7-yOgbn|)(JP0U7EjTzgGfoaOF0^Ng<5F`n87leu|HW6vD?8q>6a2z zyN-~#K;htE=ZoYFPT9ZTJ)Q54Xv1PDy#HDYVg-!(mwz1LqKEVc4elHNu+qkR zs%v-@_dd~JCq!mb4D+JE=_(ayUW7Y!BCZuD%w)J0&BDK0zEodDP#G{(;v>e{)pYnc zVeiCYZtj);4gp3LGNm!4Obx$;TfSdjPuCiS-x?Cj98(?6Ta?HIgz1H`+d#UiWT1wR zxZGkcg4WK=|5l$!7%D?>hf}cAGDC1@{Dx_7F;@|2%Y7r%5;LYG8Fv+|Z9ctyS!*g4R_Omeb)U;W zZ$cZj8>2($mD4h*j>N%k`iK$zu^+}wu~^5^;&{BG=~`PNo5cFqyf*eZ=O12|w-d&R z@@X~kh-1*vM`TrK6_L({OM~6mUh^XtrNRc48CH=;YW2cP@$aZ8u_a)}^Bc;_uvn(b zSa77QoE+|sLegD4Q#LW7StCyJ0gY{zhE>AOp-Ht}Vg~DXA~F#+9gk>G#jKR8kA+dV)WR z8c!-0q!~1rwg`IPiMobDtwhIbGi6CNSM!ba8;20y#j5S3L)OH4^txk4`R!-Qn9jalh~rbGgkc8L1w3ub?Pa5<2xeZXhD z5sO^~s-89Q2@A35fUQXqHRe_!UOv%N3(N~`$;N}SrJpevVLMBLrSJgf0MXEqco7x- z1be^b#<_~)jyAuU`!vzMBsb(4t;VgU|KpmY_a&eM8N^NKnic9?hPUoX=-<1Csai%i z!^yP}MWl+l1yaxLjA20qBup0H^uGKm{Fw0W&(N=DMn--IfArXF_4(jNSM?sNm=k(R zigH)lNYyKoM^QyWX5Bd8p+Ba>EfDD|wB~obV2Ki;;)51fN-LqLGa+=`@YCyyh1-wn zF{pD!Ex|uu!Yihs+7M%qH_v4mhMIzx#BHAf-j`bS{KqR*Rwn0Xr}3>XOn#G3Ux-^* z(q`Lx2PMKSKeV9=?P8DE2GbA$spcY7jv(({t@$`2@E}57vC8yZNksK-E&k+0S(Nrr zpHH_TO|ik0S2{UmZXk_!m?^`mhb@$D3jHZ1)PmDsRHISa#e9u;3J^dLDJ&}+!$DKUIK&!xJDtRysyt?S-tz73620@?9#pfzPfzDR7ai^{2NK41+!Wt_ zf{$_)Wi)7ny$p?}026%@5$z=(KImxq6*HF0K3K>SdYq?r7hU8-_$y&PUCR6u`}$i$ zrUne;ff#*xb*uEWy!N+XWrWHwo47oBy#0xdFU|J@H3|zg7u=@KeMYuTnDjTG0H?{$ zlIUhW@QKhSnjLhz9~E$lfpi$ww+eiz*-fgQ_itt?p?(e{QBQi`JVd)#4|WmJfhQ zuWk>WF6GUmqml-`Lbs_qRv=U)=d@CpH%mzq$@IaHXehufBzuR_MT!hO4H2S_zq&s9 zCxj&XUew(y;8St?B;XeMwmI=`^S)?#yBf`p{thn*qL(RrF2!6!fdA1<<+d+%B^IID zLe0gl4$qg$?LV&Y+d=$0D<))f(Y(EI4aNEnbTT1@`VDk)6jgs|?^#0-D`#TJ*R!XI zHI@D%Dd;aUESJ(VASwje@M_n=)@l)AwYr@6%}rFIJig;-~K~_Ki|S5x^&% zBH3;rMB}9S$)F~0d0VuRMCyjf2i&N^;i1_ipygGRfMZ7vACL#BlS%}LWS|3A5nfT= zDK#{Xv_~2YlSVUo&CTb3p6!3i{DfPB+Rst_%^vKEbTc8>eaUa4uUox){2vojB7e*7 zPSi^j;9ajV2+Qw3RYtvZ(Oitk#BPp<4xP~i9DgsP$AQBB+lr$wkfHQ0CkNZ)<6_*5 zS`{=i{*=sNIp#t8t04t1MnnmtlR6}RsX(f*e>L*82{KzUm%#?y(!kg86S;zTG9Tpb z$ir-HqPF?j@Ez3d()w@QOoFV3WXHy5ENeD${W5Ddg?MPND*Z403= zLREp-72Vx9#BCI5ghvbBpLGhN>V`mj^3ltU@NG8gBM%1Ern?!cC~zOA3TWkEjYcR{ zsC}yj&}JnPxrDG^Q>l=FAg2D}{qsL{1K|0dC(FgSGW_s;%{bK|1TVS~QHT2b;ugyM z1iXZ{Du9Q9izQBB`A$h~Pa-A~r-6tC&uX2+cik=`Olq}I3fR30)1k3VEZKjDi#25s z)md)?HENXztT@bC84_5B7Kp$fIM~?O5C&es?WDOpushNjO+!P_46FoFxe0i~AK5^| zszV=A!w_w-k<`U*)Wh<=1>l5noJBQSAl@KM3B6cLTT{5?+O6CPa^@ zzGe!#PlKF&MLa=JDfVd zUqAz2AkF&10m~GENmy@BE)k)!g-_L5IdR}vvcbp0J0h9vKA?idz-pzUH{dSTE9GdT0mK{hmU3KXHz0afeTp>;twi-*>A1z1Ju4A_ zp#YS6vITeKR>mvdtiieTmO3&PR2T~#dSdRsb{;@G!CAD%=w>Oc0Ro?{Sh^B8NJ%m5{>MmBPnw_t&80j{sZLAozY~5J2vI43u}F++9g&M+ zdmIi5FJidoy9-t%QO$~Rl6Pq2h=HijW;kE*UvK`;PIR%Z-rj@_l!6{ZjEV?aa8~WR zg^zN07k8||Qo=>Z`=%SyJ~*PIkfT;6iFmuDaK|-z16EFE3u@aGJ0X9Wqrnrd3n2$7 zLC^RYnJ&B*++w1)kHKZ>B)-h|honR$WM1KMn;(V^Y(!%(An2e8+5S~i#{af4d!37I z46$baSCie4nfjtz(2EOa$IAZZUS;N-Vhr3)1|MX*g$xay(D4?}mt2oOTq>cs1}fKs zi*DOPtoohZrA_lf(jTBhQx}|{nY=L9O9~Icl$iN_h@)4J6QGL6L?2x&e8R*CqAz<) zf>4Kn$tDu`#0{aN$Ea<;EdAE5l`vJoZUr5?3X^vAmK}V$0b-(fYz#-#+|UmIK8Szz zyXPtdGc+D1TtzX?)~fV?@T?kyX|XBc#vRv61jYCdrk57F=FVHlNCu zWLb?qj1i@vz_^{fm!&k8PtZiR@2Jp1j~G7UVT091^MUL2Iuj}L2a?GsewTb|Zld3; zdTqWA5OM_7@SiT-M1S83U259tya?oxgEI9$T~vRDK7y>jYcC}HVRu6Ol=*Qy-qLW9 zq#yzX5>ZAOUwic2K#>W-e$Vs__qzqiuCpTr>f^|zm)_$`ik=%TGO@+?h)|-G3cK!3loCuV|DP9p>v8(XKG%5#dMz-* zZtf>?0uwQBDHj5*UcpOZHxG#k%yj}XCvt)aF24VxzSY66%q9IUqpFF*zzIer?p-J? zWO|9YAQys{d}N>*8sj!}m<>iBkU8V(LgGKAEse%dNMaL3K({H95;ly~SX5D@prJ)Y z4DA16Czs)`%fxmN0*MCWBQ52B2mE^}WUG%WC@P{4@1P#wz03A=PJ+wx=$UO7UQt;P z+bfY`lo()E=;+<71qm+m2@QBbY48?_E5wiryUUTBDn{l#o3 zQk7!QpDDYH#sGsYtR=*nuCC=h!IcSvnK7}F1#Pjqh4O5NEbm^Yv<3?M?b=>>d(0pW zZn3mLkt2++0eP)A=r1F#uK?_t{ibE=7U&~jiW*%5a>3qlKJ=+nYWa^fgNESNL9ag7i zk3>bQlA5{zsZsEom&7*Nd- zhy~>MjuuhGTzDDCZ81>b%0dmUE~*Vl|I-)5$;mH{Z398zb|O>%O@r&IKU|mM7Mpfl zTqe=rqeG2>R=AH2PqN>Mqd}>gDU8ep7F7fRamL~edQ6V&^rE!t63B2!kl}ok%ag$^ zL>3I6$PzxNMGt;u@GA(->j}HH9;NX*kAO#5O5EtWS?s>QW?tKly#D_+!w`flvhK$C zAC7|0HE8?$t9Y>lEFVaSlSSB5v`EwWpKEy@?Y+I&Y|J8IKnH7a_!pM%|8vFq!g@bE8)PxEZ zNpT9{#e1M4A&LS-(v^=tqU=L9qYmqExdA`dlb<60HqQgF5|*tI^k60~`7j}RYf`3m z5B)B`}p3GpSd;uc_NpAi*F+K&m;_e5~N5JDZxzCpq+0R*7kHi58eqmano zVX>p!MMI3B=+$bmaaU0GVN)1S7oW6 z7EG$G%Iah%ywBSqetuS^VL>miwMJ_rXj#;2eZu_1?dPZmA3XsPzEx{?d$4y$Rmab! zSo;cdzelW%XV(SR7|ph}7yf&S#E5ctS0|@FFu%9P2>wy;5c($XbVw73U|ya_ooW9C zKKs#=mffxP!T|!+H!p{dPGrs*PbiD`nZ#VmP@QkJ?uEBhU}C;5bUb_Q$mhFF8~)R4 zWbbaq8w)lsYxk<#Tzv_0$~(6;M`8q~x>E!878*Egnq^Gs4!jCA&X9jCp1Yd8S36tK z9}(bocUL16jx=5@%kSM+QZ7#%JB~!7J@ofSo2oE5UtROH`(v82^=8OOY2+aOpd?Oz zyJd#EbFgwIa^3Mk`QBX$c=qrn|G2;ye>3Ndca6~wCMFIttf#_nhP?b|oV}*0w2aTb z)!GL!3d(*NFnx6Psh=xyUn%lH-X<7QbS`W!qkQH{c+1Lo`L^D{>-<_iZS4Z-7tCsNu=S?$sV3<2#9)X3Zsv5pKwEcE6^wcoK~A^0Vgb z$;n9yPgBB4*D6Q(GntC3J9J!5NAdCAtX^(;?uw>Np}ng>c)E3cF)}S9j5OBoWRD?E z*v1`S@Yl?9Rd<4&FQ-Ij$MoL@?ws^)$-YtEJ$;U*?oAwtdmubjA$b;YP_nZtf{23g zaTkwO*1lw6&_}!nIf0VaRO(`AseM|RRty@oCS`Ti~u6RpU zV($!Tng6kF?cR+&!~bKneJXvtIOE)PA_?ah+k5KFo;j@k&phQVuiWGUae2oQYo{>B zXDkTYH0UuzK3!&&H&3WC%QMU#b57_^^Pi!n8OkMWzXCLPLaJNag+hvI;rxA4$`?PA zC|j4RNGv?;QA@7=t(SWmql*PIgLx>?{3YjzQ#aJM5A7N7CVX~G_(Hmz&7>%7b4J)4U3>1;dnL#_E2?#IZITBkf#0T?$-~rc z;7wCr!wZ17Nw{8vb_CE8Z_qJ|*%{f5`hNs5pBY)P& zxzzr97&d*v&hDhtqRg@6@#3G>Rqw;+%Ix02r!bq+rQz$im(y~H*|N%HPyG5=b@*nb zSIWn~PEF5*Z7!r=uR$K?6o(n}WoRdU+Hr61(_85>qSewvY#%my?H>sW$~z0^u5CMX z%h?GHgTt{^f89zoZky12ti4!L<12r_eP=83Wf7U;AN@HGkc>CN# zqgCoyv*9$N`|CM#cjWN$*13u=bRzcM|Lz2j%`xu&pYLj`gI-;%YyyyWySqIg)qK19 z*||h>t6o*>t>-WA?{ALR+9e)s{cI^=bp)GWc5RmHaK09y>VsUcgQU2s)hL|uV%l~-pD z2?g|NtbGehE>^k@M?^CKXiOSLBN=2U$Y}zp`>^@xe-)3QlLxv;ot9*|^wbCHqq4@Z z8-*dTa>k4NeAqlLNG__=A0jDY@ZyVt4ohai7bI8eMa~=+#eDQwutVWND(Oo}1h4&n z%30LiykI6l908=iX-rQM7r=tS#o@DpCY?X*z3d3*X@$)H0$sb#{jdJU(q0htE9CX} zB^9mZvcWK~v;rr){-n?E{ax;0b(ufb0T zxBt)nDkVHb$m}`d-PL&J-_KxaZ+QuRtJIxOEMYx)|EaAt8AS%<1JMo3kk~ItyV0BtmLo8jj@^)VX&gin>)# zlhx;;hx6EK$H$-p#`RYGZn&lxNfjY)Y53t*{%3#gvBa_FmhL>{Hi(R2e40(I?k`N( zxuY>I#%lT0wwa{FP3Aha5EOwbxx#ZUesRc#+!y6ct(i2mSGCZa?)C~lm`arWdnzT4 z<95ZGv^@Q!d&ZS%WsArWLd)$0ogP+dvZ8vN};U>=E>rNpvYc!L8Unr;Jtj zCw@Lk*PvrWHnio6QVR;THd$HdkGHAK?R6y%RM<)YzFqpvsL0M+jC{@RB36v3m(6zH z*J?QC!*J&ShV>(Ckus(`Jfg07TRL~C038-kCa;f;r}J)H)yrI6sT!lnQ$-ba_;wLP zV?9+*XnDU!v=%09OIrnZUS+S2!^@#L&@YN5=#nwS(2jEQ=VGWvI^=|M>V zjP-{X?Zq_`kB^OqbMx-R(D=pI^5dlA5TmV9f}W(g+@2q&Yl+lWz$m&yJg|B1IDfUmn=yZbT=> z;XT>!qh~Z^nshOjI3~b!#Sf1D0KwmivzhZ_?|U5-&%dc6bYl=z2ZfYBWA|!N<=~3kea#p@hB?t6(M@!#cVcZ&thjCpp^tGE? zCe5YpyoPFW_vc!wY11>77_q{BNv<#IGUKUlpE=YxWjnK3=7CdkoQW;w5=A*}O5lXym)6s%&P7f9r=O9a%g#HSE z$clA=V2oz6kr?4du>q7grz%nbz-8lY=rxpgx$2>tfYjAo?oU_5*R!vUN9gTnXzPJY zzL;mm1A$R#1y5|ESq}qy*gcn@K4`^Aom1i78FED4ykhG2iy3LADo`4;!;vHt1iDME zhrn+EX}z4iTjdP7yTyQ(5O-M}k~Ba-$Ne)fd22vx$o&O@%pxfE;$+2DA3*F#QaJZM zu<<9*Qh~Y(-kNYslT*Z>@dkW<{7LK+1fCC2n9;g0bAx{)6OQXP?>Uz#K<6w1a~}|(Og#XK)Zh;= z9XGrBd8StO`UK8rZaWHm7S`Y1!ai>G7XIYc20=-x%i1&KVX2As+EO1~0}dFc zD$F3H@dFr(4ac4bpCea(1=7RZ@=VxXhM`ylJp_A$Ri}Vh;$k!?ky5EQyE;{;qo&gvFRom-viO?m=Q;6`tu(RPtE|KZ139g%?C)k8*!I`Cy_Bp;f+x9*Dr2> zOUL{YJMj?8U0}Ms159@)%QmnxxB-Wv?yQG(>R666&KAHO=tJhWtc*vd0e{c%c{IDC z0PV-$B}s2U0`H3noU#w63VVAEx_K{ua~@!xu)aJgyn%2#_|G5ZiO#-wB0e=gv@>y# zCs@qg>9_&N^gp>}CC}240$*gpULKS3=vPkhrvzBzLn~V4cJ@LIlr!q(+JSHU*Z$T_ z>zoTfl*}!S)ZDQTLNQ?Aa}yhraEhNsf7&AQVbS7NR>Krpo*Z?p+1>jV@<+yX{$bV8 zx+v}%hFKp+9fQ>MB8I}W@rxx_=RSQ)g)pfq3F1m=dTH{3N$terNN(_V!- z_A@dNfjmb~<@T)dU%!co_yOqQ_0^*Arf}}5&e;C=nbjehsSuW^Yr?m06$3?ry|Qw% z>h_HEZBGr2mUpgI_cSnx1WEgcjWFHZZKB?>MJsiqzx%rsN5EXS(FmKt^)D^J3_viI zt(qf6K(am%xomC;BnzK`Axo&QUz6`Q5QMrRF#ZDRD@4I>)hfK+-#>kQ z7wE*@fxs+-(!j7Y5G!ub$kQFl_`a?O`U0W|`8Nwd48hu+9aQZ8%=x2Kb2KttL%MMrE*Z+ zIaz1{XdgCK2fz8xQ=3H+Kgb5i?C8xW+kK-(w5<>Dig)$?^wU0XAd$ZvmyO;{EjGP@ zTby3PJc>6&^;qM|yBL)+VGbq%1t87hl9WZ0q&n3x8RT_clm3Y?x~-XvehzhdKr};6 z+srie4g~6SesyIvPw`qZcEJ5s{B zP#34-2~$G2LuF*&9rW{YRP+c0n;&qy8~FXC-vT3Td8a&M?9*}M0;Z~Ub#8Ma`x<>t zO862pLGgQ+^`-b0fhf#t>%A@#E~)|EKL*uf2zHu*)jE-*Xe}vdPN>A_WR55!<1FWX zx@_z{H}|8+wpvGk%VqDjO`Qv#0CarsZu4W@7HW}WZr0Wlr`eh&)a}R~s)EP3q9?$a z!ICD7faD9*s}1b=HSC-2Ki2nfj@zrGhoGE6uWvIh3JZ_vs&5Q>?^C~_XXbCd?x61J z>;8}=GFa%6;dOfgq+Z%7=5nMOZd5VedBZO{$vn@$L150lzvm3nV==MU5wb7*`dczq7`VN(>YvpF1SB6OS^z!g@6mbL z?LJbRx(N_t$;x9$WdsZBBQ(gWpO^KZKL`FgN2!q22Hn$PNpD7sQKZrheTBf+uH|PR zsjQE2?sxiwczMAEO~$j0Pq%vtKINs0ob$0W^gAk>=?&~Uo>XN-j>JZouu0hS z1MVxYY-cxX*_`_@s;Knt`H@*|v735MA8Egavx+MW_~su7!G`ilMx1y= zeCm-nOW`sFm-+AM$GrE(e>beUkS;_eD@kTN$h{${z+itnd~o3WW_g?TyP&cYhuj~h zoIQsqho8;?yv99%i%_yz|5>a)Ve)^>e|K6{oDFbn8aC&x%^W_7IfVyLXfKOVqQHfi;oZXp^B4p-9eY~SWIB5n2fh09F%+LsYByc5gJ@6-Q zCQ#|2>>e2JCB-m+0yTicV%vv_^4NWG(b@zg@kz~`)lAlc{i zdWEHD{4>2I1efjIE8tW8uxHk|Vqil*oGb3u>yOVuGY<25MwQJUB(C%6%+e`Hxl1-` z&JeV9?ZkemJt%UtQ%Ya^wY0n694}of;9Q2~M!CSJoyBmMrov`j-25x|meaNkSF8-= z!y57ND)a@|bXi}sc20>6u-X4kI!x&@qFT!|=CTRGIoKT6m-56B&n(>PlAlb$a-jtN z+yXKQZYTzK^uV91OMce*)seV7vbmjpYl`A34opM;^ytTAJG1|3lu&(Y+wMy1RNffB z6LIm&0%rG?yUo8dN62$X#rMZ>0W> z2`g}OcyAwJTj?&oL$8!BfTzIn&1Oowdax-zYwAZ0W5grpln16#g5HWfG@8mH#@cOl z_RGff9x_7>=6(*V5C=BGqhT+$^+Tq&zA(qG+f`lDRuk@?&X`vb!X*hYVL< zXz$w)?^B2pwv_~@(O0DDjBf@%yb$x^-rCp(| zyj@B5l=k=D!%<;;&uP4+=h5wZBymeXL~!&pcKFunQqiwjxAskr)A1*^Z=ZL|pYPuY zojlD=)OT%_{}Mp2SJ}UQc~yJ(O4cB&M+S$Z*1qk%lBbcMc@Dl?NO24A_B4DkOQ|O< z@GDaPPLV<8APpCJ$r6j>hv=f8%d99$vP`d7siGhS^6W|t( zclTdQu)f%h@p7CiY-N*>CFXCA7v#^At{rdJ{xwMd2Y0{@e3uZXQ~aUw{>fkyRd%sU3=w|Q|oR(up6%- z=<_7Xq%~a&=YYYWVCVZ-q5Ai4n=S#nZt3jV=WC_4=H#Oq+V5Ex2ng1?n!Zg9?yli* zexKOU>{ZymGFy9#hJ&*|;FiMzIArTy4uWE-y*u1KPjHDVC`UELx<^&}#y{RT5Hoan|JZxh& z^q1?-ih1(CZkN3*Xl$5X-&=PhEzqyR@n+p#TO$CG$5@!GJD3~yp6WyC931T%H6qczTYDQ)0_{McHq)-w%9AtZ=UVzVB;(XK%laVuq#2~L{ zM7R`)iG&maSBIMP1BZ4=Yf=rd<{7$QQMltKi;5ibkUewB=+2*c)dVbU{LlxieBJx( zTgt{#O#YOkYd0DI>o;M5 z+yJrG!Pnu*HQect+VoanZSDAkMltTlP2SGH+Ltsm7Xfs9BzJ|$pV#~wj_<{AHVOpT zH2$dD4p&|*pH@sC5S_Nutwd{Z4Ec%mjlzAv{Y`y=1WiSx1rq^{IGIa z(wjT8ez)y~dDe5|AL6#eQ{va6n*6f(BIWjvO>jit&Pyh~Q~$2Nw;w3(bzzpuHS>7- z^!oPl>PSx*U0dg_@)Tr2GOS70^e9%dYl|V9Tq5K76VcqSnbS>$QT?0wTQGmd%_+ix zPx-{fYnx?is}Sb$(m!-rcO<^-n~Yejc7Cv-FV!|gFRna~`SLC%timF!KQ;KKdr4|R zsvAp>3_0nK_&0_hbkoJfUH%GCiNAEdzKa#vAyB+K7aw5w!UFvWh^-7*wSQDgSb5-J zk1koWG9xeO0FKI4YRf&enNWoEqA4_N22yU24xryuMiMD2XzbMGJ_^ z(ockjc~hF{Nq9%^%MwlT8ka79RbdvSvrJx!`_s!xQd_b(Sz6yX2o-yo zE0cz`?%8tVHSfvXiPJ3g z4x!izSXWtwz6W-Pn)+fqE~M=B6J$nz=FAZgVB`5~?9um=uX#-}&IzwNuryCtIwm#f zY`xuoS3jY2XPik(F;%25Pjur7tP&?-;}q=Un?*{EdD%1i=AW*;!@8T$t5O!9x2yK> z>-e*GY+aX($B*ZTT8Sxl-t6NR_~$HWmR%(^&#XsdNr!;gpV5J?msTAr246UsglgQ0 zZE9P4-7dzh42m%d}u998_%Q)6Y!Q{G%mSyghQ6HF-6zn!n;F2s3>Gf=r|*W5$% zhx-HTfQ=^~3NE`kuU#mId-UHtR!{mrtQ%hf*fj#T1yhZdikq%K&35-c)h;!hrB)RF z`w_>Z)*FIwqsPp(f%}nl*vHhnjaA(0=c8$6A8PBcnk#`4e*MU+FU>}&j8Hv{q?mxQ zj<-v%ulsSvw1vjdnSgkMiQhCY*w_vG*0#{jq?|{kWliX*J?#3e^=k#Xo2=Ct-&dB{ zdxk^Zp0@{`mAJ^uDNlPk9*wujdj5GovrpIE@X2h!g@T2coYbP-OO8WgSFO;p{Y-h& zX55Nf<9W{e-E#Y56U2iir#UUcr85q~j8~12oOS2V`Za>toE3AVP*SkZ5#z|MVf%(K zqCT<2iBqlBur`Zt(5B>8$|L)t{0NGaWgGcDwC8iRp#L_&G2bZS6*onXykg47*tSEmGI@2 zB1c_+#VGfl6}QYzBj%#HRlj^07I6%Afl{E`q9?cV+bXaRl?Fi3K(;obACUxmB`WuR zxuyrZ;K=@!a+8sFIpP&J$pKOfcUv3YzvfBSsm z3ufI*al^LRjaV)BGl>6`tZQIv?5YvRtPV=w-lu-x>%ikF4Zfe}lMd(o*PO8UZ(ha(L&-fV??b@xIfm&Q>?J&p{! zgXHoo|I)yh+2^Fcm?sPmA86En!6*t7;lYtZN#vQ(CXRHyc2d@#C%VLRxp|g5(%Pj4 zqRQqytxZMaDRp)fRdqX+|E5S&->yg8@hFf%JUL+P$@O{@=QU%kYlMpXL+t_a4=f6` z(rc1J7IwpqNmWu4eH0Z(cyoE>9~TDZF@%j2d)7C)^~aPPtbJ#<8ATiP=Bj(M7u)n! z?8>wIS*J^H?@>I5fa54m{m?G^!<4g=vHBLxL>ysAKqoYm8unwp;o7V6U2#S=KTFrQ zb7>@8+J$&%0V_U`d+mFC&$2Qplj>WL>@ zBkH(+Cng#uh3s6Ef>_@l!K%2JSf{lL9Kk)6*37)qrJ7N!S6`o)?-bdswgyPcBa^7a z&bCIWorD@HEBrw3giC?DM{PEFL4^8{KwJO8t)h7I!e1KCd2TT*%V&TK_-KFn2y>hZ zT^R?vGNr-H$@p&^1XctE@@A_)hXUNqHN}VQ0H+LV^!loqr|%S)NN9j|}-KKj)0etI;+ua9PsC3l4m+UnYzEs#0beLeH$UBo_9b}qAW5jj*>5o2>ro)|65Ao0uINI(|+_lW8fVaF>iDg=)%<@u6 zK^SQty+z@TMFOR}`kYnMFS3j?nop9?N)q)fGwt?_I_jedt$5On=aX4IW-+mesvdPX zQNQ(nol4Zm*rW2j=$FXPX9-6`aa{F+&Im7yJs0y9aUepS_EkGf|?n+gT#XZw7uN53jtqu+ju2}jfHXA=mc zsJCYnul1LS{>Z=8$Y+$L8W!qLjG zLji~hl0~eF_VU>(ez#TH`59~JoYc*xQSyx;u&yt(a6&vVqTaOdhH1;B$s?^ZxW##E zI3p1GAl8)M65S6+UX!#)b9!>L&pF^xt#-P>g-fD6al7+=Qes;anKAh+_*II{V{@+| zE8MY@o4j%p-n?)6K->k)Xaz}eYB+822~a{AZPD1NN-NUUB|kJFW+ArvU{-^ET+a66 z*~%fyA)B4{N<0DSYvzrh#fpw_glONWu1cV|Hs4PW$4buInGZ;Z*S~Cb)OiW-?2H@j z&c*GlsO!WR7*Z8vthBnn_g`%daeRX~-Q_nGo+kws8z{~b^IJ><*XsC<UJi(meRH~DxqiHl z@^12@)H=b0yC|O#=8XVri1;kneU3BGn`XcowK094Z_&j~=igB^KZ`OrEC0M~8#f>b zW4F?UDn+ok{b@!h@W#(L}xET1s)b;4{X&F}qUCfHf&f z53<)H6K}Eaa4%$OWegD)1%4WfMcc0%|Iu%AsWeI3{qb-d1IhITpUnC?IF(c+CG%;de#%LBeH;)5fDa95yRy zY@9(y1{@ZJr;BBUEx#eYRi>Og&dVO)*OmX=_K7_`=MTRJs+%uE2?hw zqYS0s>7J-_Jaw^;pIOV<<;RFMIm8KMhIpG*UVnE`RwIX@C$qsP-h-0G-ZB7kY5?e&TCIl-ZNGJ2R}Y-){?#z z{EVD|vB*U0!h}FGD^84_$8W`JSvSP@v%eWut|S?v>z(cy7{A-2ex*U_L+#@1mX}t| zOVkx~ww_)W_jGvA@0pq5b)zbqrol8&jj|@Dg*Wly*KFk>wEcDa%MND!P`lG#Yo(M~ zvk2)dkH0#GAK2@-3lPd_x(w5yI?LrV>@GYRbtKb=ah!>^OI)i*v_e5c$jr2*#O!}{J9?Wc2kpu_$gu=L~! zovFV`v;S~dyXbqZCpThuDt&UBh}y0;+*)dD3Lfp!Rbb&a!U>)t>RvHcQ~$i6N|4Wv zHQH^o{n5^*y1US4OB9VlW4Qf|pqby(b}<@}6;*rlW*`wI!xAF>w!Z1M%V`&DIX#2g zJ=909${Y%lI$t57?#Wd-J~IDXGF?cXj)zQjb0aAu@p(d1til_cOGO%Jj?!?SiFoNt3~gvVrs0w`<8>+ic!0e zs(#fOCRV$3=~V2|Zt)F!)yL^MEswGcO3zt`Rm#?T3Yj8BUuJyW-}87fVH`NRZomDW zw&L!VFbM@GNAJXnK=UofgADSAzUPwo_vLwrbUyQ@YwLs^HJ=YX`t-|<aHZ`X5G|JI2xi?Fb*b(5X%q7|ZTx+bBwcG(`!F+uE=*Z7z{Wstf5 zmGf5q1JQ%`$X|H<}-T5(VRb%CdO&KIm+nBn}!@`?WtU>mLst7Qexp zYixJguXE+it(*GhH;&!YOT2V!eXXQOiyoqwuP)8WJ#2C&&6=|mevFTrjao(jkR>&P zBwjpI7{d(bCVXCCkGz+tc$@K-*x2fE6Nlr2L`6Z@Nh)@p^9|8%a~u~3s2|-UXn4@y zEC0=0J=5Ue$sD_K>tu_*#me*I*tJ*^0td6L6KFjRB)#tYBD9I|^AD%Pl9Sk*oSHB+LBWBTHcWIS9C$uN*kVTKI-TaQi z2^YI*OHUZKZ@Hxl!A(I+Gu9cjuF9MjC@V`3)rmVuuq`oGGkUE%ig0?v-Wysjjuzbf zUe;8F8_x1lc4PlZ$y48MYs9~}c7%MxX>Es6ybZ@%g(sR4K2cO-R(MXz_#9MihNuCcm8mwR-^JXGbz zvdukh*JsX6vj0C@*AcL;2cXA$JJpVFHXBeK@YichmY{&lF2UsVJUkJeFN6O&0_lyK zH$`|7<;lfe8a!gVAr-Y~DVH7c<)~4=_H76F{gcBl)LUoskSF6ZnVtR%vT0mHdK!ST zztK)qGLbpy;kS}Rv%|r@Z43X4pF>t)@x50LEQt>T)|?LS#q;x+s`bg5gJ>ksK~mTl zIS*;>BH5f8ikH5V)7|)=f9kV5#Ts4QQDk`T@Hsanwi`MK5G9^3WcX4~rDQIWOjAIW zA?Nge{Q3X1_vPVKuWj3rM8+nBNGc?Ag^Yy;MJmCN@gDEqq{=&gI$U>k*xc=da|K-^BHQ?TJ-B0v3@TP~m+R44qtC-;EAf#Pu>?H)M? z&trTMjwBnj-%GzSKqu7Xryot9>%{X08DGye6FSx!rVQ-Xi%-$jdI4AK9D(WE3JYh_ z|I?2{d#aSwdB43i9oD;&Ao?D0cFEv*3fO+y)q(P#e*70l=pQ>*^}LL5qW<1S{$po+ z_WpXl|Fcp5?>8z?B?L|4M+ze@DYGe7g>^h1o6b!YPTQx3o_z4-%|Kzn0Me$xtu{JN z!IP z%Xzw-{Q2f>^&up{!IqjfkzSVvB(XLm(jrs4So;Zr_XaFTy!Mt9L1}zhK82IM!0a^EODy+TglORTbt)-2uDSv$(_DzdPEWzG_Y_HF zV1>`1tH^Ljdb0wi?X)6`bO#;e=S0K{n(KRDCL-LA+@viRwBb6p4vqXSzqB%HG0MC^ zuOg(t)|`4-Mcgz88is}n zy9^R@f%aKfj=#0KF(0{UPQh33L8_6u7qD~;5Ogpw_|+m)r5?&rj%G+*feA$2d@Jxp z%Y}`yJ0HTfacUBF6v~WX^3! zYIC*Gvf3ClT1>J*8v7(e&i)ixPp+{uk*lBe$L4_M_R$t7tUdHaGdrw+!*S+7ciBhK zKgmIP3Ki-9*jk0Zs#qN}Wx-b}-%A#LVxKmtg(FF{2ukHYw1%7qiAA?8aC z&3SL}mA2R0@a1v>MwD z!_l!jC1;3q2mHN8*ZkWTK{j;Idt`0~U1ThbN*>eYS1zXVR0sgW;qhz-$H21})}ioH zA{RC*CYAQXdx$*v{Yr0m+D4&UZhB{{2YYwx*wULL+vk9UJynN@-H+MRiaz^9sj$@L zPvs@jENZ>F5$^+X>8*|ugD+(4IBD$#WESkP_=V@e%=Nf)fnUj^EECia@lB8+50^n;-loy$V8%(1N(Z@};anIWPvUFfQhHh*ESww^ z6Me{@XOlNMQaQcZ><^58vJHPbOz^i;d#c2F!&5AJ7kYZGpP#;xC7{xmb^4adgLiA0 z_uk%A)59aFTU6O zs{;0dn$EHzEfOR0-eTGyTX^YcW*VPvj@iA=@& zTLpqgg_Mm!b3=0BytNPA`IA@2*nLDw`3&L*1Y|P%FLA3~2u6xG)-(?eOzeLwTT|-5 z*;5U?Y+4UoBSXSYHmwUNav|mvqx{XkP(6ah_kqQS0P6q>uqbKa;lG2CJ_rvF207qQ z26^soFO1S2Z@rHYUP$#0$>qw&w-TlK3FJdcPhz#km0jWpAB=cB#>Lt%6D1ko0NUjFfTMk(3Tk2nh$u841kiiX#m)vsW|4Y^ z)oF%=Z(xI#kPQmE`6|5lIT9ujRhodV~qm8;0vv>;L-y*Nm-e_NRy+<1?98buxh-zThfb+mfCWb0P|TezvT)c zf)(J|Ofeu{8y|#t^cGNoda+pqC5{{col0x=ub`MVQRe#j^(n&+ri7tav5y;Ia2!v1 z^kCP_1)5IyoXL<{poIziYn#m7fQY7bSZ-vT1hoDhNRY-eF-s$RXqrh zOYW-$!`y=`JgEJBU&2~x?$xJo4uq}cX>Oq@iT8Ozvm17EyORnmV890V9DDzxZuqYz z%KvhEm+W+r)*y;=gn@x83>uITe8$;3>>x_)@j4*XqwUG_qwxL1vo+AvfJNduSiL$M zds)~!VZpys8RZJ+{gu~F1z4qS_b%hak82>2l*MFK0Qb~;sM^|c8AD& zWWy-E7R;QRzA2$yPGCco~+HV+SFPZlVS!W68<( zl^5PM|81UsnG*J3c9FPeXc}K=xO|YTblQ8sD<>d2am?X}wEI%k&m!ee(0$j1YUVBP zbDMbGr8|795n0L!C*QRhD<)+;$i+_f~-cO z-1ET8rbi{gU=@q1gH~bPMj7>tOY=L3@hHeNw)xH8r1t;v@M6f$ftEiRkA-=&qQ8`YJLAYK`qK z#OK}^!3`DB`oH-u^$=)js)toPPF zgQzcYI=AqPJa?|I{e%uC!sbWK{zystynKjjdIoF!?L+R})zp1;(^R*U+*uGsH_C(! z7C;-^cM&9&(7xys)`}8rjkd_k%Z@0n>2BC!@%69X{^7x$@(461KXjvj92#Nx`U@nt!L9^v*aziKV-1@1~?SfCH!S75J;if38@Kp`^)^) z)?C1%_ckEE7|j4Su;S5?pbZo(=7tMxrywVOxO)Xe5PJie&(Q{}6Ng;O$pZD`_6DiV z^`4~Do%i-Su0z4FRstgiiM%ymkSqCnETkow(2rRZDDkuv1i=)FH|dL4T%Wh;)xW;k ztsA6nPQDiJD2L)v_RM4c>DR#Z%NZ_nE}JYlFPYklDHW^UpwgbIcK+`erIB$ z3-<1pI`QPfv)zLqTQdV1E18U$r+x^L$RoEB^Xm=VdRz;_|8z*yFk)NNJ_+@ZXqEuf z%xZiR0wR{LGVfCDhPXABgo{_}52c+bcbBpRXl+%;aqVm5`tperHHYp@zF=Di@mMbn zq2+s)=ibPmTgv~ZlULPE$a{XeQbSon0tuI?lV4t6D(}1W{*-M#FS0IT(6Wj3yTCF! zAR!!N2{?u+vTpgHqDorfO|8pnGq^h&xV*VAP%IC+yjmW9_(VDd6u6vjNSLq?XUkj* zvXK_USE4>fVS|=X#7-CWe%JOy_MKXL&J*W`zHD;Uj#=HflG(4>-sk`+9p zj^+St`Re(JgquOlKJUb5cY4%;*RzW|>AMtPX3O=yQL4xc+C#n4_JIDJ!PoqP2T-%< ztQ#L~{69hZpVsnknI{K;!Sdl+wA;$CLdN1!n+F(~=z*Z2o7hXWbQjHqdmg!njlT6K zgt$JEYxjeAK8|Fp0AqL}!fx&o;-K2dEGezl&hW6WY1mEh>w=u)_P10TSW2sh%UG*aU5yS0#~=0T zb^Z5jS5GKfx)|w^EaR3z%h3&Q=PZ2(Wee#9t0;o3V%m|ok7%hU34x>syY%Kw=F6t? zq1*CudzLC<@&@UNxO5v;V**q%mdEG1ziqT3-V$;kCSnsGTy9K^z!+7mE~##@Q$N&J zhkF-Otf=nzAzj+4ya>o!3zruX5>v)xY&qB6#Uq4VGF2fqAGfKlQ|WC}u{HtLqJ<|< zRxoM?3$GP)Sho&3P=!UA=aA4qCT>DsYFw`&Y!u~}*#MZY%^}HnOizx9q_K^?{~@K* z;5Ebee@jJfo1CcOdWZsV9Kn?KBWF}`UbJApi`%}dqSuvtS{+*`m6(WK+^5RuuI&7wpx{#s1laU4ji zkAWP<7>Wm`XPN6{1yU@qFc_g+w$LhTE<(g4I@B;FH+l}rWEYVwRUmu^%05hUUK%nI zLJ`8F3?bX@&Yc^J(r=8|*68X#<+K0PIsa9u{Qp~>vs365YVq%ZV%d*d>J~S%o6b=z zUdiPoBXOgE>tFB=0dca!5UQThuRK6Y2KG#baWzsapwYE3eD?}?@)6{1vP*Yy=Rkog zEyLdr1lmX;;-WcseT15_)wTbMm=LD&V|Vsgmjp=0PoW_8wz|bDLahXuHH2DU_n3ti zp*(sG>rY6b72~{vj&q3d-Y#cRIh57YpiI1rLwd7SlKTi7|2}{!l*~ze2odiU z*~t3}Z*C>#Hro9Vnxb)BaiKq6ri6!5QR!qxqa>7svdkb{QX4OQL_$mss}s~6{DN05H1@&L^j>H?0UYLHP7aG++g? zu{OgO?LqD=kE??g0Vi8zQ#EQ@M~Iu^o;< z(*f`hGc62!xau(HFS+hsDXMjS#Sc}lhu_h%fdk0vz8~X#&d;q8J}e{BUJQyiQhGNW z7!$c7;ni#FA+4dBgX&&!4xRqGzAbHZhcIVp#FNVI83W{zEfR#1Z%!^$SbeM+cor0k_Nz*JKy4MX}5pS_LV_wS@ z*W$~C=~fh*9Q6La7~bjrTjsaXC(zTVPiBrV0S;1hz0Sb(Iu&JhKa9xP2^Q64^iDb=`<9;TTn@J>O&bd z1r4qd<)#IFDw-!_*Ls(Wh7~1O?@8%pm<0ON1lvJ4C@eBnu+(KZ_1%TWQVQ4Ec;xz3 zQ{tC8eqLDAbXeHjL#+YhAf}kUt+<7Xn#E38%z zyB=NPTr_(~x(>gviOo9dPD^rz?L~!Gimpa==EOt0?l@iJ&v?W4e9Bk<{M75YrbVF`E1vn!DZ1R+?ddy@ z7aKz*I$4*cZP9epOz-Yel@I8ws^|@-;GwlHS(irBA#$#U^1H?AXNot05`(RQcl1J3 z^a7S_K#DRs&{b@94uU#+^*|chkwA2IsnOXrhYOu(rUy&^0{-r;-{yhyt7;u$C!LJ~ z<5&%uA?4KIH|wAMcJ<8Y+o-E{xcEim3(O|6JoC~ix^ClLn;X-cYvR3|8_Vr5Md^7L zWG?+xH$#LozeU;@;->pLxP5 zxB@b(_Vh>0nOE({Ze*s~Pa0C;7Aw4Z1)r)4ZZ}(U>$Pmn?mXwc!59^_1}XRw4pQsJ z)}e_qs8sHDobVBw`k-#a*0dii12Yq{3?pF#%zU_*5zONjSCvd&tyBO|TkGkLyW5Q! z0`GCS8Hg@a*PL$<4WTzHbG3vJU#hKEFR5)=zwhjN@OhuJ^_SdCy)*BLQ~OumW~_Tq zpgZbHuQAiE28Z4-Bs*bpN%cTd2CeT#Sd$VS$SY0LD9mOWtXE1kR2?&4zdxMjD$N29 z#`3Cv33}{(bmv4(gpPeHH+%a8OS#~<5W~9azNgUJ5i0u8Lof}1WvtxjaaggNSAAgm zgI#%u-fmq_@r$o4o9@G2Lwmh0EOdb zM~jWbKCi8(sQYyZOyrraVwgLW)$%+_W%ypyX*(9dmC7$lZezEGYZ-W5_2Ely|U*p2C3M6=XM19%0 zO6wj+!)QaToEFLu*;(e^4fUz8emU;I1?X%W|5)utM|lVG!kw*$DR%Vp=8YTT_mUD5Iy zGwKRyGy~$pQref&y^sLeu+jrgU_rHqBo77s{)Dwr^skl8yV;CGsjlYW|qXXt=4?Lqln97z^hwfhlY6bCj?my&xAf7 z6Xb|FyLzAP0?)*Kfdt^m7z7;{!sdU&V}wom%g@dsL`ID+qI@`Y-299Ka{!yzAzIkT zo&54Ez;h>wf3H_lA8b(9(D~RxsOk3e%xBSb@+~d_w_qlG09`|qOD95H&ik^cr*^<5 zo|tgYhii&mnH|87eT6}EPo-{q`i?jsKdG!Qkci*zdn=bk)byT&qdl}=7B9A`F1E&I zu9+tWxNb868isXZ*@U|^p;%vC%wNk?Ufw86Oq@(OO_>`OkKUr1jfwoPRWK*7bkik{>%;#`yI{x{)A%9By5hHs&zQRHbo2w{OeDavqlQlWxDlb z7Ey1(Lfm8C4<*D=_sfP6FQ~}co@1cO?aB(V^D(gf#}3Ik@-&EshF#|G&(0o?X?~ zXs1eFdr7~h0|l;Sf!>pNFB0c;gxB69U?qPY1iO%TbcyV8-;KLr{X+3&>BBGpk{$X= z0gh})o!1$r*}qoSw%%O&OzTUBf-a?u<2_R2;%T6YP|1x)hQg1PR(z?Cys>XPR?;%dZQH1l+PL<$%%5%(sUQ{lKwC!Ew*DRT zYDz(wG$9)MY%;3YB+{$@VK+bAf z%_hm}T><1V)xIq{&4)DF9ov@*)$!s)-5X!m(@w;y;(wuAuAlA( zVvC_lNY@ug1=Q+dV^($Gp&%4pz@^WyWWs@K0LodSrs0rLvbMsL5)f4st{VMs?dbt$ zMYY8m`l zA#}a)?+QBA&wR1L64trC4hu27^Cgy`*5_JBSuYv!ajwP-%WM06ZZ_qWqh9&a>=#Nu zIw>GXi`gd$RXF>5(ec^b8Z79n0ln6lL zUtiA#GV}gx$zi7g&kP0JFks!7aUUO`_x3;eqnRN|R2?D|i%!Tz(j7$1M&5IUY3mg% zt)74`RY_&{f-IppwqiaZ#rB~^HZ!sWgvT$A*Jh})w9>7`tVUQvhe)CDG6J_7n>((g zLvh7MrfBWU=!%IF>4Z~BH_2;9cPFmEZ7i*Fch+%-SbG^4&$82$9Ut?yZM)WTJC#L1 zTeeQ+{;3HgHrZte``Z!ug39zUj<<0PRG0H$Eb3|II^0VIRkJ*fK`II0N#&u*Q@plh z=E%#6eRtYEr#RcYJ+Bftq^7D95xRglt>Oqnyb{f>%5~>hDXZqbF9PlRn!jDYpB{@t!jErzEE60Ja>@zFyNZYR!&=rtk*+$$<<6M_B$N!g6WZ(jb~{T zLaLl;#9mgES$3+fuL(#>Zz+8GNdic)&~2U5N(<)mw;Wh=Q6*@Z@0!r`@O#6fE=|{;2p;~83_^X4&cgW-rcXUmpl@-@fRj_jj zc!F?^rbMP&lRFbt^q-M9ixMip6nD2&fR$q}6RTV_fz;mOH7Wfuo^orLkebe~P3GG8 z3hj;np+2|(yYuc_Wm&Qt041J;4KpXZXGoNb@Y*RDjfRPD0S@*swdf#ioZRFk+L0Y# z=S)?Jw(z&!3TB*@o_)`1H*;24w{{Gdt(c~AR>m=vY8b?skNAJ z_scxcbP@OjaZW~eH+7wbFg!IB1QWkrX@k*JWe9K{y&1qrrHZ~Ql=`Rd`a_Gyzb!N2 ze=Y{CF3b)Cv0-B#`<4Ll#x_++LizAoDvJT72a-TM1oWl-CDp4JH(x;u6XE}Zu7TS& z-f(Zlvj!=lz2@Tgdf{*8e84H{yUh%GF$f0g0bGZ%9@K%GG3zU{*F>RX2|f&l%_Ex5 zKmW)GGh{G>pM=Sz?*{L&MmQ;qgwe1b)qNYGA7K-{>H}nM9vIL2Uw#1T-V!B=U~*gN z`q7fV3sX}SBUFj8!dvB}3DPb_h)DND5FXrERAas-kO-RSSjD#LKW{C1hL4xv8A9LC!m7K`I|rw&{o|lK zu>}jsc>Eaq$9_fNRitZe0ugf3yS?k>mb0S9OAFc}wfXitro3zAV5%>YLDI`4{K{61K0?48(>f;|C2`No0lb2G z@-0KoO!2q-kI8x-Ye3$#YC*2ju;MMa+VVF#6MtS?^tPOCaAtr?LIHY5WhD2)vUbp+ znPeZWqbVEbUS!d-*0V{QtC1S~mkT_!2NsxD(-dvc(-Kq)rJ3E${gUoRfS-y+8_4wj z#kM-n!G+ASoFFWH{gtDCeOb`9EbtOhwib4u4xs$2BTV?j|Dhj)XS2MuZQJ(H tOR{H`|5v`?e_ra}>_q=B+wc{~?O&DLEipR_x4}P`

F7&fR_Te*g-*-9G>T literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-2-extractcontent.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-2-extractcontent.png new file mode 100644 index 0000000000000000000000000000000000000000..6064df3455b5468f913b67b35933f4f0202b047f GIT binary patch literal 130427 zcmZ^~1zc3!7dDCrQi_xy(%m3PNOz8ONTY->q;!K|AR^s0AStbM4j>>U-Hn8Fci%n8 z`~Kg1@AvWh4Rhw4oolbX_OqU~PN0gC>^;m!m`F%S_vGcI)R2%++`wl6{SLT8q*dXG zgmhO7Dk-TVFDXf>;$Zg*YHfyuBo`R{98Fujoe-uM5hiAF3nk;-G)nj*F$HwCH~Z|& zR2Vo4pZ!veWeFeXV=)tc`;tffDfyK+u3(VyRuJj~GXvJqXg;RYHTZ7b3A}1=#<%-? z=~va0qv=~>8&73vJEVthwc!(fH1WS!k#A>m>Q@&-ITa&WzJ-R(XVm=p6DHcXyJfx{ zTZOl!#tpy6T3(b~)f3($7fr!J!b7ze;7CbC+Yv|Fwk(v>M?zBVNK|CQk>|`8gWSFS z&g~0n*+%RL46`lr9VQ z$Ui7~-#nFzBf%}49e9EwZvF8$rg%06=BPc+nnZ0_3dO^`Z)3DD()8{3yrv&@o}v22 z(8QcXpH_3U2bweQ8=3EnVR}C+V`O2+HO6fI^WOg*`bSMFm$sh3d+MW~**c7+?~Z+> zQTm|NWcoEz+p@gJp9b@IaG7m3+KUcYg_9afvRoP4|2Z6r8^LJ^^XR-SSAll zcHSXZQZAmlefUbrtVGUW7}bRl9MSrg}C2>-_T60{xmI=EM{l1E;J^^&Q>K9yt@jmaVW(uDmr zH^kpQ?AasRHeen5iQ!;Ui<8RT!GF20 z^Ex@GQR6ZGuiKfaq^!5|8U()KTHg|Eu>D5C?N2j{^7J128_4XV%G=MC@4MYPZ4}%l z7RSMpkhb_pZv0uB(&Z!m4>7MWFc8H8g2+%ybf zj&ei}SnhB3sf)su89(Hv>VBoN3=EMWX2icIIp2$6K`(&HCIRWCTYgk}XX-s~Dnl>U zGEXUqw;%QkB=w(8V0h#vFIZXeAdOs)IeAG~nwY;o$|tw->p8(34nBBEVNb2pv{EUo z-$>;q)ZLOjTe8z~EZ~JU{ib66`8JxH=w+}N-9wypbQlURRbZHWoB|`A23|QXHm*`* z-JqnKEG?e%M=dGWN}|ZXv1a1tmga+I;pX{f_Bon4q(G^Dnz8U&`EXk7oHrT@x{z^c zOF9GkgD??BCKh5AekN3=Oh%08PtnZLF3e==)%p7xD^GNit;G*%Wh2}dRu(cB3Kj@= zr5)~{Jdu8DEWMM}tXj;p$7scP5Tzbf6UEoj(UH|b-qF{Ayjay?6?Md%qUn=*t}3nG zqV+yAB9~fKC#~3Ev9g0LnoomyC}LGs@e83!tcq0*L(WkSg$6ViJJ(thwn{q0JalKM zGnZqi&63daSy`Z^v}Lp<+8}%GK~Cfl+mOyG>A~2l=4#g}-yj;d#Um4LYMxzgLaX(` zr`bEtA~T>l#9!M7So+sNh}f`f5ERG;eKpH$8SWTn9vLqf${8vu z$k|9sP8C+HQ8@eRQh}XilG8NYKBSXdVzgAw(Qd`a$-pU9V{744;}+#)#ZzTXFcA>@=^~J?neM zcV9iU=3IXK@G&>nU0wp>LLMRLsCkCTf_aopY5#JZY7uAGXsh;%+WyL+V}66g?u@1W zk=}`w3;H|sRP=a}UG&NHDT)Qp<`sh!kK(4Eq{ngo;`*hPP!@lXNRk-(Bn4_{?la+U z-T1AoSlh<_=UiQ8v{~9Ba;HqExmLB7bslw|KwizThLzD8(=gBQbDNL6cN0ky4|rp( zRrQ`vuud>f1i5@oPE2mFtuq*5{J|Ki`a_i`-F7r@^vMR+#@af?)8(h{1PKJW1!_rc zLx&fe>qRCx|x6e6W7bxVxs%A)}`2aPC%32z&T3*jlaj<*1>1dcipe4YIHTiRl;#dO5XQ(wLCnyqd)6VDQl7k3uFXe?}e z;vX7-Bk@BbCSdXfNpaP>u8Z$i&*l8y!^hFzM!z|~yx(Ng)bg8czSsWCyw>L$l9#wt zpDjM4gs-)oTS_LugM`ORj%R><50 z-XNYDz9s%N!yQ^JS{H_3rG#iFDjiXsu*oP11$nu#7_P@6ug`OLgmVgWGIK^4v=!rc ze@{wX1XkgPDybzl$2(C5hsDH|gVkV&P;VlZGtYjWedMS!IW$QmBs#5N$T5>>)aaY= zJ6Sb+hjl>EQTml#jGdO5R>~VXZ3ee!vWPm)GjAo+?!%vF6cLjH-dzXab%aCv0 z#eECV$Qmh^G&R*Df*z*4to@*-=84_#bBq&3RNZY^xTG=LiS%x2Z}zv`?WVzZIcQnX zXZT8Makhfqef!zc;?nh94d$=SOS`z^`R)F^)%m59XiifyU|!M1nW)Z{ZEm|gNw&wh z=kVvka9vwp_1wj0?mPysDWZxl)Cc1-vrwZxe^&!dnyvW>FM zvNFMYF6;%8YqZXFHEON~<7D4za;D)vd*sQ&i=K^!?bAzV{f7oK6y^0BNb_PB;z4hR z0xSYL9)*2axpxt)6FjBZu4vHprOSrRkSzdPkaXIoc|mSSbt4@Tsu~haN@C4sN%oXo zVA_>>uQ|P3-o{mlKDlYfh zd4+>rUA2MR*DlTOS@s^?3E$cImEEkYEUJ7Vaa#N*uWKa6?&ZuT_Mdy$ zpPm|rtUpJ3`=F~0Y4*Sc>1#Vu@9fEEE<>No$OlqBucE28Q@)*lW@NEN8f>~cS{6BW zb-ewiPVe^))@oiRGLAGVW%upW?(Qb%<$QX<=L%T3u-m-SUWz>sw5zPPo+Q!c3rDPs zT=vbMMAkLqXfPYe_`N*sPp&xcEkrHH1G=#eB4rz|0#{)mU{c&>jm%(NnBl0ULJg_ zn>d)6**aR-Ir-e3ivl+=Ud!n?A|VmdAwIX{)o8at`=d||Z6|HTX96a6Hta7=?TpRX z-E3YX+CdU>69AVsW==0D-E6FF9R=Kksjo{2fNR8U4rBeU3Nb|Rmf7+2Ub2M>)zIKAz*-|3feQ9jx>?BN0jp*p#zrW)&bA$e` zCtJsxX@LoHAf9k=vOnhdw{1{W2ys_H1?pyIts@1s0Wt%9h;VZ8a0y*k`2RinUzh)> zs{Nm;oZQ_1t@@uw|KF;bj%E&$b~d0>Cz1ax*iGgCKD?fvMOL9Nug588sHnK8R8Gw0sLe9`wgyd-D0ee)iy;!5<`-g64!9MwVsSt zNg!SQ!50TtHSJTi+~c*}xRQ^B&@Mv**CO`^w~f2`mqQ@l*Ze`H`T4 zD=vNu_0Qq+nxP7?r}>{aTK;+Fw&spKrEaiO0D~2V)?Zr22KwVtBHcn4Lqf)RgM{** zOUoEBK@?a5ncySr|I=gATPWnVD1YmTg`@afo|q>-ekoyDh)pJ-y)|8@UWAMBpL%`{ zxMH(RL^>Vs5uNx|)4gd>uaJba{@bBg7oON`&e)gq-`C^l)k48>8~-G9Ao0I}El|H{ zhzz>B^LL&bD8+d9LI^}jLu?O>q>rQkLo#fHT)=>R3L9q)&Co|tO&9B=~ zp+`m+GDC)Z|IgIv@WD8V$%wD(q^e?2Zga1;4cPrFAeB1MZLX`e1@X18G?;Foz?jJ% zCZPU1*++3m$jZ0xix>o4kD?!pDe7mi{(t&dqyYmHDUZ5)UB_7g$uC<8Eyh;-pQ-vi zq6Y<<-pUUA9fb{ITK8NnuVweQpEgCx%7(rAtFwB08`o*~Y%?z6321h*zJ86%#(Ie0 zrd)8MuxI&3?Or##!P())Mv?DDURMH7r^}cb5fNW2spIHI#fQFm~I`s-^NoJ2*d&632z-Mp-Om zq{`8}W+f|eJv+sxxbZDoO{M#v_*UE6ou7l1IqKQt)1F%!{au1vwJuHf_(r3c)U8vz z5AE#@s-3>C71dAsobSj^xXySR=#LZ{Y*eomLW0kJ2T)koUmQBFR6DP|8Zq=NP=pDM zeYi_JH&^YtWeWzWXgPT_<7+1wfbG2EakvhBZr~i9*`k>Av=95~EK;m3{Nzu`%cc)w zpj~q`a<|l=MBmGkjSiNgXTdtj(Uh@lnd1@uN-z#Y_DS^di8f)kQwVXWOncw)v4fZn`fz*>drRul{=w2mC3|Ciwr2CRC_DQ_nP9di@6$M`>wgkLcB8h`=gi zY@u?Bhzsar)I7!el`MsKY0%n4MNu*sY>xfaJ<9)u<>M|F|P+dtEqy3{>Xz$lx<$clfUL}ze(^%V_%$VWiX50v4-PHf*agtBlVYN*)WgXAiY=dW|Qu^lV@A?Tm3A*i#yrZI^4JF zj`R&aoo(3pu8nVR)@+RvnKCQA`lN}N;tSxXt1ivB zvf8?npyu4BRi>UvFwf-0pD>Fn0%=bQ#71wWpB{DFwQr+iu;PDqendd8Vg3`4N|9a!+xEcl5>Wqe;)hG4m-$ z6?+of$}gTGSuMhQzn*MXIr?9j;`jxbzsc*p&9kYyzz7dzQH3sw95VyOnb4 zax~?d=)=G}W{Q8b?z`L0Fy&~p;x&n{EcDxlf}jHI%j%in(#lo}-^q(<#;I3{Hh+8q zE_XtGD`)6hE`W-;)<5^XI-8l=ik;(~@p3$>0hS=NYKFhX=6eg_<`D>!yxJ4JGVwhfNLjx;+j32MP;qfMF+p*4 zEL*b$KU;@+ZmLEh_E`CRh~RkL(e&CS!BK$ddE-T93scH23uDTz6{%*Zf5&rS%YgCp zRF|*9)o6pZs8hFBA-YM^zRlv|Theo>u4FrEy@Fv6vszXzWX~V$8`55ST=hL=TCS6u z&p4t8&i0=U;)Z#Q^i{GflgS-PAgVf1ARZrugRA=Mt3+^R#EiI?IqmFp#f&KFgebZ1 ztA}%|q`4(SRC@9z}gMVbi_@PB1S9g`Z4y z)7bmx%^rFuz~~=po1rQCprFqG80l(`3v0h{b4f+t*gXE z{NMaxdkMF}xcDUZ%uz3FEiUtC2)pVY>?fwX=9WIaV24f@rp~Z*O&pwMp49pKjBwYF z*rhljphv2W$=qoH1GD){zpeRIM@y%2$9_u5tN-$lI$f%Np+ z)s^C)*4>uFk`L+}R@VQZb*@h6*IK__o@QWrgGe;kZn3)Mx>#5|{1-uf6}ujlp!l$P-ZdQpgJc2$yW3~{s-z<2qHJKxs`T5 zTkAx>Ddd4bc11Lh`v1YlMT~d-s{U9)U!DaqhekcRfqd@~sPi*^&cEf5P;}{W6lBF` zq47LSH+K~Qyy9+bMEe)xA^B1`8vD+c9rTs`KMbR1n!eT3wiZFX)x1* zv0<=pzta`edjAWG_qUA(Fp1wX|3T;I(v(pl{tft+QjBQVf_?M>2)5}i58403X2t*n z+rwV7ZVXOM&#_x7w{GT9fB=&Z+*kis^gtFghd1e;eW@})bA#yB!7?biZ)1Sw#WFE! zkET6&q5T=PLjW@*mH&KXwVKG{DU{cKPAbtH=AJJbN)DB0CXX#rtkN#E8tF#hg9&g{CR3osM&++d7`mr+mzAfJ?&$EG)OH(pI7i))tNeTe_616RI7w#xT zGANe=@L9c;4EGHtkXT`CM1`qXwLXGAUSYWw+#n$U`s0r{bg^!s4^rIXu(SpTQ#m;B zvt3lKR%&-USh5W}iB14Mn4pW*H-3BPUh&1zjHprnCxgzBENj9N^PcjZ`PK;l+Dwb; zFRPt5#)p)I4>ZoY?0nR}9}YW8pZrW4_31QTT8LCGUm3`r;GM8GK|r07>eKa-o^=3= z%Q)c2KX}J2XeI#s`@~kQxe_Y0EBE{;`ne%=uSe83uY;L>PmyoRR9R@}-P!cY*@zq! zLQw|fT<>{bueTp1khK4c3>?HV)8TKY{Zy4DGs`QiwrpR_FuTj|r_53xRq18Z(K-h} z3<47CEOjSulnyB6Et}Rw`3y_Iyi8P|on=3ZHwF-))*GM+1fU#ybL)13=@q~o>mO-C z9_y6Jp_~FxH?iQsFAChBNL!CFO_uF-fYx2WGG$gIXy51=84lPlZ^bjPfSxJhw=r0n zx~{*yFY;JNinVgqzl*-F*nybOHVI#ZQ2WI#&#Q~0sU_exj4#|an{DtLE-_bTpa!mw zCpbo}5srrQuudy?-}B2MExm&Fg)@MyOO-`026g&=@y&P@c3)ncne(w+o&(%BF=1D4 zW8{4hy;qmEtC8)``gFHU9;$1+GyC@bIGO9@?^8-~R>5DuWjdHIsF>zD+0t?9lQu|P zY+_{l`1UlL7xHd-AUm|6Y*<$@23IleapSbto}xKV>7+v^50gyi%(UD5Jw1yr?~__S zv7bI=qti1EqpRD zb8*Ulr`_j-bI1XXfRn*>9ao1Fb_Nq8dUnDLAvx<3KP6V0_W;axI7&KS`I^F91Bqp? zCJhLuksTj;ZnQDZLB+t<_DR-P&!#LI3QNBGK4b!{*t&LM<=m(41PI(ml24$Iw`;6cz(BC>nH+_2GlN%#7JjrStQIE_Onv$W%EZ| zC3H>i>?DUzHs2cWMCXnN+o6-uRiY}lpsV2C7@A;&p}}>_MJV335i(ogBYBmCr{s>m zII}V#Ly=tPyG_`FzfMH>cbf61P@Z&|htf9`r-p^zRV=P?Ui(AelJJ@N9DvVK z;H^$CBrc9y6*)(Gwm_W1-lWb03~2)lL`m9wD#oe%QnERoR>T+Jf0hGmT}@|<7X?=I zMAw+g-lQ#z>adfezVcB1*S7fFFmN({zk5WHo!%TKiOrueR?LpUY5c{6OO6yDB1-Ig zF!Y?v3d0nQ^tBjQ&R%i`FN8ncqdA{I~} z7<`yt^(Vo8zl(3iI#l$sagWb0vJM=42tEs;Xyc)X674&SUFLe8 z5hLHal4=0+g`X+CGV}i3f)ve`_aT^XdQ^vae{*UAg9_mm3tye@vCouG*se*TL}z;V z0zX~w_d9&!pa;?RLz>UyS?Ov0*8w(HLU99IbsgcAShWI+VH?C+EH}-YlO&;S!4%ev zeKMy5T*as^TA3uLetD(Ilf9Iy^=7mae|O-mC~Aook5&S&D>*Zuf2~4U>)tL2&3A=i zw&Gd9BABlud=D*^r4(Oyi5rvVtg3Kv+n7TqTk=<6po`-2_f*be+%!_rdV999 zyO?g~;>#}WyEcDFcR1gale+m3x8)!s18KYQAK)_c>NqoftYd9puicgy;ULm!5N!!b zBy!UzMak;oo%j`8fYJfHl?fgB_V$u-I@uoMOiyJ(GrT+k4pkMAf{qrgqi`pbGEIpf zxNEUTf81KpB$eXijfzRD|H-Qii0|S*|$gSD(#9mul!Fx zIl{O)%g%yhquXK{_XM+D3T~EgvEsocH7po!A+fP5l?I{56XfbM6~WhuHKpF6jOWH@ zE3#jqzLh2K#eK_hr>{8rnW^)@f-WtAsfM}fMtGl{6xSr+gV<9-M5H?3=RQo9+0F^7UAa_Ge|KTx#I_c{|XSl#P*m9-`ORD40lYZ{=B z@5XUQYM69PXV)jhtD&-*+A8kp6gaci=lbbG`gr?@FE}rZ8!GdcxaT^kDHIc--Sx6~0 zT|c$+xE{CtyT1T^I%;7?6ekA#YWeFfGIQ~IK8vviW!zZ~T&!38y4_2X z@bA4m8@gHcBjx)n1*H*sgmkifVnx}DX@XeGlJ-oDeBdTb^M7J3pv1745W2Un zbNT$WF)b%yRzw93B?Sw-D^)RO(h?y}VH=^c;niB~t(^$(0Xeju>FEv0y&4Uas4j%{ z!XF96^1T>UnvAXyZLJ%Xm~oI)?0T3L@=U}-N?(ET2l8+&+NW*yMdb{!9Qca#k;tK5 zNfs6K{5mm`^Dz=YU5uz$2cHMt##30RmvPR!mzQ7siOhSyFK1}0{_4_}fi93_bpZGV zbEo(^IvH8YGUd??_&Q?E-T8Gii>5W_RX-VY6&{?;w$qJ2VA=la6A@k^6Q zl-GebdGfo0NPKBxGzI=lxmT-ktKysnqhB_jH4EML0={eQ8+Ew^FO#8HBNB19=sJ7H zCI9iEU>M(|@?f&Da6omJiBTpH)MU~@J6}7LiQ-vs?eeE(astCxqc{9uV|0+!y^!wd zNz9&wjwbCJQ)-sjUx1t)$@pcVXE`#wfX7yE%4`#;9r^3(A}mj_dJ-=PL6RQ01FkqQ z)+8rb7Y{Gc06J9VG)KVC5mH3qHtWZ(Oa=LLUYu*}y`90z9xEMw6` zq^&4ooSa|kP?bOQB^fvf!=)8dCn0=d# za%}2{*d2a90nI-!IQhJeE5e&D;?jLn1N}F5e8Qnz1TuR!xH}O1y31c zvC?~w7C4B@Mt z*weTUrQXh7OyjEU2lG>|Gr{xcfpdeeIJ8$vPLuvfji1pfeBy`0O_MlkwEbH5!Xg)9 zB;cHy)5a(;3t3iR^)5QG@hmj7N(FVi+GmKyl#=FA`T|`aZ=lA331>8@+m=V;xTTqe zcb#u$1Pc0)_DjLqI$DjeT?Q|3sVTSLrOFXsd&a#g!0~uwS-HZBLaFi|%uajTO7`X;z+SB@(@S!2Tw@h&yA7E;=Y=0osE2q;eYJLGER?_lT^pNB`9 z@9cjWwP%r@-Kj}ifh&D*C^`6Hu7(a7rkK;8dsgbjeA&scd3;(dO&+YboO(tng}(V# z){5A!YB7da&q1c;hD30{Z$MM7!cZ4MD9JfW?F=!0|HfE}d3W=!XBv)8xNV=3GJQQ4 z@G%|iDSXer2W<@(BuaPsKJ(EgNDC}?gp;!EaPsRKfB=GZNf&SDS`i(d+%pl)pTQ*N zQ#G!x*nTvAEolW625>pKZ|{i=N2yhthQ9ccd#-aXHGuQHuuu{3$)VDaM18?U2H8a> zC93C%PHc(qS0hTy`5$SCoFs-1O!EJUlWd<)b(%}#?Znv5%`3F*+htig*MXY=k5G2v zWdPaSzChKiKNf58XM3Nw!ypP1^zLp@`DM9WhC_aYQr@Bwqb}?fBCc~Vtk;tn%<8GH zTgI}Y!jJbvUo^rVK4+ZF%927#7W#;%gS#BWe&8gLa{1rnQePiIOm$2V9`7`#CO=ff zZ8un&b|LQkevb7CU&I#KNtf!ClaZ8vqF}By>d9Il3Im_k#o3XohzB>jr8({D)7D=B z6h4(TDVLE=`Cz?T(3DMu|NPmvjNHUJ3GF`;`;uf`7s~p*zmg3Q{DnRw)4!{H{j zsQmm#BCfj3HG^Jk(Q8Ek*NhX~kMxFZIzTWwhO#j}2=gY<5cG(x?75%_bSd8g%Z|?- z_*;>Qrr}Mq3@Kjy7&zgf@yEIgL@m@13g6;gFN^l}V)}(~14!Bjf0PFs=d|Ip zx*z6UoYXucBp{~33sNw<`itSk@P}za3G40O?YT0_dE7T5JlD$tgb3%j5K#3{R>2C|PQV3*)Ov1zb07ms49g ziwL)zv*!|_5>;azITcA7K|wBu{hMm@+s9f>J2CwZ(MMM z0UmV3Tiot?XsO6xftZovng3xs--x|MaHr-)b%(`?w$(Y}AN*{uMdg3*HvOHUF5%mK+vAs6v%*%OYu2v2QD7@@jqxO2B4)W zY&Nq0ajg#+{>Pu%r)m5ZGY%sHoBO58-AoJ7VlOq2A11PL&{uhcfPn}s6ev>~(xWa; z53CaWl>Z$fLO9~7mZ&fgsGa~mOkDby)Xj2PrhsNQLaBri4gPIqNaBV`)kW&VzG1kk z01Xlq#Jc$JvfM(CQUJmGCuSW4YyFv!oDHgfT=G?fOMZdVC-%20XjbLPwS=y@NXu8W z7L3;b=HA>Zcrd!lalV8F)xJ0~1DJh0{{6|?fZ5srwatI*HS zX0E!O1KqAWvXDeZ2(4*6=buarKQ+WwM#SmY#>=e!&aameg?8I(6na%#C%l+j8#^DA zeJ@yf4@RI1!^ALnr5BwICm7!6xqMy7 z-p4S!8JnN$>|n1~34zzHpTqTiS<8h|dME}uI7(;UPtPc^?mk%Lr{79cC|5r6%^~%6 z_42&<-n)Bzd9ifzwqbsYG~};H{W|{cAPw7JUjex;hw~a)*|af|fcqS!p!aHJWPy*Z zz9?&SH;Mzw_IuAfHoND2`S*ejmdxOjQ$4K3oD0Lxs-AnO!4SxWP3P|5VegS^S!}jQ z#n2Jm{*jJ1Q~9r$%~plzfQIxw8iZPdb!3Jh^cOI-BZW1a)gAq*ykO`3K|gQxm&H`1 z?JM0=Pjm@yk*7~^s|b3RxcsoiJZ%k{qAV!ZuN1|>a^)-HN%}rE`&3D^A@=C%WS~QI zF*jynq_p;G%gE(yzv_5*w?bJvNm`cN2et)0<6BG_76}d|iS_Ty^E#tF;1A*7B$_dr zQtnP`;^_-@Jv}BcH1xP!KP&0ob*c}5*C)ordU_pSW{TP+_@p@Q?aidH@2>WDwQ4#Jq9D=WR~gB_p9Ej@0Z1#q>M!|xC0hjNkNDDL|$F+mx<@f-daZE ztJ4_5?C)d1kHP>?TtC@#xf`7xr1t|}S9$QpaDkppf9eo0Yc1%eXS4EqC*i|!#$FK_ zi^@SPgR-(;I(^4;h3u*#hg;Vxdr2p9YujFD&zkAj@UXw1=6jT0KG$}{aYnLxm4a1g zx4L*1(6D!9((^YjNGw(DKU#_!;epUns<8b=_2sL5tKGc!IwHsKImc44*pyEqhViO8 z)T<>rt@CyjpeFFkVF`Cj-!F54KQl8g%gx;9$WAW{4|fNb%1`0%E=BSTj_+hwNfF!m*h&||sa*G}p<6#oQ zqJ70f7D~)BrL?&7w=Cd3fT?beWY9a>oPtVV=h)D5(0;WCMEP@zl$G+pqSVB?w} z@bjSlpTSmsD{TZy{{aKxHkY`I=%}2&p`@xn_RZ-X%#*?ytZ0bewz`SsBTaPZ}uXXdo8 zoRFzfx>k5_KuGC&uZr^Yk*vS;v^icfC0Ig?^bt(VDy8e* zeH^5lapR$D6f`_>T2`r9&COF-#{0Woy$SNojeHa_)ufQLV1KWzw)@JEVd2;iWYJwmCJ|?ppYcC|-{Fu4Snu}1$ZT*K&G0z4qwXa-am7}qOI4x3xb0=3 zJFNpKY!slz9Xgeck}7f4KARh{KL&AJ0uqkoBXL2ykm;`_rP^$RMCqeF1W?hSO#5BN+1zZ~>BL zWIjLq-61;r=C(CRNn(Q-#2UE7dtaU`4a2=X1p(LIsV1vOsJmWL#WtZZb+eovh;VwS zVeW@}Bw8FvD`)^g&?C1fQ5Sn}Wd!oHXT8D_g04hDJE*q35Rto!gOLf~Mr}!bh6#I) z_d_^r5Rgs2p&i@zq|?YcRQPh95rn0R%Bdtjm@+^kd(bGnz8ToJJSg0JZsZMuY&KU}P(CoAnU2~<^J>7gRy?3J^~v0DBMAk?U*0yoAl`2O=z)|TI9 ztC89km^_hJD`D? z?S#QH$%pxAZ&7sS?ve|QW+^7d=uS8I-B$Ehy1KlWUpt`P-d`DLg&)shF^tMp%6|Y! zWBu3y%alYzAk;MhVxEfB4In~A#NfGBP+<)|E+J5M=7b>en_W1NwJjF_x!Go*`MoK_}BxYL@A} z?!+R~;n<`LTjs03I*o*qfZK+GjZgm!U7xxI_@f7Ml&2}@{xt_bNBiOrl%^>!1(&lV z^?rHIKjEv{??_}h=7z6(K^>#R34K9`)+{Zc;4Fw$n=2eWJTRY=E77K0hhEp8z$l3I zC0BT~g1(gL3Lg03Hg{Yu64)G9X%N0BA>I`jV_zi{F0E9u4|2K`^{3U9g@@Kp>n{(A zp3$B+LqUFC%d`FZNAV!(lxLtoui~lU5`gv4lJn?~y%%e(ZXn!(X}AdDl%%l~7(xZi zqIf^%imCmwHO-Cb99-q=DTH_hzEnpxk{{{h2Omoe9OhN9v&m}#fw{K2Ka&$zt3P&I2 zBI(!o99&$Wcm}wPuZYFU-!7_LIuv(yl||(}+$J(~`yw5K!X9+q3#e^J*~I~!vNW}Bi7)+bAX#uv!o_<^&Nf!c^gI2!xZJfl^q2k zdh56Mc-vO;WJGLbf*+Zg{QS&WCVy$eJNa5t&ugb8aWK3G4X~&uJ#NI&UirtF3Q$1k zTalfCS&~tZff$QnWDynW$pIohr*E`qBKtBkTB{f&d(`Z$`%;IuffNPdB{XTcPw5~F z@vH76Xi@F%Cq!yu_x)Z>mi$?5r;Tx*_+QDLf%}6w>IKh>47)|uOvO!CM?hA@0_lMa zq3m}PC_k3Z@|OVCQzJsb^X(>sOAul2EVUIVW_0N6NYGS~kfMT=I?dFA)Qj%lMM)ZY zq1B=bM~uV{Y<-{dQ`UibeJ#I;FTXLO_IS#U*#tx$%L>ZJnRPTG9m`a0+&~n=U>fA? zPJ@s;R-#D=e?kUCFO3H{V~FfS0|7Ss&cj%mdT|HSd`e%D?nVT$@GSsQ#2_f2PK@TP0R=k!4qbP3G1h!#G@?bB%DJ$lTj0C1JT1<`FY3-n zIk|QzrPwE;SLY>=*wWd+=C_|?=%q#R6faFLB1ra6#o3H6+xHvIR4Ayd9bWjfy2NyCIr&&6@Hde>7YGJESG{nhKeVaL}ex7tP)ob z?Ze-$R@RL%Fu>f><{3WpN!7P6BF=(#JPsp6kk0=k6yy;{mY(8Z+%4>u>uqZjXgg|O z8c_R&AhQ+`i5c^UCu^4v`6X?&Tyk(vVk|%kHh&4ougNZ#Jo*L9+C;{7KxdMBVN7yANRuNS$0io^dlq%(w+ zyLH5Q3z)?A9Uh}C;~=&_F=D4ZV?aSu> zK$dm2TZ~0Z6QGxFob-b0BrCX7Ui1aEAV3t9D4%fM-qA8O6U@EM#`!#ktW4#utAMZb zv9jIB)P6vU7Rr=T$iOg}@WyhNcw;m&d<##MYKYTeb~2-~ZJa9o)GMLw2jx7&&?mju zji^t;-zy|B6nnHu4L`B>I3eg;cz)^cMQr$5HH+Jip*W=d*OMM0_PUVMq`OdL<%LN; znkzw(lylsbl1pkU>{x>(_`SYOlXhw>fzW&_V?P=1&n~Q;a!!y>xWMX07S}e+uSIK@ zO?#~I?8Iis*TqCiYta?fB`lxqmuV5g(r<7l8!&mrx%Pq5%$=BePBs_@sq@^(h85JTjXrLQn+t7`e`Wr zAUwIR_}f#%NZUY?YI>_wBcGENgHA%XsBYzDp7t0zWfwVoH?IT>`OkSK0AKE@?x^~| zw?pJl5w;RJr0q|CVj4Hg!>G5&(UKAI)hI{7vE#{fh(zwel_kZ+a_o!Xiyf|z)lwE& zzC758b}DA2P=*@@Vz-CwZ!DlbG2oDz4M&eFZHSN#En2$odn~CRO!~s}`!l5NYR868awd`ZL zoZD?h!4AgMS+M95u6 zUveg9G?aksjT$$d5IVNRD9)-$$gmM+yMt~0g$q&ad24JyLuAGe{(rYO)!+QiF2*n- zWE{qLvG2&C%nIxS{q8x`%H`S_#S$nRVT-8m%wi47m#HI>mB)Ekbg|QS>AoF@?dznQ z(eJiIp6~g1nJ%W|@Mi~4Fj_^gQ$p@4ZXU$GYMgJuDU%e`U&PGB6oS_=*NaR$x=w?$6)g#c^`G2A-5{Tfi%x%;^H8Py_)g4 zdO|Y;o12#**&?IB#9WK{KB=j56xl9Y-#AEWnm?5P%uIhA=*oU2t82J0QGd2s(<(iZ z(69$iSAlv7IL9R;goHn^d)h-Jqk*QAdx&U5mvGN&g*K`ovNS)?<%gw~O+o{Wc_&O6 zBFi?TwoNEd9op|---F}Uzl?+Sk59~W!x1a2+;s=e+fN#mPYpB*P~_kky$96$3mqg8 zKBbwCLSOdbBK;lpAoYu@7`tz#6gr?Ep@u@vKYM^bG6P`40tMjE-i>K&;4-E&8x0~)8?1Q(!NLsy5SF%N*UIY{= z3@ggngs8Ae^(*0U$Bl)4HXX|o9JSdo#~xvm4^YT*X@>lfk^@v7go}1|Ou}D9B$X?k zfj4?Wkv%O){&_3QEw2T>*uCCfP6(^lkOCL#`bU144suI$<(Vz60niciAMkv?+i3K8 z_Ddkq(xSLCI-k83HH3b;8g_Slbs~02IwGTCo{ZxPB&r4tK*vIe-g@bPeQgU`iL&F#dx3+0(u_+G?+AN%7zBSY} z76hfpTA@&57r9Rox(SPKJC4kRFZKsS1Vn&88Ke}oiz{y=+CH`9WT53&9wqZ7$1HW3*JIwN@q*FP?Fs1 z7Ay)nIsqZMQjOL#Y1QO$CWRU%D1Qun37$6r2ih+Y+F0>+Xd&tTs_(th$cXb0#=3k0 z#0;Di33=fu)4qW{>bVW+=Lt!WN!77kGbaGE%@7H&N7**Ho=2NxBZ*F4tc0Cx6-hN& z?G?=*G4Sb4P2bxCh~4TQG2xR32YVAk*DzHZWXN*i!6Syr9cJ_FiG|!=*G9=Bw*c=w zam;J43#8u80EqhwcpzKn#L!8vHo9%vifu#S>O8TlJNSQSd+%_p+wgxJ-HJ4iHa>=Ou=Ae_ziZQ5I@)@;5?n<4%IkW{{$o2}nJO)O-ll$QEucu$x8Cx+7 zH;U~@hn8Z6~Ut8dNGnbBM8wcn>$t?o+)FTcn zpQU%TE;*6^`^$A%+SxxB>k6D>tr>$9lD9n0Ikta2|42^b6eyCDgS;*;GW6-UOJLnB zh~_rAPU<(i{_*+ESCha#N=b>PY;DP{iNhsFNteP@auv?F|*{ zN|X?lWm*VY*4K8;bh)o{B7}J%WdLGUj@X$rXHs#A^beo+FXmm>bF7p$m5a7~@#|({ zOue&iE5}tHcPz1i_pxjT`RuF3CAT=pLICp zfL5o&;2Crf))2f?|DE5|t8?siN3?uK_s6+z7fstkLG)%ZpIX=2bT)6Kn4@)$8Jzp< zSS{-swtFseuJZSzt)APoo_Ss6&7E!1^801c;7LVIP0GaF%aOEC+RL4Z{4?vyg6tjz zvfP$*p=9s_C`5gtTu?CUq1VI=N4V-(v|bpCT`N0kD)%P6Jo_@N@?ZdJsPznIk;oQk zyRq)VM$8bB!RFL+zMkw{_RF?II@#Jwj@iTvV?joo3iNR*M80!mq^8D)5<(=5xs&FD zJ|!T0-YV0=%hM=;#AQBBn46V-J@sBbc~)IN_$I9D_Lv$u_}Xz8W{L{tY5$EGUCsB* ziv8h0qhbX9B4fGiO1gQzcH%uk$w^{Jse(q z|Ba{Z#SUzcowo<&bazg&bkOpXCF&?y+barbTub!&ErEb~QyFG>PjAMLqnAFlKvc9oUEP7MR|ZbTh@9~M_%V}*NMq)-NPf&kiLaO z7B7KIpOA?``}2-zK$+)8yy96ndD#@VDb)mWsZm)pULPs~3Nl!k-5hTXvLz&9Y=;S< zwcEXu{5ECFai>vKbXH}6U)FsIxw>2CUPmc$8%RO)0J=W%{j4skFW**VE+j=Czs5)y z+t4A%o=bw;hN-yS^x>a`EbPRJ zQs&iIH@PbkBDl9V3t$`s3jTI@11{`eRiyL#7X>~GR<*xe0P`nRcc4JhzH_wgGE8y} zRd!0xC*YAos+;&p0!dx-9+GxeDAVu@oAE(iHpK?!IPL5<$$)c{udyL5bx3mSK{1$^ zG|JE17`ZAHtXLfy!qU$cI~al<%rdhp;O#A( zR5p{KNbi>O;xq2-YG&VZKM6?0WiUYj=*U9#ezY%&Fx4nFR{n02!Qd&9St(|BT zs}zZU@bRWMDsc%fRkO@O19iG|QZA~cFu{5p@le8*Z1%H4Wl2hvbr(B;!Q})jDyTmA zBM5K%dHAWVSm8q`rKM=W(wW-*G!nu`809cOq(xRI+xF$6uPssDrVOAxOpC-PmlUMK z{7H;FzF-9vAbe9=ryu|C->r{H+Qrd}CBHr-hZkQ&FMiN$#EW}zyBNNP(=fRIA=Js_ zU=hsyPGkU;!BiA1!Un@FC%jMGA&2U@kN~ZPT=$cyM42>XbWXw&n>>H=9EsLTSPVb4 z2fGN0l%u=lJ1>fOW>~5uQmBJX{zx+#$zxw%pNO;6ybr-As$1dGgkVlhXL}1QLzA>w z5MT@vKjA2ep5J4VreS))`1&5Go%OjrRSv_?(Siha4pMGnhxn|3Gs%CL++xwLlP2`PR(Yk0s@8d8XG<96#K+1x;4~^c@tQX86O28r1Bn zPrkVEUaxeRufvHKW%i)XVLtgbEFY4Ch1yYa#m8U4iteR*_z@4FFjvNjRonPwxjm=> z4GNIsg<-9xXBSp}wtN9Lg+fDVecofmU<1vdgFgTB{xjc!EuoX^AWGcc0fDCd=6P*b zE5E{t9?WfaYq{xg_w`yhT2rrJ+#+=CytePy+_4>g$1Z$B(QDh)3`jtJ%NvD$g?xmF z?|glE--&a#rE)rJ3Yt7fsmu_az(DDv;^}OXK>lWU;e6-`*X0I5WuZtwFrL{gDEpI9 z6Ucc?z9zvMcp4NYF3Z;CI;%phWA!@6HMznKXeLvf4R(DkO{2v+Pnjzh#9p*=TF1qc z=9aTnm*K5(`7;^U)Lp#9sQg0M;6!AXyaSO(=7njuxADMXkWTkYXhS_HjjzO$yJI^= zq4<;60LcwISZ$#Pl&SzF=0$yZGr{W0#(S5car*fo`);oT896o`pg{E&|@5!V#2z z%$=1uppB;_N7V$JyrAEx%~1ST-#utK^ci6@L#^U#&6f{HgyzJ6_hWHPC)$mQB$0fm zJDTVj=dK;_B`4uoFuMP0bVsI>WM3q3lw1pk&Dp=_{f{XK$ssR3hM~~PG>VdY)=45p z>9vwfpE=5b)jSV?rbsGX{Y4w-CXAxhDk)c2j19#wgs`Q((a_YY1m=nHD?D1Ka3mW8 zj`xH3{SId@6|MD{_E#ObuG?}`{UdZ&4X&>*PC`D1)NdU$cO&+N>8{lL4>fq^umA!> zLElu+OOs=sQCECrC_IgGS?LEPG#_`z4mSW`^Nx1l*weErad`(6>nf=K z4(boK212#z(6A!;H?GeQGg<)8LTnC?y!M}JYdIQ4#9n~9rR>6I3cbPF(U);T}q)rs_mCfnZyWt6VlQ7+XOmRyk@cHzfQs{%JbUZDcy{_|wS@K5SFU zZ@xrU#CF$f2iB|q(A^@tqnC#kSA=Nf0n+d~QFb?DL9*LT%b`nq3euk`_hHed93OsJ zYakPYxXY62#aC;(^q8E{8$u1kZz(Xbh>~IsB5`(2ak&i=F}dT>v>G$5ws6y)xQw;o5}2Sd)(Pbc+;c<_gZ5ralP9ehHlKbowE2bL$8fbS%|**myx7$a(*kIVyi-PMwU;FcmT2O2GX3doE$ znGJvS6qty%iOko2Z$QSdY6_THyRne#J&5C`hf+^d%vvmiKHsn@c6KA>A&J-({|bpc-a2loz_plMMn` z1Jb}V>CC!^Ef7=iOUg5w2IG<0>(Z9>0L1FWJvq5uvATo@2kmE57QrEA*(oksUA?h~ z>eXs1fW3`+*3XDK>}yxx%JM*vgYK~P1MY_K2EoghFnGru0!6;a|aAn6yns$bQ0E%m3aJdxv~vF9R-iV?Z{cA;>kf~ z$zESMzH6~;DZyg>Bo5kPOK+@5(jZS^M`z!d&qyJIs+l+A{Jn4A3#~?G0LNj!Z4Z6+ z21vd4N|bhPVmHrVVN_!`)WK;-fm1ovKWBTH9^vcSIgsoNXRaWfVvhXLM_WeXcz=`~ z?~j`MNoG8kYM_G9YWcG4K5zy^knenGUkGy6!76l6L85u$E0szvRJL-a2wMTZ8gC9x zrjgEeJ6aAVUyuO4(_e8v2XvXjZZe>xKj|U-R1a-`SU@5pyi1${a9Jt|asUK_2jg3C zzU9VG79EJpPx!L@0lVnM3ZAa)ZC0fCRS99pXZ}MUYwacsp>scbqYe_P@r~%^IfQRT z;pI2M6VRpoBFlk)`8O0qB6RtXF%M=6SzD)LavOxOC{*CYukuB%5aUh@bYg*N;fQ${ z7-RMASl;IVJ2z&)b1cKgpG z9v7`ea;1XaZKIo3di~=(P#u{An&KTqCRKY69KEe@w)A11VAR^h_`C4c>j(Ic?C!u_ zJl^T(DIndDVng8otN9eAd)e05*+y(%x>imk!7*O_mk5djV53d;MQ1laZ-dwWGC?^G z@AYSpq97R>a(khJcc9Zb1r?&KkzhKzO{F`s$oPB!=b#RC?c<22AVFII?akjuo0U7o zP$x8^CbR7{{kX?{7@`%P*ki)DU4dYc4F$gz6>U9!sJdlz0A70hUUb^qut#%6$w&hs z*I5TLfKcO>=zH+xp!s4YZ2=UnKLgsB_SOi7$X6Dmp2bh(F`fs`=_JDDAtI_et{lbE zh1W>ZB0w-Wscv|g%M5{Yng!LMPOW(aDrmS8J)7SPRf}ovpb(Y8PmiO9EKA@|remgCVB2X0af>By@x+5(t z14{diJ)9BGZAgNHU|X$Q-^0B3jqM2s*;7)0cc?k$gwP7 z0TK0Us{wU>A1YI(?Cdw4Cf{bxD6&%}he@{pGS~zxoymnerB!locjaY2 zijFT1ogx>if#d%HbfU+YoN*B;O!mYYqE`Z{iVMYvzkn^+fq7p&<&FGq!zjb9$6r&_ z5&%WE_Td?gc6Fo3G#AIwcQXn+MPEIIKnE97-3`_wdgZ*0oHoR^^&J?)^eHWXjm?l_ zE*%0+L{_!=CT8Jb{rM^()0hj5LSML#Jwkon&~C(Q9_5HQ$(Du>ME@UY1yubyTf2cn zLI2IQkja&c-WWjr;LjB8x{-g`5Avg&8$cd(CK{;5>;OUYk-p&hA^PJSZTZub_PHg~ zsH1#wX_aMPBkH$$qZ}$cWo^N<#TB!yHKO>;SUQ9`w1+tOBs3jScM+z6hS%Of`N6TvFpJ)er+BAx_e_~GN^uFRM*z{SXjZ&mLd;MB1}&pY)gKk3LZ7t zoPy7q`8w@uu_VbwsULGOf`ive^BKa_@6GD5S+R>6MYIEy%>V&A)pPsvX#>I=}G!yLzqx zmu3X$OE;7VThlA9piPoRxunDj$0E{npp;2s(?l_m?figvrZSCi&#*H%ertfo6eyzM z*qE>3@6x^3mZX#e9B56@(Aot!AsdK0)9nFpIl|?n@0sxSj`cuapvEiv#B??g)#+Q0 zv*uD_3UVKs!A}}{k|9J;%3LNSdJQ`GkW%o{?%23-enP>JanWQVa#?RTO3DGyQi{;s zi(Ci}E23S@L7@et5Elv?L8u02y+Z0FRie*{pgQC1VilvH`F);%s(l#|p{vRT7X34b zYp-P>Yx#{g&s6^9I3VeBwhoGC$@&R^m|SQ>E1GR9=(P~NKbv6Qp0^x&*t(tTMMIQX zwr6ieU6qZ8)c@+EsZ~J)n6S8Sn$^WZH=JOE=axXQ_)rN;VdNhzRP|&vU(JvUVS-++nc= zw+2{Kix=s{G&jm~-+s&&y#@8E86Y0XLs^+o%IA>K5^M{dv|d4`L-c-QkHF2(Eek)s zcxy|H_m)!-Re~nfI&$|xVdT;5o0rG2z;DO%r zEeS7B!miPY!?+{C`}Ahvh@VYeN$$gR;6|~a<j?0WE9 zD6E+;Sw7^)R3~ZCPT(HN0U|N};?*H|7sOfjZRii3z6l9A_)O}5zLe%Fi_{*NvL@6I zT24FbBey}71|8QYA(wFe4)}QZ@M7uiI5Yf_2R5x%I@B8dF<2SiJ2_LmOLxl|g51TP z##Qi$x$b zL!^?Dch8(S)Cne73uOuCg}ZI8t1{nmyhC|!gC`RPr?5Mz$#Od`i!D6 zo|b|M5;Lh2K$zTFxYRfN!h*HK4l62`I{d)&yQcL1DW#?4~8{CeKH zYvw16p_3&9RJjTLu^zmYTUA7{1^#@+?(;`H1BOt!A-hBw&SxAbzEHl$4yo~$Rt$`BzULSE`q%&B3NEjGUaXNyGr?4ClOfL$Rm6=ozBhUW}XX`$^HReZ| zh-fGZBKoY3<0?Ji_5IC6woQRQCW0{j=8(wDLfl8l9X%o8?4ZT%K<+TLV)#Y}A_K#J z1k8B$J7{2GAm)eYBoCz&xc#(O^VR|PX#Stl=~Wh zUwTo(0i~I^^S^%vU*BCwX=xOIEBUDO4yna^xZFOSP%}#YKZ24PYGtq;*5EB|48dwD zn56Xx8#TvAkKLabyMS4O*e}}uP}i;v>cBA^!-J7}4x~s7m*8{;z~ANBR8z}QvRD+b zjb^pMr<77dg>2fC1iKL`G!rb+7p2VUr%RJ67oXHMzrH*RMc@Z8fgaRZT=fdR@! zJUo?~uB^xDffr&8(#yX&qaeI62P&I9gv|s0^+YkS42_tIAPDU&fNabtyP!V<(e|1_ zIl+>$X&(u-0?2vRS6O@mF1eBTe{Osy%Ntbajzm7qc+0!~e5lD+U(E3cfYMPn@`QUG zMtozQ`Cw>BMHIO6g8u*t?gRAlx1f9*f_kl^z?zd!YQr%0+jc!W8UeYK1C)bZ$t)_9HK3jpMxd`-w zJ~=Chay0tq1^A!BrDFT}xuvDU7HOpydV{c z?zaF=7EtYd;!*{bsezN}1Z1SIt|EHIUd^ zxJ+9Bp6GqZc+*r(vu00Obupib#YLi7aFA#fe4x_@r(HUCeHCHwOSqki%wwFTEnwy6 zNEgb!LZ!{Twc;ulu0GX!`{}{xBYw^c)rXZosd@z&2sZ{vHv<>go#L5n?=vpo?dAg0 zRloTxgQtK}sx!nEW?>8uBpvQeO(gu5N2_`42spMrGFHPE99 zY*-20xkmO0@7A^!Ur~l}1c0(LN^Hs0y`@0m(EbsnR5MH0wA*--QB}a@SMi|mzB+5` zXnJi*htccUyFPP^sL7wZ*o(_T(N)glqt&tghdecBp zq7B0J8F0Px5HoXd7&L#Oa5{u;efJodBR?sDwqs_;(`?hC6*X7&pqxlP#XJ8s1S|Um zzR#Q9qNy5=xQE5JPFVnwhkw0P%i+hf)s|$6LvH_&yHDV(|IcJn6u7(HU1;VI#Xzha(Vu>S48g@6@rh(=eNw!l6iAZ zyR~Ks(eMqwiTUKr>-T>Qf|I^dJN;cXCcq}1+2$1;WIC>(Oovr_-QWZQNAdPJngUgm zS*z6SwIf<1g{D0fwJ(5>u}FxNr)G#a+~|}BuLzZ}R3jINFHh4!!9Ma^C(hsX@b>0% zmpD-sU8AWVkQdK>}o75NonfdA7~ zb@tTs&d0K%qef0~`xrsNMPZWQ>?@`ah|v6b?}@O0H0GXjp^^>J!fopfh|s$)Fb!3F z*MgRz9a4w2JUmnb6`LAWnq*Fx>-YGY0uM#DY%RgOcqT&~rF-A& z*(}mV6*Q9UESX~ zjW|V+RK;(9}E^c&n#zyM^kQj@weaHD7FsNR_eRM;Zq?whC^o=Ti#`&)- zq?a)y9!%vq!WijVoPGtq2vpE|Qm#&Ce-)w_cTgTAn83Rxk#6~XgFr8n4!o0AlB_(& z5gnjC%>&99dl7KY6ml|spsB;6&W-H`KanU=j2&OZ(%W8JrlCUBEvAd$s1WT42t*%&tD)iCKKa2DsPsfb}i3{}Ws;oSu*bra_V=%Qlx8N55 zwZ$G{zYgKzQf??ffZRVKB!$KEK)>8TH_&ePGaXz{B81CjiCFvBiU%hm4MY-)sI zKqS5kJcvBSNFON^$CXLwa8b>GeYlRNdPG@dsX7!U0q!>y(cE))DWdM^*Dz$GpFfh9 z;qM_RXnniqat_Xh#Al)*!xOJN{W?~Ewg^KVA^ge%DzpPB5@7t&EESFLq@V4Q_?p3= z!80r5L>dUgqH%}}fs2ku!8unEfHhYrPD=pY=Jy z4O#xn-P9&fB54{G1pFd}c3bC9#zgp`GM-8HExNO)^Q zRJ~tT4nBmI3sfIqLjL8dj@#UrOLf^a9VZP?@H{ei=X{4 z45%v|H*{D}3w&tR=Tn4-kbS!JcbDBkX9#0 za}H&);QokUpJA{MyCN%vkH5lntv%@>xY&)qz=Z8_M+|6C^rKHd1CBxtRM7UF@qZ%E z@v25&aU|_tO8M{S%yPni-_w2p=dF^(Iz3k&L}TR>ac_3fc3F-ZMEaro=Z&BkfEZ@0An6EBT34!A%rzJ*i~G79X<(ce!W$nFW4r?h<=8okk>V zRQOg`x$o85{K?~kfS6(|6XiemiSHjy9V`p3WT7tp5>Z&gu!5t?p;9o+<(OM(_o*3b zdk|Ho8DD3+G)o>1@l(#|9cR<~^+u=u3W=`*%4Pv~Xb*Ig_ZME7MC~l@ds!Ed7ylaj z#EtH}X_~pG$Wk6lnjeF>(~xEI^La9@WyLs&e%7CFea|*#n>su&7Ww-=9HMDA4sIZA zBf#w%XSf^t4y1^bOg5BbM1V@D&zK>=-!OhF8(BAwAEL zCPUrp=!z8c%g?m&nox;kG-|7E-j7R+Bu^&RJWsGb9{_*bS?`Go)$i$8D|&Pj>kIt~)O{eMy2@iM!B zQ=|f`g_;%7dLC4|y);^5abi}NTpy@sUZ~!}0?vsB0V(>`Y1V%>S@i-Z(^ zWTrgvXqwo${TsLp%o7YSXtqdS4vag~dGz$( zlR^Lwp{iTuk?Y?t?xUxU_ojwwCA_>tRw~FzxUV74G}Q7!r)cbzKa9zM6q?)DPnA7q zJk6BFp@WhCmi6Vjjz9T7?)_K0kjEf1Rva}16O4aau~qx1xDLB*kO;(=?9 zVMD42f!>xJLrxQgd#c5EBS0DBXT^=y*9Yj?;Np_+m-2M+^)_v0Ri6(DE?N9e+`bl$ zy{9Xxyv0QtqlI%yj_f^s(=WEPLnZ(B$cWWmdJ=)k-L*L${T~?RssGu2l>cfT$FNRu z8VPHjC$s{>D&H_o8gaP_>_pyi)`d|@S8{g~+iS*x__4$<;e|AWQPc$SM5yLra&u)| zE4@TMU4AJVi>bOq3J5-q-__X@;HCKPm{hhVzjYa>a%VlXUKU#WMz!7?jeupw9;xRivfK zeRh+29~iFU*DBao>RsLY*5e)@f@Gc*T2nUjo;|z$^CxkdW z#>Vq9X@N%Pl)aut! z81v|DVG0AOnB0#lGO2aH5Hm*G-D1sD#IEqpQ*J=%2q>t|P6bb~7O~wX5#^>BcH*X5 z`0W3c!8J2shl}jwXIBzg@>-5<)=FmAL!y5VKr60j3;||0q-EN4lzT6MlD_%FOe~v&`5#w;p*(~ zh#hi~E|wA}=X{|;-y|I>vj3$>ED4r`TBdxHP@c-gIoglp;=1Do+r4Kbm$F$ek=Hf_ zrQwzb$Z)`jA&!ZQPxtcg6?Z~e-1vhe!FAgE)RGVG4LW@y)XjtG>g>uE}Y@}MMXXfK;Q>Qhm8C+hLaiSRtlg0v13Y^&@B=lgq6e36b#+V&fU zeET1*xUx5CkxLN}V#sALm23K4Di69ckFGppx~*1j_@S+(QU=NwQbMSFdi9YZ1dbMEYE^qH8<8TFk#dF#i%7N_B!;23_R_ zNvzo2tM@VFQsw-FMg2v6M-&)FxOAP4V13QWrI-}UeGaA45=As^`QI^E)UqLTCDFeW z=QO`AX{vYHlnD)nTC3ctd>JT0-T^lWb+=FcN%&XK{jw>`P}j{XIV>}ktN3j%3Dc{1 zH_XND^5kyQ@xvx?Klk^04AaDywttNZlfmfSq}GV?t*NI8{WDCfvN@9A0XL})BCF5z z$1=V|i%kAH^5_ZyVKGs~xoSp|^|+|jnYq&Fw-I$!Td!40A5`xMzg874JCl2y5EFRh zLr4Po0&A~IupbOBmz9ad{*7KJTg9Q}sobx1wIoccOoXmK+kO_2J{=9H^dorviXJO} z5td|&sBy0tB2e#vX3syiW5T#`Vcb)G9I8;pC~BhL5D~FsmR?AVmkuSQU*@DHFdKO* zNxk21@BT;Jy;*2I49Ma0HyA6__d55ygztZ+dFfx=DlI8ib%pI=(s1WYu;2JK z|9+xJF7NKl0oX#Rcs9Zd$7MgqIuWXtHbvb2$TpmGX7ec{)kb)gQ(Vm>F0_cD|6^Uh z8VsU;iX>8#nJ1r1>}HuELqd7I+=piW-b?HUc>ODy+A<8Xei;%mEgL;yTGnRx)8w6T zaT>mFPN*ZipHQVoylNX{+1;%iV%m04;q(snoeH00v}nZ8{K&)ERL}l?_bXQNXk;FL zQxC<$3-N~73zt&pjq|1B>2NEo=lHe<^3t^}WVw{^#0MsFJIBzzZCxS;8kiHj#Ej(k zm)Id2$Jw357t#W2H#>Rtd0xw$e0A*9s4FY^>1n}dc41m@knTRls6>c9%V&Xgx59iT zVqn+sW2qH@DH0In6CWp*-2OFIBuO3ERe!eKNd?0Zw)W!kHdA%U*?Fs~tj?HjB6sHf zHx4^6IF3^AKm6Iutlh$nR+_>c*UoZ-C+t3d;*ULOOzw0f;p@+~D*PRJ{I@6AnQAZQ zM5n#+5Zh*S{SA_o-t>fk$b#g1ty=m%vHVPWo2L=lXb?4IAc&Ur47gb>vD|*buzcez zX2~@)F0@EP-DWZzBQ>6^Q()0CF)sdusbIsmhguz{Ak^#U_hnpy&yso&i|yHapju5K z@DChQEMdNRomh5~RNC!ua#;q_C)zd!=DQLMSL*dZBIKV0za$`3Avwf| z&LyZEEh!>~3#U0vIQ5J6KjUhIGuh1XTlW%^qrdRe{p&~K)y#48@E%=@87S-Jnq`?l1X8RIw%6^ zC8zRx(>+=B`rYTzfyjm#)DF!YLNZT4ScTcTqq5+5jw`T>E04(Vt3T=E(UzU)0I58^ z5^ELQ`iXj?BN9E%jGKFNlADLr>J!GD&Fu;EIi6opbtH_gLsSRcM)}Ez zwDt>~SODOas~RAhu7?WDoPe)YhrkOmKYG{VU+_v4hCX`Ob+l;5_868Vf}k2B2|T`C z?ynph*lrhX)=p%8Mp}%;!J0!3t~?@e_aP>6;GH9|!_6E9p+UHF^PA%Dk;88FkJ$NA zEp7~Pu>U+?;V%|BL2MzQRQK9#)otSS3x=z}M|9`VsT5u`eQ;e>1muwx5;8Jh8GZCT5H{NKVuhJANZt4;~!ohCnE?^@tbWGSfu?PB&$*w4=qc7{j z0tBwUeYpDu&LlU|$MpucmwxWl)Ooa2l)vcxNV~jwyz{hryS7yYM|_MH6zGC$1zPmNA`Or(!QL_|_0Cl#`)5{b>mI6L<+=ydt99h6;% zBj$DnKApLOS^f902wAwJYduLkTn02vAvK=0OjA?Z1~n*tfc~aJ%OOiqjH>k%(z8Ud z?ZtldNLcdVC3y@bI8%#Ww8VUyXU~N@{yXPu6v|1QmX;2BYO37d6N_#aoCV-=i*|)J z?tJUMskDdR<5UJYXS|z-Nrz7l|B;w?}VPODtUJZ ziD)>8=-pguAGGvHz(%#DWL6<{pn+~HFo7GRUYo<}`A0%l_j1>d%YtatF=7^LXY)YI zQNmsYr{oy#YB&-ma&;8ZnSAyjb<&G-l72Ydsl=Vzd}&o;SiFij`k({gl8@&B&4Fkm z7Kt~ZJ%+@gZ<5Rev@rhjgZ5jK- z0O}Xl1Nr3I40(`t0(TnBXBRoDgi9t^UK7B1aY$qBKaH0Q3DN%lzxTNRVG!W&VtASE;FePYznr5vDo9H3b9l8o zUAcE0I+^&uML$Yu60QCsUO=heml;$1{duz=esyN6$6a;6BY`vX7Bx<8)incAwiFYa z`-fYt7_n2fM8(Lh$BBkDGk(I#BIi|=kU4Uz>dd?6zP5zCRL*q#+5BuLdhp*80p%qU zYbtNPWnY{^i?C-q0^**H->*}f1y9%hDT+LzJFcRX3+>utV<9<&O@PP^zUD`azReVg zoay74-wP}UpT{i{{+)!I)H*0OuP$Vh!Kd)TDtYVDl8BZU>Df=c070PEa-%Skp1axe z97)Bi5UKC$hglr5r;Z4&nTih;fdF+t1TiZ!og#e-GfMDF^ZlRTRELJy;Xmz;bS}nx zUh0QZ${v5{OBt)tQ|TVuRC(&OKFH|encnYQlPYHA+hft|rxn8SXT+)ON=BS>PPTRZ z)T4yn`m(a;RdI=%W8>my)iGiX{*&qMII)yU1C^TLRaExruLpPjSFzjikIYQ_3eVH) zm5*?rs>1Hed!X3!xdlLA`%d8!q?Z7a(G&B zA-1hb_D()=DvQHIxq(x%mENi@VpPJDtvq_=SI?gYeslpW+xhT$bqgCvl5{_oJ*f<@Zk;+7fFL{<^W4^AzyMx|%4A`u1AK zqsO&`4b!y6V#az;*LpMKW@Kc(@tMANmbkmaSY~;0)mtNW@0#P@T7PZH_3hqmm!{*S z?S(E&&wdQv8X1vpAGti^diLaC+>ISQqQNr8LnM+bt_d(DzT-Q1sVx6UKUWq^&1ju(F@u*%_ls@f&K{@z3jWna^r5 z&>GkoOyBXppTK9z`q`f`{0NbMlGt*7x5RJGxM?broV(TcN{iI%r?#ziB5aZe*0!@p zms3{LCcdX^rTAF~^7OYGogc$;H?4S|YBICbi>gW)iwF=fEFMZI+tS>q3?3OAD7nut z5_~_O-@jrhA-`8>Z-L-gWR+&!h2Ds&+v^#jw0#4?rVL}tDT9{Ve~#s8`F|_ua5>Kw zKHFeQ#K2O)fGY%Y)uMysz`B)2074a5}!zkuUbh`r|A0l$!cA;Sa-qfd;!Q_)cUW9 z&N4|NhILJdY>>rN^Q!b(QvvJbqz0!s=6U+TmsuuD`jm|GSpy^#uEOI*vo4|5 zIo6|tag!a5{%V^=vOC)(4kE6H|GL2^K*9y6`%iyC5H|` z)$N|mlR|1=G+0#=LZaAV+cMBKfQ?Q?__txtHs<@pLY@!uO;f;71>b<5dy2*-;dB)1 zX{qucPD}P@y*Oi1bLDAFsJ2_s%d7U1zFFBnPD{O7O7!S^l# z*mDQqZ^LdpJ9X4Oo`8emB847$WjEOav&K({Q*a7R>O5?fSJ!(aa=0vySC>I9=P5#E z=D@(Wy6Bys2&%?^+nJ$=5wtq;D8rQ{By;{c40$SUK_rZORgl%kVqh?>6xa5k!Nfm+ z!91Hr{?R>CBelhkJ~V7>uSVs*de4_#Q=dJuZ~M(S|5==hpCV&*G|Mc*4oKsaJ|rYC zVwKbP@v%o`LXvbQ*>?%gSE>_JRhb$U52}P_vgRx{CSxz#30f|qbPf|a`G?15hp3wr z;KT3^`7=`G>OGm3Jr6(tq3<~Qs#fK4?+Izk?Zqvnxrp4PO72#=4E_8lMf}2@Z;Vw;td4LLu64T(-JR@nsW* zRJ;$7fM0{b!-NL>QRbs!8Zu@NE^7k62P3vV{m6ul+~^nU8@0lFTz)mq9a5(-^K^Cq(LYv9R$DuU0B3`K&z zH5G4ld5kmXv%iSh^ts3gmLz4wXnOX$mQ&HpmHggT@R=bE8NG~Yr3#0>LDU<) zdI(Cva(MvKN-D5fOB|+)y0hyjxC2|eScC6U{&P5sf3;MJCvA2aufSFDM{DW5W4uo$0A8BDsg)xL z90S@Yz?|;T$5a8QMw?ksyJXn8VYo^7I#jabH-IHvban^Ykgp+z?A1-UJV_1Mq+7uHcVI!wz&6F?6VDLK!;=x(wg8_LUBVZHWpHc3HYC zpRfhsY)8AhTk>-j+Oyi?0uyi4t|7-0w9G#60_lEY>NNi4bqu?`hxcY*1BW2 z&JB6;o=(w<$aI}!tUyfx{7_xidT%|jhrvna8Lpx!IH0{kqN%V)+mq`ezJ5|l%-#$<8^z-!;^hBmvSjNh( z^KK+xeaC5qt~9O3c3_JzGoBo=%`DES^>7tVpe|}Al;?3>J`3kQGq5g0KS20C(39}2 z!pl;Cw@B0pr zZL6=OUMD0Z7?wh}zrwA2cH0o)=jaY&w#R@vww$bcpWkP#kOxFv(}<}5M875F^FJ`- zuK7PpVP)44jYH-RjgHytK~sSz*xCRJH2^P0#}3L)mBEe3A$JeO!okt3qsx$S92mWSb@ldxA?UF6 z6PZf!5`2=f(L+YPBC=Eni1+J*f(F$If`LiH4W8;5PaT&4eENP)KyTsNqDHad; zosVPk-H05Br;#i$b=u*_^KR+ns)GXGAvMz;C||BYN8z(6)0`OEg=;{g{jQ+bs)}Cpu|+4;y!ZJdq zZ;or;Y$Eecov)*syl?4}G&PZ(;oOC83V7T^R$`KWs?|G4S>#=hvT-qO>lKsO^od`g z)@D2z3{mEDQQ0`RgPb$lZOhMmMSXt0msyAUtvSwhP*HO;#=2mJqp$q_68=<%p~vpt zQ$m$q;_-Ml%{>gptw0NYQ~Y(w_QtgBnbH=fBcv*$S8P1lLYrULA!bf5!Pb}v4=L+>Detp{~cD1SpQ=elF(nE`;*S;5vgoR!2Tna)Uwi_iE}=YdOvg@a0Q_F zmnng4Kf&7MVc$ckd)1d>1@~j$e*&_Usg3aFFHuir%R)t50P5&OZ<&3!^z3>)qfx{k z{zt(>we7a<#55@m30e<--WfpcOlfxlgU1d#=+l#GCLSD+jW!Wp`jpO5$GGzwd-J~K zHT6{+%iv%a6(ZprhlCfWTlePNvh+}@JYjq7G54J?!P@F{y@L+*8v{mhwE&XVnuHE9 z+Y_YvCF?I9N_s|M&KdrJE?C*FiW8OU+&OCHBT9|B6en^r+NC#oPE=?-R*^UX9aDMf zZ^Rd62HJBqdQQ+RRc};0z~-MEI@KfibPeZjP;11i6|&4zEl}rmCFIpP!E1bunKz|B zb7ONCBJ}qV*?yJWPa(LSZHAEvHKOGH#<);jvkZjvt&>$zTm?yo2G}Hy2zQ4C$<0&7 zctuCo)s&Wm#-VE+3?I+0O?5Pt$c36pknI)w^5XD&hcM>J9lPdmY=-QhVBLFN|VErLGHG!ozxln=a^f?+=Dl zq}S`;UTwACX5$)K9G6MWzswW2e3sXws#v^a+ZmYnQ(g3l7>?bcrYjn4kiN#<6qT0z z(H9K8-s&f^?GN8vJwuO;v6-I`wKK~Tb7Z~~dn)HAp?_|HR&|hJAJ#(w-U`WynKwj~0Y=8B#P=dpgAAZ5=p_PmaTXXhhX><>pFUGj| ziLZVCNLNvJxrEf~3~>3$CEF(K731|vXc<=W(1mWRN2htcuu~&SqdFI={OpUZOWz5; zw&p+ZN{ypJT4q*OmRV9wX2BWkcgIpFTr+UnO6gYJ)&)xC-xIGFEm`k)lb$W{3|Yl) zyJ5%A@8>b34%FR0aSKYY5%$$HSf0ti0QGG~D!v*vHN*3S{yxn6d2r5g3A%}WBMa7I z)0l1Od{|$Nt^$1TWdyIc*gpTVe&0*6AAbGm?R94O_1!GzQ1CN+EJ#5-YHyanNEGY5 zR+t0Zy%W&D!JXF_CDjL95kC~qBv|(JOIS8mY;5d_h=uv1~4dd5$=QoTvJR_#w}`?*soujSATmj$jZSktm# zw(LTXhj7K)7&mBE>};v9ml)Ak;NI#HZf-Uerw6Y6Eyz{Wr>B8TKebF3xo+C@%c&8A|z@n%~eqrv12S=gJgkoWkZef5!Kc^2|d4 z>%NMd51?7=cUtp;qc)2}BGDd~lFK(rJ=5#x7G5ePyyEbEfL$A$%hIa7y_@-U$JZK{ zw*CI5Ewf7O$yZfW%D>5ZcnDEWeO(V|PWDmgbE{HJ=c@P$)$O#f^FW{Me5B}Jkn7?; z8rqu=19n)%5BrcZOf^)njMYpmak7YSdG{^WnL)NH;=m1d;?Q4X7HSoq5o4kKh9^kh z)!+Smpj)7@TEE_KcpNf&nY?#CdkmdEQY0FF>iXQU0nHA5DTC9`lgs%o!9$<>_BuDm zU+iew(#xo?$br_aYty&I6kl7fhrJurHq9^?$dvF3^6^%EuX61lhIV zzL)<;+k1y&{l9ELiQ$+S;|gv$;>DsSy|cc z^LbtM{*L>&@B6rq-|znKbG$#gUa!}4oa1pm&c_*GT$r~1d7-RsU#qdKvx&m($URA$ zyAoO_Xj{HtzwL#EiXeZMfpCIy+4-TabOSR46X>eRRIVco;{nBTtEh!BnAs66qlK54 znHDt_URGJPYb5|M+llXIX_KGbpe7dW+;SoQ0HoTy7{P6)7!!eUBqHoS4DAH<%g}qT zgWT35na;;$yPoFJq-}E=j@5jg1JRliE{dmxhpd5(wx(BF_WqQec2(56AJlV2CLVWA zo3+Qjwh2EalDCYKtyBoa+aUtKM&HF&WuNf7i-lQBQStt$?su|i(o5gJM;&+{rXKCG zTNM~dyR^ZnB@LwZ)h{{b(boEGbTK~3qIAG~jYGp&SQ5-o^CcB<2ZZ+}Rc?X7rm}cb z0{!LbEE9?QMc2ad8HnB0zL$Mrq75zHZS7Gin482D^!mZ zjs~yILQGSsmh@ndm)vsWZOpibRzT9*mWGYDWaafTIZ^Jr3FnjCcOqe4Lk{Nkjxs){ za7$Q#-^va&ldkJ!k&BK0I&Aa8KB;+-*CpRun5Er!At3g3`sTq92wmnn7~_>U$yXDz zG#gpdI*hA!)&#rRV<`;1jn@NP&b`>Q-X2J?{4z+5GYR~FUcDNT_%+M1Kb;v4gfa%& zxaUU+49Jw0*)9ar?Q8I?o!+V`JR}L_)POZM$#V}&!wP-?3wzJVk$a&*QbnN_MmA;F z%tI8CX_HJSSr)BkQrpTBs$CUFCJ6Jl%r6z)5@8O+bTojRSB49O9kb3&1*~}QL6Sm0 zs^dX$D1n~KVb1a*Nl1s1Me?Bms+=W;C3*8^XN{1p71)*HTEG=Ver}t#62-;o8sv)q zC<{3$#O+=u;rH$s#4lfOZpqGaCPZu|0CB&TWqq&y@kc8VcPy?2ZGt@=o?8bQsxxgt zQmw#XYnqMZh*$IPM6-z8M8g-VyHmH>G6Fm2zY}}e)^Np9D>EctwK*X12ffz>OiEGW zJ@8|r#JvKJ8X6+xVrrM!u~3raLV8l5XHQ`*#k6JqFYgG#+D1k11XXUQ$;f36dfm~S z_;`bF;U~35*)O#xzq;n_nf<7?klHo}*z8)~D5m%a-BWy{Sy$U%Ez;c(rD zyY*t!(M5ME`jgM;dp|RpEHgJ;6q24I)Y#%&zE_a=0AN}9YuuDLEP}nkjFxYd?g8KK zE+&U=R6`F}0RF|Fc4c(`TWZmv+dP$k0B0@up90QH1c|e~M}=7qs%{6$E`axN8N2)R z6wnu#$A8Fo(0Wshv#~fyDfj`fc@&>6x#MF;1sQq}(-8m}crh_wvccbrmF6ygH&=SO z78?=xe*%Fmum$gIFHA+NJ7CL3i_EB0Wh? zfEUq4I3xcATPGOL1Q;>qOOI|A79??SKS=wUsZlxPNS!P_if;|zkN+iyG{(ap z^C^3_-+~$sB*c9%26{GvQiQt}*75G*%~OCopNIEPd8?iQ=rKH?nv`k0?UGofwlXf5 z;HwOdBP&4CnBl7^xn9Tx|i?L{1ONKt|r)bE;u;U$EIn<#l3}1z{QCZ%m=E*%kaw7nV zpUx@AvD0LWV4Yn0jH6E5p&fqM+@->Q#F>1;W$}s_lu+~GB$cg8&MTnMR>=SmI-7~r z4o}hXO_*a_BsypwvwCU3%m&_y%b~y5nSj(83KZpUpI+?4M#CE_jzeC68uUB|3_TE?n$v%_7^_OOVsdXb8LfhcJ?m82cid5B|l_*Ey7HSo1fQ zR#%hZn5;~sittJiPlkv8gUFYhHdM_93#(cuXN-1^5k7b2@}Jn82F|FtQ8{lPw%}+X zQ0BygpCW(lMkXn&Vq@Nc7R&q;N^EU5P*w*ha4MRKoIq(nawbF2b$;aF<%=WK<7{ts zrJvt}BP0!{V(O=>0hUDbI0EV);vZ}l-r>;nCq%&2BpYW7UX;@g0*R|!paPZm&camr z;wK>Gg%RrD%>cmliw(Wr*aoNq^g)eT8=4L=fa=@nJXMduLU~$b zOykOT&nHNfGAxDBo!3Gl4Asr)J)4wpDbsf2FBNiLev%2(56NnNMnoJ3isY3_=$H>F znr^@ERQu~*^tYlA?SPH9jM^Y8W0ZSpjjI|qI645&S{i%ZZvtra@ROx($-iM0@cC{n z5S)A1YMfx9dn2H+96WKpwBOcDO=rsatEeSu7C||Cx!R{~p$l9aruVs6x`)@8 zmycv>rM9?cJqqs2_P}|KIr+jDdp*nGxKb8RV@n`7L^4G_wy48QY=Ciz4n|stQxczC z2XU}Ga3}JnFWf;c?Xb*u;1+Ry0#x2L7n<34swn{k+}i987|ngo=(`kjO3X237W9-2 zK&EDOT4<~rcwTvkAH3b-Z!}UkR7ApzZ~Owv{3odHIqGi+&frS7K(faLJ+Y6g7|CXQ28bJaYYarGFC4-q35UB;Z!|mqjz|g zq>0Z^E%_5vYOcEc=!LF}bP%sg2X$yxcfi7OfX;3F)U8MPsjUlGg@h#2Q~VmdsNPs;d3cB@wW)9EaU=C}6Ct+xV!-+U zY&ro0emobd2lVu!Ix?%NognmMo94IvgA${3z`aYPKLQ&t~Yi#yEO-vec%@FO>UrXAr(Da0P;^ir*d%7hT0)`;wIZm0CS9= zgH9y(c@5|?mI{WfL@}6-INaQ=3-rjcEjtsRnRrbQN?lOkP>=@YsJw(nNrBmM4o@I5 zIYHx4-hMiNM)Mr`N7T)v!s`#?IZqR(b+WK}evn*IV~<1#fdMo>yXUa8MyC4Aa*OzF zxz**VWok5KBmGQzwOWO=m1WOVQHYoqw|v6?Bolu`;P0Op_R5L zms+Rvk;ZehYSE!%r?ijdLvGkCx6oP%Wo{j1fKPAxi3t#E0jwcW3G9|1-D-`!?V?=} z%(}Zl1g&?Gje8sP1hi4a%V5lw@GR-ly?Y3&vqk78Or}Y%pX>1DNXThxWyqDli`1MD zRDT+9|A&y&Dh6Eav)EEK0~1>2wwNs=4b>VU*bG%;P^yl+(x^CfS>%)#=#^HNXd4t zU5Fibfm}Kl3T+p5XZ0Gi&~(hT7LIWni)2f@$WCH5*MpnX)tL6Mktm6~YYSft?;8pI zejH*PzHmmN@tfYHw=fv&u8<8x!35Xf1?KM8h|y>})?i0=SYrwkk=-aajj@iC8IOuJ zS|GKE2f01NNcE^sXUUg05`-WN&s_#N!=gKU-m%rUgOA{8TnXvF0k-X}+Xj~6e;{ZB z{qFpu`YdqY(1cXJ^<~jt&rNJ-`Z1i7vq>HIn@*`#?I>0+mvDew=AgXa0#&89|(XN>DY%pK17_u@?fh;H0nkm zFnd!ceXCO823{m)6MFkKm;ItQy-`oUt#IelN@(89;#4g1U-}kPGMriEQmqr8V6A+0 z^5l}xuazi8hTN^SpVamou1jv@6X9oIp>Bx#{&9uA;5IiriX3fFjBu;+ey*N5L~1No zu|_XA%0OtgS2TP^IiD56bTQHVcx<8q3FJ!IPMs2Yg@ zSD1ljXI28T-C@{=@<8RpDJ}km=Q$|!y%rGxPYXq(cH!NWzD=)PbE0~H-hX(a`n zF^_Gj9(wwp}b{EmAhb(UvK?x*e^7tSkl(_B4>dWC%7Mp zTXE});AW0VDvIH%t>1E}`%ag@UJ@=CbzBZ5U-AX~1681UgStv?yv2Z+&I`L=s3ZqV zV=9x-k#N$)>jry<)nxoP7n=Byq|=K4bKF8^!Uyp6wZh)4d6)Zz(8Hp4Ipv(DMy;gT z`=bg=GfsH>Hn8*6Nu+5w^FxYiq^X89FUuH~jG%&M3pG3bJM<22uo^^);w*>z9*{37 z2|HF#okTUapJcp;>D?YX4uZT=+(`WuM+g3HizwnGc8p*e74vwMYUx7VZxbnlX*lA& z^GL<95A0IDdC@b-s*FHA#l#jcj4k24JF={B9>UGNaP-HZbqE@VR$On{VWD2cx#NF+ zl~WAyRLV^s8SoIhqItf3pm5j+1i9y;S0k@SpRs~jFFB){Y71rTp2Omh=40R$O$T1~ zo{@mtGk;Et1ec!5!G5PJ?U@3pg~)4p-k)wGU=Z>3!z437R z`?4<19UCcRIKG4E>v`g?N`4RZ<4sRi1FxNl+2N6Yk zKBr%!6%C9Cvq-5I;1#|;qFCb!g7;*fF@FL%Pc;SPJg@mP-e8yNJOrz4%8HdZW+s^= zUqz4(SqTe-=2QwKDBUiARP$xK@(tX=YD21-M=4261SuYW!;XuH{Xj{hqt`=O7cj|e zR;q3AXR=1{xwobUaBU?)&Tvwk(mE(%GF&J^+iSIOR3edz*!(mdPD`$br1&pQ?^1^1a@guI1szyIAl8Bz)9~zBe5lK z8B7+y$A0iQMs*P=@F?B9HQkHtlL(TBwk&ZHHsWeJ8s_0^1=*^7I{lG!;Br-&AFhm#HZtOq))|-?!^NIQ9r}h?@F93_@o` zT7Z-*15C5fy0`-_Aq2C?d2Gt<_XI@WpxMWDzU_t_6CWJ6o@h%M7HH%1U&}Ym1*|9^ zl!x*`#?th#6hdC1V1FM2`ea4?wM!RDNV)3ZS9<~o4`9DlNeO#Opk*r?+EOs-a(6ILCwyK5O}I|(>=U0*Ct!bq zyG4aLlTS`k1tu@Z)$%pAyDXs>-Jix2NeA)BRy;t@OoNqEPve#ZcU z!1;Bm;tp#~Ep-0bRjbzsKs=Uj4LCyi0Aycv|M|JlG7bg}Dl?e&2-?W!!dj7ZmCWz0;_P*qFg^5bLz#2x#C9AB^F;3&|Sj+3hG`cYZE~ zCG)UaA^H%)jnzfG$Ab=IK;ol#@&Kw2cy5iMw7 zeb{zHEo7}D6wCY0P_2xIQ)Z)F&R9LC?0BCaFLXqi=T+;})rRdaix|YuKtqGZTl)Hi ze5`<>p^e1iYd{?#Z&7u;eBvH$UU9>ofhj)6$_ASAY6~=_9=s2|8ZyQ!N46mt(1Tqo zXW(PYxMw~H%&Rz~c$6z)!sepNEL^O)4W$!I_jrlexM87{joYyojH?`7d!I2Jpztfl zCh8{~hF7B(-{qzi23moRvX3+!4TQZ*N9oR$F!%6PjXZ}s;x+Kxx4DVkb~>P1oB`oH zr*XW|=QT)5e^z@;iq30Dt;GiD>>4Q$A}Lo}kXF)xj>VbP^G;0HuL6E^`Eolh_PP#0 zS2ypG)IQX7$zQ*5(;fEo^TJ)de?c5$RwmaWG!MiUfPPJwe;!~f9YbwN&<(_wLNriF zJ#BmJ@k5C1xT~ANsG7hSfK`GHMBdrW|KY57fuZ`W+V_!P0xr}1G|Cmt_hMoh-rP$@HAr@H%*iYu+MpS7Y{6>6X=+TCtL9^O7Yd5hEGKWBEn2_?yA&AaE`@@ zcO=#3vgDd1qsXBi-%(-ipJEwmblM%7Wqx*)qUw{LDU8&K>#Qz^p%=6L4Vzs~7=&4v zcGQZmovb#W1HY?hDIaO{oxRG{N{&et3^GV(IH9Ww?J)?y@ci%ig|I_O=Y7_dueIQy zZBe|F8*0Sijs?IKFQn##8X2%tFq;Bd3dvMz3H^>Ry3dnwJL23|B6mPfW|X6090$3< zB&eZfPRnk;6YzglMQogOf()9y%v|3zQsUY%4X~eI2aXsf&TB|TS^a9YiONSOV@UX2 zK|h3`t5a73_Eyr?EOhpC8IYxf<>>W%drI6bXFub~w9sNih&&LSstsT`Kz^F?$8w0n z^Ffi)^zd17Osi4~tb;^P4%47NP}qB`oDx3C!GF13Y46QKA=ChB2 z&P;v_Y=9c850=5J&gs7+y3cO*b&uewey@%q69avn-ebNHxUM>MPI*iTHI;_d+f=lA zfbM;cdW8Mw808cTeuip`Ezq9Kd7FkJ(fiUa`iY(ISXb5(EkIy%7NGo3E2)LAzCPM= zsGhGGAvX@rK|{R?^2~2qRbQ$J6OUG{SF6kCSt7ve00YZ+M~e=*J*b%h?6jm~&4xJ8 zGtWa~cNzwuZuxDBsYv&wqlKn*r!zV%ZdmTT=G{zrTror#pGiQA$K&qc?>4Y9(&$| zu;=afzZu_u@eJVv=9yifoZJUDy!ax8zQ*@1G{7xxM2@Fj=+;+x_cz3xGoPf96x?Vg zj08bk-783P4RJOv3kn4Rx|UWTFdzLmHv@(xUKC0yH)@|UWt)uJ3TKcq0DZShksQ&l z1C)1>2inh*RAdr_%kc1%)){Ox(FVcKqu+$sE`G|z*P1maZ;@-(@O!>!BClx8(jb3N z;B;eDg&mT^i|i}YywDf;EWtxLNr+?`h;5pHk=fT2#d5E~f!q|oO!P%%_ZoH3Ir#c- zIi-Kvxw_y9PiUtcjzRw=7RCatIPaCr`tv6tYH4|@*ay{!68INKhAYovlW-<8xFcZ_ zOhI38+<4{mLdu7-Ibh(QN%|rqVvZ6#3J(tyB{E&XhrcFwbLr%O<}$GI`0g+Ut-0WM zGV{k8_$P(Qr6?YLTmfN9=3Q=V5|my5A=JB{SYR95b@cA1kU?xL{IznM7GgL95M$=* zEAbNNF`3jLs8QogOM`wx$Z7`nF+-dX37@!=4d*hD6<(eG2T>ljDd);Lc$qiuxh{aR z!{mjmLnj!F{@=u&(@#B9#pg=i{(p!)Lt*)vAy1$ii{QhtO^F~uBJORQw)#&sX(%YX z9p{Ec8Hd!~Qc*nE7RSg^2Xo28*8EgpIig=Zt;9 z_X!e)V0xSb?(lhS5~U;7z;IF=^DwZ5XjA3O?pX#r0y-+lSFBsE@L_Mv2;7*-`Xdx} zsE7igjt=0|ixmGk$~MFYE0*K~tAIIWS8yX=sa-DpEbN48Z{JyL?TpC+b|SXv>cCT+ zr?GQMr#w1%Ne*%(E+s|ro-l;!qhJx?P+;U!wLs}h2jsiFSt_>?nh%NKb5POG01=3^ zR%tHEpaO`(t=PN$u}k%cALSSfFHyV1f4VjrnN8@XMo;d*$O=s)I_1uYddqL(bpz+m z)4dVza1QW*)t~yU5BB*j69lmkK_+rRpyv=ah@^(AVl)@GK066P`C`DnVIg$U%~wVg zl3B}vhdfzwt@2qTsLY}v5j>S3)SC~3U=_MRp&YSkwFxATwQa!sXju0@HwDrKdsv47 zw-1ii*mZBw7d)+d4JDPsmARt!pn#?FeFPXs7OVdIzaj&2zIY7t={5?(I4O59^LY>{ zYq-0)<$y{|?q$(IdC=rHPS}O`fcyM|28cgqAra8!DXfSd0-$)!!T`cM|I4pU0nufj z=tbBP%F9uVnvdqrThg!GgihmG0IW2T!auf*0wXQ*bWOZ1++i9ZZFBdPYct3lXD5`NxX$Dedhy@LS*MF$zZ`I|a` zKT){QuxC4Da?kq$TPFCXmM}Uy(mW1ZqZo!q; zSxM>?b^~#1lsf5+rNIVa$Q0?GnKSYdRj|Ul(@VYl(+*P#?{1b{>woj#6u6lFVqGuP zs95|~gysYhqPx*bL`$RW3L!T)P5266m#|zlD&6y}tY!>e<9H$<#YcZY3Lr@F>u7Ma z`#3AB?_|#zeKxk}&A>@vVnb9^P@LYGI1z9~XlMp>{`DHR`ik?6A)IfuPJ?-YiudP$ ziVwf6o1NnFD7r1hg4`+qs0i~AcGv=4^$MB_Tz_^Rl8U~Gh_I725G}vDeN88U^A=o) z7ELo+?9nB70(bVTSo`oPtnH{>lU$pJQl2~~`6x(^aE?cPl0buO5Ln*_h>Bj}T>@#L zgl>>DDO~)>o?y_{V;wz}By4q>#+x+~H9SOr60?jtqkzv!Q-}tH%vPZQH4kRlU25ZE zu9jC-m`J>|K=2YMuO1{`lgovuRR0AMq*>~Qjl25B_f^RG+l*|X86mH}n}Waq?8`{r zKR^FqahJiOKe?Lv8SJz-WH76>C%oTe7HuMrG`2f!JO7P95;c3kj~C5;d0mxy#|j!J zo^{z#E`>O(g8I&nB`c@}UPbw^iOxMY69~n&$_ha~tIwX_lOX0mi%hWMUe{3?A$bVv zU1lNPDAG55!$llm9z`a_OVkW+L#G!pk4+p&i2zO~{@wO18wX1*KTv+Uf~uzV+ueOb zEyP}#d`NY^1`wuIt|wHw*W{qV%*Q8q3N>jweKN#5?hUMqHKlY|OJmbn_5_bM-}6EZ zkuu}wLD$dyiYwp=x#6NiIhHbO_^GIY0!zzBjTQB)`IZM4@2bu6%0=!xPXZ9}4&%zH zASE!Qtb(QFV74Ld;)9K?P}6;K3=j|0N3d3SSCv)iqA)tAEBsvMgkKLy1~a>UX>Cme zd-`+JWm2j-xesQJ=3J!m=2N}a$eN(u<}h>{Anc1Q)jKG(zJlocHIOL)U8~wimA@;x zEja*cJ9}3)BKHfamyDd72vz*e>(iiGi78kNxCMCTY)Adh;xD?O9C*15l~M-W%N2r{ zLB;pD`<4GjS5wa@ax!R9K>g{&rTj}!W+~l(Om>a0gPp4yHgwhDLza=SqIw3@7H6Z* zJEQgi_6!H}%y9iqNtrz|Nih6n3_n<(bwS#AL$@z-4X8;ECi#u_ynSs&zY;>r{5erW z*-Rqbv7dz+7)Gvb(-3>?Qv%3x>a|kjm-v(}FsV1XA{&Z@|2DCN7G2d~hU$#?k*Do4 zxwkTa;1O3IWN1^+=pw>uRJ47m^oMD9v#%{T`x8p6LHe#B>CQ9^X4XO# zKLe27cZY%S*RA9lUxHH}ejba(5Ca=epulFE1EHn>{cLH4e2wa5(L>m7rPZ==G4PGT zFf^c+oNDb27`jd%%u1T=D42}@aYA|{U1%b5_egU>TPloq>0@38w#cQu#bjRW`4M3d=C34@7msL@aGPMC>e(`!yORZ`c$dgOzbuJ z@*8<$tK@l?`=_g~#u0EpH8I2oP6gIC{0-cbd8*_RR7>l3*A3}kC@ry8olS6n5{jK4 zbw20|nmUv;9hwCM`ihH$Tavjq;IS$k4)NZeoh+UhaTuSmDk+8XDpcT4Ju(xpznxb9 zrDmVspe!7tI4cffsr1rqBU0bqhxfYE3tNU^VQ@HEPLHO^g5sfHvkI%ch3dZYQ1^Y< ztlb_5R|d3~UZxH|i+yrdoC5r=&<%kdbucf%ES|5#CqgJl5lN#l6OFEyx4eudt5smT_x%E5|%shspHZ_QyQ z_L(f>)n_{sO(sr$im#**e-@0oX#Q*qNWWeQ;9pdAh7wzwsL*pPNlQP0nyBJ5_#+*` z&o~(pCg{e~IGZh5h)6xy`Y5l)Q#s>u1TGo|tdCxk#iy*Mvwe2=WS)9~k&}b|H1Okk zW4NJm{@Rj?<_qhM?m)bQ+K~)Ww$k~(FYx!Cg35Vny1v3MCD{r9HGZgCK|1!*CwYcU zdU;XlLi+gLPshe&B6(6UhIkKkaK`WfGZe@UcKy1HahjZY{`kAXf(5Rk0 z!8K{qPc8Js`AJfD0CIAWL+CzXJ_>t~346fFr-nKWfrAA2IJZf+1VBWY&mcn4`qgX` zRSst1+^~o?FFRg!tqO;6DxiVUvR1%^&~vdWK+zx+y0uZjbDzOSDe|LouFBjYyangh zKN>MXApoS42X~cR!$HL~PfC79Ls}YJ@y*5~Jg{>@*nZ%QlwkS(!Rm^iFdGB&EOOs3 zmGys4J}l-pzt2PWWePapS)d6z)?0m+ZH$JXC7y}<1t~swMpV6BjHpDY3R#>`oA42Q z8$hkq4O4cpKy>USl&(wtTgxu+GFAwUCWpP^)^A4v##i|32&a~K>}mj*PP!EGV{%0p z3mQvrrx|QRB)oKE2?fi}@&FvM(#wfpw}6BbLfcpTI>4QAv1x?v#%up?G8hzBpia*J z64!$@_`g8U|NqKhAiD_YBjG8y@_+xOA@t)a$1Z+DMTjs$2(}*IIyNAM^G)w4h$QxZ z04-&9W`@fo!$y-0sdb_HGn%cfBw@u4EUoNDpl>(jFB^bSoA1B7r4LHYLlpl2edmd!x6AFj8!uCl z&w)sG`u2U4UZd18Q1(0mDf$<2z9FF9KkKg+021kd6GMzQys~s=U!A;;vOx|0=hD8y zyTv1a()R;FFb!deGGE5w<$Iq-6iVj;5opwso*SC$Nlst4qdbw}IkFQxaO&OL8G5np z1Ow~^gU5=3HywP%S*`bPsZ{x^rYJtqE9Nt<-_@jOiY4D0TP8b*vd8dru!b7qzueN} z(2GCs^|ze_Zec9TIQM83RweCf?cyQ63ru?WE-mcUSsp2m6)(_NpM~jJ)x%S(#jpJ? z{$bL|GNr+DH|MHK48VH9JTx9#SjZFk5?Gdq2tMFZSFkON-rmpo>FMbadOPuQP_s$M z$O#hGng9^NJz0ISrvV7##k0_EH|cNIeXt-pup8=qWU51?K ztO1lkioU0~s_*t%(~o*y4cv1tH_f5KooAv+RS26+cjGJ_pv@Oi(J(*`Pqd^R0CF;= znk@fdwCZhhbB!!5O=!^l#Cm>Gi?^CBl)VApg*!FubMz<4v4H)$y{Jw1Nf(J9V-VYM z(bT>D+w1^<0nlHv>s3?7Q1J*Ws1`r{;n@O$W6}rbdk-J|hqS@@rQ3yZz%!53(G|m} z1o4@u>ydx21@H}Y;TJw<5EWyc$E6L?kB10I*hqoc(;Bh2yG@-_Ic47trTZ+zUHBVJPe5lym+20QNfnnUBcGgPMosuPGCVVn6G+;u$uDm| z6nfNRdL${Mg+QwF)3=0)KZRPN*vs$*_EHRWb-<~*s(${`F-B=gpJUf3VDC0Gne101 z3Rn`XZn>DI6o;z4+<_TMFGrZFeN`3KTRnpoFLwCa)J_u_w4D3-#RpcW1azZ1KrQ zn;S4d2n8S7XCwq9h!2PE+M~#dcJ6 z5;3l{KCLbwzxr1&Dw7G=t@tkLo;Re%Q9vyIBQwUOU9OT$mPr|}S~Y$rq^pu*WFKm$ zSuo&@);GDIkos{)CBUjB*l4^OD`7`!vmQ}~sbuJ}f773}#H(Iky{s_kWtBMy= z=L0_lwfi+nF6$o!0km}6xc!{9m9~(m5q~ z>pO;+f_a3vccNUEpmT;{v#9;I_M0@`gN(VaM+!TjnsXzPX7l1wuS$2>UnL5pxB097 zFv{AZ(>$ngY9~Vu{|vJR21?;2XSfAdvJ4PES`sJ5BlezD#An=W`x;&3Gaubiy>kO8 zj2g}AXoUU7hCZaSD5&~V9mO7YJRW@I`{G*4jscEY`CM%G0>fqmGj6NR#C*=|(#*G0 z*h*BS*mKq^&IvBSrC@)BSI}aE@&HIhT!F}w44<7;NeDwGt?)B;Ts8I^N4s&&PKK}1 ztBQPvY@-E>H>uY*t~&XR{*|!!O) z*wrRsU-0Lq@0&2San{K1if@>_Kd_tLaQVvs^N00b?$xvD$M`e~`))Y9w{vG_emTAl zFDyt9P2$0ZiMp#=a?_0@`JBLY_@a-B-&dl38<|e_6kR`H{)Ul;s`;;x-b7q@MVU%1 zpke$`>?*uo0$WkRab02{icB1|9n3S;?Y=Y)5t2XFl!Cq%Mq=Mi*966#ME=Kt*_s<~ zQ)Cn#{%PVFfnU3o!D#>a4We|LmFsaG!D}J3KO_;>bE~#IH&qzKnOnlvqZIo&+#Lxx z5FDXV<^mH=-I%B~+oaZZt~6xhBM;ojCqX{m6L1BUy@|#fJTQ z@%9$Roz@jezVAZCKFn8AhvbZL2Cg*}&X0-DX&)(^wU&3nw*#X6DSVh3sRJ0Z{+uz{ zZ1W4Viv%~1D7BjT@3024QMUK8Jd2ds_;p`SZ7U@^HM0nNuK6L%c!i;L>;pYjS18-$ zK=XZHp{^D!fy!x8u@Pnr_YdlO(5TLo;GQ(D$Z&asHPYJ7nWJHLhCN;6gs}&t>zRwpM>GAg2x;a(_<*qPrcpq1BX(>)8 zD#m?RBD-v(aL045Zk#LJRQPxt(7LOu;>ZmsxyMK5`g=sQ*z z(2aze&n=1dd;=r_#7Px)2;MRh-(5br*P6p9T?u4JNgO!&NiyMK`X6EO z)(cE?i++2byPqo#jb&pASiAM<1OzE$)*gm#{2eYY@^gsrvnbC@M9Lfx>Rvgyml!3( zvGnV{%yJ6OCxVC~D78j+6&Zm?1l&!L4_J4DZ6#I&D!d=(mJ2|tBKF#SkctZIz$afy zEKvk$X!d{FdvCsBbZtvW`l-lJJ;$RVrHnnrVh|pW%q4i({%OZ~s!+-YSCW&|!_F`VOfLQRo0yRhPtm3L~eio52J`Nce~`*|BOQ##L) zTeHx5?pgN*b0X*h2`D2QrRucuXKO+whD0 z83Z64n+^3@qwiEtBSp|1%!XGP%Uy8J6M|6AEaJSvQ)s7D7m03%A**=D*DZe741xF- zKS?6gyO+#NIx+|@^7DTu7ogZ^wZkoF6(Rh>{NBT>K=t}#ZEsx}I0S-+&LsldVu7G` z7?_W5IT8XL2ZPWIg&t{V>PN8>DxodKSDO{Ht#d4|tWC|6uIg5S$r) z;w7;L)w&5?V=w?fu&6+^^$}b4O1X44e7ZT+o=;~{fSdz_md(WR7Rh!=85;qi=Pe=7 zbc!`<*Y)5P=6!0eJo$Sz1*sqB9>j-^xa(oz(%Phke%7cIMU9hX#&-`p;fLmnP zI_?Xs(H9KkNVC8`46RdrO8S7iu}PYPE*T|@?BqGxr=P-^3Wzo_FZ+&?ThzayF_f~p z$AwDHTzPPwsEe(h9`l;)5Cu_Nqzy;l!UR8Y_^CXiFoNP+Cs7&{CAl}1T{0uXW~0&} zR#|ABnxPH`>S16x{!Z7TeSdRNB76bN5g6dRc@a9Q3}F$6LnyI9|58R^*s37JG!;@& zu7Jw9e-ws>nF(z9l~fbHfb~M0vj{*CsF3vmbe71qddvyRs9_2G7-1YCY-B6Gkj)AJ z67EZ^kuN!J#Xn49pNxuviUqn`6YvU*3o$Jk$WtCf8qQOtS965%_2D>n*Bl-|a}loa z<5h=sKxwBv`%7asuo=Z?ZU}vM@r$%Z7KsDSQla&cqxNb_Em?(CrH`yKg`-)j{DV(mkTeymH_2vSI#{lcFJK?4h-b#XYRtrOzL#T^bB9B}Yi{R2Lc0p}YaJ`uWow8f#m%RL-We*3kVu3qSB@=ju z<@a-N<6s;7@RGv3p;{?EL(CJdDTyQVTTqQV?R{OV#kw1>v110>H{rAY7T71R$bSD$ zBjaCNKkt(<+c6RWDY(t5Ah!1e!pNpJWM)D95ia-YkZ(!N|^7;_vL);<(t2ym%2w*hCv}2prj2Q~Z$!vi*m*&sO4^ zodfys`~wD~GwHCXMN8+`wX*-kr(X*kGd;f`nFl95_lR?++EdA2^y0KjzN?=U5nQc8 z7f25aZw0IMSL3|V4}QCPOc6thHIRuv)J+HS9%646gW)msNNZ!*ke`Q!F90JDySd}e{5aQQVo>Tr~VM*!~|ncZcyK$7dH++l%$IbdnY zo;KqD@5NzPRBR)n)PqOd+K!J$Cgjq|@!xt;w-aEXN1y|&&dW>X0^L{-=x3^Rkf4Cr zxui-#8G2vgZ@Ed9ws|hudw^a?H8s4=Hvb&v02pWh_~iM3=lJw=CoMSAT3F1<@WF!! zy$M7gJSdhW4~toJ04-)A&W?Q51VgmU3Wi9=@6u&#>}UxG!7iQ0+@-HUW5m zx7j+pJf!@luwS5l51`Z|o<_lP>}3G8pHa^{C*!Wcn*0^X1cuOWy$FfOJc#pV;YF@; zYWUuV#0t@c4Xy_fk{`Z)_`f<5QfKtr3%|)kfV$%e%U;t<{n8c{*rkJ8bPsZaz`X(3 zyRjM?L$r`Y<^u5IbK8NZ;C%SQpMTFT?kUOz(h0OL>VoRojiLGL0Vv>cE}6E&zWifF zSjef;ez1E)=Io7v!aSRk>$ib&l!FSY^Ypl{{_N4g1vGOK@Y6T4d(F|EJ^;u6v@B;2 z0YX9p%`9y(x$~{q4K_$9WhID4X1G@HHaa_`CGc7T+b?2aJ>OBG^{jUXLJ_oF^r5y3 z0I5S{kAu;fPLIi9LAxu<2`}Z&GI0z&=oxJ9y|dN3s8t+Bb~s+jJ%>+7Iu5I@o=J@U zn3!;5SLcm-Hn8$Z4dG4fYu-3n1Ak&B{SF3H+O)mSoq7-$3Vzjij%pUXFD$Vdtm>cM zt3>!{*Z51CssF04^Qb6(I!@4o4#W%5Z7c>>!VVu`7#$140W3(-2blb2%E*PP@In~X z{8XbC4qFi25je^?5*U*^9Q)iD&iadRly7}|p9}af`|-z^<8Ss#*(x=#MGa^nJ_FwK zJ5T8$P`%fJD7+YOz5IxX2&0#wDyX=xOauntg2=ECE}b*)=-C^cV(c)a-{$Un7ds`D zt6kf_eD60GqgSy;O-1PMysvfhQa;76z4r5eZ^a&*bzoo z9=%C}{*-c1`H)`m&P2jlr^SNhC;b|qKh7WjCA`)jae0<=Ez-n9bS);LH)7PRZZXuR z*f&jqZhSHF{Vqju?fdyigH+7e`tRGi-r+x68XDfWB(v;qzqaXCrNWEc^aOR?cNF91 zSzdlB2+TeY%g{FXz0Mr291Qc!0q*YCk(ySSctP@|5MEB>Q_Ig5Gsw2LO+qBYL5KLp z|3-(Hp&?Xk(VGS@(v9v2J|Pu6Bb{Eg9G*c2&v;m3X^`_b94akvn_^UTC+Lnm$ko|M z=1s>SIC6gKKrx~F%YIff-$g|}4N3K=##XV%8Je3yE}ZSrI^O!?LkkHxQ_2n1qXL>g zImZff<|DRtJ?bRCAlYciz06PD1d^H&vjul#{Rz+Mw>gbzy29ZHZk^@UcjZN_a5TjX z_h!pi#wN36N`1cB{oX~!!SOMZ_N4}p54N{zUxn?^Fj3?a&R@>l)6_VW711@6e1Dm( zJ9J2Zyryl-(gw2!D$G7fixHAe+A^EGw)DmV8bdc{|M@(6&O!aP4t5_Ie z&{q=YGOdyNEDNj&GJE0ITc8R`&zn$b#X!X&p;BXRw?ig=h)i;nJ1W(+{h^xIe#^R8pj4H}#|X+WK#jXkp>8^*&*{B-+;ewA#@aXO9gPD?XUPx2@`T zY}ti`ZIM(6T$9<9bn*6Sepp<0Y>2T^apmJPuYejyG~B?X5Zjj)NJ&3( zye&R!x2{9xZX%SP#ExkLi9MpQPjMos;44771EM_j$H>G|2N6 zD>?AL@r)vq>pCAFglyYjmFJUBX34`@MePFob!WfIiyS}Qj($FMeCp`; zedQCnQnPd{E-u`x%%}MFmcMmIeH)X06?txouwy9!zR0HJ7TRV*6Giv4@ZX5y$85cu z=I?->wZ=UoOmgoFlTA(ffPbF#?QCb<`KLVNaZApcU6$!{|Aq|lL)(LAIUSK+x}$3t ztVW-Jqj!_XR!8Jk9d|p3NvDoi1@)tW^)Ycb*EkCye>8{4hT~`Qr5`A=!Og#tjq`)| z6%OY7bN*!TVwQJ`GvYailZ1AM4|jlsdPirTWNyF`oAIo%Q2XsvVGH zf&#PJ_6z+4VCFH86R&3On9-cME<0{)f=k9xGl8n#dbsQ|eo6suu2DKLYlTC56JD@` z7eqbkNR2kvYJuTAYVy~-0nR-Lc&_AbLYH|#7lh}JE(jlqbEiG@6XyTXnhr9yraJ$S zXlep72w5fypbI9IlwT##0wcsOV(S4WAN)n4exSNq4lx8d00L=1ncad|M{M)v`G-oQ ziBiaU7D5Y{qfXcwS!)73YI9b=7Rs@M@TktuG7sy}qp;JNRPp&#l2YX5HE+_Rry0Z3 ze)yD&5%ipdr#ak0D5UpEIH1#``KF%Oyr3~5=D`6X0ThcI2Cj|7M`CTfAOf%;%A^U; zfh_C{i*Ters~w6o(C4oOl6Rht?_$Isdi~l>uvOEg+X#KqUyJpl*1V4~rPaR}VAeVE zPv-2G9o8Xe@_SqrWJID*v^}AtO&t z4lgJ-H|a5cc5YTLe@d}@mf|Z|P_BI0*C6dt^T`qB+%&eSnD{(`xfM`-pc6KpA$;wQ z<#H2jWj&2~06NY|PF9i3uetR(?W>(5S7P;L?4_hpvenM^@k42~ntBCQP%Rk7=>F7< zQ3TJyd;C#?>4q^0+92H;+Q zvzAEk2#nx_zw1kVW&&Z;b2#DDx3lWRQLF$nu8Q>so#;X3+T>O5j7z=(-iZnIbw3RV zm>Z3eK(ePHs)2&WK${BmWus8V4E$8ML;K#twLmD5uWRZ7QPk1+Tgo4e>U4tsxZgyc z@<9kxvj*wtfst2dr2$dsCpf2nt57lL1b{ufje@;XqYw4O{smEfT^kz9>-(!zZA<;1 zl&aSa6|pgrG$9fNE|0^U3R*^PW3)bd3gVy|Se?&ZMq^mOnquaMt_Djm1<$@&vsnP&q$TEP|k-|9R)!_HxwArBS!TY6CM_2z+ zTM!qpr1u1XxxUtPm@%iw_;ns?;pjSDE})7JGNgR-@i$>r*ukJ{1zU|3Iy*s^%Rfncz5dyWa)CgRK;%f6QHkHgU1|(sKoz#b(56fe zkbju{>7!AUmWA>0xd=c3 zpf92ZS`{&IA?0)PWj<4rmARU-pU&!cf zq4U45J9*bCzwjkuuw5?ioAn>H-gnnqud4fFbdE(6kgk+mjXQo!|S7O3&ySR{1rcu=2AOq)X;&?$H`O$Gyy(KLXqD+4~53 z!zz7fYS2ad-P|Fa-=Y)hb`_&(IoiRxBumd2ZAldMv6{WTt)TB?Qm zv`_2@=mB&$)Vp?ruo75`r-|hOm6B{1LMkZx-J{-(2AZ`FzPy3K*JUZMJUVuX>`vYF zX8c2%8P5ZsGX=AszGfC4%$7cII~@N}o#(sYxAAr#+iETYQoYk+(-BE6o})=pw_grjzjLhfcf}qBXUQvkjgEM}J|M9&NQ5-qIh+rp zC!1eTt3I%6xJHwyInI5FYxhT7LDvt)W4D#?*+@AWyHpa18e$F*KbDomD@yuhU-smy zs^TyHr44Zj-QtJ2<`kOYjaO}rZUkl1<)>@8e-@XrZ@RM{A-Sw1V$+k^`hwX9hENqU z@fG81pc&j2J%FYaa;PO<=S$w(oMt2(JA=qWH$GK4nXZAn^+{*t;e>NR@2I>GeL;F- z{LT|MsiIx2aP)374XZ#WRt~2X`yd}iFQE(cd&hnDliv9+jiOE;i9W5Le1v5A2)rr> zk#l+ULz|nN%7Fx07Xs}pRXE<5iTVEEqV&2Dqt(X2!ux)sz_WOgeIYQ0Lm?_Zq)6p(6ToJQ0>@d6ffbzd0KhiPD{YjcC; z>BoN`2}wfAIukQgTNR*Xc$(-uFwkEnU+#J$(s6B(iG1zGQBAzyR)Bh}civv7D+z46 z#ecr+5a-8p3qZGuq)}{w)>zA?f0$o1o*++k7^=_&2>uG(>8c;e6~Ez(=YW>;U}p;? zA6%cl6Wj4Pj{=HL^1jaPht@YSq9o_Aqw$Kj@KrAorXp#b%W!SayrX*?jzi1mQF!LfrCV$S2T zBAyFIE5kNJ6}jTOtG!vYBv#hHXqz-IIX9QSgG`Wn$1hVV1;9MraXC{^rn&U?`L%|o zk;)-_iP#ky*hRmW9W_dU1kucJ`h-#jNnl5%iJZGSTO?~Tme`ka8eKoq(sqk*7=A%b zNFH|<`BUiUge=9jtKGLI%R1g$$kRJMwPu}{b22};`|WI6uewB6z1`tj9yhv1}|rEAKxfJm#SM~V&Z3@Mj&W%z9mc-2Dp zIb##F6*%_5-c!$Xc{qa|9NgJ@^>{uK6PdkEL6Lr9;*8U1ZR-vYX>|`AcRU9@4E7RR z1J9GSm;8a4e#qd!nKU4*^kr82VNOcR4Zn17FrGOWd{QSETI>eizuxUvY{KWem*~4; z1riAyj9&Nl6)cWdxY{Q(3K*o9xkD?)U5?9!ri16^%?Fz&82K0mz4)#TZfFT`RRKR} zxNPB{3aZ51{zBFwzsaUo6^&IFjDgBQTw!hgwbIqDN_^$`=VjVk!~HB8>36GwcTn}M z&@?nQ=;TJ{@AY}mj)kcLdVyni45>#e({23qM9ox8E6$Z)7^}QSF>IK5w8h8+f%-s0LxjLELYp1 zaE^B=#do<)l#$>W8LI-p(5biYZFmx{Hw6>E@Nt+h`&K!nA{Rp%Y`${d7Q+?u{Sl|q zTMj>POC+n|x_AT!+5(`jmlBi37mYv9+VbLJ?5EE{>#tvy5m(&3&zDhn*M10ETpYi3 zcoYBD+zQDi5*vE+?sKF+ojXQTE`i_Jp01~|GRg*oEpkOo49^VmTj#UT78hQ@pXX_P z@#CAB==@X9QNdquc|`C}?0w59jIpQP8ZbFvi8>9!d*d|;FBlqa&!lu18hR0O{aijE zL1g6k@w2Le7g_yK4<~H#9S6qeaP#|1DGx*pR1N*Xebk#GnLN^CT>)X5<_-v1P@f>xdr@B7Fbt$_HhVJEbaXg5T zAX&1q_}xP9FL916OtY_8G%oTvj@0-PDTo|?LF-JI3ZE|hYObuK)AD3+3lhrsNa)~W z*>hb4-;SX)#hH-7uY%*Fm{5{0sVDTL3K$;dwKLD~1>WGGSLw&VxR8ARS)NvhJs<#e zO?tkguzzFBbIcFJ_OZpqp79csoZPDBP(5@UUEU{)6#9p!=Qc`pK^Wp{RW%t#;2;X!3rRK>qiH_T4$CeKH<0P zn;uC#B*Z0PQ`ETf_Jj;UVnC)G=2Odv8jWAL5A-Ea4+xd8#f+0k4hecVgF4m|>9(4Q zh&C$u)nDfg8Jn9c?z@B}yDV?eWuIBs=lscnN3L`NZzsIPZzCY1S6E|?7z~p*QFCF! zgXt$p5$E$NsO#M*7x7O-)S90kk9;0YD*d95RFlKL?%TIkJ$AEP(PEw@BcSXwJ--xM zKy~M~_BXI!WnUlC1(Dxqrt&J`n(!i(K0?jUI}^ViV%q6Dw>bC2nLR)OQJfo~+dB32B`NFw=cOhF`>sDJj*V&Ao2A@ArhF>$a}B}8(h zM~>Q$zDwwWrgUr3PEr-;+F7V)R?*t)jo)h7g^n*mq#{(B6BxkQ&oZ5U=Xja^m+*%A z;Nj-iy7WVMMO{B1aYA=sb$9|bo$%WijEoiG=}uK;w+5Osj{iPLCf{BQC-+4d(dh^G zXLOXg`V_19H|B#L9~7JijhkZ?S8ohc|4yX&CtaZA35@NKr|cw@=88Of?Pp^dNx)$t z20z*-4j-oG9?2Co-eGk*$3nG6)^q|t4s+rY42n{^$Uwx$a^PKM0|-_QZ&kdVq{^dU z?pSO3wxv`v8Ldk4onpsg@A?#L6O~uaE>r`b{qRteyXC%yO3GtaGq z!;ULmLq;8YC644wcnz?GzdB+nqYe;HCLieU-05<%C1mKouXwUlul(Tya;wv&^)}X< zKCT{wsW5h#4ZO#Ay2dzVX?F;(-4STg6AqxT6J=t_fnO+63N_58OzK@k}}VkDuhDD zLJ=Y}nTkR(E0viP-gR!*eZ9Z;c%J9?{*L!Oj{BeM&iA{|?|H7_vp#E`YoQiM%S`I6 z&Xfm(D8B`=FqqbB`2BRrB_g~wm(st`?c&qs>cC=*%A_>QCY$=!_@~4BnM}7-Zu1iM zHP2nWpOaygV5Bv6`m^)OfdDn9^o9C8_r%t39U15rf0w|q)bhns@Q>yHZYQ1#Gc~;; z)BT`t^6D<`&p3hX?D3d=i3$Vp+DEULjLAT;pw9YOGgfN<<+%OYRHoxCp@$C!P_GOO zOLozDI;*sMu-=alDF_(TeMbjv+Nla$vJaRJhmGyY6ma6@%BWPo?EOolBa() zsbp|AG8O0FbUZ6&vKE2W7(0a-F8JFVbc0h7TzNRijX?_p@`d=IWLHC!}df4$ZWq2-U519s3^)5A7R5#(o&8AfO z708@Y>jn(E6X^K7P+_5NPht15j|Aawd|Vk8aKHG)fyJ_*TpSIc$+XWogST)71~wZD zUrT<&{ZipnU8fel0rg`ZvDMF^xj<0$h3`jA#!hpWgroaBt0H^esNoC_(o4I_k-to- zVmo^&-*BwnLUZTNfuqx~QBt2B#Nzf@WrlzoVAt}~KUkQBWe&44*n76Dot)kGkwrgs zVV>;6clRb-;@_@*H>3RdCC6D?Ynb$TXhEmhWT!%$;uQ`+o>x*`B9VP!!&cfzhFp|w z-|W?YyGUFuD!P8>2+x8Aa0D-o z6|fF3z%8nw^5VK+Uvz8oOwyS=A8q=j%5lwp76a%p?7-wU)1^v%#f;-ho&FOp?Zjly zbry@(gY8`8l|Qu74o}z!4QC%Zh&P#(rgEZ>8(R};)5SY=#V&THh^dFC%s^o2Xl^>@ za(i$5ohVLnXHG#A-*!`%__lvU=v&Cd^%YXNw`cQQLUi8EY}<@~y3@k-yNO+M^T3_f zOZVJA?xXVEugPF?Dc&{t&BDe#a=BvSNUY9;xyn-cV=7lM=htm2oomGd?mXPXGqS;X z{AZx1^2cd$H|s9|JGR%Qm|tG5-)sXRg8wbB5Gq$`7A?efCgO9XEM+OkGTs;b8B{Uk zSbC^qtejRCyf_an+HTTDbe3CujwqKV*uw3t7nG4EmpRA-O z{O-wcF7=mBt6caUH^Rx~##;`jjutPEKu~)0SMdqXRGW{Pt*DX)LxDTsS|iz}av}I< zIFHq3j@5SB<~9oviUalb>XAa43hQ@2XFYB_QqDs!;^e=6!S`@$;IM+)z}+hv5IvSy z1)Wba{mSYD{pJjE{BF*^uTM;cDQywW_pZxxbrp|L?rzVwCZ*}M5x-`Ow|pRGKRCMK zA{)M);cTS-Oa6t29R+4(K`$IXzgr_c_=@d>7BAOAUCIm7{g323Zf|Hc-1@7=?7wk& zcNZ*R7ZQoPzz(~h^l*q0c0mXQdrji>lnqId<0C=250e=F8az9L4LP$dpLsR#qi~Rb7Nt}v|4mGAiH^ffah=cp(c!R&Zk7iR2GA=Y9Qh0dGG&E{GfyPts z3xGn5bb-^yqVz&4w~(sPf)s_4uh5ne8Ya&bG)<<5!4m1hf)J~-EZ_x4fd{y?Gbm5~J<^}e8zv}k5J@unB zGK_k0-@v?4On{&7NSv9a$nz6If|pWcqFUe0isGCdh4#>tMJtAqkKK?Hx>j#7H__$N zdY=b{k!tD?%xpgy1_8J*CWD%2L1%zpY1deJj>`#QB){Myz6 z2YmntDFEkaWy9Sz*jPP8u9f;b>cF;TUN zf_PVBLpi5u$196J%fYUUw zVn^9XXRv zqG{O!0p`+)d&l;GUK)TWNY=RT~eeHfhQy} zH4KmU=OU6I__0rEL7JBiYdj1^(G8o0@fmpHBs}3Jl{4p2`WJ2k<~=!+YLh4}MV*o^ zdovBbp&5Q=#?w_soaQ||^tBs)IuP5&Qzud8Yi!m!2ehkGeH>k&%hn_}dnDxqacTZ& zX;Jd$xe1(z^Tb-eI1N8^7==-yaof#bFN`t39q$LS9#3~NxD7d^sL2U7Vnn=03V{$^ zos}_73o9J4sI=X@D4fU$nBIb;n-3)BcUfN7gN9?HbxgGC`HZ2`WHevMa*&5x(XSxxH_R|#RIT?2FjHM z58#Pg=)|0~`o3TN4R^_m&O8ur{r@2vcgW>{YP4^SFgT?O_}>T&C|-cc^t1;(jzxA(AQBN|W-u1Yn% z4v;1o|BHIYi9audY75M4~+-YX5H z>nhMXPvIij#nTV1SPA7w3KLT0WN8XV19tqb9O#Gcf*eQcm1ZGKI$P^%RMD&C$j5lHk}U`zO$RsJsL$F z{|ccO)$|5x5qkZ5Qj(0@bTWr|v4)aa2#f#5X+Ov{Jazp1;wfP5Y12;)xAzZ>H{BaR z6M){4zERfy3i@ccJkxE0Ze!XHmF66el2zDRIkyVeOCR_*CPX}qa*0TSZ+~}k{g~GT zD`1t0WZTGCtVSIbm^gpwC;9Z2vf;P(XNaGE`b2nG#ZbA>CGogAaR60BT?r5h&mf5~ z5{Q};+$SO4T!j|zbalbeV$hXxApaZ>%$L2ED(U~%({1HCG;}~iXr&&(-7c|9^B7*37ufx=rC^AFFr#BR4^NM;N90LX!@-WqGp7$ zE7b0IDd6x|Q6qn7L)0c6fe7jQEg|9|sK4Zaqmnv~YB?kOLcyEu&FRo*&?|Fbjoeri z1nW^rG3v~6<~2@kqxaRV8x_z%Yw&L+*LCQRr{*gI>8l83YfS;}1;y7g&}z9OMxb>E zwmcs?|FrqaL&-;HjG(U<{J;QQ7=JG7KKl@Qg62FBw_{iB2NQMQnribjq1o!IHaU24 zZ4xU{@~dgcb!95v%mf>%4|blmDP7~rE@3t|&drVuvAjYaR#ygn?&+lU=mVrqZLP$W zMAj$VS~$t^hz-*XjOoJ$6V+~*8zB#IIXCkhj(tH97ep6h`ay%m`Retx`Jw5-Bau;> zyxa@!FouB1Z>1-8KtX>emP@-I)tjRx^t4ppnKlONmAGo}IFN3!&+{%+8g5*?4;o_S zo~(u$;sFg=n~CNCo1qa2@-b=3GhhIm)L+Zr0+!D^&Zroi`a1uTgI#r4!}&h>lbZ|r z6QLjg3+;P$ zXOGj}_(8t!vO0lBC-ln%_`^t!x1KaD`w&zTi8@9GepD1JBKKRFdxe0m#`a*FPn9x) zLVWAs`f`K%IJA;006Tm1GwHFOYN&kgsF?m)!qtj?;-yCf5aX1GKPVUNFkNPaeZ^5L zXj6}UvIxibX|jF>V9>huw`3ihR4X{C6oFry=F9)VE!r3>!=uAcpL5yb5GnO>V5^oC zK*sFhD*5za|MA>3gRwpP#`me<8{d+b9M!rImvNiJ{7Cm;$(0D&8R(EYVo~@rlKNLe zBt|!x@=Y`R>M`}uMb2m{P#13&Wt& z_VqyxuXA3KkdQQLs+=|?vxlDi29UgpxeS9}l7bFIx%in!i&^#LrHM;T}quypl@7cI0c1&j?`N8)ij8gtzr-iW; za(nme91jK=7iAUMPkuESX<~-l&2-R27o>^m7nk??&GGK+BO%$@8UIE%{-53k^XdDB z4-Krop<-n^9)zP;X`)vZD6yI^d1)G8?{FkSLanLj$m45`G%#j1>m9St9tq?53oB8k zilo%|{6Cp)odvJ{{2X<>b@Nw;+()YUt`5N&1>4bO*EgfRe6*Y{$>r4$OUe|Klo)Nj z*;(JY3In$s9J0#kwK33$Ekh^y#Oa$~{BD_DiI6Z~x%_juO4M~y<2RJ|Uxk8`q)y4t z9g_^uw!s4Y%CryUM6W`Rf+T3`bQV>WKmt1gGP~-DrdH(0gq3<^xV&RnLVnR_Q$Pw$ zPjMab_^1i}ohG6CiDXoHE%Z-Q|MbFjkKWsAZoAYD~{ zD8TIOY?J2~r?(;fE&zmaR6%hAh3M6mXot~-c4+)8fSfVZ1Sph3y_<8ctk1hoXa!de{s;Op0`iK{x@rT2RjcGmzgre1`>w0RgQm{C zIogM>%s^`CIFeqNrS{h18BP==oB?WWh5=yS#d}V7XAhT~7d_iA#hP+Q3q)P;V zx6RanaRSijnuNnS<<4I+(*2}XWn>4VSffcWl*Tk;D%B*UIPv==B4wBJj(&*dXCfoy z)>%`sQv^CQFpwJAdEAWZ8tfSoEj<4+vY^n$xM0grM7^o7EB)lrb5H`q4R!owluWXQ zXo8W>^0kxP*sx9!mqRv657prP_E>choFZ!iHY}yp?bS<6H?z+?8w9;R4tK@E!V15i z7#71Iyob66+sGIf*`Zl6XrQ#sT%wS?_S52o&$hK6bpr=2PAS_526khQ6l}~hIRukn zeK{06RysLWKTl|>yC?+bZWl?s+`E^$b4vGnE9(N(d9;G3v|bw>0*LOeY48##)=rys&r7dJ{*At7KQ~a98ck^tPDe#mSK=SmX6c zLbP#;Bp7BqtEA5uRwqy~1hz>9m-H0Ar;(b~hPx|g9H222T(v=CS^D$ilS#ULWX`Lf(ERPV0_E0P9<3kKTmEm8nF_T)kEf4mI zVi`{d)@yO__U#49J_M4T?b+*hDs8{q@^t5Q7lBfrvaDw@r&Zh?@)pHZF=l-) z06XwNHRC3Hk~9sg&AXoKH&ZVVcOZ>H%I6u5-cB&Vn;!mk%en@ zuYOwh*xmhg1~)}_4-EeGfqLNJu-sZXB|!~;skj%{R!&Na_ZtKIP5ZEq6U(Rp8@K0j-ytk3-Us%3@Kv$uzFi@>M%)5XR}4r{ zEb86|T_xpHPTndifDA?{02Hm6u5^wYse4)RhcS)Qm7WwQ+4PRW_g1)M!Rw;WBE$51 zUSQfoy-w{>Qt}wD(FUn zMD&~2W;#6*(Q&&?au88g5Yf{;&t+kCUA7AwxAVQPA5k&vzE4v@I|!W>ryxkAACd5b za4Z4hkX9hqIM9eG7$~U@2$T!B;QSg=EhSbqI8SmH&Jnn30&mv}{3?lMB_XiGfvP@q zZpS+dEp%kaf2K5;XiGY+Vj+L{)GO2}b;vF^y~DKtMa|Fms55!Q4+rdQxkBk;+s)68 zN$dCbNx`U|$j-+Gt_fUf3rCQI#tT-PSf_`CF8!31%5zD zVxd;XH5C7NK@iwEdP-ZMB%}ahl+t0>cDh2J?Z3Zm$cLcSG%K_Tkob+7%=Mk^!3>8e zV&U7V6?3$3o+^R$;p$y=hj5B@aPB(4E*Mr2Kw+>Mn3aFbGy!IDd=O~vm461{F70C% zU@B{9D=F5{4jmpJtQRaOH9h2rk~09#Cvjb4eX$B0%vG4-)%EJRcHLRoInJJ}jfU*@ zy3m76e5hdY?fL0tkb)T06KzFI3SB98g+FhrBkXazvCM>tp=Om-4^0eb8zTcDx>!ld z0mqVz2eoX}H%guZTLY0u3-m-aR5khA*2dZ(9KbomxMjv5?_G{eXekUonDp>~f=)Cz zR%2&-YYMtqb@?RvJo=4hK>ddQx(b1x+M(Wv)aSLE_Fga*tJ5JBZtRMmp9JR|k4nj* zNfs+=XlZAn`^&&(214{`D34WzgqM7rC0Q1PqDF8xRZIpvy~_IjUIw}9w0R(75}N4G zcMR4-zjvGOGyAcR*%cik!v$8Apywf_@!y3##$W^&L4xn!$$VO3uzm%N(|C5=K>YbK zgPiL5yFF+2Qj546cbplRB9?^8LAk4%bL)>t3Q3rISGocNr?z1(;cfVSP}j-Nc$sRt}A|TbXVRU;b$c(;8f% zLZk)icpb1A4%1`Wyl3Z@L2VS)slTw{$-yRYn&uzEjDiIkkKjLn#o)t8Nv&)s)A~7S z)A!3EI{-yhQu!xWrP^5^`-k@HF_0d{bXU4eT!oPYv1ij|Q{+9}-|6OO|5}4W35Sp@ zsHkHVPNh)d+rHMzApZaX2Dou*I|+O+?MN<<)Lh9=KAn8l>~R`$j6N7m^)tzvZ0{|Q zS#Hn#0G8AfJk7gy%LLQ!|M7Ij4GeNH zSAuS2crX#J5Ia->bm?^Y7QFXBEb0&J9z2zkXwLUMIioEn*Uc_nTd3;ls94SWiBAcm8hai3Aw`QBdl|HA|82yS~DG z*HDUL&HG&)mWjJ3<$|=eT;zWU7x{2gop8^z&<0 zdx7Emo^17^@0rZ<-Wnf^E_Bvd1^ z^drLd>^-nAzT|kZiSMrr5`Fj=X7?6J!Q`vsuWH_Ovo`n~=6Y8LJzrDKL_Dwl##bie z>eTF>(ht_k)@A}a2T4>#6?|FMScdiv%?|I;B=MA*TVN=yZN~DVvs1aA6}yk$ zbEn#C86ul0XaPT?u~J&zf|meE=?}?g!E|!fUCOEYYsUnVau_7#^vfWb$A8!R#VWeAqfpygj9-Zr%&CT$BI<#*FswKMo;@7v}`erP;A^HMB ztk>2T`Br8{zI%%rS3W8g+Sg}&*}J?d{8XWNMccDy3re?KOwlLVGx8zE* z&1&w?n+z)b5z!QUoDbF0sF>vj1%i9e6(Ep(&S1SBjnkNdaUp!wr*xfGtJCZuqi>Tn z1@&-3e`X<%Dp?k=(z4|~#`eMa`frf(gZ=YF_V2Ck zfGNUFM5JeYfHKL8L8UTV^THGkjp{6_9H%gYxhU2E`^B{%p+w|wpvA5NCP@SdZ#2Kn z>;>JDU$uh)@AQ}V)=CVQ@Loy~Dl~`U$SEKz6QK)Rs`pC9bf(CaQ0hR#*T12dT4x(- zfK8F@hGvu3UUVXFo#DHrY39;I7m=4W9v7}}8L(AB9kdONxvwrS!TM=n{hCSDZ0~)M z{{Gq{0JvscIWU=l#nQ+=pBMuKLR5lbwNN`d1~yZIJt|z{0aWVS*nC1k30?X{3CB<3 zXrks-xL*IbEJ%FL;b{6@;G3Jv2ldR>2YMLfgljh*L-5vI{wUXT?tTWM2IHxAdd*Iu zuFOLw`~|OoT3&)_Jv{<~`58vR8zRR(#oXTXAguy@Z-sQrcFcQA8i%iokQ{LJ~Y z7NXtSM$ftjdknt)WQe8_ej%D9n2stvNE3kLOE43o)lY2R?jn}{VGdmi(XfR^>SVxn zimZ1)NcZivWQ$gW?V_1Atj3b!RMcR*7(NY}%v+50)X2-;`g!d9iT`t8O%#w10_rCK zVR(JI4dJ2Q1Co*|@uL~gg%I!|XZ^huQh?isKv%Ogq>7kKWGOJZgzh8%M_|{Jf8m&O z9qHp-Op}C2SlF|f;-eVo8wJK7wnqC4YaNZ?D0~+QW&p1^mLF;?xAYIv-B;Ud3`n;# zln3G92daQ{1(wS96W$@Dd!(xzl#??UkS>G&{l|y9$TFZcIncZp9v4@zd5DHx!`Uzr z(3jA`4;fXqP!a~IFBQ|AOuBpm%lcjgKqhxXRGyuQY`7btdEiq?SY<+C;%OB#m8g6e}W>%X*d~?TbHPpcMZ7n|1sF#nbQr3h3P!p|3KLz2Vw2W zaFF#~IoI(xur^;M(9i10zz$CpjK}r?0{7jkYeAA&8h2XdxF|7Ze=i4%nl8z;#)${P zqSW-B9Dqd~hec^FX8-Y*KI{pYiW}Y7_R&6S$Ppa2m+*uE?W_?5VktVsDg32T<1Ml+>+fsQQ&YI?vhLkX{O5;pSF z7n03PcM!bTpO-iH0lMY{bgkJJRlSS-x38iAu>Apf#{&uN49{_LaM(UbDNA8~QX5=U z(RcHE5d=Y599U5;sQb99v*?c7U5g0o!Jd7M@<7{nl1xjbl*p+_|07`)YCW)L4aU= zo6a~I${;kM?$iTv;{V`<88g_K~e}I zyG&T_db*XPJKD3;|F6_ZPub+z~FZ67RV9tw+p_s zxaVC(N-&22_xd9>23+GE#6fIOXaka(MjXVKeMN%KyJAwvnkbY zCm=qoDZl>ltne9>jhup3+;QMHcwbv9cm!V?G0`APgD6M##&ebt+u7{6v$2rOy4qfD z*QfTM=%8$U14BS@-*ql^8C+v9EZlR0oS%(^^c2|Yt%YE1{$0_$c$p}g)#Uwr3sCJX zv;;^qdFT|nAX0wg%SCWq3Mh8UhD$I@04}uPiVm9DAPUsz5L^_=hQ6FjFudaPU_rO| z2+%mLFrp@vL6xtepmy_{T6T{&=!u{V%r2RPVL&2YtIvV^YsM*UZLY?_5S_Oj4PJom zI=3K|(lXgj=Lr+ZREMDA{c2qxsp|(b+mswTlm?`}WtfK2jneW0+DXC{;2u6|MY`g{ zkZWIDpfv^=^9R?y`E_5CF|w;LNy&A<$fRoiy7G4`e2?C6{t@1((f-U6Q#M%3xLpDF8ZS0dr1ei`j1Z55P&SFs=x=*GsKJPYo z4to1|f}$^&wYyQziw}Y4Pyk(9_8a?z0ecIfOVLS~+Hgcg{DJ4v=Vz*8`h0xl=*r)J z;$mAeYg$h&Izwu^sRWZmL?1#GrG075Lc{*YWw%jKRt57K%nuE01-PGjs5jP=nFQ50 zgZ;~Od#EO-K5iQQ?a3DN*1+e1udHJBj=vqa14d=z7l%Ly3HC1=aCOaxYt4Y-K-6zG z#gxx@h_^&5_5ECND``Ilyoq4qMi161piM!r0h4|GVy}JIXCae$rSQ@tac|k zas66AzVOSmon~c5NI=?jIXAXOTQ)R*GB!wK1xP`N^ zLKn<)0w~VGs><%IL@|@yI4S=NRG{PhPa{*|k9b}nk}M+e-E8<9VBN=moPy4eSVnE& z3r$BOu5~6v2GyM#6WmKprdU~!swj`qb(*NYT*7?U9}I2kxne+%U*|<8B?-1bYoG8T zW->|vzS}JpZjE*2Ho55djm&b{qJ9Xo#vco&9&$g#PRxra(l54!3%Hq2Wn z*D(MmEEaEwB^4$I6y7U%gp3sDsRg@}f92bW_VWbP#v~D@={=4FmX$X+YU#xF z&bvC{r(0Gk{dZ=ms<{x_v(dn8wBz_Um!IUv*JZJH0byVbajbZ%#LNdVn5b=LonQpZ z8nIMYO@nmlhYm(Q3yK#oD6btZnI(?|&wxXT2gBc5{)oqyZo1ns>ev5PqKX7uz(i)& zPdp9oe0t}!N?8lf%Mo)dvwWn5ZUGwNJ_#l+31FZ#DH9BCS%O<@tuQ02Gqh#%QdgQx zDeyB41Xa}}Nyksge<1@e#UA-HIIB75PJ6YxS7KMc7sOTBkhQl7)g26^kTeIdx=1wy zBx=Lw>5bI|=~d@A>q5&Yw~v2Oj5H6m7ejyH&n4FTi-x^R8Ng2Rfq{P(sA`OB1h|=S zMD{=&82Bd;DoY-ySwC>{u%|mW;b$}T$3|B6qKxH_KVk7dBA!;F>5T%AokLFiLCvj; zP;rLli)7#WHTpQ$c+NJt;qL>Jq>K51eYJi*GSJ1aV{bbj&71@4mSHNlZiYaaZMS68 z4)72*-@QRxI=S2VFpecn^fnC1PKk1m>@O47`E>^qMeo0}wQdb&GKoRVrUm9#vZHaI zzVLs%J3F$2u7bTwe1BGfO%JjLZi4;1dk7l<1!-6;FCPcQ<_nN+`cGX2*}wd45y8lu zB{)n=Gv>Jgrh-DwyKMkE!Ejc0cLJ11a6GHZc;QJ-d!B0_+%&Y2)c7!U;Q^W&3`1tH`2jM=#+QQUNr)*$Vamz;6i% z&j0t@+Y-A(+Km%oyd(`dhp~n|>Ed>=0xt%wAm~(ob_Rwti23|H)Rleik$UD_0n6vR zCRGW#Y0^DuFLHQpok^E9LetQCtj*9Mk`$)+RFgN~=b5({&``C}($WTB3+0FA03q$~ z#;{-(5!-liyTKfgP!%kobBaav`zVE6^(jOyGOK1DHQ)&j3%&G&I*NSO;SrjdBnf6u zj^BfdCA}$2%*~yOl?KbG38-A-Q#l#rDgc#xPM1Sg08$ttVzssW)>Xh>{-(?@*xNj% z5FCb>-YoV77$-Dvthp9yAX=f#xQ)%<*T8AnXeM>B_gb4*cTDYg}}SCcw_bf(_beHFKhLo-NhNd z_7X!?1O^uZNlJ$H@La-R**Rgel`0Jn0PK&1&2bT;yFr!#^sJ!}@K1v@ws`9EA1_M- z`+W&ki#?fCbefg5?$Yg#@RmTuDR_TES-AGuUFYdSYT9;oFLh3fGYk6HWl-wW=8Qx( z_)ai8>+NGa(oyvIk>L|67Z4I zpsI~k{a?-`!W0kZD!A-prJZ(K#o@x8ukb-ZfaYlDEvokt0zpSE?m-bu0-#XiEfYtVv7VN zx=G_NPop5BhrsTC;$}`m#4GDn!wGk8JyWD^WSsj9Wa4q?a_3^slQ5vBNXAU(ZSmw{ zjkhKwO)eN~ULd|r55Yl?x-4E;^FMi9aEoCak}R-rJ=tGd>-R?s z#u}u3(6p|gPL_6g+>s_z_M(AHOAt(YFBGTK=vIMT!ly=X!-@7Z+EKxYiouDd)tC(5 z+?Bbe&M6`6V~!KxHdDYRmF+bg)F<4T+(&?DL>cJ!JqI0Ki^*l-ZT zLm-Av>RKgk_Egnn?Yyi3k0J3uSp?UwHU@|;w#sehgK_1~88z$xBW&l%fUER!d%0 z`yjQP2u=KprL<-A&UhEKm%CRP7v=+%^X|=lfnvo%BfWz}!C^=ClfBpwLP^e8P-!nG z;Dpe$Cw7>hrN0!`mt~62Gs5b*s3+0Cxba1SeoNxp^$RpBHcHw*2+W z^ir+|+Hrm42FZ+|tr<2JfKrNK_lGaSko$w&_VvWm{WlmT6d&HiRDUg{Aocd$glejA zYz6VS!``SvV@ zuB)O?I35KIy!>{BX7|Oq@ZueX1|@hg3%oeM_iGys;+CTLqx7PZuq#&u)2H+WpqIcI z=z(x?o_gI%=~6=o`D1q7lI-z=Jdf!(-pZp5*am86JX5zA2Syj1PoL$Ne)|-l4J>yy zO{yTdV*lgt6zf<#no6Q-yLd<&N79tn_&sCO*w2EGZ9L&0Gvbfll60yVECaY_k-i%{ z{Z{F_F3en3nE|Jws;Fzx8;4}^R zLW}KkV0jwvfoxpdw$H8rh6)vD(qC}zQClOtVAYGQ147ao8e(!~exEa|rWQV4i*;x( zr|`M`{+ve0b@W~SD5Y|Tr5FPmxeo+a?yq4)!vDSVbx`c>fUSkmzOg)l6^@^cp{qZ? zZ0emumY9Jsxlh^`GCl$#Te}qgyuIWYaBfFHm&o0CM?lcnxge=@=*Bs|TRB-kXf*kW zf6~BAt=Y}>WZi*Mw~83<9%C%_DV>+=nfEF34+h;&CQ*DLI)F&97}_2PN6{rX(rof=Oqn(*#mSF4LT^5kSJ z>2p~D8bo7;7lc*yEalvU7n&hYx|F;2Zddx3^!l|tf;Tx=q&oeaxt1nH zDU;mi2KFrF(z zpzq0h4_eIDkd9G1Z3J>ysrp5Y=h{QgJZzq*uo9tsuh8ANZ^%=Ph+e46;@f$dP4W1jD77 z^$}|OSoQef%L*s|3eFVPA!fS;QVcV~G6LEX1Q;;b1_`5fQB#;JdBh`5+<0D4zr*bt%rXQNazg|qdl{I za3cvaB`|}tbYuLU)Eiq<64Lzy%{cB>$fyg-_vo{ zrDZ=3x*DVyM;?OF+hBMQ$0?$rYIh! zrkIj0cU*qhU8mOt+ArK~NWj45MJX0-(606h+pP4@@M|eQWP}Lj=_A#P;0k^qSKzZ} zHwVT^B_SWAsEv^=?jMr-+8diFV?Nr$bzU~>4%nSa9(>M1mJ$_@bJaS7gc2(%ptE2k zdWK6-JXOA|xqJ@55ES*WZT(r-6+5??L4%PER##T4JEAQq0a1day);*~1#plH+ z7R|y%1Q@{lJ14oT?mU>(Z%>08KvnHvaav_x$&rUw0wk)$zWC;!+({wRy*Kw}Pn+Wm zUtL9MKM)If$9u3g@kqs*AS!m<313z1Iq)!CehRtG@GkUq6=lm6*+dIuLUqNY0M}o7 z^*Z9G%DkY?pXphxl>o#rin{Om3_ItURWh8jax9ZfXt<>C5YnX&df;LdN}j0Z^EtB%}xCb)CFXY9Judya10 ztOIlU0X-=35ExqOxSJfOg?V8n=kOw}%_QioIx1}oan5H1yK)Td%D4p|+TnEf*fgj? zAqun}55Yd_+{Y{ZFe%H(_SqrON_(!rx^sxredvv3*j9}kWEfR(wJz7}LdeNSPt@_w z>JL9n^smMjLWof*7We=}DV&^jftc^g14<~Nn%aOAa0ENZ*CCl#ZAG$&JyG2d8fN`e zJa`@tZCB*a`hiWO6CUb4lXMua#H^4!&V@OMB$z0k$h!OggMHmC0+^=0u0v4dL}sU{ zq$8%1pv9&aOM=nW6qgGObc2PbapV9PXRSvIcPm1|$>a5FuY5Q)!9LNU0unW7ZdBIj zWj^fq84?1|==R%9Pn1>_4!A4`2WbJzo?eQ2j2vQ|u7a7`{Ug+>nfbJsyPSHRv4WBp z&rb3y=E}idP2^uW!5;%B&DmS%jwpmMz{%{zX9_;-0ukDj_u^QqGnc>@OiQo2rb8y} z8RQYOEF?2PVm)XeRZPp;#HNR8zKrVF`C-qz;m7;mV7N#jn~82Lp}wQ7d%XaP#dGWB zw;+cCVY<&;xF=+e`(gRHP9cV9|G|D8p<_Vc76);hFageMx8`oiLM;jzz5f6#8|X2A zI|qv6te0aTAD#!D`E?zKdC4Gi2Jg7dXbs0%PPBo`L?UFe=I9D9DVC4g%e`+(mEKGR z6gxeTxsMFDp-w_*K9zY%-xYpcv*^Yg1L#!-#2oT{HXmofkHIvf76#k29tVYTD#9Rb zGF&~^fqQ|UsMaP*#=o**zp;b79af-Fe#nR9twGfWjtvn-tv;mxtIb@-4em{TWF#;^ zf#TnnScg*qOtS7L#Ab(IXgK0Egkk&UR^{c9UqOYY8D{2MX!{_cY6cxkdK;qrl~`Rd zviH<&m*cgn4WB_8EQc9Ew7eMj_=qvyy#z+|@uZBZ@bUe`k6-;aLTPnl0K4r-#JxcH z6@uT)y-HT&vz{2qbTZMm3%DK77o6OHSQZvk2vpYCH6H4zdo_^CKL1x`v#>P(iSfC7 zK2*8c&J8>>X`PV+(DalCHbP!Z3&!NZM%be)pdxDoQSquF0xZ7#b`sKdJ`j*|;KWk` zDdhN8;egm(Ht3K`)_!ae*+N?$j~SzU7IryyijT+fJVjxbTWpJ z6h4~~dOG%#M6jbx2~J?F*QW&A!3DO1Z{(tWE<_?Q@-_xlsnSWD<%Um|Mdk)bo)`!M zj7`|8fQ666;ytw^L1Db#iXNgBfoB`uW%yQEx0^lUxUwUibHA0(0E$ z)kC7-N1b6_L-PT3d&2(!Cr^)Qj=nk_MWb^G3YbJ)#v32Y_ix>DXJH1P4#5*9>S$y? z21?N398t+QD8gSvp1qc;2U_Vyz@OAv7W9`HbwC|l1!S+v4b#c>Oa{CbRzBOI1U7yD9k~W)o15onX5%Y=Rp<)^+1xh($BKs>0cs!`Ei-%$1^;M4yfoT4#R7d}w+7r0A zF_@ANbuzIZOK`9H=Mr z=pOLXfj@2?#pq7_Rn!K*BL{wG;%lybfjf$h4VBYh1DP8r_>g14bBt9-gA%F;8Y^QX z8v}QrqC6IM7oJjvVBl!@E4BzPD5`s>SbZ7~XA>4md`{(Unt7iln9Owl+>f1X@SUM` zeaAB2^TECXtNi3mZT^7hF4C;H8HPd&Ypeq)6CqbLbTSBz74ON1ZU?}SAN1rzs7M8% z{8kh&L<(P6Z>*|Y29~$|hX3->k2eiiun2R#2#lW-G$yl3BKB`Z(`&UXp!oO^elvKmv=0Uyi=vEh*3@BzCHkSNf{SCmI33)t}2uoyee zOZ#VWa(2EPvu=j4(#P!DHTY|qb!)E<5OTSMf5KBcE;rrGP1Y%o2oiYqcI{rsPpJ}R^_!CRW!xmS;#lk3dNLqy zhzCh~+?x*!w-x$P<=h03p3#2=z4NJa8qbp*WQE228$2A$76fdh!6L!NxuKW3u{(E^?A{{yANm5nIOGHt_GF- z@1{?Pwh5pc@mLOAu_pwAQT-|(s5pTmZjg8vRXN~cD9{iQWA=agPK#JVCeh!zpOgwW zh>^ua?K{{I(g+LHyw5+ULAm=>V;vzG^c-riM9Lmj9_>wr{RyMyn1liiCmfcM0P#0^ zR+sS$Zu1x&XeFd?gt6u{#am9wV|K0gPJ$j)6GQZXUEOL`J`=?{c6e6;+D%MpZ-4|W zuCaV*zPBp@uYK=Uz(XtWc|Bs0*Z0HD0E>Fw7?Q=6OLa~-PhH~9yuQV<_ZE4E zHSj~!;62rFFEx#be(%Ag;)C13H4v?U@zeNVh<|W$f+^1XpyZSIfp^1YS3%NU5S90A zM(gtrzzt9(-KNQtZurSH12o5j8cA$M^?QRO7%xz+C1+<+g2t}Ms)T2>ff+agKl<*y zowW`MWxzteb3#`=Jdn_S!7`^7dRRS#yc8Ji{?n{y-w5M9zd4Jm!gDIH$wyO@Q#6uH zhL1yd1s(6WpkVsxAc-ifX#o8&%TE2a%N^xXCI>p-_N&iTR=fwTWuycw@33rYZ#OL6 z{+RF~_so`jURkkk$@f0K@)EOEJ4?O_eVNew&ZRxXfoAvKC(yFqkv-F>(8E-`?dA0d~Oh)ye>VxAUv@1?fdlt zzi%%tC++=^&F4(1aKGk(x9Z~zZ#+BdwDaoeh zB=irq^V9t1mi9%}W+88#$iR+k*GV!fj@`|*tfBKnExctG8)+vr>7KFS_kcI}nN_#k zPMJY6zr)=zYYX}E4xNxRS6MfXml+Gyy_@|t)${BEwtxB=jwEE;41CmyB>u)X;V)YT z`F@*DMxTCZeYx@FyO`hf%=4>%?XokeJ^AkboJ6&m;Q2m#0(7x_GbG?f6u&EC4{YIs zx6nbsrQ%zAqrQnH9JF$`QO=tQ=?}kPVxIXye=dVDp-b(WvEQd8Eys>@HiP?_-)tZF zK7UUC-wDe|(LchnxqPQC>muiApg_ZRf0(D%-{UgmlWl<;M&%b(#SyX~KkWbf`pICP z*>=06?HkY2$NmV@x`kWr$kMVMd3<<8Y&wae)Tb#vUw7Z;+s;}}nN@~8u0_Fee8;y+ z)$fcboXbqP`@Yoo!|2X{rHdckr}g#>7C7%|?O#>2avgJT4jOk_zYi!oY_L%|Ci}cL zZAtC!{9aXZ*-kQ{^2g=39SO(8eG50X>fw&M+D3jT%0u+Fn;Mx{%57 zC;mBTM8C;91;37N%enPey&KW-nQuN1ULE?|xNsP86qPif26)f$Iir8$XU}RfI922T z*Y(H9Z5R5N0$w|c^B3xVZ`pb?EA-95*w5Z4XUi!uqUeXVL1u-p=Aw3v|5<~~xD_GM zVD%bJ008Jl;fqBueR5mhh&i;}of6rm-7RaDN;UZV?L9Fm6&kqByeKI7+5e@8>DMMy zR1EBRa!NM`xe3J6&?WE1v4FpenwNA~SjV`~iWXVsYlkLt&WS)c7^t`hY@|3|@(l5~ zV0x5XATKlN!d}DOi-5VLe!1kh><56Y{%}9pkPg#WZC9Q_AsiO#d%qKa0Hv{~Bl_=2 zzyZ_CW8X|LV9KRNJ(Z0iEns?nv|JQ12ntDdnY0!;+ZVw`f$!LNtoK;y#37k#P! z8xz7r;PKurhj%No{sf~s{`2%axZ&QjyJ|G^c{jEApKXD^ZW7*m0sewjj}2TD#p##> z@PUE3SceG!pStPB?*t%RhJuqes5Uiu=Zoy`{|XvlIo^P(vYS7yKq;LDH&JJJG87VXxGv(-Hr|>8Z-7T%00ZfeTPQ*Ih z7Eym-;2XT`KWSMDv{^Vgmqn}E^nc;x?~y4uM;!QPZyLyA-z*APdsqC2&X0Y?FyyWd ze3KR&aCZNqJ1PdJ{O6mTzzKnC?AtL-w*u3XAx_Apci~wd7!1hrvu=6!XKHl(Bogg@ zTEGcg{3IlB1k=QQ%qFqrP zJI?*{P-?_Er%juEZA0tkmMvjtJ$kM3@W)9Bx(i~o)G>IT&< zzv`3h3O3am?$OD?W>}3!PFtVKkSn(UuYVlCX`DfGpCf#Y$UnPN4dC#fG*mb`lVLJu zdCY|*+uNGDZw*SG81iv1#M9hzf6kn0YWVa3@pi2SBP=M}D}EbZX~%KtC>5N{vg3mB z7^;&+Rjzk=jKHY^R70=JXsEdrepjMOi0_bR_*8dia9x}>6wIMZnQ+bn%PZ)dDM&9( z5o$Q%1OhdK1nm=Xz zw2#0Cg~-ICm?D4ogR*~mw0tjyz!KUQ#Z-JS=YLKu1YqYDm^1kD zAEz-RIa*SDIs9lQF89>V9ren(f*xzn*M^W>EYd z`Q?AuBTn^u02IwlK~oBe?v;^mP*L9FJM4gjW%>`vej+kcCR(2SI}Xcj=0@(qX;l{|YeNGlMLS`{h0a?2j{u@msQW$T(;8^Z zC#!_!$3bCxVfV-LEUR7MN5n&4&xQUdy=se;IL414nsY)AnFQT(Z$r^5KW60O~(o52Fftw&f3Mf%@AGomVEQ`v6NY1GTtK4PnjA|II>T z!v(7Mm4w3}`~$*=lNu07>jmi z-Fwu?Ede?vvAqqb6}KNDQb0NXmpNABbOF<<2+g!AfoIouXH^tBW8f;$B_&8~FjLz9RmN<6SI}zzHjR1f6$QiX zQS0!;o|4UAmv8J#XYm4JVhDZ9lCK0XTqjL5Ii_6K1LD1S9@GVE5xB3@QyoI7e}Nw| zYbuu*y1$;BB)qa&U+{77t1w$Xu)Z=7{;2v-&_U4c%>Jw^SIC#`;s|9^ma8kGqz-lB z<2~+7+8u?x$lvlPtOt*n`+ZIibaY&XITQt*zVNc1yVH;OEIxUdiUF09yO4T?3eWSD z#-G;hm%F)5|E2>>*P|a#gR^?ax3TeYx}*eE%^zsuOL%Afp4^Ri#r>8FvswRM`R^*5SFH~vDN zx1>0{5$jbUr_cTD=#+GEy;)db%K;zHui#1We~3|Rk>i`>QdIVVW%N!DO;&U#xtn2_ z{D+Os(MHg7w;X#(MZxAgTMrG|W`R_^1zyK#?@PH}_k@yonirf^b+%-?SdbFrdQkTG z^No65OqRH_7L2DZ5_9qjKvdM%_saeDZR&1OC_Xqr6-#Jw-25jRB${GU5Y+DZ&R8srF@CU=(L}7V9x!K0cOsxy&if6Cxgh)XL zLry#J=d?G}S5)YDimN4Lp3LS%@OI&MyIy{BorGJwYq2zK>WuAQo3-QZp$juYADzIl zyZLwjr(fE*RmrqB}m z@=;&rV3mZ|mOyKgB&@D8n@&YKPbp_Ezh`S*pyoi;_NH=$o(sI|PRE&Q0b?zuDXrLc z);HFs)!lWo6p0tC|{D{MLMFsb5d`CSL%1dOp0e^b84##FIjg`w~xXt@Axc zFYcFP9Fgj$*m@w-2yT<&snbX95vTc^dLK`)`p6o1$B6I;d3CE$3MR`+#v7oH6;VQje~6HXuZENu@YTDxDhIHxzB zYYj#|aEUXQc*03?PFC3al(0Og^E2k5Q%gv0GjvT#gzuxE`X{!YmmCfihKhmv9#!nO zkKU2*G#xx5Ef+||RnP>~jq0blbDK3~tCoLE&QlLW<~zmN&z7cXA7_dhSNQ{ga{lNb zDnW~C>uD{avB?Y1L&rPxgs+b{Lr+{Ny>YpssSHD2%FmGYyaRCAEaJ?5Jh!Kna9+F~ zcX97KYPiu<6KylJHa=KOYQA>KKyeGm^iCJYO0EU&KW<(AswwP05}9%Q#5ofozf0`)}dfaj;$YoLA z%~2q^CUiLM&QF8>4s_4xH?nYzykK3;avl4cP=M7RF@2)Od z42LN0m$@9pQ=ef@w4DER!BfM!PVLz}0rlbHkulnQBZ$L$X2W7JNPy)4t)h=|N2yJ# zp#kjTtBqJuSHa)yM5sjK@2^_9{p+A(#9H&yudj?$ z#)}TW@?+_Jd|~t)Ur^y028C@qeh;O6T2X5;gUF`p+IL@VS)-caE%ODn^VvRyBIg~r z8L7O^)-C#D%M9eS_qg1q92AIy>Y>jY3O&S&l7E9<(Wfbef%9pmr(<H zT{WdEXy1utEJ-2J*w@1YTJ~kjoixkv)K?hX!4r-#d8bh~fz+mfkx9KS&t8+KSFxg2 z%lh)UI9}E+)fP!ij&g3`ejv-(l@3f5A?Bqwr%2*d=q1iZ{IrwG6;GV)6TUX^s)_@i ztjEnr%yl;4co+COye(EHUuPhiVSnk(A$(M!E#j2E;L?pt8HpvGwXbuPCjy7ws!X>O zjI2S)CFN&bvu=)g?&q(1E4y?fV;3A0Ca=yC7Ox{6HbW)~Ol5z$?+m9z&3Wd;l=KD| zPRjY97;!RQ<>O9K-AGzEPd(>UX#H4iB69-U_3vb5CIyOY$yBC^i?&UFzIqgL9#Rk;ZzgCz z#xDn)i+^=~?WskW&T4d{tx-LuRmp`=*YYEvVJT+%e2*QpUm#!Jj(gQ-exsMHOzLe{ zT`tY-xMh52juDtqX!dt;Tb}J~<#_EYI2hamj;d)pC{G6&s&~^tJU>uE=Zaj(>iIby z)rC_KO`L{kqP%eCLf1=L*~P!&A?=xGiNtqkZx>!x|4M{`jJpe4@_^==x`^&+i)OPx z)@v3d=WVsp35BlpExuY6xi4{&Yph1N{}6?+j059Xkw|W#@{*!qsx6*+#nSz{bb5|w zM`le-KdMyRc3_)2BL98bEKarR$5EJ4MrrKrps6OuYy>)C~lrb2fd{! z(C?i()EKBINoAhDg`g{6Z|PQ01}H?woR$-w9%Ubk5#zU?dQqYDhH<%pzV>g{!5%xg ztxU#1Zco4XPYkm+$LCd+YW<^`CX||W%`LG)rf=doG^<=!+s@|@j}Mx6wSovJyf&i4MRy8gdy~E|DUfx;dVZR1 z9=;1RL#6LS+xy+)tDa_V$$+ltT=t<*bktnQzEW!3Mye_lpb(O4=x~9%B#VD1TDp2KeU#k99Pk}rZeN-_^5&#y8^TH|UCr{9KGf;g(i7k1=ANF>N4h1EZ8yNu|u zYl;wPOLLxSCR+bq8_vhM^Q8cq?Rbnkl)m?=)>qqvkLMhWUJGGTg|@Z&J9=aC74a8G zPc(VSFgzVIRNsb(hRCn(o7KZAFK~KumIKSBH%@)}>)!Bgs zF)s_~NABxW8>nhwYe@mj7OT3*fOkE|*j+YqTjzA6*!*gPnknd~i%;X(l^7G|&E@x% z(O?=AXW!-CBU`>050RtMWuRwuejW03@fZPbCj&ZMZY2x;)I44QqyEg;#*VWa@Qx*M z<_0Q@@N+(wD#Pmsz?YrHkq{qUfDSp82!3?git8IDyt`{@eC#B3JQd#^S+=oz7t)$_ z0u;F354~y4x3cbr)J;lR(#5|dt9&Ti%((h{eZ>S+deC!or{KKreX9^Ix)9aGAiscE z1`>9>I@J%~#@~w&9h0>GLM3;lxEd2+G9I`1uC=9d$>Q_e8mn(FDfg>blY2Bvlh5zD zzL_=7O8i?u!}`9u?g{vcTCC9O$Gsd4IE7ujR7m%J=(vD>-D@svx;OgfGwT<8 zknYvUmGAuNyWQ}>E%+E?3?=S^iO>CgA3C}?!6yAj~0Mh4&Iwao)zk9 zloG&w&N5lSs062AED~4SM;H(!pHSSLls-W9m3XAT$|My<@BOz5_UV3&I}J>JHJn9F z!e(zy3O|~0)JOS#>~|>{6TQSXb`inK0gf^Yj16Rdg}afgeahGqKMm+n+_5!-$4-cEX>v zEb~q=PkfAcCX|rov5_>Umjo(p4m(rPB^O9B?cAYsm3qAw4IotGc*m1L2{+7MfK((e z0TvGY_XUIhS=>U^^j!Q2q{eT^$TSh_ok% z5+c2yzb8T4U|64uKCFK}BB40X*Yd5tu1> z8=O$%(WGK`MB88${TlxZDG@1Lzq>(vBXbqlt@N?P*VXNH_xf8(PcNCE7@ zV1K*$fCmLE<3konMy`Ah5X1Ie^!ba^4Q@&KNw;2xIGp!h4S=>vO+*1SYH$_oce$4i zte3QW&6my5x1wo;`$83VqI1w>Y(Vw~G`_Uv;J@99{|8`iW=}#WFWXs#AW9x;vYvQ; z@1mFiN}9$apGi*%t_(g`Ug*{Iuv@rgY5akbqEX-@08F@0ry^i@jm2EPv2c}E$k@9! zUCq95_`GQ~NZEoCa_qmr`H#7Sl*HoADx)AnM5GT}Xty>`o#zAhk_&s1ixW8KQ&qFc z0v4c)Js!T-FwwwX^v_V%Uo#cdku(p8waYm@=MR3jQ%;LOhu1u!Z~|}AtSuUnxT-UV zatT0`c|gGM2P)gQfX234`Z!%YMev3mkGZp6ihWfh6xTa}wtGAEH-ztBP?Zi8gem*M z_RagQe*lY6-CLqtjnE6nXl!F%6&hn3r3(9tG;-HgY$SHXZ~ox{wV@-7Gx;L1fQQ!x zCh<&eE4KIV7bChK!rR-U>l6R2e$2ZjLhb2wIo2*N+*I?9I2)jZpGkACaUhfrQ3Xg9 zY92y31^x*iQxq>Tq_VR_+myF^QlOReOWl(nkVX_Vx5bMl5=~K&2VK+!ltk~z5WhL& z{`uq|NCwDFia@@qXX*AtT>;9f?hQE79A(%3_Fzbi*b4Ln7RO?~O3!K#&j!S$E{AE&2AIr)Vyc2%8h z<8>R|K(PM;#xD_dO@ex>%IjA!zxz`TQ^;pUpD#0>>{NYerhIpPu?o_z6J40W6bXof zA8x>G=qQP;F-mhpVgjRLklYeS9Q#uT?hPFb21JDnAmzjAN4&Nv1mEMtZyGlBU|dC* zu~%r{B~6VW=%-OHws6Uv13JJ*+PKWLLizxO?Uw`dYV}souR5M0jwY-3yldORoWVVn zR_Tp^!b?k%;wa`n0c^iey&a5UCiK_x;b+Q(8zv!Bf>AU~!N+$jI_aUD8 zCvJ-A(BF`{z1$wTz-vI{m3k~-0qC6qg?E>1nKPudl1bc&!tNV^W>#HP27{Cdv%#0+%u;zMK&H zVVO9c3dTjBsu~(eH_m?oAf^e`I#$J~gmu@f^Ofo?$SVjj){_XNtro5ilN06q4nN>H zBaGVh(D-`yN$+pPS}$*}L0kyrbVkH=P2Q~y6|9YSs6CF*{`28n&n|TjZJ3oqacyC9 z9XJ<>P&;|3nz&wc|2Di-)7&@w=!yRRv(($he8f1_=^6e+FYqPAaVIs3Ur;a+-3{sk zFQLgXQp2<653~bYH2FczE2r97V2LmafrI`$^DAC_GWW1RgcxJpytNOwka)on-K(xJ zUD^s1)H9Sn4j@Xhh`PRz>>L!_%*g;7>o7|dJOasTGS0cF{7h|<3I%FpoB>c@dbnT+ zA39XA%QE;!jpfPcJWyIbBQVLmSDT5b?tMyZG28Yz%ni{01EtlTO{hTez_ge%`=Y2a zt~UZh$*0DMF(B&SEiUwi`Rke+cYOdXD!-vosqm9|)y$Z%DIQ!%&3~8=?l@)3uPG1K z9wgHOSX>z$*T6mds|=f11IxZyu3&pui)j@#;X8uY7tZ^tE8}slOjO^92mB{FVe^r^ z-Hm0yd*$qodP4`I#~&L#du?T^o60pZh2OXfAXBT!rjl%8kGg_#lt*^9e_jO4WD?XE z)lUI7|B`qX$pz5o6tYm$qtyvLiW5js^4i|U_D~fb)wh?zmb=0Ad;YV6S04|sGl@`e zP=wiSc@Xsu2#vg*KyC@bNL=a>u7odwh4@av2#a@H^4el$m4 z&vpa2AESm&k$XV#Tgy1VCCyNW``LB&ZTCUUzR1lw&8?qwn}TgRUR|Af61D=e0X*J4 zg82I25LyKU(_kG!Ih;e7z*YSY$2jcN?>Gy&Ff#16A}4g@43QclM7Gz$_z4(PX}EYS zEiS;UFf2?DOKsi+hJQYA4ppC19N7ck`*U-21qt$VBeDC%lsRedE=Y3s+rNU$diP6Q z?n4qwGVC$7iq*8F#3LpD4Q|;ylt=}t-Sum&&4Xz(KID3xS}GV6D?`<;M`J`!)rpYw z0}=C}l!Uvp-maJZOMr_gG^n5kg*7%LYH2XscM?HLb1c$vevmo)$OMnE5QNd|(Fg`1| zmfB1aPz8a%fWoU^znBxzd$y#6pK9^A&q6f3Cfj5;U!l;pYWWiHD(wi7nv~Vf1FdF=21d;gCA3kbm1XQsomLFwTnNOz4zHGT;&s4}}1O zrNYCX7?#HFhvFl)MIAm1SAgO5yi;i1dvR(Uf&Pn|HOoj;Slmt=A0kDL zRfemR)5^rnkh+5}{{=(@38;%1#58(nibo!}A~WM*(*MhA6n;qw(mERZ7t(pZ*OelBcyEy+)vmjZ60uim#bY7ej5}*=}d^?2=qu52oYwR zZd(US^}yZ!w&!*-mPLsbIKI|A#ATiYS%ibVg)3DbbPV!OFi(dd1wGQZqt+B((%;}L zn}e2v92iVFY5)+ZI)9xWEaw7RHNnl?7uaAzU9sQRc8YgC)We-593*h~7 zflps%2>u|2W1cZ*JgvBH{|&`+ALp5w1H?b;)L`j?6~r{YYbL<>SP0Nholl8MpN3We*` zC1T-IBmg22sGJ)@9ToR(Z|f8gDs|L6-j!<0g}k&fTC1taszD{LPKew%fzH@HlLfO0 zuz=b-TK0nyfiOCf7p^0x6}zUZ43}X8JbsdY2W`@m*g#`d;O)^qnqNAz^?GyO|y^sq%6gX6U5=%#K$Ft->9P&qYNwuv@MHHWTt?Goj)y&%Ot^UxJ-AP|&cb zb}Z~44br+?g|LlZw4Yp)dkO(2YZ^cpL&M=nYMB-PGq(aO&y4&P@LdNvj(1mItX$ZH zllup^d7+DkbP*n*GaSZwMdRqQm9N8`r`Y4^n_*9a88W zG8Zp-lQ9e*2+Pn|g23%y3T6M%peMfiSqq)R^d4VGqjEN=*b#-c^AygGL!iF6u(JR)^h?SO0{wb-Gt8rGK7yO& zSltDghZ`K2raDA(dqvtoY@T8jG1O!|XkefOoq-B6x zq&_7d_;|aG%MgTCF7|&;wRy%7A``>AFu3pQ>_?xWou^!dAl&tzqr}aCsM%HWkeEOK zaN@Ro(t6IFt|09?lI z)5+}UHiE^P5%}7vo+gCn);AmIjnIzQ2q;X=W52i5B!Sd(Xed*k$mbIX45>;1yS)ja zK)>RbIp#dhT+QFzSv=PB@-ATEdLhL-q_TsX(jW;mcfd$Edrd7rNM~fje z6;aTgCRYJoq+|??bClP)CQs43M7y=+^k$+0vEn+}-sr z_C<|In+T7?j=a7_6*_~LfY?EE1RulTiow0k-jdbXpn{qM%C+Y5yJ-zpc7g)4f$4p!3uDj;VhFSqssGtwL)<;(4<-;Yf9D?}@fYSwRv zI8!`m%~Ni0R%gigH3W#26d-+(2EZIW1{f5TJ$_xZr-X*j#v{WcZ}Tw&p!lIzics?k z@|Y}fSN1?v^ixFp!lmVlUTa*$pHstz8FMQ}%jFMWg94QY4dlp&3|AHWp0Jh3^;Y7G z9l$0w719||jS_cTv9+56q%gnNA8rs@S=1rsW-l4HQGuY%2sdK3uD#7&9p;&i(Tl!r z487FmkSNKi129Z`sA>dHU9y$tcbyi{31zs==pp6e%YG>SH8M{`sr7#FVj+-4J?GY~ z!;p}aV$jF!fSTppr_)$%bHc--vv{Pqj;GLU&{YsLLU3lm_ULcA*=4GMxN-#oP11~vZB{!G-?mU~_XZW85t383<&Ss~e-bK`wKDDQg z`4vnW18A3|-jeQiq%no4s;9Th4fyNuV3%vOpzls9n-P~dfAHe92Qx$t$o>y7TW64G z%Mj$@Iu7zS^0Oy-2vo8pr)|Thn*9za6_THK7|y)|-JD#s@+hlLLQm;HzR`qzv+usr z@bB+G&in0YY+P{ZoqVcYVtjt}B;H8~6^9OW2w;)$P#!`J8Nsv6r4zU`NU%=<8aITi zBs4N1M!{1y@=P>KkG21 zj>Hy3I49~bH!m_@QrSo0LzLHEV3$t^_ri4R2kR@r`-1D#*mdmo@r4IB7@?7t7n^2Pdm2=X#;HgNw(_5oV^F6dX_=nthu|Lpxw zrfs=?unv;X(Du#$l@PxXmtnj0@zH^de)S4v_Aa78@LGLj38F$ z?=J(=PL!{>LMFyYj!I6pBKA#$F*yl~?1NAR3~e4Zh0I-NxSoCO ziAYVy`M|3Kvp4dthbwcm2)1a_hEs6d3b%=9U?*XE8|rYdYwPDUJ-J|?6<6OdKE2&| zOWoDY)%C0R&+(pV?-g-=@*YSxT0;t&pC2j&lPq!ysvKT>zuJ32G;RW{c5BE&(+xHI z`N6HajX(4nK(%E#8!z2~?R&()Jh~vK^T9KthB`$BV9p_u3J>2PTDR2*sVUSTSYBsc zCkR;$AJLJErTDP5`S9n&piuqp68p_&0wB5@!e#OmWjq0R%=0t`KNF3Wb^bXsU5aKQ zL+r=%vOI#%^j)&xs3CtS%;k*kg^>d12rh+#KONLL?-+2%1(xY(ro43P!R|uJ-Aq1! zq+g0-SLc=OhYJoBeh9m!VS-i<(4~mC3)C{KAyA+XRS4+JX4;&+0{0r+=rh-5ku zXemSPLjO)4HnDm(;03an#&+7p*rGBtF4UVQ1TxU!SIILxyQ(nTIWZa2yEDSHYRQv< z1|y?{b2>DZSb#eJWTSr{%rum`hA_`PkOH)Yy3@THzvdYg+TR*<-%5YP!6e2eN&#kKqp5cOS;B}KL(kt7r~ z1=Zv!e&Zd+CXL-RV-Dk|Dr0|_mA%1reH(&drd0%j`U<&Mf&#&JniG(t><51bS95d zd2KK+@a&la77EhrFGJ}iM-LL8k7P6MkHG-S$i|297Go8+KN+MWGzyfMKQTeA>(Jq! z1aDFLywT=a_=z=`-f}Sl+dGuN06*orM8Jzvz*_Lk#5~5}f%JUGRt$Podf}NZ(HvCp zr+$uw!$izE`qfNs68inU&tB_FnrfsKg`*u?0LYid3pedn zUTxo62ZuWq)~E~C;&ec0PUK-V(fKpL-+3-0IEPQ~@qY0pWm)e!uu?CN87cQ$|DgL( zNQM_hx=22;!6c1iMB5kW=4XWwf{`3Xq?As$&J*37uNhL~hwdNVwb=n`aWY5LD1;_khs349z>E%*zmT(8U4}eX0Q-B; zTG5SZZyY*Ha$gp#C+W2+bXC4*!gD~9OG-hdj5bfM)hQSU(UKGEG7{Ym^6twxi+cHe zs2T-&nS>lL|4& z0JAyXc3)MPdxQy66ZsVUx?4e5&`$Xm5Ws3sd`SF#)l!wgF6A$H))a&oaHeG|RCgd( zPN{O_`^RR#Cy$WG?1Hh(AOsD>ztW|ERSskzGG^$oJao7jcYOD(OO(nGW`BFOxCSo| z96M`flF=v3tHbZ&3uEQ7VfF>&b%=r!0oinHt_Na$JD3+pybg7cR8&rfP)y)=HA{_L zeStm?s1RH3^q|ljq0D-rRlx$9IfxPc&EysBJo97uxfn`0oxkEgYT`uqH-_wVT5>vI zn4oP1MFXXa4mF`DT*=xMyzjw6nnT`D@0uSJsFQ&;Jqcx)*OopEsy_yqdST3*ZpBJJ z>UE=pF#s~2Y7EzTVD|!sogVF8W8Cibt<|nB17O4yIO6igO@!cXnd0_KO?3FHDUWpN0om;-m?}lTbuvl3$Il#%_lNL&6+XYERzrBky{T zkwh3^2EPh3o>soa-bp8224x`a?m&kaBPWA_L{9Sl;${7sdVo=fh)<9iiCb8CCSzQL zrk#gcCDa(+Tf(-BWK;Qz{v5S37(j@$bhjeAU?gdKf{jM*>&F-IW7V*CoTBIVJAM-c zAWJzO;GVl=Fri@s6%+w*8-ynA0OaB4>1Uhu#a0IR1OgEdMO9zZ1X(ZEdZLR{;DJZ3 zlM8k;TRFFNp2yQ;$1N9x4Syxc-s7P`Etg;b3-@JpVu37>jdCX&%7#Q(pqkP#^DcNk zhW8@qP=hDrpf1w~!3(^?4={x0lwy&%kPO3P+@Y2g;*h`V??3W!AaNN~;hq(L`)(-n znCs}?4mmNL&bb5re|x)53!teGX&rm1YxNBj#{U!4)SF7SyrU%eEsOolzl!rKj-q{= z1{U+7$56-%bG+JMPcTLx*5* zw-RFSR5F^9xDuf{+zHur~p_v^YT`r{VWbgeE{6#9e4ZcW!>d+Y!_Ghc%ey`Z4 zxft2Axh+0!d{ln3wEJ}g(?p<`pmd4KFPS`C0jcz1e1bfe80aNd6 z^+GP3!?VIIdfuzeV$|RXd@YdM^~cbnFsvpPyR{1wcr?An|6PsR+)s+o503l7mAXP# zv{MY>r8r1G#$Z2&yUz<flw1B37&nLuqj*lF#6KCUC$|Wk#!OBZj!y^i$iOi$w%YEDq{ha7h z?TCuA+p|`bhq?l!bO_%7y7oW(6n8ieml^Q8f z$lFk|NNt^^8cq@z0ShP`)=C(DEry676$O1gMmdZ?;e#*0&+IxE3s`}UQ-^nYHfn3J zFchDk#{`u`v5pe^eB{UAZ>j*wgQ~!|l_Fq#FTuUqJcQ{i6~cp)Bmix;K6a;*Ji@nl zjyza=O7PaFj&W5hB=#-*@M8q;?-rClhT0kmM`32|WL~rgJU2V`6w#|wbF^4M!s?A> zYGU^#=z&i~5XYX_Mr#&^pL8C-UYvluWfm%ybaQud1>w(00*CDKZ<*_$WMfD?h>JbY0-B-s;=K&9pxN}fJbNjCxOzlCm z(r1l^9|@aHv)w%ws{wuldTf>)LOvKZZOH_S!Pcs#s{!X9#W;{X6TN-z(plhxVeJV^ zA)Qa1mS&BBcDLLjgafw^-#W`OJ^!i7XTF~KG(8@0*GRUeG0=CjsGw2C*oF9(-@OiNJb~7@Ois0eyxx`Z_uueC|F8fCp9;gVQ;u0+1feqa zl*G|6x3%@cNM%dqAD#u)Cuv3A!)z4{M?w75Q^hHb-4TB62x&LR$&U)%pCu>VN4;(N zY{3-iEBZ@TCv4DY)^RCNLGl&mIh)pj{OsLusv(>+a^x(m2EJfnAeACs>jNI)H{e<( zy53~^tt>^a4>|ktVjt3FF`jlu<;)ZqF?7{A6fImkgSLgBvr!gzLD$_~QaSXTX zXnsoZ+i;wYCM0J|24$nRmMzSmk96%t&H8dnio^PNYwrg~mJ;g9F;&A&An3Cy`xKuk zhUbt4ys}~&drkmP45Q~(^XKrLs(VcF`Zypw&qtXtNvOy=A08?g{CXmE?FB+)c z)ONY_8Fl%Bx-82Z%j4wb8Np5X5O`c=Sc|tYCKR_eE|_Ai2_QFdow^H5J3MLM6bAeBD=Wp9cy|1en`T)2}id#@#*m z0wyx5!?VVdHc7-}?j(o01{CER8{KPG9qXlNl9;bH665|5FG1XK_(N>)hpeo74@TOc z&=0W^KjdoUc0x6o3EX9H*H7N#8)grV?UT7WKO#QydTxeG~u!A_CjBGoXt_Fhf`EzadJ zZjWv5FthXXEkl{fh16YX3Wv+f4jnlVVZfe`&xbDQ!7WzI1QF5Gug>~tdjS=}5{+;z zl#ivtQuko*1}z^r%$0iL=?o9VC7f8ZBUG8LWQq0DYG_v-;?%xc+l=wrP(T=Q?7^_k z?{b2Jp-+g6kNLe9pN>_7KQ>~-2NMJ#Rp2uE=mbN|4cwVfqInhY;i>j;B^%?4F4>Z3 z-Se3nhN{1o!*bue0!ILCCx2i?oUvMkR(DX< zut{a<^)R2ji7r!mjFrDK9m6ax$ZcOpv`M!ENsdOQD3MT~OEGn_^3=fNn3H(VLNYH1 zikB(>BDkUQrcuZcT*? ze|m8&^}5SQsF*f$_m?A0p%U}6-z82FODPqKaGgrb>T&^BEn(JRsYIc`%_ga-b39}uL$AR)z#aXo!Ht}&1 z>Q(nO!$av_Otjhka4D{NE1gY=r%2)5^+9mY51+4yHh*Uh>@#rVOuIqBFSUccWL1qm zbby@$!b2)ZJl`bvd^vKMAeaE^AMM@%Jb02krwLFS`j=Hqk*`e4hu9{@b`r#_??06; zrpuc!#eNGZe(7v(qrod*b>?qp`}p?`O5)Gu)qE28bhp}55~fd}{_2D>Nj!Bu0i^%F ze9Z*>3T!ISz-gza%CfNu*B$l(-mIX{qaZ0_f@@PLF!dB9!1A4gf+p7=W`>x^QGfW5 z`S57keQ!$lF|i5hTneY<+E@Lgc#@>=52jZSjJ;OkB6};^{$AjuP7Wru%v;kXhGf-+5(E*$c2e048GEJywj(v-<#@ksFhZ)AXr}jY= z%55;}yT3yvK3`zV)PFDdxce4{dumLizhvP0dTWrh*E75~3nvBrdl=uF;O+R)|80II z$>2RbJNK$@=I@n@tjET^ByURDi9*ROH}@~gYRy?Rpd553?{8hkCj74n9uAu&djlh0 zs15o-t(^wZ`9-0V!uu?(ll}Wx*Ne*Q!BmQl{@8(8^8-|uGkN~ zW0x)=6t2yiMBL~gaPF}`dbnjR8nfoNbUyD$k>70g*ndZ3|2>xj`^v*x-SR#K!45KO z1h~D&a+;O$YIZIDu!Bv^or9Vd9@IuJx8`xSQlViOYqzdW5U? z=5I0Ym6H9jqX8@=*PF>#ylk`1L@CQiz}5M~sWpo(y!zLC*Jw1!mU@=4*mDI!(U-2I zz)sARni{xEi&Zj>5G+EpBduhi9`7AnZ16%3X-iJ#Z_ zaC*q)R)5m%-s&HK(iu4WN&cAlrNMzyiCUSF_sqr>JH#;zRYCdp4S!<@H(TUUap*_G zvc_hukAVp9589l~A%@m)!5!P!g-3sF)B9jp1hr4kQtjF(1De{Uw$6&xq&AUXKVQm9 zwQMZ7WzC&^-kY$K6%C32z+|d;;#5J4Bf(yh|67{xiT2T(p`sekcJQVO)z9C3_TBY* zF7uI$1J%1EZn9~MlkI}|RniPo7a!Q9zm>dk<_2NIZdo}=G5KA7d|A2awmNn9MOt?8+|j17C^`rqKLU%|I(lS6 zK0;cjCX7gUz{W;BH~O1F*=nxSB+Wru~lwD2t{Xo)p`QsdGWUd=G%MRT> zPGd#k)QVY_Vxm1OdCyt=F}C=v1fbDz`XBr|tru_0f9GAal%w{#^=(4(`>saA%dw@K zYf}05wqCVWz-WN`icXVRq(#f zVx@w?gJb6gM7+vpa%E|K>_ghpm99L`F`sNvf+}^+bKC$N0=ykYU74u;YMDmj&+~^l zZYvmM^V4-7)i_xDrv}(dw|-ibk~Pww!d8h;w0Gle*yn;ieVK<*TNinKerN8lRx}d@ z|E!(Es5KIPvX0E49)uW--oxJ9GA48@-p-gb3-iz)0Ej8-5P+ z!%pgVq9uiYFBd&z>W59+%V6@#xh65HDXrnux8r3MpT+L2tw%t!J%E;b;>=22XaOuT*Xb27=6M0F5ht-y5)M?Q&~4}93MoUizWWyFuvQ;1%uz^ z$716#o8v6M*toweR%>S}gpV$0mH1wJUBb?BkJMF$fd+s0qQ11@e$HQNzM1PEOIWFw zjfEC(C_^m~`VhxZ2TlzMemcqZLMlqToWQ6!8S3~jv4x6c=F=ITGS$I!TqD79v@|!P z_c2&;AT{0&6CH((tzrFd>$QFv{V-vj7aVS-TMv)z64y!yUEPyj>};J>qGOyrA5fH> z>u@bMaL1BsaZ%Uq4LMuoik=9D`$z*%PbOS+ZbjFJ#)6=$~%+gYn9%C$k^<*eZ>m7sO}I3(A< z*G4R8%1Y4{P~2Z02p3!HvvOWoqRz)S)nw-dlQ!5#Df6*gmvcEN1V&LfmA)Y7iNH?6 za=;%pei)KcA(FekSgT5t#J(L)xXAx5TyVZGjc4kNf$nqyg^!`f4#S#Od)6@c8CG~} zeS}_%mMVjf{ELQ8baisS!BM~k-MCYFV^vEwkIoPY>AbFN&-1MxO|s$oBI`zIWx)l_ zTF+K(ENkbP7PXWgZJ<;Gp`ZrrJQo38tH4He9&5R@8|@V_pW}KXE7}rf`7PgP2eMl; zFSDq_VU)lZ;CEogAm&DVl(;}w~dx3#=^heJM0(4oX6i|2Hzv>k_O|ucM0Nb^&hv2S-&W1Dphb$ zEW}Mqj_34?g;*ClFFC0(oyJ~Q$^P#+G=9?hPzv$uN-Jcp?+ghZ#`{4E;w-4=DfE>4 zZkB8eIG5ky?x!4MD&<@e?GJ577zB@jut{}O1nyFiLp#xxs|oV#Pd zsNfUxmt@yqQ|G!p2V#4oitpZvTQiq7rNc>v>^?O=GMg?sn(Ma8)h8l^KdlS6JcEfQ zR@|n(wU64~s1j(Yye8uLKwoB}=^)MGEW^P4Ue|%m`$U$(?RLc$vq3Vv1o!GwRG0Hu z>ohK4^Q_dPV^%s_MGk5ufwHt=Vq3`td=>9m88EqcT!|7n8wA~%6TA( zUG2BJ@!*?LJ$G^_e7TZzw$ez^l`vi}6Ul8FlGIcc&2v#^;C`p&*_UMf+%6_~s^&s< zqki{M-_;8W58Me@f?$DLhe|i7rn^i+Xz4Z~YUD?k4G8fD^jW)NE|F8-Yh4&2XOEnV zye@k*t-?Kbz4YOhZ{?TKuF_`4rIO~&2bDR>we1_EifWN8Jz8sbUs<(U#c3*Y+X)7| zuAZ#=PvaB( zmV)-?xv8M`<06rD(oB&Q)=FW*FO$AzU;M36p0+Az{Co1A57*q7-R&PPUg}J%H434- zwIml%ljAuDs)p^d#?iPKveHpwo70 zr#`G2bdGCAHhP3yAt>=`Q4D1eENAv{9dBQWI?;aqD`lc+;EXSN@=g zEVbWEi1pFb3f(tJ8h#U`%6kkX!V=06S=Ql{U@7h^OMR3m6;>M)Z@1&9U3ieJDK4J5 zkpu^pRU0(JH5@KK?!~>VR0%r5H7pxKOY^fJZ)1Us;-1!L&sX=K<`-NnCv($nY1Vfv zQ~a2DZYVQJ)uyyy*bzUcHR4$r0lnMli>Z#XVGpHT@F&)1KD9@GbC0Eln`BYJo@-9G zoVM)`8GQNgj)l^TWkGbjgfPS&1EqG0pp!!P^+p-WD|9tiFvQI!3a{)+LrvRgPKpuy z!iV!DLDh6VOY`#z)3QL6onaq=fP-_aM|nKS(5-X|hgdyR0FjCj)UdAcX~H`$`Bo4S*KBa~ zmLb!rCp-Z~45sB}&m%J{9PF~T*x!YVRS`sIe4A_w=bvvh;|wcr(#o_HIL!5KSnNmf zz!q{PXEUW(=T6OtuhhqxdJVYEP z*r6 zBYj6CD)wOLd3)M!FFOf)SA4_S$a_s6;-O9gwF=g`sO2|ag$ZS{<4q**-JY!KgLxm= zW;PD=&8}mLiXwCTJus0p(sC+`AlN0|bNQ*>Y$cwm)(cP=`SbuLfh<&1(uH?ZD%@)% zT-Y^SYs2zr`U}jU=SA+mqNq>bD|E?UCCSt4^el~(8O&ywmdD*1E$SenG<;g02{DlP?GKTob>Z9$hQ$~`<0&$^Q`bt5 z@sBjHVT`&wXUDu$`3hP>z$G%!5;^tw-HqZsOF5eFk8|1W4Cm{IGcR+$)qPTsYWjn9 z-He~`no03+M1kQ;iwR{tR33u+u6Q(RO>U|)#}yBo|Cd%DU78if9qPdzm7SgtzzupAp0Fx zpJmg`%TK3gUfAe)sIQakFXfV^oqKcsLjG7-k}ijq(l>5GIBX3G0dMt0e8hk6j7UvM zmG&%bl#>SS=y(?W-7}YsVe4-ft9|l*SkC5X(8r^mU%7(5)lBKSyspc?dokzTl8^U7 zlZ(I&_0*URDq6=m&zzegEhSo(sG7+9fykcHJt)-9;Ha-1mFDUt6Y*ck)8`^=wN;yb zREs#m8uU}S6qAI{avMTtrJDeiLrp+fSf`fXOL?6jt%`QBprd+)#}YL3V2DupR`S#&>bI1wr!8%fAU+$&ks7-P#=RMn_JIt9E zXAxfEF)=L}UFMQ@^@cu3r^@*5=u0G%2jDST&Tu ziI$bFd3bNMD7tw)lcuP9UF*K6)|tA;i#b{>Nz5@n2Mm8r$jcV>^H7%`mTF?6|nE*ecerlVs^GAQ@`W4l% z;+}LVNw&P^powI~TbDQUo$I=iMU9Lca~l|;NpkP;URNf^4TusUHuK&FEX_aDC-zkzZD&l8}3}aGPwQSkPzpL`|CL@1tsjo za$Zc|^6c_f91Eqf$u*;s=tbMjs7{WGK9UU$%iVcC?auwKCo zU)yDh?88dYf zk{Q{U9?00lv9Tx(iJ!eE{)<4@Pq((+^JZAb+RP^(^fPGe(#bctT+W`T(f+(ARZ-eh zH7=4)O)T5uV5k7q)m(iD%cPIdwFItAQs47x)(1z#VudgBJpPt9NR^vjGfJ-NhvjP@ zDtDBgO^-`oOfaXcOa690DeqQvzkJm#vcv5YVHV{#ZpT??ET$p{ zz;hCVR!%6S3XZs|s5$z5!7ZL8Wk(zel19&1%Tm8f`^3q{-k?#joV9Kv*x$5va}vU% zVhMD0lT=)=!$ZT#MM;I4o%mHzLn>UYmePe~ZrVF3C~0ZE^qoM-V+KNV7Nr{X7q<#J zZt{R*%0|ItGv)Iv6SEZTUm%RM5C^PzGgOI=V!2kIG6v{UOPIAs@ z-r&7|hLP=6QN6HvEo&$vKQ;Y_+=~Djml~yt#p36}T+ljZ8;*KDoB+C;v=&w5U3qRu zSit7vlYC8!Z#8v1g{kfoC*(W{&97%B(y8TmoGok=oIvmv++KDfv(?K53Y53yTITz; zU-J;f)>zWh>C4qI8eMMAzX#jJF-W&hp#%*jUajgoy_CQ<>Kp;^ly|}GsFkN~HaEwo z+K-+FM;{=73A!q?FwqQ+``j;yjLg|A5-J*Q)>Ni|i6pi&N`8hiSv1ul;#Ak?JbC9* zFQU0QMTWW+>VSEo37U3{-LDw(E(|?i9nIVPhA0&7VH-dHs=_O^Vdb8WV|TS{nwGvm}mfqj* zPvBo2bYUI!)JNO~q>`~KYFAZxCwnaf%Wkqy2JLK^q|}P;OMnU+J0@FYJu`s!9r2w#jw2M zoV1X(3X~=B<{$1@y;jQx{cZe^Y{ju~bQQ85Tk6jt)xJul($$_)EPU%Iq;$FUBxd4w zzvfJaiWU@7&M#E(FB@(Yu~gYN-{#5%k`M>3sjhG&r4WrjwFpiE7JczE9bVJDc6uJu zV%4fA(A>t$qx|iSab?$D^Si%5a=x!13$X2Up>~{=n>kR-M@pVhp8>r5 zZP$&_yLyg$$Plgyzt`MKH5%;86OG*eZ zK?d)|*VnZFJ^f+Idq#eDi*AKzax9Ml=JX6$6fOtvaFF3J&jMhDRj13|-8ndcs&HE? ztqDLC^5L47KH?47J?k#XGpmo7d+EM8<_XL-)%}ta_AKE^fW&<}=ro2KCo>B^*GpA% zNDx1A0AX^HSW=P3BJRKuG6Et44SGUB<_I%%;;ytvwj%inGQ|L#e-6|56B<`kt_@s* zvJ-P3K<}cy`syD77P{Rn4an**d+*x(ofR&mqbfvq75%*sHLJcb@?U0YLvJI%-nxre zI1s=c<|SSozWBYK8v*iU72vWwHRo9oy&H5iAzm_Hz@%qUNyOrpogL9DbQocNg9R!? z19JF$iJ!DkV>c_5<H4gjRHSTC{%$nxOdS$M#VH(#<3@4hMj8+?d_9|g?r-2tc= zE(<5H^hQ%*cNqR0%-FkfK^fJQFeZN{qUnS`w)=9?=pe;`y5_BmzUVs8&^033L9{QK z@9YV@cSLLuJupDCe>U?GYP<#=-`&({IOpxtyZZrFgo^0M`51qBG$br7b?Pw7h& zh#)x!(HXF<#HWX!mEiEmCc$6$3Vi+93+bE#a1;PAr}?|Bl05-PbN*XE@;g(1?EsgC zzub9GkwF|noCm(0$Es0+Swb%X)JekEYRU1_tWA3Qo-MgG&JJnP5d4~a?h<(fbX3r8 zMLiIRN8GI{2ZJBP0W^rGG&y@=5F&6{I|r?qz1aU%D%;%{%xxD5_p`>UB)maG(oyuU zC=pP92D37Tdz}Gpy#gExXDf6T&;p}HY@>o77zQNCqXn#`O1<_F<#V4U|9fv|j3UVK z-!_zm=c7o~W=E6fPTD5=cJI9bjI#J{?XuDI@N@k8MM`akq0FonN8^`)@BMB+s84{^ z#bwX|G~Ep8z(Jlm<#=o?}Y5|;224&2kEmQlw<@3bOGp3b5vc@Jb=IOd-%Gtp7*6=utyl+!kz7) z&I*c9|JBc4(ClwTkODLpn#r(>mL1;#d_p$$G8%?L@(mWCK1HS3aHjeDG`Oz;{0n1- zX+PYg#y1|Lt$qfnjXVL=Z8I5XVq4c6ZV;?>Ip<%|cR=X|R`wFg%9XA4J0O6+Dv#bA zG8{0~&*Jw4l2#_H1KGTgefnAr@CseQxY8QC$gO}#Oaf5C#c7<~d{(Zc?Y4tXfON$$ z0n~hsT9FMvN4<&OA=ey;IAjDpmPT~G#5O)l*yH2+e&BJxT>gm|~QgKML)Wcyni{ z1WPyDt9R**JL{LnWotDOv$+<@UNbq&PcC4mpan_@iK8Hl=YZ4OE0IT$$UI`u#=Hed zhNFFL&re%?+K<3gv~&0S+lw6ZP>)s|G$V0(y~vzJUDlJ(8BUt~75e3%7uUzWF>C1V z?bqo;UQ4%)V9Po^!N9cYXrR{&umSPVnE%r20xERjObPZ5?3i|!S~nUDt%pdTB{k}B ziP}VM1&YHZhmFalsGP@%3#$kz;*dy7wqv;@H0T`|L+yfK{NvJoH^1NbQODE_1OaA< zXgIYV9VMub%eq~GP8hK!APIqsfS6cN85duw+B+-a1b{{%sOUOHgqH;)W`_4Gxg2{< z=dwS#C*S9k9X^hXA8>U32fZy6WnRU{EQJBES9CRBl{Mv8A$o^+g1sb#zg@bC81IV! z(wy|&$$}<f z5vEBv;M*Crn{i-V4t}C9ctfir56j09WB53elCTvz{hdv2|4>+gfHfY(pW`7G>#&X2 zdSy~hPE!sY@O$OPp<754@^GTS${|R1<))U4Y4Z6)(mx|2M{Yu21<`fnQHVnuz{TMT zpJwQEGN@PL1lmLaV1sz;30bOuWk9qO7YupLJ^_}LO0KU^wy+eMP&Q8w6Cq#r0zHMj zhZ3O{9|xxD3$%UGDzw%Mg4>LQj5Age#rCB-neBGlA9le&G4ctuu2q+08rXeh z3iji`c#UCZSJjNd6bNqD3keooAuJ!fJ8iW2`)* z1b7p-Zcy+QY9{;yym#w_!rin&0Ha@F`ASS@9AN~k7Zg3kb7fW036i&*M@tZcL&OPw zDVni#&P|0ZR{*z?bxbNG*31J#$or++zSk$^%cP$Gh&Xsq4h4Jv?ph^y@qhzg-#nGS-3pk!8exs>gT=@WF#)-T zX6aT;rty#khbXwoMC%-+(}sSixyhx;yYF!FW#9LY*T=Zqj!L>ao~pc~!4eh}3+`rx zOVawghKcekKceOk^=j&xuO`^WVDmy%<=WTX?C`uJ9QMzdefb@3_@`j4PTu{pV)jVb_>Aj<+!pAA#eY9_3M~@t$?R<-L%uKt}?lL(N{XYMsRsNfA98ISB89<`f@1O zzV#Rzq3N$M{d5Kohgv4GPSPw414fp(H39l6SvjvOwW_keVL7=}AIK$>Q9sn!kY4vF zcDVTkX6LTV)>DjWjx}1AZ-MriO{E*EC}*+IyhqYeNCfpjT^E^rS)w zyQevup6;$&4rhWdl^JH#rd0n0G&M5hDur(rSvW3_=&O!2Tw_cAym*3U%Lj7j3g34? zk%o?BYk8UUE)d4wr{<(RpSbltz*%_3-2O>iFNXFE zw3t@*O-I!s-KJoHmDvR%J9@sQGN6S}C?4+-rs-f?#*ZlesCPT+xH(Kz&_dy0t$11ns0O8e|8ksA~L{ zH(twsfn*tD%u!kouq|&`C2vNW#)x}Z-{*M5HTao-*|=V5wz)PKdbkQBx*zqCnsdxm zM-OL~hpB8TWKy`>Ixz<;_Oejc33MOMgF{dZZ0Py~`x@mIoFQ|5$IMxX=v5$+W}2yik#R|*B0kH@3`+z#zPQ>DS7m`u~H{7h%^`)2=ne+ ztTZiJ(+sehn=8!jFQ8}4Ck4|{WN(*!ID2c74mEZweBuG2w&Wh`)GHElCe zB13t+lg}Z<3!DpAbiVUfZoP47jjeKpZWg*_DJWxvH1gik3at{fYK%$j zexvoZ&?xpwE2p8NK50LYag)CV$lR6ug08~+X_4=1)$cxrePDzvmu+x)c5XQw5B>CK zLwdk6>tz>|#gqDi%ez)^C7+zIne{&mVE4(07~}+kUsz8CuBq?aVqPp~S<<^Q@ftab zgGN6%d6s&eT7IuDGWoKLAvv4RXV-x#>o6^K_j@_$SaV+4f-@l@LE77$r2>>04mpVO zIP5^yjGTgQ;6ah>3Uov3E3Z`TJ=ANYFmwTibeuhVb-JC+!x@~EgTsx%bf>#nzl?%2 zlqQ)7{G58hd7q2Eygy*h&qL>4Z{YxDvK8rXRnsmMANP*Yi;c4rEm(SJzjn5+8wE8! z$IJ+zeFK+y=xo$|=7{%)qsbuf5Ln^~b>&!+`q_4l7;6D^5MYax2=CCIEe@q~sA9U3 z?+KJiJW^SCs9%1irg)f&WX^pmaNnVpav*^bSKIeSE?H97quqUa*%~_Fy&_uFwfn*E zBWKkB*@1W0zuoBWRGeryr?Tn=Y4scEFjpUYP;_o0IB>|%Xx{|OvGMkwK=&k_#aBD; zs)e0`rX2o3$(ii*056BB)4};w>GMMzTq+}8fN8z~SQPuwlFF?KeqwEU^ShZJ4SpBV zg2N^xJP+JZr*c0Rj7MFdbsVYX2mi^pXkWK%Y9A4Rx6>-^g^umV^>v|bk9Y2Gzx$>- z5}9>TKlk-sKZXypZ}uM-%g^pNXZdgz!uQ3)9p_hK+AIRkhviQ_@EKy1bf1f(2t5wU z5j=}f*==rcPq>xy4JwK$1*{K>wCZ-&ceYI%rr5V`&i4zyqoeVtDGlm-Q zu#4wco0Pr&*Hq{rQ$dq&7||75#nxE~7-KVu=vDppDnB5aqL8}cyH=1wL$a-E^8u*j zZU8CP_0i;HPauStpHHMEdqj7ShEj-w>$r7ue5@ppqq|c1I@0qkl!8EYAtl`mrH4*t zS*Vc01??I~0}1=*9pLeqE=iFOWVzlyYj_hXr282pgyJfomX$^ro-7FjhM~!kj~~D@ zqGsgKAYn2^;4pGhH%_lEVrVoSQUj6*ANAz(<|jVfIq+HK8T13eSu|W+EkMeCUv}To zgq=Muw9BD;pHOQ^wHx?aBCS{ygFz$ArkhR^uz?I(DXR}r*K0%zfT!`)zzj>=k43X5a0sGWsYdt`nSaY<`70G!A3F) zlVqYr+>n9Su#6m2FY+&Launs?eJ-dRp-jyu7Y;o%fK^QDwU;PO3e6EXKwpC&T{cQT zq5%9V90)v$3i9ZSYB0ltLS2W62>5aFG{iX8K)wHiie3@#y1!9}=ja~ZGXssS8_KZT za6Py};W*D{pT-JC6G&&!b@vc{0oO+hPl{qI1<99Bf|;-eSd>B-kJb?R#fF{(*^Ch; zc(CmvGFo(#)N7Lr{<6jYGCx+Oo;kRFdKl~($j}Jg;VnT^l6@BCw62I5?;Gdxu2i?;M|^USB2%%mR3KY(Ci=#0#t?;7PnceL4vTrHEqwzB{r-Fa#XR5=Kt10hym454Z^k4VYok#F6wA5NAda zyLX|K8a!Hr7)&o!k7dqj4@h`VLNQ0^SMVEP0JR2#B@C2k@gN3i!84=8!K?|pbkd7# zS&Ar1ZFhTL2i#|0bKol8JjuYJvQ%)T!!b4}3AkjdfQ9a9c0910Bp(HGzwe;@q9sL# zsfX6e$&c}%P zqEnSyH(Ws%G7cQH-s2F52UAV8BsutQem2FC830vdKd{BTa|x%$HGH|!8YM}b8}Y-Z zfVXW}ZT?<#174Y+YkX18-cVhYT!CCx9JJEcLNqJG{d^8_B;v|Dz!Q)FiGhk~lbleb z)d=vn*4yAyXM$j^HJtJyh}H3X2Z*+gV?A)u?Dtq*?V3qY7K*a-gRHw*SSl(!-kj}j z&V4rq!P#YjGm#kEJurNT# z7O-VUaS}zH`?qVv)vy`6^9UFX*xHH3s(~5HiQn7P?|OolGoXXxRtL`V)b?9-f-Pd{ z%|6GTy4_}PMEq`u$){T4Y1ga;8cedXmEOq31D`0wgU4d_)H&i7Mi)-5?Bm^bT7|p@wW{av{TeBHH<9GzX|O5iLXk(E7$B zxiqg*7rw>MQKqh_X9}ffUJEh;J5%H~5z4w1y0i8+y)e1IM(YBsxJdx}BY_Tp^PJ^=yND#)neaLXuYCy#A~^hsbT9Z9fsg z$is+|Ts$E3Y61u7_;`lh6`Ab~i~R0~#dbR&TX0mJKJhc}H1Kf=AjOnsIGy_kISHXA z+$LK3OvMDF%($DUO&)(*dv$jU%DTggSFdjL_I;*dC$iLMP&+qD!1c#jNP852bvJSa zim0LzriX3vP4PS#+`5K)JOQ^zN~>=I*`%5V!s|+`;hA^rnv-1GTCOa!8(zK75y00k z1LD6`du@S~Xc8EU$T=sD6RJU!HF$IUFez4t3J3{oUzef$OlWZ;lrk&raEnTB>z+{1TfdR*QWP8HRxG`cQ=;je_IvxuV0uNuW=!UR9Tz zw@ijjKVmpL6GI*^;0|%fvDx-?JCpvFq&-+&Fv`vxK8<%|XaAiHMT}=?@n=W}%DevR z@0y~wlRjROR0%}4NGew$(>^}N_i)TRw#clG)SSzPq?@Q>kFsy={gO}>y*cvLwM$Fs ziR5CLpgy}N6!^lMVfd9o{Syw5NhVLeuhyhSDijAVSs+2QG?CQqQS+nh=^1FR!5v;v zkYRo-)bK)@5Lo#9PBY9vy6|A;<={?GSgD>wgc|N2`hn)@!`qOSqJ@X&)A)*e(`=KJ z$fQB8I}+SP4MMDD(3r}cClHB*@5BUCp zPE7z0pgZZ3TjbVAEWMs3^Cu@??2THMLLzwj#TWfRONa2s6!(15vzAFt!Vyz@%hIXx27|7z0VXWBppI;dNSBGJ}xSa@TVpfgKELt z5w1QtX7j&wy-$=lx(HaRC#+S>tlG_Amm4Q~}AT^0@iofM^_KWl1&%flR9Updpo2 z;oe8?w|wA$dO+)z3}yF`iFCdm8l0#d_|M1GdM;HoUa;dhV)x=FvQ3U4q9|v$jr}*F z%#x@87q8^=>8Oq6U1ni?-HZJmI8#Kzx~dmrvzk>nlPsZO6)P*0awq-No;`fWRL;xk zXcZMYx3nuUITZ%#a+o*9a4S-1+DooLB8Ma#oh*poWD-&&LO#EKayKecZ~+LTp}OnT z{fnT;bEuZZN8^jmOP7dxw%>?HSP+Pb%=nn*w>f?nDgr#KnHk=_{f#jM1b+W^|7IxT zXS}!5zyZ@*;`I3DXO2SKndsWMY1+vzUGs3ToHmrP0kt0T+PHT%6T!He$J29nixYG` z8!J#M-2fx;a{q!{6)||obcY-E5;mX~J@BO#B>i#vT9C(h4o*Wh{ZSlx$yN-G#XD@7 zdq_Rhz?V{5Nv8wZ#aE-|h{zt?KIRu-1uTmuazw$c6NHPV80{NiZu7tgTVbyxIR1Kl zz}-bcSrqk69NDrkZBaO>4n|g2@za6HehdqSbEV*ubvR|d3UVkxOtkC%tK>b*!&`3&^k~$~ z;i%yI@5E=nhkSnsc^qk>m7nNot4@p4y*E@O_kf-*9YYwS)0W}H3_)n@$DF@pa&di|RjJ$Q@;aN<01 z^5Qv&_211WezRXVGLL9labAZvIFy6vap143aN(DanOo06(R>-k32TS!q@yFmZ0FykSRu`*Axi5uBYmlGWXn70$;4~jg-)#V-r3rGv&A_^4 z4T-Kf)(gx2{yi}H<<5-I0sZ0M3n84yOpG6-fHE8hB*)5IcpPqZfecayf|X5z4Lnrl z4wUoaIM)pEqM+mqNt)}A`TH?Cp2Uk`bb6? zP7yL5$qs9*V$bIo?C*_C_>COkg;Mz16L%bcT}*ZY?BMs5);xPX+vb2%!WyWCPowm& z)>rVh>=7$8>S=}|NbBLRytjTJ;19fs?YF@LwT5hv2ukDxuHJV1g-*65a-{jv3@M`~ z02WyNTAOze>9=KrJ}~BlCI};{r62)?+_YPm_2`r>_Waj??g$ndjTRlg-I+aySjem% z8GJ(RG}gd^Bgg2F`&#kXp=TwoQ}Hk;mL=wy&a7|)NRKw>n}4V=pl#t)i?>8XtSjKw zCV=Kn0$|1Z9YKUO6LK=aLco+J4O1{rId+z)0~hTK26BGw&0)ar#y;drl>Ui#`U9@L zbK$8f5a+UdD79QK^6|p)$lhhRG^)rImt125I!LFrVLv~l^E!|{l5GQm*2Ag$6}V*h z#*QBVLwW{kaMqYV;K0YRJrt`^)<@K1K%XgBmU-hVoXbw`u3vr%|C)IRN-*ms4$ zW?md4Q-!4Vop$~#k?d2;s_{?_4kLdE3z~t5y~4T~-^djsIG8n9dJ!J6nYxztbfpA{ z?#hr)9s|bA_~HH^V5%+?Ei;9vpWA*+o3$B#K&l+{y?r@M8- zvo*hMO*Z_ukky{WzPF6q~e-K#jVkDtr8|d+6{*~V!htZ>7*cNl5KYO2;KnLDnpW0J^>y>60M-z0vt_w^XK>?aMrmg!w^*@4rZ2c9008OcdNyL`m@ zOS1D((drW1$CCztL@w|rL_J7BA%)7tW2o?u_LJ%k`}o$Jjm+nXa<>3)&uh<^=fU4FjL}VNgD8NZ z>q09fvnCmwbgEpDB!K0D*Z%rhh0kBcWG|Pn0D=U6dO(8zKq>%ne#sC*1X+x0k^w~l z#Iibc=QCp111Qf<9F%m0dOOP00(L^%VxA+az44EjXgb%6@Fq8M92j#jutaj$2fN2T zVTl5T-Gun&kZ<-K;9`*ea_X+tN?J}$ zx-V>4`5s_V>+7JGf|5O-YAZ3(bEr=kAK>edXZFFlV#LXao9QG_Q+{Vt55w<8Ze}oH z#9_U)Jht=k;Q0Nw_BgBp@O}i!{=ysj z?@Ag6@KzG<_Vkc{RtlrHQU`C|6$<7FNM|REtUpY(RH-49vuwj9A!w77FXT4FVR69i z+Xkekywzj%UW$@d^FXMs5FSJAk*Sg5j0m(>pc+D4YHkN;-u{~lgW~#T280i2JS?M$ zxcCMWQ|?@jsXPVA*GtXa; zls-h-P$EZ_{y~0ZQ{}-2VDwRja;)FWe2=KVHE}$oNOIGvVTGgel-h_7I;5Ve1q)qP z84{$@eohuo0x_c>WKsg=+cbdSQP%jiCk(wX6vFf?R?DI6`)VT8UVVdXiNkWJ03Aqd z1#mo7kTm%7jcJFJ^FT{FVYsh!tWspzt#K9|C(J&l0bj^45O=nd9AZ4~STHnnv(WxC zi;o04?+UKu&=B$bqifLRChhn;QcZDi0AdSRGtQtg^l1j}N{7oM8C2G3c6#yFmuF1G zl4_6NXM{?;C{nlF&}9y)XH%>R>HhHgX<}8hsgU^6cEjR(t4C1%O^_06d~u>OT^D2r z$5narEXP+@pv;p1MShx$p27koC6G@NPDFoSQUX={tsz0etIAZ~_FfVf*Mn2g6m{X0 z!MnUEQ8>`jxv%!$xgvsLZexUPvqpNIiSYZ0k&8;#%MTX4_>yvEK*4wS)g>SuzcJTU zW2DK4-5UE??W~0g-Jy)q;G_g>Oh43BxW7!-|C(n}h5DHf6h}gwr-&ahn+!a<8|a?d zv7iUo7<(~^$0?08wc*Fb-*BC+#N5QZvMIb=a$r;VM!*J0E(km}O{r6y8iGWx@z8Is zcXUGvyo)S{YZ-2~$(3vw$^`$EXji}5KmXae&)I1hs9i%ln&Zx}_1a%4M)bl8>+k$! z#lehD)ME}|9q}G!`yPPbmIHoLy zL_4b8XQ=#RbY~ZA^zBQ8GgHxFj>|Uhm{WQ3`e^~=LVz&IJ#mQP0=$1IQSdXt z$q%ucTFML**YmHG)+R!ZvBCBV6Pd6Gmr1Dwyj54Qq|ZG5cyPa-0{jeL|CkRr3Iueq zPVA|ThCg;H5oE0&8h`GlB0I1Nr3E(RP?*^ZF#4G>M5|Gvi@^gw}4;^0>a2^x` zr@f8!X?ncGc-Cp+fs8{yF!ytQKy&0T49EX+hC0gXnK}d`J`O zNkNMQum0z{v%IZIKuMPXAh=U;0ao82<@mqhhQM0QD6?cn{MRqbKzVG1;CF@8aUi&d z@L5ezVsk1?ZiSSL4X7MNt=Vf=5>diLX*QZWPUwGNv;~J%Hf#}#i^2^$U5S6naOocb38AQzszDjiABR|ldBcV(d*>4R7`w3)Y62DJoEn`ma zmOBdpVY};uZX;;QwE;8mN+B-&*}OaA$7mA*J#%)+NC1U#zp}|+gDD27n;JYV^GL)V zpMrC7mc+os_g?^h!7RZ{VE5uxC`c{4#=0L`K_C>X5DyGfk7dE2%D>zQncM^(A_1BO z`&m1NKpX}7$WYmWy_jC)+;00ms3<&vQ=sGpxA@<4XXF7)felv!D9-!)3&MzTr0!J6 z#e}YV&Pbs%!1RUN4wP_RfsQsEh{lx4*J1Aa@nD>Wtk9@#Sguw~0 zAD{CyzJfh5g~RW6zW{JBP2y}W=te?n;h>@_9($i8{WH(H%g3XcFohxpd1$A31fZYN4zKEItun6nHP3c%*E_e!rd7>(0mJp!maMr zJdjryfY7$!>ozRr?t%&ZpDY+6-G%V)^@U>%6nO8HKnZr17@>*?nv&xv13?;{%N!w# zyDW~TkIa$!A|58|0f_3Lb+sTYm?09F0-OBy904Nv-J}9bb|OA0-+`)s{YG)WU`bTi zVQ1Cqey}Eehw#*bzFQuGW>tG`33G7%jqnp8H$J+b<0x7{7XV539aR+wkY&X=a^6eq zQ707f&o6+Awu2m7L5MF}QxW2j-?5Ac8e_Y{NVgD>WYsBMBe@S?kH(kf{dv*3fV#8Fz1Q6@Yy6KU zKurA+Ea>6%H6@C78stIw#sWHcDW=25> zKd8`Kk5pk>mf6_w$(L|+$U{XESCtKS`-o#v#~Vx{Oq{U8BUB}B1{%fL;$jFIP*cTo zNhSewxWWSoh5VP8*2huH;}23?O}sF=8xkHmZ=c4;XK)+h>r(!&c8`*Z;?wSbkMlj) z2h`cOo_C|fF0H;9_JQ`ZU$`-hNFx3oASHeR^{-zohkpNed01K5k%HE0I{k@xd$r>BdgfU7lCj2^UxID53Eu`za(O0GNMcPACKFQ z+pC+neYO%xl9H9zSB(&!heza4}U;A;vX*;=~2~`s`-dcnOgmM7;jE z&KUtK)da`9D(VWB*GgEYjt@FwyIrfpxPZ!gX@i8^`+wXSFeo9C#k8om0pywrA6IJ) zNm6+>603cEv+mZS^YX?sB(?%O?HQ^DBM}eCU%ph~IW&iNDbOI6f@F1i$r5O2Jyksy zgE~JYDFriSC~Vzl`y`Fqi7skppfqtB^@e=ylfd~}FG{T~(K~~t9eMW3gx71--^shb z*M&yRQJd%DheBn%TF7PA^wmtB;p=lT2&W@A!mSkN@!t=jdx#P`CR5$qpC2%E$a*yO zoR2eph+Nuz=JCRSnD>LTAAp__-G~`CeFJ*SSX!r3FY(?Rx_258`;09ggO6V>N6@XN z=Ss}aGv@4*Uk`_UkdH`y`vV4$ygLBUZ{95csT{g$Pxx=_aMIJ5&rrJMN}FG=`_yMq zUK81zMN^mGtF4ZSx-xn*FF#rUcR#RQu=S9Tk};^PZhtu(`ac#TUQxDE(XjRS}hRhDM~fEa&>{dj`Shhd3XgGUD#Up^>+D3NzyzT!(#8VZK780k;}xAD?WGq*~ji$B9eOdC+cgz8PRC1|Z zzwf)m+{^r+OzMH_--KXT_*TJ&67-=a_wF8NvWuR=%j!nk-^h@7WKUxLS7X?96FEqP>4D_=XdxAcBagKr@^X? z7AAuK?Syy73x#8k{P%2o2q=3iu$3$sq z8bAB#8E8Teo1V(^xnto)7WA8WK(w#_&SG%uyw5|-Qqq6tOWfiCf1kgXlx4nX*6~cG zI)fxbTj( zn~B#CRURXsmfeaUW34)}(C=Ni>3gukb>_9$C6cEYLzQN|>{_Uu?yd5$kDBXboZ#V5~ z|J&_YB1+rLx9T3U%;(=L)~d994#>eM6JWE&U}b42L+r2F4rCcsR{%?XktbD4B2LiThrEDdQhlfwT>h;~E_eH2#_S)YU4lEbF6g;x zLF?$TEl6IRWQeqqen$Hvr7YQE^u zPwp;c%J{olBLoEZC1Y>@n4~D&7Z!{o+n`df@*V0;=l!9i9lY-VeT<7lpU5!uxjg>- z1UdsPOWm^Z@Yp?sL)2|_b8BU{Hag~f1g|oq^H~SeW)nr@3wStVx1^1-=k!nX4xz4U z?EX8rh?S^PY<%McO3C+i_3dc9_tRP5tSm>btFZnL?pG!RS#-Xg$>qeEa(& zWhtx}_O7Ws^>CREZn2(|>K{s3t4)Ts+Go#)401q@1K`%Y*Y>ae^@LLIRY7sx@Fqh= zn;{96=ifwU3Tffn6*j_fjl9bxd?pl9#q>~lE@(FvaN5jBE%Gc3_R4t&@bmV$vXvIO z_ox>WiH4x^Gi>;M*AX9r=+~Rkkj;JwxY{cL?*R0a04Av&S;;x5SIHTJ?tByIS!oJ5 zcjp>d1N4WY!)?C*i!tP!tDjFI4i`*Mo{^|4Dv7u1HuW%-&RpmFvi`fQTx zP=N?9L3P?sYJiiR1Hm=9_l(kBaj5u<{8ea1k-RqxWjjDS>NLEA-o<#-x~tp+;e{)B z)Ct_$nQ=SALk4;FUlhze2K^cJv$OSk2WoZa7Ago!;h*2+a|@qz=V-r%goL<_(9{c> zVfTF-kM{d>^6U^nTE+ez(H_C6;Th}S|9j$!G5V?e<5WWEh3{40l9*5BjH?D-x@LCTPI~6GZ-dxEF zW0dY+38CY61~;>aNyfJ~?u>;ouU1-+Q{Nkot_r^$XZY=ESro^G{>kQqd4L;>Kt{Sp zHI~OJLBwf!G5j>7fO<|39usvgd}UhM4}w?5yjR{AI`!tWbXDBmyb4-2Kg$HH8hd}O z&d#q)wTjqvu=g}R;&Yho&Z8)Te0X0Bx3<(Ph|EM>ColH! zTz+o(`7!_7ni}~8h-@5UdG(5|K0gZXhbM3X{ByEihANHG`hcTSW?Cp$M6S(z;Z2b7tC+g{%H%8UDPvoH(~3f33fgzN>52z- zT%VPD{HR$`0rl5RnW@c{sci$>BKt4PXQaH>nrQh=9#|GR^c2IcHGC%R^Q&w4ZkdGJ z%!c<_&*j?TV|l5mRi5Yj!QoC~^K_f-cB=rMjT=L@t219#JUup+p0-jLLQ=!0f>Fw2 zWaVyms@&d`9_3QMor+icmgz+tr)S3@AgOR}qZhUxpXKgY7%F#wN)Sggy{NMzL;=Oh zrLI#CUpXwt3)zj${#lzJsMv0g7jc^T&DsG?7ZbnlmjPyT^K7Z-O6H4q)S%`@~Yf9JLOyys9M5G;CMxnIbu_p%o* zu2LZ4)ZYujqkJ8wOe^MA!^(=C2I$=u0f=p~uN_2YrdH2%oH8k!c8e{8uX~r5yNm|( zm==OX^SE->Z@a9=t`OdHuRmvREhP4M6oLwY=X`i!flaHIAM+~^avBn4od;ad(6##6 zj*QUtr=3|E-lp=i{Y5X@U%{6r{!l<`yR)LHr04Rpb@4*Lpm^_u7Q!C%7-mOw??W1} zzZU+`@0SA z;O#4ON7i{j_!ir=GPVK|JLSMpf!c$NQnmm(cgyqJ*%t9wGLv)}@rT!>wFrQAy$Dad zv;AkA;yQ%l+q2igiU|0RR7Lm`(9g!)I^l6P<%8IfL)Gx_V%0mcA?&UDLR*K9lisF0 zz9g~4x+Hu0fzXzQ0PStctsgQ?)L8zj$xAY26j=VTD@zhhlGuG$`OedCbDCHE!V+xX zT~*9)(Z9CA)Xf23yfWsdUrEGxPnf}OgK2Fqd=d^{z57RDB(&EQK4F1Rn8y5iUlCQ@ z`xDY*Zc6RJyyo?6)GwQwXcb!|>-01k(?x54@!6D1{By{P3$AK+kY)=)?ge8+!%8$e@@oq5PK7i$~=@|)Lk|FZj9-#h{7a5+y z>T%FFWNNIG^z0#iTDu1f*OZpjJxY}Wi7O)f_f=r5JyZ!Fc{C~qiu%TwF)pPyQB=g5 zRV?K-+1>DLDI2@?#JlWFb$rz^C3*)kQ?>l$r7s@Ww?wgYbb?bz=~SF+#HD-_ig6I(SylZof@E5)ZY_lmJmc9FCw zM=Gj~-d51m6EnSj;@fdWmlNL{6~i>Pf780Qlsw#=*s9x%jlZwxb748#+qUb|_zVB+ z@I>mIr-^6C?0IdPl+(9woaNZq-;&j3*YcWZHNo|yaI)ZOxumK6$L)UZZ)qj+30x5@ z4=xauYaN|qu=|-Y?oHeN%H!HEz2)@T?+%HJ-+c!6Vt&dj5RB9@5Xvo%A7ts6el;kA z`FUl5Y5K#UHs+_z0{8Ujpbh5d^##G{)j@BJ$DRdw)&q+X=f^^$DtA6F?riEUZGC3` zdF{L`5&qA}&WVWQ@@cJK^p?)+JiG5tde0wq|NPRUwPs1MXd~7TnNmonBA~?J{wP23 z-+!agf7Ju`cdQO+wqHhoFhApSisZ65#quEZYxGCD@W$--S3^mXzn!DoV0Y^ z|9+%lvdmV8m6>q&4>xs$ochsxMtU+Lseemv(^9ZlDn{do7T@Zb7axbe%{ z82tB$31>_V;({F?%F#*nCDMhd{(T%~D?)*K?!DdL6mU;?i6wPAmb|;nBm0!c@>Tj( zs{d|WLKPv`5*xR?`@J`^vVV9f3L=`%Ob2i#tfXn}P6Ry4vx~AM;#O5&WxLPCL)2Ay zsXCtS&=+D2lZ)vl^8ejf4{lsSrCqW6JzAIX5!N!o)GiUAB z52xFm&|sKq9{rWEr!X_2c3l_n6X;Jmt_nBSyV|(>y?R2JnUhWOB6X=0$&M|diX8X} zWQ`CYI659fF#BWo(Q;s_Rf!!;q;%+{JkNxAk^OgLW4Q5@>c!37?-?+|%mf?MPafIc Zquux0NJa5G;U4%$MM3@iOL?<^{|`Ug4(k8_ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-3-errorhandling.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-3-errorhandling.png new file mode 100644 index 0000000000000000000000000000000000000000..77829f943fd9d81502cc6ea9c05a1acda2badcee GIT binary patch literal 112800 zcmZU)1z1$w_clxlh`A zYv#Yuyw2PF@1-Io@*^7#K7uNzwN(FmOpQFo^cZ&wwj=Qjs|@FfRnn zMMU07iHJbn+1r|!TN%T^NQTC%BB?3$VyA1yMhlw3BIE`xA;jPbN+Z+xp3~Ekpn#;q z{W1+CuwQAR(qflq6_b8WGZ6ywgc%%#A-*!!emfJ-L6f=HeA;x=Tz9tYGjzW-P{(q$ z1S@#JDM8*RHVNBQu&K2ABLkcc+okX zABh$ZKc8B#(TD^xP+>3;?YJ2-Qjt!CV2&-yB(-2*6#G);=s;3TC4w?9-~(N=UYhpV zRlO|eKha5+(|Uor>J0NmnUlbd66RYXM1f)5L>O(-JA^QZ*AE24#@)&9<`(A$DiE?l zvla!(T;T2qY#Tkwv3RM)b@USaI<4#Z{N5L$`=vkm(IKUn3{L{#Y9Sl9uAd4a zwDLq$I$s7+B4WD(7~p^VBErECO6tiW#R;*4952}}+j!g*Kr1jzJqVt9-}OF;llQp? z9^;{YZp{Fl2t6Ki`lvGyzZ8DlV|>#x7w|&eHp@yB-4)J`2pPWaScep!g5WX%hDb#G z7~_aQt&VX?OD*r=DV^A$g+;a}@XhPqXo>o{XbPt2*Mcudo`J1DdMzMbwRilU#P~q< z)z9O_;93LUZShJuK`gfn^0I@m~{;$oOg#acx>SS7dakxAfG zZ9MpF5byA2R$wo9|4a}6@+U<#h{rdrOe)ad{g<5s}r^c?H z)m}etf1hi7_lhyuO`a*POK$I&t5?SbQp=;hZ0fAqZ+ix4Gg*G)_$ip8)}iNY%SXx& z!AQ0+oK8=tn$5bvkM3L#4L-rc98I;gwRJ4ljeLN)tm`;fzEAsU(y9>53}c#FhiRAf zwIdG)&QqqaEiE4@+!wd})0rKdO)KiPpCITtsSttWGc5gc^^a*`ZOV*T1Mqp71aIMs zTe-`@Ru{$wi%oM`mEGAnqs@Tv+JuCTXlJjb{~AWUH~vk)SKa5ac?2v)nG zXS5(WL8wq{3@bSnm=x29l9`xBf;QTz$0QefHkg%+D%wF3#~#(q_nh>5i~{wiZMy?Tm>E@1SwvF_L0*O*hxa8rIMtnDU>m*!RTQ5wx$UYR|#@Vr;rb#F10vup|hR1 zon4(5oqU~Zo%E|@t1zLWV`Q^2jZ!h>=taKD(i$>zVipwIlo!$b)HH8!-*C|&(&SO2 z#D9*bjd!LcRH`pISKeV!PqPxbB$tSFTi;pFTQ6P5J{7aaxM2|sFc3S*?^LXyIit3u zzWAc_rQr)lUteE-A5q_EAKXS=pXHY;+6)!%%zH&KrLGS_d9mL}71gsVv^Q$|=;Aq) zX(wZMCFHWO-zC1YETSs9Dk4@k|Azj}N+o@lV3Kz7*<}AWhRGfaYzx_{Pzy1OcnhQn z`fnFSag%hD>bnFNv%4z0gS#9PNUUafhODG)r>xkP`xBf6C$e$5=0&)Lz2k337qZ>n zpMlYGl1J58>8<&#-yFOwl?W>|&g+`$o1&ebE1fKw{9amgke!yvC)Xf-SLj@Wo^M#x zG1WV%{_UskR{c-`_!C(Y*)*4XsO{LTaAGfEOl%(y|yG|CLQ zQrR`RFuALwC6=5drUB-G4=Gj27peHEQ7jqeI;P(9{#I?}Jr!!!cE47e^5TuNH{klk z`%OR8f3PYhE#@w6m{PXX-J_Xen^Lt7VSkZIkb1$MXr-vBI{$W_c0SCxFfBE$)uu^% zn!24jO0iv$Eyrf2c!uQw^yd#Oa zfO_=87h!2B$=L*EM*ffY-%j|7%8K%erm58AlGuMQh(3hYVMWQmPwh;0ghWIqBvk{m zK^3difh%cRpjvR{puRA(y-OSwvh2``k%e^W_*UjS^M4hwWD_*@ErYX)p59{ch zKaIIZxsgNEIF@J@F`&B?`q8v< zkTsdhb~8YIKt0uz?B}>?%^*{%5_5s6&G7=^8TFa{=7Y|@nwH|dv-j$K0%k{S9YvHk z%41%ca+R`{yP@B-e+24Dkbh8G*jJoV?eo%bRp;h~AB`3F{RW4tXDyL8PJ`bAWSA2i z>76v|^>OWzKh9okH21qGn<>9k;niTW5$lk??W>5lRh83lDjUnQU$f}5=-oYJ`)!#r zmGWJ?`m(>i#A*85eetC-zB0o%@0RqM$J1vmXczcs90|7fl~Uz9o0}_xXlw~=@tPr; z&ue&RQ_3#}Y{Qo=YIbbjEMGb`eDt`AOd>p}I;^VVL35@r71<+qYHE1zqCH1g-cYpE z?0rU*#<$_orqjE$bvJgYy-Zx)asaa?_#hM(Fd1wX+=mzaX$S2gLOo(pu2)WbFl*47 zPKPcSorhq_yK`M~Qt=>1CQ30fo&ev9&VrDWo_ooK^sFp~3!kA@+id&e0q1$c-cd03A~!yLy{=XLx?NG8rPcJ0 z{^mplEtFdOTiUzbAWQIceOb3|iyPU6_?6t!plOZ0ZBxCr>)lyeoePRf|6$Q`aii_W zmQ9RQflJ2*Ct@$qaps!-eg{ufZIll0kk5Ty!8*!?)P~YpLDlfrbiy69-KKNT`?GYM zY@Ccc>8FY7pSO7z{Q*8V50j%A6U%4P_ZG(&s~qmO)h$kL^XwqDk5Af54Z}zM{M%kL zCov}{0|lK53IYlb!nYNh#e>rcw))G5=$mNhpE(U8_f=s6UJdrZtXw$56!yZ5tlWe% z>v%uLy%O~{i6=SED8CPS?DB!&nXg3lZp>TudF zq2y7OMg^zCvx|Ak4a*=s<@XGK73h3$K+XL|zp{0yi?He$m;%Q>54aScjn$+~razZS1HIakaLxao~34 zBmL_MZr~hxnSm7Y*CURWe57h}?;s+!_Qnu)dPaIiQvT-<2!z+($b|d7sQBN-fnR*2 zW{!>@xfvK-TwLf~Sm3}Eb9NcUi^`})&QV^{P4TC#EYdsx5#8K8F&;~yf3l}TzUj_buxBhGLe~PO8SCpOo|CIcnJOAI3Dh|f>BDU5* zn~wbd&DY<}{_oAd3-U5RhyFjJ_#@}Pt^$JQf6mMB@1*fRXMEEi2FxRYxu}9N@C~do z=nJk6I8gujhMtd{t4`z>!@vl_NQnw5yTb0Lp)9C$;CE+87hj5g{`?u!csbFEWL7Lt zwEAUlLS9J>9k@bOa+Xxa*?5^$I#)DX%zSzJEMvLxscy(4^{V-5$Ya@K_!K{5?<=C3cddKV+1b($luDlg{E zSX))1c_cM2gSy(6_=fC=Z%kvVY@S-eDWh;)g<4m9$*ZucM5jS9az_!QS|qIQ=JD>0 zdX-5|WkU(1%)YCFd9&?9g~7!$bDtaXG?z2!jrBn!=a`K*96ffbFFRFAwd9vx*M_~- z{-GnkeGwUCz%?A}UWVRYlpb<*cwi>Ja7H+?v-j!mb@K&7h!QZ1e{VcuOzGyWax zWZ%aY=d?68+28->s`85u&i6R)Ko>cJgU=eou_4wTQ{hPxou8iGdmnw?06~(X?3{A8_m!S8vpt_IkvIs zU;9f_HI{^oJY;OU!4E?&>}MzrH$EqNQYSe_%B{Np`+ox``G`9{T8xSkK_r~Sll z@-(0PYq955VkF#;AaOprB#TEW`Qud!&7XN4571A?&tHwL9s1X%rCQMkl$ldv6_iHz zAJK(aPC7cct?Zhdu1#!=oi#5zE=}=R(ZheYqQvsnrf+)ovh$$YPz86SxX4q$xK7K> zM`+=2u(;1ll{NmPZl)&WICxBBJiI z&WJ|pKP@a-^V~SLWA5CU2`vaUOhs!l%*ylMc>h>9pLvmXxUy*3Lx*EcwbrcpwuToF zm`PNM{diD>@&0v}{T>#gUGSm0(yh*j8vngX%);YD2KUi#kI7j42KHEtW*zc}bovk; zgCh<5j+=<}rtjE&X(A@Oj@DPD6w^-TZLU=4%=Em+8KAm3Tpl%%MBieQE z>uQ$mEWG>WSuJ?zcFB-dvk2?7zUxpO&mo`7DJ-sfXk^NBDO;1#4A=J7$x_@{^T-=LQ_gNF8%og9x zNc~ujA-vm0$Ia6leoSqIoDPc!)0>FV8?(Q!ZU1YupC zmc4}+?#&Xf+kU)%ZKQS0moWJv2C}6a2_he-$RUqkikiJaRJL>-%a0Q2o~4koaVAZv zX{?%rtzQH21>(b#J3|&V@n3z~r%nen9gKVS5bd@IG!^OzbQL}|F{yrPRtHVjdGT>w z)w(k)ccI<+EY#7K{roJ#zb+xjzwTpkPM|U@zCV+5___QLx32F8D=(?V#{(0JyaJp2)Yv*HULJCzasV`M6Iv$sd zcm!R@o}D{9*G+kSz1N65vKetD)*SdC)Yntc(;xl_uQ6QFSN9>)}HCiDr+3QQ=xA$O_9AXg&g`7 zr$`l>r0#9KN&kuL+Pum&|4>%6>(=jHnNC~1%1RGRacSsiwE3Wo^uc6?W2*SwDBvPNrAFayWxd7t?I>Zv4;YqQKXqq zq5^X}pLC5bTM+F!Y;8#{_C8EJFx?s>oOf?0j@srti3_$wJl<6<#izc%uXC?>tEBTI%mqJLi{%~k4==>y6H4w+GzaUkd+GbghE1U*FVKIV*R zBYx-bxyt(RKFd?xeEvW}y(SBC*c7ZOb2Hdy_VA1_7zQ{XrCmKym!4%_#a?6%MQ=WR zewhSHc=5kB|AZ1IE4hfiQa+~P)*_3G2zooBs?w#7_&9**bt| zJSFrWcwi#c(+&s?N}uWJ_#Zq$4zh)#uOFdh)ZUz1i%4IJ`xgobvZB;(3-7(GlMxyf zmHz%euwe;7%P~tuZfE_NrePV@zmdEF}R?`+$?Ln!Vl7MvF+i zm~`*@cJI3}P;JH4l84MjUwoKs;$g(>Pg9xiI!(DQM@toZWrwNJPmg!D$^2gK*>k5G z{r%XQRwQODUs-1U49GFrvL>}t?Jo)Yae`g9(p-$TN3tq55;Ub}D~!9&c#}D7)2*s! zP3*^{sCs*;G6+xz;GeySK5BW=35zodo~yBCkRY%VwRbGijUwQBuj6sRwlh%_N8qwx zr(VRMt~d0ROR21G6R(XJJ>VHy#jR~8SL-h4|7Cf%;-!dg45P2_ zX;}UIroseg4veS2P-|B|An2&d$N8g}b9|LI^F&vXjqxFj2MS$`^H zmeH-zKoV0u!Px^)B!|O#j`iqUaoH(-?FyViM18(Bo~yvw0E>A6!$F{7cjU!frFo73 z=dj7g(I8-@?0V2{^w}i>A@pC2B!6|>mr_rhF~fbAhA+aRyhQz{IDPZA0bi;6PF~{f zs6eN6)5X;3MxySn`68jqf+AH0li?@X6jmvPZc>|?MQ4_-?VMfC3s$PJ%Qv%L?Ou1f^Z7uOI#O@bI09foaPYO`RxFSNcLd zqK@-cEz?@OY2K!xbid{4G1Od>BE->Mmdg;2z&t)gMj1umuzrue?(udnPq)R}ey_Bq z51nT}y4LnIXG!4kTEzQi1BZ_;+}!4Ag0a@~@n-Av?x z5Bj9Ko+m-+CP2GQ>7*@}bCwqqJbi^q1(BOcjpuMT&3A`9eD+I~=1Q}s8D6q62I7oy zEg8h#*IB$LomfG&uPzANug(fGykiw}#CI+cb2`v@WWT5_=0=oj>F-Vy*-t1+P;dso z@0%^uo=`5&2WH`>uQWJrX9Z*T<%ycCHmGcW>Ie#>uY4{WT%R2Dw_?bp_6wDN)s9KX zKQ+d8y%KN=>}|^*Ffo%^B1l{-zVHa+ksZMp(|h}jrDftJm95)LO%3S@h^t!}J~}!+ z4^Fe*cL$Stl3acvyc=JX9C2$ggr*A=h`j=j10ltlHD%ZsYCIw4qs?A-l!qd+LSMAp zHsV7sypkS+UeCF0rCq$FML^5HR^T~cRVLX%mPOu`j;HOtzd6hEez={pMINNe@O-~J zSzO>19KyTx)uD2x>EUv|O0SJCgym!1?@##Wb1E!i;U1tO5EaL==g(@yCw$J2nS!EA zu4@q*MyroOI66td>~h#HX@x1t9<*bJuv~wpu|;5GF7A4wZ`jX1=-^iJ9Ke$$W3+qN z&WW-xd8|sx@VZQXLBy*z9gzc00;Pl2^dbb1%dOzHEGkuMj*7oG(=s%j$I_1@i&vh6 zF*H+0@m(gpRj&{|3wSDj1xG-N>%%H2i+>X3+|D-jV37PQ5b|1toa`k9Bgx1LhE?b7 ztnxC6=Q97}wIQ7WiG5qWOd@@XK=b?~JNH^R%X)sQ&Dd8+Ay|)W4#j}$CW{!M__f?Ns)MAhMZvbAXsko^*=_Z3tTQ2V~F?O$=}`LG=gJkM2!0P?=+_= z^>j>>P}2C738i9UuX9CGo7K)x?qxu(*=rN^-NMr$%B!MVO2YC5b!S67d=%JN<7r{o zbYo65KeCUCs&cxat@=;d{m(`D(He7AG5 z;AMwlzI437JD+d<0U9FPMNWyK=2n$$bzAAF$HYalj~+)&s&`?W<=w38`61vhnN!o! za1{;lA6@)>@da)F3D z0tKfN{%!Ib@Fq=L!4dx5NfVP$#Okhd*C{Fj+iGD!U3|IRb)k4>6=69JiQ8uaI}1@& zA6;O1Fl+6;TW~=RjV6r0inMpMkW?Z*qGcXTP*)W^ za-Kxi8u=HnOS<_yTqG7e_KYmUv&HV^-Krtx_iK5i@Fs=-wOJwU2;r^D<5e1hnLmR@ z{G%1g98v5LvNFZDp2rAwqk;>>U!!*AkhQEu@;DMqH98$Ga4Djb#t`|VrDdm>Od4k@ zT_A-Jk(rc7VqAt3@o9u%=8Bj`H6l}EI2S4kEuN5~r*R#wg~0d~i4*a$X?)lYxCo|f zwPnMzq`w$NmL*dLqnsdmd^|@&9{M@VuQ_er8D@<}n$&@f{1zbuX~tHRWYy~CH>xN< zCxrL$dW}27hmceh#frrk*AInfqSBc}9|P9E(xP*wLqqc`zF*Os>!&9@s=$GMhL)1? z*N0}ro)RF4XL+QO-1MGh>O-wvKV6M0pU35dIfPV^%8C{C;CYtKR2ZILsMEJ=GK6Ki z^*TVAV^+5l9e21Ohab1$sfK9ZgZ=N!1S*fl6$DD_)_8KnDf!r9%d9U^xsdhPVvBmG5Oi;t&r9`Jv?qiDmPKr-+FG;dU6z0PP^U6OKKOyh zkx4Al{Jkt{s&7@mw=--CJlQY&2H%_j_~l-&5|Oe)xFZdEDQ!qUaGq0lN&Mpw!}1xp zd|o5;ecJ?`Gipel0V7g#pZ&w3C?RuqkYSqwJN+pG0Za%TH zz>3WP{3N0e1~SuIx>TRd)vh0V$fCA)cC&TH1ByP#KD_ynRET8xqD?oHozOJDC(cE` zI7Xfen4*hAZ;I86Q&^wWmsMc|{V3(lpnjtAN01EC3Rc1^Jq-d&DyEA_ho52hO;=0f zGWd)ek<-*PqxpLtm$Rm9swDw|i%H*pSw@1Q9Ij#SCaHfu>V4Kmk$g}j>&X_)p|&fD zN_x)$Yc4xu+wy?WoQ=L;9o$5X^XMnwyktmA==S9!=0$J}KgZ1=Sm`9mU!s-yg{)mF z2U(%OM}%bs3J7XxDeH+^h3}5Tx@xu>0}j(hz60yyN+E)0*3qk}Wd9ZcxvinJ3p@iQ z_5|&fW&t_H>0Wbp0d2*@{fhc`-byx#62+}rHA``)sjo`9scFg9r2a(XAj!Vqq~28O z*zs8e%BiO!Mf|4b(F`JgECU3>pLn4YmUut1y09mx!@O0q@ovWVs2w&)VecqN_d zc6@%-sr~5*@% zN6z)C=dufWf5$WZWKq6|>KU*;_eWgC!e<4Q%iK4f4UIQ^{DCVj=VUP&6hUmlf-Ell z<3s}Z3HHMYFcj^!+85-q_@-=&?7!Vkjm>@`*pqa8+<~zt$MBl2bCWDYIvhDAZ3Dzm zI3C_(tfXh~^SLOiD1kWdsN!0H9PjJVuEN&{q)P%QF`;Ojl!JibDUd)TSqI7TikmQL zgr;@9&gzVG`4SA4u<~TyVZ;n|aVGLQA9Y6n<@gn1q#A+Bvyk!MLuT4uI%m>fNE7p& zhAr-w6XvwOCh6Fw>RbaHhT>(8P;Qcv5gQP9k0txdCP1OMzr=0F#;*rKq)&9D*(geM zk0a$il)qX5e<#~6od;jt5Sy3{f1 zV`o;e+(`MCPNT;4i$#_V*VEqxrLj0~2s;XXTn*X;@tL^LkFi=#$sX_W!kUIx&YQdDl%SvR^7GE)d71cMBlM2+bJ%?~!qKW!7FWWtTzFn2QM==caK2l<0u zxt3z=GLs@-J-}rFj;O}Afi%vBp(6;^9K!rA0jxqv+!r9|CkL8NviN02(8;!69kC*G zte40Yn30@+QA4!rr}n9wu^$XE(5y6zk$g7#nZ~A39O-K-t_)lx-=!KKT@+Y2{7{JX zw-c``8@Q7#kf_h^!Sx5f&J6;S>uyoOF6aJ2H}MP54XF6_Xkcb1a{#4W%C@;#m5^8% zdm{;q(mlu*6ncz*@LBsB^r;Zu2c$=immq3VE2soU46*KYwCF8`HmiLOxR4wRllhSw@QR}sSa5`NKtSG zu<&5E5blLwLXdOPY9QJ%?1kYVi6~iT%@4$@<2FZ*sg2)W$)F~+;gC^!WY^p-q<&1u zyC34<>7D>EzidPYe+Z4NpP?L`cnjp0BBi((8fsmZUv z3wZMw-MZ&BVzUCPXyL7a)!ZS!5D?8OWPLljibfW$MjYlG;FV;<;g4b72KL8X22X@m z2T8JiN2Zn5Yp1NF@eD*3*C~My>ghzL&L27<43x`XT;XUI0YzPRz!=n0`=OA0y6g&J z6n$F;zLy}Il5KpNBi<5F7_vP~&Y|fb03Tu6y#63|>!g#c_+jrQ7ov&yi{vCIP$N}n zMaFb&=tP=sjTUgxpXS;>N$!NuvR?jHJ&Ncp2-~x`h~5O$fY5{s7-511nmkkyq~f#++~4> zfXdwJAudyJ{eYXIMBFc*^9(ZG8TfmeLwEr>yr;U1ATA7^D93#c19ZqAwX$p08R zCBiEW95BU&e!Oh8LY5w~>{H;s9Z&YCu2sH@9Z#*~Bu%<|bkNjNyaL;z0&x`ymnX#~ zw~$m%tJc>NaOBZkuw*a7VUBg@hZIGm*5djZr#Ve9?+qx^91Lp&N<%xqC;DjK0>Bu& zQ|`J5I|kqnLLz6Ln5_4rWcX_~Sj-3b99T~tbBf3cB@EvnBUeQoNjM zH{hOIxZ0AU@7GGL{FMCL-__@99sMAuQVo~E1Lah*0oHqdj?32`T2X(+AM2itSD_?D zA+(ES)(#0wdbN~GZt(h?3TacI0d@4p)VT6 z1f2KYI0b%}o%d2wu%BB$mDPC~&7itj$XNCno={yF4Po`EU$0%a%w@C7mnV{f%>N0m zmZ@(^b)7yP({b!_i7*e6$v9~0K5ev*xbf}aB>inNe_%>40YzwcH}ZiDAAv;@TM8on>*@L zTxa~C8z=K*VK+&c&SmhM9gv3eBU#BtErnxYBpL-_BE!HU!-wRqV|4o6T#LP2&3+lq z$jqNYc3+t$(d*J8*~$gy`kM05;h1syF{kAPGB#Ncpq-4Y>`c~&z66ksC09PWIuLzg zXRJ;!r1kk+ud3WgGw$;O9&1h#P7xZDc9V??J-y8Qpm-YB94$x!2NGBFw8jb%XZ;<5 z(JA}|eyJfi3LO5+)&}H&8tzLLj}Xrnqh6NBCX2Xq+1lK=Ng2j zaMy0-h^W{`a%$L75D z!w;AtxUe|dX0MXy9S)cb3C&z+7~vOrStAkPgn$Y`gMws|keAT;bNalHm-!C#IWz{8 zw9jiWHsctFK!ALV`x_=gT@v!^?SGOYDV5(CWvi*EIhM%{ zIf(90hjW4`yqC()LUdEd4KaHHVKTtUBL*6Pp~8WtW4d*jq;y&`Qa!bNpOFCO2vP}W z4e?vSTwx)twHE1@goONQtbU*h2`s5)ZLkHNPAFI=k0yfUCmUPqR0g&X;y+!6unT`g zEC8NLKx8JHVX_55jaJ1sHPD9o?&$Eo0hX}`_>caPnb!59Qlf@ z?=Z5dgNjr#sonVVv&T9|I^{jXvWV$$$*4M z2OMJ&zM=pU4v)l@f+YMIG`)7S&NtP?w+Wz}6`-E|>Qx}jO?$RA%uq3Mc`)Cf#ALXh z8z&zNjd=6lA9s+BKil~Q4a25ysmswS<%eH(yT`?3K?k^<8jT(;DF0`*4irt6yxju^ zXoRzq6--3C_ABB|#k-K&hKkwq>fhyDXCHgI`}QEB~bb1*Ixi%J^E?^Pz1Xw_BF8%~v|pFUhI_n&kVGicg2 zRfQAse5jr=Kvl2(n8gcZh3xMRYWh*AzCv6Uof5G#DT$fCtNjpee!R6zvkfM6j<(-; z^sqgcC{l8|33<9&_SpcUP|@5@yFFzMhqU6#q<&l(?A9qnG?U4(kHO#%Gg0@i-hB9C z+k7K_2cvt;7a&*eb$@;GzQNgg)~=fnivqG7CGc3-zhu*JfO*2{chsNAkZ4`A_yLUrd^DK8AkOeiDNP-}o9qCFg>LhE@|H-#WT_Xee zFEUSYq?uKA>p8uw>;eRnm=E7B!nq?f0+J$cpb3HBT)xSbv!AMQNlfHc~Au)wsAhjoQ{ zBRwpvkLH)m*Cv2|3_$_bK#165D)K;A|KnbuK|B3>n~Jj@8;5!z?!;Uqh8g+2@$|Dx zFGb?EC9)(N91xh7p@So7pFqwk$ZzLkTiuEv;2Pt(+yCh_2anDv*gc%aqsrs755-)9vz~|Uy1sPE*MWDO8EV&g-7ppQTl%7o3QO_fgJ&H#W9Tde( z2oui)>mg8%O?}s04qyN^i0hH+M3}f+&sSd%xo?wTWNSni?A8OJLgJ%Q3F3?JBxSNf zA_QjtP!Vr7;hi+s)zBq>MAXJ zq`6Vc3vYXe$QtM@5lny#;rTWxLiSDL{b_$a^E`5X%G@QeqSzJWSoT;!zVlW;Bd}$3 zy>Id&m6Wzngf?n!+mMJSPAt$&NZoJGceu;kFLu*YbmTkng$kU%?{ShbPENRzqKQ8z z8M`O*tEJyP19IiNjpBO31VH+iSuq_*1eIAx^Mh^op&AIw?$`s^H`7i+tfa5~il*Es z$9{{JB2DfFZrt$-KK2hs#6G{yL)U5fv|W? z zWtdlK+M%1!<>hCIroyAseun82aXrc?>2GZ5OD3t-n{7x{MSb{827xt>J7dDeC)xjJ z%RvDuRMX361{}VDYUpTiQgT^a;7Wj+f4Rj{V;xadvHH*aW@m5`Fr7_9mVx@GKWvwq z=T2l1O}H%?$6qCR9M%<7D>mU&z9ZA)CcbpupHb|_pnX&4 z@Z3Gi-|e?3U}>nV21Mb0Q2Ybux)R)B2u-(Q7RBwrNw1v(Y6B(TH#A{A(?@tI+%Kjl zTu#csj;Y^daM~@azon#?Zr?{UC;k@gIGAOKQ>5Eaj#`-3&-`>Soli6B(SVNT{ulLI z5of^O#}G!o4MJEI{sU05^AJ%_*5K$ZyrKImtR7jm#3zutU1N~Va>^{A92+ZD^&Spu z567-lr$eb{yPMcYtGhVyJJl+cY*OZ0xyV9`r6<5JNLNSw+pD@W$W1gp1u~iu4Y&wXD-Qh# zB(L{e8}pHO>=c@U$CQq8&LFH%WpCR)OrgO&o@EtZDr-ECakfW__Ohc64Ve zP&*q!_Uw+%msaJYrwMZs$?pw^+)X>(A@%|+2h>kn|5|_YXiC^Uyn(pytCSh6H;Bx* zz@lIe3u_HT+$|uw*mHqAV^WD!eV**1t7F&o3We;1mN}$J z2gD#4o~X=9gae459uyt^fxprSC6l_4wuaE8D>kr-;HOIlxDgG-ki!gAb)oYHNsxMO8gMAMgI_?zGQZe z;y4r8SvlRx!tG!I-Szq)D+NcsA`bN|06y48*W<$a+9%ehx*50i7Bn+h6oh1;Eg>7i zIdQnQKQI)kS6n$)Ex|ld*VTM{RDO=o9f?knPXdd{_z!}F7$QX7Kt=pyEr==!>4A#a zes6>w6Pf5wkI!&QjadR5mk z<_cRC2gu|PUd8;&$-;g2WcDEiz>2vjvXNjT!&0gF5VL*V`$z(znT0`%is z0CGa3(_BDiCs!eZf7I?v`uZ0@AY5I8eUgy#KX4fE0fN(Y0txiCz-hl005Zt>7bW=^ z%d$Gk-xC1ghQ#n!cnknx-+~i8EA-mn+>V;A)|Y*r;>n_T-=}cc>QMIrNOL{awmC*Y z;6d49@*Bm$TvgmNG`yG60Y~7&wc39A2Fbh2gFLI*ilDJ$(yvfqnl_ADVD{iZg_%a` zLPM=9|8q$WrTlE>i_~jF2%O7|KGF^Ve;oH`n0~hd)`SdT`lQ|iWUmixSNCVb{c;?O z*I@vXyYO%p35rB@m=v?cdWKO&65}hWEcQydfaN+~_ zNb#r)jqi4+O5f0FRE`+$17NB?++t-pR8W$DPczV|dAIK^kM3C>P&5>&I=&IKI@kq^M6jamJZro; zrj(P$-Gmo0VHd@JTR07nZH@pmIx-D(E( zf_~SFN!1IsCmb87576*n#)q>0*LI84+ms>sxddqLZBb{=&4w>-XPdnMmFEox|8-`1 zI{PJc3>B0YK|NYfP$SpKmF_quRbjs&jSHYHA}`VV6gJEAZ3=PXN#ZZ>3*-p_RVE>e zjmvBlLPV3sZ2Z~t{-oQ!4G|{tsrZA8J9pC+O#t;LSQP1})0Ju<)bi8kjn3=` zKA}4iU+<@fD-Do=2*^3xwrcW&ySpdAxk~`@jLGXkkr|$+p@Oa76y^DxhIWpHT}d6< zOht`SVj}@R>kl+@OtBme4J#ohgsQfzmyxV$Pf&UF0|cK$#%sXwNQC-4xl(bMcGYnJ z7qA~#n=}l~*V!xDDLE2RQ{UNvMH=s&xY>oH+bC5HO~j}|^vOoAPu4DmF61zoNY9~5 z)C+422td>PyF?4pD=NM~cWfutw^YCiH+t9IieT`4DZd+&VR+R@>;M?WP)-Mbb0^t| z47*>r?pqwIC;pd!gCP0z_^?qqE+2?-#Jpq4P>L;cMF z1I9jAfUkFcM^A!&g37N(9IA?~Z^{*DVE+!aWdlHRU+8KU-cL{xKJ1e0(*_HQse2q$ zC1NK)okH&nlj$;Tg??0dbHwq2uRL&94BZ4HDCBcB|xa^ z51@pyc$6XHZ5Yl(ie_TBf^c~ffJGiN?2na#eE!G4Wm3bT6)WuXw$IsDrXtvj08cD@ zG@&C0z527nOow%g*>8>$J=<@)0K+hqkh4f0jSq}|{S%8NiC-b;j}?0bvAYOJlQ`CV z1jw*cEjLYOqon2lJ}0p6ArX6IRyc0T3eqP720Eh=eiznW)XC4u}J4L0O+gx-HEU#i|OwqKFxg2tNAq-friZKS?W81Vepw&+Fg8_JGGJ z8T9XdK?UJ?U?>?|nVp+PYU_$S8r1r13}+!;E> z=-p$~8-Q&|B#Yz}{)A3Qi$v`CG5a$>TGnv}Y*rmWL&mb}r5Z0#NCXtQDnLV}lpu1; zC3+A|0W9kWXq*F=#UvccOB|~(mN6gCj~HsUnq^Ki1vr%eS-D~NnpWl8w#TLq)SL7K zSiZL$z4KL;RFYwe@06a60Zd%gsOZOwY2A8)5F;P8DvNl}(_SiOyH!!H<5qZMz&IxW z#_T5Um{$<0#NHW>yu?I?{R3KjhBe9Nt$KA&QG`-8<&3uUyD&o}L+XG4M6U#~7dhaC z%g;=dYI3Mo7=^wd=H~@(3GB@NENBGi)9$ohZT^T*a`yAn^W90AvZhOoWWZt&HSz8i z<`@AAJrT;%I_BG6sQYQPP^$Zyi+S7yXUG@kki7ARu^X1Oy}n zi38Fpf}%7i9ZE^3DBUPsf`ml~B1lV%bSNSy(jturBHZ~SWVQizu6?U9 zkZ{*~e{aVGpo>yNj=@wVMc{UenJ|tN9}oOsSm#|?sKn(inck>DlcPn;Z!}-L;8+3( z&k4Y(Kf6V5*cQX|U3-zqXQ^!Bgg$hFU|3ngSfn%qo+uYi3G4d#0oG!SSsxjNi~kyfiK{p2{*1E>@Ofa`+Wjcn2IMl`Zjpx29_ zE5DEFvfT?mf6%FK>NjHyCX%_JOH(lTaygN^Q?AYQuAH~{jXv>@VgRM3& ztKS?@4;M*skzFc%OQk!Fpoq1EEn3197&6SfC(tAl{dvmi>e7BC6W>$0*_`ik1+jo4 zHcJb>ABW|}Tc%_8szl0Fz>SJfpblgFfPFdX|Jw_YT+AaD@;T;7rA2piOSlFYy-WLN z0dBN0q*vnsalX%@yFyl4)`(1Mybg(UrLIxC@$II=6M$cxDz^0^XUqO@GK5lnYd04i z3gFP53xHggu6%2h{IsCPo#ACCaWFO8<5K-PgHGKN-t{4;JATK%Mh0%B+0oU+gvu_u ztBy^8QWex8*A7<9DOxcoeU>i@3f`O(DH1Ge>fW3``~G5uWMIKA>21FU*-EQDYffs< zs#^qBir!0>x6&GRKrSt7o#VS5yAI^u@=T>e^a;*1aj9XCm~e&UFv~wuC`26HV9~f( zsr9gKNY9isDM)*2b4=J@e`%$ulkT(mJ*?E9+FlZH*O&k7cR>$8L>lwx+?yY-(oOD$ zm#?`#ymk=oar;UC)nWjh7oG9@W3TW!w%nX?B-uim)1)C}+s(v%_{wEy#9%ikISIMN zzp=3vE|4u4wCD+Z8=mL|_P2k`uCUU7n*(iFL%7lv()jFJ|9+N1rJ>P5LhOMI8Z&V( zb&<{=AgTOl_K8%|vLlpBN?_YU3D%OqzaSz=c})~U7W^Th@}vv)#M10Lh@YXeb=Fol7jbF>0j!rf7>0OrXE>A zU5G7xb?OIa&^Yod@r>(7NfrKP!-@#PpPpWlni^Z*Oh*qOUW|7}N(5S^F%v1r>{RXZ zXm5S)uhX0D<|Vwt?A6-arHWZajIJ+rKbDGw25)qsC>X>_>3Rb)D0@e-+p0My;kIZF zg;fDM^a!FHa0!wl*PA12!8eT8-+PvUMrng(XcVVB&K`ut>yMq^&#sjF-R%l;iD}D6 z3NpLTQWDTpD)77WuO$6A2|e(B+hhYLayc6@l*dKyUC@$=g@7>$RY1pq9-m!7GW7$}Ut+a6lM8yIj4Nfd}tdI^HArG_`~+)c?XE8AZ6#Pl*DZKPy-61|D~c8 zDQ&L#K~?mBaKt4XXL)Q#;vg3Pf8(wj8D{w>V!eS?Rbr`i0oM1Ly7Wa;?XMWqnf~w9 zo$XBa{;rUV76=&8LdhXz&OXDipYhmI{GCC+964=-ry-DN;)nK%YR-9tp_Rk7fuZ;Q zi!U$@EK#_$l)_ z$|~Bz2KG35xCfzctwZb-f29KHe6a zaFv1!O*MS|@sUxZWD<-p(l?0>!3P)_LN^|Z-0CLdYMwk%LwgRz!BjbGbG4u>Hn*VR11qZbEd*ReAK!gwTQTwpY!NFM(w?0Q=VWG9Mp zW~Wdg1}J*Jv?^4;b*ZsCqYqgvvEi zg$8ou4xQ0|VWhN^f=1%>Ps8Ym=9dG9El)lDNvI6XF~w4|!;+Gb;g_h#V_@mM-(Z*>$UCeYSJBe{@T@NCS2GZF5Wgj6bu_>(?S^NfXX_yc;i zi#4np%6Zq1pEa$oHF53Kr@p=>n1F94If|%wpK)nTBI7~6qpdGV;AOs)*GkQTkmQ`0 z(|D04x8bX2dQbFJArMLwNW3P1>kY(`{EO?zSp?EFQ1P6XhTfdp>3OFjejyVXfD!cU zUzWLi`S?*lvT#!(z>Kdqtng<#K>mI;Xb0vTl)F+pt$!7dL^o>yGNb#s@+SnMk2#MI z_lep$4CEo3tQLqGd~j+5e@RJ6$Q@a3Ya|kv%yMQYG+X*pb#MjkY|I_Kll1%0otV+* zhNGujUH_vBzGoMS!PVY*Cc@%#03~sa$vBaKFcCX)8eH3LJiA(<3Z@1AcyZiQBom<$ z3ZLel&7XYuj$iXBQHN+6Z0CPrUJw`!unZ2^WcW1-4o^Y{C%s)s_Z%v=m>_OSkccI( zm0LoqPYUpWt0j_4W^FGz7X%SvCLQ_5K98p;kHRyzVYbKu0*$~mD)|1#DVS^~DflaZ z!;W+lh@doatU-bg`6Zmkj@S-IK^7ut1#KL54t{b{T*esbObOz{=eAQ7)34FMVYA@n zv!qf_k<9_(2I@Ci+giHmzQ5FE_Wa|pDKK;7e)X2k1!QmNQ2e=)tABnm%i#dg!NAy) z6+%1{h#{H>N4Af*11;g_H)9D7Tm8IN&W@}c_E#IrY8mrC_xK{P)3WnUVe+juTrn<1 zgj_Mn-z#pS9|OqNB_R||3dgthad;2>TgEZx@n4!mn^OWiuWwjtQFX||!S<>M&*IL` zjX(Q>&?xiaz)hqe0c)#0S2cxOQ^5rMN4K24>i20tO9!}BM?D0E3CqL%;lM%2(2O@f zIy7Zc#Rvuyv2j2gqGC~-qsR$EaA>9@-K(rQGw{rPg2M$YO>FqHzra~SuY+4IlPAWf z6Ti2a`LV}o!u~(zcb4PcUynZ<({;RC|G9k^TpaubLS*2F28*NpZ;GgM8l$|A4$30i z19#*<*jBZ3d?nmNzB=5i@nwy_H-`feA{euzF2M;9w+?Awig97@pNg(tkwHz zcU?LnYMz?31+LD@AuTNiC+)>}dAR=MYWZytj|`-5pVaCq2sO~vj0NJEk3M}X0}bsz zA5KSsiYGY=>z7v0{J_ES%7E;4#Y#=Ie@(`XY9R7+^d5SP2S?Mt#%@QT7%&h)U0a@9Z6OR|`p;)*8ek&}i zXOQ3z*As_NAexkj?Tmw4_|;z1Km#QM{2+4@(N}P#xABicjqmeEI|S9peFEXP;0O0` z7qawkS7mvcp{kfKz;A~pup(f<0@3n+>+#9d$zA@>K}oULdH_dXg$1tiMIl!LM2f=) z5XZh-y1=#x5J@x3>%*%dQlv!9$h>2`>kcTbmgx&vaW=3e!jF!tSHb!!nD&2yI`={U zDlpllV{qXX1Z!e=W;XbzNGkYlvLYOVkcHmvDvm7i__J+9!W^9a=k^K=w3%!{|3`+d zK9`Jea}p=4K9K*e9M2M$appnz7kTxMgddk@zDdIi0^vrFc^>MQLJmiQV1Dp0Miz4g zVO5M6kmttWkXsT$9<>Mmxk-9YvdR#Cjsjg^6%|E~oGlQ|hd-SE&$m<(xAmRY%LJ`x zNJ9{Lut|%!k0XB$UK<>%NCt4E3g#?O*4Eqt4hpmlx{N{OVY^>;dL**CvyjH~r3zIC zJ)9sT*Gdt`ueSf(eKyU9J@McgfB`A(w-G|9hsNCu4iJiE770gNXaeO&-fPbxhsW6e z-JGO(8H`o`rT5RboF?&e%=b_#LWo3p+1t~$byO^B#^ZQKx^ zo{eN6ZdXIRggC_V;R@jhV+P_wMh(ClM@l22+`$HamTI;h#_gO1**Ok?9&&!*1XUOs z{-`3tmWexf?O*?7%_DLqP8v$eD`(XXspO!6`050)*YTG}g+BnN+B6J%z}g@uf3;w9 z9;bz~5H0*%yF0F)PK2z^)h!S`g}668&$TCUhxfrh3--af_c$VAamV3wA9Ff{@Hh7t znTnYL{S5B;k#4g};^$Be#{`w)6_~`GJyr>~UIX(P>>F5-hVY1n|8e}Y<#Kp0P%O%v z9*(-hfelJE-G7GjXQnieAm|m>L6VFyV{7VBKgfn5{Dv#6A^e8^$8SLx0LccTso>1i z+@5#^c?tE8YNSh6S8*(Y4Z42h(sg#^*e!oWvFsb&Q%#vYj_i~Eq4O>wU{;li&Mh24 zelq>{N~LxipTNG$5v#3^`8!<=+lFYkTLCWj$3Ij$x7`Cs1K^Gkt6X^21~*XyV3&VE zS|FoX(DXC#XH&Jr{L zUeR?1z3ZGf3dH`~->>S zO_?SqNqA>5FlE8)+c(G$u&4!mBmvedD=S<>5y-;}yQ^k>?~O1HW(?s;93I*x@399v z1){~j+6#j|hiB#RSl=1@{^Pv3_n0>DC8iN~bN|CjTtpE@g}+&33|P!qY`X8G&6ym5HN>>yK=f2O9cL%MT`gpDA91D~xme5pNt~~k)&FTlNmw07va10(EZ}h#C#f{txz}>rvX+_hQGypNr}c;{hSs=CRV&=1*&&59Mz4^BLe}U2Mb)`GVAvUZ74YKWJi(~ z=mco#pg{yK_rwB756~(5KO2!0ux$&@dtf%Wve`tv2-O48FfbLFAarWuBg)nTBfj%m z480MLr9fmZ_k;~@g%I2d;0fpD5X1XuQufG@+lHkh5iRG}`-ek6t5s8G$Bt%19rY1z zJ@t45(jfsaz-c0gye_PPFaf;$uvO{U-4Wz*>^0=gI7id`p3)mt%oDhfys`4lNaR76 zznubxc8R@Z;9`Fox#7qjX9+LAPSgb{Hn_k7{C&11o&L-jqyknsO3xKKi4Y}r0<6A9 za%)EJdRkio_81(dEhm0~6}RWWe*${yS33`^B!q03kdG7Zt#Bv&M=rhmE446LBaRMU z>ebYvhR_S+f`4iK&H~CdfFhui%Vm;6k_Hi@GCLA1sPC8Nfo6o!VUDr{6U$-c8xW zr4jWj6(w2IE(za7?_nBf2?{Xp`mdIBni|znkPE(}9atMpNz78*Y`w7j z(~$j=W>0pPQ$XFTz0{%f4B0{NKRe%k{!wX#sc?Hgl2m_>?f=m=OS{Do3wN-9_nlL< zEH$W&Pyh?0)L<(epf6TkF70-g^pb1N7uUO;$>-fIeX2i|;rH!KD0>Z}NI_gC{Q}gUhn3D6fV;Z(n${%IHxIW>Z75h}zwhLr;G`t*eMKv3cRo&aJwk ze1VSVN7rXYw>XEb*C$>jkO=VJAKNHgr>i~q zR%z=WrSWmBz31ueeZ76!)v=^ol0T06T=tV4*!CFQ)vXk$&vyRpPxG_t_V1}l>nAIr z%q$~eSL7!fWH(mIidKjZ7wnj10@P5At^_;BGN`Q!3 z^O48-UD*-B8~dX~vwSB?^fKbJy^bqV9_c=~whD7oayI4KZKZ zP)BwhLTU_wJv^!y1tgiObE996isTGJb~%CW1m;3VTbXNBV>ZqXQ2K8LdJ@#Nzj0<0ywW$|v>fn~lD<=Pj9#2i=S3>jii^!0tK!^U z<7K);{`hz#f1Aq^*D1^@qSb#1@fq==3=v`7P_0E1J+(^Jh6-(RXI>)lXSaDycdo*Z z?1YH#iPFY=wpRGUlp6#~Nc?l2W?C*@O`r{OyFlI$SP`wZyOL)S@vdT3n35Eog{^^P z^ac4ti01tpJia^KldMZV$C;f`VQ@w-L2K^o={i2O^Sg(wTo|Sg%-dp7=3NxWQH){^ zFeY|a{fT}FKVYL{?(1$gt6?&&LqyrwU^R=slmGY@6gw8&B*`5Xn`W7xJ?2wmX9%*T zPPY0yTUZqpO;O$aq80|aO?hR-@IlIT34nl*|6O>YX#ztE06Hq|^1(b5L6Yfwz)TMB z%w^+1S>w~coQ428XM3}6 zC8z*+uj1f+nrI?|rJ3@gl0hvyTQMhTe-ThBKQ(Bn7m_(%zfg4{r59F}RlS z5Rs=Tj3et>e|bhQ+jENa#~}237!_%;3HC~U=fm}%004FxX*w}!jW{7rS$t23)_LvA zUGha=z}#t3o;jJxi)xi-bFn1;4xttaF>I*l@GPKBO*$?n2Ol}n2<1TT7aQEg*pO8b zlqea>6~^d=|J(!YQC+6T#Py>bBrm1|b?TtBI)DIwYaj6Hb54$oLVwe!t<3H`o3^GL zo!X(GaLU$HVQbn5)0bA!{JNTkK7eFdy-cc+bO3-u?~~^=m@PG!n=)klYCKv`=-UMJ zNNs2sZ^Q6F_s`Ic61e*oURFE^P`VGNpZ7*S@qz681dRJC+Q_`!Oj^UddL0l*GWxo& zCSdYh9pI_tP2ck0zETFMY#ANlqx3qtI|Ib(Z_a3=b$d=RUfaP}t#CsS zMLGC6m8hYyTjW;CK>8W;hQ^RTc}md(@~Y3{`KEgSr7{PjAk`-?PCyr!=Pkg==87N_ z(DdQL71EBJ;PI>)cTs1JHrOs+DHcr$e+><3-q8%AYSSO@d~%f5$ju`=pxOD`XvLv) z(zyYErUr4T%?XER;RngwqrW04^+Fz@X954|p4Ih$m+RpX8vUy5$YN_4ZSvLS# zv)*(5+cCfB^9_w6d1-dS3F5cEV9D9;sSLv5Lnt(8Qz+gkbu$6Zig~ZGgG*iqH5jG9 z>^`I;$+WL;^qEaLX0u1;w}WFZz?XBrr(W=Ik~eGCS_Q1!^Xq7?W>Am~??=;pw zb{+9Pr9dvziK4!9W14as8GZPJNa=+0kI6=tenq;4^DlF)0K?Kh3ZM*?W_PwU3=t(< z(R_cNJ989)McO#pp-s+D$;?8HesuA~#d`&@FkVxcDfWzO$jzYe*B6#wUn~sb{z=Rx zs4X!=Q@2KJuT!Q)ofl@OU-#<7!r=2yuZRYt%%Cx5_^n6{>1w6b`?N8B>Hq-Geg8gv z#wRhfeUmhQBTA(~qyTb#VUqux0i94r&Vzv;Qr#jFF$>QgZ^sBukPllOAV(GemNRJ-drAg?c+4_N-R9B4cQVt9orUE|{8J3}0Xqi+3$(_;fomr?Q=fpG`vB_mIxa> zZ?SmF`4={DYD_khna(-t+)mp}SdLzflF9@mK47jY_IoNYW1uIft7S_%^@< zOFzf`(0&QwKbd@jEEt`i=h{!*NcetZorF_~JDx24?w)(P>&v{)ie|fo`D@;sL#~_s z%3MpTRm&QOIY+9dDlbt~6lQrpd9e)x0M&=yQG1|LrP#*p&9xBe#XH*c2*j0b5O#P| zUV(N?)tX0|XPhN}b?vAI307m+B9H8j9Gh91PvqHGeYU3q_SQmvd;TD6N&QF?a-5I5 z1OSiWA{Rr+hMheXDdfu=TkON*h#AdFt4RpA0H)+GIT2;1l_Dr-dNJy&s0^oB)fUXB zbKs{Q{F$l||NAhf*lbF%f0Q6=zM=SsMmh%`p{8d|6`JvE(ezDhNkPLpv*jGNDj$q>Nz@ifxCHFNOp(@^_kbeJibhzEqTGy6;kZobDVH!0K>m9l zXA~P6P2pq~w$jLD$nyJn-LE%)W%y}w6?AOOzos+w)hUGWm$$ZTP7Rplm$}aMguCBM z306{}O$oaIJnk-W?OV8eICR=sKjLQ-jMw_(n;+92*>~a2(!6T z_|I`=wz{qJQkpT7Sc&JGh=J{IvXka_*%UZTO73n8O>r!~TJ7R3@!a251E{^;0V+oJ ztkR$E9;ZDKwuysZfGy=p_<-g#X^Zi2VwH9XVoEl0376SEsuAU2u{)vBdV_6%*=H%& z)WnIW<9=gH|=#4mg@useR`ePWDbWWcWjebL@v1@QLc z4DP;qO>rq;+fJ_7pGkGOZrfx5#;O)&&$^w7@Zy7Dq7-Y6)NAAM&N=P%L$^YL0ljre zggRP74*R28ajv|Ib{X(T*(sR8wIQ$Erb+*KnIq}_CFP3PrIbK-X{ zukPO0)3NjtqL&IBdqh3`DH(ubiEp z9yXh^@;)kV~t&O9;VN-XJ#f5GMW!-U6m_5skKcFKB&ox&5&imng0!H}Z=4Ho%{jSs6n z2SWz-59j2Wd+@;Fs`CZp(iYAL)%5au{ewBYBQxQ~}rr5Xb3LpAc2p z5hMVF>HP5<+Ft;oD<%Y>yI12d0Fm>leu>t}Yk)v3zVh^;Ls={Bm6)cbI5Q5qz}EuV}&2Iu||r_%?^sQ(+Ikr z$pb+3>frirSvQA}hPyeARm9)#q=oqfTg5lY7N@ee#wJgOj!4g)46v^xe66iN;#e@RSlRS(Sk@}_i`*mWuc&R~RY>h>sl zHkfjes4h^ahXjVHB;69OvmKyMSzwk!3UiR}w`lR30K#>7FFHEkugoU+RIiwoNzyG2 z@aACX_E?b)3|!m_-=#U;%_%7Rdr27a+h<@ibR?IVA%?&-uX3q9>~dwJvCu zu(Ybob;Pr6k?FWs6WAD(B7aL6J>3N+g#0uEV=G5m}rNh930 z+F*g^3=(1v_NKzjK|Z8qINW$1=`dlAMml{hla256&)h&Gok>Z?9-l=CiSx2kgxYJg zAxyB!mhsC6(3IryrMe%lKHUUwYSLU#6nBIT9wG$5TzPXRbTyfPa0@+`9U*Hk367BVY==-A1)la?ZE zDRjne@glYZ?D729FL9dQh&Yw^0A$jGm0Sx629RzW=A8@?+XQ6iHU1iz=>$!C(}~(H z5#t1;AL9SFUfh9ucIdYkklqND5c69-su>&XzUZN+R?XIK8hrH-dDjIUBKEe+i`g&Q zikvtuKK`YcvP~X6;`K9$f}5tqgkySZWm1;1FeG2xbM;G$sMb(Yb({y4lrN*3d(D|E$4(&u$x==q(aq>; zmt|+j@^^FajUxAOSu7}ApRia*wT^{2m`XsRoinqY*+-+n*RSEv&p|Tt=QA*GbyI+F z$0Htk{4I!gARs;i6Fdg&rH*@vdh-Am3l(-8j|*hli6Xgnx9=3Bu<)^img^6~P-?CX zn0QlF>4>5pv3SllB6VkRNY$Xq>G|XZ(y#Zo4J|f~P3Z4H8%!rMr{tc^B{Zi)yVEA} zZGo{@j!(sTzFtxOa3Xg5mP;Zz$zcIFPOVt3B0I&+e;PKfspY*GqG8vdQ{5!XzgK$O z=I!v9`kxrpr51k}qF?c$rN|0Ef0Z_drDj2BRWl2HDJ5PO|8rdtlUGAk+U%1{d6^l5 zPX;T3{>IgQgL3}Xq`T7(IwXGVsLRORT79&1V@`)Ab+z^B;V-*%-m7Vt1A$$1KQ)Qw z-W0i#F{G0G{xtE%Ee?NkNzbXIgSKmX%CU>6roU*84^9I$Q-GD_j-gnwP9$hy_rFgi z;G^8AebwR*xBs1T_*g@@>0YN*Cg0PWVBfguWoIE%!;KWT3nDL{t>-C#xioi}kqRRA z3#he+FttSUV1V5Jl?${_UoNvyY>J#n7qwSXA!G+$B4E778IKM8=U$KtCVxn6>tVa2 zndK1*n$=Wi~jBqYkx2LC#`?=$kd0$%~bRNV!v^?YZYt@@@0;K zcfVsk5Hvx1AlrFIIdx>J5*TJ&b(iFC`M1eE{^8?`;Dt#hrQAnPA@Egv*Ez1d>pT0r z+NGz=w^HQ=1Iwo^a1_LJ9x9{yS>Z($V~G=SSlfS2u*q0Y+1@6Cl+8Y&n}9t9#*wo9 zYGE(?7hF>TlD%~iPnUS92`_sUUM@uyMhh;8mj9)S3iH* zyT0S>@n&#hWeGaGyO*-|?q$gCufwJ65mj1_L;E8G?XTK^}rUFbV%7gCPzSXa9X9T8^f#DU)HP_lKnrXbkawyLq z#S);KDFoNX2_C~&?ZDNGyD=3Lx|tUX>VsL_X`en=(H%FDGV%izNkQV7C!!fRc3Oh zG4@?+bvgtm3Fw2ArTl5%)u1;ed!3sLgA0&b!|X?|`u+SE3K=#1gqsTkk2fJLIMA1S zapqPi@&Ss9CL+e9{I45fu^ZnW8ak;W-7>@Arm!D3nNB1SVa@(?|LR?=0b0X1cf_pV z1i@nTeF7P&M_*I!oN+*je|FPd94sV<&VFYnSC}}gQv+ko=Nk_Gl#HxO?@TAUK`Zqj z`1%)-Y%ixXbj>}6bmt{#Y3p=8B?-yPc~JKG(A#ylerJ}^;Pq`5=W0X8$_LM^JK|W} zpkZ-v%)UhP2ej^4I)DZ(kAg(#YsaQUU#dW*>6epzb`&7TYv#JvK6?*L7nk0~>`|N) zcQW#4hwk;!_mOWFgku&v3a`NFCI7uC&zXT?s zXKZI#wt^X*z-cyh)Hz=JMLikXo==G@*tyI5$r;^(zL*QkDqzPF2M3?jHBUo|%2=U! zvva|*(o~w+xrI_BVBfa}{EF#$wiQ#zx1Xw;?pHQvFIJ4C9cl;&cyN!||ChPMWl5K5 z6>qRy^N>7aGrwsI-vGHYCKwW;x!z0I0>RVJPvZVFmWP59V2FGG)ZoyMI}On zzMgtkiG^&wa@vJjne%YC!nfqI$1dseB#g@bT>6-(CB3Crbr=%QvY(;FeiPCbHNV}oIkhrW zpsBV1+FBp}GD$-mx!TJ9?EOtHFM;*DecaVO0%jS|_e8LlEW>8HU{eg6*%ct8g%!zvPwCk6~pkK_8|ZXm-JZHfUZC`q`T3r`&@fp2r2dSmHCOM&0Y+CGY|9&4)*3` z-5|p^2=?{ttE?!`<7WM+f} zT0Nw03{{qh->dhZO|_o>!~gNry1=j1U6p!EO4I8-N@c#(M;mNMjXfU2XD>oq6eZV3 zF8W@PVASKchfcA?wsmBV#`fUer^$UK&2+KH&t@p4nU@8M>_sg@ z=BSN}gcNH{Cue$+e&nbG8*J%%Xl^woPKc1&v`93C)Ah%h_LxO!^y=5ILGc@f4kuxr zhB|V(nR^#th*z%D=NcVwwu)fd*w{SMJUwbg2H`}SV*M7yr20k(wbt#y1}GJ+UsZ0{ zTL&+*@J4%#%kw!Xzi8Zf(|`@C|r3eAV!|g-|`qg@YFo+R@*| z`0oZ3Ain<)Q(Juoq5vws^*KlpC$JDitK1j1BP28=N%!2e< z@}rK{rXTYzbeZ46u!~c&)zT6If*x9D!J-?+AwvY(&fbqJQ84-uYDE&RAmlG<}Ym?}l*a^jURf5yZ&I8f)OUTEUXuUgFLZKKS zO%vYa3(~m&qc0wVaeSC2=E&fer(^K~5%14_Hc}+)S_n~(TTPu+z86mg)XG19LtqK2 zakAo&|-TVgk5pwi73>U|iB(wIQdgAEgDYVAvPuiTqx zIWWGS(=Q8|+H-S=UMj~jZ-O});D56u56%6BkO@!gfl^=1YV%PraJw%()=`{=ZK(0fzUK_s60@t4AQXc+(JLD)z zluB^@-qgkCJtI4t=I%eumHkPg7{&QzqZQSSQO&<3EJXS!2d9^1J|}|4R4woUZTR^u zj1+Q#=;AvG8~3&}M7cum;XI24)nLvtpyD)1nHTGl{W_(ClC7`@xMjJ3a&WT;s>|QQ z`qitoqddE$F8ruSOfP_(iBL3;HB2q+L58|PVPI7AF4spyrQSKnamU1o0z<`}592qk zNoujUd`XqiGoA+wTYkjk>-8L>M9oRgbcd-7*B*%;7}s*s53&#LNDupM`r3tK1yZZO z11vWC>1F+E-&(D~DEuU!kKE-=bBp>mM?>Yl9Z1mHC3av0S3_Yb=aB;$sCv-3B$qJBuAmOg}5VQeuu;Csa( zqYlI$LP%orVs843OIF03NR8v&<6=25PGynI+_VkUh|ma&4H?PJ47}otY7d{i!UA3) zPoR+Z`KTs6men6&;PSBFa&Gx9{_)y+fpE&Gk?ZmBculXzlBQqY+VetGXOsrBjJgk? zjx>;cuxDKZ86DLqyC*yy_JZ(v4msb-3zKT$qj$WkV0uEr$?IFo%^Oh&9V%}0@llk9 z2Rd>8G`*!c@&i;cMdPNvmgJLXsd9xK@FB7rb5Qm7@O)zAXs2d$lDJyE?q|VAC=4+< zv&40=o^EZGU8a%H4)B=oYogT8O3trdDt|YA#>FJP8an1BLCg57rnN@Wy&2r#AYNrn z1m7xkiB{wYT1XvqK$mFB3}tF}WpZ-R=mI4u0`DdJVu_+AnkudYy3Ng{kAsvAd*7lM zM7tIs@yF(*JZ+M}=5(5UR2QZvM8>@CG7|d^&XemaDD%!$j+(f({5i)|FBmpz6k&L> z`u*(ILnE@e8!vgByx6~AwilGhJ^3|>ZYylzPUtxIaw%l-b+bXV2ryh3vv z)nxfRNU+B(-Im`g@HIGZ`0(F!ozRZbvhV305bL0;CSEiDPmxBiMM`YrwHT zmF+&ReT`?>ZQ;$5_7N*>siT=+)Sh+ z_BwQv`hwT`O#_%LHmx~T*S;$C32x@&^63gPbh#2cqA#Rr}d&mnaqtvDxG4IJxeIyrlDRly|m;aeLt`qEw z#aduxhUr-pEm2ZiXZcYq&TMz=KG#(4(RXweJF~|kZ(b6upocc>|9p|hbYgb=;ETk% zuzDUxVZJ~lxH{12NB#&a5MVZ35B#*7$j zCexoVZY<^u-eyAJnRtY1OAp!6VpJRo9ic(Mpff$8yo901LZn_RKAnc9zPf{wj&Ogq zkq8BE@<&4+amM3hJ4Bz@M!Y-xa&l01(5u~Xo;HdLbBP`0I@=xJU~z*%cv2uCfRNbG z5&)h(`2f(-)4DW6av-Dwk?qs`W^)PxLO41%-qneEbdLKFdR@(IqSgzh3yXZ4-B|G{ zc{OI&wi6i_A28)I^6YwB?)pmJhPIi-A$X9GqJxw02SYdAWTsN6y9salS*52~j1|(0 zS^yz5{b+N;`?@ZkrzhtGb@jy=HlsjQ-jtt6UB78AeroptS7f-SFdR&ii%~>Tb7>`QI~l!lpdWs!AN$KcS&E~Fh{O;o2#p&y zoRg;qoliZ1K}k2ezZ*2YzEjUlh!9bO^gl*um@Jg*UJUB(F{m-Ii~cZc ztBJSJh#d_%ktzo?&*8*h)^C|}S^VeM$VEkoQso*sQ4dBen$46xy?b!}slm|kr7~Jw z$|l_PwrJzUWlvM_I0ok9)l7QK*?xL>)^L_L?o`FA0d6a=!Bp9jpqc3G8w_}8l+o<}mlF%PJe(|gf+S$R_lQ9To17-h_ zV_);F4h5jkO^$X{aSuEvowgB_X1dcEgft?JEtlt!_jt^VS*?4co7`z2eH_iIBSv zMp9JVGFU{Tap0@cA zxIX)t%Y)`u!}`W)rOHOGL-GRbfA1(#;;8B=IqL)h#=C;LlQkGKK6kx{>RySg$brr?9M?c;U?gaSb>x(54cgWj4?mS0!ZO37IN(V0!m=9TWc!Dk~(B8cC5BPr@S zYvXAwh%(s_JFQDf7q%#l1TCc6Xn(%0_sWe82dz4FS2M{gu~Yq5k#M-g|9mFd2l|HT z8}@~BXI@}tsS1R&r6XPiW2MlwRztrt>~#w&d;_G;lA7??)5W^IfB>;cn1zERAB&Uk zf=(l!BWsVkP!zu=5BBxXshxZmVRIW%(gcK-=ZD#>Ro_K5I}ptava3e_XV;PphKfEI zp#14JSM{jtw+ugRksLz{DPrj_ZFNLP4u;833Jw3UiNfF#J{z-yvi+?AJ8U_@*e7V7 zXcKu~-5<1iWGGAyOXKXz@j!1=`*OX{ho3)U+`T`zMdSVp7Zg>iZ}(hJr3gjp!xd{hY6ayC6yr za}H%P*jZ(^@ySf+&-|BbRR7Bhp~Nf#yVRf6*{Sjkb8y~09UqMCCs|Flo8U>+^sVL} zqmO^?sR_w4gc}@kvG(g)*s6UhBH@*~h@YV9^w=)h{w%P3_wcp3kErK)W=8z|iOZ~0 z8}I5HO;a0c=YDyDb|a!j8J=LI0wKJHCD^r=<1(NPMzSFq?_OgvPEiFm&Xx}L-2{!M zLV`VDZ3>gW%!#Oqwsv8gNhVMgU4FXri!7!0mi^J>sC%3Qe|X)tt%W@Q%M+!t2P@4y zBtVN*8He77Z4Idq?CP^&Z^q(651bR?CBj|WhsOM$mk^iQldXBiyDXm4X~M6PhbcY0 zU=u=$!dwa^*eoHVT2e{H8_A?^*ewi};#vk8w3#9YY{Rbi(}^muiH6zT`E?%G-o8+w z)I~q|a2rP6k!KT2G2yQmDzqU(iQ?}+2Bma7Mx$C8?=TQ_NZr8>!fG@?2 z>XY@^9eAjM%Yl!f(9YAzK$8Z)(1!?3lz=9TveP|8h|g0co30u#K}NT8GcPjY;}H^v zbHAB%;8zsOkAf00uKzg1I);cL$sQ$`QoSrA!VCtK2)4`cg~gulpOU!7DxZ6ggCOXv z!zkNr3T`RFdDE}jLEVISeemb*xB0FH$E0e4{+ua_z((4_63R0`m^1LRGvu{u*?7us zUX(i;p+$f)+!Q8QyNh)u#U4a_+7?rMffV64u2;;xdisfm%c7GW=+Yy3?EPV&OBB#W zD2bb!6;Bs%Mgfe{8W_k9qz(W7MA{Oc)d08QNr;zq?vQmIQ6@k!#?V~`vV6D*T&rz5 zMiq|ZTEjq0%pStERV#fiz_nw*wRYb^$3G$l4)&Wm#d>-#L0hb*DCIdE`Q;>~D7a7l zoy2q4pa1f5z0714+`VWs!Hsbrn+rgav%o8reHSOr|I2PUqeFxu)qyJG^~%$jOKu3O zWD|ajoJ5jLvz6pgc~A;Ssw3%t)Qpg};3*|HtV0i^rKhOX$i$O&h)wR$5p7uHmoAv` zoKkqpy&v9BQKO(YRwA!gD0A&4#9ookd})R$mG~|wLI)Z{2;S6j=}}JDzum@i6VV(( zB@m!#xlr&99pr+>Ha1qfv>H-8do;i=jo?&S&G&wI&(IS@Xta*}nHh`b2tzen57ub@+|6T|NIH!>(n6()-e`46X`foBG%8o!{75Y=0N)I`Q3aama`Jc+gotI)#HHyId*WZs#NCs-h*Q+a-ED zH(p$nJgg$`F*qQLQ*V;DRhMoA{mCs8OtH|F%v-D*YsdVgZOfCY+jg(4t=ZMv$9VD1 zlvpu;kbSB-Txg>Iv?MhPsQwL&nd^% z*8FtYt86hm+hAHd&fIY4cPk^iKwvYR$t+p?Rt4u7(n zOI_Z(ynE@a!06>O4W=eNg(4cslPh~aXJ_VScAgG@-`+2JII2w^JUv{Q{K-_kc6j}F z>$@Lo_En~}a_cT>>-#z-Plw-mdFd`BHyG@XA?t7C2f*E)_7zRP^W$}zMdF9N-^Wcx zxNABn6IhskF!m!Y?|O-0VE4II_vbAO0RBBrPx4HVRCUX_fg zxSqc+1*7c_a zWP6OL)(Z*){);U?4|4A@d!Su+^b^uXQr5ydTIa=>V`k%nW+ojLW%v*7=l3>PC_HnL z;nzNX_vw@(kH^u=C2B@lvNm-!d*+MB(@k5St`{p0`#0PXo>OC-OSZ^6!UMwsP5ECM z)IB->e-ZcAVOe!iydd%+B8ZfL($XOzDTstfmq>SsNJ~m10@5I$grw3aq6iX_(%mT` z-3`((>ww=kcmBBdJ~PiVGylT-p7)%+*WP=bwSJ*{>{+H#_IC5I&thsHk9*9ny|p9H ztFKYlMaa|YTlMy5(Oow$m8sR;ll8lmgwGSkS+>W+W4e7dw^TN^mKZ8ea{rvHFX>y$ z7P_h)HveMXI&c@tab}s>{YZRJ^wO#DM0Kr(kY7zuR`z_P^c*|F9Y*6w^8y&(n~vG6 zAED_zr5<9%gP?JB){fCPMgPfPk1i0McGEEBz0Msh$(M^HLp^ud}g6M@8ir& zhI5ab{^87w!y;)q8$|wC+1SAzzfIrb&Xg)^-uONqEqf9BqH~F$u%73Jo!imy3bD=W zvIGfY_T;?2#hJ3Qh0O*+8!glyg57&3{wE7V8IGRz7AamWhiUITT{JhgXm_Y`x5N@j zr5oa-PvVf>@3DcI<*hHgdgSlWWqU89e#if!%3vAV(!Ygk>n%cpwwXqNtN_n@@lIpR ziNWp$96fhTlo&9k{Y8?{c(15xPhr-em_aUgbU7`Z;NPDAaRl1kcSOYoxnj(420^tG zxh4zN+?8Pz)bF@bvJ6O<6G_|{1=v}iDHV{LK>kH5R;8Bsh3E6*cFbN>)HllX0~~K=Rb`XP+e}~ zzE$uI9XRWE3WCbuKL!6i8HW-t481-4$AGBb#UFH<~o#zehHjVrDz4z(?*UMr6hAkEU4eTj;^XFw~Z|>J3v1U{$cu12;;H8$o0g4 z{JyivOtn;r&42lF%hN6IFEkBg1+qnL*V-)R9_W3GZSk)qXmQj#{=?$6ankN3!i4TG2UrfZ0I)+e3xWHp;;nSHXk!2W2qb4;wYiSG3l6k{1q zenC~=3^4lvh4o&ihvN>lc#@p}P8Ww>)^{acpgFL3fFL=Po$s1$0jR%$+j5Y@e5Cj` z^sbyD>S`UsNO*#vgCgs7(33n;nkgSI0f?=Nqx~)rQ!Y6)ZH*O$_A*;;elxQ})rCt4 z>|_D3n(1)8P~M@(1bm52Q5#bYJ(Q<=l3G@UeNUkvM=wsLNQml!Kgj(44cS!^b*1e* zw6Km~Xd-lj9TrQ*oxR^afH9RI44+$g5#$Jc?DtT7A>q+|tHiVg4KD?Y+(NbVn!#jF z?-T(%N3Kx!_o+hy$wG49_)c^In1iL#^toK%?$K`NT^w~e{e``f)8(66n;Yz2at!T3 zp=@`l#^%HBNGCwUr(#)Zn_b^|;-Y2@N9=n{{ZfbQE$Ds;FeJO@_kcvQWQ^CuPrr7= zdrW3@qW_28(3o|FHqd!^( zfLvfJEf7VFfG%&zUYCnnZX|di@-PwOp-M(1TIff{1L6XE=1ppI7t_yIdY)4LCF~ z@E@6cJF)?RZNng0)|CiJ3MR)u`3&o+~%o>H&lbcYU=uM5 zzcS5wjNvpweYeF!IYUmV>5&&1m|ILYQR?)42M53535TPnr5C#{_vH7q64QSEFs~o$ z`S8_(_azF9H@V$zuem<%PC$p1%v1kc2Af{l;AqEi)$rDs+Zw;V`-WeQRukz`s75<& zkG;?+eKamV0^F)uBVNm5?pR+xbca6T+qG2Ssx}VHm@I$)WxZK^Ah&L=<)YP(*WZy> zaUio;vFnVhw`eaW*U-GD`Qt6ohVj0q%|00e4%xaX_O~r-BbUfq{!__ZreRyapW)Z+ z=CslF5n>NIKRd5r@-8}vqD3#Mphsl>sIk`bxxs7mXhHK+t$bv;Y{3C4%(*^a9NqwgO-2r4<^o^2?wPOBarACqSf8 z`1Sp?2O@+&Qui~9m*VZTv49U#zfN!fWiiq+UW9C%udh_OIS{@6m>Na3jGGR{!y z-Z$ox;2$%zzP~?EdD1r#hvV7)N@-x+D-oWIUw&fYQpLt z{qsfd8$}K)2K!D)^~MT-%=2%3l^D$~NN$QUUBmcNX;_6vX$382s1gS1rTYS5!ALpNukU8xnv z#w%)m*rI;|@DU)fn#UY!G#PAWQ4Q!?7xzl(13hAd^G)?NVjlim4-jw?%Nxr^XL5!$ zV&dgSPEdRn= zp}+ml%M6X1gUeftmu&t)zw-JpmWHwtcG(2sqX`_lWIjC#N7anzwoQOG(8vqKcmbrO zCEuBxrSgeS4zF!>#K?2NgXQR23-zJziZm-0uPiE>a`ySZlc^a$kS|IRFx2wY+J9NE z8vLp+l8=?GZ&7tFI=TAFCdIVtn~3SHH=#p7OI%(5H*-xQVS+c3ul$^!Uu7^mB%YKVa4+gO=7|YWou2o zNPLYj_LvYw##FkRgc?wu~J4jY-XXK!oQ{E4F=d7UsM2ghIC* z&I=hK#j-<}hYvh%m|~X!Yhc26dd0JiGvAO)dNOP0jQQ)w zMi$E2-1Xj!$&Jr^GfWN&z)3vJer#tRw7XGcT$pVZ1h^UP)kz$5ULQg1ntGMon(2s( z?JcOEzRSSHUuQP=*6-Nr@f2178lhzXsMx_WlZb7FUGjiIj*@)d?K%6z79;!SClQ%x zwW=$CnfXN1Vl>s!urKD@#zZ-1f_-Zc2qDjwTAB&!l7F>&=|!kgK3&vcosFEr&LPk= zBq$&koq9yyrFM+fESs)8M_j4cuVEY-Sqb5(K#v7e?b|H1>E|^jzcRNoZN3DpdBFh} zUO&fId=$CUyz3>O54HenYA6lBGgC+ z+EV>L0Ty91c%+SsRxq`0W4)KhCMd>L*p>2aban@J|Kq|d!CPv>c~3$HR7*QGdO zI%q1(n0$tSzViH!1V!V%mv|>qtVpEhV`2>25-Y#y^Cmr`Y5#Vkmdx5PU3*kt=)mrm zE(7E0aZ4BDjY%d#BAm5ha+Ag~3r~&OJ?u7<^$av`$(l0K@Mc?qB(2%p5ra)3J^d!L zLGF!fJi_WU-tHqm#LYO$H`b*t47h<=Q*qzd7aKw+$~~)=XhRS4BUGt7QZdW!LAJ=b z0>m8J%yj;7*bZi+)td}*(ovWv;Iw0GaY5`j7G{Me@?*n?dmBH&Y+=1~F8KMw>if_W zPT_BJq4L}@4Lv%ywvTi|b=IHO0+!;|?uNLG=7n=#s+ z!do%gyc!AWZN<+@A)s0&3dVjIX59$iEOmMCx@R+{(@y#0@CWA&sv3Y{D{p3U=XB3W zq#v8h-wStecPOicKF=Z9!ksYMSICA(Z3jnZpK8S>!&+ z?LHAUEO#X%D7fX(5@Si3oljFLdcv#Ivefqp(UO`j!umVS@gyEk#9E88jC?wl9_$BS zOujUNlaMqJY2wI2tVwuRSi+Evf0aWZLRF#E$uF=)?YPG_`@7-|dY&oeB%xNfho-b8 zf}gi;LPIw&Vkyp!on9O=vaHW z7k)8T-z_sL-q z_hY(l?bha4TdjeeCq??>S9%+4e6a~(195f&Hg3+sMLxS;z~4UV)}?Gt#T87?W0RXy zx}cX+l@TTRy(KgUgd$HJs7c?JOmoyMK6ur+$4h6a!@HQwo`QMF=XyrUnrLFgw7KJ< z4~g7Px|+POy^A}Jf#;e!SM1S6<+eA^Tt_LF)c5PZ-El9%l>>_p)$lnT^ z_QcJ|-w@Di_}Ss|M*et1;JV3EXG30_=GTGxA8l(U&G0-^w^&)7m0|pzzO9}zGjZ(W zVc(vWH|21tEKPaUGX1fW#mW<1Igl&7HCsMTw8B-(ikV{cTFG-Q4LU|Y`=rwO&>d$ONhQ{XHF(}On5~QA6eJ5rX(SI`aa&8xW^;Y?O2Tf0var6BT*G4mloVNP<%9zCJ$J_VKAFo*j^b*d=;A#Zy5fPU~ zE9rjv89do%v@V0LEdR^SGjIKEQAdZtW+QnpP9D}hpe~FkX;H{eA7*FE*BQe=4HhRJ zH54tbTbr}uQf`f52K2;|$Q51W~dg2SWh1=K48AWE8hK*STB23f@YfXGk;{M3v)?C}Zciq4GG3(?mTRcy{ zzT+>-v4+YQ8N7vR38vZ_4Ewza^Be&~F7;d=zIeH;ej< zGg6VQQ60;w+K@8omt;|GU5se+B`JS2`w;r#=3n{ZaZc)UTCe1JP4MwxT~-X;3Wyng z^KhMg^$W_nYD>ar=$CG0R5I(ruJ=I;dbgi@avacpxDxcdj>$Fu*?7cb9Xn{93!?UWx zr++7lw@!A!aG%DaJ{n7B6l!zztYp$+;We z`#cT)QZ%+K7j2VR$JtIQZ#!)5TlKcoQuG|)?bf1(6jK}0OW!hJFl?04L8GPgivWL@ z(RaQ#Z+@kZ@O}6?minfxC}yDJJ=)e!@%3w$=kT7q_Fa$QDE5^8hn#>~#)vz-wm?2* z+eM0Y>^`l;4LW7RT95eFW4eqz>5<`DKJu$-^adXl21(RslH}&3wAMtwqDR(BCH0zD zHu_1^-tLq8w3ZPcOPvYOrwEN7R$9KcLj7VVa`+al+3zSw)F1s0F(&$;u(3Wf ztJAh9*YnYA;)o_~O>qobxQ1`xAvGcvT5DvBzeFm1BqP?jEOiO>E8cm<^gF?Yi#{S3 zE>BA{q*dQqOYNwPm;L3$ar#F^Pp@;p1(ldP<4#YV8An}S+v`0-${}}KRjxp@s5@e! zLsYo2FLDJ8Ep)u^5?zM+Y!3c;*|pzbb?BId#7@5<$6F$i3aYOS5h^Z|rZrNZ>S({N z`qjD;TNN4|;{*oVOX-+N8F`oRhFZ@7u2y6c^%}mgy7VMNmHE5RU2nrZs0?JzDzIIjfA$x53n)Hh2Yf#jYvNfCc4#Z? zwu3B=&b0AVQ(&wJ)f!qi{$O#*5Q+N82vj1k1&U}17@o0Ga@%FFbhI`;=)v!-qgnP{ zhQjlB4RY;=fg(u`_W>E$5@Bx^?^N8F`RJ0dFqB1I(Kczn7&5}euA@s~0VGlJ-=77b z&T2UAU{^uaMGNvr0CRfmeLhy2cT79caliQe{aJ)#A>QR;<{kWD1ps^qF7ErHZ>}w! zy;=$pE+e{kfy2+Uey6yHWRDebfqYp}H)@P?N)|5ky#Ee=J$Yydly@C5{w@ zF1&xI6R9tI|L>~kXN&|T2P>`LAD;eLF}Vc*@CvYzi{wxT&XN{D`piOk`Wc6k>w_n1 z{9BwX;T6S9C0u}j_B{GuW=dG9ik=#Y1-}&gFOKw_51*DE-v=l~{&=o}5K#9QM}HLY z)_=G9)eXf}*0u~@(bl_w*Z=srGziqP#6bQew1oFsGOu?58t-bcRVa(Kwe%aDW7(IS zix~I+otzuAjd!}!F0zGbSh6TzUIwVX1n~P_U$KEpa2wTPxB_@J8*m*Ct0i-ffDop9 z*#iLYy=l?}T8@3w16MQgpLw$so;5LEDH9%;im?yliJeEt5~XK)F0cHFkAULu(C|M{j@#ywhsySQ<0Ck-`~Ys1KESyepb%B|I-aX z48{_%>&bmH?TAiI5=2F;GbbgJCiS3HG2)JtssY#-019rRSVXs8FF2cfLZv1X+=ELN z2A6rwdjr6vmu=TpcLd$@9;i1M{nKo+u8&yrRW{l2AG&odl5`O& zQ}90zk%a`hu71qp;AjRJE{1=bBfz5JYbc~P)%vYgfYQN?w@xq!WzyFt$7^27yQ`x! z+&kbjN*!ukWIZ}EiDX)a0DNgr+qbU=tfv0WRx~K!irEFL=57&bFGa8o6yep1Ww!T-a(9c{ZN#M zdz`4}+vOQ9=3;q4m3WuauNHVFV4nmd5>=(M?<}%_Hgc`rV6y)7t zevAGll+nhLLwN7^KL<6GCnEk7u36kK5Ssc(!#!UM~e``-~KQc*xkZ8th&ushi2UI7G1%k$_PzcRoifzSZo>x3eaeJ;Q5JpA;|j z&ewF%ieK=JV87NAsQ{{Yd)1@ul{1%iT_P0`_rAxDSuAK5ZmX^!hPDY`ti@TchOupodrJ` zl*WHMQk@(OR6g>*Nmq=_0}fwT7V(_gqGY z?`4w(PX~yf@ll57V@@$i%DGHOW$qxv2L7N0-Hv@g=XbxpDaIkb8rjlMaCS1}JN}6+ zL_aXC$1eH6@R}vDbKm@Hta8!aKOScMTZ8|6OF`Yi*1Imm&)i7k19ja*vISBBho{oM za6`Za(v4LA)Zzfb7E`OrG_k8;WC8J^UJTI zaX!V}?m!{Qul>8zbsI$IRdX{jVn&{wFW+O7qz&qxm2;a-V%iX2h*z*xhI-h>%Br#j zTg!k!1|0IaZy)~>1}VX!#3SDUyUl}#pWqiFVq?ci$_s0?u7f$tz*B7sUBgv<=)mul7@=`yExj*R(ih^lkXOo17H50 zh)=ZcMH3YlOtvYPd!n}R-JT0J1-Ltl01&-rLc8{7mllg!_VwPtvSxvWXc5O5+nQ|C z2>%U_iv1ZgT3sYN>1MbYI#s9D85v(OmtART{qZcD;Y|qc6ee};3cC)SX&6FdN}(qd zR$Uop`>TWcbC-3NIc{aWkVM}ncI(YsddyFIf-<4MIiY`}6#9PwYww%XbBtRk{zm_E zbh;kud40}=U^Uslc}CCtt9N+Yc3Quwp9%yu-n?K-C%6$h>N1?{^&yf^=+fct8hhp+ zMS+DhG6@r}C@Ull$v+KgHTCVRbpNA$ev$L6LG?Uqpz%V}Fxc0NMeHCCNv8oe8ZE5c zWFb~wnVwWRzijpO?5X}F1@4DkIf&^vkv;2;_ZuWVZ(&>Lpw+VL#QWE#9K_!s-DI=NmLqQfDPV2QNQoBo;NO%WPR%s^B3rJr z5<8@I&gYNow`MpE2~*YsDn)_!(54?9bm@K11KgMYo7rC+?jF$E7lcpu!;jk5K+&fQ z4u|;In8~`G*H})excomJP|art)ElK36w(%;So`$rdwFO<8Give zHoLrb9zX#@6G)?B??+^|7Aex8mu z?+!ibcb!kZ!8e_wa6T#gb|dcQMO{HS%ZoQ9qo^Um0td$)A!?`s&|+Bt<(BuMyu*#y z+n>I*e*kCr9s)(diPbHkg|5^~<eFL&+{C0t<5m-U_#Y zMc)I2X(<>XG9H^Tos1Ir$_(QIk+!8MOKw3V_Xn`Cr?4Fw+*CXttD|Jw37Qutex|R; zs*!YDqT*r5pI)RV|Eax#)(PQh2AV&VL`BncU7(W;VH1F!B~x}TSfz)>3NR04>K$$O zC~|f{o|x^#|1W3rmc|#6)urLWk2Dzr%c z#>dToLB$2x;&aNR>6r7JZvS3StM=7nE>r)6lk;^UMJ5BugbXz&Q&t*E4Y8RwmnB>N zKJInyr?0tPl`|Dsr5!iwi0=X#Lcd99#|W6W?f)Zbb9xmtqXe*s*&Gli$Sth`DGEvs zgBQRW{PBEk6_GaD+=uC2vxKgAJD@9OGbZc+D&gsmee3%X!TDR~sj1E$MvB)&+)YW2 zziaw)wcyq(%K4<0XAz2DE?A^><2gh7Hz4!7Fi7%7^1LxZi;@T>5kEcIt#llk2ih9l zZF6a(f(X!veE^oxRERn)=z-Dj>v{sT;TzV6EeYUVKM^bchm+1&@IB6F<-}{h4jFZlzyXp4pA+O zoN_kAIX@Ye`L|w5q8?+lV=~p?HNI7tM>mlS)2^tHe}1do!wmbl{)Vzr3lZ^Dr~s2}jLXU4S!RFg~0?ySMRt^}~$9rbXI)IZzy=gj6V?ODgk`aPTjJCM?&z`zc^Q zeAkG16zM?fC^yYU^6|cQSHg)`;SEEq%@HZiyNJ9-!~O?KJs%eY1!1WHr~_?DxvzKO zoc>sYP7ERhDgs`c6mhw>Gv)enzi^JZKPMVQ@xdvQ*2^s z+rKm50Z5-8#Ms%~&aLdoJEOG`oLziHpQ3S&Tg*GAf9N97QQZr{IYoJz?#4cTYvkt( zj7M|kR-$MdMCll}{aDvLN+7?ubTw}6?NG6;u@*Fi3tfC4bWS%#nL#Hzd+%G64!ZvO zRQcr}-#M0B3C{z-&e5)}Q)6EL%0n!j$N%oCxfQbi%Mt`oFhwMe3tsTBiouN1Jmd_ss_l9#HqUicWw8?06mNO|M&-&1vY$a`3% z%KHCus>b~9Pu0xbefCkWi)@=r^$S6aqEz~4H5I1+M>Q4u?Y)b=e{lYPsi`as1P8l+ zt`!%myQ7;L80n=shlmw`aM_!F9s9PYnBq)8PyzIs`8)gtHtngPYmX$bd`?UXjBi=0 z{WoPo(#VtzKU@PbX)~aSBb!1`hw!PK4>~sW#^G-sKYGu+gbc-G1rR2!G99OgwRJ+|=>v2u7jWyb@$o7GUh$#xyFekYK?B|)4t6&mzWpz$DskWZ1u zYxHfPl`ak!vaEbF|A^Z%Zm%gIVlSbb08Y{fng|Bc@> zeyCMr^qc~g&NVA`*BhePvp@)4fOo^_Cw2|w&(mxw_Fus+NL&hHcIgpi05o_C^?_}G zZ*@aihd#6E`LG#-D{Pz#FfExNPtz$uYDL?&fSl@}W#iN(3|{5EnTnLd<{xvX1v+6D ze+cJpz|z96qC7c1gDMoLIPqD*qt#m700&w3w%P02LVvT(@NYTa)$X5>AF2^-DxCJW z6r4&O7JlNa_OA>_tZvaJBS(JI;zM;`G@+w)u`{KkS2&cb+KZJZJM3}RAjQv_c)iU5 ziZjAzg$+r}UvPj2{-+}dXtk&!e7v2cglr@q5}4IzJz+o?=t+BTmP@t1ghXn#Sklg`c9 zPhMO%mX0JU)_BRJ@m}|*A9p%S~{pK!0Fyd2hs$Kzq`|A0l~;{*T}ZK z2U_2Akl;o7*A6Rcvc;^8UKN<9@^6`1fv0%>ecaP+4$1@eQSMOqqLb*m_ktmg?mXXr zqL_w33LsEi`k+m^*9?iT(Z314yDi+oKK{0GNzhvsO)8A}GV>R*N2e#p-WpTtJy~wXoMQP zFDGBdM}8Q57=-0A;&F5U#sNa&Pv#zHNop65Ie@Ru zC`61yL65Ty1hM9$YZO6CLt~SKJ^yd+Zka6KRdDry7+dnqqr`tJ-CMn5Q@FR&& zzC=uy-nkdw4p4*MwgzJK4U8F(gqjWbSd)k#xH6Q8F9m8-HaW&4il7P6a|ZwyG?dga zSq;#pEt&Z({r&Ihv+|AUUBBq*hlzkkIe+txegid}+eyfcF}yQ+OQK>B*NoT!I4t%` zzZ3Sn#;5kYHV>tws@Cdy0{mT_g1Cq<^03ZIBrSXTV^v*;lE9>sL_4V2{|?ZmDA_N* z!W(WW$d71P%>^9lm(s*0yw0rWmj``N69I||cfeGQa<0=J{(ZYRxuNQY_1>$H>%rw} z4Csdx@M%%<+1|7&5b|td*h|cQ-R=7#_8^0ZRqG+7F@-~QeUk;C=JIw#&I7=_oAir_ zy7ufuqI=CBD9;g$OC{RWkAZR}ei1vfH2Y@*hObxOs6lQSdF&kFfoe8-iOrhxEZZnU zvJH!l^k1ugYa1C|*gP}_NkqjM@4~1b(Nk3%c0r0KZM2ecbdmCQp~9rO<79juD5rj- zm5@=kzL^sqbe`-h?c8n7mIJO^T&R`gS*s*KfAj0gGRrf;NhirwsTo8Ut&uTY2MT)G z7c-3X=aKhe+Ov$9(Tg9rQ@7iF3mm99eTE8mrCj$GodS2&w0`5`TqCTPc#&DRUsS)J zNx%QReE4fZyz*fWfyTz~H3n7DK zHGC z6Axt{QNS5j4Z(h)+Dc^M;r~AT4##wL@q?4>2GG}PyCcjAJcJh@@V7T-&Y+O+ROjAV z8S&MBGF^U_ERjAhWDUC0q-!h7jPn6(=!(1yMSW`g2jQw`6TCP{AG`Qe1m&DAsJO6m z@1CpGChGZA{ROmy`WEA4LAw1TtH{t~o0+8UT z6*p3bq3DBH&)7#ROSS<`Q*!f>JNx?aIsLq32&Ml50gyNe%p0Sw{YT=Kz8L7bOwR#b zjCc3koa(4twA`VMRMluAe*X?4rJxQ(yRttC$1DF5NS7OeF`}ryfCLjHFYn6M{gbx> z=oJ07SOcLclBB}*5C@rTS_QbpJ9aA9U!2nfqw)^ID1Lkvd%+xmhBkRK{&Hst_qUzO@UoXtZ9~jr6o|x|s&gc_8k3WwR1q9HL!e!2d zN<^q!;K8QjQYE~Qfk*P7*Iqr(uA%);ud!9jQklx|xj|g$d%|^>kbh9)8}!8D;a%VZ z70Pk&OX}u;$n!^KB=yS|sq#fZ>-QzugrOKjILTt{)aa0ZO7{@;xLQn5QzlR>z|Cef zQ1C#Er^&)#v3d3C#rzkirES#|pl*iZ%>!ixc%FS=CAU039U64~+njBbNC{xN2lU+S z!PF;bbv%*N?wRo6rN2!1Ny70S0*~Q=+zW0ZR5#&sz9Ok4@HtTDIx+byi#bkp26Xph zxQqg^Ta^K^i>v?{SZqw*l7$fn@%zZ+z(}hIUPA)kaARVL178)B7$5Fe;{iO8lBpJc z&>3Gfs>Dh5I(0}0kVK!P(R3o^!df73;!B2Me*8}WG^YUDUzF+^a4#8kq^_V+8Q@)* zPZm!!lV&{QsDxJ@-F020+@qtEZ4cIYb-}yMr09JYJUAXqq&Nzm_yd_UD*cnR-1~-9 z2C&R5P_Q*&EqsE}|Mr0tixR4!v(q<%vuEDHN?13Q3__v$!t_(J53Z4o4~sL1@~y>x z6wZNyOFy8!3v&d`0$BRfK;dMcBY$eR^_PxLh2=t>cI~o=fwFIS4ovbRpkvDt3NsI! zGp$`lOvbx<_!13zm6Vr^cG4)Q@h#S3sqk3TNbI6e2L-%=$on*qE=%mMy#W<6$>HnQ&`}FvM*HvM zxWETD;mnJf8x#9Qi151?zImn+eVn4Eym&dh38II-gNz48@v#q#BI)r9wNK}=1hDKp z#JSe5pqWD;=O*5iK_mtkUTeD{f?XC3HXH0!+p~FYj~0F17@O=ub=CBbOcnu7D=Cguxrd4k*a+hinf} zV02!=9K8DdnpGLC0NuVmfv(`m)~h&kMiS*)Fg!*(d>>$pHAGQ>0C+LU#OJ>yXGtZ( zVhMv>n9hKQP_i^&gZNVWTF4qU2wZfoRXx#t37ATW!Pl2zd{JS1u>`vEajCgtC3naa zI>z(;Oq_8Qk}yOJwtBUnvcMJIz>O$%KhRVQSnmbify?}arRYcZYTFjkmf?uC-NkI?qlR7X=PNn5 z2O?PPnw^2selYvonyqFxWPPJJVasSHf3b)xLK;|v*!P$}pUG-SaD`Px9BF5Y zFV_6pzA+qdnV(fkzcGpKil}j^Se#LV9Bg1PVf<*qTAxp^+2*8|{`EYZVN4>RMgI zowXKSmllx4B?_x0&{~8BX5J?H3m!=vPL*D^uq*TFnB=A z0nl4!pU&uQ!%`~gCc zQy!jkVU_9e6s-G8iM!b4BWsF&z5yxUkRc8A6k&nEh-82vP2xA-Wnm*Nb*90-G9D+u*e8Mm6{fq4_DcQQ#K}Zj9Eha zzajnS_6NBjxK*a^tV=MY7BHlmOLzqh$c;{xHPJ$R(f_y@3c{#TFCe}yG-gp>^KQH{ zdtuH~EN?6jX01zdQVN;1y1<{vtRX*<+RiMcj2wGt->AxPpS*&9bB*vnC4nIR1V8X~ zQheevUMESbcqMm;OMFExXuGf*5fF6-uUB}YJN1Z zl~%IaK23iG2M3uQGl0kbICa zi7BKIbvp~v_W1v4kgh&&=i5He1tp?~+Yxsj!P7|De&B!`C4+5rChHq!w+aLpdG##} zcHcNd`1h6PSKk5M7MBHr_>;4e^L;p+dLG|4y}u&Pa}iD*Rw+puIrzQ++@j-a1Igz9 zw$MznyE#R$koxnlir<0J^oF58O}+oq7s+O>xB3cP>hq;R@y2ez%7y@4z6tqICc9N2 zEvHY(nx=ja=`kP&{+B6hn1RF)$%s>~=_1^2Dne+Yv#5tE1}au)8N|5BE^`~#gj_P{^>OjH8b~_{eK!h zW}hzp*jX{rD=%sLaL?MqWdC?FHyRWsR@s|^KMwS+5wmYScQM;M++p{hd+wlY#iep6EHAi zg^b*v_jozSw{0VSs^TX6!!>V#GX2zhJLSITeDY%Ove1)ugOg5;8?BF?x>KtSr-^=Y zc(^P%M)=I!ur7UXW_K#2jO9S&-qzGiL);cI|C%xpsoebd=5tA+4jb#Z@x$)Cw#FZ2 z`sJsG8=F(DCfE3FlC~AkoCnGo!WQgzNz0Dvsb%($BP3&+`A+DY{MAFKPAbI6_m01+ zUo>W}3~Y{I@(?~kGaRXNT+g7;6+=xmKxla)G`Ogpze9UVPBF>L?#WxwxGX2;$IBrU{F zuNBUOiymYrkyHkWdu zq*u0t>@r=k4O~vlccL?QD?UfZ{F?Y5b_LO-6Los=iaSyj)o7i3rl;?+U< zBz`I==GP>6oy~p@j_DYna&g?Kck$fYu_4>Yp!-55n?MTVuW#~mvAmkeVGT$4-ntub zH$2x^gjV6(dX>8%-lF3*=*%BZnPl4srbM7OL{IOCE}Ny5wO)0WEWt#yiI z8FTFRHH*u2Byy6UCLUSbl8NyiIxAXDqq&Wm+Uwa6KNtQF&xFD;qghH*iPZRvHm`{_4PpIgZ6@`n*D~MthVF2!g8QlDkh95D$X#kuc z7JO)AL=G(JQ)1D6a(?mqZ2V|R^{KU7GMY>vT#uT=yF`igFk2^kPoC&n@#P+d#h;X& zDSPE#>N4xeq*Dqo&`cOUNU&zPcAbdlnrMP6c$mmTC+czhq51pc6s_)DSDD?#t8};I z)OD{I>&VAZamMow>a&@7eBGGjgIf!^tWvA+pk$aZn|MY4jdrvNjrVU1_!F9X(8!}r z!q^7AlTi0TuhWr77#we~?_kIn>$I@Q_^WwkcHY9l&x;bNy*xM=7@#gq+>xPI;AQ(% zGaBr_X$*`RUN(**gCQ6H^$!*EYxd$yNgXpSmo9@JV{-;iKYVbBo$87SqWAShL#tfl-kO#B?{)7E^+Gg)V#SYf`d=x z-AI7ZQbj}}C$#ecMC&X#Jn#ePz1^1|yyn4>Gq4`~E%lbUPJbo}=T6>>?WMQddY2vt zwNTmcWQe0q;59ZV_^LM_*|%rv=^aWO#t2)~Q+XHT!=KSKB}sMXfBD>a{C-oKMg7@W z=C0h&E{hjEHB=|H3*#79xX5AP$}pzQ875U(B6OC-4~dJSOQbriK)i3DghO9s%w|F#@f0m1xtM#4w_%&aOpz_)_Qv8qfEUGLDgalWlUXXG zz_T#;H5Kvl=!TYI3D~G`)|>TaMaaeT%G1lmQi12gTWEHOf^hnekO^o>+JJwfbp4WF zDF+~cU;OhJL{3=v z5-xvnTG84+Sjw9LNq`7wXFRRQ)h>#Ly7)UPlZq}1kp7JXyv@_YA00x6Z!Z6Xb7Y;; z`VcpuEVYeix6MfkijqIfxagCDNz45ze>+zKVaQ=Yu56rQ}Qr!KmBr8B6NI z9*JSWOZulI3ir5(N->BfLatwE$r1*%kW)Lqo?R22Kp1|k<`ihKJoHyaRrZJe>#va` zxr#9x9h{m%^FEoFB_Lo+HB|s|YS;^LSn&iP#~Ta5Ky+`Nl?U81@or3Zx$2Kr#$R0c zXM23Ci6J=w6=_`eoGco$*PQ|$Z{S3xGi@(3*{zl|ISD7ar5C^Pw}`7I;0ww&~{3XWYNf#}xPEdJtwSFb4*`H)Kw;7Q>-4W;Bc%yi}086O8@o7kU60DbhR;ZpW&5h6+O|?_WBFD2V$R0`%Fm;KMQjJZBcy@Yovl}1 z8g>f&juJ!J4R|so3^v}<&A4I2M(FIT|cHS?;K%_jxzm09$R z8UAft8nOy2T{fqmyek}2Gmr9Ue+eIj-uD%$x^Ww$62y4)4y*O$^=;#`z*%Vc6flfk zo^h6S0DRaPe@|lmyz}m>%08gLEEbo$?e@=EfAFF>0;FO)0H@coCK|V8tu@zo6PvOP zbrabh5G|upXw0?6&mlgWhcvo_MZlMcZlatKA2ZKUiks%@>mtRyPwo}f$R5QtNjC2J zS?~wKf3diV`{v7GeRj6^s*Omc=W$4p&sFq-afrtGg9(&B*H7kixo2|K^OF9!0ZZV~ zPbY9mKTMSrkol~#oE*xwO&Dv)qsN9$WWt4tS2MaVkXO|Tmhv?q3lnvKQvkLP?a*iX z7)}W=JI=lX01hM`>?|!U?M;n44(RIsGeJ)61x9u{O0S*)u`un>__fdjy`-wzFW0+^ zfu=ydt@8W&eXb~BOpgZg=XbQNOVYL5LC28)y24f|>-WL?`jr4|*(rMGlg{pTkv`=6 zjnWd}b(D4O+zz2{#t{v(oA7ym^XOsqdxq{^lnKKdUd9>fTv53k z->=>aGyk08miNw$XuQ69mJAd>m}SaIEi2clC6Mc`S29T zA$RFkK=48Sxgg@^!gOMo+<9VR?Je)6nZ#k0cT{h$RwlMa`RKb&o1hG-w`N4~Q1dl^ zi`hVZ`Pk>{l0m{og^hIS^Bg0%hQ;U+E&$;b00?Z7kbg2>?0lh5*9*!kGhh}oO$M}o z>tYA*K+3B;Hw0&VxI?&J5iDd&kGI2Cnc=3+wJ&y^<`x?5tlIko3Cj~;THkJ3T*C0r z|BwIxO23LHwF;p7Lf*P^KNi-uP`YV1t&yuu^&*ISI3ey-y+|UmV1TCpU7J$zK=T`Bkd3;%jwaSl3dhNBGr{Fi`sn+w(D(%!=^V znqdTk<8k%oKkqY$T<48qC$_=OrGr(CUVDNtc3!jA{lOMr_$9?c6sb(_B#9&WDT=!B zJ1702zkh(S(x`juy%5gyU)&*os&04(ewmL@#$qNfq-r9JUC-0~@qJa=1ArUk+n<3% zmLy<

st${vfL^@+yVhL1)hdI7OlS8O?o$Wm3-eZI&)k3)OH=0sTD zJZpFuqwguLSlU{$pIbfBszX-G)Qs(YWp%cQHm2CnPt>K2A^ZQ}>^q>b?%)3-LNpzXf`S?jLu;q`3w)dOO~yEwPj?0ZQtwBn@wKb$N2dAE(ftCP?y)xiG{Z zdZv@6bW>h#ZjOCwgn-=VyrdEH1O1c-=SdTO#TBLnQ4bp~n9JQITgs@8L~Wi8#8DTg zxwbzhGuYWAK^o+?}BF1H{&G?Ifbn2VWt}SM=6LpWYc>P4! zq(o*|Y|UA&7@>0wes}oZV3Mw8hHlmp=alt-QjisBG=IsAwj})vLrw$0PP-{`vy5>fMBsC{C~#dT6enrNf2u$jb~Cv?^^nlNQy%hzW| z=#tqa33uY3udzbG$Z~(uHW@$UZIavVHs^lfx=c16n+$Z4ltZ(F7OBHA-2)iU*Xu4P*2ogQ7FI95{C2`wSS@`=)QqGbn+<)D;c6YdnE>fes_xA^TPbhmcwUwW zcePKZ97Fp)&+{<5an<&(yQS%EQbh;nIhKA?aEryU%CJ7f!3Ut3Z z&t*V&+gvp6h$b);9aGwRbbs9~!6cU*P=Y(uo?)2mnf z3MR>8_Ox0DmRPw-FU|IctL#hrYeD8FEaC7f=Pxasw`wlUG>#6wB=3Jc#itR!?(s1r zVsWoW(`4FOkCMk|k#CiIQf4%O@I9}@DI!}xr_Ijk36rLVUV;{8!u@48Y!At~^P)C= zOU5Rc{T6(dB;g5>KI)WV-_a~}5(%iz?~P})BJ*GPD1UpmSRQ)F+=4=dQAp0pG}B+f ziYK*BoWT09SuHjPZ*ddXa{7Css3loqb$3Ijyvf~csr66IYtEKvvabEsbu6~Gzux<5 zDI`N^Tso}d^Ew>WBKScQCaF1-c5Z9#@*?Cf7yW&z*ko0qaw7|wAl22*7LDd$Z^^UKtoYoDr;O8)@IaNg+~N5%c-cO+~vyX`kk#_b0^KIGC**wb|H&hX*z>-ks( zEcRDLG@&Hco8Cw`B+UmspnBROz*)0 zRK4&H5qn(rTP*gX)~gD<{Z-$M2_#~1eMMr761Y)a;?0|;Q+ILI%*s%HbRJ~JaXP#fha7A5w+Iwb`3w?)eEv~S>`7Zf>vDu&0 z%@0ng)L8+jU93x}-UlH9uj)T7bnd%W{9@Iit$SETbw+kMk;q@5UonZ~E9z6#!@X}e zov9dXJCoF170yxn;0@!7g%zBcD_rLtGE;~ubEIQ7)*p*}qePi%;oK#*pPKG^uG{MG z)yk)(_2P#w)t%O_Vqe+UdVhw3RYj6}t)yEhM+#NUXo4_}3m18prP!StA( z4m3FxugGPljq_jI%*V<73H-phPdMiL!I-}Wb2-3z zkpC(#(d|t4odrR2d~K8DTHY|5Y4rH8&3CiI0KNfIW)w;6H{(8G8$7@A?2@_TQ^Y$+ zW@XmR_9lNrbQ3k6Oq%$1yXwP%`k$;+lGINJe}4Y`dN_NozDZQl;+^nj0Gjp@l}t@y z0<}xvwo3BRse;Q_BoTC`inol-m}+QqR0UV9Kh&x*Zb)dl!z3TeJJHKNx1o$2>-_YJ z$%>G0#So46CTcd99m9&sCzu`0<>^CDK+QIsLMOcofiaaz=Yrbb`ucivzSN|BUJ?4e z%te$iRY+s1;y#1BH^Dk-B1a%;@GNaZyN2*qvRWD?TAkr5Z{4XuQ`|buNubKb5d}I< zRRq^{`~96#%Vc`QG^2J6S3erf+6HEg$Xj;TjS@3k8CCV!kpE4qz)f)6?5HkJjUZ*4 zMM4=oAOAFlD9P=!pQcA=W&950N((NW=iQr-x%wl0jAlb=Bh5FswriQT>w%llYtjg!`Tq!b(S(d-J(P}%;pqMeVl%Q{R< zQeFKe_3vt`HeIwb>5YH*%?@vEt>FgO{u>iPsqVGR49P1U&75&7A>V>SQ02Y;!K8^s z4zmSZ1EVTDI5a7Yqnk@XwPT$l%w{C0ht!3_YdGOjUPXtXXG7x=F6x(xwlCWbl=dfi z03+liH7OH=7It{?!}uzuijZo&j!MhWT&!z$>?z2W;(k8Cj`$$Fwvh`~S5#7wcQ?BfF8_p?$y z!M#bqlfK{ZbhGm?tn4v~^K=#s@pqE&(K;LT!=(4GEG^)dw(93?#JN&F@+Hy}^-eD} zblp&FZn=B0iO@=V(OpPPk!peuPtCCBV@a}e*+(*?0V}txevA+7bC0OTUj0OQF^8>w zT)}^1upV(O$(c8U{JBN05P8w7&wO=y{eCXc>fN)&KgreK&;6I6F+{tAl?{z89C=0m-ap);enL z<>>i4dRt%YYDeq`ni#Fu-{tS6X2XBKaA)Tt@l)og8wi|;J-FVGmxJ}$yR4LU6{? z?O^qTcjK0db!(rA_$@j~Dsf)MQhuf8X=^e4jt6iRC#uOhwXEL%rtx$N0j8URZ{`iB zbh$0Vpo@;=m8K8*E>kubuncIhR2q@i`s=fh1zkCTHX8?iwu?pXSI!cWusg^HHW4tZ zN-ssSTT4cB)1zs>i1o-5o|k`9`;bRpdr3}v;rz~@@F{j54pQ6>jc>EpjD5b>Y{*^M zbL-FMC&V}GaPVSreaZ-Ivewutjr^xSZ7OHR0`AL>NuJ25 zXCIq9P~DmrYiIB^4r}}x1J~)>_a$LHvZ{F>lX0+~{3KwGx@P{^+(p0ni~-)<$2;D! zy(*Mh)G54sOeE}LW1J|>_jb<+yxl&iZE+@gr`De*z$Ayc=B+BnXmtBfVyzN@)094C zi+)Fm@>R1XchYteVJ?RYA+q-Hy|`q@E6C)A|4NC4dc#=IQ-k%s@g zu&!;Cf#AzY?Lp}#KWy&LDl7u803cBUU0=@!gV4n!vQSj<`r9Vqbra2&hdCZQ%YDEm zodbl%$H{En^kjz{-y1(|&DiF_4N+sp{m&9%VWkK(Vp{5a)ebcySVXjK+0&e#l^j0B z#=i$T8tV->^h!T!T&AE$KfXOK^uwQ1XZbRc5G4)R{M+(%}dLF7sGK@3oR7!L-B?P**Ux&kbFPFK-ku zA;18&9iTYnZfkc7AS=63YINso$sJArAO`BvXMIv7Jl_1GmK{MFDyi)OyD7Kmy7>Lk zu&w#P%bTJ!0n-Zuh3s*cZ#p)~Eb12Os>6V?=Lb=g?N2sfAlrS65P?-?SBU@n=a}|2 z#Fkh%762!@2WX{g8?yvmPZJ{v`{@+oEI8!!XS4`a|4sw+P^i=;PF#xqihRi)=+F5ZGPqLu8;FLA)SY2OK~$!9@Y3>r%p#k&NDxK%jTE z?wvm~EyIF)eB`6wT3y!19r|z<^WJ7Xa;R*a8L=%)y3!Z@$}fBarQ{o4e-B)%0k+Mt z9(egI5|A5!8#1my6Xoa7AVllqiZr9#7y}B?3H}a{&kM*z4Nif`w#pwyBgnm6gr$5$ zpv}+fYf%ynWYRTyj?mb{2h+ll36wC58m$2R_IA*~98Zxu`=j}87KqHMJuqqroj z`dAP#6@KHuLz_OKdyT@Xqt5lM9!c+Lxz0Cea#`;NH?$4&vP8UVIusbNUxI|t;UkUdBaBT`?Y|V*t zG00x6AO?vStiJ!WZrG;P-w>H3_en56Na^XZ2g<479%G{)b_t1#vO)5v`riW!fc;Y7 zYED4tWTF3~+5c3t0~&-^UMG+yLgJEvm;Isl_7DiwxGIgnpuZ9eX!IR%n>Bw7J=xCi z^OC$pe_xGa2CI7DG_+>{b+=W(+%@gIr+<~{4kvu zk&fN-O=bY(e{2nK*C)9iYMV-JU)!tov`5RMajpj%#`6wfNMFpeY%eh3JEwp^0e zoe4x014>I-COVI}avMezJ0CPl#?K9(^B2J(4}F3%b_P8Gw7;m4d$7OsbH&K7T`e~! zduu~uj@Ih_sQg}B$Tp9Z5y2hd6GP33!VhHgXgBrgFuH7n?PGI>!oIg*JPE1gfL4!q z9k1wBW1$u97Nb}Y8;XK07Se{WuQ6Z|dK+tv@FJaT68!2=uO8MW<0qHTiIw#GYaTne zvwtHB-0<%Iz1$sj2}_ni<58oYN%~=A+xqX*f6mAQX`tN}NXw3>khL~%Kmpbco0Q%<(x;VZHn!g^nxJ~sG z$r9KWS$O=9c`$~gc*1vS0*^um)dc&fD^TpTAtpN86+WjIc52D^4kN4$r_nihqrA&c;t+mxiOc5&{OMtZgnKz$L#Tb-cb29tQa$N;EoIZG+tQ}uF zbSpzYv@&uzyl{3ZFJU2Z#XOD@M^=R5GqAGd{yJ z1A$Jkbn2oXWab;6+Sca}JYzI-&*Vx$9Os5Le$QkhEA|U|s_BU-e6Sp<#m6zL75~?C zT4g)ECd`D}b5!jC^+mTTCc$P7tnAzMxran%Jv4nEf#YWR+DLh+5}2BFwqY#aOEmbL`C|3(5WbJp4Y~hc?mk%}rZoRte4`Y*681!9-ZG}V- zkt`6^up?=CI0*&YUWt58y^=3`^iXnGcA;xdIXOrx@48Ahk8FCOC+5R)^7oh@ym>oH&jm$>89sBiV-vg{fMr_hyY}fGf-dtoCFczLW}S2Nz6U&k@r0zb-O2> z*35z(lAN+$|8c$ZDSrEXZG&(}_u^y5I@nR5D{*CWx+f!pY!@0~9U%~Sdrr!!9J-Dy z;cZ=!S<;r>sUkxjO#y`6^8-J_yymSD>LnS8nJr=*cy5z)hlQ?%DTAtTGQQPlo??Qo8e#;=cmy*MLno23-}3Ch#;15|O0v_= zS?c_yU1Km3XN~pPu6^9msNfM<%8(HI)Twc70Yz~5J#MdCW8>!E9PWm$Ihr|5&6~m6gQ+eP2j2(jRaYc@AqKKJUz{#8a*r`EyvoO6I^VbkaPlM2O6SV6 z|M~v6D8+#Z(KI%t;VB#WMH~X)>-%L{R zP$Su77meE4PMa%j56&~KfUDLQv3q;(EtW{hy}VHS*Cs_UZ}jNURCxXso?+I088GKiC~@Ims&lp^?A z<)4I!Ak~vnReikY;E&Ip+#jvCmilMNl+GUibWXim`<)ha zlyopaBoLq4|4&le-k-+G@`*QwlR7Fww(<)hLJOXtDN_zwG_6;MZFG7ksWPzm$l$XU zT-T*3Asm0QD#x7^$Env2JATDuxjGNYe4;t&@nhBggAC|a0N4QTkMSq;lefdIV}lgo zWoNRWDw4F$AN)H1G8t`L4djjML!fSyb3~^$`P0WDvHwulMf@%XtZAS` z!PJKJP>RAwtY$?l%7OIV?ky-S8Rb=Y9iY>W`rv_6wt0e5 z^?rE1&N&8NW-?Ysqxa1O%>Rjn=~U`=pwkov=8cp0Od^afN(fq!R`77 z06Oj=~h1IY}fN8 z;vPZ(T+IO_gD?XPzKxI3ydhnCsarfh{n2;!2*1Qt)6!HlA6axS{I6N~xhGH1WUc^I z2`OIqQdIC47! zBHBu62mRX-^Vkk{wQCAOouJY3D}OK3PJH0OJcrMGCc&p4UVoM%k~3HGNA)K@t^}Px zeoS6sQ#Zv0na1<|!y?G;il)zpI|B`}B+Lo{-Xbnyx_-GIOaweK8d?zlU2&br$GNZA zHRIfqz;$apKV4%6$Pi8k=^!_u7)~GpbJha4|A5rR>jNF9pFdEa-n<8L&kzr#hbzSB z!h?oR;VA(IuXFkZx14xZ8TkGUKLX_WxExFegp~KDFQW#?tFD?H{Mm8UEz-{-t=}!V zRq^Q~QV9A83`SwV(pQj;LT&n+)0k4rr7wJ(ghe)R{dv zXwKryH-6xY2XUPQKjlK&>DazYT1}t3r4HAKg)H1LH_|*3)30qZAX9du{-l3=ltvF?Uo8Yn@CS?9D660HRX?k5lZLR8A;y%HF_rLLPqM1qzY?sv_-UyF8E#-sffvBzK z0b2m=hyBpBg5k!PNk^Zedmpw1_cqZEjY4G9o8;5;;vh~MkzAg$@yolHiW5RxN0|O>w-}6sP}GU-m48bcbL2|$(v?(P415n2o+Rcvm?$;R?P{| zF>s#D^*vo$U%Ob|K~tHjBvm{S{NvBs;r*Qd@doE`uaU}rp%N4ULC+t=HVMfe@_Q=q z_=BVfEDMVVH)YFRb6D?~GzJvxC8Dh(lkM|V3#UUK{INizc#DeP2&xn)*VK2mj52Bp z&gg`y1z8YG^~=ZngY_tE(1qM7a6F$#^3=LPUFCp*!}MNw(;nPs8;z}f6>(FK8J2>$ zq~C3$nEw||^e|^&8?7G84P+>O8Y?J!1>@hM+NyO!g2F;L$>HEBg ziJExB=2#Yh0dG|g-D<+>m59VO3ux`Aa80z%(fuH1o%JH+3|6ra;SMG-lPs84(!}~y z3HyG}w|BK+7kC#EztqADp}bHg)*q|9R077DV}#Qe!<+WixVjsdXvO26w`kk4<^=jN zutZZfGN*YTZe*|}zmK462RGT*K>imbC0;fqew4AMCi-#uAt>U_o4Sj?Fk7!RFkj9; zpJ*p49(48O(hRxkd{ogJB@yvs$vthVMMS}Lmy{4K!AKJR4zBkWO0b#Mk1OD_>99%F z7Y!;=BDqWZ4EM%5vlS}a-koNA`9{F;OZ!9GPh9JldPc`EXz-&@#7>a657C*i;O*6S z_x-a|zqd^S{00AG2(&B-J|>FD@9I()ip^h<<|d*tP!kW#(it|t?_Ejwj3Q1SJ7Q1G zT8w!HAxnRWSF{t)!B`7&!wocLsr+lMVviBl6NF z*eS0_Rq6U7o!if>-#6G{>buzHlXQqe3^p4r?vj$P=Z114LEg{DD4xZ`G=dsm`}N@N zZoM|HVD#BRkN2h-LQ;7F4o4-0?`n5!$FhX9dfq5fZ~HbGIlY|GDZAhS8e#Ib#<9=* zFLsmJB-Vmp$cx<#mfu)kP6vGfDC{?OSA`!$sp(W?7idzl3*ASD()VtXygB=ZaJze< zY)@qSW3$)$q|{e2-^_Hj0_G*SUIms7g1^4eQ;=V9tsDmap`Wh_Lj~{zd^51kzaT9O zf!vd9{@^IQ{2BFXNay6cetfZC)skJv)-A-pYU~YQn*3I5fZ;U@`g148BL9)4XbP^h z8-zM>eV}S}FfA}5W~~zSt60JsBl^$POm$wj*mK9xqLNQi?7N69lFA>^!#V{4nOG9f zXRB)`pQ$Rb%N*At6OS&`rQAq5EkR@L)i?B;qIbXK0;O_@c;#xIDmMhWaweEgUuy-+ zJBU&!>--xr^)^HJnx2x@Unfr=Uwf%stK$l#Gj4q&HDQ{QofdXgwwesdvahwP0i#maA8g2*4Gx>ajAR7V2&UStqwcrxQi)A~iVOp{O z&l)EGOiA>eg@6jAhub(&QLuT&;c*sHDtn8 z`}}3Ou4F;Za~aV9z(JM-|2$>=#fjos-m4QIF5C_Ed|G@_C-gK1ZYKnxwsuCwHnXaj@EGuq z3<*Fg=MrV2EH9ov5^`)vakUviif54iR|K~UAO9KrTZ9BE%!jVs)&(EPf8s`+b~=Oi zB(E6v7HwHGI3$Rai#2e~TVWZz`YH zhuQYO`gOb~c+AM-UG<0)7^zlhTrCrQ$RzD^A@A$mTgNNg=NY^Gj+WNKy>3E*&Ckzx@=}g?M}!3ce~s+sr~pW2 zkoHN!;W~5tSPVXx$V;C~#I_+}v(ZD8M=@e`>D;Bo2mR@=p`Cj>Aocs{!D; z3FbyZQenE~FE>>`%%DS7 zOf_ummMRmvA_N074AiiHiDt-GnWN@fP9nK7AucF8%J$xN^&bWZ$dVKbk zC{8Xp7gv!w4LsDRR|EPMCojB9j{(hqdq1R<|EBCPFq7ei!)Nc$Ch?x!-yFpD9HS&J zSi~89&XQ%;7codH{By0I;2v4pC%^cUOyH}}bi@<@s|2xfPpD{seW&9~}0 zL)AJa@sIvgY#$$l-v1a;XH*%BGII9)bnnp~fBQ3OSWRscwyY^~eVntMSXVC>${im} zRcN^HT*)nts-ylI!iq&9dI5_A71JA0gnjhf()-Azg3810a}&aG4cLU`E3NCtn*iFO zuv%M>r8<~}mW?D+r2|Gzb*7&jfy1NIiCA{A+df!NS>U#_s6%(&@gr{^o%&n} z`%hd9@V1zZ=6%Qifh7tDS*^(e&U-%<;N;?XoFPnvov4>iqB+?K{SyNgzsG+O4@&`_ z3l*ytbn>$y8eo=MyWOsOFc{@+&fcn{+% zEwjkw;i%eJ;4V(P5bqaJ?5(Ox%G=;r#33t6h^zU} zZDsk_dsogCs*Em^^YsfDC{mirec?+t31tVBm zMeNuX=WhF(GaVUa4*7qdDuK!2YqbNs7Jf_orQ_2XB`taUSdxAh-rR@(6dI6x;p8?# zI0Y+)C&is`dV7*7qv*B!t4!p>Aqm0%e7GAvjD&36B4=nmYu><0#g1VH-*(Ce>OSn5 z6h)ZG%Ca2Y%B&uqb(6tyo^_vkm{u{x6oF}^{8;>_#BXyuzu{62>QU+sEoQgIjgpch zh3M8T-xn;}^)w<~%Fl|^V7|#i@_3%*3Y*&Uhum9#*8Np~<6~RQ|77h!b|*6md5jaL z&s~T}Zt7v`h&<)d-PCGm77b^!@d-VkPF$ z+_%|sao>AQjB8nF@4ln>X8VfH&)56#so)>Hr%R?LSt<>@Ou8(3k$uJ!WAwPhXtUHGnzBbi!J$CrV_<)1n;R=oJuHLBrTcaS)BZ$QYbD3$y_*s1PNi)-e zZaTdxgS&>u@NfK0vcKCziY1>bgW1L=Ef&4F{ZK3Q^gmhv=1a&|Djs2yFM$i!nDLPz%C7cE1;NZTFf; zDcq-3G1nNS#Yi*W+*Jl+qjGei=8Scw-anXl*J|0g8C)YzeE;)n=iBju>^^>og}C2T zZU9dm&HA6GzHxcI>0uZD9T(EDw*pf>tu{j@-qkrQgix?J? zi^!Sv@WGR@YBqPD6np(!%-_O2$}J6x32|@Zgw5k%C0HVmQ_^wCTWRC*{5jg|@1gr$5d&whCE=}2bA{q2V zCr#~&@*Lcuqp6)ySZA*>(2HqBVWPB-mYerq^M}(QI!I4szhcfdHhVsC&XmM^eA6DQ zQGnZSuJ6S<&{BXlse_c%%K@HmgI@7--^Y25d%@GR-y_^Vq={S!$_qOCP9hihEy6l8 zq@>I~!+?f%fQS>;(2U%PqF?j1WHp4x{9?I~uJR-St7qF#x1Pv>lt8y?i+TUwwJ-_TOEY6SRs&!w2cjQ8DEz0c!(SJa3G*W9g}`7g;h*p<963|ehN zx_51>etgvmPg@t+{$@tYTElc}9b}1xUlfg&3)zp3XpM80I#yW^TsA{1QSY4}P7+;) zISH3;^&|^+6hJ2__X@BMq8-E8&$CArrscb7oCK?GJU*1qIsh+-gM4I?W1LL)jo}Svq2tlg=QtiyueG#=UEp`#hw@4FAnfIf zgIM7&Ueh#n&eT_J9Nxoxg|tzS{rw><-1_C+4lwbARk#)$pL1EyO0hnU!R?d1(`N2abDV`T5eD zMA3?OOb($wU{I8AB#Fbs{(}%G9Mw=UKs3e|REfn`pAdP&+*4tfg|7oGAk_YK@P=k` zMF;o0`%+sBIr4M?_BSI_Qi|qTzd-NCfy>|uFev)HJeDdGwyCAOz+r*7z?Ro5anw3BVJ!04NZERJNOFN(4vP>1)azjNzA>xyc0yqoC>=4Z{5c* z#()X(dC+O<7n^P)&gh#}4$dL@4)jC=)m!EZAKZ2RK)zv`PPM7Evz=+g=-(i7EI->#L3@z?}Q@2RV9E>@Km;5vPnsG9$L(zmO@_B@4 zJG4(Lrbf_;&A*P4MAMFi?LPesqvu{8)nwyx(XKd!3St3{q80LyofyRhg{FvlS%RoBN*h?JMXJ*lQ1A8t8g*+F!$D3MuCb@qK<={$&E<#4K8Os05x{C9gUjvdmE>mG>`)64xVq=ce5$30RJg71v}s- zqf#w>|7{CDlIG|t%k?n+?$2|HqkJp@^j5e`LDx1}8mE|11D=YO0PKj;9r~Kcp@G#< zyv;w3?=?psB*HvOvODUrGGGdoTOs7}i`I&k?REso)Sz$c#C)oYJ`hz$T~ z)#$Sq(wN@OPDvzc68u-u=hnE;sX|Rmmg~+)Y}&(i#t2!%-snFjcV2LV=?-=(XknQolI4uQ z`*|Bk{nI;&Xa92tegZxYZCUf*jscBWVK!Y*++6e@osFA5qYUM- z*eK31Mgl2m?seF%%Fjj|iDwqv925godAjWjyQ^X_onedDFYBR#iN)>77-c1kb-h19 z^PVWFfrT*rtJiobecS=Vov$uKO0rf&7X@0?9pTOl5qt7IR@1a~rx}q`-9@}^W1_wb z;&pN3kVIUhG1QZw*tsFGSg~)|DapPxuZ^e39#M|(F_j=70whr8O z8i`v1d|7saL|5PMbb_+UU22B@jzE#1PcjasJ_LRBI1*2dEaxxYBhH;GkkdrH;d|Ot zm8AWo*zXto#Hf+M>ke>2#S2fWO&_odGowi9r&@4i`z(mnY@F`NrA1^O^5gqQU465(Cd!P`vX4pMp_V94 z3TWitGn*ai49B|d(13YqGIB4;;+rVBo{nN^y<8G+CB}wwbVSn6V0lFaeY52Sq!z4b zn_SwISe&T#xZ)woRAny(vm*9(h6`XxxV{j4IxPzI8cj~%P+sG&t0KGBzj^of$KX)< zX^m@9()(0CXMBRPELJ}KO7bI_$BV{(Xf?h9;tc3eMxN8SmvL3e&-@Wd#ukE(<_!*> znpq}^R6<67TmRaoT?NmvD=2h$7 z$ozC!EzWfQPkK*vqaw*}i}Kh@*J5gI$%V}C^_QZX3rV5+&t$Ftow{>gpU_)1Pa{n( z*g&gkQ`hR7g0{rR?B?8(3fqg-pLBG{>wvMlICW9vz-J1mw7K zkgL4Hd6A-Sd4G&UbiI~ejWW@l{TI2+7DYR7cI3BOL(nOa`CUWuOFnbteBX^GfeP5C zG5IMt3rv?11IKa5{bpHQOF`0Te^lzw0n3H+9Cz9^T(zx7j9T`Mz5ZU%D}i8_b<$vp(-`a*af`ao#1q24^EDMuSXH z@s-^DT-_KNZTEYk&+9wo0uGVP9qhVVvl zkN(IIdBiH5inkv}W2wD`ufF<43h<*8{DC+3CIW>+Zy%%@KmPo^>}vS_c5g=;eMC01 zWdI9_80Ks8R}c6o65&Os|ARX`rS%qmn%OF##PA*KKKrvD?~4$5tj%MnbGDLM6;QJ% ze@O8tglmrgH0&d2_};j-Rc~yl=h@E+YJlNqK7~w?6E~{&XfrgAACkYv^E7@peMJ3r zK1MPnE{vhz=BrmY72x|AFo6Tm^2B&*n;QVY%tsbCl^A&88_4@(i*VuR)sUYz_5rM) z4vQH!K_=QNO1d2Hvj)q%$G61bC1(-5{#;`31I2iN1sU)Gr{i*P3r0N9S!o88kBCJh znaKK+;eL1TA43ra>Kns!8QfJacGtZYpUAETGMi*#uvubF z|I-4e;ok5R8{xg~0p%qJN|laV1A3}c?Qy3&1$T0Q-o*}rMX?}X!?dGBa6uK>PMfO~ z$?*E0{>yg4p?I39pC}MkZv#ESsh;a?my&=1Eyt#j${{r3;7gGVdBZbEB3gl2JEZFJ z|JePOatQn?bMT$zz-4NOB*s~oplh&DPZPdrGQjs+v2JlPjD1l2!^jhhF{2vKMT&bp z$49f~c9mV42IUwsXG!K(!j;=>K!duYLN~YOMn?n(!r?8y9;{9ulI>qYsqT2S6;6N0 zn*R36Y${x6Gz!9z1=D$8Y8m509b2r%9U%k$hEdNDl$0QpsG1okU~L%`MNsix(=9P9 zXtJvu(Cq^<)s^L6ygw0CS2ZL6DLB()d;_(0kAGTTXK3gA;YKE7ulEWjFdB5UB)_R_6S#)=raP1R-q_wo2z<`h_HaA5mxfg#_5FdF zIYTNtg*3n;89}HUs+K6Qd;~fr9Kj5>BVs)V(EN@TgYJa+ZhU_?Pmr^jY{ch@h0qb1 zLd_22I%}4Bn?_^VcZ&9r1Y zGnpsiXq-HYP1yHHtZ4)ba zfM=TZDwHorv-wJZm1MpjGR6zDPVU}o66eCr|9G_<&fzRa!%V&e@74_X;C?UQ!7D3k z1+wZK*>x)1sW zv^VyvDja4xYkn+R1c6phHsC|JAoHVVNlb;3w#WB#kFl7&@W=e_fZOlO>!`q1BxcmA zOLY8n!E$ylHQ+Fm=RF4`*@A(sgi%3{`!(c>Wb|p^PB$L~2$=j_+WTX;p`Lqt7G#M* z!R|3Iu+ul6(fOd}6JRRF$k?-n+(msg-|EQ3!Jx-n&sk8faB(-}!NwNru z>c<5?D~+zQVxPDH%Q@2+cZQ>=Fcz?;fd z+|j>`L@uicJn1Saa>uUjwU3aYR>WSD*1673Kdigz@E*`D*a(G_=BWRm6svGYNzNsm zftNeABv(G1PzmLeAUeQap%1<8(Y;`}SLGy`^_|B7j8f()9ZOqVh`KA!n8I zWQr%*m+qpoP(p{$( z71@}3j8H|zCgMRHy)*`BX``g}Jzvpy;KD%Y%y1R%Ht4HvLkPX0lsxe;f4CLoUs!=+ zeOZL4$N?!w`I@!GA{LB+@+9C%=sZgrAVuS!dC@4DB>ql~b2NHuz70?x4y5IpIYRi_ z83HN>;K37>nskbMg+Sx(4^J*}`1xQGWCx@+jH|0pQ|`sKN?b3qrUL@3_6M)W2)08J z3$7`OzeijLzZ1m$D7cK-Qf;74*Jfw@H3FUh{hts369)+asKNzu)l=Ajl3yeaYLR{A z=BicK!M(&tY=4ubbNhuc$a7!<{p#z0f+)lP{{qxWYj*mnEk1^v=K~DX$K(h2tx1^Z z8Us|&8X20FXt3Z4#9gHTXwG=IG6udKbSJTg?OC&viieUggBLskbb0R&iBPc83E8D0 zYL|R>n{g3ucYB(%T43Jv6{ZA;Mm zz60cc61V1pXC#E)Mdz%nN)d~Qjmj<(BaTEzl#a>eCWH2AV$jkn@{=7&@ zdLU|ROhAvkQ~&H>X+}SU6qX5lR?+E2)i`9-YQ8%@b2i>pLn8%Zm6)tu&iE1_p%5MGH>?=MAz zAswbfHY+1O5ypRScXg~BS{L8Fr~$@a4m44$HPs;PfGk?}gv)#4vW8KMx{RM8?z5UU0E^ z=Eh~MvHbI65e7cX`B|EQ(|&|W zWVy^XguC!Nzn1ePc2^m+kgWp>(!m@lp1sqwzw_bm6$E@nniYPFOE}rs`t2o9ol(@x z`OO~-T{4QXtFgeZ)wceU&qyq}o4I@-r#TB0c}{&RFStN+#X%+Se_JEM!LRLp7!V2n zI#msVsYt~JS))(`X%;Fk#g$8UYQfewzdO)JgInG`Vme{+7rop-oP&Rx-j>|iX~S#R zk@SdX8`1M(9cEj|@LUf@5*h~{L=%HZY!QV-&71LZ9ISor@VJ$&&2$MQQQtVpnH-^Q zYw&!7VONS>jdIVHU#87yZM8T+bC;ZC@O8{`4G6$Ue9Y_=M@(hrUE~x#K5f~m?mfel z5t3zYBP&g4x=_ahp^wMJ&1IqOcE5UoY+eaWw9rg<8Yf^7vY!Sum?l6a+svd5J#K@x@4LsaRex8 z&q7WLx+uO_&kzUueW_yzo=7h8C{PG95-XR3xZ@VkwJdy^RN;TJTt_BnbXG~-;$VOC zizx&1HE9gg74s2(3TSLNBR;(K#NKrB|Ez!de>i&!s4BOpZS+u*(t>oCbSNR+N(m@R zZ&4aVQaYqX2}MdmQcw_-Q~?R4B?LqfQ9=+?#|z(RUz^O^+dURe0g6}|<}d%;6teaIMTM=$(}HRo%I;ZXxH z%H@oVBanTFA^Oh^+=OmmI+0sQ9`XuEH#0!3jTtHu*GG;|ewu{j=S@UlHsc%y;`zysRb?U`DDh@w;&MkVZ{$Tu$0x z4fsBxXL91o7(WoL?n1e#8--=|neI{b5LscP9;kZEMg4*D6yifa1uoSm9sedA!n%>G z6~KP>gSMfz5XiqR(qmFt6teUJ7e}+!^@0SPz01@9Fd}EGJfp98J1+3H-#vWuM^FO2 zAm{U)KPPA--38j6+fqkwLX^3B`9YS?+q^|EwReFb+>J~k<%7*#2m;sE!EBjB|Kbx! ziKY-j*A|NnU@fj&9wkp+eT<4vQB)Wq9sR7SEfcb~iQOLuvgD z@NCUC;bJ=d4|k_KRrFf8TA->|BM1tQF6KtFf>LzgX)BP+&LYo`n(C01W2g&@im|lHiQPj1K;Rz zaG5z1l({NUx{*ht$&S1TU{a6tX;hP@zo!!s1dOL2Gkg$05lz9C?P;+lVk7e+==Vna z=fx5Fm`|?aq>z^1J>r7_VC3CI?83R2$Lihwy8X5$0XVuzFKI*i|Jc@3#@eOidk4SeoE4HIPdqT(!!p>rPyGE*^-9xYEA3rgOR1x&i5 zPD%0EYFj{IpH04Rc6F+LYv$gS3)QJb;eH7q73BSgU2L&#iu_j~r~;Yk?m|<*$80mk zHk%*Gl79X8cs{2uO}zE+D#VA*N|NfkNOO4=V%Xus;do?Wt#7hUjy!`0vQ$5}%$ZUm zGFudT5fDb0gpe%hMHm&d`6Kr4or}v52wj-=>^@&cbZw}==BMdd^F5hy3jd>jm?H`p zYkVsB?SV<>PjwhF`|OO@Al5&L)r7bPbbt3D^XpX|Y7|AIUE1wo{E(V^d5AM9&gKYnYt3X7P`*!!04M+8@qEov zC*?{|G?cpqwbB;agwOxIe)jV$f#0_D8dMeCNoHQN#b3`o%zd$REvjSnw{q2%>~X@u z=F=Xfe`(2+nIfHS|K)gMiscW$4-c|*wJc~Lq+tDCyp=I(;L0(L^q0DGcVk@9Y;qPJ?(+Ts-P!BMe?FO)J#ySj1gUy~WP<}WT$Bzqq&kSs4QcbZKtR3* zC~-fxqs;cLJOh~+_4WAq;V&1;;%hfBtKO?8?yIZu4|Ew~QXE<S?CQmHuvaf8 zCAc*n-EKSC>pWDP`vB13Iq%)3o^(k9x0xsI zs7^J0`4g|m6uim)Pcqio+Q6jb98pVDzKXw;0RqgMT#NY9A6+B?PGmK{eDrUyyY~GG zfZ#el7||0k4DxO>3RDK93| z!!5ywHs4$TxjEAAZX|kH<|Ay%%iY4YJy-9JAG|QoTlVV~Dg*X%M|eXClt}4CXans% zz(aW(=kGM~I7@)&4=SNxf53Z%mv(&>C_DA-2c8h)9nP>ZVj=5Bi;AED$iRaZ6(zgOpvF^9>$Gwd$eh%8o6vNq)B>? zO0&l4W0y3MANi^06*ENbSz0v|W1pY2F<~I-NjrWtZlAUQ37vfm^^=GTPu@d@=%$|W zll%W}2*?Eg%K1EfqjHO+QKIUbp%mmeG}PYQ+@BF`cENuf%*vFT^rzSj;P-yGm#$*R z1&~!rYzoTsejDcL5%Bd*#e;7`?2FxRD(E=x&W8JW_}pW;evE#QBdi|I?aZy4R0mJL z&wem*DyQ4kPq8D8ieHn>nS*D?Ac(i@m*LL^-RZ!Icv5q$WD=M)XFV&Q?*9QQEF}Va zP$QmIIEW)ae5>ZA1MUx{1>T^^n01MGvOQ6(~^)9Ayr|j_O={q++ za%NY<#Mr-hGPM8T&I3@zo#?3JH<-ZZud6=V{WlHd9y*up!<;$27Soln)Co5L=UDlz z11U>jck1RpERavN>TBFMI`CF0Uyk1Tdy}C-3y9%e!RMtXgGXazB#s=sa4|osYs4+~ z-SPamqNjMom9niCcp{E{@!|)2lwJE^H9J(rg{dpRJ9JdLg-zV38D_$2v^`tD;B&Ca_Rq?x=Xlg-=>dm-ocEC` zb}1>H(I%rGzsEPr0v)ZNG&0#(`NAfXW{(4e4XkzI(Su=$t6xZZ52(`ru#~vjLExx! zWVWO5!J@i6!i$ph^!H0%I(TR8e~A{SPx-3i9)U$PRSCVhzr8_&V}DRM&Gr(_@L;kO zK4ZvyqCrFwzelLLY_}y33(qWm@;ACH|#AZa3gb=6wiyur7v=d zmYH*yRTnnhGzqAXI+#u!^kpmwj z`uDHG05mda_cEWktDT)YQ4;1k(@>oK;EdBIAKcy!0_ifu)5@?$gxX~Lr$rf;3Q;y) z$7LA?wx3)O2a*`WVAQ22`$w1>0lxP~^lIR8q$>LDv0%Hux4Q_|oX$;7sU*XzYL*4eHB*2d_(nT!Vc?Wb@gulEkJx`133J*mH(_ ztbLZ~fWyfYPDJ85NW&mws*RF51`V=Wa@o&~wTCe^-0Z4hoiG007J2BH!18J4%L!=+ zz-AZv`pzDUKMDoW9@y;U!S;_cLsY@Z*o?hVhiA71W4_LJ2726lp|QQOzl@FlT>Za< z4YRWmapUEfv`JrZ!1{1cF3$Y(&jel{&_LYfL4Gu13}!u(Cvf-%;GGnKdD(x zJ!bik{p;H!l=$kj4c9R$q9;nvYc%BHo+$8DNzP-BxsFSMYw`;BMxN1G7j_mE0$TjS z4y~kP4Yhw}(mn0uTJ`6KuC@#srs4tcj3;N>6?T0rwbrk38>xT{eYxL@E9_=96&;1~LhCtx%NqgOET;2!ojE-SG9T?2G%{x@ ziiKGFDkO==?JN}&6t%~F%5*kherlh@hRZJ2;F^8<^RMQ;5BD^7V?Hh zZ!&SV?q1#cv0;~_#9V%VLvA`@_z%_NkmX-T33^L!`N7i8;qj?;2g_O6Ob^X( zV&40D;@OWIgOgXT-6y0GEh@dfymHw)%WseO4vh&vkHzlQ%=BN2ieg$5zL8TmcY2)p zJz{p;sO6DC1R2Py`J^laMfaZgh-MIlb$0%?O1#X#o%kGoGgJFhqir|XuANg&h*Y^| zZ~gWddLjFFJTW}*d{gl zD++1L$A2%5Zcc9ABS?VfW)d&SPV_`RK@a-EhZ~Q(`lr06vrL!8O|3 za7c9b)`^8-wA!xNm22{neD{;UtlWw@ZbW!sMym8}6KUlIkm?FeAo5+$Tr#}lK!W!1 zW=1HJWts%;YLd96*X&VuM}HE|N|n3Y9tLqQ^k`qg>Aa1|gVfhp^#Ew2eH!~Fyt@J{ zS9Iew6e*K6E$l{|UGMEht4f~KuGhy^1P^9zl|0W6#w(x&Q{uIT;k}_-B^obnU(a)? zk;(asrYOf0_C0CF)<4C0bJ5xI&Y5?mGYIDGwdbW&w{F>6;{i-2- zpk7AxwQcdWr*n6NB|fla@EYT4qS0q7OZ5?dmX11vO}0}1XPq|HS0{;R6ZNe+fx~@k z@@*We4Xl_i?tnZWwVSXo*3X{i`=UwDTi8UlW@X32j0bF0#%b%PS4x!AB;euP^k6in z=S9uH6-?a0m#0a~R~+T07(2`HQ=K!&9U2$X{lbXYns5#f>GHWd&bapeJjvC4A{eBH zY)n(8)@@$j=(J6>obeU@z&X$cg^08Ff{_k*@}mmG1<> z)?&z>AgUq);^Fw{cf$vJAZUcuorWS|7l<-t+wOvRT?Z&~uB`*M{#5v(Uh|7(rj?pd z`+GQ;1yFS(2;_yjrz5`+sc&b-JU&~r?-6v|2o!Y&W;~F zbZzvvgxJf$F+F3=#}?1lF2aQY#nbB{!^P^c)E*EArWG{qiky3x{C(`1G13?ZM9`bd z3m~Mz=J)$Yt@jQHZV5q+w)^G}(ez5$A z&=cWxATRTR1rmG>2;#6IOaA36#D?|ECw+9=1;YfO0AziJF}sdWnW8Abrw0p|xI$F- zHd6m|wcqbjg#i7zh8uCOByq~~moZVR%67Ii_1Q_&=UMJ6oOxtY*7dOX{wB(KqULL5 z(B|>b>nZia&&)tVv`Rl)-ci`PaV!JEb^S||?kj+zFhlJ)s{PlUf)4K)xAYj{wp|}_ zfh@95;FspzKfk^GPWPBXTcBD|0mG67P&BK%(hiGusm%K2cw>AxbgDCpBsuc*bgGCe zRL6VeK>4v0^t1)}^}cjM-m4U;%ow%43>(;%5wtp8Kfk>?+gJWE{1qlFfjZYmwwT`C zyvAj~J&d&n>dR&jgja+7rW+vfI(lh*ukLKRquzaThtu#6RWaET`nwyFPbM$DJB6o^ zRa%%Q@@K=DlfspA_ z?wE6!QSG;1vUfn|v<##mFJR_q6AyV$C~E#izvNCLXi{2n^<{bhK|3ZlqSkojejRWl zj2^Vsex&U5Y;pb2v+rvrrh{pV@rz2oe($=(kx5R@xAf6eF-L#KpTH;Sx$0L*t()s7 zR$4m4I8QIM!f%kh4!FQW4IOc`0(s?vqA9y4S}S-Pp`<96Q?iP5LFvxgb|pXQehrEK z5YmtsYBGgK&*FU;$+QGTb59J!FdOqWsu45+spIkLIOK&*0BF0myfo3OcF#dTw7?rq z2H^)^fcrXzIZ@y9S0#|ePkchvf{$}EF&K2hcwel85b3-O*ZZ4I2+UL#vNOfD&A6gW zq{g3q^-B}Z0w4*hPp1euJ_y6?!ccXoA3+6XJ0pY7sXdo0`8+7qve@JEb`kOoW(RJq zn1+*h)1mDS;xnv$|6QzXo>h3w=Gsso;# zZj`Xmi6hnZqlda!hnC|#*l%$3Pz@y$u@H)X zth6@>ddbkM?TkMIhu0>{f4*cpxV@pQDE@*IV`y*O)1-HSnd)};>(u3Z6+T62=g?f~ zwpZK_E-?EZ0-4B(VEd#-nVZ@Xd^D_`+G_WINxTT0v5ieVX4Mgv1M|0V%&l(X&>y2n z`gZ&(96t^@Xd+phe=Ct0V4tVxXx(0sat@E_XYJh(d|Wa3@JO#5`8XF1yVDubCBHmju?RoSxc~og(%wk?au*nwu5uD9(j3?D1uMbZl1=Tof7NMx~ z))79nG9$P|5llR#$;1;uX^Ai-GM&?qbrJ04UIh|`b zB6HIXpCzhiB0Xw^_GXV44w>_bK6G_h(>W??vnuY%a*L3VwY3R@AI@<5I4+In zQM_>DKqNV)!=(BezlUCG1OIVbl-5hfS1u zOExeZAr7CMlJ4k}CX53+&Sk>I{w!ge;sZC|xt>6U=~zw7A^%fJ^y9ii;+UqtGMx;# zdO650O3yGd8WzwL?`d#}y;&o;^9!Js@766rGFPsPNVoFNU-Gd_KPTB}is?%+r1GV9 zJkOvN!vKZ(L|KIN`#|=7N(u^@>&x;scm2O9ae7@R8e|uKH3{bDa&FE9)$t(5`VW*Z zikVi=>xEvUao;?e^5yy}G#+^Gqt?Oo7@*X0g7vp~*%~>78t77s`7>rC9BGjA2@dcW z%|FifFT1@g$qag}Vhq`A64{!C=KQ*pXjf@e$HXZK$DXVCj)*h5UOkYnwZPBE5lgDS zoJ7(CM$*&{hoatpDeoL$Ek^ETHAV3C`#w#i$QBP2TcmJp3G7qmAWg!35|Qg+5v>e^ zvF*Rk+lsroC6++bLWcWMh*8{SDT^k|apw2M-eO`(JI%kE$%*89M2~au;SR>|#;D?R ziB0^_2>qg&wZz!!95>aQB{_4b_4f@=F0ot1TA71{-zk?UNRq>R(!4gcNQ*9~QJ!MF z+FV7JZ1~7TcmZ0m)GRU(>BeA;gYS==IEy({Bsr({a)L!etX{Qzig~1-2)x$%;sjgp zN)wfzqY1fVg2}5?i{y&xrHZD&=kLXf9^^Wb%Kx#Ry^=ajJ6wZRoBrD$m3oG@9tlUzsA>{z*Hcr*T5r2gKik5RQq+pt}CtuHTf2~<0K$zc~zlKop8+InJNBc^QM zwoFB@g;~7;Rf`CJv0K+D=G5?Jh&Q0TO(CAY6G=)x&wU{=+_UG16vrbC#avch$&q2Q zaHg=>5(1(Nw((nWl%X?hbavwLB_qIygC@!sgQ^tPXr8w)sZAU`A#KbiX4&dXt9SD0 zX}o44cVclie%&{Tb;}sWgrd>lPTG=L`xgB`-Jm6-$-vPn$Mo;y-?eBbxg{E?Q~7gn zX=sn4Y!N`>z$xca>_|I#RB2Gs$zydgG%e!l&1ss>GtV6>{hFg_bT!W$O*pcWp}~6L zvd8AqMa#@PiLVGA_;QKib$c;#N{{RKN>F`Kks#H?B`9KOqsW{^j5enyp%;zGvIv83 zrXE%$$6=L=5r{cY)0257*;SRhUe`>>OV!^c?$Y(q1mpI|$DGlLC&tgZ3`0`7Bkr|Y zOtd|@r!n3GHdHN_J3JH4nRa60BR(~+hR@$x0J_4;vUXQWOf}*;F8R7pv5$W#(RdC} zLf!UA5?yu1Y)a4hkx(zW9nrHG8~VSJBe$M~ng4ZBUv$QWGgR33lEVjIf{W2}ZarDI zxDU_FfCt$_>Bh*$9UZ9^TAFn`(cD(ECzq(^fZh zhNfm@shrsQ_Kz1Qih<<(#kk7feoU72b60kV@)zWgyG1W)rnBtvnlH2I)oE9vQw6Q> zq%I8Hbj4h~64h>4ekZZ#n4%GMSgf3IZyi(P8a*RTcRB4Uv;%es*xl+2nZNsG(@=|z~v-r2{1R|YMkdkno$c&-Q&;Wzx$@YI;yW}=Ij3?Z(L|d z#iF=Loq0w!?+Vvbz-HLTaa0~-+L;(uXKTuj2^Rmz*?i}wN5v$YD&4Bg8#?7&T$(xd zB{AFkpcyB}_BMmSTpyQ4m0@cM3BEeo@rK2czBeMCJHs0xFponhX2KM+@JHm$Ri%g( zAkD9n6LjEl+?qSQc{%ma_!-Gp@v_;QgmLMrVNVfE&y{cE@@rYev_qj~RX6Qffe5Mk zKI8YHaWfE?TQy2*`+fr>E#F2c>!<(fy%gPkc6><*g|3seyVlb*j|nGAZnl}volWtZ zi=s*EAnc=mbu0aRjBB{+uiG~xn_jhjr?j-AqL}{y;^@H>t*kWc?Cc%=tiB^XV#4a? zCe1PD0JIi5{Jz6^W>qDoNyn*%MKwd)k3-l+TCKW+Fo}?YF2%0XXqOd^lEWe0eO)qdgS#7n(X<6^%vY<&Mf=vSlA!)sakQb6pIUfNeI0$B`4 z`g#BI15R_V3!IWxj`zo3E+6jPp}Fc5oL8LdVyMdZxIY;)phQqI(-B#A-tSA5>uHw= zJ^G({j4SlVYefeQ?4!DAeli9d8GdS^5;tzqdvI=2tef-h)!`a}ftfT-VXJ}CWCvQF zI_AMN?W!Y6PfCh#()jycUM#21$di8*Z!36F$Z6B!*xBMQzAQ;e1%}3?UID{PFE7c~ zMGw8xetNO>P;*d$$z+@vCHLv~B)hz7c<=8oHi=Vlr03AYet9j2P7M146`hGyfMNM7jw87TAel|EmRjyq^>SY;!mhuG}8*3dm)mmg0dm{O_E`~q;|2xCw0`gI_2Lg*XE*dT(fFaw?nEkJ=;;d@kUc((DtT$kvPt|arC4z^INbbV zO6gD39=jRJtr*Fl?xc^_VmBotE)c^}x#FKh%rVV7stMX#0DQE5f3%?gPxZUgH9BS?a+@xULNVA23R z9}M}U9S2uK3sgSI@UY}QZT3UxmS}BrP42Drgu?oEj0Pl&rV}BnMUEqJ5+xqgQSJ^B zRhp*d_MKakOJs$!1X(ifoP&MDbV3)-K+q+?b2@pczR`iyqQ)h?CSNsjbMKX7RgFYY zFVT{0l4NiX*56xZNAJ-$$}w_ciZjR8OZn-LJ49}CA8X6=pB;A^N<VlJbedD<5n0NRAxi*0@32$;%_kcM_KRb_1a!YWwn(3Va6v zu{&j{G?&Y%(tvPw2LNAk9XF6z`fVPaX*kNmsu193-R3i3l?5o`4==#Iqyp&Y)Igp_ zp>i0m$J#f;50&-_8~!bL&m0GNo28@bR=;-J&whLv@{(EBLmd#uJkjJVZ|B6MtQ!OU zI`0amW|7-ShwSbcm0l2MiGGg3>jlERFy(!Iyqx~4-`|T{io6Bkk2zd(01xqjggf~| zE-AJ?8nn&C$4mub+}qT!%gBS=!o3`g1%7gGO~#yDi4i2@6PpFz9%``e^{rop!kB_p z{sL(`^b{Kr2|;r2fZ*-~ti!R{Owag@K$s+xxM9GUcSB*t|CuKU>F{oD`=BR0-`Pw)Voi1t~{uuD{Q5&nBGX zt?vtm`NXtt9_4`aN)mvNPOr?o5rlLine*qs)i;pJJJJy20PW$-LRDipl`*~Pl3G9r z=ej`UPh7p2A!iuQx^X$W#lD*p1Wr=@J?UF9GW3BiXvXn$Y-!ADp&# zEwtl$LUh{zXsk{wY#MS1`lY1E7dXNrg;jr(pt?#6LXLYp=cV7sC#MP2wo}30BO7X+ zyxCN0{2*Q!8nyj_IqHB+LIG?$)CyppF`p_8P=YU{9QvDTXtTA*?q2&bdafP1vTMj6 zwbBa3La?PA->RF{UMZW(#OJNg0_Mh(*+^A+kVpCG0a(-10F}H9I4&gG?vNl*pTY|= z*r#TCvvi;&i$Mzg*J zZcX@Gps~G!Mo8afzUmFleUI5=;a73I#ileK{}ZyT{m-axgJe+t_XU|VYBSGEk;lCW zc-}tM#{MP9zb<$=5h<)u_XR1#?qc)8)kC{W5Ox}S&)0TEVMy;tE+U&!5T zI=gx_oFp_R{JpVY{SRNF9@s%|6x+kt2xfp)TAg@S?T<>05$4du(kJ2^n#KicMnp@V zL`JBcV?|vyKz8je|H_9E=co>~S^=|{!3~%sgD>;nQKaP>Cg%2*OySZ?-BLyhOO5Of zjukkSpY>!UfxLC&4qybCIi-~YH{Q%f^vOlcpim|h4@WC(>qF+t>4$N3 z9qnP~M>TDE#BlIedCxz#5O5;0+_RX1OgIl5hFkxm^;$4>+6DZGtvrB9z7rf;zJpk7 z&hLai^;u~A?E9F7JDBrF^%2|yMyZej{KhY4sW-D0Drn8js(jOsY6744gn0X8Iv~&y zceF@avwda(a*F-$Frr;R|Imhv=!{?(@l*xJZ`f9k&bLf4f8@C+s_VjBK6Cw$>vsGC zSu9`Etor2dP&#;x0($J-BiZmXsjh}DTV=;4Vaq$K&V<$0ma>17hi#{;U$DHM6U$s< z*a39?U`UWEx}}Wnh1%n;MqZQd$iKFf8ff&}=iU?)BdDWZ-`zzxw+ zP$1r_91hgqth9DGg04DzYB=IK5VDnlaA+b;f^phk_;I7#nnNbKMl~*`f4WsSJ9Q3) zK(T+#{$)R^WDW740D(kT`qB?8%-6V9(i=kIF!aB2Xi7f4bOAPd=pGRqO1P{Sw9H`J zP$J6M(E~QSIQy6MCgz=uLvud#^joj)CtN_hG;)LsqGX4p0{kQSz&A)<_fN?bCGR`) zl$~1J6ug51myxFzk>jM^WQYkCUliRMiQM7Ip`Eus5s7bv{z{h@4--`36S@C$TWbDa zxd-8Gk5hl{ZE2C|l38S zEcN7Pp0iz#Eni@HAz(XnpgwWC(b7eE61fI48>Q*!%bdu?d71gr_;KC+%LTT+De>r4 zxAWMEy1^D%ccqGo0u}uZs7$G#_^Au(DLmJHjx|;!X>zln3%tC~0fbbQJ^syxi1sfd zUu`eh9a?^oL`i7fVk{*13b)!2vNk1GU;D5K&t94BLKO`m$r87W8s)mFuF4=j|M^{k2qAOLiK?&hzN5Kq*N5%n7v>G?rb(%cQUxx8qPXtX`p!7!pRSw}hp*?eT#LWh1bdR}D;u``4jJ zW>=ECdSAv9F%Yusf#|{8H4p5xbI|v90Rl(4$MCrnMyuLnAg|@`KyrAHQO>JaF$cJA zB2&Ort?51rhZlhT$C1dH4>QH>R5&u7AFowq!T}B|9?Y4KW4& z8e$aCiCFN10j;?e@cl`9)p-v{fB0m|z6a+Ddg91>ygqUs(FfN1H-SCz6`1_Dl|q95 z94@~MIl*ZlQ%HPRQ|0tM0XStm1~E+NO*SQ~6wvd{y99_$ab+^Wpf5 z-KqTS0AMlYmH$Kl71&)Bl8l2BoBIR4M(`&0EUT_5;7d`WKRzlLN?&>PDS7p`P*-Bq z(P|8um6%~{tiHk$EvA*LMv_PbY`7>0C0tyt9U3!;8sY|H>9iqe4p3om>*0C652go-5LPn*qwJ*PLJ{qMo3b7evQW**#>6(F(O^}jKfPk_hQpY zJD;7?Z%E2?cj_25y58pxzQ^}Nkj);G14>9Ks2=pjlgeMHz=W*>W$pqvhn4xm6}8#M zCF+rdCK2XN1kVej#(ylx$oQ`n7e>AH&NRz$IIsjX8?ima5jBEI{+*8F1eqx!Tr$bLk3jC(eOJeU(s%54A0(+29NB_uQ{b z@>|Hj1_o)yA(X;BxamPReJn_S;S~E1KwlJDrdBqC-67^hSI|=1-l@I-!S)7O@QGMt z1-ZitxL`uwdNUk`0=c%w-+nnW^2`0+>EL#%LHs&BCujy5=%mg#FaXy`ex1M$I z1-X&dWa97Wz)k)#NUa;}iP4>XKjYKQ>VU9hQF2lKr(_6;0hgH1-cxTF#V!}@0wc_J zIh!LW$|xU&SnuMOC2DjfzX}C6pmJ|vp9?r>bje(9z2fM8Wx39HB97~4hr%OD zrkp8zj$yY*9SHCg1XBr;l=o#zLllZ{ql0cW+O#WeqYVy>*34FCZn`mmc3~o7 zP9aqjbj5+G(E$Z3&&#iXGg=aIX$Q7ypP_6|W(GpQya~2N&ejfk5+GpQLbk`B!re0e z87F#V1*sLP4w8nU=FFx*(V=_P7_E#qg%!i@Fa;E?as~~=c8Y(t_f9STo>Yp>7LA{N z9=>e;`Q8cF+t5|JV`pm>v8R!Ai|~oo>S?eDD_~%|0G?h5F8$j)4N?J%x(X1U(m{8C zWgEnK1z?>$;4)sgcrCdWEO@$N&~q1vivdQ#3#|ZBpj|uuIf9dRec8egCMUsTI#J83 zt<~97P=fh5hng96%Leb%iJ%TV8^>8UbHIA^E=yj;(rd`I$d*eRg*|io ztcaRpSEac?u?Tg41hy+3gMxn*-EBNV^5oZX^xbfz3PBk2`o$G=j*vLy+)&9K!~lhM z>3Odmfg5fMobj4vxBBHMGSG*BR=MS~VIpAjprsX0vHiFHT)9tR^9!L(p$Fg4?n~=*K zlC;#Xg_i4C`hJpSz#wmV4!u2p1fuW^+1TIAgx5ZKrw%{!v_Glu$Kee)fQw)gO#q-$ z21oLPBS^wHmTcYkQT%$mI$60I6HW|rHZNViJ9`P_JjIfcXX*R;`MN|}U zKMVNL$#IVvdvjS|l)0ICd*!-~O;?Op4D&kycY*Tyl^e_o{%8B1xO{qE%`aNyS~tka zR(fU;vU=Gk(@$QzA^?yIOtu}oeIW=?KCg!aLN}yK)^>D^--F6xS@ihbIxMBQ@%PV@ zo&w(jAn5e}rpSMruMsZ42+95fLH$~`M=LJ2qGq%H?i6%eV~`p!!^FeI?!WO}0I(`aD_^@8 zsy40fH$@zMJ--G=^h4tna^NKLLDov_!On&_ocF*nSt$Uev-1!GNF{h>#^YPJK?7Bm za1wb^0+V#F5$TodB<++nW*wK>CoG87elxEMLQ^G>MXi}4XOSzI)z9EqK|!cg81@&p zbwr*-cL1H;FKY%YMMZaW^F(WOc(3228*IeQ7a^Y8alKqAPTqBpIup%m6CyWKoA)1|0X~&I7zw=*CA*%ZI%N{ILv*Kn zWaXEn;njCODODj6hjr|oqc)99-4e0qw{FrKu>xH1Ug!p1`tZ#2!MHwAfThiadT?Pz zeZOD(03Em*BKwUE6nLBno(VyQAc}(RaY#IMfVE9R4>%rke^genM{tL+ROs5IxJP`+dwp3=gfx2yK=Sz?(ZuaAOYu}%{Su+{&7QIjVQ_K za+8WUGrh<3qy&~+HQ<E<1RFgN{Xh+0fKZWN%wBX{SdWP=A>a`X)9qWzj~D<02ukykJ}dDW|5-|B2kB{;wjpAk96FKR?SE@n~A+#1=y*yLCC&e*lGG3EJgZ41d#yzD4 z_kY2H1S2S>CQkonm_EMlVz9Flm3{D$Qn4kq6;dFB6m$&o0R+S5s0iuqsN<8f%JXlY z%gT&^duy8ew2z-bS}95+k70KrfpddfZ*(5-~Y*|1#HZS6A;VL$bj{QBKYeqo3!_^ z_0Qd)PuMfah!T%QT6yS){p)_fFK~^C+ddplt03r8-FUwVKUY!I z_rY`8jA03L{F;dvJ#ig3Wl4w7+2X{jU+Z)_dLX@z9lPNH`ql7VH+akpIAp164svfg zK}!22W>|)P-8bqU0g@F!Gp|b|L(;RQw0M`aSA^~ki77v^y6Ru%pv~^1ThS#Cxue_C zZdT2iI=-R|2SbMF9ScUq`Z-f0y6=+7@iT*miCZUW)8_hryMV3_JUBFG|HXpmRYq+f zT&((+??8PTMX#@7s#khiA$si}*In%r5F6Y%gjq#Zp-}ulbd;fnuwA4f#qA2`u*h|! za$^781e8$*}KS*-HJ8?<)H`^8gOp&Awf}V=^g3|ioIG$C zf3knreT4$xK(>C|zn4Y_M)f=$8Z-Yj!-c~K9;yD>qx;Y~W!zj9VR3_Qu8h+py?G%MAHbWwkSlEpNk9J)KP5w`GZzE9#``9 z$+S27L>_I%gWc?CrmJS0g0eV1!c6XYk?wu?EIYIUgM+2C^W31f z%vh@Cc%|3>D1*UX50m*Ph~AK?xpS@%2jIg}-!7ln??J*Yu)nuX4OGwBa}(HYugjWa z#hAIF>lMJmue84F_yQcWslF@v6p#4D2J>@1fG>v*(cXa_-gP@Btm};yV!YL5-#jQE zL+4$oY>aOf+HB^W{ptI%_7?8oqJsMGgBgqbcg9FB*GMYDj7_zsSMTo+WW=yD_Puz; zUloETUi_1Se|LP+MJt&=ZYeD4RgzQ!cx&Z)_(WTr)B=b>p@D@hVa+0i-KLZ8U3{DW zHJz_+Kr~{p^D)@b*DmS8aHq#9c5@i}2Tq&6H((4Y+MTcm8JrVzNKH?TLe)<9-eWtp zgA=8{A5Zu_D0<`v$8>bn1Bjrk7&bm<;ApHS;)7rju4H4q z>Het!o~_3I!0gW9Or2u4LTjOsv3{@?p})}X?|Vei@!>d0fHjj2cNh_DLQSF@o%?Hv zlqg}fprmY9s?g?oY1t7qnt3Ca`DTB~mV{wGp$J<0DSbT35L0w*uM#ER;Aazd{u=8k~xs#a7&y%@XumLkyhs+kELD(kggS&k4Rb%&Ng zgGWx-p|Ujk>HB9PU(A+W40F_j*D8CFbL-%`rrMt@Ef#I#U7*HoNt3eveZO!LgRMA8 zZpPF4Vl_38`a}Ahl{A6f^UpXgP(wxcgf-S)v+BN-it#BBg>%04BLGqYXqnizh~aXw z9$`(C8-^77)(dGrbKu6+2WfThZ%(7Qe@`Uc=^EYv*abA7@Jg}z-tYv8eLGPf5C}VO zF!hsa{=rF3MAd>^`;gSsfXt*Qq7Ro}!g*QKG3Y19zHgY<0g`8sh5KWUFC4YlMLe^= z{emlKRCgE1PnR8w982j8HS3^5V5&2Ww`k2SZHbY|y`DcxC5*HOk_w+tnoDl!R+J#6!IZ3>>ZPzfn!4wZ= zXeO3eE|d$r=8mfwZ`uj69CBWEP0w=>`m5wE1ehx zoId;+JiL%p!vt`9DCE;#(x1UKj_jwW#5Xp<5uNN`3!DA$RpVLM99|z+CsP`0I}yLA z;Bx4KZ~7sUrvB~9N77Qjpz`rE48d$StzPv@(vQm)T8Vac!&os#zss>~nDDhtl^**; zb;^n_;9`jW=sio|C`Bqvd?z0n*Iwb&bJ$B;{XVZ@g?~qJN@z-+x!eKRRF1{_tz;$- zoP)OZv7TyWm^`A|@jZ*wlx-8YW4vItrwJh|SKMe|WIqT|vL~Tak9Gv47jU!L&v?Kk)EO zNFeU2{pa!0)o5o?U;8DGUZ7ZgD}U){3;z=$xk`fmU1#0$i7lyGdVBZq6}PpmJIlL7 zBQM!0f9?`pal5)F_vcT-3wv&bw3(VotkK;}hU2#LV~@Nm{VwV2Pt!nqD3Lcxh&{!& z`KCT(P%OLs<)!NhEl*Utu5P7Er`gJwXfwpquO6!OveBTh6XVK8XG^*C-y=zXt9uEv zt3Wu|^UUMe$+yp};{7JL4_{5@Cp7PuT{v76HPKz_8_BexE-+*TGI$6ID zUk_gkQNK;U;rGcZnLhq}-9}6HRUg&v&#}=3c?|)`2D0A;6)NM53bL=jK2RPD&~Bd68G^v zL$RM{v8jUp&%osW7(a>e5X?+^>v%098dyk!Ix0STASeF?8`kVEqq!Q4ljpC8rZkR|MDxTicP;~3{)oBZq ze)p3W`U2Iw4tYwOC0fmLj5jMQ3H6&hg)ktF&u>J*RZ~Fp&pq?&`b8={Li5d&tUZN2r@0x!=D0=P)W7)P4kY?_SpU-uma3i+`CGxM3EJ%iQVMJLyMEY+*hN8mpsykE$ zSAbV?s8GiPy(0QiefqtNZcIT6W8XN}lGR*7c(?%>jZ|P>q-DTB%K{&>yzLI&>TF!Id0g9l$T1c98 z)@^kocKN%;WP0exfUnpmU-pUho>RemR~T3|-X${V-4%zzA3)ZG^T0k+0{=ZajANqz28paPlW z=o-?m1W~{|(`(QFLlnlUyJV!c{SKfIc$N4pt%I>rPK>H&mnkH`U=nX_5!}_ ziH8D*-$6dRl!i|q-%h9N(t5(tPlWSqQXfY5KoZSNJKC+5caZ#OB{iwlY#O=p97NAyqvz;=lHdD)}+cXHr%eKg{|39CNe;y*wXTCy~O!8KZem zN%bpyCWHX&*sh0ky^kqnE+u=|FI5v<8gH+KcY3pbKnk+#(b;kRpm87}23^@DWHPLw z%GShyaCw=Ecdgs!aXBdIbrx+QRaLW6E+xhDmfm-5Bc`Cp;B5g}?H}?oO|%3B8f+X# z!-y4ulUB7*53*jRFuZS4o!d$=8m+vU9j}hxfne7(;GPed$pWr@d&Ns)b-$R%K0IA( zP7^c3?Iv@N+T9h(*eQ&3??ivV(m0Cdnbl1?++aL0GACwT8M3$gE}TNKIFdYK1WIV$ ztXyISvR`S1$Fluq7#;z-PVO^w({fAw2#Vq?gh_|QNs>~b2FyZ#3y2MiJ(7JxTkMmX z39PQ9Pf*Gnl-&otvEbtYTqerABWXvDo$Z%LGH8MT&wE}9lHK!9*)^N_46^0lf!ef# zUZ%`|QR|wV9%OaAG8_naWlH97O+W}$_;^)$_j71T?q%A!qr}C${DWHp?={ccriHl^ zKsLQBoKVQ{5Fs@Qts@=LnKiN2`aN5P@(URUxboweE546F$wa&>eMP-Cubo15aP#t2{VR6OhRlkwZUaW*9(hE)O7Q85<+EUYNUjqjeLjLKu+D>q10+3AhsenDV#^KGwHM1i_uJ++OWH+` zD#9G)q`AU6{1hNlor+4FL5Dxo?LQvP**rgY_+X%x1ij?dpS&vtW}!!0u9h>9MA5IH z4q0zr?WokRfhLiQihK)Gbi%TF#)K^ws3I?^F;UDoaf-^{BBH!36PWXDWPo&!!P%OFO^QO=XE~dX=d+eVkSAQZiE*;>^xJ zRovV0G`_hTMWo3>*#pEk&Og9z=&J7%H} z$~11I*Mu^0`KH`^N}B3He9$`x$)Y-{8ihpP=xO>b#Me3*b_SZQIxay|4X9KaK(fov zz&b>9MV$YCQeTYqENn#BUiE_3J{k3&!-kG@tI81<>zGBiQ35mI3qbHh1(z;ubl+6R zh*;HMNc|c~xQb9RwhADpsIu`Hlpf{A$@rMU7@psd=O5hDKSMV<3O(5FcehnKN93AL z(|l{*nR}4qWhK-?e3e0}T5}gAbELOqD*&cRP1Yz50`0gQjnZ^-p5P$5v1& z{Sj;a^m43~knwXbN8!sA)k(w`g#W<({=xjKI;R*%)lm7DZ%->aUl8y8Sq$;Ekn$9} z^?-W(44D?!mya&YZtAg)_T0FGzRsv`)}@TXjjk%c2ol}(M`}KDT>7`J(WyW>!r8jGDe5)zOIL3NP{P;TeZ|QCF)~U_3 z;I)T(SF&lWhTnJ~_X*HVT#)$Rd!M}gG@+De(dg^+Yn)t$`Lc+9DJ$K@(lvuqM;nhn zs(8Z5pw=b*$?MLK1{dgKCM-X!`OR!ec@1-c4loU$t}0$^!T(gOjn7NHF+Z#`+j>=e z-uPb3M_Vl}u>@vWo8eMZ<5{#`)p{2c$nS|M(X6sl0r8KYX8hdy4d|QI-7yD@T|M#VgqKrah?~&QEN0E_~ks_mv zqC!YEDXUUOR<;nb_YB`sWSt^OB;!yv$2{hJy${v*_q)G)ANT(LxwpsZJUpV~^Lc;X z<29b6#6_G%W4#@}-AT55dFw5jPl=qs0T}KZ%Co)wD({0gjwHw}0NumgsBiEOI zGiRsx9eExm0;t8G90TN1zFIRb|d6_zF zflTDB5v>XY4FOX!#@FQcb{SVAL}b_+c~+N8X}q-7dd$j*wBn;p zGOTq@2pei^#cON1d5>0fL?}M7qGXjjq~837n*P`P=pMD$BhA zHzju|sY3@xRZWc0h;$}o;9D>UA_Q|l0v=ycFK%MpVhrZ+Dryh_e8EJ%4SBPQLRsd` zmN#p_c2r*Yb~Y!hbQ!dx{znu7ZgLLqFz7%lZI!o!PoV0aU$#n$c9+gTuLi>0#L|W; zv{A#Ij8p`Ypu|e@_dHv?$NX+bSL-*SC`r5i_W!zipcuMr>r%4%{DN*4{p^9)^ zu{1j$v>8pVJ;Jqw@=yd2EZ}$pn-g@gC}A+m?YgNmoEbr*CoRSjM8EGMGYm*hfmTp&C#`YbWkr7df|M{mEe<|8YTl*@q4qnv5Oz~{6t+sxz^y*X z(Dt{!3?OjNO1(IJ+g$N1EQ@Rgyz8lzw zYIq-IYnNeJwmnAmu-0BdY+M?Ea=zIl-kLZ{R0#MAdm!15r2H2Yi8r+S3U5H@^b~r$ zY9SoTrh2603#6c*;x)G;8wNm?l~4yGJ*?HR1<{8OMGEZ@!;@MZ0O5U5C{RK+jY3Bt zgy(m`VME}M>Ckin83X{iMD78^Kxs<@w0Mkb2a!x8QjG?+OpS&2sLgXKrZkpV;Y9aEO2d$8=Qf08hQ za$4}I-?i><5w4+BSrQXssKfGjD1?` zQ(6f5=R(!FfO<&aFM&(1Ci3TnMyQD#9=mTs%(t`62b9C#_mn$~fZ31x!Wr+8!o5k7 z@bvjnh}89>GZ$yPA!SGUj$WlFM8yy&@0tvg;L74XeT1D6NShsJjL8%z(&T6A_5r!O zS@Ax}Gp^->y>0LmyG{-OebV751UR13tW{Yj{U|=N=s3%XWz}5%i)i94MJxu73>Luj zmQU2$eqE~1;kC({;tkk9wG%e8c#Zu6Xq%Q>aeZ7%VW_;0tlFZ)%;rAKE7Kd9(}x}c zlIqhBejg?x`9vpj=7zSzn2ysi0yq=|JkcgO%&amr*9IDpN&+os18XUal;uEkTy~=X zmXxcbgs9?W)7g0tdceGh09{5USJ8Y+vRO%1z{@6f4DFA)Q;Ns^shn#^A*kAN>0P3gNJR$GYhr%<8||Kt6UtKw<&ujAr(iSWI95tthGw1A>VhfUhwSo4=<+5T0u?ecatI5 zcyqZv(1(;oMYjz4OchX?G&We%xUvJzMG}uX4wwQEs1qIb-}a19v3?r8X4)^5s>_Z)6?sy|8RX)-#da9|zM6AB zGr|z`O+rDiD5|q*w>AQ|XLOg|1aZ8(@_);TRhNLI_1vrRZeA651ExZmQai^`5)b@f zEQrph1+vH**#BsqU_EK2ri~(@jEYC^P)xTCMi8uoh^2aS&y?XB1yi9(f_!s?_~=E9 z4;}AyPNs{3V_+_`tElM*glZnWim1K}&=hFh*^cBJ^2oU9^=zMcOYMXe%e(~#l6{v& zbxSX?2aFb)CT-b`6{ldc=51k3)BISbsN!N zurZ#u0%+iV0o?{o5cfJ{RG~}R!Ybo@t`q7-Ke{_zf_u;@{Dw92A?gY^J|ieCv9LJMUFQDgU0{_!u4M|2EjIAB%K2VK9Y!bGO=O#H2O zuQCdY6pH~Z+M<2(hoarEIn+a>FyJ|^5!R7*mNgTiGgOegZ3sUnbhOHbU*v_H3LfdL zABEh+Lk>uDM(@BY{br;XSdwSLpnZn==920IwbFm zM`1AuCSGp@y8sGUj>H6xy$U9aZvihC1!M$;Orf|>N2r2|kRxN*BZ=l+*}55_aY3Z- z)eyFO{5YWtQ-wAz5PF-iHMz9+UvnN7y#5W&12lc3*BbSpBDNHXBGng@O+(3w0Py&p zvruC(&DMMC%^q>bQQZukA5$j26+k}Vf(jfC5`16lI;gdp0I;`_6+^b21g-fQ0CFeH_9x8ry-P`ku&IaCM`8h{AilyKB)8A2qn zSGi6QP1^Um{IIgcGCTDo2};-v_$VAO&3q1V&$X9xz(eYMjswBDNC1`n9HWYmb)BPf zJ_cxf&OMArg1y$pOvJ(ANm}0T130C5uaJvWv0ZOXWr-F6)S_nc@SWXs>oQ=<=-N>Z z%^Ni|#wp{+7*U3C6&>rXuj*6H^8@iWX0U(JtH@Z@SLX;DRgftSqH2+66x6L>BA)>q zU`|J8&ynX9QxBV%9$<=BJxXg3@3Ek-pLf~&kU^2^Zn{J1#9WZ8`=ILWz3^cS=m7OT zOoui=fyc@ntB=opkAUw2i|5i6gbSx&k~qgE&aT}rYoTg`K+;YdA6+5A!y-if7O$;u z0P)(OlK_GBJ;_L_uxVq{{IeVDYur6NV+dgjg>dpoUn1h&4aD?@Y_OZOU!qB}ATS_Y z1DM?mPbiq?Y6LXsXi6#{XK^^%&;Irw2@uQvG(ry|V2}JrV+Ow*>|_`=H@Iuf&!mZn zAh78GOlg30+uc{Wo7XG`dofRg00@DbXAnZX z1sl}3I#?<6^uhhHcX1>pOm-@*yVcl65G&xe(W(2e^M}U%3d>0U%j~(_?YkYnao8jG zw#AamO=`6@u%oVZA1^}o0Z0G7n5ngRbiuCZ4qnMwpkC`L!VDoH%$Lsy=Ru__l>Y@$2DM-i@Az5Yz|xgn6sqk4 znHO?Dn|8!mG>12$|31`f(TxVc7Y#t=M%jKrfUc>9Yd$$I4eVmhe?gG&{z5?y!esm) z#-H3AE2#q-k=#NCMZu8GG{O0S1qrM+i}5e{C%=|h3xWn;K3yK4Eo}4R9?Q48N~~jm z2B^UZ&bvU55IJ?98O%l^U^7q!#X>CDNk#${<1}Elt)KoYegw>bl>j<)c+;Wqhk)x^ zB;rOT0u2oYP+~&8O605h*8C~Y87v&U;EYq!08Gc?U?>43f^6YMwABNQyW+$E@o+wk-bM|6A6WY?=?i4~&3_TaAi6Eo`RAU;ld?D~upOd~-8l?SaT&RL4j$9E z{y8gye(`b$2IM}oh=-SE22-zL;>EAEGVz0Wr{<##VkoF;Wobf|9yu{KoCrl1ZRj^T zG}^L)n#k+=xRdQ@!G*%?_=#JEXY08+~;Yf$x3DDK^Cr04&-2IN~;Y4&l)2Uli5Ys7La@4E$TU~_aQ&dSb*rPE>ypjyBn=U%oh zsXp%UKvw)(OLXcYM%Vas9HpDR9JtrHj&U}(>yo?Ai7k}ZF=^}^_eL8lz#mrcc^N*S z$+#z#$4zt6o~n<^#eQyIHh>9LnK!q1jr}x)`hwW}%m0osQMk^&?dBKbM1}3Ln0>QH zAhtHv>ZT_?pY)G}33vi4!p{k45jMa%)+X)KE@Pv|Pp^YQ&GaeLBr3Z3K6>|0A1VTw zB(?MK%RamIrNAMnE|WlC>w==j@(oK^3ky}5H& z$^l6F{x*Y5n^Z zf-+W6w+=a9ulxHS(H9`onp`X6j!oGw!M*;-T|B1#01A+{UfxQ7P}NGuVjUs7Mulv` ztIlsTbC0RMobl%)gK1EXzA?H~=q?ru2;YI6%bc*Lw_a?dggOP~)x9?_>;3vAC=5A* zq>_S|3bA4C8j@(@f#jRRl== z6iKN4Ln-Q7w1fE;s{xG~qKs{cuDGewSZu*j3Eyc!-`O8!<=LKx|NZe?R8L?8+FQ(9 zZ~4tI{`vSnso(o9ca(blx*o1RQUo!s+jSa;?H9o>CEd6Dg~^9&;|fKSj!zW0wY&xW zS%fCUuZMYT^9g*JBR;(eRz^?3I}G8ltH!-kaV>mk=OQeZ-*<_4?sG0UEF?Y(+z}hOYw+#W$d~^ zgTHfo2Zq8-@z0$Ca%@X$is#y_)uvC&<`cNf3V^ibB>KPKdI}Zcfn>@0M&GX`}O<0*l1w}JLTQOA^dMf-}mmQAF9Y{x?N%F zJKK=LK5vd$A8cFx7+9wB2pfLZBwes_fjRQvdJOf2=a$C!C45DJ14gVof5kqo)I8r^ z+-_RGz-&>h9G@Rp)|NK>l#(=#yW5F5{F3YH%l2^o5U9_)Mcn-6qHE(J1InTAS8?mL zQUyb~$@bm*80?1BhazV%pF-qISwoJ!()(F8bUJ}!YEQaQObK&{wClCH^AZ#LT^2{a z7F-#-Ql5!XkIR8}9CpI9*kUc>4K9KA@W8rO&fUO?OQ%0u%N?>#Xbi}-ciqk~8}@KT z=k5HSCzmQm@_M>OICBhqiEyV93<|CevX@Vde80`yAxIb8>DC!~{@`A`D zxAQw>;-QMu|zq-uD4FoUc+dv=#fU7TMRxNFCvvj5Sck`U)S# z`qQqYbmtwFnkp}T)Y#YBVet{?(YBTvSoY5GVPm3Z$Iyy}TWbAuvD7!bxx>=>mnGUxm(A7igy4xYL@^R(kS0ezN{s+hRvx+3@3sO;0B>8v4^+dFQc$A=Fg(Sp0j4-W`S=laZ5l@_>zh6V*w zp>f)ct`r}c-jV`$Nv|}S6}?kEwDe(NEKVCItD{PFNFKKYtaSQvr>=OdvnOEir%7rG z6@OUIW3<~Qa@55g7ff)kD;exA92epa>eW(v(fpPDWN&wRyHR^q*llYaHJ+9p_BZb= z>WrN1tVZ13l<{+w%(#K2z_QORr%$JK3pCcZmo`mP9vi7TUc|@AY@5xtujZRDCy#mE zH)ksbm!*i^xc=ka=2(BP(c8yO^iHuaQB#wA?)Oh#`l*5Y{oYRCiUz# zwCo!ESIOY7t;W}^(XN++ldp1Odp^F5E!nZaR=fAh>}Y+A#cCvj-9)aukxgYmmJR(^ z|GRgRJS^oU*b>+JL~(DZ{=aqx2{m@;k{xu-F*-{Th3tj zZ8D21TUa1<>Tr)OSsCV5O}}TI$Ah~o)Qdy4VjF3b-;JauXG%7LBhIKdZga$VB~N;xXPh42t+}>pG&sfLwL!L2S^r%=p3+&9 ztq*_qhf+}gjZd@Y2~XTg%#KRd*ocj+4>0P_-8j{gsPgjkVBy2jA2ji2$()7z@jgH3 z7eBhL8mxrXH=t8)x(i`aAZFy~2cz(vpuZ?(LN)bDrv3yYEET zYeWXX3-0rc{&g9P2YRnLfA4?ge|M3&*Uf2OKdCoJ<#rE13Di;xi04-JO=<0{lAmF* zh7QtdIVb1jtM%ek_h&kDD+i`5c2)(=7{7BnIAy!Ds&r<(InFE!vj15*5|aXcK+P`lbrDOlkz9kPIwC*t5|Cw13{`yYJ_ zq5ayF&K_;3@yq#b^T5@oy2ZqS;SfqjRi@vsf2Oh#;top$DL?th>hC{U0BxOTLL5B5 p!$bHY0BKhpDT<2tKls2^`C04H$>=>MHrwEjnzE)+&WX$S{sWpn$i@Hw literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings1.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings1.png new file mode 100644 index 0000000000000000000000000000000000000000..b28e44a636b0d6d61f9cab9bc50b9e103e38a115 GIT binary patch literal 49246 zcmaI81z40@)HXbHNehakAl;yJODZAVAl=>4ppp_IB`wm8bR!@!lp@_RfFMYB{Ch;t zIsf;*-^VrM%z0)%``NkHecx;C6ZT9|1{?DpCIkY(mXnoKfj|(ez&jir6_LM- zZi!hve*8@C@nfoI&JN~Qwq_8BY*@S|nwDxWak}o?7%@u(*upUi_ksZoR;LgG-9pvxa}y^OBkRK<|b#7Nf>&~SJSJ0rs}(aLXC`?LQl zMUtE{s7SQF{B+$)j7=e$fdwHzaS~w9NI}~bhwNCF%IZNN%6%yctax&q#bQryA-(j> zx^2Xj@`PL_c;6 zDPdB+$BL%JW5^IZie!(BR$uqQBRc8s>0Y|6Z2#^v1utw3^{JO?sQw<736aU}OT;=V z*qP_+d@3eI3VMTg0~l}Kb_cT~Ee4<9(-BO z{;~dwQ*=*jBxHZ!-uCIF=w`^z{pX}dpJ;3F1Ow9A6`!w_(`eCTnv`e}js=p(FD;H9 zYSlEnZtCh;>J1bK@;@PcQTa`NRd9^|NWd@lm}nT$By6{8dfA+oLPd*kCNmXjVxUw@no4qPKaevP%aw|C6f4{Jb<>N~dPFVa4lw<(2i zK`c`037oQ0J96$I`aI2VPs>G%3?M0gz3+r*--dM(D29hgElwtjN@#ST?U)wPuEs&w zkCc;1#)4GTCQy!Viy+u$UrzZjh-Lwq51TFE>B7A_Buyn;PlU5}!5tEDJOT+R%TNmA zNG&S&P{Pk*J~3hs$il?m#y)&Nh)+#mO2>Xz;uYd#U zvREaC*M*rn`S-2EqNGU}2(ce84`W!;381h_JRPQ8zgLAi`-&%%ei&>0Q5C6gAkK4$ z>K9kgJPMQNEGz_1+qpP6c}ST%m@4iSQ`iI!onnr?IW(elqE_tqQ75d|e&17Quq$uj z({9&^fDhVSK<%>T4w|Rv)f+L|yLdm*(~)`ZhsDSxJz=0#BdEd0!B=c={{Glg<^h3Q zsD`9R-JQ6w$xf2auFk_w;m+kwwk4V+NSNd|n#tHEx!4Cd1p#VLbe>L0S<~v%9ma?- zFfx-c^E09_<}hHyhsQI;yEBoiHWVMI{ovM4vlTyjAoJF1<;P0SO34cGo|H50DYsOx zvD9vEr*b9ZK7$Rz;XBoLjqiB-`ucMFDEhwkA;RkWY~CF+WvKgQUMNecc4@rId0R-W zto@-iLB0{VjE0!lTjLYzWd_4Ez0QKnJUQD`CiXpc3q zwS0A$wUl+dHQIN!!oz~NQPxrI4YI?@4fTzI4c_l)4=wMRJfwcK_mJ4;=XbumUHQ0= zRs|&ay(27NXFhnT?Bio+Cx6v?$Yv*E$GmmBL?$BNET?O%Z;WYts${faw5+6H>qAl}AE9&+8{xlK~~NXTl!;-krm#XGyIZ|h0QWt;;O-CEC^zSWJM@av}xeq8%DK0N*7 zk`9&bJ{>{a09_hghC+$_vOg2-|(v)cK3@ZZ*zv&>`_VS)e zEjy<#OU*g)W*=aPP-&=zMuUcJ5p|J3QRA4Jjo~Ka*rPE`yHK86DP$>!Jc+i-x|-80 z(@fJ5?)hmcX>In+`r{0r8KRXxD?iG%pD3E(-oo12{7K2T&i7LAw%|j7CNet_0kea> zuNKn&F0O9Rn(R7_2a8w+UdpG#mfjHa}Q`S|t_V^QuwL`UcJUhD>jdTH=e zOHcX2-EZk4fuQ4UJ#Dlf##K(KTD6+q3zNX{pf^~%uYz8sMWo&-(LX4O)O2r|sc5Ra zfEsTv?YBmqx($>CKjlhrWpmSQFd}hEcAPwfwLm@8EY)tS3+Zs$OLaUs>#K}+&{WWI zEB%(^ylmZP-Mg{gKvD_-WY;&nrHF>8e;-1f1a|YbqlgAb}1eaJQ`vd(swWB z^$+aJH`;Gz6?zr)2eJn2SPfW1a0JQb{5n@;N0qm-pGGT3#gmcRvRae#u?ft1Q15qU z*T~sU)1E_*t&|FB6{T7=tk*_x0AVS^s_@Sx-Qy4Xwd&6CvOGgP!6WL zoL4=Rnoj-$d!zZl=VCwo&WAf0=TEM`pL{yYIfMrLpI(lB&G>C--+GW-QE7YP9-H#rAvvk%GIKQ@dO8>`E8t4Y@BdD~I>J0!msGQ4ma$z|Yo6-OlLXC8llC!_o#l7ZPC^1b8wcwOYg!v!gzS$A<4 zYojO!5l;$*Y7i-NaInK|y_il=^9dGyIuehRkHXMXwCmiSp;Xb%OBbwXRe?CVFKULA;<{#ASmD)0(grckpA;s8i4_V^!qv@1QKEeLH<)l z5q!e`V!#`I&&?;&+t(1lQy>K3?U{-Aw=`l^CelCO!@+M5an;9ia^O?d#M#Wu-o?_v zRlBG(4P3x*l+|{DK!|DKZv;7&2RoqtF)KAKS1kp30TTy1HX~C9V>32SJ4bju5Ft+i z@YT-D)riW|&eq;Vz*Cs|cL@RT9e$aen(B8ER~un!Ern-Pj~$%NsCd{o*f^*~FsZ1h zgq%&y1ym%Z|J)A#OPJcy)zwjeo!!I3gUy4R&B57%os*xRpPhq?or{YVlwfu7vUfG| zWVLspxvAuDJ(6ZFCeBulu2v5ARPcI@j2+xug{i6G5Blftrk!SriE zKfnB+cdEOXIX`x=1CMkS`FFbh-2C4k|J*3V4)6JYM&f3ee_sXTEP^S-{?D9=VCKX` z!WSc%m86my_ymg#{)Zq1{$aTJgnw6^p3VPQ0)dD@3e+FZgTVH>6)bL!-Jh!n>F|D?5WWL0^A-5kNZ4rWick@DzY-4Y|kgypq zRj?_u9xYUB^}DQcIe{Qj;oS=4LZ5mr_JQb6(Pzk1mP8m$L1KL1OUL3ZW-3q$f`~y4 z5l_i%e_a$y1ra9=GZLd0gCK*KEx2woV);3d4=*6|O$S3(^iBB+aGkVhQNn^1l(%W8 z`i%0s{ypF)EHr7(IB-2_pn@Q;j$%h5epk z3Mudc6xiQySO3$@9R*0x7%!I^B=9Y|*l@y2+<5Wd?GeiqLO`F|lXpZ#b|nbN9F@Cc z7Yyo$pg(vELAIj~Ay*`Q02+iRB7bvvLJrh3==4hSD=ld7Y=!vHO}~tvK!SL3kwWad z6Tp48Avok{zXvmv2~?jSb?eNG83A2KSBBUx@OSlhX+W*Fgm2Nm1H~oKDC}>lUk8_q zqFHrN18qV3KPk8K<3R$o2v5%EueXn`>(|obemO;<-}-D)?!KinRb@peywir{eC76Q zE{#c}SXRAAgSKA|5s8yoeh7(IR%Z`2XzXp^OHwd6x)|k-s=N^;7P~mepJ@V!rgpSC z-g+SNbW5Y_{Mv`mg5fZhXj{!BkGn7H!+YP7P!xq4qiuBpU7WO+F+}mE)t$+pwW%_= zzR@(GTYORX&Thdcn+neg5w9wig2HuB0(>FHVbVhk{7A_6t7(6I(jr4a0QJNn27+87 ze!`I!7cfwPAH=g(zKjT_Q4eDxm7aK8woapM|5d9t3~D*k=r^#G9$< zhu;gO9E+0yQ{-iW6D{8M0+C1b>cKwQ zvm=X~EFewQIc4isIm7zCXNn_Pj}>dnSuraUR26BId>9fr??2=`RCt9+wBAD#EiVk) zyc`m}mLJ!1!Jc*O!2X`AKx*iQMRYZPeI@fa1Sj5*vS~R=(3~}0do?e^?*qTf>hM;NS89c7VcRX|+f^=pB9aQjd+xqq8gPV|vUYD!LA%WdAD8A9H@Svk9o3b1vT%loZQg)PVI?MW@&9RbfCiOzkAAT!+@~M2WsJHG? zosX6oids+C*k$@w!RHHAKz;b`&BjedH^tN$J-%b;6MJ>}%YVf8cvfcGzLB+Jm~TBZ z*z)Y)$H8V$5#5yA>GnPJd$2m6*pXPb>>FXUmO(n?yXe^^yl2*Rw+U zG|eIm{V#T9-gB8@+D%o39ZgzLT2EG*ZJ6oQJJarFjB6HaiKCG@LLOOtd$1)CQODkL z5@bGvx9L(k_t7;*rqjtYCL|;Re^1Ic6I)%Krv0AlDvAOM4-WTlI!eZf5>Wgsq6a( zDXht;hA(AJ0#~5W$XnW{`Cj|0okiUpud$y#b4OH$Aro9p zGWlk$l*Do0Q8~rD?E@~@g4lF=@RcmF0Gg6#k?w`cCzeSSs28MhIr*X|=Dv!SQ#!-i z*cDh-T;y5)*HHJeP^}WFjMBRYy<+n>ck2IeQdHLRp#w)S^2NdW zeP7e|LFTvp7SX~-nXjRHavvRT?V<4Um2Ue@^u;y~w|SAQ$de1{zaSL7tqHgEWDj|A;SoI-!$YS8U!jc(~( zQED8xQz;K`1NPV^Uer;VGVm#T^LKn>&Vc|t^(fDt-bEW@w^4lKp+$aL$ru_6Diz66UP;S&F|Mmi~{@yn5F8_ik=W0cAI zg&-`l``asS6=%#)7gA#w|CUVg>prgiwV_0_i5h{<@Cu=BJ1l%=C6|ns49;_Y=XR&s z6-J_??MLiQI}#QL8G7Tz+RLunGn4pMWZ(SRj(_B3^wqp7j~`9<*^`?r)oNBB=-KPn zPuo|KzMy8dE0At*{Xwmb#mS;nK?$~X#Xhi3`P9zEuaZ2vl`m8l_53dn2Z=QG784jB z!dL9v&IH!(##QJ;q!53Vp`5{JYWX^K+o(4eDYSnE7MQ>3!octVZWbMFy>PVbm1r$m z*Um9%m1p5;2&TC4mMH>mh0`X)6?)r-*Jp{|QS*_Mq6V1N(IRKEm^#E#OMxt>=_ss} z#wcT6hohP~5Z|lQHF9G80*T~T979I5NA@kJ<{>C|GBnY!xWPDyI#X(iDW>oY2zN{9 zRTm(aXz{yuGFeSUTdQK|*gq!M%jha1__85%yq5009ZUJCViAkwXIaArllJS&=l*q+ z8w)jE8+9F5`|W03*0l{RhVkp_AEuus+XnHtMA|nWYMeerA;m9(RXQY_rF$-gJrviq zy!B0Bd-*XBXHnrMUN!JOAIb6``8ENAto1m6^5^1KsAGWGeZ*Afw)!&vQ>dY-@|CLa zwVY4u<&n!i%V@@^YHjY4X!CwX&#U2B{-QpdYpX`@Y{tvUIzS>;%e5On ze-h^dNkQSm@2U^oe@>(_iZwM8pZB^e3jBD`t3c8>^A-xmgAq-J+)YiVF;)8W59Cy~ z&jonCX9uy3VmnDikBd>`v@kqWaWLNm7Ot&5;vNt~*GguKokCsv?#%CoUQskhd}!@v ztJaFKKm$hob|&z$!)5-B?3q0D&} zL+N`0iAtBs{&bo!uN;-o$I0K?iW)BRF26rsc`dNi&Q^~IWiGOo(k>a^3%4kxnZ{&I zgvHM%8oXdEdEY~sP&DboMM^3%RMgOf2<-+Aifx3ca~t34s-n#LFkY2Fa;}J_?5bBU z>rmwj6tko!F+Or43U9^vRYH9ZMKW{MhE5jeGgwp8o*~jprg+h&dnuc;`UNlnzDh!| zK5=(8J}dyDOb~szyAco#&~Gzac420?e3iPxzZ97Bkeg+Nk6uIQK6@8wggUV?%bZ#t zLLQf+up$YbA%q$w-V>dvggpLJpFe>`7QH*WW0{NKMV^VG!@}d@Ab~h~CHFFybhb60 z!j@O}ExWdnOMU%1K4^xq4<5p-6gGlGBPIX5IBQKqj=Jw3f}02(^SE z!Cq4@s6uM;S#GN#f_`!#q_kqw_suL%@D64vTpGW!6EiiQubZ_hb1|`di6UF zQeE}wP2)^!HGA#nDJ{U`IFmBXZG<0-Z%tHW&ydEKVYM^XCNKFa)epSJChgm7qGY0+ zA$FkbnwGIex0m=vk8s0e(`D0b7^!ySIm(aBG2d` z;+v$&Uolfl+r0;zOKr2I-^jv#-rIVIy8N3;U=yM$0Arp+3kVF)Q{u5^@UX)tn6S?e zp1at-%*1w~YbOT$hXL8vZ|DB)01g!{K&am0e?Y)S{=KR0(cOBF{F_0_@_=7VqeYu= zGXVKI={Leh78Bz}m?PW=j(N-$WguA09+0m3$%SzK8#?E*kC_1I+(Z0wekFNAu7{)? zpoW0J5A|Z`6J(_KEs>I`Es>GMsGcC(;!G(oMYgPf-lU{pmnPBtZrmf@vgah=A%qB5 zJk(cxN@P^e!vT9^+$&FszF}|kJ0q_KXYjW0B?L28qh0L0YFW~vtvhc(4Ix)_dIsiF9Uif z@~4f^b!K0F6#+xEjx3LAM_k2SWS~5_=7*ag2ur^^JnEZrh`nLorGm+9PiuJTld9~o zc{ktV)DyPce;B8CJgJC_k6iLzs=XPIk8T21Kwl+awt<~ynPOMN`Hhsg>QfxtdjMPw@cZjd+E89@5 z&*fk@$^P4(kC~51NTJu1BtJ=jB;xhq5#Z`C31fmZiia!HVI9yX@4kJ~wYD9tHP~=Y zvpw7oVPU4+CLP`Xa$ue+g|NmFqtN9?!}h)i5Q%INt|Xj{S9$HjTqe9}{N6YyKrsx8>y_q=sP)mQrTGBABV-aYiiu_**n#> z^jje{j(G9x%#Kv9Y`@4PJByP!VV}B|j7~aPT(!kZM z!bGHc9uv@;Ilx?%OL4*xN|p<16e8dtp^^V09qk#;hAOW6P(TgrGa%E%0cO{vE#)RO z%ns(U1X=zIdR&DQGlK9hD$7GhsUoobF=(Ifna4j{aU?}HmV$6f3^|GqZ!Csuz}rbs zxBekwy_Y`Y;Wwn2;>hPA%6OsJ2yvXBO>Msmi{Vu@U}XD(dWSKR`U$zcK4FqSyahJO z3Egup>7>XvE#jxez@iY|4F~*RG6S@kymP>MW5$YLuVr9Ll9J~IIwI9Vgcu&@=fw7R zpXp3Yp72&Ry~-R7B6U)C_DDo<*djUF^?Y`_4mt^ykVjL?+%IN|r4>9=rIlu;l;VwtcatJq z!efMicA>5M#sZI^fQ(mCg&j`4L+j%B#b^KPkjQv?u|&d-`O-*%++PPy0IC1 z8O6wplWf*U)P@0`(nGj&kH(eHn2Tp)hrU_?eM;?~CoY&!6ofeO>CFKq^>TAvH-+ZG zU_?%vuJ11isl*ISUgD_HnPlVmt4zPlRLBWD2K~~&)2g*6S1(ZM3S~F2*C;6ZRR7Yt zFmr&V<1lX^O_AlL6A=44BLWidWmw*UI`4K)%WXl1NiM=GdDWVHb-dS~P9@b=|c!ah-IRFGL7pOt0 z?m#>d_*veOZ`ASH%C`#*PnLFpR{CQ?x3>cj`I?<17(sXM;&YrX1Sh7bvsJtgPlB3; zYqykka=)V8>_3*1Z21$nvgK-}7zP#9XIRdUumT|}B zftZnR*y{Ja(Zk_bFUBE(-B1z8xLl+8%E15;|D;NBpFUenarh|f@bv=rB1N{!}`bMWIV4s~v2pXE)m=VZe>G zTBKF!GpEy%q|D4&s!p{v|MkV1C~Rmr>v0-Q6yIk?X6882vev5?`3&h^t0Y#n?psqh z$FQ741826@3ulY-t=dH2%kkE$oE%X@pWRost#Wa6I#X-XA-I%mcA;f*qvkZd*muJ5 z1tvke1TL3qv2)fE z0;*YKlopW!b9aVRR87@6&@OZI8xO~HVW(pyx_N19xW=^OB%*#zwwkt%8ScJ75c+o1 z{b>pW4U*3?W0e0GRdN86yZqI#q zayV{iM(zi59?4a30YB1f7dUfU47vjOi^lHau*^Rm#yzcUX6Kd0VH9+rIgsV!7dGk< z0^PN#kjc8MY};tCksSHP;7s!ga6KyPS!}>jKENo}ZyeKVZJH^5N43Sd!8Csy z++7u1CFn>99Fo#@{5)9}0K;M>ebFh8rgUdc&%LQuyphZgQ1qJrD`4&v41^YZ*yQ=uo`*nm@G z9EeBDO=lZL+lOGn^bIzLvw}a38+g~lB+r8O!D8)mE;v*dEVy(f{j`o9p$fARQHGAI z)z@L_fX#aRE--Uc;qDeNnh(MQ?ubJr5~S=ngKFM2IFX0?1Zk`kK8bCYhyfHq+w-CI z_a1(LfpZ*JMyGs_CS?*}@!j6Pn#U)>CW5U|2*2oQcrxrfn;k7ORSzVuG@$9IB@{f% z3eqcC2?G2%VGV3>E^YRC(G*?>ca%a;OZo7@bYbr&%FRH*5^!1Vj~$KAG@AyRY5vdo z)3prtEuh8rE{*_iQTnI?-57Z~adl~zx znw6f_iFW09u;8^j{w=4gDYG@s&C2{`EC!RW1=L8N+-4SsQu~Y+suw2d^_uBgqo`LW zCO?K6Cqu(#sb(7V8(d@8&W^U2hZx1Z0bQvx(|YD2Y=EIrfFjqQWfs^+_FbtkA6uoL z%afh+GkIpfJlOsSjVh)5=u^(Jj)(vzZX?UDhIut>7gQnn9lpg%dG8H%d+W*|UeqbQ z!Ms*^v@r@@jDIHbE^`Z*u6xv1{f8$?*DQXcxDlEn?CC!}P)a5+SxVT-R-KdM6b0nc zHH&JR1oYND0q!@Q1!gLT2&@hZn3ut$)yEi9{WH^XN+Q|`%X>bigjFp4_5LjH>&yCT zqsRsO_BXE4Xe}GvHtD8SuGPv6^b>UJ9J7StCB>mp7KGK}5!yA&kE9(ZGjSi?39{$K z+yXw-==LxgF!y2@Nw=m995j*vq$M(j7~cVfg^Es?@#sXq7<2+Hgj<-Ybb<7|)%F_P;Yr&+i0-b$7RRa`%+ zh-u5=%s?Ui@JtxmOy%jvL2|e6%HM#y-#|C@&tSABdh7?Y5hpV+k;JRvn&u!O zxkoA!wV`&5OA39u=Z6bD(8y6U`v?@#wJGznT#6ULnF^#qsg#vrQ<L2N02aRzv-G)=6z|Y7R06R#rMvUo+8mJIr9&`B72H{mP*@^iyYU%i~I5>4N zfARssKDo!t*Tt>YlZCBnzEQC-BN6%*k?+dNwLGI6R+0$=FN6=lNgs_WQ+>Z-25&+?^y$R}Nu?0dVf|G`(8*-_)NAdMWk9UMG4 zVXZdj283&5M)Tyn`Ma^-Id?XBkxLAX8Gy^e^FpMf^kFzM@A}N+@+jd|iB8TNA=fwL zw5lRz;wi)wvoSwfEztKg&# zr2yMb{6N%@6tC)ZeF2a?F%~#-#ac4&Vy=+We>1?d2!?uw&3_Wn;Z+xP!YfKq$u3;| zf{?6tnP%-h)bBjR7MvAQc--ie()CJhezjh=z=iHU2aaWDpbUy$u zKU=2WEHAwTIS*+u^#-FeVGd7KaX&dll?oX6>`H}bX4bP`(6lsY_!Yfx*aiX*^izyF zZp7{&0bdXeo@69L4KG+66~0G0Wsz9nVbMr%ZLs)0l}4%T&pfM6%DDLv`_9F?_lR(9 zV<$)gBo*1zi?zNUimkMMLqH=FqvlQM%s7&>HIV}1Drs0r)eP6PMfU&;PJm<1Z0d&aW!7vTjYVv=&)O&w@2(L<l4=!M!6%dF12m_#KJv!i4unodNIK2sh7{5%f0KxVSq7rb+-3N|8 zauUDw#L~PufWW(AbQ{_Dahqu*FiUQ@c*rI8pDihY-FJnv1^KyR2{i0rP-P27QY1b= z`wq`Un8k+AkY6vs+fDzc-TtP-kAWfk!@u_a!@r7ks>5q$-KU4QD#hc#<8V3{&;tV4 zZ#uY+rGgJs(ujc9OrSQZn23Ph6!lV(MIkUyuINm*p{I>LHinZPYz;3$3C2KpA- z?r>aQ9?y?VRol#}02G8hTB@(>$E^jT!ZZ7HbNv~j85$r2S)yCZV?@F8_`#p)#(zA# zK^J)hSeXQ9wNn4e0F&!5CpVw#`m-}wqG8|R3t%Zf9&JslLK9e_`dFLD{_6X{rCGHF zAgF0*5GQC)aEkpIrtH^$anMUJy~8|kg6Z0St1`1ht$^Tg{p;hS)P0QBt23MT97c${ z=fDO?mrlHQ+>92z%6w!q_H4GnO(7{{0PoM3+IRkEz><7++W)jIAK7jwU3lch)owTC z2*5{@7O76Cm{LLj3~g{*&rPHy20a@qQd)!PONVzgmo~x=bF8tE+8nIeeaQVPey@iUIT2Z#_lP7 zhEL^%kja-@FP6gS*T1VOx)y zmp=eti)St_YdK>YNEc3dakxRNsTrSHV$dv*1|ZG%Vr_0X{j&t7ziQs^9RC_4DcUy3 zeQ%|=YJX601jvbOC>#-LBHdeLo(qAMa{vi5)rJeMrMj5naFnfx9M8Bozak0JpbnoA z{cYj$y{V`y-3X=6!CuKCl`^Ko!`k8ZjL@9V%ntvlvX3W7Vn3 z1Yz=C(Y15@swU?bz`4au(Jq1r?%Sm*kQ`A3Kxyxh*Q+CV$ePD)s;<-1Em0MZ&ml2d zFQ~?0zNHH6+Fq`idGTP>ulGa9s7GkN_w6}#lgq$i1WU(tzKGaxgDmyomp`Th@xP`c zxCShT^-L4<7!&RG0j4 zX#HE&X^=tqYtiup5d1PkW=lfdv*j{LysFRTabou z{}<*?Y*HTAb6oMw51}ihe@xije|DrKJ}|dWZm(0Pst577B?i~d`=!izt&wj`RTuR; zz|7+_)%mOY)M`RL%a;y2vw;Avga0`xx$CfaVG7qPMCK~CsGer=$YJgWcvUazKEN}i zf{{-c9sRVvbOxrxV{FXXniEzvJsMkR4~8M>0fP4w`y{IrMu)=s%h3_os$n=?ba5qmr6hX!&cJgKkoC6*;a`0m$ zXXg!oi7or~7yzVY0hHd0Hto5|K35%h+$x2`N*F$8^k>Ks|1;!3Df(l%lAaFlCDP8g zjq86)0pUA??cF{>lnannm9UhcX{TFj&nmS9(0PV0 zMSURCAx^Z3Rx214`@_VnAN9Jyeg5vEI!~c4>10?jx(BI3b;!7+h1?T z{tp}7p^X>H2=5jW+P~desD!d*%TRcA%HleH;tC>{7N_F55oMK+AwYNiAFh)FE|I3( zH1!M>_6cl%rRNMd0Dt=;(*5542QsL_fI0k`MH<|H7i3#sJo6hnU4YDp1T61@GCK-4 z9>q6!Pa7dfs~HVJbR<%oJsS&fuzXL-&8}%iqPGK>BmORkRr}`h$(4G8K>* zW(hKPT>BXFhUMtfBz@11s%O0G@&L5?J*i6tQ?D}BAMC)zav)g$R>8Ho1Mr=IuOnBe)}G!^q=WapM9{4z5&Tb0WN_OD)I#a4KQBA6!_Nz;7GoKHKbH+ zGoA~$bMif2`&5nBYxst1o8Lh1h^G@XAIr|7{UExIPnmhMY z0oHzxs(3N<>V`;AQ*{P#`mYYA+4S?=LX&yk`<8YPC_zh+d>sb0G4v?xgRpTv)BR4I9H#RQ2SOrZtzsXw9gg6&7c|h%yf-10|g*Zm+9%l_Ccmz zT+yed2Bw$HqXnuV6h1rR>aV1uDOfdw0Q1oWE0w@)dnWm%V=hR_8)w}I`mZuTFJYS2 zPMv@D)dt0*|HTC8+Qv{qrr`(#IL|EZLB5W|1E5gH@K?ig2myYpUmGQ*@F#A96Kq64 z`IgfNHh>U0f_=4@*g6bQoZ4>HIA~_H7Dt4-UUGQALC`~?F`qyEM?(M4Mhz!o&Nm-; zpEYe33_PhcSyqC%^`vdi&1vk(PsRPq+CGUyt188(nDHAB+8uabl5yVH*LJ?k;rJwV-NC?cm9(7xL@?Me#(MZW;ZgFcutkr-6c zmiL5ar(ZSk=6nH0hR+_VFWq;#u@ zef5j5C3Fc@9uB1eXzxX92Fa3&@H85Iw^4P#S`tQQfM9NlvjwKhY>>sFtIK;5vwb7M zAEbT2#tb8fS>NJDR9lTmuWoEtjVM+|P8bEgINoNh-CP>g_PZNdi@WUWLzxBhy0hQl1kV5043>5Zgp2 zw%osN1s=}(>sEGvZ6oo{{PG(={9j!Q1o8AYv)#N?3J);{!x6mGe-M0|F7>WKIvvhT zvHY_T1-ww)zxGbzAA9#-2Mk8?{~i&v1J3C(RvZ@)#^13?<>Q|JGRVDlwEwq1hM=#| zz~h%?hVY0V9yTy@FTMZ=@;6Z026FVoafJ_JVks8P&5#8ngT!qA`fP-tzuN%LFMm6$ z^-Nao6FBV!;fn&0c%I;e>hVGH4-bn*_y^Vi+WrT7vvSlq!=Au#|BtRah99j)a=#An z&a<@6)!ZI$Ion`<0dh~nG&`6yOZA?p=G)x=J67-hGgd5yEveSyrQ{9zJeF`7Wig3V z7$gRNL<4fyy*Fh1TzKMw=KXgY7GIODL)lwD0tdpg`f~U(^N-^lCgYwC&K!6UGz91O z&kzy7m=QFjO%!JM>|u~QwjroxNuW#tm{_2cAp+KDUXTzxRT!QSWYxG;lMVND_a0hO zZ3=-*w@$m3-7loBTh-$#z_u2E0NvgcfP-wC{i*zMU{lcyffXn`Fb1+}#2i{q=>QXZ z4<{asp{*xNSOQ?D%{s%=$U*4qD|nVwqP{!adIob%J7Fy=to+s+2>zot0DF*wM(Hzr zgNT|o3*Z+ef5#vJ z(Q<}eEk&^B!qdtF(I{lE0SY#q1;kbr_#m@u(aQr>#$1mXCz&aeXkk-KWUNu35DWwO z&j8)QoCfJdQh<8X80h>utJcm@J~fBu#~6XEjD+7#5@73KVIo@{?cz02CY zPuLw@gm@Ao_(f?^vVuL+Jpk4`fC}&TbKBSJ+Shkkef{u?=4GDHF)-UuE|Aq7qPq%M zh1=Dy>#HIBqN2T2&y#+9=8i$(AVe4SR?x3Ms&8(!-JSjj{)Cy zs;XDJ(Y>U@ZYNe@xDN}sWev?VkigB)qBtb9FYlG`iVc~DL;1otcLAG{`m%!oEM-ev@{7pZI!FV-f2MsnZw)3@|qQTlsM z69m~y=9~Pm=KwiA&@i(Csz?r7Qg&m)ss^BH?WdE zCiu+)A%W!>p_if^JU^zmPuC=C2|brn0>n|Z8LCh+ zuo$8|sbth+%a)8dd0_z~7)LWMN-{qqV>DL3kUF@6p4PepD(&*(=x4HqIatf~4UeZ# zd_Z2}Ukcrm^(R>PUn9n~u09;T)v$u}9OU>n)3j*!CqJ@DZ~>B}fU!5*u@PHET0(ya z95|3wU`Kh$I6KtBw&0KTf%;?<0QrmH;MHa!@_K`LIF%+c> zv(yJ&{v&%27XUs^S4Xw2{2d{<9@Fh9mj7V8pm#_L%=4DS$YO0|2yyeTgGK4zlOC(E zM5L6gkxli+(W}n4_`)>w;75Dz^U0xW)W0D<4xA>S*4!q7>U!)FJE!Wj@mF178#iI! z`K6#+q`{h8*;as@>JRm*xp;L8H1%E>+JG$j^Ci|M5U(Tkm}V2XZbQNyhRZw0g)nye z1z*Qsb;~$~DmhOW(Vy`?__I7n+5c<9DSd$I07S*l*ri$(L0ZGB1aF0KwH~v=T~C4AU&L^{C5=oCFuWpG9x@NUA#Bz>HxdX0tHj` zDM|7N-&Y3uUbdEF08$$Lhm>Zov_3>G@i)&$fafvrW56}>pSRgSwuR(tIC22u(Lb8_ zaI?2Zy=Gn*8@z)8wn1V(e|<6j|K^MTUo8Upb1mAx5BCB73R3Le|EjF90m?Z~-j(g$ zf7OyefGYdSZeh=I+@gEi2F|KH*#q{hfp%2J3~%Sj@1tOwZp4{ne-i&p??2lpY$_OS zU@2X=jo$g6V{biDZL4UM&Zm+4tLVTACxP=js%mJ3>P6BZVS>4kjVUm|@Fr8S^%DXN zz!_?#wpF2uo(`qwXs689Uj9SIQ37ro&j3^lF9;WalltFBx{h7Nz^TB%e8O^mOPMrZ zmGfF4eCo9NHM1YX1z{kX9jK$d63@K-yYeX&bJPt0s3hM84B(1L67UJocKrkbUyl=+ z(nIinw#WxEV&?~9H1wCCx$t9dV=O`BfMJ1G5k|3YZIl*=?MxkKSw=jQ`rAfFwE`7d z2Yx-cG@uBM017g{17AZi0zuxuFhD_+adRJ1cxdI zT3aw%Rs$fcP43SxOA~}^1jZfI^jbLOIWEW!6@;gIS9Jsjih%=_s6qH|kmYa2amQ5L z1uXAXj0v);fe|U(CeV^Y2@JFU(f^1$?72W&u&PBu0+ANb7&b0G2QXXk<9jRpL3THj z!Hgj`tixaI$n+T`OVJ4Nt;2(nPVl_hEI#UdB6xOed00pKotqWK&J8hxODbSag(|eL zC4&U9#8Vd9qsuqpX}MGAN6QI%#Ivj6bX4p?-6y;%42lc%MtgYL?wg&?$8hNfyq3U0P@eSV*9P5*pyfW@IKn_6>JiTb zM-5!(eUD*v;UiR;AY-@0?@ReY#!aWLeZ;WA0uBWOsl>9h{Y^v#$_GWPe+JmK6p zzVNvS0~6i&fFqym9_ZWfvyK?to4!#JB1nPVvLwC$Waq52!_DsOPwP4Hpa&&nV-tdA zzgu{D@Dd3QZ6S!t;4CR=HNLpNEjS}<~k5Ko(k-;=P_t-9@c?YZ>H~?}5S<6{8 zA?+{0nO%hw zCKqUsOa4FJ-UA%#{{I_Br#g-7nLV;cMwyuz*?aFjB7_JTMP!z&%&dg$WF@jHGD}8e zB#C5{mAKzuXI=m6{@=gnIiCAj$Kg0GSNNWv?`OQ{>-GAaEO5ysK7wlh`phd0(+(&9 zj0T+SHLSMaIoQ%1o-b$5pvSalkU%72NN;i0Mksvgw3LczqU4zpdOnEWMS9GMC7^G~3yICjxy z_C@5|DqBUV>>I|_2w6JTR?ULb17TLgWcGW}98H#-7_f(r0MUiWY%PXs!va&0RPDB( zAc*256@cN|rQ0tS9!-@02q^=s6W0ag3@EhDUlWDbQMRWajP=D>iyzh9obbK`u;=Ue zX>u+!0Q@`JHX%Ft=!}eMs`$dE=8=R0#w!CFVg@BN6*t1p!E#Lu=Fh?cAj4NrrzwGw z;n7YThvz#LL|kFcON0$CQ||)k@ivlh_sVCfQA7-GFD3gTut-t-PVy9B6X0(P#ODWL z06@iti8CUjF(8VE^>Qh(eR%>Knb*gOF=6@e=yinto6v*KOCzaEKcPDLwa5qbbk9A z@;aa(kdPu~B$`=S3?FP9KNdPIo&;lUGxW~DofT* znDeZdjPC^J8Qx_DhUor%>_x2$&;;T?H{=<^r=Lca_ZC@Io*O|legPI1c_yL-5DkYy z@#abW9L!Xn*FJ`TVWJ5TYdj}>{Ni^`l*h2MoapbTh_P0_(*NMkvm{*7!f*{0fz3|G zdMSMaW2~10Pvt@VUd{qNwm+k2(apT{8<4Ymgwe%^M|;7gU*ZQo=D2nvr%^iK%QZeY z1KmDjgRV#Q@Cv|qHUWcjgR=cB$sn-cjd-m@t$+)JP&Mu(xf~N#AZlM0<#c((H`OxVzz%!iy7-zwx z{ExnCFps0e-h2#4r@PxnEq?@l3BGiJ$UwI+B>(XeLfW4%dBI7dDjp87jG-hJMKUey z)lB5y@5)?SWIgi5Z_a)dM*uqJh%fH)*C8AAMo-*48KCju2{L=#4rKY6@Yrd@l|9p! zyO<1fpgDZ&n+0YYG%7PJ!UEc<&md%O!02|?AASkgN9HG#(BsFEJ9M9qi$o49d?~{Z zPIVE_>XyX#ZPYp`3=O5RDAWHf$b4VYydl z1xel?^j0$yJTfO2 zGvBOHW6X&Zn3HeOwpgfPI1|OQZ2(EbrbGW&oytVAOFD~iVj)D|U?jY@W%9_F^>PY+ zI2O(henOisW}a3hm&IjX+-+-n90{G=5BDQ)F((~4)y|Vn8Vpf+@k?!{(;i1Jvye=^ z8ZJkMx4$!=__oeg7CK0+%yj-7eKUld(H9SKW9#d>p@17m14 z`?$v7gb#v`J;78Lp*SB7k{BlmPoKh1!h{WNF%iXxLxry}7?2%fm|O!)VaLKrci40+ zBn?obT1t>wMYoe_gvt5C*#VMpxhcj5)m|YOg)J-!=TTeZn1JY`aIoOVOI&IkFr&Ns z@Wd&2=jc<&=-!$X$d1EDI4wrhI$`Y3n|^*12z&Hrbp;zFw#5hezoGA;hV zpaFAGX8TW@96OpkRm9Sw2!$=;DBv5_lP4sP!TCdlXL>fpctRI~`Mv@jvkvmJ{Hh0c zoiKxZlYHV0#3eV6(u6hF7j{DMez6atB%3gU@Ec%6P*c!jhe3WhV_U@qAFpJqc=`?I zbK8ia5(jlJ;*V$eBHD7{DMFO_s+h2~i}scbEF^5f&ik-}<+q5l6Ioy! zG(@MGAaYKE7>-!x(>A^F|5s7dktwhXBbz~?JaCRnLX$STkPIb zQF)pll6qd)9_@U-VrEGsJ%ueUKS_Mz-dPaH7^M#~>DN4(k<6eE-nek|icv@fw0X73 zT_Y&ba}wG8QMz>lCyowx$R;HtX4YuH_PPgP*~oaqw33T`g2-isa}ZivO#q<(c9nur z8A%ZxZgC7T)FQ@}+Y*L~ztLaA}&X7J`|BbM0RO{hEkQ>wtv-{i2Ouz@ncbz$1feB+eHJ!yV8Bbm& z?CB#A*)VM0;?1@{!O#B84cv}oG&KxYsIU6U@KF}nycA6YdyN7dM2-o|NAL5e;FAl4 zh`uPn?m~Rpp2<8pq;I$gd5(%uo7Lj~fZGPUg<^x@eF0*;PtAu}6=Kq-;>5Fmwpp0m zD|ocJa`bVzHfpeR)(tukb}LTW# ze$tRP$BdAf8GdAU>c81LBx)jAYrlX6Q*$gVnga(W3%1nB;HErzI7lOiAr^TL{nq*vM7I z17~mzHpu7+K8lm^n7^)bMJd8x+rolq^YKMh=_6U-@XmvJMvO5$f8{s^HEAR4vPdR; z+`CN_M%{@{e_aFwD?j3YcU~Y7W%vfH7NlFs=i+R*toBL#g5w-$5!VPeIo_=OSrD)o zefV}Ee5zB*Fo$lI_iJs*$M5YwHXONsoB9IN)b%c(MZ2K=R1L73=!F-;Dd4 zKB;@hnu>p=nSQK&Zo0BfcjwvjK6k8b(;q!g(sX5WpGN}mFRe#?!*0`b1-{#DZmD@Y zAlABK)w-)Y_)&VZBl$UPu;cb20E|Q4f29ApxvsV;rXFIs$@Xz)BY5-E<<#{z`-g|X z;=IEx?=F1KGec_c0N8FlRQ#~}*^{scCNc~wRX4LP-)tn}R=r|Lv8a2MHw#ovcz3@p z&c`*G_sBgv;%-*t{%GRBeSAum&Ki9>M^ z%JcEBLDgj|yN@eZfW=w)aa{c2#K8OS8qFD7zb7;w3n~?TRqb|fU0*l{w9c42mDgoL z-?bP_>HapXj9lTd!u++v=(d}fd5EmPVMubssKX_LgXL@Ru1Zwjpn64vG*&h6&8ii8 zBLdM-McT$3^@<0rxR8@}AVMjgvlZMWUGz!xn}UEB`#3ToP4*aBYU zd*_PY7gaQ&hL+dg7{S6k@~eF(_}1C-z{h%H+=#b!e820ESmSc9t={1MxcJwhmX*+J zdzo(u=foc2-KHoupH^8Lu+X_P|23LkI;Ho1DjOF!9v<@*r>Cme8TuJ|t$alXVV-;^ z4FV%gLJmZX#fn<`-u>RJt>Z1K;>or8_-3I`FSL#FMTPitCAGB5?{7Y?{CxJyytTED z<;}2lmsGa#wco)9J|T@hoTS{LPe%t91ry&>SuF)qJ`X-GCI|MR~xt)5ps zH~)R*%02!40b|(&WUF|-%6M~dc=^Hl!lCTu8)CHCS-~42kwEI^G=pd(8Y}$r^WUj} zOK~jZKMO9POpHKjk;9(kVpGbj(zK^<^&SSTL^SNHlICuH;_(L@Lw$sYT^ z;7DQ5Pscqc?ZF`ApsKsh^0%jh{;v`$_J~BCssuXZC6r?U)pNvW{cjNFKfTcZ(|WDU z4$1!x`;$FIu3D^&SH+tD4!2O*PW6A}eA=feE=i(1xYBJA>bC~Yu@gjv&~ZDhkzB$5 zT#T2hO0BOc{!_q6^|vmPdWo?aQFLcy*hd=DLTkX*CtY`NCx|#lN127p>sjpNK)dUn zMg<(wS_Pakvo%KC#~BhE#UH+8nD(Z5H&b%{9QTrye) zPH$%puv@?VbV}iN5;3+;T)C>z=>3vUzXD4OLaz%sVee@kbdUnK>z3>z4+SJy41&$b ziw*{W00P#bS?~H6`7=ST?-Z>T4Q_ZFUO+r)I33kV*v69k7d^-$&ZQhjidkWDF(_cf z=@~m;!Fy*QpJ^Jkteq*9ZGz5}F{-e*NY1HM2t6zvXlWw-TZ3oU{6?mDTn?dYS^)4r zeZX+LfPJsoH?|H@(w5m}2ZW62-rcf0mRF}zK%nWweMvu_C$8iNS`;4!kUlQJq_lzD z$n*qSfyTz(G7+ej&Sy;spUp87D zE{c_YUuG=0ZBZ`v=0K@sMTmCuy_uwPyzSPRz^mjEdXrU z|MgwjrXl~5*`wbA4#9vByxR*X8GOnob4>^vR`(AZR_`)`4a35zC$V|p_;t;vRxd(S zzr1)$A~StqS&eY>C-;?oywbN@>LHb#AE8F% zt5xgt#v#+raQt#|PH)=qWkKauXrC`B0(`oKA~A$kP)H#y8TuIlxdzQ7b+=In(YYLs zrwetHX*K#MtIL&`skeK@DE`W}jjuK^BUv9q56^o{BQ3=Zv!|X@!~L7Ltgr8O1;WBH ztS|2Qyf?XUTk~T!RPM#0KkqXgdKlsiFlh;6#&LnCXFVoFRSF63xhHC`@oWJ2doCEr zKnr^rnu9)v4a9(l({#TKSTdqYJs$H$)Hf(+BAh>|%OC`FKM^z)Q@+)^zt^g1L^y$q zqHJ>=(3L3Ep3@%ANXxoGB&j;eAnEHean`BO$cRr6{xrxRm^njF^^58}xAFewGcsDK(J4 zf8WPOZE$O8LgcpX;vx8BY>OS1Gyr`a}&`FJ1k5t8@7nM=Pxbl`1x*ej3u60*>kmV5fTDdHDSX z;FiCM>SjA@uD(>M(c`ZmPR3Mv|JQ5(^hLl+%;d~lON#$ieA4p>4Q`wiXDN8As&Q1OO_dG3 zUIuz_qk#Qz0^){l6tG7+1Qr3(R@;H7AujJNSEg+Pzq58e4yY4Rl-zoqO|TCh!$BAZ zCB1aQAmBkev@l5dzx)LimB}TfqC~p|bl~19SbGz|Po&sFudXhNn8_0mt#Z>>egV{; zXFdc@pYRH(3ern0D)X=;0)w74K^Nf|k80 zP%-Y#8e0Vxodwov888esT0xmlxPPuIcHvoV{Dyp%H5~3D!x9~q*vU~q3A#hg*a*Ra zF`f_Fs?!HUsvVW>KjBLCz7_zbR3h{B&%nsKz7K-Rys%(@Vx#w@Lg9W!Q$j|`qK9X_hIY$^xef}BnVR%XzWY~`te1#3>xvm7BP0f zTK-Wo4}G~i-X`cx@0~T!QOQQ#=CCRx2fj$O<+XI%`ZQDH}Nd!DWSb>r(=Jv zto8J!U`iTUC+shIFuXiyD>4oE(0$)7M?jG(UJZGgz3@FqI$zK#wI{frbB&s{TU`>l; zXHpv!Ft688szvnvXfFdf$v^$|fXB0xyz{`XD2FK?+kLdY2{Mqu zoAB4cZxi+3I_dTKG~cZzwR&Rx(Bns`CXY3uzu=67py;*odlhUMX(?nyd~Si=XU97v zq?S1>Yj!#5Dq$Knx++l@^L^cv6PzuwPY~!(5li#FzC~8&c08p6Y$WUa8eyd*cyZ1S z1bNBLnr#<1UvBlxOh(Y6p~`PmT<7li}6MQP=G?e5LX0<6nlCip>T16dut z|EHIv?m)7v(UK;ZX!3K=PLnlSNoUgw_Cm{t0;N*Y2lR z46m>E^D#G0^KoV0y3ZX7T?WT0vO4qnOVtJ5<`VqdiIRGcpOV!{ORY##WOC~lzno!4 zn^mat>1MhL^)9g6vx( zk3ZFKM%tugJ$HQ9Mhrz%W?Y%ma`k}6s-+=#3lu9jPP+h>jdxy!VB1B$7pD=hw!)r6 z`AU!Bn^zl+j1qWe{*E&N>(}A`9r>+WEo2hKZafOu>=PScE^EoE&F^~v2=fiv=8&~J z33-~u)3)Swee7m$c~XNvJ7}sgr)wd5s$b^OeuT62;1_$K#c$9OFv{H9@jKYf>_8MMX%j=lc2$iA zU%#BAZe|JwnZJ5s{$DR^WGl!*R}`zbVLz%kdzlk@5OzzYOs z2bn3t=^jKj7SMFAGe$dJ^Ynh%QFfPC=cpr z0IV6v#4h->6TdANY*zjIk`#4)k&(RHF`JzFcBz}%qsm?%G?*kk?29MKvYAjGqNE>= zV$oBm4=v^Skvz?;UNnbQGYdfvgK%Qat{4BBk=?8n{QEsAFA*V_*syM<@?Na%>(>t^gBX z)w`6j>dQWu_jix|0Mhun=>sU(Zmo#PBGc+9PT;r2`C0MZE$5nd?F5e-`1$M%&F@my zJsOoVg(qDY6O-Tqo51M2U(oGR{PlCUvO{U#rI#llYq&=1fePni%eyjrV-i-k#IL_o z*_w0y{*1A4nUWD-VFf5Da_8^dS0exQv{d$LZ5vQDO9>Zg6|61}G}C!WQ>m$?bFjGt z7V9nebx|6duqnqSmF-9fTDcT{pdX`*=m=P+iIv*Ycf{_m%~E6M5K-(EDQ>xfz1XF2 z|El?n?61x9hg^s@#CP?3T((o!e|_^Yy6>!SDEW>Pcmer@_UjW6J#+JA5O2;ktSGl1 zJe3Zz%#*`j>dtct&TeL@3Q~MuVfoEBYYV6~MqMT=SIatizNfeKX@j_-X_SxH%u60E zEg8&I^!iDF332>AhU*Asm)>Dzc2wDc?55Ua`YZp=*N{X}QQm}t>CO9LTC&v!)<%4n z3Y_gt2TM&SnElZNDqGZsFQRez1v%i4^)LAC=0q0V?wN~>1T{b-;fPcb$MvoXEH&xc zwiYPE6{?5V4uT7u%E5IK3R>@?R;R|L0y4rFSVB->`)jE{Sg#|BYV)AxJKN9wmEy$M`fMOn0x zz2h4TY)`quVb-;Htu}o@JZn*6G0T44->F~mr;1j@xp&u^N$dFPQq(&`4>p;c&&D@v zy;F9;L0&p$kMlJfwJgOAh9gxO8b!IjjifOZef6N5iEM~Veq57QVaYM_{*j5)p>M4Ml)yy6g2T`Z|TT z-~|ypYGSg`{zSC)0tfj|Edzn4hoIGS?;EwqhnW1RJyQ;F-&npA-3H59mn1=e?IH_+ zcH~^p(UHXLP8D@;jTd{>)Nkk_++kiENe_6}FZg=+cUWBX!=l*1-7}LUhV`czwn(xd zurP6y4O`0+aEC89uo{DBi$^hr{LsBN6MmxZEtViJ{}?>zW>lCK-Gh;H*QVr2>a56> z58x@SSZ%udBAK#pWViuqOJB)w=t`Ai&9c1YW? z!UIsgt>ts?@Mhuj+gY&aW;Lzl2CsEcHbObdh5d}fOOhr)r9S!pW!$}Kf)9ZnvS%eg zU9#y|RMgHz0q;YJoqbN>p@%OncjQZwI&L{dGMG}5Xcf7|ULLD#5@b&3ajHnCf8e4X zi&@;^g*8&fk))QHaV)By*A+$I4EsCNXaB=Ye@UX;0odXo*!yp0xu9!okbeP>$FfOu zp+9?)=)E`4hLE6w8f29eRo1yiRQ@dgSv#~@B_3NSjO8QV7tslr=++xVrr`(|0+bLK zjNLI0Zli`ZprWxVq}~5Mkkt4^b$Td z)l&JmZ-BC^eBj2X2jtq)cffpH1Rk$&M-TMLrDx<;4ZE^S5Bc z5s}MpEfDTIa?_sxA@UC8I2w7S+Z1{!$ljbrnwP`Im1)J^?Njl(m zlh+}A4y5`78So>8Nc>OAgPvnm;a^7|t0Q$Bk#dNaWgeT>MuDEYr$lop%Hx4TWC>_X z8C2UTL0ywlGg~qRQ4E@^eyzeM-wb#dgmYa2Np2@jx)RSVZy9O}uW*g+7X*SYpCtkm z*lg)opGf27mi4GC+o-&hXq;_z%!XS<|L4ui=R(XNdBNDiJF;BjFeV3g5o$({)KYlf zJ~kKIT?`hBNOjiwwe`*Y``s$LE_`mSX3&C`U44M^SnSV!1T@`qLOD~8)Tph=1-6_# z25~QDN`5ng?%6$=jhAe2t3=2W;{&Hu!NyNyz!FP|>v(R!!0h6|9^UlFh)DtW@*g9H zRBf^WiPf6ci~BX$)Ya818sAZ#&OPl0RLb;0a^TWO0~pY6Gf1Lr^~V~#*-kRae8^EU zjMy4Azgz_I<>Y!k?xi2+o9Wjst>|l?cYHV-izwV9g6_`#f#CyyKI7?iHb(hX$%TSD zo%P8JL#>qmS;k|3mr>4$ba1!kHP0Zos8J(*i6v3MEG0oVyueVr*tE+c4V8QuRqZ3xi~ka!7IIA3+1KsTLx$demq;mha><_bUmrD5D141;(ZUrc>(&_s?uVjeRXUOsn8A;@iOD(x?ymkdGOsgc!kTy9PC&)DPP!k*wbZdt)C5JRbx0 zOcau8a|hpGb!l8}F@FK`qYh$w_Xz%>fp_Jf;h<Sm@efRlf=$2O358(YBozPmiMHBm>Cb^wjTd6b zd{#EkIRG_Wkhz1D?~ldz69mVoNwW)~y#x3p!Lo9kZ;JMj1G)P+G?z0M_U+P%m6`=2 zBKbpg}j`6s_ghun(@R12VI$=y-ss zYkSU<$eMtKfv+1q3_CO0Yc+?0p~~AZ^VQF}QlEt3jCmdTz#mP#U?(}^_Q6gDbLNd` zYb^G@zBM*%^9 z1P(+`m+vEVV4U{|ll0B~vup>f)I=nzDVR4de$6}X3w`yw3eqr9gAmZw`L&iSjL5S* z`U1t|FHmO9gR;sXaw3>W5^f z)pQW*T=4D&sdoVV`V|?td>|R&%z`sbh`y2OMTW!zAO95#3^uM|3-ZP@8!#bpg-x* z;3XT4HWq_Op=K56Hh%6g;}*2mef6)apiyHEb@FP+OvM62zYMZ0!xaj3Z__Djn~^$Y z`3c5=Hpzgsyg2BAeS9fB4b7CmMC(}&2rx>#sXQa9-4kie1`d0}aG)f-&w(V`{o~Zn zB{D6hAzXZP{r4V-A9C$3UVmrA!n0J-yvoWVT+|$fZ@h@0!M9eY)Y()L?$wsjUx$K; zgj8>BnSzeaBfTP*m9c6bJmkI%AC8?KrqDtp!r5M(x`LRfB)bdw(SH(VjooAZLYT^wiVxluWQVTplp(xdiN5{t!1L48tX$s_S@%yDdvF z4bH*{U&l(WTk7;ZUSq8$I^fSq)Ls54LwSAvr*4~`N}DIfTMvAr$U-e6CI3j@_=IjChxDh*XRp zKR@UBfr~m8QPkS|nse>zBjE&Kgd=wh0V90e)QYfs^45B5gZK;WKbg}iFv9=I!qy__ zr%xF4Jb=`(;MurNar)WQ{+wJ;i0ymkY%gR(4YcrjlWMza#@;grSc-R8l?uz`%T=>Z zg!(^qcy+g$k&UnL@5Ww@_`9)R^5DR(HpM*^j~kwJWS-{#Wu9mxeM>B|D+=%fOqcQ! zgyor~|2~27pC=%ZkE~F(An!-{0y&tf_kVunELR#Hvx_B;qA%F}8StTq-H-SvQUaHr z{dI<$|8j;i19vx!bA0JxIFb{w!r)^+{IUI4?|tZr%JUl_44&I| zlwBcF!{6-cuSHGFmuE)(9c@MU-}n9B9c_EbgPjk2{6PGIOOGy-<<=BIXM=r-m;+US zTcVWyH+l#;{%`O2H^qH{(*tU_vLfcC2HcWMg~?xF1EyK0WJ(7LL!U;lhzem>$sF!q zK^h-={cJ@Mcew8T9drLbGyHdrBqO^kEE?}tv}~v~{oa~ff@G;sI&fnUO7^UgUm&H^ z0A(FT{6;dw7=R`RBcfMEDPSR9B%Q<+goXfZE z#tBMX$Vq;aEtU2fO0XmJ=daxpXqdrq``CK^B)#|!2i>d+%6> z{4_ke*y?j%hA*g3=z@(e0XDoalfSD?dkb89RchH>Pbg?mk`!Eg9b~hjKM&+;Ll*l0 zRET>Ur1i@;W&-HMJnxCLAq?rhMNrDof3*+@L{U+&k`j}_aC3+*ur3Qt5~FWsw*9RL zNdeI8pJYn%cjr-2!9y-QXlvyTtM{juci_6R%yj;*BM*rY1=P1Cx@_Qy>3~Ch)hG2S zw8lhj+OYPZwYso!3EJX_ob_8^InQ~ofuu?7J(~k)e>hA4f?BuHXTd>2@YxTtleEG& zAu&<{IU)8O`Ti7MYPcCJ7m{+8c|P;{IHawHyMqKlpaG5h#A7lQ5sTio`Z}K<9yI$wecR+k&*=37(6li_Ty1Ax31JYpQ^zczdrx}bl&kO5C_Rm;C6t1C z1+*6y&4CM-wvF^``X!h7kd~1Au$?);Q4Q36Z9DzEb}ZI^8MEV%fDMBLwGi1biBIYG== zN5mjmcz1^W%wNMT*0#{LDd9x{b;~48l<$FrP^S39$x*~13*anIH9xhWNad1cp}N@j zdKidj3!EwA?^C9*-d3QisUjh*4@2&Tx&{qgSE!_PHuLO5^@!cc(G{xLKR`x?^s>)@ z0Pc!x8IeCHiAcE*=kbJ&z)nd?-!IG~SEfNn1uFyu@j4Hlt!%XkpzVsqjD&CEhGAX0 zsggbO#O|mP$~+^6UA@-+o%P#>Z-47pN}G}bjvbR3((fIGFpsP|_BxlpP}R+WvMd(l zj+NiyqlEm6mf)e(cUmSqPe}KcIk}8ShC5qFs@8AL5-buvUaNirS`(Mn2{!;1PO@GL zWY=h41wQi^mq*DI%=@`i2KJRNbjKV0t?&6Z&<9|$y&WyNhq>wfX(`|H6it|arQhzd zOdYH5^`}*ROSq1Gd>K;J*tKdy9HYLvSA}1vmu>}t=Qgtos){b}vJ)~N6fmv1fIsnc z`dV*7hX5hedY)PmQSq5D&ue4|cO`UCudL<~QH_Bvknz)DhKR|BAE;JNG@uDF3EPjR zWB*gi=d%2#ptL&iq?au8?o+e+ORnpemSG37ULV^Q-6NUwyC`c8g7Fa7n@Be!+uj_g z?dtoD*2>mRzKkB`*Zr#`yCNpqlCc^N8zIzwh1|i4 z`kN6w!(uW^w2(;R2Y)7m)1pN7oxgNGEHsPg>#LMk)|+GsBAyP@dS$@>?m15_Qsk3p zwrW02rjFLgAiiALbt^zOnWWJt)O^bp&;@+IAwC&COzHg;Vi#yV4&DA_6GD!si*|?R zTd%&2*FO9pd+zG$2cO9^2R<2Iw~}P)vbI5NvG7pfyE8A;MYZU-=?+Vc7ZI0sn)kTGkjk! zeN!UVbUOCj5Ot?XU5$=Y`|Cqedv89cn{2O1nyFO1%Vf}`>pCC2Xf3WU4Hp$K&C1vu zwFOB2)fRyKj^p`6AtroaY!wbP+6MirWTuNVmgE5^P&wlZiTrC7cmRz#XUqabXmVi-;uBS>%5=g{GE?~)u;dOr~WjM7r=FY z+W%LRXytHl#}$7D4#JP9s4qyv0%{=0fxvz4tb+)^{wb0yviw((q{`tH8C3jo zkY>n!4JLt$K^s6&I%GePvs4$KbN1_@aMa)M2RaL9?#neD#tfbIj9aA;6<7vbxRYgC zK|8aJOgLy3=z!mSbD7JG291N1zH=XI76}8$r3gyk5ARaA%C!Mr(gR41%d=r97@<6; zO--%|B=7dWq58}r8<(Uu8B4*f0B{nu)7-;hVeU8_lW-~P`wnb8@DvfCw^P92yESg| z9Q*(z6q$N%*TBD)$c5u@7b&rlc&WEJOrqtN{Y9CP>vCrBf_E3vd;!&QyfvpJPXeWv zQPj0kF7cZwH8uSb(8oH2@pZLfW*`)lBS-k^P=gmrY+eMD$}b#>?rb!du+@h{Nwr;V$%UfJ#`1 z+xE)1VlYzp=Zn57Vq-o2B9frPc~t7hONkRl?>0d%$&?7u>I)U*-CCz#_Mh5h6{4!^ zzX6?3oDOW9CiKxz{o;Aja-O;;X zlVC0)&XR9q0#3q#mohiZwG0_@qutM?k)9qhT?+ICgkAY5yxE52=&z64$338e-=bwl zR^+%43okKH9# zSRNUCeg^v*L^*@mSEn;x@!g?DHvns38K0W<*9FYVjxZ|-!LJ>_J8SpJ-sa+Ffafm_ zDw(KvoC?yCr5{!9AAH~glmiv*d)dcB;%Zzos@f=^foNgT4Ed2K@i!41Ip;T za?Eh2lLrOKhvN}pM@vbfNSK1`3fS&3z?ZrBET;ZuIQ$CDH9SA)Dtal|KtSjVGM^1< zH(oL)U4$`Y6vng&1DG17BJ+`ML@;bu9MCe9t!|T3Rw9GR){H(ud=|4*Zl?^m5Rnrp z91c2cM!t?99F8>(*ahUncwI42DA)GrU!OV>=1TxdP^Ve0sfu;B$6|o?!ITjxG$NF4KXJ$jp1H6Ye zA6MMeTg4bXt@#%J+Xl`JnJvF>mvy>5Ki+W3REpeesUW+6V)KDneHjM~=c& zRrZUKz;6@}EmRgoru_W%Tge`gRCt!Kc$#1~-XD7%UWkVXi3>n!sBqrdzSE`s(W)kC zam=1*s>{L-kF>6t^s(?WD`` zirde^QtyDZi+WBxgahxug=1^FGeePAiq~ItiL&T+XL zIg?BC)u%W^apg0mc{^b_(0JI-_dB!Zl7}vQR7~PvJflT=O1;p>%{i;Hah4lCK?}8C z6EE?uBX$w07CK5fFeWoRcKf=5E>7byS}K0!f?RhG8!UoEsvMFM>=kwc?q&EfT?am1 z#JTeGx31lgqaydh*N$wbg;36-)cSerHTHr2MN(ofa(U*QGYhIi1NqxKT=7qnCqLU0RBr5-?04JZ|_CY%;#QB!;$UvdVGN@xQytR(>U6I(nJ)@6a~t8^|f zGLw+uXstggT_Gg6&1~qW{__LmYo|nCYjIXCmK?Dg-ZL!xeTVFhrcv%oJ51Cy7>EMJvh&uKk>_V z69rdS5*-D@>=YZrZC3||lENVcks%FN-w$$UFW5WosM~^I`;f^oild>lV=T|f>44t7 zQ{3TA?(YR&DNNlW)u@{>m=l$h-jj**P|C*m3pqK2UupSCx8hsJDXQ61{E{qMH>9F9 zA*`|ZT0#q(2SgZ=*yx7L9${(jeOE{9vI;-30d74O9lvem&#%Scd|$bj1iVkV`afrX z(q*gh8K1@PobWdn3ng=Z#U1O~1%F-eja-2O+?WkB#_fb3JBQZPQ#dywQmrNQTBNi; zXwi>^CXZ+jWFyz)nGEj%+mgf|;{mG@`@`PJk}DYD>7z%(`~*tjIgpTJ;b0?N{J{h~ z`zHXpaBZgkWN>JbZy337k7_`knG*+~+z%m*`9yJY0C`{LGK}Gj+pKbF^-$!<|ax04z|UdLCdFmYimJerHKsK?+zE!_30SR!@hRbuzs^ z&(d<{-1uQ*=jC1@vg7f7A1+uDwPtfkwr1UbxH{OQiZ~4%t7sPY;t1 zWhLly589m*1+_cx?w<8?xe3$p&w<5pYpd=B{VujmlGDF?-t$TTL2F-RpNs zLK}Bux_f3&zk6E>dU_mAMr^??PzBJXHr!&r4sBYPS%mtBLZ^L;7iBM(X4IQF=aAl(F}U`0p)^?h83iv- z0nMVxLg^2QlwQH>oJ=VR1F!0g4SO=QEIHLG#e39qOQ!p`y47+u8cP#~mXi6!jIN0- zC_mIT5V)E&S2X-cJ!}R=up8(7C`QT7!%NnL%8k7RC{81v0w0jzd(C)g#YOWGCQ{tN#t)FfU#FUbQRFxMls`A zO{MN5q4Hy&QFvf{W>CIpGqCtL(^Dk~J~9lkDIt_~>{#NEwAKQKK7mM=0-B3aK&J$1QdsB!B=P$ao4253 z(?Sy$VmlwBcN)C2>B3d1(6vj6&8NmsWVH2XOFaTa$UUj6o{~U0xnLN2uzz_3B?pGeXO0dPqxcN&cnG{m}&$CySk8H{)j~EQk+Qt zT16g(Pr_*zqz4N^r3n-t5ey+gI3f#vU+gmDdf?hjJNLbt>>2DyaoH~&0<+#DSb;PP zpNqw{M{x-=i5ud$2Ao|+C=0a?&CwHChYWxph(S~tBtg@42`(UbO7102@B$LgVeW@n zcDEpzxX;&{4;6tCxH_){Qf6Ub*Zr~v*48P6gX3?}x&xJP*ZJ=Fje$LA8`R}~`}o{i z;DFntia9%Yo3#!elpCoHxBI7s%CPK7-j!QcVO5b>k?s*Q1w-E};BX-vl|R!L$MW^dsqF13Y?cZk8X#Q^Zt z#Vk!sN@^E8I-4T1!k`7W24_ zT+H<8P)J<^pjHj^c20No07c0aw3mlV^&cj*e1~qq)O`eaPGo^o5jlgLdrn(g~$4RKJ-G#LBSjsquAX!@Ae5ia@l8M zp~o1GJ4jmHif=~sM0)=I%4}A)U$VgXzGlq!u)5&Ib&hj)pJsN7)JrF42wKayL;X)U z%h?GKHXhYU6dxPYTs!BurA}I-#jt4wf?d9?30OA0JhK@06$n@kV<3IaHxJB=2$Kei zG1qs5`~)_$!#20d#;4(eP}L}C_o?>pXywXGC9u^~-*H^y<728uc`Uq6am;e;O&wp_ zQD_i-=~Y3UFppOy7tiS?!tK=PfS#D&WdmY27Rs08jOpODRYhK_`x`n^3~QRgtyNkd z&9!duH>WGCC`RkgJzIdjiySCM*R$b?;Vi}SI9Wde8Af4?z8AxqVpHsu;~)KC2a>jy zLvN>AmgiT|G_a6|WmCy(pMQg9pbCs7S>_4wP)=rp5<{q@U7HkJ{14E$L8;edb)&gUZ)vFqFHzaB`g?L?8{ylrcc%qF_}RTrC}j)2U)5- zG>r${$SzYvW=7iOP;<9mFFuRvFK1=E;r5jLDR%nthAHK-b9SSeQ?-X>l4IxP(WbQ8 zs7hCudTsYRBGSI64Db<{Mw==LtY;kFY*TOmQ5Bzd>SikO zP-xFNI1eJ7ytFA&@%X*u=eJ4tT_j+IUN^vn>E0JwO9>N_EAkl5*o5&F0NgiqY$>d^ zu0guOo>e{w1lD`SA)Zfm-AUW|5@85bmPT{}u0%p7$}VO?BgRx<{7q;u`Fqjbhbt8k z;0uwY;V$A&jpchy3X~p`RaE!$`FMZQ2%`?KUL!ZRFtz)|Cx!j1NHRn@!1Bmr@-;Fl@ga_0{7>*vwpLRW zpA+w>Tju0yD4lhMqpdc{IX9%9!n@^5>{Zr=)u!c7e5Vg-bDgCFM}bcnD?1$$mI(MJ zwdvU3=HC=dXneAOS8A$evB4^sv;N7e`uXFZ(8}*Y=&OgLTR7~3^z%RS32bC3DALDva<2ci~As#wL-UJ^;a)g=y~!DJKDWh*=l zNV!YaMqAp3w@gY-Z>Jen;LjD{!mS<8R?gdvP}cD^1;X{PhSUU`vmLj3C9gc|QlYfl z8}#`d&?{8E)=O36`1)S8#zF1vugO(eB95FN>cp;D+ljf79tpi^oQ8Pwhez5m7B#OO zIk~>hyJI!6z@X87@r-$atm~kP{U*!8K&~t+@d0bPmk9I0*EHi3?{l9!amcjJltx1> zH95UHeuwOS86n9yIgX^i#9ATCOQ2vNJEjWJ*O@$9LW541hcJgSNF;eDDVi2*2p^f0 zmn4{AHtIND4Wb623BR)NEWV3+&CVG8067y$MVQ*9sbtB}B`CZ?(b>9Cm*^-?Ms)mc zB_>cCW|cA{qDXlt-!Z)EweU_YdhF8?Zb$|c2G=Z3D)$Y3rPb%6MAbw=od9IU-;u$Pm zB@(mL4iRyv&_{q#0&Yr7v=@UIf&=}@W!>76 zFsECb2p<=6Gd}-lnXn;56!-FNDVa0CZP!HsTSO5UCkgn!7waKl@_lpmy{b2zNJLL~ zs^d^w__0U8t9|>#-8fNQajOaMOBFd>V!{n_QF>QLbfLcl2w}Kbubxz-F8q{RMp+zL z9bAATgk0F+)Z({n1a|Fgg(?u zJU>h-u9^*j>(>>WY(2aQ1j!Gv5!+YEzzqjzWYfZ>ww8KhfO^OViiaZjrP_H0nc#E; zstPKzML4+lgP_3g3UCM?qcmre_m$S#!A@!6a8E9zEAY2xR3va_cik$P+WgQTO zDnR?JX1=^wbc@L>|kxL-)FngFIZji=9mEDC}z(V!=vu(yH$%N~_v9K5%5y?(a z;@=oJK!}$LKjHqp`YRB)FaYM~_SnbI$W<{S%|LCxHx~dp*>zXHThTuKOGFWRg^gUG z1A*3kvpN@fxMZA#;rum&?lRES_xm4hU3pZ^-@C3Phf=3O3DF#yq=`zM22(_#G>8h7 zR2q+HJk65^QD~5&fl!H3X%b&Fi&RKTner7pI2G@|`pBdMOL|tRgDAzZwsb^P=_EqdPT9O z3x4P>ks_S|l(f>de6M1fVPZI+a2dC-LU!j4?NdZSL-;E}zL>&W^oAYrenu;wUVeuvsmZ_hmmYHM*(#NL}E?q@i zp)JZm$g;PU@}sCo3u1XDNy*W95fL+V&qol#S>$hqmf;L4!2}&5HH_wTn3$ignZpNa$WCFRJAB=7A>_L_*9X za_h?Xzi1>K6D>FGavja`yX_tB?0I-=&fl*F6w9Me&a9%3hVkhx_EZkPrtbb~-PHO6 z6cNtzFCd~kG>I$a?pgWaAQ)_Gvplv&2Zc|uQQPh18XKb6%9fcE6U?w=*wA?sajV3DE^Npgj>{-DRxKa zFYes$A`(QI07K8?0wDklD_QYgZCJwjG@noDCJ3OfTUsvaaFtMc=bA^2qOWUyN8lEfKNg;?Gs za$si9zw)BTB2&E`YF{AMN)D%-QN1-H{g)u0lTtj@ld*w&PjsbBBoEEBtS|rabiH9y z+(SQ&6ElDRc+H!eWj-8n5JR(4LW-LhQ=S8x7l^5rk=ZinQt>2l_!19s2ozRL(#<*Y zQdKNMqUjEf{jM1Bpjb(wm?-MxL2jh8rz1x97mL{X@RRq0AyBq&{a?2G=&3Aw+@;^Y zPd9!dXNYejeKx}x%0iz#1SXRhxd%-`=RXVIF{{g?sO zmR#+=hZjd5@NWm^#W@vxfZlVso-@>%83f3(j*V4aE{z~TfTC;14yM~>1f!PeAWdK$ zc_Lb(vF`ximLG6Pw?Wn zV*Oc&aQq#&z|QuE;#X>W4jBD@QTul&CpDVcU}>0DU18N}+nGxSbad1y#{+MZ>1=85 zq_bnqcSt63$w)im1;G={N9$ozAPs8|P`#R$U{|d=pF0Cvqt!7NEIzxdT$JLZFCDq_ zGQFOxf(A*h1N3$i_@Nv!`v>i)R82WbA3-QP(ztm=pb<>N35|hG>CBRX5-YkHr2CI= z=-(M(QDDk|r5zGextBLb!G>`U_E%V>b{S#(@z42B#^kbRh-@ z(uJBzKR|Dia)TF)wX;J90JMyI?+Z^Sn|kD_bS1jC++WCh8yYh<1`liB9Q}&J)vNH> z4&X;Sf%W8|?A_Z^fw-T<4r6G0@Y23D4lOd;kyXgr3KDKk#H##{FK%9pe^^tY(^LiQ z3>8RSu=Ntj2t&el{0QLYj3;R6tJZS|w61^OPIzMxUomf9KSMKNRFK395Wais7z2t(1aOxt~f!f*d+c;ZcP+7aM^MWIDx z;uGnFgY-&Qv0fCgS>Kw!0Pxw0wm6s2&ZHf< z+DeFQN-3>r_q+2aFvl%UNq9m+D*Iq09UkKu+UO)D8~8FqUmI5Fd+eVb&R$OSPi}B< zizvm9G8UjYRQa!b*BxLHC)ew%~FS!iSBHyS})dL{dCZ8y@p)sB@|PAB@T zUrC=+C0Z>RaNP6UR`R;1NLzzaL2Ns^(!1%bwGwVsy^J0=>SW{|2vtrmuYbi5nW4ui zQDj&xk%r(-IHTXo7--_AKskvL%Xd#U03K zoi>2qOh5O-0b?AyfU7u>A>#C652+;G5`#udhkrP>jU?I4-^&HhyG+NgUEs$2wcAIs z#Qz*d>frLv+cMI0sQ)|ZCksim3>kI( z?`9^vA3k631GYD9qQm_C^_uSB)Yl9A^310Vub>%LXn6Y-%#e;HQf-p7->TB-Y`P|Y zLM|Nb_qMyYJg+)_the+qH*k@7Y>**{=1e>n8jc+f^`a%(bJ}%Ma^(oW~nc4-CQOx)m$2~!RuZ0zGdSW(77uoakYYFaR^jrTNJDpfbP9;iCQ)$ zcs40kRs?3Wp)cL#f}stLR>j}<-&U{&=~9`$_~w__$KNOh`dZbI(ZOl~lcVs4*(C;n zr{!RSE>w?7ocJD%2FGk1OgN8PmK=CWTOZA_AKi0mIfGxBg&p%xjMD*_ z?12%hxY#kI=BuQG zuc^p3**HsMNOlyek-lkd?*fh`%zJ@U?t_647DDJfgo`8*Ce%V$WuSn$v}Nzd0(*%2 z+mZNx^2`3fQ!mSo2%1Wg5)Z0k5=`%MJxe8Gm%jK>@@%U)A`7t=#1u)$`BbP_fsPF8 z%hz|d8Vuvnn`UeF8+a zU1|wsA8{Q({G^Y4%|(}iC}aW9a$Xo2a)D3qVg|tg{Qh)+!CJs<;Beve=xdp*RrJv)Rt`wattuvoiOYd4cwBM zey5rBz0{AtTK=LG_*e%>>mLDs=fML!7D4HtcBg{7pcumLD@0n1;jY_=A4N|iGf?@D z{F5Ly2EK>j`x4}|n5!+Xe&ngq0>{>`u!H%aLU#~Z9p=^h)Xx<$&C)p`89Q*2uybO; z<3!GthiC1s3HU|o3G{0CxW@s!D|8ipV4PsWA082KRB=5~hn-%&c_Hpq=xo68%(@JNKnQ z7iZOXaNNX<=dWm}#$;gl5*)>$dF&~36=NRguUY|rZL(fb;FC--#x8wE1*zDg$TGQD z5`7I*%iY*CHET@t~EzB4IPAt zt&5a2h{sF#0n1_CaPh?sImZY^iD<XCml<6@;ThAzpy{lf`MmrKzGa?wV ztdhH*0h`jio^N|22d&Q+HL=SHIiwOPJdtLOB|~MmA;!vDyUjAL!Eas!La05zYjCmB zV?~se{E!G$JO;9*>7(I9Pne0#ciGo5qCZrEZ7Qb|_I%WCj zJO8%!{fLpj_dH6LDny0GOJ%Y0g3r)I2$hlH>A3tC!SI3DkS)+J6 zdfT2qAS0q>jL(>subW4T!AHw3T82z5F~_!3G}v?94r=|VncVH5H8z%K(ltbzKGrJ0 zT_(A@Nr~OzZJOA_34`d~2>{&xa-soM>3ssa1RglsR8Nw|W%J=gm|ClnKmG0RHjF9z;8_OxU}$6Ojz{l*m{*`ZWx z0-MB$@jAo7tAU&P>x6Bef9ga*rslD%SG5++PoDTx%ef=O9kQ?+A)188NBf*o1D${M zZ*T^Pw4Z3vX5Pnzuas2^E2rG&=jG7a$J?OKbsN+>EC-}(P{fj(Lj%I)*$KpD8Sqf6 z(Vc5grO(IcM7fbycXuoOegf#NNP2C&Ub`vjXqDd-bVu_|fNIQ_YRQ2(G2tyEba9ci zjXHHE{Y=ymsZ2mrZYXXVW7ac_6suM_=)8Qw-Q>r=I9OtP#p-sj;bQdetnvM{=c-Nl zjUKkr0^iv@(0L@g5I=jK^|LJ_miq2W6iTh7zietb=ivDRS?7noRC=5N*$_0{4zPty)fO%E_PxgmWsdUX6k^k_~+x;Ug zcBWY3%bw@7I}xO1$h zG@|>b_}M|k>?s#3Ok@}arsCO$E7DA8i#|*{k$EC zJz)|b&*UY_h1tI#JyUv(d`1@^NHQfux7Y)g=+m zs0E9h@A!Z^o|pVMwimcDKV4%qk1y#c&Som+fxC(U3C6^oSbLK(wL(RJYBw=Q_)bZH z%a5U^&DGb~g&8tz(VOwy9%p|Jp7X?!&1LP31&{sLrFZS`bHPbbzbR3G1?NB!xXmJnJV zVNJ&#N`0TwJWu3Ig`#OT+c!R5>S$~o$%`_ze7~>ue+MZ88^xEkdfaq#@LLd2_uJ^6v+&(V2Ep6dbuyGD4vz9!baJ0?3#{2yaAewuh{4)HI7tcF? z216w`EzM*OYba#J)_Gd1law}dd|dGI14o8I+C_T{%?s3j7cl^0rjZ_dzHv-)RY(X6 zrg=b}$YA5P(S3UUc1z)D1@~Vbm1{=tEqH%)wg!&HIwC|>zja{F%hh^~Zbc8}?XPhv z|5ML%OktN}I8n^rKu00fC1Dn18)%U%+M%l2mLwAw&NFpGd}@uo^OXf<@cmF5~T_LU-zy~uxn4>#s zdascan{klK(?WAUw|Kx{vi`c+W2?Y`yw3?D4pGq|Ubd!A!c_qUQ^jgBsiQ-c*F1cA zxA{*Ow3w_#n)S)_W(sW? zHQJCCW9vKYEE}```ABih0pQF9y7d^F0d+U)oGZl^M)Ps3qMC;8&0Nnd6Y#57v~|KB zZ83*5);R@dzHR-yrhJhvK|kQMx$yScjm>dk=Qtg*yIbq#*-78X^P$^gZ1N|E?i7xG z_!Y-Q57+SSw|zIWjaHmE|X1%yj3&UV)w18Peq?x zD%mUsga`KiT-OSX8N22k6<4)>FY2A;hJmnrWdf!Rdo#@spJ9HZ`vD0uLFVVQqV5J3 zXycjxN$S_X5Mj!J51X3^)fvCiatg#JuZ<7FYV_+d#CslYFJB(2(P@-S#;(vg?0*XG z&e@x}XiI0~O@VT1&DAJRF7{wb_;U?eLp1UfHI4=uk-D157M}APk6pdBKy%~trnvpS zH(RgXBm6<>oy;qrKD~~%9ovvZUGuBWrO<~VaPL#mcgn3v^9GHRse2=?X$)Wm9`Vdn zdE8QOkU3(mG%Ez3Yo;}<46y1p7;*gL!4;xF2B>VzJe3zpyMu@M59xn2l<|9h{_`T? zx7;O5&zOauoFrHWUb6{jgA>@uETp?To4x%+m1y(#`wowUDFkFXkcygU9%F0Fy~WucmuHg?&f5RqU7P z@T_2n$Mk`;v^9iU@pkFvo_z;NQtzgb=vu~%Ab2FR9sQp7?0mr`jt^^&M#5$g!zqTs)>~038T*W0Q92m5Lu>A-hLPmxjHxvZdJQot%I8 zKkN!1Wd3s)RzP}=T)L{ddZ`k}j^rvNb<5=W%+jSKjYb*F^XQ6V50_;5?q<1Sc%7a{ zejMd!P}u#wY1%$S*ab&&ZN5${V29C!5ZK4++{g^ojUf?k3T90<0VM?w{y+WRpGKXq z$aqy_!ZZKpS1+V{LMm&KyEDO(2zG`vj;`9zqu t{6rb{h$4PyU_HXJHvZfYA>wjo^oAGwvgxXDD`vx=ts8e3-7v5V{V%-J@CN_@ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings2.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-4-processembeddings2.png new file mode 100644 index 0000000000000000000000000000000000000000..3649f7a563fc0754eecc9557d388a1d406bb6dc8 GIT binary patch literal 58035 zcmd42byQW~_byB+hmuA>x;vGQLwAQDB_Sc5QWDY)l1hWL3Id`aDJc@tAPCYSNO#?} z51((0_Z`1G#{J`taUI|}oW0lDYpq$&eC9k+n(7KTnB8PQF&5h~w}_vZa7?BOvgZA1-fKE- zuHT;v9JpBRtLHzSMTTz)DKPZPjUo3Cl0;g*xLjBI#_BPw14lc9-&;dQzawbY5gUz( zUWNTJkbC<*s_akG59v=YYpz>JaHu46v5*LExr%V-W}@#(A?-YUuV{>fq}7|L#)+rI z`xdT(jT-8mPh`{U`jM#Q`>yFLHDhe71uvu&T_FlrCZzIo8V&9xYiXRZfJn+Pv0yx; z2O=F3K0e7lcqLK}IF7P-Du-n2l5191YW&X)lH0#gbnVsmL~94pn5BGwbro2=7?|E@ z{4Ehz#XzsRgO+z#xV`aL-zy5HPBy$_j1hV|+VRknkRaxo2+|@eCFzCbqf%tS+*|Iq zJ!rg^Tv5K#2m}c!X57Pn|9gZVP0As19#g6W19Q?9?~in2VlFjlMb#8T4n^Tkn1EHJ zq3W%`H2SpD)UyWeZ&5aE2WB?AQ<%?HKeDiM;agyKtc1M?g+=Podi4xO;pj}pa`sxt zVNXRe@(!Q$)5gzZg_tHC6&O9!sw1Q7m_Cg5H`x9`Q+cPxk_mmQ%`CD#7K|+bTXfktTas zmc*93p(vkeme0JyOKI5DshCYu`Z3~Oc874I&IjE>L84TAtcIQ}C4lE~^x&vX)JFsF zHxh3Tn*Za<$Kmn}y}qG4iE;JILPgMscNr zq1Nx1(%oaEI7&mJl9AuR-KH?8=b1G&D7w7Pp|)t@SM3R9C;gVF(C{;nkvEYPj!k>!y)+a!;g=`WV}7wNR3NTb)pXBs~PGuv=AWXu)37z&s0tt6`t z2kG@3^-Lo4-qc_5O70nqMC|vI|2>_O{1dUc|M=cf4PzaiXi#>W`s39qdIS1A%XbEZ zf5*G zF0yN^TQws1kZdyR30(8D+KcX@_^Xt*Wf!Bz29Z^T@4KQnwPKwF!|^cbq$m_`6Ph0w zx@5<+>GBZvp%&#)aG+MSid5k{Ad9v-RZ-u6LH`>~2!|_3LBar>^STY?^x{MTA*J*ijp0spGs;}N@BQE7No2ENad&8 zQ$`b}!$b)dR(3LWVb)u$MJyPp(Wz{yUTl;)4Q~&0*ZB>z9i)yJ6khr)tuGZVy;~yL zlXJs8<(CVwklQWp(5hzLXR&8FOwmdCk|NmK+gsdA)jQOSvRvP5pK{EWs~3=Wp(Ur& zsUKGKvYbxKu%Oyx`ExI4s-Q00*vkzCwR{rIbWQs*=Cb25YF)eXJLL{~IU5vXY-6{_ zzL#^4^*kkcs`@eNsoc}lr|6?x<%ea-W1M4#8x)6A8+sf48-k{~?d6kACg6Lg4*_PCKUhnR41J)%nHD9 z9B?eKtw|kt*LX;I?(<;_5RtuqAZ9md^Tu+?CdILKcr8P#lDB`d+u(8I@aM4;VUx^( zH><-FKc?3&nQk-DG7%*AGi5X7s=ZTPREtqN&Y0yd%;4?g>(hVz@zr7Gz03suTsu>n zfaw3r$6-PYRY%zRQ2N(QnXqsGzG5srWLkYj5_4b^O8jL&rz~>`aQx zLxFS$Eu)9i9Mf#mF;i=LgdN8-jI!apu2}>pXbbmw@vIy#t+_3%ST;}Ue%rpB6`9mp2Z^Q!h^0(ZUM^{ zt6!f@{dOBkT@e}Z47Shw^HA`a;0L27qslM!ey{yLp5ug*&_{pBa2*f%J-^M{y%^c8 zpdmPjGI*I2=riTF?7em1D=_|)BXD#4LqelklkjY2)0gLK5_S@q{teI7{DqqzG~-_O zU3dpfU*zyPW=!>&^rc_LUs+tsTx}cXyfu8gh-?6Nfg8eY@~j{G|88hKmnxQeCFLn~ z+4jDT|3yLsp7dAgw1}C<_p0kRA9)3q`mMeFam15aHCg3pj@$0o-Z{^?_`@}SQ9t&} zJ#&28Sld{%q|F}Uq~n6O1;w(I@l5gxau4MsqsR#P`BzjqIK8wt2PU7nUb(6){_Qa8 zR#})P7$wjqd`dXWe49a^!HYRg{dKAbt)Zl0;!KLPvXbId8Xu2@%SHLFc-i~1qOu8Q z1GNl+`5D>EsCvQ#b?wZKR~|HRiD?;iKpM<1b=t`kZAu=N9J?FNjLqB?lblsH<(|tl zYYTimpRJX&%P}JAE@$nW=B)ojKR1ZcfZ02h@?{h6`EzxvfupRpnT{E8A7vjqpMGgI z^Z2Ug)k8~m$zNo>3beJ3i3V9KiVtGiJEr%NE-+5jZavyjXqK_yJeAwa`%zL=zSBM$ zT83V1r%I^)C&Nkf`Ora$oRnNkf2)m)O>O`ESHjLKA%=a1nKtw_BQ~G@Ewz@=mVvl&8 z|w)nL7>9>u)59aM(kH4-osXO}K@YZvp{G#GW_nt0yc|c3f zr>ni&EjWkwa0JtwFKU&l)>l@3_v1WBdyr}rX@vPnZ0dE@VV`sC+|y6%&g^qXo?l%2 zj^i^Zw?6*;_)!$ci|d`t9|q5+FWS#ceo|I_DVuE$*r&=CU-oM={WiOLK73>{M_t#l zg|rC2l!^%%i?EI8B~J`r$GMC%jQgebP0ge~zu%G5lr!RvD8+0*$CBch)>ffPf>wMg z#XSekr<6imBD2rv_B#selpLR_Gi5jar1~}M!?rm(H6m4_kz<}?C;sR8QIGJTd;cog z>OJnyCbnxXTS5n4{%l8x{1Umx)nMk(u;f}+WbZKX;d^tshB3xxw==f$jWB!siH7&x zW-UJShw{g2v;8)o+?<;lOuWzcv+JK>Jp2B)Y_6iw*`;L#H&gP+7#BI*TPA2eWc0*KszQ4*W0nbM6-0%{^DX9zCf!D?0oh5_oz! zHk3O$x37HhbmwqE(AT-H#nWe60PlgzwaM(4!R_x7YtJWllXiFeN;)(&BsDIj&#G4{ z`X|zy&FB8!S;4szEo2eD`4B0DxW5PK_n{Y3={KYwzfWWNOarcxiDd(z#-4ue(vA^HBfw z9+d)ILz2>wQBnf0I+kuvo;bPNI(w{3kSl->FkBQ3-I0(;7@=QeC2fWsF#ounu7QVv znyQGUvm=+em9xbYE^kK{XdWanZxQg;@rj2yjklwNle>twI33~+5%3=Rn469Uaf^q& zIGur-CXI}<+Y=fAE*>r(Itfe~8X7S-D{B#LS^0m4gTKV-Y&|?&M7X)1J$uIWjGxQd z&4!y-SXh{whmV_&j}zR%>F(p?VeZZ8hKfO-ifGz-KXEXWwQ~e#2A&}y zC@dg`xc~omoi|FPV>Lhr2-AJth* zA|b(%lw_rJy^%MwF)s8*&aQKBt3_+~5-X!dIVJV{2#uy=TxV`b2~-So<7dy$v}Z}T zie$O3bf4wddu!!z>G`y^;x`4sD|XitHPv3{S=(es!ZwvPM>|u;hmPA#N8~w&rsopp zmTD!5jCZ~wqtGCsNYGXJ9sflc7iwP_$X3C*ClDL8=ufqeB*=@VoWg?dZr14SdMB@7feyERv&7T{B-uu z#-SE;Ln}nSQ8~<*-#^|_#D&hnlrTv1;5Ex0DsmiJ7D_^l>ufic#43Lr(=7>m#z_*+ zM^@a{!97_1H3iV!RQMT!&QuSaW+n?~?mP*hlGAXdtEnPz!4zJ7fR=ks@{mrW$OTjE zPE249hiXwb85*=WxFj-6?lM;E1ulkAXHG--n#!zZlx&vH8l!Ix3x^NJD{-Isu)C!| zB(E5y9q^El@nFKpl+F0kyi43_qva*C)DuP~qPOxeW61BmcW!jyXFYIRuC%05e=k5Q z4huqpWiaKn3re%RcG64PejP;Mvq2$;oDnq{lkkV!1Bq_KB^%hjr*KHB}&WXhWMkv~=(jq}d^w7Z*ejB!wTk zYjLMQW4a>Ay-S42Vby_%4{=L7Vi|Jnapc7d*Ci}o#4go7X6b* zh}anI&BpA$|48Y5!80cwzTT0$?_&!uq~k^N5NgjraY2Q|g3N%p@Ai(|PRKW98sA^; zYeYUXsB4YSOwWUn9e5Pr-Jy=|AEmy5due{-AP#pZvo1=N%M>QdctV3lzyC9@0J;d~ zVnoU({SnS@el-5hXv=)XW0{#CgpI@Gkr;Zg2b!t>u3W?6xY3AQyF0YkNTM4_X;*3< zrK&L#7^0O^NyENw3ey*fqKK@_e`Y890O>$$%gvUr>poR-OjntQTqb*~x3lS$6Y(ef zIK`fz$%_fPV^jKFe+dg(#3>!vP%%;T_4(ZOo}OR|*d^X$6uolYZxx5`D}DLJYMCn^ zRf%px_XaRS_Yz05D|@6HV@3sXqUT?=QwX}IqdL(?Cu}!rq^H%mcRic^qVy+raKhgu z;8J_?BC=}Au;pZLB=sKkoKG&-yJ~K&)3RYV5$pbl8Zpn@@@5K!BE8-0CMPJCVF-SZ}`;1o-Ft}fq2&>z~O>78!6Tx@t;&0=7RrPI? z;0?9Qbp8eIvL?^-mQ}1YW{YnZRO|DR-25&xF%IXrDh}Kw=`X%%L2Dm>sNbWNQ|SNj zQ0=9J-t$+V`q`z;iuYppiP_^{Mc*Cns#fjjl}#v!_YwN=ppz2P zW3+^u8L0khiH#=bRk;ol9FLaD91(rXVZ?v=n?ka0S#_}8xZH+i=#?MO0&c3rh?|~7 zmT5Fns@4nl0yn)gkL5k|v2O}Ri#3^NZ0=d>qn_ml(eBj;Wt(2(f?B?U%WU@+N4{_h zYMrU}#%bi18#~c-subzi)JaBeM@*k}eZrj(CAJw$URx@hb%x2w-LiqSmq(>FQd3tv zI?9)T=+0GMSD<&+@%pfv#K6gbpP*g#?&)jX0q$3@->Vl|y3``g3EYw<^}D;-;2tJ6 zjUtWIw2?9@F5=wwli%mMhs#22oMzo68V{Y=OxK5teI;k!s98C`KU9MxF_ok1RZ?D= zS^I6{`?h_9n=mK!?p8%jGgIio=2T6YVU2YJTTGx7uoNUT24GUWjtO*3Ws6&=$aso! zLC+L|auwxZg9umu83dM)X|E~@G=HhqP|ZC0fM; zgO!sJ@`gP%ha;E|yG3|-i&JWPvCQPgcU z;25uXl-?llMeEi)Ho|bhF28fPs0z;cy9brEc|b?MlLJ&@tzOg0yj0dg1kL32EK*y> z9fF*Yph1ibylCJ$PAK^M!1j@XYa!R65w;LHRXJksF8`BQOQzrC^{+%-F^TCXurhBi zX=6yo!jVEP9>2y62QE7_E2uKeqERt+o6}*m<##1ZM>mGgdA=_Vo-ztp%Ui>mwSpKf zOhye|OlX(Yj9{0(}F!@qUp z5JX4G7QCJi3T$=ghb1-TVmSP2NbOB!R|vzpTa1c+rz4}mZCUwItu4OebY0xl*+k25 z*j-bW#ZK9;arj{EkX6b5&eud7VIp*dQJShuDA}H?Lc4THx$H^bCZfXM3R6TpLz7RS zQUmBD{|=>y=K63^)Az}ctd|&_v@+qEbIs2!I~I%5O_dkB;tC7nMS4SCCdqLD#d6P& zsD(uOn-bX!Q*PBuC@M6sTmqMy`KRd6D=KNNq2Xs-X3b)bvyHC;PUg`?PM4AoiBseT zqCyw;mcOsZ@(&~y)M?~OxFO551YFwAHu`j|BBS26(js{2tptwfVy7Kw+z$d6sUX{wkAJ>kPA4lEMBfkUgztm3x}zT?aGRBnfV|8dUgO*u4n$2WxQ}# zK%L*QlZBLKwUyG@ce5+$+~-GRyY6-7{jVSJT@$Jc1qWR0DlG8XkH1wKE7z+Tsff8e zKOO<cGCXY{pQjv)|Sk}KrO(Jl@vTO@=f=~SVgjg zyTs=zGo6@g-?+~sWSbjjV}_OAYO|;m+))0EzpEJ#KQ^w=%$H8HwLWDIPvGRW(O`Nl z8QAR4v~ptDa(R@Aohjm39?Kk>eR#5%FuktpOUtl{()rT2;^|nqR~DUIv_XQzS?Q%Q zOy+d6c0|2qK=Rt(hm(ZRfI9HvL!;A=yx`8G(#EL)aGo9NgLf@TCkbRay(r?RKUgZ* zZ_hUI&8Bf0E%Eaz45?34KT#nQa9>dsjphEFla({_JCgB7OJHE;hnhG`OG6S#ovlv0 zYVnz~D2IKG6!nWUc){b)V6b8M8;rG`xS8pc1DA8;$H5uzH2WAGwcBNmejm>Te%cs+ z$1b_?(A1~su#bPxz$VA1xN&=|BHd&<)2ZRx9m`!t-(O3YlOHVKIk#MnlMB1OVO(MC ztt!d!?S0K_8LoPMIALa_K>fTH-~-kQg#;??-%+MYWZT)#W|Le~fQO;=<$MQ_tbt6# zosVI35xLK1eTM7Qvqiqu%Egd;9Di@ZJ`%Xwd6zFXY@udQ!ak0a_iMRUVL0}^`?iU( z$%-t|<~C$D-(E8E#h)o0==ZwRQ7TBGC?o9&CmDD|%f{3DQejyy@O8Y)b$OLtVS7PM z3V8(ZV*%|t2UCtDW=3Z1$mNL1`sIM*m3h?hMn$Ev7J~q3T}6j@?99IYTLWJEiT5Ua zS$+iUx}}dlzzJJI*heDesJ+OW@!C82jQg}|+eM%K@A-) zJZ-DCTp=l}i1cS&yWaNF!TQLP$**BpPjHvtC?#(SS&+4Gg!ti{Dl!mNJgnqej2)ItA6>-YKY zPV^;h<3g&myR@>@yR(X0)1OMZ;_gi+b+}Y#{bBaSD${@Wh|lUmZj21V!ryx zcZ(HFrbm5W_b6te!85_r;#(*<0`AcfYHXfs@~XZgPucA#*CckxUGyXB)ECCs*Y(*qgo&vfvbbsF>}!fgOS6cCyaOll&A;B} zCGLmepXE9w@z{&`_8%s{imYb2FNxzC?kANhXhrIfDQxDmES3hA>z2{AXNwnZkPSp& zbR2VtOg~7UUukE|Sl9AMOR$GrJ`(bCG%+$?{gqBEcuruC`9-(%%k8l&S&~S+sA%I2 z{%4H)=vVm>R4=jgyd$ShKRM4yh=$BI`F;N^#?fy#`lB)$kDEvs2&wlowf~NkYI8Yu zd~$_m<7$qd6`AAD+7BJQHUq;8dy`$d(!1pq?_atysb{f_>_}ShJ5|U=5xDK%!7`yJ z!(bqYF^!Vt5>CF(Q*|smFA;M@Mu+@b5S#;mmm!_}o|Uc`IYt;^wxQU9qDIi&%Bnm9 zqi+TlndG9#gXiu8LyNcQ8y~&uq^aatx!<^ExjK;TMgCDyZd1cH;-llVepl1qhCiD; z&A(?`%>pm=!k5{I^vE|%oHhm?tn)p=Vu1;Z)UF-)X$GToi<|~ktvT#H``4qiBpE%I_+yD5C;NVG zOx2V$b(_DJPU_^?9xJKqFmDs}-g#`tKg}sHtuCaHw6*x|dB+Q4BN{5d%@1kue?2=< zR{J2pvwQy3+M3rQ1d^u^;@IB#l2H3m_|VY!-cfRhlSyTN$i&;14c!p&{@V#ky z1j|(r;2W3OQ4Sv&^7j7f2=IO2Q`jzsw<8KLb;~EbO>k!g4U`t>v18#)~%lzOnEzBp>_5cm>=iRtBI^WK|$ zk+Pcl>lFJcuLV6l@dc0#rjV9jt!4Kea-PTVuTLjR zwL;!EpFL$ICzSs|VQWYte(*y^x5?LYW4I`0WjL|P4g%LL*O$&uhYG^h%M1OX!`?xh zns{{cQ;Xv+mi)<5E&8Xw8a}F?{Hbgi2ADDGp*V=!QND*8asUH+<(5V83Odawg78o< zT%_Wc!Ct*bk4z{%ltA$q!HI+>frY`1^e{R>^ch)BH>1*t87fp50XMI||MK_c3)lUX ze%&{U_#$U(g>kZ#Ki0sM-Kl$ItayJBg%^uNrZj zYtBB}oRV4oo?c?|#p|25*wXWh-EN(H>99?2jz=-X?D7hfo+;Ri)swfI=NdfOKO`=R zop4`Gesa$3OJBTR<_@H-J4}$g%EKgP>qx=Ry>o00857`rGV>#}Aq8p$ln6v5J=;p2 z$kN8>1UF~Q`fZChET zOb}=4ETJU3+f0Md*oZ}}3?39nW8G*FdF09u8m8o8f@pxLip&B|!>9sI%-J5*vOEFl z&gakIQ{9hG_390*FHUx~75Bb(qhod6!Xo(*#hjxE;!KjX)8CnTzSxlmwm;xqDhxyu zm1Y_sT$jXfH7FhtsK4S11F=)?+1&F_n}$m{y;32!hmsViSZ^^M3+xqSL=iG8OJU}- z44wU@hRMx6z`#ZUX2t^4H&VdA;3<2)!51$_qpV$*?Fq9$HOb3^7-S@S>HeJyhQ#UQiyq1ZMH4tC{l5m)!HU)>Uj)2%aV95(w`%KPkM-H zwGs4j4~5%8tu&B>;ZZ~Q-TS}F4byT}0a%B#)^$W% ziQ14PShzc>^x@aG>*5yV`Vw`fNep}OJ{+n`=Kfsr(obWkkTj%IP!KP=*$$Hw;9fje z6p>K66_(o`kAhzpX4q28H^b3-avY zFpwGGMTe8~Pgv++$j$b>z8f|A&M5sofFJcz0?uFG1>Adm%-6W{6>B{NS7Iks(~7B) z_c+tJnf0O5zPR&9yC3V`?FEbXR(&gT7ZdX#|&Zn+%-FuU5y9BDjj}2d%aI7t{}Fj`ReRXN5#1-0)fCNZkTkcl^FJRw2lqO z1)OFdq3l4BL|hVDU=_(SE5M+d%(1lm=@7wD?Fr!+jyWV#JO+esvq+o`N}TSMPCC}U zI2h5`n)99oWbef(JV$i>;@w7KAx!s>h6r`cn1=+|Mh9RVWM)2>{xnD75bIEWM8N)~ zOuI;VFWus#1m=2@3@|~@zCCx-fp#EDupac7#}F48NhK!}7iA$u!vQcsXH}JqIAm#n zlcFKIoRzM^1)=yKVr(_ToC-3BHxR=k%~ISM4+FE5A^h^n{F`$697KAGcOwodTc1f7 z%6||55=s&sAj8X6YhGS6@a_l71ZVJ`Y?g{bHxS)8QIghJBoMn*|5=ZfwoS(^2;@sbsDPH} za&nm5@zEDA&gTb13bjwZla%XKkUX#(4h0eB%?DX4N-qE(&7I9halu)#fG36wYfF4g zzP0e~)2wngb)XpKvl&+?HVgsC3g>PS3TXt>($&)s|2Js zJ?uBI$j4E-CTv$DN3Tpf%+!1S7Atl-hoLk;a_Q0^K*DStC$k>R2vp}b2?;hjM9>8R zoPuo-^2Gix;5^{z#%>t9@3-Ra3D(uvjTGZ#yY;djjOu7ineX1M(0^Az#&3;?#sDx^ z3l5XnbjMTPfA-hdV%}%(8?z25u9WGPX?qcUuU-QLPKwMd$a_%xh5O%69pRao_wan#e|J zp;Z=hj<@vDqa4>RGLH^?1w|#Wf3(iVGpP_-tgnC=?MavVP5kMYlwX6UmGgaOB|~-ldO=BBL_^x7b}Bw zE)x!32gfK=IiJDPT;Gkd@kCmgrAt5)KcOOIR!MVd-v#p{jUa3FH)`-0Ej{k z(b6Tj%?*;{4_V+4WfLr9ul8kNat({|qPDvWox1tbx4w)Y&$uUSGA{j^Sxf)|9Wk`Y zmIHq24LX%K{EjfSn0MmQ*0e%rG;y-cUq=6(HgwByWS=(diMlq+USqr0)=x&&hxKkx zAi|5{@5=njMCXhm#XJL0zVKv*rwCViMGpXbi@=Zt{_&cL=a}Ao&kfyjlP`59@_1l?Ft&H4it&>;N*U^*?oMScqn~a$Y=L&81F`^}~fYKv4a_ z+MSI9Y(qew`%GvA(795}uU_7d&>kQI_sWLM*~~k}Qy*2YP`24tvxyOz|5ybk_y_PF zRs2&efi0c`!m9??7psz6Jgc1raxrhuw_E1Uz|J=x^Ja0ZQee71jw0?$GvIZMqIEnM zHrxF51t`TRpX%GZka%jXi zap*p1hI<=i`tiF0GmN5=<77X7@eBLL1ecz9Z%%&5c?nW5qLLPVLh+CecLGCFDj^p* z*HfR(58W(d*3(!d9C;B`zL5slDJ)tqfE>gC^_Mw=8DanZ6q`~+H(G9%J}Hc5`(uwq zmG|xf0c`L|BS`I+jTK44amerIwS}N}_oW^P3LB(H|09!fkp(iyp`r!fOAP61=C0!YXq`&Xp zE(Y1MEj=R2JhkVt!bTaQ0ih$#OP&)T{|gNy>7i>!G%P6^H^^bkrq_VnBgrxj6Om)H z-9!={{BC~WO>3d&*2>eO+(Z&)D3Y{As={d+5Wa@6u5kKAsW9+01~*a5+(%Rs7mr8w zCLQg@lORHr9dmB-KXH31D>RfL2C$fZFNJr|Vidq)0v#8OV!TluuIeRrjru(Lc)L%= z)k#VGp$u-cOlN)XJ2$R`*eD3Az`a66paO-V_S<_h`T$H5s$~f!&HAiTLZL}5N9@&r z&|++sY9`=fFK|`?yk1VhC5V86H>vu|QetQuCkde5bp4aTJ_brPPDQ`Z#CQXiPo-@w zOmlOj1ot_Bo7x`MjuZspX<*z50FgtU6DD^b6_=7i;67_YCG(f@ULpo$Ohe7XSQ zKl<#Fe~}2X1-a-hT_(tVNnBsJzZQMQ1@fPm*AMLFp(=snRCO4WY6d*3yRzj<*`&cE zpGoa?&%DEcGjPH8?Z-KOes-k?Jq;P;_if>F(L_e<)6D`{r{gTP%GM48}4m%TziDl^JI z0fG3(*h4c;Ind;RvtCizTma=uP$8WahK@rn3sUlY;EQz%<>MkFatDD`oIypBc+tOk zpgy1jc2d+MHXlkDxvEgQ2qr25fLt3Kz_QpkkgG!ZY2@{B z8GU@38q4f^_Z|4`5m?-(k;A{grXg3k>DPP$Vz2ctpVe%yj5XqvQ9BcdvO>9r$Krm8A1wHQauZu$fM+#=HPOuMT zREdEIqq4Y~=H+H=AdarF8B9aj26eA0ugytXM?#s_ao~m=L?(g)nR7T8c%>t;ke(XGwz@6WG5Fpvi<%mwz;(a}Q`mz-msg5&^C^?O%THXFc_Rq4 zI-9rhKw%O_hKwy9Q7ia!>j(csvq)Lr^`dlJQ1EcOV*x;j(gE8~D*{gIdE&7I@uQ3^ zBZJQ_=4?iFZ?n3+_xlA$KwiMP?*|Zz9jL9j1r+qge!4&3DIm?D4Z@Bch-e#HeWxJX zO_W*S4gS2#h)gImSg4SYHbUc%fhLb&Bx!G4Nq*O;i35{#lH+_B1NHL*e6`p2mAFjT zXPZPnz?bNgG$F*kTmz~i;h?<4diTSQMH(pSDdr1V_mTSk4o0_advVJHFcsfFUnJD4r_r}7WF*OO0r+Ng6su{ehP*D|G5Uuv4OIQ32skX9Q_uSyj z9r2j!v#O4-FEe^~epDG$Sw!qM;dvj44@xeXFKZRbPnm!5+Is1ZHTGkFp)=+Q#p;Dj z&b9ZD+}$aeK@IW4l-PX`U4EYKbpHMI#qzJ~qB9E9g)D$}mpHe^5xYf(cDuqtZrkHe zw64zr0EK$=gAdYEk6HqnICnk00|MRSW=($>-eBrn0Z%k3BX) z$ewYFSmlil-pmSLX8gXxB+r7{=k5e*h%W>6&=9OHQt$<6q3Xl6NkKu$hw_ZQ@V^tLevY=U z3M>!{y6Ioy%}j%n15rX5L8&EGya_GT1rP*>-*{i-=VXrsU?*W9?wCD7@dB4*vjK59 zI)^k$jpnZi*0VInaJOY`QHGoc1Cl`>6fZ}PSYB6fDc=U(0W%nMC!uxfP9nM|XcCcw zQ>1xA{P7+zpaBX_>4KfLo}!z|&w=Y(5((Nc0ja1L$1Wb$9Z0#5f^QnUI@|qS8DX2V zO_?VpXVv1pDXjE?+#=Hlis&d#ur*MdC^qf@fD+_;gg5!#%))I4z|qrOg4EeCy|0BB z1grJ8BlNVe&!7{Gtbz5eD_AMKb?~N{O7hcZ+GzpDpR!_T+5N5yQOv!$8{e5Yhl7HQ zP_~0(L|w$hj;}2?uM2AH_%P^gS+M8r6C>uIUu}>0W`MGx`P4Djrh3$ zC}=2m%VMT{F7P_L&S7fdXZ%Y>6>X3hm~<=1;`M|D0g9`BGUTRL;NO^NyhPwLaSWht z>E3!A+0loOa!~HONJnFiLG&EnjUGvJYYONso7uxj}ne zvA`WhWDe9f=qA8+3Td(`;L`W!Cui#2@-^u&Osyn3xi|2RtPZ66rZ6?o{~{Wip~(nN zJ@N+f&>s;zF*2DoN;)=(mGnl&qQ;BewQ@m}`GEekF#{-cA<(CSGvm8Nv@PW!lwmuOfh^HmaK%_Tet0m}ZK8I17oJ{hc~Ur7(6D5X1)H zCG4sZa5J3>`!X%EzG|76$L^oNMAWh1P6+MnocW1XCQjG7Ff@oVY^XMd8a1oLoO)rZ z0l?$?hK=$IlIUSb+gev|lsQ+>ceLLfQ_*_9+qB2Z5l!k}6*U_Df);5&Bjwp%NJB$fWo|-i1WxGN3ppAsX9gqE z`?TwC%4G}PCo^*^Ts*ysF&009UK;bIP!SNGWI^{KRBC$n4EE6zPxFOV_NkVpA0y>h zHFv5lc46u0Jhr3DXM|}7-EcFgXMC-j7kHX<+S&D^JW)z=ytW0}!0C3PAUexnY&Y6o z>WsR1M2yVHF!m}==f%s)?0J9cC7CGY_aQZ(hROam-f=TAM=(PF_D+(OXjw0YeBQ|x z@pDv$+i|Ey$(NKaE$Dh*e7vVK8*uKp@gqMZvosP>txCFS5S3p!l#&aPheyeK>K{cI z>bg;8qYSfhWzR!ga~!3pK5g4St+;rt6mpHXfgWIBu<(`*uxT5}JsHdqw_E#>U;SDg z`IV}H$Rdk?5F#y^guz+D-^poFgJ{{x%9K5!tXr)#;DX9j%;|P^uKlI zPe{F`cK}YkhZ?z0tB=|~1r6AN4hjc3;FeSY(VXxBd~GT=iuCtAM)K@}mlcIy$Anw; zLf64L2}7|6nd&95g3t&!Q#*UPpmsKKSQVBEBRMUbUd$pk$~<#ol(UBls?4C15-p$` z;S4Yqsj`9cVP%7^h>BGy31)&+Hr4D9W68-oV4>n5U~W#)G6I$mAi%2%;puXg`O*Ql zTpmm?XLeHxxL5De6Uc;Gu11AlZUz-Ovj?&qn0*a;M&Rs6;UhzE9nuIL>M zWXg7SiLyuf)(DkIrioEzBwh4l)cagcKLCy(;>PwPP8Sf}a?%>`C?(GoOmr>H>c^px z*q48zG0#9Hw*Z$MnF-3(`6oG^+`7g#?P|0UyzaLv_SO(C;j>n2K4C%)xNU z#+1E~RC$Vv4ns@2G29M8WPuQP2S(%N(ex;2l#kSbnH^p-FoCj+Ee7dVUjJ>w4Er|_ z2ONGHC-n<*JT$Usx(RPYR^20wFch#KMy!QN>lp(k(<2Nc30%LBeZ0^SA|oVNgQ6RDS@1+=}ok0dP=zjv^h}_@|R2f-b7ZRL3i*m+HBvjYT07m zY#8&mH>7bIC?Z+;lJ=Mtds3~ebUs3&Bc_V5F8^ABUY8s3dB{*f0p4HkF(9Wr`X5Z3 z^fHkWa-pd2eg+I?ZEA>tY77j2@1HJDnf83x4cz!mIN9J?@m7{^-k?mI(cvB8g|$4+ z0x;h$QyvJ=BiX40TA{->HK|!oHezI&_wjK!DknjgDF0;L5 zBF{@S0f$M=$6WNG77S*$%=8{XZKVf#gzco@fr5}I1efXVz*v9?ok-`Gt)%2@ZCQ7wt`ap@_)d4=h#MnGg#`HiN!(;JM!_go^=Z@r4l@0w25- z(F#gxDqBB4e%CN(xrIgmEKe2bmQ>e``gLUz--fhH84j2*RY&-REtn(L>-_xV5lAcm z$}_s>1!iFYOg|YDCpIqlP6RMCG)+9KEQE^KHX52k_^=oa)QuG=MZNm&lP!b<*+BG) zOjIm_Z-D4z3gq4U{tChtkc0!b)%}zBfQsr9cwFHTP~uL;yJH{gY4i z!4SmNw;zP(ana!6Ls}qw0w(VCGe|f@8<_Ck(H?QEe78 z;+rN~I2>vt&h|+9lnWH^tn9(#8(n~kqoBSxtY<8Xf=mg{)KgO~r~W30x;qlT0fc6c zDC7zDOfiT^ZNZj;otl4;S0Zfn$Wy8lPSXf|*I{Q^y59qw*{Bq;&x$f)aeW{45v=ZID4G03PVaa3aR9}2nrTKW(e0X(eRZk z5cJF(*nE-w#fI>^8_vWVH<&q*7wjoEiB<{@bmMmI{9^%_?5oxkR-LtI_Vzqq5^%blvlIr1(|bUvrZi>9f87Y}*Sqiw5KhIIfl0)# z%G0%LLMrXKf4j~NVTzB<&`Iy03_*@WKAq`l4i1>vp_Dex>Bd?9)t9+~9zqE5KjRb`4AMqy zcL7YY|9QN*(f?*Vm?9X;hEAAdZy5tAcwk@-L6J}9F=)FSE!SheCPh=B19UO3n;KOl z<{yaF47T+p1sU;g2#dt?2O5XKMZoE2VoC7^%|!;3;+)C$UPn{)jIgpa@nuD%W@YDCXLS>OnqZB0S??hnxai#!V5f*51CI3(WEc@^){+=8<6xz+Yt`VZMmHSzw$XJ2vF8Qd)bG=-0#V zIraRXN3fIMA-HS9D%?0(zj%bizyY+6q?wmTA&jb65X}=gI1<d#AcY6(DZebES} zyl=eiL2XnbsI|Z)Aj3Vay7wgXe~5s9Wf%G7y%>jI##z3u0QMN*Ks<@SUJFeS^8*ni z*uYw>Vc0{cDnPrc)X>|dKzYMMbuykyZB*?4lD+-_Ehz5vN#>>C+2IX7s7AsFM^{ye ziYQ}4q6ZF^$R{P!87%sgDSi(agBuf%Wf%6-sp=;Kr!=T|J!wFt>Xs7KnGrJ8vWcN} z20P+J^XO|IvS8uE;RMd&hk#=qdodKiZl9$F?b6_>0pMqTU#Vbx3exj5isAJeC$Ztc zyM%*6lNZ)%U)dxN$ey8>f_%3Cz@j1gr74}|eN_kU9wIW$yt2RG>r|b zMUxn#xf!mm#E$*0s)X4L!o-FbUBrmzeEtfmc8Ym;6$dT};1{D1WMwax8|&7pe)lY< zPL!)>d*0x79&^T5CmWG@R(|&6wW+Imnnda^^?|72l4_&Wip@BU?YbG+b$(VsiRa~R z^}!2UUd6Zz=ks;VyIbvt>YcByuh;!0yF8u;t76DOimBCf3gzKw;ZS^cV&@LCi2BPb z9TBEc9U@t}OHH-R_&3EfAdPnz6uk{iEtoMwlr7sg~xJN_p{j&mG8@< zi=x{)Ma$mWjSNcQi;V-d{*-0`4Q@KS%IN}Q=krXCbbeD;#v7Sj&&vTxS+=RJXC`PT ztnPnadhW68_DDkaLgrb-U!C3F+GeJ(0?1R8&JsZk6$(gN_&tnHWyGW3z6nhsFr)aZ z#G+f8`1xbUJE55CGQX)1T7DAzC;o>~A|7*r1`hr(m$y-K2D zYc5Z1xNBpc%OM!tlobHSO^aj;XgjE$d2=Y95?Fn({wGrmtc-74rjeioeDl(&vb_v{KHj%|H2>R203u!+VQN$ z9G1*k4zW3wR(89Kr_M^A8Q=Oy0m1-IiVwnm6SzHhTu=0EK@Mi zhsb$%$k2>DiA01kIIJ)OK^FDvo75+d`c6=6L(}aN;Cw=2LS5I$w9DKJD}HxB-3lKh z2!J}tJZ>T&s$*CTZ(T=+rICh<31T4ab(5^62w&9?>r!ROWuJ{|HLiW?TzI#f_Bytg z@&23)=FlM)4MYn7SjcSAFQQhnJCRvspW^lthgDbyKLq>DS4E1Dizr@6<)*^pJP~Q$ zJU{C8gGKiYY+1GV-GM*U0p5UdH?5>iB2|SgHa{)o*l5Sj+ZUlfv(m zlM|E-`O+v95_zZl=rTOFOj??O;1w+N5&X;|6J*n2Q|jO&`349HKY z(0xbmL6j+IJ64ku0~8)L2pV$%UDOX)56-W`0EV*k6d_H$bK}_!)!8(TF8~rVP1;*} zUf!Dxe`N~|EQ!Y2eZ@IlzmCjma7d@)6vURa5Cy$^QEDSeSmPQ1O+~$CQ)6{Zs~odt z&)rlh!%E$9W(Hfp=nhs`zR#g^-<+2LOmB}4M{l}B%8g=uRlt2R4L!f5mGG6T9BS5p zq*?c$F#!C{=Jz)%o#s$f6A`W#&ay{uruxt_OuA(~Fe^ab&7u`ILh;UwIP}2!jq|rW z?)_^tZ+6fu6u!NlqG~lt({q_MjYn zWvdQAs2s#20a(iP&r_A_BkUQTvOueetd2MT?kxUi`>Q4Ne%I+``Hre{*nZwd-#+gk%i!j0ZxpDUO+x8*OM)i5?E5Ny9jYwHa5|A(e5{V(SZxgn+bpRKymLyFi z)#YlOD8`nKbx5FoR4KVVarj#hgLg^B@>uQYF3k2hxZVyYAaBO~q?3ui1thec=OAX_ zBIdkS{NVQ$NiF-9AU%n)K4Xhd$&y~xT>BG^el8abo@4vZ##N;N>V&pPR>7T5E#&d29% zVRF5|Yq=TTj}Y#z17Q9e(9#li!~&R4yw_5f5X#ewT=8tb{Fvt+f*M3f|0YwV9Cz%) za1sPmUQd^=hRA#8O&z(;v54da3*&q(5ZSHL5c~;F5ldxQGZp5@`f#@;?QJ#-@%`hi zIWeo5hX`0d@t+Je@LUU^JjO){EX-S1h`4oytOlcv0AExlY&&KhK_@yO_G_v9PU1-Y z`_n>Jaa;!RA9Iu`z5yO0x(^!4cSqY~qXQHHWX8pJT~_~GRtIwhG==z|+m6=_Z?Apj zUMr7VuY+ylLeE7hN?NMj<_?`PE-K!q_t&AxTlsa8SqL-^c2bqN#N?UVC-pn5Nkt@? zM{C~azDLr4_nPg9vcWi#zkInd(4**cggLC$+TWPR^2;z&_*$F=Q1qggK7JjO*!qwu zbNEf?+NHh|r+6ypy*4%P4&*4i&`~*ZQKddqYUz#ZF)At01yU?r9*G- z+tLUFu<#q?wy_^BbNt!c=quBs%3sue?T{O;1sv6HfN|j0#1|W zu=xyu(B5;@6D@?T_wlrdWK-HuQae7?7qC5+1B=hA(qO6{niAzh?!iSB+as^y|L$W- zqH?vIn>@ElNF`ucZXsePJ8x!_?^@HX$;L30aHNR4l~LLlJ69|bn9A4#s^{avRg8S8 zhC&;K?s$jM?900!I*lenPoC@4Z-%y^Hk9YW$U@J4Z4Eosl`i`|W_s*0tLX(|hRYm9 zy06q&OXWFFT&DYuCy0f~RklS$+1Waq@i`-^8{MA{l|tCStk$rsC`~N2@v%qm(%feS zH9wpH;bjgpvCZyFT>!3+A$*~d#D4<-Xk4WZx5nO_3>D|<4jS=R%=^l8_j{IU|NXhN zpgab<6_)X5^z!-@4$xZfD#1lwkWF}dY0$=>xli@l8W@16ZfSXOmm2MBbUWRBH5WdE zd+6*`q=(2p6Szo_|FbfBdo{e@zWuwTH7kE}&&b-z8 zyoi?%_Ps-bZ%UTi*`1?Q2a0i0q`Wy`DrX3hMqj7+U^L%(*Gchsjp^&Xp8JpM_gWE- ziAvwO=c3{JSAi3_!rX3T6ozvo3~9-}$sF^-{u_?}mAKb&yo;2$|M%oTi2nHoUu=|g z&7o-~OOmfrRF6DY=<))UoJNly`FiIZWqc-rI9XO1qLa?oSQ;#hr8GyN9%t0!3;mg; zu(b4DuHrWzooXn+>2pYKbPMq71BdcD<+qJKsmDek{@IQf8Pc`h+P9qV8nLvGp&_6O z6kqCopdTt%$xOizlQgto0*Ug7o$g*eF;es^LCH4p0nPfL zNVuNrQXk7Knk|C5e)0zx3wh%qzL0?PPRlPp3;;N1y0{=X%ZQvXx+Oh;zxbws*Y!Wy zop0GQRAfJs-tN3V;g;>JlZAFZDbo~7oa2+ z{Q;niDE=)neVrm8b$-YX#;gAPVP1vAmqvhNbd2!3eE5pQG`-x=GmGc4Oq|+V-|vVi z#ssCfbWeeyE-vE@yOnTr{vkhj=Y~{)PlIC;(AA6e9qt+9u z&>S~@AF-l2eGFhmdCq@!=5#;*>Ik#@4&c!%Xw6%dIsq@J%J+V+Jb3Kp@KJo4ORgYI zga?|E5IV?n<6)fcoHH?}V7ccpuKl_V^@n?o?NbSo>|?Qx`_p~Pn8}2}0tAr>jcQYM zg8?NPNi1{RcK)#dZLVn8TV?kkqhXfJ0Jw4)dzrK;YWtzjDaoc&CwJbX(ehF+k6Hu7 zz)oJeo0GXzZ0NdPF)m;kHzC9@J;zOu4~k{G{wf z;ij+HqzK>Ew7!F0YQJ7{!kM{QF9 zXlG>jY~MDp`czRE6Y9T=Gej{TKjMp}_&3h~Rs+fGZox`a{m*eeiTV66Wh?}Z!pzWH z^b!5{t}MRamPPb@ZjrkJvuObklRf;-Hg93H4&~x|h4uS?JU-QWyt>^FmhmB?Fmdwi zwM^iJk501j0MPsjVBDTSNcBZ-jYA;n-`x44&Xa~hjRq}A5X;~@@&3K8AE1Pjk!DaP zmMRpC{}Vt}Qn@A+`#|*d4)BSV4L$_d3N*4jPlh=VVbS1xGz3@4v!LeS`x)JRAb;>k zm+e*_$*3aZ6{0$oHrY~({_MeLH_uy+Lu@h+oED}(FOS!K2NeF2k)Y8Y9BWG`G&lih zIGKa~&TP!T-7&j1Bclm39z!Zr?Jw@&-(9$sI4k8!)z?UBR2UU^HZa4k!ps)Hr{*Il z7r=1y$m332abA753b@JnM3cxE8Qn_jED%O`r>S^G9A0vsa^C zYFy#AVTRBSXI`E?Z2sWiwv>NRbF&1=E)FC8ztJgGWJ6I8xxV#hCRN-hpY2KMss~K{ z0B`{c&HK`wK=a~bmA=Qa5N8LNKzbTs(+=#}+tn3&x@D#zXQE!20j(jv#-Pqq6oREk zJoS4H$U9m7{8}sU@n2q`sMMpwKq!zGS6@S@Wqs`G=_(LzB~{D;td5<^&-2=|8xhsc z-`I|v9WQV21P_`UJ?%kNc}mZYz`jot!j*Z@iuh_sC&~Qp;7Gc^HUx5u&qc!XHLHZq zD`Q`7n?{+y-gVWBe|P7x3Wr*XaD?lF*YpzZ8eow)D6*;mwe!@VUh=k6zkHdZ9l)O| zpr>CAi0(VKndG4e!xPaa0f6hmsBL{N0#rfx2z!&WfO^y`F(gQHoeCMuQ>8UFdOH=> zd?Di~@#zjH$Sl~s=Zap>o)&F?0jt04ddpaGG%hu%&K zW3V>KssbuJf?v%M6~ERcYXeB8Yflu27eBpxMmO9e2dsSqfPOAtf8UkB9h+V3IM;hE zU+qki)K3nV#laBR=jJ98V@mX)vu-h-EEJ)n5}Vj~PA!EtBM*RVYFdqyWYMwRl*l?| z;K}#W`r}owsd2>EP+@X|+j#S}ySVj`54J7>`zl$6$eN-u%4gL57!&=8R=H>Cv?4Yt zFLYm3&iz5?lVh2V=kgGY@KI8F>z})Y^sTx!uLA(=YGV?5Dsb(m_-1!)5JVyVX@Tbv z+9hOp2}>bJj<^66gy@-#=n66I?Yi~ObJXt8Ja-XZz*TH3tQm;o0Yi)$C^45Mi*EhW z){xRNO*<<>1ipz^-Y}L4-J{s}^&HqB4dfEue3(|xOV1#zPT0AHKi!C=D`E5$AE7Xz+E>-un&G6Q*Jpn=(!T`}7rb=Wr`j zJbD~o_uyRe_5Xo_T&G=W_C+>2kw1(F8|<@C+N!bSF=A)2`P`m%Q^;oDI+<=lg z(kR%>r4Kc7(0=Xr<8Js(2rintoyD8=NhH>0ueRpM>CNGG5mG#6=ts&sf z`lTtoEo+DQq?_WTrv-x%6`oGf=%=^kZP`>4c?mA{;c1?KU1s?yqFJOEkOHp2r~Yl? zE1nn)(`T+cjD46I&@dW|)TQsN2>iY#JGE zor>ra4z6Km^jmv=i5T;ppjaO#LuxMb_4kRWz=P#5@!F1wPaQeV2HLkM72d(?X8c(l zDaazkmTB|Fo%_i#9p4O$rb^0)&s`n$jonk<0LqP%!pOr{)&_AIzy1*W;7xKXkL}Oq zXH{;KCrBRrS5+MjL}?Z0 zqrIjA4fyoE8LB^`jY4;aHDL3Fi0l>JD$mm#)o?Zb=Nc2M(#Y@%0~o{run3}kv)EUG zZvJQ_KL+5EEOQ*|XD#R-woR%N8f}n)z>{a6AgU;W~cOnfPiARq+2(JL5v|H#Vw&m+}7|fcVa3JV@WZDXc;Vj*M+Fj2~hS z{{P~Xq?b~N0E1jA5@EFTKiOr(q7O#v>$+^n0@f`S!C?08Y26N5#zj9Zaz#K;qW^~4 zy>~|XI7Br7L`CXutp95OfrN3%#_PC=VFd&%%B$Fr6NPVy_0(SB0K5?66+EIts1<)G zgXhv4Ks!hVbogTGT^+z%A!MQ82Lwlsp-r#+`)IirMKlR~^brao5@w8^NiIV12KJ}~ zBHCp2p-2_6wDqp*vnjR%_U&i#OHEDBm>tZ}$Xi!a1dHbM1@ulZPt^n=?&XP5#Rjxo z1|>p`cn{SVC9OP_nEKorAVq?jfDqDi12F>9otDdqWP<cKlGav?B!uw1QhgcSDw{Hzao;P)CRir^?;Byt=hq05Af zm!epj`(_axK+{6c!v86P#&O=gb7Ibm@1U}v2Bh%F5Wf^d@_%8TVO>zkX%sVbC&8i( z9xUGD^h05<5v3|%({b!KU&^HxvZ9oLm|+uCL`1UeXeHbQw^qh8dd0qRoz=<b7DC*09Q(0Y(L4MI(17Ya5OKfixz*#W7>n^XcGKdh^FW>XeMOrmn}jDLF| zL3D_RQgUXiG>T>-K^}LVX3gL~KalS6UDV`9k$&wb?T;KI7YP9bQKlxjV|o{8a(qd3 z8@(6#^@uLD`#+=Qn`x{DWYy3MR(FOwp8Xd&ni2jAp0fz`D~=VQHJaacn~A>x?s%ql zKB7WR>bDaD&9C_J{mZ##gq4%jju7To*3?Td2T`;;<>6feE?!!&J46JrwMWdX=z6ar z(hpFh$bm`UAo=xF=Xj~Y{V&d^Fl%f<5!b84Uful=iJ-6h#5cm1h^JQ}#OoN7`%X;x zFdAxF>Oip3Ql1Cof2_pgaj&0e5cw^Lauj5v2a?RwE!$}Tqz!@x5mn}7iY1tX#NwBi z{}v-w8Nb7i(BR#25I6mKy+Vjwi0Q=>A(Ov*)3(VU$RR(AhSNlzRA!Wo-WoLiTBvCd=6g{eLVe>b=;1eun!#AEX z@6bnH@r1L_qbeu(80G412o=jh(1CIZMOb_-t7j0>fQxo^SjK}tNqK5%m%p}0v5$q> zLygJdN?QG&tzw7sZMwF#FQv;MK%oFo{s}Y5UrPe72dbr6P^k2Oar4nX@p4C`8Ql@s z)+9SwUv&^2}$z;d53NPV}a9`J4y%e>Bw%|qhTo{-9S#&^=ynf088Jcd&O zDk=yfO(Tfpj0JEa9EK_!PYRH#pE|FfTGgHp0aFVt&e4{Ip{8UO2tZG+@Vxd189S!t zRdqY$h3j~ll;npuxkFySL5|(%48J})E$3_!Ph#0REhOi&Bv)$7SHqU z7>eO-h^&W!mT4Q3Wp!l}ts|?Uqb~KkLNA!PvN^`mRrAF8M`Y(RUO?ombM9tQ1-FXyySMuWIxf?88r+gd#hO_gUMBg7&G$3@@!zh&> zRMZ7F1!0uyet|?ZvLLe(ed5~X__OlRBcecfPDp;^@4UZ|pD|6>3e3q4(U6)M;+0ko z|I9NF014|5mImRPJdZ;m5Ry_)xHVzeJ1!Wv3Vi{&I0s$2@2b=CTs5ksE8l{FFWfYX z1ETi6TJ}*!tSBKEQofM%Gq^2Cj}QZQ@~rz#a8Nu@&JyibTI6QeHTYMeT!H@0?#|Wh zYGU)}iv_)dnv)&<-i9yexTs{eqfc=BGx|jHVd!!hV4PQg&Bq`HCpRP*+=5~BzpZfE zGf6`bJe>=EFC>6>{GT(=L$(G3&~D(;noH=OR%8_Eia8DToZ-Jzk-?U@6N-*#VkA|3 zCoLzB@cHqxyR=u9ZN-SaK?r5h9IzsNN?T1o$ao3K0v<{S|2_Tw#=!z(_HS*v+jaers-Ns72rU@} zvBcM>ft8yN;i(pAZ4lk^F@7S=Xe~)7%Xi$Cdl5~@1XssE_Ah7j3tjc%zQ)(zkVgsL z>7*pDpHrE7^;miDd)Z!6o!(-(^+$fhDHTGE{Th_?^~z*FVgK_0*t(noa<&OKGYD z3=iD&MU@GVi^&4HI^92kjzDK#0Arg2{Bioo=xN9guL}Zd0sIG%knLD1l+G{EY(Q?n z*y{+$9;Xkt@At!IR)eZid>@=^(j+q^A1c?P8aH4y{du4R6< z?41xFtX;x(mUujg-2 zj@a8x+9FZVwC(2-h@TFjj+0|4Do-4pX>=~@&Pv>6nFu&tt)Rx z=`I)|-KZ(?ZIJn!@srvotMwg*6*4WIf3x`XnJLE!NvfydsjCE@ZUNy(=Q79lkL8L9 z5+Ey-cmb)$IfH3$@tBFSwD^x z1S);;w91u^OMT)kJ@({?2vy$B#vHr5KW_$-m_W+9Z{?s3=Ian)2*zn?gm2AH+&_7V zyW`T9tginky*&G&(x)F!U$Dyrg(UM^XT+RJ7+4oIm&WVsS$~*UNbb(#Jhqbn#vYZe z-NFnqeZmqhlAe=Gnv|p8J1)qo=!|=AsCH*)Gg8Eoi2PwaNJAleTzPY}3)cM+mRHoE z3v3031oeh-;sytgPDMxc{-0mm^bc)Vzb-vl)C5)9?<* zi)9>{xxo>7&wjco9_X|>Q;+qY!`kQ|fCbjd#?n(+!T0W(_Kib^%~g6<=SDqLux`N7 zd`>F4-7uA|q<^T{k-T+SRG9v|#I)yyGhu3_Ly=rpdNiklkwjmwhB>hGPAZ6jG*v_V z$i4R7+C{QzRRpPCc57~a{CqU=FeDP`hgMAP!#T{0DqsqHyQ6seDMb=jK!u%zqCLdP zNT~gm+ZM#~Wn?>Iun!&HoIcv-hFRnIDK^dRfYkFL%Pj{L#`=+_(J!lGp-2}OV7Bie zfSNg9wJNSKR^S0>SQe^|Xlz1@?u;kMY5e$cflktCUgNZ+XVvGsF(R9@UUUdQ8shFK z6^|pYv;P*Usi$9&Rr*QGVJwnLmLKIp7IuY#tOrL3Wj`EM=v&n373U7w-kntW?D<-i zoiOg2kpOv<*A6q&=^+8fPd*AWOf@T~Ml@!xKiAf@q5AF_co*2AfAJL$>TQu@2M}uj z3Ci$c{J<}fpZ0HPz`8^OSeVpU+qt^yKYo&HN@CHO!16)%F7n!l2GzqF~;kulyF-b#=ATS~jjQe~afYOzzGULaw&=CQ?E162Ox&R-nyLXxzsV0OY z6^6H+vKP=SfWl1-KHj8qfJjt*b+!}1?I$9yh=$e|6~k6y`JV05Ye#b;KM-*0FKwr% zr3j?Ob{xPY`7*cx`2sChgyHFKOwAuc1Te`CLFde~Pw_Cx#gjgzurewLmM{7U2qFPG zRQ81Z#=$yt-~5(E2Nn`?hj$2TLRBl9*%E#wkHEhjKJ^E;ifA0cf6G$MhMK1E6NS9O z!`C&>)UClaeTVk<>D7ebLY#wpMwZi#ZBxn*Z=F|0$;V}Km_1}?|zeLr@q{b?prc9<>YQ_@?{c+V&WMSDT*&sX~fX&KGU)h5@_~M*U=<~-3DmBm$9G5qrPqL>X6!ydg7*kEu$P=)c6F zVw^=Gi6H`s>Av!IDj*5IKJjk8!@crJT29JyuOv#RS+KVVg@-5QsE!ifm-UhD$GDg! z`}q)$D`h1f7PE~#gz=B^AvUM3yE>4~OOdHTwwg3tQpMIXYTaE2z0 zE+wfnp{PG>(pen|ScQX0QQu7#jyZ%eWi>Q8qc%+B7K^%y%K8tF}DvVA0doZxsXC8Id;5dzw} z7+F5b>sN3G!^)QPk>$JmQeHQVNjgiLNTQ9EfZR?NVSo)3-M+}~CZI)r2o}83kC(gz zS+i#C_Z1;1CB}0EwBPK?%sNt7qG8qej-b^q$s9aferNy<5BEfm09tywuLNF3hw%R~ zF}u-wuioByd^!k_UA?i_10sl=F$vVPti!p@n{Nd<$~HBGYN zLs@St9B(zY^0!y^L8W3#CW+E0{gV_D;BA;b;~5SdMIKyg!_P&BX^CwBA+fVyEkn#F zwVx_sGd05Hgs!K?;9q;I5RNJld8101+A!=^a?`y6_N&2O&G3mhG72}Q$q*92 zD?94b!gz<>5ZTNLwWpJqFwxSb$k;fC+1iS^;N$Vv$2B=$Alr>P{!@M$>|(gDs`s6G zs+1BVEPU&sXzHA$gRz(0#0jtvkaHv;PiXzrvK=q)BU@Uwc=2rROXSr&x%#Uj7C$gX zL0FVaS|gewJCoyAC$RWW}`o+iNH@o)mSyMB3dM+cf zomDz3@jPGP`D6HvS4t<;E`G4_jA8J6H-_FenZm|PJVIg%FWf&C zVb|n9dNwH-dE+_?dG`gg9VZ9yuU1bWvk^uJhc)`_|NMM7tlIov6E)=Fojeb=iqvDe zhrIX>p~I(D2wCj9xLqsa}W!+9k6L!-W-Wu!C&gsjt`2EhMJj6=|pK%~BDJ98I{UK014`w3V z`%GIY3cp|IbdPt*5%R$%plSR6_`Knp|MBxMaTddP81eNQ6>&M_^3o7fie2P`omg>cQx$brRtqNx#O7Xa{M1=jW}rKt|{CyLHGNZwZQC1k5n| zQd7WrFbNNnFP8=rNO+Cymf*qrnzu4SEb9!L9dLfS;&1BT`fi=XfZ2utcv0?=F(17A z;IoN{IfFlPesG=y7ej7Psl)DZ4OH=kW73U+BHmf(^YJ(qbw{FxVO90I9@aNqIxraO zzX!-+0ez5jfk_ndi}68hVc`E^GGGDCdZI;?83W)M$Fb>w5Ai#fI`nBS8H{lUTeXmv zG7aM0o$SY(A4w}?_Qv;r8+_WV{AiX^@xd(y)p=l{xD}kPTX0s* ze4BOQdnQ>Bqx|lX897?!s{kD=-X5mv^sP$dhi^d}GX*<_bx6x6O>q<5re;aTFJjEK0Pe)9Y*eO*82>0fr~r=@Pz6;4w0Iku`k z&In;(vB}&1*co;q1gFO)rPm(AZSc9T?c7bh>6GLQQBv|#J1MK%5r^lBI>YwLKc>B? zk(pvvsb0^no$WkjZ=VyE5plC}SKj^gI(*sA{<+(Fk7fG=(mbNS@fi{57i``tTRb5Dy<65*1AAdhh5n)$r((*m;a>D7=m+j*m={WH4k zaae=7lt&pM54@u4-|t_ncmDd#^ZiKJw%GXo$HzS0-`9({4XpMIXH*%abln;~xQm^? zn&fcY{+%-yPY3_9Kl_qroN?=fN7>7$iJ99X`_@}wKW8j znGa@vKWU#q;3X*xy(0Lkw-!gjBewToRya=hfT*MySZe0; z%l#OYWXW6RuevJl`I&Xu!6_qS(m2kbX`zWX&a!v?)?lEiWXQ;=^4_UU?6LC<+@g-L zSohPmeshm@p>EuAmb@N3H?vBn27_iOGu~b27SQBD6r^#35cCscp=gjGq#k>#4)kua z$U@j6RK~xErCn}J#E=xn!op2|!(G0`g)C-ioKQ;zJkVI3F+*))jn8i;r1TQVuRnb~ z>YZ;!8Fl|YEeBgFLyQ#s4)Y~O1_rVKSxpXwtON@jyaI9p+7p$r{2dPD6%{q$eJ3A;7m#ZDp2M2^?o`Mp_=e3`}ZCCktJW%f9ttdu+?(VRF3i zXXt*ILfx*cUx8K`60lSW5B?YSw|GGBG{6DUsJEcjI$^6*4b^KzlEnc^A!gw_pkn%# zGe4CXRONE^e%9~zP#49CfP(Jb9q{x5Y_CCT^DH*`O@O9UCqR?Z`E11MWENRX9`et(h&F%sQe`L2 zXdiQ9HW^X*=^^2r6&d!6;z4p8pLZoKlu;vJLCeqMsld7v6hhaK`)_O^)%chlI5h^U zubIm?WHm4RtjKWixC9BVU*>I4OR&^fE%`+=#IfjTavkU+u<#1`srI6gHQZx;J(8f zVZMln9?hFfn*8JPEwucmN@`4x&ze4*{RxeqyTVr2xGgB9QprN!ELW!d1nOeZ#tuj$ z{VoakgMOwV=k{>$O4dxIRgIoxli6$hHcJs|t3b9H_5#$kU*yoT^y{Tbm&%e74xoEv z{h3iG6v_wV46oX^MKKf?o0!@I!$J8clrLwHQt=oxmgfP9!LdNEz&Nz|G(V4hYYRFw z{T|SWs|86-CQiMnw?N{qunW&28EIzV(FIiayJkU$I>l^_ZV?`_Ip{Zf z0HDJA6Q_>dH^3qPgzm^MJA;bhf<^B{AQkj|1K{857xj&{phdja%xEwY*CBMBgGaAQBdDFDP?Yz!X9K~p z?S)JL6W$-53(@Ms-T)EN0tS*DAPy*nk>mi?L6P-K^Y4}gWAt}b;g2n(TJ_#u>Mu?S znxUvdp=Xe#_7%Bus-bsPAuUr9E{vKtXkJl{9p<^nD@!(mz^52&`XqjCn)RhGk3M90 zy%bRO21NIM3mUc~w;6b)U)ZAax2<*;1@^474jgJX@>k2R#wC>xK}|5D?oc2xG+Z{2 zj{TWeChd{lNe%8$`c*d$+?Crap%1zD`m^vSB(0h$lAl!sJ1yu7YgacfcwCiHiQRCR zhKiXoEF`R-->JGMMTGQP4QL%J1_cGbUMlOVf&T8$sR$Tds#-HzKIeoHzr{O7^?L>H z))+7IF$Xn=cN*=wfO^jubd9o)&`c$Pe$l;)OC^4hVZ97mFBMRT2$(J~3)Id9Yb=4> zs_jKNWhhHZptYtSb5IT`V7-XP5fFL4aPz!luvEBfgeEQnDK^&~B?~Sb1MR_Ep6wG7a&55f}+&FC4wcP&>R0)uJ6uKmpYR0f{lS~em%k;SzHU7NMa zPgdvBZe(i^HHoBX$bHmc2L;yh`lZQK$KJJjN3N8}d-zt#Ck^JE^Xfs%hZFh<$>n_d zob#FzGM!l@W4|3pzx)SX0BH9^a~c{3Mq z47x#$N|~a(wITg%EddQJF;_Tb?PPP9d!Z#}{AeA)EqQTka>(u(4^n0yJ+?mT(8cY1 zL52NmM}Pyk_0e zx>?wx*}ZfIUR&`b#~P2x1mJ>e%>;i!u|-6Afzx@!2)Ji;AN`sCWby`CYHfc#bS!T; zqCnl+!)(EaEOdh8jZUSt<)Z=T05PRAJwFcnPbgv;$Or`vr$F9Iee&=lNH+rx30qf* zJ#-;7+oc(o89vQcbWbi-mSk(2{7Y#viKy;rZGj*)YONmVO0W7Qr#-Ie zZtyjoa-+@)cIxr%XPCAJ-N~dY?)MmqCQTMw(eeWPcL`G*3>U?MYq4*svo3uqIPaKK zq)F5Xdq@yB&~ey}Nj;qHYeUDmUcQLPmr1j(aJ}4?Gmf$}t|@M<;V|hY(tUlv@}ULu z^}Ys#elB<;sG!QH3P$i5N`?KNP%Kf%v1QcN0E0@hrk0o%Pemq*o)$K8r-me#6=Sc{ zl2{^WZH1DA71j(Mev=nNB20xk8LD)cmXBBB-tdG>oMGsY3g-%Ne#E|joF?lX3{&9V zVDrPREj?MXZ^Ky0>)3q2m5Q(WNbMG+n-}EUtL~IxnV#d6VD zj(I5v?a$cgWV+bf4=tP>TDEtZw3>zV#%Y32ArT(8;PZ`9Sd#5gkyZjL`5~BDL6EN#WAA(W%PfGDB%JoFW&2f@jm4@zzYw6DLEdw;mExYs$S5gneT1eR3yB zyU9mN4l81>GwzS+x-Ti2Jt%w-4i~ZJftfW-J0xa)Q?U-FrR`7My@}zMPkZ*+>H>XnjRuy~M&xS20cWAOK z@+M11N9d@J+=E;8NRGM8HaPJLFH%?LVnY{I1=)6Q*P^5K z={8b$+;jS1%Ge+?cG2O|}WDw0a*cXorXCNv`jA+9^beB4Nm zLcvi%;bW+BM#A4v@JVPcVUT78pdBs9e4QP&wet0G`F%`W>8?{4sdF_Es1LkT!?{;Z zkzQ23J6MpV$lBwV?RQ*WU^$Fr4=6P6Qi#JD;Z~@2r*!OHFy-&vrji0i8pFQymJk?| zYfW8nz{vZ*hW??~Xq4zI^$INJhdHG#v^?4v_X9!HxngxfqE`4^m?X*@kGvm8Amco4 zW)B;8vS)jTA70q5m;^#jM$X@vn^!n?v7oAdpb zMqw9-Um3`WiWvf$(U5_Zr9bn)@upYsiZ*-H97i5jH#1xq`4a93XL*z0N#ux3gyNjE zQZkwR)>8!*iq0n*3}l^698WkXn{@;#pvcaMBTy-SEe5HW;dBY*lb|GO!>5}NQ~p%! zWki+Iu-Z?*YiT`-5y(CkgO=F>E1NRvt4K*k(A2qP*p`<7(b)JKXafJPSUr5Vi}7+# z(M`0V_}SKAW`7JbB_DVCb}&u2_@rng_Ot>AGq*|?vs5s#U4z8NQcH^}i|e&0w`C>; zwjN)_MQcDi)KwuA5%1?wuWkLa$qWj|*`ltZTY;zGK5AVaQF+AqNU5oUS%k#vCHV_W zvcm6&sg2tB309RULuIz9aFl$^VdmD!FN3g~+n0{)y*t|D8*yHH)XJal54wqgxsWta zWU_hnHj-b8T5q?VPQFsh}lG03j8C!GW?X?gE&A!hRFRA$M zI&D=_X=ckjC_74Ir-eal$|}TN=%m(BjSnS{$o4mGrQb1(RxF@p zDYIjy!q`PzK|r^dHCcV8z}zj=y)B1>fNRk$N||rS>19E`gJu`&qoSa-mfWa=vinG< z1qI}uH;}Y&n25rrdI?3|J)uq>EJ(1Il}~TXvzWD#ouCk$Onyq}WwE@}_d->g819U} z>AXTE=YSn~fgYh%wfd^Pebo|5FYdA1BOhWytM{>~E}twP#*DRGlF-r`O~?s%`jC()0H5*2N8@=g?1!H^km|vAbk!yqQeDk{Q`wRV zdCzft(N054&xKItp6LZ}UiVRYyJ);OyRB>7i&Pv~Cdw#+8F`Xb2u~_9;)gnw%!_X* ztb$4JAi20Sk;7>J4V5Zwn#Y1Q`8Ur03La2FrHG5~Qe!NLfTW#eBJy9cr26ia?j)#l zLTwsy_Wx}RN64!nyqW>D(|KMO`+gX0>#g1q7~Blw#NQ>Ku@LG;mPHr^KP~mOC}adF z+FCB%%5L>qKxmh}WlK>4&TtN3o&GqVlJbVU>UVWDWOA=N(%lkbq(6J20*q;BNz0L{ z6R|I3S~uWU5B=npbrn{dH0ktKQs>o+D>YgGr7e#H{xWj67ihs-r+Lg&uL60r3+~a} zQgQ_*OJ_XyhP`yd-yL{Hkl9QbVt6`y?NevEi~?sV8zt zCFpZEu7sVh(~jb;p4b5B^IN#1Z0lmg?dd-8F2Lp;Z0W8KE{VvN4nZ14GOAF8LcJ`# z+Dwz>+IZv?3)42_jng279u2u%2atkT0aNoB&=p!?3g|12khC(CLDq=?59=!ICS~Cg z&SdL~euZ@aw{?Af?U?KRdw17l3gMcobK)ZB-1q?(_WMhfy-DNXE79m5ko@dbVIxZp zFUUwU>jM(bG>p44dXTIg;Pvx0OfL2hYPW`C;pQ$SzV)kkq(9m<6K|OVu+_T??hjA~S*WcYphW;6iYTU;6$L&gH;X-q z5pt_7jfhRE)3;$&2#iCz#AW6F&F9i!mBn$G&o%~5Sd)7Di-08msRGR>rjU0JJW(Wt z`uRN;hO7%vOo2{n?GjMz+~roGq>})p0U4e*OZ*b2E&PoE-7y2fO*~THsM}pUM+114 zin|}wc$@riX&Q7=-Zj=Qhi*8)H8N;_))s~$rZiwd&AUd4uLo!LAUF27N3bf;2NBce zDavtD3T40hO@S{sRC>+*Q)QY6y-Z_kcCn&yn)NE_in$3>kHmzuYk~h0>sqcHu`6>w zXj`=sU6Q{sw1{)z6^IcnEq^G^fCzxzJ^6FEKfPT8fkOvV&^+&U`-pVbJPtEN`qw%U zUEj#&gR?7xMl1u1fxrFVEiA*4H@a2RC;6)bZX{hS{@ta>Vf2~wBb9}zS-XrFxTY>sWB0QAEe*0!r$=dGEl@$Z6hwnT0?y^&XEq&O&J?fk#9yu2O%UK;GURv!~>S)M3-*(BS9gAV6s)6cq( zZ3Irx%REuQY_(BXn>9W$FQJ)%B_AN290LUM%Sr7pS12INi@ag^?(QPG6@I1-8zrPI#jJMFZtjHt^)9_y&k&v*Tl0#}HTY+@q#>d`N zt?en;m=R>Hp84zgsOy9k!jR`+OUeu{DyGj?cBwUtG^}i*_%ZV0a=#$@`VcdCt;kz} z$%RA;SQ|Qm#&KjThJoA2g+Ttw9LA?a6iAtj`4&U&-@LRmWiK_-8WC|`yh;Awk!~KU zvml0TaR_lqKGx3I1MkX1P4Lo1rQZHXRKBw3r7I;PGsH(4gGH(gDV-qb-1Spj5wsdC zs0rkKmHN+p@|njAARBB1J@L`ebu2Zk`40O;c1`ztZh~Oh8hR}0?>66?7-KJ1{bR0R z{D0Zn+23V#RA>3;&q7{yjUziC+Rq1EXOLyhnM;IL?d3=ndC1nL@f!-Mt`-85X~K6L z0`4%QkS7K?RXgb8t@E#x6MRr-BYoV>()34GvM0bd>+zbV?~rYSjYHk@hD+AnBbZ%e zOpYfjUcmf~2#ha`(!2oQ*CC7a9=JZTesTDSVWDZFfJ{My{9k`ULs;N6z(%4Ypk18E zUEksX##`>E|F~ap6_KzNb?PbSa zdb9wyj7{gnsy#yFFu>AqAV$_!d^ImVk8O06c*%0EEmcRzXkT4#*htd+)jK>nuhAu(k`iJka&nhRm@uEKuWCI*!B* zNxVi65gOj^7o@tH0Lr-R0WuexygXmoe9?hOkQYH4UZqZ#i~SemDU_cXup%$8AVlow zXQ%h7YSi$qQz(3S?{x zmOUQ4yAmMRTQ|fP;Hu(qdI_B$U+x?1K%MXo$n8CJ8U_1O4!6aHBUhjfz_o0aAIoyp zOdxN1Zwa)lp2Vh=U}Psh-&PnZa64$cqgZ$Wuj0c*!_|!fV)Yz2VV5zGON{OIgj(&E zvk-Bb)0b306Zc=--`nUZ%vWG=b`=sUD3$;_r4MhfHp6>YIipzb^}xmT4qPo-_qhnc?1lyMites~Pk zenkqt(*>xk!D0ggL`s4}9?9G#9CqNY`!2}3rlICW$*Ke|8*1jjl~j3U8P2sHEc`EJ z2zT+w*<#U%!6y~R2VkcUDg<_^9-_So_b8k{e3S@@C%4I0CrsfSHrr}YQ(f6HSrnsgt@by+ z;1gDjJ;LPM$prX>qq%cPT#Rczg7ew(t@Zes0Jp;YB16c^z_0Z%1?caemrhd9;NV1x zW<(5ALrn#AAkEe1+ie!b zsLU;}KJiKuP_VGOu~hX!%Hb$KiUxwQNquO(JmM0C_L#U(ZK_jGDqQ&2erZ&T_F6Zq zdI(`{9fBAf`Va}V3tOb}KWIiHMdV#T%)P6(oHsmKTG>wryubB0!>uvbuf&ca<78|H zHjYj+d~ckG3-JV^m?}DM4s+3SOy^m@Z*GvoO~b{C1f_5@LT6*-E>yQ%&Y&*8>44S? zH-rs^6W1+Wr-@x;#ywVQA69^-CmtFp+=nxB~Wp;PG|F$L)Hiuq=oEi5~bMs_C+KyKi7aN5Um z^KYs=pm05OmkDBoD`ymzs}dx=cIVrtCio;q5|7x2&WWuDOa8abyi-guN|{q=?OQ<< z(b?eHGDC74-PZ7YmHf^6*4Q*;fs)4#-*~~~>BUz$f>u$NS zVJ!Hc(-ItIdco!!ih!$*9$7rtL*ROe>Pea244^ z=BD5o%CP@DTbUrOnJn1JQ-tu-Dqd4jCC1(ZbN=FMk|I6XK(1pb$`eR;_`3pgqbMg(|R!1=#AtAbag9o(-z zUo~8s%6`3s3N249g;X6-IR7%heyhY|u%WCGuW5WZ22K$*MQ}MAWAM$jJAf2) zzFLDHtWNJHoWoe4$12sckruaHobWkJD5W%7W~ov~_^&2ld_+)(Dh4&BfCnFh<4Xe1 zr|pmuv-fR%KnmnS9eN!aSl@nfe!blZr3`%;i#*j7HiSf;3O%SJ`I;JS;grh%+%NT8 zGo>fWnPqtI8-n*=+J%#K+fye*!BoybI2n6_r%D=9N;cb?wLmIYk$QD&NxNWN61a&r@LQ9t}~+n-7r09H6yr; z1!zyInS+ZL>jggiIb?5t0Fvd4m81uVRG%9#$~P}&z(o*T=&%^o*rQEwpR#V0_8&MW zVrg*WZw6cSAN^{lLgXewcPA(K^@J}jqpD#OInv@7;(t4a2y!bXutu7UNOgrUqvT^1 z+=k3u2bl*8bSo`%HjpDoBjH{FzDIMmFXKEAnERhRf+V*TDh-MVh(IFQ6*4S*k`H~P zvdlB7aQSB)_BW(#PM@LcrYJmWEs)j+mW2)J&jEr#0s64))N9YL=7u7f8b4&xXUu@S zI{=v-xAm(r_q)hF&NC&=B(ZP4S6>-#ulNbnCq{?_UN6Bv?!w)_#pjXcUtL0_bj^I6 z^pdB4J5*f&?x7%E>(!9dYE@cQUXx`12erHPZI07uwy?jxA_fUn)oW+kweC{?{`Eo6 zG2%morIvcc^uk?8_MVd+3@a@Z+ zR}6|6HQW3&m-RT_p%2(o_EQ_&wg<8g3CvD^&*VjFQ>XIF|^9l~6!H0YSdG_7>kezI)GozcJqNo#7bg9LIk4equdqtvToa_n&Otcq$_x2R?i0 z5jRabgZ64$6!Uy|E3lDZ2#5 zg@dW5p!oSvrsAB$*-%lV>=2ZbK^X;WA@_*wwZxzgdOyzYi%Rq;-y5V9MhAt=en=G*8-u^k)&|V#+O#)_0 zp|cltH6F*l^6K*uB0&4dBlTkERMS3T17>}E=XYj3qSg4Yn3bN$ee&>ediIyE<(KiP!Hz?vC#?#~ z`9q?9utQ!Jox*lhy9Rl=;T)giL4MAbS$PD7EO>g%1-istlzD&lvnIXzsgx4DcoZQF zkXuzAG^;J8by*+Gr7}=tX@4NsV;}4%T5|<5JUm4-6x{Dm2OMDeQqaD{ z#a$#_yRPaEc=l<&V-O);r_((T!kXW|2Uoxg&^rmd4ciU}vL^kIJ`gky%K3BpYn8Gc z4!I@Uj9cUk`0gv@AR>1<3i`lBZ`uYCGqK)KtR5=*5yQCoX^>Ls9H0~pjBb2T>a{8=1xUW5jJj+$H z1EF=Lc>6g+=E`KQ(}2L_2@K(1zRsGJ`ls~R=SA%$NP;*q!f@&a7neRXru*qeA;cTv;D=P<&2Iaclx6{PEskC0uM9D>KY@$69f(9teeAM&C^Emc`S%RQsDR zBb1tv^rC?gzOdcK@|P5^lIy0NS2yXy^Uo;wVTfezA7Zf!RC`5cG-b5pk0~UtgG$DE z$bkA?WB=abd1%?91)@8%K^~fII;O&lEh+6_=JVwye0?46%c%WoU0eY4|L4E!CW2QH^UVb7FL21Ye90kC;6q_=cL- zBl4~sg+XC^AqJQN2Xf{=5%{w0z$BZbvvr97?_}(xmTeA`aWc^d_Ls?*!|kI^q{zb( z-5-CmAU<#phJE>h5kKh-Eit`rjBU{cHR~h@DH-Aw;w;(9#{E zz|*Boja92UW-PiLu>zafGaa$7?gZen%yg%z-OzIgzv-8H=YzL{a1<=k!gME%(~~Zo zOm$X&{zhYP+Uwk1J?Vm+sZ;07EpZ9?GLBR+5x|BF%NF~wR;$W zyZHu0(?@a0#!UXvPe`n2j3&l=gUUQKNAO#tIG!HS7{@p@jG~W_RVrmZN+>zP4=709db00+oMwOjNFD6sAu$Kj1ron=G|#(p!r?Ux z=Y6#@@Cmz;#MA(Rx4`2*Pu_eS&bd&^Wb`t2xF@{JHS?ng;X;)l4x(Eac{uJ1siH&= zj)>^OPpg3oZPG^a3I}bc6J|t)e0Qqs1;&N3`lRx4Djfs-SguR2xbM9LRTaJVxRaLx zoI_hS%k)WN^ck@WML{z#j9?s;>I0D=XKPPJ0e6(WfG`E#A+odvh-=S7#fFe2fL%MI z1#ko%()d>iK{%Er?vV`#T-OW~1)24aes!?wLSo5Q*=nl9Py$Eq9|-yMtN+_v~@W7j=ZKdCi3-(r5zpAbULm zl>HxWqGx(2tWl6t7w%qD0Z zdS0uDC7lm~yN7x5-PZLf8ozYY`g+JZnEt%?$w-{zs(O{8=s5_n@=2~&q81#ityRjHt3IiWVw;ITR8$`DW5Dg zk(`Hwj=L~NCyfVsXeAlMQoX32Tll)I#TIvVRPf_vmo7)p3&AJ)9*LaR!RN|AV^yyd zQEkMn{*Gd`#1`EDu<{^HCkiO5r@Eh5c!d?wio^1chfg$AJUM zFzgDZX-*8s4Tj<|yJIjIxUsC=s|Y()Y8Citjz>p5Zdal=m~)uaQ4~An-gJR;DR#NJ zaW5R6XRVe5D@4|D5&DcQ-Uv`55|{ z%+}HwSW^%@_V`EAVcfdlir$o50rMCp_=xA8DK|LDC33V@xIUfDos@{6IYA1)5mqp# zMLmcGr%Wj`%HR$lvg^>RxPSO+Y{To{f&_~|1-}PQK~AkVq5_r$ zxPujYJ`15&75-?!3n0c!Z57T5ryx5dfFqxPr|F6<^Wg@r$ILcEx0nkzqT);sQlhnj z7fmD`PD(3b$Ka?h!!}Gh*o$r}dsvD?uhMM(&G97Fo&R(^c8WU*59M@gXNwxb9n}+5 zOkje#MXt;XIDGT}S8nPx_qiQi9SedN8C$`v$16H!i>tEoT_T zzK$t4QJUTu+?&<_83`T$tNH%_a#LR*3;hgjE#})pDeV8EIUjs4uBi z@)VFstGTe=vOfIw9`}XlkNy^DB8c73}eh%gy}(K&uFewVS|$hv=iIw8`NyQb6O zsS+Di*sC3b?-3|U?f?)4$|Cz-tGaJ{RvnXc&!DXcB@8+IMcb3^np00nH-~7+DR+}* z-t^C5;R+P+?*pN!bSy92*`A9g(pJN_7S-_x*AFaroSbNRGCTFTL*~q{m(;^2efz8` zW+MH&V?|0_XEaiF*`-q!IS=@+$2>`%;;lw35Ef%{z7NSRfPT4_eui+wKbTATJv?o3 z^#(OJfuiT5`r-8XLsenKj!pCaL+N%cJYouR3ZC_sTkPG|UvzK%Vq!jH>*M`K>BaB5 zsvp`ZTP3w3&!1~8-eDoym?fp?d9&@Zc;-y~;8sa{eu_ZA*@^h|qn+Vj=%*Zxh^)I< zVgK#(tFFJL_vDNDrQ@Lwit6%9T)%P}`kJiIEu8ShzgKf}-N==7bdGlC#<}SP(~0=a z)}2cQUz;BqZaw?0^9#DLIQfc8!CWIwmIOse$j`qZJzA8yFKDX9S-p=*MU!23z2#z? zr4FZikn`6!RWHiRRP$Yz>8}X2TNGYWVk`j~j+Q|E56-l_=r{8M#sE17S)x=}=zw16 z`cM>A!DGJ``*(}BL)SAoHOptCTwNAK4S)V5Jhi#*qA7Z!!)ZZ!Mq|Tx<#1d{nT*u+ z*Zw4-528G6e~-f7OKqfFq*O|MH~U>LV&|t;*3-Bi>MN&~ zFXhIEn#p^S5_3W(zb^4pVVAx^nkG-Uve#&m4x||?H0m1EW@Gl!4^{2(Qs9ZU;FQKu zZW=71K)VID=#fUhmyNNa>m(UxdqT`#5*ohfwG2NXcH!yzs4S;{^^e#i#YI52*>}pj zbBUd`8touq`R`-H!G)D7ETavZcVeOd7u?{tSI#E7HPN^Q8hTY__t};uNx~K`^iN__ zlS*e$6upbe=Mj}hI_R~r-z7S177NCv$f&E zyK1IMi9LHn_Mtu_ur9SlCTMU5JfXPs90Rp=cQxicmA`S=ED-`Tf;1HL!|WlA?ocC{2v{fRdrV7JI+TQHCc{j}fKq zv}lY8a8g8ul7+j~xN*MwDc@}|B?H$GL_{^mdEDq6djHF)q?#q+H5Is{{0H8uW%Hi* z64+-atW{@#&(0fD5>4pq9;7O^Af^5bJ~+e-un4B6}0nA|nrt5#@xn4E8nMuz2=f&OG;yIqx13ct?ZMl|c=i&4!hjMW)!M zz)(GjP5&N*L%4_=y|RNsf5ysa*uG}4mZcz+!2)0WSQ6*I0AJSf0DQ>>wDh&;;)@#b z<=#=*dNq(oNWg>8I!AKCDNfgfmuMxZJaI<%9_;iJ32u4ekJLd61v8f+F z@5n_=UEJUmW)bS^(TPUMsC zWvC`~JiKQmfSOz=Y%MkWwjH9W1NP)QRQ5Wc3X zfxinO*5+{M_B3RA?g3Sz{^(?p0QjK^U-{uXL))VHF(G5VVeW?dUtI@)PgnB{9;8bw zJ};U5IJq(+Z*Yz@b@qBmyU3us`;94syW(vxz!5ehOZ+-GFV;r5H+!jtS}uZw>*oY) zD7Y%h6Au0iQ$z?ww?0v;?cEN#8{msG?8}Ept&s~}bA$YDsiy%UdX7woK!2Y=d6!n$ zBLAzgYF{tw^*wG5z;OMp65T>~bkwO?#zVcY%GlvIJT+fzpqEcxiy@(44ZbUPIw|Qu z?!5s;c`>o;Ppcw%B7sP;iZ_c#scn!BT@e2D9wpWJGN&E{ueB@we9MXUJ!u{t{Nm?c zjxkTsNwv5)AC!5S3#F~74N&m>kQ@G9+d$sBT8ZlCV6<3>GXLdJTkZt&@fnDC6QMz; z9?ENi7uOr_xMS7NAJ1Z_@&Jhj9bSbi=*({Ih%|eA{7x?YwQ`Elt>APnY=)KbtFe-v z>bRcd3&nl-5%?+Ot*)H6lEs_&&c&?Tt1q`u8g#{*_OI=-Sb4dscZ;V#LK1py(6qV+ z((o67jGAD)ST_mP+ot9(1jRdWj_DLc6L8+2_{k?BM|t)-1K|#2!>`iO>t5l@oYYGe z*9g|`{sq=#YpBz-qOsSYO`@%?_zKrh;_HHwdeb*Wtj^F8Jp%y)`XQfbmR zOxnSaDzg6YM92d_K`7AI?6Y&2x+ame4u)`CRzEJ(&d^x0LWak!-#aZs5n45mE9a7H zTAUgfOL4dTrQ$ylayTiuIsV#@h0#spxe$)A1*m&h>7itQRP^N(pCk{k#LAXq>PVbN z>Py9-<|~;%d>J`DRhfcGp*8ybfwD9km{qhgf(XE&CygRE@vO0m93JPYXk^-AEFV=x z>p=l3n?+E%Wyv4stba|;lQ<1!gn^$1@?RwlinIq_uF4n8DhV$ zfGj=sXP+jk6JwVW*$pzSHXhMAP^1AVW;6t%gbf?4%pANz2S4A|SA<*FE%zH*Oe|Dg zcsFLixb=qc!Gj&}MXz%yu5B~PF1-%U??>BQK!|@V*sL87&-Y7aupeYrR+}NK)bR^M z2iuk6M6F0d*sek!%X%m$T*;JTz=YFZdy49Yf7nlG%V z!`r*x=j(XQ@NA}Mu+Oa|B%7ciAv3hEbOEChIi{c~7?efBaZ;fFJKOw)DhdpXe92U_ zUI3Jqp5phTi;$vtas{+&zFJ3?_bvxy)BU2DW&xT11iYT#D#jpnv_>fK-Ebf^eXM2j zO|j+pJS@g}xlPZ7sz+4Ef51)R4Efo7Sjp$j<5R=KYzmD|P&^;}%%@hSO74^f9VjNC zbB$t7^a^xD`ck)ic<9R{1Ht%-Q;#quhRDF zUH!*onIW<7UoytN*pI&hghTD5PV_a`LW zr3z??4bGwvVDpI4ZwOJ05VRkE9hY)#%F(vi2MD(2)|VDjpPNW_SY8DwtF>Q2?4h}h z!#!;;zp4Yjs10^9qb1o@6X%T8b6wRW~i-G^HwtI(PX#5J=m z*7b(E_l>2Q<%d2dXB#{?CfAMnLtFH_^pr-C;_uIPN1&&T=$L(!g=!gONLe~k)*CBj z4{`n=E)y}a2(c|~bfi8cF$4AFdsjg`b?4@yksQieTAFGwT1nX9ml zF*oTvQ>dl>Tjd%-Q{pBz8=zyUqYD|)205%kpbvm|)5&{oL87n`{=%0-9yTgnu>dsK zQ#&RpXH(;4QN55Ta%pNFX*(s%y>(9~O zWln55NpMwdFg&B}k;ia#pj zSQeGg^nUoOBDFNEL8qWOR-#plq0K@|;-4n}o8+f6cN z>#2CIDck3a#i>=dq=T~PZCbuWjS_{w>w78FrP>#PNj5T%U}&-*iMs!!fOSc7AO#AY zNA$O|nN-P>3)&AjF+u~g&!6w>^QENbAJ#oM9Zed4&_%SH?*VyOs0)?Yqwq^RTc4Hm zDEwN;hDEE=ItV`7pMD#iaozfPo?l&5dK2a?5hc%;ar?d0Q_0y$#gLFzKb6 zk$hGvmT=*+ZEBy2Nz`EMnL-oEXuANi;3AL_l+=J}tfhr+wA#%{zo|iq@1j|xcw*n3 zFOyHd%?W*PA0|IMSdrWxoFU;fX@t4M{=IqQ{1S8nAw11 z?z6JJOb0rGR>k_0)(Kg3gQKBh9E^9$^=V@+Z-vDknJ&MRgNlbLR1QR66mKucOMf)b zTRBfZX)WQ}aN5w5K!|tjSVvg&#!f?y(ZZGbV_73xuI+h-m3m3DkV=eCQ=U4V8^@c@ z*{CXyN7y#lr$&g8c$CCB?tS3^yzPmSCf7%D0+{nhzDD#=&(lU)m$Fl_lf@nmy63l3 z7~H29LTnLeEp{M?t#~q4gzlhNv~#1BKU+Opkg*b-n1rp!z@R5g;tHiYG|!$!IyS}E z+Uym(RTIa?lRw%JimBxKIOg*hD$3W$;D71iaXvSAu> z|Q!~rQ<0wHC7IlKs_PZEAfJ=JWg zDC+xV^cre(jH=|>C(ITdSV9B6o$T$vJb=objLyD=INGf-Ky>sN zUGkCJhApSn7v`jVZA{26lk&3oED&R!N4#z|6=5NYe@WI$FL}N-+k-7j_bG;N3U7og z+e|)e@8JzhEqj)`vX+g*GQ<{ny#V(Sv~hYw$f{dTdtc4f%_m{<@gmQCB{O2Z_u64w z8vId(3`QH!T}Sw*kQ-NUGT~EY7htTj-D9gRnIg>R4SH~al_n$d#m&JN+qQI zxSkvCENL1e-I3i=$J&KCy4_N-wW}r*Oz6gWB4yXZCg zms;Rb_ARcT8re8}B^U9o%K(5rHYsm*vnAyjH5YY(3D&P5O4 z*v2bUnbLD#Vl}F+i!Y1cID<2oajb0=^@aGjpVDxCK)uLY;93)&al{#3vW%Cc6nOYa z)u-lvt3R*84DP2lO4U>^((9Ic#~NU((Xal8%ZNB9k<-UzIEodq9UuHpMQrT_FVzk! zn*IkOHrTotxL4hk0GPmHoUl^^zj#-wmi0KX#UJ5oxLUX!=)e96U)zVZ^+}of-?V`@ z{gjGOtiP{KSU9CqKSQ>Y9!g1&i2Wl~En_OWgY?2JcaRoen#`OGw0{9uvHiacdmv9* zI5k8>AS8p*JOqCgh>dgiGa*QV+%@v_Y2D?uuEDOV9H#Xb>!_V2$`_!zE~tjrN}jDR zLv&4SRWbx}G}=*#Uh%*BdvR$Nd(ht-GGLLpPV99T4qgJWR~}UM9PUr4@v(rtML!g$ zy}u>QH$l=ucp%T<9(-T=;#px@FU^fz0x_b|-1qj&6p{U(>T3A;{%`9cFNi$?g5|O=w4*0*v1E4n>jF8zy z@hjg@SB3iDNwVN1@Yy{eT)Q?MDMJ(S&3cTn{lAcCE3urK9iZTS^wk%V0Q+po+9BB>g3?bk`Le+cOb{f{ z#K{JiLK~Fq2u2xwNHg&WovCg)&%*wfqhNQoz?s@%YXvic*wGG@VJ8=rW+VeQgmVd? z%y|iVK+X-qR@mKY|N8OuJlfZD-hwAw4rBkY2UtVS_9=5xf!I+9iML$AMCUvB&A(cj zY<_6@?DDPZuz$y+UGPK(9aeUsvF#8_&XOAuYee#S?d&2ZUsC7}BE&=k3dmgvd9{n> zl%mp0@`~W?3XatsKtvVqwSGjS`VF!u37{-J4ko&5l^2ewl5<|8we*U8;o=7j(dpIu z{@-8qKlOIGe+)wfUQ7a@(Of&&?8udyRVGZK$&)!!bMfpAFQy+q=7$Y_jMyDk)VqLD zJuLfS&(V)DYFm|&2dTo2#TjnzPS%D}A`D1k6tpMzlW70c6eNkC@Li1m9%QInGmmUF zVqqd1Jp1cjv|-EBM$ew3?d_-@OfPDEQ)$itLj@v6SEv4*P*&j+bh+tvbM)E4KoO*} z9#ATN_D#+S)Nd@IX>#6=6u`4x?uM>*>fR=lsF)e7z@sR?xJ$*E@6G;0X~9$_L;rIq zxxk@fo}LlI)&d}^a9)~6*hOp8psdlLK=N+DB;M9MQ0>zs?_kAg(p>E&!(ho|L?&Am zGxvN%VAix9&H6n~*xF**i#&(wz&ytTz{uIPK)!79F#GN-{wq(r1%)-K%7)0LnLWG2 z${`k_msm5E)1x*-k(lx=nkhdxWUB-U8)sh>$aa}EjiUD)&rcy*K5pn1lJ z|M5P6^WvpDwVvDZK`@!*MT&o?OhRss2Wez)i3_TdSKIwu z1)G@}wAo|ySPEfYm}*;LtWDk*)O2NcG_H@)38B%bf@)k$=9@K^m+|Y#w_+2Oc``$S z9YJ;XRZbuFFl!RTWuEI$j2y@!&yMgpAki|swXwpjt4jXtiI^WDdAN~*0vD4lZS=23 z;R~+*ljOpG1<=JmG+fZ!uaD3n56M`G!_g&;jHi~CezeF`29>dqMfe1gU{o}tS&Ec= zR`s(nkxh;lG{&hARQK1GuHnMTInC}{?)|HzP_ijQWOGYo*q`IR>BeXcJSDBF-M?Ur&qW({QJto3R~ z=~V9=SKEEU1E%7CG*BRntZe!7fJE^T16v0;(RUaY^QS%vfiKn}#Z|+N;TONTA(MVE zkx(rbe2xE1JGuqd>A-{pVeNNSbiVFlIHeGFcEThj1i(-->m)SYN(A-)E6~VpyA&uP zJvy3FKD_ZmS${xoIrv|Kw){NkYQF=(GJEQq_qVs^po2=I+W(dC=Q6BlHS@*KI%3$>a$x^3<(s|=nr%!pix3K$5x|9ZSSn8;E7&(0K`((p zmRsPaomuHRYMfUFv)!yD&aFr5oYd;o{t6c=q z-{uyy+FR*S{Asbnave3ex4O%trx4Nx15zhX$$hFNM-+7*`3dDDUdw6iU8W{4ol~w- z4-2MY<(&=-xe2#u6znHU-5Hz(@-Bp&b2X(E?Q`If&YYD|GTgx;YMH#S zct4-`XBfn+ff3aaw@G}x$kKx47lKCPU9NU2j6?zfqIJ_Kat zR=J021`)M%Ya)%BSicuVzA1jvn2wB)*&!Cdbvqgb3w;#;8{G5_CG+*+L~~Ef zg1cma7SO4tm>CfF{-MA;UCxhcvE>qU&A+ywf{ox!k~>{+Ub?%7{w3OwvMB4%{e^wX z|KNCww5ZNu*ew|8z4QjCZHoZt7zZ&^&f$i~T!#3&6S>(fOd}V3M~BV_a6Lt=ZoJIE z&I~N3FnB^7Het@}fPAtbqKA;7dhydaf$d|TZNp%ul=OwN^~_t(-UQg9mL%oS$=ulu zD#h+Z#%myHy(W^@mzXZ=`NBH=`nEiTEHmi63pHf=zgV#;l4$NDAvmeMO83oJI4KKo zk+qh0^u@;KS4^9cd1pDS9LHgvdUAlUGv`3%HGjLd|O8l=;)m>7WlTJ64b z1$hKcQ4CVYg=;@jI6*sQsI6CcwfbKaKEC4l0 zy5W*7o?iYs7mTbmQ3%JQm%;^f-hI^1$#`IS?~3M^zhmfMF%qN;EPPXb!Q^kZ_r(hD z2VX1eOh_Tpc0dLAA%t`wSocpBKOdF`(Z2^o=QJT0uG@CeY4g~WeGlj-R&gfn>pRK#1N&YR=%z*{RyrR0|skZUnJAO^P?z0AR4vp}K`kJ|(@ z*MSe#rp-F%Tin<&YKlx9f^19?-kr;Ao4>agZ_Ga?^Cx=*?cWSY=b(jEuY{_R z9^+iy8%$A-D1;u>uv=^AhIQ2E51X2o-g%6#n-pHzEZv5gO5^z2+G@*JVGSm|@aU+e z!uU{=V(0;;gS=}PR4bZ~zO$z(JopUWfHf+yA=V)Pxl5-ROuE1d?N)mIG0>-cg#B0o zG8AifhfKoUT2zu>8tzMOvmF9?H?kTjv6bSKEPPuHgJbL4mJUtyi(^Ht1}NpV zdbH^3o0-b~r$bA}4e?d-(v_gm)?&wzl2y@tZigh^6cXI5Ki5Oqj26c9u_fEAFBhv` zwLb7`tNacI3115C_iNh}Kba!FF-Px03;wv@PzyM18`IB=23L9CebDCnbHf2}Do^g? zgAAxEvb}QM;GTe78`oIl`wenJsJGj>ueK#XM*8ZUp6~=Hfj-}d+G4sNX%uvaR;XZxD>(-mr)M)xcqZ1q#m3G^YmKW8K_CsKhS z$9of!JyDmHTjrevo#g$X!(zl0uyXx1yQKN#??6zK^U|w;SxB7HkfeGB%DP=l4O7hb z29DE&Q~%0Is3fezxNog5_e0iV`(-Lof8ZUW`SI7)I@pk$vhvkH$=xR7rTny#`pu3T3WtgjJr1yK z^cM%AselQesA&L;IY7n`aZRx`^X6`CHRuD9Snf%&T+7}=zm*wgCCZoYqnjDWKS0Bf35PdOV@%mgWjNIRsLE?R&MRu z)oeZ4>zb*%i!$b!V!t0nz83z;iDP6jIj|4`FivBP@BBPF-+`k8v|W!h1>Wp^wl=?k zTvQv2?OB%pdV(!G-V@wMWJ-l9s(i|SLaaIh(T9c7@*#d~HLfqVe{8P#PgW%+QvY37 zNu2gYKsM+@{GN%T$Co4I${KO0`*^q#k=@lU>C~D8h~JeTQX)m6+||Sf1X;?sc#ya# zN*W?6F0Abmf1n^Q?B~j<5)j%!{P)cEN#bKb!hJ~xn-u2$_e(Ib93ECU78BV3S-iVn ztE;YJMRrwl*tohHApmZjCvzFE;rl3G7Sn_PDLl022^U!sR?SaGr-Yna;TfB91lVfE zT1E<=u(d4fdI*4rnIE)9hQhOW0A!z+@mdqSVQSj9cun|Y{+XI$*Sh7l+v?|;C|;&wPax@v zlqf!Uv$#MyA#+|_Y-fqW?73eR%H}SeOz@8qIPpkpx-D|;>iw0__EIiGK3pG-*JIWS zFqJqE=*!J=A(xbC!5j7#z62;|3r6HGAAf-P2StJ(se=QwGw5)HiI`N=6WFf7A8NHd z1HDOi4tZEVr61@~&|;ed@qe~eVC4e>ftWqH_>bX|1GpVxEh!tIBu2BmKf((Vrc$dZ z>c=Q}nVGFBaP2@UEnRdUK(gW^s*Uj1#_Ea=5ug|twWNlZ>5B&<9*54}xCZI!>xG@& zO}BAZP4`VM&l^s|3GHN^$|t!gUTU!oR8VhpwM4x1$tJ}qy~RzLDv;7!C;+)6f&cq# zKTpS?7TaOKDtsGkUzdZv8zECCuXSPBWi2*gxQc1xyMmg+@R3@=TH=t5vy7ls)f^Bc$AdzY!nmFwX6m;^ z1>h+$qNkJ^Cpw&uI=MJ^w~k-H>dhzI?mXlSFvISq5;M&BT;M#4GS4?@;DCnkd@>9~ z4sb2`x^Q9%=8Fkt&cv<93%wJQ2{`ok96s8%H_A8W<`Kk;yp(tk6E(t$Tw8oW_`nvA zEj*o9aWZuTG5?)vXMr(MtjaYjm5RgwzmE$jLCjr(qRY?$r;pnAQ!1Dc0ojujDZ2>E zd{CV7aIxiiaJJ5@jKrMS4{H35{LlFbxE?sCk7PW@jffujZn+YC216b=iSzi~uJB$S z*KAl(Kz0|i6z7g}#qh$h3ISq1<~WSze1RNthA=#i#c2AE>=Pz3WDp)#7+_Kb=X{zJs89i7RUS|2XL#kvyNz|)1(xl@+wgvGvxaj&p{`_9GWVcvoLR;QM&|EHpBG2nio!=mKq%N+oVs)Z{HMdaQipB6O-E z5{I_|&P=(L{6h%zzVdCQNm4DdeVo1zsB!d~0SsuH`H@b<8BGNz?JD8!f`+l-C~$tC zPbly00Mw;SHzOHrz|gn&BWx&7*gS3lK3=ghX^FlA{+I!~HYKO!!=4L&Z+E&(kOuqt zn~HEbKSso7E(HOb93K8QLq>0UW!_u&SwFg*IFIlXnAFvQBzDwCnJ2|7nXbdFgJ<{^9F zu)WH*WPi?43yx0G3Vbn*NV=Y?&T`xnGXsLsao-WjgY0N1oFvwR_zJFwPzF7~;lo8o tnBhPA%ivYO&7rFBW2yovt?@gDm-dgIoUwW?xd;By*3efkR<*wMe*l6CC=37q literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-5-storecontent.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-5-storecontent.png new file mode 100644 index 0000000000000000000000000000000000000000..32edfd280a9607c008eccde31c02bcb46e3a2eff GIT binary patch literal 74751 zcmbrlby$>P_df`T(lC@rcL+##hja_l-QArcjdXXYlypl=Nq4t^G)NCI#6AP>H?H6A zwfo18&OI~FUFV!T&gTRX%8F8G$OOnRFfeE`((gaOz`PWLfq{FE_zHM3)vKQa1A{7J zB`&TkBQ8#+?CfA}WorflBOQ^ffuN~6@H$5)Ax^{+7XEwiDttVFh%Dl}PZvz|(rpGv+%>6d+p#7}vroHKWEnxI+XSj*| zY86)Gh)0TQP+|tQANzH<$>;k$nSr;ild2-{w<3hQun6ef20c-c$Ou)a^#RPs6))Z| z>Q|>)-`75NyhbAt&P9R2dg;W+oSTVoDhhLAT_LRt1EVsSsqhXY!&)XHhx#JeGatud z(5W7$Xy{ZwT|pNWWy2jNNsWidi3a9JDwz`VUvn|E8UJwN5P?7t3@2ZYke64Y7pO?o z7R~NGNbVAUPk7h#L4n;*GqHPs==WJuf8>WxWClq?=??*gn*phxoL7!-A!`>8_%7|@jfs0|eb6>i=awT*B>jc+D#`TFu zI*kys;?ER2yr^yXIJBE|>I8Y1IN%4$-%h5H8bjE%}!LDU>NU$+ykS z0}&SV7X}um3&?)*^>hqOn8wIG+aaHW5yL-{yZ4VppsCJBy&E)^KwSu@W}UnqA&*%{ z3A*39)ies_p_4Q+NJ$muFtAlk`9z%m3xZO!A@=vCbFqm?>&}sBp1M5_U-*0Xm=ZE+ z21+a`h%T2O9K{?J`*GJ7gnQlB-?xFIVxQDS!o9WgO3q6)T<;9U1lQy=815(8)~#n~ zF&Vug39Ww82vT%HUl8+)^-nM1V2Gs+6%Z0dxj?R9PH-o`mlEg?3~N8UuVJNOs3g^( zckl@TO60Fx(5dLd2~U6)GbxVP!vPCJ$6^qwmP5Yndvs4YClbULO(*&kgw#af6c`e5 z$rFrYBF!e2Rb9=3`^Owo;|_ND{$PeT194K#i*eMfac@LW$zNgGIr=RlTy=G?&tRz2 zW(E47j%>CFt|R4tIf-+O9#{CuDWlBz@OIECk8mI1t#kZ?36rB-@m?fkb5)s`ev|K~ zP*tB^9x2T~6u}m;l!9=b4BT6##emFC3?*?RDR-)Ge=$vkYc9tqKiYlN3;X!J>48=F zOmiyie1zcN&4Tbj*x|V$A-I;h0mT0)yHn9{r;1XOGS8%36MHt0IC*1z=2EkvIkdI6 ze`6q!?{mO4p?A%s{5JnAFPP6i{tEY`NM>!~6Q>r39%KJ7eIENdUZ9c%N)!6GUBwv1 zQ5fkC=CirEOv?q27ZZEeqa%+mV2)=yJ3G7An#R>(z)jspYj@eT=D(D}*kCL&o3Nbn zv$_lL;C$tZJF^QBqCVkQg`PXX+5bYh4io_)Q-~5tzrr@U&~nWF(y7LRJ^Z2|kBIR_ z=`X%2Oj}s~U-ng`9G@xw!1JIneUke_@biU+5{4)2Z72T;z9) zpBgtcP7pkWAEHI5-+&GgbKtqiBjRMzWa+5Yuo^JYF%>)8ro}y_sIc6^)!%#k#7m4= z=)v#l?YZm`?Ah#L+MwKkiFiLrxe(tf6HkR+@<~lrTW(Rpnp%(MGERu@EdxFS@7tGe z3+RxNBa`Wq-RX%{o69cL_Sm(uZAHOUQVCvv_x=|AE&u!aOu`xChFv1aSmLy>N2TWN zIh_sNWs+)AOA`0s;9%h($>78w+*Z?|P0|&8?nnQ;I~57lUiFZIgdY?tT3>7Qwtf!2 zOXgOipGnx4QpkU;oT_Y7LR)fGLaJu<1O12X$DDnl8Ty%5GebX^XZo#QTg%r+SW8$Z zTO&*}{kSYioOw5+wNG@pu>Wy?WS@H)fy0u(goA?fjN`S<;WSUtseIyhs}lU;fhop` z<*!~J&N0!xrB7&bFxd&&F&yEPOMNLeE9jjaoTZ;zET1WvsVpx!`kI|5sL&z{DRyr} zFElCXo*kIc`cZ4J(>z*)8A_Q(c}@8`x|r0-tH)<0zS+OocZF?5uzAi^cyHno?$Y@p z<$!a4WvcJe?(*AZ(Z0jDaKRe)2b=G6(*OEv;vrJ2Tq0<;XvV1KZ){n2S>CX4u%U9{ z;8$=8Sj}5}H~DLkWLG!2o2F99Ix^pScln$)idhTFEa@bVv2A@Ghl**p7h8Siv?Ec|hvO5Wxhbqln~Jka3w z;I7tb)2VD}^2zY2hoFVNri`pkbD9nMvwqClw;A3irNli3H+G*B;I-hh<#}}B%{6Py z7;rdS9owqV#=Dx?*5bD-WF?g8+w7;{%iGS`j&VPH=jp$6m&0b4wlJ(Woca*+VEicl zaIBS6rd75Ht103rq9tOHXKv{Gr}-B|v`{o%)J^ohv!avzb8Hw$tV=8F{>{!JQnLX*QWYKXc zX$`kr#Rfz$yDuaZ~qxV<4oa9 z)(O`^ZWLI^sJa|>h+4C5+;d=91G}|2`dr1N5g*n6tFPxrb7v|SKcI4JYx&@zw@6&o zQnK3ae@>DuxaHHSKd`z3nFQ;tkv4Q3!EB1$i+%~339}3vB!~;$L%WaGie6C|P|zF6 zAF+F<|1J!jpJ>&;=dbjP%F#EuSe2M$B0}4D*2Fwae5)Q5=RMyVWb8Z?X|h`vNmf?9 z=ntnCrbLUBa*T4U1P}bc{k&r?BRlvzgv>wnEO#A`crIEFj>Gs?_z0Pr4Q!kLI+YaI z*v?fCwWliSBK>r}rHAZ?*kH~zSM(Wlcu`(TUMZ}OSTs62v^DE_Le8_BJdiww{*|ni zwmLX=Y-3~!gI!zPNc})3d7DOu-TbjXWAz0_1MUin{vutMn|o!5(J1`|)a5=-R{O#iur%^;4An(gHY;#7nZ#7kQ(j z-EO;OH2fN}D401fxFmcP#^ewl3;V~ignRB?P`>|Q`m+NzfZy;9&Vb`u0!)fXGff$D z1qB#-;5#A=JS+jsOW+$U@D_q4{O|9Quyimlo}RBJc@)#Q|?fAVxbU1CL<#ga5gpP`|w`! zKiz>}f)tjnu8w@n%pM*dOdjk^4$c|KpK-`TrRJ`eI= zF>$tXbhUD@Cxeb_WbEMPDo8;Ao#?+`&vcr3TK&&V_AdWn3t*5LTEooB#KQdF zV*_0UpilXftvtkn-~KlE9{!ciu=7e#=9QUBTV!`ntYHDyc`3sKv75zzSk82drX=ys}Y!U|R9 zir@Bm;EDlT`dRjZmqm`el?IFAuJk8(g3rGmAFeunxsLMgWKA}??EHeXio#KeK>uG- zh(JCvpGBp@0}Z9~Uc=QH)Q@l0QpG939nl4E*BYkLW1Ucnh#yLY%t2dxf*&FTLf z>`AiW02u_;(`n{d#Zbb{gj(g`nkYoXvr0D%jh`u@z{}=lH-l5iWq5+Vx@0=Ozi%fVdfzLM5Llk3$2NlWQ8+vndP*8s@VPJ#h&SaG8+@N`LtzEm5HPJI&o;=PB$w(Bg%} zFe(=|u(sy(3fm#j&8!^;5O9#h=^_hdVC4NLaA~@1q8E7}1+W*&kf_E*0AdJ>tJ2h- zr^f;q#6tEbf1=(JN(`EsraV2;dnyrf4hlMN%0K~79{H&FTNbqE&E^D<-%Bm|`25R@ z3=mh?2SWmBGA+0W4b^)sn_&zphLl(@ach_d>%r87t_UhYq4 zCo2c03ib=T^jn{a&COJ1CSv}&L^ljiokIC{`j9cRFET|jQvIGp z<}mSDxTx~j+KB)j=(0#RyYG=@5(|m>XRwS2`8?hPnWm)k^}m*iiX-)ZtKV|f8aQdU z9aFOvPib~Hrdr89a8OiR8fmCXj^-0umnzvKO4c22=K_^RNmg5 zRcYXN|L!Z;?s*i$ju+3{;*mh!*G6mLC*1qnX=(3EzizEt0`bE`PSkya=h5uc>-Z*% zV=DgalXl}?n0-2H+Kgc^_fceWalHiZHL~Ikh2b_UK{S5)~bt~ z(%M6=!x#AM?lpek(8Sj%LM|RDf{HgHLN_&@N6pxHDedR3DS}GG)T9Bd1Gg>U;D?%D zt?d1)MbQZpxpXDVY#@KdVb4dHOj~MW*zLpNf^KHtaIDW=0i-^ zh1dtjYlV`ogx~46Y@~&l{<>oygtdrGiO-d3K9s2^q+H z+0x&+6O-a0M6cg+9sbg++Mj)Zd?$d)A%Le-$Y)|5WpD#!gepw$B)*mNhv(7f?$>eD zkTJ$EgX`+nn@;B0wVN;|GP!pGwM1SMeP_3jUYi|~)=!MJ{lm`F)VcjOlmQJ7qrAnH zjQmxlL@s-@XM^Dz^|k1S3_}?E^nLo^h}Jv%)5UazVP*zKzAr=pH$`U~{K5+sN^1fB zT2`#OkRDxzTYf)F11+o53g@om+(DG?FAs_<4*ptZ5n6wHN`;bl*68*5&DL(rW~eRV z6DuFoa{OvN?H=DX_t&6!BryETq^oQVvXz zz0vklcCW*NQ}Dt!k3H&x@@c)%rWhFlFU zVu4Iu=RQ2=wW(ek1Uv3n+xFW-=e=x$!8Rp7n|tD)M*=my1hqq3kGCKG`PTq!+ik~} z&t~aW{In@szU<+iT^ldoPAQ+u5wB>rC|kbqJY8(p8GW}@!5*gTcdXg8DcoxVv1LuQ zZTy#Y77J-lSR#Qe3bo8_y9~r#ZYS9sKg!zs)p^>Djcg|n&*rLD1Q=#5pY`(`I9w@s z4=NIZlL$$PXt@Q|S{s&r3!3djdoOl45!^c;oc@O@}1lKuH*z@#6{EX-Nj)`t1PH=oZHb=wUYqM0&A9h&VyUbPD&rWm}DX@0A z=I}SN<;z@yz70Bl^W}01@tdwd;y0gJHM;{@HP@~#8Vltk=eP{~xOgpFytV=@+7rLn zKX%W6T(rhd; zY%F!gfB(w26D17dLbSsqS|qEJsO-4fGq4lf5%5v$?cwxSskIgi5 zAsVB;PR*mdbEgr?A{Uk&HPNDUUGi~M4jDqOMr%Wwl>sx?(IMhgJBj6@)=yCs*Lb5W5{98wae*2@=+L48iwhea zT5#bu;GzS?W$AHYf86Gmx$XT48GC*cOp|X#Z*=rpxh$??ocq#Wl)np82nzxVDPfp< zSe}a)mvWtrcy^xEjP)TJUYX{yoO-L}~J?zYUX$rrAj=j}bpbKK`d z%ReGi$nAj7#(Tdk3Sr+WI_|RP9|yPN-fBAT>g30An(dPoGOM@FmYDG00Ge=U+Ulj2 z#baZ6N6gXEz@`SHJ;Zf)sWV?Gtev>oYxhN0{;|h>^~&Lu-Wc5?{+-h%pe$N)Z^Y#; zaaAysN&hGV#ocY}Ap@6(0j4`F7#1BU$V=Kc1|S#lGbtG#L-p7tbG*LIGYeQ%3bA>n zYn2n58n6oc^%9ca4PM}VxtG5{@?gF1`+dmZZvi=%6*GEq3I{VxEBvZ`2DilYsp2n?kb5nry7=nGDS5-kV7 zeCxdpi)|UGVPgak&1pNqx{jturr;MZW*MY`PSa5j%+#pv#oh>=YMc=Y9Sf+^F`l5~ zI{@63e_S25^GAe=f-bzyCY97t9tnwq5Q-ofRJOyO5M&U*j7<`|i_8;(lmr-EkWIR= z!g-<&kOxklJS`FjJ!h!1`4FT(F`A++OR?;X5Pd)^CX*n zU?T0dS1ga06yq#nyLD?Qllg*5az8XAj9NzkgY4(cP#Q1Pq6##p0_?r0f^A0X?gNWYd@xh{j(V>oRX*{v_KOO zFY!7a_Qk9D^gv3f7#zg!2L3k%e{Lgab1XxYJbky4HU4&?Fh>cXOuWkJ44@At@%$4w zS)xk2_J(|!>F1x%?N>kA=6A*D%qLd8cCy*}h+O(JY$~+gy%N%R0j#D3ay};xDk*hf z8ImJeI;YcbDX37W;@|<(ii(a6_4!s|L=+hhu4)=$YSHYRipGY+|7TLT`?e^HqS==3 z7#O?J*xf|EVj^U`jE7TJU`*gV3YA2FXX-Ck6o}7FTa?fJPtv-)4FaXjTR0-=1~n)4 z+NVvz0X^b{bm^E+N8B3@f!H-}1@9t?eKZbywA46oTstPj;IZPCYFpj=x(2*xGXy#A zU^e+36;GS%`{A{=CApT{%*O(stA8~YlX9toU$)NEKHhdBkoK3=j?$wrR(;C0)7~Dq zJ6*J?9EcTuperq_iNw`i^V%WiXLpL`oTc-+zrR|u<8kP?Z^%xL#Ls?#fX=uw$IM5` z5eY($$MZ__+D2&Q1X?h}>tgMjO8KbseS#y&%ROpA~39pPT{EvQ;y-t3EQtaYtUBKB!u zs_wx8|M`|DC>8)Y;`Rd0;LBQnr!*{c@TlO`vK<&Pq+=(?XY^LfsTbESso8-pr<*g0 z_benR9rS7t#rKYA5NTN1p3Y+$O#m{ZVK6YrP`SL#TTUx{f01bR$#-Uxj21Dkw6fj% zt(=nZy{q|W%Zyv&uk2!SuGfIgO@<3BNa5$?z-RBxcX`0aVD<|&2f8Pv$yniva=d@w zGS2CF=sERDMZ-{#_V9%pfM)K{UrZL9-wvxP2~q=<*|=R71VZG+NA+|1W@2wrmc==A z9NI4bR`-$Au9CmXaqL0@^DFX55d_?9;XxW75Jc^c+pb<~#6?jC8#Ak11J}$#`86Wu zu-o~Va0ijcvJ)36bH`nc&yYQ_JsW3T}*h#Q*DY}<|bx8qy*n2Mwxh|3l;M$219MQku#C`xV zE2hnLC-`5lBlwJC4Zd;uWtBtrQ;eu*(kuMiKAC?Itk8@?I-5jA>zQmONn7ZcXwa*L zHXU-XzUzdVT#tvSskGdUIrB@NP8ss^w-ULHUwo)6%3T_PI5PvwClZO_TFw@Wn z7dC|&1fD0tYp25?B$Vsk4)a6xIR_JP2SR;XSx+}~JWP*Z_y^EGKKDb*UTH)RO_B<^ zX7b=4B0n}tG<>_GI)URlh3uZT7nZm^WAyfxwU5XoghS)6D!J>>yT-cD*|&W=y?6$; z7Ivo!(o9utDtGMtH?gs76^d}e`{w#CmwKNabL*BKr^oGDFhdwF`klN-=b zOTW^c`EdOBpv2U6@u4Lert33guH%6kxXLO8`w`*39k8fkcYW7+4bo!O!S31s)Pu(F z4PcVwDJ5aSvnXZ9;5TU1ABfFg+324jaXW3i*P+og$Ee1>lpsDsKP9a7)+5Kcg!#$_ z&UH%{1cd86CU~W3)#RpLCU^lc=4>7rQ>114n6V$!~H?UFT)n%(r&`j12jj8Xo8JjZ(&LiO_me39(g$}PbI$30(?6s=i%pNw^!C5MoJ&2*{xK%%+dHhvgDX~@Tevn0n}&M}>{lJK z{qovTKKI`{DkHp7TE9?42trIqaZ9BB3&L-cG#!Zn9$Is{4zK8s(J=6<*ORSaXI=^Z zbKwt=^-}Rw7FG2S*~?>Ef(`sb{Kz&D?p05d1pooPx5xtTPJFZ7{vH~;^*51LsR8cY zWZ1h`j&Yy{cx6`Y`&*=ZzC>{Gx0o8a+Zkq|>fv^Ea-4PB#ufJlTQ$^K0pm(QldRzC z4`$_pZnshm2DL1*sn`4=V=TkF=iN2~I$2~s{1~ydmR1m6FcJV+ zTC=SZ6{7=}_H+F>74>JM+TgtJv-vpkEdCKvB+l-Yj$#SU!&B=cSn${*!oj(P{%7KVSY5ibk(^KS1lKNA@);v3QQQ@XC zu`uLiAPW0E85S|1#kor9y_+B(HbR;F`T|9Glo4q=TNJMbdjf@RDw+!0aH)cg&B; z3xVfN++_s7QEu7D`=XHOX8%q9E9Cs*IS^-emIkVVcZ6@oEk($9)#R2RRzD`Juz1

z((FQool+|PToo4N;y4dN*ZL3^l%&YB4iBLXZdm>C(5!EsZ!95%} z;7Qh733!AU+}xdy9Su*R6Z6dv0dk2)9L|#9n-0Y7;ST@KgGS<$Blh&~ zzQYM*R$SU;ao?b+#C^!co=FoW<4>7Hngww`WuXc>w0KO~nQx3v-U2-`Zu8x^T?Uh4 zbX{|*e^{3^reY50Z8utw^FcpZyWdzz5a8)LiY_;x!@UKI_u8r-DrA1|9#&M)QNvW* z^ynQ=!4;#u2Zs%_+`A42LoGEV>^~xGMtP_)&-))6FyYb-&8eOxgarDPFP>H%zTjOu z%=YEtcx^4x4KQD;aiyKac;3@9TJ;%;ID)Fl+(pBKc;NFDEG|V)f=|Mz=RquZfDz&I z1|1yOqyLSE(P}(Tv>-_X3&0L3A>#15_Dx%dF7v{!0IYH66$TT{1i1k?^wV}^VL|^e zp6v^m2@NG7rL!ukLknbb>@a>bBA#&)$KZQWaypYz7^q&}pjMdig|O| zM|!lH1M+0ubOx4F_1Jxz4&GcIv8zlunAi8(u_c(u%PB-zz!iYUi`3g7eGP3ZY8`jO zQ0LG`j$q~B*n1fjVP*AVykaS7?j8eO4RRbS-@oxvhqfk&Eua}z@z-Ak%sVIoiW$Bz zHPf}y{38n&LySK7;m5?7^K7|?ngw40=DIZ{aM0}Tt~s9X4?k!eFpS^iKXJeq82CQ^ z<5pFD-&2Xs_Iyy;pDI%79*VneXOF9%Gr%SojI>+xyUNCWAp2C(9dSIT>k=*KNAi^r zz6-Ea`}q#arnYg;Wr4*PUu0p&sYNCy=rZDZ8{9xH9-4%dLm#ZI1p6pD-HX}-Fol@c za@^jc>$R2auB1c-i%Q?fhPpsL!8-%aBZi`v-KNhP zT8#<@5|ZvY&yQ0p?uNuga<-I#T6;$ZRg~S4*{i=;`LWm)oz9w=46be?&bfoT6|?yk z>UzB@Nc>qT-8Xt7Qgk%$21@|@*37te&1E0YxN0=`^1NtHS(5GrVs5fxvBsl}pRqf` zWMnfedYUL|{&``h{h0zcyhD;hqx}k-g2jiMa?UL1(ua(5aI=@=QsmMGnv(&G&z?ka z{^S5DmjZ6t%fT#OsGkz30}KZviX6)+dR}@3%`vCm=ipbU%#; zzxA)YBlL+9aM4g7+v!jo?^8yCI@FkpEEpAdGUhg+p3XZqQ*?qv5vZvxW|?7gO<22| zW{oAC0z}_y1n{4-1)hTU9X0N?}+V1BULe?u0XyI(Z3ps6X)c2eTaL6nQsLe_#MUd)U*0MFF#CRI8;INs<9O zD6JTju}$zGNk>;04?jr*aL8p8`s9A4ihBiET^z-1^3S@k4j83CA#FYJlkl^*>S5R` z8Qxh2NIFUF?Rx)w&Vm6O0vWbv=|4LUSdQc&W23LypG3M5^MS|)jb4kZWgvyoYnePD zz<+zton8Ix@0f-|$DKz=(SQSBk03;?Iau8k(ZOr;AVj zEYxP8$K=+~xJ(`iJQl84BadCtv?K z7SK`qYg+mM5f*f(sv#=4paq~0g~(6@4g-!~i<&$7SHHvJ6o5}~O}j+K@QTPd_AX%EyWmvKiBC0un_9fJ zEbnFC)9eZPry-kz#+o*}G~s`fDl)z1In9MuO+{?J+HF%PK*_8!#CN=f#pySav?E#e zrOtq^elP|+l&kdr(XiaM1G)+V&cVwr_U#yQpUFuCgD}NV3V$@PKMLg1>OahxshJCh z_1RKxowpizuCiRP9$%`0xitj#P9cP7D6KDf(S)r%X}yCD{8kD5r%!(25(Q-uKD-4b zUW$b%aSL2-_-YLc6PP== zO1L=2_lh7i*Yq2v7eJ>{@lGZ_((r=dH`;%;*WYhx{9cbX$me%i$1SrY&N9-{Cd22~ zUpq_eTd5Y1@X$s@OfpHzJuG`O!X~iDJYI2V*)bT9>vX`kyIs)kBd5KTS2h)P>@U z^4lPQ*3KaT9pmjxbI@_}zS3kz>i2H(+s#m|w;fucsUDEz!uI(Lv0YSUw&Al(CfZpa@ex*3G-HT*8d0;W%t*Agq%w$^WoF!pwi$2!{HJ_;h7Pp!pn&F6J= zv=SQM*Uy~S@_Y*gqg-Z#8GZ_bX4=0pN!4T28FU^oAK?N2G=7?WzrVG)lZHq@wqh`N zACMa=XD+Ym1liUfN|b)y!UY6u0YfcS0y?wrQQ%{<$5C-Tb80S2&#IuQLv@O_bKFEHC zRbO_l&!jT-UqeFIByBfJSuM1+WrJ{_1B~c)l~b1FE|1X2PWI@VUxW>5%~k0Z*HyX7 z3gi>zefgqbWEwmmxWsX0d)q+2Mqrv{U`iC{-9QYBoYOD{QAbvVWxPqpcuGK z1ilOX@C#qWDAw?Vwlt<(R!~D+*)PU)zT!o5)SS%q36}J836zAnE#jNOXbq4;!x(FF z2>9F;)KP3S%g|~)srj}e8U*a!dMC-}GZE-9dC7mgPCj;d+eUToBdNa;imy(3fJ=gq zKsPcticV$KK`+kc@$I1VG<15O7-wbbiIwUDCfV{FL~%GU(yN- zyWS|Q{@{OrPjZB}s{yP^^3Co_afidTL&{q0Ht*q;0;Hh6=aJnC=;|dV(tNj@; zOWTecrZ}VFLeLguHgH}JnZS+lHrQNm=?&?#nj?hs;_k7-(hWl~RYTZ~N`UGjF~ll6 zbXZgSL=_&&+b0}UhV-}1&i)Pg1%$LnHdgCAKobiH72ac;_(B!4a=OSWn$y;_g-dGG zf1w^Sbl8TG^}2q#EKENrq7j*eKU?K0dcq?d2cJu6+FSA5IlVU6pD@HO{5)4!D%t-V z?d8i{wDiIDKEpoFJ6c~%_57e(f_6x;DJ_Aay@D{2+C6p zP25!jFQDL5$T>zh?_@qk0`q^kG39xhU;1hO*Z*(x|Noa==BcV_%)jg!rVh>10+o4( zXncLyZk=58RO%gru7@_$+zpF8nvQ4qca0_e%?=!09LQYQ84fNDWPndp1Y&n6a8`9A zP5qjmBcy{ck2F5N?13gHXeWpe*tsN-LA60PW`=3u6f#3eSLkWm`d+3jo1XZg!j*lH zNBoM$+Kf+n02Io?Ao0K_SZ8LFW&X4I5C!133TqZ9dU2zQQ(MyQWBg9y27Ua2nO^c4 zlLL(jF|`0(=-$^(>yfo^7e-C&+MWMF=I){t5$FAV@yQeN#X+j8A&=2UNHBa4!A<76 zhk`f}4mg>*kqKjKT>}}GulLr=XBLCNlAXzrMKth^WEK3EpI-RwuX>z@_K+wb*0NZw z_K>i`G%bAd6z!)vp$*ht+*S|NM4?=)R4<_(rhVE`)>fIdiZ*QKeKdsV(+;%W9-_O9veH{+&JZ~4=bm>>Ucu*@z9M+GFYN5S!duyCs+t| z#cCm`Epc%(V08F?90~^y#nMzhalr=v!!9#1LQo(HxSBg2{3C}KzVTwno$17^(^^+} zPwod&@9zkYvpYQPb`;G179c%UMGY8&0i&Rg19-~85NN%$ZN9ahT4{|BfTM}OPnpKj zp(6EmE^O6dVF^!{>-b68^qLfBh~NDb_aNFB4u+%{ItvO8yNDZCab_tQc@jmjT*TTF z@XQk7v~%%79C_5>gf3!-J_{6*Us5Y8Ea|S{SdjWaAJ;I`{+519`2Jy)nHi!CEMnZ| zYU0wUWqWr*>jn+@<^o$(q+3=-@hS$6K=~9*-K=1_8gA-Fd?>+fcvzlAQ)? z?q$0c9gZ#5V!h2yW>QhEzxKbMo9Ot0-gT}=iysyFk2vr-EaNKIw3+f;7Nl{ZUk53dQ-zAfC1qfUqu!bPivEm5_1GWFPH5;FQkP;o;!dtcS)N_O1w z*scr2!BaZD56Wq6%InqT&ih@qAFnqtpb?|9te!#zo#h5z8xUXw`d_Z#q}w!X^D++L zzEj!t?<#C$D_%!GJMin)vP%ETeaI8=2wrkfNAx}EMA!mCJ;{LcuU`!19TgDXFFS0z zT7~{mLd_p&w0m!@qob_>c-{|04>jF|l>$&ng><>rd|d(&y#Wu!7w5b)8|5O!uB~Btj zU;gW-j72-hm?$4kxmw!M!(8mgh?#0ch>x{ImOs{o7{_)@R2|tiOUIjeTKINnw zudd8?y3J}yH2?`1g`@oJb1Be6x`bm9K#l&bb+HnG#8c!SurQh@|}SVN<~ol9D@6}IKCm-8Q6gTHL@x$RTU>HFAaN&T&8`Ii7h zOMeTwbiTyv&krLZxb@ZM5{^A;wuzl^_<1(SVD=l|qO2d$FAo|g|GSfB5(8vw3|!4B z74&2<%gh*Y%ll){fQszU(F>Z6dyJAaAh<=rG>TbtK*TuL?>nHGo%c(MZiQl4VSd9o zLjJW>meozlwm)R-Z9pSV&@_R~pPBZz2dFXM&#wVV^@kGLTLy)^6MDGbOpNK`Zq0j; z0@7#T-C#GfBx`8f)8%(5evbp;{^o3DS0EQ<2ypplDfgq&&7J^S%9{_Y|~^6uRm6IGTMX`j=Zw6>WC zNzGZbBm2TL&0^nBYNT1@c*$O+Y8pp;#uPw#N)8gfE5Azx0wkZ)-)VzeADn;$iqt#u zc6BEpE~H}=if>N0rKh#G^P{vvn+48)#nJa-Lh9udZ|!50p>nFLd>TuBQYergQNd)@ z&o&9Ln6fvhs_XQ|%T9s1<-6B#mw$Sv;h$sufXCG_G_HA(#5XtReWU?VI2c_IK=^g2 ztK8INP$2Z~6mWw}$p&Q_P>zQLFM`1x50{l61SApd4mx*a3Aj4+kGwT?_7+X5xN0r@ z!tdOWz>D?Ap1)h=?}x;b*M?DGmI?{in+HLIr0mR?a$Fn01w!AFu13s+k`w@s6+Xp< z@M`ie_Xf*g>3yW?`znUT2Ii)}34yUBK&}IIz~kMH@QZuS$!J+S>@(Km*7HRHlh%|% zc5HQ7+h|aRxtf`g)LuYZ1oO*U>*~WM?FU2?eK{bT8Uh@@ltIfCx4ClOACguaf_;09bH4l>AsrH8&26Qe+$M(2-fx z>6;X+l@RY7wgqpcfF;-oi<4r2y~=mA>)S20qOQ2)(9u?zYu>zj|BhRAHDOSWNYbm` zz+=_RqLXor3DO8n^4a(9@V_7Y*LhrJ)NM@Zia=2nK|p&LnvSU*iP6DYYg%ELVwSv7 zY-%E#L*eTM_|augt?xW}i#iVMSDNda92(RkK+a960*LvSwsYN!vXz2dwMv7q-Vfe~ z)qS#N-_4SPL6NEBhz=D^+h;wRE4eNInr+@@?{+samGB*g4jgyFZ`6%+=5t(S^F*Cl zcWfjnGoa)rFVe8sQj@DeNC_QUonRaK%WmoFmw?!@4S7ify3z3F^a|s>VkTF46>!M$ z)ODSDyMXA+TE;hfHFQpk)=(H=OwRYJm*1Y%V$hYMzq`rIG#p8k>bzBW$|_5D@YE+; zczrIPpk#u1lUTt9TSo+yy)2i}w<`udNA+^+90?$8*}5^j@OD1OPMtz+U21Y1c0LZ0 z;nowr<6lFEs@m7UZfRZBRz$qt&U8RFM`|?-!_Nnj0UYoXfg3U(hFoMpjIJWg#+-cm zFjwb|fM?Zx*s+RU{g?!ru;LOC=dk%XUReCxEY<1c2%-50@ByGjnD4iZ8WS8VKzxpW zesPIzn595AUNU3WIvV(0etcWGdriB(|1Nm8MX&Ez%r10IjR4%l7ZEmFiexZTA?p zIIWKJMq#*yVTJ}(x4&BGsf!gqHY>tbNz;xI{iqk85z;7W|9J(-^Ph{BpOUJHbF~B|I)`CEHX2SPyHml=sV?M$$Xl(vt-cP(*>Wh~SwF(%8kkvRi zt@GO-YzVKh11CE>yQ7N<`=%O6z@1ZPpjdeK@C7D2gJiSo-bQ9cZoq>UGy%ouSG<=v z6*TG0(8b1tW$11b=DUWzSFe7zJ|B<>mh7?Sr!kxY;|wGme6eCV6r@yM^!sxnfG#hV zv+nq28c5=^?KlYNPhABP{I=}0t2?>Sg+EXE2|N7?)_P!)G_8k3t*-(=#PrICHOP7a zKrK!(!)$L_q*CuU#~!Q3iai_zzN=i7v38Xj=8kh@;rlM#k2)5=8iYkvmeRF&HN|E& zh|@{p*1VSGLkf9g#Ef^yeIO`5^v0=Wu0-nt6QCMf7#rq|je8ond?s>$Q|CUW7W_r< zGZWxk=SGIy18D`X*-c(CTfv*1o((ZKo(A2X0_mBH0!Cxp(-J*l&h9If9S`)9w)Lrh z5PZE1)3K9fEWyoqOE) zJa4Rb@3p?;*!#!#)9GZ6x_VsaIeI@cX0n#uWN7_^nedV=1gElSZ1)T=|c_v2QV z1Lu^yHLDz9KbJYJ=+1)zwj+Gaa7k#?huBdXm~2Vj@+t;(57bt)x2?9@xl2>cy7r>sm5TjluihAC=Ydc?Oz9m&#FjjZDc@DMghu{pz!wI%nb&vpP>yZ!}0O zg!xF@vW!x8mJxp&qqGU-+v~ztw!sg_9kM-b8#AWTjz>su9IV&&usAm8b+LC}I1Qon zfYsT!w+9z)cg>YfND&?mm*9QycBo%E_^xcc zDYvy`IuQ4Me1;X_>74$~Ga7itg$vlkV}Vz4WIHaZXSYnlG&Q>F1HGh z7oB>=-z{3i*w%8{8y03|25lyWuS6cOIM#2xTq>BGCGN8s!*X{8*jT^Gq|I>t5aoFU84yL_>t*QRB(rxn9bYGM=7# zXPo|r`XQ4Kz0bW;B{bTdCH2b9EV+E8W5);%&y-Q3M>Gg?r1{s!)SVUS)S8n@-D4go+Js?s)TgbXrnmp66-> zJ6VYv3yRmB%w(lv?3E%G20gJg)Q0o;oQT=;9k+Ngy*$AP58rBIDwkvJSG}xeyO?G5 zuIj%7=Cghs^#rsme~O_Uwf+Ih^q&e|{QRK3tp$HR@awyWy>vIV{cX3c)g9c|N9co{ z-p1<{!Fic$ROgYcOA-0^Rd+CDu&6SL&yDjE1*^3@sE!>_{I4Qwc@(AtEBws^d)8U3 zQ}@C?B)CgFRlpcFWtb%xP;4LOZ55HsS)GQB<~bMNF_gHmQy02y`Un*=V>snubDgBAMtHgs$7^UW*^CHVtQ5bW{wHe*#{hFEk38_Cu|Q z-(lJGcB&85f1f)vjQq+tDi-wk6QhipaZKkg5u4;t6RV zW1!@ORI(yljXJZfk-v3W+Xgp}J-VV~w!7|APWx5`$6V&Xj)DeV`egn(z6{R#zR`gz zCM*Y5M4dEi!sKG25mS&Yn0D_T0Da@Yz3d|20u45_fM>G zKVcia7nnNv%dFK4(|?x8sv@%N)=%{8P%2cFEUW;|&DzKvgV6*V@0p|0F;@zkV#k+$ zgn6avo)IE~v$nPakB<4iZDifaT)6%A zUeRzHiRso|sQ(Z?o^jf)T2ltMAUL>hwfBuWF^W3t-A7dA65jGueL;VTRuo_m7?Lzx zCQb&BpvcMBZ@iVLkmCYyMC(h$!jdvleY_a9c%qbEFY~j`Rh&CcuU71(?tAOWMw+zx z_uXTrQTLHrYkcPOjG})*5WB~vm&dVJ?cq5n+UTEUyizj#vEP-FFHQ_ zKA^%OZJ7ptnheBHbA!eD-2i6$Fipct>^QQaR@YvJ`Y1G>vhX;2&M`hb&sQ6!F(Bv6 z>q2{`U>C|JYJ@w%@gw0n)hsW(|Xpg+&x4=t4J??pM}o~b53ev zeb6?kNoZ2dR@iC=mX6}nRI20eUOdNOISqA^t*E@x=Ie%qF><|n6w<6(raX$HhlyUE zVLfR6Ip++xChz$Q%)_33Zfz`Q^gh|(NJZ}Vak3%FRKXR~WnA* z6vkihkWo;wtW%d?Z+E7_$#TIY@gzae6fX9qlIk4RC49VH$lX2g*vxlQz>#93 z8k>$Q%O%Jcs(Xw$b}lGb#C${&AXar#9$xGO=gu>2Q-N@0}7fX!*3%vnFLdB}gGf(7}ROK(>gzM{liP=d;< z-xE^RillJ4jo;+plF^7$@0Jj;$+6~1Fip^{ta37{7d_3Vg)f&QhM%OGGXYW>`6ny@-#XNN^#IE17eYu#fogdsvvj#=scz?MHUsJj>vu{PQsU5YqMjdsU+;HP?b4 zkX^@ZHuF@I4#oU6bK7{Pw3RZyGL*i@@vXSW<;KXQ4*oAJXhzQ#aZzIlethh=7wf8OqedVm|DMHeI^A?ewd3y#C!X(2 zS?^B5bSLIkg5XlclhxB4=}ha9%e}VMMz@ygvw~r=IQG+fI7oEk4OR3rc_;VI#Pwdx zALVpnA>dxmq!1@Z=3}KE`_c~F3iQc9^E(L;JgL``hsmUqMxU?t^Gcs3;uzC22b$~^ zD&kRp@?+FXl`KHFVZur8`zgpeI8v9l<3|CXX@l7Z)u;vVzdCaz?lH`0oV``6>p+2w z=!H^cu{eWGV6NTcvXiF88htD{!1wK!M)ovCTZ84eg^O>*&-<;=fRSnTf%&aym~G8G z#U>#%JTRv9g62yoT8s}ptz)sil*fz|q<_U0PC%>AoM$h{hpSzcQb@4>V4 z!h`iG_EH21K3UXEVoc+dl=1a}Pe81y>#Ur_4$0R7k~~w1+>XLn0+;gA#vAzVREaG5 z|0S8TX=7Gb*w!pH8NK2-k8N*XxoSNpcN*YI#Kw^ZjWF|uEBg}gX!Y3pty zsCxGX8@60le18ELVRNb%WOvN58ZRlZh!n?giO!h4IS6v5RL74@R)RNG%4;~*zWTqo z4D>2D&AB&=fl)UF8DFbq%O5|NxUC4z3Dp_o(Xx|ic3fT-mfOZn+SfTx+ivDWmWMNVnCZVQ|Hdz_D#i8uuf)t`bOhK|B4f`Nqp=cdxp+RAQ%Mo1tz;*P0+Ts3q9a+TORp>K%FzSDHw+!y_ znLeprzBWX$UDi)*t8w$5qeDMO-)<)s|L5HV8}&R-`SLNI1e3M%ZM}s|>@=a5ff}y+I9Szj(jf7tyCPbdwwZQG=U) zEa+;K$tkwSr@@v?f20U=5n?G@+g!y_lxw5#&qL3f$a&O{IE~eo^5whYIA@x4Mq@c{ zBApTE&eiubSc^k4MTxN%9{T-PyNtiG2Wfo_LLR5j^+u=*xN!zwK5P1#k@C*IV=+JW zMw69;RGLN)1l`_OH;SoGEX$U`q_PInadPJx^03lc zHYe1(@L0|qiR)~(_E@9R`H6Wo8LD`hH<8~gjK(j>&EMV~P}tUu7m?((36+MbcNxaf zlU!w9oeMG%59<^f~A&gE@?C^Fa6R^}@{Pt&f8O z$Z`i5ZI(&;t{<){#v5KJ>cVPpQ?FI#jXej&t}hUmLD%;;mTqm zNvWw86ik}da%t}^=j?kaU zl{@uvd=$j;URu@dyzQp1nzetIb@ya!+eaNwQa=s*+t}v2BZNIYVyBaKbo_|lU?$V> z_Lem|iXXjRxcRkrtuCE5Wz!ukR1CWaB^|v3#@8`A00gySOOqbk(TB$@NnrcRbvV&@p@ znZI-fr)v9x%|&hM%85YfDXpZ;<+QE&iMd2pCv@HHKkcwqcA_pzM%p9v{MTr{43a zRmyG98H$ze>a9**LqvO(j~au#&GDKzR6p zy@y>#Yrk0LvnVfW^)u}r6@#Lpt?nH@czZup9aa;7YAMKUTRh|R9Xh&wu6*8*8>rG0 z-_A-EIk?Lc1wR9Y$%KyKcB406%HMkx=jk6W6CdjUCo9P$u%U@`Z+N}ZHYw(7s!P(h zJT_qYSZzw}o-Jst)&Ts;WziBEHr;{K&JoVXU+;8;AOuJvGAh3q6AtSrW~$O^QdyQY zL^kxxtr^Or*6#=6eMQCr%~H@5|G7bT9;n4Y#E2!HF&f(E}NGcdIKH* zey($^H|E%rA>kDRx|DN;Hg=J`zk{BPZ?M1m6LefRHj8znw(6v%f|NotPEf_paFCQ+ z+Zo^bW!_Jp7I-FW_NLRkJvk`Y=i~i*1vDKl6f+V{f(yP#=)~2Mpk+>23pV<4@#o2M zv_0)RlT6lDZG;J=U+==%D?Ot>q9SL-bnBA4cn7FA69M=Wm7?tl*Ke61RZQ#|5~<#~cNr6j>dyl$Q#sfCpZb28Xnz!>iiLEJ#;nMS z+wO-Uajgz>NiwQG-?U6V3roq@k^Arn`LXh})U3nthi?``VM7ONek`Nd3-{2nM}G(n zN-~{jaQDt1IFO8-{Vl}t#ungi+QVa`?ZT>po&|?O8`Kg%tf8Nu$rpVnQ{?}ax7nT%U)Tliwx~o zJ7(*RAQncNjy4Op_qwr4(V67I1rU-Q&>Nm7ES}zdl?ylh7X5@wuNaPwAbzeXz}h*O z?aidv%4WV)QX+m5ihp<~e`M3!gH$weppirCau|PwbX$u+;qdr-$q`E^tAX%yeK#DF z!?STm*oBE}s~};je<0HUayL(M(}Ig?kJ{S&CNe=6?ubvXOT^{EjJA(ETyl24rSU-D zy?!Yj9qfi~XJ`H~J-w!ghkpDO>TtI0-9CiDHjVG7ZB7Y+Ds|C8Y{;Bfx5*}m+&Cr# zBn#@UG^VvMHo|l11N}AP=TR@r7H0%Ywu8K1TqJlyOba_NhR@gY>wAccc)LRH1bn7; zRFnCpkol624VPfOQ(WFh_&Gd)TrhKEw-ZOKdd{EpU0k|IWfNoAJh%qVbGQgp;r9-W z*ohboz!v<}Ji96+2v5!~Tz5W>$0W_Y3kMt#>C;(_xJGYYpRbudmx>C=Y+?su<5(X; zxJJqNP&rm|4cy5nZaKT=^85551W1oJL{>K1kjZ{`StGVQUE}sRZkcyG-a`wG^v14w z;_}S)teEHNA*E;xic<7WO-e9aN2%pEd!T=&?oJ;U3vI-SwPo7=E0%g4;+0A7 zh74z~uP@%NjdC=>Clle~?<=ASn>OafQtf+WRY+I6EJICzFG9U+J99 z00^ROCuZg8XtjM!^`;p~$uFD>_WM4@k_9`NQOdK7WS6)ZBr9J{9gp#yf>b8dBL^FM z>9GZeJ>l1SM9xp~%nOUC{fP1m9O1fL)B`#}D!I1Kp6og!^RplQ!~XIJ+0JLp|ViflEnF-%xncP0BM_94@d{OFy_m9<(o%<)j@dgl3$ z>dr`f)6v_!(q(=&$d$i*O;5+9C>#P@29R2x={8#k`wBNVoQ_v9o?P`lFGQW<;rx#pAi)@JZle_kiw2|1gY} z1GEKMa(-4zmgnrtxh$d!^!MSl)sFWI6>@Me{MukUgk|odCOF| zIdDH0l5ipBBQ#!|2b2wEpKCLplDy-apLH8rA#;&sGBC9^RH-8!`lKi8U8{H5eVKdt zT9H-v8pJyHRGo1~94$OIcqVvsZY#m8?8J0~m?C~Qd6~IM1|7u}lPEWExIi#2CCzpG zB_@W+g7ABRnpSD>M7#Vao0Qq`z;EpDMAASCx-Gwxs`=_82Lc$etoIB5TM3di(Z%O+ z-{z<&MAGjT1Vrz>gJnNO$=_eZUt*1D9M1#! zpHX%dB`=!+4ME7AHs6o=D-xV-J!1>`X->~%Z(}obAmBW;%Er#Gw3-0G>vS)S(3Ld- z^@l<<0!(P0c^(;_3xplFw92I#Ox~CE_{F_cL zxy$*KSukYh^FSz$Fr1+e6O}{!RT~~^C~+!w=3zgX4^ z^lDEu?k9GU;Om*yxs8Ng-MrirS!H|g@z5w=&K(+p{uo~Jfw=%I8KDq~YD#S1^QRqU z0N(D1vxJANY~!{3b+60YbOWT1yxU|QCAIn?N|I^K&{a8`nA|+lwumNuK_{Lx=GP`v z$)K793CV)#uK{Ezfn`pp>Y(6FE|6yySb>Y}o%WPINAm8DXON0W7SeHr|01~T;#dfx zL4LTc?6NiSjZ(+|^0uSZZ*Wy(_!{GqCO%p)zUn%ku*wkZ!NlX(LaD{bviD81bu#;U&E0>$1# z(jKmRiN={;)o~|mM&uBboS@>%>#I9Wij5kf9bV*iuUiDYi%-53RRnpSloo~yy_NLz zb-dLfkxAx{-y^lQ+fimD%hGf2=@BN82CQ_1kA|EgU<6?3Sz+mYaE^WB{BP#QiX;?L zOD+BUb}hKkUlZk?3J4Hm+cx!JEk4dLyhb{IlXLOy%F;91q8ZT{Xh#>&jwvH0#@dpA zw#b0@dSoHzU|Q+s+102_Wpbr|Wvrv<*$Dz&mtOHk_fM1PrdRK)@B-Eoa~nsF*89|S zgyy6{k@O)asGTMA``R3+%Xvt`wkJ)Q@KQaZ5qS2q(vJ81n9dW2q>pDcLL^FEs~xF4 z2kU!a0S{)=2nH977%|m(Sp$m|F&(jjxwoh>B831kNh?c9g(S1+*74HMVPKXsg&d?s z^=U>#aPcx2XYs%5rxx~;ar&ye?if)z8XX1C{q~l8)V1#Cjr}fdlPbVbEX3=3r4B!pu45m8{pku3hv+1Cr8i74fI_0pr}dR0Q(3NInj3oxNK$E{?zTBjK;> zXZ(1E5hbd%^+7Lb)dz)un5hzvsd7#lf)(OADEZvkoj9jBx{X>_&;C%_pK844W#^Md z-EcU1cK>1!=PSy-_2hcWKwJHJ=~l6Kc+32o=71wi?cI)D9Qd>(HeU zGX@!lq{rUaC$BW*XTEu}&sD08Wdha(n;HU{@)SkkVuEwrwc=YTrG3BHI-#~9@t3uxtZJZ&L-_YZ4bo6?Jt?x+D~{f} zpVTY49-=^Q}L5>otLKV%K<9NAILVv?G5z#!<-sax76Gr*dOzr1)v z69p|BwhXTCt5*=x^7u45u>stg$}_+uFDQ|IIEW24WH@YXH?@zsk=q!Aagj~Fzgf0bkFs@0mk?VKU>cBItu;*6@I`=wt4XrF!BsJ4C#ufxe& zFYF@cIGd4|Tcd4j^_=SkgdDsDv2d!oD;i$`ubQluhczIWEbjsrCa4`Y=30g9d`>Mf zb*?9a_pQR}XM??{`O~W%?i%M2hOQ5V;TjI3i@GhQn)R97t4~Hq{Zhw;t&#t2tCU_g z5YMToDEX^Lu0ur3ovM{!%WYVlvoe#CmSTuW(XGZ6HmJi-L1^Tl$L6KC*ku(RB0S$gRkBqN+zy z1DKRts4~)Ck5uX+jPA+4f%Ak$Ug(iWn;Ws_$VJzh`JRy-oEI|8tA6);mCA;SY{_-L z{(8Mh+N(PIqDE(0PigUuYK+bu5Uc)@m^MNd<_Zwx()A2=ceJ45ITwDd-3_M?eeFGv z$Ze9SGUdu;M`mGA9Y$glifh>oYb9JpYt&Xczmu;o$3HTFN$~qcF)Op9t-i=qpVgU+ zEGE&O1Xb_!v3Ns@S(-_p6qlqlNpzN2go?2(vJKN!x1-haou@_!x=G8zWW5|q5?+DL zd>jW)HUqlk)ndIT`9i+a(SEqpbP%78n%B#_N_StUH|O>vrcu}<3z3@kbIib1>4sTn z4=qRK4_&pDExK9NzCgzk8<1kNReL(&eOx0VWjJZAd~vZ^ll!}LaqLW6iHbVn!A9U1 zO~jV~J-oy6w_Y`z*q_{H=H=kUvaN6bMpZZ;dIk4?*W!rTJbQE8@s+oUK|Wpkh;>z| zIGG+RcEqD;&Dq=oUZDvfwMj3&$mB6<20T zP2T$}&9;yKV4p8vb9xN8b8M|W$R4amu3%g!p8+l1TPDYgugS140mOk$ z+1J)dKn?s_d=QRP(MREny-t4(l!<43$wC+QOnhGnMx}SfN79amV9(nV2{Hj-+7#uCzCNH^)Kt;D#ea8y_70 ztG!+C`c*E=0RQCm{>i$|WT^GW^QSAGO$Rm0)ZD$Ao<-VXO>&|o=!hiT8_kj~& z|G9WfL~^RScmvTW;DG)fYL55{E4Am64HwxBqF^IBkbxNWzQ}+re(4_jFJP(v3OyR? zvvmTn3j;uS?Q_xWGqys+(xS<}^)zcO1saf>4nl76{mUp0$jTsY^gQae0 z4yB>-Eo1(h@&7=dfU1nuMDG6qR#(#iFvAQ(yGy;etc&kgrW+7dAUFcJ`0BsTM0{>~ z6#!nu&R)f`3e00_CmEKU{4$vMz%4Zh?d%ZUjeh)pK%GIzB$sv#e8V}6##G_`Kj36% z$QmfV`Ij?>#MSX%4^GoA4J`Xu)GJ=kN%-Fn?nd4y{{Jj*QCBvx2b}QSM{fgYJ zBKBtrac!qRGdZB{S|ft*O8gnDH6u?dC(YC3^B-p*UaqY7fZoD;pj);9>Itvh`M9~t zZ23qr9RInQ6nV|sXQq41XE`?{xm%EZu~33gc{zYYH=NfGxsvw30P{lr0_N4=t4K9L z+!AemNVdK~k`mrmrxvH6q`m@*9d8QN^7{bk1TE-BP};^dhj)NwGgEmT#IS+{8R9xM zgVW|v`#p!&=yYRL{<%hASH7k~GA9R~MIf?9L^GB9hiTc`AbDoRiD(rHp%hriJ>tYX z4xTpvCZXF)Yk&)jLB#Xuz+-V9$UZLGk6F0ffNas+Cnk+1B|?F1Ej2<$o;W8+rZGbQ z^Y;N`LCzJw{Mj*}7@5@2?}=ydX9qE$>wU6S?0k#o6z~_BLAaOhJGzeqo!5*m zH+;Z_PFY6G$F+CiI$U@5O(ump(m~&^8Q`1KEYDLOKP^)pW&KH$a_5T496WH zqe`iZkM|xtAEKVL%+DwhFB~s#+z}+oAeI3VR`uzQt_=-@ znR)b9SexoY#|8?S4K+kBrZB#D+a3Igu8qem8QjV9aDyHyB9s6^u*$_?vX6BjT6qP0 z$x-(c*O-VthrMWYu>RK$dnRv=iX~+hOQj)g=zaLNHWYvII|^?NWrbD0m`*Q%3m%l< zth|19r|;cBQB+O>Mx)hpwlkw-nx3u4p(*$bkKO>voe7e+;u_)bsTMC!7FUyb)zGk~ z-q$^3_Mbox3q)>OA(?k41kmW9RIxic;CDS5Am}bp%N@3U>rb^WkA3EU5G_* z{l^LDr3iEZyk!Y!QOVk;LsB@}F~5P*bC^8}QW;XXihR#xB!mW#ojQt=S)}S31H@%`KoysY(cf^ zL$@I1Yg>gwFi6k+>(*kXA2=H6=dFmvsP=S^ASa}9675LV)&n6<2ud zPBvOQ&bj+eMJKlF(R*`cnNL=UJ#S}2rUgUBI| zFPy174o9_Cz%^LyxTB#ySjOVrS6hecf~*NXo27MI!#*|$x7+|-(PzMzY#yWv%QXNv zgX7PN9L*!LPY=(Q=jpPLf7yldW*lD7<#-I+m(*LfzFv_zAANdQtE$!y%>9pl91!7y zA@6;nnA?AaK`qzo(j=TdNvwS(8v?+aYgk0!FgzB=a#U%%JU^~e9JENzzVNL8N3mtb z_i}Hck_j04dxH1`=iy%kmLrZ@(6a#wFH0o(v)3AE{CflN19Ry00^bavWt=BS!VW(| z%%X3-jGJQ-xGGW+E3+7E2jg>f$!K8#kq)*%8Ex>5Szy@~ z87_&lsLtihN0s?qo-``3L%hX#W9avB-@!>Ahl<7n=9iejdGXjIMQb#%IPg>x;p5o9 z4Y*be(l(I~t~UISYdr9+5{vJTx>Drqd$+gL^Q|w4B1xn8TuW*z<1jRD|7lTFN-A7Z zKZamHM*`LikU1+r>Il-GMyGr*kfRw<4D<=X1uK>fm3%Hha|RJs%iwt<$xm^V(qBA! z4JINW{}0KN!3MG?V=D1?G;l*W6juZTk$>*5&kk0MaMMuSUjQ@2pe%A2ukYc#mK~C~ zUF;6_`!l!+F^f~;i_y^3&SRy4Sip_kg~$`LpFI$q0oKyoB89JS_$1A~MOzj$peujq zwLJR2oFCx5W3?h)2K4j^nEO9I$KCGVq_(kZJ{+sus_akvRd9C!Pm$I57|u4!X$X9n zeP(Bk>>bI>mrH2w0C;QWcl*v8peMaw-d&b;uq#K}2Lux#!hnxOr4#zzZyL|M6C4$B zRQ&W6v|HuDWm650gh9q&Li2Y3^AExKD>3^O7sg$|0Yb)i!`x8^XzJWup_c$R8VAr8 z;K`lsdjP@VZ0Y(loTG*5c;2Vgd3{(WN?-|gj&`l~{!t7|dDAdEbMJg4YvAK_0aYOP zn2jQBiMn>o;5>yZO=p2yq5&>7|MD@-filwtj2;BkD;M?C;nPUBMoPMsWLmNUz68H=eI z)9IJJ0^z`_gZ0)>`VP4D$@9|!2hO;VygTu)m0`Yl(domBT)Q@<@gha z;|RJ{>%A~-y|ElR4Hd<~WAAW+Nlc8~0r+Xf@cJytd1FLA>cJ{;iskzhRyK0FKnEU0P zQF}|iGaXzOm}6p^9o&Cts`QXJo||#2WovcMUKR9m+2XToC?63sLIVP3{mOTY*fJg2Dj7mJ{W1y9h2E?#w z2Y@U#HS6>d#!dBR?GFJ7H98oGDSnOd9wcA6e|SN}3Bcj0GJR84NBl$YLHTMjHU4&SRfzYQAvD(+yqAd8I{)IxtXXS2$H$TYZMHmnMJShR<0jHPR z|6Lycf1gLslZZ&%+3fv)l@Q)AQ#e8X#Nh3pef~rA0*p=AzjC4ci6}kRPN_Zzg@Csc zN?15=3MhWEj6;3F0#=hTs_IVwB^^j>CZS*ZlOFwB+2t>~ySO5Pzq}8~Qy*r<+!T!1 z{v~0Dj<@SSYd%k4Uq04y-IQFY{rI53Wty%Yo9I&k_7Dj@RNncwIN+}W5U`Fj;1P^H z9L7(07$6z_muW*(45%%p@gdM7Z_rHALG9uNrMFK^3>BdPR2=dCpSc5GN|pzSn1u~D zwNAqu>G=%U>=dHtrL64!qA3Ceqbb-Z-EPn<3^S$YIdU)#6}zs@hai0vpPw1vfjMUo z?DT=YUTVM||G}ndzsKzO1Qo6NE+l|?H^e`*B*46Ub}(fsm4Hl=NA`ERW(0zY15y04N<0!wNJ2tJa)s^*(l z&cWjnK1p(v?$UEU*Ip@rxA$QEyByz)gVg$1sczOapyVF?DzH6OQ_R|OvQ!DW5YNuK zW$ya}I!pgozdkeaZGi_BroTSLdgaUiklOgBbo!1iryzH&ToX*X2Jgr3(m!>*|2pZv zWcm-yVs4Si&HW&E`We2!uTQkw+$RtvwWfjlN+q&{gB~ChDG_i7=|tk}?<Ag}>$c zyHiB5&S0!Kwz`?&=7C+v#euzk{^v}kz?qUnVdnAPSOkkdX9_XE2h&PfVnJuBHcEtn z_OacwHNoilIG! z6i$FUevBFt+Vuarz&{EXPZxetbL{=Gxv`5U52I50({&>adpJE;>gYeT!OsL2_5GWqt6RL8P zVrIo)8X?Ev4t)xfc&OrlG{S@%mKs=4QYkDh4YO9(`qK2Xarem61yF>H&d#6r?%BD` z7L_U29Hw36HCh8gRNli^AZqZKAI?*BncNe3|G;cC2?R=OUciWan(jqG)`M~Kr)h9; zlc{z@-P-SuqOCEN@@<$D{z{w@$z&|&dt%lbqf<$ z$-a@|B|0A${4t!Ly!e{_peJ~AHVZ1rv@BQKErJ&~);)@Lf29jJzPl22S{?W(6JR@C zdwXN&8+f?l=k_xsM0Vulv@b>fK2Y0?4W&iIfNXpPt@uUd@cxy~Zo3(Q z=AmKm`U@)v!kY$?xuifXw|``_la9%Id@ zF7bLu#%~u7o;}zwef~1vVp~7Pr?@-8eDE#XJ!aCDezhc3`4K3)>FC`&Jf>GnT$70% zk(4Xdt&1v=vsIVZsBsuxXb+%&C2YVbH3*d3qrVEtPXOB8zW;G|_q^;}Qw^)#znAwX zrA!HaWu3pW?tkNfj$gl7)_9Qfv-SFN_qX>Ra6Ic?i?6rc&mWPEypwG`>E=MJN&W`F z{N#bfU7MLDIyDO8lb~L$M(E*FU-XW>)nT#2zfTQ@kc2q!#a#d2kJQMHFPcZzoK|51 zm)d*MF*nf$b|T8(IKZLfCAUV#@|2aC?E3Riza%n)^4k*p$Q;?oJ{D2isr${lU>;wl zwLGOxo_oRP4oR9oB3{G4+3NEvE^|$=h)8y-)yF4GGX}Jvd=DrpDr)-W!Y>RA=-DwS zCzdv{s1y%^w}dj<j{Ks}&BP!a4HOUBEh()t}+?^?~1+{`MfCWhyv^Fk`%FfCs3w4vRwY{zRjj z{TJ865QUZ~@(0`#RTj~6EykTF&7{pmp0gia9+(x!m4JM(Z}KY^jvCr>ZS!Hz;y83=dVqIS)6-l{L%s zm=|ZWhrt?}vtm7b)<90m4$A2x;q<)LV~>(>uC>(Ab`=s5*>n}IuP*a|#-54f_9tbx#q?)S}U0|h;It2WKDi}7Q$YR_W`KdsQC%D8XDDn(jG<5N1O0FY|v|K zcyl86Ldah{RPwy=^Hl^&Ud4VRxr9J+4|8js(ZPB264F-T(r=-|Tem!(=nCF`m6@i$ zN+cin{&)I&ElTu)gr&~j;ltVNJjyu`M##{s3jxM+mDm?wQhCqVM>I=yNYZ9nyc!G6 zfJc7nMO3(tN+90$&Ue>u!bD543_T}y{lqv0$R;9WS(GwZ@zJc3>xhJ_=lCk{JKhkb6i)uFXok46A6yd9(ls3eZ_}w zSCWTfPzXh!C%hwvM$7~wPLlq|hzyeYqMi1exqDs}wuf;Z&}`@+*_Olj=U2f>mmmw~ z2iDQW_Qwf5ynBDAD)IT-?zmd)^NIIc^+Cg$50LcY4_(MkwQ+K=CRM=&-oM^0P zNILxZgmBa#5w($t37r=FKNlo3A0Qn@L_mgz>H^tky)icbnD8G%!=AZV=knx4bfWf5 zUIhBTSjeX51pRYuz_dC-rWM+Zr8|G7O9zf}QWiRkTZrz!zAZH&m9?Qn-1B`L2R`$a z0HfU^*y;J_)aQS$cF{>1>K>)&okXrWMjNVbpNdY%xCD1VV=6#nB8C0;V?tXrgQTV4 z*@ENB?6t(7Ks3?dln)qGSb$yF?W4k!^A$&g1UDcU`XTUkPU~y_Wg^(f9V5H4Qrt^^ z*>CJNY|6eT8gHpuWvTm}sz#;PKFNv2HeP3R$DSzPBSVyce}Yo0DvQCy`O6)*;c81Y z?OL0Zs;}k1^|!d^6gMVNE}omAceb;=>cHI zAiZOfZ98t5H1m#;e$HiqZiai}<5%vrZT$`}6Ji~*2X|0shIrZOl(2aA1y5@BW{V^5 zuo#hk)^X{JzF%wK-H0HMssSB4C)h{m^s|V8WxP2=_n$nctCOsW>I0wUCQZ><4Vs=p zmoQ4H_hSHGcptbKdyk)!Crnzt;Q_LqyrLon@H!R~lxn5F=ws&PgkA#NnyOKRE+fhK zY~Da|bSl3+WYFVlZKucLrz_3ES@l1vCvq5;)nzyj%Ma%(Q(2CcXe*^m`IJ2UAgMkL z-D5*cbHh1uIFn88Rb|UfzrV(To4p2C$H`WoR&PRI527N4Gw;hyLn!YO%f~c?GikIP zi|bjn+FW7Nrln4nhM&k>^zBUUzW)gptM&I{Lk(A}0`a>4jux6}tBDU35)Qp8SXLw( z^DyF5VH&NC;1RYJ5a#t+Hi4A+1%RP`0?cCvt_kroKxQ739GaaStPj^%Yc+clxX%8Pe+MSiFnJ*1o=s1YT0_42a$>>4BgpkZV~>~j`(RW_xO*?PHc-wvLjBu)nma_#94oczI?WUR2=BTVmu9IEwSNGO;?D@E6X zpa@=4=W|$Wh}ChNjXb|svqs!T z0;8)HPhK_zy1i5=)-3%2(2B8~lguEl4A)yI0ECu!4EJT&G;`-Jwa;^+7NMJzy>BjP zyCWkQ9EHa|!O@>LWujIGGs}s@&>4jQ2cqH+z+N?OW<8#L)dK25rhW0uyt^uVu1uEK8voP547p3VP2u#F(~?}?)&1J} zQ}1(o8~A-wrK}M-o()QI4UGhYCZ?EIo2k^A`xZy!KxJ>!@LI6krRF3?K{MDbaCNH~2r9^&^sm6R?8EQ{g!A;fMQjR*6>229FuA$ z-OyY6#c=vQ>Y<8TPEMVoRvdpT(!H0=OaU$Yl5I}fJI8&*l1FTs1ppC`-k-rsWs4r?NP8$`;jc2tV6u9$Z@-)&lY?l*MOEquqzCKnyTOOqBb4aS&Hv zCzSt-x%ZB0s{0m26+wyxkfta_ilL*3^b(LFAVqqQguqE;3j^eRN7}PBpkMs>Fi@J`!JRR zVS|cEKx#oy`_!!ndtM!JE!Hz)P^4c8hIP=hoq;!nnos)`mEJ2#=R0@32)LE= z>#ISEqF;H0j;?EO-ec&%@*Uf#W;5=5(?wIshCD(5NOif7f-|KdVJJ=JFX1E+P5E`RZq8=nr-k1>nb)OlgYV!)xI-8vYAn+Wnk9gL`ri}j#2vpd1$kkE)vhj>8PnVNIgO+G6l-s(!}1AaZw=gS4D7!n3*Ev$A}0kfSFtw5oN+(c<2&% zK{83}J@9iM{{(*D2lUUc$91(qh@o9H;q_ieAwD)L}T2yD6tlox8bG0YVd@RMv&Qeg?j0>8B zT&Ur4n#Y&6u;KhFxK@vCyX-qv&>-py@=Ef4xdY{THQC1s%B*XOn^Q2%LVCap|MC#> zg_i&jKV=IvF_51<9`?WfYA=v1JEPu6zM9gs`b~C+b3{RVeEaI4ylo5LRXaFIvhcj~ z*Mzs>v{M4cs$0|i!4F-5(sf+=`LVab*L~L0Haz zi`(P?=`3IpZykR={N}U3hg~r`u!*znZ3O_l5Z?)#<-d^Rq1qE}bDV*W~Eqv~)&Ws>fIFGXM zNwBcT)7!=YDHJ9__D>j2e4X7f$aC<26~D|Q(E4~E)esBh_{;wr00IU5@!tS%&))~8 ztRH`Vr`CXBj236ET0{$ize{l-Jn*lEKJ-7tf&%K}DmWKL@I0K?5ctXIes9>mg<1_i-R6 z?5+P94^omWei5IPZ{}A|ZC_39g@uHJ7X|E~7w`RdFNTSAb%OL*Zb1+qe@oYOom-i2 zs-X;ofE{l~*z4|!_fQB11=dSYoL}~T9X&gOHhC5nWDdP2+tj(r0Jr=0SG~E_x=C_l zyUqAL$%x#=z(e^h8K{pVG4x9%*Lc?X0Q`x^e#wW_gKQ;;Ka9h*8|m^iuXepqt_&ds zxPE_KQ6s@6cN1=1>mdxHgV!|*MRXA1_Js`tMjGbai2^8rVskp)A0?q6%q7L3_}-)O z(cB2ywDD!c)v=b~$~*Jcv$VrNfT2jDY*6q1-Wx(d9Z&85^Ll8g`2M|YSaR1d+`Bw( z>L3ow=75^QMmf(AC~=3_Y`5?**Rk7sc>=JukKD$+zn?eC3GLY7d)$4D_I~E+AG%k8 zVXVA8W#Eo2p)S;oSXKi$1t$2Ld~hs~FX0R2Ehy^@F<-jOhaCMW-n$)y;m~XU>Uz;I zUgVYgpv2JYTe&swRSAW;c_7I*K*;5zV2?Tnpzn%^$3@C|&_6=vZyZB<2wd48?>t=w z86xqlW7&jzmER3Pk~0s)(!HQ(xlhSY;vU?`f#%TSW7tC(1cj;58#g}W7nTyoZ^YfA zdn4Jd)V-%)`7PR8)p^+sWn7ERtE4ZYd@PF&Xf|d`R?WYke#CkcBy{#d{Jg^_nJ-nq z>tl}By7Z_=gFG?op`HsAn0dfARiYVMBmN*uKpTJz0(PIT2co6JsR-9ALFl{=hIX4S z#(+WE@;@K(KiN8RAId;cGrbZ7XBWV4--93IP}ZrB#nWYMTPNBETN11EndevyT9K`e}+_E2dc}y5vg0bu$azC@DP}?u!r61k&2`Z3c zsq@bC7;CYEk{_Vg3GLBo&1b1Ho+2Qt7!UeS*ph8F`SEm=L>beDap=ZcJ`D;Izdf)_ z%>foyF)z!LaAJ_2xIuS-?>9)D@EYj=EDElpeM?QmTFeO5a~U`|{PY&jb*WS09L*bkv#PJw66>i4(ngfqCmkHP#1*skAiqF{F+D z?zdk;AeiTL^qy|j4c%bNezzsyOG=oo ziV7!MhaS>FKYSQY5tKr3_GZR-IPK3)p=77+BnL>;rut2I05&TDXtBoUNkPDdJqMti zQ|mkYhgU-3;Z#5s-zY%G+Mo&Fu7=NP>ty}Lk|Jm*cBSXphZL7!I_r(HG=k^$9wtr_NqKZuiO^8k0b8HafgOS zVBtI?K%;kR`Fel$8pii$bjZXaL za`Oid&6z((gO!I)xCcCeR@MP4zwF7A{IfFLhod(n9J+aW zW}EI-5appOfCqkObN61-pojQHKcXS%fhxlTPdW$v&_ivdQF0_mMf9GA@94J5^&a}e z0ASh?P_W0cZ$0Jp)t< zgg(`2PkJG(!pZt7m5z(7>W+UI#>MIJECUE;1Heu`Zb^^A1MR@+5^*-{_`N#F#8ViC zl4sAXRvW*|eDn01`+0}V2pSL(Z;Hcre8ex4{yrW6Jr2$+rj)W_CzyVyOC1Qm%mLUh zj1|zxSgsIB|3{#23P5{3FX+G+PCFHSWPr?=fvuML7@we!|1u!| z^DNhce~<{zQrlnhxLez)Jn{6HV@2M!()K0j)P-0PjbQiXbl#{-z#SYWZ_V{u;_6X7D(0=2QD3q8-!#pNl<}3WjLk3sPFT zW6yi}|B&cSBK-QX?DOPTIxV>Pk8!pcbp4V4TPdm{6PU4+h1#BQyE#F3R{~&h>cEG8 zC|J}0+{I`04N@I@kTXGA^tU3SIK(mc0mrbtzXbcsF_wF5`$;N7 z`d%o+T=R0qhbS0t0vNyB?0V!6#xGS;b~)*tpw>JrFqC>j`;9wzzL}f*=uzmIEN)6vR`}b3qG60pH7~f9$_H~e{`be#R{$J`Jq3~{yQ^xmSTr&V9RT#Cr|;~aoj#NM*Sx^I`sJ=tpP~cWYhR)Wh5qRvGE8Ub z`->P*mqXD;Me_R7m)Idw_fDu!8!XHl3>IFE5>{4+1RGKow)6sLn78_yqNHBSMhey| zFc)%woyyZv{un0&$p5qO)|Wq^!ySK{7V8t-)|$MEfC-(POJ{@uoX^exoTO^SxBTU* zHa4J#GFH>pd`N8baV2kK`G3>{UU|XD^bm6Pd6|TP=5n~k|7Vk-@Q8oC1%BIa5%5U} z^zYC5u0f!)0nkkk^Jh>TC+MH+dtA%e!PHAf1sU{ePz6GPbO~_E&$X9(9vbv+8x=V8 zRFU@!w+(j13E148qr3aRUqIWs%93ZdDlCn|f6i%S9R_IZcr#u=0f-Hp9hp6S@!*e* z4*T#Q*5y|2r~UjtAX7(MXbM6^;O^UoZs60OAN1?h8y&cmsY6I9xiXE<~UBO?{@q zrxcVcs*AqASVHHZ4P<}r=>=tqjRS^7x6h@QV~I{vt%<5X810yj2qy*W}Oid^@E? zr2^UWmFb%vl+UjL8%K!0ruYXhp>Vt3?!tib%@+R9P&-Hg1Q?&B0ImcnfOJ=yKajw) zt^t*6lzamBpz*CebtNR6GM`fHA(Xp9DEHKuVf?f74+8-95^DdxePr^}>I%f42IF%h zwBKzPgbmSkAax4rsZM>R7qB;TbH-sgil?nX>+yqfj>GcXe( zG`nDGkNMXps)pw~rDd+Z3EepFy@#sLk^4v%_HOrl{z~O0$7iXQZ!vI}UB7+0k3uF4?rig$wZHPDsUFYBJsFvGJX_8SPyUgx7oBm65h_<;FoRw#Qz;cuZrcsG1HzplwS4Xt9P zCOqUpT?z0n6?kI++8ZyEedw-eldwF+?S=?8A_6V%!Yke3zR_6YGbBOH$^1j3l3Bc$ z^2mfHcwX{?n*3j}*g52I0wKVZO5Q4np!)!J`-!!Q64gxe0?>~Zi#?%G8v5F51!cFd z|KY+pad9X0w{9N-^AQ33jjw%%qyfZPkpMvrzki^Z3*J}oq@|Bp{OSk1Wrt3O!XezS z@disG=_ zJF4+&hO)G=^0rWH^5=ZOHxecH)hUhxgvxr>>F4li|3Ok$Ey8=)md{v-x+v z3k)`s&wF1!c`mSRR-ii;1y!fM9r z!n^uJakNg~99-|!97*}MFh?U7`(^a`1&ZuNaAc7gymh1hC`Wi5!@s-rDzmLU6s84O z-$Ns^n(~N0z)riKzR}U&%me2o|7K;(uJCE93INSHnms93xD>iu5-t*%^Otdg#k$Xh zb>BvlGD+8h|IZN@>lbqMWVxK?|38RpWr6Wj^pvxcU>0Vj8sUBO*g>|R4@V)6AwbYw z->>X{9pp|5@)h{1hmJVNQ!^DB0#?{c2Rpz6#eaQET-;CY!G zO>5|puqL1h@%d+jXdz8R`6en%Y&?=$eFZ^IU<$JY&^*5~b3 zF?%044L-n()j0k41LYiaoOI0#3ydDkcz;Xd!Nm(6S2O8>+IL%5|A z!b;r>`2I4>&p--llo$YND$Ipg{-VYZGD0tI;c>McpTM-9%tq#X!a+z;WwQ?*grK<# zgz;m^?OFYLG<7QB`QKt=^6$1My6p|VcjPe zNZkWgcR+*U$`LvT34qSY52br*?*Q2CVWT!m>;(O0+8831+JS5<`}bq{z8y}dyPpAZ zJaQ|R0s$`!z$0Ho_vCN72N?>`qe+#`_3D6;26Yp7*8ZLa1sH>4S!6=qa8Bu8%#1N0 zFY~TecW4hUq#2CWS(kE!Nw5Q4Mv5luLpg*7OTf#2exR@$6{PeJEAUDGx)aX@@-=k8 zAnt!h;l;3MCb0e^x148x*B?AYeI#P^?%2cHbtxsWqFExR%&RWQ=AI8Qr`Wabdw_cm z-4*+5DOfN6hs`bhI%G>-%m6--ct3@*s60???fx`Zl%Nek+`+5;TTvC}2s2Q*0^Gl( zUs8s!4d(_XW!X|R;ctTAa{*+944XajNVm=do7*+=$k;a=AZWxW(gUisGg1P`|B02i z`Pb%({xyxoe34eoUp<>7iVYoLwhpeFD*|2f*TD-Tx z8|Pt8G!*bp08w-d)lK?~pydD1%MxWaGKaz*KzjL6$_&2M(WuA3|DjP-`7st)konI|uch?4H79js;yJH`u z7$krLL-_pzgv)(lZy;obN7Ef?8gT%Yj}4&HJoFbc^v5iIuv>+kR3U*stlYgu1=vzn zbtATHNfhY)wZJ1Mzd3_f)=wylQY@Q`8ekt9CHB7{F`W6=$2ep@bscxJ0u+oG{161}!Y;aTP9s~H^zm%>0Z!UbogtA@x<)ry$ z8gLJH%1fi@|A5sUAO|^=km4Q+viUk+Z>;*i>{7*$w{2<{7w=jc^DIi1w*2oWZ=LE+ zMOkHko?XnRgM8f<5OoukW+?=90Uluqn{v`$>OK+tm%5Q=cn@(ZEJWRfceWJj20_$q z3=slq<2Q9-zwMA|Ujj`SE1<4P=msZ+y2*gL^`-0#6dMV#!h|tI-B$Y@1kStTZM{aG z5gqdXlj!o=WtoZ$fs(==_jllC%rLkF7eIO;xu-^^4*|h%F>(gkOA6nCf3VabT3Eb0 zal^*HEl5?YXy8$V8~h6;vFZ}@=EqyxD@J;$yCq3S#~`;a7zXT`-+Cz}*3DA<4})A- zjD0-Oalo(InD2c^&-TRB#jmVxo80V^h6?NvUTZkdc;Sx>P^hF<6v z#*EBmv|l4*Ol^Io5>{nOSrvQr!YvHkYnwl&CR&CL^w1;pT&qq}DE(!)vaDc19j2z-H04@xIHfE2MC zNT0z&vM?N3Qr#npCKRw)et}QlebS`)5DbWIL99-;cF|@Q4Uux**v=H7OM+B93J4v< zxzebqf;Fy|sxd6edQ_8+1#j}?;@8WPi1f^8A<-nilP|vQUi|9Dt7`FK7;mymn`wpX z+HQzXOjs`3@tvAFj6z^+&}v4h>BV zoX+xS{0LyD*J*N-3j6B8EniP$le)gdxE$UHRu+tQZI?NKUtLQ)&X1TC9WkI zu1(B-oa(XJT~i=XUuwMQ9p44sBZy}X3_iMHy_wqC<&CJ>xtQFQcqJFUvAtb%ZJCX! zxprkShNaCdFlbY03;lV#l~h$}w_+SZT+5vGgaOol5Fv5;=8*}_t`_GX8vZV zap!GP`LfyAU|^6I*{rD584`XV)W0BW;is}yQTAG6mnX^8Kc+hXNHkC7O#+Ai~Ib@-S*YBu@NTL1GNG-XJ0l3kYFNU-0j-_TT zPVLVJcXnh75)H>zh_Et(Ng>9WDjORdf7_0?`X~2bRIQrpc3|^nk!>(`4DkD0#O}+{ zfFk1Jl{E|R7cJi1QZ?(DfXw)1u$oG~d-YR)fQ#GKo2$XonB?wF-}hrYq~!HWXZCjS z?mz6cQJyltTS$X;;Wmvz1FfTQL+gzo?-r3vM%+0C)4pagQujGjjK0MymzM3F@n*3S zF@!iu);RzArrTy?U{-#DA^6V|`fo5Z(}L`}U7ToGpruUwwX-y;ccGU2D#Q2l?8>Y% zd2)HSIV80MOM}IM&}nwq#j5hWId~D^KA(EqX7#qJr`aE$Fk^Po#u7I zjJD9jT3fZFGWxEwp0b3_^NX9leKPJg4f*zj5$^`qv_@A%9@*GaYA;S^qPQp8b{h!A zo3n4P)v4U{QiwgJBsb=GxTy{DKIkUwLu=)mpPbc|<)Z=WsoQJ+@Z_DyDiw+~;iKDg zM5|du!ow2{3I*Fiu3nQR%hLdT>uGP)CCDGHwh67RANg5IIY!+VHb7D7Av-2EAr**f z^PH#2lUD_AT&;vo4^x@Io7(S%Mjo*O8vt*laY@68f8C<6?i1dB?b<~%Xc*)DA(yy2Nr1jXhq4H?xQNBhaB?l z4+XVzIUfTpd;7c(N4ftqGx%^={k-I~ilnqjm-~+39z*Oo-uWI*>{_b`#}7e}6mznF z&hhkm%_O#GCgI8dkwr1`E z>7OfSM;}P#kO>WT;YZZ>g{iR8VP%N_oO$K8CuBY_fbUqi3k$Xxrx|Ej_QSbGjQ)|i z$u-C??B)m2($)*6caL^OM^Y%n;-@-2JgGIHO$m1a26&--#QWW?2V9jSm+Q;p&5xSz z2&HO;XK`yJx04-9Om?*GDWPHrt4qG81jBVvutCisy>A8(zTlz&VIA#zq-UCDrG#JF z5qGFus%S+O9`|2LZEqW+MV8=07F&vw&eR{~zZlQi1o2P+b*g7t2hhl@X>lS)r?$=u& zR5J*GS8aLu+y@@*mRt&?$AK7Wp53L^)5}l(4>>ag{*yB;&(n%>jNY!WLwa|G$C@6E zqQcS{sd7qN)s9kSQ>(@6QlM<7Np(%@xIMjV*m=AMS)m|)BHfeU~-q!Zh^`w}j8_==N6FxwV0DajTrbDnERK8+7 zh37P~QfljCcXakLsc>c*g)>Y6pXJfGYl>geXBFI5ujczm>(^vC`GkuaDwO0CiS8xq zc2bg~{6@$yZwzT*eGG+aJ?rq9Y5P6Tt`Rb>d}LP(H(7qau;N-2XAjD(bqDl<^C+$O zRJ<#An;FU`Z-1kQrC$a0(=U#klIw=5W*TaDW3{;}+{^1f<&QYT>wENQa+JB(d+GPP z;J%j~^#JXDAVEC+Cx`5lTh(6D6Q0=qX7a$p5tAyHwpA=uh6-KrWob@nL&I1MXoWuy zMwOF?)@OczHdSwBR=%9arMj29vVh?VUAHJCXZb%RcP+=1g!vK|2VI?#WiQwHa8-Udv$waa9^t{AAu-Mi)V z{Mx~~ub{xa(QV}ZaBH;Ug4{`!pt~2-s=#d*9H)C}Pq!4nQj9XVz#!A3!BUT@3t(4S zY+Cs_ev+i@3EJ7Mi4D}aa(bpUvfn#G(S`7=i;$Y;xx00K8*{(cP-Cn2Wo&q&L$|SG zIs3QL@9e0bLpH$4uAxRL1XwY_H5z`7ciqBYrLaMa+Yv zTyBTNtDnW=kbU~|pxK0|Nf78;;PL5%Gje6M*b|+81`HqGo`^T}AQTnULT#CrXETG% zZSt?i471Txvznd+K-o1wZ5cqRKlwEgOqNY>r4D#Blj8j1mMSmklbL2#l?Hki&F!po z)q!c`*#x3h*B7_Olj20*K|5z~wb*AZ=gr-k8GzEWG+Qd*;;Zw|0_Mlr(<{sp)@@K!^i|jyd}_+W z>|317b7nIvM!!Agev z8?BsD^5?3wsNAekq_y!5wufNDm785P{ZnNEII@qBG5N&w8oq&9k&L=;*N#g>l2*Eeb zoQZh|`W)E>0?PmNaX~IZLx#vi6qzHrzAjkIz-THBTvqt@;K)hyB8Rtr>g1YUUD1!Iv7R!S_F^%H=?`N?2&k(l;DR<(bPG#bEi7uCFB;KMqWMJ%(s{K=msKpC*>n6WT)lE<7)n@MWmY8n61Q*kAvKr5u?i2T;aNYhCx_thxGOFMKMGt(8` zei2E-zTmG&`W%edJh-oOE~@2{0o3gEWJ>Ph%jee_(onmbo}hsaX(+gQ$K|-48)n$8 zBt;^=zt9BRb?=bGw3eZpUFvAawr3w{Li)N^E>99*>Qvdu!AG&6@8p-L-fm^FDbStF z;-KBJnF1&$@dEUvBVNf>o=m@OKu!$APUnGsasAB*hgeOI^7=`UZ54&AJ9z9P?x>gD zg8mItHa_5R$1D|GbduQg#S%v~`E`6V(O~`pR(e>I7*}!%pRK2M;h1K!JweLA^WmHb z%xYtMdM5jzwE=#xxmEg|h=Ns2ogsg+LmBPqr{}-QE%cftUW{%ls$gPL^3uWV^}f>&CWy87o3t7(ecgQ=Gh9&IMcWOar+{p za{p1qxwCbIi$~=@tk%B}u1{a*A=y1l=X6cGSY8*u?DnCs67A}^`ck&|OBHrkZ##R8 zyei8=bASYR^J)Dq+|v-rJ6#U%7P_K3-%K36)2jokk8vvjxV>eJz;O#&U;E#|BD9`hpoRc-COe%gc<@%7fzjo}arh_u!=0&G~Zf z*Lf-i!EsxS6M4Iu^R;KnEJzkLZ84wvKC6%T)%ytCP$s1;WAVEUO}6n{N>M6iW2a9> zC7JTPtn~zqf^NHVjw}S=-fC^{Q@%L`+7QTap@P8GnFSFB)Z)Mq+3($-xcqGWW<%#| zabFW{a5q6Spax&r|L;yKkZObci;;zc5e=J4i;#xHyf0rzSf1qjb&tey7$W^nfyzbPp{x)pgicbs7j*ES9-L%#@&Xt%=U zF}ElTbeEk?vJaRUePT1Xth|4yS>hFu6LYP<(;v zUGnLM2t{t1o1mA{*3iOQaY?y|Cd_Jy*n>z9%WV2>mO3Jqg0lg>fWOlK-qpxc!=-uKmX+xd zu|nw86&a~naQ)S!$v!H;z+i21F}bJZ6wR~oot1~G1-*0GEH1M0KAGB9fo&}b?%I3j{K(bI-F;hU zc;%O9 zDc70M14keAX)@jy*_PN;^Nb$0BSqe;XhZ9+tvg!_F~qgbnlu)vyie}JXoZa!BNxr; zqP1!pWAD7`QGt8rN+R$j1QQe1rpP{xFmqY-FskQ$%}->{JmXJ+E3f)GV?rWs=ibC3 zV1sNUsabi!=j8Lb7&TSlpZrTJ+eO+m^%A!&mpfPtl`2f@cZDul4TCt~=S~jX+b6b? zxb~DTit4%GJh{ztbuo8e{p@Uf%GP-FbZeNg6bkcn%ZV#Su%Mbb8}!n0{yw!8v|8+= zS!bi`gzXO8LO*zY>CbsiGo4GG=Bt^v1sw;@zLh%q{E8?nuF-y@HA$~`Cal{92@kqi z7O$=fYAS3)H3Y_pu8 z5g_GB_b9jjY(fU^qtT3ajOI@N5#Jb^C+RKylkWv$RzE)yp~BfY!5*n4l}K|vH?}n2 zMOsg|A=BMff`%@xPW;kA^~B6miCQpa6|xJp{@lr^cRN<_RY}yQo>j#xeT0ZNjc`tn zxL}6MmankU@hGh)D<(JPIY}o`=wQKyOgG^M)Kw+J4lPEO$H_fuTEu!6jVtMysyfkH z63Nx`?Wzr9S9MxK+@#49j^zqVNsa4j81l^G#r+#CFGRvqhu5v&c5C&?HU>w52}k9* zeaY@03Z0<;Drb@m1Wy}ZxD^h1#II;RBbr;-9YjHtp?G}*L+vov?jiC zP?cs?TGN6>&UtMSYPX$cF|S%P8-indv|>^@y4oBqo)cJ;L~;=lf=3VL2AitRxuJ-&1H|3d1BqqJ zuP?UJHDKYDh=Frj(msRUx<%%^(gDN4O%iLwcG1o7JIF+xbi-V4&iNQl{V^B+pe?Oo zC(L2=?~?~3n}5$g2hS9faaxS#$mQw9nG-)XdZ+M+SxNtOVRx>LSpiY`>uRtkD&N@c zzOvJnA*q<`p5Pfu;-7$7R0=A*Dxkg4c+@5CyRgN@m+_qrZ%lH%b2^HD?!>7zPgl0f zVWuB9ExGhh$F&+%LOBKZYUvg19n)Ihm>ES z042wyFZKTKmrdh})Dk&$Ps>h4lZF&UY&SdbU_G#BNXedWNu2rB$$r!8x3?J3)t?K3_xTq?y6tNAH=s65IP36+$u?OgDST z#L+-+rDbt>H)VNFGm;6v$$65M6jO2NHDpRqlP<|+x}!Ynx=(SyED!Qv+~|KDJ5_Sdf>0BtMoo9h0oX%{W7A zNaL{vCg!1(`h0rCIe)5PC2c)BkrAw6`&t$fl&G zRKW(vd*os{f49%I!b581jdR6inn>?S2?6 z==scvIGT~LAuNAPBbmkA0F34hY)WUmt7*6zH8l@@TH<4PwdFg#EUW5I89M!L9v5X`#yLNe(VkTerL-uGzDP)s% z?Z3{Uek~G1kZ6?@mrEUNWck?5)9?KJRVgNGZ=Nil zRdzaTIA<_VmO|MDK}idX?FHDF%Oap2W7}v9HytFRXu)+EHq&B1CuPWW10}|hxCh&6% zPMdn^7kXj!!2#%=auqV@>DY-+JPcHay4puKGV;wrfwe?ie2}MijSg%^Xyml}{oFEV!=R~jPFK5>nb2%l;*z!6=I z%iK2_K87*A*^us>kW@vO+c_06w>D(j;UCu_w7KmA6|Lqx!3-&{YR|8gV*Z_tS90A* z2BTZ(iM}!H=&b$6-NnyiHp2+xIqj$g(2iar>Rx8{d)gSx<}RQd{}la6Wa8Daf0 zkj&t^d{BLM{(xyT5O4;uK%0^A>Vbs0uDDiO6{h0YkBKHeA6m+Xf0-=b6$*fx=%_v; zWDaZ5PzT>U8?Z0Fu`?85fX1$$8*zH;{jj{YFda$uA_t3xbvv|h`W9ih60JymprI=_ z`l8wuRjk5-5hrP*tWwaMt>1>Xe+*+=SKd8#XW3g*wP~xTx@~I-@3Lx!!*}kKOs>gb zdUj6gjDc+HrO~{-B7v+z%IW7pb;ay^sq&UZr_6G@qS7ymcsg?XpFC3BpvS!djs^mL z+}m|EEmZy7|GB+g*x$d#a1iH?cS$h{GvaCLfArZc3^vl$pDZjNXUZt?YsHr=Ph(B` zoH>!d6YwqCFnOJD;0X|CsY2_S{FK@#a5i=cDk06jmWj8oNnz3PG{v_$ zG$*%N*dImX6D+t39C)Suf1rQlqb$QhI*I#bFh?2~6I5Sg@h%{t?t~_D9*H3IHcj;7 z9vP{&8|KQ8nsIR*f{WWJO{{jXe^h??gZx-S!RnUh662(iO2fm#Omuv)HLj3_T0i?h zwu1k%L(uai46vw7bQ zJLhhPSSig2vLWmR)<7lE-gQ}Sjh)WSDjs$m%vkWZTJnX@GM2mB^(NXD4t#E%$X{AJs2O}~x&`qg6M2E2n|QPJfS282M#|Fd$*1LuFIchL z{X|PQ@T6xhpQ1B6QDS#zs$&%QsM~2>9$ysvp7zQP$rPX8aWUScY7@579cXti_0)-% zdI2j6W{N2u7EGpRI%_-q*S_S~c5>pI$;-(}v$V40Jhu_pd3n#OPMLC$A}a}2EDXGV zrl6E6d(if_ev^Un<(Dsn{ZrOKTt76+$4L)-n>w2AeqbCmd8KxwmkD05jR53&l%+*?KIq%ODxP^63oO?v5^I-sjS3BN*US5zV^xGv~Q=&Qc&Kd2i z7Kb9->_P;JLlVl_stuAvKWtCj*#<_O;j|w2o)^BFu~$NQq`$=AOyD7}tt^)^3-Po; z-ooQQ?eTILP>fY zENxeKRV51Jmn^Zp%t))N(l%PKlABar2C~7)nOqn`$TQ9gLIJ=zezqga$6O_JJo_Q{ zlOu8b`ODmNcA#nOxX&y*bsJ#KKpzdrNm*WFyws~w`p{w{FhgWte9$Y6+|3Sq6wg)x zb3vQVb{GazDlyZ!lAx60F{P7tIrYdge&{@M$ogFSZL%*2X+C&{F6xQkT$OKrK}{hL zp!KoTPjjm5HrgV=a1HA)Ew?RE+Wz%3JP#%h-30lSMCEiH(hEl38J~Y9SlUQZphK63 z(mjbyja8?uDyIY4HMNaBcZ_gY58i%SQAcz6nE@F>rrDxfNxYgtra7be>h}GG@r?AK z(bL)y1nmo`OrtW{As#>kLP$|dP{8yxd|pA<_M$r}D?R9W9_rT(DM@AF(W&#u>Fx=O z{E5Q;36+JIR);xl!EKs&M*rSsV-m1ErmDSLYqRDYiShSx+h7A0A!F^%TWIrDOLpM3 zg2VV7dQjPlB=>a~>Mm=am~?0y#xK-RWo;R#bvGG;2v20&$~l9L=ZBj>L?+VzwogLz z0{<2`T}eisXrg++5KVX^=P#*+`jK%brn|i-t=WzfEj6u@ZXI*aAmD23i^<}GXWO*y z(~21arUne5oAF}#J#G5S*N&+K`)E8gaDWDp^n&E$mEL?8SOzjqF&q2-)Q-?2F@Wc* zp$6;7t}f66J8|`_5Kn(!a6fn8+&RQtgJcxQEE-rLIMHi@i(xtGy>*INUPT_{TT*iO{q*A^|HP@k`;tY`Ym#KnH; zWBtijRND*Zv13Fqh0A{VZ4sCaUm$k8xVCfb{FV88TShbmtKbmttLThMp-R_8cmU9m zsYfCPh8MJOYFO>6FeB8Mg@0k$fTug_<#iJu(?T)d#3x>&Sv*$p{1(0?Y<34sD@Iz-1vg$*PpyzFb5NrCJ4nUVmkD3s$0TW3q#Ajl5=R~w&0CpyOoDo zcy?M|BC!CCfw6>MB+U z@s5IUFTF9of#hsQMF!)=L}$8Qf+0=(_69QnX>l-rm9dGE7X!0{v6{T5_+Ev$*~={! zOqTrdaVE|D>f0hr#g$unH{x+Q#D<`evD;$6-b+Qz-7Y?)s;dpV)&{R*?4Q+;_V>TR zzgeQhP`s7(u)RDOmu#2m`_oo_SgYgQ8eN1(S|e6E@5+lU$q^eP!<2Ot8Oz;Xps=QT zM|cFVI=RZt3&aFGAL1Ol6_6NXRaaJ*Wu0FwrMqUTS!RuK@VA)`&pHYWTg-8zTHBDQ zyW&P%6ra!WpY(SNTBctc)#6tp<&86<0;$Qt_>!$iH{zJb1t#nI7Zc7TA?r3 zTFEvYH63!el@w#MVYGy&qefxK&b%jA8&6Ptt|Dpa`8wGl)!~FFN@D?7!g*{wA+ST+ zpTK@l#93DVG;Y$%=bFu4cKLG&R9(&AgXf>VYx*Nn6DOw-qf+zvhe58 z&xr5>(^e$*O)zG$ox&an5|MOVp&bQPO(L0Eu`a@mM;ya zEdzNhHc>mPusTl1Y~N`$>fOi1)+#0N!sP)MKINd+WUKu@z_ILLR8Bv!t%!vxJKzl# zbXMn3olwK&Z)dnwCyrT&9LOI26hN%c3nK{2yGp03(qk3kbDt49$EFHmUo+%w+laX+ zR-=Tn;*^SrGyWtUaP1;fEFduXe(Yk^Rs-4EH9zs^dg*fOsm0h+BM9;7Z@Rpuvqnt% z!ExRe*i1nf&G=E})$R?Im%0UkM`v3Tz)6BiO7&c`*0C*^_jc17f19zMxY_!FYeVWi z549+&+1h#dj%@EynUi#8sFbGrQR&DUO+3iJvD+bgUPmVwsO}@szMBvajO)06IHFkD zVyuMLdM4>;+wMDbRUN#rWiaMivESH~km|fMXeDLTq~8GZ^fnoOWF!uwEqlM{Zk-$* z2~IlMdFl!gYw?RWF7$4O(Q*q3zn)FTj)hC$24mRTK>Evc#`GY`P5^b-R9Wb`-Tc6@ zwkE};V0O>jFa@LE_wm)Cy1Kc7ds?(gYA^A-XZsiA{oeBWJ0|KNE3fvTSFIckTT~e| zFzYtRUKiju*knG}*sl|5AScFo;@Xw7GVxLx$(q*47=_5A3*rnxj+N2XmviH_BNy_W zd+;9Y8FP7pT_j?0-E^F(g>%$^&A_%T|K|4mBZF@7hNiKc__yD2MBn|cX?^bda*?^_ zCK_SlkrEcQE)%b}aZyxeW>q_2c92-Z_raRxEk(5R6EBV%SKfSB<4`y{Bfi+)G~AO3 zWge@;6LD9%oKv*IEF$kku`83B1~r=;S96`UYb$y@eZRLrB`uNV0P=ivPhOrs+P(|a{IQMHu&3){Y zS;5^(1`(*NY>84<7S_a|kz8Ygv^g0mkx@6zogo8^f%R%PZ=_eo)t-xtb``lcb(Tm> zf0Gv#o}dY8@Hmb;?cIjzzW4i9pUt;Vwc_n>b#aAO&;>>#s7y_PO>0Ig56+9G>Iu&A z1RYYVr?Yj;A~$ejy8{xSoo9o0iqQoYr}f9++00i}#(ULbsqCPlDtKp8(xajt=FxC= z-tt*^;vJzX=q_8>t25l6Sod!1tE)RbAZ{C5b-oQmzgTli{%@wkfgg}KQt3FgL`su&i#(rvR7Klj* zA0<8hkq5WT+|Azg0#_ft1YHJ0-~KP_MYHIB=PIh29;ljX-hOpVH!cWrEpLJv%$cPB zZInlw*w@6b{{YuGzX=a%0jA~6gQPh1fA|2n@;pqR`1Y+XtpAuJXrYq>QDab!B0Zoe zZ&)0T`|fpXVDIiK+#M>szKGy-JoHsX&0VnluIo{Ase=nSbKcEs=1>(`du<^C69Uy| z4S{(SZdG5TBK6OGWOW>yBE9F2QMwL|`zr&pc4ffSw29`|31~Qkfy5|C%>;m3LEl)I zTrz=7ZkJk7F4DnoLHrN0g3yJ;LINnVO>iJi-3CYULm-_sIQt+PI6}4Hd>W@PTY3W& z2~1zGd(k2XYMkbwL0_O8>aDqzW1zvZHrzaLAR4eFxoTJ*F~IEy8jQphjc3 z-^9=^C87&RXaS_>SpSE$|CaR9Q%ZU_g>DQ#&eQe(`4~0~=JR@|c~m}byllTBZEZ?q z$12N-E}q^3(}8fc*Z6Jr?UU1k8jOR!(g|PKZEvde;nz|}?~WgV3N}1nrZ`pVoJsi* z43z@RgR-GYP)ODerqP5zm4JG)T~VAFp0z_B!>$EUE0N=Rt@qDsqF10lRo(iy$`s~CXbmWkKdT0MHxTcF>f&_OGe?^A; zEC}J%@mBZI$jjsY{kQr2NByVj;Xm9;%A7jH>(ayPLBHpDFgdyo^3;;7-HN2dXk?(a zva*YXxzUxN^LJ}A1`Lw$nC{N59k6z7n7TbC+0G_y0ho#t6v z0aZ!TIFS4TK!;a<&q(pwoBRKt=B_*(>h*7zQc_upgo+Xsp_46H+q50~7P6!;_OeX~ zNu5p#QAlLT8dzG?pXdH; z_kBiT5sG^DzY!f1xnNopuA!>EM_N^1#uc&^pglOp4j1?40}vl?}NFiR(a`dF4*gGhh{I_ zUq~tUf316DH74BWK+rl`$*S28!&LHn+%4v9&$F-VW}K*3*)5T;Sy~B>B~)wL_9|WV zO$t{03guPA8&C2_l5X6|pI?B<&jty^x{&h@r#X@)8$Ydz(N~HUnffF^FD9X(6|16M zd?_#ce5zuS%2wyF5~W9if6hWWJa12>HQTeM7a;VUUz_Yn>5Fjm=ax=(_Z#?pmWv|o z(0!`@LBB-4_GWLj^f%VjWo6+zT7T=*XK2FwS~6e!%9;cpsg*kmHPNo{{(|bpceCgh zQtk++X+}3ecIhS@o>;lrud%2u77yhuU3=c(2K;bG4&LN09`vit88;bd0N-|0T@Sfx$k-R;G!Oh50WPz?$_ z?S37T21K9xRp+_V_cXwikIM^@XF*WC^~&4reIi$y8MJstC%tQ6f3*@O#PVuDSk)Zj zUw+}~BNQzm7RcX}3{QncI6a#OAt|B=rCE4|xFG~RGwckh5q8Bh=HT4V4TYA>$EOyN+J*-mtL2&r+)hwX zj61PKFV2CLk7X9XR2&1`T`Nu)qZS9>9ngliW9_Wy!mhTRaECFOHPPCd6%V1VX$xac z5Jf953wo{1EIk!}T_LcchX8c%?&sP3ZZ`M;5iZ^~*RO9*>IB10h^V|6mXC%XJaIo$ z*y*x(hPUDDsD3)SS~bws`t&4pFTRb@=Y6xOu(g=$v|H&HD ze>83so#agru$VOV6#Xpm5V~}iFcgd8r=F;Tvf5dE*AZLuls0yvkW&A)(0Z)fT>ej>u=V1%@o~iR)fJUHM78>jG}^n{5ny* zF^O|r^S$G=zfbTsoii$TwFZxmqslcyqr_LeP!xv&gLtlxU7q13RD|Zc|5-A0s(Kf%<#)H^C!+asDl4~`*Cu|S{GNV`D)Cg+Ph%%MGxF*-O7L9s(~h~ z!tF$~3Msv5WueNj8>GC;chTaoao@r)W-<lK$}Ddps4=6|dmVvtNX2Q=6xE z1TqyU{iQ+qpkiO^r;BL)&sLI4>{{Gu7 zzirZWpf4rI!PX?FweIHO9SbSoq)2aHhx~#`XeggL1nW7#4JU_ZftgPczpXs*JS}2! z>ZfkC-KT^`q|aHNNIzGR@Z{8)E>K;_0Ic+3&4#Tez*ZlH>fD%4GjFO(dI#ID*<-jl zJ*w(J*#=y#5_j0Rwqkbusu*C4N01RDe%oFA!A#(wM7R~9k^tH9KZOg1G85(^5AM8= zI*3F&W-)1U) z4D1Am_W;2Tz+69|jF9L(vD6DStB;(vTg@fedFDX%;$isgIghssgDik$k+08g@9K|H z1Bz9rSjC*-eTfluxHc;xfQawg)Y+P9=@A=(?k=iZ04#E|>242CX{N&mE7FKt>b3Cv zK`vpepi6wX?Rf*~vjH&w)Eo;02~4}$f;x;EVZC~I*~(dr;Or+aqtK?3auw? zfIWIKS=ZOe5)7#2(Q5(V?}1;{MO#%#BcAiOZYF_kH=f{7)1~NT=d6D zuL)?@zjM&&u}|Ae5;`9FVUb@{l`vHQQ>{8v2Ri{U;9uC`U`nRBSw9>|E`PIO*=+Nf=e zrZfty+sPOvouj9N5EmJX3d|T`(?8D`5RjZ`@o6cSXP-umM+kO)F1w8f(_#S!AryL6 zOabRb%L{V|odn)2*r?5G&|ESIDK4@%J!sF@grg=i7@SHa52SH}*?BcE$c!ig)kJQ= zdh6;3uK;0KgA`B^WIi0t-aH0~k2*qfqkWvz1EQxi}YS zqrihVXUi-*dfiWs)RLNdGoJM;9g4f0p++k{gG2mUFGv#$s1b{$WI9u?GQu7p^_l%( z5NnI$ggs6s3fQQN(E+J+4l-#VG7DEkh#BOK-CID44}Y-){q9sZ=^FO!|*774wOY)F932&4)hU+&l!{J&w=+&}%;i<%AL_ zU9*KgCLD5*R1WvYb?a)QGIFvD_)jg+@nr&1tuMmNAHQ!pcX4Al^ys)LFM{`w$C|iV zSv*iU;}spQLF25pa!S~2!Br5{AwuJqqicG0-5I;TtkTzy($jgPh}97kgE_Gk?9BoH zZVga3?KN!>GoZE|)fmwjAMggD&r=oYAovhauwel;_82dWhng9rUGIVxwXZiGgc`Y7 zwqk>Ml>E0@mdp`GhG16q&RN}9Cy)UtsSX`O=f=z#X!*&2rnE|Vh~;I@aJjlc@pTJ8 zRG#YQvT#TOaC02+vrw{s*qc$tlm%E>K{}kwa7h5n%3)#ZUgLwk2A8ETV9oRQ+{}fj zO`&Zfi zsqG15#v6w~!qyX#RSf7>xYBr*0KSIItN`MJ6yX1Ge?4W9s-K*6{DAB-_TeVG9Q;%3$8|) z-NQcADkWu5S_!2Ftt0~07=PjR_G=pT^!e{Ipn?38klP}x=_ z4F_uIug$7XozO>9`#$|@fT930*3bWWQ-9ro&{cECLYsM*ok^NAwI$j~`}5I=Q;f$~ zR-&f6G=N)hE0MaTJBc8CA4ZA>k^Cl`as$MnSfe&@-HpihC>!<&V(!S#l^( z$wQFkFgUhSzTyjC-b%HR+12FW4k!48%zv#FN_+Ww3rInH=)R^Dor)%={hud>dfLIf z5>wAM>F|1^u94yVL3mqbdf^B42#tX{aa@7y0_ubuPe~q`x*U%bh}qaYC{xXbT=eRw zQ^LUX4u89a;dt8Z-Tg;S0?s_qcuicB4CRFn<{8ut&nSQ^I`tU%Q82gB_3+_e#{(cM zS9<? zHBt>I0pGp--A9dQ-NWk+gHgakBU6vk-GRubZlBcK;vl(t;S`6VQ=ox}uv{nK{djcq ziu~INUWyNsGR1QxnRFgIUVK(g&t@VE{U0{q5$3F7IuyO%=d5T^!@_)@0v`HWFM{r- zNh|g}vs+70q+khnHEp2s5<33@8eHEK8|lysBbW(!4PnvX@|+8rvobqi1OZU*b;q=W8#RwaOU_xSSIP=;aaRHY7$ zwJ89h67I#hHuJ|JxA~o`hf71Uca6ez`Eu>{{&hsIQt3u5&AcHeuNp95ol5t%s!uJ% zymuD#8(y0n-3wmOh=@CUdD-P}H&ezWQwjI#dZ{_E+Rl7x(0}SIGu5D(>b1BapSN#% z7PQh^5t)D2Ju(0O@oRpkKoKHZC`u%U{D3)AkO+PV(PF|^(Hvcx_5Z6)S#uOc#@VE) zd}oZz&5^sUds;Xm)y?b)F3Bq3}RTC8ROu~oT@ShLE{YM7A8D#(Konoi@p6p z`_v>Enl7soA{@@t%PCDIpFYxrq{@vvrnJMMnZq_%TjvV1RvqOSkK5Os*gM5aZqN|> zQiO7g!w7@)4#CV^5;-IT!nFl+_aU>NW#`lnEuGkUS%KA=K(V00QI0~#%U{YKd^Dbc z&W3%~!uMZ6(axtEF{sfGZ$-aFs>Ng#))LL+6Tz6d>qs~E3V8zQd3R0bYZ>uLrN{*? zz_e$9U)+KGmO3Gpl<1^7-Lwf9KrS-wBtSm-*$;j{cqbJ!CllQNf=le`*z8pw#Sk>Z zgVY($KvrAseX0zCC0RzTG~T#kb`)dQxm|?e*!~)5_gt_Sh_384@n7MJz7l&ZTsT%l zS4+`3zds>G=oAXSNijM@lb)(!4@8~qK6s&4<=9i29K0*(RwC^AmWoYU5&c-#nKS|l zu56!rr$c2s>ZZ}0vc}AqrWkQC=_J!QFu$Hfd)H%G%(}#-u{ig!S@J3u05bjH3~_*k zK%fRJ7lYgW85h0*qYk`%|4k%4Uc*8BatyLO|5Z=o9l02`I#+JaCbL4I1mRru0997j z+yvWnnCoA>0?5SxlOIp=Q`)71yn{n%qR%cn(X=EKVQ$mUWGmFg|O{6E)6l&z+76t3wuQ zFN)#1`7KX8!jLV1;4Epbv&BhsJR`0t89*>Ks*48K_fLcSDJkTOl0v&@cGDNn4hXq_ z@?L8LC;jxtUYNs%eGooix`l?y1#Ev-?f3XuPBGR0<-H<0(=l3QLOx?^i1b0wxX>s z%wmV;Cr60zWlA)kuP@rfKn!U!lbJ<+so{2Xn&KdEht?c;TQT9>wY!9opABhfQM3>r zZ(p=|{C@2uMW=D^%C$lgm#35Mya83V0I0ZvxCn^gC4pP?sEl4IJWt%nRc>Lri-cm) z5S_AzAn)^yoCkM{RDfe(0TdQE=m#!CAtyY5P+Q16QG-_rkFAmq5#I86*--Q6$kuj% zoSjw4OqD&r-4umaHq)b@6P!!Hpi2UJXk`?7Zo=a^9aC}f4u41nq$Gqt8rT;vF1TZb zRzOJMNGJW4bFWaM2Ab6sqxH}skVc0f5F*zn7K4jJ?>}XQA1Y;^TH+D$ph8*dwNoOW zJgG$m8Bc%nhRP&;J9N3gjge?`3lb_Jwj~r24kasQ7|(cut72w3`5CZv7JumZx+QTi^p`F#=kc1)oPp1xjy}t8=(E&5T0!BU=w5W+@GJ$8 zx-N~w1{N+AHT>VB6DUnE%k?^Z&PdN}vYgxgQn19#7A=#?N592?U8xk0xq0ar9TE^W z5MewJ5($co8f=&L8-fEJ4;NrdRS7tchMwvy*=&BRg-`HF66R6kK(&5)0*YpeSM1Mv z*bDiEI)JFvb%20%e3jfoNG%tcLp^;qN?z`t_ab+Cm4qvR>k;~NE+PaRfG2g3$x?^x z-IMlfwiOUA9exn{lrBq(HmG~;@#uJ2hWL52pVFPN2Gd6gHZn!xNi*DBezi1d@v{H-SBzCSU7%EBbW{yl> zSBynQ(G*#$r@v7Hh~=cM8oEFsm6q^&iC|QpN|G^IH{GL{WLnUpc5mUYaYpk(x3(Ie{?{OXE-uSa1a2IrO%p;nR{1Q1L#s|6%07^ zg`wc<)4Lx;{EXfD4P5N6b^ zVXAD>*pDa`C&B@~3i%T?*$(m^h(Xt5B=$GS#!En1I})ri^O{lIeHW);on38D&lbPY zY`~N4C=qEOK2VRi2$Go(L9Rxd)2?t#>8^-jE+#I)eKcrzk<(CMWQRlds+ig&dIP}d zh6l7Pg%Up~ChdreQ%s^u1Z!$+=1^Z+(u75U5FmIN%NhyuRxCnz=IjL&W+{jG$JY`r z{KZp-kOj$m%s4@dMML!ojZ8g3O3b$E0E)f=>#+Cg$8v!U15=k{k&A*{C`7E$e$0RJ z(@r@~PY6ux7rJ2YU4s%Hl5H41cbFSOedp8)e!uo1NL#zB1$ zrqhhO-omUJCxb_K;K}x5L0Sp>5*~fA=%;M(vjgwu5fJtwtIC9|s_z7%2i%po? z;1hRyU2r!2wy3tsf_xmL{W9kS^Zz2mH~NGoD}yn4IoHw2HwpPcr>NLZmO0&($9Nwh z$UqGwVt$8g$Zw(=d}_;09cxVZ&|;zCmCE*=H%h{KO4D&+wOb^+%#|9xMuvktcPGSs z1+oxBikVA(XM)nXflaQ2^m^4IKL)=Yp500hlvKlV(Iv14o6gm;Z0rU&{fZM;Bg8FSK6zR-H+1D@x>S^09Q@aP`7T`%ctJ+IWXlsG9^U59*+Mq& zSJ{j)KkkPcd%a#XJJ$d32Z4yoN2O*p`9x!{#}nX^B*FT<3=Auo?-9Mo(@c}2Fcn05p%x#Yy^5~yH-|Aw^zgH-+D9pDM+Zjsw> z2_YuGiVxhYx!6sPE9YE$GDjZla|hPsa~F;B8AU7un_VU`T<_cAl- z&>aa<-d{6xC+^@&3JFgHx`Qi6$}vs}6`G;BBM@9gZtR=EnsB%!I-k~til`-LBPJ^0 z{N}{7j&Im?&p8FCAsVFiK8Sp^#e1NbAi8AABsNpE5T!4+(X8T0D?Y54IP1|~C5(8OFsBz-n*NXB| z-_TDvNp$)57J#wga%jikI2m1qY^zaGBVltdkacm(jJ{fR_d}TrIS^SFS}~IyjC?q) zQ-yH2tyd*g`U6$+kw!=!AWvx$n0s?557$le8QOM(Y&J&=M5G6I-0RkT78niDzC2>d z;=*tpRv}7vHA>J&q}@Z;eZDc6t+~OM^|<@Eyn3s90#zSbD)i=OkO&c5&ZTMjx8rXVh!VkM?&|k(< zH5##)D~3{I<1hZn<3R3EX>l3xaaZ4a=O>plf(8d-E`yt@2B>7xclkO=8r$Rly}FPx zJz;NKhtkIXpQ;Nf9+6qrs*j}p>vg5|>Yc5k0$)2(W=(&s(C<|DWM=bL*{hcq7=vRs z)q3?L9o)+ZZp(e3i9cg9y40w%;s>Z#xX!0XjY#~*7e#->(gzg&SKeqMxe&+){_TtJ zZsfK(liixq6fBE(HEu+7i8PPy*Qa??YZE~;2IKzzJ8}Ep9>{Zh z?oCTKlcbh_|9xt`Z_s33^hclEt@^IB`19xE6-9)~hSmWV#Hd72tSB5h^l;UGED-#K zx5}n()r~P4I)7h5+9UV+qkyQc@uEo=^R|>jZZ=5n3;dvT0kA~*6kb3D% z5h7n0EI^cSJhSVEg3B-Kco>P-ApWx_H_zx%vs^zfAoW+Fk`_W!#sM>>YnY%I`Aw@w z%Tj^;-+uM=Pn55hBf7uy@Bca}u57T>r+CE!Bviuw_5j+qd!tcphAw`JvtY_;M~K!d zkB7?N<#2xfi)b@s^N*aoi7Sl1w9HPRzV9VobYh8Ee(prCk`2D6zbh>Kx^rrXKdem& zUMP%6D}(rDt{__W75x?j-8@E2=a!wVcXz0R?{IxYxYA1}eIa)1&ztJUAVKAdCa6H% zt@Ys_4AL9K2um5B3!35y2UP(J@`ba@z9$1>#a-GoWmkfhoWH-nZ?`(EG?FQ5I%Kx2 zHHyAYJ*}3B7f2&na9OP+t4n@YX!~wLU;(^CmY7l7#zTUBs5}2Uq4&MvxMH4NY6`I6 zH&au-^o?BdmtiTAe=;<~>E^iQ4UOrNp&9bHo4UNAk)jUW^LC62S$ci%8vh%=?zwMo zs@V698`|;IzJIVEY^Sd)VgjwgZNIA*{Ichw(P4l%t#Qrlg{)=ikHsR-_=eqAkx;8&K6rgOH;y2QCZP0n*ixYL&_Hf0JMbD0%jcvDv2EoGyQ% zOyaQ{@n{zt@x9oKdnT~mBz*g5VbMhG)?EJz|Ht@*uxrXNCCqNUhHl;Ml0>#u zHg^?DO!zt%%i{~W|ID+Q-S;-9bNl>_qA8>5!V(_&(5eHCFL6CPMO(Z&;l0OV9NRuw zu4zMg68rfP>)fFew<|5hlx1pgkdAaG5+hwXw#g<|9V95?Nq@4E2Hw?*5N5n0p)k=? z7QgV0l{9W3H}O#h!^C~C>?-MYtWwKads3wg`Fg7@sk964|F-awXz^IE0cnV0kt=P% z)#-AW#P9xjD$>=0C{<7;F)@v+xFU)lZ7z1||2SRG5^fQ41UEP)ig!$h$9P2TlpL|~ zu_13Z!dV#ghhf_~vRdbl31lclPWo7kYq7&7G!W$z)d%6zmb(Z6geGab=P^iFi|-CkT)N%C5+ zY;!wuE}uZwsxCaXc;JC>)~M)E#zaSkd(?bGYS(?Dd8Qc$<<|Bq3*0W`F1d+?LgMeZ zdDD^%KGErP!c@y15yBDt1|0jAsK5Ld7F~$@amj*=UDLbN8dhlQW9-+n(FgNtZrikS ztz!IjHo*s48NaJrE3#}mbA4^k5e6T{Bi!rOFrE?)DLwcu?#5=!QByV3r%DeF<|tY| zx`^3M%Edy1(}T@&AH0fQIJgbxl@wL$4-RK!){(BUJBaPc&M78K=H{gc+LA+eN?p7b zVun?H;mUu~rq5e{F}MD$x!z*rK1r`CR{YqBckKS}Ggp%1TVCg!w4Lj-n0FTK$$jBX zPHY<-*_LA)b*ufZD(*$ZS`FK={8(I1{rzys_)+zD%5H_h8@*n}vxi1y zj5Djw3<@w+XR~_L52gC;v7rp#cXv6L;8&@V)f29At~Kqx(ALKnTq$+7Z+ZoV2(ERD z#QI|gZL}7h+~+8Q2DyCQk#76Tl$cND2D3NfNIq6xgqx-7DtI1c1f@3O+zj6N@jT4< zO?us;eGmIZ$2!~g=nV#4X?jz-BA*@B859(?>DaAn?ooK0u`X}jV%V(BEqTqTREv5s z!ATpp@=U`$1nj!JLQhMb1LE~wx*is{ic^XmXCgIq3U_uZ$p)EkK6%MahEwp;o@1O9 z`a3V`*>||NrJjBVpZ|42cMi_iLju=LxVVsiKG$E9;AVGa@MTMCqUos5Gf7(~?vl0H z9Bgf?+Sa!TwTUqj*@>LQeMI3~q=%%YkCYrqd*^pkBY~#%Z_V=L^an{+@653Fc4FnZ zQ|@~1`Y}b%(5dVz9?!u*XK;75iA&l6>zEdXSY#AH<{zu*B!?eMi%p z4TmX0FW;0raFb$p+cRZ1KxmYCUQ>{Qy}B4PA(=955jEH<(P|Xl@R8}|?1B2mbteKu z-b?o7xuls#IOg!p+hRk*2OW}ylYe7u+|szUk+*TzEvx%hwWT8>EfpseX4>VphqvB% zM)XkDwAD;Xc!u+!Q0R469UK10q-AW+Glf>0YV_)v@cr!I{ z|E%ii%*?ze*F<$yb(2C<60MnPWTR`gMZ12g%yZ*5y4upGVQ+JZyBCXVb99z`e&Bp) zP{bu}4kw2!r6=LiN7G!hlyH_8EEFDKB-JqjwKfh58b#q3j*8@+K9+K>;f;vVLgO0c zxhCW0hAKG&!o1`ADmiJp1$FVf;uu?@fiD&URVs{CzdwE=ZeUrifx*??bB-UDlDd2A zd{^8W+@PPGj_zWCnd;&0^s`niuUt5=6Jk%yWp0@tu5#g;5ML!IK4-Z`3zI|dc7>V_#f&p490#OG&C3QKERxTbRGOyFkvA57is%jABohR50?;c~0)r)X%jx(`mr^ z9o$9XiBXr|N?sO`0s4Hp=@Tz-;tqdTc>EREykLiyac9eXXlrwj>+}E8ftOoEHZuQ%3#`8GG7DGm zZt`~l!(T?>-NFbjkvR2xBEsOHs}SmZm`*Cd#TNCdlPsUU zy|iN?5(#F(gP=h=@GxbiL7xeOoLW{$>41PJ4W`L6ph&Zn3&|pY2e{>6m<>AAVH6FW z>7~f)Ai%G?g2bzG5jaqSlqQiVGHsfQAkKJ)5C-!3p@6XS^a!}S$GM{v30ouDh@r?{ z;qD3Uet4E=_0o*%9w7K}-q;_e>`S5_Ka}$9Q@G)i)S@#GgDk75R?|buGRfWD@@qJS zfU=hsRT{Q!3SxP^{wk-P|hq!Cn#qU*G&hoE)ou3E~#XE)QZ(9 zRtRD1kTp2rB3QV22b2SmmY6JJoU*C~icEsM(?E_7AzE^fK8fUscM11ROaq~2w3qs3 zXA5v%a&^>njHpI%J==kw17JebNL~BKLJ?KwBNzsa#1R%k-mpyGj*v#J!TUdL-D?^K zaZyVc>L(@%vguo^B>EEObUeV+f5~P_m8hBI4ZIfs%Ds3DG@=H^wwR3jk{%*}8WNDkh;-AfnQX zAAybh(&x_vzUB)F20|!hAP*fU%z@&3&3@g<>#m5h3c}J4;b~B55F|k{=n-&=2Orjv zg>okRe9Gmc#YBuD@_fKd*S_eFpkA zd09uZ;OQlk*r=UVu0MbdXCOwhX))#vOAL+>0x1-#jlI_r^i5ay+6=NfRl46NgprMA z{x#T~@Y5Lk$Z`1wcIkKY&oqM$+4%d=G>);)#*Fr|#d~278_VwqX*YNp1giRUvxzB> z;rW|I5(`4!r()hKF8ZW(A}fj-O1e_yZK8eBOx;2V+hb`_!&MnR<7na=0t z(##h=f=}$-j*h&5gPhEEc6N5JG>)r-TsL+ftvqIYGwo0eehXrj)`;eilipo`4dy9Z z+?i1b9pQ^x6?EYMX4?UO>nDT)MUT$jWc*4sJH`%8=NBOrd+PfnTSS)QxYnw`ld-S??{6C-ky!{f5RVw42~_ zq|h52ltY+I2oBQF80lmg>Nl!r-%*iJ6*^m|McpJR&|E^)#XdG*$AvER;P&+PT=nqx zZ1ga$ldpq>icOL)#I{JsQXrN1s>*1~E{a>e(WShK5um1_!=>YdT%}h%u;5E6nUYp+jqX;%AVUJDD(%ddeM$Ko ztanN8tV*a#Zc2z%ElQC}t<^I331(<#pk{_jnP&Peu`K24LM_ED6D*;p8B4E9;$|3T zwDt+E7WUQlNA@|Vq1nvwjM>Q8&)Kl74yU<_&gA0qElO~U2d3yJmU7*dFHjNlQYJLn z7;OY>=#DVHN`@Dk6!gvx&eF~;ew`_qsr*`Ul$(*wFW)TlQ0!WdRA^k%Jv%U?Rr*bT zr)jhZHHbW!{FWRmvY6Pxz30=f*e35L&tGqU@i)ym3+_!^fn7OXCLXZw|C;K%vboB; zD%!Uj7c5xeRJO{WlRECNiG7e<<`6=(Ml?b&#j$4DWyWDL!iGdVZ@bMJ`~ijtHPEpCJ|gECA0tK5csxcp7> zGHYHk%kbM__0+nQt2F$yXx1zXJu~m0pRGHq`fD_89Db~~79^PDZh;L+44J7nsauzk zmGP7{&#GGKAJELQ&uZ9&a3G`+q+M|&Su4HQ_(}hh_Gh?jaYkB3hi$9w9Ca6Uv{IK6 zd!Frl**xnJ{L#T7G1o3v051kF8&3;?jR23y<@tn}gpZT6%dpGRzMEOZcJeq&ud()z z7S|e=bsjZtU6(>0XznowTSxCLqn)mg3#*P(3EMoQE`C;N2O69oIji5dzOQU<{FM5s z?g23fi#)74*_`mfO*#2gj@>z0cunb##k-EBA6* zYqQs`fQ3MsXOow_CwCir8}ifeqnr27$IQ1j$qU1}!%5Fk&qgnz&nH@$ciVLLt}$$kJLGJrM>OLbqLN0KM?l0L_G8B0qE&iQiqu-NJoDMBXr}%ByB#(;}`}%r#e8=>}trrmXNXyu0V;dF&(Xojj!W zsbrg|5yPGMdG>fwRq1K>bU+Dop@kf}!a=evuh+z7k+`sU`$&hGy;}Y|Z?BFZvL~`9ybpN~SI@VQT6HyS z7qMblzh_6ca_!P=|LG`a=Mk_@n5a3%gSQ2tE@+^NOyXeXLE^YHsfik0Ik?7a5JMH20!jy24Vy0oH z1^KE9<%Hp|&JN9oPoP zd)kNnKr7U_riwoOc6ahCi5vOl5wm(nyVfRMw}*?2#*eTchmK2D%3AE~+qaR^1h1W& zU5LF032MtvR&iZ)@dq_e-?XoHeA%B zs3@rTBywM~T{be8XlJ-`jI@o26vkx~b*KU2k2%s0vU=qTQak`MzIqq&R?qu64pYqA zG=cOqtLia=n$8wvy8Gp3SK#)e6S!~d`?WLp{jvfu6mdwBQSj{1(QcRBa!Ote8FfK)J!pFl_vW}+!=DlZR03mn6MK!DRJE0DQl`Vt~(Uoj>2;UxGlOfnR9A$1NM| zUuiJBZ18`_5I`gV0wSy;DlH9ss~9_)nAkd*+d2DcXpRFXVC|)}oIpUZ-n@Q5rIjg8 zf&0%{sA@WE%FFQ>+u1N0ey}q#VRW;xf4vU~pBoQwXk+4RNaAK=ZR^D2#!vRU1P^fh zdYXxh5@gpmvoue5O3pY166Z2c9w{IDM5)4l6w$6rb47N_>e{S;cK4KlG#zMrNk}+#9IM_j;D+orRl; zwU(HL4KQaw8v-mWEF65lEBt>h{ZEtsRaNtUs=7so8`aqfpAv{L|RN()eZD81M(3={G|^X9-ZDAl2_?a{av-cB=MD9PIj*E zFWApvy3|l%2IQ{c)CLC7lwo)Ce!m)D?hr=ba<;buuZ-4==CRXzhxaNcDa|K4rfsZq zv@ycWF(9yi-@f2rU#Zy$-kE%%5CZ%24g(2<;syIp0ah>!ywUNuhG-<9f8KD2*ME?Sm3wAffLzFoUljlgAK zz^q^KZpLA^R0UdcU2>4D9%5|XRbesefjL|CM-JM#H^}n3MtyJFSRjDu4~>kaC~)@y zD2l|riBi4naj>W2aKBd+uA%S64&0S0(=B&*zB*3;vybRg`R&TfGd9qHZ=*c2wRS6d zSz*hqpTh0hpE!V`NxH5n$yODOmnHMY(fmCtJ}-noQGDA5VL8F4%l*rpEFVV0{Hzq4 z?+Y@?-p_YC>s8%IQ}x|*kXV{RI&N!$Yj>LiapLXXPhC!4Rl{$GLj1s?_f^ii;F&_4 zM!BhgCdw+?nz-(F^R7y8IBnA#*2Cz4qT>a!*bW__(7P%QbGjcx(`=h)fTFyIRsQof z^}jAl8jhO|dc1Cay>`6ydQsN;+X%-!;OOiXtXKCD!H-=DX9;-qNZ{M#EB2wmevuinZ06trXRa|zCy{WO8>EaW57 zUITuQexHD}o3&fyD)b^II0O1b__U0)d$N1Ov-;$J6H9pd-1#5{?PXy(uSRp7~8M7to(U=K#n^O*D4 zga$>`4~Y)52n}uwUYvai1NrN~52+7JkfaZ5yg7z6;1K;O1EK!wFUSyw zmCKotZ5TBIVQ`4zKT75;Nwy@H&QA-tNCdoa>3>?!4p3X3RxF80e-#8D8K7*GjH`?P zs;*Uh(38L*yACbAe`?R~5SHs=qQ9+JF7}^N>EHxZV_O~0knCS2M1%@#*~=gW4{)mJ z#`Cp|M z4g1dAZOM(E=l@>7jfw425iw1W;n3kz)h^I~BJ^8er}QN0d((J79C5O|JYIxrnWpq^ zECQEf5dB&XYgx_4t(loI(MbFxNPP>OnJ-uZ~%n2q!6`=zl-vU=SAf0epzLY^OQUlmbN)5 z%TMcX1q$J)(j->E1Zkn0cmTbMh0yc9GXYvL6I-^LDvV6FYrE@#!nY>j_~onudKN40 zwqH`zBaW?`|0NaJti54$71TT~`xG2Y_TBVfD!C)I1EKam-u(I)ezW4!o==a0|LM4~ zNYkcnMyBE-0AgR*S0s;UI)w_BjTDrDO}ozWO`?wDKum|e&r?AmJD>%zT@XKS)J`d& zqY&2xZA1&|;jRJ0)4x3Wso}Lz$v#5AQG|&nj1qEC(X6oK^YT>6WdK!(gL@BXEGmK9 zRd5%tpEmVspUH>bYQ)Xr+qIC|wqB`EROs?fG=Ds?%r(8YRvL`8DBzPBP8q+w7!$+{ zDsM}1>xmbC1BwBbXaXPqo%>o zm^C*rr6|b2*m5dT-A@)C;xxw3S!w-#QzhQ9mRd7|Y15=A12(dyMeATmVK@^)iiFfp zMg)97$U5D9JB3ypXOeq6ncQ%nf!M!e{?ke8Ok_Ayo2FNp?VYMuWj-q2DSPKGf`qT@ zPaDqaZX{kY^>tvuu}&4WEK3TeVU#vSqnNOIJFp7~@NRC;HhO(Q!6Eb>ZcdmKxVJF8 z=;r2H4%q@FDn!8$sItmEY2}QvyzfgI01es8d(uMAyfxY8 z^{M&?o)wH-u7&4AgMCM`55aW6`vLHq@|8=tC zFyCg3s7#^jm(7xk9|wS-R6bHJdq3EKQjGw^l%%ijK{I+ds>gO3VNbd%W)A%E8J>}v z$n#WKjgxDHv1vDVUGU{DdUrfWRGJ_{U5Kl54LuKQOMP4}lTYjNU`{d8L&s%C&1hE3 zj6r$_kRdE&zEfW}KqwRP=+EDqrkG6chVY~DTTD{0%lq%)!lVTgduz6w57JqtnjSaq zsOO0nXFc1{$?cIK-R6FM_ai}FTY4)R6-VwWc@5Z1YPq|4(E`d_V$EF|&}0J$6+?#G zTl+YyXq3ryMYHVS>-MqTJykT#t_zV`1>= zylddTYS=gfLU=IAJma8kr0`5F_FMs#k&mfX6*QTjj=7(bxz2jf#rvWNXWS?GTsDEZ zS4{WVFIj`b#?fqDEO0rPwS+Tk+)A8^4i*8IxB?bRb*xJ>P0$sxx(b-|Acr6XTrwh` zM~7;_I);fN^RuT2(TvB9wK%w`(ns;O@ZHA<_1}-=lK*RNafEviCUw zNs;d?0F9mb5k+T1QEYB0|j2g_;b}OO-s3A)Vurl>g?wxe| z01*Fij?442rB^~v1RQaXqMuXCR}kve%7B3G`Q!+o zyRLZMNRdo7Kq=D~o8^NW4zLzwc{epN7KzDJY=|+QsNUqSh>)}!2H$L5`Zq<5kv8^; zx#<%3Pk^}S-v`9^Ox=`?pu6vm@$(8cYDkbg(M)@B%jAB8amLM8vQvWM=FiDYTG{X3 zaj@?}l`KAR^vK=xm_)ZYRrTZ!8z&1RkcP?c)VMimK508`BM?@a+x)U16crRO=K!+p z$R_=(9hhgCCO(g|dBc)M-vZ#W_L!#G6dqGOA(yj|LA+o^l$)!Ydfw_A zuys0<&5*;MVo!`VnG%#^b@fZkRp?@ZdokIHfwz$+7j|8uNhmYdeSnff&WXUm>>9*` zz}{lMaCXqKr4WUKOUO8?^b2nYcB>H8b6rdsw?_=w145)*aUuA~2IZ6Xm*+v)aZf0u z&&)l{C;RuM-Bws8%`}1y2HP#MdkQMtQyB;^FGGPPrI0!DuE$f*9&GJjaFI0So9cA7hq5Fxk8G+kh(1rzpd6|S1b039t|4B6m| z9B+8OKbS#Vp(s-zp=Z_wm(<|W$>l?#)l~XCSL|z2DM{!Op9~n88A}uma#)WnWUh4) zgCCG`b%wMf*g3HhT=}8sedX_NK2SNMZC2_3+n|0jquGmh=ZD|M7(pqEVB=M?(-m#k zn6=KcUA|1?Uf%6w)f+(xFY>*oxIBS8!hqf3r4j1tt7SD;#oe6tE9G355{ze#BCt`K z+nSd4H5-I%B$MQa-g*C&j)6)lyH|vYd$nMer8d@2`f}Rl+7&;f(QUQum;P4SVUs%0 zoqPN$$`LBzQ&2INW22RHdILK4R>%<35x>RedoFKEhw~m*C5NfRJmnU^fNNx^TEZ?< zfui*X2))T6^#TWLTvEU#=I%n>rpcfO$5F%4X*^_Uu_cmvi4fx9Nlp5!9Uzg)wM$a1!yvasT$`e&FaUi4B+-wpbOc>Q+Q&Yx!ZfBiAczdEkH@_8UF5isPGe)zAbgh`kJ|p`G+knv zQ6u%Sq4Zap#Z!;~q``$UjY^^W-uRvwmqa>knFt?kaAZ)wmMVIzBSYwrw@XP#rEl^$ z+>j7RnYX;j4B0e*?b5$1RM7z&D8UYxvC3X32>40ISG|_drZAThEn$oAhha$;7Dx(; zMlDG)Dk<96X{rIZ6Ag>F==fZFUwf;;x3tVM-o!v91{sYK;ajaG$y9#<8{()I0hcv) zDejd7x5yt*c_1Va7z`dMTfG30)QETZS~shF<-YtbnX76)W|kA|Y$DS> zA}|{9yV1$)*R)-dqyaJmi?`|~*)8*!7-?~9H#VzN=YZc(Y&CC#@jj2dfi_sC1@>iV zNI}J%&2YykK{}fV;tLF7ysRz7o2pAxUU#Dxh}LjQ1PX5oIa{~_Z&8S`E@70PT6V2h zvql*%vz-OZ)3j_c@9NyikO-L1(%zupFh9a*(uTb?Du6*~M5=>9%m<~?J|uzb47?}v z#r{qo+}~jr7Y}bh!^w-TQYZ&Y+<6fK?Qa*Pjlj2Vl!a(b7iI7g6iKl6u6;Y4VZK@n z7qo>WeN!sregxY^>3OGyfG672qhQ>;aMn~8z_3cmd{?$Qg48uLas#E*VDsG?7Wu0E zu)jb2Nc@H4FrJ4)Q&WKh`5DSxxJM{^53!S0A|5$g&yh;yYQv7YEF6S|^rPh0elT(Z z6geD3$4)v6ZXIT7qts=1MDF^+hA14C+z?MK%D>^Exh0tUf`yq1mHaP{a=yRuUyLsh zsCGvE-+``1EvzumOb8&L52P=Iz?lK1)|SatErY{*d3H|wa)UdpF*H8 zuOSDwKb9|P_`*axo6(`C ze;pS_kguHxITP>EFm5d~gT9Gy?RYYulih^O55$PuZTA=1NPJ8r=*yIQUtD61+|dWM z*khp-+s1S80)3m5yg}>)efGUvolQ`(IRG;jC z&Q+tkqdlqr@-mo#y!-2ITLiUAx1@r_J?$^Jy}d`MOB-AtZ4ZX26Hc*Z`FDExf>m}y zx28`c=QsFh`aZgPBWASGANGoNl(@arj(hkDX9c1vx8tHDYc8iQiqpMO?}PV0?mwwF zKAk>GkC6*_CD?Y|+)UrdnzUKm_puQTOez9rpZ48*F61BXSR#Md3sDpd>x<+s@)pe> zANMM`Oc2S74;~i*C&yDFJ*tl_t{t|jm=q_fn5>}GjqiSR4Z3J}C%-t8AI_&+pex@* zEy=!BL3w03$pD8;JT%M!;WznCo|i*8Ydb{3La=Hq9X9f$?LlFSeJ9;%(H-kv2R^c% zlnuc!95lo-cA)m@TU6 zvHfH@q(j5+)t>6Cr2fPp*%wVG=x)xV;-ZyzG42IkS7G)dHB}TBkA2c-IY}99@kA`y zc6zF24r~b;Hkk((E*GPF^HqPNee5S6FP0DKjyvG@{k*pj%ECm&+-B~%WH>wL$}gk| zQz~tU=UP`m{&v9%l_K|+qqEIga-#UMPceE2B)Cs7gMNGINs4$U6e; z70Ktw9re<_9P4F``d+qYC8@!ff~8L}K9)qVlXjQSEHKm$PPC^kv*Aq*i8jB+-aCJ@ye~a&nfjhAK8 zbDKE_U-k1QKh6kJv$0+60vKa>I2N8x;RnApn%{a&hyqmqDCkCo3Yy3*+cIW|shzLJ zFwA3O4oO8dlOFrE3>yzn)gd#i*CzKdaW{m-9&nFP~l`d4wi>f?9zU~P+$0F?I)^%d3m zcI?yUnq1D!%>P?0`h9@#tiuP)2OMG<@!g_fx#em6Z#7Cn1yn%*x3(#68Fq}ckn6@fHDhpA=KEcQ=3xACMT_t-QLT3>%3A1S2UYe8A2Afa<9)W@0 zs79lngpY*B!*!u(GOy*2J_po00IpLI0B?Gg^t2$;E4F@AX#N($FQ?OCLEr(kFy|5S&LSJ3;3}yyexPXP2 z)aFGfl(jAzfQdk{24IQmYn;lhf6gslx%H}=dx;{ciU-9^@AT`l^6&q zEeG?zXi`(L(e@7IR>=APnLSsqy7|aOC&et0e~N{je|sg!7G~>{aH=JN!MiR6@(CkkjMrD_(-nU972F8 zg`Z^h?bibUNcB`1cJ~0_Ba^`6sc7};XZSDx71X5%kgY&bKgTB^oG8_Kbt27wez#d0 zoR@-ZS_{JO`Dxewwe&LU{6X!~BtWsQ1;obpV8Jr6*rH zt_R50&;ki1hBu5ygEoK@xb6o{^e(BgaT{-<_0z`()XX>`@z;L#1`rAY{yvU<bh_xI)$DPO8DO6IsFP!{X$^jSVV4xUU%5;GtP+!}xyr>QX`!cJPx zRb=2zGywUU9yD3{ST&B6f;e?^imh_^FpfJQ<8^5v+G@1y{_=E#4M2Taw+|=nYQIdA z4TJ3S`^(E00Bs@nmaig6NV{1{o`^BCVBB7qT{zW(H-drfi-Mhyc~2+;#zJPIg;-8b znrS%0_jfDI@9QiJmdp(y4~ce762h$730Xcfrm` zt;3;gj9rIN8%tOFnBWUv%`SjrOWqCx;0*DdTphM#BpzmABLJ^y;I}f4jlCVBH%#ed zw2&I)w(w}2Y?EW4F!eFH_N=| zrw5G;?)BX<;`}Yz{5t%KL6ur(@d-H9lXF0#q4m8qL&x3Do|4{{!A-smggQTeL&5Hw zCmD?-hyvtT(Jd(Se4efh+MllB69K%)0B8_-6VA_|vh^A+)9ZQxjbQSHY#zvafbSXL z71Cr7&#>#X1L*$rg(KUW7RsAPC@hkr>EjB;S)P9hr z2*Rqe)s^Jw9T;~FS~-vv?F^;L!~wuY&h5PYc`Q5tmbC+toV*Z@a8d~tK;9AlM-<^t zGCk!{Q?X5CqI;%NzA^qA9XEZXZ>bYAEQ>bHN8jN$;W9xY<^c3nh$jWg@v^x0Ta#|g z3IJ+@bMK?|qlOqPyYHw~+O?cXBx7fW(vAV!jYxpJNUDo(yl^r5@cU2OwJRCu@8dBF z-JJkdTp~84<1|vRDD$kR^1)}?q8-@qpo5>iX$VvOG1WCm=K-Pc2a++8g39L}fP@6P zGysZ6=6LPH`R2fyS2LrdiJY4v4L8_A2E~k0gyUX(Q#=^1X5E5z>Y^I z0=)h&J7J!=KULe+$vNa={ccg{D@3%L*Gl=sWs^j@En{m5e_-Kt|3Io&R`3a{#nUwh z3?pk!PE7YbXj@LIVOIcb=P>ISfk2+1+bPdR1S=Gf`?S4nNUsR@ROsX0FV36(=EyE` zo33+*Rt}^W*0?j-f$;n8;B$w$u;#Akkp_-w60|rUfCp>K*fMb%A2!E z7(|MyS#7Jpy%o2O$Tt@2Vha2LH=k~fbsb-TWCZ-lf=e9!6{PWx2pV_23IVvRlgWZ` zG(8@oo+RfbRU;Yw=Icc(+4T*GAmx>$(e5=b7(y>yg}IUg6k4X%OXX9*l?WJHfG!-L zw=#|RK~u7@)b z>2*U7peyp_tgog00?=AqYe3@Gp2V@V#B{q|p5&)6@b3NlJ)fR#R@6LZwxgw(Q)L{kSL_W@p1QkjZaDC zTi)Y_yr@#WGeo`ns(>4oFTTF9)ET3I0m6Pw>Ag85)nUmEK&9bL76G>~U_&Hl!--8c zj^gfeE&{Uts{k#6E;K5ZEy7^8g>%t74}=VBE7PJdrR(6k(2$JzuICki;njj( zLP?O+Bnvns*yb_!OTNpADuH4(*wygi8btis^*-5O@ThjG>l46wFmywPPVi*2 z&haZr3?}lRzFi9@{+#9W%q>SJ|D239Zjz{w^k%PP+uI~i6 z%4&x5r1ZVI0X(87d7l*4ezCi8se}_ac?dk=rsK>hQBNuo4(6nTTuxr)o`zgKKft5l zN%dd{^0F{dJS28|Qh0?nc&7GU#gPi1#6N1NB+497ll8!V6cMnJnz1-%FU;2ry;3A8=47I_2!HW;8T@ zD6O#S%NrI+X4$`VfFmPJ9nDqh<3Q%onerR@t_?X2G1Nm~1M(h9qUpYsFk>DUWetZ| z8%!wIjj)1FIJ}1#k4>)W%3Q?6yqG7dF0uhA;VJ3F7LjaAq9i>Jivj)?LFe{n0D_i1 z#=N5l^_4o0DC3zm;?zl?k#54klcwf&D@;T}XTN_p#1_nN0v4+z>w#piCik`kw5NVw zMUkKVakd;iXvOz=JU}IXX|1}KIOx|EYoW0@y9jt5k`k%SGhpeG937{(77hFn!OAUf zH{%p@YhWu=M1b)8D=`Ro1|XEF-VBjq;H1B@YSw_kkBy_5k7NLsLnaefN;mDsw@N}3!Y zjJFHf^xNhMcP&hPrtRa0XJH2pH|NFr46fwS=LtjpB6$S3eREbv%u3`a7Cpl8{r!_C z-7G(t-G72~X&R_OKM&`a-I|Dbp%8iO=Bx!5-mNJgyKVNz8ZK+WL~5T0e-u<#O*ZLe z<{=|JuDed@FTq=(EpkWGV&WNoWX11<9!=27FB zHIb%eBhhhA-6h9i=wOR&3wj*!-NW@7(+M>anEGZmc8ZJRK?uOo5h1&^@;RVpGG0$# zWEm$05?XS-u2@&giZXTJ5#{uA%6aUb-~_2)IOWtNB0J(S+Ony(AOb7$?!wG2;Oc<* zZ_r{#G{+EqV6SBsSo~7ISeTFi3ITjKN}2KxTNknDKbY~~o15qPLmRACos0Nzy=EW{ z`Mi7+K<>T2XgJ;S!sjT-J{N=UM;Cmv->)5twoBWV0p-(2&hhgcnIX{B16WBPgSP4< zg@zN5T{Ch=uQ1*z=lGo`)A_D0S>c&Q7$k!R>Fro4C?$_2ubhuyVp{|T4OsPrpqi?$ zv3OEBH!|16C5?|mcEBJO_R3Ki=@P-HA7STZ-E6Uk&iAW)@O=+NUV4YE*m$>;A%ZI+ zScHY?^W!SSIA58?p`#qWrgw~=c^ZeF|A0kY>T?c#KJ8w{ybfLgGNstded(N!)a%c~ zx4&0R^2ymZ)^CG+->QPg54FRah-X^Xygx%rbQH%XwjC2KGfjYkno8n>bDE(gO$EO_ z@{|>U`-%KwK#<$?PRROmHy>wDA+dBd4>iviW+Zg_%{5$zo~CZ@W(SZ8%xu4@|MYo7DjIn(oe zfH|~Q1YDt2ld){bRJqM*J%YC`lqB807$hTTb({dIk1hvmSO@)m?#Fb%avFGZ+Wxz* zKaAXq8nuCV#+4>CQ{9{d^)MrcTiD(Z4zfANsJ1(u zs9T0~D%*HR`P);qC7esI3>H2Ati^gAFl@&Zmo93OMTn!2Yj#N7DBK#-)h^o)Dm6y= z>vHD1?mvxGS>0)LTydtby9xF>s-G|1Qz^n6fl<393yc$2-Ab6^ad}Efdi|Uo+rT6z zy-u9df{m=bGp4(wmgM`kBuU^P2K&7{ASgw0n&rG2ywJJUTr|Dd0*(+mPqOUmckOvo zCHGZM!C}{Ob#qT3GF}_R&X2Yk+5ns6L~PKdJjAnv$DE>;WUynX--p=pE&jSmk046S zo&ZkzAxSYSO(oPK07~Ffb-H>H_Ei{PZJaHE%Cf9PN5L;C_0P_VfCKP{u{5>DKnwtB zDs2*1t#PX69sP!(WzX+=rau!hZ$|v$t=9PLjnoD)e~wEI>Dmu?Y0hOM#AihAo2Ye3 zkO47_ta^q4XYx~erkaX?;!6GW!C>{U$?IP$h^&iu(}>C zwHWk5e5?4=E_mB>5z4@Dr7 z3`x`qUy%n|B?B%sP#y7|{l<_jrs*hYB!@6K8)>Ec_pnGMJ7|BDT!S{IRd}@b^GgS$ zOE`X!tTVB!-SPT|BKAG}MG;%5NHRDAmy|H`DnxAMnKtb*-1Mo`vsqwoEjg*#_NR%x zj+kA#dab2NRKOfxZx00;OR8}U(I>!P>|FQ zVL85cBGo8|Y3!2fU)e-)Oo+XMiI&c`hcM?{PYV#~UhhUS2K@*5(t|EWYVv*lU(X%~ z1T8sBZ0SUMA+47Y_#_chW+I>bNkYZ{Yd+F^b8AFJ&!j6BZrNn=J;dasR(BltmDn&y42f6jUmkiedhQmPH1^*{8*N6g) z^qu+UB)_)lx>c}+7P=mgGzOaK_XlLI`%}&v7fJGm`edkDcQ%Z0vi(#|6W%pB(hKzw=|UAV(6O zWnB0ZZU9!MHVRM(-8^C8T$2b34G_E5U%9tJov(88mkzsJu5}{_v^PtTO!Nm|fBOg2 zW(y3Oi3Y}~&h94jJ5d;e4y->^po9Wp&*+aiB;z^%onFdQ0JaQ;MW+K{1)Nj(l>Ho=By zdpPE0F(1zklV)n%{9**?Hng@UxEd{%ZxzL|08woCCm$u~$v+9d0($l;;VNj%kteTE z=QaRtG8X_uQ#G##AHZJ{LxC^?kTvwyPd8XcKkJdR&h|iDUP$~C_?E}h)m$(8oK9Yt z+i53sIKZ;4CF(WF1QwYESYo5q{`db$KxCju(Xy?wEmrCass+6!NR3`O1APDvknXlu zkQ6^fUGZ}b0PQfkgON3o>H%;UjW`-e{)Pg)r1!7P_*{S_XLLAU4fga5#9+$gRI*I1 zwetX^^-3>R{c;E(155(Xx5m|rmSrV^Ty91l&~vYd@Ne4c*ZO7c0z-+%R{}GDP!=!( zTz^x5E}sZvSHuvmuo|Z*fEq3M+;Dr^jSQ*DP&+OHrlNFE{w?Y?FzkILeC|)?h(Z$v zR#L=p13T9(|NlL(=i3cr2Y{47ryoh^Ixk$_{(N7`hb(ZLjRpfxU**fS9zw1iKY>Bb z|LGb72-$RQhheGGNr146$;20U=s}B}9ew%n#-I}t8v_W*i_eiKZb+~w6*KA8^}Sp$ zcYb|bnW^uZoBjGe9N?ttz>;{OifjPDvP#jXqq?Nm#~|9?OeR9YwmhB>8BG@}f#nV& zW$?Htn>=kL>5JVQ|4O*6%39-w{UGk^N}id|n&W531M`ZX{AU|YX7>WI1*Lo{MwHwk zFy&x*h%j0J?*Ux68bI>H06#*#gK87N`8CMjG}K>=@`eM;jvh>Fp5uoeVPIV|Kxq_+ zss1j=d%Y*r@&hOs6gko20Kyn0DxO1_a|$Es|*F9n{K zAo!x{Cn6fRT1NLb^8_Gog60xY1bd}68TkiR!46>lp*-XNSr|TYuP94T=rUP_L<&TO9YrEJ ziW{J(%1)NEF!0`(yRG~FF|YDD zhAuq%yb)@MnONz*G;Hd+E^2->D1-)^0Hsp)4A`lq6Cg|7gBVV*)>tClmv`gt8&`#C zw#s>UnhHPb?N&|`vGAx*Y@3g;^yZ+)_%0$Ej=ukl$KiRpoa)AWD=ooR$}l8Nl|jM$ zb-K~XYzsSgC|jQ;BexeRFsQIgL*q5=hJjb%Ar|NiXcx@V-k*@<25^xLxnBYL_HXJj z>nNA;fH4rU=(;Y5zoL+D_SQcDIR82TrlA5jdpn-^1hRP!C^6z;uwgL35`V7)gxLFz zz%xAEN41>SF|?Zx%wRK=sA8mXu@?g44%l>V=LwQz~u~96iqZIS@q86Bn6n}%y3a~w}h#wH_PIE+qOWz&>0I*SlnmP@QA}c&Vpm=2; zhxGy6Ow}66kRTveB7Mq^{7=-(#`*B(#EIHmaj@x=3wgm30-zj&+QA=*PGu|Ry$QRi$V22pm-@xcIj zR~5ke-d1&M3c2kE2myYvE)z5|Q8~xKB0x9n1)k@@NQ8V)UjhK%eFA)|1wqy8ANY15 zsN@pG4_h3E=W6EJ%D1r`he@EpYSx;j@C@biohLmc_A{Yf9WMlE*}p=9WpfPA41=qE zPp4sD^)RleoRF$PdkFA!le$fO-~m8MLIo0EQD~&|kScamQD1vm;2Z%}UAC3w_FXcX z$mnh>sV@i4gHs_Z@CN`g(7w|9)rhBw)4~l+kklIt`jj695E9$T-kDP)JdTmRD!c{& zZ~RUxp$U-D*Ja;TH6JoVTE##yC;(A=2IFh4pbnhU^>hi4a1pY3Z+u>Nvelx^>J_N# zm5z8Yj|X=!eSnOF<+p0!Apy~jByd0v7_Ryt{U4^B z@Q*3aeE-T$mwDwfhZJRc$l$6Reh$rwkn|-6dt{@OOU`Z^*uZ<_s>C_~V7&WZ`D2=`qktQr z=Kuc?_uk=Lzwi5ac#^$W_THPadD@%ok!&(jkwn>hXJEu-iQ2{Jp&V?i z2zmxPL!k8O3f_YrP36H}isWGuJ{I27lho?kI6EJ*UsaqJoaPFRVh?fSJiKpZ(*AG+ zdW|*OKK8M{3B?CUcWED{pt(*KbhbqBZY*SDl@ECxlSA52n>r1E%2ilgX52@&jxOSDe{WrJb3aVd{v=0Jak6FwB z#I{WD{74m^pe^d3XEfV6PqJLBTwK-_W<~E0eU_yg9?`E-F-F-im~q>=u@q?#0yBS^ zdBeb7q@`4Z!!8lL7T3wB$X>^}@u{wM%(EWdlXrN-NP(KpMX5lcr2Ny}a!i6@d`ALqC^H#Bx2 zvm6XvJ;^LN*#DiZ-+d>Ob&{`2lbz(NV@m6nA(ir#_tQT~@<>iLABEq=YC@0$QB%!9 zW&@dQs5S0!apAqnBq+Z1xT~=m?;(;*p2AyHg6wtf`4Fv$U5NV(H2XdezsX*jgE1pK zMZtEQ1eeF1L4N@A%FNZ*(jJY}OuTkAJ=|2DN!qvpUOi6@{$o=p52EP6~zNF-c zc&7)tL_$K%i{$mY?=D17_(Lk6Oms4~=QeC_@WBXF^1sH_Q<_bboF@HvW*^A_ZfmF0+_715No<5P=ULjpFRZme&3 zmqo0Od>JxD8u0;XDDP!Tr96n_If0DCL9hLDK@sf2g7 zCulxqEvIB53Ep{N2RX^wB%McfVT7*V!b~_wZ4E5We!VWHuE<7%W{pKM5k4;!<1CU9f|~na7$B&znOH zA(Om*f^|ek$Z_5W*6Eg@>c25o7W2KJPdL#zQbdu6&^cbqII5cxzA5mQQPIs7Qxkx` zv;)vm41<`wtvI}U;(iP!8*^)qy?2AM;Lfq44YIT1ED5*N?Tz=)m`5?7q0$N9G+LLE z9(3mQ)8i^a{K{yweZAfXw2PM+1^tK^4j)On5 zj=lR{vsDX|>F?$aO*s8^kLdc!j!`boO~|-oUIhM%>!H4&8;f=OjJuTpDI+QQ9!|t3 zwfcUZuzZ;huEmJ}|Fe%h*8a2m_0t8-=<{dS*zT$E%fI=>ao+DZp^J!WFy*#ric*}x zd>$PLGTaFcgyc|8ML)8VdzXT--z^o5A7uxHCw_kL(LS|TNn5n`g@fRTpEdzoYBxNy zoip)Df;FHu)Q(Rp-WMok!qAM73muLvMsCM6@U@ zbsDOA8$U+p3m_ac&8w}&hhGxfN&+>f!LYf6t z$T?Pw%c2T4N2y;hPBXA1$^Uq7g4+DZ1B8cyJ<#fnlfQ#K+F*kxPOx*XJ@%gRR>KZl z-aShp4`lb;hst&QJPdMka(o)HV-I^BCJgp&%uYDoK&p6bUzQ%?BBL0d0uo-ah@%hg zx_=n_7gG+rUZkQ6k6%4U#*0hhjno3+xF{eDd&)OSpIw6t`WchlRUyp1K(a$(yr#n- z$iubGqD*w=p;{-uwjBT;Z3b3(5(x&z!c}~nKoSK8zqd0GRg}MSR0=)u{Ah+~+GoYl zNk%IgUFK2|^O6>IG|^L9CW3(%AsX>FA7diuT*@3V4)>2&5E$-qj-D zy}ba797IjX2x2%Pi;36xpZ@F1dqW6LVe+PQ)FQwz&I`aN+KqN2Bwq*-KHk59RL@T5 z@896^i3|!8%j%**ViCdc zlnPS}{#9h?5?e{|@FZ_~`Zum4>D)}?6>os+o)&hPoX|!2$Y9^){&&m!OX|#oK@woPN$=Zp z>B1Cb{5_uZnF+{~#AyPK@WM1N(Nzw3QhcA!zO!I|fx>>)Pi$O=WAAmG1I_+3C_rt$ z`mXv{0R#>YVqM{2OrSmG9#D+}B~1(2k_Qz39PZPENdv%XqIdkIBj{yI`U!d|&$bCr zcsvFcP7DORI2PQ_RAG?Y&UaH7P+n~D!F#xcBwT(cLwRFO2R1L6IftgRhgf!< z1<_~^19(#pa0?<>SCiWvGoFWVXkK9panu2rbLFCCD=P+yzZ;)IAnGoP7vg)P@N0b~ z+7INPESZo^g)6f+S$_HiU)xjm&h(39$4Yp8*$oavd|nTMFdnq!8Us=b34F3svas#< zfCi2YaxI}PpyI^=3Zd}52%(D1-w!7yN02L8`CC5AtKv%Jt~e)?DzmE(h0eWeq95sS zTmKyWav0Tbu$+ei zmhxfic3hU6|04t}?S@DpeCy{|L265G5iK@0shp(sf-!w1CYv9T1~agadi2ml&*9N|DHe=+`&m;LK>UdT{t&+j9M z5Q5B;LVEQG#1%C6+}LBciU^LQ0b)mRo+dF{BL}_-x%VP>0`LMLC3wB3BA}ZkqI70h zn-d}TL{JOXTzEpA6}As3iD_f)#b@|?w{^j$U>-6x^S9Y=%KA>v)js=5&R^4*D$~3D zB%DOSqCJKk?S%GYvEzU_t|fz*-Xt^er{jqe0MzDeI1|9)#Nnh%{s!YK`1ScuoBA;$ z9Xwn?aXWlBFwdZ{2n=t8QF1}b(sB(XT`A2RSS_BgxveMv2Ogv9l1o9LZzz>4?&kF~ zq7D%j5Ff^+|FC1+1z44DD1%vo5GNGiMV5AEEWEMY5QZDLi@4}I$SAq^UIXo}HXr%2 zv6{m;5vv6(>EGlD4ANrBQ5QeaD+KsySs`@O79kBHM5ar3h)yCd|MmOVPZEXwMT*@4 zs42+S0U%myVQ|JP$--Nhr*$iw@`LQd97|fPaN77-rDY#mFi%{hGz5EOZ&J_^nw+N}JZ-Tu{diNY2RNhG4 zNcFvq5C_AT77K^ui~1}gRaXxJ>k>kU$~~$}h_6aUsQhx4qm|r4B(*GFw8%s7*seU48 z65Y8e&w6WJ%W9)W%Y+^Jp26qGCBY%TriY-Tb>fArcL%Y7Syk7y_s${F+(w#;R5KXg zSyL20Spuc%Gg0+8>rObWoMsrK4nuK&DWuLs-*l#dr(EX$#&*H-31GW$EsJ#}w?M-dR zazjzpNUeX;XF?i1_AcdXAM-q_=+3Y^X`<;avX*gn!$4L37z-%~UA;?smnu&Yk~{My z7Q9RS0A#9zdeL(Ke7X+%t4azfXE2%wp9c`!D2V3T*L3Eod7y)mj?Bgbz#f?!w*iou zJ4teyzKqt+`iBNJuekzN4MFY9(asJZK&wp@&H!l;9WV9t6tP7l7PtBD2Ar~czBwVr z`~ixK5^;zOevFSvDncw7o&R^&2Z4a(!EtMEOa)V9+M5HdX-iNK-Z?zQlcjpOmF{e9 zTUDlv3oq}`c(-_{scq+Zgdzd&0 zIgYM=KJjLuuqvk^KmGyMF{sdi0VdpRKxP#oxhOl7`qPXZIwDRB)7Ds^xP66 zBO%W97TB1qA;aV`)Qlo=r|RvKKx{*h8xI;nOLyt2AhR}gJfe#@>-qpWmaS>9+tyE= zjU#v+XDU$Mu;XUvb@PKWjTh|Z>ZM)r9LWeO1IqfY>^o71=kYf`U)a1+J1F@^;3_Z1 z^G*~$=g{c}!p*_tew7SGm6;cTI;lzDhoyi}W|HT9L{0D6n9_2GN=SymtD9_%ZeBf- zhnYy4vv|JmxisdU#~7W@z1cQaY|WZiq6&B>9RHT|7ym*gOk^QjkqzvbsjuaEj4~Li zlwN+kT;>(c9`FJ{UNn5i@lF0AOxRZZ!zuZ&fswOqv~ zYJM2i67@(`+Tryu$&Hx-66d?c0&53Q?NTq>GxhzJ&>l$}JzH5jQq~n9AeUJz$R{r? zMrpHvaRQia+PLOH5{_{cKmf67MyjFJ7|#x`V&^69rl*2?kJ{P1MJ~(NPhaajw5k^B zcw`pPHG`zlsp&6>)?|K6w`&uL_g%yqmv64R6tYwzV@DM;<)`zh!90+Ks5!Tyh5P0y z!83*E-#gl5xBdBl{6t&Dco1-|*Ruwq$AZ{yW!Fqq^AFhS-(4xHFmI-Jd#a8760*yj ze?c9I{{4q?quV!X1r5{(_~|h>i<5H=D?SMz)ZbyHqgx)NSET&abz$D8dR_yt-Tulb z$*$ZsyJwA}B$7F#OFs{h|0LhO8KL#s5Rdh4bLB;?`V48`Gl7uRkIw*$yK)48x3lIS z?IH<%=`%W2baN!M`t;gRUkkin^D1Alp$=$_2W25$+CrO=v@dLx`RzuS8@QWs07!%rTOi|NKFvHB#h56L zlmavJvUJdt*UQK%le$!_x9ok1%_g_r$1z}3>6uwNQ!P13JN1!1PLwvwnf<9gz*1(M zE|Sd7x+WGuk|)k8!=hbRiKF8($kKL$Z|+pIMGPnEvRaaq1z-F||Hd__UaKBFM7WaV z>`(77m!+{wt^L$b7Dg~gBtB&QgDX}%K`ka_W42+1F)MUwDqFX&;H#hAyubw^GdSKb z1d;B|J~0PD#zS?j`4O>i_8i#1uGlr=uv1wSvPrTU_uJPCG+CpANa?*VnVj`D`YIjB z$nOXO3Hi_z)@tog76+ElR%C$EKoFXLYmGOf;mC#3Her4WBvSep>NtYPLpolj6QSZ6 z7tI#ji-m8u^aLe+=9oqL$bS`no`F|-pM!KOO%#Vq1G56x4)WrF&N5{oFVi3j+E?wu zeKtPBYOgP07W0E3fEsa7sg$FQMa8NAoBQ6X{@b~~e+5>@-D|%JmugvsQ`<8kqj1$^8bAQ_ocTmL9ls;;IYR zrK%m3JhReE)i{%N8gCKl&qIv?J}27%4nh(8e^Qh{k{Pl<(+QwUj)BglbSO@^ir2)? zoWPrW|KRp{N>!K3I_02M%u9%-jz9{AoCKua3I~Aw|EAtL+uVbF7CI?P-sjA_=C;*aSIo6R}svNAcDv? z=Mg)iH)|49;IP&U{pwP>>YLxD^~SGN3*bqRg9KgOUGH5aN*fPJs-(j+&7v>Ci#jHe9th40bMIz&cpuB_kBeTeZ`{=icv768A)^$g;0|peKW;LOJSss5!ohW3M1hG8X6_OXkqfg4N5|K5q1tIIM``l+k{oy4i8{IG7HjsG!R20db30w zP5goMl2dA(02exc3Wf&T6-6D2VjzR}lBGbv51IM5PVgEW;DqUmMeb^i3luCNsJcNkXJ{3aK@+&l0%-ychR=we2KMW^9YF8feT7Q{3& zyUc;KwQO;B57qYf?JpvLQ1J zsO3Wuqz=)GNHgUO?S*A9Y_aibGC4#G*7nA6JIP&n0r?`D;2w{bUX5wq*F<(Ocq8{4 znUSEvg&Q7|vd~fR8)6Z!N?>ak0Yg^Jmx=Y7?Wf^}3XSCbnwUiT@S7dqn=@Nasb z4#A6pG8_g$bTae-3kD4eguGA>*UXQAB$Fl}3tkLO443o{84%FCuYbI40u8!55pid5 z0wY#-l`P#jR8BfF+vnt}^f0xM=_>5L1P?F||Jgk}Cr|F`jofS+!y=HpFC;{~&}A&#LuxP4B;C(AF(CnguJNk`pQsc1g9 zWJR4QIi4SQ1$M=|IIwEnuScfa_tzueL1YG+KNdWO-Udu8kE(bbDaULCOdUnFF2HQ} z>EN%&hvpkNW@(UROe8sVKjs_MV?imG++ZOo!;e|<0d?*=USrpBJfBkAM{3^x&-aup z3g1{lsF*alD+l~?;#f{K0p3_Nyhh+tadYy`Ft``ZV7D@Y-bzu>A-#}Fzb98d;S+^K zLdf8OjiOkNewr0y=6Gj3HuDR->Y7=AibK|GUrpDe6tw5bBtL#k;h2V2p@1d3vh2L{ zUzS!J8h#hIrd=#KgdU#a%2P2@2%3B_?j{)VcTz(TKVLZy5$d}rx)gXc_9nhBs+h)z z%?B`jD%^|sJEH)D-Gn_LD%i7^JPmDhyu{wz*W!Qu8Hp~)j`7s9D>VhsR`A$#v$Em*}*0i2^6X1qC7Fa7G#JC31K@Yi(r z$tO_<~$FY-1d3_=6JrrFw^3YF&zPwvU zK7!|`>yG}v8z49iqcUH?r^o@GASroN1su@Vfj!}!gMcue1Pk(XmrcxKQAeD!&{jU{ zo?iJ{=-GOwI`Y!+$wrkCy>v)JWBz?c(M#j&TgE|GZhQt7y z#ca4NFnR>(Q*OMn@8 z%o4hc+_cs6ZB8263dvp5AN((S;Gh29RI)Tfpk z*Z6C!KCzq|M=te2LB7V)PW+IfNt^MvC$Bf6e;V9k+FGshp?J|& z_QIC$yWXf`>6T_@5c03)Z`t#&PQJb4cJg-t2ZnN2)}?ppi%#5ij>A{w44dV1_rg%J z#PZ}HA2m&^md~26vXPj(nWqW8whxl+^0zKW{9)AcPb_4LbG*&de9x{@M@HXqcA*bL zyOozR(pMP#k|{c7S@Q%I;DUcn)R0JoAP!3O ztE!2-S$c4!!dF%nK)=|qMs#|H_?}JV&UoH4ftV!Q+SaFv8}uqmJ7otf=?~lP+4iv) z9pR_f-f3li6XFGm-kezTm z8@(C4-+m^OPtSRBsN|PlC0w!{^dX~R75Xh9S6V8w@uwWzTxhTp_@2D!&~LM1@(s9G z8+7)Yx$zbGiw?aLZaEL1egprTDDi^_%l7XWUm6Bf&RGxsqTe?){#wEAI;y}-Iz!xY zI9}qhJ+!sqv+Mu(ch|Bt?{D|ljJpZ~cZ1HZ=E(Ux-}Nv#s6B{&2-zi)a;^0K*f1p^ zn#ttd@6BF9`jp_U44}wN>c0%+lNR3p#_^^0`46j%>p%rw_=mkmN*#@q=J@(2xP$Mx zqLhh8+5a}`PKs%kGbhLfP`lPN0H>>>``at+i#=(Jl@U8!T7W+OmyYWvFhIZwscL(b zSE66F)fLFcDycIwgC@*z4O>F{NYS8D$l#uWUIhB1b9mCi;Cu+2J} zG2*Z-B+o@(s4x$X8%|`p_+!~edv}HQ*lxv}H96x`--EgveIVlq%U6qTzBPp_8gM9K zODIktFIW^PdTxEDr#pj8tSs)b0FD_2r$ z9(SeFzv7V;Gt_cD@ATB|Qt7cA!IG?e{ECxllKbeZf66H{f~Jx0k9}!%@4nYB@Nxj3 z%%^VFAB~caDHxSmJvKj>?j+WNkQam-#8R=dzzX~tJG;dfNu8?2vt--TDaM>CJ4-L zg!x4VoXZAasG0VB{<4cE$J&KPX-v@XgIEA1Z4ZG~nWDA)`Gba zjdYQr(KK{a`S;!i9~o~GuFXKV(+(gNzUCKTt%0+B!C&j}*1h%S&-wNrN={Xpi)wGz zn3RI!yR?8fEqpuVieF2NWP5v-&el$#Kl;(7S}|2e$Qyp$WuvaQQB-OWohRE z8QTnP^$L-$C$={}>mj0j4+FeR9%*E%gtPu;3zP0#EX6a9Nah`d%W=C;Qf9LcMs2py8KE9woMy$01>2uyk5& zFcfPM^_uVQ`@G$1@v8i@CG|b?2V#b`E~lEtrGT22r;r^ zIJ3#~BMxHdB64lEeHO-%s6UoO^ZME%n57ecL)0pvJN`VmN5FHWR){wwm1X(FsuEQ$ z^9JlKI*=}w!q6SyNS<>Nkw-F)H|*#=cfG@-0%)>quMZj+q<-N#w48)FICF}Y(Owsztnj_3Egh- z0mE2EM95b`$`)+Yb!tIbwlgRN_up{Rd4rh#^lkhZpMfpF#+#&*RdfXWPjfB#++B`JVGc6-roN%or}J6wt~}*481Vs@4j}!+gF{EtP@fUu zdu4VVGDp*yXQL&`SU`u3)_a>Tue0V4?T={n2EaGhR9=`rw84l7O9IyF_hV)!ElDrk z#Ln+=vkQ8Onhanm5W2s?ctq}P5Gle$yjh)eS|#=o!nRE2SMB_A@PbzY&NiT&7MAoh zN;T}pJ3PBNS2X%k69V6X~*Ah>SW+T38gWPkT zVOYdcEXer-VTiN~Bozbs!!q4uek{7 zmM!O=!@eutN*roes~K{=bgF^9^$&1Fqqe{gKYz1jN|+s?1WLq6X?I9{DThF`%`Lutn7Va6frm8Wx2 zA`0qCFAmnKM2;W|F=er{FOP-wi}m0oT1^f!JgzsNZ4U-VN8L}ND5bNVg*lfRoO1u# zjTpL?6lN;rS;-{C8xqOt;nGIAf$jyPi^%@|9}%@MHqTo2wNhtP1|h-@GEzU(dDlzMwGAXOC&2HaF#R_jq! zVw_rcmD%w)Prn@-u`g0*cF$g}E7N5WbJu!{QQN`|?tq@sHn-JF}!x z_&w%!PY}Hfk#TpHm#&Jdn0R7~G8pLXkHWo*3_w&%Q4Zu>S_l< z{gm7i?rs{_O|-t+(6?j!gaE*EhD!XymLFfJ6f493v&U`x4e$)!2;c0EGaz z=#nLnzq*BxGUuAFk}yf}J7mN#>PGATzC85>R#I_l@&(w^(c0 z&{#T=eRfvHIevk$NR63D_L|l8$}c3tQ3}07sc7>0l(IfFp2uT)L5~j><~$PBKL(1r z(kbX~tzi{Ym9Dgl)@D%9%H*4xvX2Im1-FE^I zeu7BZ`5CnQDI)aduorJW{w|Opbf`E#*)cWIZJK?4RSNt|WTVi3Xni%*DR!mn@LVf!>uSj^`HgdJ7$u*6R%{d6M)BJuZj=burUa*yxYkQVQc>Dy= zoLVE!C=Bcdel%=cRQlosTtnOid+a#I*LXyucUax`9rA|S-OpS!OBiaW)xb`0NxwcI zVSZH=k5NFUrxAZUbn6cgtU3j(>2APzb?M)k5}9{7Jt}=h zC*aI$iWqYGUD7i=_GZ4t0iM5KEaQ_y^>y(x}qubX&%em+_{=V zBKcu+G)^h=K*)^pa4-Fv{4_2-v8z=Jkl>nc)YyORj=d*$LvCNElG6*csMSBRhJGSK z=1m1N%9Sr7weRu92!Hh2mv(WXBAV~2gP>Xa$B&`a_6pO_p)%PzYJ=nSdyJIVErsR_ zz{bsf3OSX8yTZmpbZt|9$XVv=(*^C3o>#2V4CK?_-q$-WHe#r!ASYO)K0RK-BTei- z9EOXjH+gRXt4?PvoJu!`!tBf{Sz~InA36Du16F^{%S3t_o43D?tv^2;n_{Wkfc#rB zE}qzUV+99WecMcrO~Ot@_mawOROb{wR@Ete`CXHxz5p9cL zS3y*?A1Af^#2|q@Ym{`kL8{l=Dsa#`DfUr&xnqgajHIoor~TKv_5?(X%wc^AmzY%; zltNUo1rpDi>%I396{L8~Y9Xl3*dP4ZZgjtp*ri=^1gaCoCYwD^sk%pyB)BAoAo@Q`@Q|K87*z zp3>Rhp?8L{tDQo{AISG$l+!MsD|03#!Epe#|JuhE{nPBUzJ4IS3C{o~9zzHE!>mbM z2G8le^h1_6i7Z zH(@cbi+VfZv+AC$e*Dg=GH~6wKnTfW?ytLjB-@Ci(zdwgl;mLF&0hoBB~pr=M%ve0@19Mqhf-4%3^~wl9N3nRtdkTFkJ% zWF*vBvrBl<$kNcWP040Ew<1g3aNhe^g+T!x2Mld*&ja>V1Cqw?_ zN86+^+|3Lw`(tB;$1Z{F7?)W zH!;hio4FDjjzCWH|96nnqKhowXOLWp{4c5DfBEu%fXn{> z4__XHs1GNfT>t#Wzwx?@KPWN9u-YS#(LhZ3|CK-1RA%7s)BzJ8jcZAXU`(!jh(u@* z^2!q#?*BPo9^r;8O##OLU$lIPt43(pKYY*Q2HgLX?=p~vYEQpCK?Ap0;pu^}P!`N z2j>!=KUCTN3N~cCpflvJfA8?;?Q}L;%kZN8-lK2!DQcvYhNus!=ctH6%UzRCDG{DF zj8ILuuWVsTO?039oS}q!*ld(5ZkT9n7{PojE+$`$0GrV1C_U0Hq1=q0Gmg8fwfU>7 z_AQlPZv6R{m;LpY>2Bsjq3glZ&%TX+*tem7M^N-D?uF0F5Noxb^E1uaM%?#nl40AU zFR;V&-;LsT=rz&l;ttj0VY85K1msG%v4IWbovFZ`6~o^%e=-FxbbB%IS~Zpv(u$yw z*wlCST%Z!Ojo%#hhA~ucOTfx|RIz`VA8%17B#2#<4WEdFnJ(cpE=eFOsh|WH;c~gX ztG2Y32lJ3rIO~p}w#K^oQ39=~avY5?(@jQk_|Qg<|1$W${AF^&5W45f#KXHlHq`%k z4G4W&;N}qEMnVAiuT&uB4t?y0eJ%-?fvm!_Bykcse zjq&Ag?IfS`oLu>7T4e?tvVbC`;=Z4y6x{lKFtx=Wpn~?)O?X*t*|0cPVeAb7QL878 z%K23)C+4%6hD&#n^2Q$8+9DVnr%q-9?+;*--_6Toa2qa)JYs357?thgztB`YR(tUR z)SM;Gsxt@*_!=A`Yb*g9$r!6L7xq5z`8ZEq7x>G!L+(4cS72AWk5_VL#kIgWJWJn# z|2$itX|w%h^yJFLpVUFA=+}6E=LRqFevzfIrzp%#rF5NKFdhfLL=v;2kmJjwaf?I% ztzJVhUrThM(KQ7>0$#=#4lSjdc<&mz{cPs9KHa&?w9PJHgWA-@JCLBD?QZ z!HcnKOJhKOs<&9bO2tUZ*RkP;mJR9})GT%)^K(ZSU5{Vk5?PxkH-*fAdfLYcr$6Kw zs0WJAA#1Jr!V~@Jz@HZ>D`Fw$b{MoFpZZjA8NA;kTh~Tw_`+;15cQ4{F-7H}#grYP z86dM1G8bH}edgTdEJ|J@)!Ui)g@zSIoE>8>``b(GjV$=RiS*nS`}0(UwL?5+)mi8r zRfEv`r*y~BwDvMCAmtQwUY0B}9BLAgdCrx+8PtOlmmEib9KoRVDE~$Mu{K^bI*pgxlspt&eVI4bT zR*51Nw53g2=UN zX5LCYSo-KL!W-G%8NdyjMJv58NL$a>RifNL+?o0@F7`4t9q4}2z^+nv9}cXOb1Bgf z9~8zXx{Fc_XR)iA!F_G;i@KPsyKA?Y7$%m4T3}QW6WV;w>}h;QBG`K#8D5r8S|yM? z*X3z=j(|wOIS?3?RM=)UEHAopfg_~qx6X}mkl5|RD{5 zPhm|j#%;W;pX}0m??m8$RV7$gc=u*Edv>4%J~>V3tuAuJL<0pT8lLba-ncGwH~yDa z;auwMB~sLiB)RbqJy+cVg^M1&%kt+q=@Rq|`lp+?*d^+|aQc1?ciaG-Hdr$Vjo7bW z5WV@;TE4f}YMLg)^L?EXIA~kS7feK5Y{Dnt{enZhnsk z=V&_xW`vzbaU*vKmoMk+c#~sQM@bG=|M$DFHn0RkvO>z+gUFEf;g$5rL-1Sz)seAD zgdL6UOfE}@KGj!-((2~P5C$&6Q5#R{)40iPIPd&ESN8{}l(G(Q0stV>7^k8kbivlwj0ZY*=d1Mj3;+E3u`xQ> z%V$!~60?N<&s}gLqwc7ul)GLa;xa7o6AD13!s6NQj3jV|^}VKBPrzk!=1znZs@qNA zLX++v?mKebUUu+$A!ohjRiASiN8%jB>vbYt`O?>BPZ2-<^PZY9Y#sQ&BdyRYQKOT* zQa!UDk2*xYMxd>A(WUZ?nuTi>d9-q<;MujHuG*;ODL-}}cd&`bEWDWXg*t*B({A|K zLqiYFzI>>XM>}2IsfrmyRyTeX=rYb^`{^E49gCxa`COq1h=oS$dBGM^Wc zIU|lY_9Wz&O}_?;S}(E%v4&ET!~IZ8Wq|}ZdUN80jXW?w=QXPnP$RmhjMXed6e zViFFivU*%;m%ad)JhdfgbMvV`9(V6nx&N+oE{U)>7rn=^AKXPMmfN>aveZ=elb^zI zLhF5I;afi8(Mb0WjfrtGq9`YpX?GZ!tBQUnN1Broo@{|4Kgi7EYj7wg$Xrg^vnb`` z{h2JrzuVlQs-yd7g!IXklt^K$EQBS%Zt20>Cje6`$$d^Aw};h2c#7b}41UoPNw)fb@ao>>-NW`lXcsWcFIQL2b^}g z%0IHDZru7aJQg^twwO%^iT4koW`21D!(brkm*tG|_V#vyLkS)^y(0Hw;2!yPYj^Dx z$jTmoKruxw2=1aOcrr-t8!W3M5GP7x`v}C7+L5D=Ap_1}*GSR?V$Tv#uffbWTiFCSl^1Ed_3qwdWD6BR#tkT|uWrE=B<36*yK#k>mw=ty?jzp~t5q;^N6Z7X z(|HPOI-3#5{ z;^BEdvufJWBIuy$#8V6U$T)A!_ZY!(?39+>JhUiaJY6Rmmh*%x}|75=;k=rY|m!$#NPEN@Wv z`|pny9F_Q!o_TqP`7Uh9S}-Qg16iUYWd;RVUw}o+_x#pwDrkO9z6bwkD0l@D)ScTf zDbDaXBNfJFS+_wfW6udI^KF_a*_y5)>c##t{b@yA;9H|y-sW%Q0jBi^T50JfH>m|I z?oUX%4z-K&TQIQWGKl}4N1WK+z*s!|oDbdItVhUc!Ji>;UPt^>K7)kxT7rm|e!kpl z78%*|(?7F0pQPN@zbCUa1r||OW>8t&gKTiE8FExv^>o*uh2BKQ?=NM&N!pUpS$IQN zIf7)eo;k;{p~GXtIv$!)yl?W%9j&$$r^0ZFk*Iiq6c~)pKe!5iz4g-?J_NSU1AUQL zhNncuZ+@N9eDMo=Vtxt;2TIF&nwHg0>0G?Mv%(6kpHz?#gNrm>#>4~B{oZDRptT(5 zM=^ie$D(rwG$Zt*?Iuy9^qN@*E6@Z_7q+!QvP9?p%p1ZJZ^7xnG@+05GSd&}@whXY z_1ZOWw1;9j?`_VBPAr_d9cuT0&_R~;VTUTF{eQO$VoaXwg6o{8lumhVL33_3pAOWN zmq+@o$PV2~#ht!QKR0S{(Q5{O#FD^YKO9Qcyal@w2StpzN1CsHmU0D4E=uvhVx23J zSEhJKd7)L>fD;18+jQ|6&0lfA!Lv~RH=U<#JW1yp(nHR)Q;PSna=lX)dOC}FQ92CW zA69Zt_U3D_)NzKdM_)ZxWTi?tf0{M z?dhjW#p_g?po|jpcqiaC*;tWz8*=~ERI_UJRalLkEi58+GpEhc?6gmBMEKdWVSGeZ z6_3P-lRs2bIoIkZNy$X2l$_vD3qBkcF;be8O4Kk0Ybc{8DY<_#$Qcl9jY28w9goV;rsp_bX3gBortTH{kNLD82{8X0Otdk4$2@z{ z`4O-mM#CI_JinG;cMWmKthRYBQ=1S;K(ppf+90g4$Y9oBXBwMb>ZKc?!J&8%e+DmB z!AF;+eA@BcsrXknNfInZzN^PPwLIhqJ^#;cPeJGLuG(wU@xr5VyUPm$KT4BM{iqEo zmX!69Sq|ii}J9pFMGw{@mpuDt@T+M>wkyAz=7|JRk0Cw|=2Z z8(}kYnvoidALX-voWv(SU%XI$Y<`p<>92lxq(+($_cUo`f=s}wQL_$<*O&CHvZ|v( zLSHhX4(@*rhBHp@>+Dlx_99W8x=1V?E&L~zij1L1@5kQE=E!BCtu(F5+}s9-N!&2G zttRvIZ(O*zQh!`KX7j^mJ8}+#|NNQ>qiJTH!@df;%7kQ}19pNWD+}evG)H0LhO}~_ zK**g7@fp_#VYxMtLW*-(0-WQMx?sQ^6I={wS`Vv1Y~+>ewS zfuEp7PF7LJdkt$r_|~m8vX1Slp?rm9#PiNa17j?YfX?$;nTuM{%pZtmJ1o?3Ii+mtI zWC>mTi&?=nW#RLe`!vgX@_6WzH^w*<^MnqD=pzZ!C9Y5;${rl-vZ=*6xt#vMPhOV{ z0|^Eedc7_%zIZfML?P|3(zus^9-d-uV^wEfW!)$ zx3+!4$d!`z%+)>zWTKcA{+%c{)@$qUMR`ptO-jDPEuF2d;wnFx*s;FHtF{7urmd2T2()vCMvJ@fvpDz zbKp)2`p|B12^zr};Ao`%a_-)w0FrAw5AU}{f?v(58mryHtAz|Xi@L54LReQJWOvY#7AQmRTdw__vh!i zj=%}q;O7A_X_G&+Gx+ejamXjJ=8OQa>f1e3##ExXKhqcebiZp>M+|n_y?VhwAh{3+ zWofeLq6V17UM5hl+#$oocBlJ#mi^SNZ;&{z(sFMwC<#PZvMhe-!CZF`Z9>ZA^QzHhTrFg^oz znsPlhq8D5O?|IZV_lDwr0B~@TDL|B+Tpv=a#uqfU0ixc-Y~8wPt_b0T@zu??EJI59 z&s1~vI3*SVrr{OK+Lv;ouEJV#(bv z69(Te4P?!^^aK>iO*v+sdb#*L1Qo`C6p`AldjKYl9ZUtdMBxWa9x6@TMi6hznSOfB zc45rO*kSZkcy`dx?3+i<{rK~&1L?6i_H6IPpQNu>QlM_s+B~-!h>7ggV_C)wEkrVQ z7!d+Y%@n5F>!R5Tfi2dMpocwpHtkfZQv#rBPU1G{_s|IE=P|!mK0SM4Qe$QMy(l<) zn5^clZh`B7=8kj_cj*29?0hjx+k87MqH+_N&}Q_W@kWpLmiJc5zi-_Qk-p*hn=~mu%3cl<7k5{t_PU+<-Q2gVQ>Rtj*2UCv)NYUd zr2~)HV_}hezIK-5;-x3jacClP2twEzPT{bLFF88=;H?gP7IiC+B%BFr19N;o^Y&Q! zaAXa)gFFUJ76|m3sQ|vbMNk?0`U?R;PWi9U9M8fx&`twyVM9Zc`*>uojSY`uHNp*G8&8;jOKwzBj8y_HE39UxCK_u#go+mCM}&kx^KN37rM z3*e@AFPVw6g%7lEv5qh%10VQw@Aw9yQFDNyBIfbTLt^KeaA`3(+9d=2MLciV# zvU-p1kZ`MJKqmtKpPWzAj0mX-L&p*rm?D`)Zs0QOrsFmH)nvywKuiF4JNyu~;A~%4#EV`Ti6^5Das%R=uQg=^9`nZeCr? zxJsw4k9YoeljPgoB=t~v{AB$h_c@KO`6j(PZ++J|i&OE_#Pgf&0yrJ`Yp7EzXO#qN zTFClADmqaJkdI{2lAcMBr}-kMe@n*cr)_1E_UP~EO=Y|i@Fw0DzMKjdz)}8lYuvxL z+S!H(;kbNTRqSQE4VXS@vykWMef%Wrl3XX{_}8bCLB1b+x`SAjNV4Mvbr30jO58)9 zw^!`=U~Ug0dT4d`WNwA8G&U@J*Kit_xGj5Otl6GM|1n~X7JJo2?uN{!|LNHfjbmE^ z6rz2{L$Mf!lHIBd3t+M@kS*>z>g*fbXQx8&-C-dJ#K%JflBhz=8z0vUT_hX-YdI3Z zB*HGP3d@zr;+bsKFaB>^qi^l%)}Xk-uKlWocD?#lF*OjH(1|PZ;ro;zTRV;*kFQwz|+495a%i}-yCvgFz}VIQ&UqN5S$5c z**9`7y*7ADRm+G{UuNA;uL0PU-mwSNKOj>pK!L{DZ!R@_c zkiry6D{+SIPm<#wJ-iRn=Ft!huX$E4=tWn)u`S-Nl2;pXTDFeqP{(-;1Pk2t7oh$y z9wZ2K_Aw!GXO?*Qk({*X$F?O6f)! zD^W~(T%_C$h?eu;bAD5$%6r@hWd%H7ej09>XX%g8klF`TybJc5W7lnK1V%F^Tlsy9?HKTUOCQN9w!k01*oAio7l=AfiZPx@-{~$ zB~w`{P<$W9-J}yH^#KbU^Qyp3Qx6{if7!HRG|b0T7K?l_80%g3AWK^bWAdN#`KydrCN&IvoffAnG5q!NvBA&nv z&t4*Hb_MTo4Edz?4EY2?A6yI(>RE@$vBQYxJwu6gqy?((4Zq}X80;``teZ#9wWlT# z0?`lHI1l^$7*vB(S+@Sgk_s$8J$P-{&&Dz5wh#~`S@b8C- z(1uju)+>on(ghMU+YDr6+uILA>4N{!tNoUR-LPoh-X8sa_&rWEEZ;O${IsDJu_B;y zv!crCm{tC_JGf3JKEoJsCx+98)v1!#VoZ6>5`vQNBpbwZto2@%De{^fxvJ))EW6VE z{Lz=iFWFvad&GPFf`0t@bxT&ZC{q~4f!SUO+7|Y@24Y>bQ#Zvm$k491)u8S9?Qqrs zkaz3z)e0fQw4?|;cjo3nPfBuJM?6`WPDE}z zEqb2-S>Cu^QC=6`)D?}Fi+-rv&y5=5v)zR8c)NvETt4&OxNpD^Be^Eeel^-!|X~xxU!prGYk~P*%6MxlVY~L&!H2J?cM- zlhP(5HnUhw!7B;KD-SUqAtRxfgZo6>sd9aBk+S5NSfv&EIZD(eoRpB9KSZms5EpNv zW4b2@-NFRF77(Z*LuqR^q~<6H$+^BddoY7rSCfE8ddzleU9M#CG-1riuA$r9ua#7U4b zO*DsiL_cJjQbU6$;ScxC6U=$gL=4iUVG$4CJ>aEf+=If$3&WSGkWXB}MN03CFMpd( zaPj~4j4t>8bw0`kz^xU*G}_R2>3CQrA)-rYk}nRKZ$~I&sdNTmszXUC9OqhJ^Kp08lwP|5watP-eF|3hv;ejlDv)VG6_)}?sDLD-=ZVYvzLHtxZRaH6U= zs-*MITu`{s|5J$*g_4H7hq2jMBZYwHgWt)8>1geba3$oj!xLE?9_1gwm)#-@qZB7c z6LQC3o=}hMxkrX!N47EAgEGiAmJF@9TNqD|{sN%g-@wKq+QO_WK-DS|QFxJtmz(&Y zc#i}5qn2C27vQ-3yM4vJ|D4>He)_Sl^BW3o>R98*eZDSDE}x_B2h^G#^E;O$`IbT@ zy(yK9yS9AnSw*?(vCLZTyGuPnrx)x{QH`nBZp^C@S-)lU!D8fjl@ z`=e>DODPH$Kd~c zSGLPlGb-X-1M8ClPxDnVbn-WkLO{|nemj_IAgI1`CYsy4epDa1>Gmia+c$@O+*uvm zCM!H)5jr3udhMo;M3A(zRgsE(iX_`v(RtUUX@B(g+~jgu{E?A&Q!kw|x(?qCsO5e) zc=&c*Wv8NqiuD88^?0m6gzI`a`$bNE&ZytHIULNKNQTzQ;-0C7r0knn184NCb0L=m zb5KwxDaHUzAimfzo#Cd&t!NSN30u|&KAeNn%$}#i{F^4 zG+|UPyWWvjnWQn@&K5Gg$$XIMQg-Mi@l8L%7zjBZ=5%i0vQHxP3)q|3IbH4cr zymj-!b_RG(nqAqF86{k`2x?U8+Pao-C9N&+;(J|%`q7W|rvgvsKW11!YuBNPW?f zc^zw%3>BJXs2wpPtVn;#2*f`c?I=s{NYZfDx+zylyDZNhXq|7ePRmSlhRFZzMfb^2 z^WD$9i=FzaX!KUnW(&vprGx!Nc3fkjzvIzVOFgJC{ zb$v=u##1mILSP)^j-mqrv^s2b8#Pl~msTQjy4&hKG|0REd9oF6Wwa+YOG(Q2MXqPllWek%mp8TZusc4faYFp)`yIp@|jBER@^2{{=*fr7M9FCUBwlmi?D6y*&uMEvsQ&%0rWWv+km z8wfLeFa;nJ)7)=h@$~_?N#c=3sr4;DUZmjSm~kRkP_^r^_Oto#&lx#yK(eml1|-7n zS7fes57}MLOZYNL$K8)%=AbFfBdk5ANQ>3gjO+OUw>{jbaGh4>Wazu73r6* zoioP@MDHc#_9114oB_O)_iNHD0^;6WS1+UT7^8U-@bN2KhT%l#mnNFd&Z@3Zx4pGU zH5_b@yPx?&hy_-aXXV^R>TbMup6kbk>FBH`bdZq_KnXht{YR-&(XtaD_2+f`j=TSh zoaU*{r27!_UQ(Ld_WZ?V_Ll#{241d0kkxD*TK860_?9_WoUu%vxr&PsHQiQIAn>*j z{(~F`>ip-(uNU0Jl2> zoaP1l#K!MWKBsS`rd7Lv-uIH_<_c&F3@nT{yZrw4uIlV7PRMRPpDIWP_6ezaH1KMQ zQJ^g#?4TMEPp89+>&@}Jn$CdFSTgH@G?d1MwqNVs6Y&=$-I8M)gPfQ|PLx^`GhADA zOdZ{Y#APg9(=oN{bj;qJLMEq(p(>i=5FB4xZGVjDxB6`)UHq)NPCZ|?b#WudEUC@$ zw)XplI{&>$B%tq28#ZyL|F)9SAZ?k`E|6uUU*+cj!sIR z8o)#5Mr8pMRP{v(pv9k5$|=?5Vpo@bEl-9*1+2pt{dx83K%`HY_c=~2(4RLFca)R)gHDr-+Vz$3TwhEfGwm6B{22ZCB+&!{90BE0{uTW2n$7eEDaos~BuRm9VVjfbnzb4<5+I(G%mV(m6v3M))I0t%Ij=*OD{f_xfI^l=g-d*sxQAD-|IlXnShPC z5wJ9n5j~L36nWD5^DFB*-*RCg->!C^Zw;*v0!Gig1qzQz_LIIbT>%4U1Y9@*iTkXD zW!SK;0-PbmrA5Z3eglR~$iHS~ zYj-r3FvUlIH0p-P>^0|CZ7w%pY}%kY#vG)&T!LC|W(K|*h;_b)1_)_}4~Unn^;juy zj}>9_c#r3WrE-Yp#09N?5w@DH2x}`l)4UPjXD>=zWY!ZFxy&`;Wqd&lPA=D>O4Dif z54xHMuQT_a)y{dwykv8?P*U6C;Or-rXOF)hoiTi~x+R^|-t~zv1UBfvCmjp_fT#Km zElbCx{gPIX#7@NM5XYU051y>Tk9_+IA1ugwY08^d?lS;*^~uua)i~>>QeJk*EgU|% zYI6`uDHkE=hLN^kUq9A*5m}My7}oE**lI5VJoGYA#W*@C_=T&yg;;n8-uT>Z6OT!@ z@3>tL9`+7(PWy3(Hx@E;S<8=H%G4~eIk}!6I?#Jk?$irZcHFV>is}O<-VCL+`uOvC zm1Qr$FJ|2%eDkC;+J|;; ziOJMi8x6g?;SuRo`k*qNs=d@^{ba=dLfL9nJC4((A5X;T2`W(Y%GAi_FL6~LhMKhs zi$Ps6ukD(+Lmgk&8Sy-EOk{gqqX|vsJ$`rfSzz>YktyTsM#TpJnMeanr35yT$174* zj>&HE&+!%}3%}Uq2^)=2jc-L7_5;Ax0V2!u_VqmzfnY}BnB7tZk#lLMR2Zt}{xzNSxhYO<@?|0%cu zb6F3|t#c58K1e)SVSN^AkfF|1?UYA;i)ayDTAm>h>b z=Z1cUMVKlXJITlpxc3^++9OY2?IkB_+BXEzg-o;2pJ*ct-AGl85Bv0VNmp%5KUM9WC%y+Vgmv&6U+pjR z=6P}{Uiyg?c~#Gzf9rz-fY|m|`SXJ!ysT7Wv!f$xVMO&6V}9NoP!-TK4#s=QGX&oWvfY+B-ut-iVs(pN8Q=!qDdr0U*z z0p9Y^d)6FGb+>q4Em+eBiit^@Oy8;e65?BgJqR4JyUhHeUlPX;VU#q$Q>nX+my=VI zVyD9xBlbTTnuu+13!C7@(8V&)>QICnL#%Kzk|}C|E6=wZOSgDM+sf1Zx4*B&1S3so zl&*R%5&*}KQInt8j6j0oTFvh(ZbcsKw+M}5r>LOV@rgR(cO9OHzCPMci_rcI((Yr) z4bj%WxNw}Qv|5sFaQy>m$LJ>`KfHVW$~XS|EGOuM?(;&F7^?w3I~C;ci#01pdMzsnMD6*Wbb!K=-i3)zUp{iivd-rxki}Ei?mhcnJHDE?S8$XQT@(XSxmv z#_+fJv*l9h>E4)P3Z$LV!ehmwyY4gu@C&Nd;brIPI~b7uCE#Uuhm@13S;>EbPUiyU z-<}so96#{aWN^>f;kp}D&+dqP+x6py%i{`K8`T!+g@fZ9!!nFr(B48UmjE`54x4zTuQ7wczA-gD+B zUj!)chq8lJAd^&k{MsPf-+9#7lY5Ap{f04mRtU2JcB7o{!kA3urPqbnEcYfXq=gG) z2HNB`${v116t>zP7vE#UOld*K@4F4%yX84v22McOKi#XBa9sFv0B}wB+6i=7&H=HqrA5S!hnBIZbhFS`TF(sA5I zJ{>hqELUdgFn3-k><4((%cI6{!kz#0^s!2|vX>A5=nRcHr@#8I0^`XABFjrlAR&<> z)$Ra&Rpl}ubO?+(7clqk0*H|D!zSRsraC$SX=e=ES^>i8aQp*=&px1}5RgiQ8{^pQ z4d3gJEFRf{QRX6ie)PrjZholx8MJ)_I5f#=HFxD2*Cw#b?huB--MSBKUHw>6|Y@E@;%psNc4ZPHOjcaAOh>F>L0E@MF5!JR<9h8qY zd7R7ipxo~}MhXxKW#SBI;%5iNIF|2^-`c+J2j0l3n;?KV&dBg?Wa=p zpa$2Gz+p=n0Fqc;X19TZl%lr1;gzA{Cv)xqMOXwM$>bQa>CLVD1bU5hxX#Q4*PCq! zBbLS&EY(;A{9HH8o_or8^-8o1BPUess|=WGXx^?9oh~jZ&4&?AVp(9uE@FgH;wPeqB z<43#-PuB;U7wFK{tEk!KEkcKPU4Wy=+1)<)?+z*?NBi?minxtDyO_JFo+BDOSg>PT z;t1w;!rPED?E{dqpe}{d%@cRgX$?BOav@%7=I=I?12=q?iyL3MT>bs+i|&EE#$Vr| zicyfw{s8#CpCBA&AblCi%dLqy%5d~|R7-%&5dWgX9bhAO`0*Cg&n58`iIh|1<=q<_ zU?E?Pb+7oZ{RtheV@Xc6|1vHsV5!sm{?R*Oj%Jux@O_f#mBxa@0@kLW@>V`r*KyS! zE*(~m&Sw%q;*GxQw5mXx4jz0KOG(Q<7AND#Cs%R*B}5vGVFD($Qt=4y`AU(;SV(dh zK#wUlVZnV5u#@7%WpeW3SUA@sYyc_9YirRo4&>-N=3+tyR?Y#Sl=Bmcsp_dgH+VPc z#np*yh4z^WEVzV{ZRjXDIqYDzD;xvQg&LAmP9AluJ@q!XuTSyYM3&sUgry3z5&uk{ z)AwpQid*l9%lRJo3ZAwgwe91?^DAdCbWR6WtB~FO%u5q4X3pxfl{y!-N;!=N_r>Q} zq0SIP4QDnFa9{grW)%e)i|gRHkA_jFRP>E8<{)IzB`DB$qKTWAnrUakaUl~9$wft+ z4F@~s4Q*yyCc%mrTGG3`-*?c0Y7yaWzMH7mC0ctl!UK9W6~V3ZtC&_k+%P`RXZ1 zixPDAd4!GfjXbdjeBn~2;+xl}&3}WWz-xdzOH2-29G))j8`D`deD7ONn_bq$mHLGk zbK={=ReL47p{SPNZCa+C!;&2`uMc2dlqv8^1S0Fg`T{GnxTtS^BmjYRE+x0VO_Mxr z4DjL%A-B6Xq*<_1~_vU*tRQFKL(Ci z-#9U|K+QH%o;=FX{PMIi!C8-XwHFV2KV?{iMgivKdy7!)1?GtfUHq}U#=|xvu#0+9 z8I3e97bY1maW#K%TQ<(#w`#FeH%d8Q-x$Z@%UlYmIRoCKH;ADhy60^H=0?ra)o$=H zF#BWMpZLLIix^)CDk=^UAHemf@67(iE{*K@Z&- zjPn4|54|7qEAd<^zGZc9?ijM2AE~QVX$!*Jh5TLO3LJx4=iYY=ScH~0QaVAlP+ekN z@zIh>I6(V_>o5niX>24a2F6jNFzJZ>V9ZL?oTLct z4<2Jd+ZzP)h%M3#9o-Ws_8o+m@Y-)-Ja6C7Q1l!>{_`i zoDY%0k0I*O|46WXkz1h7qISz(Nu80NfF~j*n-t_LxVxqsG;c}lVZUMcJp5^@DEU_4 z`1G@l&PNYs*%}Zf5#QuVkf%*eI_F0esb5T;WF#~Q{I?ar`FIfm;5+emoDWJxIP^4| zIt0sBzkB`g?*#K`I7SC?h=~FJ%80v0 z9D>aLHU@ap1_NDP3$XK;E&c$&0Eu=JuIw&gB+dEG`&huyXXm9Bc_TA+e&@m%Jc^lj z`ycIi%)djkW05Aq&LqO^MGSpgI6EMhWoK92g|PF5Hs zlGU#`^yh0=c}^R=hZu}A02D;^>7C^;!^kQK?kjT-k3_-(zpasgVT&fT* zuL|7s&uKu+ABd?j$h(bvtdBszH_lkdVq~&$p0M@^|JDh0YP3HTM0j}(+0i;^{FWB( zgE@~Vn(Y`;@H|tX61#xO@DpM($H1BT0yb9>L_^!__TJ;h)ad`+aBtJj-L1vJ_m?i| zJYN=;g4KtVN4y8DQBSP`Fvw(%^)7z~^Q&-ivV8z3`UZfYHs1>8rGh32z=YM;W0C-b zz!>D3-Ny#WQ6LYK3o(J1)k-K;g_WPFaEM@DM+%TDmseTWdRD;- zJqG*qmoR2yy%cNPKM)<5y4MND*AYNI=BuuJOhOnp??G+h*hnR?qt|NA0dcS*5S-Lp z0N}3x6U4#+$S1+j+1pM@T^Nu0r2Ul=>5-qUrRss6{a@4N;fwT880I)w#j00Z=59 zn(tA<*||^*%gWXv0VG^>*5PcpXWtsVw0LB;CyV*@a+n1x0pUT_^^`aqe`I?w&d<2h z#^;kZ#FI>FsD$yj6UgGa5O`9hU7iUk%#3*Kh)zqpItD+6A_4cw4>BMz8}G`_Xm*Lx zkmD!UDjzJsVcuCqtn&pezNIky0x;^PRhPSEvwgquO+vOKDgG6NiN8kq@n-vKO(LHB za+uU&O97|JjmVCxt1m!9e3~PFX zMN8q}3e04$U9e0HkAYoag zJH8KX|Ib{$dfEQMwM&DVE~WG&?N{V$wHm>SvxgwO>*T~HZb{fHYOYtjI@TEK0#Mq^ zZ5-$NAdH_l5y}?kCA>AR%%S3!%yaEtL)^Jl4~)9Pcstj=)&6z}RvUofS;!%K))bBz z3(vJRu;G(%9EeW_mTpCz!sXAVYpc_UwFFtDL%gavkqIpRkfNCWNT2-dd!(0o}5X zt~Ma0r17@bVi%n%?LMV=aN`W^ZP7O1Tlimoy#`DzJunILAuH3tNNa5R?8-$zj_qG; zc@45%BlWr}_IaRJ+o;ZP7>dRC@|Pn%6H@isAIcqniPxmZKHKAgA44PKy-b`0gFTVb z6M~$F&xjN!g6UPPu4Wh$>}z)f{J*3Uq9wZ=godyLkl6VQoQC)kbr3zxX36={1dN2j z;W1tA4W-bPse6#mLFj$IN3Od0odO{BbmDQ_*rwYN;d@xwf##4}q7Ra}V*X%X-|@vh zqkr!4-Ie0 zRQ4(~WmgNR>DE;}q&yMHKcz5x%du2}{n?khxAlxzUrn(Vl9Mc%gGWUQuG%YyKM79p z^#1Kmo{;w$#f((xoED|C~Voff=nm?kk)08uzC6B7)jNgZ?RA4vT%Ppl%G z=!(o{`orCEe*kPVEx&lz(xiKS;Sm)>i=N)XAoCyhj;k${cG$N=ok)4_$#9%m+5#3; zE`&rZRNGRYM6hv+d#jeTi6 zZLf5B@%lCinY`ElmugDOIItz5vt$Ak>z0t%upjkH#yzq4RKf^rj3` z^`Y6$g1VQ&+da#rxhjoG!g104^e1cFc^6A_R314%75C*+iXam+v0u-R%8XZQDw83* zdmUsAJLG_bEbwb%xE@P|#~ni^Jj4*w`^w~R%b$M+=V1O!@Ww~}h9t2y?*(y;(n5=2 zU@vC=R6^|7Y>K!_=Q0!~Ok^hP#j@hQZG?tsv@Y(u3CWo(+pFK@nGQ#Fz1-+phm_T8 zFnM+DlZk@A)Dh##CUs?EEbYbO);^W;lyLgg=j-VpS+hIiAXvgtz8fqNGzwm6fN08| z?@zy$V!19&hlGuWH%O{~a4E`4RG2hKr2-(-dyMgIt(G&U^sQIX;C+O>)n(+zIXZ{ax(0N_Q9gPA1Q!casfKZ0^{PPv==fJO*bLwKrP`!Gk?UAeGVV^>1Kt~$l^=kZM zh7N=4aN@*V8NKEMRVSV})~-UZ6)z7XGiW_%0Z?2p zBNCkCTwZIXOt%QrwnbeMT{ocTQn&#N-2fqgz{$~q}D4wfg0r}N7wc!|?3%OvvtfpAj`Fe~H9Y*T>eu8Fgf5;vh#M44}FNcu$5j@fS z@p%iYHz0DeV`+j&Uq9@s_aIn(8Ip9JHD61g8IKf*wtDG;BBlv!F4*$b@q{#+EAPUg z;-djlnAjk7p>xH3zwhOUptlgzWjOz#`k?pRVAT5W?}kXl2D<%+lFz?+aLb)X{SfEj zD-T#siyeQE`{~MuBdH#6#=i7qW|~B!#hJl#-o+PbJl(+;opeMm?d1eBq5$G{Q7XL! zpr&JZru@>UL;6vB+L6c*yd_WUQC%YFhg;k-<~0|y2FdWf=Rt?CuA3Of^DqF8xuwi` zC+UVYug`BE@z{8F3IbA~UF5rIjHy^N{m)<{nD&-%8*d61t=(WD>b8A{VkaXIxqJ?B=BrS& zqXe9cUO$*2DIOo&z(NIlLBjrdsOcK-VSgZ7n47n=2F&5rw|{nbgG%C z{+o6zH#HhWVLOA6g@x?9qv<7-m3Q_iiRz%%FtLD;FuX2)JYGQAk(K+x+tL(ujZ1)I zN|zWa_ub%o`3%bTZGOfw3w2z$-$E~ZvfB05jdOX81^YxVe|>Yr7AD-Ep*vwFJ#q7l zhQZ9GFtc2dn9MK}XWNYPN2me6B>fHmF=zjHSlLcTNnZ8~LJZX|D#cJjGJ5G&LE|!a z%OsU@Z6#5+(VmzfLotvr9q?B-$ZMltHl+!$t}SqkIx}qI}^zy>!Kx+g5AINlSNYC=~rV zkLr|lYTqhsH}}u=W_y}2yolRF6B8P}S1ge|E51Y*0lh!nTL0=)t-@%bVvrCro_gvM zmQ^tB7?zGz?0l=?Yr6M)So!WR4JD2PnivO;c{}w##|0-1YK@w12qVo9sRyT0N2EUvoHh5~tb~}lrrkqkc=X!Q>Q}J$s-%USh%co6GUNOW^mLZg%5NwPU96!*mE#d(jIQl?7NSnT4WJS`snTDTE`y3Z zc^;s?<8QweDm>I){%f`Zq6hp6T z6Vs|W=#N`IR_k>8MxjN43W}F_wr)@Lw4>f!&Ooq%C@hkrOl# z5__J4vo`{TfinBb00L*(2EhWAGixG}Y)(bJe@=bo7vTw5Al;Z;1suXaZ`_GT)_}$d z%#@%Xq-NT>G$vRfy5JmfQnksCa;1Mv^^mX0O>i`au0k|_@ZeI?$~bP>P^J9Yf!{BH z-^TpmpSX-N=bJ10#3uVqtNSNtY(_#Xgx@7<`X?dNpEvVq=0fM;OG~A>ME8z^b$%yI zvd=Y=gfCF@#`lGR{dMw3Tc?}*Cs5eXzj31vEH7=9-pOy3y}x}{r$$p?Wu*dCNYLl^ z)DW-AY{<*;{kj&o0?sA1kuTuBzxPCm@+@DN2jx|x^F>FmoiokOK`N~-@MsI58Kdax zr$pIC)-y-Ry^wrFRB7FS2oN3qKlUp-mb#W-HxK`a{}6z%gw^&-hY&!GF}3G?{c10{ zo3g}}$z-wPt4xnL8M)4NTH1Pw{_cz?hNM+_4tuokzhgJ9pIP_55Zq`_{f9R|PGT`F z=i>~J*VWTL#8oOOLD7saz%fl_HdN-I!Laf*;(wzON(@>k&n&628f`S`*cM-X=aiQ4 zUP#rK0roT2-Tmw!O@Eiij*G|wNPfG7218v0?jmr#bhM>_c0xKjU%jcA$a-qNs%gBoK<96$I|(|fqA!mwTO@=tgyCP0%W6LN`cJL>F-dGM_!~jF>3?iZyaP)7^eF#0YlO9h0Ye4Msm&`uo?1n9q)dg7U2@M6fv90 zFvECQ>JS?L7_j`RB50^6w&!L(t&^15RJaOq9~s)NK-be=v zSfD}w@_@GPB~T!|ut2Xl(22T1HpX`MB0yYqmUQkyPplz?KVG?|d9f)nR1ec|jR*fG z_{2JMqX3L62F2=xx&-hy2wOmKSvnWF557|dU*`Hw<|IfrQy!2tIr`k+yBJb;Gm3a1 z*9Ch6ed$d?0Rbm+1QoQkCVbH5sod+fd8pcgq{sA@wa&sz`w?PWs<0=ZG%&nn5H>kk zZm+Eu-CY16R!s;Cc5P4=0+d$-gV5y=f4lM1<$)u6$Rmm1aaDl(Nc8h`RwD@qEeV*S zAxyct!&(FunY#TT4ox5k$M<$n>d0KqP@?)TW+WP=axPR?K_&~|#jNfpK;b3BBsGCX z6UGqInH0D!q-3=37qJ*n7+ejt47j&n%dg#pbuWZ-cv(3tn=+Vq5YYC9W&^2Th0Ze9^DrbC4wEWVvYKlKHexKg^nR@`@LiWU_vWQQPeS)LY| zsbN|WcGr!=R2CE@6^OD9ge%kA<(v)jiLn6T z&knez5xiv`tY`y-b&B;?$e>2OgYoD{Wq%C~_zdHGnGfl=1!&c(h&wEzczz%iA|f3X z1`kg@?kYcDLkBam5B`$+6NLy5x|{JoC*Uh@a)kQ|97(!h!Xneo+=_l1(Iodx||HXhy#tr_>}Y%FFa9ewA=D&-BfNr7GwKFRI!{5eyaX}j zk5s5d;jy|tb{WUW!hl!uZ(DK{&SRj7iSSGV@fr+DTjMXsPQ;%)mW;$C43x47%HWw$ z0vdT*aHKdP*AhOBG2fHrSr|xSoNr*p~s=5Ic$sS%{M*BQcAvl;9S{2d2cM2!Mi+nMl>I9^#!8HnV z&%cj1wMJfFqTbJm!yx0hO9~F{a|ebl?jVeovmc)#E|PhdHER6)=F?z8H+)gc15MNY zxJZQj943{p7dq||Jjio0;u)b~54=#~C(|5^62c4aIL!V^%mwM1Pn*svz;1?nRMyEK zBXl}Qq99UDQy|*O!yFJV3?GKv&5AF;;1JDO1ew-{^3VjM55!W^#6=>zNDCohKZb=F z;V91?soI04M&_N=M>Em?Y83trLMH^oJwaguHsct1GIM%3Csg64pRPCxRO!FOZ<~4N zK_2u3@NfBho`*gqfg}Kgi!iYp>sX&qf*HcIp8KIa`Cpys0dtD+x-DTpp@9~fENh)0 z2II~Ot*i(RFT{_r!^QSPv_pGH_5v_KH=ai;kQ&Y++ZJ%pU#9+?8=|=eNxv}>_&eRh z^h1;++P@!swogX@_RvUWXlMF)ePSgUnOX#*SL=f-rb5v}r@H4RnQG%iSl|zc3A(j- zh={?wEt2eK!bKv(J*Ahxi%H&x>|k)ZFwG?Ba$9+t_S@k^SfTS5?y4AXzM&)~!G?vG!@23gS2$R^i+ zK}a|8jw9kx>;jIkrtIK`2&)$z`_a_;e|NC> z1ycoRiyS;Oe>~Efkedt>E7^0t;5*)w5*{W)#>)vCu@SVyiI0fFiP*qTcRs8*D`T{@ z;gm{;)W-t{(2*}F7F&W|EQ29!mT_mHLb2{p>%$NpIO`v@Hi^)Q95J;Lx}k=VaC*@J zmouA?!9gIou?kNqSvSn&s-l}7r!`!O}h>Hb_zGgbo@ zDg32PTD}H;j2Z028~c=xD$In?NGx!LLvEP-2|{=tI(_dT#&Jh?005!4+xQN`E0FWQ zrSi0?7|tEGlL!P)5c2Nk7FSJ`8E5xQgw6uZSS#5>Y2o!NY{<%Tiuvqj zWrt6Ehoa5q(dnFvS;vMXE&PD2`U}Nyw@fftSBBsV`n6uRQl810%oDJLc1LUHLCBs?ADLZ#9Vs zZ~0B+@MKK{jNR#6dCBEi8@0yZd+*zZxgW(UuL-R$H4q zv~Kmqk0z#<_4UxW7F=EvH4fua^zeOcSRW35(DN4 ztM^yOUHC8S$P170ar>

qdBK$AP$3$AQwD&O4e7I{9n0Ucga9v2so@Ja`p^b=w-e zK5l-w_|2jN-R9QiS^pQJ^+3i=l1yt4sx6}N-|`gD+?p2nAu{Q;9N94Acxx)>$n(7n z&jbCH_es_dn>W#_g{qi)s-E`FKFHvmvrhX`osC=c^sif@+pbHoG1nZMZ+(BAeC=L9 z^J=luR`=Zzjc3RF3g_xwm+Af?{yOl8qgOBBZTgmbQ?Ccs+un@s6&l4ec-Ln?IyAfL z`a-?wTeC%8o@M%9s%>6xe~0$9uMydE){3?d4`GWo1Q|D4O;S!vseL$TC41FlkRs5vu6vHc!o6t9xqsvAHcR0WlZwEG@HFW>GZ;Bs;G-= zIH|sZX@1nRl(SsW7W8(_)KhXY4=Dt=- zkjeLKH0_{@%**=t-_oUjW~??!k5F1T9tq-P=5%|pcK5*_tZ=5?Q@VtiM_~~e1qokI z^E+8|nZK|uy$=lUw6BaC7s7g*i3$jlKh0&A67F~mnBhG_qcsces{ zyPVLKLy|kN+v(t=&gN|2V&YWM<^OQzWY9Oo1*)j^W~EgZ(bE#@LzQEf%`5zV-#YTb z+E$@Aq*0UX&$)(Z^_VPOg)8S0brmx299cg9fS?|}`QqI29~8HdkD+_Q^%6vMAfj*B zesoC>OGis`vHJ^Kk57cW4h50l$tR=!b8%z3a46HRTYPjEgj{in#3npF_Y&!us9Cm)~A>^64_x?#? zC280^7KId=#mGHHOHQi#1(=l6qJ z@pi9r)%)5z8?Lp>F9y%2rEb;MMQ-nYmi>{2!cnPr?uaECi5#S6J|VjdByD5&-R%5-Gmq7hXybSKJ(w_qUi!s#D~;u% z!LDiU>@ocEAkfu*brg!1#~V8*d*`U~ft!VN9{vVFEs$S))IJI}dVd+7(0*+Sb&1;g z56@3xql(s5_uV#jEQ`Oj`6*Lo*+`uH9DFYP6T(hSDjK~Jlkh{xtO+1!oN>F6FSJ$1 zvM*dGy~C8*NVZe)fl2$vkZHoFSPr6(UL6#>_HBDV145 z#WrN-To3jA{oeDw=bZJfv(`DS{%E!Lex7@Hp8LM;>-r2mE?K8aO6ZucOuJ^rv~e?4 zv!=F0)j_rCI{;dikDj>Nd;sAtZ7$wAs7nz6`QZeR`Q&*5(4p=guzgN2c>$2%gl#|! z^#T6=*m&h7I0sxtNCgm&1N~Y+k_wS_Qkk6wwlpJivck}1guPvPcl|08aLxKy^Tg4+yq+pI*-9;8c zwti9=Q08RA_g>y3R<4KF+p-TOXsHU<~&z>rBEUHo9!OwI`&EZ68n8Mzn)c>UZG=CUkpx`Wb}-bp}+9X1MciD zJWSP_B=p&gX-T+gZ>G=|nrkO+fwcAzSU`^=NdL6M`v{GA8hcRL90HcbmghT)2rwUV z2U$!)$_YCld!Or;1@u%kAph>YLPbC=v0DVB0=3OASWEx`VR1Fg8<1)Q7|hAzHt!lS zaXRvs%pC7mnr|ZZqVaOl90}jx*r}tr2(;A!+2wxYY5;G4MX0+kVLsiFEcFTi9r<$J zXl+sm#AGPE8kV3=J^QsKUFf`SpxIVX!MasS$)d7kC~<@GiJB`=JwaRAqBka4w?<4P-w3k}(A7Z-8g7yP9bXDbgL8mPB63^Uvasp6jXw z0}=(B*$!IEU%0o%3Dt8o8+Sds!4~YtRJ|iS_fOIdV9K^CL7rl`wrf9NyB5Z(!nRbb zHlao}YB7BCNs*k`s}J|!B!!1M{?{iU3CVE>Fc5qb!Evi}n*4_w8r6kYCUmgpYGdqQ z6@NJhN)#qQ2v-~$p~VTXrzf?avpu1IjCQ1D|7Mae8?Wpegvk^O#l@ep%3e8j*{pX_ zkpc6q0ZA^?{G&^92MSq5y}?N8WA>XU%*JJRAvu#(7-4y_kiMvc80JmhuI00n^Q;k2 znl5;H+AFlPc5?|ZkSnXbF_foz>!N%sg;~L$Z)VuJ-$5|rd3tRDz@1~XOX~?J42u62|2P-os1U6;y}*>eDA-j zC`6rH(g1aVg|_Y&H3H4+bsnTMZA)BqLA<-srSkN@~R@??Teyz$xgLg$Th$o;o zhvEX_rx^RI(?vYL&|*H0|1r775x-;*H0>9;_=9c+L9d=pdM0h82sPPP>ED1FwPddd zW&N#LP{+stRH~`Tr&J3ENnU|5_c9xx%i&?c$J%|}^m7;?^A+W!Ww?>;xVB2ovaE9s zAy4imt5aw$7Y{5RPcq>~zry*OaB;VLaJ*WmzyAu8blmpA91p-EKJ)B9?2@%4#TjYv4*H$>z(P3cRu1JS6t(#;Fa-7(D|A{Ps|H<+Wj@`ckvt=tHws`O zO{bU+dB*^5>1NDE(~=ygiSr4_oR09c8CnE3RZ-j{AeG6n=#?+{RKIiwA9cMVPDz-o}5G4nF(3d};Vc`}B=tDaR~`yITsB zFMN*AtJ*&>rlKTgn!)Dw(HBV0X^xK(?V+>&yi7ND6L2`5zOfvtW2%!LX!^1vR$#+|kB_RC9* znKn&Ut|`;^jLX{TLyG9!q4KVIUTKIE!5wbde9^6upVb=Vz9R?RV4zUbuOJq z7_6|HZ6VYHASz^-ufum(;K@P^<9vD?+3Ao7W!!-v#)RQ#UuhwnV1HzN^{f$%Z=gK+ zs0Qb?EvxlPHCqz>tC}MS(I%MS5o>uKrNEBrx<7ITGZv<~Jzv9w$5@Nb$Lb&IPOqOV z+SF8TD$r}_ua>jttdx|z^QO^jMhiJ1(GI zd*wN8m!iyQt*Ue?SWYeUCG6-5^xe{sTK)O->!2Hfqo}fhW?5IyXD5_#PJ(mviw4FqZVYGDl9wqL-)KowG|ah~IaNgn!^hal2GGcQ4OxIR^{!M~0p=N}VqdUu*m zM{iPoGbw2Jd%NAYI{W-X51rSpg%!P-9H$Z!OF*cN2v|8pL%p&XL7#+d@Mg@8Z_*O`-I)5u0wG00MHcmTd|fEkTU&;+|@^ za?CysqSo@S3?FD+Yv*RgrvzvS`e78_pHvq$dlvm;_V5?a(3N(9qh$Q2Lgx+`NUQBI z5O8mvm{(Wr44iCv!r^s?V&_G1mp?ais_*)&LDPQ4ri=Vr8)a<0BY2NtDuR1+xBsvT z-4_=iCtumdsXQ}teH(Xqu5z5FOYq42P1=M=#ir+LIg*~fvuHCCR=ft=V&Ow@K+45) zFrLDmBjMXpFLn*>yWh|vpImEQ% zmo%G7Sb^Yl5Ft3thwG6as{Nu*elEB!H6TKJO3jD=PH1hH){l5Cyeap?XW*Yl$IZVW z7I20{8&7jP@8DDqOFq@V+!S`cs*joY_d`d9aT3*f9HXkmIP5_QyfW&gysC>E*6 z#>R8plbpur);--b?sW6+adUwCMDdNEMdR3rOO*rNU!ERfTkcUg`1s91ML9QD)j%bs zz3fLmd->*cVg32$0OZUGf433bEM$pyxlNCOK=q%6*g*RB`UiP}&SYVIV-ycmAK(!D zcvm|i{lAsqpe7VrUnp~@wsWVpWB~1i3&9yO$dMZD{0Cn8Cx7w(3s9k(Bzo#MU^KVv zS+-a&Faa3DiYS_PFGqCtFQTa}q^dJqd;4zW!bh5i;!Af5@;sYZcbLeidKc zF4i_9Zs6WqB4(A*dk9%$PzcHgNg*%hM4C!!B;dP+sok&!`7D=$b&ykyN2oT$nMT(K zQnW@O5tr#}rbmmSjBxIjPlP_F=VXEL89zc)aU2|cVQV~GbH4z{OowNrF7rjP%ZDO{tKE78- zwIy5$Vk`cL-*dwjAGQ(cQbgZt9P$V65X12tqnhGiV3EfG@S0l$(3S)WXE{J4s{!IA z3(h$)6_5Y`NjJLEuOb&KbR$!L!0{BU#$^GF;-p-GAg;Fn3|2nuwb90rHqsV#XVnxC z&v3AG%L0L#4mm%m?X8tzNeI+ErjxElW+HgGsIC{ng8l%K?SZqE0h~GcuD<}b2U;;y zW_83r#KN~4^s7ESfI$4hS3d(N9PVM%2>=gk??RAF+;b*zOn_SEOUBN9M+o~TZhz5~ z`sxyw`oyvQNM29v_Oi%@aGJOfLW)Rfw|InoV)-_T6>hgXWNK zgKgGCHk0B)I^=zrdmy(VJQbWT%|Nv{)tv_|w-^G$q~Uylo#%ixcY%zQHgS${Sx@Ef zLEFh`U=(QpGVf(*mf36r6Zo6#Qtd2_9tQ(0qP+_ES&=e#ehD=Nr;$%#(TD!5m!KGK z!e%WCiUZ9j_~R2ffXLen`}k%|2<-pS?&ZEJ^ssqcIpqk99b*6lhfD={f?u>#PQW-~ z`Xn=7N4jQ6&MUbZpdlJ5Gb4Z~ujOQ?%ipVDy>OZM4LAFY2(SJ_P@-`8)E~G;4NmHj zZ|B8Fj1A<^L=J93w-JJ4A1KJ3)M3I((o=Y*{70s4KMH8t&9Gewah#l=S1i$fHBta)8wQHn6x7A` zdHRnwXJ3_Y$$5A zed)Ofd@k-dpQQU&xt--Ei|9R0@#O7G;N14~5W>tP$mPMaz{?Y+Y>fnXE_KBFc_upe zk>DPrC+?NQ$*j$YK0%3sIxV|?}Bh1Oy1o;45rL+8y3sJ}tQ~(NY(S$*g~EKIFZ#<@CH5A#~7cu#UdOtrQuSxPOay zvBa2gAJUskaH#KCnm+Nv>5R2QwmTimTw*=utj8PiOV)%?HZOw%`=83h9G*`2`$VyG9Utq2ETytUud=Nb+o+@`x zKlLTUlu>kE+Ffnoor!YDNoON8+k@8(UprO_NrLI0HZ0(AKSzHrezU}j2cN5)k9@r8 zb=Z8neNXC&{9ol2>fZ&upmT6I+c4T=f(!&QMwC7BI$r;(()s1nChMLC!j)c^+&Aoo z+TuuSwbNEUk8DY;#*o-IyZ|ci5!Fy5YlnwxUymB+>W-?coe%k3YuQBk>eG36#c)$j z4n9!rMLG--r^7!XO#~W5Y8*x}e7YG3Q{YDX>&pvA$4}17k)`E{*$qmrCCXWhd$8gk zN!$ClLN2aDvPn>~vKp`T5ggQ1o}4TBX8g%(h~d$`WM;9McaNz6aBbE+u80;zLZ?*0 zzc5vs9~ATBbY6>s-NyLjUc#hK+0c+vkX;n(hB%9|$Eb<=+&R?{#a39@wqzYlsddn`E*f>&?E%3ra^ACh- z@i=yOS-MiQbpz;X9GC-7&O2ss}B+rQp9DLv#L@KSKV;=mMt12IZfbB_lOWsl}> zf2jTR$BDg68oeV3{Iz9i7DVSt%;isd{Qj@>1^~*c^#s-S8AKy&=(3bD1SsZH4|rYL zgbjT%XW%mT8ufF4@}3)p9_}s%DC)ikF?`dsB<&) zW+KUf=EpaXO2$NFH6cmt zHP-^tZq`6SD~J4g&)JK1lP>oRJh=x^lkcXT9@55byA**tkR)(Nj!LEAwLw~LwOX0N80GCIf7oK8Os^KqRaYX^k-0 zK73k~tAc&Z27F9izGnu%gZ=%`Mn#4nf{0v=m^YUI$5(LtyRZN$^TX!;NlguxEkAJP z>nD?e1;;65w<6qtXcGwZW)c20GB}{Fh>T5@g1YNPkXnRCVBxRk!>$3zqEt*HD?C1p zXGfmKqvoxffQY1uRL%e)ELboN<(Mo$N=JjPt~yjHoKAZdQUTwQ2G%-fCrg`GPLtN4 zJCYm8Ci@Pe?`XkS_}`FfZ#9a8VnYhlL)1VDmH*NG-=E4M51)&_0XUik$8ONNvf8@Y zpMMGn*o+9BtRKWG<5xzwP>nHr6NIp;KJ2lo=s4@1=t3DrTaGE7%?-l2SW_ThC-#9G z8%Gta&EG*?NI!Hy#LThj&=L|$b|7G4;cmS;S#6KSYPx;WI?OrywqQj&J6= z3Vfv{P#!jfLeS*+L&&l6=S)#N5%K|rFbR)G=UTv2*_3@TH5QiPu)&x1=?({X*}#aX zP5^nxRHb?YRQ)CiQMRLyU|(F_PfTXE3)!{@R$q)F&=NE+T;n9NlJkCW2nmsr&*B*E zA=#aVCL4mHCjLg6M>8hslGT?av8!}t9zYkD7ZYj3L|Nkzf7vuNUz)qgcnB@6$HCzh z$QiD$po}Ii0bfa6WIGb*h^65V6}@Ztzs0xCS|)_^*WkDA0f^WUDeCV6CnaZr#<)kY ze@1#v+5geKLIkdgR6p(BkWy^Em=NuOj5T~LAA-j zG;+2tFRE%HWQ}KGka#~UHZYun&|MD+U;gpA*K-Q;lo0vq?8Y_9q*40irP7eZ*t_5S zH#0~+m*4scG|7DBKuWD+9CGfW5v(U;kuaB8+)DGb=e(c8uSP|XW8sfl$Ub;zm2g0j zX0(NG)7_s`aa|hfV+9b;OG3%D5Hd#E!io?hUZdmk2lSL~9KXSX0jPHfa@3mLoqp_s z#6~$&7_nGom~edR=XzNBm9W2d*C+NzfaizeLs~iYGVkF#Dn`q|jyBmMTredoqh*FE z;QII6-wSwsCPS&vEop=*)M$QuMT6V!SY_SXG&#zNQu*;_a9aMGk zTW1{CB#^Y?7PZ{D#Tbr&+=R?ONt4Q((s06jf;XDmMBz>{JFN*$HT-v!2gU{{=Ks6G z1F$aK_WMUa*E(PeWeCGn6iCP0o_4IvE?Noz9U69#UFK~NuR$Un3xX9l`uDCGL z!i4A7sJ^?5e6bwj+>X!dD{~@=aL4Z zBdtcOU6(|gnVqOxpPp1ql2A(9F-_84s=tx?N>WP0(QQ)#6q!e6eq0%l&aUaD{eL1f zu=o0u;?GD}?1?u5QZwlxC!2(b1T`g@`L97JcQWyxm}$r1+^_lQik#@vHoaT;pb$7& z$2mnx#Z`x#;h^d+bj$AwByG*Z7>?n0%Wpz%WXm*i^GY63=c&gnrbus7g)x}ZZGcKS zPva;V&=4T_@SQBW7T%Z?+Kf&*Qr0@aL+sT;VSlQtL*GM~%;4ihTjH?iVnk|dTa>i8 z$C*t;YM0T>!S~C%)Wxx%ogRKi?5}iTq&@NsOD)tJ&Q5`b$DOr_7AgEu0fYI8wrtCC ze+H0J#Lx0~&wd0E6HhV>zdp~9Z~PtfA!XIBwUM@`h&i)v9~`x+;9$791O<3$B&jD| zi+>LSCMOj3w)k#&61i8g7W=#Xd`J!O6D#5KHHQlLLUL_6ORPq<(h!5@RHDLnr9PLZ zIOq=uMou{(&c{n|nvMCWWn%vA+=rL9KE;Bhu?$@{ygCmrWZ^DaTZ<`@Z9m* zH75uwvX743NyfEsG>TXqJvx8Sq>(1C91;<&v@vDxxB=6u{!ggK0O^$fNyTX@Flg|0 zrQPR>|4m>OVQux<$Wdz!a)yB%Gk|G_oS`}bifZ{Mz(!OH{!eq>yJ-(^~{ZDQB-9RMIO zQ`Vk##Rs%6LWwFgDHxloaVL!7ce6*|As3nrc$gjlp%j3XfeVs60)3@LEZg;@suX|J8FJ8;w$0KApNsDhV^Bv_yysR60BQxHZz z6dDJ5XW}FuPcsGBO6?uhZwVAi!;1NyBfUurYmq4i7_cf(k7-?$6SSzo(=z~rJMP(e z2mI1I9g{qLHbmcZ8g`U~X?m>}4Um2_&jd78Ylo2PYHvIAMzDL$oPjh|F&IRVdJ&z> zsQ2D@%Wn|rJx~rR0lOXF?d=VF4}*>0Kd0g0;$z?va)C_pliS=ibHMP-hT2(Ml|2yf z8ZG@v2Z94TpyW1%u=YRh&KV@x62E9l&pv%Fx9xb$6M`@H9juR1DhekeZV=AU%E5^) zD`Xx%s!lXf&8;3j9}xE&Qjy>FQY70jEHiDh%Je)FYmjm4n1y&&r?ObTQ2-atXkn6; zB*7)J11X5KBx%7CXAdyUFln32L(7QYM({y=`l;WGENX`~0NzmzCyOrI!SaIe@e{%p zHD78C!hg6U0E_U~kb{&bptPEgT_|g;9Rg~OqCBuu(0@)`fJlc3+kJD~6jCE*zu)?T#IL?*No)`V;rITghp!( zQOXwV<%B=yC_c%jznuaCks1+~(pfPc5I{%*FLIYw%Y9n0D%VF#5wS7MEw`i|RIZMz z5PNOw0-{=|co(#Q7EA?8--JL)m2Vs}Oaj4_U$?#Q-B_{gV{9foKCG)}E-tAOb2K8C zPunTXY4Wj|V||HXt4^k>GDgqi=;JSy^*bXX4%}REDfjxUPit%YOV%#SeadtG{b|bX zx4G@yi^AoGRz>4o6yWi>_qndoTC%J9FmTnXd_*3c_Mw+TQc# zbo#1iE8mssu?@$i??%ADWtKm?lUOaA_488;|BS?G@5njyyc*IbNoDTl;C&daW%v@))aEZAjrKDV{u`>~unuWlD-T(NgAm)>P%^UrJW7Z``$OvK9B z>U?YH^`!X7QI>L0rNoH8=&o>Ax|ID;|6RMGFmjDvHwESri`oz3MVrTNDIAcuZ_+8U z2@#^TpbWuk4Vtc5-}+pk*YQboFvpVBUx^bHa8t_jlg*8;j2iOjWkD*}daF*~^-wiz zM31!dx3~CZ@1&PCw0i!r)@WLyg%$;xf=BlJl4*+G#RGGnp69vUUsB7`pBS#`KPMuY zo143oq>5`jKlX5WwrBBl@etF}$I`2J%^G#y@%{WXaaAi1v?=V#lwYva{D5h#pjWvh zT6*Q#MR5|FfM%Gu{_{~05uM##T_>VF)|U#>K10qxWi>w_EVMUqlYt;BKu5H3>%7#;gF(;i8%4hiHVO$-Jv5?g2LEb&uImF69^ zx_ntl<>vB4ZS9YpP07x+iHV$C4o1gkwgbY>YYW$0g%Ubcm2l{S;akG!2*FZ{hVoPt z-^O8e6<2uG?6V}Nm#?kT5L+P4tEY*sw_YOOIIK1-&oze$VOe_h>LmBrsUbaaY3aivLr%jA*Tv|#j`j*JC`)bN8h;Ge zT5#(qcx`^{*EcchAA;0NiaDO)(i?sp1%p?<-$@+pmCbAk#h4BRRqdq2PFwwFs zgnAGANi*m_JqidCvH$Qi(J7yb&z7ctKQT&)h~z8n z-iN4&k1~|5^^HOG!em$?Vw5`&N<1jlCwcfaq2ywhlAuMZl=M>BUlmXGj&@^;DJ@j~ETHU&FRh5d*E$I@{VED;J68y3( z(?M9eVa!Pz_8RP>2E6LJlHh z@kx4mJg?wsr9dfVaQjYA+bpik4Mz?fn;`K>55admq6P*cAF+iSbEeVTwY+oadF9J) zZXe!ucAm7c1xNJjkj&Aa$vuriSf>H%cZ2$2yj-wYd;P8Lt$V>w7~D+FlPW+@Jn&F1 zJsEZ+71(SiEtz4Eymq$jYCPA*$QpqcHa$D*d|E0&nTTYX0-TN|a>$`MR3wnXKd$J# z{bQ{Cm&o+ooRq)6|5wHlZIn`-tnWx*Gf=YV2cbM%{U-YqeE%M36Ksk}NJxBT{65w` zI(p@~Ra>lZ@dO<-PK2c92QsYqx?UKf9$8-wMXw|Pt}dI4W;Q~s+gVqq)WAdo({P5z zrzd~xsl;^@4;r0k=t!gYJY+FX+P4guA{+|X=5(IWPQK1Z)W58r`Ct(-)~Rk( zJ5aLDaXSetPWpJC)myEPB5t#(HW7?}S&hUQ^-w&%h_*iUuQ4BKxdIL2C8oH4sFW}} zAi_Py-HsJxQdBb$0!%2b6&rAz*E&=M%MVS5H9U_Z_W~+FDWmVTp?jhh>bWKdXS6O% z^V`5|w2zMl35q)EnGGdL8jPPDY2HZ~k7OVWCsSx(4H-yK5oK?A#(ltb8qG{>z zs&gx3pu%A$nM?(Pl&-;;2ui#^Ku=PQQLq#OtRiO5pmZYk05qCMsWT4$$;I2sk81N_}zUwWNUE?ho6-&JvoFd)t_AX7LS zB9d3M@W$!t>kHz0dwaiN--(i8qa&duY>L_{8Leb{Te4b^&M-9N7hJq*yzvZlz!tjv z;g&S1Z##6##EXK6h@=1-d73+i1QnkMBa3(LXvL0%`AB4nDJT?Qt#lj@Dk<%ufvbzb z7&R*u(2(ro8tKrN#V|ueK_z*^Wgp37=#aqIcyTPM9|Z%9iePrCf!Ej(?j=q8o70dT z{;}|um)lnfIJC_5`fO~$N-(n#kvOm{K|APyia?6!vIRq6??w-Jhw8WXbXz7QTJ1RZ^+RuY&6GcQI z0@NetZJ0^a_@Qa*2YMAz^y%>Fse zp$U|FCsc}9eOnbY(u;|tcl&wCH`5$m&%*)aPl!k+dEu_ylI^d9pe?6i9COq)!OmC+ zu6iXeDDqT>4D|ASvaPzQJX~$2^CWkW(kM&;r4+9V3al%YtgV7#Lt$P5>6EO6;#nuL zhMve{TTpQ1F&-3{#`KC*6y-*dpaTmv-9;g6P@zG|`ZWxxLc DwV;2! literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh2.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh2.png new file mode 100644 index 0000000000000000000000000000000000000000..0fc1410d5bbd7bec387d9d743c8619c1c0fa604c GIT binary patch literal 72757 zcmdSAbySsW_clr_fkj9vBHbk|-6frZv`7fjUDDDG(hUMq3W%a~qjYylcXxmH!oByq zfA1OR{ByqXjdK{_dR))*+A8EFYsI5-4$I5@}?6eRFX zSfg_=92~lcnYg&JjJP6*oO^bBESagu`=W4r<|Wd)aBwP}sR~S3GHjnj zZ-_={?mju2CZBYo5T2~`{H~WTGz|iI=$;wqCnO3#)wg_};6t3zE&73;WW67Jfb3&TT0w_MX)15L9E() zNkK?Wbdmr^BK~X-bC*c7igi|3Gw=E~gVeBrL%!Xg@li*#RP|&uHCyx}5p;4SY%5#O zY2?%9miZCP7qn@HG=cd?6AFP+lYRJN=_N&CpYw+c3FfQ$n^FxH%831as}(5 zmy5H?#0-mkl|p5|^s-24cF+WC#1is?by6N&C{22&Heo7>8%Q}*jkOjR^6}(A7~B*9rirlIiC>T4B9$;MR@X+`a2fCPkNK$tJN|}P0B2z&ziWSzQpkh z^CQQa->U;_+u9d8eEHsapAo(-ACzC=ALTva^NKlru$M10H{ipm^;GXkdpAQC$2`8T zk||o%{hT$$NX1?_=?0dAv9Z)Q6E27Y>u0?^w}^1NqfJdsEpt`_+hSB7l~&);ZWM*8#}5qkRG ze+Bparz2<{iP5VkPjKX#*jd@Q2pL-#zC8X!V&U6&jyf7~Y(Q#9q1dusDX7~-?kdpR zmjCSX(F=5H9kNXqhBgI%+kX@7VXT6`SgY#a*SS zaGZi)NVrtu$A(O_6129p9=8g%F19i+P%glQNDNX=#MH{fP~9)|QIpk{o0K%C)}uL& z7NVnPBw*yFzek@(2aOMnXNY%ZAbwu`=}2vzLo3}<^n^<4z1!0IQr^<%rH2QS_L%1! zl75Dg`yX3X%IOd3Ea;Bop2yY1ad&ohe(WUa9Oy(?uIjXiJ7vgJ_sY6bk$m3vA|UU5 z5rvA@hjP8;%1)+uZZ(FH_Zw0Q*$hz?(0s~lC1l08jrUn9tl)KC+i2$~!`S5Kk;0ME&xP9` z(z65=YGf}9oPXT^XjIrT+A*S4^i_Ygx;Gy?kTQw#jPhYb0jZr^tNTn$wO6&r4Eu~= z^_ZjZ`oJ;5vHelPCg;Y?(63{w8a_B)^&Pgbj@^8D$OdKIo9LF;~d*)+nZaYJZn7u{CNCN`D%%*g!o<` z9SoR0^LB7_>UNsma5W8IN$O{7Gt&N9>s;=%z^Bfu=k$>enYYi*+TLr~aJAWG;_+{5&+oGRS`7XrrNrHb(085T z?Ka`Q?7DsQmTT1fiTBp%m#A8WI^NmTx*E?lAv2*=k7`c^58istdd%zYD_5_ns|H-fk(5hh%ECgA6PPr zt&wp4psI3;*GFIc@hGgNb?PAI3VN<^PkT?QUfhuBT=F2RKfkPKuVvW35c#8-Jg(wq zk~P2Qz)`-WsANM=qp7Xwx1OiTyf!O-THRWyrj%cYOn>ySr9NlRH?`iICO)J)v|qV? zxuvPAa^>u`aFu}5^1g~X$_sjvSE5#-ru){&ckEpR+J3;hfb_65{LgwvpTjkr>!-ie zmS1%l?kpTOM4mhKl={iBCpa=Y=~NpK*d^Oe952^*xv0HS!&4W~X0w)Tk-g|FkGIiK z&~_>r%(GuK?=c%nB)`n_QrZc*e~G|X=#=uJ@P zOBV?}Dx2#^WE6lU79Q)E$pQxk@t+c;jxZDV^z#gkE`K90BMtS_~RAIKq^oPAo zUA3O;ZL%h_f=m067*PR z`T1ggMPFJ5@jAvv-I2%DVFvyO{LD+)+u^gX7kS5Be%|NTBLkVkbBD55=6lBr+;45Z zH#oUXaba=V-s;WP^zC*Dt$B{`$L#NS=eH^;2`gQTU6ijB_lzaj7|iY5U%|K^%3~P0 zr2*⊀+~_pvivK?hv_@AGi>mtHrq4q_i%dW!d7_H%kTW(L+gw9+X5Pj-& z=J(Myit`Y#B=3>+B4+jWwm7YQqT$z&MZ?ZO-6YfTvw1qC<;aE$^7fqx8l4_v{6KOuO+f3Baw)4?I$eUAVK7i0zp z`Rf@)@Co~g27j=3{(K_74}?Po|HA=)u2~3wKaIeih4}Y1}q8T zD+f-%l@DB6y>>JpbG5RxcHnarq_}&64_w2(W}zUvd&JQ~kU~>InM~Zq{xumFGb=MI zg%Bzk8JU2+u?e55#IwI%2mceKc;o13%g4gv;^M;W!oh4~Z_2{P%gf8c%Fe>h&IF!d za&WVDG;n3IcA)%I$lv8iymm0MH?ws#v#}@4ipPxVNeC=xXpPH;4 z{^}O!APejc3mY>l%Rgm%EW98xzxO?IMa_c`;{_9oE z|Gdh{#s2R%|8?iT-&A*aZ7*(P1?qGZ`p&8u$dO4EBTY5d1^;=M#1vO#cj5nGz081WramRLvEBD;;{J z-g0}J{|!rqK2DY5JyVP){U!5dU&_TKLrCxw8Oembo`i~sxC)PLi)A@|at0ZCId8cS z5)>h6x3wSn^Bwx?R(th&t4_PmYd`+V|JrSDlEHCWukZ5xc~6X@{~C=#O0MeW_!pz# zLUwu_GGBX~Gw<8o+hq^k!<8O~i=kMEG+905y-9v?0F&{}?(L1z$n0X2Fv*;@aj&!w z#2zOrx+5eggX$HhZ|;N{DnW57C5IBi09ICXNFd{LYF{33#WpN92uBbqXM-b61&^96 z@q!Ue5+PJ%aOx!ss-9s@@j9UhfHT%86ZI=k|S##aKAE@NC#u1uAM zLe--fG)sBdh=@t2jIV18pNbBR_Q`|L89({VW0DFvAd7y=Ph1_{;YaQl4F_#agA0)F zW(=O7gvYUkDz{hCxr!lBWr3?VI0Em%{d91D+9LRz(iau9un^8s=siR_iw+?hEtz}P zRs8kRZ$C_C2JvtS1S(}W1U&ZmQp6P>$Ono)wa83bt`$7S83O&o1wD1xX0@SGetb6q z$%B)KMzJCkpem|zZ7n(+Wza@+401Cc^tL>1DAjvVF-38F00q1xnVEBnAiL!_l?Va> zc!)iE4+%;Y2p15Z%@B*!uB`j^e-;BKL3}sH?QA~iL4iY`V4^}GM7mKGv@PKU(N_fo zH!SIt^!s}4OHI+DU}g*zeZ**19<*-7z(FfP>AH9Vs5@% zJuv)gaD1dPps#qo{Q=z;rF9?K?bot{9>$YDqkzXL5CGHGIR@__f}n!^rxq$JE=M(t zkoET>X`rXARnAG=ICK%PISr665=^SEl(XOpS3m$=ql#8T=t<@NQS|0JlpxmgW3&vX zFfLuOHa&;j-ECzq(1n2yTob0K$#~uv=Cjo^5}a;*`@>Wh`ISEs#x+>3@*T7r`|_WEYbX$6Y5TVBaS+z0L~geE zU6h2Z?4$;9^?Z+6KhayuP*%(ZEGA09jX(c|H(C8?m*0wWfF>Bo%vhBcgjK=hZVXr%+h;HU55!(9N0X*o~vp+lfIIDm?*AK}^x!%=(QGm4+*0 z^cFjXsGljX&U5vp>p3yKZ?5`$8!i@x;SYJDN4c=}Vvf#xarT!+d=@ zOmB}{hi~5G>!8pxP$=gW+WAfUh>+z2_LKl7sk zfxlu-*FhfjHZnq1U%zx;WBu{q-KxGRfna{qvuVK`ub&x;w+(AS2W&>X-0W2l& zo+M}U9a)m~()pO=21}37RRcx+W3uHK{XG>2+=Bw)J^ILBDHnY|T@>64A8%7y-IpfW zW+^&RXjQJQlwNn$J|I_5y4ux5%?$rN^MQ(V zJ(>(yJd4dOKLx(t7bTl{pc*A(J@oygecU8p;e^+b+-T|<|HWdVs6nKB_RRPi)`0Mm z196TleqSHg!_ux5;fL%ex}ZQsa-;nva^F0@Xaf?!Acs(1#ts&AhLa-@o}()nGu{KFG}e8ymZiI+?Wd%u7WVp&1>eZmZPwcp*? z(yEz8`FldxISnWo9w#^uZu{9&-Rr^>9xdM(SUZSg&CO2|i9bkMM>7eIi5=}G7UQc7 zoXV5bak88;H2u<*U^Qdm6nhH{JIvFih{QQlcD|c}^-CO|{d-T~Xha)CHi&Vl;9mGc z=8u(F0Hf>iXBeT%djA;*RNUZZ>$n_gSYLqARV!xmWyrOA!0_tI%w@el+3+W3z!+>) zus9G5CL`=0Mu`0snc@~vp<*waw*o^ap}H*iBfN=QkPMOlWya-=G42Y z&=3s9BV_UXK~*!bs`9wnvoeyhG9JW6X9o{(wou;nHCCk1IY%|GyhkyW2cQ8ClkP|A z+@=|BCV3kL1*<6{$vADc@4Z!RnAzUTwJ(L5b|PN`R|l3wr+AiHjY@{DiBKD`aj*gJ zkTCg0j9Z|a*Upt|j6*3Z_7jgqLugIvk53Xp$QzG1SRTBgO}?AAaR`IP^Sv+}blRDb z<+-X4DBeVn`#5Y^vK$Y)pMFd}^gW`wTL&odBJ|Fb!{Jl{Y0a- zd&%C5JXWrC4cdC1o4T2NBuC?`Ui>1Y$q%_Epyp)KLNekp=k%s7EkItlM!zD}MN_z} z7y!6+;>!xwQcH-TVE&Yy--E+zP)z&(u713!Wbj@ykjq7gR>GYd+$^%rFW+-him(^sY*K8~p=V#r?9KY(0OTDkREMb$LjYcqKBEE_A8X>RcJnBZKUi3pNsS;YU?)0-&L4m1Z3}kYR}Tm->$c)5jLp1 z3WTeU&bXCeMv2cO@^6-wv`=*vE19Zal}oiMAe(jZvb*I|&%N9H8Q)CBG@GDfy?)V= zO+99_+!@~s25$R;^G$zavaomkW*;sNU3_D%Ol;bHV!i^d(FX6^cd_C`*SzA|DpiKt z&(zL4xb>#CzJ+{k|MB}g@GG#TiO{(FtHLKlb}}i^=WE65H)N9fluV`3O<&ixsQ1S7 zy?Iluz9(P2i?3seOnbBGHPLB6VAPbrthb?S-$AW-d9qDB5w_Uw#@XBGio)`+RF;rqh*nifm#lpT73mXKH*kQTSy{_mJh~R%6-L zw;@HvnQEu>&)+9IV`77d8(hV`qFJue9eR1S+{lQDS9)UZm3PrJ1RUadb3?!!OWn%ttVe<Lo3*E8kc!QTuler! zm+nzm`i>`9d!y+>eNx?LlHrkr>K$PfgeuFlbB=fB7l1eq(Vt9dU{iR3z1RH`w;>w( zoa$bD&}j~(&r6i+wu_@7rD9|~x8?YqI#EBQdqTB(HO|&NRo}FapE|A&w#6`pO<(d^ zl}5)n*bw=Td8Jxc7gFkzE{I55)$ae?@jOfQM~A3Xn2${J)VZJD?`9ELnkJ6_=yd`n zV2i957@YNw@C}64l}!ryR#V?g=JZ@Z4HiyqFkCX3OGPymDX zIMVwJS&+^EokcRk3z4h9er}%qR(8FSW~5mv}RiBU1W$Y_7hF>;|*xq^cEV z?;yy16(dEm{_2eae<@uPE5QK*w{bfbcdsupNr7LnQKpcmZo^@p@Z52W?fc#_Uabj<3Wuw-nj2?8b{mZ&Ay2=)+{u64Dc;C$e9*~Qnk$GWE~~v{oVrvEY+&_G zX~WH1ZCzc+XYGsQG5s0WM>rQ?us9Z0_$M@0``lY|jci!Lxc39l@T>b0YOeSkR(sW( zkr$+HfhFz2aM=9(qj~_7=QrHxO-__h(T0o){ls}#`c@hSR9Akz_`v-CUqeOoip(XL88YD=(fRjbGdfn zv5hU2vn3;{YyL%;xay3Bg7c)hq)y78DUor+I*u{(X7vuB0?^B8iLh*gaAxSFhTkb);S{zz5eDILmr`&)U=Wqti1O;F)FhYy&7^1y+qGrWVaNjAk%weaM&y27FwmWX zspS%u(wnYN=k(jF*sWc8pHMFev2O59xhKDKK50WE-5e_|T`(AXMUOK5^jM}O@!8wC z3MvgGXqCT~WXsg|eQP>i6)iovnB3lF<5%gQT#rXI+UA2uOG;d(thmYD-{>}2(%Sb% zq-vZk=6!XI!c9w?qF6#@f=_TAP1N@)+(EvkrM+Lgnqz72lqBLc56Cn_ASNjZUc@OA znJw@cVep7Z#yy32YzQ~YjsTw8uV@G}n(<$%w&$92eL`CI9>}rgX(KxHw0X{E#-Zqj z5>6`ovV9)=n3Z=pdsYGO$oSFAS`U?zUKHMGswD=R2WlrYDA(V$d0Q!#KDwI8K%fMW zW=6yWCY_JXT_lg;RhFF2Q3-D4D3`GkeMDkvu8Nj=g)S}0-6f`Tmvi;VPuo?-Tyamq zE+t#KdN4NUc(4?!RLxT8j}_-&t%f&2NVW8m%i0m8wIi0c*rJL`$8l?X;JBp&NClAD zpBO>h_M48w7&oTWzLtg`aVU`Xn#&27OM-Ku1oxTpE>4=aD7rc2{j({C^y^zx;x`hO z91dx1yV-tKwh$%xr*ydmDb+eCm2Eup8YmRrBJ@l5sJky0Ss@%byFB)&e8B51t`IT) z;j>z~M2FAX2lXFBex3H}G;w6OEhcJ+YpUU5gR%4nrMu1a)3QO-)Qy1&B@s*xgRxneSQ_`!Rh=}l9gYaF+>?hCfv$Kg=Jf#Kc$D738>dv3R28M@?K%D zrn_WR2h5_I#Ra5k76+<*pM7gV?sB8<<+`4$8~FLH&QiI_$R=3>WF|Kk1#9h|Zz_jr zQQNfh4(o!z8aej)dZu6Q)Dl6TEk%em3m04ZnG#E?>nuCEt6yER-$?>aOhG*@SJnj zy790E0HImZRebPh)l{Qp&1}z1nMQbr8*NqRE|t_{ zJ?>B<1pAHQ!o?HjM*Z9Q8JRV!{;^pS0;(+UI&<$`mHLXY2#q&U=z~A&S?td`6%cnt z53D!19oLk9{A#__^^BD1<(~84c;9KOX_ebDPq+Ha*uM#5si{U{hEOWgXk--Nq6|(U zoA)ALARoRFwyn^I?8~Q#lWLbg_yCRjylhy{C?#@|RvGw+#ijdKhfIkz(S*bMhmt3T z1^6r*4-_@o`~@|ZN+0~HQHs!SCyP0lwtj2m$IORUmG;>sxH8MAiw=<7rJb#3AV7-=TfhZJ5u&dJ?M!`ucfL7dU%%br# zIyRLzBU%IsdBZL1qamUO7(MM}CkrNH`j^elPF1!c87iNSnsOJ>_H^reFD~&G>MwWr z$=$t%StE2m>Pip4Z!x6cctO+=aXx&=W|`1s5G-HMq}yKgRBQ*~Q(&fjz?hdkr3g9q zZ(RF663%Y*9s3QBy(ABWZ-mQ*Mxe3)^aTFq4*c)&`TpdBZ4idsU#tW`pAdkylhfLb zsB0AV+pp@A&sD^k$|F(w9$@_~Tfpc&)8=5$3~*wLF+l6!Ll{52SNF`GIy9v}e0e4b zJv^lEpl4=4Fo`}^1p9Bj0><$dIei&EL~x)&qeXlnOS%-kf}*vG%sTi#rtQN{ANI+R zJ%l}t#~!$1L`bGbq(hk7D;z}?4MFfC_p>7U8A;5z)Ve~27xh~D5xmZJAPgcW3$hnt zCP3Aig}k5Khqk`=(VeF%n__k5227)_&FgX-zsrODIThSrddy|T|GgdqbnP!zT;Hru zXq5SgqF{mm_`Q1(MnN3b#D0#hSRuT&s7kW=BVoKj>9>|>8ObaP;RuiqO@hXeJ}Ldf z^ozLQCnFAq8(=qzNaV9RpX+&mUIVSfhY8Gg=kw<R z8YVbQ!L3t_Pm(#^xc$)u=ve+eYR)zY>a$VEnSZHSE`NpoYlik zM=eB&!Si3=VNWDW*`1dqmEFH@)0c@7Kh^}3s<-nP-m1w=L zhec_C!UtOzuqi%I5qU7uNe}|@fYko<3?g>MO0_HxAzX+OzEQ5j2Q0J7mPksi^rfET z=BUC`o7rg$qakSL-davn5&Qg%MXF7GmBNJSOPNM#pec%y8x?4Sxm)-%ci;ScG}_bL zR+0_#)15op&pt?owZfcf^JuBQ{?Yo7)Np|cB_)v!SQlyuf3KQulH}plMo3XPDZY>( zJ>&qJoo;h$;%nR5K#uws_meI96i$l-JjfGd5bFW&_=)}Ohgc!zh8bD-4WF+L+(!Ty zKpwSzr+QN@Taj;_>dWO$P4OY7(3yyQrjVWj>QB#x%gxe;*7g|6k?j^tVF9bip1t{& zHb7O{RHTTh!6z+^Vp_rMY>n%_;m9G30o#FCPN;T;c^>uUpd3qxO~dtzZx&;U*9HhI zfep5$is^hG%!WQLv{Oc@nYmKl0BpVhfYgg0_l3@04FFYUtY*A+4Fa=*`zKR2VF1o+ z`>4m#C_n-?PNp4{00CACe%+N&SpMzJfSA;L3`7hjr~U!nAZR!3&{z1=e(TGxz}Yt+ zvUU9 zr|s84zz>WT<->Y%x&0$^+3mMAb+GLhn*>%)(o-{> z_iB-1&tiYGF@Q~~5~H9r2HV+fT@VIh#!~4l9)xq0WPmM1F*;tkLW)zEYqr=`K5KIx zH?sP7KO?lFVy-_kywm;yeAXVZzVd18q;DV_5Hm@48pXy^pCzZD^2vUJ!h)`$=g`Ax zW|s2S7`3PCY=5zBxbXSClj*93P{!7a&C*>VOj;~xT-KK1Vh z>{a$vzewCsx;cuqzM5#4fsvulPon@+_T!tQ-Tp>#X()g=&n>IhKNjIjBln9TIT)9b zhR^k63hN6T^^0#f-`-r$ynOHBFHi1v4(NU>(6mMzYFn&kuaw(e&GrP$|29&Ib@ZC9>da#zV&H++AX<+7#3uvd?)OMpS}5{~)uA zr9t3xzU)lgtqLq`o#6MkiIFFdHQxs2`b>F`po13l<_2TG43`-lr<;AVC(?l%eu zsny)@1QK&W&+b!v&)nz%D%U5-z*8cD7`W3ok~IFWQKZ7dz8qqh86B@UDXG~}O`Rw9 zJQ^^(y*_n83-Z(f(4@HJi(UGyWO%06Md2e(^SA@dNvysXMzI$MqIFaj1{CMfK@5K#QGu6i;I>9X%t`@PiZYhVFo*Qiw5v$|bRhn6~G z?TP|t%I10p^W;WMyqq>iA0){kN|EQV?lqu50bD{Ei@kH1s0q-xAbKU4tT!-MBk&4H z7Yj2kbLu}I*Nnp)n;`z72&SIEdn=N6jCW+Ba#F6jAN{JV451a-+>`+Jy3h~ zu7C}wR9pUKflGFVgh}#g$7yF;%d+A6WFzf1Mg$LrIQ9!lPxIw2T;7!U&FC{X?0IqlP3F*fdjZEhxTy> zP9THVXaZhY$bN zUQ5^12U(94s9Hz|2j3r1*!&NS1vpG+s_+jl`vajrkc-q{V`z96(e#202}2rXud>=d zVa|fdI|zLXyHWU-)t5oZX`lWN{QVA$Kb?|7GE5|V7XBz(cC^A`>|cOU{0`GcWBqM6 zV4(i{ME}L1@gMy6rxdck_a%tv2u0B-Uyc6-yM18Y0$52WOsf}$mR{YVCBT6|crUB= zAEtBCo;z6wRdW+2@p$~Fqtmca1sOO@U9hm?cP{C4Z&>;ZEG8)T&+4d?TB#U&`J8FJ zUmTOkCX8Xg%{ks#R+sq4lHn!z3O+DC%N8VPg9w z*u!5c>{3N0w3&gT33*ZalF7Pu|JfZZ0*1eZ$+Gyx=T?9?$iL5dxu23Kf!dp?!5}mD zWev;2oaGV_v*8Yfpla_$R(sRiKs<^GhU0*5BjcKNHQH}NCJhFTD#U3_Pa_=coGpOG z;=3s_6%ax|IVu@NoG?Lo#1C}6C`^gm_X=7IT*yK=M=ut}^fEz@u6Ehk(Iz{ zwpH3KwgR8Z^fR0w*82lG$oq-r2CS)8B_6LI7Wp^?!%&Ex=`!m^{knEKMj@*+om40a zA`hu{+N62ud93aW6TAoC8I^RCxNni)++5oD?(YG>&)1WQ4<<4X_uq^a1f&%v{xH@B zF}OM=SY)hE;4pj~L?YSCoq(niaysj-gig$-M&!Qrh2Unwy7uN`bD@JaQw=w$SiP7= z``a6GMV@(xT`RVfF-Pb`RLseF0<$pe7p7Tm`z;PDDh$}f_GD~|g<%<#o$ZR(q=LnT0qFgRhA_WfGzwy+P`BuW3NTtw@!ka--#D6^eocL;D<>3>}-@l_GA(mui0NI;%G5t{U zX4EL@PI*nw8Ydh{!|iqD`rt(mgtpqYkY&0ozI9O6X%k!d*66`>E+ur7O>s>-2|g9 z4`Csb%H;$d_NzMXTvx2Kmb6Fm_kZ?BJ3p##w#5#NXT3rU%FYJFCj^6QTtq&dPaS|; zDP)abrkS2Eq_HVexNVqWxsf{3W?no6XyiyT5CFt|XYZGWx0>ofu>i z0>eaUCN2ld90ZWDLhzcJ@HDXC(DcqieKE(TlW0$~@1$P;CZEv!}lXoS_=s;;4 z7v}?#*z1+FdCvT;F?zenfB_5|a4JvDB%3K4=b_lG7y?|~ zsgFwR8774j%=wQ%)K&VN6c@KZ(>o=pia|!CH$o+lv@5F)0n!DN4+^iAp1Hbxi7CZX znO`<>B9O~DR^0%wh?a?|s$9B_cc*fZ2!*!P9mLGih@Hoqs;*C_W~S_=^z%J0PZ%O~ z6<{&gP6QQy>M6hx4N^#1q{&KP{!`b>YE7oxN-L2$^2??y@=^@MPy%jOvzZaK?8##X z6;^WmSc?V_N8(h$7Z*`od*pBhOLzdos!?|UXEuL~oFo#iLGCM@>3M1GL1^Rs=5csURASCmq}X`j^CjE| zA3@-00q{kKmP3EY!dfnO% z(A{c617vD}6R1!5UECs?C8FzkFXW_!C^nNU_8yLM%Zi$bp^yGhFk|#9E4?R96wJ|8dxi>!1t5*AF zIl*ovUSnA10YYjBEH0Ll-^_lRxQMQ22O;gMzHbpnU`cz*Tl-69NBK$0Q?n0m9p&fO zl@RPG@5Dm3Z&d%0oBmrW|2)5IOl!KH0yDwu;0zm7Yssg zVFVl|!F)~|pD-FZ0MiJ?0l*m3>rP1pH0!hM=vULe|1q%75May|hXoeNKRaK|oY~6+ za@3NT35d}w08E1if*ZI$X5SWHT4n&L(fA+u@>jZ_!qA^B@qua&q{j#ASXW63elY1t zF&xu#d4Pp)y9`vCx}*1%YahZSU_cbW*`kcDTL41K|Lw`Qhae2@94mS_D2a{z+2U=0 zic#O8uA>2|Fql(Of#8A^<4ChKfyd7OzeE3{Y@I= z20j1OaC2ce=Xt8`rJ?5(ADY7vP{qIaGXfVDJoBj(>l*-BPV~LJt3(6!p6v;>nbv~bedSO0NR5)~dwJK}~bZO;PEWSr?FJL}_b3x3K%(IfQq*6@R zbv|H2_crK(1~}B{>T>^@L(e^NWmTa%VMY{S=AJH-?{>IZ&G z>|}w?`&@FZotJ;(1cpbLmQm;!^Grt#9P?Hrs9Ndv37!|kAh zHXT6P7w6gGAPQi&*pWfM3N+9cUce1GEn%e|0DiJNeF=<=GtAC2i2g&y^xA6yzWtr(}~#Czd?L8l}$H_RwS?J@bMQPC|^~u$dL$ zzOQH;75a6`x~6ayBo`W$kV-@nJySiMb6vzZtrv=^6Y1<@F^1W=on65C7_5j%q6{D|^GJfK1xFdG0;U9O4akx#Mz7cPjWgIIxAX$J zl*g+*^33Fm7>V1u_mv7)XB>wVN>2TuxWjdphN!+EckR;^xpBW-j$R_O-tcwJPPN>e zEI)-{YmD6R&<+rxJ7r+t<{L=i1!?O$Y@KBpv>1W1MQ(>H2Gkvn(pi8dcgHl<;_?dQ z(Civg3U*61@%OWKJdAbwTIHd^WH7h`;D`T~oyy@0*42Bs-uTSAbii~7_(Gd)$4eJL zI@}MQrD$B=2ZB;-;%t-bA-B|aA5t<7E8a=iv4 zu}lH@CCT9bw0hBsS$O1rab>8e#lFUU!8KJRFC<0#)4;NoWp&MbgL5Zhg8daVIfS%N z7EC5&K3qUP zo=vVY2r)248DjD5!(D3WmLI%J2X&&cF-6TIho>P^&U)W^9(N+v zp1^GrFBrjSKyIw!V3PzhIVJ6S=wc+lVtYsJ#yDVf10N*OxS3x(>8Rp16J$RVif$2K z0HQShk2$Xz!WWswxa^wARI5`3WPp|0yxPS^0I9B;Rvy{#`)7 zsJ{Bh#sia-=xE{3;hk5b@v6<~=`xe1ry9Jy<4dMbsjVn%OL*Czf0S3Td$&^4e*G%0 z0Z(ct=LT+jWl!B$v$^0%Q z)j)75NV-FtfQ2Gg)m(eP8B`~G48U0+N+X(i2vvfrw zwWHWwmpRzJ_hA|aMrCl>v-?0y_?L)a1_MSQ-*OSPg8wfbEV|c`;dmJ4Vl2+xZ6)=p z{RK=nUl>_dsu~U0OYk0>NeZrI`3+lFyCR-M^6$_Y_eH`&ER6!GzxH1K0LlSI_DsW$F(g#Ji ze)&aG{DQ{wYX28O-vD%dl@p}6EVjMpLy!97kM1J14r_w!%hu8cf_)H%PphoNI0fbq zvQXBzfNb)H*is5Cpq=adw2c!cDwQFNyu(?H|7CFm5J1*1?tx^yVWSVc`6IKsvIf0YQKf$3oz|D}g5VMnvL;bC#Ww5KFON)3uaJs%*|M%EnC zl=c?OhQJa7{^YOMx##OfYGvRA9^v5@M3b}_OJ~XAG@5^uBK1`JY}@QV_x5#Q7)l0lGxt!`(C_Db;v) zDrZX}>q9ZcNBKKp_Z0^?BKXFl&o~o+Qt?1^73z(le5G&HMjJ4}9GBIT4KLl6fudnt zj=KGf3X2$=m0a%!Cw#tU=U6bvCqowGego0-0r2RNRDQ0RszV(dGKj`>%ylx2vIwq+ zM?eCj=-_{eQZ?|5Q1U$&E;T$3UHYGsr?NJ<^1hHylw}|Uy$_NQrQYxUj35jHQSyHt z1V=G_4gNa13V{AXTUcA@14gkPZ?O3#a-KzqDa3|=Q#k%|poM%#B*`E%PZ@g-XtXWV zeZv4=i`}Kghk&%!z!Vb@E|&m6Ui{zhIFjQsTYm&;AXwDCFoc~aml${ujD+zx7$~iQ z?;qlVBb#Kt9^i!Kl3eJ#G+8!;bsay}Th1BZ3Z%8{}xo zNP6^w?yCfXda%fv*a{GxUYr2|NGE``ynyPJFno8iXrFvNt`FN6d_lG}TqEz9uMqVx zFeHv^Vl-x7vx{FGnwZImpgidbOgaaN0QXuWX5k<4pNM@MF45BwX|)aE)Io!e%iK-M z&tB(t-B^)ai2SbMLQNb+TVJaT8SF?ZQte;UKo$c|oZ@JGeu4+3Ly-A8v=w8}0%8g6 zWFq9TzW&_OWNhF*x_a_ck-&w)R0xg?IF%YR(kT|>b9dDCZB%zS0S8{ecdM!L4#0gh zWqrhc4u%?imm$N16$eB&fk*5pAgiexb_})lU6LS35K06S7xs=83{MKUqGO)_N2JZ2tZp@fjkB$+B?D4EHa z$W&$$G9{Vb^Hx3kdH(y^`+fI5{{Q{%<9Lqa`L(Qd57&L&*L7a!`Tc&b-~?If+yl^6 zNpUkAPzDHRr>w+RKvq=$Tw%p@QaZL?1%CB_8;7~DDPc~5RG@yJCG&l7R6a)7ggO-I z=wkg8tLF%_5HD;t{^T=hA(4ME_QExGQO5pz3-2e6cfsI`#mm8s9wf2H<1E2_%aD)# z_x@m{S3%zYq_*T@ImdsU)9TX1z?6t8umRwIAO;rNe8PuJK63 z1`CfT9&F*oW#b99)Xf2ZuI^~?5%70YI>DiYpQEougERUB4l|Mdg#^kpTsWj;4JI{8 zSNOZbbIf!!iWlc6pH0}csRHj;I&q+YRUt^!A5eLp;x*j&eCxnB=^8pikj`D^{jo`f z%^ zUlsry>#v{uw-FqP>m3RRv?1mBUBvvYl0zV zDdHf5hI7#Q9*9!Gfo&a%?T~Q_m4v^S_fxJa6Vf5qM=8SPNZlGzsmDs5jxCCN%wmISMRx|pxlQ~5Fn{f1Y>*htq^w({~pAwf1&yQM^=V^ zeWP&@eGvbr4Gi0xt8}>#rxpL#4Gh1&KGi=5wQXa)V1_W5VOHST@|qoPCm>No-1|r? zmwoEeIm?-?M(60lewQ|MOS~5d86K^Iv{x6$j6` z=sxtFEV?F}I2T57~bAJM5+fN-G>;&3_G9<<8F<_m8kP* zvp=J@mmHQEIx{kX*1xh#K+gO)?*o71&C)(p1cwXk&d=F{QSX{PVP9|hd-?hBLd5v0Lx#Tl*;fohXT6k zFjO4|I1w5c8qL?{yqtI*_&h9M^H?yd2j29|vF0k3<7w^GD- zxHE)R(qbV8jm2kJ$0in6+W##Y z#LWf&;`9F!2EcG1jQ{r&{(Dw1Bf(QT0!_xK;ue6SR)A-t{osEvXgt|yK4o04?Qh+a zN%CJeJDOH|gh47V?1{qvCJ^I)u?ZmMDB|r)rFAyPm@e^-5aHAML@W9r=S$FQp(7r63UP%K*Mqw~A(q*#M{ z2}b%+-y54!Z2J4@@Qb)aJMG<)EVCP_rBi}KtEA`YH`$MBUz~5|h&!}6*}I$mtBSTa zd$mPSMm8w(*$|Pt-NrdqM(@@)PfeCL(o@yo^ILqQwr4!9SFmJsPv+`^`QW+qn}qZ(N8k171XD zBrlLOlw?R-TrmJfurqcWNVxtHL!9|q{;TkL+rf{liED$Ud*Sk2`XQH^+e7d%vXuSX zv1U$dt{1GAZ2F=tz?GPG%ePXZ8VDw_0;wUXA?O(`v-SF1Gaqd5Sb4Vc6^$YGw@1!*wEjG#eYBWUD5dxJe4`x9*m}k>M>Q#9RyDN@u*!KADi!U+Rl4Y(ai4epr z(W|{&tJ)2k{&YX}2arJSEAOfHF5h7+HHiz!E9ce`{O6GI(umm1ymap{2+6i~dS%|F5UTGp9Ct=khA}PWrA#aH~Og zuDzx%dDitr^jiP-=5{j?nTA!NoaY|+m1e=8xfLYtAJ~4i89FB^dtOkNA+cNu}ObHPmyiO^JgxXUJ>ub zfR?^~h$G-~!*0^b!5?PI5A2LrA_2IfvDvm%%N6a+ett$rW#~X+t{d$?5j0v3qIyj9 z3RX4xKe95CCa$^Mceu)j=C=}3&hE0ad1XJm8lbvYw88p^obgykJdY_TQX2Y7tf1AY zR4L9Py%44- zvlm!RjFlpyr!?OEg3sU><%f?vR=OY`z{}CKZ9!lF|Lgvv3>|EUJ)x!-3JLP#@{Mgn zABh+R!`sP`p-7Jf$#h&^aMo@ktybomm2;@IzZDQEX$S!|3`Vi4el8rWXVnm8fJn33 z8I|d~$j3H_oGdyi9rQ^DnXIIab2`Z(r$Yo|p~@OT0#=o-O{_ceN0euuXc184#gLTB9Ro(jgWfEK>*0*Dw5G`TjB6(jyeZmhbcU4ipzURXpguIlQh?Px z&p-AP(>qNQ376^Hag1~d`{TG0ao|k0_}RUp@k{Jg0Ua z$$e|le2LnkRxL~%yODK3{Xe5I=nWc9v;6kb^bo@rRXruH+=5B>Bz4xAW*Y@nE@5g; zEA?gP2C!xz4X15{5HVV(rHr(WnlvL?sPL|XN2Sc^@^p`9n%y@Sn=2I9>oD^9f8k=# zCL}y_k;qLzM2FWE=!ejy@9+7C`X{tZZOd~#4DF@~T|F<_e|qk^pg&?#kOJ*u{(%G) z-{l7Sq00%xj=O;63Wt13{(Vr8sTYIsMEN@{*LlJ@RQL;~xxD*(ta`xV21N00Iv+=G>L%BoP<-w1S48??16YBO=`@J@rtQsXy`1n0-ysWLSkNrOf`bsWeSXEgTVXxP;E0SC2=1M6B%v8c(p- zp;s=xB;li9waDuNo0TZNr_XgBF(2;a!a!hDJmu`{7Z4z(L5|RAy|1rfse#`IG{- zbEG;)cI?#c?}oYa;^WlUKhH~)`?7hTm441idHrX7jQBfte)~bUVs%bEUbQYtwCN?_2?rWqk!!J>Cxk4&^*4jfC6Yiq)@p|t&g{8JCq!9 z>mV5}+5r-wr;BXPU2;YfN)AqtFm_sH_q0lPL-le(xCqAj$iP;wcW{k?VG3#|!wd^b zK1%alI$0xa9xL;}?=4ZOGP zqe(j6x#;Aux%k+sX>}ZO@NE&b9sD5uaMj9E|Gto?Rn<+~im7q@|sASEgGc9vnPXes3>}+iamMey43<>mvx6<7bO>fnz znH#ju325cME0sUEW65Mytf02}eX2`W=DsSK(|8znrI}j^cU>OT8Y)M@ti>WOEOHAe z(N`awxK|5RNO8xWr%oV67ef|<9G-&AnVTS+zgE8kw?E?6Z|roh)N9i*$8~9xBJy^% z$2#+;gOkQCpRPtFJ10~1^U17r#eQ4A-K$_a`Er7BxiOk;DL5wX_JdmBVNmRSY4ZU^ z+|6Y~3)WzsaZkkYQ?onvrUhQhmIwN&M1SWa?mhemeM6I}X{8P|$M3FC@E3!8aW!x4 zH+=^y`xp+-mf^K3?s8v{J>{PfDhP5z%)T5mTRg!MvT_P)xt}wlBw7(dho1C=``)kq z+T>5A*FD)XI-Zx0lZrm~7-(lAHd!_Ic}FhlX8&{*AW*Lz*uZCS#CKGPrt3~#bWkYc z^H4dje{e$d`kX=ic-G7&V2zBix9Xw%@#8%!KbmhxsbkjKEiB&L>dVhMQevDnxed8x zipb}?E6k^_see?g8b2J|E%{insi7Lm_9yvw3yeyK%P{9a`Y5?^sp*}Lrj!fXqe|cr z-rvH0xV-og#I91~_tHD2>M!@RIr*2tZSM?kA-0>EXNq#5l)L|!*~f8G*Zz*c!!xff zZnJX}zh1^nmbJxeEpQuBgRrmO-0+M?Ta;7n0eP=bnW7&d4QCmeN!km|L@>ReWa$6= z_#6ydEPKXJ!0=u^wzDqgx@g>gv~vf1yv|g8-H$qXJP-D;CKZwSvW!Wv2~W@qOQhs} zr`_*Z`Wg=*+FflBQ9%GBHa~;Z6WH_eMcnKwrc{~w(I(E_O08t)Onf)T=v_r-=&-~Q zqy8Yf%o{zl);#-DOfwPNpi#$X<(y#R@~Ppi&__@@Cv$~PhNDGUHSePC8jpXISqh}l zREb!H!9wiL*AEUmvcG0I`fKe!*DUV%A3R*S@z~?*$&Fpa8I9JMq?_uYAXMEjmMe1i z#NS;BD>uaioS7OvenN9nibYv~!1Xa1X6ilPc9^%^-5WCcxWgMBV3Nzf(BsRrB5tU1~ zw_(uGd>2gK=gpu}MpnB@ng~1q2}ceQQR2v7pz@y2jxd`U+zg5|o_<#SVHrkA&C{)j zx@~^xY=OrOl2k`F`VAwYWJ?iHsx_e*JTEO#2Pc$mK_RUNl{mbr?3p)FWbtlz z8*OHa;dqFDO-OY$NzB3WlJOD!xso(jrK6osTs5D!)=@rkEpHR_6H?D*3C?r&7_IEz zm73N{qqNXf8Q69*r!XoQ5z1kf%{6R)$@`m;CElHEZ&BbRjJdPnly z@#JItmomwZf>HDR`a8;i&+b7-B6B~xHynyQ9<21GN}fo;=u@)mMZ#*L!6|CuXO~_$ zu9Equj|9=1&W>($T=AEUzc`;&Nh9S-O}HlgowO7~bqiBGuqb{>QZ$FU}1yi zgjfoT#8TDVSw{|ThTKP6s-73jU3t3Vy@Wp!Y*e)ft}&N;R366LPUtEk2ul`M&)w;) zR!|4=S6Jl|9row?$XS1wd7L*B`dsejv*wIA7M~ zq+}>TV88Xz#-V=4*e6T&Q?mR?mv_k?&KSr`*w`M@vkqwg7@!;Zvzg1XF4^@;ZVm5P zHB{GfK>J?!b`xTg-d6IlaXNO#jiEDpUV-g0Ti+(KpBoBIaafrGGD(!xpzp>*HMAFq znt5tf^62q%qw=}!@!oxA7^zLu%MV;nRUJ!T&%~rYe?(wRj)3tbc9qVe}sorkvImP zAEoUf+H(O%s84&`Ag<;&)d_GKCZiUTNF@-jCgc2K5?es7Rmj2B{ncH-O;1Lih{o|w zQl%>gm{q-%ou!tnx{V15++zjrI!{=*p{m6tSk_KiISm|C4tdLa(zZ-T=6T%fUzKA; zm1<3ItvP<*iu!ZY>f!6ZxK-w74wEo0c}qXJ+|;eS#LIcA_eUkGv`2a84~i{|MatyV z8h+1fyLD1k4Yq=vN#K?7X?brhHZDH zXj`d?q|Ztb$HZ4o1Q$9;fcCEMn#({}$KDK!&x#%Kb*pfcY!*l|F7IjBU<>dV)zg`s%OY#cI81Fa}wYgeo8M8## zny9q&{YY?l*(xbxDBG#Nb%~`^?$OPYFwa%cEmf^Xi6$l8e!6kp-=cvs!KDIqmm6Z*a-chakR#7llEBKpY z5ZiWzI-*$?*6|j}x^t0PnD!t91Le`=y+W=B zYp1K#LdQbLr;&U5X;E{ip4z3cft1lEC3@?pf=u<(EJqSlN{#F6!JBVpJvYj`$Eun@ z-uK>{YaDU(&o88ca6Injn9%Sv{cYl8zQ^#}%m{t;B=T|#{l3$kQWM4q=FCTM@H3wk zrC}#b6$Vd60D(xsrT^KnjeoJx=lkl?Vnq4vQ4>5>hafJ7e#eG+nU|lb2vOi&r`5oF+ zc$^N)FuJnW)4$x!H(_@2ff_njy=AW`oXF{{vjmHz%Lh|`W&6s|8IziYdx1}=N1Xg` z%e`jx>|M1R4k-hz*MQVf!SWARS6PdnN0_olZ2^m;{MU}osN1Mh)hcf@IJ=s?1%jEX zX`WCuy_Q4wtl3x6-xpXK(n71gwvbI(c>p0u0DbS=!`=ta8M0)<$V;4q9kkv`?kU>; zyw+Ok>O(ZG@+K6f+3Bz^hXPFH0`z5n>Mx4AL09s&LKjpK9UIRFD4g}3I9W63NTLO< zc~}p3^^QHl7UMX*>PH=$q6591Ee|?%1CR7H6&>}ct+KR*;mRPr^R*xz&@GFt5BU1X zB!qpN`OLL*5+o<+l8Kk@Aj}*l?%CHGotW`kBvq5KzIGtCgwFpW*ZMo2?ORv!@O8PY zb(k+g;;)HQ$1+{YYicfBca)0iEK2Yj&ed$TAlD}p7dB|#C&jm;mvH0@E6!0>2!a}8n(vdn3UoFhIOj?22kfXA%+04dQOUv8c<_2Z$ zgnasCDucy?oRd!v1kz&Uf8c48ub&JJG;TSOnX7X%tz%Xtjp1*yiA4vJO~8i;;Ee~A z9$uq)aB zT9h9yIq-(rR$AhXO`3dt~AeoayJEGrctDLj%L*iNq`mfxj3TI9)v8>v(vjiGiFcP!*t z1N1s^7@~|rhG<*JE2kZ9*r|bIFB&JxBB>`8UN9ZHeK>`HS_aS)-RMLzh;Tx z;FpjY(?7NJyO5%|@ArI~LXocl?=s_MCM)fR6`9{9ma;$X=3?=;-F~ynDX9=!4wAAu z4$NS7V00YGzc~6Ex)O>YKM;I}fXyMRTLEYE4p;v+ma?-W;$i5|`d6T3K@^+gVE_xP zfKsmJ_ZcSNXs?Fy@6M_;r_`BCD-ggaz_JW4kbVs zif;h=mgrUk7SlahHonjm2_^$;xv)YOPZM6#~b*}53;Quqbzj`6pn8LUEP!kv^0q5U>Q6nYXtLHp- z7&t^sQhWg)XbE{UYh9tmtr0MYo;Vl9oM#8+y|J54Hi&Buu^2}Ikxb{gskE@exT3#H z?f;*!7fA3+^rTZm$FaB)a4LUCNY@{=LWc#j2{oc2ak>g>T(M&Cx8!nHL(Wpw56PRR zL$ZibYvVuAUHr`~Dfy7~?eR)S=?mQ8!#xJ+B;8e~GPe3=Fr(H(LMVUS3lhSRcO4(G zerW2EM}6@ZSD-p59s8z?uBV(p(nt#@!rzloCx|;`yMYtokm94yFTm|Jbq54|Di`I?fmMzM_>C6=>nO2Iwq-HpKgS~) zh?s1?=!X0!<6RI}Hw;(#LI7j%jgbbViCkyols60`r!24+{NYu)2Bcdm2KF8b_I>Lp z!ttzT{Qj<$Yl(V6C@|8ssBN@s^`_;%8Ck=fGDpOY`t2ahz5MMa; zW!Qlu<8>YHW7b6ZRmX)F{iH`;-64xS+4mknsPf zQ34ak7XnW55G1dy1e(jw59=pZKtJ(G4pVcxZV}Dm{><$1{YCqZt~jFRSA+_1I9JIu6hXVEk(H=FklBF-4wU{}O) z26@xZ8mO(8kt~u1CRZjT_97Snf^&`*9^#dLb-~h;o=jO0M3OE2_3Ar$}W>+afmI#%na8QBducU1cR2}7e zT_jH`NingbcQKXxpFu0H{4x=1o{Lej+_5fehO&Lo$~Qev$j#-k*c2axz44eQWN3y~ zuW*}|L(Su9^^KyvBf-W+Eh8}U6$WZj2OVjRbd}Xem14}X9w6UZK%Azw(}u6?O#Ca( z{4+uZiLD+W)iiFZPrqZ*36MWw0*18`gyawgMC-*X^h3Ya=by@3uo6$V0I*GIUGWRo zboZA#2M5d-K|Zb9hL7X5(l)iJ+khHZ1WY${5A(O(Yza}HfyUNX-H{?00l6yElvP>k z(0!`|a?%hazXj1uTKLq?B+Y-Ix~!?8NAVqB3kD z^XJX^M`!@s;zS(?9{FC^Y(rtc?gOQu2f44uf|-WA}ot5SLVabDVqxe%g20 znUiwg%8kT~GSCa;j)rd`Z_scB))!=x1m9Q!#yGdfs?`QKk4m!k#^5RdWq3crd2t4$ zVLyxHrtNsvK&6g^$K1HrF&!DYhS1V0HuwkDcfADm17Wa=aj>o*tHi%uOus$ahg#;b zUNnYZPAixpze6-`zsvLzJ1-1>S6YV#(ftM|I+3swz$g1ACS=gL{C_eK27*_%2NME0 zIKI|?aTE!GsJTvq5D!3vet!e^ObFcnrOBZ73>yfQF)imB0mf3gNUkwM4vRoWIL*oS zmk+f6ZTS!}PW+`>!OVO80b-+VfYlBMgrx7rsVz@It5g95Y4RR#j~^^WfaY@HYPrk8 zV9YhCZ}p$h@3>mmTIGDX;;B)eEn6fi>wcnexEO@x5r~p%3TsVuA&2u)vbck3c{s=nm{ZpX* z*o$!1)~M`8jN5eL^?Ja^#p-mI7M`1sM68EeyC8z;a(H`)-s4n&h>H=?ChLdyXT(qEtZ`~DnEb1ean+g`M-01_pOMZuN&I}ajkjlQ*C1z7R&iGG6@4tYH$_e>8N&hfF zc4+wSAu{Ie?ifFE6tY=?4U}!aB3myQm7Y2jywcP?31!dcb*c`N>d9i=F?Zh?y|hVO zg>EM4cu{rp9AK82CUCgQ@vv}YXl1y;Q{O&NR9nazEtjAoxg_K&wbe+YaELWpC z58GqhsoDK?Yf2P6ugV>BIvZ2Ge9FfUi&AK*pgaD~voFy|#TOZX^q+FayOI+Sqxh}R ztR)oy=9deKS-oE3o-?Arkm=e$XP7zC6!c@hCo5u}+tZXel%k$d(xt%D>@oV(HlhKw z7k8A2u6zX{)X6849+{dHlHY`;e4)IOjkoC3~a{1M+J@{_EnT_J9wxB%fZnJmH2 zg^U9&P8`{LV{s!FLS@SUauG`Se>x}tcE9ag-mk{jhfcqL6aj4LJWs@N^zH(9Bk6%+ z<9^A_xH%QQ3@wybaW+9vK^eC9L9XujKWUI+*!<;xL;vhRLz_&(AuT@45r4`keR_!b z&IcW9v*QI`tz5Bd!wjc3VOO4v(>2kT6zafE|}!YnMq*a5cA5&4_Y7(~H(D)P_TcoX$q zm9S!kyC`G=q!hGZ3o~>wivmPTx*zTXE*CNRlf-x~Rw!p>gv9^&?CZOJJkX1}pXulN ze7{z6%bQI_3x?l(lyg0S>a$PdiwdZHUy1{pT8ow+pn6G|ZX%-&<%1al?)pebye`Ybmo8L|m7gCZMkF5?Eeq%VFzufndMPF5&43FuT%$oTboXilC=%{-}^Y!*j7WD3fn%d>awx zJ?RL!4qtg_dc@ws&`kYH!=(AA%N$4&#oxqJ_<1(;gT+-d_bEDmmcZhJMYIyu9pqem zzOaVoRgY$sMs^RH4xS+ay9bQUzp+P*(s`6>g0_y3QF+90`gu3yo5@d5@jLm}pWED$ zf7B_^`*Mi}Aw}yU19=PTCFm~Cxqg1VP02$^Zz`9*BNT${f9G6VW`xWR{;AYN)3n`f|hen{`)WlQg}p{XJ&P4)9176bTXq& z&;b;>%ZJclHMmbIWoo%DD6;QUm%%}7Tk(B`1B8@*xC&IkH2G6^iMB_?iM)(Y2@>3)eQD+)vcPQ5yg zm#eGiL4IBB2S2trt93m{Z%m%+>65OuXGB)B#>GdGi!4AjGg-JD!(3QDNWgdwXL$$? zOTf4PA?d&SF#k5DcNywC&c(uf#LbOj4z$P*h_@gAIY)!Zn0(_`n;XAxcY^&pWUeb< zt1ZzrgSm1KO?xz@0KrrDch;#L;iy3M2a$b#?T)9WevoWK2VJ};I(i$ngDFEOO2fky zP9uQLT5yeI#i6PHwAg`Xe_Cu1?f<;ymuf>B$N^`3VAqJU!)CC@-kkh2tY2N7rrRCB zDC!9G%nO4el5n5=zt38@{$@{&Ato=j_&DAuDr=a_AJ%G}Bd|A-wHG8FhapazQZ#P< zNQzm70lI>OhO4~zN>e40xEPvZgGVO=$2o|(@cHr0JbfHI?L(cKv?vI%L^goX{5nok zqJ;%QK063M7V!!c-8G1XfSf@n?Sl`y6qa&248;^Rf065I?E zemPBZvGm$KFa(-;d;wQ$R~dd2)`lrkH$A~~vk2(A+fYfkCkecn5y-er{n}W1ZY%xs zOTZ!kshz>w?}S8sL$I65Fc3S^?jZWG)HsF{POivq5F5))Y?4%lSjVyvm!XQ~iKf@y zKR$7b7xd;NbBHw0#%`MpLBu@*X_&8oJ$4p#U$NSNn5PmV!wOJ2mi4+j-daGCG!CyU z8%SS4M)(F3GTyx$nlrD#r8WHR?$tqLi3P#(g3tw@I5pHv6QAL;V)fm2K&qaGd3wWu zsmraq3aK;e`@~7GI%yK36nx+bV2LQ$1Qf}$&bANmP`LU zWlY87==}+yQM-04(a-s;L>H9DnkBQ}w;FGJ9GHtYKO~lV>CjE9A6IHW5!5 zg!EQdfq32{F*gD7(B#w^#_L$=eh|{$gJXUQX$Me3NL|GVhNh>W; z9E)A5|D4+0@_Gd=oftiCe{zoVa$@^s`&-U^vFd|600T;9bLe*G!_Fvi?Ftx`)TcnX zNRx}W%?12cXhTwQ#hcs+IQ<$xs_|NPHr zwvp~1+Y2XBN_&f=yot-g{|tkhbYMBXau!|6tme*)N}|Hb;|%0ltkJUASQ%q21EsLE ziZ6G`^n{!_A8=cN{%TU7dQI0m`(85lU6uMKSPOsKo((UQk$|XX7zl~$CD`tP&(^wf zSnpqJQ5thT(ijd)9wT9WC;5kjR1)ijLsLRE@@cexMoH$s^U9!@|Ft0X9&g#Bjo#29 zezhKSU+}Mg9U2WK#{GR<#M|$1TWxcixB^E)QZ~_0mRmD<76RmyR6f4=?cqn79M~0O zW?#?d6u0rPMPNRikJfPcO2TNAI5{&kivbeA!|<%hpM{U#g&ge<*|@_$BZ;~s_r`j( zC@JT(cha5CyqqcM(wCHc3TzFQ?p&IPte#hXvzM+ugD+Uh&nk$waztQ2Py5$9z30wQ zmA@9pB0A;NxU-)dWRISkuNG5%6#w8n_Mwsix^Jjty7?P0o;*^`rcIX4w4y0df94WL zkFhxK{iy*4btwwM_r!^60u`C#GmrJQhwmR}C`nR51#kZxQo8BwDnAWL{ErhbynbK% zX{Bty-?HajVXBNd4Kz#sIJ$E^4+|eXHq2OeG;M0B0W2l#HY%3?>b&Rfn9q7XPXMm% z@0@#uj2;WHdx!mdJJ@sl`Zp6<h|)u{zOl~ADRtt=epF!JZDmAXpeTGDbuz2uil;iqMB>`LTu zcdqvd!dG~sWq#do4EJeuy0N{T8tyA0-0AaGF<9^!*Pm%+m*1__rsrO@=@6TUTqCJo@x2PzQ)h&iCX;ArK;M!~e^WV?A=$=9?_6gb_qrX_l_cI? z`~eW%c?5eI%T4a;UGls=I#Js5<#zE@)fdNIe~(I4RnvRur8J4LKpJQEQmK;(^|?@1wzsu=e4r5-i%sLrJ7I}pb`YSe<7EmysCrPUUgag~srwpL75he3 z!_ZLbd`yAF%F*oJOGeZSk7+XGmEw)tUP6}TDLVFiWyz(R0D7#Pyv+A>?`N&?`dMN# zujQbcR9t-%5IL6$E5Gulh@A=v!%;oyby)4J>()QNOW@^^N{sb=r1~%xqKPR1_j~e9 zP6Hvhr^I;COG=enE|ge(8I;wAzx*?ALUwtpo2#%dg*8f{f3$@stoD88l1%Id;~lyq z2qBAIb8@;f8jPl<`0lKGFx^BzE=gvXEc>?j zncmPeaad(52s28eyD0y@O{DVwE&L3We^Wcz9hQJ(P2wtqeNM#69JmaJVju{kLjvvsO60?n|L(&)Xno(d z!XF_Am_8mjaCqdrqMV*O>n*#R9B6)bvdc&uU-mcVb+5XOfr-eO8gG=w6M_Wh1jHXW z(DrvI>OPp9h`Gk2t%&+9|!_P%&Zr&kITw&}bMBU}ketG-Vj|ge8!xoe% zRn9k|>J-BR2d11KS`#O5BFy1eFnjCTJlr^m^ppFyoOB=%btf#Ix_QnjDVn740D%+A zs^l|NG}MY;`|j;_3|j5!;%@5@qf`%I;wBQpXt=*pK}g&p9%~K1G(kbAb9;DN=Al>fkr^B4#AJ;yXDgv_5%9}7+9!7X)3^)w9cu0W9EpBgl#MV1YQ zgv5!3ia#m*P36sD$38oV=7N5>e0k$7TSj9zrFDb~*mMk8rEt84+R}+J+F=%FdvR;i+$inolJ#GUy!n z1qca&YZ@+Q&FE=d_q7>i?l_+X%U?GNY-(}Zc4oPX)LyP7G<6|z{ern9u~%hnpF|R zz|XHKW|aC-o8E3y zF{LFh>XQ5OkW5xbgt?wnnsbrpQzNdA&w*uG;+dPV!F3=}@I0^5r9oV;d;ml9SZj_7 zMGa4It}aNOFkFim_eOvqFw*3+g*eKCi6BtjepPHI2zL-gi6zI4Q$n*4LxZD-{irC3 zBM!)Oo$@s&N1^c=X}2+@H@ewvZ6U$^>6z|9NHc1{;mbUQC=y%b?v;tPg1dMZoBUz;@m5Cgv_ZQ*Bi9faukfF7Y?e}1Lu9u&> zt{$T955hIWPcfc{htOTj@TW8K*%fYIY={-lLnCEfo})$GhsNZlv^;pf*CZc9c>*rz zzFpP@O6}h-7qJbzF2T@=iAWAn!d~O@Tul#(30mg_`Y1zV;a{q+dQKAa{CR5-)SUL| z>m8g{@cw7OdC#$ge4@PJhqH07ghlfK1q%NIC_-*OJjt+J!eIj9O!&%4QWsdo2g0E+ zZ@HmGB;W#OS?gqo%lzT2jbvIrN{#+zBBjLUPLAZ`*P6W(@Sx zFYs{vC5czz=^|ZB=`97ku5e&Z&Yyt?V4EiB1tSUrhy6w1voZn+4fz9v((Et}BAY}X zFM+vj{UE-`cyCskFndyULqc*xlMEeR`LrxE->idqro+w z=?Le57x)J>UD-p~X9{Q)Wbhifs{;+g0{9NH?g5vdJPzfA;Z`}6A@3&YdrTd8Y4 zy%M=xqF3|t+{>@Zs!w9&Kkd4-#;T}J`W>;*03diKyV#X--p)q*F(GqEqV3 zPQ_CHjLn62-Eu(;yPKW9NyCkij5L;o?cq=sOh)Z~-rMpOXX{6H2%l3&I6;E`nVWOR zEyxmKN8gSzQ&3w?427R!1ZkWm_=d%ogOG%87tVaKIVJCkFnkT3!vdxp{mLbxFKqpQFX!=1Rd4xa`l-CIt<|s!3Cd9z65ceqG;lJWhvT(+6xNfdR{`!>|DBl8V>wpB?dGJ`%`a|&uCTI;gz+LI< z`Mc3T>;7hOkzXX-XK`)+WpQCSr~fjCK3aU*XAU6^koD={(qHCKX^AeUInn?Oz#RHd zY5-&dE`e?K~C>Nh}R<4LPj|JRn`e6Lli9E{M|w1eR5bexfB z|9a>PBsW?!2}pAV;{IT3J{1(uk}x1040toJ1ysG?*OAwv|HB4|oH`+AHe7t_4B5W6 zfm7RoP$TI~o*QGn69GUMo@rq&r1+9wbValpJgIQvw(`Czn4h*)5Gcjb5-#JbCc-T*9gKjvrMvn}gdWQ)Vg^%O?UY ze;%^m4VsMbNE;sADwTehP2cuz^fAAEJtpNxaj+&P*@8jmQ^<~Vd(MvekCU`|%uB7F093*A*^f z0$3B<%{Y^{c66^3Hi*}ps)`c3>?2a0P|NH%6*4olX1VfXH2Mna(dVzFb-TwKr6(;~ zp6zl*pOr$-Vi)-O-GxO`_+;;+=7P3;CRpvG&ui}n#f$UE+Iu{4&-gaFwbRjZ(8}QO z_mzp?r+%q{l?5G!VgYR5B@i56HYah(`pw-tv%Xs7-d|@g?BwnB<&OEvq%m=-{@S_K z5i^$L9A}U7`Qd(00O#Jb^lGXt-naJkDxcsA+rayHQ8C{*`>0<-E0;{?q+mN&v7==a z^0zl5CW<9Cj5i;#rlz}3cGTWxt=RNf$Pr_GyGd`!HZ5oyfG*$pzI9P7df01WDY`|p z^YhZ$=Z9am$%>Dzdd_b86iuy`*4DYBUa{3Y4xgb)_U-w0#ra|1$XZWuJdf44rNYh} zJ)A+aRY%OpX+c)DGdH_+rXoTVAGl&R)0szWFFB<0(Z<3*#uI<{DP_!*0OykY%0059 zVOOs)-NkwEWU|EF08ZoiS0Sn?h$>_hJT1(OwTtyO_vji2oboou1I&0UDZFCJdt2i}(#9#DQWSvvwTu#HEw|}jgg4iZ5W3J3J)Bc(Y zBj8}NIehEos^u*5zdw_DFeMEcz^FRyzZ zhNpg=y&C`7e)!62qr*Qv&6x|PE4dOm!e(1(c>f@M8Nm60Sl!?P?3d=DU*mG<{u@%p z_Lw&YS8 zD`uy;8x6W@x35K-(buKQ@!cue5bv(p{4`71JzP)G4MNYzR;7c zuH1RL%T|fc`$fzMRv48>WhVjz)LvPYN?_{vhe3$93;oG61C_e{1HD|!ZoJl z8%X_9vI{+7fj_nHtANQFz8p_#(hOYWkq5s2dasw{mq!8up;-##`e@8z+T-`zkhxRo21zLLkCU zht~@@a)W~Fj~7D`m;V|6HD@O&MjCX4y5v32Ef}?sqj6I56Xf{>z?dzNNf_RPzV0n^ z`CjrGUX5ag_}QbE*vHS37PBwe_qIxl#%%9nXH?K1QR!<;ItIr1kAGaD26aB{UhuvY z+2ykKdLvebG@y*Ik@R6&xWWMLgY!*am#!@J-wuvkA%C9jbK?1&eOg<^Bqq&2L6Ev^ z0hTpFt~iYumlU3B*e5_vhB?Uc99q4Wsx&{07nI`p6?IsZ(1eI{rn)6mX|CB$Tbj&v zyosEr_r-TNhy$lY(@&bLw>*&N>icNHFd2G2<3n@Ht?4?q_stYcpDKh7uh}nYba=F< z0=bJ}|A`f&DT6w}%9QHZV6B zrMk|999?T`8J?R;3b{$XoTb=FCtZ61;aIii$;!^>m_N;_ZAzH+>)uMd%lqEUUvp>A zs>p%}W|^k}T}poPP#BdghQ`^!CpY%E>RtO zE$4YxS&hGOTcYn@^OfNi>P$Xr#s2##=OVv)axC*rJM@uSYs9XUeKDY2M3jh~$db#j znHF227@2}g)~cyUZ7H*&yjW!Je_Yz5EU(?*vTYWy9FoHgz9D`Muyhoke1~X;T}JNo zhsTE;<%X^9cs|25OL^U(j-qz z`gz~U808O)*OSLgC2w@0&pJ-3+-b&yMVE{>>wH# zhApb5jif!+zI$FfH4$;Px&3^NPdaC5j{!UcnG4O6r>!1rGN;lje}P=T0mw~T-A@}# zn?Oo8|4LJo_;WHzop3x-bo;>d$OeIqUxh9IfKCpj2#bkkzFNqRUETo;+w`T-Jir1^ z-eD7D_cum`uQOGwoKjALx2}oH88fkwR^CK^3lF=%ejZ~ZPBdmGs=|gm1C-%)B`~&L zPJ$YQ;s{Vw^5G03yggbc<%uXEzRp9S`TD?FSqHh~Q*zN^AzcxlL`3Xi4RS}(mRbER zkml_OT$Li3)JVHn8?$lH&B7Gegt`ggXH|wuEpOJ!n8!-DZ-$M=dyVGl+3bq5b=1R% zfH`D|?*)KU=MZUEzQ5!j3TLFa|)Az0fuTZ+d#>UJ$ zl2?3Szpm8vCy<w^W5xjZxz==p6)a3!3Vc3w0cwmE!n7-ZtG^YCc^ z+s!8z1HK85kDtM;E?zH{e=4!E0=sBNJdxVp>bV8F@q&*=M21wYB_Q+ew3d8a zxs%aBg0b2)0rRGe0gRPIJWuz@Wxz z*pqAu^4gdM-6FyWWEU&joEIN2wek0Y&g#C4v?eI-W@aClYl3Md8T!WOPq4>FGB|8| zyQS6i3$H4kK@|`eKi-F>yxpNx)Cq#es5jLjYL@3S_}mASx3G0u-`nUi?E<-Lq={qC z?;UyewK+nb>rC>YtM{MdyDxjcrf`Ifh#f$R7yE-*y1P#tY$hYOzR{QN*Z|9a6)IWT zBLOtkmaSqso+%rpumPZ*4{Sw0%UIG62Ax;>E6AQ7BWs{sdp+l|@|ejNHVqL2Rnh;$ z+Np5u5P$K!E7T)UinzDohkhbe^AxtY@%n(K^D zt-;dsbaq`QOD%_7NA)abdej}tgf0l^RN?iy+^o(g@_Aa)W-2HY7Ntq0mi?T6y&(O% zKyq25!pb&Hyx64rnJZMJ-1(-jpT#L#=~8O9zDJv^akfdg_WLqZ&mFWvQQLJcURFxm zV&z<5q;PD};f(81HtRLR`#lDq}HqKXRgcg-N9UGdQTZJ@?C*cTA1V~ z%;{;Q9lk2*d3^o8EjwHHY0J8l6}q5kxmRQ?-&HrDTZ-%Yc6^0hCU*4QUFPlA@!^;)Xl@QlQT+z`*4xL#-D~J)x?>jD+(V6SttY#vpch(i0}#u$Ma-3Dz08`>O3! zt{V9{e6U6T7}i>P98JJ-j9Rl4H%VeUfH{soI3crD?9oL+PWltpfWEWm|M z242$D?IT8xW6>Lde0)-Om58({^JuL`BhDeKI4}YH%)NDk=p+Ng(Nw& z)>F$`w6%gs9xj>1=>$rYW+Bn_jD+(Cma3+ipUWodS&W18YyR!009YD*>PlS^BxHme zGR6F5v!0c4=($@&2l1pe5bwHHGOZrc6KTJ_5g^CE@!eH>mIl9 z)qQGg&nK2%T)XkmO9M8do}N2_;iml)m})qsatL^=aj-%&IE}c9kEwf+X~=U%N6EHK zKhz|eY$>9u8*9m305=9!a}M?iIM%*)K=Q%p; z6$RAtid$u0gF~C1If<_fs)~IaH<`oP}L@yl(4kaAaXL{uG zag(0l<o1W8NN9~1f4h(Ef{-{u`5x^QSO_&Wo z8LL8&1oVuLCykacwNBSlDsz0orckU2AVChNdaSLS4tBRZ zb`YPb6AoXGf-XNXoPu_kQ(AG!XhuCoJ<*){=DHSX_ zst7z~dV2{bc&tkb-u|?agmQiCdk~5BeyJU_KK)jc2-}m@QIadkmgoV6p#DpzHt3}0 z_WW`XA)Q)D<m3#Dz?425~hko9_;(mNrP~XH;AplgCrw@Humf7GL3n=$Z9(YxG2z2WN(+n}X$o!kA5*FrU3{Kh>j}*)bd5#IJ=8 zhAKKa@adG2ZvFEn&Y({Lw&ue$YeCC@`x0tb6BG z$lH@=k(n)$#eY8LRzj`w5lhZHK8FhcRhyt z2T}dD`RpPz>h3LY2mM&pEL@50RpY+~B|e(_3caT{6ic@(n;WW=uTJ55B^Ack@Z_ym zr!QEwPeUmp1u!D*9iL7Me5htM1tV%I%x;;lrt3aKf*FsSH(16p<-M%0MY+-9d-Yr6 zF6H`yKCwXg?S;l?#6#BytF0Mm5SZ<%q^`clK~JrP+8Ne1N1p6?H@T|8q1%^hJp9Zc zMlNvdnXV0fH_2-5Wm_^B)otz|jbLbUztKF}(Xmnd4LKlZxA;^}i{u;%KF;B(xh}(h$@9!| zuuUy#_z%z)cHEq~*&S>B;D~hYT|8-aO7087_UuZLI-W{~uHl%Y>%2qTQ4&+Ms9kXW zXP=)PHANJ)r$sNbW+6SbSHZgJi8)YHh)vI=zP}?kdt0Lqwdm61HacQFI>;U%y#DI4 zf5!Ua%UtfWONE)B?uz8dNw~+eK02hS7De2ep?CARG>%=_d!EcbW$sjK&x_2r*Dr2z zrg}y(=S;I6uT@T)t*@^-;yT5=-gg7~oX^?}sSurfj)xcOIn%PPHk-lYdQv~r4S$er z)MNDCMm@tz@^xD;l`vMb@jb0x*!t1o=>?U=`@ z4-N%1@{JasTWC-ANZ*wD^z8Qr--V@{8?;{8L*wntbiB%nQK9Nx+l|BI4AX<`R?`_n z^EK)2OR+~>?fXwc`Hs%-`qMJetHVR}+4jz|e6i1^FjgnUdwT>E&R@xhi;wgw)h;3V zp+3@C2d(i=47#WLZ%SGmmp(|~CmZL(uJT4}r{ZXAH}_AKLmd4qJM{4@IpoWff-g6L z;zn+ohktj9X=BAJGmXeHSgYpKXRBoI5V8f9WT&Il6Cz;Yd{=#)5eL`QHpGHy-dHGz zc(K>jig6@s+%(_{E8()``K71zKJvJA4enu;ZnT7V4laeiIXlv>SrjcN<1RN2ZGYpq zE|+KvwniUMye92FzN2g^7pzA=gQ0eII>nfyyG=l?^5A59R@_-)8jBYfh-ywy`j$@? z)^*1%-WIeusCsg>%p_jqu+&Aq+Pe`wOLeWDBy0l826ev57MIBf1*2uT;7O2_9(;an zaSnjb)|cY;x5WQWkYu^~MVdp?WG{(w#DSaMGV@V3OFs*z0GOmyA+!C;so{Q8f}c(u zE6g4Pd2lNFklypE(Lq$+cnw@AP|$sC)Os6sU`J8vBV<+j4+}TD({iX@q=RC_gR#!X ztz0Pmvyt^zu1F{N09WV7k+4#|FpPe&;Gr>XWmz(;9}(`tx-;^MOK+X+&8k*^2%DDS}LH*+7eJ5QO3o{2Ty$wRUiW z)cj1&qbYRoVE`0@P!5Qk{7<^6&ej@;Tj7ucxuXAeAL0RtBS_Jg5as)2(_72B@+bZI zU2O2XP$)4N4%c!NjZ7JUm*E?_mgmreKfO#7{ccSXG?p-P)0M%zx6Mqo>7I{E=k}EM zD0*0q2WXfKJ0RE$5=+E^5>YK>Y@fT9_B*)Y9*m0@K=abMJ4B~6p>A@$IOc|* zBxF9>b1xtb@<-q`aZ;z-aQe$aI@3R-)ddS2-a;g0yc@b-j{R5r6r~N?qY|l1tvQIA!4BNf{jXr z33TEGkg1-7h(f#MC-BOW($ZlBjxRLVZhXA**yc29Dc!GUmJe`{k!^EqzTi4CWCaIb zs8K%HX+KVk)viDgpbHm#p^Lyb%=-wY^^pn7HcKJY#JwNVEKgrS>*hncsNw1x%@IhH zJwZzRO6wL1?F1;beKh*6jGaNbx9o61RL4hyfj zI?^{bLmZ<69OHIrL_1V>IWymvpLMEgf`Yxoks}aNV7J?6LEW4WNh8P~i~eEp(F2vx74k%Tiw9&4)HO(ZE`aR45_|+-x^Z}{uiarz1faA2 zSGZj(o` zU_fj3{x0>I(o~% zXU)~aZL6)mIX}f^z#6cc2K1b(HPY^2mgyTRwTc*`7rq?5$M0x!=@1z_MqFKaWw5io zt<R9+04dgVtloi2cs++&j?!O;?vaoT;L zWsZsGyTElJCC~Ksw@dvc77D#Y)_f%Zud^;Ws%ESI;#!6$TshQl{Qk8wVtU^@P>t1Q z3(O9EVBqO+t!Z$=tZFv9+b@ed>>WKe;q)S{#@9jjk7rC*Yz?EqcaK+{3`V16n9LbG zWQ2wH$xhWQ%Rg{xQongb5Y)IiNb*UN-gc^*_sQedvvWZ5;3e>h57^)6?Em=?=1cX{ zYqq``ayRwbSYBxnU0bd|jn(wqA>)Gb=z6Vr$n3N|dR@TXUfe=VC75*9otYEP+GDd* z>0;oaq$Uyc%^G{qL`Nc9#`YFH)Z?j`s9)oPwb*2WmsO#vhC)Theg}l z7D`*}jwZ=Z<&0KcPht;X=24qy(psXtAYSk2V^=k#K}e^1Cw-Ddw(jEL*7`1dlV&N% zu1w`KN#4Ns@eF_ZNbYT(0EKZJERI`GYGG-$+q!?3mIRBC3ER+f8`w^#7#Rj_UWHr- z#!73X?@y-??n}*&dfW*(H^S-ztsU_bk580*X?yh{p)lUKd`#}_{`8>xdsN{KxoCE4 zO_vbrkRuBTE?`?ujF=%q^dk{y?#%8|687Zw$N^JnKhqRw+1SUc@wPJcv>Wg{D<}`E zmsHQH9NBG7OD}mNFQ0TSs^e!#84*u*$v8hoBWUsiJ^NgwPL!mSouwUEq(J!0{V5M<%*#r+6Lq_ab)&R-z5e>kx zzs;b#)!=g)@FlSy+MU76teL190G)mKm+SBe{#3{mS@$bs@50d^YGOX)Q}a;N>EZQ( zp{8fx1FYaM ztnN3;07mOj*ji&9-e(3$OaXx6(Ecu|fOs?KcLO!=RRVKy0_Uf1rD7npCj9kWR7>&B z9KAfPP3&HUJ3c)Cp4m4bKx3GTZ_uLacWv^l% zO0q4^Q@HAuM)PHC6& zv-BII^@YTOX5O==jRz_4WA@*O$v=k?F|vs`{BgmPli!PgG4O`yl-LJr${^E+U{7dP zi+l^#Q31wTV(72#n<$o|bQj@_z=~L<%=8uz!!eWuJ?bXN8 z%GWdHrtJ7@?%{lqKj8HahU$#_5`}0>L*zSR4$V^#g-o)g{*-{+HYCyqq(CPaS(VOM zu0wC#P9NCBihvx6dnfA|S6DVsuG>3V|3tg*DfW)t0R&!)aftRIX>7F(CPUw>3V}9&51bt~ zqI|aj;MZPGCEP3}w<5Uv5wR5Z3c>Bbq++@^Nt`g$YJSb0< zcw(75DL=yA-xxywx>3CHB-5{Scz{kGPr%vtK_zia7dZy!AUxGRX;^V>+#MB{Pr;{y zE%Ei?Z0tXyGZ9ER0}I3xoST;b4WUzg>uVp_o{DTcQ}WSSK44(Sgzl^T(1J2h+x*-4 zM}qr*{=wgltrl*6a24JJbWI`D=@OZfzCQcZsq^tV<4rjK@_;2_jNDsukaksMI=#=bLzz?wg}MNg1Ru!hPI!|)du1}J}9&J<^$$T*$GO*_OntjqmHHS zFDQLjP_%_g;=sAv0lpx*&8Q~w>KNcZdOit%P#Y%Wr6;g5R@)K50|zXk%fTF25``)6Ug^urfbFK>g7_E?iso&qhZ}2wUy?gG4I!r1Zb22gyi?uUz(C2T z46_D%T=G~6peG-$Lg4S|03ZlmM5b_n)&v7yP}aWA0T&XG^!SKEkIfY?&0%yA8?Gwz z28Z~G97GkcDC5JC1AAOE>rGe?WXA^s6bVGV)prNaTBVg55+?gsd;qs5Z47P0Byw1P(w1XAKWE*K%->4 zFQqT%U>rr>CHA6vO&)wE$*;_5h~xp6yKI|vcgX;*WhjFtAAQ4uMSz2>aE|32qEmL@ zDliBuIuxw@&Q6n7>}pDtZW#MCgzBypqt~EF^#$seZ{Ig)aXA6O+5STfxThl162@m( zd8fe>bE9JU`nqO(me-AT=@&~-v%9(UKC?{%2wv)@FW(KI9tvt75jI5n@w>+POS+M_ z4lhbevG|KqqNOtry@Rq=;XDUuRN(N zpmGF5jRUwW<|bk_bdkaN!42oYBcz095-9^(>S^iAup0EBhEqZj>`TTkd;^MIg)o)P z)S}y0n+eZ;1MVGY}8u#v2;Z&~4;#`!Icb zf(2?-_-Cs;G=`IhUZ%ZULl-@*ub@1U}E}kVmM5**;ybc$-X)n1f}1d3_-I>-^_V_Df^8ir34E zAKE_5<15$z)s8+W(PgjKQ1w67NN=AwF%5~yC-mai9z6ux!K{D%A$0MCF#5%t#xa-- z!?jAJa>sw_r83OA`jPx7x%e5~!ys*d_YH_T`A}kfH|?Mp>&Lhkn?@+_r$q=jA1UQt zTAS+VC}6{ng5~y7MZMZ>=_!|5qPZ^VO$bv}UYa#RwKqJmKAJ4ptAx7)=?Q=o`4rut zu7Z&}-M}THUu`IkPtVWK!0T&eheI}MR}5Fm+lq{nFYnrOu=ww7Dj#sX{*kGX*`;^X zuD*M|OYhKuRVN7a4ydv2(L0ia-r>ifAa=s)Zv0`XW!W{LcL+WQddJsKdb2?97=CjF zDRi=bL}U{rb?cq9FyQpbKGZ3bFks2M1f6j#b+^oCU2MA>&f3}w$IJSHQ#z@|e%h0* zIO|Yo*5{Uq8n|rQ+l$jdJmk@d9CNv~Aj#mmP-p+?{o0jb_a*7L2Z>UTczMkjdAF-o`1m`)%f$!kT^0a2 zo+w;PZfZ)p@UgP6%G6m`&&IyOLsVYydg{S)X%%Btf)%GeKnKFGoPDqX~DzbFp5Mu~#PYzNr`OZsnk z2upbBT`%&p(!^c?jy0vwMH&TJ!x^0WiJ-xk1ov?}4{0=wsjnYFqk9t;*No+3vsyE? zkA-CX613IpmhME=KP<8NxbWsN^xvSM%(L(@%tHkB{_3CG;0nEHfoHLBpeQIZ$u*#5 zw{b*OuF>|KBd?04-rW?oD< zgpEOZ(L=)R8C8I4>jMT;@%n|=TrVM$Q2CnI3YL;T?;5_k&#{>^A zC6$IlVyrZ zLd$saFN>sE+Ge zl4TB80&;Sdu)!sdafJ|vM3fv1QX4v(WOkDS<>__dI`Zf38ZfEs zqMOpBz3BZ{;aAx{ae;(d9{?MQQ0=M`?CZ9*Ttcg$aM9gOoN1!m5?$Gw(_M%hyxD^e zK;YaKz&CcT>n9X5@_)`nS>^kxrvwH) zxM{{E@Mo@KAj)W80ffX8&-IxM778*JOBP46?$~um0Ov#QwJqh-Z~o_xuM9p_qgK=kx?2zXQ| zyKA{37wfMf|9j27%+JlUnMMro${%hywX`p&mR@p-41*t8qzTD)C$!faq#TOYv(;CqQ7%$0}c1_zCP)C7A zcJ;6mo6L*|Bhm>0uiB>=V1{vCZLg}bv!U9-G{hhs zx38b51RnHTe`I&R^z^wsByBFJwsBfxyw}v(#n+>w*WhxIJF*Fx$%l4waWi8qVjWYD z?*=kTFO0wabg(vMcsSv>b+&ez=WBSPlTD$JD0p+H;M}1 zeChZOFo97X7%)c-Pw+g2@J>$Lh6vthit?t6Gu%HSnOg-{Ja6hg__R)zoVD2}FpDBX zWhH5rc$Vxm0GyuKoS9t6eU&E$=@fO?A@P+^KwFBN*_985ZkNxIsJ77H$99Ma&ROz zW8IhrUD#==tH-$urJX%^cco!nSb3c=>-g2JbQ*toI& zVjnNF2MNka&5ZF0h0+v;v#^ozXVCIQExs5(+z%*C%IO`x9ueANqrOE$0GxnVpW zlV9Gr1MLsmL+C=papFor31~~fzcB;L#$_+TU8Jy|;65UI;PgBOf?Xg~I|e@gV4F;B z;;=wPOrCl7c6%XdNuXD(x~M=cHykN9);c(?yS4Awe9}LMli`nH4q;>HCc+jN{|d4w zY0;(X+1%uImn&XRcG|hD9qCG{mtmK(occqr?Wt-;_fdZ?CZ24^7f@?aXI$(!Qq~dkUO{(?xmGM9U`sFu*ND44{XO>M zz6JL)1UUOCC0qiaU;HhJv&QYmN2zApq-9r&7xp_yLaU?z|4k-Kb6Ot&&{oV6 zi^sR7Uh?+Rgi*53SP;!auXS%4hDo2fTI&?p0+VilO7mxHqIewVr?W;QURFJ|T?!vy zKM6eww8)>tOa2%Qu9Ih^Q*7o6ru*FaT%&F}$Fyi{I#ykWXfqG0XlKtoEAfVDQ*${w zi0*LlsfYvd;`3YlhdtTLo(vu5aH}Oqy^itJt~yZ|Te3heO$Aw?TZZwTS3^VJVTAN& z$4Qs%AJaQ{h%)pNJi&Xqv?i zD$5}&#jbjRpR+?Yqn<)in_HgZEk`=M=SBiYE?p3}h3A4Sz*%+ix^0E@h56IL)N45T zR;dE-TEEz^`g!nrmW!1$I0fL6=xFb3%o~qEk2@M5F|P=Z91(z|q(VC@QjsP5+L#OF7xzEP|~ z&ayOi5jOndb&SZ>_?LR;KwDy~XHheEpc;qI_(k`2AKaEFVyeoE-71gh>RUWFKk3*9 zHLXYMxt@fxP$zlPlYH=H*Zbu4W-3i4%CT;HH9=9uYy997*)z~g^S`VQuM=fqt-Y8La#9-xoea=piZYIsTn zPd`6!lgT51`KsU>h0Te?%lJHMuY&UYXX*=hd1p6pcf=CNxe}L&1%mzbs7IN3C*RJK z2sTh^NLj?&Lg4XEae;-2?{lBD=G1Lf_fvbjB#NUx3jI@_&$UQpxpKTIN5?LW<;Yb)xo^BplI-2bS1ksgQT+Eo$Nn9ugt)JuS%u7?5MbF09*A z3BDJaqCC%X3n%l)V$Z7~t3g>_GJ><`{iFr4u9lNZ5S(GQSKdWW=j z@c9718d+J1lCnF34-a}LG7vQ0MO|L{!81YW;E!(8mmsjvT(h=?+vXSrc)CHhJ}3j< z7{XR^V^-@Mgc3noQ;_G*gPy@GaW#nqaAmy#z7XsoE)q@In@MWeUkJ>5__%(@Q}9>O z#Cp@gXa1rA9_okEf-BD@;-hicxHVO}h`lV?oS9|(aO63`FYPB}?}uH`W=(OP7^^)= zu1oC|rJq14jkG`tWh^bF`MClD) zUJeZhwdEFuh~8^GzQA;dyn%C|#ldYL_XpV@Z@SJ=ZF6;H7HDUMX=-#k&iVYSfe=Ns zqMy8?VOVJmhbAuQqHO%ym0&Uk%7eg)D!e@q{sMeu@U`V*wCsHXa2EpH(x+h_qyege zVgnI46zcH5-*>dNUUL0wXrE4Pbk6B%!>{YF7z3-i9Qye3xrT`+d6cn(U;-Qurzz%w z{Q`!~kZ1p+bHFPmylsx?hQT~Cc-4YTs8Vf zesh9JI26k85HTICDlvYn(ZXj?ml}vqiSstTe(JS`@eibU6ixo|8m}n91bP%?qsLP) zm4~_O3ePCu%yIjAxEw!Id4pvQl1-h551wkN5>64;m>QKd>twdbAb;&vA|% zaoF}@~u@z57__}2&EeAnI*H77XFa^kPCh-{5cGlVoj zjq4ix%@@Z|t;ig{(x=!)?1R%V(l4HqwJBj!v%u%Wn(^j<8GjM63Xnwp_<5+khn*mE zW`n1bg70m8j-?9M4DvryjsrMBI>C+J%7EpDksOxfWgv)C{4*Vdy+%j=iAw-KeL&&- zzka73%yjgk@Q114gttMQj&-`1v`u{et0J1Mv z345Rq!hcJ@z8WBZ{XRkrd07yqr#!qaX!x+ZX94=qRzb<;e0u`U7d3cgolhFdVr1>nq&{w~QjUjF#|}jd zjAmP#oLy`N0e|4XK@cp&WLUudt^-I7?L!3Y=UD{M2J%qwUvD5cpgagz9R5e(cvOvr z&q<~NJKg1m(jks$yuC+bCiX!4cRLJCV~xMQ6s(kl13}?ockp1q;^8aKdJXtcFW^Ik zdJ&a+b7*40SWx?QD#0N|bq;zA3W4%MI~r=1G?!Q~XF+A@0pIYz`Cz;sPA1!*VOIX= zIFm;lx^-VR1^u0>oBE;@$0A^h6An_d-*{d9&!`1(vjj)~YYGRPAi|FDI49s^7qD31 z5O4&;;+56jKY0E=j%fLTNF6fq0*^p>t?>WZ0JtI{*d1d)$DX4O5rYh_4&pjYe)8i) zS032vAm4mnltJ=@g5@g?GCMStt}?|(;Ha>}=%}-5Ka^g+N^$;gdq4cNA1lY+S^6J7 zl5G?*NiwKz31P;bI8Q5w9SU=SvODhI?3xNoG03lj)&KJ5m50!c&`%ToygWl>6PLf# z-lMIWH><)4VTq_DQzVUF-~+sX;Weg`g7BgQz5;y{yz9nPc&uTgkV$rbLWE2?a6{%k zJjXF?h$VomoP5XlX8)!CmIL4crgq{ykV!q@QUK*{98Bu&V5?GP7Sg>lN{V)s0wz@= zn&Tb^OWk4Sp91YP%^^}47hd=Zq)+f~&_;lja`TAg)_D*i9J4-gAT~zPN+W=vPo2gPHuXV6){Zv-7X=@LIpt zp7|u?=vgYYu=8P6k+H}A=#?3&H*~?|4{i^AZt(s{x?ym>s}KPK)$-YjK^oQ{oZ0f%M z>(Z@?zSqtbhj*6pGc=fP7o7Q4M=!G7g?A(1ta6=ihlHSU?A2A0<}MlM*N2X2{MdQs z{b6;<&CO@>+^>VxA69XvomV?9`Wl(G@uVs(ZWB%W56I!4^1UvlJDtYk+uNWm95uW} z)5$JoR=h*?j$Qn>{6fQajuRO`OMH$JO~)h`@9YR{O>O;5{q>U~#g}!%!iQA)GwCyL zpA9|nsL_pvP3;w_J8K#bZ-+0Iy-ZG1P~C9L5KIS={Ng^@B+dJwV1f6_!Y%+OuHmO1 zPp4~;Jfh)93H_Np$FhfmCV-lAeT=oV-BE$;JZVsc6@?QVAF~^p`Q=a!L;BXa9%w13 z(x0CXe%Z6X^fNxrOBUBJ4Tj#iR%g2B%st5(1}$LK>E zp?+8Rt|LikztTj^g|if-`SOYd=@*{IiP*tzC}PkiStkD$AS~-`lm`_>hYk-oPB(2| zy3cb)3UE2_UVk}Go-+mvi)SYmR*;Cl)IiB=v*iI(C&#C2@{Z8>dZLxI>#Qeh-RBqY zrXt2+8l802^uesluXR4InTYwFLc6?B7A7S+ z@kWbvsiahQ)NeP@FzLlg}T6Mxe=KtR^uR)2qzsCSz zC`LEi_1{7P5L&=yz`=6<5&b?`B^EdyyZDid^b_$5+z$5P7f$sG|(Vz?MQn5hQ1bWfi2V`RU&izrkk(i{; zucxoS$hW@V4o?n|9V=KKLvnBYs!{=sm&hLu2~)6KWTLQ2L`seAsc@Q?OrmA37Kdj( zW0F(8B}xk!l=4Hs0D*=a3vu1W(0fQ6Rs_#)tpYmKG}^kf2P~`#+CI3v@8(YwDB4r@Mtt!+gQ)rY@?36pLxU+T<8u4$oj1i`V3Gu!DW9Basv zHd#6}8fA}l{PMnFo`%`>P}>93pr5SZY)h?1mgPdYU&?@7W332D{ZIj)Z-Aw|Bf>w# zKe2^cFi7nbxfM6rbgc7JJHZH?Y%+atP@u*~M#ER0Rw*Kpn)EalL8HHHsZUmqO#-Ag zU+h_uzFU$Ay6U%RL|YF)zw+(H**d^4^x%S(ztJgAA z7Yo5?tpG9^XRZRk2o!*r&H+3EWeJVHz&4TO@TD;{Hw8MW?gB`v+o7~wy=yC&PdI6m zJ7D5wvYCR%L57Fj^dmw%PFmRp9Tv98!{0Kf5ob*w0fPLeT~vZw#0EApcc)j_mYZnI z^1$TqBIpNLY;?eOGUDphUG3(46Qp&Cy-@MZgZ{&LQ2%J+3i}>N7Vuqu2t)yW0FR4t zt2%DIs@O!ta~^;q7wg%$SKp&~d2)b7C`j#9!x=JQ#Sq_nAmMn9mHoFH|4~*U;IQ24>elYKky3YA*o+zD()u9(Bp>zY9lZ1{<$d)U5F0e!y9$ zNzd}78#yYi^j7D*2KFt(ipyhf=UYlUK@5>xUgpX6#1#bUk}qz7lf-3X#Leun*XCjd zM$Bj1eHtnG&LC018fwJ>O=w@1gPuYXHFkC#vbZauaQE=d^@)iSHpqKD@?%Cq@KO)? z3Z(rcKv7g$4kj7NZ$Q+jaCe0Y|I9W3Y8GopbGU!zj?_W=%?|3@7l4hB3VkhFKY&0z z71TT(sB8BwcEF^K(gUZaj$M4fl50t2)NpCXuyNwHa|X?kc`?G<7lsB^VOwB~Gm|e} z(ZZreR2$pxSJ=bO=}pQ^oP$em*i49{Jnn@psC3?{^Zs^%%Wj}(_|A6_P~0ru^sygz z2ic}8DB&lIQV44`ic4J5T>$|?TN<+^m68sMj!TV-(b4{+mj{ZHlIz4iLrbj3Osdx)EBV28q8imohYrh;1bXC;q$%DHJdV>z>0M zhTgB|AymuNnww`$S+lt1t7lO!7T5kudP#OnP$s-ugK>SEBw(dz=;6+G*Dsx=h12$4 z)ngqtSl(LcMsqu`pW?uN3M=leZ+8I2+hr}CEnMNFvFu#Y$E^ju&Od$ele?!s;OyVG;xo{M*qxr1ZZR~FNVJCRzb@VWQpv_XiYl~>rM`RXS&6O4u z$6+ZL1Y54&hpljq-ZLsZtz-todVs3Rnt9ODPKg~VC*!Zi8el;-@4oV(%tjSSb6-(2 z$#}o;k`DG1Y=GrBpKH|_LDGborWW)(%8;D;a|@)`;>nI>p+DPUNnzAMQGk42n`sEQ z0>B)Q8#JkJ5+;FgxVOHW;Y+udh})vESJ)P6mXHEvq@;9&`#c%Jy^24%EZK5tjgW4eI^m@G<+k`)Xfxx&2d5h(E5_JIJc z@bArBmTywrv`vUlKm7)UYG}AdtHeihGgWA+t_Etz28DhPBy@!**6j7pQG1!m$WMXr z>Q-B}8^^7-RMlo^J^8wpI~gAyW_i$h%G$i7d&$jp{TFQ~@JUAmms#j=wsBOSea132 zdWue8J&D*~j37uO0u|c_Pn+&oZ1VeEDRjoM>cl}J(_FK`yg5Ee&}rRHKq}yYDrSob zIT6=D!J?agglon2=jujSgNuEjS-Db|21?-c@U zP=DaU9l(W+Z)&%3vP6U{>(gIn{6Ns))sr}PmoRQ144%S@XY(H2Wi}~@o64Cc^BE& zGFvQ{dKJzUC`B=I8A(@IW47Y9!d*Khlx8KwxjaEYrc(Sy@LiQ?V=qryUy6il`^`L3 z(cf#q@|em%u`O_DhB+(@mD;WeC3is|rK0tc1nEgTY0LI8+_Lv}d<=y&r*#5y6WqN-{%UlXJ z+#td>=j=-*#M~gNf^@7?iPWr`%M{+ejE4kww}N6nDbhA+2Q zLE1EJT3(OKAg(Xz)BvqYhK1Fp`XfiUs?zl(0tsH|k69^{ZhG8}2dl!IH^|jQRoC*H zLX4Qz1UD8HQ?6Vl;0aq#xW85ZN?@pEkU=mZ=Tplo)LYK$=q;MJ(J5k=W2SSDLQ5SC z2QznwACv8y?gglloX{=j2fG{(DlQ6GMyaWMIhRpH^fk5q)05NBmnIHNZxv{8g>aJf zu?O4(FG=&d@HSw)a?@y!ylLb3rsd@N)J*kJg}%T$!8GK!te(_AnI7bn$$}WXrh&7Os&Nf7~>)b`pD)IYqrixFl)sj?sS_2Cx|Ji3B zuS3lE%`}_ydMA(7I)3bv^R@p_6n8{W#rfgmYw4I`#o3NuYV^U87fyc_4#Ffs=}vo3 zaLq+x)9cmp!UOWxZ@9>UKI2!1e^^yDj3Nei&VwxB^Gn9GQovFqJ0+_5TA!ljSG!WP z-+@5Fz`G>wSf+J8JMFXJlX$Xz>ct%BvMO6H$$-?x?iWGM(AZSZ*Z z=<{hCoR~hEtG0!d<6o3mRO-lWoMO4G<;^NND%(^XXfn+77{~VD06d_6&M2s|++XmaCskrzg5e ze6X_=SMGAl7ZeVho}@P{o2>7w(f6=6tMVhkCY79F!Aq$rmMwwgg3R`-v}y0rUYqq$=mdSD1K%V!!CA#U{U*KOIy2p;OEn`D-?(GDvq^!GO;T|hPMRO$+4Srt-9b0UAG<(d(#Or9+l@1Rn1CKM8q_lF9KwDmy({f5oJPzDkvKElmC#G!87v5>hH{Pdub z@GjUw#TivQIB&2b=$AHEE$P{O8=m$j?wC&l>KXgh#;f59a_(vVFhge=f}b8(sq{%l|LB-UQ%X)N~!Pfa}Kk{|&-} zD`+8;A3}RL;r~hh+TT7r60H~_yz~En*06WUB;Pk^D*k2A{GXPg(?Zb~G;Vfjm2mR? zlopC#0TM(!zrit-7Rkfi&x@cY+keA^@TI<9oowreK(Ciw+Ya#VPaV1+`F{lE8d42B z{6=pdgL+E9Ek_r%XZi>ceGYZ{B&UB0%ruuT$Bo#4z#u|W<|wN7Cq4}49ve7`sh)K^gJ7LCM=toPQ9C~oRtLHn!mT>eE(jzad2jbew-04B z{%}rY(r{L(1rw=;i_OkxheA_EV zKg2@hLug?91k}m)A1*)xB5?leIY3Ksu``LU!^Zd;!CIx~1Z0RgxE_gmxI3eMmf(|b zQ0s$^i3Q-RFcPa8DjOn3t)ssIqoEK~qw@a!K;p~b*Le=I`9khAclndhimormzBhN{ zh|hi58T8ShF0K6!|AUnxiy14zNcKbXVj;aLAXsVxMj<`FC5-RYWgPrxVo0Z0*w@M9 z_wvSUCeS$-P0HbwbeCRqw#M3V}jt*fXqwb2o4e!EV_Z(u(=5%R3P8?H>+1Mt7{ zp~rMvNOJoX_Y2q-{}69k{Y8@5b#kLv=9-u8%UfRyx{dCT1p`-zj2Dmbn%_r=L8zy8 zT}Qc(@$8}BW7$XO?Row39cn>wIns6z{}n+Cb31kp+paVxPeqpKZY!%gmg`P3^0_?!MlSeT^GF~~a3Qp(oU4!u@U$j8*#@B$3R$T)hO5By*$)E$ zX&F*HGC>p@g#!FXp&=H3Pd@h6Z{-$_J%i^T+los&zI%|`{J@h3Zg`F%A%KLGyA61V zT(N`p`X44tGMR#-S30X!z94p~dNFalm)rb6hkBVj?xZwCIK%E<%gEUPrfB+7z|XL_ z0iIi0FerZV;gzQsX>WnFJ~b)9s?Ge2z_4`_RZw`&o333M8#3aY2BY%dEfYNi@Fbop zKR!Tn)EvoM3waLk)A8`?{*SXEz-B<%$>$3;18#b2AKZpQ6?ULfBr$wB3Cd}UOB8QV zztplPFY-I(r(nO+x#e51hl6-Vi0b8CE?|8=hBrBiu4J_DSpp)0^x`hO=CKwOYizji z;#!~nK)4>Ix%<1$h*FKGo$|P8n2A+Or&09HaSsvk*`-ES;mq@AJ| zi9+Sy-XFD-*gw*fgTei@@AUcZPJvG$cuZs}*&IWF35*z|G_Q;%8EQOiiCcKH#KP1z zN?qnBTqgJqbw#u>0&K#9jW6Ca9pWY1wbN})K3Yg4!;80nYH9y7eC2N}2;f9$LPx?! zMsy3G1i@C~8W}MUx<^$p61=tPEzfRMTVf_H@>Ae6xd<6Ak#BN4r5;ChZ&?IU-C}(E zGrvTN!C(IE{$E9C>)gNbsQ&x%Af~=yp;6=$9rFSLOh1IO_q4TrYecc`ID=f2>nR{4 z3V<>+mH0bGmTLc`BLSU2y_q6@V&1XiVA4QP)$yGcYAoH!c( zL!sSb<|_=EDD5(ZYR$V)fFC+`SWwIdY|I?ooi!LPx`Cyjx>&XO z*2(enkC2fqhk|(82WE{Dy|p-Qlks`zl;f@~wvp$6si}ICJqevqs-hJBBMkzg*bsiz z=6mv7hoW!|?ohAjQXHeiBF?e!kdT4Bs&M&Eo4V+Svl$6fVDuJ@%@P=+aVA!h1Rb$5 z)Q?nNZcT&qM*{dI9lcACTiaG858$zn7PyRa#2=JFei8!XZzFb{WU&uxpO2D^Brb>b z8v#aQ-%NU?3(TZ_AG-IUUCP{kwO0Hofk$iH2$Cl(-dW*N9NP!XUD5k%P5e#B`d^y^ z|ArPoHFjC-T}lMZ{Dptae2~IHR2LGmry_f6V`}n)Gx>?q6n%Yo7%MXWm-)>v|&m)9B#V*<(G54;h|Br{nMTFIF z^Dr{s&Ozo*_w+(Z0E#>dl{*a7De@TiHymiD-Q>1TAzd~2lC$O9@4} zSndO2LGEsTdrh&Jl2T=WBEChC|@NL%I2UWOnO7aa!tWy(bV+I6zXn z0FwO#k6+Kl$lC1hr92?AD+f*NS`kQLU;oX<`5@?Zg?(=y;HsYjl)~K88d;szjNe`F z6B#tlLT#v#L$3h2$#^Hn6wW_-xfiOO`MF;TgxG!2*x=O+zVWsCL#T*Dm<|-P3tnVX ztqTAk*!9ry)vn+F2vYLi9uJUmYD$8UzlSb3226wZ*hmD4T`id>Is7p3@TV?BFrC(| zgtTXCmIfpZb`6Kn*RTK}OS;NkM^@k>S_r{Tj~XMu!h;P$_VjY^;F6JIH*YtXhR2|a!FWRmtYl@V;as= zgs|@dLJ0o?gs`2#*3YRx04}wSAQt3^d%)nev5*gT1&Nyg^~rBAMQOJ; z3@A-N&KqP`WL(vga`e`a#T>X2FF{8{wd${+&DKS=q0oR3gem(0R^&^72y0&kc1P-i zThd${3g(d6_u^tMEZp_yg-b{y4~Zn0rFs{~kVPBz6E#D{sK zkM_gCh86-Y2z|X)?!C2IbZ!8gSmlAuYy*iillm zUnB?`@9*s^bjY@@m2k5(jGxOSkXO+Zr(|mcY^#<}P4-a`!(#5s<|2+m{5OG3Ux-Qp zDlf(4!|;NRwujbFm=lwH@EhX+IHo>9!x?;9dPyLN^~2Sp7+G`Y2A*62^feSo!h6^h z!K!f<#6&HPB+zDw`2T6{+QXsT+Wr`mJwy$%<&b8aCX}3_D2*YFLur?YkQg#jO3tDg z)OKztG*eCy+D=6jLs5f~p_~edBBC5Bhf1aIe)g{S-S2n3*LPjtAK!a@{u*;V^E~TW zYu#(z>v!M3-)-X@{POG@;MrSo6o@rM$~I6r@jNaz)~dxM4g4TVWNiZ%>!*-Wx$r7Q z>h3M0s{%lyd<+x$b?y+&`~7gE=iN5#W_kC8YYh(zd=y*S6B|-yR;)3!ZINp=+L^FA z2OP-~<4-RHl_J|ASfiWui`={kMw?0Z|FKex{)2rCN(`@gmqvRE74+{tc$mf4Lio!- z7ggJ~amV~b7g4wR;b#cI6(P)5RhvwoP}dVC2u&E~XsOo8VBjAI&2*UVc?07g_~_Ic z*C=D3?mtgmh+~Z?*L#D+XiLEIE(!_rb0rn|+Y(nUJbu#+5lX56;+8wG!%FwQL`cU# zKEBjr8tQi_)&E`$9D-!DqX-1ikFGXsWFHg*sliI0zRrOsa1gC{C0oS0p>_z$wN`+> z%;g>mqVrw@OGLuvhXE_$$!K^oB9c>cV<(1Obq^AHmZ1?o^kC}%sWf;ztw|y zDDsGi>yhap!eQJ3bfpOpm1M`skyUWX* zo)2GZH!eFq#a%2VoM=%=zqH@@K+P4IIjv*|*+iAcLqGxb^y{zAsnLDJywO#`_Gy;kwJiBFw_GAy2w zo6X-}z_{%~a!A@F7eBlWwk>)G+t>0VO-AZgq(_O`No!lP>(H3in_+X~hdbxYgF~ds z;gB*GTIb-LdaWZ+Z5{+^GSbpr9H`S;c;1tP~d>8Rw7&v8f z>b%RY(WVhp0At<#lqmpFi4a6m#@YYL-Df?1&HEJijl0#vTlz5nW_hrytU$6k_80De*JS?6$V zhV+G~0N=z#!0@F4ry6V70OLLnUxIUa;f*M*K-3d*GrzilItpn6Lrz^K2Mu}v2mBbS zz=X_PMCUXBz@f!5{e2ZsGNKX02-6~kSj2TMzZ#)0ZH*r?z^=ep7dM6_-SD-lvZ#-c zs<&KcJ)8k5kE|C*%c4k(JMTm!E}z??XE1Og3ZY4-4hr|0JuR}g0a-6g=aA7<%a7%~ zR?ls$KQ%t9z9}u{;vM^qrgc(=9M@XzY`qotlP_-#g>e3+2elpDD>w$Rn7tr-cdY+h zU?USzwv7_=>Fpxry#2}_mT&c?!hyZe{S>7eij`1YnwMloH>g6pZnzcf9{ii7t(?X zT0b>95MS$u>Xt3!p7M1c>1_%F7Vq%?oC|ft^gk-iyM=&3D0utJT zupjXD5gahiknMbH5%^nwW1Ok4LW90hdME%}Js>C9gXB~|&~ZPom`s5g1$>Q*yMKW4 zj_dzbR6&HO1#v7T7o^zCJh%ZInOayKl9#hU5VHu_Q_djhdH^YeGhGpaWUeCn{(jZe z7=Y1#D=@LU|Me5!8V!bVuLi+z^g=g1yG6^zYvLXfs35hIT?3Lk5Yf*E%AQwd>vtH0 za?okwwO<>!30V8E9?$QxLRNZWJ#fc+fVbU^6f!dOAJ2x;)rgMe5zP#PneVrQ@^eDF zo#f9Soa(WA*6-*TZzOVjU19beeCel#Wm4BIkKMF;8}|hrvBzYY!HJ?fZq6d=agpm3 z7k8}sm?*tt*^Z;qjv6JEB~lp%uTSW1edeDz*s}FP-aW>%Bbn3`iwI)f>odAU)^}D; zPL4}s?KJ|mHYe}uE;=egNmMCJS_cGWk`^Wbz`*AI9$pnG&_`USS(HkzO|6=5dkB2R@#RE5eu)OiiZxO-Re+Ijr|OvV?Ow|N0T-W zW7`n?>sG?sJKSl&^XgB718>ev<;x@JgePPpjY0Cs$@I!_OBq@eU$fv*=8&E>)gIhvwEx`0HymJs-yFLKo<`sZ$#2CCi<#w0kT90v?m%F(* zp*R}~m08VAn+@^tqE?}@m_I~hK1U6w9Xzh>kj8>oh11zq7`atRST?A4G(v@&1|(U1 z47{eNZ{@2&QQz9;P}tO_3ZP|9RAbkK%*_EK_id1xhMVb|^buHU43Ny1OwJi@BdUSrTvs<@(|m>wJH{-;am5207!n8W(Y4MZ|&SYJ<&@XF`~X%EjMMsv*~%tUYY z#Y)}Pj)8-JCYBiPR91SCQPL~JACydp`(5lVL3ovH4hi+FFvHL?%i2nxz5U`2>~Z`WW*Emq!_h4d0sEz$s78 z; z6k#$^HkdbYn8}B{{3x!6t#=j+>Zmr|3K$W#71!}Ia9E~N#F*q$p#EBMoP-G!2bY%; zEh$l#)Bzm1s2ZU902?_qVkCvz3?@UHoQx|qW^W6ts)jI9rK7N3)p@opa{!K+CAT|g zUC8EK=0WYA{n<(;(Ywn+jWkCzFy4e86I!N&!&IyJZG7V93f|-D&b!`-DFy|}{o)S7 z!S3OsjM@cK6DI~K4fEg%fceup6$FBNlz$s6DBeC=Wf5>}O7#SH4d|3{Dso|6`btV) zhI^Y$#{fPufK!}rM2XRa3YF@Sl6bqr3K0O5vYK9Oe*{|Q<743&6Q_T#$75d@2$@uW zRlu?in~U;3G~AbpBEGQV-Kuh0l7-Z%XZ)|9XXkIW!QuPjy#zngKNB0Yki4sDwOKHP zQ4go*b8C_e@k=AEhf67w zj)a$A6au=Xk3P0rv%XMQ9d`n-ivwnhxD^#Rrp8jAVB?zFaYnA^* zr#>D|?}cNC4&|}d%N|=QVnq6euI<}g2U{eY_}TaFXHSaY~#1O)0E%5_02^l{meiHFna}eI!hWW27Cny)yY_Yc^sa2P$oK zF04ALr?)R!<(vvf9KBy*gF{EQKwY_eDQ4)PN$$a~t);aRuN8mCj5y|WJh|{9IYNqA^I^WWVUE*ayT9 zmT1o9($e4*WrqN&L4KE$gT%3qvA^XE+Zj(nQipnsGR59b1A$x8Pp`BuSi7*%K3wcR zq-f-p5pnNNLiFS^Ka=SAosbunh-pWRGW>X7=e@u3r9;j}*Ti7`O+s@)@nwJ;l)Rc%?_`#}4 z8>PsxybHU%)f&|>_lZAmg`FYikDKQdz#U9qTYka29{S*A#$OLXNs3Q`b8H?5WmR7$ zKiS^8IyM#YYg!4^b5r$nQFl^joSrbxJ0CK%?CdvN)XyLyO(q;z@bm&@0@xY(_AQe2 zzR`!>qWL%0tY#h-r8o)*U`hC2!H%`ok2>>sE|PxSs@sBsItc<68hDfdmP^Ll-4id$ z(v+1RJZoLt@fa31#Rn-X;AbfGyhKcoFZWGBd~qyO=BMr9=qL<+&qgpyA#y&|i7T@1 ziGYLKigE(njg+nWbH9>dCI7E~Qum0*8?*D{#gDle4$u>>x80ILBadSf@xGDgClHT9 zdqI;LcYIZ-Az9Ubd51C5%EJ;dyLizSCtsD2c?nn6s1R@)7&T{&?)B!bk=2t&cZidC zW=C{nMHz1}kjvsVkbj`(=+QD0{Rda{k2!`Z4k6vpZC1}=I^2t>(+>c!Zqp`~Hvz@? z0|TzDH)PP-SF$rlA?@uLU>*ay=gr_fL(tbh(`Lx3m0sgxzsql+)6bwkj7>lZgvGMn zC+jvw%Nrbb+7Pf$4Z13WRc9_AtI~$h>IX++stF1mA)Gf>z)F5^WLzkQ?!bM5 zC6f+n>|;C!LNJPy?q?~OjQ^tk{557Uhi-!Nf~(Gx3YTQt1( zPW-0rQ{m>GfbChH0sX9b2L>Ak(ifvVZXbE3NM2$Gvt$M-dXsU z$WYhS<#YWzUm0QWS`=u$N#~dg4tt5t<@5N~_XO%CdNp*lxm=8oEnA2I{yEuGQcK4< zy1og=yw<1axwQ1fS82Fu3bo#7Aq`rO~)T7O9VO2?Sl*gt9}i-4x_4oOyY_iaC!Qxay6?SfL1# z#Wb(g7J{0PxrwpTc8LR@*E~sVJ^be{a+IJPyu6A zH-sqLm<$v~QH9Qo?LMI=hYrWVv*Ab*uJuX+jPL|>QoWejEFQ(U0?*soAdJDkz~ZU$ z!p1Py9t-2CjU(?&CZjPa1Ugeq+vDF}i%b|$xf6L4OU!iT zfdwW@fQfl;W{WY5u`ogFma~jk`@!qsHbYxq2we?<8M7ODE%yZmjXMF&Ny22oSL5(0 z=U~i&Nwxs=jZpR}D=f*?3{$(D1<5KjQ#es^#5cu|;mTRKvh5lDS5mg-KkcqYNMfFh zr@~K}C8p2tdlLV$vtwKL$f7NvMancg!({?69BMMyAEre~fgZA`=v2e5V|0Jj)7266 z0b7nok@J@e3*#u~-~;#KzrFU>M;WL&KenzAB)Jj@f&y4GOPFKVUwY|d7V*&8uq-XQ z@>HO$FQBaqxtK99*Ddh+Ns*FR(q9wqG#Ld|NJzZsiG&fq8K$lI<@<1&RbbZ_i_f5p`?EhHe$nkp4JU`q$Yf+sjN)G%fJVTeLt6qf4z3_2o(_t z&jxyJ^Z&`*h5|lM&sksDeDeR(+#Ul+e&3#z+vaelLTFsfMVK;sA9VO&=S==uX6720 zR$#>SFbHS^Tw4?h;|F~xzl?~Z)0ODVlggi>Q5Z6j&J=!5zO@o9hJ$fLWgLXTS5ka; ZrtwvY!Jdx2hfwfmv17McDcLphzW_oxjr#xq literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh3.png b/website/www/site/static/images/blog/dyi-cdp-genai-beam/pipeline-6-refresh3.png new file mode 100644 index 0000000000000000000000000000000000000000..d90853fc1daa6fcae2eaae658b8936ea3ee8a696 GIT binary patch literal 53972 zcmb@t1yq$^_caPfKtj5^B_*Z18>Hk=!T|;8?w0NjC8bMBL69yf0TGdqZV`~~xclfY z-uK@7jqm@Bdl>_Hd}7zyd+j;rT08QQx&kIT89D+20;ZCptQG&;V8!h<(VlE5gK8z64kt}po_}3kir#-Hv1KgO!&;0eJWLeCGTh3LF;*2)8RtE z$koQDCf?I|M9CdN1%^Sn3B-PUq6qUB*FTg7SX{qoOCnuJl5Zm3#S$>-iHSnLTXU~I zfb&| z5{2EP8tBxoFyg6km~%x=VmM>sbvJ!+2+#WZ`<4kb?Nhp_1lBfCRJ^n!j1Mr(3C;II zAdNI@7oK6IG_2}WOixmV(PESOf;sOj1tLQbC>2fB?j}p|`UG32?pqMQWHqu{N!S5p9#LF@9^4I7RwXjM1^{%Y3~Ug zIU~hM8UmG!{2ul%3TPA8yb-kM`ZkB!tesc2KZK2BAVJ~7Oadc!0*T~3S`=J6N59#- zr(NAk6WEWLvVwf>4X-qdEup=R-b-+d{jAo=r}T*ZhGozxk9_Mci*w?QIftW4>5r(( zmH9`MtSiEe;x&E7dDQoJF~pjs(~2StUlCqt%mie1Vr$626x)PoW zhwBzL-Ed1BK*z%mhsk!&rzL)dZy%bHpS)#kz!3?|?o>D3sG*0_=b68O;(rUGOkG}@ zIEFTS2y5x>Umge&ei3j+?p^mqbzS6}(21~r;wj-?iPFMYAfNt2WA^?}ta-dk#6cRC z7)@9Oo9c1uBM6G^oClMWna`&^?u`978yUX6gYfHHXJ=>kLeuBR2q#V5I}2CYZ!J1B z!g&xZGn??7UT1X|5kq`cN;|WQ@5Tg@)`T58LF_v)&VnRy(CMTo6jAVDNBWN0(VaS6 z_@C|+yc>9DV%e)xiZ5lIk}BT!|LtnrSX2DT^SMW@IfsT2;LwA}Lu zDzg|Uje7)sm!xlkBm+gHR8rzY27FvPyr=hhs96+P6WsbO3W=t}AJQ`=xF`}kV|WD~ z(N!mEFo%`r8I;mmN5;vMGUH>)tb9g$eqR{*fwanJ#!a$!D089wc}$-%Hu>I>`vqZ{ zB4~ef1t?MbtE93C8a+q<7JCe%cA``7{?RCA)Jf|pI?`LR z_;$bdOxXACeBk>Py}i4h5;w7uj3hYQs5wadw2=u)8OqFzI(QAZSh(t)tzTt46&Uc` zA|A_nG!iFAPWO=Z^!6P0i1n=Wa4gd=BSgx6p`T7{QA%XMDht$6Hc*+7vt~5Df1DuB z%)&;>Cd7iwQpAjw8kNeL>ds23{h{(m=LfHTwyo3&gF=$m>W|f;)iSBdzG`xz zRT*jNz&QKZY`&M)AueXY%Q5If4m)u>wjF{u3ely{ih92deq)`Sc{5QqQT?WDCqFw+ zOs!e@vef-OR`=+EehVoRx=yn1}*5M6ISQ3(Z$8Q|#V-+04+a<{qBvgPOK{X`DC{GR_<+-1sv2`TLLS z`zZHm@8cy8-_O3EtM*29MJ-zGG-IB(AcOl8&!@+)>R%pbl4r*A=2|_m^#A_CwzHkF zIvBH*4{&jH`{XvegoW2DwF8$)}TZ!mX&=44fj9%vi zcuo7PdF~u}^MA8u57_=z8{eYVDm0(j+U&O}ZY7@S`@v7mSE!Az4g31jm8bvrs~jG? zjOkCtpVDvQZp?0FZhqTh5-EG(1yYyh?v(xL9$1%<1FkIT0=P^i$ z+x>)zr}>rn#j=!f_vIDj^yDNWN%47k*HzgcxNB{XOnErnIH|1c_89i5EPu!Qil>Ed zjX%$X!tj{EoherRRjMnkzJz|lY>Kq9lHzn450|*(Rr$VHSyfq4*(4KGErWk)R`xow z2|r$4E3@aND@|-dT1Eq~22+xDH>sj!iC)R6i~j7yEU~D>yz&#yg-nyqfLBY|nu+`D z<039{77l3+kDoox4P=Bed8Se(wQ^tjsXrY#$?BZ#nHBR=_OkLCmR5ts)%ewoty(2F z%ev=lX`T{{vQ!iwMR)gnKS;bnJ6A(C*i&eeF?(<>caZnFq^5kY`)f$q-C`?MeD$9h z_9A{`MO3N&j}-^8GM(rZ!KBrTyM4Vo+0j|VZ;X%L#OS%V&DOTm zT@9J-E+4kXox2TJ2dnU;xpKG}et?lWy>y&DUTYij(0Q&ypet&?Z7Y%4) z;8yje$a%$j(0X8Nmv71D)wfsG#tkP!A1d7@%daX zT)k7O`LVvfIE=}c#+PasVTk@-boy1z@h69v1?%@e9M~34+?pMIPUA8tcj|ZR>qRi# zIo`S3*77i(p{!{xn{V?!q{)D`#N@}#EEOO8?_(KUfv39yIUl_G5(3@NWfK5$tv2h z(wg>4N&RS64&@Kbt=1#otHT`PeB#_o<=d}kZ!e0Dhk^snuP4TGzb+gqUs>-RFAI1( zG_<>UedovFbG$X4ZyxlNsi?xo46(wjE zx2=Z|OgP++uz2i_P&$C{dGS1k=ZXJKGNG)$MJnxHZp~E;Gn+la*Y4ZXP4P1impg&2 zhD-YxTNOnR964l~kvn-KBi(MBmG?#TlreB~?(oa`s*S4Ly)Et^&64Q3cEKnt=UCi0 zF+p1P4S5CO{|+cc)HA4(g_;@yE4W5QKtd!#Kn7Qc;8z@x{GV%iL}r9Lzwbj35W=kx zkp6l{9sGs=B!FM|H-G-#NeV-_3;u@(em(Obf4>bO%DeOT^&LU*3_(g;Mo9_$)i!s2 z_RQYpxr6J`3X2H1f##^F?}C6p#0dXIRMKMD1Lc3S(t)}{)l`Me9qc$@PaVvjad_G} z!plJr^%Mq|cF$a4G@f?0_AbJnVsyXX5C+%q+njVXzh7~+5u<~uJ))6uaDGO^&%wpP zMJJ9*VQ^?=t$Ubv1ceZkLwQ{hhftL$2b8vGNqoadY^v}-rs!pY6S#raR!;8RifUExPop3iLcWv%SMn1MRP1$iEd{(k@e@#Nnc|Km&O zzhCn4{@0iP@#O#hrLM~}XBh`OP^GK*zvlX@+W&m`*N38<@Sgw2Nc?{i=&Hj z{$n$7bfNPOVPHlID_IR4@E1%n_zz+h_=ow=U--3UjQG;vB?5vZf|9J1jwj-FHrkb5 z_suPna+I=_X42D0e!}UdXA9}JlX5K5gKewwML0=p&C!GIZ)cx2RUK>jGkhoffcxw` zIlt~{ne0fwlVqXc4PS|&tkHAdk&RCqEyaEB-wt17o%^mIzw{c-GRSd#5r?EmGl6uo zaC;fxHPrs2ICI@^B%DDK!jOmdHHwwJs%f2!a58~b#x+Yc2?0$|KCr8+7Tw%ki3TaC z@$yH1*Yaq<&0E&_U~F-c{EQ=I2txt`gn&q?9CJMsNm+$v*Z);l{=4J8Da8&Ya9{l~ zIjpr^QbHHwe@xC9sy(7~H!r4~dyMc_p(gMD^> zG@T@bP&vjQymolZoQNACy`Nt45+>TueMn8eXU>F($4rih9$okPSr5Y_Cy3MOcw(#o zn|cf>4I*BIYG4-~kF-4K(cWD|bPRrEJQ_qPSSS3(BUVsR5#4~NVh0RdVd+u5RfnL_ z>1bX7pPmV!%%iE&#CGixgz`fN8G#6P&5#3@|jpwh{Q=RE%*B@G}C^6x{WEP;k z+;=RU8F1)-+u>s-cO6U%*zFm-+WR#zX-Ha%pkbk@BXr4<381b7)N0UxC`Ry@UFg5t z7m6lJM3ipg-^AKg1M6lrBDk*s28SIq!EAt=0*xV8a`xKwDGc|0z12ZQf)E4ZDe5C= zqn_m_ZnP=9i00hlnwmGH7of}cLE*irL-|EHpu(+81jGc%E6}H9%Mk)48V=B8r$h^3 z&`5D$2x?>v#k?(0kdAR`og_noFL*pqO~0N1+6a1_A?DM-YSR}-8U28_I}rg*Cj%52 z$`&|~EJ=#iGpC`zKhI8&A+H8uz`+L%&5-0J#ZyFtkE9Myv?3%F6ih(ZvjHk=K+T?5PQeK72c% z*5+jAGQ+iGD$upr==6m7rpAkdAopC`QE8eOk-WYLVz*K)v-^83sO~> zj}o_&A+o$uxH;=S>>r)9mFV4PrU=M5dF>y`iWK@2JGXC7^x$wJmzuD8ZA0YnP)qQJ zYe&WpEB4pj>~^CaPSN^E+I^)P{?nI3br!ZP_i8OlM75<~A;UL35?m-PEt%!$hlAPC_0^oO z!72{=v?1I`FEO+?OUOlM*3hBVt?D73mkGJUD}JJ;u=YB_JOY<=&6QrxEMJ%O&a>`M z#$l1fKN1w~+Z{l==~bJ%vCkNMv2-&z5;4D!;Z=&LnIDLqwv)Z3C%*f%nAa^Cjl+A>2hl#T<{^@V-g zPcA!+TW?_UFFl7`xeg&?PAe^su3(2o%;JY$Q;l{9DpV)@go{?fdZD5EVS*!I+L)Si zFQ-Y^vM}sV5aBjePY`3-Qs2wGcRW#0t&3EVyQ1>?k?+NMKkw?x8~NIZfa9zt<0~6m zR#!xq4U=Kqp%~`xXggw~$BI|wG9TZjn55o~zx8}VfKA^LHM(f?(d1z~WIufU_2YdK z(Ssfnf_REG>;ukM3PESxdym>>4^tK@+b8%5-FHYEG;TI}>25%)2_#kywPiQ}%K5hw^mc>KcFdCshSc<2Vt(Cz z#lV=1aa^_`Q;kGY3`A1wV8~PWFzt|`W1XWwMARLQtyddI##e5!jn>wUlC(Go`v!lD zCJ4!BN8qnr!{dlm*U+UtD5d-g?#JPeDG>E8V;e86&@tLn@6R$tw*fb;NCV&cwpnw! zUUYR;6@4Z~OF%#?%t8naHNKNqlUGw?!*V1C0nUwuNfHK|L||qIZyNw72RxVLAlOr< z_yWP+To5m=#bQ@K@z|Z^eouM@CIjtR1}5AGDpLNb6l~yK6KT35C_IO6J$ib|PS2h@ zVotuX({tONaW@>Zds}r=|6$-3-p)WIJhi;6Yx#T(DjpXCBE~HAu@}-9kl>hZGo2T@oa4pY>j^`nv z2Gd;CQLIXgGNQ*04MCQZV-&Yn{r%E5ywSD!HOyL2(+m3Ng)cQIR`H3>yQX#Ne-A4H zT1qAue@Ym7G+l^X151MRGEI67Gr>p|*^lVomdu-l+YD73NEZYnA|G-uP>Ij-;NU5RJS&qB%Q5fI<&Fm_?_sdj;irb%O34~pA;UI zxV?G{4Y)q`yB$hQnCa_ADqd6b582cShz*mW0 zEv9U|=%!hVsB^jauusqPkYN$2s8|VyWnU;aBs8RVh_jvjWZuUCk^GLJ?bE}KJJ>qx z?V5PSH4)PBFk^WCHGkCg0)Rq95dT>p!*DOfmV zPZFbw&Qp=y#)1rfyH`+;Ikz1;jSNoo=n&lqnQ5tJY9Z%B>syjH-KEAPcW+U#1>ClD z#@i!{4pcgBllh6tCBKx8|D38V2rwZN@x~zL_+$vRBKbk-+)qn& zWna^YG7koA8Iwx1v2cmX??}_3y7fdW09)d!M!;cS^w-%+{OQblhhKJ$OJVckm)k3` zKE7Y@|C#MFBJ?DlN~}dbmdr~_)vG{0_Mq)@S44u{ zQ)y*5X0PJ$n}gl9OZ#fQ>L*WretXjzhDq-I-M;zFWVMOJuWoFKPkTccr>GgZxLGPr zFk{p+zHSvw7$tvYpV?(>&5x9sfZ6MJ5e9*cycB%x!oIbb49x5w+XXchNY5i0nc3;g zRdtZ2Ly(0B?qG|4zU4`&kZpO%dx-J%XI1mase^yya*NmD`uyRjL~i{otb2fv-Cwx* zS6lPeRq&6ci+=j}%CU1|JR%$fL;I#8>UL6x_x5umWSNa(?q`L*$=)Bg)KmXgse}8q}Z2~BdvkomYyE2P`G*R-r ziY!fLP2=JVj4O5LH}&5RsC>3-Tgt?aKFj17vE+AsePcK&vfqQ>s;wc$S1{i993n5J zs;4Tb4P8_r`>{m4XjPKE?zS^CF)ENVAV8Km;1wDoce*=wz30qQ=CS!Yyc{>&eP`zU z)UD4H9m8sBOJc$H?QqL$neg&=CsXFww{5eBT|O(Z0z3!Z7#^src&O(+a@??dyw^1~ z^olT8-@qwhR#gY83`?=hAYpZ`Q;trEq@rH)JJKkHJeNW9)EV1gWCDCdz{K|_&?K4Mi&Dmu8&02W!rLHHh z#i!4mU*DnoYyRL_E)7NgVmsSdd%OMS{ro}q5|TEDFa^@x>}I4c)KD&e6{V1_t~iUQ zFj~ydqE}7p`41DEV+@d1LQG$~_3oi?mBlN)&vM=ddb8ebNf=d&Ei zbhS}4{kdw&vA8DT^7-cG)yl!zP;0ST=7p8``D*gbTi>qb&4T!o?t5ep`>_i&+HP-r zZkt%x{m<8kv9;wI!^g7q-&*{j**cr|`Q-(cYx;%HQYbM`&VH8|g#Sh7M~x~>|mDW4=*FhE&(2R`ZP%cYWsf9-9N+jgvPl~^Hym28$D6P zRRn$9UM`FksX+X#<><-ynvxrCKNH1)Sah&-ia&ka6(b`i*s^TszN5-;o6%&JG8fIs zC-$1x;NkY)XU7uWq|xArKNL}!(hxnJb|65T5l2Ga*;TKvu)d3phuJ!)X~ zl!aB>77&qG1v1yvHU;L|E*I+Xlk@R>A}6H9At2%k>2ZZT4$C+}1aAMJ?R-6x%qv9G zqJeYV=(I+Z0N9GZ3(`O#?8ojg0p?7{7cEK@)feC@&?LsxRNp$C!}!Y6)fdM?O6H5{ zRf~IkU~FTAwoV`oKuRJ}EmGg)h>aN}7NlcHdF3gM&o%s;$|_&|azf2KT_l>d@U&gP zoP>j%OpT?0^^#wudig7&nQZTNJQ}!K(v2XUn01b^#IBuw2LnvfaqMTDRLLrWy{iMs zyy$NaM?|70pZKWFWj?e@N^mD49StUaVA!C;$-%~-l{JH7nd4O)^;kW$)FTaPTyQN# zJLKKcqxZmv8qn%Bl^*(%KOG=sT%Pk-(7ky?Ws73*f$JA~XC_&$*E;Dw@HwT!39h22 z^FGxH;}0*A`dDd9?R^Q^;|K^LqbsgOn*z(9s@2(c{e|mlBK#CM)mWEnbubyumTQ=& z>->HxraUC!e*%$$;rgPH3O{%KI-{1sCDpH%dsHL{COYzN=Ku?;Q!k<3+2}zw0r`kL zRFPhD|K{>n#VTzGO~m2>S~T#wMxr-oD}xKilO~*n#*xC4w`+Jr)V_N`3EE79-@h3; zJP>x88M#65-WJr(od71W~3vnL!(`^CBTDD(k-`r{cf&9-rmH? zp(Rm-X(=bgS&?nw9ZE4j`t`JWZU`H#A5&1^72P1ohe>`Mi+L=OZLS54U=&PIcESn! z!4#GoUJ}c}xW^I)@z9ru!OE3+@I}0?qxikSuC==T#z3JgPK)B~JZP-6$?$nw8BXPF z!b0vA_T^*K618#U^d;++;gVec8SL;m3GUFR!Ym)IHZFJG=blGw=*miJM4PfYZG9~x zWYcH4j1|Pd^^h3GjZSb77-r(=j(f7WQ*`Zpd|aY@&^v5tkH`GbEJ*9YlNRVTp2@rC ziL4C8+Lx8mUsSARWp+<#E>3pXRvRV<+`k|I(-}CNsbK zgvg&llr~5Ue=#DIajdkQ#h#G>zB>w~GRbq*%-o^JEeJiGc>bXbMWX z+VnmMRTj@WKiu$HuS)f#y88L~b#V9ue(|4n&v6I;*tChEVu#jrs8%hGWZvGKwofEM zDZ{;b=1IKU(Pt;=i4-NHn6NXQUfu+6csiiRnEg=9%R(BQc1Yn}fb&qj(4Ooitxk_W zB#4O$($EEIBIPm-T<|iBwzgW`wA8wB)?=gIgYe9inel~OtyMLgVyQpa41{&R?1JRl zS+(cQ%jJ*v9EHD2FXc`RIyz7b-Mlj0P_5klY?xx+T_bV*LxW22rD0fR7X-HNwKLcJ zK74{#m{U+XL9rSCS2(r*XQe$w$TRvy>vmvmN-tJfY`X@lae4odIj1Ttt0tKYp$Ei> zsu-WLMfCdpC_-j;XM{;J!HkQz;yKU@?H|KF_V!SPeJ#`O6CO6AN+fQ2_*l)VAFzt~ zy7_c>N`NTaaLz`o#_~fG=ol*`<{5w;9P1Z=@&m|=vayp>?}qiA$Y|_wC8i*wF1qZ8 zP;0z?h7N!FWQ8g=^k_UXhxHeqE*TclflvkqtCdLLLS&18h=9obL1>$n6p;1^Cq79N zDZx2}(Nmby&G|+}%Brv{oI*qcqDRHWi8<+RqC9$5_C5X(0VSbwoG9iTLIJiioR+ zhFm@!`(1ru3h1_RuQn9Wje3F$I=FtAf)8il2!C7Scl+1i+>Z<)6i^XkmDilVNwyF$ zS~CielYNgT^jH~N%%&NE*_|S}4}nfD37L@XnDD3n?ZX}G#~NoOjn=Zi`Bp<~rB!pq z#ccL}PE;_xdT2$*KkFznR;mBC(%-20{EG@_B%?}Ngx|@u!b97s&-L59?F$=nL|N$jvQL_G@d(M0tnK*wmKhdl;1Zx~UqD46-(-l^r^ zJDu~$WgQ2^gU!f?`?U!-@HG@*44ebeX2BdE5j|8TTvHR6BZv^5PPe5QsLHh-$Yx^&I`uEizVa3<7_+2?;u`T?v0-4?@u$dXE~YkFsU68{`pk)@~gH+Byg3{yK~JI z5+qN}KA@a#O;&RnytjXVCAud`7*4u(x)4ANM~WZn!Db!}$KmnO!_jezfJPF4)lheK zbvTL)t+$>~Hnj#o-W?BH>?i^H!L^K41?+BaI8;h`tMI9np!L@bWQ_`ZQv+>sE+XN# z&8Vvv+^*^FscPOW|KhX$(lUjn!+94VlWKV?RjDuS0Rgqg-wZGP!>}k>0Xq?xSZV`n z&$_%f;H*yx@b+H77vF4)9Pvg3$BSPm!qFw`r#|X{O6v2z4{krB;1U5El^^~ej-1Yv zPG5zl$SHGy2W&$v@E*cLtWfq$xtV zj>k~nz0SKg{oR3xcaY@q7nNz*aj=nkcAI~304B+4I=&B3MOh+_sK~`;fgB)=j^$ZI zM2#J&y*WO+EW)mvdBP_i^Ii(E6p}TjvNG7UUDdHQf$dygi0w92P4clVruw@};nY>~ z&j64l=eGIA&J7r|Y#Bvp=>1OfCip~8Ec$n%lf(9b@he!6?roS)^q+6!a#~N67kyz1 zt(h_n#_Z9z%x3&%65!W<4+oK6A&vn05|04Y09x64G6SLO%laaTl;ItO({(h+m~NDW zYxx>Cy2k4z&$B4co%%_nldHzf{OBIB-Sxr7_IZbcJ4z;jvuq}hJ5e~(S!6JuEs4pH%N*uYxQ zU8+X~M9_KR=bveA6N!lWe>wG0ZY)eP0nG5hb2U*R<*eyoknNg~ou8fvcIf=0C>>oD z|9E>^d)jSNbd$bV;a%hFn$J3(?8N_U3i(Ea&|ObsJDRoit}_*t4g#Wcvw2W#@9S_X zR)CC>m);O`EoYd=;eK~$_mgjF^gNli?8V@0p)q*({I$H&ySop7+8cDu{K9^BwrP1p zFAx`&RT_M%S^ZcEMLe{HD^ zMH1-mQcv9!tc}4p^fDKg{{DEg=LK?*3*gG8&X0(qKp)~k4dVzD0dDp0& zmRd3y5nF3yr=SU(DM>cLeAwG~Ig)0S!b}2kP<{s&zKWsG>Fyi;?k`41iCxmki`-a{ zsuOa6o8YcJa^)~%<{|Kb?x`(cAy2GB;cR=#JYLNA$&`TEmqMj2GID)nlkH=G<)eXy zVJJ3QFO^o&1qtm^8LlxwzNcC%1^ft14s-r3q)(R?{mv~LJsesz{=Jz+(V5b-<8b%f zkwrSOA>nx%F_VFQn$E6IR|S|0qj1eo;d~Ccp__Ti3aCPFhm4nX~tM zmGscw7swbrjZP~%UXT23*F#fzp+PSwrS|;^*%d+yY95iCnUMaL?*3yT8p49I89=D7 z!abu0>t#5Fb3`sx!`qMQFC9JWg-@D0ZiqAJI*Rq+k|Q3~1T*_1iT%C_Pg^nS61E_rzDX&H82!6BbGU1kE z4SisC@6nI(2~M}!U`mvRjgMj4dd6<42ikACAZxwFMnL3jIRg&Gv1~Bj?jvo7d_2{g z-Q%6_dam13n%RdEc+BvKvPgMiD6+p$N9W$BzqfY-14zbA2bOmYZ-@GOl2*mc;77ro ztb^E2p=J5p<{z2wd9skr%SbCg*69A3Jt-BCAGZ#jSSi!x(?XZ^z0NwGX<=YOY7(xEx#K zsSX;)0AdRY0(CWl*GJ>J%3z$Ejib=^dr^`SqD=V}^AHfHbQX_)0M+5Ke0i})OAsMS z);inL@P;T;kQUxQa2IIx6*{p!6DV5c2fS2xj^D*aFdq2d3S z<3D)^T;Y(a!wQZrClp{(Ai|-w`CDVxQU`{uLVWZ;Lt~Op!Nl5IG*w+o0!Xj1C&P;o z4&SW*9rVVT=U}B)9a#Jhx^%cfwgtAW6S`h#)52l){~p-gBjJ)Pas}eY?+Au14O$th zMKE04r=b1Eusy8>dSx@Yz_L1PmZEUnED2dc{ey6{0jg#AU&MU$@=+ICN#R=Y8Q39v z9vj**l7TJ1^opC8;ZZCc6IiGIg}n9XP%91Um5sdCQkQ|x#v4lKJUx83mSz~$e!b(6 zZ{8PoBbEaoaLRAZTjOsGA4&81y@SdA!4PLOP-Y`@#C)PK1!f<$02RjUTcy6)oZBR! z$AaH`xK@pqsAus4sXP{rhGXT4h$L>#3Y5#Ff>Hh>v=obulfi8sCJ1B$x#at5Un6>f z>#cHL^2%uhOt=@QuQcw{&xoCFuTS-8JXDjA%5(ofHjrGfWUcKCY&4j89;a@)*b%f0 zpb4QMJSsqn+x5+?kJ5vn{rV)+bGK=uhse-2aX*U~jN=%5A(;QO5D1w9$0DlRE|f(N z`swox-`n#YxXe2BMNXOo7{`zcs8e`vzUIrZY~BZ;D;m&1DCvuD<+>{G?7lxf+MP3` zaO^;wZmIBr#~H{7&-c2qk>y$6Pg{`hr?+3p**5Pq=yzX%PpvO3fY`|z@bk_49FQXt z3|}9;_cLvi`0hC*N#enm+64OF&L;}5gEiq~=Q{?8Lw>ORbltPqt1JNCmrZUj(&7G9 zaOd5)wJ7$y;6|29559)zYx~JJhI%xyCIQ#F#=qWn~ELCkZ5Ag6aMeEyvOg&dZQ}!(F<{l9oBc#z3_ijDOeOeBn`iFIY0?I};V@OeGRLR5Fc<%@Oyw z_rQu|z0DT!cs$Aj>5o>R_R|qdS24X-F#i~%c?#buZ*#m3x-d9Bzl?MxYjdpI`TdgLkeL8^mtlH_b)ZH>a?wUFV!`Ybzh>8&$dhoKk#jV-m-&e(P z+xm(N*Xrtq1wrT_RScDXSNLkL+a_cQ6L{dii4tj zn(I%b?-e_pr7tuNZaiC#4vx1m;xuY}6hp?RteVNMMrzt6v+^;v7ld%wOi@3K$&;VM zR+%F*z%~_-{R@fQ9kx=H?LMQ#Ztza=YK8h1N90$FOB;=^JVYH1eJk$&?SBXvY3|bW z0?hyBv7IF9JFD?xd4CjaD)t=jEtYy=88Vv*&zzqh>z)mzG-e39l_!K{ZC%r=W%3iQ z^u-f9Z^Rk+vEH(RS&9-s4pYL_y#(;Av%{55oum1X0w~59<$fOyDO} zCYkj;9A>2wBdqJA@+r0JwxgRQ_r@gWS5;h5&!jn$fkfViJ9{4qJIg3dzcwSjuClgQ zrjfR2Sw6c>gCuw=e8KAKHl;;*(1Rw^&KHX-F*Z%VLYRKTQPmg=o(oL$CQ@UgpmJ>lh9xja> z%RV7AOIJOBBiL&7jDjnqLc7qdh}!LO0wid!6JUkqmd@stSBY2qt{s7hgYB~|tmeme zseTp}FlD=H$s0jC5nnq3AMb_3Q|tryLW_E~uosB8W(Sb7sdyi4zV1>i(7F{@e?$~P z&~v(Qdn2$y%Be_GhBW0cHKDp%mLMB%xHgbgA!Fbb)uX^d6X+3Qs^YM1fM$q70_}nx zti($^4olCdtMPKX0ICG1y3iUyy`123m-|x2NKe`#S7VQQmUDYl%$si5cBJt`ZpvU| z0ayZ>xrWdy+V$Z~CNO2Tc|Lgj+MVp-9Tlao>1J}@1Vb=w+i`XjaC@11zzfI_Tmieh zfybclcldW#xzk}_oL9&fg-r4_uTDuCl|NXYU_dZe^qr?cSTtphJaz5ArmG`!dj%VI zseBluxlZoT^7q)WX=TXyOrwLuXNTyt0t^RD`3A>iARzPx>s{^hp5(;o; zDgr_fndaM~>{{10qYR?ZWpq%Wj5aiGN=~ezMig{8t4I9ds50l{2 z2NA%Pyo#&-E=aX>ScP^;930>RwMhw8+@jzf5?nLc2VOivfyzr3?)qB5hVreNi&Ms( zgb@f;f&ai4+{N8bMPP(TJ3f{21p(2=Br+0s3sqkrlK3~?UjGVsMoa{PKd->HL*4_h z0^P?L(v^~7d|<#=U?F{dnzVxzid-UHSC26t9oYd^nTOLb-(=}mt}UR-d`ELVadZMQ zLXNlcJt|SCZPQvB93kocH~=|~LE8Wg*14d~!#Ndd1KN{5Mbp_vr)7{Lg4gPEN(3i~ zXCTQ$ZZt`?K9bGmf3|cNXo}>um%Gi;;(iwqZj;7(xVc=g z7oLa10=;@BevI>er^dDO-ajKj7u3Wqf12EucWssj?exL1;LYX2EqVlHOr>qZw8e5^ zvO1oa_tEEho;cKCQPU&j-1;{SGs;I>6DkxQvrfyv3bsj4+00q0grXr%XpfWukO5Gcsz+3Jwm7@M*zOWr!K9|k#PqoK^_3rjgd>P-^t5x31K`% zJjC2z3N z!4sb8G1=zlbA0caQQgx>YW28TPXKh@dAslT68ktSD~eqL=5T!p*iR0;?T^s@a(rim zxC6b}ONuY~Ak8z9F?4(H(9vXZiISN#9#bCX-W)M$#Nj;sJT|(@8Q=hvR(ROm?8ld) zd9d0xq+ETE#uWTXvifoFI)E6R-j8d5Aog;lT>GuC-EJjr&*KA3a{D#$j)AOb-+KJm zEdbR2*vaNraIlQ**cME@)79r;AhO4R!}hnkjD}`1g2jG?l#dz>7XYxJRePmlh%(RN zV18@#lax3Nb}{76QKGHm4Hp1z&&Cv}bx-Re$vq8PJZrtNU_qMqb}o*8mh3KguEbtE z{)oFz8)KA|>6A;%fWrjW0A}9RE|w=Wmax2hLIu&ZWjB3KQ-YtuZE<~ZjJ!>{N)W;G zD#Xcx-?Et%Nj}4&^*Ako9&d0LksqIH)u_fahy;Yc0y~*zfnd&1)qdk$J1TyO%3g#i zEObrlJ_Wn$M!w%^aR+P27S`-^wci&Fl>Bm^^#JE1nIfhVnUvx1^BB-!VnmiJU9C*v zZw@^m#}mj~7${g_W*kC#qOMv#)-)7I=sl7*gcE2lo`#zDD4o!jr{crU;MTXQxp= zib^3vgb2`x*7cI_eN@3?#`srQWUnWdDFk62lFwX`=S-Y;}276ut-)A@Ga8~mY_sEG~ar1V5n=`!k;Al*Sw z+6YMMZ6yxHmvSBbwaq{RN<0TbbP1}sC{sdc11Z@7! z&~rpX)XKg7b?0fQ>#up^(BvV5x5z7Ec#24fC*f2 zB(?PX;@_FvfXQbHt49&`y&Cv|dFv$(a{}ay^%k(W_rFYXO?6S(hjDXR*w;V3EZBu1 z-yOtr{lXz5rXV=(`PN~fZA-jInLqq!)xu<&YM%U&ny%L8*V zO{c0Tq)&R({%%gJ-0ZjFG+t5U)&Wk(wLZR|^DmGK?v|j**48FuH}ZV$iA&=uPJ_tU zMaCNQ!k7^6t|(9MTXM~S?82}1>7LzI8zb4F6z4dU!mcv9l`Cw=|(5>XIxP(EhzrkZr^adnLdL+bWQcy_l27MG)~&O>s-yr({Y$DGtzJw1&O3X5(@~#>P-xO6LOBV*(Y)HE}{^!RJEvs(QJUh_nojX}}b+{?d+!(V%#Nik)9Eig{kB+s{4t7)NPne!;LAKgp0F3r7a^ zc)*kJ^*egLsZ$x5Kv~8!U+2q4f(HUuqepuHSDbjK(CU3mfZ7C+YsDnIG?z5pZoj?x zfz+cVqVo47sd20i{T<;P_ALny^tH+eZ0um0gc;K6d6yas!#dFV=6J`)iOSy;!WFob ze-tRFY6UhJXaaYqfB3U*z zL(`JmWT<~7@mBukY9Q#Z5AA8)moFo0{67}v{-=2Jzmu^cnR6XWy>c8vbCO1byniCV z43E4ZRjq^~zhl6qzJ$c-chIAk8@UO7Biighn=ZQD3*|R85#$fY>4TKDFo^AepuF2ylDlUUNLGl81aX24j3qMfl+<#Gv$v>6% zpKr3T=TDNF(**V4cW|bKVi2b>=7QuW0*YF83inKq^%I0g;S-hx3*|mIw#5k!>>M0C+h#~GfVAISR6uJ?rIdyi%5~W^OVqQIJ`VH) zEmavDzRG_mN=gnw_)?1?JV^itd`=~2?{w4!WUa?n zF(5HU861*>r*VCsc~^Nv3Tg%!2Xe}6Ilb!0;Aoc>dbG|%Hlie?13*>!N5eIgI-Y@5 z$uUWFs1At()X(wVa)upV4P4|vjC}H6!^IE@N=mX0_M#0+0m_S_lHA*jG6rzCYICwW zx4bkGpQ*U63;GPLkv@=mVF6t0QdQc=!N=yV2o&u=cNw@1>qU13cgVGye3$$^t?CDzF zMw5V9;nun*hJQb(4n9I6X7_(DKo zwB>%UuYHSh>+n#d9!SFy<68!MS#4GbQr97@b2Vv>#ZN|G$e}L!@4gBO} zP0wa7(kv}lO#p7Fe>YWE`rl4d0-_&fHfbqHcru{l+`64H36cZA5-Aic^kC4LWq}vO zp%rIB^gt72Vh1{D~5ujJ)_x~*5x8*j$q4fW*dj_04pj9L^ zyl8le!r-cK66a6>vJ2dTV@Xwp_J5q9O%R5F6KMHz(FE_{^bm^*LlrV14JtRVPFjPj z75^N7wV_*~1P-+n)SZv_)Dutfk9DJ08<5vykj#Ju{x9C%Je=yj{rfFLEtDx1G7lkS zjD*Zn<}oT^84{VNkccc~7MaVCS+UT75|&vCp+sdKOPMkc&-vALU3d4r@BQq39Q*g| zBD^X7~Dk`&XSSsk6E;z6z6*GkRtM@Q0S1!_1tDl4$OS~Wx zsZC%D>!JRk!l-*hW^fs!m!d^LC1nOfwEYR=zg?TLE?Wi8w8}L*xDOJ|#bM0^_vD|! zWfp!&v#nN_58pRA!@D(4dyoUN9i9?zcaOWR1pg;fvB1^)M2he&D3YkJusG%*59nPS?79ZX zy9EJ-3{o_MNIsz>aRW4Gyf8U)4zR(x61G3NB))p4h*#?T;b{&sK!z*r`4%Mta`-gY zs-+Y!YIS;eAR)#murS|89tg=Cc%6z9V6_qn-1S$<_xkps|9p5JM-UCNJt&;03@ z!T&>|)czfUmjQSRUyXrB_0P)rhe6qh3ci)ESPP9M;sd*P4xYAt z;yfD(e8W9GvQ9!0qr~+1<_D_{lQN}3xn{G=vaoM2{$=kj;}7KuDn@Qv+_*hBPW~v|m3Z6& zqFk#+9#*>WI8RUqypBSqoB3JJYd-CsL>EU{!n!-xW%Z%9B z!C-GRTUI@l_Fmso3t8Xc_InmidmT8yqlK^l!fDSy0>kvLcOP}1{21(-aL@+RyQOU| zSDXf1LeXPy*d0XS3T&1}X!oy|S0KW!HI7S993Dk+KHo)3>Ox6)6k`dxvijI-A4{(^ zlQkZJr%eOPZ;qt%5fuW#{oQci@9dRLovMbE>#^AD;w{_vZl6QmL?3CSqDAhiv1%q< zCj7DI7#T#bff@ao&R;HUuoo60Da8%`)4w>;tWTu%`tC)^+)3~~H+ErMA*eUr?(wQ+ zXzK&VkBv%A(i$!4{9xauNS}7xc#~wF;u9~lbO0+JBjZp}&z!z$Qre{{;xx!zx4s>6 z_wtLQO9z6)hnXc`j53>S`DE=mwcX1CO^N&NzIXVmJc*jb72N>ZEMoo_n|@7Cp4o)` zxO?f@guZOqiM`1hmb194J`8#-p=BwZ>&CB$$9iX@nT)_uyzbB>-0yYnYHo zdS!X0=TT)rT=z>qJ|a%^C$sW8iHc_1ul(RJfArg7E*yAkR4S9~*bR74SF6X!j|nYN zaSvnQFD?c?RVj8K*ikk$!oN(lod)} zVP3{_NWO^wlWc^2uf#_4w20Q{iSqoN@lIM68g%Nfq;Gt`@c5_Y^>mGMtV@9d2I>yg zrD_k^Ge;QJ&idv|MNMK!EzRCs=5#*Elv$x27tnwB$a4m%*940tXFXD>m}N}@7X9A} z1>+aXp0l@I*|qeOjTM+Q$@27_@Cf{2Fy4`{7r*`t)vf9yexsy}&|h&G71DJfP`&ON z3wqJy=9a0Logl^tcPnJWe=CzNa+7RGR@BF|$)a>spN1lGvvFFibNa-cFqQ4e&0XB6 zNW1>Hc7LbUwonBw<&4Y2jvpXNpt64ch$pW=@o6;i@%vB2`n|G+CnM-)llXrKFLJO> z93;KY+Ys1+d&KOu$_=d^h(}PSPAD}7VXcBE;bUCV`Wcqx8Y;>L)-;< z0)@7^BFou@8|^d+yZY&3RA(fB1z-f z&S}P3Z2Q_ARfYI41&Y;EY7a9e?mM&4;PcCCV3lgZjCWUf;8JAcxZ5HIqYFl`M`o zUb7(UrK4!kMAi|mI7ABqKiKs<^~n%^qy2y|$M4V5L&{JY$I!{qnG)sp`-QiFUP^4@xR zeUSLxcjIOpwVAT}8GhIB=b~oB^(tqlm&Mi}pASVir+fV>>(|~@`+f1(2qwRu>A1M? zcev1N&>*Bn_}d}vi|xMMHZ%M!SJ8YU=9+8iW6JPF6y}b)Um@GAu6w3NgA3R_h024 ziSqmzK4RhD=?j{|!dX>&wom!;q<8O_;dAt=3MU~b+$V0lrwo7ZlE{77xk(YD-(;k& z$7Q)W-V*R}`pN%;J(>p#OJw|i^FzONxaken@d!jGL;LNsM;9%qwKY7c7Oh@GdO&+} z5o&eEpi2L=<0m4OftB@EollS+Clza8ON|7X^6=VA$4{dAW#;F+cAy5W`3j0|yJEKt zN-mu0fnV==?sU1l8GSh%DmP|we6x#)d}_dP8tUz9<6jUtGt_~mEqEp;AdIF&EXN)1 zuBJP2Bl4VYk82@M2=>T_g`;(#=qmo=1^Pyk;^QnnkBUiXhFpw}Yk-4`uIJwhQgyf= zog~vfFOUA5-HQ%%%Q2>r(vsct$=?|be_GsgZlRv=;r9dQHZX5CKjx^A{e)X6)A#&T zy}3Ji_*Kht?w7*LFG@2lz|38_)~hSGFL=0YOV`@$ z&+5k1tF)cGQ=_==!=H>q;$+mt@vE(|9+yqjjb?lzqAkL7m-OGY-kgbBpp<_`n(&;$96ZwgEuc{-UxPfu z`jJN&M_6S)$G*7n^Q(SN#0kfu5NyV!C5AWlyzu0HER)M^yJiv(E`!MIqiIKzG9SV( zije~Y56`ekwe+rg>rc8|0v^P@wIgJ$b>o@>=Bh2!;*e_M^w!IXdz#y=ur1?};^x|? z6duG``r)D-bLJ=vM;BSa&Fv_$cMIc*-^LrH2p6f*$o?t2W}EK8i?t9Zq7eT-h6{TlYf0>xuop16la8@1e(6#Ea>fw zSSCp6bdMkR<9&HLWaaGW8`0!w6+ujp)^}Ju0+>Jpb4B&FD|@s?EzHa}cOKk6y}5!o zr6{}^+$W>`t{$rTUh&3=LF7ta-78FG)|(wDYMlHDR*J&FpE1%u7KDR?Emp%>TeA{E zv%a#2F^xG@v~xc^ec%@&gLGLKY50oAlHD=ixvn+OR(k5i=lAdwf5}vHG)qJJzIJbW zTJAOLEovGxq|;yCfLz*5$g?j^@KI54Qqglrs5N_<_O@^&<8BWl#&$dRPLbA`5eBtB zR%KL2nD#24R+fk2%*elmG3K58yz=(HW(;wL|7R;{4Cw=l|+&e_@OvfFWo6OdV5C zkF`1zCxJUfSjHz&EU}cVp$vF0KQ;=fGM80b9h^=fY!&lVY&aPW>iPB2Zua!X-bqTd zyAY||x$R32L33;|WoZ{86Gnw@d-*9fyRt=8e_lB45ko`Uu|t)Efc7w92+eGm#lp;2 zz_@|JpyWhZArFe@ZtAl9|-xz*}Jj|ZAH+5de`TJz{m1|+M zGR)+hf#D^`&FHf!TU<_lf{3ms7l#{994%pbgl>>zH!beR=rDS;Y`*D{!T&hOjNQ0y z>>-S81|Y?X(;n*PLFSL(Hs9Hr!o9Sw9K9LGGbLmyP}28qJ6NdS2qk6qKsSnoh&kkT zK#a8@rX1*f#WC~`M0PD79n8*Y47PO@WO+om>5F@$BD+vxDn~JODhgNcsUb1`d2zgT zYZ2Kh8gg#B8}jW{e-7ygBxf&L_5xA!<{Cish;CACD+pY(BP9#}lyzsMC-|6~(8Y?T zHELDUeI@krq4LMeOibd6iezYkHp+`tp-Mv^MunvkoQx?3uE`Cys1SQ^%P}@sju)8G zBkMhg#@)lDC<05Xd5>}p%_t_?M~={-lvNVf&59!F%zYHJZBrg(?^ZP7UT&zwil*Z6 z+rJ9lkSXafv$Wh9Ol97HFyV=d9n<9MP=M)?pVv*uF_==n_N$tD5l1V!7eTtIM%O5Z zAJ7+59K`(+-s_|~N>7p|2zOj~ULx!y#RJ(RDsP{C2@*r1!PL54_7uzY^$z~B zi7#?^8g#Wpn`nWBmUN^r9-r~yw(I*+z&!+9)OIV(9KS}&9c9yS?pL2-faKFUM{>qe z3>SHz@ahxpWL?RLo9M%Lcz0BpHs0I^2O`MXgM{MDEI6u3Q$x1&8P%Utg<3VqOX%xs zyo~0;2__ysl3<-^3A&!eI=!9^&mW)9T2JJ};e?M;12I9Tw(V3?<<(Xu^qG0G4t4>s zfQ}`L4LQ(&mr)^9pKQlYP~?FyMlssmDouGvV&>hsVUeoiIW+LRyfTHHc8>Ilxz42Q zEhX2je_rvxH5Iea%j0?bJprUpqVH+ej3U$0Pcd{Iq&#LQD`h@#VZt4^*1gN-JPM)i z^zgUd0^QOq<6Cmcx4R#Aa&EUw=79ZZQY=LWaaO9}4hfW1MqjebJ(zQ#i}dVzJGYL8 zZKXc%POfY#DPiTE)6mLA1svDt=gtKubcoaT;Q4rKFYte!eaf)Q&}_Z1q#QTd$vW- zI8nf3@3ffzG}Q_>gVPjIQ`%j9QJ1P>RJZv|`=HcS`d3!!=RR7v%H1_Ukv(?q_97EW z=6AX!_t9NLD*-cj=n;KBrMT0zF-kNgVU3J%ci#`FTCV)$49s85TfM`%YSc3-B91!z zlKvJ&SMT?wyOC8Y#B)L5Y!OF-qK;YqA)`Y?iXwsesP@#XMhdNO8;&iGHP;4ME(Vt4 z-OZ6JJ{wXRNINI_tH&%lYt6P^On|W8mqFV5@#+s&?Z)mAS<)#{be1%i<}GV{4dfK{j8LR^WKs#7!IZ&Zd+yug%I&9^Mk^iWYq0}bJVRG8 zV4^h4*IMiibDM(YJ}co@ETy+)QSvcsF4b5KOMB4^w5jAb<*z*)h&GIMPc$@@=S|@_ zDjk$FpZ3lSb7%J1NMr|_k=N_;x=`1%^Ic!$%;(;+OQc z2U}b#0?5aeJ#6pniZ(Ah8lJ3jhIw2JYuH9~Z5N4=3qaIUEQLS0H>Nj3=-aUrUgmev&LQ#vo2F#iGSe~-CIbKJ@9Uh#n|#?l@i zwFLNEBJ0_N*%Y-NS!7%{;ll)ZlLplh-ULWyr1Y{=8sjn>&@=0$;wGzXa-NG+ylQn+YCp#JnW~amiJZ~y z5|q73c&7IH>fMr6`&cutP<_(MgpU@b4V+SLjzl&!ym*2-hefh;DKWm$7AXp0m}?%f zWW93l-o`XuFl9UHDLAuoXL0m+7+lY zux>y?MlX{l9h){gw@l6~WfP@k$=7a{U$T>`)`zwKevuD@$)}Q0dx#_8N^az%>iuzo zo@HL9>SK7!n+fJ4Vb>kgS(xVEV|)cDg4|abC)?d@t@3b%N7jrABHkSj#nQ7#NMH58 zm{Zv-^XikrHB$O zaER%}Sp$P>BvPSU@iEmj^i0CX53nabX(T%A7#3~uJ&j?G>bXG5)kak4EG%xqQU`7%YO(|{(?Zb6aclVUE%I}uh``j{v0o5#8? zEpRjX&SfxZnAj%OeK#;!&80Pc-JtIe#}fiCb_;o0&DC6HY#eiZ9EXxRhJB*jyTLQ> zTyR{vB*`a+l2;w?o>pms$7;vj+l()fBifzd4T@`AIuSj@Et+Ui zsz>F1Nwv-Xa5|gfaVgTX+WCWIbGi{cC9d~ZLdq^ZJ!`+hQ1e7nN!Qps(KupJ1I^iQ zUpb5`*@aTiYXP-9uH^3jjn;g7qbu-&_E^JVR?l4f$M5nXWq7i!QvgXOLN)1cAfpb} zsHgRz%D&8g2?{zmYfa4m7dImN4K|~b?Q7Bl{|D(8VAH&u$F)}bLQ{nO^?*FJ0aYhd zrU2cn_>UGV{H9u&!818XvKDo8`G6b&{@|&8%cb*iBSz{~y69W`Nk|Q-CJh+N{13-{1?JO(P?+e77otJU<+#sWpw5R{ZUeX?DaDiezPYK zGZiAX4F}|0P13EYWTK^ibFT8cJPGreaUdkoSN79B5CqvmbHFdwSn43SMz{Kdhr=bD z25DB(jVP{O)K4<6^*bS6(n&``qw@RC(hwn=%B_0(wwT+WK6%geoj>jJ^)p44|Azao zZvkIl`Qn#l%0FDXeizgW5!%d73A~L6NvXs0dwFXFWaO-Tni@plZrC z_Nme?61+)jd5F$20-%n$g)8*oP?hyZ5}X77hMKq?KfN&8Xnb6a*((t-g7%$_D!uwH zqbN$oO>QP&=a-3y+nBP~P4IMf;dQ~Aga`L+8Hn+8!1EQD%pACgx?*HEuVDn~KH>?wgcZx-aw1^cD~#Cz zM~6P=VM+n|p6(A*3)eXXMufD?>wl(QFV*K1GEKS(BH9xDRN$tlcBYAIYCVvm)#p>g zWS?I!6+|4*3TMO`bJFjW7?&hzl+oa=?i`h%M5L|=G;~K) zo+ygxtbwJih928BOgUs($-7yGO__vOSy=wbm<$V`!$GpsK%^6LP_8` zw9FAeE0P<%A9MDc%~Tj3pWCqP{x7uD3ez`0r<(qdI%U5Nvip0AmNvFLuvm;7x`-II zyD!f_1a&&cP(gSpD<8F z&CHYJX2tKKrFLzI^r*mFJb|VrjEbvRf)WkOOe#+rv;xN~!F@wS&A)&WKnrA3{7wNt zksF0oFdcl#=bsN<%}uz938H!c7C19NIQzJ^urME|-UqcH7@_aHe-oGZBIMaILF$;Y z122hR&C(3{T8oEz5A>0a5tOjwf-bl!H(q6p6!t!Hw%0weYv z!^surWI?h>Vls5N+jvXSQr7lBCmFvTj(}J?cx;EyG=>yrq3}YT=)d-?4%0B(=s~bL zUEqy+H9_R4DiFu(eUAxFhJzStX&V-xy9Su!9*BE)gfy1)mk$%K*45J-l&IYV{>}Y` zLRCzV==hftz4_XWMQPlM8{n0`{H)pp3LgRFOG9^5>k#5_1GXQmq*% zi$>FTuW&DtdPxpY)xWcXPG`P;gE-QV-cKHG&YJZV1kycn0Fy{DTG+a%!-$+ z)7?&mJ2ZBW0vbnZPv)Ndany)5f=n*)W0r~113TzE9b*^nFzWd( z`D^vJYfZxS za_z(fP}wkF`6sr~V<*5B)qm@?>w5Sk%IK}*$kE5bBd0A$Tx6%uk|aVe3@o$uzy40&x+oto+wYJ5`ZQOt%(9ZLQDd}E4IJ8f(_)btQP9*3?z_h#zQ6Fto0EA*4 zdyuTfPWz4_R5bcf7Pw{7W0rOLDOCP?fA5fvL<_4r;RjK7bY(Z1bZ-D{>@2n!3?5DV zWv_CS+n{X1bD`9%2=Ddd^F@Tei9j4d>a`Ewlm7dams2AAGD)Ch^*=pK{rzk0%M>xY z7}MvDsck$0n`=LXz#o>sG*GC|iTE=S5S9WaJ8l7hHK6w;la7kVVs}NqeX1<@V^)Zw zI#WcHjxdNSGr2WU&6R=k$_jyQgISOUAqrH>+VNz#1|vr082~B{z?Js%5|6=clNBh6 zqLzb+B6$9dd9tX@A(jiw`AV8s#S#Q5_(HJP7D6;&ahefxaQq$c#Wpd!a17%MGI)%W zm;F(<7qAFr?+9dD0*wis&L{ToHg#oN%DL3Sk$TS=yThDQzI6 zy4ilY)~8*XLbY7^x|xDd9Xv;yuNKR~cO-C$YNc%E+Lzwrb=MzMKCJxU^5n`JLmkDcy{)>v z-fSM740?{F^*g&@%j~`D43^hL9%ze_b$}pnG~;!L03lSs%(Uj z0`R&gWgD@;lt(7OYQS+kM-;+iI|s03L@huO=*bo5NdU)(f)@aax!EPN9OZR^M>{RI z5so9xQc;Ttqh$_vnAtP#jNLmX7jQm&1;5$c6~EcM&0P>mjn;y)qUEJ5IjgLhoS|Za zp~dUXuOPmC1w=8uVZ^Xnn;E$KiDDVDUVc?!?)whV#Hxiv)FK=_W`E7}C=LGvD%lH% zem=P)a6bC{S7dy`Q3ZUbxD9b+!RFqR$q$qNjJJa7*$3t=1lr5C2j_;@^q`8D-nK({ zX{!|R3jjSr>Kzyy6qjeFDWzMkxVN9!{Cb|}fsOMrjmZVRf%^8y&s_j)E>$5PzclI)%`NHDEN7MOtv-6d^CZ0VV255OHMe8Xii?apYbh zrG318TN-TUzeuSclYa7@exTjW*9vr^c5u?=m_&Yum$z9hN>2OuP&Vg5y8VMO_1~zS z3M$?L914B>5p@`gU!j1-YqP|~Jj6V{S)NL&Uvj?5Tf-9``1?0ntO#Nqnh`wp)q$P` z5G-6I^UkN=xP{br5X*Jx@zJW5)QzLMY&zp_Ay90DclG5JAY{5O*Ld3B?_^Z!1fk4E zm*q7QB`xvVcue2yO+B9Bz`a`w?~!$tT+FH|JZh_|zM>b=EXbMQA9Z+ufXMx?EX!1J zoclAqd`*HRqA?n@sX=l6NzZ}+;7!b)6VQ_A=f3!Ga6&`wPKq#jiLw?;`8^vjZB50? zeg%02BGE`*BqZ9hE__=iW7{O_Mh#NJtWcZkK?2Y%)K) zK+$Tr=Z!m8KDP_g`{KKlnLJqe&xY)Xz5DAk!&4{k1Co8VpT(g`!Ti=QV|@-qi7%dp zzHnS=8Zf=xi?Ds1`tI7_iSK!lFVkd!)or|`c;*{Z+P1&R!G?e?C(?oCl zFhd}Lp5)*9ia23R--mRF1|SfNCyqR*ILu(4+)KQ?!5Aoe@&aQ^`0J!}yh*WpZjFw|7?x=r{hw5kNa9*^%d5`I;wuxudEAHrVF!_b~Q7cbw097AsZoeTLB{F;p=o>CvJ z81}xF?7!g;^QA*A>g0aDtma>HsZ>G?eAgwfQe6-Uoa+N*4jj*g>{|GL@L~U3CM^Vw z_K7`^asAudKfR0Fe_Mc`;=%#gUx;`Yx%Q!I`yxC1A5gJe-2Yq(FB{to^hQq&M3eMK z{PgcByrXDSQqJE6FeU@ZOgR)8(|w% zdQSn*C`)GRd!;DsA@+TR`V1aTxqs{bPcLP{Y6X#WZ#gR*Xf_Pbj~tVTBf6lWQMm$} zzr6?&Cu~5;{ub%aSIau>*k&X&Kn{G4Ai!~ldA4B>ZBx{<`2Mo0cf(X+t}VXc<#q#T zEzS3nvhQh(fPyCuj@~oh`vJ)1{ngh_!H$9ss*!*qTKPF_@qEyMKUMZtjZQMsALs8w z`2L~!hc4yM7R3=rHpW4cMF&D1ZeT?v0ryq|_IJ-g2sKL~#KGlmfM?^RxvDt^8EbrC zd;L9L&+qqn0hH{|)*YA^p=h+n^t<~A4Qi?@Q%EyOARa&gMo1TuSy*gNU+b1H5cgZT z<_o7C+fh7r39xG#;LNfG;K=(f#N*A}Vnh0z6~eeU1ck=tEnHCnd$;D#ueG)ig7^nhC9=DdZ<4>gQnJ)~D02~Vu-t@)$rBN<%9a?h z7{4A%z&gf$==qb^M@NH-ki)r>1*d>iU=zg+2q!uqx0m1b7J+QQ3jGWq8{kZT16z1; zqbDR)_cxw}+qjE>1<)2&s6r3{NRabj4V*)2Y=R?ktf>27VXa^5Iz}CiwDCFN_KTRR zgZb7<&pu(pf9dXDgbg4p74;n0DIii%xTs|Mi_9Xd7P1vG{U>iCT}rNYri{)*mT&Vl z^+5(VS)iG1?kXH+PIMbKE7T|7F?-wN*D<9uybE#0Iop|{pPaQYjO%?qs-19Lsf2Lm ztD&m(S6#WCiCeI}f6YCg@Cu4%m*kWf=uTal2s7k7V-D)jUzO}ip*{{8>T-x|YiG+Y z^Vaiyz{cMwPylRne1Syj{!S%9`B74=4oijN6kzcgIrv57C%l`7$JyJ|2V@NEwrhd( zvei|542{s;$bk_?kz);cXCQ180%0Thig(W}^dPvkD;7>JKA-(l4j%9BEsq@4^DC8( z7!mgg^;>};u-!;_*s;={F7ybf%WMfqxKRb@Gg0m!wO1PMa z8m&oLwPu{uqsXBGju?T*7ftv>wbg!9NObhzm0 zg2Ge;#tixitn8#a*R+4`kmQgJ(bser`vJ#`G2o*&#&}<2Bzwb<+~jL$F(Rd#-qXV2 z=9MsY1Y2Hy`%7?|PIpVnfGOGP5eHIbMs2*LSaRhRS^~~aq)|g(c*ygcx8D*SIa{|r zMs^nIgK#L*74%w_W{{G64|#_MgF(lFd^sC%5%#{IVs(@l!F8gaGeT7%4ic!k!(_XL zk65M7i8JRo$p&}r+x0E$FlvzG|2=I5phXFp4P}WIZcs6rX+7Nu(@#RwaWho#5aWT}b6iU!a~k^2 z@LW~VxPRax5H{SK(GJlbV(U0+(F-M#71~qiep-QNv@kzf8IZM7k zwMvX$`$b$k9%RvHK+0?M$Y zN)kea1|9Ws;!bdZBV#cQd5L)U=(w)7%X(MIF*HI3S`8ctI=s32mgE9+#j9oJwJW#6@0*d&7QKD#es1g>b@)GkJQYXndnEDlZ{fv8 z>OWGQeOG;JYxOITjqb+}AT{|+2n)R#-kdG>KlVUoHMxz2-4`0dtcb}3LXf_*j;oo- z@OZbFX&ZU45>IGx)wIHaNtM5Ij=YKCb1T738%y0NsCL|hrCLm3g=))^cZ*VYo~5JK zPPA=-!x?$WO-e(iO*`%hg;PC!bX74CC5ky59X>g1lr1faUSosu$D}V=ebhc7ypl99 zAb!l7+EuvJh&fR80tN8iYllvJbsa3dBCXHKS^G#ty=T=Pv4{Q5hK)}`z(?9=u z_E^aoOR*qN+@k_Fhd#xyMx8VeCifqDB-wg>1R>C0<>~(Z;0(sX4QQaFtued@xg74C zswE}9e0Q9g4`Q|c(4v9%1lR%gM+4IDHMm5vV{yYK%@+l4C>()nyjs$zby@{5@U~GO zDkuX|&@~>bF1rE2r(X3tC;O?8IAe^%1`9RhnyK}k>0;63rdFgcfGcPpMDR%S@gWHK z?wXGkv+nNd?o{*+g+fwt$WPiOJP2e8hT#9tJ*qHzH25H%bEeOn$*(_bFitbK~4y4|>Fm1#sN(}L2G5^rcCQlsDznzx>1cS%#gY2e{JVH5K zQm%VR_;bANN4?2oQ+7%+pBHM&Q*SHpOxIrMLK^uQ1L@t>&T`AYkLwY{!_x;mQIlw_|D3sFC=I!v!8Ek*oi?uRs zw#LOtxZ;sG@ly6baUzZjVYKQC$tGYLLuc({p-wp$=Rh10z{t1Sdb?h7B2G!bI`<5< z`XR*A;`guSfJsDsnvFzb1Z8GnPS{Jc**9hZSjTju>nYKBm%HGJ#h9Fczz+NgE>6wS zzJ|$6010~5{HjLvK#9ta+BNgqV4_1ANC!9x{CCLr6xu#2r48FSaqw^M3lf7eXd2BA z{hp5$zui5UzmDQ31te(=2L9)o8Fu5XAXO%H;+0|j-;m|~9t!vW?xBFhpTB!3{96gx z5{yaA+<*Cg*n);p3r`06Ul4(*ePiL|{v{tEu>tdITfP1y$5R{v0E>AFiCtrkX=s!f zWJ4*eRF&6pwChQ#R1ckRGJ|IP()=^ajll>{g^x55#-h<982cSvfz54fgnv)P>L~`AQJ^nlt z)j94)#v{53atP)-_E`asQT5@+S~4(9QR0}VK;D5b{sdQ7aThv38pyMCwh^oG8Fb5= zDpR8r*q-8ipeumak{Xkna=4=~FWky!B1?~;c(TiPbm$$da)Tvk-9FL)e8ClYtT278 zpV~($;b}9o;a11EZ|4H?F9&9YBBFe1LaYPOXRzZw&6HahAWLCD`eXX9A1F{Pv&tDa; z05J=Kp$`XMsYWQk8$D*QvqIh?Xi>srPFQmQxd1%bdt>Xe=42xu5xn54mK?qMmgceoLeAemRUuBft;j z9GCQ0CWt|dkvT0uJaICp%qKM2bbtd@CCSwvYWrFzq z3vznJaIu|%3SQ+WNYNx7MbHfqaC`c0#vKIl@}ux=q;@R`8Uk@lPfX!oa>XIdxQmLN z(Mix569z?!q+8CrED`dP_RauB0cr^HK}7tVCxVm_k22hIRF*J_K%G!QM;zy_P|>~CHlAzl;$5C>gr68PlGGZtzNyjsZ z8T6@q&mS_$8FU)HQ(fs)WfT`B;)eH;6r-h9KMj9VHlYwxBhZ0&=UQBoa*>+%DE#Al zSKG&2ajFEc79+}!YRGPKMsJ&5r7lFlm(%J;3lNb^a3Q9A*hIjTpK#m--cbm2`KBUe zB}|adK|JBWF0MH%?D5AJB4qkVcRjnb-z?CRE(Zhx5V}a=%$)5gvOs}1{-RJPEr~MH zN{BQf3QqHXK18_PF)&i}-hJoR{<|$VVi?(wz@n*8u&e` zZKV;257JM3MT>!xgbQ+4hz{An;o$~bkSP2HFQ-4ZHo*yG8d=pxJ={OklSUS6`GgVi zVd2;E`ncXG5RfBJy|5z1n3@-9h8|Qu^8nUqWEw6UVIU;ox{thHXmychi*Sd2Ll+8z z;Qu^&dp(7SX&Bj^kw4QODH6XMDsPLi1V`-W5bGobZ3 zHH=uUY;?-uKL!sp5`qy|c!-=e{`NRr7*m=2)l4EY$bgMnJrU{RvIKKKKIRz0zfwk8 z!YigkMG#S|0xL=6)~09l0u;b?=ryiFH{yCk;dkofCMPs_hfx&; zeQbojOB4XRPD4II#gT@88;BQEKu$3*rlBr={2gS%SQ)vvm681%etgYWR03$Ce@)`^ z!L6@g`~DXLf=ll{JV|5jS4sp$ux7{C@Yw~Ohs_Lj2n4cpTN?iMxriS~EcbL^p{|0X zg(m}48`4)QL<8aF_`)EaIM_JErFZc6h8+oKL;kFoi$wpIjRLynKAxkeBB?_BiC=20 z-<{9mhIDr$9POrL z+bAsY<%55}E|@C_u!`>{v-JN3>PXD)gvyZl_ zG0E!`A>qn|FIf5(ewHi9>i0{gxomhujer)R`2_PNsIUhKz{OpZ_W^^po*#0ZcAYfp zS!VEztvD=t>Fl*WS&G6#g+DP=1JJlNqeBF%qSdd0p*nVrTM7AhT4<=9vq^f?$r{lS z&Cr0Fs%*|sW5AG8U`l-EveUcJn8&MbEsdBRQNfa0UgKBlPmSE@sD+jkSh5zvIAh05 z*^Jno%8WANV-1?GsK1&xNRTR6x+*dnGUSObB|#_>kRPq|`%*26w~`G}Id$JfcgL1l zNKoF$j6gr0R(9B{XiUYAZrH)%aB{}Om!GSb;8WTT`r3RT^xbZF6z3h*Wm_aGgl#Gr z^1Lf6G_j~Q#GB~xc^+sI@#m{nOn@dqUoPxdKhgpMVyT(aS(;IAXfu7yqbf=TWpnKx z(jB`Y?EiWjE_sZNLTTv*CwwYlU8nXS zoxdL3MFv{px(WSXThbO=SvdK~;!kzrKBg8I}8RB=n-;uG%;mF?sVb8)yN>s&l{ zU1;(y!8^m<@7r|EN`%E{_f(b(*>{6|aktjzr(-Rq#6PkQS{Dtfv1k6|%S=WMg-u~L zHU!gvC86Za%{F0)qkl@qU#TXszj6=kt1?(75=$`vdPA|;X;uxme zp@p>>GNoCLtmi(H7eQyA%=bfaOhq=hZqD_}1AZ>#Khp9dUdH*v$11`r_>65zr(qGC zO7%h!q?X3{J#Lbip9@E54c9PKYobY=SIxKYQ=t_YBmEmK=GN}B@S^fBv}K%?wba|( zFcaQY9BW3^yVRk+kL``q1$DH$$f+!`P?fngY^UCA;C%Q^h-th$)LO!B{2YD3S)tG{ zlIt-WV?s$`cRwS!?zmP?&loUjCXN-62fjK`Hny(m)bcM6?_ z8v1^gBe5&XS72OtEv8nWsfF-yVeD%|@iMz32OCn$c8*&#Ok)+sK|P~HT_Q;vX_d-QCnkHelHllNI?kJ;-c50Ha#99feVR=~i|4Lm9w&44dfQasP|ID&B@{>|f zv|vo`(4b>#^WK$Sn%(cqqt504OP9Cu-x|z6(U;x$F`!O-U{=d3AVN`z|AK}kQAr3r_2wsQ#Zg8P-^0TNAX!D+2Y16EQ(Vs9v<;zt>pDsCexSu3) z4u7JgTKEj|6ywa1XYZwp(GS73i4z+nNP-H;fz-|DS-z@UfXq`7d3KcAL_ST!T0WiE?=;^M}J_&#a2H z>|ZkUwYQC}@XG|Ig4uIgN?J;~{%S{z{)0l{L)j9G7m~R%t5rq-ClLp|+w|@HpbB*l zF{nQwP>vgdn$0BuixgR3@^~xWjAa!-n(ct1*m$}c6i4pRz)n>@Sf49izI;cu`VBRL z&(jA-!s=;t*`T;Dalv>F3qWkV)8UW)h;+Kyb>9P5`6OrwlaWdaqTSv>WGT`?Hej!} z3b=^$;1IQHLMf;rJ*v_7_aGCEBGSlq1}LdJmf*tn4CMHQ!g0pNUbfGg<#+H05o87 z_>IYH|1}{{I_7BPy_R|M-iV`78fmrz#ipl)p62d(*Qb5B0?f#M;752@RhCctiZx)J zozwrlpxH5W=owQbO65^JX(V(Yvg?yxLNIqXK|kQ!vIFW8Uk6XF2mugw$6I=G+_Tz> zcBd0WlFB*t$0@wGC0}@n-z7b!TfUgSU@t|^Almd(;Dh;1r=P(8SON)_7;6JCLZ*t? zU5C410M5v1jv+;ic(nL4!^xk)WP%HC^73VhWzF9D z8BD99C2K~P{U`u)8A{On&x2^BX{`(cpcSLuW~1%`U1deb-nhcZ5k&%zU-aay!=I}? z2z3LcA3(AD2GS=c-Lc~C3+RCAgNVSw*nc5?Od~98Q>K}rfz~F)bVqlOG|)9xghmk( zNTUz}c&Qzqvcu zf4{>KIiVKS<|alN#qiYCg|5Ke-oQR6?#byQ)ZoD33`?S>2}C=0=0%AXcI4$G6u8$C z50*a+ZaDoJJR;LOuv-PtLz@WUoO&iQDmn%Ipan|aJon~R#=H()gYi5ZFYamu7CTQ9 zrY(B<>(lI&u*}jcFE^Rnf$C!1N4EwVhi|Qz*L|H-4|G?BEO zT*n`@Jb^Z=Yhv{1Lp!M+QTbVoXFm3OMTK{GHxcv_f;i>ZxcFkaH8ks}1;xPCiAqWg z)l+)_7=^KR!Ab6C!C5f-?7coq;;zQyXr^acIGO+O68l)I^->h4)I6Y|(=9iBfS9o! zpf7|-l3JG21cMLNS1X@rDrj#6DaZCqFekdrr1I)p|G*Zx7^H-;20bn|M}3wK2eI*Z zICBbCs9;I9)H-_}>YENMBFKV?tqy3#8q;A*7RjrkY2>SAAOmIN8!dE_0Y$(m*(-lH zR=4|0h=QMS!8U~bmZxh$hI_=Wt>fz~%at&FlJQ_XK>*}e9;rzRb993zBWfF|x^kz; z9xAY<^oIVCi#RLdx?o1o=u%0LVzO6F+JvpXK5S35DW}~%#!i?`y3?UAfO!pD2 zT``b*$Qy#aOUH3=Rb*!1j-e3nFv>DnbP#Cix5khmXb6HplDa^UXB%4QPp1#f51u4d z_`&2(zcE)-;O9_X)^)hcQR_337e*T%`+yPc z@kbm+X@_ZS4ohdgkU2ye2?~iOx~uAkTPj*qP6z-K%@eZq{;IXa%SQt#T@-pk{4suH z_z94SlzuQ3BgGs{`t+O|>0)6)0foF%gM|`p+03-M> zG|RuG<<=rR*<7gNok1z!=}jaY_`qd2xDJ}Zm|DYx`4j33GIMBL<(JJdbfc;Dp$z5ZoM@S9@n3Pi4RU`%IZ- z%1nmLGL<1iWX_l=DwU;BC>au!dCZh~$W(?1X&IxGF^NP)EhV!^l$ku_T;GMB_C9;> z^V{e3I{%&h?7yC8S?gZ+eXs9*eXr~Dd4Cp%&%%D;9#$y3#5GDhD7C6JlhD6_jNuHO zrt2ZH$my=Hz^_>{S72NVZ#L<^qpt0skPb?tzP~_?n>HI-z+WQuiF&pFm}7p1!L=(J;g(XCE{g+Bv1Je_K=58 zW{%yGdMpSfoS3lKl)U+DqGQlq7O5CLGmQ(6jQK}|s|k!RL%5Z_Q#i2;-GS;)V8UCI zxHcbr=JOl(bSN07ep(MV6z&?qa=h;gePb`qoQL&$8g|6!;=v5jgU$^BB+vHf?@6H} zWykpMK8Rx=%(z-|W+M-XIsVeN>dNm&t;5K#Ge>f#UWQx^;#0x^c~Iw&v{scoo;l7< zJG#5L+-iJ=Jvj#-On`ZDKm4Ip4in1-KOt;sjbE14@LrL$C=ph}A?m4$UVzBd?I$<9 zaPCKQ$~Q_;Pot-9Y7An>EbpA%KRd?uRdv*uf^scR?rRk5egXUGvkEOM4F&p#Fa&?? zT6{*m(P=mGz=R?Kpc_AbhG6m^?EmDQCOuNn|9qHKA`e?V94@-1M4F$m*6d{x>=#uX zW@^R_`w;U&;c;q;h;+-TOJ5k6?GX3jaIC*uU%Q}?zWkH!chi)zw3Z!yDvPqOY9(j5 zDBs0ecMl`EiBF)SWHjVv{J`Bjr8vKyTJy^$_&FE|N5@BE)M;L(4hRKnN{Z_*j=JIw zyugxwOg(qYGUd%}E0@tH7P^?(dZJw`KU=uyF5|MAD@bvLX5-is%?a!-tneS{tddSn zDO%|tdgoqjXw|$=q&@>AB8$OHOt>4QKtL&A(7X{of;{;dD1^rF|>BZ6NG(E7i@m1(|gQ=zl{0o97Yz)hg+N+%Q(LhhrNv}RtA*jKyyJ1hCdL2+5QTtyb{XA?~F z_IWsQyxkJhsOKlUnPhk1QCtnS4it{(O1iLp5?0`NP*nS?`RF|&r z0@uC7ms-gH-1C9_QcQ5q5gopX`u>F7q5<{EC6a_%R@M*MaJYna&0gn(A}W_$Jq#Vs zn3}ePkWgd83u&GC;3z)X+l>-O!$Lrff88=EOu}{K#oA+;`#YBlRYU!#Gm+M8m!8(p z2dXI2sI~CjOgi%j^)(-0%qX_35gPq-*{P`!+erBJuG<0I(1OMUG7tLa=XZL99q7dF zyNYFqRW1^QtVApF zL7;-I+d&i*yo11W>H?;*>5D+MqKIsTbSKtZKB>&ch=PUCx#Er1ul*}81;u8ld`fRo zeB`;O96dcFq7)8w4LkBequ>kDrWE(DQB@CBL7cyrzhz>4HUUE|Z&Alcv+d8nRS zW)fghk+8&u zkm-xA{Z0as37i~v>zS2;2twX;{KCB0*$ zaZtj(_9USxAL4$@(*|QDZJ&o6UCG-sH8Uf+3K*<;U?3@ubf{=S7Cmd^A&euPKC>DD zpFu;}7X~JlF;95NDNU7PhU6RYssA8ETEteQMBJo1!5qdVe(Mz2EH$Hg}a zR#IHErKUSWvwe6XR{2lrGm?ZFOVrcb>hU_H)LF}t2r`k~>6a9BAd)sF)Xgu_P#e7e z`TjNI+d_(@#)L-_~$cUxm5?-1w$L?Z0UF{40mP-bt-~Mn#?GNgI_YtR86!Or{ z?LV}hK>C~Q;0x--g5Nja0c32)G#)y>fbV??qd}0x?6$QP-*Q6a{VBE`|3gFq z2)m(%h&H?r&5l2klR;|&Q*_SX-LQYxb4#Zvbrjz0>IkDA)Q25NIsH$F1V2y=;IJ5l zyu0e5|2CHzrpb^?Z8qKcpXEE0yjYv(6sZT@>|RR(Of>;IhA+WGB-)WC*ZK8DWg7Bs zAeJg9{e@7Jo@1}Q&ylUU`RpDfo?nUb7ktA2d)__gpvaZcss&1EE7X1>pzyp92 zV|gsW7K$@HhhwP{^g^Ih2%iYNt6FgbQ}u$;L-4k`33USPd>uCO1f`rIi9c3xfUu{e_mJO)QTqrOfL z6mj1NI)K$Bphd=Ea*);g_|t2ZYl{zk6Oncvw!l2+mm&8dii@?^q3FTroj{tnYm1Y% z8j4pY&5mQO{ik42TiwDbAm2<2yR#TL=-#>z6G5Zp;T&v;O$QVJ&zc7-GQug?@UDQ& zcnti%$7Y+j%w=TSyc5kC2~J{4 z!G~2dksVR-Kodd)A=U3^grPGqZBNjm?wBhWge+dFAB{=B+(<7tn=|^mA`z4N$CsV69AXL zFa}$A384AzzmE4g1Tc?Rdot56J?-vrF2p_eSKO+nLfb&^OLK|fS}l&|B)5r zPZ@a>Wm_RN-2-yMt1~|@Uhu>E3JJbqg)Z)D={bs-A3iP0NWZQj?9rpT?gbG znuCrEx9&6y8pyfyV7Ay8?Q|Z432~OF8KwW57ckU|rDK9tTwjoY3zeFv;mrl{q4trdo6oJXEyMa~T zB)ufH^>4AbsADh%GnOJ-s$X=Y3HR8?%!)u!Y|I%0TJ=*1ZhIfX-9sJ;1wDtJ3qN!{ z^|WfMQ=mVk1fN{iu#-lsEs^65b&eemft0U} z<>ec>9u?4C2fUC+>y7cY=h~dnA7q97+?0yu*(1%;hE8KqU&+p%#L(-NW9#3(J$b1{ zGpD|g9%p|}ks`tgunGL=4*dh0a5Mv}8id<33|(kI8H4E&f*?a9-QS9dNoKo2OtS;m z`d`bPxSS@{Y^>y7&qxSG3FaJw#>KbKL-6Z6`^Q5SyZ2~3R`meZ!tXO-zkB#}Z7Exm zypsco3xVQJOAq^7=W$bKjl)h6y)_7rBGGcUPPv))0Q!sX(lD&zU!be0Gs8mq`wroa zZWGgFgQukl);c|p&zesRTQmxOlw7$NlYKSiQ(%wq3Bfuyugs1t%E*?r`ILTu#aR&i zVsAR<|2LLG(!OGB_Rb&2{YMIEqx?5O__7|tX+=Q^&B#i|RgrLDHK;bz?xLl`Jsbik z?>lzWpB zQSIzt9X%qn2eO~Av?gM&ZWn*@>USYx<|zv_QAs6W{YryH`?Hr#@{w9KP8ttlY&r6G z|H_NLj%})6>cd^03>Piw@)0qI${UyP?WJH=0`#v3O_KL~id6Tb1|RJJNVSulK#abc zzM*r6ajVXRiBC1;xfPZID<&I7P~X~dPi4l=1jh;(BBhb%nD2@AsyWmP8mxDZg;qD3 zy)m~qmB-hl_hyERUV&a{_vSUffUZWu8YoE(ujx5iWsGJG%Q}U?`o#QTajEr5z3L2YMw;MAeU_o})m&7jwY9 z!Z=RvE(CDW>y#A%v-SE-&yW2hi&xRzoNS`=$09;nt-0;=fOXPi(|RezVA^5qCijFnILCy$?)$}i-v=3 zXkRDuUsT(8N7P%GbX@vlY54Ez?VG%Q%XWhntpN3uKu$5}?~5A$JZqEp@5qA5gY zl8p&6BQ7_l^@l0RO9To*LUi|uuEWU4j%;pY;J!!#>3=K0#gJb|RLBrf8?f7BYkYZ| zd}{H-6!=9<++I%Pn2;lO5yaW^fMNC>ANbHqgT)QM{pi`TgF^&m>=cfBxq^A4Er^`q z9F(;Q@SHRkAxw$@TlVH1fxjfS>vyDGt`kCukqBfl%IHASU9ZW=PDY>y6MS1{FW~5( zgFEe&=W9xsCeNT7kY(FMDxydbsU>DreU$6Q%;8=D;uMXJr<{$BMYN1d)k8ZEuFW(R zDRo{MLBDVqGhR^dg8cU*;L^#3=bbBfa=*-#{CGOkNO{d{6q=|H4_tVw0VDzTgD~o| zg~h6s)eeFdudUi4gN-kY-;%(fatwC#2YS!N8@PrH8Q|q=l*nNK%M$D*%#RUTlHg08 zNVzwMA3-3buFp;?7H))wAAgqwIL}PJm>`gtp)Gm)5pus-8>X( z&=yN4N+%unID4;Hy&7)^4cev zhd#_2v5Zes@+w;@Y7Un>+d^eTzQY-&y8SS6RhEg$&O#PB);CJvXv-}7Hq)IDu?%>L zYr0{c4wKJK`M4CQ8o~YtS=0OhwJ09&MevU>Q~;|$6C*R9-f61+%cRdY$1~;Vv+|xrpp_xza>s&hOWOr2&)Ax-}rwOkheak{PmVk2qq+=K)X+ zM&4l}oq=*@_Tnpn1|*TnQi&LjdnYBI`d^^cx2VU6?tO}g5VrPh(h!Sb zmi@j8V_Y$yV^sFR!=nsx)6JNJI(6#F(NnC4@q}dxkD=w&CtCm;a|8KylwJPRfk@5z zoq^l~!aU!NU(&GYda5uI#9&x3r!)`+jtfT@e_<+pBXD*h%OGsP_+gLou0j0Sip|qz z#v)DZ5sND{0okywU8{Tn=^PTkYka~a+7mpqpZ+p<_SX?V8tJp5PaDe|f~}+3n7z_> zx_YL52V2d4IzMD>wF4^9ML` zuDsD?zA62^f?Q2yZ_Ii0;3-{xhcR#-Io(1pTPILk^!_R0WZbH+_a|Bup%CkN<>7Eb zU4rC!Tqj^yc>i)~sg!=Tkf1u#t*BWG4dZ=I*5d8v0%hvtxUOf)Nq{G1Z+qiUUt+r&lH+h?tKkAcYfhr> z+L5+dhY$OW<>RFj_hehjL~!uqFJ3R|C@K^qkqu;sN^8X-XnZ~VDEkgs;WHg)2zww-wV zj()m;eNRiN7^|oxu|cH>hf+W-A!Uo1pIm@M6eB~0uX~?r$`3bs3`vqbPa2f0ZZ=GJ zs0)vL(1e+~dFp@^XW%J0Obq#1BbnPgdv475TU!|5Y|#eX=U%=+ngDKyLD5}g=m z3$8*Le_jPRD38&W?wK!sO+nN=kL?Z)f5?tVp4n^eWIN+kFvxVePQwo$_52#Pd|LEEZNS!AbyT$p)fADCSJmUvC(t!mgHdb zCyP?%ou}hM)}^e&C@@X71&0JO@#5l=I5j*}Y$8DxI+SSh2@lCQ+8eL52+6E>g(DCW zP+&4j8_cDSBMUN#1ZC5>{Pp>w8?zWQeTBRHlCL1#7%ym%nr{S z;Nbb&R`)E&LW8;sdRWG@tkjzpH2*&>XoK1jC}2^CI|Dt4Co_K6jZkXkLMUqRDWbnE zb8tlexvK`=da$6C|92L&|1rfB3dZ?Y`>*KAeRhxk1I*+~kD6!15BVN>Ii7z_<_X3W zZr6XRAx*Yw$i27fp|8RVkFg3RfT+yDW~>e1hs9?wnYC{vwIOf%&y(YoD46AfPX+&eB9QwhidyJGHL_j} zX~XZJ>-oK}#ua!5FCoU=_GKJD(jc_~4`QG4^`!(T7AGpSZgK5{sP{tak#y8BVw1Zh zhT>s-JYc^1O;W%VyPhJ638;le8DVK)Y$6I%y0-j%ACMP{W|LerSVjqQBvzy^f%0bx z)@>BH(mL1>I0kh01Q7k404!5SqxY6)srT(gF;Q-m{!)OdyYRj^m5M|D zC<2g8cQQUNR`r6rBM-XCtFl3)_jLO`N2tA{uoG)^Dt%B=iz2?zA-N4~_E@*6QBNbu zvcrdxeJE+0<{ivxQL#$gNYrQt-d>po;-P5yVX~i~hq==K0m6R#tNqk{xXl2NLNuUa zq{Dq_>-jNB$(#5cz&*1w+ThRNz4NSENrvstMgq5xkRw@<&(Jq46y$073bxK@1`bTX zTJNP^?Ewp3Zob#>l+_lX{8%jr zit?#~zS(u?&d#I#!w&$jEkcWL^y1US_p7XJ)lgihq_fq`0u`$y3#eVqrlxAEz2jNF z-v?A+NAA6$!~?cIAozldxae zC|1Cf0;~21w5pc46+lTc9@=n|&1DAV^<;n+>+`ru;Kb2)T&usJEFK$>6Ld(>P-$mk zLYv&>W!%|aoR4O2<=ifO(^-YHp)`uJBi=KcPHe*u!6?D9k|IB+vTwfo9f3i4)QV{!K@o)^@9xGp~y(u zg8B$q>ZVP1FNESd*s{Zc9XmOadK*1*0J&BM+xD;58+;uW)bsQ2saJGp&#>aH*HC-> z_Rebd_-!v(@v1PYqsJtYA3IqPOXOR=uUNsqUVbdW)SZ+|%*&v07>YTD8^RbPYI!oo zFn=JZhOv2f6)B=;bIo7?<$@babm4uveY8js~iHfUJTg6q}5$;HQ+t#Rd}RE;}1yMToP!o0%DzP$u59(A^wDdR3+#tg8nbNuuPgFHYwD zP%j>J)iJubl=zMAN35#T?`MgPW1M~%SXpVc@P@CeuctpfS_Uk2QDpO_x^<6rB&ck+ ze0_5)ZDO>vPGgDeB~0Y*WDWgntJ3r25;6nD);Nb zLrge6qU$ylmp*@Fm%Efso_z~Ps@@yw>9BnCyK?;D5M~t3j^*R}*rK*R*%dF|z_}o1 zk7=7U@b>?_lB44kA@UM zKCz=BVh5kX9%`|w7m{^mC=0d7Mj>sjE2G?Zewo2(=z+(RZc*LKltl{NTIQ?5Nq&Xi zA{CeG=}oH`dk^rw)Hm5pDwz<&=6-{EYLvNg>_nl zG0q236<8b|hH4FSuwQ)b7I~ z&R_6-LMuVIUL`vbeZl5~2@2YV-XXZ-8cPWN%_m&TC^h^FdlP(TRG;XGjz=F@RjNGm+jM{a|cZcbdMyv zZ2E(svyMKXxbe8v_CfIjgdrP>8wFo*aGw#Le_8$eN``a)_U4};Hr=^>TXv(bcMq#` z(e!M?=4;=At(TX&@|85$_t7^|)rU{DA?!G}eZNzq-^S5)vF$6}tO0)IQZN`q-GlM2 z6pzi84HV98)s~C(BU9wBtv?64GP0VEeUQ@I{<5XuWD28}VL)pNx`u6nV90L1$Y}D@ zypewUDBRIo35#VHDYk$;7=LpkI!tV}2qt9ft?N(0jr9FJr`z^#>?Yq}F}79UAOt^b zypQA88%4Q+4?AAMPnaL_O7m+HfLPvo=WyVfqgxrYGtm-$R@@eLXDBFg<6}D5SY@Sw zl?$`{@0s!`La1dxtq9qctWC+0n=-;7OB-Gm`5B@VXMs~^6g4yy@BFT8J#&m>RJ zaQuo?zosMiiAOotkz?+b>p8{IJ!6tIO7;Ao{7I`>NS_U+PuDZ2M_q@$x@}*z%a^w* z9>B2%eWd&b+u|*cnCYV%=Sf@cD&IY2t5_jeAv)4^=#61H=SRVh_g&8aIKSLWGtD{O zFMq>l=3tx0%G#jYIgV%LGc~U~R-&86>1nbY=UdwEsN5(CbS`XJCVnPS#`#Qe#v8-& z?CQ*V{PfJzz#n@?W|TvIlskp~kh+n@8u~-?hKily4T<5u-pJqR9R5r8)6jLMfk!Pv z*VzXKdT9net^AUuiR{{QPEk_R*P3H>_ku((%{s*@^GKJ`-MTi9iTtiJtf4-;*3VR_ z*ML^5EAprQ3naZ-~YruVDZF@ z$2Nal0L9`eZDo*f-db#+4TXRd6rvfL$bN`^rzY^b7217#D;FJlQrpmktwjftKw!N> zJff|YgaWn~x!AYI&uy<)Aa;o`iM6fWT83lP;0G*R?9PvtLyZzO>wy0Sjbm+& literal 0 HcmV?d00001 From 3655b1ea3fb33c20b758b6a70a76c699ed767f89 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Mon, 2 Oct 2023 21:02:20 -0700 Subject: [PATCH 03/32] Update dyi-content-discovery-platform-genai-beam.md Fixing the publication date. --- .../en/blog/dyi-content-discovery-platform-genai-beam.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md index 8057374591d7..fd967e318a07 100644 --- a/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md +++ b/website/www/site/content/en/blog/dyi-content-discovery-platform-genai-beam.md @@ -1,7 +1,7 @@ --- layout: post title: "DIY GenAI Content Discovery Platform with Apache Beam" -date: 2023-09-27 00:00:01 -0800 +date: 2023-10-02 00:00:01 -0800 categories: - blog authors: From 1c5982567b3f90e922285900d71b085a6d96c203 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 09:47:26 -0400 Subject: [PATCH 04/32] Bump cloud.google.com/go/bigtable from 1.19.0 to 1.20.0 in /sdks (#28789) Bumps [cloud.google.com/go/bigtable](https://github.com/googleapis/google-cloud-go) from 1.19.0 to 1.20.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/documentai/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/video/v1.19.0...video/v1.20.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigtable dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 869db285534d..9c911948c48f 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -24,7 +24,7 @@ go 1.20 require ( cloud.google.com/go/bigquery v1.55.0 - cloud.google.com/go/bigtable v1.19.0 + cloud.google.com/go/bigtable v1.20.0 cloud.google.com/go/datastore v1.14.0 cloud.google.com/go/profiler v0.3.1 cloud.google.com/go/pubsub v1.33.0 diff --git a/sdks/go.sum b/sdks/go.sum index 52512ffd424c..f1516bdf4a58 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -15,8 +15,8 @@ cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNF cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= cloud.google.com/go/bigquery v1.55.0 h1:hs44Xxov3XLWQiCx2J8lK5U/ihLqnpm4RVVl5fdtLLI= cloud.google.com/go/bigquery v1.55.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec= -cloud.google.com/go/bigtable v1.19.0 h1:wiq9LT0kukfInzvy1joMDijCw/OD1UChpSbORXYn0LI= -cloud.google.com/go/bigtable v1.19.0/go.mod h1:xl5kPa8PTkJjdBxg6qdGH88464nNqmbISHSRU+D2yFE= +cloud.google.com/go/bigtable v1.20.0 h1:NqZC/WcesSn4O8L0I2JmuNsUigSyBQifVLYgM9LMQeQ= +cloud.google.com/go/bigtable v1.20.0/go.mod h1:upJDn8frsjzpRMfybiWkD1PG6WCCL7CRl26MgVeoXY4= cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= From 25024064ed23b37664f165dd6af8a24c972a3148 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 09:47:55 -0400 Subject: [PATCH 05/32] Bump github.com/aws/aws-sdk-go-v2/config in /sdks (#28790) Bumps [github.com/aws/aws-sdk-go-v2/config](https://github.com/aws/aws-sdk-go-v2) from 1.18.42 to 1.18.43. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/config/v1.18.42...config/v1.18.43) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/config dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 8 ++++---- sdks/go.sum | 12 ++++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9c911948c48f..e17427227eba 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,8 +31,8 @@ require ( cloud.google.com/go/spanner v1.49.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.0 - github.com/aws/aws-sdk-go-v2/config v1.18.42 - github.com/aws/aws-sdk-go-v2/credentials v1.13.40 + github.com/aws/aws-sdk-go-v2/config v1.18.43 + github.com/aws/aws-sdk-go-v2/credentials v1.13.41 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 github.com/aws/smithy-go v1.14.2 @@ -107,9 +107,9 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.14.1 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 // indirect github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.22.0 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index f1516bdf4a58..71c1c4545c89 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -86,11 +86,13 @@ github.com/aws/aws-sdk-go-v2 v1.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pf github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= -github.com/aws/aws-sdk-go-v2/config v1.18.42 h1:28jHROB27xZwU0CB88giDSjz7M1Sba3olb5JBGwina8= github.com/aws/aws-sdk-go-v2/config v1.18.42/go.mod h1:4AZM3nMMxwlG+eZlxvBKqwVbkDLlnN2a4UGTL6HjaZI= +github.com/aws/aws-sdk-go-v2/config v1.18.43 h1:IgdUtTRvUDC6eiJBqU6vh7bHFNAEBjQ8S+qJ7zVhDOs= +github.com/aws/aws-sdk-go-v2/config v1.18.43/go.mod h1:NiFev8qlgg8MPzw3fO/EwzMZeZwlJEKGwfpjRPA9Nvw= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.40 h1:s8yOkDh+5b1jUDhMBtngF6zKWLDs84chUk2Vk0c38Og= github.com/aws/aws-sdk-go-v2/credentials v1.13.40/go.mod h1:VtEHVAAqDWASwdOqj/1huyT6uHbs5s8FUHfDQdky/Rs= +github.com/aws/aws-sdk-go-v2/credentials v1.13.41 h1:dgbKq1tamtboYAKSXWbqL0lKO9rmEzEhbZFh9JQW/Bg= +github.com/aws/aws-sdk-go-v2/credentials v1.13.41/go.mod h1:cc3Fn7DkKbJalPtQnudHGZZ8ml9+hwtbc1CJONsYYqk= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 h1:uDZJF1hu0EVT/4bogChk8DyjSF6fof6uL/0Y26Ma7Fg= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11/go.mod h1:TEPP4tENqBGO99KwVpV9MlOX4NSrSLP8u3KRy2CDwA8= @@ -121,13 +123,15 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 h1:wl5dxN1NONhTDQD9uaEvNsDRX29cBmGED/nl0jkWlt4= github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.14.1 h1:YkNzx1RLS0F5qdf9v1Q8Cuv9NXCL2TkosOxhzlUPV64= github.com/aws/aws-sdk-go-v2/service/sso v1.14.1/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 h1:vuGK1vHNP9zx0PfOrtPumbwR2af0ATQ1Z2H6p75AgRQ= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.0/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 h1:8lKOidPkmSmfUtiTgtdXWgaKItCZ/g75/jEk6Ql6GsA= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1/go.mod h1:yygr8ACQRY2PrEcy3xsUI357stq2AxnFM6DIsR9lij4= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.22.0 h1:s4bioTgjSFRwOoyEFzAVCmFmoowBgjTR8gkrF/sQ4wk= github.com/aws/aws-sdk-go-v2/service/sts v1.22.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 h1:pyvfUqkNLMipdKNAtu7OVbRxUrR2BMaKccIPpk/Hkak= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.14.2 h1:MJU9hqBGbvWZdApzpvoF2WAIJDbtjK2NDJSiJP7HblQ= github.com/aws/smithy-go v1.14.2/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= From b556aee9177b17ae4568f569e2b337f28adbc508 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 10:30:58 -0400 Subject: [PATCH 06/32] Bump urllib3 from 1.26.16 to 1.26.17 in /sdks/python/container/py38 (#28787) Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.26.16 to 1.26.17. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/main/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.26.16...1.26.17) --- updated-dependencies: - dependency-name: urllib3 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/python/container/py38/base_image_requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt index a47fb6beda82..ed5d35fc6457 100644 --- a/sdks/python/container/py38/base_image_requirements.txt +++ b/sdks/python/container/py38/base_image_requirements.txt @@ -135,7 +135,7 @@ tqdm==4.66.1 typing_extensions==4.8.0 tzlocal==5.0.1 uritemplate==4.1.1 -urllib3==1.26.16 +urllib3==1.26.17 websocket-client==1.6.3 wrapt==1.15.0 zstandard==0.21.0 From c7d7896533ba50e3bf2b64d6e0eb18ea7313a13f Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Tue, 3 Oct 2023 21:15:36 +0600 Subject: [PATCH 07/32] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Java_GBK_Dataflow_Batch_* (#28738) * beam_LoadTests_Java_GBK_Dataflow_Batch * argument fix * description added * fixed comment and filenames * config rename * config name fix --- ...beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 141 +++++++++++++++ ..._LoadTests_Java_GBK_Dataflow_Streaming.yml | 141 +++++++++++++++ ...ests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 166 +++++++++++++++++ ...ests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 168 ++++++++++++++++++ ..._Java_GBK_Dataflow_V2_Streaming_Java11.yml | 166 +++++++++++++++++ ..._Java_GBK_Dataflow_V2_Streaming_Java17.yml | 168 ++++++++++++++++++ ...GBK_Dataflow_Batch_2GB_of_100B_records.txt | 28 +++ ...BK_Dataflow_Batch_2GB_of_100kB_records.txt | 28 +++ ..._GBK_Dataflow_Batch_2GB_of_10B_records.txt | 28 +++ ...4_times_with_2GB_10-byte_records_total.txt | 28 +++ ...8_times_with_2GB_10-byte_records_total.txt | 28 +++ ...ow_Batch_reiterate_4_times_10kB_values.txt | 28 +++ ...low_Batch_reiterate_4_times_2MB_values.txt | 28 +++ ...Dataflow_Streaming_2GB_of_100B_records.txt | 29 +++ ...ataflow_Streaming_2GB_of_100kB_records.txt | 29 +++ ..._Dataflow_Streaming_2GB_of_10B_records.txt | 29 +++ ...4_times_with_2GB_10-byte_records_total.txt | 29 +++ ...8_times_with_2GB_10-byte_records_total.txt | 29 +++ ...treaming_reiterate_4_times_10kB_values.txt | 29 +++ ...Streaming_reiterate_4_times_2MB_values.txt | 29 +++ ...ow_V2_Batch_Java11_2GB_of_100B_records.txt | 29 +++ ...w_V2_Batch_Java11_2GB_of_100kB_records.txt | 29 +++ ...low_V2_Batch_Java11_2GB_of_10B_records.txt | 29 +++ ...4_times_with_2GB_10-byte_records_total.txt | 29 +++ ...8_times_with_2GB_10-byte_records_total.txt | 29 +++ ...h_Java11_reiterate_4_times_10kB_values.txt | 29 +++ ...ch_Java11_reiterate_4_times_2MB_values.txt | 29 +++ ...ow_V2_Batch_Java17_2GB_of_100B_records.txt | 29 +++ ...w_V2_Batch_Java17_2GB_of_100kB_records.txt | 29 +++ ...low_V2_Batch_Java17_2GB_of_10B_records.txt | 29 +++ ...4_times_with_2GB_10-byte_records_total.txt | 29 +++ ...8_times_with_2GB_10-byte_records_total.txt | 29 +++ ...h_Java17_reiterate_4_times_10kB_values.txt | 29 +++ ...ch_Java17_reiterate_4_times_2MB_values.txt | 29 +++ ...2_Streaming_Java11_2GB_of_100B_records.txt | 30 ++++ ..._Streaming_Java11_2GB_of_100kB_records.txt | 30 ++++ ...V2_Streaming_Java11_2GB_of_10B_records.txt | 30 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...g_Java11_reiterate_4_times_10kB_values.txt | 30 ++++ ...ng_Java11_reiterate_4_times_2MB_values.txt | 30 ++++ ...2_Streaming_Java17_2GB_of_100B_records.txt | 30 ++++ ..._Streaming_Java17_2GB_of_100kB_records.txt | 30 ++++ ...V2_Streaming_Java17_2GB_of_10B_records.txt | 30 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...g_Java17_reiterate_4_times_10kB_values.txt | 30 ++++ ...ng_Java17_reiterate_4_times_2MB_values.txt | 30 ++++ 48 files changed, 2175 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml create mode 100644 .github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt create mode 100644 .github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..b490ce69323e --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -0,0 +1,141 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 6 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java GBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Java GBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Batch_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml new file mode 100644 index 000000000000..19cc12f91eb1 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -0,0 +1,141 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 6 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java GBK Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Java GBK Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_Streaming_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml new file mode 100644 index 000000000000..aeb8006dfd53 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -0,0 +1,166 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Batch Java11 + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 7 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 11 GBK Dataflow V2 Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11"] + job_phrase: ["Run Load Tests Java 11 GBK Dataflow V2 Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: 11 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml new file mode 100644 index 000000000000..23f9f34d9747 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -0,0 +1,168 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Batch Java17 + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 7 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 17 GBK Dataflow V2 Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17"] + job_phrase: ["Run Load Tests Java 17 GBK Dataflow V2 Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: | + 17 + 8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml new file mode 100644 index 000000000000..9a0b236e43e9 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -0,0 +1,166 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Streaming Java11 + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 8 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 11 GBK Dataflow V2 Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11"] + job_phrase: ["Run Load Tests Java 11 GBK Dataflow V2 Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: 11 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava11 \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml new file mode 100644 index 000000000000..3580aaca51bb --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -0,0 +1,168 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: LoadTests Java GBK Dataflow V2 Streaming Java17 + +on: + issue_comment: + types: [created] + schedule: + - cron: '50 9 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java 17 GBK Dataflow V2 Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17"] + job_phrase: ["Run Load Tests Java 17 GBK Dataflow V2 Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: | + 17 + 8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB of 10B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_1 }}' \ + - name: run Load test 2GB of 100B records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_2 }}' \ + - name: run Load test 2GB of 100kB records + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_3 }}' \ + - name: run Load test fanout 4 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_4 }}' \ + - name: run Load test fanout 8 times with 2GB 10-byte records total + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_5 }}' \ + - name: run Load test reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_6 }}' \ + - name: run Load test reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + -Prunner.version=V2 \ + -PcompileAndRunTestsWithJava17 \ + -Pjava17Home=$JAVA_HOME_17_X64 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt new file mode 100644 index 000000000000..29d0ded11fac --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..920f0a60d198 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_100kB_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt new file mode 100644 index 000000000000..7ccacc6c791c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_2GB_of_10B_records.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8c6f6f1c89a0 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..43f04dac1d02 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_5 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..c8f9ecc34770 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_10kB_values.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_6 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..653004c6a04e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Batch_reiterate_4_times_2MB_values.txt @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_batch_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_7 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt new file mode 100644 index 000000000000..a6523c64edbc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..422557e84a66 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt new file mode 100644 index 000000000000..da6c04d0d284 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_2GB_of_10B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..33181a939700 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..32b59df2a2e0 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_5 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..376d7e3244a8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_10kB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_6 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..b7bf9db40e82 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_Streaming_reiterate_4_times_2MB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java_Dataflow_streaming_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_7 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt new file mode 100644 index 000000000000..d9b1918383b2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..66911feb7d17 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_100kB_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt new file mode 100644 index 000000000000..e265724ec1c1 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_2GB_of_10B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8610a9dff9f9 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..65aa56993294 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..aaf66ef03505 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_10kB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..8698ac90cac1 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java11_reiterate_4_times_2MB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_batch_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt new file mode 100644 index 000000000000..f5a432b6d140 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..976bd20f7dff --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_100kB_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt new file mode 100644 index 000000000000..89d356201a76 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_2GB_of_10B_records.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..0735c9bafc45 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..4ea7f402cc58 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..881640714df2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_10kB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..a09c16d1e66a --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Batch_Java17_reiterate_4_times_2MB_values.txt @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_batch_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt new file mode 100644 index 000000000000..84349e3dc06e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..761fad3d11ba --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_100kB_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt new file mode 100644 index 000000000000..1b6624e52482 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_2GB_of_10B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..cf173fa845c3 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8e751d638ceb --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..2e9c847cb832 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_10kB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..dd410a81487c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java11_reiterate_4_times_2MB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java11_Dataflow_V2_streaming_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java11"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt new file mode 100644 index 000000000000..05d93f213ec7 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_2 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..2391bfe4e416 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_100kB_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_3 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000,"keySizeBytes":10000,"valueSizeBytes":90000} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt new file mode 100644 index 000000000000..3c4e04e02adf --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_2GB_of_10B_records.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_1 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":200000000,"keySizeBytes":1,"valueSizeBytes":9} +--fanout=1 +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..d43ab9d84688 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_4 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":5000000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=4 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..fd5233180ff6 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_5 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_5 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":2500000,"keySizeBytes":10,"valueSizeBytes":90} +--fanout=8 +--iterations=1 +--numWorkers=16 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt new file mode 100644 index 000000000000..50d17886b7c9 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_10kB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_6 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_6 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt new file mode 100644 index 000000000000..e28d4d895ada --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_GBK_Dataflow_V2_Streaming_Java17_reiterate_4_times_2MB_values.txt @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +--region=us-central1 +--appName=load_tests_Java17_Dataflow_V2_streaming_GBK_7 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_gbk_7 +--influxTags={"runnerVersion":"v2","jdk":"java17"} +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":10,"hotKeyFraction":1} +--fanout=1 +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file From 2e0521162f6e23de29172eebfd3fd91f64329634 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 3 Oct 2023 11:46:14 -0400 Subject: [PATCH 08/32] [Java BQ] Storage API streaming load test (#28264) * more accurate regex for table spec; add toTableSpec * storage api streaming load test * use io performance test utilities * only fail 30% of the time * use periodicimpulse stopAfter * use custom bq service with crashing behavior --- ...m_PostCommit_Java_IO_Performance_Tests.yml | 2 +- it/build.gradle | 4 + it/google-cloud-platform/build.gradle | 5 +- .../it/gcp/bigquery/BigQueryStreamingLT.java | 643 ++++++++++++++++++ .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 6 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 6 +- .../sdk/io/gcp/bigquery/BigQueryUtils.java | 29 +- .../io/gcp/bigquery/BigQueryUtilsTest.java | 37 +- 8 files changed, 723 insertions(+), 9 deletions(-) create mode 100644 it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index 471782621fa7..b38651352a67 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -65,7 +65,7 @@ jobs: matrix: job_name: ["beam_PostCommit_Java_IO_Performance_Tests"] job_phrase: ["Run Java PostCommit IO Performance Tests"] - test_case: ["GCSPerformanceTest", "BigTablePerformanceTest"] + test_case: ["GCSPerformanceTest", "BigTablePerformanceTest", "BigQueryStorageApiStreamingPerformanceTest"] steps: - uses: actions/checkout@v4 - name: Setup repository diff --git a/it/build.gradle b/it/build.gradle index 35ccbba4c360..42a9ad9f4ee8 100644 --- a/it/build.gradle +++ b/it/build.gradle @@ -30,4 +30,8 @@ tasks.register('GCSPerformanceTest') { tasks.register('BigTablePerformanceTest') { dependsOn(":it:google-cloud-platform:BigTablePerformanceTest") +} + +tasks.register('BigQueryStorageApiStreamingPerformanceTest') { + dependsOn(":it:google-cloud-platform:BigQueryStorageApiStreamingPerformanceTest") } \ No newline at end of file diff --git a/it/google-cloud-platform/build.gradle b/it/google-cloud-platform/build.gradle index 0917ddd3e21a..4c5327b44c9a 100644 --- a/it/google-cloud-platform/build.gradle +++ b/it/google-cloud-platform/build.gradle @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +import groovy.json.JsonOutput import org.apache.beam.gradle.IoPerformanceTestUtilities plugins { id 'org.apache.beam.module' } @@ -73,9 +74,11 @@ dependencies { testImplementation project(path: ":sdks:java:io:google-cloud-platform") testImplementation project(path: ":sdks:java:io:synthetic") testImplementation library.java.mockito_inline + testImplementation project(path: ":sdks:java:extensions:google-cloud-platform-core", configuration: "testRuntimeMigration") testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadowTest") testRuntimeOnly library.java.slf4j_simple } tasks.register("GCSPerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'FileBasedIOLT', ['configuration':'large','project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) -tasks.register("BigTablePerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'BigTableIOLT', ['configuration':'large','project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) \ No newline at end of file +tasks.register("BigTablePerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'BigTableIOLT', ['configuration':'large','project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) +tasks.register("BigQueryStorageApiStreamingPerformanceTest", IoPerformanceTestUtilities.IoPerformanceTest, project, 'google-cloud-platform', 'BigQueryStreamingLT', ['configuration':'large', 'project':'apache-beam-testing', 'artifactBucket':'io-performance-temp']) diff --git a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java new file mode 100644 index 000000000000..4589f79f1aaa --- /dev/null +++ b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java @@ -0,0 +1,643 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.it.gcp.bigquery; + +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableReference; +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableSpec; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import com.google.api.core.ApiFuture; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.auto.value.AutoValue; +import com.google.cloud.bigquery.storage.v1.FlushRowsResponse; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.beam.it.common.PipelineLauncher; +import org.apache.beam.it.common.PipelineOperator; +import org.apache.beam.it.common.TestProperties; +import org.apache.beam.it.gcp.IOLoadTestBase; +import org.apache.beam.runners.dataflow.DataflowRunner; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryOptions; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl; +import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.PeriodicImpulse; +import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Load test for the Storage Write API sink + * + *

This test is set up to first write rows using batch FILE_LOADS mode to a "source of truth" + * table. Afterwards, it will write the same rows in streaming mode with Storage API to a second + * table. Then it will query between these two tables to check that they are identical. There is + * also the option of providing an existing table with the expected data, in which case the test + * will skip the first step. + * + *

The throughput, length of test (in minutes), and data shape can be changed via pipeline + * options. See the cases in `getOptions()` for examples. + * + *

This also includes the option of testing the sink's retry resilience by setting the + * `crashIntervalSeconds` System property. This intentionally fails the worker or work item + * periodically and expects the sink to recover appropriately. Note: Metrics are not published when + * this is used. + */ +public class BigQueryStreamingLT extends IOLoadTestBase { + private static final Logger LOG = LoggerFactory.getLogger(BigQueryStreamingLT.class); + + private static final BigqueryClient BQ_CLIENT = new BigqueryClient("BigQueryStreamingLT"); + private static final String BIG_QUERY_DATASET_ID = + "storage_api_sink_load_test_" + System.nanoTime(); + + private TestConfiguration config; + private Integer crashIntervalSeconds; + + @Rule public final transient TestPipeline fileLoadsPipeline = TestPipeline.create(); + @Rule public final transient TestPipeline storageApiPipeline = TestPipeline.create(); + + @BeforeClass + public static void setUpTestClass() throws IOException, InterruptedException { + PipelineOptionsFactory.register(TestPipelineOptions.class); + BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID); + } + + @Before + public void setUpTest() { + String testConfig = + TestProperties.getProperty("configuration", "small", TestProperties.Type.PROPERTY); + config = TEST_CONFIGS.get(testConfig); + if (config == null) { + throw new IllegalArgumentException( + String.format( + "Unknown test configuration: [%s]. Known configs: %s", + testConfig, TEST_CONFIGS.keySet())); + } + // tempLocation needs to be set for file loads + if (!Strings.isNullOrEmpty(tempBucketName)) { + String tempLocation = String.format("gs://%s/temp/", tempBucketName); + fileLoadsPipeline.getOptions().as(TestPipelineOptions.class).setTempRoot(tempLocation); + fileLoadsPipeline.getOptions().setTempLocation(tempLocation); + } + + // Set expected table if the property is provided, + @Nullable + String expectedTable = + TestProperties.getProperty("expectedTable", "", TestProperties.Type.PROPERTY); + if (!Strings.isNullOrEmpty(expectedTable)) { + config.toBuilder().setExpectedTable(expectedTable).build(); + } + + crashIntervalSeconds = + Integer.parseInt( + TestProperties.getProperty("crashIntervalSeconds", "-1", TestProperties.Type.PROPERTY)); + } + + @AfterClass + public static void cleanup() { + BQ_CLIENT.deleteDataset(project, BIG_QUERY_DATASET_ID); + } + + private static final Map TEST_CONFIGS = + ImmutableMap.of( + "local", // 300K rows, >3 MB, 1K rows/s, >10KB/s + TestConfiguration.of(5, 5, 2, 1_000, "DirectRunner", null), + "small", // 600K rows, >30 MB, 1K rows/s, >50KB/s + TestConfiguration.of(10, 10, 5, 1_000, "DataflowRunner", null), + "medium", // 6M rows, >1.2 GB, 5K rows/s, >1MB/s + TestConfiguration.of(20, 20, 10, 5_000, "DataflowRunner", null), + "large", // 18M rows, >18 GB, 10K rows/s, >10MB/s + TestConfiguration.of(30, 50, 20, 10_000, "DataflowRunner", null)); + + /** Options for Bigquery IO Streaming load test. */ + @AutoValue + abstract static class TestConfiguration { + /** Rows will be generated for this many minutes. */ + abstract Integer getMinutes(); + + /** Data shape: The byte-size for each field. */ + abstract Integer getByteSizePerField(); + + /** Data shape: The number of fields per row. */ + abstract Integer getNumFields(); + + /** + * Rate of generated elements sent to the sink. Will run with a minimum of 1k rows per second. + */ + abstract Integer getRowsPerSecond(); + + abstract String getRunner(); + + /** + * The expected table to check against for correctness. If unset, the test will run a batch + * FILE_LOADS job and use the resulting table as a source of truth. + */ + @Nullable + abstract String getExpectedTable(); + + static TestConfiguration of( + int numMin, + int byteSizePerField, + int numFields, + int rowsPerSecond, + String runner, + @Nullable String expectedTable) { + return new AutoValue_BigQueryStreamingLT_TestConfiguration.Builder() + .setMinutes(numMin) + .setByteSizePerField(byteSizePerField) + .setNumFields(numFields) + .setRowsPerSecond(rowsPerSecond) + .setRunner(runner) + .setExpectedTable(expectedTable) + .build(); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setMinutes(int numMin); + + abstract Builder setByteSizePerField(int byteSizePerField); + + abstract Builder setNumFields(int numFields); + + abstract Builder setRowsPerSecond(int rowsPerSecond); + + abstract Builder setRunner(String runner); + + abstract Builder setExpectedTable(@Nullable String expectedTable); + + abstract TestConfiguration build(); + } + + abstract Builder toBuilder(); + } + + @Test + public void testExactlyOnceStreaming() throws IOException, InterruptedException { + runTest(BigQueryIO.Write.Method.STORAGE_WRITE_API); + } + + @Test + @Ignore + public void testAtLeastOnceStreaming() throws IOException, InterruptedException { + runTest(BigQueryIO.Write.Method.STORAGE_API_AT_LEAST_ONCE); + } + + public void runTest(BigQueryIO.Write.Method writeMethod) + throws IOException, InterruptedException { + long millis = Duration.standardMinutes(config.getMinutes()).getMillis(); + int rowsPerSecond = Math.max(config.getRowsPerSecond(), 1000); + + // The PeriodicImpulse source will generate an element every this many millis: + int fireInterval = 1; + // Each element from PeriodicImpulse will fan out to this many elements + // (applicable when a high row-per-second rate is set) + long multiplier = rowsPerSecond / 1000; + long totalRows = multiplier * millis / fireInterval; + // If we run with DataflowRunner and have not specified a positive crash duration for the sink, + // this signifies a performance test, and so we publish metrics to a BigQuery dataset + boolean publishMetrics = + config.getRunner().equalsIgnoreCase(DataflowRunner.class.getSimpleName()) + && crashIntervalSeconds <= 0; + + String expectedTable = config.getExpectedTable(); + GenerateTableRow genRow = + new GenerateTableRow(config.getNumFields(), config.getByteSizePerField()); + TableSchema schema = generateTableSchema(config.getNumFields()); + if (Strings.isNullOrEmpty(expectedTable)) { + String fileLoadsDescription = + String.format("fileloads-%s-records", withScaleSymbol(totalRows)); + expectedTable = + String.format("%s.%s.%s", project, BIG_QUERY_DATASET_ID, fileLoadsDescription); + LOG.info( + "No expected table was set. Will run a batch job to load {} rows to {}." + + " This will be used as the source of truth.", + totalRows, + expectedTable); + + fileLoadsPipeline + .apply(GenerateSequence.from(0).to(totalRows)) + .apply( + "Write to source of truth", + BigQueryIO.write() + .to(expectedTable) + .withFormatFunction(genRow) + .withMethod(BigQueryIO.Write.Method.FILE_LOADS) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE) + .withSchema(schema)); + + // If running on Dataflow, launch pipeline via launcher utils + if (publishMetrics) { + PipelineLauncher.LaunchConfig options = + PipelineLauncher.LaunchConfig.builder("test-" + fileLoadsDescription) + .setSdk(PipelineLauncher.Sdk.JAVA) + .setPipeline(fileLoadsPipeline) + .addParameter("runner", config.getRunner()) + .build(); + + // Don't use PipelineOperator because we don't want to wait on this batch job + // The streaming job will run in parallel and it will take longer anyways; this job will + // finish by then. + pipelineLauncher.launch(project, region, options); + } else { + fileLoadsPipeline.run(); + } + } + + String atLeastOnce = + writeMethod == BigQueryIO.Write.Method.STORAGE_API_AT_LEAST_ONCE ? "-atleastonce" : ""; + String storageApiDescription = + String.format( + "storageapi%s-load-%sqps-%smin-%stotal", + atLeastOnce, + withScaleSymbol(rowsPerSecond), + config.getMinutes(), + withScaleSymbol(totalRows)); + String destTable = + String.format("%s.%s.%s", project, BIG_QUERY_DATASET_ID, storageApiDescription); + LOG.info( + "Preparing a source generating at a rate of {} rows per second for a period of {} minutes." + + " This results in a total of {} rows written to {}.", + rowsPerSecond, + config.getMinutes(), + totalRows, + destTable); + + PCollection source = + storageApiPipeline + .apply( + PeriodicImpulse.create() + .stopAfter(Duration.millis(millis - 1)) + .withInterval(Duration.millis(fireInterval))) + .apply( + "Extract row IDs", + MapElements.into(TypeDescriptors.longs()) + .via(instant -> instant.getMillis() % totalRows)); + if (multiplier > 1) { + source = + source + .apply( + String.format("One input to %s outputs", multiplier), + ParDo.of(new MultiplierDoFn(multiplier))) + .apply("Reshuffle fanout", Reshuffle.viaRandomKey()); + } + + BigQueryIO.Write storageWriteTransform = + BigQueryIO.write() + .to(destTable) + .withFormatFunction(genRow) + .withMethod(writeMethod) + .withTriggeringFrequency(Duration.standardSeconds(1)) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND) + .withSchema(schema); + + // If a crash interval is specified, use our crashing service implementation + if (crashIntervalSeconds > 0) { + LOG.info( + "A crash interval of {} seconds has been set. The Storage API sink will periodically crash.", + crashIntervalSeconds); + storageWriteTransform = + storageWriteTransform.withTestServices( + new CrashingBigQueryServices(crashIntervalSeconds)); + } + source.apply(storageWriteTransform); + + // If we're publishing metrics, launch pipeline via Dataflow launcher utils and export metrics + if (publishMetrics) { + // Set up dataflow job + PipelineLauncher.LaunchConfig storageApiOptions = + PipelineLauncher.LaunchConfig.builder("test-" + storageApiDescription) + .setSdk(PipelineLauncher.Sdk.JAVA) + .setPipeline(storageApiPipeline) + .addParameter("runner", config.getRunner()) + .addParameter("streaming", "true") + .addParameter("experiments", GcpOptions.STREAMING_ENGINE_EXPERIMENT) + .build(); + // Launch job + PipelineLauncher.LaunchInfo storageApiInfo = + pipelineLauncher.launch(project, region, storageApiOptions); + // Wait until the streaming pipeline is finished and drained, get the result. + PipelineOperator.Result storageApiResult = + pipelineOperator.waitUntilDoneAndFinish( + PipelineOperator.Config.builder() + .setJobId(storageApiInfo.jobId()) + .setProject(project) + .setRegion(region) + .setTimeoutAfter(java.time.Duration.ofMinutes(config.getMinutes() * 2L)) + .setCheckAfter(java.time.Duration.ofSeconds(config.getMinutes() * 60 / 20)) + .build()); + // Check the initial launch didn't fail + assertNotEquals(PipelineOperator.Result.LAUNCH_FAILED, storageApiResult); + // Check that the pipeline succeeded + assertEquals( + PipelineLauncher.JobState.DONE, + pipelineLauncher.getJobStatus(project, region, storageApiInfo.jobId())); + + // Export metrics + MetricsConfiguration metricsConfig = + MetricsConfiguration.builder() + .setInputPCollection( + (multiplier > 1) ? "Extract row IDs.out0" : "Reshuffle fanout.out0") + .build(); + try { + exportMetricsToBigQuery(storageApiInfo, getMetrics(storageApiInfo, metricsConfig)); + } catch (Exception e) { + // Just log the error. Don't re-throw because we have accuracy checks that are more + // important below + LOG.error("Encountered an error while exporting metrics to BigQuery:\n{}", e); + } + } + // If we're not publishing metrics, just run the pipeline normally + else { + storageApiPipeline.run().waitUntilFinish(); + } + + LOG.info( + "Write pipeline finished writing to {}. Will now perform accuracy checks against the rows in {}.", + destTable, + expectedTable); + // Filter our structs and arrays because they are not supported when querying with `EXCEPT + // DISTINCT` + String columnNames = + schema.getFields().stream() + .map(TableFieldSchema::getName) + .filter(fieldName -> fieldName.startsWith(FIELD_PREFIX)) + .collect(Collectors.joining(", ")); + checkCorrectness(columnNames, destTable, expectedTable); + // check non-duplication for STORAGE_WRITE_API + if (writeMethod == BigQueryIO.Write.Method.STORAGE_WRITE_API) { + checkNonDuplication(destTable, expectedTable, totalRows); + } + } + + // A BigQueryServices class that is almost identical to BigQueryServicesImpl, except that + // it returns a dataset service implementation that periodically crashes on flush() + private static class CrashingBigQueryServices extends BigQueryServicesImpl { + public final Integer crashIntervalSeconds; + + public CrashingBigQueryServices(Integer crashIntervalSeconds) { + this.crashIntervalSeconds = crashIntervalSeconds; + } + + @Override + public DatasetService getDatasetService(BigQueryOptions options) { + return new CrashingDatasetService(options); + } + + private class CrashingDatasetService extends BigQueryServicesImpl.DatasetServiceImpl { + private Instant lastCrash; + + public CrashingDatasetService(BigQueryOptions bqOptions) { + super(bqOptions); + } + + // We choose flush() to host the crash logic because it's called frequently during + // the span of a Storage Write API pipeline + @Override + public ApiFuture flush(String streamName, long flushOffset) + throws IOException, InterruptedException { + maybeCrash(); + return super.flush(streamName, flushOffset); + } + + // When specified, crash when the interval is met by: + // throwing an exception (failed work item) or + // performing a System exit (worker failure) + private void maybeCrash() { + if (crashIntervalSeconds != -1) { + Instant last = lastCrash; + if (last == null) { + lastCrash = Instant.now(); + } else if (Instant.now().isAfter(last.plusSeconds(crashIntervalSeconds))) { + lastCrash = Instant.now(); + + // Only crash 30% of the time (this is arbitrary) + if (ThreadLocalRandom.current().nextInt(100) < 30) { + // Half the time throw an exception (which fails this specific work item) + // Other half crash the entire worker, which fails all work items on this worker + if (ThreadLocalRandom.current().nextBoolean()) { + throw new RuntimeException( + "Throwing a random exception! This is for testing retry resilience."); + } else { + LOG.error("Crashing this worker! This is for testing retry resilience."); + System.exit(0); + } + } + } + } + } + } + } + + public void checkCorrectness(String columnNames, String destTable, String expectedTable) + throws IOException, InterruptedException { + // Need table spec to be in the format `myproject.mydataset.mytable` to include in BQ queries. + destTable = toTableSpec(toTableReference(destTable)); + expectedTable = toTableSpec(toTableReference(expectedTable)); + + String checkCorrectnessQuery = + String.format( + "WITH \n" + + "storage_api_table AS (SELECT %s FROM `%s`), \n" + + "expected_table AS (SELECT %s FROM `%s`), \n" + + "rows_mismatched AS (SELECT * FROM expected_table EXCEPT DISTINCT SELECT * FROM storage_api_table) \n" + + "SELECT COUNT(*) FROM rows_mismatched", + columnNames, destTable, columnNames, expectedTable); + + LOG.info("Executing query to check correctness:\n{}", checkCorrectnessQuery); + + TableRow queryResponse = + Iterables.getOnlyElement( + BQ_CLIENT.queryUnflattened(checkCorrectnessQuery, "google.com:clouddfe", true, true)); + long result = Long.parseLong((String) queryResponse.get("f0_")); + + LOG.info("Number of mismatched rows: {}", result); + assertEquals( + String.format("Saw %s rows that are missing from %s.", result, destTable), 0, result); + } + + public void checkNonDuplication(String destTable, String expectedTable, long totalRows) + throws IOException, InterruptedException { + String checkDuplicationQuery = + String.format( + "SELECT \n" + + "(SELECT COUNT(*) FROM `%s`) AS actualCount,\n" + + "(SELECT COUNT(*) FROM `%s`) AS expectedCount", + destTable, expectedTable); + + LOG.info("Executing query to check non-duplication:\n{}", checkDuplicationQuery); + + TableRow queryResponse = + Iterables.getOnlyElement( + BQ_CLIENT.queryUnflattened(checkDuplicationQuery, "google.com:clouddfe", true, true)); + long actualCount = Long.parseLong((String) queryResponse.get("actualCount")); + long expectedCount = Long.parseLong((String) queryResponse.get("expectedCount")); + assertEquals( + "Comparing actual table count and expected table count.", expectedCount, actualCount); + assertEquals( + "Comparing actual table count and calculated expected count.", totalRows, actualCount); + } + + // From a value, get the appropriate shortened name that includes the scale + // For example, from 12,345,678 return 12M + public String withScaleSymbol(long value) { + List scales = Arrays.asList("", "K", "M", "B", "T", "Q"); + int scaleIndex = 0; + while (value / 1000 > 0) { + scaleIndex++; + value /= 1000; + } + + return String.format("%s%s", value, scales.get(scaleIndex)); + } + + public static class MultiplierDoFn extends DoFn { + private long multiplier; + + MultiplierDoFn(long multiplier) { + this.multiplier = multiplier; + } + + @ProcessElement + public void processElement(@Element Long element, OutputReceiver outputReceiver) { + for (int i = 0; i < multiplier; i++) { + outputReceiver.output(element); + } + } + } + + static final String FIELD_PREFIX = "byte_field_"; + static final String RECORD_FIELD_PREFIX = "record_" + FIELD_PREFIX; + static final String NESTED_FIELD_PREFIX = "nested_" + FIELD_PREFIX; + static final String REPEATED_FIELD_PREFIX = "repeated_" + FIELD_PREFIX; + + public static TableSchema generateTableSchema(int numFields) { + List fields = new ArrayList<>(numFields); + fields.add(new TableFieldSchema().setType("INTEGER").setName("id")); + int j = 1; + for (int i = 1; i <= numFields; i++) { + TableFieldSchema fieldSchema = new TableFieldSchema(); + // Every 4th field will be a struct, every 5th field will be an array + if (j == 4) { + fieldSchema + .setType("RECORD") + .setName(RECORD_FIELD_PREFIX + i) + .setFields( + Arrays.asList( + new TableFieldSchema().setType("BYTES").setName(NESTED_FIELD_PREFIX + 1), + new TableFieldSchema().setType("BYTES").setName(NESTED_FIELD_PREFIX + 2))); + } else if (j == 5) { + fieldSchema.setType("BYTES").setMode("REPEATED").setName(REPEATED_FIELD_PREFIX + i); + j = 0; + } else { + fieldSchema.setType("BYTES").setName(FIELD_PREFIX + i); + } + j++; + fields.add(fieldSchema); + } + return new TableSchema().setFields(fields); + } + + static class GenerateTableRow implements SerializableFunction { + private final int numFields; + private final int sizePerField; + + public GenerateTableRow(int numFields, int sizePerField) { + assert numFields >= 0; + this.numFields = numFields; + this.sizePerField = sizePerField; + } + + @Override + public TableRow apply(Long rowId) { + TableRow row = new TableRow(); + row.set("id", rowId); + byte[] payload = getPayload(sizePerField, rowId).array(); + int j = 1; + for (int i = 1; i <= numFields; i++) { + // TODO: we can also make the struct and array sizes variable + if (j == 4) { + row.set( + RECORD_FIELD_PREFIX + i, + new TableRow() + .set(NESTED_FIELD_PREFIX + 1, Arrays.copyOfRange(payload, 0, sizePerField / 2)) + .set( + NESTED_FIELD_PREFIX + 2, + Arrays.copyOfRange(payload, sizePerField / 2, sizePerField))); + } else if (j == 5) { + row.set( + REPEATED_FIELD_PREFIX + i, + Arrays.asList( + Arrays.copyOfRange(payload, 0, sizePerField / 3), + Arrays.copyOfRange(payload, sizePerField / 3, sizePerField * 2 / 3), + Arrays.copyOfRange(payload, sizePerField * 2 / 3, sizePerField))); + j = 0; + } else { + row.set(FIELD_PREFIX + i, payload); + } + j++; + } + return row; + } + + private @Nullable ByteBuffer getPayload(int payloadSize, long rowId) { + if (payloadSize <= 0) { + return null; + } + byte[] payload = new byte[payloadSize]; + Random localRandom = ThreadLocal.withInitial(() -> new Random(rowId)).get(); + localRandom.setSeed(rowId); + localRandom.nextBytes(payload); + + return ByteBuffer.wrap(payload); + } + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 58d769312444..ce8ddb683d1e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -2499,6 +2499,8 @@ public enum WriteDisposition { *

The replacement may occur in multiple steps - for instance by first removing the * existing table, then creating a replacement, then filling it in. This is not an atomic * operation, and external programs may see the table in any of these intermediate steps. + * + *

Note: This write disposition is only supported for the FILE_LOADS write method. */ WRITE_TRUNCATE, @@ -3250,7 +3252,7 @@ && getStorageApiTriggeringFrequency(bqOptions) != null) { checkArgument(getNumFileShards() == 0, "Number of file shards" + error); if (getStorageApiTriggeringFrequency(bqOptions) != null) { - LOG.warn("Storage API triggering frequency" + error); + LOG.warn("Setting a triggering frequency" + error); } if (getStorageApiNumStreams(bqOptions) != 0) { LOG.warn("Setting the number of Storage API streams" + error); @@ -3266,6 +3268,8 @@ && getStorageApiTriggeringFrequency(bqOptions) != null) { checkArgument( !getAutoSchemaUpdate(), "withAutoSchemaUpdate only supported when using STORAGE_WRITE_API or STORAGE_API_AT_LEAST_ONCE."); + } else if (getWriteDisposition() == WriteDisposition.WRITE_TRUNCATE) { + LOG.error("The Storage API sink does not support the WRITE_TRUNCATE write disposition."); } if (getRowMutationInformationFn() != null) { checkArgument(getMethod() == Method.STORAGE_API_AT_LEAST_ONCE); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 17b5c5ebd99b..3d4565cb086e 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -154,7 +154,7 @@ "nullness", // TODO(https://github.com/apache/beam/issues/20506) "keyfor" }) -class BigQueryServicesImpl implements BigQueryServices { +public class BigQueryServicesImpl implements BigQueryServices { private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class); // The maximum number of retries to execute a BigQuery RPC. @@ -549,7 +549,7 @@ public void close() throws Exception {} } @VisibleForTesting - static class DatasetServiceImpl implements DatasetService { + public static class DatasetServiceImpl implements DatasetService { // Backoff: 200ms * 1.5 ^ n, n=[1,5] private static final FluentBackoff INSERT_BACKOFF_FACTORY = FluentBackoff.DEFAULT.withInitialBackoff(Duration.millis(200)).withMaxRetries(5); @@ -610,7 +610,7 @@ static class DatasetServiceImpl implements DatasetService { this.executor = null; } - private DatasetServiceImpl(BigQueryOptions bqOptions) { + public DatasetServiceImpl(BigQueryOptions bqOptions) { this.errorExtractor = new ApiErrorExtractor(); this.client = newBigQueryClient(bqOptions).build(); this.newWriteClient = newBigQueryWriteClient(bqOptions); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java index 00ee815c3c93..fa5ffae0909d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java @@ -64,6 +64,7 @@ import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -94,8 +95,15 @@ public class BigQueryUtils { // For parsing the format used to refer to tables parameters in BigQueryIO. // "{project_id}:{dataset_id}.{table_id}" or // "{project_id}.{dataset_id}.{table_id}" + // following documentation in + // https://cloud.google.com/resource-manager/docs/creating-managing-projects#before_you_begin, + // https://cloud.google.com/bigquery/docs/datasets#dataset-naming, and + // https://cloud.google.com/bigquery/docs/tables#table_naming private static final Pattern SIMPLE_TABLE_PATTERN = - Pattern.compile("^(?[^\\.:]+)[\\.:](?[^\\.:]+)[\\.](?[^\\.:]+)$"); + Pattern.compile( + "^(?[a-z][a-z0-9.\\-:]{4,28}[a-z0-9])[\\:.]" + + "(?[a-zA-Z0-9_]{1,1024})[\\.]" + + "(?
[\\p{L}\\p{M}\\p{N}\\p{Pc}\\p{Pd}\\p{Zs}$]{1,1024})$"); /** Options for how to convert BigQuery data to Beam data. */ @AutoValue @@ -1000,6 +1008,25 @@ private static Object convertAvroNumeric(Object value) { return null; } + /** + * @param tableReference - a BigQueryTableIdentifier that may or may not include the project. + * @return a String representation of the table destination in the form: + * `myproject.mydataset.mytable` + */ + public static @Nullable String toTableSpec(TableReference tableReference) { + if (tableReference.getDatasetId() == null || tableReference.getTableId() == null) { + throw new IllegalArgumentException( + String.format( + "Table reference [%s] must include at least a dataset and a table.", tableReference)); + } + String tableSpec = + String.format("%s.%s", tableReference.getDatasetId(), tableReference.getTableId()); + if (!Strings.isNullOrEmpty(tableReference.getProjectId())) { + tableSpec = String.format("%s.%s", tableReference.getProjectId(), tableSpec); + } + return tableSpec; + } + private static @Nullable ServiceCallMetric callMetricForMethod( @Nullable TableReference tableReference, String method) { if (tableReference != null) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java index f4074cc1a556..d73ff5e2b712 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java @@ -19,6 +19,7 @@ import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableRow; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableSchema; +import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.toTableSpec; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -994,6 +995,27 @@ public void testToBeamRow_avro_array_array_row() { assertEquals(expected, beamRow); } + @Test + public void testToTableSpec() { + TableReference withProject = + new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table"); + TableReference withoutProject = + new TableReference().setDatasetId("dataset").setTableId("table"); + TableReference withDatasetOnly = new TableReference().setDatasetId("dataset"); + TableReference withTableOnly = new TableReference().setTableId("table"); + + assertEquals("project.dataset.table", toTableSpec(withProject)); + assertEquals("dataset.table", toTableSpec(withoutProject)); + assertThrows( + "must include at least a dataset and a table", + IllegalArgumentException.class, + () -> toTableSpec(withDatasetOnly)); + assertThrows( + "must include at least a dataset and a table", + IllegalArgumentException.class, + () -> toTableSpec(withTableOnly)); + } + @Test public void testToTableReference() { { @@ -1020,6 +1042,14 @@ public void testToTableReference() { assertEquals("mytable", tr.getTableId()); } + { + // Test project that contains a dot and colon + TableReference tr = BigQueryUtils.toTableReference("project.with:domain.mydataset.mytable"); + assertEquals("project.with:domain", tr.getProjectId()); + assertEquals("mydataset", tr.getDatasetId()); + assertEquals("mytable", tr.getTableId()); + } + // Invalid scenarios assertNull(BigQueryUtils.toTableReference("")); assertNull(BigQueryUtils.toTableReference(":.")); @@ -1031,12 +1061,15 @@ public void testToTableReference() { assertNull(BigQueryUtils.toTableReference("myproject:mydataset.")); assertNull(BigQueryUtils.toTableReference("myproject:mydataset.mytable.")); assertNull(BigQueryUtils.toTableReference("myproject:mydataset:mytable:")); + assertNull(BigQueryUtils.toTableReference("myproject:my dataset:mytable:")); assertNull(BigQueryUtils.toTableReference(".invalidleadingdot.mydataset.mytable")); assertNull(BigQueryUtils.toTableReference("invalidtrailingdot.mydataset.mytable.")); assertNull(BigQueryUtils.toTableReference(":invalidleadingcolon.mydataset.mytable")); assertNull(BigQueryUtils.toTableReference("invalidtrailingcolon.mydataset.mytable:")); - assertNull(BigQueryUtils.toTableReference("myproject.mydataset.mytable.myinvalidpart")); - assertNull(BigQueryUtils.toTableReference("myproject:mydataset.mytable.myinvalidpart")); + assertNull(BigQueryUtils.toTableReference("projectendswithhyphen-.mydataset.mytable")); + assertNull( + BigQueryUtils.toTableReference( + "projectnamegoesbeyondthe30characterlimit.mydataset.mytable")); assertNull( BigQueryUtils.toTableReference("/projects/extraslash/datasets/mydataset/tables/mytable")); From 12b071402035a0404c4f6e4a83124b3839b5de40 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Tue, 3 Oct 2023 14:02:48 -0400 Subject: [PATCH 09/32] Add notebook for per key models (#28327) * Add notebook for per key models * Wording updates * Add new notebook to readme * Apply suggestions from Rebecca Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Clarify return type * Missing comma * temporarily install from head so this is linkable --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- examples/notebooks/beam-ml/README.md | 1 + .../notebooks/beam-ml/per_key_models.ipynb | 608 ++++++++++++++++++ 2 files changed, 609 insertions(+) create mode 100644 examples/notebooks/beam-ml/per_key_models.ipynb diff --git a/examples/notebooks/beam-ml/README.md b/examples/notebooks/beam-ml/README.md index 77bf3fc99f15..a1fe7ab19f51 100644 --- a/examples/notebooks/beam-ml/README.md +++ b/examples/notebooks/beam-ml/README.md @@ -77,6 +77,7 @@ This section contains the following example notebooks. ### Multi-model pipelines * [Ensemble model using an image captioning and ranking](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_multi_model.ipynb) +* [Run ML Inference with Different Models Per Key](https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/per_key_models.ipynb) ### Model Evaluation diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb new file mode 100644 index 000000000000..b529449555d0 --- /dev/null +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -0,0 +1,608 @@ +{ + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [] + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + } + }, + "cells": [ + { + "cell_type": "code", + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ], + "metadata": { + "id": "OsFaZscKSPvo" + }, + "execution_count": 1, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Run ML inference with multiple differently-trained models\n", + "\n", + "
\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
\n" + ], + "metadata": { + "id": "ZUSiAR62SgO8" + } + }, + { + "cell_type": "markdown", + "source": [ + "Running inference with multiple differently-trained models performing the same task is useful in many scenarios, including the following examples:\n", + "\n", + "* You want to compare the performance of multiple different models.\n", + "* You have models trained on different datasets that you want to use conditionally based on additional metadata.\n", + "\n", + "In Apache Beam, the recommended way to run inference is to use the `RunInference` transform. By using a `KeyedModelHandler`, you can efficiently run inference with O(100s) of models without having to manage memory yourself.\n", + "\n", + "This notebook demonstrates how to use a `KeyedModelHandler` to run inference in an Apache Beam pipeline with multiple different models on a per-key basis. This notebook uses pretrained pipelines from Hugging Face. Before continuing with this notebook, it is recommended that you walk through the [beginner RunInference notebook](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb)." + ], + "metadata": { + "id": "ZAVOrrW2An1n" + } + }, + { + "cell_type": "markdown", + "source": [ + "## Install dependencies\n", + "\n", + "First, install both Apache Beam and the dependencies needed by Hugging Face." + ], + "metadata": { + "id": "_fNyheQoDgGt" + } + }, + { + "cell_type": "code", + "execution_count": 11, + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "B-ENznuJqArA", + "outputId": "f72963fc-82db-4d0d-9225-07f6b501e256" + }, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "" + ] + } + ], + "source": [ + "# Note that this notebook currently installs from Beam head since this feature hasn't been released yet.\n", + "# It will be released with version 2.51.0, at which point you can install with the following command:\n", + "# !pip install apache_beam[gcp]>=2.51.0 --quiet\n", + "!git clone https://github.com/apache/beam\n", + "!pip install -r beam/sdks/python/build-requirements.txt\n", + "!pip install -e ./beam/sdks/python[gcp]\n", + "!pip install torch --quiet\n", + "!pip install transformers --quiet\n", + "\n", + "# To use the newly installed versions, restart the runtime.\n", + "exit()" + ] + }, + { + "cell_type": "code", + "source": [ + "from typing import Dict\n", + "from typing import Iterable\n", + "from typing import Tuple\n", + "\n", + "from transformers import pipeline\n", + "\n", + "import apache_beam as beam\n", + "from apache_beam.ml.inference.base import KeyedModelHandler\n", + "from apache_beam.ml.inference.base import KeyModelMapping\n", + "from apache_beam.ml.inference.base import PredictionResult\n", + "from apache_beam.ml.inference.huggingface_inference import HuggingFacePipelineModelHandler\n", + "from apache_beam.ml.inference.base import RunInference" + ], + "metadata": { + "id": "wUmBEglvsOYW" + }, + "execution_count": 1, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Define the model configurations\n", + "\n", + "A model handler is the Apache Beam method used to define the configuration needed to load and invoke models. Because this example uses two models, we define two model handlers, one for each model. Because both models are incapsulated within Hugging Face pipelines, we use the model handler `HuggingFacePipelineModelHandler`.\n", + "\n", + "In this notebook, we load the models using Hugging Face and run them against an example. The models produce different outputs." + ], + "metadata": { + "id": "uEqljVgCD7hx" + } + }, + { + "cell_type": "code", + "source": [ + "distilbert_mh = HuggingFacePipelineModelHandler('text-classification', model=\"distilbert-base-uncased-finetuned-sst-2-english\")\n", + "roberta_mh = HuggingFacePipelineModelHandler('text-classification', model=\"roberta-large-mnli\")\n", + "\n", + "distilbert_pipe = pipeline('text-classification', model=\"distilbert-base-uncased-finetuned-sst-2-english\")\n", + "roberta_large_pipe = pipeline(model=\"roberta-large-mnli\")" + ], + "metadata": { + "id": "v2NJT5ZcxgH5", + "outputId": "3924d72e-5c49-477d-c50f-6d9098f5a4b2" + }, + "execution_count": 2, + "outputs": [ + { + "output_type": "display_data", + "data": { + "text/plain": [ + "Downloading (…)lve/main/config.json: 0%| | 0.00/629 [00:00-`." + ], + "metadata": { + "id": "r6GXL5PLFBY7" + } + }, + { + "cell_type": "code", + "source": [ + "class FormatExamples(beam.DoFn):\n", + " \"\"\"\n", + " Map each example to a tuple of ('-', 'example').\n", + " We use these keys to map our elements to the correct models.\n", + " \"\"\"\n", + " def process(self, element: Tuple[str, str]) -> Iterable[Tuple[str, str]]:\n", + " yield (f'distilbert-{element[1]}', element[0])\n", + " yield (f'roberta-{element[1]}', element[0])" + ], + "metadata": { + "id": "p2uVwws8zRpg" + }, + "execution_count": 6, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If you're worried about your worker running out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/index.html#use-a-keyed-modelhandler)." + ], + "metadata": { + "id": "IP65_5nNGIb8" + } + }, + { + "cell_type": "code", + "source": [ + "per_key_mhs = [\n", + " KeyModelMapping(['distilbert-positive', 'distilbert-neutral', 'distilbert-negative'], distilbert_mh),\n", + " KeyModelMapping(['roberta-positive', 'roberta-neutral', 'roberta-negative'], roberta_mh)\n", + "]\n", + "mh = KeyedModelHandler(per_key_mhs, max_models_per_worker_hint=2)" + ], + "metadata": { + "id": "DZpfjeGL2hMG" + }, + "execution_count": 7, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Postprocess the results\n", + "\n", + "The `RunInference` transform returns a Tuple containing:\n", + "* the original key\n", + "* a `PredictionResult` object containing the original example and the inference.\n", + "Use those outputs to extract the relevant data. Then, to compare each model's prediction, group this data by the original example." + ], + "metadata": { + "id": "_a4ZmnD5FSeG" + } + }, + { + "cell_type": "code", + "source": [ + "class ExtractResults(beam.DoFn):\n", + " \"\"\"\n", + " Extract the relevant data from the PredictionResult object.\n", + " \"\"\"\n", + " def process(self, element: Tuple[str, PredictionResult]) -> Iterable[Tuple[str, Dict[str, str]]]:\n", + " actual_sentiment = element[0].split('-')[1]\n", + " model = element[0].split('-')[0]\n", + " result = element[1]\n", + " example = result.example\n", + " predicted_sentiment = result.inference[0]['label']\n", + "\n", + " yield (example, {'model': model, 'actual_sentiment': actual_sentiment, 'predicted_sentiment': predicted_sentiment})" + ], + "metadata": { + "id": "FOwFNQA053TG" + }, + "execution_count": 8, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Finally, print the results produced by each model." + ], + "metadata": { + "id": "JVnv4gGbFohk" + } + }, + { + "cell_type": "code", + "source": [ + "class PrintResults(beam.DoFn):\n", + " \"\"\"\n", + " Print the results produced by each model along with the actual sentiment.\n", + " \"\"\"\n", + " def process(self, element: Tuple[str, Iterable[Dict[str, str]]]):\n", + " example = element[0]\n", + " actual_sentiment = element[1][0]['actual_sentiment']\n", + " predicted_sentiment_1 = element[1][0]['predicted_sentiment']\n", + " model_1 = element[1][0]['model']\n", + " predicted_sentiment_2 = element[1][1]['predicted_sentiment']\n", + " model_2 = element[1][1]['model']\n", + "\n", + " if model_1 == 'distilbert':\n", + " distilbert_prediction = predicted_sentiment_1\n", + " roberta_prediction = predicted_sentiment_2\n", + " else:\n", + " roberta_prediction = predicted_sentiment_1\n", + " distilbert_prediction = predicted_sentiment_2\n", + "\n", + " print(f'Example: {example}\\nActual Sentiment: {actual_sentiment}\\n'\n", + " f'Distilbert Prediction: {distilbert_prediction}\\n'\n", + " f'Roberta Prediction: {roberta_prediction}\\n------------')" + ], + "metadata": { + "id": "kUQJNYOa9Q5-" + }, + "execution_count": 9, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Run the pipeline\n", + "\n", + "Put together all of the pieces to run a single Apache Beam pipeline." + ], + "metadata": { + "id": "-LrpmM2PGAkf" + } + }, + { + "cell_type": "code", + "source": [ + "with beam.Pipeline() as beam_pipeline:\n", + "\n", + " formatted_examples = (\n", + " beam_pipeline\n", + " | \"ReadExamples\" >> beam.Create(examples)\n", + " | \"FormatExamples\" >> beam.ParDo(FormatExamples()))\n", + " inferences = (\n", + " formatted_examples\n", + " | \"Run Inference\" >> RunInference(mh)\n", + " | \"ExtractResults\" >> beam.ParDo(ExtractResults())\n", + " | \"GroupByExample\" >> beam.GroupByKey()\n", + " )\n", + "\n", + " inferences | beam.ParDo(PrintResults())\n", + "\n" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 463 + }, + "id": "B9Wti3XH0Iqe", + "outputId": "528ad732-ecf8-4877-ab6a-badad7944fed" + }, + "execution_count": 10, + "outputs": [ + { + "output_type": "display_data", + "data": { + "application/javascript": [ + "\n", + " if (typeof window.interactive_beam_jquery == 'undefined') {\n", + " var jqueryScript = document.createElement('script');\n", + " jqueryScript.src = 'https://code.jquery.com/jquery-3.4.1.slim.min.js';\n", + " jqueryScript.type = 'text/javascript';\n", + " jqueryScript.onload = function() {\n", + " var datatableScript = document.createElement('script');\n", + " datatableScript.src = 'https://cdn.datatables.net/1.10.20/js/jquery.dataTables.min.js';\n", + " datatableScript.type = 'text/javascript';\n", + " datatableScript.onload = function() {\n", + " window.interactive_beam_jquery = jQuery.noConflict(true);\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }\n", + " document.head.appendChild(datatableScript);\n", + " };\n", + " document.head.appendChild(jqueryScript);\n", + " } else {\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }" + ] + }, + "metadata": {} + }, + { + "output_type": "stream", + "name": "stdout", + "text": [ + "Example: This restaurant is awesome\n", + "Actual Sentiment: positive\n", + "Distilbert Prediction: POSITIVE\n", + "Roberta Prediction: NEUTRAL\n", + "------------\n", + "Example: This restaurant is bad\n", + "Actual Sentiment: negative\n", + "Distilbert Prediction: NEGATIVE\n", + "Roberta Prediction: NEUTRAL\n", + "------------\n", + "Example: I love chocolate\n", + "Actual Sentiment: positive\n", + "Distilbert Prediction: POSITIVE\n", + "Roberta Prediction: NEUTRAL\n", + "------------\n", + "Example: I feel fine\n", + "Actual Sentiment: neutral\n", + "Distilbert Prediction: POSITIVE\n", + "Roberta Prediction: ENTAILMENT\n", + "------------\n" + ] + } + ] + } + ] +} From c01b41f9758392a00fd96965c91edea7b61d98bb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Oct 2023 15:29:30 -0400 Subject: [PATCH 10/32] Bump urllib3 from 1.26.16 to 1.26.17 in /sdks/python/container/py310 (#28785) Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.26.16 to 1.26.17. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/main/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.26.16...1.26.17) --- updated-dependencies: - dependency-name: urllib3 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/python/container/py310/base_image_requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index e8a666e7b9d0..548c56371706 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -134,7 +134,7 @@ tqdm==4.66.1 typing_extensions==4.8.0 tzlocal==5.0.1 uritemplate==4.1.1 -urllib3==1.26.16 +urllib3==1.26.17 websocket-client==1.6.3 wrapt==1.15.0 zstandard==0.21.0 From be8053722387c5427f7c9f2c37c451a4bdce77b0 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 3 Oct 2023 16:08:18 -0400 Subject: [PATCH 11/32] BigQuery testing suite that runs against BQ's day 0 region (#28397) * support creating dataset in specified region; * new gradle task * create yaml workflow * add file loads streaming test; add option to query with location * pass bq location to query operation --- ...m_PostCommit_Java_BigQueryEarlyRollout.yml | 97 +++++++++++++++++++ .../io/google-cloud-platform/build.gradle | 44 ++++++++- .../io/gcp/bigquery/TestBigQueryOptions.java | 7 ++ .../common/GcpIoPipelineOptionsRegistrar.java | 2 + .../sdk/io/gcp/testing/BigqueryClient.java | 37 ++++++- .../bigquery/BigQueryIOStorageQueryIT.java | 10 +- .../gcp/bigquery/BigQueryIOStorageReadIT.java | 9 +- .../BigQueryIOStorageReadTableRowIT.java | 43 ++++---- .../bigquery/BigQueryIOStorageWriteIT.java | 31 ++++-- .../BigQuerySchemaUpdateOptionsIT.java | 6 +- .../BigQueryTimePartitioningClusteringIT.java | 42 ++++++-- .../io/gcp/bigquery/BigQueryToTableIT.java | 9 +- .../io/gcp/bigquery/FileLoadsStreamingIT.java | 9 +- .../StorageApiDirectWriteProtosIT.java | 9 +- .../bigquery/StorageApiSinkFailedRowsIT.java | 13 ++- .../bigquery/StorageApiSinkRowUpdateIT.java | 9 +- .../StorageApiSinkSchemaUpdateIT.java | 17 ++-- .../bigquery/TableRowToStorageApiProtoIT.java | 11 ++- 18 files changed, 341 insertions(+), 64 deletions(-) create mode 100644 .github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml new file mode 100644 index 000000000000..952273e810d2 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -0,0 +1,97 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: PostCommit Java BigQueryEarlyRollout + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login}}' + cancel-in-progress: true + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: write + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PostCommit_Java_BigQueryEarlyRollout: + name: ${{matrix.job_name}} (${{matrix.job_phrase}}) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + strategy: + matrix: + job_name: [beam_PostCommit_Java_BigQueryEarlyRollout] + job_phrase: [Run Java BigQueryEarlyRollout PostCommit] + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Java BigQueryEarlyRollout PostCommit' + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Authenticate on GCP + uses: google-github-actions/setup-gcloud@v0 + with: + service_account_email: ${{ secrets.GCP_SA_EMAIL }} + service_account_key: ${{ secrets.GCP_SA_KEY }} + project_id: ${{ secrets.GCP_PROJECT_ID }} + export_default_credentials: true + - name: run PostCommit Java BigQueryEarlyRollout script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:google-cloud-platform:bigQueryEarlyRolloutIntegrationTest + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v3 + if: failure() + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' \ No newline at end of file diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index 560b27aae162..efc9ff3db9c6 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -202,10 +202,8 @@ task integrationTest(type: Test, dependsOn: processTestResources) { exclude '**/BigQueryIOReadIT.class' exclude '**/BigQueryIOStorageQueryIT.class' exclude '**/BigQueryIOStorageReadIT.class' - exclude '**/BigQueryIOStorageReadTableRowIT.class' exclude '**/BigQueryIOStorageWriteIT.class' exclude '**/BigQueryToTableIT.class' - exclude '**/BigQueryIOJsonTest.class' maxParallelForks 4 classpath = sourceSets.test.runtimeClasspath @@ -244,6 +242,48 @@ task integrationTestKms(type: Test) { } } +/* + Integration tests for BigQueryIO that run on BigQuery's early rollout region (us-east7) + with the intended purpose of catching breaking changes from new BigQuery releases. + If these tests fail here but not in `Java_GCP_IO_Direct`, there may be a new BigQuery change + that is breaking the connector. If this is the case, we should verify with the appropriate + BigQuery infrastructure API team. + + To test in a BigQuery location, we just need to create our datasets in that location. + */ +task bigQueryEarlyRolloutIntegrationTest(type: Test, dependsOn: processTestResources) { + group = "Verification" + def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' + def gcpTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-bigquery-day0-tests' + systemProperty "beamTestPipelineOptions", JsonOutput.toJson([ + "--runner=DirectRunner", + "--project=${gcpProject}", + "--tempRoot=${gcpTempRoot}", + "--bigQueryLocation=us-east7", + ]) + + outputs.upToDateWhen { false } + + // export and direct read + include '**/BigQueryToTableIT.class' + include '**/BigQueryIOJsonIT.class' + include '**/BigQueryIOStorageReadTableRowIT.class' + // storage write api + include '**/StorageApiDirectWriteProtosIT.class' + include '**/StorageApiSinkFailedRowsIT.class' + include '**/StorageApiSinkRowUpdateIT.class' + include '**/StorageApiSinkSchemaUpdateIT.class' + include '**/TableRowToStorageApiProtoIT.class' + // file loads + include '**/BigQuerySchemaUpdateOptionsIT.class' + include '**/BigQueryTimePartitioningClusteringIT.class' + include '**/FileLoadsStreamingIT.class' + + maxParallelForks 4 + classpath = sourceSets.test.runtimeClasspath + testClassesDirs = sourceSets.test.output.classesDirs +} + // path(s) for Cloud Spanner related classes def spannerIncludes = [ '**/org/apache/beam/sdk/io/gcp/spanner/**', diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java index 3574c12ee3a9..4d8095c1879d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQueryOptions.java @@ -24,10 +24,17 @@ /** {@link TestPipelineOptions} for {@link TestBigQuery}. */ public interface TestBigQueryOptions extends TestPipelineOptions, BigQueryOptions, GcpOptions { + String BIGQUERY_EARLY_ROLLOUT_REGION = "us-east7"; @Description("Dataset used in the integration tests. Default is integ_test") @Default.String("integ_test") String getTargetDataset(); void setTargetDataset(String value); + + @Description("Region to perform BigQuery operations in.") + @Default.String("") + String getBigQueryLocation(); + + void setBigQueryLocation(String location); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java index 1ed9ed6cb6c3..f1ff827fc633 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/common/GcpIoPipelineOptionsRegistrar.java @@ -20,6 +20,7 @@ import com.google.auto.service.AutoService; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryOptions; +import org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions; import org.apache.beam.sdk.io.gcp.firestore.FirestoreOptions; import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -36,6 +37,7 @@ public Iterable> getPipelineOptions() { .add(BigQueryOptions.class) .add(PubsubOptions.class) .add(FirestoreOptions.class) + .add(TestBigQueryOptions.class) .build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java index b21fdd669596..0e9476e6a226 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java @@ -292,6 +292,21 @@ private QueryResponse getTypedTableRows(QueryResponse response) { public List queryUnflattened( String query, String projectId, boolean typed, boolean useStandardSql) throws IOException, InterruptedException { + return queryUnflattened(query, projectId, typed, useStandardSql, null); + } + + /** + * Performs a query without flattening results. May choose a location (GCP region) to perform this + * operation in. + */ + @Nonnull + public List queryUnflattened( + String query, + String projectId, + boolean typed, + boolean useStandardSql, + @Nullable String location) + throws IOException, InterruptedException { Random rnd = new Random(System.currentTimeMillis()); String temporaryDatasetId = String.format("_dataflow_temporary_dataset_%s_%s", System.nanoTime(), rnd.nextInt(1000000)); @@ -302,9 +317,11 @@ public List queryUnflattened( .setDatasetId(temporaryDatasetId) .setTableId(temporaryTableId); - createNewDataset(projectId, temporaryDatasetId); + createNewDataset(projectId, temporaryDatasetId, null, location); createNewTable( - projectId, temporaryDatasetId, new Table().setTableReference(tempTableReference)); + projectId, + temporaryDatasetId, + new Table().setTableReference(tempTableReference).setLocation(location)); JobConfigurationQuery jcQuery = new JobConfigurationQuery() @@ -325,6 +342,7 @@ public List queryUnflattened( bqClient .jobs() .getQueryResults(projectId, insertedJob.getJobReference().getJobId()) + .setLocation(location) .execute(); } while (!qResponse.getJobComplete()); @@ -395,6 +413,18 @@ public void createNewDataset(String projectId, String datasetId) public void createNewDataset( String projectId, String datasetId, @Nullable Long defaultTableExpirationMs) throws IOException, InterruptedException { + createNewDataset(projectId, datasetId, defaultTableExpirationMs, null); + } + + /** + * Creates a new dataset with defaultTableExpirationMs and in a specified location (GCP region). + */ + public void createNewDataset( + String projectId, + String datasetId, + @Nullable Long defaultTableExpirationMs, + @Nullable String location) + throws IOException, InterruptedException { Sleeper sleeper = Sleeper.DEFAULT; BackOff backoff = BackOffAdapter.toGcpBackOff(BACKOFF_FACTORY.backoff()); IOException lastException = null; @@ -410,7 +440,8 @@ public void createNewDataset( projectId, new Dataset() .setDatasetReference(new DatasetReference().setDatasetId(datasetId)) - .setDefaultTableExpirationMs(defaultTableExpirationMs)) + .setDefaultTableExpirationMs(defaultTableExpirationMs) + .setLocation(location)) .execute(); if (response != null) { LOG.info("Successfully created new dataset : " + response.getId()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java index 692a12c0f4a7..d355d6bb9336 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions.BIGQUERY_EARLY_ROLLOUT_REGION; + import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; @@ -52,7 +54,13 @@ public class BigQueryIOStorageQueryIT { "1G", 11110839L, "1T", 11110839000L); - private static final String DATASET_ID = "big_query_storage"; + private static final String DATASET_ID = + TestPipeline.testingPipelineOptions() + .as(TestBigQueryOptions.class) + .getBigQueryLocation() + .equals(BIGQUERY_EARLY_ROLLOUT_REGION) + ? "big_query_storage_day0" + : "big_query_storage"; private static final String TABLE_PREFIX = "storage_read_"; private BigQueryIOStorageQueryOptions options; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java index 570938470b9d..b4f6ddb76f72 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions.BIGQUERY_EARLY_ROLLOUT_REGION; import static org.junit.Assert.assertEquals; import com.google.cloud.bigquery.storage.v1.DataFormat; @@ -65,7 +66,13 @@ public class BigQueryIOStorageReadIT { "1T", 11110839000L, "multi_field", 11110839L); - private static final String DATASET_ID = "big_query_storage"; + private static final String DATASET_ID = + TestPipeline.testingPipelineOptions() + .as(TestBigQueryOptions.class) + .getBigQueryLocation() + .equals(BIGQUERY_EARLY_ROLLOUT_REGION) + ? "big_query_storage_day0" + : "big_query_storage"; private static final String TABLE_PREFIX = "storage_read_"; private BigQueryIOStorageReadOptions options; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java index 734c3af2c4d4..35e2676c70ef 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions.BIGQUERY_EARLY_ROLLOUT_REGION; + import com.google.api.services.bigquery.model.TableRow; import java.util.HashSet; import java.util.Set; @@ -52,7 +54,13 @@ @RunWith(JUnit4.class) public class BigQueryIOStorageReadTableRowIT { - private static final String DATASET_ID = "big_query_import_export"; + private static final String DATASET_ID = + TestPipeline.testingPipelineOptions() + .as(TestBigQueryOptions.class) + .getBigQueryLocation() + .equals(BIGQUERY_EARLY_ROLLOUT_REGION) + ? "big_query_import_export_day0" + : "big_query_import_export"; private static final String TABLE_PREFIX = "parallel_read_table_row_"; private BigQueryIOStorageReadTableRowOptions options; @@ -67,12 +75,11 @@ public interface BigQueryIOStorageReadTableRowOptions void setInputTable(String table); } - private static class TableRowToKVPairFn extends SimpleFunction> { + private static class TableRowToKVPairFn extends SimpleFunction> { @Override - public KV apply(TableRow input) { - CharSequence sampleString = (CharSequence) input.get("sample_string"); - String key = sampleString != null ? sampleString.toString() : "null"; - return KV.of(key, BigQueryHelpers.toJsonString(input)); + public KV apply(TableRow input) { + Integer rowId = Integer.parseInt((String) input.get("id")); + return KV.of(rowId, BigQueryHelpers.toJsonString(input)); } } @@ -87,7 +94,7 @@ private void setUpTestEnvironment(String tableName) { private static void runPipeline(BigQueryIOStorageReadTableRowOptions pipelineOptions) { Pipeline pipeline = Pipeline.create(pipelineOptions); - PCollection> jsonTableRowsFromExport = + PCollection> jsonTableRowsFromExport = pipeline .apply( "ExportTable", @@ -96,7 +103,7 @@ private static void runPipeline(BigQueryIOStorageReadTableRowOptions pipelineOpt .withMethod(Method.EXPORT)) .apply("MapExportedRows", MapElements.via(new TableRowToKVPairFn())); - PCollection> jsonTableRowsFromDirectRead = + PCollection> jsonTableRowsFromDirectRead = pipeline .apply( "DirectReadTable", @@ -108,16 +115,16 @@ private static void runPipeline(BigQueryIOStorageReadTableRowOptions pipelineOpt final TupleTag exportTag = new TupleTag<>(); final TupleTag directReadTag = new TupleTag<>(); - PCollection>> unmatchedRows = + PCollection>> unmatchedRows = KeyedPCollectionTuple.of(exportTag, jsonTableRowsFromExport) .and(directReadTag, jsonTableRowsFromDirectRead) .apply(CoGroupByKey.create()) .apply( ParDo.of( - new DoFn, KV>>() { + new DoFn, KV>>() { @ProcessElement - public void processElement(ProcessContext c) throws Exception { - KV element = c.element(); + public void processElement(ProcessContext c) { + KV element = c.element(); // Add all the exported rows for the key to a collection. Set uniqueRows = new HashSet<>(); @@ -147,20 +154,20 @@ public void processElement(ProcessContext c) throws Exception { } @Test - public void testBigQueryStorageReadTableRow1() throws Exception { - setUpTestEnvironment("1"); + public void testBigQueryStorageReadTableRow100() { + setUpTestEnvironment("100"); runPipeline(options); } @Test - public void testBigQueryStorageReadTableRow10k() throws Exception { - setUpTestEnvironment("10k"); + public void testBigQueryStorageReadTableRow1k() { + setUpTestEnvironment("1K"); runPipeline(options); } @Test - public void testBigQueryStorageReadTableRow100k() throws Exception { - setUpTestEnvironment("100k"); + public void testBigQueryStorageReadTableRow10k() { + setUpTestEnvironment("10K"); runPipeline(options); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java index fc3ce0be4b69..d061898d55c7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageWriteIT.java @@ -26,11 +26,11 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import java.io.IOException; +import java.security.SecureRandom; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; -import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.MapElements; @@ -43,6 +43,8 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -60,24 +62,37 @@ private enum WriteMode { AT_LEAST_ONCE } - private String project; - private static final String DATASET_ID = "big_query_storage"; + private static String project; + private static final String DATASET_ID = + "big_query_storage_write_it_" + + System.currentTimeMillis() + + "_" + + new SecureRandom().nextInt(32); private static final String TABLE_PREFIX = "storage_write_"; - private BigQueryOptions bqOptions; + private static TestBigQueryOptions bqOptions; private static final BigqueryClient BQ_CLIENT = new BigqueryClient("BigQueryStorageIOWriteIT"); + @BeforeClass + public static void setup() throws Exception { + bqOptions = TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class); + project = bqOptions.as(GcpOptions.class).getProject(); + // Create one BQ dataset for all test cases. + BQ_CLIENT.createNewDataset(project, DATASET_ID, null, bqOptions.getBigQueryLocation()); + } + + @AfterClass + public static void cleanup() { + BQ_CLIENT.deleteDataset(project, DATASET_ID); + } + private void setUpTestEnvironment(WriteMode writeMode) { - PipelineOptionsFactory.register(BigQueryOptions.class); - bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); - bqOptions.setProject(TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject()); bqOptions.setUseStorageWriteApi(true); if (writeMode == WriteMode.AT_LEAST_ONCE) { bqOptions.setUseStorageWriteApiAtLeastOnce(true); } bqOptions.setNumStorageWriteApiStreams(2); bqOptions.setStorageWriteApiTriggeringFrequencySec(1); - project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); } static class FillRowFn extends DoFn { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java index 611c691dca12..833a0a0829c7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java @@ -87,7 +87,11 @@ public class BigQuerySchemaUpdateOptionsIT { @BeforeClass public static void setupTestEnvironment() throws Exception { project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); - BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID); + BQ_CLIENT.createNewDataset( + project, + BIG_QUERY_DATASET_ID, + null, + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation()); } @AfterClass diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java index 3ceb6f0966b7..da5f396e8d89 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java @@ -24,9 +24,11 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.api.services.bigquery.model.TimePartitioning; +import java.security.SecureRandom; import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -38,8 +40,10 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.ValueInSingleWindow; import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -49,7 +53,15 @@ public class BigQueryTimePartitioningClusteringIT { private static final String WEATHER_SAMPLES_TABLE = "apache-beam-testing.samples.weather_stations"; - private static final String DATASET_NAME = "BigQueryTimePartitioningIT"; + + private static String project; + private static final BigqueryClient BQ_CLIENT = + new BigqueryClient("BigQueryTimePartitioningClusteringIT"); + private static final String DATASET_NAME = + "BigQueryTimePartitioningIT_" + + System.currentTimeMillis() + + "_" + + new SecureRandom().nextInt(32); private static final TimePartitioning TIME_PARTITIONING = new TimePartitioning().setField("date").setType("DAY"); private static final Clustering CLUSTERING = @@ -64,6 +76,16 @@ public class BigQueryTimePartitioningClusteringIT { private Bigquery bqClient; private BigQueryClusteringITOptions options; + @BeforeClass + public static void setupTestEnvironment() throws Exception { + project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + BQ_CLIENT.createNewDataset( + project, + DATASET_NAME, + null, + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation()); + } + @Before public void setUp() { PipelineOptionsFactory.register(BigQueryClusteringITOptions.class); @@ -72,6 +94,11 @@ public void setUp() { bqClient = BigqueryClient.getNewBigqueryClient(options.getAppName()); } + @AfterClass + public static void cleanup() { + BQ_CLIENT.deleteDataset(project, DATASET_NAME); + } + /** Customized PipelineOptions for BigQueryClustering Integration Test. */ public interface BigQueryClusteringITOptions extends TestPipelineOptions, ExperimentalOptions, BigQueryOptions { @@ -110,8 +137,7 @@ public ClusteredDestinations(String tableName) { @Override public TableDestination getDestination(ValueInSingleWindow element) { - return new TableDestination( - String.format("%s.%s", DATASET_NAME, tableName), null, TIME_PARTITIONING, CLUSTERING); + return new TableDestination(tableName, null, TIME_PARTITIONING, CLUSTERING); } @Override @@ -176,6 +202,7 @@ public void testE2EBigQueryClustering() throws Exception { @Test public void testE2EBigQueryClusteringTableFunction() throws Exception { String tableName = "weather_stations_clustered_table_function_" + System.currentTimeMillis(); + String destination = String.format("%s.%s", DATASET_NAME, tableName); Pipeline p = Pipeline.create(options); @@ -185,11 +212,7 @@ public void testE2EBigQueryClusteringTableFunction() throws Exception { BigQueryIO.writeTableRows() .to( (ValueInSingleWindow vsw) -> - new TableDestination( - String.format("%s.%s", DATASET_NAME, tableName), - null, - TIME_PARTITIONING, - CLUSTERING)) + new TableDestination(destination, null, TIME_PARTITIONING, CLUSTERING)) .withClustering() .withSchema(SCHEMA) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) @@ -206,6 +229,7 @@ public void testE2EBigQueryClusteringTableFunction() throws Exception { public void testE2EBigQueryClusteringDynamicDestinations() throws Exception { String tableName = "weather_stations_clustered_dynamic_destinations_" + System.currentTimeMillis(); + String destination = String.format("%s.%s", DATASET_NAME, tableName); Pipeline p = Pipeline.create(options); @@ -213,7 +237,7 @@ public void testE2EBigQueryClusteringDynamicDestinations() throws Exception { .apply(ParDo.of(new KeepStationNumberAndConvertDate())) .apply( BigQueryIO.writeTableRows() - .to(new ClusteredDestinations(tableName)) + .to(new ClusteredDestinations(destination)) .withClustering() .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE)); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java index d6b7f8e16412..1abe7752b2e0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java @@ -46,7 +46,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.WithKeys; @@ -214,7 +213,7 @@ private void verifyStandardQueryRes(String outputTable) throws Exception { } /** Customized PipelineOption for BigQueryToTable Pipeline. */ - public interface BigQueryToTableOptions extends TestPipelineOptions, ExperimentalOptions { + public interface BigQueryToTableOptions extends TestBigQueryOptions, ExperimentalOptions { @Description("The BigQuery query to be used for creating the source") @Validation.Required @@ -252,9 +251,11 @@ public interface BigQueryToTableOptions extends TestPipelineOptions, Experimenta @BeforeClass public static void setupTestEnvironment() throws Exception { PipelineOptionsFactory.register(BigQueryToTableOptions.class); - project = TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject(); + BigQueryToTableOptions options = + TestPipeline.testingPipelineOptions().as(BigQueryToTableOptions.class); + project = options.as(GcpOptions.class).getProject(); // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID); + BQ_CLIENT.createNewDataset(project, BIG_QUERY_DATASET_ID, null, options.getBigQueryLocation()); // Create table and insert data for new type query test cases. BQ_CLIENT.createNewTable( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java index 012afed6fb43..678708062b8d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java @@ -106,11 +106,16 @@ public static Iterable data() { private final Random randomGenerator = new Random(); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. cleanUp(); - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -293,7 +298,7 @@ private static void checkRowCompleteness( throws IOException, InterruptedException { List actualTableRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false); + String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false, bigQueryLocation); Schema rowSchema = BigQueryUtils.fromTableSchema(schema); List actualBeamRows = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java index 93bc4162409f..3da93c42a480 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDirectWriteProtosIT.java @@ -80,10 +80,15 @@ private BigQueryIO.Write.Method getMethod() { : BigQueryIO.Write.Method.STORAGE_WRITE_API; } + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -191,7 +196,7 @@ public void testDirectWriteProtos() throws Exception { void assertRowsWritten(String tableSpec, Iterable expectedItems) throws Exception { List rows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertThat(rows, containsInAnyOrder(Iterables.toArray(expectedItems, TableRow.class))); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java index 3dcde8f39cd7..f721f57147e3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkFailedRowsIT.java @@ -108,10 +108,15 @@ private BigQueryIO.Write.Method getMethod() { : BigQueryIO.Write.Method.STORAGE_WRITE_API; } + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -217,7 +222,11 @@ private void assertGoodRowsWritten(String tableSpec, Iterable goodRows TableRow queryResponse = Iterables.getOnlyElement( BQ_CLIENT.queryUnflattened( - String.format("SELECT COUNT(*) FROM %s", tableSpec), PROJECT, true, true)); + String.format("SELECT COUNT(*) FROM `%s`", tableSpec), + PROJECT, + true, + true, + bigQueryLocation)); int numRowsWritten = Integer.parseInt((String) queryResponse.get("f0_")); if (useAtLeastOnce) { assertThat(numRowsWritten, Matchers.greaterThanOrEqualTo(Iterables.size(goodRows))); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java index d5366fe29613..f8cc797a87cd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkRowUpdateIT.java @@ -49,10 +49,15 @@ public class StorageApiSinkRowUpdateIT { private static final String BIG_QUERY_DATASET_ID = "storage_api_sink_rows_update" + System.nanoTime(); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -129,7 +134,7 @@ private void assertRowsWritten(String tableSpec, Iterable expected) throws IOException, InterruptedException { List queryResponse = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertThat(queryResponse, containsInAnyOrder(Iterables.toArray(expected, TableRow.class))); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java index 6931b7ac9b98..bc99a4f50f70 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java @@ -121,17 +121,21 @@ public static Iterable data() { // an updated schema. If that happens consistently, just increase these two numbers // to give it more time. // Total number of rows written to the sink - private static final int TOTAL_N = 60; + private static final int TOTAL_N = 70; // Number of rows with the original schema - private static final int ORIGINAL_N = 50; + private static final int ORIGINAL_N = 60; private final Random randomGenerator = new Random(); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - LOG.info("Creating dataset {}.", BIG_QUERY_DATASET_ID); - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -459,7 +463,8 @@ private static void checkRowCompleteness( String.format("SELECT COUNT(DISTINCT(id)), COUNT(id) FROM [%s]", tableSpec), PROJECT, true, - false)); + false, + bigQueryLocation)); int distinctCount = Integer.parseInt((String) queryResponse.get("f0_")); int totalCount = Integer.parseInt((String) queryResponse.get("f1_")); @@ -479,7 +484,7 @@ public void checkRowsWithUpdatedSchema( throws IOException, InterruptedException { List actualRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false); + String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false, bigQueryLocation); for (TableRow row : actualRows) { // Rows written to the table should not have the extra field if diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java index 218aa7411414..f28ae588a5ec 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowToStorageApiProtoIT.java @@ -318,10 +318,15 @@ public class TableRowToStorageApiProtoIT { .setFields(BASE_TABLE_SCHEMA.getFields())) .build()); + // used when test suite specifies a particular GCP location for BigQuery operations + private static String bigQueryLocation; + @BeforeClass public static void setUpTestEnvironment() throws IOException, InterruptedException { // Create one BQ dataset for all test cases. - BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID); + bigQueryLocation = + TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class).getBigQueryLocation(); + BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID, null, bigQueryLocation); } @AfterClass @@ -338,7 +343,7 @@ public void testBaseTableRow() throws IOException, InterruptedException { List actualTableRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertEquals(1, actualTableRows.size()); assertEquals(BASE_TABLE_ROW_EXPECTED, actualTableRows.get(0)); @@ -364,7 +369,7 @@ public void testNestedRichTypesAndNull() throws IOException, InterruptedExceptio List actualTableRows = BQ_CLIENT.queryUnflattened( - String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true); + String.format("SELECT * FROM %s", tableSpec), PROJECT, true, true, bigQueryLocation); assertEquals(1, actualTableRows.size()); assertEquals(BASE_TABLE_ROW_EXPECTED, actualTableRows.get(0).get("nestedValue1")); From f6b43ce1736250c044ad90305351662bb95533a7 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 3 Oct 2023 18:57:58 -0400 Subject: [PATCH 12/32] Revert "assign highmem runner to beam_PostCommit_Python and to beam_PreCommit_Java_GCP_IO_Direct (#28719)" (#28804) This reverts commit a4ee8548424b335cb3ffdf6aeda899fd1e3ba8ad. --- .github/workflows/beam_PostCommit_Python.yml | 2 +- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index a7a214c7c5a9..6f4bc5e2ef0b 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -53,7 +53,7 @@ env: jobs: beam_PostCommit_Python: name: ${{matrix.job_name}} (${{matrix.job_phrase}} ${{matrix.python_version}}) - runs-on: [self-hosted, ubuntu-20.04, highmem] + runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 240 strategy: fail-fast: false diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 7c821a024742..30e8d6d6c33c 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -90,7 +90,7 @@ jobs: github.event_name == 'schedule' || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_GCP_IO_Direct PreCommit' - runs-on: [self-hosted, ubuntu-20.04, highmem] + runs-on: [self-hosted, ubuntu-20.04, main] steps: - uses: actions/checkout@v4 - name: Setup repository From 95af465f0dfaa123364e5880bccc3dff32a19b39 Mon Sep 17 00:00:00 2001 From: brucearctor <5032356+brucearctor@users.noreply.github.com> Date: Wed, 4 Oct 2023 05:30:15 -0700 Subject: [PATCH 13/32] yaml label (#28813) --- .github/autolabeler.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/autolabeler.yml b/.github/autolabeler.yml index 5a8a22044da4..d1cc8296d303 100644 --- a/.github/autolabeler.yml +++ b/.github/autolabeler.yml @@ -31,6 +31,7 @@ python: ["sdks/python/**/*", "learning/katas/python/**/*"] typescript: ["sdks/typescript/**/*"] vendor: ["vendor/**/*"] website: ["website/**/*"] +yaml: ["sdks/python/apache_beam/yaml/**"] # Extensions extensions: ["sdks/java/extensions/**/*", "runners/extensions-java/**/*"] From a9684c5563dc9b5779cbe874d1fead05de2cf36c Mon Sep 17 00:00:00 2001 From: Alexey Romanenko Date: Wed, 4 Oct 2023 14:58:16 +0200 Subject: [PATCH 14/32] TPC-DS: Fix CSV format --- .../java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java index e8b85f63b36a..6570b7fe81b2 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java @@ -283,7 +283,8 @@ public static void runUsingSqlTransform(String[] args) throws Exception { // Make an array of pipelines, each pipeline is responsible for running a corresponding query. Pipeline[] pipelines = new Pipeline[queryNames.length]; - CSVFormat csvFormat = CSVFormat.MYSQL.withDelimiter('|').withNullString(""); + CSVFormat csvFormat = + CSVFormat.MYSQL.withDelimiter('|').withTrailingDelimiter().withNullString(""); // Execute all queries, transform each result into a PCollection, write them into // the txt file and store in a GCP directory. From 9efdf6be61e46d97b51161ea934706a6a18e9118 Mon Sep 17 00:00:00 2001 From: nancyxu123 Date: Wed, 4 Oct 2023 08:50:59 -0700 Subject: [PATCH 15/32] Fix SpannerChangeStreamErrorTest.java and stop disabling tests (#28751) * Fix SpannerChangeStreamErrorTest.java and stop disabling tests * More fixes to existing error tests * Formatting changes * Make the errors a flexible retry --- .../SpannerChangeStreamErrorTest.java | 181 +++++++++++++----- 1 file changed, 138 insertions(+), 43 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java index bf2ccd454bb5..9ffa61c93078 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/SpannerChangeStreamErrorTest.java @@ -52,7 +52,9 @@ import com.google.spanner.v1.TypeCode; import io.grpc.Status; import java.io.Serializable; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import org.apache.beam.runners.direct.DirectOptions; import org.apache.beam.runners.direct.DirectRunner; import org.apache.beam.sdk.Pipeline; @@ -68,7 +70,6 @@ import org.joda.time.Duration; import org.junit.After; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -114,16 +115,22 @@ public void tearDown() throws NoSuchFieldException, IllegalAccessException { } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect returns the right message.") - public void testResourceExhaustedDoesNotRetry() { + // Error code UNAVAILABLE is retried repeatedly until the RPC times out. + public void testUnavailableExceptionRetries() throws InterruptedException { + DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class); + options.setBlockOnRun(false); + options.setRunner(DirectRunner.class); + Pipeline nonBlockingPipeline = TestPipeline.create(options); + mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.RESOURCE_EXHAUSTED.asRuntimeException())); + SimulatedExecutionTime.ofStickyException(Status.UNAVAILABLE.asRuntimeException())); final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); + try { - pipeline.apply( + nonBlockingPipeline.apply( SpannerIO.readChangeStream() .withSpannerConfig(getSpannerConfig()) .withChangeStreamName(TEST_CHANGE_STREAM) @@ -131,33 +138,36 @@ public void testResourceExhaustedDoesNotRetry() { .withMetadataTable(TEST_TABLE) .withInclusiveStartAt(startTimestamp) .withInclusiveEndAt(endTimestamp)); - pipeline.run().waitUntilFinish(); + PipelineResult result = nonBlockingPipeline.run(); + while (result.getState() != RUNNING) { + Thread.sleep(50); + } + // The pipeline continues making requests to Spanner to retry the Unavailable errors. + assertNull(result.waitUntilFinish(Duration.millis(500))); } finally { - thrown.expect(SpannerException.class); // databaseClient.getDialect does not currently bubble up the correct message. // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " // "in the given time" - thrown.expectMessage("RESOURCE_EXHAUSTED - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("DEADLINE_EXCEEDED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect returns the right message.") - public void testUnavailableExceptionRetries() throws InterruptedException { + // Error code ABORTED is retried repeatedly until it times out. + public void testAbortedExceptionRetries() throws InterruptedException { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofStickyException(Status.ABORTED.asRuntimeException())); + DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class); options.setBlockOnRun(false); options.setRunner(DirectRunner.class); Pipeline nonBlockingPipeline = TestPipeline.create(options); - mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.UNAVAILABLE.asRuntimeException())); - final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); - try { nonBlockingPipeline.apply( SpannerIO.readChangeStream() @@ -171,23 +181,20 @@ public void testUnavailableExceptionRetries() throws InterruptedException { while (result.getState() != RUNNING) { Thread.sleep(50); } - // The pipeline continues making requests to Spanner to retry the Unavailable errors. + // The pipeline continues making requests to Spanner to retry the Aborted errors. assertNull(result.waitUntilFinish(Duration.millis(500))); } finally { - // databaseClient.getDialect does not currently bubble up the correct message. - // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " - // "in the given time" - thrown.expectMessage("UNAVAILABLE - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("DEADLINE_EXCEEDED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect returns the right message.") - public void testAbortedExceptionNotRetried() { + // Error code UNKNOWN is not retried. + public void testUnknownExceptionDoesNotRetry() { mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.ABORTED.asRuntimeException())); + SimulatedExecutionTime.ofStickyException(Status.UNKNOWN.asRuntimeException())); final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = @@ -204,19 +211,43 @@ public void testAbortedExceptionNotRetried() { pipeline.run().waitUntilFinish(); } finally { thrown.expect(SpannerException.class); - // databaseClient.getDialect does not currently bubble up the correct message. - // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " - // "in the given time" - thrown.expectMessage("ABORTED - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("UNKNOWN"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - public void testAbortedExceptionNotRetriedithDefaultsForStreamSqlRetrySettings() { + // Error code RESOURCE_EXHAUSTED is retried repeatedly. + public void testResourceExhaustedRetry() { mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.ABORTED.asRuntimeException())); + SimulatedExecutionTime.ofStickyException(Status.RESOURCE_EXHAUSTED.asRuntimeException())); + + final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); + final Timestamp endTimestamp = + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); + + try { + pipeline.apply( + SpannerIO.readChangeStream() + .withSpannerConfig(getSpannerConfig()) + .withChangeStreamName(TEST_CHANGE_STREAM) + .withMetadataDatabase(TEST_DATABASE) + .withMetadataTable(TEST_TABLE) + .withInclusiveStartAt(startTimestamp) + .withInclusiveEndAt(endTimestamp)); + pipeline.run().waitUntilFinish(); + } finally { + thrown.expectMessage("DEADLINE_EXCEEDED"); + assertThat( + mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); + } + } + + @Test + public void testResourceExhaustedRetryWithDefaultSettings() { + mockSpannerService.setExecuteStreamingSqlExecutionTime( + SimulatedExecutionTime.ofStickyException(Status.RESOURCE_EXHAUSTED.asRuntimeException())); final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = @@ -230,6 +261,7 @@ public void testAbortedExceptionNotRetriedithDefaultsForStreamSqlRetrySettings() .withProjectId(TEST_PROJECT) .withInstanceId(TEST_INSTANCE) .withDatabaseId(TEST_DATABASE); + try { pipeline.apply( SpannerIO.readChangeStream() @@ -241,24 +273,34 @@ public void testAbortedExceptionNotRetriedithDefaultsForStreamSqlRetrySettings() .withInclusiveEndAt(endTimestamp)); pipeline.run().waitUntilFinish(); } finally { - // databaseClient.getDialect does not currently bubble up the correct message. - // Instead, the error returned is: "DEADLINE_EXCEEDED: Operation did not complete " - // "in the given time" thrown.expect(SpannerException.class); - thrown.expectMessage("ABORTED - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + thrown.expectMessage("RESOURCE_EXHAUSTED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - public void testUnknownExceptionDoesNotRetry() { - mockSpannerService.setExecuteStreamingSqlExecutionTime( - SimulatedExecutionTime.ofStickyException(Status.UNKNOWN.asRuntimeException())); - + public void testInvalidRecordReceived() { final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); + + mockGetDialect(); + mockTableExists(); + mockGetWatermark(startTimestamp); + ResultSet getPartitionResultSet = mockGetParentPartition(startTimestamp, endTimestamp); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() - 1), + getPartitionResultSet); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos()), + ResultSet.newBuilder().setMetadata(PARTITION_METADATA_RESULT_SET_METADATA).build()); + mockGetPartitionsAfter( + Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1), + ResultSet.newBuilder().setMetadata(PARTITION_METADATA_RESULT_SET_METADATA).build()); + mockInvalidChangeStreamRecordReceived(startTimestamp, endTimestamp); + try { pipeline.apply( SpannerIO.readChangeStream() @@ -271,15 +313,16 @@ public void testUnknownExceptionDoesNotRetry() { pipeline.run().waitUntilFinish(); } finally { thrown.expect(SpannerException.class); - thrown.expectMessage("UNKNOWN - Statement: 'SELECT 'POSTGRESQL' AS DIALECT"); + // DatabaseClient.getDialect returns "DEADLINE_EXCEEDED: Operation did not complete in the " + // given time" even though we mocked it out. + thrown.expectMessage("DEADLINE_EXCEEDED"); assertThat( mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); } } @Test - @Ignore("BEAM-12164 Reenable this test when databaseClient.getDialect works.") - public void testInvalidRecordReceived() { + public void testInvalidRecordReceivedWithDefaultSettings() { final Timestamp startTimestamp = Timestamp.ofTimeSecondsAndNanos(0, 1000); final Timestamp endTimestamp = Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() + 1); @@ -288,6 +331,8 @@ public void testInvalidRecordReceived() { mockTableExists(); mockGetWatermark(startTimestamp); ResultSet getPartitionResultSet = mockGetParentPartition(startTimestamp, endTimestamp); + mockchangePartitionState(startTimestamp, endTimestamp, "CREATED"); + mockchangePartitionState(startTimestamp, endTimestamp, "SCHEDULED"); mockGetPartitionsAfter( Timestamp.ofTimeSecondsAndNanos(startTimestamp.getSeconds(), startTimestamp.getNanos() - 1), getPartitionResultSet); @@ -300,9 +345,26 @@ public void testInvalidRecordReceived() { mockInvalidChangeStreamRecordReceived(startTimestamp, endTimestamp); try { + RetrySettings quickRetrySettings = + RetrySettings.newBuilder() + .setInitialRetryDelay(org.threeten.bp.Duration.ofMillis(250)) + .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(1)) + .setRetryDelayMultiplier(5) + .setTotalTimeout(org.threeten.bp.Duration.ofSeconds(1)) + .build(); + final SpannerConfig changeStreamConfig = + SpannerConfig.create() + .withEmulatorHost(StaticValueProvider.of(SPANNER_HOST)) + .withIsLocalChannelProvider(StaticValueProvider.of(true)) + .withCommitRetrySettings(quickRetrySettings) + .withExecuteStreamingSqlRetrySettings(null) + .withProjectId(TEST_PROJECT) + .withInstanceId(TEST_INSTANCE) + .withDatabaseId(TEST_DATABASE); + pipeline.apply( SpannerIO.readChangeStream() - .withSpannerConfig(getSpannerConfig()) + .withSpannerConfig(changeStreamConfig) .withChangeStreamName(TEST_CHANGE_STREAM) .withMetadataDatabase(TEST_DATABASE) .withMetadataTable(TEST_TABLE) @@ -311,11 +373,9 @@ public void testInvalidRecordReceived() { pipeline.run().waitUntilFinish(); } finally { thrown.expect(PipelineExecutionException.class); - // DatabaseClient.getDialect returns "DEADLINE_EXCEEDED: Operation did not complete in the " - // given time" even though we mocked it out. thrown.expectMessage("Field not found"); assertThat( - mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.equalTo(0)); + mockSpannerService.countRequestsOfType(ExecuteSqlRequest.class), Matchers.greaterThan(0)); } } @@ -487,6 +547,41 @@ private void mockTableExists() { StatementResult.query(tableExistsStatement, tableExistsResultSet)); } + private ResultSet mockchangePartitionState( + Timestamp startTimestamp, Timestamp after3Seconds, String state) { + List tokens = new ArrayList<>(); + tokens.add("Parent0"); + Statement getPartitionStatement = + Statement.newBuilder( + "SELECT * FROM my-metadata-table WHERE PartitionToken IN UNNEST(@partitionTokens) AND State = @state") + .bind("partitionTokens") + .toStringArray(tokens) + .bind("state") + .to(state) + .build(); + ResultSet getPartitionResultSet = + ResultSet.newBuilder() + .addRows( + ListValue.newBuilder() + .addValues(Value.newBuilder().setStringValue("Parent0")) + .addValues(Value.newBuilder().setListValue(ListValue.newBuilder().build())) + .addValues(Value.newBuilder().setStringValue(startTimestamp.toString())) + .addValues(Value.newBuilder().setStringValue(after3Seconds.toString())) + .addValues(Value.newBuilder().setStringValue("500")) + .addValues(Value.newBuilder().setStringValue(State.CREATED.name())) + .addValues(Value.newBuilder().setStringValue(startTimestamp.toString())) + .addValues(Value.newBuilder().setStringValue(startTimestamp.toString())) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .addValues(Value.newBuilder().setNullValue(NullValue.NULL_VALUE).build()) + .build()) + .setMetadata(PARTITION_METADATA_RESULT_SET_METADATA) + .build(); + mockSpannerService.putStatementResult( + StatementResult.query(getPartitionStatement, getPartitionResultSet)); + return getPartitionResultSet; + } + private void mockGetDialect() { Statement determineDialectStatement = Statement.newBuilder( From 3cae5ada4f6e50b4763801232d2ede7f8193d0f6 Mon Sep 17 00:00:00 2001 From: caneff Date: Wed, 4 Oct 2023 12:13:29 -0400 Subject: [PATCH 16/32] Fix dtype usage (#28797) --- sdks/python/apache_beam/dataframe/frames.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index a74ccbba041a..7330feb4433e 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -4594,8 +4594,9 @@ def wrapper(self, *args, **kwargs): return _unliftable_agg(meth)(self, *args, **kwargs) to_group = self._ungrouped.proxy().index - is_categorical_grouping = any(to_group.get_level_values(i).is_categorical() - for i in self._grouping_indexes) + is_categorical_grouping = any( + isinstance(to_group.get_level_values(i).dtype, pd.CategoricalDtype) + for i in self._grouping_indexes) groupby_kwargs = self._kwargs group_keys = self._group_keys @@ -4647,8 +4648,9 @@ def wrapper(self, *args, **kwargs): to_group = self._ungrouped.proxy().index group_keys = self._group_keys - is_categorical_grouping = any(to_group.get_level_values(i).is_categorical() - for i in self._grouping_indexes) + is_categorical_grouping = any( + isinstance(to_group.get_level_values(i).dtype, pd.CategoricalDtype) + for i in self._grouping_indexes) groupby_kwargs = self._kwargs project = _maybe_project_func(self._projection) From 9ed9c059fec6e2520b091446898365ad976d053c Mon Sep 17 00:00:00 2001 From: caneff Date: Wed, 4 Oct 2023 12:15:17 -0400 Subject: [PATCH 17/32] Fix method argument handling for pandas 2.1 (#28816) --- sdks/python/apache_beam/dataframe/frames.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index 7330feb4433e..f51814e7f696 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -1388,7 +1388,7 @@ def align(self, other, join, axis, level, method, **kwargs): Only the default, ``method=None``, is allowed.""" if level is not None: raise NotImplementedError('per-level align') - if method is not None: + if method is not None and method != lib.no_default: raise frame_base.WontImplementError( f"align(method={method!r}) is not supported because it is " "order sensitive. Only align(method=None) is supported.", @@ -2580,7 +2580,7 @@ def align(self, other, join, axis, copy, level, method, **kwargs): "align(copy=False) is not supported because it might be an inplace " "operation depending on the data. Please prefer the default " "align(copy=True).") - if method is not None: + if method is not None and method != lib.no_default: raise frame_base.WontImplementError( f"align(method={method!r}) is not supported because it is " "order sensitive. Only align(method=None) is supported.", From 659b22089e64957f658b9c92f9e94c6b23708bd0 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 4 Oct 2023 13:07:45 -0400 Subject: [PATCH 18/32] Fix arm tests on Python PostCommit (#28740) * Longer arm timeout * Use custom container * restore workflow * Set up buildx * Fix container formatting * Use default creds * Finish args with backslash --- .../workflows/beam_PostCommit_Python_Arm.yml | 22 +++++++++++++++---- sdks/python/scripts/run_integration_test.sh | 9 ++++++++ .../python/test-suites/dataflow/common.gradle | 3 +++ 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index a77c4e96dc51..8be303a82d1d 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -18,10 +18,10 @@ name: PostCommit Python Arm on: - # issue_comment: - # types: [created] - # schedule: - # - cron: '0 */6 * * *' + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' workflow_dispatch: # This allows a subsequently queued workflow run to interrupt previous runs @@ -81,12 +81,20 @@ jobs: run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose sudo chmod +x /usr/local/bin/docker-compose + - name: Set up Docker Buildx + uses: docker/setup-buildx-action@v2 + - name: GCloud Docker credential helper + run: | + gcloud auth configure-docker us.gcr.io - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | PY_VER=${{ matrix.python_version }} PY_VER_CLEAN=${PY_VER//.} echo "py_ver_clean=$PY_VER_CLEAN" >> $GITHUB_OUTPUT + - name: Generate TAG unique variable based on timestamp + id: set_tag + run: echo "TAG=$(date +'%Y%m%d-%H%M%S%N')" >> $GITHUB_OUTPUT - name: run PostCommit Python ${{ matrix.python_version }} script uses: ./.github/actions/gradle-command-self-hosted-action with: @@ -94,8 +102,14 @@ jobs: arguments: | -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ + -Pcontainer-architecture-list=arm64,amd64 \ + -Pdocker-repository-root=us.gcr.io/apache-beam-testing/github-actions \ + -Pdocker-tag=${{ steps.set_tag.outputs.TAG }} \ + -Ppush-containers \ env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} + MULTIARCH_TAG: ${{ steps.set_tag.outputs.TAG }} + USER: github-actions - name: Archive code coverage results uses: actions/upload-artifact@v3 with: diff --git a/sdks/python/scripts/run_integration_test.sh b/sdks/python/scripts/run_integration_test.sh index 6ad592080ae2..5ac3627a0960 100755 --- a/sdks/python/scripts/run_integration_test.sh +++ b/sdks/python/scripts/run_integration_test.sh @@ -79,6 +79,7 @@ SUITE="" COLLECT_MARKERS= REQUIREMENTS_FILE="" ARCH="" +PY_VERSION="" # Default test (pytest) options. # Run WordCountIT.test_wordcount_it by default if no test options are @@ -169,6 +170,11 @@ case $key in shift # past argument shift # past value ;; + --py_version) + PY_VERSION="$2" + shift # past argument + shift # past value + ;; *) # unknown option echo "Unknown option: $1" exit 1 @@ -242,6 +248,9 @@ if [[ -z $PIPELINE_OPTS ]]; then if [[ "$ARCH" == "ARM" ]]; then opts+=("--machine_type=t2a-standard-1") + + IMAGE_NAME="beam_python${PY_VERSION}_sdk" + opts+=("--sdk_container_image=us.gcr.io/$PROJECT/$USER/$IMAGE_NAME:$MULTIARCH_TAG") fi if [[ ! -z "$KMS_KEY_NAME" ]]; then diff --git a/sdks/python/test-suites/dataflow/common.gradle b/sdks/python/test-suites/dataflow/common.gradle index 7766cf3a377c..a713b82400e7 100644 --- a/sdks/python/test-suites/dataflow/common.gradle +++ b/sdks/python/test-suites/dataflow/common.gradle @@ -144,7 +144,9 @@ task postCommitIT { } task postCommitArmIT { + def pyversion = "${project.ext.pythonVersion.replace('.', '')}" dependsOn 'initializeForDataflowJob' + dependsOn ":sdks:python:container:py${pyversion}:docker" doLast { def testOpts = basicPytestOpts + ["--numprocesses=8", "--dist=loadfile"] @@ -153,6 +155,7 @@ task postCommitArmIT { "sdk_location": project.ext.sdkLocation, "suite": "postCommitIT-df${pythonVersionSuffix}", "collect": "it_postcommit", + "py_version": project.ext.pythonVersion, "arch": "ARM" ] def cmdArgs = mapToArgString(argMap) From f30f6c5e22046e2bd603ec47a3b9e38c80510fae Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Wed, 4 Oct 2023 17:42:26 +0000 Subject: [PATCH 19/32] [Blog] Quest updated dates (#28824) --- website/www/site/content/en/blog/beamquest.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/blog/beamquest.md b/website/www/site/content/en/blog/beamquest.md index eea893bf8227..dde6376b4077 100644 --- a/website/www/site/content/en/blog/beamquest.md +++ b/website/www/site/content/en/blog/beamquest.md @@ -34,6 +34,6 @@ Individuals aren’t the only ones who can benefit from completing this quest - Data Processing is a key part of AI/ML workflows. Given the recent advancements in artificial intelligence, now’s the time to jump into the world of data processing! Get started on your journey [here](https://www.cloudskillsboost.google/quests/310). -We are currently offering this quest **FREE OF CHARGE** until **July 8, 2023** for the **first 2,000** people. To obtain your badge for **FREE**, use the [Access Code](https://www.cloudskillsboost.google/catalog?qlcampaign=1h-swiss-19), create an account, and search ["Getting Started with Apache Beam"](https://www.cloudskillsboost.google/quests/310). +We are currently offering this quest **FREE OF CHARGE**. To obtain your badge for **FREE**, use the [Access Code](https://www.cloudskillsboost.google/catalog?qlcampaign=1h-swiss-19), create an account, and search ["Getting Started with Apache Beam"](https://www.cloudskillsboost.google/quests/310). If the code does not work, please email [dev@beam.apache.org](dev@beam.apache.org) to obtain a free code. PS: Once you earn your badge, please [share it on social media](https://support.google.com/qwiklabs/answer/9222527?hl=en&sjid=14905615709060962899-NA)! From b221d804998734dc9025dadc0d8354562ca79c18 Mon Sep 17 00:00:00 2001 From: Shunping Huang <133698626+shunping-google@users.noreply.github.com> Date: Wed, 4 Oct 2023 14:24:30 -0400 Subject: [PATCH 20/32] Add job name and worker id to traceId of AppendRowsRequest. (#28729) * Add new functions to retrieve job name and work id from GCE metadata * Concatenate job name, job id and worker id with ":". --- .../extensions/gcp/util/GceMetadataUtil.java | 50 +++++++++++++------ .../io/gcp/bigquery/BigQueryIOMetadata.java | 34 +++++++++++-- .../io/gcp/bigquery/BigQueryServicesImpl.java | 12 +++-- 3 files changed, 73 insertions(+), 23 deletions(-) diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java index b853ab792e08..fd49b759fd6d 100644 --- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java +++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GceMetadataUtil.java @@ -30,40 +30,60 @@ import org.apache.http.params.BasicHttpParams; import org.apache.http.params.HttpConnectionParams; import org.apache.http.params.HttpParams; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** */ public class GceMetadataUtil { private static final String BASE_METADATA_URL = "http://metadata/computeMetadata/v1/"; + private static final Logger LOG = LoggerFactory.getLogger(GceMetadataUtil.class); + static String fetchMetadata(String key) { + String requestUrl = BASE_METADATA_URL + key; int timeoutMillis = 5000; final HttpParams httpParams = new BasicHttpParams(); HttpConnectionParams.setConnectionTimeout(httpParams, timeoutMillis); - HttpClient client = new DefaultHttpClient(httpParams); - HttpGet request = new HttpGet(BASE_METADATA_URL + key); - request.setHeader("Metadata-Flavor", "Google"); - + String ret = ""; try { + HttpClient client = new DefaultHttpClient(httpParams); + + HttpGet request = new HttpGet(requestUrl); + request.setHeader("Metadata-Flavor", "Google"); + HttpResponse response = client.execute(request); - if (response.getStatusLine().getStatusCode() != 200) { - // May mean its running on a non DataflowRunner, in which case it's perfectly normal. - return ""; + if (response.getStatusLine().getStatusCode() == 200) { + InputStream in = response.getEntity().getContent(); + try (final Reader reader = new InputStreamReader(in, StandardCharsets.UTF_8)) { + ret = CharStreams.toString(reader); + } } - InputStream in = response.getEntity().getContent(); - try (final Reader reader = new InputStreamReader(in, StandardCharsets.UTF_8)) { - return CharStreams.toString(reader); - } - } catch (IOException e) { - // May mean its running on a non DataflowRunner, in which case it's perfectly normal. + } catch (IOException ignored) { } - return ""; + + // The return value can be an empty string, which may mean it's running on a non DataflowRunner. + LOG.debug("Fetched GCE Metadata at '{}' and got '{}'", requestUrl, ret); + + return ret; + } + + private static String fetchVmInstanceMetadata(String instanceMetadataKey) { + return GceMetadataUtil.fetchMetadata("instance/" + instanceMetadataKey); } private static String fetchCustomGceMetadata(String customMetadataKey) { - return GceMetadataUtil.fetchMetadata("instance/attributes/" + customMetadataKey); + return GceMetadataUtil.fetchVmInstanceMetadata("attributes/" + customMetadataKey); } public static String fetchDataflowJobId() { return GceMetadataUtil.fetchCustomGceMetadata("job_id"); } + + public static String fetchDataflowJobName() { + return GceMetadataUtil.fetchCustomGceMetadata("job_name"); + } + + public static String fetchDataflowWorkerId() { + return GceMetadataUtil.fetchVmInstanceMetadata("id"); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java index ee64a7ab9ddb..0b5e063c0b5b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java @@ -28,8 +28,15 @@ final class BigQueryIOMetadata { private @Nullable String beamJobId; - private BigQueryIOMetadata(@Nullable String beamJobId) { + private @Nullable String beamJobName; + + private @Nullable String beamWorkerId; + + private BigQueryIOMetadata(@Nullable String beamJobId, @Nullable String beamJobName, + @Nullable String beamWorkerId) { this.beamJobId = beamJobId; + this.beamJobName = beamJobName; + this.beamWorkerId = beamWorkerId; } private static final Pattern VALID_CLOUD_LABEL_PATTERN = @@ -41,17 +48,24 @@ private BigQueryIOMetadata(@Nullable String beamJobId) { */ public static BigQueryIOMetadata create() { String dataflowJobId = GceMetadataUtil.fetchDataflowJobId(); + String dataflowJobName = GceMetadataUtil.fetchDataflowJobName(); + String dataflowWorkerId = GceMetadataUtil.fetchDataflowWorkerId(); + // If a Dataflow job id is returned on GCE metadata. Then it means // this program is running on a Dataflow GCE VM. - boolean isDataflowRunner = dataflowJobId != null && !dataflowJobId.isEmpty(); + boolean isDataflowRunner = !dataflowJobId.isEmpty(); String beamJobId = null; + String beamJobName = null; + String beamWorkerId = null; if (isDataflowRunner) { if (BigQueryIOMetadata.isValidCloudLabel(dataflowJobId)) { beamJobId = dataflowJobId; + beamJobName = dataflowJobName; + beamWorkerId = dataflowWorkerId; } } - return new BigQueryIOMetadata(beamJobId); + return new BigQueryIOMetadata(beamJobId, beamJobName, beamWorkerId); } public Map addAdditionalJobLabels(Map jobLabels) { @@ -68,6 +82,20 @@ public Map addAdditionalJobLabels(Map jobLabels) return this.beamJobId; } + /* + * Returns the beam job name. Can be null if it is not running on Dataflow. + */ + public @Nullable String getBeamJobName() { + return this.beamJobName; + } + + /* + * Returns the beam worker id. Can be null if it is not running on Dataflow. + */ + public @Nullable String getBeamWorkerId() { + return this.beamWorkerId; + } + /** * Returns true if label_value is a valid cloud label string. This function can return false in * cases where the label value is valid. However, it will not return true in a case where the diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 3d4565cb086e..04a665ac9947 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -1364,6 +1364,12 @@ public StreamAppendClient getStreamAppendClient( .setChannelsPerCpu(2) .build(); + String traceId = String.format("Dataflow:%s:%s:%s", + bqIOMetadata.getBeamJobName() == null ? options.getJobName() + : bqIOMetadata.getBeamJobName(), + bqIOMetadata.getBeamJobId() == null ? "" : bqIOMetadata.getBeamJobId(), + bqIOMetadata.getBeamWorkerId() == null ? "" : bqIOMetadata.getBeamWorkerId()); + StreamWriter streamWriter = StreamWriter.newBuilder(streamName, newWriteClient) .setExecutorProvider( @@ -1374,11 +1380,7 @@ public StreamAppendClient getStreamAppendClient( .setEnableConnectionPool(useConnectionPool) .setMaxInflightRequests(storageWriteMaxInflightRequests) .setMaxInflightBytes(storageWriteMaxInflightBytes) - .setTraceId( - "Dataflow:" - + (bqIOMetadata.getBeamJobId() != null - ? bqIOMetadata.getBeamJobId() - : options.getJobName())) + .setTraceId(traceId) .build(); return new StreamAppendClient() { private int pins = 0; From 3e22be4d310d93223d37620663bb270f71a7e23b Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 4 Oct 2023 16:40:08 -0400 Subject: [PATCH 21/32] Revert "Bump com.avast.gradle:gradle-docker-compose-plugin (#28465)" (#28828) * Revert "Bump com.avast.gradle:gradle-docker-compose-plugin (#28465)" This reverts commit 544745ca72868d3e385654a6cfaff444c012d00b. * Add buildSrc/build.gradle.kts to workflow trigger path --- .github/workflows/beam_PreCommit_CommunityMetrics.yml | 4 ++-- .test-infra/metrics/build.gradle | 2 +- buildSrc/build.gradle.kts | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/beam_PreCommit_CommunityMetrics.yml b/.github/workflows/beam_PreCommit_CommunityMetrics.yml index f044b154c0ab..bb44ca0b5464 100644 --- a/.github/workflows/beam_PreCommit_CommunityMetrics.yml +++ b/.github/workflows/beam_PreCommit_CommunityMetrics.yml @@ -19,10 +19,10 @@ on: push: tags: ['v*'] branches: ['master', 'release-*'] - paths: ['.test-infra/metrics/**', '.github/workflows/beam_PreCommit_CommunityMetrics.yml'] + paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts', '.github/workflows/beam_PreCommit_CommunityMetrics.yml'] pull_request_target: branches: ['master', 'release-*'] - paths: ['.test-infra/metrics/**'] + paths: ['.test-infra/metrics/**', 'buildSrc/build.gradle.kts'] issue_comment: types: [created] schedule: diff --git a/.test-infra/metrics/build.gradle b/.test-infra/metrics/build.gradle index febe2849ef56..f1ecba05f84d 100644 --- a/.test-infra/metrics/build.gradle +++ b/.test-infra/metrics/build.gradle @@ -106,7 +106,7 @@ task deploy { standardOutput = stdout } - // All images have the same tag, it doesn't matter which we choose. + // All images have the same tag, it doesn't matter which we choose. String image = (stdout.toString().split(' ') as List)[0] String currentImageTag = (image.split(':') as List)[1] println "Current image tag: ${currentImageTag}" diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index edd10ee108f6..0ca748e3eb04 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -54,7 +54,7 @@ dependencies { runtimeOnly("gradle.plugin.io.pry.gradle.offline_dependencies:gradle-offline-dependencies-plugin:0.5.0") // Enable creating an offline repository runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:1.2.1") // Enable errorprone Java static analysis runtimeOnly("org.ajoberstar.grgit:grgit-gradle:4.1.1") // Enable website git publish to asf-site branch - runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.17.5") // Enable docker compose tasks + runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis runtimeOnly("gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11") // Enable dep vulnerability analysis runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin From c2666e114971a4727919d9c95287a4ceaf486a92 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Wed, 4 Oct 2023 16:51:02 -0400 Subject: [PATCH 22/32] Add num_workers and save_main_session flag to auto_model_refresh notebook (#28777) * Add num_workers and save_main_session flag * Add WorkerOptions * Apply suggestions from code review Co-authored-by: Danny McCormick * Add back removed contents from a past commit * Add workerOptions to the import * Created using Colaboratory * Created using Colaboratory * Update auto model refresh notebook * Apply suggestions from code review Co-authored-by: Danny McCormick --------- Co-authored-by: Danny McCormick --- .../beam-ml/automatic_model_refresh.ipynb | 1133 ++++++++--------- 1 file changed, 529 insertions(+), 604 deletions(-) diff --git a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb index 67fe51af1253..9cbab0a14178 100644 --- a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb +++ b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb @@ -1,605 +1,530 @@ { - "nbformat": 4, - "nbformat_minor": 0, - "metadata": { - "colab": { - "provenance": [] - }, - "kernelspec": { - "name": "python3", - "display_name": "Python 3" - }, - "language_info": { - "name": "python" - } - }, - "cells": [{ - "cell_type": "code", - "source": [ - "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", - "\n", - "# Licensed to the Apache Software Foundation (ASF) under one\n", - "# or more contributor license agreements. See the NOTICE file\n", - "# distributed with this work for additional information\n", - "# regarding copyright ownership. The ASF licenses this file\n", - "# to you under the Apache License, Version 2.0 (the\n", - "# \"License\"); you may not use this file except in compliance\n", - "# with the License. You may obtain a copy of the License at\n", - "#\n", - "# http://www.apache.org/licenses/LICENSE-2.0\n", - "#\n", - "# Unless required by applicable law or agreed to in writing,\n", - "# software distributed under the License is distributed on an\n", - "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", - "# KIND, either express or implied. See the License for the\n", - "# specific language governing permissions and limitations\n", - "# under the License" - ], - "metadata": { - "cellView": "form", - "id": "OsFaZscKSPvo" - }, - "execution_count": null, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "# Update ML models in running pipelines\n", - "\n", - "\n", - " \n", - " \n", - "
\n", - " Run in Google Colab\n", - " \n", - " View source on GitHub\n", - "
\n" - ], - "metadata": { - "id": "ZUSiAR62SgO8" - } - }, - { - "cell_type": "markdown", - "source": [ - "This notebook demonstrates how to perform automatic model updates without stopping your Apache Beam pipeline.\n", - "You can use side inputs to update your model in real time, even while the Apache Beam pipeline is running. The side input is passed in a `ModelHandler` configuration object. You can update the model either by leveraging one of Apache Beam's provided patterns, such as the `WatchFilePattern`, or by configuring a custom side input `PCollection` that defines the logic for the model update.\n", - "\n", - "The pipeline in this notebook uses a RunInference `PTransform` with TensorFlow machine learning (ML) models to run inference on images. To update the model, it uses a side input `PCollection` that emits `ModelMetadata`.\n", - "For more information about side inputs, see the [Side inputs](https://beam.apache.org/documentation/programming-guide/#side-inputs) section in the Apache Beam Programming Guide.\n", - "\n", - "This example uses `WatchFilePattern` as a side input. `WatchFilePattern` is used to watch for file updates that match the `file_pattern` based on timestamps. It emits the latest `ModelMetadata`, which is used in the RunInference `PTransform` to automatically update the ML model without stopping the Apache Beam pipeline.\n" - ], - "metadata": { - "id": "tBtqF5UpKJNZ" - } - }, - { - "cell_type": "markdown", - "source": [ - "## Before you begin\n", - "Install the dependencies required to run this notebook.\n", - "\n", - "To use RunInference with side inputs for automatic model updates, use Apache Beam version 2.46.0 or later." - ], - "metadata": { - "id": "SPuXFowiTpWx" - } - }, - { - "cell_type": "code", - "execution_count": 1, - "metadata": { - "id": "1RyTYsFEIOlA", - "outputId": "0e6b88a7-82d8-4d94-951c-046a9b8b7abb", - "colab": { - "base_uri": "https://localhost:8080/" - } - }, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }], - "source": [ - "!pip install apache_beam[gcp]>=2.46.0 --quiet\n", - "!pip install tensorflow\n", - "!pip install tensorflow_hub" - ] - }, - { - "cell_type": "code", - "source": [ - "# Imports required for the notebook.\n", - "import logging\n", - "import time\n", - "from typing import Iterable\n", - "from typing import Tuple\n", - "\n", - "import apache_beam as beam\n", - "from apache_beam.examples.inference.tensorflow_imagenet_segmentation import PostProcessor\n", - "from apache_beam.examples.inference.tensorflow_imagenet_segmentation import read_image\n", - "from apache_beam.ml.inference.base import PredictionResult\n", - "from apache_beam.ml.inference.base import RunInference\n", - "from apache_beam.ml.inference.tensorflow_inference import TFModelHandlerTensor\n", - "from apache_beam.ml.inference.utils import WatchFilePattern\n", - "from apache_beam.options.pipeline_options import GoogleCloudOptions\n", - "from apache_beam.options.pipeline_options import PipelineOptions\n", - "from apache_beam.options.pipeline_options import SetupOptions\n", - "from apache_beam.options.pipeline_options import StandardOptions\n", - "from apache_beam.transforms.periodicsequence import PeriodicImpulse\n", - "import numpy\n", - "from PIL import Image\n", - "import tensorflow as tf" - ], - "metadata": { - "id": "Rs4cwwNrIV9H" - }, - "execution_count": 2, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "code", - "source": [ - "# Authenticate to your Google Cloud account.\n", - "from google.colab import auth\n", - "auth.authenticate_user()" - ], - "metadata": { - "id": "jAKpPcmmGm03" - }, - "execution_count": 3, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "## Configure the runner\n", - "\n", - "This pipeline uses the Dataflow Runner. To run the pipeline, you need to complete the following tasks:\n", - "\n", - "* Ensure that you have all the required permissions to run the pipeline on Dataflow.\n", - "* Configure the pipeline options for the pipeline to run on Dataflow. Make sure the pipeline is using streaming mode.\n", - "\n", - "In the following code, replace `BUCKET_NAME` with the the name of your Cloud Storage bucket." - ], - "metadata": { - "id": "ORYNKhH3WQyP" - } - }, - { - "cell_type": "code", - "source": [ - "options = PipelineOptions()\n", - "options.view_as(StandardOptions).streaming = True\n", - "\n", - "# Provide required pipeline options for the Dataflow Runner.\n", - "options.view_as(StandardOptions).runner = \"DataflowRunner\"\n", - "\n", - "# Set the project to the default project in your current Google Cloud environment.\n", - "options.view_as(GoogleCloudOptions).project = 'your-project'\n", - "\n", - "# Set the Google Cloud region that you want to run Dataflow in.\n", - "options.view_as(GoogleCloudOptions).region = 'us-central1'\n", - "\n", - "# IMPORTANT: Replace BUCKET_NAME with the the name of your Cloud Storage bucket.\n", - "dataflow_gcs_location = \"gs://BUCKET_NAME/tmp/\"\n", - "\n", - "# The Dataflow staging location. This location is used to stage the Dataflow pipeline and the SDK binary.\n", - "options.view_as(GoogleCloudOptions).staging_location = '%s/staging' % dataflow_gcs_location\n", - "\n", - "# The Dataflow temp location. This location is used to store temporary files or intermediate results before outputting to the sink.\n", - "options.view_as(GoogleCloudOptions).temp_location = '%s/temp' % dataflow_gcs_location\n", - "\n" - ], - "metadata": { - "id": "wWjbnq6X-4uE" - }, - "execution_count": 4, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "Install the `tensorflow` and `tensorflow_hub` dependencies on Dataflow. Use the `requirements_file` pipeline option to pass these dependencies." - ], - "metadata": { - "id": "HTJV8pO2Wcw4" - } - }, - { - "cell_type": "code", - "source": [ - "# In a requirements file, define the dependencies required for the pipeline.\n", - "deps_required_for_pipeline = ['tensorflow>=2.12.0', 'tensorflow-hub>=0.10.0', 'Pillow>=9.0.0']\n", - "requirements_file_path = './requirements.txt'\n", - "# Write the dependencies to the requirements file.\n", - "with open(requirements_file_path, 'w') as f:\n", - " for dep in deps_required_for_pipeline:\n", - " f.write(dep + '\\n')\n", - "\n", - "# Install the pipeline dependencies on Dataflow.\n", - "options.view_as(SetupOptions).requirements_file = requirements_file_path" - ], - "metadata": { - "id": "lEy4PkluWbdm" - }, - "execution_count": 5, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "## Use the TensorFlow model handler\n", - " This example uses `TFModelHandlerTensor` as the model handler and the `resnet_101` model trained on [ImageNet](https://www.image-net.org/).\n", - "\n", - " Download the model from [Google Cloud Storage](https://storage.googleapis.com/tensorflow/keras-applications/resnet/resnet101_weights_tf_dim_ordering_tf_kernels.h5) (link downloads the model), and place it in the directory that you want to use to update your model.\n", - "\n", - "In the following code, replace `BUCKET_NAME` with the the name of your Cloud Storage bucket." - ], - "metadata": { - "id": "_AUNH_GJk_NE" - } - }, - { - "cell_type": "code", - "source": [ - "model_handler = TFModelHandlerTensor(\n", - " model_uri=\"gs://BUCKET_NAME/resnet101_weights_tf_dim_ordering_tf_kernels.h5\")" - ], - "metadata": { - "id": "kkSnsxwUk-Sp" - }, - "execution_count": 6, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "## Preprocess images\n", - "\n", - "Use `preprocess_image` to run the inference, read the image, and convert the image to a TensorFlow tensor." - ], - "metadata": { - "id": "tZH0r0sL-if5" - } - }, - { - "cell_type": "code", - "source": [ - "def preprocess_image(image_name, image_dir):\n", - " img = tf.keras.utils.get_file(image_name, image_dir + image_name)\n", - " img = Image.open(img).resize((224, 224))\n", - " img = numpy.array(img) / 255.0\n", - " img_tensor = tf.cast(tf.convert_to_tensor(img[...]), dtype=tf.float32)\n", - " return img_tensor" - ], - "metadata": { - "id": "dU5imgTt-8Ne" - }, - "execution_count": 7, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "code", - "source": [ - "class PostProcessor(beam.DoFn):\n", - " \"\"\"Process the PredictionResult to get the predicted label.\n", - " Returns predicted label.\n", - " \"\"\"\n", - " def process(self, element: PredictionResult) -> Iterable[Tuple[str, str]]:\n", - " predicted_class = numpy.argmax(element.inference, axis=-1)\n", - " labels_path = tf.keras.utils.get_file(\n", - " 'ImageNetLabels.txt',\n", - " 'https://storage.googleapis.com/download.tensorflow.org/data/ImageNetLabels.txt' # pylint: disable=line-too-long\n", - " )\n", - " imagenet_labels = numpy.array(open(labels_path).read().splitlines())\n", - " predicted_class_name = imagenet_labels[predicted_class]\n", - " yield predicted_class_name.title(), element.model_id" - ], - "metadata": { - "id": "6V5tJxO6-gyt" - }, - "execution_count": 8, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "code", - "source": [ - "# Define the pipeline object.\n", - "pipeline = beam.Pipeline(options=options)" - ], - "metadata": { - "id": "GpdKk72O_NXT", - "outputId": "bcbaa8a6-0408-427a-de9e-78a6a7eefd7b", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 400 - } - }, - "execution_count": 9, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "Next, review the pipeline steps and examine the code.\n", - "\n", - "### Pipeline steps\n" - ], - "metadata": { - "id": "elZ53uxc_9Hv" - } - }, - { - "cell_type": "markdown", - "source": [ - "1. Create a `PeriodicImpulse` transform, which emits output every `n` seconds. The `PeriodicImpulse` transform generates an infinite sequence of elements with a given runtime interval.\n", - "\n", - " In this example, `PeriodicImpulse` mimics the Pub/Sub source. Because the inputs in a streaming pipeline arrive in intervals, use `PeriodicImpulse` to output elements at `m` intervals.\n", - "To learn more about `PeriodicImpulse`, see the [`PeriodicImpulse` code](https://github.com/apache/beam/blob/9c52e0594d6f0e59cd17ee005acfb41da508e0d5/sdks/python/apache_beam/transforms/periodicsequence.py#L150)." - ], - "metadata": { - "id": "305tkV2sAD-S" - } - }, - { - "cell_type": "code", - "source": [ - "start_timestamp = time.time() # start timestamp of the periodic impulse\n", - "end_timestamp = start_timestamp + 60 * 20 # end timestamp of the periodic impulse (will run for 20 minutes).\n", - "main_input_fire_interval = 60 # interval in seconds at which the main input PCollection is emitted.\n", - "side_input_fire_interval = 60 # interval in seconds at which the side input PCollection is emitted.\n", - "\n", - "periodic_impulse = (\n", - " pipeline\n", - " | \"MainInputPcoll\" >> PeriodicImpulse(\n", - " start_timestamp=start_timestamp,\n", - " stop_timestamp=end_timestamp,\n", - " fire_interval=main_input_fire_interval))" - ], - "metadata": { - "id": "vUFStz66_Tbb", - "outputId": "39f2704b-021e-4d41-fce3-a2fac90a5bad", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 133 - } - }, - "execution_count": 10, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "2. To read and preprocess the images, use the `read_image` function. This example uses `Cat-with-beanie.jpg` for all inferences.\n", - "\n", - " **Note**: Image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." - ], - "metadata": { - "id": "8-sal2rFAxP2" - } - }, - { - "cell_type": "markdown", - "source": [ - "![download.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAOAAAADgCAIAAACVT/22AAAKMWlDQ1BJQ0MgUHJvZmlsZQAAeJydlndUU9kWh8+9N71QkhCKlNBraFICSA29SJEuKjEJEErAkAAiNkRUcERRkaYIMijggKNDkbEiioUBUbHrBBlE1HFwFBuWSWStGd+8ee/Nm98f935rn73P3Wfvfda6AJD8gwXCTFgJgAyhWBTh58WIjYtnYAcBDPAAA2wA4HCzs0IW+EYCmQJ82IxsmRP4F726DiD5+yrTP4zBAP+flLlZIjEAUJiM5/L42VwZF8k4PVecJbdPyZi2NE3OMErOIlmCMlaTc/IsW3z2mWUPOfMyhDwZy3PO4mXw5Nwn4405Er6MkWAZF+cI+LkyviZjg3RJhkDGb+SxGXxONgAoktwu5nNTZGwtY5IoMoIt43kA4EjJX/DSL1jMzxPLD8XOzFouEiSniBkmXFOGjZMTi+HPz03ni8XMMA43jSPiMdiZGVkc4XIAZs/8WRR5bRmyIjvYODk4MG0tbb4o1H9d/JuS93aWXoR/7hlEH/jD9ld+mQ0AsKZltdn6h21pFQBd6wFQu/2HzWAvAIqyvnUOfXEeunxeUsTiLGcrq9zcXEsBn2spL+jv+p8Of0NffM9Svt3v5WF485M4knQxQ143bmZ6pkTEyM7icPkM5p+H+B8H/nUeFhH8JL6IL5RFRMumTCBMlrVbyBOIBZlChkD4n5r4D8P+pNm5lona+BHQllgCpSEaQH4eACgqESAJe2Qr0O99C8ZHA/nNi9GZmJ37z4L+fVe4TP7IFiR/jmNHRDK4ElHO7Jr8WgI0IABFQAPqQBvoAxPABLbAEbgAD+ADAkEoiARxYDHgghSQAUQgFxSAtaAYlIKtYCeoBnWgETSDNnAYdIFj4DQ4By6By2AE3AFSMA6egCnwCsxAEISFyBAVUod0IEPIHLKFWJAb5AMFQxFQHJQIJUNCSAIVQOugUqgcqobqoWboW+godBq6AA1Dt6BRaBL6FXoHIzAJpsFasBFsBbNgTzgIjoQXwcnwMjgfLoK3wJVwA3wQ7oRPw5fgEVgKP4GnEYAQETqiizARFsJGQpF4JAkRIauQEqQCaUDakB6kH7mKSJGnyFsUBkVFMVBMlAvKHxWF4qKWoVahNqOqUQdQnag+1FXUKGoK9RFNRmuizdHO6AB0LDoZnYsuRlegm9Ad6LPoEfQ4+hUGg6FjjDGOGH9MHCYVswKzGbMb0445hRnGjGGmsVisOtYc64oNxXKwYmwxtgp7EHsSewU7jn2DI+J0cLY4X1w8TogrxFXgWnAncFdwE7gZvBLeEO+MD8Xz8MvxZfhGfA9+CD+OnyEoE4wJroRIQiphLaGS0EY4S7hLeEEkEvWITsRwooC4hlhJPEQ8TxwlviVRSGYkNimBJCFtIe0nnSLdIr0gk8lGZA9yPFlM3kJuJp8h3ye/UaAqWCoEKPAUVivUKHQqXFF4pohXNFT0VFysmK9YoXhEcUjxqRJeyUiJrcRRWqVUo3RU6YbStDJV2UY5VDlDebNyi/IF5UcULMWI4kPhUYoo+yhnKGNUhKpPZVO51HXURupZ6jgNQzOmBdBSaaW0b2iDtCkVioqdSrRKnkqNynEVKR2hG9ED6On0Mvph+nX6O1UtVU9Vvuom1TbVK6qv1eaoeajx1UrU2tVG1N6pM9R91NPUt6l3qd/TQGmYaYRr5Grs0Tir8XQObY7LHO6ckjmH59zWhDXNNCM0V2ju0xzQnNbS1vLTytKq0jqj9VSbru2hnaq9Q/uE9qQOVcdNR6CzQ+ekzmOGCsOTkc6oZPQxpnQ1df11Jbr1uoO6M3rGelF6hXrtevf0Cfos/ST9Hfq9+lMGOgYhBgUGrQa3DfGGLMMUw12G/YavjYyNYow2GHUZPTJWMw4wzjduNb5rQjZxN1lm0mByzRRjyjJNM91tetkMNrM3SzGrMRsyh80dzAXmu82HLdAWThZCiwaLG0wS05OZw2xljlrSLYMtCy27LJ9ZGVjFW22z6rf6aG1vnW7daH3HhmITaFNo02Pzq62ZLde2xvbaXPJc37mr53bPfW5nbse322N3055qH2K/wb7X/oODo4PIoc1h0tHAMdGx1vEGi8YKY21mnXdCO3k5rXY65vTW2cFZ7HzY+RcXpkuaS4vLo3nG8/jzGueNueq5clzrXaVuDLdEt71uUnddd457g/sDD30PnkeTx4SnqWeq50HPZ17WXiKvDq/XbGf2SvYpb8Tbz7vEe9CH4hPlU+1z31fPN9m31XfKz95vhd8pf7R/kP82/xsBWgHcgOaAqUDHwJWBfUGkoAVB1UEPgs2CRcE9IXBIYMj2kLvzDecL53eFgtCA0O2h98KMw5aFfR+OCQ8Lrwl/GGETURDRv4C6YMmClgWvIr0iyyLvRJlESaJ6oxWjE6Kbo1/HeMeUx0hjrWJXxl6K04gTxHXHY+Oj45vipxf6LNy5cDzBPqE44foi40V5iy4s1licvvj4EsUlnCVHEtGJMYktie85oZwGzvTSgKW1S6e4bO4u7hOeB28Hb5Lvyi/nTyS5JpUnPUp2Td6ePJninlKR8lTAFlQLnqf6p9alvk4LTduf9ik9Jr09A5eRmHFUSBGmCfsytTPzMoezzLOKs6TLnJftXDYlChI1ZUPZi7K7xTTZz9SAxESyXjKa45ZTk/MmNzr3SJ5ynjBvYLnZ8k3LJ/J9879egVrBXdFboFuwtmB0pefK+lXQqqWrelfrry5aPb7Gb82BtYS1aWt/KLQuLC98uS5mXU+RVtGaorH1futbixWKRcU3NrhsqNuI2ijYOLhp7qaqTR9LeCUXS61LK0rfb+ZuvviVzVeVX33akrRlsMyhbM9WzFbh1uvb3LcdKFcuzy8f2x6yvXMHY0fJjpc7l+y8UGFXUbeLsEuyS1oZXNldZVC1tep9dUr1SI1XTXutZu2m2te7ebuv7PHY01anVVda926vYO/Ner/6zgajhop9mH05+x42Rjf2f836urlJo6m06cN+4X7pgYgDfc2Ozc0tmi1lrXCrpHXyYMLBy994f9Pdxmyrb6e3lx4ChySHHn+b+O31w0GHe4+wjrR9Z/hdbQe1o6QT6lzeOdWV0iXtjusePhp4tLfHpafje8vv9x/TPVZzXOV42QnCiaITn07mn5w+lXXq6enk02O9S3rvnIk9c60vvG/wbNDZ8+d8z53p9+w/ed71/LELzheOXmRd7LrkcKlzwH6g4wf7HzoGHQY7hxyHui87Xe4Znjd84or7ldNXva+euxZw7dLI/JHh61HXb95IuCG9ybv56Fb6ree3c27P3FlzF3235J7SvYr7mvcbfjT9sV3qID0+6j068GDBgztj3LEnP2X/9H686CH5YcWEzkTzI9tHxyZ9Jy8/Xvh4/EnWk5mnxT8r/1z7zOTZd794/DIwFTs1/lz0/NOvm1+ov9j/0u5l73TY9P1XGa9mXpe8UX9z4C3rbf+7mHcTM7nvse8rP5h+6PkY9PHup4xPn34D94Tz+6TMXDkAAQAASURBVHichP3Xt2xJeh+IfV9EbJs+8/hzvalbt3x1tQe70WgSAAnQDClySGo4miXxbV70oLX0b0hr6UEPkkZrRmuRHJrhiAAJEoYwDaIb7dDl63pzzr3Hn/SZ20TEp4cwe+e5RSq7+p4020R88YvfZyM2/rN/9zsMERAAgAgAABEBAADcH/t39Ut/MCABIBIRgL0OIhIRIgIRIgAgubPMO3MUoX1P5kJE1V0QQWtg6C9KRBcaYP+PSEQEhHChnQSABAQEgIgIQIT2eEB3O+1awMwJ/nxC00CC//8vc1bVPgJ/Ipo2ABARs6Kyn22/AAgIEY1k3O3Inou+D/Y3K9gvawTCRVEQuS4wrLXKtsi33bZKmxPJDgfU5I5V27B2Cd8OTVZkHhdE2ouydiBBbSjt+1Upk/vB/BVGMmgHnGzLDODMsPqPtfFAspgjcPK4iGvzHpFIuzv4MXCDY2/nYe0HAAEI0c4D/6sTKVENjgTMSXlFHnYc0UCVABmiF3c1Ydz40MooVB+qMas65GaWux4gkiZEtOCwLQRw6ERE8211LS8tRNDmW6yBD8HInqpbmIbY8UIrPCsef6wfXyAgI3QAIIsgK2QkM9DaSd53zQqPwLcEEQE0EFY0U8nCDiAB1qQO1a0MTJlFnAcUMjKD4MZ/RS5oIWxELTwy0FEa1DBOfgo5jJKdI4YgbfcJ7YXRCsl1zw6UQ4oXMZHrC/l54W9UcWrVNqBao9HPInBU5KYu1eRMrrNYURzWoWbpybUKyE2SCql2kLz47Im0Ipn6qNUmT4XBSuLVqDg025G9OPqORGsaDP0x5KBG6IjTH2PbSVBrl1Vz4JkT3fRZuZvneys4cNIwI7cqHKxRVG1WVw1ZYU2wDcKKIiwFoMEnICPSgCunIyKrtbDSUQjAABkBA2CrOp4BMsd/tdlee2deBEjEEMlCzoPeAdad4KYa2RNrbbBvCMxVGCIAMEMClhgJALVlZXSzYHWeIVYD7fSv/89+rBG5GQD0TbCnIFZIc+A17SVws6zqnZWQ66vnJY8Nx4U1jWwJr5oDQNWguFPs0NQEVaETERkgq3Wypp+djqxdzmCsrhsBfAvAmW019UzGgvAz58I5Ncl7QToGcx3XRhV4kwYAPMUiYk3otoNsFT1VOwmtfMndiqG/rMcSrp7lB8sKjFzH6iDwl6gI6UuuUGu/w0DtG4sL5i5DBscAjKzSR6ia6AYVEIChRTw4c8LJDo2OxNW2WCPJcXBl7dSGhyGaMWJ2grguERhTyCLP/0Z+9Kz0EYDVeL4aXrTWQdVpOzxeSismqYGPvVmNxqoJWZuZUB+Xumirc+qnrwiGKmvCWlDVvfwkdV/WXBwPGnO+QZmG+ssSn0Ukq+upisIqAgGOyFYnWgUXrJ1iIeJAzBz+aq32H/3twLLIxQasYhrN6FUidqNnNaZrngZS3iCqAYgANGntNQTWpofnYSBk9tJ1TNSuY8zIWjdq4+UsGqiEazkU681GNzLOdKumnZkJiMzCnGEFBqjo0A+Bx5GXniNjWuFdcy5WcPFS8K/qSwcuzwf1Qb9gNKxcyp5BNYVPABqcMVqHZiXBmuKtdcga60YoosYOq0aVtfB8c1bHxBmK5sW8MYe2gXXLFZmf+mS42YKPVgyl+ssZtbY1K9Kxk9fbwn4QrMeqrZvlrSkvddMCc5gb1pp2JyIEBpW5y6xBaZFMTpOvkqzzCqp56uRLTom4A9C2k/nW1k6z56wwaG0M7W211hZUXvwV8/pjV9w4g1qnaMFRwoXxRCIC7ZtcXRj8vZz/QLXWVoEC8AyP9U9o5zwQMYQqMOImVTVBViaDa5aAyouvjCBy6s9JcGWWgcOhJ27vXWG9LW7swPlADJCM2UtQ98l8c6GmdiuIr0qp1iEfgbH2KzmWZYCOL11YgBzHOF+ypqexGjE3sM6tQCsEz9++PWYK4IVGec+ZauC2SHFNNW8RVnmoenl311JZxY6VqnGMwLzbsaKjKvb1DSATs7pgV6LDiGs2eLD6sSB/oO945dza+9QvB9oaOwBE2uk7BjbmgH5GW22+QtXoohOWHpnviTV2KvMIkIDXBsTadVS11MCXMXZBzI7SsS4JRLQxCERC0CuTpzY2F6/jfqkrW/uNu4kdQj+RUJtfGTorwnXLdhGsCrCGkJ916K4LjsTRqW/ndtSaWtOkNaPMDicyrNxLE2IxHffidFJG31Nng3ioOWPSXdtLYGUaW0l/Cda9MWrOXiHZVWiRNtY2klUg1S2qLoDzkwDAxZjQAcO1zzRF+wFBZIjcdquunA3EaLWNKz8iAoiqM/a+tckH4AXqNYUNctcutjKDa7EJ86WdIXZeQCVuhAtT35zAvNCdiC2B+fiEH+GaO+mJ34LVaQFnTlrNhZVycghw4q8UGkPSNrLt4vxewyAjy40rto8lW9MjYoZcrUK0POx64Qfcwh2p0qoWqz7wSXRhUNz9nD1qcWdMCh+/gy9LW1zsdKUhLZPVLuvGzjWz+saLxKgCN6CWC8kHRKsGG8TrSuujuxR5htJfpkdsX4RVjlXTyPN+TQtaIjF6AgCZ0eyvqoNX57E7wrET1rBQYyQHYAepSjDoEj+V2nOdrIte12TvXWcrympkVsOQWL+5HWhy0RwN2t3Qm6FARMZjW417VPepUbQzvmoWoe2XYStHMdWEdKoP/HxzXfSZOh/5r+59AbXGeKpM/BWXyd6uzvUrP7tb1O0qrExFGzdwTcVqNvmsIfPsVHXFslLdEvDviFZasNJUcEThCLHS7g4iNcohx4jM36PGczUX114daxqhjkj06SqyQR8A67QyP0iGh4x0GVKlwoyRt8oPNU0F4IOFYJzy2pFkjRQ3jk43oTPprMZyagGt+nQhN9+FmhTdl4DMRWqrcUArTJ9W8m11t8XVy6ATpj/fDS0iAOgVR57cXK6NN9p0gxuzGgM64gRwwS+y2qyKpNpZ6uaSmYy1/tbEYBR0TSPZ3lZQrOYB1c6n6gh8BZyI9T8AAKKmD9AzXO0M52QAAoB28LHIsdqkAqBpen2ae3qw/OS8GXsNp0BruRDbM9TE0Sejq9/QBSMd0Xg95Nvm3WWnRV2a2ArJd5+qP479UFfBkUoEULu0U/KV6oNVcw1rJwEAECGgrvl8tr8rg+0ZF6imZ8gRmLt3pcfrkwOq36nGpeD6i0RU5THtMNiuX3AELjLlBS1fM4tesQqs0q4JrlID5BSBgzV4lWHvYnNGK3RGRKI+XLUqA0AiYnaO+ytWt6GaCqvBxxmgNdQ7M7QSi0sokzsbYWXW2ps6m94lVw1xIlgHq67vap1y/1pjzBCzs+XAWn3kZvuqiLyh4gRIVir22uTi+37G2hdDIPIFE2TJsqYM0TceEcEk7ldVTjWTPStWVloNgx6vaO+8gpJVX95fuvrtohPv54YJGFXM61T4xUvR6qe6mbBid5jGEhEgA9KeJmoNsoKpCBdqdzd4Jc3sQZ55EEyG0waKzYT2YPNE467FAH1I2SPFyxSppmfRKkHjV+Mrwje2o+VEBLAgRjchABCIISESrBSIVPzmbk4ezXRhSNALcsULr4ubAMgOGNQNa2/DOt1c6acVVWeu4SI8K/UfAACga+G1uv706MRaY+oGmLt07Ve40IdX4yI1qay6VmiHsQ7ui69Xvof6ePnbXRQjuFFAQOZu6rujbatthQg5KazC3JkBorqgs3JYfSJQpW78Bew/lp5qsTZnXTPP32hiYisT0ZCiV1Xo/JVKC62wEyEAdwl3G5ZAJG2igACrI+rnIDilie6CF+jKcZj3Ro1N4NMxrr9OHVRcUaUdnHlDfgSweuubVOlAZziad1QdVJnsTiDOu6prrsp/8a6tlznW7lgHUDUPV5FVfeUUXV04vq/1l6eDCgnk4r71bjuh+p77QIUxbCofAPwQ2Pa493buM0ThjU4TSK9ZPwSANsRJxrolBrZMxjGGLZmstDhYlFteNQ3yjXDq3N4OkQi0uWzFIC5XAd79RjRBeT9dqSZa329wISrHkxV7XeRKsLzsZpqLuPtP1WiaD1UcwqGEHLH6SVYB1dl3KxNtldouxHesjGslIljLHrhv3Mx5hSYNXrXSF+9Sh1pNBAgVlNHJz7d2BdarMvOjAwCgq7Pc0XaukfMxLWM7Dc0Y0+TI04q7GlU7ALUbE4DQldVlhepiv15X2hGx3qizlS4YIzWHtKJPcl0Cr+UulJbWJ5q5lyM/p4Yr0DGHhopRfD/ANm/FyfNdcCzlm1u5q/U2+0m2ah3WvIXKlPDMVrXKIG7FFFiNDXqS9WlPp838B6w3x4p21TR0XhSsvrT+EnSuQtlpFSJH5EAARBoqVodXX+hqzcw4sKrOrvJg3XBVpzhDwt6W7DiAK1oCXAk8+SGtcTMAEYmqDxVMfdKwrsKdTVLZ4dY3NCZlBdlKy1paR6xoHWoq3BzPTFn7ilqtTRKn9SrFWnWHfMvgIl+B9wncEXXZVxMSHHVZOXptZOMHqwzsu2475+ewvzr6fA1WCsuOjFN4tUbAynS9ACnvwXhKqyPVf3+RUJ20Lxy8ckylD6zYvM1jBeJrAF2g0LIMVQDCWu02eYr2sqhg4Fy7ipaZayf6Y5xgrEIC1zXjqDvCcgMGKydWB6xOdKN0kBGgdm6MA7EZS+ZPNO4XVnks6xIBkC05XXGi7d1ddT06UTnh2pwhWfJ0QjJmADq9VXP+6yrWNMrXwmJFcUZq5q09x5kPXiN4saD/tzK1jOZf8f8qrJi7uuGqtK5v2avodBem+psa7mus409xDm/d+7FOur+i18v+zFq62rTvS1tVNchb+eiJAXFVesYFcpFU1/PVqxHU7Sn72c8KAGSrVrVRWlUSuebMgbeQ6vaMMSjc5LOpT+bTJOTQ4sRUHzQvAxehA49R9FekSuExqCpMnRFoNbKXOpqlHd4CuOBmOlOdKm/Dj6sbt1VAeCUF1vKs7uQMLNdARMYsN5ATXcUdVf2/u8KKIGtDgBVFrQ7NxdJPU88AlqFXzMqLp1eYISedSpWBo7bqfJfErqYWwAVD1pv7VHuZ08kreGfXkDXDmL+NWbTkmmjzKibMY7GPCEAMrRqtKQV7J6dZai3yxzicWHFr2yUbNTAFqEgucWBZqqYpVl6OwKA2+Rzuq3nsZ4G/O1q+9DyEANoPk9c1XjsjIFQrCMyksUq58nFq96zcoMq2Ii9Uj4aaiFYzPW4OuLmHYIN3hrMtj7tX1aj61VfE5DBR00O1H1d65tpbI3iXP3Etqt5ZavcGja7BvUbD5gQLxAopXmqreqreNKehicjHeKAKNiIAOb6qjR4BIIqa3lztK75C7oh1bq+pM4DKlkJXEvKK0VeLOjDjWtSq0+sHoHWqEQz0ybr5RizMpkur0fISJHdkHVTgRtTTrXtPzltGJw7bPquf7aT3g1ybLOCMbrJXMHRkqyJqp2rDWL4idpWqzBfeNSR/WSunFZRWCueC2eoU1MUTVoQPlnHcLPT5sIqGWDVHvYnpPYG6MKu2X3R5V039WsPJ1toCgY2Dej0HFVycTnLfoikWqfcWoKYyyY2aw79BgWuxuawfbj+UK/asuZsCDVb5gvcevI7Q9SF3bSOqFj35yFRlezhp+EY4uYM1RK1jUdnT5LtGADWZO6MC6sPjpoO3n4zxg+B8BqiKOhyuSVu4VVdHrLwmB0E3AyqsYjXwFt8rKc1KJhUOakPmD6op+BV68KKg2lkXbk1ElRzd5az/pPUFw7F2DPnTa9qmIkxwQ+ksnKorPgLgZr+TCQIAc0FFEK+6eBeYyePN3xMdD1XyWJ3TlaJ2ROeNB3S6vFrJ6kspbHIfyCwt8oyLzkqoId9cnrmWubkP1veqW07Wd6rG12YTmF/uUo2UBQdU2DEKyPbb9qVmjFVThhBqJdhkKrOp0hpA6GJS6JgCnbnm4bhSmuS5ocaaVB/kVbCuGp2VpF6Fuzf0/U/oxnDFKnCX9YFYkx/xP5gBrYjDOXHeHvBWTsUP9eSGE40DazWVrTpCFL65vkHsFbyaUarPXy8uN1fAfnthqiFAtc7QDr5vnGkOA7xwRs1yBfCAqheNmFt7oiIA5iI/tUOs9C1sa2e5X1eaCbU2vaJnKwHUPtcmEVVXIQACZMwMle1Q7farI+Mv6zFa1XTWKjw9bqi+DgEv5NBXm10P5l0k44u2WwWymmtfb6qbpWiVD67Axk1rL4NV2brEksGcFxiBcwhXNYGLJiMC1JwG31Xj6fjjV9EKQKSdarYH2JynP74mQavk/Ky3NYJ+7nvmswxVK5jytrR2PeeA9WXQxqm22fkKzmi9AQSnVmoRUqrE6iRCta5UMnIcDPWWGvlWysoitBZPcCxoLobOCwNEs1uBu5rTM26kV6jXt6Cuyv0QXtDvK15szZFftTL9l5Ukav6WVXHkvNzapPPjWHtfN0Jr4jMjiL7pVawJHAwNFIicU26vsDJ5bBfd5HJLPjzmvKVQV+JERKSNHL1KcwNPNYvrQm+shvdHEgGCttFL5x/VMOqrRTx32gahNxaMFFxAEaz/zQCYn97GIHIzAW0b3ayo1I2b/dWV0V2V3Kx3CqNyLiutbgYXXQS2ukI1qH40fUiaavfy08DwsSdCN8ArPOSIFi6+XtXydVr1JoTx2KpBraz5utKujeBFVK5onrpercbFCMWYs1Zr4kW95ia87aphHkCwa/otnfkyBwtQ5u21SklYFq1nUXxrPIUS2dhKzdbyU7Am31pNgI8Je+WOZOHlpoanIDQ/2Wag5S9mDE1HUWbiWkI17VxR9DVtgIZQWPWxzlXuSGcuufCi740RKlVaE2sNtgNjruLfO4cLyIT00NGM722N3V/hEg9716RVZf2K4qrmHmKVz3SvV/Vv/XVB718wlpxfRX5kXm2qlfwFB9S/BRsTcL5TnczNZVfsX/NXGA9BW4fXmHLerDbmgzGlK/PIX1o7oKK9v+dtu4DTfVztSN31czLWCGgKNwlWnA2oT/AKwU412C5hRf0rwnNmqFuLvKLWqU4GVFWSr2b/rPdDNW3kysN8gsHalq4S1BqxHtG2Zdqrb9+1uklQp6gap10A1Kp9UiM2x5SWuhyW6p7QhVOsUn6lDLQaKYDqQr499RGpCbyaX37+umnvRw9sl8n10hhj5Gax1zaeGZFZ3e0miOk9EGiqxo/ckLr8kLkc+WSqKVGprDqvx4jcR2N42LbXZOwNn8qgIbfiH/xccxkG1xgCIiSbx3fGluPv2pSw3GML+Wvz8yJZ+Ex0pTRqNoDlL0f4Fz1ia0qYA1cBRH4CVzrolevV/oPK0LDTG9D99GUp9S9lshqPejXoJeKPctPnFXSuaH+yd0H4khs5RHmrsmq/+b7SMx4b1jAwhqEG0tVMAvSmEgEY9S5s+xEAkEgTugSvAbC5AQJpS3Lmsyt7RABysaGqEAkA0OYjagrS6keobghAzLhsrj++BMMfi64prjyOnEfFfDoObHfdncmbMk6MjhGhkik4drkwPABQHVYzb8zNwXrXNZokr7PsjHVhaAAy2zq69qObnA43WFV4+lG3zhNhrba/dhBRbbLVSGTFE6pfz+uFeocuzlCopFGnbK+VfLXkyglOvFgNvVNZda/DKQy7N1g1xGjULOnaLCY3ZPYvc6rLmHBAREgayVl0hibde/dRW7+HnIOEgKTRagyjzmrDa/4lBAIN5O/nceHJ1azl8GLxzfSdNE10BqsLzlnxomsfeHQ6nxvQmL61YanQuUoOjmepJi/bWu3bZTdQQ388YlV7X4WpEG0tgTEVnJIy7XQ2ghWDd7NoFVXkzsRV5q4gBMAY88C6cMyXMN9/5oXO6fTvjWK4cIBv8IV/fQO8T+J0JPiou5uermukLXVZmK6YdkREpAVWBKZtNNNaPnbDLbIkWs2sFceWkECb5cg2SoeWVhj5CDgikptW6NzeWtfQB6zrc86ebaqkzSw2R0NNIVZIBANp9PUdXiSGcR3fO9FDNWntHIOV+zp2cFUEPqFZ1xYmt4AVKVJVlI/1YcKq2Ssr5qqoUoUrK8RatmPldZHsoTZGr77oYkLyP/eq3F9HmXVevHCwvxdjTGtdk+bqbLcdqc6xGs9OWfM1AGlwYncAsjAQlXNX67PxGMjxW4267a8X+1YVR5oiOiRwaWhzTbKmkRGXsynAarAqfl9lNbyPr62B6MwKBzg3uwlMagxsUt21xR5B9urmpu52nupq4rR2e/UZ7PwGBFczapvukr727qYozDIk+Ru7s+uRf2b50Q6CBza4eCrYyeQA7eysVzyY2rh4WNCrduorqL3gZlUm1n/B5kFXwfAKWFeuU2uDwV8Feqd6nEjrtsYFPNlOG+AJfw83HOj64F3WlXaQb65T7uDu5t9aB5YIEN1ax0pYRITuf4AE2hp3hMhspM5ONa+pKx/IrSQhNzENrZmW+CVK2ql6oz7s8uhXyn5fHbkaFV3IirkRIjdCteHzDgB4ovUwtarK31Q7I9NJzImOagXOXl0a/sba5ITqVuD7Uj/Rf3Q2d2XJOBOsxotVrOG/+NKuDTUQ/5csCqrnzMjSkLZCtA1FdNuEW1E57nOWEBARCdAK3C5CRiGx6sp2fS05Brbt0y6uY+To4gz1lqJFp2tnFTfWnoXRF/LYZpvF42jNGPSDh4jgksCaKik4/8iBF4zjh8Dq0TcAQDT7770KTXJtqA+w7UG1pIGYLaomyyNQG1lcGY2aLVSjXqgUIDk8e15lUN3JN9j6XNWYr4DDHWbbUNUVOIvK//uqx4NV3Z1VIl/Cu5WAnMfpf/dBPa+OV+VpNZx1sqGqqGXMCrhCtuU0gx+HFjczAQBA6No5XozV8BIoZ1hUVGr1lzOnzEy2MKvW/1SD5jDqsOgCjE58bjaTE1dFUYioSXsM2oll/Vj0Q2VdJOdmVz2og6PqWPXOXsRbVDbTa4KmDvHAKhl5bK4UZa7MTfd/x6P1e7pgRk0uHjkeOm4CuJiMNVIc7KjWIW+AuS9r7rMRC658dBi6CKxqvC98U+t3bfRda10Gvn61Sjms/CE/92qsbXFRc3D8SURWxZOu8YJrvXFINKHPX/kZX8+21e1oo6S8biS7gJ385gLoM+4Ort62cSqwYiKn1ezmu5os3drHcph4PjMhUtMd26zKEndAMZKoPlWK6aJ+9EDxuoicejJjagREhHa5N9Z53k4MN5o1xnE/fhkJvEpC1cvVfLmTatErj3U3aq6/7nDvaNYogJxX4MDiW23TMPVz6yLyUvpyI9VpMts28AY6WIm4G1uGAoDarsrmFj4c5wNB5nvh5oibr34uVqzjw2PGHjJCQQDS2ulY1zr71k0yMl5eFTsyAQVA96ZqLAFjCGAel2FxAGBdHgsVUmaQ7MukZhCBMQC77WhN7lWays/4C+7VhbEEWEVK3TS3Ss4vgXCArcw7gBUGdy4Ueu6vq1fn29aYg1ysEd1MA3D7/FTfVA2tq2T0zqXjBt/n+sUJPCtYE2+l81hxR/1enjRIVxu0ebl5G4msR1gz0y9afpbinAI26t+FDKsIFPnpBHbZcZ26jNTMKDKs5GAeplTjWqiAau0OvyDVwMrcQCmjmD2JuPwokSbyMkEAZfvpLoHM7UNid+Wyjq/W5lvGGCIjAoaGMO3mPtqqZGdqVO5dXffXMPHlH6l2rMUSVOJmfjZ4jqX6mVYtMqpRu6Nc9EJ0NoNtqnc78BWjfqVtVq1Wprj/FT0OXqFS27Ia8znlAwSEjDkt5OC7cpIfb3BNttxYCbbePHKNdicanVyFhowEELVNEFfnVHdEBCIBXBipIGOOqJzRgd70NjF0Aqv+DVdbY8BckYiYk5fjXSs9czuTDTdJ/yrLhCC1BkclWmvrWiETnHPBOWOCMc4YcqZBo1ZGFsgYETFGjHHQhNzOZmYqQ1dTl25ymIhgDXernsGFjysc7DjLO8jVWebAGlW6UXLvHI5th91Y+AHxs8gw6KujXqnyVcX5qkJ3N1nRCeiH9EsDogY32kUi0ZGoOx1qgK5+81h13Fy7v7ODoZo5rArKVcrd2Xg1S916jE7zAAhVKjsa9kwCk4i3ZgGRX7pOdrJ6Oge/vgpA24y+9X+UUjVvirS2q9nMUVpr86/WWmmSWhtLxQCUMRaGYRAEURhGQRAFQSg4h4BAkyzQApRzLoiIkJGt5ajK1bwAqllZCdjHny6Mk51etVH+Eu3nTqupafexUqsVBH1iE3zLLiDEfFXxtgdkjZc9MtDbEDUurJuMF3SCw0s9gfwlx1RMVwnrS14XVD84f+kC6D0xO4vDuNB+LrHaQ+h07ZJGnuQmhZkFKObzBQGYImQyiXUiItCkwdWAGvCBVdlau9JQTaS1VlprAiBSWkmlEUBrJZVWpEmT1koRaa2kUlIpg0JFpElLJUmR0koRKSIA0lJqgEacdJutfrfbbjTbjYRRzCAEhgSaVElaA2OccwLiEIBWZtg0EUPGnJTr3i6Ck5gdBled5ATqe+fH78IoO2u3JkkEb+pRzWq3I1LdF83WHZ7DLcOt6DSXU6lFcNwlPNjJq9SaQqysuaovuFJtWPObLvo9dSL/z70ukHfVPKfE607eRSuFmN8qjaxtYztqL0FeKCst9d0nIHF4eqqJSJPSSlkgWn6TSmvQWpOSysAINUklldYladBaAWmtldIGjUprTZq0UkoTkDKRLQKlNQNCBK0tzxkAmC3kydpwIKUsZamknAZhmWUcMQ4CGQQ60GQmBihd5KA1IKcwAGCIjBEn0pqYzUe7YUXPGZXzawMBUE/2XBiPCpjODiQCt9ae6JURssis2JQcbNxlPOGhHQJ7nDXnoX5B0ySsobbGlNW4OUPMVxf4eB9Ahc7K79b/GSS6pNqXqH1ErbU1TAl8s2n1YzVZX+FRqBSYUwhA7umL5KXh7KKaTJ0yNH6c+IvPPlOktdbSlN0ppbQyjCKJFGnUWluKNY9DJQJQiAjELVUh5wztw7IY5wwJOOchIuM8YAw5RwDBuODmxRAZ40wwzu3WJoyIpCxny2yZLTVQGMWIjDSVSinTJIZaS1mWWkrOOJBGAsZQK8aY2ye6St/UctneC4aawCqIVGWaflCdYe1iG17KLvtSw2ltABwTOEh66kBnETjpV1WjFg2OiN2Yr7QTwJWr1p2h2hnWliB3NUMzVvXri0bzRRhdSLsbO9Gk170cTHPqhilU0Ux78cr4uKCAPMOS9dWNsQgMzVbylqG1k8bKfBMvjl8CczAB5IiaMYYoAELGGBMUMMZQMM4Y45wFXDDOGWLAOGPczFHOOeOcMyY4t941Y/ZXxgLOOGMaUDDGjW/ODHsyhmD8dNJUynKZ57PFYplnSymBMQVEQEprpTSg1qSkLFVRaMaACDnnSmhOmohpAm5IBd3WobWcvNfm3vyq2z6vDJj7vsaL5ELCZljoyx2O2mesX9iF78AbBubZZORrWQzIqhv40h034RBXblG7tYWUb3HNVrnwpmraqnnjZ9wFB3/1xEp3V5ijCtNVBM3WuGF1M3Q6xFbPMXc3++1FjVO7r7hz5ZrhM4aMIWOccWSMc8ZQMAbIiSFnDJEJhgyZ4Cg4BxcEMgJx33BEYAZ8BnsI6DeEcfEiRHNBZIiu3Aq01qWUQZgJEbClKOczBQSaSGkg0kohggZSUpZloRkHRC4DCjRpTVoR48Ye0mi8Om8K2j92eBkD68lZ4dVs1FcGcXVcycMCCejL3eGaWYn+LuDcT7COl0v2VosPnS9sdzJatRTrd8cauMz3nv/cBHIHuxZdPGPloyM/589e6FCVWa31EWyEa0UCDmtE3sjyxIsmaoPIjD4yc89mIaqQ5Uoc37dU3L5+gwA5IgCabClniGiZzizfMTFH5vddAIM931rkzGysY5G5kj90tQ5+3yKDTdd4sIar1sBYqSQXQgguEECTIq1ISyk554wxjVqRKqXUqBFRBaVSkmsbuTKz1xqZlUXoYt0AAOZhzL6itmZMeRtA/2ez0s7YW40xoR9cIlctZe4DaJW7T4IQ82ZFzfkwDXPjqX1hg8U6OUlWBkTVYIsEK9tawyqGBHv9L+P7evyoupG/qZ9Rq1rF0179ItafN5dy85Tqixc1gLaocUOENhpei4DU5oIhXjHoDmx00LAdmKQ+GvZjCIzZ+mDmaceZHxo0kOFK8M5P3QV2g+aF5QrJKj0LyDiBZgDEmOCcM64ANAERlaXUoTIxKamUZqQJlFIEWnCujXlKittZa29V1Wq6+2o7kljjpdVaAagNTDX0F1fzAFj146jXkl39flj3GzQhQxeItvnZuifrTMlajKC6mbtPjVNXScvGfSpqdwL31ii5ca3FLC4SKq7gr24ZrDRnxU4186xmkddVv+diJ0DzzpxONV7zt7GGgTvTm14EACJNG+TN3hW7yTGmITxrL5uLaCMTRjasiojA7EN/q+fOVdMCAVETWE/bJZycyQyaMaUUU8r6UMA0oo1REZlgFjA00VklpQaUQkitAyLQJsRkfR2sZaidbVPPeRos262ia4NtRV8HQT1iYgiSwC/EsMTnj6xsxBqNoRegCyMgoDeOLbNa/qxOdjTvFLSBGtR64gas1pTarb19XKnvFV6qRhhr+W+HVLTWkY9ouQmwMgeqnK6bRN6aMrVvnuEr46DiAMeZq1wMdevLhxcED5gpF3GGA6GjT6uyjUK0T1a1IvZPtNBam0chIkNknBt2dKG4+hYPhIxbBwa9dDSR0hq15pxLw9wAJnVEJJXDCJjnfJLW1h5FZUxPE2PV5GRBBLbFQCuKnhm9gs4Q+LJXHWc+vIc1vYHgRg9tssN71nWCcQOBhCiJSqXnpSIgqfSoKENr7iNn0A1EzFkoOBg5eHuUIfjgqysaomowTU7O+kW+M/WQ5EVQojdzbWFEld+yqtweq0kjOi+whuVagMny3MrtKiaqGyy2yhGs0jWttRh0ee4LM6ZSTGYmCIYINoJIYPdpRLNFqDMWV0pdjOCYQx4xxhCJMWRMIPpKOMM4HBxKLHAMl7jhRBeCI1JEzMSLrIvFCMHYpkor52QgIhKCVlqZ8KvWSilhph6ZJ0BY9XNhlBxp2VX2tW8qfq0FvK0qv2DZ+fY6FeDlXrsZESCWAOeZPCvkSVZIpV8ucwYwLItny+z1VmstDJ4tFuOs3EjChMHlNHq73xrEETP13eiKtrxt6ywQbxxo10nH5Gj+rOhlwzfgZ5EjtJVwmGu6m5+e9cnpzBqyqPK4qxiq41Gyyr5S3ZWfamdD9Wgh8MWu9taVavelw0RAJAQPHMd7i4V57kRE6zbUuJe7xI+ZWWicJEQAE2JyE4wUQ9SARJoZHBO5XZerdhs4Kq2ZQSgiMiYELxhqK3NGWiulgKEpvzD5KSWVVgrIZk7N0xlNAQqSsTeo7tPolUyF92+8PVl5AKb+1wxUXdlVgvfoQKdSwT5kWxOM8vJgoX4xXe7n5Wtp/GJZ/tHB2d+8vMMAn40mCpdf6YZnRfHT4fCG7r3RiP74ZPiz02ES8K+vde902pHggd2f0sKKKhihtTOqSWOa7cDg2mg1qOuy01dud0jfo5ozvmKlIK5eCirU1V5klXJl9PgfrDFji9uwbhzXo1D+3qyGh4pEAYTZU9Bk261zZOjT23FY2UJeZbKqQtR776ZPtrWmDWRntxWDcTYrNVrzlxmiAkCzAYrRAwRaa6k12YpVICDiDBC1Mll8pU0+n+z/TaQLyC8M8ZrL7Thfk2yd+KoPq06r0QMXvqyMeDMBEM6yMlMkNQHC0bJ4upTExJ+P57OiuJWmz5fFrVYzFfywoGku201xuZFKUqdK306SqVInhA0hNNGnw/lZrkNEjriRRruNOBGstuUuOKfMzpyVgIVtk1epNbeGISIoRUopzrkpaLzocrkhAa8bV3/03zqL3JUXrMz66joGZGgnjtM4hndr6HQ2qzuiNmbmhsLoOO2SMM7kQl+fx40NxGqVLAxtOIQ8PNEiEam2PR6iCSK4XtSMPMttruFgzWAExhhDBiYuD9oSu5tohps1aa2Mc6Q1aCCNNZbxD8y02sr/BRfPcErJDzm4GpsVNPgxc+LzysuVYEMm6dks35tn+8v8+TK/mcYa8EVRPsqKrSD42qB7fzrth0E/bq7F0X62vNPpXG00/uRsuFB0NW48Xi4Zsq93B52Ac9BPZvMfHJy/2UhezmdLpW42G391e3Cr2+LMrMujWqOdJ1U1zprH9aYSQFEWk8n45Ox0Op0mSYLILu9eardadeuxQsQF9e2MGPQ/ei70hFdX4vaCPodXv6iuyc+TKTqTtzr9glkmLNUwZ1YhmuyQUwJ+dMERKJrV6wx8N4CIbLmxScjVXDbmGlRvMtSnSQVy5meI+wxSKqmUIs21tqkBBBOd12TqWWoPEAULP+fLO6+0GkbyITHw2sTGSiuLvVLfgC6GVJ2liealmhZqoek4108X+cez+b3Dk6+s9UeKSlWOpD6dTIjzb3aai4D/fDonhC+U/MHx+Vf63QDhZZZ1gvDr/cFhtvjXe/tNxt/ptglxK0kvp8nxYnmz2eQIZ1n+o9PJ80JLwMtJeKMRBojCZE2AGDIC5986Kw2cp6K0yuez8dnRdDKcj89OTo7LUgVJIhiWs9Mbd95tNdvWG6gpblzVHjXYVuq/ApeNDVUAtiIj8FEj8H6pw/HKZKjF/ZzkLZz8FBAm2ehs4Qo3zHozoLX1ujz5+1VfjKFG4D6rROgvYAK0WgMwby7bhyvWnWXTDJsaJRKMG4s2YJyAFJk6FucEASEyYIwIbBWVUkbFO1CRBmKakNlyXnebCl7WiDRSXrHNV1Q3+GnphpAQl1KNc3mwKM6LcizpSV40uXhRSAHwm1cvZYg/G082wnCsqRFFqeDHUj8r5PU03Y7CoywPpLqWJNea6UFRTkspEa63Wv/w+lVQaiELjTjRuBbHaUSfnJ0d5vKtXvtUqqcno4NSDkT4rU6cCNZLEs44J1VqnQq2mYTdgHNnr2uCIs/zxXg2PJ6Mz8siAxbEcXrl6g2ttdIglUSZjw+fNa7fFSKoOMKPBlgoYW0hHq0ypEEsWTbSK9cgiz2H8HqMz4q5fpzHpfeuagreFCw7T8H+4w0rj24XtvOmmLml1lozEzdyKqHiWyJyQVoX6vMa6cKLuTCK/cgYmBQooiyVUorsniXGLjXsTsYGVUq5v1KEIeeccVZdx6ooqpQ7IFAtC14Tra1Xssay5RJNOJf6PC+Hhcylmin9YpFnSsecPVpkzxbLd9qtsZR3koRINxnrBPyoLG7G8d1BNyf6s9PhdhJfTZMux+Ns+cs76w2OPz073w6CS1Hw+Xg0kvp2q7WdJjpb/M9P9jbSxq2WjgVrJqkupy8X+bv9LmbZ3Wb6YDJ7kKnNNJAlU6DH2fLPjk97Ufhmu7mbhJtJ0Ap4isCWk3xyfHz4YjqbIxNRnMg8V1KJMMrzXBPkpQwD0dVKFpkBKFWYpAofiLWUhFd4VSmfobP6BIcqfIL2gZzWACCP75XsSC18BhVtgz/AkKbQCOgD12h4DsF6sgZAFdTJcgmgjwZXNmulBtBW56DddsLYqG7WVOkytLF3v1WYMRy5iyAoIKW1sVBtbIvZ+aC1zmVJi3khNVssgY04E4ssWywzpZSWstlu7Wxt97vddqtFYAIz5KkTELkLJeZKj7JyWsqlVO0wGJdKk95IotNc/fTkfJiXW40kZmwk1X5e5poEw6KUSymjMPx0PDs6PV+7uvtut3W0XA4Q16MgQfpkPOmF0XYcxYi/82Tvdr+znSbdMPxoMmkwvp4mcyn3hmNVyLjbTYVYRsnNgD85PN6N49NMjYpiLU4I8c9Hs0BTP2ncHgx++/GzQZz+rZtX8rL4i+mCR/GlNPro+OQjxot8eVvQr241OpBPxpOSRKc7KJcLqSQRLPNseHiU5wVyzrhotdt5USpdZ74Vf4nsGFUj4o+oRxRW8+ZW9TtF5wvoTa2f110VF1vDb8Utq13QfS/cvr8e4PalzdIf+zUCaO0XalS8XNnYzPkSFT2hjUFgbZcRtmIQg6kX8QEG43wwxjmwqu5Fa9sYIgSbONCkkUgr/dkXnz978kwpmeXlcpmVZQmAUqoSIGk01zc2vv7BB++89dZg0I9EoAlM6RYBHGby49FiUcqCKNT6YLmcSxkxcaR0qakjxHFe7u29vLo5SJLk0TJ7NptvxmEvDDnnL2QZhdFQqs2Qp+s9xvl/Op+cTGedZvP9XvuHwzHXcq4UAd+bTlRRrsXJN9b696ezo2V+pdFgjA2i8Eaa/Nns7Acn56+1Gg/my93B+neuXCatn0xnqMVGI2VIf/Ly6JsbG3d77YSx0e7Wz84nT+fLUqujXA7K5R//xV8U52fddvKd3f6vv7aN+WiumAiTBggpS8nEYrGYTmdZXsyzPF8uAUBpKKTsdTobeU5Nb15axemw4j0IN96v6D6bgrH8WLcNLQq9f+KDfQiepGq4ruH1IjqJEEB4uxacpqOac2R6YJxDi1MA+8B2RN8Td4qzPQHcNPLTjlbSW1iFwBnachPS2t4XgTPGEKV57hECEGmluOAAwETAkJli3TSKv/bee99+/71QYLZcaFmGYaiByrw4Pj394uHTn33x5J88evwf/uAPdi/vrvX7nW43iRMWp0eK/8nZ/GQpsdNm3c41rXdb6SBOzsvys8kEcsmD6Ea78e3XbxZaP1vKe0/2W3H8lUu7PxsOl3lZar0s85SxD7bXP5tM9ubLsdK/sr15UpY/n8wWSv/1zY1mwP+XJ/s7ofilqzdPsuLH55OlUkEYbUfBh+ej02X+br+FAf/sxcH6jau/tt7544PTf3F4+pev7WrGQ17+dDj8eq97o9V4vszeKPX96fgP7+3dubqNoH/2+PFkPNHD03gyvMKKv37r8ru3rz1/8UJK2eoMgjAOWcCV0oBFUY6n89FkEogAkGnSSqv5bCGVRO72jXMPr6+zoBkIH4r34ZaqsMIhEcGFgVfY0PmfgATKfOswfbFsqrJXa34/uYCa8EBHRKNAnaGADswO38ZZcfmsGt5NyNcarc7WrNL/jke9rWubYeLhPh1qVAZHxl31HwEoTcr4Ssb+AEacE0NyMVBEFjbbrThWOJxOJxzCVqfNGe+t7bz/3ge/cvjyP/zHP/mLB3ufjsZRFERJHAjOkJEQDalDScsghptv7DX6nz8tO+uD1zd7b7Waw0Qj4wutn07mo6xAwTe31iLGPz4ZZlINWo290SgbT1+/ujspJSPsR2JbiJOiGEupFf2d7bV7kzkCAuc3eu1lkf/7oxOYL5uE37117c1ee/Li8NNHz9/ovnW91VwsM6XVsNC3e53z5bIdCEySHKh4cTxpNG/2eg8nsz8+OT04n6go+s7Wxh/d++Lw44+bctmnrFku/8Gvfvfa7ubRyZkIEuRqMpk2GzoIQy7C9a1eb1DOFvnp2VmZZY1GEnFEwlmWl2UJvmbF/TFFyh6aKwxnQ0s1a69W8kI2a+n50cfIvQvv4vn2GBeKX3UD3I0qdJF/ErZN4tjISh3f4AKUAJ7MwNsnKzwKjvap7sVVN3fUT7WPAE7BGA1vU53IGHKGDEmTVtrsUmviSSYWZbP4WmtSSslCU5o2tNbHpyej4ZCIAPlsWfbXtv63/5u/8Xe+/0EvjXSpymVeFlKWCpVKApZyakzOwp/8UfqLHzQoG8/mP/3wi5PzaYzseLGcKV0w1mmlJVDAg1Gp749nu61mNwzm86zf6dxstRZSf3Z8dr4selH8cDh9r9O+0WpMNTyYzv7wky+y83FPhM20MRCiuP/gW1uD1zvNo6zsJMm3blxZSyJkvMgKPZoQgx8dnrJF/vx0xBgfxLGIo8eTGUh9cD7++fODnX7n8lrrf713//H9BxvL0eXl6QZkf/d7X799/dLZcEzIC6lyqaXWIk7iRrvV7THGo7R56dKltUFfkZZlQUBKKhHwOEnSpGkItD5YdVDWIy1+lGsaGS06/UhaU8GHP8wPzluqxanI+TIWMzVAmFONKWlDjT7s6ljScJsxSohcqY25LlttguvgCuzq8wEATMUeVZEsHzNykclVZWGSSQatZqWUTxeZ85BzYEiuyomIiLQGCOLm1sZW2mwWZTkdj4UIGs02D2ISjV/61rf/q19+v9+OC6nyLFdK5YssL0ogEEEQhEEyPmn/7E/ig6da8P3ZfCHlWhTebSRrUYCE+eHx8Wiy00rbafysKJ8u8tuXtjd6ncfzvET+j+7e2mq3fvj0YLfRuN5uX0+Te8PJ+eno9uXd/8PX3w04+3S6yB8+SvNlnjYbYfB4vvizFydXO+1C6d1mM8iyX+y9RGDv9zvnWfbnL4+HpfqT/WPJgv3R7F/9xedBFP/GzvpXOw18+fzwP/77wd5nV2LVbkZ/6f23X7t5czKZZ3mZlTJJG51Od2NrhwXhIl8OpxOplSyy5XIZxvGg12EcNcE8K0ql02abB6FFnTPT6tyJF1iqBlX71423c3TAsaP5T5N9KqXVzn6sL+LjwsVdXZVDPQiD+9r/apaEK10jUpaZHeTRwdZAyVycwFbemeXv6PS7bZq9FgBW6+YAwCxPNv4+AXCzWIQxk3GVSiljJBCgJptFsLaBJtKgyGa9GCadQbcoRsPzuNWJWx3ORZnnRCgl3X3znUYj/eGH9754fjJfLAVjXKowChWRJCBkLFt2P/qhvvra6Pqbe6XaXOs90jCV6mw4ub6zs9lMZ0WxjthP4stpdH883R/PfvXapWEhrzTSPzs+H8Qhj6OhlM0o/Pzo5FYzvbOzmRBcbiTHUuk0vvz+ezHC6WKpBH+a56+V8nQyHc6z9NKV0enZv77/bDMJi15/VJb//smLv3p1a7GYH7caj6dxJ4nf3N0a51mM1B4eru0MGo1oPYq/8vbbZ+enWlEUJ8V8UZRqs7++WCzCMMnL5Xw6ZWQWIOSqyIi04EJrUlqv9bob69uMcY8LeqWi2dNqhVTnTtiojVvg4egTKgL0lmC1SLFyVKy3tMpnHrXW4fLeC5FA5ySZiCZaRV7FjKByyVfAjz5XSWSr5SurwrXbTSi0hgn5yaSJfATU+PLovHizqQgjsLuEABn+RCBuwlCMEUMCWwSq7R6inIfRzuUba+vbwBgQlHmGZu8dFhBrXL319s7ulSdPHv7gF/efHg6JSAOVpJVSoDUxZEyw5w/ak9Hk6t3DNKFlWU7nDOnF2SibzDe2N2Yo1gi+GE6vd9v3xvOzvHw+nl5Pwm6aTjQ0EQ/mixKI0uTxIlsMJ3/88Nn1tf5RUc46a41AnGbLH7w4FEm8EQanZdlJ40+OT5iOwjR9Y2Pweqf1cDT8yXiaPTscb/S/c2nn/mgssvmTZ8//jFQrTXZvvrb4eLPTFDHAW6/dnExGRVlGcTNMmlu7V4MgQsYG61ukdZ4tVWegSZ+dHIVhNBgMnj2dSSlLSbnUrUYzbTTBaS1HiBVisF5O6iHr/1S5X/LHw0pxI9aQYpx3unjNV10lspVKzgy1VxZmkZnZVttSofV2NBGzsSJvWyKYQiY7Xep+FIDzlMB13J7J7PNAEMyWoOjMl5olXrMJqgirJgIyRSGkAbjLNzKw6U4FpJQ0G0Bo0JwxHoSNIFCl1EpiEAAAaaW0lnEhyzII0zff7W3vXPr43sM//cX9WVYKzoHZjf+AQDEmhsfd+bhYH3Ru3u5t9I6VHp+NFkl4mhdX0+gP9g+CIDgATNP0pCieFvn/5+mLnAevt9KdJJqXRab18nR4ab2fav2VW5f/zcf3ck3fv3ntvbXeVhp3AvE7v/tHrV6/tTb41lpv/+x8/9GTN7a33u20fvL85aX1XqIndH58NtqEnfX1JAl2d+/tHd7qdw7m8x+fnq+laSuCdhRtrq89e/6Mi2iwllKWvZwvNrZ20iRhnCMXiFiWeSnLOE0n08lskbMglEVRSinCYPfq1bTRrFtZjlIqrBBUQEQHniqcVOGsFj4iAl+D5QHnrghYjXi9ntmPexW0XHXRBAOosoA26WPylszkOc0P2mhZ53FXTOlor7KCtc3jOyOaTOBCE3AG9ikNjkqNLLwv7xIRIJAxxjQAkS5Lu+rDGMIMGXAOQGSynVqZzLwpSWYMgyAMw6gsC600ApRlQUoiY4wzpbFUvNW/9Je+uXHtypXf+dOfPD04C8JQSqlLCWADEIKx2dFxfOX6r1zZ/pOjs5dPnl+6vPsP7t4aBLwTit/9wY+//s2v7AXil9d6XY6//Xs/uHn71q/evLSZROfLbKzkj5I444It80QE5csDJnV+5fLpMr/daYYigDC8tbN5rRn/7pP9NIoHzVQge3e992I2/539I5EtMUlUWeZET6bzF1mOefbpwcmdy9v9gxdqMh0TXt3aIeRKY7/bHQzWCHkUJ0naCIKAALLlYjI+f/r0yXA2LfPl/t7L44PDIs8E50zwt95//627byPjX+I3rGh58t84twbrP7n0v/PcV71vMFrRee7GOoSV+tHavaqleStzxrpDfomE+c8kkJSDhMsTAXM7Z6IDZtU4e4/6dYlIawI3cQAdRRH4HXZqxGlX5ZkFyWaxMoIpgzfAJLNRjrkqAuOAqLV2G+5ocy3SWkolpVREKIQCKlSptVKIUmsNCEEYJg1AXkjc2bn8N7//S2/cugKchWEYhIFdGsiYGGyEb7x5OJysJfFkNIHz07VG/OB8kkt9KQxZXo4KdX50KoT46b0nSZ595/rOw/Hsf7n3LOIs04BRPJ/M51n2yWiqW+13b9/49u7G4XT+py9PPj6bRIO1uNe7M+iCkh8+3jt/+ODl4ctPx4uvbq31QWWHR+XLl51G+nKZX+40Xy7meZErhi/Hs9cuXc2v3N4b50sl86I4GQ73j49m2ZILxjiTspiOh2cnh/svnh+eHi/L/PHjJx/df3h0epoXRVGUCuDt99771b/y6420WaGz5q1QxZAmSISkfeVXVTHngt6Ol6C2kIicQQguUF5zhWHVzLW39HrfO2fevdEE1kmy8we9c1Zz4oyuJ22YFarl9cyayWQdcFvN5EnQGwcOmQjKWZwVrP288QFWH19AxhHNZk4m1VnNVcZN36T26XhZFEXAAwh80Ax4GE3nhVaSEZVFXpRlqbUQIgwEY+kyW66tbf+176azP/iTvWeHTAiBTGU5aJLd/vuba6WCvWUxB7z19a/95ntv/mj/6CAvf+/B81/6+vvv7qwLwf7s+Pzw5OybX33/g8u7+5Ppp0enaRz/9MMvysn8O3euPpjOPn70vNVIw36/1HSp1/7dR3vroeg1k79398bz4eRXblyJovAXxYTC+POz4U/PR7fbHX0nGvW7T8az5+WLcDQuk1heuvrkxREFZ4Moij/49otO74vpYePk5PHJSXl4MCuKa5evdjq9KAgF50UhgfGk0QrTZtruPnn6ZO/Z86NFxqNo48135O23KYjspgw2W2O1qkeJ2yTuIpDQ4bHyeIynYcfx1bOoslj9BHCQAq8w60Reu4r3rEVNv3vYoAM3ObSRS1qBrUElAPug4iq+5U60H8nRpJ2E7gD9ZR4cY0wp5cwHZMg45yCYLv22ZJaABWOMM20iZHZ/KEVEoHUhS8pRA0AQBoFggne7fQIqsyyKYqVkWRQSSGlFHCLOVbZoN5pvXrt8eHSuSsU5E0JEb73f++Z3Z8ienZ0/G03V6fHVW7cSxiTn/+r+0ybAd67unGT5jw5Op5NJ7+zlla++9YuT8zu9VtpI/69/+vO21v/Hb78fCnYljf/li8PtOPpLV3YDoJv9zheT+Z+dnK8R/+nJ6IeP945Ozr/91u32G+88f/T09NHTq/1eS2nJeLC+dnw8/MrG2no3/eHpmItoXGSUZTwKW7NpHqSfdbaXhyezklGmP/zi0eP9l+04EVHEBUdNQoggCKRSs+l8OZ/P5gsVis6tOweNtecvT+9ubtzptokqOxIvQAds0bNTxwR2IaTVZl6Tm/p861v4wn9vC1TDXvtYMyQu+PJOnVb+u6Er4VyiFXSu4tU8/tXu+Wkeu+J8M9srH0+rB9LQl8LXwhb2mNXb1Btt2Q/ttiO+np501VOzSZgmUqQ1kSJSWmktzaxSZZkTLAo5lZRGQRqIRpxEYSiVIq2zMl/mWV4Wi2w5G48GrealS7vRR18sdC6JSsQrN26/s7a2HgW/V6ifffrZ37515Vt3rs/y4loz+eG98d2ttTQMbkbBt9Z7P1jMkktXbm5v/+zobA30tVbj58PJBsCTefafHjzd6nfY1StPXhz/84/vh+2mPDvvdzu3+90XPNBad5upSkKlJGXZdn/wl1+7EnP+dq/5//rki48++jxsd7qN9O1ea6PX+6Nf/OLlF/f0u+9/9fqVGxH7BZMPT+U9GRXNHb7VvDzo6pP9h88e6dmcsmXQ7QftTsCQFblOGjpIZKK6V2+xze3ifHI0nf+Hx88Hb9weRKHN3VUJnosq2EdtnBFKDn3eUvS1oeji7WiTiXXMEUF9/Z0lvi+p5/8S3iISjrpNjGllJXj9Kmj2fbJGJHFCRNRoV4m4xaquYS7HQKuzBJz6tu/rFnFVr4cm1RkwxhkDxkyg3tZ6AjBAYynqUmqpSVkzlDQpIk5EQEqrZ4viRU4fjo6+1e8ca70WikEkEo4NHgQRB025VjNZtIke7b/Ms0Igiig+275+XyT3Hr+4FIpmp1X0Nvfj1h8enH3+4vDNy1vrN6/96MXx/XvP9Hj62nqvc/Pm3icP/vjpizIQv/34RdBItzrNNrDradR/6+aPXh6dHw05gztrnTc2+3+ky0ir6dHJV9fX/vLO2t1OQwL93//t758+2fvub/x6X4jf+/mn7//lb8Jsyc5O4/7g4XD06U9+/o9/4/stpSGKQtAfH5+Fm4PF6dns8/saUW/vKKX3gkZvsD3OKNrlMeK4VFkpRZr0moku5HyZdS9vnyslCorb7XLvxdPp4hcn59+9tMVtTBG8H25QZiOXThGjz3xXbpLRcuTYrgYRYyNcMOEMImrq/kvReeHl4gAgvCdk5kTd1PCAM61htr3Vii1m9+02WsEbl4gAJururQUfob3QCB9MZYhS+4fQGLGgN8G1RSFppXnAiXHidstvk+w0u0WadZ4AUEq1LPWS2EjqAtnPRrMNwX56OryWxO+t9Q6L8koieo2dU2qPqMiam+9/99ep29dJ40qURCI4ni3+4vlBvMh22o12EAgOT8+Gs9PzvNv9+2/d7At2rxUfzxeT4fh779zeTqK73VbIdh/PF//0j3/y1TduMYBbzcZrb9w6/K3fPz487dy8cqvV3H3j1j/9wQ+fffT5L/+jv3+yyH/nsyd/9yuv77aT4WL81Uubd/utjzb7f3R0PnyxH4fRP/zG+yWp//Hx4//13uOT8yEfrH/3+uX/9PnDPwQaPHkM52fq/Q+SjTX56Gk/ChcZUiFnaTvrdZL5rDw4WQQJRK0tmg9Ph0W3y0s1PDjevHa50+0eHpz8OE06YfD+xqAGn5ptZkfcfqtJGVPO6l+LHLt7j1PmGvz41gKcFWdhnUJrnEU2RLpySnUWgHsON/qZQC6uUKc6e1Gbf3W7c9U4b+VQcCit2b/abeC7Yu448mbVe+IMGWMcmbYhU1TalYuA2amXkDFiTCOQtIs7takc1ZqASGsk6gbIQA9CETJ8Iw2/PuhsCEFF8e+Oz3/rowfPxvlBRvtK/D8fj0927m5+8K0fp4PPWPy9zfVf3Rj8tWs7t0JBp8O/fm3ne7sbf2Nn/VvXdl6enuHZ2UYgbrebVxrptSSmh4/6ef52v/One0eHk/lbvU633fz86GxG9O8e7k2z4u7Na6zZ3Gk1vhhOf3hwdnNnKx6fqPm8FQVxt/l/+fnnj18eX711c1jKw1l2Y2vj9x48Pn3wRSvkDycLhvyDOzc//dnPijDavX6tH0bdRhw8ubd8+UL1ByyMmUaxs71/eJyfDyGKIc+ns6wcrHciwc+HmaIiiUPBTk7PdbsVhsHJ+ShupE2G2WL5o+cvJ3nhuIB5tDhWQmujGV+35lEAEIB23jmRSZsjOq7FOor8WPuPGsgZDB4EUB3nj6zhiLkIwCocaxetx49s8BI8z5nWo9bVkmp65ZZQY3W7i1BNibglm7QaRmCCm+0btNa6MFtCutYgIGPM6HRt/CSllCliMUvsEQTovuAfdNtzRU3OTws5HU42onAjivTp8OnB8Rvdxgfr3WA8+fMPP8ey+DubvQ86zX/6F18cLbPrafLffuu9S5e2/ulPP/39Fyf3zydf3dncefv1MghGhQwYy5ReRDEUy2eHxzd67buDzr85PD3Pyw+uX7qxvX6rlV4dtH9wcJakabvdRsCvbHTbcfCLpeSt5uPHTxDZB+u9FodZEG698VYnEA9H0504utrr5m99Zbl1WYH+83uPu2F89a03eLMzHo1+609/ErQ7jIvpxra6dpNzoYgKWYrTo+z0PBusi80NkGq+zKnTjjiGAStns7DfbbQaAaLotAVgpmHz0hZjLOS81PoCHlbYyBR5uPo7qP1nQeyeweUvYQOT3uV3cNK1KgtvIzjPujIQ6hjDGvz8PgYrYVj7lVuupMEsbwezkzdzS+bdXKP62lhyzYSay1WDuw2VoSvy8wUo5ktmQ7LAGeOME2Kp/SI552khklnbSaSUIqXIxURNcNRs6LyUshkEy7Jci4JRIZ8uSwqj97vN6NqlPaV+cjJaD8NvvXY1L+XhPHuz29puJE9OR/uzzOzs/+2dDRTiu9uDRIhJXkrGkzReiwJFdK2RnM/menQ2fnz/3z15OVb6H9zYPc/ynz0/EFm2P8/u9DrdQPzw6Yu3ttcJ4MFoPpplc2D0ze99zJOPT4f3J/MsCPibb/z4fPI//OLeh8PpvclsbzjS62tTJf/oT3706cPHv/vDn015fE44kTrttN5O02+8/0H/8jU4OysOXs4fPFbjGUax7K2p6aIoqNWIwpPjJbH48m5jNhnPFhPk0Ou24jAWotVrM6XGy4IYC8PggsazKMMKhmbuu81T/ZA6Jl3Vy+Doxptz/lcXQAfvalV4ALdxQQ0bmgg1OSIkYU09gytnEtdhbWKV9V1KqFYsooEYcG9rugZVvFiDOxprw+zuQGYD35oZ6tHsthdlHJHbBeCklXI7PwMCQ8Y0gEkjKa0lUWgr87RSSmp9nqmXpWA8OMrKlLMn84w3kmYUbiXxRr9zI4mutxp708Wnhf7gK2++M+g8HM22mumVmL34/PPi8ub+bLkWBn/tzpUfHp3/+uWtJC/WABln24342Xj2jd11CSR/5ftqOFwq1Q+DTiAixGBjcF/Rdl7uTedzgGBtcIrQEOxfPno+kXK5zMIo/vruhiL66qCzPzx/+vBJp9//W3evbSfxOM+CNEpGIxWKxt3Xfv3aztPheKrViIoG8J3r14so3Iyjm7duHokAGbvUbEqVv5zO2OmIQGLajHudxXRWnJ6VRyetyztJuz1bLEe5ZL2WOB8WgGvNdO/F4fpr1zXRIi/7UbjKSMYvd0Co+ezmLQJUxW0+mujRWSMjuPhaiZK6uVABwwcuAXws3T70lXlG9VPBX9/qc7soA2yWvE6HtVv6r7WbRjWvfsUA0P6JzbXr1IIJAGbzXUQT6zSekKkrZEDc1tvbpW9aa2kwqZRWsijL+WIxXMwl0W4ScaBBHHHGp1pfX+vtpDEHWA8DTrop+EkhoSx/fbMfMDzI8i/Gs4OiPC3yAPF6u1EofavT/tpG75Pz8c128x/eusw2N39v7/jH55P/8ZNHEeNfv3YVrl7/0dl4f5H9Pz57/J8OzkINnTQ+zPI7vXagFQ0n393dvNJM3lzr/sqV7a2ylM/3N8KwLGWWFXe7bXj4sByNOeK/f7T3clnkivD8nB8c9ZrpRhB8Z3tjlGf8yePZeDSaL/74/sNPDk9CxtT5SC6yNhNfu3ypV+QciXZ3eKez1u50Ww3otIMb13ivv7G5HirZioM4DHcGPcqzRVk2Qw5Ex9PFy9lcO78CaqNvi9PIu0CWusyxhggv2G810iX/zYUIY3V87V42OGWVradxz9JmxD2KvYdUXQ6rq6FJ5+g60SOAezpRlYKqgkc1WvXtcwFfwkoidur54xCRM/MIGrMdmV0IL5Wy9a8EnDEE0EorqVSpZFkWZbnMs8lkfDI8O11kPIi7YVCSjhljyP7qRv+vb6/1w0AA/M2dtW+v9wFgLQq+sdE/LMphXt7ttbZD8UvvvrW/feXfPNr7ydnkrChOi/JsWfzgbPI/PdzvpBGQ/snJsBEGC8Y/PDpLSV8PxC+vd9/tt7+10f+DDz8Tx6e/sbv+w8d7y/Hk1y9vlk+e5tP5J2fTm83GdhB8cOvqZDwr8+J2p/Hw+GwqFaSN33jn7tfWe4M0+rdfPM7ORpmCb9y5+c217j//6N5plpcoFBfrg36nmawF7On58NHey7VGozsanp0cP11k0dkZFAUuFtn5+dl4mknFsiwHGGfZQaaS9XVotobz7GWYXrt9c8nErZvX1pJwt5X0IqPlSVtr0ngw2i7zBk2g3ZJij0BFpDwF+kH1IKu7H1jzklcIjZzbpZ3i95lCR0/kaqAIiDlr44KfZC5H1RsCZvi0Os6jHLyn5VEI3qZkrB5YRbtDpMcw1qEJlW1qK+cRmV1YrEkTSU3SuvPMhKC01lLJLM/PxqMnBy8/fnjv40cPh7liIiyJSsJc67lSD2aLmZS/fTw8XGQbcTSS+ufDWQDw7qDz09Px77w8+1dPD59N51/ttb/a75xr/ZPhBJXeisO3+u3//vbu+73GP/tPf/FeFPyf3n3tb1za+Md3rkxL+X/7F/9OHx0nnO8NJzf7HQj47d31QRzuNqL/90f3//hwOMmzZVG8Nmg/HM8ejmdrnXbUTg9m8xzZMon+/cf3UZaPTs6fjmfrrWY3CbOz4+DJoyxtLAjTfuefffYw23/eyhYTLl7MlrLdUwHb+/iTrFSD996TcfRk78WSC9XvQ6PByuzs2f6Mc0piNpnAoyezrJCjcf70ebn/cjhdCCFUUUitx6UMhIgCAWgtf7LLZv36xYor/PqJilSdF1FB1J3wqsNxwQdy7pV9TIyJZnmqskhyH5AINAnXjlVnitzVLKCdIvbGgHHKHPfVjNLKgibfW5dltwu0XKmrPxgdrGsbhJvtyBEAzENqtNZIxICQiAEA4xRw0lpqtciWo/nyfLY4H03yxTJoNvnmtZ5UgWDnUv3obPi3Lm19uH/8Vzb7gygYST1WWnC2UOogL+fT+W9e2mgG4miZ/dbDvXxd/t2rW7nWv7139K9/+tEO0d/5/rc3InG33/nnjx4zJS/1Or/38PnfuH3laqcRhCxot17rtvZOFSP65huvncwXB5P5u1d294Lwn/34I0a0zPPtNBZB8NPTEWolFvP3d9Y/Ozj5leu7x+fDnxzsiyh6ucxn09n17fXzkyM1WNdpqvLia4PuT4r8cHJWdgd6mV1VmifBpw9eBpyJZmPv+Ly9vk5PH8oXL4CFEKei39WnZ3g+Ut1OO42LvGDZfCFl3EzLZSbiYDmdtgIhlWbEpFKzotRE3EACcXVraXRekUafcXLf+iG/wCweQnVN6N9UC57qUSF7G0ddTrVaBBIQWIBaiHl6rkGzNkuICIhh5bG7s8CWz9VMZnKumdtEh5zpXQEYa1tMoTufyO6IgnaFMJpdwH1ZiKFWAFSKFkWRLbNlKeezxXQ6L/OCNDUQR+Ph2WLRSNIbSbjfbIzzfBAFz2aLv3Jl63/67Mnv/+hn//vvffN6K/39e0+ePX72j3/tO39yePrtrcFoNP3dp8+3W43rrTQGKtJkPjwLArE/XbQbydvvvrWVxHf77YPR5D/uHyHnbDF7rZUMorC52f/5yXC8zJ49ePy13c020D+6tPGHQL9IgkfjxZ/uHd1KwmSr/zuP9qjd+3g4+9EvPr3/bG+xu8Nff+OzJ88PlTx+/mLz6mWxfamUcO/ho881KME7a2tFo6tPR/zzTx/tPWev3YZGozgbrbUa5Wx2Pp1yTXpjhzY2BGJZFuF0BFKFyKaAvWa82Ntb9vsyinvdjp6Nz4m2+p3FbNHvtYHg5XRxs9dhjPmxqVwiAL+U0iBgBbl1LluFKdbR45BX94+8ljdHO4YCJG2fJu/oi9zNhL9abSG8jxSAe76XW+RfD7JahW51/mqzrVaoJ2a9Y+jSAMb2dqf4qJPT3eDMU4EImqSUeVkIjkzKpSzOx+OD4Xg8nJgqJSklaWKADEEvs/zk8EW73+hvXum0302jaam6SfTZePb+dP7Weu+jZ/ufn493uu3rW4N7z/eeHp3s9rpfnA5Zv5PleRoGL+fLWSEL4NMo7SNdWuv90cHpkdTH9x78xuvX39teezlfPp0u7rz37pKxo0UWM3yr1wKiA33zt1+c8Cx/vxn/6rVL7W73Z/tHyeHZt7cG7/Y7R8vsoNduBuK/+85X//jgJBHs8qBbIF3rtr9zZefT8eTFcDzY2exzHCKPkuR6HAaCnutd7LYkIZUSAoECjj75JGykFCdquYT5XNy/B2EMl7YDpeXOjiAqigKUZlqzIOw0kjTL5kW5jONCiLVOGIaCEANe3wUJHJV4Zeuw5FVijfLqoKzDl8APtz/Gesyv8i6aZ2lbyJstUQ06tYsYESIKchvUuFbaIuXaRKkZumAJz6wYJjCbjqH18WstqJgXAc2CDSRA9E924ASIdkMHIw8fOK7NV2SMCeSAUBQFap0tF4UsT0bnJ6fH0/FUliUQMG4fOmsWN4HWw71nZVHwN74yThJkPOH4TqvxdJn/4dn4q93WndvXP53Ov7I5eLoUjStXPjse/p/fuPlsuvjTk/FiMn90Nvz69nojjjd77Ww+eTmZXY3CX7u0MSzLB430//v08O1e82ojaQveiG79wcnwlqT3uq1L7WYzyvqM3Uzi33zrllTlH3/+8MXp+dvbGxvN5NHZ6Z9//vlH9+5f3hzI61f+3cHJ7OQ4DgTb3u13msvx+Wd7T54/fNhKkmgwkGkSZhl9cVT2252y2BRx0oqXo5E6HeciDDYGRRgiF4HgshHxQBAXGkAUWUFQHJ9AGLSaDX52StMJIuYcIlmWp6fTZnuWxkkoGpwFgsdCcES0WUpPJaveTy2O7YbEbe+4aoZqupjHNuPnnSTnU1viM/FMUw9l4oh2h1BA//gd24R/9lv/wSwPAkeKZqGFybdWUX1to6jmxuaxG0ZBMMadUnahAADSimwUyHl5aLfXNUYCQ0TG7N7J5gwCG9Qsy0W2HE8n5+Px+XQ4GU9kWTICJcs8LybTyWQ6zpYZAgRC8EBwzrTSSmsETJIYGUPOB5ubV+++HW1dL0SERP0oWgBqgLIoIiH2lvl6FGrGJlJdMo9u4Ow8K07GU57ElxrJIAoXpbw3W76cZ3c7zff6raXSPzuffDGe//1r22tp9Mnp+ZPh5FK/myAKWfz48wd/sf/itUGn3Ujm58Ozg/3xwUss8ka7qRtpORrHHJMyu7Q5SDvNaVFmo8nLg+NZpuL1tZjxG1e2H9z7gqTSWqfNxmKxWB8MeuuD8fERaRqNZ9liITgvEHnS4GFIUi9bHb51ydS/8bgxyTIkAMExCBLBaTZfMCaSeBBHyzxXgI1QpEkYiGA7Dtth8Ob64Eavw6CSf50eyVdlQAWyeuiwBuNqcQRetBUvHukvaBCozVIgswlZdbvaM8SIRD2F6FW4Jm3dbfsYdq38VvBuXtnZhUCgmd0s20ZMNWkCrbQmIjSPmHHARQBVm2GGeJXSRk3LUipNUpbz5XIxm56en52OzmfjUZ7lpSyLIi+yXJYlaQ2c8UCIQACilFJpigKRNpIgjgCx0+5sXrume1tLHp3m+XC57AueBmEYJS3BR1KlgqdhECBsRkE3jv7j4Vkm5ZvN9N2d9eeL/Omi6IZBrmQC+o1+eyMKfnY+3B+OSqW/2u/eOzn6n/f2954/S89PppvrOk17XDenk6/K6fTzz5+PJnme72yubbZ4JJqv3bp67/GzIgXB+Vff+WpZSk302cNHe/svl1mezbOgEe5ub1watIOrW6dn40GvNRpP2kn/+u2rf/6DH22u9+OQt9bbx6dqluV6mXOSLYgKQjocZk/uMQRgjHe6zWxJpUTGIQyJCyjydrMVrK2HYSDPz4QGHcWzKOJpQ29uta/spmGAXnnBRQiiz6pXg7USbyGXOvIj61m2ArQ/y0HTRF5tZZ790u0pV8UQVl5CSmlvbc+ynoqpoWNIiKjcE2CV1syt4eREyu4Uh9p562QedAB+vZBSSkqzua/WpZJKSgD7pFqyTwWlvCzHs+loOs2zLC9LJaUs5Gw+G03GZbZUZSlLaTcE1YQAnDMRCMa5JI0auGBpGvabKQ8FRvHm2nrjymt78fpPDs4wf9nncEmNh6OzrNnLm102HZdRkwAeI1zdujRTellmAOzW5tY0n//kYP9yq/F+v/v49OiTg0NdZO/221+MRgfTKVvOscgeB2EjYleXc3r+LEK8EzRPz44H7dbbb91+vLf34ejs8o3do9Pz/mBAUr37+vWP7j86OD67fuXy6zevPN9/eT6f7e7uChEwzsusiMPg6s5Gq9GMEAJkoRBa6vffeW9UFE/v3Rv02u1mmhfl1UtbGpDOhtcubWV5cXw+4owrpbjAbjNdZDnMRpyxrMg5w3wx4QCMcyoWej5UaRKSzhfLQlOcNhZhYzIcz5b5WVZ8bXv9jV6bu/gLw2oLLefgVkuIVuJPZC3EFUh5C7YCcVX4bBKe5kTz/ENy+ScX90RnG7odmEkTkVBaItgHWDGG2vvfWnNEs7rKPT1bgdmdnuyGvyanaUpFAQg0mOL2UpayLAslVVlmebbMllmeZXmelfksW0opEUiTlmVZSCmLcr5YTheL5WyuitJuCWqeO0vE3Tp50sC0RgTGUAjOuV1HH8VxpxF306TdaoIQFARs4/LPoXPvi/tXhnuXIiXzJSjZYkzIZXt5Op0uWFEGAZMaFiePG1HcScJ2q4MHI5VlrxeZOi7uPdBRlO6enRXnpyeCzUaTq4MecX4+HCPATKrRaLLe771x5/Ynn95b3+xvb27MlpmSspHGh2dD0nBtd/unn37623/4n5SWTHApi2VRPHr+8uXhyecPnvV6nWKRkVJhGm31exu9XgDUabeOTs+nWXgyWzz47LN2I+m125e3Nw9Pzg5Phy9eHAVxyJCtd9vHJ2el0o1mYz6exEkcp/FwNI0QW712WUpcZKVWQnCOoMsim+luI02bDS0VIgSQqXJS7D39Qspn08VfvXn562tdtgowS4dWQ37JEmRwURrHrUj2gdeV1epVpFf97mrWojDhKyRbSufKUip/yDNoCQACEEWgNBBo/yBRbeo2NCmlbHDVTh2zolKhL00iAAClZVnKoizmy/lsNpvNZ6Usy6KYLmfTxVIWWZnn88VimWdaE2gt8zKXUpZKuwfOIENun2pjFijblDworQk0IhcoOAejLBiLwqCRhJ00bjcbcRxjHMet7iLtHcyzG7S4kuhyPGZaFYTEWTfBSOswwDBOwig4G8+CfK5UTjLMF7OlpkajIUqZRmK+mE6OD3Y2NkaylU2nd65dns3nnWa6v/dyssw5MMZYp9v5gz/5s2aa/Nrr3yLS//K3/8Novmg10q2NdRHHAYNOKI6ns29+8FZAsNDqeDjKtC7yXM7noGmt32k1kkYjPj4bCeQ6mz1/efTe++932q1///v/sdtMMylzonkuMQyhKN9+87W8KOaT6cvxZDbPOt1Wr5UEoJeLZRSF/Vaz12k1kvj0fBgGQVEWy0IWUqVRAKRJSS1BE3CBXKqCyTjSfDaccv4XrcblNN5JogpQKxissaFLRroPFX3W9h686MmjLdp3gDZWBVK1nY2PbAISahO5r7xkxoSxQIgbl98WUxLZBymCW1ZhlvdWZgq5DTdriUopZZZnk+n4+Pzk7PxkPh7LItdS5UW+yPIAmdRqkRfLotSlVFIVUhERN1vbMcY5E8yn7BEQTWYTNSEQR+QcQs5LAKlUzMNmFMVR0IzjOIyAYCFlg4s0TuaM3e6kR8t+fjoJgiDP1SCNoyDoJMlkOonDcGtj/fh8FAouhBBax4jLPG9FkSiyzfUNEYTzZSnl7MEXD7bWN7a2t8psqaRaLLO0kbxz906/1/3wi3s/+NHPVFn+5vf/0nA4BMYODo46ve5av9PpdS9vbGitHj58HoRBwNjZ+agoihfHZwcvj5SU/UH3vbfu7G5tkFT7B0dT4vcePSkWs1/79V9f73V/8eM/DzhLk+Qsy8Mw+OLJ8wihncbns/k8y5IwjMJwrdduN9Px+QgB0zgSnIdROJnOkGG71Yzj+PR8hJDzKJRKkgYlVZrGhdRmG2ulyuloWC6L+WjO4ni6u0FpjK5g14MTfbWaNzXrxOjIk9w+oy40qV2U08cjrdfjntMBmuwGCcavB7cxGWoAl3x3XpsWBv5ak0b7hHcwViSCBs0INZFyRUNKSSK7gI4h5lyAtSeoLEtZlnmZn4+Hh4cvp2cni+lMlbKUKtOqVIoRlER5UUql0S6yJ0JQBESaA9MAGkAAlQSAIAAER47IGBM+ikEEQAEP4igIo6ARRUkS8VBoLngYxs3mUbr2uxOVsPmNiG80wvOJ7idpu5EA6aPTMyC9PuiGcdxpNvIiX+/3EPH5/kvGeWdjrSjKo7OzxTIDxP76xrVr11SeSa02L106Ozl+svditlhub25c3tl+ur8PSEqqLx4921objIcjALx+4+pX3rjz4aefvQTYXuvPpjMeBVmWX7+0+/OPP3vx4sDE7zhjr1290u92P/nii9licZbJl0+efft73+l3upgvRcB7G5vTvJiNJ/PpvNNIllqNxpNlXiDptX53o9+5vrsxns6KPF9mRZbl/W47y/OAB6BBar27tbG1Nnj0fP/g+KwZx6WS86IslGokcRSKvJAh6AhUPhljL5Bau0eZO2Ss+jpQ24XG56Vx9RFt9lcHYiQDwirxCWjMTASt/fJfN/7WAYIq1O7PQgAU1jTVWqPSQFJKX3nKAEgIEzMqykIqleV5WRaFLMsicwuFCBBLpbI8K6XU2aLIFrPJZD6eyaKUpEutpNK51mUptdJSSjCLQ2zk0hUWasUYAHGJKIkYAucsCYTg3CyOY+bZx0ARMhGIOAyiKGo1m2kc5UBBGK5tbNDgyk9l+mL/SWM5/d7lpIk6C4JIcK1pMp1NZ4s3Xruxu7P9/OXx8xcvACgSnAiajXR90Ot2Wo+e7J0NR8dnw1YjVYCdXu/uzduz+fzTz+9P8/LF8XC93WilycuDw0iIACAnGk6ng7XBeDpdFsV0PN0/OrmyvfXhvUcPn+31d7azxWyyWLx1++a1q5eeHBzNJtO007i0u9VI0zCKAs7zZbb3+Pna7s5wOnv++OEH77/7jW99689/+vM//ejTfrcZtVMgkpJarcbV3S3OUANs9jvtdmuZ5VKqUspGmkZRIAIRx7FWMi9VnhetZrqzPsgLeXQ+jAMRcM4Qi1Ii55yxXEoFiKBQq3KxfDlbXmulfFWlYw2XUENh/ScPJheJtD/bA43z4+qaq+oN0ohu32W3sSi5oqpVU5eAQJgrS01EpEgXZVmWpVKyLAsOKIKAcU5EWsk8yxZ5Ns8Xy/l0OhmPZvM8y4hIA+ZFCVpHDJhSAoAxLMqyVNKkJ0mTlrKUCjUhgEbQRNzlOYmII0NSqJEYA4KQs4CzNAjSKNKkF3kBpEUgEJgG4pyHYRAGIk2TTrMxnc8ni2UUha20wQc0EOLNG5dffvHJ2cFep5220uT47DyNwlazubU+2FjfLDVurg0Q8Xh43u/1sqKIw4Ahe/7iKI7CVqs9ycvj8Uwg3Ly00wjZoLt9Np09+vQLu202UiONbl/d3b919Ucf3d/d2uAAHDGJgs8/f7C+sUZRMFvmB0fH77z/Xgh6d9A+GY4eHRyR1qR0Mwg/uPva9s7u6dEhZ7hYZFEUh4y9fPLkm2+/OZ8vijIfnp6urfeOjs/W1npr3W6v01GkdjbWpVKCiwBJK73W743Gk8PTYasZa61ms2WeFXmeh1H08Nl+lhWB4K1GMo/DeZZLwEgIpZQmSqNIKlUQKQVMSV2Wv/9k/7VuczuOtOdI7Vc9OgjWV1vYx6rYWOmq8iewD4Eh/xBEq/cqe7By7cHZojXnDNyzu215gEBkZJ65XqpSyflivsiW2XIxmc1Aa855EAaIyBFBlfNsOZmMFpPpbDafLpZZlhdKm4amnLMwCDjPQZOmoiikCTMBSEVmgy6bbQVAjvYhBwTIMGBccKbRBF5ZFIoQkTMslMrKclkUseABF0EQEJIGUAg5QBPZYpnNF0upVIJMypKVy+3GIKdg0Yjzs+zxZHQ8GseBaDXSjbW1Qb+viRaLRb5c7u3v9zutMs8E0Gg4C8OIAeRZnmfLVIg8CsIoWd/a1oTHJ8fZbNzttUfDYVmUYRA0m+12q722tsbZg/l8gchK0pzzLJs/Pzi+vLuZpM2rV5KtVnN9vT88OX7w4vH58Wk2XQjO37x5dWttsJhOy7IMgiAMuS7L4XB4+85NhtTrdD7++NPZfJmEwZ0bV9+8+1qRl81m4/TsfDJfXN+91EyTxWJeFFlR5M00aaSZVroEtczLl8fn82VmiQxRcB7HYTOJI8aXpczLJSKmOpJSIWeNOC2UEkmiFJ2PJqO83E1iiw/wGtmqcxPkAXBL6WoZbKOj3bN8zOoL7crsCBHJFSNTVaVELvtvyXole2V0qtulg4iEJm1L0ZUsyqIoisV8djY6Pz87z4tMgw4QQsYF50A6z/LFbJ5n+VLKZVGWUhFAwDAWQgDIUpZKlVqTUlIprZQiUkpLbbOnAMamRPNoWq21WQAPnCvGjD2Sci4QNUBWllJlpdICkTNknAWcl/bxSCiISMplKYGglSSDwaC9uf2UtX/3fCqnw/fzUTPkJ8P5bJFRGj05OBr0unEULbLFaHTOGV7d3Tg+Oc+y5dbG4Ma1q51uf75YTqbTHaL5fPH4+Yvz+WKZl5sbW8D5YG3t4d5LIhJCcM7DKFkqdWVnfa3Tevni4Hw8vHH1+htvj1/u7R/s7W9vbrz/ztvNZnN0crqzsdOJoiBgj56/DJP47ddvfv1rHwRxslzOwygw9f8sEMBwdnraa7WIi/7G2ht377QajVa3vZzOF/n40uXLN65fX2ZZyHkjbTSW8+l8JpV6tn80WWRFUc4Wy7KUSmkbrAEATVLpuVJBEAjOdZFrgABgmWcMIEzifDbXPEz7g3kUJXGUCqFddKcK3tvMYG2jjVoyp+7gk4vuAwJoDW6xsg2tG9iRKbcgALelPKBGV0f8SsDfx/YFIiolS1nKspjN54v5fD6bz8aT6WyilkuulSKSnCFjUsM8z8s8L5WWSmmtOQEiJIxzBKUVIJIGqaSSqtSkpDYBeUQMze4TyMCWh4LZlNY8m9Nk581i0dI9d8asfwqY27fJJHwJEICTJg3zPO+kjUaaNBuNKAohavTaXRyfyNMTORo9PjvMizxIYgVwY3vr1vUbSdJQWidxEgVikWXzLNte722ubwzWt4KoofT5YpGJMCo03r3z2vOXByfHR8046ne7rTRpthpnp8ODo9PpbNbvbybNzqA/uHNj96cf3X/+4uDWtfh73/zmk63HP/rFJ6DZ22+81Ww05WsyYKzsd5+9fLm+tRlE43feegOIsmXGGD84OjobjstS5vMFIF55685ar3d8cDAdnl3b2TGLsLWSN65cKvP86PhwMp4ss6JQdH5+Jjju9FsMdTsJIQnXuq2iKKeLbJ7lpVJKKkQw0z0rinaa9JtNwwpFKRd5vsxLGXDNGJcq7SffvX5pJw61UlWu6BXrE2pOknGRwBKp3/5I2yXJaBeyuWwmeZiaOJTP2ZPWzDpolUWLtQ3hDSsLrXUp5SJbzuazk/Oz8Xi0mM0Ws2mxmDOlCFFqPc9yM8OyvFBSEgBjTJitHAAJoJCq1PZBpdzEoZQtkUPGwkAEAdeMmXVt0ixf18SQcbMPqNbkHjuqiEqAkEHAGRMCAZAxpfSyKAutBeMEUBApUjFApOSg2Y2TJFeywdi4KEUcJoKdHp8kJKMoTuJod9C5feUyY/zw5Gh4fjYaTUqlwoALzqezpdLAg0QE0WBtu9vbmC/mWSkDzoXgw+FwNh1vbW5eu3R5b//Fw3uPs7KczTNCAk2Dta2rly//7OMH9548H4+n3//er6RhuLm5vrE50LJc6w8QkQueL9ud9kCww7/yy7+UBmI8HDa7nZCJsizmi2VeSgDY3lq/de2K0qqZJts77+ZF/mJ/LwyDQb+7zPL9/WehwHYj2dvfe/D8aDpfvP3atcvb671WNF8sc6nDMN7Z3l4sl5/df/T4+UtZlou8kEov80IWUsUUhKKUCgA0gCJQIqZGlxifa4o1dQLBkCqv24eWHFJ9vMnkkmw9vFXYJvLoiNvbl+DQqXX9OUYm6e4rSyrCdHUkzmytdvEUWqqiyCfT6dHZ6cnJ8Xg8KpZzyjKlFUPkyEops7LUAKXSpEkAcM5CzgCYVkoTFFLaPecEF4HgjEEhZV6WUgJpJMoBCCEOOTGmSlkqBYicc84YMvPgTRfGReSBMOWljLGAcc6ZMVOkVKVWjHFimGsCwEaatJqNfrcbxkkhJWTTgtJZrqVI+OBSSsvs5GWe5be2N+IoODw+/OiL+8/3D5ZlIaVuN9JGFCRx3Nw/EFEjjeOsKPO8BFCNMHjx8mCjP3jt2nUiPZ/P0yT94O23tJI/+/BTU8NbFosyX671un/7r/7Kh/cefvT5/WtXLokovnvrxkZ/kERhnMSMMc54HEVf/8Y3ci1J6vagNZJqeD48G40Xi3m3074bh5e3t25ev8IZ29/fD6KoVPKLR0/2X7ycL5ff/cZXEHUzDZMk3hh0XhwczMaTXr/7/hu3Nza34jjpKsVFoJQSjMWd5nq/vVhmjSQ6PR8fnY1Kpcu8GI4mCICCAwAyBkzwuKFEQI32RERc6ZYQdaa0oXVjHDrutBCygNKueNRp+1rRKLkaJVDOeaqVnoB90AC5bJF2BRmgNdmqKp9lNTboZDo6m4xfHh+fnp7MxqMiy3RRgJKkiTiXpAuptCJNGjVxhmZHGmOjmOdoEwBnwIUIozDkopBykeVFURrTGoGAtJQyB+DIlCaOLImiKBSylKXJjUoNSMQ4cIYIkWCMcxGIThIrgKJUpZQaKEIRBEGBwJQmZI0gEIhSK6bkMlu2kvhrnfjjvLgn4vHNN1U5vy2gODncOx91uq295/tP9w6yUs3meV4UZ+djxlgrjbVWy9lw0Gkdnk0Vss21te2trTt3bkkpJ7NlXmgmlFSy2+197b13yix7vr937crlIAyJdCONte589xtf/Y9/+kPB2WI66Xa7jSju9QacQZJEnHEA2Nzcvnvz5tOnjz+7/zCMo/lkqkm1GunN6zeDIFhmy70XL37+0Sc33rj7+qXLy+n4z3/+F8+PhhGot1/f3d3cSONeGEbtduvsfKy1/qX333znrbd5EDMRj0dDAAaolC6Pjs8+ufe00UhbrdZskQvOG3G40LooSs0QlQICBIXtpg4izYVqd7e31v/K9Z231zrGdkVbiWkxQ3YLGQI0z7T0vrvFrtP3jget2kfn/dfj/LVNGWv2g0/vO+MWwFoI1j8BBPHR558OJ5PpZCzzpZaSa+IIKAJAVECylIjEOSJxwYg545e5QDtjLA4Dbda1KT0vl4tllmW5if8zzgQyY5dIQM4w4oJxnkZhIESBqPNcmVAYY1xwwZmZe4ngjThinC/zfF4UUlHAMYmjNIkyqRZSKg15URZFOZ3NE0IexXGQrCXRb261QITPs3w8kk9Ys8NCHE32T4Y8ihfLfDqbc87jMOitdTuN5LVru2+8dqvf64og6Owd/JPf+r0g3rt77fIH7761tj746Onhzz96EPHiv/lb39cgGu3OX/r2t2bTyWw2XdvYjuJYnqs8z8Io+d7X30ta7aKQO5tbrV4/SVPOuSu+hm63u7W9OxufHR8dTGazZhJvrW2sr69nWfbpJ5/de/pccNgYdGfD04NnIo6Cv/rLX//F/ecaYHNtjTOBjDUb8f0Hj+492v+V73zre9/9Xqe3rrUu8mI4HMZRMFsUw/PR/UfPJ7P52qArlUyTqNVI0zgadFpKU1GWi6KYzpeax7y7RlECLJDA3t0Y/PLOhoGYtRstKtHuZmM4VWuwTwM0Vpgr/TXVbX7rMWdZevhWVigg1OMADt0O0DaGiqu75Zvovbj/+KHKc6F1yJgABEQym4JzxqQSnAkec7M1KEFeFHlRaOeREQITQSAEcpaXZZ4XeZFneVFK++wmJAIOXJg9VpkQvBFHURAqrRExiaKAs0VR5lJKAhOnIgTBWCMIQiFMrACIkKEQXAiuCMwDaqSSGjRyliRxHMfLPBueH7W77dvt7W/IJo7pxTI6ba2L6ai1PMry7Gw0KxR1uu1eM9FSfvP9N69dudzrDcIoVqoMg/DWrVvXdj/++N6j/SQKv7j3lfDNm1e2fvzhgzQQg8GAsRgAdacnZfHk8cMvfvzzr7z3/tvvfWs8OhsPT7O80MA2tna7vXXGeBTHrljWJpL7a+tJowUMtVJbmxvr65vno/Mf/vwXn3z4edzpNNstfXjSCNgkZHmccM7eu32Jc2SIUhX5ovz4sy9++tHDv/Ir3/2173+fMT6bzQBIiCBJG0cnJ7LIHu+9fHk6FEEwXxZxFANgEkdm6/RSaUCc5AWLG9jdhCBm7a7iYavbeWOtIwxyXBQdXRTI1GqC20jGZIdMYQcZ1BnvxkGTLE2iO9aF6O3WjOSNWqt7fTmfy1c5UcFKuTKgkIulAC3sIzq96kapCYhCLjjnJtOUF0VRytIWdiDnLIrDVpoKzhFQMKGlmtvqT9Jamx0cgEEYiEgYpS0EF4o0ADDERhIpHQJbUgakNAcQCIJzzplGKksZMJYEIZGWRIr02XzeCMIoCnkgBOPdVuvmlSv93mCR50maJnES8iAW+m7KZ6rxbrf127qM8v6VsLXZagh2QqQ2W+nl9X6v19tYXwvDSBEAchEEhBAE4d/7m785/2f/6uGzvdl8cXx8evvGlW+8sb6zscbsA/hYFHGZT8fDsw+/eNpb382z7M7dN9e3Lj1/+jiMk6TRFkJEURxHEeecITJAjaSkCsO41e4209Q8u0RJeXh0/Gz/QJLO5nMAGBfF6Wi2Nmhv9LutRhwHvFR6scxHs8Vksuj3Bv/dP/iv775+lwfxYjpSslwu53t7z/YPj569PMjyfD6bTybTXquVRqGUsihlo5FIqc6H42VZLotCKSUaHaW1DkQep81u52/fvf56t1FKiYyZCJHDUBWHh1qwqbI17XuPTct+9pkYFkMuFK+dEkdHujaw6jKb6DwudDEtF1k3jRECNDdBHLfujpmNQpQWDAVnGmhZFFlelEWplEY7A0gI3m21Ntf6RDQcT4ssF0LEQaCk1JqYyQYwxhnjCAhIWiFx4+4IzsNAMMYLVWgixpkg0giFVpyzNIqiOMzLEhHSJEKAolRZWRaktFLLPIdAsCDCMARCSbrRaCCy8WSUL5dbQqRB53Yz/Xg8XhMh27n28/sfbu49v7LWu355O0G2vr6ZpOn+i5e9bqvXG0RRLMKItC4V9QYb/7v/+u/8i9/+nU8+u392cn5wdHLzylY7YM+ePm61e81mK5PZx5988uG9p61WtyyW//b3fvzg4cO/9/f+wdUbt8+Oj0LGojjt9PpBEAjGzMIBk0ojgk6332l3ZFmkaTqfTfdfvMznyzAMpVSz0QiI5kCn56MHfF8wxjiP4mi9371x9eqv/co7b73xVhSnJydHe3ufnp4Pn+/vLcsyWy6fvjweNBMGsFgsu83GlUtbjTSWspRKDcfTk8m0yEulZBnEMNiUSavIchGlcx6sp8k7G/0AmfJbHxlXxlKd5UsXUoeaCieya9vs4g1vkvrYqFnDSVjVgRhAe1fJl0Q5LrbBArJxKLSPWAIgIKGkVIgISEiC8ZAHjHMA4EwTQVaURVlmRZGV0i38RSaY4DwUIuAMkTjnQSCyLAsED4MgK0smlQlbcIaCSJaSNEWhMMF5xtBEeUoli1JyZGkYlkIXWmuiQARRGABjQRCEnMdRpLRWtBQKAwxaaRJGYa5pXsrZdDocnXNOSlMcJePxKA7EcjBYUDJXwcEie342Srvd3btfhUc/bTfi3c3NWZa1u/1mms5m8z//6UeXdrc21tcbaSOK4m5/g4uw1Rn8t3/373x+//7PPv70+YuDn3366LNHz9vtZpqmoRDn56PjszGxoN/Lnj1/lufZaDx69+13Xnv9zVk05EHQ6nTSNDY2iR94RCzLQmmVNBp8eEZaTaeTg+PTQukwjERARZEHQdBK404r7baa/W4nTRs729uv37nb6fbm8/mDh/dPT09Hw7PDk5Nllk9n81lZAuPrnSZTajiadFvpG6/diKPw4Oh4/+B0sVwi40iaVIlRqnuX8iDWRKwVLqOYgvCDrUFDMLtEAlxhkSYA0OjyR+i8bOeMg9PgBGZz5QqeF7LzVsVXBSTg0KixngL1J5CvKfF1JNYJE7lUGiBENJvGRoEIAqGVLgoqyjLP86IoCynd/olIDAWKNArDKCzK8nw4jqJAK2X6RESkNAdExs3m3DlpxnjKeBSEXAhkGHBu1tAFXGRZwRDTOCq1XshSaR2HASIulzkHxIRzzuIo0gCKdFnK4WIZKcUAGJFZx7LMsqIoAVicpmmcKk3rrCCt3mrGzy5tb8fJrTTak4vR9LCb55PZdDqbbm3u7u7SycnJTz7+ott4trnWXet1WieH3e6A8YCQ3755+83X747Ho4OT46OT0/lyeXR6tpgvgzDZXONBIDhjKkrCra2drc37D+/duv1amqaL+Xw+m8RJHLrAjYkYSqnLstRKN5O00+loXYIqTFkjB9AIAedX1jtvv3b19dduDza246ShlJ7NF188vE8ERZmPR6OyLONAbK/3F8uciPphGAb85eHJ6WTa6zTfvntrc63/yecPv3i0BwzjKGrEkVJqHMezsCUBismIdbq6O9jttb9zbeeblzZMEZyBoS2udKGjClPkYkuVUehodCWf5DJGfim5NxGoxppk1qXZcGdVPuLpuYZUv/xJMGQCgQPEIoiDUDCmlVpkxSLLZF4URVm65w0CAeMsCEQSBlEUxVFIQMu8WGY5YywKhFSKI3BEYkwASKlyJRljwOxDwky6qCjKQpYI2GrwRhopqZFhgLzFmAnpGUMCOQNEwThnjCPjyErGllmeSdlqJN1GI4lCxoRUMJ0viIvu2nocJ9lizri41GhebTY30vgX87IkosHOsNFqqglNZ4fD8635NIziwWDQbaSCY16o+09eaHqOhBpZVpaMi521QavZGAzWu+3Ozsb6+3fvtNrtKAyBdLPZ5pw/e/rwj3/8IRAxVKAKxrlWeb6YzecN3mpxt+rVpCzKImdcrO1cSRuNxXQEpJMoVFopJYNQ9Nrt7fXe1uZalKR5nj14/HRZSMFwNl92O61eM+Wkx5MpAyqKcp4V/U4zK9XR2ehsNNla79+6emm5zP7khz8fjqcgBBdcE02XWSGCIu3LuMWJxSJcNNuNVvPvvfPa24OWts/F8LFO6747l8ijzmpdr5LNr7YcqQqLmrVrNude51FwgSpnD9i4fc30rEHfpuy9WUEAIBKzQJMzxpiUSipVFHKRLfOizEupTZEHACAILsJQxGEkhACGZu2Hto/owjAMmZQiECIQZHZFNAs47QNLqCiLrAQkEIwjw6KU56NpFIZxFJrWFaUkoHYYaKK81JyY4BwZkq1W5rIsA864EAqw4Gyt2RgMBgQggfqDdWBcI2s1mygEMZ4myfsxa0fF42UhARut9uNRHNJxYzhuPn/ywZvv337tjSAMHzx62Gs12u32+WR6dHiyKPIwCEopi+VCKl0qNV/mUlO/09pe61+/cmVne+ta2mp1O1euXH97OPzxZ49PBZyeHDTb/SCMhAi0UmUpMQg4Z1KZ5zyiCAIRBDQbHx8dnhy9jKKw0UjNyqxWI71yaTOOAkXs6fO90TSbZ3kplVQqTeL5fCYJDk/Ollm2zItBu1USHZ+cTsfTNIluX92+cmn7fDR7+vKIEBuNlCMCQ5HEQ+JjasxEpDUIpEyEZdL4jWs7d3pNpezTZRjaLb4McWinbpHQrn90j0YgW7hJFd25Pbh9QZOPENmNh92xRABGszNCMFstmZXyPk7lZ4UDNyJp8/RNJoihCEQzjUqlZ8tMKkVSyUKWZVlK6ScK54wLlibJ+qDXTOOiVOaAQDDGWByFYSgAKImjspSkNZAG4L7RpZRJFEVhsMxyAkriWHBuqvQ5xyQOT0fTWbZMomhRlouiEARJxBhH8+iPOAozKbks0yTBQDDGm41G0kgbjWar3Vkoef/h/fW1tfDS1SBthWEYx2kaJw0ukijuz7NhHh8tM6m7i8uvNYvpXM2/ePTF9cvX37j7zs727pNnT6aHB1EQ7GytS6nSJELGxpPZ0dm54GJr0Jgvl9li+ej5i+f7L/u99uWtjffffqvT6TaaLUbqP/zpj+JQ/Pqv/bUgbDTa3SCMyrIkojiKzJOdGGNRlJRFLou8LMvpfPH85cH5eIqcpa1m3Gwo4MDD0TRrNptbm51SytPTsyd7+804VArSJO60GuPZfDJfjMfTVjPZWetfeusOIpVlOctV1Om8s76OnJdSglRBFA7ny2f7w6kGPZ/xRhMaretrve/cvPyt7QE3qCACArMPmNFt2kXTbVCSmXXrZLFrF0Rax4WY21KZ/JcubumIcBVyaJU3+WfBgLbEbbzp2i4SCKBtCh1ACx9r1ZpAkyxKVUpTzImamA07oeA8DII4DJMoSpKYIDfPfmPIk1CEUSilBC0FxygMi6IEpavNShyNB5xJIRhiyJmJkHHE5TJfLPNlngsCXcrJZJpJRQBSSq2p0UjjCKMwCAMRl4EG4IxHSRInaZw2lmUZ5jko1Wq1ev21brvfSJsiCKIgYpyLgG9E4XY7PZ5l/8P98U/Pxq2g0dncGBfT2fDw/NGD13Z21vtrX1vbHA7PHjy8d3Z6XBTli+MzZCwU/Prl3WWWCcZuXb00mU4WWW60Uhjw0+GQiVBr2txYOx7PHz7f/+rJ8fa1O0EYg6sSV0qZrfUZwzAI0rTR7q21z06KstSAZSmTNO11O0kcD9YGu5ubWlMYiIPDw363szYYrA36Usmzs1EYCNB6d2Pt2tbabL6QSm2tD6I4ZoyHTZamjaTRjMJQCBEEotT044fPPj3NZixhACXwZdJs9/v/zVfu3O6kUmm3IsOqVJfStAkkF9m0jyFgTvGiY04bM9VOudtfccUUdTFOs9bXFX+a/UIQzO7AJrBktxAnv1sjeXsDzHVQJEEQBFyRVrJkSAhUSqm0AgNsRMZYGIpGo9FKkzAMlFaT6VxrzRCSOOIMw4AzhELrUirQmnMQgmnFUGvBuUkpcM4LKfPpjDPebqSc86Isy1Lm2jzNjIBIa5JSMc6QQBItizIuyjhWSimgoJMmnPFRlsWcN4MgkGU2mpwul7rIhRBr6xth3DBPQfaPQDMPA+GcDxrJN9Z7B4tsP5fXuz2gzo+1SJbTfJKvzZ53A7bd77/15lvHh3v37z+4cWmLiWCZZS+PToui6LYay+Xi9du3AHC+WDKkJEmbrc50kSuFUbP7/lvpjZ2dwfbVKE6iMOQctdbmvsiACq0BASAIo0arc/XGbRGIz7/4vMwyFoSFNlngQkkVCN5I4sFgsFjM+73BxtpgPJ3O8jLLsyjgj/YOllm+3usMep0gjNJGK24kcZKKMIqCQARBGARpEP3RvUd/9uwsU6AZQy0LQBknX9lZ323EZSn9okskF9OxhGdD9I7rwJdtvLoLQ5UNclamJrdpElZ4dlFL912tGMXfUbsKEp/uJxdh9ZcXaRqbBdFKk1IkpVZKAyBnLBQMAJCzRhy3G400jQMhgkCYwATnLBAcEaRSUkoE4AxKRRwxYLwAaboXBgHj5rEIyBA5Z0IwAiqVUloLxgRjRFSWUgNJJakEHohIiJBzEy4lIE2UxkkUJ+W5WmbLWbbsNZvpYIDIsjzvdDqETKkiYFwwIXjAAyGECIUQXCBiGolfvb7zznr36WSeMn5/NHsskVjzWr8/oTxEdV4qmc0VRrfu3M2y5fD8/PD4NA74Zq+fxmGr2cyyRac7SBvNQATIeRhGZVEIIQbrG+ubu5vrm0GchoGIwsCg02wdYB5Hhpo0IwKKoki1ujtXbk6X2dPn+71Wc9DvP3m+LxgOh+e9brcsy267vTEYKKVe7O+dj8ZxGGQMO+3219/pLpZLBGQi6G9t9drdMAgwEFyIkIkwDE9mix88ePCn957OxtMoSYK0PQui9W7rl6/tfP/yugCtwe164BBWcWkNg1U5nMv0XMSo2+HLsW3tdBsBUB5gRGif82qPMgVw1oNHxtzTFldaUd2USJSlYghFWZalKopSSUlaAxBnLOA8CAQXIgpDHgiNEEVBHEetNGmk0Xi6mM4WjSTkis2KQnDebKST2VKqAt22KUrpLC+5wDAMm3EiBEcAKdV0nmmto1A4W8fk1YAjk+i2M2VMCM4519rmscIg3FlbW5ZFTsS5oEBESdJqtjqdPiCejofnozMUQgRBxCJu8w22poUxdrnb2u0081Ii4neW+bSUgzQBikZSPl9Ot6L+7e3rnTjI5tOjg30M0pBjK42iQORSLubL0+EoEOLq5ctJo5PEyWQ8bLQ661uXe4N1Yedq9fKCNirerOIXQjAAQtzY3P7at/4SkCpm492NQRxHjUaj1UgbaQMZV7Icj0bLPAsCZsyDRqPBGaSNZpAkzWY7abYE4xwZFyIKglme//aHn/3kycuTyYITYRyp9c0pD9rt5n//3s3X20mplNlmwCfdq4Q4QR0ZxrmhVx7ZUYcsur0U7ctlgcgyY5Vk8rGnyktf5Ul0G+RSVUxSJQQsgy7zXEpVFEUpldndXQNxxIAzxtBYB4yxgPM0EEkUCs4JSCqSSgWCM4b5oljmRRoFSRwJIUaT2TIr7OITTZokAYtjJCLSVCqV53lRSkRMQtFI4rJURb4EAEWglUa3mC4RnHE2XWStNBGCL/OCcx7HsSY9nkwyrUnwZpJmSi2yBRCAkkoVSpZSlrIouQlMuIcy2NAbYhQGdze7m610nOVlqf7J88OfnZzvRmEqdNqg87JoBfHV196+eecttZxl80mRL7gItKKz8ZiLsNVpB2EcRclWu9dsdZqtVhIFRHY3NT9y5Ha7MMoO7eNNkDGWLeZK69u3X0uS9MWTB0mcaFVMpzPGxXyZtZqN8XisVTGbjJZ5SURRFDXTWCpM291mpxuHIRdhFISc80zKF5PpD+89/MGnD2WhkzSlNC2TRrvVvNZq/N07V262wlxKR2UWMpqIedq6sBWtz0z6b5zC9Y9hJecYgUelx1dlJfhtkMDFmHzk0yWrajEpD0p7fMXmKMwazbKUiohzzpnJypvHwHDGWKfZ6HU7jKHWWkq1zPLzoWy3Gv1ui3OuNCVpI4rj8WQ6mmVpyFuNJMuK5TIDIrO7PCJxZER6Os+WeaGUCoQIuQBAxrjSpVkehVpLpYxzKBii4IJxRRBFYRRFWmcAtMwWw8kUlN7udiMRnI9Gs8WCtO51uuvrWyJMSOvFcg4MwSpaBAROYJ6hDAAILA3DJAg3W/E8K7+fF5+ejz4cTr53aTvH8P5k8uenZ3ca6dfWumtR0G302o02B5RKNjt9IijypeA8jJNGo9Fstkw4QrnHibo13NVScbtyABFAm122GQBqzRgDoss371y5+fpyMjw6Otg/OjkfnXXH55PpjDMYjsen5+Oz88mdm5caEev0NgbdfqPZYkIwwNPZ/OcvDp+eT6bz7Gg0K1mkG0L3BnGv/feu/f/o+rMlyZIkSxDjRUTupott7h7uHhG5VVZWVVNXD6iHgBnQPIBo3gEC5g34BvwM/gGfgEcQDdEsBAwGQC9V1VWdlZWRkRG+26aqdxMRZsaD3KtmkdUwD4rwMFPT5V4WXg4fPvzVby63r9oqIEiS9eaDmZZJtMXqcO2jL4sCz5s5F2P5qWtdqypbS6G14F8fDwa6eoInU376zvk5f5oUnL8DzxOJ818QnIhIFjNTs0DUhjrllLNUVajrqqnrF9f73XYbY+z7fhjHw6mPMaOmytN2u62rqmubKoTHY98Pp6E377jUirLMlhISZRFEFDNEcMQIqAgFUlYDdq5sMiRGWUIFRrXDNAHYPrZVCNuu8T48nPrb02lb1QTWD0dE2m02ZspIJprGU0ZsmdM8ReYybksMZeaPCNakHAHRgdvU9F+8vm48/4eH056pYni92VxO+RZgBP9/+3h/ezr8V9fbb7rmZ7uNxWkae4QyEs1t1zVN45gZ0NCyloKddP0qZZ/kDIgqkkUkpTgODjE4V4y18lVVN5dXN9dffX3z6cPf/Nv/+e7T+2GaP9097rb7f/WvfnE6HrZtuHlx02yvBoPvP36eRFOW//jpy8e70+NxyEiz97rZpN1l1TX/21+9/a+/voKyurTcZX0yBV3ReAB4GkxbvSKep92f5G7OXPlnCOjq7J5DRmuc/4krPLtMePZry/fXTtEZXIVnwef54x2AiSkCsFkZ4wRGw+S93282222nhnePB0fETHlSUfEes8rh2BvA5QX1g51Oxya40XHfD+OkCxRa8ClA77z33gePzI6dihASe1cXWhoiOTYBImYkyQIAjGgpZQBm6scxZgnBv7i63LTtpqpCcOTctt2GUDVVU9WNCyFUlfM+isxTX9eNI1LNBm7dbA6FUbMm6GYITLRv6v/q65f/xVdXWXXMcjvEl7X/3eMJQP9st/t3D4f/6x8+/+cvb/5Pl9dX2yrnzM5V7abbbB17Wq8yEeGzNU5FNa3cGCJWMBWdx+n2y6fD423p28XH++bVW2YHgCbK7Jqmy+S7y5ff/Hzz1z5c7ff7/X6O0zRNYvBpmv/m999/Oc1D1MeYT1GOx5O1Lex2v7y5+K+/+UoRX7Xh266SmP/UJnAdVl/xyZ/8+MlOFgqIlYJqJcHhefao6CSWeLBu9LRzzf+svD9nq+dS60986tkuz+ntT9/RU4MU/+pf/2c5JiYyBO/c5cW2bmoiRMNhTrV3SbKZtHUVHKuqc9y0TUwionUVNm1jAHePh3GY+2Ho+2GcYoEARQSJC2Llq+Cdd84BgKkiADM7z6I6z6l4NRGZVdXAOdfWFTNVzjd1aLrWISFTHcK2bes6XOwvjN2YIhEz8367vb56udtdOKJxGoi56XZVVbVt530IvmJHRMjEZwNVW+5ByYzLPTKzJDLFPMQ4xKRAd1P8/eH4bVv9q9cvGMQMkNAAq1D5wgssI6qIolo0L8qstYquECOISBynT58+/n//5t9++PTx19+8veyaerO/fP1t8JVzzsyyZCISyTnO89j3p+NxOM2nByb7YdB/8/EYqtrU/vD+8w+Pj7jZNCFQ8JuXL9/suv/ml6/f1F5SFFEp87PnvO7ssZ6F1zOQRM8IdIsRPWHvBbSk1XHqc2dXhnIXaPOnVCZ4eob15VZ5nMX6n9niUm+cS66fBv6C3jsCLLl9XfuuK+0ev9+0TdM8noZxGAwElFUkqjim4KpN2zZ1gwgx53kaJaUUo2pi1F1XNZWTnOc5ncZY3piqxikmSiXfR3a2Fl5N8KYwx5iLwiIAOmbHSOSd984F7755cSkKv3v3IaesKlGaUNUxZwDYb3fsmENQYjH17HeXN95XROwcEzsih+eW7/r5yxxp+aYhlk0CpYBlotr7XVPFlGeRbeWvKhdTfJimi6aufEkVClP+qVwoUr9cemNyFrOyJTE1SPP48e//X7d/92/VwH/zutle7l++9t4jURl7LLO1atb3p9PDl9PxkCRfv/rmwxDf3304jPnd5499TC+ur/7Ft9/sN81fv9i/6OoXTbP1bCoxRhF53tdZzcbwJzd+ocqXHy5nSHUZdysx2sDOwNCyz1rPiaeZrH3IpTyHM5z6HMuEp9rombU+Xf9ztPmJZf8zVwoATiQnySBgYESsBsd+fDz0Vxe77Xaz23aS0zBM0zzN0zTFpKpJtKlHQ1DReR4rR7vWU+sB2pRlnOLjcZxjVkAFJUTnnBGqaYxxRGyauqoCIqpaElsELYiyqWNum7oIh1TEjsv7mUqqAGYxZdPps936ENq27XP2RNL3OUvgV5tu50MFxETsXXDkaBlQwMUsVw+6gEErOL36FaOl9c8VEHEOTJvKl4iDVPaewRnVFhE7a/QjYvmkZjFGU5Wph+lR5lHjIMPjXj/u6/z6z//zb37zL0PbMTsEMtOcc3knagqI7XbXdJuLsf90OPy3v/3+//39+3q333399n9/uctgPvhf77tvL3YOLcdoBiJalpg991tncOccggFWJQZTPa9dsfWzI61UtMKvo3WebEWOFttRe25ATz96+tLnNvccpTprkDxXznkOfP4JDQ8XSVCX8qKxkFIepgkASgU/9P23377pmqv9xe5yv308HL58kfEY72NOd0cz846DZ+/Z2uqi6rZdE0J4PI2H4TaLFIUwAiBmdmwAqiYqZWJE1Zo6AELOOaU0pVQkx5tQX2833jskSillk02oTsN0HMZ+HMcs+67dbToX2NfVwzTaPF1d3wR2TG7OklL0VeXY+TX4nj/tchWe2eh6J88/X442AhoCOfTK3rkCo5R9dilLVq29R1wWlqrJOW1YdH2ZQwiZSMbD9Om3px9/9/7LXR/zv/njY/fmN/+rv/5ft123CAIAGigVGFi15ImE+DjFv789pZR7F/76z3/19dU+OPdmU99sujYEzy7OUxIlHyQmXbEtldXPrUofsPpme/bZy1b1J5+3atTDsytiyz7jc320xnAzwJ9UXWcwAM4z9ef04lkqCQBL5loSvAJ0rK+x/tbyeFsxWlM1RGeiTOSYpKzEEgFEXwUzBdE5xpRTzHmeJmJu60BE45ymOXrvura53G+JcUpJhxiSjtMEZUAewExs4cIiOzYzzJhyUtMpRgCrghvneZ7nLALMRJTATjFuEKtAiuCRN03Tj9M0z/00j0kc04urfVVV4N1ls/E+7Npt8CGKfL6/q6qmabeeHRKKqYoxMCIy0Pkq/gnA8bxNcjZeW7zpMhZgBqpo2ZjAg0s5JZGCyzt2pfnMzFYmxBEBkMi53Qv38/9y5lcP+fcc/P/mX3/9s5//AgBNFRxIzmpFLVBzFjMTk2FO/3D3aEg/TvGvr/f/y59/TQWmAphE5iwofUQKvvLBm6mrUEcxMwNDwpVe/FMYp9xwNABewaDF2tYEAJ9V6XbGJ88t+OfX6yz6dfZ/9icQ0k/h0jPXCcpg8bNffMqPn52f9VeWH2HZ8uEdt7VPYqfTwGaGVFV12zZZdZrGmCXGRGg5ZUKovGubOmZzjKI2xtQ1gYiGcYoTxJRiytOc5pimKABAqsEbm9ZV1bW1qU0xBc8IeOwnUa2r0BJlNTXrnHeAKee6Cl9dXY7T/HDozbQO4fbYm0jtvahNKVUDfH/3eJqnfde9ur4JVT2mNEyjmb568bqqayT2zpV1dZIVmJAUFjbikxN9ss4ycrAe65VSbss1ZPToTDVlISRHJiIAEHMyfXq24reZCMBE1MjdfPvr/auv52EAg/F0ZOeWBjRYTinOkwHY0moWVf1q01SM/+U3r4go5bTyiC1QYdMyqsVpBEQXAjnPdQ1xKpsqFmVqhCXXXM+gLYNrslok6MpFUoCVybSCSvj0uz8x8sUB/7QGen7az50nXSdCi8iSrvSZf+Yd1uPzU1+7Pn+5I66E/zHKPM9JBAG9c+wYAULw3vuH4ynH6JhjKuFYg/cGIIRiMIxzP4ybtmKmrJYVH09zP8aYxMBKnatmKcasmiQTUhGzGeY4zjMRBl8jUZaY1Vow59gAKu+C9+OcssygOsWoYNttVznu+8EBTTDdno51VZ/6McDd5X6bU/qcZ5QkOb1587Ou3SCgmGpKhAjgGdBIi6h5cRN/0pb8yd9XCHAx3GJUSyWUAICZk2QoU+OmKUvKOaYcswDiRddsqkBoKpkBqiqkmFRyShEAytCgmJUiScHMhBBaz7vGm9nYH6umY3YiYqZEzAgqeTktiAYWx9Fs4KqqNjs5PKhKoamVxPOc0i1TFiWs/0n+95wyt4AbS2IA//zrmRn9NBNdje9pz9ZPSqP1kPynn/L5v89/ef4qLqacsxiAqjCRD2G/7TabRszu7h9TakwtiyCAqExTfDgOjOS9qypfV5UBJDWiZrvZpKxTfDRAVSOipmlg3XNvavMcU4zBeefdlFKMkRCDrwxwmOMcUx1C19R1HXKW0zAOc3JMSKhIWa1mDoSFGHCcJu/4zeVVUzfvT4/34+Cb6mevvjLmKcXD2L+Ic/SO1ZUYXSSYy709ry1dGymFX4uwEnDWS1pu6PMEoCwsKQ4TdV1wSoimwoxmnFUt5ynGL6ee2P3m5XXwFYdwru6ziKjYIseiAkk1g2RclpOjipRIG8eTbzbsHIBJnJmdcz6ntBbLiIg5x3gY0bmq3Uz9oUDITzf7nGqDncP6E1b0nAXyVPuvlfXT7z5Z0p8Y0/l4P9n3swv4E2N99n179rpPYf1cUK1GD4tLB7eoQSJ67+u6Ct61XV1V1TzncZol5xB8FeoquHg4nYZJsjjHAJpFUspV8G1dEWJKaYoppdw1VYoxiTJRUwUgGscxQraUVCwDFpkxQHDeOe9hcfJIRGoQYxpjUpFN21ShBbBxijGnnEWdtCH4uvrx3YfrrmWik6S2aVTs/vHovPuLX/z6Zbc9jv0/ffcPX7/52f7iCpC888ysYJBVCQtCdMaqEZD/tMW8LHt8usSLk1lqj+W/ts5IFAo5oGOuAUBVVP7N3eGHMX11e/jXl1002FeVI9xXTGZieppmBCPNc85q0PJyYM7VNxJZzpojgnnndZ41ZyNi5zUnWyoMW1DXcc4p1t12Hk45zmcDWe7xWiTBuj4LdNlZ9RRe4dkne2ay+J+quP/ETS5G9vxsPy/V7Se9ejv/FgA8yTTj+QiVN4DP2vSufKtMZgbvEbE/DcfjKGpMOCC8vL7Ybbo5ZyTynsvT5Gzeg6jGlEx1inHTNuQcEW26RtQAUUQNQHImxMp7QkwxqSqTg7KHE7BspSGzxlPFfOz7snq3qiomFlm22agIM9VVcMEzQMscQuhTvOz2L/b7u/7YbS7fvv4659Q/3k45TYCB3t3df3n56u3F/jrGRJSpiEgWimpBU5jOF3ptsix1/p/6jKcjjcvC6me3BwGQEM3YqPb+b+6O//Oh90T/+Hh4xfr9FBX5v388/pfb6v/w9gbRpjkxU60xiCA7pIAKBEiAyCQigIjOaRZEyoAUvOUMZiqZvJeUVESl7K5SAMhxHnKuuo2Z5TQv1c5P4/jySc8+71nyd/aKf4KcP/eUPwEvn2OZy3qPhS5z9n/nFupT8vDc0FcsFs7Txk+3oPxweRtODcCUVFRojlHVUs7MXFUB0AFAVmWmCh0SNnXtKM4x5ZwIzZTBOyIGoKyGWUSECTdd44NDs3GK4zSnqGZlVcgCthXV3PJeU85ZNXhfVX7KmR233hPgNM2gIqJTTF1dt22zbdu2CneHY+29ZiEEVL3aX/zFr//ix88f//7v/v2ubX/17c+96qfbL/3D/as3X19fv5qnYSZyzjdVretlLUwSXdZFPiVIhShz3jyGz6KkWfETVnrramWcwyGAmBVWChL8zcPx//JPP/51G+7NDlG/3rV/cbX9NMt71e/j9MMw/aL1O4ZCMDRENCWwlShgZlbu9NJKIFLJ7JyAGJS9P8Leq6lGUS1NAUNEVZmHU2g6U805PYvuTyX2k6vDn9aK/0nk/HmOuJjPfyIHff7M9uxXllPx01ThbNn2TGvkWSkKz3xBeaQ6RhOFnDVJpDkBoqFtgru62LVNHTyHEGLO85xyygjqHBfQNKs5gizq1LZVqOsq5ixJh3E2haapnHMGs5mVTiCYLSo/qp658N3LB2UiAFC1yrlN2wKCaFl1h0l0ivPN5cXLywsxm2OapjmpxpxCcC9vXjSb7T/8/rcPDw9MBETv775c7y9ev3rddFvR/Ic//Pbm5evrq1d1qHNOMUUicqX1icDEBaImwmXh5wqYn1O05YIut2BZn6cqktMiwAQIVNaRAiK83jT/x29fvQ58Efz7ab5pm5ppV6X/c3s5ioGJrt2dxVkgoxkSnVsva263JMdIBGbIbJKt1CIGRFSq/udmJDlP/bFqOjU9Q/e4OrY/MaxzWY3PCEf/3PLKd/SntOXnR/d8ifSnRmk/bWL95JntKcdYar4/qczMEMpaV3DnF1109wi9D0Qcgt/tt4FJVO4eDsdTn1M2FdWi5oPM1DbVHGWe4/3DQbRjQkfo2naeYozJnOaUwZQIELng82qaUo6iVfCOfYwp5VzsJYvsmrYK7jTNlXNNVT2cTuM0eefrphHVEHwWNQBP1Hbt22++3u92f/junwjx1cVV17ZieuoHR3R9efPhy/v+eHzx4tU4jff3n0+Hu+BDaNqqarNkMHPOMxNTEThjWIC6Mz0NaCmI1z2OCudpXVVBRCKGBa0D1AWp/qpr/3c/r2LOIvLtrs0iamaKBNCSLroyZb+UKi2eA1cMdgUOVo9ipoisJsxcFqmBQc7RsFRsT1VR+YuKTMOpbrppONlPgzucyyBYq/pnEdz+U2X4swACf2Jnz//yp988pxPPDH1F4J8euqYNcnamuLQMbM2jABDc+vaW9xB8aJqanTv1Y9vU3LUx6zTFOCciZOcsC6oF75goxtSfBhGt6goYL3abtq6c901dDcMwxwQI3jvvuESWmJJkFVEtRSuloj8OZoTIxMMcYxZX+kDEu64jJMd0s992XWuAn77cmhoEt7m8PPVj//iYRXzw1xcXTHh7f5hVfFXffvngiH/1q9+4qvnx3R//zbvvX7/86te//ksFNDXvfZYMgN65um7qwGAoubD40QyQjAgXPbbiHmRZAl0KbVVl9lg2qpSiCklsWUdmSMSsYAWHN8kgGQqOCbIM+C5ZAcIiH702s57d0eI/EUFFDQnLqkIEKEGc6LkHPduf5jyPg6uqOI5PdnNO+NZHPuXW///tcgm16/88fy148r7rz/75V7HdgsuuvBRcCqeVFbA8/RpU7CyKvLycqyqfs2jZjVTOlSoaSJbbu8eYEhEBYt02lXdt2xDRPI2HY386DZIFCduqqqvQeN9WdajCNM0q4p2TnHdd470Dg893j9OcVMqlpuBdCL7yzsxSFlObcpbj0Xned93Fxa5tmxhzziCqlXfsuGnqw3E49H1m3G12P75//2q7/ebtm8/39xfdZrvpTv345uXLm5tX4zwhNGZ693D/3bu/ufv04e3bt7uLq7svn9i5bbcLdU3OO18hoimoWRkUJuaiRBy8N3zyNabPTz5kk1ITMJKBZRUEICRFxKJVbFr0xtY7sCwJBwAgWgzCAEyRGJjOFrkUYmaABEiw2C8qgErGspDKVERyTshsa+b3VHQDGEAxXx9CnOcla1yhh+dJ5BKF4ZmbLAH9mcnCszLo3Jz8yZIk/BPztBU7sNUfnst8xKcmp9kqg7OEnnOGQ7gUS6aA4Lz3gGh6pt8Alu2DoNM0pRQ3XXtztd9tuxBCU4WU8+3dwxzTiupp8ME5J2qH44l6RMKymogdq9g0pSmmw3EYp5glI1IdQlUHJEQEWYhAmFXnaa4tBBdP/TjOaY5RRMukxDzHUz9mkV99++0k+o/ffQcASNQPQ/But93WVY1I7WafVU3leHzo2s398XD7+cMvfvHLN1+9/fD+B0kpSmqr+vLi6ubm5eWLr3KmmCMiiuSStIhKW7dmy+7dP7l5VpryOT3P/XPORGRPHnC5E8GxY55jMY9VMw5gqYfKUB0TIi0M6oWvoaZGCFguEK76mrYM8Zb9vJJTGbnS8zv7aUSO8xTqmoj1zDwqkO0/QydW612LFvhJunnuaCxpYvGChGtg/6n80tmnmhmu45qr08ZlKRIuj1yxvifnamsf6+ltoWu7ulz1OeaSJprqtq1D8LcPh2makXC3bRihDc4AjqdhGoeS5xJzFcK2a4eYYsxNHerghnnuh2gAqtYP8xxTznkYx5SzAniCUnk6IAbKkMt+PAZMoimlw0mGYfSO67q+vLzo6qqua+e9iDrvFPHh/g7MJMs4Tv0wusrv2o4QQ1XFnIbTQXPabvauasLh4X/x1/+ZIf0//6f/cRrHum1eXl7P0/zv/+5vfv6LX445vXj5FgFOhwcFaJsu5bhpd6aljWmIBPh86LtsQxEwOCvbqMhiAbgkfLisMUFCzIpMWSEX+KZw+emcz3FRElyW/5bKHckB6vMArypLhgqmAMV9iirquvZgNZaz5yv/TvPsQtCoiya3mf6pr9OnxPeMjuvq+nBFJp8kZg2QymPO5rt0n3A1cQOAZ0UPnh21mpbjtiAUxZSfoCU9E/cKVLtEfbdtAiGa6mmIc4wiQgRq1rX1OM0jWOUYNR9PhxxHx6wpgWkd2Ll6jknEbh8OKQsRT3MkRFObYxYRAyiiWVNMKWezMpkEWS0vxFoBM0dUh2rKcY6gak1bFWYTMgfvnPPOcUppu9lcXVz88cN7NGDmvh9yzlHFJPXjeH152bXd8Xg4PD68ffMN180PP3y32+67uv3db//+ctMN3h2n+ePtF0d0eXGlCGL46dO7dDoS49ff/ll/ug9VVzTzUYt5aBlqOk9xmC5pJYADMFERlRKZVcW7gIi67ODFcqOXmT01tFX0dTW+NYiDwbLaGaDQn/mc262RDgFW9TmRnJJIXlb5LNT9n3j6c7sop0RMGvMadRcLXbHes/tciO5qRgC2KC0v7Bo4O0GDwhzFp3pr9ZAGy/rDAmuu45rL1g9bX6BYvhoiLU9sBgiGeB5GWf1qWbMEjtillMtnzCLjlBCyqJU6pq5CcCiSEbya5iRMtN20O9B+nB8lDykP46yiVRViygAQvKvqKqUEgDGbyawqRUFHwdCU1yn4kp+54EUFVStedm9Xwfvg99tt3TaO0Mz2uwvv/e3d7b5tQXTO6dgP/TgR4nbTVt61TTPM84/v37VNQ879+Mffd93m1YtXd7cf97vdfnfhvD+ejuPQf/funQb/6tXb4+HxH//hb9M8/+qXv9psvzDi5usLW+jwGgBKwnQe1zSzLLlI8SAWxExKTV6mPNTUMZuirtt3yx94qpiXe1lo+biMJwJqWT9ugIrIT9j1spUFz65pdWUl2cgiWsYTnu12sXKiYDFBU1Xisk3oyX8aPmWOCzVkhYMWuYViOSXaLvSi8jA8H4enHGEtXlbDXtBiWIujMztpSTRxbW0iAFKhv5XsG5ac1QxBTMGgsMUwpTzFPIwppaRmpdSum6YJbs7qxFqgqqpTzqhKYCnb4TQ9nsac1TEpEhFLlj6mXIfgPRIP45xSyqkUFQtrwS/zzMhEazqiKaYk2QCaurrc7oCgaWoAPB5PhHh9eZEl398PouqcyyLBe+94itEHf+yH3/34fkr54vLCVC8urv7p9/8Yc7q8vLn7/FFNAen7H38AhJuLSzP75S9+WW13t18+f/jxu6ZpLi4uI8C7d38M7FzVtE2LzE23sWZjZt45Ji7WqaYxpUIzWLM+IEIClGWUv2x4fAq7Kc2LldiiGgyICFRi8AK7qqy3r9w/QeSnnK/cZVNb3wPiUy/2yb5K/H1WERdvVbykFZBVZdGkJVp7Rc9tbX0xfIrQa2pamKZPFVUhNNia/xTLfNYNwqdGkT0p6BWPWV5DdS2zSule0tmnrHRx6mbm5pjAICbJWbxnx6hmIjpPc+UdVt4MuLCO2c1JYlJTGcY55cyIGUBFDDCmOIxCCETglqkIK8a3MM8BHbF3jpmYmZ3LWaA0UQC9c9McY8oxRwyBs+Q0A8DlxRYMPn6+jTGG4AGw8h4BArHv6ix5irHThrxDw8vrm4+fP8U4v337zd/+/d+Nx9Nm2xnCOIw+BMt5f3n1+uVX96fTH7/7XV1XQz98+fLl17/5y5TTOM23/+7/4xG+evP2xatvDv5xv79SCYSYVyGgkuQtbgnRyrYcRFEEQJHs2CEgM+esxWmBKgEoMZqpChEXB1xIXmaiOWnO5D0RQ7loy7xlua1lcGjZdyo55ZSW8FeOBDwVFksNshbnZ1DIRBbB/GJuqrAKKp01vNbzsox+PquknhCf5VydE8dzYbg693ORtuTThmWOdAkBq+9dnhPxfC7O/twQwBTPTXkwxz7klMuZ8Y6JfEoiElU1paRgnjllnWOifhBTUTVVJuyampGkH89oHKEhUk55xsjOMaIZJpEygwtgIpJyxgLBSFYzx+ydA8CSpKac5yxM0k9T63zd1I/9+HgcmjrEnKecdm0nWaY4dV3TVvVp6KcYk1ntQ1XVHz59+u4P371+8+bd+x8Dgdbhy5fb4zDGlK4uLm5uXoSmiXP89PHd5Wbb1HWeY319w4C//aff37x49bd/8+9f3Vy7UP3dP/zdX/z6L+ebPlRN1bSIxMwhVLR0vJQdn+//Ot5UdAMyswOwLJJyWu9kyUORic20JJqw4FDFnSiqAvPKGIH1flrxj5qlCFLM0xjjbGoGhkaL5xFd8wAzKN3bpRhbK2wrKxFWwGyt/JDOQfrML1JYhOXxTBxerPVcSz0VW2cY9AwB4NnMivktaTTg+owAZ3r0E3awAvzFrksaUJ4QXdt1x+OxFGwpSXmrTCiCc8zTOHnuxjmmlLabpgqOmHxw6r2oVlVV1ZXkHLOM05xSVjUkMjBRyTmrCiGWJxVVAJQ455x8Dt45dIwOF5U8R66umIgJGawJoWKXY+rnSbKodE3dAKOqzpr7GBsfvHdJNac8TdPd4/HDw+nv/ubfv7i5dsGneQ7OHY+37z5+BsQ6eCYapylnvX+4y9MUmsbMfvbNt7vdxfc/fG+it18+OUQR+b//9//dz968vf386eOHHy8vrtpu02z3l9evgg/2VJlClAwAniBpzpKYHBhkSSK5jFUgErrFJhQFiZdweXYPhEhc8FPNCZDYLbtBwNCQSxFpalaYpGnOklU0a8Yy0KxqJuUtrcjmGoqX/yx3XmHtkRWzL4iCySo3QmVH0rMVXUuye0aFDADLiponn7omjKV0On89++vK+i5+8qy2eH6Y6TM4dUEfdHXYAADgXl5fmsg4TM4bO5UsKWY1IGZE6vtJRDdtzVVIKRECCQpReWNE2LWBsD710xxz8OgcI2LhmKYsVoI7MSFhSkW3PIuKzoBQOzZRNW2qGj1TFlhTEwLo5ylnGebIiI/9OKVc12Gapl23ud7vg3Mh+K5rs5lmOfXDu8+fVPLP3ryZ+z7PM+93v/j5z/e73e3D48PD4fv37yNY03af3v+432xjjI7dj+/fnaZ5mqdfffv1x0+fLvbb4+Hw4mJfef/+4wfHdDw8MuLNzQvLOV1eV1VjAN4FE02aHbEQZ82mqiClrBbNZ0wRlq6IIdHCMTc833qThVpK7CRHS8kQFRwVfsBaPpeSSCTHeVqTJVBTNCqJqYE9MZUBDHTF/FeqK6LpQrrGRfprjcugVlQQbdnDsVq4rdH8yaDU7Mw8xuVUrMZoAAAKdtYggLM12+pX7VlDy85w/pKYPund2E8M3PXDmLJ47wAw5xycY6JpmhHNOQKAnHWaEzOfevXBVz6ktLAumqZqKp/FQuVfXu+KUZ7GeZzigkQQEaGI0uK6nxcQsQ5BTeMsaujUERKATTE2REwcswxTFMlKdNbDceyOpz6myETe+Vc319f7C+f83f3dzX7XEL37/Onjx89dXX9NvO82X79544Nnoq+Cv/7q9b/727853h/2m+7icq+q+/32TVV9/DiMw+C9+/Nf/uJ3f/je1Mbh+Ob1W+/dH77/Pkve7XaHu89//MM/fvXV21DVdbclYnauabeDSJJYh2ZJrtZUzJ6xKwzQ+VBcqaiZChgsfA4AAEUm0lUxgKQUX7AgB6o5l7nNlauiALrkkiq2SM2fsaaFS28ABEuHfXGLRYV2ibrP5RdtdbdwLnpsIcoAgKEaPBU9qGC0ppJrNQ7FjBfuz3OEoER5BSRAfXLpq9z4OUeys5ozrP6/fAw3p5QlL5mAihKGyiNBirnsPDJQIvRZ6qbyvmLnppjnrJ4UoSrLgOq6Ct6PU/z45Z6Jm7qOqdRGwMw5C1iZ6CU1LZ5D1KYYg/dokHJCRF9z27ZVFdqqhiW82ThKzoominjqh1I4b7t2s+kIoAr+5YuXMaVtW2fVx+NxmqY0zxcXF/v9TsGY+XK/u7m6urq8fv/p83g4jeOYyvQSIjPfPd4fjkdT/dW33xCiiDw8Hl6/vLm82H+5vWWmHz/effN2bJoBTaahlxQPD18Acbe9lJzqpkOirIKIiIxUGCRWEr6zUKgVEQQELHPlCFDUhnSBFYtmTvFJarks1pWcJGfNKeU0p0UxGBFVwExtAbkWrpGB4TKVtqCturi/kmguosVrP9UWSsC5vwolSVzWKZg+ddLOmWxxo2gA9KxFCSuqD2e/f3aF51T1qWlUMktcIfw1xqyyPKtnXio+QNc2NQMM3vXjpKpgSgjOOREtA8kIBGbB0aapxznOMYJBTjmBNXVu6ooQx5jnmEUtpkxmITCgiZbN2yriRAUkkyoYnrnXMSU1YyQQUJGYU1a5dpel7eoc++D6EdQ0CUA0MCOm4EIUeXg4pG13HMemP+23G2badJv9fv/u/TsQ9d41VfDMMUUD3HQbQ3q4v/3zX/383afPn7/cEyI693A4uo+fReTmcj/N0+Px0NYhuMu6rgqmO07zpm23bX08Pswxtu0WEKrgRfI8nZCpblrvAwCknAByQXKLTFcp1WHp1WBeEH7IOcOKqpSVZ7jeYBWFs+tVlRjnNOcUc86Sc85pwXdK+DRd88mSZS4pxNpKOtfpdvZYxZ2XDNcWMPQpzyy1O61Li5dAfi5h1r75Cvg/83bnBy4fakl8n2eoZ7D2J+XQ+hJrqHkq7lf/C65t6qaqnHdlGKJoVDChGaQskkU0xRQBAMmFyoNqzFlzruoqqw1z9s4R4ek0MHNThxRT2SxjANbYFLzcH2QWWMDAp/kVMyNE9AgAIoKqWVQVLrab7abz7Jhy5TgwE7NjIoBd1znnHk9HMUCDz59vD8fDVzcv6rqAtm6/2e67TkRPp36co3fsvdteXH1490PbNiGEKc4Xm+7hcDwceyQCkctNV4fq/vEQvM+iZvrh05eu3f7+D98/PD4iwHd//GHT1k3TgGmOU1XtiB0Ano6HZnvhfAVYHKLmZM75pde5msqSc6qmnBaitCgYIDGVsXNVMy3cZFAoSnwqZTvIlGJUEQNVlbLg+sl4ylc5AWujG88LiwtOuVZPz1ua6xDdancGRsumw2clzLNhkTPiDmiwHBJYnxEBlzO5ErWW/AFxwUefg/p4zjaXRPWMiRWaCK4VfxnIdwgYKj/GOQRP2MwTD+M8z7OqFsQPzAiJmFJKpVt1GmdTBaS6aVmAHAFQzNliQiLvHSMGRwaYxLIXcmyTCZxR3+X9qoElZRVmt2areZhmDJ68I+eAqKqqrqouLnZJpAm+Cv7Uj4YbSXI8nk6E33792rHLOX/6/JmdQ4DPd/fjODKzAjim3WbXHw/7rmubGsF2m/bvf/u7vp9ELTgHYA+Hx3mevPfH03BztVcVIgJNc5xVJSV9eDyo6X5/pap395+nmK8uL8dpHMbeOc7TWLcbKrJTgCKZ2BXRKGIHBqpZl9TTJGV2zN5pXsY1i12W6geITKRMHeWcUoqSs4pITstocOGWPsGEz4J8ufOEQOuylzNgeW7hrKZ3JhQtRlh8KoIZnFuQ5ycv40IE+ATWL65vVRhbSiQ711SI+IT72/qEBbh66tsvD17fJpgV7VIDWPcuA7i7h8cXl7vah8izCSPzdtMh4jQnYfHgiLmqAiL1w9SDOWYfvCrPcybTTVsz0ZBSXdcxikj2zJuuEZFhnKY5TXNyzjFRybtyVitI4HI+IYuAgXMumxkii8qcBjftd7vdtkveXe33b1++OI6Dc3y13zl2D4fT6dRHScfTMCb5cHurWU7zlHPe1LUB5pya4Ku6mlM69KcUZxWbU2yrepymLNpt2kbyxXYzTdP9wwEM1OzXv/r5m5fXovrjhw/9ODimnOTL3UPK8g3xZnfx+eH++x/et93xuh+OD/fj0P8qi4lImhGZiFyoyHtyys6LZsrJl12jOSOAqgCgqjIW78kGqqLE3gBVcrFByTmLzNMoOYmKgiosY/gLYgMLfWNJDkzXOgNFtRTrstAAFoB2NaMFzCle8Mx2IwDQ83DSmm/YatxmaKBriVtwAFh75etbOhf7TzATLm7IbBlBO9t9yS8AziO25av0QRZ/utRPrj/1cZqK0qepwhzZcagqdkNNVU650CXnecpZidk7V6ux9yr57uHYT9FUg3dVFarAwyRZVdSISAEAiZkZ0TmHiIVejkWNSu3MNCvnuPTysmQ1lSyaUl2F0DZVFbKKY3bEaMjE2+2mbRvN8vv5h9M4xml+PB2naUazT6qO+XK/e3l9dbXfV1X17tOnH959GMZxGqcpxs1mc3Ox7do6xnT38LjtWlV7PPX/8i/+7Nc//yZJ/nJ39/s/vMtqwXERsDud+v/4u9+/fvPNTHA3zH/8dPtmnOd++N0//f7h0P+Lf/GXu92pcd774EPl62azu1yKJMkSah+qMqFR3F4pPBGBijwTqJosBZNIQftTimqaJUtKpcu5AqIFk8en0LuEzadypfjFUjTj+ohFDRPXqLo6UzrDkKuV6FK2r89cCCCLn1vLdizmpE91efG0uKpEPOWWa1qwUpFLurxWTctvnmcVzxZ+fofucOwRkZ3bbjehrjeAMaWYkndeJINBzjmmVAZ3QEUEY85d8ORDyhJPvWNOIkDUNaEKDgBENKoycxVIRZBwKTOXiqFcbQAQQkQiJSRgIlRRRqir0FbVNM0GVoUw0BgcAeAUo6l556qqIsLD6ThM09QP4zAeTv0cIwIQU11Vl7utqX748tkUvOP9pr29uz+eeud407XHcVK1u8cjItZVYObf/Opn37x9lVX7/jSOU9s20zSpalWHn3/z1ePhlFL+7rt/+vWv/+z1zXXvXR5Or66vQWZCPTw+quTBeQS4vLpxOTG7qttKnGGdDXI+lFOoJmBYpmZIlZgRwETLP5JzyjHnJJI155K72iKV94SFLzQ5WPpOVhL6c1xagRxcCpW1Vl6yghUtX/PjJ/LA8s3FQa6Gu/yIFnMtAfgpjUQAKmqMT4CnmgEtT7u8UTsH8lIuPWvCrVIrT19rgQ8A4C4udmYmIsd+qELYdM2ph6oK3rmc8zQnG0a30BwppVy0FOdpZudCVQXvCE2TpJTDflNVVYzJzBoXqiocjv04jEzkvXPOaZaY5QlaKDnVgtkSoTMEQyTmdrfpp/n21Hsam354PBx98HVdBWIi6odhjvHj7e2HT1/iNCPAZtN12K2QDc4x/fHDx92m27TtOM9meLHfFRIwGXji+2M/TXPbNsMw5ZxSSh8+fd51fcy5CuHNq+vjqf98+2gizru//LOfuVCJ5q3XVz//um1+8/B4C0D7XdvUVdd1VdUBkqo2XRdCVS6tr2qRhEhZMwiBmWp+DkOrgmVcrAELyJdzjDFFyVlywem0cMll1YkodcxSzdjzruMTxoxr4vjMFa1hdGXCwTlJffKAK3wL51D/pDuiazuyhLtzeSSqRnp2eM/B1vIiq09VtIXpjE/nxOA55o9wttq1HWauqsIcU1bLcwQDFQak66urOSbJqZqjqmAi7xiJYZxySmXpAswREL1z45xEJGdhpqqqADDnxIRzyqdhGueUk4TgHbsZ4notjBANsaDRSzWmQkQG1o/jdd7dXOxzzsfHw+HY18G/uNzXIUwx5SzjPKmB937bNBLCzdXFVy9fxpSmeQ5VuH849McTAoro4XgChMv97mLXfazC7eHYz1OcYxX8zdVFP8XPd/c5JQEL3n/4+GXTdduuIYK2DteXm4vt5jRM96fx5rq52F82bRecC1Xz6tXbaRpD1RFR1zXeVz7UWYS8D3UDCCKJfe1DU6aZJMUzplNOJROtJfJiZCnFeZ5yTinOWfLC7Suw8XqMAQHLHCkUKchFXn7BiZ5F1hX3gZWuX6x3VWWCBQJavO0y6AwGIEs9DmcjO2eZZ5s7HzMzW6ux5bsrkADnR5UibMlxVdfE5KliKz+n5+gVLpwqBXBfvtwX6K4KgRDLCJvkNKecY0pZnQ/OOWYGM8mLuLo6ZmIQ7U89EGYRJsxZ5zQwYfC+9IHGOYpoFoljTHOUp+0CS9auxXmUTTMZkAwAR+2/Tz++fHndOj8hSZ7NLGdFxNMwHIdBTV9eXKRpnqa5qcOc8vtPnzdt5Z0jtcttGxinOToCIqqrSrMchzGrff3VKwNQyXUIj4c+xlxX4WGacpYvtw93D4df/2JTRMlF01cvLt+++urz7e39f/zu8XBwRDlLyrcvbubLy6uUzYUgpmIIgM75qm7jPKZ5qrqtgcV5cL5iZjPIKSKUFbumqpqTEbNzCGgGOc4xznGeYpxFpFTutvjNglCVuLoMnVpxoIqIQLROL2HZ/XkGbQyfxM/PsXaxicV+Vj+p68rG59+FMxp1xrJWu7GnX129/9k/r8H8jGqd04EltVv94zn9XTz36pLPlo2lSLp/OLRNBUhZlByLmpo6orZpBrOqot2uVbWc0jRH0QBgCuYBJEvOqeQ+KiohnPpxTtkxbdraOY4xoYGYzTHGORaWl5oRoq3Y7FNubgaqiECqHjGLHA595R2IJJE5JT+ObgieMHjHzJu2Gab58mL/9tWN815UxxjfffycYu7aZk4JweZ5ZuK80cv9znl3Xe03m1ZE+3749OXu4dg7x8H7m+vLeYrvjp/3280U09bo4Xh88+r6+upaALJq3VSmmtIc4yhG//j77/D7P75586aqq+DcjDEAzIC82YWmi8OJ5yk0XZY5zgORK+PzKmIay9U3tSxJRdh5IATElOI4DnGeYs7FenC5m1qwQTUjpNV/FUcMJfIyGhHpMhp65rydgaTVONdBElsodcsD1yQBV+7egkQ+C/5P7lCfynA4t9SfQU0FSyhHyM5euCQNYmAL4HD+3urjn3lcRCwCz+XzuJzzMFNTV4QUY1K1ENz+4qJtG2Ym1N1mk7I8HI7kfFXXh8MRwBLxbHNK2cxMFh6dqG27pm6qlPIwziK5TNHTAh0rITgmM1DVYqnlzZZ1RmoGZRmI6q7pri8vMthpmmIWFT0+HEikaZqL7Wbb1Z65q6td11R1NU2zmKWY2rpJHO8eHvt+KNhKVVf7lKtQpZSHND8cjllknOZQ+b/+q18Tu3fv3t3dH+aUuqaOKZ/6sR+GaU43lxfHfvhyf59S8j7c3h8u99s5SYrx4XD6/t2H+8fji5c3F5vN9eVlcEFyStPgQx1cmKcBCR25lEQtRxNids6X1vpCvlvmmRSZUoyScwkvjssoMzAZAIGKSjYreBMCMpArK1TtfEtLHFomTc7R3tbYuth7yQfWNvqS9i1i5uXYlEJ29R3FVlbyGwKYLK5x0T5eQACzsoDG1mhNhnbuRuFakRlgoRCUiqPwVdYhu5KcnNNoOD8MwJlZnEYH5p0rWgSqELxXEQCLMQ/j1DT1brOZY5rnOE2zmbJXAxQdC26SJZ81VVSUEERV1VKSmFIxU1szKaRS1Zdc6Xy9lgyKCrFScn/q1UxyNsmaZeyzY2q69hTj4dOn2/uqCt4xA+ChH+7uHx4eDzkmBGDvysXt2padG6f5+z/+yLxsjClMqz//5c9evbh+9+HjtutE9OXNRds0x37YbjY5p4sdiOT3Hz7fH45NXTFxCH6Omcl9fHy4eziM0/xPv/99jLHfbhxgW7fBeVVNcSq8u3kYoKpNNUsyU5mn4AM7j4CGRkhnEDFN4zT2c5wJlIJDMC4SlQAm2cSySrQsksyQ2NkSuBdsbknxCtKJiMXLrpno+bqW0snW4I2rB3sGkp9L7OdpACwgFyCu436rokLJ0Jb/eSZsuy5OXvnNuAjrruG7hHRd84zFwJfS6uyKz87LYUGFUqpyruttacE/HE+OKcZ0PJ2mOXYxHvspptzWoW3rUHlHNIzzPcI0zRqFkGLK0zhpzv2i407OOchyrtPVQAxMDVTWDKN8FDhf2uINRDVOcYDBED1yW9UnHUz0dOoR4fLiotk0fYxzSpumxhHuHx4fjsftpiuicG3bjvNUlj/d3d3NKRFi0zRmqmKIuOmaH9+9/3J/3zVN1zZNU7+8vnCOQ1Vd7PZm+k/ffTdHff/5lgjfvLz2jkNgJP/4eArOgxoZSMy3n28t57ap9/ttXQWwMig0W8k0cyRya3RGydHMnHOEfHZRojnGKaVZJRWGPhESIROqKJhoTpKipBlVaQGmzFZl01L02KIYa7zQ5sH0+TDqkvHbOadcSylEOI/4LsTNNXdcy6IVxcSlsb7a8FLvrdXQalII5yUjT8nkGX1aC6sSNLW8h7WHUI4BPU9YAQDRVVUoW89MlUodENPj4fjier9p64fjaZrjbtshoedl5crFdlN5R3g8nbyKmBo7z46z5Kyy6dqmrQlJ1UQtx5whQ1EgkDPndb1F5Qroyj1bpNUWzAERxKSufExuzDPEeDyail7InoJ37ACwH+fb+/vdZvOLb79x3h37/vHhMAzTaRglppwTMHn2cV42iLJ3SPTjx89fv35FiA+P8a//6s8v99txnrfdtm0aM3375qthmP/ww485yeHYbzdtXfm23d09HKYYv3r1IqbkGLfbdretm9qlFEXEOVs6OqXmMwU2QCgk7JJpmZloJnJLLqdqKqYCmkspC8CgIKo5JU3TPPXTNKlmACAkYkNEIzV0WhpAuojIF20INSurpwDPFQwUJfZSbi9bFQCWSgB04evgYnDnGuicGCxhzbQoQeMZJTzDorbknraIvwMtNXjJMBcm6Yqwltuu5ZlLfxSXE2Nr6xPOR8AhQJG9iCk9PByy6ByjY3e931Dt6+CHcZpj3jTN3f3jaRhNtR/GTdfKwrsBZhLRnLKIFNLUpmvbtp3mKKLa1OWNapaMshSaz5gLiFhS0sL/MUSYiySONW0NhQyKVD5MznI89dM0b7q2auq7xwfLUp7l3fuPx74fpinO8zRFyQkVyBECDFNvIuxcCKG28Jjz61cvguMvdw83lxfbrjv2PRiEEMTADC4url9c0bsPH/7w/buPn28/fbl7/dVN225fXl89+Idv3ry5utyd+tN+t7u62O/3F03TMTM8u7trI7d4NTAs1Fheot7iXiTlmNI8T4PmCCCBUAkFUFVTitM8zTGKKpqAWRlkgqzeBSVRK7RxBMDiYpTIMS1usHBSy0/hp2o2WCLVApESLCZki7jf01dxgGfwH5ah5CUZhRV4R4RC7afVny5eeWXzrYo9oAhPNCszxDO7dC29znDX0ogwV1c+JhymGLOcTj0RBe+cd5/vDs6dALAKYbE9xMq7LGoG/TAhQlNXkiXGJBJTSipCRNM0/uEPP4QqOO+rKtQhVI4fAFJKwXyClMSYyQdvBiK5iLMvufQyLqiSNKt657pNpyLOcVkYXCj6GezUD8McvePgXRXCMExfhrv+1BfB0VIpm5klAwAi8t674Nn7Kcab66vLy4vH43HbNt5zP/QAVldVERA9nfrNZkfO/ebPfqkqp+Px05f7FGOM88vr/X7bbLfbFzfXx9Oprpu6akJVBx8KVAfPv5aKBRdqE5GBETECIaKq5JzSNE7DaRgOKBlNlAwMRHLKaY5pSsuwHq2VSMlCk0cgBWQDVDVEcs45ZkQUNUYzKFL1CGcwfJlORrVi17hCAmcbKpXKgrgv9m2ABFqmSZ+sfCnRC7BQyCZlBmABws7wPuAa4RGL7hWU8aiSDyMY8FJAL2+3GPg5IzAAZ0jEzjlNKY0pEbNqBYBjnszMOb/fdo758XAcx8kH75xjxL4fYozOORd8VnXqYozFf8SYRWWc5q5r99sNIk0pOuamaUacsxqD1HW13+1EpO/7mBIIruSpBYBDNTM5HY5FWGae5mUAHEALImWJRXLE6H2Y0xzjNE3lUBZ421YADgm9dyEE7zlLQqS2ae4eHtGgl3Ga59cvrq+vLh+Px4uL6+1mezjczfOYstvvL3/182/jNLbN+8dTXyCI3c2NITd123Z7ACPi0ujLOS83dVn+fTbLsscBl5mPp7TNUpofD7e3t58kjpAjqjgw1JzFprJ52kAAkZgJiR2wA3Jski0hKnPhvSPzqrhZcgRaQCiiMv1WOA+LEdDqHBcLWRuU5dqKLFjAcsitNIBWsGhFjojwJ2b0VKcvhlf+FITnjCzo2u4ssZtwmUQ/Aw8r/Ln2HswAwJUqpq4bQxz7QWOa5rnsCTbCbdfmXMUYqyqI6BzjNEUmSCnnnLNoXdelSxRzjlmKClfpMLHjKaby5bgMLmdiUtOU8zhOTVtvNpspzillSWmeY2k5AJGYMVoWmGNk50SfWhBqCgKFCI2ISXJCLnvEbf0qRo4AzAxExFRXfo5pnueqqt5/+Nh1m66p1ezt6xfMZAbOeSJMKW67/eF0PPWfr69e7PeXqe1Cu/ub//C3265pmrqqKnIVs/fOWyljVwlZK3xXeMJ3ibjIJgMRLNW3AnAJ+ExODWOKp4fHNE8oCSRbFgETRSAUIGXnmBw7YGVnzgGgslNmR2zeO2ZHVNZSo5qgQtn6DQaiBgDFDkqeVGyjEJ0WV1lo/kvpsoA+a7fp3M2Bc0JWCBVl+zzC8wnntY+12nGxxMX4DIpYry47P+BcTj0dl3VZyvmZiz92m+1mGMbgXV373abtx2nsRzNj79i5rmsN8dOX+xBcCKHt2hxjylKchKj1w1BOmveubZuUUk4JkIo+o5l1bTPOpKJoggjBOUJUUSYMzicUyskxo0HOIks9alBwXVRMqUiAr5XHCqcgEhMTmYGUr6KRuyLNpcvimJ3juqpSSnNKRpSLDDT7fhic4/1u808/fLi+GF6/vL5/uD/54367HafheDzst1vv64fD8XJ/9c3bt1Vdh1BhMS0kMyNiRDaRdTYI1y8oeQUgItHTfYClE5NklhTH8SQiTdXmNinQPPQx55yWvrshKRqiOmbvzcgoWxUMiUmNSJ0zREMEpmWrYVnnp1IolUtqsTCTDEUXUJwJrbQ0AQqdTBdjWqr2pUBZYKIzBL/8t0D/+GRb68+xwE1rG+n8W2t/vpwJwnPjdkVs16przd/h7IZNzeWcg3dNXTvPAKTwOI0TiAXvEanw5It8vZltKs9NdewHxOrUj9M05Tkuu7+c224753x/PB1Px5QTR8eurFxlJhYil5JlSWZECITTVFSbivSIOeaVPLMc3DKbz6p1FRAwznNSQUDHjr3zzi3D+yKmiiXpXvsiZZ0XM4vY8dQjknO8CF0RH09HU+u23cfPXx6Pp6EfwaypQxZ5+/p1XTUA+PhwD8j9NKvq26/eYMkg2RGxAZZhYCyrjvGcQS0V3QKcreopiIiFSwyQ8jScDsfj4zCcSvkfQiVSpG/IKKIsGm4lAOecUxbDDMijS84xO0fEwYeUXF1VWgVRVRe8d6svt3NvxgAJQeFp9ZGuE54GZdoTABbyUSlnVxTgGTj67IytBrloTZbwX6buyaiwLM4oTXmiUi+qAZ/j+YKDL667nOpl09cZDTVTMDcM4+V+d3mxH6bp0I8AEOp6HkYRIbY8ZzFlwipUwPzw8Ng2NSzPguycD5aHcZqmk0o1VdeXl+2my6ZxmmKKNFMIgYl16d4zB984NjNQjbnMziuY5YV3boBIRCKLoiQgECIQIyAHj+rK+u4SrpjZQJkpxlTaWk/HVoXJkXOBnUimBc5TNJCUsgp7j4D39w+FVgIAbVNnydOU/uLPfl5X9T/8/vsXVxddt5Wc5mm4uHyhgGZISLZQLSyLQEkkzplTwcdwEbl7MtBS7ZqJ5HEaHw8P0zRIWnOglGLMomjkAIyxiJRp0dMp4AgSZNGYiDiFEJiJjRVMDQiZCEWtnINyBdbeh5mZ5nOfU3Hl4S4AT0HG1ppmgTmXuaU1Pq9/oXNnaHGm628uHxlMQdath7ikp0tqscKquHbrF+9Lq/Nex51/0iZ1bV23TV2Gg2vvQURCdoRV8N4HVRnH6XTqcxqqugohlAaQijBzW9fZ+5Qz5wyIaY6fPn6qqsoHj8SQc8556HvnPRGBllm2SsxATSTHlJJkWxp5Aqti2RIW194SMzsiAUMjzwucR0TddrPpmqEfHh+PAKl8BFyPLQAyc9e1Taicd4QQY4opDf2QUyZEx06ylCGklNIwTqqKCN/98V0InhC+3B1ykr/6ixdd10hOMcW2u4BlldGCv5TUU02Zym5EMCvq+7S60dU4kYrcvHceF5AX1ExMk+gU8xSzKx0OAgUC5lLiOEQHAMSGTMSAWOAR74N3ntGZgUpWAiQnsDQwF3LQk5kiFEp4wSuhvJt1+LiEeIIiTrhy6Qvw9CdPhavBLvG4YOjwfBhkSQ7OOaYVrAuBVO1MeVk9CRhaliUxXds1CwiFAO50Og3jVFWhpGuA4IMn5pxzSqMPjp1j5yRLjFFVYyQEqOtAaCKSVcvAmkiWLPM8j+M0znPZRpTmmHN2zjFSUSZiJkRi76qqUgBOCcg0ZStjuGC2rLJ6ymxKsKtCUK8i6gCRuW6qtmmc91VTXyD1Y5jXKr74g65rry8v2bGqbruOGdVgHGfv3IFOOUuZKwcDEzsdTz74/W47x/jp9g4Ad9tOlU7DPE39zc01AJqh5BxCXZCgM+RpsHTY0bC4f1ySv2XestxGJFpq0zINl6OUoWRywQOYOWZYKjwzMzBrVKSUzwAA66aaEh/VRCQCZBEimh3XIYQQmBlpWaRbxgFWi1m9KqyHo8hKWonVpcO8QHLPQvozJVsAU80ATFTCMaxaOgvsClC6LeXt4dIseHqGwlx7KreeJHDO2Wtpi0KR/St+1h1PAxOlVNd1oIR1XbdN/Xg4ZREVTSKIGOqKgKZpHMdpnmd23OWmaZrC09l2Tds2p364u39ERPbujOIyMTKpyBCnsv7HETtmcux2O+9YqjpzFoNxGJd+nFkWwTUyAqD3vttunPcxxpyFwAxMVU/9oCervKur0HQtgKFaKZUMoGqqTdc1oRrGQVRR6auXNynLp+C6bXc69sMwlfbPlNLh8egrH95+dRrldOqnYazrutl0VxcX7z9+2e8urm++YhcQIOdYIratze51P5CtIX5B6UusffKhBoAgItM8HE/HYZhMs2d2zokCsqKCSjZAIvKOiYhpDdCqWXROOueckqSkZaifHXt23vlQOQAEJKfmPAIgE6rqUtEXGUe0M8t9yYJWMoQuNCV8yl9XJmTxEPhk5nYmSZfIvSp2L2nl2fqWMH6ePF4h0jUleMKVCjRTsl4pqmx6dtfgYoyFPDuNIzG/uHZNU1dVDYDGOsfEjJcX+7ZppmmepvH+/nEch5zySQfnuKlrBX/sx9MwZlVyblNVOeXCCmXvETHlPI5jnOeUEiLmlDWlTdddXF5cEt3dP07TjFjU/J8CQ/nQzOyrOlRB1nwsiw7zlFJm4q5ti9txhJuuDcxmFkplxuyD2293zNdgNs1z1zRf7u69921dN3UlYo+Ph3mamCnGrFkOh1OOiRD7YUw5i9nrVy+ato0pppR8aFyockoqWVWLCvOStC2hcyVglJoDCYmw/FmRxSxpmqZpGud50pwyEzMjgGdCY2VCAMfkHTMioRGqqmQpUgk2RxmneZpTUkMi77hyvmsbH7rF+NY3Y7bARes0CBVDNANaS8lzwrrkhmsyivTUOoJnDm+trZfwVip9Lfsd1rGTUhKVfSlECGtdf75Q53xjQbbsHNwXX6uLR14AcWeqQCg5IyEjz3HGExX1wGma1KzyjWQZ57mq3HZ7fXmxu729/3x7P/QDmE1hqurKee+8++rFjfcOAI7H0+HYRxEHgER1CLvNJub0+fZuGkdVaZrGAMZ+IMdF/kpXbwRqsAKZzByqwAj9sTfQlCTFGQAIIDCHqvKeYVHzQDLwTN57JCoMnKaqEGyepqap37x6iQgpp5xzjOnFzY3m7BnnubnYb6c5fv5y//n2vqqqEKpxnlXUIXaBX7140XU7YlfKSheCZsqQACBLWsH5pewsi2myZAfOsXtyKsVPIBCiqmURFUmiKWVCgDIpgcBEROgIVvKxJs1lNE2WYZCUYoxziiKGaHW9aZqubdqm9j5474nZMRNS8VNJFBEcczG9BWrEMsX5NDEHS3WyVtxPE55WcgpbqylY4UnEFapcLLeUtlIaEfQshVgvzrn0WZ5HVns/5xKm5b6X5sGyZcVpFkV0zvu6IqTDcbh/OJYyuSBH3jkwSbMeHw/eu8uL3cuXN8j88cPnaZrmGHNKoQreezSrm1pEnONQBTUlYjAz1aqubjZXSPTly21/OhV4KIswYt00oQrjOK54w/IxHFHwftO2zrlhGKdhLBVVoTUG7x1RkUuAnCvvqxBCCNM0IlLXdV3XMNE0z/Mc1bRrm65tX1xfVp6dr6qq+nJ35xjJuYvd7uHxUNfh9vYRifb7bZmrRDQi9M6FqsaV/6U5mRkzgREbq503Fa29uaVaV5HM4IAYgGDBtJGQQwhV8EsdgyQqKcWUpTgSzxgdES40ZQQtLKEi4EpgnjARAFBVN9eX+6vLy65r6yoQO8eMS0wH0Gc7txft+iVNVls2/gkAAwAUKjSoWqmTnhzbiggsQX/NX5dLYWZarH11jUimKmvgXrzgs448IJTXWoGwdRh55Vkv4JSBrWssXCpnkWIScd4557qma5tWVUSyrZV1TGmcZxU5nfrdbnt9dQFmX77cl0gXU5pjmubYzrEKgZkckyn64FW0H8cxxg+fPhFQ1zRFZX2aJnbOORdUDWzZBFj6WgAIRs6Ftokip2FIMWph7gEQEhAZQFKriGrvfPBqdup7s4UC1o+jqrZNNc+RiB4PJwO4vkhVVbdtJ6qq0g+9qe27jhGPp1PlfV1X7Fzwvu8Hx1RVNTuva1Ve9GIRUFUk5wWct+X2LJo0ZT0Nu2X5UWksFXkDREQUyVkkppyyppyhCCESlQEFM9UEEyzlFyM4QiYEYkBgIiarA3vfuarebrcXu31T184xlm7S2iQoyh9rzAcRMUJCgpVTVMBzpsXmVBUBDQt+hmuGWlJDKDXeikYt7nRtiJRyTp87VFxnoJ8wPztDK2CmgMtIdHlWXeP+uV+1Ml4BAEprfUlmEdAxe+9TSkmSiuYspfh3TMGxmn78fHc69TfXV6pWVpoSczkxkvPpeBqY6rpGIiaqEZHRAIa+Tyk5dtc3N7s9jcMIZkRUhVCc5ZKp2DLwh0xMbGJjnHLZnarGRIwEjN57JvSePfESIAlNCxk5jcNoiNeXFzFGJAzB13Wlau8/fqmCB4C2qXbbXV3V9/ePP7z/xzcvr8tNuNhtx2l++9WrLHkcxhB8FSrnvAEsx7WEJDWRnKVMXUqpLXipaNAWZSxaoPD1sq9RTCTnQn5bq3oDQueY0HK24hRK+1TAMqJzzKxIXHzUZts1TVe3XdNsmNgARK103WHFtmzx6LYAQFjE5stWkKWTtLxyGRp+FnlXZZ3nRmaotLI/zUpEWEqcxSjPqNDaKQV4srznF2F9agPAkr+u8PCaBz+lHWYG4JquNTMmBgQkjCmZQRUCGJhomTrJEhM+QQ6H0zBMMyM758ocZhWCimYRx7zdds7xMEXnnJg4dteXF3NM4zSaQRUYknG3IQTHTEybrgXV27uHRcC/pDvEQCR54d2pmiOq6trMFDQE31Y1EBWjAUQ2E8RDjNM8qSgifkiprarNtvPeV1U1DFPOWQ3meXbOp5y7pn5xsZWcfv+HH0vNXIUQk1xc7HebTYqxbaqqqgpSAQCqYklK/00k55xNZalGS/FROptES9JmRcjWFJQIkDCnPMd5mKYpJjNzXGyFAMAxxag5WUqiIqBSArT3DsuoFmIIoa6but003ab0uuaUSyuOEN2yFIWR2GwVU16rt5I5rdUPIQCcx0oLE6kc86LcXswUccF6V31aW0eS1gpqrcbObSsoYuWGAMtQgC2WVox1IR8uQrmrehmeSSa2NEp1ZUUBOO89GjjnRBUQnOOyAfviYi8q93f30zQXvFBVJWfvXQh+nqQAwq6cfTAlc94XHlOZD4wpxZybGqsQ1Gn2vq1rAGDm64vtHNPd4+M0joExeD53BQ2AiOq2qdqNSpJBHTtw3rFDQgKoHTNREiEzds57n2IcYoICkZgCuyx5enione+H7fHUI5GpXWy3vGmrur7Y7w6nU07p4eGh6DOOczycTkm0qcLxeCTiU9+XzNI7t6ZlRXRDRUUkiyQwO7c0ccXt1wTLcBm2XMBtUxXN8zyfTofj6TBNk2fk4IJjIlQmBJ3mFMVkTmaKCHUVgiNfusXOhaqu267bdCE0YDjOsZ/GApYRQ+V929SVeS5COrbgU2fnhYigi8hCSaiLuxUtK/LQAAqlgdcGREHH1kFkNLBz1X8GgYvpPqt2llC8+kpb6qTzFuc1A3jiJp6FzVY28xktRURXVVXOuWkrIhrG2cxc8Ju23W46M/XO9f1wOBxTSrTsldK+H4vYFTOHEABBshhYVdXsuOxUMFNGaOo6eBdTTDHWwVd11TYVAHZN049TTDFOU8r58nKPf/zRkhaYm733oUIiSUrO1XUNZR5a1FSkrNIgMoQ4p2kcUxYog9VgubSuRQzAN/T4eHg8nqrgzSzn/On+7qub67sqTCldbDdTnNnxy1c3CDCneH9/qCp/fbkfpth1m03XFEJCqc2hcJeWFa6GAEVMmcjR+U6t0anYZXERJWcpGfYwDqfTqe+HYRiDI4SKqWJiJqq9l7Y2kQkkpVwElACM0AJD8BQCegZSmcdhmPNpGMseFeewCgwMmjkvK0aX3cl/stoLDASFkBQAlzZBUc43QaRnB2ydVILzx1JVe95jx2Wnk1mpulYlvYXdaQBl6mqZiDz3mFafuuSwazBfsodSjZ1fBQzcfr83zd65lAXnhACXu+1u2wGBGXnvmqYuxA4wkJTGcSqfIaakoqYaqsCOmcgxVcGbOTFIMZqoGey23a5rDsNoprvtlomHYTicTjnn3abbb1tVu9xvf/bN19//4YcSetA5ruumaUNd5ZQgSxZhIMeWhQpqLSlLzlb2GqkhAJbtOUspBT6EJMoEdVWpKjGdTifn3f3D4fb+4c2rF3cim7bedt0U0+Vuy4QpppRyU1ebTVdVvGmb7bYtTQFmlsJaAss5F+9C5MyMaFkqe3Yr63V/lk2BmRbZkDnnWPRw1HSOyfFCzchqCOYYPXPZzEmgKiqE6ha37fM85jRFvevjnLIVAqggKpiKqqhktXPBsyy3KD6Scdl/XogjxfEr0ULFQxJEJHRET+gYPstDYdEQXz0gPCeL2ApqrqAbFDirLIFeGSnLA21F9Q20KE8U1kXpmp4helMDAueZfN2WOayri13BgcfpbrvtLi/2L64vd5sup/THD59jTKfj4fGRzCyLDv2gqnUd6roqSbMP3gCHfkgpzTGpaE65rcNXLy598HeHUxVC27bf//h+ntNm23775qZr24fD8eF4MoC6rlJMyNy0zdXl5W6z7cfx8Pg4xZTjrCKjiuZsa5timV5acnIzBTBBRCRkYgQUVXIMBC+vb64v9uW6DOPonDOAcY5EtNu4oR+2XUvsri53wzAZyPXl1W7bimRmMtUsUc0hoKioSokeJcU692bWlsgaAZc7eq6SDABSzvM898OUUkYERkCElHIpTHJKKSXJsuKV5lBBxRQkGZkKaB/jGGXKEAUNzDnHTLVDLmurwUTVs8Kic3jev2oAoFj2WayZcoEtzRCBkUqWSEAGkBHLB3ySOT1bGBaCvRGCyaq2gFhWJC4Z5EqJtDUrhVU3qlyeJROFFYGCFTpdW6ELpI8Iqm4YBvY+54ymTVNf7rfTFE/DAADOuU1bi+r98dTUgRGOJ+y6lhCO/dDUAQm994u5gOWcAVBUc1aE5Soc+/Hv/uN3dRUuL3ei+eFw3Gy64/ET4+by8qLttnMW5/zt7f2CZRC2TXNzdRXqKkp2jst1AijZn5oIwJPw3zIys+xEI1vYT+g8Bx9KzgAAPvhffPPm8dSf+rEKXlVD8Mdj/+7D59Mw1nVV1/XjYSCCUz+1zenm6pLYqaGa5pwQpOjDSE5lWXwxvTJBIWKAsJZTCE9Ts08VLgOC6uF0fHh8HMbRMXpytmx8N1NNKaeSAjJ7RgITQ2L0rmxN0WGYslq2wv6h4F1VcVfXPnjvGNk5dmKoBkxlXLgEFF1bkWVr3mo6q6vLGYAXGMzMSnOfzr0HW7n3sJZ9z5qU8LSgttjb8mC1Yr4L/LomrE8Zgy1lOwAA4wLJLWnnuco3AwB3d/+w6TZ1UyFSFu28u9l27dCkGA/HU6mKzFRFiHC36fp+VFMfqpJalFXe7JiZ4xxjEgRyjpwLRNS0bRHo+nz7cDgNL24uQqhev3q57br7x+P7D59fvLBt12mjl/vtl89f2LHfbC6/enV1c01IKaY4TvMc8zyLyOIP1kjxFDsQC4Nsyf+ICoQxp8jOVRjGOJ+G8f2nL01bf/Xyep5jGQkZpul0OmXRlHIV9NAPY9+/+eplynLs+8v9pffBlk2F8RzwVGXN64u/hjLHLuukh63wYBmjLc5DVIjIsYNSPs6as6uzD56dK9W+sYHzbCulgwAITAxMdU5SVoUTZCZ0zgcOm9pt2hCCRyIpI8tAWVF0aQqnnM/+HlciSzFTVaJis2vtwsxY8Jx1Azetqq5gcEZZl9q+2Fk5rCsmtQaQJX1d2vFrXnvOD0qHae3+rzmnlbwWzhSAAgE4RIwpMaP3PKf8+HiQ3KkaOvd4ON3fPVzstyEEBdtvWu8dO5YsOethmr1j7z0i+uB3m/Z4Gvq+B6BpNmZXVSGlHOdY7KYfhvoYXr9qHdmbV1fb7ebdpy9ZPn395vW267569fLz3cPx1Fd13Xjf1cGHahz6j2hzjDEWreEzjF/QNWLPBTMmIlMpooMqYiKi5h07A0Qkg9NpyFmuL7aaJKtuu3aSVFfe8/7T7f3j8RRCeHl9daxCSokRqhDmOHehcezmec650OaLyt+512K69GOYtJACFHGhjBGWaaSlri1Oixm99545qUrOmZAZUYAJ0TSJgCktjGcgMFQVkZg1GywUIjMkrIkdagBFzRINwLJhQmbnEZ0ZpKzTnOZ5nqa5GMfCGyd2jokZkFxZH+icqhYBVyIEoLN3LB+yMAxU4Tx2b0tBsxhSacevQFKh7q92CWVIfw36pa5XKLgBLV53wT9X77OKmxgAmquaJqV0OA2qaip1FSTrpmu2bcfE4zhOc5xTUrUUZ88MAHNMYsbMxBScA4QQfEqZHG93u5TSME0ppeAdmA7DmESmcWy7ZrvtRHWc5inGTdt2Tf3p9q5pmov9/vrq+uu3r999+ARMofJ1XV1fXV9fXbndxvi3h8+c+n6eJhEpnsuHyjHbKjKMgFkFwUA0RpBShaQ8i6YUTye32W3die/u7pu63m/boW8Ld+bVzeV+t5li3nTdHKdf//LbYZqiSDkEw9B37YaZNceYo4oyoxHTmkURAREvgt/ETA4KOYCWYdqCg5aStyT6wVMVWDSDFehOsJBcEZkwJpslg6krZsrMzlceagQzUEBH4GmZaYpqGrNaVjMgBjJnRIwiNk5pGMd+mPphlJQBrMwXOOaq8iEEdhzKDKRzxKyqwTtjXpC1QouHwpVHXuc0114RnuuedWPzMi1ccP6lolqxzhKun0DUdVxOn1WSxSJlzecXFMzAtU0DdfV4PPVDjwop5XGKt85tu3p/sa+rKqdU9qjePx4QoK6qzaa9vtzfml3utm3XTNNUenebribCH999FBHnXFZrm4adi/MsOeck4zTvdtvjaXCOHKIjAKQ/vnv/9ds3N69fzznNcxzH2aY4joOjFzeXV5f7i6tu8x9++49fPnw8HQ5xHDVnAHDeNXXdVMHMspqKOCYDO51OAgCZVLKplMS/ClUc43GeEZEc3z9WF9tNEmnbpvIupkzMnum2H+I8v371YpynOaamcobSDz2W/cEGZpbFQJSf5jYL/5KXze8/BfQWwOlZyVLgd8ccfCBUhwAAOWsZYxMAYvKrriwxe0fOccWEoFDm1s1ETZGQyJCSARM7Ju88MhsyIuWc5jiP03Tqh1M/pHkq8bocIu+9d857R0zeOb8sPfdVVdVVVVUBQIkcEZ3ppMuuxRW0KslWaUrpOfVZu6ylT7mw6kzLaJqt4MaSpCOWPTq0WnlJBmytLBhpGZqL88zOeeeD95IyMYlpnud5nh+PQ1WFtm32u00IPsXkvNt03W7blvp9t9s0dXU4nD59vq2bum0qM1C1qqqc45ur/Xa7yWIq0m66z1/u7u7uL7bdy5dXMemU0qaprrftw3H4fHt7c/PCeb/dbcVs6vsvnz7uu2az2Ww3u7/61Z81df0fu/b9Dz88PDzmafaEvq6NCiccm8AhhHIMtt2mrrKUPwCqxgDTNI/z7J2rQpVFjqd+GCdAeEn0+e4wTVMIXkT7YVgmNAkeHk9vXr2s64a5SE1JycsK/aWYGjM5ZMJlCKRgpEwOyia4Qr0lOkMzi1cycwx1IAACMFVLOUczptJ5J+fJOy6rugkNzUiTqZYOpQAoEDISOWQXgg+LPiYhUTZOOReGSE4pzlOcxnmeUaVM0TEzMlNx8lwmI533vqpC29Rz3bRNs2lbH0yJjblIQiwZZylo1rrz3EfiYlhqpR6Sxa0uWJusZq1rQ2lpMsGZSrKSa5/lb1r6+4DueDwhESMwsWvctm0FIOUiO5z7YYwxxhjbtmna2jMHR455Ur25vmya6njqvWNAHMYpHE9zTOM0Vd4H5yRLCOGXr199ur33pz6nfOqHz3eP37x99e3bF5++3B5Pp21X13X98PD423/87RCn4MPlfp9i8sTHx8Px4b4Nzabbvn756svD7Xw6BaJxnsFMY0opiTko/e2iCWaGAMxUheCCr9pWpvnxcDyqkBQyvzla7o9jqrwv7uHh8TiM8xRnRjyehqv9lphSlm9ef1XX9TkKFW9ScileSh+ErAbiEFUZCpy/gNNnhKlwiCyLmGRTUYPKMyEpWBbNKWdREVMxIlRzxFx7xwRQyHlZRFKRVzVidr5yyExNFXZt40MooLkYgFjKBAAIKlkkZclJcwaVUrcbrPMoi4myY3bBzXOwMl9F5BwbgA/IRCIFHi3EDlQDybl8PCIs43dSahxYkgKzsjF3CdxSli3COa+FonTG5xbXOq+9QgV2/pGZuXEYAJGYQgie3TDNarrdbrY3V4dTfzgc52lOMSJAVQUkvH14TDm/uLnqNpsY5+1207WN9+724RhT7vshjhMBsOOH07C7SCH4n33z5re/+66qw+HUG8D7j7evXr64udzPMY7j5MjGcfzhhx/atlNCQuy6hgDv7h5MNQT/6tXby93u9c2Lx9vbY99LSnmeQbToOjvvyHFgbupqTpnBCLBq6zevXjh2X77cMeH11X4cp34YPLskeY6JEczg9vauqirn3P39PSI1dX0/nNq6lZy3281ut51S9N4T01LWApDZuTsPBqqCCCLFUz6BjrZ2QWE1U0JidlVV1VVIKaJpYUMyITpCsIJSSFa1TEgelsoJc0oxDSklEQNgdlWNVeVqh23FPriq8kQIarNoEiO0lLNkycXsRVXFRNZeF6KqFE4goiILUc5sat67kKosOcZExLR+TEQsi6zxzPVcNywv2czyuHMHH/AZc54WT7oQmYlWqGrZMbLSROCpcaVLegSA4BAJCb3ziNiPI5h1bVv2Zu26jtn1x2PKGZBijGAuJQnep5Rvbx9Siiml66vLqq62G4kxHQ6HpmtD8DHlNM+3n2+/a8Lbr17udtuPX+6r4DdtfXc4PRxOlxcXVQjTOBZN4X6cRQyZRPXF9bUPPuU8zfHx8WG7u9hu9t++fptzDN7fO344nXKWGqBI3Q7DoBBdCGKmgMwURIfTgIhlXoCYri52++1WVac5TvMcvD8cj48Pj23bXlxcIGJKkdgR8uPxgI6TCjNt2rr2IVA4RyWAIvDLS5GEi6oblcGeEqnUgIsM1TJxVrIrIvTe+xBCCDmnMx0MAIr/K6BjykaQQRTAQAQkx5SnLGU1cPBGjJK9SgbLYNmUDagkuWiGpqZaCJAp5aLOvlKiYSnenpYciKqiQKmAVFVWPcKy+GaF9E0AHRMRmloRn4BlkGON13YWMoFnlNAClC4VUklyFBCXtc+2etelNFrnR8r/GQI6XwcAIGbnHDtnqqZ2PJ6yyPXlbu+dSsZproIHw2GMTPR4ON3ePRpCCN4xl/H2yntE7DaditZ19fHj55zyoR++fHkIobq82P3s69cPjwdTy2a3d/ffvn2bLvafb+9ubx/6YR7nuQqx67o5p007brt267oQguU8DaftZn95+eLPQqib7vu20/cfjg8PMs6FA2pmQ8o6TioCiOT9cRjv7x9LJOo2m6YKpbk0DsOifQI2z7OKxhinadZlcasgsyENw9jVNSGNU9x0mRMqETOvedgCvkJRpyPjJXxhqUNwaTItzFwod0MlpZhzLuMoomKS5yjDNEvKK4Vy8RyaJBGoWhJRkUJOMTMiqEUNyPvkfHRjJHKESOSyLbRTQkOErGpgxX+W5I/PZwhsAXnKxyAkdrSo1xqYZpGU85KflEX2RLAoZ67wvgEuumKll1foJvYUOgrAtLTdbQWsy4UyWqceoDSl7Jwi4DpEurhZV3QZGNkAVHWKMcZYBV9VYZxiQcjalolpTqmsRCrn0nlXhdC1TRbxzg/jBGb77abwmolwf7G72O+qJjgmEdnvti+uL999/BxjOg3j4fD48ubm8Xj68d2n4/GUUp7GqdzPh4fH/eUFImiMh8MRAJq6vX755sXli227vby4cm3zH/727x8fjjnmMjBUSAnMpGpxGBBwJHREoa58XdGEVFVTjKdTryLe+1M/ZjVDnFN+OJ00ZRERyuyYEVTy8XjabDomkiyZkI1XqJnKTA8qANhZN6SwnEiE2T2bqi1egUpDf44xxjnlbAaO2EiL85mTTHNcFIDBGNEzBbduvipebuUKC6AiipmKxBhHYgB0voBfCGXQouz/hGXAlxFhHYsuGQcTQZk/sXUAuTQ7DHJWp1r6SYgIzCbCAGVG2Vby3mqRZ/BywaHWLtOThN1aDOnZPMEgi66ud5F5MgNFKIt412lPUDCHZo6dc26a52mezdQ5Zu+K5pZ3DAApSco5WEEASUQNLHiHAMM4dm3T1LWI9MP48vpCzT58uq2b5s1XN23X7rcbVfvjD++7Tfvt29cXu+3d/cN+24hkNQvOGeAcY5qjqkqS0FQxp83n21AFU5VNF1Os6qYK1f76Zddtfl7XL65uXlxd/3f/w//j/R++xwTMDAbeMRKlnBAcmJFjDj6Kfvly54O/ubz0zAUR1Cwpp8qHnCKYaYxFb2xOsfNt1zTZ1HknotMc67pwTRacE4nMsNC0zYCJHLOacWkdFTdYeoLPaiRcun4mlj0B1U4ylqctNYtzLqdU/B0wMoKpeUeO0AC9Z1VX8FXvKHhGIDFMApANBWsHXApn1Zg1xmSSQZXAmACZzIxLMki0IOQlA8bSrSmsYU0irmi1qoFZFilbfYnQlMonRESwUiyspbgVd1mm5J7mjHnBbhefWnrFZcSq4BoLy3+1Wjgrh69oKyI6FUkpMVFb18xsZpXzxIwIcZ6D7+qqApvMrEwbIlFdhQJ7qVpK+XQaVVQBNl272XT9MKScry92bdPkJF9u76sqPB5OADDNcds1Kc3brg2e+1P/3Q/v+34owI2IxDir5FylL19ut7utqngmZjw+Pn5pPnKo9hfXdagqH/71X/5L7/x/+z/8j+9+/wfpe43zlBOxI+fIsqpYsphFzbxzdVWJSI6JmBxzVmFmIKtDJSLsuPQr0xy9d03beu9e3lxdXV7UdX1GAgUEoAhkoiqQWrFXOM/TFHIGmKrxsgeqXPlSk6BnrrwfmSVnXcIYIiJTGXgBoqfGtJqBISMQsQAqY1F+V0UzMqCshMaVq5grI1ZEE4hJxzmOc0opS15asqVmh0JtWXyoYdlju5CqSwZb2HIqqlky5YJJgRXxRirJI4oKIYKVDW24wqAABaVHW/v4RebgCdXHpRFma5vDZNG6XzT3cGm/L/ZdcAMnYpjyBNA0zcV+t+0aUe37YZrmcZKmCqPpHBMilgaDmbVtc3WxnebYD9OpHwhximmcpk3X3d4/Hk99OWgiYqpTTOM8A2IWAdPLi0tEmOb5y/3h/v7xh/efU8yExMxF2EPA8jSNw7DdbT27VXTTDodHHz5WoWq3F8zUtdt/+eu/AID/qa6///6P0+lkScoOVslJRbSkRwgK4KaoIsH7y8tLFbl/ePDOtU09V5WBtW0bQnCOh2FUkbZtmrqqq1AFf7HdOO/HccwplsLVIXrHziEXljygKqgYOjiDfcVSCQUQEFdhWwAids4D0BxlTnmc0zRHyQX21sLsoDKqU5TTJCdznm1tOoIYAAExsAIahZJKEgGwiKacj/30eBxPw9xPMeeMYExF8Q4IkLFMrBQneE5OIOsSAVQsZaUsRLlw82BJO4rntVULYrEkfd5kX7nJBTXlc/WOiCvzb4kqixkuoiVSvOw54KyggKoRmRNVVGW1nFLOeY6pwNCErKYfPn6JOXdte3190TZVGQgvcR8ARTITEnHsx2EYS8ibpjmL6KRVXe02HTEP09i1NTNPU5znJCIPj6eY8u+/++HUj8TkwK/NCairWk1Pfd8dT69eXjvvUsrTPIUQ7u9v67qu6oZdS4Sbzf6vf/NXu83mP/zHf/jdP313eDyMj8dxnrzjFFNOidi1bRu8F8nDlASwypkAtrtdW9XOk6myc5cXu912o6rzHMdprKtwud/VdV2CoZnVdZOZYpxzkR0yFikkUcgAZtHMiJDYOSQAKIkQEtPTyKNpwT+N1CyJjnOa5jTFmGKOsUhUWfkPIhAgM3lHCZSX51hLKNKsmhRb4OBDP6akAyGA6hjTcZiPx9PpNEzjVAi7YKXiXmaV8Lm+kkFxq4Rn2P1JwlJUEUnJcEXaiWjZu0aA5/K9IJq2Up/WLxErsSAvMX0ZIzyPdzACPBtFWrNYK52Bwo8jBWcqkiEjeO9yln6cJOdxnGOcY0oIcLHf3dxcXu53zFhCUzGlrg59TzFnQk051XXlmad5VlMTTWbznNqXTaiC907Xaax+mO4eT/ePx3mOwzghqCOXzZwr6xyEENqmnXM+HI7bTeO9c8xf7u6nOe62u+3mmGIMdYPEnvBie/EXv2xeXb14/erl3/7933/84X0YpjTNRIOvKt+2dVUVTKPIJ2Uzk4yio1mLVfCha1tmRsBt13318kVKkRDrqmqaJoTKAB8Oh9OpF0Mt60cyMBMjAljKgoDOeUA085JzKoU8Cy2Rs3iKJX4BQBYxUVWNMZsYQREIMUBgxy67Ocac85wTZYiJvRMm0rXqKjFymqkf4zCnOYn3wxKzAVK2aY7jOKYYJSXJmVau+1KLEBLAcgNxUaa1VYhgsdJ1sYfZMtJbyu9zik1Ey1KFIvVTxIYX4tKahK/D44Qrrrmw+59kSdciHp6p45QsFlZzBQNz5Yp67w2ggAumVrpH5NzVfvfi+qJpGu9YAWKcDscTIg5D1zR1COHqwo3DFLyP83zq+3GcmCirEmKc53EYr6+vjgD9MJpZP/Smev9w6PthnucqODB3sd8C4uPhFGNSkZQyEm67dp7j/cOhrjwTeucOpxMRD9MwTUPVdoGpkMSbugkvXrVNc7G7+Mfr33789OX+/jGXESUwIt50zaatifBwOFqZaidqmsYzg+mu28CicMRNs7m+anJOzBR8xc7N80SApjpNU5aMpaYWJRQgAgAkdEzOPfFxEU3VBISxeCtEOC/h0ZTyOKc5ZlXNZoAUPAXvRDUX+V9YFJ0KMxpMMz1Rh8tWxCzqsuacx3Fynpl96T5m0RijpKSaVaR0DhhtzTqXCkYLYPlUfy9qVnregQBwFn9U1TLnU6xcoQgzCSAScVEgxHUhaOEenjU4zkNwKwj7BKWdlZ7X7xReyfkcrkUSgLu8viyJ0zhNwzDUIQBAjDMivXpxc3mxTzn3w5RFihzSPCdEQCIXwnZTT+N4O47jOB8Oh2me26ZGtDLp4hz74K8v9oR4PPWEMEcZp2Pf90M/DMO42bQF5ri+vCjFLBM+HE79qd9uNvv9rh/GOUbnfcoCgHf3D0i46bZtu3HeEy4pOTm3313+5Z+1b796e39/++7jxz+++/H27m7sRxCZ4uyDe7nb7OoqNM3N9Yuuazdth2bj0JeCwBFXzhNRCKFcl3Kx5mk69sfj6RhjArAijZAKW4UBmDwXgkhpHZaq9KnjBypAvCTRZqo6pzzFPMecRXPOIooGa3O79ACXDmT5pXMxa2eUlNCxc75M2RAgRlGNySTnlMq0foGZFhmwFRQq70nO+jTPxjAUlh6DrIVSCfZudatgpgpFuhEWCYYyUmIASISqi0bVmYas69rLc9FjtkxaP0sGFnB0JSmYnhforGQ8h4BznEVVRQkoZjHTum6cc6Y2T1MuG/tirirftU1p5DLR8TikmCTnOaaHh8M0TT4UnVtYHAriy5urly9eTCmZWdNU3ofTMKUsh+MppUxEu91GVA+nHlfN4q9eXn8AIIJffvt6mOLd42PK2RGrChI/PDz+7vvvQte+9Z6ajnEhDDlHznVN017tr169evPN19+8+/DuD9//cP943227b796/ermRdM028120+0WEA+5LKlMaZKcoQD1gDnFnCIimshUzSI6TqPKIqirjOyYEIiQbZ3bRCvoI7GpKS/PvwCB610yBHLes3PFCJ2Dog6dRUubsNgnsitEOzAoQmLl8UXVh5kZSU1zloUwkXLKWUVKP6Cg8aWuIgIuk+eFf7EilbgO9JXcgxa6arHWkgoubQJf6n9bXKCaiepKgNflZCmULGoRTVvU1IqZPXXVAUB0ZZnimsICrMjU8m9Yl9CVpMGBQfB+nGbnedNtQggiEmPMIqdTP05T2zSd95uuqUKY5nma0xyTY/PBplm9D1nUwPYX+7atNMvx1AOY926aYZomZmJ22+2mqYJjPh57WE/vMIzeu8vmYp6jiHablpnmKb68uZzm2TFf7LY5y3EYsggCpDmq2u3nLx93P1xuLr0PhStUrjISMpBv27quL7b7b15//Zs/+02McwhVW7chBGbnuMyEcYlTRR8hS1X0wMqMmk8+zU5FlWVj29ev3w7j+HD/YIsXRLd0OYjo/1fWlzVJciTn+RGRmXX1NTM4uFxSF2W2K5ErM+n/v+uND6KkXRDkClhgMcBMd1dXVR4R4e568IisWglmAHp6uqqzMj38+Nz9+1idn1zQWEpRJAmhdlP8OftWk5mllOdlllI8x0VkKRKCqFoRLSIkAkgamIOKKVyHs91HQy6FAJMlU3XuQdEqOysiCEBemTumBQQI7LI+jfXzNtK669JqGSqqwaskqG2BZlIu0MhgtdejToSMykxaQU5rkRnQ84HW4L01U6tbRvVv1x+whoRWd65wfR1AcPVSJNoOQ9/1S8mlVMYbVWPmzWZ4erjfDN1lmqd5MdVSihO/Oxf9MPTvnx4B0aSM05RTZkZjTql8+vw6zfN2Oxx2m5KLiuRSxnHsukiE0zTNc9KisevLPG/6+Hi3/+Hjs6o93t+fp7mLHSJGotp5Q7hcLudxkpxU9Le//YeHD18hddxGN7w7gszM3HX9brev9wXqsHgtN82s0QICGHpbEsHQQohFjYOYZgQk5qHv7w53aUnTNJqpAZmq6/ENnfkABBiIKl5LYOe+8yfjqjAyT+Pn1+c///zL68txGqciDjeKk6aoqKhTpNZ3qdRHVbLFAA3N54mMauTVai6ga0mCzQIRkZkMkQCL2G1W556p9i49eiOymvhIlROnmgDwWtiLmvtJIoA2fK2qa9LpDrH6TjOvnHTd8GwZurYRE2g+3tYtu+ZzYXXYZgAQ3o7HGFtLvZRh6O/2OxF9O51yTiJlnufz5ZxyPy/L29u55GSAOZOo7Laboe/6vgPT4/H0w48fc8rMpFLmccIQlmVZ5nE/DPvd9vPnl6UUr+aJKWIU1SXnl+Pb0+NDCHwe568/vNvvt88vx83Qh8BLSkSwpDROc4yhjzGGcFnS88vx+z99/3j/sNnuAj8iYOCay9hNWIkc2u1YCwJoGb0nr6xKCFiggLgcHnMIkhEJQQARh34YhgERiVlFVKGYGUhnUkRCjNbc91ooYcXFEb1iMjVVkTKN8/l0fj2epmnKjc/H7dLLlrZ0Ve0T2qav+yesOiG26mFQm568AvBQc8/A6ImWeJG+Tk4BGJDnkYIGBmKgIkgC6rMr4lTsFfRZf6kBqgGajyJoLanEqAHvZiJCiC7S0owNWhy4LjJdO24tC/KkuDXuYW2YAkAIxKaaVZGZCWMgKdkMDrvtdhgATIo8v55jnLxNstvtzOWMnE4CcdP3IllVY9+B2WWap3EG02GzSTl/en65v79/ergvuXx+eSPip8e7nPI0L0hoqpfLqGp3d3sV/dNPv9ztd2h2PB6ZaLfbiGDX96/H03gZN5thM/QuIvrp0+efPv707sOXsd/2Qy/1Pq5OAqhlgQAVAGlnst4XJTJVInMw3b2R184q6uALEYUQu9gZQEoZfBSXsYAtuVAoUSSEQOi7H0De+AarDIZmagqmueRxPL+d3pZpKsuUl1mKz69Wxl2/t2ujUOvwbt1+YGifytvooCsg4+nlLcaNCIM3JwFc9kUdpScI5DcBtXLZeHPK1LCospqoUx2YqIooIXmI57oMCKZeBiliPZFXG8QmdLUS1xi0lYL6ceg6GVoN0LtcazIDdSTP3w3MLABiKYUCMWEfY8kyp0REXYw+R7Lfb0MIOWcA6AIV0Wle7u8O758ei5Tz+XyZplJK3/dfvH/69Mun17eTqCEAMavZ5+cXJuqHoeviZogiQwj8nN92203fdy5DP47jMAwcwsdPL2/n0VSnRZBOSLjZ9M4abgDjOKtoiNHMTufL9z/88PDuabPdc3hX0y5ne/zLzHw1WWtbyv7hGzRt0HSaPaAhAsWAhCKEIhx1v9v1fX86n52d3gyJTYXqHp96YlmI2Ut1NrMmpuYdRSnldD69HY+XyyWnZCpoyqAG5gsRRlghyfbItbJU1b0BbGABIJhhRTRbp7FB78hM3n8l8uFiNXBQBSITNXhStdgKdXp17WqctR9gzbZuzknzSczV7RVRpnUx8KYcXKWMW37piBys43kV62xYl7WEWGsIFFXPygAsUOAhUAjBzJ7fTmCKhkRUYjkcdtutSydE3G4A9Hy6HN/OMYanh7uH+8Pr8fh2PuecD/vdbrv55dPz+TyaqsvuDl3UUn76+dM8z09PTyHGIroZBgBMOe0326fHu58/PTNzP/S7bT+nwkgIQEy7GBDhMs4xhE3fxxhUShEdpznkEiMvRb/7/gdTtSz/4T/+5v7dB7AOIzrFu5/mFQJuRnnNyn1F08lCEFthQYjAaOjxrmZViJvN9v7ubhwv8zyr+Bqw525WREQMUJlNxYRVVEkNyZrFGwAUkbQsaZnzsoCUgCaEhAwVbgQDqNTlqutz9cul1nQxrOhhhSuxwuBQh4YBCTvGyAhIapDFzFtchMHFnRAV0GWDwMWV3DE2tQ7fVwb3kQ0tU1M0VFUAAjK6mZzzH0b0ee3q9lazXuv0m8B17ba3L2ogcNIGz3Bw3ZtFCA93ewMspcw5iyoDdF1AgBh5uxnAUEUCD10Xp2lKuQxD/8X7p4e7nWp5PZ5EZDP0h21/nufTecwixIQIMYau786XUaWMl5E5/NXXX0oX53kexzEQbYZutxmeHh/MIAa+vz/Qeco5dTEQQgguRCmXad5vN/vdxic6xmlx+DCGYGAfP/5cyj9eLpff/Pbv3335NeAWQsWr1nB/a6YtiGij3FAEcHYh8fBmLh0qJTuvTlHVwGEY+hACE5takYLkQhlkZiLKEQFBzLhKZNwsjps6WsQhumvSym5cGVyl+W6wSuB5kz+1SgKguk/w1LM2LeufwQDBZ1EdlzVA5y8lAmaMTIHaOryjN17yAAIYke8V14gjaiKmCiLAbP4ZDE1rBQ9QkInwirRjQ46qoa833k1wRVrMfbOfqNZeu75LS1bd8B3+N8MwjbMSDsOwDSGGAACBedN3+8N+6LpSJCX5/PLmyf7Qd/f3u/vDYUnpx58+vb6dnAT55e3y/PnldDqbKiFRF+72u76L4zQ59QO0RH5Z0ul8GS/TF+/fpyJdjH/7619N8xwDH3bDPMNhv0ulmFpOi+MvXeDArKrD0HdddxnnlAsiRGJDez0ef//NN1nKfwZ7/9WvzIYQAt2YI1xDjyG6Fps3bNVVNNR1F1TNtOSU05KXRaXUghiBOOx3hxi7k15KKWoaiVrCVyFPJ2VgDkxc/UR1f6amIcT7+4d3797P8zL3s5QCKqWISMEiRQTMyCmTbgGgmpkhNoYSJlwVDQzASbrALBBFxsAElZRBTTUiMFEgjETtfQEN1FX8GgZaTcopSwkQzKVOTU3VPKCDl2a8+mtPmB1sa7lTm51b+5a+Ue8zTOhQEa6Aa2WE8bH566vWFNU5DsDCOI4xRuw6ZBpih2ix67549wiI85xqbtvY8D03mZd0vlwu49jHsN304zT9/Mvn09sJEZ32qO/idrsRKYgQ2CcmZZrmXBVXMRV5PZ7u7ndE2HcRwAihlDIvqe/C4/1BDVKKx9NlSfP5TbvY5ZReU9ptht2mI4J5XpJACCwmb6fTH7/7bhiGjsPjF19Rpb+sYkt2Q0ForRIyAxHnGpOa+bkWr6mWoipLWtCLAyJk2u52T48Pb29vS81ZDcCHmdANXW/ZCsxUFEyaIg0S0n63+/qrL4ksLYsvNqioqlzGaZ6XeZrnaSpLKqWQKoAFdBQIGI3b4g9C1Qx22fM6gk4YAwYmYgSArCZi7JA/YSAicn5iED+S9VKB1ll2aPpETeLDTyY28NIfELtCcsud6gWtZnU9sjUFsAa/39ZSUAv2inf5oamG5ZVuS3v9r0M/9EM/IMA0zqJyt993MV7GKfbdYb81Uzf81+NbSnkz9OM4//zLc8o5MMUuppxTyjHwbrdlDt5q2++2fcfjuLgmZwisIs/PL0S0rmPPKQ1Lt+kjaNkOXZFSSpmmaZ5HAnh4uNvvhqGP07y8HM9pOauUlHJKadP3wzAMfX8Zp2UuMUYA+/zp8//I/6Sl/P3v/uvDh6+67trMWH1nM9BKoVNDjohlM8JVAAAU0klEQVQrvZqolmyiOaWcZibKOad5jl1HHIZ++PDui7e3t1LyspiqNXHJCg/lIlEkWhsG97holVWGiGKM2+3m4W6/LAEQCTEwRWZ3iCmVcZrf3k7H49vlfJnGUdLi9IDNkRpUGseKwjuvCSEGxuCDz0h+4hyAgNY+1iZG6KCStQwB0Nj/Cq6dr9XkzEzBGKyIsoFVeWFTA18guX5OJCKHe625hVoJ3DyCa24KrU/QlBoqCtaK2mazAIgY+mEAAG+YbTeb7WaQUhBx33XELAJD3xsYB96Fbd91x7fTOI5mlgmnKQGCSjFVV84khN7FN8w3zDGECICvxzMAPNwdfEp/M3RoWkqeTYjg3eNDypazT/rJp+eXnJM3n97d7fuue3l5ndKSMuVctBSVMgzDZujnJaWUWIRDPL29/f6bf0bmf/gv/+3h3ZfO/nzrO/0fUZ/WUfBSybF7KVKSqJaS1URUfTt5LsnA+h4Dh81mc3d3fzydVM2joWuEurn7ZISocuU0WNP8+gUhdTEG5sLeyHTRTQ4hEtHhQO8M8ldf5JzGcXp9Pb28vr6+Hi+ns6YEph5IoaV+DUqjwBhDna9KIimrgSFSdAYyX1ABUDVXAyX0IU5AF+9qdYqDkQAgarG6TwfjIXBYz7rBKg2vWNeDV/j1mne2fytE2tgc1vdABGCsaBo2LMKal8WbWxfSkjwJ22yGu/1OVHNKRLTMy+UyLkvaboa7w/7x4T4wPb+8juOcixRnU0FERBFx3h816yIPm6GLQUWYYxchMJqJFEHmYehcE5sQRcsyT8qMaGPf+XOMgU3AVMZ5joGYsKiGEB7u73IpqjbPKiKXaU6lDH3fdbH49EaBjPD6+vLNt/887Ha/4e7w8BhjXA+iNXTYP3xt/YmoFKtTOwUAfcyAiUwEwPq+rxQHgfuu326Gvot5SX43vdwpIszk3SMRp0EXqgU2rc7AzFJK85JySj73TQhMAbFGW0Qahj7QQR7t668kp/R8PP7y8y8f//zx9eW1LKkOVACuOyRdpMiIxISYRKesZtYxxUAxUDWOlte1C0aDRo8P5hu+qwFds1t16/GMYuXmNwKDtm/kS/FNCWSt3GGt8lcss/72ViN52ipqDbwGaA/Jv1rzB0MI5/M5MHd9L0VO5wsTiRlSEpVlXsQgMOciqioERHQ4bJeUx3H26biUMjP1fURERo0h7LfDdtOfx7mIhIAxhi4ENT0cDofd5sePn0rOgYmQci59F/vA87Lst/22C6UPS/bDDSKiJRFGUNh0PAz9ZZz6LoyLOgBNYF2kwj0nzKWAaWQoy/Ttt38Ydrvf7H7Xdb3vErWehqeL2cSk5JyWZbzMy1RymecplZxTulwu0+U8TvNlHKd5STlpKSpKIbhevKn1fTS1yBzIwRfMRTopMXSe4NYUoiVStgY0oixlmhcmCrEO+/SddV0k32NzbXeiXb/B3e5wODzd3x/22z/+8fvnnz+ntIDnnWgGFKOTgRERLWJzElGL7AeMAyIhFKgmogBeqRCBKDQYx9bGWgvX0K7XmhHXf1SNqxRj7QtVf+m912piCP/PxmBNWf3+rzm6451XIKBG+YoDXAslAAsi4tG5SOm6oKq+u+XEYPu+N7Pj8Q0A+j5uh76PztqDSwrjZSKSYdOHGEwNI3dDT8zTtKRlJoTtbui76NFov9vMaVly9hnJ2MWHIW6HIVdRoqHrY99Hn7rBSgBquWRE6EL3dLc9n08FYb87FHGiIjrs+t12k4ucx7kU3Qw9MlrKnz5+/D/dN3eHu1Jknqd5msbxMo7j+Xyaxul0OU/jNLrk2zy7Y1MEyaWUDIjEDKrMrAbe+0OEEHjouoe7u6en+4fDduBoToPp6aC2Hnq742YOULuMhnEI93f3l8tpmWeVUkrxOXaHG82gw87bWqoCGfqu32w2zCEXOZ/H6TJKya7KbIDR1eh8lUp0TiWLej3KgYNXRoBkVgB1PSV1J7giTf57nV1RTU2JfdXMTGpC1BDZFnDV1h+t5FRt2xOc56FBntU8bd3ywCv1/FpI2fV7a+SH1ZXXEN910flPOAbRK6Y2dN3Qx1JkmiZiGvqeENSsFCHEh4c7MJgOyzLPuRRABAZVNrW386Qli8pmGDbD0HV+t5AQUi5oQkxD3x12w9BHM8tZN32362PH0IG8XfA8zmYKpqbFFKdc5iUzcd/15/PRpyoQaMrLp9cLM5nBksu8ZFH10XH77/8YkBgxO5qj9TarFxzXxAjqkSfutwN2Xcm5C7zZDGYQQ4yBY+RI9OMvz7mUKRc5HqeUXt82D4fdX33x2K0TVQamUkphLoGDEjuICC3CMtF2s/vw/ot5mY8vn6WU5Gv12lAEr2gYzCDlUkQGHQKH/XZz2O+7vgNE393pAm0iIyIB5iznJKUIB45MHVMM3LEPF4MaYsVcnSyy1s8ebREAQdFFPAy5Si9ALZEcuIHKxlvb5dV0gAwIkW59J9xaFzTErNke4K1vrT93BQpuav+bFwJAIGIzW1LCXJjR5w4J0VTO55xTJqZh6D3zm5dEYBy7/XbbdbEUSWl2AF9FUhFULVIA8eFuv9/vNpshBEKRcZ6nSS7jlHMhMJMyT/M4TiaSckGiTy9v05J1WS5LHuekRZFIAXIWABAz8WxJ69SPj3m120yE6BpnDdhBW7sUBsgUQyBmcNhTxZ+CAAQwJCZmy0UAus1GU0pzssBzLuxxvGgMrKWAWVablgWILks6nS9/92//+uHurqiIlFzIAEMouWRiFqnL5j5LogYU+O5w//WXX0vJ57ejw3ArDsUUQlBELKLTNOWS+3642x9UrY+hC9HxUCbqmAlRDHKROUkRDUx9oL4LQ+AQAiCKT0itjsjvk88p+7QIgrpUMQAYtLUNU1UPCBU2p1r0GxgBrrdX0XvuWOEqwLrCVZVr3LPC+vv/vy9uc8/qpKtZ37hY8CIJwYpqYy7H0MUuRwQTNSbaxk3OZV4SghHRdjv8+ssP7x7vmclUcu7PfZ9ymqZFL+NlSSllM8spv7xeFKGksizzkkopJeUiClbhx3ppdX6mHqu6boitaXbTCmqwBDa7A+v7jpjHcfJDSE5I4LEMcf/01B/2m74//vSxlLx/fNrttv12+Paf/rfj8GQmZqRqpkKBCEEEkUrO5CR3mgWgeNXffq2ITcvCRJ9Tmv7Xv/zm7/72w9N9kQKZkIJHM5FCRCLA7PHNPwrGrru/fyqSfzSYLqfWLii5MC5zylkNRORymVLJm83Gw/C8LFnFAALjEJkIFUBFp1SKKBP2kTdd6COHwNAcJCGwuZKx943AsClyuHX6NEkbdyqiQK71B0T1IBOCoSkh15ebI/dOzF5dI1FFNFuqXQe5V3d6Dd14PRPXx3vbC71iyf6tUHLy5Al9Y8EZ9IoQIjEjwGW8iKgUQcS+6/a77dvr6XQ8zSm5bsU4ZxE5T9M8LUWtqFopVmn1aibTnH514h5dvPVgNXmutSZWYgH1DAZUCakWmW5+oqYCxIioCiLZcy4T9SBpRZylo8zz01cfdveHy/PLdLmcxwsEvFxO5lNkhABYZUGKACr5vTUwU9IqKaBtrZ2Q0JCYDbGIFjUEyHn8n3/419/9p39/t9+JFNMiJQsHQhLMAIgoCGrkhIcIiH3fPz68E7XnzyG5uA8REqVi83yelrTMaVkWIALDE5/Ssnz85WWeZkJwvnA1SKJLKqkoEXaB+8Aup+SR3Wyl7qqYe+X2rF4R18UkA59RMbvRj3Oks9qKGxgSVO7/6u+rHdY/tI3N6voaRXRD8HHNAW4zgBs3uXaQbssydzSuduqqJb6SYmVJvl4j1mZbmkTZZZqej2+ilfy/fmTC2lpoJQJWVj4lbAwpWHtdgCDSph0QsfaOgYmM6s42EiNCSdlXPZG5nTIjZslFTJ3iAwhFPBkidRUT36pBBIBlGv/0zbcqlVQ7ny6fz2cRicQYgvkcbt0R9t1bD7VcJ46d1c33exAICAjNVPyB+Z0xe7mMv//2T7/7zb+LwdKyeoyq6mCmzExgQAZWPQ1z2O/uum5AxMDRZ2tSyqfz6eX19Zfl87RkMNAiry+v8zgdT6cyzQGRyTvizvUiRLSJFAPHQFWLEcBXLg3RAVkD1FaA0NWJAiBBZchBbfi8udxUq+9xHQiogZtW4hv/ayc7gSY5fq1+/H0rHEa1adRstMX0vzDF1XGu7+/1QvCRQZeq0gYZqpo6S4CPmdSBNChiAE2/o5Ua0LABu7p09AYxUWMd8DlwIkV0CW0MoePgFYK7F4C6mswtlDiJMIhKyVKkpkn105m07BsQjcldgVNZYtVGA1N1/9RSPehCrFmA4xwtDtXM1dPbGLz1ylqRca2llvPUARMBB4rR1VOeX0+vx9PT450VQsoGUKT0XSche6EVYgQItfAQUZG+73fbfQwxhFCLNpGnp3fv34/v3j1/9/33f/rhxx8/firLwqBsQqaAoICmNmXJRZgxBu4jd13sYqSG5DhCpAZSR+zB5wZ8mUIrY0MNWW4lAUGrAyEidI7bhiBDS5QNXSCs3kqqVdJNZolVMQLEqq/0b1q7t6s1ri7zLxwnrq+oxGMAEFyVR1Swvq7CqvWtoYqGae00GFRx3CpgX/1oiyX+cg4hdhGRvOpUNS3ZT23d0DbQItOSKmq8TgqQ3xcFAAKqd0FN/fIIpW2ZWz0O6IeePG1vC0cYWK4pbFuaRIC6OuZsa2ae+SMGZKe1r8hKQmLWXFwecm1DOQRpAAURuZBqiB0xTSn//Hz86sOTc9XHGGIIoiZpURWLilh5AXzXMYSw6frA0Yc369YDABKGwLvt5vH+/unp6Ztvv/3lp5/yNEYCBhKDopZLyUUNoAu06UIXA4fg5Q9jZRfzR1H5u7V6eh+sqv6krntAnbOzSqy/XgggNk4cvHGlNTez9Ut/4Zo1ErVhwesNtxpTbQ33t2DcreOE1pLFduGIEJD4WgFUC7WVuKyaf8Vi1y0T0yY9iIDrgun6i31QB8FW2vaVMkpbc78+b0Q0tDVNrhJEgDXBUKzTWw51oI9F+bVRQxCb0/ZLVIXKKOd4HiG6rwVfloPKCOzllBYLTP5snTLJ3bKJBGJPNOv8uidrqGbq6bMVEcwAQcwuWTwKeoOVQzA1p05ELGZKRBy6GHo/vYEjVa5Or63FzCU+LIbw8PBwOBy+/uKLf/3jv3z3x2/H1xfJUkRTKbmYmXWRQwgUmCudrgJQMdCGuzeXYT4jpw3P/EvPtaZ+BFUIgpl8lg4JyYeu1dwH11vdkKgaM2m1O2gWA1Wy+9Ykrj1QuO4bXd0qrtkqXOslNQMIfoIcFHA/4Y8EW+BTbaOyiCvBxNoZa1F9nT8ARwOaL/KOAbpktyGtVJrNE3qeBC1/bR+hJkHg5Fa3rn1NGLQ11xBx3QX09EDluhpYf+paollTP6nfN1W1AoimRlWWDwCsqCATuC6Lj3+hVdm06h8MDdCMmNOcUirUU8qFQzAtuRTnIo0x9v1mt9v3/cYNlOoGRXsHADMrLu7BrhGDzPzh/Ye7w/7Du8dv/vD7H/70Yz6PToDXRRpiiDF2zA5OO/zu8TG0FhGs8xCoq/34HSGAutJqoADMFJidv5KZXfpbnaaHzNePSK3uK5iBmrfSDUypqsyoGQFBW5harRXbI7uxSAODdZMJrwG/rot5IxqJQDVwDKZa1XEBWy9r3SgADq3DBUbEiOsZrWRU9QJW4/c82we/Ddb2N7QawVq6oLCOfpEUBwV9VNHM5Si6vjvsp5cXLeLKj+t5rElxjVGK2OgrbnqMyEziCJGfKN9Qg7prVtMEBEABIzNP9LniDz6dWdc13ZzNFOsX6MvhaErABJCWdJnmGNgAci6qMs9zEYkh9F3Xd7ELMbDLDVQc11o9a35hKgZiVlFFzx77fvvll7+6jOPHz8fl9SSqMfIQOUYOTB7Trfmu5ubqLVh5ofysqvojaoPrHsF8x44CkEsmVTkGf8wG5s7K6tq7Qht2r12g5tFAa/sDKmiDayMNWm/dh0wBoA0A6C26szr9a0dU1cBC4GDkV9+yzBXz9yQFVwqK1ZFdE4LqJAFahmotCBgYmppJqb/MBZkBEZCYQgzU93le+s32/a//5vP3352eP/nkm98/DhyGAZlXuoHmQ/2Wo62p/urHGxeeX7yWQmui0k5KayZXdwrY+hmInjzUil4rhQG1zNMqveD67mCEqqiqhJRULuOy7bsQ9ZxzkYKEfey7GBFRREQzC4OBS87VigRJwVRFck55RgAIaIDcVkSRHDp93O52hhRi6ALFyEhciRPBRF1WGOlKKltp3hXWR2k1ufSPpVp/DpGIkchtE4m9dFQzMedchIpauas29CE8n1wz5JrOX2MgoK9IA1R2xTYnbpW/pBXWWjmhK2TQ7iy0+RJ/r9CYxFo9ezPLX61hHUg0xXYckdZThboiKus0YU05tCqPQisIyTcfgTnsHx9ou3n788+B+fD4eH55eXv5bG1AC8yAKA5DWhYTqQOD1bdd8d0GSHjMaFnP6rO19javiVe7EbWkb6mtf2y8Ge+u1r/CBgBonrbadZxOGwsWqojlnEpOl1HUlJm2my1VsFZLkZQyGDEbMleCFwaj6ziJFEHweX8hysxBW7L3eH//b/7mr89vx+V4JIDaO1YpAMVA1Vu8GJmtldV+gNvYKtQWsHtvtSKVS9DpJzgEInZtEiBSF7kz8HWkla+UjMzZmKAen+Y/cS1IoO3lwU0BtP5XVrTJy6D66JzpxLHv+hRW6wsVWGnfamlus83V/66bMrjaH4A76ht7lurq6oPFa+FVk0VCNB8QE6MlA6CJ5mXG1kZyX4vMHAIA2rI45o/12lrN6G/rw93tTNj16AEAEhN4U98BPzPTumpOgOZdaPVHxL5eiTXpqEmVrSxbjfN//WDr/xTBEFR0zmVKScWQMDCbWS4lFAakEDRnAcgBkFwD07nNlCpMiBS7IS2jSVYzJEk5OTMCMyPS4/3h3ePdz9MIIoC1/WMAYi7dC4YE4Pp0gK5600YLPexXIohKKA5iQOs4FDMyM3GFPG+Ki3rP1JiaMmqlGSOX4VLzHN3M72ZlnKwv9lu5GkmN0c3D113VenbcsNHaipQ/7v8LuhRpugWd6nYAAAAASUVORK5CYII=)" - ], - "metadata": { - "id": "gW4cE8bhXS-d" - } - }, - { - "cell_type": "code", - "source": [ - "image_data = (periodic_impulse | beam.Map(lambda x: \"Cat-with-beanie.jpg\")\n", - " | \"ReadImage\" >> beam.Map(lambda image_name: read_image(\n", - " image_name=image_name, image_dir='https://storage.googleapis.com/apache-beam-samples/image_captioning/')))" - ], - "metadata": { - "id": "dGg11TpV_aV6", - "outputId": "a57e8197-6756-4fd8-a664-f51ef2fea730", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 204 - } - }, - "execution_count": 11, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "3. Pass the images to the RunInference `PTransform`. RunInference takes `model_handler` and `model_metadata_pcoll` as input parameters.\n", - " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input is used to update the `model_uri` in the `model_handler` without needing to stop the Apache Beam pipeline\n", - " * Use `WatchFilePattern` as side input to watch a `file_pattern` matching `.h5` files. In this case, the `file_pattern` is `'gs://BUCKET_NAME/*.h5'`.\n", - "\n" - ], - "metadata": { - "id": "eB0-ewd-BCKE" - } - }, - { - "cell_type": "code", - "source": [ - " # The side input used to watch for the .h5 file and update the model_uri of the TFModelHandlerTensor.\n", - "file_pattern = 'gs://BUCKET_NAME/*.h5'\n", - "side_input_pcoll = (\n", - " pipeline\n", - " | \"WatchFilePattern\" >> WatchFilePattern(file_pattern=file_pattern,\n", - " interval=side_input_fire_interval,\n", - " stop_timestamp=end_timestamp))\n", - "inferences = (\n", - " image_data\n", - " | \"ApplyWindowing\" >> beam.WindowInto(beam.window.FixedWindows(10))\n", - " | \"RunInference\" >> RunInference(model_handler=model_handler,\n", - " model_metadata_pcoll=side_input_pcoll))" - ], - "metadata": { - "id": "_AjvvexJ_hUq", - "outputId": "291fcc38-0abb-4b11-f840-4a850097a56f", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 133 - } - }, - "execution_count": 12, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "4. Post-process the `PredictionResult` object.\n", - "When the inference is complete, RunInference outputs a `PredictionResult` object that contains the fields `example`, `inference`, and `model_id`. The `model_id` field identifies the model used to run the inference. The `PostProcessor` returns the predicted label and the model ID used to run the inference on the predicted label." - ], - "metadata": { - "id": "lTA4wRWNDVis" - } - }, - { - "cell_type": "code", - "source": [ - "post_processor = (\n", - " inferences\n", - " | \"PostProcessResults\" >> beam.ParDo(PostProcessor())\n", - " | \"LogResults\" >> beam.Map(logging.info))" - ], - "metadata": { - "id": "9TB76fo-_vZJ", - "outputId": "3e12d482-1bdf-4136-fbf7-9d5bb4bb62c3", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 222 - } - }, - "execution_count": 13, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - }, - { - "cell_type": "markdown", - "source": [ - "### Watch for the model update\n", - "\n", - "After the pipeline starts processing data and when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in `.h5` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier. You can [download a copy of the model](https://storage.googleapis.com/tensorflow/keras-applications/resnet/resnet152_weights_tf_dim_ordering_tf_kernels.h5) (link downloads the model). RunInference uses `WatchFilePattern` as a side input to update the `model_uri` of `TFModelHandlerTensor`." - ], - "metadata": { - "id": "wYp-mBHHjOjA" - } - }, - { - "cell_type": "markdown", - "source": [ - "## Run the pipeline\n", - "\n", - "Use the following code to run the pipeline." - ], - "metadata": { - "id": "_ty03jDnKdKR" - } - }, - { - "cell_type": "code", - "source": [ - "# Run the pipeline.\n", - "result = pipeline.run().wait_until_finish()" - ], - "metadata": { - "id": "wd0VJLeLEWBU", - "outputId": "3489c891-05d2-4739-d693-1899cfe78859", - "colab": { - "base_uri": "https://localhost:8080/", - "height": 186 - } - }, - "execution_count": 14, - "outputs": [{ - "output_type": "stream", - "name": "stdout", - "text": [ - "\n" - ] - }] - } - ] -} + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [], + "include_colab_link": true + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + } + }, + "cells": [ + { + "cell_type": "markdown", + "metadata": { + "id": "view-in-github", + "colab_type": "text" + }, + "source": [ + "
\"Open" + ] + }, + { + "cell_type": "code", + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ], + "metadata": { + "cellView": "form", + "id": "OsFaZscKSPvo" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Update ML models in running pipelines\n", + "\n", + "\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
\n" + ], + "metadata": { + "id": "ZUSiAR62SgO8" + } + }, + { + "cell_type": "markdown", + "source": [ + "This notebook demonstrates how to perform automatic model updates without stopping your Apache Beam pipeline.\n", + "You can use side inputs to update your model in real time, even while the Apache Beam pipeline is running. The side input is passed in a `ModelHandler` configuration object. You can update the model either by leveraging one of Apache Beam's provided patterns, such as the `WatchFilePattern`, or by configuring a custom side input `PCollection` that defines the logic for the model update.\n", + "\n", + "The pipeline in this notebook uses a RunInference `PTransform` with TensorFlow machine learning (ML) models to run inference on images. To update the model, it uses a side input `PCollection` that emits `ModelMetadata`.\n", + "For more information about side inputs, see the [Side inputs](https://beam.apache.org/documentation/programming-guide/#side-inputs) section in the Apache Beam Programming Guide.\n", + "\n", + "This example uses `WatchFilePattern` as a side input. `WatchFilePattern` is used to watch for file updates that match the `file_pattern` based on timestamps. It emits the latest `ModelMetadata`, which is used in the RunInference `PTransform` to automatically update the ML model without stopping the Apache Beam pipeline.\n" + ], + "metadata": { + "id": "tBtqF5UpKJNZ" + } + }, + { + "cell_type": "markdown", + "source": [ + "## Before you begin\n", + "Install the dependencies required to run this notebook.\n", + "\n", + "To use RunInference with side inputs for automatic model updates, use Apache Beam version 2.46.0 or later." + ], + "metadata": { + "id": "SPuXFowiTpWx" + } + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "1RyTYsFEIOlA" + }, + "outputs": [], + "source": [ + "!pip install apache_beam[gcp]>=2.46.0 --quiet\n", + "!pip install tensorflow --quiet\n", + "!pip install tensorflow_hub --quiet" + ] + }, + { + "cell_type": "code", + "source": [ + "# Imports required for the notebook.\n", + "import logging\n", + "import time\n", + "from typing import Iterable\n", + "from typing import Tuple\n", + "\n", + "import apache_beam as beam\n", + "from apache_beam.ml.inference.base import PredictionResult\n", + "from apache_beam.ml.inference.base import RunInference\n", + "from apache_beam.ml.inference.tensorflow_inference import TFModelHandlerTensor\n", + "from apache_beam.ml.inference.utils import WatchFilePattern\n", + "from apache_beam.options.pipeline_options import GoogleCloudOptions\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.options.pipeline_options import SetupOptions\n", + "from apache_beam.options.pipeline_options import StandardOptions\n", + "from apache_beam.options.pipeline_options import WorkerOptions\n", + "from apache_beam.transforms.periodicsequence import PeriodicImpulse\n", + "import numpy\n", + "from PIL import Image\n", + "import tensorflow as tf" + ], + "metadata": { + "id": "Rs4cwwNrIV9H" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "# Authenticate to your Google Cloud account.\n", + "def auth_to_colab():\n", + " from google.colab import auth\n", + " auth.authenticate_user()\n", + "\n", + "auth_to_colab()" + ], + "metadata": { + "id": "jAKpPcmmGm03" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Configure the runner\n", + "\n", + "This pipeline uses the Dataflow Runner. To run the pipeline, you need to complete the following tasks:\n", + "\n", + "* Ensure that you have all the required permissions to run the pipeline on Dataflow.\n", + "* Configure the pipeline options for the pipeline to run on Dataflow. Make sure the pipeline is using streaming mode.\n", + "\n", + "In the following code, replace `BUCKET_NAME` with the the name of your Cloud Storage bucket." + ], + "metadata": { + "id": "ORYNKhH3WQyP" + } + }, + { + "cell_type": "code", + "source": [ + "options = PipelineOptions()\n", + "options.view_as(StandardOptions).streaming = True\n", + "\n", + "BUCKET_NAME = '' # Replace with your bucket name.\n", + "\n", + "# Provide required pipeline options for the Dataflow Runner.\n", + "options.view_as(StandardOptions).runner = \"DataflowRunner\"\n", + "\n", + "# Set the project to the default project in your current Google Cloud environment.\n", + "options.view_as(GoogleCloudOptions).project = ''\n", + "\n", + "# Set the Google Cloud region that you want to run Dataflow in.\n", + "options.view_as(GoogleCloudOptions).region = 'us-central1'\n", + "\n", + "# IMPORTANT: Replace BUCKET_NAME with the the name of your Cloud Storage bucket.\n", + "dataflow_gcs_location = \"gs://%s/dataflow\" % BUCKET_NAME\n", + "\n", + "# The Dataflow staging location. This location is used to stage the Dataflow pipeline and the SDK binary.\n", + "options.view_as(GoogleCloudOptions).staging_location = '%s/staging' % dataflow_gcs_location\n", + "\n", + "\n", + "# The Dataflow staging location. This location is used to stage the Dataflow pipeline and the SDK binary.\n", + "options.view_as(GoogleCloudOptions).staging_location = '%s/staging' % dataflow_gcs_location\n", + "\n", + "# The Dataflow temp location. This location is used to store temporary files or intermediate results before outputting to the sink.\n", + "options.view_as(GoogleCloudOptions).temp_location = '%s/temp' % dataflow_gcs_location\n", + "\n", + "options.view_as(SetupOptions).save_main_session = True\n", + "\n", + "# Launching Dataflow with only one worker might result in processing delays due to\n", + "# initial input processing. This could further postpone the side input model updates.\n", + "# To expedite the model update process, it's recommended to set num_workers>1.\n", + "# https://github.com/apache/beam/issues/28776\n", + "options.view_as(WorkerOptions).num_workers = 5" + ], + "metadata": { + "id": "wWjbnq6X-4uE" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Install the `tensorflow` and `tensorflow_hub` dependencies on Dataflow. Use the `requirements_file` pipeline option to pass these dependencies." + ], + "metadata": { + "id": "HTJV8pO2Wcw4" + } + }, + { + "cell_type": "code", + "source": [ + "# In a requirements file, define the dependencies required for the pipeline.\n", + "!printf 'tensorflow>=2.12.0\\ntensorflow_hub>=0.10.0\\nPillow>=9.0.0' > ./requirements.txt\n", + "# Install the pipeline dependencies on Dataflow.\n", + "options.view_as(SetupOptions).requirements_file = './requirements.txt'" + ], + "metadata": { + "id": "lEy4PkluWbdm" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Use the TensorFlow model handler\n", + " This example uses `TFModelHandlerTensor` as the model handler and the `resnet_101` model trained on [ImageNet](https://www.image-net.org/).\n", + "\n", + "\n", + "For DataflowRunner, the model needs to be stored remote location accessible by the Beam pipeline. So we will download `ResNet101` model and upload it to the GCS location.\n" + ], + "metadata": { + "id": "_AUNH_GJk_NE" + } + }, + { + "cell_type": "code", + "source": [ + "model = tf.keras.applications.resnet.ResNet101()\n", + "model.save('resnet101_weights_tf_dim_ordering_tf_kernels.keras')\n", + "# After saving the model locally, upload the model to GCS bucket and provide that gcs bucket `URI` as `model_uri` to the `TFModelHandler`\n", + "# Replace `BUCKET_NAME` value with actual bucket name.\n", + "!gsutil cp resnet101_weights_tf_dim_ordering_tf_kernels.keras gs:///dataflow/resnet101_weights_tf_dim_ordering_tf_kernels.keras" + ], + "metadata": { + "id": "ibkWiwVNvyrn" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "model_handler = TFModelHandlerTensor(\n", + " model_uri=dataflow_gcs_location + \"/resnet101_weights_tf_dim_ordering_tf_kernels.keras\")" + ], + "metadata": { + "id": "kkSnsxwUk-Sp" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Preprocess images\n", + "\n", + "Use `preprocess_image` to run the inference, read the image, and convert the image to a TensorFlow tensor." + ], + "metadata": { + "id": "tZH0r0sL-if5" + } + }, + { + "cell_type": "code", + "source": [ + "def preprocess_image(image_name, image_dir):\n", + " img = tf.keras.utils.get_file(image_name, image_dir + image_name)\n", + " img = Image.open(img).resize((224, 224))\n", + " img = numpy.array(img) / 255.0\n", + " img_tensor = tf.cast(tf.convert_to_tensor(img[...]), dtype=tf.float32)\n", + " return img_tensor" + ], + "metadata": { + "id": "dU5imgTt-8Ne" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "class PostProcessor(beam.DoFn):\n", + " \"\"\"Process the PredictionResult to get the predicted label.\n", + " Returns predicted label.\n", + " \"\"\"\n", + " def process(self, element: PredictionResult) -> Iterable[Tuple[str, str]]:\n", + " predicted_class = numpy.argmax(element.inference, axis=-1)\n", + " labels_path = tf.keras.utils.get_file(\n", + " 'ImageNetLabels.txt',\n", + " 'https://storage.googleapis.com/download.tensorflow.org/data/ImageNetLabels.txt' # pylint: disable=line-too-long\n", + " )\n", + " imagenet_labels = numpy.array(open(labels_path).read().splitlines())\n", + " predicted_class_name = imagenet_labels[predicted_class]\n", + " yield predicted_class_name.title(), element.model_id" + ], + "metadata": { + "id": "6V5tJxO6-gyt" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "# Define the pipeline object.\n", + "pipeline = beam.Pipeline(options=options)" + ], + "metadata": { + "id": "GpdKk72O_NXT" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "Next, review the pipeline steps and examine the code.\n", + "\n", + "### Pipeline steps\n" + ], + "metadata": { + "id": "elZ53uxc_9Hv" + } + }, + { + "cell_type": "markdown", + "source": [ + "1. Create a `PeriodicImpulse` transform, which emits output every `n` seconds. The `PeriodicImpulse` transform generates an infinite sequence of elements with a given runtime interval.\n", + "\n", + " In this example, `PeriodicImpulse` mimics the Pub/Sub source. Because the inputs in a streaming pipeline arrive in intervals, use `PeriodicImpulse` to output elements at `m` intervals.\n", + "To learn more about `PeriodicImpulse`, see the [`PeriodicImpulse` code](https://github.com/apache/beam/blob/9c52e0594d6f0e59cd17ee005acfb41da508e0d5/sdks/python/apache_beam/transforms/periodicsequence.py#L150)." + ], + "metadata": { + "id": "305tkV2sAD-S" + } + }, + { + "cell_type": "code", + "source": [ + "start_timestamp = time.time() # start timestamp of the periodic impulse\n", + "end_timestamp = start_timestamp + 60 * 20 # end timestamp of the periodic impulse (will run for 20 minutes).\n", + "main_input_fire_interval = 60 # interval in seconds at which the main input PCollection is emitted.\n", + "side_input_fire_interval = 60 # interval in seconds at which the side input PCollection is emitted.\n", + "\n", + "periodic_impulse = (\n", + " pipeline\n", + " | \"MainInputPcoll\" >> PeriodicImpulse(\n", + " start_timestamp=start_timestamp,\n", + " stop_timestamp=end_timestamp,\n", + " fire_interval=main_input_fire_interval))" + ], + "metadata": { + "id": "vUFStz66_Tbb" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "2. To read and preprocess the images, use the `preprocess_image` function. This example uses `Cat-with-beanie.jpg` for all inferences.\n", + "\n", + " **Note**: Image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." + ], + "metadata": { + "id": "8-sal2rFAxP2" + } + }, + { + "cell_type": "markdown", + "source": [ + "![download.png](data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAOAAAADgCAIAAACVT/22AAAKMWlDQ1BJQ0MgUHJvZmlsZQAAeJydlndUU9kWh8+9N71QkhCKlNBraFICSA29SJEuKjEJEErAkAAiNkRUcERRkaYIMijggKNDkbEiioUBUbHrBBlE1HFwFBuWSWStGd+8ee/Nm98f935rn73P3Wfvfda6AJD8gwXCTFgJgAyhWBTh58WIjYtnYAcBDPAAA2wA4HCzs0IW+EYCmQJ82IxsmRP4F726DiD5+yrTP4zBAP+flLlZIjEAUJiM5/L42VwZF8k4PVecJbdPyZi2NE3OMErOIlmCMlaTc/IsW3z2mWUPOfMyhDwZy3PO4mXw5Nwn4405Er6MkWAZF+cI+LkyviZjg3RJhkDGb+SxGXxONgAoktwu5nNTZGwtY5IoMoIt43kA4EjJX/DSL1jMzxPLD8XOzFouEiSniBkmXFOGjZMTi+HPz03ni8XMMA43jSPiMdiZGVkc4XIAZs/8WRR5bRmyIjvYODk4MG0tbb4o1H9d/JuS93aWXoR/7hlEH/jD9ld+mQ0AsKZltdn6h21pFQBd6wFQu/2HzWAvAIqyvnUOfXEeunxeUsTiLGcrq9zcXEsBn2spL+jv+p8Of0NffM9Svt3v5WF485M4knQxQ143bmZ6pkTEyM7icPkM5p+H+B8H/nUeFhH8JL6IL5RFRMumTCBMlrVbyBOIBZlChkD4n5r4D8P+pNm5lona+BHQllgCpSEaQH4eACgqESAJe2Qr0O99C8ZHA/nNi9GZmJ37z4L+fVe4TP7IFiR/jmNHRDK4ElHO7Jr8WgI0IABFQAPqQBvoAxPABLbAEbgAD+ADAkEoiARxYDHgghSQAUQgFxSAtaAYlIKtYCeoBnWgETSDNnAYdIFj4DQ4By6By2AE3AFSMA6egCnwCsxAEISFyBAVUod0IEPIHLKFWJAb5AMFQxFQHJQIJUNCSAIVQOugUqgcqobqoWboW+godBq6AA1Dt6BRaBL6FXoHIzAJpsFasBFsBbNgTzgIjoQXwcnwMjgfLoK3wJVwA3wQ7oRPw5fgEVgKP4GnEYAQETqiizARFsJGQpF4JAkRIauQEqQCaUDakB6kH7mKSJGnyFsUBkVFMVBMlAvKHxWF4qKWoVahNqOqUQdQnag+1FXUKGoK9RFNRmuizdHO6AB0LDoZnYsuRlegm9Ad6LPoEfQ4+hUGg6FjjDGOGH9MHCYVswKzGbMb0445hRnGjGGmsVisOtYc64oNxXKwYmwxtgp7EHsSewU7jn2DI+J0cLY4X1w8TogrxFXgWnAncFdwE7gZvBLeEO+MD8Xz8MvxZfhGfA9+CD+OnyEoE4wJroRIQiphLaGS0EY4S7hLeEEkEvWITsRwooC4hlhJPEQ8TxwlviVRSGYkNimBJCFtIe0nnSLdIr0gk8lGZA9yPFlM3kJuJp8h3ye/UaAqWCoEKPAUVivUKHQqXFF4pohXNFT0VFysmK9YoXhEcUjxqRJeyUiJrcRRWqVUo3RU6YbStDJV2UY5VDlDebNyi/IF5UcULMWI4kPhUYoo+yhnKGNUhKpPZVO51HXURupZ6jgNQzOmBdBSaaW0b2iDtCkVioqdSrRKnkqNynEVKR2hG9ED6On0Mvph+nX6O1UtVU9Vvuom1TbVK6qv1eaoeajx1UrU2tVG1N6pM9R91NPUt6l3qd/TQGmYaYRr5Grs0Tir8XQObY7LHO6ckjmH59zWhDXNNCM0V2ju0xzQnNbS1vLTytKq0jqj9VSbru2hnaq9Q/uE9qQOVcdNR6CzQ+ekzmOGCsOTkc6oZPQxpnQ1df11Jbr1uoO6M3rGelF6hXrtevf0Cfos/ST9Hfq9+lMGOgYhBgUGrQa3DfGGLMMUw12G/YavjYyNYow2GHUZPTJWMw4wzjduNb5rQjZxN1lm0mByzRRjyjJNM91tetkMNrM3SzGrMRsyh80dzAXmu82HLdAWThZCiwaLG0wS05OZw2xljlrSLYMtCy27LJ9ZGVjFW22z6rf6aG1vnW7daH3HhmITaFNo02Pzq62ZLde2xvbaXPJc37mr53bPfW5nbse322N3055qH2K/wb7X/oODo4PIoc1h0tHAMdGx1vEGi8YKY21mnXdCO3k5rXY65vTW2cFZ7HzY+RcXpkuaS4vLo3nG8/jzGueNueq5clzrXaVuDLdEt71uUnddd457g/sDD30PnkeTx4SnqWeq50HPZ17WXiKvDq/XbGf2SvYpb8Tbz7vEe9CH4hPlU+1z31fPN9m31XfKz95vhd8pf7R/kP82/xsBWgHcgOaAqUDHwJWBfUGkoAVB1UEPgs2CRcE9IXBIYMj2kLvzDecL53eFgtCA0O2h98KMw5aFfR+OCQ8Lrwl/GGETURDRv4C6YMmClgWvIr0iyyLvRJlESaJ6oxWjE6Kbo1/HeMeUx0hjrWJXxl6K04gTxHXHY+Oj45vipxf6LNy5cDzBPqE44foi40V5iy4s1licvvj4EsUlnCVHEtGJMYktie85oZwGzvTSgKW1S6e4bO4u7hOeB28Hb5Lvyi/nTyS5JpUnPUp2Td6ePJninlKR8lTAFlQLnqf6p9alvk4LTduf9ik9Jr09A5eRmHFUSBGmCfsytTPzMoezzLOKs6TLnJftXDYlChI1ZUPZi7K7xTTZz9SAxESyXjKa45ZTk/MmNzr3SJ5ynjBvYLnZ8k3LJ/J9879egVrBXdFboFuwtmB0pefK+lXQqqWrelfrry5aPb7Gb82BtYS1aWt/KLQuLC98uS5mXU+RVtGaorH1futbixWKRcU3NrhsqNuI2ijYOLhp7qaqTR9LeCUXS61LK0rfb+ZuvviVzVeVX33akrRlsMyhbM9WzFbh1uvb3LcdKFcuzy8f2x6yvXMHY0fJjpc7l+y8UGFXUbeLsEuyS1oZXNldZVC1tep9dUr1SI1XTXutZu2m2te7ebuv7PHY01anVVda926vYO/Ner/6zgajhop9mH05+x42Rjf2f836urlJo6m06cN+4X7pgYgDfc2Ozc0tmi1lrXCrpHXyYMLBy994f9Pdxmyrb6e3lx4ChySHHn+b+O31w0GHe4+wjrR9Z/hdbQe1o6QT6lzeOdWV0iXtjusePhp4tLfHpafje8vv9x/TPVZzXOV42QnCiaITn07mn5w+lXXq6enk02O9S3rvnIk9c60vvG/wbNDZ8+d8z53p9+w/ed71/LELzheOXmRd7LrkcKlzwH6g4wf7HzoGHQY7hxyHui87Xe4Znjd84or7ldNXva+euxZw7dLI/JHh61HXb95IuCG9ybv56Fb6ree3c27P3FlzF3235J7SvYr7mvcbfjT9sV3qID0+6j068GDBgztj3LEnP2X/9H686CH5YcWEzkTzI9tHxyZ9Jy8/Xvh4/EnWk5mnxT8r/1z7zOTZd794/DIwFTs1/lz0/NOvm1+ov9j/0u5l73TY9P1XGa9mXpe8UX9z4C3rbf+7mHcTM7nvse8rP5h+6PkY9PHup4xPn34D94Tz+6TMXDkAAQAASURBVHichP3Xt2xJeh+IfV9EbJs+8/hzvalbt3x1tQe70WgSAAnQDClySGo4miXxbV70oLX0b0hr6UEPkkZrRmuRHJrhiAAJEoYwDaIb7dDl63pzzr3Hn/SZ20TEp4cwe+e5RSq7+p4020R88YvfZyM2/rN/9zsMERAAgAgAABEBAADcH/t39Ut/MCABIBIRgL0OIhIRIgIRIgAgubPMO3MUoX1P5kJE1V0QQWtg6C9KRBcaYP+PSEQEhHChnQSABAQEgIgIQIT2eEB3O+1awMwJ/nxC00CC//8vc1bVPgJ/Ipo2ABARs6Kyn22/AAgIEY1k3O3Inou+D/Y3K9gvawTCRVEQuS4wrLXKtsi33bZKmxPJDgfU5I5V27B2Cd8OTVZkHhdE2ouydiBBbSjt+1Upk/vB/BVGMmgHnGzLDODMsPqPtfFAspgjcPK4iGvzHpFIuzv4MXCDY2/nYe0HAAEI0c4D/6sTKVENjgTMSXlFHnYc0UCVABmiF3c1Ydz40MooVB+qMas65GaWux4gkiZEtOCwLQRw6ERE8211LS8tRNDmW6yBD8HInqpbmIbY8UIrPCsef6wfXyAgI3QAIIsgK2QkM9DaSd53zQqPwLcEEQE0EFY0U8nCDiAB1qQO1a0MTJlFnAcUMjKD4MZ/RS5oIWxELTwy0FEa1DBOfgo5jJKdI4YgbfcJ7YXRCsl1zw6UQ4oXMZHrC/l54W9UcWrVNqBao9HPInBU5KYu1eRMrrNYURzWoWbpybUKyE2SCql2kLz47Im0Ipn6qNUmT4XBSuLVqDg025G9OPqORGsaDP0x5KBG6IjTH2PbSVBrl1Vz4JkT3fRZuZvneys4cNIwI7cqHKxRVG1WVw1ZYU2wDcKKIiwFoMEnICPSgCunIyKrtbDSUQjAABkBA2CrOp4BMsd/tdlee2deBEjEEMlCzoPeAdad4KYa2RNrbbBvCMxVGCIAMEMClhgJALVlZXSzYHWeIVYD7fSv/89+rBG5GQD0TbCnIFZIc+A17SVws6zqnZWQ66vnJY8Nx4U1jWwJr5oDQNWguFPs0NQEVaETERkgq3Wypp+djqxdzmCsrhsBfAvAmW019UzGgvAz58I5Ncl7QToGcx3XRhV4kwYAPMUiYk3otoNsFT1VOwmtfMndiqG/rMcSrp7lB8sKjFzH6iDwl6gI6UuuUGu/w0DtG4sL5i5DBscAjKzSR6ia6AYVEIChRTw4c8LJDo2OxNW2WCPJcXBl7dSGhyGaMWJ2grguERhTyCLP/0Z+9Kz0EYDVeL4aXrTWQdVpOzxeSismqYGPvVmNxqoJWZuZUB+Xumirc+qnrwiGKmvCWlDVvfwkdV/WXBwPGnO+QZmG+ssSn0Ukq+upisIqAgGOyFYnWgUXrJ1iIeJAzBz+aq32H/3twLLIxQasYhrN6FUidqNnNaZrngZS3iCqAYgANGntNQTWpofnYSBk9tJ1TNSuY8zIWjdq4+UsGqiEazkU681GNzLOdKumnZkJiMzCnGEFBqjo0A+Bx5GXniNjWuFdcy5WcPFS8K/qSwcuzwf1Qb9gNKxcyp5BNYVPABqcMVqHZiXBmuKtdcga60YoosYOq0aVtfB8c1bHxBmK5sW8MYe2gXXLFZmf+mS42YKPVgyl+ssZtbY1K9Kxk9fbwn4QrMeqrZvlrSkvddMCc5gb1pp2JyIEBpW5y6xBaZFMTpOvkqzzCqp56uRLTom4A9C2k/nW1k6z56wwaG0M7W211hZUXvwV8/pjV9w4g1qnaMFRwoXxRCIC7ZtcXRj8vZz/QLXWVoEC8AyP9U9o5zwQMYQqMOImVTVBViaDa5aAyouvjCBy6s9JcGWWgcOhJ27vXWG9LW7swPlADJCM2UtQ98l8c6GmdiuIr0qp1iEfgbH2KzmWZYCOL11YgBzHOF+ypqexGjE3sM6tQCsEz9++PWYK4IVGec+ZauC2SHFNNW8RVnmoenl311JZxY6VqnGMwLzbsaKjKvb1DSATs7pgV6LDiGs2eLD6sSB/oO945dza+9QvB9oaOwBE2uk7BjbmgH5GW22+QtXoohOWHpnviTV2KvMIkIDXBsTadVS11MCXMXZBzI7SsS4JRLQxCERC0CuTpzY2F6/jfqkrW/uNu4kdQj+RUJtfGTorwnXLdhGsCrCGkJ916K4LjsTRqW/ndtSaWtOkNaPMDicyrNxLE2IxHffidFJG31Nng3ioOWPSXdtLYGUaW0l/Cda9MWrOXiHZVWiRNtY2klUg1S2qLoDzkwDAxZjQAcO1zzRF+wFBZIjcdquunA3EaLWNKz8iAoiqM/a+tckH4AXqNYUNctcutjKDa7EJ86WdIXZeQCVuhAtT35zAvNCdiC2B+fiEH+GaO+mJ34LVaQFnTlrNhZVycghw4q8UGkPSNrLt4vxewyAjy40rto8lW9MjYoZcrUK0POx64Qfcwh2p0qoWqz7wSXRhUNz9nD1qcWdMCh+/gy9LW1zsdKUhLZPVLuvGzjWz+saLxKgCN6CWC8kHRKsGG8TrSuujuxR5htJfpkdsX4RVjlXTyPN+TQtaIjF6AgCZ0eyvqoNX57E7wrET1rBQYyQHYAepSjDoEj+V2nOdrIte12TvXWcrympkVsOQWL+5HWhy0RwN2t3Qm6FARMZjW417VPepUbQzvmoWoe2XYStHMdWEdKoP/HxzXfSZOh/5r+59AbXGeKpM/BWXyd6uzvUrP7tb1O0qrExFGzdwTcVqNvmsIfPsVHXFslLdEvDviFZasNJUcEThCLHS7g4iNcohx4jM36PGczUX114daxqhjkj06SqyQR8A67QyP0iGh4x0GVKlwoyRt8oPNU0F4IOFYJzy2pFkjRQ3jk43oTPprMZyagGt+nQhN9+FmhTdl4DMRWqrcUArTJ9W8m11t8XVy6ATpj/fDS0iAOgVR57cXK6NN9p0gxuzGgM64gRwwS+y2qyKpNpZ6uaSmYy1/tbEYBR0TSPZ3lZQrOYB1c6n6gh8BZyI9T8AAKKmD9AzXO0M52QAAoB28LHIsdqkAqBpen2ae3qw/OS8GXsNp0BruRDbM9TE0Sejq9/QBSMd0Xg95Nvm3WWnRV2a2ArJd5+qP479UFfBkUoEULu0U/KV6oNVcw1rJwEAECGgrvl8tr8rg+0ZF6imZ8gRmLt3pcfrkwOq36nGpeD6i0RU5THtMNiuX3AELjLlBS1fM4tesQqs0q4JrlID5BSBgzV4lWHvYnNGK3RGRKI+XLUqA0AiYnaO+ytWt6GaCqvBxxmgNdQ7M7QSi0sokzsbYWXW2ps6m94lVw1xIlgHq67vap1y/1pjzBCzs+XAWn3kZvuqiLyh4gRIVir22uTi+37G2hdDIPIFE2TJsqYM0TceEcEk7ldVTjWTPStWVloNgx6vaO+8gpJVX95fuvrtohPv54YJGFXM61T4xUvR6qe6mbBid5jGEhEgA9KeJmoNsoKpCBdqdzd4Jc3sQZ55EEyG0waKzYT2YPNE467FAH1I2SPFyxSppmfRKkHjV+Mrwje2o+VEBLAgRjchABCIISESrBSIVPzmbk4ezXRhSNALcsULr4ubAMgOGNQNa2/DOt1c6acVVWeu4SI8K/UfAACga+G1uv706MRaY+oGmLt07Ve40IdX4yI1qay6VmiHsQ7ui69Xvof6ePnbXRQjuFFAQOZu6rujbatthQg5KazC3JkBorqgs3JYfSJQpW78Bew/lp5qsTZnXTPP32hiYisT0ZCiV1Xo/JVKC62wEyEAdwl3G5ZAJG2igACrI+rnIDilie6CF+jKcZj3Ro1N4NMxrr9OHVRcUaUdnHlDfgSweuubVOlAZziad1QdVJnsTiDOu6prrsp/8a6tlznW7lgHUDUPV5FVfeUUXV04vq/1l6eDCgnk4r71bjuh+p77QIUxbCofAPwQ2Pa493buM0ThjU4TSK9ZPwSANsRJxrolBrZMxjGGLZmstDhYlFteNQ3yjXDq3N4OkQi0uWzFIC5XAd79RjRBeT9dqSZa329wISrHkxV7XeRKsLzsZpqLuPtP1WiaD1UcwqGEHLH6SVYB1dl3KxNtldouxHesjGslIljLHrhv3Mx5hSYNXrXSF+9Sh1pNBAgVlNHJz7d2BdarMvOjAwCgq7Pc0XaukfMxLWM7Dc0Y0+TI04q7GlU7ALUbE4DQldVlhepiv15X2hGx3qizlS4YIzWHtKJPcl0Cr+UulJbWJ5q5lyM/p4Yr0DGHhopRfD/ANm/FyfNdcCzlm1u5q/U2+0m2ah3WvIXKlPDMVrXKIG7FFFiNDXqS9WlPp838B6w3x4p21TR0XhSsvrT+EnSuQtlpFSJH5EAARBoqVodXX+hqzcw4sKrOrvJg3XBVpzhDwt6W7DiAK1oCXAk8+SGtcTMAEYmqDxVMfdKwrsKdTVLZ4dY3NCZlBdlKy1paR6xoHWoq3BzPTFn7ilqtTRKn9SrFWnWHfMvgIl+B9wncEXXZVxMSHHVZOXptZOMHqwzsu2475+ewvzr6fA1WCsuOjFN4tUbAynS9ACnvwXhKqyPVf3+RUJ20Lxy8ckylD6zYvM1jBeJrAF2g0LIMVQDCWu02eYr2sqhg4Fy7ipaZayf6Y5xgrEIC1zXjqDvCcgMGKydWB6xOdKN0kBGgdm6MA7EZS+ZPNO4XVnks6xIBkC05XXGi7d1ddT06UTnh2pwhWfJ0QjJmADq9VXP+6yrWNMrXwmJFcUZq5q09x5kPXiN4saD/tzK1jOZf8f8qrJi7uuGqtK5v2avodBem+psa7mus409xDm/d+7FOur+i18v+zFq62rTvS1tVNchb+eiJAXFVesYFcpFU1/PVqxHU7Sn72c8KAGSrVrVRWlUSuebMgbeQ6vaMMSjc5LOpT+bTJOTQ4sRUHzQvAxehA49R9FekSuExqCpMnRFoNbKXOpqlHd4CuOBmOlOdKm/Dj6sbt1VAeCUF1vKs7uQMLNdARMYsN5ATXcUdVf2/u8KKIGtDgBVFrQ7NxdJPU88AlqFXzMqLp1eYISedSpWBo7bqfJfErqYWwAVD1pv7VHuZ08kreGfXkDXDmL+NWbTkmmjzKibMY7GPCEAMrRqtKQV7J6dZai3yxzicWHFr2yUbNTAFqEgucWBZqqYpVl6OwKA2+Rzuq3nsZ4G/O1q+9DyEANoPk9c1XjsjIFQrCMyksUq58nFq96zcoMq2Ii9Uj4aaiFYzPW4OuLmHYIN3hrMtj7tX1aj61VfE5DBR00O1H1d65tpbI3iXP3Etqt5ZavcGja7BvUbD5gQLxAopXmqreqreNKehicjHeKAKNiIAOb6qjR4BIIqa3lztK75C7oh1bq+pM4DKlkJXEvKK0VeLOjDjWtSq0+sHoHWqEQz0ybr5RizMpkur0fISJHdkHVTgRtTTrXtPzltGJw7bPquf7aT3g1ybLOCMbrJXMHRkqyJqp2rDWL4idpWqzBfeNSR/WSunFZRWCueC2eoU1MUTVoQPlnHcLPT5sIqGWDVHvYnpPYG6MKu2X3R5V039WsPJ1toCgY2Dej0HFVycTnLfoikWqfcWoKYyyY2aw79BgWuxuawfbj+UK/asuZsCDVb5gvcevI7Q9SF3bSOqFj35yFRlezhp+EY4uYM1RK1jUdnT5LtGADWZO6MC6sPjpoO3n4zxg+B8BqiKOhyuSVu4VVdHrLwmB0E3AyqsYjXwFt8rKc1KJhUOakPmD6op+BV68KKg2lkXbk1ElRzd5az/pPUFw7F2DPnTa9qmIkxwQ+ksnKorPgLgZr+TCQIAc0FFEK+6eBeYyePN3xMdD1XyWJ3TlaJ2ROeNB3S6vFrJ6kspbHIfyCwt8oyLzkqoId9cnrmWubkP1veqW07Wd6rG12YTmF/uUo2UBQdU2DEKyPbb9qVmjFVThhBqJdhkKrOp0hpA6GJS6JgCnbnm4bhSmuS5ocaaVB/kVbCuGp2VpF6Fuzf0/U/oxnDFKnCX9YFYkx/xP5gBrYjDOXHeHvBWTsUP9eSGE40DazWVrTpCFL65vkHsFbyaUarPXy8uN1fAfnthqiFAtc7QDr5vnGkOA7xwRs1yBfCAqheNmFt7oiIA5iI/tUOs9C1sa2e5X1eaCbU2vaJnKwHUPtcmEVVXIQACZMwMle1Q7farI+Mv6zFa1XTWKjw9bqi+DgEv5NBXm10P5l0k44u2WwWymmtfb6qbpWiVD67Axk1rL4NV2brEksGcFxiBcwhXNYGLJiMC1JwG31Xj6fjjV9EKQKSdarYH2JynP74mQavk/Ky3NYJ+7nvmswxVK5jytrR2PeeA9WXQxqm22fkKzmi9AQSnVmoRUqrE6iRCta5UMnIcDPWWGvlWysoitBZPcCxoLobOCwNEs1uBu5rTM26kV6jXt6Cuyv0QXtDvK15szZFftTL9l5Ukav6WVXHkvNzapPPjWHtfN0Jr4jMjiL7pVawJHAwNFIicU26vsDJ5bBfd5HJLPjzmvKVQV+JERKSNHL1KcwNPNYvrQm+shvdHEgGCttFL5x/VMOqrRTx32gahNxaMFFxAEaz/zQCYn97GIHIzAW0b3ayo1I2b/dWV0V2V3Kx3CqNyLiutbgYXXQS2ukI1qH40fUiaavfy08DwsSdCN8ArPOSIFi6+XtXydVr1JoTx2KpBraz5utKujeBFVK5onrpercbFCMWYs1Zr4kW95ia87aphHkCwa/otnfkyBwtQ5u21SklYFq1nUXxrPIUS2dhKzdbyU7Am31pNgI8Je+WOZOHlpoanIDQ/2Wag5S9mDE1HUWbiWkI17VxR9DVtgIZQWPWxzlXuSGcuufCi740RKlVaE2sNtgNjruLfO4cLyIT00NGM722N3V/hEg9716RVZf2K4qrmHmKVz3SvV/Vv/XVB718wlpxfRX5kXm2qlfwFB9S/BRsTcL5TnczNZVfsX/NXGA9BW4fXmHLerDbmgzGlK/PIX1o7oKK9v+dtu4DTfVztSN31czLWCGgKNwlWnA2oT/AKwU412C5hRf0rwnNmqFuLvKLWqU4GVFWSr2b/rPdDNW3kysN8gsHalq4S1BqxHtG2Zdqrb9+1uklQp6gap10A1Kp9UiM2x5SWuhyW6p7QhVOsUn6lDLQaKYDqQr499RGpCbyaX37+umnvRw9sl8n10hhj5Gax1zaeGZFZ3e0miOk9EGiqxo/ckLr8kLkc+WSqKVGprDqvx4jcR2N42LbXZOwNn8qgIbfiH/xccxkG1xgCIiSbx3fGluPv2pSw3GML+Wvz8yJZ+Ex0pTRqNoDlL0f4Fz1ia0qYA1cBRH4CVzrolevV/oPK0LDTG9D99GUp9S9lshqPejXoJeKPctPnFXSuaH+yd0H4khs5RHmrsmq/+b7SMx4b1jAwhqEG0tVMAvSmEgEY9S5s+xEAkEgTugSvAbC5AQJpS3Lmsyt7RABysaGqEAkA0OYjagrS6keobghAzLhsrj++BMMfi64prjyOnEfFfDoObHfdncmbMk6MjhGhkik4drkwPABQHVYzb8zNwXrXNZokr7PsjHVhaAAy2zq69qObnA43WFV4+lG3zhNhrba/dhBRbbLVSGTFE6pfz+uFeocuzlCopFGnbK+VfLXkyglOvFgNvVNZda/DKQy7N1g1xGjULOnaLCY3ZPYvc6rLmHBAREgayVl0hibde/dRW7+HnIOEgKTRagyjzmrDa/4lBAIN5O/nceHJ1azl8GLxzfSdNE10BqsLzlnxomsfeHQ6nxvQmL61YanQuUoOjmepJi/bWu3bZTdQQ388YlV7X4WpEG0tgTEVnJIy7XQ2ghWDd7NoFVXkzsRV5q4gBMAY88C6cMyXMN9/5oXO6fTvjWK4cIBv8IV/fQO8T+J0JPiou5uermukLXVZmK6YdkREpAVWBKZtNNNaPnbDLbIkWs2sFceWkECb5cg2SoeWVhj5CDgikptW6NzeWtfQB6zrc86ebaqkzSw2R0NNIVZIBANp9PUdXiSGcR3fO9FDNWntHIOV+zp2cFUEPqFZ1xYmt4AVKVJVlI/1YcKq2Ssr5qqoUoUrK8RatmPldZHsoTZGr77oYkLyP/eq3F9HmXVevHCwvxdjTGtdk+bqbLcdqc6xGs9OWfM1AGlwYncAsjAQlXNX67PxGMjxW4267a8X+1YVR5oiOiRwaWhzTbKmkRGXsynAarAqfl9lNbyPr62B6MwKBzg3uwlMagxsUt21xR5B9urmpu52nupq4rR2e/UZ7PwGBFczapvukr727qYozDIk+Ru7s+uRf2b50Q6CBza4eCrYyeQA7eysVzyY2rh4WNCrduorqL3gZlUm1n/B5kFXwfAKWFeuU2uDwV8Feqd6nEjrtsYFPNlOG+AJfw83HOj64F3WlXaQb65T7uDu5t9aB5YIEN1ax0pYRITuf4AE2hp3hMhspM5ONa+pKx/IrSQhNzENrZmW+CVK2ql6oz7s8uhXyn5fHbkaFV3IirkRIjdCteHzDgB4ovUwtarK31Q7I9NJzImOagXOXl0a/sba5ITqVuD7Uj/Rf3Q2d2XJOBOsxotVrOG/+NKuDTUQ/5csCqrnzMjSkLZCtA1FdNuEW1E57nOWEBARCdAK3C5CRiGx6sp2fS05Brbt0y6uY+To4gz1lqJFp2tnFTfWnoXRF/LYZpvF42jNGPSDh4jgksCaKik4/8iBF4zjh8Dq0TcAQDT7770KTXJtqA+w7UG1pIGYLaomyyNQG1lcGY2aLVSjXqgUIDk8e15lUN3JN9j6XNWYr4DDHWbbUNUVOIvK//uqx4NV3Z1VIl/Cu5WAnMfpf/dBPa+OV+VpNZx1sqGqqGXMCrhCtuU0gx+HFjczAQBA6No5XozV8BIoZ1hUVGr1lzOnzEy2MKvW/1SD5jDqsOgCjE58bjaTE1dFUYioSXsM2oll/Vj0Q2VdJOdmVz2og6PqWPXOXsRbVDbTa4KmDvHAKhl5bK4UZa7MTfd/x6P1e7pgRk0uHjkeOm4CuJiMNVIc7KjWIW+AuS9r7rMRC658dBi6CKxqvC98U+t3bfRda10Gvn61Sjms/CE/92qsbXFRc3D8SURWxZOu8YJrvXFINKHPX/kZX8+21e1oo6S8biS7gJ385gLoM+4Ort62cSqwYiKn1ezmu5os3drHcph4PjMhUtMd26zKEndAMZKoPlWK6aJ+9EDxuoicejJjagREhHa5N9Z53k4MN5o1xnE/fhkJvEpC1cvVfLmTatErj3U3aq6/7nDvaNYogJxX4MDiW23TMPVz6yLyUvpyI9VpMts28AY6WIm4G1uGAoDarsrmFj4c5wNB5nvh5oibr34uVqzjw2PGHjJCQQDS2ulY1zr71k0yMl5eFTsyAQVA96ZqLAFjCGAel2FxAGBdHgsVUmaQ7MukZhCBMQC77WhN7lWays/4C+7VhbEEWEVK3TS3Ss4vgXCArcw7gBUGdy4Ueu6vq1fn29aYg1ysEd1MA3D7/FTfVA2tq2T0zqXjBt/n+sUJPCtYE2+l81hxR/1enjRIVxu0ebl5G4msR1gz0y9afpbinAI26t+FDKsIFPnpBHbZcZ26jNTMKDKs5GAeplTjWqiAau0OvyDVwMrcQCmjmD2JuPwokSbyMkEAZfvpLoHM7UNid+Wyjq/W5lvGGCIjAoaGMO3mPtqqZGdqVO5dXffXMPHlH6l2rMUSVOJmfjZ4jqX6mVYtMqpRu6Nc9EJ0NoNtqnc78BWjfqVtVq1Wprj/FT0OXqFS27Ia8znlAwSEjDkt5OC7cpIfb3BNttxYCbbePHKNdicanVyFhowEELVNEFfnVHdEBCIBXBipIGOOqJzRgd70NjF0Aqv+DVdbY8BckYiYk5fjXSs9czuTDTdJ/yrLhCC1BkclWmvrWiETnHPBOWOCMc4YcqZBo1ZGFsgYETFGjHHQhNzOZmYqQ1dTl25ymIhgDXernsGFjysc7DjLO8jVWebAGlW6UXLvHI5th91Y+AHxs8gw6KujXqnyVcX5qkJ3N1nRCeiH9EsDogY32kUi0ZGoOx1qgK5+81h13Fy7v7ODoZo5rArKVcrd2Xg1S916jE7zAAhVKjsa9kwCk4i3ZgGRX7pOdrJ6Oge/vgpA24y+9X+UUjVvirS2q9nMUVpr86/WWmmSWhtLxQCUMRaGYRAEURhGQRAFQSg4h4BAkyzQApRzLoiIkJGt5ajK1bwAqllZCdjHny6Mk51etVH+Eu3nTqupafexUqsVBH1iE3zLLiDEfFXxtgdkjZc9MtDbEDUurJuMF3SCw0s9gfwlx1RMVwnrS14XVD84f+kC6D0xO4vDuNB+LrHaQ+h07ZJGnuQmhZkFKObzBQGYImQyiXUiItCkwdWAGvCBVdlau9JQTaS1VlprAiBSWkmlEUBrJZVWpEmT1koRaa2kUlIpg0JFpElLJUmR0koRKSIA0lJqgEacdJutfrfbbjTbjYRRzCAEhgSaVElaA2OccwLiEIBWZtg0EUPGnJTr3i6Ck5gdBled5ATqe+fH78IoO2u3JkkEb+pRzWq3I1LdF83WHZ7DLcOt6DSXU6lFcNwlPNjJq9SaQqysuaovuFJtWPObLvo9dSL/z70ukHfVPKfE607eRSuFmN8qjaxtYztqL0FeKCst9d0nIHF4eqqJSJPSSlkgWn6TSmvQWpOSysAINUklldYladBaAWmtldIGjUprTZq0UkoTkDKRLQKlNQNCBK0tzxkAmC3kydpwIKUsZamknAZhmWUcMQ4CGQQ60GQmBihd5KA1IKcwAGCIjBEn0pqYzUe7YUXPGZXzawMBUE/2XBiPCpjODiQCt9ae6JURssis2JQcbNxlPOGhHQJ7nDXnoX5B0ySsobbGlNW4OUPMVxf4eB9Ahc7K79b/GSS6pNqXqH1ErbU1TAl8s2n1YzVZX+FRqBSYUwhA7umL5KXh7KKaTJ0yNH6c+IvPPlOktdbSlN0ppbQyjCKJFGnUWluKNY9DJQJQiAjELVUh5wztw7IY5wwJOOchIuM8YAw5RwDBuODmxRAZ40wwzu3WJoyIpCxny2yZLTVQGMWIjDSVSinTJIZaS1mWWkrOOJBGAsZQK8aY2ye6St/UctneC4aawCqIVGWaflCdYe1iG17KLvtSw2ltABwTOEh66kBnETjpV1WjFg2OiN2Yr7QTwJWr1p2h2hnWliB3NUMzVvXri0bzRRhdSLsbO9Gk170cTHPqhilU0Ux78cr4uKCAPMOS9dWNsQgMzVbylqG1k8bKfBMvjl8CczAB5IiaMYYoAELGGBMUMMZQMM4Y45wFXDDOGWLAOGPczFHOOeOcMyY4t941Y/ZXxgLOOGMaUDDGjW/ODHsyhmD8dNJUynKZ57PFYplnSymBMQVEQEprpTSg1qSkLFVRaMaACDnnSmhOmohpAm5IBd3WobWcvNfm3vyq2z6vDJj7vsaL5ELCZljoyx2O2mesX9iF78AbBubZZORrWQzIqhv40h034RBXblG7tYWUb3HNVrnwpmraqnnjZ9wFB3/1xEp3V5ijCtNVBM3WuGF1M3Q6xFbPMXc3++1FjVO7r7hz5ZrhM4aMIWOccWSMc8ZQMAbIiSFnDJEJhgyZ4Cg4BxcEMgJx33BEYAZ8BnsI6DeEcfEiRHNBZIiu3Aq01qWUQZgJEbClKOczBQSaSGkg0kohggZSUpZloRkHRC4DCjRpTVoR48Ye0mi8Om8K2j92eBkD68lZ4dVs1FcGcXVcycMCCejL3eGaWYn+LuDcT7COl0v2VosPnS9sdzJatRTrd8cauMz3nv/cBHIHuxZdPGPloyM/589e6FCVWa31EWyEa0UCDmtE3sjyxIsmaoPIjD4yc89mIaqQ5Uoc37dU3L5+gwA5IgCabClniGiZzizfMTFH5vddAIM931rkzGysY5G5kj90tQ5+3yKDTdd4sIar1sBYqSQXQgguEECTIq1ISyk554wxjVqRKqXUqBFRBaVSkmsbuTKz1xqZlUXoYt0AAOZhzL6itmZMeRtA/2ez0s7YW40xoR9cIlctZe4DaJW7T4IQ82ZFzfkwDXPjqX1hg8U6OUlWBkTVYIsEK9tawyqGBHv9L+P7evyoupG/qZ9Rq1rF0179ItafN5dy85Tqixc1gLaocUOENhpei4DU5oIhXjHoDmx00LAdmKQ+GvZjCIzZ+mDmaceZHxo0kOFK8M5P3QV2g+aF5QrJKj0LyDiBZgDEmOCcM64ANAERlaXUoTIxKamUZqQJlFIEWnCujXlKittZa29V1Wq6+2o7kljjpdVaAagNTDX0F1fzAFj146jXkl39flj3GzQhQxeItvnZuifrTMlajKC6mbtPjVNXScvGfSpqdwL31ii5ca3FLC4SKq7gr24ZrDRnxU4186xmkddVv+diJ0DzzpxONV7zt7GGgTvTm14EACJNG+TN3hW7yTGmITxrL5uLaCMTRjasiojA7EN/q+fOVdMCAVETWE/bJZycyQyaMaUUU8r6UMA0oo1REZlgFjA00VklpQaUQkitAyLQJsRkfR2sZaidbVPPeRos262ia4NtRV8HQT1iYgiSwC/EsMTnj6xsxBqNoRegCyMgoDeOLbNa/qxOdjTvFLSBGtR64gas1pTarb19XKnvFV6qRhhr+W+HVLTWkY9ouQmwMgeqnK6bRN6aMrVvnuEr46DiAMeZq1wMdevLhxcED5gpF3GGA6GjT6uyjUK0T1a1IvZPtNBam0chIkNknBt2dKG4+hYPhIxbBwa9dDSR0hq15pxLw9wAJnVEJJXDCJjnfJLW1h5FZUxPE2PV5GRBBLbFQCuKnhm9gs4Q+LJXHWc+vIc1vYHgRg9tssN71nWCcQOBhCiJSqXnpSIgqfSoKENr7iNn0A1EzFkoOBg5eHuUIfjgqysaomowTU7O+kW+M/WQ5EVQojdzbWFEld+yqtweq0kjOi+whuVagMny3MrtKiaqGyy2yhGs0jWttRh0ee4LM6ZSTGYmCIYINoJIYPdpRLNFqDMWV0pdjOCYQx4xxhCJMWRMIPpKOMM4HBxKLHAMl7jhRBeCI1JEzMSLrIvFCMHYpkor52QgIhKCVlqZ8KvWSilhph6ZJ0BY9XNhlBxp2VX2tW8qfq0FvK0qv2DZ+fY6FeDlXrsZESCWAOeZPCvkSVZIpV8ucwYwLItny+z1VmstDJ4tFuOs3EjChMHlNHq73xrEETP13eiKtrxt6ywQbxxo10nH5Gj+rOhlwzfgZ5EjtJVwmGu6m5+e9cnpzBqyqPK4qxiq41Gyyr5S3ZWfamdD9Wgh8MWu9taVavelw0RAJAQPHMd7i4V57kRE6zbUuJe7xI+ZWWicJEQAE2JyE4wUQ9SARJoZHBO5XZerdhs4Kq2ZQSgiMiYELxhqK3NGWiulgKEpvzD5KSWVVgrIZk7N0xlNAQqSsTeo7tPolUyF92+8PVl5AKb+1wxUXdlVgvfoQKdSwT5kWxOM8vJgoX4xXe7n5Wtp/GJZ/tHB2d+8vMMAn40mCpdf6YZnRfHT4fCG7r3RiP74ZPiz02ES8K+vde902pHggd2f0sKKKhihtTOqSWOa7cDg2mg1qOuy01dud0jfo5ozvmKlIK5eCirU1V5klXJl9PgfrDFji9uwbhzXo1D+3qyGh4pEAYTZU9Bk261zZOjT23FY2UJeZbKqQtR776ZPtrWmDWRntxWDcTYrNVrzlxmiAkCzAYrRAwRaa6k12YpVICDiDBC1Mll8pU0+n+z/TaQLyC8M8ZrL7Thfk2yd+KoPq06r0QMXvqyMeDMBEM6yMlMkNQHC0bJ4upTExJ+P57OiuJWmz5fFrVYzFfywoGku201xuZFKUqdK306SqVInhA0hNNGnw/lZrkNEjriRRruNOBGstuUuOKfMzpyVgIVtk1epNbeGISIoRUopzrkpaLzocrkhAa8bV3/03zqL3JUXrMz66joGZGgnjtM4hndr6HQ2qzuiNmbmhsLoOO2SMM7kQl+fx40NxGqVLAxtOIQ8PNEiEam2PR6iCSK4XtSMPMttruFgzWAExhhDBiYuD9oSu5tohps1aa2Mc6Q1aCCNNZbxD8y02sr/BRfPcErJDzm4GpsVNPgxc+LzysuVYEMm6dks35tn+8v8+TK/mcYa8EVRPsqKrSD42qB7fzrth0E/bq7F0X62vNPpXG00/uRsuFB0NW48Xi4Zsq93B52Ac9BPZvMfHJy/2UhezmdLpW42G391e3Cr2+LMrMujWqOdJ1U1zprH9aYSQFEWk8n45Ox0Op0mSYLILu9eardadeuxQsQF9e2MGPQ/ei70hFdX4vaCPodXv6iuyc+TKTqTtzr9glkmLNUwZ1YhmuyQUwJ+dMERKJrV6wx8N4CIbLmxScjVXDbmGlRvMtSnSQVy5meI+wxSKqmUIs21tqkBBBOd12TqWWoPEAULP+fLO6+0GkbyITHw2sTGSiuLvVLfgC6GVJ2liealmhZqoek4108X+cez+b3Dk6+s9UeKSlWOpD6dTIjzb3aai4D/fDonhC+U/MHx+Vf63QDhZZZ1gvDr/cFhtvjXe/tNxt/ptglxK0kvp8nxYnmz2eQIZ1n+o9PJ80JLwMtJeKMRBojCZE2AGDIC5986Kw2cp6K0yuez8dnRdDKcj89OTo7LUgVJIhiWs9Mbd95tNdvWG6gpblzVHjXYVuq/ApeNDVUAtiIj8FEj8H6pw/HKZKjF/ZzkLZz8FBAm2ehs4Qo3zHozoLX1ujz5+1VfjKFG4D6rROgvYAK0WgMwby7bhyvWnWXTDJsaJRKMG4s2YJyAFJk6FucEASEyYIwIbBWVUkbFO1CRBmKakNlyXnebCl7WiDRSXrHNV1Q3+GnphpAQl1KNc3mwKM6LcizpSV40uXhRSAHwm1cvZYg/G082wnCsqRFFqeDHUj8r5PU03Y7CoywPpLqWJNea6UFRTkspEa63Wv/w+lVQaiELjTjRuBbHaUSfnJ0d5vKtXvtUqqcno4NSDkT4rU6cCNZLEs44J1VqnQq2mYTdgHNnr2uCIs/zxXg2PJ6Mz8siAxbEcXrl6g2ttdIglUSZjw+fNa7fFSKoOMKPBlgoYW0hHq0ypEEsWTbSK9cgiz2H8HqMz4q5fpzHpfeuagreFCw7T8H+4w0rj24XtvOmmLml1lozEzdyKqHiWyJyQVoX6vMa6cKLuTCK/cgYmBQooiyVUorsniXGLjXsTsYGVUq5v1KEIeeccVZdx6ooqpQ7IFAtC14Tra1Xssay5RJNOJf6PC+Hhcylmin9YpFnSsecPVpkzxbLd9qtsZR3koRINxnrBPyoLG7G8d1BNyf6s9PhdhJfTZMux+Ns+cs76w2OPz073w6CS1Hw+Xg0kvp2q7WdJjpb/M9P9jbSxq2WjgVrJqkupy8X+bv9LmbZ3Wb6YDJ7kKnNNJAlU6DH2fLPjk97Ufhmu7mbhJtJ0Ap4isCWk3xyfHz4YjqbIxNRnMg8V1KJMMrzXBPkpQwD0dVKFpkBKFWYpAofiLWUhFd4VSmfobP6BIcqfIL2gZzWACCP75XsSC18BhVtgz/AkKbQCOgD12h4DsF6sgZAFdTJcgmgjwZXNmulBtBW56DddsLYqG7WVOkytLF3v1WYMRy5iyAoIKW1sVBtbIvZ+aC1zmVJi3khNVssgY04E4ssWywzpZSWstlu7Wxt97vddqtFYAIz5KkTELkLJeZKj7JyWsqlVO0wGJdKk95IotNc/fTkfJiXW40kZmwk1X5e5poEw6KUSymjMPx0PDs6PV+7uvtut3W0XA4Q16MgQfpkPOmF0XYcxYi/82Tvdr+znSbdMPxoMmkwvp4mcyn3hmNVyLjbTYVYRsnNgD85PN6N49NMjYpiLU4I8c9Hs0BTP2ncHgx++/GzQZz+rZtX8rL4i+mCR/GlNPro+OQjxot8eVvQr241OpBPxpOSRKc7KJcLqSQRLPNseHiU5wVyzrhotdt5USpdZ74Vf4nsGFUj4o+oRxRW8+ZW9TtF5wvoTa2f110VF1vDb8Utq13QfS/cvr8e4PalzdIf+zUCaO0XalS8XNnYzPkSFT2hjUFgbZcRtmIQg6kX8QEG43wwxjmwqu5Fa9sYIgSbONCkkUgr/dkXnz978kwpmeXlcpmVZQmAUqoSIGk01zc2vv7BB++89dZg0I9EoAlM6RYBHGby49FiUcqCKNT6YLmcSxkxcaR0qakjxHFe7u29vLo5SJLk0TJ7NptvxmEvDDnnL2QZhdFQqs2Qp+s9xvl/Op+cTGedZvP9XvuHwzHXcq4UAd+bTlRRrsXJN9b696ezo2V+pdFgjA2i8Eaa/Nns7Acn56+1Gg/my93B+neuXCatn0xnqMVGI2VIf/Ly6JsbG3d77YSx0e7Wz84nT+fLUqujXA7K5R//xV8U52fddvKd3f6vv7aN+WiumAiTBggpS8nEYrGYTmdZXsyzPF8uAUBpKKTsdTobeU5Nb15axemw4j0IN96v6D6bgrH8WLcNLQq9f+KDfQiepGq4ruH1IjqJEEB4uxacpqOac2R6YJxDi1MA+8B2RN8Td4qzPQHcNPLTjlbSW1iFwBnachPS2t4XgTPGEKV57hECEGmluOAAwETAkJli3TSKv/bee99+/71QYLZcaFmGYaiByrw4Pj394uHTn33x5J88evwf/uAPdi/vrvX7nW43iRMWp0eK/8nZ/GQpsdNm3c41rXdb6SBOzsvys8kEcsmD6Ea78e3XbxZaP1vKe0/2W3H8lUu7PxsOl3lZar0s85SxD7bXP5tM9ubLsdK/sr15UpY/n8wWSv/1zY1mwP+XJ/s7ofilqzdPsuLH55OlUkEYbUfBh+ej02X+br+FAf/sxcH6jau/tt7544PTf3F4+pev7WrGQ17+dDj8eq97o9V4vszeKPX96fgP7+3dubqNoH/2+PFkPNHD03gyvMKKv37r8ru3rz1/8UJK2eoMgjAOWcCV0oBFUY6n89FkEogAkGnSSqv5bCGVRO72jXMPr6+zoBkIH4r34ZaqsMIhEcGFgVfY0PmfgATKfOswfbFsqrJXa34/uYCa8EBHRKNAnaGADswO38ZZcfmsGt5NyNcarc7WrNL/jke9rWubYeLhPh1qVAZHxl31HwEoTcr4Ssb+AEacE0NyMVBEFjbbrThWOJxOJxzCVqfNGe+t7bz/3ge/cvjyP/zHP/mLB3ufjsZRFERJHAjOkJEQDalDScsghptv7DX6nz8tO+uD1zd7b7Waw0Qj4wutn07mo6xAwTe31iLGPz4ZZlINWo290SgbT1+/ujspJSPsR2JbiJOiGEupFf2d7bV7kzkCAuc3eu1lkf/7oxOYL5uE37117c1ee/Li8NNHz9/ovnW91VwsM6XVsNC3e53z5bIdCEySHKh4cTxpNG/2eg8nsz8+OT04n6go+s7Wxh/d++Lw44+bctmnrFku/8Gvfvfa7ubRyZkIEuRqMpk2GzoIQy7C9a1eb1DOFvnp2VmZZY1GEnFEwlmWl2UJvmbF/TFFyh6aKwxnQ0s1a69W8kI2a+n50cfIvQvv4vn2GBeKX3UD3I0qdJF/ErZN4tjISh3f4AKUAJ7MwNsnKzwKjvap7sVVN3fUT7WPAE7BGA1vU53IGHKGDEmTVtrsUmviSSYWZbP4WmtSSslCU5o2tNbHpyej4ZCIAPlsWfbXtv63/5u/8Xe+/0EvjXSpymVeFlKWCpVKApZyakzOwp/8UfqLHzQoG8/mP/3wi5PzaYzseLGcKV0w1mmlJVDAg1Gp749nu61mNwzm86zf6dxstRZSf3Z8dr4selH8cDh9r9O+0WpMNTyYzv7wky+y83FPhM20MRCiuP/gW1uD1zvNo6zsJMm3blxZSyJkvMgKPZoQgx8dnrJF/vx0xBgfxLGIo8eTGUh9cD7++fODnX7n8lrrf713//H9BxvL0eXl6QZkf/d7X799/dLZcEzIC6lyqaXWIk7iRrvV7THGo7R56dKltUFfkZZlQUBKKhHwOEnSpGkItD5YdVDWIy1+lGsaGS06/UhaU8GHP8wPzluqxanI+TIWMzVAmFONKWlDjT7s6ljScJsxSohcqY25LlttguvgCuzq8wEATMUeVZEsHzNykclVZWGSSQatZqWUTxeZ85BzYEiuyomIiLQGCOLm1sZW2mwWZTkdj4UIGs02D2ISjV/61rf/q19+v9+OC6nyLFdK5YssL0ogEEEQhEEyPmn/7E/ig6da8P3ZfCHlWhTebSRrUYCE+eHx8Wiy00rbafysKJ8u8tuXtjd6ncfzvET+j+7e2mq3fvj0YLfRuN5uX0+Te8PJ+eno9uXd/8PX3w04+3S6yB8+SvNlnjYbYfB4vvizFydXO+1C6d1mM8iyX+y9RGDv9zvnWfbnL4+HpfqT/WPJgv3R7F/9xedBFP/GzvpXOw18+fzwP/77wd5nV2LVbkZ/6f23X7t5czKZZ3mZlTJJG51Od2NrhwXhIl8OpxOplSyy5XIZxvGg12EcNcE8K0ql02abB6FFnTPT6tyJF1iqBlX71423c3TAsaP5T5N9KqXVzn6sL+LjwsVdXZVDPQiD+9r/apaEK10jUpaZHeTRwdZAyVycwFbemeXv6PS7bZq9FgBW6+YAwCxPNv4+AXCzWIQxk3GVSiljJBCgJptFsLaBJtKgyGa9GCadQbcoRsPzuNWJWx3ORZnnRCgl3X3znUYj/eGH9754fjJfLAVjXKowChWRJCBkLFt2P/qhvvra6Pqbe6XaXOs90jCV6mw4ub6zs9lMZ0WxjthP4stpdH883R/PfvXapWEhrzTSPzs+H8Qhj6OhlM0o/Pzo5FYzvbOzmRBcbiTHUuk0vvz+ezHC6WKpBH+a56+V8nQyHc6z9NKV0enZv77/bDMJi15/VJb//smLv3p1a7GYH7caj6dxJ4nf3N0a51mM1B4eru0MGo1oPYq/8vbbZ+enWlEUJ8V8UZRqs7++WCzCMMnL5Xw6ZWQWIOSqyIi04EJrUlqv9bob69uMcY8LeqWi2dNqhVTnTtiojVvg4egTKgL0lmC1SLFyVKy3tMpnHrXW4fLeC5FA5ySZiCZaRV7FjKByyVfAjz5XSWSr5SurwrXbTSi0hgn5yaSJfATU+PLovHizqQgjsLuEABn+RCBuwlCMEUMCWwSq7R6inIfRzuUba+vbwBgQlHmGZu8dFhBrXL319s7ulSdPHv7gF/efHg6JSAOVpJVSoDUxZEyw5w/ak9Hk6t3DNKFlWU7nDOnF2SibzDe2N2Yo1gi+GE6vd9v3xvOzvHw+nl5Pwm6aTjQ0EQ/mixKI0uTxIlsMJ3/88Nn1tf5RUc46a41AnGbLH7w4FEm8EQanZdlJ40+OT5iOwjR9Y2Pweqf1cDT8yXiaPTscb/S/c2nn/mgssvmTZ8//jFQrTXZvvrb4eLPTFDHAW6/dnExGRVlGcTNMmlu7V4MgQsYG61ukdZ4tVWegSZ+dHIVhNBgMnj2dSSlLSbnUrUYzbTTBaS1HiBVisF5O6iHr/1S5X/LHw0pxI9aQYpx3unjNV10lspVKzgy1VxZmkZnZVttSofV2NBGzsSJvWyKYQiY7Xep+FIDzlMB13J7J7PNAEMyWoOjMl5olXrMJqgirJgIyRSGkAbjLNzKw6U4FpJQ0G0Bo0JwxHoSNIFCl1EpiEAAAaaW0lnEhyzII0zff7W3vXPr43sM//cX9WVYKzoHZjf+AQDEmhsfd+bhYH3Ru3u5t9I6VHp+NFkl4mhdX0+gP9g+CIDgATNP0pCieFvn/5+mLnAevt9KdJJqXRab18nR4ab2fav2VW5f/zcf3ck3fv3ntvbXeVhp3AvE7v/tHrV6/tTb41lpv/+x8/9GTN7a33u20fvL85aX1XqIndH58NtqEnfX1JAl2d+/tHd7qdw7m8x+fnq+laSuCdhRtrq89e/6Mi2iwllKWvZwvNrZ20iRhnCMXiFiWeSnLOE0n08lskbMglEVRSinCYPfq1bTRrFtZjlIqrBBUQEQHniqcVOGsFj4iAl+D5QHnrghYjXi9ntmPexW0XHXRBAOosoA26WPylszkOc0P2mhZ53FXTOlor7KCtc3jOyOaTOBCE3AG9ikNjkqNLLwv7xIRIJAxxjQAkS5Lu+rDGMIMGXAOQGSynVqZzLwpSWYMgyAMw6gsC600ApRlQUoiY4wzpbFUvNW/9Je+uXHtypXf+dOfPD04C8JQSqlLCWADEIKx2dFxfOX6r1zZ/pOjs5dPnl+6vPsP7t4aBLwTit/9wY+//s2v7AXil9d6XY6//Xs/uHn71q/evLSZROfLbKzkj5I444It80QE5csDJnV+5fLpMr/daYYigDC8tbN5rRn/7pP9NIoHzVQge3e992I2/539I5EtMUlUWeZET6bzF1mOefbpwcmdy9v9gxdqMh0TXt3aIeRKY7/bHQzWCHkUJ0naCIKAALLlYjI+f/r0yXA2LfPl/t7L44PDIs8E50zwt95//627byPjX+I3rGh58t84twbrP7n0v/PcV71vMFrRee7GOoSV+tHavaqleStzxrpDfomE+c8kkJSDhMsTAXM7Z6IDZtU4e4/6dYlIawI3cQAdRRH4HXZqxGlX5ZkFyWaxMoIpgzfAJLNRjrkqAuOAqLV2G+5ocy3SWkolpVREKIQCKlSptVKIUmsNCEEYJg1AXkjc2bn8N7//S2/cugKchWEYhIFdGsiYGGyEb7x5OJysJfFkNIHz07VG/OB8kkt9KQxZXo4KdX50KoT46b0nSZ595/rOw/Hsf7n3LOIs04BRPJ/M51n2yWiqW+13b9/49u7G4XT+py9PPj6bRIO1uNe7M+iCkh8+3jt/+ODl4ctPx4uvbq31QWWHR+XLl51G+nKZX+40Xy7meZErhi/Hs9cuXc2v3N4b50sl86I4GQ73j49m2ZILxjiTspiOh2cnh/svnh+eHi/L/PHjJx/df3h0epoXRVGUCuDt99771b/y6420WaGz5q1QxZAmSISkfeVXVTHngt6Ol6C2kIicQQguUF5zhWHVzLW39HrfO2fevdEE1kmy8we9c1Zz4oyuJ22YFarl9cyayWQdcFvN5EnQGwcOmQjKWZwVrP288QFWH19AxhHNZk4m1VnNVcZN36T26XhZFEXAAwh80Ax4GE3nhVaSEZVFXpRlqbUQIgwEY+kyW66tbf+176azP/iTvWeHTAiBTGU5aJLd/vuba6WCvWUxB7z19a/95ntv/mj/6CAvf+/B81/6+vvv7qwLwf7s+Pzw5OybX33/g8u7+5Ppp0enaRz/9MMvysn8O3euPpjOPn70vNVIw36/1HSp1/7dR3vroeg1k79398bz4eRXblyJovAXxYTC+POz4U/PR7fbHX0nGvW7T8az5+WLcDQuk1heuvrkxREFZ4Moij/49otO74vpYePk5PHJSXl4MCuKa5evdjq9KAgF50UhgfGk0QrTZtruPnn6ZO/Z86NFxqNo48135O23KYjspgw2W2O1qkeJ2yTuIpDQ4bHyeIynYcfx1bOoslj9BHCQAq8w60Reu4r3rEVNv3vYoAM3ObSRS1qBrUElAPug4iq+5U60H8nRpJ2E7gD9ZR4cY0wp5cwHZMg45yCYLv22ZJaABWOMM20iZHZ/KEVEoHUhS8pRA0AQBoFggne7fQIqsyyKYqVkWRQSSGlFHCLOVbZoN5pvXrt8eHSuSsU5E0JEb73f++Z3Z8ienZ0/G03V6fHVW7cSxiTn/+r+0ybAd67unGT5jw5Op5NJ7+zlla++9YuT8zu9VtpI/69/+vO21v/Hb78fCnYljf/li8PtOPpLV3YDoJv9zheT+Z+dnK8R/+nJ6IeP945Ozr/91u32G+88f/T09NHTq/1eS2nJeLC+dnw8/MrG2no3/eHpmItoXGSUZTwKW7NpHqSfdbaXhyezklGmP/zi0eP9l+04EVHEBUdNQoggCKRSs+l8OZ/P5gsVis6tOweNtecvT+9ubtzptokqOxIvQAds0bNTxwR2IaTVZl6Tm/p861v4wn9vC1TDXvtYMyQu+PJOnVb+u6Er4VyiFXSu4tU8/tXu+Wkeu+J8M9srH0+rB9LQl8LXwhb2mNXb1Btt2Q/ttiO+np501VOzSZgmUqQ1kSJSWmktzaxSZZkTLAo5lZRGQRqIRpxEYSiVIq2zMl/mWV4Wi2w5G48GrealS7vRR18sdC6JSsQrN26/s7a2HgW/V6ifffrZ37515Vt3rs/y4loz+eG98d2ttTQMbkbBt9Z7P1jMkktXbm5v/+zobA30tVbj58PJBsCTefafHjzd6nfY1StPXhz/84/vh+2mPDvvdzu3+90XPNBad5upSkKlJGXZdn/wl1+7EnP+dq/5//rki48++jxsd7qN9O1ea6PX+6Nf/OLlF/f0u+9/9fqVGxH7BZMPT+U9GRXNHb7VvDzo6pP9h88e6dmcsmXQ7QftTsCQFblOGjpIZKK6V2+xze3ifHI0nf+Hx88Hb9weRKHN3VUJnosq2EdtnBFKDn3eUvS1oeji7WiTiXXMEUF9/Z0lvi+p5/8S3iISjrpNjGllJXj9Kmj2fbJGJHFCRNRoV4m4xaquYS7HQKuzBJz6tu/rFnFVr4cm1RkwxhkDxkyg3tZ6AjBAYynqUmqpSVkzlDQpIk5EQEqrZ4viRU4fjo6+1e8ca70WikEkEo4NHgQRB025VjNZtIke7b/Ms0Igiig+275+XyT3Hr+4FIpmp1X0Nvfj1h8enH3+4vDNy1vrN6/96MXx/XvP9Hj62nqvc/Pm3icP/vjpizIQv/34RdBItzrNNrDradR/6+aPXh6dHw05gztrnTc2+3+ky0ir6dHJV9fX/vLO2t1OQwL93//t758+2fvub/x6X4jf+/mn7//lb8Jsyc5O4/7g4XD06U9+/o9/4/stpSGKQtAfH5+Fm4PF6dns8/saUW/vKKX3gkZvsD3OKNrlMeK4VFkpRZr0moku5HyZdS9vnyslCorb7XLvxdPp4hcn59+9tMVtTBG8H25QZiOXThGjz3xXbpLRcuTYrgYRYyNcMOEMImrq/kvReeHl4gAgvCdk5kTd1PCAM61htr3Vii1m9+02WsEbl4gAJururQUfob3QCB9MZYhS+4fQGLGgN8G1RSFppXnAiXHidstvk+w0u0WadZ4AUEq1LPWS2EjqAtnPRrMNwX56OryWxO+t9Q6L8koieo2dU2qPqMiam+9/99ep29dJ40qURCI4ni3+4vlBvMh22o12EAgOT8+Gs9PzvNv9+2/d7At2rxUfzxeT4fh779zeTqK73VbIdh/PF//0j3/y1TduMYBbzcZrb9w6/K3fPz487dy8cqvV3H3j1j/9wQ+fffT5L/+jv3+yyH/nsyd/9yuv77aT4WL81Uubd/utjzb7f3R0PnyxH4fRP/zG+yWp//Hx4//13uOT8yEfrH/3+uX/9PnDPwQaPHkM52fq/Q+SjTX56Gk/ChcZUiFnaTvrdZL5rDw4WQQJRK0tmg9Ph0W3y0s1PDjevHa50+0eHpz8OE06YfD+xqAGn5ptZkfcfqtJGVPO6l+LHLt7j1PmGvz41gKcFWdhnUJrnEU2RLpySnUWgHsON/qZQC6uUKc6e1Gbf3W7c9U4b+VQcCit2b/abeC7Yu448mbVe+IMGWMcmbYhU1TalYuA2amXkDFiTCOQtIs7takc1ZqASGsk6gbIQA9CETJ8Iw2/PuhsCEFF8e+Oz3/rowfPxvlBRvtK/D8fj0927m5+8K0fp4PPWPy9zfVf3Rj8tWs7t0JBp8O/fm3ne7sbf2Nn/VvXdl6enuHZ2UYgbrebVxrptSSmh4/6ef52v/One0eHk/lbvU633fz86GxG9O8e7k2z4u7Na6zZ3Gk1vhhOf3hwdnNnKx6fqPm8FQVxt/l/+fnnj18eX711c1jKw1l2Y2vj9x48Pn3wRSvkDycLhvyDOzc//dnPijDavX6tH0bdRhw8ubd8+UL1ByyMmUaxs71/eJyfDyGKIc+ns6wcrHciwc+HmaIiiUPBTk7PdbsVhsHJ+ShupE2G2WL5o+cvJ3nhuIB5tDhWQmujGV+35lEAEIB23jmRSZsjOq7FOor8WPuPGsgZDB4EUB3nj6zhiLkIwCocaxetx49s8BI8z5nWo9bVkmp65ZZQY3W7i1BNibglm7QaRmCCm+0btNa6MFtCutYgIGPM6HRt/CSllCliMUvsEQTovuAfdNtzRU3OTws5HU42onAjivTp8OnB8Rvdxgfr3WA8+fMPP8ey+DubvQ86zX/6F18cLbPrafLffuu9S5e2/ulPP/39Fyf3zydf3dncefv1MghGhQwYy5ReRDEUy2eHxzd67buDzr85PD3Pyw+uX7qxvX6rlV4dtH9wcJakabvdRsCvbHTbcfCLpeSt5uPHTxDZB+u9FodZEG698VYnEA9H0504utrr5m99Zbl1WYH+83uPu2F89a03eLMzHo1+609/ErQ7jIvpxra6dpNzoYgKWYrTo+z0PBusi80NkGq+zKnTjjiGAStns7DfbbQaAaLotAVgpmHz0hZjLOS81PoCHlbYyBR5uPo7qP1nQeyeweUvYQOT3uV3cNK1KgtvIzjPujIQ6hjDGvz8PgYrYVj7lVuupMEsbwezkzdzS+bdXKP62lhyzYSay1WDuw2VoSvy8wUo5ktmQ7LAGeOME2Kp/SI552khklnbSaSUIqXIxURNcNRs6LyUshkEy7Jci4JRIZ8uSwqj97vN6NqlPaV+cjJaD8NvvXY1L+XhPHuz29puJE9OR/uzzOzs/+2dDRTiu9uDRIhJXkrGkzReiwJFdK2RnM/menQ2fnz/3z15OVb6H9zYPc/ynz0/EFm2P8/u9DrdQPzw6Yu3ttcJ4MFoPpplc2D0ze99zJOPT4f3J/MsCPibb/z4fPI//OLeh8PpvclsbzjS62tTJf/oT3706cPHv/vDn015fE44kTrttN5O02+8/0H/8jU4OysOXs4fPFbjGUax7K2p6aIoqNWIwpPjJbH48m5jNhnPFhPk0Ou24jAWotVrM6XGy4IYC8PggsazKMMKhmbuu81T/ZA6Jl3Vy+Doxptz/lcXQAfvalV4ALdxQQ0bmgg1OSIkYU09gytnEtdhbWKV9V1KqFYsooEYcG9rugZVvFiDOxprw+zuQGYD35oZ6tHsthdlHJHbBeCklXI7PwMCQ8Y0gEkjKa0lUWgr87RSSmp9nqmXpWA8OMrKlLMn84w3kmYUbiXxRr9zI4mutxp708Wnhf7gK2++M+g8HM22mumVmL34/PPi8ub+bLkWBn/tzpUfHp3/+uWtJC/WABln24342Xj2jd11CSR/5ftqOFwq1Q+DTiAixGBjcF/Rdl7uTedzgGBtcIrQEOxfPno+kXK5zMIo/vruhiL66qCzPzx/+vBJp9//W3evbSfxOM+CNEpGIxWKxt3Xfv3aztPheKrViIoG8J3r14so3Iyjm7duHokAGbvUbEqVv5zO2OmIQGLajHudxXRWnJ6VRyetyztJuz1bLEe5ZL2WOB8WgGvNdO/F4fpr1zXRIi/7UbjKSMYvd0Co+ezmLQJUxW0+mujRWSMjuPhaiZK6uVABwwcuAXws3T70lXlG9VPBX9/qc7soA2yWvE6HtVv6r7WbRjWvfsUA0P6JzbXr1IIJAGbzXUQT6zSekKkrZEDc1tvbpW9aa2kwqZRWsijL+WIxXMwl0W4ScaBBHHHGp1pfX+vtpDEHWA8DTrop+EkhoSx/fbMfMDzI8i/Gs4OiPC3yAPF6u1EofavT/tpG75Pz8c128x/eusw2N39v7/jH55P/8ZNHEeNfv3YVrl7/0dl4f5H9Pz57/J8OzkINnTQ+zPI7vXagFQ0n393dvNJM3lzr/sqV7a2ylM/3N8KwLGWWFXe7bXj4sByNOeK/f7T3clnkivD8nB8c9ZrpRhB8Z3tjlGf8yePZeDSaL/74/sNPDk9CxtT5SC6yNhNfu3ypV+QciXZ3eKez1u50Ww3otIMb13ivv7G5HirZioM4DHcGPcqzRVk2Qw5Ex9PFy9lcO78CaqNvi9PIu0CWusyxhggv2G810iX/zYUIY3V87V42OGWVradxz9JmxD2KvYdUXQ6rq6FJ5+g60SOAezpRlYKqgkc1WvXtcwFfwkoidur54xCRM/MIGrMdmV0IL5Wy9a8EnDEE0EorqVSpZFkWZbnMs8lkfDI8O11kPIi7YVCSjhljyP7qRv+vb6/1w0AA/M2dtW+v9wFgLQq+sdE/LMphXt7ttbZD8UvvvrW/feXfPNr7ydnkrChOi/JsWfzgbPI/PdzvpBGQ/snJsBEGC8Y/PDpLSV8PxC+vd9/tt7+10f+DDz8Tx6e/sbv+w8d7y/Hk1y9vlk+e5tP5J2fTm83GdhB8cOvqZDwr8+J2p/Hw+GwqFaSN33jn7tfWe4M0+rdfPM7ORpmCb9y5+c217j//6N5plpcoFBfrg36nmawF7On58NHey7VGozsanp0cP11k0dkZFAUuFtn5+dl4mknFsiwHGGfZQaaS9XVotobz7GWYXrt9c8nErZvX1pJwt5X0IqPlSVtr0ngw2i7zBk2g3ZJij0BFpDwF+kH1IKu7H1jzklcIjZzbpZ3i95lCR0/kaqAIiDlr44KfZC5H1RsCZvi0Os6jHLyn5VEI3qZkrB5YRbtDpMcw1qEJlW1qK+cRmV1YrEkTSU3SuvPMhKC01lLJLM/PxqMnBy8/fnjv40cPh7liIiyJSsJc67lSD2aLmZS/fTw8XGQbcTSS+ufDWQDw7qDz09Px77w8+1dPD59N51/ttb/a75xr/ZPhBJXeisO3+u3//vbu+73GP/tPf/FeFPyf3n3tb1za+Md3rkxL+X/7F/9OHx0nnO8NJzf7HQj47d31QRzuNqL/90f3//hwOMmzZVG8Nmg/HM8ejmdrnXbUTg9m8xzZMon+/cf3UZaPTs6fjmfrrWY3CbOz4+DJoyxtLAjTfuefffYw23/eyhYTLl7MlrLdUwHb+/iTrFSD996TcfRk78WSC9XvQ6PByuzs2f6Mc0piNpnAoyezrJCjcf70ebn/cjhdCCFUUUitx6UMhIgCAWgtf7LLZv36xYor/PqJilSdF1FB1J3wqsNxwQdy7pV9TIyJZnmqskhyH5AINAnXjlVnitzVLKCdIvbGgHHKHPfVjNLKgibfW5dltwu0XKmrPxgdrGsbhJvtyBEAzENqtNZIxICQiAEA4xRw0lpqtciWo/nyfLY4H03yxTJoNvnmtZ5UgWDnUv3obPi3Lm19uH/8Vzb7gygYST1WWnC2UOogL+fT+W9e2mgG4miZ/dbDvXxd/t2rW7nWv7139K9/+tEO0d/5/rc3InG33/nnjx4zJS/1Or/38PnfuH3laqcRhCxot17rtvZOFSP65huvncwXB5P5u1d294Lwn/34I0a0zPPtNBZB8NPTEWolFvP3d9Y/Ozj5leu7x+fDnxzsiyh6ucxn09n17fXzkyM1WNdpqvLia4PuT4r8cHJWdgd6mV1VmifBpw9eBpyJZmPv+Ly9vk5PH8oXL4CFEKei39WnZ3g+Ut1OO42LvGDZfCFl3EzLZSbiYDmdtgIhlWbEpFKzotRE3EACcXVraXRekUafcXLf+iG/wCweQnVN6N9UC57qUSF7G0ddTrVaBBIQWIBaiHl6rkGzNkuICIhh5bG7s8CWz9VMZnKumdtEh5zpXQEYa1tMoTufyO6IgnaFMJpdwH1ZiKFWAFSKFkWRLbNlKeezxXQ6L/OCNDUQR+Ph2WLRSNIbSbjfbIzzfBAFz2aLv3Jl63/67Mnv/+hn//vvffN6K/39e0+ePX72j3/tO39yePrtrcFoNP3dp8+3W43rrTQGKtJkPjwLArE/XbQbydvvvrWVxHf77YPR5D/uHyHnbDF7rZUMorC52f/5yXC8zJ49ePy13c020D+6tPGHQL9IgkfjxZ/uHd1KwmSr/zuP9qjd+3g4+9EvPr3/bG+xu8Nff+OzJ88PlTx+/mLz6mWxfamUcO/ho881KME7a2tFo6tPR/zzTx/tPWev3YZGozgbrbUa5Wx2Pp1yTXpjhzY2BGJZFuF0BFKFyKaAvWa82Ntb9vsyinvdjp6Nz4m2+p3FbNHvtYHg5XRxs9dhjPmxqVwiAL+U0iBgBbl1LluFKdbR45BX94+8ljdHO4YCJG2fJu/oi9zNhL9abSG8jxSAe76XW+RfD7JahW51/mqzrVaoJ2a9Y+jSAMb2dqf4qJPT3eDMU4EImqSUeVkIjkzKpSzOx+OD4Xg8nJgqJSklaWKADEEvs/zk8EW73+hvXum0302jaam6SfTZePb+dP7Weu+jZ/ufn493uu3rW4N7z/eeHp3s9rpfnA5Zv5PleRoGL+fLWSEL4NMo7SNdWuv90cHpkdTH9x78xuvX39teezlfPp0u7rz37pKxo0UWM3yr1wKiA33zt1+c8Cx/vxn/6rVL7W73Z/tHyeHZt7cG7/Y7R8vsoNduBuK/+85X//jgJBHs8qBbIF3rtr9zZefT8eTFcDzY2exzHCKPkuR6HAaCnutd7LYkIZUSAoECjj75JGykFCdquYT5XNy/B2EMl7YDpeXOjiAqigKUZlqzIOw0kjTL5kW5jONCiLVOGIaCEANe3wUJHJV4Zeuw5FVijfLqoKzDl8APtz/Gesyv8i6aZ2lbyJstUQ06tYsYESIKchvUuFbaIuXaRKkZumAJz6wYJjCbjqH18WstqJgXAc2CDSRA9E924ASIdkMHIw8fOK7NV2SMCeSAUBQFap0tF4UsT0bnJ6fH0/FUliUQMG4fOmsWN4HWw71nZVHwN74yThJkPOH4TqvxdJn/4dn4q93WndvXP53Ov7I5eLoUjStXPjse/p/fuPlsuvjTk/FiMn90Nvz69nojjjd77Ww+eTmZXY3CX7u0MSzLB430//v08O1e82ojaQveiG79wcnwlqT3uq1L7WYzyvqM3Uzi33zrllTlH3/+8MXp+dvbGxvN5NHZ6Z9//vlH9+5f3hzI61f+3cHJ7OQ4DgTb3u13msvx+Wd7T54/fNhKkmgwkGkSZhl9cVT2252y2BRx0oqXo5E6HeciDDYGRRgiF4HgshHxQBAXGkAUWUFQHJ9AGLSaDX52StMJIuYcIlmWp6fTZnuWxkkoGpwFgsdCcES0WUpPJaveTy2O7YbEbe+4aoZqupjHNuPnnSTnU1viM/FMUw9l4oh2h1BA//gd24R/9lv/wSwPAkeKZqGFybdWUX1to6jmxuaxG0ZBMMadUnahAADSimwUyHl5aLfXNUYCQ0TG7N7J5gwCG9Qsy0W2HE8n5+Px+XQ4GU9kWTICJcs8LybTyWQ6zpYZAgRC8EBwzrTSSmsETJIYGUPOB5ubV+++HW1dL0SERP0oWgBqgLIoIiH2lvl6FGrGJlJdMo9u4Ow8K07GU57ElxrJIAoXpbw3W76cZ3c7zff6raXSPzuffDGe//1r22tp9Mnp+ZPh5FK/myAKWfz48wd/sf/itUGn3Ujm58Ozg/3xwUss8ka7qRtpORrHHJMyu7Q5SDvNaVFmo8nLg+NZpuL1tZjxG1e2H9z7gqTSWqfNxmKxWB8MeuuD8fERaRqNZ9liITgvEHnS4GFIUi9bHb51ydS/8bgxyTIkAMExCBLBaTZfMCaSeBBHyzxXgI1QpEkYiGA7Dtth8Ob64Eavw6CSf50eyVdlQAWyeuiwBuNqcQRetBUvHukvaBCozVIgswlZdbvaM8SIRD2F6FW4Jm3dbfsYdq38VvBuXtnZhUCgmd0s20ZMNWkCrbQmIjSPmHHARQBVm2GGeJXSRk3LUipNUpbz5XIxm56en52OzmfjUZ7lpSyLIi+yXJYlaQ2c8UCIQACilFJpigKRNpIgjgCx0+5sXrume1tLHp3m+XC57AueBmEYJS3BR1KlgqdhECBsRkE3jv7j4Vkm5ZvN9N2d9eeL/Omi6IZBrmQC+o1+eyMKfnY+3B+OSqW/2u/eOzn6n/f2954/S89PppvrOk17XDenk6/K6fTzz5+PJnme72yubbZ4JJqv3bp67/GzIgXB+Vff+WpZSk302cNHe/svl1mezbOgEe5ub1watIOrW6dn40GvNRpP2kn/+u2rf/6DH22u9+OQt9bbx6dqluV6mXOSLYgKQjocZk/uMQRgjHe6zWxJpUTGIQyJCyjydrMVrK2HYSDPz4QGHcWzKOJpQ29uta/spmGAXnnBRQiiz6pXg7USbyGXOvIj61m2ArQ/y0HTRF5tZZ790u0pV8UQVl5CSmlvbc+ynoqpoWNIiKjcE2CV1syt4eREyu4Uh9p562QedAB+vZBSSkqzua/WpZJKSgD7pFqyTwWlvCzHs+loOs2zLC9LJaUs5Gw+G03GZbZUZSlLaTcE1YQAnDMRCMa5JI0auGBpGvabKQ8FRvHm2nrjymt78fpPDs4wf9nncEmNh6OzrNnLm102HZdRkwAeI1zdujRTellmAOzW5tY0n//kYP9yq/F+v/v49OiTg0NdZO/221+MRgfTKVvOscgeB2EjYleXc3r+LEK8EzRPz44H7dbbb91+vLf34ejs8o3do9Pz/mBAUr37+vWP7j86OD67fuXy6zevPN9/eT6f7e7uChEwzsusiMPg6s5Gq9GMEAJkoRBa6vffeW9UFE/v3Rv02u1mmhfl1UtbGpDOhtcubWV5cXw+4owrpbjAbjNdZDnMRpyxrMg5w3wx4QCMcyoWej5UaRKSzhfLQlOcNhZhYzIcz5b5WVZ8bXv9jV6bu/gLw2oLLefgVkuIVuJPZC3EFUh5C7YCcVX4bBKe5kTz/ENy+ScX90RnG7odmEkTkVBaItgHWDGG2vvfWnNEs7rKPT1bgdmdnuyGvyanaUpFAQg0mOL2UpayLAslVVlmebbMllmeZXmelfksW0opEUiTlmVZSCmLcr5YTheL5WyuitJuCWqeO0vE3Tp50sC0RgTGUAjOuV1HH8VxpxF306TdaoIQFARs4/LPoXPvi/tXhnuXIiXzJSjZYkzIZXt5Op0uWFEGAZMaFiePG1HcScJ2q4MHI5VlrxeZOi7uPdBRlO6enRXnpyeCzUaTq4MecX4+HCPATKrRaLLe771x5/Ynn95b3+xvb27MlpmSspHGh2dD0nBtd/unn37623/4n5SWTHApi2VRPHr+8uXhyecPnvV6nWKRkVJhGm31exu9XgDUabeOTs+nWXgyWzz47LN2I+m125e3Nw9Pzg5Phy9eHAVxyJCtd9vHJ2el0o1mYz6exEkcp/FwNI0QW712WUpcZKVWQnCOoMsim+luI02bDS0VIgSQqXJS7D39Qspn08VfvXn562tdtgowS4dWQ37JEmRwURrHrUj2gdeV1epVpFf97mrWojDhKyRbSufKUip/yDNoCQACEEWgNBBo/yBRbeo2NCmlbHDVTh2zolKhL00iAAClZVnKoizmy/lsNpvNZ6Usy6KYLmfTxVIWWZnn88VimWdaE2gt8zKXUpZKuwfOIENun2pjFijblDworQk0IhcoOAejLBiLwqCRhJ00bjcbcRxjHMet7iLtHcyzG7S4kuhyPGZaFYTEWTfBSOswwDBOwig4G8+CfK5UTjLMF7OlpkajIUqZRmK+mE6OD3Y2NkaylU2nd65dns3nnWa6v/dyssw5MMZYp9v5gz/5s2aa/Nrr3yLS//K3/8Novmg10q2NdRHHAYNOKI6ns29+8FZAsNDqeDjKtC7yXM7noGmt32k1kkYjPj4bCeQ6mz1/efTe++932q1///v/sdtMMylzonkuMQyhKN9+87W8KOaT6cvxZDbPOt1Wr5UEoJeLZRSF/Vaz12k1kvj0fBgGQVEWy0IWUqVRAKRJSS1BE3CBXKqCyTjSfDaccv4XrcblNN5JogpQKxissaFLRroPFX3W9h686MmjLdp3gDZWBVK1nY2PbAISahO5r7xkxoSxQIgbl98WUxLZBymCW1ZhlvdWZgq5DTdriUopZZZnk+n4+Pzk7PxkPh7LItdS5UW+yPIAmdRqkRfLotSlVFIVUhERN1vbMcY5E8yn7BEQTWYTNSEQR+QcQs5LAKlUzMNmFMVR0IzjOIyAYCFlg4s0TuaM3e6kR8t+fjoJgiDP1SCNoyDoJMlkOonDcGtj/fh8FAouhBBax4jLPG9FkSiyzfUNEYTzZSnl7MEXD7bWN7a2t8psqaRaLLO0kbxz906/1/3wi3s/+NHPVFn+5vf/0nA4BMYODo46ve5av9PpdS9vbGitHj58HoRBwNjZ+agoihfHZwcvj5SU/UH3vbfu7G5tkFT7B0dT4vcePSkWs1/79V9f73V/8eM/DzhLk+Qsy8Mw+OLJ8wihncbns/k8y5IwjMJwrdduN9Px+QgB0zgSnIdROJnOkGG71Yzj+PR8hJDzKJRKkgYlVZrGhdRmG2ulyuloWC6L+WjO4ni6u0FpjK5g14MTfbWaNzXrxOjIk9w+oy40qV2U08cjrdfjntMBmuwGCcavB7cxGWoAl3x3XpsWBv5ak0b7hHcwViSCBs0INZFyRUNKSSK7gI4h5lyAtSeoLEtZlnmZn4+Hh4cvp2cni+lMlbKUKtOqVIoRlER5UUql0S6yJ0JQBESaA9MAGkAAlQSAIAAER47IGBM+ikEEQAEP4igIo6ARRUkS8VBoLngYxs3mUbr2uxOVsPmNiG80wvOJ7idpu5EA6aPTMyC9PuiGcdxpNvIiX+/3EPH5/kvGeWdjrSjKo7OzxTIDxP76xrVr11SeSa02L106Ozl+svditlhub25c3tl+ur8PSEqqLx4921objIcjALx+4+pX3rjz4aefvQTYXuvPpjMeBVmWX7+0+/OPP3vx4sDE7zhjr1290u92P/nii9licZbJl0+efft73+l3upgvRcB7G5vTvJiNJ/PpvNNIllqNxpNlXiDptX53o9+5vrsxns6KPF9mRZbl/W47y/OAB6BBar27tbG1Nnj0fP/g+KwZx6WS86IslGokcRSKvJAh6AhUPhljL5Bau0eZO2Ss+jpQ24XG56Vx9RFt9lcHYiQDwirxCWjMTASt/fJfN/7WAYIq1O7PQgAU1jTVWqPSQFJKX3nKAEgIEzMqykIqleV5WRaFLMsicwuFCBBLpbI8K6XU2aLIFrPJZD6eyaKUpEutpNK51mUptdJSSjCLQ2zk0hUWasUYAHGJKIkYAucsCYTg3CyOY+bZx0ARMhGIOAyiKGo1m2kc5UBBGK5tbNDgyk9l+mL/SWM5/d7lpIk6C4JIcK1pMp1NZ4s3Xruxu7P9/OXx8xcvACgSnAiajXR90Ot2Wo+e7J0NR8dnw1YjVYCdXu/uzduz+fzTz+9P8/LF8XC93WilycuDw0iIACAnGk6ng7XBeDpdFsV0PN0/OrmyvfXhvUcPn+31d7azxWyyWLx1++a1q5eeHBzNJtO007i0u9VI0zCKAs7zZbb3+Pna7s5wOnv++OEH77/7jW99689/+vM//ejTfrcZtVMgkpJarcbV3S3OUANs9jvtdmuZ5VKqUspGmkZRIAIRx7FWMi9VnhetZrqzPsgLeXQ+jAMRcM4Qi1Ii55yxXEoFiKBQq3KxfDlbXmulfFWlYw2XUENh/ScPJheJtD/bA43z4+qaq+oN0ohu32W3sSi5oqpVU5eAQJgrS01EpEgXZVmWpVKyLAsOKIKAcU5EWsk8yxZ5Ns8Xy/l0OhmPZvM8y4hIA+ZFCVpHDJhSAoAxLMqyVNKkJ0mTlrKUCjUhgEbQRNzlOYmII0NSqJEYA4KQs4CzNAjSKNKkF3kBpEUgEJgG4pyHYRAGIk2TTrMxnc8ni2UUha20wQc0EOLNG5dffvHJ2cFep5220uT47DyNwlazubU+2FjfLDVurg0Q8Xh43u/1sqKIw4Ahe/7iKI7CVqs9ycvj8Uwg3Ly00wjZoLt9Np09+vQLu202UiONbl/d3b919Ucf3d/d2uAAHDGJgs8/f7C+sUZRMFvmB0fH77z/Xgh6d9A+GY4eHRyR1qR0Mwg/uPva9s7u6dEhZ7hYZFEUh4y9fPLkm2+/OZ8vijIfnp6urfeOjs/W1npr3W6v01GkdjbWpVKCiwBJK73W743Gk8PTYasZa61ms2WeFXmeh1H08Nl+lhWB4K1GMo/DeZZLwEgIpZQmSqNIKlUQKQVMSV2Wv/9k/7VuczuOtOdI7Vc9OgjWV1vYx6rYWOmq8iewD4Eh/xBEq/cqe7By7cHZojXnDNyzu215gEBkZJ65XqpSyflivsiW2XIxmc1Aa855EAaIyBFBlfNsOZmMFpPpbDafLpZZlhdKm4amnLMwCDjPQZOmoiikCTMBSEVmgy6bbQVAjvYhBwTIMGBccKbRBF5ZFIoQkTMslMrKclkUseABF0EQEJIGUAg5QBPZYpnNF0upVIJMypKVy+3GIKdg0Yjzs+zxZHQ8GseBaDXSjbW1Qb+viRaLRb5c7u3v9zutMs8E0Gg4C8OIAeRZnmfLVIg8CsIoWd/a1oTHJ8fZbNzttUfDYVmUYRA0m+12q722tsbZg/l8gchK0pzzLJs/Pzi+vLuZpM2rV5KtVnN9vT88OX7w4vH58Wk2XQjO37x5dWttsJhOy7IMgiAMuS7L4XB4+85NhtTrdD7++NPZfJmEwZ0bV9+8+1qRl81m4/TsfDJfXN+91EyTxWJeFFlR5M00aaSZVroEtczLl8fn82VmiQxRcB7HYTOJI8aXpczLJSKmOpJSIWeNOC2UEkmiFJ2PJqO83E1iiw/wGtmqcxPkAXBL6WoZbKOj3bN8zOoL7crsCBHJFSNTVaVELvtvyXole2V0qtulg4iEJm1L0ZUsyqIoisV8djY6Pz87z4tMgw4QQsYF50A6z/LFbJ5n+VLKZVGWUhFAwDAWQgDIUpZKlVqTUlIprZQiUkpLbbOnAMamRPNoWq21WQAPnCvGjD2Sci4QNUBWllJlpdICkTNknAWcl/bxSCiISMplKYGglSSDwaC9uf2UtX/3fCqnw/fzUTPkJ8P5bJFRGj05OBr0unEULbLFaHTOGV7d3Tg+Oc+y5dbG4Ma1q51uf75YTqbTHaL5fPH4+Yvz+WKZl5sbW8D5YG3t4d5LIhJCcM7DKFkqdWVnfa3Tevni4Hw8vHH1+htvj1/u7R/s7W9vbrz/ztvNZnN0crqzsdOJoiBgj56/DJP47ddvfv1rHwRxslzOwygw9f8sEMBwdnraa7WIi/7G2ht377QajVa3vZzOF/n40uXLN65fX2ZZyHkjbTSW8+l8JpV6tn80WWRFUc4Wy7KUSmkbrAEATVLpuVJBEAjOdZFrgABgmWcMIEzifDbXPEz7g3kUJXGUCqFddKcK3tvMYG2jjVoyp+7gk4vuAwJoDW6xsg2tG9iRKbcgALelPKBGV0f8SsDfx/YFIiolS1nKspjN54v5fD6bz8aT6WyilkuulSKSnCFjUsM8z8s8L5WWSmmtOQEiJIxzBKUVIJIGqaSSqtSkpDYBeUQMze4TyMCWh4LZlNY8m9Nk581i0dI9d8asfwqY27fJJHwJEICTJg3zPO+kjUaaNBuNKAohavTaXRyfyNMTORo9PjvMizxIYgVwY3vr1vUbSdJQWidxEgVikWXzLNte722ubwzWt4KoofT5YpGJMCo03r3z2vOXByfHR8046ne7rTRpthpnp8ODo9PpbNbvbybNzqA/uHNj96cf3X/+4uDWtfh73/zmk63HP/rFJ6DZ22+81Ww05WsyYKzsd5+9fLm+tRlE43feegOIsmXGGD84OjobjstS5vMFIF55685ar3d8cDAdnl3b2TGLsLWSN65cKvP86PhwMp4ss6JQdH5+Jjju9FsMdTsJIQnXuq2iKKeLbJ7lpVJKKkQw0z0rinaa9JtNwwpFKRd5vsxLGXDNGJcq7SffvX5pJw61UlWu6BXrE2pOknGRwBKp3/5I2yXJaBeyuWwmeZiaOJTP2ZPWzDpolUWLtQ3hDSsLrXUp5SJbzuazk/Oz8Xi0mM0Ws2mxmDOlCFFqPc9yM8OyvFBSEgBjTJitHAAJoJCq1PZBpdzEoZQtkUPGwkAEAdeMmXVt0ixf18SQcbMPqNbkHjuqiEqAkEHAGRMCAZAxpfSyKAutBeMEUBApUjFApOSg2Y2TJFeywdi4KEUcJoKdHp8kJKMoTuJod9C5feUyY/zw5Gh4fjYaTUqlwoALzqezpdLAg0QE0WBtu9vbmC/mWSkDzoXgw+FwNh1vbW5eu3R5b//Fw3uPs7KczTNCAk2Dta2rly//7OMH9548H4+n3//er6RhuLm5vrE50LJc6w8QkQueL9ud9kCww7/yy7+UBmI8HDa7nZCJsizmi2VeSgDY3lq/de2K0qqZJts77+ZF/mJ/LwyDQb+7zPL9/WehwHYj2dvfe/D8aDpfvP3atcvb671WNF8sc6nDMN7Z3l4sl5/df/T4+UtZlou8kEov80IWUsUUhKKUCgA0gCJQIqZGlxifa4o1dQLBkCqv24eWHFJ9vMnkkmw9vFXYJvLoiNvbl+DQqXX9OUYm6e4rSyrCdHUkzmytdvEUWqqiyCfT6dHZ6cnJ8Xg8KpZzyjKlFUPkyEops7LUAKXSpEkAcM5CzgCYVkoTFFLaPecEF4HgjEEhZV6WUgJpJMoBCCEOOTGmSlkqBYicc84YMvPgTRfGReSBMOWljLGAcc6ZMVOkVKVWjHFimGsCwEaatJqNfrcbxkkhJWTTgtJZrqVI+OBSSsvs5GWe5be2N+IoODw+/OiL+8/3D5ZlIaVuN9JGFCRx3Nw/EFEjjeOsKPO8BFCNMHjx8mCjP3jt2nUiPZ/P0yT94O23tJI/+/BTU8NbFosyX671un/7r/7Kh/cefvT5/WtXLokovnvrxkZ/kERhnMSMMc54HEVf/8Y3ci1J6vagNZJqeD48G40Xi3m3074bh5e3t25ev8IZ29/fD6KoVPKLR0/2X7ycL5ff/cZXEHUzDZMk3hh0XhwczMaTXr/7/hu3Nza34jjpKsVFoJQSjMWd5nq/vVhmjSQ6PR8fnY1Kpcu8GI4mCICCAwAyBkzwuKFEQI32RERc6ZYQdaa0oXVjHDrutBCygNKueNRp+1rRKLkaJVDOeaqVnoB90AC5bJF2BRmgNdmqKp9lNTboZDo6m4xfHh+fnp7MxqMiy3RRgJKkiTiXpAuptCJNGjVxhmZHGmOjmOdoEwBnwIUIozDkopBykeVFURrTGoGAtJQyB+DIlCaOLImiKBSylKXJjUoNSMQ4cIYIkWCMcxGIThIrgKJUpZQaKEIRBEGBwJQmZI0gEIhSK6bkMlu2kvhrnfjjvLgn4vHNN1U5vy2gODncOx91uq295/tP9w6yUs3meV4UZ+djxlgrjbVWy9lw0Gkdnk0Vss21te2trTt3bkkpJ7NlXmgmlFSy2+197b13yix7vr937crlIAyJdCONte589xtf/Y9/+kPB2WI66Xa7jSju9QacQZJEnHEA2Nzcvnvz5tOnjz+7/zCMo/lkqkm1GunN6zeDIFhmy70XL37+0Sc33rj7+qXLy+n4z3/+F8+PhhGot1/f3d3cSONeGEbtduvsfKy1/qX333znrbd5EDMRj0dDAAaolC6Pjs8+ufe00UhbrdZskQvOG3G40LooSs0QlQICBIXtpg4izYVqd7e31v/K9Z231zrGdkVbiWkxQ3YLGQI0z7T0vrvFrtP3jget2kfn/dfj/LVNGWv2g0/vO+MWwFoI1j8BBPHR558OJ5PpZCzzpZaSa+IIKAJAVECylIjEOSJxwYg545e5QDtjLA4Dbda1KT0vl4tllmW5if8zzgQyY5dIQM4w4oJxnkZhIESBqPNcmVAYY1xwwZmZe4ngjThinC/zfF4UUlHAMYmjNIkyqRZSKg15URZFOZ3NE0IexXGQrCXRb261QITPs3w8kk9Ys8NCHE32T4Y8ihfLfDqbc87jMOitdTuN5LVru2+8dqvf64og6Owd/JPf+r0g3rt77fIH7761tj746Onhzz96EPHiv/lb39cgGu3OX/r2t2bTyWw2XdvYjuJYnqs8z8Io+d7X30ta7aKQO5tbrV4/SVPOuSu+hm63u7W9OxufHR8dTGazZhJvrW2sr69nWfbpJ5/de/pccNgYdGfD04NnIo6Cv/rLX//F/ecaYHNtjTOBjDUb8f0Hj+492v+V73zre9/9Xqe3rrUu8mI4HMZRMFsUw/PR/UfPJ7P52qArlUyTqNVI0zgadFpKU1GWi6KYzpeax7y7RlECLJDA3t0Y/PLOhoGYtRstKtHuZmM4VWuwTwM0Vpgr/TXVbX7rMWdZevhWVigg1OMADt0O0DaGiqu75Zvovbj/+KHKc6F1yJgABEQym4JzxqQSnAkec7M1KEFeFHlRaOeREQITQSAEcpaXZZ4XeZFneVFK++wmJAIOXJg9VpkQvBFHURAqrRExiaKAs0VR5lJKAhOnIgTBWCMIQiFMrACIkKEQXAiuCMwDaqSSGjRyliRxHMfLPBueH7W77dvt7W/IJo7pxTI6ba2L6ai1PMry7Gw0KxR1uu1eM9FSfvP9N69dudzrDcIoVqoMg/DWrVvXdj/++N6j/SQKv7j3lfDNm1e2fvzhgzQQg8GAsRgAdacnZfHk8cMvfvzzr7z3/tvvfWs8OhsPT7O80MA2tna7vXXGeBTHrljWJpL7a+tJowUMtVJbmxvr65vno/Mf/vwXn3z4edzpNNstfXjSCNgkZHmccM7eu32Jc2SIUhX5ovz4sy9++tHDv/Ir3/2173+fMT6bzQBIiCBJG0cnJ7LIHu+9fHk6FEEwXxZxFANgEkdm6/RSaUCc5AWLG9jdhCBm7a7iYavbeWOtIwxyXBQdXRTI1GqC20jGZIdMYQcZ1BnvxkGTLE2iO9aF6O3WjOSNWqt7fTmfy1c5UcFKuTKgkIulAC3sIzq96kapCYhCLjjnJtOUF0VRytIWdiDnLIrDVpoKzhFQMKGlmtvqT9Jamx0cgEEYiEgYpS0EF4o0ADDERhIpHQJbUgakNAcQCIJzzplGKksZMJYEIZGWRIr02XzeCMIoCnkgBOPdVuvmlSv93mCR50maJnES8iAW+m7KZ6rxbrf127qM8v6VsLXZagh2QqQ2W+nl9X6v19tYXwvDSBEAchEEhBAE4d/7m785/2f/6uGzvdl8cXx8evvGlW+8sb6zscbsA/hYFHGZT8fDsw+/eNpb382z7M7dN9e3Lj1/+jiMk6TRFkJEURxHEeecITJAjaSkCsO41e4209Q8u0RJeXh0/Gz/QJLO5nMAGBfF6Wi2Nmhv9LutRhwHvFR6scxHs8Vksuj3Bv/dP/iv775+lwfxYjpSslwu53t7z/YPj569PMjyfD6bTybTXquVRqGUsihlo5FIqc6H42VZLotCKSUaHaW1DkQep81u52/fvf56t1FKiYyZCJHDUBWHh1qwqbI17XuPTct+9pkYFkMuFK+dEkdHujaw6jKb6DwudDEtF1k3jRECNDdBHLfujpmNQpQWDAVnGmhZFFlelEWplEY7A0gI3m21Ntf6RDQcT4ssF0LEQaCk1JqYyQYwxhnjCAhIWiFx4+4IzsNAMMYLVWgixpkg0giFVpyzNIqiOMzLEhHSJEKAolRZWRaktFLLPIdAsCDCMARCSbrRaCCy8WSUL5dbQqRB53Yz/Xg8XhMh27n28/sfbu49v7LWu355O0G2vr6ZpOn+i5e9bqvXG0RRLMKItC4V9QYb/7v/+u/8i9/+nU8+u392cn5wdHLzylY7YM+ePm61e81mK5PZx5988uG9p61WtyyW//b3fvzg4cO/9/f+wdUbt8+Oj0LGojjt9PpBEAjGzMIBk0ojgk6332l3ZFmkaTqfTfdfvMznyzAMpVSz0QiI5kCn56MHfF8wxjiP4mi9371x9eqv/co7b73xVhSnJydHe3ufnp4Pn+/vLcsyWy6fvjweNBMGsFgsu83GlUtbjTSWspRKDcfTk8m0yEulZBnEMNiUSavIchGlcx6sp8k7G/0AmfJbHxlXxlKd5UsXUoeaCieya9vs4g1vkvrYqFnDSVjVgRhAe1fJl0Q5LrbBArJxKLSPWAIgIKGkVIgISEiC8ZAHjHMA4EwTQVaURVlmRZGV0i38RSaY4DwUIuAMkTjnQSCyLAsED4MgK0smlQlbcIaCSJaSNEWhMMF5xtBEeUoli1JyZGkYlkIXWmuiQARRGABjQRCEnMdRpLRWtBQKAwxaaRJGYa5pXsrZdDocnXNOSlMcJePxKA7EcjBYUDJXwcEie342Srvd3btfhUc/bTfi3c3NWZa1u/1mms5m8z//6UeXdrc21tcbaSOK4m5/g4uw1Rn8t3/373x+//7PPv70+YuDn3366LNHz9vtZpqmoRDn56PjszGxoN/Lnj1/lufZaDx69+13Xnv9zVk05EHQ6nTSNDY2iR94RCzLQmmVNBp8eEZaTaeTg+PTQukwjERARZEHQdBK404r7baa/W4nTRs729uv37nb6fbm8/mDh/dPT09Hw7PDk5Nllk9n81lZAuPrnSZTajiadFvpG6/diKPw4Oh4/+B0sVwi40iaVIlRqnuX8iDWRKwVLqOYgvCDrUFDMLtEAlxhkSYA0OjyR+i8bOeMg9PgBGZz5QqeF7LzVsVXBSTg0KixngL1J5CvKfF1JNYJE7lUGiBENJvGRoEIAqGVLgoqyjLP86IoCynd/olIDAWKNArDKCzK8nw4jqJAK2X6RESkNAdExs3m3DlpxnjKeBSEXAhkGHBu1tAFXGRZwRDTOCq1XshSaR2HASIulzkHxIRzzuIo0gCKdFnK4WIZKcUAGJFZx7LMsqIoAVicpmmcKk3rrCCt3mrGzy5tb8fJrTTak4vR9LCb55PZdDqbbm3u7u7SycnJTz7+ott4trnWXet1WieH3e6A8YCQ3755+83X747Ho4OT46OT0/lyeXR6tpgvgzDZXONBIDhjKkrCra2drc37D+/duv1amqaL+Xw+m8RJHLrAjYkYSqnLstRKN5O00+loXYIqTFkjB9AIAedX1jtvv3b19dduDza246ShlJ7NF188vE8ERZmPR6OyLONAbK/3F8uciPphGAb85eHJ6WTa6zTfvntrc63/yecPv3i0BwzjKGrEkVJqHMezsCUBismIdbq6O9jttb9zbeeblzZMEZyBoS2udKGjClPkYkuVUehodCWf5DJGfim5NxGoxppk1qXZcGdVPuLpuYZUv/xJMGQCgQPEIoiDUDCmlVpkxSLLZF4URVm65w0CAeMsCEQSBlEUxVFIQMu8WGY5YywKhFSKI3BEYkwASKlyJRljwOxDwky6qCjKQpYI2GrwRhopqZFhgLzFmAnpGUMCOQNEwThnjCPjyErGllmeSdlqJN1GI4lCxoRUMJ0viIvu2nocJ9lizri41GhebTY30vgX87IkosHOsNFqqglNZ4fD8635NIziwWDQbaSCY16o+09eaHqOhBpZVpaMi521QavZGAzWu+3Ozsb6+3fvtNrtKAyBdLPZ5pw/e/rwj3/8IRAxVKAKxrlWeb6YzecN3mpxt+rVpCzKImdcrO1cSRuNxXQEpJMoVFopJYNQ9Nrt7fXe1uZalKR5nj14/HRZSMFwNl92O61eM+Wkx5MpAyqKcp4V/U4zK9XR2ehsNNla79+6emm5zP7khz8fjqcgBBdcE02XWSGCIu3LuMWJxSJcNNuNVvPvvfPa24OWts/F8LFO6747l8ijzmpdr5LNr7YcqQqLmrVrNude51FwgSpnD9i4fc30rEHfpuy9WUEAIBKzQJMzxpiUSipVFHKRLfOizEupTZEHACAILsJQxGEkhACGZu2Hto/owjAMmZQiECIQZHZFNAs47QNLqCiLrAQkEIwjw6KU56NpFIZxFJrWFaUkoHYYaKK81JyY4BwZkq1W5rIsA864EAqw4Gyt2RgMBgQggfqDdWBcI2s1mygEMZ4myfsxa0fF42UhARut9uNRHNJxYzhuPn/ywZvv337tjSAMHzx62Gs12u32+WR6dHiyKPIwCEopi+VCKl0qNV/mUlO/09pe61+/cmVne+ta2mp1O1euXH97OPzxZ49PBZyeHDTb/SCMhAi0UmUpMQg4Z1KZ5zyiCAIRBDQbHx8dnhy9jKKw0UjNyqxWI71yaTOOAkXs6fO90TSbZ3kplVQqTeL5fCYJDk/Ollm2zItBu1USHZ+cTsfTNIluX92+cmn7fDR7+vKIEBuNlCMCQ5HEQ+JjasxEpDUIpEyEZdL4jWs7d3pNpezTZRjaLb4McWinbpHQrn90j0YgW7hJFd25Pbh9QZOPENmNh92xRABGszNCMFstmZXyPk7lZ4UDNyJp8/RNJoihCEQzjUqlZ8tMKkVSyUKWZVlK6ScK54wLlibJ+qDXTOOiVOaAQDDGWByFYSgAKImjspSkNZAG4L7RpZRJFEVhsMxyAkriWHBuqvQ5xyQOT0fTWbZMomhRlouiEARJxBhH8+iPOAozKbks0yTBQDDGm41G0kgbjWar3Vkoef/h/fW1tfDS1SBthWEYx2kaJw0ukijuz7NhHh8tM6m7i8uvNYvpXM2/ePTF9cvX37j7zs727pNnT6aHB1EQ7GytS6nSJELGxpPZ0dm54GJr0Jgvl9li+ej5i+f7L/u99uWtjffffqvT6TaaLUbqP/zpj+JQ/Pqv/bUgbDTa3SCMyrIkojiKzJOdGGNRlJRFLou8LMvpfPH85cH5eIqcpa1m3Gwo4MDD0TRrNptbm51SytPTsyd7+804VArSJO60GuPZfDJfjMfTVjPZWetfeusOIpVlOctV1Om8s76OnJdSglRBFA7ny2f7w6kGPZ/xRhMaretrve/cvPyt7QE3qCACArMPmNFt2kXTbVCSmXXrZLFrF0Rax4WY21KZ/JcubumIcBVyaJU3+WfBgLbEbbzp2i4SCKBtCh1ACx9r1ZpAkyxKVUpTzImamA07oeA8DII4DJMoSpKYIDfPfmPIk1CEUSilBC0FxygMi6IEpavNShyNB5xJIRhiyJmJkHHE5TJfLPNlngsCXcrJZJpJRQBSSq2p0UjjCKMwCAMRl4EG4IxHSRInaZw2lmUZ5jko1Wq1ev21brvfSJsiCKIgYpyLgG9E4XY7PZ5l/8P98U/Pxq2g0dncGBfT2fDw/NGD13Z21vtrX1vbHA7PHjy8d3Z6XBTli+MzZCwU/Prl3WWWCcZuXb00mU4WWW60Uhjw0+GQiVBr2txYOx7PHz7f/+rJ8fa1O0EYg6sSV0qZrfUZwzAI0rTR7q21z06KstSAZSmTNO11O0kcD9YGu5ubWlMYiIPDw363szYYrA36Usmzs1EYCNB6d2Pt2tbabL6QSm2tD6I4ZoyHTZamjaTRjMJQCBEEotT044fPPj3NZixhACXwZdJs9/v/zVfu3O6kUmm3IsOqVJfStAkkF9m0jyFgTvGiY04bM9VOudtfccUUdTFOs9bXFX+a/UIQzO7AJrBktxAnv1sjeXsDzHVQJEEQBFyRVrJkSAhUSqm0AgNsRMZYGIpGo9FKkzAMlFaT6VxrzRCSOOIMw4AzhELrUirQmnMQgmnFUGvBuUkpcM4LKfPpjDPebqSc86Isy1Lm2jzNjIBIa5JSMc6QQBItizIuyjhWSimgoJMmnPFRlsWcN4MgkGU2mpwul7rIhRBr6xth3DBPQfaPQDMPA+GcDxrJN9Z7B4tsP5fXuz2gzo+1SJbTfJKvzZ53A7bd77/15lvHh3v37z+4cWmLiWCZZS+PToui6LYay+Xi9du3AHC+WDKkJEmbrc50kSuFUbP7/lvpjZ2dwfbVKE6iMOQctdbmvsiACq0BASAIo0arc/XGbRGIz7/4vMwyFoSFNlngQkkVCN5I4sFgsFjM+73BxtpgPJ3O8jLLsyjgj/YOllm+3usMep0gjNJGK24kcZKKMIqCQARBGARpEP3RvUd/9uwsU6AZQy0LQBknX9lZ323EZSn9okskF9OxhGdD9I7rwJdtvLoLQ5UNclamJrdpElZ4dlFL912tGMXfUbsKEp/uJxdh9ZcXaRqbBdFKk1IkpVZKAyBnLBQMAJCzRhy3G400jQMhgkCYwATnLBAcEaRSUkoE4AxKRRwxYLwAaboXBgHj5rEIyBA5Z0IwAiqVUloLxgRjRFSWUgNJJakEHohIiJBzEy4lIE2UxkkUJ+W5WmbLWbbsNZvpYIDIsjzvdDqETKkiYFwwIXjAAyGECIUQXCBiGolfvb7zznr36WSeMn5/NHsskVjzWr8/oTxEdV4qmc0VRrfu3M2y5fD8/PD4NA74Zq+fxmGr2cyyRac7SBvNQATIeRhGZVEIIQbrG+ubu5vrm0GchoGIwsCg02wdYB5Hhpo0IwKKoki1ujtXbk6X2dPn+71Wc9DvP3m+LxgOh+e9brcsy267vTEYKKVe7O+dj8ZxGGQMO+3219/pLpZLBGQi6G9t9drdMAgwEFyIkIkwDE9mix88ePCn957OxtMoSYK0PQui9W7rl6/tfP/yugCtwe164BBWcWkNg1U5nMv0XMSo2+HLsW3tdBsBUB5gRGif82qPMgVw1oNHxtzTFldaUd2USJSlYghFWZalKopSSUlaAxBnLOA8CAQXIgpDHgiNEEVBHEetNGmk0Xi6mM4WjSTkis2KQnDebKST2VKqAt22KUrpLC+5wDAMm3EiBEcAKdV0nmmto1A4W8fk1YAjk+i2M2VMCM4519rmscIg3FlbW5ZFTsS5oEBESdJqtjqdPiCejofnozMUQgRBxCJu8w22poUxdrnb2u0081Ii4neW+bSUgzQBikZSPl9Ot6L+7e3rnTjI5tOjg30M0pBjK42iQORSLubL0+EoEOLq5ctJo5PEyWQ8bLQ661uXe4N1Yedq9fKCNirerOIXQjAAQtzY3P7at/4SkCpm492NQRxHjUaj1UgbaQMZV7Icj0bLPAsCZsyDRqPBGaSNZpAkzWY7abYE4xwZFyIKglme//aHn/3kycuTyYITYRyp9c0pD9rt5n//3s3X20mplNlmwCfdq4Q4QR0ZxrmhVx7ZUYcsur0U7ctlgcgyY5Vk8rGnyktf5Ul0G+RSVUxSJQQsgy7zXEpVFEUpldndXQNxxIAzxtBYB4yxgPM0EEkUCs4JSCqSSgWCM4b5oljmRRoFSRwJIUaT2TIr7OITTZokAYtjJCLSVCqV53lRSkRMQtFI4rJURb4EAEWglUa3mC4RnHE2XWStNBGCL/OCcx7HsSY9nkwyrUnwZpJmSi2yBRCAkkoVSpZSlrIouQlMuIcy2NAbYhQGdze7m610nOVlqf7J88OfnZzvRmEqdNqg87JoBfHV196+eecttZxl80mRL7gItKKz8ZiLsNVpB2EcRclWu9dsdZqtVhIFRHY3NT9y5Ha7MMoO7eNNkDGWLeZK69u3X0uS9MWTB0mcaFVMpzPGxXyZtZqN8XisVTGbjJZ5SURRFDXTWCpM291mpxuHIRdhFISc80zKF5PpD+89/MGnD2WhkzSlNC2TRrvVvNZq/N07V262wlxKR2UWMpqIedq6sBWtz0z6b5zC9Y9hJecYgUelx1dlJfhtkMDFmHzk0yWrajEpD0p7fMXmKMwazbKUiohzzpnJypvHwHDGWKfZ6HU7jKHWWkq1zPLzoWy3Gv1ui3OuNCVpI4rj8WQ6mmVpyFuNJMuK5TIDIrO7PCJxZER6Os+WeaGUCoQIuQBAxrjSpVkehVpLpYxzKBii4IJxRRBFYRRFWmcAtMwWw8kUlN7udiMRnI9Gs8WCtO51uuvrWyJMSOvFcg4MwSpaBAROYJ6hDAAILA3DJAg3W/E8K7+fF5+ejz4cTr53aTvH8P5k8uenZ3ca6dfWumtR0G302o02B5RKNjt9IijypeA8jJNGo9Fstkw4QrnHibo13NVScbtyABFAm122GQBqzRgDoss371y5+fpyMjw6Otg/OjkfnXXH55PpjDMYjsen5+Oz88mdm5caEev0NgbdfqPZYkIwwNPZ/OcvDp+eT6bz7Gg0K1mkG0L3BnGv/feu/f/o+rMlyZIkSxDjRUTupott7h7uHhG5VVZWVVNXD6iHgBnQPIBo3gEC5g34BvwM/gGfgEcQDdEsBAwGQC9V1VWdlZWRkRG+26aqdxMRZsaD3KtmkdUwD4rwMFPT5V4WXg4fPvzVby63r9oqIEiS9eaDmZZJtMXqcO2jL4sCz5s5F2P5qWtdqypbS6G14F8fDwa6eoInU376zvk5f5oUnL8DzxOJ818QnIhIFjNTs0DUhjrllLNUVajrqqnrF9f73XYbY+z7fhjHw6mPMaOmytN2u62rqmubKoTHY98Pp6E377jUirLMlhISZRFEFDNEcMQIqAgFUlYDdq5sMiRGWUIFRrXDNAHYPrZVCNuu8T48nPrb02lb1QTWD0dE2m02ZspIJprGU0ZsmdM8ReYybksMZeaPCNakHAHRgdvU9F+8vm48/4eH056pYni92VxO+RZgBP9/+3h/ezr8V9fbb7rmZ7uNxWkae4QyEs1t1zVN45gZ0NCyloKddP0qZZ/kDIgqkkUkpTgODjE4V4y18lVVN5dXN9dffX3z6cPf/Nv/+e7T+2GaP9097rb7f/WvfnE6HrZtuHlx02yvBoPvP36eRFOW//jpy8e70+NxyEiz97rZpN1l1TX/21+9/a+/voKyurTcZX0yBV3ReAB4GkxbvSKep92f5G7OXPlnCOjq7J5DRmuc/4krPLtMePZry/fXTtEZXIVnwef54x2AiSkCsFkZ4wRGw+S93282222nhnePB0fETHlSUfEes8rh2BvA5QX1g51Oxya40XHfD+OkCxRa8ClA77z33gePzI6dihASe1cXWhoiOTYBImYkyQIAjGgpZQBm6scxZgnBv7i63LTtpqpCcOTctt2GUDVVU9WNCyFUlfM+isxTX9eNI1LNBm7dbA6FUbMm6GYITLRv6v/q65f/xVdXWXXMcjvEl7X/3eMJQP9st/t3D4f/6x8+/+cvb/5Pl9dX2yrnzM5V7abbbB17Wq8yEeGzNU5FNa3cGCJWMBWdx+n2y6fD423p28XH++bVW2YHgCbK7Jqmy+S7y5ff/Hzz1z5c7ff7/X6O0zRNYvBpmv/m999/Oc1D1MeYT1GOx5O1Lex2v7y5+K+/+UoRX7Xh266SmP/UJnAdVl/xyZ/8+MlOFgqIlYJqJcHhefao6CSWeLBu9LRzzf+svD9nq+dS60986tkuz+ntT9/RU4MU/+pf/2c5JiYyBO/c5cW2bmoiRMNhTrV3SbKZtHUVHKuqc9y0TUwionUVNm1jAHePh3GY+2Ho+2GcYoEARQSJC2Llq+Cdd84BgKkiADM7z6I6z6l4NRGZVdXAOdfWFTNVzjd1aLrWISFTHcK2bes6XOwvjN2YIhEz8367vb56udtdOKJxGoi56XZVVbVt530IvmJHRMjEZwNVW+5ByYzLPTKzJDLFPMQ4xKRAd1P8/eH4bVv9q9cvGMQMkNAAq1D5wgssI6qIolo0L8qstYquECOISBynT58+/n//5t9++PTx19+8veyaerO/fP1t8JVzzsyyZCISyTnO89j3p+NxOM2nByb7YdB/8/EYqtrU/vD+8w+Pj7jZNCFQ8JuXL9/suv/ml6/f1F5SFFEp87PnvO7ssZ6F1zOQRM8IdIsRPWHvBbSk1XHqc2dXhnIXaPOnVCZ4eob15VZ5nMX6n9niUm+cS66fBv6C3jsCLLl9XfuuK+0ev9+0TdM8noZxGAwElFUkqjim4KpN2zZ1gwgx53kaJaUUo2pi1F1XNZWTnOc5ncZY3piqxikmSiXfR3a2Fl5N8KYwx5iLwiIAOmbHSOSd984F7755cSkKv3v3IaesKlGaUNUxZwDYb3fsmENQYjH17HeXN95XROwcEzsih+eW7/r5yxxp+aYhlk0CpYBlotr7XVPFlGeRbeWvKhdTfJimi6aufEkVClP+qVwoUr9cemNyFrOyJTE1SPP48e//X7d/92/VwH/zutle7l++9t4jURl7LLO1atb3p9PDl9PxkCRfv/rmwxDf3304jPnd5499TC+ur/7Ft9/sN81fv9i/6OoXTbP1bCoxRhF53tdZzcbwJzd+ocqXHy5nSHUZdysx2sDOwNCyz1rPiaeZrH3IpTyHM5z6HMuEp9rombU+Xf9ztPmJZf8zVwoATiQnySBgYESsBsd+fDz0Vxe77Xaz23aS0zBM0zzN0zTFpKpJtKlHQ1DReR4rR7vWU+sB2pRlnOLjcZxjVkAFJUTnnBGqaYxxRGyauqoCIqpaElsELYiyqWNum7oIh1TEjsv7mUqqAGYxZdPps936ENq27XP2RNL3OUvgV5tu50MFxETsXXDkaBlQwMUsVw+6gEErOL36FaOl9c8VEHEOTJvKl4iDVPaewRnVFhE7a/QjYvmkZjFGU5Wph+lR5lHjIMPjXj/u6/z6z//zb37zL0PbMTsEMtOcc3knagqI7XbXdJuLsf90OPy3v/3+//39+3q333399n9/uctgPvhf77tvL3YOLcdoBiJalpg991tncOccggFWJQZTPa9dsfWzI61UtMKvo3WebEWOFttRe25ATz96+tLnNvccpTprkDxXznkOfP4JDQ8XSVCX8qKxkFIepgkASgU/9P23377pmqv9xe5yv308HL58kfEY72NOd0cz846DZ+/Z2uqi6rZdE0J4PI2H4TaLFIUwAiBmdmwAqiYqZWJE1Zo6AELOOaU0pVQkx5tQX2833jskSillk02oTsN0HMZ+HMcs+67dbToX2NfVwzTaPF1d3wR2TG7OklL0VeXY+TX4nj/tchWe2eh6J88/X442AhoCOfTK3rkCo5R9dilLVq29R1wWlqrJOW1YdH2ZQwiZSMbD9Om3px9/9/7LXR/zv/njY/fmN/+rv/5ft123CAIAGigVGFi15ImE+DjFv789pZR7F/76z3/19dU+OPdmU99sujYEzy7OUxIlHyQmXbEtldXPrUofsPpme/bZy1b1J5+3atTDsytiyz7jc320xnAzwJ9UXWcwAM4z9ef04lkqCQBL5loSvAJ0rK+x/tbyeFsxWlM1RGeiTOSYpKzEEgFEXwUzBdE5xpRTzHmeJmJu60BE45ymOXrvura53G+JcUpJhxiSjtMEZUAewExs4cIiOzYzzJhyUtMpRgCrghvneZ7nLALMRJTATjFuEKtAiuCRN03Tj9M0z/00j0kc04urfVVV4N1ls/E+7Npt8CGKfL6/q6qmabeeHRKKqYoxMCIy0Pkq/gnA8bxNcjZeW7zpMhZgBqpo2ZjAg0s5JZGCyzt2pfnMzFYmxBEBkMi53Qv38/9y5lcP+fcc/P/mX3/9s5//AgBNFRxIzmpFLVBzFjMTk2FO/3D3aEg/TvGvr/f/y59/TQWmAphE5iwofUQKvvLBm6mrUEcxMwNDwpVe/FMYp9xwNABewaDF2tYEAJ9V6XbGJ88t+OfX6yz6dfZ/9icQ0k/h0jPXCcpg8bNffMqPn52f9VeWH2HZ8uEdt7VPYqfTwGaGVFV12zZZdZrGmCXGRGg5ZUKovGubOmZzjKI2xtQ1gYiGcYoTxJRiytOc5pimKABAqsEbm9ZV1bW1qU0xBc8IeOwnUa2r0BJlNTXrnHeAKee6Cl9dXY7T/HDozbQO4fbYm0jtvahNKVUDfH/3eJqnfde9ur4JVT2mNEyjmb568bqqayT2zpV1dZIVmJAUFjbikxN9ss4ycrAe65VSbss1ZPToTDVlISRHJiIAEHMyfXq24reZCMBE1MjdfPvr/auv52EAg/F0ZOeWBjRYTinOkwHY0moWVf1q01SM/+U3r4go5bTyiC1QYdMyqsVpBEQXAjnPdQ1xKpsqFmVqhCXXXM+gLYNrslok6MpFUoCVybSCSvj0uz8x8sUB/7QGen7az50nXSdCi8iSrvSZf+Yd1uPzU1+7Pn+5I66E/zHKPM9JBAG9c+wYAULw3vuH4ynH6JhjKuFYg/cGIIRiMIxzP4ybtmKmrJYVH09zP8aYxMBKnatmKcasmiQTUhGzGeY4zjMRBl8jUZaY1Vow59gAKu+C9+OcssygOsWoYNttVznu+8EBTTDdno51VZ/6McDd5X6bU/qcZ5QkOb1587Ou3SCgmGpKhAjgGdBIi6h5cRN/0pb8yd9XCHAx3GJUSyWUAICZk2QoU+OmKUvKOaYcswDiRddsqkBoKpkBqiqkmFRyShEAytCgmJUiScHMhBBaz7vGm9nYH6umY3YiYqZEzAgqeTktiAYWx9Fs4KqqNjs5PKhKoamVxPOc0i1TFiWs/0n+95wyt4AbS2IA//zrmRn9NBNdje9pz9ZPSqP1kPynn/L5v89/ef4qLqacsxiAqjCRD2G/7TabRszu7h9TakwtiyCAqExTfDgOjOS9qypfV5UBJDWiZrvZpKxTfDRAVSOipmlg3XNvavMcU4zBeefdlFKMkRCDrwxwmOMcUx1C19R1HXKW0zAOc3JMSKhIWa1mDoSFGHCcJu/4zeVVUzfvT4/34+Cb6mevvjLmKcXD2L+Ic/SO1ZUYXSSYy709ry1dGymFX4uwEnDWS1pu6PMEoCwsKQ4TdV1wSoimwoxmnFUt5ynGL6ee2P3m5XXwFYdwru6ziKjYIseiAkk1g2RclpOjipRIG8eTbzbsHIBJnJmdcz6ntBbLiIg5x3gY0bmq3Uz9oUDITzf7nGqDncP6E1b0nAXyVPuvlfXT7z5Z0p8Y0/l4P9n3swv4E2N99n179rpPYf1cUK1GD4tLB7eoQSJ67+u6Ct61XV1V1TzncZol5xB8FeoquHg4nYZJsjjHAJpFUspV8G1dEWJKaYoppdw1VYoxiTJRUwUgGscxQraUVCwDFpkxQHDeOe9hcfJIRGoQYxpjUpFN21ShBbBxijGnnEWdtCH4uvrx3YfrrmWik6S2aVTs/vHovPuLX/z6Zbc9jv0/ffcPX7/52f7iCpC888ysYJBVCQtCdMaqEZD/tMW8LHt8usSLk1lqj+W/ts5IFAo5oGOuAUBVVP7N3eGHMX11e/jXl1002FeVI9xXTGZieppmBCPNc85q0PJyYM7VNxJZzpojgnnndZ41ZyNi5zUnWyoMW1DXcc4p1t12Hk45zmcDWe7xWiTBuj4LdNlZ9RRe4dkne2ay+J+quP/ETS5G9vxsPy/V7Se9ejv/FgA8yTTj+QiVN4DP2vSufKtMZgbvEbE/DcfjKGpMOCC8vL7Ybbo5ZyTynsvT5Gzeg6jGlEx1inHTNuQcEW26RtQAUUQNQHImxMp7QkwxqSqTg7KHE7BspSGzxlPFfOz7snq3qiomFlm22agIM9VVcMEzQMscQuhTvOz2L/b7u/7YbS7fvv4659Q/3k45TYCB3t3df3n56u3F/jrGRJSpiEgWimpBU5jOF3ptsix1/p/6jKcjjcvC6me3BwGQEM3YqPb+b+6O//Oh90T/+Hh4xfr9FBX5v388/pfb6v/w9gbRpjkxU60xiCA7pIAKBEiAyCQigIjOaRZEyoAUvOUMZiqZvJeUVESl7K5SAMhxHnKuuo2Z5TQv1c5P4/jySc8+71nyd/aKf4KcP/eUPwEvn2OZy3qPhS5z9n/nFupT8vDc0FcsFs7Txk+3oPxweRtODcCUVFRojlHVUs7MXFUB0AFAVmWmCh0SNnXtKM4x5ZwIzZTBOyIGoKyGWUSECTdd44NDs3GK4zSnqGZlVcgCthXV3PJeU85ZNXhfVX7KmR233hPgNM2gIqJTTF1dt22zbdu2CneHY+29ZiEEVL3aX/zFr//ix88f//7v/v2ubX/17c+96qfbL/3D/as3X19fv5qnYSZyzjdVretlLUwSXdZFPiVIhShz3jyGz6KkWfETVnrramWcwyGAmBVWChL8zcPx//JPP/51G+7NDlG/3rV/cbX9NMt71e/j9MMw/aL1O4ZCMDRENCWwlShgZlbu9NJKIFLJ7JyAGJS9P8Leq6lGUS1NAUNEVZmHU2g6U805PYvuTyX2k6vDn9aK/0nk/HmOuJjPfyIHff7M9uxXllPx01ThbNn2TGvkWSkKz3xBeaQ6RhOFnDVJpDkBoqFtgru62LVNHTyHEGLO85xyygjqHBfQNKs5gizq1LZVqOsq5ixJh3E2haapnHMGs5mVTiCYLSo/qp658N3LB2UiAFC1yrlN2wKCaFl1h0l0ivPN5cXLywsxm2OapjmpxpxCcC9vXjSb7T/8/rcPDw9MBETv775c7y9ev3rddFvR/Ic//Pbm5evrq1d1qHNOMUUicqX1icDEBaImwmXh5wqYn1O05YIut2BZn6cqktMiwAQIVNaRAiK83jT/x29fvQ58Efz7ab5pm5ppV6X/c3s5ioGJrt2dxVkgoxkSnVsva263JMdIBGbIbJKt1CIGRFSq/udmJDlP/bFqOjU9Q/e4OrY/MaxzWY3PCEf/3PLKd/SntOXnR/d8ifSnRmk/bWL95JntKcdYar4/qczMEMpaV3DnF1109wi9D0Qcgt/tt4FJVO4eDsdTn1M2FdWi5oPM1DbVHGWe4/3DQbRjQkfo2naeYozJnOaUwZQIELng82qaUo6iVfCOfYwp5VzsJYvsmrYK7jTNlXNNVT2cTuM0eefrphHVEHwWNQBP1Hbt22++3u92f/junwjx1cVV17ZieuoHR3R9efPhy/v+eHzx4tU4jff3n0+Hu+BDaNqqarNkMHPOMxNTEThjWIC6Mz0NaCmI1z2OCudpXVVBRCKGBa0D1AWp/qpr/3c/r2LOIvLtrs0iamaKBNCSLroyZb+UKi2eA1cMdgUOVo9ipoisJsxcFqmBQc7RsFRsT1VR+YuKTMOpbrppONlPgzucyyBYq/pnEdz+U2X4swACf2Jnz//yp988pxPPDH1F4J8euqYNcnamuLQMbM2jABDc+vaW9xB8aJqanTv1Y9vU3LUx6zTFOCciZOcsC6oF75goxtSfBhGt6goYL3abtq6c901dDcMwxwQI3jvvuESWmJJkFVEtRSuloj8OZoTIxMMcYxZX+kDEu64jJMd0s992XWuAn77cmhoEt7m8PPVj//iYRXzw1xcXTHh7f5hVfFXffvngiH/1q9+4qvnx3R//zbvvX7/86te//ksFNDXvfZYMgN65um7qwGAoubD40QyQjAgXPbbiHmRZAl0KbVVl9lg2qpSiCklsWUdmSMSsYAWHN8kgGQqOCbIM+C5ZAcIiH702s57d0eI/EUFFDQnLqkIEKEGc6LkHPduf5jyPg6uqOI5PdnNO+NZHPuXW///tcgm16/88fy148r7rz/75V7HdgsuuvBRcCqeVFbA8/RpU7CyKvLycqyqfs2jZjVTOlSoaSJbbu8eYEhEBYt02lXdt2xDRPI2HY386DZIFCduqqqvQeN9WdajCNM0q4p2TnHdd470Dg893j9OcVMqlpuBdCL7yzsxSFlObcpbj0Xned93Fxa5tmxhzziCqlXfsuGnqw3E49H1m3G12P75//2q7/ebtm8/39xfdZrvpTv345uXLm5tX4zwhNGZ693D/3bu/ufv04e3bt7uLq7svn9i5bbcLdU3OO18hoimoWRkUJuaiRBy8N3zyNabPTz5kk1ITMJKBZRUEICRFxKJVbFr0xtY7sCwJBwAgWgzCAEyRGJjOFrkUYmaABEiw2C8qgErGspDKVERyTshsa+b3VHQDGEAxXx9CnOcla1yhh+dJ5BKF4ZmbLAH9mcnCszLo3Jz8yZIk/BPztBU7sNUfnst8xKcmp9kqg7OEnnOGQ7gUS6aA4Lz3gGh6pt8Alu2DoNM0pRQ3XXtztd9tuxBCU4WU8+3dwxzTiupp8ME5J2qH44l6RMKymogdq9g0pSmmw3EYp5glI1IdQlUHJEQEWYhAmFXnaa4tBBdP/TjOaY5RRMukxDzHUz9mkV99++0k+o/ffQcASNQPQ/But93WVY1I7WafVU3leHzo2s398XD7+cMvfvHLN1+9/fD+B0kpSmqr+vLi6ubm5eWLr3KmmCMiiuSStIhKW7dmy+7dP7l5VpryOT3P/XPORGRPHnC5E8GxY55jMY9VMw5gqYfKUB0TIi0M6oWvoaZGCFguEK76mrYM8Zb9vJJTGbnS8zv7aUSO8xTqmoj1zDwqkO0/QydW612LFvhJunnuaCxpYvGChGtg/6n80tmnmhmu45qr08ZlKRIuj1yxvifnamsf6+ltoWu7ulz1OeaSJprqtq1D8LcPh2makXC3bRihDc4AjqdhGoeS5xJzFcK2a4eYYsxNHerghnnuh2gAqtYP8xxTznkYx5SzAniCUnk6IAbKkMt+PAZMoimlw0mGYfSO67q+vLzo6qqua+e9iDrvFPHh/g7MJMs4Tv0wusrv2o4QQ1XFnIbTQXPabvauasLh4X/x1/+ZIf0//6f/cRrHum1eXl7P0/zv/+5vfv6LX445vXj5FgFOhwcFaJsu5bhpd6aljWmIBPh86LtsQxEwOCvbqMhiAbgkfLisMUFCzIpMWSEX+KZw+emcz3FRElyW/5bKHckB6vMArypLhgqmAMV9iirquvZgNZaz5yv/TvPsQtCoiya3mf6pr9OnxPeMjuvq+nBFJp8kZg2QymPO5rt0n3A1cQOAZ0UPnh21mpbjtiAUxZSfoCU9E/cKVLtEfbdtAiGa6mmIc4wiQgRq1rX1OM0jWOUYNR9PhxxHx6wpgWkd2Ll6jknEbh8OKQsRT3MkRFObYxYRAyiiWVNMKWezMpkEWS0vxFoBM0dUh2rKcY6gak1bFWYTMgfvnPPOcUppu9lcXVz88cN7NGDmvh9yzlHFJPXjeH152bXd8Xg4PD68ffMN180PP3y32+67uv3db//+ctMN3h2n+ePtF0d0eXGlCGL46dO7dDoS49ff/ll/ug9VVzTzUYt5aBlqOk9xmC5pJYADMFERlRKZVcW7gIi67ODFcqOXmT01tFX0dTW+NYiDwbLaGaDQn/mc262RDgFW9TmRnJJIXlb5LNT9n3j6c7sop0RMGvMadRcLXbHes/tciO5qRgC2KC0v7Bo4O0GDwhzFp3pr9ZAGy/rDAmuu45rL1g9bX6BYvhoiLU9sBgiGeB5GWf1qWbMEjtillMtnzCLjlBCyqJU6pq5CcCiSEbya5iRMtN20O9B+nB8lDykP46yiVRViygAQvKvqKqUEgDGbyawqRUFHwdCU1yn4kp+54EUFVStedm9Xwfvg99tt3TaO0Mz2uwvv/e3d7b5tQXTO6dgP/TgR4nbTVt61TTPM84/v37VNQ879+Mffd93m1YtXd7cf97vdfnfhvD+ejuPQf/funQb/6tXb4+HxH//hb9M8/+qXv9psvzDi5usLW+jwGgBKwnQe1zSzLLlI8SAWxExKTV6mPNTUMZuirtt3yx94qpiXe1lo+biMJwJqWT9ugIrIT9j1spUFz65pdWUl2cgiWsYTnu12sXKiYDFBU1Xisk3oyX8aPmWOCzVkhYMWuYViOSXaLvSi8jA8H4enHGEtXlbDXtBiWIujMztpSTRxbW0iAFKhv5XsG5ac1QxBTMGgsMUwpTzFPIwppaRmpdSum6YJbs7qxFqgqqpTzqhKYCnb4TQ9nsac1TEpEhFLlj6mXIfgPRIP45xSyqkUFQtrwS/zzMhEazqiKaYk2QCaurrc7oCgaWoAPB5PhHh9eZEl398PouqcyyLBe+94itEHf+yH3/34fkr54vLCVC8urv7p9/8Yc7q8vLn7/FFNAen7H38AhJuLSzP75S9+WW13t18+f/jxu6ZpLi4uI8C7d38M7FzVtE2LzE23sWZjZt45Ji7WqaYxpUIzWLM+IEIClGWUv2x4fAq7Kc2LldiiGgyICFRi8AK7qqy3r9w/QeSnnK/cZVNb3wPiUy/2yb5K/H1WERdvVbykFZBVZdGkJVp7Rc9tbX0xfIrQa2pamKZPFVUhNNia/xTLfNYNwqdGkT0p6BWPWV5DdS2zSule0tmnrHRx6mbm5pjAICbJWbxnx6hmIjpPc+UdVt4MuLCO2c1JYlJTGcY55cyIGUBFDDCmOIxCCETglqkIK8a3MM8BHbF3jpmYmZ3LWaA0UQC9c9McY8oxRwyBs+Q0A8DlxRYMPn6+jTGG4AGw8h4BArHv6ix5irHThrxDw8vrm4+fP8U4v337zd/+/d+Nx9Nm2xnCOIw+BMt5f3n1+uVX96fTH7/7XV1XQz98+fLl17/5y5TTOM23/+7/4xG+evP2xatvDv5xv79SCYSYVyGgkuQtbgnRyrYcRFEEQJHs2CEgM+esxWmBKgEoMZqpChEXB1xIXmaiOWnO5D0RQ7loy7xlua1lcGjZdyo55ZSW8FeOBDwVFksNshbnZ1DIRBbB/GJuqrAKKp01vNbzsox+PquknhCf5VydE8dzYbg693ORtuTThmWOdAkBq+9dnhPxfC7O/twQwBTPTXkwxz7klMuZ8Y6JfEoiElU1paRgnjllnWOifhBTUTVVJuyampGkH89oHKEhUk55xsjOMaIZJpEygwtgIpJyxgLBSFYzx+ydA8CSpKac5yxM0k9T63zd1I/9+HgcmjrEnKecdm0nWaY4dV3TVvVp6KcYk1ntQ1XVHz59+u4P371+8+bd+x8Dgdbhy5fb4zDGlK4uLm5uXoSmiXP89PHd5Wbb1HWeY319w4C//aff37x49bd/8+9f3Vy7UP3dP/zdX/z6L+ebPlRN1bSIxMwhVLR0vJQdn+//Ot5UdAMyswOwLJJyWu9kyUORic20JJqw4FDFnSiqAvPKGIH1flrxj5qlCFLM0xjjbGoGhkaL5xFd8wAzKN3bpRhbK2wrKxFWwGyt/JDOQfrML1JYhOXxTBxerPVcSz0VW2cY9AwB4NnMivktaTTg+owAZ3r0E3awAvzFrksaUJ4QXdt1x+OxFGwpSXmrTCiCc8zTOHnuxjmmlLabpgqOmHxw6r2oVlVV1ZXkHLOM05xSVjUkMjBRyTmrCiGWJxVVAJQ455x8Dt45dIwOF5U8R66umIgJGawJoWKXY+rnSbKodE3dAKOqzpr7GBsfvHdJNac8TdPd4/HDw+nv/ubfv7i5dsGneQ7OHY+37z5+BsQ6eCYapylnvX+4y9MUmsbMfvbNt7vdxfc/fG+it18+OUQR+b//9//dz968vf386eOHHy8vrtpu02z3l9evgg/2VJlClAwAniBpzpKYHBhkSSK5jFUgErrFJhQFiZdweXYPhEhc8FPNCZDYLbtBwNCQSxFpalaYpGnOklU0a8Yy0KxqJuUtrcjmGoqX/yx3XmHtkRWzL4iCySo3QmVH0rMVXUuye0aFDADLiponn7omjKV0On89++vK+i5+8qy2eH6Y6TM4dUEfdHXYAADgXl5fmsg4TM4bO5UsKWY1IGZE6vtJRDdtzVVIKRECCQpReWNE2LWBsD710xxz8OgcI2LhmKYsVoI7MSFhSkW3PIuKzoBQOzZRNW2qGj1TFlhTEwLo5ylnGebIiI/9OKVc12Gapl23ud7vg3Mh+K5rs5lmOfXDu8+fVPLP3ryZ+z7PM+93v/j5z/e73e3D48PD4fv37yNY03af3v+432xjjI7dj+/fnaZ5mqdfffv1x0+fLvbb4+Hw4mJfef/+4wfHdDw8MuLNzQvLOV1eV1VjAN4FE02aHbEQZ82mqiClrBbNZ0wRlq6IIdHCMTc833qThVpK7CRHS8kQFRwVfsBaPpeSSCTHeVqTJVBTNCqJqYE9MZUBDHTF/FeqK6LpQrrGRfprjcugVlQQbdnDsVq4rdH8yaDU7Mw8xuVUrMZoAAAKdtYggLM12+pX7VlDy85w/pKYPund2E8M3PXDmLJ47wAw5xycY6JpmhHNOQKAnHWaEzOfevXBVz6ktLAumqZqKp/FQuVfXu+KUZ7GeZzigkQQEaGI0uK6nxcQsQ5BTeMsaujUERKATTE2REwcswxTFMlKdNbDceyOpz6myETe+Vc319f7C+f83f3dzX7XEL37/Onjx89dXX9NvO82X79544Nnoq+Cv/7q9b/727853h/2m+7icq+q+/32TVV9/DiMw+C9+/Nf/uJ3f/je1Mbh+Ob1W+/dH77/Pkve7XaHu89//MM/fvXV21DVdbclYnauabeDSJJYh2ZJrtZUzJ6xKwzQ+VBcqaiZChgsfA4AAEUm0lUxgKQUX7AgB6o5l7nNlauiALrkkiq2SM2fsaaFS28ABEuHfXGLRYV2ibrP5RdtdbdwLnpsIcoAgKEaPBU9qGC0ppJrNQ7FjBfuz3OEoER5BSRAfXLpq9z4OUeys5ozrP6/fAw3p5QlL5mAihKGyiNBirnsPDJQIvRZ6qbyvmLnppjnrJ4UoSrLgOq6Ct6PU/z45Z6Jm7qOqdRGwMw5C1iZ6CU1LZ5D1KYYg/dokHJCRF9z27ZVFdqqhiW82ThKzoominjqh1I4b7t2s+kIoAr+5YuXMaVtW2fVx+NxmqY0zxcXF/v9TsGY+XK/u7m6urq8fv/p83g4jeOYyvQSIjPfPd4fjkdT/dW33xCiiDw8Hl6/vLm82H+5vWWmHz/effN2bJoBTaahlxQPD18Acbe9lJzqpkOirIKIiIxUGCRWEr6zUKgVEQQELHPlCFDUhnSBFYtmTvFJarks1pWcJGfNKeU0p0UxGBFVwExtAbkWrpGB4TKVtqCturi/kmguosVrP9UWSsC5vwolSVzWKZg+ddLOmWxxo2gA9KxFCSuqD2e/f3aF51T1qWlUMktcIfw1xqyyPKtnXio+QNc2NQMM3vXjpKpgSgjOOREtA8kIBGbB0aapxznOMYJBTjmBNXVu6ooQx5jnmEUtpkxmITCgiZbN2yriRAUkkyoYnrnXMSU1YyQQUJGYU1a5dpel7eoc++D6EdQ0CUA0MCOm4EIUeXg4pG13HMemP+23G2badJv9fv/u/TsQ9d41VfDMMUUD3HQbQ3q4v/3zX/383afPn7/cEyI693A4uo+fReTmcj/N0+Px0NYhuMu6rgqmO07zpm23bX08Pswxtu0WEKrgRfI8nZCpblrvAwCknAByQXKLTFcp1WHp1WBeEH7IOcOKqpSVZ7jeYBWFs+tVlRjnNOcUc86Sc85pwXdK+DRd88mSZS4pxNpKOtfpdvZYxZ2XDNcWMPQpzyy1O61Li5dAfi5h1r75Cvg/83bnBy4fakl8n2eoZ7D2J+XQ+hJrqHkq7lf/C65t6qaqnHdlGKJoVDChGaQskkU0xRQBAMmFyoNqzFlzruoqqw1z9s4R4ek0MHNThxRT2SxjANbYFLzcH2QWWMDAp/kVMyNE9AgAIoKqWVQVLrab7abz7Jhy5TgwE7NjIoBd1znnHk9HMUCDz59vD8fDVzcv6rqAtm6/2e67TkRPp36co3fsvdteXH1490PbNiGEKc4Xm+7hcDwceyQCkctNV4fq/vEQvM+iZvrh05eu3f7+D98/PD4iwHd//GHT1k3TgGmOU1XtiB0Ano6HZnvhfAVYHKLmZM75pde5msqSc6qmnBaitCgYIDGVsXNVMy3cZFAoSnwqZTvIlGJUEQNVlbLg+sl4ylc5AWujG88LiwtOuVZPz1ua6xDdancGRsumw2clzLNhkTPiDmiwHBJYnxEBlzO5ErWW/AFxwUefg/p4zjaXRPWMiRWaCK4VfxnIdwgYKj/GOQRP2MwTD+M8z7OqFsQPzAiJmFJKpVt1GmdTBaS6aVmAHAFQzNliQiLvHSMGRwaYxLIXcmyTCZxR3+X9qoElZRVmt2areZhmDJ68I+eAqKqqrqouLnZJpAm+Cv7Uj4YbSXI8nk6E33792rHLOX/6/JmdQ4DPd/fjODKzAjim3WbXHw/7rmubGsF2m/bvf/u7vp9ELTgHYA+Hx3mevPfH03BztVcVIgJNc5xVJSV9eDyo6X5/pap395+nmK8uL8dpHMbeOc7TWLcbKrJTgCKZ2BXRKGIHBqpZl9TTJGV2zN5pXsY1i12W6geITKRMHeWcUoqSs4pITstocOGWPsGEz4J8ufOEQOuylzNgeW7hrKZ3JhQtRlh8KoIZnFuQ5ycv40IE+ATWL65vVRhbSiQ711SI+IT72/qEBbh66tsvD17fJpgV7VIDWPcuA7i7h8cXl7vah8izCSPzdtMh4jQnYfHgiLmqAiL1w9SDOWYfvCrPcybTTVsz0ZBSXdcxikj2zJuuEZFhnKY5TXNyzjFRybtyVitI4HI+IYuAgXMumxkii8qcBjftd7vdtkveXe33b1++OI6Dc3y13zl2D4fT6dRHScfTMCb5cHurWU7zlHPe1LUB5pya4Ku6mlM69KcUZxWbU2yrepymLNpt2kbyxXYzTdP9wwEM1OzXv/r5m5fXovrjhw/9ODimnOTL3UPK8g3xZnfx+eH++x/et93xuh+OD/fj0P8qi4lImhGZiFyoyHtyys6LZsrJl12jOSOAqgCgqjIW78kGqqLE3gBVcrFByTmLzNMoOYmKgiosY/gLYgMLfWNJDkzXOgNFtRTrstAAFoB2NaMFzCle8Mx2IwDQ83DSmm/YatxmaKBriVtwAFh75etbOhf7TzATLm7IbBlBO9t9yS8AziO25av0QRZ/utRPrj/1cZqK0qepwhzZcagqdkNNVU650CXnecpZidk7V6ux9yr57uHYT9FUg3dVFarAwyRZVdSISAEAiZkZ0TmHiIVejkWNSu3MNCvnuPTysmQ1lSyaUl2F0DZVFbKKY3bEaMjE2+2mbRvN8vv5h9M4xml+PB2naUazT6qO+XK/e3l9dbXfV1X17tOnH959GMZxGqcpxs1mc3Ox7do6xnT38LjtWlV7PPX/8i/+7Nc//yZJ/nJ39/s/vMtqwXERsDud+v/4u9+/fvPNTHA3zH/8dPtmnOd++N0//f7h0P+Lf/GXu92pcd774EPl62azu1yKJMkSah+qMqFR3F4pPBGBijwTqJosBZNIQftTimqaJUtKpcu5AqIFk8en0LuEzadypfjFUjTj+ohFDRPXqLo6UzrDkKuV6FK2r89cCCCLn1vLdizmpE91efG0uKpEPOWWa1qwUpFLurxWTctvnmcVzxZ+fofucOwRkZ3bbjehrjeAMaWYkndeJINBzjmmVAZ3QEUEY85d8ORDyhJPvWNOIkDUNaEKDgBENKoycxVIRZBwKTOXiqFcbQAQQkQiJSRgIlRRRqir0FbVNM0GVoUw0BgcAeAUo6l556qqIsLD6ThM09QP4zAeTv0cIwIQU11Vl7utqX748tkUvOP9pr29uz+eeud407XHcVK1u8cjItZVYObf/Opn37x9lVX7/jSOU9s20zSpalWHn3/z1ePhlFL+7rt/+vWv/+z1zXXvXR5Or66vQWZCPTw+quTBeQS4vLpxOTG7qttKnGGdDXI+lFOoJmBYpmZIlZgRwETLP5JzyjHnJJI155K72iKV94SFLzQ5WPpOVhL6c1xagRxcCpW1Vl6yghUtX/PjJ/LA8s3FQa6Gu/yIFnMtAfgpjUQAKmqMT4CnmgEtT7u8UTsH8lIuPWvCrVIrT19rgQ8A4C4udmYmIsd+qELYdM2ph6oK3rmc8zQnG0a30BwppVy0FOdpZudCVQXvCE2TpJTDflNVVYzJzBoXqiocjv04jEzkvXPOaZaY5QlaKDnVgtkSoTMEQyTmdrfpp/n21Hsam354PBx98HVdBWIi6odhjvHj7e2HT1/iNCPAZtN12K2QDc4x/fHDx92m27TtOM9meLHfFRIwGXji+2M/TXPbNsMw5ZxSSh8+fd51fcy5CuHNq+vjqf98+2gizru//LOfuVCJ5q3XVz//um1+8/B4C0D7XdvUVdd1VdUBkqo2XRdCVS6tr2qRhEhZMwiBmWp+DkOrgmVcrAELyJdzjDFFyVlywem0cMll1YkodcxSzdjzruMTxoxr4vjMFa1hdGXCwTlJffKAK3wL51D/pDuiazuyhLtzeSSqRnp2eM/B1vIiq09VtIXpjE/nxOA55o9wttq1HWauqsIcU1bLcwQDFQak66urOSbJqZqjqmAi7xiJYZxySmXpAswREL1z45xEJGdhpqqqADDnxIRzyqdhGueUk4TgHbsZ4notjBANsaDRSzWmQkQG1o/jdd7dXOxzzsfHw+HY18G/uNzXIUwx5SzjPKmB937bNBLCzdXFVy9fxpSmeQ5VuH849McTAoro4XgChMv97mLXfazC7eHYz1OcYxX8zdVFP8XPd/c5JQEL3n/4+GXTdduuIYK2DteXm4vt5jRM96fx5rq52F82bRecC1Xz6tXbaRpD1RFR1zXeVz7UWYS8D3UDCCKJfe1DU6aZJMUzplNOJROtJfJiZCnFeZ5yTinOWfLC7Suw8XqMAQHLHCkUKchFXn7BiZ5F1hX3gZWuX6x3VWWCBQJavO0y6AwGIEs9DmcjO2eZZ5s7HzMzW6ux5bsrkADnR5UibMlxVdfE5KliKz+n5+gVLpwqBXBfvtwX6K4KgRDLCJvkNKecY0pZnQ/OOWYGM8mLuLo6ZmIQ7U89EGYRJsxZ5zQwYfC+9IHGOYpoFoljTHOUp+0CS9auxXmUTTMZkAwAR+2/Tz++fHndOj8hSZ7NLGdFxNMwHIdBTV9eXKRpnqa5qcOc8vtPnzdt5Z0jtcttGxinOToCIqqrSrMchzGrff3VKwNQyXUIj4c+xlxX4WGacpYvtw93D4df/2JTRMlF01cvLt+++urz7e39f/zu8XBwRDlLyrcvbubLy6uUzYUgpmIIgM75qm7jPKZ5qrqtgcV5cL5iZjPIKSKUFbumqpqTEbNzCGgGOc4xznGeYpxFpFTutvjNglCVuLoMnVpxoIqIQLROL2HZ/XkGbQyfxM/PsXaxicV+Vj+p68rG59+FMxp1xrJWu7GnX129/9k/r8H8jGqd04EltVv94zn9XTz36pLPlo2lSLp/OLRNBUhZlByLmpo6orZpBrOqot2uVbWc0jRH0QBgCuYBJEvOqeQ+KiohnPpxTtkxbdraOY4xoYGYzTHGORaWl5oRoq3Y7FNubgaqiECqHjGLHA595R2IJJE5JT+ObgieMHjHzJu2Gab58mL/9tWN815UxxjfffycYu7aZk4JweZ5ZuK80cv9znl3Xe03m1ZE+3749OXu4dg7x8H7m+vLeYrvjp/3280U09bo4Xh88+r6+upaALJq3VSmmtIc4yhG//j77/D7P75586aqq+DcjDEAzIC82YWmi8OJ5yk0XZY5zgORK+PzKmIay9U3tSxJRdh5IATElOI4DnGeYs7FenC5m1qwQTUjpNV/FUcMJfIyGhHpMhp65rydgaTVONdBElsodcsD1yQBV+7egkQ+C/5P7lCfynA4t9SfQU0FSyhHyM5euCQNYmAL4HD+3urjn3lcRCwCz+XzuJzzMFNTV4QUY1K1ENz+4qJtG2Ym1N1mk7I8HI7kfFXXh8MRwBLxbHNK2cxMFh6dqG27pm6qlPIwziK5TNHTAh0rITgmM1DVYqnlzZZ1RmoGZRmI6q7pri8vMthpmmIWFT0+HEikaZqL7Wbb1Z65q6td11R1NU2zmKWY2rpJHO8eHvt+KNhKVVf7lKtQpZSHND8cjllknOZQ+b/+q18Tu3fv3t3dH+aUuqaOKZ/6sR+GaU43lxfHfvhyf59S8j7c3h8u99s5SYrx4XD6/t2H+8fji5c3F5vN9eVlcEFyStPgQx1cmKcBCR25lEQtRxNids6X1vpCvlvmmRSZUoyScwkvjssoMzAZAIGKSjYreBMCMpArK1TtfEtLHFomTc7R3tbYuth7yQfWNvqS9i1i5uXYlEJ29R3FVlbyGwKYLK5x0T5eQACzsoDG1mhNhnbuRuFakRlgoRCUiqPwVdYhu5KcnNNoOD8MwJlZnEYH5p0rWgSqELxXEQCLMQ/j1DT1brOZY5rnOE2zmbJXAxQdC26SJZ81VVSUEERV1VKSmFIxU1szKaRS1Zdc6Xy9lgyKCrFScn/q1UxyNsmaZeyzY2q69hTj4dOn2/uqCt4xA+ChH+7uHx4eDzkmBGDvysXt2padG6f5+z/+yLxsjClMqz//5c9evbh+9+HjtutE9OXNRds0x37YbjY5p4sdiOT3Hz7fH45NXTFxCH6Omcl9fHy4eziM0/xPv/99jLHfbhxgW7fBeVVNcSq8u3kYoKpNNUsyU5mn4AM7j4CGRkhnEDFN4zT2c5wJlIJDMC4SlQAm2cSySrQsksyQ2NkSuBdsbknxCtKJiMXLrpno+bqW0snW4I2rB3sGkp9L7OdpACwgFyCu436rokLJ0Jb/eSZsuy5OXvnNuAjrruG7hHRd84zFwJfS6uyKz87LYUGFUqpyruttacE/HE+OKcZ0PJ2mOXYxHvspptzWoW3rUHlHNIzzPcI0zRqFkGLK0zhpzv2i407OOchyrtPVQAxMDVTWDKN8FDhf2uINRDVOcYDBED1yW9UnHUz0dOoR4fLiotk0fYxzSpumxhHuHx4fjsftpiuicG3bjvNUlj/d3d3NKRFi0zRmqmKIuOmaH9+9/3J/3zVN1zZNU7+8vnCOQ1Vd7PZm+k/ffTdHff/5lgjfvLz2jkNgJP/4eArOgxoZSMy3n28t57ap9/ttXQWwMig0W8k0cyRya3RGydHMnHOEfHZRojnGKaVZJRWGPhESIROqKJhoTpKipBlVaQGmzFZl01L02KIYa7zQ5sH0+TDqkvHbOadcSylEOI/4LsTNNXdcy6IVxcSlsb7a8FLvrdXQalII5yUjT8nkGX1aC6sSNLW8h7WHUI4BPU9YAQDRVVUoW89MlUodENPj4fjier9p64fjaZrjbtshoedl5crFdlN5R3g8nbyKmBo7z46z5Kyy6dqmrQlJ1UQtx5whQ1EgkDPndb1F5Qroyj1bpNUWzAERxKSufExuzDPEeDyail7InoJ37ACwH+fb+/vdZvOLb79x3h37/vHhMAzTaRglppwTMHn2cV42iLJ3SPTjx89fv35FiA+P8a//6s8v99txnrfdtm0aM3375qthmP/ww485yeHYbzdtXfm23d09HKYYv3r1IqbkGLfbdretm9qlFEXEOVs6OqXmMwU2QCgk7JJpmZloJnJLLqdqKqYCmkspC8CgIKo5JU3TPPXTNKlmACAkYkNEIzV0WhpAuojIF20INSurpwDPFQwUJfZSbi9bFQCWSgB04evgYnDnGuicGCxhzbQoQeMZJTzDorbknraIvwMtNXjJMBcm6Yqwltuu5ZlLfxSXE2Nr6xPOR8AhQJG9iCk9PByy6ByjY3e931Dt6+CHcZpj3jTN3f3jaRhNtR/GTdfKwrsBZhLRnLKIFNLUpmvbtp3mKKLa1OWNapaMshSaz5gLiFhS0sL/MUSYiySONW0NhQyKVD5MznI89dM0b7q2auq7xwfLUp7l3fuPx74fpinO8zRFyQkVyBECDFNvIuxcCKG28Jjz61cvguMvdw83lxfbrjv2PRiEEMTADC4url9c0bsPH/7w/buPn28/fbl7/dVN225fXl89+Idv3ry5utyd+tN+t7u62O/3F03TMTM8u7trI7d4NTAs1Fheot7iXiTlmNI8T4PmCCCBUAkFUFVTitM8zTGKKpqAWRlkgqzeBSVRK7RxBMDiYpTIMS1usHBSy0/hp2o2WCLVApESLCZki7jf01dxgGfwH5ah5CUZhRV4R4RC7afVny5eeWXzrYo9oAhPNCszxDO7dC29znDX0ogwV1c+JhymGLOcTj0RBe+cd5/vDs6dALAKYbE9xMq7LGoG/TAhQlNXkiXGJBJTSipCRNM0/uEPP4QqOO+rKtQhVI4fAFJKwXyClMSYyQdvBiK5iLMvufQyLqiSNKt657pNpyLOcVkYXCj6GezUD8McvePgXRXCMExfhrv+1BfB0VIpm5klAwAi8t674Nn7Kcab66vLy4vH43HbNt5zP/QAVldVERA9nfrNZkfO/ebPfqkqp+Px05f7FGOM88vr/X7bbLfbFzfXx9Oprpu6akJVBx8KVAfPv5aKBRdqE5GBETECIaKq5JzSNE7DaRgOKBlNlAwMRHLKaY5pSsuwHq2VSMlCk0cgBWQDVDVEcs45ZkQUNUYzKFL1CGcwfJlORrVi17hCAmcbKpXKgrgv9m2ABFqmSZ+sfCnRC7BQyCZlBmABws7wPuAa4RGL7hWU8aiSDyMY8FJAL2+3GPg5IzAAZ0jEzjlNKY0pEbNqBYBjnszMOb/fdo758XAcx8kH75xjxL4fYozOORd8VnXqYozFf8SYRWWc5q5r99sNIk0pOuamaUacsxqD1HW13+1EpO/7mBIIruSpBYBDNTM5HY5FWGae5mUAHEALImWJRXLE6H2Y0xzjNE3lUBZ421YADgm9dyEE7zlLQqS2ae4eHtGgl3Ga59cvrq+vLh+Px4uL6+1mezjczfOYstvvL3/182/jNLbN+8dTXyCI3c2NITd123Z7ACPi0ujLOS83dVn+fTbLsscBl5mPp7TNUpofD7e3t58kjpAjqjgw1JzFprJ52kAAkZgJiR2wA3Jski0hKnPhvSPzqrhZcgRaQCiiMv1WOA+LEdDqHBcLWRuU5dqKLFjAcsitNIBWsGhFjojwJ2b0VKcvhlf+FITnjCzo2u4ssZtwmUQ/Aw8r/Ln2HswAwJUqpq4bQxz7QWOa5rnsCTbCbdfmXMUYqyqI6BzjNEUmSCnnnLNoXdelSxRzjlmKClfpMLHjKaby5bgMLmdiUtOU8zhOTVtvNpspzillSWmeY2k5AJGYMVoWmGNk50SfWhBqCgKFCI2ISXJCLnvEbf0qRo4AzAxExFRXfo5pnueqqt5/+Nh1m66p1ezt6xfMZAbOeSJMKW67/eF0PPWfr69e7PeXqe1Cu/ub//C3265pmrqqKnIVs/fOWyljVwlZK3xXeMJ3ibjIJgMRLNW3AnAJ+ExODWOKp4fHNE8oCSRbFgETRSAUIGXnmBw7YGVnzgGgslNmR2zeO2ZHVNZSo5qgQtn6DQaiBgDFDkqeVGyjEJ0WV1lo/kvpsoA+a7fp3M2Bc0JWCBVl+zzC8wnntY+12nGxxMX4DIpYry47P+BcTj0dl3VZyvmZiz92m+1mGMbgXV373abtx2nsRzNj79i5rmsN8dOX+xBcCKHt2hxjylKchKj1w1BOmveubZuUUk4JkIo+o5l1bTPOpKJoggjBOUJUUSYMzicUyskxo0HOIks9alBwXVRMqUiAr5XHCqcgEhMTmYGUr6KRuyLNpcvimJ3juqpSSnNKRpSLDDT7fhic4/1u808/fLi+GF6/vL5/uD/54367HafheDzst1vv64fD8XJ/9c3bt1Vdh1BhMS0kMyNiRDaRdTYI1y8oeQUgItHTfYClE5NklhTH8SQiTdXmNinQPPQx55yWvrshKRqiOmbvzcgoWxUMiUmNSJ0zREMEpmWrYVnnp1IolUtqsTCTDEUXUJwJrbQ0AQqdTBdjWqr2pUBZYKIzBL/8t0D/+GRb68+xwE1rG+n8W2t/vpwJwnPjdkVs16przd/h7IZNzeWcg3dNXTvPAKTwOI0TiAXvEanw5It8vZltKs9NdewHxOrUj9M05Tkuu7+c224753x/PB1Px5QTR8eurFxlJhYil5JlSWZECITTVFSbivSIOeaVPLMc3DKbz6p1FRAwznNSQUDHjr3zzi3D+yKmiiXpXvsiZZ0XM4vY8dQjknO8CF0RH09HU+u23cfPXx6Pp6EfwaypQxZ5+/p1XTUA+PhwD8j9NKvq26/eYMkg2RGxAZZhYCyrjvGcQS0V3QKcreopiIiFSwyQ8jScDsfj4zCcSvkfQiVSpG/IKKIsGm4lAOecUxbDDMijS84xO0fEwYeUXF1VWgVRVRe8d6svt3NvxgAJQeFp9ZGuE54GZdoTABbyUSlnVxTgGTj67IytBrloTZbwX6buyaiwLM4oTXmiUi+qAZ/j+YKDL667nOpl09cZDTVTMDcM4+V+d3mxH6bp0I8AEOp6HkYRIbY8ZzFlwipUwPzw8Ng2NSzPguycD5aHcZqmk0o1VdeXl+2my6ZxmmKKNFMIgYl16d4zB984NjNQjbnMziuY5YV3boBIRCKLoiQgECIQIyAHj+rK+u4SrpjZQJkpxlTaWk/HVoXJkXOBnUimBc5TNJCUsgp7j4D39w+FVgIAbVNnydOU/uLPfl5X9T/8/vsXVxddt5Wc5mm4uHyhgGZISLZQLSyLQEkkzplTwcdwEbl7MtBS7ZqJ5HEaHw8P0zRIWnOglGLMomjkAIyxiJRp0dMp4AgSZNGYiDiFEJiJjRVMDQiZCEWtnINyBdbeh5mZ5nOfU3Hl4S4AT0HG1ppmgTmXuaU1Pq9/oXNnaHGm628uHxlMQdath7ikp0tqscKquHbrF+9Lq/Nex51/0iZ1bV23TV2Gg2vvQURCdoRV8N4HVRnH6XTqcxqqugohlAaQijBzW9fZ+5Qz5wyIaY6fPn6qqsoHj8SQc8556HvnPRGBllm2SsxATSTHlJJkWxp5Aqti2RIW194SMzsiAUMjzwucR0TddrPpmqEfHh+PAKl8BFyPLQAyc9e1Taicd4QQY4opDf2QUyZEx06ylCGklNIwTqqKCN/98V0InhC+3B1ykr/6ixdd10hOMcW2u4BlldGCv5TUU02Zym5EMCvq+7S60dU4kYrcvHceF5AX1ExMk+gU8xSzKx0OAgUC5lLiOEQHAMSGTMSAWOAR74N3ntGZgUpWAiQnsDQwF3LQk5kiFEp4wSuhvJt1+LiEeIIiTrhy6Qvw9CdPhavBLvG4YOjwfBhkSQ7OOaYVrAuBVO1MeVk9CRhaliUxXds1CwiFAO50Og3jVFWhpGuA4IMn5pxzSqMPjp1j5yRLjFFVYyQEqOtAaCKSVcvAmkiWLPM8j+M0znPZRpTmmHN2zjFSUSZiJkRi76qqUgBOCcg0ZStjuGC2rLJ6ymxKsKtCUK8i6gCRuW6qtmmc91VTXyD1Y5jXKr74g65rry8v2bGqbruOGdVgHGfv3IFOOUuZKwcDEzsdTz74/W47x/jp9g4Ad9tOlU7DPE39zc01AJqh5BxCXZCgM+RpsHTY0bC4f1ySv2XestxGJFpq0zINl6OUoWRywQOYOWZYKjwzMzBrVKSUzwAA66aaEh/VRCQCZBEimh3XIYQQmBlpWaRbxgFWi1m9KqyHo8hKWonVpcO8QHLPQvozJVsAU80ATFTCMaxaOgvsClC6LeXt4dIseHqGwlx7KreeJHDO2Wtpi0KR/St+1h1PAxOlVNd1oIR1XbdN/Xg4ZREVTSKIGOqKgKZpHMdpnmd23OWmaZrC09l2Tds2p364u39ERPbujOIyMTKpyBCnsv7HETtmcux2O+9YqjpzFoNxGJd+nFkWwTUyAqD3vttunPcxxpyFwAxMVU/9oCervKur0HQtgKFaKZUMoGqqTdc1oRrGQVRR6auXNynLp+C6bXc69sMwlfbPlNLh8egrH95+dRrldOqnYazrutl0VxcX7z9+2e8urm++YhcQIOdYIratze51P5CtIX5B6UusffKhBoAgItM8HE/HYZhMs2d2zokCsqKCSjZAIvKOiYhpDdCqWXROOueckqSkZaifHXt23vlQOQAEJKfmPAIgE6rqUtEXGUe0M8t9yYJWMoQuNCV8yl9XJmTxEPhk5nYmSZfIvSp2L2nl2fqWMH6ePF4h0jUleMKVCjRTsl4pqmx6dtfgYoyFPDuNIzG/uHZNU1dVDYDGOsfEjJcX+7ZppmmepvH+/nEch5zySQfnuKlrBX/sx9MwZlVyblNVOeXCCmXvETHlPI5jnOeUEiLmlDWlTdddXF5cEt3dP07TjFjU/J8CQ/nQzOyrOlRB1nwsiw7zlFJm4q5ti9txhJuuDcxmFkplxuyD2293zNdgNs1z1zRf7u69921dN3UlYo+Ph3mamCnGrFkOh1OOiRD7YUw5i9nrVy+ato0pppR8aFyockoqWVWLCvOStC2hcyVglJoDCYmw/FmRxSxpmqZpGud50pwyEzMjgGdCY2VCAMfkHTMioRGqqmQpUgk2RxmneZpTUkMi77hyvmsbH7rF+NY3Y7bARes0CBVDNANaS8lzwrrkhmsyivTUOoJnDm+trZfwVip9Lfsd1rGTUhKVfSlECGtdf75Q53xjQbbsHNwXX6uLR14AcWeqQCg5IyEjz3HGExX1wGma1KzyjWQZ57mq3HZ7fXmxu729/3x7P/QDmE1hqurKee+8++rFjfcOAI7H0+HYRxEHgER1CLvNJub0+fZuGkdVaZrGAMZ+IMdF/kpXbwRqsAKZzByqwAj9sTfQlCTFGQAIIDCHqvKeYVHzQDLwTN57JCoMnKaqEGyepqap37x6iQgpp5xzjOnFzY3m7BnnubnYb6c5fv5y//n2vqqqEKpxnlXUIXaBX7140XU7YlfKSheCZsqQACBLWsH5pewsi2myZAfOsXtyKsVPIBCiqmURFUmiKWVCgDIpgcBEROgIVvKxJs1lNE2WYZCUYoxziiKGaHW9aZqubdqm9j5474nZMRNS8VNJFBEcczG9BWrEMsX5NDEHS3WyVtxPE55WcgpbqylY4UnEFapcLLeUtlIaEfQshVgvzrn0WZ5HVns/5xKm5b6X5sGyZcVpFkV0zvu6IqTDcbh/OJYyuSBH3jkwSbMeHw/eu8uL3cuXN8j88cPnaZrmGHNKoQreezSrm1pEnONQBTUlYjAz1aqubjZXSPTly21/OhV4KIswYt00oQrjOK54w/IxHFHwftO2zrlhGKdhLBVVoTUG7x1RkUuAnCvvqxBCCNM0IlLXdV3XMNE0z/Mc1bRrm65tX1xfVp6dr6qq+nJ35xjJuYvd7uHxUNfh9vYRifb7bZmrRDQi9M6FqsaV/6U5mRkzgREbq503Fa29uaVaV5HM4IAYgGDBtJGQQwhV8EsdgyQqKcWUpTgSzxgdES40ZQQtLKEi4EpgnjARAFBVN9eX+6vLy65r6yoQO8eMS0wH0Gc7txft+iVNVls2/gkAAwAUKjSoWqmTnhzbiggsQX/NX5dLYWZarH11jUimKmvgXrzgs448IJTXWoGwdRh55Vkv4JSBrWssXCpnkWIScd4557qma5tWVUSyrZV1TGmcZxU5nfrdbnt9dQFmX77cl0gXU5pjmubYzrEKgZkckyn64FW0H8cxxg+fPhFQ1zRFZX2aJnbOORdUDWzZBFj6WgAIRs6Ftokip2FIMWph7gEQEhAZQFKriGrvfPBqdup7s4UC1o+jqrZNNc+RiB4PJwO4vkhVVbdtJ6qq0g+9qe27jhGPp1PlfV1X7Fzwvu8Hx1RVNTuva1Ve9GIRUFUk5wWct+X2LJo0ZT0Nu2X5UWksFXkDREQUyVkkppyyppyhCCESlQEFM9UEEyzlFyM4QiYEYkBgIiarA3vfuarebrcXu31T184xlm7S2iQoyh9rzAcRMUJCgpVTVMBzpsXmVBUBDQt+hmuGWlJDKDXeikYt7nRtiJRyTp87VFxnoJ8wPztDK2CmgMtIdHlWXeP+uV+1Ml4BAEprfUlmEdAxe+9TSkmSiuYspfh3TMGxmn78fHc69TfXV6pWVpoSczkxkvPpeBqY6rpGIiaqEZHRAIa+Tyk5dtc3N7s9jcMIZkRUhVCc5ZKp2DLwh0xMbGJjnHLZnarGRIwEjN57JvSePfESIAlNCxk5jcNoiNeXFzFGJAzB13Wlau8/fqmCB4C2qXbbXV3V9/ePP7z/xzcvr8tNuNhtx2l++9WrLHkcxhB8FSrnvAEsx7WEJDWRnKVMXUqpLXipaNAWZSxaoPD1sq9RTCTnQn5bq3oDQueY0HK24hRK+1TAMqJzzKxIXHzUZts1TVe3XdNsmNgARK103WHFtmzx6LYAQFjE5stWkKWTtLxyGRp+FnlXZZ3nRmaotLI/zUpEWEqcxSjPqNDaKQV4srznF2F9agPAkr+u8PCaBz+lHWYG4JquNTMmBgQkjCmZQRUCGJhomTrJEhM+QQ6H0zBMMyM758ocZhWCimYRx7zdds7xMEXnnJg4dteXF3NM4zSaQRUYknG3IQTHTEybrgXV27uHRcC/pDvEQCR54d2pmiOq6trMFDQE31Y1EBWjAUQ2E8RDjNM8qSgifkiprarNtvPeV1U1DFPOWQ3meXbOp5y7pn5xsZWcfv+HH0vNXIUQk1xc7HebTYqxbaqqqgpSAQCqYklK/00k55xNZalGS/FROptES9JmRcjWFJQIkDCnPMd5mKYpJjNzXGyFAMAxxag5WUqiIqBSArT3DsuoFmIIoa6but003ab0uuaUSyuOEN2yFIWR2GwVU16rt5I5rdUPIQCcx0oLE6kc86LcXswUccF6V31aW0eS1gpqrcbObSsoYuWGAMtQgC2WVox1IR8uQrmrehmeSSa2NEp1ZUUBOO89GjjnRBUQnOOyAfviYi8q93f30zQXvFBVJWfvXQh+nqQAwq6cfTAlc94XHlOZD4wpxZybGqsQ1Gn2vq1rAGDm64vtHNPd4+M0joExeD53BQ2AiOq2qdqNSpJBHTtw3rFDQgKoHTNREiEzds57n2IcYoICkZgCuyx5enione+H7fHUI5GpXWy3vGmrur7Y7w6nU07p4eGh6DOOczycTkm0qcLxeCTiU9+XzNI7t6ZlRXRDRUUkiyQwO7c0ccXt1wTLcBm2XMBtUxXN8zyfTofj6TBNk2fk4IJjIlQmBJ3mFMVkTmaKCHUVgiNfusXOhaqu267bdCE0YDjOsZ/GApYRQ+V929SVeS5COrbgU2fnhYigi8hCSaiLuxUtK/LQAAqlgdcGREHH1kFkNLBz1X8GgYvpPqt2llC8+kpb6qTzFuc1A3jiJp6FzVY28xktRURXVVXOuWkrIhrG2cxc8Ju23W46M/XO9f1wOBxTSrTsldK+H4vYFTOHEABBshhYVdXsuOxUMFNGaOo6eBdTTDHWwVd11TYVAHZN049TTDFOU8r58nKPf/zRkhaYm733oUIiSUrO1XUNZR5a1FSkrNIgMoQ4p2kcUxYog9VgubSuRQzAN/T4eHg8nqrgzSzn/On+7qub67sqTCldbDdTnNnxy1c3CDCneH9/qCp/fbkfpth1m03XFEJCqc2hcJeWFa6GAEVMmcjR+U6t0anYZXERJWcpGfYwDqfTqe+HYRiDI4SKqWJiJqq9l7Y2kQkkpVwElACM0AJD8BQCegZSmcdhmPNpGMseFeewCgwMmjkvK0aX3cl/stoLDASFkBQAlzZBUc43QaRnB2ydVILzx1JVe95jx2Wnk1mpulYlvYXdaQBl6mqZiDz3mFafuuSwazBfsodSjZ1fBQzcfr83zd65lAXnhACXu+1u2wGBGXnvmqYuxA4wkJTGcSqfIaakoqYaqsCOmcgxVcGbOTFIMZqoGey23a5rDsNoprvtlomHYTicTjnn3abbb1tVu9xvf/bN19//4YcSetA5ruumaUNd5ZQgSxZhIMeWhQpqLSlLzlb2GqkhAJbtOUspBT6EJMoEdVWpKjGdTifn3f3D4fb+4c2rF3cim7bedt0U0+Vuy4QpppRyU1ebTVdVvGmb7bYtTQFmlsJaAss5F+9C5MyMaFkqe3Yr63V/lk2BmRbZkDnnWPRw1HSOyfFCzchqCOYYPXPZzEmgKiqE6ha37fM85jRFvevjnLIVAqggKpiKqqhktXPBsyy3KD6Scdl/XogjxfEr0ULFQxJEJHRET+gYPstDYdEQXz0gPCeL2ApqrqAbFDirLIFeGSnLA21F9Q20KE8U1kXpmp4helMDAueZfN2WOayri13BgcfpbrvtLi/2L64vd5sup/THD59jTKfj4fGRzCyLDv2gqnUd6roqSbMP3gCHfkgpzTGpaE65rcNXLy598HeHUxVC27bf//h+ntNm23775qZr24fD8eF4MoC6rlJMyNy0zdXl5W6z7cfx8Pg4xZTjrCKjiuZsa5timV5acnIzBTBBRCRkYgQUVXIMBC+vb64v9uW6DOPonDOAcY5EtNu4oR+2XUvsri53wzAZyPXl1W7bimRmMtUsUc0hoKioSokeJcU692bWlsgaAZc7eq6SDABSzvM898OUUkYERkCElHIpTHJKKSXJsuKV5lBBxRQkGZkKaB/jGGXKEAUNzDnHTLVDLmurwUTVs8Kic3jev2oAoFj2WayZcoEtzRCBkUqWSEAGkBHLB3ySOT1bGBaCvRGCyaq2gFhWJC4Z5EqJtDUrhVU3qlyeJROFFYGCFTpdW6ELpI8Iqm4YBvY+54ymTVNf7rfTFE/DAADOuU1bi+r98dTUgRGOJ+y6lhCO/dDUAQm994u5gOWcAVBUc1aE5Soc+/Hv/uN3dRUuL3ei+eFw3Gy64/ET4+by8qLttnMW5/zt7f2CZRC2TXNzdRXqKkp2jst1AijZn5oIwJPw3zIys+xEI1vYT+g8Bx9KzgAAPvhffPPm8dSf+rEKXlVD8Mdj/+7D59Mw1nVV1/XjYSCCUz+1zenm6pLYqaGa5pwQpOjDSE5lWXwxvTJBIWKAsJZTCE9Ts08VLgOC6uF0fHh8HMbRMXpytmx8N1NNKaeSAjJ7RgITQ2L0rmxN0WGYslq2wv6h4F1VcVfXPnjvGNk5dmKoBkxlXLgEFF1bkWVr3mo6q6vLGYAXGMzMSnOfzr0HW7n3sJZ9z5qU8LSgttjb8mC1Yr4L/LomrE8Zgy1lOwAA4wLJLWnnuco3AwB3d/+w6TZ1UyFSFu28u9l27dCkGA/HU6mKzFRFiHC36fp+VFMfqpJalFXe7JiZ4xxjEgRyjpwLRNS0bRHo+nz7cDgNL24uQqhev3q57br7x+P7D59fvLBt12mjl/vtl89f2LHfbC6/enV1c01IKaY4TvMc8zyLyOIP1kjxFDsQC4Nsyf+ICoQxp8jOVRjGOJ+G8f2nL01bf/Xyep5jGQkZpul0OmXRlHIV9NAPY9+/+eplynLs+8v9pffBlk2F8RzwVGXN64u/hjLHLuukh63wYBmjLc5DVIjIsYNSPs6as6uzD56dK9W+sYHzbCulgwAITAxMdU5SVoUTZCZ0zgcOm9pt2hCCRyIpI8tAWVF0aQqnnM/+HlciSzFTVaJis2vtwsxY8Jx1Azetqq5gcEZZl9q+2Fk5rCsmtQaQJX1d2vFrXnvOD0qHae3+rzmnlbwWzhSAAgE4RIwpMaP3PKf8+HiQ3KkaOvd4ON3fPVzstyEEBdtvWu8dO5YsOethmr1j7z0i+uB3m/Z4Gvq+B6BpNmZXVSGlHOdY7KYfhvoYXr9qHdmbV1fb7ebdpy9ZPn395vW267569fLz3cPx1Fd13Xjf1cGHahz6j2hzjDEWreEzjF/QNWLPBTMmIlMpooMqYiKi5h07A0Qkg9NpyFmuL7aaJKtuu3aSVFfe8/7T7f3j8RRCeHl9daxCSokRqhDmOHehcezmec650OaLyt+512K69GOYtJACFHGhjBGWaaSlri1Oixm99545qUrOmZAZUYAJ0TSJgCktjGcgMFQVkZg1GywUIjMkrIkdagBFzRINwLJhQmbnEZ0ZpKzTnOZ5nqa5GMfCGyd2jokZkFxZH+icqhYBVyIEoLN3LB+yMAxU4Tx2b0tBsxhSacevQFKh7q92CWVIfw36pa5XKLgBLV53wT9X77OKmxgAmquaJqV0OA2qaip1FSTrpmu2bcfE4zhOc5xTUrUUZ88MAHNMYsbMxBScA4QQfEqZHG93u5TSME0ppeAdmA7DmESmcWy7ZrvtRHWc5inGTdt2Tf3p9q5pmov9/vrq+uu3r999+ARMofJ1XV1fXV9fXbndxvi3h8+c+n6eJhEpnsuHyjHbKjKMgFkFwUA0RpBShaQ8i6YUTye32W3die/u7pu63m/boW8Ld+bVzeV+t5li3nTdHKdf//LbYZqiSDkEw9B37YaZNceYo4oyoxHTmkURAREvgt/ETA4KOYCWYdqCg5aStyT6wVMVWDSDFehOsJBcEZkwJpslg6krZsrMzlceagQzUEBH4GmZaYpqGrNaVjMgBjJnRIwiNk5pGMd+mPphlJQBrMwXOOaq8iEEdhzKDKRzxKyqwTtjXpC1QouHwpVHXuc0114RnuuedWPzMi1ccP6lolqxzhKun0DUdVxOn1WSxSJlzecXFMzAtU0DdfV4PPVDjwop5XGKt85tu3p/sa+rKqdU9qjePx4QoK6qzaa9vtzfml3utm3XTNNUenebribCH999FBHnXFZrm4adi/MsOeck4zTvdtvjaXCOHKIjAKQ/vnv/9ds3N69fzznNcxzH2aY4joOjFzeXV5f7i6tu8x9++49fPnw8HQ5xHDVnAHDeNXXdVMHMspqKOCYDO51OAgCZVLKplMS/ClUc43GeEZEc3z9WF9tNEmnbpvIupkzMnum2H+I8v371YpynOaamcobSDz2W/cEGZpbFQJSf5jYL/5KXze8/BfQWwOlZyVLgd8ccfCBUhwAAOWsZYxMAYvKrriwxe0fOccWEoFDm1s1ETZGQyJCSARM7Ju88MhsyIuWc5jiP03Tqh1M/pHkq8bocIu+9d857R0zeOb8sPfdVVdVVVVUBQIkcEZ3ppMuuxRW0KslWaUrpOfVZu6ylT7mw6kzLaJqt4MaSpCOWPTq0WnlJBmytLBhpGZqL88zOeeeD95IyMYlpnud5nh+PQ1WFtm32u00IPsXkvNt03W7blvp9t9s0dXU4nD59vq2bum0qM1C1qqqc45ur/Xa7yWIq0m66z1/u7u7uL7bdy5dXMemU0qaprrftw3H4fHt7c/PCeb/dbcVs6vsvnz7uu2az2Ww3u7/61Z81df0fu/b9Dz88PDzmafaEvq6NCiccm8AhhHIMtt2mrrKUPwCqxgDTNI/z7J2rQpVFjqd+GCdAeEn0+e4wTVMIXkT7YVgmNAkeHk9vXr2s64a5SE1JycsK/aWYGjM5ZMJlCKRgpEwOyia4Qr0lOkMzi1cycwx1IAACMFVLOUczptJ5J+fJOy6rugkNzUiTqZYOpQAoEDISOWQXgg+LPiYhUTZOOReGSE4pzlOcxnmeUaVM0TEzMlNx8lwmI533vqpC29Rz3bRNs2lbH0yJjblIQiwZZylo1rrz3EfiYlhqpR6Sxa0uWJusZq1rQ2lpMsGZSrKSa5/lb1r6+4DueDwhESMwsWvctm0FIOUiO5z7YYwxxhjbtmna2jMHR455Ur25vmya6njqvWNAHMYpHE9zTOM0Vd4H5yRLCOGXr199ur33pz6nfOqHz3eP37x99e3bF5++3B5Pp21X13X98PD423/87RCn4MPlfp9i8sTHx8Px4b4Nzabbvn756svD7Xw6BaJxnsFMY0opiTko/e2iCWaGAMxUheCCr9pWpvnxcDyqkBQyvzla7o9jqrwv7uHh8TiM8xRnRjyehqv9lphSlm9ef1XX9TkKFW9ScileSh+ErAbiEFUZCpy/gNNnhKlwiCyLmGRTUYPKMyEpWBbNKWdREVMxIlRzxFx7xwRQyHlZRFKRVzVidr5yyExNFXZt40MooLkYgFjKBAAIKlkkZclJcwaVUrcbrPMoi4myY3bBzXOwMl9F5BwbgA/IRCIFHi3EDlQDybl8PCIs43dSahxYkgKzsjF3CdxSli3COa+FonTG5xbXOq+9QgV2/pGZuXEYAJGYQgie3TDNarrdbrY3V4dTfzgc52lOMSJAVQUkvH14TDm/uLnqNpsY5+1207WN9+724RhT7vshjhMBsOOH07C7SCH4n33z5re/+66qw+HUG8D7j7evXr64udzPMY7j5MjGcfzhhx/atlNCQuy6hgDv7h5MNQT/6tXby93u9c2Lx9vbY99LSnmeQbToOjvvyHFgbupqTpnBCLBq6zevXjh2X77cMeH11X4cp34YPLskeY6JEczg9vauqirn3P39PSI1dX0/nNq6lZy3281ut51S9N4T01LWApDZuTsPBqqCCCLFUz6BjrZ2QWE1U0JidlVV1VVIKaJpYUMyITpCsIJSSFa1TEgelsoJc0oxDSklEQNgdlWNVeVqh23FPriq8kQIarNoEiO0lLNkycXsRVXFRNZeF6KqFE4goiILUc5sat67kKosOcZExLR+TEQsi6zxzPVcNywv2czyuHMHH/AZc54WT7oQmYlWqGrZMbLSROCpcaVLegSA4BAJCb3ziNiPI5h1bVv2Zu26jtn1x2PKGZBijGAuJQnep5Rvbx9Siiml66vLqq62G4kxHQ6HpmtD8DHlNM+3n2+/a8Lbr17udtuPX+6r4DdtfXc4PRxOlxcXVQjTOBZN4X6cRQyZRPXF9bUPPuU8zfHx8WG7u9hu9t++fptzDN7fO344nXKWGqBI3Q7DoBBdCGKmgMwURIfTgIhlXoCYri52++1WVac5TvMcvD8cj48Pj23bXlxcIGJKkdgR8uPxgI6TCjNt2rr2IVA4RyWAIvDLS5GEi6oblcGeEqnUgIsM1TJxVrIrIvTe+xBCCDmnMx0MAIr/K6BjykaQQRTAQAQkx5SnLGU1cPBGjJK9SgbLYNmUDagkuWiGpqZaCJAp5aLOvlKiYSnenpYciKqiQKmAVFVWPcKy+GaF9E0AHRMRmloRn4BlkGON13YWMoFnlNAClC4VUklyFBCXtc+2etelNFrnR8r/GQI6XwcAIGbnHDtnqqZ2PJ6yyPXlbu+dSsZproIHw2GMTPR4ON3ePRpCCN4xl/H2yntE7DaditZ19fHj55zyoR++fHkIobq82P3s69cPjwdTy2a3d/ffvn2bLvafb+9ubx/6YR7nuQqx67o5p007brt267oQguU8DaftZn95+eLPQqib7vu20/cfjg8PMs6FA2pmQ8o6TioCiOT9cRjv7x9LJOo2m6YKpbk0DsOifQI2z7OKxhinadZlcasgsyENw9jVNSGNU9x0mRMqETOvedgCvkJRpyPjJXxhqUNwaTItzFwod0MlpZhzLuMoomKS5yjDNEvKK4Vy8RyaJBGoWhJRkUJOMTMiqEUNyPvkfHRjJHKESOSyLbRTQkOErGpgxX+W5I/PZwhsAXnKxyAkdrSo1xqYZpGU85KflEX2RLAoZ67wvgEuumKll1foJvYUOgrAtLTdbQWsy4UyWqceoDSl7Jwi4DpEurhZV3QZGNkAVHWKMcZYBV9VYZxiQcjalolpTqmsRCrn0nlXhdC1TRbxzg/jBGb77abwmolwf7G72O+qJjgmEdnvti+uL999/BxjOg3j4fD48ubm8Xj68d2n4/GUUp7GqdzPh4fH/eUFImiMh8MRAJq6vX755sXli227vby4cm3zH/727x8fjjnmMjBUSAnMpGpxGBBwJHREoa58XdGEVFVTjKdTryLe+1M/ZjVDnFN+OJ00ZRERyuyYEVTy8XjabDomkiyZkI1XqJnKTA8qANhZN6SwnEiE2T2bqi1egUpDf44xxjnlbAaO2EiL85mTTHNcFIDBGNEzBbduvipebuUKC6AiipmKxBhHYgB0voBfCGXQouz/hGXAlxFhHYsuGQcTQZk/sXUAuTQ7DHJWp1r6SYgIzCbCAGVG2Vby3mqRZ/BywaHWLtOThN1aDOnZPMEgi66ud5F5MgNFKIt412lPUDCHZo6dc26a52mezdQ5Zu+K5pZ3DAApSco5WEEASUQNLHiHAMM4dm3T1LWI9MP48vpCzT58uq2b5s1XN23X7rcbVfvjD++7Tfvt29cXu+3d/cN+24hkNQvOGeAcY5qjqkqS0FQxp83n21AFU5VNF1Os6qYK1f76Zddtfl7XL65uXlxd/3f/w//j/R++xwTMDAbeMRKlnBAcmJFjDj6Kfvly54O/ubz0zAUR1Cwpp8qHnCKYaYxFb2xOsfNt1zTZ1HknotMc67pwTRacE4nMsNC0zYCJHLOacWkdFTdYeoLPaiRcun4mlj0B1U4ylqctNYtzLqdU/B0wMoKpeUeO0AC9Z1VX8FXvKHhGIDFMApANBWsHXApn1Zg1xmSSQZXAmACZzIxLMki0IOQlA8bSrSmsYU0irmi1qoFZFilbfYnQlMonRESwUiyspbgVd1mm5J7mjHnBbhefWnrFZcSq4BoLy3+1Wjgrh69oKyI6FUkpMVFb18xsZpXzxIwIcZ6D7+qqApvMrEwbIlFdhQJ7qVpK+XQaVVQBNl272XT9MKScry92bdPkJF9u76sqPB5OADDNcds1Kc3brg2e+1P/3Q/v+34owI2IxDir5FylL19ut7utqngmZjw+Pn5pPnKo9hfXdagqH/71X/5L7/x/+z/8j+9+/wfpe43zlBOxI+fIsqpYsphFzbxzdVWJSI6JmBxzVmFmIKtDJSLsuPQr0xy9d03beu9e3lxdXV7UdX1GAgUEoAhkoiqQWrFXOM/TFHIGmKrxsgeqXPlSk6BnrrwfmSVnXcIYIiJTGXgBoqfGtJqBISMQsQAqY1F+V0UzMqCshMaVq5grI1ZEE4hJxzmOc0opS15asqVmh0JtWXyoYdlju5CqSwZb2HIqqlky5YJJgRXxRirJI4oKIYKVDW24wqAABaVHW/v4RebgCdXHpRFma5vDZNG6XzT3cGm/L/ZdcAMnYpjyBNA0zcV+t+0aUe37YZrmcZKmCqPpHBMilgaDmbVtc3WxnebYD9OpHwhximmcpk3X3d4/Hk99OWgiYqpTTOM8A2IWAdPLi0tEmOb5y/3h/v7xh/efU8yExMxF2EPA8jSNw7DdbT27VXTTDodHHz5WoWq3F8zUtdt/+eu/AID/qa6///6P0+lkScoOVslJRbSkRwgK4KaoIsH7y8tLFbl/ePDOtU09V5WBtW0bQnCOh2FUkbZtmrqqq1AFf7HdOO/HccwplsLVIXrHziEXljygKqgYOjiDfcVSCQUQEFdhWwAids4D0BxlTnmc0zRHyQX21sLsoDKqU5TTJCdznm1tOoIYAAExsAIahZJKEgGwiKacj/30eBxPw9xPMeeMYExF8Q4IkLFMrBQneE5OIOsSAVQsZaUsRLlw82BJO4rntVULYrEkfd5kX7nJBTXlc/WOiCvzb4kqixkuoiVSvOw54KyggKoRmRNVVGW1nFLOeY6pwNCErKYfPn6JOXdte3190TZVGQgvcR8ARTITEnHsx2EYS8ibpjmL6KRVXe02HTEP09i1NTNPU5znJCIPj6eY8u+/++HUj8TkwK/NCairWk1Pfd8dT69eXjvvUsrTPIUQ7u9v67qu6oZdS4Sbzf6vf/NXu83mP/zHf/jdP313eDyMj8dxnrzjFFNOidi1bRu8F8nDlASwypkAtrtdW9XOk6myc5cXu912o6rzHMdprKtwud/VdV2CoZnVdZOZYpxzkR0yFikkUcgAZtHMiJDYOSQAKIkQEtPTyKNpwT+N1CyJjnOa5jTFmGKOsUhUWfkPIhAgM3lHCZSX51hLKNKsmhRb4OBDP6akAyGA6hjTcZiPx9PpNEzjVAi7YKXiXmaV8Lm+kkFxq4Rn2P1JwlJUEUnJcEXaiWjZu0aA5/K9IJq2Up/WLxErsSAvMX0ZIzyPdzACPBtFWrNYK52Bwo8jBWcqkiEjeO9yln6cJOdxnGOcY0oIcLHf3dxcXu53zFhCUzGlrg59TzFnQk051XXlmad5VlMTTWbznNqXTaiC907Xaax+mO4eT/ePx3mOwzghqCOXzZwr6xyEENqmnXM+HI7bTeO9c8xf7u6nOe62u+3mmGIMdYPEnvBie/EXv2xeXb14/erl3/7933/84X0YpjTNRIOvKt+2dVUVTKPIJ2Uzk4yio1mLVfCha1tmRsBt13318kVKkRDrqmqaJoTKAB8Oh9OpF0Mt60cyMBMjAljKgoDOeUA085JzKoU8Cy2Rs3iKJX4BQBYxUVWNMZsYQREIMUBgxy67Ocac85wTZYiJvRMm0rXqKjFymqkf4zCnOYn3wxKzAVK2aY7jOKYYJSXJmVau+1KLEBLAcgNxUaa1VYhgsdJ1sYfZMtJbyu9zik1Ey1KFIvVTxIYX4tKahK/D44Qrrrmw+59kSdciHp6p45QsFlZzBQNz5Yp67w2ggAumVrpH5NzVfvfi+qJpGu9YAWKcDscTIg5D1zR1COHqwo3DFLyP83zq+3GcmCirEmKc53EYr6+vjgD9MJpZP/Smev9w6PthnucqODB3sd8C4uPhFGNSkZQyEm67dp7j/cOhrjwTeucOpxMRD9MwTUPVdoGpkMSbugkvXrVNc7G7+Mfr33789OX+/jGXESUwIt50zaatifBwOFqZaidqmsYzg+mu28CicMRNs7m+anJOzBR8xc7N80SApjpNU5aMpaYWJRQgAgAkdEzOPfFxEU3VBISxeCtEOC/h0ZTyOKc5ZlXNZoAUPAXvRDUX+V9YFJ0KMxpMMz1Rh8tWxCzqsuacx3Fynpl96T5m0RijpKSaVaR0DhhtzTqXCkYLYPlUfy9qVnregQBwFn9U1TLnU6xcoQgzCSAScVEgxHUhaOEenjU4zkNwKwj7BKWdlZ7X7xReyfkcrkUSgLu8viyJ0zhNwzDUIQBAjDMivXpxc3mxTzn3w5RFihzSPCdEQCIXwnZTT+N4O47jOB8Oh2me26ZGtDLp4hz74K8v9oR4PPWEMEcZp2Pf90M/DMO42bQF5ri+vCjFLBM+HE79qd9uNvv9rh/GOUbnfcoCgHf3D0i46bZtu3HeEy4pOTm3313+5Z+1b796e39/++7jxz+++/H27m7sRxCZ4uyDe7nb7OoqNM3N9Yuuazdth2bj0JeCwBFXzhNRCKFcl3Kx5mk69sfj6RhjArAijZAKW4UBmDwXgkhpHZaq9KnjBypAvCTRZqo6pzzFPMecRXPOIooGa3O79ACXDmT5pXMxa2eUlNCxc75M2RAgRlGNySTnlMq0foGZFhmwFRQq70nO+jTPxjAUlh6DrIVSCfZudatgpgpFuhEWCYYyUmIASISqi0bVmYas69rLc9FjtkxaP0sGFnB0JSmYnhforGQ8h4BznEVVRQkoZjHTum6cc6Y2T1MuG/tirirftU1p5DLR8TikmCTnOaaHh8M0TT4UnVtYHAriy5urly9eTCmZWdNU3ofTMKUsh+MppUxEu91GVA+nHlfN4q9eXn8AIIJffvt6mOLd42PK2RGrChI/PDz+7vvvQte+9Z6ajnEhDDlHznVN017tr169evPN19+8+/DuD9//cP943227b796/ermRdM028120+0WEA+5LKlMaZKcoQD1gDnFnCIimshUzSI6TqPKIqirjOyYEIiQbZ3bRCvoI7GpKS/PvwCB610yBHLes3PFCJ2Dog6dRUubsNgnsitEOzAoQmLl8UXVh5kZSU1zloUwkXLKWUVKP6Cg8aWuIgIuk+eFf7EilbgO9JXcgxa6arHWkgoubQJf6n9bXKCaiepKgNflZCmULGoRTVvU1IqZPXXVAUB0ZZnimsICrMjU8m9Yl9CVpMGBQfB+nGbnedNtQggiEmPMIqdTP05T2zSd95uuqUKY5nma0xyTY/PBplm9D1nUwPYX+7atNMvx1AOY926aYZomZmJ22+2mqYJjPh57WE/vMIzeu8vmYp6jiHablpnmKb68uZzm2TFf7LY5y3EYsggCpDmq2u3nLx93P1xuLr0PhStUrjISMpBv27quL7b7b15//Zs/+02McwhVW7chBGbnuMyEcYlTRR8hS1X0wMqMmk8+zU5FlWVj29ev3w7j+HD/YIsXRLd0OYjo/1fWlzVJciTn+RGRmXX1NTM4uFxSF2W2K5ErM+n/v+uND6KkXRDkClhgMcBMd1dXVR4R4e568IisWglmAHp6uqqzMj38+Nz9+1idn1zQWEpRJAmhdlP8OftWk5mllOdlllI8x0VkKRKCqFoRLSIkAkgamIOKKVyHs91HQy6FAJMlU3XuQdEqOysiCEBemTumBQQI7LI+jfXzNtK669JqGSqqwaskqG2BZlIu0MhgtdejToSMykxaQU5rkRnQ84HW4L01U6tbRvVv1x+whoRWd65wfR1AcPVSJNoOQ9/1S8mlVMYbVWPmzWZ4erjfDN1lmqd5MdVSihO/Oxf9MPTvnx4B0aSM05RTZkZjTql8+vw6zfN2Oxx2m5KLiuRSxnHsukiE0zTNc9KisevLPG/6+Hi3/+Hjs6o93t+fp7mLHSJGotp5Q7hcLudxkpxU9Le//YeHD18hddxGN7w7gszM3HX9brev9wXqsHgtN82s0QICGHpbEsHQQohFjYOYZgQk5qHv7w53aUnTNJqpAZmq6/ENnfkABBiIKl5LYOe+8yfjqjAyT+Pn1+c///zL68txGqciDjeKk6aoqKhTpNZ3qdRHVbLFAA3N54mMauTVai6ga0mCzQIRkZkMkQCL2G1W556p9i49eiOymvhIlROnmgDwWtiLmvtJIoA2fK2qa9LpDrH6TjOvnHTd8GwZurYRE2g+3tYtu+ZzYXXYZgAQ3o7HGFtLvZRh6O/2OxF9O51yTiJlnufz5ZxyPy/L29u55GSAOZOo7Laboe/6vgPT4/H0w48fc8rMpFLmccIQlmVZ5nE/DPvd9vPnl6UUr+aJKWIU1SXnl+Pb0+NDCHwe568/vNvvt88vx83Qh8BLSkSwpDROc4yhjzGGcFnS88vx+z99/3j/sNnuAj8iYOCay9hNWIkc2u1YCwJoGb0nr6xKCFiggLgcHnMIkhEJQQARh34YhgERiVlFVKGYGUhnUkRCjNbc91ooYcXFEb1iMjVVkTKN8/l0fj2epmnKjc/H7dLLlrZ0Ve0T2qav+yesOiG26mFQm568AvBQc8/A6ImWeJG+Tk4BGJDnkYIGBmKgIkgC6rMr4lTsFfRZf6kBqgGajyJoLanEqAHvZiJCiC7S0owNWhy4LjJdO24tC/KkuDXuYW2YAkAIxKaaVZGZCWMgKdkMDrvtdhgATIo8v55jnLxNstvtzOWMnE4CcdP3IllVY9+B2WWap3EG02GzSTl/en65v79/ergvuXx+eSPip8e7nPI0L0hoqpfLqGp3d3sV/dNPv9ztd2h2PB6ZaLfbiGDX96/H03gZN5thM/QuIvrp0+efPv707sOXsd/2Qy/1Pq5OAqhlgQAVAGlnst4XJTJVInMw3b2R184q6uALEYUQu9gZQEoZfBSXsYAtuVAoUSSEQOi7H0De+AarDIZmagqmueRxPL+d3pZpKsuUl1mKz69Wxl2/t2ujUOvwbt1+YGifytvooCsg4+nlLcaNCIM3JwFc9kUdpScI5DcBtXLZeHPK1LCospqoUx2YqIooIXmI57oMCKZeBiliPZFXG8QmdLUS1xi0lYL6ceg6GVoN0LtcazIDdSTP3w3MLABiKYUCMWEfY8kyp0REXYw+R7Lfb0MIOWcA6AIV0Wle7u8O758ei5Tz+XyZplJK3/dfvH/69Mun17eTqCEAMavZ5+cXJuqHoeviZogiQwj8nN92203fdy5DP47jMAwcwsdPL2/n0VSnRZBOSLjZ9M4abgDjOKtoiNHMTufL9z/88PDuabPdc3hX0y5ne/zLzHw1WWtbyv7hGzRt0HSaPaAhAsWAhCKEIhx1v9v1fX86n52d3gyJTYXqHp96YlmI2Ut1NrMmpuYdRSnldD69HY+XyyWnZCpoyqAG5gsRRlghyfbItbJU1b0BbGABIJhhRTRbp7FB78hM3n8l8uFiNXBQBSITNXhStdgKdXp17WqctR9gzbZuzknzSczV7RVRpnUx8KYcXKWMW37piBys43kV62xYl7WEWGsIFFXPygAsUOAhUAjBzJ7fTmCKhkRUYjkcdtutSydE3G4A9Hy6HN/OMYanh7uH+8Pr8fh2PuecD/vdbrv55dPz+TyaqsvuDl3UUn76+dM8z09PTyHGIroZBgBMOe0326fHu58/PTNzP/S7bT+nwkgIQEy7GBDhMs4xhE3fxxhUShEdpznkEiMvRb/7/gdTtSz/4T/+5v7dB7AOIzrFu5/mFQJuRnnNyn1F08lCEFthQYjAaOjxrmZViJvN9v7ubhwv8zyr+Bqw525WREQMUJlNxYRVVEkNyZrFGwAUkbQsaZnzsoCUgCaEhAwVbgQDqNTlqutz9cul1nQxrOhhhSuxwuBQh4YBCTvGyAhIapDFzFtchMHFnRAV0GWDwMWV3DE2tQ7fVwb3kQ0tU1M0VFUAAjK6mZzzH0b0ee3q9lazXuv0m8B17ba3L2ogcNIGz3Bw3ZtFCA93ewMspcw5iyoDdF1AgBh5uxnAUEUCD10Xp2lKuQxD/8X7p4e7nWp5PZ5EZDP0h21/nufTecwixIQIMYau786XUaWMl5E5/NXXX0oX53kexzEQbYZutxmeHh/MIAa+vz/Qeco5dTEQQgguRCmXad5vN/vdxic6xmlx+DCGYGAfP/5cyj9eLpff/Pbv3335NeAWQsWr1nB/a6YtiGij3FAEcHYh8fBmLh0qJTuvTlHVwGEY+hACE5takYLkQhlkZiLKEQFBzLhKZNwsjps6WsQhumvSym5cGVyl+W6wSuB5kz+1SgKguk/w1LM2LeufwQDBZ1EdlzVA5y8lAmaMTIHaOryjN17yAAIYke8V14gjaiKmCiLAbP4ZDE1rBQ9QkInwirRjQ46qoa833k1wRVrMfbOfqNZeu75LS1bd8B3+N8MwjbMSDsOwDSGGAACBedN3+8N+6LpSJCX5/PLmyf7Qd/f3u/vDYUnpx58+vb6dnAT55e3y/PnldDqbKiFRF+72u76L4zQ59QO0RH5Z0ul8GS/TF+/fpyJdjH/7619N8xwDH3bDPMNhv0ulmFpOi+MvXeDArKrD0HdddxnnlAsiRGJDez0ef//NN1nKfwZ7/9WvzIYQAt2YI1xDjyG6Fps3bNVVNNR1F1TNtOSU05KXRaXUghiBOOx3hxi7k15KKWoaiVrCVyFPJ2VgDkxc/UR1f6amIcT7+4d3797P8zL3s5QCKqWISMEiRQTMyCmTbgGgmpkhNoYSJlwVDQzASbrALBBFxsAElZRBTTUiMFEgjETtfQEN1FX8GgZaTcopSwkQzKVOTU3VPKCDl2a8+mtPmB1sa7lTm51b+5a+Ue8zTOhQEa6Aa2WE8bH566vWFNU5DsDCOI4xRuw6ZBpih2ix67549wiI85xqbtvY8D03mZd0vlwu49jHsN304zT9/Mvn09sJEZ32qO/idrsRKYgQ2CcmZZrmXBVXMRV5PZ7u7ndE2HcRwAihlDIvqe/C4/1BDVKKx9NlSfP5TbvY5ZReU9ptht2mI4J5XpJACCwmb6fTH7/7bhiGjsPjF19Rpb+sYkt2Q0ForRIyAxHnGpOa+bkWr6mWoipLWtCLAyJk2u52T48Pb29vS81ZDcCHmdANXW/ZCsxUFEyaIg0S0n63+/qrL4ksLYsvNqioqlzGaZ6XeZrnaSpLKqWQKoAFdBQIGI3b4g9C1Qx22fM6gk4YAwYmYgSArCZi7JA/YSAicn5iED+S9VKB1ll2aPpETeLDTyY28NIfELtCcsud6gWtZnU9sjUFsAa/39ZSUAv2inf5oamG5ZVuS3v9r0M/9EM/IMA0zqJyt993MV7GKfbdYb81Uzf81+NbSnkz9OM4//zLc8o5MMUuppxTyjHwbrdlDt5q2++2fcfjuLgmZwisIs/PL0S0rmPPKQ1Lt+kjaNkOXZFSSpmmaZ5HAnh4uNvvhqGP07y8HM9pOauUlHJKadP3wzAMfX8Zp2UuMUYA+/zp8//I/6Sl/P3v/uvDh6+67trMWH1nM9BKoVNDjohlM8JVAAAU0klEQVQrvZqolmyiOaWcZibKOad5jl1HHIZ++PDui7e3t1LyspiqNXHJCg/lIlEkWhsG97holVWGiGKM2+3m4W6/LAEQCTEwRWZ3iCmVcZrf3k7H49vlfJnGUdLi9IDNkRpUGseKwjuvCSEGxuCDz0h+4hyAgNY+1iZG6KCStQwB0Nj/Cq6dr9XkzEzBGKyIsoFVeWFTA18guX5OJCKHe625hVoJ3DyCa24KrU/QlBoqCtaK2mazAIgY+mEAAG+YbTeb7WaQUhBx33XELAJD3xsYB96Fbd91x7fTOI5mlgmnKQGCSjFVV84khN7FN8w3zDGECICvxzMAPNwdfEp/M3RoWkqeTYjg3eNDypazT/rJp+eXnJM3n97d7fuue3l5ndKSMuVctBSVMgzDZujnJaWUWIRDPL29/f6bf0bmf/gv/+3h3ZfO/nzrO/0fUZ/WUfBSybF7KVKSqJaS1URUfTt5LsnA+h4Dh81mc3d3fzydVM2joWuEurn7ZISocuU0WNP8+gUhdTEG5sLeyHTRTQ4hEtHhQO8M8ldf5JzGcXp9Pb28vr6+Hi+ns6YEph5IoaV+DUqjwBhDna9KIimrgSFSdAYyX1ABUDVXAyX0IU5AF+9qdYqDkQAgarG6TwfjIXBYz7rBKg2vWNeDV/j1mne2fytE2tgc1vdABGCsaBo2LMKal8WbWxfSkjwJ22yGu/1OVHNKRLTMy+UyLkvaboa7w/7x4T4wPb+8juOcixRnU0FERBFx3h816yIPm6GLQUWYYxchMJqJFEHmYehcE5sQRcsyT8qMaGPf+XOMgU3AVMZ5joGYsKiGEB7u73IpqjbPKiKXaU6lDH3fdbH49EaBjPD6+vLNt/887Ha/4e7w8BhjXA+iNXTYP3xt/YmoFKtTOwUAfcyAiUwEwPq+rxQHgfuu326Gvot5SX43vdwpIszk3SMRp0EXqgU2rc7AzFJK85JySj73TQhMAbFGW0Qahj7QQR7t668kp/R8PP7y8y8f//zx9eW1LKkOVACuOyRdpMiIxISYRKesZtYxxUAxUDWOlte1C0aDRo8P5hu+qwFds1t16/GMYuXmNwKDtm/kS/FNCWSt3GGt8lcss/72ViN52ipqDbwGaA/Jv1rzB0MI5/M5MHd9L0VO5wsTiRlSEpVlXsQgMOciqioERHQ4bJeUx3H26biUMjP1fURERo0h7LfDdtOfx7mIhIAxhi4ENT0cDofd5sePn0rOgYmQci59F/vA87Lst/22C6UPS/bDDSKiJRFGUNh0PAz9ZZz6LoyLOgBNYF2kwj0nzKWAaWQoy/Ttt38Ydrvf7H7Xdb3vErWehqeL2cSk5JyWZbzMy1RymecplZxTulwu0+U8TvNlHKd5STlpKSpKIbhevKn1fTS1yBzIwRfMRTopMXSe4NYUoiVStgY0oixlmhcmCrEO+/SddV0k32NzbXeiXb/B3e5wODzd3x/22z/+8fvnnz+ntIDnnWgGFKOTgRERLWJzElGL7AeMAyIhFKgmogBeqRCBKDQYx9bGWgvX0K7XmhHXf1SNqxRj7QtVf+m912piCP/PxmBNWf3+rzm6451XIKBG+YoDXAslAAsi4tG5SOm6oKq+u+XEYPu+N7Pj8Q0A+j5uh76PztqDSwrjZSKSYdOHGEwNI3dDT8zTtKRlJoTtbui76NFov9vMaVly9hnJ2MWHIW6HIVdRoqHrY99Hn7rBSgBquWRE6EL3dLc9n08FYb87FHGiIjrs+t12k4ucx7kU3Qw9MlrKnz5+/D/dN3eHu1Jknqd5msbxMo7j+Xyaxul0OU/jNLrk2zy7Y1MEyaWUDIjEDKrMrAbe+0OEEHjouoe7u6en+4fDduBoToPp6aC2Hnq742YOULuMhnEI93f3l8tpmWeVUkrxOXaHG82gw87bWqoCGfqu32w2zCEXOZ/H6TJKya7KbIDR1eh8lUp0TiWLej3KgYNXRoBkVgB1PSV1J7giTf57nV1RTU2JfdXMTGpC1BDZFnDV1h+t5FRt2xOc56FBntU8bd3ywCv1/FpI2fV7a+SH1ZXXEN910flPOAbRK6Y2dN3Qx1JkmiZiGvqeENSsFCHEh4c7MJgOyzLPuRRABAZVNrW386Qli8pmGDbD0HV+t5AQUi5oQkxD3x12w9BHM8tZN32362PH0IG8XfA8zmYKpqbFFKdc5iUzcd/15/PRpyoQaMrLp9cLM5nBksu8ZFH10XH77/8YkBgxO5qj9TarFxzXxAjqkSfutwN2Xcm5C7zZDGYQQ4yBY+RI9OMvz7mUKRc5HqeUXt82D4fdX33x2K0TVQamUkphLoGDEjuICC3CMtF2s/vw/ot5mY8vn6WU5Gv12lAEr2gYzCDlUkQGHQKH/XZz2O+7vgNE393pAm0iIyIB5iznJKUIB45MHVMM3LEPF4MaYsVcnSyy1s8ebREAQdFFPAy5Si9ALZEcuIHKxlvb5dV0gAwIkW59J9xaFzTErNke4K1vrT93BQpuav+bFwJAIGIzW1LCXJjR5w4J0VTO55xTJqZh6D3zm5dEYBy7/XbbdbEUSWl2AF9FUhFULVIA8eFuv9/vNpshBEKRcZ6nSS7jlHMhMJMyT/M4TiaSckGiTy9v05J1WS5LHuekRZFIAXIWABAz8WxJ69SPj3m120yE6BpnDdhBW7sUBsgUQyBmcNhTxZ+CAAQwJCZmy0UAus1GU0pzssBzLuxxvGgMrKWAWVablgWILks6nS9/92//+uHurqiIlFzIAEMouWRiFqnL5j5LogYU+O5w//WXX0vJ57ejw3ArDsUUQlBELKLTNOWS+3642x9UrY+hC9HxUCbqmAlRDHKROUkRDUx9oL4LQ+AQAiCKT0itjsjvk88p+7QIgrpUMQAYtLUNU1UPCBU2p1r0GxgBrrdX0XvuWOEqwLrCVZVr3LPC+vv/vy9uc8/qpKtZ37hY8CIJwYpqYy7H0MUuRwQTNSbaxk3OZV4SghHRdjv8+ssP7x7vmclUcu7PfZ9ymqZFL+NlSSllM8spv7xeFKGksizzkkopJeUiClbhx3ppdX6mHqu6boitaXbTCmqwBDa7A+v7jpjHcfJDSE5I4LEMcf/01B/2m74//vSxlLx/fNrttv12+Paf/rfj8GQmZqRqpkKBCEEEkUrO5CR3mgWgeNXffq2ITcvCRJ9Tmv7Xv/zm7/72w9N9kQKZkIJHM5FCRCLA7PHNPwrGrru/fyqSfzSYLqfWLii5MC5zylkNRORymVLJm83Gw/C8LFnFAALjEJkIFUBFp1SKKBP2kTdd6COHwNAcJCGwuZKx943AsClyuHX6NEkbdyqiQK71B0T1IBOCoSkh15ebI/dOzF5dI1FFNFuqXQe5V3d6Dd14PRPXx3vbC71iyf6tUHLy5Al9Y8EZ9IoQIjEjwGW8iKgUQcS+6/a77dvr6XQ8zSm5bsU4ZxE5T9M8LUWtqFopVmn1aibTnH514h5dvPVgNXmutSZWYgH1DAZUCakWmW5+oqYCxIioCiLZcy4T9SBpRZylo8zz01cfdveHy/PLdLmcxwsEvFxO5lNkhABYZUGKACr5vTUwU9IqKaBtrZ2Q0JCYDbGIFjUEyHn8n3/419/9p39/t9+JFNMiJQsHQhLMAIgoCGrkhIcIiH3fPz68E7XnzyG5uA8REqVi83yelrTMaVkWIALDE5/Ssnz85WWeZkJwvnA1SKJLKqkoEXaB+8Aup+SR3Wyl7qqYe+X2rF4R18UkA59RMbvRj3Oks9qKGxgSVO7/6u+rHdY/tI3N6voaRXRD8HHNAW4zgBs3uXaQbssydzSuduqqJb6SYmVJvl4j1mZbmkTZZZqej2+ilfy/fmTC2lpoJQJWVj4lbAwpWHtdgCDSph0QsfaOgYmM6s42EiNCSdlXPZG5nTIjZslFTJ3iAwhFPBkidRUT36pBBIBlGv/0zbcqlVQ7ny6fz2cRicQYgvkcbt0R9t1bD7VcJ46d1c33exAICAjNVPyB+Z0xe7mMv//2T7/7zb+LwdKyeoyq6mCmzExgQAZWPQ1z2O/uum5AxMDRZ2tSyqfz6eX19Zfl87RkMNAiry+v8zgdT6cyzQGRyTvizvUiRLSJFAPHQFWLEcBXLg3RAVkD1FaA0NWJAiBBZchBbfi8udxUq+9xHQiogZtW4hv/ayc7gSY5fq1+/H0rHEa1adRstMX0vzDF1XGu7+/1QvCRQZeq0gYZqpo6S4CPmdSBNChiAE2/o5Ua0LABu7p09AYxUWMd8DlwIkV0CW0MoePgFYK7F4C6mswtlDiJMIhKyVKkpkn105m07BsQjcldgVNZYtVGA1N1/9RSPehCrFmA4xwtDtXM1dPbGLz1ylqRca2llvPUARMBB4rR1VOeX0+vx9PT450VQsoGUKT0XSche6EVYgQItfAQUZG+73fbfQwxhFCLNpGnp3fv34/v3j1/9/33f/rhxx8/firLwqBsQqaAoICmNmXJRZgxBu4jd13sYqSG5DhCpAZSR+zB5wZ8mUIrY0MNWW4lAUGrAyEidI7bhiBDS5QNXSCs3kqqVdJNZolVMQLEqq/0b1q7t6s1ri7zLxwnrq+oxGMAEFyVR1Swvq7CqvWtoYqGae00GFRx3CpgX/1oiyX+cg4hdhGRvOpUNS3ZT23d0DbQItOSKmq8TgqQ3xcFAAKqd0FN/fIIpW2ZWz0O6IeePG1vC0cYWK4pbFuaRIC6OuZsa2ae+SMGZKe1r8hKQmLWXFwecm1DOQRpAAURuZBqiB0xTSn//Hz86sOTc9XHGGIIoiZpURWLilh5AXzXMYSw6frA0Yc369YDABKGwLvt5vH+/unp6Ztvv/3lp5/yNEYCBhKDopZLyUUNoAu06UIXA4fg5Q9jZRfzR1H5u7V6eh+sqv6krntAnbOzSqy/XgggNk4cvHGlNTez9Ut/4Zo1ErVhwesNtxpTbQ33t2DcreOE1pLFduGIEJD4WgFUC7WVuKyaf8Vi1y0T0yY9iIDrgun6i31QB8FW2vaVMkpbc78+b0Q0tDVNrhJEgDXBUKzTWw51oI9F+bVRQxCb0/ZLVIXKKOd4HiG6rwVfloPKCOzllBYLTP5snTLJ3bKJBGJPNOv8uidrqGbq6bMVEcwAQcwuWTwKeoOVQzA1p05ELGZKRBy6GHo/vYEjVa5Or63FzCU+LIbw8PBwOBy+/uKLf/3jv3z3x2/H1xfJUkRTKbmYmXWRQwgUmCudrgJQMdCGuzeXYT4jpw3P/EvPtaZ+BFUIgpl8lg4JyYeu1dwH11vdkKgaM2m1O2gWA1Wy+9Ykrj1QuO4bXd0qrtkqXOslNQMIfoIcFHA/4Y8EW+BTbaOyiCvBxNoZa1F9nT8ARwOaL/KOAbpktyGtVJrNE3qeBC1/bR+hJkHg5Fa3rn1NGLQ11xBx3QX09EDluhpYf+paollTP6nfN1W1AoimRlWWDwCsqCATuC6Lj3+hVdm06h8MDdCMmNOcUirUU8qFQzAtuRTnIo0x9v1mt9v3/cYNlOoGRXsHADMrLu7BrhGDzPzh/Ye7w/7Du8dv/vD7H/70Yz6PToDXRRpiiDF2zA5OO/zu8TG0FhGs8xCoq/34HSGAutJqoADMFJidv5KZXfpbnaaHzNePSK3uK5iBmrfSDUypqsyoGQFBW5harRXbI7uxSAODdZMJrwG/rot5IxqJQDVwDKZa1XEBWy9r3SgADq3DBUbEiOsZrWRU9QJW4/c82we/Ddb2N7QawVq6oLCOfpEUBwV9VNHM5Si6vjvsp5cXLeLKj+t5rElxjVGK2OgrbnqMyEziCJGfKN9Qg7prVtMEBEABIzNP9LniDz6dWdc13ZzNFOsX6MvhaErABJCWdJnmGNgAci6qMs9zEYkh9F3Xd7ELMbDLDVQc11o9a35hKgZiVlFFzx77fvvll7+6jOPHz8fl9SSqMfIQOUYOTB7Trfmu5ubqLVh5ofysqvojaoPrHsF8x44CkEsmVTkGf8wG5s7K6tq7Qht2r12g5tFAa/sDKmiDayMNWm/dh0wBoA0A6C26szr9a0dU1cBC4GDkV9+yzBXz9yQFVwqK1ZFdE4LqJAFahmotCBgYmppJqb/MBZkBEZCYQgzU93le+s32/a//5vP3352eP/nkm98/DhyGAZlXuoHmQ/2Wo62p/urHGxeeX7yWQmui0k5KayZXdwrY+hmInjzUil4rhQG1zNMqveD67mCEqqiqhJRULuOy7bsQ9ZxzkYKEfey7GBFRREQzC4OBS87VigRJwVRFck55RgAIaIDcVkSRHDp93O52hhRi6ALFyEhciRPBRF1WGOlKKltp3hXWR2k1ufSPpVp/DpGIkchtE4m9dFQzMedchIpauas29CE8n1wz5JrOX2MgoK9IA1R2xTYnbpW/pBXWWjmhK2TQ7iy0+RJ/r9CYxFo9ezPLX61hHUg0xXYckdZThboiKus0YU05tCqPQisIyTcfgTnsHx9ou3n788+B+fD4eH55eXv5bG1AC8yAKA5DWhYTqQOD1bdd8d0GSHjMaFnP6rO19javiVe7EbWkb6mtf2y8Ge+u1r/CBgBonrbadZxOGwsWqojlnEpOl1HUlJm2my1VsFZLkZQyGDEbMleCFwaj6ziJFEHweX8hysxBW7L3eH//b/7mr89vx+V4JIDaO1YpAMVA1Vu8GJmtldV+gNvYKtQWsHtvtSKVS9DpJzgEInZtEiBSF7kz8HWkla+UjMzZmKAen+Y/cS1IoO3lwU0BtP5XVrTJy6D66JzpxLHv+hRW6wsVWGnfamlus83V/66bMrjaH4A76ht7lurq6oPFa+FVk0VCNB8QE6MlA6CJ5mXG1kZyX4vMHAIA2rI45o/12lrN6G/rw93tTNj16AEAEhN4U98BPzPTumpOgOZdaPVHxL5eiTXpqEmVrSxbjfN//WDr/xTBEFR0zmVKScWQMDCbWS4lFAakEDRnAcgBkFwD07nNlCpMiBS7IS2jSVYzJEk5OTMCMyPS4/3h3ePdz9MIIoC1/WMAYi7dC4YE4Pp0gK5600YLPexXIohKKA5iQOs4FDMyM3GFPG+Ki3rP1JiaMmqlGSOX4VLzHN3M72ZlnKwv9lu5GkmN0c3D113VenbcsNHaipQ/7v8LuhRpugWd6nYAAAAASUVORK5CYII=)" + ], + "metadata": { + "id": "gW4cE8bhXS-d" + } + }, + { + "cell_type": "code", + "source": [ + "image_data = (periodic_impulse | beam.Map(lambda x: \"Cat-with-beanie.jpg\")\n", + " | \"ReadImage\" >> beam.Map(lambda image_name: preprocess_image(\n", + " image_name=image_name, image_dir='https://storage.googleapis.com/apache-beam-samples/image_captioning/')))" + ], + "metadata": { + "id": "dGg11TpV_aV6" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "3. Pass the images to the RunInference `PTransform`. RunInference takes `model_handler` and `model_metadata_pcoll` as input parameters.\n", + " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input is used to update the `model_uri` in the `model_handler` without needing to stop the Apache Beam pipeline\n", + " * Use `WatchFilePattern` as side input to watch a `file_pattern` matching `.keras` files. In this case, the `file_pattern` is `'gs://BUCKET_NAME/dataflow/*keras'`.\n", + "\n" + ], + "metadata": { + "id": "eB0-ewd-BCKE" + } + }, + { + "cell_type": "code", + "source": [ + " # The side input used to watch for the .keras file and update the model_uri of the TFModelHandlerTensor.\n", + "file_pattern = dataflow_gcs_location + '/*.keras'\n", + "side_input_pcoll = (\n", + " pipeline\n", + " | \"WatchFilePattern\" >> WatchFilePattern(file_pattern=file_pattern,\n", + " interval=side_input_fire_interval,\n", + " stop_timestamp=end_timestamp))\n", + "inferences = (\n", + " image_data\n", + " | \"ApplyWindowing\" >> beam.WindowInto(beam.window.FixedWindows(10))\n", + " | \"RunInference\" >> RunInference(model_handler=model_handler,\n", + " model_metadata_pcoll=side_input_pcoll))" + ], + "metadata": { + "id": "_AjvvexJ_hUq" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "4. Post-process the `PredictionResult` object.\n", + "When the inference is complete, RunInference outputs a `PredictionResult` object that contains the fields `example`, `inference`, and `model_id`. The `model_id` field identifies the model used to run the inference. The `PostProcessor` returns the predicted label and the model ID used to run the inference on the predicted label." + ], + "metadata": { + "id": "lTA4wRWNDVis" + } + }, + { + "cell_type": "code", + "source": [ + "post_processor = (\n", + " inferences\n", + " | \"PostProcessResults\" >> beam.ParDo(PostProcessor())\n", + " | \"LogResults\" >> beam.Map(logging.info))" + ], + "metadata": { + "id": "9TB76fo-_vZJ" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "### Watch for the model update\n", + "\n", + "After the pipeline starts processing data and when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in `.keras` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier.\n" + ], + "metadata": { + "id": "wYp-mBHHjOjA" + } + }, + { + "cell_type": "code", + "source": [ + "model = tf.keras.applications.resnet.ResNet152()\n", + "model.save('resnet152_weights_tf_dim_ordering_tf_kernels.keras')\n", + "# Replace the `BUCKET_NAME` with the actual bucket name.\n", + "!gsutil cp resnet152_weights_tf_dim_ordering_tf_kernels.keras gs:///resnet152_weights_tf_dim_ordering_tf_kernels.keras" + ], + "metadata": { + "id": "FpUfNBSWH9Xy" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "## Run the pipeline\n", + "\n", + "Use the following code to run the pipeline." + ], + "metadata": { + "id": "_ty03jDnKdKR" + } + }, + { + "cell_type": "code", + "source": [ + "# Run the pipeline.\n", + "result = pipeline.run().wait_until_finish()" + ], + "metadata": { + "id": "wd0VJLeLEWBU" + }, + "execution_count": null, + "outputs": [] + } + ] +} \ No newline at end of file From 6f4e2852311db381a622dec3ebf26654c9372806 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 4 Oct 2023 17:52:46 -0400 Subject: [PATCH 23/32] Fix spotless on master (#28831) --- .../sdk/io/gcp/bigquery/BigQueryIOMetadata.java | 4 ++-- .../sdk/io/gcp/bigquery/BigQueryServicesImpl.java | 13 ++++++++----- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java index 0b5e063c0b5b..1893418dedb3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java @@ -32,8 +32,8 @@ final class BigQueryIOMetadata { private @Nullable String beamWorkerId; - private BigQueryIOMetadata(@Nullable String beamJobId, @Nullable String beamJobName, - @Nullable String beamWorkerId) { + private BigQueryIOMetadata( + @Nullable String beamJobId, @Nullable String beamJobName, @Nullable String beamWorkerId) { this.beamJobId = beamJobId; this.beamJobName = beamJobName; this.beamWorkerId = beamWorkerId; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 04a665ac9947..1b6cc555511d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java @@ -1364,11 +1364,14 @@ public StreamAppendClient getStreamAppendClient( .setChannelsPerCpu(2) .build(); - String traceId = String.format("Dataflow:%s:%s:%s", - bqIOMetadata.getBeamJobName() == null ? options.getJobName() - : bqIOMetadata.getBeamJobName(), - bqIOMetadata.getBeamJobId() == null ? "" : bqIOMetadata.getBeamJobId(), - bqIOMetadata.getBeamWorkerId() == null ? "" : bqIOMetadata.getBeamWorkerId()); + String traceId = + String.format( + "Dataflow:%s:%s:%s", + bqIOMetadata.getBeamJobName() == null + ? options.getJobName() + : bqIOMetadata.getBeamJobName(), + bqIOMetadata.getBeamJobId() == null ? "" : bqIOMetadata.getBeamJobId(), + bqIOMetadata.getBeamWorkerId() == null ? "" : bqIOMetadata.getBeamWorkerId()); StreamWriter streamWriter = StreamWriter.newBuilder(streamName, newWriteClient) From 161cd6b1c8de9a2a68b6aaf81cc5e13f827b6f50 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 4 Oct 2023 20:18:54 -0400 Subject: [PATCH 24/32] Gradle 8 support (#28756) * Gradle 8 support * Make gradle help --scan green * Replicate defunct com.palantir.docker and docker-run plugins * Fixes * bump errorprone plugin * Fix copySdkHarnessLauncher dependency * Link to original plugin src --- buildSrc/build.gradle.kts | 17 +- .../beam/gradle/BeamDockerPlugin.groovy | 325 ++++++++++++++++++ .../beam/gradle/BeamDockerRunPlugin.groovy | 143 ++++++++ .../beam/gradle/BeamModulePlugin.groovy | 29 +- .../beam/gradle/VendorJavaPlugin.groovy | 2 +- gradle/wrapper/gradle-wrapper.jar | Bin 61624 -> 63721 bytes gradle/wrapper/gradle-wrapper.properties | 3 +- gradlew | 19 +- playground/kafka-emulator/build.gradle | 4 +- sdks/java/container/common.gradle | 2 + .../io/google-cloud-platform/build.gradle | 4 +- 11 files changed, 511 insertions(+), 37 deletions(-) create mode 100644 buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy create mode 100644 buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index 0ca748e3eb04..968829caeb8b 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -44,20 +44,19 @@ dependencies { implementation("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") implementation("com.github.spotbugs.snom:spotbugs-gradle-plugin:5.0.14") - runtimeOnly("com.google.protobuf:protobuf-gradle-plugin:0.8.13") // Enable proto code generation - runtimeOnly("com.github.davidmc24.gradle-avro-plugin:gradle-avro-plugin:0.16.0") // Enable Avro code generation - runtimeOnly("com.diffplug.spotless:spotless-plugin-gradle:5.6.1") // Enable a code formatting plugin - runtimeOnly("com.palantir.gradle.docker:gradle-docker:0.34.0") // Enable building Docker containers - runtimeOnly("gradle.plugin.com.dorongold.plugins:task-tree:1.5") // Adds a 'taskTree' task to print task dependency tree - runtimeOnly("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") // Enable shading Java dependencies + runtimeOnly("com.google.protobuf:protobuf-gradle-plugin:0.8.13") // Enable proto code generation + runtimeOnly("com.github.davidmc24.gradle-avro-plugin:gradle-avro-plugin:0.16.0") // Enable Avro code generation + runtimeOnly("com.diffplug.spotless:spotless-plugin-gradle:5.6.1") // Enable a code formatting plugin + runtimeOnly("gradle.plugin.com.dorongold.plugins:task-tree:1.5") // Adds a 'taskTree' task to print task dependency tree + runtimeOnly("gradle.plugin.com.github.johnrengelman:shadow:7.1.1") // Enable shading Java dependencies runtimeOnly("net.linguica.gradle:maven-settings-plugin:0.5") runtimeOnly("gradle.plugin.io.pry.gradle.offline_dependencies:gradle-offline-dependencies-plugin:0.5.0") // Enable creating an offline repository - runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:1.2.1") // Enable errorprone Java static analysis + runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:3.1.0") // Enable errorprone Java static analysis runtimeOnly("org.ajoberstar.grgit:grgit-gradle:4.1.1") // Enable website git publish to asf-site branch - runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks + runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis runtimeOnly("gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11") // Enable dep vulnerability analysis - runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin + runtimeOnly("org.checkerframework:checkerframework-gradle-plugin:0.6.33") // Enable enhanced static checking plugin } // Because buildSrc is built and tested automatically _before_ gradle diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy new file mode 100644 index 000000000000..442b35439cae --- /dev/null +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerPlugin.groovy @@ -0,0 +1,325 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.gradle + +import java.util.regex.Pattern +import org.gradle.api.GradleException +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.Task +import org.gradle.api.file.CopySpec +import org.gradle.api.logging.LogLevel +import org.gradle.api.logging.Logger +import org.gradle.api.logging.Logging +import org.gradle.api.tasks.Copy +import org.gradle.api.tasks.Delete +import org.gradle.api.tasks.Exec + +/** + * A gradle plug-in interacting with docker. Originally replicated from + * com.palantir.docker plugin. + */ +class BeamDockerPlugin implements Plugin { + private static final Logger logger = Logging.getLogger(BeamDockerPlugin.class) + private static final Pattern LABEL_KEY_PATTERN = Pattern.compile('^[a-z0-9.-]*$') + + static class DockerExtension { + Project project + + private static final String DEFAULT_DOCKERFILE_PATH = 'Dockerfile' + String name = null + File dockerfile = null + String dockerComposeTemplate = 'docker-compose.yml.template' + String dockerComposeFile = 'docker-compose.yml' + Set dependencies = [] as Set + Set tags = [] as Set + Map namedTags = [:] + Map labels = [:] + Map buildArgs = [:] + boolean pull = false + boolean noCache = false + String network = null + boolean buildx = false + Set platform = [] as Set + boolean load = false + boolean push = false + String builder = null + + File resolvedDockerfile = null + File resolvedDockerComposeTemplate = null + File resolvedDockerComposeFile = null + + // The CopySpec defining the Docker Build Context files + final CopySpec copySpec + + DockerExtension(Project project) { + this.project = project + this.copySpec = project.copySpec() + } + + void resolvePathsAndValidate() { + if (dockerfile != null) { + resolvedDockerfile = dockerfile + } else { + resolvedDockerfile = project.file(DEFAULT_DOCKERFILE_PATH) + } + resolvedDockerComposeFile = project.file(dockerComposeFile) + resolvedDockerComposeTemplate = project.file(dockerComposeTemplate) + } + + void dependsOn(Task... args) { + this.dependencies = args as Set + } + + Set getDependencies() { + return dependencies + } + + void files(Object... files) { + copySpec.from(files) + } + + void tags(String... args) { + this.tags = args as Set + } + + Set getTags() { + return this.tags + project.getVersion().toString() + } + + Set getPlatform() { + return platform + } + + void platform(String... args) { + this.platform = args as Set + } + } + + @Override + void apply(Project project) { + DockerExtension ext = project.extensions.create('docker', DockerExtension, project) + + Delete clean = project.tasks.create('dockerClean', Delete, { + group = 'Docker' + description = 'Cleans Docker build directory.' + }) + + Copy prepare = project.tasks.create('dockerPrepare', Copy, { + group = 'Docker' + description = 'Prepares Docker build directory.' + dependsOn clean + }) + + Exec exec = project.tasks.create('docker', Exec, { + group = 'Docker' + description = 'Builds Docker image.' + dependsOn prepare + }) + + Task tag = project.tasks.create('dockerTag', { + group = 'Docker' + description = 'Applies all tags to the Docker image.' + dependsOn exec + }) + + Task pushAllTags = project.tasks.create('dockerTagsPush', { + group = 'Docker' + description = 'Pushes all tagged Docker images to configured Docker Hub.' + }) + + project.tasks.create('dockerPush', { + group = 'Docker' + description = 'Pushes named Docker image to configured Docker Hub.' + dependsOn pushAllTags + }) + + project.afterEvaluate { + ext.resolvePathsAndValidate() + String dockerDir = "${project.buildDir}/docker" + clean.delete dockerDir + + prepare.with { + with ext.copySpec + from(ext.resolvedDockerfile) { + rename { fileName -> + fileName.replace(ext.resolvedDockerfile.getName(), 'Dockerfile') + } + } + into dockerDir + } + + exec.with { + workingDir dockerDir + commandLine buildCommandLine(ext) + dependsOn ext.getDependencies() + logging.captureStandardOutput LogLevel.INFO + logging.captureStandardError LogLevel.ERROR + } + + Map tags = ext.namedTags.collectEntries { taskName, tagName -> + [ + generateTagTaskName(taskName), + [ + tagName: tagName, + tagTask: { + -> tagName } + ] + ] + } + + if (!ext.tags.isEmpty()) { + ext.tags.each { unresolvedTagName -> + String taskName = generateTagTaskName(unresolvedTagName) + + if (tags.containsKey(taskName)) { + throw new IllegalArgumentException("Task name '${taskName}' is existed.") + } + + tags[taskName] = [ + tagName: unresolvedTagName, + tagTask: { + -> computeName(ext.name, unresolvedTagName) } + ] + } + } + + tags.each { taskName, tagConfig -> + Exec tagSubTask = project.tasks.create('dockerTag' + taskName, Exec, { + group = 'Docker' + description = "Tags Docker image with tag '${tagConfig.tagName}'" + workingDir dockerDir + commandLine 'docker', 'tag', "${-> ext.name}", "${-> tagConfig.tagTask()}" + dependsOn exec + }) + tag.dependsOn tagSubTask + + Exec pushSubTask = project.tasks.create('dockerPush' + taskName, Exec, { + group = 'Docker' + description = "Pushes the Docker image with tag '${tagConfig.tagName}' to configured Docker Hub" + workingDir dockerDir + commandLine 'docker', 'push', "${-> tagConfig.tagTask()}" + dependsOn tagSubTask + }) + pushAllTags.dependsOn pushSubTask + } + } + } + + private List buildCommandLine(DockerExtension ext) { + List buildCommandLine = ['docker'] + if (ext.buildx) { + buildCommandLine.addAll(['buildx', 'build']) + if (!ext.platform.isEmpty()) { + buildCommandLine.addAll('--platform', String.join(',', ext.platform)) + } + if (ext.load) { + buildCommandLine.add '--load' + } + if (ext.push) { + buildCommandLine.add '--push' + if (ext.load) { + throw new Exception("cannot combine 'push' and 'load' options") + } + } + if (ext.builder != null) { + buildCommandLine.addAll('--builder', ext.builder) + } + } else { + buildCommandLine.add 'build' + } + if (ext.noCache) { + buildCommandLine.add '--no-cache' + } + if (ext.getNetwork() != null) { + buildCommandLine.addAll('--network', ext.network) + } + if (!ext.buildArgs.isEmpty()) { + for (Map.Entry buildArg : ext.buildArgs.entrySet()) { + buildCommandLine.addAll('--build-arg', "${buildArg.getKey()}=${buildArg.getValue()}" as String) + } + } + if (!ext.labels.isEmpty()) { + for (Map.Entry label : ext.labels.entrySet()) { + if (!label.getKey().matches(LABEL_KEY_PATTERN)) { + throw new GradleException(String.format("Docker label '%s' contains illegal characters. " + + "Label keys must only contain lowercase alphanumberic, `.`, or `-` characters (must match %s).", + label.getKey(), LABEL_KEY_PATTERN.pattern())) + } + buildCommandLine.addAll('--label', "${label.getKey()}=${label.getValue()}" as String) + } + } + if (ext.pull) { + buildCommandLine.add '--pull' + } + buildCommandLine.addAll(['-t', "${-> ext.name}", '.']) + logger.debug("${buildCommandLine}" as String) + return buildCommandLine + } + + private static String computeName(String name, String tag) { + int firstAt = tag.indexOf("@") + + String tagValue + if (firstAt > 0) { + tagValue = tag.substring(firstAt + 1, tag.length()) + } else { + tagValue = tag + } + + if (tagValue.contains(':') || tagValue.contains('/')) { + // tag with ':' or '/' -> force use the tag value + return tagValue + } else { + // tag without ':' and '/' -> replace the tag part of original name + int lastColon = name.lastIndexOf(':') + int lastSlash = name.lastIndexOf('/') + + int endIndex; + + // image_name -> this should remain + // host:port/image_name -> this should remain. + // host:port/image_name:v1 -> v1 should be replaced + if (lastColon > lastSlash) endIndex = lastColon + else endIndex = name.length() + + return name.substring(0, endIndex) + ":" + tagValue + } + } + + private static String generateTagTaskName(String name) { + String tagTaskName = name + int firstAt = name.indexOf("@") + + if (firstAt > 0) { + // Get substring of task name + tagTaskName = name.substring(0, firstAt) + } else if (firstAt == 0) { + // Task name must not be empty + throw new GradleException("Task name of docker tag '${name}' must not be empty.") + } else if (name.contains(':') || name.contains('/')) { + // Tags which with repo or name must have a task name + throw new GradleException("Docker tag '${name}' must have a task name.") + } + + StringBuffer sb = new StringBuffer(tagTaskName) + // Uppercase the first letter of task name + sb.replace(0, 1, tagTaskName.substring(0, 1).toUpperCase()); + return sb.toString() + } +} diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy new file mode 100644 index 000000000000..5297c7018139 --- /dev/null +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamDockerRunPlugin.groovy @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.gradle + +import org.gradle.api.Plugin +import org.gradle.api.Project +import org.gradle.api.tasks.Exec + +/** + * A gradle plug-in handling 'docker run' command. Originally replicated from + * com.palantir.docker-run plugin. + */ +class BeamDockerRunPlugin implements Plugin { + + /** A class defining the configurations of dockerRun task. */ + static class DockerRunExtension { + String name + String image + Set ports = [] as Set + Map env = [:] + List arguments = [] + Map volumes = [:] + boolean daemonize = true + boolean clean = false + + public String getName() { + return name + } + + public void setName(String name) { + this.name = name + } + } + + @Override + void apply(Project project) { + DockerRunExtension ext = project.extensions.create('dockerRun', DockerRunExtension) + + Exec dockerRunStatus = project.tasks.create('dockerRunStatus', Exec, { + group = 'Docker Run' + description = 'Checks the run status of the container' + }) + + Exec dockerRun = project.tasks.create('dockerRun', Exec, { + group = 'Docker Run' + description = 'Runs the specified container with port mappings' + }) + + Exec dockerStop = project.tasks.create('dockerStop', Exec, { + group = 'Docker Run' + description = 'Stops the named container if it is running' + ignoreExitValue = true + }) + + Exec dockerRemoveContainer = project.tasks.create('dockerRemoveContainer', Exec, { + group = 'Docker Run' + description = 'Removes the persistent container associated with the Docker Run tasks' + ignoreExitValue = true + }) + + project.afterEvaluate { + /** Inspect status of docker. */ + dockerRunStatus.with { + standardOutput = new ByteArrayOutputStream() + commandLine 'docker', 'inspect', '--format={{.State.Running}}', ext.name + doLast { + if (standardOutput.toString().trim() != 'true') { + println "Docker container '${ext.name}' is STOPPED." + return 1 + } else { + println "Docker container '${ext.name}' is RUNNING." + } + } + } + + /** + * Run a docker container. See {@link DockerRunExtension} for supported + * arguments. + * + * Replication of dockerRun task of com.palantir.docker-run plugin. + */ + dockerRun.with { + List args = new ArrayList() + args.addAll(['docker', 'run']) + + if (ext.daemonize) { + args.add('-d') + } + if (ext.clean) { + args.add('--rm') + } else { + finalizedBy dockerRunStatus + } + for (String port : ext.ports) { + args.add('-p') + args.add(port) + } + for (Map.Entry volume : ext.volumes.entrySet()) { + File localFile = project.file(volume.key) + + if (!localFile.exists()) { + logger.error("ERROR: Local folder ${localFile} doesn't exist. Mounted volume will not be visible to container") + throw new IllegalStateException("Local folder ${localFile} doesn't exist.") + } + args.add('-v') + args.add("${localFile.absolutePath}:${volume.value}") + } + args.addAll(ext.env.collect{ k, v -> ['-e', "${k}=${v}"]}.flatten()) + args.add('--name') + args.add(ext.name) + if (!ext.arguments.isEmpty()) { + args.addAll(ext.arguments) + } + args.add(ext.image) + + commandLine args + } + + dockerStop.with { + commandLine 'docker', 'stop', ext.name + } + + dockerRemoveContainer.with { + commandLine 'docker', 'rm', ext.name + } + } + } +} diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c31482d577e0..c7a62237086e 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -23,7 +23,6 @@ import static java.util.UUID.randomUUID import com.github.jengelman.gradle.plugins.shadow.tasks.ShadowJar import groovy.json.JsonOutput import groovy.json.JsonSlurper -import java.net.ServerSocket import java.util.logging.Logger import org.gradle.api.attributes.Category import org.gradle.api.GradleException @@ -1252,7 +1251,7 @@ class BeamModulePlugin implements Plugin { if (configuration.shadowClosure) { // Ensure that tests are packaged and part of the artifact set. project.task('packageTests', type: Jar) { - classifier = 'tests-unshaded' + archiveClassifier = 'tests-unshaded' from project.sourceSets.test.output } project.artifacts.archives project.packageTests @@ -1560,13 +1559,13 @@ class BeamModulePlugin implements Plugin { } } - // Always configure the shadowJar classifier and merge service files. + // Always configure the shadowJar archiveClassifier and merge service files. if (configuration.shadowClosure) { // Only set the classifer on the unshaded classes if we are shading. - project.jar { classifier = "unshaded" } + project.jar { archiveClassifier = "unshaded" } project.shadowJar({ - classifier = null + archiveClassifier = null mergeServiceFiles() zip64 true into("META-INF/") { @@ -1575,11 +1574,11 @@ class BeamModulePlugin implements Plugin { } } << configuration.shadowClosure) - // Always configure the shadowTestJar classifier and merge service files. + // Always configure the shadowTestJar archiveClassifier and merge service files. project.task('shadowTestJar', type: ShadowJar, { group = "Shadow" description = "Create a combined JAR of project and test dependencies" - classifier = "tests" + archiveClassifier = "tests" from project.sourceSets.test.output configurations = [ project.configurations.testRuntimeMigration @@ -1639,7 +1638,7 @@ class BeamModulePlugin implements Plugin { project.tasks.register("testJar", Jar) { group = "Jar" description = "Create a JAR of test classes" - classifier = "tests" + archiveClassifier = "tests" from project.sourceSets.test.output zip64 true exclude "META-INF/INDEX.LIST" @@ -1794,18 +1793,18 @@ class BeamModulePlugin implements Plugin { project.task('sourcesJar', type: Jar) { from project.sourceSets.main.allSource - classifier = 'sources' + archiveClassifier = 'sources' } project.artifacts.archives project.sourcesJar project.task('testSourcesJar', type: Jar) { from project.sourceSets.test.allSource - classifier = 'test-sources' + archiveClassifier = 'test-sources' } project.artifacts.archives project.testSourcesJar project.task('javadocJar', type: Jar, dependsOn: project.javadoc) { - classifier = 'javadoc' + archiveClassifier = 'javadoc' from project.javadoc.destinationDir } project.artifacts.archives project.javadocJar @@ -1915,8 +1914,8 @@ class BeamModulePlugin implements Plugin { def dependencyNode = dependenciesNode.appendNode('dependency') def appendClassifier = { dep -> dep.artifacts.each { art -> - if (art.hasProperty('classifier')) { - dependencyNode.appendNode('classifier', art.classifier) + if (art.hasProperty('archiveClassifier')) { + dependencyNode.appendNode('archiveClassifier', art.archiveClassifier) } } } @@ -2210,7 +2209,7 @@ class BeamModulePlugin implements Plugin { /** ***********************************************************************************************/ project.ext.applyDockerNature = { - project.apply plugin: "com.palantir.docker" + project.apply plugin: BeamDockerPlugin project.docker { noCache true } project.tasks.create(name: "copyLicenses", type: Copy) { from "${project.rootProject.projectDir}/LICENSE" @@ -2222,7 +2221,7 @@ class BeamModulePlugin implements Plugin { } project.ext.applyDockerRunNature = { - project.apply plugin: "com.palantir.docker-run" + project.apply plugin: BeamDockerRunPlugin } /** ***********************************************************************************************/ diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy index 061ccf27cce2..97d96e6cf1eb 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/VendorJavaPlugin.groovy @@ -126,7 +126,7 @@ artifactId=${project.name} } config.exclusions.each { exclude it } - classifier = null + archiveClassifier = null mergeServiceFiles() zip64 true exclude "META-INF/INDEX.LIST" diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar index afba109285af78dbd2a1d187e33ac4f87c76e392..7f93135c49b765f8051ef9d0a6055ff8e46073d8 100644 GIT binary patch delta 41204 zcmZ5{b95%((seSiZQHhO+qRP@<}68$AGx`lh~)fFsaj;MiX97^(}w;JdeIg_EK=f{qUxfgB1;VggKi&SA9v8P52JERa$mc|$lmVvfx{qi69Qt` z*fZ)^M2W{1fVoSt`s`K{cHR_Ov(<|-{;q0kYpPXc^iz@|EJ@}9Rkl^$`6gZKo4=jl zD2P(#uTcD*y0DWmpF`Dtw^s9E&tgueEt9jYdg)|4^CH zb}H5tkozJO-=TYiQxbjRT%6PCU;JFRQPXAQ>rlRozg33`4{|4|Q{K zP#?n!$RS}A3u8%zoT~YzxuAqLNKTa(NWlJ4XU{EHZHY-(GB_2uuL{a8BMNcj)?=dUUT2HO%1h(mqG)ntl zN?SX{UvgL}$DJsYuN~%ASKh2fJrBPH#2??t43t8?^fOWdaMj%wk$O`DK4(tRuA(&E zog=Ry-f5N`!=ByQeZ>yqokNEb4OV)~d*KM!+n@>L3iD=%hcWl5GV|Tcwvb**xo{vG!%lw${AnQ~eWceyLLtO0ikN#30gs-w0?6D+m(Pg;;(saJJH6dgz zPtR9$S)TMwL6Y4dX7dYUtY^k@&mLj>shqlfVB>uT4%k z-sD&k5#S$1G!f+SeqV-O07FX!@mC%6H?4gT42hV?{rCiRc9Cr9B1@ZjfX@!wme?JN zAJ(4)af-zesN2Gr=Jn#7mg9j8%5Jvi=KRdf+^w(o&rhoFI@|08W-G$DW;^7um(;k@ zrb`3p^aRVime{Nq^@gWKx`2>bX7zjX*(w=Bcc4S{A@7F|ytuV3;DP(RjMa4RmukeWjWwVyaGM*D6m`mn7ZGF34w6Gb!;w3^St z3XgDy{pdd{y~uiAiiTGa2wO@_XU;qFfTIXAZ1RMapg5FqfM@t-DJO(?zaynola?z< zq^^3=9HQZI#n>+*T*@Eef3h6)^xyrwTYa3S(|cxi6h6LV6~ufKNVoA3J4aC#kWj^9 zU$rtnC%FQ(!JWlPz7l4OHcH%})DUBe?Ui1bJ3TXHGHytpNOUMTkK>O63oL1f0R~Vl z5Q&~zYZ~evszs-g;%QS1E$G6>(o=zr5zmFhgtr72k-dOTT1h;>q0$c5&3}By18rDv za{zTEQI@fiS&|kEha>S}so7LsIpivt5vVHE)F!Z@B(20{Xj&vc)i}Ts+cmgXxl^-r zfwK2C3bRXuS7T6w6&q}%IY$i(=8BOF%1u8n14+J?DQ}GLQC#%nt^E3w&nfRL0C{8o z1kK9$eh2k$?D=o(;&vtePX11A3JhkVfk_W3;pVPI(@owrFYG{UOG!MTtY45i(<@=S zN=P|BG2-(N7nJ4OA>%O~hs;#8d_UEH{2?oTcEBiMv{=Wp@WM@3;N z0{Wcat;=K&itA=*;~J(LXVs)n9~I^r%a4*|S$W0d?v?Q`{73J$KUQ_4q5`Sm3BFa? zV(eIUY#=VrrlfmrNrXeQej79wf^Or9m^X&%-g`DZ;Yi7Q4g&5ll^1FyD)W@D)KrVh zl|juxjkCEccS-qdV|;zxQr12&?HgSGVY)!2YK($@QY2deG%8D@JL$e1pta(|J1t#f zUVXCDHNM8*jvht|y!My(A~`L3>E<-DLux~cq>XNop^vVrBZxb}{nb5fA()O1T|^S(dc&uFNXX z1sOhL*eHNaulR)m@PE4EooRR|dslppGSL#gzaw^n>yc)xGtbqT1u>8fY9^T3tzg#i zEiiQ#AQn{r#eEUXY8>4+Mrqk_>>(;GWqA*KMTOm5@A`nZy=B$H-C?( zlF23*1jz}fkjVi@l)zFo10^&Obb)3fg9^P$h^iuJQ;Z*^a6Q;+qAQe?3Q<`pw}KAg zydA$doAnNj?u2d+;V1>M^FI~FysSLcf+g$@#ZKq8d0w`C6|MR|Uw=akFnT;VD^Hq9 zGI0`KoFoRj=k7WyyDED&OeSn42gbbM%*0-x1h2w>1ew$^fC2Ap-F{$PBzXEyrTj37 z|B%v`@t;f5{H-YoPp3c_sfip(oYcsVma9E!ya2Bhu7Ag^4(~_@9b)^=9|exlV;ye0 zkAQyjF?9L1MD~jY>AmX`ua2~kR*f1DUXh7s5&IJF3N2tvARh{hNrID-Z54AhWLP_F zFm8P-mQDjHL1nr5DLdug=(vCHfzCO_8J zEp}ADT4%<15ZL|G=AWR}ikGH-x5cS%WlZt>I`kA<%!uoz)|GAv_cah!+Y7h`@it zu8EEaceVwi{ki&o3?rRBTObvKrW@02mSpQ zD^OwQTHZFUnC}|{%QT@iPD87>s#7FYY?`8-wKN+V%kaw0n!Qwe`tejf7W4-uxMOEn z1(Vn|#vOg*_XsP}L^5eoi@55}u8>pdAD)HjFyCfJ{bvuXw4_?hqm*=Twu)n|8EDp} z+eYBWOMCAP%?4JsQxu}o<$vt%1F1M%3xE{do_y6?#u;H-jLua7VHFrsRou*Me7~Cn z1C`5D3zl~V=mjbZ4Z8f}+vsFcsxmOvll-$BZaszJJA)xbGmlU*&%|6w=(r4LQP#B8qZL4?#bw;rFwrUZe%Cl!17wp@m_D+-Nz5%B% zt?XT&zh679<@}P7y-eRZC5R&rPv$LF|5=J}QFnrP@w`@`o?^n3b3RXV&PMS=L{r+! za7Y9Oi2VBUB6J$%+V116)8xd1RLkFT*ANt*svo|*ApHGBmxixy&&1idy8M>im6v+s zJJG|fRAA@k#dU*&gHSrR)2GZ}tv0Z_0F(hXtyrUQ8WJo|a{ ziMWa`=&FqTs``tea&48@-F9e+V6{bmb@ln8%ChSG*rBYNCc8(^^J?+vcr*X*TlY&* zP`b0Uu~>NJh1l@*S4;+P70*S3Lo7CoK;d@+>MMofl3$_1TyK>=!U7BZw?Q zhMTsbi~7e-*pV{h+ncBqVqK6o#QCoWmO$-zcY?U8;xN7nN;fJ;JVN(eDNSd5D)gSH zWK>0(S2{&H?qD%k!{JNVuQ3tZ8$}9(;3+1vRTU|By?jbx*xdmg2%wem?uec@=pCKR zAWz(=cMrn>0cfkX0ZqqumOvoKy0-yri$e3lL`0*=|UF=Xo;nNB$R++7AqF1I$2 zZvCA#{pqrH;X7TditMr4QzKe8tYz#nZ6q5vj(Ua#bB__qb=;@iZ@p%Qg^;6D$z44A zh(2vy6$HmZxDhoJsL3VEmK6%>Dlcc!l(mcUV1(oOxE z`#HlgHd3=fPzjTlsAlQngunQjxoK8k2*4DNg|*W#mnx?YRFxRP7EwrMqU2oRtpMtL zxPS_E(=%Mu@}{MByzHWw_iUI8VR?cgJ+c;FlPE8-W=Lmm)AL@i#FS3`Nlhz; z;VGl5;Zpo)d6Bv+f3=idnJQ{bP%GC!Xo6b_hE|$O4RtQ1jx#G)d`K!=l&<^(h*HRs zB7rxC9LE_a62%!yf|{!|a{rJQm%q?~j7l6=D332v${pu@2oAIzsn#Y}EYi#wau{h| zT8}|=`?eb;fUz6p&v2*t%b~K}6e!jLKTlnsUkmQxi4+N7N)oF?5 zTBO*$S5&^zn+N6;KzZ&f-aD|>ic#ydB&->9(A|2Jp9(0;J(sv_dtpCp-h&4Q4`;Z; ztbWGaBh8)gv1G?HF&5~?h?M$s*1Sf28_pImEJRH-6bfwR%*v!e)#l5NRLD3#{KS_? z6i(c44`X6_Mr|W0@*mtFl(dVhZY50MO0@|5h4iNR^AspN6*hcRG`6RM{DAR0fzaxx zV^i-Je6DP8BQrssMCdS^h?JD)5qQ7|;!N=(g7m^(s~Gl)4Og&J6psP1 zS#-)hqqC_6Ngq1M9Xll{ONfA$!bY2bGt!YanL@r}o`q{yizFpUBV|F>HmDP_qul~O z^GDy4;p&fuVEdH)0HvGx#b1NSl4Sbqi(j-^TH$Q3^GH~D=O2WcMmU9To-E(KfTN=~ zV)vf^#83N&^X~F*ARv#BARzqz#m{|KLg1d#vNHO3zbD?#8U_w+git1$g-`;5xUe$X zC=rVy`=D`<64u&G)^*ypN5$PJmeHfpod_yHB+?h?m&%yy7VEe+t9F{(>2}@&kI>fF z+s7r>cOCjmY2tdaNYSK{BGq!Y5@&;41-eS~ae4e^_^jqDow$f2$l+U8qkzFuJKz*Y zyf0<&oj9~w-Nv@NLb^%a;e>7_rS~eQoL-xTuFHixV-<#6M&RBE$pouJrvuD%_jE(T zB~*fuRYWbf#*~#bP95h(cA4KKmjf@4j!Rod?_6o=<1O>g-bU)3M60w*-6Blln@L^m zRnr^#E+18zbKXAh@xiqtiJ8OP~2gfQxKj?}%vZ1~_f?(Z9C zL1*8t>OW|tYIseB)Gr47Zak-ZJu`wqm*TA1ys4|yeHr@+$MJ_R`zUv}tj6}DCO(?F zW|-yiM?*!7TyX)@e&&@7I0EKuwOJ;{&^GR!Hp1gDU=P@A4c{fePF)5E7=SV3jU&u^ zd#Eh(1m*qk&ErnQVr`y$*p%_iIR%R_(>q4QJNeIT4pq!2+l}oKFM5j=*+XU%j{wQA zT|*h4`o-juGLfXPh%Dq>_~hq|7U<7%SbL;I(L1W5?h?rfd~-sIc}U0&M7L+e@-usO z=5d>(eL2|aGfzk}5;vLEq`Jc_ zqs5Y&%;Ky-dQqvn2U=F zOw&jV-Ln)N*Q3%LSYyx$ROZ(q>N#foT1`&!0S8&Ft3nwRVG%D&0Jdd;1%i%e5t7B} zGzV`_Y(mT6gTebw*5Ia#73$3pH~W?>3^o3vR7h{gD8*E+DShpw=iDgybRfx-`BpQ& zuSGbVQSGry{Na0-L$+z_pusx%wy1Gh=C1r>-6SPYk7W{dS1`&b5t zEzQ7aO27)*RV5w96QYtLuJ>RA3jB_2a`#38E9Y*J{li9W$7iPf6V`^JoTBPy7DcT3 z`Z-Ny1zBK8+;Yb9V!Rr!BhXZPv?oqaTe-2q3}zbMH88WL=T^Sa5@wyQl{O&5q#<3e zYR58zpYSxGw<6G|oQB8x`KQu`T395ovNoz4-fYk{FVkk;Wd(rLm4k@ucY#~KO4xG; z&h#KT{d?LxwY#!wrShOhm&1UIjZ2~3>k<1!Rq@vvu(N->cGf5T2XL}eGRQ?XOE8_+ zUs?=17A7jj4id&@jsb&G#$gk46NK}NPIPR9aVY)DON&6w*03?B`6Z$+U_HmM@5@aIrq<(LqW+FVb*+6#Ws3C^mh`AE%ol zMtB;`f@JdQ{w+;T5l%f~wn!IcE#@Jt@qJRJNR0}6D}rr%nzpt{ff&dZ%>-ScVmm?g zzNyc&qQ&v*w1&`RS0(ia__dIwDCMU*6yhvkTIg?oSeRuZuxppe0mTN^i z*2P*uOiV8$ww9$R>NwH7toID>_M3muuJbZpxtb-f{!1V>F9X4xEUkhW$}*1c{PpI| z@9XyJy+2=1Tp)5CR7YSKXdMNI9oS=y^hZTe@S5r=jkHH(5O%cae)MKQgHaSv)drF< zIa3`KXPpT;*1D<*m}dOC=?d8T{E1g}Nrf?lutJUL#vqQc7Dsy)X^eIYR{UC>IGvm8 z>x^B7KC@&>E!XVr-NQVJwMJFgYdt|Rk7hLOIyDzhzU`yHOBQR5ahBewS^quqkrsi~mA1?%c`k2!r9qq@hyJktY{Wzu2NN8NPL*fCZqund&7Vh;v9i$|t(GgzQQ{bCshm$73o|_+C17X%@WIp!y1!r7O zBnLC~&VgBG`mY498zb+QaY>+(#d|TbvLv z4K`g8D{8%IE&1}RO^fq3Qv*mHSg-B1`+^e7A5Jwc;e>))Za=;WBN6q&4F`CR$Mdk% zX~mY<36Ag)EZeZ4C;L5Dn!YC#qIUebb&ywbGmyT`KrW5~LiqH@2b(1Ip`ofC-Y;Ab z=%z}uCS`64lmN?QU~iVV$y0ve>=t)of;xX&CeJL8H=jWVq>=0tcM_IRP1rXJz<+*w zqe*%9%eR%F{Tb$~3>>MX;%@4NV$hF&ywyaKa@E20;eJLdG4)rCN?c}PRecm;OJ0JN zn2e+@9+jyChBuCiFagZwS*E^(13zVyMvYzrd!g-kiB1UC$t|sJ1GG=c(5=&k zsQbbZgZ(g!P17t0l&&CA(jB0 zr!^m>UO@LhJ~xZ3O#f#x_i3j0mxF?U6#q?V?tdh6-U&)F96vJ{x<6L4AiozV*8dWx z&;@fH|AM482oMmq|8o-t@JbO)1zEuJqw%UKMHO6J1tq#gZd!Fj2|0#?QW6{uJ@^mf zRT;gbXZFtMS@A3QSExAgf0!OuCoJ$E;MPl>Y_3*wIhmbT`Tasbu#C}Wb{`XK@-lXF zWmHR^P#Xj;ld6oxf+BKpiHJ?~>&Oir3?y}a^9Yyz!34eCpTPXibLV!Gy3oEW&Yw6* zhIB0g=_wFft-}&wlcHAo<>ei7$n`$%(S=T%9<`~tKg&+~kFw(^XuUbEYh3p*J&t3- z^*ja&>ZgJnuvv{d>rS)?2*ELTvZnL)nDGSb`Oj#3ZO1-|ga_totxMKL2pn5a)Q6Q?=n`#eE)1NynuJ&48uPJwQ zj({)t^IEKt*H+@=0Bpa8VTQ0l#%zXw5~_JUhUr?g9Y;`tx(X}Y^9^rtW=Wxb_)j+s zT&mfmmKIC*)=fN9rpVY>?%A=g>8Xr@>F5)m5O4JoPd63sWXG0?}Iqw62ipHvB`4ka~b zzdcfVWZjRxZD!CCh;H&F*G;$;69D*;F^(36)rGec!BBJw(IPC=I-(~LEmzKUET?EJ zk5F|*M5ah3j|0$kpZxVhCHV@KIG>RInF(?AFZXH5{lc=q|Fn16y038c@n?A1Yna6l~&_zku`JHXY z^SIu0ma~1FY<fY(69) zfHMlon>SWBSih%>gZ>une91BIl%-(wi!W`{**>{JYG@~YHV;Oq;rI4a>J+>#SyaWj zeV+%lV#qmkyUo}Hz_U!bl+94RmVDC_l(9Ws8W^%a@E|tSm@f2fs_dfBy2pn!j&Iq} zrl;A{nq%l+?yC*xfxd`(v~rVgml9#y{RKGhmmbR)CUO@IcT}9U3m!B1hYoJ@eKMtw zUF-LFYi4?JY3H`>#7H>I39Jn<7zpv$Z9agSh=HoEz!OLh>wN7UF4HC`QI8x}<}i4m z*t_R0yxkw~z&iBkQB&+*MTW5R-9O+4sN;a_1CdAjvoeJz-e3tIPhM=uv(&16RqZH9 z8rDhHrpQjXThC)>_{FRg8Nu)DZ<6o>2r)?(s(Et<^UBmn_E__U6;1_}PKC7D5B-wW zw$JGA89rg)PFC|JzEL4RzCfNJx`m5?(r4z`QPe6&Fz@!?%b6#j5>Lc34s4F!m}ULI zr~T+jTTr>Kkdt;5wxgsSk2t+^CK$}{Ju56RdA2E(D3IKEZR{KCnYDW^&K+>AirD6G z4=|c4I}fR>A0GLnj6RsMN=#vGaFpDmYLZU~L^hv-EKLU}{(xVbWIbbS$KUeM^GE!G zmz;=G>m!1IPv)XII2X(+`%XI65BAT1^_pga15d6OlLdZf-T7@g!HJExx`Pu#iK-w! zVHvbJdTm%>VlcIJ;$ZbC3- zy4aR1*cMaCVWq*sWLz{i!TVf>$wxwZ6>l8884ccG$aX#X`e3xP!GwxJ^>9<_rZJ9Xa&|wh$)!wYRWh>RnHzY+lWWY638>&B zcWF! zqi2Ug7G}JzyXDW=u$EsPx>$DTh%qLF2lk+USORzX-uEz)0@L+e!6dLIB_trKWvCC4hfn#Wo~-40JnH9 z@;iegoEv`sfLR?KHn2;>hnQ!^T-;X&_NGd}TZRH_;U6sq7wA9^Fr^UnOtW`CL67kG zAc6J)mTpN?6OH*wAog0I$*Bd{XsH{NkPPZTM}6AUX_Avol7}^f2yfEJ6)I?FW2#cO&=1Nu!!TQ9s=`_n zs13DPWMs0+P0O1sfr_TRUHd1|Z^CkZ7JA-vzvQ^i3ru^?`}-ViJV1fSr+<1|50_uu z0W*Dm*GKx`v)*c}ee95H* z00=Ib8;p+iUh?o*!^riKDoLjpYQ3k`;ic4&nP(}3hB!r%VO8VI9IE@N2P+biUZo2O ziIkK_$XaPCVKJO4A!3Gijj72iN9Lm}KuLiyTF|4E?YoNuEg#!Hwfnm!d1WLP^P))E zO!EnIG`16Uu`+B`bJI5T?205*3HH?lRkQ~i2pTq(T1(o}?F3k~=+pru^B}FVMOr{s zR_1l;@+z&BmqnN@odS=VWz{adSk--_zNN249Eq!jXP&NmpB5(dYISduaxW7r(7Z8n zHHAS=d?UH}45GDp70^Ut<$HqP+KP*oKG9j_nWvRT3$~i2N;NI7!+7d@@{Lw3UAmWs z?pU`;bO`81!Y*CmWm7_$2IY5Qj zay`V%gBvpjimBNb4JM3t``oV@2vN)g-yTYqJv?`OFU)Q!b$Jxn;V^+)=d}y-a}74k z0>_Ywx&~p)RR2e2ZCu+K&2?!PkE2xXVg?Booa9+zbI_u8PoEjs(+_lLE|me%ZXVQD z-&njRL5uGfvw;=4jn@=GzheSHu;km*j9;5h=xozgYwN4)44MMGpQ142fQ%JruW63( z`j{1G5l~_nD&3>Is*U`}<(dbZ>gFyA{!c@ta%)uF=|&C5HMl;d0|zh=Lkt*kZm3AQ zBMzfr2d`7Y6~oU{kb?;k&6}aQG6Z2IwVHU=RCKu$hiLJLH=5Yfch)Kh?C_S}aJc{F+2Bm(sz z%4SR1@Rsjo=IHC@v#ITvcu?9)fAWk#%*>;|mG9D@&6sS8ER7n(qJ}X}$!<%IkivGi zfnQ+v>#uqC#94pGUMMsPt|wzcfGqSe*lbjQ@~2(3FT(b#GlJ(mLf%g9mrFfi0R z3gRQQDce@ITv&7Rcv!87&rJ47#OyI}wpD%)X%~bt^|{5Xx3{x-{5THbI zZZ4Ypa2LyVK3DA27$o51WR-0%8Oxt%Z&`YxA~ULiOsye~1B);GBbPBPi;l> zTri?_ILsaI=r>D91k^@!CQX*$=ajRiI>lOGwzd@l8G<5vuQE82sGfc-z)VsVcq|UUJ9d&4ioxIk^p4Wlou|i3+8re>zwMK2?38r zTqHX$rqplA7l`mqZdqQ1Iv*DI)AA%|(H~Q{!fFLDTZK8nU*iitN28LKneQ)U3;3S> zK1P!}CC|37VLfh|Y$=~jnW4WT1dIN_KDWtQ8}3?m$nQ^_<4+q=5}_j-;)%GK6TDRu z?@4^Si>(7{xJ6jndq$Ej zk9{N_fednv33xE^K86a_I(|@|jm9!DskL-WCfntpT207$sIihoY1<_!gsWT(L4}m5 zEJOY~^zGV!>TS!(t10BYw5zmj5>H(~`nL7tPRDxE&4fVOcD4A^vtGV-+i~~3z?!m- zvFhm9JlpRsQroS^>irf_O<@5pB#R#e7mdqFz>njvV@k9Q9LMHt)WGo?{DRxv7XmrL z1SqyPRf1SZUJOER`YiIwqvP#Y)^I_sNtiS^^CoGZ2GrG z*`JARasbZ+}NrixnS z7~_RuSD?$D>h!GW5N*}J4^2H6ctOZQ`w^SN<^V<={oW7#4D1w8sMlQ3f=kmFOqrFk zv}Zbpa~H5f7`gbx;PN>c@{~(gV9EPd}v^P^Xy>%Xqb4 zxDri#4uiV#$g;54=O0||Z&rP8mdCq^ilV2jp4@6qSQ5nR09V~?f8f*!q_p}pXXS3p zU~T$xV|4&|c{a?ak1_6;+%00fWpu2R>};m|Sd&K(;YEOeYwrA+U6-5BoosqqX;(*8 zz?O2BsPPOFB^Y%#MpilPp`y78)xP1frX+BYt$2megrVy#>d`^5TQ!^lIT0UnNqSHM z7H&Ir%$Y|03D84t$;n~>J~u&lFwR>+D#@`gE%wn?tkFo%`#f2Qj&7!WCG(;r_6n1E zKcDj;VO1lB_Ce}o5K?j_bZsI{OBHQRfCm%MG+%*9QF zdXZm)Okvrk-L>|p|LQ8V>Cq1;efT?&T!%(5rxl$S?b8+h0ebuhWWqCRqHt8A@5~k> zXk0rUT%7jehl0Y+inM#LaVoL1oUw!u^4VA!HJOPlY0(oMYkx<#mw2M+l3F0e^n2>! zb!uU=KW4_TZzTUo!VpFF4k!@iH+ zQb_n6imLKQdT+8b5X~`cAM@GOnGV=f{P4N;{)zbIWw8F)t-(++JyhI`kt=tQBhkQz8xi|X!@p~s8XrLY=>be(%_lACSA!#Va zH4~O3FtHi$H)yEs%Tm#X7mOz>Yrzz@qOcQR*rV0qSNDlcxk2JoWf-_m0bdVDf7>>| zZWPOV?1Y2uW(Giod6=mOn-Unb>k;lZ)WcC-4ak%{cFOnxu*THA{qh)NC^1HWmilwU0CeB(!;XA~g9= z*s&NH5pXK8soYrAuO2Ahtc{y$B&pW+ZEGxSYs?0WVx$@feVt(pDM#xYLHotX1jW$= z!()$t)@dTvi7M8qjnGjg=zs| zbjKX_sqHSYzhSnH)%2OdorC>CvyZVICskmoO#i^Z7aJ1(1`#@bAdD&3jBk2@4m)EB z&eEBqcXv61uj2%Lgp-dn5eyKIU->4LrS3o zRk11RP80O*)tS({f~WQ}!$c)b@kC>$X+_^7{xa~_v!teY&nEq3qTAcugN?1QPS2)3 zS1VmjV5xQ6vYv(A3$4+W`e%8DCo~E_kI#9`1<0VP>s;Bc|0s76v4Oi=AVQm{>wF8Q z#b8bKIGEWP7h*h*Jb`&cwKif>amK#_vc)=3+}-&3IAi#CWGxmx3Iu1xi-`lS9h}C=0E|umAtpQJFQ%9-?NMI*-GR3}R!l$FQcJ&k3;kTg+{sV^ZZT8< zQf*w-QGTOBdaFI5V%~9^R;6Zyq8F2-C^u5mn6re^laOP_rSKaWicZO{${BZ8E|d>F zF;l+wY!286%@sB5VDGURYHT}-;RF( zV0^OYR~IxGS3XRgDVHnM{^(xiQT3e~XUoBj|iLno{kdnZBhn-PCI$aGLh_lG+{Yz9-yG ze!0UVxNJ9LK4BW8gMI3_F!M@XLa6iQ`aAj*^D~|$(%O@)^*6ZL9h^dYP8R=M&aE?n z99h|*2ICl>m)c1)zWh%ZUts@`9{d&m8^g^POa|^Ws~=ik8&`+63yabu+&%spZ?n4CTS%I%T;FfzZ zV5wby0x5ZeI!9K9xig`C(9PKU5a$IH$vS--zvW=%4$K^j{FPUA{{}K!Qa^I6QeBC6 zdPLNQUv=(i&o=0&U1o7TxqHFR8R#|<*)3mE%PX#^Xk0h)C2CD2=(bm>=gwkP8(#P9 zj02~!b@zg8PGIS81wY-C>v=M&5sz&2qJWfr{^EWgTJ!_`CL~`IpTL}A=T7qKd6ZqyAd&IHl)*CNy>Y*!87YW(I6|zKP)lN>=|#^5HI@ z8z1=?zq^6yL0a;hLWL?@OC!uk*E^?mi`@yGEe)+Z*KkC*zZT~?xg4WFfCv%bf_`zC4p}RkirLY0OqX!jiT6jl2vO5zFna6Es%ZE zXkqRduoi$qkwB~24CqbK>Zabp-rBT3vgg|{lyu47S2#r3PQw*OPrd+4q+w-V+C$PksRZ-3e=6)`N%sEBD(sBLn*@_ft8; zX(q7h`j-H3!+PoO@-Mw%@_;Sw|5&%SWdElVMl?q%-~E>ci~Kui!Ipeyi=QlMhY!@# z!qY(i3In%<7GqLtgrJr536$T$@ez?LY{^iqEu-NSE|o9{ch1W3U`3nH`Og1#m`@BG zNEfU?qLE8~zm5lV&Xbqg#3E7J8bYlum?yGI%jaa?v zSHognXH(;1Ya;4?%dVBxhgFludT3Zs@Ir|LA!l`Oo(G1pd+ za_(c1>fF&KIY4bYQKP()s_Tc*-)d!4)ycLETg(LcrKab*@;D8|ZK~C|O8_;=gig!h z<~I-@z%Vs`Cnb^A%$$BPkPa|6ditD)$wj5+?^!(|cAAkI!tIUVoQFPa40!RtVxXD$ zNl6=OCT`O;5vZ8RiWdfopa`KJc(mYA>5zWN)7Vb1_S3P?S=VC=#ENx=$0|qS8eugO ztIC;RSB}ti;@-^RR>SpUhXO|>k?B`Qdt#Tp=Ev9pkd#JheOTZa;QQUGMI%%sWR+;o#qloiKrQ6ezta?;MI=Le8O%FdVgKMqOTOy%2HE%=1 z0<4Qeq9Nh;#;SfA+y8J*9!9{k({6VWx%}Y2Px&wcS1J(*}(Y zMss+gYjCAk|4=eYPw42m>!!9BjZu$-UFFU7${F$0CSTS*b{PmR*1Qn-mM3jftkVur z9SK**wjo*;DYK3vi&z}u#C6ay=L{$H%;NXG_=9;_h&0#7B|=bA1(a=KpH*?e(-U5c zCe7|K|J;;yYt;O^E}wfgukgHR_zY6Gd5#(QnmF(x^b+4`&_?t_@p}_b=ut6tYjg>mJ|l$^HB4 zaK&@jg|_U))j8g*Jt-+0#a->CYy(37h5Oan zjp3pNWo4kV=d9ei_!>+dHhOkC!n{}C!jUJvu8#br1Dn@%4ed=2DY#tO=4aTN*?ffp z{68U$5tINECcC06;XrWA80{A4y)!7DlDmb7q|l~Nim<{{RhiEq5*^xg9929KHRhPL z5>&n1;joC^hcKl)KZe^5qtirokNM1{kEYq%8ccD9a*Vq&rkuF6KC)EI#KXP^l0y^f zqV%R(vQ1)oWxm9IlnS!H;)GO_sY~2-QEW}QROOKiR<~%~G3NA_X{-b*xMLKSZeR#fdtJi99%71Rd`iJYTn~&6GmQ zRJVp$Hyv|qys$Cqc&;q~jR@!q{V7sl3nbolHf9|=n#2CapI2;Lku@7ZVD|Fm&R@u3 zN)Ft`OeS`s1q$m;6^FDfkSFLiAef41kSD-J7ce4cM+zvE0mc@+z6p(nAxP7|f(XL8 z{1X2rie)gfJ)Yz0e%<@L{>l4mM3!0K3g=<%JepaLC{Xxy3flsSO$R;dpqRUT*xK46 ztMV*4H94|rF`C~q)XbT_^_6KUBfd|ifjzN7VRH081PR&TQm()h_*R2fO6BuQ=&{$5AmgTr%K*?2azdM zH;G>jEYN;V_reljo0QnJ6FdjSS{vVsjC8A9;D0a%aRE%}Bgm-?LECuku52RFErNJv z(oSc{u}vOx4XmeN+k!K5V2Z2j<7Nc~(i6Zrr(f1StbM2C|Ds}y&$o*Ve=u>2G!&<^ zPUqVr?o*!!y-B6r9JJUyfEVtTl$))XpOg|`mz@H{agXDv_P)P`)75T0#NzG{GEw`hXny)_@7UT4*X9EiPhB8 zLeoP3LX^p1hBqXI6$#WXAV-#7{GnN(9!tGYkVeh9^aF8{Bu3G4DHBH2>DiHK-v72% zXs{1lSMy4*TN9|JrS+02Lk@HI%`P`{(>K4j|MRN(Wm4$R=NI+>&2MtD?bA z5pt-E(;34(@I;v8Jq$n+3bGgp94%*!F^(RxGzuJ%5)1utghqNO7d7c!1X8ktv=A1Y zT0SES3D*k~BsAFO<8661vP@xC_tH#QI+8nz9V=iBIPRjF&1@tC$5X5= z_ma!*X|WS*4`~OIs74T)uwADaP|_hDvT@h=FGFlmUutZ){TEyo6o1&pr?6aHvb3eN zHKtanL`^}7UPj{}tqcL45@gM@%&u0OldPac9BqGvV;$<<)3G~k0T;btI-lG)$8ssl zv7XSj@ht=O7)&-FPqvAecuTyN?;dnhS(XQ?w!g>f+$I$2nAiIB$kC&FtQ4OK^#3>& z)EX`_3^?fOVSIB)uVKKJHLcVk9Swtkg1}K{iQ|e2>DEt|4KD^TpU_tke#-nTIVKA~ z6?Z2a+(os@m162Vj8bouz?x>3U@xYdw!u)-V}wOc+9mb#&e@$cOsuZeO$W!14@uUBZXn&w3sJDAeZt8{>2z-y?Jj0F9Ok_W z#`sS2gpohTgLw=DS2)mAhRnDa4Pe9d3~8bDXx<<~S3A?ashy*~lYv^{dIlk&w)ekq zEW!U>-1#0td_z8X1MgVixF)8fRw@=pC^={{)eJ}HasKDz6NSit)>LR-Cc9{t(Pw= zv|Qx;p-J}ZtbzA2N7dKQbOz66Q)yu?YAzP>p2>-uHEsVk2VD(yuCKO5N|npuDz0Bx zvx^OZ9ibCAvZ_vVntyB9)VstI>$F^nZRN~$H~rbxakhzTD-tWP~! zHTKSR!y8c@JpKz79>G?+F7Uvg&+mk>lwM4~jY|VaGv*ZGN@L?42r@y}JI1RGz4N_z?kbMpbw#2i!r~O zo{VCUNvlrNc8LoyLt9%T4Uo=~74jgvQ(>!|o4B!~4+fVfx-D<;%#=&X5TC zMDGIZhLoEQ?z;$vo`rhy+x z$}^nt<&qQJw$7dBs7ppG|Rj7Y|XQ|B*t=R^Pf zH|L(xbB3D|;Q9OiJ&091JTiVRXse$TgUEl4(L_(!6n`&PV1nvDF|o?;A=XPe$?FZ5O)=SsAg7a1rGV2IL7sryQZ_PmE0BWKixf|uA8fGVmvXaj z?T<%!e$$Tu$2r}vFZtUaU^jw2OG$mBj3<4~IF>_+OEzs-BOZiYH!}3eZM2?_r)_ci z^p>pUr1iUM03oVBR!2MZ^PEaw7tWQn_UxyJlT2*Pvs&Yd6@+0>pH=B=IehekaAsp| z&#;UpPAA+sY`x3&C1-bB=Bw2v;i%`-xWG)UZ_)u{q3e%ifri%eld=+!#*sZXmXWCj5g{)USL*MG;}`s)fjIi&u_&U?w;}CI z$~hSNK<=PwriP*GJr%6F+E?w|!;a&m!^qlcLG=b54)n zdXC?i(mI#C0u}^IaQTY7+e$Mwc%Kg#`@gy3Sc!t+oeT>=7&6h?2ZKOndV3 zxJvc$1ZD1~^E2X+ze0B>U+XBQ;W@20pZnjEY6xm@UwJzR<)BeRk;hkVRB{`PRx&zy z16T3vFgA1KgCVq8qJP-)BZ=rR#F1&UvC?XfV{=Rr9S0#JnK6bWMrR+eu@%JJgh7Z? zB2AA)2_u0ot%|R3Q&AuyZC`urV9*m&YI9S}j@&7Suo<;_5EQC8ovt>FN?BoO7%Va< zJ@$o<%x?3Hs0ipu-BHU7pG$Q4Vv;;3Cvz#rd_GEn z--ppDjTWA0fz2`+UYc4h&Ln%lrY?!8<6Ez=C!m6JkbQTjijhK3Dw|X^pj_r?$D%8P zy@`ToAXg&mO>K&4adO(H#ueNmKDvEf96)@VpIwR^Wqol+TiUZGEVm;Tg+*d9a8ItE z>SWmAuS^m4()<7KWmxeVA$gx`cHrkgVgJ7a1<@Z5cy&UTLizY>@A}t5Ar@a+4Np1- zz9nT9sT3K_u%9+{*(Xs;2AwARew@d*+im7J{7vlrhvrVRMAcK2?7J2n>5Fi!I;_Xv zJ@FQF9(GqxE6?5^xzCoK*DD!9?+7~#ve5A+?6QM85CPs=mB&Ti$k+6z5D>(BUBRre z=WWI$KoyvZe(X+i7PHqr9F@C%TmeddNqhnVXeTQevX0(~_`I!fA3dB2*sP+uM}Ux6 zp+6qaqxti#bvH^@!na+T_twA%s@OkAcv{+IG?@#FXY-58N-SIZhcP zIkGj;Bvvr~Cf_iWa$yb9L9S&#Y`kWpa{vYaZ;u7s_5;on;%Jg#Kh{zeg?!J#-0yBA znyZf{kkxPh!3S9dzw~pK9NE&C$007VV`-sQp-BEAy7BAj7Fn`}#xS+%sn@*2R9Fns z#OtiODHOz;7!8@*(W3y9Kpsjukvd*Mi?)p)z45s5&>-8#ds`d}ZXd9${!A~sJlc9d zC;D`CT2s@a?-n}Vtt&o|5|gKlrPwTepX!MFsIkB-$qY9Cv~9RCb_VxljjdPr=S4Dz zTeCqn!oo}KxUCl9+Sxz26XDiVEPU_}r=iF*fifaeFb8JsZx=Y3@QZ)vU&GBhshb@T z+|83oQ&ks7E3=OdRycGhi^Ha($$lvSN*r7o6QsYfC@`s?>A1tie!8)>rsFF1R&AHj zM4b({lQD8gNzi}ao8lE~?V#yY*ys#(a=70o)Gm5~Ur1fDb!AsV%o5niUG+4&XXKN5 z6zgVz+fB?)THGprlw7bD@Obq1Dz7wURIId3#Xw!TaJIb6V}Hkrh@;%=^MXGI2C{GX z8)cBy=(<%4%C@O<(nNh>LqSN7b9z&IE*>$`a3X3M(Y((J^V3OXbN1%pL?RRRq#!~Y zJZS3+GK3p^#n~~4KYcP?3U1mSo`Zu%C1Eyo7H2R(g4p%|S!Fd+ccu@_;pJlT+Xhz5`9FjF`NUpI>#Jyf`Qah`b|38_uy)e2%O6wskSP_E9lA@05r>1Jv=Y_}IJG+8c8J02AdIHVSE0{@dl_Ye#TK7Q_dAkn0PLDUG{y#3-^evfIyd$RT z=l-NJ65?HB)P5}maFa)pEw!^yZ6r%!ZT-WhJHEk- zJ8qkXMSc(UI`5L&@Ba2n>6qRlFjsPW>FI%}C%S(58INZl5znal7T+T*%?A}tHYTNw(;JyXptLTrmXJkO1;zdChEz?a(tz& zDc$-;=jq}&&Vo6D;s866eY4;Kr3R|SGnh^mNH~)=jPR~k-$P$wc zvv?hOB2bI=oHh%ZWfe|-nRh^so8hz~cRN`N@;hq{5X3@+ah|>|DKivWIO6C=AU+(t zBfT0+U4*g|vaz(Zz{nj?24$5RR{NAE?XG3yJQ+=~ryStHLJ-@^@=J53;I)#V*Bw5k zn(>=?)J1>8QVPzxjl}C@E~%V80WQ0k-lVni$aD$4Al~;fc*}2C=8mu1vdgqZxtut# z*WoS@zyd>nqDML_8~l`9xlaKMOZ?TKRp&bob%tqD7Idx^|M^qZTlpXe?8|@@jce>r zCw_~9BmZNoSKozc`kH?4dl2rehYg7%($#>VN7l&o{L?00Z$#zmB8H%avVlNx=tr(J zH)gzA7g^8y4xk*Q)|?#aGX=~m(!pH4|8t@ND3ARi&z+qiD;;g2RB&4xbR);jG`%YZ zzeV`gu29wBfLG3c;Y`*y1DwiplbtbICTD8ghxYg1yA3LsO}@h+L4N-uE9J&AzuI0I za9!}}0>Y#!5;33NC{{r$ox(8Mc5oNV62hO_5>pe zSRkuG4rie3wM0UUI#ihsFED4hSS?`4z~G?pu(jfLlo%|7NsrW-`$e9vCZ6Q#NfjPZ zDvW9GG(5LEq+L&GnmR=8!nSA;Cbd_rL#-oLNxKE+SAM3iM?uPvd2~dRXEw{mCKo)m z!$g0jo9GESXWpN_|@@Q0qVzA-wUARtQx#pYU0>IMe- zc`IX%Q!e?OS@*0f4@Y9Qa!GX7K3PKwW@kuX{|Xn|S+=f+1!t7VDO$QPVSlUx4j18e z#E+O(F&wyx3k#H|eix0m#F6PINt|f1h#WW95T1pvmSXKx@}XwXIE6+f;z&CWE zNEc}5FF2Q0p@&;^ClL7M6QgoYsLj9WGKJJ3G_&cqR{jq20HiX4iRg_j_j+s zC#KNrBR>RQd19^;Ltp8v2tu37ecs7cn)uW(Da1e_KzJV+&z~6>+Yx&+2^AV3w#Z65 zJ~O@=5lJ%dhN0_BS$tLpxH#U-Nfo z%BIzC&Lu*`d*mXJ_RQ!2PCfMg#ONHj-J^sxj!EK#C&Ui|z%Xkx(d2!Qs^Ht=fl?Yq zFJx1%?lN%%gPg=ngzD9Ft${Rz-1(ae-|b6vlJlr(^angNkd)8zBqt0X5R)hFBk$m( z0X2X(K0S;=-nnef{M*JT{fHhK(3rgC2%O)kzqdmCfCkj|op6CShmZLF?L%a?XVt5_ zYgV(5g2hiG6u%=PhxMC4Xzz>=%TvS~Io15rD6|mN>%9^rnN^o4qlkM+e7ME?;M`ay zu??9+Cj&apADPje{Af0SYHCK}sJ!$smVKruSa*h$-;xHq;njCo?2Xe=a+ioF{X>R% zH?4rg{^tG-z6^NRTlxFf{jdJTbwX4mVL(9kd)4@p+eEBTh*b18!XQ7Z9SSxHP?ijn zqHgooEajgmCgKO_kQRFz8!0>D#}w&=It%P*2w0|2k|mC{@HnbxlJ$*jc}2SeDZ&e5 zxAhJ4vljHftS`@Sa~&e{QexF~I?(^Zjf82;;ZQ@|Wv2nIr`EU6ckZG)W}a{SLUa+k zF|=dLY3-3+pLk-Em_PmClBl`CzXOI6kJ`B$jdQc4Ue6kI9 zG^J`P@FNN(ZJ(ROF=ja$K8^p7X#!fAsB`4aFXVO(_YEud`9HzXbR6P@ps7S+x7Wn; zcl;LjLEy@e&y9oo$AqGdMte$jE*y2^yH)7Ffs53d@J+=pY&l74nd6tc+Ln?rLwFay zry|T&YwDC2py9@mPm{yLp7zg|R`zPU8>45ZF(@B2hvN=Vnn%C9&l7(Mg?jsZ&K|@i z+rDC3-v^nsXOEP%<9E-}aK$qjhb&i}@09o3!$8x$DR&G%Ja0hp+!)boy8!hyCKa~d z**XMGB+D3_{_xp0E^|K7CeXZUJr$s#nl`s$kqvVLEw__JT#4IAa61W)X*aR05Q`vD zZv3SCve^UvY`VS$pXsuzE4RCD)V_smQjeP!=6i!>dO$-f>JsT3lo4EK`|z@ylN<`k zM#lfmNe8M;QCOA%H{AK592Deo#XF^T0^C1&(7pRhvk$sXO`esR>DknnF2@eIdD6-7 z(cg!ACS2mdNt*p|LTfVG>zUM_meW`sPi31dY^I&)W1v3FNgrl>rM2t5+=2Zr%>S_A zDY5jktHRf>Ij{yB2N}Th_jh~se|Kgk>L8`5BV7As|%K&He z&%;z5wJk?{&I*!Mbcri)e-e*!@Z@WYd+;EPXcA;h#BefJAIE;^;C>Z3x{UBZDe<0} zq%BKzwQ{Ry8wxjr<0P1JsvE942RmtX!Cv{-wDb2D;8 z9Dxbk;l2D@+&LR4b z)e6Lt=l#@k&+3xcvr@CwjlWNeL97pY6Rwod)p;riA*X-Fe*Pg?qU#6~2*d&rwk1Dj0UY!#25u zxZJets$>7aB-4e5mr_02;fn`V+$1f=xO@()rEQ5VF!crcNo}%I*4(bZqf)>XRMLt zVGtbrB31S#ia>1%HMp^&tT~(`8}s0Ez#fc)Z_K^j=yFLpw8Z$8{{L4zQu7qT|sT=)|@`ribR$0-MMKqu7j_bkjh*3{NYp8NYe|IW#Mxbq|?~HBW z8OupwvtO(38={Hm-APsPYOe7+XYc(eI=M>qK}T1EH`{k48hJ|uI-UO&c}wzsG3Qqi z#>LUHXO#MFHh6k&tvnFRN61}LEV`ZBI0o6HodVI=Inn=%vh%V>vfsT^@&JyAch zsKJkbH4SWuNEgP5p=<@>yA~;^zRa=%+U6cnntg>#bvqK=kI=ZCxDXH^@qxH3;|c}# z7@uA}Q}6Iqf*rw6kVt|*Gkl34|LL8+?tn1R<`$&6XUI{zgR1*tKt$56^eeLt8`7=! z&+cFAOZa>^Hm9&jabZKY;7g?UUO1;0`glaP3$MY=R3OJ6ojqA8VD~|xV80|fPric0 z=z`UYLF6FW!clC}!Gz#O>`K)B+f<-L$}mNg!WW5b?kB2FA0r6;g%(p3+Vd}#3)_2a zAQF7q?KezuM@(KN)!!7_X}0Q>bDVK`%O|*Tc}pie-&5$cTuiYCT+j+x&@t(&epXYF zcBN+^wdTv5Wd4+%Mzd>%`ttThQ6(xN2!4z#Ju5HzJ7YfyGoj;6w@>CytA83$7-8*UqQ#Ncdv z>Mf$iX=imf$4@gT^}Q?ydgr!Di?!R*;i&q1l>2eMfRLkt>8XQ>w0vb#;L0k;@05Ih zi`)g^M6sj~1A@4;j2| z|Cff4!hC_7@_h?ECD9Wmg&?K#lN^ozKTpOkNPbeJc6|0q{M^nO{H&fxDb78`_@9)N z-C{2ru}>PL20o=Aj|NE7JpE4qp4*SZOgVqudONZ3rFcL+NyUWL zx~nYPp)7ZCDn`&Udk)IF?wWA>Kzd{2UP{K14xck`;|-rlHnlR3ELW z1e=gJCTUjE5|CS4#w5>mb3am>6*T8NwO*T3D~p}x_2s0`Juz?io6(f)U(4mt=CBFp zsfiY?Q&*l%bx+8~FvwTf2U|a)O{7LybMgZ5x++*mvglM!V}U3ZPATR3kLCJ;hn8IR5jkSfzvbZF_ zDY@(t576}0_FeFl9gYD<(ho**hvN^Cg8It}Q*1yI@(ylrBdt)08B9Q=)!^`FiJPM( zP4EZXWU4!4or&fSuOm&xRg)E@p=lR9GkjM_HhBFv?>L+KeZ*j{+w{J5Z>ix<@ax@x zVC0*^=mbc?5o+-KZ;zjJ@Gv&G(H=c!D`b!5Dd2Qd+vn9*Y5dwmZ^kph_RW>{!9KH@ ze}w2-zt*NgNAt+AzfL2i-gq0L1uibg;!v(9Y5y^!T$GBi($6*MC_H2)UR$u)7rFT&u4XP_?_VZpHGb zpPd@h&NhmwBM!&m*H;ovj%~sDhzC|i1HYznsHHrdve1gDQ|3!$Ruq0Rbsyns5{=+7 z`Ssf)XJ2Yh*|$?jsig|^HKp0<=y+N%%4M+!4nCNN-%%3!nzb;v#H*SU4O2hJN3DGQ zb9c+3$K({HfN8HxZGp>Hncu2@hC%1-APZ#8r%Y@Sm^v+gyfFxHA6nV@1bV6)6E#>d zE-Y6rX=wG5jS`0DWjeS2{+m|+i)Q%L*T>VSO9+eJGmv$Dkig~`kL=;v*7-r|fFWg8 z)QF^3c{V(2L^5w4to9X-e##aob-Ao(A3qJP$_g}yOjhT*oGym@)!ZWfcSbDAeG(|9 zo99V5lXkNvDwKAc;J1uf3p1tsDOFv^MXQr<3WWiHYl zi?@BW$){XS)>}xI(7{)cuN#DAe13b@F6oL{0AqAtja@OxSd)%S$IFjJx()$G6RxTSw z2tb0%TO{n;cbPX?f|#}gE9MZC-+t#sXfQv|2 zRneo9{nOXtW>O_#Ukv^~t5P}u-B^c_QJtCsMM8v=7DFBk4xD7hBFe@6vr^Hv(O2hj zA^Z|$;EkZHJX<%_Ho}}rYeqsyY#M1AOCgz3h|MN5!5U`wNtI)o!5q7VjGQ;vFRK|{z|w-NSStWS>-RU5W+RiT)7Evr{XLb5uZ+VIyS z9+~$bex2*#`bO=2Uy_fj$~GT+ZG^*m*>{st_pU*aDl1~XJaj<7B=aH|WK*SRS?mx& zhosu?JkSA?!zg9{twe6Y3HR<-DI%kw)SYwSPb;+BedSl@hC&|0Z-B9gUcqNnwwisF^I+8O_FP1j&no3I<3VOUJG$r#lu>Ua&Z$s-P_XG$_6)px}4dG>hK z=fbL}8Xtk1Lvr^ixIv;MTe7@!MN#k1BSw1E#RVf3u{Q3@sd`8k#P z&2@9Fzt03HO=ybF*Zbf4J#M;O{rJc87=66;JZ~o6tj;$fH>^*c;OoQgU75>u;@=Va zbO=y)9qh&VoH(YAQu*YNCQva8%uE?N?oJu{rnZjl%jfEKv;SB(pVQ23{hf2kdwrvd zdX2GmdAE94>U8U9Q<=M6{F!?!D0S@Sf#hq$|Gc2( zGC$l{h#qtk1?Fei)_hFdpJMw<2)-xRr;ch61@^idUGCbMUj-r?WzVeKKHRy4@;icV zzIK6#dt{E0eJdrNyyKkG^YltEGnON|{ON&JIn*A7p_j-Dm%hQ)bN>kUz(qOYLm)xs zIG3nLZp+K>nW$%FkT@er2?ioRH7ft^MuZv25+cV3G>>1q76y4KhyrY2Ays$tczXo4pUq5~A6J#o1 zyMDk(N}Fb8=9jVD5dlLMU~+YifIKATH;Gu-ptJtRW@LY=Ut0NAGdD(bPz5_E$4`L< zO}55@w-bcpN~o9YW#u>LNo8fwJYk--h!LI12oXSNX>W%TYMN(l#nL5!1OK~q^Hd%b zZoI6+9_*)uh|x@=AF|m?0BJf}A4;U`T+GsFt8Mjh>!# zDU%pgfJ5^zdDX5k6Ygmp9*{?8yhCME>8|~Sss=YeW*RWUp9_4yDr?-7pV-#Oh99XN zK4Uc^q5?he9jBw8Cp{r?$`T>`+z;)ly$k~=)!8S)P6o#O9A~seg8kRyv z)C>Mzp4Iv;>0{#aH}vn5+@x$3IWBL8DU2^g`W1T$poeny%WyYO8Er+ST799}O^1=m zI!z9;0So2d7;7ivuIc$9AYsUHl@_b4>;*~M<(6`o;%XPbjcV7%z;abSGkSX28c1m$ z{mwi*ge^O&=HebDhZ|KBmBMzh1j#AlVe5&2{Fi!vRmk!x>q208`Q(i)7eKo0C~j1A z3)Mce-lH^7#>pVRV{bt6IeKF0F&)@XXvL{5&hK{&20 zzjpO7D_#$@BfpahtF@BVJ@h|V>c#9^9t$(1;?y`ow}I=NiwTs?#}w-+GYn!SeFQ^# zvW9+UNAW^`G3*ylq(wfc-AtMoacK@D!N3CMp0RoVsg?_g?)yz@SNO3kRO=PbeRn+} z=$OD@rf&OF&_TjBLFe^*wO}jOYee)NOP;$kmCYMq^EcTb5n{(cB%9FWPCQ-6{`?C<} z4^KgKT*evGh7I^^yPt~gXtu7EP)#8ilGCSwGZL{(nT?)tEV8Nz#ZWUAU)hwd3^FoV z-+uZe1Cmx zx7@jpegtmgD!M6H^meG^omo-kaxtDHa)zo0=(G-O2E(q*N@^=pxn;$FO0lzl8Pi7r zG#>^9w?_T8$Xpiee-WZ$>#g!c6AN86D28-OLD;1G4r)7QuRjFpWy^5wXA=-!-~yHk zxK0#8i4L}dK43Aon@(Dx1LBETPDldQ3T9@l5IYjCxSqV;{Br-1Tnpj7QGfjG2X=*u z6?wyD(qYl#%^SuXnge{d_&b3+cfga4&?22Sw-`){{^w z#XN)H8*C^)t^BdTiz4&=WpWYW`;1BxZj;qFl}v+=N&bCo{lEBc#DNM#_2f10g>6D; z57)5aVg`ZEg=OlvZ{ZCupTD(k^vFsLF|~ed?X*45);yi|k@TV^LHmR}eOw{G>NnabvAF%%>tbweq1#@kfumar?aA5QIfNG3pNXRI z`)Z~UF17mCF!XzRpp~^;bLnLFdg&sZ2i@YmRHs|*ogQ_G4`ouaG+zV5aZb+;$(u*Q z={3t#q!8-GdB)hyH6o=iP%TRar*9VUA@0Ny9C2!tdC%T5yG@7oax)I3pV=Qqg_nTR{K{j-%i13I}HQcxhL`TcyY`PsifZ$NtLWf%hhhS&c zxKoe{gnLFqPT4wC_6xdrI7!i%hBBUI7B4yKd}J|v802;bs9>1JYtT1X^bOSEiRl&R z+FIKUhXwFdLE`^x_E@v|*2@_)qpktFib5!@yd%zbygFQ1w}h+Py=-ia5BWSkFZ_KR zYb&3@;r;z|O_PNg2vSu{7^xjjGss?UcE$cTOqFXA4XNB&_UIDRp2*%H)AJRtA9M9faR>cp z0^M)(a$Y=i)keuf&3l~V)ya)iod&E}w+jX~ON=3THCK(sGvQ#-35~VY3Zw__Nvm^H z*VCd#FPjgq1=#eTLj1FYpTiqv`wA-k~fvbW5$m%H{ z<^qPWMTpLQz&*X94uXD<>+=w-$&@B`iPOkdr%{%A)Ftf_v6p^>jyAW}(lIhZG2I?! z#N(oQ)P{$nL>I6zmF48-S@WDH&fae2Qde6*PG}l6wWMLnkGEx3Rv+cskoI>N*0zvl z8ch{OM`}N1>m3UkAG_4*4_CNEhM0NtsOz7jgMespbBC|$-x66E5q(5yCo^@ke2|le(0J^MKO9&ezi{WE zK8$DFQ3^+x3<*Q@jx< zG8QKLo28-a#ZbO1b{pRFF;uV-QhiYu_<aqX3cHw} zvO_uoj(=}s_Bfe^WpIc?31k$m#$~-c=A3ap+dGW%XqOxB9c<#ONBP?;HjE8~u=>14 zq7#XuSEv&(XDu4w1G#bWYT+e|P}Sc}=<&+akJ=Q}XgQc(&_~#o@_Xg995}8-~{&U*9Wyx<5`+us+*u6VtJxIHfVs9k8CVvVm4+kxTT^2$Xg!7p}f2 z$KM6vAzN0Z(92R1=vPXf8WE^YjhIYZf0YKxPUBRleA&X-j5)x^s8GC0;-Nk!$~%Zc zzsdWpW#GiJ3_Z#gtT+BU$K+6In3n5z**-9FSAT&&B_q6dWIbsQlN~Ti*)onT-Xip~ zzGl(2bV{0~#_@@v*!@*>Q1}DOHZ+p(o)=>l&Du zgYd>Sxe9fMnHhZYWo{pfdJJhYaZQg)tnQvQ5S~Fw`9Sd*A15yAYKo2+)6;xLIV9HP z^Cz0YJs5Gr)T}F@b%*oTR&b*crdB}=_(E=hrSL)~AYP(7fL@*MTw0lab0~a`f>YC<^#OotFa64{B`$s53CtnHU_K59z($cjM7Mh#JO|lHB;A2Ve(q8ok*^6rka@iVRNS40x4|lv0(cQG?kwV;) zoL&S?w9f~W3hLGXv5I(wJeTC`tpS;DVfW$If-QWN&R_7v)W6A73mIPY1b~_bcRauy zO4zKiD9|pOMVX37^s7U=7J;Za$2&FTG-c+rh38o@;@V;w4rQTbgUB>&Wwv2!>fuJ) zpbl$f%j;5>%i{A>vKJDA8uoQW+d%Gv1rUNygDIhaoRJv3u8hsAWvPTFqulyD8X=#$ zoc%7O)ykZkvnw`@p_tKE0j{>L=$H?XnwYZAm-T^O@bTpRe1z%vb+^ln`%s^R+JY8hF__?;xXXU zsC2gVI7Q$5;Ia;J)xs;m1?lo&;S-c85!9EM%4&(qhQGAV2#CvpcXXw{fW*EeEZy-F zIS(}6Cok(GnWjluhXFXB9{X#JA8d*u^=;Fi)gr_p+EyvB=cpe=XO~6y#+4S<*BVX} z5a2g1^dRPl2&3S!w&<)A+nTL}hZU2lz?!yrw1#WLj$r6h$&OG#u=R}+hE2HY>Eg@r z5XBlpN#@E`ZhYYz2VdM}Jn0?jBZ}Cv9Ye^ z$$vgEosoVxb{jt|Dd)%FJbk)xrkw#Y5RyKwu< zbgHUnnW$+w8~~|MNtVzonQOZ>lw2~nY&BXV2Fpe`W{aH5>6N88|j% z?_0C5xE)f;Nh(V4Wb&>(YLJ@q8yg6(GB1C9Pl4jn0f^c*Isr{^rJ8>(!1fS5_T?o= zQO$SmC28wPEu@$>$vBGJNwnLNjTM;Ov}V5a1+kJ53|o-wOrPaJ-g%9m%^33BM!Sn5 zT|$VUe+=H;37q5t4pQ?QaqhU{1M<`@@u^*eAiH&MLe9J?sp;pq0k)(CvA(iR=Ou*I z+cAX}AeZY(-CD|B%wz%8))B8T-IHQ(YAXI0RLUu;QXh|AXN*6lX%nNJ)i%#df@}T) zt6$_aKlh8Cx8u3UH*hAKC;bGmk`^pI)Gr5kT8-6rjeeCKL6Y<_1`|vIqI!&AYe!w# zcLCGCV1MCF_rg6S{wrlZK-wR;4Fj!y<^@1h^ z2t@?&O}fG0Xu{yqNj>U9@F8ym5wCfl9wj3Is`*1lib+F?NO-W*P-&!VvF#;i3zbWq zn`iKf>qju3uNBiJLbJq$d4@sb`8$!0YjA78NJZN_A_$vCXYa=Lh5j8eI#)bne*2Ud z*a;w4T@h+r4i;FM?LFBpAREyZeW8Kr#BhC+DqjgAtVjQtS_QKf5&aSClzTK_4xh+l zC^h|C9?NMq+w&qW1j?jHZ#WNf6A8A1HjFPfLDjFbxabn*l2^17+2nv>vBAsaM8GV9 z0D0k07G|N~F)Hsdt~{^TBn2)Ek?imx@S9Ge8?4aidmA(9dJjUv8`9n}CNCja8}5!P z6N>0!`^Fy2VdbW$x$HS(g&IVf+o{7)1%%!4JRe>9!W^BMP;0k08qXG)P?1OYygVva z!nZ%WFKC=-Z57r2;W-hg{)9A3LkYadW3A$PCK+`HbWV0&a9b_EI_cKeHU-Pa09gHg43P3Q&+=Y2?ICSx(dSra(ZTAU;{2&N{KaKrdVTO3}4rKQr6TCufWJsCZ# z&%T0%*<-L;wbkHs^y=;lb&qhw_PSPUG=_W|&*vX=NSgA>EOT_yu6(CbWBlJOz1)V` zmPcV8NS=K4=!QxXYE$VNRt+vXfQ+N(oZ+j|vDX)rFr-A4(*@$Pd-gS#Qhb%il=0@M z>OP`5JBqv?$Pw8HW(vWyWnx%D-*pj5ZTml{np=Z@mQ-qyds2|Or{$&CTK|d@#MKe8 zLVkK&P+dQfQPUHUQOiiACtgaO_9^COJ@FJ6_5okYiBPbawmUFwj30S^0@#vqWqu}; z&8Z1D+zxCZ#JF|pldHB^yTq@mJ%mT+na}j~whs|V-UzS;67rFN(V}?UK3Zl86 zE(dz;&~Jw2QsdVG%w?$O*V;218Na@Jc7^-XFl=4c{rhIbIf*f|`sie4M;_^)3{B^= z3@{{#OoT-fCHvisX-X+VY=B)Cv%yzwoqd)PT1gdmL{Yh|*3T8h>}p^)gro;)SAah_ z+M5jX-duMC3S&p~q#eeXC2abz(p^xgcw$vbt1-WB_lG;k)gXoP#OV2y1ONYuAztS5^1h-6c7If&%i5Uhfs| z@1Jk}IM068e&2o8*|X0)XV%$ky$5+a$t-O+vp%JXK>xD(-krsO6w5NMxh*E&%v1b$ zt-Q)&gT5*Cwkqo?uM8r=x$!EAj!d26n<|s9>H<_snaSfOyGL6tOTahq@4jj%-d(~JMyr)g6GR>wiCojoOH6>bOMw`qY3Wug zmbS@h0p%-ZgaauYHS)*Ly}lk$7YyB9yfPgIe~y=SyG1#2l;>;Kw#hi?b$wKPz)+et z)gr0XOpeIn{53xBsuQ>L!XrGaFhyXR4RIj`nNr-{BpR>b*SGK*3RCUT4 zpZFzhX|Tk7oTRT}cvKxeFt^DyC2`;d4>;YgOW>d6a42wakd(3OkE{qqI$asi-=lSq z4XK-ivV$+(+W_vJ%=vs`sA}s{0JRZ@=f&g(W9#5`P9b&mG-dCEv-1NDrS~}_hxDGJ zz*JW84~48vaQ!Nq+4VKURj4+KKz%OY`|(-FX%?Ynl=oE7eCmAa;JFB_)p*xI+oVe* z>oU8+iVxf28p7=U7=9bY42Y@(F6`SAM=4*E$Azs-WZx8$aJdb1O^SDs>cfg4Plc8| zZudrR_9iAgb;#(mguncdkLc?lr^l(N2sD7c>_K}`pCbJvac;M4cqow8hpajPq?brm z)Gq6=MMx1UICf!%UA?8{VF!4XoTcjK?%`78sJUbVdfXSZn$i9C6Wd+{w(`CZa#0P; z5SgLr!8%*3_F0hGn=C+Gk6@`@khg|rJ4>12^LhugHYdPAap#Ebtcv7gHPa97zJLsN zxz2v7#u>{jA8~Y>rPAA_ul{69x-N^aTQt&p0^tp~EB0ma>VNnUVBL^x-xB}Ylv0cN zzWnEYleOcYds7I&(W@S*9h65#3{E}#+lM?%ry6kHQsT2tXy zf0-YW(V!BvIxmt8VB+--O0Lzs`J{NNK-T+dDw+or)B4uE(M@HcNF|eRIrF$EhJmRL za`{8W-oM?`B0eC3@!Q!>2fwG_W!v?!&z=9}kta-6*DaT-X(gp|l9w*Fbl(>)MSd^l z^XItvkeYFDSp?9sXRnhCS|vH1M)#(^2W6d+b9KX5y7!3dH6)!&4D=xs;_VxF^q!bA?|5sv|;=RxXhZ=W~D*%B|eba9H7F@V?NJIsL{C&I>!e; z(I{6&*Elbqc5N=54ek4Ocqj{-kbZEcBB+&8uWl1aVfDrtHky9&1SDDFxf-5&SC&!q#iHBua}%fkF$vgm@#Bnusuh;T%zZOYsw zJKueBI9jd+CYZzih&H6JUVc(mJi3tFX2Lf7t)|;a@T2+sbMIsJ@0IS!i4+-=G3vvz zDJ|xMvt=el93ykz5fYx$s02H?rxMumYn8v#_ITD}L~EkLJ|8yPvPpMBzP{!2HE5sh z@Pan)O=b!#V#rxd>FDsNbBTh(%Xd8!?!|N2kvG8>IY2krO0+akyQitROb&aLR9(zj zom~}JhN5Mvlt@jF%u}0SI!SrBNKgH#a%Wz-*<-}T=aq}4OXI6Wj|Nv8nw%D^iV;zg zHoGVd-}0d;&2BPT3DGb^Yp@YJY;Be45G;qOPdYxx!dFY5n+aUp-q=PhHy>CFOEj zIt`hv_1TFDrMo$GcT=wjxxtC2wmC7I{CNH&QzyXFjDRPPI?mB>E!WCzXJvIatTkY%*rM`lmF&rMA0>}~AEpstR- z^@0;%s&IX4JEnHGIE1r|q}o+I-$f-0q`nL-fMa9J&wHC9Jt=IqEF+e>Th(l9-Bs>n z6s^B`iY(H*Max&ysDypE?{EF~^tCB_xnDKU+o#@6%$3ZqF5t8Ms(YKjFTQ&)GU*w@ zv{JCQB1gVX2Tf)+k;K-my-M6;JK%xESMgQ>zTnY>mDhjtneoxoxZoEY>;2o(y_d(dxpL zV&D7{qD}d}xu(fsudxXrwB#`{63|JU9McT&n1>UFOgb!lJNogOImMnb)e*j$H_6hz z8EAJ5+10+d8cg=vVA~WBO;Mf3ABz(ciLS*t<<6+5kvxw(NsFWHUXI&B;g|RxbrF|= z5fe9M7wYDFaTNQJrWfCHlIYn==n3Y)RgmeJsrg2l)!OIkmj`Nf&GIi|wSfPP56fpAJ(S;SmHSQ!HWTHsrm9*C$2 zt|E;?2V0EpJ7vLqKUsf{8Ghj9)Q}EB2hN19hL9I?p5h>i9(96R9mr0@D4B!kxSxkz zQd>}X+53ij!5A_LH&B2smZgEQa8&6k>UDwa7*$uOGm}EGU$QD_TGAK0CzQ4jj%J6h zV|O<%Lc~;mWL${KZ6b)CPVeQ<0K=KjP@~4Ks2I_LNIEGvG;Xj8_B7I%7V+shUri2K zR0R5CC}Vv$PYlL6qwTZaWL=Q<%VulddX1kDH{!xts|Wtr`VAUzM?6j4s#;$rTAVE6Ty4zwx$mW4A+NDIVR2Yj=-@}^k4#! z=5&e+qjKXS(pwyJjA3qxZ;DCA*xLm86dA3vs}d7&Zy^|cC~MT(*(Qm5PTWMf!EY$L zfw;KNG3yihy}{=q&Bhz!Otc8h-_WEW{F4po5z%}W9V#n;>MUAQO*vJSHw|$;(wCPo z9ZFp}1#8cyiA1HzJ;q>t2kBy_VSK4V`;lIYkBwp5g;zlI%66C2$5Vx#Am;tO8TehJ*9i5hDxM z;8EbV^VK`^1D@<*;9Ic;CUPl0e&Ww)v=>cj>AQ}6DlK@-C^_ux>|5*8ynJ`e*_e9s z{dSWH=4q!K#s(K_j{$&sJIpQ9Fk+} zB8N)VkCFm!h6K)8Cx<`S_r&g|_|@2Febyc`??l6B5x3DQtZ07Skv|dpvTu(nIEA!_ zcj*NZe+^wLMv-iJG{Yh&tkiKv72C}Rs4radx@j(7dwJF_I44+TNR)>HiB5K;s)SIV zRmc6y(%ksCo4(({wd}~vD@~gQxt9>kMb?(g_b$4SLlmIF5ms)}py3$JoN84PknAP# zc2$nuTRJ6Gi|XAdrR#x_kEfo~1`a*ZQ>s?XVP)+g)c&k~WbqARY}y~vaCGELzz}CU zfz1$CQCf5Ki^rnO3z)62CH=w^n>Tc_!^djrxMJJg|u>%tz9c zP_!?@bv5tH$Z7%miUlyZ>09-cm>C>|&4ORe!=VnVGF5@w(GvyGV;Gv0OQ@Mgm`w>7 zr=4-fi9{Ykk#ymD442>)uQn4M8ZdEM0!hP>*^$(w8HDGK7!4MY8w2CSo5ApKAXslo z=AcPxax0Ky^TcGN?$Nul6vOKL-{ebj1H zW|)9JoOm~wGXzFNYJPE7>P)G4779}ADUi_g%(3k4K1%HGJeW;CuDqb=F+Cmj`N(fBWLG>lDB)of8 zGh`R0NKKxYNb)%aNr8_>av1ZCic~y%^Qc1*s!i6RYGUBnxQ9my%$%NnY5n6bRWq+= z;p58k7|1~c<@K-Ht6vkJTzAARd|w&;D}5Ow;{6TJ>Sg<7$DMG;<;vxe=`+zQoB)7S zX@pKh({!AU5_5OCOq(813N=;%$B9_OkZz;izfHCGfPtc)oIYti29IbhB7$F0H-gTb z-kTq|GMZc12BG@V7V5dmx?br;KGMD%=XrXiy0bh`>-i1hK(*SFd->ftdf<}#`c~vR z;L)Qki2o1Y!8q*mE8HIxmzxN2kId)eAcwHxs%HW{No*f1WPO)_ame#a8vedPenMlT z;a9GDy`-p_r;3c?Y{q!G8=|=>rzm1{>uOo%MjsTf*r+z=cjMY zL;_;JDx{**1!8*Oq_|CN^*RA{4l&{CHB){i3NW4z}6$EV$2XpRO!Crf+nDf zyvVYe>`9|7lArr#UO;`!x3yHML!rEWp1PP$-|W{}Z2F%n8D>{TXVPqBlDQiI6O{u= zz40biZ5|PajfJTu@*V;Kw}`PR@=qBWs^t|0uaz2Y6vvhrUzrCcujJd3;ZaXC*ywP$ zQ%01sFu{qr0@VvU`3-VF+hJwBjkea)6UZ?k0evwlhJtzt6Ukoh4-9Vg64T0flAnZB zU@>WsPBS{yZZfuKZ!$tQA8>y_P{CG+ce+5*h)T8jofJu%U(*y0;g%d^Z642Y8;zP>S?GQTN~cqPwr|`9Vv1^nuXU4tJJh} z)%7!7>NNAkUp28`EzWUE2QbkEl|+kOjCf%o0nJM6Y%!ckSGzoAeV_G1-$F*8U%^^O zw~|SOCu*DD#4})TFp`R!)O+LIMhUWP4zNM+Ewo$H6BI)^niVXR`k&cH>zm(1_iah* zJ(iL$SRnHsUozU_wp%me+0+>=$;0BUB)={ocqS_Cd#~wnf z#+-tf`%?t7fc--n4KYGsWGRGqDuXW^Kr1z*eO!N(5liZQ~a^cI)GNL zSw6c21{LVjp^c8_rX3j98d^^BuCTa(BSrT3?TidgTa8qL!5y1a-F;9E{Q{uijh(yt zLi=BWd`&&?h?9Hw$5<^`aratEpMKcABI`NyptL7Dt8y|a=Ipuv#BxdOO*Lw9Y*$k2 z5OpYi6YTN}44T~gR%8;Hs4Hdp@c_v2BQNiHufhu7w9?gxAOF~hUiIDUm|U zx4W*kIfVtYM&J9Ji@S5`wFTwTHAp7qKToEA?@7YZ zco9O;rFG%z1bd5P7@+57j9i(cQchb@_(k<&OUjS#sxRt#t8045*6kzp4tegtM%(tp zIngTjh`IJEcUCyx_9>B0{tCcUwz4VWZ3tyAj^c5#7L$@S(o9YemoJ)ZPrigRQHKUI z>39JzI;08l(%}0-kP1n0wVh*k1mepuQNd2s2%>5RUidpvT^BU-9@rsIGW0Nzbi#|l z7fHFs=J~#qeJQbMWPw^ClTojM6(0LG&P!f746HNL64(^_rB1~5r?}g`%pf#4LxoO| zY5~uT3tr&!b45)67$6UEavfnK3a6r0C7P8fPwyjqqHCfCS1&S}#{se&&08sEuea9G ztFv6F%TsBMf~&Vf(pl_9%4}M~GccakUZ|4;&6!P0@l^_fQb;RmtSfh zGhg)7TKi-5p2*v45!rsoqV^2-La=<-ANdi}t)dgKb%Vy%tH;)WXD_N~-|J%6jbvBM zyAY&i7aD2njNGtpXb`^Hl;b3XT1vS)5y-^+LU4{oN+_Hx5t#3ex10Y8<1Y&eU{{8- z3E>MSwcvucs6}bx1~Li22t2BfILVJ(Rpl_IqAq6>QUg0%NGrtASqOAf`@)vz{(5(M z<2jWct|puEAsd)&8tM8&EB#ZTzq4nGwp+ut0x}xx1&Y{JTsAcNbMIv^8OJUxyrXuj@kr4FlVas}Ow&yq4PcWAjGc*XaCl&3vg_ zd~cl!@yiheHM?)~`sO<9YaO**D;<$OW)ITL4{)h+=SVOJ_C(koy=)Fa>6BQQ!poZ( zo>0v>8sI%j9z2960CE{|z}QASb5t7$tdwD`ftz5vDv3h#`($Zd@y^_Z?A9w`cAaFO z5!o7IS^Cb&&Ahg*;Uv4GXi6Lr7N(wW+3t)DL*IhYa-z$D7^errl9aB)BENc&P$X65f ztOJX)RyA})N(ur-i4ZVQms4-t9eYsVupg?1yuy6$Me~UZBF4I5*qkk<+LXSVi9*|ckezycy=zh0cR#>5}oE07D zF#(+vjJ?YLtA`Z)YE|?%rgoJH+FD#SgdT;~tf0rjwV!QeLQ*jK`cG6j5h>VZT?pEO zP>_QA*F~`YL|=O&X-vU58~XQ5^L*$+VDAl$dnU4$gd6a}hQU44PLYHXSZ34WUL@D@ zf<7Mu`q1kIZr{{p{BsVw^ZnMh(4?0OXwr)a%%4wD;35xrotX}7xJ7dh-sOQC_fxaI z7YY~sgN7*zTmpc*1X#f1TNL-;U+pvTGogQf6;WVd_#eR61)y+2w%?Qfuk`E#A^8V{ zNfGd*un-t~3+dkY4Lj~4uS7X!L1)=WvkID2StjOI`yFVvv6M+c{51tzx?LH>u-3^h^NfKRN1 z|26$BQG&qm_Ze|ujwB(rFffiTmYkL@#%6ZroKTGdJI;Rw_=!HXLBRNfrm+LF+ERdX z_Yv>S-ve!v2iKza6Al31?ce`)9>^p-fLl00;pA%nFV%$qE|K3Q@BglN|6o*jS1_(K z3okwQgDL%YZ9S-~{(y+)4kdE_J9izQ>EH}sLrsLxd|jOXS)-5c7Z!dnj-=0T($6}> z4F5xXP(btnaSZq`5!09Ce~5p|6~e&KJRr6Q|0bd&1|a==0i1_)aC~Y1^gt7iiT>@1 zg?2Hf|LPh%WP~oK6R3eb@0URp775IG^i$SG>4**vQUI2GO9YEKX@ei;HzFu_bGRQLS(_pKHdRIK{J366$;Q%Ip>WSr2!$xQt+%$(5O v<39-MJ>YA8{009z#z%UO@^{eTcg*1d1!3_wg%V2uMgS%YANotV{S)zjc9=jx delta 39055 zcmZ6yV{~OPhb)HS zzXM2kh6Ci(M*<>p!vPJ?;G51dAtC>%@9@tNVL?Ekq5pk?Fv#hvbszAIKnbABRZp)} zN%R|qR)jG*7+OmS^jL=qI%%)3ME^o$gM3Te4+2pv=w~V!8;ORZXk949Fxl|JiMZgkGrZF_RwgkF{t!MAmjs#D zoliD7MXvrj5Y*)I1<@Io6KOc03~+RrX|gJ+!xl+%}`Qk~+FEQ8NZyJH9;A zM!+on+=b(NYY^2k>JoLQ3O*NxzlezqW?*o3wbP_}{DM0PJjq9Awq`P%<{6?ua^Ae1 z>v>TvMtWsDYl`;*LJD#>rj&YAOkwr}qY>Bb2%8Tk;^46~o?fv%^By3Djq7H=k&M9F zVqe)g$~-V{S7)zT%m#BIuIVBgKtJZvm>Nr;6<%qMcAVs3`Nw>Hcz_;H+u7ON>+#ra zDpv|4ER|+l;qP;6EObJ5G3Uv#{e`L%pYb|h8l6)dyMuYug?t_Mfyt>Uw1#V(u+FMy zOfySG)T1?h(;15Au_yop%kQ=*91T!Ju9d03lkDcVR7Ei z`K;5mxsQN_TC3x5SvSUc#YBwQdxG=#%3|PtM{QyN@EFCL5bRLdqTn;Ek*}@dF znqpF4y+MqA9c92SG0u?ty+u)P5mso#pM`m-c7Xx3w-{Wk-U1KecY+|W+ZHD4a}m3r zPrgtItB8C0aFSLR(S$U(&vcE!5<-Cu;%rUi&GnYhjTq4~;~p+z=ICwn!sx^295s6A zPT<1Z#Muf-t3jV42}P@nOkJ>Ms>K*w+flg$l7_VT>vB4Yv;jv^MOpnBEwI{m(dy~j zk$+=#u=(5N9=JN}i!C+qq#fq{mPq$&18>(BBDo-me``9E5fSP-7cv-FILG+vr5ACZ zC4T)MfBql$YIirv1I7{y1MC&igptR-ed&8#=v&K)1T?{eXq^J1NJhnOiODeJ=OvU4 z>8_L&Yke(zsRF?Jq8Z7QrP=PlUJGKH)|#@LkNVRox>%c<`Z6|hw!S`|uAl;Owwdxx zhlOy9cF1DSsBooB2K&kA2vm7Waab_w3{DDlTta7GjDy0qF0rph~JgcM(cpmP~ zYnHjh?n@|ffM{bcw%`8M(+PTymEd#KL-!W`GpX@!nX;)xax@ zwl#lU0l!1uT&43QNM%~dRZBn}yIMB!iR3%{H`!zN8XhR@Q(0>|dZ%)aB) zDe)PdyAo2H2h%Ee01SgIO-HMEIH95#GLHqePnHWS4gI^?kPVbegw~ySbHmD;ukja- zBAxGnB6~(3i`J={u3xtm89^FODx%Za-TGNRQgaP8yV4p0ff-y#!-4%|l>eNv|HAGc z)(VZ>GSqfz=-=avmAaUD9$L7>oYuS zgd@Vc-=SiPaq*SJOIMvp_RpaRh1jp3xD=Za-3#Z|D_RxHiVh_3{!ac6LVS3<1Oh6(Ew>O&~F@kGMBc}3AZfdd94lC58* zy!Tv?;7NllfcJkmgU%_@_O_0Ur`L$GGvXVZ*MX0vqx>-nq7V@S)f`et`a&>aDgMBU zN;6ce=o@$eT0ie*59&&+ByB0RC?yd6K=xsMNOx1@iS0RZPx_-O z{d8k_fIkBPFZ)aE_u3lt^ zbcB^4yF zv+%tEalY`M&ubF=JVQ*WRocgD8!DbOJ~CC<6)id~m6^QUL(4Z68v1RG#XV-};XTt% zwu`!-?g*0IOND7Unv9CuCw6a4aYaE*?~SET*2 zTwFX3)G@^ZD3x9@hbwE)mO{KO1hTRm?)P}-DGUp@WR_H*Tj=8XY1pm7SyJVh} z3)VHL{e%6MT3s9xJW{JtisL&MLzW(aFm}PJ$4#YTcKnpG(YimW_c6g>HPE-$lGOx| zjxmHU+MMPK?^3W@&o+sZm@q`ZvlQ$Lal%|^gnhzw%}S;?T+#j^G=M3v|Eea8vIURA zD1gB?YzZHon@gD9WpP!pLC!k&)4h1_$0&FuBhG;&uU5lkY^B5I#T;8t1{78%1?ddF zw1yPp0fmv7Nw)y=3^I@0s^3{d?FK%;kXlI@m5@CgVlIIr4%({yZ&1DLh{;zUMuY)K z2ZLsAW$JIl)AEoGV4sJN+3zzSyMuNacszdfVuPP4)i#+1b&^Z0kL?Ukx4C6{h6r2b z#pT$)2IndAD>YW_!=r1McI3R>+tl5D87UKC-#`p-K;7ZknZ9f}RCfn0F6q<*e2>{4 zc~EIik~7s)th6tKANKF$2$HW07(B!J1iPhezu~;n{(eVy4og>)i1aKWx9HKKqmE>0 z1iXsTLk(>;cggz-&dAVxk@_`wnIL`LSld3H!BRMr#KQTHzE>vHO(u|@3?2!Dk8h-Bi27L|7I{gd~B_fTL>N8!%$a_cJ_a7hO zvJYWi)Xcpx&Bqp?|BPW32Q=RPDBy?x>Wp_{<|gS`^ttFjO9*@?>;Ts#N!MhaOl2Oe zB_60HozkeF*YnIb^bjTx(qs;j4yMykZN4X#60E>RHFHJhOc-WkFQKLcNbT;9yGY>& zH4!5RHO(kJ6w@wD6I`zodwvcwbsccUbi-0Q zTl-s7*&iT~AUFTYRSBRm1_3Zo1`-HH%?S9^D%Y)ms4Rpgk?AulU>t3>UZ$XsKbKS) z{M$@$zSp=l?GOnV`JTrzWV#!8y>uiw&DoJhz^sWx%HefA*>=6*&iM?uJEjf9wZTbW zpEVL@q~=?mB1Pln(PvZUP-a#(m*Om@4WSS%)Z#IdYV8g((mI#Y>?X{64CUG5j{vZ| z#jTyp5^JjD9h$LtQTm6+V!nuPJ%wFN3FSsP{Z%wrMiH(5ae6H<{g=37Cdep+9@C)zG%u&$+GEI zCgS%ryvP8RtFtY(BZ`m1#$GY&0tMXk0Y)SnPGxs~KgNj*md{zp*rUaZ-s)BSb-(Jq zGmba@Yd8-dtFoN-plGDY; zq7|W}{zJRMN|t%?Gf^k+)Zay9Lu@>G4ca4!?Qc+wuw6JxV`|Z4y84ZVGBoe8GtdKa z#%LeY0c}o)ZGoFE)mP#3X4v2}Z(cvxq*d=NTrDNQQcCfQgaof;DhOXIUOHEW%>-(F z1reR3-)$BB+BoDIZ5{sZr2%6UsvTw4F=2=NyfM5BH&^!+=0Ed|Zjmq65+n!+1qyIq zm=f52fe)C`I`c$RNB{2Om?>TYl})4(cNRNMatjLP)vy&WZx*k?q-B7gNI)!Rb+=dy z{@cq~{%aLM30zrCNw>N^r-+b~?+YXMaCdLN{AE^dkvyo`?|#bH@w)5pdy?Vz+dUr0 z%&VFNy+fLE1TWb(brCk6EKK2DV`Lf^BaDHY#14Rt7)Lk{Ty&JgE>StW#!p6k-VhQE zlN4dRd{qC#TGq*2&4ekK7>C&o*g)X`dGsj@N1KqB%txtfQI#Y2a#2d_ zQ@&@SS4b2k0U@tdX@rxBK{qB2TMHP{30j|6nIIdew0GwJb$-9sNi6BwYF{0q&R9zrY7?V7t_AjYzYdHete)EZR!;b#5(D7vO@sof z#>l{c>*52``KnPtQW?V0v@Mh$_W6JS;>IK`*xyTWARtam){^HKSIty=HIAov>%BSS zXa8SV#KxzedqdI&FH_zjV0`b z^8+Z@>6Q38(M?fDQkr3;%Mv%&~zt4t%EhDBe#*16J+1*SH_ z3r-WPM1PL6M5g&lvZUMQfGZ08fcvSyejO6#Fdouy3t@=ZU#n`9xRVMnxD`qJ)d%nV z<~na4uy*Mld4f5sx}_tBoriP=ttQk@d#xF>$=$ixHfjpt^QO_+sS^O;zxuiNwt5n0 zl@c%GF~K>eOPeCBkkad0eR=h`7o@7OVUjHkhB+F01HU#~EPKMTUIwc>YRIp2h9ybx znNBSX5zob=ew+N@feUc^tzQU&6|w{mD!eSF zp^-E+Z1Eow!9QUIdgjO5h=U0%cM*#$f-8GT=VmXYFtNS{n_`MJ++WS$PWjoRT)T%x z=X+z|W@o_7jlc3^64w^A1tmOy>)vi$hI_np=TYo22f+Za@2}$~Z9WNZ_Bc-u$hyvG z43g!{e@f}GMdvaO;4$*9N$o2l^BuqV6Vx_!z;MDV4DZMsHfG;E?HQ*ZdE6n?9d7_C z!0Rh2Nzp^)w}(o0^c{wGZ8S^D!-Ln}5GG%|d%M7RFa-3N!Y%M39E~c)>2SB42Xurt zTNjb08ITq1WKU6=Y?n~f@E-nFP=qX53T~HHCKMf?BF~M(2_;TiL`j8k`emYw zGy%cD^|$Hy)&;z*H$u%G=%=6s5}e*(!v%%weNms|r*TCVc?`U1ex z2a@8@VR;s*O0{XI-BlzZIafB4lP0C*D49j)v*~wT%uw5)Lm0u{Qk?jg^Z`#{j;Jw~ zDw)@f^n7!V&`s)NHE9>MORbYr3Pr?bH8#?$k9dMVv=68}>8;;hU>@EKK*HEX zOg1W<1;8eqN;cM+cGgUW{s5QMEmhZT2V0T}2`D~1G0S)#2k-f0S4EG&+6dKK=J})J za`j|diCa14R`mqC$CU3^UH}InR9hmeQ&qiF4zgkHB6{uaDZ7bTp#hSFLe-u^Zh_CM zpE0DG_#HoKT#b5>c#nVcPY-}-OSWe4rLx!Z8*0g#q~5lcMS8wuAhwb`Hxx1~h29;u zBR{H_ojGJG(zTEeN!pdFvBJX>RWY<7oN}2PA(BG=X)xL&VZdn*Spbk>!npR(qZr?l zeUeXScL#+wTwf8%;hS<97pbPU^$GbfUD0o;8PmL>IYtvU7j7_>D2}_kO2q9NSO?VI z0gFoqlETw7k@MgipOag#zb&JhBeWVG(e4N|OzjXk;+x7M1nvt|(JWu>xY)+DQ<0@$ zWj}Z$5}`Y>gg^d^tq!i!dv$EH6sg7gjhci9i@l8XYaVd907{nZ2>!JLt#U;4&SK(}z}|*?*p*M7jDs z^!azXlrc&ztbtgDh7&Wu6;>e#3C zT8xi#UG_6uRs_?@!+*Ujx-7jBo+9e~%}+QrpK9&qg-4U;@Lb__D+r#?l zmU=Qy^?~wVttLt2-Tv@H^I~@xfEiZ=hc#DnMJ*41>w9j|^yp5xc3LTV=Sze!0is_& zrJn;BJNIJZOE(m=&R0!)uQa7-x#^YJ*XXQ^I~C@n#0g-?+_Ois;{vi5odIY=G=JQ; z2ZN);31qb_(&47fj>9aY*k#)dD%9_W;6r-0RjxRO9`CqbF2oU*dF3aBn>Yc3xh!&@ zvi9oR=GaKN507@faYd=V^T&z#17)>$I#&aDinlC|&C`8y8XZ}cvN^c3Rq9LbORnCX zBN?XKvfK^2-^X8sZ5$W+HUP>KT!c|-(#MYFGf~{Z{A@>}jAS&$TqbnfLQtk`;wq_M zOf}M^U2qolbf;K0tQ0qpP*pfCJ;?_ZsAyt@b zlKEu%&hZ3=2D~|ed_q1xPxy58g%^&p_TtaDChtDvHs-72B_I9E-9f0 zT(2%D^WfLP_6EZ5z=p60AxA+oEYNB9n}rX@o3k|)+e&Yt0Kx@B83RB`Blny7#>(?z zPFfKZIH(!)=cfE;U9b619s@p~-rxeolgZ7PNDK1%-kE4k6oNt22Y<+rV2aa8OS$sK zQj%zJ`Kb@9qS%tG;`ngL0k$4Fagrd+R{f2eBGRG^^b+ga!ClN<$24>Gn|Tzr>H_M4GOZWnxJUY8ZG^}5Y# zq4IlEeu|U+NwI|U0IorLS!Kq$!~L*if>&@eF(6lUYTQI<6f!q5KSQU-cx_0eb#Bh; z1!6W2?M`nLv(@^x7VsQwOv_=e{Sp&(h0|*J{-4OQzKN<|8#2(%zyiVUYH6hHnn7_|_Ay=f6Y(^lZN9n5d zxBnhni7QK;AOAwx%9%2aTZ8#&XOV0agAgc;fQuPIQc02-896Z`2WYXxk-xh2m*PV7uz2Qw`jSlxvr#gB(|wn@dK` zPic~`IO1n8?`PKWPJ9+g?G6vDm*AZ0JppSzmI$cy{B=G8yCgQ^F8MBB>??;9jbB(9 z85?kq7-s<3H=UzQQj0+|?XM$}B=)4pa)D}Je+kk4V+GapQBobn1LwfGB4jCuHfaUr z85Sy)Wv;WwsnQA6VZIIPq$%ugg#Uzj23_A_87K%y5f}&v*Z;sAiC+o~*&nA#hu;en z>3=CcZaT`#3gEtg5a3(k9|`gG>~ocJXl09nEG|yBNJuY<0tmLMW@aWy&&U37v}QcM z^;z2{koY-5d21*Ijwyaf9*eYykF1M(ODMxWndK2 z4daVm^LfL6cI&hZF=^)Jl|0IP1uva$w)g(I0ayQ9LYm679mGVc0A`Sh^!egss3X?`ey*XH` zpk!P{2%4HW;a>-`GJQGW)SfDhyKnm2*}CU;}s-^R;2k&Vd~eUt4ybJSnv=q z&3WU0b1@z8_WVT}C|-RkWh@cFfQ0~o(8QxQJ#kzR?%g53x?G=M@Q)m8)92kbQi2^J zfNVLx0%*ezxo@@Pwjz1be|N0AQ?aVNWDahi^wJ+Xs35#t1rm4pqi^PJLHlpntM?3% z?nlo*;KXEJ=me~w2pgd|)Lg&OHyuawkFjLM*`q!ywfi=1up=Jj=|gbCY9ps?pq zFCz^%FcIvxPGf7I&7j-5jYe0dkpCs@A9*H;1R!7f21s!|-%AQ@xRIjIXetgXdv40c zws|5|TWI~4IW8ut)2{Ip1~oUJr$6Bvi%qoBxJ?hGwFL13V6ud0KlVZW+vnyIPffWW8cbH}pDP$QF^*M2S7+jBIT%GeH>~)QQoIwHjY)%Dn-^CFjB&HqH82wemg7LW~m(DqtDHO00g7 z0O?4x()pnPl89sfDvXRHX|r)o6OdZtWIEuz8xkN~#M`LK4N}}6Ox_<(eIxxZyNrYlT&pZFc9Ht147; zywp-QFMnp?1|BS@&nhX7+RTh^gT7HB)f2*86D~6r6zc=6J%V~u(@AjfxWZdYAjnN z2TQrd-LM3LBK@USIT{Ni|Cx9lbcN4hsF6o-fvnb*&mw;}_KW_L@T321a_4~1E2W4x zPc9i?=5!hkro*&AMDvLR#hK;XKt|Ku4<($T&(0l;-9q$nxyxB*+-IMhjC@(DDtiD$;|KSM zMS(CC#TWZdnQP{>b(dE*j>BpK+>tR6<47+cgL7`t7xoDKs0LqWo&0hc>w(YlqKHF4 zkaYPAO1|EAAPl$thR^@}f;BxgmjVe+J6Ih+!11gr4>!a2tkhhuxUOr#L3p+65%+#_ zO+Ivw79gPEk zX651J6tlQoo}htjsrQ?aQ;mv|x57wgRl&>e(3SKKefWe}kd|sI62Zt{rZ__lhuXH| zq3sS)oWQ))Ewdk{-(*d;mL@BUXwN4jQHC+D;{qEjqNY@R7Bzhqy+Ui+t5FYs9&&8S z7r)MMXvk&Jwl)kr2Zdub0zm%@692zZo2``ZARv)sC`=8WAsnZ}_9<3Vjbz>r^#9G& zh3~!BhNvJQXVgGueT;vDJ177KQ#)f<*BotcBej)|Z~uwu?THx(5u$#osd18x5Gs^G zTugSEAGqK{BnnAcX5&m~78D}e<-zV1BdayR?$kA<@tpYW@8irdh zezKZ1_88jly#Ko57D>^OpQhK`FWEPKC%S+WUQYvouSa7L*5eBN=0kwu`JLs{Yjm8C zU;e%*frDTEVco0L$?Y!NhyIYD%N>QlYQZ5I0XJ9SzAC|xGaJ^J6Cc)Jr(mQYibB%} zGjDR;F)C;4gH>h?K7)njE=_(H^sw3@S(hG@@(@Zu zDJywcH}nIrx)@_x(-nYew$8q4yefK#D#q^JIkWJDbxl@!^bkuV68_q7U6G2yl38Qi z28%ebB2#XOSyN$6nS2SuynsTS2XhftrMgIGMolJ?qJ$@vCfsp?tS)(hMy1w*rkh3d z$|YRWqVicze>O?GQnO-WqeF|jsfR@pNVAet<&kre@+R{LVF1uB|BYp>Qe#T*J!OJ# zshBzsC)3ozq5v#0&@NTul^ip7sXhN^jBB3%;NBsWdZ9w=ZlEn}e2S`FF%{;LsCm9} zL?h3DiOs%WkE4$YJ8WVh$7CTc$%hTIHdZv?bdfavCr*eBIbMuin?Ajn44Kt-keL-9 zZZsq<>oXGE8}NMo=xz&s$bi@uL8?3Sd^9PP4bGU=uiHh6%>*1~Rx9G< z``8lMSX_1qCuA`@QFF?*r5Y_qrBvByEZbLC)#|r81I(W))u$LL(Oy&k^`!~RPh+4& zF_fEDwlx}9kjvEeJAzSAbk|4pr3pzF%Wc63iYF?NTEoHaW~tcqQn`8O0uUeuYcM!M zI;Kb$R-;y}^1014*usJ_V0KcWhDzx#w;_(WS&(7HTUR05GG<_0VUo=ywGVN=4yMa1 zXm|=&0b7ekbo1CO)?#zrMqJp2MB@x}CT{JiVwv*ff}WKZ+uK$Wn$#k~U-wd~M9nZbU@xwf_y;z^E+Ef!m-9kes<~s4xQ{rVdQ=FYIDYtVv{JC* z@Zdo-!7Y{BoHCNX&bPAC1)-&4WV4qisX`5N13J8>omfq?;wp@h#gYDeL5gWgvQTByT_YdR1V$uk ztMaP$>r_sd2Oo--EtnBxOX6SGQCaS)kV-{gd_oWt$)}v)5%73!H z0h*R56R#@D^ZVbxw1!zg%YMhW4rFAIK7{|E#=Tx3%O?rJ9bt@v&^c2y?3AqJ<-wl! zKt*AQk%DxKixbP5v0|~jR3_JxT()b_OTeFyMRV|GDm|KU;OzQ^ng%73v%PE|eyIG^ zJM93+LhM?}UacX{b4Y_pKyEg87r1V>4{)Q1c1;>8#>pjS3v-RILTlZ~0^0&tecxJO zPAYp=JtZ@9%~^ID1dOdTK?-u>uUapbO9Lsueb-8*54Z)Y=OO8gc9Nbh78r7{SLrk1kNF0<{Rc=W9n%O3RSfpdI8fWO$Y zv>K_}r`3s>_=PHYyd|m`6;PR((^_`t)i|u+OR!LqK_r9=+FKTnHQ6;9x!lPqV3Sc~ z(cDpCo`h)|WgP}T&tfSs$$~XHsn^>dky8fY6?_0@!G@Vnby= z@*#g*53)vqXRnGuIe!xG7)Zc7rVw?|2!Bno<2=HAYb5S6Dr zg&b}qZkr;YtHw7{85woO3DC!2mw#7x8bzKe?7awOxym_rP9ru1b^&E5>RGoPI4FO_ zYjcbJ`DA1czf9%FsZ^6J0t7LHO~Xg3^aU9n*o$5#W`Q#uUAWZcuT;*9G6TA29v`6Z zBy~Pwn?1Veu%9Uvc3r+ZQkep(+@ju0b7GlYWhs$p*-_W+7B!dwM1SH#MfJbDh*oDE zmd}f|uwC*l7mfnJ>L$R#{fhGPD@+FxdT8qPmrRxhaqjF*YlwG~0qMlMNl+CFY_!;f zCf%bS0WJLr4k^>7*~;obN80xYZQo)HU9p)}p=>1ofduKDUAktx#ZFKXSAw zipZpcS|Hz^vf)r*{GMK}ACd$5hhJ(2$1v*k-BvykXlg`^znr^Q7;KCpjEINx5i2g1 zhe*&z{}m5GMetG1W2K4)?rcOPY0q-{3{1}fkZ@tQ^Nwu$0o4Ts&w=q^qT+F1dC=e` zc_$=n$)@BF4wOW0vgL7!dFfenf4<2%gf2(o+YM8}xn{7raAj18C<$7&-hPQ3|2vVuf zhb^@epb+gT1~4vKR{{yBQ~{dS|zB&##nUZ!dV=ziTR;DtXna z8zQ#4-E*BwCOm?}*g#^R8fqFIa_=seexw)8Hj7nbqOWpcP7SGXl~vTQ3WouXgD+np z%)Lxl&CM=kCu4}W*Z=xg&j;^YxMp>Bc9pBdx>&|G0c<)U1T=7AF?l=wJY45wamAWo zmBZCi8kti$4<)`xPx3e3jgnT_dK46$4(ZNt(Uu9^xp$qzX26#55wmXNIyMa^LypIW zUXbh;gFVnq7UNDO`)$;~>L_?d@t5__{+XEGvyPSE;avq^5qLY(uIrU9=gJcBQlTVi z{Aop|2;jG!9h#{WEtnI+ER9frQ1N_E#5uDC5Gz0H*o3@=un_?K6?spqH?eIg+aJKvqrtjhOeT@ zJ!Q*TS5x`8-f`k7_mD+dorUurI?QHbv?Xtg01@t0bTJ!=$sU5^uaCLGHRbO9*QU*( zB?dj}X`C529F1vfo6&!GoNDFo@e)VnKWjwBtvE~v$~Uq*%PPQcjzU+5Pn#Jw^AeX3 z0r-1t%|(QPcxC04X+3oAcoBQD8sC^-Ud(OpZIv93*`1>I{@B)Dx+f1Fz4SsHzS#@{ z01!=Q#%%RKFatvl-zmN$ZoDGpnG~I;T!O zcIS01{yQAs*^tmE)!82T<{*bB2Vf8{)@8_2Ui0^p&_y5ECe$${Y(2w{Ay*5?WE*y7QT1{748ucm3l;q;=BuKdTpddHMS)bMe#XN|+M3rGY zr-GTKaOeQMllOz`9Ij7j_DvM6z^&0{e3_AcjJic7B*~G$xok18c_!}U{v+JN7wgz` zqOp-YE%5xjXKb;}^#;}(0=Qw}EE{*EIw_j&8%sKKQ1*}KKMwL9=Z!-};Kuc^TK7b6 zVvD)YWB0#QMsM_lXZN!?<&lYuGa5jVIHcsFzkueuEH&>yQ2V<92o7l2x5AV}SFswY zElbe<@DfDd6yJB07bT`I2zOfkH;85A%&Wb<-+!RP7e<&B)?U#y3TSwObwo6>u$R`~ z*q71tDCz3g-`kv0V{n?QzE7rby`QGTAJo>?YFe6U2edsRTfm%8+940FH-bZ|)0oj* z4Q%XC{${+6KAw0)wZdYwXGv`546!Ho_artoMey%Q?bW=`6`!Ca5!|1Fd*clZITGep z)#}U*rlNadLCck31h6Z-&l7rwliHk6JLMQr`LH$HdT9(93xxf7#a4~4T=17<%8l+- zte^*)3C>+X58Zn(T5u-g_v0TpVBC2*fbF~f2A!^>l?DCR9?7ToX-H_*RXe6>npPIS zmBl{7SdrBlHf^fVu)_q#LKK8XkxNs^PMF~k3paHfy}F?40W_)%er&mi31HMhF>Rc< z|0omXikR+Xu5j!$+Ix(Q@caGYigdttErMc$zJ--wd-ofR{@V|+-H%(~6PaOL` z_9OV?_?fmyG>4-_20&gRDIQLKCbl9gRf*A#F|Lj;reBiPP?%bc-Cj)EVph^PPON7| z-v-Xz!uCzF252gG9M?dr0)OIfPoJEjmOpRJw822;UOO^0p*@`?a&#i++JsZw9!$4e zPm3C}{Urli;qnVqU{n!>6+lsBLH>-w{6aeIw=?P16^-L6==z7L=$2q~wGK!B=1}`1 zkyH|!XY3Aa;ZF46{NZ3InuU;Yg5*Oh_0U{9C=Yqc0FX%VTCp9XdF>k11 zT#v9mhNYv5#d36>Yok8p4b3uqSPn>geI~rS;?xS=HDL~r{QRa3b1bPw+#LXvvqqh~ z)wTc7xY2)CFWRh-xLMTCo# zA1R(0ob~6Zdjk+4sFU2H<2&C75XkSs?h3mQ3< zG^L(7T=-IbPPNq-KpyS4r-&UZ_~7;i2Y`D)vhL_>qA1Lsx$+G4{-Rzu zp(=XTnuIG_u*hL?m_=;5m&##@P%WF@AEs`K?p1qB#w3&uO6pLF&j|4t8c8aiE}mjU7pkN3b+FxMmN11vSnj^1NshU zSweGgy!#aOGcdJT0~z~zpoj_KYaQAf^ZJuNW}r|4e$a#6D0sU~9Ab9SnI<~jB4JMa zTo`zUT@64PNg&Z^a@iTA)lasMxzcNOZq2b{r0lA(YiqbN@ePw5OR=jv_^Z{b+0+w$ z<>5j&`wVASXYQLr3&*>DP4@X@hty|9e4OPq0en#W?*?ZRp*}qHza_Hn8=NLFb6Bu{ z+h&SZOaHZP7I2Hx#Anm-U*f-YHz1rH?*El|?MTu7x5b5P=)WERkF^p7I@|F9WPP{9 z(D~6e676XH%R?!wVMzSRG$NNVRmMY-4oa1rmI8sv&Bn~mrL8*JETr7Tv>YFiuSoT0 zKeXE4_Q0M$s06OZ9nLYEgrdE-eNS>-4c{+r48K8~2zHMW!_vmM^;Vo``rQd`j_(Bi z+|N6S<_Iv|7Hpm%&{1`XgUOx)z7jjP*=`$x%-#FU_t3z|!P*e2kDQ_M{@mhR)2y68 z_RcF?U>n?v3U*B7&!BVSu#{M7GG){KEzq)O_XyQH;RThhc2PSXph)ZOdee`b%Kb?p63%zK>ZM|s=BGHG4pq`wE`Z^ZI1uw@THsa%=oSuOm`l#P z<-dmDtM*P2dpEXkcy&Yp4X@~7m*P~B_If(w|9zRrdpw4>U&U1v;zY=jT_s^W_{r3J zC95n?;$1)a@`0=PL1i}nb_XW79s7L9CTo^gCq}Q0;prE~DnKQnxu4Dz-zGPmKhE@o z`s#rE3m3^@LZ^y&(CoMiP-*3nxP2yj*L*eRflbcQhT)wxWwW!3#-Tk4#+&2;+_Ou8 zN;C00d-|G1E37@tja1*(oVTUu9qxjkX1z-}=x>$hIx`*+gSeomrTR>e&26?{V{a&E z&b)9~IxjILVrFjYz8t1VuyW_4j>4qDU~p8hVl$pA*35r5s+}~F^x8+((=19b zi?wh0E-8KzDTsv>D4|Vc#$<}ngw_gYhPio^tA}&YjLmG75e+q?P-37skEmCwxg=Jwe&UasCWV+ptLu?# zWQFO&w2MY<&UWHv)a*Rc;6iY;mF`k4;)6};*MuA@n#~uQ`-XyZwZZ%j zE)FqGf`nyBbqk|cqE)-V=+GKJq4yJLY-4Yq!6P&y5&2VzAB;N`$wycw|+Aj z>OzhN&%K&cCGt$Gh`MBLLQVx0T?>&@sh@AwBuI)5bvo%?{~L;!AY(9$fi!cBu!w|N zW!L&*hBW(=5yF%pMi(u#a$d`jJIy*rUiR_}>_3T&6!N9gj(iUl8hvLJV{&?m_Evm&6&9IFg4jALw`-L=AgjuT;YgXfV+F zZp^MfU1wK1US{VEcL5+?Q9U3J?M=VBK(F_E^1g;0+f%bya$n>zZ_9I|5~7I2>)1x>lH*2^+1DbgYuU5E(@IBoz4U^i1-8N9w z0|czkn1H50_+);*O@LzQQv-nsRM(FyT22O~@1jTh8xivC{>@$a6ods-`g+fiPP_dO z4r@M)R~?onZr0W3%hU5FQncHM#Rd@1j?GRx1gpVAz`^}%#~&}QpAu`Q;?2Px0pEDT zY3JN~Jxb(APC1@jpz{n1Dy-+K_b>}B;*HzFrq{GKd6uWj;*JVLsy=5sB1Jwg4KXQ| z!!{#&Lp(JRTkHJ@KA_45BLn}>n2a?jUDZzr5D+pTu{#G~OBGEM`;GfiZ0 zgNQT;8e@w_y+$n6JOn+cee)*61aY;4$C*XZyvQkq980C@Pw z+a)b+(%nD3(X{*YqR;s)@cGFcSoA6_0fNS2Fa!aEC7PbhL~e?EGkGhV!bEDUApxIe zrZ?)#@+Sd+G1eV}jA^DfpzhrLQ$#_jHq1$X5DHa=g`}O>6rYpaZ#~k$k4zZOSb2Ri zx`+so!B~1U0F)@aF!bNZ*Cbm<4s)|hMjB3VnVYJo0_!X41TFkmjQSq@mio?7Q_3Pq z+ewRf2$k{5r0poi%<715?5Ss}dt;&+eOEferp^Fh@2sn}wQ=z3k$cKz60v96sx!l# zZEOFTOEjOZ79K3};+*~}E|r6=DSeiw)d6#iKgywm@f7l>=iR^X$#-c-I!)xuKv{!o zeCD8GaeCXnlqrAuka~8jK8B#%()@$cWO<3H189)0il-&gAy%%BIur-F%OKs3`GIpy zrwaiH*9r-oNrTVzTqKMq^W>y-urJB&ma5uN9V(OVjdn@t<1eadH$ioC;b_P_p^bCM zFo$U?=aKEyR%(gLF#OYsX9y@HcdwaHilKLgwf_5L3Gq1c&?!Gp?;ERUQ4^JO5-c5Q zqdAq#ly3wQ0;_OHFgd8f<%+SiAo~GWhere8RGFY3lpYTwOhyd-D>vc|-DuJ4OrsU^ za-uIRa`fgW%6hd>Fb_mSjtkI5=Td>P(QVK*p6B>O()#sB_Md*uyoZ@uUbI} zlNCkz!Mw27QKleOsH^CH!HO8$L;jNtEuNDXrD_DWlVH@Sg?p@i3NJR|K zJy{6DecSV5pGDSi^*@EZu)PCD1bQ{NY|(1)phvn8yZb@GH+O-hmlzRtd%uEz-jWW@ z)6y_K3e*9t?_0GHjxViqBy`ARQ}q|E8WkyWb5=(0p3XHK7yd;4UXjqO4hl4-pO#PD z6>D_1DjUtZ;86#v^O3wj4(6aJQVDbFmUJ_7V{(J`5$X-k8WBX@ zAoWeqVR$eUL8RfAfX3iOWa5*T;AHPm*YN^_tsA1>`JQOKdh0jeU_sfAY=5B#5TpPO zB*~;IlCz%WupLnfCRFNf_W1zc>2K(*qAwbs!mrynnnBYbE23^-F)6p9mu73!j_H6^ zf;Xn*ssG$c`8VTZt!UgkKr~r6@VP5r^|aWPQFtc;@v$H;fGdVH4qm)Y<%821nLMgT zWMP*S0cE=wONgG!nODps-BfUp1RreR@e^>X~d$0A|k5^?&`{wy?r=C8@b(;E>5x62ERAK_f>IHF~6{N3u3-{1hAfPeMj-yi%=~?5it}vvQ)|_maJk0 zN+6#ZL8yeH6E+Fj9cjSLDS4buz@g&ti-F*gF@SkMpFu|;IL1siyE){~ewpmwxGM+! z{5PrL<{mS_$7>Az|4CYc36vzJe+jwO-~WIL|9+8p30~ezfDBm)VWGTP5G^Z+#9TF$ z`Q>H-7M4UQicqyI(l*&+Uhcm{JCuhDT{*WG5NQs-r}eB|OOU-hMpSU9G}%V)|7`)P}J zliGR2+yJbf-<~rW5VvRb4gpdF%30fskm*1@*wkbJ;QZ7wHcj(>IIfqa4$IeOy8>Hi z82mXlnyt}v4bGPIo3L5^HcZ2e3=lpDv|YZ)1fjzjx_AVb6ck9Rrgz8Nzel}`Ou3YI z_Rl8MZL_URRt|h!Mzsmqd1+b9PfCfB7=(76nuR9ohQz;%UaNdzWnHrT?M~1!i9`~5 zn~rP)tgB=q>11UorK;*VJ&-}FENp089R;7nogAfy?b>=wOre>TTqnPk+qc2ho#rOc zvfG@;r}UfXQMnJg`z~zfVeS;PHWa38f)5?i_A#|5m}>xGoesPHy5g|Q{gnkSmpjp` z34>f=2#b%IDm*5@KaKIY*;!?{-3v=&p@EVOh&j`M&z*7*s$O?_Tx$zH(5CiTp+rbw z@iE#}{#SEg+F8A3629=4%jLh^nx)u3Fl9~eQa7^yjYmU0=Xtcu3$f~GhasL4i?>+R z(+dNv?aHN{fIoB6O>E^eI?>Nkzhxh(l-(`Mbo80qD;_Nbefe)-Q&4y+Fzu&ibD)#} zjQN}1(od?TLt@c3oF`NBDHBm2s!#KT1RrDTjTG0~qdAx#=WSC#?}7pXPchoRk@77q zU_!g|7$v^#GD*C{ziAiRg^ZaRAd@jS1qhfMfk_K&2wo+`o?>l7q@LBozhG%!aTz}a zB>+fxr{o87Aq)5IvQj!^<{+E`&gWAAvh5a#IdqI0+qNIpXw&YHd+0|Qdv6D1sri2N(qQ0@L6}-Gu2C3h{SrTE!<7l5d&`PJGqoUhiykNGn(CrRhol{9PDE z=;B3t2PFr&GVR$gx6U6%B48RbC914_;l}6+e!}*}GCQK@$2d^@A&ki4$Q1-|4GhNR z0J~!2Kv9NqElD91PP74(2x-cbP@Tw8bIHHR`GHX7bZQ$_vDq*RGw(&&W zvqTC?3Jkt=_6l15S5tCccV70LZFP_EZup&0qMuz87YI^Y>!e4H?dUZrT4(I+QkBA#3>S8s4b&n>kn9GW3H}*|)bmgodd5I#J4vQT6k2h)m2RDc?EsAi zOHxG_Swnir86_`c#!8{wF3p*)7EXZT!%ESCcJZxljK97s&KatWVwF_?AvbQ;=Ea}= zYkFxw@+7-X&0?c@8{EK{O1p=;53N)5qw~k#QQ^w`&ug0s5)f={j-$=a8%?=10Ys3u z>a%s2IiMbmBou(nHu^ETaz4^w`GImUbk^HN>Yjx_jT?PUuwpRs?_A7C&lbRO+I@IY zYV?(jsVX>5%^t@H<0Ez1C}R*NZUat^VYD$GG#46YvV-mjY_#d7$*J|sr_xlt^!$NF z9~Ex$mTcC6v#CeK0oygh;07!vocV?a*pagUD+lPZN+a2bBnf8yPYpgs^mX=ZGvjQW zWUzviB7TWodV>CvNm3*l8#N$&k&Sn0Q-wB!L_1a`^KfDp18!=1YE#3E+BcG%k z{Up3`5eKyjA)NgJX6rovlFji3uDfX}#@!(7oL)v*tzvABht=ZwU(9U7cdF$QXAAzR zpCVA$PE5%=W*b|!;AwH|VJoXRyosZzTv16xr(qzisbl_88;319^cP@`dsEz=HDY9mmWUEmR+sAha7B_^I9;qE4)YyY%M3ctUUGp! zz!=@|&xprgF}zW5xn0`yf$|1yyl+xk8E0B$@mFq~cW^XGUDwm0$(LC?f_sFyBU%h$ zR~K>)Sl2>zz9Hks6nZO8hc--5i5_V?g^`Fe91A6s z0vEB$iqC!^8c!HpA8mfVf5rUez_DUrkOU_jQa7`Jw)y98Zb1}+Es?@1wyG7t5@ahC zqN|-ekfpdGEZ@XfvqYi$8X`u~!!M?P?)m?qFCn!m?_2+DTz@EmfC&F12>#o}HKhgV z^B)&i_oS0O89N~+0YfMQmGm6||jd#nN(B z_0s%uk%cLU*brTQQ`P>wtoFxq_sX;8&iA{X?K}U;_F6pS{LA}C)8}N{iO&i5$wo@T z>-tTOcU)-l{mgw?Ai(b=FXr?GZ*rKCAEE!hnomuF8?-{-RBiFVxK4EDzt< zfb*LPJno67Q7ZlHrxC}Hu&qyde+gx0uW*ECUI^M95q#G`1YpQ54P#uj`gT@ZhccLo zQ6(Zz45kUsD?|R}7DET{_&yC!YkESLZwaJc zYY#kV=2?N5TZK>DXC*XG@rdkBx5BC66)hB78%2+NRhK?kjnM{SK#Z|5EKe5(7s{Qg zQ}W1}cX5DL2=Iq4=@%Z>v0ak4%68>F8ZfWCMdOO_SEuTcE#5n6PR%_Sls<*PhHAHj zscv2h%J$wK-?jQ0f;$WaPw8#!xL}fdd8hPI?D#;yNNp~jqNm7Fglcp-j`ErcXZ5|E zqeD!;H|q9CVN>vcSR-ZOh>=*N<+qlxKupMFQylM}0zl+u2PFzdEJ~x27K0gSkrge(-n-1Fgo!W_@uylt&DNde!{nSFxa?vd$?ka+B=9$fnmhHUKR~_u;ERx zF6qL85?=Qz;~+D#Kl;;JVAc)qM0;Bbb_Vgq=qaWY%w3B5d}APVt&S-EyH?HJ%ngia z5&(u<9%9Uul;dvP@(^SKA5gJmr{^dVRF#~IZy9a(>s8Cgw5J5Z{^cNG<(rKl=X#b7NhZV)q#9zg@8is9FQk=Zt>;mWvOE(6-p7P`;-NEL zfGYtsz-Z(J1+s)y&lHYS+|}Hu9iWL+7!gtO@G;u+3*IVzcky7;k31%93x$8~E!k6;YSjVl@fMv5T0GfuLMqyIBkY zZX~1wrq9G5lkrI*0}+oUgvpNSJ%M2R&te)4nQUAufjg06?=@2p)4Xy*1p+*nU$fjkxW^QXUFCRDqeEP(CG3lk28 zDT)ctzv8`Y*n56+aA+FP9Vt|j_C*HQ5|iic_r!HFyXh7?u@*z`?+A3z$(ep!%Cgg+ z$>*eG>W`gW97m8M`*VE4gWTn2NF8FkRxaeea(iD``XejWj|{x|bG$AJFf*&0S}Rp6 z#=@;iHaPRI;N6OQ87$u{JAleNTAzyHoV|_I;uDG!texo=%TLXBg#+%vOSx|xtU1cT z*;BRmNU@f=Caf(v2GcK;QE0wA$=?H?o>6?#?iMcWTe))$E)aaICh5ei#6&Mp&Q?C? zJ5ie?xpgcU@7dw^hsXK|9pZc5tlcp^xp!Zme}>_OxF12HtR98m;{e0(cLNeXV>pWx z%w=;qX=Slc%*-yw$*uYrp*^K{1)r&6H}l?DU%`BVQD6g0^YpCWNcgj_6Q8J2JygLX zZ;(+rhb1iESokWhmQgu-HP)XrJLxQG5+PY-xyJXTL60|c#^1RTi*v7`mX9kC=OHL^ z7Chr2tbF}5jB)+OSOESRhe-55w-bO+x!9pxWcsiN1Br1vi6o}Lj{pnP<;trlA60M) zLeMubF4tMjL&kE}H&fjM?BU#=M%(UH#;9-_WA$B2;EAs`3_Ua|4swI~;4Kw+<||yM zV>nJ!VTxHZG%IuJm8h}n{MHo*1DZbi+lOa z+eoPXX=^Lvlj-22LAjehfNZ-aB=0eGE(E+UXZk!qqQ3myakNLytGEZh8xQt66gm^@ zr#RN%hdz2JK%tC3n+T&+@DiHi{tEZqFI!N0Et00xfTPPeR09ucmBZz_c$4+L?KI@%w2@16k#7@d!{9OJc`f_t z?|B#p8ato|WyLO1gUD}UuXDwL8#9|ZEW;j|CwMz6_%VViS|>vv`%vWWiwHlB?+2z+ zf=#>32lWs)X)@0~oyj)So+9ssvgr=?IrRQE|Gk#}aXL)^yr>2oJo}dIRC7)oIyXyf z11euic135!)jp@j6(pVqq&t}$XmNUFx{>!5t6I6K^pmRVZKI4dGg0fEDiEF&PkGv= zp8?QRL^K%IvhEBJ4?6G!5{hr)tyFfBYWqOv(mcy7fg6KY3*?=QY&Hn)9nT_&E1^0P z3skPhAFHYUAVP?Pr4{rjVPP-Lt1t)WfOBN8wzVpiFVIgML6(1^f=bRIx1q)Ac&S3A zv^NIpT>DXkB8fg9UlKE|23d-jXBqTmw6fF9R>E)x;9^=uUk8ouYB9M{R_9Ny#~(Kt zhbu9U)4Wwse93aVilNF%*#6nH@%Ub^iHC;p5PPQdFPWhDz@OC>+E35}GDwIa09nrJ z$stFB?R?s#HW}}_Ibahn5fSDl`ZC~adOR=ER=GosiXS?N^_uyJKHOQ7!Y_!WrP7kB z2U#!9Z=mBh(VXE{X0IyT9{=**#Tg1!9s#4SAm05Ps|)S%M;Jv+^}xGVJ7H6I-0`d| z=7qemuk-Lb+I{#6)v?~4LDJ3!;EH}b^F9Kd?UtYZ?xr7WMqx4ns3YWh$OB((V^mho zr$kl7N%hHaoLeSZ(TR^1-WRyww|+jegAL|ZjAl@f9Q^Fx`x?&$5^NgtfH)pbCT0pXTKBu$fCu|%rz|8*H9ATbl>AOl)>WMT~9Mb`Xr z`>u-_onE5j=hfl}o*<5xo78a$V|ML1$-rdjQ4Sc=teyLklCn}8l6 zP`NE8{O{DM;P>T94%<}jHRP$P@`D=F*_3{{&aTA+_)F;vMIS}IWSm;Vx0=OUqU9uTG~Z`umJ?6Rv_BYd>#AOGqV^6S;xUvl9?;_l zalG>&??QH*TmN3@B!yTC2B0msjaVq&V9;KJ?bRz%-2iUekW~~*s)sG+2wk6LQ~ePOd$GXW=Gg@zLiKhxfp(SexM zP=a=DqyU2=F!9oD*(JUuMrMA&3b@FfkgBuoB}l8=w&=KXXVTNlbK}M;F2NI=+-ojT z1e^k~!&I`Hbx>AlNbSxGh6WRj8`~isL2RFK3R@m6cG{Pq@YY%OtJ8&903vP)yhYUO zy^qDmFFsK}TK86TKRFk|M?XX#+23mTQw6{#@c_bdp(sr$?zkj3LNDRl7sK=5qUJe> ztK5jaAVq^A3-&Y@(ji6b=l+HlLOA*&@-rrni=#)|ca+nclAK(ou9)%1mfm+Ts`=k8 ztiBQ}e-gfQ1Xmns{{`|-#q=cE@2J|-X2eX9@Lbq=LGa+kXJz44iUc2L!663SmOFP`K)+vO(KQr?x@cacRz> zt6>uO;uJDLLr52m^NtKPhl9d|cp;IR82}UeIw5N@l=nUkBo#+G&<}hChDhPjn1VRO znDSTOnk;*|kZe)73cP}5y~S>~;aUG2xlfz^nYPz6tfJn&X2$cgC%bB#B3g0N+>g0R z7xSa+4xqLSxuA?q`;;nW!l#Ny<1l?#bxtF5LiVs9`C@F0e|pXh&kIQmp{yYf1i&(V zQ5x)-{rs#JPy%Y(mBNCFuz`S}y8Vz4=i(olbFSDv8;(m7;=-3nuKe;V_laQf1A)Uc z&@3T5WPv=^9l6o_L);J7w`>=iL1H|T!-gkv~_U{2L@E>tj7HO7jj+m1gQxv zqfbw~$Xz`^NMQ912^N2FER=&Y-go}wP-oRu&ayf}^gFQY-r!NT6xQ%u$l8;%J2{#v z7gOj>Q4!vZSaL<+yCK)#8IegB!4abDLn`o86+$kr<{{62t@oAs-Fj0&7*H~|V{i;= z!iLnl(-$B-TL;zwwRCcGhR<*zl>Y{1)-P@+dG?N0rvA7mLG&Z>1Bw?EOigKzbBH@1 zg(%R2!iazu^1`7^07QZ!?kC#Q%=-pe8OWGnQWZI_1u5G&VM^0>S;{5$5GO8Q&kMm? z)7DBSXx4Nj_NJs`a$C9XWWFGo*M$~Q>_N%BKZSt^_CYzJo3^l_&CEi;7V zNpRd*U8t? z>griftx`(7Z#Yt*rZ)%kG^hj2Yl}}kSN*Q-SC^G(Zp*4}kYE`c4xo4fzBnC-I$bZs z#ooXV=h*{U@exgGWWxm*Nv2)1ePDT%MdF)&zW*V1^GDUH5DV;dr%+67!R+0Aji#vp z^#guVr`4SEi`?wmDCs-mdxOn}wws8kNW?OD6RDMAqx~ zFafco$iue4rfilb&=hJzf`f(}vVdR#7RnVR>liSLUrTd!OBUcQZMM?3zermC=%=C@ zGD8k$+y1R};Ty4BVAXYj|ExUp{xNk)){#LElgmgJ6+uEv-w~S_I3~x$8b++2xkq$r-B&mtowX4?+^9k~;-u6R!m>eJ!^BFPZPyYk zJD_wh*qT$DtOjp=RivSeL%mdnI|tMMZwh`TS>jB*F4;)Z+RWiHqV#sVlrn2a12^bAN`nVqE4e5tmObPpFgJN3F8B~q}h%`*;?NrQKX+VuUhztmENWNb; zfu($fscAWAfWjLN44ApQ_1yjd=`P)seytCv*{wk8F5bQ1DA~n+Ee;e%@qvz^dSxRc z@e%KHzvJ~6P?l=K1=|^cA1&npl5ppx{>%%+r+j5}{sQGsx@89AAE8Id9HB@3go>g3 zS6cfO?6y??xJw|It+NU{r)J%XBeq-`lbkw!>hl63}OtX-Ilgnt@hlBC>$2=o*CqjuW!2br+i>Cj8T<7sWyf6c6ezeG+vdpz>AR z9g70jQ>)6NwL3s)O}TB?@HngZ--tFzDv;6}q)ltOxMD4~ipZMgsWRJPHJT-D3mKJ) zujDqy`!X?vqmOJMb&x4no~1wBi?Z!zeoe8aysIh5d%H~&bGJN}&=Mb5bf}mE{(uw= z8)>D}l_$BVlA8H~p>{AvOlK-@Oj@Z=7B;6Qbd5b!8*SgW(adP4Pk@-b;7D_0=^pV- zHAB;f>K>*Ee_LAB_nu#9J``(O(*+NHGI2pjQtDv_o`4<23PHjRx8*4?p(iQwyXs5& zs9)8&wnq^U!3UZa*%x$bSLCn)a*v!hElH8+G$MMlItT8vnvFH}3tuzv|F&e|(qR^Df<$WXIJ5T&yK$(Dfw? z4eCoxOi)`Gf?}}y5xUR$v(pl`{Ow2sIM?C{$itM?;IR~{ z;J=&+Lpo?>64!D@MPxB^WJNSpyusxQ84CF_uN@4j-AQnhwcp-%q_K6C&!L{hg7`hX ztL2YqvtM%o%1bX?)rUlisDuO;pOMYVDZ01Ik^GeHVMavG87nG`Hld0CwXheyDHv4J zdTizmPt-PkMepeYHgb+(gAZe5;t zzv56+=373zS6sQU%jSw`^8Iuj!dR9S2>nY$sq|Q|ePeWCN0a9WO%%9;CS*^(*Al|e z6r->qZy_~1&EU4nv3KDnv?{q7xNtgLAo7Iawt(D}ItpV@z(p~pWsbul-ke)k0!^I~) zXk1Cb?TQWpiV(Wo-9q#8=H-|Vp;yTXN%B4{C#%{8b^Z{OJnHv?Kr%w!mCUhG1un%8}+tVyZy1Z4hBFtWgLICF(*e?WkFp@B@e z2U?ufgYN9HTfg%E{Qn?#{+Rraklz1jW0?PFW3>NQmFPSp1CkPT|HG*X-mz@aZQl~A z+IdwilF)|IhDt{m!jKFl6Q#FG-`%Q(F*URtv#a*5uRt+YORyk{YsyK7DUq9RoYrUI;WL6tuN@L# zSl>Rg<2+#r(8P|W&Ylg@y>7*ow}IL@fz^tGRBP>AtF^M7v1GGI!mBo7&}aQ(bKiR6 zHN-D%r0jG}`RKmIT)sD73bLB8*6@eC_&X949cUhuxJeLoZgEkCOU0z>n!fLP;-Kwe z|JGcg(`0kK+;%d`(^-n|CVSyG(oAWnF_m-eka>OwD3M#Rk0H!_IwxoM@|I)WR>qQd*-Y$0=8`WcpM{wuwCTG z{@;EQE~9b%9TXs-P+A}$g8zO`VF6W|kUl!9OZ?;wonw!#(izRd=-?LXm9j6UO5v92 z^NB39vgixtxAC1MzZwGB?bidTtVF>C71vA@5r(LUlyW7d{{#?lhqnIxLzGXHL7gkm z^V4Q;#w4j{d2wT%@Uh)7q51O<^=^;n`>z=hYQfioT&T;Vm~LcACdsUsP6-II5Sqs! z&r$xXGQ7Z&eNFiN)+Gpa4};0Am+lZNzl&VNNjb{gMk1m9Kx1iUt%Fqa+8LKSradd5)=H;>Ds?sYu2;p3^Dd)H48lez#Asbcgh&Lp8-@NmOu3umbAQ zHcUQo8&2SxDIYft57w%dO%3>~Gg1MfLr+{G`mSucpH3iq91K*GbYh9c4sHoqv(QQ@ zG0_W`by2iNFm)zmUO;0DcT0A*R(XdO)dK!H0Bjc*Z7ZlXwo&vZ5u1GwWA(`xH50qS z{MQ8wTTpAF3abUovP(gK;tH7Us6aS9bT~_JeNq=o{I3sM4y`PWgf^gv9Ix<-hqf2e zldLEVv@K%>vKI9aWg{r_7&m`@8t7%TK^>G!{Xkdn9I3+XZbW=e`xDY~df_!eF#Mq; z;L~!O*1lxoa>l}^dpHMHNeWHllT-7_bM7^Opo_NqH2&DOTj7A-!Ur*7J62*%j7Dw6 zv^lGa54l(somq*Jc?f`_rP|Z(=6q1htj5Y>zrI@Q^iUl9Bx`O{y44t5R(eNc9HNNJ zjFh!B)r>0Zvdj=wEe+Kb*fkVdGzFwTc^E^=%`F>YX=OZrUU}UZ`JAfRwZPQgAzRU| z2y0}T%iG3`D2;U`!@NnxRyiIConkj|T^<};8TAAb@6E=5J%`X2aRc`*s!M!vxy;_ zDRXUGX=T8`g#V7W6|;#j6g}czIBV+2zuja}pN_5}^sSv@g5A(Tm0gpW&|zIY=h>T$ zwlH63y0l5J=hoeiM2eBLsfSxAx~5lsmT{}r`wIXY1e6Y zYDpc8+$QQ1hxG%r8CH{{`#L>Aax}OSrP8z?83vEQ4FDCaTXTf9G^^e5V+*Fwx%^N& zZWO^_j-X>MAp+wz(%On0Enc{@kQUY9Zt0ZRjXby@fA?`#f<@fd!O0gwr5l@``#*wh z8BhZ{iGddq5tXV`;zJ8EYzwX~UBz1Z(}q(lK24*ZEk03;CAnp&*XJ_q`=Z83VqMo6 zs>G?T!T~8Ta^}Y;WT6b4_%IcdDjBF(Za$Z`F$Rm?ub7A)N3EywNTi?49>P%oAw zsqI#q(|4l?-~8p=sgXRh8QC=?B`2sC=2C?rxv69^I0eNf3HNIG(3@B}e=1O6A!{4w zw*U-Y--&Z@vFGu3t#JaMXQ_#eOo}CUMGTfKO|0p?Y~Z@QMW!3MYD`Ne5VuepQ>m0^ z%T9IhWL9O3>v{yF-5|EQ3L{LwmyjD<_kq71R!;w|8_VKghl+}Ek81FNf1M1peLBv} zg^4U$8Z~-i%6Bs`fg>+Ea}cA`mDn^ns{+hiq>xam6oAx^dAeL6pS7k<+~bkvsfQZ( zwXIchdKMC-J#y`2E#8L@gwgGJ{yWcK=eA?l+nq8aEatOJZXgulxfI>i0wkde_r zVUN<&oKL1r&t0~t8a~c{%J$ncc4`}xTAjw0G&O}H0%z}y$pMM_ zA*PnJd`wluW)(D5q(T+v8Wy(LI-M$LIesV+F|5kyk2DyE^)juCILt`*1CA=ei=AyF zm2_&qGnU}u!j>jGIhkZj$WJA+zJ+Q|UFf$;7H-Z&v+xQ-T;6GX;{-O)8GZy7$kRX;^%h5C7dQc&?=jd64n4y!Lf@>k4QUP$pH4c<| z*b=8hV2A0+G}Q8vlm4EDTrqz>ug!vSHz0?|;9Xrd?%MNg^Xzg~D1w1%>3*Vdx#-M+ zKlLI1ccbqNKh6z6#qDK*M3``V_ekbGZ280t|90(8xiQxAHi;T?4JH+FZV%B_^S_UI z^~bEqG%wQL>u`1!T9v=~%$NX?n>p zP*t1;u_4x}9Ugi+UAo0uLXszkuEt~Ij>(3_%(L-y$-up7OIkO34J#a@w|6AaNcC%u zGtCi$g$7lHBRya}Sp%NVK84h9AiyGx2f~HYE$wIzjv6z>*M>R^2_Vlgw27lglc`S- zqPQd4Pg)i)!j>0Y#Fa!?!hJH)ER%()s0Wa2s5kqj=qGbBa&!gB5u2To3ehZvoW_9$BA*}5p##PSy0xYp zR8vKrEi!hfS#J?H4d557^dDE|`bo@LelrJomApNk!ugNQPr7dvrO3+_}$(b15lb!#a4#p}5jFh`uS9{IPXP+*a@vnrg< zVf7Y48R^|+oxgu{YGIq!6{%13B;TVZC2N+SCW{^X0lH`ggTS-Z)qfp8stM7>r9|{zn^79JftTM zwX8m8ZOW<@#n;AI|F%dD6~R_x+#l#HS-Ml&K+D%<;gppmFGpqObo?pj>2$MO_~ph# z9fBsi1Q_YC&Gh@iwK7*AFGU<|j%lo|cx_wv&0f4F?+AB&JmImY@}jG}@Fy47pgA#b zsnt)DS%&~F@WzFzH`}PSx$D)S{J@HG0U7ddS{9grdD+P7qZm75=67z!Vf#9Dy~{Ys zrHjK+KR1$YLvXwn%)C_DNS7L(Z>GHCvyt@c1f=rH4bvx1)GSRWEqe$!1Y*rCmfJ)@ zdt+S*EJ})Z13)7j5sIfJNRUevjiXr-q;%=@R9FfJ=ZWmP3&8Eu8neU}#lHNf1cuD? z$Z|4Ma)Y;rTY=hdg`O7GWE_DvTG6LT-PSuuxUcsmdlEDm19ay zZWe1AhjNb2&et)Tp1)YBGFZvniK?*Uf9iSePg?IGOyDi!=kCcCT`|>|i3UaL)#i4i z%82qfejn$#2o=4_VSAOaHBU%XNo$$;0wyB|xD8mT44+riX7=pHj}T$iob5;+m<`$ zK(6wci0&0b`6P3QiscU~{xrNjd=Yi?0l=X2 zc+8Vulnvj2z_i1~qDn$OBzMFMCa9m|)b!e7K95vCEGG4a_M|Ohi22t&H8f?BE%{6| z1eh%dvqU0Jks=ehSp3C9txosopIvxmXU; z$8T~OJuva_f;)XxQ0~Tzg$8530ds^uK(aQJ@NdH1!>fbLl=Dtj&ahKm ziDU~dZjVbY>eeHYb_TICLO!mtVtq?9{@1@{Ea@-6ylss=h47@MnlUoq?afsXtC%L*4)M(|)-K z4W4LT(m#iEzHACOk1XWHwygH-c7C}Si9lQWy2QVDVWHkfWo(Euw*>pltJXjH@ZWGg z7S?9CQU~HdA7p|6%jao50UjwDZ1ap7xpSB0CQrd!rh_8f=k6XI(`9pVrMTcP+y+u} z`pUOdVotC)njx#Vpjw*y>$d)A(*wO&3S0?2>gInVyY?1}vP5*tviEZk&VIsO`BQe> zed}Uq71#pJy)aN!ckrB~Pn?lqU>Sl@?UZCBUFHUf`ne_Q(AM<{=Y> zbI*|9CiVpNm6w3ir+Nnl^ObJ4RF?P4Y_{c(KFaphB2|bSbwl#X_k^>5v6TD)QWSjf zEBAv)IEKq0`SoC@3SbJQH&Y~Bm(F`JoqctJ`jve2#{^dQS5-~X)1Un`pgmG4eNw|| zGhC?R%5AvbX0Ti*vMtG9wQFUypSl=N^II4zNv5Wkc#&aO!|^A&P}VNDt?rg&mEuG% zzG)o3D3OD_uBz}#ww|W^Q!ySDBn;>}vJZdeg0v&hETyW#j{#`(TxjequjT35e^eEj z_px(HX$kfDX?a`OL%75j>;?0&Em0I05%ExECIk4%Rt$} zlV`s?5(<@57P_PH6v7m@NhF{bH)2#7c8ZnAf6zf6w!R2>6X$LbvXR=Xe_LQ0(Grc7 z9#d&WkKB`4bOCaubUbE^$U!4~*=He<{2sDqx!1_BDY{~|iPJ0PXeT&bBsar* ziWo#a>I_jPJrO)07=@SOJ>23J zNYJJV=UbF-x@fihqt^U-Q{0XwKS$-!Rf@+Ka-q)tvQQ@XtoQdaYI4B=wF)n{iT0O^ zR)i1x^Fq6yRX9OLO-L1cZ}2xCR~IRgM=V)LV}6ueU?9kp?_wvarlL$|nI*ie^Qc`P z^>fJA;~s#|>R$!FpU2<^Vf&}~@4k3A-?}(HGRFR#xcTEhIDMWhqtY^1qb4uC=(&SI zYZO_YFUGPT=(9J>$I(IR9SOlJa`>!r?WsKDizVp9%$dzQnA?uN;nzd%BX(hj`EJsQ z;FV_Jxjkb346r&q&}`2y6@3-FD{!5bc^n0AsUUz3h@Gkb~I|jci zR@LGuXVkKl1o0!?lt2zpUj~1e$-J4&8W$H#{fsa!IdOI{eOMW!=&+wvSJe7@X;)qT z=p~>aI>VXUK|%J7*u+oMl7CDyK2AGiyGL12ivSQjhR}`PE=5#PrVz)1A2aQm1h154 zKJ+Mc=}MkRy{Rgf1Hb!}p#D5nF?Q1S<^xZ31tz2K@lacN4~6lCdZ}0B_=pv7<%%M2 z-wSC!uj4~s+~U-y7MupiO)=OeN~EZV_XN~d<4|>0iqwpybq0SI%FrF-%4WZmN)+%v zJh}@M3Ve4amH}T6xFaaj;pNh;eHF;APRt-3;^j7&LNgxG4l8ieg*iG&7lDcz@ssB1 z4btOpjgliFp8$Quz{Lj5N;U<4ewn-|bLA&hyc>#3T$E%XVe@)hxRB&-3;F-`h6CtA zMt3QRp^QtHB1o=4XSyqZIFbI5@I*8EyAyR{r()IN+-61W1UCm@e8+P_byFsZK%&bf zz06Zym43P`V8zL`#VLea&YQE4+D#j72pO5B!xPUrylcdsfxhhwjYuqG2RR{KgemES z`k)bfyGt)GXwHlF!L6R_r3HWalm+a4?q|-?7x$PqH%6S{L!AUs@O?}E1Z31}&S<{b zSE8&e#)Hox>SJ|1;n6Np!0F(FJbWs|3Yv=)v_P^7?QE#L;!`Pz3fV=h!nplYolM$; zD|6%ww3s)dlq)L08*(=pw$vTL2orHZzCfWWet$(kFeJ5*$7`bw8u&a+M)Q+h^~#eIi}N zfUn_**m`$CR)8WS4noaqs7X&8P(~G@{=_SnfEwtQ$UzeLkhq#>y~^HV4j4201asB( zW=;L(R$M(K3>+b3(~a!AZiF|6p%QR0=9BLQ7L2{_^EX6?gAq&;J%rm?;z8~(+WT)) zLW9w8#ecj9h%cB8zKF*HfYM25rEZd_S074nC*@3^gr8@wkvB1{EE+1lOPt zBzRLqEcy$cvQQ5IPb6FAZFpZYX26glr= zOaCihHpBsvU+;1Pk=^Y>8N>S>3;ax|huWYsArJmAkRL{W%<3B$p5CDVzu5&jksj2Z zDVsjz-V%Zy^q%W8KzJ|%#4cn0!O2pGl~#CaExFtpq%acjZd*z?eMbo$POsX+0M%I7E+Wpc$&y2dvw3-cQNAXngI}d&C*Rc*3CDDFOou)rJ@_LYwj+m&WO=I$21z#pUV^= z>A{vz6aTqLo(=D?bLK1g!MoX{L8i340n^t;f&5Jp-1_O9@fsloIPDgCZ|O(Hzmmp ztFl}SB)@6Co?d#%w8TLDI~$aqfgklW{>{0BCy9$SD!oUPmnbtAmpmASu;KeutLHLN@k%TDr@yhPWy_4K4jHp>oR zZl82Z$X2S+SuDvqe&oO;MT)LERa&Jx_h=o>^^?B^d(6yfm%5RB+`0ce)JhHK*LRa5 zG#KoLcAAYFMSr!Z<6dc*cjI!LNC_W24+Rq3Gi!dMiq6g`y)${}u&~`jJDdLjtBCE) z>4ls!fGf*_LCz`NE;A^itfzP>9H1u zc2sxpPUmME)TOmhhpHFd1F4I!l;hI4oeijq?5Jl?w)En;Gn?wFW9qRwU1^2p(rTU~ z{WFT5*p!9}w-(5KwDW(;x(=u&nyyW6g7iR;5(vF_kS0}1M5=TY=~Y02bOT6HDT0fD zf&r-(nl$NMq$ov@D$5Ca+_~3vtooBljYzUf ze0SWW0O7p1rSMbvnQD3kIpj@-bPeV#R8YTIP27I}y;^TPytNnR=V&Aw{knr0sc#|e zHLHgEjdJ`3^_=T691(M~*-lP%MC|CQ2g&|Fxr5i`)AZ!W4<%VMF)#~i4Pq5koJrQ2 zv_hyvsGV5slo|P;>K!IK){prlRH9yR^vjkTYOo7>LsaSUlsv9fdcS0-EM`m#QBn~y zeU--d)XR4G%Qq*U~gySg+GB%3`5&c1P!lLZ*yqP z!shAWLuw;>S@DSxZd!5|uhyimT>RD~%gvi@QiE2~;&|65e8H*nXa6_x{S4Ui_6~V9 zG6LTC2p@aGPT+_a4dMP|Hqb0fjoO-QY2e=(_Bueq8B&f)5RED_SM;WY@e~*OVrp+L zZxONZhueu6eKOi6X{gUo$oZ~Shc9k9R&3U*K zcn2RJk=Tkf(Nyy0V9Of+$^LLSVI7yLx6fKBPFN=-G=Y**N}Ke*dfa`~HK2Zz{02=#A}b zVjWETl}RfX<*K4d9LRo@HyRX8_?F)aZdIY}v?FC8>1_Q?IrhD>%@mh%mvpRCEJ!=w zOuAFkXA5~LjAt15eT2A1E`{pbHE7Rd#!t#nhu$%`O5OFs+oIl=sNSFM5%p!9ptq}Z zfI1t459Xqk1^XcbHFZkrd24wExn3hi$eJ;SYQ$qtcDiF6p_K<6ULgyXi$5PS!A!#Pgeo z@6)XFZ?@AFN1%K6#IujK%=hyw!I^(hhOiWK2pB<1A9UgS|Aq2~Jz^ksm>y+qG@wiu z)vM|cY6w zWKP_2s=Me^wph2WG&-==J9(oK*M@LMl`4t{Pl2MT3>MoGg_DLmlGVK^Y40)unIilo zFS+_TffgpB92ROeAUHnwS_ow+fyB$s^_ef@Sa~hP_9P~afaCeL^?O6owy)@@tl@dG zp)I-b-K%eVNjT--@jG>OjRod@;e-YE3tREj+z!&jv>{p#kuRmLWjoNqH1Ljubca_3 z!N5tHa5DuzL5R`HMlHf^dfZc{B(`$Vr>!paZd>YP31_Qb4_h@X**P4H-y&o&V(Ps zJO59eNKcX8v3lj9)(IA;K>0k;TiZgB`rFg4m)YNJHWyYcTz?;O-_{}HDSJFcK4V3o zKm#(dLFc7y`9~vPy=?q6)ReBn)9Ya*fT)H|vV?ZvUcAFA-Sp}Qn}O!vI#bAaG@q#M z&wXhg4<`}3Pn$ZY6;tCN5Jwc2_=e?IP=&gdvQ-2fg`Khd`~HPEMEy-(DeCr{su3jW z8d{$h-X9-UurGh|qI|u${EEVFqn3PDQ>Y2Yx9tl~uHWsJZf%kLItA2%Lt7Iwk!ms7 ztfbO4?+lv;f!_oi?}^5z0NW#Jk{-FKf_sKYvCmP{>+#(}vB1=aR2()mwdyg6T)vkd zQ;1UuM-AHRf}-Oww(A#K81P^l+M zur11ataS^{Lx0{%|K?(UXOi1y) zzR@IOOPYVM#Clz%lPATrhk{a5#=Gc^^Q@DdbHo96WF!1AdsL`zEG1bV{!w(OOyiA0d8BvM1+UtRP`d3%SMN7V@ zS?OQCg7TOUt^4`<4Y)ixL9?)~wL_6l6nG#nK&_ahQ-$sc`%(W}<~${5t$YwMu>vQT z)6WGL+1huM2SyKwjtA&6OSwqMJH~=(m>eu-f6OLJ zKkFagHM5BCsqpk`*A36NajCrN_+5WC)g>`KZ5vfyKU%WnW^urYRPS!iIRDnC7%ki< zhIh_gTvNNO+WNA|I^UJ}rJ+#imYS&AkLAgudFDjL6S|e-PY_4!idIbxWNIshmC4_} z6wZJ3ky?o@xLKT1Qm%?X4GWEPt7O^fqJzJGO&jtlBF5W^P>J`HzeJk`x~dEahLk1d zr(ZzviFRZ#K2j%q`LWY{#7WHQ2`NDG8FM`}G=_=H8rf&%x@F^iW-b>;!t7kL zL(t_UQ1dQt>ombJmCHP3)!{$fi=SXzpc#o6^V`Q&w}QUFC!F&IH)v-1k9iRG*8ZVa zMxhfIYu64F9$Q5}jTQ08b{WgbNM0El5&ncK^+mMtWtz|0dt^T%H*E2Oag{4QS0>%O z9K6wM0BLxwAIe-=xWGB85dC$L=$`9r>5}_=^8WWXHT1Q`ns!NFloop!T#VTU$exzz z7LF40RSRmnTy1pgL3})a<03nvF<05zCMold7Rj1pfVh&Twx+qK#hrJFy|iwfP#v8d zRF%=oP5v$i!kC1TITQor=~AIRX+POrr|PfPMZDXp^zxVhnjfaWt{mW@ZW&@Uf*&y5 z?OZrg9g^nWwq?=Ls*=;tnMxEh;!o0=G;tLdnMM z{rRJ`{0*O&CvVyUDXGu;mWUAa+}DDKn@5Ugfv9l-pRh%cX)f832 zgDSF355b&+93l>N-WLb#T!__AxgfOC1cji|9@ct~kJ{fJ7n}IrXEIlhCC) zG}rT!JUV*w2(q+yF0X#S5G@u>+9#^Fvv3`1S zDT>UZ1p2&uO!W;vvhNoZy&Y`fxn01}FZbR2hStgjg%z&yhW@gKq#P zZp~OEOTU>OYTb|#)}vzKp-S;7G#RR;Ch?N#VdR^9>QogkXC|qEghymm9;hu}#uQum zkZX%nXs%2*F=GPeXhG)(BXp{}cFp9j(JeYtVjrQe<8!YnKfgQyZs`?BF%YsJ93W{U zg@>6GY;@>UIj&OG846G!cjZ-cnOR8sdR~o4E|dBTdu--_gXD1_|Ort2jkKJ>{(3OGA)%@5KQDKLpd$_a7|Y2UZxZL{Rd_M~ExmNzT9 zu3{A-d0AUzh@0R$d2)w6s_g0;oFDPjyZ#11S{tjnbIWu*pVc-&l3A4qNpEkH2p4JU{i^slq<%m#PPmDNu@kGwMFiibr7Qt z<)dy5eK&0rBZIZbq|!`tp~ynCeFWe0G-oRL7JjFvJahLtIrvVr5ED z1fOE4YC5o1u(eNt;i7h1oioqyqI;_F9+0D`OUfmNwNh zRk9wgkD_aw!*x{Wycit5PpCJ3pv|nz686xr9 zFW@Swjk&p;{cFmoFbi>utj78_ZxexMYdKP#}Ju7o5L53BLVtp7~GdXdeEjB(op@mh2Wx!DHe%C@{Dn zb6NuhxE2-vVEh+(z*6Sob?|t+=nNj6mrlwEdMGfpbb@02ZUi1N%upcayTX~=c1wFM zjlh@hmS;p&zBUpz_GIMV!=!J!fHnx{04&$ZPlkszl>DFCoUk5CV0xVcC_b&K zTZiL~+kpZ}5U1x11w=N;@Qxh;`3+T2aCZZaH{cA&xyl2gKTmc8o($t(8|N?Nx#Hk> z-gXjkx_#5$-9f|`>EaFs%R7AzPBF5W^LRt9zy{`m;Qu}ze?!fKosWI5Dcw#8z-tWb zjEPv0uKzItJm-Y1-vVqt)1L&smgE%s|AdVHd#%3#mBHZ(I$8hy7jqFodN%we=5$_OH?1pr)xVU}IPI yj0UWUF)R%my@1!S$bUKvYaRfLU@!o{#`z)o2tshY /dev/null && pwd -P ) || exit # Use the maximum available, or set MAX_FD != -1 to use that value. MAX_FD=maximum @@ -133,10 +131,13 @@ location of your Java installation." fi else JAVACMD=java - which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + if ! command -v java >/dev/null 2>&1 + then + die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. Please set the JAVA_HOME variable in your environment to match the location of your Java installation." + fi fi # Increase the maximum file descriptors if we can. @@ -144,7 +145,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then case $MAX_FD in #( max*) # In POSIX sh, ulimit -H is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC3045 MAX_FD=$( ulimit -H -n ) || warn "Could not query maximum file descriptor limit" esac @@ -152,7 +153,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then '' | soft) :;; #( *) # In POSIX sh, ulimit -n is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC3045 ulimit -n "$MAX_FD" || warn "Could not set maximum file descriptor limit to $MAX_FD" esac @@ -197,6 +198,10 @@ if "$cygwin" || "$msys" ; then done fi + +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. +DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' + # Collect all arguments for the java command; # * $DEFAULT_JVM_OPTS, $JAVA_OPTS, and $GRADLE_OPTS can contain fragments of # shell script including quotes and variable substitutions, so put them in diff --git a/playground/kafka-emulator/build.gradle b/playground/kafka-emulator/build.gradle index 486a232f9b99..2d3f70aa9883 100644 --- a/playground/kafka-emulator/build.gradle +++ b/playground/kafka-emulator/build.gradle @@ -24,11 +24,11 @@ plugins { applyJavaNature(exportJavadoc: false, publish: false) distZip { - archiveName "${baseName}.zip" + archiveFileName = "${archiveBaseName}.zip" } distTar { - archiveName "${baseName}.tar" + archiveFileName = "${archiveBaseName}.tar" } dependencies { diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index bf4c122ca91f..cc427494ed6e 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -63,6 +63,8 @@ task copyDockerfileDependencies(type: Copy) { task copySdkHarnessLauncher(type: Copy) { dependsOn ":sdks:java:container:downloadCloudProfilerAgent" + // if licenses are required, they should be present before this task run. + mustRunAfter ":sdks:java:container:pullLicenses" from configurations.sdkHarnessLauncher into "build/target" diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index efc9ff3db9c6..c4a508680186 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -307,8 +307,8 @@ task spannerCodeCoverageReport(type: JacocoReport, dependsOn: test) { sourceDirectories.setFrom(files(project.sourceSets.main.allSource.srcDirs)) executionData.setFrom(file("${buildDir}/jacoco/test.exec")) reports { - html.enabled true - html.destination file("${buildDir}/reports/jacoco/spanner/") + html.getRequired().set(true) + html.getOutputLocation().set(file("${buildDir}/reports/jacoco/spanner/")) } } From 0b938306d135bbaadb3123daf0455f67baee4a9a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 09:42:10 -0400 Subject: [PATCH 25/32] Bump cloud.google.com/go/spanner from 1.49.0 to 1.50.0 in /sdks (#28838) Bumps [cloud.google.com/go/spanner](https://github.com/googleapis/google-cloud-go) from 1.49.0 to 1.50.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.49.0...spanner/v1.50.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/spanner dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e17427227eba..89f119a54908 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -28,7 +28,7 @@ require ( cloud.google.com/go/datastore v1.14.0 cloud.google.com/go/profiler v0.3.1 cloud.google.com/go/pubsub v1.33.0 - cloud.google.com/go/spanner v1.49.0 + cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.0 github.com/aws/aws-sdk-go-v2/config v1.18.43 diff --git a/sdks/go.sum b/sdks/go.sum index 71c1c4545c89..582f3d69f88e 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -38,8 +38,8 @@ cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+ cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.33.0 h1:6SPCPvWav64tj0sVX/+npCBKhUi/UjJehy9op/V3p2g= cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= -cloud.google.com/go/spanner v1.49.0 h1:+HY8C4uztU7XyLz3xMi/LCXdetLEOExhvRFJu2NiVXM= -cloud.google.com/go/spanner v1.49.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXXd6Dif1KoM= +cloud.google.com/go/spanner v1.50.0 h1:QrJFOpaxCXdXF+GkiruLz642PHxkdj68PbbnLw3O2Zw= +cloud.google.com/go/spanner v1.50.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXXd6Dif1KoM= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= From c89d44df1302464d85feffa5fe54474c8652b6ec Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 5 Oct 2023 10:12:36 -0400 Subject: [PATCH 26/32] Fix whitespace precommit after gradle 8 (#28845) --- sdks/python/test-suites/tox/py38/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/test-suites/tox/py38/build.gradle b/sdks/python/test-suites/tox/py38/build.gradle index 208a1d9d39ca..bc4aa99c79b4 100644 --- a/sdks/python/test-suites/tox/py38/build.gradle +++ b/sdks/python/test-suites/tox/py38/build.gradle @@ -153,6 +153,8 @@ task archiveFilesToLint(type: Zip) { include "**/*.md" include "**/build.gradle" include '**/build.gradle.kts' + exclude '**/build/**' // intermediate build directory + exclude 'website/www/site/themes/docsy/**' // fork to google/docsy exclude "**/node_modules/*" exclude "**/.gogradle/*" } From af491ebe33c8d9544df8a10e7def68cb8549a920 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Thu, 5 Oct 2023 14:27:12 +0000 Subject: [PATCH 27/32] [Fix] Broken Link in Colab Notebook (#28837) * Create HealthcareUtils file with shared resources * revert * Fix Broken Link --- examples/notebooks/healthcare/beam_nlp.ipynb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/notebooks/healthcare/beam_nlp.ipynb b/examples/notebooks/healthcare/beam_nlp.ipynb index 5106aaa607d9..4ba4a5e0a739 100644 --- a/examples/notebooks/healthcare/beam_nlp.ipynb +++ b/examples/notebooks/healthcare/beam_nlp.ipynb @@ -146,7 +146,7 @@ { "cell_type": "markdown", "source": [ - "Then, download [this raw CSV file](https://https://github.com/socd06/medical-nlp/blob/master/data/test.csv), and then upload it into Colab. You should be able to view this file (*test.csv*) in the \"Files\" tab in Colab after uploading." + "Then, download [this raw CSV file](https://github.com/socd06/medical-nlp/blob/master/data/test.csv), and then upload it into Colab. You should be able to view this file (*test.csv*) in the \"Files\" tab in Colab after uploading." ], "metadata": { "id": "1IArtEm8QuCR" From 5446776d00f6d3970ff859c38e26343274413c57 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Thu, 5 Oct 2023 11:50:02 -0400 Subject: [PATCH 28/32] Support for custom MetricsFetcher in Perf tooling. (#28671) * Support for custom BigQueryMetricsFetcher * Read GITHUB repo and owner name from environment variables * Add test_name, test_id * Move client to the fetch method * Update skip condition * Run on self hosted runner * Update readme * Update README * Pass test_name to the metrics_fetcher * Fix linting issues * Fix lint * Fix formatting and lint issues * fix lint --- .github/workflows/run_perf_alert_tool.yml | 2 +- .../apache_beam/testing/analyzers/README.md | 54 +++++++-- .../testing/analyzers/github_issues_utils.py | 34 +++--- .../testing/analyzers/perf_analysis.py | 55 +++++---- .../testing/analyzers/perf_analysis_test.py | 41 +++---- .../testing/analyzers/perf_analysis_utils.py | 104 +++++++++++------- .../testing/analyzers/tests_config.yaml | 2 +- .../load_tests/load_test_metrics_utils.py | 11 -- 8 files changed, 183 insertions(+), 120 deletions(-) diff --git a/.github/workflows/run_perf_alert_tool.yml b/.github/workflows/run_perf_alert_tool.yml index 6946011f0617..1bd8d525c2fb 100644 --- a/.github/workflows/run_perf_alert_tool.yml +++ b/.github/workflows/run_perf_alert_tool.yml @@ -30,7 +30,7 @@ on: jobs: python_run_change_point_analysis: name: Run Change Point Analysis. - runs-on: ubuntu-latest + runs-on: [self-hosted, ubuntu-20.04, main] permissions: issues: write steps: diff --git a/sdks/python/apache_beam/testing/analyzers/README.md b/sdks/python/apache_beam/testing/analyzers/README.md index 076f173f9d71..91b21076f88a 100644 --- a/sdks/python/apache_beam/testing/analyzers/README.md +++ b/sdks/python/apache_beam/testing/analyzers/README.md @@ -35,16 +35,13 @@ update already created GitHub issue or ignore performance alert by not creating ## Config file structure -The config file defines the structure to run change point analysis on a given test. To add a test to the config file, +The yaml defines the structure to run change point analysis on a given test. To add a test config to the yaml file, please follow the below structure. -**NOTE**: The Change point analysis only supports reading the metric data from Big Query for now. +**NOTE**: The Change point analysis only supports reading the metric data from `BigQuery` only. ``` -# the test_1 must be a unique id. -test_1: - test_description: Pytorch image classification on 50k images of size 224 x 224 with resnet 152 - test_target: apache_beam.testing.benchmarks.inference.pytorch_image_classification_benchmarks +test_1: # a unique id for each test config. metrics_dataset: beam_run_inference metrics_table: torch_inference_imagenet_results_resnet152 project: apache-beam-testing @@ -55,11 +52,17 @@ test_1: num_runs_in_change_point_window: 30 # optional parameter ``` -**NOTE**: `test_target` is optional. It is used for identifying the test that was causing the regression. +#### Optional Parameters: -**Note**: By default, the tool fetches metrics from BigQuery tables. `metrics_dataset`, `metrics_table`, `project` and `metric_name` should match with the values defined for performance/load tests. -The above example uses this [test configuration](https://github.com/apache/beam/blob/0a91d139dea4276dc46176c4cdcdfce210fc50c4/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L30) -to fill up the values required to fetch the data from source. +These are the optional parameters that can be added to the test config in addition to the parameters mentioned above. + +- `test_target`: Identifies the test responsible for the regression. + +- `test_description`: Provides a brief overview of the test's function. + +- `test_name`: Denotes the name of the test as stored in the BigQuery table. + +**Note**: The tool, by default, pulls metrics from BigQuery tables. Ensure that the values for `metrics_dataset`, `metrics_table`, `project`, and `metric_name` align with those defined for performance/load tests. The provided example utilizes this [test configuration](https://github.com/apache/beam/blob/0a91d139dea4276dc46176c4cdcdfce210fc50c4/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L30) to populate the necessary values for data retrieval. ### Different ways to avoid false positive change points @@ -76,8 +79,35 @@ setting `num_runs_in_change_point_window=7` will achieve it. ## Register a test for performance alerts -If a new test needs to be registered for the performance alerting tool, please add the required test parameters to the -config file. +If a new test needs to be registered for the performance alerting tool, + +- You can either add it to the config file that is already present. +- You can define your own yaml file and call the [perf_analysis.run()](https://github.com/apache/beam/blob/a46bc12a256dcaa3ae2cc9e5d6fdcaa82b59738b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py#L152) method. + + +## Integrating the Perf Alert Tool with a Custom BigQuery Schema + +By default, the Perf Alert Tool retrieves metrics from the `apache-beam-testing` BigQuery projects. All performance and load tests within Beam utilize a standard [schema](https://github.com/apache/beam/blob/a7e12db9b5977c4a7b13554605c0300389a3d6da/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py#L70) for metrics publication. The tool inherently recognizes and operates with this schema when extracting metrics from BigQuery tables. + +To fetch the data from a BigQuery dataset that is not a default setting of the Apache Beam's setting, One can inherit the `MetricsFetcher` class and implement the abstract method `fetch_metric_data`. This method should return a tuple of desired metric values and timestamps of the metric values of when it was published. + +``` +from apache_beam.testing.analyzers import perf_analysis +config_file_path = +my_metric_fetcher = MyMetricsFetcher() # inherited from MetricsFetcher +perf_analysis.run(config_file_path, my_metrics_fetcher) +``` + +``Note``: The metrics and timestamps should be sorted based on the timestamps values in ascending order. + +### Configuring GitHub Parameters + +Out of the box, the performance alert tool targets the `apache/beam` repository when raising issues. If you wish to utilize this tool for another repository, you'll need to pre-set a couple of environment variables: + +- `REPO_OWNER`: Represents the owner of the repository. (e.g., `apache`) +- `REPO_NAME`: Specifies the repository name itself. (e.g., `beam`) + +Before initiating the tool, also ensure that the `GITHUB_TOKEN` is set to an authenticated GitHub token. This permits the tool to generate GitHub issues whenever performance alerts arise. ## Triage performance alert issues diff --git a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py index e1f20baa50a6..82758be8f180 100644 --- a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py @@ -34,8 +34,8 @@ 'A Github Personal Access token is required ' 'to create Github Issues.') -_BEAM_GITHUB_REPO_OWNER = 'apache' -_BEAM_GITHUB_REPO_NAME = 'beam' +_GITHUB_REPO_OWNER = os.environ.get('REPO_OWNER', 'apache') +_GITHUB_REPO_NAME = os.environ.get('REPO_NAME', 'beam') # Adding GitHub Rest API version to the header to maintain version stability. # For more information, please look at # https://github.blog/2022-11-28-to-infinity-and-beyond-enabling-the-future-of-githubs-rest-api-with-api-versioning/ # pylint: disable=line-too-long @@ -77,10 +77,10 @@ def create_issue( Tuple containing GitHub issue number and issue URL. """ url = "https://api.github.com/repos/{}/{}/issues".format( - _BEAM_GITHUB_REPO_OWNER, _BEAM_GITHUB_REPO_NAME) + _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME) data = { - 'owner': _BEAM_GITHUB_REPO_OWNER, - 'repo': _BEAM_GITHUB_REPO_NAME, + 'owner': _GITHUB_REPO_OWNER, + 'repo': _GITHUB_REPO_NAME, 'title': title, 'body': description, 'labels': [_AWAITING_TRIAGE_LABEL, _PERF_ALERT_LABEL] @@ -108,20 +108,20 @@ def comment_on_issue(issue_number: int, issue, and the comment URL. """ url = 'https://api.github.com/repos/{}/{}/issues/{}'.format( - _BEAM_GITHUB_REPO_OWNER, _BEAM_GITHUB_REPO_NAME, issue_number) + _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME, issue_number) open_issue_response = requests.get( url, json.dumps({ - 'owner': _BEAM_GITHUB_REPO_OWNER, - 'repo': _BEAM_GITHUB_REPO_NAME, + 'owner': _GITHUB_REPO_OWNER, + 'repo': _GITHUB_REPO_NAME, 'issue_number': issue_number }, default=str), headers=_HEADERS).json() if open_issue_response['state'] == 'open': data = { - 'owner': _BEAM_GITHUB_REPO_OWNER, - 'repo': _BEAM_GITHUB_REPO_NAME, + 'owner': _GITHUB_REPO_OWNER, + 'repo': _GITHUB_REPO_NAME, 'body': comment_description, issue_number: issue_number, } @@ -134,13 +134,14 @@ def comment_on_issue(issue_number: int, def add_awaiting_triage_label(issue_number: int): url = 'https://api.github.com/repos/{}/{}/issues/{}/labels'.format( - _BEAM_GITHUB_REPO_OWNER, _BEAM_GITHUB_REPO_NAME, issue_number) + _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME, issue_number) requests.post( url, json.dumps({'labels': [_AWAITING_TRIAGE_LABEL]}), headers=_HEADERS) def get_issue_description( - test_name: str, + test_id: str, + test_name: Optional[str], metric_name: str, timestamps: List[pd.Timestamp], metric_values: List, @@ -167,10 +168,13 @@ def get_issue_description( description = [] - description.append(_ISSUE_DESCRIPTION_TEMPLATE.format(test_name, metric_name)) + description.append(_ISSUE_DESCRIPTION_TEMPLATE.format(test_id, metric_name)) - description.append(("`Test description:` " + - f'{test_description}') if test_description else '') + if test_name: + description.append(("`test_name:` " + f'{test_name}')) + + if test_description: + description.append(("`Test description:` " + f'{test_description}')) description.append('```') diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 7f1ffbb944e9..c86ecb2c4e20 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -23,7 +23,6 @@ import argparse import logging import os -import uuid from datetime import datetime from datetime import timezone from typing import Any @@ -33,9 +32,10 @@ import pandas as pd from apache_beam.testing.analyzers import constants +from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import GitHubIssueMetaData +from apache_beam.testing.analyzers.perf_analysis_utils import MetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import create_performance_alert -from apache_beam.testing.analyzers.perf_analysis_utils import fetch_metric_data from apache_beam.testing.analyzers.perf_analysis_utils import find_latest_change_point_index from apache_beam.testing.analyzers.perf_analysis_utils import get_existing_issues_data from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window @@ -43,10 +43,10 @@ from apache_beam.testing.analyzers.perf_analysis_utils import publish_issue_metadata_to_big_query from apache_beam.testing.analyzers.perf_analysis_utils import read_test_config from apache_beam.testing.analyzers.perf_analysis_utils import validate_config -from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsFetcher -def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): +def run_change_point_analysis( + params, test_id, big_query_metrics_fetcher: MetricsFetcher): """ Args: params: Dict containing parameters to run change point analysis. @@ -56,14 +56,21 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): Returns: bool indicating if a change point is observed and alerted on GitHub. """ - logging.info("Running change point analysis for test %s" % test_name) + logging.info("Running change point analysis for test ID %s" % test_id) if not validate_config(params.keys()): raise ValueError( f"Please make sure all these keys {constants._PERF_TEST_KEYS} " - f"are specified for the {test_name}") + f"are specified for the {test_id}") metric_name = params['metric_name'] + # test_name will be used to query a single test from + # multiple tests in a single BQ table. Right now, the default + # assumption is that all the test have an individual BQ table + # but this might not be case for other tests(such as IO tests where + # a single BQ tables stores all the data) + test_name = params.get('test_name', None) + min_runs_between_change_points = ( constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS) if 'min_runs_between_change_points' in params: @@ -74,15 +81,18 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): if 'num_runs_in_change_point_window' in params: num_runs_in_change_point_window = params['num_runs_in_change_point_window'] - metric_values, timestamps = fetch_metric_data( - params=params, - big_query_metrics_fetcher=big_query_metrics_fetcher + metric_values, timestamps = big_query_metrics_fetcher.fetch_metric_data( + project=params['project'], + metrics_dataset=params['metrics_dataset'], + metrics_table=params['metrics_table'], + metric_name=params['metric_name'], + test_name=test_name ) change_point_index = find_latest_change_point_index( metric_values=metric_values) if not change_point_index: - logging.info("Change point is not detected for the test %s" % test_name) + logging.info("Change point is not detected for the test ID %s" % test_id) return False # since timestamps are ordered in ascending order and # num_runs_in_change_point_window refers to the latest runs, @@ -92,11 +102,11 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): if not is_change_point_in_valid_window(num_runs_in_change_point_window, latest_change_point_run): logging.info( - 'Performance regression/improvement found for the test: %s. ' + 'Performance regression/improvement found for the test ID: %s. ' 'on metric %s. Since the change point run %s ' 'lies outside the num_runs_in_change_point_window distance: %s, ' 'alert is not raised.' % ( - test_name, + test_id, metric_name, latest_change_point_run + 1, num_runs_in_change_point_window)) @@ -106,8 +116,7 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): last_reported_issue_number = None issue_metadata_table_name = f'{params.get("metrics_table")}_{metric_name}' existing_issue_data = get_existing_issues_data( - table_name=issue_metadata_table_name, - big_query_metrics_fetcher=big_query_metrics_fetcher) + table_name=issue_metadata_table_name) if existing_issue_data is not None: existing_issue_timestamps = existing_issue_data[ @@ -124,20 +133,21 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): min_runs_between_change_points=min_runs_between_change_points) if is_alert: issue_number, issue_url = create_performance_alert( - metric_name, test_name, timestamps, + metric_name, test_id, timestamps, metric_values, change_point_index, params.get('labels', None), last_reported_issue_number, test_description = params.get('test_description', None), + test_name = test_name ) issue_metadata = GitHubIssueMetaData( issue_timestamp=pd.Timestamp( datetime.now().replace(tzinfo=timezone.utc)), # BQ doesn't allow '.' in table name - test_name=test_name.replace('.', '_'), + test_id=test_id.replace('.', '_'), + test_name=test_name, metric_name=metric_name, - test_id=uuid.uuid4().hex, change_point=metric_values[change_point_index], issue_number=issue_number, issue_url=issue_url, @@ -149,7 +159,10 @@ def run_change_point_analysis(params, test_name, big_query_metrics_fetcher): return is_alert -def run(config_file_path: Optional[str] = None) -> None: +def run( + big_query_metrics_fetcher: MetricsFetcher = BigQueryMetricsFetcher(), + config_file_path: Optional[str] = None, +) -> None: """ run is the entry point to run change point analysis on test metric data, which is read from config file, and if there is a performance @@ -169,12 +182,10 @@ def run(config_file_path: Optional[str] = None) -> None: tests_config: Dict[str, Dict[str, Any]] = read_test_config(config_file_path) - big_query_metrics_fetcher = BigQueryMetricsFetcher() - - for test_name, params in tests_config.items(): + for test_id, params in tests_config.items(): run_change_point_analysis( params=params, - test_name=test_name, + test_id=test_id, big_query_metrics_fetcher=big_query_metrics_fetcher) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py index 094cd9c47ec0..9c7921300d9d 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py @@ -32,6 +32,7 @@ from apache_beam.io.filesystems import FileSystems from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers import github_issues_utils + from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert from apache_beam.testing.analyzers.perf_analysis_utils import e_divisive @@ -41,18 +42,18 @@ from apache_beam.testing.analyzers.perf_analysis_utils import validate_config from apache_beam.testing.load_tests import load_test_metrics_utils except ImportError as e: - analysis = None # type: ignore + raise unittest.SkipTest('Missing dependencies to run perf analysis tests.') # mock methods. -def get_fake_data_with_no_change_point(**kwargs): +def get_fake_data_with_no_change_point(*args, **kwargs): num_samples = 20 metric_values = [1] * num_samples timestamps = list(range(num_samples)) return metric_values, timestamps -def get_fake_data_with_change_point(**kwargs): +def get_fake_data_with_change_point(*args, **kwargs): # change point will be at index 13. num_samples = 20 metric_values = [0] * 12 + [3] + [4] * 7 @@ -69,10 +70,6 @@ def get_existing_issue_data(**kwargs): }]) -@unittest.skipIf( - analysis is None, - 'Missing dependencies. ' - 'Test dependencies are missing for the Analyzer.') class TestChangePointAnalysis(unittest.TestCase): def setUp(self) -> None: self.single_change_point_series = [0] * 10 + [1] * 10 @@ -151,18 +148,20 @@ def test_duplicate_change_points_are_not_valid_alerts(self): min_runs_between_change_points=min_runs_between_change_points) self.assertFalse(is_alert) - @mock.patch( - 'apache_beam.testing.analyzers.perf_analysis.fetch_metric_data', + @mock.patch.object( + BigQueryMetricsFetcher, + 'fetch_metric_data', get_fake_data_with_no_change_point) def test_no_alerts_when_no_change_points(self): is_alert = analysis.run_change_point_analysis( params=self.params, - test_name=self.test_id, - big_query_metrics_fetcher=None) + test_id=self.test_id, + big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) - @mock.patch( - 'apache_beam.testing.analyzers.perf_analysis.fetch_metric_data', + @mock.patch.object( + BigQueryMetricsFetcher, + 'fetch_metric_data', get_fake_data_with_change_point) @mock.patch( 'apache_beam.testing.analyzers.perf_analysis.get_existing_issues_data', @@ -178,12 +177,13 @@ def test_no_alerts_when_no_change_points(self): def test_alert_on_data_with_change_point(self, *args): is_alert = analysis.run_change_point_analysis( params=self.params, - test_name=self.test_id, - big_query_metrics_fetcher=None) + test_id=self.test_id, + big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertTrue(is_alert) - @mock.patch( - 'apache_beam.testing.analyzers.perf_analysis.fetch_metric_data', + @mock.patch.object( + BigQueryMetricsFetcher, + 'fetch_metric_data', get_fake_data_with_change_point) @mock.patch( 'apache_beam.testing.analyzers.perf_analysis.get_existing_issues_data', @@ -198,8 +198,8 @@ def test_alert_on_data_with_change_point(self, *args): def test_alert_on_data_with_reported_change_point(self, *args): is_alert = analysis.run_change_point_analysis( params=self.params, - test_name=self.test_id, - big_query_metrics_fetcher=None) + test_id=self.test_id, + big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) def test_change_point_has_anomaly_marker_in_gh_description(self): @@ -208,7 +208,8 @@ def test_change_point_has_anomaly_marker_in_gh_description(self): change_point_index = find_latest_change_point_index(metric_values) description = github_issues_utils.get_issue_description( - test_name=self.test_id, + test_id=self.test_id, + test_name=self.params.get('test_name', None), test_description=self.params['test_description'], metric_name=self.params['metric_name'], metric_values=metric_values, diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index 0a559fc4beeb..f9604c490fc0 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -14,11 +14,11 @@ # See the License for the specific language governing permissions and # limitations under the License. # +import abc import logging from dataclasses import asdict from dataclasses import dataclass from statistics import median -from typing import Any from typing import Dict from typing import List from typing import Optional @@ -28,11 +28,11 @@ import pandas as pd import yaml from google.api_core import exceptions +from google.cloud import bigquery from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.load_tests import load_test_metrics_utils -from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsFetcher from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsPublisher from signal_processing_algorithms.energy_statistics.energy_statistics import e_divisive @@ -59,9 +59,7 @@ def is_change_point_in_valid_window( return num_runs_in_change_point_window > latest_change_point_run -def get_existing_issues_data( - table_name: str, big_query_metrics_fetcher: BigQueryMetricsFetcher -) -> Optional[pd.DataFrame]: +def get_existing_issues_data(table_name: str) -> Optional[pd.DataFrame]: """ Finds the most recent GitHub issue created for the test_name. If no table found with name=test_name, return (None, None) @@ -73,12 +71,14 @@ def get_existing_issues_data( LIMIT 10 """ try: - df = big_query_metrics_fetcher.fetch(query=query) + client = bigquery.Client() + query_job = client.query(query=query) + existing_issue_data = query_job.result().to_dataframe() except exceptions.NotFound: # If no table found, that means this is first performance regression # on the current test+metric. return None - return df + return existing_issue_data def is_perf_alert( @@ -123,33 +123,6 @@ def validate_config(keys): return constants._PERF_TEST_KEYS.issubset(keys) -def fetch_metric_data( - params: Dict[str, Any], big_query_metrics_fetcher: BigQueryMetricsFetcher -) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: - """ - Args: - params: Dict containing keys required to fetch data from a data source. - big_query_metrics_fetcher: A BigQuery metrics fetcher for fetch metrics. - Returns: - Tuple[List[Union[int, float]], List[pd.Timestamp]]: Tuple containing list - of metric_values and list of timestamps. Both are sorted in ascending - order wrt timestamps. - """ - query = f""" - SELECT * - FROM {params['project']}.{params['metrics_dataset']}.{params['metrics_table']} - WHERE CONTAINS_SUBSTR(({load_test_metrics_utils.METRICS_TYPE_LABEL}), '{params['metric_name']}') - ORDER BY {load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL} DESC - LIMIT {constants._NUM_DATA_POINTS_TO_RUN_CHANGE_POINT_ANALYSIS} - """ - metric_data: pd.DataFrame = big_query_metrics_fetcher.fetch(query=query) - metric_data.sort_values( - by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) - return ( - metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), - metric_data[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) - - def find_change_points(metric_values: List[Union[float, int]]): return e_divisive(metric_values) @@ -175,7 +148,7 @@ def find_latest_change_point_index(metric_values: List[Union[float, int]]): def publish_issue_metadata_to_big_query(issue_metadata, table_name): """ - Published issue_metadata to BigQuery with table name=test_name. + Published issue_metadata to BigQuery with table name. """ bq_metrics_publisher = BigQueryMetricsPublisher( project_name=constants._BQ_PROJECT_NAME, @@ -190,18 +163,21 @@ def publish_issue_metadata_to_big_query(issue_metadata, table_name): def create_performance_alert( metric_name: str, - test_name: str, + test_id: str, timestamps: List[pd.Timestamp], metric_values: List[Union[int, float]], change_point_index: int, labels: List[str], existing_issue_number: Optional[int], - test_description: Optional[str] = None) -> Tuple[int, str]: + test_description: Optional[str] = None, + test_name: Optional[str] = None, +) -> Tuple[int, str]: """ Creates performance alert on GitHub issues and returns GitHub issue number and issue URL. """ description = github_issues_utils.get_issue_description( + test_id=test_id, test_name=test_name, test_description=test_description, metric_name=metric_name, @@ -213,7 +189,7 @@ def create_performance_alert( issue_number, issue_url = github_issues_utils.report_change_point_on_issues( title=github_issues_utils._ISSUE_TITLE_TEMPLATE.format( - test_name, metric_name + test_id, metric_name ), description=description, labels=labels, @@ -253,3 +229,55 @@ def filter_change_points_by_median_threshold( if relative_change > threshold: valid_change_points.append(idx) return valid_change_points + + +class MetricsFetcher(metaclass=abc.ABCMeta): + @abc.abstractmethod + def fetch_metric_data( + self, + *, + project, + metrics_dataset, + metrics_table, + metric_name, + test_name=None) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + """ + Define SQL query and fetch the timestamp values and metric values + from BigQuery tables. + """ + raise NotImplementedError + + +class BigQueryMetricsFetcher(MetricsFetcher): + def fetch_metric_data( + self, + *, + project, + metrics_dataset, + metrics_table, + metric_name, + test_name=None, + ) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + """ + Args: + params: Dict containing keys required to fetch data from a data source. + Returns: + Tuple[List[Union[int, float]], List[pd.Timestamp]]: Tuple containing list + of metric_values and list of timestamps. Both are sorted in ascending + order wrt timestamps. + """ + query = f""" + SELECT * + FROM {project}.{metrics_dataset}.{metrics_table} + WHERE CONTAINS_SUBSTR(({load_test_metrics_utils.METRICS_TYPE_LABEL}), '{metric_name}') + ORDER BY {load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL} DESC + LIMIT {constants._NUM_DATA_POINTS_TO_RUN_CHANGE_POINT_ANALYSIS} + """ + client = bigquery.Client() + query_job = client.query(query=query) + metric_data = query_job.result().to_dataframe() + metric_data.sort_values( + by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) + return ( + metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), + metric_data[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) diff --git a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml index f808f5e41d74..ec9cfe6f1ac0 100644 --- a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml +++ b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml @@ -16,7 +16,7 @@ # # for the unique key to define a test, please use the following format: -# {test_name}-{metric_name} +# {test_id}-{metric_name} pytorch_image_classification_benchmarks-resnet152-mean_inference_batch_latency_micro_secs: test_description: diff --git a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py index 92a5f68351fe..01db2c114efb 100644 --- a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py +++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py @@ -38,7 +38,6 @@ from typing import Optional from typing import Union -import pandas as pd import requests from requests.auth import HTTPBasicAuth @@ -650,13 +649,3 @@ def __init__(self): def process(self, element): yield self.timestamp_val_fn( element, self.timestamp_fn(micros=int(self.time_fn() * 1000000))) - - -class BigQueryMetricsFetcher: - def __init__(self): - self.client = bigquery.Client() - - def fetch(self, query) -> pd.DataFrame: - query_job = self.client.query(query=query) - result = query_job.result() - return result.to_dataframe() From 94a579ccdc968c6b01a481777e3140ab361738fa Mon Sep 17 00:00:00 2001 From: caneff Date: Thu, 5 Oct 2023 13:37:14 -0400 Subject: [PATCH 29/32] Add observed=False to be explicit for Pandas 2.1 (#28823) --- sdks/python/apache_beam/dataframe/frames_test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py index 6f7a63c29164..fffb5391c760 100644 --- a/sdks/python/apache_beam/dataframe/frames_test.py +++ b/sdks/python/apache_beam/dataframe/frames_test.py @@ -1021,8 +1021,14 @@ def test_categorical_groupby(self): df = df.set_index('B') # TODO(BEAM-11190): These aggregations can be done in index partitions, but # it will require a little more complex logic - self._run_test(lambda df: df.groupby(level=0).sum(), df, nonparallel=True) - self._run_test(lambda df: df.groupby(level=0).mean(), df, nonparallel=True) + self._run_test( + lambda df: df.groupby(level=0, observed=False).sum(), + df, + nonparallel=True) + self._run_test( + lambda df: df.groupby(level=0, observed=False).mean(), + df, + nonparallel=True) def test_astype_categorical(self): df = pd.DataFrame({'A': np.arange(6), 'B': list('aabbca')}) From e6a1062b5483f62fd1152418535f6746dd69fc6e Mon Sep 17 00:00:00 2001 From: caneff Date: Thu, 5 Oct 2023 13:37:23 -0400 Subject: [PATCH 30/32] Add map function and skip applymap for 2.1 (#28818) --- sdks/python/apache_beam/dataframe/frames.py | 2 ++ sdks/python/apache_beam/dataframe/frames_test.py | 13 +++++++++++++ 2 files changed, 15 insertions(+) diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py index f51814e7f696..b7aa130fbbd8 100644 --- a/sdks/python/apache_beam/dataframe/frames.py +++ b/sdks/python/apache_beam/dataframe/frames.py @@ -2978,6 +2978,8 @@ def aggregate(self, func, axis, *args, **kwargs): agg = aggregate applymap = frame_base._elementwise_method('applymap', base=pd.DataFrame) + if PD_VERSION >= (2, 1): + map = frame_base._elementwise_method('map', base=pd.DataFrame) add_prefix = frame_base._elementwise_method('add_prefix', base=pd.DataFrame) add_suffix = frame_base._elementwise_method('add_suffix', base=pd.DataFrame) diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py index fffb5391c760..6e32acefc61b 100644 --- a/sdks/python/apache_beam/dataframe/frames_test.py +++ b/sdks/python/apache_beam/dataframe/frames_test.py @@ -865,6 +865,8 @@ def test_corrwith_bad_axis(self): self._run_error_test(lambda df: df.corrwith(df, axis=5), df) @unittest.skipIf(PD_VERSION < (1, 2), "na_action added in pandas 1.2.0") + @pytest.mark.filterwarnings( + "ignore:The default of observed=False is deprecated:FutureWarning") def test_applymap_na_action(self): # Replicates a doctest for na_action which is incompatible with # doctest framework @@ -875,6 +877,17 @@ def test_applymap_na_action(self): # TODO: generate proxy using naive type inference on fn check_proxy=False) + @unittest.skipIf(PD_VERSION < (2, 1), "map added in 2.1.0") + def test_map_na_action(self): + # Replicates a doctest for na_action which is incompatible with + # doctest framework + df = pd.DataFrame([[pd.NA, 2.12], [3.356, 4.567]]) + self._run_test( + lambda df: df.map(lambda x: len(str(x)), na_action='ignore'), + df, + # TODO: generate proxy using naive type inference on fn + check_proxy=False) + def test_dataframe_eval_query(self): df = pd.DataFrame(np.random.randn(20, 3), columns=['a', 'b', 'c']) self._run_test(lambda df: df.eval('foo = a + b - c'), df) From 30c5e05ed4dd0478295ac0c69553a0758fbce9ff Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 13:44:16 -0400 Subject: [PATCH 31/32] Bump github.com/fsouza/fake-gcs-server from 1.47.4 to 1.47.5 in /sdks (#28812) Bumps [github.com/fsouza/fake-gcs-server](https://github.com/fsouza/fake-gcs-server) from 1.47.4 to 1.47.5. - [Release notes](https://github.com/fsouza/fake-gcs-server/releases) - [Commits](https://github.com/fsouza/fake-gcs-server/compare/v1.47.4...v1.47.5) --- updated-dependencies: - dependency-name: github.com/fsouza/fake-gcs-server dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 6 +++--- sdks/go.sum | 14 +++++++------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 89f119a54908..d817ae549857 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -67,8 +67,8 @@ require ( ) require ( - github.com/fsouza/fake-gcs-server v1.47.4 - golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 + github.com/fsouza/fake-gcs-server v1.47.5 + golang.org/x/exp v0.0.0-20230807204917-050eac23e9de ) require ( @@ -88,7 +88,7 @@ require ( cloud.google.com/go v0.110.7 // indirect cloud.google.com/go/compute v1.23.0 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/iam v1.1.1 // indirect + cloud.google.com/go/iam v1.1.2 // indirect cloud.google.com/go/longrunning v0.5.1 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 582f3d69f88e..9f43e9a53abc 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -26,8 +26,8 @@ cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7 cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/datastore v1.14.0 h1:Mq0ApTRdLW3/dyiw+DkjTk0+iGIUvkbzaC8sfPwWTH4= cloud.google.com/go/datastore v1.14.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8= -cloud.google.com/go/iam v1.1.1 h1:lW7fzj15aVIXYHREOqjRBV9PsH0Z6u8Y46a1YGvQP4Y= -cloud.google.com/go/iam v1.1.1/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= +cloud.google.com/go/iam v1.1.2 h1:gacbrBdWcoVmGLozRuStX45YKvJtzIjJdAolzUs1sm4= +cloud.google.com/go/iam v1.1.2/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU= cloud.google.com/go/kms v1.15.0 h1:xYl5WEaSekKYN5gGRyhjvZKM22GVBBCzegGNVPy+aIs= cloud.google.com/go/longrunning v0.5.1 h1:Fr7TXftcqTudoyRJa113hyaqlGdiBQkp0Gq7tErFDWI= cloud.google.com/go/longrunning v0.5.1/go.mod h1:spvimkwdz6SPWKEt/XBij79E9fiTkHSQl/fRUUQJYJc= @@ -195,8 +195,8 @@ github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoD github.com/frankban/quicktest v1.2.2/go.mod h1:Qh/WofXFeiAFII1aEBu529AtJo6Zg2VHscnEsbBnJ20= github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= -github.com/fsouza/fake-gcs-server v1.47.4 h1:gfBhBxEra20/Om02cvcyL8EnekV8KDb01Yffjat6AKQ= -github.com/fsouza/fake-gcs-server v1.47.4/go.mod h1:vqUZbI12uy9IkRQ54Q4p5AniQsSiUq8alO9Nv2egMmA= +github.com/fsouza/fake-gcs-server v1.47.5 h1:o+wL01s01j/2OdkIaduDogXw2bZveq9TFb8f+BqEHtM= +github.com/fsouza/fake-gcs-server v1.47.5/go.mod h1:PhN8F1rHAOCL5jWyXcw8nPfLfHnka6D9fT7ctL9nbkA= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= @@ -348,7 +348,7 @@ github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcs github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= -github.com/minio/minio-go/v7 v7.0.61 h1:87c+x8J3jxQ5VUGimV9oHdpjsAvy3fhneEBKuoKEVUI= +github.com/minio/minio-go/v7 v7.0.63 h1:GbZ2oCvaUdgT5640WJOpyDhhDxvknAJU2/T3yurwcbQ= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/moby/patternmatcher v0.5.0 h1:YCZgJOeULcxLw1Q+sVR636pmS7sPEn1Qo2iAN6M7DBo= github.com/moby/patternmatcher v0.5.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= @@ -497,8 +497,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 h1:MGwJjxBy0HJshjDNfLsYO8xppfqWlA5ZT9OhtUUhTNw= -golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= +golang.org/x/exp v0.0.0-20230807204917-050eac23e9de h1:l5Za6utMv/HsBWWqzt4S8X17j+kt1uVETUX5UFhn2rE= +golang.org/x/exp v0.0.0-20230807204917-050eac23e9de/go.mod h1:FXUEEKJgO7OQYeo8N01OfiKP8RXMtf6e8aTskBGqWdc= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= From 31e1c7a8aef905a54c5382930e43e2d36d2d5773 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 13:44:46 -0400 Subject: [PATCH 32/32] Bump pillow (#28810) Bumps [pillow](https://github.com/python-pillow/Pillow) from 9.3.0 to 10.0.1. - [Release notes](https://github.com/python-pillow/Pillow/releases) - [Changelog](https://github.com/python-pillow/Pillow/blob/main/CHANGES.rst) - [Commits](https://github.com/python-pillow/Pillow/compare/9.3.0...10.0.1) --- updated-dependencies: - dependency-name: pillow dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../kfp/components/preprocessing/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt index e902ead34151..706adf9de0aa 100644 --- a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt +++ b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/preprocessing/requirements.txt @@ -18,4 +18,4 @@ requests==2.31.0 torch==1.13.1 torchvision==0.13.0 numpy==1.22.4 -Pillow==9.3.0 +Pillow==10.0.1