From ea7823a9dda531e13a8e72c540dd6c520c1e7da6 Mon Sep 17 00:00:00 2001 From: Balaji Varadarajan Date: Thu, 6 Sep 2018 00:49:38 -0700 Subject: [PATCH] Docs for describing async compaction and how to operate it --- docs/admin_guide.md | 149 ++++++++++++++++++++++++++- docs/concepts.md | 17 ++++ docs/images/async_compac_1.png | Bin 0 -> 60344 bytes docs/images/async_compac_2.png | Bin 0 -> 54164 bytes docs/images/async_compac_3.png | Bin 0 -> 70516 bytes docs/images/async_compac_4.png | Bin 0 -> 66932 bytes docs/implementation.md | 180 +++++++++++++++++++++++++++++++-- 7 files changed, 339 insertions(+), 7 deletions(-) create mode 100644 docs/images/async_compac_1.png create mode 100644 docs/images/async_compac_2.png create mode 100644 docs/images/async_compac_3.png create mode 100644 docs/images/async_compac_4.png diff --git a/docs/admin_guide.md b/docs/admin_guide.md index 42b2185d6..9b8c9981e 100644 --- a/docs/admin_guide.md +++ b/docs/admin_guide.md @@ -21,6 +21,42 @@ Once hoodie has been built via `mvn clean install -DskipTests`, the shell can be A hoodie dataset resides on HDFS, in a location referred to as the **basePath** and we would need this location in order to connect to a Hoodie dataset. Hoodie library effectively manages this HDFS dataset internally, using .hoodie subfolder to track all metadata +To initialize a hoodie table, use the following command. + +``` +18/09/06 15:56:52 INFO annotation.AutowiredAnnotationBeanPostProcessor: JSR-330 'javax.inject.Inject' annotation found and supported for autowiring +============================================ +* * +* _ _ _ _ * +* | | | | | (_) * +* | |__| | ___ ___ __| |_ ___ * +* | __ |/ _ \ / _ \ / _` | |/ _ \ * +* | | | | (_) | (_) | (_| | | __/ * +* |_| |_|\___/ \___/ \__,_|_|\___| * +* * +============================================ + +Welcome to Hoodie CLI. Please type help if you are looking for help. +hoodie->create --path /user/hive/warehouse/table1 --tableName hoodie_table_1 --tableType COPY_ON_WRITE +..... +18/09/06 15:57:15 INFO table.HoodieTableMetaClient: Finished Loading Table of type COPY_ON_WRITE from ... +``` + +To see the description of hoodie table, use the command: +``` +hoodie:hoodie_table_1->desc +18/09/06 15:57:19 INFO timeline.HoodieActiveTimeline: Loaded instants [] + _________________________________________________________ + | Property | Value | + |========================================================| + | basePath | ... | + | metaPath | ... | + | fileSystem | hdfs | + | hoodie.table.name | hoodie_table_1 | + | hoodie.table.type | COPY_ON_WRITE | + | hoodie.archivelog.folder| | +``` + Following is a sample command to connect to a Hoodie dataset contains uber trips. ``` @@ -135,7 +171,7 @@ hoodie:trips->stats filesizes --partitionPath 2016/09/01 --sortBy "95th" --desc ________________________________________________________________________________________________ | CommitTime | Min | 10th | 50th | avg | 95th | Max | NumFiles| StdDev | |===============================================================================================| - | 20161004211210| 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 2 | 2.3 KB | + | | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 93.9 MB | 2 | 2.3 KB | .... .... ``` @@ -158,6 +194,117 @@ hoodie:trips->stats wa In order to limit the amount of growth of .commit files on HDFS, Hoodie archives older .commit files (with due respect to the cleaner policy) into a commits.archived file. This is a sequence file that contains a mapping from commitNumber => json with raw information about the commit (same that is nicely rolled up above). + +#### Compactions + +To get an idea of the lag between compaction and writer applications, use the below command to list down all +pending compactions. + +``` +hoodie:trips->compactions show all + ___________________________________________________________________ + | Compaction Instant Time| State | Total FileIds to be Compacted| + |==================================================================| + | | REQUESTED| 35 | + | | INFLIGHT | 27 | + +``` + +To inspect a specific compaction plan, use + +``` +hoodie:trips->compaction show --instant + _________________________________________________________________________________________________________________________________________________________________________________________________________________________________________________ + | Partition Path| File Id | Base Instant | Data File Path | Total Delta Files| getMetrics | + |================================================================================================================================================================================================================================================ + | 2018/07/17 | | | viewfs://ns-default/.../../UUID_.parquet | 1 | {TOTAL_LOG_FILES=1.0, TOTAL_IO_READ_MB=1230.0, TOTAL_LOG_FILES_SIZE=2.51255751E8, TOTAL_IO_WRITE_MB=991.0, TOTAL_IO_MB=2221.0}| + +``` + +To manually schedule or run a compaction, use the below command. This command uses spark launcher to perform compaction +operations. NOTE : Make sure no other application is scheduling compaction for this dataset concurrently + +``` +hoodie:trips->help compaction schedule +Keyword: compaction schedule +Description: Schedule Compaction + Keyword: sparkMemory + Help: Spark executor memory + Mandatory: false + Default if specified: '__NULL__' + Default if unspecified: '1G' + +* compaction schedule - Schedule Compaction +``` + +``` +hoodie:trips->help compaction run +Keyword: compaction run +Description: Run Compaction for given instant time + Keyword: tableName + Help: Table name + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: parallelism + Help: Parallelism for hoodie compaction + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: schemaFilePath + Help: Path for Avro schema file + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: sparkMemory + Help: Spark executor memory + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: retry + Help: Number of retries + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + + Keyword: compactionInstant + Help: Base path for the target hoodie dataset + Mandatory: true + Default if specified: '__NULL__' + Default if unspecified: '__NULL__' + +* compaction run - Run Compaction for given instant time +``` + +##### Up-Coming CLI for Compaction + +In the next release, more useful CLI to revert/repair compaction schedules will be added. Here is a preview of them: + +Validating a compaction plan : Check if all the files necessary for compactions are present and are valid + +``` +hoodie:trips->compaction validate --compactionInstant +``` + +The following commands must be executed without any other writer/ingestion application running. + +Sometimes, it becomes necessary to remove a fileId from a compaction-plan inorder to speed-up or unblock compaction +operation. Any new log-files that happened on this file after the compaction got scheduled will be safely renamed +so that are preserved. Hudi provides the following CLI to support it + +``` +hoodie:trips->compaction unscheduleFileId --fileId +``` + +In other cases, an entire compaction plan needs to be reverted. This is supported by the following CLI +``` +hoodie:trips->compaction unschedule --compactionInstant +``` + ## Metrics Once the Hoodie Client is configured with the right datasetname and environment for metrics, it produces the following graphite metrics, that aid in debugging hoodie datasets diff --git a/docs/concepts.md b/docs/concepts.md index 8de1915a5..4097efbb2 100644 --- a/docs/concepts.md +++ b/docs/concepts.md @@ -62,6 +62,23 @@ Each record is uniquely identified by a `record key` and mapped to a file id for and file id, never changes once the first version of a record has been written to a file. In short, the `file id` identifies a group of files, that contain all versions of a group of records. +## Terminologies + + * `Hudi Dataset` + A structured hive/spark table managed by Hudi. Hudi supports both partitioned and non-partitioned Hive tables. + * `Commit` + A commit marks a new batch of data applied to a dataset. Hudi maintains monotonically increasing timestamps to track commits and guarantees that a commit is atomically + published. + * `Commit Timeline` + Commit Timeline refers to the sequence of Commits that was applied in order on a dataset over its lifetime. + * `File Slice` + Hudi provides efficient handling of updates by having a fixed mapping between record key to a logical file Id. + Hudi uses MVCC to provide atomicity and isolation of readers from a writer. This means that a logical fileId will + have many physical versions of it. Each of these physical version of a file represents a complete view of the + file as of a commit and is called File Slice + * `File Group` + A file-group is a file-slice timeline. It is a list of file-slices in commit order. It is identified by `file id` + ## Copy On Write diff --git a/docs/images/async_compac_1.png b/docs/images/async_compac_1.png new file mode 100644 index 0000000000000000000000000000000000000000..2bfbff9012405a2f3de6de379b71e30b40a260cf GIT binary patch literal 60344 zcmcG0by!yGx2+;bmvoCLB8`A_mk1&$B_%B&Al=ePW6>=jA>AnDODuYY}8;kGgE9Vro0k=64$`L&9Sf}d-;WD~~yyxDU76O-rrf)Xdn zd%0N`2LjYSIHD-jb6h(Y+=#z_=_8+n`ujJNh4J7|;P*Uf|NH^M;)8?6 z;s5&Yum0Qy-{j8~bUpvmtrl?pT=9Ro>3_ZzIl}+zrvK&n#DxFzlm6$M{^#ejLHTpV z|Kg?p-U{pL-(UT2pRbep=Kr*k|I@8_{(o*to$aGXk8a(%h4n5{PFY#m*myD93WHq8 zJx-`)Qq<6JzR0-yF!3S0-5e+0(&px-!=#(4l$0@+(;Ey@j)xi=$D@{YZUmv8+BME0 zB<#&&V{vugWyB;T=Do26%N@67er7y5`nT)z>duda&S!;Qj+OU{YTwGr%1V_Cah59v$5N&yR3~F7E=@!kz#8JviroKm6Zs zdig<@SHNQ=|MPtR{o~T+#S52}l*}(IOjJ9uGchqSFl48vOOBn&m*wZn4>4c`3b^d& zq@<*jm#gu+yStwU`4*&QW=eBoDG2iMolH0S3puFWGMAB&QBdgpqFE+iUQtm|RHWo; zX=lf|E#46vjPa?rx0iZVw7|;?0nyUZk_bB>-Ey+l?OurxH5U`nv9L(7{qyLwmYKq0yd}lOM%@W(b8~a+>+93gdZwlo z?q|m*2OG1ko{Tf_@ad_ksYS%7sHiAahTjRz>l`)`mWW5oty8nJ8PJm2quBIwzI-VN zWr~iD_Bw$>(%#nQ>*FIlrX#yh?~M%aSXo)QzrXM4>Dh|B@-V%mMCBpDFY5b>XecQ8 z1qHo5Jyup$I9_7yOpJ`QPZC^D53M6q#L|gawE4}4Sm6bWQ4~JDf8RGa*uvxe(96us zEaP$P1FNa8pFh*RN5#gLx*CTnE+v)9GiUkYMPRLNDl1AvM1%l&{6FWRP4jk-y1M#o zOIS;PObrLEnOB0jNsi{Bv`jizFQ*v`_U5~cm;jgpxh@i|A8h+QEprhe! z)-4eAOiD_EZ80`5fJc~R2*xvId{%DN985BlmXhM7C6GItVn@u1akAW>&H8I$!RV9i z`KgnK$Hk8yiYNlrBQneefaPpf)bIJkr6}a>Z!~6 zSVgvKUUS>)kdTm@J#^mm8X6i)q^l$zH#beqBj5>8Ib2B*n6^(oeY(9n-Fet|IUP0nV>f<^ zkB>juUyBV7zbRNpvd%zFZGC=vl&Jiqcr@b))sh`{5GFRZ|H|bX=jLL)ii?X2rj?bI z4d+2vUS3X5OPkLA^5x6T9{2&`1=a*-Wn@G{M@NT>imES$!&K40;P$_S|3!juqsh+B zPG?8Qr?fN%28N)ZAnG0l)m-zJFJHP~A&4@2R0*eUCJ4AdTutSepd>hNPKkW}Qd(Nd z`cvbx7n~S`6a^6#1>E+P#YO+yHm_bOtEea-88Pmn5SYDq!N|zi97^G`F;UapEUE0j z@Ki;m6O!uL@!rz6k4%~xnwkZ-n0w~u=OH`L)6*{zAv7cu6&4y88bUJrn=8-1AXu1~ zpu|2ac&TTw7a%>`-LX z*`V#>auPzub^ZGFhJ=Fo248fO;>=7`1UEQ1yF}LMve5`U%uGza$g#1pLFgRYB7?d^ zH*Va>e)`4vaFg2gZ&`q3yEJ*b2eQ{)xi}PT?681<*XPa6%|-?Wut=!bF(#&_Z$d)_ zt)_&V!ldQoUYeOzIqfW*pPgV+@ZKJApKm7+A&1onrQq{h=00t2Z=XuzH`~v+rAYYVR?@Z#vd?USO>ztmMLM4M^3G36nu2|Rvu$BvflVc7zJ6UtVu!8?y$t>-E>>1m9UL4Ka^8KYEWB&5 z6ca-_t=>23FgX_)6?GUGclY1R^9AZyt=ni%&*oRpYdA9m3{O92W%2UxRFs!T#l~u? zt3$2It*(BSZ-6mATx1NNm+>TVeQHWqUq3Z1t;Bh6xw4X%yzS`dh%Ricwzf7VCMNcQ z1*D|*uC9T>L2OJ+f_1D<|CC^;w8wjdO9KN>2?#zw?h*4USceT;ThoH%rlh2#qJngH ze|I;IO-yn`f-Ud}(-fB8ryT~X6!f})SBUAw*A|g#qO$D`oK?ypl zp7vH&QM`6pNu?}ky9eXg8Xw=jZIBGu&Pa4${`Kn!{TOtrNWrZTx( zBbhV@>O4Grj$N=F{}yiiC85A~@7{TNc}+F3pmBtD*SH*Dk=&o_Nfvvk0!d3+`W_99 z*lBb9*~x*JG+P$yD%1=Ob#<+BtK`zsr>DamkPNO}$M~F^`xo~7S#y)6jixtvkk}-Y zxNYt195yB>NJ%S=I!Uixy9V$QHd!>(I%fh-LPv+JhQ>%&7c)1v1Aeo+@&wB4(W@hFu74E*jX;(h!f*b-px!2=c?@(_i!>c%@-_6;-BHT-S`QPKv z{~2TgI{wq>{J!?TM*IIBp#J-Z1M2-70l%^M{}bH)f#!ef%fF_Yf5Y)kW{8y@7w52r zymN-sG&DPj7e|SIA6Hw|^Qn>&9{?KD?Lrh3l*1OGbCJKP(vHA{kFN^q%Y*GgZu?c@ z3@dmZ!0p?l=p2i}bq3tXf6(`TzY_nqKH*=${r|8H|FRPQsmJ&a*ZzkF;=k<^{-vY& zUpF6fcKDEZW5UD3ApjuTUb5=+mQPJ=HNJT_pwcNc6P!lNxClWK{wzV#f=xmZrt_y>b1D| z_ySy0Y_bQk3JP289~f@c*FcUax1K%T{pGDZIx>PoE_BQ94hswRH7|fSFUi@BI^N!q z@eyh5=DBq^>qw)Wv^R= zWQh^c`qz&%XJ_Yp#a_1S(iUsi3He+PHvyLdEOwQc4h%#?M@Oe);a5{p**iVj5x~c@ zhS#MaC)dpdWQ2lrx2wH9Ffg!TaCLQ+gx%=tU#6O{wY`JttdIunsHEg|(aZoWvO#T!8eo56C|o zXlZFlNwowaFg=G(22NSE%Rz2FIu9Z;vY70SqXgKl}KI%gP3q zJ2!a4I&V5H_hkyY9h39f%eo#TX6b*2iZ(kl6Mh2`(s6xvH}-HtSn>$qOXyGi#Ttu{ zJpHOtso@iMv668Jqo?qmQ;?9{W9~_jrfns06N%P|s`$4AvqoT&vpP{z1x1aAhX)#> zeB&oip5T9X>4NI%zK*urjmi-UT*$Y3O6f`MBPC`~*Wm!7s}NAp5wGRzHBJpT#2G2v z2bf`LVF9JVmo7Ee(E-O5P&xc7BQ5P2Dwf2eQ;Xpbh{k^pnjahZFD)$%k{SSiFX8O$ zY#P1d=a!ZpC;Ju#UeBQ~dq?{?&gKy)sur*^T4 z0yHNj#PDD}&xZ2-{4U`S^hFk2DX}|ENTl@8f2L<=qoex07pn&VJn%*H?%lf%v(1>j zuFp9ZXJ!zbpb3&95r>AQvC`A?>I)SW6&^l52uDh8t1I^CTFr!r0I_`$2LTf+ao?o~^AYiKa-lDOfqy2>U zw$f@^^pKQ=UH z!r=l`i~}3}1+Zd{zn|ao(h^D-ChiTHCr_lLx!L~d`0xO&$xF6AH!?Dklf#n5UkQiL zFq&>)T_z+VLU;M*v^jOvys)tF2C|yHeW?lGGTH>HjRc2CdUBnbl;X8YR&hRjVH=vx@Y;ay8?}^)#{l?>uQC-7%>lOwY zMn<1a6sPeUx;q_@qzA6VQ+zq(3jbXW>(&s)2L|ltS|cd$P7DlPYiiTU80Y( zl_?YqEUX6)9)yO40sDi7fdK`A-(mfkxw*Nc<6$5H!|v`b76mV2h~zZTWvZ&GW@al; zv7{s<;`yCkK>~&Vie%RE5)jM8T^uUVXH^yw5}N+z1Kq~et5<;#gHZH1NdxYWdM=M} zDIKG`>-ojTqif~WdNQ4l%NiX0Pi1V-^UB8`tM@8f?>(-;hu6Sg(z(mR@%oNC|^xjrbJ0+sobqp>uwZXel?1wf+mEouSxK zTE-fa<&VqdNfy4~h44_ykb{Uit38QsXF;OH9`cpDq*0t^ijIw73y=9U)ptV*EZo(LX&%F0^a5SY3|)bN5rGD%e*zwZR*ov1GO z2)?90%q;?D58z%M6T;f!BC_Yp7cZ8nd%6<@01{O@?EsZ?2kka3ttjtwJ*1dFXh?EF zei@ikP0i7O02Ig4vN8wg1KivMBmp03C@O9P+>(DmJ_q$Kp4<9X$H~ST<7GT~E^Hg4 z6)!2;j$|DdMrEvZT7LceuBy4u>Z6_knN(C%1idxne@OCHCgs~H5Vn z-Iu6ncqtI5{{Ah{hF?QKnEL7K>q}>~KE?y5v2S~YmX$UC(cff zBz#T$uA7%u^*o2w$ zJs^A_6`|5uLNS(>Mr}HHf~^g`y@-cIbKE~qB7=1KB$0)EDtm^L#by2O2bEfZzn`P7 zUdNWseXx_^htxku^A5@`c2P>+iC`;+dtdmD^Pj)?J0DIkd%A1P4Js@h(k%>Fs40SXq+Py1_t#S51o%_X?J=L*N{- zHFP~mreST-Xuipzl;lH{pJK5k`O4|#r!Rk5wr+Cc-9Hh}r+jtZ-(dG^{U~ykYVr7V z|1f;2;X77;X0H5@hZcsM4Igd}11Sfi!U;zBl?Uru?5W&(+9M>{{k3RpdXiswCh8o~ zradnFcIF#ttDa_A>ZHGN^=HPfI0XnEDYhyEUorMok*V7$t?IRuaJ;6}W=zp#+U z)8Y;94y`YC(cQdtZhK({f7_$Eegcz#kv1*oLd69Brb z9{-ZsgPrbpcjWqwftWR>XUe&C^9gunk+0hCpLl7>e2mP;WytR+Gz)de>v~6WlIFYK zS~W;6L6l5&o9{{DSob3j+qMr*t(b}?`mB;lrt%%`jg7G-Ggzct<6Ew8H$hMJoZ+>&yF+of3x9s@HQ)Nrb4bnk0nk8)2 z*l(Y`mFwiWD5?JyJ{*Y^YoL|ixbfDem(nd5i!&ILtJ0~9o~7%Yh*=^soiVSB27|KQu!NS0?LBsR&=i1xp;pv z`bQO`p6kaod1#O{*cT>vh}?Iypq~By=SQ2z&F+WiLO09Hx-~7;b5?cf=U?zl$mtRxF(yhvlL;N(Z}y`+Mxzr)!f}IA7_n;SNlw{rG8rYh zUZzdR9f6#?pC5lv6VgGYC7q%hUHpE4Z+J{Ol;q@E6&krF&POAS{H~ytpSSPI-RnAe z(V1)(H=X^pb%{7gM-^%YjW38;EpNI4sOcx$cRfwy)W@xcM_YG zR;qNFe3T9S!C2HZZqcO4SZlX{1?^a1VS@1kLanaQ0trv`EN@IBtj0yzZ=Ta+B|W#A zJ7tZ1-V9ioa}1EW{|cAZ9%FeHXirx;z{yaVwr{`q^8woKBufv@Feqc4i9#OV0tv_- zm|NM|eKU4<7sRjl{JFKW6H`;1#j~*)KtKw=vP!WbXogU;S|6RN>wh*eH(y;^TAG`K z0_1h6_W1dsSx+S)W09-%bwVp7ARsW9qizFji=#Bu7EezRTifFDB%N|Az3B#DKub%Y zPwoM002L0pX*V}F=<&9kpvnI`u-w{W!Fus}%yBI%yg0#^U~R3&nlEmsm?D^`0Rykfb$MgfJgC&3`6&k4q@1 zw@Y-_+s<3{bA3Xji)@9%cw4hBM+`T`eX=`Jm`CE$rBvh^FXk4Dj4^7Q`!HS#7&fSC zM$(xvVE<4e^SWbR;}`f$XtLsdnkt2_ps1$a*F?A7x`$#M-)7X`cqmj~YZ__MZWeV= zp48r`f3`ETs<7pKaE|;j&h`}+zEePxf4E(F=GTyRCUb#*JX?kmfy7iE-abgjF?WE`_ov`lfz;^zGmmC8=50o|Wk zT-jvoT5o4+ySdakI0ij%Pt}Ph(26N8&fOhjaJhHPbYtn=-6x@BT67x?~XTnJXltboM;Xdi+OZ=rfnmr>c>(OWTl>sIsBqUIIf$5){Gl)36m7bFF zTtkDjgPWawXLYy;06HKI;j4750M8p68VZYwdY?Nx9|NEXqu?9(@xv=`1H1+RiX{eP z>p^f!f0CFG8;dUDU}e?Q+Y4-2h5Z_-LtzssNK1hJ?4Q_?FN#SJ`btY}@LeE- zd^oFySyyAJGMaAnC6j=K;$#T=dvnYd*VB5TkCor4MsfsRZQj-Ri9D}e?9b95*n)Ag zdu)x4I9w?1baKDHxajHR!il7LKgokB8nKY+O@8cH)SmK@*q3U9aSvE$t1)d|50k=r z+mo3_tkzRU?dacX;p}9yOnBDkZN{L*`pZ6{95hRxc^zxa#Fy~I=k;K_w1=-Gdx`~= zKt0p8?OK=4-cfC%0Tzm znapnKpd7QR#W6x;D`6wPW~`!0EKj_FLz2UnKZ(pxo!*yL71X@8t*7KDw0bnhED((P zG1mA?doEA?Cx6USy7|7ic+V`zkj!!ftXcUs0W5t~IuY3W?y(y*W5RFft5h?^oPKtz zQ#U-?$o!1;u9+9nt)u5j#PadcPiMDtxkJfL8cn;YNt~_EZpXwSH-&sZ=V2m{-*wyH z5~7f8nIVea;4qE&KIcP|UdI|FW^PSaxv+E7=5A1*ub=y(%BnOM}^#3Z1V)!YYQh)*Q29;XK$7iEcwiA%orUt3zF zw6!T;1>2#bzWKnQIxsMRcEfvWZ*LDg4=BunH@`=+_}9}+3t7i-|HB_w(<5ljV6&6; z99#>~8fQBA$e0(dvfQKdBo_6+k~?ucHShMQs*EYlu7*Wm)0_Ar-N5jXbDf$E3qI@G z*-r{$LcNJ>yW;uljPv!Wey#{*_NC#Zyi;O)G;M%NG0Oz#m7ZlQYEjMGK2E45FcdHv zGVXbV_XQV9cNx*K7SHnsBHJ=T$RzJQaoeR222{vWGC8b1wogT!`GviziDzt1uAc7D z!)b4tT~MSEB;%Y@x*g0f!Nr&`lE(GCr$ydIDIV8^C@8E(cJ(TaR)2j>ony-=mQ#C* zRFN`rs)ETb&KvAVm)6Eo1@j+6+LAGi9}G?^G(!EgQyo}!cTLh~gdY}_)Gq=2GHdDl zKzkro@adh6&k>o`!8I}tzeldc*2#)zOa~bf?piVhLI$dKfrw3949)fh+2q%e6}7EQ za{&reY}H_L`en-f+AtF;;MPl68qAcl(S$m$CCN0GB&Z^4bDY!~U)h`N$1Njr-=S6{ z0Z$71L^*0~)riM^yL^I<-R?Qndw=`XO^#vrRx7`T*QSMTWsC`-gxQt;XJw2B@A%sl zKb8&{x*z1se&BsOdCSw!=UYn)6}Oe{XsLyj60prt>d&9wk|ja_88ZfI-rlkb4NWTa zvq00nK_n%`#l>ZN-W1T<`p{Ltb_qwEpPv3vISQsHC5K55bcr7s)x8337Q5{1?d7DU zMY3TWpKx(@7OkDWldmU|w70wa01Q}1JfKLudi4rtDJqH>W%p=j@!MU; zD?ZDUbso>2#hLK^aZ*|43ybxgF0?OYV(vvV6gl$*!%9r$qHk5zOFK|L z{Cpm)=lnt}xWtdeHlez~EbvS6r_ycRyF1p}rCLSYvc*!2_7OH5@4WU#h=U6_$*-Z> zC$PV46ul#kr{i^sAHv~hMu#arj)`8me=8|sfz^ad);8L|`opq1$aELsEY4L@OgAaghs~qzw|Kph{X(lcNXnX@ zu=UuWp>;f^${IZ*OIG40T0W*iDHE^?+|HKyNzpyS5c-&4P`e~U9B2v5*=nG;pr=(* z!g~rVesb5-r%%C!0CJ9(@Z!RPXdkmqEx*}7&bJ^U%aLMfoTayKZ$K*!B@ko>3&3Sy zumJf0)bR!bLzLp`>eIE+GT?K=moFV*l*_?{xFm*<#(G~WC@45P^X9mX6r0k0OiaY~ z)ULF>R~jjJmz7n%9061Cc!|c9AkiE%wg7||y`6sw+zn5`T7AMe;!-q+XX`v^W?NrX zowk_NbQt#u7*g`A+NmBnpW+~*Km8b(a^|Q$!^}dttkdsdqCq)RHRp?Ph(B_sch*wx zdq3#g$F%iV->cZw=XfiO6S}ieRzyjXPs6rdlc^OxBzB!1wW1LCez+qPz`!3rf1+byLBx5o0eZb^tveAwgR%=WWz(YY#z=__ zj)~(AL?`Xzn7P%};9>=e2iG!M2we$0G{As(X@`Iq8yvG`s9Lu@h@pw(GR=JKfahVr ziF6D-vv@g(3tO`-!1`PUt>O2Nu%SSbV$fpxGTpm33^Eu9Y(5R(n6WYei3Wuk(HZb_ zzJ+zNI7gpWl|9q`@zD|W`*P%<_OE8ZX`#=L=j&b5cvfVDxhp_JbB#AZD}aiOj6qO( z_D7lwi1-i19MO<{$cC9UOEECv=$wH)Lt0uoCBq9Pj@J$?i-|wxH{t6O&Lx{vQ9X~2 zbZkjza`ho~2SFw1=Fgr!-9OyYr@nuhlb(Kkf{`A)T%OPK)mAHA4%UPG;1D`roVy`1 zvat9|UOe;r46Y!C;(%^2V4>adJrrno4%Ce=9p-ye6B9ZnCJ8M)`qrQ?x->r=6()bX zDlgfjOT%QFjMqO(DlkEpjpje=){*B`D>MFm@La?td61_cewca%~HSEBYJ$lqbNDmU!$nY?Ano9Q5X*2MK*{bs| zLU#@TAXlfZ7S3F9@-(oDt*%;Jt?b~2<3Z_6zO8|Zf})nM>)BHU@eJKLnu~%tSiDSD z05ppd0Y6K~G*X~16-M|U)~@Q|qeot?qH5DgKsri4do(9Z5||Ci5+ z;&Lh~jyj^*!MZX8_F4>=qwRTMMsU_4FP6fZ_X30aIG&fl2pqPG003|1TVP{j6BmUO zs8zg5%gtqOBF$o|_BY&|s)w_(w7Tj#(g~b9FrmqNCPX?evWz07;4bmuLbs8Vw{Gx3 z%{Sl(pOMLf^rn+FhHG&2N&EVi*A88s7TO4U4W8>73k*9=0)+F-7hH%?1a)9Ki!P1R?YXygVS z-p|(W&w=s2I9(L_&C>6Ff3|*?NA-4qA<84uv|djuiZ$nzG_|Pa=<*MsI!!IIV7eBFhOR-mDQ!Z)AWv;N6&>I}FGeQMxTuk@M#7Xk^$ zjte}?s|vW%a1@>oPSm)#_=ozBBfq0ZX6yOg%ev2Xuv%YaH98?P_~H_CBn4H z6T#St7Lr0E(2PtG&8FY%aW?3o0EGtMgc*N{OD9R;`o)}>lIX}9nn3x9C zo9_?bfB4|bC9DpXxR(GKW|~k@3X+rcL`1G;p(9{^Z^{J+6v&z8!-YsMok3`!M8GU~ z{ezlxl`uK!OPyv?cRU~R7a}9A)UQzdbibOg@$vC-aS<)EThEA_o4$Nm>u~`B!+DN+ z(Lk;idT)i(4(E)mC#>az87 zcPPk)5~CJ}a8Xx){+6^J(U0?r&@1Hn0u6xTeVncU-3*N%Q;7_;=`EQ^G-CK2nq7C+QC?x?`2x?!=x5Int#SK;IHTnS# zSe>k++VW+v0PBPRSe!mxEyZkuQH&BR8$Iw^H;0hnBFepaEpKBE1y?E@+5r%M9bbrx zBifQ%>lT22#-H`gE5G+S8YK{2%Xg{ns;a1fEhQ{R_p6nI;EG#Sx3~}cr1i{`E zR$L?(VB8{zZOM=HDa3pTBY+U(6t{tp-aG{8sgFd6wI-?N`c2xaCvs3OiOu=Ggz`A^-BJ08bXP0AK}ltntH&Yp`{J6f8zP11?fhMP zo;azeWk|(`s!AID`Qa^`_O>O$f?ue{Rgti(XNL;?0S8f zCZ2CQi(3qq;SpH;D%pvNo}sxE`j+&lqNF}bMG$RekpYMkq6N#o@mG^oMhn0i@X~vy zNqJ238lRF1N#K_|jA!2emd33&Q6ru)U~W0*z3sd#rm)(sz;13?zD6CqJds2K%$P@w z{X(NG%bZgD!v4*+iYjygf;!F?vRW)_squFF!S*>SLSXfh3IqSAUNqY26xa-p9VLfC6)U0J4_Qr=Bm!n$vOLZ?RaEwq~D7m(G zsBciY6;0+Zq{n`b+tQbE{pdV$|G^l_ixqP=EX~u!$L()Qfc)bo$yqVFvt~le>C)VT ziTvuV4cA#mNzRMW;@+T$z*7WPz4XWXE zCH#$WGUKpTs_FwF>g+zCHKzl-@KR^qAx!>m;yaAx$cFL`Tr$LrQeErkEA&=4tIhB7 zMul=l@tDtwre)!)DwA{0;Tf!K+|>WX(=p*_E*phg4Dvx#^{#3pR?HZ;9`0QBlpd3X zC1b@`)9wCLqaHVfN!+X2l$G`i5Xn`lS@qjmOliJ`5i4`R7SwhgBwq@^XBy{&pVH=c zfBgg}TuNq#2O$I=AlDk(Uq9*yqCrEXp0mI)L1DA?bYDs zCQ8;H3Y3TirF~>+$sGB>#frL8#~^c=GTDDCfxwiA57YhywA(HbCzFqD=c~ z@E6_B@psPY?)5I7C0`SKRYH!v)gL2h)v&};y+*sxG-8K;{6Wixqo4Askerb5-a9et z`GDw-A#Ib&-7Nh`DI+K9DMhX$L&|MnYjb!dX1cf@)O)_TcH*Y1&w4$VYKhZOI1&4l z+(1#TlZ0#8Mqn>l`tE`*ksNj~hrigC0(!sM&vA*zz{kapv!B3mKPBhJEVuf)6w4~6 z_a=;0#F#Je6fpLZDk;iKFzs4aw{4W|01$xj7LNlGtGxHZmcxa29z588LPfHTj!!_K zH^c+x%I#)!+n*Fq?0m1XXJGpDt?T4nedmYV=;v@f%e}_l2q^6?W%BZNsrnXequjP3 z&@m~#t8k`cZWbgGFY(O73c-G_$`ywI5%BQEq2yz6JUg@zWX-d=R1apWw;^MhEBB4Z zA`{ONQ!DT>PNZu$ir&8VL1Ay3PW+XHSrNIk`;NWn;7jyTf8<8)zJf{l+SPk4s)x?6 zQyCE+x$f15ZclX(YiiQ8U3Hs~D#d=Q<)!)=Gvd(7ckn$th$Q9_qN!LYBnWDTezVNj zXRl{(BmXRzjxCPaa7252Q|0#Q%{HVL=X8UMn}im0hc_H}^oH|=e?5KAshD=zp_tw4 z5%b3)pv>=_oKM5W!((^coH{&o6!i7R*)~GA-Ol6*yzi!02^uXY*LZOT_S>Z$L5|($ zT24tDU9cR?BRHB=54v-mi=>VHTd?)o_n{lu}`8m#;KZm?&QQxJ2>K(A^@w+=ZsFD zU++%TmqIJ;J37WSNGHfCUPKWY=W)7)2hT+=d4ZCFVk=o3ViIUCQTTlzjUG~N|ECH& z{G&zf;wQX%5yVH0Z|w)Ncsut@aThO_XysmywzhUd5b;q}HOD5=;;&!P@7^_{UqGeM zOxn=4;m5nr$2E{!leCh>+Jvj^V${6~e0HY;|JL3d^LPN$2uLqQEJK1F+mE=PAMcGs zhfl?+RYkEj>W|vxzcO!OG;h(~(%gk&HVI^;TK)b$LLG2~VH(Qg60T6Lq;;fij}(6X7Ey6sXLZz111XA zijoeQ)U{PZW8$%*nfTXf&f@km$DO50e~GX6J?7INj+s4Mb;1MkbcRMYkDLupd)zB= z)-)pVaGQQi{#-24__b?kban;c2LZDFV zIuCxW+DjRrL|9bL9$8Facc%(V6EDAG9^JRxU=H5P?Os? zK|iIOcf1(8t=}gq?O0^;Ubh(A!-V%rC-s_Tl~5HW!4C?OmH6&Nylw6ce>;51@CDiK z+!`Uu%?t@GZRP}ODt(wau?)Sm5<<&89i;q^nhPG9-4*JdC);k7%mf$1mP76ueRfD1 zQM}uc3oIPBJII{guQ0i*#6ovYYndaQ2=*5yrjU2mI<(l7`~ktiIaygVlN8i*DtJrn z_>uA7RY2Lqe)+>Ws(&x0W+Zl9q&AS*ra5uq%U#*Uv!ah(H!fhqRzvPMe3Abi7VGjn zWBs`*k0y<@<;k0RhCW@p2Jz{!pncHvDuxUTV?PvMR7)qXaI8;Sd3`~%Nmo6R#k6`~ z^J-2CTi0DJZ6X50`<9D{_^Wje|TyuVO#b}=K0#7mLHbPE@%=moD6+0*OXzg zekW9>TF2IyTGPYD+h>(lh9Ie*S#=7<$0OhSw-f*PfE8FApc78#WXRie#DQsrK}28( zoHl5|1#-h==*1ZItTm0$#?E{d)QXRjR_=;ZWOLc9u2U@=btgg~NRzSiZ%kT+qO!xe zjR%uR+DfyaN6yzT1qCba^+EgK#9*f65G>VyDp}n5A=L=<%B;Hi3GFFOXS<0^EoXl& zGkupqj9GrF_k+r$+}*Z}{Ej=%=i6qTT}I?ym*?#!ESu0G#pcwlxkI!KFN*;?%VZNd zS>@~p2&@0z4d72DP_d@TIQ9BzkUhKti5f%d2wZNHi&NPL6+1Ah3KTv@UcSd~Tk$LO z?ueIHV=LY*+1Pd48ufhsUY-|;=e3;d*cJni1T5}V-^9s^AvVP=)uvfbvbLxs3l!OE z2|FflEMzfhhG2s7ufwgY95Gku&>38EHIS!ZA6L`vGKKjLD!*@WYE2WPT|)QbnA|$8 zFzv~9#c#&gaF&`~|M23wz#PHdz5YZtd`ayO%azv2H?Fsg=yH>_z6i$C9rple$2$6 z+AS!2<}qwtqgN?a_6z&~mv#f3H(^Q1V3ipq4vpTRt7Rc*MSAd!8g6w36Q%KDP3uP zq;R(I8GJYf1;nt~z2d9WHmF;*!I^Ru|X^4)M}~CFxWmR#AT+vwgP+9GUM}8lP7~_Q`HFc9`O#C!L@ZyVl_?UR-P7|rOt_5qeLoNnyO6q;?9-BvNkINEW-&5{7tZsIv z)IAXPoN+|{G;$K;u%Gj^rL7TOA-0>;=~q6gK_zcL!Bgm#q_;je^_#v_XS26D@a-x* zt8?z2P`mN?BEppZpEwGK88HQg`fF2$+ndv37%paJ4F2&r+mBFf8!fQ)JEw8R?5_T# z4O_w7!8^sipOx`o?$~WOE4hu(wh`n1w}8o$^%PrTyiHlN&mBodK<6bi%M!M7SDZOn!-t7MVz7yHfYGh(f>KFdjLG9~VnCSlXkV5_4nQRl{ zTzawGeA@Nj_jK0QOf>e0{LGL&Y`5z=eA%*gWOI|MVoRsK4xPDE!hv$pId5NK>FkK_ zGM&2D-nf;68THg-R4i%*VRYRc|Ba9z!>h#{9_Liq5~-Yhl~!)c9puU8zoZllzI)UO z6E*V*tLcTPhQ)-CZ?^@AP`RkdB#wOX4-o{>rKJ(sjgh@{(C-MyR1be?R4Li+wV(cM zjUvz%Lb*cjh($ztHpPMZ);NoO=FH)u4ivoZ^?s`RQ#2u_*UA{5?ia=f1h!=?2*=~@ z4~43SkR&YbciBg?GrPVsdj9F#8(pnC!?mgYt_Y}rlsVmA)q%kblyl#_(T{YRos=~l zVzpp&n~*rsR2bNF44}0YV(F?qQFj!wH7Lj)^WQkb{=~DMv{jl$?|tmH#HD(@lE>7X zMU7Jq_4=-p$HNQkFpES2m#-$XqIZaea)`*L+kLlez*!LOz7=eBIY#m3P56eQXFt5kWr3Tkur zhZ`l2S1H{-5jphSUi6B8Xuq|Bm?yt-(SKrNE2Sl9@aibPO~XS1w91eYzf>LL{SHcK zgNoE?*lgMKv@CJ`52z+m;_aL1brTo^>l4aYltem~IR=4hBHlz2r{X*!J4a8D`>bgA9vmRDij-Z{Y-%lQOqJa3FleZnH29PdSmh~c9_d_9np zwMwV~AnVrPY8J2j828PtUPE%n8CK5{(=9{Sq8%j--YSUnbfz3mzm`_YNp52hyRYcL zV=*rRr71Zff6ce+-P+^Zd&U@;S{a@%6UNev&2FRG*+V{Sdd$2-HVP=Y(@sh>fI;zD z&!dAIJuz#(e&I}~j*~IbATdw3=P&0Cb`?i){^)#h%3oAKMo5=@LQJ*x%Uf=3nKX^+ zrF6&6aKSSPiCQ7$AyD@C$vQ36`33fh@3iZl9^@oU8?be?xsLHG-P06nGhE;1Ug56M z#Qs7jx9sdj6Vv?bkx?*{SGPR*Ydze$khj0JjEj=nFziD_B;>Lm4Bw)Wnw(4x^RNX4 z5grac;i`|4U={9nje_gY9{*^Z*YX{i@dfuFa-qv2Su6l26cn2pw+UXT3Xa;pR9c`^ zeM7}6OUa>*9~uRFj_)8o#e6+LEA>uFPVpJB@_YQ{H9gpg0l|S~UFtM9=@YP`1wKzA zgAqm8RnHy#Q#d50A+6yJexsD4=MR(U^e8(htA2E=Ps-Td(0=)9k!Ss$d1 zLf}lG!$=9Lf{abl``{?DGFOpCcbKyU`icmHV;%--PpCukFU+M@hKW{tY#}593VR~J zr;yuX7FQ^4(8?BEx}QF}w(|g|bKlFOGxDpmT#nWN65pK8D?9B2Zre01kI`$%Z>ywG zRd6sLV4O%jdh%`Jo;M7tSHA8pF8cT|b>-#v_3YM$ILx{5vHp6VCC2MGT8`m!an8** zG~Yf^Paz~kIm`XZcSwV(IN{@uIFlz92~W(+x4MK&z+U0vZA3oyNyYIN_H=mB~fX>M@om@H-*k@M?`#d$0IxUT)qjQozywzO>}eZ=5;lE znEr`#AXPCTlzSq^;y$h7E5OT~bxT9HU*7n0#i3@?@0i`y0UP}(*pBY?u)Kpt63#|b zlel1W=w;d4rF!ObpfQ+M1*CajdX>m8cWVF8TSD)1YJTmX0=~r(TZLU=o*5ovz@I3u614*qxxQznUe#O76A)3I%oqmAUYp~nVOo`TVIaQz(|R; zQVvLF7x#dW_CWwg3Wx@Etb!;fb8UD&ZZNYVsGxXyrI|NIG9f@6W@7AmeYHP@DC1W9 zSEQKwyf85l)>>_Fh5>Fzssfk|YynHDJ_XHdZC$mVDk45CcPD2*R$mb1-drAdF6pF; z<}{Z_l7qGZyG2iZBPo_Go%or~2fxkwt7}VaVf|XnfvnPAV{*azQ>HvrJ9Zrm!(o9= zLs9d4+}(HCe$31<*MM=f%y{?DUh8}V9;~{i2By0fyA$hx$lhBSg!vIR@a)SAm)q&; zCZ(q83k!dBWdoGnJ2Bx71V2nUrUWk$qobm#ms?Szb;NRY4-E|s4ic>l!N38!vF}?L zxw};E&Yx%#NqIuY&!kmp+xGU(L;d;o4{9I~-)P3eT$`+$tRq{!@zj1}0`soq<+P6I zIe0Z5%gW-bb3Vl)=YB0)`f?nN#or0O%7BKpL+w2~Q*7Ahi03a67D%Q+>HK*w=lE^dvf5$54)E9-P!&$cGZ0V-#l3LFI5ut&FV^ zk2Kz@sjpk4S=`Jgq@LI&9yjLEweg%!dL6$M1_bx1Z!j=1OY-y8V1Nw1KgHA(?0B>o+hC!)_x!6N zbp*rH9M=2yMM+=~QGpOmINl$V%pShd0KU2c<`}_n{C*OK0*g(5!f>K)0gE=Nlns1k z1F-s{1#qs(D?x7tON`h`{;8ke_0QFnmB%1{!0f!^&cb69=7CEww2sx1_#&8uU7lj` z8c=kik4Q*JhVC1*;;8XqRDJ%8qE77(lNr-oq6ILS`6P-p83ZDr(V@+T@i&-qgn_+q za}1PYFQGIT=%49B@b=ELGMk8$zFf_?wEN%so2@z)6HhIdctd3TWkW3z)3`G?z&;DW z1{_bkHY37};N#T6;1Vn{-hb77@B7$1dtE_WaDlJ=C8&1D?f6|u z0RMGsu6B(^l*pFbK~69bb-TN^<(vkjF3EddRp>ix%PT7&E(I%Fj#v67fK(@z3G;`b z)Pn4ysi6^`3}3}lIyy8YoCGf047vEr$vdzYOcSsZ-*R(115XlY$PJ%BaIAJa*>8Jy zceYy8pv{ET?NWBLtk}drB1+HM+z5q{m-EzjuVt>?pkXp-{y&Vp1z449*Dbt7kPhjT zPC=wW=|)Nf6hx%EL_$D9kS-~aZUGUbOG;XlZjg}f25I=`!tZ^*_nh;cbN$z~_rBOH zV6EqQ?t9*2jydL-o{t_?Sv~m8lRAIriF?F60$C_{ zb6THHIpx6lASW~hXV}vH4wd9lIf?9^$b#|FIJfafYT1RY`D&nU`|Irly_OlDK0!%o zQE#~Z77_=d9R`$|f%KsSNH+TVVD97H+*9qcW9Z?sjgN~ zRBUc-lJ&Am?;{<$C)@AJsg9yaOu$h=GCJ>y(x_7M@_g0&1=P zkf0zgUf#0uaw25%Nr?sso6XJ52BXj*?X83`UFgqJfOY5fEDdaDzNDu|CM5hoG@`Tg zdc9a-*~io=+!-R-Vt4pZSiJ&`4<4trVPBj6f&KU87Efs}m5|nK^Z6PtIgc=uI~wEY zU#*(z6@9K*cl)a51)o}4SwZL%gZ}SXTm7Vqiwh_ceC$C)eLK6$4J7Q0%*-66cy18b!eecG z29+I51E*K7NE*a(%#!aK_A#KWVGrf}5HiG_QXftlH;u(Xs%TKk2%!4yxe`}YXi z7xpt4yk}7O8)<5qh4OY2);csPgUlfk(XIO7{rko_I5^a9q~0|;i~s)P#|0}=81jov z`{P{`1TzZjci+EczC`bN56PoH|V&RiE0Ezt~cxKf8>>~r2E zAz{Ab(?HX^y$078{ZA+ zm-m?r#&mRWDE)KUGvzAx#D4zxB5AT=YOJ!x&WyR>)vjQe2!GoRtQ*lhHjbE~9x@W~f^>fYFud7dn zCQ~gBHhwiUKywB3`Rxl7DL}pcp;q)X0q_1-DD{FuL5sbbO4L2hM_Wsafq@|+2o5d? z5WTiP{I406f%1Z|r@Q-0S{e;AGpe_!>n=m9sPpcg?V zF^SmsTzetSOI1`kivow)ISe$A3KOR@#Z5Sr>HqSz^1k=rmz6(0s7` zB6eMOq~<4cLn2ppE4_e2ou3pEhj*;FmmbvrpA5Yb1m(g&M+Z8plY;}#CU#Cv4%8G( z;2$^%em%z=q(qhRS?Wrxblo!@co|Fj>bdT-XT57+8p7S!1Py@(RrwYTt*xzrTTrFm zemWrO>;_%r((%H@^9Wypa#2xH=xA%3G>0ZRR8-X9vQmH0szjtnmJlB@N`b=(R!R&6 zPeTWKhuWl=xO>h&E81Q!gEas`RY{rI3Vn~zeBQh@cC$vS!Lx41w@i)ghpMi8VGg%1 z9{#d_QZCb%q)mZ=xDq6e$i;CnIr*`gg!-6W zOl?nUS~0s4yZ&A}ui@sqCG0o&=ICX}CoU`O)jb6Vujn3$M2x8M1r zcbPz;r}J7UDC{B@Kte=>KB)uL%xiKl85E%LURGyG4V8f?%yV}hy}Fo~N}7209U}!tt1x3rMcC;Ad*OMkNdH3;qXM@o zJV~UGhf|km5b6v&qzAC0M_Tn#ZA!X;1eBry>wkF}&i%!eXo50YE zZ{Ej%;?@#)6AH#BL+||OA&x@(rrVSh8m;#*13c&uCe~iC?zj{(`4MC1g(>$oG9^XD z?yQ|&s)29nVCR)OxgDAXLlbDE&bBTuwt8NZrQ*Rlz=CGo*G{naHd%+4Aj`Y_Q zFXp3F9~u~g)87lYOvPCCMz=nbXVHu!g?e_w`C}k^0w{y_J5A!Pq@*M?0)Muoxn_C> zXnaxI1&%y8z~;spF3-1Kt&bYi-k}#2^&HGq15sGs`6FRk-w1K5DDl6Kl&Ql91yl5V zUwy7p1`DD2M3@D!N`PQ=Z?OvugJR#mKlrS7ZV3lqj;BptQcJ2 z$W%r`^udc%T8~B5vHX@qLoZ`e{=2B}8`{`-fa%sk*VeslNm&h~oV z;CZg|EMqWUt2%&R0_mUv^{rc;;N1aY zyos+)A!1%|Ga&NP3_}42*&UV@Bo<)f<5~T_&Si(r)C+Jb#J7L1{A}i4gm!;fv4M!3 znAl%@fVSAo`T|r!&{HJ{zycgGa47|gK*RyT-Jr+c9#+^)yL@SK@+25h0Z5h)dJp|B zDEWvU-u*yA)s`Xwee<#&5YmP)LmJP+s=HCo;4Vv`)+ z7U0*h9MRK$#p<%|g01V#$BQm%9|o&-)LSw>3-eccyIbJ#gbP*Wd54#${{$O| z?99a*JyoJU=nH=@GAo?(UZ(rCi^puN$s~M$63S`I@$c_CyArl^rI4LE zBp!d#zI^p+d|&<`S&e9KB;wI->ws9i;V{H&-XEsqx_5WX)Z)rN-J2FXd3J-=iTvYP zU+u141tZgDdna0X>oT!)Vg`5eG*4Xm_wsYS6bDjYo%{JK2d;+#b={%dgsJu_t%v*; zh|PpFInT<%^8NdFmzzCM6PX8n<4PpBrXrtgO@9iDZ*U*Z*{Z!5kQorIF5AL1Yj12m zsozT+xiAWn(YXvUG7GinOCKsMV6M1VRJ}E(`*yf{WAJQJuZXSVSJV6lu1$4YYU*NF zwGXeRj#hFwygaO}yp#sl)^pDjMVl+0=)BVZ9I-}F^o^(HM*Z<=D%Q$5GokZ)r+~fi zw>(IufH$lM&b-9d&Q9Kbo};%_7Oivmo$Z%wL-NZ0zG*B8cTL56|H-7X*G@eYr~xxEnT>s!Hk~}5!CSOU$ZJyr`OQ-hmC!fQ;TNP`S{Uzsrh$b_ibzQ#`Mj^6X zSD&n>$3HhP_e*rT^vhutZC5^18Ah$}=H{nJm%TWNt@ECq%dLJg8!P0KwR7${H)EHO z!^@o0oQ9lvtNiL}h~%&&_DI zw`Xc)VzKhDCoFxQrF=5#OGQY{b1^~Q$*r2{=-8{^K8*q{SZ&)_`|pCmlLS28W8?Nuc%g2Irrq!0tr$W)97t3{e zcM`{Ng32hT@%bvrr!6nrL_KP1`##T6_dOWguB-Iqw|i-II7iLFJ8jWKx7*xF!jR0VPKCzGEkPK_pm$k z6`O(CI=u?pi-se^q$Q10MqRI+QZF5gv8k$R9ZJ!q+Bto8viN2~D>0P_67JV4oz4wM zD-FpLTxl%UIDe|`q*4ver;@Fv$nDP-dM>sweyW~xhHv~@I2iFdjJ~aht+1aUT04E; z%X81Hz0xyWwPEkXD|JHjo2|l#*x6#kjRO}eKa*2ddeNlq1ys@Q%%5~io(>c)p20m> zwFG0WZk2b|WW6Jw7&Tcb+paMkpZeu%$H?D&=Z)`ojID&Znwr9GJbx`jPrId3>7F(&V_j;icZz1sz8*OUcTO#7 z$=K-e-1QortDbzzMzx52C**&JFjh?YB>#HNbg&#u+a4BfkIK6yFzj>bSLqQ!LPtuRZHm`O#3A zlvK~=t-}@x^X+vjRDWCDVUIAU029?MRWF~sDM3N@ z+r1^+i+0Wdwi&PPKlvlSzC9N_!1!7xlDnu3Aa}tA; zU$Ob3rQWNoMrdfdJTIqf{SQHa$6X^OObk42#YfF{Dw2U@Mny@=qkkm3_9(2mJqG2> z&oy&6pFFM5yL&)@HCnxBVDh+&$4=@bV1zmQxLf+rCV!m1i@mzc4(*!bvw;wMAK1dJ zGhQCF=zaP#E6LZ#eCzOT=e4mI{@UIAY3ljaSw(Kv3w94f)dM}c7>H_sy8=JZ^;-L> z%T~ve_;qn5<2q`YJ`bU)y25TI*Kx>^EBIMnocn=ELdt_xfw#VA5o6Lc z#}`;^J_};i_rnUW zib&gZvMiN?-l;D2dLS~Pe-TggvPmOW5PHJRSiyG|F%4{{^;1o>kOyNz>6KG1jc+AD z1CchP?@up5w+?u~jWkzbD?r6wr80Ck<6Yzt^r1nt1xo{ znv4y4dU~FW`u)c{aex^oQh~zNCb@cCi1>Du-=pt`%z$e+G@-gtNlQgVMoNrA#^5yZ z-vq!u@>gh=0blYF9YJ!J-hSxU;pF7>2zvORi4gRpy%!g#73Z;ygj@07+`$PnO(B{O zeKTo0Iy)KGp-1B0=6~LU|K}&8<>uz*+p)~btY^*r;j>e_g zH^{Qei~X~*vhL~rgvP(`!>z3!FZ{uXzHJ;N^iMQ2&bk{*>~+;b5}TR(du2tN`4`w{zJC4sG32s^o)wT`w)q3&;KViy-fx5lpzA-Fo7>v5 zTOS-7tL}cvRsx#i9?{d#AFGPKl7ZAoP5A-#^;QHPBss|Jir9l$D_crIo{ENB9G!~Q zdYam7u(lM_F4c(HwTjI9=uW=zptyl6UX_K%03UKfxy#ZX{I`eDTKj{yMxssKmpT+!LKZH_oUur zy>fDzZ18H>abN%+i^I*y`@s9i%YVnclWZ*S4dio;%`g4{a%eV;NJ+7E^b{o^ASirR zUGi${xn+C$%=-GX*ESj&gTR1+86=ec&fab9LsKZQ5oZ@-_vC$nwsi%%2iT1j;O>BU zL1b*Btf0FL4Xt%tfjXtmvuBi^W7*2UhYhDyFPrM?(`1YRw*fs<6b3OdeGULiw=R+)L9kA#nXYD!F2d#jaQ6IN@ck4Ahvlwa@h zG48;A0mgg0Kt}+axlhEr?_51v1V8x!7sqUrIk*~zOMrbm(7!`}Vbr zV^EAukQ9UIvd!ez1L&}PQ3G9F))Q5c1>hiS-wQ5l6k=&2K=?sX4t%$PM*(`u89Z?2 z-;5i6%$H2l_5|Y^F|meMQ?96T0-iULRDnT8bWKtDiP+B~KX`gcQf>7%hW-;2Tc(SR zP<1w@>L_Ttf!@Z&-5UHf0N$UuC@9@MJ!X2)J_;?7-@CiB)nvoSWj1p8l#TK~y0~y4 zb?Lv+JaPoHT`=Yjt5ZJ*ZV2K~h>w~id;GPxGv`bZ>$qZP5t`;F@G+sV$%>CckJJ)2 z&d9)U*O#Wr*lz#|cv?C->VD(#c)Vv9U>65m7tMDk=fArEajRs#KYSPgB|oATcrEq* zD;y7LM!5fXbz>^w?F3Dflsa|n>ilq^krVd*Q|J!_dfo6VSg}KwkrPmAlM@rY_R->c z_3qWM4@fH-8pJ9@^uPTRTS{vY`m1cTS|KfL=^l(Hd11xv2xjHnCO_aT3#Dl8VLyh( zuU@xMY6I}b1zXM9DAEntM<0N0ucga(c6Qd{EZ5J0hGe^jbAB&`Th;ETbA17e?_d=K zt4SFt?E53@vlR05@s{B8jLGsXY6_%{*;$`Yr5jL&uCvstdNH$<>J^@!3jXN8tg(oi z&^$!D7b$&cH4-?f_GWq6{1IG(Ceqcpf~Z6Y$lm=4*1)Q%qEwSDI6DnxWxUPW+81MH zsk8&^u5)1m(3EA1Zw{WIhtpou;FYVqJpv7|H0^E47SZBRNG19#i@M?jMH?{lY@;$) zXU^y7_4-teehv-M^DklB=kf$;18=`ws{$Lo0(LNz;;3azN1#C1wug@_Oiw=tkMh;# zgoKouJCvXviRRB7jOew$vl5%`WP+2o?fR{@sC!6VW~Qc0ZA!``1v+L123$7dW#IB) zhOrMlSUP3q_P{>QQA9&!OJWS#wLU3AGv6ngVpwG~mp~^D-!#9Dj);_h!$dAh!mNQ4 zSQ-O=iI(y9Z8NkXKGW4LZnnE=e$o#{^1&>nLm-KP4o^SF+5FLTg*3R-A?V_RD8b0Y zK?zrtLa_J7oe8V*Z&Q| z3$=OJXLAGhUu`l7Jd165i1Npe82UnC!9q_UWop}8dlbEe zHV)irF_H84U6^eUp$}UZ*ku^>Li-4uWY@qL%dF=XFId-8a=|If`RfKMYBZ0mmHC?3 z=#$7N7*Z*)UTT0$MtXn)o{SKuO&jKGA)*DBfdc?AaC59?7>Md6l;XjTEz)%It5XbW ze0FCj>Zmc$ov<*CiEODzNaPx!)eL<7<=+YtE&pqUa2cLqi(C_Q+(|k5 zjA2u=Gn0cK=~>WiJ>dT#l6Mvf-LkfRB)1l6#I2Bz8l}55WLhT^KB^%x&B=92j{-bPbbSTO{n#E z^z}u)wMTei@>>H}+PlMo0Mmd)#O`@}rqfhG4G}8to(F*;AyzL775Hz>Zyy|BV`1^8 zFsl*#*IoS|pb72&3(Q=>6~u}KSi{&0?*?B1+$P31M|=kF*z14&xc~8i{{2S( z!zKJ*KJLFh(AQ#2-c%@X)YFp(^V^J{9rt+s&9Wq+22>CI*Kkb1MyZzPr2$a`9}f?h zRS(Qu-JZS{Y@n;C$@44_;?>suJo z19dy7d#$pos;EqrTRnjRLQdF#dBL#)6jrTyUfWS0Vq>9zKM&CWEbQTIfWl^HcQ=3C zIe>6>F0M$)uVA0*;_}cR22ZJnX(`fzSKcT2=rvyHau}b2`tI-Z!#3*#D}*>(X#H*L z=ma4EG3WC$n0K^(Jlh(=Fbb&}z));5zJ4&Unn_^7g?d#Ax5)!&MBiZ+m3vIO0m0hq zux47f`lX)@*DfMtK`aAJmtbn7CRnOzXb@I=1l*rkPp*5o4$ioLT%cSIv8gBoF8h|K z`)GH!Equqg`}ywP-aAU666202xa-mSa5%wxK}}LJlay>#{|gLtiRkp|GR7(#4ThOUlc+&G&mu{G!cD>cu*cj!@`=V z_c-nGy4ce$HKpwZ5G$FiP-{E+6<74=A6=L3<-Mv{p^1QUN@?A(s zA{_6q*p!gMtTNp6JdYW_2C;6Al>&YgwGMxrfK2WIvj%Iyg9A1M!pYlmh)pVOM~rf0 z)D9GZSl<4y#-$^WtN7SDJS%N}`45Zrq({nAI>K$xV(oSVl~>t)X6Bsy{Lo`CI!Nmy z465U~3mJWrT;Go#vCCRm)ev>dOFxzXHfhtHhcg?}hg%|U&|BYuXU;}}kKdFUN*st! ztcXWy1g;wiFZDmeXNeZ7VPZ^|8W;0_i3Hv{Bf(ynf9iW;M*7)X&fx$fP*s zxTr>;R>JE+UhTWviXqOG#}^ zaU=i5=J-1^{jzeD@fYSzO;t!WpaYYvw4EF-)a$`l#~CyPkt|>|-a$(G%k5@dh0d2S ztO30I?{=nbCf{?)_=mmr)z+xkHeNnoUiscUZ`0MjqX z^WV&t9p%C4`W;{m5c56ALxh7?U!BJ0T!m8P6KbSa2y^xnRPiHNxo>x8vq6jCE~=Vl z->wkNbbd0@Ad3Mi(Lkz_Br>c~hF-v7S!I=#oUE)81jQm!X<>&hG&Q7_K+~u{dK70y zJN|$iq(CtwoG`9x8FaqjDUtf*rN(4zq*{v;A+M6D`W(?ak z)8{xn*(I<^y|(HOyHi9GV`Cpby`Bjtr*k(WA$2U4 zYa${f)KI(rI{lu%I5a}S=#?P%+1Xj})5?UKhgSzPDvB@)O>)pSYqshsh=>+eRywxl z;J)~0;*#^41H3kFx<$rwwxa4~h!~ZGi5UVDm>$u`spiJ8Cs1$(@Hq z5Lm%)H7eyy+gw54(V8yQ4VxVrRA$j5(ci4S**QyFBjctAWda^f@hk z{`k)?yDf;1$DlR+6E2<^1X0G{|AM}i3E^w!XDszVg{-#=oOMdT8RY)UHfg}kcR`v7 zQR{aN!ZHzdE4K;8>*)#`3P}Xk^+(l6DmO4N0L}nLV1C2^>mHaINVim5E6nzDF_eYH zc!MNV*kxNEY-e)C7TTjwu}I(aWsD%gx~RLL9~_kows3vS503r|R9Yzg zc7JW!n9tUULZFnDqX`Zmu9%ozIH-QPSiddE{Q&YyH|33O4U88s6Qg+qB)nR!k68q- zU&kQE!xK+|9nw>q6QXKVM_NGJi$y9dB4Tge`3p4U^m3NP#-Z_bMZ$uDaY>(QcEUP< zpQP83Y)|uh$$dtn4I1_D-gIgja}tClA0yy+0*SRo#J7w=;EA%%=7FkIRSF%!*_^6N zY&p=QM}S=bNw(~ytA9N3-7y0Xps)4a$Q&H!PANv7i}L@Vp2zA^ybC|0kw{B|fj8ew z!rGZv*UK)V?9eqHJow>@PC)ukTs_)Rtyw}?<;cpzn~3Au+z{zpvZK4Z9@3sMH_+{n zaz@xPz|jYDOkpah`z7Z#5fRa3l|4PUA--h)`yu`}r1oEQAE4o@NCMg275YN}+^hKe zf4uNtv><{iy!wrQW5ECG3;zucrKKK1gMs#So}Oob3N^i4&;78a0Ma2whGNlheWI3n zn@7Hy@A2G);vZ)~PA4VgI{|q>iPDS!-3xu4fP9*neDNEu-u4JQkW zIo2IXEY-o^;7Yy2g!rmmfEh$|E45_J=Rfwj#Is@Wp1Xc>-kK_R*B_+Szl#&zL zF0c^VtuJHoo?AY-uUf%Kc1ML5YlaC29Dscz83By?(lIjnRY7WS(*~*O1^7Kj#l^`d zk!=@6JpMKcuvp%DM+Ur#jX1m~0dR>1=)p|A$bv{S^_8!KZV=VL{1q&-Y1V!1Aumq7MqnhY8@tAgGB^KkY*GD}(M+LBw0X5%r9`0eBcrgnA>5PAO$&HLX$ zY-eII=M-UAZV=_ci34$UMoX&+*r07(hH^-z@e5uVCBmLRp*sNJwKTOZ%7Hi8Q)6$(>1C&r2|>Q7d0w+Vfx zD{g&fQ4Vw_opFnCqYp~UL~E-Fq=2FBf$or@K{R?HjgQA7*)MU_-{7iZ=)JI;t zv}6NPj3DMZAF>30x|{AN*KhPlemw+yQRB2GyVfhakC7aR*h#br_{%6yUJ+g^ob3-6 zNi&!K=fMcjthu?lOJqbPzX03ZLqhQRZOZZ7C=xrnyL75AX}}%N0qaKMv+9=%Bzo>Q zDJbrnc9O0K7`w=V5c&lUanwsdcJjE7xINEYhUwgaF42X7l=e0Ir9GljZ}{Qtc34!Z z4baRrP*CVl#LWbz*L=Gp&KN|%RpiZB`)l#IHqlsOQkep_6E5(i$BC)|2$pABybyE` zR}0I@VIFEvA8Q#JDhGswWpUTq6t5k0h=lGCkI>>16DeNGE|aF-ZPO=kr+RYho+Hax zKi+o{`x=i^XE>TzW*~9B@3a~NLv6}#brx{+w&5em_u}c@Al;wp?d>%S&o8p^;epZ! zKr8)9v|n-RkV@$z{ zK;uhpEc6FHw7)9f1n+xbjBTx9E5=uW>uk?b-v+8hbW0)#7CdFFq#ig=Nw_Yk>nw-z z0J8xxY2Z}#9df`hGuJRP0jT8|3QMc$r9o)g#*Z@S0VS`*gS*$cTbN@x^}@MGDPUSw z8bo*&Zlo}0LK2ebG~mU)Q~rv&gboTKwtAy~{;_gUke1LW#s&jRI0=i1_b342oXB6m z{*f!c*#e-BzBj*>3Y5rUmw5uWyt6koDA%6Wx$%|wMZnQxpwjlF9l|+Kl%ZRX^j6DZ zzA_=U5zkPQyg#znexF?1+-(Mg)mOIKow=DUIO5Z4k+ChB=Z~V3XB$Tu-cnc4Wl8%t zc0|xhnG13o5UqkF_Z+x}0Cl0UK3HxYNt7`3xJZ01`vAgkXgg^jKdsEl_jqN}&-%w$Mlh{bO+gs<7k@p& zxW?Bzy7mz4j#&6U-h-M0;9Ci2becYKV_=pLv-02!v7W9^UM1jq)SW2sPHZJ3bl;o= zNs<%M-r0;L%RR5qdARXQ?(*=HcOQ9>g&YV9+nlgiVb;1zlAuH`8F9u4mfCWiIj_eX z+SHw{YqH#gZ+EhGB5nkrzlA7vq578FHftWzlq@3hW;Bon0Hiu2pt%NsIgrGsm3bEm zVj=@|j|S&~tdP`F!{{XC`;?a_sTIjlI}fQM8t5t`XW3H z+$9>12M;hRsRNnSpy503ru?TDRMG(+03P-E z0T^)VrUjulW6iHasBPH>v8403UEUY-=Cv~29t~yu!q;Pj`KTH@V7P;UiY5ADUv}& zO8VW?!rqa*{W)4DD>uxILfp2`easM zxI!*I_EX=J{!DYs$6V~ZNIyy6{*IvC$+3#0&>_QJBT)4ZS@$a%pmdbv2GEhdRUEvj z{J~UQ-uIZn>;^66Lr1kaCnxD4LCZjn!~tGj%E?43u6jc`23zYbW5TQOEg*8wnlSB;m77%|-y-}= z^!sCbFJXjPy-K2=Z>2M`rJdn;UiYu^;{o%SC5}&2b>jwHOZvggkz5>(~t?;HpOdzXI?7#@AN}|NlVl z|7K%{8)k(r7?Xd?6=%CzoVR7w%a`ie{Khb6b`>b!nnfO{tRwAtD~BUc@KUP z29rpnyz}bIBqf^)Nuy)t?3`dbXsOPN!k3Y{TCKd4xN>m;$_b1pt`gEpsfo+KH8daP zJv{pzV`hBpbS!Mpp7JAu?(YxJqNc$ZKbY$aJ*v<>1|{Q?VGeO4O&DYfd?BQQLjtOt zFf*Hkh{(j$^zqQ^6k0IpV`mR->6sw;2}Y1n(b1s9--ik!H!cPS8w*QrYLqTaBa6!3 zwX;)rJk&dglEA_^9Fd%?th72q}9KRD%U?C3PU93N>A z?dywtPv|x9JlPu{G3hvqw`tk^_D%er)JA@s-J7X9K)Z)zV}Egmne0IPfvO*dBZKR* zuWD!98Y;;ta0!&cE)lB4QEgpaVf)YQAmN30Rq3?$_>PfNK8X=nRx*a!-$c3-%tQkH zqS@fa*wAnZ_5D9v(>+5w5MtdE@S!)V`7z`pMmf(Mm4)s$qig{_c+WXUeJ++JUZ`E_ zF9@{d8Pt#vthvVtuZpr#fjoB6pdZcD++N!EEb)tYds`bgad$R0 z&cK0jRX(ki=QlaOd2bp} zWi?e*Gzz0c{*f^;z7^dd2mteSQi><`Ut|IG8xcKkeM)Eyx(n#2S5Q_iaM-&c#6(a6 zFq=*CTk9?aZ0qnZwDoX6!V~7C6NNdA(qTU-h;IR^lZo_u1_n9}J(>em;hRu91_xks z>JL&_x1zs!L#r6nIs8I+&pei?AEt>N{l1>(T<4Ey-iOgbCL$?mL%s%kkw z;s=D95h=8cKU<7MPk7_k5`j$W&nh+R?}O)T7*0vxEY;Pw33nbQrW=D3Gt{_TB+s+*jslzxZebst`5`4M($>7g_(ziyJzN04M<#?*=k*ZB_cu5MuNE<+{VE zUu6~qRG}<474G#z{)vVZ4mPh^Fo-xmf$LTzsK7a>qRe}V$kuN<+~7#{JYnUMs8Djn zr#@lEmZOM7H*Bx4mfw=;p7Of%z#(Y*1)85zzK9h5j9(6p5zL7ZVmf`7)2>U&U_wUP z)yUjo?gN#jfk#=FDRt+J7`&6&a1%75zwHK1_Sd1)n~_T1v*>4teP}}Ncc<&yE>Pnd z#M6w}OR1nu4{6Ku_?5DUJ4Y6Ed~#wq#d|y5Q~`Y3J>E}<9?uhyf0zKaqCLoGAQDpa zAKl*HBpQRli@Xkoo=6j!*a1g3M>jVwg_D#GR5J8s3<+EL(M5z-$CGmB+fvmiWG9*7;?Z$d_+MooWF@WPjmS(G!VJ+s zln-B_Qe)pS{{xf+!pP(w`6ql7e?ws&;D8EZ`g1gvt}J2{m3(AXlXBeMYkktcVd?RM z;0I=AGja4%d+hi5V^d&p(7cgO#60#R7ri(#y0gTj3YPmIo=lf{!EPO=oO=p;s%U1M zDF81nF)_Ly1|IVDSQ>GowUumz>o@Hi$WR?$htnv9`>p$_bVpeXWOJbb0iMcZ z7H(ge_@&Z^P1bHOaH=0mK4UNP-Ef*I-E@S;3v{Z8n zr1OOjdb7P{+>A46Mp(^pzTf&S*_Yg#T>a;4fiq5WH5V+Q)A=gW+KY09*L!Zre2d?{ zj)mm~Dp$<#g{7tTAwMguMdPn8qob^zcL3g2fB7BjgTS39 z@}m&0ZLF=ayiGaRNt)-_ezIW|f^NxQyp$GmjCG}YnU4AyRW-(NY+Uvokm9(;#9HxR zC)G%U3<|l0p%)qXGu&_8H>x(5=iX^>_*Ep=0ohiKi@@fTkY&o)l#coWbUsgMRoF=9x# z4Il#hBfRHQX~i}Gv5Dz6LEtf`pH)2;kzem+XD5uL+%vCLQBp#e$J-8j9J~yaf>u03 zwcsR^^`a+}LX&KaYjIJ@t`{G$lnx@ENz)HiPC3(K5`?ODPCFi;JDIqM;Pi z5Gth7ECinh`(-WVN~e*1dvRy>VO~GxZG6Ey_Axy337^Rsu1*>+W5q?_ygGT$2!i3Lh5hReYv$P&`u%d* z=Rlu#r5+KFyiS^fz!Hdbb@m1OpW;8Kd*fQ<8^enir95guWon7ZyN4yzYuI)d!R8Zt z!+6%5_b_k{{XVuazBzJh)p-RFr{335qdbqcXTYoE=%NR|_L(=+`o3xg2`MQ;y7N__ zyzTwUpU@jHA3cI?R>3$>NF+7zW+$SUGY=iFa0Wr&D4uugQs~1tWb!B3tlCXXjuzoK zZU;Ol37OW=EX>Q*9v%>Y%58(j`=AoQfb4_$H})m!{+ib2!gvQFuTOsxEkg(I$?2)O zeV%pP9cF^sc>3fNg%CVS6_ijL(o?SHyHVjw^Ygyg1XAII_%y@$bVp^$3Tf~FNa2J# z_O-vSCX-FZC90kVG)G5KghwR07mO~e@fQch@{r5jpvA?s-{N<>Wt?OmT$gC9 zcmt+hcmBgOH|woWruT3T--ieCW%PjGO{KscL%DvoL>`8bG9s$`P!|7P67a=dyXi*N z+1zZF5{$8yEBf|>95HJEaKRS`798*0-Uy&2s)}J$&00EhSK3W-2e#MN^zLDuDvx@m>_rpU}^(ok) z<(UR2_f1F$rmUn}lP8ROo^8fQQt8D*dm6yV`vbwdD(VzPm>OBufE)52J84EvmojgQaXQ9dLycjLR3MdzsKMqd|0-rCkizBQJ1 zIq>V#eHf%+AXkM>=n;i?6K_`B4LPJ`Ui{30J)C4Qh+${`p`-&kw{F5k=VmBpF&C3| ztqWK31OQAWb9! zN6~2#^+{V|W28G=#HcXGe399W{r$4cI3kU2q|&S-9|PJAO~bnh!eCJgkA$B6@xpxa z?_mqPCwtFPhZ&H^T@ogii^Y|cbk8~Y-u*~3JmNACpzO|N)!8wFNz91ji1+!Ncr=+v zv~LMQ8}7Rf zz|Q?z`U+@80K6+_Mlj=`Dd7b47uuH5p7~o`i=*#E){~f z?HgsBfmR{^v57j&s+7ggs8mR{WO_ z59iw0d&7|FI^TFf0nKM@fT6W7OZqFo50oN!K0Z1a0+ z%`e2xx6>mI!gdD(Y;1=_btD_|+<~FvK@wy{s0S{l5J`;9+tys~148u#ximt#byWoe z#n_b|?n+v9^#Vfw1?91;7p}x-SJ^S*tE>9j)t~#9?4AKE9Mm))J=H$TeDOn;p?7E~ z?xBA{h^tX{ZJ-m2j!<}JcTvc$u;?2;Ly&{JNurl$#Cjzqfy=i zz4Pf-8(}2fl<2jAFHg?G>k1!@QZ$I?B-hRFDZKNEC|do&TNR;x#_X>fIq;h#%7%DQjFHs^)dpO^BHts8}L6 z)y*z?@>n`u4!%B=69yw?Vmi{4LEk%nWPjNPaY>UrMRuLBPB0UJdD@VXSbLa zQ!r<9RdCPHTe?tPcYdE^#vg>t@)Y;&S{MD(%q1!aa0E>ruu^BsC?u zuyMX>&6Mbdo2%DgUyUnjYDFDjI{v%u12rc^_{+i#-@es1*MC_Fpvuix8%XPso1Z$g z(r;E2efgtd?VmrJ0~I+Vs2h>h2~GPrsiHYYYS!wMm05aKtJ4nK1LWmv3qwiU&KZ`S zM*Xg$*M^8V_0oR!_h)`6+*_MSeKdJlb=%<3V&L3j^w*)QVddV6^0KVbZlAhdz+oX( zwwZE3ybkG3E?1~(%4=vA`iOf}v8C?dztWQ4{?D7^H&aq2&uj&vE>*6o#b*v0?_*>+ zpKqKS$Q7EV4yQ=A>uAg`);;3mxD!+q_oN{^vXn%?T{$Ep<845=S=FN&hNEXbZ`J7q zs_ps)dhG|+d_Ra8jf`u~&-44gUX^zr9}BsdM5RkB#|-LNaB<_?MWqd<%aD{+`as9r~*;#oAl-O1sJ8eGsmFhlz@Wc7g`Y7QsF;S*y{f67nV~m)ucZExWrVix9)x%SF z)&?t`UCwK8f9&B}3R8XJZ=^Dd%u?JwtxmPF+bhU!Y?WfFS?cnrkJ8J&#cWW63qx)@IT-XQ2u$bKRDiQkum4Z zv~w98eC89ku(nmXY04&ao;9z~^m(7?Do~s)e`9+oyos_;YCJKqYctKfvEI7+WM`~p z^~aCS+XCk`9_Om=4Jzl{g*A^qG$dV4K6xM0X6N93BEKepi%-}tV$uILHm_uNo<1iq z@8<3K~lCg(j_d9wM`F}(Dbw&t^m!DgL(7JkR3(41+a(_Hoa zU8nuevZIlY!jrgD{I=I7#C~?onS`a)s|TRJzH7WbQ24FifPX8OoyD5lEUs*S`JZ&RRQ--9EMbwE81&AM?rCSNF{k zC-D=%&zhX&*|`)`?hTqFs8Vh7sD8ZF)CVq)U2aF?rfysO+FQxIxtXFFw%ZpN#f&aU zi9^n3=PKiDtBzqO(+WXV)W%Kt$gp!D;Adj@?dH@J1KVoU1A)E&kF&Rq%5wd>M(Hl4 zyF@yrTR=rR73nUKmhO~L0i_W^DW#=5B_st@QaYuS?z5il{=MJ(o$-F-JLCMbhuh)c z;kobYS~2HbbNw3QJ?Z&y_rMecwg72)=1WS5*k4A2RhQB$ZR4%du4EhQJ9_;w8BMi< z%gcdJ!4UDZEw23c6z$t)?4?93-$(YR9p$6u52uoHZ3G9YbQ2SEX=L_HcT*f*`l_xS z?){*>Daq04pmi?X4QDd%o7opP*Z1V5E5n@8-iF2bV8sHGuG=1JgQ1aTpHE+SphnIN zFoQJ-eMifKTO~7J>X&~~&l&l=?6%rrr_ldarqf+f%e0{f;@_NyRQc0-p zwzAdz`y1=H>Jlp6qi3JLIC!cPG<{sS{>CAkNJdnBf%v_}qLRtEIJp6c!uX<`@U-JJk3xkjXBargFs)t{C7vGi54CUx7B&8u1W_+Nhe zE79q!sqlMV9Da~+9CqWsROJ6w8^8qoM~edWUH5FHjsrDGZ9{!OTZ=q7NSAHDu#==N8uZ;PK(cc;&$V^|q7#>bDsf`ZB1= zMisjs+t2uZ;AmS$Uqhq7R*-_cbb3?9-OzNmLurQcuc^d8PYm&}{#Eb)73BY|^8P7b zppf~e*7;A-^3U@`lsSJV1pa!8{4MMLc@6*XAMkIT1Oo_vKLe%#U=|Vmw7zd^c(-SG zmrWJ_mTHxusQczvR-8uoySBWufZ!P!`ew&HkMJf&)_eKPGpB?NelsjL1m(Wkmi3ws zCVYKm<9%`rRtm!2J|_ee(q>yLTg}LXB+nxxI{cp24w1iq!)r83na9Xlh|ZZeWPBqy zfMfWv@RO;!)z(2NOd+BE{pA#P|H#;hmAM=Z!EwtP3-#5Z<_v+y!RB6;P1DF<%SS{k!|z0`2MeA^uNF6|9T_;!)yL$ zF8%MD`+Hgbx3~W9EB$W=196J}{ojB71dRZ^<_nziow@P>16tT6MiPzjzO2Bjd?4bn z$PDRW#8R`<+`GhIVnYvfb>VA8jK;q*@C-~e(a6#eoFl@qBZ0dc&UmPKTzh(VO_x=XW_b@xdt8?yhwb1~tlBnhnp5F5`#&mB&e2$tMM`#$10d5MaJBGyYC_*9`GnrQ?AD0w_FW zWMmM+4eg6v2VM?#3&N!DYkx;dxP_ zXa(y8vE?r4ludxg559H?WX1I-D0e==%!qKK3F?;IgCRT#sSE;@OW+Rk(70}_RHntJ zuPfa*>)~c$S?=aT%O$>h=gzgM>FIhCp_|efQldmEHWY*0U6c_&mH8eGvf5;?HhqhE zfQy5}kNza-Y)En%U;S} zMOoh5RcfO*ao49-4~BsM6V7>V|5n7-DlK6#*TX`Sj?6W}yGnf%jpNp>QPE%x6N0T% zsU!_&4!$>LvBb455!tD+WyBrtR{dOFX%OFSAAF&*)h-97U9i8{X)?M>JySB)xFn}$HF(Jnpab@K`7+rDBa_w|I=hd&Y z1+_Uy-w=Ms<>u)5L8sl4>_F<=yM##)*n%lSyY~>pirK2M#IY-P_YM89_WZc3t zqZfA2+tNIRi4$yz=e8SH4l(K~V9*TcZpM-EHy7N*+(PhHOmb9-oYhC|z+jTkU-s+6 zyY2S$=`xxc>(c%pHW+l0y*Y#>ZJimHL*7x#`%9DW+a_%&^q-FkJqeRf59vjid%C;3 zFX>o2+<7Yln-TNhfP<^+2Ds4hjm3yMqdd|?|;;VLAuB16M0&`-toWCuNwSJDpS6wy#fDr@_*q#a#y4TQ3>j&_SfwgWCTvtk^v8{jv#YDm{;O)YqWHTCG%cLQWy@op zWtk%1QDA@f%$Xz%{1B&44b{_fGvdAB(5==wAd`z02gpX#^Cv#W>CZK zr$*6Q1IIu%4|HN#sg)!I7!YIP6v<+L+P~|^Wr42?)bx8x3xuzxG*d-~|7*DCm+^!Y>l_l9FHOx=en@r9^DCx$o*5&&iGfP5ll5 z`X6h1gA(eIhq;v)H{wdxVeCgzFnJ4Y15FlFHjp>JsYOka)j8r3&b6O?ye*rO?oZt6 z^X%0g3gMD+EG>D~%RR3wKYe_wSS$C-*O)BUOvzw>Rh3 zjfj2u*(49fw2cG^O@q3=FunR2_UftWGE9@j6thjx12wxn$;{{?rRFWB0%a|rN&aIz|i+)I=+Lc-* zXK;m#H#|+wVs@{D$03nJ*Hts}+W8$qQ{2=8m{eNNlT$V9#UWusY~pIzuJ8Mz@MYcMhV=`wrbKJP~%2a?>$Zs=JdjB*y1B= z6ch5l3=hrwra;q*#ND3t;EyDDk%J<75)-y@zx~N~-KN5H0JpSKEc_v21NOx)U0jI< zf;`$T_Bh^dH;1FL_9rGQU(YAaM)EA>@Sl8;NX*Hq7Lcfu1_5_T8tqLaair4*`$5i9 z-QpS3jxN=5hUSJp$nGUP4Vwb_=01zvbu*B2EMACp)%l;U=T8xgq<2>jCa@^WZA}kV zy%{fUI-F$B&p;vSUg~{{o0}^&Mpiln2S&HVa3TPH&lj+Y0~|oi;c76fv|`wn)RnYy zkqa-`za5a?R!1RYedC^B$!zor?F0t%$asRq&v!d2Uf$kHzenQ|#%b<2PTS!A#%m!8 zH0Ti|*Hd1pOH&HwFcMLh#r0>|lWP)NE(Tnltoj~quFoN1*arA#vLcs0D5grPN-)}U zLheHN4$kY<_-Q`RjExPWtmH zSrE5zMhTdgJDf!gnx918U(1^W(}$OYBV3kl91&{x#DaduyEbqA$U>Lcr*o-3hsJ1x$$^QV4y^7UvNpzk+6qB(!~<|9iKKpM11}X23D=b`bhhpI{ z`3E>(=R1;f$W)qbSXrhm#N)u|=ROx`z|=Pm=IGS+Q;AL)w*l$J?uarkwwjMcM<$kE z;*^M6%gt~KQw1<3-)Y#9Haq(9Q@=nX+dUsM2q90`%G6)@FJpGiMut6|r# zEI^+vGm}@QEwo!O$&c+t89tq;b$MUMx2{1wk;AzFFXMYf**MR!ZCl@d&Zz6tsXJdd zI|oLpP&7*q*c;nfZt8y2aWPgb4xg-xYZm2v1v5PU9XYXC;Ea(n zj5eHy-2`49#P{^i*9)W7;!r8FMvt`Pe}N44&UiGb`nB<9e+k3tvd`yRHM7-)Q)XOp zjESpAw*5y7F}yfgpFez}#dwP?p)~04bO-74G&@71U1a%FTY?`Ms(Ql&7WSLmAe~Ci zs|F_@DUV}R>MXbwAWDDADosC1F8BmXYq`l==}1;RLsU}|3c#jY>Vq*_xk?k?wy^Uo zAoYJQK2fq!*nV>A!Hw*$$(LBtNp;mY>@HOs_uK?s@h2qw*x! zZf|a4+@YK3Uc<~m%Xle#TVf?c@;1&jH(laGG%`3-i4XB;M2Dr>q+b{js6NL{=%o^{ z#A0g0ub!OV!CFWFHd;K|2RD19kW<@ZtU*_q zvF~+q8E^buHVPXhWM#X6vNd{i4MUb>)h5$>gUuK~vql?J?&6i`8P^zcO&LpvnAc+FDO6T*aa zDPNDInixx%qvzBqFuizeWuoLF?0Lu}$o#PU&w#Q|@~Nyp=A@cq)l>;`c@;-|1;G}F z2ZdR%;|rE0`0Xo(@aUxtdALa*)w_StmK|&zLHWfdvccWl&3~v1d~HHCW17^w%9of&mdgTzfc#=;lKlL{^@o))hhv* z4mEq-QA@=G&RNr&TZfnuu_u4*4-ZaD|71R4YF{r&sq8qGfca|Hn7y7y4+vqCjZ(8ZBc z#dBKx;iv_E#-VV`G^<07|M#1Yj|Y>7?#CN-IlMnKp>&hOY>3?#-y#>rO7BC?;Ga4+ zhGKMn0KcJ7--jgLD{m(?D)zV*4X#T&jTJeasacGBF$&FD;O(D? zCyJb@52hBE`OQ= zcU1VvQ`AkL++II{cg(6OUcrUB+K!yTnWrGP7B+@q?pBR8YQyD`0_ve$!nCp5PX)+& zqr|;9i;aCtmnZX7sq>@!ep1FalpCD9vx)4jDW5XSxW@rIeA);s?o+pyYcu%Unnve- zIFsqzMY33fi=tm%Hg4w_dH=`>DgeeMPnMIxNvus`r?CQ}5sb0>ZQaWN;}>5DNe_zkt30;!r7Yii`oUa{ zkFvV&w_Xu{)BVrnCg34-e_;Av(8WP%psFXqasWwTBsA?DGG+8XsQ!M|zV=rT1!+u6 zh{?sC1ayDp_X>^i(n8luK~g^fr6#mI`O~w0A}h-qqv|r@F;I*RT4^5OUtFH;tS&G` zOvCInb@TcU`>Qud10gU7C-b|Lm{4-J70#(Zr9Ze>MG(+_Y($t`NqA)NTig}kml9z5 zx>xJ62nrUM6njW|uKLnyxnEmdDz_2DjF+H`zDjR?ktZn?5`_E+oxvD}4e!x;D-^le zC|(KKkAcya*}0nk--k9#ShC+Ve%qT$)S00GFq0WLbn!e7REDPRRzgrki3Szh-Uc{A zm{#Ikf?W0n@(n{87$y4=AS5SK+I;#k0?IXE!-zfzxR1eY2@2g$v!-e<6?b4ty3Dde z#9#Gf7Gs#k6fSD<<@u_N%+a(PjIp~2%8r(ekA)+iKcTUW@of2WHaMDU zli7Q^uUZ9iL3n`VQn<^Yfc*hS?TPRHnhU&5&-D?O*-~G>#G`#|#)kb7W#!jwn(*P9 z#mkj~YIxbrx%3;Nc ztlcB5AZ)P6nAYs1;bX?$iVq8Mv1e5_WBmj%yo8`Mqn{riK0Xs}YlIurw+ zr+)1n{K4U@XpYZ15Ojc`f3G0!`qdbG?Dcsw5o z@K-E%Y3)KZBE+q(asI&6f90VR%Zfd;ay47B?XjlHcvO$cElG2que zTw~0)6bI}1ZzDD#p`(w211MJxz*2{V7{efET>AO`rF7cNbjBB%v&{;UH_#Hk^_aAS zt7c^$Jp=FX0xUyx;N-#XXw}k7>B!z5zVWazJTgQ z5Pn?*74wBJ?EX+s7sj7~pJ@*P@-1W}m`a0j#+d?5X3?~#XY-Nt)gVTOf)~sd?}GSV zV5NIM6}xed51R3yB5&PObAmKVZ>^2IHD^UH889f zFkzK^usPudxff>^tQ+=u{$z-`k?|NF>;5F%gDZ>J3j@{yL3Pl++=C1jwgOSN@o@4n z9LYW(wJ(xc9^2%sLL_VO3zhX6Z#oR#zwWDlFos1F=Il#X7Szu0KQf05Ja{_@O+c*XH+HG3VBF*<08fc?0 z;u%x==`i!B;KUjQUL1fG#l2o^a!t_(U?n7;;0JAS{8E~vki$4ppbR*HU_GE+N)1>z z@+Y9exdu5`X_wXu7!2sH20%Dkg>v)1!u_m zfP*~1+-ykVe-j^6JNp*vd?FWL>^P=jvz%jww7)n5oU|j@A);UH5Ot9zXeZ!2C)*SG zD=u@9j9dRk0*#of<@Qki&QuMdH()t*_N$j1Hwe(a#8DM(c)+*7Qxr`Lm<@nV+L?c5 z<+l`yp!N;ShASs_GiXwMU2$cgdv{-r__G;B`9W-H2J%pIw=iz%@a9Z|$dQQRtC1ap z6;5^4#+?AJ|7M++CfMjQCrhskOZv8h)i!^b#A1@SS@Q63JVXgO z>TchE-cb~(l2+Krt8T1X9Q3Yz)2@$FJmBU3%I_DE|Ml(RM+Oo6&Hp&zC|UCV-uvwT zigp_R^FNjQuY;2Pf2tw=y;b4=hkyG2?{oj#ulWDrb1~B{4{KU3h~G}}fH3Lput?|@D>(eIw{B%B@awQ1l|CxB53a1QZl zG;Iq3wJv0+Uk!M_A#QC=RIt)t93O=eN}J|m@#zss!-?1#GL^jr+GeO@SP5>qOCy|I z!9P_y{DMA2=djeM;rqV7x;0{aS&o$}sSUw#@EDz{->Sqag5LsCEsRtdg2P}mDQD8E z9r!qb+r)Wbk{Csg+<-{N_56-MG*2cz-FX9cMJWRFR$G9BkzR^Yrs+O#5|4S)0+EDh zerIqvO(MHU-32QGaV(J5hJQSUdj>-AV$Jtm7e}+ut9O8^7gFW5);gpXC>}sgT?a7| zMu2`mj|d~j=FfIeL^ON>!%SP#7=o7STRuT>a^W9VZ$!}r=_4&({QS<1IreNzCTeR2 z;imZ=G*0_=RiWE~@4N#`TP{yw7?#QruwwaFI=7mR@;=Ee9o@-{{nL)i8%rZ5kQ_i` z3tqmw7`F}vCGnxh2g};eLMrEtBgvy_6TPeq-I`4A=jyNX+7(_I0!x z@)+<`HtE^$f@8*ADfOkDzdy36a+tV|wElooos`XdoL`9BE6MH z?X&zI-;JC*11P8VcNaCxLa>sE%=)=OctdM}lnBZ0b5XsjG?g-IZ@8tfgtpfxNrhxF z8?S#-er1>5JGnn1wCTLst?xYZb6B5cN8`vWIaXDuGfc^B3k~ zA@y?he{Ggzp{HHyP?f)7(aB1ibL{;5e*fSDp^gz(^8`if(fX^g97-KFluOqz2{P`f;xu-y=m2PNxj zQ4sd(K}xPyh`J8-LO%>PY}YQnfE@SUzQ}JS)yCLsJ{$%(ossDTZ8dV065*t*aHLP2 zJ9&S3_y<4~1UEr$=p2me4Q|GcO$XnhMnR)Qq%6q!h$%xjlTf^A%Cm~Qmtz(YRYR=^ zUsCrIj4(h*CZrbLgtLa41~ukBKpG4v%5d!86Azz2gWu3vs$aFyNvv+U3YUhIfKJj2 z>TmmIc>a2i2)XECUn;`gmo(G=a6(&6EEjS;hmCVqK&pV1n$&?#LD;o}z&`Gp*E~i( z;4p1UayO_Eamm90-5AveCJU5%TE}2V_sa~*ruHeIyr8Tg zeQ%5PnZ)%F3SDuy`v)&g1sv=rE7z~xqhVC|g03MnjJi0^a2X-%WRBoYVpU2O zv{|d}L!*P)tBYAAHNherw{ASCXVg8Y-!GS6d+&b?R4o@5kYc&{;w2D0<{qw{0^f_5 z@ZI$Zr)wmkpWE`63m_ahCI;t$>g#gOjcbt1)&)s6jX+#5t4 zE}K%gCA-I(>17U^L8HbhYQ_f$FpFiMD&&DZ^=Y}6LTuG#)%a`EM@efLZ^T`2ymTKj z+1c+iB;<7r+n+vO=uZ~+tK6-%xRyp6o|Zzp&U3b0vfw}~fFzjC^=gp$7L)niupjVR zAKZLJvnz-6sAPry>iFKZvJLY{ynn-2k5#Q)I8L&$j~%?QGAi*l9Eu*8fxXx<8m#bS zLAGt4y1~aE5#)%(QloB}Hw4h}c-3_7#mVa1hs1J>W3%TgB7nMvDL|o3*3+d|vhBQ) z_P7KQ@ycGKI`55JyrgyQVFZ5HlhkG^JCZIstd-2;Wbf^jA=F7Z2Xmo>0{81;q?`qZ zfu`~sH;bGPf==c&hptj~Ko<7B?fPFllA6W(LPll?k27mUOj_7tHjBsyaZHIYm8Ex& z2#1uT6PL&h>aoYT3avy-RxO#D40?7GpO)nbrQ97dE3GOBG(}OQw<<@0>KTRsDG0`U zihARlDR0WIpV9;z&l^T?O*HhOXhD}I1FS0a$g)r{j)gt$N}S4YAGx1`jq9rFY>B?k zt%CoE&8H{*anW^_(4Wu_hA^ASwzd~~L#->O3w;M^9PwDvY6e^)a6Z1pl%~%3)bHyZ z^73jZs2y%;KnDT_l5?G(;4%p5*#tuWJScTEjQ|h8U+-lxE-nQV8tEcLI(il2W*` zfm@g=!mJ!%zSYir6R{wI3iZ7+F_%E^rb@c7YTTpG%#=D{!!jFi_JnItmE~BX!J?a~ zZu3*$C2ZAumjAR*vj2;kHHBP_?J=w!TuDCf7MFxrRwPcV-*0d2sZ?arl6j#u42j>Q zKfF);+I?r<$?F&;{NtaNk_Q7%`b?mdu7EHOT*!;L6e}-(s4d(yuEfdT!CyYwYq>lq zb%M_C8dBQ^IlER4G(3YsuQ-cC<~Y(x4i}w5zxYyGzi|Idv7A&wdH_8%POZ>-E$~3D zise?BOWN|+h`#N38R+~Qcq?B2%ncMEDfVb z^^AlM-G^6j-Wb^i&p{KTbC(*jc6bYg<`srq)2=N{{XydEu$Q%jRGDq5yx=YBXy~R*OkHP!#J`GYR;uLhe37@_}XQ-kH1}gD-A#w z+V96ZXT^@cBTG5=j`39))}mQlJK#FBH?sN5!soIX&r*>mcMH-IlBPv$PtXr`f5Fq| zJ@&#~+`!q%1ad2Fjs6EmZ=)$4i3MH3bvESgA|SNp<=)+8=!pe&C{IOY^k)hz0WDC79iI)CX+ zw`;JO^&q7oA5(8Z-|*r`0UdZGE=dO+E5kh zOZ=dQ++@dOj#n{hR#Ua*H&68nS{>tkyM1V$toB5~o2c-`#JO%sF}6{oS9#?P zAvZq}Oyp?}i!JIV^`=-6=cA;n7KXEWqYgiKu_CFIKM&cKO2iS*jm1gBz&uNmn=bX^ zL`})@2+-zHVW`+Cl;t`~$vn@q>rCTCO+**Ch&?JVAc5r%d8RdO`DsgZynk4}Jw?oxv9nrhyDGV&~QYWK@hfwKXIS zQ+jg?))90{qGCW$puWa@U#}!WV**IUni&L{tTsK0Waj!ig}{8v6Do#luTS|tBFa;H zgQFGy1)#hLJW3S9phYM~bmU7EjwL z+p9jj1;sJTomB;T(#zdQ8RsvCyScEKmA7#hg|FS?CQ1U6Snxtou%(1tejG0X#kkdp z&m3ZE{7Ys)4)`dc$aPl?N!FS6Qm7{4_88RMSc5QBMGBpX-nD(2rnc^vAghU+t9_07 zf|_Y2En@?rpK_jW-$-%P?!}1vIM{|s&lQ^RJVm-s)*-$;m`d1z&D>sK$hb@sg3OeW8CvrbN z>T+g8e^z~bc%W(%9Ci3nh*`9N_N6#8*TchfX`5FHSQXH;>km{as_>^&zvJ>EX2ez) zv3KnhdzSe|m7P}UsET?l;a2EM-y>7}$BwmJ=Ty+a6v#zT={*Y8O?D z-OLg%K#8630d4!HoMnGnHuu-@8;E0uB>vi@WS^%}me|8EF&}WRy3U)gU80@H3feo%Bn?WQd2Yv zlw_}-whlrWJki*6y5k^sWa1Ye^vV}zJZJHLKk|ao4x}`n3u~i)d$sG%^3#M-tx4u$ zuG4DJ6tx4a;O4Y#`I4*MnIfdZA;^Sd4bxK&f36^dKh6b?ouvJAE&sFNn;O)osJxZP zX{*cF!ohUm3k+CFb5u#BcZnRh!YK$$LgW_PX_whiZ);8?pKbs_WYEPVFg*dvGUy$- zKLn-w^xPBf!5$A5yEyLC5B{uIkJ5~F{UlXsiQ`w0EhepdoWt$mlz5t!^}KY^S{8K( z_p3rd4bXK?MDXHAmQ_^=$I+i1uscFa)^K%sh98-E(1G{iwOB~dv0gYL?}sf(ft0F!Tp#D`GsytuEB_X-C#_5KaI}LVt^9Af290c5!S6gGVQ$@d&}ST zVHfNWEob0h#rnx{QjNnJvp11Q@X|4u(Yb*9Jp6nC8qncSX+NY9mY=AxVB3L%*y2su63w2#;V7iIo>uV9!x)Hv4(u6nyt&*my_+&A}wyL z;v(jC7)xo;T`FKf!_HhTS5XsUDRtD$qx@pl| zHIxHS`+ie@@)BlS#{;f(>D=C}rS!p=>6Ae20lyeA+8z3m{^XXHgqAN2>3#JH?>{{_ zWn9GZ`L{J!zq{@a!R4&$B006}2Smb~>Y0tOFd%UM9)AK9VW z%eazXa<5!)-R9$wuwx6aq3 z;fuYv`a-Dvt6OO}Dy8LBe&D17z7XI=s?F@9dk_RJ2tR8Os`wnucwW_+D2Wuqs*9hX zekkF+tG?p>ZLc&eLxN^I*U{x=T?yx3CXQ^pI={w(P}ZA`;|{*DgVLa@B+XaFSZmR3UCquf`Z z#54k8VRN>@)9H)+^#vQLHOW13?3Xyr5lkyALQQGEO6e-iG4{0fmR__(4A-4LQe}xy zGtMDe4+L|81Y+K%KVLWjY6%2Rues;dL$}3OD&P1%n_UI{LM)AO8qNw_6sVcO=Ar&`oR5Nf3DFSHMm90T>a89)BD>FAWB7$_bj$43#F^mIXniE*p`W{ z(@KdPQoweZr}%p@uqpOkRj$^cwTxGvSK1Ct$`@P*md_qAn@qny_Y*Bvu@Ois(!RPLz#dE3tQo_= zB0=*0>C#OL0xvB19_9(3k{T}=qZheBMlp}AINgD83SN}0r#U%mOXKd0J+?bXS$PxQJDt3 zIV|7<9q5Nk_;A7yp4K2H1_k3(Y*a4uG`m0c(u)n-3=!cAU0diy~%qMULgDLv*%vOV=?e z`GPrIB!HOx%4aQ9B73|38uQ)OHZ%_aQ4`)SH6x1wriAFS=UxuLW^kJd22sB6pcP3O zQ0swJ5SHcUDPd;aFKGSRJ^>gc6g`-@7Ds-$oc8s0W5J&SM6`pY1P4RtGtsiAmI!a` zdQ$q#KRu0XHXnmvYr`-b^tnni))X89B0s$>z|b4vV&%Rz46Sn|@F${P;J6e)jPw4n zW7P@Y>Mm1!bCz zJq*m;Q20}i5dH})K0^YANJOOlMIu&%rXYfx-ctyYnEsKTv!)(+O8l}&WFkC`X6X_r zNljQfkMx8=epk^Chm6a1gR*NG0y$ND3zL(*dx<<|tRPy8N?4kTK;KII#e)XYknE?_ zPICW*N~F_#{9$q<_jue!PVvS9oev{{7e4~=hlr~~t0|cgju=kOCpV>gCYdq;du1QW zHQ1PKxw?%>89+HBwghl#b$eBq^bJOY2_`S2dRU9;`C7n?ROc2Lk}XY26=U%UOAJwg^UP*-CX7qQ@-SjcKVZdKUMc}b-sjLQvNkzs9F36a_1gOX## zhdP1`=;EM=8nSA{L$Zf7^v<5J4YB(X)G7~(&`bK2;K@W8#3a)zR zeiWkxs0NEr&~esO9q?U%Lqwf`AhNz&(y;By@Im?RA4S0%9I>k_&<;PDPqpomgf9iO zf_Xo*P4UHd3sHJL29BzV7Ne<0gGL-GL&_>#3ARH^f!t8+qk3PD?;Sq^X^@{d`ota_ctAj%Qnrj0?9 zGjbwNUN4N6xUk=}y2^1T4Yx=rA8BklTO|ev?cnRE)u~5N8%G7*pH63W!!If?b@|PM zVvUT(S2Nl)l>dxxzgw^lgmHkQO_0BRELOf9N5qeJjn%U%ZlbFG-wS<@x>ik!K&#O9gPldt4`h7o95d zPExe^x1K|~ASQcv#{v{`8BjH7IG4)+_Q=4{+z46P<$irRawrM|8a8r;NV*vd&Fx$; zE*$(>jHXw+o8dMX>_WBh5s-M&vUaW_JmSaSi$0L*?en(V{%jQ64M}kxeFo6s0mwfh z@u@LBLPi#X4SRls=B~5Bcf$&1*nlYz>;yao1RHcYShi(9_`7LB8&lE5>$D-dl#p`i z5&d?6N)Gb#J=&)_kn`iE_9dYVx_$jcq%@v3RyD;$ldtFb^K%Vg*DLTlGa>StGwi<3 zl3Y}dfWUR1suknV89xcMHH4p;goSNGP{8Rg&Xq zY|Vn?OuR~XvQwMM?@YBVd5;$`c9!_^)SbqE6oOKh9&+;Bt;enyei^4Eg%&TRlJaBk zM-4ownaNXcQ+Ov!Sq9ne5MG@?f`;(?$9J+?x3 z)xcm$y~gyk)`>Kg;M=h|cP}4)F(+n{yS~MoUYGfYy&fuy9KVSB`jpBjc2zxXXpK+{ zk%d-UE6{y{tWz7#j9}N1hh@x=aq-+Q4t~I@!x7{W6gHl74MOI3>cdFIW$0-%(tgJ^ z0Z~LM1%d0i$v>St@L7~zvq8Wem$Nukh=$FBEx|CG5}Wv&v$~@>4h4k?&siHXXNoB7 zFag0(tZt94TLgRmg=@+ws1nUc&;O9W54}}=+fpZt;kItM`DZ?QHU-!EJCbqa{Vv|% zUcMH`FEZm!T=carK(>)l@OGS2(*aou4@3@ll@z@gLZdSyDaM3~QVl=I9tZ@PAObon zkLTuie1$Ci=MYmdY@DH^yhSLm(Qt4}U~4I;Nk9=dMo8{u1WfRNChZ@x`@#&$!|d{= zj5nL12Eyxf=}w4)a+unP6^H~uabwGKc;TWfW7c8JyE<<_p%`~GLs5iytqN;c#Pyzn%Au^7XQ!<262R?yxO} z26<}`ovDyo7Ai(M)4%2%>t#tc9I3R^zAd|&)_1C)mCY7ajaq06Q*NMNC(sLifkla2oo%qZ3Vc>j7|n5EH8t_iOCK`{{%pt~&@smL@fH-HaahX5#PRwS9Zh_>zK^1z0GCt| zgD>CbOQR2s0E=JT(+qK}P~Ur65IM#zCuHtp+}Z8zx?2^#E;^A!YQ4B3J;GSnI#zTO z^`>f?PJ_o21ethzvGv1^N>Iw-hIe+h#o@D`;HZpX8DQQ5OzY34-G-h3ieLXySj<-I zkCZN7YP>I@yw8ZDtBgp`2k!^%4oNwy`p(=^cWxYUYw`I5zZs&ekwP~ck6%n>`~3!Y z=6)up8aQq%fCZ){L95`A3(Ew~eRULT1t2TMl!u z5KRL@g>h;RX%~9*d#|phMY_s0)lypfp_VPbJ=w`ddoE5A|g`DGjyoQ9kxPO-&3OOV~^uoh|?K zvTc(-Dr}+Q9`8RYng70i0ul5f&{C`o7p#Mi?^?#Cs0PM`a0bcuOYwuq| zYuyAV1F#V*nMbR0P!r9-E&_mtvf9Cw@p9&@|Hp$GnK{B1%G1)oi^l>%QV<9STmDcF zp5DJUN{U%`(o}Dq1>DsS*>z8%y^F9+3+sZ`*uVk~iF8t!t^ zcZ_ruk;_n+)`R@ z%Ea5#-cxm{d|qQ$W$v^M-~HM#myI^?h&4fo0- zC~f0e5RR-ik+IJ@3qL{?pgdlRZD?KXoZSTzO-UCYfe3XInRb?k=Qa@+zYMyG7*b9T zh%4qkj6ygRq^;093)~e#7R*TQWECB`(R3l;xR(C-d z<||3a2ELIKRdU~qRViU%)-1)b9wyS_72cb1&!NFv>+q51)M;y}Pv|PhBcOJ-b)5he z(hx3hqlR4j8$ir@raP6cP|TC=+?RRRNw?gH`;X`cgjH*i*Kq7W&7u-9dY0wJa zWiFHFU?>7=0$$58$bd#>2$YDf`}I054b#leGxDYRdSa3(%!4dIJVn{bk7*I&Y)CO^ z976^{A)o^Z7ARsq5|V{%(BfTfd$u${g}-YEB|cpr@uu|hYi*+R75J-dyZkpv$Co|K zFCAbtpribFef>(>W5m>z(QU!$?~tH044~4N&FcU_6Cj6U8=12S=&6yH+kOM6;tXXH zz!<0T_vpShf^LNE>Qf)js~fG|Q083%9im~}65y|btuk*E@mR&GgZ^?Cb35l5J0+B9 zcNM&#jQC3Z;|4X1&}qzw-UL*qJh|G+AeeHhO zO_dae1X2ziq`dX#W{Mx(T#^X*gFGRWy@(oCa(YfBWd89KdSvr1m{1SDGq`&;dqrn| zOUtQOG1OI?^HF5>6;RnafNDi^mWV=!r0_GOt%0f6bd>edP(xex#zVL{)}fC4?!ep$ zUL(5oS%`85Iw;1$Mq%D*{vUQiI2M3?EECSAef}HAAqE?xwe!)}qTOG&@Z_p;TUE zfxE&aX%>gI4g%V0f%3 zq(d)d$zE?xxoDZLgQVV{Mw7(F9UTj>t29w_-`UVkletbdi#@t{j#SDfD=6$ z_e_Y^331cZ%7BT!OY}y7JuM0Oo+ZG|iFSR3XesCWIpNB=$*)TsXuQAKb=j!T5r z#|tjI3eit)jVvTyh3jMngdlel&kzb7PI|n+Y)kesN)+O0Rxw(Hr+pHerd(0KBEZg% zo=!cfad8764|?7O^2hN_tuH9}7a-ONw17nEQ&9xv<)6`~K2VSJ4E(*^cr$wnuH zzNH=rd>qQ~tMzty*y+(G(1~gm%z%UDA-W0IOF`$jR~Z0b&^lZ~MVqB-k88Edg9BTr zn8o|ctRc1%Re-V@%MQ>02I@1WhtE8Ie z&5*ULA zk<8DO+uQbgQK}p-|0T@*Ti!Zv0syJ4Ku)+I;xd;7gz79LgH6f$sz0$qG?ZB_$=7Hy z5vm+SfJ*c>zq@Tvy?}JLjy>`#&WhYTK=QxeK@MzBxQVriQ`Cuy9BTb^-1JEb*KiSW zr-LZ*YaDLEYO@8mL9xit0)4(^YEV@aR3ek5GbrB)FegoAKnCdj#q@iqdvH`=Io#V` z(o@Aqxi`uox`CUV!_L1l5#l*iv{KmZjXEE7 z*fDuq&i~Z>42b?lL|8V8Cb4l288`xhXU{v5m2kZ>vVj2DhZ0PC%fTv-L=28+3?FdAYQpGiFG|?Mc~Wq<)O%Vy=Cw9=?X-%Lx2Cb zXBgJ)4H%+^*qzoC`m0q>RLI|eClPKC-Hi`B6xvL}s3+v9lbb~7*vPGD@<2Ip^!>zZHyaoiny}!3YnBs|U187et!TnzDO(?DK+Lj!pBZ7PZ^5v5 zi5+_Fc}$8>zduc;1sTi+$Wmx?es#;BOADUV_8*`%9p1+x>0V!_JhfzfJYV`?QhX4n z<%{ne#w!JeTT$`$(K&LMAJoUCz=6IOqZzToLqy7tuWC7brS4XY?X2J40JJ2zt`_SW zdKA5Q6&80%1R|bXenYKq0vQ)j_#bYZMZEpOU?7j#wP1a^VdFYWv|%$eO@ujI#3qx1YxV1fH&bF6*2UngC+>Oe6pR literal 0 HcmV?d00001 diff --git a/docs/images/async_compac_2.png b/docs/images/async_compac_2.png new file mode 100644 index 0000000000000000000000000000000000000000..368628de9f38956a034dab4cf2279599fdd7a687 GIT binary patch literal 54164 zcmb?@Wmr}1+U+Y+(nv{zl(a}oN{FbGAV`BCA>GXqBveX}P&!mfO1hN}>2B!-NJ{6q z(f8fk{e9N|7~>v~FO?PLaIq+`{`%`LT=_dP5B~b=;+MbvLZQM$ zfxnbUD0BSv*R#LmWp1gtj{c~|P*&HhC^>nUkhR{Q=}>caA1Tz=Y8l6H+n4dBukp*f zox(3KT=%*9>T1bb6&?c_0@+ALI{DAWjCgk#=u%ocj|G41Qh!`cTzgm0Y+aqFwKM2F zq<5s*rFFwAx4O%H=u`qh??)hoN$+=Te2Sm`?AMP3SbV?#NNjfg3W3zM?%(fGPQLW( zg%nAm-!J$q5dZ#!ln?Xox1cEe`UECg?EmUg@hJbhi~hGusY?FuF8W{Z&jIWAC;ok> z|Lvmx_5Ktu{GUGbKbPuYrTag9=zs1{810{*{_9HqbJ72{e*gEIa=PL z_}J0W@%W$%;r;wd9Z50)I)b;iCu&}Ve_g8BI1Dm5cwH1@Z zCx(aj9|t!y$ZSW1httn-ypD{VTUc;j9WD0p@u{j3b#ZaQaj>;j?x*AA>=+w6YIsTH z<>j@rvs3A^emgNUGtpaiPB#P(&9vlX0xGKG1W$w3f`Wnv4<2+YyK-`|a&zP3;_~0PAtiZ76o+OgQP=_9 zB+Q$>IWaMDYikRMlG1HIko7PtD+?DFx6ZP@s>ge4t~E~3W^HxV|2DVgXBjCel<@FN z6p}R^-@Yw$CRkWn;uIpbkv*?S*bRoe62+t`aW1fh-Q&`$dU1b)g&tS$)P4s%ge(@P-2sD1;Yy0!gGs=h_JA*m{y@m35$y22$PbL1wMX-O2I%v zb4kD4{>f;Oxvd>vy+%K_cRcLY+qc&k7@qGd+p3^gSy?eNGhZM5_>ozrp6O0nWo2b? zF-eE@c$triq;F+Kh2Jx?iy_gGk>h1{45L5BO7ZvpijI!Hb?a7HR(iT=do-uGxcG*L zXPuYf!4=;(Z{FnPaXV7o`aYh~Gc**KCY=n6O~&!ac4d$=#pU(u*ZHoSk1-Fwe0g@= zk2Y59?^tGDPnFEmF3qi}(TjWvAy?)0^ZO^;CzwV4{%D~_`VZ99Q1iSVCd(#oZZGw* zv$6R#SDR&~rlw|OTp=X1o32CC`H-2(dJ{V~F0SWGM+ffV(NlNW(0C!c$68vW`?AEV ztE+eK-VF^6<>cffKvsL~eSH5OVxq2R`mmy+;@!J<$R~Xt6?`S@-(zE81&4$fwhDE3 zcbB=WCk6!tZJ@eWNn&r>so%fP!OCi0h5RF%PjVzzmX-zv1}=qpTReSgkox)a=O<5| z6c-m0AisY%z+4gJCckp!OR@|KrzeT)!(=aB^n&Cm4!)WWN!zH)^?^BYKzYAvI=O;nQfyBwkz{DV76SN-3JP0_Qtnu=I z@UHVUDqDi`JN;2iOzgRlr0;Y$L_8WATAd9lk1UjsZXFK~4<{$5?I)(Di>s^D5Q^XY z=dV^euS$_2d)nHtC0T78e6Cm5m{EF)%P%_#Tgrj$-LiQc_lW>{;_Y zOOVcl+>eWm?WUun`!X^@o)vQc`}WJ1FR$t<-n%CqFm+&f^^ek+=IG=^-xbZ1DDH*D z;HQf9G^nejM4(5L5Ne360w%lKL$#ODzRzYjYOt0s>E6CS-CB?#=PCOfYR{&5@&hC` z58=T`p=n*ARlM(u`Y1w-^o^GmDRrrZEhUA8s*f_q#RKw%Pzh~G*uwGg{Lsx_Uckat zX5|YQM6k%C%HgGQ{n-ohF#d^cDA9kn3nuPFhHa2V_Kv-DP z8RS9@P;_;5tt>4Kdh5m>*m}-1T^$vBI#l^1f>sES#~>_8M{A2<#q^eYHITH zL-oYN!y_dnjgE;)`&?e`c5=9bfk!3zWo>OOIhoGE;U`3Z;0_hEQ-Dn8E%DIl^OJT` zT-=bsLG^FaUS49TH+gtew!gR<8#7+NuI$=cl9EE-iN)z(D0Jn@6-i1e99-OKHp4f;rlGVA^H)LEoWp=aZY&@l~gflZU8$Se#asFH!U-It3K~g=`7knF~ z81s#$w{PF}_VzY6XQjZ*=Dx0ylamASWkx_r85bR`uC6{fJUlFM+uq*Z(^Hg6o5o`-;n3cO2OO^sOeO0MX}Wvbf4c8TfgXLlYI z9Uj2KNQYrqT`z_2e7gv3L9px%_1_sugtcv9;_CXywP$Z-K*x;lV`+BwN=N7Qf4|&S z*TsWh>nCCGKh2c?fH3Fi3P8`bztPbD92^0D`UmO&{P|<6{{&Y5EBy4IknI<=`#S>v zbk| z^z5Hog%`CnG)f_!ULK1Qlakh)ogV#z>LzJ?7ATNOB)R|ElRt0Z{{?UUvn=PI{bNc0 zgp2?G);do%Io#OU*%<=Q+S(ebFalAF?EmsbQdZXA)>-t#C^9;_q_i{$eRzC4{>_`8 z3DFqkwV zji&eRVY@G_tcdD_n3IFsoQmF(m7JWMm4#Kky}Nr=P^Wiv^i{NG z-gWP{|5{}yA84R2009vb2m@_!{khPQo14qZ%BpNEQ~)WIkT8)V9auww@uRS$1X5^! zcXdo!T3Rap`1shUIdu9{W#v&@6gz-vf>p?8-^Pz0KLRFWzjp2WUpK_WFWgs=mabo$ zs07sKyfzMv{Usf8sHvGkF~ zzTn~RE?_lUGu3NUT+9lM8ib_LPVqyai=YK~CFx7TK8 z88|sZGBL5S>xY(?mp28J{@(Ao1X3imi1R1iN&yiOO7YR*;YaG~=t@!H;jA1SeoWaJ z8Iq6M+uJSlQAYgxVe9VRxua9%svTF1Pcr}!#A1JvvKYy!ox*7T!vI4*btT!{W zNl&Wm+quq8MPux-+ljs%^^SQuk8yHzBshb*+d`_#!pscFwWY1? zD)}S+rW%8bPqjOS5{_;ekldNco?D`7Ir8{nQd3tq=hG*gBOGjOY{Vr#G72moz{|AU-B)vAO}q}bU%kRIY?X0w5v0-lnUkpXUEJz6ABV1_D}1+h^khD0LsOG==n5Qv9_gfwFpnQ;J(B_bla zclYjPG5s_8f9g!t3JUH^Kd-uHDPgZwGIElVXf4aum|knV-BcnrBcP_{WM#diE#|qu z5zZh7d>Lq#GhoXef5-zNb{3YvrOTYEYfDSzUPsQVszl6}wo_|xYihC%&XZOqby1YH#s6#&P6SDJ9b6_k|ZvXFR)X##y+-HA<9A@(BfIXq%`exvqqkt|Apq}=JfaP=N1=T938({0|ionBqgV&_Hv+7 zHbn9_Y}ff?r>X+*e61Ty=8j7yhf0<9L`_Z2$|}d~0ye8K&eF(Xd)?dL^Be8FTs_Kn3%$7B?5?MA1NuV18sqe z92^_~>C0aR9fI!~7gr6iyrd+3=u=8cT|H@X+`1KhcRl%ecs94TmXXM7?Cc;(JbQ#K zq4)mZqROELcXzr{lKR7k{sdvc!GPbP-b)9iv~%lKU7?`~&=V0B2FS)QC|LgzLOiAA z;>C+~=)JJtAWFP8D)K{Md$_o|yR%)p_NB_-golR*6a)x+iFpjfKYa-#p0i%GQ2^V} z-!20hC7~cd!cB+&sCy8sIJy!?0#*~!u72zH5EmGgP9rNc>oU4<;$d; zT1Do4AN%{&T6p3q5gCe#ikq98Ku!Rx11J3A2};Z8*chxd7EgOy+w~@30fIXgQvZT; z$RT-m_X>nMiQ_gmdOK59mW(LX}ybbEtVUJ+qKOhRA*$F#=nWNV>QiYcn zN_Nrd=_yoN5O!YiB#OGB2-DPVqevYCA%b2AbT>^C8n}&39uk)oC}svg2@qq5^~&wH z=Hx9bR_5l6WxQ1{{z z_?w6)OHTUqJO3l_ZzoSZ{&&9o2}eOe`~@=+mrz15AvB;1fP@tq5~6VTE+`?Sb9IE+r2F3b6q!32*SD^&u$M2(fnL>vDGtQ$1XC_lu4 z_Ix%oqxH9#NKcGhr)OZ0R#p88YuDP^+Kq$)cDTRaKQKUVudECTUSwQcTuYeYhn1l` zvd$UyAA>C|cOa9XO%{bqIXmxu(kx77h~d@`7`-hk3#HIMFc6p_#180oS2=Kmn3(2| z4}JnThTwut2@enFHSP3c3c3t?{6Iq^Fb^Lef3|OY+;g@m1Q@PgUeZk`tY-MUrl#h^ z#KiS=c8%QR2YPxmBLMpd^+9+2XH7|cKqfev3F0oXp1?};|E{i~u{byP<*pqTGp@XU z#OTP#rLYe`g@Ns*ln4dz%(AJf_CXz1)2nn=a3w^aIi5N@LeLj)T*l-%;NreC%}wF0+KhjeDo=MNubL0uUh76RF@ z2exkWC1c8vfPjD$ck=u9#;p-d7cN}5L!vJ`h7Vc!cO9!$^jTY8_B}e67Hh5$6GOPo zudj29UV{DpIa%!i+815hb@OXBkXLlwAYelKN`USd#jXK>^SsHp?=08VNzE}}7FAb! zL1%_W>FMPKvJ4@ubMMqt7$)FK7RC54tbR@|E?u3Sq?ha`*vLl2tXyyt6HG` zl-;qnzoE7dc!5A46#^4+ykC&7R}Q)u=vW}Zg6=UIq9Tb4dWp!*o2WjWZEacJ@;W-> z!Q{MtEZwhOy+TJvza#A$jv+I=R#=Ww`Q; zcS)FIZR?-&TPZ1D5Jw~>ziiGn^+Dy_CoK4U9y^|E=l zm)z}nMUK(?0*ls9TX2sn4sRor>XY7=}^7GpGe#_gt#^AE7 ztgNzL3ihxqbWKM`zHjI0eO-%>^6c?qO^vvYwvNtuCCtdk*kt2CF$B!neGh~K=zve3 zvfZ?T$lic8)M2t5q<^M2Gdt^FJY+iq!Y05G9P*?xDzdtz=Jin@)bDFiQBhNb013~b z!RZMf3AK5P?V8sQah-X7oz;#lc251pn-5vW)WvM`5X9pnk)opA2Qe<*7=~EerC)>s z0|L;SzxZ5RXt^@0jThpBzVGQ27BtO6fNSU7^4N}Z6sJa}|2~v0TR%M++zk@3R}r4Y zd$et@e(~_vf(+gG2}kXsXUE12?Rlj|^THZTW06y(S$B#u<@XL$TQ}G8e#!-GBDg1Y zLq47>0_V4Rt(!xFt6M7bjRJ zNS;1@3VOL;bC=ZP?;c4{v9U$H&6KZJR94bTcpu|)hW(ugz%^*N;i7k)vb(UY^D24A zr#A7#UwLbwQdk-Ibl(Ep4C>uPrHjpE)d6(Bb4WJ=x+Bb?++AIP%3+|R118bZ)=mP9 z?CN$G7&1WksHlOTi(Rr8pu6JW-~a>c@9hQtx(rCu%uMK%`WiIjGOP>!Puk30a@`aW zf%v7Ops28!64zOqn?Uq6W;rJ!Jw1&ilg3i&m{ z+RjE1D@m;{x!Hkz)~2iE5P(rRmFeFDYY7Y>rIE?W#Hgqbe79X)%aY5XX60+^T;=B0he@Bdb~f-udU`@C zDvo2+{Pc8;QP9!F5U0mb?e1Knuo?OM0rVxvVbHtG&CP>IIejxF+_y3H4Krpnq?((W z&bb{u{RKM%fry9*Gk{u`l5TG;b|y$~5m-=VXJ@CW(ys!afB*iybReE#ehZN`H4)Lp z?7{ACg2_0{|R82VLhXiaqkjkA@evEFeU(x^D`J8!q09 z;cArhZd0WGEsC8_8xeBJ{tw>G=puyL~d+s>`ftRGcxY< zU%^N)`jP^`7N|7{h=Qk=%~t#_5z$C^$& zo$(?+#>G+ATK?^8}rSAs}PQWAb~wcTvMn>T%+ zUqCnMj2AAn9Jv8{a9LSdTie}h=4xt@&}e{yd-6fSMZpcq$;fyl5fBvA4x9ZZBBBYK z54AKUHMm^P{bariYfCPqQoTi+UM6`{c;(oC(5ZR=a#) zLZm2!Iu_~))lKY8ko-UeV&zbGhmP@zO`^YBPZ5p*_jP z#SdK_u=uoI8>yvY8}2l=v|v(Tuiw7q((S0?(rwWX11T??EhKfF)srVtNl9VR5X>*0 z7lmKB;g=Uw@%7ASCQ~MSdgk(*%?R z?*Z4a(rwb_sB5R4Gl8XfomD1V3YkX|mOlKGn}qJU3y(F26byW(#O1x_JlXEd{}3M- z)M4?IJ(=KjPx0O=Ra!~CF&}YD>E)ONOCHlE=}~D5bK=mt+c86?I*WF?uE5hSmxGs| z@em!Mdiw4|sd?X|)ik4wy}C6N-!i3$R+&6_XGTK!Db~C3$|*&a{6d5Ehe4|JS|Ycy zHumXHG&B3dTvKbzcGhRH6?`K5F5~9Z+$&FBs3urC2wPemk}&k2rJ;yvUCzm-aTS>J z53a7-IM6N`pw@rCbx+21BR(>6+r5LtI^o?r%?A&*`bvF&H`lo-=I)dEM4IS(OfjC$ zUo_Sf{jB%7(4H){Zp{#Vw*F+(?$Bh~pcnPVBhn#tXiwakHe=&{$c>vd+Dor%kKcJ^ z5vXF?IYeKK-@l*FCZ3=`)|I#`Zs{~PFOAH-s$mqO8T+d5=VOKGG`n<1wI|F+!(2g_ z>OMD2ba=dXTgvxC4-PIqWqGa^!A;>zTsAj8Zp))~VqRjT@L9pQCAK|~zs7q`E!@m& zGg<2^Codz1=;4k__-nmWVaLMs2HkC{(wQ2P+(xm#usn-|+j$nsN>UC#@!PBBQulmP zDy*-*P(;!y9bWlryoPn>PPv(@ed*=Vw9g!0FJpNaWXF%UUCK3iJR&%_7beVCA%bM# zF5l+WXW1Nnb90YjM`gL!ay@b2r<2w@?~(l`m7GQgi2!Lqli?WVM|vlBXXmWv%Ty~k zM?x896Pu@QX66(r==+kJ zIifTfeux({u0eilreeby6`bKXbdaiv<~E+apCsAF9wN>?~e2wUCCD`6M9!mT5;!tn4Nt1Uk=A#^JS zb`@u+vsdkHw0C0$PyHH_t8q2=GMCBYnIMUg6@~;dqo)={Uq%hQW%8HRo1=J{YfEoC zV_8=INPIGF^>Y-X57T=wfI4cj>O{wo7G^_04r@T%b3S) zO!XfFcb#uz-{&(i)#0C9V-q$##ISW4zhcI!(Cjx%$sMULe4&&!!9O>3#GAPAtC~k> zYQGHo$w4P2DWX{VLP-cEI#Z@RWzhm1^T#I%%fTLtgN6|gjhT(^swlstytEypRvh(l zc9bMYyg*R%#X*C_6n-AkHlt(9Ca0oz=;0BxcSBi>U%UfI=+*+7`}@(O;|s;)s*}8f zZ@r#|eJ<^|@qz($&-?vp&YPP-3!XSOleMygQ?gVP7}X+k+zSFl0Sw3UG&dYF|=r0IK4pnBD-6q*X?=LBh_pC zG8>A1TyqcI4;Eh!{iVeE$a3^aT6LVU_}T0kMMj6rb6uJUtOz390aGcSJns)Z4}a`F z=J;7@x=lcI6I+gq)TAn7Gaxi>rPIIsaF{bmU}wnjJyRAG>`t+2pjX*Jvk)M#4 zz{zZNKEK!uol>u0gGgl7pTKO~KBHAZmYZN)wopPU-J|LQ{fQlmQ$^5(UuLdSX;4_V z^g;_N;r7tCFWt7m2wzRrDeMt8N(U*AgE}RhZ>0J+i_{;tniAOI2;Y!d-3T2J41F5A z{O#S%X_tkHp8{if&-^jmAxH#wf1e)D2;z;a=~ak?ZG{D~qbfghrZ>*M3W!=JVB$CJ#J z^MS{6jSJpYkW74P(uNeq6rC8V9HFJKYJ;z>1s_eLmFkrv`^sGZnC`W1(C#9vXhUU$!nDtf5#>#Tfq_x#?UGReJF z+!B&?JLb+^Tk$IHw3W2@Y7qoS9G)>(tuHn-icidxecsOfkT-kRtRlB9=B<1>ZqO_1 zIBe3;Z#?*8I*iHt;w9~3Ink*iK~6tklZa|RT{10TDGEco#+CSUv;4&3`w3nX_7e?H z!(?$)qaC$zS!21psnzPItyL1&a$@}u#K+R(m_wiB3(VckrnbAMHqKvOPH*-WtJF5Z{`P~4k??&re*V25pM)WIUKHQLOUonIl7r6 zGn#cnl>m1WDj8;V{SlOyw983X7}$DTSC=!PsPoVx@#+hB9{pwIzqB^7J(O|_K}hZ} zZ&HIQuSFS*?l_r$MVH}pIR4xC0-Mh!)~ZXH{M!(wXYqFQw?pzzxh_Kje_;Z3j0Yd- z5>xp;XBwrbK>~NiKnRczbVLd)Rg(A2S07^DTp+)z8)eC>ymqj64^hstCboP-aIYlh zG;699B-r0m?kYQ!XcVG^ z;pIVJGb6cAjOW)hi@mIb#u}vPmCiJiMy8UUR{i4h<$0oGu?tEy98)Izxpoa^7o#@C zun7GK6TF2}?Rrj`9p3Yuu4LmjX?QPk;5FedipUCy2xLzUVhSUa;;XY*niWQ;WN5NQ z$v%H%8rw`iuBAwv7akJZUh-1^)6sdbh(__Du)BCQoVPu%cup9<=J*2B-`>%434d;i zzMEtH&Tp&eKjcmG+#PX#6>xr{#rmI*@qa(Qa`La2&T-_qxI}D5`pXOY+olS?BTKW@ zSjLE(wYSBc7u9}wPLcYrC2K+Tnqv~8NxB{?SA*Y^nbiBJf1&XTf`zDvIO0H&VphqE zWns_cY8G+D(Gsq=Bj)+s0_|{0Czj@B|NCJq8>08W?5)H|{a2DxHC|(5V;-RKgX(cd zQ*{WJAk%srmCjKcm4Xa$?zN?p{I$UK*@ac25%x6SquyvQWQW&rqHQ&Cjs98y#43) zwy5{9D-Zx(UFs&i2M>Iw>BM_)XMroKMg5A6rsg}dUoHZ8YE5+ZYG51s{jLsdzicYo z+;@_v(<}x*`B;1eqoj#RU3zs@6)Rg|ihsB`U0{zDf6*){g z`pA52KFwjd)y`e(w+?%zK*ETP_{r>BQO zzD9z$7ex#oj7_S-pMh`$l3zG_giHVu<>f73o+1!6SlQXny0b0F{n{E(RbVIvcNYv+({Us8>!=MnWw!K; z;!I3v`bprKmOcxGoo^ZA>~7p7b-B8YHS^6+mKf3Ic&E~G{H>vc+Wd}Evmfr_mZT2SMA6-G?%YCcsa?5Gdj}U|&Xpid8SZva;g;wsSarJNWazy1LA)(~tX4AAo3gv7XfK65|HH}F(f zb^x-xUxCcr#XleHRyCsFn(vnAG1Kq9OKiirwl3)p1cyXyfLwX@P{ONMrtY0p8?=AQ`B{;Y6aebP zEN6n@tc_r9Fp?d2ls$tRaUDfOr*Q7 zk5^<%J+L6 zOgr~2jE(0aSykz#W_3wN*}wtQV9SY0aq50{>h;1Kh74MDjhk}0DJcc!eazt*d+UYA z&_pe9NL{r!bL0y$6-zGd#NcmhJ+Z0kmT}mf;9a4(@gtbuB(qg<(@Bfm=}z@(_#3)z zAem&Hr)I?sejB>4$nY{!I#_5=BrKkzy_kwvbp`4ypL8~8cXCz8x=%WA?Xlo3?UWxm z!iqre@7#79c>f+79d~J$U|afh>qwxzOq;9cLA*DM)HxnP>5Sh>1cbx*@dMvUHZ|7H zlJoLb-|K_t*QDR_`Obby=#g>>VnRMa&c_u2-^}&=aN&9LqX%g?;gc8ig;V0k({OH` zy<8tZXWgwL^xXtHcZ@-Kb#mgT>i_l)Clvc)0K0TDOt62MgL+}&^-Kuc2Lu{h*qfhM zl;v*;He>T?fn)jBBRE8mk`EDpB4R8u)cX1j9t z7EZoOT7CSs#SJ7Mw2oIennt-IWOvyrkAncD*Sad7G^upB@rwI(HYN2uH^jz|IBFMj zcDti+_(QNvpxYq|@Mwc0KheEv<6-S(y!zY6BYl$+)f)2y#pKt`kvrqu5*sdrh}hYF zjRm_Mc3j|S0EEIPG1gD^KtkwgE+XwZ7?v?|o^d<*i7`kQS^U+LPrBs#DKbq9@N^vS z$wlh5qI3jXFlO}+P*n$OnjZWX^U7+8Aajc-%Zosdn(L+|=#8z>sU(eEEAt)k_*k34%9 zYsIR(wr{Pcj?hZGHD}wdUewOub=++;#Fg1~AU(6+^3WT+N7=h-*af`$)q^+cK}w#k z19K0xSFhRblxPOz8Y!9KtGSBTSQI$#sO<})YNuN|kz9V-*Ic@m`NfN9bN{Y|X44XV3JN15Bg(h;AFHSkR+WG=gFvpu!p+T1QL!_v zb{>Ruf<72tY^EotATehf=b@(&Jdycg2mf4L_V(SrXgH$IlRTXHD?nFO^*NQF@@s>B zOk6*vo{WY}7;V6$BwiM!uE_!1`Lc6!n+o!U-hcjl`Er=i+_Q@*7piD?>W~De zt37|=g?Rdf$oYA`rtA%HI73R%o^zKHEJryXY^pu?G82y)?1s%X@@hKTrRO#F8~5&9 z=R2IEc@o+3nmYJwcX=AQsWNDVqoe|v-=^RFza)-g(^W${F5DrB%!d5 zP_jC)>EPa0V1RU#Uf5#+gQ^g6f+y>1k4r(pjJA|Z3FZps`srF2=kqW8`v!pqPL??< z5@ZTnUMI>>;$;`ZOEJa=u16pO* zmYlZpTqRwd%kBz+x7RhTkzgg6XIv0cSP3O$WsW4}xQMB)oo+&^3WrZ}(^?I*EB(|< z^c+$t=t&byYb~UP;on+@l`h&z(0y{c-JVBF@6fzxWJG9wbFQq?xM;+{pT%W5`F9Wf zbsh|*qt5I1S+VUrx#S}-*^HOJiHv0YQ5GywwaQ#33?9x&&PwfD>K`fH6oa4Dl*6uEB-FrkF?Q5FOF=g0c8fTG9%AUK{(xrv`t{jf1tQuJD^py5@v$8Dt$t(%_|yWfSKT+M#qz zUY$n^BVA3FcaeMm|@8SU?4BQM-C<}|7^(I7~GIPy`S=O$^gnN5{hWp;xA z*(K0k#&nnx-;Z83P#e~2-U*^oxUHm#?BMTv=fUd8DSBh6$|tYyF8>It5Oxo3ZMnVE z=a7OX4g{HON(|cXD2_)UWcrFGfv8Rxl{-^Tb{f5lj0r6zH;}gVAq~&lZ}-R*gWxQEGC!1?80}`YjS+;*7Vb;oACiBuTixg3 z7?bKVdS8&2M}o!9W7Z!t(big5>E%(D`1kCD_xF;Y_)R}=k z1RC=*g+t*W%AADa9_Nuvx~PonP8oo@+Cae38*EQbe7U$oD?`3I#&}V9awfm7-rn4K z`><|oq2xApN(2u}5q(3n`%gcsrCsYtyqz~;;vfCs{g&c>&$RiX`IPJPM zPS@T!Dd4YelH3%T?wfkM^39(o$om~ep=g1&ThrWLQB=+NJU!a<$5vznS<~NoN72|A{dFZdofS1TODPxm z4q^0C=Pi7&$7K>^czmu{q)47&g1KEqRn@}Ux=B>hpS~MRG^y3oU_=qHndG3S?-?F8 z13wB2%U%LG<@wQ`mOU^zgPDnbyxUA(UjA0{*^QE^abS926ZHOlxMxn6z)XmzXkgIR zXDh4q^*z_PzJNvx<8%)FeL11j;;+X{KTng>jz1JMS!RAn0+G;HohyDP{PXtvdx8D4 zic424zCDPHbrhGuy<6eh*?e#DYlCdyBsJvfq@y zl|JY3M}o%{HiaCh6Xx|-8A3EQ@GK%5v?beWDa@1Mheyy7;FMufXg(%GL;wD3j9-7OP*R$QN= z;Iz;gyIZsUxr>LC@vlpRYHyiYcgg2mbB@)5dFCU&f|d>Xd(Kvte3?AvR=9Yz23k$% zcZb^J?~k7K&evNkcI*@ET~-O~pKoAE7TNYK?^TaYM6_GZMd1o|vtGVFka*E6a9e{} zE2RKNPVk+N*zYL%@r2KnUOsFeF0s4n(^T}mFe~e>w)R*)9&rqjYue<==pAv;9%s?zb7rhvw6^Z-f+&VLgP82FJ0C z#(euvjDEVlLkknps;4;9+I)+%oOtXy_sOI`sdAC8jWP0?9v)$M%E|6->SB{DwX&0* z@_438Z7{#%7C+E;m?(cYq%a3+XJzTlT)&A?a?UE+Onlk|iJxg6;fKn4!#QCSTL3e& z_04a_aU}{&yNFCMwXbVimFQpiI*O46!9tLI(6dH4-&M3_)A7gv9N1&pkO8x1h4y}}x0h^$)OBroh+(aYLWfk(DK-|qTfdXMP)AxN0Y+Q@E5Ot+*WkL&P+Z z`IA{2wDez?0}3>)h_mI+@&MQqU+Z35uW(vHK|z57xye;v3;-)9P>rRfrPoV9>O8@} zx4E|`0s9N51a`q6074s}Sg=%>xtN%k0Ja5>BG_)|2J1>XY%MNbdI|OpTCdr2Pvq7h z_BKE&GUZE5w#qsD&6iTkJ7lR6$*MHbvxatC4qTsI z4koG*Ql8dKGASLB8`VO1XXM34*A*=hAzWaH*>tH9XX7>22wM;~JpyDZ%+YINMr|H2 z9S4{b!nHBX0h8D9o&qv`I3Mt0qq7&aok4`e*YiCg(Ov4#vH~#aZXi7La6NjOQBvJ1 z)zBKmPvSkgpK8Jo+}G~T`dRil%)o44Q#^_EZX6ynA&)%SW!3HXc*Ny%#H`ybQJ{U` zntVBuY?d>a*CXO-5bCaF;4#AS@TM%syB`;EGyfGPM@7jQ&tQ< zko9Y-*w19rk?hYK_-i;sGf-`B>IStjTHGnm-{g%a2?>{8nkZOP)?e~R>Eo<6Wo27E z;PSnTWuuaLx5PVk_EFr^Ami9Vs_r4lau)ksiWQ0wyJyO2wXF9`DZPB|rIHZSVg=Dr zqj3$i3bYU7*(E>G7E)X5~5cE|i) z)gmrf(X>a()1J8j#oJG1MQ+n02MTFW8cofdHvD_!JM znEGJ*Zk-0UR*A82|A!F)u_8I%o?@vbZdxxQLy)%?IGRIiy-ucwC!IqvPZgGs7hT60 z0WP05gd`nYCHXz2IQPwCZFgzY6{qiIy*2AB;nhMKmN&FN)qIw&w&3)p<_7+wbtr|v z{mb74xoL9q_a&^Dmoq}mbc)!aN4NnPjW|uu=9#2njOV{j#GUu-bJ5JZgK4wsv=;o3(y1?D?@Q!DR!MKUdT}g6dKspO9-7=cPPd=q z3yg4*dEa>9*+2hrp~>>fVR6pPpVYR>3PntL+O0+Z^ryt>y2%jgQsF45ZPLWpmejiM8Z8y43BEtT2k*z*dHmrg!;#K@k+_`|H7@r{C`Tn_wXV}&Xe zpc#3lr$(K6?&sb~26{0{!CBND7x*mALtxe+$nt3bgViWCKudltwyFWT1^<$THqE3icB0UOxQ=Mz7s~nE=6uF!3Xd=K@{kAZkzPiX2N>Ort~-e5&VP^rdrVTA6biP#l2$1xqup#9X453-e=Q4@vVs!SrtEU+}!+3ZQIy6G*C zj~NF@(*;Jj1s^22f$|+~*&MDvhVthvwP;`K&ik+(S^rjlUC{w(^rq=HR5&Nn z3h(tbk8>{ZcMAUUcH2Ds|1=f-uTxSp(tnRl|D3R%pLVL^ML9_|nUySGRQ@ouzZVg# z=25G45ag~gBk8}`bx2VM3exFTk01wLt>}4dx5804FwcVc!2KpE5YE${E(q41o725j zP$nM!aGuQ=q_J57O8GU;7~p|;xm-Ci)J^}LoN0;$IEIWa+&vO#{V{dbgF~5hv&!3} zradUuCgC$@*Ma@;=xff;fK~x7U9488uLIl%80QMb8>w(|4Mr_1=SPHI_p`HG6dc{G zzXbqjwn^w;;<6)R^myNNq;KoE-t5BJxvKx%2i5g1^_G5uqaLlaBy%yoCD2mBWtK42 zTjIUp^cGnWpFkat{@p1SKK=FsSB@!hw^k)Zaqu&q?cZ>!{ts*P2e4ngs59QsHOuK*}lbZ)x?!wVN+Uv$QsQ1yi8gtU2%V+ z@%i%Wp}?X0-)qe9<<_s+lIMjrx_sjfGzSg{R-9)zUH+d4qel6(!*{$BR*FWv{b2&} zdIS0r<8wkW`C^4?V>8pD?ooqNy`&zGi{Q)Cu1ACXpU!&`V1jMpZJf$e$@kU%xPGD1 zbSIkoSVqYSJ)(iL0j4^P!Cbb5cK?I1w}7g8?Y@OK-HmigD@dmZD2g-)(jgs^(jeUi zh|-M`f}o^;bSg+~Dd`fCQc^nZ+Ir6WkNb^rzcbD_!?WG-t7koH%{Av-{xw>_hNSU@ z{dDHjFWA5YEldC(n-&uve73)=c2D}pH;84g_qh2}4qw84$t~nqQrt!4 zCjI@%Y+X9PeZLziUoXv@?aZ`_AU-4Xuq20R}5@@>Yhtyzt?DA5aq@bzW*>_4QGt%QK6o5On{xnT0WI{yWqIzVa zh5TS+Flcgz(AFp;2QCwmhrZ6#c}`v^|?F*_$&dKIZqD&yV)cuh~vf zEA1DcC*)k3!1-iMmBc+_{{gX>0ZD#f8|GiVv3sM)8>Hvz92`VFL+JiET<;7 zbv|CHl0G5bxW+N+uJvHd*iCC-g-!PTiFAYXN9i3+a+!^g9mSU2h1S3mQ^8!%^noBO zOxdPuw1cPXr$M_CV$tL;HZ6*eZ>d)6RnKJKK?{G@Niz%y(=~`Q?Jii9-}CwueUU4hyPrh>Q2}AbmKcoV&(nRCO3t((d^U9 z@2wj{WaD<9$IJ%pwaOCxJghen%d~HMrPTavD`PkLS`TY5l^j71htdo7qB{twAM+(L zCT9ftyWWhM7X)^8-(AL*e@SaX~nJsX#_sDNFWgv1taKKA9z0BcT zu?4l^Z2L@AZNMkpX35H8amt=M+EGE~;|Q#Cz7Z?lGM zmX-%3*iuQHm+`fjXD7aWNi2_#41~X&EL#d^WgahGL7z5su0N`MI@M6sKJDJ#^NK3m zgn6T(bfbvz8lUX&&{W9Mu=>NJUdDZ_9~0#jLr?Ey|32LNX=&t#NiZK{kKVWIQ>_?2|dGg z>f*+im2;1?quH;04p;MjY#??iCjB(C^Rt9<#I~51y=E!)%+^izQUuPe<#y5#`-aEL&4AUPiMnv$qfunB{4_ zTv1y3sr>q{D4dMR=_oh1FdNxAss7V0L@<3CVWZTqiuPavoTKM4xx6er8oh6ol1Y9) zYZ5II+IvmcRdmmva^ql51?MA*W2NjTJd`?+}P505pohjE4;-}s^B zRPkJKO~U2rkD%;}eYSOLt$|+|aXv8lJdiz3`^J(Y^S!mI_%_b-q4Nbh{3vJncTe-H zX%w*s`(l2+h(1JnmwQjdHmLat&e83ul+BKyf=^qmHET?f)pu(}ESRdN-08BIg`!Nv z)d~9pC7zhp>gip^_r5C4GV}1WMhz;S{PP&+eI-K52`}59&VR7oT8bc-RYpq-T5xh6 z2t3h!Ol^I(bxOY!aUrNH&TRak{m$q{nom+$?a|h&$bsV~iN&q#t*ms_CPq0= zxTi&?8f4!sk?<25oaENrmp-$Mc{2^m*P@W zyRY=0t&cxbV7RpNLl@7et-W3H+dz@P`%9NDy?Rso_VTS4sGr4zqbfQ?6Vb=(1stU{ z0=l>aRJ;-(id%mO3U{z&Aq9d~YD?8ijT91gEGa$#e9bEc6@Zj=Xr+y>&Ci?L+9t=x zYa1H+9q;`FhTC&K5s_gAeN)pckdxlIv-RT!d!W|9$%%hiiX~tQ;!YDDi(O=TdU{}u z1)d)$wX`$`J#8$rY?JSf4GvD!H%|)}IXT2gZZ#;b0R72V1~W0>*Sxe!LW9P1)i16$ z9;?GeU_t@^q^G&E2uQCVct3zJOs81)^tk!_>fAAq^+X*;%fz7t38&B^!n+KV-UT6*UwL?-g%~BWqJ9a6^A*_*?IOxOWYN1 zyx(qjf$o!WdZ|=L{a{NuYlR; z9WE-21QOewhd@CD&myzSIW8mqL` zxRjlg^zlPkD5op_rS=zyGMl27?7QR`Xx?IlA4H8RXL9DjC&>Ec4N(sw<^Q@gB*9*u zo>;DE%C)rUd)wRiL_`sq;FLomM?nuNZ6HZZH9izq6F?B*hXq&#XQ7RPyb&mmh}^I5 zOpZ}brocWsczCe+N0^n2h7`2?_>@L4G?zui?3$?P2=Wb@9l89C8(qF2+y}0A{mr;RnRPxAECVE(_RAYwvFa<8IPVvyx#$= zccrXV#b!ybLy5!%-l5rP6}|xWjkV#Mgw#~~=`idZB`Z%shf&@%c?x0%34sR-H+9(n z2iETh4VNmL27DO$0O^w>Nr{Nq2Ya4CxHPUM9r|Av1KS9UB2oU3iv69PSbk#OMwuBI zGXeW1KH?yrg+~VZMdgU9s3;u8`{1tQkCDkx3ktnu(kT%Hkx0;HD-M`GfQJIh2B^Xb z<>+y^KhfnV_4f3Ddnl6xDC6(TxgTzS7g$zd&d#X<1s+DpW!;kC6-i5FWqjL^YDpL1 zn}J3b?H*eOrC<6Yw=IHc zM};vPbbjqJ=}t*>cJ}PodqFX>py0{_-RU6yrM>>E8;H?Gdms5s&uE0u zyefQNAYd{UMx=vd65j1IOgRvCdM_i*k9JoU^E_Mj;o1RBk=Hv1D5G^=KkF<%5i>J0 zKmIvgzh7qF4w7K*OHxwcFhKumy3&@Uj9ACk*7jbL{~YA$R-MEwV9EFXTLo zyy&i#Va0z2vO%pjm%tr*3?!6p!!d5I5|Wa%(YBs~e!F&~BukWartC`O5SmeWXxdN| zE^7nCKboq7f~EQPsOsu!Xf%Lk8jA{#T%Y18_7x?OXGUO3MWzsbQ}VO)a6RfVh-(K^ z6jX?Tpb5k%U_emRMd(5(m(y>deB}uqF6pos5W2xMY5o^Pnw2xK0`oEZflKTMj!B;v zJ;3s7ZUZ`u5@cpEKS?EjPr+jYwqJ$hW&POo z+Q%&5nehvDCB$21v?_>_)4isaIPi)HNBigX|CD9_``7>Lkp4g4{6EfdnG+Wov?CrQY(JO>G0){W-EA=4Jid3n7WD7e z7^2I)e*GG_;m(jq)aNNs5H^A*GdMk)T>5))`k4L{B&dV>_mJV@{XKTLqAGvi^`Fc0 zBF}~x3HTNm4L}0%xxat!_wTpw-u3tNJlvdXg`FsHAr5HRVD;&@0{by=pO<1BPdGKe zMY7#dRyzSa>=K|COizCd7MS1)wdUP2GCb@$1E~*~5s8%;U9Scwfsch+-&TAxJy`#_ zhJo#>*d8{cc$8l5?nbq)FN=#WBUjefjSUR&6!zD?K9lAQ0)i>KvU+0{m;09#-aZvRf#iej^`Vp2W=w9E>cyZmq8_?1ZKKbN8eVtA z$@|LZ&Y*d^Oue=P=>wcEUe3MvxrW_c57He_{s#-ufJ;e7wtOU^{m7dU*e-w`3#P8Y zC{KYCl$AO?%QrC8o0^`!t`kRT)7R63v9tuHUH69zPo&DJyTaODj-d?Jvjoi1*KxVC z!8aM4e((8j-2%0GYb$6C+)Y3KXmI_h5r6rBhMxW*Nb~W3a2GPx_J01n3cl2kEh*)l zt%DIASTk^a2R931Mkx>2Y;!7~JtO)3vJFBwbcWaS3$nt^GJlFvPrRL-5jX5z4WMtOy?h3{sqokV^SX-Oad4yO1-I+)Z z<2h#-C{-YB&KV0jhzf$FjLz*N-0;vP?3})zGeB84uJd?=&j5XKFk{GJS3MlWmq$5O zP1bo{z7WX^dsE??57bSd6yQ#cZ%Y$-Jd5Pw!it^s{lQo^k|t`t5E_cUBmJmI3qC_6 zXY>a1b=sx*XW%GEtgo!7riR+=f$GBQ>1eJb;4~q^lHx%)Zo*Z_NFu2MR=5!;-?3zb z-Ky}zJ;!PTK4g-H?5PMyG>m@@7g2DeT{hOcQnZ_>7q(Xh=$3+-*^DRM_YD z%)<@1SGXkqL`8VCib1$?`x$x=+*q7zAz<1*23dVu0@$*#H-d2@-P!NbAfjH{U~H$C zbu}}71W&=+nJz)n`ZHy4Vg_~&*nL0}G4{^ns%xCKTwL?fNGYLQdb}Zq$U;kacsRtM zEO@|xDh%aKh>YZF{sr)YGXPDA0Ax0?kd0$vVg?@r*(q?Sw|dk zUeoE`Bsl&h;SZ0FwvD#D`~K2)6pt$S?B%a{ieUPq*9>Ni%n-dIFsKY8!VJW~@o{)6 z)h6akff_wBNgLyn2|E0JIhemeCD45s>Mn32N3$43K&k=`E+RNjnZz9R z;Sts+l=eR$Win~<|KMAI?*F=+Ud)k&D?jy8YXNh>E*W3GvPkNZ+z-nBw&0_9#l)Nu zuoPGSRE@T5P-#>CM9gg!!q$sgx9^{4&-Q8nUayKsY?(csp1o56tJmv#O9pWyc8R&g zvb2Ce+{s464zXv>8I&h^k-#y(%j7&$@6C?|3!YL1T&0M|`dmi2?ta^EKEE~zSjwL( zgtki`xdPRSosg7NT~*aCRB%vLVIJgkV-Zlrp{Mkxi;bei%DK5&y^^&L9fQnp8fihCIqqls{2l6-_YYxo~D*hCd z5BvJMBL*^E%nAMbbtmHHfMc=WXip%HmJ!=3Monx*T|h|>tW@nY$fA@wn0*2uH!k5o z*in&#nVppD9D+l07#NENW^uUw@$rLlzHvt&a@)m?9w_Q&rl&7X|C@H;<)nwVRKRL? z=c2Lb@9kypbWAbIT1upIWv9W%JV(x1yFw8Y%xPz5YG}*UZN~QVQ-!qd9gUoqQBj|M zGp_-Irqr*hJ%cP3r$~|q1u9fTs_cZ4X`8RwOKxPFa%kg5g_T*+OZx~&sH`c@*OW(C z8iZ@2-ER|%u(h^stv?alKnf^aKS_`i?X6tG{>~!}!vHeCN=Sr4Pc1bg_B)!jR)uxU z^|g|L%m3`HXAXKE9=qT&HWkFn&(9xM#dEh5eEnbxT)O2W`P|Sl@ny3ro z`v#SSm^9f{-^?+@XFgh0{EMA=vc>V=iC!jUus7|>#iz!g^hY)}bV6eJsqu#QEwl<* zFPuHn%SBc0e?W3sojDs#P2^y{3aAzdxgUb!WdJ?{w#88|p?-I>$c`#EH_tm6gRQ-#BrVjZ>q5-R_fxoXb!$DjS9yb!yyN0{I?2e0|*grsi&c9V!I4 z_>L86b8Xf%G)!4RO42hhP+C}+0tgKZKmp61z=e)SP$6TqCT#?UCc^0v#m+W9q@Kx? zc(Cy5))U=QQ;?u2Lit+g4I?KQ+D!W)%o~xAJR|b_+}-_h(YhxoQ#Obx3;gXkpElhe zc>{Yg82$@06gAw;wXapL4cM1+a4na)}Sdn}_1PgF-fd^p#nykHHR1mKqaL$R2 z`9V+#3Oa9(B7s61?FDv4U;7Zk11?85SbDw(9Sa%;cJSQh6~dp(%izOzGZk^=Aw=bF zJURJE;IDn>H83!s;8YFv1MO4DH#qRnW&q40Qot6>AFN=(S>KiN5aK!rT<{aXz%eQp zE+F>BAkimP!l0K} z#;Xd|Q&UrbpR~3Lp!|8750~g4FFa)t%J&6|beiZIcvr_AfyZ)RE{j#<3rW)TCfLtu z8X?}^PtVF~GH=J0$a04qAzG?^bT~?Wtdq_RQai8%WgyV1OkTqMubc36duIm>Itl!; z;4UkQW>cQL{N+Rs*8$ztTpL0CM$EeiB0BndTZL8ZlPAR`C181v@{;~8H|Pl7=V=3n zU|k32*#pt2rw1%m3El*PO&R2_0P*2Xd##PexGCChz@i5n&@L;Zpaq;dStj4A!*dTG z$%uAjP*m6AhQS(Ova{RQkAbfRmHOa# zdSaq=DmoxE;Dgk6JJ!@zN75cp$q_ics=M1U(Urhv?-=Ic;OK17Z`oe7hAM<7g_fE5 zSqMNHqWqu1!}%p1$}qZFJ!{S*gL3H2>acMZY5p4@dhkG_r+?xC5}?sUh`U_q@O(Lj z#>@6qVqyyP$ET#HA5Vma6R?7c5#k3vDV~d%#5wOBqxCkcEABVy;DL}5Lc2#kerqlI zqQiOj$4>R>U5BvNN=UOp9&aJ-*Y7Es?ZEqqkCy;0Z&PBuVYz>=W8p8(z$ye(@Cjy*ux35q_PVp02rHztr{I6f1*C zGDNbOghvX79tc6M7ajUcFe9ZSNY&;W6jb5(EMSXu>tfO@%JPb6ZJyC)0i#(>YIGjE zHM004#ZP`ZW!q3w*u{bgcZ{g` zdh2s218y(@VBN~mvyHKYOGwzkY%JxqPUG8qDPn17(%%X58oG$w&t2+>%B7c9CBdD2 z#T^n?cQHPu>OFgo>ukZViPG(_`vLr&b3Kgeig6cuk-^l(hY92ft+#LGyg{&P-^bC_oFyq(e z?rw+G5#|1}W=Xr!$znsvg%#@scYsY2G+Y5^trQ7OSuv)%WlU)-k*C?^JsVI1^{Ar@ zRdE8!o_}GL^U{qQH_9HsW{F82bj7f$s{u0&4b6}vL*OwPB6-sa?m<<;gZ88J^RbA- zrS@nW_K+7J9C@-Ww{RxAU2JWmTzool7fxHc} z=v6ckZedXDa;n={FG;(mj(bCN6~M`<1VS${+D)(_c<($dr4_sc$SuZY)d37_!W?u` z^J3k%2${XMWh9egNAE<9%RSC-kPB+za`^A`i6__}&P0i0A>-l?yJQ)jd2V8=6*Pj2 zT|^<_l%?K-2ZG~eu|#R|>YK?#J&bl#7XX``t#+dKF>mT8I$)A?V`sl@$@&P)Z7ZLq zf`eZ~q5tSsK)xWF${Tug zAN>p_!i7@y8S$oJ$!P}#ri)(qdJHz%b_CIcc?)@+a$1cn0MBH8N%L9Zs z2WcZbUIb4OH)WE00p+jdj`n}WxL#0Qhrx%!-uy5uMy)Ku*yRLU+Mru<(<}>PCAbVm z*Ie`248{}U;D{j7#8BnoBWh$l{GX# z`e%E_Mi$ITa{GDz-TS}H`oQJ@(01dq<-*DtuW?~k#9WgbaqR^q6~Zn$TUjgg z&}Ew*)9y{OAs3xR6B|+7M1!xeTBNK1no+OGwWqJ{^q!)Er!5-6&L+1y3=o5q+M>W$ z8%-Z^ti@7wo1Q0RN#4_k08@TOlMQp~t&zA?EIGF;V9q{+03C!~MbaZWRsb+tCRV-* ziQuOSJYB{BHK!4`tFp$Qam;x!zr*FTiG34}JUFxmu@8G*EzO#`F?KaY0WAB@&D)KBn zk#9|ZibdiM{x;djAY` z7b-lgINCK^R%)*V+eC8iypp0)_n@^6FI!myi^Q}Uagrtnnm-A9BU*5zWXfW1Wdln^ zW+tOq>r>nW$MFg*Wkgp>IjHLA*VhSE**Q1_v&SbULWNi&4&xXSKJwy<2sCDbnRGDX zHz=n7!hET^ya;>1U{aM5P1ip{YM3}Bh4b%SU5Gyv}scPuS5rYBecS;ZD*gLf! ziiwbBl)h#jh-ivCxW8Y9fe&>9MAj2WDt=0~DqQTHQ?FpgK6ds(=_WE+PKif23<4x; zVZne9VhWTiOH;5fUH9E#z99BApCJ@)jxp21_wn?XFVD#ZVz999^zqC&?yzHTDY>?J zWu&Qd<&CRICYUkJ{?!WX)G)MZwj5X{srD`Jq@hxnd+ zJWZ_E+&nxX0jX?!%sa1$Gh*>lD0l8zBA(w=HuEdMRJJkvzP@O?5PrM+qyRBd6)M%y z(P4(=S?zDN8dc{_ZANyPf*Va-^_o3nts@h+cxhq|sbP2sgPlwc6*>8h)B^4u9y$Nq zpG&@ovCeJl9#3X=n%J0|6^FXShL1;V_?wli_@6MuTZG1MV*9J%N42|W zbo##-Y0$#>v`NdQB$#d6*WWKjGN+{!HteH*v3e){j3cPhM>=_lv{-_{gli|n#NA*k zz1lU^Qiw$*W#pk%C}+A}Cd&0^k?J!e*+$<4_bYx z9DIG4<$XEZ>QuA1bmWvH&xPyk``y$m1J`3F)=QV^G^`(N-D(OM`25-GPtBopq^J=eVIUu!h^*+tSMrDY+OJ@H!adF`e1nPq6NS&}(iFo0GF z!qD#njP?WXe7ZmiH?Xd$rwc4~R8v!HFHJAN>`5jUr;GwyUL9TC7f}@DPEBMr=KauO z+4();qWgX85%j+M8vS;Be-EJB5f;6nP;aVgyhCQhEFzF*w3m`qz(_{g2HEX3I?B1Gn#!mo9v-A0eWH+wRrnP903is^a% zg+rZdDv^4J=fa8ZU#^uo?N@wyR7#Q4$GtTe(sr}t@YV0gz%;XWKQNe@J~NP)CpRv# zD)W$X@yzE-;{EoU?L1ZCu|94pWBq2J=nx8xNCe+Vb|92p;H>luGKh_tYhnPG*d3sT zAHbxQJkS`Xn}C!FP{O>ONf@~Vn21%OxIK&}1as(U$&GyW2arX6eiR)o0f{oS#$l=p zj7MuvO`&wd#=$Y#un2{Lasew<)4OO=Vq#SFFc|Z}kAB8%DQgr$bcLIH6Xr*h3{6Xc zMK30PD=qly61U>>zBg5$lV#FvT$j0G?M7+K+722wHwd^E7N+-w>RjjBAb(ory9;Ae zTc0buS}p8$xBS8TmghcYFefK4OrY9TdP-IU%1IIC%~yQk99jCRA6*?kp=_UoC}sd; zdj0Hv$=N{M^XFvL)caZ%Y?FWvOJbd(z|xD7?Cjy%4Ia}zFj^D>y-_+LF~Cj-TuE|j zv^jW+zyMNhy2nKa=epaV4qy|+PYM5e^#d1sHeKO2u6S|dbyYBuBXlR9bI2QjGs+vO zH|+tEfpHHjNI%J~q1M)rH2r&I9{3#RC%0-*T~V~E-OonKy;}gN;Jj`@8IwSrF9h}b zV@L~74Hk_I3A8I^-XN|g#o)%C0JpC7Ntb5p!8hXB&(UiHAGUHJh-h$v;=Atp0e&JA za8v?TReq@Skg)Yb%Cm7`TYVJ4>q|K#A2uzw0CRN4l>`ju8;g!l=|mqZRjrH4JUdW9 z+bz<54f#KzBS7J_!qBFxahinM2OX8baKhw(1%Y}PG=q^mhd0##g%a-sFjHW@;G8gl z3wRF4GsmM{7Yb{LY><3Ryb7(GH#)@`nVBe<>l(~+xkFiic6qT?Zj{wSg!37#uni2n z1t3mh9qj(6koN##!uIG@F7idIiizoi{OJMV=EaYw;kJ-i_+zZQlD9vyQZ4X2(6DPo zTA^lmK7s}a>tc?GioFkdtb472)(d8C_U#W>#zx95Z(qTwdk^K|E?cHuXoe_j$m4E? zHU)4t2MCCY-}Gtsi4yBrAbfcQ0kxatNv~oc8$zvDHB;)sP_$JE%<8=QGCKYyak5xs zydsToV+bk0Yb8nM`#(U!a zDz`BH;KmUEaAn9(V_X2Yz^dZ4t7?HZG^uYh>E85^+A==bVY2pRLpY7LAl~`9MwEwg zU@qQ&XWT3JOv_Ak63my%M4gdVn)LwuL4h+$&9ADe+SUp<3s7NLxXN!I25$p`&66rh z*yDqN1ga0X-LaEg=7_{1v@O5Q{QM9M=&S@-{UizU`d6`v=Uu22OYWm^O2}3e(3SxY zYf0$F*2N&^R846SA(4_dD@*fb3OjruNw_U=zGU-jeYUw7D<_q*@BA~&R1$HWldH8t z=3Eb^V`%pRdm!#O1Fzx2JH0tT0dRC3g!Lp)>2{Gj{6(rC#J5=}!2lygAXW1-=n7b| z{L$=Ye1By6oufO)B8WIF(LEw$%lLymzkj~4GvF*8o?=jF7V%*pZ!#06!%zthJWG2m z(-I9S`W#Kf*!cJtYQWGSp-1Bh4GUWlb%hYQ2;;(82Y@5hO6e5<6faV-@5oLBjxAaa z4^GERRX>WQunD!-Uq$+(IDwLvUKpA3aBm&&!ws7%MoKZqD2$S8az@;w6#9APkJE*$ z@fFIIi{4Ujw6ua?6T1_pI6kDJ3BNdP8=WDq*kL3Us*OO_{(wR`8$$9G5sk{*S+@i3=md74?f;5%ns zYThhbqiCrG+Iag~!PMnx7^9U`@rrFi^6?l7&9B}7+ju~MX}Xl#5hBuRE)_S;Y2%+i zzdjZp|6W`VM5pHMjXI&-qdk|Laezlgiy*n zr+NnNWq}`P_`M0ibEsubcD)X@$n(qlup?3r`LK1-EOBj&=J%1kVpf=mImGNIuj|Li5ZhT4Bn84#Ng(o(KlNMmz(!E zyy#ct>Kv036V0()5fR|17yM$L#+Xjbk)|;$%7Zx=W<;Zn$Jv5h-Gf1e5oU=*PRdbf zMw;xBMQT^P)EHdx&jLVQ2SVwI?AK$vZ;FoVn-2y!BBuIw$%wBQU6H?HG(mvoGFcttkU=@2uF~ge-OD^!_o6jav13=zV>_)>E$6S#s%1(g_Jk9d!(RE6Jx(0i+vc{| z^?a9-!}X`8jHE$SncAHaBbq0h@tXw>(R)MoRAIORzhJg0WHG(x zz9HYBi;V2>r-^nFGjrySDvEV?nr&iCvHWNq8!jy|5xjo9Cln&u5dMUZNP(yy~wcO+q6OOho*H^d0|>7adFbwpyyO zwd~GHs5m-un4w-ERjyO(Mwh=Ic{z&}b)W_3S$tICLtQSv5TAB@-iurWH%6H-B`;Hi z#;XKdbjx)KN{}v=SaJ6yAXF;@3o5(99$En`&ts9(Bo8xf1Y7M^4m^M7qriLldlx72IrOu@;G=^iZ*6&c?RE$^o^8sKoK>5CK` z=Zz2cQ49diFyw_xCYDJ!nS8fI-*m^|BS$%nu4)fht=ZPcs4SU;j7Fpo#aEiX9$v$( zG#})dlD+M3Wia&z1M_UT#QZ5R6A#;Dp2_N!ezBBa`NKBk;| zug~tI9eFCF8is#bZhy#gfAbjlO%!|YPfrM5`TOU8q9VNcKXL}t8&GfiheU{Ox1#a2 z^0jp$twxq?vD5Rn4i^zb4|J`6$do$maf(|9zw7TS?dNw?;1rh)4m@kPboF&XZ60kZ z@84e)ef&6#iDUeykM4HWt?GA=5FRt01G{0L?g!0yzro81GSpDd9B7)`D=8UInOa92 z`FeUzoUvimY>Xu7p}KBxG(jVu-RhRleLua)XrMX_TI57WK3c1Z*rV6c)@KiP?x<7C zV5~8-R=Zv)_+cDvxAO7hI8%9TZ>B8)g@gMHvzqC!bM5P+QH7hL=l-203JqVh4DI~n+>-qahg6)9ejpO@S zsiTRV9avRoS3dppbMlQ)@@3=XdN)lz;ceyVJj1M&Hu7~;y7cXP5%&8F87T%TZ+5oN z&6d*7ZV;{2-BM4vaMcA%rIk~w=3MYng~=5Rq(<(yEWHsEKT);aK}%JE3-5S%h*l&i z+28hGJ_zepqd0x5sy>c?m9|EKZS`0_Bn;!W4NIoyK=hb+8mufXi; z0k^so@1ImbGBE*D=uHBpB0^FF?N64hB(k>7E8GVo89)3IKd9{5n0KhZV4{83c(-LX zg+bKUSu;6s{ED;VkJ;UKib@U&oRzhgHk(}N$R~n4#*mMK?4o~6Q_q?oD$r{sCThF> zXk}g>lq=9xrIq8JAQ;E>3)YvygL}Q-WgDk=Sm8qkU_a`ATD^~XTlaUlO0^_N|mx`Nup&I4Urx8B~))>)Fz(hvHL{k}iy zcG=6m^Cx=Pl-OE-hml3FJu>V%HMfY(9tZCBwW|k=*%Wl|*G-~34=&c;3mUe?>^&-z zcQK?sP}`?|yC%hAJ<$4fkK?|M!T6S#SL-zPgVo^DChJ7<;lqgTc{a`qIFTablQBfU zMWn9y9ktE9!zgMv|G{|LyuAC0)#T`wTIM}EszV!kEvjr9VfQaVUw`)WU%%;=d?r|# z+)!INxqI;t5`u}4Tuup$=EIr&Z`CAMci+_SVlSCC-ZA<@**sqg7S9%ol*UA>Q(rfR zywKh@S+X6}qv2K4Umg-lo69_Vvp&UM&&b*s*;u=4=(%-Va@ro-c+c3co$JEX@1nOQ z!;~zj<$t8I3#nh*!zR{ik%;6b=IX=tf!nh;pJo_mCeNZXMZQz)iKe+ay%FzqK3zvL ziJZAO+wv9LA8AzGW4&3J)xCB3s~$C_C|kTk@f)q5Bp4S*Z9=~kJtS$|qK|v8Gg9=B z7@p`~w>^&oepChAPJ>)>z`=na0OJNu%;_E1z+JU5B;Ey{A~oSlS&7NArVaRdZ`H7s zDl_u0pf}1j$|-AF@kWTP@>bv&kg1^1PP?nUE#lkbe9nLMbpPpC zlWpwe@2^J*uHX7{MnuP{rF1BmtQ_8`+@p#QYo^yE zXP8+3wN=T2$1K>LcGwoB&igO}NtyM$P*E9P8n1IEV|y*~ysk0hHjwU*?w$Pj@$3Sz z<5#wxm4(WRZO;0+D-9dWQm#)st(r9^LWeJOtL-c0t%kOJ>KQeZ58A9b#osurpc|g! z|8uDg{OB&w54sm#S1tHh8zGjMYQVR&r>bM4yFT?jFxyjL$aXBJJb(Uq>G{w@K*@L+ zHXxoY_SHK)R!PfDb=6whawnPGKaiIXoR!?^`0#a4p!Zy}CN1r$r7e@!vVGBHyzmvZ zJm*p^gY|EK4G}%tzUtLqu^paQaPnm^?bX5}-$V4~6&uySv2wkpAJY?`If-(yyqUsx z>Fzts zllmn)7nb$|0w=Skves{xw1#H8`d(8HG|cP-z-p%Ook06!%aPybc}>eC{-+;Dtx7XB zNpqdCv(^B8gAd0PmiH>v@CS{(Tl)paydBS8%DT1jhVE=tyie3fYYhs_8K^qF4cC%t z&{Amis?EK+ z$V=M~X=FTZjQV~RKlpN-_F{1H?Ze#Hy_Z!~okuzKeo)Ij+!CX|G$@=tQBPfHG-WaA zLtp=x^UyRp*~CxBK>gi3tYR!o(Y~6mjhIK;gReLT)TRTE0dNBl*Ys?o>4 zSiz+H@I~xpTYUAj^0<0Hg+#6=v;X7E~z`_Qd@0Z0pu=Opo|neG^q5` zGpnT7gPHK7)g2qe6o0ayQVl#M9_s0LdR0h%E&Gf3cuyKuJZs(@20d5S9-YQH3k+#Z z6#5`o-rl~We)F!Xj_3O+JN65dRK8Q&>gsC6Wb)!^T7yFGrfW0f)A=u>4tj#hAY z#(I*^J(uG?#k_WUND`AKQe0SjQ&%L*hQGq#gO|$lrrN46ckkR0^qAVkxQ%KrevVPl z@^CzfD<692=aZy&w<|_rkf`)3hJyN_cNOZa|D%DJ*?m{{y?=^Zoa`Rb*G%@nuNi!BR{7e* z23NS%>BARAS@)!p)j`{_K^^*5oa&4c%daNoR1qEqQle&@9; zv#g|BU1BHgH128E(P7|wcnBva;y(ZRyMx&GrS8R#T1xgS$nv3mNrnM_Za9&=&BByc z%L0=ozudgMyUrpn-c4_=%35fdx!UO@&b=!4I5-&ZuB0Z&`At}r6_T`z`tPVP(hu8k zKyHoJPH6xok^l4f{*#UVdG7yavZ&keKXTZ=xAt$&{$GFo=i~o%YyVqfTgS1scP*R8 zJqhVl0YP%~60 z{O=Ws3~9>WgHj^iU+Ta%T3S_6(Pw*6sXeMwkP8MK2ns$Y^a$sW2TaY-kdD*cQAw%0 z*dm3vZ4u0bRj|bryFY)fkfEOR5WyPs<{PjYk|GpA8{Z@|@t&`n1T2R%dw3Bul$!0^y zdSdDsV<{s~8eo+fvKA3<@&0}HQk~gd`{yKD6p{zQCROG0Is@6YAzUflWo92%1^b?| zGIRxRzS$IJyt{!uuR)$l%J)NPC)D#0b)}~>01&65@&eEW01;sZ7*v}1GD()U1Nt20 zK-~mJ=&!NzZve2aTc|b%o=QE!A!eA{hmJo8pXg#M6I=nh2T{=abba!8zg8`9OQC51 z-4niVFc|*Vlw`n>hx*qxKm14-=;`+0fh6c3K)bpBbCMdAK60v3>I(>zNZ2E!vGH1< z#Ws*~X>DnZP+zkHY78mB$c9YiDR;4ja zYj&3-5T;$+-rAD%{*?hDmoHNDzMh$cRApqOq-8-cIKlHG3!ZGdIh>X#Cq;+{g}@gi zgFr;C?pwoKW5QR{ZgSq~odT7McCKVN#U0*p?$1(lMkyme9$O#>wRV|38xJO{lkGRa zt~sBU4@2|FltmC2n#&xF=Xfp}8X6~#cEkbY(CmPw7);(YWI2Z3T}m%0W^B5AtyM!q z1MUViSDLCS)85nILpS24h>)?X33O!23v*(=dkS@V9&TG(3VDGffp9i@5u{^;HQ_a- zr7rTk-&~Jk=Zu>JubEiV27*SWD_FHW2`zjrqS*fI1O4C%w&Z%!KPY2kR3say=)&QULx2;$RzCpvtLn6oShJBma`)u zh8!-XgKQ$@LkKJOfddG>X3tM&qhRxh@TjQD$ivm*Qi+C{yyzdmx8a(EX&zC8v^RRwm7cqG4Men@ z+;`?JZ+poG)NvTXs@F4^AMv&(1S7io_A@5nIH)Ts=EQVxJ(36(4*vy0CK~3l2Q9&9 z;ZlOKE;G_X^&dWvjbO6pq8(^$$cSv=6%jH@CFI%+Wc67Lq^S04b4HO)E~&F(xt05a zh+WvG7vW(z)W+L*jO=w>K3W;pFk017v3XZl{1BA+nnU9#d zJ2WytK@%FTzxCNiWmYXsZzHTeqMQ-TZsh$XM6dn6KMh zK~dGlBJ{yH3yUD~WcFE63>INKR_-tGv1)a=nDhlDJ9*l2@QjNly8R{_2JU)#Muzdm zsAzIKD2;6U(mI0)ZPO)VO?GkHv97CRlTW7fUTF2)m|tR(G_ERn(=sYe(12(NB-Ou5)Ffi(fJh?XD%y^JX-;Z;Rk zPdQ;RDG^X?4JP;oQ-3q=q!bO;Dl-fIoQDzU&pqcEh@T5{w_kyFFNSssWw*vBK{ja6 zr_00kQ;66Rl=NO7LrZAhoyF=i9CKh|0__q0m#*+nRZ(~#8Z9}3+ffSs-;=XvNwR9~ zFp~%CK!^a?-%dbO!Y1Q>3G_nMxKYmd>|zWSn^)Gk5T#1Q4ED@~szT=TV5~gV9HM?j z)XyAliidFzh8-0)K(w%5WO`*%QKyXDoC*<7bp}?;xFGt&sHcHAYW8*0!;Nc_8u_sUzcb<}X zXH1huWApvv1vQ`cz7DYhlyngGhMUQ>RQisaTup*P?Lm<#D`ZKZJ(5I|QV-TgimBye zpzlE#usX~`u_~d21hew&-|W<6^Np_aH?W=J(jMhRpo#Ro%%9^Odpw!ag8g=BTNVa) zd}C)5_$vBM4n_ylHl}vsJ{%MmE$d0Ib%uR(3=$!r|$^Cwd$ z;&D8iLmILsQy{L3QOd!z5+OcxKBQ-?8U+j^EbINWs{W zVDxvix|UxdB%lPCa{2?&pu;*k-3UU_VN+qJ%3}a2#I~z~E)Vo(%{Z||1!x&;L9j9D z!F@T2%0`GG?!^mqX3*A?j#-Sm0;7K?BiNp2hz8Mz9_h+~mBlHc6h4x~Yha-tqeE2i zf7<)XsHoSsU0Q^pk**mU>2e6^?o?2e20=s+l#uT37(yBZlTtxYY3Wu$kp@8!k&@=z zv-gShe?Poyowd%#bH{aG*L91TUdz)<0XZl=OR<9rQvfmzwsouW;tdx` zOq<}{P+AfHJ(|4kIPYeQL)Lx++(GPNM*yK}2wr{a{g}pG_xVXkW4aU1Oc-z(xGfm% z#Bm0MY4-Me7}|vWKdL0cQ4)M?D_0cJ=Kv&NK{tdS&hSJZ<9Yi%lXFTPV`J$gDbrgx zxuJMWAXetgZ?Rg5Q%t;HQLe_(UU9%n-HsRk1F0G{i}>;$wqg5iB=sG`L>(_fD4BUA z(02mX$Olv?snGH*Ds=_;#~J4$bEh3L$@_6P+(iAAeh6+|XHfdFw4@Ay+@zx*px{fBqnN4Z97(`-^sB$ z$i`Q8S#(kEg?$lhw)OtmBE=mNlqoO&&gUz;fq^!S(g#MWhNTM^{0ontcHNGl%b$Nj zRK%!**=2k-@L{w6oW8W%M2k&t0wJH03(WN$^Bu=+ZlMDYB_~0Ga_*;?MCq^ajKnaU41|aipuCL6m3ss~MmG19K((9R@L%2)VphTSo_v?CJ}v9t#4m z**&0kgY8O-_9}Yx{3;#MFdBsK35pT*#TP5%#U=yxFDSf+2-_zu>^DCpuZ6Qf9UAcm zN*D$jEoYWl!>QNuZK1~7KZZVDuKw-zXkz8t8<9bp8if9FPp8Un4-R{A3B2z>T2dK> zgGJ*DGBb--^C6GcagBA)Pc+d8VxXl^f&=-!Enc=vQ^T}p0^v4HePc|(wgvb{5=&6d zbZ-^2ZhKJOXeY@plCmefinSJMKw+F2*|j?KROm4k3w#ll$u(iS%{YhUs1UFN$~{s# z46fq5UWVZFdL^-CfDsLyK@=gz)Pz(;9S};QMJ>a9*&?7xyKL3ga)&L=J|mi|=#rDB zM?|Th1Y@+sPW6|2={*S|37yM>V-OE>*!T!C7Mn#;;Y){5t9uE|hVI9*xOBwfLa6DP3OE?dBHw0UHXx`C5xM4pXLQ`Y#3LILCxbxVOJWByYOUO&h$ zUfvGiC2lJpFiG*g66hQ6lE82*&At>-Tth#ml_TdPOnmFa@BG`4{^5ir3F?gVRB18o zR-_@sku;mLF^P#}VtQOq9ffjc8PdIG78!=zr~Z zQN3^bvsLwkb2X9ys^P;-{iZEf4_dpePu23QjQE2V?WZ3xqW~$ zfGpls464BR8(Ayj6n8DcdJ&{*&vrvxre#!7@5VeVs+;-Cr}ytJqmM5+U1PpIKh@ue zM~%C)7IXB0BfP{E3kq^oRS7v=ldgMrCB(@w3l`=vG3A1TleR++2K+vlnVG@Qot5>@ z3A?#&6i@(@NM5EdofHX<>FZcnSio@g>8Y^_wj8&-xn|#CoIiLdgc%nS7B)EZNc{(O zbmj9!#61KOqSYhdHY)RPXI#BpDd9#(MU|SDcYAsR49irV(hEQh;746mP+M0gL*i5y z6c^>?`E2+0G6?>Vssfv%K0|MmLb@8D<_A&O5m4!}vM+#hz&cdcSTXH`U)R9j14j4< zL?oDr;7xseP$wyTRpeM97b2yp^-fsQ#?ucQ!RTz^Dkr^-!Dgpl=Q`ZxE}P)q%2?_ z1+6vchFW`}NeamWqg}E)?bnvc1N9( z_&o>u9CUPb!^6TjTxJ<^LAoDuAi&OWc=Y=Nq?42N$vSuk?gg|VlWuNqL%b8i1m9GBkXp)g3Av-gD85-akNaEwtHIS!|~|QP_hB|uDs07{t`pQ3GxqRYrM7p zH5bShP*eQ2WD?YsiUB3-n)uB zJnRAMAcEkckKKBl%K5*XKfQsZZPI^L5C2bolE8@e zvwJIC20EGWB%DuhHUB-vJ*aK6(&dNO^+}k4HVqXCoM0$E3KfGDY=~yt+<-Uq!RL1Y zNEeG&sTir$@KypBJd{!8@O!==hV{NUt?1cOp zqQjJtBM(B@I_%iB2!05%0Bitvpi(2%2_2!kH_+RQ7h^jEmLd#a2L|pyrwthyuLxJh z%S#rG-UoaEZW(xaICS^*)diJz0k&qNvYikT`2;%a2LPC1HUXW4ZoBA4K!E&lSL{U8 zWQeNC^f*1oO1#;>KMc$8{rA)TKSGFqC^G;2C6@R7&wpS^q5po_|Bdqld<%TzE7!H1 zCpUtXLxO4#S9_E@%>x(`I+cUwx0Q|5d)~;`RziT`@U^h>0UPGe4 zRj_P3KX|YQ!0E~HF-Lfza0&v*3X5-?h~s((2FeR0^Ho4%eoIi_-6ApSl`CfM@n1Y= zFsfv@e&a@>j2Y}v=`s=Lz3=7K6mU>0bxT^l={|my%(O{HXG&CGW|X(^io%Yo#36&vjFw}3IsTi2V{dmi_2Jr1>s)1$anA~0OfFv(?Bke z)%_t)0?luybVy@?55UQu%!~9luwU`StO^juVxHr}k2k7oASzNjcRo3|&&KlT2vCOw zyzplKT>*MXDxiqu*cwpx-CBnxx?~a{gOQg|+qAWyP0MvYy|h+54%q;z@gh$m#2p=x ze$2~yhZYEq-eAE9_@aeYHOaMk8c0t(tt1x&GDNHuAWD@bt0S8v7mRuAC^blj4m=Za zY2@I_}S%Ry>K<+D@Oo95x z+^7Dq2X~a0h%44fFHV%og1EpHyepR>r{}{%a4h3V(7^Fle6Vg2leLi%`Eoolg*eS* zn@>5)yrBTWdOnuwR(sqS5PNaJv5)s7Z0xST{UwQrm|G||^nF4|J7M_(vV_}IgLpUD zEgYP;5?N$wEo$t5c6YMBkPH#gx=0CE=SG9a9Oz=O2AYCo*HO<{zqvZA&AZsa15*o8+hJL{g!+EVIHj+e>JivxRR=ut_F$AA(NeX3)V>z`<6{{DpK| zuhCRVke+MH1E>4hmFGBY?v`o7+XZ~W^HI`lktdV`e-?QhlzdUf$D7TyAj-7JlyIv(Kb29PMTk;JRm^UPm& za`?1zVGlpkp^{P~3c(2Om^zuqKMd61gv}6534_1_q9-JgRe+R<$e21r_JK|PSU>Nz zLr>cA;SV2CaxtF`D=yA0F5wUR2abg)DIu530-Q*h`L5OTqtXE!xy4*WU9Tx8tO1e-#iFjOc1@<6Zhw>M+$~C_X9?6Ocj!C(W$FSt2PKxOY%J)Y$Q*(XQ zr*|N{@bua%fgZa>C{oLe%r2O+T=tiA?X>E^*1}-)F?0XR4MY>%FC)qKqw#85(YZz_ zS(K84@zX0p1Tn8>S5&lC?o!@Lcmjg(#@E$8&lUQW-<=S+z@X)Pz|d@)Br zpZ+jz3=o5NVG$ObGr2?DpEbeC}%SDEk zX82WS8>B!+fWmGcn{M`lY6glqpTWBb6)vl~BC{yz@s$0XHL#UX>q91%h|y{oxaRLL z$Oy{rHiNMjA27h5wx_@H-Gg`J1C0MMj3;PXTU5(hvN!{;DI9OMv)nZ@{K;syPz(;Q znkJ+dN`Rk9~i*1A?F}{}%S4z8gKqNfeAosgjLROaP29Iw7+4Cy1Dr&sA z9aANg+LKNBw)S5!IUF+bYVvGo^-Uv)^?XcEbTr_Um64L~%*Ts>%i^QMB26$lBc>x3 zBxn_-CDEcAlHbM()HiSFF-1H5LcdE9SGU(gsdYAJI76@-5|Ch(?fg2DcF6Gn!+2gC zkEqKETP6<)E1vEzSfCq4SX0k+OAb;Bv7)sw*PK`9${9HKO^P%T5+%t5`?3Xy7=}FWJ7u&q`LwRqo{RfGM#s_|z{S zQ(eW3{6hmtN*2#PbCQBsNq0!BeUv7NU&m zO&AoK!0{%y4S2eIoCI{0$+&+Q$_Tb^!UTJouAmF6!3j!;w7HK+k}0nbMo_%smQ#xm z9cCpUH`Uw#r)y0u-Sk!&oowO6SDP0;mVAMOTS+mmeh5yN_KR>EOZ6uZ=JQM+l)3r4 z!BK>=2x@-l5R!zuX*9_rh{oC~QFw`8QHN%+f`BR*hLJHz+Mr50$5ZQ2>TyNS`K{k2 zny1SdXiynR-fQ-l!=tesH7m$Od45jH^}i3m z7AB|^%J&d9RG!Mq<>+;qHXa*P87D89Ys_{e=^~~WLnZQ<2aW(GS6?sFNc>^xE-ejn zMF@mml0k#D?@gB$s-M=FFnQ3%5=%bm-ETn4k(hH+|J1XWW*hAFd;w)bSuoheBQhB$ zBNBSp7OK60u^>vETrw_5@~srO0JNd5!#KQ7$(|fsB5?^%(!_;g(;bYx;(FpXX_QS8 zdbVmT@@=tF#aHQquk7d|@`&U+hL}c1#q)b7+>W)j!nG?%)8`o4lest}vTcmD=`KR= z`%GljYXIG0m?X$@ZwBl)tlN;-IC!@m(2SwqSL;jRqa(foE;E$5932+48eyGVN6gP# z>l?O9f1k{N1I35kKgKNN(MtRbWZznFHW9RRw(8F*Nm;1k9UTxhFj-#Ng{++-L;%%Y z@lNLuWf4{oDa@y!6KaJO@*F9npttW2!7PDpm4PMI4)x;tD_3)X1;FVNNG2q8Ts8#n zpn?LxDOo<2xmNX#F?Vg(U85d2z*N&PR`)FO?pZDB?Y2;W@|iTFhcJ=Vd2~RnM_xZy*t6l(yWawVVhYO&gHp zw0C+MsU@2Bk_9V;(+8s3xWD`~V+#c2STRB%icZ{=%iMXYvbr>?dwEB{9Qps8Kv zXeGi9XzGmSpuDaSR1h-uAaYnhu3-39y%_Aj*#ym{XiU&Xd~*1O<}kjq+{xq8Q(()S z!cnVLm3plsmq?H-(}9(L7w{A9h*}T#*mb>8=V-M!3WBaOH?9XvlQr)fF-P zD8;x!e$@fc@8vscz&|tiVhitmB~;GV&it9?yZu6F20tjK20H+Dk*{QQXPxC@r#H+Hea22x6$Y7E z1X6Y(rX4!3t6_goU|L2-$IF2C(~H5<`_{eKhz0Ilz{j!PJBPt~upP3YJ#d^!g7Fs> z(>ft}TK3MU31j&~;35ThfBN6AmcIN66hAAU`<~x=c3AoT>|xM1t)LCe2c9T)^4UV} z{oe^j*zZ04y#NRWoZnMmYe3O=Z)>p!1Y{ZtxQp<1KfK8Y`crc$W!8$sV}$S47` zw*)^xeP4jl)z;8X*b%f{jz6(C%MU#3gL-*izzj9h_I#`*(4S84v-9|Xa1BJee=`3Z z=sY(uP-`JD#7Q-bJ2mDLu|2z}ke{MayEE?r94f1)(ADJIN+bgwmy3g4ynon_J{QiOo+ zb9Uq@%=Uvn^M1%z2+YN8=N&pSg#F=s!ykg-;=(ko5n54TUmct{fbEbe0#W)MesdUn zPr#*tk_CR*%xI{C>ex?0E+a6-;GaAKDGJi!?yqk?y&3!rMDDGst=+~YqU5=00N`O6 zG_c_U=6;Og4#aQ4Eop`lXJGU0x=^`^1o4IxP-DQ(2q3l>fS7i`#I%~jVTF@^NA~_t zT7R{NTxq7b=>^(Rsf~uB$|OChd;O9AQA6Yeb_+c8{jcCVv1sDA*JP!+VBEZ2act25 zJTQ8w&;gFpV)&l($S5awk2Kg@X>#z+RZ#c(b6vRyhpXVERP6?{-Em-L#$@yslz>r>bB+_qn5W~Ne zmI_(OFa>C9ykFsS02kL}X*6GAJGz)OoQ5l0!oHzlj&lm0RUQ~3CMz7q;8O=yQQ!(f zZw1E))NY5j#clhI-fhYYtELKGisgE?3Z?l~$ny{EE1h+idHxzzR-1=q&@g;S=uw3n z?flTx!;Z((U7Yw5fEt0bEuqkh)TjdkLn{JqvQywHH4V)em;gZgUd>_~${=6MttBl; z>Ej#ot7BkgWyPp6Z*HL%8wj$3WdPpZZ)ZOZ5wo!Y>`dTh^L{vm0vgJh1Uk9?4E`}7 zBfw#3gdYLTTN7|hd{xz~jYtxZ<)N*B*%kQztfJ=Dd3N(%E-sK2a(VQbScp9*!Qe&q z6QBjK!g`bMJJtK~DpPE0-SAac%N9$s3mYV4#^zQqvM;*c5VW%%rzJUr5>xkd z#L}F{H~pBvkA6QmZWc1@q8ErJG=E`rFc;14K3aDE7h}qWvVT1WM<>1`#ZAV$HL>Am@c>AN^A#s|1r@t7hZM7sE0kV%cPPT*(cUF{1lyAaaO)9AE zSSLxc5cFQu3k$AmppAAFCH^&8_-qS6hxS@1Quh)sP0yNj87%OOVV9Wzs|kTOkduPX zcD)27%4W#{=bwQKb$^>Qk7tnViaLqgwJk?}OH#U|ZfJ~puF*&zQ11aJX5fvRz?n8S z~B`h+gD6&T}T3yGS`+x-&2oKsCU@KG8MK5_{sq1tM_4zb z%JyT$6PJNpDJt8x#A;{3LYvd9V>|r(3xC@9bu$Gb8vwyyXllH-55=<_&6>TYjkiys!RO;RJY7sToS&>5z4PX~#_d_w4>{2&xP@>VC(6KFU1ghzAs=0`2<)3iH#g><=k-9bze1 zWjmNYy|}?lqn;akw)@3eGoTLeLp|NIIWyldC-Py&?w*8A`I{XFPrXkUJ9wla+h*qjsPJB@hhja!y`wO)J8Q@`&=lv zq+Z;vHAkJDYa>J*!8(`OCfevw+eCRUF z7rq9Y{F8{SsczBI9JR8gNbkIX!IXtM#0=w00ZGwD^$>na+|k~mTFsQYs!Y|1aP*_{ zAc0i@<}?V|!_2?tiWXcQ35zBXqpj)+RHv8)>IBt1iNv-#Gs8nfl)z> z3&1JB9t)WnCTxvszDaktyyX@oz?&LE}SwyG*T{?i`Lz^h5O|x{UaA{AdFAlGj?uuOCZ&C!j+L-2?10 z&9CYHTGUI<88zL0%w?h5SeMOsyOgb{#`TW6yX*oNc-jm)2A^G1VGeJsK`2MVXwBY!4HbY?Q0Ow|TWEV}UaH zUYjjdY#c}X#Q+cgC%wgX(2(u4t$MmkW?%b5>D0g8(Ncl(?Xt})em)+jUQ9~YHrK9`>{r8S&kR^a1T_ADe1X6=D^DJtku(PGeM(GnaaP#DEF4P-ofe z@#v)na1*iAltzRX&6=fA8~1>@@h}Gef+qRi`MjLO-U{`#8n4e`E$HP`+K&{2_tupJ zdP+QI{^uzzj_tWqeM8c~`lXi06Zo+s*CpK#P*hH->8@fLTh#4;4TRF9yN zZbDZUtpje*X=(UKR-JA|B&h#|{ZpgMJWEaxl(phi zsE906_&a%t;=0<98>yExMzrxRyx#2OrVAf?E$|TMqs~a&MF)u12#LpQ`!se() z@~zR!`4BZgoAOA5pHJjf$t?%Xh;@f~ULO^^fO!g$GY(U_q*$E{zSbwks5q-KEvmh3 zvX^-D*ybR(um}&fl*r{FDj25uFZk_%=~5>KNCXE=Hy8f2myQs{B1(}J4F-#!=u_W3 z;s$KyTpRy^G+NWQiQ$xKnp#s!a=Caj8(^+h^jwgy^;L&sQM_r@O56TT=gVQ8H?^X= z=2+wOXocJ4M%#3ieI<{ji12LcyVbkb;Mwz^y&&`&iEQN@Nkf0c5u_1n-*)$?+T246 z^a}=%wR)u6u4>@`1NzrU=|?CmjE7(jvEV$nghCD))pvWHSw$LPraJqs<{tkYw5 z%8jPZ|GvG5YL-dX6yZtVdT9%^2#hIimNxS>@gUy}ql2tFl=F{8s1w#vukx#4zVx|V zm!3`kz)9_E^t^`Kbg4*6A>9QQ(UhVz_HP5RsSdC(xJnj!S>PyzoyKcA5k} z$&lpjfT9dN`w?OAUm(V~gUSH%eV*)n*L@nwvr}55E~xggrrVllC}y@EB<@3=uYl4Y zKT|y1YYM%GwL=gP_?i8@0oFy}2>}RyT2vIEy>o}#&4-Ld0g7Qu>CkGIw~lr*M!}flO|P_;^JN8h$YALhwt06KHp+pT5$0$Y z_>79+O+ed(wb%tT;5~F9P^(JKy#je~=nf~g1Oppe!#rs+R;;*ZBa$izsx%An#6S0svE&WtG!3(bS>N_F1KE}X z1G|}^0fThs1s!omOEmz5j!Hz%^)P-Dv}#BZC6<~W?w*302LM?8Lq1>p z%1{&inO#A2juR+(irS3MeFK8xcBXimF(zr@Blw#!6oS0Cky0m#jzQ=#?G|?$bSoyV zwWa#0D0TQhcHO58q>I-q3_4teM@3D2gO4dzq7UY%ZQJvZ;j!C{xPms+)YV~j6|)cN z7~<&<$Of?RFasaQ*cc_yf>u+>@2`crA4o9iRMF+IY6crvMvWQiNGHp$xZz|aTf*E- zQ?x7`mlOM_A2X_^a;A%;zsI_QgR+CpB+NU2)UuSutxGKgKFe^)O-Ka7Eld}|+AGY6 zQKn=0u?H4Q7rMkB(}y*8g#qkB)trl>JR-gQwq_#JeIV8Et&HHSk_e6DGWSzkcGT=M z`3cZdZUE}S%uP0Agu2NV~7pB-;Y!YT`~Is}Ng2<>lc?ld>I7nC!_;s5a41QWe{eX7gWx?bd)fW0yjtUL~i<6foC$B)ba? zfZ-G8!oWRfkGKZFjzJNoTX5a%BQ)6qGkh89)MBZpG;P<$tG1z>uXp`K(nGGRdKvEK z!;h>!MyhWmi0f!BbOszOWjAs(dCbBR4AMR|SF@K;5P_@j93c`0L))3DvCo0=lF;K< z1Q#o#1_D9C%ri>~az#Uh1BXN=DOiAXnaCM6h+ipjzbHgdAF6CiM=p$?WelJ}G(@q% z(1KKYz?guaSXo?)f0D zpcw>SFqpbtW2dg7(grbZB@%Cb38Iaf1@I)gPUhO9IWroTLvZokKb$;pddxkqtI5;m z1N%CI2)rTNRGE~kJo0RUTQbVJb`1#9wUH6;wg=^IiTQ&)eAZR#zzbJ3$YSZNH4lw7 z0+O;o1Pa88xm>F1Pv6{|?f}iK(iIl*z{oH~-8-LC4L)$d*+MwyBOx9kJ`$n*wzy2< z$>9xWcVovnQMbv7#)a-qH}9h~iOUL=yO}i0-0LS{dFhLUD)x;PK{A`7d z!D}|tW$mh|uZn14CT#(5NFzrAgcnMN&N4d!>*v>`dSScPd!U9mcc+B5)e^w4(1GAe z;a9sM!9ai;y%xKbxt<9!dZJtVJa1`WXIZ?I2!?23^C#mVLg~!~kHH8gyV0`Ofb?v} z{Q|r{e%xXb=AH=BAJ+wx^>$ zM`R4gbhz)F2ZOXugf$%$q%(=~QZQ|vvAbG{-N|91VBK(Q1f`+MQloK*YOp{HOZjfj zLhum7^S@TCLdYa4)ka^SagzdwI)Yv&s>h2r~-oKt|Q!n76Pbu3%knP{cq7EmGu+(S|$aXMq zBc$ovCJvFLnv8i(Jp}iy!^o@@n#{Y+e~8*NUB}OncA&+&fl|sj3AmIUWfV7;RRCPy zh)aNv30&c4bQAY(4av#YjE%?wl~sEP)gKDQ;}k8l{ct1=-T^REezRMv+F_(|oC?(n;k8pzShCy?cN>^ro(hNyM+oYuzq5W? zhQJbg@CuF)dm!%4Oq%uAP)gW|DBL_r{)=>Q;3|tJfHis%zObyhII2?MQu|XUDora& z`WeWM-;K9{Aj}aUX;JIY0}`nAwM>^f@RyudqdQOWh4(Zff}qP`5#9oIuy4~ez+BMI z^mj*uo<;iXA|v4_bDCB-5x9e7IQ%A3>aFMqj7?2l`6%X65#n$)VU~)V-a}GZZ^-13(oS$yWv&!t>ceD95Jjs;iRNM8x=l0ft?*veM`^VC<| zoOW{1+b?Wq-hP|HPw$}C?|_9AAh+H#vym}c1Qy+!d8jQM7DX_ z=IL9y@B4Y)_xb(4_x-+q{0w_vuIoJ4xz@3c<5)XfUr(EygoWhPsZ->aFKHN_I)&GC z>J%;;5ib0uLRtU(sZ&o+UDi;&;W@OJPI!ZUAiMF{>0RZRPvy&Gjc5xSJko#wDy{&M z2tzuJ2qM+E?5~mDOWcEPV-vUxrJ4D+v-pVu!c-7fRy0+r=$eC`O&^nPJX-cbJkq_P zao}suVBhoQRsHtj1)mis%Ka|~jhw_>K~!o)TqR$QZKzMafuE4Pe|{B&tNZ6iB7)Sv z{y-x6*K6qzf4z<$eIECp@BVs+e&%17%Tl5Jzg{4Yydir4trE=Yd zQ)5NNMOoPtD{bk7SFgtU`ufa;RHkmCv?zb<@%{U&gd2sOLtl-esv9XvO=OX^~mBwQZ zaUQ?5&g0!)e{M>m^Fsk_4D|Hj+A(FG;_FjWQ+Bd6@^EOnx{=oO*P_qPG@Dd@^pmXxOFQsLq}a*-NYotbJ~eyTI}4pVE2lb z%>?KD)~M9HvsI1{SNxA-k9R1EiHR?NwK!Od^}liB2HlFxIQ{WXhw{;Wo|O|GoN2@=S^T>O9J0D4YSgQ2mONZ=(wZU)zz@bNLu1^hHIOf_0E%O zuC8VL>t>Y?mxe1Tr-bCh`1m^d`)?{d>J;iBcqlj&@TJJA@#v*f1NAQXl)|IwOP4MY z9hklT-rU^W5zEQQ$#wg6&HZn@VI>IC^(a?W8VeGVpBO{^`1trK0(vGU&-w2HYXK4A z;XkWg`6&EE;%dJ=s362%aX>Uy-3jpOdGE7$hkvyGrw(QN!83dt+Djk_4($pFqNEo zRAzuwrB9Qm!}ojxIFtNnII&7HNZ$I>B$9AuPrl&lB6C$5-R0!uTJwk!9%jCKr~Bmh zpWnl&{`1G{nd5whhK66dy9)~nFz8V&ef_xFTKVS-j$h!d*E5N;;EMp74ncx73#FvO z+?(ijnyVIppC!3YzWliiFZ^f78h>9<03D*pN3Wq?(5H&Klg>%e*X3A*Ug0iR|+=|8ct6C zomFckLU^|S@_@JSI=_5*a&#cuxbQG1H(;!p_d#Sbj^x#^x8L@&WqCj~{6C)5b;)rtoYsa;Z7EsI%g%iy|d>p03JJQBSqLXZKm7UQpw(#5dWC zBdBR;P?D2N)xF|jXl|bN{5iIm!zQ%mDGO4Eg;BY|_lDe>V8|w<24bJKX8M4kpr%2g zvDcY%p6~T0owIUDKQdXjn!OMSGv#0l?(8#n`y^aY8pjz#?W9sly1Qa{7%UjO8g{*W z?3^B#?DYJ;UTHuX?b!X}udZ`ngA7lCE5VWXI)O<{q*^>UWSrya{xhClIZw* z8^R%vNq#3MCjo)CZ?ETlj9wGauDdT*)hbu3mxd-KjcsxxQLwto3S);B*t{{VqtVqtkD??F)?gWlZR+ndK= z&VJCB6m?=^BEpX9)H{*_`c(F!!otFi*4E7M{I_qL<#2<7gX_zQHI4c# zQ<9Uvq{}nv;;hHABt3h^O(ab-ROeL&f&ZYmpkThpqzoNpUi(-#S?N_uO1M_b$cVXu za{gQxxomiJw5FJ-sOb0Yr%z8A;00eiD<(!o$SwGqg21ZIa}A!gFF7+y#mHsq*|TS~ zyjSBA5?ZP=m(;=y{%qx#DB7&T0U3_s>FYQ%%*@P;v3dRG&B{o1;KO^puM{zbAIi$C zqVNy*TEl6BZ(BUJ=Gl9_y6W5~qT#nWe={gFSuaImrY*<$MfL`(#NDvQMt>6%=4UO5 z8?(JRj2g2MJ%ep+wh$_Vh9)NA^&dZeOcJ%y%7fBqF;1+OJrFb!Irj5(wRS|>GWwW zByuw_pdJcMm-j$TtWn>@qi5!IynXwr#hW*8jDJ?S@Ce>c!sBWku5i2{A%Q|;+j#E` z6k9MYN8A!B%4{)&D)sGKOD8kSkZsdbty4a5ZlWV%r>QvT9bu1ib8}5hOo(Db7^80@ z5d2$eZzCfkt!wgQiL6HD%Kz*_K2OsO7Q-@pHJ#vP5h^5XgPAMebzzjkd- zwuDMZNOTr>HczeA`)s3PxjwJnCz9OLBPS=vmSB|n-V0kbuvy55%!${Hos(G;M?XQ*Du=zSO@zCGd$aYA1{6;-|7ta}uB9xG%Rmz_C+AB7C`g25*Qzmtu;ve{och-J&09xv^w073}QHFn7*76 z^J;D-Ms7JdQ#G}zk!n}b^drxW-`zNPr_F2J-gaS$-D=<<43t=DhvFmku3b}pyqKJl z!Ys+o+9%J;$EW^Q6E}KUK~$8|jEcU-F74OQM|-D{D&!@8bO~+|krGxY!MKx^u=sSO zFRSfQEdfja6$68{FDVzQtE;IHUS3|19Xc|8Hu@`*(Mv7&6R>z0f4KYU>l>YojScJK z{1;)IOHn&WNKe=G^&w=Mbs|x1Z3MW&lDbpF(vCxL_Fzkc@VL-KPk$_~PPSZXgUA2; znWVd~{Qfr$+@;ys&=69tmGQ<{BPd()aMo=cBuW2<5{LG<>B-51jsE_Wl$2?0tSKmi zn>VHIw1#M@ZrNHOaH}#zR9*o#_NWL=)RsbSz2+T zjVw)x(BgcnM!!ylGmo>u25Mu%Nn^J3lT@y}4PnGSEa9xS!i{{nDzAOnRo4QYSy`MCg5-JLd) ztZ=3a_oC9%*^6$t{YXhkvFV5zg{e5spG$jd>lxv5=gwg}I>|l+QBaOWSu$8GcE6PF zV=TzeKhskAaJ1&h?uY8?T?lS~2eD^^WYMLLMd~uVx~W027$cX7NZe=*7YC@G6r4&? z?$~rX%&pM*5&O>v{RBhbzhhCsq_>LAWt}IU#>7xO*oJbdZ#D7!bb6Y+hsX^wGH-)( zaI_4p4Z1XRH6A_su*$~jg6!$-y```J^!Y%4|KaL|{5IHQRaMn&UR*A&5BcjuLqklg zIRvZg>(3GrTwPpLeF6$15^+MJM>rHb#o$pGS&=tn!9YUF zkVqa)es@M>I|+rV{8LpmH8tEA%?p?QKCH)iHX+G?Smk77nk4$fGhb@>^QLO!&@*hw zMIdiuj5gNSe-vBTJ2{CCHMq|9eEn+Jn)32xWWxm^A=1l?bab~5+x*{Ucc>ln`Gpb- zVchyI=lG~Qz>Y*mN7L`!uom4Dbj!!^rl+MDl~_g>B|d*nble-v$tmYXG z&ieojt|}Ed zPPZ;XB?tGFf)|Yx#m%qX-Q?~UB_wEa^s~mYP$-!`H<=?V05{PE(MS;rY9~c~oUBoq z)vcxB{p}UT0{Q|$LH5+=B9wLNUzjR$P!Y-z6x}^37wo8ny`^6V2#TiCMAzqc4OZRj zW_cZaCAwXg>Fe>}?gcVQ0>lUvnfk)7j#x;=zs{FZD7Paq{MJii_ip$HscztEGtMx` zYTOW;fIJ=&92`#5W}u{`gpdx3`*Y5x1E~<}_vSWn$r+JMYXHH6W|JDDC1xXU)G+@+ z_`n8mW~9@bp1+7fp|+Ms%}h*o^?s|+{_`q2%z)+rLELLr32 zn|dwP0t@V&aWc1KgwsphL!uO1v61mz9%4-7#5?L(pN{QO`G=nsiR3JVKE zvkiSMoJ7XT2hio*hlj_FVMBwmkxI-njJCJMFu7&)#rf8LKxkGWz-R z=gO1hzg7kcQ`WvVV?Llgg{wmy9opBgU$}6gtfoc|x?2SWw!@J$SyuoR!AWITUtT0H zh+cwZM73O6D(3F)&K68Tb0{SDJt``yrluy6QB;thzqO;I$~9 z$p;aHzxc6=c0Fn07cVwI;llWTz+A7b8;Zl#?eyV7cO?p|yF)cY?M31Wlc8Y0e0_bL z#_D9fv50a9(ankEP3j(srPHalW@f`4!x})L0ntT*Eu5N} z+3eHi3?D1m&sKr77R4s}Xmh^5N*{1r-}T%#Z^CC~AzFll3hV0u3vaH^bU_e6T-qfa zBb-KSHh;w`Yn9VsDs`bFopo~K%O`bu`^6_r1?tB*tW$oSD2Zgx=WU~bG7R%$%?j{iwiO`rxk!8!qztUHq`>B{J4F7cJ>GVW4|@l;eUF1w_bV- zL~A!JSzn(4rY5texVV@iJ<~o&rKPk@)W9tP2M5RgN0GRgSZP_Ayc4uTtiOZ`SM#5Axd`kDT$kQq)0up^!Qdo~ST8~1MCo{;xH zY7gZ>r`7e@f3BOYyF`%6%FIk0?f2{Yd3|Qc!?c{70Eh`lV`+G!j?*(UZEbCeK3h?)|Lval>gh0mNGI_H(4*}++OYuv0W7Sn zSoQq!w}cV@TD%(`TO&GYo! zd2iD(zp!vZ;r*MZ((SDMpDDW4Pc%zzJM}-_`xkZi7a_Lyk-&TA^9!I?hA&l~5f)Z_ zvIEK3%L?DSHH^COR>{5YC60>^M+PgMto~egor9)EW!8ht^A{@oLuY=$qLsdkvz%! zeZS(Q%BM%WX3=GpG7X&X(ES-0+WyPmb~@LK6JZj*gB`pXTQK3!$<@YAh}(8HXP8 z2XveMpLn>o!^6X&##ork3JQL1Z-+yoMUPP2R?O@@dVWmDA&kfrEQm&N?mc}PN zE$3zDr%$Shz!Z#WeFD|#WFh`U>Dbv-(@Pi3X-}IQ870QWo#ErF1nMR`o7nJxKcku0b(i#==5c{PLU}yy!1b1SdNNg*1q&H5NR!7EORT1cI>0$H&M1 z!GjnM#i7wr3&~wR35j(8b~`(QsJ%NCz_zzpASnO34}Xibb>8{@DrX{f9)Z z@xT>;4fX0~b()sOv4{*0M@lus51uJ<9)Se)&a6_zo3bo($$jwg*qHtd!*CRXY5XXmz-EFC^k= z8qZa4Z*MQJ>h$zcSOra7--RFV=!J|_)FF`IeyKl1`aZ_l3b)Cs;Naj02?@zZTv`Sw z)X>my!Vq1cU}U}+f{(VpOY8$6`N`Up=34!yPoLm`R)u8skdTnTZb93?PwV!4FtQOk z4}Rv?0cs%1sLM{AZ{6UfIS~XjX6EM$a&iJ)r6nYWDjZoaav!`9x+Qi{`r!kLflNog z2YN_455!$cvsK0!;sof$|D|ngn<7IVq{+FHXtkZBy9W;h@hck|jyAj4tk}w(+M}3z zdU|e^SiUPRW{{$YlO-pQ-r4a?egmYpIrI3(OPSJ6V-B}($EBnYPpT(Kz@7taZ`s91 z_H=g#mtMJa2|yQqD5w)n?1U6-#^&bdHDbOD4-ZF@maFLKkW!3Px#T>1_7whd@lOdoZ#|g%_U(O$9|s!^q_A4qY2uKVYQ}Bdtnp_a;Tyld)An58dg(0%)4s9U z-VA9eDPqFxyZYzDboH$lKlk*sd!8aGe)Gm1gqpSR32r1b@iA#>X~5;L9^J5bo5nUmv-kIh^eWNiP=1%dn-%9`1?^9J_`Zv zY`cLBn~(KY=`Mf*O)ag@U0vgKUbI)DI{~55(bE$kPpc6Vox{UN7QcHpyg9)A9L^S?&LnufT3&(L);mJGLTzxI*sq$YXCjQ7b$Z<|5y&|YYxCar4ID{r} zA_jpMO3oURbHaW;=q~`)i;`W=4RAeRcIa|qITQm%NnB(6ttzszIyq>9qj=AqJf^P-|8wBGHJ$>bz%~dDUm(uNjP0DS~9B7$uzuxwnW9AH(-%5aTozQGJ;;n%^g-( zi;-WEjwpg|ilWS_^tLMWVehGJNnIHIN-qDGMW6?14Olry{?nr$9{vJp*w4>zRBXkZ z?wB$?tb~Z@8%Ge~LpNf9J)+}~?~lVLeZomnStW zy{EG8bD^}c)r=zMRepzZSMcVFE$h8oLda3-hpjpj`^TgUbpO&p-AaN-6}k?DT|lz{ zw3@)!l>9C$Yw~cf0d^5sC14_?q@9=zOYDN6E@_^HJrq?;f?Kef>+-2r zQFmDtzD`s_%QOuV`@b;K=yiOSS_MT#zN^W%@7@LO+!hGooSX|F(1WfsNk_pZ1FE+S z9v!r)lHwRT-rn*%B{uOLz+wsjAU&+;ayB)6a-f>yRB8c+??`^NIm;NkeAdizP^*4uU^ z0%bu*N0;ftaFvdNBoDi}37BuV%#JeZg1r2G+a;)~(k@f2@bGo~&j*PNLV_b7a~1Gt z$MEy>tK?>lLK<6Im*0l|gp@QA@(i@V#-5H$F}l~TZLh9g($RrrBqb^N{KbpzPoJo_ zvV-F7FiI3i>lPEc?o?ZlvUV9pS1E{yR7_!KLXv**-!%pa%S*;<{q-sV7cqb~diob! z>LEn9(M-R;qW!SggY@`QV1e&sUQvtw}m`GHi9mII33t7WRw5%<>1tdSs}O zvQ7fV9Ri*LAXcDC%d<+y;nNvg3&4qT%0)IcUd{v+3_2bF;S{Q%*M`>Acw1V&@@%J} zrOmsV@(4B}Nl9fMRshm@`=rH|WtB6|FwkT`QMe+~zua!mf^hbAV4(NYr#=Wv;_~=-1E4UXqr0J%exLp3O|T*A zEu2ZiOe=yXFPLy$L{vQjBK5ITifTlEGgNfw_nig4rlN%iyjb2{s1ryr~OgK`Bfmn%9tlMVV_2$2Izv1-u%6crUA z+yP+`U6%o@X>Mf|>v_>}2wzsoXX|`6DxyjS@GBfYF0laXTkh^sk93fJn)+_QOb7@H zX7{?YGEIQxCAN2r_pc5F^XeQLRoDiU2^c@1mxwmgRgAk?S*?QmB zR?7eApug}|ODr|t*)b5`eO%5Qu;vy5ds0}qzdEV@8rLa=7HB2vuaxwO&z?z$iGhlZ z#A)P5!ve@GoG5G%D+>k&1|VeMTj58fZjGc*a4)Vqu>G^dD~6eq>OeF}KO7+5 zPkVS}F|G%$hLPVhH!%@7N{~tmTXmCN1pNkh8zUW^NN1S8tUWaMSN^weV^*zJ^SG+oX|ojcxG!g z2>|T@Hz)3fUhPsYD{$>m_FC+Hvn_m$wD`$@`ECm(r#Ar{7ZPk2(%quY^L`)HC+v682Ko0bEQDoLL8J$vS1X!@o`we8aejN) z`$YMbm6aIPM{x$`?I_yx!^1;7T-+d!w}AX6^1%(#T%GXjS?;wAKj_7i8iImx4^Lm# z(Q)?obFfA|N_4a=6~75@?~E92-bAVlyq!WTqbVIH_%WBipG6tlnV6 z{a>>pz&IcvVEu;?tGCD$4mRtXnd>mkxaE)dCVQf|kG@b3rPy2=zV*67lpX==N91<% z=8N)jNd>0%Rh{=z($a(k1l2V)4<0-K#}Mc`#CDa?Ai{wI!uQ0#Nrw@1-Zh`WS9vZ3v{t%dAUXP>14bE|NYuVndkp)+%*?riCserPuF&-eoH$C6zt=5EvDYfIbBvznaqcgh>=H@{ z0b|6Ewy5?$ZU_p}06)^|1=Yb-kVzr`0lu#S=KN=w-L+K7`v7tR2wTdaSCRt0$kuiS zdWPma@CM}N=7wAXIR_Lfp>yXxK3*CE%n4Bj3kQ1m6F7OIdRub)}%mZNf9KCSgmuvccy@%rTzzrXIm z3b$AVkY1madtFy|0IQo_I0f7YAUNvrue6}$ z#@!_?NV{Iya7=z8-^*qJ@KJ@ETD zZ-PHlTt>#&6G_yH1rG{TYQ#4xUTJB@M^P4_6asXIQe#~CkiAZUhbISw$TMe7jR(}% z`$AE4b-lp04X!Vur)lcnV0+>b;B)2fc&t$ce$gORUDYojKB>%GjCBT<85tU??EtN| z;&-|xHg^+{2QY48_v2M~z5soomnMTeQ2+4$D7Eqp7e;3L)oQlx$zdrbFs-7mFW~;R z?`1PdIbLb-4`H+ys6vH-5CBo7>3r5p6zdQd`o)VE`o_e>;qYATZEd@Idne8+@Bk3G z9vyhu+b@KL5e*is(s;_)Lw`R%1W(I_Uj6ViHEbZ%0rziS2S>=J>%*#Gtbv>5?Onj*r3vOE=fi zuU57!V!DJ*FYl7-;m^jHS&sO`sTncDxRL?x##$jJu zI5J`A_d3ZSe=~`kAu5!t7s(oGK_$`A(GeDY2OM5Yh`gK}u(f2g^!KcuCnYT`EI197 zzs<_xf_+wkp8H{n65ZSkSW|THqv?F2UBfo-Sl~B|ioo z%Ht(dz}P$DcL&DCI6lWL!9G9zs2=d1qW6R}jfw0n2vyO!d|5z1fEhyhII6KxnP#3F z9Y*!Fq^PJ4G8$wqXkxCW%bh61PaYD_`5|LSCB_iaqs>u%dhD^U zhpMCW=Ipy!+>he6%Cvh>?TP1;3i;dzhWMvPDja&PqJ^X9VtkI2HYVDC9uK} zJ?4HP{mV>{TFX?etEKL<(PAmVm+W+$n#Sg`#rD}3aF|;>*6ahL1NtdT{Cr1jdIS>$ z^t$fWSN5t?b7igX7bYTN#}RqP&I5ZRIPbc*BF4?7yCN%&EO3PtguHD_s_8*v0VMAx z>K|nR$y!IJ%U*9%g&ir*!|{Z!QK@ZKMg1CAk^UGK3)Sl;G%&CYK#Qo>0&4)7LXZdx z+eGlQ{>iQc8{={$9#=C+FjR=0NuF0j^uI1jmo zPf}o7kp{C>*B{e$m(c&{mf1G}CHe2$MK5KSBd;m#VkrG;e@olSs0NgACuyhs2pw#8 zpMb*G=yDmJ$M$2WIh{OSx{TWOeN2da#XJ0Qrn>*^Pd@yYO`}4bMFAR#?Il<1 z?hM8K9(+ET=I#4))Kx1wMGxNp%_)U1G?|NoYrTV-;lUN2Ts^ip_WJ^UKdxP0Y^-ir zod2ZRht#s&zQwgM9+YY#8qe|lSag=+!7-gVzu4JeUo>NY+xeQRdVG1;Mhk0^fAKfQ=;JpZsnXUos{n^_tXUL2 zQBBI&YXRCl?N{QSua*oq(F}=EC{>3kh8P0k%hCIbLe-j1< z1}!iZ5Fx5FWi*046zJKL)b?C;x2i9*OwHTAuaR*wl=+wd_`|n4_|7j(qs=q1J4<$BN~(YuDUlfeYB(-BnYLSb`=%D~WgB9o_8d z<1_pFcZ3{tx2FF7{`j;j#OMwOFoj&_15M;a@$KxS>fTb{4iN$8;u28bJ3f;r0=GwM zLBKZ3yZ(;yf^e_xlBFOTZ2}0X^L=bAWh$w!?;#Xba4KZGJRJa<#Df}vI8isi z(#ArIX-G|FsgsqF@q})y>|VDTPvC(~mlF-*#yqR^L${Q0WRPVV>OGLXfX2Aio0?Qe zNl7tE*ll-wIW8`o?sPH6w3|`5hY`5s*DNwG{*JWf4BcHw8ZnD^>%O6Te`ow?eLS=> ze+=9I?WOdxP&L;c3k_}ilbTW8>b)%XCyc1QZ;yX+S&{8``p0Vdm4ED}xj(md$e17f zDo66pbevKgityG|&6@C;@0PmB3wqKsd7rlOwh$_p2TX;U!e^gI-HG|?oXn|k^XB~> zg_-!z-$FWd_Y}6W*IqRilIdBPv@-v%flJ5G=d%qUf;P+lmIOu_7Fq)(CFJU z^@gKj*GuSw!JlIe^G!$oYd^&)XDNjJo+0)vVuaQ&-XF1pG?>*@mPdIjo-;LK!&I47 zX#RqhUJ~=`Gm4myl{@i!HpFX|_`Mw4npV@~73uWv4@`1t^&TXQVn5Lw>aFxg$=Hf{ z?{o+y9o_Bk!typ3*a=Z+&rJ$dIU|~xMZ3KOttO)G36iyFk@~*%VT?ZR9>vK2NUzKu z|3&DAeS{pVabEzKk=2Gkx1dmHaUjHU{As*1LBH=w7XJnCNUbCA)n!XdB}vH}2y9dTOskgSbxq#6H z=moME72l%(rIL3`R>u`p;E1Vm`GwBoUQbSD0w{#d#T4|p05yLb@+{cnFHszNxVcGA zX*>c8%)&s)IH-fYkJnQh51|p&o7}0)^aeWFqTc%^){o5v<|kTOTEv4G?wQ|ae@9*p zJt>{d?Q*XC3H?uG26;zVE0`7}CT5In65xe|dAz?VnFWqVRTqmC zV!X53{&yOTuh?(pQ>*W#jTPc_9~P}}#yY;t$(0$o;ejjRbuS}#c|<(K^cmX7xXXnEmfxuIu!gK@P!=fdxzNahMihbW&&`ehaye)cC|{G}2%Fla*pt zV=P6`K-`AnwCDnw8uF#;cJ&{BX( zUS3{gjZkv=>(_U|(zYkY4}?3*9v}AOC5ro8nyRVPpTLX{)&Nt#(~1j&?`_r?EheC= zo$W~BsiKU+#d=)`cWttCkWet#aKewFn? z+DLlCUeQX%a+(x&{GE9IvDHAaewFVJs>|jZmN(Mx&eqyn9$z~qYdJFYxbp;UxXBD7 z0&GtSV)$#qNjuV|e3QRd^#;T)C|oWNZnp7Cb#U6>q+`R>JQ*_TC=t6@YRR4MI!yEF za?IY@R-Zn|xkA@i<*r*Z{ZQ8vk@MG=7rNuMsV<*(CWO%5k17M_tZw&cES}g+}jaj+g~S&{Q&* zk0BviH*fYsS^_41W^wUg@#lqj74RDH6u{UIr-CQdvb~MX1~?TKfx35r7HH?BIJZF? zqMl2@9;8CaB&MRRZP(We(??qi1C;S+`S^4$T}lIsU8D?Z@uUd>m>c3pnLP&>zQTiYihsx=mQESY1R8`dXEG&pxmX7ghSXm`N18 zIV@#mz1}2hdpd?`MBC;!ql>DJf9k&chn45Vct;It)-#4F@#=eLULP;-etmY>9=mrJ z4!2n6eZ!2Fr?m(Cr9A#0U-p-CWgOF=esaj^EjICC=S3I8wSYFI7-LI18}hE8_^vCy z6xPAPxA=altHj4WZ?#)XO8k*K$c(rW#k{r#y~OcubRho_ipf5RdG%)oU(})7d=SnD zTEb4fy@TA6SG!Y}1NV+K?RuCz_})}s&}x#%D#fuheU_t~o+CMBP(`3e{w!i-YUJa+ z*eeZN?u!IA?lbX*G>6Y;Le(;tzT_shzw4~BZ9ps=I}mRSFcuxwvJE7>auug(qEb7s zlsHYppa1dV`hDr{TPar!;+XL-zq@KY^FWr|FZMdC`0Lo0Z`ia1vehTLS(kEkVvg{= zjGnOvXov@O*4xZ{^keu;wSIVJ@NUCr#(Ulg`bJErk}+4CZUwH~`C8Q*Q&Xt=lUhq} z-t~#bwACl_>Y0*Q1AO!2n~8a!9utWFuzkDVXkUD=R-cOP?;@zFueYGmty+O@83-*f ze*1%8!kUB-pCvn_Vr+1b$?~MP;p5{Ya&vH4oXzm3cxMi3`sWuypuUPD(3zmm2QlKa zbSw{7_I7o34Gc&N3+qxTvGHGdpOu9(>~b>XK}$muB)~?s0MNEQqZ)82OyhvipYY;^ zl*^O`WW_aNjtQ6{kx`J5N%lAitZn22#QpmM3f}{)Jhq*8*KJ|oqul*G2?dVJ5|6lOW)H&!|m|Y-4&bS} zK?mnpUU8!|soVAIFrEDhTKGGGaiG-SwX=(5m;ce+jKc|&K5@=JVcu6zK!BV{>=dqr zfq}TsmV>M72H3Mh^uT-x1`4oZM#@;Fp}26@TH1h26}ciVE>8Ws-gQ>5==d`m!oj@k z9=SR5X_K2bgPW+gwtyK4j0j6E)4lP>I_fb`JE!qh~5PX5=I!0d_o^tuWC9Q$>={R3Wxw z)33|;UCwM=!G92Q3iu&##A!}|7h`$oV`0}fW3X%D9_q<>WBE@Hf(~#Z#--vyt^^DR zW@u*AP|mv7(vXy-JYugkqY=5{0S>^<4HzBTjnn`74L`Muut7i52L{LoW#3Kth__aH zxaf1eEM_`iL!rbvh+v>zpEpCeaHu4jlc=S z%>j-iE-(e!kBO-YP6PP$!6QLB;Oylk8!`fXq>TU36A+H{yY!pIabbXq2^S3Tz=fYd zLFccm+z8X+?7(!<#)PT>GPcIFo z)Srp~#nh|~x@ttUkWo?7IdbWb@81)XlWYD0WDn|`7j&-XWvU^eG6zp>Ne12FAwYmA_%ft_(Z$Ld%&an~DFfg*WuNJw9#q#Q< z@r$7Ihc1z_f+8FeL`AHHi@3isT3ZS3Y4EO1s`@~zfwSdfSy^O61Sm^>=~E~)s=Hek zOY{WDOjPi{Sh^Kj=zzg)SmLj#nh7hY0H$$}^;P{LJpw^S_RaDBeWdS+=MBVbu;UCv zaZVIAgD!>8qWv*==3$fxW`*Do+_SY+4TFCNo>0oXahTP_eVLNti|x|}7gPZ4uSlxb z)xg!lxMtiv8HBZU795)Ibwx#AT3a>1M*Ey3Xc&d!mXLTfRDK^NfBj4m@CJYsEpnZS zoK%9WZ{4^tL1p}5^@=kcg&v_7zlcbP3P?EW9afEB#Isf2mol8V7tQ}Y39gEV1Yz@v zu5K_3sxzw*IAV|m;@V(bDW7lgqPY0lT5pqnR$jgl2nI0U6|g?#tU<%dN{ZZCU2Sp~ zA%WrYxU@7fN@(bF&ot=I0>j(%5@>ngDsh@M72ufxp1^uhT_bp;;JfE`mWxR8D*NXn z9b9td$6yI@cBaB-$pxcTa1dC-|4b(QdZ7Y@JwKW=LWRfq%9YQsbZA)ueEdvG+7(&w z2!k@;_m&Ojp!^tUXt)n_Th0-&F!6e7UJ)MTp+xX~{grM|%6wpXuyaiY;b@%s&71#yLyg7MB( z0Ogu2#KI}5im-&~&!{UAHFaPR1s{frt3fxku3XJv7>o)FgYo1I7y`F1b(=E;OOB*b zejhkFVc;{r+5(>?bB0;bTapHt?enI>-`C&@Tr{tPA3MN+CrA^hTH@fBQwTg5uAshv zf0%^NvUYm_tW=NzgI>UUR2KT`z}k5tuB4I@?5R^DE(yEQo)+_1Hi!HId4pa4F=$S} za1-zOzz8zraiEU@9J&cpP*8xniFt!QR51i%8fG+)uzAYxu$t2ypd16cLu2_O;MLdh zariKUNg^XqrLWMjhLSOJkHny~-7I){c|kwCw5-Ml)M?x^IA_*}Fj}(%*i`a>s?^fe?#``4Y@Zx16dfEiHwW1U11-qmh*E zA^8OVSuXD|DM0~K%BbC}#{45S!NC{?EvjuiLW+@EkKT?B!W3#J9GG?g0ca_nN&`+R z+0s{_*KV&4^4DFUPtgwC&^p==%_UPOQ1W>RH~L!*xtxUy!&nNopXoY<_VD`J;fP~w zVC7+z!AM3)6q9-#RfUnO9NAh&Tf}!&n73g^|y4kIbks zMmiLkRZ@7-uE$1WmOvwXG>zE@JY;efQc@qm@_1{p9`XeS-0S%jaXlg> z{M=6TlG+G&t_1*J5b>l0HGCv)ett)M8BeAb7WNIty;mo2@b{oEwRlAfGeKDR3>ES7 z=ZjvyZtd*E!09~C@TZ^PW%i9w@HMgy9bv*@^A;Kl;$D<~RnEkwiuu@MmgwZ|05 zcCG|oU79i79h=z_N3d8)LGVtiUc2T5M*xy$L1AG_F@!o9lUOD=2yQ7b{it>MrR9QT zc=!TNen?PIU3Q2eWW9q9MbI4&E0U<3>H}ZifX39Ajt=8-9t{+_zBQo9H8XZ}cVZ%g zg8)DOPM6iu)ZCmv1LM8Ni-Y1}_pe?hqX>mQNE^1m3MPbvgzURhrP|xuq2xd=U}QvK zf)f(H0qqJOAMkzw&@c$gFy3*}r%N~7z!C>wfK8fP*3aptq`NX*!v@5IH z$DlW|WD{3I|7Tu$yBWOI?-@Uk%C}lh5>%{yhhR4``3d%~=4YC?$7e}|K|%rVc&@3?Bh)5AS4VgbJ`6V{Wn{o- z`cS;Pb?a6K7i2IL`6Kw;3m6@<4%rqZmgLTmybq8)$eYW3U44qE&?Te+dQ~nT4w4*E(F~vEU;y*K>tG_L3y_DA!b$A~ z<;M+~tkkIF$ts=1^UY&0T;woXb2Iq@pj~Twdsa@)SU=onFtd!AoJ3kqf}AY22r-%2 zhzyaFm34Rf0JB3?zd`P$Rsy}v1xmK{D2z}1{Aprty}kGoFTCX4J74h@u@~^p+S*#u z!#gll@ZlX4u|=*6R{n=lLBGLfs+JMd^Qpc4bgqcZ$t)V^n&7Soj#4e5Cf?${m|Xwn z%?xTHiS6 z(FLHs_q+S08eZ7Fv;6!RDgXY^9}d8`%cxGyh6Cq`py?Zej}N)Sb7cbZpxSV7?jjhJ z!M6h*8*!~UlqD#q&*>inT-3OBt#@e1+m;WqpSry(q+u94gYjGU`-)-+RW-$|%MfqY zpXzPg*H!~2e$`w{SZWkCK6Ff6NG}WNn~j?P#!2hA={;dHPCQQ#Fs8h6J<=5CM(e%qO{n^E(|yR7O?^ zDE*+Z=bPl$RZX7>rqW(iyZ-d+rEAyVqoY6_Z)V7P#e4QFR0c|}60qh%LA`PF#>woo zjf$C}A@@jTo1Ea8A!fG7IJIB`rovywgU%Wn_=a?&A2ic7$q*zklm0Z97baL++@G+6 zxlb((o*KarCx_x;MMzUQK6p^DdYsQGZ?zM84<`_2nO320W_Qp1AHM!FuF7uv9)~v# z(hW)@Akqzz3KEI}A|>4@-KfN-8zdy8Q2_;!kd$sD1w}wox&);AKezXNzUTa&7th%* zZa+t4?|ohCT5HTP#~4!(fs}R}E5ll1t9wKj3d&W98#jK*ePZCBf!hv5&gM*Wpfdg1 zDA>_sRn4f;O(FpYB_~r~!dx(SaNs=^zmom*W{`LVSToHCV&&ViUMVTZk}Yq#Oo&Qe z+dr9CJCivZRvD-PHtel47suWGD&bcK-j28Ri+>JNoV3uS)jHDM_~RF^b4^4_lJCoP z$)U2<9Q)^)S44l21ikY%(ZAeC#=@0OZ6;HzMJS+_b;B#}SXbmN>6z@=aF9UO@n)vq z@9fIY!#awO;{8r~{M3@xmuz@HH+wvD%hu?b%eq=E+CF97Sy#bDL6J{%-ga9( z#4W;CaNCiRJ)eirayv*%i9#pz-_!UJ}Kq zDQ8q_PFB`5n4h46i)vE7JVg)=>V#2fZh#7%gh@t*LI3z*GxmkL(VKvMl2^VnO=u4v zRPTXan46oMfRGR*uQOl3ta{T67iEMotyL!CL3&O4YxQ(>p?t;p>{1uj;0AbMvTcR4 z8YQ1SuBfOeKE^w6(yGxSy8vP|qR{<=gU*>#*wJZ9@!$S{Sf_FbP?(MUtP)GPTy|2Z z>ds!!Mf@^;9jDKS&eZv&D5tGJ&EtcH3Z96oV6tY0jCaQ;X513L)}mU+^*KJC@2QfC zH=oT=%gY|$ap=lGw05|4ju{g%O3m=yDGMxR6Y9OCzVXW9^ILp&f-AdKwo{R-57zI_ zX>%L$p98DEY{84KztQE1FO@A)GHvWptX-_;?j>L@?)|bm5u^~U^^qMLPYX>~r+$84 z@)a?A^=*k2`#?ZnxRk%+#ztPv?&10CP(Ql@W&w}3&S7@q-Ral(qt|)$+Qp!)?>zS%mZg0c%}KoHKl-@CDjzv81^3F21-Gkf{yDcy z*mN-fJJj+)5p}0T!?uzw^Q~?Db(ECe!Oo5V5|Zk<@!T!o!~NX~K<&v$S=NkL9OwVpC-Q|dU zT{;eiJHaSBI!O-#b8LtL0cOujBEB_@J~$ycnd7?J8%mJINRG4I^7p-Rw-?L2`)ih6 zrH+)R`rSLZ(|7|l{ZuaMvsW$#g^amHw#sGK`svHRdfYhG1%g+iiiZ(1?bLj!&K2iV z5-**si*lCvZ`WS|NiAjzZ)6%5J{o5&j&AQ1#TIof%rz{Ze4ReWF}tx;NU94f<|F7E zc5;iD+F(Zl+AM_VbnUJAjbG(_ZTY6&oI=XCeU*RH_iW;8%l5d{lG}I!`CXCIgZBeJ-4jea(3ewcbri% zUvPclhXvdC^S#7a;7jqE*^j=KpAQ%lsYCMLmUVO+oV{JYJ1b8~YVM1!^{#MF;b`4QkZC8qCV(AxpAd1wiu$v2hY(=i1Q zChW1U+Xm7vQNYQ--!!|;CPR`Hs=oaJ7OM|?vVD7*jQUHL1F{I1FsJy^l`7LBJqW_y}i9lG)Qc^ ziow7Ms9TWxy|J!}s;UB680PO7f~EQSu!|9XSe_7B5*^5IFTeZKvibzVjAnk_x(>u` z%RT2=If5(2KoF{CT2a21Yt)xm>+%eY1rDFXsZm>_h>%;;b)?`X#<`$pN7qkl0xgP= z7>4t0?4K(Pyh4>Xu4B>b(X5s$`~H!`E&aTsQ}!S$mVQj^Pt@!=VR=W?^4f|Z2GMnf zr(4eS;)O5ccF*pzpMPHR6WC>)tTB&^a2lZ`*1q{!i!ziW)O#{%ZG_F7e^H&WzB%i{ zJv!RkJ)R!(V9-j-qC?e@jDqf0ak11N=j!0Mv-3F^7mJP7=ff1wr!#i0HY|Uh)Oxz~ z_RV1CM1H$joi32M0e|}C%NIDTLl~#)pX;Ph z5oiLOSOsVFQcm;iUxmD$%}LSkZ`(+xf^7C|A=RS+53)8vnllUDM; z*w`58Nr0+QK|l5K!WD|j*~A4EG#0)RnGLZiO1+0vRZ>uxUs{rHfRlrOQ;GEm!mTDa zJ0DMd_EZ}U>t!nJnuQXKQN|m$X1dZo4H(y~EJhX(Icq+AsBpDQ)ry(2n#^Ne5j5okc~tSz-1x3ibHvo^-|ci=9+2a;|A9O4gb@1rX{mq*k4O55v##q`IZ_$zD3S!|%coMFhG zXyHP6erzIXYUvAWx{JDPyqs6RJzD6<{fzjni|WtGN;cnY%SzDleSeUGAKyTqH)h=& zSV45*{ab)p=1&3VrE$wDmzGh+`pP0J-%(FK7uJ3keubObXJ=c*ZI+>X5wvuyw|qt# zc%6y{K8X(LiaYEQ+iLeAT{~J*zW0ddyAmvAh$vcc;@Dhy{Kt{@@?&n@E+NsC z0PRT(uA}b;J65rJTr#d5-)3LgKE30R#@}h5yJ-w4($`O4&CL}_V!0a zXQi2lSBO^%h|Q%#mmc<;mB5xlaH@KvxfU8c``|i1FqM zC1raM`wZXs^>U9~@RWj+ZB$c72j`PC>>xHa1<}#>?z*R32fHu#o$FYB28$_0XZ)%L7Z6&LSW~Y z!{U$U{75~;mMKs5T_8A}S!~C!?KzuG(FU`ZZc;WNs)4lvcR`WXjIlb@Di9U{7S7dj zmt*K2EVu8{NTS<3BNDd99Q@Bq{s9~spz?hWQv32Bb5N85csy2bhxI zv%1lwvWw^X_3OZi)`$5xv!bfGH|T&t#v&t#+THGVqF+#F1hgVYRzTM(v7@_h0*Y<%2JOtmVaq(4EZhUm-(rxY5b)ubq^AcT? z;2TvNX0dtZyNRvA#W(p;aUJNKvn!)^d&zGU?3S;&dF!mFdpv#}`R+y0Xzy8?g0WPY z<$%ON$n335=YICPYYe)6ap9OGV+yW0VKd*f(CK>fxJ4v)BBu7HmbND)s?Hq_u@je{ z)T~z}(>Om=(J-Sbi`NsGd}u51UhkzfrvFi)p^1HSBjh+t?X%JZl=oQ0C)y@8Zy5MF?jemZ`s4 z8pF8JA5Ee?z30D6S{CNM3_R!^W)Z{o>PnZ*RVT4_^B#B=q<$~kNi&IuYT99`>P1=Y&CKBcd;8nwOVH|Pkk%Fd-cmq0zPfIU8!e#8=m*<{Z4yO-k)B&4_755j{>Nc??3V!DIthi4x}TMgoOJXb!m&jUWs5lh`G1w5xK zofp{a4V-T4KE69RpV~+Ceoe$%8^5%A{HD}tJkfOQ$KDuoS>du+B;Eh<1d#}?+C+%3rzBHC5PesmYa7z04e`RgJP<0eF7PV?SFj=VVBSkiM(@IJ;K z)>pLay>~zKkli!os4fxc5#5Q%yl{BDy^u~v?t5dEJ(b!P_kE_MtDE8{-W~qxs@sf4 zE1k~*TprCU=Cmh#vsgl+kJ)N?|HPd6{A}zS@j=(!bj#b;+^Mp&+9irxHzn%bcTA(2 zPVvY)&zCo9FV}38=rs>WX1f02|0ev++J1*s|HW=lsq7_uC!6bA%0($5nu57*T5oDP zb1vP|{BBV#GQTKJb4e=dE56LNfr(qb-aRiuZKGcGrsI6AWw|TCu6C$m6ovnik}9Qa z^4U}OTFZ;HwjcLDV4$TH8$V2N2>w_>%b0mv`st~fGxIACZ1gQ5-Snr}jOPtsj;uf6 z&qZqp*FAY(AufGqX`|DqO!@zA2yE-h7?^{IigKKVj@TpSHvRLWDybBg-hxfOEcI8AA!|y-Ws-H|0Fs|=n z4e_m&G%iRE%7>W$;-AucE$=hWcX<7NmLMJWx+f`CNkD&W&0U_QG%24iC33MM+sv0u zLq}LzGb*|ARZpL?T_pC>NXzeGxmGJOx+ZrpHwnB?a@S(iCD0vcD4y0>4|;`eJS5RF=e~8Z3hzafCXC;XVAYVn`|ffA{H3qyRYOoE-m0-{dZ#(eXrh-$$G=BHWa6 z$jeK_mqKnnON@!wwJTx2He%cJMDmQyGCc3)0r$Rm@dLwFp2lZ+tFOj1I39jU=p*>H zr%(5eST8*(n;5y_eJ8%WBl_Glv@c@O?sJQ{IoEpJD>hca)+emSzX=2D zbX(!5QPy-HAQOJtt%kR7UD@f~7pIR28FLG@uz@^12>m%Ent#w!oRG@OF6<7Eq(!{< z&BDFXz-eAi!Xt^HmSG({CvRQ*+#X_O%ij+$?1>$d<@A0QC9M_oCXi(|bU(yxA{6Y< zx!tW-K+#3hMRWt_Mp(!;l!3C1t;tQ^uw)-C_Mr7TK$r?f!ka_)j~~ zhi4fVCz8iJ_|;q%K8Y0OeWYB9r_3<5W_m$V+rYNj8nB`xu}|(^l7 z8lRX@mEl01*CV2LGv4-Z-bsJ{MYiajm;U-U!%94<(2VQ{+76D9>1Y82#k0#l&6fJ7 zUeCxLemt!B>Z|Y;ndS7!fi(_+9G+R{M!zsUeuE&`d zQ^1|Zx51YWtmrn7O#1@}Z%7x+Mm!4+zvy%aXCo zh)+uDptJk|LID8E(6l~1I~yaj-ufmO>6Qna%#^Fw*%MH^nqftGQdBlE{ zj(Aprw)>eZc}OyEC0T5$v96SVs^GX`;U^16Qt+fDuRHBXu(Kj@uKSK7?e{!=fvUID z6cSo_k1ULT_<0YPocoKc9bGxw^p=hK`q@(jAhE+XeZq05Xoel!;qAzBB#_TF^1yF~{^f@?n_Ho0V^e2clia_>YK zITXz%Q0#~WUlcuDGY|b^QC_Dj|M1eHh~-%3X!+tDJT4G>*u?N zFnsv%6SPr*)%)5gf~~sj0HxZhkkD09xr+t;ZhB49n=x^%jcJ0jOSGCwy}$?2PS0+}(USo_md?1n=n|dTE@9T&wTVT2UzO;htB|C* zS_l2dz09qhWzy+;8B%tO-$=dd<_vj~b*3-2G7|&D;7Gv?fHxIpBLW~mpJbVZ!}WX?3LEP&Y2n2m{y}ZnRAPooe)wC=UoWf z)$ui)!R3`*W8;Hue0+WuP*}{BvsQpEDUP~P-b1h5gn#_WZcEh+b-m9}>6l>iH2z&+ z+>H0{Qeas&v4`U+(QV+h?2lo?#`YG{A4ay)fz0ae#!+lk0E!cH2vRkyOnF(i{2?{%kcOgX9FOdts%p{u_vqe%AL6p> z>oeRb7WH&>RZ|7u*45E*#_@=o1N~HzRk{O&7|5RFJfIW2O-)BCiz4J%#Rcb7PSzo$~lW7+5P7jz4TWoFmSA;oaY zVJfFs!UHWkZ<|OLJ_gNwH`01n2Orl;`>?9IrhstlCp*oZk{jA(vZjFYz%7n=sTkt0JxwLoL>lH_5Q@IZ>YyHXBhcVG@)B@0I! zS2>zYgq!WGB=W-`V)YbJMH){x1=;sZcnv_duF_Rm?&MYUKay6v>5rJYep9Q+jnTVC?t}+#oMk7YM+f)T81JBG zTdUFw0Q1Ln#4)#NHsuUO$Bb{uNg}asV;;}6|394FSuEtGKnhw@d;8n^`pKRie3Msk zasC+}F1SL>{3PWtgdjz5|jACF!LIt{s{k_j*?=jB zUj&+h6mb4txm|S&N7rWQlSv5bv~g2w&jU&ZlS)+?*YF~5@EzNaDbM_?)Jw|Q_Pcmm z&B5fB3Ra06Y3|96&tBU4vB~Y-$$l~?pGG}axtBV^=eo|zsFNx+%fWMdYDhsdz5m%M zUAc6_V{zA2TzgBS`9ZzaQy}3KEvI0oz|cPskBDO2c|FbhF1cpk@GUQ8w-vL|A5Yd# zVySr1j61CX66O^YOD+R6BBtxHR#&JFtbUK~tyzEHa3@P`7HD~%<^`t=;3ECz%jozV z%6MuRMUD?M#h$hdtR%aqRt^H9oOkcS<+~V-VdpzW2)jaM zS!M~s$Q0tM?CfN67mC3BLE6R8X(up=BP?Lp#Ct)o)~ zXRVFJx^unEv-jCGbH)j5wKq;Z@6J&SmK3j^x|dC)CxrGbTy{O*3YGjqNNg`WFEjs+ zTSWh6fbDMArnO59?v}2rkghbl6TR}JXMWo*XRBqqRY#}|HvJ1MTAX!5jE0M)kHnD~ zx076=Xpb^>->-yR#@~86QLyUR*^@pJZh!^+vODaN8g3mv{j$ey#BGbqI5zkM=qDpV1;B5FvD~L z!#ec@bh+&u9f>oo`qAY;MoFjM)x!?yG!|fupM-cS(3K)(QqSbRgPdwk!2`+5$vHbW zC+W4tM3Q=z_X>naiHQ#(d>J2qMtD$Y(4!>?ugkSPf)ex{pp!9A54ZA$T+Txe=!|xe zC=j8NJyIDZ%Mh1M2xTBg3sV@9SXz@iEh#s_h)YweQ25T%uZ&<#o?o=+(uUAC?2Ib6vidGaP6$ zIrsVP>-z-{r^=VFyT7ZoW-blbnV~98mbiEnzr7XV0FNyDP9cX@h3lI}m!TuV zmz-1?Y!@xlz;|zkIEt5A-ro8{(lRp^MYQUpo%LuD>Ob2tvEuJ{ zzXr3Q(BGSnblJ35%_eoMCT~H%L8kkd(TuX~bU3v;;>IG>RA3Evz**G|q6cr(C>je;lEBAsvp;psBFJkG6%F4VqZ`74(m8GF^Uhl;p zL_vo34=;ZeeHl#)a4YDzLGKINBj}$~4;6w&2J*YuPd-#Y5E1+Zc&{+we^^+PweFEe zsZ7ruMnNndJ)tgw+A_2a4W(&*4}2vfLVe|XS5Q?G9C!HFSXt%e5=EW(&_Fgr{J3wU6P6Rxi09P6>`!VZZTK!xb0W$Dc2?+@mZMnp9 zLJ!o{2Zx5PvFMD6c|!c(%!~OC|e;?=iJ4G4apv-j5+vsAwmP`q5vu%5EIFD05(3n%)rp# zvT%pL6z-&x=HJ)}i2DtHswt2y0d8ryPP)3fnwpw`d_lxHcnW2ej4k$6Ha18zZ{?l^ zU6FpH?-|6kKvbtZ;w&E;N~rk|TS7duH2g4H0<6jpXgJ``{QE_Zub|t!&~|bv2@69d z5FR{4kXCFUlNd3O>}}v=uq@Q5-X^K&-I`(*8|tZ zb1$!5zY?aE0Zq`~Ln*$zAcEQ~{(ai~i$kxL22(7D)9rtp#IMTmeVp38 z{)53(kx-x*qJ2RBa+Vrb7?WIe)}W(KV3UUayuBcL|7g%*Mo`%-s^?F9_m4#^usV<7~|}g z^+RXeJbW9;zCCb-kDI?;(NSg3=WdhbBBdc~bnqx^wG+$v^*c`V(>#RZ=A=&cAurn% z8(!Yc>s?~#5pi!j9W#IN0Qzlvnrq|u=~FMOnaDdt2WN@WD9x38vgnGx>;3ATN6PzW z3PR6&e~g`|jFJT>|?8ZoHh1Ca31TuZ)fP# zD$5lzxw`r}%5ms^+RN{clQyu3s#-_Vij2E@M8+S5HnNNrh1y?2OVUc7)9aV5nUBFc z@y+8FFFE-kX)0SdV#jHyyJW`jYvVc*MpADY(a*L zwW-)>FkV4{9m}w7_~^}jD8#LvMED;Ri6Um zn671wRX*Z2SDJzr7PP~lxR2b`=~#lYER&Z7x+};)Hl6!vlD5JGJyBophL0l5lBhC0 z=Ay=5BD9oPZ=SjDC`BLkw?8b984j7e{X?y=FID1>d`#!0(Wt6z=`?S5h-#W=d;h58 z>qd(}vX7bLr-vkjhi`uSi^N=h&nNf6P&GLqFs_Js`1sDu4gGqy`-%Yq{_g>-XXT|h zs?TRgxGOwMB=Qdaz%!jwKe&BLdYjAeyv6JF73*(Di_uE;&<4Ln8B?or*_L;FqvX;~ zYE}FF|Ey`uLE~7+i0$j`J-*0G*j?g;NJv0cKA9WJAWSg)IhPhPoJWB38maq-)`_95X8(vT~%uRO`QFlfEUv&K)W;gy!lZmV_h+~w7#y>9jmC~ zdVhqtd)oI1X`(`8qsao$wnZ$LenlC4NXi=L*g0*mgpQ6NW}~3KcbJuHCH-37;b$^x z5{@mmKS5d0mGc-cD!NnNrlIfSuQKO5?offg6BJvE-c@yE^D@*`1gBRa&nmsg-{> zE_3O&7|bbLD1(OJLbuhEGFQdeL*9!d(u6Uq-(6}V-;tDWlu;|z^wQfL<`jOzY9rq2t1(5U zOR%32>~hzbn`JkcurJ{z;x_nQn5u-&4S!y|mYbhk_YpB%9YceZ21zCTKI z*P7OK{c2Dw;Mp=_{=gqHqh0R1*GP!_oQ>D0{eJZyfCy!W^3@?@>_MPrzdR^k8en+7 zk@yHmAm@PYpu?kJ2X3cnWw}l z_jBv@d0EGOn(QrJ$=Me1xwZW7byeYh)#QyP0T&J{e-wm9V)1tzF1t9#9A-71F#-X} z(4TvEQO%%VcBQQGON+!@-aUqp>$aS|XGfhEXcmEDP(vkX8Ikb5S0nMVKnj z9Vrhy4DItO4m%!KfD-KA;{y(mk9R&Ho+0;o_!5 zWUW4^WN+pN)JkYivTFnM6&oDlldc>^iBLO~8*MW^N^bYQIPNoehyTJkvc2q7UyHPKbVTa$LYZoP)9hYXK#Sp9hxdwv^_JnpU_zQMG#!tyB5B+5=G z*Ztr|L^jjl#s+`i54$&`K==+@-OkP@Sgsjpl|FU^#mB&U@@*bCO@6N68;Iu~UO&k4 zEW7(Vg>ITBdR2FAt2JdpHxTyU$H`(XqOKe5O$M%ZLP<1M)_LW zwWnKJz(C&3#Mt`Ltd4LcFmXVJM!ha&dg6Q*HO~9oRB7&Q0hlMX5`ThL%6rVvdb&Mq}S~`rv1)6z%J!^3m~=H(~b} z!RFj{{4>9;;jNC{IR(>x)#~*3wZO4W+2fQL-{5_#^N$;^wah=F_q>VK&U)(5VST#u zwYDUG#T>)?&us=PrN5vaB_oht{`OascJr#(i`C!16$v;%i!&Vg(B_}Y;;$AYPWdXj z%{Zzrr-J&-(+IiYXA@{6Fsu5OVG0V$|E+)iuAAkY_jPqrFqo#JL-COYzX{C1-rnDb z94>bFwf|R5{C(qHfb@Ern)0Y(9#F8ulP4_yzdwU&sQ(wlqY54s&|Lkec7%#c_%0~F zfER^^24=o^^s0_~{5uU~Kec-9oOatF+GCb*pc^S$7Y z|E(YY?Y;fK)e5R-_UuIgw6TPRcY%p@`+%bgvBQuMx3RG?{}*=sy-5GQWhU6>U^orr zcrD}DW8{H{`pp}7lmGc-&@BUf8P3FG4#F@|yHUBj;;<8Dp$*!{(%1Hg?L8b+{JB&a zO(Up=Kp+t+3O65K|K}RZg0U@|(f4rnA`V*{{C$2<(z#>80oIPFy-_38{%$^o*8bC3 z(#8=h<3Ddm_L$@yL^Z-#_E!gTtI(Z;RQLHEN4>_nP2cIWeIHL8`1!mKOOu1S)~HHW zc!f*n_U+)%e}o%` zdchaThWZl9m;V0yF73#7;IsSmDbf2r>WBZ{$v4dJVD7P}=K&Zk;L#H_&i_;g>o?K2I@A!jl4kb1MU)BdSFmoBoTzG zss*<4tkOjCrvXBs2m~eq+GEF6PYp;P65>IA?C~iBz72Y-$;PH&QCZm^ATP3 zd1^i*Zy25ct=A7lMKTdxQ7?4C&u}_!ElXmct1Avt@gt}(9YDPdob+Lj3sKietdu?i zyY)H~cN7ds)c}7!Jaqfed;mpsNrb@4LQgWxX@R;Hz6E>*5SEKS+gZHVL4c?NAw*bM z7_5zTQGip_7|+zLp@cxj;veQ8ROl_Lwz?0?qc zO@Zryd=Tgf<~<1224W&BA;H|V3tl)rTh0Sssli0QmflKy|j-hM?j!?+o z**!Q2rImYv6d@9L%f!{of>_{>N`2&wosA|2#W$qWE<1qq7aolg5)YOt7?uL&QV1%e zO!58IhUaZ*e2OGDrThbq#8~L2NskBoceMxVfB}DNcNfxvv@|s(A?^b4>B9%YB03V1 zrtWT9Iy!{MTH!Wk5}`FkK3p;dqFse^Saa6|fas~1f87Y}z4_VM94Lvs>+0)GMH6VYHs~=!0*Up#BtIOQ<9v%K-d_3k$p@` zw4!;-Rw0@Bzl)(!7nT8>U)Yx*C7~qk*}-o^X!JAjsgw0~5)FclcX4qMtQgSu#c*W= zbEL8|E^bUt2G$_ZTaZqp2((do`T;DAEo_WI)pH-jD-AIIElV*o!{zW#OFKb5VzrbX za&uuq4;{42QYQ1YUQ+?r%*@OThs-ssWP}I$-is^CRJbQc)Af63Hc6%hG6WDR4xS2a zNcxa$<3|{3YGNosY&XV2>KPcZO@4!nzbOQ?3;ZvG)k26FiSm6u2@x?PJHEFWyoES# zFNY>K@CQK?{B`UL*#2PWgV(O{J4kPT@fN5pNTL8gn2)zNs^!{tN+2ieb4+1?`=HKk zg{&V-+vu|gkBA6mVmJCStSKPkz~7w3Y~ts40qt%d>}(J;!2AzKnBZ{hHfc^y&b0+0 zdY#)-9bpqfM5|lB@Ny!7#5nP zrOk-!&;v3I+JitYMwwMaT2NudaVh`jdz#GVz|IH8aWKfDwkGWCZM0=>7^q-vO?~CcXZxR` z;9oJ}Dv8G$G||iE$=-qchjcMDon^b5#3c5qX@1_KJ(B{MWr#oyYFso$(g0& zu6J0r&>=dSAjSDB@Px8nTL`3AwGJ4&e2UFsS|H@QoA9TERA67uH_4n?T%@9=#&KF& zSh!8f#Bg{|U7f@imw*7X+3#S|6E;StbYcWizf_@Q@sVq1tgm;f8uwb+P zOYx%4esxfZ!!Y7%xJQMB_+9u6So3HkXdX7Vvp3-2fJ|&u_yi^;>627XcJmmtUQnpFtU#LS7NzDTCtBW?6{P#fN(vq3+G=Q8sk#-{5t+`j(5~9WXCh}^kUlC+1U{8%B_NHF(nS{ zawtUb&m$qcGKV;WOachjy?d4z7boxuaT#3zOm#*L-Zj8Uu2nnt)a_d;*X=A6l0Lc* zWc&Y~^#8zz{})O81!ZsH`Gkw>-OPxLKmPz(IB*^A+WrFtQIBG{@YQyymzG^Tii|_( zH(->orZV$?j@W-O3F`O!<(^S@6V)DgrC|0Bz8M_2yP})^4`J_ser0`b{Qm+EU}xb$ zLf8O$Spjn(`cKN2WL z{U4Yf@&-sH&VK1|De+$M=c2ZNlVk$)FekWtsr3q z^h^-O6k!0&_O0Q|MyM)`Em{D$dRvK1(*mE zA;O^wS_=S5b>6|SMmVDOt;yiHh4vNLM$`A?t6;zu03pDJrP}l}@@8hgpbdl)QhkAp zDOqS%RaCsJ$Oo4Z1Wm4|&XvP()D!c3<3?Yf-xCt)Jzwfzh!G%Mn8Up8c?r@}A*u%| z4Bh7d>r6PFFT!z9fd70^nUR9xp|*C%Gb_43QQ_e*M?t1g>Gdnfe*%027IkaO7){h} z@@L_$wzd<+YD3Msq}z@>z^N7d`kdb4Bw4afkVg2oTfy*6JHj=G75hbU2L84UGS zy?@_|<2I8O#B7qJR^Vi5&`C-L*f>+!q`Z{KCMOuTwiFE$h3)^^z&Hl}+^=6Rm?e23 zzzIXdY~$uJL@7eDMkx*TQyH%<$RUA$Lq<(Wi8eU)w7#;EN(j^AO-LW=%b(F>T?>s{ zUrWjN~E^;oCljn(ya3%@9eaE0R}7(exWd_1DyfuUja3vitW5<||}QT@PG zC{bX(rlW&{l{9k<>W$tsl4T3D*iTf$M4=;)T!I`0GQ65bE6qEJ;0RMAbQxuY1R)4_ zfsUd~DBJ{b2OUnD12L4zNx zRiK>!RZ;GN&vmA3dp?NWf_lRzQ{O;G#~OqgD=XonEm9k!&CNFq; z=l#@wZ_fEk{!>`HFh&Fu8=J*U@%HU|nNmlCIr32IAA-gVDGY}#^g#*a;ClhigudMD zHFAC0mX5&(T3YSi?KLx-P=`PQ_F3ckTH_W#WDNN{$QFRh2qO(!CI&jPX&{vfst#D8 zBdr&HuV0)4Dl4H1i`AfF1@U54Xs8kx@<5f!{}AIfWO7CL)3Q?aw(=90mo%1?Y{4%A zn3BQ)d2A-QoxTKi@C5-#_MW+Dk-IzM|}asp!o zV8;huAoabeFqT*o4NX&BF~N%IJ{0lWXjXF+x{+iz_j`Rr7AZy$*gxS*zTh~G8lJ6( zCLXsz^%megC-KP$R>JGdCE0qA+|&x27$hX2p@nsKc8014-gjOb;rZn1h6#yxZ*KC* zV<;LoU8O=pn+?Vzg*prpqT1)Fq8agC$mHlO^FdbHa*b*}DxDmTDVUi8O0@3XWJt7u z_?aN;-Rx=i(6k!Pe$~CWEB%BpS!{RovnQVonaC|-us+iKV*V05294Tg4NTbKgz0YE z1+j@=TEt#}fr|m>q|;BHVV!8;&xmk00uJ8yyIwbA(0c?0uYVI6bnm;dx!C$^A;Am6 zv0yE)@$fkDt>2WB8@(yzuWzyeL4@kmx@6SUp&8>+3n!V6-!TF&pF7S3@V+T~U}Wg`DQ5H13MlT(4{71TTy??afEN5Y!)_AMJW1CR+2 z2Xoa5P1plQYhegO*39dioKlC$T0mMJNJXZ~k`mo=tLVVMR;V>VP)397D8YP)wF3N6 zpl#|`QpuYM@Sv8U)oe^RGPWW*q6Q{bL&j)mV2me>1A7a&&(3uE)pm`j z2VJzck+IHHPEWria?%u!l9E4$hlJnDq`I9c=q8pqza60S{MSZ+L${h>8`ZdkEwrEP zD}YJNfy(nER`-Fk!rRIVK504hxeJIoW}uumm%AMTup2bG1$mW~`!L>v{~Lfb7=hdj zd7u!9V}1ckS_6|Vh7(OlH}?r-aR=&bQ&KBK9juCAb;dIx%afjohuaRb>}MXFPtoFr z?^@$ZohCp`jftmt> zU#yEp18c_{o?_KX(#XLR{v{Qd<$&S{7JffPYXuyOTZH({&WQqersUGphF9wF zS1qUd`>Jv2856u8qV@z_zOF3kx+V^IfN=a+X zE*^Ln8)N3_w=+;DtbdHS3Y)g$WmLV^+uJLbE>{mV%KlzoQy*v;tgNl)${}W~gN#ZJ zLrmZrEKcU*Kne(9eSr~zLqJKxPK&&M_imbmE4$!LRi)AP<#vQ{g2W{=9Gj$+&#cYO z%{15~4Yb!b9rr=GEx>P*-gFJW{Nz+e48$XF_h@b*>u3ZuE)6wV5io;O)q?N}BW3)o z0hL^N(EDcoC@q>G{({#dMXHFRrZFJ+FvbKdT0~(_Ca@F7VioJSdPp+I!VMfpvvO27 zo|u>zJEiI*TAvf2t5Sq%Iu2-qk8)K z_$)P^n+%h_?1W^KV4{zv?PKtsz_}D06f_uKLcWbdA3>ijfT7T+7>aHSn|HQQuBo}K zaw67X*zKl!Zo>0;m^iAew1us?f#n#&&wnV=$)()V#o!af{pn{FV-xZlPjqAFhJ%Ul zF;m_&qhU7^PxhG^Of6T6&3(>tiJ8UU$A2-!9OH>80sgPAeWd&lVwQJ5BSO8Gfzq@C zEZL>xk4*^Htrv&3V2~I^jG~efD^b(;Yzu{3B%&q<5Cftd1rvJXO9Ypbn?om2)TwnS89RIbf{^n4m&6)3M&S9&SHHT2b*BGia8~A ziP|I^ddeeRT@zJwB+Eo{Vj{oQ9Ld_E0E-L(;)!|JO>M+5NE0D)EILE5v=IiXS1jks z%9*Rdlqy9|0KQ&$ffEodJK9Nxoee_Al$-b>Ez77V6b`^#rrbEHSyE3Q+`1KzkWdbp zWeCFfke{?dGb$y5uKBc0^=9o65YL32Ex;=n^7hResMB$d%pYC>#a}V{+|xvSTbMNp zh$!$L;v*E*pR230e=xNGA9w>Ix+X9A%nKo+C9YTZwYFkl(oGhE-$TYTo0M))GG#$6 zW#mhDH*>uArR-ETqb{%t^x<-Y@)@$kaH+G7l?ZD^%f~UTV6Q=qXim_wFfukKPV-NZ zd+Kqma~Bo*??bM1Gpi)Rog(A%HV!FeJXKxruj{A%0n;M$%EeULX zU@C%i(!GGkic;x)W-vlVL|_izZ3n_6wc*(UhJ`@RoZ==Ewvw4sSw%(rXov=8bB10c zYW@aL9#LA2+l;|9Z_ttv_kd|mA-pW({0mc8l#>&0A(tsijYd$QmjAU#JX&60KmdcR zZ&WTsVe;F+ZY0nX6NfEd0l60M-l2fV7G+>(sd?uNIgQ6sKfTlS9)5xd8cEaZ_c6O# zbW(gxoIurY07KTX+hJ19`nQo0MN5qrFntXy7)T0GP+8;H9MWOXkY%^Qt_^1uS$1qL zh-zmnnxwbr<%oAf@1{x)Ze8~<>$M8GLEyiIAtrVJ2gulFYMgT5O`C+2Ia8u=hhSQ! zS8K5Rv$GMQ{9=j+%@4`RZ=+P}jFNIN;vFj-1X6*&OZ+a*;dwLng986p@)-996X?zQ zTig=$Za{QLwddy4;h~q?(R4!-9&L+VBKmP>8-tbkZyqKlRHh>43pliMnB*jW;pGzQ z(i&c2Ce0BP7pLic1rv2k5b5ohh{V_$;rn%vqHCu}0eL)_imW5d*ttJg_se9JF>9Fb zm~f!8dJ94?A&4^wcOAkxmE`0Plu~XbhpONro0d5>^O*dCBzv3m5Ye@*QoWxvuH};d zBJ+-vN7$-%ZBx5qsQw_$tAr7!#>|UNr3JWoHwswmBE-XOVl%qv!b2z23Q>&5Zqqw3 zWaZGgg=yT>)1%d3kc6^XAl|>XDURm;7TTLI(ytKr=TCWxbE!>llz5_T3sGc|Y14ca z!Np3&S{A4zAjcxc1ak?l-EYI0kNO&?_wkf5ExHs3n+b=)dA^I3z#NJ?ggOY92@Gju zQ=0J}L&=7p)ZP>_4*Gv)fiB7<^L76?ruFaj=!a&y&&CR4iE3%01dIaoG&G|Udo$9Y zw9IWldSN#RB$}yHW9pq_?CoO=}w0ouowM+7?{jD&bfnjv)-Y{x@F zD<-k^<~l4*^oJxSXf{nu5#bTg66tt+RZ*J;UBJn#ZnHc)SmE;3TU@uTZyI9^F`H%Q z1bp~FF$@ z#?YAf3~|#j_*Fa2RpqgKyxkhtq5ITMhv13>Qlc2#vGpl34`>9;SSM^{BO*y#EHvGx{FQMO&%@X%d?lpx(Tgpv}X zgd(;ABAudygLF4YgD@bagrX=7(p?e*igZc~DBZ%pNAD-z@Bh|%*T2@k*0b(sK$zjW zuJhdI-p6t5V_Pl#_;KK&@tce4U6Tf!j}N|HKiP48f+X>8vn1^KY2Zo2hrB9-G?1t( zSd!#4!Pqg?CEcXIi?<#e7+CoDFn-s~?Xsu;r2~vD?VY=KjeGQuYzpuBQJnQP5g#R* zKVEksC?gy$7Lx!h3nEnhZD59=$QbzT2%QNHC@y?|2P>;Zn74SmCz%F$F=V!&|Fjm6 z4BcA5a(!}gBNz;3cW`5O_t$24YMp+)o6cRRG}}xYVOc;4ws_MKD%<}cSyTGP56wiJ#5G+UXB0Gb(ZEEKW^PpFOLM8RErq zl-NrC0P_vH6e#cAyeS|eVm=WFquW^6*ubR4P+z|qCd5%uMFFxkpZyLdNY}Y-Kl1df zfdL^eUI4H26>J8Rc8iM^D%^K%@$md%ZgrZB!)q9q2$?9@S^zx*)P`!AhWFvl6H`;u zdhF0cxCStz0*)Q%4=9X|`wnviQy$#86Oa#^j=5%}(3EHe=MXF*KRm1g5UH=R5qB{K z-2b5|f|K(U^hpS^tI}nGgc;r+7!UvoU=Aa@;v(B3)neNc7Loi1*gUBK0GCrp$Q=^! z<8rV(j)OBE&;Y!GpD1D%K^XpYh43BB3}#saZ_6_SH^Ht7o41;om6Vp^{XJu|7KqBCqEl}YU%UYR z2WFl^=bKL!@Uj6YoCe5M_?e?A;()S-QP#da;JylmGPrV*Y@oQNJgX2Aukj0}Wy0Wf zkmUj%wst`q=-dqX|2zS2jf{y;pJb>D0N40i)!NhdLN0WzSJ3E=zCMJ;-){N!!(iK7 zZ^NFOoIC>MSN_AQAS!^L!ORj{kh0Xd?@}Tt`0cE$l#ewK#Lzy61qhvfj)iLZtN{KC z1YPSZ$p7t8D6G#qc$pDQFGKm~(*mescV}l6Ok@QQXjr_Ml;`>U%x2+KQ@>!cAdG)j z1aoizagZy(EQ9SlPCj0~vBD!7R#qUyieWmXkId)G^I%v8wa9-hEARMM0G`p)(BPcf zUSG!v0y{17sCO_A6vml>4?TANCyaHUnVIpe(>MW5G|)XO#?j@arKjbgz3tZR+YbB} zE?~vdUCGgafx)CDFxT2UIKXlO0CNJ`44c}=$n0>jj|*77dRtgLXc=~Pc0oZ>WJD(a zTZm;!N^A(v+4dOJLXSA+p#J9U{K5heov5JT#bcvIEUlc3vkpiIe+x8)earp$QOzyo zG%YO(H21L5xG3JWx+M_*8~3wb)P&XTF$Ou(NA`b31_vt?*a@D5a{wI zzP_40-B^pl-9%5NFHPlmCq3umb#!$};uIgPPBDBHp56qkYIj$VYek*lu-c6BbX#b8 zrIz=*UX3&*@5s&zmpmy48%Edm-^eL`rJ0&IR!<_9te?gW1l;=Ft5?R0wN}(mpo~@w z-c@8?NAHlA@5+V=W@b}(cmy%Q3bvR3jpqv_#lZoDpHeX0MWIjtfWR&VmBv@^U4ZC<1d1nlu^fU^khWm=r-1-n^Bb9&?cN+I!F#8015X7EU^q*=6M7S$U^C zShr1^>gjv(?F6S_H~`pb1>YAD7KW+srY0t6!{S#e@Q8nnl!BM0#{K&Yip~xWDsC%v zs0GSf;7^}nOiYZ4={q58U>^jd zWe71u>6tB@bYR#%BUs?o9hznXflLi*u5h8J0Dxzl=M9FF8R%9Sf_0WOw)TmLFVLq3 z>)x0|bQd!BMWWA@!?QaZ#(gZdnh@E^gVJC+)YSh_HDh~OKA^979gzkSNCM7j-grjoA@JwcdF=Ok_ zuSqJ#7GlT^t3fyb2R48jE_W|qCNLcZ#@pd7l)B$=TtV3IDiX}~2!BjVhTv=04>c)L z`f5iHf&H|Q(DRC$6wa(*Yuz5hg{?Kxp8D9t$HqQ|Q&`J=4d*i6a|qI%W3P6%wt%+= zjaDZm~*TwHQYML#4L@4KvWZ3wWHvX&)aM(mG>4IYs7u29lgGzU`*I*dB{|3{{vi12ni?IG9oH zNyZ+}Uwn$%!%hp;yCUSF>a4GHE481#7_O5Z=BaTW@L%ZBv#+q|W+s0R%NvkaAYF)1 z{=DK@AgWmf4yl3Ff!+Z-WslK8{CItH)0A7|%9V}n?aQAbvqOFd9$B1_y&3aT`a0aT zCJrA}ZEecJW01NOLM?UcQ12yJy5Xw z3}Tf}q#05mz3I}N($puoB)EmERX6QX^1x#XG!VZ+qM{PvXufsJBxe!?JIK@HZJ$4r z4h!=0*RQVJ_pri|nnBfn1Su38y>6l@{>eL07-ClYZ3u>aSH!aez78#rFS6USTQEa6 zU41pHcM`(@J7xVz1E}q z6yoY|aI7=Ye^17(70)wj^;4F*G$@QcjI+GC@XhCC^N8EnKfqCBg0Ok=gn$h?p9+Yq zJS-^o*$AvyS~OZ48PoudgZ#?{ChjP5RpYYO5BNL-GPIYRpL#3Q=Xsa!2yw&IX|!)< zNVQBQOtAp#XC&Eq>O64-7J@=V+{95i607@tbkm3Mo3WdZo=>_FcrS~0+U?cn4V%+j z9qjEPSP=#*LdFGb0m0-+Cp8jHRD_o;sGiC=|72+`fTGg)CnQf1-yzfht4fx4NwMCF zyb|bk#(I0Y7T+lq7nv}WcrCzvQ~C(0>R=q~L>VYwRBZ^$?|x+t`HDMrqE^i=1)T$ioJ_pxC90cxFTo;B&$ z)Ox~03^+K(z7@~OCG9J2O39q5p-P&1`{9GgQ!6{Wwo+AlWd&mdGaK@Kb2YImrAYcN zkrg|3U@cxRMAtm;2MzA_GO@380nm`RCdJQBT$~7u;5`={&dd*-Zjx`5(e{nLpF~d< zA7=Zb`p6K&_QGm7`e5ME>)7XC{~evq-j67n)@i=5M>~%5kV;)#{o)lCa|H8+z=|zZ z+hKBS8GCT;V#rvrC6yX0VdOegqIW<#wjN(W5k5`od+LyeNxTf4PQ>de*g=FA86Iv% ze8$_CS0)}t>jc%;6Drobko2qa6NNyd8aqDM6sH#4@6qOF{640pv#odmrYr;oTvCz< znBCP?P0u*J2Vsl27gpn%+~McoI_eDH5B9u|e5XlO>E3}oejKb$BRZ9AP=p2e;(wEM z-Qea0whwX3N6#hA*u(u(n<9BFdoT8rZ zreoZ_jhR(?#g3fN5#j7N1O#Tu@j@Mg4TH-4F8E?|L$~#mZK%ztX7baYoXD z{OqascYzmqJ3qLqhPmSG-~9)-C>a4R~3y4B1PzW~Q-D00Ke$AXSg zSS|XUjA+|yWeFb(e$C)Crw9cRJV!7pC9zxKoOcM?_jPI2>TxvqYJTv1;gT0H#q>PZ z^*%T?aq&D?=_~DQk6Tfwe1=RPk#HJKi5JMoyYKEaa0b0-qIsRM26i`Z1Cl9}@#fHW zL{@@rS~+Bh~W<9#E<$$TZB$M{?!chrJrVOnQjk3SLXT( zRNnibx7Sq*m-pSA8pmp4u1mssU>GM^GEmYHxSOS)2Hu(o5lsxZDds~cYADr8?#k-w z!x9Q_(DBYILk(eft6dx{GzVTmMu@yAQ6r&rkvkspU!l1qvn1oB+-`;mU4kmdrN-GF zJ3-4*Y@7WG?sieXc=2t?MM7`%%OG8OUAkPRW7CGsot^oIjLCt%u5A715vwyZc(;rA zlZEEychq@p5@s0=S!M1KPlaxWo1gvJCZ60zYPlA2?o1bMoraQ;7$TCgNJ8bzieH3k z9R8yzLK(7}X_5d6CLi-j?~@BzxEF3*uje};-boye^K)nMLbWb3Tx#Xyv;-dyYC_S%9m!kO1jgtL+-eQ6?!=Oi zTR}dt^FJTOQGC=Z6_n8Q=>b53QTNddeuJhT3g*VJ_!*nylelXOJu)@?G2K6Pj_3A? z{3px@NP(v$If7SR{F>(L$dA#Em{SWf)@M2-(`bhTI!Nq7?|PUDhqU_mAk;|HmPgCQ zEQtn=UVt?IzuyUC90^x^e z>&LnZ<-cBhNqTen&gZFF)&ITZ zdT8w_{VPZKWQDe~*d^_$84q^($8ux;himoE3-}ZNekT^8_s$j=jET@ zi{-{C`R4^m2G2k5@!#)y1rw(5b8+16(#7Ugw(Y0~CkwIJbp~T;_8Ux6a>o{LwY1L# z=IQ@Z@|^IOzgfijVm>OE>Nl?z)%d61#$699IN)6co(Q+MCACxv!7!4nK;cVPPi1;MHwITGbumJ!{MoH=P^r@WJ(Wj^=46L259bi1x<@ovA z`})@5SOLZh66yI5SkEAQ{1hecG=`mb;CZ6K^YZ+YO7~q7w(Oh6Mn+S>P$(!UfOo?D z%8CaJQ6wS?hE|v?VGj`f0Rs$#43D^Yp8?(|TMaSY9vki!Fo$5&Lr%hAL2ebLsP7kFX5jkgUU6TpZb^z)5o)CpWhln1ys`^F(gfZ5&<3kvh+e zS*ik`Ye?HyS!bVtFo+MAeqFR@NWOE?h}glFurY*#J)Ei5tiifj7X^B0xTyfln)IZ| zrpQ3S=j80{{TF#+ zEg4w`4^BxuWpns2)Z(!`S9?Hr4tOjeNBO$Nyhd%|9kGC0?}JJKNVunKQw{Jw0O^)D zQ~?sk0Iu&(=RV(fCUMCk5=Mhq>X|(?JD4Y^Lc69kDjltC_l2`QMHKjSw6yj6bXg&>z&czde+ZV<(p6*Hg0yhInj`>iOJ8R^Kv zknfCXc*DS?%qsT>`O+cNUOtRwA`k@1Swh*=RofR<;>9D2L`TwIxY00#<$M|-x78~C zgw+!;1^Mh3`7fcIm&0IymD$!}A08n&|IOF`~lTear@Q+;Jqie9y zI#2jLZI1B`;xy=Rc0tkuMGYRU3I$<%f!}ZoyF|XLG%yX|(LNQDBlw8j`Fn-{ZNDS7 zST*$Az6-$ZTFR22brv9>aLQ)+U%C&zF=%Bu$3-LnQBcU-e|LvuMUo~&^3v2xyIM^H zUp}WH6BCZK?zc(3=d1~F2Rh>gnp_J(pQKGI^HMWNE3|wq?xB|E%n_)i(8C%Dj1tAH z>GwbAor58B8%f{=B1W14m2WznFEEL0&F^-a;;Ap_p!+hDuP>7c{}{|?I}>^W@=#Nn zVfkX}%qBtQS?A?}ZpG}G^Hpck*Lz)h$XHD8!2}v3Ma$17&!(C|oPzr)sjVvny;gQh z#o`s*2h;rA2J+8vGc?17<`~>Wu~Fp<88tNm^K4~^=A#5H+k47o(oq;3SP2z6+1&A-^MTAy(iFqT1Z20v51K*L5*HRzNZM_+J`=bk6MSoL&%tc4z$o! z&;=u`TQnsr2EsZby2?+V*7(K<)5UKM^?Yx;NZdAM4|`=T{oD&Th%WeM%92SE<|Zu{ zUuyPo$ufbHWhHj3O1C=}iXl4W67yYN@-p)G9T zfnZ4sI*XnZ*Ns`tTa<{Hp`g)>XwW+VaJkh)rPO?(H^sTo4Y114{hmQ-<1$w z(=F=?$5_&|! z&e;rf;aUwPKU98FhO)}Jquh*;nr%>XGRskXI!g?TFWuW#q=pHRLG#e!iRtPjRB~Mn zUp7jXm!X2BM(SU7Gj*<~?1h=3Sr#m&0_LHWAOH}4kaV}4w5_9Bwe~6NQG{OL>mBZb7nY(xqvy3 z<0Pv~>?DgP0wKkzB>bRk1rkx>GKZN~o z3}r89eofE$_T>6*K5vjfbeEWpsM&V-GIHIc*4u7=@bYhNvTPD+GA*6{xiG~?S6HOW zCsEAM;-O)QRZMAs`sK?FwF7fWC#1byXqfD2v@3s!54K#CBi@=d6w{PYNHOYG(piYQqbx`Gb=;TIyEa0dW1b_45lAl@xe}e0>pkO)g+Mgw4W@DvE07o;{{ADB{P;Omy>kuAItEmd$kzvAocL*{Safq-yoP7Xfh^mpWyu+zgJCxYmLC zSYy%r4*7DKofqv}d=5={r*%IOe8IH11)ZB9#`$6F+`}IJ{W^;~vvS-Kayl;l7G4s^ zek{%+g!waa!Goaz&__r5C!2_D8Nuj%$-T`-f2J3bx4ub_&2HGcNvgUv84q|B>CZ=3 z!nqBZ|MQ55z_)0p2H{>~D1hh)iMxW`s%?lCA)8YqQ!b8&bwVd*z|xFkrKA6f7Oxjr z(n36`IC(KE@=nXj+CV&)5ol$D+(gI4DCcITnlUADVEWV@m<^i4M{=WnGb4k&+B_+E z2x$y#$-DvcW{t7|JVJICH6OxK7z5cMbBmhl?6W1Edy+ zooFVGEQA?z7Z6rR^Y&@4R!u?u%$j^4lVMz00r0ro z#kvqx)G%5*hLm3D9=Q>ZqeITcKvI~?K>CB`?cVUayF&AvImw`S^h*#4m_#9@;#-ID zd_DHEv^C+(44R=*TcID!fKaO3$~yleI89a{ZAa=1)U-Yc zStn&(!2yoT0~A?4vwEeEwma+E;u3J=B8cOl1`MBuoMsg^58d%1!LwEA4Hl~vf{l@K zJHtYg7AP04B}vR#QxP8;x*d*sQA!befsY)B0B9CY3*JEtd)EYaRNMP#b8(*NezG@) zf*?uf-Ti!R#m(jM-#mZH(ycx5Y``2PFpJmxyE~&#Gzi`|H8x&URW;08sLS=vdJ6K# z?<|hTf%nJpdzFOlH=~wGCUF<OH?^O6XMd0n*D5pE z*_75+UGZT1gd!MmFd651r@!g#BHP!O3xH4{M6$0wv>KQ;>FfzzUx#ben|*DnA9jZv zUCJ&l>hF@agLAMGGvng55WksexSxKCEJI}fTI%Y(kN1wcZ{RroxHjJ{|M2I45{Ixp zU=)Ae@y`VlfRoDKi^N{dpX(&czF`v3V`kdI;C@ZaCW z-n;+tKL5Gbe?J=glYj2`zklVQPdC@P5E5qLid0%)E|Zwgw8>{@j`kK$Wn^IR z78=_UvhM*AR`%-)V$GA)+b*3S*6#eY@X!gc)?7KK6y5IJxOM2OVsZ6*&7o3`bZ5t> z>h&8Y89@>qocuM$LS1zsR%6z%&@zL<- z`KiaHL5|&}m%e?KeO__@XkzQ&U`Qa>qi@{1)gy}enoNm%w%Z%jX=f3=4i~T7C%cO% zhO-y6vZowM|&QKj*gcWW8;o^6uW7@(DaOg=+WRz;fjDJv3&j@9a6|uiP4x z7)*SwwL;y9oz6v_OUc*hOg%mFx6XdQS>q5}eph=zaQ|6P-3F{Aleub(Br@t2JjhK|M{zv+!rrGrmBzHX|O*ki_G+00pw?8wh@WeuR_m?{r^OKxQi^BqTnu(34(PuFC zJXvPfUzJOAJ!63tX3ozu7P|4ak9bk`_mi84QR>~i>*b9SJ*O8BcYpKt*nc*Z)0Ix$ zNT~EYx+Azi(r1aOpnso|$8#eU6D>}~`728vqe42lP6>-6XY0JjFwyJj(!8o+-tfsx#98RR{7#W`axE^aYg?xF*Z4TuJFM>A(jw&wj+>E^JnwmI z%C5$QalDt#nbW_DMeCjsi0HPsr99psHK^nj@h+bni$Z;UI#PTrJz&iMI#)ljtG4ciU0U`>6RSWv!p@#~JW z*h9@3w&F`wt}X+%Keo~*+@AgLRGMmy3bj6H3L9->_3bgU9Q)?g1feKr-Gy?zva{4> zgKp4nBd^W5H2cXox=u>>^YZg^nPR?q%U5B1sHU=4mM^cg7wa2EfMU@~1 z2z(Gco)?MwaiWs&^TiK~YO0>8ESsj*(ZX+~R@o0mcNDH@Y z+4_T=FD^>|(x~=U`cQtzXBlGY`D9yXoX(!Hsa7HcKb7;c;`JH4n=XYs`h4}U!q zwJe-6w~?GJmif6)@X70bgA-FeW`+;1IwU(Vu))2a`1yS5)6-Eq3Mc&H;=kS8C&i|; z3I$Jg%2e|Yt*2!9PGb0j0^A0U8!F@7KN>tATCQDc+;p!!w#~LV8Xb+S`AI~W{ags; ztX-8NaCbae$0H~_qAH0}0_76ff3Tj&bNub8cW?z2C)Zc6PTHC8e)RbVyR0S^VS#x% ziK0LKLdYsyODFA@wULROFJ52hu|$haYrq3&5}c1sw5*}$J>1Rvy7H4CKYYI?qPbb9 zhUjIdr0C=eD&4v{0=fP{D@kZB@tAE)ma%A{5Ptq}JRu5s zVvRbfxVRvA7U^!B{9Ettm9)*XGG9FEBg#46pU?W;iBTh=DHq#iw9?ZqnH;I0YP`zU zvGc4hOa2vG+ve~)3cGZf)E%*uGYzQ=FH6pK$QHYWF9dF_vp-jlYTRY@@<@DmE6n}L z-d@{vs{TPDY$`gi@G$`2JJ_{nzf)#TWa$@Dx<(j-dBty84*MmoM|VsAcY@f?%}lCa zZ5Iq)p%U!{bk2XD>LXt($&nYewKabfdur;*NHxJ?t#^j|oLfmWV+a}f74=S3{PQaJ zK)Ke@Pg5_KvgfkyPFGQVdhvne^uwQDb}xHQZF*>S&WNQYpX zmm4kBxC}@)^t|`|gxpM@P|(r){5%#u;op&;(!afz!(~%QvfXEmLy} z?^jFOPn@s|9*neWYP((yTvc3atSfXr%!=!Dp3C21VrDy)jHBRf&-deCQ6SH|+-b!V z$2mcX_CYgI&x7y!Z}q3DvzKgKd2REfQh(?dJYL+nu(Uk+vpG13X8L2x`%5erUZA6+ zBlWys2WOuj@408^*tVpl=YsCSpu1}F-3OjH)M}0{Zav)}zS5XFKK6NV=SP3AXjT=G z8+3#(l?&B%6d%3m?pUMx7$xIz^mNe6dOG$?ll$(+EoPP-Mrp~Ah4LtQH!8-M1U4Vy zTT`wh-g~XI)fb+Azk*R&THGkk?iG=*CW_QLq?**o&EOI8PC7haMK!z}$Vg?TaEZRJT0jms^;>RFlB1=HCc{Az6t9Mhsb zibpd#Z_dwn2gf+1m9sLic^$S%c`Sv{w~muv6LbE4Y&kaJ$x^eLrOhMU8oqpM z5$unQx_%>G<1ed|K*^$LQ04IQtEerN!In7zq(pxi9c*g%_ecKZN&o&PWR8Dd|4D=X z{P3T2@_+f}!HZec_^**?b?QvNm1uHUp&4||Z}3HQFVZ#b5p?agZlcju8w%g(SWf+T z)*{cL66g9(d%pJAm+H&($p@`BVNp?a9^d$+q*h)Xdlo^;^!v=6;InvgyJuPQpNW>@ zx8BYBX4X*jO{?}P*E7elJ_1ZCCdD%Y{z=Mxx=1fVFL;79@pNvE_64GCw)5u|zLx=C zL_je4xcBwUm%GcErY5sNe-beOa{s1#e_s5}hyRuL{X13ucMAPqxf`S~e>1*6FR%&m zpBwkD6#0MriuC{SkN@??|74;N2@rC4cv~Ca&fNbn`8vISfB!hxX*;>puvRVY%@@q~ z(>7P(A4(^&_fIL4SG#S~;kM-mohnV+te)!&2L3|*k0k6m1a)H`DLs2xX7eRSsPk=} zMryk3^Lux4NAuKArk~%>i#wCMDd{m5A6A-7oRLo2d)uPN8C|TQl)HI5Q>7E`+V#{@ z<$GB@81CST?05Y(U$ZgsNgE0;ybdT?BNp(2qF&0mzZ!#gO-OZ1eBWn37Oz{x`OWLm z`A2v2br08XTXu%^?=wkStF??=8-a?+;XuD4Moy`QFjCksGX|koZof!RIo`;z^gsWL zU0T>h^{+(v@9Y0BH~4Snjm_Hs{XDT%&fhouuR7(Q(&yig5qsPJ`!W7~pQXYvg`O(+ z6mY+ZKb@NyKtvUp{Hed{1`xJ9jdS90=~YwN zT?*eEe;s>j^bCrv1vaJp`KW}yo9xe0`@6}of9e17O_IRiU@^TCD-dWvd%(4QTd=GE zm2y!gxJ_{I!1~s(^=9V4Mufk+i#nnE1Fzs5YHEs$DE(`W%RdBHf z^lAZf(&jJk=kE{7LCvcs5?fzAY$vKK051imD1uSk1avom_;!#^t${q=p$g>IF7Ce% zp3U3>1wAO+p(0i=s`mjT<+8fE(kk@|wA3|&QP;3#Fp!?WkiCp4a`%Gfet1Xd(N@A! zT0|(pkO&#mR`Mf9dwUoUudyG1Qp)QqVh2#@TMS=il=J{55eCB-^iWKA_kP(s}Z0MN=4;O6Ol`xbQyvSxn2&XmR_cXOJZ^V+dGbH7`22Kp=4xw1V~xbLd=$6%fOD z89;##=21z5DIZgg0Ooqq_qqkNwC>wKT5Po4k+_P72g~_5lol9pbobck;DZhqfj5XB z)*z3A2cmRxv>C^DUGFi@=jz`(qhJq(KlvdvD8jCKolvBsep%nh$f&`OphHgMa>g_W zY{aQd&W%Tfg?*PM!SHx%Yk?0xf1WQ07mn-frfqb?VT^X zKm;i7tx5=?mRK1?YY^1GB; zvFJ~w=#zDAwn+&1n4^_q46zq3dnT_)fcbC{R6fua@`*Q8)4v0sme`JbmLTCfN~@_O95lxM-zohZhp+GGSj z6C&#YusPsxrcU3l+u8k!S1Wv%Ab3|0b~-G#3(h}N&hcyAv#Z&rx50m3ww}_FLJBgM z=_J>!x7Q#79^7XrXbF`DW(wNvA0WEBh`p|*E2a*av<8f#<+=q%K0J`l2?b4JQ>_-0 z#ke0MnoIeISz)^)LT(n5p8l)^J_H%|tPB+^xs9yx^PjII*i>9lLW$p2B_yrWPBI0Ls=&n8Bn210On1n|2IAVV6WG>ZA&1PA&$SSa07*^77bAF z&BmNecF-Jf9Vtnr|H*5-E{+2ovsOzRHLk=YiOQu@uI{Mz}1NKM93{ z$eAvt3I{VWW~^p)YK>I8qBdVCy%D#es^jp7T@JdJe#Ss5<+ESEk5!;1-G5zJzQ@G7 z3=E<>=qsEs-D4C@;>D6?p8(3X<#j0bF%}NOqZf8z?77$}j7b2>rdT`^w}j^a4P0oH z;z0;fl1<2z)uHG4-S_uAf%nMsYke^Bd&-{*dy1ag9(SJ9-6CYO{2WgrQ-G;R!R=K8 z=?EipP{GM&1g%1|8!_F_ z(ekKbAKKv9UXtL>dPH}1gRlx)2R*>kfe`^bG#cK%OP{!X>s)MFB;$~-Ts^a&Uyx;) z>URwl(%&FC99bs%Je;bPR_u?okkGAonq3j&zaMZZHJrG>BF&N=-6habbn$7LFOC~1 z?Njz_!Br;f!;I0 z^uSUSo1))iWS1JC{y!wAMr=Tv?zEw4kJ*vS~kLY9Z6W zQ~MPJu&fh`rpuyQCkzo}Ba&0GmVj1^wgoRj6Dcj?7u0VOA1){dR>Vi&*6p4 z!X=|UAl+4d6K439frJBR5=q+b_pJ*l#f66_^}7-}v@FGWkyMgRy@sc(J5lEc-ruLP z>t6YsUVra};N2<7D|(Z@#v=l{Mcgyyijsv2maP`JeAOM50&B;Iza$Ozx*qg5nc+5- zJ{ffao~-IBxm|AiP{LK4)-Q$jM}+i*Q7a`WsRZKM+uN!@-)b)T z+6@rjBa+a>*(zRqJ@_qYGht7E*M0A`|K^D`Qi4$#^>hF5@TL0&-S=qz@+YIMboS7j z9~QIG$eDtT)4UuOp9zI^njoz*O6QMaQ#zhYG!%VM7EaR7rr9y8+F$@Q zjyY6|rp&;y+O!f{&G$kQ4Z}GVh93q2q1uE(-4`W0MXTYAYu8)WJ^XWvx;H%T{6uk102bU^`* zS)TYFVybAWZw|DqQV4FYDEc(-FzL8nQ3Zdl!0SKSOaMzzEcGAIV(bzbU2fSc&{VgSwPeSn_nFqI&hGZ zL=nF!s}TOItZd4SdED?I9CAZ~`Di&@Cc5+G1ITG4%T7Lto2zRBEjhiT3ArU20!hoF zk<2Sf+An*p?o{ALiQ^?A+rd&Eh zeacrf68zlhTKQ4VIGY8(g>1ObQ=bLT2Xh5WH@^0yzziq6UKU*X;Pxb#+(!uKZ`Tmw znhZ|w7VBil1oLVy6(h?}4HBUfFSMTi?=W7&>mSRk$;Zmz_ zc%z+KWayg=6;2-+&+Q=>?XY(KJP_XPP71FY*I5Nb^x+B`Upm2n8Uny{~S{cmpy zKZJm8{>r4rSkP?U^danMQHw$x8Z^6%5@au!+4fvpZS96$EzCp;SwUoGI@cQW5adi4(eS+e@E3i)eN2X5 z`r@f_+N0yN;Ed8aq==?~@*}Ci!Lr}*gj)obO-3&k_*#4IDHbn z1jw|>Byl~) zmt0Y*Sxh%Jw>_2vuI%&Igv)jG>c-(5qqpWrm(9wn=Y$K|J|ta0>vZL=U$Tq4M#Ed( zomx9kHE&FuLx>V#H&v3RMiD=pr#LEc3pGO=LEozl#{0&?{JKmtpW6Suowshv5t1A?pKY%XJ=Io4>w!e%R@XN$&55D7B*; zUev#_K`b-;7q+;<>O)CP43#`GYE=sBbEu&K7UozJJMfK2-*d2Zbd<7Vm3`XW(!zs8 z_Ik){fI$>kA3@JE^t6KL;H0@*dBqDHHJ5KnsLk8nS)RV42FdB}#>RJ@x8|;eGP_J!occ7`6B0r$BgE*lT4ds-X22aDgR{{S9oLSYD6^z5oD$0Kfq`%)V>q z;NWiPSs0Ewl?jO@9QNWZBZ^RVDg9o$4#!UE&&hL!POoP9(mi5V`p|!G;+K38Vi$<1 zkSCop%y|s*TL@?{;EkT1-lA3=|ATJ}WQKx}tHabikbDs{!MIIsXx-|6Jou5-dt+3A z-X3!90j0UpWf+i#T#)}&Ju@|BbRmUwqbG(7wCMIzK9?slOySfwaGg87EdC>xqIVVt zt|aI?G?RFeWpprR7;lhoE`UmKbGF0u<8|nSaRA}vP>EF^=i+dISlJHz69~Ebzba_X z>#hTwfY^d$>y|-)h5DKs{J9j&QdVEGLS6B5a2CPKcV+`l)Gh-ZtX(P#KtWX?7{J0? zO9Z6sAh$MyVx+rx7DT%Q_=NcQFj4e0$f3|unDaxmp7NL4>_V_L8CG%uPVJ14IZ^B- zsdEp%Cy4az0&z2RO4orj38ot4AQ0e_aqUye>O5s&b8|DyV!Mb2i%c8_LvrN@&5wz! zu5Kwoz6glK?Hn!faeF7H9KLGMW#FCHJ^9eBue}5PM5P@XTE=d~hqr`>zdbvaIw6#l zlIkBFm52Gt$Vve`Wm7i)n}APj{>o1fGo(zuJvC}qclsZ6yoTSm0$R>Ni|q(YebB#Q z5xT(;-4M1qlO*}_y9s&$tfCjZV+CNz!%D|ofH1bK&fYHqxd<#F9`}T|72_E&^BLv= zetNci4_1NiMt!|gwyaNNSQxGuYzL_0$KsB^G`H%MeQ{wHfDVE}PTHy$ z@s%mckAO@Rnh3z+7C>+86Ut)Xhd~gerv>xC1f|ls{!}j-v~=>y{`N)&aIS#x_7Tw{ z2(NkZpxw|8mE;NvQ%Q?1?BxL%b1&cEiESET2ADmP(nDZRPy*YBNmyu5zrx6RzHS_; z-}%s@1xwuD-+z_ogL}r3l4q62evxYrPbluvWo2b$P>e6~Wpz5CKLJXl8bn?=*{EW2 z6BIVs+2(F(F4@glQ2@X8|wlx zQR{i;Ee6)=i8o2>x$;ZpkG_RzC%;L2DFLEjQPA5vx`UH=8MM>V3El}w!HXq)4)RE| zHwIJi{m^uejnHwo9lw`KhbxkM?f`(n+G%iGsn5J`;7XwO2C>R*Nb|21NDb}J4PLNv zBn_WQLe9xFJOAq!)m_lht-(rw73T(V1F!>_>9JQ(P=H3GL8&?a`New>76PEyktj9^ zYWP)#0#;Vm=X3%R(42iZGX=73oH>}VATKNHn+*7ZKe_aik*)oOba=LMoSfgGGJ>{@ zNvL1Q5LWecWTHoJV6NxlH?Rv`Ia5L4}wZ3gNePaCg)ewzg1F?bAgD^$rZGm#w zV`9*P#Lpk%X3r%qzJC9aNJKTPaR5uiQ!*7aVrvlH^6>ogbc>Nk!|x$8wY9Q(>$!4V zZnD_T-rCY40znhvG{#ey1^iN9$HG|zjjrQIiGZ&De&5>R9f5z)-aA=Ba#a9QTsau} z(E5B$2lgtQlj)2XtyQGD0=~k6fH3rHdU}23TJ|r0JkV4O-U`yb2dCJOI7j(wlUP&+$vC&J);W3A-;N#nQI|uzT;r)pA+X z{Lx}&)KCqaw^QuzB|2r#ZQ@cE+mAgaG9I?NXImUtc5^Ghw z6mzxL@b(w>{M3LvH9fR>MSPcZBpKUFiKC8X2qoLAqlqy*-5K2gIZG5=-Q<5<+Wl_q z&P!ow5fe<1tVGAKmpLfbTQ&yCsPcZoxk@n-Z>ZY`9gVPa2n%5s~?aCr)^89GI1aB zeGlKfUe4WCBKODKs8Oj{rEA`|)$BgU&SxPrU6nNFuX};jcP5-Q5bn%iA`cgfr)a?^0Yfy%QRe4qf~zOlh}yhg-a2A*SNG|*Xm zJmLQ9?gzbXI2Dx%S@F4G63NdO+*Y{|FTwEl%^dJlW@2veRRlE7-gp9eAa=0CR6WiF zw)yMm3QlY=1XTa_m7(|d#q5)Ez8H=;LHk|7C)iZbuQ6}$7_kHJC*EQq=(A;UqFv7=1okjx=NNhMQ;k|Y%+j+rt`A#*6D2#vxqWGHieN`sIgl|)IB z`0dkY-F5G}zwi33`^WW{)uMB}&-?wn_OthX_5(Ov6I&nn^_|o1LO z?OFZVGkOV$iJG;9T*=QRmz)_H5Kv@|*pbZe6LY69ht0K~dEgrcWro*1>i!A_$eYO^e07$YQ@A(= z@wj$h;2STigdRyg0h&31k#!N|5&Q1`N05a0EmF3;6VQ`PomSJylM_ zF%2E)>P|SlX8SdoE)d~rHoO=;`g!3 zFNbkWR{PZ{;{-~w%*rDC8qQL&oyte0-rM4T_6kfx&GMac`NJ}M3#z=%eq z;W)HN)m`PH@^NQcMAxqGl2>Z|2+KXpU~Lo$o`ggf32$A>k998Rst6F zx!8$&rBVFi_uU813Jo00Wx1lw=fRTNymxrR+%I4-DnBCHV83>*@#Y_MDl&?<*8g*X zqp;_t^BHbK#$k$_Q36(shhPhFSLik3wsDD^}fXaJWBvt;3xUJtas zGW&U0>wB^_Ijf2f5AV0M?8FGie1}p#O^(VI+nuh|gNanazS9i#DfSd?>Z(LmF&xxZ zXUey^e*|+{#z81*z{J4P=;{@=>1{g!#c0Dwukb}DDN~e{TYdq7RKG-u!)-Uv(^So01Dl6Jw%P1t+M*C8Cog~V=?pnO zsxdIjy-z?!bhC_}I3;-I(v!wOW1_8xWbmc#7iD&_q5LdFUe@R)2D?sD%C(jPCC{{L z`w;BQ#P8W6Z4e#1N-@3g@*LxPC4p?RPt$!$$$Ss@@KJHX-82JkOMy`tk{i&-<)nvJd*-t^q!xUd+3WDXGlPw~L+U zx?l4?sJggX_av7+v*XRr$2WAMNUP^p?%kb@vdEd%R7K|xyQ)uR|r z+zyT0x8V&%Ii>Lvk2U2Um06Va>LzptYZ&;wxk8wpWV(kxULCfiPU&%`nu28W83C$f zkE0zQ7O~hmV`LWh>3ffU>@GUFr$SENL=g5jLzh6~p(bDKJ#d4QrGRtQ8j3|w*E~HQ zE^MY3I*;roO1d;dDwU1G()-Llj+hqnT`LeXuoU{HqFyO20XHV%M=4FpZb|;R!tqs1 zlCqn)3JJbO6Ex1#M;gxjhJaJUC(fe&>ipSoAJgY;ZLuetA_~+@Vh6vho_=O(D)lX9 z>#!^>h6e6XyDikNw~6yFs5;`if@@vx^ZN zU4;$M9Ah%4adus5j z3)a_q_0ovu**b;R=e0hHyK-H&)gE*W@yWm`eWX>aDZ{RF>Tx`+c3AbRfHIB(l8t&I zxrfNY-gDhktkw_waYWUGoP63>00D&d9=Lv*&gF)Rt}XShqZ%P zM;V3HYycDr4|xlT5BvMS%iD5B#wlo3f5ayjaVPo$IRbxLDb zV#k8GME~hB21;%Nzv1plgn|-R4rSJQF?1fCl3= z^IO7Pt8k_#2zVQ8s9bF)q0G?j!XrsYn3M|9J~`$1je(M%SfdZ#hAc2wd>Gt{U&1qWsre9a6+)N>kOye-s?3 znheK2-SZYmH_&lT-$?V~(?6)iH(>J9)z6KN%Yvd4UpP3Y@bjtey}KcXUdtvhKFs^1 zrj6cldJ8d7Ol{I6S(3)?lO=9+CH6hl0l%7<+E?fsaVSA=!ykw*@Q9_xN>n`%{ z&Ns1?3kDx19UJQ>nu!6MHk6ccxdB@NSrLohvk2Yw8_edoofNLJftCgfuDXvb@JuRXY9T9OytSJ8Z>5QvQ&uPiW z8GpmZZ;oDOB9N|1pAj;q4ZyWhq4xyT}A@b-fbqKr%jU8LSc5AK;N z+~O3lIOo$IKc8ulS@Rt!dQ;55Abo}h5K`-8)Ch4p=x?n5)a0LSOO#-hg1E$GqZZd{ zOo*$S`dO0c$4I6x9!p!pbpGD|kVpAl)aPjbzE@5!*T8b4?RY&yIh&TJ!a?q*(ze9eFU-BWabHAtrSfc(PXl%S=ukS%)#?@TQ{d`nw+t0Tll~y@3wpK8qe7^ zgLGGnHj{!jpB0kVboQL`7UFhkLuO@%UbtCXO{ccKLsr2woJ_B_QVfj@;qJ*e}00$Ar8PJDF30Oh!5;o+n0{n6ph}cFJCf%;yp=Y@ePuTszQi zWvVeK4Y`usZ~!j(fDNvWJaH(TzIqSd7b{k5GN8TiNnLgxW-qdNT)|Q ztZl1~9$MaDaiyqe>CYb~Mn+VcA8z@sk_cA)*sqX1t1lb$iNUc;iN(jG(%a0#gS5+C ztLu0C*WpsB!MZaq-@Lh8V}8mHhTs3q+yDPyF#ct9&HfI8{Y%vVjTD2HA0R=#{|aKD zW4S^jOp!!#{JQnb9S}C9PM=mPRy;u4>&VDZhbH3UfRgz z<VZ z1(a}{eH92j5<8;D{e%vHobC4(8b*6|W#2@`2)+~svR;D14E}QGCEcwALS`Jv8kI?D z0uDF7AcGgJzo!f$Ok|Z{cg^DNN zQ+BG+Le5R}*_D5CJTzCsq68EN2j55ZztsnyzGw&w3%H&?Fs}3v%6%68ygJxDPZsd9 zv(u;G;rhX@IX{~pKJsoyNQa9ekJkTEJWGBsB>40ym+bcs;MmxL4CerKcO9NR1WFhe z@Dt9JEqZ$3ovYDUn=ZD*3qAsKUDO4|<@2j*CCjB}SA2(pkQ;0Hxxa){)4Mwr_dl`s zqq9t(#L-8$F+t46!bk6G!X>^dP5nj>MO%YOcovP*C|*}&HH=qbo3_nMbkRT?+(`t58FNQRQ3an!teivBuJs-^JS=<_Snx!W4(QmJ zG;CaARelwm#7Mpa?gWfIi*s9o7*j=a+|&uM-neZ7;&`NVIJ;-TAIgVC)1mkH}`t&cYW+|+NsK;1A@+EPRwl) z&6H-F%&qhNTi{ z421QA9=fa%%t*FVvdrwo>arx$C*j(d$Da$tkCS=H_-GB!e4}>I@qWf}t%bD4K?KIg zH&I3<-mr3s()icL%Zi>)4<15oawQahG@QQ+?f;w5nOqIG^UOD&EuQV~Hor zn=4Q7Txl{cPusQ-tVxrQavkOX)dwLZ$75LZ*nCx`G}A{Zp|m{=pIel>+(*%~PBI=D zzyrp*I)v~XkG6jt7{3%tTkGZ+Zv9_lf?9c71GK)GrF-;yo-CGby)aah#qhf7Am>4P z8i|A8_UFm5l1Zo4!ex}kEH^E{v;|RV3HNuyyothJ$m(RyO$~+pQ+8wI_ZF zeF2lRxb(fJ(S$EO8{_88lg*ZNB9ihDoKbbEJwq#=?w4Off9lfMwzdjq6WgI4O~Ukp zRR4P_1rcdBuh&;R>W8|al~H$X_30~raL>!u5@zWJ+gcyLPgL2(65PWO0v zwC4M^C`eMzzLMf$Dhs#R=a#vKW-KF6+4G=CDXt{z&yp6N4L9X`>*61&{6@>{=(S;xXZO@$uQ)Oc_%ej5jLFKQ`c zF9cP$IkS79%&(=;&fpOtSyl^E34C z$P7V8ia=gsrO?D+x@U<85$|;|EHd6E`s}ZvN>`>DB5P!WFlnb~UZjlcPz&kHVC^OS zvg6nyUilc*1(OTiqlSl~7JvL8nUBm)sfk+72Hg3{tLPhXv@* zyqd;+yEoNBY`Ay$CC{=h4g|=ARBaxEQv>Ja(`Ym=Nw$+?6f(i%>K`0AE-Jb2PoR1C zC#;$%s8pm{acWEMM*v2N)29(WdYHFns9;ljIiv{21PT)$)yjh(MKN5@4^jUotcd@{B^zh zWb5x!;b)QWlEk>#PBu0sem1|-)e>C0SrD~);ISyf6A!jWv_rJ)Zo~N&$5P9zNgk=< zfQXbJMgAh`*SxGuPZV{F)CZ!KNr^+;a|D8*yqS@Ky${hAzW0v$Nhu{xEqDWb(kJ@3 zG}btktp6LLN6?WckZm|WBk$%ImJe4v`o>|y%tt-wA~!Y*@)pNJARa4<#CEsw*$6mS~f0k)3!ivmLfWj9aZD9 zqf&^p2&?OWp&Q|L#)drty@i|R<<8Ii_z^u^_GD;iXy1dVt8m;B+567Y&P&NYfJuAc zs0B^XGC+c-QJdx+T(c?MO@n@p1xdsLPdEV;S zE1!)<#jc<9u#@sPm`v?lVX@tU*&ERv*+$&=pA;4zVkrSjL3~hnsGsSzv0=M264MIw z+B7zQvSBhfD8Oqd?szp&KblV{K z{;`X}Yr5?{jrCB$SK!HlO*10ziv{p3w&x`}8aTLY9xyo~lWJh;#Qbn)T3x|$XE@CU z=S0qF+;dF{cNBS=IFM^TtelNWC9Zu%i65h&e&fIe5WW0bvJF&nyX}jvA zNVCDHtIdXSM{+${VZ`)HY55b9#{7phV+mhbcqH2g43@}2z-8pL!a~wbw*v(61Qtha zouIbEcP}dy4OCF9bF@<38>^lNFE0i27+AFXPpVFUxQjpSEgjFm^hB18Bp@WZ`PCl@ zC=x}v#an``L}}$f_nDn4HC=B7d+O(sPPra#Vwd>Hb5m~BN39>#v*67XWA!bqu`#?Ufw`P_DlIKID9B$+Cuw6~l*pWTep%v`%e3YtNJQ8|J|kSqH)Fqj zRiBeD%t{_0M=LMP+r(@mJj;ydy-;`+|IozE7*mJ9wia2_b)9u)4+6$DF5S@lKtFKk zx5D@E`+d*yXfE&0wsMpatu$)&WR#?ecTU7MGCV zEUi{0vgFfK+Z>5M%FlNF9I<_v>g)bwQTf?a^7>%6pUMWUNc0oxk_lFGc=g*a%u-eX zv4$D2hxk`}_Z2;=9O7{M3Tyvo<==!Y$B4?ycVD*V@3PAm_1(oizj(85s@Czv9*LVL zom9j&rr-G#pX!;+FApdl%xU`c8zM#K?87rs(vf}n=f#)&5`@s4ME+P0dV3)0 zLk;6P8O-y+r%xmF?&Y#j-Bks5^oUWb;8q%e{ha!Hvot%GoF7`yxIB70_a6*v|7T7q z9TSo;y8OS=Pw9Cm|2DG!ufN!!0Okp^@3ZkM;yUVPJwI1hhIQ08*lvbFY31okV)gCy zLJ#L>Mz)D}hUe|V&fjoGG55f~FM?nb-tYq;Xug7SLxn;)xR3ZfC`4P$`!NY*oFLi< zgC#mlRLZlUqi5c3bj!!~A<&AKp;Of3+|b4lt!mqg4;Qi87j~3#`Uq9S(zt3Q3-`JB z_~82MSTeTRcc2$n5IHoc66LVLVY^O)K%|>|^X#V^6`&F{RaB;7kV8iOT+=!}e*Qir zPoe+&<8>!I4R}rrm0SJ|Ko%YVdJOW`t;^9|+{CX~e1-)xUoX-B7${^l>jz5UE<#0; zMlwGLdKh*Wq2@z?QBqU$M_s3f{^7<(cX@oWS61ya#>h$uq@M?VB{Tt~?)vfR9~fh? zB`O63*3h5nQvl%6g|DRUdXC}^q5HHN!FdJCuUl(^7>v=CKBlH@kFkfXHfYujQWr;M z?x}D+tpEg;LYIyiU2CQQt<@PA$KZ zI?l%qmmMQ$Qg1@t#=sJ&$&Z7zKj{Mmuo%MZvbVfLMKX8Q<5#c51Z8|5((2*zX(pd; z*$E>W)@54WADTiY&>&(i)v6M@0eFB(^OUTmml}LAEu?4y1emq8kIkOHmua)eCz#?yz=lcqZU)9%brO=%4&5tBfPtz zpi%CyuoalF48!Tk=(v|)8Q~6w1L?wG_#=9xWu#$qMZ@xGST2!&$7>Wi)5t2x=r)Z! zQ8bGnY)o_MqjH?r+G+_0KfhRM$kjc6^VnX3*0P+ZoV1aI!1XBc2-6mu@0JqBcW#Tj zohWYZR!WSQ6*#o(oT9*mbpr9hc^@*j*w{N57V|b`|kun^269D01}@n7^1fY z226`uva*Hny-4nQ2={)TuUXmkE>6g&G7_lR-s-o@G^cKl%j&aWM7hlOBEPR{>ju>uZRd9Wv+ovMg_5+h^l(M+dz>5`jfDu>kp&mQajYbAKhQX?rO#^DiRDG*BMY;mztmM!ZsD$tGkFp2 z?s#P7X$DEGcku|K->q6R*4in8!~Hgx5Vq7(am_X&KJa`9!_P}Sc|(taw|zM=*&F^I z3rx0;r8`|u?UWmpyCc;-voP!Kb_1n5X$o4cva-4+aNY zoUEn$xaX>8>)BQaf8)!w?Z+ z;nudcXAxLz_1rm5JINPj9FrK;8?Z=TJn#3qCcf#0)16N_+Q&`9Sd7=(EtN23d-y&p z?E7s}N26q{CQeZyNUIchTbZD}b)8Z{`3L9h|(fM+`8&{vF! z-6<;kI69MNdc6AJxwl>Zm-DuseR45|XJXnP49pu*(zBlKZbg5uuI%ke)i+#qrkou& zn_trWunF$RxW@n|q%-WB)ZmShE6CzMb*^@s!^UV}m}3n=yDPCg@=>j*K>GWnjrN<= ze0|uC-D(ew;hfcr@8JyojdgKV>MnhuFE3e=%rpvq9EBfeJG&+j^*Xu2h17Q3zN9#I z)u!mV!onC~-M4uJ5F#gW4g1peokDXX z*;)DOA`Y8){on(1KYLPzO(El%G&}{f0lO0JF=X@%x?UR@_oZcu+-ThIsX5>aN8wcz zZqS#mHSmLs-LyR|oJN0ubJsiuajK}GfCS{!0#dG>o!#2PnS=YT$9YXY#qGdv>tVBS z=<9*JsAcBC{<96g9QW<(ROs*TXS4HE_v{pvj!mwU zx0ClM^RQ|Pm^FNXyuWQ^^4tMUm77cUsYX?p%XDY-c9}cc`b?gq za5ith)y*FE;6V8iEOUh9_$pkD(>NnFwNW8RXnoJ~puDnUKJQpMFMYpSTs%X4dTa*3 zL5zkG3vKNWu1UV`LFs(&=Tv^q;+~xf!zOO=)vJ4MQFED|K1~>qy3OpCQm&obHJ@;L zq8;4x>^B?&@7UkUC^lVA9Nn``IRAH3Is37O#SQewb|tz={!%}ub_|3z zw>5L6$#44@AgXLNwp~%6epF)F${AbgNO8^2GlR6)y(UPhHk`>TkM^=1_&5=rbA2xM znO=`Yrg%@RLCo5yk27gAAq%g`O+#*D3aLsDB&}mq^JiZ7TVre0d11W+i-);Vy_%jx zg>@!H%WVN1PRq&ZEFIVXaE?%U;%Rx>)l~=MzUoS7`wC;?W$gCGlhCdoE=T`i;NXtG4Fnh6N&la}HDjU2>v;U_ zIR5ju8h=M?nCRNtfByF0R}%Y6E&YAb|Mu f`S+EUZ!ELl^$qumVJ03V;LB{gmC-Fjmx%uYfOEiC literal 0 HcmV?d00001 diff --git a/docs/images/async_compac_4.png b/docs/images/async_compac_4.png new file mode 100644 index 0000000000000000000000000000000000000000..537b2ddd3185cd5556bc2f4852e341176b8dc596 GIT binary patch literal 66932 zcmd43byQXF+BK}3?ha{06bS((MLLxh1SF*d>6DI5cPJSB|-LT-ZlhqF- zz2m7j7Sb=Qcuh@R2>WodKqG_88bT`5X}%fYiGTJhZk80q-v^PG;Td=gq5Ag;thBQ~ zN$~|v--63ZX`O-i`$~?!zpvCl;hjD8|8a=C!2f-a|I;CUUS;_|oyY$^gxCL1E+sRa zIpDA~H#ax!maD$LzW@HOJjo-BzR}V9y1E@E9q;qrym?bwdte#shRglu&!0mcNuRVA zFMPbcGxmAQb8>=HXL~SVjB z(QkXyYRYeCLhB@SWu$bD`DC|)-m$J}^RQdW*nhcV_}Mk1JFXY!_o&d04SPSms_FKB zpR4jL!rbrTZ9H7>VBYib5_<5C=l72v)aa{PxxVXdG>7hzHyTckeL9Cu)KFGE>0ZCT zUd>prYX9;=nC6kXaE0py#IPh*4d?ooZXUtean)1sw;~s zHfeNla435725;G*)U0i&&>-;+Jw1KE@t&34moHx`Dl3N$JqQU26FL`v{}ytb^xa+> z{a)oraL@F0eSQ5;+1=e;N=iyM4-cz{4{wT!78-sitE)RqeEI0nqYs|z*^P~jp`oFv zsj1=NIQu4ATEV@1`@z)e8Ml7@{E7XvZ*cH^dHHy)2d_u@y98Y?@y@w}3vqHEqkP0^*kdmENoT7%*~+Yb&UT$1kT=wygEH-G<5 zU)t5)-ab6MbGWyOepNz+Au1w5yGTStM8j(w)6YmveYusP6<>~)i>n|IJmK})}WNx#&1 z>^V9A>8Xj&as>g+HI^^M`W5&aIAl#L&DB!Pj~_p7H{RIT5K+KAuP?{hh8(YPCwLsE z9WL;$xHx{_*&2=g`!pm?t|A?*tw%~sgA)_UE_c&nLDWRh(o#ma-@bi2WoPAk70E;k zrl1L73`_OTZw0#iwnT)4lwZnCi)a-&NXf{Mfe&nLqxX2)_1)bCF=-ZAtF`jf+Rze@ zmuQC~cIIJaBO@b6LKtObWcaRh=*iwf4mnOWFlzo7tFVQl8Sx;Tgt>iMUbkOhQl>AF zh%91yh;}vqe3L3Jg7!}Y=h4AVrm!QE40Vr?Su;vHcufn}Awn$$`@zO|v5BM{Yf_`j z4}6=EN_%=@{2ydLdeTLMRd)qvLZ(ScxA@HS&MF4)dnwu7YeqGf@OKh4=dmw!&CZ%# zl9ZR1f5m_4No^y7UtYmW4<$dnmPjAM^`$x4&S_)*cJeT>l4DLC@Jmi>`I(g#zlE~dCkqu%cC1iZ7nTdzjPN)nz^J?wB*Ltf@B zl`^M{kQkvABfanYm4DpE)|Nk?f4!Z9LLRU6X@)TqwKZAF72f-07GKf?vsvRG1SQ73 z*v~R&h?u&y^jy~CMPF3 zsHv&LcEc>c#?Q%+z&ZC5VX~*2N~28MaRZ+{d#0(W86ksal32TQ=T39xFF8bPQB>N; zRA(nACuiqJc6NvH3+ZPCwJrof1my@rR837y!6sG0?iTo~DQ^^cfo`#ab`zhkFS0#`?mW&7<8GJIxwdGU(mKm7 z>h>jOEFKN+T`BDSWl)T z$)Ku|65L8zg%A{5g|4>tH0eIBz$=C;IG^r9dfS_tREwQ8^m80)S>@9uMQa`yN4qecHgA14%-uwk_?;fWMMM%~37 zz)RQLegDYDhEoR;wdaMf`mrwQes6|2!A?a*1vHN*$;nhhwvQgk%gM1=3JD5Io5KCX zf2}~?jM%`x{tSJ6fVaN4A@F^+dw;T{U8oZdHulADP1A{Ni3JJm2~`P#rBol*J|Zzj z2zV|ov#Rdz?`Ma5tWLHK*$x-6QsljUJ;U=>zDMB+027(>DT;URk`J{|^|@}5UXOJn zXj9ylAMAVsm4VNtQ-GM0g+*$MgCt#Cn_%mlr8H9y}RZRGW zoMRH~VgzG6G;3)a^VBoj6Xe931F%m9d^%ipbcUcF@@+nU_AJcRd`;##nT`LidQtiu z5(6@3(m@SuEYhG7a!mx)tdgIvZ%Dv~>@$HIHt= zx3?!9ZEI_5+Kr1xz-}FE9 zl$Jj4&Ea2gv1c-rfTq&^dnM!%fTY%rj;iu+1uS zr=p|VUmM)wIK|vs1ef}DRwk5HR9f>p`mS+u-gKFdAZ35NGVb-W`$9VAda-e{3*eTT zSOhFL5izlvx;o8~qKmEV;%nvSq*hLDxI-;V<4jCRq_i9+s;8GOjX$G z%E-uQYk%GUvq&baqM~wFODn4<;!?_kuC6WvJ-v|7P*#=)yi`}KFCm8YwHK> zG9jg<{Dutt7yht_%wfcc$o^|CkQ_!#^!JxK&nYM?cLBi73^=|ZIrCh}v)Ozjr1*}r zyZb#2jX<21x36EnPTY1v$oGa96%@1>S2<28+=FVg8=35}G7gX#89~6_`%(-iyqz^V zAoJo`sqxputaw&dJv>J6H9k)OS^8_LhPKYjWntrvW4b*izk zzTRkl3YSN>jLtIhMkZTqTI6j=L_krJI-tBcdZc?954Qrpdm6w}~&+go;xg17Be>{Fe z9fm0M6^y4=o&4#vZ2P9jxn#9!JfUve&QW)zo?h2FkLtA%WoP8l4b+ zv#oeuUVGEi54a`oq81ky{rvm{1qI2@_gOu9^lLCrJ$j02VsLQq+c&6az0VR7BBG*X z`tsksYc+=GgKEp+`r~u5*XB>CrDqubbO`OO0Ic#0jb56lt@`lcL%bB*3~PO8NJw{@ zU{^~^PE8Hne)Ov#hDZl2xz zJCySC^K-Uetg zt~`C`;%x2g>f9HHprK@BWWc%an;@y#O8F6A$8e;qUt!Um! ze=?|YVAa{$drEov8!jVcB`#U>gh`6NIaAzzaB%R+_#>fzDzW4>c6Romp`l%ssAJpX zN1FHVf3W%X7C+_E}nB8~e;OW2vjFANuY#Vw@0urX&<IXsIjigcYD}l>&cD|3?@P|!`+t}F`Jes>L=1Gw zf9}<3-U5>Y#ilL|aa7fKwCuh?xYuQJ3>V4!Osa8v%$AU{_etZ)(Q0SU-}VfA7Yc>? z{Q0xOhRVlPqx<($5vs;4$OafLw6rL2R^R6VGnZ)(w_skB%6hVSoqL9dVLp_?S_Y`o z{~@d3LxFJm55@{k5nx~;ewwNNZ*mQ-tBnG@ogg?45M7|Tr6rz0 z6cRp|&t+}83DEG=7#Az6DO8@((a~A1)rs2hukWF^S67Qj=V;!OlWU!635TRUrkOjX zqr^w695XXBOfD~ z0UjRvi5hp2?vvaFbPJR@ataFQh;hd2IcjGa@K4P2)=B{vll1HvQ#AVr9{&Y`=%7f@oFS;sK4JCU=0rs544K>!#7%aFt_Gp zWg!RMH-D-MxB<{06sDNL1^Y(3&(YwMzpu7l?(2#^wD0zS=#y!AC5MhOj6`3?L z1-JF}k$#S54Nh*HvKL5v`ZSojG`gl{ADaCdHP!FmzxD6mm-Y=cOVH-dX@NTfJRx9@ zCjQA9cjfYm!a|~O9U~*6miF7YgoJWlTUzLIzQVk`B$ct|P3u4ottR?W;JL}lVuWGb zxu#blrKw5wMY@&D9fGLRZZrr(ZqXXcPe>Tv!b{^}n@+95M@~-8vi_gd24BF735$+% zX=diUGVqE~(b2(UQ7f~d(=gJ6g@gdFxHSGj%MkVc{rl1frl!tE`|e9YRUpfI`c@bEB&frd3OFgPT{#}Dd^!rcG;t-whxDSWJ1rxZIhzy!r_tkJSQ z*TKlZFr6kOBm^MKXTI@AmKw(!py1Nd=(A*SSu zQH>1^4S*zPXXA820uoz|MKDOGwznUT@Tpz5Gsr)$`mr&z;l94UOsJ+6GcxR6&aIna zZ$e9kAze!R54M*@>Qgjv8?Q%*>`VQR3SfK-NiFd@0uRv1nmnKw?Zt(K=b4!&Vu+V1 zDb+ULIFnTF+zCpCNaK=_V5HV7SL;Bp%n0WEw;~j3{H72 zV=o_{-WS3og|ZQ)R5g$~SkHmP?Ed^2pO{$Timi=J2~94wNc#<|o@q};M#las)fYlp zq5_d$Df4ZQLYJ=28c4zsc&E@1UC2Bh2nH5_D6^ae2ih#pvv_`uF z5`)nPCbm8zfD}Mwp#K0Jz>>6kEC0=#w$@fD3IL7aYO_3_Y3bIK1SAKG3?VJ5*Nmh-S6M7=OiW)AFHI?4=G#4)i>4W;o zl}>j22fR&tAupc5QvF_B)GULt)U*JXwbsMQ4<#(aTZ8(|oB30Zu>O>nmrsX0XbudM!-=aBvFWd{3B)fp{D5dt$hms;>dMLr zz({L?!nq_f#jM_YC&a%kKL3xR1nWp6i#l;*NdoHFXJ^ITD*~og^lG%bh{iGm9?*iR zX=$MFt&UYJdNAevm&6rBz)e8M*QaZ$Ud;EAC z3iCL8O-xkTe}DY5J2fVTu>VT{Ud%OCVG2j7C^Ooo{sFTwie;Yk3a2lTqxcW-&iHd%xupNd$2c#Q> z{~H+PWJ$SiT)U>Gpn%O3X=de*B?zR&$3k~Ex5rB(tlkooI>zgp=*I3pU3=3r;H zgreaiTY=`>&n+yBRnYtn*$UOXBaSgzff?0gQKuc)YK!s^G5Qg)&BruWNr9UQLd+!PT( z-4HQ`B1c3N3v*OcWvGLmo?iajw{diW_&6N6<;BIkHAE+};`z*D_KzOn4R_peMpE1C zzTQz#pr?O9#ijG{!^xZ|2qdR^|LHn$^gV$4NBSA+#K_19OwrK7n^~6q*{@$G-`)$y zq@%NW@ZcOpD5j)A8=d%S)$#YQ3=ekNJ%o zH{MD5Y;%8Qkc^FqS)-uB#=$Yyl9Kue_4ArZ12^tgBAep&%EaQ_+*5#%Mn`VbQkP-G~x2R`YW?Vz#tMeHnp7Y13)($X>| zADGXOkk)ti-}lM(Q>X*cB;+ujpPP$qNu#^Bvf{ACY!t-73sA-Pxrz5`4Ru4txn^(B zZb(StK^t`Y^ySMkG}g0(aE}H}xH=aoxpQgO5?2eLBpvB)=;9p=p zo~sXXA5?HsCJlWfqk4yl8X(GXu(054@T#0}pQqOb$X7?plA@v2!x$P_9H56{J=N_( zhMPUUdtc??kUFoZ*ilw?eTH1nA0h$5yoLNhIhv4D?hDjp81(^~L6ricH2i0CiDvLx z$QVqgK&>xKHwDMW#xC?`hW|1%NhyN4w!BS0RJ{rpFF*WZT^a=6(7NVc z4IX_>ZSAc;e|WgLDZkzV0BK+ajs=%_r6lAJ`U@Abm}U;zf%>0o(`>2C$^ z$u!Nh@{u>W{T)3|!}OF@>7cTnp(Y^nW6~M3vduJWtS}zg>ii)gc%l;kQqYNkPb7_} z=OEj2`FoMNyJA0>CGg_+2zAfI`_oCQ?hv7xaq$#@o@MUYBzc<1r}Ny|h;|Cm>a52f z_3$d+H%eP-8}8cOC{mgl*Ll9NslM~i7`&eey~J*+L7bEGEvy+(Vqf2ACDvbJXNS%W z*d=nxVti~YEiut~f9n_M&$Aq$#fB79K@9_%U+$;H69Rz}y2Ug1HYk444-B(9q&`cRc}Lz$too>_Aa*baaGv zCM;YvFmQLt0hB;c-~JpQ9|JcFxUS_*Tx@LHwdUspcTexV^#(b?*3K|v+Vz(KoSTPx zKD-Uaxw;;uitcay`oB?4wT(WTT5~K2t#d!i4z#W=3NR~B9rhL$NvWxk@n-)15{KlU zeO4#yO}v?%&WrL13)3`I{b-M+$H&K)kdXNC<45ztHmJsW%#4gifObtxNP16c)yv$n zSE0yYzm{SL$qO4B`#G;kSm-}Vi2S*T1yOM?ecDi$(tovMUw7v283JRLzJLG1k{8@x zpxQjYE26Oa0GWJBNlAfjsgfpuX8?8_|AOZ1?2AC^Kz6~s=%?Y((bWaf1Pc=rx9}|H zATWL|wo#!2jHA>x9g9`>e)$h~9W|Yf_gie8-QDQ`9U;9S8<$j?bkB;r#-|VbRV!Io zTXUM3p(k}PGMb2ph=_|L{ugKBTQ7j%si8D_!8|rGy4A(Y`~CIVy^&0tpSOSoRdNC; z0&3t}Kzi`y-4qE*;?k!4=x(db^BEM>OAZ!)NdY0yGCwvG;NwFgg!Bq=hTQe>@@gsx zIMzDpl>bZ0{9FuR?ISUE9-7K4vMpBk`Lq&swz!Ti(8}OTZ&)ip-}TqVdIR=N zfv-D7_%cw3DtQN+tDcCXIpA>Kx*lYxUT_2IK*_QRSa`$umn(udct66d z*|}SO?|ziy0m%j4j$!K4L+#%S|4Rsc89p&Yop@axE4_TOi9>?^t9JbB-KR{#ktc7# zfTQo&;-E2Fx&L2-^xw1beuy}lzBh0X7+1u+XxxY6b|xyFT9f{+YbyR$F$r34Q0L%F zwCB|`Uh4GFd&A){?&cK~ECN8)YZ&f&D*iC>jORml)KV@R8judT=WVy`k2OCxL}`A4 zX&06mJP(G3hCn_*&zKw;iSP|fS(%wBbzjs;%%5lsIB5g)8(sYWeb1ZjrrAaZ!cX9QH8(~pVY^Ioe6 zedBL0cw6%1>t?oZiHQp`Ok9>Ik&v`0kr0Xm!TPC0JrCmeBQUG9gb@WV;j8b-AAhg+ zsY*`12dsU9eo6GJSFb=Q2ns^ny7%=paB>Ct`5^2~)p=1EO_Z8ec znLhCEMPhC7iHRLNEwX$uan|@7Sde#}LH2Jq-;_4psL+OLMcscr|N`&eBEW6F`W7CHeZG zJd)kLqpV!2F!FVDG*UI<3i{l`ueiiPpLKP0&BgJ}*)Q+mV(ki=Dt-;Dc7_v&Qjn#_ zDrG7@IeR4z%N9l}u$pP`G?R_VtVy#px1I}!+QPmR^dyC8S_=9xX`%)X;_?;^)+vKF zJA09q@8YF2*?ak<-;O&-|I#m8%)V0LgTc!Ek`1)avte=KqNC{n`~c)L&4iND+~Sca zpTNh`2N4zgLMAbh0BkJa7KmbvPfJOWx|^s9k`M3Qo45i)Y*l!Ui4TAFW&%UWX(*E+ z@QD1`jqBGF#1gSINgh)csmP*07zo*r+zP@1pni$gf;(j=Uo!_7zLoJ0Y`-P%UM^SA z)us8Q?k4k~Cc(v`EYH(2`V5&7Y<~dSZ?|+e*<21=&(6}4U119*F0iBl3CcqkKN95} zwH|>RyuLU+eX0tFgoGp|Q4%`!XTHEBjZjflB~#~92M5^D@NjrnbIMG`X)CYZ{_{tE z>m#Bj3gxyuCfFK|ML^AczYMbpe;Me62l>smr3d35T>F^?LX@yrsbz2;Zg~2DoTDDO*vYB}HU)1lubB~$(1q-w#s;(Wfmq($ z=DtO38zO-kSv^^X3p=JqHs~((KUp>m7%XLNdBZdOy1@F!(ZH3RRk1w>ujkw@X5C^N z6Cr|V6{|!s@6gZkwkyD{E@9;izCLl=h=2$Y>cqWU6^5@(b)K<*^WqsyLkR`)2G#I>!do_ZykBl=G+eq zhJRBYl;7xbU>}u`N3PUU+8g+r``OxWsPC+5qE1wh3T7f6E9oFeW@IqAP{rJ@)(H8G zg}lbi9kKyF3zE?S&}2bAhdUDyK`ZRQ;Ei}n0KDL~JU&^21_pwMojnx1V=^$#ngUC6 z<;wGnjJQH;vY|NYQgWV)K5%=$*Yn^hOJAo~b3OAp%l3HYVsJkpsmtiR*)yC=W9mfU zy)T3vqZ+z`PiyfB8&bh3kg?f;HGsr1R2J&!*yyOVEc0a4IdTSucVHyR&4qRv=2Z-3 z1Ch2nVG=b53=J} zv3=TuJoAzGii(OkeC5^U<&*<;Ae}+`mqr2gem+y7IF3m?%5p^1CQLo&JD|u$QgjA+ z{3|7jKYMIes(X44LkU;|vdI?V->;?mFLSX)OfU;8_inj%Q7sT_CDY%3YQy_3M7$%Q zVnoBQuAr@ZWfmt$FJRF0MOEGYp(g#r!hPP1pO)gahc#cGe&Ci9S`S&bqMIv>%{x*= zI`q2765SvieTOjhJKWSFcVw(l<>4KY@qOZST$1s295*#JSb$1l&Jrc$zmtZ=KKw3S z#cGUJy{rB~<KjcO8g=C6B4gZF$|Ogr9U*=(>0QpYrV4eE>R7fhsi5KL_F-tSJ? zB6}s!0ge)C!fLbzKXW2k_=;{#_~k-%u}=3{oOeRT@!`725(3H1p~q%iW% zvA!YI;U+*3xyn;ZD;!v+h*-jqKa$%6V`Jd@U^hgz zBVl^I$<3`|$k4BDVFvhLdjC{BJCdXVbK6bA+e&Yeyk-f0&jZ}8G(lU;KXU0viX1foD@Vs-AThZnKDb`V9Ytf{vst6g zVPiE@rqg3D&CRjJXe|YqT)^KUvyNdDIFFoc@Dt0Xo=ur5#>B#!(IwP8&OyXha`W&w zBS2k}uVCG0P0~btI*~y^F(7u(VA^e>U!i^y*%50VM?1oky1pLn`1?&vVXdjjvj=`= zQ`z?(yFM<~?7ZdZUDa1F*ulO{(2~scFqJ!P+Ex;WYt?UO`uDGI<_gC}W#9dNuHMh# z^P5#vcE238->3CEm99eDVVm^cUSfstQUJHpiA`@df6LT+!PLjsHM$%=xz&X}k+~3$ zneVOtg?RFnq0?yDrAf|@2FHPo%;UwM$(%yRQAAZot64lv1Ts6tlduHvokJ_$IT+MQPy*|V=^nUn5 z=Yx(_3VULE9sU7Y$0Er+3zSfguhq=w*=i~>9~#kX&qt~pOx0pW=S;#BKeZF)4P5H& z*>=I-%DmjCVP$dI6AaKy2tOYmA8BbM_~G8Y<91;!i(~1t#=T1V>0He%_utw{5JKa1 zUN@qo%Swk|8iTN&C`rtC(+vzt5>-wSA9KOmgbIs^i8&>tqsQ^s!tH6Osg3JA*F9E{ zVWchZp`1d!Ezt!D2PT?0MzIiuw(r@e6WWOqYNmlUkMuqv4leu8dKgFag2}49|Jcy7 z&SHNZ0)31u2jVxFa?qs{faU@CGsqfriVVFr=0Aa;w1oAoos_yxhvmi%icP0<8@o`~ zoLd+1{K15(2mGZf3#DO$?*&Gw!&8z_K=)E#%1BEFf2N|RTns)g?nkm2D;Rp~s2Qaa zq#ZRC^!4cwv3~ueDl`)c)l$*<7m!#&F?Zq^L@%lwFmK`u?1vdMO8PT|$Lxl+C@L+v z2IFI;-QqzuBwG;d(CyC9YqV1wl%wLyo&?Y2WDLFMWj<1|oQxP9lxW|ONu3!W&wQ0> zVC*^S0;RF`$GdOD15exiQN=fjx$5m6s?$zP`TrtJ3ekyUJf2)IxX0@IN0`xH$)mN| z!&yhU{7TPrx-=K>+vCp_^!POBGo8jg3+Ki!P)Ob^zrEM$&l7o@`Aui+T6r2f{~lX8 zJ^jP{0iQ_WKU)Lx1d1d~yU%i)(TjNlVtw{W(uRL;#R5P5v#=u?RoWmlZg;9Z6zNc8Gd zXai}r{eH#|%YUu==?`8Z)Z??V{`}^3&_T0UbWzc+ir-(P( zQ*V&wNr|-OO3ng9yn`TpJk;EuZ_|`+y25pdp9I-) z;xtJ1<4W90z&@&HuVo({nA&8aBIBel!3b=~9|C*T+%6?zi5jCL--yZX+svPOnSyRy z6`!Lxq}DHcd9{!Ci4?)`Q!v{BGY^AHMR|F`(}agEuC9qZ0|NsLjEsq#azxT&3Nf^% zpj#q3r8G>P!)4!&fc7=OxugR&)V9{m>|MdR!)c#I6|%p zzoPyD^Fbd2Les_NdOGdfsMHgf9ZFU~b1IRzdKC{5o57Ou(bBZOxy^;LB@Ofwk#Z#z zP`W{TzJ`Vjab{p@=gyv7Y2fVmj3VMIiw$&(-t=2e z$uE;WW#er=B*UriUGcCs{U(aYGW@4ZEl$AJkiO!tLuG26=)7z^ zC11OgrWibo$$$ms8+nB3w~kvP&G(Dl6N>6hE<-Lyg1 zCj9C#S>3NTK>PD|JB{UGNL*OET!Z-{d^p?u^*YIAyv(?jOy^0rVH>P1-`c7}-C99S zdgJ%GzPBvL!$%fTjZbz9dzU|4D-^vp)vk|OWRlM!zBPB=(ais@Tiab#|M@LEW-8;- zt%}TGlk+{SKZSy^8!EnOF^M(T^=JO3^ier?ynk|h-J$g$pSg|i`opmFqB_ABibPGmapc^!^{-{uo0 z9Xa&}lf^;J)!MXPyDoR+`P!6T28rF5^|n-96!t8AeP4_UXg=l*i+0H^-FFkxqbU-# z8oSa&pUgv;M|ilVN>VVIb>0SJzcGu?W2B$)w*&PPS;>{WYm#NN!zZ2x68*Vsy?-V> zxMoH5 zzuv?h984V-kH2HJ3UX_dc(e6uyEe9 zbP1IB)>Ho!T1-dn>)Oo1Yc0ZuMwT~vU!?s~eezONWTu+`X5f7!H2G2!S{c16@6Xc3 z<}HNIo#+<*ybo{@#G>uK=ZwW%(BGx$lf^cU_aqb7Cj9)PN*3%THsyEj=3H?$xwcZz z?oIiQ##qdu)#uX{)wt!MklP0I>a`5V7qmTl=|bBH6%!+J{$LHXFsQOmZwY5VtJyNX z({bLIQ1Ih@{daqh?gt6_n{D5vc67HW;H=d57s6^R`%&|h;o0qZBLR==$+WlDc&u*{ zk9K))zvd4q4`Kfr4^dwFNM2|m z`Vu#Ft?cLTjqa*qwS*N}toJ@78r+&^+VzM!k`7>Ba`BA*446kto!7`zz&-c{_bWqf zE7xZ?>h7yszC*t*zS+~E!#TJ3V=;ZB=kybwp)bz1u|p7SioeCqyCehu|XVH z&+R_!*S!J~$uGoPCtP%9o#(aLinS_fHScbj%92@8rFzn;v?eCX_b0%E=RU5#F zmXRwJkZEmR?>TnD%1g7l6R>wcw0Fm}0M~!AN*wukb=i?(a5O$noprq6WE3bF1i;K%fQ!oxtByzS) zG23x*Fpj2+`@qgTs;)e>mok%7eu>2_48Yog*{u=y4`AfMegh^25C#E4(T)8f1*6lF zYZse$XvRC(

tFBVJBU8GEq80<$Lq>bAUmXuf2AV|n=&SjEntKaUOoyG2X9k?`5TrynTG;I5Ji))Pf63OcA|1=g#M$A?g<_X&^@?$Wu}`H28dbdk+L?VId*vfc!StLwE3J5&ZSWpz7bd z_Zo!tDy901>}_|z_AzV{z}OJ|$_L0M%^Za$!&0XE^T4||H8lZgD4WwKT^BkPJ(cIV zvtl$a3DlRK$aT_ru#%5zY;S=xFNyN2sMm(Yu0VQiR#wx*1fwEsFU8IaQDD>jlEN>$ z>)`b2adlZn#$*DktVU$QF_z%)&`|sM^bYfNAk~Yjr3g%X$}K*T6LNGXi1{>)R=Y05 zt@*uf-17Ir4+;Zov75#&r`-%!nI`#pX-a{T7I=s$@WeQW@x8g)psfan%AT4(hw5Ttp`b-z;VE=ZTJXiZD3=`8X{?uzYYyubetjlxckrcYE5yyR6tyMdJ%q!h{V)BE;no(cFy4R zDBpNr>{H7z{+1)2z&x#%hRPlOh>+Cx&z@0V{sy8St=29W+*L&iz@X1fItPU71+76I zgGvhviwDKYT)O!2E-i*0zzcdtYXshbxr*-`Kg!U^NE0Ks3?h$=j0~jP6tx`r-P5f` z3f6#76}YfPo`I#R(sSJmQc?~(Bs3h0?xkU)zc^wHm_rX>A|Y*kd=5rN^b)0V{F7+f zLE~ybBBiypwKKaWLEIamv7bS$(R;YLwbn5zA|q?oP_`kI;J3hRhp8|LoGenc`P6>0 zmqHrMJ5IN{Dk`$sU0j*za}p}asj3pTk+mlOLUHO|{z8Mb4`#~BN}*l$jd8XU62=+& zxai$UE3R8qQDGM6GZ7 z3u;(Na}T&}wlNX)w!_!7uX$q%a>@xOAiy=%N3?}=o6IHV)~o#d2#Xkr!=9WRCvq_R z!REKnbzta9VLL(mUiVcfnEOmnk%+sohe6fR)K!X8Dl^JG^+iUqR*g^wK760WH{js_?Z z?7!u#!KD=(2IDT7cT`^@NCTnW-72(iU%a4KSOKYAE&CFp4HQV&9Va6z3mP+-FD6o9 zzRv9z`NQLghxz!@803T#jlO$j`PiKTK_oKFcP=hTC2AVOJrSVG8UQPtDG_5Jj63E>3c6(ESzXozSy|^b$z-^@FZ9W7F)#+N=U@c&MK`mYDPP%Al6eTCjvy)6!@9kv zaDRrrE)feD>O>ZK6Ka+~Og<3%U|k~wyWK(M=oIc0P)Y}jg5qNp*xjH;PJ;&T15121 z>!F;2bQ)Qaie#!3Tja$*u-`IjNoSYX>1*oCm$6B=z>d&)v&J4uPk}-P#fdGqKG~7q zULu0%!6B3}iq(N=_CDD#!lv6PG8t4y`+t0}u;{oz{^tK3T19Fd65ZM=H{((a-qHAY zb%~)#SDI0 zp*piR{1B1#4-WSBJSvpfF-^oi?di;SqfJ^{5J$?Z!pNL4sLJM|r~DcUwmQR_hJO$@ z1eY9kM^GX9L$#D_5?L+j4bx8?fj#Dm3~b(U4?JOSFxYHhubK2>sH6qVMxy5!yy2qq zVgCn#PnH_PFW~(9F=b|X8~qbmWiIxVptq^cv|)LqzZw=D3y-vMA~GSLKd-okc{5wcW_3D^{rPn4=~`!?Pq z+B5=tKQp94K%cAy4FlV64`*jl5Yl92gV$cqc8){aRziX?SV`Nu=i4``WJq09Cc5$I zQ;O?u&~2Tfn6CD9hQf{{suS1Wuo%#bcVm5JGw?uYzkdCCF-l5Soqvc0mCeYViAk!k(pW023oY7o*kt5&5ez|Pf#5$-MVq({r&e3S)&PdE`tSIwgRO4#P4u= z%$wz#)N#70y4%#m-3k0Q2n2#9+B5nMcM^>`jx7D)+`R%FRt*+h+$wiz98GLj{TRW{ zg0+gtl#d|$o6%6VB!qZeG(@Sv_8vkNZE_kK*xy~|^5afEU(f)w77Z$F5$x;!G6Alx zu9}`;O`x5CO)mNPY>d~Ujb{p-`1LUTrO|{ByY%9sqMFwf*QD9kT&hLl5R9@6?@LMo zE3=sxc!QFyJwjT+m(h*iCmV4cahXnrgzR?TGH4Q}1MvS88o<6BqQP(giZvN)DM8RB zZQ2DKO|qv**pg`;#xkB3^cFQIB07RK@o{F2>D1K_jW|N!mY*Y}-3VU_!dzRM`z~hy zdh^WlSr0)lLsnH*;**^~#sxhF0Ut%k&Bm6^hyO9Sl&MSa>bS)D67EAQ@k&tCqN@ma z6s`v&BsLIy+xqq1g5|WSx6y=jK}$au7TA2U3j9`TKkB^WzILs_ees#+OElj0TWSoW z<|ZcK_1@!N`r2_4b^V`q!61_RSwl*tnV*xjjP#C57C$K+9bJe486Pc~K#294@+zBP z36-lhp@7^AaQR95&Y+r8s4g#a1ZMHj6*kXt%dye?VKIdNMG-lKIyWiPL>5#+ui}r_Pb$pRV9R1AGfMHrTCYJM?ZHj9^?` z2k{kn0d!^HkOSBJvEJ5ae@~Wi_$MK_mLvg^u;p0;6|iuR+|lx_@F&wzLijLfvGJ^j zywh5eD)TNgZwHviPUTX6%f?6z&wN*1-REnY{LHAUhaCvXTg^AJ+L$BHL|@2TO0+K4 z=}2SpLp9^tsTackH*b1N&&3@O9>v+2W*2LfZ#gV{cuZCJ*TwPIZ~+I02_f(B?P0sa z^VOhl#IDyu=-jAlrg+@cMeE&G_1ntdclmQ36x%rTxJZ<$cYV^9@$}OBa+4gT^U^Gw zdgt9e%0e3B#dOPo9`0I`-{*ID>g*z`{P6A8lM2)-RtGU{HcQg%>#SvddTI54W+oW# z#r)O%k{=z-_?uMY^nB0G;B1?b1}bx`pmt<#qrbO3&U1CO_{aoAi~Ic#-5(Ld1e&K# z>vR-?x3v5?!}gfHYnDY@=e=^y9~-+mX;kE45t(Pw8R_wFVh&#&z$MbytV~Yh72I0(FaDJu zGchf(8rXRElOWgEsXr5+9nHUZ6l*&1zYw)N3}^OR!=@H0OsA<)UDhe%p^H5pXE_2h zk)Io{pv|u&XJ*>`IIe2DL<;k;EjarF>>*Dq=ZMP2!f^;eT`APsDj$ zu;6{T+sP@9HUG3K$s*iwJ8AJ(J^H{LX`r3ZGw;ld=7{HMbkIsR!@7Xa#Owb&0I|O8Vv(^GU)AZhUuih19({g$65>ki*v!k5~c8!o&6z zrNFT(%=Lo4Yu`ab4EH@8>dad8Z@5SFMBMTa2e|1?voBSi-7iG<*!D(Aey8#~e{AEE z)uc$llUo^Xo^Kkm(03lk7bdukZdj9%EuwzX_fK4$nu5*(H6UQSO>hyooB4-CYPTGg zstpm@7wRR=HImk&2^!B*J1?aA1=hDdubH2NOz+m)7(MC>o9+( zf>P1sz-C=x$4gdXrRtkjgZS`0Hw}$V`^n7{`2SSC*QdXfwoEOatVo{nl%S#+AGkPs zl;Rij-QC?tcO#;tbW4Xc+zFoZecyk_xZ{p-#u=P_cD(Pq)|&CmXFjv% z*sN^c``gKCSWmh04tMxY9sUGTEXCNFYd@(h|EPAXxT}a^B8UE00Yh&|T2m71Ah8Qc z^@z3B8nM$k3Fo4>?)(HNAkThdh|7ysNIHyLiTo(oXfCR+%vt{mJ(4P-ZC$HmA?P2xCv(Ak+GIsk2cZ&l&0wCE4HuCWQ@wgN(KftX(N}FmfLIy z^QcNiUd}qDC+tTOv-H*)6sT=~haxIAlwopk$v^zooX0y!o*)$(z(`r!W~nT^_q;wm zLRs{l2$)`=eS|J4Qv8vrO&K9n8J4-#L!HVfH1^%Gv?)1=u1d}KP-0r5EMrcSLnm3l z@#lbtLWH8ph+X?mMx3jF>%b8=vsFoyORmHpG0M z-m=RX?y(<@{oVg@Z76){Nrd@3AJH#oYLj>jmTgu9r)gKkq6(vZGQGS1qxoQyXbs>5buwIt?sl0`x)@A|O`QTh8S|+~t+1_?mq@1Cm zXiu;AY}u@m;n0^fn&3!u`Q2sOv9b3#xJw^$?)JFU#BS*Hy7s&hpIZ!Ce4&y*Q>~9b za7)-ZDa*tVVgi-o2rC=x|JNf&Q_X!)s5L_;V!DFWxyy99j0w=jRWr6*oJ$OThTJS6 ztFihvs}q&6I0K<^B28nu)+)tI%@5IL?h_i^pcU4mxSd0)B==Uc|5{zj+_bsPT@w4XcdtA<8Qa||ZVtm#Bu$M1p%6=XBF4(UulSHTJ z3hqNMAGZPp-09z@yh0~q*87Wr+;VRY*sL_!ULD5UX^o&s;0|YVy*~D{lS?Jl(I_wu zY=Bbz5=+pDb=mxnjb3oS{7xgtn@*r>aT2`Lh*fLxg6qUQ|7F)TG_bsF4{+2Mc=nF< zTyCxtT*uQHkxBmPBh?N*7AeMGLCI3JJ$J%SS$*gX8(Nhbf<+OmbT^>d;E! zuNj6WlXI!NNAW_~s;(u**P0%=oTmG5HP(*hymDT@uHf@+~TsZr1aIsrkM96Z^lNGZKVUy74t@ABsB;y_Z4zDJVM{DenZOzVpi6nc1eGG5O8? z$1wrez91eAQrT(HF~g)8ieKT+u)8_EPgL zTCh`QAlMHcNlZLjrDGjq+frT^Vf@uhv*_D${K&HAbP%d7#Uvj%e?-~t4jGG)l&8dK zeaM{bqjp;+b^PBRa)@F=7VgLY?x*@$(bg5s)z|wB^ImomPfLSU2SsZI)b|Z<&}xSL zP^F$&@2*uK`|dVB>p|GM!>`RV)%h#Z%$fl(>T0v!P7=1>1xYrdluVcDD=JDi@+xprp)Hw{f^?hJOh<% zTIYz)H@Bm>VqD9Y+!cV(L|;|&r$1jReLDtqZ-nyC(A&d2Z#O$eB=J_Y+pmUAhKWH{ z%E8#*hZ~rc|7yIHw!S9cy z*3|Tj)K@=lD>ljXMaA9ANfTvUk?J94akl;F+i!1-5W3v~xeMs4a=@qrKAEAXmltp; zZ2`&#A(r|1c_8KKy#NsoXm|BGqrORmmtS3dvNhKUTsNRY-P6kkKoiIH(oS-2E?!(C zFsX`+djX_}T7yU-H<0!$O-W8BaQDznRU^YJTcbax#kjxk#tzQho3e_pyRiA>j8cV zXT<5*un>NJX0EN9*j|Oj{CGt~yL@W0V8AUVioojT+@bqOA9sBe;;YwC?ypqujXMnv z1yVb)+diznbj4HhR-(U&^_is_|AHH4pFv_GW9@`Z?6&zIZ4d~o+09vZ8Z?Tt`H*e7 zKCl(id%L>Y+WI!q;Xz)n$NaC(XqL|hFVgr7zc(JQNnzZH8ru4y>Wb#jOFf+Y?AM-b z&9>i?1O-N(b*^f)w?!j<8LJ7lM$nq_LRaXAZ7F__7YjkAq14~`9WD9UWT(ElEayh| zXI9x9;E$g?k4jC^|MNzuvPR-Z@A&@LevNG@-l>Nng6l5(YqTf7vS_qNYivdf*FMaJ z%9~tUd{o`#=Vr8!p^)q4V%G3GRQe8bwZP-`+0L@1i=fNICsQv<-aOTX(mzO&d zHpNUjKHrMxAX}cClWOH24H9tnOz2Y(So)4(f(7{m3zmva<=ylLno5f6wlV{?sXj{yzth8h(Sy`l%dJ)ZVNd^$Z+b)8yOFEI4l6N&D$3^xq3 zP9;@!SO`=#!g+zdGT(yPLj5P2_v@>$Qp1d~A{U>#43Q7N97N)cT%CQEq_rKiO6kVa zS2bViW8ty;-l|qhZsfo=d!XiZL~JhcpkkSV;9Rv~jl>sg%RA%~!h2YC3L+CP<{wg$ zj`n{Q9Qv|po54Y!e!9dOLvyk*Gvzl}v0tTHP;_urn|5XUV0=z-<*Qr4`cgObkzHPk ztow%*?d7(<=#hZ}*@{68zI{aYAT;g8r|1YE(VU^zw`A;r?|Vuf6NC&iBO{^@`on)g z2SL5;71ZMUwsFpuH^=ht2)M63sZf1f2thhJ zccZ=+r^zT!dh9(`tcrM%I<>C9G|!uG5pRGX$FBW#KZ%bNDTivZMCyihcuOaMx!P~v zS-WUjeR+XGUvph@SS+hX!eoRG%xi{cequ~SG@6f5cix~{ z-!wnMm|j<(R$KZ$=&>^76?;fyf>OMbg=WHI*g-T&wg8H3N-45m=z0Ko!#a%zJg&a{ zj0m|fKVR4iRYFvE%=B|gI<%XGDg%peO4mkccnc9w0zJ*<78cO9@cd*4Ght`L> zLug=YoM4c8@9r*cfvI2bi2;18Os%R?jChbt1KS~^qcZ_TbIYN86t2{sS2Hb_p)(ED zLWXexCdUqD@lp@a|>Fe zq2_FL23@!UzeY!^pkjz43BbBm8&=8KV;$w9|e86%%@pD(;sr_ghINd)aX&*grfhWeLwk8!?YMq;D#yhD$V(3>Wh z6j%|p7~%PQxtHU22`=scbukXT51|kJOVGORZa>#S+!nk=5~mjx!+CbIPyojf8fBA{ z8B1A@*mY`f+(kRSgA@bk=-6ya#*j6QJt_77?lAx%hVyAVu>!F13Zf0y-f7UL7_kvi zQ{%?iRL*@`gmUHsih)$&(NqTW?bweW5h6D9k?7h{%IU{ZmNlF?`XzPuD@go?;=||e zn)f8Co88_op+DaOtp8kQiVg!(uJ_^pzH|enm=X3ac0Bx{4eks63r@q|KYq$xY#Z!& z9V2&u+qFTN6nt2I7pwwkpxoO=Csz||HMfO3gOb>XSYZbO0^VL)4YA7>uHT~^7b$Ll z%FXM;9aA>?|NMgE#jae`K}CAC!De3UPw51~599)rVLVYmDgmMK1T{lcWMu${rTL<}-5M8w3+f1tA26{en#mHF~}Dy}^>i007Hj3?g5hWg^$i1qr( z9jFw8PTOrZHasQYg!8T&9}9DH3O@5o$RfWJi2dxWtp)8?)Dmj-Z~o`*&L0~X6fJXv zps48hj~}n09tX6Q;|TotKoCb)&v<;p&T}LVWZl8eZo4eRTq{-}2fvC-$tNGh*A34Y zQC#sBhzvs5r@t$<{Xro6 zBe+|C{`@mk@O=JY3Hqc!Vgl$Y;K?riV|BC^Z?hKfrB?jWUhS;98@g@9HZhvhF;I`o zU{@tpJ`g5&x6-ni)XPP5Z}}m8U(#X6tsFKir`eW2MI5g>hPCCGOBeq%8$$$FS$wsuCP*tlw=D29z*M#L_Oip?7*n4=0qY5s zC9FK)CMiAr?9G^|l5+@xV}XSDG{WIAF}O&Zn;aY*oScs?PN5v?;yp_m9l^e8h-}xt z9=ugbljXR+uQA=N+TpyvKpdj(>oWHNZQbb` zO5LUpwbq@yk1;QGXg3Q#Cl94o;PLFP4(cQb(OGxPT2^mWomrLKMmtp`AK0*TJGG3Q z`mK_pv}Zg;x-1p6-}k0Z z*N_hC57sT8DS99El7HJTQFZIA^!d);Yco_C-(7%9og24UVEW-?oLF)2gVXbn;2YXL zMHLTKo*>(L*qCcpkbCF;6lEumC?3Q}fq;c^M8Sj^PyY*az;Pgx?28u%<`u-%OQ19S z;DNt`^8x$;F3e#e<_%Ccpd<>UZJ=a`0*M34vubYFTA`yghjwam2dD#gG)nJqb1VP5 z)}DFM?_BnB_7jbo%MjA>e4EL8hY>lypSUc{?CDU>Yhwsl_ zxDnN|uIhaKoSS{_&5n)+?{WN-_Q6V* zM>K+E!pl~-%AKVBln80<&EgkY~Ie|*~>lc zWjSn1t&LwigQu(f+OJ-yvTZ(5Li_yY`mu7)A!E$)UBUPsO1BeQ)g!LH?YHeYk9&6u zckqmS+Lp8e%3Rm`c9I|m=$z0#T5m_A(9KP#HmjfJdH)*Yn7XF(q%M;^)Adh_^!iCa z3|Z-$untu`r`)J-mB-9(IohY^$JBj=+jlsha68m~d(SCUsJ?an9+<^WrV{6~^*lw2 zzR9jo#lRP}gUM@7*PO;YPL;;tlO+TS%U}1};WhF`2oQWzS*U&tgx`L-(d%sm~zcSErvf8kFuntESZ{=B$-;3ssIqkFk z6S17AVaLWhN7l4wDItw%{EA1>BV(UBvQ+K2>&zDTx@Dihr@fyv-6~FoTnRkOQ(y5evVj@dwq15mhAF^dwOgJlsWP~!&Wjn7nigyfC zIWNvsT6&~ppYh;*@3Lv>9Ym#2SX$74v#a!i%k5|um%}o#<6gJYbRFyZ@Kn_x$NQ_R zS2WHx_i9r*Va<-#G-~(Ebf)?!NFVO&oX+W-&P<%`@}C`8<{htzMGd&_HGE(b*Zz^P zO#N=8{?PJt)lKSfFm+bL?TEHVU+Yn=R6yD3VbziZ<*&16_)_PgDC*)gZXf3Dy_Mnb zAU*(|c?@EbVU<{*tdWtK)RfQoR8=C>36OH^tVrhA74pSLxVGB7B|mmU#<5#@y_QFK z&&axD@BZc%j8}}nKNIe+?b#(9C&jmo6HKQ!QOj8QU7r<`bK8Mz)N+qY`ch$)oJ05) zq)QoIlwTazm3~EwY6vFWFX)(5LBKCEQvdnHSaqqQ;Kqcr)g*^ZVBYzDdQ=U6VVT;=T+OfeiPXCn0ko>P@t}9ebP77su=51&WD8D5%xVGe3&8=-y!i z{R{?%8N&^mi%?fv-Q9PnZj4>s#oATc_yOms8oo)Pcy}#3&ET<~W^KBQ)>PZ!4>|X5 zDYB=VaQb$aRCd8!RGjTE)$MT9HqS&gW^?Rp^89hSn&v#Q_QU--S!C9J#nX(( zYp<(XX!+8$!HDy_2tF08PMMj$-2MA0djX&$12)w^<%?2+mBbF^Qj~Pge8A`2T8ML+gdj+n~9v&TyE?+~T6Nt%t{O7emjg$|6m7t5AdD3q$Ighe? z&f+$XJ4@1TY$sE36=Uu+ybU$hK_jCjstdADJF3O8!EdxqRGaP8dw5$=cde zs^H1eD0DpFUw(R<(?J(a%^}&%!s_d@>R<-6pmamdhCVH~bTf$O{>n6Rv%4cjaV>UOTc{kSEE`>nx`-X^z$jU6MoqT zn|KqQ`M7tlqx4X!R=CP#!btuz*vB9K1c z!{@dJCw{jdKjUw3{nu@rr)iE$H!gCIi@@Y>=Jq$i`kRORpCkz04p}w<^7r{i0M>!5 z@b4ehf788x!q@*Mg8zBpZ+iZ}Z~OaQ|7Ps}WRHyQ7y079zXLyU@sW${^Is(5f4=ko zm16$Ag^LSJ3HzV7{r8$LjPt(<>cwqd{J)D^$+`H*Mb3Br+5cwc|2&9`hxHc_LD^Bl zSOmNRX8RO}yp@Es(q+4w|Kc7U6%o`<RM;{39o`ebhgvdm!m+ODtOpn0DgVINlbPx?>-2jt8hp01%^#TF~ zDINs@0V+F*BTRtkfW+>>Do5GwX?AutKv1ChcL2HTF-#W<1ll&VOq>ff-)J>{6YXC4c3Jqi)KD>$n;ugJM+uGVVq#?s}g7HTmVntmJ z;ZHFa2M%Zz&};Ndgbu&)!MB%umeDr)zQM`%FlR$_wQeU(C$k;M>%$r;HW4K z05wH~g~|NG+)B^QLNFujtEIt9eXy(?)37RjxRgIraM>6wbAIvVd{xH))EEEW)Ky9s=~CwKdJ{>=fXgSQwB* zgq>+EXsBNH(gMZAgn&Jz6ndwvOijohw)OP5j%lgU5f9*9z6{Uc1CZ!#6^~5yR8&ZE zFuIxL!x4J!nRMokj)JQRrSOK(TgsR{rBoXVuyCz+c5d9f>HicZX9OlPReSZ^lCi+0-~o(RLQ(O< z>!7-qY*#j)vt1lT#=kbLHkm@i(Kbf#wQdyvqJT-LwQ)gnOC5$o?ZG_drOUyXg{+8b z<6at2$1))}{OWvQt*fgG3fAnz(@?R%G`! znJ)irTobZIk0_v!XBhV<%iNIttOh+rzzpBkiV?YU`NF6NGyV;Aw#;Cg!6c^H=g?{d zhV4@<$Z=y`6D5y_Gp(MhSpjLL6&uemY?TL`P)k7VGjIF^NSlnJ;{5qgn7TC>d=`i3 z%V$8Zg-#QC318@9vPyvJl19e@;iTdmbaBLs1){5UKWDTTATGp-#|F`FA}?jQJ;fLV zmSoSL1PQwR#3qZG$FFx(s+}vwm2Sc+F=#K#|PR?UM`2>TI-+T!1SRxc4>=l$4 zEL@6#TrHtR8_=fI+)3MKVFnH7KAik7p-@TS78Z`yeoreV3hhxsN-NNYV?;M(14ua7 zK{(XsXfc!_G$WI;QB0Vnr`EyTg0K=`JRit7J((W4IXfT2`iYzA=;%1U0uUJ6Ok#N` zu%_=xCj%8&2(F;2x;eEZ0L7uN;JFqG5*94#HO3D-O1_nxvT@Y-0yP*f1H#<8wqB=0 z#?Z(EpuFBClViEl>KlK33Al~JFQ~;O8y~@vlyk7L^)xqQ+yk3Ka{AoT65qC=YH}WW zqoGASUPhM!3_=mpO<;oCsc|qeHudzp1RLh7sn-e`S%VqMSX`L#8MyhF9ngF;e6Uuv z4m*6Xz8)ptSGi(wMl@wl8Qr(v+6n}c2Vfr7!_;mbUtD~Ad^M5{!wkfXXMy4M?R4<=o)xTNG{(hHvjEM5W(dUU^4T+x?S077R%ua}#kG^Q*A&;Hr# zK3{TfqkP&k`Z&fGaLi3W<`FleQImZT4~*I-aLlyOAr?q2Wh^RN$3bLBzI4M zl4+x$_Z9l}w1R+tedu`%q9b^+SSHlyMzlXX(dOGK$|5he+^$KQ>(a1i1a>%_StRzm zU{mFw<_qvgV0B?kRAxmhj=a}avy13|?5Ygn#QHKS4X3_$QPDQ(*-a&-UI{-uLt|so zq#9W2u2_*OC|l&Hmuzir-UR@7)2#O2i-6FlS;A-B!)T%NjDZ2;epGF@@+-s68R{`_M835fpB)&A#G|L1BiuI=J`{xSUjxyOs3_y)WgZuUHk z`g?2tz1oY%_3yL&&()rhA!|<$f4hafeRy`de#Yh=>Nrh@w)kk? zbik-q5Kq*|M|{z&u%3g~qfiNaTG8ZL8?(FJ zkZi*ISFbyOoMetp?J1Oaz#4Ho?wDBjgGRFG$&Rvh<4li;&AdEZS)0Lu=L&d_)8DnH zFTglHxi>vrVPg!!b3o5Q+9>yH?kq)hgQaeQ@FtSqa&2-FNjg$ySqt*uk7JgIq@g^q zHa-0~OC<*aAE5O(vmCj zT%kb~JR>NWeVMGo0m{Zr#$PZA@*eKXD{%DAJz_BJ2vm4ie^rHT+&d=c zHG2YMreJbiM4J%_p+opLF`>3TT{8~xd2nu$_pZx>@&s7h(ey_Qjs?&KI2oPy<@mA?X+Q*e+B^wlSSMxmO?xajBW3tAo_!r3U1S3#Y= z1ZMN(?*qMv={ljd*wIQ6Of-s;5BdDL7<5Ce$2c8N5qd`?suc1%4Uo0l&Mm%Cc{U_N@ z*!||j`|s5JG@+t5Aj1$uEhObr^=6$<&L1l1@_q?zsY7oabsam91oMe2V$S>doOt?T zq_DKaX?=N6l(Gqcsxh}VPNJ*%eaB&MN(lY3)e1aswaCW_yqrxAv%Qn6g*c#v_VKi}u z`yK9>^ycj2CDz`yws`fQKM;ZnA~3HN7SU9glTk=03bjkt-vOdis3H5me?QC8w+)rX zI!ClUNCOzhVgz1!-h#=vmV)=2@&|gC-jT91Im5uJ*at`DqlA)Q0B+20NPDs4IjNfw z7kahiaBp4Aa$J}KX}}Qcy!%6ya!YcSzKZ~#|K(U)zyR8BcCAXHZGuSumZlrZ`1K!e zHdTa2D&JH}gZk(_P6~N{CVkb6$3D%M<%lZ4s??gfGhP0q?w@Z!;rSd=HI+PVg$8#F zSfryCN=qCv4um@s4@6gC<>n|Rs4hUsXj7ZbghL6OAWx+HPzi!(bl@2j=OLX4KBJsbA|xp4^%|R-AgBhNG)L8B+P__%^*ME+Tdn=SCVU{-$0p#xkIPYHbB;d zGB8<$Dv_KK?YB@N(RumxR_Q}2{u}Liac-w~Rr$%xOuoFn2QHfP6DW?)UddM<+}It| zDKq=g(bB?n^QH-e!Emy&bZSLc6T)GZN=yv3N}>-8&gKjVmBe8C3<`vhMGi=@B-Ve% z$H9pfapJ2u_@SzEgTcBIs01op?Xk|o5I}Nx%7dz76;QQM&U4_wUYy~ zv9sl`+c5HFsViAGcDgv5a#M-KTbJjBs*xm!%6vjcH=rSRH}ahx?Yi|f@|iqBkBuWB zT!_S-iS4eSAeRm#Tc`;HJnF?JC`;-zI7uwn$f#k3Y=#=nPfDf_oEZ;$MUi^;hYjklzk+uDwqNSe0-r<&Kvbu zR9yV~OP6k6nREh~P;{G0Q1E%SOpo-~H3|@gxg?-afa(9`u2-|a?lHt=#pc7oZFnzIY$+GKlZ9#G$<%i|?zDbQO-Au&woC(J-#=%CipS(5FS5}JGM%THPw3e| zxq;AxDU-kod>1Q1Vdo6W$d~$j!!ypHzL|q!qUUh0nu0Xls>qF!>f^;Z!6&swlu zvu%({%sYifHc7EX2)s&!`O*;t6qwW9BERs(j`JZY!LW%c*+MDaW#w@X=TXUga;u*# zgP#@&aD52su>%C6K=KIA zDaHyP2)HwKqIRhHN|42QxHAJK)YJ+r_HdulmMnXa+Lgk5gPb-NkqNWVyFHTU!VEmn zk7tbAmroInZPVzFPWv+g@{INGY53))C?@t4jz)qUCm9~(=t7XDuoB7bPuot@{GNhS zO>`)O(_!_9=}to9)!Jbt|D+hrOa6Dt(7nTQ)7RnIxXs;VDn4^E;&W2VDkuyTtv6O8 z5bg*WN%1EX?WkTRQbf~t0*m&LAQuy@tUObzQ-l0mb5i9{SMWYvtDy z@V(j!Zpa*#pz%B;BPHcG?xDKQKsH$G=7wMcu)H0iaNX9%2GnZK6pyt&r@_nvuYg&6 zY@~9k`EVh4TrN{1NMw{SP5Q~kR&;S?FgRUl`<|F7{r$@G?Gr*=(lgn|KdPbxI-D676?nFqCo375Zu_K!X+VxFXxp1nW{jUx(tvLjEUUr zXGZYY#5H8Uj`;e@Vfw0^idhkY$52-itq=W*JAtz^6_T*Y1hXhy5XR=!*TWoE?q-@Y zQRAPD9Zks0lj};L(QT{}Da~Ml*lcQ&QNAm3-I_q=l9yxvFlf|iX6A1y$I78)BU$R3 zwiYqANnPifK!92iMzSYcpd*<89I>qt8pDiFFk^wCG7XwQiK5r+Q}}b9qF%r5c?DEF zdlDd&(51Q=A^Nx%{3+ZkssF-G8-mBK#bib4!2?}hJS3sSIf^mu-7({ORs2il-_UjY zo#Z`#VL)9VnLXNBvVl2^aI+8iVYDRwgtHAGdz15!2g=BSV@OKgWuRh2OT5c#l@8Bt z{u&;eq?yN;1iPCdSyyA)Z$v*(Nm6HFtX9u%X+4H}o z1|Mf{e*TS`2iKG!q*)BeYqE?Ts3XG=*~WAdJ6bM-OM{6ofYa3?Fe_&jM`eznaHPXP z{kg*Y`V&z%0~I9cfGVJqA|2Se6B8e&$Rc=5gPdhDpNo|mnv`E*O`CbWJU|~*$mF0n zAxtNuB+*Z23ZklnZUC2ET_8kIEb|6gy{F)P8`0JSP*C`RK8ne4w10h*p3y6|p584o z67Ql2CVAYA0!V@L8Mr^uNr-ge>0cvyaiIR5mbe*k9h}F^hu4tdFRYluxX%QxqxL;0 zt=CD!b@6_s@DZgZV(ByI8*~$5RlhBbViWOoe_@{R+=5tRPNaLH*THq1L9VkohK@(> zi&JZBJ;VhGz-P=d>A6AlnO(;f!iyo4I@YI|M0PWkB<`u$YF=ASB-!<^fuS_{{QBjB z4=wJUl13_*fK-YR&nf!u3=AQm2CS*HR9#Q8$AfcKYZl|Q-LnZuHC%rnEMhni-D6{# z4L8+rLJN$+ilV=G9PeOMH@4_Q)PA!`MgI`UD(zI!in+_9W$eS>gazfPm<2ieB5j!vg^A=jkZW|9I2eiNF0p5BGf5c7JB}WaJ?65amsCa zb(~Al`hXz}$U8DgKMsGSQE7Y2zMg(0+mxX?n(K^0pNU4;{$mNPjo*FV&R+mU5KyT} za4B?=J|vLhGauqG>LQ!J4}^{b)hzk8(uqgDp zBl5!Np8|QWSWp@lFxD|;_XiXpIW{_2VlK5Dl4tUjm2vueyl!Am@Jdn($O>0_5sq0j zUMK0zaOJ8@7Xy_fpW?G}EBZ~p65OOZM?n ztkT^zGkI&ac!$RU8xi>7gQyO&?j*?d!+3HQGm(AddT|Hy_9d)Y?k1f)%$?-!F^-e*aEZ&QoM&dPZeSWe*hG9Clvny z{Lrc4+nrLNC<7fc=unzfy?F5zZJZlD*Ti+`hWN9i;|7W&T#)+Ncq!zv%k&b*hXe;B z`?WBLXlP!>K5=ybk{-ryW@ctFm#5GYfrUo}v(Ud>eDl9~@_*Cmi^TpsXTAV9E|TZ} zuLL_e?8yXBpaIB)1Rcyr7e1SMjo;@$NHdri86l5`X$HTXCB&S-2LMgh4W_t^-~A00 z5BAa%vBTyJA{dD!3Nj=h%uwU^iPX%zZ5CQ`rdT?~4;c7@N3s^!PT)0}< zOD#}f0lQx~(cZq)6hH_wQ9$PaL@|J#-e|15!@N zB)<>2O57P`xv;4=W@}u<0UV`4o(IDf$R)p_w(_Af!zBC2d|g<=RiJj{xtVs zcNm6JfEC zISrYxy;ZI2$;IAc`bldAPd*?IsqKyePAF)Jyc=n_JJsj(yjRh zG89O=!R*x7Ea2V5&pR*K{d+GQYoMnO!Wl3tQ~!^9KBUetn9DanT#21M54t66F0FYx z!juaTE(4r)4W{RWglt2Ga#u*`oLG4;g(LUfyH0TGz*a@iLG6Y+(<(J5R|CzMw=kl( zycDW5fWbidXT`fY2PHDnlwj=Z^H#iN6bArIDk07RQf>bOHVQaIN=3D>HY>t5^EkxT z^baZ_v4ygV-?v-6Lh<-AMOex6>aa;Sc6W&Z9sGN2Vr3DWYM1BF=RS>5fe`tSAe8gq z5s(pKB2j(nqa~%NK zfHVm|aHfIo|A7Dcb$wU}5NM@M*Z5#wzpgh>sLpc$Y5u2LwR&ArGSP=e8FH zyhXm*-_RqAJfBBy0lSrkh6YRuX1pH`IqD}HlW&RlKQF|{(rp3S77H3#kRm|sP7^j6 z6@K=AmqaNAilJ*@Y_D?g2t!Z-5c|2%FC2#XgLxol!V?Ci>yJrLMuf%U#}|Q7!xGgS zY|>zshHStg!YL@ibx@{cJgIfqE6RvIfrZDchq*!^W&0wK^T1``_$uKa87c4j)D6GT?F^38gz zJUzg<7bfrmUF>JS2a$PaTU$9yqpit%2|1P}jQ@t|=U<|tUYxkojYbE97}$;yt|-iq zpr`jptr7m&C|it$2DMxLkb7>uIB1VCpR3b0JFe6R-#=Trfhh=ZC!C5~skG4zw)PG1 zB;QYvj9?$yvn&0S?pCF+xx(;x$0D@^u&(C478a&vTWC`%7tGbLI*fY7snhhx>bliU zA)zXNw~e5802Oe95|ED7ZSC9-_1L)ILa%!3bxY>c5S<#-?kYVXc@`BBg>Du-Ygm+4 zK-_)XL`JSDpGqICGwqD-j^^jkS#`zBiH!DZ^M?248PY2T1#M=b{pK>`8hdX zfsFu#opUR?vjb9M@jkFU+&-s4tyX>zh+*Iv&MoD{#2Mrq$;E)@K$W!kdF)XB@tynM z{F_Q)W5IUSuY-ym$l{Z**QT7ny2GR-OUPN)!GJ*ht^T<}1Cx+7kug{eLTNx8!|pt5 z;F#iut8Du5%Ji!X>_6Mun@ik0Jm<6NblgsO8(|7hL}Hw%QpW!N{@?0ufBAWj!63|3 z$l}Hdz=3*RxuEOI>n$LrcZta;;T{T=Ym@)A^Qt`E+rn<3D`IVtWsuP_*w~VtD*92&RU;g1M9&G4IbuQ}mAmPy4|S*u0vbg2D5x z1Fi_=cNsGDMbZ~i6dTQ?I=Y09iK-bS2rrteCPEt99IP<4=(hT+?!P0^^8;s$_SlvM zk~G81?Ljpf6Gn7DCCXqfM!2YpAdxk*5?BS;G;F{8Fv%|YUM2*y7H9(?X+GHR29a-} zeU>#G9_TIf4QvjcojyK3tnGw5P`PZc00TTr0Gg7F7P`i4>YAEi*hcgpA%UU%s#GL1f!mMqxkAXs<3nfkvuvR1uu8!gCDR` zMpoP1@nM?Tm|mWqR?If|_@vFf3HIQ@QeXIGu|5dC-WIO~0+6 zhWNB3%%Eh>5s*$^6+x!2N$m)O5(({Qew7PK1jGb#KbKJPDi>UT|M@^4H}}KIUD-Ze zd?TO%=s}eZI5^aCx7hgnd`<6O?P@^tYhqpEec zlznuw;-7~>yb%R&pL^=sbUh%ailS5UjuUp+f}=mlXV&<>JpGk!10J8TfB=S{@jy6U zcGwV8DyP~!G-(MBUWXDnAxd8Q!vxe<0VFj;{vO*r79*dL;^lSS*BKa0&tR4U2ZI#1 zn7WKEqyD5{GL6X*@NOWYG>XvLP{cKAswThGfGw5eg>#kDd#X+@fRZd?WP=|SgU~xG zU%U$+4{yGIRzNdGjFV!Nj67&gDOU(GjE*SFAie3U8g+XBKyVrSoVf|7fz)}AXut%H zh_3D=L<2G(=XJ*CrC(nD86@=W=g(3)WLj!!u)@j7NyG*S*3l0JKXciVe1fqH^#|#? z>pj8kP&<)F%~ejrGj(5unD1KZFgQ$%i15k#5$8BregJ!pMj2C6-EG)~H=3@>f}b8f zYdEVm#O*U87H@*_h!_=8AE+)PqjuY8);K?UsA2Y9lXGW!N$|KpOx|5wphF*xB!T6R zZabLFf(?$;{Gy`9{H7OO{`KvjaZp-}ckfY>JyVe!%7F|X0<-#;IJQV6LZM*`-Il`i z`SZs3Str>vGL+&1%5}BT&Q5Ip)R}zMgx@GQJ!Td_Vk#$$Xdh`D4qB25qEVO!sZ?4C zWbAXIp%%^ZbPy`^$0Vt%F%am&Yn?2fRSdqFMW;Bs3}{>Ojd-+>HY%S-;f4k;v%6Q~ z$yn1ex;~Jeo`u9=^^vHyI&or`>NAUYCSGy8mMZ{pMn@gFeGD&B0!@IKcX9}ae4E-A zjZ5vDN9PkfD?($d*Es%JjYt93+r}g*&;`{H{vJ;{gG6=VW=X+T>yH94=ILecYb~Fz z>G=}4k=0DkE&oJV{K>!5G~#~iq*=85RZUfAAM4botAVe=*X?A8M|0G=fH8LNEW5) z@;5cK-+>U)oweE;q4|M2S(^2AV&8_%d`uLN6)7@$3M-<8*)e?4eRq@_tooeq&DR1Z zSBYlcLQcXUlt0q?{{9*IX9y5uZ3#z?X+NqV(doIbT~d>%g4wW+j)gpBpLy^Z)d-Xl z5DKZYlGi;C#7KQIJmQfYH=HFZ-LWHr=&)-L!w@>s^=&UE_51x-H@h)-n+mz*4Bxe$rh1;7OZ+MLk2*_D}5sK6MI$-gPyH%eO$;K9zZmd9!F|5k{=h=qy(JR>uN&g}RES7hLx+ds zLsX9legij+?OCCBa5N1GiIL0$0~gG9JVuJ*jCbD_EbmWzO(V6krwdxSR5rEDfwJS? z3eA>4^}n`&UT8`jKtj?wK)BWPD40LI!JWdr<+Ulx929V#eC^V{8ZWZ}T&=oGm`l3P z86 z{K-f{^p!$Ce{TH|ajC218iQ0gWp1_xnf`|lJ%oB#TktrwHwzua-ID!FOTInGuER7#O+|EOnL+^XM8GkjE zKZHb-+v!rIWHp3(D8>Nqq2$x)Hgtf;I{dRLcUp8z*+UV3nM4x3LE(Io09IW=T!hkt z>;6b?-Rog=R)i1Jy(J{IQ7aFqwLGyOg*#qi`^*EtH{;v-_++F_XC@sopg{~1 zN&q`;ecWS8X}TDP)4zXXFFvPzEQ*flX?2MqJWj3gPH4IU?mU`AxIRQu=r5b_>1oJ65DT|)eLrNoK^eHkNf1g+O zce>O+*?VS1jV3~%hbE~;G}W>K^Nc-tk=#N`sgD^#I9O~X|nCL zEU$5={oc277x>I8)oc%$07SWp^-eF(*T#_#ehx&!uu&qozl9Wg{T!`bbhh-|Rp7o=F;jcFxdv12s>Ags4B^_WHPe z#eU@CEC;+}uE61KmIAp!^h>Bz2_G@owxgXJ(Tx(hOalVI@&Y~duAVIxkcA4}Z(9f7pJj$|U#r>F6pOmI*> zFt*Z|(Xm`xgN9T|6K>j9Nn`;Y8@SKzc@IczA*P323(EL8u&KR^+ivfCYQyIFcv{;{ z_>0-$xyDDlsC-;2v=Re4Yq4w5Z4Wx?Rg$HyGY!96Jci;b;UcFHefSNmsV_V&G~0TT z*@4t*Dfz~LbcTaad8Ad)5?K{Mv|f0Oe7^6;WGn`{Xz0e>=?s7aBZHRDhbFs-wf-O! zN9EO~Mj>ei#zw;mqZ5t~GnwS%O|5Z=^Vs~lsi~;w&>k~XGu;373?xzh*j!nxtHe92 z-|PzHS#Bcg!r1WCymZ6`rU{ZQ12cCb9IB^NE%TI^@gi)JoDVS<4DO_(DBFA{c^sQ8 z$7#(xXm}?*M4wEIhv&15H#2EoRNwseWdH}t1>fo232L2Qp8r4{YOvHW68;6J)2J=n zL{}lpmi9ZR4hP;8>asqXa|FcWBrQKYbU#cVf0P8g!Rj?CMy{fFeKjZhtOwPGiP2s=w?>Sznk`4ZJ8M_3g0~6j(6{&)#Z`8j?g3=HqsjkSF?2X{ho#g zWT$M7gAW`X87ZtcU_>S3?l+(2E|$%1;o4{Z7Ih-|=s5rXq3o^0s%pP(VOmOBI;267 zmM)c)20;*{OF&XU5b5sj5)h=6Qt3`fDM3O?Km_ToGq=z4zQ6Z7*ZHpVeg6S#@6BFo z-D}=+jycAdPW539SyHd@uK+j}|~oLAG2J)tf_LGP0Z@_DJUZCOCFW9b0&t$n7{A1llayIJN|Wrh2>TNzS$g9k3@U>TK~uN2NSS3imqlFO7a;Eg*JUlgBmT!{2KX z<0puvBpRt)`s(>}S;cRqs+P?MM;a7-PY!jGhb)skKVGbTy`O%ud-h!7_YLbkPs0s! zangzQpIn$_14U{-+Uso?Zg@$EWS5Zf*E5@toHDybryrv~o%I-6 z%9t{F5I%+D;_^GM^Xs_x*h<%19Es`756xGl@M@PggXhb6TQ=P< zmPy?-cd9*`I4o%Bc9jGC6N~?gm~<8$jzBx+J`*_?aw z&B-^-SUQ*$(((pE_J=miRUemBis$B}DCu;+fZ!kJ00e392I}jhSvSCXKn#gMAzvMF zdOx8X=*J2gYLNuXKxi)bN{$7JY5TXw~ClY6dMcGKd80d ztbjd2L4N*nJsU#I0DTwsm@T4aBp(1RzKM@>LJoW? zzyXHPKMniw-Ht7(fLI=4JHIcaFbeWakkInT&M;An1Npv-o(kS%ubs(XC~Y-#Y+ zpZaJ+ww03U-YP}1KGaXyZ~-VISb+?J#^Jrn(Uez4i6J=pW|vUeuM!AU%tdRd0_Wq# z4Re|(jymrv2}j%?UG0z+hkOL!X_9ShJk<~|>YQRja;c?hz{Fctj#;TrOwil1q;VOr z_}V-e)FOCHSl`>d#x@No+n%#;I$3dikKnKYr;ASH#TrcL9*Y>DIPzoeDxDK@>Z;^^@m~a^Ft*mnmh?XFs`iZQXbXFeK~)*}={RH0+HB+}%F< zj?gmDuN%g_-*GdTG{O|Y-hA|ef$atGaREw)jsZ;9wuaIaFsD{b2AZirNr>>CuU`X1 zeoMQw#%@-`?M?c zVN*NKV)4JYf&HuWkavlL1yhuQB?w8od@Qup7~ST9E~_H$pQWX}pUH-&XJ=K{C)JH5 zsH;PYx&w7S>ul*KvuYQ2lI1w&1M>*jLPWRA@BHaLf156J%gm>Q*x2C!aH^%%h#sIq zM@I7yCUF_6^iyyfA(>hh#T@$&EX4T{qPFuI=)8hfMMmEZ3PWg2KYCX}tOw5p%$R%D z7=IVBZ80z~gmFoY{(y$a{5q%t07z*Y2k5!s`LuwNT!)d%SLlBMDbr&)v==1ldLwJ z5&gbm<$Y41^ooJRuzt34c~Gf3LYnTc0$SbaXxn}K#O{E3fe#b z6@va~bnzWIopixcpaGrla$%u=g_*#S@O!MN3jFkU1(6f1z>*%OmnxYs)=SIC@JvIW;C^ZK zpjmz$v?AkMV3xs~c3A90Az0WLa+ug~4KSnu8j;yxG5It7`2*~k(0E(EW6k&n>_2dH z==})%kgeojL(WXX67!WiDv3jvh}!J!Wd`XF8Khw!4&>w}VugY0D)QljL}@&afDI!g z9nrOc+X0JoC{t8!5qVc>L&ANR>F1Pv1=5?OZRItlWAMz6>L(UzE9WHu`Bh5hDR5Ej z?d?_HrvXfbV6T)}O(;h6C9rGyV08tC3?cELjY*m4z<9O04z;S17YWW&oJ~mAMMgEx zhi81JYaAKH1J!i-hVPP-<(mriICF60K}31YY1kA|V!0CP{Az#A_;ovx&C(n?4npo zD_Y`mP|hjwi{(Sf@bU}VHLXyh$wTPtDVd!Po_D^ep`OF^btHM*jVHuJg-gnY^wEge zBH(~1?IwUen&sRDv_Myq#v?RgUZeP($Yn_T_!{rL>GysPeGN?jagO>^2gVn>%N_LNlCWqSC_|=46ZI`RnLrR z0+Th|C_cj>V&fl`5|%1rW}K6h3@q+ z#v6aml;s|E6m*rSSHz}{^i?V=B)^w2H^5m9b;#_x|Lw)M95)@sWW%s27?Z-$uCJgk zN3v>0bjWS9nhm7#1;^Ba` zQ1foc#4wx2TBN*#J@wXp5XBqcu^zFZYr`4jBqY)=d=BMR&~QxHe0`oTvX9cDEjXcB z<>DeK4?lxqBQ^S>L*Mx*ZBqJOARi`=+FelvtPLjfHo6Yo?c9)GC&I|pZ^`!GIj;`t z1c}KNC@3j;&eWzvd8(dvmy`AZn(W{55NWpD81?)*c6o3$+O?R9d9k3l8l&l8GoTL)qAIo)8uAhnQw=LtKIEtQYO_1+)~#(8{){3k1z ziQKxB(C!U{>#wZfi4NTz73sn|{x&-nKAH|sGVpN@d)=^^ca{OLTPTsX zSMvQ0Xpi0s-kZ>ie!@juR7z6PE-53CMZR$BMXbaJ8T#lxEaeU^+Dj3P^lNKh^ZHS+ z3HEFF$xNP_xjKnVU|f>pb)7m!hmeR%RZW_r5bkF;HaZj&PbWg%v?#RJ z$SoJU&&gJ$!^j_Pz{@r&b6`E882h!C1|pzlp^{mc$4VmQfx)F(Vhse_3uvasDcaiF zws7Grs}pe;Q7B6vvjdTneq*lQ4WF86JhnaM3Yu5a#)Fj@YtRP+@POE%7R?dA&(y_GFQUJ7E9{MNsJt;K!jwoUQH}X%T4F)jB>wt`AuS;5#P+&j z#__91cA+_m)e=O`zbR$PS!bIi*XEJ1>}0`upohxei_$lYt5WFI z6-Ag|{u8Kx-M`SRO@sy|0c;?Y|+}-trXXaOZlL#q?ADNz<86$ zU4febX#we`1q!hOf--Pcxy?+FA@MrxdRs7h&h^`CM`%76W1!?yVM6E1d0^Fr zGQ2_okN0Q~Joo+Sg3&da9k>^^$Y}=snWD+n%HCK-FqEFVBox|ZxzUqJ5Iw>v;uuP0 zF&KVBZc-isS2mG*)0bsL|c7v9)Xd^6I;N6ffRb9SeGPO1} zHT@%{hYHdW!IVCQp_}(?Z)_Q`JRu#gO(WA1_3k4(ku!eewk4#8F zWC%JM=6#H+*%_;V)s)5@j9~dY0Wy2fn&HW6K~|%AW_z$c-XX2Y%fZnv_C_(V-?|O$ z8Gan&cPCE5`0W9ez1rJ*k!U10`TfmpV+cVbsPYnJKF77~wo4jiPizMlB?d;U8@mFU zyH_-^n7t3nh_U7UI}@f_QaViPx_@Nj-{-7Df09tA_m3LK8*1X%?-ls31+(ukJ~UzI zM|UvNA?2RjgI<~s?_KUv=m^`IN^wMAV4dP(`v+e1x-Z`vM4>B+DQ|+|Wg6&Zxy)m| z^(S~}rOMB5vS_8^zb4i6=?XXEU8{)Yhv8D6N=e}a+bE2Rq(`r2A3Gzlv;XgVX)NOz&Ca0eH1ef&3nA+dq~^J zuH}E&{r#AE0#7oKWb=^T_e3D&rAi-*;e~cOIHgccSL(OwKkD1AXr&GEdC)bUdUrBj z=UQM(EJ=aN1F8tJkLvi$1B@GEhDykkLvL`p#vS=^!`V3m-9i_1ZnQBL`+wpPD8C`l z(L)};Fdt(qy%Bv|1L-kq(0t=Y$H`OSI@A;DZy%`Ploo}tflG}WOjg6X8$M(e0V|ob zsY9bn*-Gb~?%(U(xV&Tr8T4YCg#!AF*<7=Rh%?QFCR>g|WT;K3Y__A2i(;gt$-CP! z2KQT~S2W*{2>jrCV9L#SKpY|;G8wNWzKT$Pj(m`poJafE(;q{39LP%M@N=fcU~lI=A<|9d-H{_gpJ zWGLPqL=K(hHt%lvviMZRwT^?=0Qi3mNRnqCJ&iV;JCdT}141())na2q6^r{E8Nvzk ztw{lx>6Mq4Z*On={lL8K7nzut^b6vFGLga7!u%C9We!T_0iuLhc?R#PYY!i&oK6Ak1SBe{wEEPJ*gNOf$Z#u0_y=N+l|c3zA= zoFd&ibt1X`-J7U#;0a6dJ6HVN2t403Q+aq|{dzaF})u-vV4uELFg;Kn`Vx9nOXewfQgvEt6VcxfVxD0>$iWWRqp3`z`n9ZH;eX0`YI zeso!@RlGI+t}MG9&ET3BAM2m(L;3li& zF)>Pk6KDUU*W;95FnMX7x#Xf-b#-Izxd%an>JMWJ%=_PL<#D3>-F_)IED$$L?)<5F zHsmq(Dfm^Qtc3Wc`=+GpleI4&i|)~XFc6QAV?3h9O*ywMKEtDQDbud{Js@&&ooeVa z^5{zEQ`aHSU8bP|!NY#^hE%n!h>Yd&nnt-<($Ci-UXq?HPpMyve%sx-YVM2kq#Kk{`9StHy^-HG4^z&Q$Cd8V^ z_Frpj|B;SMjP3n(KS?yzsAeVw$v|1;{PnusWY&q$-Dnn`+qcDUv3O3~-HyyUc=EL5 z>!fF2=P}btY_%N9ol9{h{k{EnqvyNHT2ANnQ?>SS(T3WxvaM?@9vx}BZCR($=}8`v z%V+)lGY#$NN*N2ztKQ5M^21r#uxaX8!;50lP9+c8Pci*tV{7f*6^*N-gB%{z4qQXBS+OZ3 zwKjAI#)BD;-#NRDE$efB)){>#_&%W|k&#?U#B3@}U-RV0VRgv*{u7t_jAtB%XMcJW zQrX!@XtAn~-JaTFc|GVhxL7s!OPAixO;p|6jxYNmH8L>$`PCHs;}yxWpu6d^88xL$9(n7w|=e?hXwe7E7|K`o7xSOzpwH)XwMSOi0#QouwNeiiAr_) zxG1UH^y+T?@QoiocaEI$CBv>8Y{i(tJ!Wk^7#Pv9RsY|p;ZM4MZ1rbO_vHT$8~-Y@g) zH5bO{5=k$**Byzq7wxv}kNezOLh`93KIAw=M&dmEI5BZ=_Lg9cpfi1mn?du`+X7iS zmqzQo+pQ%(tgjeHW2C~p8C4l_&mnlcw|HthGq<~I|M)8K8s8;Ei89f}kL#|n zoYxG;^;mN8sXsQV&a4Ztwa)Twoef9NRaI5y;?vegWjzsJ7eU(W(-ilhCUJ7x7%TZ{ zb9IrbpBVI}Wa8~=upqibs2544?ZC%0<+6PYrL08C)0B^ra6 z^y{)R6mPe`T4o;ihCT@|GZn}VnLkGFd#AxA;UDmg$44-8%293i(fUYRmt5Xr0$gWpL!n?t5gY z#r@0}|6)T9o{J=R7P1B_^hBj56?C%QPA-y}5~rlDJr;EUcgOS7_QLQl%JT@FEj^_z|-qpVQf# z&8p7h|w{M%7jlmxYYH&Wm@ohUC8S?$nC@3)$Mv*xdzt5iIH zTXHhH>}9Q`pz3{iq@zW3vQN3YfnRR$EOT3EevT#>>(4kPx_AAq!lM_+^7ao~#Al*H zNve%>pS+YdAPL_uT+qwvor%P2%og_=d~Cm8$Wr^Y;&bv-(T7~6{B$x|8*a%(US+O) z@0-pK(vzyM;ivY~eoU9V{bctr+jgb2W|6q<^QJ}dkM$ABxdBG+#Kma+A&Yz=B(IVq0?cWp`fzHwCd0cK3td4r{uuDqq{@+G9M* z8*V$++OigN8A$AIDCgZuSh{}_$S$LU3=5?zcJARMT~kVl-Ee00ewsg2a60kS>oCuB zNWVXic&VGg`!LFH&+9?g%{-+o&rcJx4UtFJ2K3T~vO>+w-v){2*wylRdRkAmKc-f+ zV0pG*MMuo% z)(3X5c06@eypGq0(;V@Nnna2FK}1hI5v05&6q#AE?;fPl|Az}i!g5^ztw}s z$Cr_G+7Zjk(p%~7C*DIRJNyy~hmBNQu4OILilC=8d!#f|@-kZ^i`eSh=&B9Bwe#;! zhT6zpBxz(HKJ{;1Js`?*qdkh-_DkMt_>p&>e&{J~oZu>x<^7oGEe}31#f{mbZ}aZq zG|a<9R`dY`_c7^xlcaF|{)APH_5>60lY9>IFdTrNQ11AW{{4x_3ac9N;(!<_#sACE zs|TrK&aJ0+M2X|$62kPC-u?c?UPU^4&|LUvnbH`)Jy}R(>-($i#PjA4R~6Kf?yiHm zHw@-JBVOvI))SZbDa?M zWl^>pwTIDxZ#~*4?rwg!E?!+}{vJ84Wj|c35`PG1cTtiqKCU~S7^Z*LU93^Ba9BL2 zRO0czpwXZTk7$h!?X=DO2rdy%w8;UN+vUgK0!F+2=)On(h&Oz;-Ymcq$H;=GxTxo& zH|;ljJX&GLOOx(;XyQ)o@;ya*K^^JNu8``EX~Z$+cp<1 z>0Oi_^Ij33#prY~cRyNf*|+5%Hyo)+*_UYEDBmgURy@D0IE?eyzHI$!F>=?k@N`cz z4Yjvc+XYsoh~(^NKR-WEPdtW>{j<@6&hF-U%?!XSK?be|17$=-MVnNzCB(r_0u{hY z5SHr#3jjFv+9lcuZ59AzT>5IDvJ%AuloB5YrwYRy)YJ&B1uzz$t9n2S6$Fwl|Hx(B zb-B2YPfx-4?<3f&fWXD(Yi=i~JwSqD>t@6ZAioVjtv$lEn=m2*JOp7>28gW?O3AQE zl@(~}IzexqY`qhRB@~&^g2@zfANlS}Zv%oNIAA|hW4zg$14eF5;7|aRl9OL=CB3JBhl`Z)QCQq(*$(ErzP5VnmA$j4Rmj9C0`wK^fGM%`_D{` zoQuJ*8FKNCxOrKG5 z65wS}IshF1ax&6?JUx67G&vKw3!9ryq2(jH3G3)n$tegU5CTbPZ>;{-Kt#_gDtQG3 zUC=Tni~ETJmqaf5^3wZ(!_jvzoU$8O40-DK6=qsz*o<^`zJ2j)gri%3K?f71`+{9a zD224#82n6ZW~;aN_st?*9UP2;5&n+M%&D8yeWYH?fBx9ny_%#ikpe>lkO~Y9ROU=9 z_Mj0(WK>W(^%-Z%~gfdm-(qs|F{VFIuxQf$Cq5GTd$&Hca_;XxC-&$#^c7bipB2$rMU zFiNU(!B1cofdIDeu@WUcOQdIC0uCmo(97+4SRl2Lj=%4BrgVAsmAwZQsnP~~%_U=K zwl#Si+;W)(<_f|fk6zeGCWu0sk#j~ufrTvjgmA#?e5d=-J!k;)YG%H=Be4S_KDXZ! zkvn%LQIt(~ZV^T4^k++E%LK{zPz>5egoe_`lk_XOa9#ta#TDTq3uT8FFU(M?Ot?w) z$u`>WOV2s+50dhYe>IQl#2I)b`C^Tr;68FMMPwx(^&>SU7i*dK67OSSofQS7c7Z@P zYJf#TBCGb{lO852-8u@wiR%{Zn&RBtF~vb>yyK}0Fgu7L5*MwV2jtuFjdSTK|PiCXr9iej~`iZ~#KC1@=oXiKyl zT&0Jga5J4UZsKA6YGZvv+kPzdyZecCO!F1Z{3B(xR6z50_?vys2YS2Sl;}rgt-qiBAp=Ya;bE~1q#dDZ%k#-%v=t>Baa%UB%otf%LnpdW#J;UF?(uL>FJCx@m z==)@fMiP7_u~lFY()9CF#|7>)n0Gz}R#))tv4)n-lAQER&VY0#oZ&hdzX38 zB$~>5*AafF(jAn^;K6ycIh|ZPAmtFsa|%zStfp0V1=}P_8}DdDKR5l?nyo5tWSPs8B;o$FevZ3aqekM!K0Ua- zma|6s@?_3Lin2LHMJiwK*L8LQ`lGtX@85KsCIh8uTOoY*r=0>p2 z?mVsIPc=qn?zOdCOZhj<^24P&Go_uu5k`N-`-NkMDf`SuULN!4 z@r)g8VH@pb?{l!L(d}BrBg}Zal~hXUG9_Nu0c(wEYAnF&1zecgibz`VI@oDv$Oi0Y z*kK2Z!1*q5z4Hcl8*ICLjkT|K()_MD@@vxg$_iPf>ZL)eqdjaLxIqzpVNMlrLuXmB z`lXb@aWc|oFsq;YDmx))hRjBvVnnQkOUbBmCadBiq+s>a-1+o88wYvQbiI=1vxS}8 zuaQ9KSk+J54%Jl;WX|_~h#~M!`>M~6F&I-*84Mo?55Y38-+{mN;1^4iibT z;Y5z#3kX2_QwFq(xA&&g(TP*g7$!*0p^a#PT*&Us$Zg**>M)PwZt?VfYi34yG4y=?q$u2-G}P5l zUjcMmXa}+M&NN0yfw?e-?s8J^_b(##=6$3I?;%q|*U3>(lowzcn2CfW?wP%PG_uZF z4BC5?u`gn6p2yqtXWZih(-!0?d-aq~@`I`GC-fwzdS1Lezpb8&rkR<+IMX1@`_5SC zZj=zn^Ks+IEI#d_h#KcZDAWu#Dj}IRvjBs!H4cn^5ul*E-gZHVX;GF>z!E`!Ym6Qk za*$}z@DblI`jjr{UrBWX^V#RSL_~dd%!)~7KaGo*h!byV-O|X%Btu_}l8UC_3vwSM za9D?PigTBflT*svgfmE0n7Zz1uElzi@Dqr)QaMHaMfAGUJoa6g2iTDU z%}(qssyV&%feg+O{e1C``WkWqrLX(*%rV-;%2wCs|MZF{#AW5_cn!$3W+pIHL7Tj( zN%H*h+>91HY<-ytTA8;)Y@LlD3(?I?>6%fU%Kk7Wqa&(Ce875Em7|d3P5(?a>!rsJ z>;^gxU()73EMx=(KC%wF8rL*uG0@Q~JT#c|Y%s5lwA|1En_2J*uE=ugG{FFh;3{dS zUteN`4z+}5is`e~zP`RwMp{xr?Ku#0ewSEnZ*wFerA&IVA1IULs^cM)luqZSO}Y?1pY96F#^iQ3-0q?@=?aMiKCh7bZayK5xLN}Zbw@cQiE12YrnErjA#+sv!`p-B-waD*qb&z%)bi>qiV93X_N)wynd;nvj@ zXJ+nf&`WSFZAQ*%g&GH!N&pv~A7vnfTVzZ|pG0XlBqYS|&hU${z6@K#-XNH&>9FZs zXi0PPhHVEfLA^nv2mKVJXJ$plxw(U8(L9SpQt0G3E7lNeAYPkY(Dpy#P4I>ZgpmgZ zlo)=b4w!?TwySZ#GE&)vv4Y+Gg$`|s*q@~0(e^&=0&(1_QIl5lWeoF|`;}ivetKBa zvGA3MXl6>sQs71e>@gmiVIH%WAZup6P0^Y?y>97$s11oKEki6kkHC$P^0US zrLT$B>KM9SPV>+u>ccSWTiDtMU~&#gedLYA0JWCwvXgopP|B626{!sBj`!l<7Opi( zVQbJe14o@NMAFJ_WuvD!*aaerXnWb_9Uc;$_Yt<`{AmR>9pnHl zdk`$VUapU^hh2+nISsG)>Xtw8^(*135#bDYnfjYZjNn`uB`#wq@U;6WqMI;>-A(IE zlZ|nju9(-MPSaE?NttGnG^f+{BsG+c?}t(?mbnJW+kpJnPk}D!w-)4J-vHUnS5Zx; z9KxC02v4=t7@dSDy_-}6k1VChQI^ z3HnW01%d(tcPRarU$8`=G>@sR@I0kJiBrmzZv(!wJV&U>N8{pnJu%!!i@G#bDzma2 zUSkf2aM(zu-ZPP$&d)F-VG#^C%{odL)i0rIstBT=@rY8pM4n0xmp~XHX{S`*HH1Q~U=K1py(XK$>yk*m=UPn6m zRa}PWPV^}yE0D5*{eV#j#Y0!4{GJ90N>!9QfAPG4e^ETle-S%^MP}?5pql^#dI&aK zo14!Y$+jVP1XUFn;|q9i@0_o$dS_pJx`5&(62c02ZGd;z%2sxN;S-R>nvK0v=dWAt1}Q4vaSwleS_B0aw1Q$iq!3|xC4 z0D*g;3=qeo@Ous-vawJkws!!;OL$g3G+&B{*kmL& z43|J2oh{_J2qb1O0)m*wQ0R#spPgMDeDJ>5%f9Mw(Xnp?|KXLVSJP4(>8X9C1bt~RTFMpvZvjVeBUJ|R5$g2M?14G+s_$;KUUY94U1rjZvU}mrk z@baNBA*H0W4hsciVi*bW2XaNIRP828Q%|T-!+fxaSth5Z#0~Y-@?r?Xh$yf2M^#^Y zdcr=5RtGRo##;z;E6!$C!Af`S=Q8#Ir}=l;#mcqpu&8DMHUmX5tS*qyw3;61GVU;#WOIG}3H$--=JYG1@#$$Q`VNe2vM~h3cML8UsAR0NMksv|R_zta zQM}odFF;BU$&u-HE$c9biv8_j(iaiCxj=45EK+v|6M=@DEGf^?k&*r^C^FARvo8bY zHDs&pf6RTCfTj7(;ME=Q$p2){l!B{Wq=BwnIE}DfyjyU5v-!m(1l5n$4~t^;WlM&P=^WRa>)u~%KF5ZBbfh8OM@IjZyt!5XYHh+7B` z$`wW$*bpW!xtfmKT3hLZy+Nh|3$OR75 z%@m!4-AoN&9;;Zsc(}OKU_cH0Wyh=4k1havVOAgX&>SJGRsp4(g?b2rC{=I46`);Q zTEz(`A=1AmVS6V5FE%KZ;qHJ>udwL(roA`c);>3ivhe-1mkX<_-#$Ip+artsR^>lX zWJm+j_n>3&Sov!q;uY{~uIY6>loy~&Zo?S3mbw%SfuQ3jXntOTeacdHh#{C7f%~!5 ztCvbjN?-zcwO=Ty1&Ite>{jm$if-}S%?RDP^#yYuKmy@%$atY1WeEbu7qESH0bK5b zK@L!*iXRpE&FWnpB+%bV->SN*k7SSu0}IJR5OKf+uYF>w0=!AsZP2fsrY9swTgu-y z_uXXO*7S>Mf>n+%?y?YA<1J9hh9ya0&NvKHSm9n4foJdlYz-h~{qMyX5*e9!*M_wC zaC_lFAFHOO=E)&cj>mH;Jf^VXL2wjG07GvmsOG>+1|AZy9G5idz)UzrAb81~cMO{v z>g(&rIqq!CHP!>x2y$$m+z@AmZkS1{S78!JXcdsIx$*sD2|UKtpMWXYz;lF_b~+Es z34E$HU|#fTbn$K*z=R?q-&#suYU)GkXrTQj>V~}{}^#i&B<2sV78Zlr1Ey2dJlrf+6RNi?`3*|O$@d*yWkC4aj)Y8+%$0c zQUrGs2-$9bs3(HiDjOjs1^BT_!ZgwXMpKu|;ZAFx3@hSwE5f*uU~vdNSr3587lcO> zmPd^AKBQbMWv9Oiey4e8DZZlPErL?L~RByJBQPDdMT?xP49V0p*U!^lJV) zN|WbUR9+zuOd2f4gM#|8~Tn^YhOq|L4HqjmpYO z5NLtrlm}P@fC<&v@i8DBsdR4(3&G3J9oX@8br2USrs47RHm(Bw0^lA(VB+>10}V~g z>pX?HG2$4iA5e$=Sz9Bgq)baqJyARirhY}F2jjMp4f_Fu6a^bpE_S1v^oU&<77-Ln zQ*bLmnpp1j4*;#Vk@c|l7K1_#wovj+32_=i@RhwEzO%I@chdq zj0T$DhERdfZoOSe%7}^a3D{u}5lMU1Fh$229vS%>tl0GRGaPxnNa*N>YSobOIzC4{ zs$li4V4aRUf^=caA0Jp>RjgcAquQPWgVnZpt9me69rS9jBL|qYe2*-E9!EXEJf5e& z3JpDXIz_h{_P`Mvd}nY6n8{t+m7h^eRhrxsSTbnfM$UU(ZE7%j&wz?gz^ltQz}RHv zZrCT|b_`Wj)qaf73~pmTaAHZk{{4l61DoEb4QM++q<3BC88?)%*B!UO31GG$8T1J@0ggBNB`n|^Mr!?z+ZvV@L6ks z+zzO@+JRX9b!w)_?C-MKqk|pE14g+)UJx=wfBWSK7(mvxf-d5i%lkwTHX=gJKLa!Q zkgR=`u&a|iGqMNusQ20Q4=8~HU2i^s+P22&C$^ha|3@H&gK;Z4XfFMFo?k-#TtlW; zV+q!3#Qs0Po7T&xbe;zSs_bvV9XQB{)`NT+{s)SGs=M7Dp`)FhfyXxE#do1$0HzP1 z=IaJuTmkSul-`DmS7`pd)pw2T!_{`M-x0da@#gkx8ueUO&|-Tx1h6B3x$nx8;ymgoEak zR0sm$qAkn~=w*GSB(I_z2Z<4kZV!iClJCbmS)nb+kWx|eGdE!f1w@cTFmzkv?afbD z3If9*aOU5I#Qb*W0))rZj4tz3uLMtNt23X}_SD>r%m{UC4XfDPmKG_DI#9Az_db?Q z;?zk-x=W=x`0zQ2#QxepIJF41MOWaJBM-`!Az{%z>Z@aF^Vw5XC_38#Rc%Rsu>Asyli$ z3Rjc$m^HOvPP?rwa2&kZev1izNXpYO51~)vv{g`6E;DF^<_TEy+4|I9elzrb4G0yu ziINbTcJ5R=Ez1Q%>HIg1^^ zva(?tM(K4hsi9?%-^0hmY^fHaF3M%^2;Vcom>AXOM|mY>%tQ6N@is<(o^&YDGO{vL z{)egsowVG0)u>u`GzJ-H(zNVw@EWW&*dM54d!VQhhx{Nueq{ z*Z~hI)iqS4y{v*4j2P`!*Vn@&3D;X}N%-DczvX zbS1DuBNP9*@rVU6ydpM+bOnD2(TI4I&JgfThn-WN`2mferbclrT3Q|0#NsJey@(N3 zVg$@kCX+%mD0J$+AUkTgYMgB&PpFMCw9C=1zz3GTsH01wT9tf1p?Z`t$mh+1^f7;t zDfP2Rd2&V%vkN?5qj!Gt=$r9^N||x~&!k@J+dx6W#Qi=BWSEPK{ns#(Lt#pX#$rCv zBn&NJ8R&nbu}Fe@eKa%@O*X0ge0W%`KRxxqo1O!`CQo-~EVb=niDaT%N>AA2+4xdQ z`nO@*cP@(+M=?ESqk5-G&TDd0MxQJ|`o@6oer$h+kYih&(%+cdiSOVq9gR>KhdF-h z#^-JWY4Sewd?#5o+Af4Z3&qFBCoD{1yhv&%?koGfh9X@qx+bmmTjeU)P|nQGf(7oB z6C}Ejl`c37WyEr4z~xqiPvts1aSB0ZP` z1k8i&S%@#hogK8pn6^m9=Bfd-d#0*PmG=cEg@YLly2`P9;!R`#eW~&?kCw{YIA$Xa zSsWET^&2G+7Ellqy6)O%w_!?I*vEDgBPchPv+cocM@DM&?zZBJ3~1NqYMSBFF~pmp zCgz{5;Tfk}W+c#A5o^dBeJbFEWNJ|#ZRM8I7OXh|tK! z(vtri;f;qKc=EJPT?#z%1`gH?_fjNI*BED_wnI{pCZB5vOwx+RSR0UW8DwDjI5M0< zZk93u5xL?v24+QG)*GfAx3!m#9+9ke_Vfz7-$ZJN^sMA-w4i&`?{S1<*Nw(!|Eu*A z)(u7`88EeIk5iB!?K`LMeL;NNoCWd{vW=6?&p(KWjEU(-Kf)#MC%FGp&u>|7AE$2_ zy_K&!BtR^lGwd4W0!I&v8@rJ9)y1L;Lmd&IM+XBc{CB=V)d>xMcqHx`axHyS~~ZB(218Q&w3a&X@}bh^>H| z$|I8d4af2uNUW#>#TE=?mnK8Lod0oaW_-5BIH7nM8sPKGHkLug>+0-B{B;qVbk`S2 z^4mt%^h&W`&^GfJ?(aIh5HNZCh`l=KDNhF`e`+DpI~c@p2A$gIcm^}5DQ=PKuQclvwdMg#=aQ!tNEr3n!Pzty^_lB$i`HU!bpd8I_z60Iq6Ux z8?z~JeMBSnnW}gC`A|BB95b<>&v}i**ZtM2c;?;qwc#{(k*FqTkpkqYs!|c4eeC)l z7Ik2z9AX^v$fYau=P&H$5-O-McRZl#b_94O2?{xFXX}%w~lgfzT@!rs#sE-1Zq&<%V>Ym zBe;#E7XCMky}D?JX{h_Arc$T5vs&$%m&IeL!{?wM>j64RaG+V0+M62&C+VeOe!r@^ z`uZgXc^z=Wf$~Q<(0wu@(BY<=2UGMbG1A|Qi+*3yS&gmOD1G(juK@yY=3R~Y;Qkrr zUGTdarD&Yj!^NVPvQ*6!3EU^X0n#QA`sRw2zg|>as`iAu7HlnY6E{pHynP&bFW)jM6woG0cVM4iY8Y~#2OfJI+`|{Hh zh$rM!=MAXue^4M)cEtX^+{OV4SU`uS53Bv%-FF~#Tge;|3Od`S`2($WG- z@VGk2vKBQU)3vmWlhU;9z&v>}nnq+5pHz>ywUxxiPr%{!6U1V4Jkxx0(j&NNSh?}&RXz55Fyd3Gt`lQ9sYCG4EY9z!c$$$}Oba^wn#VWq4yz}rvvR}$c-Ut_sk6^S@E;QqH z%4&6&dICeWlt=25N9fYz6coKSbqXI8FfSjsu@|*_}t2BI^QiVu~Nw`v7?s ztGY)p=LJ};vqPWtv#WW#&tnJz2m_imnzREONh;(Al#*WOa&)ivVNz!kj3dk^Gd`V? z#CWZs)(9wOR+C!5xgI4<_|wVNqnJ19W2cDdN^Qe-OwP`{Z9NH z8X$IvIUo-SpT0mWWYUvR(Esll>v>$geXcUhwU_GAT!tR^Y;IxT+v78{ubvxEA3b^$ zB{}uK8oSbXD%ZDLGHj9gn3Z{!X(uAeoD7Ky4P;iklS)#e($1W@GG{vI9H%lwA(h!l zGF9lD%wvSMl=)px|6l(P=lst5zJ2iJ(YBwxpZmVAYpr#ywfss_Qqs`GK&*d>d<%xMZduNV}rc z5(5_@DM`t$mQ6*RA|e*4sbhD(Mj3nlVV>?N#ORU7Cv2%x`rhl6PYkp0t((kEf{O@& z^i_%PzjGmJ8Lr<|o&Ch^L+>ualZkHgZos@fYHvctgMtlswu@;fgZpz9CZ-TvEi`~m zg^_*pW$YP_;-Sb#fgmvR6@WoxOMIE^rXdg^|1jHc#ge&@TsQ_U3_~6{bBhR~LhRP{ z!uiXY*$-!-egah`r4sc!{v#NbF6SFxm#@7o)05u$`gB>TQ#g(kFx ze1R)~fKb+Hd;M0)P!k!652_Y%T-rB>6NfN7!K63n1x(WPoFkD{v5{|UvuiP*^eJj1u=N1I|M)~!3a$NDBf$_&K`sIM1_(g@Efcg)^gFiD zK>r{%vjW{L!pCN5G$AY~93jD5nHU%{d4T3HRF?pqVP)M;pyTqBA3rX2ycSs)$I$~J z>v@MkA_eIUV%OYfl_Xgg2wfjuGS56IC@m*1PePS&jBPGBP!#ekG!n z09IsbqF1&wH!nHx8k*9}pl)#k5PeP&uz+QnL$0|1Tf2Fc`wJv^9&M)Q5YPr}zl0d^LHCyKIs$cm zQ+C=hF1TpAtFY$tGY|kV3i2HD{hh9&FrrAiNF3?g@x;#5hl90S8I28XpCm;OQ~+BiRDmDc=b@*20mI99k2Af)&;cWT32ick{d3A zMZyl%hIit{!8v3o9O2`<=IVlC-e&@s`4Hd{26bKNYGyl5av?3pNh9mQh<=S{n?&EENTOQAL{g9%KAhl_x+3TA?HDB4pA%}n|28NM zXhnY++0GT&m`^dym77SEXWf376wJ!;(-(8VLO9ob$J?b^>c|-xkSUZVb~&PAaui=oiQj< zRh6`aM@zU)%!hg}^a^(~%t^{9-PMl_)W!pWz0RtPCP}7tmQo0iTC#Ove+D7BvJmI} z^BpHr?xs#Rege6$3jD|E{@eJ$W=deG1<%IefYG!IF&Z=-{$ja<1^ijphx!%1kv|2^ z_4&dU?c2s4YOd(4p}r1@n=!h`i3-h_(Vu$DGc|}DaCo+4%;A-18Y>PL%c9Nelk-MQ zxqd%TV8?0k;$W};^iFaQ(wnCjo~|hDkpLUG)V<5om}+6!rWR%4Gi!yW^CQp z$Xu+)odOfd>s^i>)Kxz6n22oLexj4~*HC5ZQ#{3{gqG#QWtzoYg5pyY+XQJ8Fs?kS zE=v+;U^fxV#bsxv`Pw($ZVm>AnT4|5cI8>57;9qz8;CC~Hp$#oNk)5wgjCUm#2q~$ zb?tJ#qUyE1_iSWxZi{$^_p8)c9JGFq=$m4hV-dx13USXp+u0-{B(l;Ib^=$D;CINF zn*d_AWA74?;!9$lDs?Ai8?!{p85lG$M@Fgd858LV_i?^^H*itzCN24FkXhJfK#PDIQp~L$PpVo#c+Q>MRs)(C2=F*=RG~$##b;G22ZF+4*&t@jZ zmnX?4{(w7!KyJ9`X~>xd@*<@LKDGbCB9h~$aI8|(UrkHkjIXGqxTF{B?O89|6yC0o z#H0=8XXEy)g5@!}v1?Oy-S-!G$jYXY-R zfh?W~4FDa+z{CV zV{0R?ExTAvjAk7m=a>}yAUuH#lx*``p{|%wNf10=1@FhIhznWMtE;OC+=BgWrxLbb4i32adW*f z1(VgtA3}se`eHA&T$jV?hS;AC*UtFvx2W$*;^ij=sWQvNlj}(PjS|bY_oa<#f7o-7 zxmm}?DE?`075Hku^-5Z>KPyZKcq}oZWqNIY+K2GwbGUm+19=jCEyNt52PBdW2zLT_c<%8;N+ zfpJ_i^TS92eDTHJbZV8H4&UmooE)LhA!x6EMJwF*F-PJXJGBk9G1itUqu6c@O`FBV zasD|UjM*;k`bVs|yC9?0OOy{DL7eOX9!mg5D0e>*6Ize~$+iC3XB~oX92*~lmoWz? z=SNks+kNOOwmwWbv+rf5wE}em=f&l3g>GdKmS^p+8GE1r9miG4g znH;U)A^48gV?tU-hz8_vTy!daQ=IXzQ?3~(w?U&AY6<)$ z2QsJK%~};rV$CT+!^3$LvRb%tiojT3-*XnQ||f?&Ld+7mxpRmU?P)OtDOs z`I{Flu0q>nl6;cl)y8C%>x8mi=NS3Qts^Ct4E|y6)M83?O}zI~G8Nv5+}E>is6|sA zst>PMB_#C^e5fa4PFyqfgz0GlbIpxi=>lw3btC8~%)$d{TO2oje^3swYji+*BFwP1 zOSzTg^T(ou{?7}cf`TEc8Bee_pGy)$M&(2TRiXN+flZ-o`j_DV+SO$zSnGI(jU1sd zt7RL;c;*qbo3#xMg9*zCQ>ZNKfA^wpN!#hqyRBBolit6$Mc)Q&W8Kf_|GNSGp;l#L zWaJVGulFtWKEcZMI@{hn)O}WRQ5z!!krVS%?@=PI+nEjkUYR4Mr>|y;oPC*| z|C7Zg0kvAH$(2Hnf;*Ac>`tfs^z@9Ty-k9HJ@=5bB%hLB>OPih64NKg7I*L!{n0(H z%``A9E7DJdeb39;?(_cK)IS##&?woRyPJIHs@&~;uGJ4$h2yhB*T1WM(w#J94prZ$ z9OQA>Iebdo0@^j7G0mO_{drL`X@nc=}YtCV= zpCv|^oy}Bgcl9Qv8n!%1*^TMj7#b2AjId z_>d-Jt0u1n5^iqGrP@DAo-wn)J+_%&2bI}T! zN4`X#Dv2uk%X3?zgnMz`#n{+QQE zkdsr0@U2U7sOF*cgzVU5sn92PP6LLttT`ZOd{+~^+X{A z#sBe7S5IkcU0QT$BGbB)H< zzgn@BKEDOZ&L8q(tbF8(e*$HTMD?{{PCmX60U}Syqqj1`Y;kcBsgkYW0w03m`r><| zwY3$kL+nnPKmSpY?}Z_Ld`Hp0ONcNo`=8m`|3x+M|L>vsp}+8*>{7)+2^&jAaG-!u zQ2E+-|5+9AzPtM(lzS-ekCal9?%a8Yjv;VQ3cTFm=!a5uc_igDCces?UeMvT;YW1n zO#{@oxPK`rsb6w)OF{R+4i`WN8t-SBa#0&eq%L$HsQAZwKrI6sHF>El_Zg18U`NY5 z+Xm35NVdLv_Z1(N)m`03z9i)B<$;$ZC=r8smJ~N(uNfd3F0L|G>4aM}$m96bLuxyy z7Rt*jE8_+CO^Ch9Gxi;(J={^ufo13b$E<8$l!JE#VR*XAaeX_xE{MO-BHBYoL_Pyu z9yCtF@6lVW6)0vU;>*QcpC97Ap_9p_aL_%YVz4uoQN6saPaQKua+JsQ4G20z7mTV=6Qo%>ovcN2OV;&l& zNKP3ky^>wJ7k6*D_=>H>K)#Xrs745O@o3CGfwO}qO+bTUhw1<$6YjXdtf)=3u{>)}`wre=Se?RaWVlW;hj;2jsEgSY*UeIq9?IDQan-A54ZbWAp zdcH*?4lKSCPkT!Qi}PJisl0q}?%_XBw{qpyduZqOAWi<(_pmV{g$Yg4J5hnkS>LEK3|lSPJ<>r|b@+ zQ_p=ZQ5o&5%6j*`?9V66s}VU$B$<_-cr&r`wRgJnUi?)2(dGx{Norl2gUz0ei5v@U zZ`)lPyS0NS)+U}qX`|oD;`yFTzrK@@9BQ5ie1k!ag^LZ1JQQ?1mb2j6n41h-~58uroD(xA=8B`ap*!of%i-1c4j4 zvx!YYoak-L#KgpyEKe7m7notO{GqKa)-24}RPYOXQ)XVH>Qjx~`*mVI0KABm4H{zb z%F7phKWXsw;w%I&{S761MC!!ogA2I1&N*RIBwMn$rr!jgpFFO!*%pNEF?o8M?|n}+)Eu9 z_rurSxbCK%Ko{JYor%Ouc&@#_Z!MG<@+Se#?_N$<0We>T_PKml+|$ZRtY*_#6%;&Y zBP^WnQcFDBKmb0lo+1jDNg)kN2J3SkIVT)x>Su>c%SR?AEUN)&;F|lOZ7`ArT*6y3uDn~P}Uw(>~PkXQeAOa!*hvt#G zo^=mdr=kxc>0DlTpHs~BAE;#(ca7BC#=y>`Cfs?5^&?cw&)E{J5KK?6g0S22=joB6 z_e537xrKyYvat@}dnoiaW7}bBI>h1SF0o$SJHfb_qbDaB?$v5GE%SVI#L-6YP)Qea zbf^el90<A1_p_-hV&_H)wz{af2ng4I8Xg%~=Maoub?7|83@Y>4mY@^=BIF&on@m zlX$*>`r4-Q_-U5F+p^I`mcgK zJV1%JXmGJUt@+@2v%cc|$NKJz*W_B#cM82qOWWjYy7TJOde=Nwq^jP;`V-gj{yyk2m#06p}s7!%kpg@)_DC z*WWoD&DbyQ6s3Pkz}_ME-O#3$U$!j^U2*Hw(a~w0#iB+993Lt6IFe`g+uzBL7}l+0 zkl16m+cw|2hJPM43o}=y9wy##w2U)^6mJk%!Z%TP%=lNo*#Xv1&{N66H(j_3PiT2$ z=MzfTU2bcw#NG|9(gwy(krX(1Ym=emo~`any33;uyt17^;XBlSZ08uyDwW6CvWqL7 zN=<6Wud1qomL5>ea|u<+0@5S>f-Cb%!X;W@N#`FRV;O?{Hbu7DiG6Pf+d?NKJ>0*w zi7vb*v&U|mviHG~vp6{Omob6MF@{Z3SkwC|?~DxHZ_v8vhedD;#VulnEAwD&Z-Kpf zAl0y4^l3g4?kt-QE%X0%q{NeRwnI_giU~Te!GQr!futHWuNYAj7Lh)PO^#0Fd<4`V zV`Q~#%%#*XOOqZmm?g>B)+191&wS@3yKP4YJ8O5vkoRuUlJhl<`6jL(Ej74DeyP}&QO*A%Z`!-VyeZykrRc#l4))1eQtyrL$>~LG{iGGl;Hz)_gbMS7}H;npsUCaN5&0-77)9IVSGNDfVv zSj;(z&5VZ`hsPj~tnv*AHQi*bNzql=OOYZP&G(XY(Ar9su_`pV9#;3MpwL&`Ui8$ols|*W*DW5Dk!j+R9=|gT+N3RijNJ; zvf|=&kLqT+M(GxT(y61ldudwoIQw$(vo^# zjdq4CDW$nPHx1oZ@px#G{9Cz#b(*)UT20h1OaD^t@cqNP@l7Bm56};Sr~%u*syu#Z znd~=;T_+bD6M{)xzq+jzid;voc{3!0D{+n%M2Y`{!unsht$%3+{w_KG*B;>qo%WCB z;m3s^6FxuQ%a2$7@959pFZ@tB|Km^o>xCbj-{1e_|H=9NcrQO*`M)37-!J^*z5Mu- o|9au?@8!o`{`e|F9c$K@5`E9oIo5FNt;4@PMwW*6^__zL2Y;-%o&W#< literal 0 HcmV?d00001 diff --git a/docs/implementation.md b/docs/implementation.md index 937878d26..0e1508380 100644 --- a/docs/implementation.md +++ b/docs/implementation.md @@ -22,24 +22,26 @@ Hoodie upsert/insert is merely a Spark DAG, that can be broken into two big piec ## Index Hoodie currently provides two choices for indexes : `BloomIndex` and `HBaseIndex` to map a record key into the file id to which it belongs to. This enables -us to speed up upserts significantly, without scanning over every record in the dataset. +us to speed up upserts significantly, without scanning over every record in the dataset. Hoodie Indices can be classified based on +their ability to lookup records across partition. A `global` index does not need partition information for finding the file-id for a record key +but a `non-global` does. -#### HBase Index +#### HBase Index (global) Here, we just use HBase in a straightforward way to store the mapping above. The challenge with using HBase (or any external key-value store for that matter) is performing rollback of a commit and handling partial index updates. Since the HBase table is indexed by record key and not commit Time, we would have to scan all the entries which will be prohibitively expensive. Insteead, we store the commit time with the value and discard its value if it does not belong to a valid commit. -#### Bloom Index +#### Bloom Index (non-global) This index is built by adding bloom filters with a very high false positive tolerance (e.g: 1/10^9), to the parquet file footers. The advantage of this index over HBase is the obvious removal of a big external dependency, and also nicer handling of rollbacks & partial updates since the index is part of the data file itself. -At runtime, checking the Bloom Index for a given set of record keys effectively ammonts to checking all the bloom filters within a given +At runtime, checking the Bloom Index for a given set of record keys effectively amounts to checking all the bloom filters within a given partition, against the incoming records, using a Spark join. Much of the engineering effort towards the Bloom index has gone into scaling this join -by caching the incoming RDD[HoodieRecord] to be able and dynamically tuning join parallelism, to avoid hitting Spark limitations like 2GB maximum +by caching the incoming RDD[HoodieRecord] and dynamically tuning join parallelism, to avoid hitting Spark limitations like 2GB maximum for partition size. As a result, Bloom Index implementation has been able to handle single upserts upto 5TB, in a reliable manner. @@ -61,9 +63,175 @@ records such that In this storage, index updation is a no-op, since the bloom filters are already written as a part of committing data. +In the case of Copy-On-Write, a single parquet file constitutes one `file slice` which contains one complete version of +the file + +{% include image.html file="hoodie_log_format_v2.png" alt="hoodie_log_format_v2.png" max-width="1000" %} + #### Merge On Read -Work in Progress .. .. .. .. .. +In the Merge-On-Read storage model, there are 2 logical components - one for ingesting data (both inserts/updates) into the dataset + and another for creating compacted views. The former is hereby referred to as `Writer` while the later + is referred as `Compactor`. + +##### Merge On Read Writer + + At a high level, Merge-On-Read Writer goes through same stages as Copy-On-Write writer in ingesting data. + The key difference here is that updates are appended to latest log (delta) file belonging to the latest file slice + without merging. For inserts, Hudi supports 2 modes: + + 1. Inserts to Log Files - This is done for datasets that have an indexable log files (for eg global index) + 2. Inserts to parquet files - This is done for datasets that do not have indexable log files, for eg bloom index + embedded in parquer files. Hudi treats writing new records in the same way as inserting to Copy-On-Write files. + +As in the case of Copy-On-Write, the input tagged records are partitioned such that all upserts destined to +a `file id` are grouped together. This upsert-batch is written as one or more log-blocks written to log-files. +Hudi allows clients to control log file sizes (See [Storage Configs](../configurations)) + +The WriteClient API is same for both Copy-On-Write and Merge-On-Read writers. + +With Merge-On-Read, several rounds of data-writes would have resulted in accumulation of one or more log-files. +All these log-files along with base-parquet (if exists) constitute a `file slice` which represents one complete version +of the file. + +#### Compactor + +Realtime Readers will perform in-situ merge of these delta log-files to provide the most recent (committed) view of +the dataset. To keep the query-performance in check and eventually achieve read-optimized performance, Hudi supports +compacting these log-files asynchronously to create read-optimized views. + +Asynchronous Compaction involves 2 steps: + + * `Compaction Schedule` : Hudi Write Client exposes API to create Compaction plans which contains the list of `file slice` + to be compacted atomically in a single compaction commit. Hudi allows pluggable strategies for choosing + file slices for each compaction runs. This step is typically done inline by Writer process as Hudi expects + only one schedule is being generated at a time which allows Hudi to enforce the constraint that pending compaction + plans do not step on each other file-slices. This constraint allows for multiple concurrent `Compactors` to run at + the same time. Some of the common strategies used for choosing `file slice` for compaction are: + * BoundedIO - Limit the number of file slices chosen for a compaction plan by expected total IO (read + write) + needed to complete compaction run + * Log File Size - Prefer file-slices with larger amounts of delta log data to be merged + * Day Based - Prefer file slice belonging to latest day partitions + ``` + API for scheduling compaction + /** + * Schedules a new compaction instant + * @param extraMetadata + * @return Compaction Instant timestamp if a new compaction plan is scheduled + */ + Optional scheduleCompaction(Optional> extraMetadata) throws IOException; + ``` + * `Compactor` : Hudi provides a separate API in Write Client to execute a compaction plan. The compaction + plan (just like a commit) is identified by a timestamp. Most of the design and implementation complexities for Async + Compaction is for guaranteeing snapshot isolation to readers and writer when + multiple concurrent compactors are running. Typical compactor deployment involves launching a separate + spark application which executes pending compactions when they become available. The core logic of compacting + file slices in the Compactor is very similar to that of merging updates in a Copy-On-Write table. The only + difference being in the case of compaction, there is an additional step of merging the records in delta log-files. + + Here are the main API to lookup and execute a compaction plan. + ``` + Main API in HoodieWriteClient for running Compaction: + /** + * Performs Compaction corresponding to instant-time + * @param compactionInstantTime Compaction Instant Time + * @return + * @throws IOException + */ + public JavaRDD compact(String compactionInstantTime) throws IOException; + + To lookup all pending compactions, use the API defined in HoodieReadClient + + /** + * Return all pending compactions with instant time for clients to decide what to compact next. + * @return + */ + public List> getPendingCompactions(); + ``` + +Refer to __hoodie-client/src/test/java/HoodieClientExample.java__ class for an example of how compaction +is scheduled and executed. + +##### Deployment Models + +These are typical Hoodie Writer and Compaction deployment models + + * `Inline Compaction` : At each round, a single spark application ingests new batch to dataset. It then optionally decides to schedule + a compaction run and executes it in sequence. + * `Single Dedicated Async Compactor` : The Spark application which brings in new changes to dataset (writer) periodically + schedules compaction. The Writer application does not run compaction inline. A separate spark applications periodically + probes for pending compaction and executes the compaction. + * ` Multi Async Compactors` : This mode is similar to `Single Dedicated Async Compactor` mode. The main difference being + now there can be more than one spark application picking different compactions and executing them in parallel. + In order to ensure compactors do not step on each other, they use coordination service like zookeeper to pickup unique + pending compaction instants and run them. + +The Compaction process requires one executor per file-slice in the compaction plan. So, the best resource allocation +strategy (both in terms of speed and resource usage) for clusters supporting dynamic allocation is to lookup the compaction +plan to be run to figure out the number of file slices being compacted and choose that many number of executors. + +## Async Compaction Design Deep-Dive (Optional) + +For the purpose of this section, it is important to distinguish between 2 types of commits as pertaining to the file-group: + +A commit which generates a merged and read-optimized file-slice is called `snapshot commit` (SC) with respect to that file-group. +A commit which merely appended the new/updated records assigned to the file-group into a new log block is called `delta commit` (DC) +with respect to that file-group. + +### Algorithm + +The algorithm is described with an illustration. Let us assume a scenario where there are commits SC1, DC2, DC3 that have +already completed on a data-set. Commit DC4 is currently ongoing with the writer (ingestion) process using it to upsert data. +Let us also imagine there are a set of file-groups (FG1 … FGn) in the data-set whose latest version (`File-Slice`) +contains the base file created by commit SC1 (snapshot-commit in columnar format) and a log file containing row-based +log blocks of 2 delta-commits (DC2 and DC3). + +{% include image.html file="async_compac_1.png" alt="async_compac_1.png" max-width="1000" %} + + * Writer (Ingestion) that is going to commit "DC4" starts. The record updates in this batch are grouped by file-groups + and appended in row formats to the corresponding log file as delta commit. Let us imagine a subset of file-groups has + this new log block (delta commit) DC4 added. + * Before the writer job completes, it runs the compaction strategy to decide which file-group to compact by compactor + and creates a new compaction-request commit SC5. This commit file is marked as “requested” with metadata denoting + which fileIds to compact (based on selection policy). Writer completes without running compaction (will be run async). + + {% include image.html file="async_compac_2.png" alt="async_compac_2.png" max-width="1000" %} + + * Writer job runs again ingesting next batch. It starts with commit DC6. It reads the earliest inflight compaction + request marker commit in timeline order and collects the (fileId, Compaction Commit Id “CcId” ) pairs from meta-data. + Ingestion DC6 ensures a new file-slice with base-commit “CcId” gets allocated for the file-group. + The Writer will simply append records in row-format to the first log-file (as delta-commit) assuming the + base-file (“Phantom-Base-File”) will be created eventually by the compactor. + + {% include image.html file="async_compac_3.png" alt="async_compac_3.png" max-width="1000" %} + + * Compactor runs at some time and commits at “Tc” (concurrently or before/after Ingestion DC6). It reads the commit-timeline + and finds the first unprocessed compaction request marker commit. Compactor reads the commit’s metadata finding the + file-slices to be compacted. It compacts the file-slice and creates the missing base-file (“Phantom-Base-File”) + with “CCId” as the commit-timestamp. Compactor then marks the compaction commit timestamp as completed. + It is important to realize that at data-set level, there could be different file-groups requesting compaction at + different commit timestamps. + + {% include image.html file="async_compac_4.png" alt="async_compac_4.png" max-width="1000" %} + + * Near Real-time reader interested in getting the latest snapshot will have 2 cases. Let us assume that the + incremental ingestion (writer at DC6) happened before the compaction (some time “Tc”’). + The below description is with regards to compaction from file-group perspective. + * `Reader querying at time between ingestion completion time for DC6 and compaction finish “Tc”`: + Hoodie’s implementation will be changed to become aware of file-groups currently waiting for compaction and + merge log-files corresponding to DC2-DC6 with the base-file corresponding to SC1. In essence, Hoodie will create + a pseudo file-slice by combining the 2 file-slices starting at base-commits SC1 and SC5 to one. + For file-groups not waiting for compaction, the reader behavior is essentially the same - read latest file-slice + and merge on the fly. + * `Reader querying at time after compaction finished (> “Tc”)` : In this case, reader will not find any pending + compactions in the timeline and will simply have the current behavior of reading the latest file-slice and + merging on-the-fly. + + * Read-Optimized View readers will query against the latest columnar base-file for each file-groups. + +The above algorithm explains Async compaction w.r.t a single compaction run on a single file-group. It is important +to note that multiple compaction plans can be run concurrently as they are essentially operating on different +file-groups. ## Performance