From 6c7128af1761f2341a4bfcca6cfa15ca79db6734 Mon Sep 17 00:00:00 2001 From: Benjamin2037 Date: Tue, 7 Jun 2022 10:09:40 +0800 Subject: [PATCH 1/8] Design doc for add index acceleration(#35983) --- .../2022-06-06-Adding-Index-Acceleration.md | 99 ++++++++++++++++++ docs/design/imgs/addIndexLit-1.png | Bin 0 -> 80762 bytes docs/design/imgs/addIndexLit-2.png | Bin 0 -> 79367 bytes 3 files changed, 99 insertions(+) create mode 100644 docs/design/2022-06-06-Adding-Index-Acceleration.md create mode 100644 docs/design/imgs/addIndexLit-1.png create mode 100644 docs/design/imgs/addIndexLit-2.png diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md new file mode 100644 index 0000000000000..74ad403424ed5 --- /dev/null +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -0,0 +1,99 @@ +# TiDB Adding Index Acceleration +- Author(s): [Bear.C](https://github.com/Benjamin2037) +- Last updated: 2022-07-18 +## Abstract +This document introduces a new way to create index in TiDB in a more efficient way. The existing proof-of-concept benchmark shows that it gives 4+ times performance improvement compared with the current solution. The current online create index implementation will be introduced as background knowledge. + +## Background +The create index in TiDB is in an online fashion, users could add one index for a table and meanwhile DML transactions on the same table could be executed concurrently, which introduces a good experience to the user. To archive that, the process of creating an index is divided into 5 states internally: + +- None - the initial state +- DeleteOnly - the index schema object is visible internally for the transactions, but index record can only be deleted. +- WriteOnly - the index schema object is visible internally for the transactions, and index record can be added/removed/updated. +- WriteReorganization - the index schema object is visible internally for the transactions, and index record can be added/removed/updated and the backfill workers are scheduled in this state to make sure index record will be 'backfilled' all for existing rows. +- Public - the final state in which all work is done and the index is visible to the user. + +The backfill process scans all records from the table and generates an index key/value pair for each record, then fills the index data back into TiKV to form new index data. We found that backfilling data for index is a time-consuming task and it will be longer with user table data expanding sharply. + +There are some measures that have been done to enhance the performance of creating index statements. For example, 1) the parallel execution backfill tasks and 2) adjustable batch size for every backfill batch. Below figure 1 is used to show the current backfill implementation. + +![figure 1](./imgs/addIndexLit-1.png) + +From figure 1, the backfill task is firstly split into serials sub tasks, each sub tasks include one batch size record. Then, multi-workers are started up to execute backfill sub tasks parallely. +Each worker is assigned one sub task and scans the specific record from the table. +Then generate index data for this batch record and write them back into TiKV with an optimistic write transaction. + +## Goals +- Archives a better performance for creating the index in TiDB. +- Reduce the impact on DML transactions for foreground traffic when creating the index +## Non-Goal +- Make backfill work distributed in TiDB instances, which could be treated as a future enhancement based on this proposal. + +## Proposal +The current backfill process could be inefficient for mainly two reasons: +1. The transactions need a full two-phase commit process and be transferred into a bunch of ‘Put’ operations for the storage(TiKV), which lead to extra overhead when we are trying to build a lot of index record. +2. The index data is written to TiKV in a ‘batch write’ transaction, it may conflict with the foreground traffic transaction and have to retry. + +To improve the performance of backfill process, we would like to change it by: +1. One full copy of index data that uses the start of backfill processing as a snapshot and ingest the data to TiKV. +2. Delta part of index that user changed data during first step preparation a full copy. + +The Online DDL supporting the above two preparing steps could be done async way. The new solution will also take use of this important fact. + + +## Rationale +In detail, instead of ‘batch write’ transaction, we could adopt the current Lightning to do the backfilling work. Lightning is a tool that is able to take rows as input and generate the SST files, ingest them to the storage(TiKV) directly. The figure 2 shows the changes of the new backfiller. + +![figure 2](./imgs/addIndexLit-2.png) + +1. The first step is the similar as the current way, the worker is assigned to a sub task(range of data) and scans snapshot data with a unified timestamp obtained at ‘WriteReorganization’ state. +2. Generated this batch of index data, instead of committing a transaction for batch index data. A backfilling writer that is embedded with the Lightning engine built for each worker separately, it writes these index data to its local buffer and flush to the local storage as SST files. +3. The index data is sorted when compacting multiple SST files into one, each sorted files will be ingested into TiKV by the existing interface provided. +4. TiKV will finally combine the full copy of index data that was ingested with delta index part that was updated by user DML transaction during full copy preparation period. As the index at this time is almost completed its build process, so TiKV work in this step is not much. + +Since Lightning writes index data firstly into local storage instead of committing backfill transactions to TiKV. This removes conflict with user transactions and there is no transaction commit time for backfill also. The main path bottleneck of adding indexes is removed. + +In addition to performance enhancement, the new solution could further lower the impact that DDL caused on DML. This point will be outstanding in the future implementation plan. + +## Uniqueness check +For unique index / primary key, we should be able to detect if duplicate key is generated and reports error if so. In order to archive that, we could reuse the `DuplicateDetect` interface provided by the Lightning engine: + +```golang +service ImportSST { + ... + // Open a write stream to generate sst files + rpc Write(stream WriteRequest) returns (WriteResponse) {} + rpc RawWrite(stream RawWriteRequest) returns (RawWriteResponse) {} + ... + + // Collect duplicate data from TiKV. + rpc DuplicateDetect(DuplicateDetectRequest) returns (stream DuplicateDetectResponse) {} +} +``` +Duplicate key will be reported when two unique index records with the same key but different value are encountered. In this case, we could report the error and cancel this DDL job. + +### Checkpoints +In current implementation, the progress of creating an index is persisted in reorgCtx, it is updated when each batch of the index is backfilled. For new backfiller in this proposal, we could mark the progress in a similar way when a SST file generated from a batch of index is ingested successfully. + +## Temporary files +The temporarily generated SST files are stored in local storage, the path of storage and maximum disk usage size should be configurable, but it’s not a good idea to modify them at runtime. If size of temporary files closes to allowed maximum size, the largest existing files are ingested to TiKV and cleaned. + +Cleanup of the temporary files should be managed for the following cases: +- When the job is done or rollbacked, they are cleaned by the backfiller worker. +- Each TiDB instance should be able to check the inactive SST files and clean them, in case ownership of the job execution changes because of TiDB instance crashed. + +## Limitations and future works +Compared with the current one, the CPU and memory consumption of new backfiller is not controlled smoothly when doing the ‘sort and ingest’. + +## Compatibility +As an optimization, this proposal does not lead to any semantic change so theoretically there will be no compatibility issue. + +### Enable/Disable +1. Use tidb_ddl_enable_fast_reorg to enable and disable new solution, default value is off. +```TiDB> set global tidb_ddl_enable_fast_reorg = on/off;``` + +2. Use tidb_ddl_disk_quota to set a threshold of local disk max usage by new backfiller, the range of this variable is from 100 GB to 1 PB, default value is 100 GB. +```TiDB> set global tidb_ddl_disk_quota = 107374182400;``` + +3. Use one config parameter lightning-sort-path to specify the sort path for lightning. The default value is /tmp/tidb, please change it to your own disk path for the new backfiller could work properly. +```lightning-sort-path = “/tmp/tidb”``` diff --git a/docs/design/imgs/addIndexLit-1.png b/docs/design/imgs/addIndexLit-1.png new file mode 100644 index 0000000000000000000000000000000000000000..3f5b7022b26258e844537b64a8dc80ffcdfce257 GIT binary patch literal 80762 zcmcG#1y`2a7xt^7AR(Y2EupkDC=CK4B3%N~ozk5f5s;Sd?hvFK1Vp3}L_oT`m5z7b z{?B_p!8v1W$3}dfJJy3v> zFJHSxc}?=Uh_Xxa=9H_4@{#A&*(h&?dF4*SHFTe`_hYR$NG%P%SbW|P4BN=KLc-u1pGl!yfX!y;0-1*3t z>G=PCihF(|2l>?dTIzfBe#lQ>X(FC6{QEdd;LTV6u3;-lEsufx^g-&Ok0NpbQO$e* zzh3IA!0$Wp^jPUd0|9!^2L4Oy7ouqWNbVTcNufW8N0x$$Jk3M@j^fBv(_p^X1ak?M zHg7qZ%7_H>cB?0$q8s0E(4*6Qh4Y8`XY))3&FB`5iCXCTMpzW(e@m-#JY_-`ocZ8pvL_m|ZX zvmc2Z{q>3b*?)%;Z3qaZc*Uh%k4qC${a7MIapx^d;%}|w?Zb34)H@GKepuxqD_W8s zLe1Y%w$_!ubY7vpD8A-;pGG?J1bK>D}X`%lYi2PEO;BPTi?N=kU4F9B%Q9GMjT2 z<8w=HoUo8<7mg_*kSMRT|X^lH5r_Q6UD`1Fro8Nas0KtEjVs!u3*YB#*l zZ!dvtO$b*w>+`9CRYzP2(lc~3)T%tx0;^SnLA>XqR768R98hG2(UXM3H8CqZ5H7rB!E4+qD2FW@MNXL9T+VYC1vzwlilefqOkjWJ91 z<+V-axE1PpvXb4mE42|5&&n8Co0ES_0j4LBbUJQ37HIX}o#&1(=6Z)F-y`3EU$LKE zjQI*P)9IhTjtX~H)k@za->umCFG|>JP*kSpn!lb6mEiaZG4?O!E|a|wtf70%(*1JJ zRD`hO|Kng6!a+8M$Y1Y2pX)DAH7L6Lfg{%g9Q z*8lvw*2Vuim5jtNmP7mPfbOG2-d|$_t*1hwzfS_b4ip!}M@D^Vw|J%I+;@x0ZAmANCR3S;jF#XEQKczucb~)_+tw&??Jr<&9-LLZkn9=yg6sEfNeuoQeW@qmWloH)diYI4?w@Wb8ZZKac9-1Wzh)Piuy_MqG&sC>S3eJc`iOIssHoMW?X8~a^D zik)ritX$XG#ueG_uoWrp`xVBL#iN7QH5V5ujsKS089kf!>1+1U>3zqV5M-=#>d+52|J$R+m z>0!MzW;JyS!{5P)VnYvcSu>*atWna6dPbVoZ@rR|OiVqxd%X_J_)=3Z|2a_cA8OJy z&bQI`X(EycAf`HsS>{`sQe!FUCHjgn}<4u5)$>chpq-MwDOke>^k>Fe=+rw+PBXvaL6Ct zxiwba;ycr(lvLJiB7X^N0-?Y*N@My1ALfzi@qer=}VUyR|V$^6x$-Rkm#D7Eo z&qU0sw{~W|XUt~RhOae0bI6yL{9NXyhmBLpoXh5Jsa!;?y6}Zu=fhS2-@pWCLX7@O z>xfdVfop4R!36cc3zTdSEMD9F$sbDgc0C*;=7L3dXZx+h=$&Re4Td^m+pLk^+mPgK zGCZz_Ha!uvJl|E8j#s|#*NBA%_n0e*5x+-^EHL}FRt;r9fOTS9aTmiF(~pQx0}pf$ znw>7)#h6(Jc9UvDvTfNHAKYHsEZm+?a3&eE&!k&t2mgp7=rqtt8YpJy%3rYDkwPr$ zgIvW{dXxz_w_Sb-tFf$+Q(+34HPc)wY0!s?(CNeob_)6@Q@SQ9bXNzj1<}m2HaH6~ z7PI+X&MgiSlpgwZ|9c+xDxRtH{i-+9)1G|Bf1S-UQzU-H?#?E5#mi(3mpxq$XmloX z*k2@dY$rIAma0A&T-P`5Dd9#oI{phr#lp$yR?eiGj7YD6r!KjFo<6J$k1@ zx}~N4Ugv{P$bvEKC&ZC=#*NA>m8gi*b96H!kB=^W9iyCVvXMHChLh8P+}ZWU`9PAR z^2lL^*`qW@xWM@ospBT!FPYV3?p*rEsR34u5YGJAqm=)dD(yoRm3VH!(11|_`@Z5; z`2y0}eN>m_E&q7e*hfRvcfG%1{%O5V*O`Q7V9qMjB{O?TnrpN1Q^( z-nTPjJ^#|cTY6_PGQn)Js{EzXm%GRnRG8rkPLdc!%S&Gco-W?mZ*!L+E8c3~ph%e8 zVJD;y|Aiy(>_$&3O|U|%$jcv`$zI;^5%ILIrmy6x%bd)FoV7LZ55@{uxUF}1WmH?N zpa15_y|E-A(J+3j@Ypw1_vc<@y)kx#z>PAO0wYB|I}D+TEPEv&sJhkN>COW8Wl#mB=WeQ+Jhr5;WvKLB%UcuZKr7 zJPC~s5yjH#{+48Ec{pSzO}9R7Sa<)4V#JnhUz?G3HLfhNo6M#jeScspM!Ywf zntu*?Xwl^GsY1Jh@*CxfWce$b$NQI;PT0w-O_^ddS>*JdUZGoPDVBnZAq=`+3vHTK zb+qB<(w(j)1oQL6{ysB1c{Me0sj1|R+N=g1QtoiQd8Gttj1Jem^YQM7#VeMlcG(4K z%xL^POAcu(>r&k7j`Y5XB2qPU%Pz)+DLECKFD&yX*s#pa%}p&V#D-Y}TKUPa37FqJ zXCpEkEzw%7mo+k?K}Scw=HTo+JUIAFM@Q$UV-Uy_4C4yoqLEj|#BC>hSUL<&~A+kC=y^ko>cc zl#b&4n2EgdC0%~HH)vUW>xU_;daadQL3Za17a*s@6I#S&#L@{*hnt2+doCWda% z8ii^GQfiH!j1PEyQ<4+(>%=3q88BJ@S zMr-g+D24-fXzQiZRS`D#l!+BAW_xNMx zj#{n+uTIjgBwxHh9T2_}2qs`Y?H%1!7tO%c($caVFZYYnb_>K2KL0p9t&K0<$`9NeoJ-7Al2Hj0kM+zx z%shRQ`1bBl>~j2qBsOLdfnoS@;dXH`)2mmncHNpH49LR+nwy0@geYKnvU768wUpG< zh|mLwiHUFC@X>Rh!50s#n`Xm;CxcH4pU>em{uz=w|EqKLWaji&&G)V@;-`Z=WY|MP zL!T$MA`B99=MGNB>Kz|bcpX#Cob)>%@5i-Bzj$GZ?sK4k@Av1A16bpuCVP|Lqobpp z_5*^Q{&z@^hvJ&Q&Rk5*M5Lt&Z+a@L^r_{n#>^>! z^Nc6=>x~#z#hIa@wkSEFF1Ut-i3uG814C~-yKi(f{<)X1Y#v@*PX`!EN0N1Om-l(6 z_pTqShtJQS%4-Fno!Pf16%z&`zIkb1ttu-seDkep+7upd&z^K)TJD)J<1qJ89KQ3r z{$H}~HGm-aT=w^Of61t>uQQihGy7Ggp3LTBP7~E{&sQacGYboY0w}!VB`7?(mg_mM z?@Y{GKK0l8n3D1U@A;+^zs}`;+{r}Cg$5;AIV`iFpy1|IL)_Kn&eihq-op7Jop45G z=DP{&mXz%5?06@ZH;>z5wW`V5;PYxp>+hGXww~#KT-qqAqH@2q`IOPs)fGN8L-Z7jmWF0; zKA2TeUj7=cnT~!}nR9>j3S)NX?a9X^zu__EgiffDKP+*f;;Q_I5L2+@2WE8FKK_B<<#6(Y-L1(;>=jp8h z5rJW`grp?Wku!Bo&EBCLDfM~>hH$|{BJ0LIRPWQ_w0L1}p-~-g&t6X^f}YP?N$duw zMXx{UPdB=r_q3d2qG6NGfdT1O5L%WC^wQaeAPDPJkuw9F)WNM<@W_!TU#dB z(EWY8@`{R+W$ub{>1kMB5_9ij_+8Uq)?TkCckQ)& znpJ4Ns7)Kius-9j_k7#C`6Q>lB+Km0q6ZKEfMczN(1|AezJ%Yt8EXxb>g!jPrwj?= z!^3OqN1U@l8XWocCE-(@^>p&-n|u4~VZY)Rj}Mwr$>|aIF62vmNU&MT~TyV2??$uNAmc7U?Ae}#J#<*K4ZQW zDqZS{Q_|2t)|1C}{$^>*<>_ai{o$`29c8c^>+9>s16SvAg^++MOmIfE-M5S^R_Jc| zillm9x?(@z`&wx>T)PyjLC?xcaj2Qp_pKdAxEF4A(!uK03PBs9kshp(z}aR6@)+IS zueNI!u=n@(|J!|+tJUDN2VNF?eC%?$q;u&vIXU@#vP{-qLcxO8bj=+~>Ui-0H z+ebSsS3*6p%vs=KGZ)*=WIQ(4@`k%Yh&gNj{?c$=%?R+{B7SRXiui%$bYsHuvN5E! zgXO-3Rjt;;jEF3c^I`_=XI|juv)=%WF>Ed4+8!9 znw)pr%E8{2wKqS^Mfclx#jN>kGP&ZnR9Aj<&j^n@6VM<+C95a`}CTT zkTO9%8f7supi7f%v`WQwv4ELJ6Rh+adq6e`MWCk86~dkPWS`t#tQtEj-CB3C?R`$PKcig zoFq<&|pD|Xu?$P#CWmT2!N{X|h zn%eF60brp1yC)v-B)cP}UNJF`Rqx%tdAt|y-Mc;2aB{KZJ>ZKY^zzLc637PY>*lGh z%fwFGlU=^c>^S6 zR8msU1LsMRSFbQmPEO*C{x(7pRo0iy7`;emSMEq(7`Okt2baI7f&PtPKZ?W z4K?-iWy5++mNyz2tbFpKF01@cp2U3n_U$fu^y{LS=;&xY3)9pdup~uAQ45R5Q&UsN z9zx6gDLvVeQK0T}o14i?M}tSqb1EvL9!Z2J!0w_;Tg(%_U^tp`rT}52AXw(qkRn0u`aXc%?E?fM z2pJHsRA(O$g0}cISxXuB=*|Te*9#o?9sVEf`*rI0##*Tzd5?r1w{)x-AC*~o_*8VX z%M`YSWKwe|ZRW37iMqz{a_;Q)vaQy^#X!p#8W}~z#E84O@xiSiOXv}i340||T&zlv z@!GwAHoLz);cBe#J)tE{-Dcl)Cu`%>C}qW!C9X%D9=}b$OtlGLHzTFIaQqADQA1pdij< z$bSHVS<-Up#wC+gRTTr_GQLzwPmc=XssTL4;>t=CloB2KbpqDqU}@}cooq4er4%J4 zdH7SHo@^_8D&eMQ<~NA)*JgjM10-JyI=#g*BX;F**P_|^;r?BBq4uC6>(xR5zoy5pPdB>;w)b9?3c>+|5Lh{gcd(3^IXQl? za>^=8gBfDfw6x{sBaB=!3-A|2)n#TrblMyf(b1vU`1Ogj+So)7+OPD7VuXNT%g)Q|DYSyQ4MxYf zv?f1yQkD#g^2o`f&Z$e>3$aVNy8ZbO({wUBG&4K5we2Q`T*+l|$@6>IC*(f)da zLqno$L}1qewY6h!lEE~id0|Fy7ZbNa{ZU8`>P5pJ7^ z7wqTebW_qoXE^L>N2@QIq_ni*Qg;m429o^i?CD}+VnAXp9G$6u)@}V<`a{1Ai~pu| znOdfQ(b?JA-bvdGtC*PBO3FidsH@NiR{G~;BqfOtE1*3#zs}acQ4r;g|w(b#Qt*EY~bLHa0TeWo{=h zlU_PiAORG_wC9o5`o_lE`g;AWb{(eK3`8qlr%j^0y}iYyCGo?@Eksbbk_oy+`}kZB z3Jy*Hok&x&%q)>c9l0vrbPg`tkEQjN| zGIAe~qc7#;^6Kiqo0i$Erv*;W&K@oKJ8k{;8(z!){Q2Yj{JfrTRwg}R<&)g@xetGy zmZ+7z11+1zO~x412I@uMzoO6h;Fe$Z@7il01FOBshh2ahqM zA9xDAN&bN;Q%rzYL8~)`c@uJojm^ijtMe7G^{CX;{`rnz;nB80&h{+YsI_wc52vSY z)mBrl-oAZHU8}u`YYr`jo$01VkVE-}ptzLOV3&+67V1E-syDhohd^a>rX|gJ+BE=b zJUuDCwR%~I7A@yXti+%lz_WVOgoPE|7-(qxhK3Y}hKCDEO1>>&TBlfgQ=?j?8cL{D7^B_^ll8_=pyJ$CdMNjVqj zx2)pf;==#C)7mQXdtzd|`YkEwQ$C+(|0P+~2ZD&y3)VpM713l-T@*5@)*&MynI9`N zXgpa?auFam?n@LmHeMG0E9)je4x#HdsxOhhPv=7GlHHHV$zBcXrjCZ4W=xxDEjvnq z@SUhFOjKX^8U#*ow7CvKooo=Ijg10U^=r8=c~k76K{T-yHo2l%Sg2nyF7)>=Lr@!Y z&14(6itEOw8bqW*l`!dbwiO){qgONWNNg9@yzzX$#{(h+v@$?@%0UMryGBj{A2Jlf zvk{T)^S?DvI z9d0c^Vte%XaWK@uSncUqS^t_JGY~By+bF23pValdMu+ZeEr$I)eJ83-XS965CpPNa z!umc#3z*<}(7bfJ%bKBs$e`ZwanB%$aq7YI-Pft7asvh)o; zmbk~|iFNh$B&DN0NLkk6b>X^V?AD|W;wItcjyWYRSU93qGsFUTyv})GZIIFwS;3XJ za^Bv;h-8K5B0t=N7bRfTmKGQ9$ZIaCvtL1)KR;$>22usxkN=i>_XXn7sX^5T8wc$w z=y>rvZO(qGnSRK^5{6g`tJhvIyN)lXj~y6s?bP;5R@(O~LQu<8k?}d)hFlBwTalod zg7$w|0OP*;*-xwk5-KXciq+d;hY`~o5IiCiN(%?)fZT{E;sldaFeX@VNd<)nENe`v z_a7ow+3-w)FHdIwT2wl@u99MWbPH_24^vCuT-~!R>>+jRHD*_cjB^<`r__=R%G4Z` zpRZdrv=Rf2>nd^SGNf97Iq%RY0y%q`3*Us%`; zYZ}y{Kj%lNtqZZ{9OUa_PMmg7?^`k=B1$2lROE4()Z26}Ms(O6*MG)O4VF$sl;B@YD}8WQ;z5Jge{arSmmZa9zqMCpLDzbNS(xrq&1R8s`hBw9cl(LA&+>Z z0Y~}xF=pwgl)JmTq26FtxX>#PkMqwuq{OI%q92Mh+{#0fD|w(_+;~}$Lpn8$OP7eI z&UZFcSQ4TJgVH^_dVaTB{w_Tu@sphJ2^~ogeN5Vo@gX5txw0vIo4i4pAETqcu8$O< z`-v(9n{jY*Li1qlGp1!f{$zh|ui>{4;z^fq;FA#FnJ5RHrnRoS)w#8WFn}%WqNIjGU>l| zc1Fdnej>r7drhA z7KVMp=NiPHnh6FbCc}|J6~tj6Heiu{0l5juYbn)zI|C&3{l~ALHZ%mWI2Af;fjwg5c8G=z5^PT}qDurUgA=4%hv~ms_u(fCoE~fagN`bP!A-{X>X> z)(faIyPc+^a+kgN{4<#}p%iFfLI2gZOE3qz_TO5D8lS_ojraFY3f|>?RNiS0n=T(k z_7|pYBRv`#$9?{Me5d{X-J8n%SECP%CT*uWw%nP&=0M%@=kRc7bd<2*!X-ETSj~h8 z6-q5=e{sn?;^aJNysiel=HqOo6mqYMvRGB;$`@fYyiCQ~@jcGu3@zF2;c7TWsx7RH z(l>ZEjg9Y14FnRY!#gG?CqFJ1T#Cu1M~YrT6~$L@nQLrG%Dv003NK{1chL--mmAa` zl{@<9lA5|%O5{(O*ROBL{cn8n9+ZmfT3CJx`cyM6^tRFcS(uf>0NOL6oVs+Q0xqa1jb0LYt4`D(Q7ma@C=|wLzmV_oMRFhWZh<;4t<$UCi|{k zR9$b#RIusaa7KlWPB82|WR?6?jqr1sg`E356jjd(uF*zexk4y`MnrV-A0CYI`K+*$ za}o(5g;%QwL=n%%$2W2+W+)t)g!+4<1=K%2`geUr>B4)%8F{3*SmrBr^KBt7)3M$= zynFL!ohqtOx2BuG_TOY>VJiR5|E~QZP~`t*ir|sYP*MIg+)(JWe2oY5

uys> zQ`4!vl&chffa&ib>)81VvxNvgi`FPI*)0E)y-1ygxr^_f94qoss@vA?4|YCix8ECb zs7LjPbI>8)C-`^I|1aSsgg7FSOz{-L9{r09fpW14VI-UQtM!LfZ2W)51T=R|IF%xk zz9i4%8_UViJnA?cCq#C1VNm5R{X~6}<@ikN^2HXHV=em4C!&cyRzx?&PEEPxq@SUg zyF*u7u(QF{C;&ZJ+~`Lo*S7YPK-b}+7Y(lP9$$@~WS|%o4Y#|WO^j<#Ns2WjQOP~K zkxg{hCtTR%k}|?`v*tDL9-5=W1A=JDe55yRD?RpXeE!dF(jB^!{M;m;jAx5zjksAr z?UZe(5r#fsAsF~R8(fqp|8a?E%NX)CZk(kkYBxb%fQ~g!SG8CQRy|7A6dBb+&1u|{ zVX;40H+N5-T<-rWxW|Zesoe26486hM011#H2t#%5Mt=JEHre=W&Mme;b zzkXPCjg{re8dzt2OCKfL%Kc!gJFbk`^Gd}s7XHaE$;QXdS_X&W!CDY{RAJY<@z@U? z5kcdr@4GA|6r|~wZU2niTkyZTk2mD|pNMXLaC#mg^KARzfENpzrdRa?I}Tp{hyd$U z4!2Oj&Q{~lQ5;mWO^(~<`|Wa?jms=4)k2&bjJ?@3^jTBM3aX#?mp2q8$s`6DyhM4g!=cHJ6G`IaIe!#a zlDS9kvqdLLW*))Dz@qs!K(>Ydd%5CpEqXLw?&mr^B7qAh!3OjC-CLF4_g=52E#frQ^bpKrXg&~H3Gx{SM#T~L)q*irQSlA+|9+!tNdDW3IL#3*)( z_{Pf#FB*vTLu^j&9R3X7*~vh^3)4Ii6;IaRyg9Uehla(03HkTvJBTvknd7%4};%jU)4c1x+iUUP+8-C5*6;5pR(u2>q*}lWw3?ARoaH*19ZC*T)*Dplr=V zvzgo3c2V_BypxLe;_=nrKl(->t{@+6-|`o$al!1c3c}GII-Wk(PdSlCa^YCQ;@v_$H= zH*dtWIowx4j}tdt`w>{d=zz1XTt&3Xmrchg-J=p%!;ow`$}?hsYoZh%S@1DQKXW3- zzLJ>18j1WN*YDAH#)mwQDV^4RM0awI$?1TSjKY_M?SI_9O?i6Td6?!aLoq$4?G?x4 zJoh`7GjX1^9%<9J@7Ee92BI;3lJ4i>pCr94vXU+$Ti%v>irYDH?s7vu(O|ykF*ANn z(fRttH7#EeyGUiLG;5;~>6nX9l9`>u1=7Yg*4M}sI8E}#5>Gbc$EaevQGHC}Kn!6| zy4`U?Es!=x0%8X%-7jCqYnf)!3zE@Zi1?D%h)}v1e-NHTA-xcoGf^BM|Cb4gsD13P zip+;xHf%op(ne8X{~V8B9(6x9Ha^8xnXUa~$G>&o_qP+zNKw`r z#^?Y?gsE!0y36IW1YM<Ln=X$R;wrSBfFmdji>={xTg6@GPABMUn=9j z3{$QB)Fy2f_Uc7r&^TFhuKhEUPV~^GP!g0>Ih@!f@wtp+!-gQeFa9VZugYbZSQNUX zF={H9RF|&mMr18YTdNmDjF*z=dsKLTH|#PJJ3rjz5Lv&G|@IEqd+;1s-tC=qhB7#74UA{ z!>N0I5zFtKE5?Pt(cSQv-7M+`=e`QLx?UEOPOIg+w@;|*wfHcDW$=yi)r6*vnAR5N ztQ%xjTpHTW_3=6q7`OlPG3R*J&HgVT2i@#uFJ_Wk3Og~b_ixJ>F!nbL5LKuT9J0J$ zIx}lpBzr-3W2urX#z?7eF^BOF7BQLWkm>5shYI@WrGnn?oDm7x91UN37cEm3-jZM! zwWUZgDa&%+wN}mF8a(;@x}B6QKE_&KHluqnWY;sP!SVN5mOCzUZg~4*)oWvd_bZak z-b`ON&_sXoJ3f7YF|^`ul*1Ussj{BJw?l#XgWYs?pNNO4{PzpQlm-w}axGvCw8iil zOvs$pt)W+Ao{O1&>xnuchnltgyq`r`vJ!=8ljpAfK?H7|nK|7 zRn>+1o}WKQwmGzNjai!%wn%@{sJ+f_0)o*(4fImON-0!MoPxg@o1^W9nbC=w|~ z#VboiBinJlXHFc2;bhN~olD1PNbOHCHk@9zgJ(}8S836xa;I;KZ#um-Fl<$Yv&+fP z(Y`l&IYIvZm`dqh-i63|%8TsP`jP-0vE2o!CsSWaMc53}N&k`RANFK#ED`-0%&FMO ze$koklS!l|gMn>Pi@MZ%BZeV?(PBBCoxvtkP&2r^P(ACDoZjo^z&df*CPSBe0YMR( zqXqKTU9X`z-UL6hxfT@Vn)RFSOxy(=9~^RbhmN6($&OYnW5+wpypeKz;(Xm$vi{*u*^i zSZ-5%o>sn8!qH3lOpNG9^r_=1w7Y?`CN=-9B~MvLKEN24#CXl=(2z-SfHwP`;f@x! znwmZ>>wT*PlU#fH?*-Zgj}h!q^A2*yLJX&hdkHw`>7;Y%HRZk1?_{X*ThX3SRa4;n z_-Qn|M~ZKpZ%CWy!|9_aGK|HC*6;r{k_qpO)bY4hVkrB|q$I6XyxGI9lJYH=kAgBL zgiC*1v9!nTXAivn7*(kFG1pF}ATTS>5mn%u$*)gH$@BF%&+Cvuzw+xirJxAaBZhN2 zP6aADalJPHvN145Rp2|Ipf;gGroloPQo@%ucll6XsJ>tr;KO}05ZiD37Ne4Vo1UTy zzBKKw#h?(lnV;{>{u(W<2_sK2hAqK&8HHM>`oJ8eK75nM)V%nAwxaZneiJ#R4%K#__b(r`8FIX= z^ji^j`b)Z5GOYH5Q($L5IEG-l@Yn4%)ZpCscqLO&m5c(Vyy*f1tl_nMGgRx>W=t$U z)Ok9di`2KR^BUJriX^5CNinoRp$>xw=qlqMn}D99Enf-@U7#(fF!K0--;Ryw>zV_U#S$<$0`^LQ6g5OeHZKZBl7Kb=^Mw60 z>1|~aHA|E(+e^42z#A~xpp{Dv0X&O$r}s#M5|@&JA+WbsnhaawCvRddbj#$RF&305 z|N6E6xQEfa2t9r#OhdP~tzf7L-2^hi8L9aSWCddrvh6OI@SIu8Y>w_>ZV;-g3Gz2S z$<+JuUuXJqWj`kc>cdeFWx~u+gLG0GDL=m}`_dE&A=8zJnAaFX%9=vH>8R*W>0jRQ z#%n-9ZJBv6>yYLnd>b=2{$z!9?D%TC(!NIklVJ}-ON@`=mznaDwBcbr`iTFChwh|~ z-%YoFCc)j-lyVTHJpXpiUWoX4s2-p6AN&45GU0(U&dEA`OABl36+kcY8XJ?5*&Q_N z^#iK zlHRW|B9c0NH@#k5jGK~KTSTS3DOx54_38*yTS%bb$3|Flb&1T%+7{JxF~;0sK)d;~ zN0YRr{j+o!MQN6!t}8EZuly%Vd9z|)W-*u1wl7s$Yx?MiXxgFMXm8>M(k%u* zyqA!{X~FS@e`0`7H1vn1PfJQ_DCv|%o^J5^eimj^vebT_yU;ApdY3*Kya{g zaozkf*B%gXN35x-3F+Ix2+p`anRs}^u8IQ-C2gQ{SWdVPleO`ggan$uo@Uv9lrJQp zkqyS&XdC@zQi7=0hr@o=W*SqzH{de`%2xE71CkuGR%bKhOy<^6MRA$ z|HjVvSn`w>kD;@=(xVMjIcuE#{cTE+o_kI)*K1^JFmGh+;zX++e%ZBw8w!Hz1>|)m0+mn5tCeZVt_t z4P&DXMJ!)S>Q|k9Ig_00Mx{|$p7z#=gebs>9>4);=K)c$vic(n=~yeQ{2Uz}o}84X z#NBokh8Ots>C;W$*8I6WP9F~Fj9Qe`89&$QR9pV*c5->tGEnOE_i=rF{fh?Pc?|ZX zK4|=dV}66kl3!Rje|feO&WFsL0Ap#K-n}Z5k&)rz;+j1=%gPn5)nqx`nF$}aG=~14 zs%5Q)wB>$5Lb|~9HcUr+X5-qEiepD#B|%jGHgTQDUroAtdJLeA*?6|y(0(XyHBo&B z+VPH~S^+S=f;+&x&wl+UL#}kfjrS}1P7V%(KYu=l_8)WwA6;;KOV(xk4iwMHS<97s z7=>_H=K>ss(yElF|2p7x$O-HOuzkdw#-FXG_@I%i-~ac-Ls>;-4~FdM?QAwYWRu=k z7l3WN9hLGx^wSr0vP;tQf9ff@Dsn_S=;E3mUW}%ZyU7T(8We?v0t^`m< zXe7VuT*%1CpxQTq7x*&H8(LLW1&HG7PjCFX41?<8aMQ$ZEZxi{##QR_xLI<4ei#PQ zoG?Ct#{LuVRsbd_VLr8g3`2=s7Xbu~0ho}~x=}<%=1yNz_--hq>S_Pk{6AgK^WG%`OmB0})hWu<+0eAA?Z>APdiixgcI6BS zLZO!&;eK-A^^d9|7x=E_JkNQK@*xLw%Ekc^B1M^I$xjwNG@29s93$iAHY8>QhGF0)Ujqyeq`Wx{iTeI z(aHW2j8?JF-V_xUx-2J{@W2F>HYyP8#9H)bcTf;Us^_6PGG^V_2tF;|hS@M|3LqDf zFnNFpoWDP+xrN0eA3K;*wg=zmeZ0r7zjz}r5LTLQsk zvpL2NS3-R+HUh*BOelZ{yse^wAg!HY4}#x(86FuC2g8C@fGNF1U)oQ{6z{7e?-f9j zGODWX1MQM)25gYf@hr;Ix4*xN;Rt?B19ljXkPuL-%z}bIlQhp*0KP7dyszS_tDk~^ zfcYUp9l{!YZN!!t<^fz2f#1f<4A8(}&H zN%x&5Wo2b~EiFNocZ#iw_F8)e9khShEFS zH*Sb9@a*pH{H%?;H()6QF8&NVI6BVlxs1WS01HU^6sS97&V99ubM2dm3g)hMtioht`=M5t#+J1a1%UD>LcS*Ez0qfje1lw3@u@YubM zj6}f6v;KETfKq_QaJe@jjf8&2`^r<>`@#`mNkGZk)*R-Y8h$uF0Ov%b@Ob9v_{95w z+gl7~(MD%(Va`Zl^r+jsDl#oiPErybkO`nFDlEr2xMYHvD8Kjj=VoWWhp`!;{0Jxo z@oy-}l~x{p`x%+N4E_Q3cYm{DU}$XYdnotx_tU={hy#?Agdy`;u=~tHMs;=dNr>@8 z{vwpP=`Af3aXo&S^pg(5G7;15z|eL?8~or31SCt?g0iBL5+L5q?Lqe+vnBNr z`L_;uU)m#K)In>-2aR0!@A?R+)BlG^Zi9(eSfBAiAYCfV{K#()7?BbYrPtRdzI%7$ zSE)|;#FlYK(0!Phd90tU+wdIMc%N}wKGn#6J~+ZfD60@%RKgTFZb=LCYsegfgM(jy z@qg&E4}3J>tl*xYGK??JPw)u{_FK^?-~-P<;lgkjI0*aIK`H<)GTli3;e{0gLQ=52 zKpZRq)}LKj8LgBjyN5tvjvKGFUFL4t+1tBFCG-5DEBU2GkFb1^^~gueMa*uT9L|Ta1#M z|LdmS~rR<$bg``4^$&dQmof->3Qp~FujrQx(q0uou&nEXpN z3{6aQDuaJQ6#$1)V*M})Zv?jVIJ@ZnUlu@JMy5~PBBDg2CJdB3EMx?A;JbJ4zWQqqLM+u+c!BAEL08k)j&d)urAXr1i<}4!&E$;;s~llAEq9lr2hkMb8vjTYeo3VwfV5Bl-=T2 z@#4moQ^N^zs%E;$9btnJ%1G%_=tWipQqvUcw2){0_%OT%BkYx+%j)`)1H>(=EJh0E#w5k+00cc9-S<1aEH7~%-f0TyTKi5 zb#^r1zeHH55HH05LIiOSP;^96k`yGaGPMwRU*q2R>(tH3K%GQ(x&zLw*6hK{5{G#W zDDZm64Ia-^F88DDoQ#a?Aff=OQtaQFcHc3Abip8U(Duv~DhB|w5c7c0cMdE$4Z^;| zO}pNLyGp@PCNMFCJOF=$q`&7Pz`Okk>=`ENNWx>+c?G!^(MmAp_R{b8P4`GW7AYD_ zOf@RTrQy&PXaUtqPd&XkBMsNQhoB>&&~rIm%j5MpwkaoRgIFK|JPQayLu2DidJMQ4 z+-dmNuWxX{_$L+w@XAo=1N9A3*J{6ZYilc+&oKzpHKKSDpIAf+mxD1OWH*2|K*A1# z4EOZ(><;Ju4+%XW;j%;SZ?COMUS7c!kAcla;6HYf(DSXCOJ*FQW8Ly1(t8Ws8X6j1 z-QD^i?jV6vO5Q6|!-9tex+tF01RG@MEO`9`JbEc9DRuA+P`MOtgntM{G%sMg5H$k5P`O+Y|eN~$ORnYxk^=XTJe zm1I)i<%1K5__Ecfn)%JmsWv48iOKq zcuo43-qzY$Mp@b2nu)%>h0eeiSxjo5|AT%9c%@PJWJz(3reU12Y9}uv#BB zZu!l;D(vK51T}imNKpIS%djLePj4ruLz)(k`8^&WdM76}p%gntC~~M)l$v<}mkI1# z_^Qti)b++4;G>y#EK{KETn***fx4i=9T+kliIS{(E}Y$u#^lc}zb zi;Gi&f)o_9B$!mW_{^FEE#0>ipg(_bF}pH{dI_hJRwTmto`Dqu8ylP6i-d%P1(r{4 zvI681k<@@|gF+Sn9|VAniDvBZ=m;XUSL9lqGEQV_DoXuyC6T2{S!S2h8 zGpN?zx9hzw)eh}kkZexVVj+ZB4@Eg#8xiOx(D~q&2QnUCyl$}yKfDvrqL-0exhhwa zRSpG5ws6`5NAM5soK1Y%#imK)>3R`T8G+t9=o&PBnxueFqh7Kyi5s^UvdvIya~0mE z%O+jGh>j|-)* zkbbAJv9Vobe*im_@y#cF)+_X#ECR@Bf)N>*_zDOrS){KJbZz@@Y+38r$?mKf5JZOHK4UmHh*~s~Sgxh5O%9A6 z+`HOxybY9E;PZ#iGGnelgKUnK`9?(Cg~CCR_Kzk*R9Cd+lZ(^Q7UTf0L^$~p9DjfV z0~A`L3EbA?FpBDkX-7YU*O zp$#V%fKAn7XNS79YAXeX4<|j~Xblp6wd$i26BDD~QcJc$%VRYwd_NojzQ)Vb4Op*o zXb{0#K}JJ>SfnO}^n|En=%NYR`E1x%e1j7+Js{|N;2dc!++DrFi5bjA4^pb-?@FXN zCQy%62UP$qY&fnK%-Z}NULH{N>rl;}efJg<`R{pGKAcxD9!Tvw+iAg}Hl~k@jt(If zb}2!%8 z-g{A!5n?P-6(HZCOLci?lIrR|Qlx4R61oD!HC!`jbYp2LP6xt>J@EJj90{gYR?EnF zDySx*Q7vF}J7jka&j(pbgyjYt7AcB@GHu|~;rt)2CPRicH!Xp2hE;Ad#dY5->*Vn` zX0vX&_;-*GAz_gU=KGgdhZJ(Y z8z3xQoqM@Qam5bP1~_h<=6xLCP07toC>a^asb~Ug1eI(O)AI|x0Wn+_!!k7x$|@Z4 zP{#gU=>P1*Lxy1YX=&l38yyRKAr1HNzrfL(@6aqmYUO{-Efmurqu^8zoFReiG7oxe zY2g3x^p#;%ZC$u3NSB0kNC|>;3ew%(-QAtiXVcvw-5{WJNH+)@M7p~{N)Ybce)rxV z{y2|ESaYp0#~fqKHy)2B1*X5bt$P5ZC5*0W1d=zb&%IKXJNt?pD_);@tcCN>20R$S_52EXRN&a!XwoK;kl!udVeh03a`q98slbX0Hw?+v^2rsMP+Hgp6P z0F2lJ&K&L~Cpb`{|6d;FN8U?5(Cz`63wYrIWC6G9$#ru8stS8xaBg5;t)x*y02sBR zvJ!SR#6CX+m|QwKBj74UaJqSFUv643%=`3dw*`m4iRRnSh5Gpn{wsWjnvrw=AUfO5 z7VoiR>pRZ$qg?0f^d($mGi{Ms_a8S(Z#O(C7xl5alh$mhr%mKW zpjlJGf zo8e~N6G}|WU&}b13{|@DgQw$D%63m$_ndRx3Ut4llX9F3b@k%X(g-ko-Wrj|nbN_> zh}b_lu>f-b038N{ZYFJRkBYNv8^62qy15^L0uJUDE3;kUxr+;ZPj~JIo=UlcBf3wX z$C3g&Gxy=?7Sza$2rTs2%ACrn-P@rL9FU-AztKf`Fe-cm_@%K4%#a(Nn~`!Lqwr&M zC51NWQeny4F-sdI(=cibxk&L;V|V|ja2l^bSAi45?m(f6i`?+SgMG_fY6$x`rYzb^ zH;Ii8=gQ@Z3E%CKS%|1nK4NRYx!rU62_p?s)a=tg*GTwo_AE=B5=y^bFMTVDO&UT} z*Q~$;gf-3xF6Le9GrEtHhrY=4aptTT6QGEQ$@3cu6UZwkOV@pfoVDq&Vu+`ZdkLV@ zp!+j0SP9LI0E{oI(@>T=vCr8?WTkz1K+r`yEJ)vS1Gm{clAle2Mw#QAz0mHls&iw` zf6(bXBzc8~zE+#C+GRV+8ky{UwG6{VtJ#YPx7-D?%^axnbQSi^d)6-&jnCsQADD-Q zw01)fVhm4XH;zdcX}w1IGu$tu$3{}>GI8$v)pKyFiI|V0&I39!(Fza+}>w#Zx99MH)6iC4oNO@$h=-#Fahl?}?Gj_YPqd^>Y z8VF@UdAo#-$;55KCDZwJg@C|q!h5OV3t?l)4`ibBVq?t zIcjYNdJOO^=d)5N?mQMFRJh>%oZw`^`f=O)D5iNG)-5w@Wk6JqjSHO1Mv}bUz9q;-i18FOoPQp< zM`s0}*>i&XOWBx^cw^rMN|-bp=M~9QngoH3o zwPdmOe1bk@+1)vsttvAGDN~`cPu?|}QV9hty$AkPkV~TXpJM{wL$%;nIi1F&5ihG~ zAtFt3(y7sd89KWR*ii2-;cdYtiK*XWhb80G%9%-fH4evfx4l6;Z8qv6tT~MVF-Bs8 zbj0QK>U86Y+FBK70fiFob-|0d1nhSaaL!z$G0@Gq1aow4g|BuK3@Sq|BV$8Jb?HXV z<-rx*`x-HUtJuaY>FM!MZjwQZpv7Hcu$l~23fFe|9*Y-g`XbQp_@9#O-yI9&Tt81) zR|>6NHlr*OVWE6u9q*9xB@1VlgCu-_EM~!jR6baUBNqB%-kPmCgea2ZFePg?iARYP@nV#IGQv3hp3USva)G=|i2|q3(#3OukBzep zX4$D?3_EXXM~1J(gkB<9JC&%KdWMWVsJx7N70GT6pmt+J%JJrq?#~nVG4FCjbxd5M zQ6f5sx|PtP(l;A@*7_AQ$PT;yfkc+ zsXXjMN(oe#x0y53=o(34;qRTvthor=#0x+uc5y3M$0l%8_O(+&rWql{DQu#`R5_wV zr#TSlt~~#vla^&*l8kkUoOP<4cB5R4tVuv!EHjX}tODQG{8ITUa)bf-|1Sa3_pMIe z&RKtVX*`iKKRVM0^!%L(QQYJGYNYlv&g!HlHk8dWiJ`;sbmOK?)U+LaCLa#UE=-89 zVh+V7A9Yi^T|87`FswdkN#tdr{I%3o_)A69iU9XKr%7=Tx!)}{l@P?|kW7+P)9_jl zAuD)f!cxwO6L#kZ;~Y#PH52C?^KA)>E$OYI(Jf1G(X2ud1rIWvIt{D zWxFPXCMG&5Ld7H_vG@*qDez zDR<$F&1=LwbSRteB;5Wyq$$e&?fu})$?EDSQ8V`F2FXX&Vg0B8k{p#R*4*dO+WbRo z@P%o#*)kL|g3%x~Z~p7UzQ=8*{yD+=t^Q@gzB=sAxDm09U%vX5G)YbjwH@2l7L7LaJQAX3_FG5)RT8C3tyBsOp+Axj4@xn(R?#MlXtl?G~2 zy%pUVm`8|YqoHKoBbwd^y-$FljBS?$C>lJ7#@?%Gaf-ghj)JlmC0Hai*T@)qhXvP& zQlOQ7;NXzob!r%xcKMQFbgTyl>I~Dk*yR^<>hz5CnsgUKzcCNNdE=Ka<~8mlRaM63 zRIk7JNt5bz#E~0>Z)a4GyH_Yv-Q=*2Cqh>Zg^k)F+y1**{LrM5HSvrU`42*ZpU&yh z9;Xqe*qJ4pE>esPn(3-#@mU~E`V3{!B?}PDAP-{mWoPyMrmv!Y_b=G!K;oiIgg?Y5 zaMHoi?*rdFO2&-D0%B_aPErTE;TyA_z6C${U~Px$E_&E4`)u5k>SsLI?&QlJkjxD+ z7810Bqly^lkDNW)71i2$j~W^Enx04*xX460Xp%!p8I#J;q|}jXg=%q~WUq0;FLmPG ziIQ}*CyqLwi4B)9j!Ln)_yK?@IT4OQs}vJ$ns~s5ROn?yXcAc;y_AQR zjVOP~#Afe}YaAd%*+&ce3`NA_{jfcc;`#p5giDg+*>~$=d_Z-2Lf3IY(Gt zPJaelT|#>7{4-A`4oX?b&sHO?UA4A>tcXn#TjfD`TKTm9Vlr2%o}XrG41ei$PeE|e zXKR{NWHc>2gg6}i42Xe~f5&_phabFY0B zYj+v?t{X{g^X%so56HYNVJO8JZ06d#mOg_sV#}kXrRo^mmC<7Pb&96De^N3`^~wy?#B~oP#Z)O$jpg0`GEueXVk_r5e(r55Gd7km;2i+#M6)=)6v1ciF%` zM$C04qf7!sTSXvIY?D^~*CLfkN@W2g9*l zfS0M97otmSO>L;Lm1>O@Xnj3Ny^DEx+pBr%!;ijI^O=2;!u8iB{W?+m?hKP?A52}^*KfA z3-`81;rf!VSFpkn+oP*(^3#fddw3jD9+Q4vk>irI-QqAP@x4lo&v)vFT`D zj|~|an08poWY zhH9h8$yxPj^bA^J4VmqLH*QVmGNl`r2#auJ>bGDmS*no?2}Q_)Al%dLjlsWpgS|(? z@e8GJK=FgU#EGtY=oE`RlO$|+5wY8Mc2gs#PK+0{a*#Hh``zKm&nflxh9c+^5}skl zpvkXkqs&phe@%O2`wcJz29v78NP z%}3B3h8}OrvBuPbeLpgA-%V zs_v{=82dBJ=z!&1YGN91OTdiU&WM)0#Oq{wF|pa_>r_tjG!{Ff>3xF?&t5t}bSv;I zc4Sopj1ktM>-Dl@Luo7u`Lt|wl1uW<-P)33jvG_3;?R4$rfy8;TSJeOhwe)})Z=7XX>Sc3%YCZmL39=cD4cKQ1deeVOmi!8l3uLoKZzuNU(4}U{>0=_3#;E}1e z9Ezc~VZW(oYLq$uOX1s@f5X6J4<@{qk;wah`saMT4%K31R+jaA=Ve@(X353LM(|(2 zk!iJI?j*hgLn))7U$|c6zaPdcPx?HdqZp#;+4M^DTa*L7rh6mFo*1G)pe=oaY^lWX z7eO7>F%an+F42bYU7h~vt9F<5mrggOeil97EG}0R0$R$SJ8FJ{R%`pM|3s-M725A; zuE2?`2&Fua^RKd!YxAkjfu{faW%O6oz5HF{qq^j7Sg{pp%K{|o^Uh_cpCaZ`*lPVQ z0tt2ABd|foo1tTyD}rlrMqH4LrJlF8&=m|WJDZ}$aYe?O^N9q5XJp!%6Au;9$O3%E z^{-$vZu(8RQI@Xd6Z6qWg%|l+BmR+$bw(P`rJUMpqNINNje;s|_n2zg3h0{4_HQ5r zI}v7Gd(<1N!nxb`_H956J}FU8)R=bcwT6})GG?#mt~KD&;{3ae##Zq5Z~xkmVvpEH zfKn10#YTi{W3lDjo5%*P9jugiTuMTeZvFO1$RKrxR+x!3g@?sswYcvYgIO{$?Cw4% zp=TYGcTmXX;vn_UyHZZ6lplVvDnUvS3I6v&d`jB|43_450F?s`LsI3WAsub3dc=IT zM}!9fCyAU(RJwq=;=(ed)uP|GUxqmbCK5-~%x&;zPMCuFiXj z9knf|mTb#D;Eicp}bwm+?)hG5CSX8YP&;I_Kh#m$f7+ zA4VK{5=JFNdwagBjGlIGlcd2hb+NnNTU-h|yXK)}nzNuG*fqa0b{(>O`=Z>GHiwIyeY9RnLtiZ5dOM17*DIT`MXC(M$oX{w z8OK`P;%KQ{j*M_KQFsU|&}!elvp0z4h(=FpA$vPo5tX5v5VCC1W8FXjH+b;jT>p%U znIQJ^WJ*kSds7ClL3X;&rF4!$c}NamsEX@8e!tXhw_9H2rIpK!Mwut$3-e|&(=G|L z+NvK!mE0h}&FDFDtJnLO>_K+0ljg{&GPfbwHuj5EI2~gmY#6`Mmsd@YW#^Hx zDP)_oUrlq{q=^RQ$g+);F;n$>JE3t`eP4!7SVirZHSTA5ehzSNN~wMN8@kkl7Ntb2GJZdUB{O5Vw-=q!wD5uGz&IS z;B2&$n)%@)6Se-vIUhqgos*H07-|&DM|4EfqYo>ij^L5BxVhwf^gL>^3`NGH`5AX+ z(&LXkzL+T#^gj({OeDX4($X`%?^d9e-Z;P79;1`#wZb~qm_zxuYn>`epg-Arm{+XU zt~%_+OgGaQkV6aL7jVo!9lhR6XKp+9;zQuoS;vJ|ejdR08hX2IyW6LJT!H>OjRsjfL5YbK$NfIvVN3!pu9~@&W72u*7~Z%( zs-F`UWtv&Uyz5snEvrUKL(YqYkiK7aU9UnI8+?@KX(E!*5UDsCuc4e+&0BFb@9+5l zsDJRJ%t%4|4^1tvvCaSPEm=vTr+&Tuo@BAZa9GLIQXM-{v)MsHX32FRxh~!C#Y+Mq zxo4J@Tyehpr|_SBSL!-z8?ARVY5Wd~t1PFa zb_JD!PV=t6d_LpoM-0wsveYhxQ8SfW|2vy$L!ri7t#{g0Rm1H^jqmlTX1-w>ej8+B zb~?Mw!Bcx3t8Z-ibu@+sCv^B|eEAqC0hrtlh%Ioi&D?et zLvp2F&mcvJGAjsgE!>jYxfR>pw5$W`BP6-oTQgAioqAzDGyDNQx9dgi-}inz|9580 zak=1o{IF5&)2=T(Bdt;eOXRNg_@qBLa%dVp;`qz-s8ho-UNB3|P7TE(rW^EQyjDWl zB7TV@ok)0ZUsVCe#Ye6sYgz{_jz>mAW0RRC4CrZGy;XR`9bTFP^3i#i_l?z!y0@Nq zYPx>5g8Dj8)%mK8D`I?A!C%7S{U5jk4W=l%y{kkgaU?SOPF`$FA;OcXWlTq+4(U^q zL&(@PAopl^o}|1p+flvLSQPI*>hkt>?ByrDxD11HFQ}X+zM`!k%gvmj4RW+1iqh4~DW}FP$q}=C zF_IWC$M;K@6Z?!CYOBRqGX>IA!XIIbxwomNe0tnM95qvis5mjU%}~azw)el@Be<|v zGgi2$W}CiUb|~6FIp9G_N(-c(IHCI3{-xJdg4Up9j_hlkKu-_tbwWlmm*^<=|rc6XvUo5 z43%ZiU?)xzTY@d2V+!W3MG@8A?X+@|W2e8*A`TJDm|V}-IlRO+r_L|Co-{LsZ!*~JfDwa3E9-j9 z?_x{X#LV#Bp~LT)+d|AzCnNbMbVsh~X;LYAP1ojtPoUPL8>?4N4Lts4fhhWpRlRr> z-B=)P^R%D*P@-(bU#GMe6b@3|fBs2T6Q`{7`E!({E3R zuA<&T7{fbv_$B~WU)Q{Wdqm1wIbEB>M@~}6O3-V<3qjL}dM=)dB&i-Qx$zra94S<# z%?8woaNqtu{BA0}3l!jvsOaIKR(@}$f*P)dF^B$TSIlwbIFd0T&3e0Tk)hetT)3W7 z8OlMKfCQ)(|NI01#t2AhPS4{nTCh6pj=V6V&J8HcZCpu6&e-r(ysj%7krz%yhjOYY z&=IKY6QVg+O%0w&MT@$Je7%C-Cyt!v!=+qyFtNx_YL_Fn_MqZso}ih_K|*dJL#=O) zFZo*Yh}AW}Fc!4X?c7^5Op&zCvF?$ngiBbdEj$N%Ni5i_HZfPU z`1ivTVCit~zXYf3adPTbTE0K3y+w@DW-ViSnmOYgbND7)^vT=2Uo!C)d?b0G{C#|q zzC3!|PFz~KPoJXuI51|=&z?TIW_NkFhi~ZJ&vUkh4ev?yQq-P0DBcObk;P^zg)mpw zZspF3)76Us^gvr%7Etg1z{h9p=xAwX1`qV&06`A0asgrpXuF1W8-2G^YqnH>L-gV4 zJVFN{oO1Z8aPeM(M~VCUlFJ#eZEGuEo3-*V`oX2KKrPJef<9 zKD8M3@c@OUTW@nsh8(yz2;l|5D5NE@iFpi^K4v9ZT5UXzD)Im>Vo3>lU0TrFxqnS- zS~(@ap+%1rGfs_}Tvizf;ZBXFm(oi&F9ZXW1sd<{vp=`vjTRmx2Lt5Mv`vsoq6i{{ zyT>Pzb;HgUc*R!juWAY+31uVa;lkImC9zqYS-2YNmgkbTk7(N|rzDnsqBWWWm;$Ka-&9THdE2k=Hkv zw$ad1PwEGM&)(&7)a+VF$zHqE-|m8sxugZQ@WO3BsKPv7{#UuH%b>`Kd4Bj_ekif; znBjtzh*q-lX^RIH+?OiD?54&_23SpZaJ37l=ekz|-QQp?>-o&jzTZi0z5NN-aJylT zr!`)_dB~x^Yqd#nuXq)AM}bEhL3Jg7FXAwPBazpd#NJla+0gF_9X+$>r%|&mtC3OD8$*&l!KXQjI8gybRo%YAQHfM@ zptuX5ihfV`tEDY>euoQ7Z1_her&e4!~EhNI9KEy_#cy_aQco1+0lmF<&X45)CPM$ z4}CCZfetQDp56ci78wE}qV`uS7!h#DWrN!)96I(9`=j7V?hHsv~Pw?S@=jpW9wXCH_^i=wXdEljV zCxvb2RT;$oepiy=k1^>%p=NsqX{(ZcVJxtI|T z5fYQ)ncTo@5D1jz1@DoC4*Oz0b%Z}IWy}qgduHqG*D+rEm+?_h_(;!dLfsX(?r+mp zb2k)E^M(_(a}^J*k;$ixC`MctGY8rcEpUs^GqPmL7@a6&v4Qk3%vy#2>ERNPT!4*1 z(p(leA}ku5flBNJAYr8eU>z{zNuU9ygN23MVqJnVdCYY`LQ6ztZ;%|1?00p=8)!GI zOR@A~lZ{L8WyGy+E$g{9y}t{705b*%ottO8nga|?lpdfucY4+$BRlnP{8GSnMA zLk^24Qv>orB7b;M6UJ?`sh>DSwiL-Ox;2X7Y{)#?hcfd}FOA515*d6Ura@j)A-E$}v}LfO_w(&U}vv3+S} z;2(EK1@)UsHQ}IpAy@m6x`%7HQMP@!!%>)`3wyG-K#6d$r zosT;a1{^i#D|KWFO>()&1U=pr6`SOQ_uAfv(|SM0CTW+AHvrel?$?)~@hq7837`@K zk>T@}N&i^bwtyY37W>o1d#w{^XDk5Yz9A|2=M|8srNowgh%cjas{b+~Uoz_QSjU;u zxPk>@?0zQqwEtiJ8(0xsmh?5WsZ5?NiWjKx$SAP}q9td5$XD;YDWQ%eY-kP}1?8CoRL8MeB+V zZXhb;H1-ZXY@VxvVR<+ZC{Y7J?hzQR4@Ty(w6v5#P9Lq)!38mQ8R4mO=8HzDof9x9Y@d9Ti`P-C%mv#Zv)swez4z6u=KL39I2ibPhdpoIHSeAz zo*?V!E3tN0!Ka*vR$e{sc)xLxS1GD1Z!$l?!N3Sq#xW(a>#V)f&p;uUaKbXZumF>^ z2K;j2L~5AB0`Qwqhp}De{Q|PRLGS}zdvI95oI^mX0KAhnZ4K{<02W#*QA|82X}W!; z|M^D6#3w-!SDkkx`cuE zHXgvg170%7>{UM*bQv@fXqFUkm?Esl{J~iL_JzsPoY{}lxM*wE#&o<3Ho=Fjbij^S z@!CWOTJC6{de_&k7abK9hIrr3nUbp3y0KstP6~co#eqc^-?HLv8J4k<8K|;{$2K^k zM}F1LNm2L2lJflmFAQda6oE-z(&2xE=9t#o_he+co3Lxk;w{>9Flj1s{uwrFJKETb``9KTioww;fZ3>0SZX&)HyScfYU5qkc z0<)G_Aqs%2&j)r>z}3oYtdS`Kd9&h?z-l48>tCF};}omOw{F(zs9CUZ6t zj0QlP&hWbm~iXV`>;Pd1lrEJjGWm{e7=e(u#vjch2VR?C^hhr?f z&ldlm{|O*`D*-8NfFiFq8L$Er8g=!Q2j`NCcYu~N?410JJu+J@tyEzGIh+$CJL6Z~ zaKWf9R!MnhK3#&=L6s;N?)-WA9JpOcNJs#q1|UYC2bjmsFqbz2g9)G@6vi4aX7+~w9-_kUeSaGt1Q8DRmnshI zL}nGE`ch!357e&#E%hA6xJg0V{gk!j7P zWuR)AQO+N9%N|j0%>9q8|NDn1jDWCt z!p=fM^PwVT$$>SVt2OxtE*o|%^vA2BI;wA3P%EW$)t!wVenK>a`7WTV4)jf6mSRA} zJ*_(1DS!5|!Ta6{^GVRxlZXU&m)vQ@VON(N4RuZrgv5%c3rZwGWx) zxg~9j@fyY?|75=?ZHAtUDUgSd{uIv&{{@*~@Ka13I@bz~wyo{4|7v9`Ab+?%m7M^N zTIhOuw`t)Xv{lGMT^HJ1qCJ(y78$=NOUiaV=lrr`z(sk>eExI z%%C6w{VtO7ivGvNcqy31D2>~~SmXz97}mI5R80QOVhb{_;MBV!nm%)uZ zV+Epzyyg#~QOvnBXJIZ)PF{KzTl~7@6EqoeE`ZCN8}Mg;?A{cx*#V``viP)KCpo*| zc}2xKCgZ6F_AAi25pl8*jnA|1x}?*&MIqYl1FTxfi9&Bwu*B=c54jghBM)f5n7C%ohB~*BkyT)l>W|?2*{<(?3=9SI%s7$bJ z#I;=J;^7&B>8}87C(sT8oqiyH;?$xCqw~bZ#`1nN6wiOzrrj-QFXHx#ywFz&KAAz7 zBey_Mv2(d)i#&%3Eal_Re@Okl~z-)*m|_E5oPfdXM6kb zxHxRW?S$;aC&y!tke*t@lsZFj-9N9Dc7JQe2CQmOVQL?@poYR zzmmKM=Iljx+6lFxZdQHbnkBv8cq{-i)C}d?4L*5yH*@B97Mo|P8DPlpdxSw zlmetVh_;Er56y-fVaSZ~_I{%9hh8fygsPzJYFprGl$x;(1D8kdi{ZC0k`h9ZqTecD zhEA}i3jaSWz!Km?;=ZnlP0XSM^#JFHiNf5c9D_FAglQ>iPPu_%`LoZ5az9UU&xzzv zB3-PvX!Q0QIlQzq)xVeLHq2^(uzT=$dgkkCh8SM1771u|=c|v4ft$eYW;AQE$O`KJ+>{o^!ES9VzGn!_33!DUKK=14akOKj-C6hq5 zL;>soTF=riTtoGt{Y6-^W|-Lp=eVs7{Eh}cz4#|!-cB&`KM<~19=08UvFo~Sqq>GN za^R&(IIzo)c9Ur6--^%m)4q@q)+!_#$<97OSkHm; zB^z1NW|EDy?=nV8McHctxbjmguZI z6#?(vJaWf7eVc)y}KQu(a*M+AnvTKR+$Mzd3{txL2R2Y{KxE%4_XhE?}&`h!L+ zN}8I3+@0s>KqM*Kz14T5OmO16*7fHBI~A?2G2~NNqyS;8~Dvw5-eyM6&w)_M0!yT3pJg^0ZXS~S2>sV}YbH%(2;PNuf4@8s_X z-H*JZ^%CMG1w%68QzL?PAF_-M`EBnAzuP5R*Nh9KZ(aGc98^@k{_A&zhk;^|y+fAq z-!x}^`~7xkKcwFTW<&Fj&JUye#nWVTO*}2AtQtcF6$umCO$Pi zA8ZU8fA{B(B%~AX{Jo0CC$k16Tx_~g(!%wLQFy0!N^SR!jx=~uH~zA3lJI|gu}<~~ z>{NM^ynMk=hS6$)ni#8qK){5pY+`8d`gbc1m#XE#IKva8K4y$qBwJl|NK9Kz&4!kx z$5jD&*K~g)EpsZN>Ti3-Xi&$!CAMA`Bg|yijc&JuFa)WW!zr$J7cg2 zSa-*Nx0M7;5(=z|Krf^97WsPmT-y_6O)vxD3qzYcAp8T7iWyRmrXJVS-zw)gh3{^6frjer+$UpFa zvv5GQ-Mu+NwYDR?BKI#JZIZNKXUi<20yTD@iF0niw|=7hw|8_Q zxY9i=4bV9#B9tk#NJhbkNtSKoKz_TWgvSnn=$%W*eNlyTiES`;+RI99pn!P}V+21y z1^?mzr9eQU1aet0H+`8zWeY53-L?&IYCuXnb>%OEyt=HLRMQB?)KyuiFbfjqlWOwdcQe%c`HxKL;~O zcsNtn3f7NVW+*GXC%c9QR;HA=lPJLzk@ide>NJbQ;*=+M2V`ikqAi-36@wC7P7gl* z`kD%|=Ady=Ipt00)urPN>uV`TUZ4HqG%wId0#ij$p#qVk$-KYwdFOYNfxz;lp7&HS zc0hfD9pN;LiM>hRAjlg6?H*727yU;IqcXe;>zh{h^E3@Xb#(WNu`bJv*zYV!erWw? zL_IuukKp2P^EzGguicD}8OM-amAms*s!8shzE7%3qS@Gl3z5TLhG!0)?b(YpRejbu ziyT@!6&KS%ZwwrpHsZEZaocTNS<|X7O&Y2mnFcr6lsCAbytpGvum3CZ0;k#{5q>V2 z6<1wRWQB#IG7&mksyPN(XhS;_s^?MpMwXypg8Thrk}1jQ84hO5c|+dcXdvd6BvVXB zPY<(7-U6)Nev5`zXb6@qE}6hD#)^aODC%-x>XpKknc#|9JUyeC#=au)UK}w1EtB0k`#slP|^5zY;{%^RC z7z_RIN>MUvmTZgb`!Gr;Qb}Y=)DQ;n1=@U>?iFHL`Kc0G7aax(PV9Fn+3fh2UUeb=bZ&ccI20*ExI75 z>s*~$XkRX2v0z#Vtky(t(8bHFMD;qowW9qDPe2lP?lFqw%gj*R?qo~~^`e-Ny+^zS znU?qFctuOh)xNu4(*f&(o!GpQ)5~kGM2DL6I)`kuwab=n3|i~qayI7m$5C5V$i5sd zV*I-p4}=-Za)^BMuj-~7MmWzh&SzJaK$g27^6qxFPR#c%ODgnN3-yhtLai>+yF%$F z`7E5W>0kX<%UNSR{~%A6iKrj3s_MSO*sj7w5HG0sH9)GVak1rET7TkudduSP_?Y^b zn>%I?WSxKZAsC@4BsOgaj*n2?8}9(s$PFPThfmWODE2J++Ms3 zCzgHWR9Mod{gH3gL0Y++a^&;6P4yYEfKQ1mFgxB)Fy}oylw$1fhHgIm_SFA+6Oq1O zI5(%#9(l-Vh(f{z=gXCrD0lEXMtq-qG}qYn6cdyR8)}}P4zP*)9Z)r*f}|M5=@P$_ z%Q%;C|Hqu()`$MZG$$>>p<=V*eLs(#ZE0nlw%G{ExECs8G~zUS1igA`AV*K_%E_kU zss5yjx&J;9(yI(*)Q*zAi2*>_%_69(rsk>)I`m9EKXtsC=6k>YaN8UQ1l3IH!b*P} zkV060TQE!d4u5#CY9jdctuwx@wnA^l*6^KeX$op^yrz74&Y@>f8uzN8D_7duFR|?1 z)dQeJLe^QFt8zO3?1vrKWyYjmINxbI&Zo-SPNF3Gj^BikVjd@xXv=_Aaa+femAui+ zc00$5|K^9)zZfpnhmEj5*NhF%W<~Kh`S7dq2+acW1=YjO@^b-a-H#TV8!twZmW460iKuv} zJ~N){%#8)id;`T=v%ezEy+Vw0(Xyn>tg@U3|DnUn&-K&8vvV4*hj`n`~TM>P`+3xNZ3R2Ien4qP(_#BG`=1d9>0+$miFM47Z{n z-YeuC@2OTyTsmGkU2dDJrTlvRz{dk6-jeLHw`7d>zb%+CHQxPziC!>08(B1j?Y`|n z*3pp=7*H3JiV>*MIn>xm(RC||X^P|QQFE!HZ&BL(JsRpDrVc1t@R=p9CEPNu6n)qDs!+d7|=C)+cD}IvVdZrAY9P9-7^(~*J0qy3xPFV!X_2+FOQ7s9AlskG!kq{=3?mJCS`BlC}U?!GQ{O zpBT9D8cb6(iF6Llv^QFFV8d?ifPl!Fe5`V7>%4_O!C$XldS88~(6nPL%I}eGnG*rc zB_9b*xlQ|@$7jcX^Rq`xpRK$=*N)9M!lwnWbWALHB4%7P`I=@z6hiP(4c~u9D^DLkF5E& z!syItSw}@f?}U^r@FudNC%BZG!0|hkiK%;18 zwpi8Ex;0iXeNEHx<~+;u9y0EA>XmK5w7RCx3YrPm#kR`m>u#j3`8ml!A-dhu`Hv(M z6&jrQuakP*7bEZX(B19Iw7pN^V3PzI3*Dp$#nt6d>h2)uW<&1uM}Vm;0XfuOQ|)Zt zckcaprQR~aASM4@6qMuxdoDCVj}B5a;OcmNwv_d8*Drh2VWV+*`(cVSW>?8k28}!>o&(Q_xV~Q!)Q6ZyV*4D@ls1t4xgvRN3P-nv}kefzdx{^ zRuyA+`QyNk`pGadufqHey_z;|Wmd?)Jm%?Ah3|b1PG1u06C<6o?k~O5or#Yr3~cG} z|DN7jwIk~3;RkK5cX3o+$4X4iA#bU)CAc%@iVC_?f9dy7qT{fl?;K=!d;rzj-jc<1 z*f`A-n0Qp(j5?4?P3*NA8}sHWi?6v+{~$- z5s$yy=2F=Fmpa^UeP6)$4tx8PrAMUO6!Y}Y66 zHmgmk9&Wc4FmedrmfP(4e4y{F*dM~CgS!0H8ACx3w=k7|Zs=u1B|rOe^cH6<22@My z*cwM%m&1HS&u)JsW_cZiVW`O$7DFeyHOSYk5(iM;g150u9QEtxR>?2>{6E-B>`hoP z>K>hE-)TD^p*Zc%W?tRJOMsX(?io0tRW;P>thoxdUDRfaFS`}#j0{vX2LGOX(Dc^@{AqeuxT9ny_-mxOdl zN=bKjt03LoDIi_aB_PruARw(Y(jg)B%=Y*FzkXhD9WKvi@6TRq*33OKYwmmVSkB4f zMR9SB>X7`n3l<&I))!YHtk+5_TG+Gtt@+6%^*Bx&;ZavKU-2-$^3MHUcDB1!GW|KR zsYaUs*70q|(1MzpfebiLvRm1-SXnJKBvO9%TTPJ)f5q5**b=CuC%ub`D(v@hNv8vp zW87D-I!rv!vYJ|B{oy4ezm73b}E=cQbV})AN?yNQ=e`*U+jpt>?y_--TR4 zNpjCVVDIt}m8E6eZK0<)HIutiKNtefdj8ej^u6(l&jw3-!tG%@@|jNRvNK?xR(XWS z&Xs&uPOhPh6>THha1`D}$7C!SyZ9uRsm3C?JmwN|+w%lwro|TS=u-#Ru($2k$0pVm z#|ZV3VPy+*UA5c#=XnEXH5X~;^Bqq!-T6+*dg5>FPx*FdwZ%fF7_W12lZT4;zxcJJ zlU^5Okhaam#0lG%3;DhjjG9{zDVnM_HX)jN7i!3m)l_y{H^@(T zM6)>V+EMzPnR@vLT~t*3>X(;rN0D2yfP{iVilLvJ?;34nIS&1IFm z&Vo*`XruFuDWD%D(T4k|q2e#qZeuLYyvgn5WoJw+XXhbVExBq27^Tcqf@wJ3jZ%&VgiezgaWW)FpaANQ zr7WACS(hC$kGA+&c~nJc8U4mjyCB=qs4PslF)yT~?#a+A%AYFa0H83{^W9ZeReNaP z{_JoPvUL{_XRNDiwZ?tdM_bd@^d@RvDGce7Tv2IyhHyoDBy&mF+n!j%mbueZ+D|p= zzonLaTVciaG3h<2UAAIIY45+gE^|&cocVfT_o+4R+jaC8)wihQp|PPQC&dun7iDJK1zdqNJK*NTPDpBsBli%lZcCUGp zpr2{ll$FkVDG^!7+Xs!xw&r6IbEyZlznoD%`*@_O_OhCI1hH#F2fdio#y1I2jxgGJy7w^9;-uErGFnRey8OAnJMGkA5pg|| zQ=maCMSzJRTQ7&&J5=lGn=Ps#i@{KrWMd;RCSV}G?9-K}RLjsSGnM0 z0dHC2e7#3)g!RlRqUe21R zPv^J3QDwBw%Cd`^u;oMcxTfznJ6y3ioWiArFyvW7`k3b@T0ZL zzVC#AQ=P|QMq}790TLzf(m#T@vMu@Si13FuQO)R`QyPuj?qU23Egy%N7&8t+Xe(OA zo90-I76AMe3=U^4G?S=JdF*_9HtK#N)JtI_-Jwh_*W^hgctX1nBKeOsfha1i)`#oXyT#?jPa>`}F7XK;12&rm; zf1iwygO%W_1bH$}09`B+r-`eO-y$;5T@K|@1msSOuEOfi$PxqIFp*H#^DfLXgX$Bd z51h~!c6LAM9d50+{GYYGOGl&s)ry@LU^JbPQ~$X$L!-cqTH+bq{qu>$X^-tT(>E=m zCbEUV1b2!wLB+KoD3p3iAfGU7QPeE{%tI*2^_3gg%j1u2ogGTZnb-?*ZGlduQj1E7etW*2Sf}dwl z@_FxnbNtJ+txp<|RL95Oxzo4wBrQdm)z2Mnz0ef!XHksNdEKZg!MhqAX?+$AI;YEdT$9u>--!Z*E=uI1QX)VQ+$g5^6h9X1?^ z+@|ltg?qM?cxCyy-$JF+*Yr3L;U1pEUYeB_3%q#u?y4`9RnYxD2o#7peu+nr)gYeZ zT_x?mRYUyi6N&%5C$u|)-@vFW;sW@aAD|4oga=xix zws&)iOg+VVSz{YUT}|jnUKArYv=Dv!dDVB`y^$A*pG}8vxS(a59%#+cn~eGr4WBVC z;uYedQVbtFcBu}N{-8=;g%i^ES!vvv{6T`aN?smI{2w2itB%rJj^+ZPEXm1#JXBJ@ z7G-{J(a)MF{id`w=%sIyf>y%I|AWyvh0aC$R;AYCV%Z&$rpbDrYT+6!)Rd+L^|*J8 z1IhAO$P$d(!;;Xo)M3ldQ9uqXMyV2~$Hued`w~lETV=&FGv; zD3xmDr;Eb{?%ADlKvi|f7;Kd}lkfYYZPQ&=(dW1*OsKeI1fQT*2QbR^vcW}QR>l7q z?yu)sc$!8RY(=YZsuWq&oLe?AW&TEwPUbpV< zQ3G;nF=&PJU)GRuw~EuHKA6eA?Q4@BmW$Lu5f|}@+2k{CiY(ZallP^JC3rDAmiH0d zf8?k0U>a40(|alRN7@7`69M*hOgh{`m%H{W2z^eN@LpYjnf*^b`Zm|y*)#!;sRA>p z!bEaox_JwfBrf?WQQ_IOG5z1DQ9YZ)3K;#*V<>}Wz0Ks&lUz$Bm8hft( z@NimbC-<{ucEO;i@>~2j&_N0%C$udL`HR9|P5D;*?@jPY&67<&V2Ca#{O_gMGHYb| zfz!vK7iNCZQUVeKpSZ{d%l#Oi-)K_q`2hY`oSVD#IZ={3iPNJpd7QyE>A5}Ko&GRu zYx3ckaUt}&bMqZEOvR5xyZfv*CTKo~VfRYo%Pg*p9wGNoyosN9MuoW33Amm5RvlNv zblY$oJ{?L@0lThf=K;=+&r|f;NOrDAg?5b37tM_&z3%dOb-l85GxHo5?R?9=%$t!y zDf)I^&M5)AM%uusmGD7>bw<+8)*Rl^U^Gg}%NKt4p}ocrg)0YrzXG&D@4bg%*pc~X zyA>nGZHbgWbhg&2saZ&8wB@f>o>bIAE{3Z)^%~FbS;s}8_vML*a4SX3F;*I1?REjn zg>|oGuDs7J)2n`GmEx~%ehId{$P%5|wALRTtIImNGBh;4DW&*z)m8rO#-i_;xg@el z`)MA80T=EzkFi5aphtS5t;YGOPm92XNg{mR;p9cewCUJo2UI>~$fWmA4jz#S#q;v34oC%l>-Tgmxm(#FH0qxlIt!#A>4-u~142pGs${|73?iM1?# zMff-&WoWylv#InCD;0P1E9mQ#x2sn!CXY`EDeDH$t{d<7sHAVAueI*U*+nZcUaXu2 zvm~aNpo*(}`m=ho@ILvxS#WZ5-B|IB+2RJ=sV_G_+23bE+t%FEtzOrm1H--j=>!Qt zV>aa4G3um7Z+WUdNKz)Yyza>RSFkl1dj36uw^Dn&aF!mW45!&_57n5Gi?e}jPx;L> z71r<43!d2vF(k9?R(y;vcz9_ugBiK4ew$oQJy_$4O@%x^Qa|Kdy<1<{>^6-4G4u0R z9{2b1zCdp}{Ip>9XHW9&c={jioj5eHhzJzLNLhZNS-jg9-&9~qz=9+HpN-?l#x(P( zHNWQx#ylPlzC=tF9u<90UY+@3g)fMxgm7>Q9Uz-%1y* z;(|O&%eBwrq>{i$R%|44m=Pu7 zOGOg*V${01bWWU=G&ia%a|+1DqisyzHx&hjM|KJrf)!926)p;~6a4{rQf-kvu;} z$|4|lv)qv}`loqwmi!-Pq2sp%C)RYSr=0IsS)UQCI!__dc)cUq+8^h{A&(h@B!e9p zOSYtZFw*B$WD^v)m-?J36w==Q#=7umU*wYYZE@lq$0jRU%OoTH(~XxIQ}z@~ag=i> zekY_{WODPk`~HX&6&K&5b!WwB0P$ZyxOT19adART$IoLrRv@d=K1MTi{rcNLl>X*x zMob#4m+OCop5Yr$s^wqA{o!OzwL*DDVEns(+dX4Vj?A2+S93^a?dJ#bLPcJd!sEKa z%Zd(xsI|f^B`%MRozMu0lrq+X4>BB?62%;wR+WgHcDh)en zQ=$8L4hc$d^?L$8D;TQlpDS7BqbUw>%37cDjW~_D%J3wT6UnkZLHxg7xK6M;quW=@ zFSB<2{C!xl8&Bs}P>{#gk7unEkKoSjYi9@14?PwIgK?QdmC0xvYJ5cu!p-$XMg1xG z!P7%Lgw3u~gxMwA9VyTnoNmKUDXN9)*5L z{#$HDE5}hjCj8_nUM7*|J?u|^knz*yF4#ynMPHFSX;%7>iLZ+U>$gk{WO&gxy11RU zRM0Uhz+a5*u6LjQ^;q7pH@&yvt>cQ#Kk3zw+ju`rx}a_D?Y!mlJ4EXDf`)?0r62lY zR7l1;*C}R{>05oQtZD4Xum~zH#tx@G_zxS=EJ_^lJJN#-8x62C@HGJ?L2~YHaC0lG z)r%M{p~NjK|2{45(?liSdE%9cyLT^lzhTacNh_6XTuTsP%z5`|#6 z5v~WgQxH`=P9kh0YcKE*dWO$`r7OyZ z5XVUue7DN4ntMB7+quP|tZqC+^n;C-jk!7I&c(C^YSR3t?LyX_$;%q1NDp6QyJCl! zF^JMQta!Pt!s0z5!6(%9jMoZ;`IQO%ul(y~SI8%OlwJi0?l!IJZ6gm#F8aH{r4c9Ir0keLDSb z6F+o6C{N6L`zs&UWYe=W@z)+X;eVG~9Nd$Qh|kC6)Sk?Kq^o_YZD}UknkHY%Jf|>D zHHFTMERhJ!rUQk7bxT^eN~Vnt++cPBxt|Ze@XoC*M|5>LqrG=(6DE>9Tx#>xQJ}zB zd=>R+E^p=@=HAD9hft95a1QXH={a1`U#eBowJwAPdI2AJZ zbD5zN+kA9jBMD|-Qhc_gNRuZt$=lJ)l;@1G&b4AL&@H?7=jAq+)6D`w46{j!&uNkw zkGp63+bZ(~pMdD$)4KBhzOQ`T)M9@tr9yf-3tmBomE?;q{eW|r0Y`7VCsVv<;hv(j zT|556FZc)DQpJ3NT?TxA#b75k@I$`wV-4{KPdf9=6aIusby%NB%G7f$gKNRb&LtPL z_h%Ectq@R7?@s89>PUH1MW`gD7)D8`DmV2CC}5Kmh6MIi)PHZD9^F{InsivOz|U{J z+1@;iFGf^6=a-GVFBAWo=u>Ike>Bagv}hi%zBsrTZ8xL#6|D7p{0yWybbnrbG2bUo zBcQYWDD4*6oJwJBcWl2u$g`J4GBP7lX36Jc^hi(eS#RrMmjs!^3%g_^0iRu3ch`aqM~J; zSN-EFpk3&0K*Ll~U8%^@lkdd_Tie%eOy%J__d2B&oY&Q#{Loon!>;G5%wJ%#^eQKV zvPiM4HKI9bx3IO2>$lOzH<-8ejE7{%3-fscPi*nmQ@?ph#u$Ev#{7k$%pr-=5_%~= zT3ju5S=r9+U)6*T(-A46GaKURJ~WckewOKpu71PozoLb|LN{^ccbNovc1tf%m}`=W zW)vIYv|ff{g?_+0=n4L!GAKZvV+jR)0o7Wpkop-cry!MXa z1~)U7y;dpppFwIqJ9K*aF-^+Y{TFd5U4iU(`>(6asTut39a_D-Z&ghX?Mw6K9rvRj z_w7gs4Nkw3ht8S8l^SRr<$N~%Y^+3ENhp!A`H&VzI%f-UOJTVK33{^dZW< z^iUveyR{z_t5Y}ZKVbXCl-2m;^Z~9uc|KH%3w_AGc+p6nUv=d0(zrfhF29QV?}cBp zU-XcRt>wTDv{olI_&zeZDXp;O^|IaL%Hw&~BvM>}eV-;JnDT){EK+Bm-0w2(DD?3$ zR@Or6$0Zr=r#d2VQR3duDZA4uu$89YcLdO}2`y1RvbZg|;>D~Uhp4ft=-|nZ1A)i$ zDoP-^piRsz;}#*$Kl=L%vZ=rKC1cmyqbqZXEsh*M4E20Xy_32%_dTAc!PKhCTntxQ zASYI3Tut;chf=KVK3pq1xbPSAc$s~l;U(1g5{GN(0_j7obsCExs^>J$NOu{(Me8y4 ziluxoYaE*6lpxoFBUk&@U4{xvK1R-hQY}LBITe-$V8Ne|R@7zQD)hXQWW1?y@W49; z(4^Ta3Ioe0$N3W#IqH<5T4t{AO}8rX9y3yVRD;IGwDYDaU9vn|Lb+I)%;py?dTGup z7xMf^VI}DssXyJY_@&@|mmPi7oFA77kpaMgYQ9cl2eTUKLtxun(^ zIn#_+CX&63?m>RN^dP~UeBQtn?S1lnD!3}T{HLqyVR1Euj(}~Dhr;98zb`1@l8W2# zPTJO(Ppoo`#Yg+fs>y`uIulh&Mrx{(jX=h%GP2qk6>N(*p9dc>%z#!Ul;-=O#TFEj zyvPECV~x!Pz8)|OgRx-eAkBBhcnBAY#;~b{OpYDa1ISOFuqMmo@k5({YkP|AGbv6N ziJshP)QZ6sCH9q#9sXX`p^=QTcxWi&H6l`2DS@psV`3YgO?BLF&_5JddKZ3@+iZw0 z9v4hjAQIjPb1Ll``D`8#V(%y_YYp?vYz22CWDOn|GHsDY7}q|{6J0ano_s1@**Mvq zh|H~C^DehH83$i0(^Bk^N?2-~gIH~R)LAOuIo7fXTVA2AW0@G6XlLK4r3+D=zs)+( zUJPUug6-bTNuwb=sRbh!+2*qANK6@xT5`AgThsw+(K388v8_o%E3I&Rt%SqH-N05) zh{MZ}l`E=H-{qtwCCt^zl0sMXn|GSOVBscufqY^TH`$w!C@sB1aGE!!(Nx>de3U^K zS>kytC5$1Zs_=8~(L;ke(|h+gx4bzv_>E(Tpz}*&mX_I`4RIn}#de1gTpa~&Cu*lA zwu*!udD~*y;l%r&QcVRCSCGk@fGTHra zLw~&g`pXMkyy7-`OUbggmuK`Wyg1*V!N&1f6(;oGct@1%%AHJxG}RM+u-~IS1WYtb zAb8{}|4TFUrN`LWZMZOmnHe+XYA+zUxN&LrWZmj_jYJ~Vq#EA=kNoz6<`*UnRV8KP zRq!9W`&0I;kl?`TIuZMw{6Yh}#}I3yg?l@LIAQvw9S5{s7rD>!!Uk~vvn_Tq*`X_e zio#k-tKoab0dRMMd)t?AjqD)u675`d52-`SRgC1Mdn40iWMbOgz%;0V2W|?$+u{MvO!Zd-HR^ko_U)(KMt zP)?9Kl zOIIE$wzVfGu`pI$l$VIUSRZXB*dVxnaGm0So^jyN;NmSRTYEGy#@lh+{l?#HSopQp z26Mu6PgyTiy)25Zwh)Z!%#uaorPd|6Zt{zWQU-!*W z>3$XA?>EQU-!0k51~%JV)(&r8QQql)Ncobg2ML@x;4ru%Ms@I^AMQ9NMn0JN?>S{G zeGmV5kaVJoLQqb5r#jz%;Hdvv(op+%z9_G_{aUQ-VNmy*K3XF#`s8rKkvU6K4yhky zlI5+X0dVszxR3+6C-;-{U)}}Qj)U?Lt$~}2r~>~2OUm_VdR82H`vSMWqDapOhJ9|q zyg1wKO>fG$R|~#sYMd&+2Swrb!g!~_TO3!}4l4qxv)K&C!b4m6?zvMWjz0yEDfQ?IG+l}dt-O!9k0T(oRk z#pCTEXmK{TvWo{J$1OiwD=EbvcL>4wzHziN$jdmmWLUQ=0rW$~k?1sF|Ji%ARl)39 z>1FgN6}`$mRD?rBhg5h3iZtty)Un6t5|F8s$$nBiGOSKT`<_ssh|*sz zZj_H5ZC+|`tngRu$+Z0sRDgOwNZ1#BJ+CE?j6>yv_G`>2*-|X-f)Hl#>IOR|v}JDH z?}-J(%OQ`r;9Imn1{c6bMZ)VA$i#$VelA$BtEd@wTdSBC(`V<*-Nxcj<8_!0mx`5( znGKlEe$6$2XZD2T$s}WmR-d+aUfZB3TRyg8(HHDEEnd|Ka~-j9m(I~`t-;8*N|T~x zPxW#1?N5Nfuwb<6P1~Tg%164P_Nx!&gKxXzuNE>=O#+<#tl10A9#cv87|7quT5uYx z<1(6>h-!B-bLMjE8EmWHYvw%Nhe8aNVlia0qkYcYbn-Mf(z%9ZCeK7M{Tnp|#C@vq|XuI~iq z2$>u6`3!6asqfWfwh~&!OVlM)S6;9WB$|ezB7?%F$YA>}A z!M*U{9*cepyC-?iB@U)<&G45|<^za-9qsTcBIV0?E*c8%z#!^_U8gJqY<1p)qH4*< zgB#8$x=ux2+bz#nBr(;=;*^I!g{efXwnSk_sE99_DYEs%(O-XS#GpU(e-QgHd6m&w zo2@2%e7X#oi2a?YKlw6Kukc4~=+kQc>#sliZ~W9+p@>?G5f(mUU&)N@T?wGl79`${FL;^v!y)CT~l&6!EOm z@*G+exM064jH^2(ov)H&CD!DbC{(5rL@5o5y2c*YN9ODm6|^YG{`a#?vDzQ~YuvtZ z-anqPs(2J3S#ziQJw5PxfSA~i9aqrTg@G%iY)}@2A!q}({J6sxdCD*cVe_uT`N2<+ zT8Du<*0d4bc(Ey$=Q)_57z_I0!cNUS%TIAoVr25axe$XSFG$bQ(WUqu{tzMi3@Uv1 z#KaP@l%W0xs&|sg;^$vC|E(C={X5a<-~O?*Oid+mF4rr8?jH+ceA>d=dS!O;nhT+3 zf7^Y(<3AYsI`UJ|(LwzTL@rT+CwL&EdHW{_8*?CDPRGO~xc+z%e$b73PI|NKXMen*44kA@JtPR(_nzg0 z!HZW`DbVpuc3bc;>yr$GS;_BHQ%3c(7B)8d_4T-K;HoWR4G1D`QDQv+MN26jBIIn4 z3I?)*qe#@JtsJ098N;&l$(zm~ z9NC<$TmU<7@0(g+a;*)d4(vDb?WS#3itT>PK0o2hDhgk5Ae8$+sT4YSxwPI(*jYdA zuGj3&`V@UTVg6QBU!N4ngg`kp0Ej9w`}_Mnu29=SsBsg9k;XX)2nZ02DYo@qdQ%NJ zfWhDYTOc#D^;nJ-ZQI88EWQU}r6697n1|c*+4H(J2U9PX^NUx+@YQe0$;s^lRaMfb z`ue*dW{0tl7@h}#PA5HKJk&#k)I1E`@?wYIK@4qfFSMp1-ez)A{nkI>IInG1P@!b2 z$aS!;lXmKJZJpbpH&!bFl;Z3&2Ps}$kDO592w zG6n5kSnrh6k!U$;d_25fM&TK2F3{6_nz)a23o$e5+3InTW6seJD~yeeE#i46r>)I* zvNaN>$ze#J`P_<#fZ!iXivGpS6iE8P+Jivf%dEdRh|NGuKLNqaum6P4Zi8keTr~$G z8XlM5-&^kVgIMVJ*jViO<&Q=u8h*Fk^znsZS-9+~kvux841`fhl-bAU-YN3KFun(* zHMcH({U-mYshK!C$HVA}v#yz;;T<)#5f7X8#SsKo2x7_=)O*>cU}XH_42CHcfn84c zyiXAiA74~WZNYPk3Ft)l_ud?CjOG#XIR}F}F}w@NWpZ$Hj~sD7)#(TRi@}@g#hWyk zB4+CQ1FJ6DcDc#|W7j?PZGV{b+<)EMcI5{ZNZ}~qYxgyWo=D>5_M6k2Ongst0ZAKtQ_Z%Wwn zySb=Ak4Hg40rgSvVu&Y(T^n`D0Dc=5_e8tbuiubf?+1Awr7aE~YcnT>y&(08M_6)v zVj{UT7EG|~_@1wV%;VDO1xRo|13&e8Z=uzRM4%NEbyHSviryw|_#a~5UPS@D`lwu5 zkGuU~muF}0rXP%JUjY?aW-Yr`y)7|%kWruh^4S^BzqwYh1#>&Q7vmgo(H^l}H-Fl% zHGsh_IG0w(WkM~T$Kl=Ex3^zuYXi3p_`{(s&Q_S&g|h>~O(2t9rd?lc&=p8^lM+zN zfEDr$yk5#%{Zg=T4ta_gDNwx>_wwRL5Aq)!9Zg>c0qZ`2ip4$QJvKe^nEyNT2-Y~@ z3{h5l9Msi8dUHVPv>d_hxHgDis?Gp>K0rYzH=So0al6trG!Bi5a&(qw!%b7x|4aRkG6|L*m{ zjoIFC-fa19z)gFtcC5*C^{P)ql67Q;;8O-|vZn?zvN_w)Ps}Z{uK9LSQW9M=(d8^g zQt;PO*69Z2x;Wm)Cw|y|pO_zg>*q!mK(I2f?xHY! zd-Ecp+<1QP)M@W(&TFbH>_lK@$91w?7r||kAw*%tml64xL4Cg)SO44ebO?~S?EmV5 zv33w0iBjROaK zDKARs4tc`$`Fa-VCvqAZnz)pd5WR}^No~7sV>Sqt9W=jEO)ZErqpU>-8zQVw_F{}&dr$*HM<&&0~cW);W>))2gQydZ@A%O0;} zqCdlJp>OeeXOYe4@3GKh8yIDm%M?ffkB%T-g6D^E`kTtT{XQ2b4u>n!0wfrgwW+bO zu?ATT;uS!^aD-s47#<$JomEj*_D{e#FYoRfZM)V4;O2P!xrL?nl|0DzBKzNNo*za~ zAG~@+0HV!QvWYsX=Qt7_phR4ve-(NI_yWW{-@*cVdkcUQp^tIZT%QIG?42IL)Z{)G z*xhG-*WM5d+YHt0TDt4=~lEHd0C%8j_c&ml=W{?dY%d7|@eVOnl-C zZg^k#=16#Su^B=_+TtZOg1It|T0Zn7@%JCk0>!N1?dh7Ll9D`7O@*(9pzQf%F&^mM z4wNWWMm;}j%rG#uO6)q`-7_{djcnL{9xnb2Zw1El83?(oZ;{jK02*7byweX`fMAyL zg;{zPW#zZIvt~qgK@d}ZEk_BHOnPx~bAujvjyZuem^17~)5m$S0K6iwURs>cYr^3%k@XX~ z6Xdv;1#$*EGqnO|KmBo0sl5{7>gniSmE>89L(*?aw?0;Q}3`9nnH!_+cA`ATRaNGsv5N1wm{C;e;y)qYp&UBlbs>mw^Am|baXNOjLZ!1oi2CX+Lwp87b&^8 z5d7+g7bYfD1QCJYLlEx%`o!+$N!k#Xc}32vvVzLWXxM*5 zIs{JkMI=#9ON;mC?{Ru?Hma-HB8hnKh*({3s`<}-Z()X_%6ya@_yhzI4hoJg0)>KkU8zWTI;9E&$;F1$!r|&b-@rgz=H*i< z5jx}*dmeQ%UNxER&Na%VaYaDj0M&MY<~YC=x&rhYc>8C8 zfyk~qGZL`d^a-I7VNY~lw#I=ie%^7#Mh^-J3!@Me^!dC1Z!QNf0d7~?(04NVhGQS3 zHOnS%+ZD80#kjh;<*5__4H;ViJ0#zjDDOcI9O=$m1J3WB{(Z1Ahv13r{~lX>dc~BI zl7f%{M@X)JvF5_y4dY;72t&AH*psH_=De*=J=*Njii&jrG(ZTNqh12vknj__9vwhA z6B83};207yrmKCp!=bLF)d!X!3EmqF3O9Xo%dzoCn-4#{O5}rCgIkt`!{*LxeG~-Z zp9k(rMXKht2%<%R#}J}lDk*&iyW-oPTLoQjh~MXzML7HNC1=ceR=tmm%V|m1h(hkcwF%a z@iWLiVh=7vRC^Z`%S-j!GZC)^Iv~Huli$Gyqx^$c5K}8{=bJXF1)qey2RgL&_e_hF&-K~VDqMlO%X_p27sOlXcGZ5K@<{Nia;*ss=gdpErDnc zY**hcIl*x;>NGs#YutqshhQo82FgLiUbbfHRqWNUN&*WB5TfqZ+;EGle9`myGhwbv z&%|W%`+HH6vHW^T#IaJSFXDrGNd1WB32WVgJ2suGJ&sHPn+*c7P=rARO@T^LGP@Z% z6&2O@iHXs<241nLADnf#cqo}Nx*Rk8%GiqG;7;z z^Lg1b$!|hEkWenn2f74AYE-pShTaKD;c4HxkFqJzA!|j@aLp8wCJ_tW@U=ha- zd856+3i?9e7b93)nr%}E|IHO}T49y0N6VDGb=bY3TbspLF1XC+8|DiS;aCn_qgVjyLfmJAsgKQm+`6HX;4F0~M z^&_x)twzWEV+Hg$r1;#dnVDB)u_duFF{0Mij7&^S;P4=$p9az^#77mR*%F-0xB*Jf z{`4sxOndq09Kxq5u!@(ysx2OI0RGkOnc9c-vk{Vc2Iq&XxO26l$5evMNfq3S-f>E_ z;zmXk^a&^lG7>yQ1wa>jb2)Nj{Hyc*AWR+qs7Q*9?aY-3lai9kZfzxo^#by}4nQk^ z<%hw=|Ll#S02amU2}?S9`lY6=a>A`Bo?YO1Jnz8p{R}yizX9>hM1^kxf7X`BoGE>`Cao0WktcsJ<`AVqmM|XyZY}9 z4-S5SuLsWB@?Y`DU~95$^v&4%(Lu+5q(AL>lyL5)4SfZNC@mw?^qusym`rTJPy{?$ zd^`^9#r0b_a#fZS#T)XLi)}uaK&)lPu}>5UF$?%ZbNM|vI5_s;2VOWzd zU&Il_Q7H(13B6CItwLW{>oReDI{0mqnS z&C9}3v}-w#bT*v_PA_p+*AvdEtvcYsgn~v9q#X$wdIpfV+$8~Ua`tzsT><s81OlDq$uzfi z;kX012}eLEFL3cHii?+kob@rJDER6|)?dT%2U9s8W+(*pEdK&mhd7C5-3cluMINgY7=-7C6lyyV2 zQ#Lp>WCAC>Yk@!ofgmbPhX|f9z4~5AzWK*Va75iT*MTa&1~4p{Bk2r;eO+B$#BmrkYWq7;(Vv<} zL3ulW<3+_(-h|~vEjl|pd*gB>e=bv(6783wmxba5~_Q14S1?kg<1918NXS!(FjQ~NLv(z&W)X8RcUdSU~5-^@{ky4RK?oL^(ttDg@yU7 z_;FAUt}phWI6VZVs*lCRoqMN7kc03#6)>8zl%v3gmg+Rd0`g@y2~>S6YLxg=-b75v=$`0`B&`d?suu4Sy!JrKO}GyH*BtS+3jsqsfH< zV!+1Oso1?Xbwf}c1|{j z1kKx;O%$t*e#c?;Mm+YoIR~OC1eF8vQ*`toAO=l1u)8_2gR;4Pr3oG`ZVv?gWB`>= zsE1MjAg(tx>|4e!P#l;>%{ib>afpDQb8~a%LMxD)o$uKB5$cLLK6_P`7#<$JVR{#M z&DS%%A|ZXc^c9904+1u6SYR$0T5zOpe346#;oKf#0t8ABK|QloIBNbEm99j!Scsn+ z%?+0ZYGZId2E{N0aeQoG{bX5MzkmfI>jsv%eJbTHBqir(D$}fb2OHwsEFY31{q9Wz zg8B6PX#EI);Za=HWdL*nAao(fs{t$hM9`ULw}wLj4I(sT+>+2LQo^Bv@B{`7Z)2eU zA)w(Em>VMDy!oqt6NX?I1?U9{A>uuNPWVet4@gvSZJbNy#a!$*Ms z)T}ms0L2HW^S%h`lc&ahh|5gL!eS>N9NHiQ$G6(B8wF^-S#ys(>&JB4ywBf^^nCp) zN%JFIb&oO)ynOhEKv*{dUgq*%89GCG`W+<=4OEFzHyruEM7mHeA!P)V>g0?i4{3Ck z$Oy2AK?Ou2?i80%^I2)Pj+b^IqAhwc)JsxqV!@VzIP}86o11q#U!l+lpbMLYNO;K+ zp=OtjQpW7(yMHNf+m5oy82fTJCIaR7|!;xKi##j zwVa>_kdhbYQJ|0^ z#hkyIS8Q)vBjP6lM?$teS*q1VqNZ8-`Zgki;I;pGizJHXudkR5Q=v*x1K8e!psvxm zD!@G|s;V5Nr;q8Yjw=tC4; z;pbcM2;`@k)@8H-cy4SF()XVRU_K8IMT&op_{~t2G8^aLZ+Yam^Dz;>;XQsp`yTPz zmyduR;Klc!N}A~0>9=l9ITkm{>PZ*8tuW~v)ie7L@lN-%`+nuOb&s#J=81|)>vewA zbJMvaQ=59*E%<(5&+`1gcS3rDrGEdh69CLwa-p7$PnP9bDWQ_=qVI74tsjmvsdDf0^cQr%dpB?Q!Ln zCZUH#)yjv|KKT$TnjtyEbc1&?K06<6HlO05*6e-opYNG1vT|6q|Lfp4{s=a3m@+srIJ5(zvf zix5fJ4C)M9zbp~_?BYIowYaoUPa3a*+2nZJrwo1KrlBdN#@O)XlXQ%lgG8PU#uO`@ z5V+t7INnCrbYyqvQh*`oFa~GN# z6dOk7I(9lEXC@0)K3m1$)F<4vOt$ody4y^4X6vJMzCA;Xl?F9ZBZv91uSaV>aJ(<3 z=5D_l*t{K8WVMr1UK?~pLqyq4C{_J68pkgCS=Vz>QzGQnz-?n}i}kWEs&T1{XWk(w)J|AbEE2HbCWh|wy z1b>nZ|F)@-VHR9rd1oIOhK)Oor>p*c<>`K&_m;uYr6%pSDYv>4KV2Q33YMW%6@E%Q ziL7tnJ%4q1&sb^CXD{e*RLmXSmEQQQ?tHI1`L?}}g&N2^|MNrfbVLugy~L!|gmGcA z7SGV|CxNZACpyPD6CWFmw4T_^3&HnC5;N2OhGrwk@#&BKhw8BEP=x|xZE4x?r%Zi# zMPh8}ZhbCImkolVzh?!BXx;2f3&Ko;f(p=Cc68frkDRAT;2Y26Yn-ddRBkl-NK;GZ zCARj=PQP3b`spw)c)Q8#&gdw|^iaILo42Y{h>` zy!iNs>HFE+eh2;E22n;G`E`Mn$M?Zw!8gSs53Y~AJmsSPSHx?F`RJ^=+b{bv!DeZGrF-QW}-y;WA6;KS3Z8Hrg*Vfa(;H(m~0is(^^=xKfM zy}mTMu$YeQv`PEemmBQTIstgT2GUcDuU|i-Gdo&q`YgzcvWUYkukLXjZBoXEOKBY{ z{3%%8&uvT@hp|PmF^-q@wps-q&ds^204wl55_#6Jan-8pyc!*F!;5rP3(nxiTsBslr5uf znOo0P50QTUGTm?PB7CIfuCLtvzZ5S=YERqgJpEm%Z|oTxm6RposBXg5tdhVceivWu z6ZhF!1H+d^$vqE~*C)?wWvV@Sa)V_l#z&yrl#R7r?vcJyxwgt%9`^5|H_nXS7q)7j z_@ddIeqAK1?X$xlF^j3i_Q}y>_QT$vhbY^+rmMTD-wl!CA8OmDJ}~d;_$>O?{o~=} zF z#C-inxTuG8&Xv>0>lC~L{Eq~F?{ZD?AMDgSoH@;uZ^g)R$YZ2EPy9l#jY zCo07v`UV9dYLhm@Qx@|19}R`14*Y%ONkNlGnJuGqZ|KwC>6SFkMGHfb(w%tIq%o1F zX6ARBTP}Ydn;)Ftg)o(v-H0pL=uPP!AQCH-HK}VxW43-ljExvQmKGP!%CNT9bH${Qz&GR3wnIfjXn6-*YpoecB8^Njyww_ zADl~ADDnz${k-eQ6Euq?Pf0Kj5lGTFbl3J#rV`2F9;g@;lG6%SX=w*)Z z?T(GAJw16c&d6-NU!zjw$EVy+ydmG5DJJY__j0wX72IXhp}4_;X)>0jI^!=-4w@a6 z$vL*##2KygpS*+Ft&#%+i{^s28C}_vpLcB^Pmn>I>)jXTJycyQ=QPt!_-r+u?;fHU zn2V_pU%zzmtlpCuj)NNMcA`B_n!Pmgw4>*XDtaF0jSQHz)I~b8X;XA%pZc9rD{#k= z<@7`AF6xhbN8N5o7Lyde9M88@r_S+VQt(;PI{$#?_L&5~daA_4f{`E!&Rb*omd{%h zOvkcrToCVipGFeJl`}T%?CYd;NEDxD>8RHEn_+e}#T^>6T&>n7h9bHGHjbKo$={yF zIS-R0s4tQ7W)E*3>NamP6j+RRNz%yF`p@po|D6TkwUqJXKlt+2{hM4d=c~<6bPtk$ zc4}#J8jV@_Kz3CyZsRW!(^bKe*_GC-zj?PpX@kj6)tSq`8I^xSS1*4m&rspO!b=W? zUu`!zPj}aBEq8L^o}`wv^=J|G&f2lv-~QX$tdzKKLvnqsG6clcEBfRMj**x18S0e; zP4Y(Dtyn9#b6?8MMCwqGnvF}`t9`d^onQ9}V8Gqe89Qz5LMhY4bx3wr^Ya|hk81hN zs3};IVH@(LI?8nUjpynui!a;Ku}#yn zdfqyR^`j0m9d5bH;YWZJVROq%eAx2JwSKD z&flCN=}KaCfAwJS-)XU!3upDra*i_@nAgM}U^Nn)Xplc;@~AyDnxXN_q{!PVCJjo0 zVdh7+;(i8bxK*`gcBI$cxEAF=&S|p9VBA7(`nQ^&BAY2_p^n4O@Z3orB~uFvfgY-& zx%TPZI;G{#uImT_IQ@M~RKJ`=Ulb}c+r-kouD1*i`unJBbj$2`^r`W6?_N@x4Igns zoI`yb>#DlIfw{nREBa!=p3K~z)HW0aF6;Kr@f(Mk4AKpl>a>CcE#)ttF*;+8w{%(% z=fBino<5iDa?8(~cE7$z4J#?#*dirip0@oxJ4R`F{hV_tuP&|UlClh)T#US|cfj$| z<5@Dv?0h_qtTE4QA5ca_2Y)yJ&}810(6N+#`w>|C|lPdOJIK?l<#8hLSSnRh+H zs7oxA&xQ#!X(~y0r(PG~dEnpibQ`BwXG8TBN^XXjFGU{q5csa#bKf9qK4Ri@gwj9! z!U^-3s;G~hipd&Ar=;%*WNKfI(GIN5GEsl!e>}B;?K%YxfTBB@oK?}|*9*frHBrur z%}g7i>&5>)L#CmZXa8i?I^FxTe{o?X*5$`rew*^3Q?@jRIC9fP)SpnrjTtz*OT&u(q^NNX+!_pH*L-%1g;}_@u0gsERn;)hc+9qyNM^zqxbY z#otN2sr}ge?-jZOEsRR181i@Wo}>tkHf5I)pLa=D{^Ckq`R>=A5L<&Y28~k`s&3P9 zKv!J;v>ibmWFQ&i>eQM-t zY}iLvs_fVcg-0yevp9=!)OXfSQ6zD;4U$t@IW`o$Pv}Va_Nv`dQ0o!HUQY_0f}mop zBma%jeBnOyUnEV?+X2q{x4Nz;0!8$;m!7{ZOT#0>bLFi}mER!N6j)MV^5uoc9vjD$ zzlvhLHaz)q*i-)_eMTcCq&H)~CaUpM>C(gKKepn`Q#KN6EETsaCL~w?UVKlsvh!F* z{Yaa|7xMzke1=27b#$=|1q%MPDK{N8Qd|TDdp`Y+5n&&x(fiV@t6TX}CCF?oGG%U@ zT{wFAj8oRju%Li?!J5IxU*eV1?3#AGa$0Bzx#kw!V5QGxyvTq$Yj^DKWPPPa7ca>? zeUN&*;n{DFeP&;uL|QYf`PbaUsW;n9^`K$zhaf z@v2(Yw0o(~p01}M^PeO6;D@fr$X;FfFWmg2tpU=fJSk)~sG-HKi=Kot=FK zC)r(9I_dsb-*c9J#GHtg3D&1ANxXQYy|YP*I>XG9A#LbmSTqg&SHAkEf}}Zuq8X-@ zm8J*wS-fD%o*dA+_M2sOJnfqMPNN;w{C6c@ygAUYOmTuNF_a}Z-zohXcfQUc?yI9N z}5BWjL8tR){=(GP4ogx3Lhe zIT^<*#;uzx_jk$C>U{6hN$5=H;!&jIPk(Lc+%dM;(f0k)LgcwmlxdI0*=cQ6LQM+H zt}~2ie72`MxTc|VyeZ+lFs)PAwan2g@+9YEvd6Xet={>eSEm&yXy3Yp#q z-AjtO!u%}HrO%Q~q_w&KYUVZbmW;>v0-4$>YA^ruRsQhk?=I7rvF&je8W>`a@JUTa z{vEh`ui5FpOJsp3T5Js+vk)z+W;QAkD*|a`w|?lwi#H{{3fUjgZ29$yNR4qHPNs3Q z+0N=`D|DZy%s95yG8BB1s*|#L`0(q(bK$oi8D7MQ3SFiNwvw<=;SUWro~BkU^}P3Y zlP|m}I?Da-=^VWh3sbq%r}IYNpD*rz^Zstwk@l~Bxvw@aOP?vcWo|wZ?=^*ud*{=s z0k}P!eM#}8K04m^*k;U++TVUJF7%AhQ!r?&>gpZ{%i~dOnZ6TdJ~pe~^8-k|HDj&5 zfOFrTE3Ri9Pxstuwp8q0Es1+YuNkqmNPDZ?&CifFjJvFcKVDxgf#?n~-Nv``d!e4Mbv9C*| zuK9s?{=IIajcR%Gv^ZDwM=l+@{y^zL(HLwnt=TkNgbmQFMhRig6r$uTvxW>eXLH?v(G0Fu++S}%H6gYvTn(%s}jVaoxI(Y(|Y3DEEyMB zb>F*-CP&jBxjTlE1?X0(Wojh0$HdydRObQ!ZAF=MfS>GG(uLk$Ne?Brs%K0%3GbV7 z=`-lQ`KTvrBz9P=>HYP(%b(-r9FtS1 zetgk-9+8r8B-v}sp|!L{Uhwo*xmSZKXH(&&Y*Yo6uS|LVF-MB|9TY$F-Iinf^+M?z zgPpF_O`iXFK0z_Jmfuc+CGt;R$upzlZ`)>ju3OFOOSL@tM^25}eQS4p74_rx*~tMD zir=X^ADurucA(;1(h>C6b^z{~UjC6ray|K$@W0Qv-|lh8?EE#Q+eOuiA;O_z^}GhO zR+2Ofc^ozTX>DI;<|xx5zCUN8|3mYS`AP*5aQO9P&CpocBSZbtNbRuoI-}%M$F`?l zOk+QOD&U#lW{<1QuMRmk6+@rAf7Dow{RWtr;HlfRPh0b&UtD?CWhXY-9`z=B-+{rs zCnJq!mRC5|qtmVJ^%lxT{&8=*S;^sglj7YW3+2kCpC!QpEkCZ3u^QN+ka?HUG%fTX zZO6!5fGgE?s{FRGMHSVCf1mT(I_I{!(sj+=$e#chdE_>~Q$lPu*%S>W=##&xK08!5 zu-%hOc*bg=uYvLhQ)B`w@mr@3gj4XMD*4Rk=2)H2HheJHX8J}BDLLo%0Ul2c-llh! z(_%IiuREI=<2zM*1{i)T{-g9xh4yZwKAn!MtiF-d`R+%9(ER5*Q?r-_C36Xjwv5Oq z>bik}6z|rJCG!*ODz1HJXUO*x%eQZXi|kCntrdH>ouG(TGH$5vD9MOmf8j?;$+^8r zhR2P!wVH0%QuAUlZ@Orhx6RhWYgDZ1ghs?0;)7_v$E~#roCI zIY&%xaGXo{V%7QVAlvMy&FW}NkILIu-L=1{!5ze1Z29-%_MQKQjj2^@6@Rs#YEh4C zWVqYnM-|KCq^!*1zylIFSDa>eO0=5Ryeppde9{roZI|afS@AytbmYoGdykE0vD1Bj z?d8+1<$1XZgbDEK@smc#OOPgLWUMT64DYkUeo(y`_Ry{9n?v7?x?SOl#Md2|AF^Lt z*$51u6dxL0{$WEdP_I;yU(Kc~v;FwHNB7)^57wHTF<(1ZxgmFAW+pwY@>)RUKRVP? zP^`RsC5=qTHe-DGM?}Ppg{98b-zmWxrC}pwSvwShb!zp#jXd4g<3^vTs@1P@C);Er zUPbNO=hGKcsfm>?El`yTkF6Kx%`TBydf_UbPxmja z=Ed?onIpX}%h_Pr*Dm;9x=$;mP{(ak6Q?`;irr(B`m4nUd{#RSaJ>27!Vl(@HZv8ja^{7+YXsE7pvRCWMyU8ReYX?u`lp=NFd`ECeZ!hAgAFfaW+0GNlEjCNCHr;9y z(k_5!E6ZuUIpi`%ef9HQz1;1`eUf|XGNUs+R+ANUqOk(<}6VOEkiapHvgku#d{G`^t1*gUdQeQ^u~g^lJx4Y8RHm zS9QgOU1BbC65(Ng?uffsmDiL=omClW!26w)PAZLsjwAEd&s!9$nd~I8E4|7-ypHlY zz%rrcTKcyh#@&7{%Om(Tte>O7BJ_}RcLq`9z~ zSReJhy6qin<`t)`zbnoyQWYVsg z1b2o--uQi@(Q)PT=Ozc<)c*wz3LB;iJ-tn`w-`k9JJ3Zqyx7IZ*F5|ydI6n@N@Lfp zGVYxU+hmL4Zx2yY#!eerX_Kch#_VKD>r9sm5q@g?u-}g6rejaiUTtGJVxyQO6_y05 zEDPsO*`ElfPzE35wXHCo9RWx43hvXmGI>hGoHyDs?#YCq6m6$U4W zD3Dm@mKvr3~%6MHv3NNK5~oA0v9nKPDD&>_3BqyJireBHCd zCmI!=YU@Nv+D3UCzU*zGdu97^se7MFpEIG)a5!t!ur=59PrTz^4}H(1C(m}VzNDjQ zAeC`cZ&qx`P5PEDuOjgD`HjPYp9$1w%z_3C+8PYs&jD|yB>udV)~NDWxF+|Ntxnmw zOO-NSS$7+{9h#zJ7sjHLh`rG#9G?`kzazv-<9pBed3PF^!gAPm79C z#>yz^JbkiQQy=Q?c<=7j>ugWHnw~Oji;s=Hd20N=pk8}4H7Pz@GhBjS@w(>LQs#BB z+h>Yy?eTk$he5#u9{Nn7hlobU`O&CvJ3*;%ehM;CHrkOFWUbgxcmVF}c2vKkC^>Ka zchAW6u!YI`6OE8-F72QDQ&T%G8;1VwCU3}(u74ZdZrRCJ$H-6DvO82G@A*nsX7X_s zA_GwXn=6HEPG|@hEUA9`?km3UWkKVu>YLuXpB|*Wr1sqZ3ma>sys5(cAfjQdaphUD z=WGe*S_|LS=Wo6;6+X12%sn3Kyu6FORO{cddz5jF`@jH8tjy{djVW!HmacIc{|UbCGU zeSlKG`txz~UHKQy17xx#b0);uY|faUqG!2fKH*~dHmsRVYj0Qf z?w2=e)t*@$|G;~qeDSG|<0DS8!~aMOj!nbgC$g;_`={q*A6rZ>ws&1ve7l2KAUfuk zbUe~K7f;gMtUb-s8q|<*Osv+cv-uN~)d5v9>;%dCY9+33rT<0hn&T)Ti0}3P+FY03 z@h?N{ImP||{N@v%e;MfibD#eIhkwd7iQGP8Wy2feL;dzENd~N#55xSXLMjX0UI<%B zcl%;yX3_J!XNC9P%O_iC$^0>Oz03C^Cdx1t74i4H9`Cnw@6#in$DARegKRE@CfR>JmHEfq;hD)!qdjqutl#;pLy0av7r%{RudoI`o!f=2!#~#LXFM*% z^LZ-URg4WSd6Mqh0VRcU+9iB=ySdC&T!;PWpKXBNtVp;5#6FVQ8nRo3>UOnqdLEj- zMz1hz4N0X)Y|BIMV5lZlRaJf3%~LWm9ze#7A$mp3hC{`5b8T_VM9bKr60(j9>yxLU zlOu+7^2}N*Q-t}Y)!er>mZ8qPsP;^)O5fF02*Wl|;Ml_$0IF(Y$h)}s#oxbwPjSxG zU=2iKl_4~S2ybv@ZV++|e9j}~G*%2kW2y<`>lpfmCJBm{j*(&Rj~aG;SQ$u07#;ZW z4;fimH`f15VVOF{-e{e~Eaj7^s2~U+aYJoQb&C|C;A%^}guxIOz}kfQNkU(o)?rlCQ-T>8jAwn9z)(6bF6j;tV(;nc=`ONkhR~;`riS0DJI*gh3^mj%y? zPvV<{;@ntIvV?O>U#=*)xh)^R-Kxr&f;lfP%L31Td7J7XNXlV&wkZA<94xs1a8%_9$`b(ud;8WqWVxWYJB9ubrK> zYfBRzEB%41_-sCm7H&)WaHC@Di>1pMrQcBL4BRDQ>Z7e%`493Vlg|q$TISu)5yJRe zGZhgWaqjo1Ig}r1fBL9eWtH>ovWJH#v;l({QyszX`u3%_2v^(pZf!1?V@|5Qt|gls z^P7WDU!GR~P+}??tE_)(N{|?}n!|&I;xFl5xpUpY;q~`pGYhF{X<<=OXD0`*aUa5% z+Y7~)m$kT0g+dWIhb{386N3)74WOIAT$pRw?RO8&R8*+29U-*eVVJ8`-`dLTERvpr zIl|)N%R0kzc#}1RASBOwV{LRqGvjKPwUBLb|L&Jc(K6VI(6?aYL)m%6($W%+0tg}A z8xsbI4lB5TToyBFyY56~%0kl&6{sOLv`Jito~o*ZwKZ;p*c12x0ZW&cmNJkvO8GZ* z{umN-`{_bUcwa?n#^}FJBTy8`hh_6&rb-gj#$yYwR1Cj&xb4)_kT|Pf;N8n6Roz8yoex7I5%w;vr zxI3F33YRah|CZqfKhC+hxcJ7VN5~teQ!hhT8pYSY>;Mxk`lg6tm0B*W%vqggA|ZmNXV7t;o+$UWo6~*ytUjAjZtn zD|=3i0C!mzZa2%iev$^J0W)B#C%)yR;V^`RP!`(twm>YSop0X0%j>|Y19Nk8c<&T+ z5}ZMj>goqv%VOg<7t;vY5EMUD^BMHR8i>8U$5i|5v56>GJEXj=hygFRxVXtTCT6t#HU+-U8UDf+m-Z`-ZrBoZ>214ED z<|ZDwZ+Q5RZ_BCZLYlRf!0I2@ZKWf)WMyR~B_*{gJOz8b*Lh-NW3{q1xVgE_EiD<; z6Y`sTFGuhus%P8}3VQ$HgGs%cAwQjJW+%3<2|wK)3X0<*A}ggMW!=^<^YX%gV=xrX z%geiqgk*4L=8oQ2H*6dJr&z|t^7lhRKKAt7s&T5fEHLJ$Gq<)*sjOsrSSx96J}S5z zop}n_ut_jd5x-4+@Ze(JVqFD;e8S;lw7j~61q#G?`}Xa7M;rx=`uemALQCHFZhHvR zc?BqUfByU_d5M9ZzTxTd%XFtr_ZoQ3{fDt+ECr!!=t@1hHk$lc#C4JmGGdF4mxQ?Y zBc{^Ls}R7u3T+tMiZUvYKBc6kk3}>hMEzaLxij?MpL}I)%@N9^BvsB8s7@i8g3(6* z&`>QV>{sw3HX|lcVPPL23~j)0Hdbap6j<`^-Mc?rdu$-{GSxD2W<$^l1;&A3ZW|mQ z2vPe2Bz92}au7}*aF4C6?XLg+JLUb?9qup&x*g?T ze*J2Jbdkswh?p36M`XidySuyZJ94BMX&IW0PZZJLA-aH`@!+N-GUSc1&#A0`WGTtW zjN6l>;W_Xb$~`zmY~Q|}E?6ceHWrJnTj>=^Jj{R?v-e7WD8p+YC&)rCYiQ6^R#p!{FfJQrpoH2IV-D=jgG4 zfq~sx0{D&w-mAx<%@D`GCLoOU_xuMq?oXql6tJS$uoCf&N`{6qSXfHf{Wq4#{ z0m#qc&!jd!yk>#Xw%E8YSQ&OZ&2VfDRZ{|R8f;$sm{V=mmr4>I7@or-RL-^m6wF@%e! zjNbb(9+(7LYld3Yuhky8v7w>is-|XGT%0mCDzs1A0MF;Jn;z!s=b2JKR5Q4>QMJW! z?;G*8pP$rrY}^ys@lxqSW8*`701R^Qy7LPPw%;E405I3k+RDMr9fWs+6!ZZZSP4lj zBqZb!gJ4kj3C*{zQ#z+)@k@{g6Q)XVbxF|3M8Je>6J^ot{5)Ju0s@)6U%RsJ&!!HBAFtiJKi&(C9@!(YU~ zXnuLw4En(=gx%24(2FAl)rJXKS&u4iEkBKm3z11qOiLrl>jYEWh?b4r2j!|h(JTBs%AI+b ztdNaT&k{0eV(l9m5>r$EOK#`%R$&iIyvj|8wuFTvAeZz;RT;hW?1QO~u3wN$?hsTII1>hwUT0&!E8-w=QbMwL4p=~z`q=M%O!pvHmL>yYO| z7V<*Kh5_Ol$Vv$J7^lB04iIBV8>~OR1-y(uO;`yKR%@=q(l|oe+VnU=#bsq3ot=1t ze9U+Lo)Q!k+&MqY9j<^KM);JWk~k_PMB`IfC?p^t0OiNCmD~7LsM>+~wy{C8;yyR9 zzaouMIf22#PDx3kt&V zGZ1zlX%Ywvx}*@45U1E3OJe%^3{f)kqLxS4fru;=<^m{+dCr7wI5<21hNM!r-2IB`=Mx>xNFY>* zte`}C=(6|ry2r(f+xe>g3gFD5vh}VP!s0eGHxCi|yT2=Qm#WI+_Yf&tBlA#R_oHs> zkH)!OO8N)^GRggq@BC#&R`%!Lk4PY3+C+y$iO>%hhtROFOKNIQ%v&`I`p0|UYMmS? z3RvLCFEII_pNY!oa?MA}AIEP5OQc_Z?dV0ZtRYgS+pFgW3$@+h62#Ako_V)G`5fgJl zYQqv#7xj8W2}zDj27*Mu_J;`86lPq%M@FC{m4WREz!%~hB029FvW@kejxoZv3%n5) z>fHJBpg49bq&~$y#8(}inmR5nZcv=49ORs=pu+2hG3Zgk?h#9_B8Q=+AKQ&0$}O zx9MB$19cSy!0A_C8O1l=5FlKph^h z1WBo>(z>r~oN`e@Qe9Z_K>?JA7O#MR8uB<&?yRh1pF?ZtfG%hGa&=GAwTA&!5+Z6b|)E(Ic)=Q@m*q!Yr(;Fx@zg zY`kmtZWyS5+Z_1m)|(15Q>ZeDT#fkq;Qsx6j9Rj?BpfPnHxR^8459w{XjAm_BT(Rj z;NaSyKV$LZI9r6F48m9{u!~-q>;6NBYF}N;e4Y7S6~25ZEF=uxeGXpPzI9bp)Mm30 zI_^WXaWIsH%}BuZkcjvLxJ}rkf#raI&LB86W9$pX_zSmvUUr+GAUzX&m*$|)0H?5kr9@C@JGO#*hfS2aVOQeckriL?enD)aT?wWPza6YKWh%ogJf7V zyYYAO^74u4>GjjMdrp!aIdTLQ!BdEPfoPG@PU_^}|L4*&k{?;(a`p(217Qb%!sft% z1NXwhsIeBP{J^t)!jJg??7e&$fXKCvnOOz#cbn#G-F}!n;XLdJmhp{+5fcdsIS6n@W@cu}#K2T2k(A*qM zpnRC=QQx_9=OoQTx`(jjeFhg`l)S|Dz`B~u|85Zf^M&sD)}FhV!6i(Gc?~MhCAC6c zN|-S{l1s?9;3%@|<+XJJ?{w}@mkD-6&9sc!!caGi(2&1{mlTEXsb|~(EIkLeGsy&N zd>4NHePInX_;@*(#>ph_M>Roo2{x8{Rm{!PMXrKIW%q;Q&c+PN*vgGR*9pZdOwL@$ zDY4CkZv<6@laiAU!Y*MDj(CSiQem%wpaTnH!k!QoHiRQzVc~NaAvr`>U}@o=B8Z1U zTt02|eiwnJ!Je9p_;~l&6zxBFum%(v2d`eK6dnieTqf`RNL`1Y_wz?8f&eDtffMu= z4{g8b?KNk@pR)hCQHO@^5ecJ?>GN>8Xo?qS78VvpfFh!Kb#=AZ#?m+NnvldeAdtfj zV+7QhDfWk?L=l{$;?75_CCkjM_ihnUf`G0L1G}(Qeb*S#b0Vr>_=(s-t8VDV_3-iI zG=u;G4WcgqXqujt#dhr2b~iV-1Ju-r?Ly3nd`-qI`oBbTvfr#L2;K^u3UYP7*8xQEbKn#bTEy z%RVL$^goO)B0?9b7?766fl#n(jn_>fRo%RY2#(-Cq+s9mXE`tlta4)n9~CE^QLfhL z^t4QJds|!EyU5eHK5RC@JpkT5NEe(ZwKWIf9ZMK1V6aNs+4&6eGEN~skPR|7!F&Lq zO3uQ(sLUN|a5+WAgJuuAtxdn@Tey3A659CBFk?oq3~7Yz5nS?$U{w1m{)$7+A{Ya_dE0Tz; zUrvR08s|Phig0aGG(FrBd)dh;Z+6bw&Ta&W@TtV=6~hkIkgKOhCnm(wRay*@Bw-ze z0<*?A7q9;vx==VGcVic_n|BqUkj8%qO3H3j+cp5-lduDTxdfk3Z$iRRydl_46EO@W z%QhNb-FhUnJ>=wh(OqQz_psW8%>h>5&%zQmHQ3-=y%(<4HzS@AWgMJl4+OIP-Kxw$ zr)isA#S)E9HqFCYmK{lemq6J+e(1tBf@IO+^EcF@sQq?7Dtq{l{Del<(f^r^5`q5H zCyY`yveIzcOz#&8izP~a${}&h!-o%RRe7HW)j+f9R9NpE{k1b7FYh#ftQd@ja`t_+ z4os^y?1Rk@Qqzvxq*a>G=|7@BJ^AtHFh>O_CGdURe@|7dPmAFM@uy3SO-=Ef5!AH$ z!>$xn1FufxP{Mv2xsSQ?Pzt;NrQ-8~FY4(X0r1?px#7@ZE!ApVGV$2^>8oQm0g4afB-0HEYWP z9dP9hgz*ckQYhY`^XoC0TSu6`gF4wN>i*UeFV5#Tw~5w4X=y1c|9#%Yg}L*8{+Mp8 zEx|O_LzR{+Qc=dj^`WvYF zkm)qO3PtIK!GQt78vr!eh>7QTN<=9`pi9<-nR>&6Be?Wu>GZ zbuFhO--BC8teu;k?SmoAVF9ueFkisJzP0PE+`efbeg+>2s}3rDkEmwT@rj8kp(hAa z@FgkV_VSWb zP+*upa?#+%tH+NYx6$dUa9u_rqb7rLWMFjEU2k|ko`H&r3Xw-7$z^6>Kp@2wX1vY3 zNy6vO#o#>|)!l}7A|gFlG3lE(Q)e$#Fc=sZ48yGF#+f@(Qhs}sPt)xcw^Wt&@+$wC z__cQ{58n_k4lFoAfNrt-f?sks6v4NnsL{zUHfQhg(dhW;X;Iv4lft*pk#~M zN=&yG)d6q#v_)HiQbbsxR<2Vz4ZMi@$hq4V@A`T&T!1Pg!h;zjpcw}@I4NSYuH;B_S>D@(9Ws;c`? z%VH@5Q6Aji_!0n?5V;j~9~sJMxM=cKEFQp*KVmwsld^YyX=$ji$JPXEzw`TwsKZ1I zrSM(EjfIg`v;Oz~tZU^w#YSu>4a%aufG}Qo`4R;;De3-}L2>p%saj1Ffymv@M4Q0h z!L(ZLj_CEx^F7O;0s*Gf-!EJBO--FQ4<}qD5%pxopA7JeY+ zY4ll{nS%hlA6k|Gy!P+?f~gF2(avyQ%D74y?b!&DmX~F-E{kai*E>)%GM?}gzkW2J z`L`a3P4Mp{t?_9v0s}!vm-X>6>f8-C1kj8O@G>jx{rSe|!<$y~g`!FE*T@GwD8Qq(pDSWP zH=($sghC|#h5$X8{|=xGU0q$jggiKE!D1}LsN+l5`~}M6zkxpU zTD$hPc|Y7!tgJGB`g;~HQ{8?ou1TvlTJ@MASO%yDMM>S8DKJYx=R5vvpc+5rxoB?M z8rR(aAyE3yU2yIwhhFd-?EnOUfq4ixl6&{=?b*9G-zhcEVhGF7TJ1~HKCwicDTEOG zG?Dvlx*TOlcJJP_M(>q%mjWsSBGsXSTB|6sd6-bxKQiK`#1n$tbm77U@lpwZ2hsGx zkJdt0aBV@6#~m1pvD$w7XJ;57lU!X_H)3|{)|x1aRj|)L67#R&EV?>nzkT~uQ{Ure zO_j}Qxc>YJF~9V-b6kTv{5@<(n{pyvzve}o3-!~taO)LQuslo-hJMM($+6^;caRAu zld51gihRi6AfB(29m{8VCnhE)AUMj++4*n7*AAQ~41vstF0|vhbak6h>NLVY6qMP; z%ud1_JJ)^AxXj!R6*%bq`lhCX8P1?qF_M#8meRGOcsF%rS!t{2FrN2ZeDk2eMt4H}%hr)w z?(Vv$kHjdlGlu#Wm6XuODw_$DT;=Yhun5IT0LmnhNNx{}isH`iVBM&+VLpsl2Csfl z_XK^1^J%2HBgfh&TufPt9izCW%3?la#E5zO!O>3KA$*o`B_nRdOC6a zNKWM%8Fm`lJyMM5dN2wU4XZTV@sn7+WL%&#<)|6VCg|tu`{6!Sr&WR5`3z1xQ^Lb; z_I>a@UecgPQcFea5rA8M#}I%0&^+U7FJM3(p(~M!?7ZyXe0lC0rjR%wxuTPUHq(Qn z$MU+p^ja|x0Jwi9I5-%*#C|5G5R?EEL-bR8+jPn8>}K{`H61y8IN;!y zj&is#p%o){co-}`N|eE=sbKXCsZ@1zA1+?M&PWj7>hko#GDh>7oCk4zB-cSQkioNy ziwC)WZenb~aeb-?e9N-#a@nRR;iE^O*1gw<#yuSz9T_yH;Sc6D+XmM;&nRyIWrO$sgRn zDy=w_m=lpL1|ah%zl*pG%H#OaqdstVy0TJYd4>Tn#?OUQOe`AKN0iKtB4wuAwlQ?d zOI^Hrp$R@oO(1u2#3+7H1+%ICVk;E5CWMgqI`T?RQIVTm{Hsj*>1_pCCtXGV$h9V-%2gWSB5Sdu`QZVq%ih_)OU1GaG^+I>!W1 z0cy9hns7NH2iILNKKSPu>hwE8yeo0qjS=qDJ(g^x?785_QaPCAEEe5>) z5A|7C?2ozzx*yCXY$|sn_+wt;hjN>;q9QQ>h}>ZhvZkF#*E-o6`D*C%0Ky0ZwY&f6 zHoK#tqadG9E{}(q{3`-{7MQ(=f{s8hmG&Rnd$~3&k=_u} z`M~<%FT=fN{B-vA_Qk1!JVj)P$&#$_0h4+a((gzSrO3YMS&@DqS(WqkgnbNV41or8 zOPxvKCkF;f#BKU88eNBE-FL>@e8K0sp50Gb*n6$QYhx=jT2|L z-FH?}NId{(iB^I?>0g}@pmQD!rQl%?(9_H7>c;G8!%N2gqlUR3K$wWN7&gcYMRvRx zduQi^qNiXO+?8WpVgT}s=2s%BpNyl3JYYsf0(Eh^Q#3QIloCXAF{{xq+ABYwIi| zp43k%4fA;+zF1}9-@z@VTU8B~gyX6YkM9>{dPOF>!IcpO~1?$i^^5Fek@N ztPDDbk_n6GJp-K&9AFP?*!|+2_5N7RWdM6{e$IMN#(PNsJzYIc;QA{3^f4|ji~5+QWGF|GdKNqZGH;m{7o9;76d#cM=EI8W-v+!DG7Xg4r2!Z9p? z6Nt9QC2qTf8G!|i)2U%L3bY}bbWz1GG&MAWjCXw?Er!=E+?SnJ=Wk-`07eL4!%HTr z5?3Rr-MR%wT1-pGd$qcHMBzYT7O{`GFASc%NK^#jj_wY+83cU~+e}mgm_Fo1QZKM# zMGGP&h3#vOK`RkUaeU#4jzUIFORFB`r}qZm5Xrzu@`(y*2Uk~iQPFPR=uWgY;t=x*mld+!Dj3MK`c!&nUlhPTkV-}~xF zflf9m|M~ewzB~;0{qt_0L+dc>>daOMLiE1FhffN90nor$+=Y~{jg|%T__w~qEf|1( z_X)FNOzcf~s}O^ipyVXtcaMRSE}#E&;@cNWOf77#)QO{J;%@pvjK?4o44if>@$z{& z1Il6Nywvx&W%OHFhUy;sqj!#HONxg(w+VW!m!gKzoFkf~7}2}|7O|Xo=7@-`qZ60d z!@8x7sM3CJ978OOIqMPx4lQUC7@Yy6@RN*$*RPd#33h+7rwqVtJy?F0t)pF5oHo_W zg_o}(6t6GOFu%1!3yf%6f@o3i)6Gtte5}Y0r_s6LXgjk0c0>MIm`;b|yIp_nJkaVt zJ8R#sLJmB3_D1JfG^$Vp%v5bvs+OX05R1YLfdPCEb-_;zNUrbeR}EtR$kVfQ#SI-oaJF~; z4l;}0+5Edae;EBr;?kS_FL9Y0*(Sz*Ffps8N^ZFOWdKI6M_E{Q zj7EnMC--`xHGc4>&+ea=1&+980)wjWV1w^}T}Dc37csXB)Jh#+5P5lB_CY0X%|dn5 z(_Dtci#3@t5}slPyYmtJn;&}?t`$S~c!9taJ3n(`R)iSBw#O}47_mUeMl5;A@V-|V zFj11)TqwE(%r=OG-iNat-Q8R)#x{45j|gTFx6t6;HuvF3Gk07DRojnl{X39|wVOA| zXY;0r9t0kO$~_QCg1BA}&c#i(ie+VGQX&OxQlX6N122vSxvJJ@Tt(Bo-RxDZaX&gZePAt&FC6FKK!2ke1T`@* zf&U;L1>s@`SY0a;cSu#Dd9;uwzVYDiUoX5JR1@fNK0pz90KWx>7F9crGOunifx84p zzl-5oCb+yBt1K)mu!@MhKo7@XW;2MWy{1Q^MogXq_mxX&dOTcbU>_twQVu298a zy*tr4CuRbmRB%KD1Rz6MMGWYH4(vl|kLyLy;=&Y#)h}+4t3>mnO)(Car2&fY-~4nC zPy;uvu_KS65mR<)*%Zn6H!%!>{t%*SB8m(&bYSF6Ozz_=keZp9A;A(;ba-5FK$yUj zS5zc^CEV+g_bwDcTBXihh&ajR>9~fgk<8Ig%eGUjV5T|qvb^yvOgSW5Ppwp z;Za`Rf{%+$qin{v2_$g5*ZxaF0OACK63bpdfi<_ZB#q|O(5FNCcQ#xwtJ4VD2;3@d zDm^^?L6H`Uh1v`qu>r8iL#>R026`6a3IbYv_q&Ml{OIe*1T#n@+X3Ukiydyu7$z8i z?ts~$&tEdJuYn_;jfY3+U^Y>^;taUb?0E3xSXcw`GO!~syntzTdB7jyN+A-;!$-Ne zNKvzbaK*hzoVafC{Wyk^jGy!BaSy)%?S|=j4$!FwGhS7$jh{eGT#SwdW@shxH4s6J zWD0e%iH;*N1<|~UmL#?&3$xekcAR!%BFq364-&0cKfhfNnGgUHCk>v@?L9pwUd`a! zlVBsE5+bBHpiD7CgE`+O9O3t%OKTavj5%XG+F`Qm6h0b>8@<#gxEu%_5X3|Db zMi5~W(D$00oCFR)d86+&Cr$j9m^z1DenNr|U;?g0@&RgxGAmey)p7;Li~vGE9R=g; zuAu~Z0XKe?^GgS3$`>f{3i2`Qgfoo(zA^dn=sA8|B!=;YlG19xOFMR zGl7qP2q=i?j}oW>lQ4KD02d(7y8wCz$^Ju&15YZ}VFkQ};c|Zb*j7U1w(|1wVl?Vj z5ZEw0@y>#To42Wbp_<|fRIoFmnE1$VjjQ(1QZX8kX$8+a3M)9kYa+A zpkmluWV8zcgB)U9AmuO!x`YhOejcj*5BB|mA2eLvst=YH?>8)hY&GLZHZ z%M4+pp@U$@KN&X5bADHbp7LjLJHuxX7X?Nh8ChieN*e?$f3U+@q#E%gezejChlv|C zB_74hBPi-cRcY*Por>stzTVOMQx328K?n+?XgT-Qv(BzMZHVo!-A zZDv3fX&oWn1esFs9y~JKJ67?Fyd4g`o(-p%dGO(QPfprEdGws9n{!)C(oiP^K`mkk zn`k86OOAV0MEYZ9^3#m9xWqHN(ei8d>KwOK`)vEFu_3>`>0Vq|VTAs~$!|g&Y95Xp zMq?Kfi9nyGz||j9*70O~w?zTftlEf1c77lru_vKU`M!D#1`ZRG+O2 zZqf6{_K}1{wF!`a@aetH#Br`i%GIq7Iq=qqWob=yWRkXM( z^+5y?@jQ4&0AZqIkN-L|$BB?H<`IgSKSFf^F}HWHn|%D_iK3%wkLV+UHfnwM?&rC= z(44)rKG{Q2iP=J;7PK{=nV_tP&`~m%2BkeIG~9r=2(^)nzz3Pqi|)*GxKx8uL{6j# z!`yPFnsrMI5^`?wBU5$|f7NN@kQBmmmNqu0mNX5~a&V1arnL1ox{Zs=r_Ss**&Gz) z9~kJ#-jj)?#m47-I^ATleK1kFLcJ`ubKA1R2{F4vvE5 zjVUJ7wrN|IHVoeS#?VmwvKO##@tEcr8`~YNmFVd)aL#%`*5Hh|-Mg8Y7u$X7PsmX) zoVV-hWD6I%7aUhcHt&pzntzD)QEYy4C1Bwa8=FWE?_|wv5WnEy&m6Z>G0-YDn$>)2 zXkefaE|R)#QI=9)Uq6f+u0?ycq%Z`{q_eY=*L|^nB9)5G)V$O|PGGQf-}n^f zzh-hAYBP=+2Zlfh$ST#R>FN4O$;rmBU);rSzxj!uH57^GJM=479vl}|nlKS9EEqmw zy6jW72KIEyzyKLUcdqHpfxa&&@C@2tTvql4)1dGsv?*mnEZQ?z@1ui0_lJU9? z!ZM7^j=`#ojE)Ku>%-jyENdR%Vo38?Lc(PF!OK0d zZ%{*%>u`!hZ-JGSqSbas@=Xse8O@ubazZ#FDbZvh_*x^QHCwhAaSwUr`wzqHNyir5 z*VVY~Ox<+xJmms8iC!`29xqyHG@1?Q_6QWLBy%p}H!Hq`CdBB*RYCWU;3SL7(x%~C zpc03h-ynpMXVim(>nTlDDivy}8s-m&YOszu+8JvlPsVlLFlp7`q0}WQHi=|HXtZ=q zUr!Gyj%m6g4icS0E3REf5yP6wpT`-dgPlhN$FO!c1_iBRF>@06=O@13(2D5P*5=0= zpiRSKrx!Fk>*ei+qbE+-h_g}t-418Q2Cl)7V;u`}>(Z6+C2h_1{?ecBM!tq)r`}4V z7@xUzhqJoBKLdT4>ns+sk&){!mFyDTbSe!eKg5I;F_XV^g%_3ciqrw>wXa!X5HN+)x%~^sZCZencHx2%ng8b#?ujtcTwy zvL2C6n|WX84@R?uEOgiB`db!!0{9}OUKvy{r0xB_ZLaCfBT?(UH;wJ zw$4?Azv)Bm{-W=>Y^3Aqqtedm5(&@GFxlR#pz?0!T z{+{c(p6h!4eCn!ne0Jiz(Qj)yP-`AfYp0Q#7eNC?A zF&p`R-xgtu`R~e#D@wS3fBcc(MI~02zh8DcDpH*L@1}QN5dORW7X6F=ueS^trgeMy zii578am{K}Sa2_><;?wOWW}}XtIAQBRVQ%V71X4{Tk5P-I!ASP^7ISI>o++LpJ&zM z3Rz-09=sE2qAKBQdVEeMzwM;T;uWdXZ;j%*75e6+aEI1ARZ|64>*^H$o~|C7Df8J- zc(DuV)(I1VY&MSSjO34vxQ0u=(BHr1cP<$yv2K!$_qU65xk~K2mQS0rKPViXQNw*7 z#o;(9=DfCOndDWj)@-#N(B`z&xraJv`MawVd^8?{WZUz80v(A_89Vn{R|4^mdle#Z zgGhp0F5lJENJf1Qe-m+-^CpK|>-s)_>_Mez3c?uOsS*my(|6V^uemmE==6iW)M;$R zs|o8{I(~9MZ#n#Xxi~(-*#f8R| z_=XtVj7LEv5mKD86_L;8WIL8w);*? zN*dW-tv_+EcbsT-jQ!*6IyD$lmq~S3HmNun^I+XpsP%p<0sl&T&!Hz{0;*yrqvb3+ zJ<~f1t;F3D#PX6mXus9OM)%vo!aA}K9;$Ylz97Wf)BR%HY?d=Sm9;v@Eo?AZtXW=H z^i5p**X3gL&18jCht~MseDa;!gp!|x(FAD)h&=K?jk^!CLK68e75#lz%6Ag_M)uzN zFV~zui%Zp&F2la)jtJvr_sVuuo7~>?_rhi(kQ&;ZBd}z8yeOp*(E7f>JQ3j$A7=5& zi}_fPIH_?=DBD!=L22E&Psy*}dR}ZUGz)%qmpeA1H-CiEV}y2lm>E4xyFEWFjJ06F z89d?|IK?}@w{h~{aXObwu(et_FX`s?1;BAvi<39hYQNQXd{9<@O~>dZH{U!`IsBmw z+o#n&t^u`|VMF0_i_eHUaVj3(HNT25cd!0DWs{sgK)*N6Gi}5->g1}2tF7ZN@TL8v zihY#KRY?% zwbL&u$vdh~Ft%0-S1(BDtj5hJA_DA5`Y*mM_ODUklfN{$|8aKAydryMmOj(Edb zS&v=$qR8VR|IjT@1OB0=%T2*=i}~%nk=va_b9SGUT)*AK zFCSS}Ac!eEO-s{cp68lV50hf@g9h#&H5mvbndIotOnlind zM?n?qQ{j57u+VdBlVW-sH2a5xyM|%9&2h!tI||b~dw#b?_^Gk}1_NSbwicJaFyZ{W zScmL;@DyJg#dy-_SB*2ZYU3^F63KNQO%TFj-^r)79J0)m49|lo%=E&GY>anT$Rs3 zirD*J9DhIeo5<_ncYShL_`CEQl;ne|cW>XK4lI_m|Jx9{OTHA7F*08~hg>E%Zb`a} zsP`$mEWpHQZNuH-ksN+ClaSY0yTPaZ(?I@0xk-4baz|PWp(|sYV8&+dTE&ON`~NnK zaH}$Dl#={t@yJmp#dn<>H|JG_?!$RS4s2a^^N`y9jW#7u*Rp=Yx&-^zE3ZGB_gm>< z530!LQ~c|dZAm&;G<2vbHK;!6O}%vFBU-}GBuGAo6Q$8%pMA^iWg@-ki@!S~nrbCz za>dNvijE`iB*f-{1#FB82?5>umNNUljgb&^M9FmIjnZy=Y==B|+`F**8q>SPjBGim z7p%FTBK^HMGZM8~%~2#?u4NwzY3q2-ts3KW3A>AnaNwUDuQ)L;MutpMEB<{j36T<^ zXHOcG&`0j8+>Vp)d7DBY~8x({_T!w|BeAHIn9E*`HFMB)Y|M7C+WhpCL#oRu^7)EHw{ z8U3-Ild}1?^yS6vlJpueli$3_`qS?`P$A*QMKKny*p7lkh0&i-9A0b`;nA6t!@ua@ zrQ@A}kc7m??o6G$DiX{E_Lg+`nQGB$g^}fJqM2$+_?Yb4nr1sXK3-zLX}CPj@}VvR2Z!*xm52u_w&p3cD78lq@m?ipXTPQt~E3TA?~Z!uPuFi zQuKKhSjZ}w+x2P<^YwW>R=={J9xqF8G^;yqGRMfn8$L};^jx11#16-DsYf!2O7{ih zMaq8s`0-U%R>+TFe+j0;qoYgo^p<{pbcLe_@*78{<2j)rAz2K!Tq^78KK}g4F<_JA zQtxOsG(1d}mzPION_s?4C~OXu`7WBcJ)#jA{r?!~Qnk4D1iF5_mO__L$Xv+Ju~Q&|t^ZeMV@ z*f8h6d|73*TUhltr2Qm#dwcu%mtOnAc$raYdHGkA&xr5<)3a!~D{n@)m_zojFeMV` z=xhbFCaT~%)Y7M?rw5%LEFYuhPWke+?Df7p)iI!>qhm=^`7;~1K~9W+iIK7UyIam0 zvYiQ0VHW!4DBuWH&wPBW5+pU(c0?52l_B9z5q5fdYO97zACG?!l;wyh-|FMcc3T_g zgOj(fN4_hLo!j7~BlbAncOT3Yzt=p&K_9=ccf@>PdLC2yC1SomN@UJl>k_uG$WzX3 zIl{H1j*{<=EWh?Hx`pAp1_ySnLaJqL&$M#d1>g6)`8>^P#8QIv7H<*veq8b;cyF_G z_UHQ9i|^kpU;&kqp1}uy9(tK@GLpbi$qS`*FHN~fiuwOx8?k#C6LbD_H1trNjpF=^ zKDO^}etJCp-@Z}V4rQ@3B~lXNhM^4BgQV0sk}RyOBF#>I>9rjFZpMw2wXB*bm)(LG zNJ>iD+1(X_7$~s!61!YiS2r;+k)zGMwOW|1Re@MtUG-lNn{wb33q6&<>~=^y^QbWF zL5(4wP#W>t+S+u3d!#~Mr;%%GPxD4w+}y?-T4K)I%*e>d$;IVQiP=dk^XXb&(ffP0@dtY1IlYP3w31`_r938-ItC z{_$t&3nubJ8A%4Sr4tcrwYGB>fptMPHji)E4!ZrGzU^gFIO*i^{R7_Vey_C0*=aB) zS>$q`Nv@po>s}ZxdQiCEb3H^>f~i-4xY>IpG4%Lq`@w)~L+F_lqZ-7 z`~JaD=-`hZY?w7Pn-q_6>s6R8#a(Zk|D4S8cMX|pgISVsL(52Sv$<3YI=JZP^76s{ zVxs-c^ha`63JMDk;0zSGlP%57!&ErBPLB34-#yqFRfn^cXC;SlqluebyCAHJ)VWeM zE~G!kverJnW~jqh!u@`GdlU7VxP;=pCtvSig;8SlX)iyo8t(?!FKjpi(`Dp zV4ACr4l1gLKlMUAG9HQgxxcmOgEqIfkMU^iFEgqi7LM=h>oZBL$A7L?NC}sGSz={M zydfc>B9(3H@9&Rb*-`q9px))gvX7&ql{k`oEdBO= zoA8^qC{|C(j|zWY5`vPqwdFpTar+MI^U672Ohs!k9zWkbE_ya=n4CPoZ4`x{`g_Su zOy|%t>rSigyaoUve%ZKHqW1P0*}4f_7A6w60ErKcMO?NVZ~TnsT<5Dj3o|lH4aQsh zI%aoM;^Qy2ogO+WC@8GYZY#1`hR!JCmdbx{xC>9CqNesjt?*#Idgz^Q$#kPDP6NohK^3p;uaZ6^w!bIZmrbn3APWbIr~kQ`g9qVFvJ>rn#xp}q@$zb+fjSg z=by@Vb9!eGA3x%^pPg(J>6Y{l43Nk5?9L$eG`W-Y4Gk@wo#W+lWn5jaGY9XGM@B~a zSYjiov&{t*8;uJ_>^9K|v+aRfbMSVz;iNEGp4xuQzR`Xr#i^Fjf%3sYmRYnPFI~Sx zM>kI(y?+j?O8+UOQvUr^u4HA%ILj(J$j|4R+1MXzIdL-;V6jp6ty*s(#<{YxAsX_?vQ!Yq*v=I)eQuk#abxNv*iGJ)#{2i3~>4MGw1Z zAKKhI$%OjLc)V{*yu{`Py?VP`)DrJ0?jDSwBSRU|2RPvT71`vV7>}+>Z1qc+*RvND zE9YwD81rvT#oKy%Ch3;knKuk_ey%}94i&r=w-PJJj`(7P09`?D%j~_R< zFXA^ECn)4u)Qcy7dH8;k7ut-E`0(MO7!MDRaJnk&EMx+sxX)9kn_uD?Zfyp%K21n~ z1`M^!$;k=Rr~?85UZtg7AtRi36V*31{;`Wf^78Tuz&b7HVDr}<>|=Wui7#IK?$HRR z1u67_;Ky7##M^#5d2ipEXOx5OtY}(B?u=3a* z)@0=<-I7|z^dZjg=l~@)ikDv=!mDEr-iWZ<*+WTcg$5=jOR(Hm1xh7jmchZ9`T6XC zTqLFT=W!J7-OK)BX&l`Z4n>5FaDqBaE_d@=Ovm66ce3))@3o`%6qRa)cetqD*WV6? zafh7)Ld&uWGP@h)lV?v4*iSCE1NL#ihEhjIsc_U<=#pU>ru!BZT?;HYgSUOFbq#P8 z{Fp+X?9KG0+15bl1~xV}7=Q}BV($3oE#n5aaDaNhXPTquHVO+1qXYMTV*rwXghc<* zM*Gmtj&PB7>;+o>P2}8Z3Vd_YQTs*T+u(SV4?|*a7l4k9mhFUA8p{p(gsu+)xsG8L zG6>wIYcH&~em&5^yL(P`=5)L1g(x}8)@E@fH_oJ1(t(b2ead}aSMA1jLyi^i`C$^* zxZzx7+`2y(`DX^IUY3V6-`ALK4tCnU611I_JXYa)abZ`NQi6#zSL2C1YiCCXR_vQ$ zhgw4XNEmbgRfX<%F%NrZS_{RDt_Jbr$gMSHq36q>UTo)Z5zV-yD&%cVbj^!plAr^E z=PVq?%Qaz3;GzEK1^AU7r%jS`2PzW=HSs zEp8vI|9A+cMuo$QGZbX68c=j~Dv9 zHk`uIThD6BV*3IHt!vFIQ*POLGo8_G9V+5d2g7yAL$`B^MVE&_V`p-yjG;e@Ye!V~-t8FQWdod{q+~m~$Fi-a^`vCgK7S@lW?UW=Y1`dQZ!W(EpjAprt$D`g z;X^zV1d7kF_v1%sj8(KMMi&C^=TkJ5-w+Ux$s1uw>YMixg9ZTEIe~1&me13qh`}|j zNQGikrH`4H25g{+@4m6<5`eOor_G(qMT8ERnT_?fO}<7G=MGpL!j$gvHA8q0wt?BIAV~4tCAib+>XLe9VN7g!J7#{3qu>zVGf^Q(pK!_Lb5@ zNSuZvv&3`NrBlkxJb+EUY=nJYkaWf{{LXX2brS~jOjWX?x2a2+}6#6ncJq? zekdfp!2UW_$ZO?0wT2IagJ+loq&Jv++UJHKTA31irIN0|o^1Is1(ZS9G&bIbXnC&q z$W7F#U5}EI5-`X6_wOy4r)gs3MPV3dDhG_BCYJd#-L(w^J78C#wZZVMTY?t`&;`mx zbhE)OhaAc$-LOoda8wrI-C}zEyW7OwN8ab&RZuF2W>vLo#T8oQN@2-h5*!EVm(gv5&0qBP$Hr65Iag2nMEnXAehF zJ}8=JUSg^eOwi3>O!K|SR%Uu+Ez}3q#B{?)un36)jzHhUq?VS_)yau0vDX)6xWk69 zKDk_{&caVcL`8%L=%=`tB3@zd;*Y)Ap}gNb&lM$h+L*jAr}SR*D8ubrLr)Pr&L6dp z=wyNEdA53P%2gV6>ej7$jm8{IiO7=^07d|PVD3kuq(xL0<8RH*cgIkEa}eP-Zjj+f z`VLe6tAf7yd6pP?OFf2^+D}s`pN*#7)|fC0Iv3ShdaY8VsXT^V>+dJNbLUPoe#m&oa6n6OF?16^yU@F#hC#(AiHb{r&s!P#1xm2C<=zBm&)uOjf^nx8MlXAE*$NH z@&;E`&sD<$$4c~2POA1?~4b}f03 z6GbUeOP9Z!1BHMTiy4z%)*71VS{JcYzw z`WV}Z*kflh)!C|y9$nOY%@-LzJy?tFGU#g=?NHVc&Vik1$S^;wlR<+1REJ~We6!J* zB)Uts9+|IO;=cV;mEWulGeo>YtijwASeTZH)9=&loPC%+V1vTxNP#`~-CzQI!WC=D z?6iaqU>1Mi^qM1HA<~%Nc1X^s^%p$TLiLbTo<8qaB=QS1is0bjW)eyqpu1py{v1fC z6^^!57}dMD1KZ;=+hRY5*xA{+vr(d3qRC6sx8(6{4Q(>C;nR#7eZpKvHok^lfUlL6 z_)`}l(>qZ<82-kB&QF5r=FOWJSPO$S^cbjxqQQs4KtMsm3p5MO9=eyW1$6-%8KcJf z+}zswkv8{TXz+*VHJ}6F{Fq)7O7s!8&TZJ**+I(;?_I1SO^Z{=gHEPVGWKrF0Va>< z12)&Fh}2st$1HAUY3xrLWWGAH`&jKKB|$50Y_h85%(RotsCF$T3Gw-CdBx3HXW_ff z7^R1Y2XFt!tXk}(E?I6giBx?;c~SHmY`>hC4&Sniex{s+(j}bEofG1LWn4tQzo);S zowT3bW25Ki375QbOm{6bkv1{AG7BNPt%hK;5xbq|d)`Z*sZS5G&-~|d4xt3J9Q;fN3`mjK%Pe+O&PBAi zxOfcA-15(X^ywD=Byr#0l~q+1^(g)1m%1fA7&2sav=G(0=WSB2o(us6yEyOD3}|^* z#Hbn#gV;)w%ZAWPMK&LkQQwn<_2bvxS~~zMq))dJq0Jq`fRC93cE<}KpJ!)HX9L#w z3JMCcz8qjM=p(-ardnNdL6fC=vvPM2)(`7;`+>~QjwBfESSz72xPAEI#ND6P$5q&C zb~=S+g4pttXqa&^=1-NhoBjeNzj_n~)wN4zCMN8eYDme=ar+iYNl6S5BqGui%^dP+ zVuJ3uVmH%@(aGY43m1HjNX<7E7Wa1hZ|n72cYMWs(x)mKrdnc5ozNwWyxEkd_wLp$LuAcNr+QYG~u{gC5yYhcPx1T7>B;yOd+_o_i4#vlA2I;6w1wUkJ&MvS;zr9K%l7B%y&rC z$Bn%kVq%|x&6MR%{;=xxW=%{gn>4X^Tbonq0*uNF^XtYu0xvcv9r03bh)6Zpgx}Fi zV737Cwi4}McFVqO<8C&@$#RTl<$$Xb{sn-6~g2tQyxTMT^$`I)Tj6Zk{~ z%SS8|X2gKNKvYkOc;M#V`+#54KwVSBbhqg;njqXo8DXAmK4FK^(WB9aT?QzR;5SFm zwSXz5Ac^+9Xvpks7MQPfE#b2G?k9(^9{MNhDk_`VkV`*N^>YOpTobnV`py_iV%J5Q zG%)*PZ|_0H8M=K+Cq@bnZ6`W0F`X)eGiJ`c7;*l^#zV~-y9|x)@RE3iuqcJ zCv7n5Kql=5hXCIWp<)|y0xcg_pM9_u=-+A@Nbm;2dm}A-xpF2IxUUe?aZ9@r_D_a5 zn+=z|9Oqk8kthy|7y(Jili}VdxhrzH!&f)#p`Jc}{yaFFG`j0imjwq02iAHNavc4; zOiEQO3)oMb%iBzf ztQyWWO3S}~aH|q`$v>dh*10n8O_u0KxX(26o3*=3|3X+cTBA17yjB;(QZI~Il~IH1 zVKu(#8e+=HMMQsS+yGhpeS2kJ@(|i0fe^KUqu0czv1{5>Uh$}z2sL$iI+SFCNd)Fu zpahK+$MZ8Z;`b=WmAhBkDlvX~*F3fIY*2(t(hp}rma{kM3VvXVXieALR}6y$Zipm} zTGWmA!m4*I0_B1v^$PV}tQg%`-2}>H2lzo1mCKK^_+0c?wt>Z2X9`a{T!JavA)X}t zK&;2X5n(tP>Bd4G9?;rl(J_h?*QHP{ z+xGR%*ecV?+x^ZOe35zt;SU~UjEwrk!0HxZpOkbH|9?&%KN0Nw6Djv4XVwXx ze&W+Oq2FbKy1?Q4s<@yT#-;1feAN-lgbf5qcara)#Mtsx!X>lqRkUvJTW#mdJ4;yC z8%4N`96UU8KRFaA6gqx#k;UvW*+T8S=i1KQnMWtc=qG6fN{C10_8uA~xYqQhm~VsQ zpnq+ACEjnn7JEz~btiHgm&AcV^MCu2IwklsvntcWe4AN^?R-8_(ZA#-@XhK7&$tQ_ z?1GyB^Lyt^r$ydJWEIh=uL)lj#8^n|Te+NB`FA}^u<&?hyVmwiKIbb$&{%z7t_o#^QQq}m zSJpk5KBXwhJ;}S8Jycy))j8Ab+Xyr(FmUscq*5IldpJm(OrTH^W}yS^C&+ekLR=UD zA38Y1LYf1Ui?!%x1tAJv2tc+*=*SN9#PYV9xpNh`{8ad`J)fbUz5tHhv_R+xxL+Fne!}2^Lju`iK6U z0P(sh*IkQdX^q%(a!-0Uk4{#a)CB0>jyY)ZrviTMwiGnxr|q$PezdoNSs5<2F#Sb% z^FC`KP;@{8(4^u{CIYGv7yF!6(M!qsNz*Rt^1j)U!$jYqzhq(q7SquosZmnpjVvtjtWrlEDh$~oa`E1h@0}-{TUwOi z4%JW*)lNHaDw)=+ne{vGDWAX6oww4I0Nsp=XatFjE3p5|n-hl1bYvZT^Ul0A7lb(= z5RQOe0h$X$Ghh>dsGgdl2mS#NEezo!-kDlO@;B2T`unHB#XwO2zp2mr5h#qVt}eDD zC7>8!*u+egTMKs~re7DpA@nV8V}xu#-$YR#%4OA|z?fE4S5|fac@AXmKS8n^W+h<9 zyu=1Re}BxYRfLZ~rVuxYi@#6oGy5U(WP}U?aU;1LS78 zaibjMy_*}^s4&X$c7N~`Bo2l-sjE{E%d~;CrsPy3i|c_H*5j<~nh5dk)W z9W+`{Na(}L%1T>G;lY7Fze$sPtznMb{42($KJavWD~Ts@XXhR z`F%Jtbd9ImL$kl6UYGVX)|W=~wuZ1i$8ZDf*L6Sm!ARS7@RPw@ypfwG4ps^C?HU?` zyw}%x!3L5&#Caj3hi5VW)U17W9SlMcQx19Dr``1YTgp zOAOKy6w(np%*2ne5NMU@!q@_&IV5APVLU*!dnCH0mlkZQ0Sf-}50C`&d>S98!DusI z^Y0tC1|&m`j58N_GP>FgWDmlnb%^%5=?~_!EoKn1Eqr`vMumMZ((yqJIv$lip#+f{ zn7Y%&(BnjKaxkXJ&l@aYd^&1A{blLT)x{6W z&wM}jK$I}i(@r|=K6iKj^P|tiJ-7ImmDwE12Xy!KIZQ+B9i2NnnL|2SW1J))~=kIZd`{%HGO`2k^w{|&p z>}Dp8Sg9LUH~M_6eQ9kA{ zq7Dnn?^7=LSbKm?yZrv!Yh<8^)Q`2ed;cqyVrapz%Uwe(>M{hOsHr1uhq?2mmiue3~JNU{+QuOp;-d z5vB)edlbqh@K)I&hu451I{v{aZ2@vI&~g|Yg@Gbq2tXNpguy8AR*Fw^eD9(Klm_2f z|7e)Qp|*ny1lrUFly#=i!_n*~3V9f0aO3r3i;0+z8=+@`7^~FjuZq)HI9@N42$XjdPzmw;jiH# zH$$9b$on!~pctnl7XuNN085)^3QQ|T?Sz3zOEf-%jtnFf42)qWYj_DSWLP%k-8Dlay7C5W*eoi6M3L7id_+B^qP z;u!oeHUJk_SO4nvM`(ZmX!O>|J!T9p8CuS3Z_kE6G3H0KElYybakdtEa$TGpic-~3 zfv7St2bXDKZF4;QUMBU6mR43>E~&TSI_Sne#u)t^nzuN=5C7NNg#}1fA7ej>ta~4$ zy!k)nExa+U2(>i^0tU2R=EFl+4WO%uiZhes3eXJ!O@p%z++9F*p>Tncy*w$e`Q+a1 zuNY&h{?J$>xONe~bq#);>aCNtM-?&_f%ajKSr>)3i0Pe^6q;4b8(QmnatI6hb`+mg zCuvS8ytMli_0wJP;XHdKH}>A|87rX25n-HUgk(ffJ{SD}Z5bFFgAMAdLoM_~AJUrg zH9!cz=h}xL%VXb65QMQ)oHq}|nsT}c`njcrMK~IbHo?gP%mv(0T3Xtqw`sV@Jf;@8 zxR3zR(`XFw3mDUO&(o;-&7>1e;iyveq_w~QWJp#fQi`9Ck36v#xG1d3iaj9Fpw%z> zG?Sk6pH@#9_4!d%AmYGT@mz5eqe1T;*@6X%a*+p)L+P8#W({KBd*o_T_! z3AFm+;wg8yV>pVY8l*dpq^dV;MOP?_N{t zjLAN>Te$Q}fws%eiq3h(XQhT=xIF1vO@ml=eY)n;n(Fyzlt*p`Y=FDM}1)VT`eDIA3U7m2zidR&u zcxkStsQ@|w6bnZK&Y4f3w)9zlbx{>0?s9PjN)E_)Ebw)J_6!q{1XFzlGJwyK1^o^| zTM=y=6Q{zlJ={=9GizvMBgU8jkytzL>uRjud`Z&Y7)*0gsSSQ#5usEilAjI9oSbbB zi#aCb7`NneI{#TOm;TJS@|=5(jPJUv;O#fe$S*t-U(hVL#H_SM+^+`iPNn*pw@CY( zZ})4O`8<|58nwDiACvFL3iOgdRmx@L>PF(^5#9d4>`bV#fZWM|e;3V8bNixUvfGe(k&v|Hh z+qlJ#JoU?67^P=i5Ft=?V!$Q|&dBWpEZ7uC>F?VIgi&%wh8Cr3sXcKO^_=k+N}^T( zx&dgA6y`6@d{PQ1E7;6f-|+?$HLY&MaYMtz^mNdhhRSh%3031D#F}(0(6t+T5TmGG zltlZTCwc?VPgAcxaB}mP>eXs`UqUhO7VYNufs2Hbo4)v$j1LL*kg}$iw$8cKgr9CB zy4oFYXNpn9#QTzBCz7UzR!k96(HZeu5&B3feC%`h`1o8@L?BmXLVfcD&t4!4hB;Qq z^J({mpwunFz}5Nr`L;`60o#Mf4Xd;E_8wyreh9;cS-Tp4>Q#_0Nuslh38X>#78Hac z42>lM3NAD<9_SPQ6d7O$*HL#alMsOR1q=j`oET>?$csJ|%|pxLFPf*G&^p>0g8(P3 zX$)g}t_DmO;~S?S5dhkqguUsP;N{P<84t2G2FsMx17pnQ0~HRf6CgJQt(-*U)igqV zN1U767t60xO>ztS>R@}R6Grn8R?J7-cE=LFs57cDIU71%Lz~Xk%#AaW6BpBo9Y1en83lo5b6*wPvS`a!nof}Z9ay|@aT(KWqLywF?_L}F&;}!hDfD0BN zm`1~5HDwUcRW>j@5+Q$|!r>>C47%&{cdalCR*`aNYhk6!ua zq-{1#j}Se83qc6MFm66R<5?$OpvcgONb#(}hsP}P$4L1TKzERbjB<8HRk;gFNhK5) z!}0f7|H*a7)D|#rONXB3x?7-^H_9j^oQ?lu889NQZFaCizE`msjE10Rcwd9(Je>4J`@dGY(;Jipt* zce{p7=!(_ug{+x2L0DsiIe-rSdo#X(V&4;pi=XAHfpD}_D4h=)$&EGBjZ>(3oLf>jUSND?nAg$13?+FxWx+0p7)TE*K%zy?V4B zC=E8$q0m#$K#dedw)M^&(E!+JKghgnN8|p%K`9;M$`{cA2c4uLU@Nf(jZKEdyj^e0F9U4O&r~>I(t-(cQk?eu z7B_r=N2Ybw;wmCXe@5zsfu^wVOjNO$hGv4ihzy>}!=v^@ODuvBqSEz-a?VzKbm!tO z|7YEP*X*vQ`@Uj!8-z}F3w4DLP3AGW7@wH)O(|yd-Vs=_x4CN&TEMId);0Esx%x4nd z&(_3IK*nE%>3DYNDhUA(KbT*)Gw1ed+$~P{+ut?73<_M4-`p}Nn;>|pGtwpop0t*7 zf5=;a#|2ku*8agGQ3t3w8qPi-ne^$^C0dW`T;_*B z^ux-->W60@?i2d%TMd)}X1FmIu4J`CTlki4a+63^c?^|g>aBD4f$=??R2EpEN7^F^ zB8!TMHe27cI*YHr*&w=6y5L%)zsxnJ;&73&E9{Qh zlz(M9SusiM{|rMeobMLws&}8@@mM% zdMjU0`G!SQxxJHu2CPFy=Y6Jf#wW7whK^)%-XTwBR&k(m;Mis$D!Ch}d#)*wOLeDL zTaBc6C%T^Ui30Mi22Lu;x%$uG4KwRc-`S_;IMSxV`{k@ZEA-0o^;3E6+4Qdy#jmCf zqH80R3i`iXkNPXKv?#EVi(;9k%1OKvx{!KXEMi<=9}Csj8llzovz5DQB4A&a`}I-j;WOG0bC$Q8k>}$(-zj&O zJS+JS(8=;cb0v|hrkVbhE^)veBF@L!+B&&k1?+3ThkSf1{bn=rTKp1Uq{HajDdA5# z{r7bL`>^>QXH3Txx9iSs7FG9KYbCjT#i?7==$^hK?xBLa6fsec?w?HtZqJFnUw7JW zB!7OCW$jVuldN2;r1RTd59!t%zJY^J*Rd$wc{Xp3zdefec*Z$&#>H31TTjU(U74px zvTypvRrs|qd+^;xi;bP%P?@dg2rEKfkkupR>C?bL_|4|nS>4}4cBuX}{MA(r3X^o^ zh)_kYx3BB&A_V~ynqsP}HM ztEvlL&I-pTiR?5Npd)_l8F3z0?c9Cu^h1@{|M3F!wTCbayHRs-y~SH9?U^%)E*ZU< zWY8%$`DRp4+lb;?=X+d1BXUpNB2jC~oR=(5ayED#k;--tVw;_BRiD;+n-p*)bN=`W zF%BVi=Vn-#3}F->uF^g7YP=8F?9@bMT-KD&?q8rc>FMG=ZeZFPEnuUh?Wgielo=X3 z7c2Npkd7E17AJc7mCQU93p?>uy2k;68~9|~)TTKzzAq`7H2rFx@Y~{fvd0~~=rO(| z2wh?EjC>;;k_0oNoYLd&}O6b&{;hATNQC|jf*JzcN95r6WeC+oVE z(7#m)WDs+ZiQ?(gbn}K=Np2%k^9s!Kz0RvsUHismB>kKcZnO10qG!NAPFp4CUB#ibL7}uHuE1ddd~KTZG;@3KTUL!NVa@vbnEH`@~Rn4@!!OYi%VK$zPv7G zmHW|qmz$V)$EfP^KXK%~s8{U4OT+!~z%Y4*n2MH@FU0>Y0p47N@cvjEdGXi-xLh?`!eQFZePZdm`o9FYPwzjlDt2e z?<+}vD&7MlY-B`V>3rVZI38+-UJBkZHXiR7tMeY$6nnq_dRe=u@zTpn;92aofvT!h6e-l7?;V_aiTaO?sAMqqG(9ApZYGT0ADw zD#xBnlqw&~q~ftJJX2aR4J&H7%YMBfxbx@f;~!foKkQZiG&9Qyoe#(BPjUKV|B8>0 zU)50i4_P(!_=FA@2Uk8HTP(W3y6oDd54Dh1jj?Q(j>Hs2=`Ctxxg{OfONt+(ZX^2_9@%>{WfrLfi;gt*TJWY}#-_oxT5SY6hb z!m~(Ea#>fogp}eu@TFRRJTp#$Lh${v)o$3oH;(-7Wr-dBAZ%^TW=?_hji%TPwvc=J zWW<6cUcoj^{!^CpVf10>Syuc**yX0Tg(Zr9v2E9%7zrfTpKBdXr8t%$&#hGWEuVsA zD;$^el7?4}(2yTDH%pb}(Fe3eHD6yhE~WAd^$Svs=HKyRd%Zga7;oV){Fh6E^q5?F zODXW6Qt=~|mVIINBOF>D8i?N_Ixm7)i+5`$4+PzuWJ@aiw38v+OD^`wOJ#%lwX(v$WBquVjy*$ra7F3=zs|8hu8_ z8QR80isMf2d{x;n6&WJZjYcrM;~Gmk8Sl&5eCWTo$QQ;E0@tVw5e|=`0_BFi6SgA^bhRu>@6tCf?v%A&D50|-k zk0!T&9aWBN@8A=ipKdeXo-OT>>s$Ez@-CUO*hy=6NBGIPybx>=A9A6sf0>Kl6@xj3 zj-bf}W*5=@+$jStBN}T^GC7fY|LTgzVA-`8D>eU!Jq^CD{($?Q;=tF1zC zo^XaOONFX738Ieo%Ze^bLj%$v7`JH~6?#uZ&;ZHJ8 zdZ8|+)MZP5wz*$58E|^QR1;i6`={$ybv81%NBQJnxwfXvw4wdGlchn^t)BNaR%1!U zKa|EY%pyP1i1}L>EeCcwmQ&)qTI+QCqkqrUQnXgFM}P|VIm?rsq29*$PnjnQ1I-*) zEc_y0R!5N1I?;Tg5kyW;69-;f#%1m|;!VL5JX%4`>GXSBhK-;r|2~KXRrn2+w=4m8 zSIdx$bjxfR6rzwVD`?M2+j<0Do^sff7?nhjaqx~ z0BtUMTZIKG!Yo(*)0~&&#@Qmt(U0q+ZyIPQ7CrItC*u+Mo%eN8_X~EQc9QdCMAGNZ z*K6J80vY&!e~S3=4fRm65ufN%#9^sR<9;4h^IBBROqd%LWBtNoQs2d4Virc_Z3`$Y4)HU^zF3?TN_cF@BVznrttP4@(dh+T|7ZLgw*Li9P( zW1ipdQHzqp{0wLKKOEv_bBX7iFZ0d)g6e67uZ z)7Ca;{=c|-3!p5!sNb8G?vzHlq@^3Bq@}x2y1TnOq@+cpySrPE?huid?z3;7_dRpI zZ-yC1y}7S_?Oc1U|N5=GL>}Y-@X*G856${CKx9VQg-uKDtxk9i!{sGhw(N5({PFK{ zr`rBi7$T)>+hPc2A@*bon~=yx_fmYWI)8A&k}_h?SNZ1nE`@%DA`VQcNQ%VCOF zN?dV75}l0&{Jq@*Vc+$Eg}GbLM}%JG|NE$2wI15sh>UJwi{PWw$&u92#}srCVViTy z@=-&=1O@&bPWE54Gt3HR^m|)g6QAnY4b4W)%*wzgTMICu=%F<%oJv%P+*=m6Tj^Ln z+W09>_Bsp~ksxOf&J+Cgb}o|U@D*nMm}7f&<6GLU%INCC>Q(YiwXK_FY%FR zH^Mqga*PVAR?+P3%?A$+&S9N$&~FVl=a2kucERFo^nn&MUAMh%~cv?sgURpUL2<%FTcbhVVm_(@-VU}B8-cGiDAggfIDib>J z;AKtfr$$L)2vj;-6F<&2K#UW)j+^s*xHsUjYL$R`+Du}=FX33Yow@YjAyh!dL4@1~ zhFwSZ_ESfLQqZq+8Zt7Z9nQrAiL#IfRCC&v(>?qT5|W>IVtCn(u#+FD#Y zN=5`VHpbpOiY*6KBncm14e~u5Gmkpx=XypQp@q&HeTTE273%O=*qXfl-xI`Pmdl~bE7TDpZ{g;DIZ>P7XCym)gblDjj_L9h8{ z#WDFpg`?X}-lW}LIe5nY9h8))HA~+fFfrDJtg@$Sz3-#FU^+aez_%8OsF`v`2raAW z2)mK-V2Phj|J*87X+aHb1jpJM%r?Bw3b?XfOs49y(V1+Z!Oua3=1sWDyhSm6vBfSn z_G5;v9tXi4+F2+-1z87N!l=ym`<*$^eJSjH+P|O|Ee(4{oSkFx8xB!ZBn56NOb2pmfO%oekaNSy{|VoA9Zgr*vRa2Xo~FUK+21VatCYw0y7jdTA$qCWPdM5 zMw|y;7MTi3|I2TTzMMTfwl>z+Dlo^w)>{}7qKBrzZu60yawA>owK+0tWS0nnQpIJ2`bod+N;y;SOrduaH(cS%REKa(GlK(ZwL3LwNu3?9NPWM8z$Z2 zbmY^(gwm(Vs8ak;*_9q5``NFJyZ=&3520SBNEEyo&VSqU221=XVzFZ&utU=r@gt^NZNV9OAJtrOq+xW7qG;@G}0$A_a{+T*4|}lYpJgP zr&W}^E@mw}i{kX5jISo1G<*#M5i0yGF1VIF{i+#X*RLNgfos4FR$+o(FaIqHOWhuj zpM}*XLf6Fp$;n4MUJ*%!UCaw=Ja%W9o86NXTLg0~4X(4r)(Kdl{FOd!DS<|Xgb0x$ zLeZ92xbNje*1Tr$pU+=q4rfUT7N=nQrZ3J%Gtme)IU>!kg|~x>jJg6;Nr$|FFUm^jk_3h0bjI7;p;e2h$#q#(Cio=!2gIe(Zds80<(SbtGEgUqXPA(A9n@apapDtn7S3f8E42rCq3ZouQIG-su~!Ry7N>lDXYi&VWh`YA^tt`U#o65 z|Bg!i1rtLyl72B+QP!j{y7KIt85SLhB|k~48dmexoUQ8ay^^8x)`ZM9zOxI*ZLj%9 z7$>5na1mH=0GCd^_Acmt2(m%s`F0Wnw}9HS!6r$X8nG91{Td>FgX9rI!kp)mA{HOb z10uGlosz76(IwFr{ce(I6O#s3&9`JzTPl|yuh;FNT2}*Hui>Io{1=BwV$YdJ4OokB zf8U`lI&X=Zbu}RoP2kXCx}zYXkjSws(EsGOcauu{Y3+uat5>6yRh2%1Y&)^ocZg{B z_23!4#oDrzRm+Ng?sKAo0Tw-QSTymNR8Olt)H7;T>^o)?JK_7ejTfiS(Y(=9w6{}b z)4}xi9HtF7tf#y^knJ-_rADFYwji)Ej|xP=6DxmlJ!jNF(Zpw`d_tdNEtmQ+@n_4{ z-S;o)SjjbkrYpwuc}pK~f3zf97BF};F?iIlXq7SvYuC|RkW_t%$ESW9(+zPJUC!4#JG*r#9ba#CFFY%%B#wQe+sb%h(r10o^(sXd=QrUWq!N)I}Iw=<|8TdP>=2Zk&5_Q2)F`)n# zp2}PDH|D_oI=uZ&f41=snW&pBe25mlv}vM4(HuB@rN;10!8*vNI@XjYz~zJ&W1tD= zgGs?+o~P+n+PrHa_!8gHDkm`-GQMWG0r5KL&=9zBhrLqlKvEk{wHzVt6n8GMT<%StA(^67%<vG|86XO?=rg4NCH0{3UEX)(nRO&>j`dxOEUDzlADYGzBs^`GQ^ z=kEM;d_+pi*b(nP1=}`*c#4g9NqkcOZf&@UE$qDJ5_X7+pCV&Y^XJ1Fm`ZuzQ_C@7 ze#zpe(XxyXp)7~0k&F;FXn3WG)@iB`?i! zVFj7WFX!6U(If=MG;ECyQYP_3pzu%zq zbT;NA3|rOVuC>4yxxfJ5;&Dz3gdmh@qd;qgB@UGMK&B1JXa%|3g~>ofwAI4et{+z@ zotJ!@$ah4ti(c1Bw?U7tioh&)I=xFpuFoR`8|p>?PBE#z-*Nhd2k`EN&J6Nn(^0XO zJ4Kyd9+k=HFg?;~w85_`AC11oztWjbFMkoqP-M?+PvR=r1N>kGT}D|8HSg|uQ+pa* zUW?I`5U1K?lAhQUIm7-n*O@+Mda&ne>=*B(k(7i z#b%YD$ne8~6w0QKibn@oN>Z3+sHpB>N?eOWwJv?)6L9~Ho{ zWNQ0uC}f}Nd`1e}f%q(tYh9BF3h|W;u8j67bj-wt?idkoxx4<5x|8TK+S>sCUU>!? z{^QN77WyLkv{4{}2io_eY*7mQnT$*^$EaZLeO z@}JV(DQ{y1@!Ez+E!&SlJ}muiqI^oNu3kDtcI2zzC*dk|2hW#G^3C#r@{9_YZ><}3 z8;FHo+T@<()VX>h;1|nJ`fslG!@<~L33a{mI_wnrq zNv%44ObeGY@UnPk(_FHg8nGzGM2w%+vNt8_-l%{Puu-W6tuq0(cIc6-D3KJCVr}p%9BP7sKM4F@&$p zWpHFk#eT+#m;Lj0!wB=ovB%C2q{N7Yyr>X@vx-U3*-gIE^_$a&@I%TJY(fe7$ueI8 zsUh65^vujxX`=uQGzfBb&fL8%o~u0pE&7;D=*O;Du;j;C&ce%h7EG!@_9)UC+hlPIhF>V`L*o6L@TI_EuGIoe% zm*JPGRGGNC+3J!7A>KR;=}d?fl+y!v*E>+88{Uk;d9=x=QrU9!bMpgy`E8 zG+pNsP>I8Yi_Tf@D`E`&T^RdU-qxd3SZ;;}tOg89^o7a6DzSloEeLjeA$CwZsh~*y z(o1khhAPBCPcaUZWk>2-Sz(5N7SUHkBB=r?zARcp5?&gYfsQkN*c2!mlO3)4EZx)1 z_6N^FjO7(LZIZU6jzte#;tbW8UJi#FVNN^vhW5vSkA3ZEt6$sJ zR94obhx*{iCr~QXNCe3viNt7)s&%7otR!ltlV@8n;}}@w=|>p4MLJHRJ;^Sv+M&NU zQUMF@sDdha8$lt+==BQ*Xp;e=-T?9+WGov|i#q{>#0ZpY6o$3GYU<{`to^}hHw+KS z;IwZ5+P@v3xqSdcE^zRJoEMPRuiN5m73XNkt9N*ev``M$#^TBZyPX5QO>63&1~Has zVQOTWl&Uc;L%z(-MBULXM|u%^LMJJrgyuE;y@0h5_?BEzsi87|U)hN3rcPx$w$%2f zVAQh)sheU-Vub={@&nhimR!OxiX{UK2Iw4So`(y*Q3)=Dq!UYF=Se}hf8;n$Q9ZH?uu zipC4(VDFa|4c7Q{@VWBZyafZ}sZ5Kua_Yp9yP<_gjL}wMMk2CSbQMQe{^V&(50?%} z3VfM6tk$>LMu&m2mQx%hql58! zq{O(BV3xy-TI?gq%!o)vg7c+i9zZ~Z;rnCsa{g2&tk9x>o+^@RZf@=dKvtW;i4W3Q z4(2PVC1;ZUR}0_-5}1IfOP(qLDp+{UGQkQ-Z{oJ5$Rza5sORdoqR5D*7TS@0il0?So~q$Hm8lmp7D(Z@g~AB&p1uOxW#!u`Lh;JVI2x(gLU7xj}qW z5>>+9dNq3a&q0W@-xn6aUc?TqNXttX^UkA>DWYF3JkE;CZmd*wY@eEFhiP3{o+;qw z=kIMAa&J#D5F+1S;BJ(sl0l`)?f<2k57T~EGnaTFsAoIvxG9GB(zZ`A^X?|2Z-LSI zF*X~5p2YwXfneEiGRNII!}p0HMK}2TRj$RDX)To3W6UFuuHa8s_K6k zx$3C3`lq&Lx8v`&JZ_S1x@;|Q!}h3sUg$UwOtBv-CSjGg2+ySGIz@M&g@r~rYT}$g zwNmgK;f==iA@^Mhzj=Oox#fwx97-Q3)-=iE{8i7hLr3VwrT*SClkWF5wJk~_d%yiG zT$>2r4{~w8uU}noul<`|N%KW+_#X+tN?Cky*Pam>Bo+HH&6D$bsblS@5MnC417#Iy zhzUxd(j?46=c6{x+-EFyo%nEufqoG&xL%=?K`V6V9HGW+)O;<@HoP_?>4NQT53B~7 zbe56~mXh4%ilqziX3$cTDEfB}1A8tlSzB{Iu`^&FhHc)pO@$1dngc_Pu^;YeEk1`} z;}&cAeVj3m0?)2sqh?%wI2$UkF#e%JA!tuX-VC@yKwxTd!gt}5BF|Z}MS(WUbvhtM z(TJm242EeJxBqbTO)~Ks3AkmWr0i85u!}I)-~Q1j^&bzt9*I*O9=zQ!4lr0r-|%bY z6XW-wO?!vXE9XQjvVVudA3-jiy_tlif{v9JVL;N;yTsxK$ynw+8fs0i} zP(T|U9ff2oL-K7PY1csUtz4>hvy-HF14%lDs55yo`)fcL^a6o0vp|$0F;Ru1Ta7`6 z5@)blzB#Si(ESZxQa~3k=j=LL#HKg0q6IfRES2rYH$e?v0>QmLGKk+ zdNyc!kQj_^u;ctzw&TL`Wd`0)hMk<(Om5dvn2F+|D?1_zj#A*iy$TA=O#_jal&Vr( zjP;72q<=3zF;6AEa{D~WsU_U3OP$LdQ$!aaI&Z5Z?$;~FcgGsDlVg|8hKdyE5AuCZ zTh||BAuj}h%z*SOkkAIn2EX0&d$ItCRge`fz&Ozrw_tngeAhz6ol%v+Ogmgs9|aLI zv_)kNiyDf^zLm_23#X~#TR__>w8X$w`~))}r)edf`YF=L{tCY%v|XB7`8SRfv@Ao7 zOjxN(qA`y-M>s++F>Q8T25Zs6$Gh+ILLVW48g7|B=Oov$IaoY$5#KQ&{2YX&OZz=v z`)#+dLNX+i6ooe-=}fo33*taNDDQ-T!#yX6t;}!9`{HPeW4c*e>j<1ka!4Y;CMUrO zh%aoZrMze+`;LH>NgA#LGda(F8d^ItH1YvKGVg^lB7F@**zGV^k?x8qCR*qld43|3 zmMmkB(f$TF9bf^IHKwx`HMz70+c5MP4)um%+|K(wY5=TMZ@bsErl#fwB=s4RAo_fv z_$*nf*3*!i%Sb9Ch50cBx=up5r@HF>zd;WJgDzT>zLiu)gU}oPv9~~k1RH%xptBJ; zI_N?$OH8=&W#RKEqH!hUutK&_>v)LK#Sw&+c@23jajtZRx+EC-P1%Rb%V=09Omm&G zK(MRw9jz6os_fy#M)c$A{3oBePh>$?2uzB;TZ(>nuhzZK(ToTk0YXyv@j})2vdjQN zf#S#*Q&8;k{WuQkXw9`lc2t^_=}Yfn-ARO?P10ZT2wnSjQwnCsJ#Rw1;JC_ z?r-cs_}A8n7B-+-XXx^l#P3``BCJeEy+OBUYP}4#4_h-e2(y}Vlh%~CD0(G;3Qr7| zjl|iN0XlKmIIzDIHLaL)Kyl6RCWFxLWb%D+My`B*at>Wp!_S>KH_z{@P!Pod+N zAN;fb=0X?+Kr|m08Y&c*#ELU1e`a4VheNCj zzoX$6NcfR#R)&1TH|0Iw4hMT=#!cQi5LxuLc4wf%4cJm@{5l7rV`;`e*_1+dP49@p zdvFz)Bf@FyrE@;3Q1&j0%jR0Dp;JrbT23t%z=6O~lS_GDe+KFVA2TJP;ti<l+vmM1nYJSCo40L^E9T|DHg7BE z40&?;+J0U%-AfBzFIneRj_#`@-pamw7fvu?0Vd}Oz6A#+p(?{bvrBm$C-TEt%jh3g zGf~ugxYQV%#hX-EbC4qipNuRfn~bW=AsO0hA=~8~s-q$Iu=R)p&eRDrmNR#%1JClw zKdDh7+Vm~@(#vShaj0ynvNf8+_D1~UJhWZB1wYB zSm$qrj8{EhG~Oqxn?$i8Lt;zN0H`aeEXjPewI@LP=0K%MH=pS|IL~LR9eVjFfdlhU zC^sfn(Y}jM7=1e@GKZi>nOM-mK~PTt1YT*9oWapKCWsrfnPp!L8vw#m5#BJPHSx&I zM$`K*n)F3+o4<-$Fcv5e{4jLK8%Q#9^KF2uPVZka{k%Lc>wTx60sh#t#opCKvTT(| zZPe}XTlsZx&wb|$C)!?aLpB(dMhxQHfd5LKI&5Zas>)t~*#`ZGG#A*X7ZWnH9wqO- z^KDed>2N&?mzOg>R-mYpFd@XkXIDfm`?Z}nFO}>wD0_1XOdJb{D}wU8lBs z`*cnK@q+oPNx^X*Zm_a*&*ZE$=BP8?AM$;(1y62rZUXQw0u-Ken1gL1O`!0dnZ71120 z$cLth?UWevu^BQAN3S@CV|n#ywW2wO7aNmh$gYufvnOYWF2h{{nMBdeqvUT)1Y7cHs<=P)W|U2`JUTjm zYz~&#kd?a-p%x@KVp{;U0vJ3;Tf6cbbEmF9siPz6xhdb|Tnhh3@9~z%9e!ZlZCi26 zfnykDX5Yjju~A3(Xbn1JNA|N0U9!wtu9#oVaZSyxaEV|H!6mFn{vNz_qxUzoI0b`s z9M*Bu+f&B?Vglq6@lE2+3ij=J7p0^tvFMbfH44*$<4tt&Z)kQU8m8V)wg!H#PbAx( z=4@F4YrtyB(gJE|yPTsr|5gm0hg;w5Rt)gMSgADh<~U;-KfJw*hAd7}n%Po}%+rkj z&N!x3lFvpga{oD^B%LprCOU-sjUQO5f;@;n0D#&HNit-Q>&h8B%@GBh!$}45fB!U0 zP``UUM7{n`{y>)md67mIF|XNI4LTk#&wKRA2e(FaL!ZwUQ8o6Qyo4+1`lBW+eCBVH zRRrFkg<*>3yNkh0>R^EcU>SM@Uv zPoSN{|D;;>%meMvI^)pmuj{Q`AC%46>yxw|4%;JOIiq&H1096}ghrehiwKw?x{wi| z3)%Y%bRk6B22VeL6THoumXCXc`udJ?nZ@tjEz#uo&0%trS^L*_hne>-*k}AlzFmep zsm^nrK3A<<0m8ZWDO~<4Pg-#?FZD`@(d-IWeu7P@`QI_>dI^{Ha0^UFx~N+OaApk} zA8K^g?PCW=f}TWC&G4~(_3riB;E3pY)WUbK@AL4aZqnu48xz|IXH7)})+N?PuIjwf zMDCy@$KQt0MvP3L3h5|dj9xWUxL=8WMDAW-XGp24H#;Iaxa2#&Z4=A|8<|8cr>V#C zZY9PQaFb8wI=leVcMv86vQ+s79ax~QF$OXc)K{$XW*Ft{sVdl^{KRF+Jq~@)sgIf4>$QIrb9Qg zvlER{cMCU*fTgRFWk^r~UcLbh10Si98Olg1;rl~QP_M)dCSX`RE3Oi^lt=Jb(0cmM zo%@sjW$2)1xg!7W!*`sI>n{3qNe}a3wynnoHd0q&v{+@P(ZOntu>&70*jF;5LVp!ntYmyMF@($N+$LO0~p2Yj>!?`6k2M zmm@+YGFG6EIM|v8K}}ZX7p+R)+7gq`8;=zEzxHXZW5+pANixD7==`}RQZ|G`7EUnH z*g(7&3xYMkuvUT`0uZ$-ikzFS}0$&48z}zyCd!rs}QS z2nC|OUfK%`vQx;V%=__jex=08zl30@=Z8jP)j1o)DN4cpKm@tYSWx z5;Z&|QzK+ReH80@Ya&`kfce>cGdIQ|#^VY;!?ZSB&?~yKz;ui!wG~XSg@5WxlE}Sx zKp{rzhA7Hmq{=`$WL8&g0YoDp7aNiX4iX3f@(7d==yTEY1(jm9?w|4o$y;CSa)K`1 zyv52PpLT$iCpkTLP@-L|X4#QW;X{|T!rS{=KDgY@kgQT*l5-?giHS@JSE(apfkbb* zMKJ|^fwiZIh@QRTGkbyt?lb?Cs=c4s0RgHc*EiB;8GyKeJGsH;CGI5hJ?HSFnB+1k zHfgPL>z>rBu##Sa99xmXRdOTrOiK zECS8a`)?3r8TZ*#6}Da^#tIaQOpW(Oq(8pp2;D|?E+L?xDNI5Sk7b6=1iw6&&1jS?2h> zN=aPbU0s<0oIRP2RT%CoYE*A&cS+pa9%To_kb8W0yXnD5?DzaPyMM^6f{JNq`ySA1 z6vnOl%3}lXP$K4A^IC)afd^vxJV7O4$%-sCv%iPcH5sS$;ChhgafzN&c%trPa*8}T zLWI6RSlu)>iqOVymdq|6-S_!y4L;7hsOu6&pIcZZ8aH?1w21t=>dWER^%D&abQNk^ z6r^7x3|R?(+04CeQFD>`l(<+}=-T*!W(5*>sa0eUR$$6Kg7 zi*hbp4t5|YzLn4NrX2vAPRKs0gVQuIsvX`xjjjaN?}*y%mNg}YKxr(gztSb?+f(bD zGMZKz`Qm}PjOGXl-LWuJhqvEtuWyf)qwL6LwHE)>>R*Rb-S%8tWtBFf5BCj{L^WZ0 zjYxp*I092}d`Or!3dx-AN8e%ZM_o$c7%Jt70%xh3Ju%IO28Pq)@w zLOp3Ke)r*n<@G+vfozfbH}{XSZ)*uG&BTQ|vq?%dT7#ym^Vv8POdGO(ZI{s?H}l>) zqYgH#ef2a2d(6Q94!n?zds(TR%>TO7tbq)dHt3qOcJ;4E%kE#pqTcoro4Jo~wq5Y-cdzPJDom3#U`0Hb<7%6r&VveX>^jyth%2#=2&Bf%xEFnr;oP^P&3F z47(0TPC;@Mr;bc}9o5wJs})~S=Pp}Vk-~$J7i4aoi4O4`?CNB|d;iXekO^7nwP^v* zmqpSVt?MR_=B{^XktdRiX&@Dc?cbVCcIFsx=Ij5P8fAm@@qe6d`!aRI!H?Rk6e^5? zO4yRvrCs=e=_Z@JJ@hbL>YpGYTwt-s*d4z)d$6B8!Ewlp@Xpd4TN-}S5kEB5wV+cO zi(WB0Bxq`s$HgA&fX%q99dVBx&@#$eT(2~p1s#qSN1-Gsd%S2{a7-D@s;U|w`OV^7 z{3DCc3eXJwFvkely7NAkZ>K}()f(_V2M3_UFtv@jYy>Gmt^=)I`%3ZY1 zRT6ukvfh24#+IxWDFeb64i2 z++qbjZ+LHHAwe%XM;lRCNlLWXb`7vzf72Yx$Cblm`0de>yCIdk-TN{fY*U#a0S;mw z4PH?Ym9FXZIc-L~aUbm&n?mTQ?=1YjCpSdM?Ga2Q?)-4}A>qh)vs#nltOI8sKqLz^0_ zHf|SISRIDvCE2>LbxqykD~Nes^p-|!K>U=gW}h%h)r!Bu)f%bCaxkT69^>~FhMOJV z@4d%pFI4o;Y64o}oSX^1`%c^$iia1>ZOBO>)KCt4*7Nb5hj4oZmUVG&Hk;?J7Qd3V zuR0A1{c4MNbfUQ`lij|>FqtpWULGV(ccci~Ec|b!;HMZz>S&Cjh9!RNT^<${(LDG& z9>lM)ch0QP4!9`SIXi3ScCIsVx{-oI!~ z--j*^5Tl}U5dQFeHG(ky-1(Ys&m$)&)rfLhi&pLM`*J^> zLk*9IymxC&qgehX4is8B{1vk%b3E z^#C0j78dr@U9jYF8j%u?ix0frP_r2!0gwuW4(_b{zAL6$=~OWjw;1L9o1*i^=wi(4p z?`Nt&wLy(aeX+zsPM5s$1Ab7j3WBy25I|AJH6k4N_f^cFG_}jVsA#ZtY9BAZ7PDbYN?L7`?V~9BP z%*+fS3CYT(xYvOv%nVnn(+yuN?4X08R>RCIlXDN9b-RP}tXw-n1ncGF4uW2`4PJCm za25P+4KrZAL5jd-$RiYcbhJ|Cnq8WsfS;l4D>}*@?=PbMn2ve4Oz#DA^@0qrH}Gz3 zqGVlGgXcJuzJv8DeAVLFrsoxk2(0zfM1e9ho)WQdyOh`%!#sQEjr3$7+^OU+k zU5T|$Ka5_sE(a1&Qo;e0Lj}cH!t=ucak$On903dvC}>79*nMybSX(flY$B-1xC#}3 zDL(NM84i(>K(=&AvUExCC{9cPU@^NvdcfJ?BB!9w9q#AifNfp~;q%)!;4A{98=!e; z^SVZlBNrI}jPE%0GWeW52f&{OJWqLAfb2fDr1xyoct>`DtMrn_cG?CXEc@3dmsnV2TTSOw48jBH;rFh!Vhw zbpfR|QrsYHsF4yg{uwA0D3NLguySpF&two49R(tc@6&m%1ZJ?U`7@xO0}Bd8`3msJAleMdVg35`3s3~R&YiyrvFhBXtz<^`vY`_=_eq ziqFrFB`$0wIN8NyAKxbUkpE%;de7p8Bzk^YP?*K!*xJR;2J>j35h17~(*pr{LWH;w z9S$hd2Q+(6+agb}>ps_{)G|E~nh?-aZa`G#r%AaFa1iV`DA$M#_^v`iF#rbz9rG5p z*s7<787O!`X-kOqe*kdRsv8#j z0dy=-B$|y-63iY!vUkk*!1e-*7S!^nVn!huG%_-R21*35()(vh6swz>!U1#=@QOfX zoF8=*;zJp41E45(l6aCYjOh0TAQL775WLeyK}{_EN`e4Us7-rNwFDOmpp?PL09JUr zewfu~fR6_3DL^3tQ9&qwORh*zTLdIP01l}4%_%fc*@K$8pvIYKxw-`Sp}_R7IH*%f z$8;=K1_v{E+}`;uASVHz9Y~qLl?pH-TxF32g&2W9jq82T`pJpy8tUGz4|O!r1^$b96jbU4QsV8h$`0{*?GQ69 zX=-e+NAv+^qQmo(kD%uTB3QFv7ZCNbfq{nw8)@W!8);5HJ$;HS1ZO{$JqKHt=>=fC zz!$Uu>ZmdRHYk8tg7p;Qcbn!1-OC21@U1r)SgVlAsDSbY3O}9#s#q5r>H~mYF%gFD z^WYPckR0+L0PLpw`7bn3Dhc4dA(Q9w(gm!40J#EiGRXS2!2h?fCb%u$&;drFfIt-Z zguSkp(VNi*H?JCI6bv#!;4|t6y>#|)>jd$&0jJayS9$BfBzQO|rVY4%kcV#o&hrhZ z9yDd$;H3!a`rv?H^rw~qo8@bRjy^#A1Ep7k0b|ql1DJikThK>qx@wr=#DJFyzLsG`p^V%rebB=a4~JM0VC!QM!vs^5OwipQI2jLby=4;PrKt_I)ZLysY?e28 zD|CdQ`~mD6{FG@_URgtr)L1LV1CrQcHU}IO$zZ^Cla1Chz^<-z_}rWL>4CRW`S)_o z_F!!Ermp!J(H>oKgC*CkW;7*)6o^ z{wru9j(9r@u|_HP!p#;fiSXrT5Nu>+CYY8H6%Jk0^>5 z?RRMXH3}kM(C2RvF^Q4d5@AvXCgKWc<>0`K=ZrQYv`2+aH8Nb}LB_*VTuAs{>GFwAZ;z-e|Nen)l79{wsyS z2O}*W-c;yOjtyRf<&L)CcoPZ8yJQ*pCcL9%%`;p0I?Q6NrtSVz5q+5$IXGi>>fKsJ zoa`R_KoG5&ljxy3m_Y?tXK3UqEI^CJ&sR;uBS0;s%vQUeufA z7iE4OWlOf4(IX*7W<%)c9%s{4l__A|W01I{ zXR6Y&%W!X`S6{dR`>a%r<7(tP`L9Q7uM?M7uDSA$IVYbQpC?d0;FU?ihc-J=DE%?L zo9Fj)oV5;?1QlMR{aqS#R%N&T!xLaLoWISNwpVZ*SL6bVMs%c2c*RrI4 zE3jSpa@jZbn-;L5gcIH^yf&nuG@{=fOzEbe4Gkiczu5r&7LW5k<0c_P9-1gzxGj9+ zBbe&QF(aypgjB)A|3|f6Frq_AH*@EZ%MTaU$yz zor%s@OHC$*shG8>M)CLkp2`8!9{L^Mpe;3UU5%IhzyJ2RcCX}dtq{C$Qghmh^R#`m z+D}s!VRobKcs{?Yy#IaeJ>1%Uw`F<(U-1Sr-0k?v(Oo;--FLi%HbEF{j0wCC(p2*m3(m*a5rQyV=J11iyY%OZLqy{L zW~WJF+A_Pzef3I8s7>5;so;f=h=$Zw)zvrCEP_XlKViEQeb>*;FdPw&&nxZ=QOsti zNVlf{4r5bdx1#QHgWWuGxBvZxe+G#4=vVT)N8OAY-v@nk-{IO5Zv5nKx}6z_y$~F4 z6I=EJa*`i@(I)?|G7tggzg-yM<8(#;ZJXeq9igXRLT((nsG~JTu1}h&Xv@MY;u@*& zVA~nwFl=tW-lGF|7>Ml-J{0XO)vKZ~;HWTQT!C~Hzno&8PZgTv8(ru_D0@#ib(I0k zrT-q%;J|;wPvpAXV6^<(e1T{Hp_gqHz1f1sN&mZ*yO2SD!9EpPJBzq_j70lneKp&$ zVt+TA<;k@oX!;J$e;c+9*j&G@^nA>V*v*PoX%G4Kz4!OwI{Be<7TP^L*x^tIXhyPd zz##rxTk$q{{wjL?j{4mP7#{>P7>vPv-9@fs6Le)jQ?R_QB+swj@=@jBAQ*9B4ENZ6 ztjpm$OOjFlqp{09&Ek{2rIv~=wn?{I4spcA*N_uD*t!hrJ1#CW!RH0~(gk{kuiRd( zw>%$UKhfZ!^>=BDNj0zacq(_-%NebZL}g`|o1TS-Clq4zT8X?>5q7^*W1f>`RCnY^ zb$rjfyMyzT4Ep-NV+3Wa6i#gyAK!^XzoT*^Y!N&4oCAE8dGy%yM@CMR*G!FalFrvT zohobM=91)NEVSg$$Qk$M?KjJvP2p<--3BY)G+;8&IY$E^WP!Z)v_Fyq37Dqn%`bp3Zs5@@zBR=^4RXbe0H zu8B%ach{i)P{s;EBDalsce-eb`ien>!=Y55{@0N)siH-c=0be6zFL)vr7_KV-;5iL z;AhsYQ5_j~UUUs21mA>`f2Rrsq{nw)GS}nJmPIPIj>Pdkj#^KaD&Dg3k@Q?nvmsz( z|6oTlGO$w@0Xx>cBd*CJ)XT_N3?>f!*XmVOtR_|x16wW8&)#*&KiLu@BQXW*GBLqH z$^CDc!|xoT*Hx``X(+x?DsAklK}Fc20>{Tz?J(i-`#uU}W^lbQ)3U7Rf;#Geri_p> z<%I%QayHcBBp0|xLBhIB&vJZR=Dp1Du(q4w7&vb#KiWG~U{fvro2a^?3NO}fU5G-B z+yK?3Qu6HfCojS_m0{-4UewUuIrstmW^hLZHTghHvs4Wk*z4VfRXUXMcsSrpE0aRt zZVvI5>Ys5l(>c&%yWuaK`peLblX zrVlJ2lmDj3S0(kcLm#B517Ov_=MGVlreI=f{q{tu)EDdrk^2zv7(h_A{ERSKV@QaA z+S>&Z9{?ix3=nc5%JDt`%?FvhiC9Q(tnc#bmr?>^WKmwPr6oi}G}-lS3LF_^VTwx= zK(-#o0A2upW@OYK*HEsoGSot#%=MR73pRjCE{=xW%^m*{L%B zdo^l<>8dF$;xEKJTUf8(s`~1ZM?r3Fx>~(sWm@1qQJ`JE1yJ8NhqaSzpa?`>9wmr9 zgIf~hJ^@kP0uRR-5LBZlQX6@DllZdxJInWp_Jnl@BQAq_lApdI@W>*=VaWNwgUK5W z@h*t7#v8xy&Aeq)|NEXWG$JpUe15Peoe=&1u=U>YRR8b)xD_%ZJ1bdPiH3PXRzmhl zvPsC^D>5QvW-F3x64@i!D_L0?$IRYj{jR6i`*XYf{`lT*=Z|jWob!B)YuvB<{koWw z-)=aR`BV1-ULH&e01gZpJa426^;{%vhC~puedr+$(LNlQ12ofF2+SpbZ}X~&p7lmB zn6SoJQl;)!02^}SYHbsMzp0l#|6HE)&z1iV2La#%$N{Y|6#{u4NPH1!i3cKoP6{cO zD8En7K5KBn+*rsS%54ISXjrd77!~nh@Jq3(no90xHxX@r=ARixa5lIXnuXF#HM(G=L-%V)rYY^3gB}nka3d&+-;bELZGb zIZe@N3jm0Xh|POSnB-88@Wp(dPS0IHM%51|02|xM2!+G0&rCen;PF5*tIz-0$lDrv zFiz8cKW#bh@cZ4UKmOurl#V14tCgqaHZWovrj)Y#L1Z?uAc_gA0I4$oVK6nI5yl%qOK@m4 z&o3eps=)=wN~04OAyJdxnA)@A)jeAot!#2h*xj9TPfQ(>U*@`g{jQS^$TcrhlU>9Y z<(q=RR*<6hrinMf)|S-NywT?ihA!cD^uf=E3#V_K&fGwl2O$~Ixa#R_RK#n1JTW-> zN}8>D4`j;JUn5EJzyFS_bRUtmhQ`1w6^6u)gS;#o&WY%wWmQeeEVYY~jN;$#oo(MU zadxH*&1hKOg~MWEZB2v>0)aG=l0@>6tQrkuDa9vN)@(PtKj9H{XDT1(g) z_wFn=!sb`1e$o0APSdv7{}NN?D0xHZc@AP|I8Y$04EXfy5)LNJdU?xw5mNbbVx4kg zj>BGoWKjy_J+J)th#za&C8RokKW%-gN&z<+de^^%iK3w7fthMBri9w<_~0gNNJ&Y_ zOHwA(6r{%!j5xdjGMcs@hJG69`aPN@+avTIWu7nV`vw`oymDy5PWD>bZs}Ln+7V2XxKAO2aA#nQ28;y}Lm(=!DFAll zu@5~YP#`9bH@p9rhcUv5ij*n-+2~KDywYSc!?D`+EK6LyD&mRbilqnfW=;G&pQ~O6 zSy>+m6ZPAsv^Ww$R6)`VUA2U_k!QVxMYw7sl{ZKS1IVn$aC^6*$vh0Dft#{$7$uJb z@`uK*F2!#-zX7R*9R$;w#D!eL=ZpNjgsOdEPKr{$Z}?e|){EKLXTE&Ll-s7Qp<%`l z^nBVr=aW(9N8`+oUK;2xURizxpyxCy9Zt^a*Rf1dp*k_6hJ)o`NUEpIC$j{CpU zi{E(@fIy*$3)l_CHjB81u0t0ATZA!FFlrmX7Fd~UG(@(9I0*C+91JS%Mtgcw-4+-| z0i$}rM`q}@Y2O}5T7BX#eIwZPci7va+OVHS1yqz`UQZf!)QBGn)9Q7K#Q(v`N(xm| zYR)^f>OJYv3kIWU*FlO)#2$1X*|8qB|D62;S=RAN`hcNx;{%=vj8AJ5BqSiu4MsE~ za+!U8t>{K%_8)+CFu$Vsap4qn`GP^V(kh)@EZ31JuE!Ar1B~$&`$70ImWv$hx|_!2~tlQi5OU@Zi=@lKKT2 z(eV=zR*j^kHQh{Az&W#*{I2pV_`^q1p5FHw{f2sX2IBh)h=gc8BTS~IbwUgnGqO({ zT?tim;sb*%t`hLDMg>0`;a4P#4@}!9J`_jF6$=hTXZt?5)2I__xWQuob1gm$zS@KY zLka|9YBa+|lNhaEK(*2UqYyS4QeD?*Uu3GzDwEY7p$BT z6gmT@gn^-7YDSsD1*fE|9-vuW4fI}rB${H4U1BpHVx)yDA}$!043qRGfy z3#D(Cwi#XL$W&2DUVj<)HycAB6IO4U?t{pWm6{r@7@3*yY*T1 zKcQ;!6aL5Fzk6WwHc4^KG>32EYhpoiMgQNmu)0APA@#g$|ESk_h=6rH%O2gsT7yL@ zqq=djKZ%BRHafbxOG`HZsOE?1F`&_-N6^dHAEwE0{EhN8%<`!zHLQ4PK7~8|2ou%L zfG=~E{yy_mA(2q^#oEN83^q4d(PFxKkKq_W2WD?>&vpEpOtuY%EJEzc+ZZ4+_YI+`bdT)|{CS;+UE zxR-&uxCY%wt<9n2zck~mAcv1jPLSUZNiFBtT&{VoEBF_1#?wR4rxx#~cKP{<8M;3a zRrdKYyG?B$GKaF%7?9neNr(@>OQ>1n^|U235i6~p`+mRD27Q>~isdlwPH9nxL zLZfilz`vS?-;?B5IVf_c1KCG?p8ve${0`;8mO1%S!mHtJ%9?*Ff%!CT+P7{|U@yy1 zc2=@8K6=;aN_Cu@^ip;~_zK&Fc$;gKn=A}5{IOf;Avc_By7yQxG3h-WpnJu{%L?}8 zHXB{Rsmo)OFPEC>sD`fmRLaGH;Fs~gBkcH(05L3&nm3QA$%&d;x*Uk%BSJNa<87MC z^NOvjk3_Qj`2f|_!OaN?v6lM2+NnC89AYdqCwGUBp)>x&(e#>xSgl6WOv`){=7M9K z6&+NZXRh8r?g=$gmANEUm7@61Qp9hVkP@u7)t89e{7m+5hI*@p4IEqeU_|8)_*1q_ zIH&L5xq7vKY39nFyK)x!RJn6EMk_{{d37k}S6|Y2v$vsc_Rqya@rKFI*NPKaEBqw~ zfu;5eQXi(sEdA;m4SK3CZLou$VPTLP-4L+9_>E7-?uCHMUrIlpaxL5~P0G@T&OCFb zUw2#{y>e2NzBApl%B_078s``Km37P;{0dLD(1#-!NueaVw^g)>bW%h8X6MM>y0MBb zWA!Lpc%SZKW{A;lcaij|;9E*;3_BCCH%w0-8e6f-Blarre*i}c3Ok)3TcE+!<#O&F zJq(HG$ZF7?uLd8HMYrE*BHgWoEmhdhgR~X%x7cK{;`(7A7d}TMIE{!AAHjPL{Ssv( z_qHANX_@5eyB-0eIlJwL{%7hX&DiLN=e6MMX_g-F8I6I#jPXEPQX>Q$xcVIZdLEq7 zaVe!CPl10u_B9_CEje*{hTSEHcy3W<9(l@y3c0j9}Zfs602rneR%~O-0d=-O0-@#7wiRrs)XK#A!&AjMO1nzAX z`#E*t8AZM!T{jZ_CWGNO+_a}bGah##!l=PPv0e(hlDWsTIsZt=r7%Ucd8L2mv`#Oc zxc|#HhV{Fl1X^JPLk|^t?I*nH*Uf*3CiK{R@~l2{wdD>nVs@{*b|lVFPTUq6;d8%e z{O1_@*Pm07PG{7($C&V>>5TCsnZ_rQ&G~J!!E3L#72$JkMQizm?uwweB5Ke6+r*sPRK(bnx z+GC-ER zY`2hQ0%DI!AqtAw9Pv1l6u9^4*{-Pjii6Lx!{hQvQVd1!_Q~{K7 zV59iEoxk$LIZU$^r+O$$_gv>c&nu*BHCu0QrgB@nE~Xv<6hF_HeWSWJ z?o;T<$P|h+_AgibcJgFX;yDeaIQIFPTaAV8r-hmE)XGP%YTJIW;iovJSWmjl|2l`T z8P{9g(|sm;o4DYBuL~3sP!d09yHM5V>CZW!0p6b`b$Bk1&iV?!;^l_E77H@E)xkX^m!_$pC>qlBZyzx|Hm3b;_)(T7X#Hb=Z_QV(;g zxLDQ*0;<1k7!|za5M+}MSXF$KAb?88acvoBUm3pQ@ksVeg;G#vT~zzuS1$7ipE$wx z&a2Fu6H^m82{x(R6DOxHayV6*SFG*Q@w5mC;t7>#WRLvIb@~B~RC!J}f%ZPC_;c#| zuvMBU>&_X&MCBV<=FU_RmM9jL9-hD%VS)4$qScV=M1k;{)kB=hms2s#|h1YBg+&o zRxln*b{0ONw~->#jY;i|ChLnz-5AB<{p+TCy4tdn?Z_ICxXyEsQN5jzlU?{Ic<)32 zHPI@Xm|m9g#iS)W`C?&N)jks|x~ceU*bm)-BIEft$+l+sF(XfJTo9)8mSQQ)3!YDGEdd4*U3fWD-c=v7x$at%PDhOXBTa#N zapb_wxVN?}y?-d(Kc*gP6~1?fUNxo+dWIgA$9H8n{`rOm z_S7GNv@dl2su1G*-{Zv-5@-=9S;uNv$KJVNIG%W_V!`Lt*1K=IlL{tl#!~=!!q)Y< zDNq~kq&v-LKW%B9HuW*x4G}<9{(19=hvz3BxrIu7sBT?`k=etRfsqSY;x9Y|Gkpgr zOVPQp!pa_nU)66qe)?sW!4e*PIkw1Br>))Xn?_S|AeR(*%>q#~mE3YZv)LYJP|NZ&v#*8=Z6*J05 z<_Z;e$iH_#B>QoVB|Ho}L&8trjVbxCy@l0uMol!2dQ61l)Y!sjFJ==HqpZxc#e#LU zRYLK*T?bx!=gZCF*iGRJCtT)5^SKbcN~kX@#tF5A>azFw7Vg+be#=bZCdS~MSkY>s z>t1!t;vEYzvaNgPDP|huv%!P?p;VN*I}WASoD_6-TT?8JZ>DdIWgvb;953Yi{Qllh z7CyoaOW8`#L!Ufz?77?Cw$|%etIwazjd<-^AUD5fDgduq*v?x{C-@cMIyXQdwn;Tgjla( z0V3|wON!RLhe{4Lx;~5|5kBM9)DTPMmb~=a#rS1QV;#fD^JgFvv^G)M;fQdwW7!ghy&hf;Ro~Y+*tfH%7Bdv6j}| z{ri)d#_{0~;HIGUmL2pX@{H$?Pb#?yl;_n|$GSh7mbr>m3=aV|Aj!q`a)Bl{fdLbW zQ#5yWlr`CD$?d#vz3I)KIhfI-M@8|aD@&cHdJ`^Z%ZFTbzK>toU_;oT}tS_cjcSAi*I#5#9RJMpyt%OpX7XF+#ZrGjgOlalQR zwPC}AJKrTSFJoT~VT|W8mwT`Ae0o>v0?cq$3O=VWXiU7yr<-u59oV0Cs~lz*K9Qs7 z_A7kM&qEkL@y9@?uPZI^(A0O4AXwim@Jq~L)b(x0Ae-(3dhLCkFHh=nXGV_p4t^37 zg1dqn%q}NYfuajqeD<_Uq&gEUxnC=rMEgzztJVcM`;a{r{ATyw7Hh*u|F{61f( z{8bjchhlY3sx859PsT(Hvp$OJv1)Oqs`u5Is8s(q^yJIuvnF;|yshvaK3(N|wUNU3l z$xnaZc%PFf?k(Kb2v;Da(?{T*q_1b5O2_feH+4D9vGVHFpzCNyRGF*#NPYt5A}!v) z;hyDAX9Cr8u!RKican5?IVXhsl8pN3mrPyu+wtb z3a3syE5c|ct#E|Mqn9J1wx!=b{M!kmZz{TawW{{8LVeZ7w?(RKBKXIMi<`%XASUxL z@1rQ8>S#>AU<*J$qLk!0wQjd@U8rXYfRI%`rR!17(BCD)3iWx8m3`(}>PuVh4fe5z zBOR%m&p21dc7r0UN(U!^2J*EO1vZmut1vXtll~PpzAt4ox z!lTcasn!eoI0qkByC}H2v`5#_`$yxQ_VDTToJhWw`(0m|tR=CvB4nGgO)zO8w(B|3 z>flRyWFN8qShVwnxUDFykA?TU+_LKM$Eo|gLpx87qKH^o!Si!({+9}(mssv7aOsI? zhLa*JUHJOFYK0^HjdtI<9*N8^Gror`?90uIBdIGF#o_;oYm;W1=#mXKQP|z81&U zE+45qxGs`pXEb=aq(vv;hlwvFXU_DD@1yA_ofq7m4i=a19!wshT#q7zS_4R2@f=+5 zk!MFn_IA4ZOzxe8nol8I^KE(GRl-2zA+J`eNrQ>&&m^>*ZfTTeI|}x?2#BYiEC!RKIu24KdkQ79Jf63n ztXQzsEgUheIZ=HQ?zh1!h^-X-q`bBT~2y#eHD^|ezjro20_AtFnmaw5Gu`X5M-Wq{_ zvbySq+c@oq(s!G>lSWsIJZ-a0irz%_+Yi+;iJN)##G6Y0Nae1jEGKo9!BE|u z_$S9<{`dO$vv2796q|yOUC)~J<$qn_fAp?DzHwdW#$HjZ&FZ*V%FgtfG=*axv2`p3 z0gkdx{VB&<9jS8mqLk8V4*jC6Ctr;X_LmG-ZmG{@F30YknzoV8dE-PKCnFsdEvmlV zZ7pnSD!oL5OCM&_dg8XQC!sGT!dEt|u5vH>vLvKio{R4eAu0o+Ill;pAqToY8FMu4 z&|dq}cuFW%Yt{O_i8qlkeUn`W32z5i2`_)^SshPWT2J46X5n4YqZ_c7d2cMbFm%MB za72(~NW^g@m9RfRtA}3sLim`;+SkVtvt&d&=?}ei)JSN5^fh1NM{DGv6j zbfs1H^{gjHyrvobRVhqK2K6L<-YbvHJ22i{JMnIxt~f$DMV=*%IG+9%dFPW2%Y1bF zWg9J5AK!5zzS1gOCyMSlr{jngI{#j(xbd3)Soy9))Orf`(t$5IWa*})kUzelUWSsB%kdC3A=N?cXk<+vqN0n_J&{8Jw5jjR>zzIPP^EuW+EzSF@C#Y)){SlH|rl6~K2GA77xp zyWr(GAS(aBpl-qQ_r5!rV?o|tz?wqFxV>+EJ6l0$UibfMU$=7!iF z4HB7LTh5b-tEHDoJt5flO4_5tJ4zI+?cX2GVHIljcQDyk7#$jBkelmzLN<1Kutq7+ zY8&^(y5r2qfF`FR{)ZozRKeZO2}G38mX`h(vu z*t$!G_5a>KV=O#BMgoNvHm@vN6%rq9G0VpZNVQ#rc>4VP@GC<|k~hUH*$R%T#OKwR z;OXaODdB(o$IN;wxbQyiR;z(y4rwQRxjpwBDOr5s;B!F7VP1tT98?iZgIrL~1!Snw z@4R$qkx)t)VKyS(Z@kjOc&yy27k{Bbd0#hvxxShh{@=`bJ)@`i)99LIwVtK1FvtC6 z9)D-o{BCKP{}MI+*uDGjn{(hT_s(m&DU!-;@43DeC&U1jeBoXfRo}qq(Q38mkinIU z(>>BeNo4nn``?Cg;lN+G{KvI}znB`SCTFbUGrT$cL-xT-Bazg6PthS;;7e}Sy$Syk z3h69VrZuiIzq<5=`y9C>vLC{Vy+Rkit@IzgWkjNO1gnYPP*m5)wN+21|7P{C?@;Fo zdgjWF6GVHTX9Xlmp9Lh^u9w(cf!gRTC7t-ewP+3}U_>CVs1%^zrotzEl#sV*mJ_d;8KkG~M$S zC|6R-Eh}ooFQ{|f>eYfr1*R{%1G^I_F+plRKm4U0wT_lWhXO|`OW%|CrTSI7H>5Z@ z$d}3)UIv;t!YhXoqtlan14mFqgE}7!O-3{cz&S^1Ei&nZ=wq<3f6sD$F{4vkqsC3U5zu^}gzK4E$pETH`B4WvV6uH0F#{>{ zwd=ms{Wc5VPZ73}dFaN*g;*=z0aN0vq|sE=4}Kyw)r(@SJ2s)tDLrEcb|~&}QU?mq zu0V}>b7x6i%;%Wq3(xvNO##|Cd;JwW7V{-U{qP#7AP7&s{$%w9LJqij6ZQ*8(5j$^ znD4#C1KJP66gdQ#m7D!2hzc^2h&O}r*9gN0 zI76U90oFS(=BEx*gIzQG7JYhvW&mQM2sBXtp=ie3DbO(|0IYVtCsnUN79zo1TofRV z!NbdIv08i)J)gV{yc-*Q}N%{|%gEP)`LBNWK z{sbyT!=8+Hl`dBmx?*cDc&?>&p!bi+h;X5$z=bA$L|F1~_c;h9ppFWKB6tklc!Wp~ zf~6X-L*&GOe&*A&wSz;@Dk1!NP#MhZ*Pn?~$YAzvBiw(WoI}t1;xAwPp_QApwKX$i zB)TV66vNL>sAgd&4fsM(sGC{p&vx5ep|u^ac=`C7t+TJ3g9n(d*${f|77I`coo_IF zDd3oQ9|+OL4;MW}LunrbunY6 z+A=m~#_4PNeq)n^5x?z=7Yn+nd^K%z^Y0)x!f2qOq4~XkaIi6rop#T+&c+MW7_$zQ z*BF=(nn-vgA(XIzQgka>NrNlS=A}gGh&0sLb8~@@1Z7RokT6DOXa5}lU58|$neN24 za^4_9^#KKl^;l2_=frz7lszk4R_Qb}SG6kXmDvnIx-#i=#Cr<0=BeuSy8HYzKj05Q zy-=uMiS=X{*dky_3~D_B8yh9TA~0XVht~=%K1G-evdcFNYLN1E6hiP}kyagdy`&U@ z2QedkRDh`$PbXIIwznYarG#PYn@Fhin|nDuSvSzH^*KS#22h>g6>ny9ur(0lwde=mQ-!lvQ4im0i$I?k_JJLBNkiXPHnH=k|KjAwMP7|=vyb8SKt&hDEx zZyK5op@KvtQ(6opM?DRoZbBk^4w~qg&ir~^L@CsDVE2h(Y1jeCIQtx;F68joE9 zFghZrzq=x!4Pr!jRV*I>gC=uRl7xpYeCF?99*ANpJa)PC3bcCBf`Wo>r^muTQz-_i z2axa3kANr_CC}>dWgZkS{>)R*%Lm9M70%04NMBq`pL(Xu%*+ahIRYHX-}?&=T$pD^ z^=Dx(>QAUZ-U5Uv*Y!Wmh*S!ipOV6^gK37fk>z{W1aGd&6}JHvPMU?xv~)z=_md-I z=fXC89UxE9T)A?eCF*_i_iRMc7!eY)8D(~u2`wgMx`y6Fg1qfbWNB$Y1}7+s8oIiO ztjAhrXIcO5dh|92iuNDE01C4D(_Q2`Kou3tkS3zPg72D}3*sI`TNavuoR28^DHx99 zd3p+a4BO~-b}F7G?ir-P6=;3uq1mK&hw~6_S5YSjHDrIG#w*7Q5gWBUauxsg@sSVQ z^tzM1$I$W`B$fzgPrb=r!xX!e+vl7B8I@<{Caf*V&Wukmd;&HmoOG)X#@)jUx;xjP(NBF(Zu{-+o7&hAL${KHvHFw8 zZ^FY@MRu(b70tAR|LjTEx5*sxjm5repaz5AP0QW;M~1ZRyqc=qw7km_(>^udos5tQ znDHkh4Bgx+L}e^FdOVWBWP$VxBw8T)1E~$d$Ds@VMI&T}6j7wh&AkerRF_+{EI^N( zgVK9Pr6#=xLXM@W3Jqb@H{R!R!BqJTgp`SV96JZc^sdK;Wf6lKx4tLbpw9{i`y@HM zyOz=JlOp^cY!mS3;4FaJ$=V|4+11}^1ZH5Du&a!&l93_!L}sGIBesS_V*#M!6xE;7 zAp|#&8-e1Eot+)DJ!AbQEaX5Qk$hgpjjc4;p!c8|N(qDhSV2b0P6Z8;f*E+UObCyM zw0<%YG&_oliu9IVLeH_r(Jd_Dgea`!7r}PP zM}J*do`~~soT*WFG4YlY=!2+uuIlsvMfAPP*Vq(^3kNG zO3rhYTa-X2$O~lmBKOUBet|>~bvpmz6c_qdiR^76Y$(L_-K{?){=56{^_xhb#sXgs zj!gor8b#+96}Yi`t7Ep!9dNc175BWu9glzOT|}648GW2Bt*x7g60-hy?gqkf1ESy& zJwDFV{t;MR_yIy#1izMFSSY0$yO!axzhAGFhnUd3ygZ=3$Y-e`Pbw=bOF~f`gGNGY zi`baW>+d>)JK*r8r7AAKq3E+JLoW2WFtabx=N0@2POi%hL!EdTfy5uX%foPNASQU* zy8!w|{K(c)v@N%CP?ZkP447yT=uHS`Av-C}atqIidaVo-K^~voYx#A$HuU~+K|ujR z`Fxa)Y6jUYS6oNLcGn3~2KqlPZaG-px0;s$CU5^7^77!%WM-*V_9m?OzBnpH!LO8k zYOeDgXzN5#AU(62MWefu@7_XzCL3!6dZiw(xz;0GGM-Mvh&nB)O%502=Np=soU2L^ za{@Qgv{VU>PXh5FO+7B4OWH-I*v`i0z-6 z*VEI}b)v3OF*0WaAliyIbG}i$pS)?mr(C4Lg?MQ&{UJ}!{ta8+NxmdQ2C+wLkTb zd;8nhIfri5Bo+bH{*{qlNFW;$&+$(O1TUwOo9a?QB!%?qod%wz&eNyscMk1iY7llY zFfdd)f3`wjroPg zwYBwz?_uZKt-MTnrqa^Vw`Pa>5u_aNZxUr@zX<4m)ac!3Nh9mpfse5vR*%-Wl>IcI zR6fOQWNq8>ZsKW*ZzSANud|bVSTcBeIPRl2ZyF}C-%@IVe1%cmXy-xJoidK`P) zU5@9Tp0bVG?h_vA@yjqLDjK`H(?T=~BYuiUtnEZn#nI{48-P6wTuzKpacnr%5Pi2@ zzQg*KMdH$Zh5eRwR;qLe76SsX5jA&uZ_>kr|KkGqo(coSEfg#ijCVmsHB~q+kbpUj z$Uj+es4NEQ@9L_}qLT>Q^{!PH6UfnU@4xu0@Igfa0!}=X=UYWFh5e4aN54) z&$7~nx1^zg?Z3CT7fZMMK?NT|?4Zs$|7PQ>FWRr$+u8ZuJKc(rMtA36KBr=zXaE(ewKv_SB zM|AOpCYef(#?Q~APk0|I49n5->Y`$l-{IWsN*S1G;z)zx9}XK)H7a*~Vsm)>^HMby-F<~Ge; zyppj*;PcapStD6~A&*n*FShu_g+QM}yXpy!=tyv$RdOiE$v(PaDL+Q(@Hj(jafp#B z70KgSFT(eSBmSfQWUT&ryb%UI4puP5%hXhc^B@HFv~=VLViWZ1ybj;8fF3t4jUC~F zr;557mgu}6UN#bdHnu`+VCfMpGjOTlD~4p?>&C~ar3KXJ2Ed60!_>OC$my0Qhe~f9 z9i;o7JB^V?WBP(cx*DAEA|!eCwBHy0~-XjK*NOo5Acr+P7)2y z3rKH2Ij%UXg=~s2>`0SgQ-c^q_DnIknH@u6E|F9cP+!173H|mr>_(kmpXCo|uYvWl zfl&OWM=c{&*ae`k!(cj?gb;dFogJbm1*gTcEtBJcL01{#B;dz&Vj(gG@7T>9MJ z{&ANSj%3YSxUb#aN|3pNX-rUL-w;D2L4ZDh^B%AFWkRU*J3BiMpOo@YRe?f;E}SJ* z*~HQDvR(ldv`2skgw_8~cLm-Hz5|>-O$8y6`$UWL^Xz~j#LB8~Z*OCp#9e;Kk`6T(h$lezg<)`zqPxrdSgz!=yz`GQY~0+Z%7=%=pHdvriFr_su62hK%tNX4I<&J*xC0r_!vkwh$MHCsyJZB31vaCXzjT<2-18fbRC8%Yz5^ON^96gg2NtMkT6cNx8h87aYiXLoSO%rqX zhn@`$=B1KF!%ddA_BgR1^M^hN4fT-NHm$m#A!B}PM%!q(@gvkv$A&rToeIS$ak)?c zE+Zc4HsE}mcj=jK(^=AHOYgk~`8tBJ!9F>OFd!o!khWO1-+s;?W{o8)syLreQ| z&`NU_vg_+xH$j&Mu0uN-(!zTxsQ)37A7V&njKS@1P6sm?L-Il23Y%tZWkrAp1J-Nz zi{W?x0ss-uaqrnN=^xK^aZt{kw58EQFMaTBJXD16Fwp;I%Apbhh*&E~iU{CNE5z|6 zl?FwAdwg4-m`#gv@keflM2s=!#hJ2ccSgQdi?s3( z>!_8dP}lI@WW-rFeBeSe7mzk1cuZb?Ag}zr>H-SO ztXJ%vk7;5Telo;xbvBX3cu>*Hp};y(bY5?@=ZKHA=uI=SwOxDbHDGl+Yh|Xm9nbzI z1ZogMMySp7Ti*I^RmNrJ2m+bC{r!@ts56BZa$M9`hZj7a`A(G$hLx_0D;2SsQ9ZfN8Us=tfe+}u1r zgiZ6}suU&F)qvs_!)8e^`$6yl--yWiG)pZ9u!30SF>vV+{dZJ?QALi!BCGrNQ4Ntn zfiYn{ULhgu8f8Y8Qj$?Yf<%vM%>W|3iq#Y~E{)tfZ#y2QUI?wh!r2tz4?_dAL{Ad@ zWh6Bz$sgpc%=rIH*a1DazLQK&DDqi9MXw+TAV3jGsxTP{&N&z4GF8EXF(-F_H!rErvhrjt?+r$-@o89jMP$NYC0PEp`ae~kaT)T`ac*~FY7@r}A1~QPd!`C;s#A0`_ zd1^>3;zT5cVbvl`N_1jvQvtoyyt54k`k?!TPXH;Z4YBJhQ&Sj7#t1E_ z09-<91GTl{5Hf&HI=Xow_Os?PCp7`*JB3SJlo}l}AsL@pKME8<)k@)k;)7@adp4ot zjv*Lj2w{w{qjVQT=qzMj-zi1xLxz=MXGa!b%`1dRS2TIQ@~Du zTpIErG50MT2;D&Nrotua)MPxBNLZ=QhfaP`hZHU$nE|vGhFyV-SK&&k_IpuRu6Q!l zc)ybry22^8!?fKX%JxHw7rkw^st z5j1f+K&h_;;Oy2&CKK>Oumg}0a~v}N!RHy6|M(ztVxe|%{2fD+f)^iDI-5xE{eb3O zWF*00c_YriN&R@_`%nFN#p0gmtsvoufU#9AsY$^5;j+!)rd<2u5Ft^GDsLVr?A4E_ zlQ(6X4IIshpSXrU!;eaT^!kD~#A*m=_*%NSriKVahA=ocDk|#!>nI-fKMUK9ApQmX z8SD_8JlJ(eG?CgY^Cg<3KbQ*_uVn`ex0~F!E|IDAK_iZ=J;(0$t$@e-e2u1-o~lRG zcJUO)hYRT@o}O1A&Or}a81!}dSqL(Pu#mzWC`60@{naHKHH61jL@cA$Z3#v_whw!&@9Y~!QuOoQS zitUiupzYv5$Y(R`DRNh#NDjdT1l%yk0lSs>N)Z&2QKul1gp0QC8YtQTyg&0=2sGa&$(3F!>nh3N>^4_8*RUt zK;0TL?P9$Gjhf6pN{4TZtyU1|pcEY6f?3+_?>CI&J;?%%-vC#D z5Ej+96lB{_Beu`jHa9ot7Zh}ysdrU{6L;&eKy>AvR|>b*2eq83651Xwl+TejP6|Di z?KOW}k}uWw z`I6i8r+{Xs(^-ytFPo;JQ5#rO*n9XLjpZuN`&Vhe(ipFF#DnIM({^y8s~EzRYslrg z!Lx8L=w+DC5eV1eMcCy-%d>xHrU$T@Q8JUW3k27ryP z|6Zt^ux;j+NMY=#S)9KqVl16C`igldibrG9iJP=2pmp$>D3J*903GG;m9~yk=f^d> z9^x8S#DBx7j1#%iUTyzYidB97m4E)Wi~i5xTEeD=;`UZ3&f0^sHA$D-lDzao)ihKy ze~rTKZSMO%>&CX{(~euk#F8ehf)~dH9Fo$wXp4*wS@8Yc-fjGL*H3rJ(0YC~jVz8n zcUPW_bjZC6mzQRhfXsMzJ+avnYDa)$zjfB)S&uEQQE&P7{F!ugup>67Y63I<%nHAw z-W#C`<$Lc+Yu^c~;dtbkehj@0AAQN`3Y^kiW`AZrM%!dY?d%o#c{Z`pj%!bISyJB> zd#m^leh!emAY_AU>BcZX7pHdiXPtDamFZOawd9kL>r+cEB=@X8ZgQsFFZe_McO1i? zIm9*9JX=qt_If7UxBF6wa#Q(X_$jYhem-RjfG?8KQsp}LRX?*lj{W&JR>!u(ulZv@ z-_OE`7(V$Bb7`Izb0uG+8hyU_TYgHN${qYn=CoOHQ?RBI6G!sWI@cE>K=5*mDLX~o zO$jtHQoZdsZY|~$&sfZoEfz#a@eaN1(k4^VzN8ekI{I5fJcK!fFr>AxHJmm$nxcl1 z(nAdUb;>KeWnD|rWRILt30-UExFVJ)O4a;Zko_iXB$o(1Yt?TH9eH~}@>`SxHoiae zAC@kY$2`M*6L*K(o=rP%7C#*@H8MZk5)pDsrB{d(iGFsm?`w#B23Drz{qskCTak|o zF95FEFb248hsKHElthFF(50evRKB2iaPWz=UsBMEFYUWO@<{^z z8geY$ERPE_2BSZy=0R>`c=PX6iDPj>Pu=pCN||ToUrAU6n*@u7^(njM=*vg#E+iO7 z--fQ|8_s?$c5Y7eNTrvRQ>;LlwI+bzjEVzxJxpHLiaCuk>c()Nb-Udd5x%=p!bXxI z4LRv*A{Ulk0Re6;u1EnL7iDbK2#O=}N-2pQF7X_}B^>*!MrkiQFWx4Fa^eq-jASCntrG__~hU$N~G;lE?d@(4q+hQ$hlE{?z|j(}HJHjgb+>&1tW zSkHg%mn;IJwMS#GcB=5K(uEVy_`e9Acv_+_cZ>I`c5cQ<^t5V+P;%@>PwBiO?~qyf-J0^`>->WnEA8?p zWzPppq}xpV4JH}SPG10t(DwqC(-C+GfPOeL~>-aMyzThdkZu)V3kTltp zy?4uNy;@$oiAt4Gd>MXK zIHz-aYo;9uU0JRiO2bN`Kut0Wt9v+GzTS(UI>NreU~g z#BQnjFDoUhIo16yqw^f@03C0IXeH)l)OI2__X$xTOM~&#;XML=x&h9xf`v+~2bk>0 zkz`(rnB$0hSNug?qEO#6XRg)wS0kJc>R&H5jp9$^yYj_3N)HyRc~H58$NH0Z_>vjJ z)LmE|$QfVEsd5ggaB7!SJ(hkLZ~v?-G`Upyjl0r@sQbZ$ECxpQ@Q&NBe{(Nq(Sx&@ zYBpFH0)brQ!WX;ai3(%`{l-2!y=4r7EhJfXoh@;!@NUo2SoQea`~K(P@O|5|dbvIC zDr;OEeK9XS5}$=_H|KBNett2*Mc0@-3UD#rtF@~*4-vlS<(pW(h&J2W`ZSy4n^LzZ zq#J*Azt-o}bAHuk^4fC9AItK)r@by=!7Z+5A`N?fV1xMbFn#Vu=Bbm6r#Lg;GWPey z&;7`@P0kp&ec&S=E;mhYwL9HSY>2EEO)U3@r668_@_o29R|0f>3LlC7j5dRZSCC@B zW02XAiDycFfS31xG*2~~wIq5jTdBS|o2UHgyAEdxuGo+W{Pu1rdx0d_l?t=dBM_P72Hw|4nvBNAm!-#+UA|FWI?7niV*vYi% z$Md*--WsYqG0N`VvHHc%GM=w$YOcHgWy+RoYd)M?PgbIFh8sm$nx(rm*C-LiKpB&<(};C+lAFC z>i?{$Vw)^k@+McqPr3j3;nF_yj$^Eu#c!Af+0CV$Ah)$Ki`NJUQvboH`9K$BVG_3gtCRC~g5$zZ8F!&b- z?%R-=)j&~H9W|p%w%OIU1-@>|*>0n9)aY@ODVeu6?ZB|fGw}d%c2Zh043)gQS3{g* zxpw#VCuy;)k9Q)kl}8k>cboO^yOGnJZ1qQuwe05K&6cU3JLUX+P*A$5|Cwb-9P|gN zf$n>*(%Q}jf-kBRvanwKzDgQ368sl=0c9xH1|K@ zf4yz1YSo(Y|5f+i@mTlo8?aSLR)xw66_O~)%FK!+WF?u2BrCF4k`V3^lD(6VL?~3E zkd=fYJ3D)?=lIn3_s8?+^XGHFyzcu>T-Wt^k8>R7ah&Jxx*JE%h-rQBpXa~ouU!@3 zdh=La=7Y8mlfQS)>~&auxZ5MoWO=0M{;f;ikJ_6B=uXQhU-Z6P-*-TpnOECdYP*rF zi;ZR3r@jNam)ORxo*H?@f08jIyfw{Vw<~{6O#F_L+Foatz^{EbHBDz{)4J~m>nlt0 z^l}{Kf=fL3ZRfd=_Q_G#xY60P;~`HCj=C7V9<03jm96L=?SJ9k;iqNnepF5giHnzb zzB;;2J}Cbzl^+Dzl=b*jI0&Zj%eo$0Xa0{15I%N-R?TTZ?muSUAR3EMUdoe??vx6# zm-@8bIBYpa`SbgLYGK0son}Fq^@J0 zeGFqI#!_3;0_v{}yZ&^pH5eUnt#FE8{%F-i-Re4->!>&RtZ(qoD9f?u3FFCH6pyRp z59{VTnB0D4{K5OQteC-VM0)=hvbt@(cVWm)&1!PhpI_!Bwf5MJHp_fN@?&DZUH%)) z?DlV>o8=vLT4ze`Em-=v?cmJ9r&~fsSEn^4`ok_lq@lv@%jsX=c;6}c$29dTDXa6Z zZ}K`hOEP8OIR8FdUtwF#CkOtNceCwdEx#=s9e>P_zA{gaXCfzKA7z!YnX4mDH5O>k zIClAwyX3ZcrBh5sQE=%;N;rAHZ`~;CcTEfH?~ppY9<`ck(yy2P)>}GkAIy_G3Yhb~ zVZ*r%W4!DDx|-mtulq4VoiR@-bvv9@juK7cO;_S9rBfbT$u2%tx67PQ zHi5Kn!r{>%PZU#rG{Ym>yf1w%*GVO8{$8?5Nh)UN%~i|m)22*yLrjU>6j{H_2T9${ zPrW(1u6)VU>D#_VR-w=Sx2`v&+j#E2crD2@PBes9{u#D>X+5sU^F@B6S#rnSpaJvZ zk?0*3aeA>*W6Cc4>l`?h>tl62-TN10%LM*CPPE(j{kIvGtS7$2_pc0jC0ySVaC~-R z*Yn>uwK%S7Z0j{h4L}GM5^)P@jc*&>pzbA8-pMc>2bEjTv}rjkE-u(r3o**omz_}jxFbo~}liT(PX z?bz(~FMa(kGgBpv;Z=cqzi6@xS8L8VJW_l_GhyiV!ZR*X@zJ!%U}bkd=W&hkP4PYp z$L2;&_B-;Q=5qwsU(9DT+mw&y(53mrv2S6i_+mbhKi11;|LaB7LIs~qxyrzg1mDTF z6H+?2xxKD>dbR|PB<*wWvY)&VsI_xASIfbp!OuRe^TO-4FV}ZYz1p2B;Mlso&&Kq4 z%NMd*e(5hJyx-l;l@dLuM)R&LyIo+~UtPy9lW8%XZ*TCpee%q$?u%g$!&CWF1S7%= zUfvU0If~^sl>YI(QEtX`ATjTHCDV>`aRSP2^eZFA#w(^{^_nx&&!)ECXq5CGaEPC- zyI-KBzT<>TyZmDAeaom5uXY~pU7e?1=o?JnlZ)mV-#gE2Dq7JQ>-e2IovZ0`dJ6A1 z#O$9LuJcmeX!Na5`IOd8?XOF-TR9dVTCDa$6J5d53mQ!cW3NMXjj=D6ju~BD9(lb~ zd}GBgO7*45u6HAKf3LyK^Oe`+fgo>JAC=>{>#vo5_l(xN74MyOmymu-0OlZ5r){%j zeZ1c;dS;Nuj@xoXc%Q1G?|ZX?7_L&r8!ih!Lv}26xg0-J$OJ%SPWfQ?&X335VZ9!j z7YrNM%*4U*pS{I14!TU5N~c_uiKLpMsGrPx(~CO$ zl-rrYjISN5)%VF?3zi@ar@0ZCz+rI=$Cf!25mu!~-0Ej~H)5&7O%nL$Z= zfj_S*DrAKZB=dfI!Lnycrd!eB#t*-$!>7cgj&dWF!q>-g?=(s9Gt3IEf4px#gF5T`JGp1KH$CVB!Q(3qMIhN^{Y@jg z+a^%uHV3je%D%33rrOr50zA3xH&(0cD1QvF|Yb2?jx6S23H)<=9+rkzZf)=A^;lmGhS#l8Q3=Q zvTx-PIa9=+=XT4H&1-cP_==l97spU&uqFB;U2IKL*50(+93mQaMgJXb(|Gz#Gp8_# z%p~Ja$p4o2xXSNm`acEDu1STgp1+{~^C(A5-tCs_Jw*=1$NS%%w_CEe_?jVHER`2` zB%H+GZL6fffiFr&zgcr!c+Qz{LvOe0)hhM=7^XN8a^bgOlH~t;)d$DU%ks~@HJ|$G zt~;{VMwEqqZm+|QYQMy;h4;6@q)6}I%%VINHhSWn1pSE&GN$|P=b91=)NKE^Llghi z>g6XgO!3Bj`U}T=m<@Y>U_v9Qu3zK}MSaiQWOc%f;SsWRudO5;KU@7CpNzUjz!cX- z{gIVxW`CBKB~A{7gP*C6-(#L0j?3>;?b*BYG6A<-kR&CwJjFWvJ8t*uA;Y1U5>1+M z*KwCts9%J7RakO7h?p{w3egPT$r--g+~|NCC2?c`@*I24ry+b$?E9 zjE3r-|77x=r{}HiHi=-D(>sS3s6UfvlYrNu8p%9!XnWnMEhO(gJZ5aOjJFbwxBo#c zn>ib+q{vS8Z)06zyy;{!co9^Xxg7~l52ap+U_^inh|?gHiu84Uw)xcZ`0gp8U)%Bd zB*~eKb|n71*E&I#U^~Rw`h>)u*X7;k*AbGQv|*AI|CW97*gCuK8H0~-~CXTLxRP_?WEK zM~aG&z&~sv3>6oGNh&`Ajd5IRPT7XaalFDQOB0!kBQE*)*f-lPTT<7O z4lj^cedDutr#a1j!1jxn!}SI~WM7BO2Dp+QT3qyAHXVvp%m49xLx`)-O=X(;g-^Yb z)S5c^7bP~7Ai0fp?9NksS9M_ExLkJ@N2r zzRLX!;yE!ZQG4>QWxOr6^~=j{n?ma-H|RnsJMH#r&oydMXez@ltpxp+@2mF;@9 zaD(n8d5b636`8)hMf(6PH;sIYZ)5dTr*)sGY8C!x>cZqpIXIuVl|Qh&GIJxur)<_6 z&-gc+*;#+jO5~D;^E2sZ+I;C`SKd(3(|K8@Zt}D#O^uXs@N;)v+O(;pI!q=OC;y)y zfvON;w8W3An#-@Z#&$IIA2ylVO7rC{)A)^FO%0+N5z2tAa0~ka&a?vGCeM)$_x%@9 zE12Z}@P)aIdP4tE0>`dMz3S6KIFrbQpJG3ADDQ_&J*F}+ZMtkoCo7XiNzHNjDObYI zoo7StS^ShCGAkuyU&#iO)YauYZTo-T(tghJ&pkAUl$tkB23}mzsU0h-N<3~hL_ak} zr2B@FWCH9)d%ANemctfFT`!;6xfjf(6#nm;yI7vy3vH-yrqo*9%zt%nJuT^A;xScX zOL9NHh?F>P)0O`G-!ZvY3E<0ydp)^Rb$4Ds;gM}EpRb#o*!QSkX?%j7BQ}dI$n>`} zfb7SobT|$R^;@0U>AL0nf~vkjV-nX%k6-8a9Qt^Ii}|^YPI9oT?&-2-R;PXc3~`G8 z&#S(T^OBDOdq>~qwY<71AazUX?d4A;Hg{dh$){p(Gx+KG4l>MnQ1G>K^O>>nzdHB4 zP>S;C4CP<@EBBTj^x3}aD7M|Ogw4)dV;S&gpFZmOP4KYRxU-%QgKipe5 z8(j)f7T`_{21=bzTL02&eNvb?bY`<9quJ{1SJ&Y{uT7i#`D0nFqsi=h@H?RVxos%F z|C8uNH*TpfqqkUWEfOo$U8(XXCiK(Qf-Fg%i%}lcsr+?Pjd$wlx*Cf%Ckw{%9X|Yy z?EiJ?^_LeO?mNdzyB3$Z2ZF>duT)ryl!rZ(5}{=dRcc)NYF#K87uddcls&Jjf5~Eu zb=9lFv(O+XVR`;sec<}6{-RgHfSq&qWUPLhv54T?2Ip;3n=Pf?ZSl%M1^x_v0w`ai zJYMwrHAJ9WRUWE4mA^+nPNL}k3FI57Y;WDVr6CZ_sj`cpjE#Inf9=J)((m7>fTbo7 zEzcK8@O?mX1r=63ll=b*wy3#*6NG|o=8Q|nKoQ97gM)*+!vnohmqQIzQy|(yTA7!D zdDP)S%@upl*P%1QOtq^8;!|Sr>f6HlO@x_EMPk7NLB-hg*k1KleqNVAl?R(hAyspH zaK@TjTHdtF&C z=ws`_xrQ+!J6S1jeG{udQa#849l27DL{4ZxOA-X=*HGR@Z~!*nop71pfzGuJ9a65DGxygBAuTgk=3Dxr|UiAF zJ2xL@plbj1s~TJ@NPk|uctL`{3ROB`U#XX)iyqnp8*y{Jds8t>`?hL2Q5a79lI;Jy zWPFlPt`KZ{kh;afz4QYHGgyVwlTdT^J1TPLlzmHrg9j9TG8W+>y)n zo}O!=ykuRo?fBeuiH+O9{Yjbbs>n@TSi39`4Hi5xt&EWMK>8j~f7Ku7q-gv^*zDT* ze(&?uMW|&KyQZz3hBqH{+zt0geE*vYcIX@x87XFG!$~|IA&UcHw6m$U2rF)6ZM`iebV86 zT~svnvwMRHye%-QND_`H!iacyco6hl4oXmUA?XTkCdM7$13>$R<&p>4zGdVN_~9U6 z{QAWX@*cPyu!rix>fpORtVw(6mfdPyyZF6qyGegs{>ztSxJO`u5JU(tnB*t&ONWnz z)Bfxv_wAkphZrQNCToxAWh^S#UpP{IUt05$IDLk0Nb|*NS67!J7~i}M60OehhplalgktOQ{`g!xROhAs%=%nbzy7~d zVUq6y);CC=xWQ7%w1O;6xQCLB7~yd`$^r)iHyFxD2w-ePRzK{$-Y1S}piCjXKkX{2 zh7x_k8l_jMu@xXy3_4;cCrl0c1z->WI2s^@;wqvrT-^1iO_yS?jakFctkh)Nm^1zo z#vbxN;;Ks0{iS}pTD`?LU_a3e9zMXv&W^aJ)uAl%Ig~GG20=6j4VmDHo-2c~loWk< zQ10PD3sSS><0-s+Q8qccr-XM9BNy}a75JGdVPr7fIJhc>tKyS|i_4zBq8FHGv;2Fq z93**dJEH5hWVQr#bzR2V;ZtD9pf(>Pv{#vTL2+GmB|E`P`wJU$HtihlN zj;;m!C6N3W{31TTIj90#Tsu@h=$KiWyV!UZcC*S%GiFa%okQa0m{wjJtPiBfgzgNp zJLKp;54(kNDE4fD#tgKten>b7-Gpm`PwzBr_!S{jF!?}S#N3QT&zbGH#uZ^$XPDfI zIp&YEp*0N+w8U7G@$qpm*9gudsHOpJV*y>Z$|Z(M==1=}zb|3YA=0PwSt7r$bbt_J z2s0axN#m%%7ZajSpS4du3h7!S%U*EVP`_d-$aKAE_q(Cl-kHHVV(7@gzyPP{9~f+S zXGm}xojV6$Xo;DsmHpGcLvHZQaP8l4{>GdUW1zVP-jUBJp67f5LY%|Brzgqym(>}F zu{==35OXm>fUOMN?@gQ;+<5Ssf`Yb#%z4P?OkP{X{AxTTBCzfhI8WKF&phe)?_z~g zp}qxkpKz`i7y!B~`1cO03(y~yUI;zPt-cK+0u5e@*FR|xJIc``I!dznyn2`^L{%W< zy@~I86Ah6>NO{0nLt}tbPV5DD^!OMNHQ}WX3C|Kr?XM%RUN-CtRU#h7`XuFTUqnO& z9c$`)KdiLPLyk@}Sh`V^P|;=kX>Lwf8lnu4Pl=EO@^F#Z(BHu~$qthVz5@vgM6&%p zYsaP6=PtC{6zz`RPe}pU2qG%Y>{gT4Kj})^IE&o7#SP<^Adw1 zE;)QT6fK;Qs`w(DHw00PyNnnYV)pFW10A#L@FE6zK&gg+9hy8e1gQjFX*zZPI;G2> zPQRKf%&uLLk&%AM`?_=_^&?~YB|rUfX!6=fn(P(kFNI)zDYjyn1=~m0sUdLmXm7aB z+5tkO0`HDHMQ8+|P(Y7?K!&tG)1puLzG?R7C8qM7od?TxZ@@_Jd9f^ih1(ngYjCGF|nRTc`hcO;bY(D`K({Z ztvhswXfp#EqDbtypw+9hXBO8;lZKY(^v;cXhuIjBh8d?n^(h}?A6g&UuXA+Bd0^n_ zsKvVZ9R8i>v7z~58jVJv@Z<>Y+rj7ue*(%bEX3YWrF1fNc&mj45vV-f`DezZ&!+D} zaKxdq3$*9I%;();`Vmz4h7KDMs4igNa8zJ?a5oVKLuawqT|XcuHK8TDA<*v0hI4$4 zqKtpJUC;85hCdZPM*AotN(^!`4D)+*wOgo8;fBR%$F z_K?FJI{(Gw7t5zypaSiVcgL~avbsBY zB5Du;kXY%N?Ay0b^H~U*GFTjV|L}NrD<1~ofY2}hR6{Om(v2S=xLEOK@EvTnvtT$;WRK*D-Mmwn+ZvO6k!4 zEp;(3J8V>4FCjw{h#u3VXs>>%!D|A44$}Z3UF``tDmda(+NRHRv*j2RaWz1c$6+Vf z$^Fx^*{$pbLwGX;+XxB>MJTdexHl{kq$r31!upS$JQ>j3+$^6MSic#g_B---rD%PU zvzLbYmh6m(nk$$G^614WU8(zB6$e$2nnCtVE_M7%50TVDt%SAH%1XP>g2zNyj!ZM6 zCe787j+WMFX^n@62Ys0!@Q2EZ1~Coqr@hf@k_3MvD2KqE4 zs5(Hq0Bvi&zFG-wU8314juS4lYXX?f0o{|xs3m4ry;_IdnT*1LxEqfun9js+Uv=~^~Jj)uRc&94H z&c?>%#*K4Xzf6Uh8S``@R_LD8JMWjn`+r=3zM$?qLhq_Wp`=G_eQfU?-E5s1E(*C} zO^VWpSy8M~MMa*z&_%LT^~_4h-Jsx()binu$oll5lHUUBgRr>0tE=kIpIkF3WZ|mC zQ9uvC1J|9Wi5er00OloUpMx7dx7llS+_&oX3ow!ng&oPJ=Z;gt_h%Nrw&wN$o&P&O`S%c zea|Z>E>NJI9WW9o7N4*4d*(G;;i)-x1-%<_;tM7z=uzg{316oeq?@HJC9VZIBf$%2 zVY%?A$VsBBprPg^HS}r_IHBc%nTv~F$Gtxj6Nh^(;7pD3UJn(#SU$EZh^7m_Ea-cKV@;|)fNN!V+s?i^2nhEKG_N^Eo)p3Dyw zxxf`O6g%XG)h2X9FzmsOA7=`U7S>v=3umOfFb!Pw9^KiEPzA19NxnMJ_c?HD2-FK& zru1E`Zz|AT0_j?cLC%M?mbq^U=KNPi3DP$zw^`kr*ZI( z~Eb_Mld1Raqh0Ixt2qoAohy7I~FxFmY? z;n$WR4V{AUv-ek-hw9#%I+5QH)T3<~q0=JVSH3>tS2`Usv&NvKtjnJ4IAS%edvDp1 zoE0`QFwinA`a*U@Wfu+>Oc3nO)Og1U!p$L4)2u$v9kIOm^Ucf5>c(+*`dyS0WUtOr zl0Ix_u__&@E_2&;zIGo^mMeFb_QPjy2M*8f8RQJlN;6cCyq;lw^~c%8_O2TR#mUJ# zLq2ldVL}(oiCttfJ*92C>ecfWCbj1)?=N3lp1ZO5>tZjnR>FxubgR~@Z)k{? zgO!jTtgNm&0Gq(>X5_r3LaJBZKKbVlp|iMJP?lu!5~d9Y6Bh`_#XePFN?zkZ-Qa&U zSC+^++S&pkj)!viL%3Ph}5&FLwU4&)~GScRQW-x>bQq+n}=CMgYr77{WgTu(JUGX<1D>q}D zPgh)&WVh_Sedgc?k8|hF(fo_ImOZGcsnHQ>@2xj%TmAi54hycVUA%S$*6P-q_!A?D zYLTcI_TpKv^l^%ja27woYe6$7NT1BAs-71WMJ?*h&?hf8k2pQe%~b-%Gd^xp)%|F7 z7&*SJVb`x;04Z)OaGB2YlT8F7D943(yB|jwVeg)WN;nxad;2MUs^8gU@zD&_(A5HS ziIn+-1_QXfgs~D5($b`13{oIg6N<<>iij1qvIbL#<$4Pmr7=B=5jKK{m-ihO)VO?^CQG|^LpI>FUzQ3mB=iC!=E(K*m~5$9c-fDD zrsfR=C%qUcgC7&smh#~Qiie?Tbvz?Spmm2WO-fG>C~?CvfJoA3sd7JowgL|jh=^F; zcH{21J0p+*!Q=e?eHAH`k?+Ao1Wyj=nFJX!RfoVcN}?$mq8_t+*&N+U+t(*f){M>1 z48a{RmXSUh85&8Sv|<`c+}`jGG$drNpR5dy^JVbxsnK>Cj0Hb7*@4 zA39#Ru-llG*Y^@O6xs7JY3VT7J{+(NMFa={5wKSFBaa2t@S=t{+Pk}<&?Q=)O`-27 zl0I^hBVNE~;caokpolK+vPGzznv*lpUM*hx^ytx}(__*;8?5S?yWPtYj?g=i{_5!H zHHqbA!0Q3kMb@Vbu_6FR7__muM7@_&7RjBYb5lP!%5U&RnuX56N6=1Pdy6;=z%e$y z`0y)9|L3jN0LPvxr|5^oqw5?vfKdt-60dTB4~2*+bUgsNCYz-#6dzO_ z@>CW0IMN2U)hz8%t;WQY^pNITSzCX=4x?1k^7N^TFJE}R{&OyRAmLAQaukJ`EhH#w zWK<7645nV%*pSu}yvKgJBD`%3h@hhqDyNGCbMhTx1x66bzlc<5xYo z9z0zthmn)qM0lbv+xOmo8!{MaoSa6*fYa~4MLF&664u_Wz1wC^7DtMYkehGr+3I0@a6?V-StAf$dL zPoBi~Nk_GvwI~V;3k$x#>A-Qg8b=4&5l#;^E$xR9r3S!h%9i)3ZpuZlBJ$xxr3MVHg#5yUNnSr6HxHUNYgw#<+ zCL9f?2rEng0APcK?+C|?hG|1nnNKq;#7M2 ze0gpzEqnpO#Sa(nIzD1W#s2m6b@bFiJ0TcTWRO6%DiGgax55v9OH@dyYCVN`x6r~@SeK#vp-2(s0+YjPeQr;y~0j*jB^5}6X%X>GQV7->y7ra?!Cn*_?$#B(jW(?9DWEk2n`^Wk~N4Tmse&B%*@P9-@o6*P;c8@+Q3TIFK7pwq|F!qbVM*x zHM!Ng`RC6NK!VfdZ;FblZ7_xP(4Q_MCGD`u46A4vdA`?&iqOfsySpP)0lp&kbPbWq zG@5MU?jcTiXI*G!V$$=a{{mkm+6*?gx8H*=eD&&8Qeh$CS=(^?O0m)Av)*R;Fy}rz1Hdxu4RM1UOY|Ly7K{8t3rKoy zI{VU2>TQ>3ca~ryAY`CQWTNgYIs@2a{txxLrhdI8>8pN({r$#$ata z+=pmvFgw(Urj1`{!X5<>0SqED6lxK|5#vCKQJ{XPtEr~r4SK$WwT%%@41G-~Qk(A) z(L|QE(V26m78U`qt+nD!3Nf5}>WYd)&^-xSeFS8)Ye|iY%!|8%C~#!|N{~kqH?|w( zL|KTpj0oc-5r{Q*<<+Z?+N`z0Zj$WqWN_j{AV+kS1%it7L-WD~f7A>BV`pTdV6hM< z)>#10^4L~+?)AdLEutEQu=_p0QhaQZHIpCTu=z+qi7p~=^6+@CZM!}YqSXzy?x>P8 z)%LrB?sy6M?iV;VSBBLh#>B!5p|XU_G(7JP?`b9cBzRKp*?#njTw3Z62kKYMSq8)P zw<$2t_ZKYAXV31CMt2cBD{&-X(U9#B{h5hih3N6o*hom$hi2taYUXD$how$LzrM+H z7|;H+wsT<-yL5i+?WTULfBxLnkhz*ns+o zPt*ME+s9|8kdr{mMl%Q#q+b{)^n1u_!@gkhT2e5H!{T9j4s#i;9b5(smxr_ZZbcDLDT zAZZnCu+^N!ZGl~d*aggww>4tW5;@K^xc;cN&%0GaLa z8o-|DEJrxFwXuqgBZx0x=7=y4Bk8qubn**kM@RVJH&D z8+%~`U{YBr0ptA3?y#;n9YcoRbgwEzgoRD)?dgdRfJ#ZtP!l@D<2Vtyn7X&EogKih z{+^zm&kpN>K14Y9YOkip*O_@vM`f|G)b$5(au3afKxhT87==dmIq$L((1Fhz$c z)Ok@*R=j@wi1qdBB?oOqVw;f`0W8dGTNSGK@gOPm+ZCZ+v=xriIh4JEM-2|q_wtgG zkdVN0#_-AI$;~*zt|6+dH#54c`Q=|#j%aoQ+{6TXIX^#7m}2yafSc@&36@2;9`17K z5{ogj%F#ffY!s%jy@-~GBzkBT-Th=ywMR)30&kmT40KVL!Nthu{8+-LQ*k3Y(q>Y3&e+omU6 zZruV!1lga4p57-025aB69aPg*H$+(ebe8z%<_aK=-rz?g0JImn0}Kf7k8U_MSNeoR z2|MJ28WTY;LFY6g*u>ExPIr}{H)&?ea>JMGRD!cW&?Ep315xx0I*{wl9bszZhE$7{ zb{l-0{rZYC!rL>5&ZNQ^-T+g@DFwJ1PN#VvYO#!rR9o_r4Fx_)?N!i}u#U&+Vk=RHCj>lvb?zM6z*qK>4OF9vpT+H=t(FF;1uJ=geagu&cf__RG77V0fM<2X|=pXxS^XJV3el}qcS?XZI^C)1VRA9-hM=Eogl0u!tVIb)GEki|^Cq2X4 z>hf+`j7z4;iwg@cJ$TA^ELejQ<|K#&t!)g)e_M8|fu`e8rVz$@Wv8kCgb1XqA`Bi- zn{UwH!r6I#%f-^!+wMn@o`OLzHSxOJ3=toKWu(=^fBqop7A+RuDDoFk`Z0X?l=3Fp z0xQCVq9N5|fg(g>p3;pEV>@i3M1d)6+#cCi7OtMzFC7f~h}cUYD`Ao3P>c$~?pa5K zPODq3XfODeV&y*do9bDuOr|~5)X@p}@xuVya(sWJh_^Lmc2<_Do9fpD^~{z|->oF* zl_+Z4`E;)`iH)qX_Kt@|>1X*kS!szjd*f^4P7=)>9R#*(=IXlCDa~caf+d7y4f|G; zxp^5O%U{}@|F2(0o&b)}A4Ma(wX^3ea*yX6Xd?tCMe1v@`nO2JYutnA%fP^ZuNrte zd;rl;9Wc~g1+L!v!iNyu9W&fa1QVVRlw~uC-0WAA zV@VYi6@6>s*JLD*rf!r15o*5}o9De7kL;<&j4E zq`dIg)av;uNy)}ys->N$Iqb=vYR?17z5My&k{H-*<%V!5zozK+s{tn{c*?VdEc3Fu zRdYg-J+7YdQ>z04+nRl_&Y)Eu5wPO%OIZ!EZBL7Ph0f&!P7h=gcpR>fGRzY(=c8iv zO@;Y^2$#W^N-W~d+i1f@*IjCZo8RjRWV;~NL!hbKA~x|=Hrifjq~!A?Sea8uB`SLO z)i^E|gC}&qQSJ;z#H5>xyWD5J)otK;Y=W`p}e%y1; z{g0d^yt14h)(sZ-fwftpY>k>0%njqHH5@-6P(;stjb{>es-{02WNn;XbK@0!B%Ub? zxF5tfa&X}cy8twJ_Ga$zTE2;&2S4q>nLFY?5Wk=_hVGq=^`8Yxk20LQL5Hb*88$wE-0(Np=l;WqO99Z zgZ(pGYpfEfr+C3__o~mAhNg<6@7RmU#B8-bk=^>w^#8|;4GK$}N#Mfv0;+r}xr z{CM5k)cao@w$a^?Kyr;h4IMHo5fNrxban4aPENkSl2s@7z7E(C(WXa2qSl%Nu@V7l zU|dmiR~JpF(rm+Fil!|+yw9ptl4v4m_h99b;rw1t5TPaV)9csGX@~2!S_Gft-p%q2 zX>&d&t=A-aGWMYtut{l*HlL!MonYPgPQ`#H?`D=gt2+5*C{&hzr!AkW|K!BMw&B67 zOeKO3hgkYw&#YiXU4Q!B_Mm{@{Pimscbdn~f9}l6$haH&y!P);%kO{kw!~yk*dypLz2{`O%cJ=S#=tAdun_zBVf7EW}bDPm)^4yWxs~`8QH-!>h+5a-XF&H?)edIoQ}iMC_ys5Uz2_6?Q;jlDe_-A zu`TB*24iTfPiF*ODnE0EE%1vx?fLP_kl>Atr?TKG#62OEK1$EqJ$#R9Uesq+6fu%u z)02CAr&r3bCQ7N(hQV?QCodWrenfPC99Rqr0ASK&kbBKFD|;q;?VPMQH^$~H2f?xOBGyNy>Nu)k);uClwcM>V#d?1KBw$R(9T5~d z2~&o62QU%QZXF=#x9{JLRBz3n^XKJs+E+eq5k_{epO;lwJ@XLP@M5wL1EofKlHu>U zw6>ZnLIBoDyg%DM+}>)@w%uY-MO_>bCFnu$64$P62eiz%iDD+f{cjtO zFi@d+^=c>0xy*cUbCI%nA$3P_2Qu67m&F@ooL7XzJ;0D>G`0@pv+H<%=Q%^XZbx zQBXk}IEF76wA?!mgn)5)|OGj9OSb?mwQ&6v06&?A-rw#7L0%xtk# z^$)#`s}~x-eJ#mtwrY29^Y%`t?;rL|OH)L!i9FgYzx~K-*4b(j5=Ju>1v%~Gd)Kn= zqp<3cT|J*k$w}`@j)Wh$A<$8Cy&74LcQl@l(t{MZKpjVzqAbxLq>vK)aGwJMYH_@>nhIPZLlFub!-c=Hd ziA0B(_VmkV(@Xj@T?MxVpP<%V&pomd6W`L#X{8`4g?m17P#Bz~9XQ8`kd-hs1F7x# z^Nj%~xF0>J8JbmdAom`)uXX6&)02pJ5HiZ#1Y(7Bx`h)c0^yvIF9A0=F@IMmGp*QW2*^&pIp+1?f4|V`x9W?1AI0C?e z(P8xdA--#C-LDLLe^T`=KPz03c9G-6;_tMw1G{ zKj28@tvnoO;-BgBn?b&TDK(o!i}#J3&yU6>$F5wvmSq+X-_VPKG!~iYV~x0sFxT;<+kPl5FL&{-Vzp=YPrbd5Q?nNc?Rt<7K(*>B%+3mXjv zF6wTfrKJ+!Ss<4HaFpmK@M!Wq00Z0v;xd$ZM2y?pbFS6vqPuu{0a+J9eL{d zNw{Vx20Aav|v5 zf7>M^68-VxM{!A1TtJzT;LP&cjy!p#1K(ckqY9V;HukSN<{;vXVjD5d3r$=_O^qn9 zX2eVu)Di(yqe&*pWF^~k^7D5hs3ba~qwEDL3Xu;+s|i_V`aS=E7mp1g>ETBC=Z4R$41)B$EUc!$O1ll?j3~y1EXQI5JVin>SAY{jR(3z1ZbKoJ&v!fC_`E zW26x#PDk*Y(d8U*0G0bri*}rB!n}%$Z?EWX6q!V#imQQA-tAUvi<6;9kWfUyO|^X@ zh%VO`FPiF&UJ)Xw)6iGYn%o;jcTUJ7{D6IUE_7JpxSI*0+(8B;v8F~Ps<*Jih6p53 z8GKPvvixB&byrDc?>l0QM84!7P!foGBxhoye0FO>cK6`WR%>reBKymqMpY0;5_|iq zurNQ5n0N!zioh98M&rT-azL+uqD82$TU*}-l~^Qpu+~H%+I&kaa%CiTfLYn0sJU_D z#$I-IC1d0L2A3{@y6>3lti=}z`ig73u!zV98xeGM1v!x@dmsZP@;P7|^G84n0D_Ox zn0p2Y2u>Dg&PbDuhnHumI5d3KA`8(yAInf(twfs}R@$U>LG<#{jsP-T13>caRm+ll z7Axx$u?{E$TUl8VEIYs*pjpk1mDw~&GXZzdw&cp&0%BNptNZwvCPyn$%QszNK(}C~ zql|CZ*xb>|!h8}izlb?4kmMN0kpH>XZ&%QP(vs22YPk)d;9T8iY5UINT#VLnbbMI9 zgo>CAz0%0Zo+<9!wlU31mqM^Hz%v10fGkCW2!@~qEb4&OhYpu&+!r3yWVT@js#B^c z&HzpYAaUE#^vYRC7HpgxH4!%vQNhq`ek#w!+dL5}h&g}_qY8xnt-&X*gAD;;N%L z4p(9$ybn1X4GtQ3F|I3fhlxJM_({xJ0nGHYb#%uC|6!7&qBW%6&nZw46R79jwVfzL z6cj}GpTI(Ax7Hd=;lG%Y08Nd*!IRfShkRrgv&(e_LCDvKM@CGyS0jNS0C9wX)2?xX zWB6%s`aIvi{5e5b>V}4f2Q!Wgl)@)Z_7YrctU6&AQQIIWl^`r(T;ycj?WVDn=X25w zlxWHcrU4;ULj|;~tc>8rTMkRH(-9w@-?Vq*ggnoKq9=ya%wX7nLd;NzY54D+6?oDF$Y zI=o-=BMvu@72Fj|ooZfQ9>LxRcyea>I@uO~Jc*}+^ysSAmM^{bNjW(*jfpZu1qOx# zH4sry^#1IZhUrVcN7G@wz0v0bFZQSVByHV134FvJHhG=tb|_% zlJI;T$Qw|&PhnolC8rYtl;o|CA(Z*svKp3|n1u)t0Y)a8FP8(~0k0-Uw~qTf&XuLD zMrTLId(@dPY&moEAOn@de>Zo*)c|{is>EoF^Cmmr*Se&mg+)bucAE=!rb01Dgg}U; zfgwXMC<)F0iW@Kn0HO)E4!#d+0|23%4-dwIDujxSo|Sa850CYL&|fz4-}LG3-6b!i z+y6?D;M`UVCnGqictxPZ$?54^_Iu6k1o8~d0S@T$ zSIOhY-%GAxM^KF;s>`_TYU>h>64$O@_b?@FD5~Py5dMH`JnjjJOUY@uueCUVgyG%0 z*Mzx4F|p#-#E$@f=ZEAstgTN&mjn_M|C(uGQf1Qk7=m2fF>zPvTrT^siWs1YB3%+p z0ib_CwnViM8aV{oD5AgDu4rS^j-E8<2|A~t__Vw{;YbkA`w^$UNsPDv+(0C*tLNbY zYGWYkLLov=;%xfYbPB;RtpV{HYlrtrtx4pO37QM0IzU1Texk?G{*{}4^pubMxQ|@3 zKEe=8)zvq9n_Bn`?E_ko(&30})E0^eYH;d1^sCJBldv;HuWSV8B?>?8*N`B-@5KQ{ zUW6$ln0<%&A|@*DP^PA)p~0X9uGo6rC8nmKtn6%mgkrxIH9L0J?sax{&j0NSHWwy| zK%R0zgg5od>z_!nL;r~s<6|zV(uQ`}6n&88i>=3)tmSV*9`5caz8brw(oZX8)x-)v zK7aGEw5<{A*7*{O?A|+0J&RPQG&*XTc_5WyqKm+*8KSQ z&`?RQ)Ppm!nJsv@6r4o_uIE0$oB<;i&3MRg4M5RGpWp@=8ylNsRN7@D>2l%4tUaiw z_n}4vrp{2tcOnFNJ!dHOSK{|{c)iofIQFq^iniHdk~GU+?ISI31&4;Hc-$ov~9^na7xKxn19b_p&8jsoWx^;C$e%#b?% zqeb>Oe+BCSt4kowh;BONogVxxD&}@Rd~gw*Gz8KFwGY8Bk*Y+vpb7;i4ueUQyNuUI z^*}LL`w{K7k&@IC{3Hs(P{7*ik(=MFgspDeK*9lVbZ{svo09c);?B%pR>CQ$03m9{ zz%Cn(_U2|kRAScU_jJ%To=ChV^=Q&u;gr;Nc&aM12Oi_K6?s}&1C3{g7&O0wgt3@F zj3Rh&2H_eM3n_(#k+E%A$K!sVn%Lo#+f;6^P4DgMb2zUk&SEV~FUiwKB24`VQ;3^% z^rpIH$ESvs)u;;f%a<<)l;|%Pv$L|Yis#}YcADRGHQd4){>X4})ihp@YA)KR-VdU5CgFTu02-)V84js7{P#-au5p2UdWP z&VuO?h$A*Fwf zoU;O^2tWggCyGH=6}eZQ2M(D7LjVR0ZUdYSjaaT&lFb2s%#yK3@m;6Jh-C}v@oCmlWpzWzC3qvi70QOSs5Kv#WM z5&;^3GyEm6Q249O=Aq=F#+T}!rCxi+H@R#NIK8%p$m0UnVGWSLB^%?^Ed#cHvq-40 z>OrVWz(SCd>+=%<9b^~K7N>-RB~MV2KlC1hfyI-9ECnKexYNy}-Ho-4;y)UPZ#J!T z{QOC9I9K2O_iR^5`kO9WFjC_@WF#j(xP12-KgMg1G8qxK!AZbA884M1sUa%Got+=? zEw~}k`$MyCxwaP)zaxewXhU9Pn@i!F_lSRi+4&xImdUetpv82@DsBVg!PSsvE8zLc z$e zdXE+ZRDsfaF2Dg}&V6>otGi@$uMYji9R4>X2u=?l&Df?^aDQ3`@~7z zj5lVYiJ`-QUX4G$XpxHtF{Ge?95hPsh0=bG)=_LZt{t4J7DOLZVFaaJ{v8{km@5{K zW(`-mDTZZ1yu&TH>YVJz-03Dex>I<67S~i!c;$Jvv<7)n;X{12QpX#}O4~DAuBB;Z zQEka7SgVe{%rhs2G6*i?sS4HfuP1bF2&}HXp-M3(l@WQl@#rJ#XBrPMH^80(LlI>^ zu+yF&dwbAr;=If{0~+#g20@qxSJc}dE>m5B@lmPy`O}e}oATd6VmqAU(qt)+)Z-KuR$1ATTiX8QR&+;zCewfN+4{bi^4mdO-~~?G9{l7ZnJ>GpU3&V6Xx)<<1~Sx}{jlt}ff@!$y|u z-gr@8gq^4c!9+R*mflozzPJnR$d(adDr5Q{Kz%13zop#nMNJl@udq+al1HOHm zyL}*ygp`belAfcdJ{jdYP?@~=Kl}bc6&d!EKp?Pt$UBOaiTJiH_i`n{@o+3@=n7oA z`L5UDka;=y))*jhb28>w?w&w;%D1E%C(MxS{u`l%H--S>;>g|FwzRv`@3-ww0$_>( ze}z|9x8O}>eMv}p{P`}!wGm^i2$%plGt>UV(JG*V6FwP?Iu!JXp1h;l!jm_V9v(d$Z1f{0|8LI)xn2 zVIiKDcw;+EjbOn=mPf#cFL-}Y=L1-9{~?4&Y7|j(2dOOfiIo$Wvas`foG;{ Date: Wed, 20 Jul 2022 14:20:03 +0800 Subject: [PATCH 2/8] Update config parameter name. --- docs/design/2022-06-06-Adding-Index-Acceleration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md index 74ad403424ed5..0627128a2ccb8 100644 --- a/docs/design/2022-06-06-Adding-Index-Acceleration.md +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -95,5 +95,5 @@ As an optimization, this proposal does not lead to any semantic change so theore 2. Use tidb_ddl_disk_quota to set a threshold of local disk max usage by new backfiller, the range of this variable is from 100 GB to 1 PB, default value is 100 GB. ```TiDB> set global tidb_ddl_disk_quota = 107374182400;``` -3. Use one config parameter lightning-sort-path to specify the sort path for lightning. The default value is /tmp/tidb, please change it to your own disk path for the new backfiller could work properly. -```lightning-sort-path = “/tmp/tidb”``` +3. Use one config parameter fast-reorg-local-path to specify the sort path for lightning when it used in add index reorg stage. The default value is /tmp/tidb, please change it to your own disk path for the new backfiller could work properly. +```fast-reorg-local-path = “/tmp/tidb”``` From 9c5283e167de0857d765fc328f86c466fc40a242 Mon Sep 17 00:00:00 2001 From: Benjamin2037 Date: Thu, 15 Sep 2022 11:55:28 +0800 Subject: [PATCH 3/8] edit unit --- docs/design/2022-06-06-Adding-Index-Acceleration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md index 0627128a2ccb8..d5709f2184f61 100644 --- a/docs/design/2022-06-06-Adding-Index-Acceleration.md +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -92,7 +92,7 @@ As an optimization, this proposal does not lead to any semantic change so theore 1. Use tidb_ddl_enable_fast_reorg to enable and disable new solution, default value is off. ```TiDB> set global tidb_ddl_enable_fast_reorg = on/off;``` -2. Use tidb_ddl_disk_quota to set a threshold of local disk max usage by new backfiller, the range of this variable is from 100 GB to 1 PB, default value is 100 GB. +2. Use tidb_ddl_disk_quota to set a threshold of local disk max usage by new backfiller, the range of this variable is from 100 GiB to 1 PiB, default value is 100 GiB. ```TiDB> set global tidb_ddl_disk_quota = 107374182400;``` 3. Use one config parameter fast-reorg-local-path to specify the sort path for lightning when it used in add index reorg stage. The default value is /tmp/tidb, please change it to your own disk path for the new backfiller could work properly. From 9c082fab48157dfd11ae1ea6031ba9ebe6c3e4f6 Mon Sep 17 00:00:00 2001 From: Benjamin2037 Date: Fri, 16 Sep 2022 14:22:39 +0800 Subject: [PATCH 4/8] Update docs/design/2022-06-06-Adding-Index-Acceleration.md Co-authored-by: Lynn --- docs/design/2022-06-06-Adding-Index-Acceleration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md index d5709f2184f61..1343f3df02d57 100644 --- a/docs/design/2022-06-06-Adding-Index-Acceleration.md +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -56,7 +56,7 @@ Since Lightning writes index data firstly into local storage instead of committi In addition to performance enhancement, the new solution could further lower the impact that DDL caused on DML. This point will be outstanding in the future implementation plan. ## Uniqueness check -For unique index / primary key, we should be able to detect if duplicate key is generated and reports error if so. In order to archive that, we could reuse the `DuplicateDetect` interface provided by the Lightning engine: +For a unique index / primary key, we should be able to detect if a duplicate key is generated and reports an error if so. In order to archive that, we could reuse the `DuplicateDetect` interface provided by the Lightning engine: ```golang service ImportSST { From 3f8e33b1047f725e3b60b108af276de29b6bfcd9 Mon Sep 17 00:00:00 2001 From: Benjamin2037 Date: Fri, 16 Sep 2022 14:23:16 +0800 Subject: [PATCH 5/8] Update docs/design/2022-06-06-Adding-Index-Acceleration.md Co-authored-by: Lynn --- docs/design/2022-06-06-Adding-Index-Acceleration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md index 1343f3df02d57..3f4760562ae01 100644 --- a/docs/design/2022-06-06-Adding-Index-Acceleration.md +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -19,7 +19,7 @@ There are some measures that have been done to enhance the performance of creati ![figure 1](./imgs/addIndexLit-1.png) -From figure 1, the backfill task is firstly split into serials sub tasks, each sub tasks include one batch size record. Then, multi-workers are started up to execute backfill sub tasks parallely. +From figure 1, the backfill task is firstly split into serial sub tasks, each sub tasks include one batch size record. Then, multi-workers are started up to execute backfill sub tasks parallelly. Each worker is assigned one sub task and scans the specific record from the table. Then generate index data for this batch record and write them back into TiKV with an optimistic write transaction. From 9a2f7ae81f8b5efd4bcb66be5d65479e3187b483 Mon Sep 17 00:00:00 2001 From: Benjamin2037 Date: Fri, 16 Sep 2022 14:23:30 +0800 Subject: [PATCH 6/8] Update docs/design/2022-06-06-Adding-Index-Acceleration.md Co-authored-by: Lynn --- docs/design/2022-06-06-Adding-Index-Acceleration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md index 3f4760562ae01..10ba7ae4bb64e 100644 --- a/docs/design/2022-06-06-Adding-Index-Acceleration.md +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -73,7 +73,7 @@ service ImportSST { Duplicate key will be reported when two unique index records with the same key but different value are encountered. In this case, we could report the error and cancel this DDL job. ### Checkpoints -In current implementation, the progress of creating an index is persisted in reorgCtx, it is updated when each batch of the index is backfilled. For new backfiller in this proposal, we could mark the progress in a similar way when a SST file generated from a batch of index is ingested successfully. +In the current implementation, the progress of creating an index is persisted in reorgCtx, it is updated when each batch of the index is backfilled. For the new backfiller in this proposal, we could mark the progress in a similar way when an SST file generated from a batch of indexes is ingested successfully. ## Temporary files The temporarily generated SST files are stored in local storage, the path of storage and maximum disk usage size should be configurable, but it’s not a good idea to modify them at runtime. If size of temporary files closes to allowed maximum size, the largest existing files are ingested to TiKV and cleaned. From 26012ecd73be0b2fe0e595e5b7d13c0607c2313f Mon Sep 17 00:00:00 2001 From: Benjamin2037 Date: Fri, 16 Sep 2022 14:58:17 +0800 Subject: [PATCH 7/8] reformat the sysvar and config parameter part. --- .../2022-06-06-Adding-Index-Acceleration.md | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md index d5709f2184f61..cda2d67d7e15b 100644 --- a/docs/design/2022-06-06-Adding-Index-Acceleration.md +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -89,11 +89,14 @@ Compared with the current one, the CPU and memory consumption of new backfiller As an optimization, this proposal does not lead to any semantic change so theoretically there will be no compatibility issue. ### Enable/Disable -1. Use tidb_ddl_enable_fast_reorg to enable and disable new solution, default value is off. -```TiDB> set global tidb_ddl_enable_fast_reorg = on/off;``` +1. Use `tidb_ddl_enable_fast_reorg` to enable and disable new solution, default value is off. -2. Use tidb_ddl_disk_quota to set a threshold of local disk max usage by new backfiller, the range of this variable is from 100 GiB to 1 PiB, default value is 100 GiB. -```TiDB> set global tidb_ddl_disk_quota = 107374182400;``` + Example:```TiDB> set global tidb_ddl_enable_fast_reorg = on/off;``` -3. Use one config parameter fast-reorg-local-path to specify the sort path for lightning when it used in add index reorg stage. The default value is /tmp/tidb, please change it to your own disk path for the new backfiller could work properly. -```fast-reorg-local-path = “/tmp/tidb”``` +2. Use `tidb_ddl_disk_quota` to set a threshold of local disk max usage by new backfiller, the range of this variable is from 100 GiB to 1 PiB, default value is 100 GiB. + + Example:```TiDB> set global tidb_ddl_disk_quota = 107374182400;``` + +3. Use one config parameter `fast-reorg-local-path` to specify the sort path for lightning when it used in add index reorg stage. The default value is /tmp/tidb, please change it to your own disk path for the new backfiller could work properly. + + Example:```fast-reorg-local-path = “/tmp/tidb”``` From fe9629fd269b956c173a679654801b6cc711d322 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Fri, 16 Sep 2022 15:33:34 +0800 Subject: [PATCH 8/8] A tiny refine. --- docs/design/2022-06-06-Adding-Index-Acceleration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design/2022-06-06-Adding-Index-Acceleration.md b/docs/design/2022-06-06-Adding-Index-Acceleration.md index 98879b3bd3d51..b0dd2503debbb 100644 --- a/docs/design/2022-06-06-Adding-Index-Acceleration.md +++ b/docs/design/2022-06-06-Adding-Index-Acceleration.md @@ -99,4 +99,4 @@ As an optimization, this proposal does not lead to any semantic change so theore 3. Use one config parameter `fast-reorg-local-path` to specify the sort path for lightning when it used in add index reorg stage. The default value is /tmp/tidb, please change it to your own disk path for the new backfiller could work properly. - Example:```fast-reorg-local-path = “/tmp/tidb”``` + Example:```fast-reorg-local-path = "/tmp/tidb"```