From eedee3ff8cf5fbd1fc663384037c9fd205a7bb7b Mon Sep 17 00:00:00 2001 From: Captain-Slacker-OwO Date: Wed, 2 Apr 2025 20:23:12 +0800 Subject: [PATCH] test --- .gitignore | 1 + README.md | 28 +-- bin/automq-perf-test.sh | 1 + docker/README.md | 23 ++ docker/local/docker-compose.yml | 160 +++++++++++++ docker/scripts/pre_start.sh | 16 ++ docker/scripts/start.sh | 2 + docs/images/automq_simple_arch.png | Bin 0 -> 154422 bytes s3stream/build.gradle | 3 + .../java/com/automq/stream/s3/S3Stream.java | 59 ++--- .../exceptions/ObjectNotExistException.java | 3 +- .../s3/operator/AbstractObjectStorage.java | 26 +-- .../stream/s3/operator/AwsObjectStorage.java | 5 +- .../automq/stream/utils/AsyncSemaphore.java | 1 + .../com/automq/stream/s3/S3StorageTest.java | 114 +++++++++ .../stream/utils/AsyncSemaphoreTest.java | 220 ++++++++++++++++++ .../kafka/tools/automq/PerfCommand.java | 40 +++- .../kafka/tools/automq/perf/PerfConfig.java | 16 ++ .../kafka/tools/automq/perf/TopicService.java | 27 ++- 19 files changed, 671 insertions(+), 74 deletions(-) create mode 100644 docker/local/docker-compose.yml create mode 100644 docker/scripts/pre_start.sh create mode 100644 docs/images/automq_simple_arch.png create mode 100644 s3stream/build.gradle create mode 100644 s3stream/src/test/java/com/automq/stream/utils/AsyncSemaphoreTest.java diff --git a/.gitignore b/.gitignore index 015df8ead8..3d129ffb88 100644 --- a/.gitignore +++ b/.gitignore @@ -31,6 +31,7 @@ TAGS Vagrantfile.local /logs .DS_Store +local.properties config/server-* config/zookeeper-* diff --git a/README.md b/README.md index 4ca8244c3a..744cb09598 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -# AutoMQ: A cloud-first alternative of Kafka by decoupling durability to S3 and EBS +# AutoMQ: A cloud-native alternative of Kafka by decoupling durability to cloud storage services like S3
@@ -20,7 +20,6 @@ [![Linkedin Badge](https://img.shields.io/badge/-LinkedIn-blue?style=flat-square&logo=Linkedin&logoColor=white&link=https://www.linkedin.com/company/automq)](https://www.linkedin.com/company/automq) [![Twitter URL](https://img.shields.io/twitter/follow/AutoMQ)](https://twitter.com/intent/follow?screen_name=AutoMQ_Lab) -[![](https://img.shields.io/badge/-%20Wechat%20-red?style=social&logo=discourse)](docs/images/automq-wechat.png) [![](https://badgen.net/badge/Slack/Join%20AutoMQ/0abd59?icon=slack)](https://join.slack.com/t/automq/shared_invite/zt-29h17vye9-thf31ebIVL9oXuRdACnOIA) [![](https://img.shields.io/badge/AutoMQ%20vs.%20Kafka(Cost)-yellow)](https://www.automq.com/blog/automq-vs-apache-kafka-a-real-aws-cloud-bill-comparison) [![](https://img.shields.io/badge/AutoMQ%20vs.%20Kafka(Performance)-orange)](https://docs.automq.com/docs/automq-opensource/IJLQwnVROiS5cUkXfF0cuHnWnNd) @@ -36,11 +35,6 @@ ![](https://img.shields.io/badge/Aliyun-%E2%9C%85-lightgray?logo=alibabacloud)
-## 📺 Youtube Video Introduction -Watch this video to learn what is AutoMQ. ⬇️ ⬇️ ⬇️ - -[![What is AutoMQ?](https://img.youtube.com/vi/3JQrclZlie4/0.jpg)](https://www.youtube.com/watch?v=3JQrclZlie4) - ## 👥 Big Companies Worldwide are Using AutoMQ > Here are some of our customers who are using AutoMQ from all over the world. image @@ -145,7 +139,7 @@ Table Topic feature for unified stream and data analysis, which now supports the Durability - Guaranteed by S3/EBS[2] + Guaranteed by cloud storage services[2] Guaranteed by ISR Guaranteed by Bookkeeper Guaranteed by Raft @@ -162,12 +156,12 @@ Table Topic feature for unified stream and data analysis, which now supports the > [1] Apache Kafka Compatibility's definition is coming from this [blog](https://www.kai-waehner.de/blog/2021/05/09/kafka-api-de-facto-standard-event-streaming-like-amazon-s3-object-storage/). -> [2] EBS Durability: On Azure, GCP, and Alibaba Cloud, Regional EBS replicas span multiple AZs. On AWS, ensure durability by double writing to EBS and S3 Express One Zone in different AZs. +> [2] AutoMQ's flexible architecture can utilize the durability of various cloud storage services like S3, Regional EBS, and EFS, all offering multi-AZ durability. ## 🔶 Why AutoMQ - **Cost effective**: The first true cloud-native streaming storage system, designed for optimal cost and efficiency on the cloud. Refer to [this report](https://docs.automq.com/docs/automq-opensource/EV6mwoC95ihwRckMsUKcppnqnJb) to see how we cut Apache Kafka billing by 90% on the cloud. -- **High Reliability**: Leverage cloud-shared storage services(EBS and S3) to achieve zero RPO, RTO in seconds and 99.999999999% durability. +- **High Reliability**: Leverage cloud-shared storage services to achieve zero RPO, RTO in seconds and 99.999999999% durability. - **Serverless**: - Auto Scaling: Monitor cluster metrics and automatically scale in/out to align with your workload, enabling a pay-as-you-go model. - Scaling in seconds: The computing layer (broker) is stateless and can scale in/out within seconds, making AutoMQ a truly serverless solution. @@ -181,11 +175,17 @@ Table Topic feature for unified stream and data analysis, which now supports the ## ✨Architecture -![image](./docs/images/automq_vs_kafka.gif) +![image](./docs/images/automq_simple_arch.png) + +AutoMQ's Shared Storage architecture revolutionizes the storage layer of Apache Kafka by offloading data to cloud storage, thereby rendering the Broker stateless. This architecture incorporates both WAL (Write-Ahead Logging) storage and object storage, storing all data in object storage in near real-time. + +In this setup: -AutoMQ adopts a Shared-Storage architecture, replacing the storage layer of Apache Kafka with a shared streaming storage library called [S3Stream](https://github.com/AutoMQ/automq/tree/main/s3stream) in a storage-compute separation manner, making the Broker completely stateless. +- Object storage is the primary data repository, providing a flexible, cost-effective, and scalable storage solution. +- AutoMQ introduces a WAL storage layer to counter the high latency and low IOPS associated with Object storage, thereby improving data write efficiency and lowering IOPS usage. +- The WAL storage layer is adaptable, allowing for the selection of various storage services across different cloud providers to cater to diverse durability and performance needs. Azure Zone-redundant Disk, GCP Regional Persistent Disk, and Alibaba Cloud Regional ESSD are ideal for ensuring multi-AZ durability. For cost-effective solutions on AWS with relaxed latency scenarios, S3 can serve as WAL. Additionally, AWS EFS/FSx can balance latency and cost for critical workloads when used as WAL. -Compared to the classic Kafka Shared-Nothing or Tiered-Storage architectures, AutoMQ's computing layer (Broker) is truly stateless, enabling features such as Auto-Scaling, Self-Balancing, and Partition Reassignment in Seconds that significantly reduce costs and improve efficiency. +AutoMQ has developed a shared streaming storage library, S3Stream, which encapsulates these storage modules. By replacing the native Apache Kafka® Log storage with S3Stream, the entire Broker node becomes entirely stateless. This transformation significantly streamlines operations such as second-level partition reassignment, automatic scaling, and traffic self-balancing. To facilitate this, AutoMQ has integrated Controller components like Auto Scaling and Auto Balancing within its kernel, which oversee cluster scaling operations and traffic rebalancing, respectively. Please refer to [here](https://docs.automq.com/automq/architecture/overview) for more architecture details. ## ⛄ Get started with AutoMQ @@ -235,7 +235,7 @@ No need to bind a credit card, no cost at all. We look forward to receiving valu ## 🐱 The relationship with Apache Kafka -AutoMQ is a fork of the open-source [Apache Kafka](https://github.com/apache/kafka). Based on the Apache Kafka codebase, we found an aspect at the LogSegment level, and replaced Kafka's storage layer with our self-developed cloud-first stream storage engine, [S3Stream](https://github.com/AutoMQ/automq/tree/main/s3stream). This engine can provide customers with high-performance, low-cost, and unlimited stream storage capabilities based on cloud storage like EBS WAL and S3. As such, AutoMQ completely retains the code of Kafka's computing layer and is 100% fully compatible with Apache Kafka. We appreciate the work done by the Apache Kafka community and will continue to embrace the Kafka community. +AutoMQ is a fork of the open-source [Apache Kafka](https://github.com/apache/kafka). Based on the Apache Kafka codebase, we found an aspect at the LogSegment level, and replaced Kafka's storage layer with our self-developed cloud-native stream storage engine, [S3Stream](https://github.com/AutoMQ/automq/tree/main/s3stream). This engine can provide customers with high-performance, low-cost, and unlimited stream storage capabilities based on cloud storage like EBS WAL and S3. As such, AutoMQ completely retains the code of Kafka's computing layer and is 100% fully compatible with Apache Kafka. We appreciate the work done by the Apache Kafka community and will continue to embrace the Kafka community. ## 🙋 Contact Us Want to learn more, [Talk with our product experts](https://www.automq.com/contact). diff --git a/bin/automq-perf-test.sh b/bin/automq-perf-test.sh index dd065326fe..98c6b3d0d8 100755 --- a/bin/automq-perf-test.sh +++ b/bin/automq-perf-test.sh @@ -23,4 +23,5 @@ fi if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then export KAFKA_HEAP_OPTS="-Xmx1024M" fi + exec "$(dirname "$0")/kafka-run-class.sh" -name kafkaClient -loggc org.apache.kafka.tools.automq.PerfCommand "$@" diff --git a/docker/README.md b/docker/README.md index 5c46ef954e..7f1dd76dc7 100644 --- a/docker/README.md +++ b/docker/README.md @@ -114,6 +114,29 @@ Run `pip install -r requirements.txt` to get all the requirements for running th Make sure you have docker installed with support for buildx enabled. (For pushing multi-architecture image to docker registry) +Use local code run in docker +--------------------------------------- + +- command run in project root folders + +1. generate tgz +```shell +# For example only, can be modified based on your compilation requirements +./gradlew releaseTarGz -x test -x check +``` +2. run +```shell +docker-compose -f docker/local/docker-compose.yml up -d +``` + +- After modifying your code, simply regenerate the tgz and restart the specified service. +```shell +# For example only, can be modified based on your compilation requirements +./gradlew releaseTarGz -x test -x check +# eg: restart broker +docker-compose -f docker/local/docker-compose.yml up broker1 broker2 -d --force-recreate +``` + Building image and running tests locally --------------------------------------- diff --git a/docker/local/docker-compose.yml b/docker/local/docker-compose.yml new file mode 100644 index 0000000000..f3666f232c --- /dev/null +++ b/docker/local/docker-compose.yml @@ -0,0 +1,160 @@ + +version: "3.8" + +services: + localstack: + container_name: "${LOCALSTACK_DOCKER_NAME-localstack}" + hostname: "${LOCALSTACK_DOCKER_NAME-localstack}" + image: localstack/localstack:3.3.0 + ports: + - "4566:4566" # LocalStack Gateway + - "4510-4559:4510-4559" # external services port range + environment: + - DOCKER_HOST=unix:///var/run/docker.sock + volumes: + - s3_data:/var/lib/localstack + - /var/run/docker.sock:/var/run/docker.sock + # use a static ip + networks: + automq_net: + ipv4_address: 10.6.0.2 + + # create needed buckets + aws-cli: + container_name: "${AWS_CLI_DOCKER_NAME-aws-cli}" + hostname: "${AWS_CLI_DOCKER_NAME-aws-cli}" + image: amazon/aws-cli:2.15.37 + environment: + - AWS_ACCESS_KEY_ID=test + - AWS_SECRET_ACCESS_KEY=test + - AWS_DEFAULT_REGION=us-east-1 + command: s3api create-bucket --bucket ko3 --endpoint=http://10.6.0.2:4566 + depends_on: + localstack: + condition: service_healthy + networks: + - automq_net + +# Only comment out, do not remove. For easier future testing +# test: +# container_name: test +# hostname: test +# # image: amazoncorretto:17.0.14 +# # image: gradle:jdk17 +# image: eclipse-temurin:17-jdk-noble +# working_dir: /opt/automq +# volumes: +# - ../../core/build/distributions:/opt/volume_libs:ro +# - ../scripts:/opt/volume_scripts:ro +# command: +# - bash +# - -c +# - | +# /opt/volume_scripts/pre_start.sh && \ +# tail -f /dev/null + + controller: + container_name: "${CONTROLLER_DOCKER_NAME-controller}" + hostname: "${CONTROLLER_DOCKER_NAME-controller}" + stop_grace_period: 2m + image: eclipse-temurin:17-jdk-noble + environment: + - KAFKA_S3_ACCESS_KEY=test + - KAFKA_S3_SECRET_KEY=test + - KAFKA_HEAP_OPTS=-Xms1g -Xmx1g -XX:MetaspaceSize=96m + volumes: + - ../../core/build/distributions:/opt/volume_libs:ro + - ../scripts:/opt/volume_scripts:ro + command: + - bash + - -c + - | + /opt/volume_scripts/pre_start.sh && \ + /opt/automq/scripts/start.sh up --process.roles controller --node.id 0 --controller.quorum.voters 0@controller:9093 --s3.bucket ko3 --s3.endpoint http://10.6.0.2:4566 --s3.region us-east-1 + networks: + - automq_net + depends_on: + - localstack + - aws-cli + + broker1: + container_name: "${BROKER1_DOCKER_NAME-broker1}" + hostname: "${BROKER1_DOCKER_NAME-broker1}" + stop_grace_period: 2m + image: eclipse-temurin:17-jdk-noble + ports: + - "9094:9094" + environment: + - KAFKA_S3_ACCESS_KEY=test + - KAFKA_S3_SECRET_KEY=test + - KAFKA_HEAP_OPTS=-Xms1g -Xmx1g -XX:MetaspaceSize=96m -XX:MaxDirectMemorySize=1G + - KAFKA_CFG_AUTOBALANCER_REPORTER_NETWORK_IN_CAPACITY=5120 + - KAFKA_CFG_AUTOBALANCER_REPORTER_NETWORK_OUT_CAPACITY=5120 + - KAFKA_CFG_AUTOBALANCER_REPORTER_METRICS_REPORTING_INTERVAL_MS=5000 + # override listener settings + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,EXTERNAL://:9094 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://broker1:9092,EXTERNAL://localhost:9094 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT + volumes: + - ../../core/build/distributions:/opt/volume_libs:ro + - ../scripts:/opt/volume_scripts:ro + command: + - bash + - -c + - | + /opt/volume_scripts/pre_start.sh && \ + /opt/automq/scripts/start.sh up --process.roles broker --node.id 1 --controller.quorum.voters 0@controller:9093 --s3.bucket ko3 --s3.endpoint http://10.6.0.2:4566 --s3.region us-east-1 + networks: + - automq_net + depends_on: + - localstack + - aws-cli + - controller + + broker2: + container_name: "${BROKER2_DOCKER_NAME-broker2}" + hostname: "${BROKER2_DOCKER_NAME-broker2}" + stop_grace_period: 2m + image: eclipse-temurin:17-jdk-noble + ports: + - "9095:9095" + environment: + - KAFKA_S3_ACCESS_KEY=test + - KAFKA_S3_SECRET_KEY=test + - KAFKA_HEAP_OPTS=-Xms1g -Xmx1g -XX:MetaspaceSize=96m -XX:MaxDirectMemorySize=1G + - KAFKA_CFG_AUTOBALANCER_REPORTER_NETWORK_IN_CAPACITY=5120 + - KAFKA_CFG_AUTOBALANCER_REPORTER_NETWORK_OUT_CAPACITY=5120 + - KAFKA_CFG_AUTOBALANCER_REPORTER_METRICS_REPORTING_INTERVAL_MS=5000 + # override listener settings + - KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,EXTERNAL://:9095 + - KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://broker2:9092,EXTERNAL://localhost:9095 + - KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT + volumes: + - ../../core/build/distributions:/opt/volume_libs:ro + - ../scripts:/opt/volume_scripts:ro + command: + - bash + - -c + - | + /opt/volume_scripts/pre_start.sh && \ + /opt/automq/scripts/start.sh up --process.roles broker --node.id 2 --controller.quorum.voters 0@controller:9093 --s3.bucket ko3 --s3.endpoint http://10.6.0.2:4566 --s3.region us-east-1 + networks: + - automq_net + depends_on: + - localstack + - aws-cli + - controller + +volumes: + s3_data: + driver: local + +networks: + automq_net: + name: automq_net + driver: bridge + ipam: + driver: default + config: + - subnet: "10.6.0.0/16" + gateway: "10.6.0.1" diff --git a/docker/scripts/pre_start.sh b/docker/scripts/pre_start.sh new file mode 100644 index 0000000000..95c9a3abd3 --- /dev/null +++ b/docker/scripts/pre_start.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +echo "[PreStart] mkdir" +rm -rf /opt/automq +mkdir -p /opt/kafka || exit 1 +ln -s /opt/kafka /opt/automq || exit 1 +echo "[PreStart] file" +for f in /opt/volume_libs/*.tgz; do + tar -xzf "$f" -C /opt/kafka --one-top-level=kafka --strip-components=1 --overwrite +done +cp -r /opt/volume_scripts /opt/kafka/scripts || exit 1 +find /opt/kafka -type f -name "*.sh" -exec chmod a+x {} \; +echo "[PreStart] env" +echo "export DEBIAN_FRONTEND=noninteractive" >> ~/.bashrc +echo "export AWS_DEFAULT_REGION=us-east-1" >> ~/.bashrc +echo "export KAFKA_JVM_PERFORMANCE_OPTS=\"-server -XX:+UseZGC -XX:ZCollectionInterval=5\"" >> ~/.bashrc diff --git a/docker/scripts/start.sh b/docker/scripts/start.sh index f6ce57665d..7da5630727 100644 --- a/docker/scripts/start.sh +++ b/docker/scripts/start.sh @@ -15,6 +15,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +source ~/.bashrc + script_path="${0}" # The absolute path to the directory which this script is in. diff --git a/docs/images/automq_simple_arch.png b/docs/images/automq_simple_arch.png new file mode 100644 index 0000000000000000000000000000000000000000..f83b3df97e4876cd7ea0c60155beaa1b1b6e0bf0 GIT binary patch literal 154422 zcmeFZXIN9)_B~8T&;W{vfCwl>L8OCp6e-esFDg<)htR7ahy@Voy@!?{9YRwP=>!O& z1qe#-HT3e|=l+f!@45G!^YMK@T%HHmB#Z1-=A3JeF~`~w8fpq4a(Z$+JUq}N#fMsW zc%(9Tc!YP(kpQpsyea#Jhj&5BR#sNyk*qAUhO3j6t-U24o?=9tKB<9r$CY&Bm}nUr ze4^aYNurCl@pD7_9;~bCJh;RhdzJ8htJxJAlk=Q69?R;>T`mz6e`>9{;$N1TUGkc^ zev6&heL}y+t0n|*E0p+qBD1uQmMeMM;CZSb0!Svv zUUb~=q5vo71sMjhvx!O5@X0>-h;r5G=$W}{NE>pVAn4)!)ZVW7x%+;2cn!&1muT^1 zVz;cc&p*_dZtkZz;+=%_g5+|6)sJEX#E9iS1f?tk7lc+pF$8EpQ>?~dBVS^3+? zF4_T((Xk>gMns!=_4ux+-wHaf1&^|RNuK?E#{0Y6XlnE=Z>fE9DiVW_Ont#b)$Y7> z&j{~Zn`<*Q#Ox6fP*8=ad?ei^x3YPdU7~Ojx~m;w_;niO=6% zkjpIwd0!%YdHUt?Q|5k>;kHH>b%_ht{hzpfrmH>blW01kPNzbgqa+(XHxX+5o@l#O z+LDd}krj}cePvO4h@d7k`ED)K8ZkPHA0Dzy=sQmL^~rZB-tzO$3|}?m>1Lhfp48AC zJBwTnc&5Owbp8G=oAH^17FrK&X&`5`%(Q$HDdb-}S@JDCW4a=sK#DS5HgNaL6^o-X zff2HmYaUm@1nw(Whu%OQE9824K_ug0@GKj?azEG1$SJ1Hu(guPn1e`2WMicHj6dXybzTCikMzB5WVHn9 zQ|Ad}&b=gZVWv|F9ku=hxl5w`Nrjj4a){Okf)f0=$2Ug^Jc1@3GuaS?e4a4%)d<`cC~;SW%`?0gGKkq&(d%YLMJ?ADIoi7%}$) z+U%}A@=B747sA;bedia1DjEEPDIVi#e|6`j?!IpE_|Bb6%Afdo`Ggs`K6RGSms|(G z*gGH}dW(C)dgh$snC zq93-is<2(A?j*yUBP9cx4$68duwH_^)O#qhe}|pAB0}J$a>Q~3=*#q%yWekq$BTG4 zaC7usgW@|@iozfr6(fZ)IWc*J$`XsehL#?`o~stQj*{jj&K`YE{joxKoyGD2?L6c& zNICIn$hAjv@1XA!-q*w!chGi7&LA&?UeUC^zNFBR-<_kQv2+`H%k#EVY+Rgb+(cYX z9C4>x99Qh#tqk3O%wtVC?N+@Pc`-#dG!661O=oL6x#L81IMFdn%BopcG!ityg&c)@ zg-klOMHEH$y6H>T(41&_bXO4%y4~)Iok~@Not)jQ-4Xh35w7q(nj3AnbPYGUq`Snu zBr-@UXhUxyctdDQFy3xu@E&wi<$bPgAzfw%%A}x9Zaa7GCcV5sPMM9#7{Qp{*lnYz zE1F$OuVHa8-J&YvV(nscV{MF~_bw_VWXmJA7hI7E>;Gj!RebRYY5iM}; zyy9AW%)2A`Nnsg~J?QBLtE={W3%pl(1^F-VU#5c#NZ5{8=UU8wV;w5T778>=`Fch$ z29F!Y_7gYV5W;DXka_xjn8g!z682B*m)`fVr?F?K!c?YJ=Tn3eCk1j6`MUYL^-`*m za48Ha;VDSLQh{cN`_+#Po*9-@-Lu%79O&flcYSC1epZ0Xl&f0zoo*W16kJplg}b~= zzg#T*kRePpvd1Syxw-)M+9BQ{ZdFcvuotcNxMmU#zWgNi=+?J_WKEG3-Ib)>7rT&M z%4_b|K3|&?7Zh!{<{%|zxzpKiEg$IS?$QkzU-GhkGnW|6x7}75>Am`F`3t1T6(JTc zChWYDTGnG>E{YM|P7}F@bvpiTG{@-y^IU}pcq+RUd(?mGd-{+sp1&7C zJ$^DiU(4iiw(xC9bE*tE&7fVI-gaV#DnxqS=Y`738d7$J{UmutFL(WtxI-=2)%}kGi_6D@ocVN=)3Tevfx53 zlMQ_$odN9#)66Bon+smNEcXPxEH^;aEGb=Z%f0a07v1Exl)7YZL9h${SFIS>DZn6w z7TM4GFR*VvyrhyBlzXn|F;mp;tEs1{olPM}os*MYoSpIIO+*r%8+-f}@!KBOEA{u9 zYiC_Nzr&{4HXX?9$O_4-$kwS(sJkxJznACeQ9?wQWZY47RAb3w&9zsiyUN5H(6gVe zlQ8&waGAP^I7)92DlV>8B4`TvozPqoIh>%R4id@R=j@M zhgOd!ATgO2awhoJ_OmS`$^pugbXs}q!$J++{)M$S)k*9E^F4mNuSLPz_Rl;HSE@y= z+^+O4CNT(lwn=n0qj?(e|3|FC=7|FTK)z-P$U0#ry79?N#$T z2IFrAay$NFR@d*Kjou)pg!+gzp$6~d2EDYyehi4h|sPxb2o0uGiv!oNO8=wn9lc1H0 zu%+{`=7vw6vM2H_Z}S+W9WL1;>O^haEZ$nG9$fhz|Cw98n%!P#(%-CU7EhMdLxlino={kJSG{NI&-4R|>iXG2HX&^{f-mQ2yi8!p|{Iib6 zb_I5anLgap@Q?QRkbBWKb#eq7iyoXv9Un(u8W zIVYs;RV3&dE#tcQJMZtBnA~b{j`wP24pENl5;Q(bD~lgd97tZ|HSuN0?uKZT8}~fj ze9GNgGn*CX#e4hpUF)Rcrs51d2vhr}nSjD7u*q z+509=eou9>$GXPVsS#o7b+nyU=Sk+NyjFtu|bA=bf7&AtwN@zGp>PF8h8ATx6^5}X)}&qoel0+9OmJ=2m=pJ(ft{N z*lnd_yA9lch>ufsGb8{bOeN%eW;$7qTvwJ_@E_THx4GF3{i3cet$y<0P-_!C2VSn} zbMhqJCLxu(#;35$74#|0>KWdKk2v1oGdvsTKzTYI>{-W!ck70h=Z-nxNv#L)t;Bc z!a_x;=jvYAd;9W~mHc^CJtWKP(kfQL^+CG&Sb zuHh4Cgg+Pg`wu+FOtnBjlo=dAN+NIv%3l0(L4$Nw~zsdH38VwW54|J}3z-4B+bWyS;Th_~-w{M#9% zlKFU%84q&HE99TnCHSrk9{qx6^ybaKTRourK{sXamNYg<{%P?DOoQ+Q`_`GK|8boa zg75~jra=F+cti+1e49+{{jX8~G?qv_`~ule4F9xv=iKoL`ed3z-Z1^sSpH{-e>Io? zGsM4hm;V{!Upe*vBI4i0Z3X5p-ul5R1G1SxHgxsKgg8S~RCMd~ zh^c+h7WHhXn{IMV1j`a6Lrhhm#rE*hrAsrx!Gw7b*O(HMDx-KFV|H=RwE{f-HnzXs zH=UOWG&MB`>Xd2e_CLzdiL6~@c&RtTEZ9yIMq}k@(0Canh4r2{_1g8f?{pR?UP2f1oNZ1sbxlV}a#q=)i=5G!`3Rwc z&qSH@0ajL4odQTH4yM4JEaZG{cEZkKgR?NR#wQ}b6Ab8tF{;E!cWY~lwd*+P#XdRCbCI34;3|FrNV>--Z~_5&&uwe#gG`;{ z#GAd^;Hbu$Auk|nlb3*U^ z*E^!j69NwPYYDR96iV?O{X6UZl7#*EloHIPrw7P^Y`NFu)X()^;^{c)cz9Gx91Nni zI@*|MYTKSXd9ro9zn)Zu&lG!Ia-H+TTS_3$+9#;9>Rcuy-R%ITZl<*K^ij1FR&R=O zJq*OqVf{Y;QHt;?XE)P$aNuw)!C&qenA^^a0AE@K4)@#tbj0sHFmFXyhay&!cAdF( z(86T)uC}(8eA9E0<^~?R@~}q)8$)Y$)qKi|jkPjH|LjoglQlk#6tjj4ac{ch&F0ft z;_8~{n~*1qF|pO z;jU?H&DlZx@e-}H%U!**Yx|q(fw!?ola#6Jf0K2|Skm;p>f_4^>(n=g@v_*YTNday zH+3gT8f$556E>0S5cW;UG}X}uHSJ9v5r0PZ^=TL%Jldyg_lz&Q3Ani(I@@w5v236Q z+o(p0kYC_nYF^iQA;$1`w`99qaD|au`sARrC$?s@7o8#D@?fqfg<~F>;n{S4Mg*TR zXlbWL8>;Gd9Ma>SfSBW0_7`JqxSu&dHV8LW2-rJHU>iHehp!?I+-{tFd1P4Wd_S*Z z>iJ)%pd0MY^QMh6pYF@%%D;88);6v5ibxe#z+38BuJ(tbGw6eQ(}hr?&_bKIU`h&C zscj&u4vSL44#oxzk++LjPUhmXUjO@GX(j3Uf!t@vI`JbFPry}x)~%hd!R_c`+q@G#t;B4@i@k*e+Dfg z08nez9Ju&b5%7l_##Z6s+ZbZ;d;dCM{oXWyni&trz{dDb1NvVlf6VP)E$9Dc@~>v* z|2*q|Hu?AS!#wpr@BZ7k_P-GH|LYJWqi9GOwEf>$tyuG5@Gv~0sgjtK6r+E@$bD1x z586#86fYL~$De;(eJl;iFXHD^XqW-iU{_b# z<7@x1foo(D3#xQ?bybvd>J};5*w|zZ7az^f&5fPN!Id>MK7@vbzP!$G`LfPJ*`zQc z!yH>L0$bS`@|2ekYuNnC!0+;%sNOfjckSIKGAbcAm-Wc7RH4o5c=X0Xe-6LXsl;gQlg0~IH-auO2#;GQ z&~rLss#UGJ6WM`$=l#mcHQUkhybIUvX&g?X^s|d}>kD}`+cM))yY%lQ7z$XAHC1J6 z8&}zB4&R(CF*w`^Fc$cC>l@m)-`FQw1Grzi;|wP(Owzf)C3a#B&>~EM#{d z=S;Xe8y>le@pQ)&PZ`5{O=>rky|;Tro?)9#v`Wn+3(Xo4k^!iE)pBd(b?>D@C>uL_ z{#3A(lx-ry2gm8X46AuI{q?QLN}h#LqfeFn$cOuDO-GL2ON&qd!h-9~{VJ^3YgOVa zD+Q@PTPZ&%CW?r9?Ri+6=+YmWQLoyZlDaN|UBET(P5QceW=L(#P^&Asx|UOa@LB7! z{sj~rvj99QrQG3w*HR57);)^Oq@#?OfMHGN_Z}NMr7f>=8o4 zN%F*c)5JaQ;oIOAY`nJ$p}9y&f7Ujl%+1Z7Ts@f4?mefo9Ml3reYJQV+0ju}daynB z2;E+^HSUId1WZaJW3T5g=DaV*eAeuKGC-5|-?ot2^Mdvx8UeUs^Nu3jV#|Tt4l8mD z>4XS&!%TJnvOLrghFi{4h`P@2wegrCU@5ZE7t0-T{Tge6YU><8GRQ#(xr?o}JR?##pM*PxH%c3}UR63^hDMcJCM115#525qp~Yov&uQQr(HHIf z;ln@4gIJlV!9w@Z=D_BenK{LJkJVh=;x7b^3$c}lW41$+<5gg&pzT2JM~^*9BmcUm zt>n~-@N#p7(D1JIi#KOCIntdU=uY%?>p%k5ea~)Ke0>w=ToKh|viiOCb$O>sMm>05 z?MOFA{%tOJ*s--}>H=j{v~Rah9*0^g?~~Pw6(ozI0>m3~E-=X@F;eMY;MPVBz;rju zEmHpCC%@a#t*noqLP+JzK@7s>JV?jy)6=W;spo0^Z?{D;?TwpEZL3im!9|vR5Cm

hHFxMex>beJtFpWkoa*%m8>)(eYgHNmaCNB%Mc`rKC&mb(~KySX_B64kNSI)+!fX?&OCj)T`Z34Ki6+}Ky<4%nI z1VT04Z|f0Y@Da#D-wDJt_b&^<2HcbTA_M7v0dA+X?dC{q%-chXhkNyLNi}H^jW>~(hY^C_0>BI{jvc&IZPcVdsrvm!A!{NSe2DSU+IZ63E~%n_*f5kt0!%f z+|OP4=bMi8L0ou^UVO4Uv!juW;_Qb5xce^v~zii@D05oQN4V*(ar$4L6$mXP6uMOjNu! zJ}o&hYw|OUye`Q(- zd;g=|QMU!}oohf^QU1y1$q9{hikgcrZb|oOph-Pl!glz?q{=Q%(xhc1G4X1lW{$iI z!h-3SJywa8N-I0(-nl>Q<>ytV20kpHyQhZ+bhpaV$|`xUZr9(@F+ih$N$~Oo5C~wW zibfo6uWGRh3F!(70`5?CfnLzyc}R#H^n+=o%{!CYXqFCk*0--;*BsIAN~eiI%9ou0 zisp0JJ0Nd-Pbqz@!LE|X45A-Q07&PK&1nu5$ebwVZKND>wuqj)#K{ShubOfLJTw!Y zW@u=bzdo%t3*9wr4v?(d?iJ6o?24bHm>2(LEXOy1RDxDs?3ZZ#!^FtRQ-P?Gua>Rg_D`SXB8G68ubux5>!Da6xSW1HL4sZ5L+|v)jR$7Cf3Kg)qfd=l3Wm8 zrH;Yk<)16@V-%eAL1Aw=)N;GvruFAF9?Hpew7yR0<~lnn(#`31uhdWT|J>4IIkNGM zz2U<9J?VX!Avij72eA-aoulwJMl+{P(eEM(E0&fdLjPRqc;n_knYMM3gm(-;77oJs zmR2l2Bgd-j9Z&eVTzr9A|U+B#8Amhkp#b{<>@T5 zXj?f<1-6=KfN`Jvfb0o)7Z+!HC~+6CtEzB4flqd_D9_4+M}Od$0uuu1{Zr4G?TCzf z%b%K050`=TaU|%}N%lP*i6*{LUAhJQ$}heDzq=AB%E&%SyD^-h%Jm>c2yA73x{~l~ z;(d#U0NZP*WhVdS-hWBQ|E=eE!Iyx6AEON3;s6?+u94k*@NfVBWrc-J zJEu#p|ETt8)B*S7+B=zF`cv-3859&|48+u!loT$Y#>Fo2s`guvd&*i4i>8-X?F7;d z3e*Maqrxl@>MpY7A>kCAvA5sP_I(QBx9NMlh^fi29kKJij@Qw*)XM2vqojfN=%b~y-GU&qwaCwcN(j--P*;R#R9|XssJjEAciYIAx3hD*>HUm2p=>15!6~aJ`n2cA;kAa{Hz+Ro$ zvr|uC_pg-L?iOIT@hN!BF^4KKte8RTpWP`eEZk~b_g6QnwDE3{nrs?r1Si}vsnv+Q zCUpYNQ%y9`0OWbyCCJahvan%Tb6_f*C<_9WWkR0JJiebjWH0 zCgsswPnxx9<>&GS;>;HmlU?$SZL~aA?Z*e(v%oCf)7DW5z@X-IXKbcnQ$7<29Jk{&|6bOQq*r}+gxgg7f75e^bcr?UZyPkE+aPiscC~1$g^+xhUPyxDYsO%Y<)$OKPDi5-S?Qx zEhAj9@<6iBV`o7#iO&MzcQBi_mtt#jY<+{to?;Zl6FqHmvgE6eO~)=FX7ZDV{%F9N8moQ$G@L0LYKXr<)wqJN6Xg8>s9vINDAPM3>0rM{LO9!4d zP`B5A&2HOYrZ+{t>>8O+bQmv1pG+?9o9rRC>jpOKMD#P@7CRTdZgZ}EfxrqWhZj-dR5Q?Ud7?n@um~-C<0QP_DV!h za8TtUMqUe3Lkm-aP)3rE$H zr@rf*hoI#=K#3Fcm{Xw2wsi$?fF7Jl9J^}r$oASunQ65H#;WZq4GkMWDwJ8cnal3< zWy$W=_N0iU*kRoAvv*@l%$v<9t}>W{wHyT9hfXQ4-MdBqwXVtSttymf=mesq56V_g zxg7!FMm75Gjx3E2&0Vh+L4lUNh0dW?mxo~j)xHn8f*~e{MS9!_nOph7q}@W6NjM&y zl-=2w3D!DswaMt@5Vvv#pq)eO# z*W=LZXv1Q~D%&dORF^`-oUoIU;l#8wZa@?itC}J_=9|Yb2$v0y98%9v-<#5`0cRYF z=NzGV{qyqjIsm=G%KBGRPh2^;bjjb%(0;VMRj1^7S(ZJJsV9A{si*`eJYCPTjxMnn zsIsg=3RXmx=s7I&`M{HfEOvIP(QhI;q|Zc#*To#$A{jmI7@fe{7QF!jvJRXVx{&vi zwV-(fV38XWs_;J%=9vTJHko-(CX$7~_8a+G;Uu@{C2U)vVJQ;`kM|=+W#x}Q5irQq z?P^w5SR)f(A>PL%k4+Lesa{h66`yV;G$($jEZ%8empVA!Mm}4NVXIyT_5tN0!!E5T z6huns*n2L?`q^MXzw6^Osoz*@OWYD#xB%ticX}{y_L5pr;MSW^Dse?i7n5P6OpPwfKJm+hYIC_d|{%<9E1c!DAbz|zw zc3$p%q3@1V+Ly^?gc=5MEJD0Xn9|Y}Y>oL99tb+fcv7m5tnVdy&miJh$&xggVV&VWq=xqS8rs9;zw?ePTfYlnTig4xT2i0Dg=tFc zMr)kRsVgBLru$?Jj)u3FBpHS6YbAC^EQC{i=hJE&7GR7EyTDYoC-PBZe5ov+^Xj8M zXdR&oqno$W)<;|xgoRK?m9{AMVfZdI%>8g6f8H0E2vsiz+YG1xyH^T;;ryPgLBpKs zJrD$Hrr4cP>kI*~{iH7QhaqQUXx>Smiew2(L0GsC_u+*lKKd1!8g!vHhGfewyNUiD zG(Z2!y$oyfgE^+o@jAD(YX2hDPM(sTVq*>y{+9nSbu(|5SlL7ObeX>5|oN8ZwPhR*Va}Ho=yvn zr)hs&is!IoQ2-WTL{}`8qYzb%)NS-RSUz0Zcdw6pw7%Ei?bg%ez3YR!3bMnQ6+(9* zZH%5HzM0EeMWyhQqh9ed!M2_nxS^r zyfZT$?%NY&izp5~yJ5J0i=m*5xW`I?jfSY}jG}ScNRe(fh)xI(XOCMJiM}5~GO8pB zx?eS>Da#JKhZ<_|J`Z(OFTRcqJo9G=5O(Sx*KwkV@o>P5T3GcY-&R)HFQAF#!@_O@ zxuk^ptlto?TJgG|y*un+(SB!3zL<|e$j-j#>Ts6}b*Hy5NXBz@%<_mq%)Y>VJs{jf z`2OVcgKbN6k*;vx+c$42KUtxRG^-C?j`uf(chKM=s%qmsM=MD*eRVd^ZPC%~%Tm(d z4Pk65G$7<~3pn>dTS#Nv751Sk{%#zwUtoM)zUl%&ZU-JSE`jXJo`Gkkn90=&qvj}-L$&G zMvUGi$EC4ttJL1}?7ilV)IkG972u>SV0i_T2C5y5joFItr~BS2Oj~lp4yS6L;eYIm<8p}mU{;}5 zVuzY<)uky$zB6k9vdz`?&;IQD>vpG`08ipJ-}^4k+~4xS^l_$f3%F5lFTYtGc(LM) ziskn0M?RC@P#kJ+4CvYI^R#zkZgJX51#^GozEc;NxZno4+y(^xqw`yU%E^LdwcUd3{oVVmw zi0-I(d-Te(i*+FM;E!u`rU>`UgTr}ECwAB?N8vHm9lw=qNZ1)S%Z_`zwfoP@1z?0!X zA#}uv1W;o#M2bBANSJQ~Z-U97I_uIUeUmrbb%Oncbc}$4pf=;Z9uje$h7HF4P9c3| zMMYkV$QXcF8(Y$YWp7`oF*&lM$s%boI#`f8DFoNgtEa8$%SwO}Dj)P4qhua&>54;^ zR_xb?VD$}qYxVq3K0~b#lMeY>?oikYs!$V`7o0(oN&oQeb)oz|B^{`Cl!$(h`{$Ez z3L2Q;pl_}!A$*GS6MJnkHytg%ecZf7l(b_p3#6VT=(jwb0lqp~F+9m(u7_wA2AxW@ zJcacQ&QE72+35{yqK^vjD@lBnRUZono)R2MEH(}(>FR?~`LKtHGF}6;<|B*Gg%$>j zzLfS-_MN_Z(Yq4ngfxGz_FY>9XJ8_zITEcclG5x9*3r={ zvMZ8(i$OwHA*Rhzl)yAcQFlvP+za#?O$FUuX_#B$I-c0tPj4)_A}^s03Lr3|fRN8J{!)RlC^2EkSMdlziRK`lV9#P?n)Gum<3E`t_>P zJp9Yw>~zZlpx+uMZ|b8KW6k|>Na1Mo?acFlfibrHGm~QERy!R*k51fY~ z|B}d?7imbUGrX-LLX$SOa+ndmMW~T>L!V0^MbOp`+A8_ zWIg(&UV2s{bR@0VAA$cvqhy@wSIR4}CmPOpz-@ zm(mBwrBtb%^t&tR0PT(jaT?-~#XCJJRZGEyN2)Xyo5i%R#q)>QR|YvB>F*b|*~-ui z3r>3U7kfJO7wIs*E^ciVssclntU=FlkP6ihyR^@cgRNO%L6mKz>3ln{ZY$zz5u6MI zRcES0=?EigmgeXd0nDHF%5blrSr>DKdfeg6Cxw-bWAp=xCIAtM?wgQ2=k9Mlq7D*d z@}9X!-F}t=3J`K1V%`atKF&&l?};YyLtepL2ZK(mlJ#ybBT|i6MvI3h=IGa5KdY}o z!$eCbY13Tw2_L1o8ivRt^C=pA_O^UX55@C!6W4}IrUyUNrh899KmY($OVQYFK*7`k zi!BB?)nH=~C-}NrhLq*=;bWWuq+SmU&2uQzp0N}9;JaUl!66Aj_uX7wXEP53J`YP1 z^!eNV;5cM50bY<~jM1Xu$16WJwtcFXZzJLmkfl*OUf9AQgb;F~o`w)^Ca5ecY%j46 zi2`77VrO@bhY_;Eic=S)A{1Q+CeU$$d2i3f#KtZ-pt${^sI}`t)9o40D}!NQ2Vbe| zC76ZlZ7=#nh^-cv=Ii+5r>~n` z9GFHwgi8~%z3L~R8X^HRNfU8O$6e=r9Arpc5rTR_lFXrYO$4bWd34QoVId%6$KOV= zV4?COP>QWUrjz(Wo5;3fF1=qFtWpcF*_SWXmpy1cSzX`S>MB*_saUfCRSeBdHu}!C zo*k=uZ#J97u1XOonL~#%HP;JCN+Kd)D-fTxl+;-Jd}l|;j0&JO;@fyepURQ^%GO*p z!xofMny6$BiuYU_x6Q9fv*NNYHE-tAKsyWGyVo8DOAjs%7_!)@8}|KfhiEE3y-9j9 z@d|$3zol{DC=Wd;_F`Rbo08f64xmQ=KLRr|`y4N^qg>s46ViXrFl6#*orx30X7kiU zidZuBf)~Y}zwPk}f$Jelh{tMOa-51}q7pMW95iBL1}4>_mdDmTj=M|8Bpf)0@3sRx zNWMzy{vrs9zU@;*uSS^p*~r~ph}uLB!`WY~UwWmwcq0FkrkPp5$V@V4ReK2M0tJ2h zVnsICWXZklw}c8EwU*$!`7|u;%HH=@WsgI1(+qUQVr3mrMvr|~KWW=10`5uwP%oFF zQI=eI3FirzVh_ctT+~qVWz==kavJCJ^T0gTUS9(r%wjTw$%Cz_{ zj&3(q-XO$F|LW>!MA&CKcIWXAPeQtr1Te?rHpM)+J)Mk4j3I{KD`ilH9w~wm724EH zl0IL;idk4#cA0oru+7NH`a`Rt8H1WvQ^!U9?%PMvY^BY+$-k-Ki4O-U>6@1PLFC!; z-$|}D{mv8z2VDV<`7L1Drv4nq_VqsMP85E_0tBJ`IiCR?k0r5GpiEmls~;j<2EJui zm7=F-Cs$CcG+2dQJuakiq^GwPO14I=3w}i7WFSoDsRGfVL8~?I!Xgizz2gm=5|yFs~$V?xYDETz~cG7>6Xu*RpRVlf|RRbDr!{;dl|(& z+odOclz~d>Z81i;nm`Ho`4TN#zj=911m!G0 zCM2E_aA;Tulz)}RO;^Vt-Itqi-9oE;!~P>`q(!=}lgT8Sh=BQELQ?zMB{P;~hd|_d zrK?);ex<8mU-5k94NYiMnsi_=*{26RmF0WN;S^Xk|7C4C#|iU~Oa46Ef^@6&`7$bA&*HARalho>%qIlw6u z^9jv(_|K0iWH6=;=@-Z2_F#H2gW>!ef$8FbX6)+8cewOQqJ3324!TiD#R{vQ)M-IH z+|Z!&R7l7Spb_zap_q=RNQHxj%XEdbDJbbB+MAwsA7;Vz1oHDiN?c1pW$1UQPY;I&~p zJEYQZg;gi!JT??dU)z>g<@p-pVUKkjIvQs=ELoy2Ui${!edpw2VOgfP(3{BD^TxJW zV)M&%cC*O5YGM(J99jvs9%jX*y4C>aehRIN#66R?ZRINAluOG@M3+5}SHgencto%8 zsE;*iNy1CNQCq{VnxdpEPz8fC);UgyjRHg$!(i=he~=pVDjGs383kFL zJ}lIA1=I&IyQ>&L2x!aiHmAa%wd7sJt07@^b;r_wG&_*HCd45sR z4mF2*Kxm^agy@`P*z-plt;ZcfYHa#V00k2pH+K)w>lE^9IY;2Du;sohHLJq9F@gIn zTw9A!CJdEE2IcSxM3TqWHpKx_AvIs6nz=Zs!fuxm+3%lvm`BpkGo6yOA^4f3aV#pb zAH6dc&tuMM%g*dzmp<|3i(bADA{p%VPTF*Sh`Ia~nX~Ei(7e*{>0sc8Oz@;n^(y;4 zRb|Z6(=r)tx2VI;jc=44sC*yy!FEj`m#}|6#Q+U+>~DC%n%Mdf(xV4kLkCiQ3*f$6 z{QcO`J2L#IE=R_jHj|*T0n76B=c@j$(fFu(ZsBqwM3rqGK)4RGIeLKz^}yw&&MJxm zJ%p{|SjBu=o)$hmm{WN<`ISx@IRus8!~Sg9@%fOL8qTbeEXrpBtz5~k27Q9&ozoO( zv{O+;mpApD@{>q&X-%uZ(wB+qg`1^U(+4Mh%5YkSa^j&Y%nji{6=mEa2 z{p_@%&f^eu*A$`XXJQgOEu?~84BbnC(0Q(mIDSo)=C-x5NfcnT4<_tf2nlnvYx3H> z)UVWgUD9Vn#(*niB1LaTK2b(PRy<7}f*HmkUlqvYT`2mk5gn#i zhppN^>2ZC(?^liM-2sN&@ z=LKm)dEs0C$oSwtCi>}`ugpnm{ zv3}KwvC}X%Kymf~{7;JWEtN_CJxPWVZh>VPyLqp|Ows`j)IoM}FgTab zpbA5YZl?6nsQRi~)^aaEZRnZWq2YRillL5v{Vz>@buwC`q)$hrxDGGWW6k~bgzSf+ zAtQa?h||yLtvU^0?r04SS!dW~xNUl|4{oWeg;Ls{@bDS62cYK~l8Dk*b*}A|s5c+o zoSU6Z?p%7k>j|Fv7z`Bq6x~cd71?4>4~F|24BZ|HA=`#A)g1ua%A%dGQtb~Wwl|v; z()=wU1@kuhsDUT)-=3Hr00qLCn&uOjA*Lc(vgYXW7y1UQzb6y5&#oZS^UZ3G@18ho z%hhH;FqH2Kd2z?6vfWFBx7irIdIA*2UghTICl+lY^0p|MQ_uhL=mqB#plrD53$8m!$2B>ia7h<)H2b1R7o3)MB0tjU2m3CiIRdiMu_s^`B|!N0+#ciUzjI% z_87WuOx>?Hdj_0{o&7{g+JUEyPSWXyh;7UEvBZoc-h%4lQ}A?-Q2D4+yVmTmcSd;z z1}Nvn%{HbAZ9?_bd!LA2__B_x2#t;Hh`SCdE;_)Ki#H^UpBMW10n~bKBw21ex3S&x zW{Ya_;VQjcy}2qu?e|NUbJOvY)nx5uKNwZn^Ujaf=xy(=jJ8}aJ@FDA-VF_nuO{fL zGL>!exV_q4(brBAf7loXqkj$#d{ITDooA3EuWFs}klO5QzfJnx+C+U9P%(bCp5ADs z+IJ#c%OJRLzoL4UkiBnZYNgbv(zP^2kq z*93rize9v+&@VUXP&Xq9&|$Tg&vva|br&;-1q~K;{>C5T?cD9>9jdT>2_Ken7!%Zf zQP-kllFyL94pf;JSBj7?Nh-Eg*{ULyq50@0plrr2tBQ73cXM>o#(r&UGgdcW{=6i~ zAUXF$Tot!ts|_<&Q%gT#M9G*12D(cQb(g3Z-(IGVR_iGK<|cq@mJ-odwXBT9pat4csF4*9_=H98;-ncAIOOZ3@JlCMC9$!2$D8qqZ)PH;46w&Bi?{*grty4Kkvn zGaH19B@VCl!3kwOfLz9I+ql;rGQj~7;hn9(9LdK8x*G)NV)KL4rUfjpyFT#qPylzE5~wuAcg~iU<_;YyLT++ zuw$hiZ~Fm=)JT`zU{KNis68&AkPBgi{La=Xu=WS6d7VfP$&2{#NBHUX4&cY(Yv zdeqITs}f+ClPep2CfItCTS+%)1l)h5{8K}}1&oR4>wi?%e~<(OWQlAcD(#o=0_XSv zX^&no{Eaf1PVhI%x9yZ-*2s83fwHFs9Mvz>hE^Zkl?8;)7F-X22N7^ZN$n=OuMbUx zLDQg+;E45xivI7)OG28k)fMAaw5bWI|B>>q8 z#(KQfW|a_Zxb`NDg&9}@cf0=D+IvSuBl?|a;$)fESGeWODU)7l{2@;Xasp(sSYi90 z`b{NM=9>btLo24-3TjO6I6vtYtZ+hH)PiKD$V}=zR4a`ngy4;=!iqQWKn=$2FaJai zI9A}{FF26QyZwK|7TZQ$7iCc!Y$>W*M)Qo=ogtD=AYj2c7c zec_BK)K6gkqdPx7^QWcfu>%4)yGr*r|MAYB{-wq^2ye&ZwVTwBb@{~yf0)I;sFPbl z_ym1gIe8ZU)Ub8}Pzcr-r*8jKFJ%hJLSv1PfAq2TUrXUyUNJj{TmH>+fTuFJU^1_coikZy+V?nVje z?v@<7hK8B>x98}2-|ze{YcJLe?3uOq^W1UW*ByI@cC)q}CJw(z5Ss5F*_n7bfPk-n za^x0V12ecJ`OtzYqZw`x|%OxSVEFz{9K; zn#3KQoa|tzXMd#Q-(ToWEqtFY66Lo5#5M`1@t)@(eD1wqwu#x;*!b*)CQV~r2zB~B z6QVy3P+U{J<3-ZG)*nJ27&N#(eZSIs&P}J;Au5_H{mKuv<)+K~_Rrpi;{0s?7K&uZ z_WS!6e|^7;@nm9UmCb^Bl7Ll{B-)J+8`)v8Y_Qkf;P0yx7Dm%;@l+!8X%(K{*CN1K zH~ESdtNOgmmThOIuDzyUp(;nup7bA?6-EaPnG9Nw!GG3uD5d`E3LM`4f=fwE0&@?_C{k}D7o*F@LH=J|7k{qhFEwn?0@N^tz{u~n zj2(yhGoq3mBT*KBgm+N?&#Au|Ph`{*BxE}5e`G1i|5HXgY$;3>P205IAOGK!-d_@_ zxQYsRxsD}e7N91VA^RIphkD?g$ul=6$l5L~^*Jy&*lTI{QJ(@_#jOFa#j^#b~12zDf9ughBuk7RxQw88<%78EMroDXT0%9v7pQ@LaNuR9Ik& zq4isJX(nH((+hp<*E@4ES4G1tmv_Z4xSz5v>3$%zfLgCJbe%7ov;iap&1@EO`zuzl zJ(kLP?A|TBP|26QXwr+cONeF=3%(W)RBaTLufY@MHKV%~3;ku=n|tqgK6Z_zcX@ z(2%CeQmxT*Gj!>Y`~!Ri45y-}CusJ8qt;@iZX5rt?6SZG54X$r|z2Y4r*v@ZGO;+g-}hVejMY1;?|ftsAe7t#YZq^bU% zQl0uMQomZutt%yLB0T?OBu^`C;4=fgN=Abx%l8JY+2G7Czg@T>io_)c3dW3l9%Ni2 zADe=EBqcQx#*BX4JQn8oCdYscKnoE%qdYgyS*?y@b93Ks@&~Vyo zLR!v_`At<(Vc=;SJpa%xJ}l-d17#37g=JpU`O+Y0JH?U_1etl2wGAcWfM*RK-MS0- znN!ZNdcuOftlR|M2>P{wkD!u-4zMk(C!X*kPedrMU*fVLMRlXn-3dRB>4Gd*L1o<YL54A+N0KI2EiQ|pmjnk0|xn)Db z*PmVj+?e;B!v~|uDNG|2-dTWpPcp}g)C?flY*uewk3Cu1&|6k>gr#uLZcVA7KDNwog7VIyXU8Ctcd4s7d!NaEcVH9o;GCQiT3h4ymL(n!VIk?wySth=z9=^=rLj%Y($imq^nZ z@3i43WWZ9&I5q?ZOG@2hksTBlV0kWkBy10~T3j{K*iwwiyVB*eU2eL+*;5W$ORnoNc+LPH+?tjF)8SKAWeZ}Gg ze+M2JQplJ>^~wDT9V1kQv){#pi|MO}JS%ZceU`Lcth}t5uU`Mv?P&>&iF52$^-C9W z(=SVYuc{;OT9?Y_HC!83omchAd7k~tvNX5`b&SkdrfNNTr&oFIMYS$BY8bL z9H~4m>R5?J5lI$7te{4!AszOZhAp|=h(>Hr*x~xP(d2>ijRlY6c0npTL392Woe5L9 zTs7qnlVv&s`ppA#&{S`UKj8I0E>4;O2y&EpB7ReA0eBb<9U~xEX#WTvwffpUvB8=Z zAX0X`+kQv#!$g0G$u$UaQOdW`?uU?cS`?AAKM$>a(=dVtacaJ>*_3R*)3Asd9sCqe z{Jsp-s+8v=s8^RFtPBQ~Cbzf_$=X?V4h}S%mRFh`@ohxWO3HYL zU7;9cJ4K5wnPx$hbPj{_7$x<`2G0{PZkX7>~xP?n8BFU)6 zP-E#V(Q+#>-R<(LLG}Zg=!N-`pvPdj|Di>dzqClMJK~QPVF#eBLZZ9n6J)*i6p!=J z0<>KWb4$_bFZY*e0b?GAPT~8qQxdxtIkPN8S98L$b9;B}BF^Kys8Df`!yp}9wOE`d z*n%5UWX$ig`$hKZC)@R1cHhgz$1#c8mE#5P#GK0PRpyc?Z(WbLtfKN#zBz`zRWD_@ z_u#xfz{LJ*;UPPvp`40RVO)&1izTC=T%7SN7Xf&TBa4Q8f8oS5sb3aT0CYQ%k|zpH zo#aphqvtiL=0gQMuNp|_{iFy-Kvd7RT2^3}C*y;G8QnI-ljMTVZv}lJDoJcMZK~m+ z?8tUb7V|A3X%VhRWj>BqKXE;7YcH+s3k~v$-%LbUok)AEkJ!zPat#Njk~P=W`&h43 zskmA%Sj-rQb8srO9ygFFyQ|79C&zPZZ&5O2rTOv7VGp&jbVfG;@x6*XcX(1Imi1SzV33mdU*oaGT$iz+|TI_0I8OzIIHyI}L7+dg&03q3YcPR=g*j zrtLPKud>n6h%sKM^U&k&+97(pEp188&uzL)m)7rgh0Azg%jV5N*t63miU>DR>C3j3 z)|pY~sKTGA0xs!#Va{E%25;QEKFa4p(AT+7vRRGCX;qBJT5?UBmWLHw#_Jh> z&?wz)8Bn$Hy<7o(^~BuWiX1@gwIHeQe6N^#g1Q7xoW<5`>Q9OPJSLmx4Rm-xZ5xvuUAX zQ!tfCJZPhx@Uk6oEpvLba$P73<{wZsNw10v=3Uw?W!DorTYh-czp~;rC3|x772nqq z+*F$OtY_)+qAs7??!y4g)Mqfd#>s4Cpx?yGcyFsbx3O@L)TToPC{}~*aJX$R>PrZ| z(RJv#KuZZ?RY89V0(_?bjwVM8jTyc5qyPAQ zrHc16dNXx(Q&Z;%TZa|5;xe03tA(lBOf+TGoz@3v^pkK9TzgXmKSVs=d2xZB{0-% z!FhdkL1{^a)#Dkue6^=@)+^ASGa`5=x>;OOUT$}J^@;4QxATT#=_m2G8o1|rOS;@o zYzm6-GG9e@uDa%Yj^S8U{NNZF8s_o^-7KesG7$RdlSH5*I!4}(#g0vKQA>%aLth_% zacYSZ;ovx*5avU)ZIW+UR*&Q2;#JfoBId!93kwUysl`++TaZl^6Y;qIb&J{h>&B&A zL4wUV(k8AHr_b@*Jyy$vGtZGw{>L#ohl+zzUmaq|kAsB#*AYO7ddjMs!PTKv-j+U5ffPai4Tz%0&AaFibykuk%=yg6%HdH3sF$?pbxGt zpgUmA|NNEpnx7`H7|qM0kw{g4Xldb6cetX+4!8VrvQjnrF0T!_O*)XB%N*$3;T_^( zuV$+n;&|Nz(pHw%N@}X3BK2}W1L5+d5IKZ;x;GUrteFm(epl_%1Nnhmt+BFVNUL1e zo0vntf6%eb$@tpFz*=N)LvA}8Nd^`(e4wH%`iN}S>FcuhHCbFfN0wh=8FsU9%B4jc zx$Z|KH2?qq5SB#OUCtLKOCx~&BNt`#DB|7XHFhfsR8jrc^lK2IW#QowtKGCwj+Cju z(R^PR_vT81f>k?6L8xpw=k%kr)ExHmO<=nBssnMywGf~3+o!SKU_4^=Q|*_e{L=Z{ zIo4ug)L})O&L>nl3ar_mQ)(jgim1v$?VInyj?D3ci#gPc$*xJ4XZrj5Rcrga*8EG0 zN|asbLwdH|Mp})l(iWLEbu)*;0(;Upjvly9q~}t%Tym_Gt}hlYaO>JK4JG@v;_(W! z^2e#D?(KD=Z0ovi#mB`>eBwwkU-)!T<2U0=Ow1JsEs$3+a1sz`jQrR-eZTyFhy@@k zqZX*o{auNrAEW+}oA(j^;Btzgbhb$OH2zFo(0Y$TMtkciahHPAH~aQZU}~=SP8)bd zj?Gj~h0vPUUGYlnLT^fhCdq?c()#RM=AXLXN5 z>l`U=K=AK1dLMvmAhjPjs*d;9D_umxn5l)i+rX!wDCiJ=;!f!Jmi{av2R3ou0EJxB21DsGA$eb>MU@SBJaVz;&1kS?mD-b!jZ*SC)kR-F8o zeGWjeee#nwDXv90P0_4R?xPB4+~R%(e_W`y$^A7{*qIKFg!znEOhEv~#H{I#AC@7% zWcOVY_uuA4m+Czx-aohh+@1zaYP+SsU)t3_-s!OG6XR=|t?eMxjn1p8@i+a$`~$1z z#c?>6mMIM7rZ!}1@ft;zjEsBnA9iy2lh5ybOUjCDU&}rV&J~DBE_8HyaE6&N_EI;g zirm8vWQ&_`yzvbU%ST7rOg>-5#^zuXtMG_*cdYikC%LpN{iC?!FL$|2S7P6uM>9f9_sSUKkaS z879v?aQOrResUQbn}~#4Gf;RrjQ+xSx9Ny$6r?S3aXYMmC zo8*`fFWLDTh~H5}PPtcBc5BzxiKMbRE#^2#%L;gMhj79x=5o*&oTP^p`v@${*om0B zE1wNrD^NcSWQ%!>1!;~54GRrV?9=R!)1021+zwV};;}vtYVoGQBXNwEs2iN_o71W# zQs=DWd;v{Jh*Oa_#e$HQTE32qt0kCQm}BHBO11iO9m`a+zz&k~$*JtIU?B7UBPZ-q3nCj*oyB#eyvYw4<(2+%yt)8 zgMKp66av4I63S_K!vqdyTVz&7%%f$9B8DS24%X#}h9MIH0w?T- zvZ>m|Xh9bj7dejpo=w%A&ZuOIBnlx+{=V=&4W>l>9QuiumA&RN(=C+%=;;?okm-q5 zP}Fp%>kj`iHey9LJ*_(Pm0JM#Q3Gj7(D4c94<-M|v=qv{=Y>6XYv=vetpcMuvb_Gr zPnp(P-J#*<6B_+t9d^mm(&QfPc28=b%gsm0uf2#VCe*m!TMdhNMs?4AVN0qpg9z4Q zIrL$DiBJ|jSx&p1BLAU4TtFP^&WNv8k~Qb4iH z`{@#%*t9fppwagw0TEGgcdDRE^s+IoYJW6MeN+s7$120qtAB|0r^ETx72~mFg ziPi*RB?nw!=AERCBB`|`7Dbks-}#HTqszCw&DLhPu+E)TTE8+u>x8?Qcj)x7IAEy@ ztc#7kqwK(j{V${(?P$RUxRf>9b@$m_ei&zSyptb9bA5aB@W#n@l>g>#y*$=@ck56Z zD+0@TZz+u4_3G85u*bP$_!11Or+Xu9xv*wz23iwAj0%gw9cj`73bH==QPJ!ZOjZ>4 z2qmN7>S8C1u!3ef0p_&3dv~!Io^LQxxt5xQIVfaC*8t*ykciyTp4>fSoDzP}9@x62z~L24JLI)ED(X3u9=_r-}$H z$#q;)-(r5r3&OBF9QSz5jppy~fr5upQ&JKO3|A@^_3l49X=#DR#yZEYt$m6qF9$F8 znfk^qYpJN~Nz?Q6IalsbiJym2VG~_0H;&0K#(nm1$Tp&X!T&bMr1U3$^faloT1Tn6 znrfxLg+jW1I=kr|o#)T7TkX%_dF92q`@0A0-7G(tidIQ8x`Vn&cWZ@Pi8^OrKYKua6yc|s>F=O~%9^cvXql!`GhL{jnM{j9qn~9HN zpb@0??pfMMYlglF>pE?u$`H+yl0gn8+X6?w&Rm7fpht&Wj)2s6dj3c9O?8EN1EGe7MrbJcHz!o(ckj@TGIfKF<5IKDsz(Y( zYhl?`7Q-Ks*bVx*X%3~YboIbUwU>CVGo~k=@-M-YA(ccaQa7LN`J9%|EqR7nhM9)3 z3^i=o6!s%OFJMP4cg<#=s0E?=mQ}q35AjqF`t)`09j67wiUg9A^|11Tbv4>*SRcmS@(q zus11sri;SF3c(7BI!(2W;eqH0;J!861aSDNM}R?8d$769RgFo6xh)fDwrG76f(O@y zlT!<6q$EhVZpmyP^QMn1a%+R@zU0^1>{;F8+o7>;45W%*9CLATnd}luz}0UlMQtwA za}Y;2*)XwRric5GzVi3;y>(tPe){Y`zLX{A+l`CWj;3_g&27$O4nMdzkT847!+o~s z=hfDAAIzz{MRITHpP*Ko_UHPZO#QO!Lo%UwQD6eDvR~$SYQ~7b?UUBK{yntd1zw4o zSI?hQTAK~0iA;jpoo2wb)mRO=`ndbx)W~fugK|t-mVJACO*OEO#QiZYb0ke# zM#l%y-HD#tX#1yG^I_aXd{m#{mzz=?lJR)Q$+1%OhsyMLju!!vhZ^N;{?_`&UzCg_ z3;0ueJklpx^w$i8L~1zH?3uSA?Eh)*2C^(xq%vV z29L|IBI0jf;B#mxx#yhBjsvr3-JzsS2E*+A#+1;rz6qwTc*abPz?sqG_b~VQkg^e7 znzOA+lC!5_;5B~&->T`)aD$EQ0;eoXpROcc(++c2cPtJ0F%?>69%K6m$C*#Zq)JM# zzRp3xn%BpV08MU1J9|M}a7VgWzlX#Jqc&$kFrA)Wuz#S>G60Y;sj@)Gr^wK32YktT z9FY<5`tdetd`nAMLqW@%9}Nr%Ed`{V_7ZXM`)d>LDpEXwvsE40ETr{x;Zvn)z;Rk*+- zCd|S*NN?28f_@XZH)&8N2ff8p=BDG8A{*Kp)RUobsO=8`E^8EuzAdh(KDz7!0WWdV znggNRe1$6;!dfub7tT&3EWWr{V~b&P?LZPokT8bidUxw!?%3&=-kEKu-5x|di%e`n zrCR?vVZj0$JN6rwEb-yTZ-o;$)d9qblRck10}=F}G~<`0TM(*AyUvk5BA z*#woTIst@7<-k<(oAEbs&u^Dz7%eWsxkkMs2yuGb*K4SvP~TP%RI0w6^(Eyc10ee! z7z(`bWJE^GXg%*U1pNV2nCQZ&2cNL-%b@(W`>r^ORNEsI>HQ!&5>DUAY zN|5GT{3Oe1mU?ccNX)N^7`b0mVh*)?u!N_uf6~>}jl*|A9NtTG5;_-LanlaQx>MGJu3O(mVzQL*1W{d?^VDaA0@N6`a3?!aPmb zjYy=GC$%xM9Giwh&H%Zl*IW!wQ2q2(f>VS^U$J+8W8Qzo<%Zn^S$?Rhf8^JSz>15DE39AMQK8)# zngLNVwrRI)Z~`O9-dx7-SKY!%hdE!-`McV!6rb`9w6P=0pZbM1PXu)Szs`qU#T-BhI&aX78f2(6pG z+4GikUj4`xY1>IltUgAbn%0)qE7)2=YU#XojYe&rwm!a4do{nQi%IS7`qtqOt0=dW zve~)0XPbf?`w3u;4ABPz&{+P-HwiV8|7Ypbel7hlBvdcpZ*okyo;Cwt1o2j`Wt?KV zOqrslFE}z9|JU%1EMO2-+NbU=t-Ff01v@+pRABD{7^R-m(uPI6Bz?_iBxU}zyF2`q z4j4eoA(BIv*qiXm_!~15WkbUcb0=b*9rT>84Rbg9>Nmj-h^H8^>Njk3^g9J}C!HPC zbf5gn6B&~hh2}&N$H!9e)+bhFd?uzjDvpj>b2n(iK?2T+*YIc0%;;!%`n(anaCYZy z1(xnD%!!GS(aI-K7PW3p}RVjs99lJQg39$?<)lqo=g_ ztWzo%J2$&jX<0*oCVD>H2S)e2zy!+(_rCbHs?l38(lgEFn@z^g}+U^PN{-eetFLB6mMcVy#5R)$- zi;q&{hzRjNKoAGW?;Z{&rh@YlkY7e zol9qB`4O&&5d7HZ6ndVWRPDT`?bXwedf|)Q_GEMh%)roJsN1OGf>O{RQ6}cL6Ii6B}WP%jVnPG=?en8Z_W{DH@DQGm^ z#*02X3=k}giO4iC`iZ2)%=FR|JVMz{!{BJ@uz+a%N!3Clh_RB$Yl4b@fQgPVh z<>h5?P<^X{D=h?!^{`m@LAKF%I3A+XQwg2W6M$|68WpX;+8cs_=Ee6f8Z4n}1pn;~ z&F}$lSPiC7$Nm*thx%`9TKu&0o0*!-Qp^=@bB%?G!?0#Dmpy4I62QVB!oZ0ctjv}&6l!)c=CGGuSn=uQ2g++Yoc&H zUlU279FpoL7)8)5nWU0QWQ<2VG(3J-o^eW4ZlOz?}SWLp2 zSBFJJypCK1E6=BBZsX-pf&sbdfne!QZsWzFdZyP}tS}nmmHxPuq>=si8AA1>ymZs3(rvVCG3!Z)|i)647OA%$6+Q1C06pYUS!q(fGZBx86G z*OIlPci)aP`zp>QV)?Jn24fJiywPIKGVX{Hs{x)g5pzE7cXf>Gyp95eMUo1>wVF=c zJ1V%K=rJk*^<++RcV!l?t*-V_X!BIFWlF1&gk}KkMI=wSymJa|8y@~e8QF*U*VbYp zhU>TnI$!1F6iSp{*58dtqH|e#zz6p2e3D=`qbrk6Glz4DKE86&XECm)+%CGq zeq5W0w>*MWbi!^gxu$Xljyx_;)XT&i&_YvGq~$muhz*odyVzO_@YMr(+Y_!6S_As5XTFH{|lISRB~ya~rT;2XRx&$uSVbGsR|xH+x>7%c|b zvW@4kwH46B_tukrezFcqW67s-N<$t(lJY51TuXHsL09yckQM%!f>zhpo{MzS-0rdJ zOgiF3!=Q}$WNk;|go4=A!D{IY!WoN28eEmbuyr=qjaR&QU}3y1E?Ni^=I8O&75R#S zw8XDjqN~fj5lV7*vA$G?@=31)%icpcGVy{09gHEWiDB+K%KkAzw-K z3sP@)qjteOKD6s0rc=t37IQyK1HBz}&;2=%>!FcJ2S-dWJAFeDq|*HL9%ZI8H>I^) z9%NP2MI*wS0j>c%)03%GBnFg20GB|U8ybpQQ}BiN_B4g%P4_P#k`#Zc+W($dE~+17|Un&aYG*o>R|p+lh&ZGZ!k~2M>c!SIF#g-^Ju_AA$Pt5lBLt{fqq`)Nu`cRHcDUaTNu4 z#9Uv+W~8H>(2Y_U8$|#*YoMcG1F5v!>ix-}4y(PT)lA2VghbE;5>|F`obPv->eEHB z)Rn5@&Y%4W&^%eNtkTB^Rh!>(xqDLjJ zf(M}I9+}QqtK}YIxN5hSgUcTsc%3Oy>HSPC`@iNc@|U^0l&z%x6WY6?GB7gI1L0Wk zG!0cj1G7AsSMFo5Cj7Wj3_k<^SAyvb4+Urqf6l|Joj1~uzOikdRbtL_+kWtML0mlzED`Vh$jz}kFWXbz2GPgP>DNi>yi~qIc(1i zDgNSUf8Wxz?PH);K-WNRkq_)Y7>0cdkg6=FIad>r$tWh;zePRrf*O?xi9>H1*k z2z>hx^(z`QwgvR!NXC*Qi(MUsu zdJUHR9x)2}-n~usTkj}KERu}0p!3bMe}O^oc>+D5Eibw%3^>2Q#jVDUOVFqYt4%!} zh7=nO297b2-xQlt<(+TdrRDRGTCzZ3H6}`I3EheaB-fCOsgRs`cd9m%stM)Az?3-` z`or-p@sJ#i=a0))Tkt-)?SAQFz|P|q6+&znV;Nt$>xDbP7#vDxM>FYiP3tLzb;+r`Jj7M_oa}?n%cFia#6+QlTVd9(l05tL` zPyha1mCyi`Z}hWL{H>%8S_Qm%?{FQ`ifJ_W~z|i@xjow+P4UGsRasIVDb<3XO{HPdKUI$ zlfS4EiD}~VH^$)Ui}6Vt+8Pz85$P|!NA(VF>KDTaml{(HRove@}u?HH7TjiV>D zdsRR(3VY=y@M3>f5@6YQ{LC0aNG?A$PI#gQwz!@&4k(3NYD@d3Cp*8`7 zVhXdVp?fzy_K3s?XAR2APncnpz=_7bxL-H7rEuL$9$&^~Q+Ed}HBvnv7>lBU85so^ zXJ^CKflLi)X&gE}Z4Bur@vf1;Kgsy6N?J$tO*`|;d94h={Y zcPzs{b}r}T+ySftX-TAjuS@4Lrz8~*zmHZ|>~RHpaD?O2MZD}gu(nsF9 z0G|e0+OR;>bYqD~Sc@v^ra7BZlZJX?6k8l)<6|A4YT~|e5y!t@3)J=r`U)~JT_{od zi$8veZ(#<7p7MCxAU8HO7iC{RjW5Hd;Fk`bTjhD-iUUx4uAw1T3+)2CmU~i;LpPN= zGktR*j&DZBfuKff7BdT|`TP8`l6!Z@WFPK)X|HjuLzCkGB5Zkx^#)-gD`O==_FrWX z!Xq?D$Tbi_z7o>k0h}j4vK+v9+WCn!yTYGH$ftsnubw?PsqPUn`$hHrV*A{1w%@Jg z<6L4Kt6d8eGq^E|D3g8)*hiZJFsXt#Ajyh~-#nOQ3W|!7Te!W;xEX4xhKDC;C%AQs zvilJh(s!|S-!ox!flHZqp65@3GsCwjY&q|WI9+u>;#Y(GG2##>Xl!w{U3(h?W}pG$ z=Ek$==ht?QDUN+*Of~s;rTWt>emsq;Pu9G0U<#IIgO<3+w0OPXY3AS&=yl4p#j z;5w12!lMwQ=8m*%kH=CfEiIgJ;(@4^jjUz|*P~VH3{aVol`pCCRB-7Bk)*%!AO(Ul zKCRhF9tpYHsl#&&OlA3%38vUkO=8?<@~-=E4yF!~(7=iR%Zd8@B2(JTLdpIRbCJ)V zuR+EFQ9oSGQIiu9qW>KxgyU~ptmX5wgp5aZx8M`(-8g$kKk~0mF6R0NPzku z0=fDi`SXjd#mphmyig%&jl7EP97(2Q;OmpFm zl@=9AH?o}rL$_bN3HuzIx0pt=zpYzu+1j7^tkb}Nh_yB8{s6p%m8918x_Rmy;Vm!) zR)$7Am)*v+0nfO0mBzYcLp)BmpwVo^X(OCCTDX!5DU|zZpW(siEM=@aP=ir2*G1{? zW4<@{HPU-`%Fn>vQie7}Ma9Lz;oy74JKyZo zP7Km1UbRvaCs3=VW40E%dHk zujF9{`CCsi8wx5kd8BD3U%yohC;hK_(2UDpIeV{k8rMGw7=cgzFMZ*M^{n5n=Zw;c z34wIcUr-uvJ7g{YqU5bHegGqo04IXS2MuqDZz6BrnvmYq)#b&nNp#y&*!i$sd&v-n zfq7Ex9Cy!EVyl%wbNg>QsxBf!GY%rKUSwGS`APn>eBHxtrY4`;l_PU#Q|XZBE=!Lb zu^7}YeTE8AFc}Jco!ldB>~Q{bS?pI;z`ZPLdj2=EjcQKZ{C53gMyhIB;+-DYc*N6T z-MXb`2PD12W4`^}jca>qs@l2|b*^>&^z4hQ z@Y9!=W+z@~VRdT}a)QZ&JbDUaB_w9!WS1;TtB&U%e-+C7t^dDFfX1(#U?ivJ{NwEe zf|0sL(xD|4<7G6@B$qxlr@gtY8wCUgCK(eqGu zMoRA^&tt6RViEulrC{xcvncBp441KN^vlb6xoRad zRPJl1rEtkK@~A|R=jTwmDGl@UMi^Dv0X;7 zT__cVZR%T5EBR~aWPA?n%sxM{h}n%Zw97X@KF3kFv575K3*_^SCfGJBIZoI8o)1r} zpCC8ZE*9ineDL~ja%Kq(L@S4gFBPJHq7@F*=e`%ok|E*FC;`wrgPS_Jg~En{Kq{cP zDD{3duYu}gVE^nJQD3~xD=sefecDO-`6gwiH{`P{P)e6szw8~ZtQ) z+%NR7^$3|p-NlU5fJp*=S9u#}Ps<{T?q*f=n1Uc*bVj|(OL_j>08*gdCEgg=ImDvN z4~B*&AC}j~%@|aQ?d%YxKYunbvZA5(j%761_d27=Y_{&YaqwU$LQ+Qjhv)8fPvK{% zL0*1t47X7C`3~erQf&QG;!^e(9>;w&8Z;}I?vn15Ck?1b{Vl4`z6SOLvkXz8V%s&* zXw)r&W&# z!yY=Mt36B~5tE}{4{?9ijzE)37JSE}FfdoBW1lJ9{e-ou-90dGc($0nb} zc(ough=v3b#A2`J_CB1oO%U`y#A>O0fMd=W5S1-4fx!xwU*UhrDevc7nx?9*J{kb? z+BsfzuXVu{`uNc+;)*^jolrb0oR9U`3V6)I3sW5`>nxNK(eQwRA({yK3LbREWEgLeR|Oz~SR3hbXF zEpvT>(U8|~M&Tm@@XtYbiN%b?=O_i!>Z#ux|4-qZE+615_4!7K`G6|kUnz-zVhLf< z7&4i6FE{R5EF&{C6ukL2U$ow}mn%zX=Zy(=TdeIJiKUZPjU+*nF+-lo0);NrePk~^ zhD$Iz_MW+Pi3g*_#>I}gf&3J}EtiFOX*5EMP(OEd^bs;*oHXH|47i`e$>bd~a2gvM zoou_2pFEk2JcY6+Y$$%8B2}O1oX|rrkt5x!_vqE`ThRVq zHYzbZwY#y=3;I@hM3&OS@*nGmgv#=Z1|8@K9{xxDY$)wlMag}xyZtO?h7WGGA|fK3 zmujA!0XQ_bq5=)(lj{F%6Q`oKcJt11_=d7yhOwZeK{jY3=GbGPUAwRYvUEAR1XSsj z+6Bi9(Apm^dhmL3EBU+{D%_@{y(LY1@8GGqI66Q<3UvLV>2ja|qa?Q$zl7I#U9u#< z+2n27$f!DlvH9&5S&dnb z;OXY8wy{*i#KeFj+NwP0#e-kT!av7$Jg{4?13e$a|3I4~s;^+MLJF7NsGam_aX1-Q zI{f0)ipe(_i!0ym3qFqd!vK-atevQQ@;AGblv?-z9-GEo*s&hwnQySgnB(IfrA{p_ zX%6h<(6+~J`HG#%n%XX0NIOvwSV;Nv{X`u_T3Z;Vr*^zdkGcF((xjZm?(bnue6im7Z9 zN^!@Ne>MhqAplf_kES2TR!ID@i>s*C+>-^_AG3XlhKE_62$;V*-&xOY55~vYevERM z>g1@o_?1vBJdHc~CEzW}VVhv*(^-1nbmtFbp=R{#{5j$}i{?Ve=zd;X&b(?)Sfi`i zYjkfIJ_y2owma|6T5Pt(u?qZ=e_=0lDel|ruq|bK%~!wb>d0hx<;~QMD;OK!1yx;M zoMio~?VjUW7QXl3!tUtU{I9IZ?;98mLkA2=H#U*Zz2ACsDEe(;AWR~XOtP;p-lO%gb%`P|1`Lc)sVqMC%M&UhjRe92hyPp2835fy z)T$ZC%zXXPVq+G!f2!f)*2Q__r&9c*koZLx4CV%tw%wN<*`L$rl6Y#&ZUuQg>7sy| z60N|J{0jIgkQ;a^*XsM`{9r~Sr0|nd?WS(vRLvw?jn4#@<$FriROXnnly8lfzK$v~ zDQg{uym)@GuGVwn`8=pL9v08|UQt_&ex*wmv?30=#8xrPSxs$(I9e=hrM1v95)1e~ zxqpt4lN0y^D4@L;jL$Ct4`@64mVG()d8`w&If)-&VEY#{_eas!uQkj6A7ftu6<4}! zoe&5XT!Tw+cc%&N?iwt(yN3h~0YY$hcXxLP?(Q`1uK&r*WZu1R?tSZDs~78Vx}m$j zbH1uwyLMHnoa+i;Ye)XHMf)>g+km{YMYQ3P|9h2w9kihW5qiu{8scwUm!fUJ~Z7+OUWXwEuzJPg9BzO!(QmE z$1qdMDuoTd>Z`c0yJdC(%l#|ny(jTjwM%T~xf&BYeo7qvn5<+i4nDf&HJAnPzlfU@+Nvy&^Ek8FlaA4_nj3-rILLu0`bU;_c z(MoEpV@1`Lp1t)2h-?y2gp9Wlzrx`0Yqx?IsoIr=+zdgeu4cOyR8>GG*WSq?9k7UI ztE-fe3#R4`;=B28qf%ld>g(NDqdT`n5pV>ChlkPmT+{YQ_02>cJi*mI3JUrt{(jIj zwI+)}>cP}fR1&?Wj|HJo?S`Hw+r8AnhgJKzKSG0TtA?nIf@rL!hVLnOZNcN zCy4;b=DkEoT)Tp=H&hU20(w3P%!a6esrZ?G2|(;?x>P_&Ma&<;|2PB2v!3@~O0v6q znh!qV!b=sPP?D|uv?svA6{rc*$k}}Fit7(OJ)h`4)v*G^5m`^KNGBioTWVOp$Csm^ z>)Xn_2Rfa|5b-~~DAssv_6`L`sAd5j6D52?(f-GU{N+o0kK}+dlWQMQb>hEhhYK7C zKx8nKc~FBJp`CQ=24(JLoV`W;&Pv6YjEq3yR~1MF@|*x#&bLf0uBhUA61VOjs6XaT zcg2wS?HPvjSZDI`gTi;s-d1fl66UQ6zC;LnD;&;$j@;daFj0%kPaX3bAuOj>&nvcj zQd+a#lI;>)R;7=Ue#gEoTTp0W3Z7oNnDU8qA8{XJgyH*85HzEc)VjX><7 ziFtqhn}rrUQ4V+Usy%~2F7a1)uk#)OMfFBMg(*xRhILA7RrRR?5`@|p0mXF%_xse+ z{oI(cX=H&*&VR#=eEkIkNjm}qBwBE9rK+{=rA)M%M z7t-^?T)f6|nZ2PIZb-GWznvq)0N2_&{}N5<+a>m|{xmgxcU*2fjj%BA((Z-(ek zF;Is6N(C4oA3aQA=s!x!Ai!eDo%E_492~_s|E;*q7_z9*(&){b-?ZHeh>tMXU(=YF z%8d*SGc_?WF~4s@dL2%+D!IAU$1m3*Ky1Uh0y;TFIXkxjVv?+IJwn9YJ&_-{4%b&p z7qzjvIaW!Dl?|G);E74o^^~Tgfc%_-ysWIK;jHY!hL(0Ov&Bt2@&zt#E=40{&X7+E z2k3NJKhgU#HjX~c^BW~d&u&&-i3ODIR?Gh9z~ zBxCX9lUGu*yz|}TN2HRUxQ2u3$C@?9yYtZ#WU@E=HdPzK!c%@XnVUcjz*6}OwhKXjZ zL;dQq8HWP%#X=Yc>tB)F-^c$fUb7-0BH0Ow$lN(*Xk8YBd>o-5EMOw~TmwsnQ!Oaq zH;@=ZyUJK9wA;)9O7r4D8jZPJm?g$BRA^&mv$V3Ji^hyiC@{Ah0g{FFj9SX9jSA}W z!l05l;RW_X`85ZTNoiT>bUQrVnfCT{jr~-5GV~*lQXinZnRT_Ya1c7Qr3t2;6M5wg z)$oL%bDj77txJ{rElwttIcI7<-j*?jBM{t394T_R__ecr_JjZXI{wRnuPgd!(cLv$T^)4MxO7g3WthISpx^_Ml#Oh*yaP(N{`rI>2YE^lb-WnOn z0*voAtNo_ybYHu;5`XWZxud6|(rwLrey+Fwxb?WPx+7q0Y&@`KcJbn-14Q&G#ruxy zF7^4E7dQ)i@z2yKQN?L;eCK#>u2n1P)1E*%u7JEavP?3+25cJYM|M)W%q57tj9FEt zh?Q1NL`A6!IRYLD6U7mE`Bp$m#qkLhiDJA3ADYtrM4;j5j_QtaY^e9-L+;)u--yli z?eV689nFr_JA2C2VcLPI{*0W5Z^bsIX^WL$8AOI7C9}+q)iZuF6DqiWVWnVAcJ+_*!~eO+|H_W5rWMj_)+D-(5}Z z$Y&q_q9^eTQ0uqLJFDb#`?yILzhL7 z(uWO%$jcvv=YNLzb*O4FU|kLG546*U#5C;!jRm$5y7!G8zB%Id?S+}q!m2^fpM zGj|4IVqgyBqrNOIE}n8b2jRHk0h6SvHir|btd(iK9*%rWXQuh8^oK-ocv%x5zCc{+ z7`@TLUDDeS;`9M`$~|Wol9YINU8RcOEHpW|MFEnNyBpVuS&2+4R+7n00WLgpwX4sJ zb>ye3A35F;>6b6vxRk2m;f0=`m|vg%N#_3H5q}3mUcZ8&mXX&qzk?xNq16t5Ud?`z z=ItZPksispxki@}dTQD}liVFS5_yjv0WIWFn8~Rr1=zsS=(6G;_u~r-N`>a7b96a548UwNArsfb2PMR@Ahc6Z-%{EU}% zz#1k8$}k(euN=8?Qv37`<;EIRaCeTNV=GR~bMGWM7s`T=b@CRQAh>(;arnFoDYPs7Y8@Pj?@C$@)QhrfU?2dR9bR!S)Z`~ z0Hjjhw_T`>di2bfOa3+*z6648DfkX8E#8<8Zxa^2263Zy%+AmEXrnDc`?8}T&los= zLmj!c|=I865vV4)n2Mrzj z5CZZy>EIGNoU>3sP8!`ul~J_WgU{|sN!N5KO1;6@OoB&F{*J0P9}v*41inTJ`KY!9 zbmIPw>Qvo8b4oothlcm0Dy#v3?aKAsqVCwvW5iAQ7R4&Yx;+RVpj?#<84Xt~{P~RP zk;!<^X~{8L9rGW%{u393?WY9NFMo(*M!&%1;l&QiPS%$0BAwxLJ?GXxL|4e{GH_83 zUB*776XEU2>#?l(uVQZn=Jz%-z-xPfUzIOk>^;&Jx7jRk93@KMdo2o>6?B zo?g-PSpysBKcjZkPTjhNZr{|~=E+Ju78eysbi7I2XGrI#+ghpm=uMBeJ!|QMNQ%wI zm|)#xm(FWF-|7<%$5%j4|46JaY+?Q`+$wm@YReQpJSdANW_joqUWNz!jeB)Hfr5y{ z#8@O71=`MF;1M2tZZ_ls8r{BJ=!+221iWaj?-*TjqG+;=Hm5zmmYru90mqN4qeVH~ z8LM>f^heW+n+R(3k_V8EnwA_;s<+cv+M4@%*!L~wTTw%bFeaviqnMx+5^r-6|LCo& zdl$mh#k98I^ls=rL&W@TqbJuisaLs!L)HA`j1>#CTLW(`zguy zlaGluTH+ihcifqIC$+QeIPnkH6i@D|KsAz5(7{{ ziO~T0_p`oX$jGOa@?9gwdgFw8cgYbqZ*g!$fo{%_d|eQfa4_m5MD#{>^7C_Zc0p4R znx|9LncLIs)4w3KUq66R6L`Hy=!}TRK%*;p_!{M;$=y$HlO`(NBAU{iW2db;&)FH)VM6hlzU7QG z$Ofk_a2#f&+XxSUmD#FXbU=SSzsu@8%wtDCE(|kP<)p3C1SGWX*QXgRj}TO8 zHbe*^_(e#h3r*#9YnjjpS?-8>xO~d7l@)HdofcSXSPkC8j$id7>^G1S&A>4_lAkcz z**3 zAJzo@v{1rDqzOQCp6Ys?#Gan_qjeI7(KITHg>2Q;0aKjN|Gomh!*JPOd44531IFJA z`u4z7=!>tD*Y&bmX-ZNDSlnQ-^w4ID6s}^<@ zWjhRi*Huc#lhkd7!^13!zukjL>3Nsa_B`!%%H^u$!yRFXL;MIn5x1=4a#M7N;8c{l&|xz&P!pu7;0VB#;y zSzEQ}i*;}{kXZV|JnqCp0wRQQ9-HQLJx@cSSCI02p~HOnK&`4Dix?FtL!|?Bk0~HQ zfd!_LDk-?3K)FNdZbg4^|9v?C!0!xzn`Y-!KIh+x6}4}lIC+6K2jceLBnO8WongU$ z5qN1@sAVKH6}8o*&^*vvPkyhewD`$s_n2YTwKVdt>q4W4jKlnn{%56MK;Zwf7o?M; z(0`MCI*|4FoMu0Ot`qoaR#IP4vk zsjKiizKf9F9iUjGfUTY8_sjkzTKM~eI>G^AL-J>oFMpp=Cs_p_%Spv93z-aODTugn z@jVvmO&oBY(>lHXS7ppU{+nYEa04Lf3sL;OWWHCw@;TXSq+;V|7#8yr2v&bB;_6ql zvf5e-7Z(>`>Qz|SokP;!@Ba6}#hYJZ#U6(a>c9Tz&-MS~Cy?&=o!0|Rl*Oq(a>@Q1 zbNk1O++PFVjBkTWKl)#<^Pk6Rp9G|(@%n!ig#Igh{`fT?dhRpT5`JkMN%Q}fCYcle9L@1^^X zHtP4u9mHR$!o%FQ^<1Y&wkZ{M1rh(-@0lIR5(@BQr_RW`WIl%kN6_o&!R&gB#$>1Y0M5y_GJeSo{;Bd9a^enFnydyEi z4Z#Q>l^+#ud{&a-nrjG`o~$$fK#;1p#25UL!I>nt9S@9F?BJv2(Ovy`Tq4O!0+#sP)UmrW&nNmk1J?rF}Pr7U>Dk?v!cfv_r z2NEU=W%HuW$q|ENBbSAVt+Z{FGBLK^v5ij4KrPn$S_*S)wHsTKeJ(h4I!DKfR#J*a zix9(w7GwGQy2_G%FtLz(N_qLv*fJ5&B>*gJbJV9=Ey9YLirwtSef|Y*be~VakmY>6 zQ9sy7)AkM#z}Wxuc6IdmWyHo=evEK3Orw$&=VxY$RX9SBttg>p2Sv^k?{7Guh;iwd z?E6Do|47Q%lBv3@#3?Aw-z;}kMp1JTI<2=RiWfrDQ8C`n)N_W9ZWXu3*5T%6P74h^ z5+a;aGr`n|+|-FwEAGRaev@Ab9`CUkEbNi=u#!rxsZux&k-E27)`iKtO*l1OmMy8a zt5Q?4vPJsi+PCq(^l|ZNaOmLM#!Y+u$Enq?sN#~6SuQKyN+57|k0?x4R%35KmY^=qS@Km zfPg{PeksMk!a{0oZEbUVdwMq`tEjxLF3#S*!c8aC+!F~22@VmF92$DK)!jd&rziqs zOGQ)q;Uk&y@BzRu@EDWH%F3eS;K-Jgfc(!<57oA#xe5yk(UII_tJ&Dd!RTWXV`CAE z@sYW?6kO+VYFdo!_412<1G&F|&2OXe_pfyyfql|E9)9;jd`=#q?^=t}FD@*%p3sRU z93|Z%GNcpk95AVFUt9PTW1^^9NdoQH3N)Qr*mra?6L#|bWC=v-FO3MM^`p~&=rtCY z3hDw~G04ctCV=VRl*DT=7+4t9T0ha=pH0myr!g6dy4=}|Tu8~vW|*Y7B{)1yGGpQ5 z@Ho>pU(N5UX}_3?Q&CnP&7a7xGT4$Fe$^NV6r_pUqv}{kfC&M3PU!PcAMs+~EO*|+ z+?=~uqw$CH;i3dUYb)zmiN4sn&JGC)!P&ptHyKBmSrK^u-uIo`k4cX^d#tZdf18s3 zdN3*rVfB4-o{Js{L&Meb%*#@+$>-O~abyAOMXRAgt-*I6?nS866@(RB}!uU}sj z%=hsri6H#!ESGh%6v8)7;VQkO)KPg%$JLdwGn2mZ+S}f~bs8vqGhQ!(b_NFzzq@bK znpxWVP-?F}JySb!n_n!s=pP!2zh4-tzgyi>l0r>g$Vp*=0Y6+r`+7RBtS;ux(K)$C zTDt;dy(pdPdBGqlBa=PPZ3pini(*$%QSk%kP0E*_S}H0s3ob_)E-P*!vUs4JtCji2 zjH;?{XTzy#hZ`G6<0C#-4meB(kmo|)1;cKd*#F*4;GeEiAPvZ$e5whY-l{PslfX-G zGRE4ymn)bXa?$nAi`Fk@i%_f*uTRIK5=lf)R}_}dXVfclI|u;GR((d82v2euwPC){ zB}Tte4-044%Yne{H*6sHUW&ycoOr#o!1$EGR7<0(>WHUOH;Hy=_Jjd;1FF{~DQ zLh$(OvZ8>Bp$xw%pqW`II_mg{B}mC|jo|+RI)Bj zh!`e&ed&0NaMWZfE_^g|Q&1GOa&)l4|N-J$Gn_FA?ej;WRg1hk-j$4ya`19o! zQzIk5=#qy}c3P|Msf6K?jQU9By-iA5+8jC1BhpB6E*1!%Gpe|>v~=KjP8%|sAC zNa$@yXRYr@;T)3(E3q&C&57OxFp#YvDNPi<9qRvMk(ho_$#2v_4`kOA(X3-%OD*Qe z)Z8UTCt)K3*0ac=zX}wfF^RmG@-LhUH*xJS)T(?{nsGwHLl@%IL5*a&Vu1G=1v@>6z z8gUa&jD4NL!bke0#%8GiD-jA{=LJF<_Uu)y4G=4k2S>E1?3 zRTYzsl~qy0?d{=w^_SA)j2+)siE{oHzrF4srgmo<@?C0kU|^tk=$VV%Kf`X{6cNDv zV0h}vAjmk{>+#`pyT12tQd!?7JCxHEK8eZ*PT5Ohqd5ICFPPBE0Z=vH>Rv2tYT;-Rh-H4g#6MGh*p$)>)Enr7v{#hB$Q_H zZ*aGas#wKd!CMm?xL4IyvMdwG}HlUhIQYbIT2TPCN!K66LPy`)r808acLjt3U20N>i75dOfUDQqGDpG z6L3R3*jb6F6RdTs#FG_RRMPMJ@yGECLA z-`u*oMF2jL{<*OIV7-So(pmGk%dyNQt^G^D2@WZOrqdD7mf=-0D7BDKS$K-_BbU*M zJU~ISVx}u6Wst6~Q?k-Z!=`rTSz)laN8K*-{jf{3W`9 zo%vHx-a`Rm{rK=!*QzF@l3nb;qK*P9RyKQ0P*ll*@WD5dqjADHs(P<`XN>hJU;ny& zn>Y!0ELv7n^6Gjcx^?VF?=4A5Y2E_s`sdogOzdeqWNjpD`abjjG``yKzv!M*rR|ZtXrA&bZ8WPD9JxV5jUK#o@~AkJXp%@lkYq zZBL*bNn19(BR^X*n0w5OXy{Spe*i^N~I0#i0tPvYQ(RuGO zHnW9T$(^z%5?JjNW3qW>f(>d0@~K>WsB4P8>K2)<<0U%LwoR4qq~R8BVcVy|tC>2P zdp`;Jso(}jTL@}Dn&k}hpw0ZiL#3{%Cb6ozDy}|sZMH^qc8SCsBE$Le^r~_!YI$vdm_IPOHLuO)Pf>`eL)%!UxR`WOr^FOZF zUngxtfAK`!NQ(*4-SaG1RAeoFpdFjZdXNSds+mNWxI)}yNK=7O3H?&61=kj+E1!xx zTfbg?K^T_CoRXqZd@Ew|a;AE;N3qmXN*4kA1J_=zOOEW53JUmSjkL=CmKjk- zBR65rUR(ZcI+`+maum&c_~?A@aORG5qe{$;VbyhQ17#YB`MQqMq%0f zFH#oYJzw8i4{wqCAw$4QxboL?fm6gE9*E$u_sot4W{`4cs z2QgT_$qi1L&Dzl5ErMfK^pYATh`Jb-N@nB!AKf@dx2fdsXoRJc91c27t`BQh6j2=W z7Tv&0AA_M`li<51tQKnG8@{|oh)PTQ0AA9z`ShZ$<2Lx1$aZ*e#3l`sBjN?@_y7Ko ze__>b(FWC^=^aYznlC({yi?&PnkkY%_`MznQ!rE@KTxZKEKHw*L}7jo7c~#W(U|GCPB6%Pz$9v`+r2`D zOPvj<4!xG;RR(%$^CkoML#V`?I!?C}U+Br?x9YiAgXIBCm{?f-SbDT<9rV;|+TC4U z1Cer1iEjv)#hX~=2>TKnva`<)?~L4@Mtow>DCDq*>eJE$REu?qCzM|l^ZC0M6tQ{K z0;Ro9cQ(-+PJy=lI`gG9QvBbbtLz}^!YUmqwvuaCR3px#fBLVF5<3WV0j&cFTa;9DEUx-YMMOg0!e-@}29OHo=Q3p) zdVT!Q`Le&1nz(?yr~a(4J(BuWZ$V$k>Zi$ZR08^qnrLTP>SmCw%4lC`sQ-cImC zva*I;Elq}EkAKPj>qEKQFP%L zhwYdVoIF)UsZ2KzmceieC*aEnlfCCUSE$|TqEg!G!KK(QPjNzopK$P)=nA%Hb!u(8 z+Z+5WU$?cjRpD~1OGQ_RtVhV8LGsqJ!$m`q6 z5&oH~SLz|ZmoQBOa$ssa({GzGFv$_)+4$cl}%ZE5IZU)>=@% zy9$!&T$>iRRt?4D{+<~lpU91PXf+2V!)p2z@4SW+8jSd3Vf(eDZB>YJo@k)^RYpev zoCN_+^^|JHs{mE-tjSdU+2C~S8BXe#fuT^fy1r#N43Dpqmme_*Z3rdyKqtJXX!pc$ zuW#<9j+dLIy+`2?QZ10+lSref5w*HS7*LbodizSgzJOb>J%=yhiwP5Z;YL|&xok-u z;F)?39w{aDHh}gU)9(zB04jg*K;GKGWL77U_e2jgf^wOwEdJli5HP^*vo>6{xCL51 zT91ar{3N-@+337=pUSFT)7wCf}-RJLR<1cM1ltRQj6%v6AY3BuJdY=7Xfb7Un4v~yYEuH5{PXj;xmZl z!*jC-gRABZEsz4r%;!5i=&qdZ*_*B}uYV{gL1az9a8S+EEYT9n^zJHdQ1jQ|aJ5K^ z)RC&7@=XbO70_uBJ(0XCiscvaD*v2#72W5S`#D4irfK%Qni)tK0zFAzO^xHU1pH70 zqJ?2cMpf=+Y9z9qv2$A&=Em5s;<7RyQwd^(skEfe`*@}7@_NY9FR(fu)L8br7XTLo zGJ#xbIrzkXbgo#d7CsEc4V2t1=PCP@HH%Gn@~8Bp^`t@NoFkvoQX0jOhbQ4{NXW0P z{FI+QAGgsCaloxHl8}r~_TE32FBI^>7eHb5o>K~ zYN{cQDUu-rWgiwHq29#nVKrUd5?Be(yNkEIoTzAxW~Rq!ukQuEAX|-1&sCai&6XPm zux;bdQ98fFCz3%#85{qm1pI!X^C z|Dw4gH~)1-PCyIYNL~!tF;<>`siut|2a%#6^+bUy)NA+tWYEbE$zvQl75t%v36YZ4 zZXxy}q=<`)H&rmkL$t)wYjF<8hiGLrT2=-$cZX9P+6^+!u|LekgPe?o)x5Nd^qpeq z>5y&o)Z#hVltmPSbm=I*H0&c>taXZskLWfZzOvb*u-;DP_tKm$(oA2|KeV0|E#UA_ z!(S_(E1m?6CRN|z9v<5vhStrV9L-CHC10_qT`zJu*X(=^J7=EfnAtVP8xdL{e;M>a zAvxpcDl(4sNXioPhc*GyA|1;vWU`g#RX&Zn559mZFkufcI%koo53_crrlxsz(aTOV zOCNCsKdv~mjP4;Rg2A!$Vg*gXk8AW-k=MZB*YSKg{o_cWSMQM0_hc=_J-;l}RlWd33k}4CsM+K~e}8=(>v7zyMnX!8kgHL7WP1?s0T>3Z zc-#7XuNTkl^z~sKt2GMf(LD$Z7nhsU-k%pShM5rxcr&&^IOV|6X4g~=q1w0Bu({EF zHU=bX$gzS0bNPufk<33yJbrdK(R;qEy4an_eWGun_|p{q$Dh81f@w#|Q-#cah?Du| z%JgjXiaNTaTKuKo1)bK2c&%Ul{tJV;PbH|> zoAes-VE7bd9)%+I;o^mT1BNRoNRR#tE&|2;kArvmx;ot|)NJ89IWCeajFgyK(II&1 zm_$0cjfL>&bXDJ0x54`qwaeT>M{#Vpoz_(Xj`tpdU2J2q3~mz+lP=J{3^GA(B6`yid;_%X?JxCt%;Kiu(P8Dl zBrvC>p;NOPU8qA_`!39b%*V%jpl`Y6>FCU+Nd$DaF&d1aJNwBplINO`31L8i0ne8u zCHR>Xl0iF;>Mr^8pmGF9jS4R|cu)QieX5^V+Wu@)LL+T`e=JjMYd1I5u1fZ3ZvNzY)PCriBa>Di=MY>JL;uhVX;6VcPCgA&e0%Fg-EtHK@CU71ZIY0s64~;rpH!)KK&d* zbC8Qgm1p?v2WKAC;H{%(S>=3=4t)aAgxl~3Cp%p=_sLM$DxmhpKHNrti&xh0At@J z8j?@c)8jn{_gk1KTXxmF@0>12+Vvj8e5e@)RD#nPZPl(jqr{}%f^p5VItifhme+`Q zMpvX4N3nWd#0>n;pY2?{FUCmXIgY(6kw8QT9{dLTZa54{LVN3${(TBrV3Oj)2HWmR zX)5o{Nhh=A4&%g1idL$tb+Me!N~*(q|F$K52W! ze4SGN5rlob)F=i%twM1*t_Ksit@=WTT1<_-Q2Rg7w|E)?GA9UD6Bk?A10UjD!9WJi z?OCxR6&)4(I=--U)8dp&AXdkCLhO(0%kwB$4v`=m7mUWFOL{Blj$AI)DZzbg7Yp2Z z+uB+a8>M}74^_Scc-YJDC@d=KDyK@7Qq%AE(nA-Nw;r7J;{#Z$;x~v`w?A%4J{?h? zPF^h-kC}BuJ4$>eAD!%0GTF=Fv#Q+fUM32&C{m|i?%Cr!ZdCUeb}F6aR@XHm0N)j8 z&7f|N9D+5)NM7h4JU_M)=GSqy!2|+v32Vme!!3wa-#i?Ti`JI8quVPR&aeL901pFn z%;ptX_yOiRue4!6;q3g}_!et0G-$S3OYOFs!DkB@R8DB!O%<6tuy}vOQqP5wFdbp2 zaV_-8^J1)bQ*Q4liODcP=%K58mc!%965(`S3y`=N_g>+L4%3m7i>5qWmR`p34Bape z6Y#kYkmA^5b}-%*7v)7AuUvb%9hCP<-|s*hFbW>L-cd&xW@mE(v2zr=4JW-2UGS+bs;TPWOn zGx%M@Iii9XW0>AtaIx$3&K_0n@Nlly5Y#1CDRb{z=H6Lln-5bJVt1LlqhA=Y$j8oQ zr>#>ElULBIvU^o6#rC3fFV)(#2p^qe#)14}kQ2KT<&^32R1TK$W|m3=rBq>BGaLq$ zWlcz}N&V6&dWR>;Pa~mq;+lB70>;hB;uM}42b?;@b~n6_yu;DOFLl?l8{xpJ?bs-y zmGhfn)it`V8pW4|N!ySwC-J}XNd>-pQM);sE+n%oR>y?K?Scto-5Zu2|Gf^NI3img zwitdCtmF7K(J1ip@yWCd@Tde8>De}uSniLH@K24d`BFQ|M(8%XPnG7}#=J82+a~G( zBj`I`AIxp!y|M#rY+l$&aIz`xuo%!N&nd{6Bh_Y;c}Ow6Ed4WbOYO3I;TuBsreSLu z7jswFI(>htj9>|Y{h-Yy60qifX{!*2_f zss~j(k|US;!I>%4MWWqz65(3P*j0ji+z1OgLZvZD`7y@uUs*k^W$@Lg1&6n{&?Qf) zmW~{ywRHT0P~N`HHz@8HQ&iE(zP)9ZN#TZZDbT{MNZWVa6@8eI_obQW9a-_i=<$Y$ zhiP;J;4j*w{uf z--%8I!$upn)=4av?qaVH*JZ`U%Qn&gk_{oeNp3i0hz$=X0lLN)MHu{MXe-W%eGo4^ zqOsmh+*Uj;ldr^)7XtZ>q2b0ii!+g-S8!?{fSp1{MAR0PLg;l7!FzfW&U$bVt0oL7 z(nu;6#x3{#m4E6u9El7^4gwsQUceS6r9zmHALhJ^jj zzwX^%$D{4ARnWWEKw9N!$^|9N~v^NTC7Q=EyO#1~CnY?KCYRt&aHn z^kg_0HSHOfw7q$m_{FBx`bNx!Rue5=si|;-94LWeJcHhd>QqriSg8- z9`En(?i6N&$!t$n() zQ_0;-UDC#;bUueEZT|f(PS}JGohD-d{nA~bz()S(kNNtJw1;uq+@8Ig6mJxLF*AZ3 z%HfHry+3~JySHg3Gcz;0S{guckT{qgs^^~Jp?qmVe4&f+zE!1RXUfL1XWQNITeOXm1M3kz5d+j-FmI;^{%p=f+xKXW6hfOaf4g zSS~lsZLSMH$^SfWIu4^@^JSkzv_9#{?{s*6dbQ0GTEF2jMvWX6UNC{RV(e<0DVN$Z zO;7*((sv?bGaJP{+;Oxz{iL8W`(&8>kDyQh;_yq`t;?r{-nxZJN8!@9UJb$aVn>YA9xBz^OdM8+8tksE zktd^ik{QJ%qQ}gj-8zDjvdG@DRXA13EEF=Wr#-rQ+~>$xF@b8;sU4cTNYQntYqW?2 z!m!zuQQ72{!VaBJnK&rao-S5K`Z-pLqOF62*K?$cAlggP(&UAVxwnc~Ru8~0AoLuS zqqKb}onpA3y@%>T~c{0lA@I=)15o}Z_-ykEKS`kL45qnJDA!Ss1S%c8ZcRVUvQ zj!V(rPE5ya$BA(ce$oE(B4>~Eu!t& zgM!LY?vdg4omjZn8wfJoo*%D^g}66SZ*^?yw^ZlS!qj*tPty5|>=vw8zD?kNTTRf( zS|Exd`N??ZSMe8_Bp^omo%KW7YPaA&-213E?4%0o{4sCIeBT<}cwS}2K2M^NaVi^0 zj8Ib1R3)uGij<8Fk^~MO5ka#~R$UQ_OZK$?hv3nguq(%yD<_Z6Y3IDOSyaq5nu+7< ziLxwkh%q@~CYw(-QjqCHjvGTb#X?rEG!hj}_jHX0QGtJ65R=7~ zj%b>++z6xo+jEso?P`1fBAv#Zqc#k9;l?|C{N!$t&Ev%D6H{-hf?~my)ioe!N$+h6$xCqFo*VHDs|CYN91R8CvO-VxxIuPiL+-02=uSSM>|tl@8JDqb!?qXg9aBwphVxy*2Q(KUZbVVBL($!{NG0yv$crDc5 z&qssH@vhLLqAlnPa%QoiISp<(&QV2Mn*fGiK%boV9eglMX(9vK8LOHXef;n2F`;t5;yGR3#rT&Q3U3L@dsZmxeroesW{T8%Vq-L`$hN z^WE*YKmJ>ONPve#;N^wATuiYjI~`^0=LhY^LR15i`L6MzR5Zb6PhR64QwBSgOvUc_6ZO4C>nv*!3ygwwXMgWH>6wP_-x-D^87(5^;;?Mm@BO zKb=c-tO_=4Zd%In04W!&jAys@#;sMI+1Bz9(~E+40WXzxQ07UoSWm z%UiH~A_Yb5?3ij)V3U{za1Yz7)X`QJ=-9%tJLiw-LTw<7db&6qH7FTuY-}!ZtSFnz zD)bKxVFYoqu{n>Sg=tE?fDm)xQ)|9|*VoE-p7e0u4JLD2s{Iiioo&4!m_J??p=vkd znbwCpFOh6AgG4}m?!&3K&fTlgh%dzf%wSXY7DiMkWDxV8$M}Zj*#-rrZ=OuDJ3U>X zW2*BF7u>gFdgPKoC!W_!I?%Ca*Zd-A&4v)RBJ?7)KKJ40MHbz+-jpV4`e|u429&-tQ+{7L_P;exthfv5P-3>K`JxEQb zwqCz`v!lV^ST4yt9F?bTC;MH*BL|j6^gb?KCP3s)qR-UDi9UrPC&i-L%mNwwFsh5ClLTvq5Na%CdX5%lYU ztmIZ#@J7O5g&o*HKp@Qz5>iVho@`#obUr_le2@`?N=XT+fYd&in>3u$59cRMxfTaT zp=nB9z;plja4UTmu}%89PV`Q^-y3cD};zNYmB-(T7FeJ94zbSMzc{{X+1ZP1;st=~E#2GG6~;U!YT0Ct&~kZnB~LigW;o1aX9YMr=cKS`!6(f*VhX!LFq=eJJzw7G>|GPrrYCy_i~Jeo8}^7ZGJcec9vf-vBjaJBs}@rqL^+bC2Zw~%JK`^&I;e6)0!wdDxPq1xN z_1>NC60))}U%$dR!CA)wS(-tBohw}nKSV-ZO@9@X6@tYu&D&%#L&*Qf+grv}wYA;DfGFJ{-QAsTy1Tms zq#Gn91U4y+fP|z-cQ?}A-QC@(@8Z;b?)#ko^YQiPhwTP8thMI6VvKQ(N!mM{%qvSQ zHJ61#l9~WWrgePo1`Lx}ic#6en7R|bel14fJe+``j}5O2=4x*C4qtEt$Zx*uI5Ptv zAF-^g#K&V%F>q{GI&JlL2aS?m&+*O;-tl{!fActffuhmYY!|vXEp|p2aiQkhSe^c_ z7=Yh41`Q2Eg=Av(Qpp{v$0imfD$ZMO>U=s!@dZ;8icuA4nvVvM6{vKKtYaLx_NPQI zWxqf--oT-yQ)tqbHr3r)3Zg=h7+eR*J$&c)w*NWz01JEhS$RV3QBAozYnfTJj^BGx z|J}}mDPC5sP)}yhWMrR*7wS=wV>iXCuo(z$T3St)+E7u>jV%;_3Po0gK;e*Uv`kJ_ zF|w)WV~k3cg3 zjqgFG%VtC+Uz|tuJjxCNzDtLjJ|c1VhGEffPUfqgr>uY|f##kAicv%xX#aIZ`9BPN zVKtdVYi=kvs(SE#%ldwo#o0hbg=;_*g`Z~W(kQa7&fTT^fp;5b41?)hdA}hdER3sO znD9;gpH=r!DujyR=Y7!|VJMWL z6!MGh;e?F0LuWlAP)8pD{A~avjmo{&F2+l(GCz~aCGs9oCB|t{PLbSao=I5!62$>s z$oh@LYZiV^0+CWbzxAon;uW8xKkG?nfm+KFC+=^VeA@SCj;pHLi1b*0WfZ<0n#$%; zvFQ8j$+P1$)xk_%3&( zgDDY3L|;4fWzT&C>pzg21rpThcM@@J7c70c;vXsA9DWeYT~tEG@ik%}p>|EvkXD2= z;lj##;yD#$Y)8;(pG!{K>|p3m)^*)oEzoDGYGsGq-TLE|PeT(u5`~k*s7zxo641Vi zd|P3_Vwh6bv2<1uRLSO^fe#Lm=|c#z$-A zVaGn1JhurY@=&N`JrVe&WVk6@Wwe#c6EtEzB01-ob)o^Eg8|Kut+L{_7jJWYN@x zAo407VdSyK<0e@<9Kq$ywoWgSSJ%f&y3V1<98+jeI3_=?Z*LnsJjS-Wx^Z!FX`iLA z5$+a&h$VuNBuJEAiQDXH1s?wR=w5)#?C>dX18Y6o+3%Fk^O9z7zPb>Nd}BTOSl>*=DHt$~C-r6Enw!|)qA}_ zY}fErox&gK@F>3RckyoL` z{p#)A^?hYyephZY@cBDjqD{qT<6fD`50wpm*TYkrc+ZU6EF8%pLwol6R4No&m1-_c z@IqxK7YP(H4ogO+euP~BD*W0wIx~K;UL(dvXlQ7>cBL7o7UA$TLPSovw}uHS&4=ql z%Vt}Ly?nV1b)oLU%yBJ29ABXI^x%A7RDC3}kvGkli~OWE53t znlz47_M4i}T)@`1ESr=vA$1ujEa=kj*Xe?cg!tH*z{-d!lhYryFtA+|hO_AYVpN)`NnQ=3BgWI`|&_v>^ z%yH+R=~OzcH_QD0{!wk!^1uzO_WsHH;g37Z0;qzKC%p<$_%EZI?zU4Ze4ad|rKMv4 z1`^+|vZzQdl7N?tS}`>meFzdS*F(m9@C#|}ia;Fn|E$&jf&nt*(J~`1%Jp)iy=O;@->Rah|`z*Ptkax2m=C=Og%!7E0)WL4gO=jIB>#ISW3A!{R<#6&kQ+p{pNjS>Ys|YKR5V)-L${G zxevf5I*BkV|I(NJTkl?F@_Yl%`Zh-YuZQyg>*qU^J|vv{%jK=kQz~g_P*GA+B10G$ z7_9H??EF{a`eVEM)hK=h=70qy2}u93n1$%kW-EBb1@7F-vunppy8CU}&G?p`1OIVz zGmbETRd_vuCMorgCS+6w+Vy0ambvxPEqB?9<|E(OD&7D7q$(JwLCATcVf3DV{{O!P zh5y8hQ3~KWq<}-?|K-B}cK84K>ii4Hpf+0+^H39t-$IJTXTT#Y2hbF8X!Ue;i36u7 zcTP;GCZ?_L?@zzwxBLB@{wpB<8Hs>5cX5Ei*){y4SNnPCwsFd@{$5KyHVD63US1wA zHT2zEZf?9hS^(rdZ$evJ4L`wU0y8=)z~cxE3|v1xW->A|LW=x^|9>wcouFqK=32kT z@86~qBV=irf4JP-a|!!{orOh-L`*|XO-f!q;&8rtp-56(ye}M={oQ=k9y$gFIS~<& zrq^dcVH%s7$^dj$D0|bzDUUyI6b;V=W=a0{0sLnVK3`Gbb3@32fBbv+wJ?W9-FKZW zGde#7(q~9kFft(~Y%IW>M?Wj1S?Yn~;soCvOf$0i{xo5;7;U3Jh6WC7h%G8&Anz@> z?*FU)9UX>UJqeADR_e*~_I@Hb(EBU81NYQ|A4X3`>J5 zN|L`iTn|fCFP9OCVoOW%U2VFXQ)+QY%aPu?t}7s(Rq2SLMxCMM>h@t>jUzWvxb zT}G^Ic8=qjdp4-RaG`(A8;xTcmQ#w z$lG|@!-*77MjL3u+eAQ1Wl+LSK5p>E0fSXq^Qt_lsMft0|YQ`K81 zQtkx^3-I3itxLB8OeyV{VC)*1_z&W(w6ruJM7SB;>OG=`djbegY+9P#{(_BqY+|DM z^x2CISXfwxrXKkFAH?GhMH*_K4<>@IsnL~ZfIm(91Z~`ZZB9lR5`dlhZyLhE6s~R> z{wvY(@p^uH4iM6pP0nHU_m;B#5E76ZC>o$I3obFL0J_?dk5tn6&l<)()r-NrVZOkdEB zjE+t+uBw$)LCc7;@U(glCM^JDkF|8mT3AR0kQGlcCH{3c|Leu*H0(A53$>0w+Bm zN1oPMG1-<`D3$ZQ6rm-GES-dzIUD^TVV3aj>rl*PxEz)^`=*U+34x%aO0DS|@B}=T z*ioj-NJPFuwii%gj9LMquLjS`RqVGoKl&OL92`6Vq=jC?pC`Y(6LXPj>k^MaFCT`TNwTp|V*X%d>b)BMnr?HO9k~?1=O>}%?61mmc z*q;2rIL!M7u!F*|dyO&qKsn8K?#k!nQP5&hyuq{4Srkz)g^G=$=;l1qHE-?|#CL^) zO?&+G7Bu~ZB$cd_sr?ktqVIVP(|)TfRn5Hdg-7_bnb#nudP5wW-J>9t44<{Pr;Ufk zU??LqAV0|*o(`nZj4L40+Invv#{8C2SLx7&Zs&zx!0l1V26E+kN?6iKQBkX+ee58Y>kmc1>IgZOA{LPDJCs+Y4cuK8a)SD(a?}scve?d zs|u{ZfaYH#@lu4L@GC(gC0-LAsT`ISh;=1khtJ4!e zS?K9CfgTQtCpkUE(!9Ac8Y%N%iSIu}hF_r=#2ptYUJw{s+UEUd3Rl_a{J3uEQld~g z`1_O)Z3m?+q1`G2*UU0=jn?xGGfOHZwve4E>IuROv$z5gk))%4ldLq=GyuhJ^yILfZh%_4J%s z^VxJRE}+c8HGQ~?Torw(wagw)XRum>ttw>~NkG;Zt1(MhOHZPSg&Q5H2e4LFuT&JgYk^og^M#B~(B`p6+*0WoCuKvwb zIqtCrY;#@I3%`p7zbDh(s~1--@>Ou35N0egwz&h}!b{SK8P zg%yI~i-VOY@9yr<@$jsicu0D7;^}sCWzx+$iCX|0+wa6EZYUF=`(Uks|CCqf(ETz; z3ebhr%-#_2q?g)%6~9ueI#dun<_-=SaL#Yup9{a zuq25~H8=Be_%uV`I0jl>w`7O1-Y9JijWH&0D;-j~uAZ8)f_K_akh57e!-H}4HikN> z*303tvbVS#8Li>CtdroMfh2gSs2Tow>1gt#K_Zw}%H}i#-xNXD8JoM0+=a0Ba}`(; z5W{MTH14z_&(#{e6wo7Mf&0Gy{=3e@Kt2jb5}-Ngyt1$xx$kOeRT15MjYoQZ3$RIv z>{FXR<>Z&##JhUwp3?ovhOomlB_tXM_}*(dM;yn*Q7e35X~ap9e>s|(dU)Tl`Dfov z;`f@Q%;OovJ2RFX$mV2t3SThi{jWBrgc1rC8$ScNhejUv{lFzHn|Q}!ci{tCq|~CI zu-`;|p}%Vj!Fw|6#LlrsUA zMrMU(Oyj%;(Pj{|kyDH>hlq2ILJFU7cN8J%g4CsB&pxf-` zva->T=ydPvB%9eXgLD~&8R{!9IU02Z zhyxh|EO(|{8+s*(K4h8R^H=@P5!IH)-?$l{nC?bUgmMJHM5C*JW!1K7AVM*8khi~E zuTy5bn6{)E$))sKPkTwU&os7++dJl=B07J(oI;1MT9brYQNQmbmN#de^R~pp z{V!6FU)hS3OjJ4@FK!b1FuG-Sh8-gxGd1$wE`BN}Q1g!9EiHO)AqCrx%B}?J3_kepw?|RoxDrw{^CZn9DrooUxUx1g{0yprR z7#qI>W;WQ&26?3AS(5f#h0nxma!yXUHR28n_MYKh9Ww zUlrHV8Nj$^>OHf2r-0@hmi2OdkafjvNH7+O@WpWXwI^eM@R>isZ~4b z^z458q9G9Q0uJ*H7Rg|Ua>eWS; zCBVT+WyKbw$F~dUtv;iBHy zUBd276gKwdhe^F%UCwcC0&$!W?JYFsI!1rDNE>Z$CkxA%X%}zjt$}^cB>wRce`FGI zC@$TcZBrhG0fAq7sCE=gOa`wIadvU6TwQAzo(@&>CsR|N)N!8r6nzTd_Pq%oQY!pU z>yD3!A~YDMPFa8pi<`hB`YxNubiSJ%rTl2~8OQApkpH(D@vj|F#Q_=Oa8}NG-Ya^8 zn@i@T)1DauQ}4*UF}a7RUF%rsRnorW7a%#AsYNOhY&@a(se;d&N>xCwLhZG0gVCE zFosYt<;T$)fcJ|{PNrjSeXIzoA7an6M{}CiUKMj+Oik6p*yaK@^{_x&{}PL?>NJ#Y3&eOd&%9T$qN&0PY9$3O*B6;?-lapu)?gy%x&brR2@ zf`lR{SN^$mAdH7-2aZ_*@51#^|5Re8r&s@R_A#i&dkQvASXh|Nb&&^SeL5)sIIGRw z8p^W@@~ph9ObO5%5ZIcU@AOVHrkFll+YSd$X$Q~Z;^0`+7!g?ZOnfVB0EvIioSJ94 z&PLgME$)ZYU_L@5EhEGBFzV*$7=wEP9NuBH57a4tJ?+20sG@*0Bqpx+J=WiUR6ApD zTY0Z;piKHOwRd+exXfitcAqw<|6^%((C|p&6R1$)HxyfD;TmKw6wT*88yq`VZSDl#OwS$EQLtoOW*=VJz-Y1 zNqEQQU=sUzKRt7wD9SITUu{TnSy>_g7+nn%{Z2@ZDsSiOl?)gE! zsW{sjy1waBbWZf#Ne@=w1KhMS&|_N$#fyYToI;*x=`-9`g}6Q7;^qR}KjB^0T^PQ}Zn* zrH@bX-0ylynpL!o4nijEbe&a5tAxm7nosvrlKt~_%)Cw{;7|~-6qyxqzf3BgSb8(! zYMXO5;FzG)7Gv~`RhdFHrOYRcAooc8-icE{LE%3BU5 z?npwSP^o`yto#xU>_MTRq!eFDpP~c2n1C6ftq&`W;?L2aS~2bOMo*NqjZM+TTBOS+ z#8a*BgPX_0QSG~l@ABQk@a7iA##!X0)TwzFJAzhXa{v*SL`_e+DIqD|7s0(m0niEZ zOBv$b;$hKs7IA+;#2H&~mx_2K?&bg4Xl>OZZCz91#P`S;i;Y6%mf+%%Rz1oO*F+h& zbII6#VHx7$lvEUUtpd#y6ck#Z%N7lHt&rP?yD<83`aR$0doE87)J2x5f6TRKAYpuL zy937f^kdN48Rv5qiSLep;TZ53BMnV-S(&rZ=_p)hZ})vbG%%B=Iq~ZJqnWumH%$US zQuLzuoPPBB^>Fb`O>{sA!sWF^`Oj=(ejoAeE{Xq?&wmG$Gf1JVZUGj`=>mcqx>jb0 zQ(OP#Oz!bpOBbkPPeZty5Iw68-OV{NNO1!fWBa-Ogf-4kB}ZP51Ok7=oGKj1kj$&D z5!s``g@s1pvx#~_*zd4k4;Xy-PImbIAAt!65~*G%PdSfH?N;pU`Z{(jJ4n{J7FyU{ zFJYQ|^!Y-9YpRWfO4U-=mD^W^XI!R4aHQKJ3{YM%f6eOR}n|AF@f`7lFzpgF= z7rKD60LZ6lpX1|YGrr^cKR6ub838z8K3ZY_)@`UN9_1$wp%lNGzTmpLx{%WJC`4kf zui-?jYQ()LtUHi*S_qsBc9z=t@EO1%0Pix2q@Vy2k%sxjZ-vU=0QCR#yuZC!X9lMx z>*Lb`7hp{a3y&E6#>__%HTdcimf=8*Nl%VeAP=k)gJuGRUd>lN@!m4hfOp4VYh{7H znCgbwQ&?a+by1I2l{6VxbF1UeL(v=tMZe{kKX&I|J>uV95C%bT3=LM!si^4am@E9C z^Ivw}LBySHjbgZWPqLYB(jPY4vB+D5#}YYE+e$G*6_WY3njy3|U0u1zW?QkyV)qTe zy+ZNNeGAAjyUZ&4`c%z_-*6A@Mg{|XfaG(ickAah`p0$qA)wJr;rmi{bww`q?i}UT zH~g(j%g9F0)8M?2^9;hYF6q1Pn(2`&!Jjp}@N#%Z&>c=d#;ria{FQSjH#ay~^tMW;e6HXy%h@BaIusBG`B4Ep*Swfg z(eKgXZ+C)X^bF2sE|rX!`L36BjMb#4?;qU}AD$loxcBw;C#=V+0BPBYSVMizS%Hk& zLdqd}u;uA>y=6ZTd4Wpd>55N!VliWmw>~yeGlN%z9+#*#2H}NVb2C4s2w~b<@{04J z#tB8opI!jFYeA+vbvTZ0wF4Zm%r=2I65dU8{q*sUX`eOKxW@V109uX%&{0^3s?`A? z_g_o(_ZO?l6lLZkOn~wSo50!4#U&GBKjX(=sJvgK42hxmAeqa@cBK7RDOm|XIg_7P z6kQhm_U!6PeJ86BZZ6EBIL;1-d=EFNg4MJTO-wPpZxoaj7@LT~I{95vbdY^WFHZ2| zsCb}>7zP{yQbHSq?;Zg$eeDuR{r={NlC*zPG*D6Y5xlMI^UHsgCSdW_-{vZ46`AehS zux-UjF=_cm4zl48VQU^9o_)hh52g>p-fU)*N=e_>>>Djg`juRmMJU9XWJ5B4`1zru zW76<9y2a^!SByD4)SL5NjXqp<6LLL~jxPdstA;^ImPJ0`#Abaw3U7H5DKWU9-z7)v zVnwD{@XV-yva`udH&1_z*J>(;Mn#o>N<#wI1)jpy9^#s|cy-NG7+-Na?N8!j2?A#v zr!=BvtRW9g@vV0271|!Y=vb>lcxHEj^H@p#Z(>d=!xXSyWn^NElfiGFnU{?^)~5i0 z#8gyK`g+oRXsxHI@XgmJr8i7(%MK{%$QJ7ek4hwlwDW^szJ&gQotioEU4f%snDm_R zjgQFn;TxLZD1tbQTm@NS;h62}qG)(;b&XlMR;*;n$?v5#6xa3jjiJD%cC-`Oe6Q*_ zuAAuKL6&YF)yXtoTO0PZoE04#pV80nrvzpVju=3 z_10w4`}~SI!Zg$vQOqdGC*n|v18dLCG3~SJw-(}ykFBusTI<=n0mtpE2+Zmy5}lY)YOc@s+(O! z1r>V;19TVdJcS(JN+@GRD5|E51@Lv->q7}n?6j;DRP$==O6;obG_aR;%sLh7bkUBv zIVhya$jK-QRKf41L)aB&fEHfXMy_fOCG%Y5Wji(WmI@XN!e{!_G8h!%80M+7dY1(| zEUtll0(gyHEWlsF=nHuwUESw+^~wC6XG-h8OO5}z7k~RF9|dHH3qahd6l7Wzqvsjf zHa9n0zf`iOhaR-zea|D0Oeh@}NbEzWkFjnlq(Y24+FFXZJF~!p)6B_HlpCCjc0t0& zhY*O0{U$#nWF`ahNv^|6C{SIc;7VOh>zlDrB<5?Zfw@B)EEI$srP?hE0l7MfPhnw) zSs6rXDjH@4vGPdy_<6|c%vd-t$}Z1Y{Bv7dOTo}E`LhS~z$p^wx$9;QO3^J)L$^Z^ z>bo3(YJS{9z^wJ>N2yOCP_v?1BHW6QP zZ$gR-e}I*?QAPXmTb_Mx=H8ou{QnV@GXl`tZu$y#R#uFwf#AVpDs)iKfxop*5hRn-yp8N4ay%bLcZ_%c+E#cqV4za6mI+|lNt+v} zWx(*+EiLo8dgVA9o@RfaAa@j*h@X>YIPBzJc=}06Kg> zwNHH~Di{p5un=n=TgJ<&s#3GJ)5MbmT)WM!-_IG{)6mj#?I3dB0t0>#&V|Ja$j3#% z`~bNQP$|m3eVbvDHyezaj;cy>OoB~{02lpHJVSyo(o_N(^GPRY6bBz8C!N&yX)$#G z$!f}Of4q*S(AHR0c#N($(1VJ9qE7aV7hD&bvjkaiQf(iSz0~kt!)Q{IFJB0c0F%UM z%}9c#8X?mmw7^8|01qagiCN8tN-$UdP>k@Asm+~D?X>G8sOeF}dut$H71xR32)w<| zkU;0+n1;b~6sh0^V^C0eA!1U{%CGTKsm_=Y5>teu{a`xpMg0H0+kps}L5S8SuK-H~ zOB$@?6A@}#2N%|WT{xqW8&z+NyuEGHEW92>2Q3*QP^aw0IS__r*hgR5#Bg@@b!$wL zIPOOP47VkZ!&+OsGwsVA{2dHEzK%cght4@CG!tQlJY}yZg;HYM z)RT4XvDR(mKVs=``K5%%kKLRDd#-k+|7^q3X~|`#4-m?$3!U4Rr33z1y?}X$2^ekC z+J5Ur1{*Ay`*jRhx&PPd1_I)S#t%v@AVvb4yKFahEi13 zzNHo!pOhg?H@Kd%j>c2hX%6ogE1%R7e=gZ zL?@-JF(^;XE}q$#I_Og;EGzGyH*8H*vbh(+IQLCR)YNMhE;mQF*m;&0TwwIB#r+vv z{~n1r6iDdlRi}%z83E7A3g;aK7a>o4G|b_cdtip3S%Lr#7M8_j|HS^_gq@uonCqFU z0GwtqBv6cAD6e{!FJ?OUMiTMM0OF3G&89$>d4qmH6FE6mTQlqsQ(2Oi*ITE9#)pcm zxfABV;Lao9GZt!WIcHc2YA}Tk(zX=7 z@=1$`_y!MOyh6F7bC40Xcx+;>hfHB6r0rEu+@+itg@q zBZ`ubPiF_D3YgyMn+u>eC^fj9Faf$(a1h2I_l^J$Pn`mPOFb7XV2JsZ&GnE6tDm0| zl)7eL8c{i{&6V^K4|Ie)$_5?8*)_7FNsM#GwPtqZm`ugp6_zN`hMhSy^g{ZYrN!MM zKQFHne9~H#pp^nG^L|)Tb$FGc5aN+?SHLvOOdaKM;<4uJA;#)8?@z3P3M{0H0BJ7v@7skgeFnq)}v|RGP2HH2wX;CnPB$!_s!8RdA}T z)4W=@w)0}!W~ze?u-s`%t<|?!avC!B6}_mv=DteJAUuU8|D3RDg4fbX3`qLS)yS4_vHo+4edK&9aD4jL6CYN4wMn0(IIO4 zB;Tt$?igS6HJ)OY>eu>{BYl0W>nLIi6%}l}u7`Qs#UPW^PtBfJu>+3GfQ(&8bmck` zAR7BO+pF%Y69ADVo*Ad&1BgmnTU(+Hu#eh83LFh&sro{GuC|VYIfyjKI(=_sS=^%I z9O|6-^{Z_+UI8hgsE7!`5_u`r7n4*TZXTH4+%Rl0**aOr7N{35BJwnqqU~qbQr$m% zNP2=TL_n({Ju3su(9h4$5sAUH5-D%PxEz;+K3hjXWujhmx`cWd6TqT*uXjq*GB8XY z(2I>YH}Sd+!J&m8KwN-4FZ1?b$js#A!aek9c8Xk{GJH49x~@!ECF_h+>FrN~?j}!A z9v#1~>&@^yTD$aZZ*A@4TnyEBbFMCO-7_&VD#w5py?(MJR#kDZe_vfCVUG$4k-A7K z_D%bN>D)Uw$O1x3;01ng)k|azNSFg7L~Hy+iqQmEpr9{=&?!J+RZBTxSHu!rQxJ>n zzgE!;57{*)Qn6w-K}JI2NU=zU6Igz18|APbMdzOOVNNqBZ6jir=T`#i{b+#O8hIeR(rnU!0uhj6$-NBH}-Kw&nXSPIg9 zwD(E#&AXN5<{&IEVcm*EIDyd!{(NaGnz$n>m?|W#SH9k93!eP#+BPFJ!=Q8TXr@!^ zu;XZkaHI*r*XnZ_PtS%&z(RS#`7H1e(4NK+Kd!<=qLG;I%Q81xK= z{;Wr9VUY|;eV*!BeD(@7kWFCVf?|VaI|R;bi36BjMZk}f3+wek&~#V*a*Ii0Q>}wv z5b^SU9jRfq9t>ULhsy>)Bjr%INf8M93oybL@7nhg<}qS@9H3KD@Jk~KgMZH^3%JAnj=R|czBaw47b}!Prj+fkJ3VJZJ7&$@OWp=TTs=6j z6&SF!%0NN{j;hvweDb0cA`Z&yKGAVeevSEBT0C&cW2KGM)=V{53Z%DYLR9f1!|Cz{ z0lb@>ywAlhfeeP8#|IjwqoZ)grhmnWhR7gTsf1r561!O1DM%|Dv1!LnA|*`<<_piH zIa8wwx*jjm0&ye^4u7H4P>_6B2((mdpKi0zFmxD@5z>@`$|s}#_>rW-Vl23OidC(v zEv?y8M%pp7uLDl_a)|qvrJXw&wb2ulx8MZx)E0B97-8B~@;6>C#1{rKDKujY&_5K<-K{x5oAL z3T^gL+bZG;$~<=@tE9C>eWC^H3!QKnLw7JFdfdJB_W&U`0F33lJl1Vm{Wf3uA?k3p zr9+Rqq~yZ%_H-?BBiTzNMf(gGsy+wIEtAi#)JT2SJ45$vhS_*kf-$ftZE2dnWYxE;+?0+Yg=WR%c7z?|xvP6zOuUX?|Rftz8HHS_}? z_4@=l7tP#nHe5YST4ik^{ljaiYqR1sJ3zFU;fVsMb5pP!*r$8P>h64N+s|)%)vx1$ z97{k}&An|0CtVk``|$#C=SJi!^)Knj8cHQH>8rt}8wsG~=YMcH zUXe-=fb|=ZD{4^vaJkDQ@4cJFyRpMc;Xn!d?p@l{V_X+H&T47;3(e4d%zR`w!M>T3 zZiFv|rDC;1b{zm`xC))+mGbfl&MLm#v~BlH%ok2>M$~O+sWWl zm@{Z2meR(bp3;X2yB1u4XEqP=8pTqvF&=@_uo0Uwnw=_m08H*UocbX=W8GqvjK0nT z{tV8!2ELOAma|Z^Sk1^X_fm&at65-4RiuAfSt`u!?vSZ>hoMQ&`U%rZ3i$1uL%;dLoI~Hy+QT zd>P`XP<-5@%V!`TJt- zu`akpT)4MaQNVx<0h@_~G<>LA_7-w%C(i)Jp|?FBj+DR@sL}UHeioutp%C)X0N$w@ z=4Jewob6VkWnEXlddZ`9^(2x^PUix(o>!~1p|{1lkIQs-C&w?yR}(b8(L(g7z5OC} z4nr?=bf4v0Z8#Z5+U!%>bJql+KKrSuAfuwe<^$Kgx#pCOn|yh~t$Muk`(ii2&*HKP zNvX-xsq^FG>Ms`!_#31{g64zb2-kww+Piq$PMn{$Z`)`Vf_{xD+=4{n|CpKijT8ui zNY$^e=au+|!)5aY4U^k$@m*>dY~!4k-zl8rh6|p^&3DTQf_3UhKMPLJOA9}G4#uM| z1QU{M1x3oQqeH=G2SYC z>3Fqb@8W{g*wAutTyTQty2R=Cg~dqd2Sn`WROjIrFXEidf=)ngc{P>|AE4NI5J#?N z-n&dS*B(DN}I-{5mH4Q3C(aeRM-#E^FB2{2cDKVH?oh#>s#eiea ztdVj7W3%Mw;p>&!F}gMb_AR_}Y{9V_!&aYePB>xoVOcLKH(pQ-g2(W1X#Yy*0A~)ISE%VOB9c`--?Xt7 zEXHqNv+s5g(lDkawdzF@o#T;#*3~w?yq?k{tz{9?kj6PEOb$%xy2Kr2|FmS*XD+0A)y1dO>drOFsjY+MJA%=Jqz%$DWj-2b3s{%jH*U0NG zG$b<#rQRJkp2*pW!#d7l+vRboYSKa18b0kiVO(P}sbdb>0xf>Srcoy_ael(Sjf0A6 zAvOkJaB=M8PTfh!m&BJ*9!6<7Ih1r-TXO_?9(NwGrFY^TlVkfOT6zQrB=0PQgoFZu zL&QTQ!PPE^%3&i8O(KUu^fTo`EQsoUm}EF2o9z)H^}wj1Zr6$$uAq;9L$1ikTlh0p zv6+oW8AU-=Xtza0Z||DzjUquFQQayFqxO5Vckg%#o#q?7@z`zvb4fG(*q~+?Ot$jO z;D#*HtPXLwHS`tY0XS=ypcCz>yMPmkd?xJV)0){xXM`0}%#h578)A-~ONSxV&%G;X(~)D~=Cm|D3@5FFI%yMENl#B0z0W z%zdbO-$0aT6ezhT_iV-|3JJnz>r^kiTYfD!norUzIoj_G6H2U>Ve>j*U_=VRiaEsC zEj7$-`gabQ!V2`3?iI8d?b3AC7aY_H(9lTfJ9JE~%`JsU0y|YDahYzfbWI{AxLkIJ zDVNhv3I-CHTUQe}RdFE_s3m$hAJ9bTkTvdk8SLv6$wT@FcEI*EUuC z?bTtIMO@CXEYo`Tp+}#q@i44`qekulhd5+>uEG3UcZn@ShnHYJ0!_>iFOOtvte2!8 z2R_G{{HQOtCkF4t)wo8w3}5YR4{cD=(e?9i9*NR4;z{}qL$;ebyMzeCx@w$$gjnow z51x%2txkY|H&}%>!rWPoBwUE-4JAmuHQ;?6QURQ9*j*3b!vx-QRws*>EenElVL)wS zz1g@Y>>Dgd#uvl(`h{Y3tON^8dfR@sR=+jrxqbv5X#_;e#1`GYF0zu>2-H;0ZZWdV z0p5&OcX%-Rx^Wg~YwZC{7)VX$a5^WBqRL8ntBAo`Z@6Cxcpy84Gi)xtS(y2^0Lc*% z;bOwOO#lNo`5BN?CC3``{{t-Wqf&&pTRgN^;#Nfq&Dld2npFw0vTst3Pe;v^E2Kw- z&<{=MRh-r7$PxK0>~{;P3{x+P&1~?Fz!(m$*z|KyxNUX`a59B!|IPlmO$byjHT3cT zb8GwisIpYlP2gNVsuZ;laRC+cbvkHJI4)FtCDe@xV7F>JtR9k{!etu^;7w9ocC!}e zY3+M3thj5&z=t!e_&gL>mZX-5_Ns4+yD<3q4arHVnIFscG(?=SJNfZpT(D?__?MTn zLX9<0btQ7 z>=t=ab{9}h-lkJ1r9-n-R^;?^rjjA{V;5n-nY|2DbeC!WZM`>ui``UfvG9OVvLIwVeRZ|BvK;-ELh@ z1f+uh0(!Rbl8x3H`g&ZJQ`>$xn6GZMO+t{5g0S;P#^cah^3V;(ct4apweOk_A|Ip! z%Qq>NH0)bF`!`#?$H2*SFdu#tYWel^14jS#_Y^y}lit=)C#P#^z)(L^+iur4S`p3o z)x02)$q%}lN#pLD5ZiKAyr<)afC(r43a*b|BvAFiFw&xL%gl{xT zT7SNZs~xjKa4h87ij}1*7@hkSP6f;VY35F?ZDOKwzv)N0)LfXZ+U>ukCQ z_F57Oeqhp`n~TjY87#P{*}t8)FSX;P=9YwqN09qq{dhe87BOLAOrSQ`8$TRN!D{mi z{KYoVN8)(Ua$;F!XNL}#*XdiD;0%IOkj|a@Op&(9N`!XM3nXVz!@xjsbDLk38Wey5 z_vtL)9QaxB(&*#W!E_R81MJMF=w5e{x9#9h`h4$vLs^~EDOMwejEu_p8b8S&n|ZA? zHG{5~@(*=g=R4!z$ux5vfu)nZU63&Dp{hf%a2s6eGvUniUdEYO2?_ z`Nwb5AQB~8U_c-!3Wt`vpclq`Z|*ZN6BZY6X>)JEU>hUK#>dwm;sW665)j}IH6<^1 z^qjXU9XDS2Wn=QP1^5>%c3*q9i+^&)BsGf}sp5JmvW3R~6PHe7U8@e>(t5>Q47~kn z^0Wr^mx}6yT(7C5n`22>!5rP7MnNN{b6KZNsy%>0#{(06ee&E(4Eh|w7A}+;3&HtH zhGjWQU0pW04;`T4oG*~9*Q z2^+xL4n!hs|0@ap$@(WFLIT}gq+ZdQj;dU}e0{riOP(qiZhHy>wcI~kK}h`>rDzJpPoyBb+wK%d?xeyTNhNk)z|UND0ubCAVWb4)+ z;}+Rc`%oa(#M!v}kHdf%&+FSaLN8O(k?estnbxmrQutl+LQH@3Cvfzwo`^EB7q*8f z&ZdceXd4|}NuXxz2vn?5don?DZUttw!o6|&s3&bosZDNSE!S&t+Jx!oaD=|J?;?S#)7X_@Jv&P` z#lZGwQIYsPu#KHASV7~X*ke*X@RnKgdeyXm>lcr0@0`Bl2{c zbF(ys8c?=sQEng0;clRPTW)x5p^W_9Dqy+6TIK3&o56If6*Y>uR=*W4=PCFt1D=rOw|QvHSYe_Q zTzh?1&!wa;()HJjqaM==wdw);JzV53n-~iW{nh|+x2`F}9ZU87O*kv1MUnL3ZHo^A zNI!j@t3Ti5-mn~m1jYgDCeC9!K93*GcUQx0w0NWwxjwjUFS`)C86LL{y5IcjrhQRf znd*IbfX+oFzGx=^QI_+njt0j&z!?(t!YjvE!(~1KDGeKvS`x z!7N!wG>>&g+uWDh2j%{wht&ZFyWp>t&OeXY&EYWbFsblA(I7~6!?A2+gmjrA&nn^t zD)*r>^I2*dV!~4*knz~{_>wiQCzy_MOh{VlT^Td=hyVUAjseotC3;sUTU|wuS~0yb zg|~z+N<~S!Z#hN{xsWN(+Fk=tw~PULoR&j8)~fT#=5s*OeN{Y=>V6|f=06h_gQ25n zZA)~?$3QQg+rBHU@olOgZ!XN}ksR8fAa2d=;=F10^0W0FRf@WRIu|mgLC^?}g>_OGtn<){asCqFK zwq`}`-PIRb*+A}9mf!bauiG$yKV(uh^HP>G}l5r^Z<}QGd2-DSbjB zFpIob-h3l3aC@S#qH3|w)QDy1xIb_1`zDl=X6>8q$fL_p-Gp2xAp&Og7XcV#ufh+7 z1~v@w0}bj@8_xOE^>YtcA_vChPYeyn$6MK6i2Y+cZZ%e+P`{i6rhCf-u3Do zM)c`l_HwNA1nR7Kp=k!Z44894?;+40in0D4@RJ7T&AV;~bEVHuMgnlytMeYG*El#B zO}ue;Sx{Ud)~Yq|d7)YM-eGUPVB&!ExCOmf=2Yu|BCte&B7HjNxmAkrjai`E*!koX z1cluJENvv$Sj!jxaZ(!$hS5|}VqzJ%T@!u(dc%qZT}6(DW-Zq`oV(9$Qlnj?t3$M) z-69+%4z+vw!JO_pgQAEFIz*ri`8B4X9EIKcq|bT&nEf1|sE)|m=CB|On%UCHCP_B7 z>#pXg8^*+2f_HbY1`Q9Ohsm<>iB*1~<8qziy=nB(mo=C=9yiuOr2Nhus|LO;A#efD z5Qc^X?=@dpPG=TRN*VOmaFb$V=Tq8>6SnpezZ*(X%6KjyQsG^IY0qo7LlHwKHD0^X z6Gt!qdb4vtzo5lFa5zgXf#9g+4xT6cyFI>;^)VNyn{z$0F4V%?fz)bZ4NkIf8luF@> zZC%qLVa4k(1WR}-nlw@LqOM@L`;#HOndKJW>=wma>H;3W8*`%NGAg$etA|P|TFlf*HjP`gz~1NyJi=4 zP$3BtZeGa4Wr>ZAErqOoeM}^6T$2Z~utubf+}zyCILF~g$VW#4ta}Hbf;QxvVi6p^ zF+w|lvBB@XGTBs?6;N&T;wlwih#o&8 z?&?jF{^)zmE8U@7YshGEFpX6!S3XF%n9*mCZyO=}Hi3oe^{<-}tKUK-0$3I7fl^lx2{RgZZRY&0RgJhn{svv`S7Er{YRBB@Vv@4N8_Vc)1u z9*Xyi5oqb1`3vTPQX`}Gz~vq3PICPNbfzES;LBlBNz)hJ{hH1vIQPd+Q)?={6pHe(f2&(zj1XRp+S8VCRBh#C zXA^tja_WN`>WXsuq?#9uA9tHMPH{N#DqoTG;!oyhFK?-zSoGmDhSd+NEl3%YX_jFo zFCDZx!P6CfC)60Z6jo&GekoO8|m7u0DgvW}@Fn7C>*6ouZGNp^+hFqRzTRU}st-*ep z-79=kGSUh{kF3WKYwOZ3tq{!ijr_$V?H?b#fIk}VZxIMGA)9kQ5xNYYl_zFT4<3(B z8$zu@=Nv$yBo$EF2yPI;dT7fW9-e(M-gke-Hh-^JQbg3;^F&z0dzD#6z>GBJLV7Ka z;o2`wJEo*rD^2G|h-roNSHF1*U89IF2r4a2*Tbl0WmB_k{b!&P%7# zzHLB4Y1f%>>P#j`>*P#U7;#W@dvy{i4sjN$v)|Gq4{hCt7v3+P`ytVEWzb|cU8V)W z?~TG)Yj%A*Tc0yeQ9aKtCfrWNKFXt+f9fPUIl(Nk=-Aw-ZQyp4&ggh;TGE(}K|CkB zjdbY#lbdT#qq&`2P|$g!=4fi+o(L21ObM zvs)^IRiqxDO{L!sQJN)Jz9y^DKg>6NZ6wtIm0z6h*p8#;h90Ecx4Yty+xphE0GBwA zF4@H|jDm8-d!_6+=rx&{7oj|wuR)ebC`e#YA;07?2bEL7@3jJ^PAhw_wxUT~1X<)n z<1IB-k0lu>vKHu8Dq3B(rq1$M`0$%mKzflKcoef>;eFE0c*;XBjICZB*+&~5gI6%| zv+S#>{(h4W@fL-QRt^oVnYmUt$<&ZHyu2x})Q|E%oAX%w zfs1!Itz*5+i!$o_kYtkmME1`l6RJ1N(U#lc<%=0j0M2=?qQyUT7LWQr57$Q-n87_( z`^F5RRbLlWIDfb2^z_mFq3`WO<46J+W}9C;9iBu14gY!DEZx2zPe=Hw1c*NoV-zno zKF9MN@Jp+A>$bon!jPDO;P)-Ux@CE!ru;58_ZAg%*VUZ|}^h{ZEClSZ|->!w+T=TGBL z*Eq{8+6~aklZz(e^LeDUxU86-4r3xDjF070BA3eihp+43a%QffKb=@IwH|_Lgezpu zRr7C=$D$AnWhgZjmHt2iN7+eE7JaO!G4g^Dbu5J+!O?k<8t^Tm7Yj~F7Xh4&GW59v zY7}+>IO2JBagk>HxG%XRtUburJt{$(BZ25Mqiv}vg?!ID#c19j&y}BhW=|*5`8+y_ zD*YZHQU3}B3KRClU;s|4NuNuEQaT*MEC1#RNoL#DYx}6;uSN9!dkR<_)YR{SPg{|F zPm9A9?CicEKr3Nk9ixg4CQDb|+&mevT1OrUAsr(Wzpe^8+KRK)8{|Y~*Sjd?;gu#? z2Rnk92_|o&w5^uSf;l{zcpG~^n`>@=qitay{UHH0$uF0 zroEqYD;_Zge;tj)M))pGbDFGjp*o%ExMlPleFnK;O7^9QErNx*O0kr0{poqhW3P~; zwtd%wJgj)}CuA?krfGF@TFu76^n4CZl@_W%9B^hIKf*cH4-$0-mP=tC&H40UTh7XOIzUu9gno*22 zqjqy~j1g#bc^fB9rbrmhE}R?pI2~DyB>l3LnMhFmO$2F%^dS3Gg?)XYdgJS}of+om z^#1s`xOqlSZCw1VOwXRlwzUht*TAS8quMa3s}qmXw#(YHJyMG6O>=KVXB{3tKh}@aHhTSpcE`kZl(|yV64$&X zYw-xe7wrL0zJRgw_IQ#IMTwJ3cwD&W$rI$rGw8fo%%r5PU-*~CYxLd*zjp}fz>@jh z*OUGiuPqjpi%?oJzhQmYezu@KflIF4c|=%rlywUd9+um zNJfn(XHi;@{AM-x&L6{A082zZ5E!n-|ND9VFMnDQ!y?;Q(NJ;HNUEy-5EXwDm|BNL ziToiDF}$y#0WO~3u*vnlJp!AI>^V+X*cUJ8s2S0MCg$%dwU22i5h1dzz$2l+P-Sbr zsGLU#RevOQ7kZ#cU4OPWadn7|ok{5zrKp5j<>QpKPTlElV|%XQNKC^78qY9twKXyb$BJ)7(I3hXEgjj2Q7Z@D1<~}#l*WOVF0IfCbss^ zTVdsn4f^n-04DbE93$c7@U+g)NO6-qD;QJ+V(Cck7RCe>xqX0q>W-Y> zfp07@b&ak!3&`gmD}~NC*;d<)%GN|hH9=pVMx%oj=LdO2#YulQMm*b7FCI)l8AUQO z`xDml=k*S=qX15%HXxWw%&bP&Ig~Tndi~J{zIa0&nV866s$B8~9_j2|L`oyzbEIJB zQc9}j(|Nuu1UxbNCOp$U8IMExUyO=mAw6P0my=+=k#RfEHmoyV$f~uxxcqNzyYQ*z_hXQ)A%$m)#RZ16zPCZb#X9)PxfZa04dZT}+U<%QF#wP z|GPj1;&c(!t=B&p^2~eu;DB6$o3_@;>5Z!o3^5TF)9ZwZ-yvR*C)~!ffy&^xa=2eOgR_hpj*`+A zDdqH<;&4yXKt|<={V$jIALb*YXoOc5s?2~q@U7N-w&p|1eRsJ*JeAP^`Z<^l0PBNVv>_{o%{cNVBbcEvCyvl#A7vof0^|fKosUH)x6*thHkN$C&;<51*6t$KgfF zGS?}2up|`l;4}D98SSD!L_lA@#7+wrb#=aZr>YtlY@3phSXSncTv`<-1necaAw_sr zHg`34U%#YsQL?Z^VUv*242&9Y?;=uG&?{QRua}B(EhpEX^|(OXX?Y#?zv9r*7cxDh zx)+5!;d0?I-TyoJ)QA{?R{G}aQ;2jdsL4c9{|o>(^XP|xf#JIk?p}EV zpaT1`uioCm)L&8xGvOy5MWQ}k1e;1(I@y}wKc0v`Qt$sKXR&xghnYsHv0r=c&YgEJ zm3l@iHmpuA&-Fy+i^?Qv+Wfo#HWqodj+Gk+hQ2YQi>^JUkm^iL4dH+Yi6_P*$=5nj zOUB$HBfr^b_}ThK#qiMl8StPV-`vUYe|=DAn}A=&2#uojPYTjs@6?~iEte87RGW?G z0v{jAU{yGD|4?jSzz12Vnwfmr0(XIVY&;^$y96Y5m2Vzp4(tUcrY16I)+Eniq0qu; zL9uXBep)B!4m;dNp7hc|$G;#4_79K)i`mJ4e)%*M>=v{pZ*RDnen3Qd%EdMLzx!z+ zxD7hk;52()3;neinifXdaMMK z)i^@c=7`PZ0w&y{6iSzWJ~>8I2;)UsjHw<1eXkM${J(cBCVNuguTvGx$DlN;xmJ_} zvpAfSS`R+c1_#~l51Hu_@Iwm{pg|3Y z@5X2P`s`pQ40}rzpARD)VlKr#)Xm%<)+$K&4mkeXolir-oNRJSeDASsKl9$Tq4Bxr zw2{>x6X&1@oD%nI5A*=jptQtS#c>;{F5~YbsW!K|xoJ3*Y>tmcD+w7>Qa%8-8r47o zBD5?=(d2_<>@?Pa{1O{2kK_$d=SbGpnSpgbQ#85ip)2B4yrulE?vu>lQUZVf>u?Q< zD%Iw$<19^)It?Ku2K9VJx+DU?(l`BdnW{0g-h1Q7#VuB5d!)EiSVjkyONqC#k=8yb z^>XY{{CT%rtgM>C!Yw7eunEaA3}Q4ib{?;yMEZ@{$>HogU!~}_M*V#y1oC)RxHUTV zeWj%#?$=}i8@szJuaK0ToZtW04W}Yl z(b+kBbz`GINM-Hp|7=2gIiKLjiwyVE$z&Fnxr(KHK{`BzQea<29HZlX>sy#Zg*@Z z8nvnyYHb53o@NyMZgi2^8mo^e7$^e^#8ic`Pv0)+BaQ|g%Js8Ny4|m)8jQa`oHDsi zDW0LuJdPbm)U4`mVtJ&Ze_d5nrr&3vl4!r_jy{WEM1Z(OmB${sV^V&(2Y#HswKK2jrtgVoB;zG<{LW~0N{`Tqsek~$ zrc!NuyV7z$Wsa#DgP2c|$8K}e=9;JRY;UH%d+n@M#9#9I%woDki~9O(*M7>qCsOBk zu^Up{%tWyoBgl8y*1Gi!Fa0joa*3NMFcnmsskWbS|4PdBV?qC3Bp@sja7}Axn+yi% zxs4{sm&wR^JguOrpFVwhO35O}%KFLisgIC z}c#8;G+LDe=4u8v+RBei^buYHZX&iJv6Q0WQgSB~#?GxHRb_-spUs|F&Q4yI{ z*5ca>qZDvZ2t4x<+8O_HjLYaxUaH+f3YZ1)VR1T!iTWN#t z@Vm?9X7DCp&Fi&p=iXi#zk3|Q+z18#(n0&VUe`&r*(n*p2j?|G@IDN|&ocAzECUZO zVz_r70Ke_=t>apw<3iP#qvplA$2_=X(5S|H)Ox+o1t!V6AyZ56wH90){av|^`8C|S z{qCCt!27QDcrE+HAx-;}{b!e_oC^-%eQdgQfF$|;;YFM|jo{XHmL{HViH8rEoK6L% z-G5f@%(o(9wVOoTb$G$Z5a-=S#VseNL+A`JT3{HSo+@xCPLz)(!DyD7W2@%}k{X$Q ziHZk_oF`tU@NJ0LxpZ(Rxf3{V9sclh`1>03@6ch`4%tkF!0Bj7ggCrf^1_sdtzhoG z^q17g=xF)8;UA;OBhcMlnqP8qv`;K{oj;jlRC@Xc_y?tGAzy>)HUNF3B#r*dQ4$&f zLzZ^h9nJqwJyFsI!o&W8ho|dR2p}F+bMwuuHH*^qSmF~-_K&Bo2;1+vYJd!8?`*#)onh{)c0;2K2k5RgxEZSy+`d$? zy;HWm0hiHX{klf^yt^d*TpR7ad@ToE*by<^WNu)g0Dm{Cx-6qiKGNlv(xBvOs68$> zr6LWDeatvYP{RHMzonzC@#C2mW{4C0={^;%reH=!Tj#=ke`K4KP7@k85(YnpkBGO~ z1}?%HU@(up)p#)~ND|F=-Gqr!bDvd<8Dus1&s#$90(N!udS%6NZroWpsI43Siq+A9pHy!RRjyN;i2nL2WiyP~2!i*wz z_olylIA+zI?ScQHy;^NoeGi@XDVaI5I1)^_HFXCPkhwJvQI{m`y=ljtO&*V@(~c82 z+2AjmqbOsV_^T@RGas5$_ayCit=+6ttlX@t@4^c|pC6+1%qRUshvH-LPc9I1ZOv7l3PaF*FE^mSHBz0*vo5J= zVL6f=)@MHX(9BkivG|m>&T>J=OYA!{Er(RZ|dKp3Dzmw`U4N3>8jT^@87&w&R9cDmCGc=)e50n#a5oy4iI`5uy1mR9nX7X2?etz0R%^3>jk_yh(laRFlHX`61kymo20I$Ljk>?(iWh@#>QH zTsGW|kTn3r`i<4zpy2%BURcykpV-7 z*?59>$qvZ~|JdV$XAWGBYdDBg_{Gy|N&fq_YlnOC9v&}=lV6M3xZl~Zn^YUt;4Z;1 zGAdMO;9D|V28D%1wRn0?+k;5x?BpP_=iuex@oA|X7w9Cx#l>euC=Np=c$;?j5ETi= z?gafKD*DUL_-z^%cCPeI!*Z|$HYxURWbt_}f4gN-KU`c~NvBH`ZeS$%*~vv>JC^Y& z*XD+6Y?y9lm?Q3o4=LEh`Ne9fsaoYu2`ZIOGHjGg#=3Yvhle5zionfpEPdnF`ww5z z2op|BOe{Jz)pDCxNDGa!fyiW~^$+h1E(nV34d3;5Wi_=t4{E$`FKadjM%op?Qw&g4 z6H|`v=_hIZH|%;pavrBY`%3>5!k?0lKD>*jylf7f{IUknIL_tCE{!wk`K7R8#+t2_Y^FHO~ z#KnOe?KjuhloWD#H3?K*`=__J?3Ve~o>znf=GN9hdWIX4PHIl}Qnkq-E=7On*nuTZ z{O7a4E(OP1Wf@bQ)2Ne}R1$%KC`*FI^heh6Qhf1YvlU^`v_aucn}RJ^VlX+QPWoFU zAmXioa(Hm^*X1m@-0FyG9wE05k>oHcyvoh3wgNyUdS0a7QSfp0y^|9mEc@D|_4Cn9 z>W8otO~jL1h{T`4h32mh$v=)S{7}s1E4-Es`N@2eX8B@}DufYbzoZ#>VdLWFetM?w z55;V!A%Q>`yb?aZB04Fe0-w2`dA$f}V@oRz*ts~cAL60DVPSs{!v8wlJQ0X_y+y+Yw zAz*-oz00PDdjye^ki)8w9UF$KgEr4dAuZ&CUPS^Ev;Uv1&?br|>me&34PbJ;?ha3+ zfJZ`)hQZraHIadVGm^Gw0-><+iotSLo8cF*0XsjB$;eoqcKi%md7v0)wyn&V1zgsY z%9W9f)Sa`xeEFg&gv+Ib%~cI(}L76D@)0~0XiRZ5sKf?gIW^ZZi*2t3226P-Z=C%G<0`7cL!68I&)sHSz{ScrJ z(7xm!wHCf11fO~}WGp^kUj&vkspiwcI0QO{JK_oox>0p8Jrz3T|MBSWKX4M6SJUmI zp6?O2KgT8H?guKftE0b^FK*}wND4Fj*#goB1`w`xW0*es^ zaM*akN-$&BbBlI-j3SRiI4n)Zimd3u9ukR*W+0ouui-5{ZK#*oceKXP0eG@Jj$1E4 zKZ3P;Zrh&7)(2N0)v$jw2&Acs4(F0eQ7drOHReL!Sq}66eA_d;CNwK3+|}_a;@z^MVRbyqfmYw9FM#OBBHuxp*pDoZh_hN|5;=BFx;$kaC-Wf zBjJF|QBEzL!nLEjTLbQv($m@4VmNg_OMCK*bZXiWMo=fQrD~M`j1oe*6F{n zTC4$>b@&^!`(H$3mvbg+?@wvS1H;);BwN3;LOMD*CNu?EaD_?cuvR_)_s0L<*#?{^ zhKD8ex3M|OEUb7B2f(NneL@4~TLX#)Oc)r1_EONNIy#iu=(FX|yA@8!EB#ya!F@7% z39eTr-S4e(Axm|&Mco#y@k~{6mmUVsck+(I)m1&)l#ec;RevzI z9q+$A%fpIboXHhG@3+aUDC~cawbupmHQTqN2GA#rCix=*1x`e9g~1&kh;^_L)kX8y zVHmUFPYP?wSWpxn@S@L!Yc^kKR4Jx>UPi&8#Y0K@p7CAENYS^U^GvwWlK^+MZ@g$m zZ>DPZ%}bZ{`1>rJin{V0g6VMG*tj@4NW*;-Rwc_5-#Ro549>7M3eT37r$agxomTb>Zt%%cx4vnEjl-ORN1P@c?M;4L8g!2-(13gG78 zp)!retHnPcs!0xU2r_|Upto}3gQ4(usu%Q0z96c2z$^47)!NdmZOR>A0@YpgmoKZh zOe%L}2$5vN7r`5~_V_zs zN^HTSYis7xPDFQ?c5iJcykA z!LD9$YFW74d5m6!NOMZ#t|1(OVEGjt8S(c zAM-xrQpNmKO^di|?d6O53ktuxsMm4-Wg?3!{P|P%{;ki2Zqx`blfFKyBomxYgCNI% z5cI-GQdaiIo$QqcWP6S58a78N@m2iY@I4ym>|&YaDhwHmtMAG7%!2+~-QC~#F14nDd2OOVhOe{daDPoj|zfTGR2O$sNI*z&6SnIL305W zcQ(50zwQAIkT2jaWMt5r3Lp#uC|LEVT=zT~w5rU)chZM1raJ<{={-A`%=`p)LadBQ#Qh66Qm~5`w+63CjzqvuUtc2P-ieviw z-7K$it^flJDM0Z})aj>{kqh zLt4tfZY`(iQ8k!r+0Fo9uY7>fC0^6jBt`k;1yYmCtSpxnUhlhf0hOuS6WXDm#??mVIBxbo}MF! zpUuh#*~G~^>D$OhYIhczwVN?W20mNuRT__Uapki#12)jLfhO&rw;ddK*}nPM9d3Sf zcEV|X-7ezsmyWT2<@)5ayZrs$#Y*&DtiM?d;l`T>bckVI%lg-}(uCBu&9|>#na@!I zgzVSQP%{pLFVmrvQfQQWevfit>0Z1Loqk#so$hn}^IKGVrjuFd#l;oj;c@1C!9ah1 zEOXPHH2!Uk@X?7oGl=Yk+5sdw-&H9@9`@N}q`whDR0+_sGa=?plVS&o1DH+JgYE;! z1ubzfiUj7}&b8uM^@D+5_pP&6A$D=0xlst69)&uk%-REHsDsY5{`77u=aXh0aJ>ET z1PE`pYL;$*nVu=4ba!60%-6i?IVM#67m)zWUvlQrVdr;W8bD}>k`{YVX+EQT3A|eL z&UU7<$(c4W4=B)^jt6;ZU3bVFXDO4}OS0LpDxg1OUt9t*j^27dMkuCf7}+tq*9J4ZPg1G@T$y8JFZD*st9n zkLNjcF#U?v^oyJd5`NtKmBDgC8LnihJdoi(@)obM7H3k0tnD_U#7vdNmm^eG8m zNqRcWdB>o4xie${@lp=$&Ti+*8wuceS6jWkzrlb~ zq;B$dmgF?WthR8bM#){XP9B|9a=alfkKW%Pk5af(mSZlda5U0oL#MUTkeOKqWz?lD zo!%J~D~(fz(CJ;z;>hRf-q!dx`7Obp%g9-koP8T~8>~jv8z-<`n`E)~NJ&)?#E8HT zaHRyMu=(PU!RAsx6$kjUFwEnhbdUmu@(`@opbZl0>MnF3@$Owr^X;yhTInZ0EW5?J za$o8iILblLA^9$$ySvZ4dfuU{3CL#@JzazW)QJ`|8PlC$P)-IdHRP_ZS8`14vm$ds z_bRi8lX{BVnHt{ffYe_Rm2kp1&=RGn^Ae9+?c|2HsA%sNS9W#X)11$4)F?#f1`;`c zD#+YU1!f|bhuj9_ACG90c>r%!19wb$%HMaVB}0t2B^^GhYl&%%5akHf>ekso>`T9Q$9(Vn=r_`$Vn7HVa*x5pmD#&oXe0jHZBPMF5t7o2%>TEFi1K zlj~}iTh{1WTeOW`2HSu%$5*g7{8n{#zt~N*0TV7I*~>!xyBa)Nz#mrXVx|!)GzSd- zxVt_gVnF~#q?8(iORjYVurmc$f;(#tZ@p*A--^H+ZZ}wwdt599bm?4yLX=&X2<(~e z3hUp4w*SqAv@naL$#?dJo|rkU?8e)yZnDG!UcTMlq_ptKJV%G%>La|vm~@{jiF*P5 zEX{p3iU(zB*AMM)RR2U5Dh4tI zAg?vF82Rym$6l<(^RwlyF-xTWl{2%V!d%|Wh{gc9k&k@(`2>r2U^8SOp-A|E{^#%O z{u;;r2YT88i0bKekvYMRDf)cg)Kt?LGlJr(HU6?d`?l8?*H3n(e{ORRbLkVm;UK93 zqK>d-B9D^a*wf1PYvYR{_@lF#k+ARRR3o5RW8f(m#UI(`ZJp5jjuBhIq9`{dc8@qdG$(D^St1AZ{glqjCm54{wDk@3W4^E zD0<=6fw;)wY4-NXCC$nXng08X5T@w*y(aMtr3O(!Ztf}8m1Lp>$cF7yy<=(v&;cB^ zq24A`94fXm+xP9uOqvHit{E0@kke0lStLM!gljFGE z^y-NG1ub~Bpe{}A-u4>B9fwxa>?pELKV(<@baEj1xRU3iVICWo$h`BhlyQO1NAXZX zV|&{!UIg2NBwoj26e3=yBfSR{DR=DI23;2pDQTv+4JwW@{;~nY9t4k_O&AOtj;E8Y zVsgyfL8n~9%)APK(Q4t-K^Fp8nL z=C^Ckj+19=h8SwtMUZWVNC3Lc@c?JyF@`W#mv3ZENyAYz2$eMPnKX^S0CbocWh1mg z{BOfo2Id@z@}s-JwZ^6O6n*X0QOFn~w-8wxqardItMIcuqDPOEe`3O4osJ{B&7BC? z|>qqWnV0vdKjeN#A4rMS^BjFyMuvO=+q$=H^a;2u@BPHUj z04#>m(Ca)x24CsG>YlZrA61zAiFs4hOib!^c4j41mM)DO#HM>+g`bGIS#m6=@jw3d z1t~mgy;W6Ft5FAn~4E~!RH!J%d zoK&R3af9DjM8G;mLd0S6#k%|Td+*3KdX?|xBS@60V4^cg3OI8Ti7G#q@wkdHTWFFx z7E$ItjQozXy^G3}hHS2p5#H#YQ1YwkGqgq{*A}<9ODh2fSD5OEkDSLzJDV&T?7E57BSy( zoj#J0wSbaN4>4MEXigPo_9V98FGx69QkLVqb)4B;S!c}`PRschK*1x}1#7a4vq3sL zxSyws2WRmv`CKEZGS8<=wTD%>c(l%6Qxu-QT0RKf^X~ zKnx3uN8Q)k%TcD0-@vg#eYt)u)`i9++c}7TX#EBm?a7mu=LG!x{2ki+SR$%V;$(xu z!lTc2vDZ}MH1!aEWn^IgR49Ms+uklN|5U4eEqmKWS-A=~{~hi*KBa$bL3~DGML%*T zh39p|OVt{Nnz$f>z1^!T=a~*^NaKDGUvfDpa7Bf=lCbL*p|ijtL$>)L^|~As zvakpg06suYr>#i0`GNW~<@;wzK|Km60khy#MtB_)@#9zXS$)5GL^o{#Chw>wnEz~< zu4)C2h?JD-Aq)HfA`SF&u}=+$dv>*%Jt(7+B^iqP6iG3A+{w+xoUtrvl$8F)EOZ{+#7waP|U$3tGSBxW)fTI(CHMaLxWhBAYmcH&y58#_-DEuY*g}Vq(=9@R9k`0YqBFkY zTv}oD>vtBWdg5WyFl1(qeo+UqIH5jQ-xCEb6ndY^b%=d=*xlK68Ho7dSMQdV<=?^y*`xr~fB`NF@ciHJl*67|#Kx(m5U={zr zS%ANetB2#XC@fC~oWa3cB=o+SeYY12uGO;pQdMKK-^e%=&BTEwfnh0u8Rs1PR`G3x zsHh+wIhgm{2p;pLV6s_T5@k@v_KhaVXLez7iOA$KRETyCWl*6d7TC)M+wnh)YCU0 zqFVsFZ>e0h8l?UDd)Te)edH}@ep<7r{WN!Q>e*=&Dh?t+Tc`rM$n|b^EKJeY= z%PXpE=YcCPwxjVG3;VIkOF6ZAi=ue86xY|P3M#gqEd&H43_$YCZqWwBEYnw=5x6#~ z86Q(|6!{#+B8ueTU$i(qMp|!q4L`=u{f;cmR7sP!cyIJYjIGUje>`pGPFCCp`Z#U( z+PjyItb!KyWRmf9_5?RC7ub=Sb?Ci$anOtIqHsxN`5>t*AaT|rNRHqq}b_NdmeaJl@HOn=n}u(Nx$cnaZcbxdQBYd)k= z|4duta(hKbvvOxL_2>B7cYAkDYS$Hxbf8A|uXSEbq!*ZZHIMr48slj=YMZQ{ZG_Mi`h8PHY2MPXitepMoVlZrxlq?Kyo zM+DMbHAtY-bX2ES$yP$(MfIFR;;hXv<2u!DW}>r}UkD>pS;F97*yq#@?I-x_fu zLf{I3k7m=-gYI3Gsh)irp~ijrczy0%@$E;$l``nb*4&KQBk>w?Ej&`*8ofD`{4;aw z_rrASNUeOFoZY@c0Bx|i_Q(NBcmjDxE@q@9Gen=){(-#`@_a8QW3dscFF@3tS3K;p zk1021onIo^ND%Cj(n)WyUQk~y)V{yIc{}!H#W=y8Ni970N62Mnmfrb*0HPC|zX&(L zWDkvl**jlC{Pj`{u>*MG9FAYt-Q6{Xh<`$eDOz2=J)Y(lkT-v@acuheGsx|YxOiei60i_Yh+vn<$jCTFq83YW z*@IN>HP=1DTw&!v_<=`1rsU@Ps*{11r%UN;xq5`Pke&Z9v~aPTk$KM?+a9_4 zLGv6lbNW3(;LFxX4vClLLeniR#)f&dl)ah!_aQC=E6t3Lnjpv?s}gB^NF?f_EwOTQ ztF=FN%&$r=c)Y0#a3&AoqJFD4;(9mPQf!eeEeng>vPMND{53{T^WACih{eln{KG?5 zv&olgHCE|ZGR+r9sPSn_&enx;tV++s?}7A(#Um4>&J;7KsFTkEK;Zf;DR=Dacmgv% zmAMO=!~mE8|I`xw?O+%IB^Rj6QoP~N7z#=%D(Dr^4WN0&GKB1r1tT*B^rB21L&K3D zrLgd6V4e40a&h&{#`p#jcnUu@20dJ07tb1Oeste zCZHNYQiSEu3`jpb;@+D;b^GTRr6idav!J|_GZ6N6j&FXe3ab3+{e-4%5!UfTQt}wS z8sNk83Sb^Dy_B~7M9!b^X)p>zg-Q$t%Sd>_oR8O?UoF!is3vg&s6PcoT%ZPh-mHTg_YGJ1X1_2S8!I3(GR(UkS=Mph zlvho6S0u>}0V_T#+v6xlqY5&-MJqMf3)MzwUakM--dO2i<&Ngs=;P^U7z~U&PWsO4 zpgbqeZw@c`kVo-^2jyUNq5NxPxoiM##)d4+21a9^13=g;1>Ca~b}t`h?#n=|&f@wz zp4#)2>wJ=BHOp>NUEPAVZ^EGAUL3NnIpbZWlb%jY#nBB4L8Ba~u(eEx85AAb4a6A~ zn!@*Y@Ca~5Up>FcZ_IgWgOw=^Cb$XHe>^!Yi02aVvg#HQVN~jA70=Ydj)(9cG%Tn^W! zB0up|0o8V@&fJuP5$ee1+Zh@fN*qRJy$rBsNl}=$!XhG2K0*k@a==lY!?NZ@9HR!b zin$}{ek%teMaE?T2t>P8?)D__ILz9k6*kaFlbSDg_sxojpwm1SpbQ_i-+V7^Fmxvm zLSbeZzC+(x*W&>j5VrJn1mUc8C0ayxLp(I^&Z?Wa@dSURrqaIHw>Oan)=9JhMJw(X z|7F+x|A_m}u%^;3il!z)0pbMCtDz4u!8+CMP4xwo|>0{)DvLSjnN z2M*cZKqiFWSq)VCkag_ti42X72!!rhyNKq~`iqkzZNeWFzzAzw3J*w)KPMpW4 zc9n36)%KDUP*JsE17+Gw1K>Hbu*wjK^UkY>*(|t%b-d5AmT9ow>#F#>cN-DI;l-rX z22{+q*?O{3oWRWj=W^o`5`|a6^XNh$L%A~7wn6fT7wzstO{C#(!ov%jDJZzPiM#rs zx=i%B#d{yzJwX)*H;+#jGki1!j@;P->vxzZOXyt!F&^fmFcRml%tazwC}`jvaq;pw zdEdd8{FlXitIo3AQCq57=igmVO^rPXfN?&UAwfNb3zfWp6Dw%5;}Atf`+2h_UIKGl zZU`|Z`X5L1PmlDM54pSn6h;I=1SBC7fc68I-{Jgm3s3}_pRDi9*!l4oZ8lPXNz(#= z%#%U%DPmA|emUwlufve}ee0DWvWoT95ih>eAPSOAq=*ZEqB+f>3_Y+ui^DLFMhF>( zx3Hr7RUtLk@p|g<2j=%%YM-uz5|&G`pUKI|Nksyo84!X#7BkBw=dH9IbkhgqHO}9G zz0ujfq7?RxJJs8!A3Qj>7bZP6?K2e<_qhM)!DXL~hw)C?dTt86zfOSs`mBw*cnR1U;fdY6GSu`K}r#deqdj-Q3G zVnHN$BLInu-!^ZDl8+%!6*odCnXI+*dL;Jb$c!$&ngHhI4vcky9={z-IS&YriRaiB z9}|Q%Hr}0Pt{49fR4eGIzOdyw#HY}Ef}N6|Iy($-O#skn2N(J!<@8={KQ-4jP?(zQ ze0insiIUZi)e3xI3?w)*9`FDB3`u|F_v-KWw?+>*YMIkKtBM4RuWFEF^iDKshwdqo zd~bAW=8RKLtv<9F=Xx100?~o0ZZOeaf#Op6K*;Ds#+x_z6mx)@QLS}BZ$rV$F}g;v zgLa8L)crRi?8oXzfKoVWlp~Cf3oS0oe;x*W{UXmPo`Ebo41iQJJY3ylvWwQ?1Ha0t zP`nlV=FNqo<661+^-bMwEsukvZ7K$Ksuy2&z_}7zfObyY$*KSG;a~Y<4+)wm=P}9D{1uB)oDN?py|f6x^1H z6tM-?p%WtQk`4r4yn%|?3A(~9j}aYKla~xDHk3|E zkGt$&+P4GLhyUt&P->VAVf)qw3N&r@89p9U-LhIk{LT^jWk*^b5c5ALyl7XR_sBj@ zkrtV)nP}dSFHCDW?15)lcXA^;b?Q_o^+x4s*67U4e1q_gx#Yz=x32*OsyPOc%Qwew zD3-+V>fQ$wD;!%`q>~CbqC?JKW~Y`m=-i$iXa?VfoumzQ(QK}|LG)Bu2$2K)e} z@z7c`a-HU8rO+~Txw8eW_*+eiJ_>f%mgws|jLVNdI=*u>sRmMlYpYGHQ0hxo#ASfw z+=qii2X&-%-~F4N#n&%+xwwP|TOLNe@w0DPn`#xNx2&)ku4=3T;3B`|?^4*4GH+UO z{7&0L+bN$}ct=f7vh-8!Vk6N92i~-D?U`SxVifN%ZfS3OU9FuZa2r z%Hc#0ZwU(4t#o|*_SnMW^BvFyvxA+tlbaZ0%w~*+R|wm?iIXxS%>WT-@+5UihJ%3+ z!s|%;m&;lJF(}ix=BT^I&+jm-VNg^-@A9punHT51jde>m9ew&o0pBa?BuF(ZTvU^K zH8wlTz{P6+{P~A44@!X2D))oZ`Wpp2iI+&i%?495Hd3^B&aYLA05LX7^lN@SiT<}) zS0dLay)`GS3S7HO_1Vh1p>Zej>!S8X952Bz%u=6*atar2eqCwV0t% zp2#UGs#8B&z37zM9~Xn62{cqcQ|2<)tCHe3MlvO+N+$rad0B(QNnTi@E9V~*wHZXh zp4_tPp6mex;7{!by?^5!;lKemD6iXi2m-Y0dzk&Sh&>+g_(RKr znJ8WP`t1R#1c0abJ^1b%MP7b}kV@koneA<)gDokk=ml0svP)XMJ_6&0I)p`vJ$9DPcBI*Fe8?jps@_Eq|KaSbmOU1zK98*~>~>qHu!mAN_M zYInXe*+(0pJO{-}`JVCLHoxsltuw~vs_UQz^p}_FXvv+&pxQo+*2#W6$d{w<%K*wt ztbD10sqF~hxbXub<2UOnp4EIA>uQX7EvX_Mc zGO|ZIO-&k1yn7Xs)8C15>oM~5oto%n2^h%EA#nl zAT7hjw~1er9{TZZ@IGA z;XcU)-JkrRhW3u-|$!9p*K+yi7upyNa5Uq3uwto{^y z1t%UxzaOHylb3V!G*My=3~=yf%cD&jp6S-*^wD<^lEsb^#ZqfPd2bKXL;L$5-R*Oo zJHo-9gnriq=x%4jfyX8CzrV6ky?^;;diJxL+c|8WN8886?yRG(+PX*Eac|ywBFtKg z_{8)hy-si2^$@Z9QxYpDiuFXri#1c5vkAqk2t-|vkH1L6wn4!DGVC3o+eY6UYX&wf z>TC6b@h#nrbS(^eTfpnnxlWeswLE}dSQmf8eq;qZtT~>q1|w$pN^x_6t=X7T=OoGa zEMsm1uTj9N>GIip+>NWjRVAS1pFrkq2p0iI%E0W?p?yJSnPTcTfv&!k8}2s7ymVHi zUx|Q_m|FH<{=@Rf5@6+9h`7M>j%$IR=E|KciAxHXcZ?fv!#i(F<8BU7Xe3Wi{| zo12s;H`M1bg;8Q72T=gFYy_0`oC-L-RFl4`F}OHgM*WvjmL}#>OS%y*z4datC$vp% zSz6+k83+s;H)OC9{zq-j5!J zviv+~KkuY86bYH^AUJ8Bo!W#5z2vq!-nayRai&-{~hHOd8y|ta=FTeb!Z_DP; zu@NlxhQ}rT%fdsmHwgUD0{GF}GzfVYQ7J^qeu?hqF4uVQ{>rX$wnmCvM;!pb3`DyGgZ~ohlU8ICOvV;B& z?0@^qpTk)aC;L=BDU*tc19jm7c0SVJG+_;q3hDx01d0#H9}vIMDK)&;J-e@ z&~P*iO!sZ^E#BRgbgAYPIUt&(;^6S9x7RgIIIsyUa-V4a(R6qdmM2 z6sdgwp_Y;1du`mJPAYV;AFZ9d!SaJsdhV7e> z=J8iA#HUX;i5?V^{%ntBO~^ixTAE05o4@{tF8wqsw$y1vC6vm{f&cnNSqpkg;NK?Q zi@-pTPpa4;*Qg_6`SRGk?`*4GNb>~-%o-1jWpM41(Uq<~z)V!`0i>Vse|ucl6v+)6 z-oo7I@kp9Hc)sIhDNmQUvi?i61H~bvt}oMc*A%pjjRolWjiQ0;{^id<8}RTz6?-dy z#39Z3^Zc*LS7{g+fLmH$+CvvTdVA6Lv!tW)=d9hWm@Ztt%=M9-y*&&S>2ZrRpo1S6 zy@Ax5+adqf6#rT?VqyCz;0}Kac$qH_SBxL}-s}@KnNn#NwNdhshVg!q14EsvG#mmw3l@)7dy&Q&2jcFxV1gpvBV^ zX3-om!M(wZH^{PFHbQOapN){qJ9Sl*3nVX02KWpCXp)@^P*?ilyZ^fW(^Sdy5Kmyi zt^l_qb~_FRTpxStKB5zU+gdUBtOmKIC{b|TuOC!0DCw2YK;hYqukGp6XBrW|Z5>$l`yeHbXr?N1^?-w44@7c@^1-%eq zMx<>BG2S2X8046Ziq6HKwcF3UO%niIE+fDOT^22U^Yd7a8nbb5$VW!)dr-Z3{d#lh zKgWiP4k3l?`udvLNPq(WL5cN)!2+}S*HHsnS|Dt;Xqj8e-X5tg%<@Sd_f5q1Vk4j(>oy+u=_Nt$t&EvU7vATIsk=b5J@YAH72oB*7Zp)x~uKbIr_wP+qLc=k#0>cqgkgR7<9!6XqWQ(X+GFxsSE?vsv_Ai+q^o-N-uedHZxabhPxUR#p?PK2qU!Sa_*!Au;*5zljJU%J&gel8mFySAmLTt1Y1=FtyxDGkzUexI;ae*ieH zWsm&{Hnc7b%l*9gXVQrn7@uS5nSOYmUE0J3JGp&tDz}*5uX~UzAz|&-vq2P$_hOq@ zFX%_cxPtDPWmj$$(8+^bC*Ei?nU=#LR@&)N++mpR4pJXf2Z@&)h*SL8+*L!F4Y$1B zmDO;2l)#|dLWI*0t8iM4_Z*T$&=^x)!uDNWkwTK{vNV@{b)}l9{Sf;u31h6a>M&4* z;#>6VXJ%h;@q$)$dM8)Qx8n*Urg+iHR{HUsO1+it=?7D(QXKUml&%4@qP9Z&A!5TZ zQv2sD-}hhgCp{hV?WJSJB=WBFRCekYHcrdm=^L-g2TBiIFsBHg>c%~ zAs5Su2GY*Gixg(x4}dDJm2rBURuJvX7+P(u>?ip{Aw$o;t~`nSaA}!H8(it!-z7Kq zsf#29zrN0SNAUg)A2>YEbTFp;oi4dkq$ktaJqo|aBWzIcnLty<4}Tc5Bm_P!B(|OO zSQk^~=!I~)wY{KEZ^v(R6YOajlw~-Nuki!{wMR4O4bk@j%xh@`j`6yNasTD zLtek7eD=;{34xIC%kJxrr>z?E+neB4iEiyNzCk`4MlD(S`Jmyx)f%rc?PlrZ2_8`8 zEMt;eCt{PfVtSTqpwQOucwCjLm5%~nvESMwTI@x|j6Tz5AfETd-pmz2;W%$SFZ{Bg zCv7V?Y42_r1^;rHSwMqzD~Pk)w0H%bZ^*Y$(Cu7RX_JL4eHYobA^iI7>+grklwnO3 z)H+PWl?b`|I?p*ozS-B3>sT81GzzoRSzGSweEj^bF}a92=8KuMNDvBJtd!BiNsp%_ zrlux(;7j&52jvayp+!!T^?w@>BRIR4LyBE%m;<5ii5T0W~s4( zflqB+!RdVN+bx-^&kU~ZZ0)yBy*Fyj=RBRc5p^p=&`^5{F13Gkzn6hhtrH?KFa!#^ zV}OBZMt;svD$_wA^K*mV=4^bBM8V%touP~^+tI^Fi_Z66rBLGwdyqMC0nSK z=-64S4#tAZg5wz&-Ry>{l}=4NJ8kr<><;)HgVHy50uJ}fy_7YX=G3BGq&joIDO37Z zT2TX{TF+pIpKLwmYOGo{FdFV}k-Z2W>{XZpiKc)P$DSVB03rPriO;kW&1ZHM6 zZ-4KuV#}!9DZ{J`n7b7OjK!t+tcvK~1?)U`v8GA{T%yAAh#5wlJe@BNqh0=@pfuTR zVdT;mK}q?AQEw}v6X%h*f7A{2>-I1$_c(hwdVJavf3A!m9Z?L><@gKl~S>qMkP zip-0a?|*XpVrE8<;Ml6N87WsP7#EVm+)wshzgoT+>k%nx`Ux6}$bUF^qiYD|4k+oD zc2?>%^l9rHGF-hP*loH}O9=NwA|UIRiO9%k(+;Tr%ZSsG6xiCJ^O(i6`D|Kwd9Etg zw}p-=G@?@%J}zoMLzqE6k=oH#q)dIh(sWf*q5GQd>dxm_n0Nm(6O^GTtmQp1 zr9}8F`$y2GxY_t^O6}8?qb1>_P&XqASw3dOywWT}dVO}#G*IWo^b~#Bgs2^sE61tB zvTs78^lPqRJC78w^*K2t+uA$pzC7bDo;+n3vN5B?Zm~Q}?i3!JDkS1U3E=ZW7EXrW ztVt50N`@A4BWB^Sg7$&H0a~co49cPsjRNvY-yzzTLfCL3b(RZx+VMy?i=yCs9ma(0 zc{fm2Hc1*?wI_DTw_tTOCFlci6)9Fg%BiM!pvq}49+OF3A z?mITdDd^&m4;9@^x~_I=+=zUY-vJR@+b7b;FFBOfhwu@39tIF4&}^HbctA2%8wd;G zdz0n%KL=_53FQL#`Alq$_>_w``C;$B6SWrO*IjMf+O-M^Gf3=o#w_JhVcYRAnCWTY zVrN@hofy>g>>}MEuarCnd4 zv*#g@*ugwT2c;9Uvu{cqus;OuB<>~V>_;&{8^lcDlBgoLY?^DWwkREuNYyJ{=LP!yBgDDY_s+^T7 zF)N;cY)IP}mCo&@JDY37;GFR#ln!CgGg9faMQE+e%=?3oR58vbW zNJ&=rVwuBOVDv?LFMBC@zj_%_QS@J;wX|X~y-)fd=T$DFTQs23CGCMq6zE}Z{3-f; zZEdT*DJ4SFFjCi(V{6_SH-vvQe*=YqGq|zzq+kLE@ zDvdJx0B^Fgb?r=PjT!>Hh|Op5+ssJOY?$ia>16yGEd-a4ev477BUMOYse7yZ(3SF` zCWSu5EvbMUlE_WK~2pVJ4xIe|z4ks{{)OPXs`$z&egzShzz z;2Hiw#0!q|8qxYoz#P!QVG@uAoA(f$FM-OIAS7W@FmA~b|6ZQkn~Bi$g8Tv?-5NfF zO(tO!hJm9}Pv4e`cM%(uwEbdJ~Gk=1gH zp`d=t`R55Qw5z!+6ZN5{wvgJ{_^85Vyvw_u5!6aPGzqZ@DEBpZnpCo2I4i9?I^&NF_vRiS}5Ngla7fWZ4p zh&~=%1Sv~cb`j7&H|m;efepH=ntLWtDDX+*kk8_nrF`pxDv@E+X@rNHd*?mp?aXz( zyx@j*dYyn0(8JlKF_?Q?C0%X3NN)+U7QN-x1eXF$c@77C*cwR&vT%)P6~rzrgk<(( zy^`5FI(g|iMQ&UnXdbQ77e0hdb-ZIwVLw1EJE{d_ja1d2|B|Nq({qv~2+-g)(S89$ zOz^d$$EWT=7Vj8A&mXU~ruJ+V=kfa&CzfLCsC3~`$GcOHoMaUqKvcX&yB;#6h&K4R z^~o|$HY(n=nzHCu`e>Y2@aerKl)H2pcO_By0Giw7D((vXj{JO*o&aJ1G(xDj3g)rC z{&q4RZ)AIxh>xL{PYt|rG=Vw3XRcnOg0vaTgFStV0Buwq?9JlMP{?G2h&rD^iHL@R z3#xk+`_yeCSM7CIY$?WCRV_5G`9f2R9OXOPciI#d;d2$K%|4MXuY)exz5_3yZ}ABf zy3FTLSM&#wrW4t3m#6Zz^#Rah5He0l;@7h&U_Chg^*-xbAdRs!D8zfQTntQ zh<-%pg4e6C`8b@epiuW@bnr_g-7J$mUz7~07^i)E3%%QK-j|Ht+mau+n{1z)n31#g z9nH5{@MQ11*;N3N*LK@dpW}Nw-5%XtBnar3165LDS}Hcdk|E`zuJ4BjZQtTm^L#_L%2%BYua>-y+RkKM)PrKd^c_7S66>ZD=tcO`KJ>0+I(x1~@tBT> z#5F|CNnG1~AN^zrS8So?8n75$ygwJ2AB- z0CZr9UdH`(+`7{L#~SZ}dD4w$_U#O}^2V*zE1h$J*Kw z3gDE#5 zO;)Wq&5_S#vtT=KZZ;68Eh>r{Xi;3Vo(pwLtmv0Xo{{bja`DBlyXZ zrJSvc>*?E)2NpJ`jP$he(XS^OfGg^c$BDp97d}GlzFPVZWGK1Y&p?_qDy<^-tv!sO zXIX@-U+!D)#usAf%t4kCPjJ_CpMZ^w_|33iOF;@SmItt7mWR!($3?=_r61o zm5URb-jAgXnc0LDiD;eDPqgTnkeb*5(*3Ly)c(FUUjSG6U%OmV>>uPow>MA`kAW1Q zc?4amI-fX{s+|E8j;kz0c6?Fu%Gr#?4;6 ze)rcz-S01@z66{=0jo(LIDc-2tOS|bhubREyZsl2d9!jXo^KEQlCL2<3s~dY=NfG^ z|8X@>KBch`1Ik5>)&0MOxJ0^aw;ggi{%0niAYdA3QoGdipr2EYKkv1Z))U?UpvPLO z@Vx&CH#h@LoxWTp|CgiU=SKhjhX4WqUZWfhY3lu-a6`GQ<$B!T9+)gxievwIpJ4DYd(wBoLd{gTN(!2&oqu4EbC_gj8=>$BsqJk%B*g@XeG9 z#lNC4rvm_KfaZ*FsU?(_r)Tj01$kq;a^=y&TgsmoA)5n<-YH>+R2HFs&CC6P;04eD z!^kK5{C|Qx0ldH#`TvE;$y&a}{0-tuYdr%<392|cMrrojh4} z!*|iz(QkY#PB8Za@{Jj^Lb~TYRb_`)HT64Zf1W~u0R9u~Zpx3C;Kx)kU4LL;bd{IN z_ZnWcW8|BagDo*N89@L$Dx{@tteCS(={Z4L?Dbzan8O6HqwJHUbnpGJS(!!IT1D$c zS4(f|a@28Tg2|3OzJ`+!kqOi1=EEc3#!S(C=aZjfoJ;Wg#{0KDQ>N%&^Q7|SOR*nD zIbe(a)tT@}mD#9Fcbmekn%laH0B?rxy(_;ek{Q?i5&v!S$5(&!a7GzebS$%0H|hU) zJdh+tB1im<4mWwTS{WTC$k@6nWDz?4!Nlu+Lm@Kx8Ca0jH%A6I_{dKT@AAT}wLZU7L zj|XZOerrO2EZqqMX{gd2xY~aplLd$CNbir<3p%5j<-25kbK6+{-cleeI(NbhlVd&G z^ZN;yCPY3tl^jQJ{9NZ~Wv-QmnfXRjI7P(6G}2iV?1LNDB(yAD@^& z6%Lep0*4-{qj9&xCilcyG8cz{(KHkCGiT;Ti%mU$jHWF90zh=}?u?fBqs8^)S2;H$ zG)H_+q!HDfoHSI6sW-qN@;gy(Wh_e{Z?OZzm-hCI%}~W%Bi~t)<)O|LA#Ibn(HKS6jGN%3Of|~uCG|$I%w?|M-67dZikHM8$_A!M4fnp5>?6wA zh%k@Rs`BN}wB|Zgr+iIf`mr9$k_ZTAre5y+SlRG+*=JAn#G6u&tc>r zfG=#Uax)H&1&4$f7ekyl{#cYVABdg|)tfr)U5!r%h>r6TwXXEQx0f0WeppM;g_YnmJ=`BTWn-H0iq2{fY6==g1wJ5UmHT*){oi83qzu1Way#QsDqe+p93 z_8YM(O@t6E^SP`{&q*WIdhr>sq$2@x$n1!pEHo9<*QcWL`klky}nW++29*?JUh&yxTOerlP+GTVws0GaO z-aCQrdAw2MrhtsJ5q_kwZ@+jSD8Sb{pK7(~+bM*1jKCz1T?uxL~Ndu?H`CKS8Zf?#x4+e|r>Uuy7}>vMBF1D`aS{4kiEoADC%0g_vqnk7Kb zYMu-+?jCI)!TLJu>s$^Fg4Wu({1^r{WGjzdA-c^}HOE8Q_Fmhz2UR3*C%@YtMQx+9 zCA76;irm`o?|!GM#%$Ow`|9bH1v|f;VhWd@_9>jz&^F5|_`WK0G`5{sy*NZgacp@Y zQzBih)dUEm$osBkus&}R9wIMYeQOs{=r@t5KC-o>tE2O6PMftw)9*lG*-PCWP+c({ z8wXE8jE_T3-p|X!W7^j;wYDs?8*$PCfB;!!xoW$%ZM=W4rxdkMOFLEM)abN3py{`8 z(ga1t%(et6CG(DiD*)5gYXAiS%1?@b?Pxt&hrBjce5c9fj;B&tF&4)!ah`)L{QXwq zVHh$tHm8pZfeV8uV<(N|h-kvY$!Ug1CUOJ;ycd-k@A1*q?48}D zKm>MSHu?OSQX&RyI@FDOF%XicAD4H*1aT4K>9g<6_cuY}+Ilnldl3;4k>XCK#kbUr z%Xar!SUIoU+qrDNzU}1$^Jczf?b4$vI#}9LC=V;xn4+hp<;%FYf7u>W+M<>^k>Psq z;d48DU?a$|CXH9yvoxE|2Q16;AD7>nnC&;G!j=OzHs6nC7VohGy?Ngl^i zPiUxG>l=`fo2J~-!hc`eEJlYO^X9Pop5F5hI$=txil#4te=eTUb%F98_;=f;Q>US3 z9d-IZ$<>!qblz^k1^E6OriEl z*MkwZtSqx43yhI|sr3tHafioi9eCB|tpZ|yBww4p{w&LEekOF_B zeolnEmjk;cZr%GqhugmI!F6f$atR^w-ZR|P*L}U3^@-J0UpVWu^PwgaYdR)8d|hf7 z)StbhCsVw-$4-+o7F4)jx_ancFf#iUNJLB=8W{7M_`pVHlWS5}b>!yly@BggBlBWm z_CAN|?mme%I`{S}jvZiW>eZpffxuS+5|b8=_sGd#OBdSHmgxlCaWKgTs!^$^yS`K3 zk8B66rbs8lPD&8%7 z-xJueMln_7HF$ZT=t<&P7HLIk)TBndw3l@;b#I5`c)yCrVwTK((B94R?Jt#ui9_W$ z=e3U1;U0p=reBk+K;P9N?eoFyum2(31Q>%;O((~|3^9?E%TPn2T?*In{#)ZxPcb<; zIhqR-l@)mar_5=+ch>joX$Ws~zfPyEz1#Azf?dkApLapc>Yd}noc0wDABrzepL!dQ zEcszK^zZmOK%H%vm-mmnHfFB(ShPicc@eU6nRVWxinI;TKaK@1)9*2#{7BL(WYG}; zkdF<=c5c#BpJR%hw($0brVb?1QFkUtVJs3&*WdQ}V=?BC%nraw3EsDivYD80@qSu3 zTx9a)>2-eos>#Q|g~}p3zywYmABIm(r#Pk#mR+PAX7${mSi|`3_o;qexrd?swE0vF zHn>pN`4+q9_I&zg{PT5U21%Ds$STS>TvBmlRPvQd_l1as3pta?GS%Gm!esB5w*X!t z`p)*u*u>fr)AIaYJbfXAze6gWQtFvV`< z=!tE)t>J_HDgSJ+p&{yVwC=aYxAa-n)x$m)v3!ryGcv01_MTNFsoE93w!TIINv%=> zho`e&krbvt^j&?!FaM@rtR7PC!{qf4Z+Xo0xjhV!nC>})d8Il%b#90F_*1#AwobW6 zZ(m<|glNb$ch;f9lx7|aRn?Zvdz7|9S0GWgF{RZ=UWSQzgQ{>@FEXHnhjrp&xSt#c z56|2iKa~Qh=~j_n!aIIo-=KP5y8rQOAd6?PQ|_5X?Tb9+#$EYR+Z}Y*EH~Kg3F8^i zD~w>l2Cj(Yzt9lqZyNZF^6i(qRceuRVm*eUJIkrFV zQmOVJtDgyS=4ombv-Jj=Ni2<4D-_wsgS_D7lQ^3d%2It++>OXx>lZ$@#I!-qO;AK_A(3thYJ1 z!&9PrKjsaud>=WhVUZeZ;k#LobUT)lZf+pfl>_X%UZYv=v#&IrdzK*+Urf-;!^yW{ zaI|zrzw#;U;w0^T#m6sQC>8|dSRxoh2#%s+( z_ldPV9evEVH`}_i9_Y$vH@eP-ldY4c(fNW*%A5wQJr)}=Q)(J;9kY8URIDs5{KJo! z`z(SFVkNx`)e=3{xzEY*^72k~9o4Vxwc3}7s3^!QaBA`1pJ$pwU`ueYkUJJ*H_Gt= zm1d)9;b3j>dn=Ena~!*V02F%zyzGA0ulW?ey=v+Za18YlVeO{Ew`mb1px9OCHO z^3kl1NUn2C5nVw--aCeZuc;oRJ%aXQfTWJDJIzr*EhWuu@0~QsW1P@16ijlBFgNbS z&(vw)mJGmHAHQ`=<@dnlPPlB(iZhd~y+r2=+a`(GK8c54)h~00P+gA607dLf9!vA7 zvX<-CH<{`-_k&dRJ2%G1CrtN}tSY!H-#`N;lanXvdh>+r*O)l~Q6E>rRMbN)EgoJO zJx=?YqcCV_lC)R;Srt|eI-sh$Y`Ht$SG?@_y$z_e5hi3kmT5Qw8axLaZFal8@{~lK zWjBSjUi5@U=&Z(%!)fEFb1-ugD7YbHz>BqIy~anyS~!WNaQn7--8IUm&l>X|yW5T| zE^U{OR9$=W6x>~8hLuT{ZKnkXdYKO8Q-VG|mw(YdE9l#O31-B6&2jZJT*^-M2DHn2 z<}KB8s5Bn|OTqk0-znCkim8uFu7RY!Jr?SHvPJjo(4+PR!NDn;iU22SDcNi4*msMU zKHF}f%y61NVIp4^CbAH16@|m(PeBX_7z^WV?16UO3lV3Cdq;KRkxr^xdM^Zfz^? zLG@hgLD#mHv~0SASSk%)bUA{Ntpy%JLwJkMq?kppyLIc>1CbOSzttF-s3pb~Pb4#? zw5C~d?D-5qDf`)k9K3-E1QIta>=Svgi8<n`kUCDtmyQn0YM1k2WJ%&sm&7Ilu*r9bB zN=`m+Ne@X?UQMdfSwWpA9?F=plKbceqt`_{TO{puni49HU82>x-t??K`U)|FlBR&& z$`RU&LKJ?j0_Hy@{29>TUpoS~PL9-U8?UXW9;6l+qaFg~$ONRmXje_>?-wbb zsurY_f&usDww!(Cz%@qA<}lOk1sqQyj?-?mi; zWWdve@a#o%B3za16ILB7N3thi)}pQs`H%C*#3`d;cVV{3>3otHdY3B1cZ91V>}l@^ z-36va>;e{>u7uED{dg**k?1LyDf%BAnX+LZrHC4?JAr6^JFc$^ELT-+&9%1k$nYI_ z?>wo~^p%8buo+=Cs@78m&5+23p$e-+(iEo2Pr)wu&Z*|$zQoDN&o{~oSA>dIwsvV) zSbQ4ki@SOPzXS%#m)e;P?|Q>-N^y!~r)Zety3laT&ZjGRaO<1j(I_5tT|%{6DYJZi z08}95xn7cmLwK>@)yTQtF3L!ao+qiV8`7e*PqGx7bb(xSNG+JGgdmFj$kZ zxX9f&xuZjJBexhKl4I>U;cLjWaabzd^~o*~g5Sfc`bB>zyQJ^XTBXxuwA7y8!qY6C zd}ZH_)rD~MNa}PJ5_u0Z655+Oa+C62A-o-q$VKR&L_`pv0ezljRJqzbT%FQ)6{7OQW+>GDx&T*!_ zQzZPYXE8)U@~Z87cqJn@JNvu!H|W@k;_3GH*aXJiQP`ASd!O+^Q=x&0bzcQC#oE(6 z0cyD!x+=mwtOVR#Oy|e|w2{N1pC-mF`*6dZ{_{U`J#+yW$P*D5`iK1vXgzb4le1ZB z`^NV-}^1`@?` zwkJiMM;O+_yJk0sK7d+c*$LJ9+^|Ca->~J6Bou!_ZIT_lMGPafP-c=j}3gP(oIkdOC$;A z4oiNI_gxPJr}m(y4fa-0f>v{ihTH2CTd=0^6wCSRV5yFLWxg9{yDuY9QJ&2&jH`_4 zMyfycJu94ScQCB)0d-?V%?~Vh(NssPk+BF{@j^rai-_Ax(@cRl zmWFmpB6>6F?J`+OTc6X{Lwnk{%AdotU(XJ8mz7)En0iBGdIOD2ZuQ<{)G`0`h zfV61@WyHjDhnY5~uy}E*V7^oxE`iMecWlWUv&Sk`@2OmXoJHk2OXg@t!rJlo3mNFp zkH*Itd66w8G{R!|g0sqiq*zWAkjI=Y>_cozuUIk2IF^t$2(+BB9jLVLB%}`peZx8x z-_p_d+>VqmOM+AGZ3y^rvsRbh2O@?Wr?**SRUUDbFxNqV`~d-Q57=wq^m4O%&AP34 zj@*oYz$sV0Ng|Jx;&3u{%hS(g)d>G;lDy9l?;pIR`eB{|AXmH6VqEA;e1^yBLngbv zoPx&;i`)^(nSDxxG@8y^=qZ}6N7xyvP>6yD3p{&3qSMK(&M`4)MzQ;ttPL9^ovo#% z*mlu9D0S<70mJ=n?8tO-r#r;?tJU4vPa98C#)g;s`udz!14*E9BXcrmKwyb(9$}@> z=IHS{nt{q^sY!ijS|Nx5Y?9Ti72JBWZC0qrqn_@wtNLI?yyIPuDb5gZuK4qQlXywj z$o)2eXLjS7)BaYHw_!c@LT7^5qYC$BuD+PL?T7m#s{FM~WNZ%gB>5#lZo|9l71E9q z$H?O1qDe8$Bl#4+B`fj5<>H^M4vRK`7iCB@O#ja0kmaViR+#(V6@meZJo_3RjWHe9 zR2`z3;DMPJhB!*QHHgeO&5G_rDUT;kAM?o^&kDF0cCf&oc3(j1ujqf2vCk2?I#gx% zc44?$2~!msyU{iVZ!fS&kV5bXe$=h<(l;>RfFV8u<#f(N+e%6HbaBdQgO-kL<4B~LBM#;W3wg#krmMPJBF4RV^Qbpr(XN9TSZljZ}O49OKh036c-Q6k% zX#k^R;-Nl&p8W#1I=~3y_5xdg$@V4-{Zz<26;jXA|JCshXa(+bZ#&6*|3E@?`QFF5 z8yR`8iW@2br9I#Zr?(+ldLMD68@?*amIjF{`2uz16)$DY%(8Ucj2!wx=^RpSJIQCO zC0@;F3h$N`lAE)hziM1BLcwW^YG#9NFpJo|Y^!nA5K8^_jciut=3P(g+m?N?tW_wr zV<(wjfFpN~Ja*F8$I#)9rDZ-u-HT^F`rx603dLPmWG@k>{&e_BfZ6Pr{6br351alp z9fv5=Op4ruir$|~3$Rn~0J5r5?gbkv8r>o{ub6eWUnmJ$c&LxCc~^C~=cSd|7Y)R( zF570bP@M>CIT+#{i_%*`B#1WXgSz5Vb<@7g*qu+t9ERh#Q7%$i&``r-r{thdMhDL40fGB?nX)Eseq5T05Vx6S7xtI$Q-|^0ELz#swpk3wXg%xdVncgRGP}n8&3P`QLky!dbnvE)%f|2+Gll5NUpb?L| zNpXGI&-69;nR9)DdeOoVZISgZ2!$agrzG#Ai{^BaTs47iq5J6+9L6_;^o8AeiT+uHzwYACjS=q z%q1u>f}o_jT7x5lx{3Sn3>pqF1(LqT zmA<4dVPRFOHRgH9DwA`#`pXO>fN(dR#;Lc{$YB#Q`J`0E@~2VF zNl*rwXRy$;PvvxiS2(`q7K1xP^`Y8383osn2zG@B=#n8mnW`mpVYjH@If(P3w_uAp zhQDiw-uf|sXG!va1T2|HB{a?Q+F2o~bt)J!!EWb)+dKg0)M)Lr3x=iWh&UTHyLv`N z#l4gCY(_G}BS1;ox_#k!S`eLmD)mAaKe4s=`W8XX|Tz+ppy=hC8Rj{hC zkYE3(Q!`fT>P8n3jmgoQ8HU<#Tq0g)B(8U1(DCcLtixEMp0*RDC;VhM&VzUxU3=u- zyIhN2%Rbe-O#hvtdyMo9Y&U_nS=?PVj252m=n$8N#h@tTk*DfnGP9-d7_{r<6{r zRJj`@j~4Z~e?uqv5poxo2-)Ownh-g&PfnoR>55qUOxw(lQnN?vOJ)?7A0FA-fm)w_=GT3j>R_cw& zR4w#MjHuv1O@cJtt~Gc|T-#>Zi$&{FehuRFsJqHf8oEZ@T{Q45_6*uS!LR37JI*Kj zSqtBMHBtjfAW&%)px!(!{8;!p3d!UGHi}00HQvYMbpqxMm-2Cw7gYA`Cojvm7^nHI z9UaOBn;1b0NuqB_GEL~b(JC;?q0ii{{#Y5KtlgGzm;Ynk0Kqbt!BjT-z>Lo zyik!*o}cjo{wvqxh2EHg$5uTp_oH1OotD?l?7Jmf1W@isv8PW+6h2O4BA{Cb{uTRFv zUtC(ng!8tlbbCWx@|hFn%WTgrj@Ujg@s#qO%{MR~bSvK~Ve;0~*Hv8}ZOGPo**Vfz zqt5tZAQfK;Dgy%6zn~BsqJ0g!yz}7ePz&Bj=;CL<`x2D$y4ij-yb(K&lx>cymj`m& zH7<9##1+sxCh~gVd_p{M^JJPQ-x`0|5{-(2)O)rF`sInbanb{rfjaae+P`tyyO7-J zB1N7a#Qm+@wgca=9Ej4w-J?WCFq^s$$SlHK>?)(8H9v#NMX>$)a9P*OZ zK#nW`p(rZSp9wm)F=?>|P@3)@ZEI`GK8VXN02G*OA|gF_JYYgfH-5(B^=0Jw9~(h= zb{hSGmS2dAi^M$)%s13@42=_v`S8J@L|I+^Ezm4UDKpazH~LByv$%QL0EE3Wx?Vf& z0nTCLT=ww=2r)prO627kKR@o~61E)H5K17)(iUqzR0<8bEaz*iaara@VwPycuUZH3 zTC@#ZJ}Rx&+AbSuT~%3R5o>k14q+7t&D5sHdPvv>VAbV$W*6(=+gB-R z#32bV64@zQ?NtpWRZhcKg&S8=wxR8*L*@9k-sAGaYmjBaK^RR{SYCK{*9&xid$sm! zb0mA*GHlz-s&mB4A483+wJNXQ@1-AIuN*m`28~ork>j0w%$$bG9+`UW-d`rf%dF1` z$NDTAVGo|yku1?xR8l|XhjXzZ2feOkszR-@x$Ubwj{=< zzP*@;-P^_}P~dv55}m6bh2{7deGxBG-F?ykw4pdOa|@2X=@MCk1}y4$jWW~ z(Bb!4idy~`z{jWnWkB|-UG#o~Y_=~C2#GKjwrj}9!GfoTjye`}d*`>AL8|fcvsJ%x z84@PW95_y@bU?+yZ%zTWfq^J0pt5e0f3uO+gfNwy_czKWJgL)+%6sDtg%G0hipZcC zWo+$hX_-AnwXrgx?7uw5e`|5o`zViW~K|6wnMx)nz zz3Y*&q&miV;b+sfeBM9U0z>GLtJALd%EG1#a9R`c z=t;n>d_})rT(;uvpjUZE*5bp$P^rGWr%%nX^t+<$lyYZB1-CMZEnyo~-BV@q7b_DhnQSvpquH9pLY)#sPTCwiog-sA4`Lq4x_>{Ij51pz@lku|o>k+I!NevTA>X zT$WXx|4<{Sn{B?x^m-(ei(gR;(2j6o0$lT*tl2Ln2 zpZ;xAqnXssnlOUd<_>iOc`mnV8dpV(it0msAdHP2Q03}iV5twTIY5~mJ}}D%*B_Wo z<&my*(sx45y3J-4Nj1rIbdix=Lan%LI?={Ii=Z}8v)ws0)dC~oHF5G>v)3)_e^aGH zm_Nn;^uls%KX72GH}te4_<%lJCmT_o8r6a-KkB+;SrgIxz|mm`6pWJGuotA#TO%s@ zmR0>mZ&$M8MJ##i6%tj6u3cy9h;MB5GoeR+0X5vwDpCWA`G^tjA?c4NN7m^PdqjE= zO1=F004wOGu>5*hk!-5GPf`T>u_~CrRy1MWS1B#%P$3Ma_mL)6aP;ZRyHZR76pSUPY?3I5PzZ$%C} zFF7Rd0ew0Ozh%R;dyoMx z>cvm)x`2O@0Q0Wj?1IDlf4n73)s5{UyH;KCLhoR{`{_oc`r6vZ5)KBuV?;#B>N8Qz z;YPVF%`tmFU)?yM3-g0UN&^hMx&G-Ed&$J0B&Sd2Kv$95KF|w!>i8H%qbPlNXP2I- zJ!@<~PfwwwurL;&sJ1i0R&aU>jP&#gK#|3ENe)so@Ln(49Oy;yo|ClV1NZ0P;1HsB zIN+d;g2Lg3vkE8GO8C<)bHhnffn;CoK#%AqDg13tC~*SPe>wJnHKR*s2}KAdip`uRGQDh1{Ev*-D$?~VywP7PPIi+EZU|kjfby$Ebk;jUoZ;A2e~2M?yXxu7 zyG6FL0C+(!k=Ic>r0a8I$|@fM&3NzG<`hD< zWW&NAUjx8GveINTAvuX;4mVxINj{3SOCIaX>-1dqO4q9e5r+e75wmXMtFO}C<=G@x zb5{}9SI7;=r*zPETvXyL|3DBKt_Qi?2z)CJu+^C_+IFKn`Gxr78+_cW_|N1XF$RIW z4~T|a%=-61N9F-)uV+(;%?`U+@x|E2W|fEDwacr}gqp^q&T}l>`v3{_4!vL37Hdo%Lg(zd-g%s36`_I}ZK zwE_K61rmtuK4$oFzXK@~po?IU(ctha-ZWz>BXWvb=9>irosNm@3U2w7lLap(d3QJ! zZgwnhHk(=Re`vklJFxCv=U)XCf!CI2)<2=hFf#qTRo*#IaW*D#MjE;rw-3>BJiLJ{yxIMF^tfM zczK;;3>>rQ%O*7rmGr0K{2MuA4Vf2H*rq#)OU@SyS5J^%0v$4kwgKj5g_4-cvC|ym zLEm#H^6=GPnvOjxM8tX9Ql3@$7vlZ8wm($cnzY}pG=%2i%X7yB$Hnkvz4T8I#G_dE zo#Nm=V)%~)W3JSwuhOR&E}ulc-F(zudwxm}H+ z(b84Y5Ffho9#C-MFquL+9q7Iy>XXWOr3dH|6DwEOw+Gljofp zuX>C^J&JU*(kx2w+s+JC@7)&L_~2_d`T6<-;98yO9Wek>VzuzC$I(Q!u95eYTnYFk z$jGS3rP5v*+?AAn#I~=akBkng1p@!@#u514aqVR;d<7J7G#69HfWQa2!+&bCEVYA% z5ynWM8u`eT=1SJJ?|EiSMn8T5c)6;90l>XSjV&^t45tsb%GN7+K=954cpb>X*M&@4 zSpD`zN-0oXTszgSaojA{Rq3b@Ge zqkM`-PHF{EUe%_f>)%(Q*UVruAKQHyn-;ik(^O551&e}P6c>2SrD9)v|J-GN) zlf@{_3XetJ9qcus`(^Mi1@q2W+)Qn|6aDZZ<6@P zc!cC`6#0%(76#8)+}QCZ+X~Fi?`a3JF%P${z9565C%j>>&_zqw&TEe{1%79l;V~RL zQKec|eygjcCDsj|+3-Z?nr#i?PPb_Z5~&x4xPUA<5(ZfrjR;o;Q4wuhX48(WGyiLO)CzPW)@)8VgKZ-unx zm+pOe+kf^P8#5TqiQ5^PIUT~T@S+|$DJPH@nH~*ctD0j?>)^>3(xH&*PQOVAMrT3O z#@|JLg#5)j*1f}^*d*5So%@MEkkmi>M|JdAIu%X#7a#}rKCqh>d;yw;ngd(UQ9IKV z7w=75wu_g~{n`+h4c)w$*D-4elf)@huaok+KFsay)WROxG-KA&9s;`6!ls1P0*Fu-eQDGOQzlXENe4LouDgeG6Y-NKMGWJtgqzUVq|CKkB;sU zRmL1?@Nf9GWPHXy^XSSCVRX9@iUJ|BW(+b;;EPK9pEeRnoY=)C&cmthc+=Ki@JIO1 zfQ$AkbW4@gbm-#kbDnt_*6hsf#T!uiShq1pAn`8n)Ay(OkX;{QKYlTYtxVjtuSDH% zH-43TSA$xz_`BDPcwUj_7~E!BOw0l zDo>x5yKmY`5XwCC=H{w7{e|v$+?Ds-wz%tD<2>YRxVaEh`v6a3wBwNbu;?K92w?cI zoNrD<*2y3j;>+{a#Wyy@7K2wQ7J3R*Ps(yNk@QkHoZQdjbJjl-`Ad9>U6vmZ__n*h z40^gIRGPT|% zAN}Kpk6?>KQx~T#y`0V$m%ntt!B20Vyt02&s1dV;wwkt4IHADxdcHG%*7kf#Dq3qz zPaWdZZe;Wb%&t|e912w@RNCLVc~eKGn6}Evx)Rl(?XIjRzn)#cg-h6^@xHrQVl?JU zUS)Ps$0RQlNmK)wu{;4A>)TKkQR3b%9miWNc5_P~R`qqa#;2MfowS55aNh|tmz0)x z%wPEgnYkZa`gyF?p?N^yd+iopJoHU3s*iy%IwseZ))o^~Po4Nq{BC@S@pEQ>EJxF- zPqC`^69KoEK`+HTIkG155|BS3acT?&yqc+e1r{npuhRz=WS{5nQgNW&1qb5mPARhM z?*5}KE?nBDKBah^A~$pK$$>tkI=Yt=aM9ntcFAIwlR`4X3MdaZbzvHBD0DM1WMNLi0sVQiEm5;78G>dui z{+ivr%*%s*S}x_9EtlXtZ~NOGfysq!_EH}S?pNRaOp-8s4;@}QRq+{8kF%_e+8HcX z4O>U_LP6SI^A1y~(l+Q*sa2kc`nA-A4OZ$Di1qg6rS&)3tu9MyjhAIoxL>S_%UQT z0e(BXb&+8It)`Xp+ouR#r~4k?*Ewx^qnPwkg2wOHx8oXg$v$d%{OmYmYo?4wHrEfXIx384q| ze)WrK&5Q&{y9-U5=W+rCu`QX?a999ph-u&>9TDj?S|DhTtAs3gvZP%alV=1` zGY&u{*Sluu3q`V?UsWGuJ98#u;60b9ESG6nMP}K!L*V5R9|LM}<5Urwq(sgM!bd+9 z%jHNCYwfYtSeBRpu8)382;Y}Jf#x*^MGrO8Ww{NG9ibHT#5dhBT`G-Pd{(sROZRYi zsp65BUE&F)ri!(Gk&17}XQ*FNtAR}y!hOVEJkAa7POu^{4_mzz`z@ysocy*CDX+Jz5&fYpB#LhPFYMh+?6t4zNVS- z*9LJM4~(3!$*FO86%G^{o(ds}?N=(FTJvmfQ+ zR%-e5hNyb`%FVj$yR6tTjb!lwok!B0&qxuU#N|agb>3t$I8ICpyccw ze`y*rD1<*lZIblW1whT1v*w*fvMP?F^*PTLUw9EDVcp{DepdOW+&|z+Mo)~4oTxF= zMe%JZ_sf4K<)UTjoU|{7g|!KzA-(f0)aPS0l0@&u7aLeocX_GKoWPP0lqve%g{(~% z^DeI23RoKbSpB37x9^2kqL8ir(-0e=%jBN4i(6PkyIKNpgSHu=$S(`J?^OyKm&Wsc zZSj_aHAiQI19-0Uwl$g@pXy)j;~DwpSIG`A*F{Y+C)9fiHAiS*CIsl0_cguwx_QH> zksF-kRf~rBm-F7sWHW(AYqYmXqf5bw(1r$1#~WvdUB|Z%RL1WsiliEF#FkRaDa-cT zw#r~K-;v695*9|VQ5)2qx+r|{qHe85a>+fr@Vhnx^0*K0lMj^6?D@d=gzIa>F_PY> zKYd&}jle!FUEyV3d)=%OJ-SHN+fRbQbh8fEz|Imr6F1?a^&h!#cNo-{OPmbh6u-r# zi5oGMivra5Y``#VXJT<6?x)vnps&w;&}LvDP$vpQD@L=Xz=^#om|fRQ#3A?js9R@h$z z$NVyOX1J4{G@rcq6k{K^h!FI-271}4k6LFqpn%w!MSNNxN%@fYr=A+${z>iOWuY-rxRcyF=+~)J zfWW^Q*yPrZ8yX1YSCMs4c|1?It3AIBN%x*zKIu$XWm%a^j6Q@n^ z-}}Og+vJ^`zjB^PUkB_IiXyqzuQ^X&(m`)J$Ykj3GT(jKH{vzCI9J`k7p z1JOH(J+QN+in?C`P_Vn{F zRwwA_VrOQ2E-E#6zL8Had_v)30Tiiy}O7)&25~+jf|*)>%PU(J{`Jk*LJVmJ=uaF zpC_~4s&Vuac|?IkQm7j_HlQQ|5`SHocjI>1HP>EG7ABSn7oSGJr`k|B0)xR?+miUp zt-Ioaf_xu6R*Tm+CFK?yZ&o1hZ7#uhf8J(wqo$iM+E7;Vz%C^{?w6_65Fl6KSm|VY zGB3fyaT_lsKPoGKYwl|HPBgo5=%@jcu{fB_Y6m{&>ED(LN1URUVS(?~oQjJH=4{hv z1!JI6ReMp1-&6mUEL3mXE&r<&ZCv;l-^&y#n8~nUBZ=guLeZ`$C@^H5XPf#}qw2CY zXg5>uvTo)CS?=Lh^c%vy{k% zkuswx#KSl4Zi?cfK|M+HtS0oEZwk(-cd5ky3|%@j_w>Jd6gapvT8+mK_i@%Wuy)V8 z#(Bb*_`=uM7ia*)vevLn$(Jj(heeNRjdLA+|KM61Y%hcYH0Mzhy)6db39al23rDmy zrV_1Ch#EuB(uWd=;hPpeA|>KN8NK+Hthvvyvt2LoTeeC>c_O@!m6f`VdSkIg*!Vlk zZjI!Zn?kKJPz{XpFjrIN)u}mrTG>$Y7$8C0+`}iX2Pefo@rOmU&?RR zD)473i>MC8-)>r{mafc zBJ>#sxjZiUE>H7s>+TXAwQ@-C+}<ZT8x41cF&v<@+iJ6)Wf58hLHqgwZ;9~;J$MkQ5v@#&oZW+{=RRr1VaO@o-j|pDp7Tp1s<;eK z=fiC~IGY(ofd5d(|9$aLTs>_AS5}-2%V(sdd}bMkcJLB?J8b6=mb(ai2i&}61`zx#+|wII=h^F4GK#(%zCblOP4*FaYAB znKbUW^wn@J9k%DeL-k!c0m_=0CKDs9y}lCO+i@ z#gMQS7j!)fHr_ol82QM(xfz?{v|Iv4A)ek z#P07#HO?*c*HT8#L~}Xrg`AdT2s!)dANTTKB?cI*jTg+RunDy($%mc;cfg9x&q1-> zQazvNgTWcZttD$Zdz~E>vy=S<>1PST#u5;Nml zuNe7~NS3b~Ih~p8jaUHF=Ryowwvw zkLC8%I*;pIQg@4hv|2rkKH+W8;2_`i?-=#rXA~-mPyFc#SEtJ%u73*D-?|-;CL=&a zAX)d*elJs?{CRUatW8DAj=&Mx5ISimJ5_@S3CN~5(LGDd7HTs(=UVm*XWjQ$LLGCW zy?gVU@!RJNoRY=EeCpUy_f4#BP-XMJFX>zJw&_kNQzx&BM$$@YMYhg^x9r#3^@O4i z?lLhkgHF6ztenhNTcof@`-Gp$_rAWT<+USco- z>iY5ngyAv?5wjkIdsD6YfpN});F`&trng-BI>b}`jqD?qVnXRT1~=`9|Fm*zcgf6u z7~y>(My!7<-Xg?B?6Y9JGsLS-vFL8IRc`XULB+|X`2d*+K@VdZYS2jdSJnGvtK}Uo z@alU!mR2nhHBalL1*?7=D*w4~jDnx3Ha^Z(b01pdxQoD&nTR-!u?Nl<+MnM~l}fa6 zZ}Kj;V%3T_7VObldp|O8vYEv-ueDED_Bv2$M=0+>ekfFyvXBh6nohdNjs(h&);bKh zmwvymZ}m#=J;HBV3gF;s5Zxzvvy+ikbZ5u>CjF$K`xOggP(?k|r};qL4BMhQO{00o zy#>`1r-1x9XhE^3w;^$4=h($G+papGNJeM`t2Jm6;X6b2deD5Y^}Ob@bgng&KRI|R zUGUvihO1}$v8QR{@-HwjupIzJ*CB~AeOuE)FtfaYPc>NT9lef*aTajIXmLTb0QTHY zX6;zLm+H#_F+(^6dYiZQbXuLSqT}D*_8&&@!Ap)SzzQ8v^i4bzs?mcLs!K~*Y*h3b zV%Rk<;$S!GK+e~Gu>Bt9UIO4*;U&mYh@GPd*lKi)NcL!~JvNhFaNepl?EZH8{=AHi z%z@vo!e~tos?98Xv9}PZ`&PM02})UWx+Y}rh&BCN_|mJ{y<^P5U*tu zZ!#^{ujlX-uZguR9o5lgGU1ni zTfx@L$L1x@PbDUx>G;Ljo1QGZdFQfF-NPQFQjhjD!eHwCm51vjKA_7DlKVi+7fOaB z`AlTol=O0u?Q%KI2r=Cc2%tq8gI$+h&WsHw0u!(qW|vn763~TUNEI`P?wU0Xd^ie) z8ovY@#E$x-vB%6pPO)5_n3$*~Ti58=L5s@+PxiV&PK?9^KE_l;h@4A5V?<_+?w20h;e{a5%=4>lK72-Gu(Cd0}mDK;S>SoYsv&D-}a1@$U^}@oG{>;h3&2? zB(b|Mjvy{Vs&Wbnw6_*?Bli?gfl!SbSbA){M5|*R5-m~l zB1=i&0%HrF)1S-7gX1004r*6{yl!T1%j{Go=>OTLPqpY?>RTW~fA1%6)c{J;pwau35MY5mWB3A~ zi%H4Jw3J)R_MTo|c9rkekuL{P1qF|qc$X-*{0&q}YHD^rl4_Bts)8*ScYl%vtRChl zBO|wdnSW}3e?JHC!;Q>Gjd;7Uccgh9`0q5K-eZlbrT}hI5L1>O-Cj&O*^Gy_qb23@ znSSa?j3a%uPj&LGvxMi~K3?rzo7+iJ}5hf zH~7<=cNb_No+c!Z4Rr_?Ku;Hvs;A^UWpOWX>7)u5?xDqu1^H# zLs!3g9TaBi+m(}*`@VqkOzgv)4Kuj|Gj(?WqDpx~3c5^!rr1{U(OQaLUWo-oyP%-V zIhsJ~VxDU6WFIzOR`9O4XN4|*=XfGvlX)urypEzYTN&6bd-5?`J6gmB1}lreHlBk0bNk={%MdX3H#WGd9m;6 zks>Z@-%d1O*qc~ir9I+)(9*NO4H#Yb=hCw7Q z^EO^Xl8}UjjIX>*NXRSTgL>{oliq-%RNusjVD)yMf4S}okCWuTE+Ib?ol3*_8-M-N zrTRR19Z>zNz*Pab0p{;cR@iP%td3mX4dJ9T^a*TiuEz`%r9)W)GQCA4C5NZU{)4a% zS0S42877=UUO9!(DizOP1I zmYQAL2@Jfu_7$68d@O^Gm63DFl9F-7koZdy`1|eKK}taT1@hj!%H*!9)0LoZ?0iQ3l3XK$m`@pU>{{0c1)<;=hP^!4Ti)RXa#v|4ZzZS5PF zu#!3H*v4}lETfBF^_39tw@2Ak=IC!#wVk-7~k}(NNM^4KApxzU1 z%NII%;si7-!2NChFE{bv4L8`A)dmv|&gP}o)w&7*hOI3kZ1{03Gcw$1k^NP0K4t& z67AU5EFY<{I@UcRZa0ZvGO73dh+t;5O$(0J1aphWnxQL(CF))X@c`}QU} zbxA>}A%+B1mwjb~h+4ir`2$*p;!wD56$`|5mm*cWW~cU{Z?#PT#pP$z(iozTDxNrIl+`~#rc=2s%H!ru&Z=lAZYPP4{9JH@nQ(91CE94X zdHEI#e~|We;BDTHUY)!haU|#$AqOM#^R@H5Qs6Nu;`X3!AJ=}0q*J5sHc??-?Azik zu5{0vN0Tby5HC9~pEGY(7?%ZQqq<$2<5OMCx4B4)uSx*CJ|@%<0V|QF`G$=Hs!7DC z9!(GrkQL^vUym;KcFcLjNYCscDOpfbg4wFfv&0bzYH`-y50ZW+N%l?0E-i8V%0cC! zowuD6dgij!RY%4s^X#nyD=RHu91Z~E>bko2?oS^9_N|VT`yKsGAd1+fFnC3NR9+N_ z7SJ9$_9of!<@jp6UT}QE=&d6T-aib$d#02ysD}1pSf!eX)@mqv#x}~Yd@tUTvojiJ zv`q)1)@*5e2RII!ds5Cy|Jql7BO?cg#a!SsRlnZLu>-8`QU*2M4v$8X0|%q)i9uNJ z)XjCzYEpwwg+d`CuZ_A$REQM0fs{8-e#&X>@s#vI+$w1t2;~~R47u?%FS%KHu>>yW zQ>FlVp^+-lB+D_gGYHhMOPpDCCKB5=SZ;Sr-X&62gvnK>ldoh*L)WtRi6~0)iK1eL z;+1tLG8xs9{m2mAxBvJ^0yTj39N++RpyX}-8)tp>2&}1baRYmmKk89m#bob=580T4 z1OY5b2tU8@nu>uXuk*uA{qQg=LN0rm7fJ615CYHrnc`4bQ!={_R6dAJDolPa?H(&z zYF}^+hETXSv0wl+b$u@mc^{l2#WOns6q<=pWA6bdjxLWgWy`nh&hyXUQxjlBOgcCt zxnLmf(XZ*r>kJH4If_LpU3imsQuNl3=08OH=^X``=yC(8Qt|bD_@6mfd>ah?p7OT| zsQZi>iC0zBiYxMHre4vMBW7Ewxsd`d*)aJY+O+>}0$|s<=>tI-kMGrThfY8PPq)K) ze*|&o<(3zVKl15H@^QLqyYJVjTLJ1-xz$9D0Q;6=oDBA7FnB!lol9H9&Ct+>s)owa zMcK33XF)GUc3Ivb0YkbJ^v~kQ(STaYwrPgmDWqrkOH5DhX_-fhyv=(0eYA&i#EIPDv(86PdyTCP3nRk5&m>O*B82`5A*S%kt?Dmj77Is8LVXelGSfnz`Z|t%j&iYRUGkXz078<^`JUUeFvxtC#eOagynS`hC z5af41`FRxUOFNBH?8qZy;p#r5b%BpY|nX#!+#$GiPiD~oqpAXY**r< z4*C<4jNAbyAL?BC)f4l=DebJZ<~IL@vrjTR1UYalgN%ZZ)maBxuz;~nvAe_>At3Pc zsa$=*RC#zsyVjJCLyjds6Iz_T*y=ly@V}ihAZ!pHVC1C!&FgSOG8FY`0g{HM!|-U? zTDgGlNbrCA)`6Yj89*hV$Vv4r;V@7zD+h#b8f;nzL^o|D(WG=#K74ma4qzQ2q`4a@YWmi(1FwD}M(aq)&=T zNVctp^IltStBDj|V>x-sJP?i4`$F{bd_|#ZJeel_kIY>fJm6MrktT77zjeQ>zQAyKFDv5Rjknf6nl~i+%=(^_)K( zu417773aD+WQss%LRn$2zq5qwQQ#R0Ct~&D9;q6s!lbXK(f@OV4o8z=cCdVyst{h4 z-^=&eoY|S}mymp#d@mZ!V%=Y(|Xx@4i~4T|Gn6r^|Y4)m#?GlwHtG8GR2EGD`~Vyjk&mUnraRA6pm5A-jQ?{)K&10n25C8M zCE{?xo)NT4`%0EevtCg>7uTO0Dx||1{NBUvDr!eR89AH(TPFWK>3l1*2V<26bA^WXjdKfkbN@Hs0AkKg$3gP<`0iX`YDjBBoXER94D4jP0;Mlvxnl4E+Q8gpc! zhYE%7*6h)ps)4((aIe7s@}Ldoz?m6qI|qjgnwpxco11>)cH&}U4}pZ_AeG=*V zl`9!46BdAC268x5O-xe9je`N`(bUuw2uS9;62p^|>u^%f>NW|O_PS;AU4V5xJ(rgr z&ur%X=()x74)+g?m_9Vxx+;D1WpZ-z&d$yU3xlpzfj=c?4|Jdf>0SN*8T7xa#sNNv zXPk2yspd)$HjD4=CA?3Q*Wl4ene9C~F;xl1Ozo^1x3;&(Fr{(}32F5H$ZehN$+`|S zgr9EgG*whIG&JNoitkeO_ir?=^VS9EMt0QQ2PHNhKkoHv{!juST*%`e^-E0R`tmd% zL7|n2Ti-&@i(34M*rK|CiA_!xu(Go1JPrPeee=c{e)(;mO2n(+(e&ZD)4uQj6uSlh zJFj3qBX#u8i5&_*06NUKxZ*ooOH(Xd76wJ63XVL7K;XXHbz`n90atHzw95#tJdr>8FrH5dh-X6F~K|D?wx2Tdvbl0>+2>+0(4yu#yaZMDNC>h5%X{i-xnVOM^Q=X%xg41?wMR$l$bKvfPq zXmVWK^@4%|Az`jNcMNEHudGW0Iq2?ZchWFzPjx~w%b<%`X7zUjmvaXki+>d<$4NjZ za$WKGuet<5`w2Ny@3n3@)pDWlfje<)@qJ9pq`hlRR-jd9se_dFM*gw4SyOTPrqPdL zc#;>j?}v&@j`?kyFsLxv-AO9=VjJ6Dv+{KlU}p8Ga8`z-)5)nsUM_Ie5RLm_5MBYC zOru2(Ok4N4L%0MK^s5M&*7H?4;(7^f@vSlC zw8Jm{89p$RjFN-Rh`WggkktR!2;jZ(Qf5U0_UBKv_0LIZK*@vDV;Pd-*JO*p{(3q( zEnyc0EPs9|T@21g{A_7?M|1r2hj^rcW<@#Gg(KGrHP0$$p6eeT9;S^N+xgLM&G9N7 zUH!!O{z+lxB&`eJx1h@byOoQ=#>J`Qe>Gw780H+kgCw2CJEhwPi zXA=QSvGT-CWhJF+2Aq-HH*by^m-D=3J%ps474N>jdeme?EhrOHX?zcMBcRqmbh=2Y`)ue<(XO#CUSC3iCl z3!l8~cpMrUdU2Sk5?zpS6r!c|X}7erRC`4++|VYgxLC~y6Q7)*<;B9uEDR967n3s^ zAV5n;Vi#J{)HKanLsHU%P#^>oyjkx6T1r|eFV@jWyYmc09tHIHgNRF9ebBc(cjOoW zUe3RF*+vc!=4$Mp9yuK5AMXLw8QL@cSKll+=;-O40QLX)@k1I{4D=wf2NQmR531{5 zxNKR48!ES%{uVmv-Zc>t&8;$@lym;tty`a(z>MC{n}QjFLMAUs+*F$BOyxe$qvpoC zAnn+d{^t$ceIB?F<$5L?|Jw*M7-_Wz<_gQoi$cH-4lbYxyG7V+_ubtNCw~jlAQ?JP zm{$q9J^hhdODF$VV4y_x=H}*%wvmy*(9qDivm$CtLdJYe!S|7zQDc8jETif`Jdu3( z|7sD+T~IXlB-s0}y~$$6h1;5(8fbKvj} z_ZUZ*f>qmvLsvXy3OL(+m`M%cA4cnfr|5KweeVy13j6&*3ZQ30_#l+?=O35cfNm8?`5(PrK+`R8-VK z=HS>SOYsSS!(IgkC*dMF5%4dP`XPbBqa`=N*%fmd3d+;%s4(X&U5|0{6DeV{=p0&<6;X zwc6?-BLei#`RV^1N%WaxNlBrG`#3o3xR{v{U!!FnjD2rJ#m@f9h}veZe^j#XZG8Oo z_4ReLl8K}%qcv_Z_zWWB@Ct*HVaz(`#pt)xwys`dRuvV!FLr)9oi*}8jynqe z2>Jg!H_C<>gdabBy6@!l;fk<}G65lBmNvUMOrQcH!wocgT-nQ;`UPzBq0qh?6$WMG(H z@9vtueEr&HVP!QY?{SXRf!1|;--C25zn#_5l1I9_99Qo|zveahl;8o^=M=zX9)`3! zfap>tglO4DI}V;Kiog)=?58oae$-?u|n4-TpfZ$GN%JL9Q+1?`0Ly^* zC8YhrlEwvX0UH)Rq=Z8KCO__GoNB>u3 zYYxtFa^9xU&+7G$gWe~DsVcefb)gg3)m1ae!#^k9U-Zft0_xZ}#Vv9E}HN3ph1GBFBv4T>9wMEQwcCuo`j{ z9UZR++V%Mgd%L?YfnV!<0yJ`>{}%!cF;P*4A(kEzlaXShK%N2WTwOU49V8U#fn?Vw zfk1eidy|A7D8H~O+bt`f)%bbdyXC+vOi_OG{*_XeJcositr#z8HQ4@8Oj;(iJT!GU z#F1LN{$!iITHQQEJz}9_ZM@qsH_Tpc);dO-Ww{w zQMZi0Y?@MnZnP%6#Pz&C^KtiM2sbQ^Qt4xBYe&ACl6B&g&@h*)!FwMYnAs3XdcfHDaoz|wpU1p6#e|Neq@lzIl#(l>utgxIKk}!I4XU-? z2vqwoLh$vzRGf*UfsRexUyS+pFl|>Bge^+DmA~4tf0bO{Z@yf?c+;z2Xp^55%YPm{ znZQ|K_53v)_{XmL-Pv8z06Xt^Xl0#YDA5rxsAes%=^Y4oF5u)%zxBJO;DmDA52{O=a$f5s#~tkUmOJiQOr zq@kKt^zzRe`X3~d#tKB|mC@$&vq$oa8T=Of_4<{`wift%*rl?fT`w zjmaXmU;6qOY;A4l78kiwQc`4>Wij-lqM}xycL0T%F6`2}qKr)4M3on6N^E<{;Ov^q z@)zbRr*Gl1a&q^ir2|1K)0L?`?&nqMtCBoB8n;_q?9}+h$2c(cO-yj`^YgO=5rIy1 zpkI;e2tfc%xuvZ&xLiz%ze%!Gtf=Rctg4|AkrSFTiaV>TA+M$N_Ip&f*l+{H&@bP- z8IGMi@=km+tPULA*hJ4sK%#O&1o`=T+NNXWz}a(yH6A{Uo~ZSB%y;u9GjKXUVhxpd z=UxZDbIhiAyf8E&@`}(0r5MCy2Lf)uZC`&FQm9oTU%!(qvEhU<_T>xHkf$FzyWQQ# zk6!^Y%{@uU#yV7`FVq>SgIL#R{n@~|-vaAKA9XVZq?rG*v`aSwbSmsRiu7u3sfw_G z=1i|yQ#AV-m1Url!!R9120=fx_gg+wtI(Ye5ifNC`zaf?%{LrcR{2}PHh`#HQa!C> zU&joX^4mv9xEn;0YpblpM0IBuo2O1ryoH<>N-ti#*gjOyO6?yGtn&T) z_roJ;B=qHTvaK|)3^(QHm_IH^XOG^PUo#05Y)jkgdRs!hUw7j>f}vsHrFh}h%a6SX zcuwe=%Neks?e@Mq)_JhHOE{oxT@4C8{1048E5DHRmVlw{(AO|mn zi~_H*(6c3C<%C6!p}G8dXs;KD1O_dX;`^*vImux z11yh)ZkzW^oNwv@9gCi{gzQnDOkBi~c*UYvPKZ|K^f=nRe7HNHr08GmeMqiXF8A!2 zG8skYix(=c$LVvMjRDv%y+k!Ay9^kBO)c@Nx8JxIm#9U<;Gn`}@98#KcJe;TDU3VSfG_ zn%j0G=h7@*=sa?o#F2whyS#b*x*ayAb0HsQ2vdP`8k4aRgE&y8%ZNq!gT7ZEVt3c& zaJZr#M&YhDrVV{Pxsdxs1V9L@;}sh6R(Hn`nAf!@?WXG$r+klX2QE;xCGwht2yv?# z>({udou?DEkRC-%dG(em`e?~~0Q~Tpo%Ie9W<=53a9vfMo&^4a7F7$c2uVt{_aH)~ zWxFV{+DE9Aw#F%~JmJ^oHApxhv3*0i|ELL%KK}mrIYP?zi7IDi78dos3l(V$2Gh=W zzRtBiczH*X{p;ZZ z7Uf79gTjHAm-kT?<2AN9LjJ zb@)6l%tv`B^8Q>=bm%0!V8>bHhphSxaH`R_-=Up8iIV*I<)21H7?M_P4?HrKC5{Fa zJCrBTviPE8MCWjDEO=bh!wDjuoK$YDjWgcjyisMgAjzD}uiO+CuA<5zf9DSVLbdx= zMjmBUWO#MOR8F}ACuf~q{*8XDOGB@7KYnzE=ry4MJml z&;<3n;GKyGvYy!3*aT$24e6^?X&)%1sI*V#Jc4q|t?$lxl5wRDR@81FOiW}g)8cZ~GA%?R z4T;C~hY*gbqueS~z7R)$q>u*xuaSuaOWnBrx-F z-@V~=+;pFFAx~_JipmR?pqOV=W)(30Mc{gVJK94ohy{5?TA8e3w+)wjVZzI7%?%u3 zpaIN6{Y=WNdo=cjRB2#2ZWk~_yvSzbH8rB7hCSbp8|T)~)oX)4+C0|MGF=yImiO*s zxIc|@pYfRy+?0Bnc-wln?xM-M$NT!WA`6-Yun`H^vS2)P8$;XMe9p8M5A@*aT*Hxa zhKbLg8;+@p^hBkqWJMiEq0X%*$Hc(xXpAp);o{)lFVzQhUccju{QN^^5|t(r-1hdD zv+dmBGwAB7stWth_CIWUqai8KAWz5sK}^0lf`}R+gY@O$mJztpH8iN z82<47#&k~8S5xvdEwvbkI_YM*)5X~o{+=Wu&W-wGyvJgK_Gr=3QpZf2iOLQfZ^BEQ zd{TtX#S%W7^=W&^o4uk1vvYjpu5&^lpI*2EUF(RVX&}8qV=`Pgx;+=c{gwe=;_9`D z67!gHmPrdNbsTv}JL&5l#Ju?7*Zaiim%_wSWaTOSP=&IKk3LzC5Av7~^PqK(YjXua3 z7=)70b6F$mb9Xpy%`Psc`kCps@sbT@TyR?(D+5KGMe&IsrP=(5XU8h(sDt23!wCfU zYTdP_@#vR?6LOQF{J<|@>lH(Jd*z0CL6T&U%cV_z`C?T>-j*T^%@=)d z9Gn*G=zn(Fc#+(8yj%?tFLqr?qHav}tMBeC-q_0u$@P^`9N*2S^a);1y~3b(w`%oh8>e98yARRk~rP_<3~hZ}e_ zEK~`=JM&5lmC4X9%v}n{FXYmzzMh*Rb2o?pG3{{iQPKFTEe9epz=S#oLhEm24wU#Vn!_rXYs^9J`Sn#+R z@Jd42GMw~M8x=3pXc1da#{!Fj!u|VvjH)Us>9~$JyUie(hVBQOM&$bGFS)q5dV)N2 zjYa*?wHKE%ccX3(mMOLrgsTO@C$zBx7x}MQ-Tw+ra5Tw3|ekq zi91o%72e1gGyyW9+t-Qt>Ln=Wm%S);b*Qw~yC27{_3Y#HE#E?AjE==d~tw`}h zdq7uf_sZTr_xlkO1Ym+d+Qon3dJ4;*-tq6APTI`?Nn5Y5@+3^^Va8e+_J@Tx!}i}n z;z;*ptq#Zw8ELgA-o$1=t={8Hl0~QQu{F~LUCpg_x#HpB!7?HZoqnt&HbgfjXlH&J zxl%%Zx^zd|DDG;*(Ga&{>#t@AOfy^d76K5TZi~^?{ZzIM4bI4|7u}M zX(9Ibl5#2Jl-rcYooRC9t(iow67z1pwl<`BX%=@rKAG>FulU{|8~OSyzv@l&!tI7P zCe#y5w`3W@%Bv8iQLE)3);b8w??C8#b&oJi$h1e1?ttE_$)hAYE~Z_B&ei#hYe&6^ z#LkDc5&)lpHCZKqgrb*eaA-UU<9q?U-fT)o;_R?jk(s&YbPr)rA#%kRZ_{_A)bdqc zrCyy`6h59wX%;7L^v?dcy(RN9*WpZy#M^MIHqcFt?&C35z!9n*<#SoYtZ=f5!R1b{ znen5Hcgu&`)~_k`ZhV=JHRzz!YoTOZ#yRzG;yLyCC|Ox+Kdo;$;DFA$Wbm3*qiB$X z)I9+KvQLAjw-;J)*)@vzX8H6RL+M>tM~r+cNE;i_-S!DqI@54%6rH%!$cdfZc)8*y z3x+xeOl)QBPKiGZIMqcgyQgz(?qq+32!Kl%{~QnL#N;oP78XDtf|Crn)2~l1rTjX? z(`@ao?Q(d4m0VDE?z@Pvs*3a39!(1*szZ!@TA2aQ)!v)!Y|HpX%<$f6p(|Yjh@rmM zK^Q)nVHACwk#=+jl)ag?^u&dqn%d5~Rj+Qe_w(oRA+xpKtCAmxJH`x(4{;H-o_ki0 z#Vo|c9_mDjZ+#@sX4`EJ-g#;8+`ewm+Q1R64%5zp1#X1KWR`8aMwai%1+O^h)i{=^ zr#ws$?9jy?OJ{l}gYitg3i4E>#0i=)g4e{wCbZ?-sGVoQ9TBjCxEdN0|E3s16IX!A zMgt4T?^+%(w_sPhO)NtA?omNFFX_TJt60_rojT8`u6W*(N!p=T&I?_<;NrkQEL?Nm zsXM`3;ei%TrDojzrG|sxlg|w&CAFtL<19RFUep0*Zn$IpX6E$#JLyAfuf|q}2_$$) z@>5fXQP@R$?(2?^--o45xz3Lu30wBZrG9eHp0bHY0vZF0 zpcGH|ekz1}-gsW&uC}gsx2hIXkgSMWNpx<{tWDZEj{Kt}wNT;l`hs2~;x=hww-BpV zI_(4jYmCr6Y8YNVkBZzg%0dgrh($eO9F<(po&BbL@gf1_@D2n{$|Zar=iycUXVTAg z^VfpBywWmmUtdB~r^M5%rKy)0>e|#!Aq8t%aPjaeH0p;3RTI5zDnM~$NzG^b^Ll9eF>?G_XfCSG|F(9}Y>yC1_+;UMZ_N{iTydM7#d=+6r51~{R0{2|sz zJ=Is}?&%X)-WNIO4YvE!ej8$k2*1;YvaIb;GtRN~pfqM}Son(Ngj;EjT%uOnixlym z2``*x1Uzo#C>y0ipff&IK;T-3%URyyG3&M$ew0Sb$(G(<wtT%P!7A`flTLZO*2+yhYESH@R>l zFmz_lzY*y`{;?{5fSmzt`ZM}a^Vdar?h~rMQ#xlQgQ{4cEP2AqKTug9!=ap?J3lKh z1nx<^S2vV4&QK4P7lO*(1@Ikmw=6|8ER8tUGgnm9_arD!9-2h5CP6lASl1n$-ggR< zk&#izTIfzld<0X2kx<7IVopA1lu=mPO9rvO8h7!zKPq{ROohSg> zudJ8m?YEA_mcxR!#HBRPa~F1pqvGbT59b?~>h1Rn6hu*zP7%&qN0MXhPo3{0dw4Zh zL6O$9xu;IM&eQM=6ZWY4l>~Y#=9A%fYQ%M;&fye-1qf8&|kE7=0C^#Vsp$T9#DpY*0ojxo6=;ieJNxmx8M&z=U7 zkTqu1*EcN-6s5?vF23P3SUE}>S8TOXP$;~{!txe{f}Z-cCL~3hxU5(VJVZrs*Y|k= zzGgB+bgL%1kGkRZgEt)g!|nKMHB-Y@BP_Po<7uexR3sJ)TfhDVno=YHh@GkT$80{? z|8TP#{V#yvFwO3g5~vCHmpwhBjWJ{e*iCO8a02E$U>xiN!*48nBA&NUTbL)lPP#UF zr{vpq*5~ZW8*Gd%{MXC!Wns3*M+M*l89qN*g4pq@GoPG{rUrI8dA_I~v1&OJP=X!( zlDQ+QZo&mKE$G|{S<>MBLJOKJ@HN$%o1|1sBQg{zHOGzUMGDO{P)U3k#!~D(8(j91 zKWpA+b91w=j8#m*uvQxJ3HZJ*9`LEQWf;GIh%L=t`pN5qI(AKZvZ{P_QZWrT1Bb!; zIec=0x$3^`@`9Gh$!^uk^78U{)U%S)88r}?=j&i&VU_V6>6@07*wUuT$gQ@~`*9$L zeKL@)(a#rC{PfMblWg-9->;!hk6nnr*tAV{ArJhqrpCw1ZDX1AN(M@)1#Hu;an}4C zC5i>|`5j35#D%Y2Qys29$SC3#p7P9VL3pjb+R%t`9r0KC$(>~3DLxAkZ!Gqx{;Rk_ zNsi9h% zA`a~}-v=So z*bV@U$7~$XoDxW>tzGBD8bFV&w_;x9RrKYH0ecU&W$V|1z7LvPfB5#2(bOeajp-Dhoe9DPyBT1LLDtqY3 zi$L|HIx$k^^w!ln^$Hkmc3V}{-OEPWVJV*DDiww)=!>!WZ#FbQ$NWT%9c_${p|03cRpXob%ZP+ROZkg2hH{uPkQ`zf`Jn*pv6;LYjH3;mtUr3+ygkrmM>?Uc>ErBEPjJ3g&l zV^mXF=}vC!VG&E0FB>X%zC?e$3=Ux?+hv%j0^nsxNL<+wXJe3sCdq{+WNF?2^KDi{ z(oP$_&$S3j&S4&bPo*0+7P+^j-Xy9QrzhOJ_7O^`n{;c~>mz!jcBi~33gA`ZxzU8g zv{un!4mVgA<84NYn}R+Tr06pP|FNH;^KdK!a_G7?b|bGE_v%>KoHwce+Gt7JiKy+E z%lCjFSwmSl4F~UB2jGq>G_Ecp&ByXHUKYcL8%sq3xX}Ka$lc|~*+)AeA?D>KKOzr4Hm5^zFUl~P{E?CSFi@OL^86Z?33D;8^I#kN(H@ozf#kYq6P z$^&9W756b}<2EDRy4;6yLwV#(XXpLzfjKWN-;-yPj(VpK>k~qU1IK=RDA)cBIX;i= zC*;MR?(PYalRg)o1^$zDCJmqa=V$M}1F^hygdlW_Uv3fs%b{4g2&Am0&30SPFFrB-AJr#oRm# zaD#YMjj4IJR>9g+mA!D13o}Q7yQjg^_(ZDBb9A!-Ra-Z4*GC5}AMXjWM)HbB`a`8b z`8``6gD~HNCJ;`X*rgM5{aGoPiUVZi@IEJhh5k^MKT<5J7`757#ubf&XPEpC1H`3XSN3Ddmj4fT5~X=qWj872vpT7 z|6L3{ciV@OZ*?fN1sgqCv0{>7M^I!>^d3|+6oW`DIQLB}f8%A$?0HFwbW8H2B#ve3 z1DHO*;V4b&Izde@ayuqj*$3*X9!x=AdqapZ?7oYzTwTDq#sAu$LNyc#$tN43xOg!h z71WU0AZT5Kbmuzo=o)#?XOH#1lpChysoF+2u+&tI>p==hTka${6T@N zCb$Q=(H35#V-MI?+77-Y7)qR4Gw~gW0=aru^MCAKF8(xcMNB>_=SKO zGFsMAW+wZbPaQjFW`-1h&;hWx0>@kP?J5j!VyMtUrT)?c3*Ir?2fsAcyXI!bR9-A|EuoEA{rY=|LFsV;Q%0sj}V)v&MKjlV*z$bFRI-S+@EB zC;x0wCg7NxJQ;ed6o%3W0N_{919i<121-;e=u#^M%-=Qd8L&meqJ>|kNbvI0-ur)M%9`a(oyVui_Iz0C6hV2WlC*8crS|h=;5cO2dN*u@rH}=RakQR~e^z>@2(uA5}~gNW`EXiWO|IAVwkzJ>)kO z6Y<+=d&l=h3pZ5sj#k*=ow+YEpKR3Eu?%um=`(S5b7p@a+i(j)evadgBRa+Kr?{wn zJnx2W{ngl@mYeihX0@Q6j22PS#w#Up%k{H`ZUZYrok9I{>USu|X#KMfS(PXh`vX|j zmVWE0lxayx00~waLCHM?vmDL0iHR#)yW6jgW?Pt{a)xbcdb0MfL^x#E!SkY0&c zdp{(_O>VT(!7C*(PW#%!hj6;kS3eQ_l+zjr(vr=BaicBb!WvuEDgzpE#?xt8n#FC9 zf`wJZ-F?M8IgB^Z=k@af9z}#aDOzX}K$K3h1SZxToh_5!at}k~E3^#*V&&!M-*>i2!p7o73Kt>(e3O_jHtyV1Zb2B)cYZ2b?S(tBT7pGM zgmM?}g!7)THt9TF*@{Q-axElTM>}CpQ|%!lskDv~$F!&CoU^|A_yw5$M=#<`5O{9bW7rf9#(^S%4oM5Rj6V^DA+4&U=kuR99aEPlX%;d=?>U(dF)9*nJ*%bt`{ zI}f(2?s$cJfu>iMJvS6+3+D}qdv8yB51d3g3{OtVbX$$(-1&wKz{bO&D2J#toSo9g zcdqOc@6se)YSvG>I(9gB<#BaYNZy+Fj~qil6XT|a-y!zl(o>Xz#(CX-xbGgj$8uKO z(a;y-$=Km82MD`lO~I$NACb|IboQw(My!FJmh}ZpMRP`Ct`c@<#&=-@l*!4&Hm~@Y zT1AD&&o^Be%be?{!QM%ySZN^w$*garpO{&$CEU?iY!U~AhlSb7n7xGgorBAZ5upVE zn-@qlT85Ai%B^Brh=U66v$htZuDI->CU+Mo!)mgqP(jOrJNnFiW6HQNzemuYTizU3 z)`2veiMElj?BZj>+_IrBvQ{<)VLruKc}LmH*lQ1#bzWu1jf}jD>kBPsfhk%&qE|?; zD%dr)eP<=Dg98hc#A<03Z1(TMAZX!HKnZuR60P7q#s4sTwD$GMzP@=+*O4!MSIc2) z8RCH9z;?QihZMD1yZYs`c1(!9N87C1arHsEfAEmiw42vQ5}I2+GlnO<&l)6fW4CzG z&%`{C=u}<{-|Y@ooeDzzdT$TS4_^o74IZVooqgWI#@(7;4Zwc7cQA6V&=mM7CUNx_ z7$p5MFtKnxP+EMh7H%c`{htuh9}jmm4Q(h~bxmnBNMa_V-M#NQ`hBTP5Glfm!nD!Q z*jX9af;&I_Z43x$yILbbpRPy&7Z%P>BYYn!;On`$&K|&Ltgv&eG3!nkhFLFmz13MG z^YVqVkH1)Ph2phbSh2z`yc@?&*7voyBVzgWzfcP9FWM*ucPd> zo_twW;ygE=a$9mnfdKU6ES2uAudgq*o2<4{`ZDF<0To%+8WPWuk-3xaBYWHoI7QlC zhrX7h9~|btWNoZaj+lP{u*r8pfO$J)gZG(hPi$QMk8FN?N{>}QWyLpsgf2OY68cO$ z%6#vMVi6>upy)`GqTManJk^H<_%^(ND#E=wRVVV%;TDimfw+X)8gX^nSPb-wY`PEG z+4YO}{v)m=RY?rbM|4Vmtid1e$--Ai7UrLD8Bm1Xu`DYFF_wU-==z4UWG3o)&rh zKNtGPivRiWJjEa5R$aZ@8kaV8)EQYT&k8IL;vdG9V-KdBHY|e;rE^L=R1BH zRPw1mMk(eep8xr_WqgR_0P^i+@?Q?|`x5psFksmF^ZxI*d4hTGdYXOu)qH=5~a(>EGS-@1gtuRX1_82V!Di5Le%ml~Vnoo_;+1-+urdsIf9LGjq1s zxEX)43BN1!-+psQa7D;lF!G9o{v~(dc8B1TvkW5gQm)bZmseCE)_EtEHa6h!WKss@ zqQ<4At<8nm+n>)V3;lyR5e#kw{7m<xy?^xtyDwh6H-R6C zv(A}la2^}`Sc%G6%3sn0Yb$0ynpR|`d44+L9y#j};^7}g`!7aBav963AUN3T7!~xw zq|GWkz7Fc@I{cr>>N|BX8?6lHmHFtNJNgoF2K#Nw*^PaD$|IX!BA{Eew6wF^9`yO& zfB#ES1bjIsVCyMVd-#clzXcQq`k;Awnee^KNf@d=P&5M3N5IU(14@wP+hr_d)clM9pcb* z?$RLQM(THomz4!`Px*SS`PGiS+iSw}BTw=0go-T$VFWng$ji$&v)8zW;Q9LaEfuWm z(8W0S2(j*VrO<+oBbnbMb6@|wtUrk^Np|LKPHH|S1)7(L=zMuSdwpE}la33ROZU2Z zMjkt!TLP%Y1pujNob|}ykp$CE9M^45IG?@0P^#`I&EOY6<@>2`2i%b+C{Pzuza(nf zIS2|E)6O<84KC|K`M)U+9MSu@3BOb;DTg5ZE45SX<>4NCcdaMwosHPbZfpy%}L|bF-%`H2>ivMeDs*765-sPY~`d`-xM` zEx_ESPlt*vwuWYFcUVF%PfoHP-b-PUWMB?BI6GPDQ>LP#g1~D(+!Wmooeph~*g{kX zxks#OiHR94o7S+|WU&kOX02CZR_M|{E#b8QKo%{VdLZF2 zBT4K1>1h6V38Ckvr&;A>s#$C(B~Tzy5NRk)iD_Z;h)AkJopFg|qlQ~_dnwasdaBNG z1gxDA*46+UEp75p=2fEe=P#<)IsJoc+?aQ6uDPk#2mNMq56%e+mT?g!Cnx9lsxVEz zP{W|^Hi39<&NOXi(x1JnKDWM746>7b2t?dn5rtP60Vt+E{c|agdr~jVb@sh@@nXfk zDFC%SU;E}C&chQ{(x<%uAqviV5EtULUd0MFAZuc5s!4Yz7X~dm2QCARfDI%Du1$e1 zGgBPVvt1Duy{1NlL%UYcdw(IZWo`ut>g+t+2u=P5?-anGb&g?|CZYAb`dhp+Qsrzj zqJf3OcVTID6{4Zxm+R%}sjWs#LJ~SPHC61f^Ty$z{Y5038o@tpeX)QD?v%^oe15+1 z0=C+&?BdgGFMTkMh6DBfx8n_$B$6E8=2SBMhmG-{-TRt_0fQ4Zzry@WW%{2vJ%0~E zo%sc}sDD_A?;1nW5=0DNgp2h57T*5!U6%_-*)qC-kp`{@p|Wa6-tb5HJ}Y>~8K;Qs*@cXZAG literal 0 HcmV?d00001 diff --git a/s3stream/build.gradle b/s3stream/build.gradle new file mode 100644 index 0000000000..f49f3c654a --- /dev/null +++ b/s3stream/build.gradle @@ -0,0 +1,3 @@ +tasks.withType(JavaCompile) { + options.compilerArgs += ["-Xlint:-this-escape"] +} \ No newline at end of file diff --git a/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java b/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java index 9d26588509..f6e47a12bb 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java +++ b/s3stream/src/main/java/com/automq/stream/s3/S3Stream.java @@ -38,9 +38,9 @@ import com.automq.stream.s3.streams.StreamMetadataListener; import com.automq.stream.utils.FutureUtil; import com.automq.stream.utils.GlobalSwitch; +import com.automq.stream.utils.LogContext; import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -67,7 +67,7 @@ import static com.automq.stream.utils.FutureUtil.propagate; public class S3Stream implements Stream, StreamMetadataListener { - private static final Logger LOGGER = LoggerFactory.getLogger(S3Stream.class); + private final Logger logger; final AtomicLong confirmOffset; private final String logIdent; private final long streamId; @@ -103,7 +103,8 @@ public S3Stream(long streamId, long epoch, long startOffset, long nextOffset, St this.streamId = streamId; this.epoch = epoch; this.startOffset = startOffset; - this.logIdent = "[streamId=" + streamId + " epoch=" + epoch + "]"; + this.logIdent = "[streamId=" + streamId + " epoch=" + epoch + "] "; + this.logger = new LogContext(logIdent).logger(S3Stream.class); this.nextOffset = new AtomicLong(nextOffset); this.confirmOffset = new AtomicLong(nextOffset); this.status = new Status(); @@ -176,7 +177,7 @@ public CompletableFuture append(AppendContext context, RecordBatch } finally { appendLock.unlock(); } - }, LOGGER, "append"); + }, logger, "append"); pendingAppends.add(cf); pendingAppendTimestamps.push(startTimeNanos); return cf.whenComplete((nil, ex) -> { @@ -192,7 +193,7 @@ public CompletableFuture append(AppendContext context, RecordBatch @WithSpan private CompletableFuture append0(AppendContext context, RecordBatch recordBatch) { if (!status.isWritable()) { - return FutureUtil.failedFuture(new StreamClientException(ErrorCode.STREAM_ALREADY_CLOSED, logIdent + " stream is not writable")); + return FutureUtil.failedFuture(new StreamClientException(ErrorCode.STREAM_ALREADY_CLOSED, logIdent + "stream is not writable")); } long offset = nextOffset.getAndAdd(recordBatch.count()); StreamRecordBatch streamRecordBatch = new StreamRecordBatch(streamId, epoch, offset, recordBatch.count(), Unpooled.wrappedBuffer(recordBatch.rawPayload())); @@ -208,9 +209,9 @@ private CompletableFuture append0(AppendContext context, RecordBat status.markFenced(); Throwable cause = FutureUtil.cause(ex); if (cause instanceof StreamClientException && ((StreamClientException) cause).getCode() == ErrorCode.EXPIRED_STREAM_EPOCH) { - LOGGER.info("{} stream append, stream is fenced", logIdent); + logger.info("stream append, stream is fenced"); } else { - LOGGER.warn("{} stream append fail", logIdent, cause); + logger.warn("stream append fail", cause); } }); } @@ -224,7 +225,7 @@ public CompletableFuture fetch(FetchContext context, TimerUtil timerUtil = new TimerUtil(); readLock.lock(); try { - CompletableFuture cf = exec(() -> fetch0(context, startOffset, endOffset, maxBytes), LOGGER, "fetch"); + CompletableFuture cf = exec(() -> fetch0(context, startOffset, endOffset, maxBytes), logger, "fetch"); CompletableFuture retCf = cf.thenCompose(rs -> { if (networkOutboundLimiter != null) { long totalSize = 0L; @@ -254,16 +255,16 @@ public CompletableFuture fetch(FetchContext context, if (ex != null) { Throwable cause = FutureUtil.cause(ex); if (!(cause instanceof FastReadFailFastException)) { - LOGGER.error("{} stream fetch [{}, {}) {} fail", logIdent, startOffset, endOffset, maxBytes, ex); + logger.error("stream fetch [{}, {}) {} fail", startOffset, endOffset, maxBytes, ex); } } StreamOperationStats.getInstance().fetchStreamLatency.record(timerUtil.elapsedAs(TimeUnit.NANOSECONDS)); - if (LOGGER.isDebugEnabled()) { + if (logger.isDebugEnabled()) { long totalSize = 0L; for (RecordBatch recordBatch : rs.recordBatchList()) { totalSize += recordBatch.rawPayload().remaining(); } - LOGGER.debug("fetch data, stream={}, {}-{}, total bytes: {}, cost={}ms", streamId, + logger.debug("fetch data, stream={}, {}-{}, total bytes: {}, cost={}ms", streamId, startOffset, endOffset, totalSize, timerUtil.elapsedAs(TimeUnit.MILLISECONDS)); } pendingFetches.remove(retCf); @@ -279,10 +280,10 @@ public CompletableFuture fetch(FetchContext context, private CompletableFuture fetch0(FetchContext context, long startOffset, long endOffset, int maxBytes) { if (!status.isReadable()) { - return FutureUtil.failedFuture(new StreamClientException(ErrorCode.STREAM_ALREADY_CLOSED, logIdent + " stream is already closed")); + return FutureUtil.failedFuture(new StreamClientException(ErrorCode.STREAM_ALREADY_CLOSED, logIdent + "stream is already closed")); } - if (LOGGER.isTraceEnabled()) { - LOGGER.trace("{} stream try fetch, startOffset: {}, endOffset: {}, maxBytes: {}", logIdent, startOffset, endOffset, maxBytes); + if (logger.isTraceEnabled()) { + logger.trace("stream try fetch, startOffset: {}, endOffset: {}, maxBytes: {}", startOffset, endOffset, maxBytes); } long confirmOffset = confirmOffset(); if (startOffset < startOffset() || endOffset > confirmOffset) { @@ -300,8 +301,8 @@ private CompletableFuture fetch0(FetchContext context, long startOf } return storage.read(context, streamId, startOffset, endOffset, maxBytes).thenApply(dataBlock -> { List records = dataBlock.getRecords(); - if (LOGGER.isTraceEnabled()) { - LOGGER.trace("{} stream fetch, startOffset: {}, endOffset: {}, maxBytes: {}, records: {}", logIdent, startOffset, endOffset, maxBytes, records.size()); + if (logger.isTraceEnabled()) { + logger.trace("stream fetch, startOffset: {}, endOffset: {}, maxBytes: {}, records: {}", startOffset, endOffset, maxBytes, records.size()); } return new DefaultFetchResult(records, dataBlock.getCacheAccessType(), context.readOptions().pooledBuf()); }); @@ -321,7 +322,7 @@ public CompletableFuture trim(long newStartOffset) { this.lastPendingTrim = cf; cf.whenComplete((nil, ex) -> StreamOperationStats.getInstance().trimStreamLatency.record(timerUtil.elapsedAs(TimeUnit.NANOSECONDS))); return cf; - }, LOGGER, "trim"); + }, logger, "trim"); } finally { writeLock.unlock(); } @@ -329,7 +330,7 @@ public CompletableFuture trim(long newStartOffset) { private CompletableFuture trim0(long newStartOffset) { if (newStartOffset < this.startOffset) { - LOGGER.warn("{} trim newStartOffset[{}] less than current start offset[{}]", logIdent, newStartOffset, startOffset); + logger.warn("trim newStartOffset[{}] less than current start offset[{}]", newStartOffset, startOffset); return CompletableFuture.completedFuture(null); } this.startOffset = newStartOffset; @@ -339,10 +340,10 @@ private CompletableFuture trim0(long newStartOffset) { awaitPendingFetchesCf.whenComplete((nil, ex) -> propagate(streamManager.trimStream(streamId, epoch, newStartOffset), trimCf)); trimCf.whenComplete((nil, ex) -> { if (ex != null) { - LOGGER.error("{} trim fail", logIdent, ex); + logger.error("trim fail", ex); } else { - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("{} trim to {}", logIdent, newStartOffset); + if (logger.isDebugEnabled()) { + logger.debug("trim to {}", newStartOffset); } } }); @@ -385,14 +386,14 @@ public CompletableFuture close(boolean force) { CompletableFuture awaitPendingRequestsCf = CompletableFuture.allOf(pendingRequests.toArray(new CompletableFuture[0])); CompletableFuture closeCf = new CompletableFuture<>(); - awaitPendingRequestsCf.whenComplete((nil, ex) -> propagate(exec(this::close0, LOGGER, "close"), closeCf)); + awaitPendingRequestsCf.whenComplete((nil, ex) -> propagate(exec(this::close0, logger, "close"), closeCf)); closeCf.whenComplete((nil, ex) -> { if (ex != null) { - LOGGER.error("{} close fail", logIdent, ex); + logger.error("close fail", ex); StreamOperationStats.getInstance().closeStreamStats(false).record(timerUtil.elapsedAs(TimeUnit.NANOSECONDS)); } else { - LOGGER.info("{} closed", logIdent); + logger.info("closed"); StreamOperationStats.getInstance().closeStreamStats(true).record(timerUtil.elapsedAs(TimeUnit.NANOSECONDS)); } NetworkStats.getInstance().removeStreamReadBytesStats(streamId); @@ -419,12 +420,12 @@ public CompletableFuture destroy() { } writeLock.lock(); try { - CompletableFuture destroyCf = close().thenCompose(nil -> exec(this::destroy0, LOGGER, "destroy")); + CompletableFuture destroyCf = close().thenCompose(nil -> exec(this::destroy0, logger, "destroy")); destroyCf.whenComplete((nil, ex) -> { if (ex != null) { - LOGGER.error("{} destroy fail", logIdent, ex); + logger.error("destroy fail", ex); } else { - LOGGER.info("{} destroyed", logIdent); + logger.info("destroyed"); } }); return destroyCf; @@ -450,8 +451,8 @@ private void updateConfirmOffset(long newOffset) { break; } if (confirmOffset.compareAndSet(oldConfirmOffset, newOffset)) { - if (LOGGER.isTraceEnabled()) { - LOGGER.trace("{} stream update confirm offset from {} to {}", logIdent, oldConfirmOffset, newOffset); + if (logger.isTraceEnabled()) { + logger.trace("stream update confirm offset from {} to {}", oldConfirmOffset, newOffset); } break; } diff --git a/s3stream/src/main/java/com/automq/stream/s3/exceptions/ObjectNotExistException.java b/s3stream/src/main/java/com/automq/stream/s3/exceptions/ObjectNotExistException.java index 9380a208e8..21e425777d 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/exceptions/ObjectNotExistException.java +++ b/s3stream/src/main/java/com/automq/stream/s3/exceptions/ObjectNotExistException.java @@ -18,7 +18,8 @@ public ObjectNotExistException(long objectId) { } public ObjectNotExistException(Throwable cause) { - super(cause); + super(cause.getMessage()); + this.addSuppressed(cause); } } diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/AbstractObjectStorage.java b/s3stream/src/main/java/com/automq/stream/s3/operator/AbstractObjectStorage.java index a6a3fd06ca..775a9df1dd 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/AbstractObjectStorage.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/AbstractObjectStorage.java @@ -61,7 +61,7 @@ import io.netty.buffer.Unpooled; import io.netty.util.HashedWheelTimer; import io.netty.util.ReferenceCounted; -import software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException; +import software.amazon.awssdk.http.HttpStatusCode; import software.amazon.awssdk.services.s3.model.S3Exception; @SuppressWarnings("this-escape") @@ -378,14 +378,12 @@ private void write0(WriteOptions options, String path, ByteBuf data, Completable int retryCount = retryOptions.retryCountGetAndAdd(); if (isThrottled(cause, retryCount)) { failedWriteMonitor.record(objectSize); - } - if (isFirstTimeout(cause, retryCount)) { + logger.warn("PutObject for object {} fail, retry count {}, queued and retry later", path, retryCount, cause); + queuedWrite0(retryOptions, path, data, finalCf); + } else { int delay = retryDelay(S3Operation.PUT_OBJECT, retryCount); logger.warn("PutObject for object {} fail, retry count {}, retry in {}ms", path, retryCount, delay, cause); delayedWrite0(retryOptions, path, data, finalCf, delay); - } else { - logger.warn("PutObject for object {} fail, retry count {}, queued and retry later", path, retryCount, cause); - queuedWrite0(retryOptions, path, data, finalCf); } return null; }); @@ -508,14 +506,12 @@ private void uploadPart0(WriteOptions options, String path, String uploadId, int int retryCount = options.retryCountGetAndAdd(); if (isThrottled(cause, retryCount)) { failedWriteMonitor.record(size); - } - if (isFirstTimeout(cause, retryCount)) { + logger.warn("UploadPart for object {}-{} fail, retry count {}, queued and retry later", path, partNumber, retryCount, cause); + queuedUploadPart0(options, path, uploadId, partNumber, data, finalCf); + } else { int delay = retryDelay(S3Operation.UPLOAD_PART, retryCount); logger.warn("UploadPart for object {}-{} fail, retry count {}, retry in {}ms", path, partNumber, retryCount, delay, cause); delayedUploadPart0(options, path, uploadId, partNumber, data, finalCf, delay); - } else { - logger.warn("UploadPart for object {}-{} fail, retry count {}, queued and retry later", path, partNumber, retryCount, cause); - queuedUploadPart0(options, path, uploadId, partNumber, data, finalCf); } return null; }); @@ -845,14 +841,10 @@ static int getMaxObjectStorageConcurrency() { private static boolean isThrottled(Throwable ex, int retryCount) { if (ex instanceof S3Exception) { S3Exception s3Ex = (S3Exception) ex; - return s3Ex.statusCode() == 429 || s3Ex.statusCode() == 503; + return s3Ex.statusCode() == HttpStatusCode.THROTTLING || s3Ex.statusCode() == HttpStatusCode.SERVICE_UNAVAILABLE; } // regard timeout as throttled except for the first try - return ex instanceof ApiCallAttemptTimeoutException && retryCount > 0; - } - - private static boolean isFirstTimeout(Throwable ex, int retryCount) { - return ex instanceof ApiCallAttemptTimeoutException && retryCount == 0; + return ex instanceof TimeoutException && retryCount > 0; } /** diff --git a/s3stream/src/main/java/com/automq/stream/s3/operator/AwsObjectStorage.java b/s3stream/src/main/java/com/automq/stream/s3/operator/AwsObjectStorage.java index f9c38b59f1..5e71ddbaa2 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/operator/AwsObjectStorage.java +++ b/s3stream/src/main/java/com/automq/stream/s3/operator/AwsObjectStorage.java @@ -31,6 +31,7 @@ import java.util.Locale; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -48,6 +49,7 @@ import software.amazon.awssdk.core.async.AsyncRequestBody; import software.amazon.awssdk.core.async.AsyncResponseTransformer; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException; import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.http.HttpStatusCode; import software.amazon.awssdk.http.async.SdkAsyncHttpClient; @@ -324,7 +326,6 @@ Pair toRetryStrategyAndCause(Throwable ex, S3Operation if (cause instanceof S3Exception) { S3Exception s3Ex = (S3Exception) cause; switch (s3Ex.statusCode()) { - case HttpStatusCode.FORBIDDEN: case HttpStatusCode.NOT_FOUND: strategy = RetryStrategy.ABORT; break; @@ -341,6 +342,8 @@ Pair toRetryStrategyAndCause(Throwable ex, S3Operation cause = new ObjectNotExistException(cause); } } + } else if (cause instanceof ApiCallAttemptTimeoutException) { + cause = new TimeoutException(cause.getMessage()); } return Pair.of(strategy, cause); } diff --git a/s3stream/src/main/java/com/automq/stream/utils/AsyncSemaphore.java b/s3stream/src/main/java/com/automq/stream/utils/AsyncSemaphore.java index 26e48e8e6f..93a033c8be 100644 --- a/s3stream/src/main/java/com/automq/stream/utils/AsyncSemaphore.java +++ b/s3stream/src/main/java/com/automq/stream/utils/AsyncSemaphore.java @@ -47,6 +47,7 @@ public synchronized boolean acquire(long requiredPermits, Supplier release(requiredPermits)); } catch (Throwable e) { LOGGER.error("Error in task", e); + release(requiredPermits); } return true; } else { diff --git a/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java b/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java index aada94055a..92695b77b8 100644 --- a/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java +++ b/s3stream/src/test/java/com/automq/stream/s3/S3StorageTest.java @@ -52,6 +52,7 @@ import static com.automq.stream.s3.TestUtils.random; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; @@ -179,6 +180,119 @@ public void testWALCallbackSequencer() { assertEquals(List.of(r1), seq.after(r1)); } + /** + * WALCallbackSequencer - Test after() when a stream's queue becomes empty + */ + @Test + public void testAfterWhenQueueBecomesEmpty() { + S3Storage.WALCallbackSequencer seq = new S3Storage.WALCallbackSequencer(); + long streamId = 700L; + + WalWriteRequest request = new WalWriteRequest(newRecord(streamId, 70L), 700L, new CompletableFuture<>()); + seq.before(request); + + // Process the request, making the queue empty + List result = seq.after(request); + assertEquals(List.of(request), result); + + // Verify that subsequent processing works correctly after the queue became empty + // Add a new request to the same stream + WalWriteRequest newRequest = new WalWriteRequest(newRecord(streamId, 71L), 701L, new CompletableFuture<>()); + seq.before(newRequest); + + // Process the new request + List newResult = seq.after(newRequest); + assertEquals(List.of(newRequest), newResult); + } + + /** + * WALCallbackSequencer + */ + @Test + public void testTryFreeWithEmptyQueue() { + S3Storage.WALCallbackSequencer seq = new S3Storage.WALCallbackSequencer(); + long streamId = 200L; + + // Create a request and process it, emptying the queue + WalWriteRequest request = new WalWriteRequest(newRecord(streamId, 20L), 200L, new CompletableFuture<>()); + seq.before(request); + seq.after(request); + + // Call tryFree to remove the empty queue + seq.tryFree(streamId); + + // If the queue was removed correctly, a new queue will be created and processed normally + WalWriteRequest newRequest = new WalWriteRequest(newRecord(streamId, 21L), 201L, new CompletableFuture<>()); + seq.before(newRequest); + List result = seq.after(newRequest); + assertEquals(List.of(newRequest), result); + } + + /** + * WALCallbackSequencer + */ + @Test + public void testTryFreeWithNonEmptyQueue() { + S3Storage.WALCallbackSequencer seq = new S3Storage.WALCallbackSequencer(); + long streamId = 300L; + + // Add multiple requests + WalWriteRequest r0 = new WalWriteRequest(newRecord(streamId, 30L), 300L, new CompletableFuture<>()); + WalWriteRequest r1 = new WalWriteRequest(newRecord(streamId, 31L), 301L, new CompletableFuture<>()); + + seq.before(r0); + seq.before(r1); + + // Only process the first request, the queue should be non-empty + seq.after(r0); + + // Call tryFree, but the queue should not be removed + seq.tryFree(streamId); + + // The second request should still in the queue + List result = seq.after(r1); + assertEquals(List.of(r1), result); + } + + /** + * WALCallbackSequencer + */ + @Test + public void testBeforeExceptionHandling() { + S3Storage.WALCallbackSequencer seq = new S3Storage.WALCallbackSequencer(); + + // Create a request that will cause the before method to throw an exception, + // For example, let record be null + WalWriteRequest request = new WalWriteRequest(null, 500L, new CompletableFuture<>()); + + seq.before(request); + + // Verify that the future has been completed abnormally + assertTrue(request.cf.isCompletedExceptionally()); + } + + /** + * WALCallbackSequencer + */ + @Test + public void testAfterWithDifferentOffset() { + S3Storage.WALCallbackSequencer seq = new S3Storage.WALCallbackSequencer(); + long streamId = 600L; + + WalWriteRequest r0 = new WalWriteRequest(newRecord(streamId, 60L), 600L, new CompletableFuture<>()); + seq.before(r0); + + // Create a request with the same streamId but a different offset + WalWriteRequest r1 = new WalWriteRequest(newRecord(streamId, 61L), 601L, new CompletableFuture<>()); + + // Process r1, but it is not in the queue, so after should return an empty list + List result = seq.after(r1); + assertEquals(Collections.emptyList(), result); + + // Verify that r1 is marked as persistent + assertTrue(r1.persisted); + } + @Test public void testUploadWALObject_sequence() throws ExecutionException, InterruptedException, TimeoutException { List> objectIdCfList = List.of(new CompletableFuture<>(), new CompletableFuture<>()); diff --git a/s3stream/src/test/java/com/automq/stream/utils/AsyncSemaphoreTest.java b/s3stream/src/test/java/com/automq/stream/utils/AsyncSemaphoreTest.java new file mode 100644 index 0000000000..bdb47532a3 --- /dev/null +++ b/s3stream/src/test/java/com/automq/stream/utils/AsyncSemaphoreTest.java @@ -0,0 +1,220 @@ +/* + * Copyright 2024, AutoMQ HK Limited. + * + * The use of this file is governed by the Business Source License, + * as detailed in the file "/LICENSE.S3Stream" included in this repository. + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +package com.automq.stream.utils; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class AsyncSemaphoreTest { + + private AsyncSemaphore semaphore; + + @BeforeEach + void setup() { + semaphore = new AsyncSemaphore(10); // Initialize with 10 permits + } + + @Test + public void testAcquireWithSufficientPermits() throws ExecutionException, InterruptedException, TimeoutException { + // Test that permits can be acquired when sufficient permits are available + AtomicInteger executedTaskNum = new AtomicInteger(0); + + CompletableFuture task1 = new CompletableFuture<>(); + boolean acquired1 = semaphore.acquire(3, () -> { + executedTaskNum.addAndGet(1); + return task1; + }, Runnable::run); + assertTrue(acquired1); // Task can acquire permits + assertEquals(7, semaphore.permits()); // Permits should be decreased + assertEquals(1, executedTaskNum.get()); // Task should execute + assertFalse(semaphore.requiredRelease()); // No release required + + CompletableFuture task2 = new CompletableFuture<>(); + boolean acquired2 = semaphore.acquire(3, () -> { + executedTaskNum.addAndGet(1); + return task2; + }, Runnable::run); + assertTrue(acquired2); // Task can acquire permits + assertEquals(4, semaphore.permits()); // Permits should be decreased + assertEquals(2, executedTaskNum.get()); // Task should execute + assertFalse(semaphore.requiredRelease()); // No release required + + CompletableFuture task3 = new CompletableFuture<>(); + boolean acquired3 = semaphore.acquire(4, () -> { + executedTaskNum.addAndGet(1); + return task3; + }, Runnable::run); + assertTrue(acquired3); // Task can acquire permits + assertEquals(0, semaphore.permits()); // Permits should be decreased + assertEquals(3, executedTaskNum.get()); // Task should execute + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + // Release permits and ensure they are restored + task1.complete(null); // Release permits from task1 + assertEquals(3, semaphore.permits()); + task2.complete(null); // Release permits from task2 + assertEquals(6, semaphore.permits()); + task3.complete(null); // Release permits from task3 + assertEquals(10, semaphore.permits()); + } + + @Test + public void testAcquireDecreaseToNegativePermits() throws ExecutionException, InterruptedException, TimeoutException { + // Test that permits can decrease to negative values + AtomicInteger executedTaskNum = new AtomicInteger(0); + + CompletableFuture task1 = new CompletableFuture<>(); + boolean acquired1 = semaphore.acquire(15, () -> { + executedTaskNum.addAndGet(1); + return task1; + }, Runnable::run); + assertTrue(acquired1); // Task can acquire permits + assertEquals(-5, semaphore.permits()); // Permits should be negative + assertEquals(1, executedTaskNum.get()); // Task should execute + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + // Test that a second task is queued when permits are negative + CompletableFuture task2 = new CompletableFuture<>(); + boolean acquired2 = semaphore.acquire(5, () -> { + executedTaskNum.addAndGet(1); + return task2; + }, Runnable::run); + assertFalse(acquired2); // Task should be queued + assertEquals(-5, semaphore.permits()); // Permits should remain unchanged + assertEquals(1, executedTaskNum.get()); // Task should not execute yet + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + // Release permits and ensure the queued task executes + task1.complete(null); // Release permits from task1 + assertEquals(5, semaphore.permits()); // Permits should be restored and acquired by task2 + assertEquals(2, executedTaskNum.get()); // Task2 should execute + assertFalse(semaphore.requiredRelease()); // No release required + + task2.complete(null); // Release permits from task2 + assertEquals(10, semaphore.permits()); // Permits should be restored + } + + @Test + public void testReleaseWithoutLeaks() throws ExecutionException, InterruptedException, TimeoutException { + // Test that all releases are correctly accounted for, even if there are exceptions when acquiring + AtomicInteger executedTaskNum = new AtomicInteger(0); + + CompletableFuture task1 = new CompletableFuture<>(); + boolean acquired1 = semaphore.acquire(5, () -> { + executedTaskNum.addAndGet(1); + return task1; + }, Runnable::run); + assertTrue(acquired1); // Task can acquire permits + assertEquals(5, semaphore.permits()); // Permits should be decreased + assertEquals(1, executedTaskNum.get()); // Task should execute + assertFalse(semaphore.requiredRelease()); // No release required + + CompletableFuture task2 = new CompletableFuture<>(); + boolean acquired2 = semaphore.acquire(10, () -> { + executedTaskNum.addAndGet(1); + return task2; + }, Runnable::run); + assertTrue(acquired2); // Task can acquire permits + assertEquals(-5, semaphore.permits()); // Permits should be negative + assertEquals(2, executedTaskNum.get()); // Task should execute + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + RuntimeException task3Exception = new RuntimeException("Task 3 exception"); + CompletableFuture task3 = new CompletableFuture<>(); + boolean acquired3 = semaphore.acquire(1, () -> { + executedTaskNum.addAndGet(1); + throw task3Exception; + }, Runnable::run); + assertFalse(acquired3); // Task should be queued + assertEquals(-5, semaphore.permits()); // Permits should remain unchanged + assertEquals(2, executedTaskNum.get()); // Task should not execute yet + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + // Release permits from task1 + task1.complete(null); + assertEquals(0, semaphore.permits()); // Permits should be restored and not acquired by task3 + assertEquals(2, executedTaskNum.get()); // Task3 should not execute + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + // Release permits from task2 + task2.complete(null); + // Permits should be restored and acquired by task3, but task3 throw an exception, so the permits should be restored + assertEquals(10, semaphore.permits()); + assertEquals(3, executedTaskNum.get()); // Task3 should execute + assertFalse(semaphore.requiredRelease()); // No release required + + // Release permits from task3 + task3.completeExceptionally(task3Exception); + assertEquals(10, semaphore.permits()); // Permits should be restored even if task3 completes exceptionally + } + + @Test + public void testSequentialAcquireOrder() throws ExecutionException, InterruptedException, TimeoutException { + // Test that sequentially issued acquire requests are executed in order after permits become sufficient + AtomicInteger executionOrder = new AtomicInteger(0); + + CompletableFuture task1 = new CompletableFuture<>(); + boolean acquired1 = semaphore.acquire(12, () -> { + executionOrder.compareAndSet(0, 1); + return task1; + }, Runnable::run); + assertTrue(acquired1); // Task 1 can acquire permits + assertEquals(-2, semaphore.permits()); // Permits should be negative + assertEquals(1, executionOrder.get()); // Task 1 should execute + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + CompletableFuture task2 = new CompletableFuture<>(); + boolean acquired2 = semaphore.acquire(6, () -> { + executionOrder.compareAndSet(1, 2); + return task2; + }, Runnable::run); + assertFalse(acquired2); // Task 2 should be queued + assertEquals(-2, semaphore.permits()); // Permits should remain unchanged + assertEquals(1, executionOrder.get()); // Task 2 should not execute yet + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + CompletableFuture task3 = new CompletableFuture<>(); + boolean acquired3 = semaphore.acquire(8, () -> { + executionOrder.compareAndSet(2, 3); + return task3; + }, Runnable::run); + assertFalse(acquired3); // Task 3 should be queued + assertEquals(-2, semaphore.permits()); // Permits should remain unchanged + assertEquals(1, executionOrder.get()); // Task 3 should not execute yet + assertTrue(semaphore.requiredRelease()); // Release required due to non-positive permits + + // Release permits from task1 and ensure task2 executes first + task1.complete(null); + assertEquals(4, semaphore.permits()); // Permits should be restored and acquired by task2 + assertEquals(2, executionOrder.get()); // Task 2 should execute first + assertTrue(semaphore.requiredRelease()); // Release required due to non-empty queue + + // Release permits from task2 and ensure task3 executes next + task2.complete(null); + assertEquals(2, semaphore.permits()); // Permits should be restored and acquired by task3 + assertEquals(3, executionOrder.get()); // Task 3 should execute next + assertFalse(semaphore.requiredRelease()); // No release required + + // Release permits from task3 + task3.complete(null); + assertEquals(10, semaphore.permits()); // Permits should be restored + } +} \ No newline at end of file diff --git a/tools/src/main/java/org/apache/kafka/tools/automq/PerfCommand.java b/tools/src/main/java/org/apache/kafka/tools/automq/PerfCommand.java index 4be64f5aa6..049516bc2f 100644 --- a/tools/src/main/java/org/apache/kafka/tools/automq/PerfCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/automq/PerfCommand.java @@ -93,17 +93,28 @@ private PerfCommand(PerfConfig config) { private void run() { LOGGER.info("Starting perf test with config: {}", jsonStringify(config)); TimerUtil timer = new TimerUtil(); - - if (config.reset) { - LOGGER.info("Deleting all test topics..."); - int deleted = topicService.deleteTopics(); - LOGGER.info("Deleted all test topics ({} in total), took {} ms", deleted, timer.elapsedAndResetAs(TimeUnit.MILLISECONDS)); + + // Modified topic initialization logic + List topics; + if (config.catchupTopicPrefix != null && !config.catchupTopicPrefix.isEmpty()) { + LOGGER.info("Listing existing topics with prefix {}...", config.catchupTopicPrefix); + topics = topicService.listTopicsByPrefix(config.catchupTopicPrefix); + LOGGER.info("Found {} existing topics for catchup test", topics.size()); + + if (topics.isEmpty()) { + throw new RuntimeException("No topics found with prefix: " + config.catchupTopicPrefix); + } + } else { + if (config.reset) { + LOGGER.info("Deleting all test topics..."); + int deleted = topicService.deleteTopics(); + LOGGER.info("Deleted {} test topics, took {} ms", deleted, timer.elapsedAndResetAs(TimeUnit.MILLISECONDS)); + } + LOGGER.info("Creating {} new topics...", config.topics); + topics = topicService.createTopics(config.topicsConfig()); + LOGGER.info("Created {} topics, took {} ms", topics.size(), timer.elapsedAndResetAs(TimeUnit.MILLISECONDS)); } - LOGGER.info("Creating topics..."); - List topics = topicService.createTopics(config.topicsConfig()); - LOGGER.info("Created {} topics, took {} ms", topics.size(), timer.elapsedAndResetAs(TimeUnit.MILLISECONDS)); - LOGGER.info("Creating consumers..."); int consumers = consumerService.createConsumers(topics, config.consumersConfig()); consumerService.start(this::messageReceived, config.maxConsumeRecordRate); @@ -117,8 +128,15 @@ private void run() { waitTopicsReady(consumerService.consumerCount() > 0); LOGGER.info("Topics are ready, took {} ms", timer.elapsedAndResetAs(TimeUnit.MILLISECONDS)); + // Modified producer start logic Function> payloads = payloads(config, topics); - producerService.start(payloads, config.sendRate); + if (config.catchupTopicPrefix != null) { + LOGGER.info("Starting catchup test with existing topics"); + producerService.start(payloads, config.sendRate); + } else { + LOGGER.info("Starting normal test with new topics"); + producerService.start(payloads, config.sendRate); + } preparing = false; @@ -132,7 +150,7 @@ private void run() { } Result result; - if (config.backlogDurationSeconds > 0) { + if (config.catchupTopicPrefix == null && config.backlogDurationSeconds > 0) { LOGGER.info("Pausing consumers for {} seconds to build up backlog...", config.backlogDurationSeconds); consumerService.pause(); long backlogStart = System.currentTimeMillis(); diff --git a/tools/src/main/java/org/apache/kafka/tools/automq/perf/PerfConfig.java b/tools/src/main/java/org/apache/kafka/tools/automq/perf/PerfConfig.java index 9f7db65d0e..a013098abf 100644 --- a/tools/src/main/java/org/apache/kafka/tools/automq/perf/PerfConfig.java +++ b/tools/src/main/java/org/apache/kafka/tools/automq/perf/PerfConfig.java @@ -65,6 +65,7 @@ public class PerfConfig { public final int reportingIntervalSeconds; public final String valueSchema; public final String valuesFile; + public final String catchupTopicPrefix; public PerfConfig(String[] args) { ArgumentParser parser = parser(); @@ -105,7 +106,16 @@ public PerfConfig(String[] args) { reportingIntervalSeconds = ns.getInt("reportingIntervalSeconds"); valueSchema = ns.getString("valueSchema"); valuesFile = ns.get("valuesFile"); + catchupTopicPrefix = ns.getString("catchupTopicPrefix"); + if (catchupTopicPrefix != null && !catchupTopicPrefix.isEmpty()) { + if (reset) { + throw new IllegalArgumentException( + "Cannot use --reset with --catchup-topic-prefix" + ); + } + } + if (backlogDurationSeconds < groupsPerTopic * groupStartDelaySeconds) { throw new IllegalArgumentException(String.format("BACKLOG_DURATION_SECONDS(%d) should not be less than GROUPS_PER_TOPIC(%d) * GROUP_START_DELAY_SECONDS(%d)", backlogDurationSeconds, groupsPerTopic, groupStartDelaySeconds)); @@ -260,6 +270,12 @@ public static ArgumentParser parser() { .dest("valuesFile") .metavar("VALUES_FILE") .help("The avro value file. Example file content {\"f1\": \"value1\"}"); + parser.addArgument("--catchup-topic-prefix") + .type(String.class) + .dest("catchupTopicPrefix") + .metavar("CATCHUP_TOPIC_PREFIX") + .help("Prefix of existing topics to reuse for catchup testing"); + return parser; } diff --git a/tools/src/main/java/org/apache/kafka/tools/automq/perf/TopicService.java b/tools/src/main/java/org/apache/kafka/tools/automq/perf/TopicService.java index f67626cfdb..46c6c8c169 100644 --- a/tools/src/main/java/org/apache/kafka/tools/automq/perf/TopicService.java +++ b/tools/src/main/java/org/apache/kafka/tools/automq/perf/TopicService.java @@ -33,6 +33,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.kafka.clients.admin.TopicDescription; + public class TopicService implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(TopicService.class); @@ -78,7 +80,30 @@ public List createTopics(TopicsConfig config) { .map(name -> new Topic(name, config.partitionsPerTopic)) .collect(Collectors.toList()); } - + /** + * List all performance test topics. + */ + public List listTopicsByPrefix(String prefix) { + try { + // Automatically add the unified prefix for performance test topics + String fullPrefix = COMMON_TOPIC_PREFIX + prefix; + + List topicNames = admin.listTopics().names().get() + .stream() + // Fix filter condition: use full prefix matching + .filter(name -> name.startsWith(fullPrefix)) + .collect(Collectors.toList()); + + // Fix deprecated all() method invocation + Map descriptions = admin.describeTopics(topicNames).allTopicNames().get(); + + return descriptions.values().stream() + .map(desc -> new Topic(desc.name(), desc.partitions().size())) + .collect(Collectors.toList()); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Failed to list topics with prefix: " + prefix, e); + } + } /** * Delete all historical performance test topics. */