From 520c46da42d4bb88b69b56b391b7ee69baae145c Mon Sep 17 00:00:00 2001 From: Rajesh Mahindra Date: Fri, 19 Nov 2021 13:30:19 -0800 Subject: [PATCH 1/6] Add RFC entry for deltastreamer source for debezium --- rfc/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rfc/README.md b/rfc/README.md index 6fe6827dfd91f..28340d736fc4d 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -63,4 +63,4 @@ The list of all RFCs can be found here. | 37 | [Hudi metadata based bloom index] | `UNDER REVIEW` | | 38 | [Spark Datasource V2 Integration] | `UNDER REVIEW` | | 39 | [Incremental source for Debezium] | `UNDER REVIEW` | -| 40 | [Hudi Connector for Trino] | `UNDER REVIEW` | +| 40 | [Hudi Connector for Trino] | `UNDER REVIEW` | \ No newline at end of file From 8a69601744c15269e07fefa2b44f7e266dcfc70e Mon Sep 17 00:00:00 2001 From: Rajesh Mahindra Date: Fri, 19 Nov 2021 13:54:28 -0800 Subject: [PATCH 2/6] Add RFC for debezium source --- rfc/README.md | 2 +- rfc/rfc-39/arch.png | Bin 0 -> 321790 bytes rfc/rfc-39/rfc-39.md | 108 +++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 109 insertions(+), 1 deletion(-) create mode 100644 rfc/rfc-39/arch.png create mode 100644 rfc/rfc-39/rfc-39.md diff --git a/rfc/README.md b/rfc/README.md index 28340d736fc4d..6fe6827dfd91f 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -63,4 +63,4 @@ The list of all RFCs can be found here. | 37 | [Hudi metadata based bloom index] | `UNDER REVIEW` | | 38 | [Spark Datasource V2 Integration] | `UNDER REVIEW` | | 39 | [Incremental source for Debezium] | `UNDER REVIEW` | -| 40 | [Hudi Connector for Trino] | `UNDER REVIEW` | \ No newline at end of file +| 40 | [Hudi Connector for Trino] | `UNDER REVIEW` | diff --git a/rfc/rfc-39/arch.png b/rfc/rfc-39/arch.png new file mode 100644 index 0000000000000000000000000000000000000000..8864f439daf9e7ee2e3c21a9d3be41677d3cef92 GIT binary patch literal 321790 zcmeFa30Tf++di6QEtWBvlL{e3gGP<4N}7ZuG%6%TrBa&L%Az?k6jC&&l!WGiCW+EK z4@w$TkEeNl_IW>L;m!N)@A&qy-~Zmn@jdHU8h+34yYKrN&+ELdTo3G*TQq;odwY@d>%mV7>_X1(pqF=tqKD@R3 zZ2bXN4mD|>S>wgI?iyW(Yqy5lYV5Q=7sk5%r0DLH!u>mM#5?-kIpx>qZ+_bE!=1T0 zu^%`0iWlu_GY{w!zfoH>bjdM8b^hOftjAq`eK-w0i!@yI^(XQ*L?!e4AD3V&zCC_j z`0aZJrysBVk@?*?Mt zm%_qAKR*4v0zyK$PfryDy?7ziU4m!5PTczU9}llxWcvDwn+cYmE2=Ua9gFSn37vD( zFev)oWcsPo%n^gSNenbT0yH`t_QR0m@khGo0^)Mq3=8fhTT_`w=SRm z_cQmePkWH6pXZh1#VV#po1Dmc!K0Hdem=QlGyOCRUB2C6?pm>n+eJl1Hv|^RojZ4q zE*Cy{6Op+;3KgH=x2?uyr~Y|A$lSO4BNzX9dvZ5Rez>;#$G81aRxkha!rh%#U5nt3iM(k4WNxj5%8=5(#M&uVRTo^L*bfhLaR1|xp z9LBBJO4^BZ@5Fh3r$P}sl z`uE3!{GS{$d7mXr<=-Ns-L0deqk2O<^=2PmU-y%+y{KtN&Ea#L9FKVPNFhZ4Pv5ye zle~&N)^Q9{8rd}x&a=izV9)BU8d+Cqn(nFDL{}KqQtqjzVcETN=FD-svS7moQ86)S z1?$wQ=YEsmlbIi~t}6C4!WKa6LT4U0w)B*Db(*c!40XhZd=vNGD`9kQOMI#^idCMe z@1F|qYh0iO$-2``(hQ-&8r{mH)K}H}7PK~3XqYIc$l%R)o>zEyU`vP=gk|b+tK;)$ z_xcp8yz3ltdV^inX4ITjdE+B}!H*too3e8vKbY)w{|7Akzc~g8^Q>fMkte#ouJ0r9uHa3=rPjd+qHG7##$kg?30(xy$25o|(*hA^+^1zY*@;}T29hurb*v^^vQ!RzP`31m6=C;AxZPCA0LMqt+%y+ejxvP z${t4(&TDi&pBspgMbGvMkO5IznznD$j zZ|1SB-E|?Pi!WoM+oW`+QND8J%F{ee^LK*N&h&fJkS8@s^%*ly(!{a-L{w^?8OEI{ z4$7XO^{dcsPDK!P{}OU-GjD(Y&>F!>P<%XGl^>I*r)P0l*>bwQ3dNaUGyAr~%mT3C zbSH)jnY&~%r?-{<#pOSeM{aoZo%MXPXyHN`4Gj&1GIG+=(q)yEd367TaMj5?^Iul_ zX6&7b!!oJimyFHB)RwFlZ5NUfhZ<~yNf1W&SWmuN!0t=L2R$~iH)-40RYrF1Bu4}?#g&!5GmrM!^XIoDJ~d8uH~z$Hk`p)d#pY%1`p$REj!EfBewsJ# zOY7D7$%_z_-LA{PvHA&}J3hnaGapb>TM?xm%bm;v$KyB=OB4LLJ3br+8+DP=!0I@h z#oqjC0gs2b-A9gp=g`Oj|7XtkyP(5BS9cvnB zY)nkPIu4iW9LBn@7qtExvvmpn*mGiRXjOfJ%|PeTXJ-!nOq2Hqh~HVw?cy?A={QmO zlc0K4#;`0ss5XP%%+$FD!NI{@eDKLX(blf&_yV@R>)v)7XNF}*pXlz%pCGZ~NMBp{ zOfR3L$qfVEJZTi0CiLjhqiMPLSIc~fQzgKIeWG114ib5yR%$4zh;A-SbHCfxPgQaH z#igaoEG;dmlS3IgPjI$QX6tpy8Gn8JXH56^@l|H~T2YHF$tfu?6tUfF_+{!?Ok1KUrbzKIZqF+52~r%n$D2^RqIne$D|O2mmS(c5{>kc zPgAFw^?mTsYI*VEW)g;xn7#e;XT_QRQ~OfuEofz@dWOQxi<5&18PIbc#}UPuD)}W} z#o=ieAMFbx?b=KWVF&%=ria$Qt&q@~Z#L34e1P_{iNdhr_>R@V7pH3l^L0DYiN=KkTlM zemOXpnkmmi2P?Ztd_{k3F@c?pLuSlWYXDF6x#F?=zsZbY9)dpn<>Jzk=be`^`8gwO zW)Sv|;Wq32m_JjIn_7K!xNYC6IF6S7YJJHse=`0g1%n4ggf^Tf6)^L(-`*EX z_&CXOK5>1R^1a(LuOC)Z%c)Esd=8W#*To)Me|v|KkmHJcUdGZue1N*@;Ch=+x%k?_bu-w*Z!Lc^R->}_V#9o##XOhy)R`s zURb#P`&{Z2vYw#Fk9WK{UoZOX91sI)5pBGP&!i#sSpkgwx|V+x?}pKV&%yEW@$KnI zL-LTPlyCzpaawrq?!~WRDZ@=G93&kNQB{$cwg&!AUoY>TGv2%^V@nbZrr^Ia-8>oN z7iOyPwW78HHgm*+KZ(}WwJ)8TGTroSUD>%K1V~O5QaJIk zu6XHM5{2oEmiRUYvrn9qo##@9ejjmz-P2w=|#p z_{|%TpMK2Aez^8uBK$u9gSr;~jjZ4r6VNDX8pWOm0rS;!86U2&G{2}l?@l*TM9qIl zqaxmZdUsFmGOV(;^@qR7{>DaO^Gmw+f27R zg@#P_`eyt@j|0a)a^lpxpI80_1!JH5g0wtiytjZJK>lkowU8Xm_T$gCn9;fu8Uy}~ z27e!W!@X->tUeWIT)i`j)MLK)+4#}P?>kY)&$~x|IH(^F@kcE5e_VKU|8nG)!4>3IX|7K%n(pW4Z)6yr40mb2ZCh&VCkY!&bNw zV@-io{R#d-aiop;jERYfwIFc$Nmy0nIN1ls97`L>_(`}t1Zjgc&YUax6W>bGs()*$ zky8<@wt2F(z&G!iVd=~_cY!OC<@RNXv&VLr(dC4V2(fVJfi25RxPQU-Yr8bbw7~qzLdt85!)z@OYcll$#+H(HRI( z>tme^N$9qFk4Bn}C6WK19XPC)(!y0TGU5Yesy{h!9tWzs&zLvD`m2jse-dl~5`M$h zhggWVe`X{$gjyM6SRkZkGy|!z>VFeyY0p4+ zdF(qBsm0z70^PmXlh@xcE+-EHhy}}e=Z>@mvcnG;`og@`{~)h&iP%p^ zhjqsXhKBNrgXQFmV{}@!bGWm!)9qBjZO^I(Kmw>lD5Rv8ZP!eE z(mq(J<0wJL^cWbd8B2<;=C^c&2YS7`C(Kw2ubzA~PA|u=fs!W4Em)IbBHw1zR(PLs z=fE7Ll^Z0m@?|F{Mh8}~r&v>)U0f2I-M9*^_uehG>xtw}ZVv(w$7es&Ybf`)U+wn<(#j! zjbY8HS^k!jQt-x3^PU1p<72;9)YfuaTy~@hUH9B)whhn#|X{@6OO-t~RH|pVKkk;quz-DTM(+XGV?Nvx5K2Gy4DI5NnYDYWn^SXd9g`|_EY%pXP$j(8*8sqa#VegRMFl7#w;8uX∓0x}003WpS|PR9g7zJm-ArOf0Wyd+pOxikcVlH}M%1`nOXk z-Y^1tWwO)l$C|M%?l|x4%8OQ3a^~iJet*mnv+7YYenXBD8`R>PF@B0{AaY@Mwgi*L z^m#ft55mJavE97JwYi=ws)kz@#jH7~uD(i8P_VeXd<7GeoHv^%@sp?S4Uyw;d0qD4 z*HqutH{pWT8Ew1d*G~|rsjAX^(VnGN`#MWXF%W z-i|-fm~PMULJJXzq3>_k=F_)0-k29hc2G-8qQ8;rNVLlet|KxqGF>&#jUw*YC%?S* zGDa&UDA~qt(m)I7f`EX43{bf`o6nD3XJ4DUc&*#I1Ht@zPxs>_C-#(w!Yxeu9si(@0p}CGf}jKvDjr#ZDjF*KnTQ& zi;MYUqGbE~`(<#N78VxJX>a_V=X9a}uJe`Mw#7{kj&900Xkuc5=Pdl&4=Tra|9QE} zoY=;fpSNwe67%QFS?SbK5B?;%!oT-KzHBP}bM{=68uMgVX?afRk z`zUfVSBm=hk6N7^?eE~jXDGe>)Y#a_ZUO^$EJgdz_`EY5a6TLPA(4sLsFr1c!!pqG zN9}yT^3A?9y*zW~%<7?{*r+J?u&}VI6f260z3otsCov25Zks*t-o2YR+8NdIC@RKH ztS6Df?0{a@l|?Jw`7}TT872A;q)v{0c8%Kiv2SGf)v9#f&RJfh2((OU`~#zVMLo)& z1-Eywl(!K%C#ps0$OpFE% z?TIYktx%qB){@KA79SPGg{=5UZ&gf83=@|>L49EYn@2y#dILUSWN~30XB9GO(0rDT z_)Piefe5@Hz^*s{X{D{Is%qWE+H=8^EJAEU`${M_k4j^2s5u3eTmtQsY@t$QLq0RG|uGMHxVtmfBX4 zb#isO5B2wF&t?=itr#6)%Ae@ogDqsONHIQpR$*-^bJz3Rowq7*!xBSv%iuyMB1S#B zySu4x`HOCLXT5m#?56*N4y6o}xXM7Aw;bN$JN2H%991rn#$;7s)G;vLbUsvtXS;Ujg#QziQ2oq5j#z*dP`Zy8&LGBL^J(j z3St-bp+-kq#OQ>Cgap-@eSDoZyj^zOa|KG^g_iBkLuZq0gA)=G+D8K0ua4W=+Uk!D zcDpVs5)>BBLlsl2mrW^IT^p58AAK`@R>~G(Hj&$759+M8sxRZ6oCef#1O)`LaX?Fl zXx=oc)aI^ula%fkLA7virII^t9D)WJ*Cz{7$DyQqnHC zs={`$kkm5`kkgUq{W>rf8-YHEYNeEdc@eUwHWcmZkch~m~e+U2?0oc zs>uTJIN5j=$L})+^q?0t>jlUabW0w^#CXkRw|jC@Dg#IM(H8hG=k4vs4w=RF{yCS; zPryETa&q#a_W2}+-O~=~d@42fY1_xLA?@d9)vAZQtgUif;H`(>sHb13C{l?t-4jU} z$ehi1xIYaZHwcS{f~gkU;>7ZrbKLc@)=jaYE>>fmP&KiZh9RtkJPWaeP#3?LsOS}k z8>&(`U}1uTW>XsK>sh0YJa$vpv40*^#GdMitz4EDG3RsjlM~X>IvEacE9G!H(b{Rr zL7%B4(Q-Klbj!dZrhr*IAv7ij(MV#8msqYqSVe8ESdZY)&=8}i^d`@_iyP*K=I;G7 zUxm0s-K8lo9KoTXq3!7x7#|0^8U$PlYahH2pOflaUS597?ukuhjFxP|(BR-=2|uYd zaA7RwrP#By=G+n7`x~Jsn1*Fo6Ib=`l3Lg-0byZvfh~O19-nNz)Igi)9~{gT&KP<3 zH1Eap=ZpTnPPRp;iHX`nv4LSTiyFf5Mz)))R2COAy?gg=Q<34Cq1Pvc?v5oqeysc& zx7f6n|6I+p;kq_Sb&Ez1X8z{ES`(EN87KMJ)s17l*Uh z@mfDit9v>3wW36mcm+-?!FyN5+z7ePUp{0>GnS|a!D*8db=zFAOKsDQCRC|7V@_(n z!HSv@EVB)KMr^o-m6dgcy)V4pqTl8Sov%A8G`@t%Y^dJK*Y1IR^iWYlnr#aTLTgI1 zdNp1cM_Ma|SDE=DvWSo}7XVHet7dT?=h{pJ>izL;1lK2#RL;=In_FBh?}QtbQjB#y zrozl5y47xMGOlQe#I2ca_Reu%6>cxO`8g7XGw089Cz~J@lqDgDL|jsJS(!k;V-6AX z7H=uZM{5|_CB-|h4B+mUW^KpWxV~$$laxB_829JhmwQ}Y{q}77+yGbac5jf0PoT8u z%$mNI=kwo0#irjCSzQY2kA06rW{`cy0g;XVySsv}6vwckeUZ0xpMD}f%#3#X-k!+E zkI!jlZN!O$n1BRBDsWsYcN{0s0r^5F$3V)bsyyqKAwMB}gdNE|h8`cOdF(27PhR+F zw+g3IJr8H}5ijGzI41#SRWm=iOo{BEyd6b#EPKQ*y`2v`9|!L;Piii{qmK6#soRYM zXLKJ#*7o_T10{s5n=Dp+XM0D-)?G)i20!Z^vZ*PDsAo9Wk7cn>9(cOSF_|;aZYNy} zyJ*fYOd10cDWSgF+b((DOmQUVk}9#Otuuguam~0C zX>>|l;!Ec2O|w}B1_rWvsIXOuV?8NF3g!>>OkV{$ZTQGwo~c6i9_x$(m1>qNTjVl|U?Rzx4;Z??y( zR&uGwcyxBYndF?*ZHY}!mt?bPf0)=FxlC_;GZ-ayuY@Kn8c7xUwgS=Xo*XR`osO7>*X zCasZJDF|h$EJ|3Ij-)O?-guHR=C7TIN%$WPJg?N|O16iZn+=q!Gfvp%xUDFztz|)M z@!o|}3|j?PS^WMzud%I)ib~GV>$`$RdaZA6ar9h=Mv|1KtgK8%CL74Q9xy4U>#5rc zKF4M_>}?D2qb9P=&8Y1dGJ!cu9jq~O^&BPfGWAOqqxYB<36mvA#T%AR6x_p|-Kv43 zhjaL=VUaeor9f&=Nc0FARb+!wSI)%P+*6+-Ov3T9sI`w@rk}!6w?nkD;Q_-QHW=`E zOAH<@?%uIuhjhzAvd`?a;XGOOo9B?AAP-pCy_Iv|uvJgpuCC#il2quaoPXep7cU@X z%$=bD9Ye|&&N?Qir#nQb*p1Ggk2Sk8I!X1vm(I$auWdh2KKw~IV^@&W1O)I&dsPiG zl>`0uBzGhrL-O7uF`CSvrfeBKFeJ?=A@GRe*lAJ&^&{zO%UC1Lq1npV$cW`dEJB?0 z(el`ptWj&8!Tbd2%R{ND_bGPa?MBCsAHR}zy1V9i97?56E9G{DdwU|T^Aoc=FSDFd zxhmyKEV6dDBFdRXlWb6Ivx$MqJOXaD7YA?3`-?+`2@O3ZleG)ik#@KhvLTP*@qCnvk}+}huuaPu5c;`` zQBwJ_9=G(XOI39lK4mv2$I2 z&4qgGx6q?oA|oRWk0$CKK*ov~{&cq08DnD);0(6eRC&coAz@+fL^I|9it56$(Enpt}vZFd$zf@ zaq?oZ`3PZ^wRK$t5-X_DHIjW->i{{b2!w(geSL4e}IvcEqZ5O zB%$7)mNA^evzVoJwg1~lzc|Um4T#|xC5Ls9@nvT*N@-qLc;bMfqN4b4le4tGntaqH zIho>PgK#X`{i8<4#%r1Ok%Q^3O<1MHyutAKmqWcHcH*P$!Pr;PevQD%YYtG;;>O4_ zzeejbhK8=sjVkvdlZ1d7oY}s`P|YJ`ol2zIZ@JBIaBJgsbeO2y+fO_J6?Wtwx!N6M zC&`{rcR|ta6MtZ&^_B4pBo~2VtS&c#>K^D7)W08kdc)1TizdtN*u&y2H5fgbbm3HS zHzJa_lleYO%AFz1rapEPpLJCHKb(S{ku^Bc+0*0YJeP6ty%Up2^R|z&>7c5#tdRzK zWORF4K}Tn&g8#@#STF-|F51cW0L|dZ8=u(nKuEzH$O(#nI70gQpFu z)G#n;W3D`Ae4q(l&>OLO(0Xs=T11BNgHMKPWVNY}!x&s$A|lqk0;L&!_bLATTu+`o zd$zHG3$dFeQlM26%A`VV{%I|dlFaI|#>POA;9-C}>K3*Lx$7KznZdieYW?;gCy4|VS&zP@IS1JGBxO@XVV14I;VT$)b zU%SHoObSc~74K;&tG>~3oSJvl^8^$;cUAhAznNmC%$jA2=jHjt+ycr0=!3odxy9&N z7AA_<`mVYpezQNqOVzp9{10sF2-x!CjciFLd*f(Dg4_96M9Tzfj4*r_Wg{nJH&)xo zZR^P@VPlc~LX~sm&hNx$pJimC#AK(E7;Xfv zAXxU!?S^4))LvxDQlNA&&t+s&8jA`r_c{H+@a)-b)vHrGA4Tmo{@mWaMU+Fy2bkc- zo+#V_EO^k^{=gHG9itkl)@IFPl}K*y|4D5P_1Yxx>nO{y%*fnpJix)2Y7sgu zYc$#L?hZdEyuD~MZNFoc-GP#~-8rv*C046uh|=EfVq(|A;+j^iFq*yWAsnxn$A_{S z_mQeR$s@g%IfH?c*Z0FFHJe<2w$r@CV=y@Fw$BkIp7KU+A#SxtVWoLKJ`OZkQ6H{@ zI%YX5a7(8(gJH~qjH0XFtVQPec-TC*8BnTx9oFTW`09=ICyOp>?LDv^Gb zbNceuu(-ImlUYK|^`~`9-crLA&OV{kCd7V}{sNifSaUu2*qb3Eow4K1hmz&_YgLYf zI=JgOt1#s&54e0hZCYZi+VTP6JCA7mK_rrZIJUvOkW`pX??hc+o9a9)Usr3!cP3m_Ov&%=Pp)y z6n9&X(aC#o{U@CG^Y-ln1Sv_g9okFs^e%mNcNM$lWz>g^_U+NiRX&G}l9-!@x@*-f z#_)8tUSXBatc#YGYW?>}fD!Y%Ss3n6k6p}AGomcyDEnAd-cof5k#Znk$1H4-?&h1{ z2?-A7pne!DeS9oFEU3k;KB|??UrM#L!Lg?<$+cx;%O*qd4;G;@`4=h&PWfsHJ#3ui zCqggsQ>H$Mw#sRLM-tROV>9bjPq%2%gO+c5oy=>m&4GrBLJjb4HBoVEjl;h;({y{`K?t|uB z_7TZ`FXNsTi(1Nhs15*gA&4z7Mi-ze_J52Og^m`KhO6)-ranx z=)yKu$KiokM+rTR(Dhg1*2Ly#7G61||5qzPYYlxFJ+2eV!0tI~#jU@1G`662yj{#a z8}J^Xz|?0OP12kS2{=jPRQ-3NjPncma`P?fp(ePLVQCYHb(9e>swde^AR z5_imBdA#gPw#>T+stQ0-5_#on|z7x`!ggG3HFv;15%<% zjx+n?$B#*z!9?e|6v2&gOon;nHKwU<`rD>~E4L+q^j$ca%ZDzTb6>zzc5r`ouRoAl62TLYOZ6F3T}8!80z^Z;RWL3Ja00y0hDo)fij z89X5+#_aBa6I}+f2`c?coWXbEPCf1H=#V##7{s|E5OnLQO*Ct{9M~A&Z7g^y&!{Qr z;luR-j+10vEfEG~)v1@v*NT{~gLn!Uk6AM|o#ZPs7mY&aB@whsAO2pobm+cOS|SQL zNOJ?;U#<4#)>=b&8wBGx%*A1WqQ;d)EX~=j%TTv+n_a_d zSJ;qa<5&ZHHzFaR9jodq%gQV&oL6Ay4D2uZxt3H_iJYgXAy_31LI{v@N!he2!9*wL zgfk3LF7TkM6Sw6HXhKuwHurF&H86F8n|D@v=D962Sekq^(ey*2og>*NkyRw6B+;Ek zW5=>gf9$TTH)6GD>)c7zP-;^zGGkDj!@U$24yHql^91tMa4+KHU$RIE0;!=pHZ-*P z$xDD@`Y-?b?S|g?)~&iFC|?n^VNTP*i_p7*KqzCpAR`;GTyRXx`S#<-G92u4CI$&~ zUv7^6f^DMbpVmncqwI2ya?8OcbQ%Aqzb|Y%wrd2rg`uPJS2QCIr%812PNL%m_6taK zy*4S{UViRE*mFC|$stb@I}Ym0mICe+7qhD6fyU^-KmRELNZ zZQ?W?sV;I5+WN~J`3p&P%(PF9w8Z5Jj{X9EEE1Xs+KlIvKJVJ^|;oYm!#zq}# z*P&1#!#5`;rfGA6IXL82mKy0a` z7Sr`{ClU@94yWCC#YVhHvtORIhnPp+$9?){`I~%|s;t&9=4|dxLY`U5%|$)VH8`}T z!(rQsPaWOekFB47I@IzC8T2L{F}5hN*&TF3mfvMmx0V^4W48cct#oD)DY!Pb1&ZE~ zV2Vv2J!@S1Vuu-xBnC{%9}p`@%X*`I#-#eR<|b0(Syc~!R_p1BDiX^-xiC7rQC>0I zOZ`T2TABnYa9A`>3^hu9*y9^A>oQ=drzhXFDTwH9M>3+GN4m`*Tgxz(_ z2#My0tL!AIW*Sp?{Fg?@5G$knL`PeI?fF-d{DyT~KDR8Ek6I4)oMaLBY7K!)1<{(`SR*m*n&1Qh7i1%Tt{S118P!Y!lk~w7$ z*^YLE>wkFlcQLYA@_uN^%Of?IM|(U-es87|IrpIoXo)PF-4kBPLhVVr@$4D3`as8_7cGWG0TNUo z`Pu5}y|-Y7Pi#k&L-4{|_F5%bJt)$x9i<=(EbF;`$aA1norAdyx$*XOd=Wry(?{N! z1bnzNNh63w_8#9uvz5b(k`WH<7McJ*d0SFOe0=G)ISk<)sPYn}403RY%fGKMuuXAb zDM8W^l6&AxPNcnYwTTej`h^#Ci=aG8eM-U^o{aEtgTq`;dr+*Xh}B^SxFc=f3|r;V zA?9(q^&?{VAc<-!FDyzB<6#tDZ~A8&g3Q4&TB;&7VUeGi^MLF<#|wfJ@)wEU7`{u@ zG}PO$Nm;a?)SIY+MDUBp48$G#LTstV(I2(*1rJ8z>{itW2n4(bfK77L2+0;Yt~&vK z!1-pYO9&_US-t)oO77%$B+bZs^-(d%VvFfU6IDWk4Hqe6x!v6kYzEATH~f=zdMNIb zhPq9kBVuFO2bX+F>H;huXHW#@%V2lyDp>0n>+1Jc*=)A(Eqr?V-Q6JNQ52c7fz<33 zmuZBT6bPn8DtEzT5#y2581?+QV4)OkoGI4O#6)GSrlFxBbKHES3*JP!8y_OyBLyWA z6UwEY#*Ro=Rz}96@>P3fG6g)4BeB|A{h|bS!dlhqb_alQ)9mxA_2%7&pBii_o@0~@ zsZT|L?9f~IXr@L`%H;%8O01*$@2*t=ZIa_iD!t<&&_VP_^wER;f~0~6w9X)V5Wp}~ zM)?`ab$G#p^&Hp)S;zQ1HrwrzeRqnMr(N{9ZSLg)aHI9TB}yVC84yoOqR{&k!0@1k z#MzGxo*1|_zZEg8^3g4e5LCuTRfWtAw*?}#*=|79DSvZhU}I^is#)V80j4aa909=- zip%@Oe`GA03W0K`_8!^DxxrE^^jG+qIG{+8arhflc-aY%DJq zG(Iv4?AS{r4w2WSZ@D<&3^f(sG@4}c^F*^NGj{^Zu}r4wvL3fu&BwyB}YilA&w>FiR2 z-lT8<>sNEfGcQ&Uuco;z$sfve)BhZ>m0hyz@4a1)qSgm&_2KE!0-g;2VTO?e^aUNP z1W5nFdZg921*;)31HD`>6p22v+@RVUbJXywW>SiW^c@9K(N&wkW0o6CwAy(gS;u$` zmLRKsdEZk+DSX0bVC%qcSoJuR027aotMywVFO;Puka|L-SAG{!w^1!&lZLfL5<{9G zdGQiyL=NShfp$eR#KxrH>?RcB{_csC_RwpB^pn7~)?IrDpCrw-f^@YbJZXJsWP!Dv zigkxbFg5aZ*m}sy$>qJDu*_MZelyKcWziQX@U?R|Lc~jQlD*eKuowAU_$;uoq~y?r z);fee8wVZaN~2O~lVe5J9-)URHYf>73zA3{__4zNHP4Z;4u4>DT#dD{@+BQAasI+0 zeBJ~2kt`GvI!n-QuxC)dC7`P!chX6jKx@+=zf$|=Wr3^X9wQP!2H$9v3h@+hKnsSK zND)>+fIspAqnM=4Fd!{Ct8|p?bjSKX@7&582NoHrPeV;Sf6rYQ6J|PU@a)Blz1+U; z$US*uk~<%nv1hQvmJ0!2T}lZo_E4`!Jl!(N3Tjz5Q!Syknc33fTZ7!Ww~2By9ki)K zZT3?3#zXRRC_^D}*I#CAB7mhhggsKp?Kh9uq1I0aJGZ^^v4(J&|KpDn1@?O*_uO)!H7?z=M*{lt-ns-qu*kUc&wiJNr;RN(h=qpYorR%AM zO4;h0$IbNLcQ_`zFs@z-p^p{OU2@RD_+#0vDvpMk|3O zP4|FO0$j7t-&VwC0}$D4m>nJJx;-Q}qK)_IAHVP%Wr{LI8p&6wP0oyq-(BQq3nZag zF3M=z9E$chD^m{@u!KKGQVSZ6#k3FeiQjYIeU2k7ZriGWN2va3rCd~gEkW2*VMDBI zd87KrJS!Uu5!(@u!!_9jo&X(>XpJY~+uEPB`|f-Qz9W2}x|x*Y6ZK<;MZ^6?YtBh! z+(bLdiNg{jNE=d`TUc0FTGGdQ+zIv5_Bx}~xM?&R=iWAk*Nrf1@_jHaXzQbb!hFPz zZU3$1n#}%*kxRq9f|qO5uUC8&6-8=t>fzN!{3vXTqIh2*vYkHVq5IlFRlZ8204sH-k-)%?erM=(9bU@LRAc%<#zk_ZE`KY=?%7n z8Rl^NrA6S&S`&ai_aO2f)YB}>Zb;mBMF8Ucd2CUG)Gd#LivpZokwI@jp5h*}*QcEN zUSalHcm{)KLAy7tcZLan=%jq4h;s?wKcx01P4LqMJ1g<8XYw5hwm{oQLwdpCz{d~` z@100?6;&sdM1pps7^MePif|RtmG0e98lFe+*J9}r`NocZqi7^D4KR)(pq$FcXj+!5 zt#I7C6{BTI)sR0#rW29Ed@!k|~yFa2L)?w&?TGvuoas`HF^zYMXDK;8JK?vzg=9o zVpE?6Y1iKwG)YNLZU)3b1$3QV6u51QOZzpCrHEIeqWS@ZmmABI#rugBHB?`;EGAqI~sUtdu8dF z(aP`&?R=5LTb%>Nn1)njM$B%Pj(rIqYRbzxnipF5ag=p!Fqf6LcdM0DS}w^5s7N(s zvr{ta5n=01j*pzm`?Oc=^u@5D9lSPE-Dte78-H84!HuVy)NkXX1~G z-@nc`fY6iKE_`4FMSvfaLP~0d-#4r7K82@{VIWj=9W6e04{aiLgYki$@YCN$3A~+8 z09yKknea?-7Tc34=7D3HB-i_|`XbPGB;|@0#-B7EhD%Kt=1-vG$vf*R<4^j8<-FEz zwN--OoY~wk)x-8qu<|FZPE$Q>P!+h1nXg1!gvQR@U%Y9<-v6$F?O*$`DKVN~Mv5FI z0S5YByo60S`CThsE=D9|lj_S9RSv{!C-ZM}f5?t4`EEuE`FUR;LFNB>tkeDHTYU55 zw3qciT>CHE7k}(WWm&!2g|vK;0WVNmLZKaQmy^1C@fbeCfA)-)L=%U;dl)9*umq>U z{({VTNgHZlp?7)wj#T`L@^?L#Z^)aQ*nigGc!enuqzMkaebc=$-}Q?u6tNpN%}0+q z#T$f~naBSC$xjy_n7K0=U&wxZYXN-y7bNj%9%%rc@yvCz{~7k8hXD~Y8>)7r|CTWR z+S4_;R}tee)h2KDFU`0SU&kDIt=+X0vAh@Q*(NCieLe&MBfgtrLvHUsJf?}hBR?oD zeQt@0I8_Ec15(QkN0UnVFbxoCHD@2LeL^6fogo}a1h*Zwtv^-f*? z{z!)>D2_T896?efV@YPVAPz%Kv~@sWV1|SYgxwJcSpG9$~AORaAs& zEyez}{n&N*pC0G)+Mf(_!SRsBK>GNNUAQTKjvpq3kWIhXRmB9IVt-*j(%WiQsw^x^ zTFj@1F_C$g-?f-iK(=&4+c>5~*p2iij(oaT`jb}UN=#HEQzEPfs}1xqrefx?(rX25 z=3t-BRO6=C*E!I3i0*=WGdsY6Hr$i|p)TepW7#ApsouVIfd#ENwx4t&BgSx-bzixu zhZgQQ8vbwk*X~GPU7WK_MVwe6G?{ylbeH5@bk;Hfe>}eR3cr2;yH59L#3dK5|DkJq z_y4@9T;|(Mo*M;dry56q%sM(mo5-LEPokyNQzQlvWS4C>#$J^LV!h3P4t4YO(cH(% zj-IP4=RoS}(*w#NO$$JyEhI9ayA;!Y@VF!RPoRpQ{6xa%%t4aWjF{Nf<-@E$<9?m> z^ix^WtH@b@Gdg^6rUp)yIxgKue`S*dovn=TclN;jL!yM2vk{9tj+1gHgVpGR^e)|b zh5-vMF2qLwH+{I|6(}f^^j#u7k@(E7UGmpvIj{cc4d+)Lq^T^(sdj@}TiaIG5K z5(L6*Fg+kNE}7y>a@s+91u=J~7!^}g#-(Yr*Wbq3d~MQK+5me)W)?hL_mVUr57$#1 zd@mk5dZLU35LfKAi%txzf3TsYqX}b1qz-|6B)Nd;@oD-nK z;nKj#<@nqq8%3^aIgGRtgfABe$XY@$ol9eI;v-K)i`C{H9ZXM|;!46T|1)1sM3GQ! z$P73p`ai&ec+j=a-|e$E5%9rPiSY))-lR1}kLM_TRGM(+5?fvhq_wNjail<}8Crqk zjDG0}8I}dz%kGOay9dA^8M?O4Vz@!ZDC&Wew#gff0diB`3)6mg@0~VknZJ^LOHv1; zg5D+sgzS**Ytoy`#6)ja&ZOy6CY_^VD0bA0*Yhr~%f7tYL1ur{1-Ih|%*#y#9#?R$ zcv;OQt!1mMRbSf>rQ=&%9T(*#HoKNm$30#gS1)>Bt>U?cNo@U_G^6J&F{&>+wJUqV zl%Jo9JRG_w^yQZV(?5Q{ynC@7&9tcc*ZqGCk9@eCr@vXcaBO1IuJCi8e5MnntxeGK ztc=Xbop)Z1=P#V&e}2xK6;G>;iAR#j1}SF420DlEHz^|OWTuU#MX~Jl{oyy#)fh4j z+X;UY`)*SuNLb#)A$1(W32ZsoDnC8LH&`BZ)>Oc9R7VX5lI3 z$I{1wp((Q&Iau#dC0hGsz0 z`%GHpk)szz6wVI6XUBON#e_eGn;s`Lh(_wrz`^>Rg&~vqJg{>*IiOM;^V1x(@$OB2 zUr(5i_#s5At(ULPM}kamrUs(`VY(-V2z1CC_@nneBMx2@(97Oyec(dHp%FENOg(&X`RgbB-g+Ua3SOVUah%4b*1&lP zy%D_udT(xS4GQD}KZPBQ6Vn$PLU6T3V2N;?bk_tGAo+8HL z_xI<4`SGz0*&&0M`*c9Nb`K&Ez-rbgK6yuJ>;c1x;HaB?HJ763l>Sb~;0Y9$5Y$Ot zhygPNEsLWGgK9@Y-BLVRUS|!1>3lMglHOAVSgE*-*6H(?FsE#Dq7y&B};Ot4Y`8w&V3 zZ~`zSJlbdj((gP^NRE!fR~@&Mk##X-9;DMC;74W=nmlhAfm>y`*)c71Q8tUGAQym~XRFX!ZsV`D>Ryb+(OYMEb zMFro|BQYfx9X0xRmtP`KBl_@LJ|yc z80}~W0^;KB;E;y$)@6NIAd&Oa`fs|IfU?C@3(*~#nGw83714=6@!bobV?-ObuVS(c zL(}TW^dx8QZbt>jYcgOz`?|aL?Kwf$G#sR?KHqN*2JUcy3%*Ra0}<;tp=> z=GU3TtD8W#S4sX= z43bfDD$}e0(g-|neH(zHdls(HoMiMTU%5|+u%rY+9|E@N>r~#wFo`?Y&sI1Nb3y2x zq?INFqfz%iun=YKu&{YO0;98c+bxA3KXEy+q{A3aKya{*LroEALdqk9fj|7husW19 zeVjRaR@8XhORUG`-udLM9b|;T7Be!a(3-b{gb6SThsT1YMuV9i6M`%n+DxlBb6$0< zL&A%aFDr)4u@#Tf>$m>DGu*raR=1qcJ<=`eCw}Q^A(DFr|0Vypr|@_L8b_a=l>gp@ z`TpvR(GrC4jAFGYqBS2(ZWNIPFgk7w|DHK-S|S)Y-ou!}F#R>hU+7o5&N7G;e>EFh zK041Kc_?X9BhGGL=plLE9Oe+zyND|y^|P<9c{u$VT_;kHAOkKqVgPb$0qN!@AT~Ia z)D9KLT3>f8C%;irb;VI%bfTkNfuHKfpQOb8A!3<39sg?O(0``pe?8(O6s=E+T`l~tKS|$TBHFxzgMv166hQik(oEa@%q6JMS}N^vc6oUp z@BDoPg5gp;rn_bAmT$49qzyF_aVMoOb@fWVK*cBY`*UiUHfjuVMC*rZD$`ez5vkyM z^!NpA&%oa8$$JM+%$@q>)i7zMCFu%L8J-(AwnLaO5Sq+{A~jw@7c~2A{)V9UTgh3Z zU!{M7$oXme&ihN`;cd9r6{PE!aQ6VBE{%m_;4@iLG*60ngB7A%%w}i(O6LEe zmfz<0*H-B}#9&VL1$v{Up!3Qs27`#{0nAE%yz5csx04csJ3~A-9n~Dfjk+N&%|V*H zoGc42`)HcYfZ2W0jttusgeuQ_5+o=ANe$b7f6X1G&D*wZGiDR>{OJ5^wXqQ4BcSOm z4q?g}6BDTqlv|La_`nHc0(+;=MWj#x9t(yJI@(tiu?W|hO50K0b(TxQtTiQmy$!xM zR)Qo2Czm`3U4ej5EuF@u5z@2gULw+u`y?#OdCNA-*xRRlc`5J1$kavJl`wyYYyc*7 z5KP_jReIsnCXhOxUD_-Lf@23sIg_Y<^M3!;?kjgFz*v80y!;tvXU)N{-H3(*5u_8a zpfNp{Gjt~XbAI`qFpkhIASXk*{$*sywAoy5cAm?oE_T+Kp$P&r>680?2~I5VW^BFb zS53osyqzSoMOZ3ZlJGl&h$oB`Ba5jWl50Z1?dQiHFywu;g!})3g&_SS$XYQfgpegL zhT-BkhB%3L+^YbLKL4fjL(z3IGSD_M!AC}B`b>`d6`b7ZT!AEI7xd#$g1HQAcQA(a zei6O*ZZ8m;4+99n=La%c>Ff`M^_eamZzS^YfssM?D5xu)OqJx5NXi_d2;=rZ=4-$x ze_J7R!~OkoqW8Tp2uxOHx#(Kb#7u~TGhRCjxRWymxe1FzaLJ1|xfi7e-jZ?}JTKU{Q?wDMuL(7hoBC)F6(Br@p` zdSzKWA~o3}wSP(ize2}1AGo=>tY1S`>94h)7ykFS z{XI#08>RcAf*UsMyrYEUpE~doAU+E(jCCiFnba-l=x{~J4DQd3l%){qMYt1C>z`nG z??dR4i_0FBYAJ!sOQ`Jkmu%QDogHm^dyWtiO&Y|{OmtMOU*C8__W`nyS8dBSE4ctA zdu4QgE`t+lwd$)7wt2xha%vH49_%q4_4~ic?;BFZp@e!A)KIznS9J8#3ie@nqePr1pWP zFf%egWwY|#<>)n*N7}?m910oAc%b>)TPw(*$qR9u-uo)JuV516LJs$Zm&(&;*U#fp zOrE*{{Obi1&$^QPHKE)gTys?V@o0$)MYfgoqz~>Lt)BOPjJB{TOTY9M@UWY`gRS?O zd6Cz!B!na8j;Q6?67p9G4!rvAE_dtew}2W7L_?nb_Xy^F1Gu)#A|F~A{gaZU_KD!V+Kj8;XtLn)Rc zjwdMr;om?cO32K@9|VOAI3PxS7Y#WLCw&;_ru~`!yvwiSWUSc5bhuI^BY`uI_kBgh z#cV+`oAD>di+;%FyG8ar=07=JbuG*CCw=ftd^nV!94}NH@+w_OhBHo|>vz(@qys)> z3u@790!Ii74T3<&T&9BLe}Y<^;OoPO4`*X8C7l-r;|Dkp@&tjx@#^0Es+&Qmk(2g- ztlV(rdn*wHk`2vVk2oO+bn}j#*pStyrjNd9V+RZ<2&<&3KF8Bw=N%Cs5y4NN?(A5) z07jayBc4#vQNCb0L}C6KDDHniT9q4V6OCk2)81$4q{RS4L=YiWk}c?)-n5WYcWU=J zcf*wv5)?oaug*lEd%5UH4n|>n5<|lUU_VQUh~nY@@Qu&(A1+X+ho1y_X(S@+x7p7T z2x4p&%DS7UBT9zC1JjY3=~~=r9jeEiU3Mh&GW!jGForz5)V|8w?yW}W&$C^K!6%Zz z*JKzN9l>yM!3#JYdT#_~EApf*2U;!bJyw9;0YXFbNuS0m7INs4w}aV^z)Yss*LH8d z;cDVIR(nueayYP=1*C_E57%Q<3GOt9I!TiE2xEv*O!fFK%h`%C6 ziv=KzHe!!N5XFPj(5}5&NbK_jILO`CMr5we0xMO z8P`K{4X~fh(6z!w9Vfg=fNa(PBRKB8%O$lO1eS&9tck!&9vQvMATvJZ5AkkjCf(1- zGsY5oI2cS%leP+iDq<8hnPCMv_n{9{KKRvqcpY>wAAY!og+=a}y!efL864rs`07`C zClH|K+}x^4P%b&~bVw{=E?EP?Lv()bwIvzj0Q?(4L7v@Xu&=2fZ>Z#w2@xOfLKyRC z$k+9=|GYwA?^3CW(i28SV}(}f%OLDoF(bLc=}Z_nA{rfrp^9=c*<`#fy@rZ}=PGM0 zIwZ)@EwK0nonz>8Um%Z*kKc3jhICr@=5xyRKQEli55|qjitgilPkNL#FLN9;aJ-oP z`8S3_CDKhs#&%<$CC`i#o6ckRVZp(J2d^W&{w{YH*^AVJOSA!|8vDypV?7Bb_S#b0 zScotgcfj;m3!T8xn3q5%(ClGp_s+eJx-I)TAFzxoTmD|9Eu#0j3COrdsG_eI3Ve7C z;a+(^ijtPe#>fH8Bgej*HQEm?N{Rtw0$V!wH|EI zXycL7B1KBl1qA8h4;mn}XEIlXoY4vB`68^}Ln@m67p-CGF`O6~TD?t^G%$V4oghQ(UKk%Ea~ED45sp*vq+zLW)_$a&qEDCV9yH5~!*ZWU8EiAh2J%N*=t_wK4F>Us~+$yuf`WVSy?!om?#oos3J zqkC-`N0P+{E;o{-( zmKt`J+LF9A{rU5Ohyb_$i?=rqr*eJYhgCwzP?E|J4K$%rt;}Q6&eW`oX&%i{W*MRo z&6yP%Me}4HQX#2mG!{|`6*8>fc|Df)-kKK>$KGLC>v`_yzOUiD&g;D5 zOExj81F(K#MKfp3B9`GKr6Kj5F*>r zYJz;$i!7&ogvP>MO}>3z{b-Sq0i}zSuc?FOX}nf;^BY`n*R>2(7G&?wp{Dl$>$Pjv z(4z~x>V?}5|GXYYsulVkFGQ=L#=Yp@7@&jX<0nrj?#b7DpLGiFLYe^Nz)0S;h?ExJ z|A>b*e|d4WK@$-hWq0;MooR*3H&_J;$17A}J9hHqcnvnYmpzbe5Yh88-5a-OX4=Dk zelly-k3LaRat{^lK=-{1epbDDRfHRIt3F_fhA2JhXe<)6d?Z|!dw6)jc-bi2!}8s1 zUQGQ`?gKqd!o3m_uUJvY{SUj(;ZnY5#Uy0tCQ)NdJk4|(v&OqGm3$(mDs4cF3A{z1 z9cm%Ul;Yc~Qc$oF*H5z^)@6D2*mZ&{DA`dB%u#UhJ`(ak-=bC{&%jag#X&Lm@#kUT zQg-V3jxC*28@bs#NbMhc=8JZZxl zw}5mtcbi#>;F3&{<^%nsK(V6d?VkH{04@{e$O#!@=27p>hYueZH5!7coJ`X2Y2<$? z`UZHlw}1{8Ny%U^M`v2ub1=EMSpPhH+^UE#k!2)=x8nA1&Dwe#Z@E8~LC6!KCpiSa5pzr;tgX zdV70=V`jbjTC0CX*)H!|717$0Rc$iD`9);{c|%7EqT#mz0%UfcUH5A=KIe?$(w+YwaUzH8wmIi;=oCQC$Y)53?Pk~#pobEVyk;W{k= z*alV^zBL}qicF?*G9Z&#TUS)qGK(qW$?WI+Ph?i{$gt0*$PsCB`Mq}yQQ6@e$r9SB z@f^EIt#Qrnc-Ow3c(w-x7r~_*&?fTF8*eDgn6uy`&_n|) z!7NWq9VfU~7zHe?#YI)K_jP#@n&pqLtG0#QmBFIOc(`3h^uQeHR{!`>kG--#XLPGx zpg2VkuBxn62tNs2LPne5EV(;2e};It!WOV84ccBKcwJ(^8{pTq?@p~Dq~i1o&{7o7 zsjfK${Yl1I>nTp6DQv`7`|HGP#%}-#Gu=KKbz9VKie-G%8KtOBV&C7@VLvm=1!NL- z^#Izig4Nkb_NUY*PtJoO;gih&-aXmkCgxT047YY zOap?hsb79GE0G|5}n;yD+dl*arsu}mVB5RD?EvWs!@dh z1J?o$bIq%{^x%Q|H}b5AKEleBkVDDIojH5f9<&skRp*@#9BI8jw^BL?(v9n0`H0<)Z?|d~gf`Ba6P4_<`-<8X!bz2Y2|v z_x^duNcM_0*fcNVHB#lCF}(5?022`k5s*fy9Kn`I>6hJ`&E=~PEsJ>8<~G-vc|44a z6cvzw{|nvSf*8io>jO0$gFS5K@{NQAI*hs^v{{W0 z$s|oIE$&HVSQlrUkNhAdtXc5;9G^7U7O9uN$8Jt*MJY%s)7bziNk!Xf+KnH4*!?IK z!duPA7U!z{kqLG;i9!Q)pN2})Mo6AIB6?dw2=X9GHJGc~s_cZ#PWSvci#mHqq8Ke2 ze4t89oh-AEK{ZHZ4w+MIWao*KCyS}|irkvr&4rw1CfO#^OMraf;!mAQh^@h+H(B=( zY%;BjKe77sB#>i$jocX|FrEtc5Zu|fstrtZ=e(<3+Yp|9NqdEe? zMJ!bEG^>hBXos7+OS;-TLt73{>`QqCYzOn8bPs5U!jfU+;t!b9CNwA#g%F~^LQw=HKzX1 z;0@wOxMK3Amnap4H=sB`a{{R0E!b;}&j&(iaafKZnHor2SXKouMFJnDnmwCn1Wzc+ zFG;P4>>Fm;^gK3?XKz9VeVAMgi4b-7X0v+ZCRDE{FoO-Q(DbFIAw899^Wz7aS71Sn&4JSVp&XCXBDn} z*^8FWG&y)|^IE78YRwKZiFXtIoVKVBkvIiu+CJ6k@#xWED6I@2rg3Gz#+$_}#9vugU}Meki!f|@`Da*5*v z^OL^2$t5#vmzdeq^l8WD@km1pr20Q2a(Qi-CFljXKU=37D-QUHHH!Y0Dl@3u{k9N5 zK)DPR7l4%c_tHMS$5ZgXjSrU`&nS|NwM7CM+;94j^b~^@0a{N0N)6B>5ODltIfARZ zTQMLAk)4dn2P7uMr3JMB>%$ZsazKjcp2(6M{6Q@@J+&YE5l|bQ9Pt(kNkc#a%bA|iU-%im zZ234*`QW#IW7zclzghVD`a_5wXcklgB2Qs-au}CnE;KPn3zbPwHorK{A)zrx3U3oL z=z#DqXgA9ASIA7W8jc0sOuR9e0}S>H$idw@EK&GJIe{gGqL?C8?ulN~Kr)yZ1?6Id zcacBtJGeObw`4AuIpYY$rN%S{Y8R_e+a+$!TEs6*Sz%^1N_8cvfo?h6v@zlhc{rggOD4wOxPU`b7 zB3WBz>sjKD$1cmkUEzy04dNv0T`Y6(j5aNKewr!#Ki?ca5g$fSclWE?S>vc7pE-bi zgo@bozNuoEJu`_?F~t_|f-gfmctDalwpU$1ScU>X~AsI1bs@C zX*il}(@|yyv!zF}z7fWb~qU%4yNLh(DRG zXcyI37Z2Qn{7{$2)HJY73GGE-LEM_O3m1&;otsaHFOWK+iR4aA_A#J72_Ghy@Srl# zUc>J7iJ)eHmX0M4NkxZBm81DRz>J$j+dn5w)|QqEF3gV*fd}r6j5qG+sOTy0f`i)s ze460A?nqm9&I2Nx)O zpJp8aO&XXH=ke(@({@C08o`lA-YD{z@|G7jBR+~ebq-A2$XqP)yE)p`)7v=X*(0cxlc447-#72UU6Jau zYW;$K;Fhz0FM#S`IPinokm34Hwn$;6mNa!n}cM0G7lN z1!Ze(x>IXLBD7jP(OQQfI6md77pA?2?%5>692P*=1hqz*zSJ@&7y*=6CzOL+^y~cl zekuZ_ri?6l{oBMi42hM^7eD|pQ*{7(^@p3^kx)QIJB4V+tfw%WC3&b%YwFaqrP42d zb8`CjExivAu@NmX*ejd*WTXM{ipHbP(RO&ewR>W-rLl3TW6fDyUw1UFl&Hrk4W!bA zew_lxs<~sZVe>!SUp{@M%JS#Pz6)BPY`JbwMM;vLUjB@`n#{+SNls}Erd9fW9UGK2*O$ z4=^G3Q{;lwX{bc8mk(8=#6E0I0G3Uf>|-cUL)=wBa1NqurF*}SJ4d|To2@a3Mqm%l zc{=*#Zd+vNhE}a+HYfeEewgfTdjyZ$VBuMfFFPgLE4u#5C=)!rJf(R*SM5{ii4(OIlZz&nan8g`T3u$$O@ z@d3kmVB-wRr9`RnAzKc!w>JL4*~JHG76D}|pld0pva9VQVmr=s5S}25LJE$V>X%kg zQPN*d(g}wR7I&G!C&Lec*F?(xpo~pLKI+&`hj(2$y{9+8?V>kpPd# z$bHzKtMkW(8Rw*8PPv_F7#a0lRzlXm0Q7#Gag? z>Psy$P61Y1Z(9iweEUMh~Q!OJC@$t4n`74f8#S`VXVI+8ax1aC1+#qa+nEme(Ws&UCB$x0<^V^Gh-aMw8TzvEH!m*cD*(+}gdNVgXV8 zMO)**(lEL&pWSsJdpw{8_lmm!PQ_J+0rA(m`ta8qxO+!9R0EHm3v3%1Y>U8NkEwLn zsbUdOgySj)f)}80yUQrH;YCCsW$q(P|#y99zD9uYo^JOv^Evo3Hdd#xH2MO zn4uF4%s432gPOfd^_dNO?tVOICBS8NhNN?(981x^+`_32 z#(GESBl)3(Ck0_9A!m5&;8LnZI-(;{a%HhEHf$l#>F`Up&=Eh~>p-{y0uic#Aty#p zNv6)a(EJ!KEH^f9E7~kcd&B?n14!F0p+$8=H{Z*zf*CSxHX{A zM=>piL&SHkRBae7U{*FWRI4~u*2rS=!v@7RC<8^W*^4+4#d_H$R0a%R7}RdM2+;|Y z-3llJ#UdJvkVFj>+WBX<#TJtCZSWCdX&^kWNNN^ZpAx(Uc7)}0>24h`fVO-}E!HdBdFG=D(gGNW=-6_Ow$;2f|jQo>a{3c?EY!f9RE)U2R5WM=0 zx}_@GDW;%pN*g1s0i`9!2J&luD|5n!7yHY@tWo5ZN`q z=4T?FjhRG)J{YiiZZ{;FQOncXzJPW-SC00_Vjm>1_^RE8<{=FNL=EV)IQz7Z_hFi8 zgbr-<(mZtymFWC%58&Mm6zJfPz20@J*<*?vA#v19W!HeaB%w>#CWD9#P_}~n^5<*q zb>WMp8KLr5ucY0i@JpKYn|A=3ruqejexZq+tuT~10xuz31yozyUs!MCNOHtvzeCHwS?@jf!utlXlkA%I z-u-xjNo}!Mf^xRZ=W~ssR|dU%mW}nH&~?`XWc4J@2a3XL^M`A(<;h{ib=#n<+Y_S4 zLybNJeS!pR`lg?Q#5Skt3J1S(2NpJZu??WV7hUa}!16haDu`^;$U*nXI?kz5;IU<3 zyk1MB6&LLz5)rGoOC|tN$V5gT6ENw-j!^@k3k`fJA0Xg*$tqJ+mDPV$+Y@P;3bzZ~yU+Lu2;TK9ao|=gXY!CYKOr+iXWDxsT&A{7`otB^{5O!4+W8QX9An0bux6(x%dC$>5(gQ{F{fw`4MlwiV=lqw(?{EDy48;TO_FL< z{18yQgSNty<)_Dd{8;=edl-rT;Dsbj^5q^3d23qPfy-5A{?14dTcOH=1G@w=kIQ3N za}|MG5H~8Nv>Pd9U}-;6NW?b+UA1PgV61wL#o{!*AT+!N9$migA#;VcgId|p9N}-& z8Np6U_{b% z;_T1g@)TBgY<$ntt&HJ&j?YtY9N0XO`P?!GBzqYyEziLXg4jIBQ*Z6w4=4R+w8a^e zA1~Qt;vpZng4Ff2z_TLXZO}wR3uz0RtR#Its%-^U0oI~A1kI=yq0I^SWw4{_Pe0^@ zJ;;HeWV7Li(Qn!!962X~rO}MgCua?QE%yGb{Z~ofclmE8lMNKSg}l;0QUa*@w98%i z9B>f#nt@vfe2d|IQ3uC=a83l*T3buoKpmt3+2L$eI#aUoH?*E%dt5n?>qt2?fVc%f zE`2LsP_Ffb##(FyzJ~^xlT>NoI+ivR0CPoEp}(5)UmS0H)d97d9+P*jRaQneON^jy ze3+ztnRlwDG4axlrF8K_tydiLf(?lPun$@wp{NjRQf`7~fu?J%ga9B&J_H{!4H!j~ zV)mxZ^G;J)iJ=K~II&xKxFp^%@c1Xp7c%9 z8Un%c`8%ZvLvU3s;v5DX3fbdRmG_$L{dvmf${&`C`E z*5#bYfYU&T515AVp_s*uPP)Z>$2G)O9QPHEQBy0H_aA>Z9~h-|Iv+zAbLzB zu}3~ZE%HD5GKg?~2h%QsVog&ksg+!jAqoBCCWZr<4;n4fN9k(xU?{pa?r%c+;ip57 zmeZamSD#!ql92N%>P7mWY;S!ao#Y1wW0j+Wus7u{ReK& z+#}cjQgIj(Rv3JtHix)7_#M{}NIUrL5F|YKorvfkvbGWU-qfjG?G8@*s@1EfSxLtz zQx2#Kqka~(nw+_h(R(WlQ<#BV45{VZ&5w`W`I7l0N)a)g?+}63cm8w4VU{Yk-*K{w zkrl_ecsS@%HyR?}urkQh%|SDzul@JE>|cA^Z~OG#9vsz*La%7Mx$m#UVCgX@;5};Z z{8^Q$b!Zu#p_(<+B>FLTs7W-*89?=_=-LpoUl&o7NCR_V;9J{(;(rjRj*`+O z6AVw^q-}&YQxG3Kr(lNCS{HA0(-F;GGJp^aH3)8w!EyLEZ@%{MD&!|9A4)rUVzXu(T6VWl!Y56rYzIX%3QaA0Sz3dF;+!P z3-&$g_6F>AoSek2gmBkdc5 zcIZjBcA_7u&Y$7h&f{{BBnlR3Ai4$^tAs&nYPwJv6>SC6BT zO+E6JL+I}O9MZahX)~P0S^7{E0%uZ9ElY681$3!WD;X3c7NhFC#M2TrQSxoD?@?LY zC?;a+O(+#ty`hg+OcMP>nE>in3?e#H6{5-Ye|j z79=&`@e2z~qRujalY65VO?i${Z`cjieZQ-+HB-#+7)%#rK`Yc{4zt{~X`FCUTNHhu zNM3=vwWkH}0G?d<2$JKW=)J+QUFf~?8Ky>Gtz+ILr+o~nLHP2u z6BL~I075_QJ_F_K#VN+M)Pe-vNB3A^Ni;F)1qaMunx&VEaB<%|KmP}`#i)l)K%Dt= zhj*Od&z2iKQdA9O#>}9(rom?5qYbGLXng9Fn#h9Pq4r2dgaUkf)VOFPdA*hlCV~Ni zMd(m2q3UpYSNpQm%66N9Cq9SZ`1uC8oDO+IeG2B-0HpKtrxz7Zz30xN2uuN$`K7`P zibAvU6d)y_@lyqFx@p`ly`JqU=!B+G+a&Kn zVx`PK6t~YsnTX1YWbgZKBLN^87HV}u_qV2O69f@eFIVam_;3r83?T2nLqqZ=`h2otM}>ul+FnIPIbD{8RKSHdO1%+BQVoiC z8>9ZP9Nsc_oHIc`hq(qW6Mv?!UX|LpDPllPzZ-UUEaeU+?zOdMJ*l6C$Ll`YrxOx# zN@e>B@FTA)pKJ?>T)WJ)tZlFZGOyLM*e@hxP)a2%gf;Cw?-EG4&Oj=Lg4(ec51BB~ z_Qw$3AHBgZ^%mWjZbn)U2vu08Z zpeAo{eZY02G%3{!U8X9~bfDHId*JsZNX0_nrJ*6g#hQUoENHZ|ciVxQFw`~wvuAZB zE(N=WY>J#59@E-WZ?n7(3_Wz@MI@s{n$Dz@@I=~`%qE0u!;PY+7oyB<4}Kmo$FWXK zhlmHHXBy$grQOKQ+`_E7?jEvU;U_rO>sUfxN(MHCXh1%1PZ^p}aRUJ80SS^=t$x8Z z(}xw+fLGf~1IrRF~} z1`b~aY}yF`Tp77dDwU)B{CCxbBHQ_}JPaenR&3@!I3Rf0WRL$PTAXZ)SN!?TZ&R5EW(kSlAg{%l?^jIc!2c{(tSK!K;t?q9W7~vhxG@PXB)1%rDX8 zk51WipZ?bW@=JKfDOc~Jj2HDk69GlcueNJ~=nLil|I$NZ-WTj5^h-~48p!TD$&MPl zy*vL}3&Lt?NN@dtnqF(zY(tKo5rA=w`%&b`i~PO#(>5XR%|Nvgo%~FKg$7e!^K(O$ z4}2r`X?_ zsDdcsd$u^@d!I=-7^0$sd`^(NnYfiU^nZE2L>vUWM?4SxlkjXc{qJiy9bNxyi$xM> zSKWgyb@Np!}U#9&MU&Cj8h$Nw2= zlH?YwBlXU5*l%h7c=)&|-^RZ$AK(6eo*rtKKx&E9Bf-f6O>f?xL=pJOqkqfNfI2{= zw70u&BLE(%kNm@?)$bknw3is4fyBNiPzu~CY=%>OP9O@CNK!9kq<2wd(F~^gJ8JI}_e>K6&B@LxmSZS6jWwf9}0V$sA(b33A2H~D_ z6#n)dW^#yD8`_X)GZHV|$lyqEbsTxAKf)~d_puPNw+yn-P0225tl>=VD2HE8`_rdM zpPC7b4<+&kz(=S3cNmCrRt5&5yyV~MIo?2F*x5yUHS5oi3LEDT)2?-p12tJ0%1si` z2oJGTT-&|i_M($ zzGlsuZr22e9v~1qglJ66x~U2~K>?Mr4Ey3D%-c)4e0lccqGN@pGUv{Puxg&6g~e;T zCNk0ZDpN`?v`_9)@F<>enThIYX=&XG?vXPiG8&4C4vrd2ts!$E zSfSf(`|ge-T{wk`F|MyH0?LlIxhrE?BZiN(20!MN;(6u1?apMSn&Y+4OosK8_K)x>O`kM>%W_%C zNMX^&;?6q1zI1Gp8Qj7Mt?i*m8YsjIanlHt>Y29l)AI-wXS63NIm0ai4qU?l)wv|5 zF|oq$7pe7xkR?O=L5`7s!JN;mZ?vB%IjKEXviC>%K3R!sJKMGo4{01c(uG5$D}%)s zh;JG(l7>Pc#iSsd8r~^=smazt$p^QW`YbVF-MWMqAM^RBZyM%YFqpu z*i};Eamz0i_QtL&)n`zr#t3+tX91a)3=RPfJ(IPg5nMcvyC+9mQ{4fY#yauB$CjN>LX{JhLjH6p z*i8Z;Fl*Nx8C)ssMILC)nR(Ou`&#?bdoo~03LwP9?;QNN>R37n-sA@G^1fX3UA03> zTu72-#E9+$xsqUBF?hg z%;3zn#(TK|$)z7qs}!9uRHXo9jN2OPx`r_5N3EPS!AjV=Ab1O3@7>v#W88*{9zB+7 z49+0+Q@U#z=rq#2K?>_>Z$S)@YKyc`(3%#oJBSf|~jNtXZt(P zetv?^;lA#6=oxk6#D|de%o|BpK$>FE_+XCxo_BZh+Iv#_n-)D2mMrHLg#c1#o9w}Q zpH0%N$XN$vJM02H4FlQhIxCv!9CM3w)cMht&=n&NcxIaJF|uQSrf1#~oi;Y%LtJgjguB<0ieRqg1|ZaZ?PPNZE69 zuQ}^xCHs{zSOjewu$WC@0eAF_{)#;^XP1A?!vp&W*GAI55x{5$h%Th+j9kb_`+ z7!U_xJ9UTp3mX^pG`;l*pR`C3X4)RtG-6@#TCs)f)D;(KRggmDMgQ&`<_|!nke(Ez zX_?EU$r+cgmxbczMR>xAhWD&B;+5Ab!?`jeXKV5cHQt}qcwb2H!B#k|!|2k> zyN9JIr0tvQN8dhthaLUp&COXPG}i43gAwTbal}MCBm}HIB(G(e|$ZJz4y$?KI_^$XxlIW`N$ zXnJ^ve~`EYCKO+_y1x-eIRnf+^k8aSKA}frO<=?riCa>Rga7C6x{X8*%_tw+1plDYXW0`=hAs09-A zA%u!WaT*7hvwx&Z)Vw-J3r7jI?@NN2w)gyAeE5(v$*b(>e9sFz$Y19Vqn?-1Bj3`L zKQdp#n7!0FwccBLyZ5s*0X;T6GPmh;V*?YU1q+AQ!&m4?i4YJ<GK}P ziqDtOBdVT5udD(S;+nmUu{$t%rKa%4_R5XuaKfQJ?2e{520q6j(uetl=^3ZlVe_|z z;aUcDU*)CV0S7cKuVJh&$M|Oyw3Hg4E{dlCZcWjz)6{tFwK4>Wq4Urz2*e zln4h`AkmK_c88!lkCJKXn?OOCi3(tK(Spc0h(&uWVfv{55r2vL+e{bvM?RX@1ah}Q zm5yKbNJ+H(c>fWwC=r-0VaSOIMG&Fz(?Y<|w}UYRJx6k(MRd?fKbd}HYFh5&V&B~r zfiMv)worH}sO=llXf_h2A%jebJ#7S;-s<9d@X5o7#6dI+-L(OFYI=-GVgBQ=AIC+A zhP?8LH>!;#i7{aR4QRUDjXzU63S9+6Q}9pYksc#3rf`=0ixnR*L>^bUN7->eT4@C! z&z^ko;>@zEs=djnHSt#C7#9HBk82Y!eO4YU#^bGDkkB)K``80mO}K!Cz&8)eXlcUT z(3o(AgTr~e<{ZR3#AT8RNdM$BD`kPv`%W@ zLS%0MtX9@Cvdtvt0#Y1t#42Irf8cY%uD-^(BcDN!?vL*W+a01Qdv?9KF@xd&O6Dl$ z#PeddD8f*fOe*L?c~OTBEAD&jcuNNzDq{uO=23GtMP`Xkd4@Uxt4^nD4K9IXGSD29 z2n!Q3US3*D2bu$xa!`Qjwp<0|PDn5jQ#zTpG!PSw`+nWdo}*r>nSEMaMo|JmIGuJ} zg69`|nK1B!NK=FaH=#Vb9xenag@l&3r=4>z9oD`VUM5!%3sNjXO^J*gJG>^OFO|pu z(LTr1B&QCp$7IId5lyZdRR(a&Q=h#~z$;0>l#VA*38IeSLrEKuWJ}sJeiRd1rD$!q zDJbB~ohD3$etLf1NDo|OJ53HEK;REtkuzZzMf04U1PGn6jnr0-N?rCLl!`z%r(GHn zB0;8Y?b>28dN}gTm@9Z@DmD}6yO4O!RB(cUWVK$tuDO4O*K0%5i&{SdD$5acZbVpz z;HVxGKm#LSyl6v^=c1>|(L*%^J&%f0AeO5vO8qsRNq zBX9S!=@mUZJca3AuOTMic4|cO%fy<79`=dhC7t_3um-j&uZIK;8L*5Y!!Yb5g<&ID zBWOeyY#f~|6i}GkCJZr*X!8gj#?w}$ZHd<$JARzDDhVut4+o0?s@@WmVDZchP+UL+ zbKzM=0(x{wr14~l`J-Shp|8qE+tevFCb~VH2Di2kQWVZoXBCR>GSJvvLv?Cut*5*L zj<5)gnoP3j!ycgA0D>vgn9V@y?+_Q-PDL(}ZI80Gl=r{hG+H|fSVMD@G2QD6>fDCr zH5IEF)PrHLFU#R7)20v`8-^kyD(HrjzS<$CB7484Uy&kiSbJPKqr8hC3=}a^Ss^pg zKvMRk&RMMaD+3u$7(jD11Pr8m5TGiqydOysZH#&f97pg9ksvDl55-ix!^+g6arw$FK1jxlT<-dqY)(Se4&HVb65zr;o zU7|XXTJIc}Ipn#?x4=tLjNn*dFrFDb2P_1B(Y$#L#d?&%Ao$n4=39}vZfiha^drie z=%;tLIK}wn_K;1aVIJe@$vlAC4V_6!Ok_7PO~`n;+Cmaf)~yn#6H_pjzo_P?$a{E6 zR(f55+N=2Sh8?{X11dJWl_;Ti7ZPgAzkM{CV42(#KVwr}T zfv+|7bogj)1f4pCS+l|rs6)WpDTee&sBG=?+mt;+5rSXT7g|r|y}hAW-fPCx+3bjN z#^E#+Q=5Fuz5x$<-W2^_SeaTxD*R|+-OI#~6!{>Rs_Rum5{P(?d2qCi!4FjAG1Cpd zarfCLJuZ`!vF$ad6TQDletx(0arp)L`VYSsW1H3faJb>qrD5LG^x_mM1$%K791TI5H4FO6w^q0nwDaVLE5c!akP^%E5L5)E(QZ(nC`~w{$WY%!|XMap{ zI77RF5(9D}G+QD}@t%S=W|R~WOR$JMQ8jeCDOw~&SQKmK;z3h-gBPYfDo|0$d6M7{F?9a=*aT+Q#gC)ZIjJ4q!HlEERh&WK-cAovQnJbueV$aAp%crLwxA9772zl|_XVg47KX9(-XkSlbIVcmp&s}Hx zq^h0CZv9kl^VvsBaq~$lHw&6RK&lMntf(Ur4)^Jq1+!X+uS!JyT0u%w;IpZc=I+zb zOB7_P&cfEvcEctpgs*8JwH6GoP*Hr~u!hvWaPvSMm*4TVLpqr9{oy>ETto|wxg*Dm zS+`+GITn}F2-s+Xv!KZ6y{vKh$}e5MHM|bLk>Wi~ zs5x7>P!}PlgUjAXiCh_D-Ici1B=(dyejiIa70NDV1Hba>6ehP;*%o}sLQrRpY}KS8c0*Oo#9yQW~q zro!;(d)}DPXD`#c&JrCKv@8@7CMRu-DP>7sN9agoJc^7cLZ`k13ND#WT%v!_VWGH& z{*Bldn;ESkIoXqT=>0gl&@|#kfZynvZ`F4 zy}K7p46HXVpL)g}H~JaU3H%ghB%tfG4N)hT=_J4n0uDdZ5o>5up$ir%jl|XU)z<_A@lu^uP~2b{vv6? z*2&#TXI(RfZQyAPkR1^;Y*eOC=G2fRR%2wo#r`YSJ8UwYYBol&WoIQ_zj*2X)}Z+- zt=^aKj9h7P^ytx=rx&k24RP13X?mdcwI--9OTckpMrcRDwcP8vd|SVEK1kQveYeke ztS+DIK@r`MlYzt456FCP%yP)A=~n+XMSy-vzpB4})Ynf$=_o6_<>&qvt*$%)I&X5h zmRJc2=4)4~EdDN^HAY9I_MT!^+gO7N3t>aP_K`iaM;;oZE20!t`Qb@pzj26CRrIe7 zCP7?rKl>|h{mksS(bmvmJ%{uDO=~lS`yTK%1rC0kE}x^@RNKh~e@zXy_n$nOZ@6cT z*23$1k}GO!YkhlqJRX0XcTdqrx}aQ%Z}|R)(ijo{V#fkp({HttE-yopmhvAv$g(fV z&VAyO^@oD_OYZ;yyUigt72SgLSEBXOXW9O z>OC7hvwZzLv&M54G5yu}yIL0&iX+}@&Pel_SD=+u)_KimHs?-tL&{BOh6K8oOilD} zmus8MS6x%ngjjmHv2o(XnP*b$B#JDA*T$?YX3JjCy-~Q+tn+_jh&SD%W=Pwsm*9;FIBN|EP8) z(q{E)T%n5b>?g~r)Lvf^4iD}$y>Eq0-aT72RtQ1vi`Jjlr9){WS`R;|WV2=C#^GP} zQxCLvGDqMpD|}RbB*ImfpeM?L=3;@Cioq${TfR)!RX29ZSsW@SS!}m&-$_Vhe22)` zLDI2O_v{AkQcujlNqc=rQ_h9!qK0!FMMQ*n4+bb6ozHDxuwahZ&q}Xd$9&$`Z~n!8 zYps=~y`X&mQr!oU^{lBvBkFVH`G<$-`PW50>s%7Qpm@ty8TLDmH@iYhGE==YHC`9i z8ogD!TbllB!GzYOVg^6&J7UGJRk_CMCTc^p z{r-vgcU!%ldv|W{&6D4&HDQ^1+^=Yx=~r6;oSi$hgwHW$$IXez9fyGl3B4~ZR%;z& z4W`lTqo1u+9#FF)k(oMc zrFwBa%VfmNuWn~!W2N?dcu)e>)|J$*?$G;dH!eD>E&iujporO$^FSXwY~EEs&N`?M)7ghpuj_HZ1G%2!HXAdC3r%UY~TL6@cyfbKz;nJ zJ|h!1z3!x2vovFs1wFv+?2cf!Yo$DaxnXzM4E%K!GC{BM$LcAIJ6z|vmuj}KJYSo{!>b)PyNLnWNmKtwZ+@$WzHEWO)isye?`QYkkLaH>hK{7WG%Z*<`)Bs2`2OoY zMf_o8abmv5FgQa45~bW1{0XZ6aJo5qhX z-_OD&aC7C_YCqeZ{4q0On@B|H)cWM`^$k*jabn6#PZ*jX`{4b#Y4zs5&x>dKhZv3; z^xV?j1Fi$GRQ^>bj z|GxS5YbGY|peJ;lmDOff*W({QZk~CZ|Mf2Mx3?-(Mq5=?PSE<8&=59F`fU63mQ^C$ z!@?V$wf?Mr_pTA($z@0xXJwUP$H7UeOdQPBLlHijy6AN^@dwBa9!C<{NUl%^Kj<8NI#(n{mz1hyr5443qSedKP z$Ns!cd4a+C6|(W=(WP55X15-CJHDy7`BYnS{e`LPYCWW{w$76jLSTdS5fT!*sq3%M3*GJu2Iv3Kz9mvWOI+TjUjFL$#w^OtHRS_%8=hoc^h%Tup0Y%izw*YTkNoT0 zcvc?|*g}T^ciJw;pU+k9I^ov=?yTNw?iGikd!IH`RkBS4q!mPFnF_s(Xd4^vrqSc= zY1Q6?C#&Hc%Wi{$FYTG`yt`MQQ1;t$x|PJw@7;ThnSK_+Ju7?K*!|h@_xcw3bS^62 zzl|33C3loPF>%u)?Zn3ugq52*4a6FNpdbU&yeQS?X>Bgh=9KVJ@2!Xl;&$`nciG)Jl5LD z_In{h14M=%(pT1m<>%{B+gSH^SsY992@7w({q{$1{|!CAix&|+oDe_h-Wz+n zvgXSB_g0ow8m+0*naIgnVXnF7DrHdv)6o)#gL&nW%#U)LD*1A@8lP-ga=E0knb@&r z_eD8|`?v$;dA}E~Z31_b{FeJ?Rnn_sjN}8~fli-+^5PvvCck@q*lnHQOLx9273@2g z_n2BYF9*{HdbU^}?8+s0vG3E~f$61NqS?P~*t2kmQm=NAa34V*h}mL)MwXAnW^9l8 zYd8-yr!4Qg9f5`K>r)4aEGa4Z49aU0S`R#w<_fY?mK0jdV%d6m#TFJCY}vBq6eJ;& z=ILCpOfplDY~AJX*>7q4JBj$hB3BK`t)~XwZMY(SWJ$pKD@D1v!%!XDwR<-m<@{h_ z)UsIN(b6#s1p_u_Jc1tFcty+FH!pWS=oi@AiI#7(^eTK%WX(v~xDlptE$gtmSXvvG zNZr{em!#l*IRGmiv2Tv=%6y*qu7~ZjHQ#D(zg*I^e<-LME*;xzG(A{#jv3dv-eN~d ztyeGi?sAUsSW&Fivg-TC-@rag(C*!))&9%r?(Fw4XEt}}w3-SoBGu0g**`acR(f4J zg&vOe#^cKZeR`DG({Rj%dmts~6ML_Wh)NxFnAP_AJ-SZT(b~x<$JWvDD9JINTQ!qB zpyoigJvD+{hxr>A(0pjEq`^0VTjxzhEr<)7=1o8ZC%ncg%kwlEJ+Ixl^9W*{ADt%T zE1S&uW+b%Y@vtJLs_xwP=bE?mMfF))Rh@soCsoI{wAA=jRaIq<(WRG3@AovYg!;YX z;R8gt6+->5wkm|?6%=rXg@w`MrD_kUYdtM5uiSIH2M48b>zslKp&m|i>zijyUT_Gv zS9ZK&(poVcu_eXBd5m=QGHV>y=hPZina)b-R*APSPwMpJ=4bXV{-Zu!lfN+@{Sj%Q zs-KrQER)c3bYionFFCTL%YN7GeV*|@uDgspR_3P_kEe>BCj=C!AAsZxv<|w@X085I zw#(-ZPt9^Fpz1+BUYcNU*^E5P! zZ>|T#U^er}y@{KyTg*Ex*yE}B@Knm7t54#Sl{So#`|cHA=kUl@8M0p$U)^w^UG7^2 z<;&vCZ@F&t(0Z$nPkevB7Y_gP`1?&B+&YoceWjb`H99TEg&;nhJ$v>wc=nq+tM2Uo zITHj0G5(SKfy0CBVtziE%&eZZwxi_OC3W`00xJD^zPqPsuYOsqdv`_i;BVow9)u+| zXZY$|*8Hp($=wm%X6iPTE&SM=&f^{1HX%-oO6Z5pqDs}*<#%g&EU z*`DGTfF`ue!Gn3%hpmZo4^vs+tBL5>L)uP)JVsZqdz|4z>n1QWaMQcqrdTS@*U%_! zY}_<);i;IIbMPyR7aM8D7iZ31MFbHQkX>i<{et*RL(6%gfIY50eNw>Xsp*8*@aybWcp5lu>&8Pi-XQA?0#G zEdTCiX9tm>ZFZ-;zo}a&9A!+4Iqh7XyI@`&yXAs+dzBc|S7Z+_ap z=XZZ`WwO@|miL#raknh{JZC?tU*z4esQac(?*e36g&RemWR~q;ryDZ<1+cygPap<0 z2eo}Sim)j-Je;z<@73OZ@$62u?9S@laLMYeb+H1pqt8Xv51hN611mdg)+~&&?f71& zha0Kpu^8az}zMq{0qRQP{ugxB7Tv_(}qpEYmF#mIxw8!)<%~A~yXuY@n+T>9pNDx)l zUAuq(34pPi>yPBx`nKNd&WaBjH73aJT;fqnMZ>$6TL8y_!-W%PVP*9Z4I5h!lB~FJ zENIC=k*2U?GjMKMdtr`Qa{Pu;f_(Y*xqj#Pzr6zMlien}!OVng*+^V^prtrBkpQpszNp*E;wPJJhFT?2GWw&#bf;U?*o=N$j zh|(RD8CRbC>y9=>80FWOop<`&0TY;~d~D*2t&zZV>M!7=o!C5g{``DMq?I|a&b{B$ z2@B$kswib>KzxBvq`u!1zyJHQE;_^87M{g!Df5o!=EI+UbXxp%Tcpck1u_1{P7nM5 z+ib_p$8btPb7g(r`6&ywt4Gv1EWQ2KSP7eR&+WhNzfZAFcXoGQ`p`P{^{(f8Zo{1+ zgp(}Jb(Hk1*~*f{*}-WKAGhhqlcvt6PoFAlX_e>YQ7--oO{-5!OU{F(2V1;1s{L~{Ym`=L;-W4tshW)V$sCWqjqWqa0?z3F+HcSY$6|e zxV9%fakYXCjK)Anr8(DcJ}iWPhNE6{>A3ivefO;ZNa5(;Q;chpdh6z7`#f`YN00jX z6*K0nP!i*Z`NUB?y*voJFutYoSxHnm1A-1+h>9US|bwp6KU zQIR^Sm5Er}uWm3sE#-0(-ntJvo&t?+@|iYWo|>i^>o(yi07`>E!l}48+MN)oo<4m# zuIbIE2?{zmz#I)!bKD)S?+Wr^rMxQ1cx zGBNiateYM8^BMDJaXX+n^1?S4P2=2xaXQ0!bVKC#uh0okQb0rr0@NL+kIU9qf7qr{ zU0rPp$AwLN>-O!}*qSeQoHuA!KI5Skn)D+frS%@SPL8c|?f35@l`56mzQAT&^Oc|H z(M=rfic>+TNvc!mFRN>D0FK#u-i8VxypxCwtYWWRd=IGuKnDt z^b(!7n1eCfRF!vXqVKuyT6gbU{VTf$fstjE@raQ|RetT0a~)U(oexwFMskaK))c$G zw|`Q!HN|MSxKVRn=4>1aq^rx9FIUDPd-qOKKtKSScrGkZK|w)A)yvGwKekRg$dWM` zDBbU}qPh9NjHSrs9j_MXn|me&Oa-I%hi6RIE%4TC@tAWl!sAA|r@C#zx!!v%othOl zYtL2s^ERZv@AmGn?=oI5sUEKs4Pzf4@PYwJ>OD@DZYkf-*K~q5e*Ec+7qioa3|*OLemr9W%RTlJBSqc?^=gK-`)R(?8_7^bn>nl-4Oxj}NxFXWWiRUCMyz*JW z9RO|qlOM)>v90a(o;Y#o@ZncmQ(^_8s&=3xEh$u)qwVCyl1v#EH1&pK<{gC|RNZgw zjxKM4Obzxq*1T}z-2T%i!Ztr1yBYf$8O(AMlX-Xo2=EFw9$h{?T3TJ}>rYxcocT*9 z#EkXrxDuC$Y4DGX5kI9_>MruffIOKN-caFjD%W=)p**3+-d&|G`eNFt6u;eDAANFa z>>LSj%f9J&oMDMa@?^f}or@hWOTXon&c1qaqM_GtnGN!^m8^?S(0+blvW455UGq!!mpmP4s0(Vhu!?H(&eR(g z$ukl*QeNPARPVG{o+#I-gq&_KQ#akE-_9PCh>CyWZs_0GJC!ft;*K}{iHJz%&eg$e z;t||0cO1vrk+qq8vK(}qN{1zAv)7Znr?^*2H%7jA;8#I}lF#L{xfa4?iMuaDb-}J; zTehrFzPz|jPUQ%mlK!InTq85PJ`HEj4XLgRyO%-pH8JEyZNn^?BZJGY8za+z*(Gfi zJ8o{1lwP5xo?xC{*>XcdP|C%4%Gr_?vyY!nTXQ<@bmID0D0ZH2GLE}na%!XWS0$O) zt&?inG^A%rXlbRKvoSrO+0gr_B&)MKv+`T?m)MyKgm|n%}!2HyB41P(53$L!4&2cb8AZ)CJXVHbDRHI zw>2}`>Uq(T@vGHaqb{?*w0mZZ9Gg0lZ^_!YhgUhv79ua;@9Ujzc2QNnI+TDCR<)mp081w_k7G>f6aPYTE0?o?V~Nub|Z3)|x{}+r;>n4y>$SsV1Smecsuo7S7?ot}WO)g+Z{&RE#NcJ#}hnz=*82 zR6oB>pTw$MJ-*itw{lCu;t5k( zyQfzu>5g%^HXEEEz3I`3W(x3v&#!WSbUJ=j(b7+4zc0jm-gMiq z^1irwlEUL;Gruuo3q_Y*el(EDS#ad}R0XL2D|LIaziXvRyy~R1wq9gZ2*v|)MarEk zBXjHOWSQ9Wq@uH={MT)@lnI=_vPbu`p#vyPL-t%v~lX zxcw&&E9Px9+xW41(}oi7&L*!c8<(!^lP|#`fGb&b zY4w(^Tal1#)tEZy*bq-}5!#pKVtV%f(DfeRRQLb?cnTFKR40y214%k0*+NL>$q3n- zkR7sR3nkfvjAUenvR9IMDk~# zHPMsJT+pl@08EV=M^f43Bo2p-jZ|3EV2=45g8jO&v%2FBqi6jO>^7V9ux#z0g$o0gr!C!>0B1l64f)Ufr50(%<=Uf*cuO8z zZ7MHIUSIQp-VAq#yibj5ZbBWI12YuF%oQ1F7Bo&Q^$jkzTct zP1mY~gfkz)b?2k|cApKsH%Dh%S@AVCd`4Id9VzB~3So~UUhi-O z@lgcLmDb&>QCy64;Na`q8!?*loxWLnr#qeL;*jFuqc8wbY4?N^9e*~^9L>t?H7){= z(V!sEYQS7?Wc-2#V^~Lx>+&Oh?4^T0UdI9F_N7)Y=ZMCr(b0TGT8{s0GXom3fcg2K zHj6{rKXV6-AKka-p;lv&C>Lt{`~`hlS^0U$1tHOwcdikE`Va7vc;7vcHw4re=gPRQev%H(Hm0Xq9xDpEM}s(PXw^jXkNH{7_1>Vkuw zb>kA~Xr9&l$~-}v0QaSro7@fc&VSi-Gg#!``*9eXa)rTO@ZLSk9MP9*1FR?s9{Dsa zPWy>vW1aGS8o>UH!0p_z(mOCPQR}<&y?pKX966(%vw9kS=B=kq&H6gaz`gi!JEFq! z?<@90EdFf4;fad^{H2kRDv1dnEQBm==9fd$K923Qb@wfXyo1PDi!!DIsAP9L|F=L~ zMk0d9BfSf(Uu9(ePPhjevN)CubHJFOdHViHC6tZm01k`h6Coi5dJ~=L{iETvA4T#t zm59eH3g}z6Z2x{C@q{Zd-4PP(d4%s((yu#hn3eRi1&i}0n_62Z-Oa1~G_bgP_)T;k zRwJ=t!yVukwimZ=+rlsY(cm>%H1@K;*$i1CHM$&iwfj$7~f4M(#gv{9c2ZNt5%J7A9Nx z;=w^tnGf0iyAXrNFg=dbYXUrR^8E{xVOMXcs%B)Gl$E&#-Oa21G;!%}Y(hd47KOqa zIa!mb;+cZ^dIyd7en^Zu2TnOV+`9}dG6qH5@xxBHkC|$XzLFhL&#cLG+CQ!P8bQK4 zw5A}Kt{r&Xf0cosg&y|GzlWOF_-j6*Ci6qW!z-Uw{{G*yK4FD-_KVG05Pq} z(x>$ThhRW|Hj4|(&dv%Z`P6~`%G8%h z|3rWH8it%UE8BH`d$W*rfi{zBw0OXG{;Fq5)z0AJ@*7}KcB6kU>jJZAWNsb=_aCkx z9B=ebqJN*-!x}jpU~xDfzZGHu+v`eUOq!C|Zr_y|-lpJlO%g7OuH;d5$Nz3TK2J2v z+DLo+_4P%TY*mY+JFBkR6$@ko_XNLJxtx~oG3TXu-fCROH2$)*W4r$7_ll+CfOJ0A zKjxL#a_6;LY-}QHatGspqTD)bJ+{Ht-=UwrKaqD3Fwp#vxrzo`fDn62<$0si)Hy(a z*D`6>`{GIV>*V{xSN-uDE~0H7%RwfdnNGxRbyj2gkC%G1-1vdx|AaU_lh*(~`}4&< zcp(JazXeUzWX|jz`R*Sz?(AZoEyWp+9A19qL}cquu$${QwAa_2Y!aC74a9$L@8G(| z-)4h-h08B^SKR@a;h@u_VtRS_Ok9G(t=*r9g44TUt&j4t93Yh&-+tQt9-`-EuzvgQ zo$#y=MbosuLV>!To$HHd`8oT7#gch{vuLSjGEI7{gO)YLtzR$x$m;+27Xf|{( zpiQyKi)I_P#pKkIT?IWqUhlNGcxFDPFiKYWcSJiQl7J6ip84wG=uS&>)ppfhlf)^V zAyTDxmw)kZIXN@XpW#oqvx%*syYg8y7i-{F*`dbwH0RvTpVJiTE(4G}i7f!b$S!sp zyH;{8sD&ivUXEK}`PrG?GslfVVM8VA*v^wkq_RmV^KiSw2L-YBw-RvLAe5-LUuoZ$Ksmi z0$JYNOE!{_kXC?At#^tNib9jil6U(!S>J8-E^p2lMy%xg+d&OmAir9=6C$G->Axv< zYk299lIod(PV9*j+IR-R=?h0|;q?Yy3IFVK0dV9Vjm1mC{Adaw;>KD>Gp+9i|L44G z)Qe+@`b5~;dTBE$fRPYDic;R|{5kZUjH~eB2*n^z3I&??9Jj{{M0220AAt9&!Nh;0LJ}L+Up`INb7M}MY6Wgk^?WLdZrFr<0uRjvCYI+ z(+UvYV>i{+{N1W<6sU;fp06%zxb>5Hvi69UhwK*bG}7f>dH?x8Uy>Byz0q_k|C16u zX?Vsv_nm@9G4-7njW~k)P<^3(Dfi8?eNwaSwB+FZZ$4|ZmBaWuC$Z|o+U8?bQEh}` zThBL_@^5iaB!fB)d?PyLPKhlo7bz#H%G`H&H+p`)EiY*Q@q-{y;i#%=%Tx#;$u|0H zr|!byTmE`iU8lTed>rj(a8gwjztChW_oi>8w(`;LW@Yu{e&wTL8b|TrW*+(Ft;6ZJ z{VYm?Gp704BTS0K(r`enfXJexU*z3`!8-2C85x-|iSf-?6py^%T=W5geDFoD5FEz@ zMr(o|iW^5dHtO7N(na9%wc<; zuGGwFCoB6lMv5;_+7#Qcc0^9XG)18HlUr1|#qn#={1&1l*zN2Hw+>TlS@wpNkzRta z(?>%TOTR}A+k$!_7buFNk^ty=bjxUf7p@+kVEjGi>Bd6f`xJLe z;m6IC2IV^Yy}AqOmWK=aWgk`CILUo8tu%EegV821S38Fr2Y zNdU0>9j|JAA{+NXTr}tSx`(`PV8Hmn!vLN9aD5s;b8RMO9fw>t=05EFybd^WR$SPk zi{%5WFv^yS{n$y5vaR&`rgx*5*0%TO(;}1{NGy3cBv=ip zjpR-ba2KHDH6w%VVOpuf!pH^(ku6;SUl^BsLV? zK|g~C3;{NVh_{F~6A-_c3Zwt6rBQnvoI0f~lgnt?3xK3%SP{IOV|QxG0H6l&qX|2+ zI44WHjGJgb6tR27Aov6j(h);Eo-1_K-kw8OUEg@(EGemMnwL+;DMIGpY&dmhoCN3? z=(^|Dj~ymA<$LwjqC2Z6HYtfYzAbT0_TQ@k1+o_*szFv!`3#S|b$LJj4u0XHL)ZYd^Enyoe^Yimc?q~PBrTC{IfWW z7{F#*@*?nQml=F>=;E|En}IU~K+~WghJo+5%e`MD%ngyyCoJv&q3T+hO#ANAhhrTp z)T`Uj`5YepEw@uLcAsHxeZ&4Iqvy#13f)kUe;~`;7rAd0aW5D^u(a+@iEe#iP7y=g zzox`q0a-d3>?Ebe&x+)=uIP<8s|5421oKO|1+?yL$j_fiuEV&a` zNZnH{2o=#7c=f#F&c*R8bWu!2SVs2wYrUs@r^)~2&f7Q-@lhF z>N|hzFtFaBkI?!fy8voVktGs=#E`AUC!M6V%H((QV4FejYj=D7NOMP&!+UzoHp&GB z+Gcu3SF+thrk$C1YrGj<>YPBi7Fj({xTCuo`aN+_^k?+&A!N&BWMF{HFdbuILMfG|%m*Ni^ciEn@dO z^+%lffM7q$m)G36)c3$)8s%T+apU6sB5uK!5>k7#Jv9gZhH6Q&=Sy)nnNhwbM{VGm zuUmf~@!aqD-2%cx=>Gr&JEW&U2`q!|R+R3KXz-G?^ir>y(`!e+qhN06!-#Hy#%T0S z6#5@~1Ff_fz7X-P@a{5d@u)xW*$Ui&P%08pvSL5aP* zQ?PO&jkA*`d)sWefQ`gPi?L_znA`7hCTPVv9Bscd>r>pQn9EI_VXw$&H8MFnQSr0D zrd9^Io2L11d;6b}-tZ1RyN*S6s$Q&sUxIvkMEdiN(W55=ayc{da2Llv8gmr#8p+XL zrybd5)}a`26*{70!}bvjFHvp zLz1eU*G`~`@SX6SS%R^`xL2YM=8ik_*97Qz1i!nLwd$8Q?;25iR@8mI85C-eN>9HU zNiEf|a`s?YK!kK3Tp~4dr4TH(-XZZEWE_}#BxRkOd&YLM%~9NJ=1gv2!M zhIlSrVWSL*$#D55aS9IH;Kd!}740G_!4CNvCXsE$e^2*T@ZK0-(~1xr@%L!Sp*^5X zc3UL?zFc}rrh=~?h!6@)qO3X3EVM>@gYs@tqEh&PyHsVL` z1;5-sy1HGtS`bjr4EMc^lkyk#9WHlEYJr#=dDCf>y63M5&^ZT&i*9?Fj72aV!y+In z&^-or79@g(vJdYP)adSg)dj^PqNv|(daw&N3Dv(Jw1NKu3_gdTg`Bi}k--|cKF1^I zP8=36w?ilvn0LRyi;-}Zp0Q(tOFq$};6@W~izW+oD->6hwuGvRmZ)BSR}<;sUA-?| zFQ0e3oS`{&g`y*b>DaW>dr%J-m6yu^FtEbVmK7P88VJ+|NKBzYE6aQIO@d1N!(&wa z%R8@a)@D^fC0=TkycPjFJAJf*6L=82a<^|1|4u_@k?X@NX1p{)ZZ_ffu8U4KwbAAb zR!LS4>0zhSr({yu4oi6x4sT35K^ztgST5Y zO-Gi|{tK?6v1uD{g097fu`w|>m9Nn~{J1|vwy`*9YI;9pNY5*8?s@C!&+TnoqSu?p zzwnes!8v{Cr4pNveIjUJ{_69PdBR=}sU9yJPIdB}8Q_jpQl2Y1uRsr~MI;=?(~aK|A6> zF@mCp!4u?jWl7SRgV4ApE4&e-c#J>^Y-dr2VX)Q72jnLp;L*69BwD_9!+krEtW7Nv zlwfp{=u2lPDXGfKTw}|(BtK1%WSFOZ+L9#LZL8k84v6tqWM&@d0v8k3u>W&_o?#+< z`IJaO=tQ#2ro4A`8 zf9TdBBgMoz?5phEGS>wGxbqHjxQaZs8tf4;$gW%t(a^1X{vzKecwwCQzYhm6q0z@V zR!W~uFlC(3fy~aRknlTbpSyog?geu+SGYLQr-GgAY#;O4=W85rK*q3*_x6g8uQ!Qo zyd$4p_1Dsl6;oD)VQLDXD^LbfQc|j`t0NkaA2!Yw0X~+zvIk0zPBV&yhVXIDaGf!g(v1(F^H6P zw6{~{s-5=w(OJ7fX2vtufD6^125mbf`~aZ7t-$L!s~Z{`Ccc07ppbDseMzv;)9N@q+|_(GD)^s0?keRdB^`&|DCzS+x?H)K9MqqLTO&mzKI(aTz zb&&!z03t0=+Ao7jbQ!M7P~xEMaGn0Y504`GHU%Ln01zBvT3T|AVfDw~5maQPX0!X}rQK(L`6|W7w&j$(a*MGQk9;GyBTB;1Fkh@lu%^?IN3xOo1c#)w zaDTh`!0FM}S91dCcY>gVLPHPic5i9h} zHQbpK@?A>K#e0!EB>L!64FvjVipDO(fk*QvpO15)(JCsE)WZI$89V3xy`qMYPaEz~ z8%lhEASvY+>dov~I3<0uA}{g^x@;fDQs)(_GQe{(GELdWNvHkA=p>!(IYVS+IXE~Y z+9t%jcOLeb@y_vu`DSMuaZ@T^z4~EmUGT$tKP?Amt`Zv>^t|PMM3hu7diw_TVea47 zKViyuKKN{Lnl&EGH~6deAlc=*CM=(>#T?m^n$4{d>s$y6%5H5Id|LkhRy0*89PWIz zo!7WCyv=C+^HUuyzF$gcXmS6;tzj`IEBR0fjZJk!*`A5STSZ1 z5?V9(jcG7(-YJ;g3i;a4hBK7eX9I};u0`<=y+R#nJqv}^?-bK?-~~}^id5t&bD(3A zV#k)F?juftDAH)Q@s%`tX6ck&wqQ22EXV`3dJM62u$cg+;|sH2Wh2SZzmu2%E$a(t zmN;4uGwHG^vA>8-NXqE5?6lyu)ehhb?s9k{#|x@c1B2AtECBL zwiFe~ewbMS&_p}{ZC0)lv>oKV4;+-xox$uHt!|UvAC?9nB#0Zu((1Cyq&L0gup>c6 zL!X^aM?#YB6P!K!@rT%wsxq2L9uL|%P{bQ*P{PHxsRtrMaE=X>f@vI4wej-@Sq*UC z3q#gG`(=0!PNlu`A3CsRqW13cU0iD%)Rm#RXFk%LMt`QZlY80~JRRq5xrZgO@gb8p z4XLj^QZdAA%}2uBP+*s{>FI#OojD>=!d1e-UnJKRmuk`R^cqP_+>?xHY3-v7?!>gj z66?f*mJh#C6YlZcy0MK(TWHQ|@XEue$NA-&7C6NF;PeEp%d~JjyT*;l<|HEhw}W)l z#YXUp8|FM@09(ThK-aAPLb5#3cYnTBVcr6fY(>k6O5@0km<(5-nUD5*YWssqgDv@>%*JaXyLimR~ zzT2Lwu=<}yQamspbdHbtfD=P$r#zNT^E-U?D)91VbSWNW z+3}?p(AhD2L`OstRw_(s)`>p7!ltKZk>=%1MdaM)CByza=Wy`Nq<>A*hnICsHPmD? z#RGhK5)_h$rYk@sXkcrm(Tq{kV)Lhlw*S@BN!uC`Mfnsggpe26m{j7%ZaxX}fK6?DL znzRflZz0K}dRwkL)#OWWi8rdab3VGDecPNAByz_g$j0^8BR79~1Plv|mbrBkv%XYE zkgZv7^U#bxesz}yHU~X@VFR;1U(E^O?R~Eb`;XQSe51M@DM_Ql zDyXx}!WZf;8B*!x*w>ov+^*2UNb`AzoF-(uot)8~KI1f9;awFCbB}rzHwiicS+avl9Hn98rDb@PQ*(e76&X=_5z?3-XSHKQMF{I)4lVI_GSp2TAH| z(C6V;Rqg3S(aVG1ouBhxHzFRmA_F?+1aXjUsmMvNCKtS>TrtZI9@|FIQQ-lYdpQ0r{={6Kb z)I#7qyaD=9#FD2j_-c6J^k%Ixc_(xro0cFI5rv${t*&s72GFH{wKxMkU~DXu>4ddk zNwi`Vz+C`h*}RmMUTGetuzUNMgqsBy(s3Li)(Hu#QZy_RYt*@cbCjF7x#XD>dS>?$ zh3G6h1*0E~Bj|qzt1mX;yUJLG(B2zBHdx=#nY~ntS;Cg(F-iKF(hYRyk zH1nk>4$SGFi0gGFmj8-X(S`~0JMEcW!tdqoA=SGfqK2Li!ULys0{%e1NhvC9Z;S1o z_9590$N3eRtCX^{_2XFj+!zLVy28Sa`lTpV_oVglz3Q18ElUe}roeFZHp~37d$LZx zY*E)s(di%M3^;TuTblJ`hQ0xYxFV17J_SW^b_+w?fRmFGA`#7kh=nwBnu@#> zw%MtqU77zL8&e2|kaUB9#}R}ehOt@M;uuLn{nhf2G*!u3?h}9y-sWmHdhNwXJb!&) z{3@kis63~k`$WezlmxW&i02)y!vL+i;lTP`5wax%z)yRCHJF1-P(q%xozP6A#Bulu z1EeKsryU(Y&bW*)7;9^e(9>+C>uz6wh>N)t;k$IrDPJvM%fOEhl!CP6Y-GU&JtmPz zOyJnD>vWxKvpoKD>aGO$uK2iee@P3~R^Vi;&I`M&-0kx1=>8!It1jDn&vV%M(*oP@ z3=;gMiJ7`g!G^?DviO;oZz#XY1r&Z^$0`?c7uKX zo!6r~1im^CK75U*&^@Nz(_&Ki{I)T-6UzwV)H9LotUO1cR&^)_P%4F!+nX{5g<5ya=j599@bz4I*Eh{76Oe zr%>U?RlpPQ_f53>(?wB?qFV^6`TUsxfDm2K)T8O2O1gNyJo)u~0BR>QKUvQAyngrS z`3isb#NkG*#~JLpeU|6b`tFEBvJWC1CGm6M9UdPs=e7@~NR~nj6X9cgh&3Gis~)Rq zj0JGZ@VkKv!LQyzP|1X>!mK3jCIw+x9%u%Vv!_K|hVg4%q52?k!ZdZw>!+Re?HI_> z*T(?1aNrbPa@VbB{AFfRjzuBw_ul*Fp?z?vD{)36c?jU=0*!J;vdd%VxFt_jF*Fuc z!81tD!rl6~YjZdnHp#g7ZkC|fXEWkA{;=MHYXE9>cia>liz3r&o7YPVBiaJ`4tEs^P zdk=a|7&2mFW7AbQK|#xfk_fvOlnLH_Fb6|0k_Nft-}l-;dK1ng%@ch`b317vt%hA@bM zIS}rwSsh8INcNv{fv5}M)&Q8~g&aC&(e18qptmydGZd0f>5fr}yIlZ)enkk(X5@?_ zMvorFn-pz*-kGTgvl;*#0bDs?i-hSJ7!AmFJ3%WH{>;GjI92HReChliLwmzWB#7vH z`JP3s1t_Ucr^)kyjX7XiXB&)_V=Ui>p zh^|YqLVr1v#{RSjmP9DP_A!-fS&GCj^*}M|fsEy8uBOsh9ck}o1_|a+!|zzV5y4G; zuigI$u9wbsJNmk%)O`7jio`2FZ#@Iw0oEP{1rQ_z!lcj==Z$e!m)%O&sx7suV`~C( zp(GA!?XRw*qGRiA_khr)Gj+Cn76OCzcKwb~p$0gsQZ*Z?$$$7v5?fwQ#KMv-J%~!`7<)Q8|5*Pc#Qilh?$$DPT9h18 zhe15~_ljpb#6xztI{#<$@*`GgG)P>J5)69R)4`x;SL&C&@78Eqk@2*n7ZcYsDdr1E zVbxgo^yiV!|zP77(e&zq4MB9{w$hlueONz_zfR~46DVDX4By(kppY!P z``PqAs-c9tFq?Nw5vZ$c>aYJ00F{l@7)8g54P5{&qvx$RIMauZ;$`pVy?y((;iluM zjupO~1w?P%KPZ?Z>Q>X<&M7vMJ|zs~;qY8RNqN}f7Ghagi-HU7D#UZ0YUTR;fgpL?PzfPa_UxnhI?ByUUFs*g3O?`wr>VQMkgD^rc5KSOX$L+N_ zb3WBjXv4?&5Q+y$fl0iGk*YR*@L`EKziOM^Ph3nS`AJY}AUtu_WVbp@@^1Qj6oE)2l{%dfDM!RcFMkxxaT!V33TKc5YQb8#q2V?#&$gox_fJ znM`Me?!Rd84qs$>(drwQ%q=c>zIjId@maM-b~d-5VC(WS z_mV6$G0F^j~FzUh0uviWIv`07>JHH37(qkR%&kh!%jZMCdN86~v7l5BgN+i1r2x3^CxTNWR1P!0vF08G=xqpWEA$)*_NgAx{Y@3WzI_ z;0T@vbWM{FcAr!;OFqm5-7^HdK)4#v^*#BzQwUXG?ycRk-GOm^FkjD$b z0}qGYqmjFFmLIGfN~98-DofG%};F7r-NcMS^&_7UvHi1ySUSEqo|K5?5w~b zXVN0xy5#L;+QH7Cv}3~?7&3bQJm0AbHWCoa-8S|Ci#SWRvU8qn{%{)Jx_fS8#*y)D zW%<%{RXqV0)S!<>D@Io-+1R1^L_!(~Nq=ZW7cckghhvy@t?sjYH(+_xBIOpF9CmzLcgO*X(}EDaogVC!lP< z9$);nIn1D{io6oq;Z0(&<2Mhr!U_fv(}EMC+);aj%kKeJ-{jFFLH7r%?u^#_qh%S-{|53uKt2y&={? z+ZBe?I*a;9OKyj@?fy0sz7Tdi)r)EQc$yyC0%yhTdQ!UthxWx0h{CP9xRW&nc|?W- z@&a^1vr-hNZ=MuGpK8+I?#)h3eM9VTk$O?U10Csb@sBUo{??p|{I@H#8a#>Bk5AfR79sG|~Mr(Wb&2hw|)f*pbloO$<+7gwbKk zE$I)@9Pjt;-*Bt;)DNMDiIe)=5@-eBJ1;%fZ#iRi_Z zyuuG39#6UQ%myGm&iYT@l09Ka7lMbIeTO|NHZIA;+8Q#kL$l4{qmR99|2f2abY};o z2EM++Oik@3>Y6$xw&t;xCOo(%)7VwH4>2(QC6%{B_wir7_NYB#Z&Exhfo_Dk!&xN( z)6A8}*g3!2Ct!xXcbNlKnV^MvBgX!QCg%by?kdVnWQPQI`HmCbr06IsNH(yt8z6H;@Wbg?c)R?!= ztHF1mEent+EiHv0;RB%%YzRwBOHii4dVuByI7%$)z-4rmjttDG&PflyDawLiLmBM{ zYt zIsq`<;|Q}2oNK0F?}w0^1CYB!6LyOM#^24!B`C`bV?liHib1x#&&LbfpHokeaalXX6aoKLOqs2zT zAk#7BU~i;zZ%$h*%US$dAm=-ed-(WDt1I(dou8C1mXAqYoWLOv*82D&$KC~=lvE4tfAIEeI3LRu*KvoQhG^NtS-Vd5U z($@Qji*~>F-Qg=it`ieVVw6uxRTbt}P%?sEic?3+`2PLKl9Fq1ejKm=HO2S}0x`&1 zJ~x7)7CHj8yLTDm>8~AEb$qW4tT#B#K|)>F$C2!|d>nEsAgYqWpBca_h-nON1Xbd` zy1H0G<+!~9_ZzJeP2oA_pee5m0P=!-`0G0tM;;5OI-|Yh-cvRR&8kqa>ieYqJhYT% zO`Mj`WzE-ceB1e{Z=U9KcF=26plvEF(%6ah>7INGi)<%geNTG6Df-f5TqtcB>d4XS zZs;#OD!vI%)66JmD%>mfv6UTZKM6m*4sULmSXx3d)&4*mR5&|p1-v5|MdEk^GAz_f z^%B18{iNp56Dxc4mko=6{)a8(t-jP7tYw##J@E<-_CaJ6sX|A?+Z;% zvI@kzDn^Y=z zVqbo>R1)NaLjcWm>*pE$GCwbrg)MUN}(YIM!bdEvUxM*HC~NbAyrd?!fnMouyW zIQwb=A_0(k9oXAr{E45PZHK&|JYCNy2v-jPC1Z6hBEfwC~Sb92e12UwKrh%PzNt&bxF&P#j>N;DKCuYupH;u zoEuGxVS?L-rDi#Xb1v=46Yc_i?JdpxTbP#hCdRqqsi=!=bxMl$3Ckk3#;dP^2EdW58bLEIb5|jP5e@ZGsYH9`x9`TY>`E$)u=}>9j6T?S^6qH z_wq2iHCB(p!Pb|cRINto+G=mjw-t3)28jgK+_ z9$&D4DzdkK{>hkAyP^T4Jdmd=RQ52uq-5axy?bfHi-s8D$Vg?7-TFadXl)}YBL0J-TJG5Zsh>%|=E1FJ5qhF6|Pc)%gBhC@v#hfW4QO6nAsv zC}C9x{1KH3fNU)ZM?1Zj+4~i-wI#Xix2sX@Fv<^l5NHAnK;!de0J%J1?|x=L4{Pz$n{;h7QUk%#dk1`S`iDU+$upg&2WZ zdJmz4lrTv^T)i)nAq=Y$39X85MGP!kLZD?aD7Xc%KytKia)&S2^^hP_1a918F&w$ z!IKWv)bENWTKoeagmQ&^x~rx(eGn80M=fNMLP9C@V^CKNU0vhgb3qcw>j4tPO?a!Q zZyeH_eSMF>pA16D;SGE$6061;o%Sc~>mX7Ma!^FRIM)suPDtHf zE&8s9t=Vut1Tk?i8i2f@0C0^0^_x9w(WCB~swj@jinR&pme(YCVP*yKQOFZDd6Ezg(1%G(0O9K+De-eA~UO15E9X++JgYhZWu@5gr0O;(KQr ze5UvGmV;8N=w*|zpn1Z*8hr{^hwlUmBlN=l%lw@G5YC2HL!RT&Q3?sK=7|0E4KXY_ z)5?enND!A*h>=tg8x4K_?eKT*rCWRrr1J5vmC2F{=4-AK>6caT-_ zf7D_KobH$Hm2%%`0cWc@yo70BNUV~ekV4y0e_(pqHxt>Ri7w*Jc%254;T;|#&EV9R z=IrI6lIbz&gB{ABjeqS%xV{g_W^>lRHB`iPg1)sAyV_;Q_Nha(-jW-itB--Z6CGYm zg2)CMh;qj{N+eF#((-L}bu#4%fv`xOMW^?$pWr6ZFwF)vB+L>32_G>azFZufpv?dXg`yHmpF{`nn|gHTY=YgTaj)AJOew& z%H%l5KENoj3*b_&fPjFU{Cqq>%McrN|9%MEErv+O5Q#a7U!CFAj{*9S`(gkRQJ~@g zr>NQ97J#D$xS++In>e0X3kUQ!O;8(}o5w=arly9Z$E9d-imvQ&Z4C@V%nd!a&g%e;+5NgjC{Ph^*@6YB#{2HOyuZAgC?gU$y9)~=3 zte|Tv*ZQemqX%CDZrcrqL5>vwdq7@+E`hhmg9W`5xPKO$${%fTA~~Ec&_;CeJh?vX z771~}%Uiprj}QaKz()qk=G}Qy(J^0gk(UDt0F`%uz^xXb7Q`%uFo18qj!w44BZ7X| z!OJE2?oUAXg}{xKI*PhukT?P0o)`s0PJ%w`c6uFr1_Z0DT7(F%Kl17AZP)E0WQM}e zK#UVOuaTFjRaOHdB}_j^d@M+a^5&A+_3V;f2m7a(fL7@Fg5-y+@a*Z;=K>17jP4$1 zS;Ed4NQsT{F(g(iw*Lsv6P>lgu0C?8WYNx-xfs)=WThgRZYh6Csx~X~Mo#CFW|_AH zruk-pIJ#LnJcZ8v$;h-TL^W&;op2M-N%pN}{`#3vL)bU{-t*kE*6sBnhfyUmJ@g}w zMR1@fDrjmJCvARz9+Y5tvuvM2_KN7}Jl8NLWe1_q|MLHqe2~CCMR2m0n%}uRfkRK9@Za=ksIl!-@q2xTp?PKI zg!b%$2tMkp#y|u2lP>RC#VtONX~(iRYWdIHej3%s9!52?O#sf0JgSHSInX|)#-v5G z!oTvUQn$r#6P@`1MFgUINgJ=g8-eZ-PHgj!Oc#and*&clD;o#-_U{X))x~ZH%XUa@E-e){LB>F#z@Obzjfl zf{5zwywi~Bx+kLH030LG*))&@-=r)K4Wj`JFbJ{@)f>r$gDR@_rT{ph_I5I8Xt5}W%9g}+D{$q_ zAz{*h5tPA+1~;4-LL4vr)oe1Qs36h~6$WmE1p~!`QwZRhV1g^KqIF&#{$eDx3o!e_ z0GLWN0ZX@qX$&?!H5q_%2qqqvfrAnjA1^Cm46#Tl)%f7)$U`8<`b#<^4Z8JkRa;XN zF^q!dcZr$Yh5sZ7A~o&6VK4zAUHJT1E-qUR9xeSvd9fa&^xTt~g5b$k%eIWcyhW!kNgJ`F~ zTRvwH@gikkZ!d(ro|HTZ_1HP78QvN)MIz79(D2-X9#{_nvV~CquB?c1>OSVfv?sTo z8X%-9DT$NkxS-e-caB9|S&pOf*6@Q-A~G>LibiIAx>Noq_32JFp0uT7aQ_)h?TE#k zSrq&ck=K*D@P$o7ImW_22gmf_xsn0VnG>pAZxbb4b*My1j->`-$(GV=(8PgfKd>@~m+R57&|&*=1u1J$sgk zhMc+~JTLq(u|TPH%A0=D|Jm6J*9A88FK82fG7fG$Rcd~Oq$2NWTQArlA$fj$WsCKv zDBOL$-F2RfI6FIx#f#+d_8w=YbwI6NgaT6v8wn)EXmj3cBhH&T#puW?C-6`c2_9g} zo`-{;qt(FNd?qL0KWb9loF6^=a6CEvGU9PcW=+DW8C~7bIoZo%%8Ow zA>B75+arQ%0jpm2q}s-5N|ZJS8%&g_o1XzHaLg?^8(-kXpkwF-qOS* z3>{ejjKM>mzuosouk8E$A(L&isp_%NmO#2TT=W2k07w#Gy~3>!Kr%wzRoC?lT-JFu4Nk568))M~|RAumWOYtwCek z-SZ2>K9HAgzjubGI`-!~%t|yL>j#?00Y3w$QmNHgg|%j96`VN`u?`uwM$ipQrFo~a zK}aF0{+4J5KvRGtLcAr(SBo|EqrOWUH6BPz#LoiGqj4GO1mUbQ;ux8!K)f}%52$JI zUfRo5&i&i07YK-Y93dg`M4ZsflEP4rBf_TQIdL*@hwP!09nPD#_q6K1wViHqYpH!a$sF)`@}}R4(6-%!@H-Un^Ke#ic&w zTK;UtLi9$w^)=psrtGONZCP1L$Z0-8+d!&K)Go8a^vQ-FY)upwK?lg@oFZLs{zH;k z<6J;)A0|62ZyE}*#yN==_D-jr;twLi4?bdl+17hu$?%-VWlSIQDR?xo$EE7eu}}}* zjOwHKCw%>xob%+$B!U5Xb5Lj@HW+T%Wsat8a~N81>Vp^?D6jynf-P13 z-o0~>o;T8=2+@*Iv01o4ODY=F~Y&?!)FDCBTMvXPNH zN1(QoGYILy4ZM;#kjO&~ghn>Ds^<{G`Sjn<2hJSQ>2Cf=OWl2yB{)N}l*YKn<}O;y z5dsqyWFCH;TIrZiSSeNz%k;qjkTp=|yGzoWoAa^VwPi64EGzB4GBOuomv_Q^rXldZ-WSE0~2s!-W7l2d_x>M?_G?DDa*jH1iO4H-pO zm(Ct##bAHEU)|*u&X+|wgVxh3_051;uwS0$c-yHLFKWLPVX}krFukrxOv}H-8Luq= zIvb9q*=y&6Gr-k3C4g!Rr_Be1a|7tka6b zBuRf{#Gk-}4q^+tdh_)XoUR^HQo?nrCO(T5ixemNw->JjEwyk+HCM*X|D1nYwePh< z{u6J(8#LCnR9#CErf=OPt3<$o7`*!D!^H4t2UUr$QsG;Yc*jIH zk#O@kSW<}Nl&)FERKo+5PzYEEmrTK}RK-l3vGSWn0luJ3{AvNv86-&etnpwXK5v35 z7LmDK1F?Wevf`2BGSR+nB5<7uEIjimKf}QjVd5uRI{qJ1-vNzvANF5Dgc35cw`7lO zvNt6oJ9`V+*()nEJ6np3%xobmS;@*?p*t&N^S^#Q@B5zrIZx;5B)PeN-|Kr_pLLn~ zc>OuEQR9%7}3(zCz%pE|T0p%TVb>50CPx)@!=mf!y znV(7qqI1dZ*caD{7Qxf2csS z0%1nwz8kPNSSr6bE_XIWdWB#6!U)6mFL!bf(AVS^PUfwJ zm?ebdzTvlh1&BB?&~!l4)O51h=yNhP_Y)SOKq3vKN#Fhi+IT3_5c5dAji?JwkFqa> zAjArI+!R3ST^8Cyt55JCiiQEih|u`NM(%ab8p&@g`$9oKnjG{?#^hcU1S;DZ5CH%Z z0QOw1jEdQUs4Ov1phJ2`uQ@_O!kSFv7qa;VHj-`q6Z}BYBATil!8ajbRSXI&Z~Aj> z4rDvXc4}b9)q*+U202}aFy6U~^Mj+c{8(sR5P+w)HaSehcrTz$5aX6~)sQCTqyBgS zW_c50IvS=(dE=#gl%rq)YD|b?^8xDVVkqcG2!K#J{{G@>fLh7Bugl3SX@nDc#99w~ z*K}3-G35f&hLEd57LS_`gtQj5>{rJF*aBv?yhi)8YMwUO{@qTo$ig61XFt36{o3w0WbBh(a zheWU{1?UvfrdBy{_n4j0CXU(=8B)4Qij`D+Z0u`0pM9rn%Ow4Vr)~O_;@NH#bcLH6+m427Iz_Qt6}3Ah(1ohGn&q$b}ej1+$v)v99sDWo~{V zb0hlnnM-_5g7-fs@THOOy^k?kSaq zW|Iqgc6g}ZhJxGmC){(0o)H>sXjOq|1oRlB|1iuse#{_lXI_Br_~J1>jhASE3RB1V zxu5Xe7KbVGs>$j;$#1<7QaNbOGf^v+E=i3sGc;0mby9`6ukyk7;Won#u9#b&V@W!h zuLdLrw?U}1WdI-eAQ|)pum=PBQn2s-3uuEe zK#>#Xco?wq)4Zyyd$eI|S2#&pp$(X`05`2HG5g5qzm!Omx#|Vk*a+$dTfU$d&?x$h z&|QcpbLwepKVF3 z{_?l%ynMQSj4FS2^v42t*^xZ|xdU!oddjWri=*tSmKI2ng4BX3_*p}@H4>1U$Fly+ z8_F)oTMDK&0Mx7Z+@X8gdUntFD^LLQ0C)8>GA>db5S8!t3mm~p>H(0q zbOOZ;6##x5_~-!RBROsC$fW-dbc3eqohB>dD>OcSW4yROJ~#jE4Fo`d1r=6vCMhwV z32kj`4F8K3hmury`9;dhps>Ps@6xZQiXM+fQJ=I@dOg;|1f$nz6j6yH7#_36ElKyB zucQ<{qK|$;DyUha7&(S6)ErPcMVcNFc`Y(1C;{_QLN`txmhnMv8b@swZTV_Xn%DKK zs<+i%#Eie)x>2;B;b|f>=$6{E_ilobry-&=?QweG8}fH7KF71A_Qq56v~bH9E#t zHeMiGwz(E6PnfXQpjNK|6*Y=)&MSjvZ)6|`z~z}{!-h4Vrw?P~*nHY6RLd-Axqz-zYfYC8;W&`R#-h;n! zb#a>_(pw_9%T;T9XKYGq#!zc;*O$w!l)X5tg-<~PWmilROH&1i1>UevQt3N79c`@fg=|{P{P$OXj+pq^x-_ii;_8O}`WC zCOaE`G-FwtO%8u|tol(XN2DJP3T(w0wuWfyWQzqsm@t}mMfEoS95oPP=Q;Pijg`45 zE_Od=A?T6&8Kuc8`){6sn~#)~bWIexi*?s@8MiQD zH7}7;#NCx36O!=`d;!o~T3WK7tbEhGouKq)qQUJh5P?l7!!6UkWcI!NeUtRk;-lfM zocQ>ga`N(I6chzu;e`Rnz`#eJ6gTKo2M*=*CnntBtZ|OWovE(zzsBa6af}#bWRDbc zQ*S%-OOH{QH~58_o}oRKO*aZ|(C^;4!ZufEsneT&i$^Z2Ai4d4*}H^Dw)Z^=P4-8n zCaMpIiSVTFg%>du5=*D3GAMT9p*7JI{JQtSH#W@dk#o8xMbs-Q=AzE<_n3vn`3eq% zEQWWC6j&0X>H6M%8+}uT4R7Xs8hk9|f7{xf2S`P&s^V>OUqo*`>{J)x-Di6Ctm@U> zTcX{P2{cr(gkhz221!aqncIfuj_%j{Dx3_i>LzeSYkzy@8k4C~NJd84v2S2IYKuyr z4#ff3NObUxUg?na_jrahuISD6`>bgUX#*b|rtbZnb{A9_kIePIohGnF1Za5lKn=l)}< zAo<<9;exU1cndC9Jn4}i7h12#odO>sOJ3;X751V5V+RG!f4U%8N)+>eo98zCLGh^V zTMhsG;haf(_u^fT8&kEuvKvvj_;AwTAg~dmQZZCUeuw`~A;+cLy{~D*#KR+F+PzlK zph#&VF=Q6A#&nyS@uq9{raL)q-nhdQ*v8#u()->E7)L6k7X5RykjkyzOha#Yc?DWf z1CDVr9TuYh*7Q+I%>8Cn*TVV>Wt?5`)dIIF{M_Hj*Ap81~ljAJNfWW?& z0UUmQX`&zz`encxS9mSkMO2)6-M7`BhC87})xHe59N0S6eyhc>>i4G*OS)QxAFF(1ql11Hepj_2U>R0RfMMUD72$ag+IKAr$%)MMKDDr};s%jp06 z`)MT`yh2|f3${*nR}uF+e2+?RT>8jyn?B_(a6EMWx!umup~k|_ORCVr(o-44c1MAQ zR0-EGBLCwE4uKxA1jYRX&4LGjK2LvoyD(ZIO49eSs1eWB@nz&A+T1r>uLO%r@_N{> z1h5scCw?f%`@&jiOQ54lV2dU_3_$iH{)Y^BaX|Mh>MRqkQ zUj0J!&ihGP=CYEKa!D3Mx{^smxU!v^`~<}@V}6tzb42X;1{{5Z4xh!D|L-xr7p2X! zOkg$e`sZPVj_3MnYmT6Va+ML#nNY`fG$ZTCI-ZHyH@eLg&OSDGcFqb7WxS-R$Ve46 zGM=(!Ao7YK0^SspC-^g5*5AqB70c1Z!*;FoH{a;?_3*l#19RNAlz0`G2nwE&b%n`K zHhWNioN>N7RlCKI*uPJ-_kS!Yj^Dm`pJ~PW%G6FT(^A1AQ*Y`p0d}uW)0g*Me5z0d z>kY~t4YxM7w^-=D7Iuq7Rs=$~2X~yZvU29NE_nBVcg$gAGI*&_Fef&BMy(ft_EPQ& zJVknXQV9hyq*%$aKNSOKgCC6=3u>r+n;7WlNbG{31aC4M&L#_d=$IE|r-*OkQbhi~}FQgv~f z7GTRj&c^84gLVY5n!#{U%C*Y=R-*`csX(qJDFMVQ^z7$XI6Zma({poaYaM5ljE(2g zEyH@8NXYsVdKntZmb|D&^&rwyHY_Ym;jRA8W*N%;hi0iZ%>OVH1kr#mppbV)4=NpO zBNHja>EMDde3t-sEDu8DkN0eh{EjSQs3ivA3syXiURW^s`+KtEy!UJ^>^zyD+sOCo1zj7wS-Zf!o9=*4I)44>S~#) z$?4x^b!8KNnX)K0tGk>ddtX9i6zK>|^sTAs=xOu6V5ZyW^k7C7!~@I}?Z+c`Sywq< z5wo~JBm5-|?Gnai)xq|^qN4T&a~(LW-)As8yC@P0XcSqm-7e09Pi`>Hxyux0O7O_c z{3Y(uV+px-8Cl6h%>+F(#@r_?nB4qb@0G2bf*4caC;at|&>RTJXtg<zg=C)?Rd{8?Kil ziS6cg&Af6j-PhB@QXWISx<78T^7VU$iaJDlv!S7BYSCw^!DG{NnCvjU6Qjsf>tJSW z9pWksHh-Pgw*R>R1@5$-O|{POp2AjTkE#p?{H9U}Y8mwVTSXz{f{x62iR$8tOo@}f z3$f9IRDYJfZ@G!(NT=8C7LYul@A&?N@qR+NLGh9&)k=EjZ>fm1&TCas7An%xI$Gbb z6E(l)#3|oIr_DbT`W8W&UP$+}lGNbgD>aPSm4D``)WY7OA$(Arf-1E}Z^O%oHHif# z5mlT%N5lV%-H5L@hUtr6QAS|pFNd;( z!iKFF$^RP0Uh@RF#zd*MMB!nExe)H3n`~W6I|Dv>$1maDFB$wJlfDwcf7682bjp2I zL-;e#-n z^bix!WBt~w+`8AHn>-rh{8Y9_-WS?6VXeLKcDI>Y?b>o=l?Ay9w<}9% zb2VyHSf--((r&p_(DHaoV2iyh`>e1obiCBT*MylxZK^Me7QkiA%;>q{cE^D*Ad>Dq zOJyLd-Mh*d4%DlXYf;)sjPIqylj-xBxqti&C`pq_U?47G#T)Fz`Gn}2Kkxc{V*Y%^ z{Jvy7h32GWl9uMAlop?>EMwuAA_}iKrvJXE>}+%*w~7f5zJWmAV3K831WK4C^2gC@ z!hgLN(gOsMcXraLi|>2C#331Q{`s2>5|NxK{>3Zf*k`6bF`c?D*WI;->RS@z=sXhQ zcrDgZB%toRFii%fM_;z;XQY9kM>5Xp06}ag*ObkVVK*vW?Hi-w=@gk#@*j|z`}pAa9*GudUQ-D1Le7BU@^?`%W+GSoY8q zH5$5%k9fo+a>hQS8WuZQx(BYCMhpj11RvoZsU#HG63g?~n_ ziST=u)RxXnp+6y&d+R+lKTiWcZrmF;5>=SS?ekZ?{%hMhcb2+^{7wbohJe{)v*pBb zb|=Pzi-*U>aPE0qsjNZ(b)8^K=nDUjJpTxMtcTW9AvPN|*K%kSd1xM0^u5|({N;!z z;)Es5^_{emA^$FuP!)Awm35tJatY&Ua%YYmB~M7H#Z{94>W29j3JFmRx(Y0>N^W~s zeRcKCiufvM)+>4Z)n26e(|x(T!LJW$GahHXAbA{OGNfi2_v1o8Th{23AL;xdM*>mx zcs`A?s>&P1PIqfhaY-z0E$KvyWi5hA^dLLgW^K@K=8(&Y9LdQgkEAf33-? ziXZlsWK~eiEz*-G6g2^H3;z-dxpiJIogP);UdJ0r+1k?6zq?_erS)wcx51H1P%zv= ze*%hZ9e$WXWpplTEliQk(zI(avz>6UnWUr>Q0gw^GAPpgMdlB9@2)(^31N#|zhfRU zOO;(=R(n0)XMFyrVMO#``q}#!*pzZozXe;ztE^&K?!#kp=@78zvE8(t?T#B&Pv=_j z`awS12TtLO6sv4s_t9|xzovJopTG*Ut5lu=NS58=GX)w?92#6qv^KYSJwsF?BboGg zIJs~!+ekHKdTrDy##*rwM<&fp-XX}5axk>kb~Js~)_EQ;JbI84^_NpS*2YjJmQ_&a zy;gd^(X$x)sp2mK~jJKgtVhisy%{(_c zH=cfll(oL@T~UzOftbm=d6pX+KwP?bnvDJgYx$fG*?c|JPjlr5t!CZVySlpWxjth) zR*M#&!U;taasT~TC))pwOkvd;gE$WXo4}|WIX2zMR5H6hDJ3OiN}pTHAV-a%2`ONN z@U~lY$58Ga9Fzbpf zb-Bm^U>)pB)a@*jpW5(5PiYF0@N*Ks^TL23s}XK@hewWDQ7o#EG^9oO{@zr@)TR4< zpYYKRLnNO!9}0T%!KjVO>BS`Ai7XkmVoj~Q94RxV5YPWh(;%u4RYOQnFhIX1x~2D! zwEAOt4aXGDjx?E~#siC_kix`aJk#PxIzRerWrR@nO`Wf+j%@qhh*MxnTx|MRb&;{W zmPskHJ1f}+vPC-9M;-~*zI4#nEJj=?FXenk>9i`@a@U0;iwV>jejH??q0K!d z$1RA-g?)+I8ucufXdZ)pU5B6K62jZ~95l|Y9KgD(dyQ#Gzf{-jxmmv|EVU^8;`+{~ zD>s_n6h}a^iRwn$8twYneU+Y&pBt#ZyQ{SQSCSML zs>Iitrir)3*VaQ6X;TU{KC&biLrp+=@v&d+-VaX`u4JQCCa84X#p%*{zt?*Ep9nh4 z1V6SLG_RD>_dm}2A6+<(+&9O>>iA%$@_1|5FjVOF^^Ol!zc<$#IPv=`r@8TE+Q0PR zLsf4WQ=)soUt=a1u7U*+S;93Axu3ON>kVVvS7c%pPAAVKcxYrC!n=x!55q~|QMZdg zH2{kM*c1#p5F!lVLz)Uma3IzF^U}_(buXDYDY&0xzW?TNvfz$ zKn2D-pWyp`#FQL5@{4H67^9<#<%KE0&|{_(C@lS2d&!Y%ybI!Cx8}spRnty4$xcCi z0)Aw(?wB1Q!9gX*jT;P0o}E`SJu08As==hT<9V9CAIhMZdCfV`w!KslE5zik7EQDU zM!4&tL-2zsC;aB2+I9lE1xzZp@*q>PHKo_DX5|NdGnX-P69v~LGvqRIkD_13qnO(u zC21dUnX(Mt&Rfr!@}$8Kmt5nTGmuK+|K^<#a=RnWjYyHHs9TX-(X^kna++rE?5}%u zKZb)ey5ZDK_pxzrst{WKZWBpIQx^Wk;;@8-;ag-Fv_>}#o@{6_xYJ&aniN7Z)XcMiXdYN5XlywEET3GlXMco*wHaDOkIPj zI#WGtOb%%7`N`o~U-eQ*kfBRrm>O5-$YxF&s8+$)c+S4#8DwCb71&10lg5@_yd-=* z3uU5z{iU9-Y+Fp!TY+zpyP5B>4)U4= zn4y^llXCg-k3P;;m3jlC{Te9T$y>f#&wU^RXQdf_XZf@%Owf5z<}JZ-;FYWQ8imGR zzs>S4-Zg-ySRhE^q)Gxs=C>R#&mp7^GIXF5`GB2y-$J#H2{WQ}jou0Xl&^)JS$6J$ z)9eM6cQ~itPSnSOLq7=6OFnNRX!6DX^(b(jNlwdru~3pWquou6t^3V-U0v3oPmz*B zIO+s<`CE^I!naa-^hl08IjxI)+tUfJXKapHoZ(sCiZH$*XBl zr%p6K7x0*IBLQJ{MzheTPVzL+o#+ySAPUk!P2v<7#$K8Ts3!V|57xpQ)9Nn)+K?`; zXge1|zGIFuU{-*#&X(&3D;6%ur4E|EqC=B9qGVNetcCdaIU zoGR~tX>KB!pFwden8AN{@q@T>7Wsyft}!;s4SXdSSK-rTEb3HPcUBp_pZu!UWPY)7 zUC|~Uc)MY3Qjue)GL6Fm_f-^cxAx`aW1PF*I z6#nzV3 zo|mNZUZVwoUgs+zBOTx>$eOPA^bN6xpP9wmfby%+TJ^pAY;SG2-!bUElPbQ4BSugL zDfR>N2=nPDrUmzDQ-1ElERkk7zkDR5mX&R0T97T`;gHLj$-sdMf7X}OQ#oCfH=sL~vzl6cL@BoqaZ~P_1{OM3-C*T18qoXqoGs7SJw%3b<*Ba%&zb$ACeO zSI;rRGG<&%Ow2VDs?jt!f4)>kbDf&w_KmSToljd=#qmDMRR}2$tT9ts29ib=;ExcP zM~kOt1#fuQq1$m-ZCv$~3|JA;%w*#KsHha1+sEoWN|T#L8X5C_GAybKO*=Yq%%FAU zh)wUF-(w59%(ffjo0kLU^^kQMUVr#+?+fSCn*q|rE8c>kb@Prr^1UK`)JNtr)Ptf| zUc1XE3JLe9n{mm8vv=B5g-n6g(5K z{N3!RKZgqo`ipL7XUY_sx>W6)9-hj)@Dq_Ba=9A4=4HNpZ{MnnquM844VzH-V7tOm zQom#JJn;fa>mdfxoqC9G2j1A;^k~h1uZUF@I9~~K3n<561+?dO+f}diHz}Tr5zO9& zLU5HMa)`qodGX50e#4X$V?VE|45t}E|MOEz0mMO6TEH1kFqxgjQ+5ZhE6Czncd7dc7Dt zv@SXtfim|`eyXt47}=?JM-pAamcX+1zGq>F-S=_(3bpkcr#lIeh1K6M7}iZ)B=3?* zI!dH8YTz`C;Bmy;;FWsP2?@_N2*h^GxPQC%haml?u%=sC^T9i5eVDr&NiFBjL^r%T0V5r}Sp0+ZX=4Wvi-=YjCp4O3?X~ zl$Up8!u>9`{s?ZuI?<8O`^PWXR4{|cKOyFyKTFgXo-;On8{>Ws|IPv6vR03=WH82q zSF`=I;1|{XR6KN@&^(rD--8*SohajF!t6ttLAycm6SiLS%0~{TE+5$5%e(rBvBszL zt=MU-_HfzP4q%9Y<^(gZgYuo^x^x38fI*y`=L>)Cr=HcGCuZ3cJmn zmV0i-P>OW5_?~bys?(Kqw~U@;89B~zs@vjY%@$(?x3NcWUG?+}AUQnryFa8IS&hy~ z5J{K+o;A|N=ApLc!VCnzmb>T z{z3oqX9yDds8oBh)`9%_!RCtLmNyCTb*Y8*d0S-|!+(|)K9#~Ql{rOCVByy41BRN* z|C=(Pnu>x!@1J5tXN027P5M_VPP;|N{gg|sXL>1;G~Sz6kgn*LAb1yR=j;@v63jIf z9Wt2}zRA){dU6ZV6xfOOb$O_+zL-B0kJ(zpWbcS%>t#s@qPxpZc*SMhU3Pb*D(2SI z&6hXy{4@2GhZkZga%$t=+#^`trhh6U(jZ{7^M0z za>6_m^-RUM?mnFDaR==24z20!minRd*J9{aL+C>>}4eOj`8{3W7xWz+kS2>D&Bx1RiolH+*k+KKRa zF-u$`&jJ&fjaMh`wwk;DMQhojQ{<`ejP~IeoS)~4kRebcx1gBB4?y z$YIYZFE6(`glB8=Hxss_b3;ES^1;@ljMiRI|8mPT#Lx3ge0(}r%oGg!%QAengQ=Se zh9=+^yExl=w)hzvn4T4_(2J`#sFtOvkB2O9cCpmMHVmCx;_@3oN$a@7 z?((ANhfA5OFjyO?yX~#jK-i7{Psn?6kHtELjJeHp2Yy zaItwoJTRH%B>Q|l`|zhgYklDz1{ks-#r73wDz_3&ru3%tK??1U)!@mv2m|6 zNHjyZuZk;=}mR_u5!|2>EQku!P4=`O3cKSt0;wDu=X zIA--6HDR)E93R^02|6lA=N0jA8#3P%SRZ7FxO>c)ax09sZ0pg}AFu3GSzj{ME=giV z(wY3ioOF#=p_6NGl4RyExxY3QqV_VU{{DmkofB(F7KWrqgQoUkzh|_<~|L4Dn0*E1v3(@IHH6xzMsOSWJoWT*le_NFHnBS z;PX>r>~uyaas@q#-miI|ug3M`_Uve7Fen1#iai2>2ML9etR}Uyu??XNfg5$-*>SYu z;~8%4GdWFO`%jHQPY^9|8xMd@IN`uSg&^`=YR=YE9?%A?H4Gdc3Ias-1T|I)B)ewK z{)pZ~BCx%I6c|bpXbYg9aH;=_7)+s?fpY_VG7v#nzZ=2451C-lCm5CIvs~Npd<|*= zQ7Yqj1^k=nmunr6XBi+1`9lsxjnyy>hCR-dK7{1WzBpZdE#XhqR~g>D1OoW~Xkhxq zt6X1`+mM%mcx|k#AgLZk0%xFiU_zLzf5r^6qNiv}J36@TK}-l*AyCT?H9Q^}AD5?# zU$cj@x6~ergD?va<@6H7WPta^crfFQ{(E*{_iSyyQDnSdKU29pT5YoI@c zVgVb*A^1_mm+B2M@Y+wAR7-yj|${}p+=aae1@@2p$#mdb+oL)Mhcv24rPYYM<#dH}cW*>} z_h`^R3uNp3Zw*_TV@3a9)jI4BN#7`)^nj%2pOB=*gh)e$4{Tczsp;>k!LrFL1dRbW z;ZKpe@2`XdkG>TiL`bonzQ9_u~;r{^hZEsA6Jd;!uH}dg_K%suF|!Hc)4V)`@a-# zWBH^EIY8msw&Ojlv1$M1h1U4v_Lg4hUf_^?!sy@1s-9ljS6|UxY|$6ba#9r{#}XCr zF{vc;!#{2d`yKI_JgXQj+Y{6fN7s@c*1nCa*WWlHCmYXgjwfrACd<=HT2m;CU;T(F z=n^{?c9NFE^?x0H(?&$zs+FasaPO6&ds+>SZzInk(@s9*0K^`Arv z6u2PPT3A@n&WkTBwc!f)>&ukzZ+W&mQR@I=wps^JB1+-|67^Osb8#ib_~+dD#vEv& zB_9$&1p^t06}$#>p2;Ftk}%rz!r}4p?uiLfFx&!G4JHPN+-W-f(`TJ7SE_C1>iVjs zMH~c8RvA8XTcq~@N{4?HumTvo!}3_xV8^2kOpl<0L1IMVA?Fl=2CZca5wEPc6lmN* zvNm4InRPxLe}`tsMyx+t=PU&H1hl76%plkR%n+chg0_Mp zI-Dp?xy{3xgC917eTApF5EEOf{_aW+N{C02#m#T&MAC`dLOLOiu|DYu&7F{q z!@v(#l<^TyR$TOL@LU=x?a0{5Ze)fuxhwUpEb?bOt_|Msn(zxCJQU1|-SV1mINbQ2 z!lKHk1d@M?{?&)=Af3*vPF)*re!b?UiBa8nd+pEFy(}|j1s$It;;@DE!ThZl>j^}a z+Ewd7G1YdXeSk~D?xJb9`UiH|(!4iEF7^CLX>}G27VvhM66kX{ac{JEgIkp6%l0#RnS0KObR|<~4ne7vo-xJ*DP;E~WFymJoE;uh%T1!~p8MXYb`HBG` zE%24Mw&WQoT3`FY;Ie?szusLIJZZP4G@%m&&IP#$5Q|K0yTOMaqVVi3h|C>br#|YD zADbCiS*K1}tY6Jq`kKajSA_|+19|(u(=%``AqnvNxN{I^o$`7Wl2(icx8)%^{JnS< zc=H7qDvirq$A6TA?a*^A_r>8NNo(B#(XF|Iqi-oK->cM1YGXfjp8b7#{_|;B@eP2y z{`GiEHH24*zwi(AvsXSSyZprZ_M|4olrz~EE2y0Bii1Kv>$L>XKoWko#+x=MnIpV7 zY+Aoy_9!c2AY|f}rq5Q~3#Tk&4y*|=PH)%uzT&^b;OCb9L6>RI{*!zH(F6WCH->r# z7KMRaK`jRU7`omNZ-RoJC&R?cg8V_pY<^zFHK%`c+m9)7^_g8MrCKdGXsoeK7)E7y zGM%8`Ti3fEL+|~WoB$!*=0{^R$9rtfY;NHUE|o4dlcS9 zMi<$A{+c`q`@Q4t$IDKa{sZ^k-;CgT`}Xb83a5YX;-V?D*CjozzwCXG+Tw(zKnh_A-4fl6zcHLZUrvJ9pIo7AnOHI zfj)Tx3Z@bIE)T;L(~2ILX#cIW>3|-xl5J#euJxxZ=uCh-f+i6kJ##q}iVgtAZ`HHA zxNzIu-I|ar6CkuxC?xz|gG9;SUjkmoKgAAUWL2s_y`x_)E(R?%Ld0&(HiKFfGD%Xw zN&|BPL|Vc>2K1E)$VmwCD6tLiCS8LU058!QU@qtrA=(o3ci`xfCu#(B9yE7C-g_*a zrZ?vxNCgj8HDqS4z;OQF=tJC(phe&iz`Y6;RmaT)JX#3q3tOsSG|tgvMR+iibj$7> z({v^P#30yWedMhdm^_4@|7FWsH6g`Jws82%!TA`YD3c@Si*RMZ&5%;_+T`JR)BOvs z=+B2Fsu(6I*tr(C)|q#`a$`vj*@FWaauu$nm(VI-b0x!`wNW=S9^Dq{s6wGDFhg z%@oe0C61y+rKlMZ{cq8nwUxmPPD``-X=%AVtxUpd{pNNx!Cms2F<(%32UT{1M($fPePa<0%g4uDxR0i@38x>x4U8l) zRZsSNPVH`;&$ObrNfEvo;qHIK*j<&uF!*KYJbBQdJ=92IjP?41YU z<8vpo2kx~C$eui%na!8mH!iksWUfMmH{kE5{V_KB6(jrTs}(6VwG~96^@~TaHkK!6 zp_d-`)?=N*Kz;i5VlNv$x;2@doI%m>@)G&K%8NJ|pl=2?y#fyVz@L<;VNigm2xo5E z_W?~lxOFf01qIU{&4)~yka3%7ShKM z3FL;*ro>K!`u2hU#R2&@T$!)}&^|It94Xd5otx!uE~(8%My=ly6;xNGU8e_t(XWX0 zE#W=a$kqWsJsNu4#gOV~%$J3#Z)WXd9lljl!rpWb-|!$*&F5QxevmW;!A^RgFZ)c& zNtkbJgI$^xNzoN5RccJQ69!F%Hp70nv11jJY+D+4WeQ0>$YlIrhaX1MrvQE^G?sZG zbnRQI4ysn{w5T-qhJcuYeZPp1evGCg!n!KXCC)5{cf@TTdfaH0Xq7({MX?OuC#4Rt z-F!8;#GWNp`uggeH=&2gJMl)r zUCx;Q#9ZvE_~;ZO?$J<1VOZrRg(v`MIHB`I3@PQ~JEAwn-PTbkGZ=qQf0Uoy21J&6 z%X_>LV0gPi?1B*+_swbSp__}@68>3<%<4$M03^PV-M;-R6%%|rn#T}BYzeqnUnLlY zDFmGYc6OZYr#3ZzvK0$GOo=nNe0ey7Xix;5JzXXFf#6J!66f4UK3Pz7ySlqY zd~eILbFi~NA2gjQ+m~U>5@PYKXX{U=EN<0E+ zmFZ!tC0pQj%>SKwx+>teBmn4uj@IRZ2P_|+A2>p|o+qG6h^G=d1)z{)sKk`u_5hv@ zQj`G+fNKDp&t<6JbFJC%oI$566WOL39umz;zUG3aRknE47I|mPZ%>VI=vc-13JE8i z&9N;nEe$U7E3(?Tcku$tWG**+#YKsseflfLHrKf>Y#hf(W;o16M5(3ScEDL*vQI5g z)^3WA4hbai1)+S}`IVK1&e>~ZwtVs_Y7a7lNIb`m_VQOx9Ixw+i!5FMJG{vdv|<^& zA{X+K6oMqA%^R> zOh_T30U`2W`iNPIgNMW32dB6&7aIhQ|M9}}#Q$x2uNomq%}i@^EB{TEvGp7zxagYSs$E#EBo0h8shAQ?`1M@j_=wSgSq7f?dUlAR z*0L}!SZe2#JEzu{?>76bL2j%8^P`0ijt6z!3z1pw-os>UTzaN&IM7ku0p8sx-V{;T z{8?A`KQ5LHKx-RCYOFF+du}bk+bx3jJA5=`<-OKbVC?^wgcDmlhNPKYIPCTvqf;;{ z)BS?z_%haQEuzpD)PAHn@((UR^J%WNu#dyp^ue_Ly_N|;h5wimfVWrG&zIGOm&OUp zjqf~cy!iR_+|sb(5c!)@EG<8tlkWFk1#&Sam0C|OU?u2RI1KM)qzA}6?OpCZ*W!D+ zVtj59d+B6*p+gj~K=NNHJq922SP{m<^LItJ_g9vO7YNiL%LDN*KFjpf3v0vqV(oh= zM&1!Xp*`x?Lr-gT>R!(oJoy~rgyp%s^UdiRUI*3&*^wbqA_>altj zlKl8W15hc`&N{oy_6p*;qX7@^zmqky9~0~;HyP5+h&S6!&{*`>Z6>&Ms2R+p}lbYhjQnoVAhTDh&f z&Y+~-HT0eTE*Yj|z(yULrtK?RJhpBJ!^b#M=I!pCui{XYMlg-%PSIF%EF{f-zMD;J z%|;M*S%WSX2K|`TxR-pDS(iEkxM*B6(A91!Cd7Yb({RC3RIkCkGVCIjF;uk3-V&oD z8rjMxyY@Xkn!&~8l2Ecdj|O0qirQGx*+Vtkcdf8&W!t+5>WKZ0*kXX!p}QlTaNF)J zOOK4(Ly=fWZVpc7USt(6XMCdT+w|{2hRGQ!*qfLZl32ik`#A89Amz6G%q{(gg@L z^-^sVDHBCEEfyNYm;Mhz3YR-2@*qY>M}x)C`VRR?3JfPZ}tzDUj87eE`r7WJ>^!0j$ZeH6MgufNKR(LB`7r0Z2t~ zQXsJaNyyVEf<0FvTO0d_4#XQv@xyQQS(;~$^TeZY@4ctI`;j%W#I#uPo1BDddEipx zA$n={a%S=QOwkpWleY~`Y&P?G4|5juG&65wi^-~Rb8975$1{~N4&5S@t=#GOAm;x{ z&&m0m>(%qI8L?R$soPn~Gj1PFqf7}6Axyye*i_NBOnTHUm-!8i5_23Uhk*Ul8-FSt zHfCp0v8V<%cJ`fi98S>f@ARpiU>TA$R?D3AKl&cEdsr5`6@osB`8r#2Q+T#P^|I5> z?z^b~{Fkv=9` zC!URijY~kJ@Ap?W)v3))Caez*8KI$G;0-CUm}uDLaNqpV4-StC@XQgh{0{_+-*gLG zpE-LXMzyHA$ETv_)CZK+GESkg@A6ju8t6MSo_>hl@Gi8h>aBEAyfFDY_+-Z@E5V zPE&~M-{%$;Z3mUb@5$%n!NJlDi#uhz-TkvXf%iX%pbmjXIeivz?iQW_Q8dBpvo@6LCc6eQ7!@_O zzgTTLytO5rNM~3+`2wi>)#9nj*_H!PdG1wP0JPr8mu#^g-Y4+w1sv!lJL3~Qb?WmD ziBswB?=E-5L1JXY`|R>3RC((xuMcdkG5X7M&s3?7xMXZD`}q6YF)@hFUwD*#tf}rV z0aVvC1m2Ph|2&`aKY5DsD?2+bzwlPiK)5vWZzO1KmUNq6XWKnnVmCY=$GjLX!zxnH zQ2s#`#z&bF>XLbGpQ%X~R#ueC)U{R%6^vFTavyT7Wq#SslYmhugaRc!_9&-1MB8K@pD*Q;M;sBQ5O+uc zbKDOb^xReaiCd??gqq$YdMZB(c`5tAZF%|9uXzR~TXdnZ7X{4m3>aA01~^a8Z=HsY z?_gn}l+j-@whzgx;Z_$4RQ1RaFQasvceF_Y{~2wa7!?vi3T=RUi&>j;{voAoxRr|2 z5SlEBPt(T(C?1r^`lrT=ykg3cl9eUP=U~ufk4c3ALuxDjz1jgDi5@hq;ip_5CYtPPuC}Uw4TgF zj!)LQstL_i9-rkr@TF>ey>uaWa}KSY-ZWzE_; z9Qn$?t#k46f>3dzfZ+aJU0SMGz&W8-XP!@kzZ2RFSf}cyLRiTFt2@X@NuR*uQp)13 zC^f!M#oG;1&AzuNCgLb=LBSqSa6#h<-*f_)8i?M5a2Zi->*kf0$1Zip@`{P67HPIS zQ)p^2CMhBQKSTizPdvb}{~28iOrl@DgbxJRq~HC)1g>(&p%fJIVF;yLTVsJ{Glfr5 z1~pgfpwC?94&LsTmX?8rvGh4URYW&UPY;nl^McnQ*>25>18P*zX4cVXTr)@=iSuMgA#~ig)vPwb;+E{AI02#6G8_HJs^n9)^EIuB^m*>% zwps&OLN6_*XcW;mP&ikDrAwD-Wdr#&=>Azn|LM%oBuT-st((fJ@j4@4Qx8BWL-&R$ zTUX}SLdmj|4!6<)YFYj|$qOnR!0Fevwydl@UxBp|^F>(i!9k*;RX8)jT6#<&DpLx* zlA$KbdPYxXl?bv3LBRv!36(-p@~QT(zxi-F>5`PvzzRio_cNh0aF=Ii7-8~qq!hOc zIkTB>s^@)7-1}@3d4KyaW2V|a#^>Gk8A!~xJE+g&5M9``*J>sVNe_jLoE{B1+=ID# z71&~G@La@lkvFO8a_kv_Ug9Qy>)~RQ*oZ&Ks%9S$w1I^N=JnsHgk|#k=c5oTgs9uW z-4Bf~x9V=fK}&K}9ZcU}mtrOSdfgzw$hdLY6``R8G$qe$59hB+)NP_ToA)%-84sZY zc6|4z!6qGA&v*32;x5ZQ#kUS2e%6@^enB^J8Gbyi%auq7(7|sHP{U8erTUqgn$E0i zPYFnXny#er>(0(j)&e23T8Tp8Ap$G_WxmyV5qZ3g0IbLu&RMUnrlw}J`|GgpC84ym zbV_ry{k`X^Q`N+MC&0J4NFRJq9WjUf1?IgSy*pSddihC4Pekx* zWr!|c_QR*cxa@ZV0AC%}~ zU%DoJ5}(-7V@dp7FyVWd@UKRN$S{UGAx4CQwgFnSdnDLjR<{X*6?Xg>*aJWB^OF9< zvYhC3w@!{N2~ON}_F5ri>Y-NBOrwXfi@53tjNj0ALUaYlw1V=6xJ_2XqS$OU=8 z&T!R4sIxFwBY-66bs*jzs!JuC=%DmNy`4R9>Ha78%O2$A)b!z9E&KSjo)1zIG1{sXYCR_X`HHFTx5^j3v?0O4$*_T2p|8fdx zX^E`kfpGo5nHwhJP5Vd9Mq8Q;<8GPeJRk|91y4-h3MmzpC7`tk&$JQwJ-*iz3TWs}V8h^*|9UC7EF86kv_P-JH(gskXyUfuWi{yiS~ zw3M;aU9R1a~I2C!RzgS$%6p$7Xt(J$0sD{(mSwdLzZrU`P?AiXmo9o$LUl9 zk0q3?(2j#4iQcQ|b>FQU8K0Bs82(p0j^l+q{?Dj;uScJK`B$@2jGi(@ginr2-G)|q zJ$pGjn3ceOKBUQ(1;V6w8S}^QI_Tj`)aDNrKdGMt>52ng7b>66|kFxw}T#y|2~Mp5L$eD z7Wa4ej){8)ECwuJ3A@gBSeO4>=e&OlVgQLQ5c5w)qzZ|Pf8+Dt_*|lraj}BQcwUS1 zPD&~@hb|Ben{AKWe`lAOKMK0D(}6{ubBJ~Q2m@xRI<`J_ccYh`tlC48PbXsks9wbr z#6lWieA%5)j7sB9jTGy};#p;CJXxdLW+hW*B?i52;JRWe)YBq~FcYH#rVgA*&Epj( zWhHi7q}X?M@I!Q?gIAA_0v|I*VuyXHPyj4*m?C%q!%ErgN<%nR%Id!!NEi+GdKoU# z$)#wbaX?Iib^Su-2v_K27&nH#yq2Y+4Xy~!XG^ZeMgbCNmj&bpX!E3BPAOxUBvpyT z-3bcG6~&Uquptqt!C`K=#9TwBbnnJZ-)(nmGG>X(fnu0>;<9Tx!ST78Os}$-6-ns@ zP)ve;ssA3O`3zz-g*VN`;XUva-6(ELxA+Vaa(*><|KSMtYlOo|i0OXZ-I$qas|=P1 z^Vj;cf3@x=uQWK%e1O{sHagSZtZMYY-N*<*vW_NrSScc^9hOg@073|0odckbXxn~> zA}BT|dD;uAbW7O$fDL^6G}}vV+Af0y6vlM{r4dXnKO*dE)gsJ26R+k!h-SBacxd}M z)$!mZlhUh&59P$fiG*Px0JM411uilgjQGcK+(4yc-RuGdk`X%SLIxU1)Vi2 zR3=XfDm`eCvq_)4dH~)h_7``jnUzRDE(B))5U>7O!r>I)K!H0PJd{fy)?#h(-iDJD zpvVb|@%>g462)K)1iR|DpucC?0b&OCthnz#g()9A)D)DI3l%Lt8BwyTV@%1)SK@ka z`&|#>$m?S-&Y-tN&HD(}uhQpfdX(KasY&J~meCR7{c&YhrA(44M{T@Z`!J(1SOEnl zI+@o*x9#xH&$tUW_}v0tVBo!DyBVxr+UYQGd8$%GTB@fyy+m(4j#<%8)D8V|Sd|+V z!xAs_OVoSZ?0f?Ks0+Djq$@=jfACWDCC2z;5rVz42qI2~_tG*%jd?ziE`01L9f$8`X+@)J>U=vm=e zXR5;M{VRJ%V$3LqR(W@Y)@+2KC^PsCmSIp`bMV-UTVBq6DE?k zC&kMan{efEnEaM6`)qoTb712Np6O+zkBfFgWXj3P;d95?b6=l7n0_+v_lTeeMjLKD zn}%G%?WCHC-Q!%W7RTpJR1oE`%@bn8u4#U}sd;ef%$}?~yd&Rzd}Uy+V3{RnypiU{ zaPpgkyN{Pwo`9X-=+GOJM#8rij)W7x0e0AHIG zud7}&k2!wV|N6_flKQ$;G?DAaya&bMo5#00Q|;qbde(=FM+C^^*2%Drq&u{Vq^otT zOSD+=WD|LkoY>_BDOvM;UYTQn2Kk=C@zs;ZVTC;g{n*Nt#43qQ8vDAtqg?OkG3J=7 z%CHX@Vg4m;iO+=J-v_0P<|j*N;*Bd2rFr?nqNJbiv$?;KveU!M#x<`AG0l{7 z9=vk$(`NfWninHJNNNF+Fbtyp^(=r5S=)9&3%5`{!3IQ^H+%d88!<)(NQ$M7M!lth zoI%jQj8CG*=#!O=Qy*wg2_+STg*cAtfm5*Gs^Zag>47J1O#uu8?!ENzl0f4}og+jS zXZp(mZM6@b-F?pY7+9&}ZGBa4%;+#i+3q3t!C6HsCFvw)Cbi<*f_J5!6sJA^Ay=wS zIZ)#k0P}Nv%sYhvKd*X5{hF3J^Zq0?)=(nq#qOQlJl3J3iW;XcV%-I$TjD5btX!G$ zkn^W~<8Sqp;&?LOI(jd?1Nm+|q6 zuW<7@?>9{2OPJAc z@5uD;X3*}Ryp;&0k;MLpjnJd6qqbofP}y4ZIr9 zp}#N=7_0-BFZp^`Xw#LDIKb7IonAwy?wTcaJX#we9rK83((sy9OMc#`H}|+QSA+M1ia#Yj^J$oLSu@{)%{JLTHJOnF z?DIh!0ODzcJFmlD4IByQmMuZhV<|ijD4Vf^6?X%1Vpyr2)sVQ`Eom8*8Y?pW+LfDP z%p2;o(>3_-i0|OJVqIr-6|=vPAbbb=oyPm$Mh@!PaxHf!K3bHrWaQ`dQw7*Wgwegc zol@|HjYrS4@H@X7x(EBVtTLA61RE)7!L zMf^7rO0%lmlTO@owfCzysVX9uPkp2Z*haI>q-h$>NE`IM&Wl=?TH(M`hAlIT)62LQBN=9tZ>5q^Dkb_wJ0aIr zF0;;`w{xcwbpAWiV4~{09dxV$%gj6HyKzD-_7^&vJzrikQgXY|X_$0?Jx{F^*{+Z5 z{SbSh0C{5&tW+4%Reue+1bU2HgTk@#hWv}HtnctXfK+$2x#;nOyRJbu*eg!khW{Zz zMrEo5sjsZNK+)*lL^68*>%;i~p^{Lv3fXVNxaIc*FfVmOt_3h@wR)pfAEy=bWZl!Q zu!p>9cz!9u?mruVgPep^Iy831&!1aS7E-vk zURSzTANJQA&zZK}|C;Say^K7x^)i8a}0IRm#>akWryYD%?nfqTpWHljgM>2LCdvocXBPo(en7R+Hmnt z!4i>j55@tH`*zPqzOR3LbYCm3fP*UHHG{odVdGsR0XyHz_1g);(KF-*5FV>Py(8;^sCK~wpAdu|xqtC&XxcTNh1oD=77 zb%U_k40 z1wObff{e5~5QWGy8W9;d$YF@Y4cgzoHV=or%q94GAq@v60+8zfSeu-WN93mdyAc%E zD{$I^mWIDN-#zJ(2Sw@@u(Bqi+AEel;}jIwA%>KBznjDHLx_hj?TACz%cwxI z5qmIa|L0rO+LKkZy8KukDHOdV?b5f&|m3q|PlnB=^mAKyPh~5O~(vK{qefhP7`1ni$xQVDdqTq{dDr;h4!IQkQ zPbjcD)f9=bfk1^YLqK2VlXI~)uyDOuo%323zpAI~H)O9}9_{#S33!6SUK^ya%YxA! zEJ_B!X4AA)KMiX==zL)~0{z2XMutIL5UGH z7bOGz(r;ar9k@S_3s$DleWQ&WS)=kdb^@o?VaUL8;2g z>FKgkz=@2n|J%X~YZ5cv4e<5RC@sS&}C^SX^%bNzWx< z(-3jKL8}I0l77&C@}Mk17zOfIcf>y$xF-~p^kBLkU%PWU3-Jg_bR`@~TGX*`AgcHZ zp8z2l{xjIzz2+uI!V)5bjv{}6?0M*ztJjRU+GS1>(VgCi6Mr{apzzQA17qg`GR7ZZ z9?`xh(F4Ca`JgLy5#&~b5s5^)mdBC{r{|?BAqe_oK9s1!?=0{JE;N$WWm6N&r~Ao+ zHB*9!H4?%zO*Jzs6hu1@82^FPeP`#8M0OUIicMdz`uN6RLO5ddojZbWh@PCPUg+Gw z3!l8D=!waVXRvZ%a&I>j1D&AMjRMEr_>-g{Cs1%=LPLaHY@Ecl?<2=FlH-;0<{7hAeZbBk3|0U+XQoPD z$rZ&|dRAQN(Y!J$TaID@J4y`kvID$n@(0Y6N~7`oSM>!j(gkR6Z&|^ACe&JS^wX0p zV@rZ-azv1#N`yQdC!}ueyw#4M3b>vy7}f{5qHY<35(x|(!LtY+9jtK7TJH~0PeiZb zVFQotzYxq2CZ#ONfDEt(!|tsS{OTZ0hklY*RP+OsvH%cGR%Wso!@1DOawTG*=r>8m zYbat?m{&^t2&tSyi(NE5Hkj&oi>WL}OEtVUzO)S_qB}3iW_7%z6ehKx6S<)V4s{Ny zVu#wvX%*s##^5;Wqx)?tV;)Mccs}`NKPvlZF_!OS_*o)Vt}H4pY%7xA_(i8rh$Nr9 z))+TwcpCe|2XoBvg(~-M5IgZE5Ifr2Ur`DPw@7fPi{2FFcxL}U&5MoP1yX&Paj##; zf!;r+!YQBDQ~{x!UDLn-az0wU87ha)%coChe`=LCOOavx+wKG9@Aw^kc(63?MO;c6 zDxmp7!b<+mcVg)ph4g09teKC9oP+cM!;xf0J$j6hyNbm@&B`Q zz1?Co7mzhxQK~p_J&?&WVUCbQ+ySD8#1aA?=4;K!o#Y?NEN(t=K+Ja%U5F!u=+2)V z2$8qXKx6~R2jW~q@HOx`UK5W5Ena;wzy0@oTgZ4&TvBp2Nykl~b^ntJ#~{uyIBx`| z6+n4a>2QuHW{#HPOq1$8!KhTJe_cieJLv6ITHwrsMZV8^_%P?N^<{7E9v~DpGxpbk zH^{fRdsJGS2~(DR3dnkEu6QBVw(WA_c9Zy94C3Qerl`mH3(C<)666eOvaCtm{Y6*# zIe~ps^7)p7+NJ4TJbUBswX~^44%zTSeoXp;dkJ=vPF5O31-0f}3d-5G6yZwQBEdCF zV{h^d(HZQvW2N;8lzMB=XIh#VLz&hbF?x-oVdbG~^AHXRMYA%ejTseHX(>aAXf$t3 z;*7jma>s*3ba7`z$XJJ|u$qP;{e98cthr|sLDcLdiRn1wcN5X^WTIM`BUp2ob1h$j zN$78uN45svCQ`!e!H~doh)H2!@Z~dL!Iin5U_#$W8I^bmb}xwH7fDe7rM;E6_cf(N zWX=WtcLaU~2UgsD@rYvk#mHpv1P6m0fves*EC*hU>wk}WL5vtG_Ome*tRz?A&u!lZ zbWcP@!Fq-wKU_>gS@BiLv$a3BJ?y`Vm-95|=Y6o(GEaP+P&7BlM{#?{=2(2}fKf%4 za{BPTlkTe~L4n@lg)jH<$#3^v(enb_tNez5p;!66R5E^jh7u~PSYcWGr(fk`YsPt3 z9mq1q*`Hmn`cnH!j7}wGDDeSbj;i*PUNIIdj3!6gQC};ml%feor;f+;P!aXLi7`*> zUc$F14z5`5Tg>_&;^~@naGq272qa=scu_K%#&M0q&fQ-I@7eLx3HUiDAx8{2FG8{=aKpY+c#8 z@C|IB9L2^aZ~ypS3aE^5Z)%nKaUM+MKcBd#pW>x9jcjfU;v|(zbPO2QB>jH>@Txp- z%UWJMfKJzl&+QN7RCnbpKnw}t)<5GE;Z|sw^&`2n2F9;W^|n3*B^aP~FpcY+Mh^K~srLgLOvK_|U&|^q+Q7dgg1blmO#1$ZrOS3gQ^T6eR z_fDNRc=a?)^LgoKWh`#~WG7yAu4F6cYMDB7L=8|O@Ji@`(XebXnbTx}R>LbnPMTp^ zG}*}~d?>!$v|mouYLZc+6JfI&uRA`JEB2^kggaLE<5e^)}g?zMr6vUD+4q|I(UbB>t_(X|EY$D>(ISE1p;u5Ifk?}sP#XZ1a0^r94>-4*fogi|DI2CRb zOm1Pn1m(GnkSRQIR(Hq4Ig*uGM{`XoA^jAQVFJS$8S8=)stN>xvR$$q>J|WGnD<@6 z;Zkc*@42}YPA*4f-tbxSN8+ps2`@qEYht~nw9-*!aScz^9{S?6?_ z3}v$3)1qM0pr9y(nUdt(oU&lwvSQJvwdUD%A|UJ>Big{3NmB^31{>Hv6FF3Df|&3U zIfWsN29B_8P-DO2*Rzq>o@@Wx95q%cFc5noUDZa`;vBZTK`ko}!1sW7CB=XB-H<4b z9D8|h&NZ|Xbnkw%UF+-XoAnhy|C>)l?3RF;@|QiGcW46Bg)a&hjEIc?nirwX(l4*{ z?LZ*O5hIUWJ?J6|e^^;>ir)bQ3-W0O$lG?pKQ=D+lOg;cLH}=mz;&}mISXx$*JfAd zxDzw65eL$U-yk`ju83UnJ4$GJT%UVa^GT+tG)1CH&!`Sg?;hJY6-SMA zc;)%WtNhmzQfKe>Z_5mA2mi@QIZ{Z(@4^tLZnfEz#(VQwkcX}6iU(#bnu)wF2|E1xtRJc<4Wr+loAn4AmO7#r}U>+e=Lbu>iTMc zGx^5t>~S&mypP(-EpoJ$22)X;IGVX=lZl@;Uc;KwSXa@GJv>c)_Op^5dog1{Kkpl+ z7~GJ|t8$vK)CD1r^YlZp!?&xG2oe@6G!!J639pg;0y1dxQMrQtDWrN8J%;RfpFLah z<;vOQ;o}=>@~{365r_nEZs7ahYMpy=zE0t1H|dtzw_S)E<7xt!g}zoI&DbAVqLtvPcs0cgPAJ*H6(owu}g_T{de64{7`yUZ-9o>Had3s@jr8P z7c!`zJO>z8uY%;S1Qo97!!xoME(57vx4!oAR8#i0O1YL3JFFT$=XTN?*#_LOjvR~g z8X)ahC}p(`O%2+vk5tjH*Rd2iU>R$>(WJu2tlWI*B11`bcSzO*GFmiJp0mwhE zFUc@}ks(lQh@R1TZL2#ByGM&`z)hcR^3}vGnpKM)jo!TyAzDsQ%*uGGF(*YQ#WW0} zI4W4#MH6#mlwuz<6{LwgAJqNxLFCUAO8PCHZtO_K6(19owQWJXY<6rxa|e`ulP;?+ z197b?_5wTFim5I+8o7L6QeUF8h=c}!ci<(25P=8kLCb?{4hyG}fLj61;|REILBRXI zqod>Bh1H6)6L5Xt4RF1bi$Yc(f!hMkR-`CICTO7N0uIuD^uU=jR`9AmUBaOgcqw1a zBRf>?J%|lXt#6M4N3G}!6bYhVYdUcye$uiU;tZ(f)R#~*xM%T7l=4Q8*_#&;E>|M} z=}dSX@JBbXdx_$AG^~gQ4o65Y%4x-GYLemrpT;V=(I$Sv^8280R}^W0JdLrY;)ZDG zE|VgOH!-k|k~?}X!Pp~gq}TpMDx3S`72MBR7_tW#_$6;?oAX`OYx=1~vD^wC2#)ko zE&9v3?lEu>l;~vSC1^UgFM0po4k8zO_><9;jW~`qtm*%5ubq@4u_39k19i;XXf%lU zP^7qDczHT!0~+{*#JU#WU3hwFkCXG3mWe^(KULNYKMN$$iS^^=tlm%}0lxlZT?x8$ z@ESt9MfixY!wmwltJqh1FmL@EF=(LOHVx>%;88a{gTSi#A?ISaGC$%0MbI6w*dek5 zh95XbgSL`2Hq1XkV8Fw}Gs<%bVZ4FCGX~Mo&m5@1f3XJrV$EIv^^*y@XU`dFzFs@H z-)|#?T?hQ;a>L_4+vh*Gb;(*@Nch#;wjr8D@zWnQ5@SZnJCILrdbXkG_x{0L+galD zL-47>%l1(1guo0W7VvleZaoBN>|akMHy=O*06cVie;DBdfPB>V?)bWAvuzF{yPv5C zJ*Kboel+VNuxkXdzOMre)ypD-^wX!`bjxme?MFeJeAk z(OtxbG^`%0E7#pOcne zs=;fxtL(y<`f?|w7>|7nTQ_RN3(w(FSg^vH7=6E;`IF~V_UQ~vWLD*4k;E974CWlm z80#ct+A{A(SW(|5{;D^q&~GY6W4@W<6@kk03MAV*3Bs@fi75uD-n1lF%Wl96!0y2~ z0{ort>$irSo8i`YN_<8aC8-C7ACPSW{7gMZydQ=d@R)xGXOJyO-N7x5MA<`d*P%s_>~Idrnzd+)oBnqe@w$HC<(FC(2$*8Q2l`sK4_G3;GpB%Xs(TVO5D z8o*_0lLb~nVk?{UcGQWOw;@04;@3-41If~f2?+_zIBKI$90_@KuinAKmyOE0B8#?q z6A{HYKuUd0pttLH=R*nqcTZR8%?I|ka9|$D&|*I6?E{;C77ixQxWuKgeXI)|I?Vj~ z@(E0Ls}(zkpbIF}wtxS=p~`RA1eScx-rG&FfDc8655){o-*5bHSMz-Nu$573ogZ`& z{Wg>|bj72uw96)dV8#nq=Z`@g8{3yRKYpyK5{?{qviK_aY;$qya@+QUjCJ~ER?7NI zZaBer!GTc!KslMDW^9ZCVT9!Y(U4#K!~6)htZWygBPIR^J)XwWTv6aCsw}pvQbi)#6&mxm1LG>fK?Ado zPmj;Q8^A`kjjX3S?+SOo=nz!zV3Ta0y%7`m`>SnwR+f>TDv*%?qLGdPI2VrM%&e>* zkAe{9-!Gw{Jt4hsre(W8@i?GB<@j#gbf}xkV8Q>d5!|v3Mm!uRJ}}CWAX|C?Yu3QO zhe1K(hl@yNoEUm1)nyNCbd~9t%MSXErp8;sh!B2)Umgl&m}Gw9`w4j6ncA(XI!C?X zoqK`5k_MFNkB)lpd_;b=<{m=RL6)xmpa@+70TqIcJ_`I5@B;DIy?hbq8=h_pvdW?T z=gX%*kgpB!4|@dSnh?%z_tkr^@4t_=Ic_rrd+0(qxw8XStW?!CfDsM2X?BxiORE4` zpDpooGseq4yNu}4utE!#^t)M*05MzZCLZTr&@a;@&ugPqAp`>(kSu)`NOVi`8SEGl zc*lJ=!J1S~UXzqb3ASQJcRCLKIvjmrqr+87ZM#jZk`*_6b)g z)Y*?!(ZZr-LS0g^J0_*`aWc69GiWc#G}N9P)V-&?X~h?6Rk%CEfBuQ`&o2>~2BrIZ zcyCj<4JH<$8O1Y{$I-_sOi{XlXJ`|aOD4e4pDN6ANfyr^&j`k_VJS^gvRa-@w;~GH zh=nuXhT>sMfbndz8RU+jOm4O?>2-sNCg_#Lhg)Lmpn&X@`v-M^V*p{3!F2$nLvA6V zUhp0wCYPL59tjCZvyiZ7hg#DWXd!M`L<0?WzG^a7x)i8)cK0^|o~e#kzw)~rNSvRW z$nz;JI+%sS>VVXH(6Zkl7fp_Lq20Q6(Lz~fQ|Bf|>qnpF35F#tP1+CMVhd70)lPry zL^8?GFK14F?dA8z&qmh=o~n+PZ{fw$Kc8BO_$|Uff|)Ry>(%z=dl-LfO1nX_HE*+w zGM21;I?5S4A$KJ@j8W5l_Sbfl<7e;nTR-32KRor|tH0HHR5)IH&^|{74Gua25_g*X zZf}1HVm)Vm>qY20j1?wrvh&;e@aGJ;1wa*4TKPBf@6PqG zo%{5S;l#xkI(=bk(0HRGJ1YzO{+MDF6*Aw9z;GnzRvm(`Jxk!ZJ|-?l;TL41$#rRAZEtr%V4kvY7(B_BNz(l<~Z;zTd3 zAu9OJ8=T!2|#?b;eQN6azBsA*ek!&VJip@8DwGzzvVV; zGg#^35a29&I;Y=C_sKFQxSi>E(^cbK|2%{{y$SgHV%x_FHx!O0gU%+OeMQXH!vQ}% z{veV|zmx-@1VtYLJQsLp54Foa-n)0NQZx|hsZVx*p}>3RsFMi!$B5w$*F6hWWvMAn zJC_G}j_+*Q!13!RD?xzE&`U!BlLnDLt~O?-rLECqmN6IK1zGr$i`V^KhqmkMo;5*j z;||B_$JLK8uhF9!t9rDKGjK{dzy9*Y3LwyapyoJjsDkIUkTI_!9=ze(4O~>pBz@5_ zrG2;hqk$br5S7*`Wg4wj{%rCYXZZx@iZU8yY}uw$Fj0%~Bop_m6-rwh##t_8SKg3g zH|9hxJvuU;vgLhOI6Dyw*Jpx=CRY z=@^2Hak(I2)n}`25=r$|RgE8Bg)K9{iy`MAcwUr*lHmJqj}pf}Q$)9SN~Q#uNOkmC z9~43%ptSA=R=6p}_)k}%XWpI?HpifK`&)EflGq2cV%xRefB|K#itf?T0sS!MuWAac zXT6EbTJ;img3bO-dvkl{)$4g5c$+Pfcyvhd`SoQMPB{3j0K1HW>AQ8%M5MXm{An1J zUD=r*+)uyj{k$^b^0~E2v-(6n<>8I+?Xm-7t0U9%sUiBl!!zQ!hxj4!^!I1Jy=GJ1 z{B|SRW_zo_WBw-yKOutyxJKd$q(YkiM?H0+W>S*TB(>5nhPe_;rlrNSpwmFHL7A3; z!S^5OV_A|CJ?UfGSz>+}i~b1?&;~U(Rsg0}$XNN+wjU?gN0{S@c=!^aHGyZpzYO3Y zYH*)F06P$fvl~98ZnGvUTV|Bsy`!%=wTk`Dh@KePfvlF9p0Pu7x<&byqx~00Cc6=| z*^|!@OC@Yc{=RIaWBJ0JdgGcTEYT75lXaBKox`T#-Gj=Yw=WIO;LYJ`nuUmINqAL^ z$J?AT;GhgHDJjWW=rcnIS@BR9w5`@HUs!X_k6r^*ro<8e|5&T(Nm}oKTNhw6Ig((6 zMB*?cmL@hL6|M((j!(k+r7Hiy%$cviI?)V*HiY-D!Pc1i5z!6vV}}y#V*6i+Ov_kP zB(PjEY7i$n|21)(!UGYXo%v#ds$o!9tvJR>p0L46Ov3&e^?DcdhmwDfhh0u=PvC6H z$vLzJXc=qClGfeRcc7On5@>jIT-`)PEg)1rF*ozR^dO}@y*(=R&;z;!`xft6i7jS9UI&H2_w*Qq8T5u-(`;u%PaY=*U@`pll2*h6yu$AlpU2_qF( zn+avgWbE_|)-S$^E~FwCkW1*NcQqv9=ZbdIWx&J3N*!?|Iq`O6my^{ZgmIdd4wMtm z3+qxDh@D_G10E}YO_@VC4HoNswhyKFI6zxRTnX>r>q^+e6ewhXxzu9iAFj`#P8mP| zN&WMW=$eVt(;z&9k1(Y3C@f??uhb5kV;+B4X+s&;n(ld z@$yttNJTG}N`F$E$DI$q97usdBJHF3{W>GCQ=}NNL%0i_#L^s+!i(i}ETz!tp|;E>}##FvzLYNN<=XC)Z+F8B5T=`bjWHq{U2zL@dM`& zfzFqfTIf$$4{iI=nUvgJ`t#`ZqUz=!A-YJKQmyUe&|~%q#0r^^p6rYWOKFGzG$r^<5^yByfyxF4#k+pjfnd5WMvi3 zTOaxEv@6zWZPZq_Y=dmN>jqE?TjE+~kHMQdJ2XE}T0)|Wui z17~*NW<#Lk48d}0mtNa1GRf1g!zNmN9}c;j z`P2*wcOHd@C0O|t30)G;>8EQ)h5&yvEMi>%X z1Wm~B4suCva_vFGjadD&1AX>K6>cUgry45-@gXB~u=jrh!W7b|v@P^@j{kzKRIb9& zITAVb2JRujfQ3t@^FSU#h@~?lZX+|~sZ%eSqfY*rrFs=W{j&!+OKX?0x)q}Hc_Iuh zdvQ=Hdv|AlHH@j(#l2%k392nHZZ^^9`cyc{uis z{|cVqQ}ohE*MzI$my@}`FHbM&T)U=kCs<8;ap=uXDf;{IqpS*w4&aFX%Nd8|6#i7Z zNX50`Jp{v~RpRN)4^JAMi@=#F9X^lS2?(VIaI2UsttLIjXa-4Lsld&HK-mPUV z*eJSO#d9W&&foc04$PpCImS0BB$drFPNF`~9s-zvp>2mw2ZYCDAtCe$r+>e;>`^Fe z?{)0W9nJ^LS!HGltVtf>FM2C?^Bj!b<4&}+sZaAb~)*KGPeiLLJ!&) zH`glKbjCmJVLo9`yghe#*v4d0ieRlefxw8dVLJ;&j(Der37xirHs@EMz9l{Hj@wVs&&ARJ!KM?}F3_6`V7lQnHy-Jgr zUC}r7vh>8rAP<&{(~^@OkhO9?_xJE`-Jw`PC6A*a^gZC!-d_JbyY1-6&rqhpwgc*4 zQXj{^Y~5!vjBjSqv!c1h>;&qO>P+)DNOl%oQKkelCf9M}Z|m_4Vg__|Z+*r;77zPl zLeuPRcyK77*0beG{L2r1Zl5sJWH@*Bhen0qh{h|S>HQPjW@pMClXnQQH<@N`D zihVJZD0}thCJNBYR*#I0C4|i7`}+DYdxN>>w(@)p@nH)sK5i@76FU%^ z^4pc$er>FOqFIvH{LLqM65reR$mEXgo!03s*lsiho{2tv`qWc2yXCiJO=;>@@#JR0 z@aiLg!;8R!1*h;`#)Qz>TA3%O6Lt@ItWyU}8i;Pc#}GZEt5Z*^8116CAE)7FcUipo zlJR)3d1Ns4kYnX?Cf3@4TE<(+Sd$m(a~Q0`{A7{E3nTpe%jMrgcAqwHZU3D4e6sRb zcerqBeT?Vzy+}WmF@AA6BA`@3k!z;0h{@V0N{dqL$B{_m{@gA*U_dKa4t>Wft>2h7M zyEwiPAZPB%o!lJ#c49p4Bv_)${(9A&^R@{q-P5aT-A`DxnHIaGD((2ynW&g4P_%v9 zv<&dMo;1M0JawfofS6y6ntA}JQqF!R_kcHX>((I>Ms~9 zGDNQt=dni{e-8T6Y4%uzf}85l%u8dZk%oy86ZQlLHcuck#Q4 zc!fw3nYWXtc9}mkq2RJ|%*MqH4fYz*meUj<^|Jk)&P$P-#zG;%O<2`sO{(FkLHp|R zbu2l{dHe!l3DgN8MOs=pVax_esaVL0N`7MqTgD|)GP$n=-?`OrXGCpko>VtI`^v_% zCMcz;Az{vMN-&-&rAdqDMp054J-r2B2P78`bXefPcATt87_x!r))&J_>9r7feFY9{ z==OiKpvR}nCpUp$*5rT8ivV+*+?NSJ7D}AWbY;=?3rGCe+5vY)X6>ZM$W};cUV}m7 zuScHkK|gk9oKyb;F`wR=tvmc08=@+fec92`@efRS>2i!->!zorb!;9i?j7&=zWv>g zW(fN-t#~oG(-8Bh>%-rPzZ;4|p$!-Idz~+y6f3=)E3j=nI91Uk^+b2&iE50OU&ab; zNa=JQpg4XJEIKjNL4gvZtR)Q1X&R4{(i|ZpQ`6KWBgC)Dtx-W3W9zx$(vzF{+eA`qVJq%iil{0Q ze10|A$$jIiDETrI=X<4k2Q`<#-XUJsjdbV_MHU)a_{_i!^1qpn6VyEaK& zOfiGkuW!}kC?JrA>tg9IU9V9yZbci1jEnbj788JP*d6*3=;DpW&YqF7?C}K1OM%B@ zv42`U4E(o2X}{_79+%_h$9Z=&J;!MZhaFDLWnwjgze zC+0lZTky;Iw5^9nl5>Mfxj|n1z=C&+tDk@2*iEfX!rIW9#qM zJr_s3K9@i9{?!7n{l9mAzT{=j;@{(-^Xs5l!aSO2XzMuzD1t2YC6+K@;Y70?b-phI zY}_$$Bbs$M3$VeUb&Ryt`6cb3sbj6(p>J(Vz{w1q^GmFkvc0Qr3tla*EfHvXO8n;< zWH=8v9+~@%cWX8mFS@n#b-&ZuP@5p2-aqZa#xfq0(){97&!cwr#iZJoZ3eirJ`3n9 zmt%(p+w*e!jW`nieaEfYv0m3JkmXA8C+@4|$V|6WGNorGYVh1u6pb|#zPO*)cp>Y0T1AbblRj_nRFV+NY^Q+M{+JFfS2aMLAcV{m8C*zrO3^)|_QHeN?{6nwa6N7ufCXb2_h1Tkxc%;lpb}HAY7l| znC963*|!r>YHDjiL1m}sg0IHQoF*J=4Q@SdSoWD4x<8|yImDuY8L=uraO|Z!sjKup zsr#yu);g+RC)$o*S(t{ICW`p70v`6CMhO!!P5)EDtN;D952@AQ-!b<$0O_DFXq4+7 zLwwc%@E3PO=`x5H;}ft$AJThZWUIe$c1g3UBA3H79^3IVVk*Gg7!n-%&E1L7u{;80 z61KmyIM*uo6|?=_(vad3yQ!VW-yWNHJT*_Ny2U9s#{Y+oH7SZac?j2qOVsXx;JOfD zOy}d^3+1}xV>I(#B2l>h-64UV0fzHk9bthN3(yuLj_=GKFYe@$$hy6&;tS^ep=(B! zmMtn@!;Lz3NzR?al*5^cS|rFlA$j$aJ2Wsgm^Fef+tSRF+fg5P23P*p)BL9O)5H!8 zLb9Hs%ag-Ui%nA`m+cd{jNCS{7^%425IYt})t8IIGU|^%;{2)8tc(N8hA8GI-MutX z)tD+qE!GbNy$-5m449?x94&OaR@xx`Z#2>a_3-RShM8BXHxU7eWg%r6dm?%(-o}ya; zsl+lr9@r-T9qssYW~ACdgIXg(#RNxHADkJG%o*D3~CB??_ zjyr^JPF5^S2Mr5qKNBS?5aWBT zz@4@RjUy*_&^yXlvsP{S>L*+DPdaak9cIeH>s|aOC|@;Pc_QDL1$W(`XPSo62i+Z% z6uS7d6lrf+d7}h9&f&DWh9VyR1XPkf zOR++rb*R4$r>M`{*-*MS@ZxG}Xp+&wGK&3~`MbT24YsOeaS4ib4DLjFGv}6>jmz8= z_#gE!jK^K=FZ0V`>3p#c3{Cim`{i%hjfR?ozlM_vCD9?;(84e?S7k{_DE{s?> z`kauUgb^-G!;kLSCb~ipDdP5vVfOUsRM`6m=Hf0kU7SK9cKE*&t?`olFk$R9;!m`f z#p#5c7I;le2T_%ESk080HrOnrALU zU!?mo)_nh!({p`egT+6`LW<1VOo>YQ)xrsi&--+3u~MgFyB6X-%>VBoD@U&H7suaO z@7S9L!GaBm0KV*dU*?JIU3mBwbmZjp&@>ql6(z5t-`tp($=eM3L-)so9PVs2TJ?WN zPyqbQw`bX8M*?!vll(L%uCr`i8`%*eU#^6{azal`>X4;&1rINEkB_YXKB;Yypm;o% zx3JfUcPtThX7a~GD5Lq7a3D8E!9cEd*EgfCSGIZDwz;AjStg`<#4(as1w*+l;BE^w zaZKaPU1Q7PcHj=p5sjr2iQXW%oOFqlcKoVbTz0q9WJER|u2cg&PVd-^-@m?GXqBO$%p*HPt+4rjygs>Sd zV?hk{>4id8zHUpmz=M*P=`Zdmizan$9^~1(zf%6sb?T>tMw9jgPnJQc_ih1TLmvDY znV(D1(LEO&2RL3^>)kgrW_U4Bt@$;#TK^?9aGM9~2hHB(-@7`vQB&pDn*4J8xBWvCi8mUrdy5#DLQQLD3& zVL!G>+|&5Er<0rS7e>6Z+sAl*D9^jErLD{tGjK$g&by1^9S7(L8rDu3H_u3JNfR|_ zvy$wP;2-TgG7{c5X1~TxCG7tySn0DfZOtTHqq3DfxrVk?>6T{uloSEnqGlOgQh|}w z<1g-Bx>q3gEWh_&{t2rFO-{K~kkoFzFYVfH(&MO&%jTbTd8l!4v(<3l3Sv1Ja#Rwv z;HBY0Ll#>;EdZsdi$y<LpLYPg)l0T+9#LBGMkgD|9UM7-lk+ow;*1RVwoZ7PXp2 z47giVja^;KhH*6xMZYt;K3*!tvNn5suYr_e8Ijb6uETt#m$ZX}88$D-9XWyXq?H z>KIU(cIul&d~s;qJ>jD&_i~!GyF=o|dlIVrR9&lkd7I7Hzy9;>b&J!djYXq-7F_3d zea*S)zr_(hWmaU;F4VHOQTBKqEvH<>7!KjX4EIWm@{Ks(XmV9I>EAIjCPam*m&rBE z6r0T59n%r~0Fm!Bn_oFcHzyUlm%8)uN!CfDj;d;;`KiE4b0zAorjA@-(*HeC0tl-u z=g|tKiO&KK25{3KwY(}ImQlps6^qYj6Q=E|z>ZSHh^$y}6T!dOD(o$yzda%lA}O?6 z^F$-Az%7BU8=_lB|Of$ufq7k^O&v^`77LzpnRO?{%)jnEB4<^DOssKlhz%D16Yc zP*uh~nZCSYXcs9ho!cP%9q)NQb(P3&ZZ@v8s^~$zq^nQDMm2PtFZF)4Ne%BP@|EqB z?H|s(vG!oS!A;)l-`IsXI;Kg*`wX=%2S3b94BXV9G%(?vk(K2(+I=v%as#Ro$lC6e ztlL+9|Fz$HB|QK)Ch4ulud{)r=9{T22}g$VkBEm<1_Hx)D2w76-eysmq3@t!0!NBU zYXgD%q5NzrMiPlt$394;v|eiH_asGj=9sq~?qj2_s?d^P$dW`}PAqu2u>}UF1s-c& z8K)Sm$mbs(uakW|)!CYt!!|mPy2den#9CpAjnA*3d`Y=|;UQ0JkG!i-eI#<1omBY7 zG4jxD?rNKdmqdfK`1@YoagjQznjmvex6SxKrq-4}&1!XN-ke&YM@MdqdzZWF=#;P2 z$*rnP-uQSAykoSdG5_rRG#J#>${uHkc^rt^+b^OAl z*ptclnc`te59JC(=^lM?_;L>ov(2fHb8?Y2J#Tv5dfba$RPIwe**Eg$>8BpZG^y%! zTg}bE=dJRmywo5Rw{KqV`{c?RI?VEw`g=pV+V0 zMh_g@X@82O)@ZXfB(rP7LF0XwANG@l!ufWoMEp3G>@6r;vi4v@VWi%YSNhB#PO-ke%?R;Dn`{b1GgJZFZqnH(j;{O3XNmra&;dK3MQ5IH6Xwl$)cf);?u< z>=OakJ%_z5gv|Y#MAb_;?y`&MdGUAW7_b%6^pw_AB{Ql8x1Jppf%2+G>Yr*)Wtow& zUwV4w+o@zgFsdiFBzz*Ns~<~7omXf0wh!d8f8HO2(|AmzTzq`~fyQ?&q-hbIMEz9u z{J7EwulEP#=L-X&iSW9vy#vD%DaqsfH)kP@OQ0}J=N3XvfR8z+-sm1so| zT~Z{WYEvGImyub$QDD5st#kJAGw-ucAINHR21e}qxvzXxNG5x1oUKS`w&U_X*_6hE z&RUm~Ib;mx2EK8;_@Pb;U&(QPy@8?9mh(RBdAA;yyGHtt?w$K0{NX*<`E55&)a|f$ zPEwtfEfL&BiWsa*n%WX(Td=$RW6QP;FGfeHRXq)n8dnPWb3JjV3hXg%p9k>)c*pP zo{Bi{rdzVf2BA}%Yx*aB+|``$uI1&AS(Ld4-B`QU^ku=@!CyO=l$p;`B17lB<{xV3 zoBf#X&bK*nwIPDm+a1)i#jET05lI#Q;HfFwr*S$P_*MEhuF6f{?mtRQ-J&~8o*5CV zECeJ|J}Or3`uTv+pAKP#>-swYGeqQF^w(Jx_tp&A-|)_>kyVbn_*O};#4m)BdaSsq ztIaGZ{}o5R&MKj6%Y4NXj_!!Rxq7_Bbx<_Tdn{L;@`a5%#9Kj*^0n+CuiCnH_efUU zWWfL@JIZPKpndh>63c}??^>Itc5N`RV#T%{_A!R;d}j7s+f=T0M{%!De=2*u#OcG# zPH7}1wU5Jx;;_zi;5~^AtqW$_2 z@9c&k3)QAWim@q;7bOPoaA8z)_fkd!)?pe9FnxCZ{iho&&6zagHBN+1Km!*-ds-{tRds z*0J`Gtmvep&93-%?$uf?LnGgk8)vBp^BW%5GwXM8ju)2ja2Ncrkyf}%S-afBt5rGz zrgcUe`oF5I`PF40{5(?fF(vV0O;YR5ei8QRCsN|5ehu;2>`S5!G`kjC{AyKLL=CF_ zDyBQh3DlGS*yKj%JWP_6lZ=jyc;~>y*SGcin9#31*Iy%W2*AC^EdOXqJ*F*^<7qe7 zadS{&osr`wy)zWH*Iz_Wa=O>|m-wjB^>aH8Du_awSRjG`J-qxhEEJntSgX64M(K?d zskJw!3l~`QUxaNAfld7HXgb|l#{Sn2eZC-`0=a&%Cr7fKKM%SOuz1Xvit!(L8NFP` z7iDmM?CICh25={I$X2gt3y%AxHK}v)mg86XL6T*)5X}(RR{A0`8Z@TkU*tV zua2B@T`g?z7|H&-R=t7WN~1HPjq5}xCv*C){N`0DpgA=)HIel0n4}-XM|*B)gG;Kt zBdDe)cl!oTJvuXkeI!A@#=q2&Om^#iFARAZ52B|a28L#Y^s$XdSZOriLt3&d7jHOw zb~6TFuz)7aBO(O13{?k10Tfjw@`YP5Ec=X06CP=7tX8`h9GrDgyM3Q{TGXjY0sH0_ z`{rlnM>urW+sZHL%6t@DzmQqYE}T8Q|J0k~Eppd`4|(!m_Hlixs?l-eY2%~Mjwc4( z?I(Fk-UN;=W4Bg{uu1CnP+3uY-^)XyreN;gQZJ>Lo6Qk*$*UU>}# z`Yz~6zSX_`;J(zIdo8N-B3t;0#Ob=nQsutBIB$*VEx|qEGQ*D-W)m02 zct&7bZBguY72C_}#|{2uHvawXsZy0%?Z#DsJxda-q9eWUH2U7tp&m2I%jY;Kmc0Y}iwQ6&;|R^JmX?ht=H@ zSMuv?-S!tc)RGQGY{*Ycc$l0zMKt5xuWwkq^tvT~DhnDEpbc=U`{G*;&0=V$7(&eh z`wr(|K`}LTg%Iw>*!h_e7x4Ys-nL!#u9`>m3+LAD&Y5pF+tj$v_k3sOxooj0KNkkA zJ1X|dD#jMNaVTUzuZNu?z#@0On!fSXu2=0@v=@<*>Q!Btr0_z4n>F+Fqol*r`%zwC z^2_E2Z%BL5w1-_^-Cy!@`lsHwR(qhdIi*3lzcu@mom8c@6wXZ7SEc>>oUVnd5>L`| zC3*EKlb`(-7u7eG{{HodxA<2dj?DSMVU6HjO3N+cj(+H=NPahSpk0kOyeBFAe|Tfj z0h3obPaF_GsqB<;<-LXfv}Wwwv*)10dFUo*{9@zviCX zZVE@+omw4PPa0<7Qjj1e^1rYVTI6)wdc&~lWl|Dt5&ZEyUmT# zjkC_2`Ho491w_8mPi5c!cpj$mUl~<**cD?}Mc3GBX3B2Qxx@R#DeMi^L+<^}7w_tdx+ zQelU)Bx=3pCwLYi@!*&(4XVhJShY9}|Kp(HPOUN-y8bYUQPOd*W_mAm%tTay6%EaIoh|&n>nnvdEg_4Ck#7 zzstJ(dBA;WWL*b&ZG)2MeXS-X9?ND|ATSB~GtWzxwquL`2rdvB8+#@De6y!%*X#PK zMkaeky}qm65J{1f^AG2Y{UO!OWzEfv;X|c$w$i;@&EHHAHLD_yT^=%LfUbVfKdwk-7 zcV>adho``PbL;krb2N?Sef%g|Y88GW_gbw7Pj;QMuF_%Ycjunmy(>c5eV8n)IqJ1n znS;@8j z(eZ~r*BI?MDzj$OYTho2>hO%#_Sb{tq&+B1yVUwOGULv{{$y*;NE10> z^psN;51;{*f2ny9X}*tO`tEOdDM$${Te@VgWVDgZ6;1|)iWCL1JnZDpiv3Ks)Jh7 zr<<+iU7F68Zuv-wr#w467uo#RZn3AloLRaY3$unV2VSU`Huj!(a*B+J5ys3Gt^VZR zcNM*EMCK#hLLpzX8uZ(5E-m^BEoD(Ajk!>aigzx1hmQeE@7t%jF=#;Km%Fry^@97lMPm+He6}wT{`ZqjuH`NJ&~7QzdrbfH?#M5 zR|qjK9rfyDF%@~YBiLT#z28>)EJI`H-=w4^`g$QgMirKZLhU85NFD?8^G?!#<}>`= z^C@X^X=x7`)223&`U?7+?h!(0d-mw%8YTT*cWe81*|pb&ZY;;DNN=RT#Kjn_RN9De zma10#REdkRXN#}MslAwMw4t!%>paCbC)4pYVSC*_tu6rk$hCq(9O$2DnqPz zMtR5QC(9Fw&03$%^}OMaeS-d*bgFGy$IGPl+&wpj=mjrFHQ zyFt{DSgHU1{gL{1d6mmPk8O@<6lvb%kThRX^27m06Cs5-mjhpMc7CTyNq+5ar{Jf~>Eu2U6sU1%`$n-n8)DAMtAMj;D0Hc9i$fuXTc_Z&Ddx1*U zLScE`64l-v`KwlCtW#H=@^CMY{<*n)|8(<$ zq;XlfU`nC{t?z;d$qPn0$CX51_l&lax_n6@#479Q)87d9-M0Y{h`I+K6G0IfsmBct zFdyUl3A6DY7ZsMUN^9gsuFqtsCL3HsR)CQzJhXZ$IoE%O6buwm zNFw4hQMs{zjx-YcQZ?tc63a?n1CL`@poJ$a338guTFgf0%DQdlmPK3uUxqWPtnXZAwhd6++#WYlQUoCga;etSoHJ_15!m4bS)e81-s@dj0xCIa%49+Nqrz$;Kp+bKc%@y}iAbBMVy= zz9)*0^Z#G;>?0~ZkF=%w;W{lNc8Y>q*71B{@Ubr|Rpj)~j!&u!IoVuK-g)hKyJbn% zP7oL%GxQ$3)Hm#WdkNYmD3s8nz6xqYIlVB%Kv`LtpbtTdfyl~7A0JK)-`8>La7)(i z`?@mF+brrBrt!82IhRw?KcZ`m8O-rH2--5_#OnH>It-@>$aF*0-$Jnl6IfH>i`mVW zJhzE(Jl3cNlM^)_M!6kh&5P3PVYFVI=7|Fy-TST;(lTX4UiheUSZ&uxg>;`wKkuyK z+9lMka)6G8!roYAdZ>GDH(z z4|?Spc6K$MDZ3buiWMx%WN@XmmWsGe>rs7Uq2&!qm=PeK8m2dznw#6&*aX3t4U09C zF5T5DahBosqs;WR^Ze2#dLw^0LBH>TXWRdx5K+JN=-fDgPo5?l`f9b2UKMln?|b*Z zSDxfElwBi>#*^#H(OvJ?cG@sNdi1G7UYOw}dt7Xu#Z=g|#t9mj{9~Jx^O; zmB$P(|L@oj-v!D=J-Wm|$XyQ@XY!@CoQI<(x+z(5=1Xp<#>ok*^0G7bz`T;MI1bd* zLRLx87vv2;ZEl_dE+~+m_hZOD7K&KTw6(Ka4lWzozFUX+Aw2Yhmt*>;=?( z$#avWCk+GDi(qg>O%K6R#xsX@uMnE23l}fmMr>h$Ae#-dZEd0mO2nncR?r6iiUIe| zU#dKieq0uU_X6`!h);opmQpWbGC^-!DC=u*S}n%8*&J%JF*miBhZM z0(JjO(>N8KIy5y)J`Q%HgOxp_Neoqu>}cQ`m!G`*ao*2#TsHB7pI@ZK?qeFUDHe)` zw2Uto3kE!BFVrM%`h&ypZA0_~ci4puVKp4of0TFI?~X@OH!*qnd)7ZVnSbDe)V&YO z?zx}8>%t!`xbl#Ts%4RdzPSEc-tYsTVk69vau?V03FE_J3z=Jo#mE2t?~T0-IUF2R zl%5e13jLO6a*E+Et9jMrD;oO7Zt17qyMI?oocE5d3@#o~u=MuYVUCCV!g}06Zn&6_ zil@JX%o!e5WrO{sXGYgi9!Uvo6z9c3Mj-q>=Ra#6lLJmX>YqYlye}fw!1`-=H#yhG zysLw}Q}1#M#Y_HP1(#^PBEspPnhPlq$5JD)Z~>l(=z>s&bo=hM=!AC+=l2R7*5;yNgdtPj#3ksw2Tzliot}162tk4k8WO5aI(9`le6Qybo3W zPGKmQ=*mA!v&trJkC_>^tCqGzxE7Y&Bp23BPpiU{fH*rQ=~tT>unmJ5VPo05n%w`Arz34a8F1(Z^mtD>}-%fhRAFvKPlAG zo+~)3=kwm79))3A4`HE8NVdZV1|d^n4IEM8P~hOXCmX!gCqaTtKh<`-GeUQM{w@^I zxZ!z%Z5bm9C~`pq&#itgmU~;3_hgLPAAk%8;F{;=gN9m$E>F`eeV2vR{0k7`nS0oK zovSAt!q9H#{+(*@)==U5R^eLIwQ^zycZ>0~W&Z1Tb1><~QH(q=f$GrbytwPAIirtK z{5CT?ui&}v-Zfip=?WZ|IwEyK`=KFj`!|=+3X?*EaN7%pJbSjtkhL3>jwt>Yv|XEz zj2H61dg35zl?aG%<)-=-hoU8WF-gJYB+?6fFJrf#0e>OdV<&PD>KbepAQ$5^S}3kv z?1K*gAcOzMVJTQ(;F_u}*$96R-tbZHQp7}9AY5I;Sg2<-9eio$b73jttYu-EvM1G& z8hp95%y{`zgCx}Mim_PtGLjQUUKSG*!3Z7CkVx0x7V71B5`?T~G;%oZ*pNr_On~u3SO+|VLNh)--koX*$kI~%%6aK( zJzZUE5spQNdJ%_B=Y~>S)<^l9VT!FpNh*(^&19@YkXk=Sr9`Xj{#NNN>g848G6^a17JIaX) zS^0f1P7NXo%!Tp9c|p9hA`8VQsi{{WKy!ugcC7R1+{tQn{+LQ1$tx~)@Z3hYDZ}Ur zldD$xHY4Nw@-z69;=JMat%d)~bfI3H2FAjv4M$YOD@Vre=sxbBC~<6J7|Ds@Z|9;j z9(*sCE`Lnu*TJ-W9#jye;RO=AL5ns0pc>R#S(!GH}Y1!UG7BdPDK z`uZN|^kCHcO&h+rtT>6SHMTS8g@io7zBv=r_VCA^vNZ>%1l($1FWM#it16k6k(lh1_U*ocZ8aS!K@ zI9{d-VE{zFxg6ecjsMkozc=vT#&?OBau(1aLIado-XNyo*YF+{W5ImrnDD#iaT|yf zc&dJ=1mp>sVymbxN_f>nirk3`e4RR~iwoB24!3TY0{rDhTl@xoGl7o{(BKuPl z`G1Z{yG@B-^^IaQXRW>go6Fl$x3_jjNTDpWSJ06Xc-=RdB76P*+CS9Wv&R%;iHa4# z%KP{4qh?Z{Jb~9A8;$_J6cA0(6@*BZ0DeA&W@JUK{6WJ-I~7=|b-%r}8SI3?i4w~M zFT4W$ZHS&0IL(B&Uq*15utJODZijp`G!9g;O8Zm6p!vgs7}FqpuN(L`h;RdP8H(aG zaRd8bgh8>EFv#c48un-{v*Nr$YP#5dwe37 zDqt$H5`TLx07IxncXtP;e4gRS8QU>Y%c%Uw%!-dTy-6fE$q}qRzzLNW;ZYoGdqN0YaVO8MMOa!wif=v> zkbs47yck%jwuw8HzmN-ol3SFxMv4Jh|vQK?vQKp~x4DZf;8CK1q_2q#xzvjlb$(fbmRT#^m zy+P>4_B3KNSA`FK#tOGA3{Ro_)PD#vkyjts+`7(Yl^w0U!!B{{Gn3tKdz((bq{O^_ zovvU-ee>nZCb7pmKK_vLtEv#+bIMMlM%BJb@qtI&M~=t)R}&bhJNPZnzAihNxAQ}t zgdKT)MZ?aWQg9>918AtMyRV z@pp%1gv2~?Kmo`bUNtO3c!uL{o%7t8vy@i**hb6eN{TLQqC3d8MB&{Ma7Yl~aGAu0 z5x3%>NEufh(%-*mEUjQtdM3Q_01*4tet+SrexbVFw81<&x9N2Wxg&r-W7$pra!m?a zN|s$AjoOhVp;A&@d?c^+1y$$;jhV~K@$vi}0A;BSxvA;R3`9QF1m03BXN7y#;}z5W zyY;06C=-K+rOuv;d2sjc29RUkt+&cvTS(i+`Qj@7Zuxx>Qv|v_6tx^XX2klG^p@fd zvb*2!{BdSW_|vqbuqC{87+?!Ciyc8FEI|v~w-`vWipZeF05`vaECwK^a#0TiDHC{S zJVg-=s|A%3d1LK%1Tmd_vtDco5mHU)=FaA}@T6}A{!HyS54;CI@(aQ&w}EM6^D|=0 z18WGnR9agHutP~Ibkj(@C{fBo&tebJwB0q;#p2{@jKLz&_&SNU>(5uUv-3lHlk~yRqAo`BEcz< zLyF9dQ7(uQ@^j>)pY}lP!IJ&SDRF z>#!6tuSPY4RX(DU&&tlegE^Lw+*gJNQ36=iWqoP~_IC`JTr(pVr3U9;W{qRvF%C-9 zw%n?3m}2lp>M*ZQm1Om}6~DPHka44?2YnM;)%97d{Up|1*s{^_487i?!{3_XbGDj7 zQ{x%(=r}5#Q}Bm(-9YBKv&@Xy)Z%CID=_BV=ZpD6Y{$~ov7*Tbb``0@Wl?TD#YA$WiZ=An#78`bCEu}On523dbKpWlqFhQTT5Ug z+09_ZXl-z*3K0cxn%zunfoej9WQuR$n3u6=uxBMZ=k@L9G}?qUnjX@Bkw1xnx}|Sm;|xB0Bz{zn>ms_JQUY(p%T1h zUU%EP&QNY`Yld;*by7Zb7010TsbZ7+)~SboB^M4%@*wK>p137E$%U5NKt*>)j7-df z4R=2Mt^4@e>z>dm!}bS-*PZsba~^*62YY726z%o1lc{9Y1RD`bj(R&*c>r4;lDu-R zc?PnKkJZ;-kQVd(u`}@-wy8Dx8~M#Y-QQES$lVp$zhW^DA5RIIG)dCnW?wY}s%mCP zK};mqW4DMC4qjlUq9p*O6<-mi9cV-I%|4c)RfDFjPKwQTXCJ${`m}c@`~Z3ZwW=jG zipClT(FaYeJ>+s!MNHo<*H69dD3Mul^wNW-=_oRLdP)_!@?Vn+OD|=lG-tA?VKyyC z)+i5JTkF|Mue(XvjTslk4M8%w@uCgE6dCqT>X@b}3FQt3 zHR!GC?f*IFX1aO{U?HxnL)A++hUS>g=*YXMi}OZ%Y*T?=6;9`ePoHkOGA4iCKVXTb^o)|Txa%!v(9Y<)D4emOdVr;EG zKNN2er_sGQI7FO*A~fqlm;G%zIZ>MG6oOreT+?$nR}M}=>*{3=yXvNXm8qwkO`n}u?z18FpczF%FN!3U^$uEal7obc?LbXY2?(6iFk^V zp_SZY-qY`9X=>V$Ck)6k9Z9F} zKQvM>kQLf0)U>sPF6?y4y=ZgNt!kfsDI#wFJjfx=9W>?+-mgopqhYpF zkC&D_W3%#OBxvib@*Cu?WhlK(*1Xc~~x3Ny#DLVH_n*MDO#;&HkK0BHe z?sAF{@Ai>7M97=I0NJFu`ATohso$Op!*p;9Yn~82(VH`Kjb}!$Zui|6)yK85dIb@m z+gHnX(YZ7d-|Bu=?&KmEDqx%eVngwbx)N=TMD@Lo=TQ{lMP~y0?5)ct%AtIwddnE2 znP!G<<(SF-3VZ3H`tANE{6nnjB{w`sZE38t(tQTj4I@@I`FHed6PuiNPY+HSO*iVyQPC%+aqNN@UH zS`1vh`pug#DXVML_k+Ps3E4-Vd>NlnH~T1q6~?CKJfyIkmoKV+O~@lvbUCeSiLq~LaQKWDs{49=Z6oUp!0of(>*C6t{%5N4*7Ac zJ*sR|EXBo)Ldna?aru0}%N*@A#w4gGdTiIvh!7AM5kP95?D1hKEQ$mDS#HeY-!Co- z?nY?QW}2GV5Jr879*A2A-FV9M!HWjZPS7)uR?$g;jq#_+Mmd7Gj&2*L9Bj#k^ z0B}}8*Avx`i;6SZtM?fRX1kau1&Yz?&F&>0Jx!`ppa8kq(bpY zZTJQf%`Bl$SwvyB?c*E!3}6+~OlA0&uVU1*f|dGnN9VqJ9I2(ac`x=-#V+)Ekn3K3eh{8zE~ z>l3$7`WD|!HIi#^lzb4prt0!BjwcSE86H74uiTyT-ssqc;7+~&C9t-8)aF8n*Zsp% zH`4+ep7o3txjH3`tsc=0v-#<~z${03=7yRMLWXD}@ZpXjSkNktKBYUm-k?yOPB|Ta ze8SU?nYeAbe$VV#GCqSQ@Akp`-48CSCLo%jaFz&kJ~cBU_Ck&MZApDwP8?3&Lkq=|3Tw8;0yplpIYzq} z<5r9|vrvuARDv%s_x%DbeV$!ySd1w^nhwAU8l4gHtBL8^w%^212%-@khSo1Oy%2Wa zqw37qo&GD|!@&2&v5-b5Tr%jL62(u*KG0)P{Fo}7cb-H^bQ};TSX2~cj&y2JV{(OA z6dIC)#>N0hZygQ=jIcC_4I*SX)jQSI^`rz^xi~zBT6g^*<&-EA2-3TAsImSO$$-6* z%`&k{!Av4uI>TF4ceTD^9<&iS=K^p+X&DMxLeyfJCYR3iAQ=!MlDaaecK$Fs!1l-R zLN4YxmF;E}QVW8Lyu3V!%s}=(dn6di*=epd2`q0I_>S(0(%1r5j1#g#NK$>y3oK{M z6}sOy$`k8%CsXxZ!Fa(NM=G3eZ|Q8Unbyr-EJ@p#1s$!j_jJ2Qy@!8K!Aoj>)=fnm z7&BG%61C%B@Pdagn>24{ctUrJ@+y0x)Tk`}I@N(B@^dgf8h{AzA&IUdS<>hV9yIWlQ-B{*2X8eGDiNCsikGi#YQSB-t&15o&&*!W#;9qwo8EE zPU~AUA2tl2>7IO9Zj}vFxG`nQA^pG3l{V>2c&mdbq_aD|7g}uJ_VJAxw6(Xl=XPXF z_G6lkN6o2TeV~TKjkINI9l3|(pd|Kk_Kek!5r38^s;oDIe}mY|>n+=DsnwwL(rUe8 zY`nd~rw>DW=d7YXdPiDl)VzPcqxM+y>5{UyU%q%8y}I<{`H-^u)aDUeM~|gN=C4zR zjmqgY_PYlPXhvzqCa1De97K~{LhTif%H5I?SSfhpV8ok;%hP!YKOhTgPYgx^| zwY|STd6rdu`SK+y)W0@dV|*ha7}b8_vlF%B2aP_z8^a&kUvvZ6YT|^9@y%Ci7BT6@ zSnXW+y?$XcD20R(7*+rBj}Pl$?GO$1V^8zi7ALjSgf`y;|LFm9Z|;iq^WWBYx+lP7 z@^$V*p(fi;iJEog3zG|-Wpl(g$L#YSFT+&9$*}qnqDsC3Z|Phd8tsu+{c&=LH(*F`7Mi@2sqmNTM5a zT<c~+pG z{JtdcF3E+MS}KV~N)TifYDa;2CCghKwNFO}lGEpr$?$he<}^N@Z+LJ4xKwN-V*;n# zD!VYHA<#x-oa%S8(tPErroDw*ogX}zSxy=X#(0`}$2Dc2QQVW(F+*eF z=$f5PS1VM1Ge*4#zyOQUS8ID;^^JX-@Me}by=a;<3150?U6bjd1=Qk^rc1s9#KN44 zdJ@YM7?plQ>J~H(zn3qG+^j2e02sGgKg5SoCmxo$^XSC0TtV^^hqisgV9NAhFoq$7 zT227UTvLUD5m(cW4G4vZZD7ZI9lnMx#Pqb4&DpcTrV6`RJztb`+Y)S~LoF09c=7|# zLj?kh0wAKS%gGE?sAcbEwb-%|;U5(A-r87#g9Vlxh%Tr!fJFcCPlQ^C9VQBhdsw!M zver%Sc3fn3;+?PH2ti-C_kkKQXYJ7c2W=kZ#MMTCv1})R!soIyF{`u64yQ4}cz&gC zlBfC%+bfuH#C`mcypZL_&k<6Y6?D>M|2JfKB2ctm^2w(;VGudtJ(m>jCCd~}y5Ny1 zq{f^u%R>Q_X>v!a(&@Xpw~x+^B2yw)1es`PdMdZIHl2-o@>1Ek>pVKN;WZ44NDsv~ za$3DCFPAGXG2h>TC*;$$?DhT!Cz7>tO_NODpM1NvRdy{fK#JR}4T+>TZQj@HeSN7c z;^=OMhg|Ot$;8A!dUzc}S^!cT;q+;-D1PqcO!BktdCaT`uS-g)GeD7rv4vU9sS683 ziGTQ-?O~YRZswmrVCbt`n62~V>9Vr0IEN4ci`QF(vNWZ40y0+MP7PRwJjkr^ga27C zVRxw)LeJD1gtG1Ylb!%Dst8sUYZra)W}aE*FIb0GP&&p&Kzfow;#_ZcbJ`)FqGWK7MhUFbd|UXsT2kzPmaA1QOU)%Ci~ zZeOTfcz)qwWG~#&2zeM>_b@`&zRlr? zjQG}AC5ovBklIHGbJFd81IIk`H)DnU4l%wPE*4v!4Q_?IWexbDm7^&UOPdx_kA zE~;CmHx3DYZep4k7n-Eoac)>jIa}HPGLv?w?@|X@S_&ftl*fM0ZC||V?Dsxx$(FG_ z6c>Bt5|gD%0s%y#6IzmJ@+(OAx3h-|7B0ZM<}~I_4^m@hEjRfRDD0kJDi)M)d1ypT z&$@-NXZ8MT@n}pF>%yNn;F5D!mw5=m5cm@NWH z!<@`k_ytcDm?U?I9RL#(P{vVE{DC~&RrN?B?WZP}UJ@_Kq9tqq>SY%oBh0ltYrAD% ze|^f`8r_+l=Yb|FhXwgx-CwD%h{+PBW!N*lb+~J63O6(B=y{I>2#2ILIGoy^cY6xm z55cVgv<*h$9RS|M8o|3eTMz(13B71Baz+l=qM?${XQSa?XobJAJ!AXl}1 z__G~9=MERp=im8^v)(cothYWVuZ0vg?~!A8-K*)55s3I}PyLhnmPcl-ErZj=I*NcN zYfs1tJ#9==5+lY23Hr4f&hCGYjMkGos>aUwnPu)Sxy&98m{=8z6SU@3hR3D{U{|l; zV0N3BY)r4W_3w6}V%LUeWC7gj51MnS3qoMYeLf((zdZ;=Iv{>;@9CuQO`SSa<9W@1 z`B_u9eE<-4y`7lqOYFm4Aqbc;*}is;aix-D&|ixrBogbonvm8PerwK0D-W6Gm z&7Xmt2FtgjQ(@gXW#Yvkyr^_$`b+HaQP1eAE{fnLtX&rSl{S_Cipl;7YeweuER%H? zqq#CTQc1p_4CCJbj^{P@^UEF0Dr1ywB^2_5=QDXh)$vqVS25^C+WUFNgp*x97t1X( z+7Bnn8#^jWItH^y(|7A>nT)iwsO2p4Gff#DZT>U%JA#BmFG+7UUCCzBO%N{BR-)q#1%UP%?vxFM~DVq!=!d_aFA5lHSNLmdTE$V5@R~b z6-ad+U~!a>ePiHJAHgLOh`^EtvwGJ-Luk8!k)T#gj9kIPJ5O-`0Yrij1(RKCQ@_d* zU}=(mI{_;ft)=dFRD*(Fx zX42=KW-il6+{AUD1r3oh-psw-9PNUWkDw7SekhNM2We4i#PBI*YF z=ySuKo>FTGm(IU1Sq@a^z2qOOXeqktr9Y7`1B7Of%? zAob}k7lv?2~icwCgcn-I>Ow=nbSN;XoHm&ZR^IVg2WXGiQ{KxXvNma9V8=EM4 zsSUtMX@Yw(dypoo_@OxM%by9$3UC%nZ|`qova;SSUlbD=AAeCwMh#CmH`-PX5jPNi ze?9nz)_gZ(a*il@kwqylqsu2tM!?Qb7rrp)c)7jcs@LSLr$!sA{c5-@ zzIsR$($py~m&dL9$ebF7QfZ8W4zs}Mc?l+kttuqbgumB5<_FW*Rfk>Skh~4uQB?k? z`}j1gX4S^fB0mH@kXhi3r2*`@LzrlxfNK@fBxp5YXJHnGuXj^o9XsZ}gu^>1FUywt zQ;&VjjM@z33r0S!I5k+71IPfnH0XA~ODW1r1OE&YFlB(F+v2ty94(hX!o@1vvzaXb zEG30Co3SUdb7vJnG0 z;@NnPigDqwE4CpP6l}&wuUVU&>&4_sudM~YN-4f7izCeDY}}h<`Rnye3Z4b#FaO5I zGRfUQM-Wgz76J)_m`{$XV7^~7o^g&H(StMn#Iv8V0aLNFy=UehONqT)b-HqnaPy3k zVuO4fjFu__Ch2pY6qif5Cy!&T0BRE641R|yF3QDp6Xp&Ce#WE`9W9h3aj;GK6)G?d z>a}FQdmx!VT6127*k71JBY25EkBfOD^?~=r8qez&^Gh~AoyRK-Ms;1*YapUGk;k?f zpIJZ~oM-Dxb&5u>3oqarV&5S*G@K}#=~D^xR?qw?CVJY4+zm`IkIg{r)8-cJ4_yO< zauzCW^co$iBxEWI)m_TXGO;`|pq~CKanI#2tqmlT*K?q1bV^8*+vOq8+~P4;kll=q zyaVd9tOw}Yh-u2AI}<_aZqDr*vrsI|?t9->M)i;eGsJiBWFmHRBc=aZba3(X3wWRz z;D6}hXRDcic3$}O6DCb9`zHP+o{8G+s~P(CGr2`YMYaLI@XY24=(dHI$YgR|l!8z0 z?Nt<7_zdOknxeFk(~RQg-SeoG^)pSxj`3fa>Wy{2Yams+j`&0FZuAG$QvsG*tkn84@Cxwxu=@{4_%A z8S`401Km1I(CATax)fU!#;o|^R{THuIhM+6dqquj_ZJDI#(dB56F0FgYjmi={-TQtP%vyY=DmFRfuKEDSiC4a`1phkE;O;l zVwGJ(pYaKZhO=y-)`pzT5fy_ZBe@v+LK>^Dm$`#jl+JJbQHpXIZ7B6Xh%bxFbpiP7 zA66|UI>dPTnGsx{SQ=G>x+a+>AaZn1G}mFftO^76cP=8%cI=l^wCOE_=Uj@qm8+NJ zl?bWNJ~-s6V)->qnB&Q#ltp@jIE9=YYr0o|$GzM1nQrFez~_y74%90d>9LEm-6boR zR7KPAx7?S6w+ewe^sQ5jOXKh^E+#%fml<)soKN}L@^0`WAe>?ka$DhOQ z#2HoNwvT(GIUByUQzHYGpUjh*X#-j$a znap26@_qrxjnnF)rWy%bvZ?j>Ca6r%moBgv5gW= zMYhd_o|Y;KtUE?N@++19g_nZwq5X9PkUuiz-$^d~9&8x!J3K&zWkz(azsFNGOMCWA z5n|tdL(}A-kraer!nG;jCue~5$d^P43O4e#dUtaA&TCFIu$>5wB)%Yeged%kq%q4Me@hn66n1nd##MF~L= zbJ-R^d08*hop9dSMAF;Bj^j@)nO|IBO$CrffkrVDLb_FF)IqlTD1H@&KgV+O2 z%J3?@9zfY;;!wDFE)Wi;*?Y(XVE0!|;5IrS%q7h--(WFEBQY9MM%d8Wd5_!SU}bsS z>qdv<37-B518;TlIgRU88->@`@Ic+-!sLh#tpNO2jd-P}nqJv$FY?TCB^K<}%7BeaZ{7wSM5LL`qw=AJN(c`{7Kc zOA2`zVHY;EaQwmEl`mtgb58YzU{g$jHp`^o62_Z!(t|YO$lj6+^jgXZIt?!*dNhctaAq5Bn$x6#?2D0Q;q-+$ zdf^>1Q!}h zsAWBa5@MDq#gW}yi>L@_4>%!)*qBFPSq7^dE0-+=hXPAC*dM@zBD)!MV9Y&`fAa*n zS$Vn@8hfxK1aTJ(S8$Vv9j?1B-(D>T3l$A0!IMKLg=qw4l=wmycN{^g!XgSNmslwq zE;5U^5&1*U$3C}S18gSnO850s?|%^L)&OFL<`wV6b?TQL-UPZtN6`kQzx3{a5e=OT zvrNQz(8Dqmwyi#4>>F=9JE8YVF7#piGBKW4A7%W**~6<`Kc{cyz57u1H~SJ_|Al>g z2N%P7`=+VP?Yt1Z{j9_TAV1vZZ-0`IC`a!hTU}kfT-PBLFj>wfYbEFCYb2&x$Qj74 z!Jl~{8hN0xv10IR2V;Vb_zYz7b2NR)23@dlq?|%42B<+M=IE`(&6nFP0ToJmJ&AL9 z0T7}Yjg*+wJAQmD8t9rv|34|(cyxQ2UG+OH;Coe&#z_-}%r zT`q2K83z$ZnMa0x=_M`XJx+DY1ddDg6|AD_Cu*ph6bC5hn=!*^eLC~?n$w2}M|JkF zC5^=vWICg?gQb1rY;KXQ6?X3&XB^7|vOjUs)Rus%_`Tqb*B1XCWq;c<4Nd`{hR?;9 zCBP@KWg>BYJk_>2T0*U01JrJ>r5qDqmrs9mi#gV*G0c1?Tlj^!Y0ZsDOHJSt z+qQ7b4gjZX8CxI-fb^BT;jfwQD+tit*=vDCKdDU1Ug>H;5T-VgZ%M5|_*J{ncn@37 zs3r)SQER?Jv2PTpYQ*l<_i8C}Z2S^-#LH@gDReoDt*sC?BRqwEYD~AD^$;R3Wduv? zdZW}om1XhmH+#*>(Mi>g|P68zKPA zk+?xn1Y`2VGi{*J2C@cX} zGVQllC-G61pnOb3rd`C$EQpnugjyP-2(K;cwtO>v{|=q}XF=kc+OBEp*_j2(XyjoS zN^eC?m2Vc)?)~q?#IB}O&2N7^|L(>xQhu*Q+vZKcTe12VTHcN5Qv*7;GCV!8jEUrf z0S1KHAaf2JB8!8VWa{d+B=BTpaeyY9-Ar&Jd);}cX%~N)4>1v0eFKSjNc8B04>N9g zVqzagQvWXF-`LqJ)!LQNXEMn^j8IO13wt{hwD!G2^7T~etWWw?fAj|%F=;OAli&?^ zR{dz7vrB8QBzI=(aTDKg4qKn46x6ml1K~8>ZK?mc=0*-}LBgdEnxb7y8(c zb|xEx(!v)Xt%woTWDjI0h=RiF*jP&~?eyq^Nx?m=RljuV5$v|Qy>n>s*>+og7*a2zlJL8~r4a^<%=2#~5Byu@M_P8z;eY;|C4f-r^?Wx3MYc+ss0>61W0 zQ6Av*q8@9%sfY$|y2n=YnpvM}kIYX8_*^_hJ@qFHCtQ*1q3q2_W z;1cwE#-_fXyT?C?lKD`n)t1;=pv2W!{=$z`k;uP`kZQ5b35ja(oM{qVUzHMkleM+$ zuNBa$?;JsUnQM9z=NPEdfP34_jEz9Ux2K}up8R=I^N-Tx^kY{+FW;H50s?5y5 z*(29?BEWBg82NrET$-Lsv3O`Cmut9{PjG;0soSvkC;eGUIn^9HUp`n9HIOA_I)UgA z-1zfYxH=*u2sjL_Z9Cay8H}JVEGv|{*jh+$)fVoSF>AyxA>?&KzcQgrgc`@`t)7R z_G4b6#73DVI_eb|;vF z5Hrjzhac}mw6qcVdTH0MVh=zLuD$<{sW*YAdi}n~O`$@RWR{_Fk%*9?A~f9wT|~wx zNit87xy*4JG)aXS{t z@Cmwe=Z-a2!TM+hv@uaKroLm03a~(qFU2^m*--^^{CJD}{=A~YsAqr(K~yCBrA9v= z&`8TMDW{T(!R?}n5lpWf6Qw<&VTbAgcLhU)M3to}3V{KtM;yr;?@mKUjngq&98Dhi zkSD?W=d(&lBFr#e4J5@Hx{RkIAp#`g4y9#BU3v3@J-C%^LzHn%Hb>809jQ4505LFy zpcgO)5mYb;8C^~9BL9K(B=yHium|J`6v%eVe8_UN2pr0KdX6PTn+`T_?0>2j@IyPQ z8ge8X@p8{U;C4$;5p}>;bPsonrT#iG4Bsdp#sDj@9HJVu9C+m@rnV1qyzqFxt446u z735#Q%)!$*RpE6`?^u^w11GI0#_@LIZ@cN-jm&7F`9GPCme8s)^1{pt z!8x@~bC>{Bx+NuvI3EXp~!#Gttg^ht)~-HvX)flX+| zg6%Y~V$@t{V`Ekm$OA*jP@@4b5KxFdT}pEWOt5>e>ZfR^^P?w+3JfpbDL)!ky@{~y zgo4CfZXocdZW8dXASYrv5H|FgS#6T&60o!2?ei%S#mJ=ACn)bSt;&x8l(agyg8N70 z1e~#GGq`zby{ewUaQ#3IU6 zEjSD3(|fI;g4w|jH!AMvEdyTt_IW2URf4OZ)3F1~C*EadUf=egz2hhQ9cHgIF8C(O z1L0Q<(C6i-({%He5&0{SC!nMts=*LB5(XL@fu2E(ATnW95vDBS)}QP=5aFEwFoTGO zluQW1l3BPQ`Y}9Gn+H!DoPTC#$~#sJ=+L=1!eCcgk3CWi_6Zl{d{Ax#O~5U&iALuD zPi+07PHQ;I_2r}GMP=J_|-5C#iO8h7ontYyGBneAg z8X8@d$RkK;{qZL(ds$=1=pxd-tw5H+xKC1h0;;QwnVyV{ZuBsGsgU zz1ViUm9{2S9!d?wF^EnPiH^U39~4rL1O~WPvb`{T&*f9lIp_X6=5XuK^>}zTSxslz z2KT+;SsN*J(Of@Naoe0g*-EzH^=MFM|1rr4 z_UZ2w{aL>7>uBJw7PD1rEl-{-+R|%oVuE!^F+G)Ra?zpS-5wlXJ~?~2B&qX6p27@3K^<)#9Y+>eCM*OQ(uDmAhS1!Z~Z9r z`P0HqtynC_NTN8_;ZDqsevec*Z;3u86vAkd6~KT72Oh0Xc&1_A5&C67K-?U!$<_It zr76jQL~$)m=15V7L2U54UxSP{eLTsy1kE@mc?!+@Cfls~@uq_t6W|w>y7(oSjVuO& zuyA5_xON2-Biui_~Ha%^mx!Cd8q24By{J1tSQYKp=X%< zZbYPIX<23OThmV9w0fzAW6YBbsOEj2qf7w?U)tsDZBihR-+AH0jyB~H5!awUBSJq8 zO%hc+G|$8<{?jMS!c0q5THjp%6AHt@={d;P(d=q{^lK(FxkraVd;TW}gOgV>XW0R)L>)^63T)5)sb#<~;?upw2=+hxgZW;Oa zoVP;EqSlf{sa4pi;hB*UlvrP-jZZKS;b#VkfiwbKhv*fe&Jf$) z8gLL1{wrBSk=QZbkKYLgX>dAqhcTyQ~3T{Dkj@97libN0O{SgHp zko>bBpY2*vL8jD?=9--d-}3O%zg(ZdCJBQlFfaA>bi9Y@bK!R1_0iSLN4pDzNXL}s zQ@Yr;G#OZGThx4n*Et-ml&tAfX{|*)3#jFs>wN;DBpQFSdRu!NS};prt>&Zsykh<5 zCa%?gbNc1$nBVC<S%*nStOq3eb#o`s*Vac8w#p(8bGNFA7$_}fDLijj{1=WB7Ez1qzRj0)2R5)zg zCMnT7xGF}UyGVYDY=Xu`P_Jqj7Jg!L z%|`L4`0Z!M-v1@DUV8l@$D8jzIv5pE{?X3jdO7Fp90VK!REIhj0@{vIYwLw6KoLCT zV%yY~LE%FUe{->YP1GskiBkXMnF@P2ODNu&mBejYe*S~GekHYp*x=O9&DJ0;zL+8h@?GqI^vV(hF7nWKHZE^HUHywIgvzx>PORmz4BpO z2t_cJ%r@$+E`rhzWRrdhF#6QHy+%f`D+ZVZ1qHN+%I{6j%Tlhnoy~U|?PiXC==iI9}jLPGUcRfhHg+eF5f@C zNZ2*{?Y0$QQ-e~|@Etq!I@2BI5&oUkY4|UO&vy~yZbd%jMZ_k$mS`v&n4=bE=UQ6+ zZ6VVXr|I@zmYDq4Pf^TE^oB)YRUx8^;RMG*s5%GAXaP5M~mI$G3Yhw}frbW7(U?en9 zRF=ieW^A(u7Il~#M<=u*bHMpt-=}vqWOhxQFM8zShc*-$Y>&X$gWV&72GHTH4zOG! zgM@}MUSHBr@q>fV2NHz=;b@`x&U|el(zdEuYWWA6gTeMxo>T=>vgw?-teBdsD z3m`S1{OD4KAt2_g;HC-Ju-9o}Z4Ns!%?owPsr~SOd)#bKxPG)|-jDaz5?3NjlyZfS zyk)d#YxeX{PxQ_0SQbZ2zh1cUPAdLj^$~d1bH7x>HQP1EWccd7{UA1N z*vzVP?O*-igM~WJI%@`|*Z2+;W3UL75PEDqUzxrgB+Wcz9x_awc%*YfTJg|=$=ws| zhmf6{7KfX>x)uow8u;tz(hFhRwQTy=SB;9HmCcqneDu=Sb+(MHq@P1Ha85HFj{K~| z#IVL`4wjBZwnap2Adu!0kb6-mEc?8|j1v?*e~^x4M{PB{jvA27>dYQGUL@fVvpk;Ni(ch${|3)YtBVn?%%3CT-VAH`yw3 zzqibdcQF=Am1_cAsyxYjPuKXDBLG>bY-sHCU~QA0a~LW(1rCin$6J)q$T5t>JMN8m z+2evb1;fM1s6J6abamYa6OvVXj+plo-yEQvD0EVZpg{qA0FHlunL&#(hGu{m00zHK z&tZGsCNQ4UbJIHl5ZGt2+LK67F(CYEx&;zmr%j?`Y#$~m<)4eq<^BBqN=2b(LHPmn z0COx<*I+r?h7`$4;Pv4x!BpB=oU+1>op$|p7_YF!Z_r+m^a-5%0Wpqf92>e~h<||A zX8}B*GXnPywr9kvmTB74VvA_p4Y)XNCZ@+87SE2q;EvQq5`6*{3GxaiY{`T@vf6KN z2+#z;*rnu&p?=7?x4i@T`mL16N1Sy35R|8_nmzi6XJG#0a z)7M2KE?_JH+lbJzCX5p1y~QRbd`sE1{@XhQ!|{2srP`%Rz$m;t;z&FH$$8FWT<+`EELjg`{YL=sl--%m^2cOh1r4mIM$ctI6 zb&NQ86PT-Ozw;37U5ijh@3hp#4Vc=zG~X*mbuUW^{-_Fb64JM=CGy$1yrTg>Zd{tJ zx9V0JD}*{=3)F+Q`K376rLvu_bH5&Fcsn@?C2C%DYpT^k$u;F0IM+$db6#^rN0md9 znl+LrRR*>A@N@mZawU#XSoDD$A&BL5|IonQvkwZ-?F%6-TCqbqv&6?yY-wDKcK7rL zr+jb6CZKRPs_hck=f<}3#Yu~YLlkwk`INw)abAf{Bok%Bq^s~NspF>q{65>Vd!a>S z?%_x;85zpW9=V8CvIVtj`ppze0ue3Y9yqj3&-|9^ci(bdIetWvGW+tOy}})yn3@WT5v?o&B0zOOBOu4u3$LJ=OJx?! zjTuv&5C8xvgC_>x0_y2E|Jv9HAMR%5k*C2^NEC2ThMOkTq~MGjr&5epyFfgoDS8&t zSllO|8u&Cr=DPWYxL_pOOHdBKJhBIy>%rfi%34C>0`tdcq@l$|6=I!oxEEG62tEv4 zCuxgZCu?IcpSEkDq-`dHixbuySXdw@Xe*&qnVnL*7&x`4S#WKlY~2!dkLrg4My8F7 zq8oSK4okMv4?ml}|JvD{sFAl*Gt;e39oYwq-#cx}b1-`1(`%Ah_s#S8uH)Sq?cXSG zjSd{}ieGp8^*>IZyv}_4aq`{fsrFmxW`d2CJ1+(mM9i@Y`p++Z@mRfar?8uR0|#}R z5K~dJy(Wc4VW4IrU>9@?aG%1^)<9ipJfm(xL~t-820}PEIIwL+hnodsf22{Zd2uZQ z@Eedbz?=v((BwkVuBf6ypL@Y=z#{}v+n#O9h7dSVH*ltu@IEgubgC$w-lKb>rpLwF z#)j$MQ;hW3Vq^=ChkQJ10qclw4wPp%HtD-PBo9_19gg=>*y8KYKr9UKkGkuWj}Je5 zckT2-?LGLL?cdHnDB1q@Ev-S9{>!5<#`d5#_pPRmj$2hD_nvAS8C}*1<&V-;)YOcM z)Nv|Itvt>ed1Ye%GmH|hf-M*Me#(M#Y+4e?r+Ar_Clq{J=-s_233vEz zB^u^h&K3k^rV&+;&vY^Ms)&ZyONjuOv{&-NNQb{7N%{fW&R+* zP~E9vWE&eq#U^7;bER99GMFqjei`ej(n_f#^JdTzZfNj4n`=r4VV|Ll{Dx~?RG7b} z@BA@;P&SR9q2jNjT315Z)_ko9W(GaK2h=Qant+(l!RXn9|J;;?+v#781ENL&XP%8r z%CLH{tt%3nMn`)f_0`bg9F1m5d(YFo8^T=k_>EPsy&%iX9hsXNf!MA6qx~d?_4v8O zc3pb+4eDXP)PoH5;-UDWJz&Db@K!U(h4265YXJ|_OJA79+K(l~^plCSu?N#7Y{lNR z8Z=^XZji;IHdy+cYMmQ_Wr2yi1(tXamfBB8mFS<~7WHYu^1Xb6wG3IrZD+lV~?`iE6gDd_oXeMSd{Qw)h!{yGuhu!<~BK0%2lqUXmHl z5#i)D{KsBhU5s~3`icPLz!sYvx>kpmCUVHXkmup)V{8`KG~I5GuIh^DY8zt#{0<^K z6j5HqBDl9;u_;a44_sGJQ)7>-4Yert!ZA~Ng<-c-xgDSUOVDxu^5v1~O8lw2hktlg zc2AQ*N>!@C4g?qs=Gx#fM(9eosEKgvo}x?&41xBxJHurA-*zLTq}1;})NnuKZ^j_)!9xuO$kX~-{o+rm^I$Eks z?r6l}hy8OiEOs&S9)n5k2kVqmqo>lYe>n&f^k(~?hCBY2dMB(9=KH#Ha?(S(*r3Jlgcrf0`0LH(H!UZA{)8loVoC|IK66(ph)D@!tLWZ>{F5tw?$; z>9~8*$%zZpk(Oj~)?@+cd!0AlrZA8CWfvG*U-eU2`J-gs8wWvX6+e4OkTcTe%&)fc z?v}d+#gf0OhdcAv5ICIqd6#lJhap2nWhh&l|DSSvX6}o2`jzD$^D?{ID0BVI`@~BB zU}B}HoE&5_2}leWb$@84`d{|H8_LF3kUH%9tQquOKd(0xJ5zTN=berZ{XZZ`D#PH~ zop=p0RN`qh9Z)Vt#!TBRrjz!DW4MCQ2+jBNuTAGrX|-o}&vh8hJ-5uVYA&OB6&fjF zprYA`*8l8bfN$d&tVku$S$?Q6B2Q-OrntoCHy##Hk0QV#-jb0y$n&tzBGj#rBdv86 zTxa1@SL!!y@|WE+itk!FNx%##sNoX55w~N2AM3{K`n!DvMWhf;ctTPr=o&_ zi|G(%y>jGW#SBC%`sDb^WN8~07j6`1c!V!I9B*?32p9=S9Vn7N<9s4EbMsF5rJWY? zw*}Zy>GKOkqs7XHw*w&$C6UhSlPIP2(^{Drl%7`{|DvnPWx(4rF^d`)^uv#DL1#ZP>^$!&41hG{G54QGEOsuY>U2qY3r(xmv(1IUXyWy zm58h$8CANRw7JJH5A{_Q>Lp)wBdZv^@$m6*g=(I;?LSOcX-%hp{)d^3%iK-KIQGsO zb4{O@rog2$H+Ik7;2e2qR*kg)I~|tLfqyEtjh#uamOsKZuLHG-$IE5o<*443-QwRQ# z4fHXx2fR;_2$H4c*)r3a(wX)a{e~YGOfrn&KsmF(X!1y$f0fRl$Hw~YuuXxN4yt;5 zKxg}Rqa7T%8y!+=X6ph(yjyCBuf%QhK8$g$T%?rGc`rD9zqMocZmN<$)qmRJc4eUb z`BTroe3~3Q8u+=S^xDfiR2uDKgT4$uYM@EYzT+YhB3x|bFu@KRnp^Z1zJj`BBz~I1 z-K9c!PRMIR##>F^4Wut%WhDp9PhcHfjqc3FrC??@7h37`y`hr(7bxX{y935T#Wvoq z$F^!#NUK}jIk>nijn7Ur!hYfC(N4xM-G19FA{W4XR+51fcV8EmvFo(i_(1C9RLkPR z+&Ic>Yy4XPM4czciJLw05*WY2F7y^FAMAOnuaTH_sjKy++uSFloW?z2RXqg`iQk?* zdp17e94J(1&Nc2M48yZ6Eb2RUTp8)ix1AoP2>pt&9P+0@bcyL!Hbe__e@vQwtr1oW z0-<=hs^{>6&=>qIygCH#{>eIfsjR=0g8?v+oWhG#i$SV&uFAwc^h9Q+^2@IhQyc)l z^P?%F-&YHro3`uTz2W?XeyWY*RPKZ4V(l6?BkWhh3l2A$n7ml}jqJb-FJFfv9}NDZ zk{KA34~M3vl9L4bQ9)VxGD7v)zu2zm>b|;|c!Ukqy5`Gme9=%#a?yHd!O8U=OrsPS=ENTXw7V&uZB zLd0k0)6;d9k7(Ni*OILf5>nCC9P@Jv0r-#5hRD6Aooxb|RR=*6p)O}Y&v9e*$ zjC|UbEn6@|cu{Z2>P0c-_C!(1t-#R6GCLM+h5Vl5bM}%WW%n->-*203o#)t8WX!Ly z5$Vj(2;&pbx-(%YKN^$!VXI?!91M-FWDcQjC(}wWrwse|uCA`fnbW5Fczwt%Dhb#& z7esN|w~<>R?HvG2Zs=%6s$XY}K|6q>)PXk@6%0@@@#+Ov+hDfU=88yi!k+25&v5x} z^8}TVYx53h3*3d*Xji0%IvFYhuK_6!hOHzY)+d?&k!1Yhk*TvztEatxJC~#h3g)C# znbW55GF=P1bRKpVb~mdPS&q;au}CWlZ6Ohbae@ihRMsW zL4965{HO?y;5kvh;_b)a6^f!<8_X!^lv?g9ri97W_syyv9;$6t&A?_4-wkC;3pA^r zi+eT(q3-xueAnzVXfeO1vz;;)nfOCCu}j=E-?n0f?c3iRqkm5e)nWg=y05NaF$CCq zKt8;@gHM=DZ=Fb#!p13tL|iT$G60ALCv~3YtyvT&q4^E_#}jGNj-j$gkZoq}QTcm&0Lt3uNy`b|&yjkk+93L<2Q0JDZ$yYYz|J zs;VkPI@}WgcJ%JK^_u5nl$iMQ-F${mDp@k$dwREZ^D-JO5=f*u&sM@^N8T}|(M~B} zfB|o~94QCMhoKDWOQ2CCeS8K0IGMp}dMjzjx9Q3+Xa>-(Mk>p=xD>i(SKX01Res~J zXZ5bg!(YngMgd95_ppz)sR+Vn%%0+ACLTFUxlsv_WSIp~RJO7_PqQZ)8b}4O$He9e zwzSr$o}N>05Yf7wo*Jf2~pciH+Bhl%Y zlsRNy#UwB=_dE(vxzzz3@JGj_zjPJ4qXGvUd5hEqC#NeIkhZ~g8ax{`YF&!8{+7}} z1Y*&)-V&6Cb8RTQ9_=@en@7qMP#)ozDt|GBE31v|S<8xq#!s|9b8me;=4UM{%c*ND z+LF!*;)XyS1nBjDY3=2!SC?^WV1Bds6b4<`D)po0!(S+Yk9|YR9CkRq{dThA)$6=M$IPMH6}K8qc9itm%Zt6 z8B5G%O9N%gHZY&{%(sW=zS|&b9Kn*#C;M`pgvB-2Uu5O0$&~%@pX|@SR;MCc`Fif|j z_>$yZ1$+9O@3Oxm!TMzo6|%8n&;|2_rvR@3KSsQSQd94>a>U$d0E9yf!Mppib7c8y z9&J&Uj$*QC>lydicwj9#0v9^-PTAW>w62b@Kjcaw=F3Em{X@7^FT$Gp8 z@DaJsj{6U;1)&3#;NYuXZ|@i!37y zaG@*T3d7zofOxkGHP#RIw%Ych$3HiwlwDy&kB^J>SM_F0UFU(Y$qoLIFQY z4{dIAA;FY4*cuoa!L=2f*-KQh%f%}RFA0tCy2R_jB|LYs$@x!@5sFvmg#{<3OAi>8OA6^Mu6Nz!?Nkk8HkyybMx{Ac*BSXJ@IuZy#tf)&ti~^oF-fiwHF8 zb3%B;$V@*=-<$1KZnI5)$$lfFuU9$xo)!8ZFe0I$q3wn4TOgLhZCi$qfW#Q3$2KZr zaOPyiJ1h{Kwhnr9*hWN&B0 zobu*DQ~zFNK|PqEW*C$Egba@lnRwW~!?1TpT;-2SQ<|j|A7c8C!7A# zbcNccmC3cMS@s4shM{_f0SIyi5(elId?0zCbX=|;;e+cnqblEaBU$=+>#cq5Y~*jj zE8}t4fRTWTQ4bEAV`B=)i}(l&3sb=M^xO3-w7IQcpM(cd6>Ea^WhMSkL#-kKrUG?H zr+opv=ST2pGdd2uABd=4czthN@Gm3JR{xr_g3}!iHE8M34hKBQ7_pga^oesT&p2#k zWPdO3d+ybx0L%NCLwx~e2YB@s{a!Lr6K?n0P&GKZ0ynfRh}=%>+!i;USjAo}RGWnC zg+K!cYq!17XlK|zTsoA5)zv1wD1D*{aJdZAJWFno9i91jHKSZa6$d>s13;Fck}dgGK|WtV$5 zv2Ebrm^QZ1mQsMjWNEvZr>G49$?eu}D*JS+b;uJxUR_R?5*6<-jv!|CxGx8&PJ}gE zl%lmYb(sT^94!NMv355Z<22)eO!`td9t$8xH}!{hj<8lt$Wc5jFpL0k*Ph&t;ld`L zxE!5+2h_dCUyxkWGI(d=>a_IIf_dYNBgR9(I%_5NUH$$Nw_GJ5gL=616a>xCb{WwQ z_ZDZ3-_S`zqtv}YZmyI0-zZukcW+5McRtEQ?ITF?AA1s<7*%B!UvECR)(ff zoT~8puS?-3Bs8*Wb7!9HXYc(lS2{}Ah#tQ)@2uea@@C*ACB^%s93P$K1LS+@aJm}a zt9Yj&Jp?+f%PHxr``i|)_$!0Q7kClb17$PFArx<9&QlRJ3cF;;K%l|Df@!H z{M+blWe^z5KKLhqFwDi>lk&q|g*QYF7GwPgEKab=Wd&yLw*f#tI(P5lk*@?HkCskb zBHpno6+y#=+KMcXC%jn0Y_c>Kd33Y;2OLjU`F=7Kn_{DJ^Y7$0!)U z+WMmMMX+u}oALHSKBeaSWec*1i|`oado^C1XG2J4jK6yL`R|wGyu$-WyzYh9IEoRtF?xJd(Hb zmxLpAyVCz_wO8I)pEE0xXO$>|Vj9d0ort$H3PtF}xj;3{Op-Br04SD#3~&V?AUbUt z_VnCu1oxQyGOlmI7z=g$%*cN4XQyj793bFII0M-va*hmUFT$5 z3ZhG+JL?7nA5yE`{d0-K zOf-Dm@3gs{l_Me#Zw1qV>wszvM=1K?Wbn`~cctmAQ~8}ck9YBT?O2*Z4T6&JWCoFB zqji_H6aS20KHqHkkA67G&4PG=fW^YZh-(M%ff*|Cghp7aC!}E2%nF2I<%GtvJgeVe zuQ7H5`ZVBOK*jBqT3d-?1`rBaMc$*~HTu6M8Jc-!m@ffz0A8Oi{zH`)IyVF%D99ho zf<{5L**}3)pk)1_x_=_0gU9l^6b>coXjTr~c_zFdNPxd<1}T0XLER5e9%fbxypvey zd@pMgsOL#vjHn4Hm?R-Aq8xC_pKegKJYVB?)AwBt2lADH`9cd1g1Q+pl62JHQs)|* zRN#9Skk`DuylGANt6wl{%&(&L`c2r#Pj$20D$m0Bec8TkK(wT`@QxUIT%7tyG2@&d z-gNK}$ovGWFDF_L^-x2ZbI7%Jxprd@$%DkNU*U|*$y?t7zx0m7TH;|}zj8+30 z63mSIb4;4$c6}?Y@AyOWt70iv-}9sHG?W3)&V_^ka!R<>;JOr^zmMhvquo<$$vbSZ zmC{;JB=lXt5G-Nq_`C)Jy+L#$0)+L|GI5G+xrv7Qz>UCbC2f=UKWS`oz0=4 zxJ_xQ{$yh7jJnz3Hc$4D4}bl^4>EtpNX+Zs8H=XM5}a1 zs8k5Zba)-*s1km9JE744yD-;;;t!7|so8#PM=2xD8BiGMkfGzS@xbx&gx|0_pMjiy zWI@t%`SN8W?m6H%MxoH&#_%QSE^vk87Mw6r{1SQMv$Ul7Bk8D)o9loB`ki-=n=y1d=z)wi zyD;s$T?ey~>yXkzHr{E48L}EMNte^MOS{&*2@SPZ=?vPa)5?%_BDAU}s5-}sjhfcj zu3D6t_5Ib6F+rOfI5p_nH6Gk6gnW0F2Gp>)D-!%159vd(fy_aH z0Z{-~kWME02jKXX(~Vc-EdaAlr)@CMe(&kz6N-L8J^~NQdeOKln{41XjM>29bL|ti z^&*@D9=y-5v0vJ?TesPcRqm6Z|AIq4v{vr_9i&4diO-+Yjb$->WIE!Da08V0&20q| zV|29ZeaIA<4Ks*9Y%QGOQ(@MzIL*j_6~V{K6ZS7ZTUAB+89_0d&=>|v3e0<_uz{$_ zD2JsqFENurClS}lyL8B+J>OXj_}DS;TkIBHA%I%6ufS}${v0s#nQT3Q|205=3wI1O z0tiPAPR^dGPUl(_OPZob%M%WhZ)kR(CzFq*vIOZ2{RV_ov|&lWFQAoK?L}6XxV8s1 z5I%!w$S?+5-Bx-p_|%zAD-O3L4V^&-OgfomtN37#hg!(M7=KuJI8P*pq@aktf^bOv zV8}PYtB%ht*$E;gBussE?;-j#O5W;pW`|bqXQ)AVR}@g>Ar4{XkjGxeECG21GDCfK z{cj9WBr&gd+df)B{XH58^ycvE^W?;WG=lmVFBs$9j)eix6R_k>|74|I{F6+pWQQTB{u^LK6}kw7qr@_aO4>9q{@>E;R=Ij}BL%hntkBV2sm@Qi@@m+rT9Tv z7T@YB_$E@l$gF1wLZEMN=bswx2t{L+;6SjjV7E9jNqB1Je8*o|K4k@tBp3m~Tu!q~O4@>U ztQxTANE+LS1N3E5KAaJpxm39AU+X!Q5AQ8C-s|;8=xm|s{aqJ@LA_N;ri}UepD+{& zUWNlshc%CM-6K@OjjZ)+eL?iiBp{sbn{TGx&K3H0bUbIBA=&-<_ zuk`r0{1+p`ihP8!i~$BW^d^^vHy54@o+GXdo+Cr-VQ#uNvv~B) z)JP}9cj8eJ>Bi^FRpZI48Mq!36f7NuYmD=V0E|_*XtaWCY=^Rh97B9@9KeH5(8&iH z#?J{#pPH4m<*-rG3mJ8hSsjyt7#qbnbMZK+Nv_Q9ysUEyp~Y5OJ?D}ZueO?IM6h+; z{Ak&a8RxC3sgKO%#jlyX?Pz(X=>A>m&ECECADzRKpZEDLKiCL+cY@E++OM{Opl!iv zGv4_N@QEiaW>RDmSq2O}22#&JeiVT|q$9OV1jR_7^#j#x;s6c9XrEt$&lnkqnhBS- z@o#A!$}tp4pF>caz#s-4q?L3e9tEsLZ?rTc*Qd$=SpZXzmNs%A5@45Sn*{jWXkK*NoA*`ldJ=f_~c1awDY3n@%d zTQW1Cq5}(Y%;>eVHZux1JI^a!$63$HH^k}hQ&24A>_qi;q4?QL+gkZ{N3ZrxgF2fu z5MVyIcjf=!KW4!DpZm^+rpB)y8(i=2^jxLk)fI*wi=KgedZ@N7JswCTH(r-oIZg4V z>ZS2xkJ>zX6C~FP!JdH_o_|T8&ToGI?MG3)=uS5l3VyaM>L25roQI?Qk0H=Q89Bx#lH8-MO`9f#;RkUDWIJXLpRA7(6 zIxyVQNJJh6`h^=Rtl!Am3&LB1rNbM6CIpW_l(qDB7HFooWvT8=xp6~oTb$#`3}eDw zy*mxnJ!$WtLr7=MNJj}$8zUn|Iqw`W@T5J1ABbEEa(~G?g0G4sFjpE$eOnmA#XDcrTo1Z7|Z)+x2iSytt}Q!&hT<$MD*(tKC6}@e2!v)b@@{Kc!a% zJUo<}qJi_S^L@^Xn-}qk72ys2Y+Ar{@wgczk#xPw1n!GHGx9VUT0g|PT7Ie>`XfOo ze*?pvAs)e(UFa4r;vgEwu}@~sce8>q@-X28&(3yQY=jd#c_U7tQp1t6%O$DtGNLcg z4yLR^VHrr8Szr+QB5r}nLZC&+zb-{8KEbP?R)lP_4Ps*Dtx%0a2#6vIACFh3n8R;@ z{ml>Q+vzSYwgZ{Rpc@9NEuiUYTEh*{5FsV z6c!Q)Tj1?+qk&cF6~myJ5iXDb^SDQMCKWr4=RUVYIpV#tX-0uzoWPC22 z7xH_^SO>xZi}rxGJ9%@~ftVs9bE6(6klWCB?wvWY^xbN*p{aj@SQ627rkXYEe_X;; zWcAOV9_6ahDhHqgBEN-@bcOXSZT713C1!|xPM0U>>%IkGF}CyY$;Lp4hzR-wHNRWp z!|lNsK*Vm>N?{8nv_|NCHu!v^Q;p2}`1rZ|JJ;z-^O7Dr${kXcp#Z^Crc>ty<{f+7 z_Hg^{WtGi3(fzd1EKTY7v}SK!=HRg%>vQB1L>~NK6NDrfR7pTL_{ykB@tJTJf}FWF z!5i7pOGmmUlrUkG*h<=HhFh(>g@P-n!g{u$o1@7_b?dJlPG7V+=%0_D!jg8i)7G1jk0d#(7?G??a zS>!+|G-&twD8wY$^GZr80+i#{iSu(*j*LGt$P&MC0DW$WhxIAOFLVqHc~_af-O$gS zaKB=#?s?$_&0T~&BWLOo#|FR*$|W*LgR(vIY&VgJVJOk3@tfo|b_m19Vf)sLdE{!C zTIx-dp|Bu5)*7O6{OOlFu$_$UL5ih@rHc$-po90ptP^OHUkH4kzY>~`0fy~(jbRWs z;jP=eCI6Iz!zx0bVc?A<9t0Qkk?*89q^3sMWTQD6Y$&r5Enx#*=ny*X0U#yX9Vn(TX$wh@qMBL+swuo4TYxYqnH4mS?MyrpkzBX#zOIKJ z?lcsB7{mfmBfea*zj{)a6N9jUAO9x^p-q)C6@wlUf>%+G+FZIM3|NBz0@4exhPWR= zk%3^1eQ=QeV<-)BEjl0Rf36tKPmTAOGu*&OaJitMuAEA6d%K!`tv=7ySbQ}!v}~NF zgs(@hyea!+bZhr6qp-su+VZn1&-MJ^x^-@#J~Sx z|F~KLksg15&-gP)?Mz?4;n!;Ak2InyRM6oenieP=HN1u^pk^igJxDyP71+_rDozv$ zix~Kh-iMn1FXhGlrTK>hRBr_Vbv89s0@M58Di@sIAASb};$XTcKZ2ilVix=_;ECe) z?ID0wXy9UN{aqtYpg~w@pc{`asCd1b)#q-^kmq+Y1i5uYIb?*mWncc!MmVWbn}d_f z#zi3tMG`>a%o|Mz+42h_i#pwf!tG`(l6KKV|CPy$ulLFdGL_{ZT*U9n!w=Zr%jp3{*&?(T$p!#%ZXs ztX}7(cZK)3__X+Akyu@-3@fot5a1V2E+d^XUljPL-thsPhgd-Xq<5Dq8$p+tWhQV7 zj|aoPq&rR5k))>Tw7goKc;ME)N}3(0JccT|V1XZXaKk}5yCf`3I#Q4=R@*413Gj<_ z79*CPeYIq4%fjYi!RE22Wrxz9N9Rr+eG(FHv>F)+?agv)&t%7tiAzg&_m*$tT)V z#4Za(5FwC@z<&;EaFRg5L7vR8adqV(gEnwGf-Zs3l^{9+eDw?(XmbmX&k3f+ufhjk z-`PgnIu|u2?U#>Qol?wYAe%{uLn8<~eCr->H#zuBal1O~8_$Zi{lDwQL-`(6ZPWC# z%zsY)yh|xu>)D|P!tC8=7X|q9&Sx!6mLqMnE7=6(;~14d990Y1Y4D-EAu<)w^izKy z24TCPSOlqF;&8YBfgcx!h0;DPXS=V7%JC$D2tP7wfM~n!p&_kO3Umo zBcm1&d{aF3_duJg2>32vv}TQ{X2D=k#d{f6EC-gO%YA3Yfv<&dnLQ2wdNMOJw}Tgf z7>Z<>z2RJt?-02IU>)>OiAQWv@y8!LV`Ri^UC10ISC0_{ATWe5e1?Ma3Y8XK+Zzm( z5Zd(kKj_!vGaC0TgUV@Q{3-svnxgJy_&ap4yN%k=+CZ^~p&Ci=54Sq)Tpk&x!BsXH zJbgYcJXmj>=BDi++))GKbE92uzi-C1V^p(*y~Fm>>9@r zjmfgx?ho$#r=mt>)|`J$M#jL-Q8(p#D$RW`q5V&pE%Zx7YHcJoS^vwSZ9X~_5;5R# z3inqY-E&ZHpvZs~@o3OEaW|ZYt1Y3+L7J~!NfdgLSgQvy4W<|g{)DO~S(?;N%7VDn zAYD`vIFqIC^_I!eGFQr^--~?z@paq%cPD z_*2D`PZgU?0!rsN^;;QwnJUudth>r{T8?_JU`qKBs<`Uh^d&t5wAn5~z=O<-?}#Q; z-xFZ6DQn=-E5CC9Y4R)b8J53)l=-Wp=V(0{7$4|@dImts=<1rW(7w< z#8-&LBrRj`!`iw+*wu*ii%%P?(-BN^V=nX=2phAP5Wa9eF3Z&ejOil4>}6e+`eW$E z?oL6o{he0MUp^($QED}Po7{5bB(3TsvI~Mt9GPs2 z9}swS%Ouf}`ONf$XLYzQVhp$nLy@m{&>-{{3;M=2LLt*d@Fik;xG;QHb;sJJ*D!Zf zSua0BRzIf;u$UCmit{I7Pg0||mo;IfDnE(q!Wv#2WVAztce@YROzMR2($Awyuk2>< zj{91Zcl_PJO=k}sRWy)5%aU0%@DSjUz&L>^!9Wp;MxvE6PQxq2yKHyXC*>Y=g=y;j zW>|1?qnJk+(5U}2ofW?~0SO)sFj@fHC`N07zCF5}olbF5W#-hkuV_EgyE>8QxPp44 z8Brk>$ejKlvQ|M}e@j5hUsriV6+pg(v7OAgMSM}GSymyClU%gt(cG?-Rph9pG%KWS zsLOGb?*#P9^YCt0Tzra%1GNEKiCEdq;R$jKIR~!Q- zPKeD^ux0)#`~kyr8S|GIkD;$?khm7H^QaMN1kkBqLd zk>68q=jGx5ut?s%ExP1OkTKeQa(U)PFkS2=eqj%=xINbfrC<$yck$A~kd@WzjPqB7 zaKK=Cxa|}UHV3INi2blu{04HK$O;Of)KQl&6~PbsXIrxZz})goa*}Y~A$34Unl}VR z;LJ?HPW1pNsnru}K*XSC!K~YF-9NyJ9N%(Jr{SL;`ZyhuWs2p-X&I{+fdz6W%3V7P z_$wLd+RN3Xj~EF_8{El-U8#1tsE+Rr4Q4(@3<&x_iR>OwYo%N~Xk?VTuH02IubQ2o zKVFDqdC&qkOY*7;x^(njUG7(~-AdC&&sr4Qo(&ZVI4C%ITM3Co#w^%6-fpxTT6r2# z3`FU&eiw%4Vf#p%b=4Kp>RmXKem28_3u!B~(XI-Gn-_|_W6~eN8)lR8`p7fvo;a3aOe$=JiL1IsMjwZaP_~^u{>Oj0 zrW1FvsJo94diD1&i#5a!#E*%UVGL_T=!2GG3(aD8MS8Nr$lcd!!62QPyRN@BHwwuy zSOc&g)f~Uvp$SDWrYnZY9s;3i>H1k`?k$Qcv5LVnA2(cw(jQ#Bm53>54sQPhj8()5*V^`Wp>qh9=b! zY4`o2{+STNv-_$8Tg`1H$slWw>$FU3h6)Y=kE238OtzNZ?Uf%a5-a#KTG*KxXoj)_ zqFNu>S^3wq#sukkw=xU;ko|GygK$bjjEz= zWJK51U{Nh;ZHV)(>VZUI$UB$L8-b3aZ_}&<&(NxbWqn*^H3Vac(!nKgW68|}n>3zf zJ!y5gD>A`({>thiGE+L4W>pOn)0OVs#NA{{%bJ0K%#F5jZg_Pv!>ID=WL{g3%NNY3 z40i0puC+COc{^^r`)MduU{dg?6Q|b^FP348#i+_U9t&BpQO!pE z5E4U?$`$UX-$D~@Tmqki`f3j}g%B6gMKd~}ke(dJmc6~X9hZ}(KPWd2KXQ)w?0lK4 zXy3T%Cuc)mJW_@aEq^T9; zoA8;m$~;c0`=uar zv%^sZohB6H3AOuC+7gHjcj;L8yWHs+!aSwu7J=Gh3!et->NczbA2xTUhLg%_v8SuT|x!s$5Q`6UKQS%lWIHX*F7N$ z0Hp=|+Eyge#Yqz7MVtKh?6IDwz zj%Y#sD-m`r)@=|Pv20OBUlq+~6i@)Bph52#aRJFLbA3pIXfUD$6Uqka5Fd0@yY8xf zRSAFm8ui2@P41TI>y{e#Kz5EY4CWwt23ISdZzm-u2P?W0r}DSO{TifK&7^#GQQLUn z-yc6@W!Cdg3y7~kN?gH=*1GD30K4h%aoUsYC76l9H&EU%p54|_F3izuWwnjK6StLz z4hiu7xQJaI0jeS+N7$+_iJm=@`1Zg5xMGgtBHZ|WQ=C@8-03{lCElB^?sdI{$(w2{ z-sz^8yFo zMM~?M_CI1GoOmg%W+bw_jux?k%oKMDV-ebIjF{KvjeK-6EyXxPGWVwT=0xpXt)mVxtW#$%Ff4@hcJVv_;J!yfk zqZ#*u7YEQB_3$R#4^H4Df=0)2W_g#pfIi>iPx$I*TSXi6SYxgqxU%e}UH@GC5Q=Fb zr?#;%m!_YDZo?>5*~qi__!XbFVEBVu3U2|Z8oL#FWtK`Q)t7#3M z3l{U$w<0oSQzT=g4h%i`_^)T_79c}ufcyG*uFu;~j@&XGsVIPcYY&_HK{iy~DDQ?G zR1%;Y1?j96%-lO0Gp4O-Tb3-%(7jR|ZCnFhBGnyc@05$=lg?JQYgaNNliRE*_ zN8k56G1rz)(6jp^^vM(DoV&_7hfHtEY>>2lI*cjwW@gID^#_%E7xELz8NZ(Q}(_4Nat3X2sUmi_e8I^oTn~9;X zNFy{UJ`JZlx4659A= z)8A;U=Puetf03_tn92*^_URHWDBb0=Aa1hN_@m&C|8k%NOst(BmP1=+|E9mBZ@N7< zlGzZO`1zR8na1x+e{StEyaz?w|N}dG|7QYR*W7Y`J(PzJo_1 zBqaEN)jwLp>~y0ESe|0dhLv`qxyo$~02*R-ioosJmTw#%8eRBtVCn9keMO__+7}Mi z&MYj-`u;QO|EKP&Rs{rKRjCyB64%nZ+X`4Ec~M=OR-$smd3&Dfz`Ckv?<)OcmDXKn z<=7sb-6_e^+z40S5VR{B6!uK#39RQTlI7%Y<&NpB=`{@wqTeZ%dK@4di(OMMJBLZ< zw{sI{r2>lR%zQvh+|i8>>j`Tfe{kdLf(S$P$cE#r-;S?r)bIVQa=a~RP~ELi^ZK}2 zO^pr8%#bGUGh61DMHLZFXUb<2$|GSe<4Pe7+VgFMdh6*PX$Gz z6T(S4HI$ZdtN~4E-OY(_@-ApRN=+5`w{LtI16!Q04?Xq*Wu(1|pjLdQGo1e(N!CmA z0Lh498kbCy%0@zG3GI^zhCp94W1J0nKyJwhT_4QDpzp=3qBCQL9GL^vALn|%J zqZi0Tulba1)~jSDp5{q(et0PV79BNSZV+DZq$FxF>?!!LQ}?g{937xqXm_%&=Fcx?UP)=KDZWj! z^vZDPrnH5{8&#f=FaP&M*^l*Tng!543&n0^_n9BB9VeEVX``!Im}cts?fRFtnPgK> zR182UWDffx@z=wS!JVZ!RsMzM-7Z_EG0}f$*V4CwTSkl+VmNVu7SF=*>6iTX>A8H5 zh|q{UE+|n3ZdF?^D^m=z;%g6JG^@u2qUVAxr^|c)Kc=n(9LueXnnRK~i6}ynkR%yW zC>5m;LKKn^B4idqp$L_z6pB*FJcS}jk}Dx|LZ(oe`PZTDe?9NjaP{%M?>YPIz4lsb zUkXx95RIz1Zv0jtYXACML4vpTB^-W7h=zKbA2eCP)J8>HJ~m}Co6BIBNYQk}?_aA0 zZEaTsFVodU#%-N5?uU8arKa<}KPuLsaa8rQKqE%uGTn;X2`Y^Z+rk_4^y-JR^?Dbg z#`W}KkDv2AR-QHJaUm1n?2J88A`%o5#)JWYr3PCIR$+39RQY-m54y!-ssSmZZ&#F{ zJv0IVGO?1p^WHL3abg*jZ+yRAs&iBA_HEmob+Z-Yv?dPnqrFbj1YH9tu?q-lcB{U| zay;Zy&Zmi#O)ECDu(paXF~122*qk7)w@DSu+Zwvd^$Zk0raidn=~$Upu9sH0IvT72 z%ese00gf>|vxtsw6MYSQ))7DD`oiuaL68dAj z9ULfqF@Y#U;7Efm=lcK}Pq+H7xh#SdyYnJI7aY~*Q9 z+xJ^*;mvR>{`T$f3tlQwA@Am}MOG>j`F_CY+u=Rx-_r z+GNXAF0pq;I`g^6eLDr@?<3px(2$a$=`wuRGg7zPyn6eVbr~~nu4S#7>Wj5l8h<|C zm}fSja|_u9!t)|S(M+n?r)m7O)YkKCzq>2%pJ4?0DWln+fab0-Va zGS*=A&I|4z`rlZ6T64^Z^AFR9ylkI(rZffdh3q_q2}>5boJ;)D;ajU111=^_3^vto zorAUBIZ;!nOYML5)j2oPQjo1|kk(-RbZ34^O3GvbhQX&69mh}?Ca;qAFIz}1`2+pL zXEQfJQy>To#k=oBt-tc!3>5Ys*d-@hO6HCawwHMC8G>cp)|RX~m8g^k>smq)x16~W zWTp6N(+S$8qX&IUv8&&y`~;R10Q(~R@872v7S8F;ZY_|Kk_z@UScqXrITXuzS`}8av@dy zA8+^xVc?}~Ry>-YgioM2KI^rRa{U){X*OkJ41t zp)O3i5}N;2GT%suJ=-LVj4IDp#`hD4wbpB;7Z!65ZO@aqZE zv=Sa;qh+0rur#{x@;@!Oe;x7)AOP5c~WSxNe@HL9< z;%~(Y9+Q&DEw;Z{iCNNHP9o(ex|7T3l!KUXN7;LA$#lary`gec!@8vbSx=srJN)P< zJ(~??zeK=a3J2Q%57DZ0eC$7XgsxwQ?e!5>>txK%GfmeQpkEId{*J_|FA}R}eW9RO zrQ{w~`H#T?4D0*C$Dr;@=NDH*B3%LsrVD$n)~@t^_g= zG-Wn<;pg4w-V5{2F*B%8e9n0e99J9w&h;jxZtU1J9R5gS%J`=jY-lhPoEPkzsnsU-0!OiAXaQzWc>$w4)Gf3qF5ko%RaY6Las_Vh8P?Ka~S}(`qU(4?^ush2y^v=OWW)3-w@T!~`K-P_DM){XBmSjL{ zDhOl`b@9qh%mvN2a`v_J6X-~>s*29#l_!#mmXiW~v3Zobfe_IEH&?>^{rT_+SA~)XcA7-q z(~67A{%tt^*~*{LDF?^}tOdGM5Ld79G807=ogX(h{dyK|A5h282Nu#`;`|ekjZYm? zo|7Tt!s6X3!}%+Fl_qv5%ujX`Qt5Q7D*4Hp@0H9}mki$W0#j(F75w4ts`KZ)9XCuoV!$cU_Zt}s5tt&~j3oBifXS`3 z_!V4PuJ3!;UbB8ZAwT?mFO~>wA?p#<0@t%zr2viS9G^kYakc1QHq?6p7cM$%5z>fV z!4Sywi*(^ncm%~`QO|DHqjL)AU$rDwO_j#yKarFkECh`R_#dES_}AG&^+ivdqWkZN zY640F)54xxk)JbrC74!* z|3Y1$c8S@z{+^S$=>F3UBPM6(!Ytkm%+ICFT$m_?9m(3Wr6CQ!XN$AOc5FWYcKyhw zH6MvQmgqG~`sTprL~+o+SPlmDtp^E-v|D29Va!v3y|F|YC)nSLnyOQpur3JP0*)SFa%RLd#N8T4m$#JM;YLAfG zFy+kQ8_d=X(#}JsF~sS_@M&j*&;r`TBnAdb{GGACqJnu5}e5@2I67uQ@6tt!8q}yoI&%Vw-+GKC1R(!f$SFb$(OlB(z`y)@n(oWJN_CQ=GUETw9$v& zHrr&!`}m@@d#D1u)FllhT5f=406H&1&%bp=Y-m+-B~KKx=;^0<0R{cUB=Y0cy8o=b zx@lBa=onvUR|-Y&-`()frW0rQt-g=lGYl}I%d%db@g>l!Gw11WN=xfHs_ty5*14TW zCq^_k#S9eNY8|J2`EkbRtnt;~Ht_EooUki0n;Yd{kSYfMyX*C;6N9%F`W}RLl`i8S zbQxb-aDBp+1ZO{1)IdkE>%_KpdR8D8=qPUG1ySyz9c~NjY)t;*sS0i5_f`BS`=lM3Rt_vui zn3v&9zN6I`VDCo_+QU*~8K1!`ME{d%6ZcS`U45e6A3f^gZLuY46)qJxRSdZtO$$m9 zyVhd6>!>OFQ>&cgsZ?~E_RwzHQgQCsS=tp>2eSMa)L*{*s9R&Ym6Cd1E6WWATV(RO z63oc&L6e4Lz0~v?^|ht@U8lR)VSspU4~EF)d)(+ynk16J>_#Fw|lMD5A;qj zv!FQ)VH#ll_ycgBfTjrES)Y z246e$)$HY%0IA~&uY}7EZEC~Etz`!%G&+V;mMb_MAVyirb z3PuYH_tGw>pgnf~-=F!vziua5%DxmSF%E9Nd5lY3O$X>&F$Nq{Tqxzhkw+B@ZZG8W z5Ew5x3enN)=x*D@?{rK>@|(*thN~Y_K^H%n{zPoV`i9R?@e}bH>nD}2Kqcd&eCZy9 z;;}V4pN}U^@kbrcGs-Tdw^le-C~a$|Hf^=dY_eJGgnH1^c>I`|!$N@o;4gud7^b|Vu9uKXwV{`=cBYP<)l`cL1*pPk`>qXg95i>y{+C95L_Rs!g)Aoq|Si0;`( zH#VW8fG@!H%@Me<;O|W#CoeRQ49}KGGXaiNb(rdYM=U2u6N>oF)L34)xn2H4%OW!n z=qVT;deoAB40I3bi$`w`&q!pSxO&(}>ZxeRY8DDpEo#Y2-0FR0pD(LNU55M8`x1To zQhH&h_R~h-GY0IZV?)h}>x)Hn)svtd3bAUbdb=3!lp7rMBCe}^l0va;u)_sxXz+sjp=1|9(+_;|eQ7$T?@qW$$+x)- z3|~Dbo*6$Zpfcg)VqZqD!>!AY6NZ|)VoY7K^pU;kyQ8|CP_dz>3Ef$Pb;A=cNEZl; z859G|x|WDJ7Qil%{p>7h$4P@JyDlUl?^&qWGUW6?{)nOFk|$9hu{QV@vRZo9dax{V zt$k73`)_MfmXxRY0RCqjPfs!%KJ78Z0B9mDU4Q*0YH)|?pJzyY3;q)pKav2;(-$c$ zohh(hwKeY*d_*BCbup9aZH3&8^F@S-MyVMV6Be#Dpb1Ny#!s^#Iya@n|RIv@FRu zQ?qqhXb_QXTC!Kb$fulWXHHczfH@Ll-wf#{NNu=gZ(@^K9lF*K8TuHMY(|q5huz`z)3tTi@ zpr&CIV{cym&M~nNKe!M)?Wh7^R?n`LV zuHZ+PVKZbk1=$W{nuH%G#6}=X2$OLstR=*Mu_#9H*mj5&r=$Sw0yzc#VT7(CABW#t zNmVysJ@CFb81FQjU@H@D~78nuT2_7KuQhK$}ejFIxW z8-%Z(A+>ZA5l&&noJ_aUBgURfM)#maTI!e5ZzKn!^)?)p5W3-n#9hklf}a}&sH}v# z11or_jLCXkbi=>-j8cVVx@Z8>-Fle#X(f+h{3>j~NPEobQltK7#$BQ(pQ;QQCf3c4g%>_f*J3Q3LN zqOTt`4d8C>jV5L!>bOa+R<UZJ+AsA;W2%?6Y`|_D# z(I(0P3KJZeh}Sr1B1G>F2{uaiN~lTjJrU7i+*vNr+5$gGF8qVk2J8kr0vdwaqq;P_ z3{kQuhoSE3e^1yncuo2dX&IUI;{Ht;tR&YTrOQ^!^hLr^;MsI6!u8JrgQdL~TY*4K zsn5*K{h)CbBQ5eAn4*%EO|-mKks?ySxF9GrZiLp3Z50hAjGLbvLHbq0DouPA;2nsc z87~Oi5Wyvvf=>IJ4_aJ~t&{w8;8?geC$X^3Pn$4;?IO+_FyvQrNs8%_Zvv4l6eL3z zTAnzYx@%aU~!=|cU^ssL{#OmP8rN6Fmx^$ z!%~TC5zlJL(w?a0L_aPm>K5%!h>jC(a0Kze^T8?smlqdS39h9YOf=-lVSB#SD%Y}* zg&;D5xeqfl+@z(gfkgtt(XI9*rgfBp*0|WZfbJ978v;JgVX)GPeh6XK9uiUF+6!t> z%xW3z=DD^F|3*;qo9&6N3dNSWFMp76YKxu{Vswqy(V957FjNBG;IDU!Jo~@zx-SD8 zYtfYyGFf1wNDg6O&`rdsn0#Q5A!o#$7iwSx@!LtI)>*`N0_G@?2!f0V*$-&Ce@+16 z0zxu?B9o=yPsea|xleq{Kgt#Dz{^mQWoGz7%NlSA2Mc)I_%?*O4m*f)+&6SJ@u-mw z5S1YO$*Oza^{nFJ!Yv_|&WJFG5kwG$XoY+^;f=Q+G{h+8p=N|&6#HRQI}2KQFyw?cD%PHiCN%}fQFw|~Rkmm~w4|S^ys)(7i9HAI4sKKL zIJ&T~waMIFs`>R~oRh8W8n|E}&I8pR%4rW`>b?fur!!reE9tLlq&5kiC;juj z2>Yj=@DVX*Bgc}dQ6fe z^W{sQsXT1=Mwk=Bkr5{q49fllzdo)B(ubV0E7Ke*XrHqyiG99-h~7xV-_Aj`bdU0{ z1Ack&8b^*nG(>tN%_N}RBsRop99gJS@h0I{1Y#jHq%h%LMr69gv;jRAVDNB#fCjvk z;;K(6yaBOyu}?=aM48y5_cO#Hra{&Xp8}N2h;m4Nmi!5cU#z~z5_)TLkgl1}Mt z2FV1r9ZP?a#)!N*oBq^4Q;14PEHH7BK^X|(C4BSILclx-GYKKDf}1s{ zmk|0x4F}6+Drn9-82{#9i@&yVz?IsrWbVSBhQC<@)#BF5PbBeE4Yv_lqC^Zivxx@@ zT*)w-izBK`~-M`|DaRBc6CwP;2RO$QvJz$1pOK9nnH z7hoe`r$7Kftn`tIz-9#SZO!uk{%CcCrwgUY(sDm)XSFz)XVDhLqMD?js7I)27taW% z1E*g4SY3hp1Ee6BuM0sS@oZUi?WUkW=|=nzU=cV;CNGjV3brg3A2L3O1=?eGZI=Y* ziwm$jJr!_9BuOyf73H9!AQcKa9I&=PY6F%mIu!U{u;+!t|Ioyu?+`4l4>me)^>{{T zjs3(_E+hf17pvQrdMY%}t?zCODJxwjA^AH7bjR^y!&?k32Um7)vn+fM2F_eXFKnMd|hkMIi%GgWh+4&c?l z^R#bwDz5mHIo4ilH*0hJZc56kRu`vkB$D9jocI^$dC{|Y4#MyS%MBqJpVc2=>>%DK z9z5ws60Z^r?|`evHKAZ-t&d5-o)C9X>H=5L<{&~zdMb1+{@Q1fMFo-rtdzkrT_T=v z!B_`m{ZTZ@QOTh8*?t2%9%rGs^sV?Q_`B()l9dBWIle_Ng9Pz|Ksu=BELLMKP0nNd z`_x~~QetN@n1So5UiBS}zeUVP%7DqICm?4I6ZE|RMG_fsbmXy4tp^t6V;xD@6%r0N z#mfoH7;Gpq9wN5p_q$^puRX%$;pG6Eya%F;OP)~~*7L}GOgw+-{$rg*{?GbqX;j?k z7QzQ_$B9N~0AVD00{$&Z5<@0V5@iE1J*Qg2YL@&Shig?EdbvI?YSOe65VTC<(@O@D4MQKvqmq%~zHK zZ-AX6h3v@u$x#3ep6HmMpKR?TOO$c%d_v%RvuFSa!~kIv&;g;7E8Xo4_QV#=x05lx zly>%mGp)R=^u_zM)tB7nLDr|-D*l#`C`r!LF ze%uwgCmj6#78Q zjeoNXP16xGJUV)AlFBYtd6^!t03_Bj#DN20y0mv(%4pZI03i)hV{&u9BzX!vo6z-w zMLNoCX!u^O=^_&tgipPHznE8J&d8gp7d)I!=sJ!2}?Kl2}I) zBVP{=+LXjP9M$HaJy$X(3j>%TC^(YLXjXb*NMKh?$*cwZRO=5dn)omneO@q0Vxr-mkV7Bcc`H;!w3z3uQ%4^^`v zdo?9R_erPO9E#_=pMN%E7y3-QO5r!nGEP$fAoHx!GU{; zvTX)i#6LKna$Ej$c{t~0*?oT0TCwaA9sZ5Oudi*Q$*EaU`u1PqPOH9?F($Sx9q^A1 zp(=0QeKlenM+Wj-cnv-9<{q~&2m~_havC5?+w8>DX-a7-UNs!WDxIll`JU1|DK8wH;SsyTkIN zD?n^nwYjc(y0L10{H~`*`x(W#?^7*znv3)CHO<Z*LPk@2hbx7%r^|E!n6jTxX zcCnGl&40Qnb+AJPCwW~o2u^ier<{gm{RkSMOA11m zLL#AAa#>D~pfr~4{?)LUlM>J7>K>HE7o)XxvqbvH^bx10{rPm7G3!#DUuZs-KOiKd>kgn;8S{cY&iEgIX+xd1#o@f8^6^XRE_3MH!(6dD6_^VLos|EQ{#2RDd z0@Jp|JEOfaR!D_~W-awZ@Mz1m(c{+_Gchb>qGJ}0Bv{s+JLZ3T*^gEd6bG>nM>8sN zq;yMoesX=&NK&2V+eeE4;XWbafn$d0YEkjmWd}L)PZ?%z8W(CU=S=T{{TwATrG%W!)DeG!9 zpBu8SUR&{#&z*(wGG$cdy2T9aXBc>z=;NfPxZFkX2bz!FZkQG5m`Iv4@E$@`1`dL2 zmT2P02(X?*e_Rk%V*!xYah<-w7Z(>tPQgVUg5{f++(iL`xgXTe+z!Oy4j)d~sSt;Q zfK?`B8WWIH`1&vTJH6N_f0fY`Z57{?jV$7_a<3?6p9(4bR*~E66{Hxq5Xwl!CGK)I z`*yv0?}Cfi3h`q?KFocG?|tWyza3(!+RUNZ(HVnP@C`3ZHTX|Zzf|j$vjQn z+QQzu-fPi~x{GlkFd6*U=<4RP9nC*pcjodIkvWzV0?OWB3ei+m_Y_<92+3=UP%T>M`lx1{%RBXR^ z-@5dzh5Na)AL_JsS~>ApIeoWkX|Zp*SQ7K>NyzhyiUTL)Q~V5*gFPPj1?H^K>=hkZ z+#-l&Bp)CF@W@6yzLPLphwUF3?13&YS|d;fLLc~`u47~${K~$k*!u$3A|%D~e-&Mb zs&$%I(O02*u0XX9o>an-#3XRBti}tKEvg?y0p0?r!|_UC&vDT&Z2+Yf`V@d?QOOWk z3OSTL8D+NPJROR4%UmD98WvV#&m|vb%1_Ok}&*xS~5d zDb;Gz3Z0f6Z>;(9+vccqX+LbyS>E^9$R~b3b@h`?(fy1}eLg9vR;kj>)@wpO8PugM zul_!w@GXv!u0sA;_Ji%$?tPbUPVu{#91xbYd)dDXsTVuoRMIbMm317)$F$!|ha?TL9C&pa3Lp*? zK4=Ud6sA_wCg2C0ppd(nf#giwfy(J%A<3ckr-WZ&=P(^`-6r%b?D4>-Hz0~lXgJY# zAomS$0!;0QCkbvQ%v6}Tpus-RM;=n$I7mW)rME>=9jHXVdmx=r!j08*W12w8X(q~DW3Zx9Wi)sOuz3^JT>EJ3hM*nObsK`vsZw-vQMS}`m%_fcFo z-b^}kShs<9V*w_9U}?dj1Vn4Hzlq!w=N@2F4DF*M!x9$z;@>n*V;9R`6;nmQP0U*{ zvihzm?B{_V5}7Lx4<-PeqsMZ^P#%bU1s}lN3D7KYn)-Ub?$b%9_a%2KSH~?->LS@b ze%Q(4^_vB%CtRat=L_o}d*{_Je0P4zENJ?A-0HA1XLHK%LhHIqQ3L&6ZC6@P>nKp@ z_;ivZmV7j$PeagStQ}lX?f^>;I&R{(inUFpfDkleE(Zmx2I}=v+0RF?S+Mrig=Us5F zz9zBYq^|>#^QWI2&$Qs1lhp-8RW_9qK?I2uUJ+pvpcY^|a6^Jskv(C718g?<2FYJ) zlwjjI^@)-*6~96!M`E>glVIDSBZj?Av=R;0qgy919*6WGt9zDGFXB%Kay8ta9P_wu zepw&gUam*4N6Dpsf6wvrJBkJ>#)g*1jXG0UW>M@Ka4Te9{)A=odK*7#zt?oTeA1R1 zH>_-XEb%-)f#My4LG4DPN3Ny@f`+EPoK!UQ&v)gr*t;h0r_$5ei(%n<8oBl}8D(65 zqT>tBMQrY*9xs_v;+Cx=W)a8;U>b`Gz%(xO{CGkGcYI!Ethi_<2+hK?f_E6CI@5Y^ z>2c);FJgaWn9@@{;yGjE#MHBYHC88*FEsBEDB;zM3r>xM7l=+>I(+qUAHnd(*8|yx zv>FVn@%0)XS2R*aN@&*0jGw&v^9u}XQyYMT)r@7sPrv-YmLh)`zY_bnSk3c%5b`Yg zs>3mi4C;lS2N2Nh<;w`eJtYO56=cT9kI^_Bpj&a(dyN_)>k;&`LSe<6JdoD29@1lB{770e#S%jFK`3&#D{=`XEW*BK77h_B}`Hp%iw@Gvg*8MgQB zDf8G>@BHXCqwdjr{bFEvs;c}8R+|r`4)Z+@2ARtUFhv*kh=%I@B5f*Ke3f48@P z7MysRk?}jlBsr$3YTdRQq63^}Jbs%ECfCS+4wU$$;loVj^h0x5xe`O*iW5(+tjuOM zG3_(ee{NRBwjq+oF_8CtS^sc<*6+8TwO19p6wTF~^v>}*Mt|@3Qrd;-<7}g590_-G~bxxM|gyL&9Szv-Eo<)x*R_5C_R>>iXb?3;Ks z)9x`|^HA;%jBfs1v+Gs;d<7JG(U_>0Ni!*Wiymrl+JI5Pg-~#*_s$Nr&u@n55d-~7 zKc6})|B;!im8a~!|NXmIhIq<-uO^_H-Es$$ zkLI^ZT)uqS=G-}v$nswo<3?Q%=CmX>*zQTc5X@2sQ(qsoUz~+j!wl?-djvGV#%(%afPnj#{WMq_0Ps^V@ zdsckQmdO2yTJD+tcaJ95cVdQI2g1NN;CBaw*|T$UZXFcn4phNORsW)XZPE|gYuX=# z_>8L(s_48rO4;`ebW_YVWsaP<%o3EqB69QOGWNm`YSA1{8}{gJd9bO!n}&Mr=xpB0 z1)+|G(n<{)pEbF1zE{>$(@+~JkDLv@AQGa;W*6 z1r6lOg+)4v25oVO8tJs(_pGtIuP>;mNRfHv$}_UDKQAiUv{X2^K4i=_<3^B0hK`~O z?L!mSPFk{x5)Z`aA3xp=S=-H|jroT3em?@nJ@0BIm&5164ddAJ+$SVdSdcovRYmme zo*6;-6Uu}-$!jXzO~x3%{QS2S6?ALXta*}_b|kd{fvHN`KRLG74Mo#K=NVUpBqNEf zYs1xavwE+tAK6htmJNZC*AShvj&ZhX#2!d1RZo^>EG;+kfICTsYC@UO4?d6$}9hlS0?# zJ?#H3o_NajrMp<-&x6{c!S+}uGFm1QP?Ch=;GggpxJQ|-M4 z0zn-=E3b&ymP;OFWBA6-*mYpGO;eaXuzcN|*Im1u1IhLG8wWPTc$L&z$()qr2059V z`LYHr&&{G$DJds!3*+Ujv)+FDCX9T=?CjYqPoHl7)YxcXVxkqHr)puP!lJc@J5b|T zwEBb6x;obUw8+|CDJH?ZS6$id59)&ItW;AQ(APOX(xwPI^v|C^cYJS1CG98>N3gIS z>xeaZGaM3u*QLw~UERpL@D;C@(bgT!>T^qYBfs(KzEb_tv6+KK651?mY{I3#Sl`i3 zZCzfhtR<1_ecMRhi}!PJKu@Us?8d+d6%Y&5M#c5Q>)ud|Hln8z(vYufl~)>R?01Sdzqi#c2BLRUNmDuAuKYC7=24Er908cN z6p8=FUtC7k^Jf7moKh7E6`+UngQR9;Pmcx617jqNPo6&?nU%E-0k;%6vYdhf_ADaD zO|4}8=zxLztLlQnks549k5j|~%}6kQ`<7TNYm}8DYr@>2Bad$gt?tX8Km4J6cE};a z_lN7K|MOnA$>O`SSMiwbXtQla*k8qIi7?!;mEpl!HP4V6*6*D0ig?6QQW9u};0#O# z$s3|agip(rRa-K$vM?A{L(IKaBp5&Yoj^$@20tIkJG2v)nN1J7-(#_%1L5`rJGHX= z;p+)=Y0P)QBBdZz$y2>6P!5BG5;+&2bmruVUmRFk2-@d6%V^CFETwPRPmNh%88-MpQ%l=E}Ype#q=MP?Wta|7|1QUa`VR$3QJg*HJwz9aD>TpDaF3CNibGMX80Bgi)R&>Bk>6GD@b>yJd`vP@UMY}fBlVJj6JI_engZbLcm!~TWiMR zqIH1TO(A!t%jH6%vhkQz*@Da#Sm~uwpHbI4MqTFDC)q@}-jyT+#2%2x)_L6F*;y7K z#9-m(b_?(lGDMVQ<&uXvHcTB1zIR-OC7vrn2=qobJQg>M4EbupSfVPd85wJ)Jl;roPMI~f6)2+ZXb|lsXDL0c)mLHYhD_av?>d0C(T;++Jw_9 z-}Dh-2F{3TaS}u%jWxl{4svd6S#rw!4wIPC{1c1HJs#gONNU!t5 z%(!nRA;;3_$cJHJ2#oKmeoYAlPi9a4PTu@RG3^K2F^}M9%T%rzl}UQ^u^o#VRan{R z9NBK$Vu|28@GC-s25}jQ1a@4V6_`>mo|g+7f5VkM>b|8~I?zlgMy?4343cr;!+Y$R zk;VD5Tl@F1if}5}KFbuKFH=y{#`%xubsifo%_t8OV2?O@&n~2Q%Q4YWI*uK@pd`GW5vmJf?;m-F4*zS>bP zEi=M6!HVMpXWX6m#OO3Of7XzPbw>jaDlStFp)a$_F5GwfzztWq(p^!yPHKCut0o|{ z{(Ybd2e7O2jVoK9b)VN5x)6J@NUO*sxxuEc|9w8E|4Qe*9RH8&oo<3`Of)K!s+e9a zqY1XJMsmH4jScCn2{pvIxVSX6wUK)?H|NRB%nXb_JkI}W0)3?e787z#{GB8dH55HE zn;ERb*)h6zLrgU`+he^{BuxkFg(9mIoTQjAxh%qZW@X#_i{PCrUsch>_e#ZSMZTwU z$WZqPE?{z!kNYaUHazkbb4&Iip_Q*nDy#TMJsqNMFMAzKi;1556EUIl6!tfU6Q<|o zRUrw_GYZ>XyenN_D6$45DoLh2I>THvowry;dK#GuC@pTybWB=6Z=zDmhAdDL~+WLH&$x?XC6rPD{lfa~=q2eZ-(DYZ2(bDMmi%*N9yjkq})@tk$cDV z@0-P2&AN7e_PxAnae*;?lrr z$DxD7#Yx^m`j${{oKYiZ746@ZfF(ge5-fiaSIJ2&Z4Sg=Bq!B9KRo#xoe|@Ze3GID zISWb`^#{=6nB!F=Bg48)4Y=z2Z}dCu$K18BOTBIyIlj4$USDhlhLzBr!wJ}L!y>X4 z&loR>Kjozj$VUim2xh$XPMktXOD8Bsxt(@hv8~tf?vRRc&bhl^OPQ(18?JYHwKTap z%BJzKyQ~djXTmydt_6W!DE;NQfTd*+uq#za5qkKghDqPnK3?Qu^kC^1Q=> zOI5~IjCId=hjoK-!ES5IJ6YN%@|*{_Le(!}>PAL-dTG?AV-cS95AyR~E%d@XQT+ic zB2cG2@Ij63w_HmgF-4w$QUvK57@YMok?9KxHZ4gF)^$S{Rx{CIE7-IMGwl)5*dH+o zY!W{jDj27SQwCjgpIjBRDFx9z*#=cZm|sR+DF!2l6#mCE~;si;|ZLLs(C)*9beUqif;d zaNRNQPNB#lIBYH@P#rT$$fch?eQIu;k`nK_rDaI=>8B^0MHU8tosohEAt3jHeI~^d z8ZlP{Z}r>iN_P+aZbA9j{~N(!toY}QXGOPz<2%+85eKT%WSdoaB%F#reaT_`-Q{0 z>bupC>5Qo}{>pP*p#6Vc+0-?o6lRNh%1U<4i`&k0b4o}jyh>5c!RIXqzT_GL%A`6XR0~4c1X$H%N0cg^7sI004qB z>zKQOTN4&Uc%^H63{82Za_^>x{$~3))IgC-8*nNx4hsv5@Hu4M82Nr#(wT%ahHNaW zPi$v`MqkSajf(mgLjAjR8`vqsZYOP^9Ct7^=e;*6=TPR;>sDoMBi=?=J04*h+VY_y zMy?djT0-|A&r+dcXN={fBHnm=UI4q#v#;WeWxqC z9WGS9t(DBn%ljN4&+p%|2vFDq7BzlpoN6kK&e(@|PpHSiS}Y$Mlg1wo3P;+kddJ2D zkRZpdwX1xkvbnqa%xSx`9(*0JdzgLuZ1!#N+}fAs(&XC@G(I~&^V=2)&9jnuc~m0% z^C#Ywq+zSA*=`3WAwNlF#iF_zj+yjyEngKWbWP-p49_g4*nRC@!+)xY3y+`p9iV+8 zykq#Lw=woGzt`kCe$St*c#+W16`;~4ox!oz)Y8&D{c8X>5?gBQ7BXQHb`f(Rtk*X% z(2zfe=z)7o&M0?xcdR7*P+{FT!d{=ej&~?woc{_vFavyPSaSTYx!J+2n8QM=4USyf zz^SPfd1Se3z8cFk&GkdvQ#9JUxanwUD&y#EFxk_$-mK%rk1{SZ4i?pQ?C1P#wp}yl zGx#XF=8E|Xg?#Iik2acXROr;xnC-JK928)pe{fBJ(S8sAmYdGoo>Ry8ZZ+=+##GFKqf4<`+{y6djl<<#J*&baS9qUmS@g@_ZA>@OF*j)Ik{`w#*p1gcTb-^ zyS#S%b5m0%KszllFFroLbS$!`df5TaZ?bLK@g6S)4=7+tN;yyhnKjN(i#! zhcw2G28MbXZB0b^jnu61y(AM( zSY$~xHJV_{H3&UY9KObJF4DFI>yv<#C*~SXjLy%DC_BzM95K1`W!1>uc5qLTLE?wR zY}O6ocMl2=4iAq(*E0^^$!qAJ^t#;))X0-LkP!>)U=oV=w=Tuwe0Vi=+1NNmB@1B& zfkTe2u5aKpqX@H^5vf}T+3&R$+(UbO-#ObDJIZw%$5`1#(Ju; zN(x%PVaI>PodqB5GJsqQ1C7hc0BVSJfslcPvomMC%nqyy0-AzQ+glO3rpjFzS!Z_+ zzAC@f?xO`7t%`e%nNG5WJbCa&p04Tcl-2g<-12;e^`1ISuU1gmpuFb0^KyfIzW+Xd zRvPUfep>U5_cqDjesU_MzsYa|L-6H=;@JJj*L9kGeEM>{uzv0DQ?CPxyTW>adjc;*{*#|}_xF$gZ;;MBp#*iLKRRnyz2I2J zF0pwtxONV_%c^f{I#xtYRCbRBM?XQ9mA5%9TY;Aurxlzu4Nh3WlR7L*^I;aLQ21B9eDX$fE1iFJF) zT$w_M@<)w&Bo_a8(TI0ADa(qhOqe~o6ma;91@Rjs5QJ+hr&avd%`^Bg*x zn^uiD8lgpZ2nQDq31Z!+QIS(r6eo1I!8)s(Jv`HM7|;^1Xslc?uE;ossl*f0i9&UQ z+8P%StD%m#Nm>D9>v+KS4PQrbu>K&`557v&!5>E^P<{>$4LJ;a+Dvdwuc;XCTg1=h z$&)AOmQh4R2IKSrh!t4gDVQV0v2nD8 zzBZ7Lh_C)YJLBq1Rb(eSJA+~^MmCOv>Me=mqP@NM)6?TsXQ9W8ICZPv=`9fD{)rjv zC9*eqs$-;>E)m0xulOpcsuXG=G{?59bLOkg=(|vWWAI`14e@HDT=au_EEogIlBWx% zq^I1%QF4f!zM;(-=VxHC!99MiPO>k)aow6zb2ur8MCo( zb%x>XC(&g(&N_9WFvk6xf5mSc#gnl_xO$-4}Z3hFmRPR>-3Ov`PYfbdDo_K%CF-pT@D{Udt}YZecqXNQ<%^`szXr|`NVT#IH#5xMZ9n7|Uuh-N)z<312uPjVy49gwdHIlGcDG#BXvOR}Xsadl{Xd2aMuatJ z(gsNR3MdW=vX*RoPhK3U$QOr)?LL;?1214eezDia3jsK?SA!8 zdL&;#>n*EjeJ^zvd@ip1C-Y`5dqtanp@~4E#o-Mr)by@zpf!lNe(UL;Xigc_`7%zr zdyhFCY}$2dO22OMHi-rrv?bBd0YOGlXiQH&<0T z>qeYzW#*t&i_nfw4deXJDbd3cxXz5FBbPbb;*>#VmoeC1cF8ynm6px84-Sode8gX8 zi(FuOC1>G9!&kk9U0MW1fsE5>6FE-_zK1LV9XNyyDW>ihm*@Ur^z*?G!C1oj@N_($ zuK#uAe8~=v~tkce$ z&nDL#`_30y!9+LraQzP_dBr+TGhiUTuE^;M3#ZJf8=W1DC$4Kq-F=;{EXsT^ZsZz^ z%f%q&*bH$IU(cj7!oiyY%}lu~g3Wz#j$z=HOpVZWLQLVQ45n@#%!ES&>Hb)bP0Bs) z)b_rDCs6j_btRVr#Vm?+oHZzttYn}~0Vo{lFWcx$&WVwcagy;Y6i+OmQ7sZ%GuCxw z%h;xpu}$a$h)^!<`q(^+$@2@PRuE!S&%81A`fmfaSuIq!jp7mQ4wX^p2r~u zmpD=U4*r?CBwIyUhvcDld@(((I^oOd=7E+mvKK!)U zf&+9k*TaWeCe2n>XZC+qlu_HOXcE7E;88}PaIIZEC-ojF;O`;o*F%o%zRn?I@WAvd zC4;Ws2lKdAvrF-cub34EUy8h_IH<7Fc=f%i=26FRZ!wj9$*(n{402|rrw{Q1{X0f1%j7E?=DU6Ra~YqVxs8yiqZ3a3txNB2w1pHh<^u`_vUgb|PlO6GQ(bLb zv*m=YZi47Zg|%C`nbz(X`w^bnv8?zIx0?G=vc;VkR?J zdvkfVD{HIBF_jCQLC5*4Z0-pIxp}7AWl3oH35hD?@{PA{{`M?2<+g&bmT(X|LstnH zPlEFvk5cx-wBIClYM7gVmpTBABTDWAc=o91E#QCEed|Q;gM3Z`q2IkDUJd4lP}70v z9-$zm;WX+@I~p8^q$Zo2b_TitYz+jg@u+7Wi;YglJfvJ2Vh*ooV!^jH6=%SDY z!>Cu5UMeJ2ADrV#G2gr4B#?XXWPnjd_TZir3co%%`pRp&NwjTCm4$p9Rpxi2Ek!@s z$bXwQZ9?mnJ?>JmO;`74Y*w-ZL7RPtgcE2!nmWC1pqc6D_8{$ajdxjJm8c`;(Eo7* zMkhTR#XOs+12GbykRdr04T(6_Rb{>#jlhTlJfRhWk@E2yRQjZU&AfQ3m}KG|t-tGz zsQ|kOw>^HT<_USY{()J_4#$hP64Ljz8409vN=C_S+_tyw$w{DyBP{-`)GoW|UGHD! zSJU@B|Aj$CEkeiKKyAlx;JvriZk21~v-EA82}aSmvN~bIjIt zds)_Q2w7hyVCwB1esPZFtaaS)YkxA?&hzQT#VT2P=r6?oKJ0&JqkWD6aus z;6{J?^hw{?7zJt+P_ws*V}o~YB?}|4_^AOE5ZS$V5T`&MQ=HDdXZpc1K zmbGV#cdGEdQsfB*rL8KleME$E+9AtrL~fTdb6V`mdtvO$50(qWRUxsAH0|Gg-#&rS z-KgKws|bH4`5@!kQ1yc%kp?~P?cSn4j$gOnFi_XiJy^bLrHR5H-Vu!dW9mD= zv2NS{GqO^VE!kN~5|WH;clN3jWlN>9LK%hZ%n(9UWTa4%BxEKDl|;#klAV?H|9tiS z-uHhT&+{Db`#c%dWhSX4O52e%5@i0q3%#j$&vx^%CJ7^*qm85Hpe+;a zm`R!TTi9Q0DV?xg7}zj0~OUcy__}ZFH}9h<=V9`}GXJ>i;|&lQ$PXzM5GZu(xERr8>1uxcs4u#UC-6 z$-oVrMUFcvmK0N3>VLOC=r|8P{L7yutobGzXbYY40$`NkYXpR@OI`y^4%EX>_wdp3g-R`Us-V7=ro9?*F;Nuv2Eu2vtz$?z zO>fN#@UfxSL#2fB2q^{GPWZTx7l@Pc4+hX)XX;wSHoh1y=&qV7v*)N;EW0K`?!8Dt!>8 zaEQ-#2?>|0QXZFM=;byj;4F{*cTt_vcab_)aVk#mXRPpycym_U>5HhJICMB@x$Ox} zc9>mBvnve`g_UH1@Brul_zB%$vMw~b(LN^?N{>IRN3iyr?We8?CN^fLKCS?}Tmlr8q0_UZ5_<#+>Ko=75CYse{}Ko&kWKAoySRO~%@q zm_fgNSSY@F-WMo4z@L8{pK!X?4uQIcY(UWXa=*VF8N|hbcN}vQx?3V7<05L^;rs1O z2Ne{?elV#M7GmOWp!0v;FWqi&K#_!ISN4ED*4cr^17g4U^wg3Jp%=t&5qExj5ugps z4Y&@_UJO1>)6XIGU-aRcDWDCT6yVwK{@7Rer`3k(70STSdl4$kB)4cIc6kaQiB~t6 zeQMlp&rH%ds!ON2asMQ{yjqlymb%d)^@;*>^4H>-)`{mCG?ci>j0T0H$u?^IvKA5Y zW6$Ef+az}2fZAc1_jU=z=65EJMOsh`(xDn2)BouogIeFJWu zv{hpv=tqQASLt=Q9jAQdN_${Ze8t#Bc+z)COY zSZNDTkB*;@U%iTU?yf4JIFxA!89b6OV5&xX4zye`rti&|vD>#5LMIXC=|MMh<6P-r z>$d^RgXS@(*n<`-Lk7wknA`-i6Fs=Z@5i>belihD5E-2CG%%DI^WzUF;B7YECdaP9 zpuuOvzK#26$Fppv`gKYf_~py4dnX@t0095bNMrEy_z=nqu{et;IYd1|EM(KwhkN#+ zD~_TFw!Zy9kV%-cHpzQx(CK!CRBzb#>TWv{Yy3T2d9Ab!E+ey-p}l%Pc8RPE&7%=> z(aV>5xyLT|D`j?$6}@=YjQQ|x8_g}V^l*x4o*tg+6<=ehn~&u0q>aM1mZH&Ed!lXw zE`k-LdHOedLouc@2yD-FYZ~1YoN|-qv^KvK7y1|B)3_*g<uUqdqlhY!r*uO+!Jr(^b)N(Kv#9}ltS*YhS1 zkG%TxXh@iida%L&X8V-$+pm{yvol<#PvZ{1mw8yo{Gs-dJz%(WcWP)h$F{kS9Oe|h zNF$Ijvar~uHtMyojZB5vnwncHp0)VioqYN7yt|@eQriyfR#RzhH*1Kr*iOS!B)%B- zYKf2j9JOWo9UV?5vz)|t#_v5UD_$!97uuxk`^n1oG)1?b>-U7)#m5bO)k<1ROI#mE z-M{Rvk7xY!!A5?EtvPGuM8@1>o?S{gMt z8I_(B!v&00Xu{)*`@vUqyAh2hLWp8g#`B?J!rU*SPIZyA*;~mu9#SSWq1B@6luwJ@nqjiAu zLDJLSlaG>2N1qBY32InFQGhlDSl)ft>#9ZCLaB9nz z2adl}T3BsaTP+*KyHS(W>XWdycb@0PsY%}q9JWWDw(@EGq-{uY>hwTqXpn9Z9%q6u zeO=5O_iMEzs9`BdD~#gZKb`{aox;&*pK}$x5&7ZC*0FRK0b`U+m1{`Lurp8hV03*MQckJ3a2bFsfX+IA8lS&?6AI_Y zu_bt^k4I14$F|(jzfAF5mGu4d@Uz6VIAkoW){$;ZAsqqS2yQ&@@w;)9{SEa^342>- z9*b`x`Y^|`FP^m)jsaz7uc_bc+0AowNtm=H^aTY}1l#2zzcTv1Q0qR!+bot+aCa9~ z@R_*wV!J;IXul?%v*gIE3b(7eSG==(?pD3l=VCMSKH27hTHd=`0kI9H^vopw{N%w| z>FhJSBr(R-9ycD{V=+HUFP!&fl=Ew>a*dH_vK zb8|C+<^!sQjRB83Nw}ko>9B>-?TeT??DQElNkIe`V-Og}z ziL4`icxQt3Be0IxQPYfTAGeu#;sO6T@%*!+;=A`4<%-8N4ZUw+jw@i~V?11PC_%D< z=eB?jZxgSU%Ywc(^G3?bkYRbb(mHo3YeHe@@pJa?pDT(~ulXQfHWjP;TTit<2|O~& z-yOe)*-$!7`x6a6?Vb(F^i!(+G`xZ=cAM$Vg#^^>MODe9H@aKy^zF$uF_JmA=bmy= zm-C#{Y(2dYYw18AHDh1T?KJUg;-u4gk%eNnf;GPwl2SQ2)C4Ro?D8$FXWp6zzm!06 zTv<`Xq@YvMwI00$ED_>LV&Xpg=p~$nB%#jQpf5j}!nldU-iqu;*k1@aEdw|QgGeZ3 zDbFrdMhlV%w?O!-Y$t=B;kWXKykSZqDDz~~rNj4)qc>+oQ@rC*aGEb`cvFm98`rxa zEmf6E>YRBJd8Y@$l7+`ORgb z3$hLEHk;_gcj^G8+fK=FS>cxUMSt_DWBT8?`T8i8Oj;dAj_WE!6eRLLCHI5{euQDcybYq)+#9`9Q<*lZQG>WAE?Xl`L+y&*RI57+#<8{)ygJgQP?Qp)Dur z0={Po?t`YKbF4YNao&8|SNQ<r^!NjZo*TISo~Z~wRid@xj5N$_ zryRxFoVbPUD|NNKYCN<1#pW`iS%rTpi;+$&_s&W7&d0A*D@;9Gw2NNqW=`KSv!iI@ zl&a4K*;k$CKIL{sKj@5>Q;@sk;;xM`Qi=m2tYlqc49_V%Q5`s=kDA!xKxhd5>tulf z)ecuOwC-2BO(pwc>RHTwU-`Vx258V^ed$Z)>4 zDlJ^aVpf;=aZB>v98)pEf%1HY?SX_uY!jQ22a|H#Eh@>UJ{Fy?H=q0u>Z&WjR|HRzGKqgfnDWdTx`J2L)X}cOPIji>*?Rc6l$wMfe7a^1PbSK@= z#z^-qdxDCXB%dk@MTLxA{+w>`T`S0iKTteVL@GGj=8TxA+D`k^_vmj#=J8$UbQQ7X z4ODM;sdlwE{&MfwYEanF_sdF*^DnT1Iw-sGL0&*+OWY<`r+A-F_ipMLy}d7DXKcrI zM6!6};Z0^@2^+`1zC?FQKxPDwi^*!(+oHlN7JMv7r}DyxCO4va^m?;V1+|&8p&Eodaxk5AJ5!8e4nX$Ihu)jO3O$c!k?j}Qv~F#nmL(Fg}kufLPa00 z{-xD0-Y{wkxXP=g^<<;sRd@=5*Y0&PR7vZVmX;E^M9+P;Sa$NrH?b2$It-Z4y=R0b zUacviGQE~0j9^4*c$>Tr!%5po%Y_3iNB>4Qxy_@peIM8vAW}8GyZln>afh_4-xoNf z%57rWr5?Q7V7OtesM&jYqiWSAa_DiUxq7Mu9&(I?hi9H12@e(*Z5R!0*wwPK+{LC$ z*}RFp5y01^cupdX>N$*acqd$+_c(1G%I)_i7)@~MAd+wO6pGMcxBdB za;XN;ZmuuHo59i$sw^7gG(IBmAo_+eoY3ocb{tFnm`{;#n z^K_e;*~1C~xzn4WY2v?Pw^I7>)KmkH$)ipe%l*#{Iz2E-UB4IbOp;&_p#7M=yiz;% z@r(@Z5mxR?xRU1P=M9#f=?s!2>0NeEHs*Q1BkyvujDFXnF>pO#jLG^YpWtlIGw zb82^G{mMTp=Fb9i9()fJZwA86EClKaupG)2UOA}!;_`AJg%F<5e?9c^!@aZFa;&R9 zD@5-7+Nl4^p4H{nHHGI7kBLtcyF7`tU^$8k{~J?u^wp&{k*>GH8`Iav$K$`e*MI4m ze>0S}#o{HUrS)hlSa#Uy{XK3Y`s86wADVbdXcM6?57_ zJ+zB1CEMm#iPhSZ%$B^hRC&>_yk^?*dv-rAEg}sb)D^Nd)zjM75ma3NA^!9w@#BHC zuZ!wV|B`%aY0=uSS+&M0(TbNMPMoi4TY1)ou~HsV+4@Sh#_w`+nWb*da2N*% zXXc8VL3Q5x;SxurvqG<c~}vjRMK z>(nrwJ|2z*9%?<%7s%*pCMJHoJFti-cdlYdF9DBYgwJU8pA#Mtj9FNWQtrQ#O{`NV zLK`p@7*y!F3}F-GElO`CF%=Bl!`uL|Fc`vISC)us3 z`Bpoa8bB*-C&dw4T^FzoCzf)MZXcMiix;a-E!UmGEfN1+$jl6?OrM$5-oag`7U_bm zLk~pQ65*Io4Z9sIB0z}zSAX*@US2jcP0Rq$2c+ZG>gX#-kavc0tSjO6R4(un|LR?T zf0i6oD*Uo|zJ`d+_B@9r@qYRs81_4{(y-2_pTehiI7E4wkD$j51lhqfLW4<2V+f0k z!IfJEJ9_6AGuCGOr-4&Wsxy=nj#m6>*CF0=rVD_&d$W<%s}FUBkE`S+G_7w>N@DdU`emv z7MMoY-_MIS=L+}h>Z|7;sg<;?YeeL-KAvv-S<3O}Z)81%=4-}|p9LqXJM1b?Bxi}h z>soZ|c3N-25u#oKeLntB6H1C>;znXDqO1|%n$TnCWxadX%H5hDHW1&MWX--jcVSKT z)ztP^)eysFwA!-QJ8jkZZ_)J945g&F29!*ZuyR5HOK^`~uybNWM!se&k{XkBjusYI zR5rbz?mGLYOX*ih_ek6C$dRb*&!rJOl$0(u)q!rQ+eF@yud-;QZyegr*>zJz!rAz3 zj=A{IF-!5H=cW!%kKc(iVAA9p$koeNwtv)R`=YJyn{~*FLsDO@yuJiYl<*YyOJ6D8 z9l~-Wf5mh8lCW^^O_6a^N$D}0+_)U$#>ZIw$tNHjZ^@U=Bgt-FQXAH_OLJL(?k*FH zl9`6k{ro{UQQ5>d=N^h*y!5o@$F%&%<2Qyrn(1~766!Y?D}f^7f`!U>ceaNm+ec4A zE8Qo`tYKuV2X%#@*#KnyP@jUGfS5fmPsvp|T|egl%3Ckauh+nh{qVjXoTa-v)%2d1 z#z9z>Vom-2lMFLw-41C~T_UaF)k^m(J;dL?jke}$===Wt>WOFPU1omhVPe2JCr$%_ zslmC`y3-JWmq?wHq-Xp-_cn~;H%r!tU(DLd=)cA=eN%f8Tfx`REaob)H@oCx2L6l>48IfB zf=+U}r!=D{n>S#R@zem}kD`8#rzzpp{QN5S(MKm0$M_V@phE2|v>YImgm^lvU3H-# zL7z+b8T}`>u2#1u7%mjV3x*N=cy2vUH>{k*1GSFy>2@fLSu4r-_{*++1_S7D&qmANfvj5uExt406^~W7 zEUsbXawdVf6b|EX38*HB^~#m83nkB=g9`RN(-`*_!xXW{)3pLQ9^21uU!LCYm%KK? zv*J%YqsAp&gLk8qY#bej)ot!h)AA4l!RF{p;n|SUWeeMdv8SO-q)K>$Rx5kG*RRCRdVpS!IMG^1b=dOs(~TEiuN5u zO3FnqFKzj>ALw9zV4bx*r1gMAeI*i}W6uOQN2mfpBD;xoOZWgVUY1b&Q9}{!J;};p zW}NJMA|~VUZQ-wM!W#~G`FQ(~UzMJpVSD+%nLJ>!B*KSkzG{gj>Lo62 zuJ?I70OdKN9WCiF8YbE+ekAd^#KND8O&YfL_QcOTa`fnWo4pe^1f(h!62jpw|NFi* zZvD+HD}p`}LLIa(Uc4Z5G4Pxf{JhB~VcB1ogzMdz35>#r1}o<^4W!1xl)D09lxVKJO5emdRC1u3VEn>w@JKF zS?*+!xQ>CvaNy{cyEGM7d1)?MMu!ba-Tq8p&$`lfv%1#74Z>R0YmvDn$9HFm(S?hb zoS10;6~(Ua?4zvvY3+{424x!d5VFGIeWhGNi4g-Ey;jLT`UKzqs8=ItBmeaCw@Tzt zeg`s*50C!!cd&GFMk~imSBMiH)Zy4I3+Rg&S1?iI%7Il)ECInyHtPgxj61)vS;9G4 zorgzs#BScuBy`;1F5J#K@@OzZfC0rWJrz1zt_GFP*^nsF{v+c+F^=RXL?RIu4OpB7 zE-lH?8ir>I(HCYE2nrF2nUeJMuNOX$SkcVOORT75;5wda)8Z)t3O*nQ{*Xx2B1$(p ze_+CHBtJtlncUCWB6JWZB)|RA*(z+KAUBTInk(d|`(M)x1;0vCV_Y~tN8@SAhJ>Z) zSJpA6s|MvXZ#~#T$JaJ%Xa~irGiWSWhw@NS8eI(v;un2T93`H6F-xF&C5kq>(o(*y zg?IJqCW;=(s*F)dSARv$;(H9&`&NeEu0(cDy&u~uB&FVe@c%Qty^R=tClo(baHzrS z?PQ^)!swsr8 z{L|;h4+daOkW0Mo8CgF7@~yc>k4Cj%HdQ#~%<`XZ4e zn`b)1AzS(`a}=Vriv#r@K!h>T;QRq*#=I*W2b(93--3;X(aqbGe|(X9&`jK}P}38v zLHe+fI4eQoJ3HS%J1?86r(etwj3_cbwqgb;y z_d9pUclkD2eyH%FdT}Trtw%8k+eXx}bb|3C2!M{Cup&Srjw?YTNk^ZT9B(R!F~rFd z0Y+we>91bCyy28`vw0|_Ue14;hu;eu2xb#n*KO*-cde%{_k!i|dlhZX4l3UjWP*f6 z8SLYSPrhzF-ouGW!OpyXMq%A5&P(Fry>9Y?TcbFu!poJsc>K@Kxn$(z!7 zxA|#qFVRx9{Z_mYGaN8A^qhC4%XglM?5bqVCdHojbz2>q)bk3gMX2)b@nu#`O^;L0 zHe;R27X1Q&l+iN`d`y}p5>BNPeZlkbN0`m)4P|sc9!h{UV8Cy>YGv;?gA}i2`TmFz zhyMdm;kS?e(VF@#BRXPJ;sNPjn>l9@W+FfPS|Exuss`$4|Pa0Ymj~j{m<;C;x!jW z89025J=iN9Ui=@pgpf?dtYK~MogA^pg8!MJ?IXp$aqEy5)VsGFr>Fs1KODbF=_|Xb zhH9r-ULBX?QC7K<;zMsfjL>KIe7)c@_Kor3H}-B9+CT<7LnW1l0O4dFH?E1HGVb)z zeJP)HogL>oTo1fBZtq-F`|0p?Ui$cBsjoj>wLU%77$7h$9whw3n{1$}e%@ESdnftw z$BnYAZWR}Y#4x|~U95%F80i558LczegJP0vs&7iqN}SWa2o~zz6Nwsl0ni&E%wM?= zJ3|M)f8Su9h*R2!`UU4zFtV_$*-0Dx!2utC=sMM}fsY#`Ib^^tFn2+gLb&~hTo;R$ zuJq2ka5)~?;Vf0{VJVhyc-x;fxv3fW0nqQ=Jtm$tDvxeBjoQv!h=zP!!0!sA3}sRc z0rYW#2%Q3535ZDWC!@m-demcJ+DPw%)UilB0(U8)krYe7;vZ%W@6(Y+%!~wlVQtP@ zs(4Tdw>cEtN*UDH*tPB%or1k=5?fQRe*OBj%jMS_I%0Aos`Qb3-@@HpW@dUP_nDdD z8$%!RYt~#wzPq%|L($vWr`KTHLBbT&lDhxrr5ErR3ZAAhpw?)ApCIk@v{8M>0F@Gh zaDcX(GEc<@;ebg;S}GOF#9^x|vW4SWP0%IA9@#gNca8sS)$*|E4?Biee`+Imt_^t~ ztMt6BE_s(k$04JkZ?N|wmr(lJzNFc~2pWd}v}E{5(oY#NTX)Nr+V4$+jU1>0gcEqH zKSU4D{UQ!CQKnRw8UA8xsgkMTz>}t8pR5xq@@e(bdjF(O&XhyM)2!v&$ioX_TJOm{J2Lckn!L^;W&C1|{>#_1Y4Q)hh31Y4 zkv2;$+pw(MU*wt3@aONk8p<2W9=Y2`x$2UyP-4iyUkk-oHO4!YH{FWg8GyT(BD-|o zvn@X}J=EI@gq+wlKbLCl=W;vyh{SxoFDoEh5|Kn zw9vwu@N2dQLHzaXKCEB(Owo*4yz<)t&*~Ur+W|r=?9rD9WfFpbKCpU2cPZ3txQ~Ts zsbK2oyaAQV`izO0WiLdNfGvoLhiEPUtRbDQd|a&|*hQvvv3U>ww{$s|S{E)%goKU? zVkbDzi;mSw>#d7Pf%_3+;%}&=~zg$wd zv$5eT{SUEat#-9ny^ofaUz}f*wY{MF?U|7Wi<(R$hk5&PDifJD=IwiE9Gd-1lY+FB zy1rVC&+$e+_RYv0wxL$CW+va3mld;L_l-khx^qX-*Z}3}pI$p$8YmZ=Mz!VHwMkn4 z{|%d1_f>WHAnQVp1D+Jmee4y}8A!h=*HT{iwc|t%zyg|mSmuljl>B{`o-w&`H z9UQy#04%+2>8+f2-r;w%>*K}1-$0wx>JG5fVFTQemU^>kq zasI=xhpdMJ9Edq2WbC!XK<+-w(}H%3Y`PL&0bVZQuCI)9hpGQGu$nZyF1D=TpaF9-erHl8y*f->sHB#q*K6Jk|aP>{553o=6b(YvuO_+o6Y0jUJ^FR3m3NPb%5fA zbPiTy(WGVsjeu|Zz6?`$+i3!h#%x>aT=MvF)w)4sv8Y?PS*pB{NrMsSU=yveUI+B; z&?VTWKwaGCvx^blSGoG}QB!R4oJ zMD#NCNI)?(W6>~*3%W5ivLK_V06ZMc4!%XxDD8JSG$0|!7$G9w@Vlo9ZbnmvNerU9 z)){W^bL_%Y=zyRPR2x{k2jzZLG-O*FkaGs*E|I5?s#Am5BVhHC^ag?_D&P}?8w~*( zM$AKQqs6?-Hm0PV4~ImOb#yEC+Ip&w=!NoUFvuz8`P=c6#Y(f?+S9U={a9qr-bBaP zk5aI4X*L^5N6@G3O&s1!mlI?ZLph}-{Aq75OW)v_3Z*`S)nbVR)p>?HjZ%39hVM

#SNTo%B=K{&~ z-ReTpOUQucEBhwTjTO9=V;9{3}`GaDv#`{Ih84s9ux7+}?+V8Nj#CwG$G=WYLNd22C^3!en-jO@F&G zk)}Q3W|@R;`BCTH?lJfqQOik5f#7L~9TJs*Nhh&F13@PwiKHtcp*@oHxI5zniM<*? z1TJ!gpmT5F@kh|$5V1ksHEs`7i|X?$C=9G&Q@A zY%~#OD0@(^2)`Kd9Kl`!?;;e2_U33338TNfd`+n{&_7@`L_(R**x6G$dGoCiO!TM) zxY6-+Kmrc|F%iB=geb!2eD6enQ;K$6EtHPXPa#GLzOgj@k|$4s1WCg3+CV1oh!f=; zrqiFlq`9^{cNsvrKqengca|@-K2Wf@%>6b8Lg3#=306rBRSuSY0A3}KGUaqw!xIgx z87~^|ts@?Kk}(mf);ZeeebTk~l2o$7Q{jw^qj?4KZFDhV7uDDfKE5WzbRm-Gazk{= z)>8iuv+-n$V2pq3-}=kS$4Ez4o>g+&T#!b9T=4eeYZC`*Md_nYe!82)Bc&sCt4u=C z@v#2B_jcEBpEuOAXTKt0+nRGAP4bdrz<>mG^b7T^Uxj#1{r85Z?CWCHV2pC^n3#Rg zxz(|5-<)Tc^5^>>Pj_tpb1O0gXwC1R%ndGaQPc*=0MyReXUfX@;3KjMk7pix8+)-@ zM0vx8s~bq_DyGdtjYAF^#D4qrh>$i9OF<^$1SvmjPTI4s2y(vyRj`xO&nF?m>qG10 zvF`5f=n7wZ`=KWxb06T&YR%=Pfz2u~kOJbxlM{8e;w0q;%dlIPATpj zDJnAMPu(xds$=JN0Kh$1KpX^x)+M_@)}Yq+J}lhkiFN7-YigxL_AK+xTrZfh2~{U7 zT)V4mD(XwM$_)}MeXAs77c(KXfZ*oIzYhP z;s9Y__Sf_Ha?sw;7p><%A|t-N8vS>Tq^>+4&K2}yd-Z0*y7NTnfQX^fk=C=7N|wa4 z2&e7u-$2kp#hbm-8@v=8q&ZQYQ7Wu3850};J`5BU2rahm)Q7Dk*R&r;6(4f|FHr*V z7(wgWI{_~Y5$FJgPi`>l(d!ZMoA3}LN0d-t?;zg*?|%JnIsNuLKTxMYfl(5_wQ6Cq zQAxC$UR-86s95{RtY>^hB_$gfRFH`c5k?Pdc2|rwco0G3g4+_D*zs0he^D9x^!)WRiW5wGAjP(}N-7tsP2>Bqrqth?F|_iE$Z*j=-eN-9-50`L ze%t2zT(XyKPu1&a@&HM}a6x|FNd6iT0-sS0KFxaj!qOIj3vb`=*gKVdx>!kP!~5b7 z?MjTuquP719(h#z|4h9O<&C5fn7li(aHYK|iq^V#3vKh(kjVg&2_2Gj76>ESS~C6D)VtjmLzYTcd=0R_Km+u^K;PEo+E$(0XfAxfH&DTYl!`5 zvMaHZG8fdq1&D}*Nnd5!KdkY-HN&E-{(M_j|0qOD(#O*}uBUQ6m!PSc*}bYPu;pqr zMg9Pj=|7wGK3chn*~?9<9#Y9x-E(RKiGsA2ouJInnGiXuCr)4wltAw3Q;2rUI~D!q zJf6_#?$``O#haNyQBF_}gv@$=9-iA{iFHWLT$kTLyV5-5AWRe8Pyu-fWI+%+DzrIU z`&lA(YF#1?V*MhwJsb-nu(#(E}PEzL&E*C7srO$3CN zW*oB%be5RK5Q>8e%GH1bbpkT8uplguNRC9*3o!$3z(WYpNUfMlULM>e_@3Y)5xO&7 zt~f3yIev_2^*i6pC}049-Pti+GHOj0@B)T2NqWpuO>!9@2MkDYv9a%&;{%GWii?i8 zrabAkANiBKz{J&Cy-6X92BCqWs>_nRc5^%U;M#M7UtRv>{-D^K?vkfZnB37jvQOKI zGo`>PkJD^lphUi%X<(sqn;VyAdh}Sxf{J$4xdUxeY&TC!xbUf}Ca@Ui1@lGl{(qwE zW>E=aM&XS?Dp!Y>j(xg$DW82y8$U_KE8 zk!6kxiNn;9*|Ldaj2p_4Tb3+Rtm_Cjmxw-tl7TflQgCokKr&n5$u1c9RPKaSI%LQY zG9eg?}XdxvNxei#L3M~@yAJ^BXX4yG{Z-?5e)8aQ2fLBcNpa9S+k@;Nr_S*=1hfJ+M$ z7?8gB&bE19G$$6}fBvJubYp(Nz)r z*@Y2mhuAgCtQ)I}T$W?b$ZH+nXDE36W?tOK&Fv}rnjIn>LW&L6-=81z3uSwo$fkKH z&Wbm!tWaAjxFDv(44dpM{|hR39YON^fE+$Td%fhy20(d#UvkghRhuJ{IeG>V)1kQn zY!(UG8%T@^Bcdent4nS-HN`Z9;`!p=#kAPvnzH5Y-cwRp{%-o0Sab6((|nosWcqM^ zXgAW1q)nZ^I{HQ3+UrXyzmNXvpuuX8->bFvV{5}}%RDH|gfPIDFe3MHOqWHqp)K^8 zf6+U10$TU~lP~rKK{iE15v;_{D@u49%aH`x6+#GtL;<~3=O}R1UCqsQZr*lg4aJJV zCfl1Y(ZrK-4A{n$m3fFEUknl@(cyh|CRAzFL6@&+`=7Qx$nD3#-<*?V&Cp=2(N-{e zi2d-fD6Jgj{_$&?Le5!d#ZEM{Z0V-B{95#Nk%p(-PMW}bQX_Xu4<+v8NAxXF!cD3H z49prD9WsLE^`*{S4WuuR>d7L}HAD!%wd_RX`}6?M8qt+K#>n#GE5ZPA!xK*i_OuS; zgnD$s_HY$BVqaiS6^3FE$EfI0 z#@UChU5u|5j+VrG1!DyXUup|_Ts%q{WvMz9V=gtHAk=?u5rsGhVtfR25jFOG@mttl zdnpdcUl4O(HdfO0Tj0e`G^?40-=Ts6el-1e^bw{(IIVEK2$Bfq5%VVsBJ#bL{H84c z_k}OI#DzjyT#&+ZAzqM#KL@`@1F9j!63CiF<5lE*5cW_-Y`FW6mR(P;Lt*MNLgoln zFi;m_t!{kN;P7RLN~F=e9Nb83$|__1=daqHz=-Crg%!6MKlQCc~d7io6Xr;%lOv2lN-!vrF_sKw@X~R-u+{u_D{R00p#I1~3X6P$pdcnM z_lIQL==!GInjNl1CaNaMw8w>|7&rZNs*t?pp&rVw>Ok6flKp!ySrlP*WNIBcEp^XU z@785*Kjyv=#v_9z`#JeZVk+!KH{Zpzxtp2`z(E6MtLbu@8 zc=6Z8&)hLBw{E&R5zwdjkU1=Tn}v0q*;+G3KFP#JFxeSyzG=S)1OqUQ=ivJC;&S~o@h9DD&sDM3T z+9p^Ta9N=B(bN)pc(i>H8sT;QE~DceieWJMqZS{3cqwlnbl2C0gx|`$T&7hKs^Q@= zTNHhz-S^c4xmef&v9v`g-rivyAA}h!4P%IKQ>E=%3I6D;+wNY2959O-jIxx`$g|Gt5ufXYO-B_ zj|#IUy(a6@4et9s!82#g+OzAAKdfymcA9O_b)GkQbXbsu=G4uytk&7l&c*7jepOrj zCMnQB0c0S}rX#rpOlgR3&^y!5QaRxZyR9^@WAq+WnYM4lSc-ncA zSc1Hk%<=&*r&3%{u>ClvUR&dKZ!w4K$1leYy^R^`BYWr+lDV&Livni{w?Hd0{mtL# znjb!DuFIxPq6)vK;IMe7iLO{IbC;O+*E@G|%v;Pq2n%ef7l;xP_|idZvggj;bV(QA z<0cUw&!yiumws4G14GOIK39fxvqYKu-DB?`>_@tdy}xI4;A>xaqgPa-r!I!dq`_Z_ zpzhl9**SVf%(!5O-jFvppDl1MM6;mp?45l5%nfWr71}qNmvXp8K016$N6i4a{SO(= z8`xKZ+IR~>v5D33iD^gwImhUVHUH)4wQ4NKeXYD4vKE6aAH`k2owl0)V-;|~=tI=L z`OaHb&ZJE!%(+0Z=s(xG+U2x$UPp~+8OD#!J}z=j6$zKrWisoaiM`0RwRbBFY&A54 zty!s61^vQTvqhPUMWsY3G-8VsGTvFK`rGw+^95}^I^)ABqOEQnWYB1J&gH|enW^)E zAJ4@4e6h37Jaj+CUctkWMAdgLVyC^|+5TFxF!TJ5fxNOq%)z7vax=47`^DUmeqM!c zNaLkBp|NQN(}zq5q}nqvjaN<}FbJujpq+_U6@q7g3vAF|GI**0zr`(tIMuJ`A&W<& zO$4mMg^sd|2@w>kZpZSUU7*Mpm@D4Do^sA!y(LK>s_(Pp^}&Od6lb)PL$A%qp2c?D z@i2*z46@}W=ioU?;Ype-QEkNvUYmYz_c6X;8ytd^tgORy#nK0V92Mk1#FV{#DYtpG zsWjrMjw0v=Yhe!6Z{inMuA)`*X{g(}vw}hBhmiRy`~MxFbdhj==sDah;mkvT0{2#q z&06l|&R@n@F^Un04r0pB@8#V3ILAbxwP9WnOWiIQAUqs-Chqb&5*Y%iO-AA@A_=gq zh!>Zc+7=zLb#VbFsV;G(KgwtiIr~?)`U!W@!z59 z-$I3m#N8^JUmrRmQ=?rorLg>6u6n~lfx+VTl*4N5zDSMBX|kabr9*^`Ks{Q=qGa-qTxmk2JOM=oL99G0cC!E&@J)>a#AmZ`Srh<4}k~gwH z!4R}3g{>^IhivyKO{eFNZS;a!&o3xXjcua~zjvpJ(_B$UvpMF7$%EpbZE zZd4S;9bs|T&`hvr{#l|e&K4AMAPUaW!5p57);GU6!NnVkCB2unv~`JVY#}8Q}P!)f^|FJB+wC=3=&!rsSUaao);jAF$8%dKlpP03L4{` z8Lb&}k{I8qu<%_OGtX`1g+S#6^=NzL%=5Y^RjcWi#$)Jwz3Y2&>gYPGR%#=4tz_AjII->L4~Jnn6O%nkLFG7u6%d>q7If({$o z9lp{*a|&Am6o%bnNYX2M_5#Uzgw9$z)dN`>>BEYq;=i3Bmo7hV#ZL9pJy=&lM>aJF z`G>g6fBh929o;Uz`w>=(qXm4BC-iO;T?RBtLnBF8M|-~uWzs-{Qp-Cdi;)d)KWeJJ#~UvgHojm)+V&Vh^?9=05}4vfmtG~^OS!Y zLE;w|$0=wn_Y#vr^;X~PZ$X709Y!|y$;E#63#gu&9q3j8z-q@FH7LZfTW6B>;Ypq>w!+Z07 z{4WtPsDrsqnct$*{{4iDyE_IhrKP18zsG(XKH5Jrdr(){wjj6eLyFCbPbMF7zqg^g zeQuFiT2Tw7&Dpc51sBd4K9n#PNyKW?d+kof4Qbe(!IVZ9l+iq~wmLEM@Pco|+5*p- z05lPRqKKOs4XvM!`yQy{+^}kWk!`L2)MS$XV$yOK{L@Zbp?pR4>Wjr3wupvFZQX@4 zq(guw$g4gqagpo2FiE{(-nVqu3x6{sL3+KNNwP*P9G#ok_0DlM2+*i9+aGB=WoB=x zaiDN~PoTSnlOoMjz=qaMU^Sd-BvW?o@X=HWD`71tQKvGCHMC=~G}#M<&C&O!_Mh7S zcy$+x?yxk`PUc+4O(UE}%KTux7CvR`5Pwg5TV8gIthBqHO1P>;14*pW5xZT+as^5J z-;Eb~fU3!+g4hg$OdL?c-L{chTkPvgM0zbbbOAv1*9)*E>JwI|-r_#GJ->4lgfkQ# z>k%&62qXjmiCbx1Rzqe93)5$4E(iEkd*0jrI-&cusp(z-occ>?3#B8gv)Fd^>%f9_ zkXZiU4e^wMtX-V;`=ZCL(BBr-a^=}W@7P-uXRpGp;bnC3cBI0l<>;7rK}vJiy*olG z2W%QS^9oey)-ks>mV`R@;L1^{Sv8+dfzX-(k4DOdX4)*8nuVptbHDqh=2WgaQ6F0s ze||hc(6u}uxA8%7ka(p(O-3eI+H8}QheCI!Xf%x4WHy@{#n>ZjL8N?sUFHDsgPYwE znwAC^3$AW}AOKsC=lLWB3fwul&)v5E{vj(b6~ByaNtE$E6Hkjc0v1OpdSizaU1S(I z8|R4|=HrB+zL>(okq2X`>~ZBY%Qg%|$w6Rrri z`43d*|3z)v`V6P-LjdeNYXkn`@_GJrm`yg^G8>7Gq6kwSi2v*IQQtgWub87;Q`R0* zk@R7yQZBAm>MCidj^b5?G$d@}bdpbhDG2vxpwvAV^6hih*X{d?xOxVLt5<4iXnzi0 zaANq{8b~uNFU($kSSg+`HXVOs^e(5#Ue{euBX(z~)pKL3%v*d($$C_*#X`w9qsF{~ ztglYA##Oz#kV>wTCpNA|3#!O~0alAd6-;?kU7A@ z@tQlDDMD9XNPxd^$vnS2D+c=INeK9{Bv5(?QYHooc=8+OU#M<7Y_gNy ztdVH0o%8brx7z-yX|*1P8f~iaJxhLBK|U4ZOY>TdWn@pXx$_FpzaOx?ebBuQLWN=yBweEI7fO(kHbwstLxq zM3z9pJ1oW3NW$(rtlcR=w8N_<&$Xx{`E{_n68pf?JikHGH}NC{iQmL&9&Fj)JBUaO zd$8yjV~h?h?!#J*bTtFICLATUPNAY;}Sn`Emw8$tLdcQ+g?(}S~X9&F(>-bv6OJ(o4fkQI~Cmv1o z^%YG#7cLbny14s@*-_7*&dItHZ$~5@oZV78ZVzvbY0=9`UbEIcnoMHr>Z$*mmPk5t zR?l;{WsBH(-b6~N*emsw7b;(dXnWcPY@pq?K_#X&gjLrVW0(`(4otgh4mLd<9?wEPr{*NxhZslXL(U*9vMySGxb&5~#+ZSVFO)G;c4Ih3ZEl93^ z%y6K4z_Rf>%6EE4+s2=q{xmB6%R~QH1O9oug(`3rp=qFcp=oEJRDK{Nn3rd5E(9vo zR7g!e0hl=ELea!L(_JiYK5_zS_NmmCkFPI{2kW2xb866)_&E&^*0)1?6Z-?;rQNj- zw7eSsxDCHXZS-w>bMQ@_FFBPDZ};E@qc+FNpvw{Gm&(b5SI;Q*7&C9w zJNn+k*(mhFX~ErrHuZfEZ-`}Qnuy-h-s!w8I4V2$khba32V!)Y8Hjiz;;#>+x>#yRK^gZTMIN%!Jh%KQX{qepk0!%Sk|PZ3K{ zERLFqL1c(9@4iExPEJk^6k!3p|AKsLfkZvWjfSc!xGk*neJR55%Gh<7M@>8CCKAKeFBf9P9o6ACGK8GD}9uN>-8(nNjv=8WEvG2$>;! zWR&a`LR3^rl9iE4k|aslg%oa*&HwS%IiK(KyZ+a8I#-<|?)!be#`F1jKF05-oJL+% zcl*um_mfP*nrba*&hEdtcUF$)*@x?bp&?ZhQOAX*E@KbGXl6FK$*6bAAT+XR@w;H! zu~_9hf0vSK)ydqVTkRtOxQ^0WIT@*=HFp*YWn^U;DId%+RL<|d&_3zT#klmFT1{|> ze`bNZZ!Um2n^rt6rYO3=DJ<53Gv!>6&gUG1VuO2~w@r#oL``3Rqsuo+#VUKP4T_1@ zCvBY$^)3%uHmuxv?e{V<7oRRy9#PFe`>SI8RhE z;|=FkI~_Zop|Om1_ZME$LfNSUK1yNbDQ|9OIQuU$fQNHmXZj?yT5z}58<+d@{U%)= zufIK{U}&6EHRa%q1t$V>doT>)v3JqO=Lj4xX~@Ax!w~`xN@Fp0B&(MCJ`QD8W&zuq zlL4qbl*HK4z^Va@n|YF`U-BA-OJh5!dGVXwWXYmuX>3}{dEM>z$W3L#tm=m6^s1UO zA}XJ(cA7UF^@-kfTQuq1V2W&YH>)X$-PzzRC4FvBM6X;3>0u|mN_ZU;SL;EanFGNK z8CAB|WEog0WABZhNQ*62@1V{7ruig?#YO7fq4{B(jVViH)lc;=E7}d8qPVh9$-9HJ zb7YA~d0O~#Rw-L5E7ovBgUtQA6l3nEu$wV*0AUktOL0FU8eDaR?h5I>cw+2A=a&cb2pIQOTTjudL+SC9&cr2509FS_-r! z_im7?1E0IRKBL$$V{w;8ia~XHOd$FA5wlYw*K~V+Yi^mC$YP;*ZoR@5I`$>6cfxJ$ zrpkel`xd*IDfo}(Ggwuec-wU^((1NcxiLt~?X6I+?W%B!VXh=UmjT;*i$<(uxvKpgarq;wK{(Qk^Y9V*& z_NTR1GZOy18$coU61QM(oXj*ibbKF6ab&^hO-1q5+KRg`7qZ=~4j$sCvHC%&qnO=& za`cF?>D97?g4@CzNo$XKd_Ot`bJQ!+a8{CCs5|OUE! zSbVL2c(<$C=kI43;p~9hZ~T&+lKf!*ne5=|gFL@}8I|&OI1riGR}Y^(S4NBYhPD}t zWlc-|jz_}mN5x7BBaY{vU+lt?FZy^M9BfAg?Rq_HA(oV+*^c}C*rzI_3il9}(xR8^P zBS9r(UL^WOq$THW9u|&z&DugMPU*>6&R-J-%JR>DxxXtkHe%v2SpGWs@=_(KQ;0aV zgAsv#eP{@=-IqnqwgmH?fnj*64oC!I<)S&KDQwf+$tc)bT9qkB58e zOyA+;$X?#eHTgxQ#l>aP^uo{=ddob&k^LV><|6ecc=J}gem|aEg0nqXn=8))C26z; zjc+|Xz|f#-*m?Z7u(J8e?~tXri9sb-Gad@EvhI)tzt;`nb5abObh`}L5+!L~azst{ znyjzGi2W}ISK_01DP*<^`H{XA{#5kRy3=i=$nzp#+Xjo*c99Y-g9mTw?YLIwdi~QY zRuLhMn)o~;b8Bzg#Kdo=Nym>B?b%~2%r2#SmgJvNTEoQpcX{J`&r8JzKG%;gaB!`w zb0uJphW7YwM{=t}@1?cVs0wm6Qz)r$vQV_0yuT*K?Md`Msm~ku9?)QN(zT{b=2~%>n9xTW_`#wcB|EEki{?h zddlc^!+sUAhk8M(f|0qxk9^Pm(O2Mp;)A`AY7*r4S%vobI~i4F^$rsqwT5?{8c%H7 z>?W3?#S5&w%7JV*byb#}IZZWoOGDh&crvRo^E5*PAHqk)*fC6WHGFf2HEbaC!Q+x~ zYt9#Q7kdW7znS{c;1L7e%+yQOD|~aVOwMLXH0vyKE;~Aml~#TmU+Al8#SjF;F~7YT?3huE@&z2nt)xD@X0vokjRAcJ|{-jsL0^zGfl-zf5`b!nwg zg_R5}D3v`8A72cyc-iqoSw~GM^}g1wEpoKMV>ar~I<1wm6SM?CIR!I~C9-ku*l?}z zsQpDhKb^;?AojBA`9mzo=GHk-wKt>M24@_O4oCN>`hWfv(|lG>U;k;qnv$g-i84l0 z4dserAs-EmIft`c2;T+`6)ydE>AEHz%dc(CyKH6C2fTad-g_W^84zTnv;WLC_`SvP z9JGdane_v$^*p#EVj1anwC58!fhn0CabguX#)cd;eLO*?O@|o^X_52WC$N$N-NDulag@C zTPw+{4V3!qzKVA$*2be&EiXNPd54ZuQTGFv%x7g~Ms>e64N0rfz1hB&OG;-jvT}Uf z|8s_pRdQ4G=l9axW_p6`e=k}R6(->=(r`U#)=s$3QfeFZdH=nj;F6;qCV*ZTaQiJjxLPm3xk z3bA_WJ|UfNM|fqdXLAqCU3`Yq%s+ti%(S#?*BQRXC(LY4FBF+<6)k=WXYZg1n*8PYW=0Xx z&Fp~H1CU1$SMtAE4`>WYm$PB}i0gwei-^Bqtr}5kz{Y0YDYcK@d|xVcZ})HVw*@r9 zBZ71p>BpuT7ox{D*zpIubz4(Xs&(Gqx6!^i52a;_=qUKUflvBwoyiNJB93$j~j2Ohg_jB_-WubZx);=#21`aNzn-Va_c+Qboq9(NxYL9jozLmTziwwi0DW z&;|hNAV#R~AXQ&J!xa`?*PM4aZhpC=?bW%5NNIOS1L1Abc6(zS#C&LmK$QjBv`mT% zAl!wYpP8y=+X1W55pS<7A+nA97L9X~sr}v>O-1d$gAlC=zASKW5e?deV5=@L$qZ2yXOz@VraX(eMRhQ%`tDqD;rESo||}F4-3uV zvOn_M?QQ+WA5U#Hx{TXG>BRlFXP8gCx_PCb_*0tZ`*%|Nn7&BXFj|>}tk~2Yir^LR zo8w{%=YDIBA~mzsz;si?J1O009ogMW9)AmW|NdN&A5GSgv#pzAW8+32oo3*wO7eIh zM&F}nt-v^Dl7<5%=LCwQn8%%_$It_q{=*u*>*Dl&DN@*i{@R!_X>#>zZFKD#JeM}s z1uRIcg<@K_wh#QoQeU>ei;-Ei>bby~sV(`!N6ZdQ+%n?k=0WgZimsqA#Vx0VTPxEF zqtpYrQ=gP~|LK%rQ|o&7sFeL#NZ<(dBkCfKQ(F!rak@$k&mFYroV(1U=!FpBima9|nR&ud==VLpd-Gub z!##QiMpsvAm!8$0#piwD3B1sxRZq~6jNY8vi;D>1kdDsIGX$7)4h4;8uXi|sJ>u6P zy3L`?mN_C@wGV3dzTTUrpIYG-AY1Nqwntm6qyEN)$-zx~ z(_6(u=--ZCdbYe2ur|5SxB6u5mq%&Aza3q~{uJC0xxiV8R0DF1Ig#GQ5PtRWPV%l@ znU7w|$rcnzrPgHhFwoM{aBQDxVAe1+V58JERWUTkQ%$)oD5$u5Q90df!=WiJuCml- zo9}~^WCc0b9_h&F9^7rl$jc<<^>#5$@`M?-d&*mL$Ck!&2BY%FG^ge{6mIUO3lUC% z9LWJF@Hq9GgV}rx*`KGnKkS=M73el0SKCySo%P;c7DXxw2vG7vRR??}Y8V^fnfMZ8 z*6g(GAab1Z_!CUEz(%lsOj=s`Di1NyqK0d;ZUzbv= z3#4AEM(_&t&7mnqPY>}E=+|g_{z^Lm7sJCETC-h=N1(q*E2y66WLK>^7j`~g*4X5_ zt>(nl0l(Lu(l>`Ze|o?^u8>o#fKfQEjh;%!B4gy%t1q+gmky1VSSOs=!vD>{66aDbA|g2r+a3gquJ4S@^p zN7_6PL;KkiUNy8Z~r*S7dVyLMHe)46y{K=l{_7((IZPKm74YMG2b&aH0+#udTjq< zDsM>iK6+Kbjsp#28D{d@(pJ|mv805>PA-i|Q?>g9mpH&^vsf6{$OyG-MgQ`__oj&< zw(JTeBSzpff`$e9^6xVTpw9&tb#+=|?fR2mpXiHE zua=C16?KXIc07}Lz~{H&bc#5dv(cz{PxX^!`!9+FEOulgctFByeu_xLSn$IO`HS#L z!zs#}=l-adpPwIhcxG%|99WAdlGwuO1jUA!yatJA((?e7y+e0zRku(e`x zp~}`m$U#WzO|r6w;d*0CLmHGJlGzPKlSPtu54&k_k3e*Q3r(WR4;z+<1xjSLa-fM7V`C-s@TSlrKBM9WT5)1`LbQ`sAeFAi6%(c-wDUb}xTEM5+tJ%mbXC2%{kPBc zFOS4#jqIIK^pmKQ@eorx++|`hyjP`W>j~MIvGkY@642C}vD&?S~HOspC>< zAt6`v4^^c6*g!Ebl)F8&^wMvpAZE7ETW90j1YYGkGVHtAH%TOwdMTnXMQqyo6ldm? zd?A9?a)r@Ecqzin3ZNaPVnD{+FrO!6SP(qHdYKOr5FNobr4-ms?3>^b$+Yx0%hjf| z0z!jA5Ggoi-B;V~33deJ;a_UkEiItcJfNfm01t#WGV~Vp9{A{OBUC93?-P@ur%zuI zQ*plqX;9e)e?93fFc3a-GRge_yXpqJOh&^8?$H+?2A2G|^@g=oJ8r_U&PB;;@adb& zYmV_f)@jd;g*Jsm(^6Gdj%LuFkGq9i1UALw4P9D zn`(`UrinI_2E&m@ixJShV_VGk!-)<)Znh^+B<(?Ax?ZgNpUJId5{U|OIC?fJxA zvC~W3yxT92f71x?HQKkSLy{w#tf1q=`J-qZJ|wZPCAF~8g|Za6M)3Lr z3psfwSzq6o`nr?qX4W`A_jHBJ!c)q2HhuWMpg&-SdR~M)WfC#xgDi zG}_A-#+&P0ygh7mRcLR)UXJAWsh|EKAr$^v%wnokX}LE=UK=al()rAuXcndOUf_4L zyMhzXFGZjYU-a5fxQHUA!Onx5+I|2p%S7daOPu@14gmi#fz#i zpNA?C+*#T5Cf5u(M?T(ns}<_Cd9dKE6i0PIaNuNedw`;MJ%x}Rc|6}|+LT9t1Yn#9 zvLu6=ySY%}5h!<1Opx4;7I4wh(OC??_`O{Kp8>Pe8jl26g4XfJ==S(hNE`Hi(Aq_oQmu%In~L%kvTs&1Pee9 zQtfItp*-E8ol5&&lf0R;J+qbb8P5e~W=h)n`O4R!%xbD#BD=&M7F;R1WBHVq)|@{l zot1@3L`~?Q)>pcU^2W(I+av>GaeN4W0{(8$KAdl~To+;$F+%xf#8_}>GoIAKmt8ah z6=LfPsgWtQWcFCs#<_k$BPNy~uPbd6Q8jB5Ke#oU}M; z>)@U0S93@-(LzTiv28<>ejNl!&}3bBvh*Q)O@FOp4e4T~zWe^)rop2|SOPn_Z3MZo z42O4u1sw)BGc@55fK$`VEff+td?$!|OV^mROtgJ9XLSR4!yw0H7Juz5V@md`>6pH; zm11VcM9dhYCZ-4oxOic7W!3(Ovz)EG+kx!sDZWZ2<3r*6VI|h1TZK8!au%1ww+NgR zwbR~8H}p2XqhkwW924lAAXwoHdf>G5ID}qCcL*PwU~-G=#f$YTOLMsONwzSl1Pu-C zq|(Ch78K0zjAiU9UDve{GD(O|ADmF{u$WR|R+a<6n%xE-JNyFm^z^)bk8UPTAqv z&*`)+$?6&3JmYZUx+Jr*WRk66(T#4OB`V&P!Q|yZj@n$a5&)Q`zMcQV1C~4KIEm*c zn%Mr^d5$X+8r|1yA3>PuJ7MQ^Iid5h2Nd<%g4xjv_-*6a%)k=JhSl4={@0`8EmZtb2&Gg4Jtm+4CM1_T`s9aeiT_9JbhBfegi9I z*UV^~;G@c&i+Lj7{L)(UcNG32i<|?#0Lfj)8>z=s_Y%F>gji_ z!96#(NHlHw_mKInw2M@u#Yc>~XzSxgeCZ#Z$fCO)6;!O`dyth3rkvwzWP)Bcr}EL8#5>TE^-{w`E#dsVFef1-ZhMqw{cUCj;YblRbn7Jt-HjGD za2K(6()DPAqC5VG^}xCGx?a9(hD@qXnu%b;^m{rqG3SpSHQBvyorJ#Gv;&B#sp(Ly z@+-T*!NPN3R;>-K%_|eWBS3(2jf#lOI{0vnTkQL|?@FH25~N?+|5B zS+piEyC|I%r?{Y4Nc-673rrEw9v~yjq;~uyVASCb?PGVmFU9q6XIvWn{i32IoJ0Jz zhgsX%3GWb<2&>T(c4Wa#kHVP0zI!Q0`}!$szTygV6NAz!5HVZ8IrtBLFO5V1yJWMXxEy9<&4#(7J)h z2mO@YUsKcdgtNx&x7q^wmxjY@2Rl!?H(%x!pb3Hl@YH4AzB!VjgufqkFd5D$40cE) zDYVDyEtf3LGxW)*$3Q7q&*mmI67ZLJ@>#qx7;HBuh+~WlU9aEbSYhd-XoKH=$N_|t z5xzn;y?UZmqLHaSA>ffOOuU}Y(H0ojm(i+lQWaa>%-lUVv!ZeT?TjLTteL8LkME|^ zx(SFrq~`UjgG_zU;5im|s49PN-)N<{F+Fg6XsaFx9geRa{?=e!Uw22W*OG!OH|N4d z$34A{(Q~o#i9c9QpLTYe$(-{V`8s#9=Tn`l5y@Bk?;2=iN-i!w5lNA%lLS4jlXvKt zjb`&?|I$RaYweYi@!@FiBcamY&L^Joi2U)MrSp6Yyes9?cV)g7TV?+y_o;jnU%+ve z!8eEtK#C~twExbk<(FD$m|;o=i%T={GWf5#NZ#zUm*m>?;G&4McC`EZdg*2sd5`B z0)HRX!X6k=VdLNKJddkHr^9c7Y1++za93^tbOIy~EP(8oyL$}p9fKfiXnsCLuTTwP zVhD+voWr$Dnnfv&u2zVG)uy-ethL@KgWaUJE~CX$M(wHXs`<0Tx7>A`9TtoP*~x zAjYp>DPnm0Liulv~0#* z@Ak2o0x?R|d2jvn`mSs}W}m%VXN^&tpm4o=@7{y*a?BqaU9Xqszf{_~?T!Pa(B?~`U)!_G&EL6YW+Woq!~$qr52+}3JLP5 zev9K~eHIwwC91-zi2uBvUb7kBOI%fnR}3-%xNqP*BQkr55JAx8K@s(2U8|{#gi|g! z&v3{kj-%|gy9!el%Y!=572XXY~M?P|3zwc1lx!=9@*)1{iACM+Mpb7?#%l zSnO>?8>3R&@QSoKQ;$B7uotVWBqC~MTDvU7HdAiG00&eK_~?7vrUK=Gyn`5=qj@_V z$J0LMj!VZ?Wzk0Z`HUhCov#<>>MuI=6r4iJ0EVQZh}E250i)#cU+u@}blC1y|S+x-9RML~~}4!W{xPmX;I!7!kKw!_wzvdhEP z=gRym1xthF;V%8Gf>!nw33~1?3qh${Zx+5EE;TzhmRx&l>Nz#ZBfNnhf-4;82%vx* zACaFNr`m4wb-Qx2L{emp;7yB%umkaf*9&nn(`SW)VsmN_W>=tO47kiEs*VeFHAyl! z_PlWAhQWztO!b-fCg&}FhQ8uq@ylH6=_8Txh%pQU@m6iai1@t07*%Svv>9#4h zIq4P}Yv#3wj;SLwkC85mNd~}-LmJ6RK2}XWGge6NU}?9bCSD$n1QZ@R9k&dGA;5?4 zO@rjsd0Aj9Cs9^zZ0IVfX$t3E$vFL_*Y@%ZOOw&q|J8m955=NYAH3}R=X8jbmWM;6 zyJ*wa%LjbH`0d6K9ZhH~{zTUXU>&I*vsxdmE zD$-WD!V(t3cke_G`fgB`+x>Q~_A;roAjs*5TzRCGJwx+e&5i?@6)MijgxH<*b2*Ud z>%Iv4aq~-Yr)-R5aeC?n)LZ2py|cf5hTYuCT)9qRxAieLN>kH!?uj<q6M^rJHcr-m&>KBfZAXLjHgW`K%qr|OC{{`o+n%Szk>StQ+M~k z@835tQ2Y?o+U@uP>_IqHz)jNKJg$Kv$rg#DFW0TIFz-O;jLHes^Ll>(s}o`o+gTf` z%cZyCf@Yu!t~lemenpSX`aJ6SRdKVtV+>)WSDMNUWJ+0^kb{Q5xB8F1;DyCAGh{0d-b;vH2$gC zx;uQ*>~q`TZHhv}ti~^u=sPfNKazjFsyxCOP!`Gs2%ixH^02zP6;yqco7iq3h1{3> zE}Gmzn57ZGAA&DYZY}sJt!`Y}k0CvtEU-N$UL_k(Pa-7)R~Zfg4)RTg5rTj1jUX{2 zbes5@iHV8M9)73^7GRq2lHevKA%j>vc?iKSgHSxLNf5m})^gNFbV@H}_JHv0TpNv*0tZ|>w(IGv%6RXSpFPcx%w{_PgD+7 z+$@sJh-ZRb4af|yY~GnBf@77`v*%%xs6)?=%t!ZG=Ng!TCZlJV0CgDjs@Z++|e z;N4O3==_6cf7C5hs+ya=lLJ~YCL=L_2Sp^X6tEyAa(BF-~VK+ zm>CUuB70trHsO>=gOSKlPsyD2i7o0HsqO5B?1eX+4Br&o9`3M75Nyrc$>XSu7y_s^ z!G0&6jS<)yoXSy#2R% zmxdwxkmEjGHPcS7?h%Dq&F$`a7b;OntjaH*WYt5JLo(S&+-4iiEvp_mq_KN`-^Z}A zLGqo z3yhoLrS;T(j0JZjW=S;|4$)J#_n9(Nz-So@4f~jIW96ff)ZNnZ@SVp&@!mg#;b6N* zCJ4F^|G+PtsF84uxy}9{;?Y_44GdVng+F0hi-lAYNaXp)@QxTkt&PqChgl)>3NyKl zGaaUqt>-c_GT@)oTK#O!YidfxY^skSVDGP4{%b9Zz%&H^VYrFI9yVy`>UFM}% zKJZ{SLqpS2&={+y?9L4gU+bi|*r%<&nfqD_&GF6N+iq(Une?T;Bl&x|&Uf0DASmmt zFR#nWBD~q?J~9bZ{RJpP<|kxK6_ylYHDQ;^&`?&}Vo&Oe9(MZ|2*(zT;AN-fq(QEh zYlepByd!(dLwlJYkJ2`YTe`ZkP!$1ZlpOOlUM`E=7@Zb563595WHFFeOm#ysh`^$i z1!Qzq%q+qKdy}3C&mZs9*sLY*a9y`suAEXBG!~d4XK_@ys65D3WRwBhMK9pZN-$NE zW0cLkLsAJg`5Aeg+uMI^GGyDuxHm+Raoa6p*-o>li?Wfi`gdHxZ-qg6Gae{fX6VXx zCbz6eZlqEauQxQERW%GBlEcjprz${6t|Xto$>P8cVZA7K@F0w~-jC0^n9r$7`I;Q! z1wEoznv=o)2^jXSxiqF=&cf(M>5w=y5jAN-65&^i zp^bJS`}5$C%!r0<$^z5WwEXOEYZ*o2JWoVlPMWr<>%W^~WNLWgXAGN0q#D(=nVh@M zGrsI>ZknY{xD(-xLd<{u=7+=L3a^Hn#h0tLxxF#@`5ET$0Tf4&Cm>^+TPW*@Bwk|? zVH8_keGtQ2!eSq{#j!hV&cUOPC!42F0po{gH8?XkvcdPoK{*bj5}~I8`}!8!3~nataK#TqpakPk57?ED()z#-T2QiuUPnI zHKWFYcOYCkSLd6Rrx8T;@X3?Im`XKdqT13C96Bq;$0$%)g8bvFLb@7cKwC7JDe&WQ zmeu~5lahZCDJblardiu(hnai2?yf1J4WLsZD{+~Us5zultKL0a9=p-L%JZvlHB7Hx zI{TSdqUh@Dt#|L`WrbR_QAf6gXS9W9W-CrQBp;}~E7b5xpK5c~m;E(Smm52J-7@<6 zmTL5oB;WDtJud^_1_IZAWz8x0dX78L{)7J0l?A(>2R~OcOL}oDbrqQ;efsPZeR{H{ zx%Kqq{Tgptp~NVOo$I|wh?g5i!MUR90ZFS5{F46TjG#ZaB+ym^U(NI9JsR;6b2@bm~?5+1$xI2by-< zx8aQaq+7v4(dyZkfb%6H%A_d~f&s;KCbu-Tc?%o>XMP82=3B((9M>z9-Bsd8s zw3!e4uAUG|quZe_m@u$1*GKI&FcPw#r(2AdZ zBg)-YdBuH`HBezPLL#@~EI!f2k*)E`<|0>#S@r9Co_#}K=X&jC-m^`I*3VVO{$FqX z-|u)50z^jn{YGUAHcpS8Fb3J0o^s4FJxzsPYqLS zIKg^|F}y)8-laxP_{Zy7FNV&F26uOVcMB64R`Usm&}lrI%p7VYWqtClrqWd)a@(gBA=otPA z>R?2M7M^ii2Gd#fo`Gf)o!uc`v$-zDGN$KP>^l6NwKZC zW%Y~0L_a=R)%N21E~ByEap!AA*N^f>{j7_B{IzIEN6-U2`w9@dcEa>OPC04k4zi78 z%+!R@JlM1Zu}-s0`$>o3HPQHTUC{-ZXNPqQ-qd!+Z!k5bY0$`NaECkwYRp3W;v@2@ zH*a{{7t3sSWVaMzh%DTiQEI!_Bj)PavXi^xU0J;R4;3mGbnx!wY@;DF3FX-18x_T+ zs#+!4nC+eMXz|7m^VbK$jl^3oinRFLJ9O^`f3jY#gQ#1V5XD82^dm~TFFvN~FMg?o z6d&@2#Q923ym)2k5i_s*ArZxKj8>9ndqz4hkWSTcX2CQ9P{n@s96%o~g za(aG#SfZe8;Cr9~{o5-%1PUye@bvyqNrKFgk-<$=FT_#rhG=n6PH>W8%rk??L=r~; ziGx5o#MoWAM|RhqCMfr{2xf{d^!ieBnHXWC{YRVr*Ps~Q6zFq+>3>Z>MneiSHceFD zLC~guJpO0-XPzZo8=l>Dw%B1Q{L+{CFX#@?`~VJ)$$&NA)rNq1CVX5Q@H*!$SCiL5 z@7=riKh+}*f3FMr><8}hNZQ4?k2wrzbNUO1Oh1Q6lO0Fz-DoP;SzNqP7hSniiIH3} zx<1!Hc}lgQ0=)CBU->)z(i}gZE6UjAkp5e!$d`ZfdC*82}0sN@?hyojAwX2*CPT`Mac zoFYfOUE|N-b57NPV;LsKKp_>s5m@88&KS4|a=meju6M&;`2*x9+#kfs%b|#K7IQzK zoX@v7XBtW$D7Oql`%v;;Tt4BUxk0K3sIrj;#l3?5fkWCl{^`m&;AeLX{DRGC1qd`l zqyOL77#7)h&bN;h+dSk#Z*A%8uSN7v+~|l2(*U@swav^AC+K6voC^rZhUVh)FIh+? z>rJ@(^Vx@#|LL0%rCV*&Z(OMCfE@PMqVz;Z2T!3zY4hqTtbvH(4A3p*rS3KYA_EBV zsr>SQciy;skqKSPLxj0vxo_{9b;%Hj9WpynzK&^Pq>1DddXz2dwSC~*xFP{*_1W4) z?GRxMfk`^3OAcxJ=7&3EU^oS)4Irr#78c)2*A`0`!UN{1YnG>v0<$CFDTLEtnDrt- z3oA?mJ1O7jJIb5y4RQX@OV_$BuS>9ME0IotpY!)3kFZK0NRXDD;jdZ#z(8gDHU`NB=7hoFuJK( zM|Dnt?QE#Zp#VkO$wzAMrNOgqX%gG^ct;k;JD#hgjoI=}f1lwO!VKs`-!?~EnhklAS2ztDX$$IXUH#1fABO2rEnL7xUtc71{;n%~2 zHs_%D+vBXevZOaOZs9?N4Du~|xNdICVag|AR*h?E1zc-Cmm7oBcc6xbsVM?*usBJz zxV9OY1QMdjn{1@Ltrnjz7^pmYix$EOt%dDL;L=z%5LdwC9rNIdzZ^YlSeN@L>A%*t zIu)m;t1Q$08wV)(4fP)8tEe*1q@ClI-v(Y$c!O7fTknF)XFy0$?~c!o&icSG5WQnn zT)Rw+`VLoq;bX$A_nj274$X+EFcT9gMz{vY%wGZyS@Mj=Vr!TRB2+Nk5@Sa(0;&c> zLmkl|ivSiw2n38p5E?WHUKogFV2)sP1V9s>05Lur6&#wMSi->+>k)@Jdhn8QP~K5o z1l|WJ_55QD>jBsnUMMYW-3FpTIX$Iu&04G=~f=EL6*wydsrhTpr*L zlIMZ=q;H3f5WTwDRD;X!vDEM3jQCCFhamt~o z8|HSuxg&U-5wPf_&?|9x>8nt-=SX&d3bWs?I+lZf0LbE95xWM{6lmhnMi9B>1p9J& z`XV=-*T>0|?tQtDJZS|4ffHr=y<@xmFZ}xX^Zvt!s_T7zO^Ky01o4OD6LL#7j~f%CX&8fcIT_#m@Un0(tF-K|tXE%VQ=gGB=Q_0Qs}J137dYVc;<-ck#gEUm zPYv&7geAZmbEY;X&Z|Nc7Y;FJcwAAY-zKm{enIErW_K+?a5QA()qZ4>b#yP^d9ol~ zPJfe}j4o@ugI8D1X_K7OoVEh{okZ@W9EkVcnDHvp+~1RM@vYT|1Nr_J-(VuTvKj~m zT(8eKacW{biN(&rmleRBt6Iiue9hCO>W_?}Rqcp3fF4j=!2UtQq^}S`PNSct3?Toi z5JOmS#k;+gKi6khNCrA`#gdgB&_b8TkO!El?bCV8>YQODZ~fg@O-%POlB=skjryk- z&@yvSYkrMtqo!wQh=^XxjaMazRd?z6gKi(agx{9`s+f5~%(*t}@AR5x{d$S@2&xwk z@%;VA&KhrtLkMSxy^%p2~7xkLx7mz#1TYHv>1gQksGPnFN30hqT(W!8lEP)RwAWQZg_Y&q;G9?&Trvc zGVH9{FFXNHE_NdoA@Gz=y}#tb(cn|XM3`Ce`p+YfWLGa!7{MKej@I((c|y!&DA%DdjrF*5pUEPxNYWpYFAF9i-q^l88jJq z^Cjbc)31aG61u$%$XrPB1CfTI!7F?lr}AfGgIm!`>j1wUtwDtwMt=a;90p@Miz=jJQ0zR}wJqwhpC*?8RY0W zKRgrT*n8t5W9Zd}k3N)bb6k3M(LrS{VdHz~-ufvY>XP{xqAfLU@LK*v*b95xG@qc=p!M(gXB*#%X8g9kOk(@1|wSTuF|pOEfp4@ zHw*DqI>XGi0;hv0_q>TtV&r?1@7jIW44W@xC*?2G@mquoDP(U813Ttw^e5mum}!7f z>@q88NEEyHW86k-mj`vqbR`-i)GBrE^51~P;E*K}=>8oI4i5`3HfWGFNHU;a#`DI; zK^p16kHkA>wXT^2S#NwLA@m2r$hjH1zjl91&{rNPYg0OlQA9$*7M#>sBTdb0ARdFA zfXTq1-g0G$WeUv{jI-A<7Zl}wM6?()1#zFCa>FIj^WsnF*@OWXt*2%dAyjYwlmcuU zb0-Fs1koA4?0kI{7&+gJL>C#i*T{a>&+h95e0e0l;?l3JK%c&UZ%xN_2;B|tJj_b` z;g<)Tvl)S`eUk}oBQW(Q`dHA>06tE4vQC!n{V^wLa+~ephtq1GS7?V`EuY?_EEvt_~bZ?$9DgR zTSyDWCR$#r?lsS*7Fskyb4G-CGL$D@<*LIxqei{dICcnL^Xu*SCnhrbgOocEycLp0EO4p#sNPYY%WfH#tC zbBaYlMBN(mskiq#;l@5cUc34a7`SEKjt|DgIX7twetE&be!GL-l7Z#NWRFbS=`f0?^EXFVtZFg5U2|C#i@ zM&kwqY*i_Tk1!?0HV-vP`IKpw!kkYL!=MHwaNc^#YiDMZDklv$Knx{DE?p?ZaJb<( z2@>!5Yh^k(ZyI~{UvPE$CwfkKp6j1~#^w5UnEmA%5red5?3fyoG%iH1uXjZL?T!XF z37Wnk_cD9O>T&h!r%(O;{lq3tyC?hWSai3S)~@VavzIO(es=K2xK_vC{x~syK9`l! zfP-Tyc!To<%zzT~PTjiB-l>{qq|`dwy0suDL09O&1^2O-f{K#E_0pJG-*9udPMdHk zK>f~sWGqUiKbsE`7QU_6rx-fHG!Pn^%eueYM1{&ETD5#P1M)*)p@p+6}(Et!< zCR4!tMuI+u88D`0s9FJILLUI*Fm1sEkX-WG`%%!~EJl3+yaYWZupkVYZKPo?je3om zxv+yky`RlZOcN(}|MQI|rZG>}u#LMN*)+S2@%%g=C%c65Y@FAv_R-a>a5dL;bnX5 z4l$ZQLY*JCZJHn;IM@EV8ASj`_>$$-x!MulH4-l&q-hw4N975y7E&B!0?E-J02a;< z1PJftrS+sIMo0)bRDubyLTc*cs3{3e7NNaZr#GS4Bz!nNmE$@YB9eUe>_Lr+uO_ix zDOLEBj@jP^xEL~nkvjisWCp*TTwW>^rYm=XARO!lY%olGs5WBH;nu+;1TJ-cRE>r| zOYoW&?1)KaaZ1)smxG~Nkk$`*Xi+4&n*A>tKtsx#l7K*=e{HdLwbX7^F}dZ{@Bh)0 zB-F{N2Ehn~C@Usr6CpP*Z~%s*r>8&qXzPvFcRtW@@#H(6swPwE-wd}b`76bWb{@6l zA%LR(=qaRxy*bx8UPe8I<&MmQCK~*SkAw>n?B`_|6_&^J0|o(a zW7XiA#?2qj#EbDU1hjbXm@a~ro8C$T60rr7lcmqXp>VbRW{!7*>0%>NDnj;zQ?=8RGa}awAcpPx?=y7;lNvFCHQ&0kH zk5gVv!O}M|wMAKNx$@)q@=rTu-(j`bz@KYkoXFSdPHQzvu8-rw)}F5 zCai`=8!vcB4$zP*I1uP`_3*}V`$NJ3x%WH~Aet3GJqy1Kh@CMu>=(}-(IKy}Z z4H&MpnboDqnQ_atG{BEubU`EDM6FMN;6!smfSwrOV*6p3LP%u#F`NQ10|v2D3_G*l zF-HpSO)22gx0+90o3@9K8q>JA;R zU1a0+n<9IB>X)hk zG?JIMmQ5$&I|A<}R7h3cdzDP@m?8iwG;nO8NbA~yAD%{>nuKA*4&X7bYzF&VJZ(9t zuwO8xX1)SKb)hk?QDk+ScLlKW@A&`tdJ}l8(=L3Rqy?c8D*H$!NeEd}B!v7DBe)bw6h2o%#L$pI@K%ooQxNp6~a*&$-TZ zu5%s1*Asu3HhqkY(v;Wyk@@Y8|IL5cvLW$^Dqw($xw$z#?>UvZu=$SulSF1sW7F*9 zmU9PVxT-A6_VK56SPhQ#Ko*&suV0KyDr0#3^2pV-v8O-ZYti}UZf0?r;u^R9t|(Jp zvT2{ZsB(+VRlc`X8+L8k$x*jFIW;WVTA$fE^6FZ#I)Q#IgJ+#FdCFPlwY;HgwJqL$ zZ<%-=$FV|MTUBoj5!vt1GIJ0*uC?*K%ZjFx&aN)TPym$M$S`oKpTG)41Msiq>W1SN zdFx$3LOEn4n;S@$?*diNwqO4Y|C-MzDh}R)D&u;qq)3{g1rB=#0qIo-Mi;+i#&czw5pOZYPuX^)k3rx0HQRBsCwO`Stl*6y{F!V zL){dnNf$4cR#k0pKF?)bPa-X7UcT(}%~#9TTulU>4q{fj4uUMikI*o#dhBZyUbS5G z)_z5oZ`Rb!)Y10oY(#yspoTQWFao9XNI%b>1t;isGWPMJF@}N|>4;Y)6|dZ5vMtDC ztX6|`!o@3bBU}aVyEu;Dev#Nf*5}PYbe{F zec=>iPD)xE9ebLzrJ@Z>hmgw)G+s>)Ub=7Vcle!Pf~Mr{cQ732 z{yk}LRK(tdi>^7FTG;j^=A}d(>EK>0c>A%5^}4kiRkz0`o~R7hir!WE)n#OsPMfU% zg?Ki>r78|obe3&WzewOC3dj~6lXW6k{AwkQB|Ie4;$i0+SwKPWp&XXD=|dZy!)vy{ z#7%b2mrb|>LE{|c=mT6Tfq6NA3<3bemYoN^f3GIUb*H@iiN$=?N8mqO0!+~U?su(o zA_V*(`H`rR^%iFoKjrZb04KrYr>4hz zlJ!d6p0=6ML2Dp?QYsbAji+@C{E%b4o$T2)Sv9JN506hp+(fXz6PjGi_Rt0<@TcK| zr1rPFhm}3Ss@ob11OqGy#Q4vrv)H@|OasN>{s)IW+^e4h+moma*L&MolrwN3LdryF z#WY{9vpmX4g48gPLx#`$KEc|BEa>jggkSrqhV!*~G0B*;`U8^`2A`P&JQ62>8HM8S zv)d`OmW^{Z0eWC;pktFasM5R+tn*ixt255^3UmQ zRuX@4HrsG)TIqr-zvK^BO<`vJpPS!INe{F{&zblpmI$faAALKfrmeRRW7vhCr5URw z9&*$@{q*zck>}i3slB~|37)^#7Mx_$)~h=60rN2XA4pzERC}{8LPA=GT7Q7tYzU8cGOPo zTORlF*q7rjtCq)lrNMi13DzuXRhPnnG<1(>oJds74XrP3qdQ^NL=Yj$ed1l8KU7$) zkllj7rYRUmG~=rnRg2NwXtV(_5tQ1GQj88j2pAN_G1x(V@0sm52+QekmA4#A9d5}V zasa&slheFZG+ZJ7M~m1VC?KUyaM>bYAVSKNv8e#=mNPbWhZOBFhV2GSV$lNynnz>5 zUFavSLR;HPDDefBwfFbu@g3w@u=)Z@Myq9FLyT6Ubx0SyhDC3^+0vW}Ct);RSe7{K zx{D<$(9nxSOuZRF?1qAsth2==!~}MXaU#fITQ57WqOxCVe^1^!aEfGXDynE^1L1AC z6#FyyS(ZSzK^AZ#si&}7-L5c^aQl9I1mrY5r*|=P8RlHI(Avk|>^Pl{>a~F@gAV;! zb%tj@_F$64g>;kMgnAjs_${6eWZMMiK;Vgz1La?iL=`{+~0 zJ72E0y=q`WJ1pdHtc-)!K8;$Q<`wnt5xGy#-I6hU26_OO>SI4BPyUReB1+$ekp1Gb zYYzL+(fQD6s;TN%Ic}ghJfT>p%o|lRQ!BWSqgxl={_OgORJ)r7TSXov@C@g-&W6ta zJ~g)mvBNDs#5B5d)BuzxnF8t{lR%>%%MQ4J*{U15Jm)k5be}Tcbe# zzx%Dq{_1D%0bL(cz;BGvpsI~O!7<+{zqd{#v2+gNWw38#*xhX01<(<21S_ktcZ1-f z^goy)_tMPOk*={C6Tci>HL2#%Qf-5tn4o`@|H#%rndDmljJNOqymc;UzRmj^I2(lD zEnLYeulCdI6;k{6M+0U34ITgaS^LI)D5IcD=OD|Gq5gMsBef_zTgvt)Z`x{mXv_VF z93P~=Fa&lqo<2<{`0(zX*mdk{%B&yzKIqp!U~t~%bHhW6j2&SWBg){m?WUUTyeZB3 zu25f17KWJ{umq_`MjqXzyG*y2HRWT_*21qlD@Gaa_-nq5z6V#~Efxk(I%D$TTtxXg zzHCtd*W)LKXN2eu9SQI^M?dzss5vHe^Zb^xvBmOy%tz@4??OloHzZVKv0z<;!WKu| zFdrWYeF^95XG)88gf)eoznfOt$e3Z-8!lXGMxd$3gSrfIJWhUv@CY@}$u#sv99$=> z&fvxW!LmpH2&N)2`*nmg+%Wy<21G`X8=8NQMXx5l4j7N9T3qjfG>H)r+6cLc!Dpud z>JEaefk`lzrml;2f>sKQ1l&Dx&q!9G)+SyBJyzSB(E`mZD&jvy?0(T&K>e}c(uv3x zh$Ribw``unq&@04(Oad`H96t)os<4&A6&Iu5{k1ItVm)ps2^BCS3B=E4z< z4a%+0#b6?WYNP7RLw!+bBkZ9_$I$|BdmkOrfeV;HU99S3+ca6ozjRGJ%UU&R0ptYs za<5&QCmW{oe=4HSSz#lS&n^LbAaHDUmjB;Vg*4D1OwTZ)1&$K@Gt@TVXGp?m$v-H5 zO5oTgJuS=Q;^x*bg0w7cON;-Y)6Y57z_IwQH52w4R0`)lKipn$Zi`$V^{&$!zkJt8 zJMW2B*a@9C`etr$)F_%D;Gb_qmv7v2PQom$w$t_04+&RK$483|*B@!geXb>G`us#y z-C&s2Lb_*=yoKUad@xxzL+%`hHX|XBMubvu=Pj~v5SjQhB;d1AU@12NZ z9oVFYrY5Z62t8vps+Qd=*!pioqFPP$gPux1r05@`OiiQUpfW zkNb{^&z~vDv2DW0C2dWF6gNu_n0fS-F|vegKoTdij<`t?iUzPpG?y^vuf6l;oL6I$ zF;D=GlIRD)=j4xbg{V94p^UzgyGcA+$v_kaqg0b>!99^-J{YbgP`x|G0yb&cd*v&I zJ=_Dm6V^1rt4ls^(YL%oS@Dfb*bciH-^pL>mB$0D9` z)Uh1h@4Uq9V%hS`{ilxa{W$hf()s7wpl8>a>Ob$lDal#W;mOyMPf3B|7L;A7#N89d z&e=5jbpwM=RAzxP;@r>19I<9MWM>BkH$hB|3`F@}vX0;auk z%Xtq3O*yM|q`!Oj%Tsx33?3+<5-fg(+2=DdGFEck)m#~c&pP6W*Ne~g=+jwrTyms5 z^!LT-q;#B~f(oOr?<~*Ur4uQ?H?AdPl&uXb=JPQL!|Gc&J(y1iDWzMxKBT0;5fKp) zlam9`MgY+|&Hi?gk(Gr(8O#a;O<%nj-!Ht1v#SqNMigv0IKFkmh=`{S>qKMI zCAET+8zZXV&W9==^~D>ERGhtj>(dd*0LjAhP7Ujx7qpsmr&rlatACRA?b*xmULS^e z*tK)}*U!9>MgIO)w|!F|ao^68>kZNSq5~&AT?6O$pY&m?qZuu~`~XEK`o2wwhp47t z?EUBr%^15%y& zBzuG<5PBn!>66eeCx;<5B956L#&3Y9Rx+`c5$p9Ie|mFZ1$PhW*V&Z1|BAH9BNm_6B(6vip& z=ltE!SS3yqr55@>4u`t>)#^AQc*F2a9jZH%+ajSIpon0Gfi41OB~V0QlP%I8sS_dG z+r)umM2NlZPz*sqij$FL>)kydOO}-_K^^STs!>1g=6Kn*X=IG~|B;svar>drx`AE; zkR1uFC?3Gn9umEk^Wv6iW@NWXZZN-pj#qJU+_%h4SL+U;*AcnjAT>2I=haQhnKv!( z=uK1RG7H`_4cS~J0I=M|)3yNN;`i%p^*ryUwov};-<^<+vVfNeW zulX6%8-2nr!l&GJJ!vK;xp13VO-5wk`Hm|{RN1HIen#QOqgDJ~df-;`Mp{%n9-+64CJIUN5Mt7bgn2`?aCb=|_l@1H#uO}54m_d?7jGFCF) z2A@X4Mbst{V-gfBZ@@ZaHnOy0V}KvV*2JR_w+miO^Jp)T;}55UAPQ_hq2p;Kla(|U zfYFvq;jof5K%lyaA&!Ugd;ud1dNE*>XBrb`*Df?ny|!gDhfqFa=k>=w_X&f~L(d*B z2sC=tnce8+V4fHJseBx~;xKj8Va<%LAzv7hG7bY^O{BSJ|M>DU304`1vJ|PnXIINl zEB1CA(T@CgGWiGns<0g96)KE;{1~AL-yfh4;S5k#;`@iUR^lO1r>=QEu?w&ye^DR4_UDPf=d(YNU@Pm)bepFGwv1_&JI_&(y{;?erfz#-wW z=uO$LoSiuDR1)QC|yZ zHzd_AYs$i~aeKGQ{7#zS)1iaR%0IZUh7;iT^#t~AfRYCN1LZLLVybI7J1yJ8adVfE z+nkGf^X+7?(db5@R|FJ{_lVa$OeT&=8-c1ae+Tb^^tnWDkry~VJ|1;KOJ&`4V)ICP zzG$?;7n1q@9?m@0RahX|zayoa9GqR7xqm<3{rqlzm4R^VkuMn|D+ z#TM{9*@*wY*X&jf|IostACK%<=+6c5dx7MFxG5uZ)Ms}_@Okde>S`myttNe<0n-9AMSX7T4aW*b{ zzr(jUsot6(w728KGbT5z4{>k^z^|{h+{yTRCf1WhpG-TDS^I6W)Bktx@)`@~G6|x^ zM5__X|5W&1Hx^8ocr5w+XtOPmZ+ zJ|SiBXdx6s`hMH5u9t!<`do~M0z*IJp3^FTo~SezJc)q(CuX?{$wKM~<^^ibL3aj_ z9{Mk-&!3M10pPOXXdx!lNq0;`3sV6t4SJ10s5VVr=5@>*>VhbkMQ=fH<}4rQwtE+x z0IjWiFU6S*cjl>6Z-BL-KuwHM%I8nZg`Z^#Kzn%~)5W5<(0)Z5lJHC@sdA0P;}?y% zP;>Xue8bUEB%(C*mC)l-#PEwJ32*WvcohGXxg)1NTD!4cQ3)hAfHfv>4Gax0H9vtc zkM?El8oN;bu7C7+9oK0v7?%*x@e!XL4cewW&%=nh6YZ;29KSAJ!O(+U&N6Nyw7MD4 zckm&9!f;s8Ja2x!0%YWmAIm-B(2*~*;yXOV+Ndyy8Sx(0jbj>86%m^=V}ep2bxLVU z>iF(guBkX2-?jVd*%lFN|4sLThAxE&Wgp8}>C0}cm%+kY_kQ^6PRf8x(R{OA%v%eT z&yUYqJ>Cm)xUkwr-dqwxq1U;uBg-+s6bh?_jvPrJ5GDg(`B~+-b|?w(2@z2-HcHU+sUl#_MHS*$BVt5=K{GsXi&S_6b1t&hlRqv)Qm$_@e7f`$?@5^yc(bz&nCYTx}2e$HDC>39iO)kzh3+m3s( zg((S1NeW4A*@DAwP;*vhsnmOWHF~G;UCTb0`mbEvQ=#JXUVY;tr;pln-HByB;m&Jo zRQ-2pQSH@&Omz8y$&&}1y zi=|A@oaE%>jE;^5p!42F4GI4H$15yZ9~?%re0~c$m<_+t0sBeuo}pkGVU#a%;j4^H zn7AvMGi+lqCfD9Bgqa)?=V8DII!q_6{*Si2w zS)ujklT))wh#x4z&~b>{UpkGjeh&(F4J#844GpdZnQh!xmkW$Jb64L{HepiI9u07u zDiCtGMP{uL0Sy^c8Y^flEIL1Juc;z?p1aAsAqFg=v&3+T$IU=u`6J zhZy!JnnPI#k{cv*tHgn)xzPsJsQ55iT3WA%ZqZz6Aegdn zb>;GbfU%*0yZrQaR^2-vAt`Kbh5^ooK>>J_;1__hsGv|f#Fv0Mx^iVXA-d4|z+uK? zL291tdyxB>cJ zLRncy^w#f_3H&Sn{^73NI&p9X7oErKc>N4CWb-veGw8y<3fe40`@t2xXvt44-5baB zEA_>rh>t{FFRmEeUWukqVA=7nUpqgPt`XU|(IG;kT%)O(bxU=%VvaY__+yS|CAGZ> zkz;c5wQk9$N#U*+ehRv09^ZVR!ri{Yb4KE|q_$C$P+N3P8F^6wSxHYfzV_&v>S;ki8`mA)Z)pqZ@@%n?p_{!BAme-fND_+{_W4`t{Ja=Nr zo2~8wQGYEgOlte~lF3OoBX-DtIdAbkmowt#k_5?B}#K*bzr3QZ5Wnh{mu z`ybfzXf(5^sqID7RZg(Hc=2{_?#|@oWJDuK;YeD8)AuQ?HN=6ua^;FyKtJ#q=HpQr zR}y#5!m01;*InOh6s{~QOBoxJM&DMvSVd8>^Yb&qnda76USx0r18DU*6V91!cFZ;6k-%io1;q;7|E0{FTwEQcGgd0<#$G71 z+Xp$2f$*e8M=1OhL00Oh9KKJC&3TNJbD=Hwy}LUX2++yN>EOWpL#fy3M-V2P5J&Ig z6i|#}Fbev`sVVpG-@ms{Lk9*N3iAQ*lAWMV3Ehp$0cIIi1FZg9+S*VdCG@eb3}D6| zPS6TjUW7IF-V}H}JgO(46MQ?d-^5ejeQ&r_wc+6tb<~uw-G_f#Dm#=S9Ur?-CLmJ$ z^&|em{M@Ww8RLzF*&miO*xulMK(g^`_B}EZyrC75 zVf%bD^R=W!FT>e=eclh%*}dFp^AS!9o&9M=81)Lm}z2G=;x_9S1zpB#d8ZNHR zvgcjjB+Y#h=PfDb3S8Pyz@on6dZ&K*8gL(qblMo?+V$(rPM;=rW3)scCm;J1XO%-D z_^RMgYY`R54Hq9Q9ZMNzCAZ^X&CDo*E1_BC)t)=1*15eY3oOF5>(@!BB{DaB0>mb0 zHIcYYxj&qJ*)F~$wI7Ru#xE*~ixs)_a}sAD%+2lCQ4WC*kxX=SbUdV6B1$1D%twzlBR53H#zIX?$!0#d+Y9-hzzHIyg*qH{mX}xj zg9jVPjq&7H3PrF+21A8KdVPp^@KytgB_$S)I6OWsT!2v~k4j3d6V%}V&7@HXX)1{Z zWJ(BROO`Dp(@24oJhN9;?4Kt*Q6%FJ9qAYReQoAx>pi@Dgz<=~nWOS2{ssrZMrsfM z^g3lvk;QFcDl=s&ivK;_FYri-T=BtS+#ZCf$tkKgi3o_zSeR^6Ewnf=KFGXj%>2fp@voO?< zO8@e%_H7~9p{5H)4Qpx^ZF6j)vE-wmu;ylq<|jzHS?FH{V@<-SJNfo0^OO9e5eYN? zL}OIO?D<~X*rFldZLFHd^l% zh4Ex`s8G zpxP%|y%Qkp!K)`xGi|OfZL({d_gb#rFt1-%nBF7_{mRJLP9_f_Svr3FxZKO?(tfvi zP$G48b#Cz#4Sr8G++qpt^!)tlZQfG{$XP-c1Nj3-CFA(wP+wl0b7%>WA|Op7(f zmw_b8%7A@%h)A#KaiZAopk`86Gm6pVjo4D4GT}rLE1n$j>ctg9WR}jq{r8ctTzk9b zyHhp{ox7E^763|aF^5EW6laYD7=-1%J{T5R@a{kuz(=R8KcW$h-2D8OxQ77&jHME1 zaD8!)amisLg+3cNbtG6s6)u1--Iw(^_lc zyIPuoen8p~anoW6Lhb~)o0*9N^B{*?wC^_vPmF42riaFJ?zBTqhN7M9%K|1NCyTUv z@n%p3Yt?l^Y`ix~6=5CV0y~f7`6ZuM|y@lNzZGtj9+r+&MAHRZ0p-Ose2wAZu4XFD(0Ul;U_xO z1%G5*Ow9Bil!;}8-~vMtGmTrp`9N<&r#{Bl;eJx_(wx-|cU(1u;OuNJ?|2W8j%dCU z1u3SFadCjcQU*tRoIYn-RHzKN!^aS9*2zgZvPNqC&@>4v&@w;ybk+O3t!=sLHLNcz zV~1f7X9%Fec4g&v5Q^_XG?_W!$P?tDOf4;4MEr*k>Hs}3Ck_hHA?j|Ar?VW-hoOnf zN$=4^Kp_Zm$ZOh>#^c28ZV%r(CWmeXt^H5fOpxBy&w)Npdhmll#=F7I#~MVu<5s)u zCYW2|aO9uCU>!J93|W+z6hlJ?5DxPSV4IHfdmkL$?A4OgJ=QbsGz7FECqfZb{q1PVYxks&#i?(lr6fpKY-2oOJ9#N>yaVkk82OW}r_()^sduII9 zyvvNio>ps~sUMkC(x;K&<}sWfF4^PtE0YAgpZfI=T@9p5N-#39AA*WgmZW^9gj+ap%cSFL|y+bV?4e)19L{Lh*c6itO-nGh5aBQNa2R$wKJ=*b!c?B zt#+vA-Me@0Y@^pZrj`E^a(`l7c+`TEv(V9u(eEh&hiY`0jo7wk!)YL(reidBzz%6gbm29ba$Pd z8=6-xnk?TIpIf8yTVuZGzjek3C^WNdL3JeqG@RiEGD2L&amH~6f`CXP=E##z1gxM! zq60*YiH#*qEr1kMi_n(3`O%~E9#4ParnvzyJZ%+Dxzv-q^NY$P1Q&OVc(-bND9+ew zM5&P8ebJAsMS_%$CkX9)@hAnEEn@6J%hg)+3y1X5>fM18$w~`wDmd0Sa$uW_b+=bR zlmI!$2umHH@yOe1AtAfGhsz3&i%Fv5Fp)4hdsbB3-}}be##5cjXp+R%NJ>eSQTV+9 z?o5DeQSYb2Qx~vDf}4G@s*!&rRK$rt0i#JWz!@W~Z3!VL4a#dC)IH>KpMCr9YYH+l6)2qWY=J5q z5lawzbBzWYEZ_8un>5KlBZN3s6D!*>T!oPjuxaSjSMY8C30OEpnTSR8P5p$j1e(`J z`loR+NC1VI1dc3{A%O^BJ3&o`goFUQzD^jx;68?UYb;{o;>yv*$bn@knb&WT3Tejg zS=PqhIbXr?hz0LX5Ih!4Nhe)6Jx`L&6^c{AOkD{unFqax*s0e) zfow~#TvAY2Vf~e3;)w#wzz}$zEm{$@2OVcn9t1v-%c(;X-$SwDZR0tnkVg~ss|=YI z>L!MN^jpdn&%}+w6$4sxq38m_z>#|Cq1`p)Y{0;T6GgA`wCvtr5>QyTsG@Q=p}X}R*r;o@B)R+F(8mO(2Yvx+1GArzG4ge-?kO%x#y&aDB^pvP z>Vj7va&No+@~F35WXS6YPk<+)uX7s~7~Cy+f(L48Fx+TrkWL!Th{{ye!FQK>goL)e zC5m}b%mZLJ6hvDNIw=^wgh5zAR?LSL#Eal+d3#6u9KmqpwX42G_~wh- z7a=#Gqje=@x0^OOQcq}h3 zm&)h4gI$HFwWOq^76ib!MGmH3AFC|Ka-p)ti-XU7JKX?k8K5PM^_9~zVB=Xz-P?EX z;GslVBFvfqdjLG3j92`nsNU8*GU5Pe9vv5F4s`YC(W4xv5`%y{y2l|i#Zhg)OT6iE zoJVE{hj0O9f6I$w$L*ZI=YXkqE9G2PFiYxz)Dw|1hhug{%r=(_WlAqx&gXo^KEl*$a|lZhKc ze*TlzS}DbTS7&o`_2~hYlmS`lDDr3Mug2vmv*#C)c#)l*BEHsAuHPvr_*E=kAPi3C zTSvRS=eiFX9~I-A8X{IJuYxxHMV8$GS+;*~5qmaSmWC)fat)ra%Sv;GTO4>`7!z{s zn*U1bIz=TVB-49n;tyN^l5x&&cKdw1`Z+}1;03?}L0FSVmW-Jt7ZcS;zxLhLpqy9W#QLkY? ze$>Fjird74!*<`t+mm1+8_&TAk+l~jYJtMUFwSe0Y0LR*ty6KWrZ~Rv7

{mDcb? zRw{o?KX{kMs-tev7mbKQ-m( zTwvSepb-kR38sMv_*x^Rwlrm3@6Q`CyK-{<0^3En*kh+W%j-{h*I+Uo(*k5x19UXB z98=V}%=sTZ7q1968byNVi$Kd;{M5ET`d))2fsC-|C0$1FtNeGOXCo~2RGUKT&i(l0 zZ|ZjTn4Y`L?D;~)+0epu1KG!tT5Gvd>Z5}ztEb9TX7(um?)hVGplCK5B7M}n)kt6S zoe47$QX8ZH;&9z{(IRk_JaK2unvDg)Ru=WZGr2}fLM>F+u?AsOf$OBYx=m;YVi8TQ z0SgxynZ2kAzKsjATxPG6ko!HPaDg$buuP_88C{(~TZ0BK%kBiPq66L%!Tbj}o*&BD zC7#&BCQF@W&~c8LtNcTV>$+_d2&e5bGQSLSi#G9mlhaXDn455!h5!k~X#ItzT9z3Q zmV&;PfK7N;ApPLm(zPJJMX({6{awc!q*v7hE25&ujF@cE)CzEu#N_bv z?7ojBSesDzneqe9V##iCiQ%>~v7XPeFGWVAVH#*ULgo6Xta2Hw3M;ap5l${a9rJrV5UTxY^g$OS}ijx z{!YOspaOMvWu3**fhGO%KRznvNv%q~e$J4c1xRe{wae)_fsJp)B%B#3m5Po{CX+S(f* zqi4tsM+4DF+-cy;4x}K1Q>T_fq)m1LwhPoC;}K_U&};aVEql0D-oVn;#SUDetrGwx zD0&GDe|Z8NyRU%U%S)HL-61w2N)m96`0N<)6}o|D=4Kmv*2Gk!B-Jm#50LSnc;*N_ zFR~)>DM20>CGKpRHH5b;T}XMwHSlmYH~s>?SNiFsg@!G17dq zi2zrqxdHeyBv)K!sO{i-yHLQ>T#OK8$e84{`kRDc8`j>7wpvndZA?oFytOHBNPv8T`=@(=4 z7#R+#Q+P3Xdncf@A?+n_t%&HT7=u{ofMvkgqBhZbIN{z15x5<0a=|OUaSGkf(7)d-?eCQViMU|Lv5W%kIH(KRm(X9}5qI;9TP#*x z?83V$^tySez1&?Gdh@W-6m`o3m^*h%ksM4*l0aC zU8yx8!OM1i?X>_e1p*0p2k@PU*`Qmd?O&VuiM_-Q91_X_df(jU+4SW-z z5&!%oWhTjcEDTUdY3Oe}uz(84Uw`W820S#9OUisCkM*M`78iyMv1M#sT`di&X>w4I zfdSS@`lTD%j&m<&`Rxhzr;s0;&+xG1LRW_!=oG?%fW@LV+;(<>PeK|)tbFnbmNkH| zI64q^yY?fJUEat`3@8C8X8`)evusB(+KP$=wJ{-e&@#pays-~u4ju3XZJ9TaIlyEL zz!$7%1qsUnq6%U=-Gq$4!{tAJ_X^S9Aj)-=Ln3`^>^<2tJ?;&WS8QXrldp{A+}p)n zcw&!VIQz9EA61#IZw;KL?C*!h5hdwBwPMFufOrmT+3+LAL)%*w#F*ZJo64~t+w|6~ z&^aZvXWaRVbwP!inz(YA1T;;U07DM;9$xBQjK34P5~K&_K&OEJK&NEm+qxBE)r$b_ z7j!Y;^icr6y);xnlqUebFeAumy8NF*ME+ssg3;Hp=FQJU@YBA1D10MHCmbuUjqOj#i{a`FtoJBAU+c=sBv`Ko+MmaSN|Jg3Z+ z$fz{|M?DDBNUIqSk;9y=#UU9T69c{x<^f<2d4BMtk7Xsj9$?SRG!t=;riw#}0nUi@ zQD`zg;FT!WtSkTbw_WL)M2|VytC0TSvDbt!; zm?!qM`1Nz^SqkW3_2^X;Vs3yS6H(m-U#eHGlr0#*5LiZ<6oZ54|6tF9Uhy~HU-BV4 z{RFUamT*{~&VnA5GZ7FH;)2_j6bl3G#?XW)gu*yE@o-cxZGNn1KGGa86$WSvpcOVD zq=CA$`ZNWyVHn3wtZR|5Ct_2{?LGU#sxlr zX4xkjw@E*S1{1|PlXaH_UQj6j_8S~q1udO;1Tk)leE8NyMT z#EPt~e{i!};>hOrc0X#L)?2@90$mOU6*bwbVO|zGgm@tFw8Gz4F%N5f8!d{`Qam(m zaV&8{<|mRA1e(Ihc4n`s>Qls?ox4FHf!s?B{H7~`@ZomB4 z8T|ZhBdEy_IWx6x{#0x*?iw~pz(nf;(S||+2rSDp!}}H<<;gTBd`qPguSXTp#60{6 z9DbH9rtc!nO}V$Nstb43<_-)i+X?cJSRR4vtZAK}Z5>6IX=JK9!3iBOj5o6EDG2@N zkdU98ETMYOjdq_O&Gw$oo~h-Zt#xNRvhWq+4?~RCY#n+*GkO61kCn?cM?U{atb*2rCSJip zX53y^86H+-9m0vQYQX^(uoq&mwT}03B9;>09^c^eso3 zk{Put&kE$5TYT=p%uz$o*UZ4@;pfI{dF#1zLa|lY!t43l2ai^DJv}gMANH(f1&}`g zMoi3F3roum`;sP!^_REVFzeObi}1#*qzPvTt_gj6veDLaT83zB(4D}v ztce6#=iPi{#Esd0u&c^-U&&mVD`w zOUw~sT1V1-`TE27hUe$T=gDv?nj!%K0XgqmvRdDrx(E3`^iBv^MMYFn6=S0kn!+VZ z`G?E+=fc;^o|o9M+`X8~BbmxUnuL zS*mb4?tfZ2e4tlJbDan0{Jm$>=64`0Q$Ej+rKofz%#V6|wUn*?E0PQjJPnO>Eww5( zIqwzUTtI7!!Cqc7SxW2+Yo?ij&AE6F@p9Dy?I(3fLn{OqS4%n=$7+>sh}FU3_08Q< z`XSa#n?rsNE>Vc>|61S_i>{>}$Z8{_PBJoUV{=VS=(5Aj!#VF!9!L|@&;1a;;$y+l z3cGD+t44u&a9n}mfE3X?C442WF)k`>o-vR}(<{Ol`q_$8ug)Q`B0cpwV}t{c29X@U z1{NwgW+mY6A)-vC)n1V9)0kGY;~XI#0((o?b340(f((Bb0_5*Ej!8owl7J-zKj9EV zRz@K}K*TMaKuwres%(_ z3ZVolF76^ZCKSarq=|?2B+&$cp+k{GmVTm3uxt_T3s^&_T5!}S(O4$rBbt;|Rhonx z`uddz6c0#DBvyoIQX!Zb9ObbHm(}=K`$wM``&Jq=5Bod_Fi^4}uM^kk1tl}617a6| z{w2)iKAi<^42EAP1?p2$af93*!JaE3kmsR}!s;9*@}pj;>HhVDD-zxSI9f`7*O!%s z`;edB!zU!73N_9<&|pL!kBS!Tez?UG)s$unM~g_U!hJi=>AtgA`z&*kfD;bMW4U3VufE*78$z2i#Y>X9vId?t2*6P#_9|CyT5;*oa_igA%`bH z(hwNb_VY{PbT$s2w#RNGd{$86gkME!#v=n9L(#Y(Il$qhU0k<7;dRE&4wk$FX4QzB`1T!+3-pmO9_R`HN&ap@fj=jL5%hxW3a6GdWv!7| z;7I;`)K;!^XCP7ZHrbqm?irEHB+PVfB+5gwd2o7Su+G{wx~0Jc#gUfyLCISc8N3oK zimfTKJYYSspBgWp_16Wms`=Ad$caGTAbetHUT@xdRa>e$RUG zuyPL^es3j>$Dkn?>&F1f2n2~rUuVY4a)@alZf~6kD;R`Q7ZnAy8=fKfO#$f%H$<8x z018@=35!R;N$iZ|BUVnKpK-twL>C)dG_@Y6FSO&%M7Ww-%VwxEz|5lR^=jGk-^07| z#?Y0=?p@E=*JNolGbYB2dc!4fc(`p(;qSgJuK*}m__>DHM+aUWAg)?W0oc|WfTKrh zOh^&Xbw!&WcGvif4Oz4LZr_eq7E7Fb%%D2?arwKHfRfFL20a@^L?%bVyvZJ9BE}(B zRm3`lP+>1IBVoaQMIK#1Tf}nWybiL|mwGfd}xM5c5C?laW9&UyD6w*vy0W@P&N5B(xau(XsK#RZ(s;SVbkJ8hqD(F?U3H zgO9=##-LQ|v`y%zJo3&Q+Vb%ra1aV2&Epmx9+ZPa6V@`Z-AWw)*Rz|7f7my=6}K3L z1*lf^NHmga5!8XNC9b)BAI4e(NIfuDM04%iCfzjG&$+lHV>Hcd@94vb%htC6f56^x1|@auL99?Uq7bp<5WE-<@B;#|}`zC{dn`<^J={ruyDm8-SO zw7=>yZjA4WwjFwxvUZCFz*&}g#dHJPq9ySMlbVxSFxfHI`%7t&3LS$@o$xLDst2oY z{OYpA;woP!>F*iVHgH#a_ zj409w_rS^uk`R&WANB|31a^f|wWU)LVvCN3`SLO%Hbc3P1inc07Fs%R-T~hR=N-mA zT?(w+Fa-&Z{q{U@Sk&+h~(=y_!$bVFctQHqbQZ zzXweN@RFZ75rSQ9xb|qc0Qo_Z!Wx3q231d0XXhCYWn7zm!dSjnCr#~YUAxj8qc6ZDG8qt2zc?N?LUkm*$*ZGx=Bf{7ktqvVmBD33SqMOe50fg?S^S>@taV z-*Na#;t9m*P#Ax}AcfUbE%Oi8X@BD#}11(pn| zbypdHxRJLL@O@Rkhr6^l3E|P9(oFggJnO(8qAw=*4eSTXerO%w>8>xzd#MF%0%-fY zN}^{~a__uUzW8dRHpTt7(YSoKwtQk%uU;c+1x<_&q1TCL zdjW#6xeiEBdfkZmzy$?XF10oW>L}_6G_>UOq2s}%=6I;Z2z6nXF8%C8)dV3Z(Z~gv0?i1(17>SZELxHTeI1RE z#J@rC;X2B;#LtYLAXlLq!6D5WA@2{KeKBqGW5@B9FP3I$I{6C4SzB3(B}^m~^T%@5yV8a-Jl~&j@As)GSaSVnSYfxRp+@9V+c8cB6#MV~Fs6_vEx+?uLd z_33n^erbuM_M+Xt&T4fcJ-rEER3i)ClS1rCjk_ zhB%pb$73)ly|hP$x{A>{v9uogq|(`1`(4f#p{?d&9a+M+$2)jvTBBQO8>=;xQTB4z z%GT9(6`b7G0w042c-}xlSJ(UcvadzvT30#5Zpxs-x>Le^t5&**;sTA#iAPmCxP}1> zCa0vt#Khc_Ke{2dy5e$jZMWV(Na0^U5$9uMX<3AN7(x=H2u$^pWzm2>XmLodhi!a4 zsWDNEdKHDy8hGW#ETLCJ=}XO|7{cC7^409~`cP%ia5=>y7)D75bEGH+mO_8%$+0w; zNMK;4@L7-Y-jKT*mYbE&aZc5`B$f)U5#oq3|hjpDT-_xPdnxw6H1@x)s^xh;#KkU@Z zlPTo?U^UTT_oDI1OZW6#s!7jkf5BwaLt8C0MR-aD?tkL7Z|FO1@_z6VOOVFIfn{kA zkaPF$p{u0=E4hZy?(P(BvxBQ2greZT+jW*+-}{HSbJ$5sU3yRFJha}C++f$jU5o~w z1iMRNaNYqmsx3@f_AJ*u=C?^790Vg?iK63k}^XtTs`a%N!%>j3yQKeIT-L?3ud1^aEE^-@vhXSBq`qXN5{N)kl9uS@)K zR9ZVWNv|Aetv$~N=nG>BIV}lD#h~dTrH^EcT*Jx$mBjOmOq{`${bvj~aVAkIXw^Zt zA5Ri=C)`LlaM7?3UD)fX54!{Vv{b@3B)UTN_Wy}1Ptrey6PX| zU2{Vtvx;qrMnJPJ{V%H#dHau71eP_XWjB0Hb9(>daHVkcrtaf+)|R=5v9P?lM}5z> zZ{up=)vQ_8?JG!zc+#|k&z3L!W$WE=ZONmRk}A4q@=gf%%1^59N^1V2pj&c`LFQMV z&4BE_9GT7aB5}?RaY?6+QAc~nl>nqiv52n7Hrbl*(tsMk2JkErE2WtXnzh$KhzT_{ zBu9QF0`Ue#G=n-66%K74MG&HlTzglDefNk0ltqhm>5#vn*}22Qre@5tm)}y~XI_-v zuV==pSb&LMEoGXFZOeK+yA-dH-yE9HuGXK_n$|Ne|7CNVdt$v82Mc~1hm_#c8QiYd zNED}WL8wyDvXC=k54hpAp7)@d8cN^HOs5(6cEg+Y`ef@7R9sTIsnrP1(1`yJl-+QJ znNat(`JdaST;~T{%liQd=G#xrFcatOS#A7I$H ze=#5!K8Wu|0Y=y$pcvI9brRd(CbiEm*h4g*rx3s#xNQ^ytdGTYDl1=rQ^NFv=ryRt!BQyA2 zw*>pH_}=ku`I7g8D;jp%IQF-3uFrSDH8H(-EtXl&=(bpdy{%Rm-R>EIGp}wwh}KqJ zw*Hb!WHzhj2BXIhc(CZ?zN)HQdv`3xu0^|;%6H$u%wjoJLpljAt%O)0tBU_Y+=OL7 zA&9E*`reM?T-Uk4E&!3!mU}6DR{G zCSi`x?nbR|Uu|QPZBxaBCy+vF|DxAY_aN!T5<^Ha2}PNmjf!05^A)rhF^L(opy$&P z0j@(`@fQP|?}{3P%$_b=gj0&r6t1}ukSJpI5Ht@VXi+t;T1+3>flY(3Ni%~W4izGX zC?Mi(ZiWQ z69G%O1#45Cl+~`=iDn#{YfxFQUkjt(F*;Hhki@zuE?$GWo-bf|;vnG@t@8AGzDyGXVB)m)C=`+nBSxyD=kZRD%ec2ZtWIh<%#et99xZ*LI(cUoCp zdB-|Qw^44`_!veo zn{`XYu%Hshv+vJC^I1+#Di4J)N<@gBfbWo&@ZX_gmBUCXMLln0!2i)wpP<4lljEOKTw{oQ#?KJs&A&Fr#qd8p~^`M2*D6HXP)% zrjF0`*Ex*0FWL9H<=+%@SekV3w~G$v|D09cR~W% zps0?>Heqm0AlB)~wvEumy1FlHi2@g<{hkC?^!4rM5eHF4u{aDvY-p7+2?NXn?LC4B z*0#VtI0#e=w)KoE0)TwgMJpANny@}3=QIseNPTBp!hnV0XiQzm=cT^x6@9J7ni$wB@|wKX*qNj7BJg9~>*V*@qW8{9pJuo(@ExF+AY z#&m4rM`vE`fX;=3!SY+LrMen(LH@_H6a~`zVykZ=sE&#bqM-nRHf^mH@n**;Qt2{W z7lcWqA+Kzmxh6p&N4~_n5gk2Ae`$3Itgn1HKbA08C$%l@(^(SPVF(pBap>>H;Nef7 z4TV^C${2dQ03Czli^PYwg5!!&C(z|Rc3j$dSDhM9@)z8yyryG8ZfVU;1)0jHUvgJ3#@D~CTX%Wu--a=kK zQpRyH<@oXj%#h%B!oc8b2OcOdaIJ{l$bxJTg(wNXK(nu3s~P$5x)>vxgiu};6!3Cc zD1`1=u;N6=&c{t;GDGdwZV}B53CW*&9U6Lu-+!gJh=~S>J+QgDTSi@`t6iScv1(GSA4|Fj>_V}_5LaIB6=jL4MDs!R3v>nE(7FXAhU-Ch}K=~`a5 zk(quwKvTzWK~+ZYhn>5wvfKmq2yYNpV5kG*M5q)mV;nj(fGM$+xpktWdN0FbTJx$y zoG+>9n66vfnMwgsf8G@wFhnZ)p5%gLY0n+!)nY=`d#T?Bv|Tv7bw zFYUww47o#eN?Sfggt)S;9%Wpoyi^5GeS6cshDZ!S;nP8R+eIdPNfgKWVQi(czD;!a zc=rgn00+>DkV#AonNop6XxISM0I}c3+&wC*=xCN%L6w=FpqU;Gm3CqSdt3}G5;9X4 zriRIBgw@Q@Y|D@X8PI`7xZnG)-Xz#f;q!23G5p;Zsj6`&@9 z4jnqi;5o^khc7=|nH(LvsVzt_%RJrl{r$T2=`Qc@HupQd@3B8LB!R-f@o91K=t!VO z`4GK{fk>*EnIUJg`){$IwQnb~=(B814Ma*LL><|-xy}1>ajkiipE_Myf5n$BiI){G z5=FqEiNrcJ2$_|6jb&NN%%b9cELrs|?EJP>F6+1lPO}%x@lwjA6P%^N!%?!XmoBmA zkU%ygN@Kyx5eHrr{QJTNzcTdt8IdqdWaE}6Pl|DVD%iDfZE3iCokl?ILaUGre8IyY zU2IGIl>&Xy5scN25{)Q`w^9kr)f1OM2n{iTiP%0Cc3D>aqKd|sX~$8$rO&ZVmn@Dc5_e3>#6_~<*R?jY_yxHa(ZmWl0xeA);~ zacFqmVj(L)H=AHni1_eJN$UVsPH-K9EP&cQh2Aci6#++29O+J`Wx`b%JTRQ+nq&Zq zQFsF{2AU#Bri($d1d`v<+nc|2LNjHx)=oyS0Ci@JG86qzZTQmj^vpa{Dw1j#jvfel zk*5bTFP{I~bV-I~RqzM;KQ0GHP&;ZJXYMN|z{Q5%8?_ zhZ_LC<4NL$#yqdT`CI8U|)UkhxCISWcNvk@!XIq7vS;J~1~`#)@b2{_g3+J2dmgeFwx zina=s9T`(fwM7Fe$(*4wW?W{{NZWmtyH{O@;Zf8Te` z_3ygQxz5?UvexhSzRz$!_kBNfIJ49)_T}AAFLu)PBtB3D(|2itxj<0-&)HW{JCnfz zXr7BG%1efZtiP-_!ru;$e;(7#Zz85A)mEi;J|080Ym80BlQ4FKZ-T^ACoDuZSu&TtxCzBQ}@Tu&=onJFG9)(%{ zv1XB|<0pSJ{q)&}w9u_%=UUSC>L>_BX{==k^%%eMeKjM6k<+GkUF;eus4&HrobOETmH6^AKC{C5&+8H=#5Y}0_<+5PtrbJ9ffj$Gzj43f}((4g1t<^ z@y^eNKUKJxp!DYTMkbaFLQlnr}lQKQKowz3M2`=+lTG$n9uAV?X+Qy2xHYKO}X5K5R7 zNimhlZb~w_Ae(bowD0uWM|S+TG&1Tw4+*yxQT5XfXa7}}z3bg?Q4g=|9r6FsEU$iN zc74{9GgKvZ2D$m=ZWH@zG!leZkG&+txx+ix7m`QP9LMSwnOyD1;FW6E+Q0*EY z%#79)GcnM!aAb*k$zxpzj4ol^Opd;o^7Kl8YwOu=Oeq~<+%Fb3!&)v7anQq|*CVbMUS6=hA#7ocyO~tb*8%~+ zpQL+>U{3Wi{SUOdhgRWYHanuU@dm$`j&LId;3a@$jysN=^vlYWcp3nQLpv+QPQ2OC zMIgOFi`?fh^VY-e#omLU$O3HN6w&dS2lKPmZYg-uB`1wrq>2!dw>aUFa$_*iNND8Gz9@N+~O{ z8@Ln2&y0&D{xZn0H-5#f({r_gX(wpu9$gBK$%&95NlQPI;84!tJ5|U z=i1XU^V;t>^?{MeD|zWR)3xj09ous(D)m~o!>qvM2dDF``^%-+{fp`zRjpk~HS1mR z{(_>8^A&}2?cdryOI+*TW=8$v?TNoC&2N9qsj%%eS*F6fXT*Kepweu}zGKa$?sK+> zLgjWoyv-4b^e%k@4HUu#+2sPk!_^j8jFW9Y=tRKTn*Z$R-}a|kbr^mJ zwiaa;6kx9!!pAQuo}5VV5E8v>zbCE5@v^e<$zrn3gvfCiZz;F z^4%b3=+9F&B}Q2%7Q?AORbYC(k;7wCoKlDr$xKQS#>u*d)&)ni?{4e+th#MisFl$@ zmck_S^B?UuY!Zbtk;Y- zKx+T~Ai5R!=_l{^9%6y8L|qhsH=Z77j|ePeD<7UI%GPBd(6!_MMKT9w2!ls5`+^)v zCl!oVI4xsw3|2dVb~CGlp$VZZp&P}>gg)gXRWW$QKrsDcUe5S43{CX{}yEo8m#G~0}^qt8+gt2hHayN z8C%Y!%xr^{hM|T&&R#(m_IWTgj;6MlOR+y!_XBhx-e-JX^pLV)$Mm@{^O~Ke5r2(Y zx+|ApD?2|IQ)l;X0Y@p@tLvIGfBXK9-6KKhy1AOh|Np^XOz|$gK-6Ry{PnnEz5wAj z@o&zYvqW)YK9>M>Gy5U7X@(uCJ3y5~D4=_YVG=-;InSRH%>^D%-544gq&;+g=+&^& zk)B`?CJ48Qlmg z1l3QzO&%!_jw_Ct;!zd-d8R@j;ts7l@=3V+2 zMa8>!?=rg<32?MO@FuYxKrCuh5dVlZrpR%lq{#!!1@l(v8k15#IP8=4^l|s(?etP5L50yfh@~HclL2=-UsYS~Yt zi1B-64%wC2e{}0b3v7ux{#;7&jYD=|5A*fwe23XI_So{>dN|N=jyngd+RDZ}pZq(@ z_>Zo+ax>`=u-gJtI9`8A#l0pD^Qh9X23JcQ<~*S9(aVhL7op7Di6bcMaZ!Pef0=-I za*8ry%2JgS{qqD^o1oi}f}T-9qDzM6k$}<x_(MwVog4!J+oWljB-)op$^>Hn9TK;%Jx%A*5?O^@j8q}|@e`wlC` zl{;QlZ-g-8*fu`jMtf%J5RZCUFqMm`Xa4w-Kw`Rxmp9C$tzQSC9P(>aJeYb*Z++45 z7>^f?hkSS#-{3MSW8Gn$+qqxAiJNeG>~w_Zs{?NiB^jzX#GIn~Hd@QS+8L)In6J>U zYx}`5adu}OQxtdX(oqeEpR7$#g!r#cSsAOL0(+Wdg(d14j8~U#Qp%i19$R{=8&T=SB zCnZc~92TVR1e1N!EW_S?qo~d8wBgukNoGtq%@u#EMLm_r8obV2L!LwNN^7-%viRS> zzP3%07`0#~rt9!1R2gH%^EQi{3C97T7|$|I zEQZnTRxF&|w4%Mc{J7Tbw9Q<}$ojgYmu9Bmzi{d-Yjch*-|Ul?|K!PrI|mHU&Pgs- zFWUECLPb3{yg0j9OV+$5OiKL%}mY`f&sh-T)x|l z>}Y-QYA!Axb&vAcD0#wjSXa3S{qBD`Ebhi+o%ptk03BQ^(lNK?Wr0?$#ti5O=}dv4 zfQb{N+LboVQ0;PiAI2>ZcorOja+1z{2c-^7BBW&`3A*bKcKJjc-3l2Q`K&TwcJBKJ zd1cyr|N^ln%AfrbttHemt*b?wi5FHm`9tFNgmS z7`d9zod=0U0HMp#T|9+Yy^}W1?;=~%fNRS02Ua@s_zUB>6H|G(dM>dO&)um&kwG}o zp!Cw~$sR>LdYS)U|1~WD<%|tGJzz)}YHVE#d4K=Dq@KfHo0wI|=o}~+!c6;oYdaZ+ z5M#3D0emW@1bVv(mM&StOZ0zWf}!OF9VLB0+)&8Al-Hl8Wjj2++VzW!eEG0rZ@QxKN2`XH z6BB1*;we`;ujO_y9}yN2u}`j_q_ymtam$~ISF_-~?!Wwd^#=7=o$a(|-8HMuO0Pkh zMj9WUb!l5x2D0`*ftu*jIc@jrpk*cu&xRGRQkoI{`K+U)YOHS6dId>~5@tEmqF!hY z5?rS>y=KlJLjA{A11s1FO=MZwCJYg~Z%Gz)5SvO&5H(W#hG$D6YKfl!R$zd2it>(5 zqbUk>=VWgzchF<67Bww--cs8BEJ3UIbjm~f82zWp6 zyMQ949CoYSwajImQf;E_hn7^?D1u4fH&TYY&?_>^7EIlC&pdK!^PL-!$0P3EEuUnW zN`q3@m-g_bGP|Fck59*Ai#0;yhYtEqah??}=U?~QTWqgch@P^-mOn?{-y3=M$6?j4 z7Y)l7<}1vTJ$7*2LHXr36wJgCv+gFl?liRh{Hn)sw@Y~RX33l2-W4dt>G0$I z1vhw?201s|j(^=aTtGR*O}{#>TbMf=CbQb^Yj$w1ETh(3?)+$H=MX#hf?6FOrKnR> zNqukgB4J@XWv+I3j>Bs@%Bu=g^d*usB9zN{TC;RdnTOOxUdE7nG-U(L!L%#5?I`q0@yY2exYAVay zIR#g69P}*#YBpD*%(k9MSVdNWZPS#nC-1@Db^d%8m03g0;zHlCs7!5rX8Y_Vt?K5P zfsF~RYquDbJb%U`9_;eqM&l}>T6tST*z-@; znO|MUbBkA0rDHGexZv<1ldTF|0-PIyx6Zx}zurzWQ`M{uVOgHg`snCVyeo3-B7ka}MV5Wv^MSj@`leu7<9R`rR?8xmYId4P2E=GtQ zbefHB3978P^yu_MGxKQP&GKyAOiDh2<$DdLJ-vr#rxv=Ue3TXB23LStml9BYLf%kHFaVj;2J72fT@x=nU#& zfnI&j?p>S9?BX{9KHa-Ta|LT#aw02{ccYXPEG}fB9$J~=v;ymk;jraWFGR5k&DpeH zc@vX@-6(tKs;gU;zKUeml<>3Cs-yxFYF$BzUB>bP{(y?5elbv?r?oGyQWnzaPiHWZbIr9PBiVCOC<_+LHR&XU7 zDRFG{wmh#kY?8EhQrTp;dbh>Km4XVtA4;PA=(;8I`=5rqEYa#L-uPhPeYRgo0x$G4 z=9>dYqOx-22Xi`9M><@tFy3JM^=i%40#c}92{k+l8lL**ei1EPt&w&g&czRT|0;e& zcy28naG>e5iaI6rr5ulBD0n=ulGetCnbtu$I00@RUX#lt+ucR=S}!Ng>b@Wt z0p~qd-zjNZvv8xW(yx>3n93*DS@Fb$rn~*1h#y$L##JnK=TX@-7ZLG5Ud8!Des+Cn zcPUCj-KtFrGS^26Kb+%LwfW(1MAxCB5{YMH4EA4HtLtCZnfbib^1<~+keyrEk7%mZ zQUsf4^XXknvFp`qiFK-jOoreGr}YFq=rB7s*C z2H4k)Dxn$#i(D;C0FI5Aw}Uu%r|!$8C^uIUDQ#Hf1oWfvclx(!oVeeM%%nQ{Q0eZ0WxpVCRi>yvnzN zr7K2xUg?(&&ey?*#9>$dE!{N?R|gLj5dy*=^{-}E?0 zxag=cc5h2DowHi6&(~VfFdyq)?EZ=B@_E+feRzj91Iu-;#K~*)j1b;D_{?^>D%J7L zljVMmzLOc0*tv@bXV}F5eS~>h!k{V;)v>SYTJNPOGmmbH26wj<4n^}y`t~rd)=%EsU|C%?SSPpe-tNh-!@A!! zSZ*e6*T_H1{ml7S6c1A0{2NrIar4`R(Z{vygK3nkhF-@;e_^x|$Rr#Aw5XubA=&fN zrr@e1-k)047d6MQpOt|wHN?OUESMX|;pcL3jlQ|9imLvJxmkYu??37#_mqvD9~Nx* zmUNLbaq_-}&etAp5r1kKvszTbWc<1{;mzL;Gq9+&D9(EH9nRkv%&lAScGNiR+Z>x@ zeE-XmTZI7$O%;aYFAb%)%2M*{*mzZQ?Kb^8;m|+q@J!~#_x=3L`q_S^d*$ci?Bw(l z7uRb{wGmryuQ8j2)nuSiQ$yTo7g87diy>3}Uq!6%nUVuv`PXi6%unC!^&79hFhc6U z1QFy=6mM5OG~41z^=R% z_+bt1webZDRXA>SQPF+pZon>LI5bNstVp+{gz_bPpowjJy!d;Ux%QaOJ{b^Rr9TIC z{5?6;aI*CLa7Bva7Rra&JQD25ulAI)oV@PMf0kMf92p*dVieyHd9o;9c{WSAo2Pi{ zu)8PJmZK?h%DBpigja#c%*Cn9&VsY5%l-`Y)r3N>|rx@8u)MOMM&0M$MpPfm?;xd<+5@ag+oG32`6kkq{PPY)$v$;{f3q&<@?b`Rav5)ymW{j|A&nqG2WM zLzvma-~n9HUsiT^!49Bjzn3C5X}w`eV&(!$fBfBQxhp+EyK?#&LumxFzu2m^NfWr0 zft8jTZkjhWiYwjTJNN0>UY=wqbGgj@CcYM@3g1Go&bF_-U~dm0Co2$YApqW zy!%>;J4ZtU8Cb~p0Ng6Tqf2cd>>RLP7+cvf;CGee@>h2$OqO2-72=uRMK0Dc6@ z9+>7!R5Wb%emCeu909=%gE=l>-lm!Z`sd)A0(#&t`Q<{V;DQhF;s;ibkh0{@8XZlx zij&k2x0kju?r@%wmk)3MIr2n#eF*n&q1=L@5=j*SlAEN2w3cIYjsoo>BTLGL)4}O( zQD#Q&;Swz2=8n5u`n*f`qK+fKD#zkBR4NbNojy(amDQ=>mVHDJlZ%9 zs~(B{NqwBOu6(bI+Q13m#a!$@)Zc@|Zm}h>@iGYu8@abelroS??d#6ERd~w+gX*@I zOG;#Q*~dQZl_=`-@pDn%61QBVR?pSGIJPNnJFD-RLo9V{9KlX!PLk7sUteb@_bm5o zXcP;JOxPeL6&%$&?)-gs*7AaA>N}oRLOm~@g|y&p^{oox((xb6GcH#(&=J%{iEagY z_OD=|0X4+Mq_3VtBZd%)ngW0dNPKVjIp2sAS^wH9RmekfC5QbR4&Eor(i8MEhzUQ; zE>H*A!H1k+4Fu1|`N4c0=Q`cPZV3#oQ=2pj=QtNSIHiqZm;Qje7`+mJ z`GTGVnJ==obAmuh81r=e0U-d74#*@ifftk1JE|?Mkwu_Un*Fht_!ASKHCXOJcZrUVaO%M$MrB7we0rQc15W3uibTsg z-qBj)dM#E*RrZyDtoTRv8khWt2filD8TTr1s|WgW(qm(v{w_n56MuI>c^KbP*pX(U zpns%IS8X!V@*b1Q$iqLv*I+2MXSJ_TV0*EeW|-4fN_f5R(pN3;lw2znsDTqDyHP}T z*U6D*vwyXxD^Eu#cN#1Xc-4S?l{v9k;YBkTF<6LN{N~E}OR6FvOX`m`Nd|mNOTWl~ zsOd7cn7}HT2vp!TRNl%CvJ=*xUCt)1k^|KSS?Gm8fkUDIM;^0}(C+OPBcZTV2DHr) zKzI!>oLyRTM;OH*I1<4VX1z6Lpbo)a18o|ax|X8k4)dBbAx5TBuvh~l=*3Qt<-0~( z487MVaj?bqnvNtE`<-g&M5EJxz}7xUAzP`R;%z1eX|1^{<$GDqan?pVbuHq2aTk7%bt%Jn|e}sCX z&mBdgF~6}o-jRL&Yf-2utIL_tkaj23=5wiD6B!`hzdyuNuxy$4vSX?$S{xv8!;3vB zsB09~dB4^v6fmRLa9t6d3^popWEtHVJ*oX%2lJilA6RqOUs!t;jHEJX{ zC7|v}(%A7+Id9<-BTG_C96M*M<4nP1Y1L#E&&KA)S@C{$yY&LRtNE9u6osh`sA;No zGLkSzetFIUawrK5yP+x2QgpPWhbBlyAkl&j14~96J9J7Qtz?8xKD=OmX>r22{=N+ zWema=EXh2w-Xf@SEQEzM-dRpuLm@A=iFTj@_2lTDT6SIqiF<(`;&f!!#3`TOZl`6j z=-^Y{(qGlRN|48ym-=eG>VsPm!`(?2R~>0@*0lDz-PX92s*Ai*_^Xvt{YrahqE>Gx z7^K$FOHC0G*eq7c|2ETAXUB(Ils?cU%Yh;TB^g!N2lXIi7o9m3C{ud_PX3Tb2XVJy{KR<$Ku+HE_KD@&JA}Z~l1yNQhGF?CeZj z_Q~Q1I_>u`v_4Q8(B)$o&tXUdCI)OxKwiZ-l`OUfYaFDCS|l5^tN1q%!4#Kphqot+ zO>2}CN8S&rF2>pIbpPD{ za_m=>3oHl^KIzrBA)E>Z9h_&rKpTbNLRNB<@c|+Z{vl?wFznY8!irq7os6#c1NFzm z-uU*DK4P5FWjqQRDdw7>wf1Q4kns8tl`v{wJCow!Pzd-tUJdmT6sy*`m{cPC5hoZN zL9kMAvXGoLZR}1GK_(rtEIp(K?o z`xkSl!3hg2r@J1#VeSF3xOicp_&XXIW>U&yL1<#r<%%0(Nd^MCcQ#0ZO#rg^16!V%KE zjm{JtHF|yR*7n5c4Iw=|hYPHh`$Y{ZjTdlrT24+r%ly@BO?IEBzDXL;|2wPnJo;_T z{ItfQ;qxX-Gh9rvZh5pZa)rwP<&g~qdkd{2rYFR46M!Dx)0v^@wo z*Gzq%;6hIW2HBuR`Uj*K84CVSZ<)Oj|A-$ zP3{A++Cp~rF)~KVvF-^96wOc#<;8vazZ-J>-yU(=6@!7^KAzRQCA3-=^NWkm{0uyu)vlXloXlBYad%ZbwW$n6zKBX8#H42CxJx zq-$J6;Ab?h*l-1%2u5}oXww;*9mjGTK;QS0X~u3#*s(&RFg3brz7vb zA@D$(K`5LfBO~w64=1(!&Q2ChvE4{IIF2Zdzy@0?xW!AbGzb;|^n$QnU3A?d-@M7k z-a44X47}3-D+9mYBnm2Y~WY*=@iqa;B=ND+` z>{nF76KuRk_FJEIfE`byIOL(9dWJ@1Yj?BayukZ2q;i&PrZmxz9{2kWcw_u)WX@XVVkb` zCNn(?>6wq3F%Q1&OsV<8zhO{KgPW$-%9_|nlLPlyz)=LEm!a)&w}OTPcA7xDK!N8M zg&)nAy!-T>9$oby)6ty3Jd`pEG85s#* zM_O7{dQ-(W`%aq6`=a^?g`M2D+&}Itzi`kiPH9(fouQPsxbc_i?}oJ;4(yu4hPLk; z1(buBwy_CcmmSg;0VU${bQHw8``Vf5X!Lh{kH0a?3&OvDqo?J?1Ja~iuUHoRygI)a~8?D%f%7-lU}hWe;Bo_uQ6q1K$(kzzz?U{ z0fDx?j&l6EX&<_~1Rkx0!_~5=>KkR9CtG#GGz{p*zv7kch{j{n&p%z&)S`H9=uXv1 zML|l2y5I@pYL!^$n(T>wuA8s=vJ+bV}u5 z;Rs;FVy{-~v#lKCUc0zg`z_zF)9ddnANYXzjL(EE%+e(1LPf=T8x?Q0G2d^xygV#o z<+ua~rOR6>WMPpqZztoM&cR&l@eC`)({!{0mMf}&Hvu&|s%r!xT$DZs-EHf7TXYhJ z8yP&uIhgmwoN&BdzfLw()ZzL;VUciF9E$OYi8X9DUgh+pE+lHP(pXN}-SHP=;54w< zC)rTg(Xvw$4@%I#=eV>iw;^r(K%k9=luN^<#iE?jG12bAA0I;{yQgMfZRKSZUa0R~ z-Udr}v;x@Zusfz^YTnb|-@g~;5oTHQDm;dW3B(2~C&$08XA3X;(~emg#rqc*3!oT+ zuLo2mOz6LE`uHsWVl5*Y@npv_$(S0*-729`LHmir0zbj`<`xQb_nGt}aQf`XKgL+)vHkx5&5Dzb8;H z$Pv)Qa~a$~I6Cj-CY!Yo+Tl|HZvT==(uoHrUDBs>Tdc$*Z}#6=rnz`>%(7el^LzCL6>y z;6ULsx+_8Nd@62rsJyPq_`{o>?d2BI##x_6rrocX8O_YjZrr%h$i?L@JbPGJSXcwf zo_5{ax{g&iR9=7Er;@Qa!T25pk+@a(N(x~%6iwH*bVje<=9hKH6J>wv8+KjGV(@*F z3afA%3mP~u*~pGEuwIlw=?<+iBd(sBO4Cdy8d+}~^r}yIz{~D0Ry$7-YruOfL;~8$ zO_CdNa0Bgi+1#gu`Hiq1NNHTDu%LH>bE>_irJF2PA=-3QtoV!!KFcJcR-I_nGH^V==E8;?%pI# zsJv%?Mqlan)3kHAi;W6(vkk|zxix2I&9{jZ^c`DSN3j{)l%!O3jeWLPLG#|?*q4N3 zFYa@7%^qd8GJgRP30%8E;3@LWzG2B_*WUiLcE8t|wjG}B!ADG*XKPy4*L#kSNfsVu zog30Ww}{h+00xT-S1jP$v112zL!)`$?>xlUV6i@``jOHFcWroGgLMZ3!-%SfYyrKk z`i>N__em{t7PqM_FP7epO@v|nyw^N-vo;&On{HH|KhtL{ z^zZ9=bPPANE3#2~HP|Sv*070qoIoi|_Z&B6oNvdLQ%|g?#mI6rPa*ARcmE>Ad|BbW z;sxe*+Ty_SX@pB>iuYWDWbL>lCCszaZSIfkq(;w19&1XJZ+95~{9?mb`1sv#9IC&p zyzt_*0q6Yi_(PWY>Ob)Li~f4Qm?5ToZYMW8%b~N)%mHOSWt}D8?ANd|_4f4%uV+Jm z)Vl4{*ryf8Cw-#KIP3HiFZG z-I6)MGNm%x7@XwzH;fzg^2YC|0>4A4abY&5h`c9v zdC9dSi-{{)j@p-AafG(RTq%LR*sJ^Xja3kHty}5-?~tKjnd)u z;N(eV3J>G_B!UuZS3U8@tG}b%M`e$1hS>&+e(i6?r=K{?8)Vc!65HCl=yic__yF`@ zQZY5WAxD}*4h*XsW__}ii<@#+iq%M=)S6{I5ri2KPV1l((!5p_gJNdMFNBm=$Ttr+CG0B;Je{gtBxAt}Z%jBBL;Lj{+}M zDF2BOwYm8UWGWQ6c$6j-%BF<6=Ir6(nvY?NR)`BHwUbzwsTJVEGd22SYDF^D?`N22!&9$gPn=n0KR-I4$$=LnKGklV*CY)KlV-*q;6L&|j_AYK?zKX&@WM;Ey|=xRTqV z{o-+(>mPx+t<@}B*o4btp6R5*x)E(A2F!b;q)K{ww;xFg$7B{$0ca1e!A@CcHPbY+@cpnHAJj?HK`Zh8 za&~&pgw;cy&o0q!2{y-lF|;V}wga({tTH9O4>e}BvHQ!W$9kgg-W6KUW|Vm@{v!(p z8OI+bdXCSh?v1Ir90Q2P@dsa_lls=RpyxqJhUaoFO$YrTFftSZDA`}7yT@PWn7<1F zFx~6s=8WFy;@tO>$5Na6U+q(BtI}@wR7&6sqh9OEl*GXaJlow>aSJ8)d~_pG>SVax z{hxbnOwQI+;k)AZFV5{mgjko)$A|q)uvbJyakC<^|KxqKHFy(CiqXrDjz7|Uq?4)? z{_MVH%jImX6-DLJ$`DfcFGFcK-F@!W^GQc#vQ(WO#}GM{p0t= zID3n)-22evQvyTUOOJ<`bT;qHUmeiu(>U?c&j)p}S(f8AE+n~CvlmsmBNCK!N~a!D zE1Xlc#4}+hg&PK;Gg1?DG6G=%00Ap2S+Q0T4r_qp^Yl1UoBqL3AWI16J)h|e4Ct>G zZgW8;1H2w4RTwD3GL|dk2&s`QECLS*d6#v%Jej^zeaXpr5<_2p@INpG`?Df13 z);x-eiZ2z1*0t|Nngq5FaVR%Wls5-@*O#r<=o>I0#3h9nOq$Lg8J($e)<>d z$#rAc-G$?bUV?z#Zf>*XbyLejE9L~ zxqm>wevcVxrG()d%7XH`OkW>bp|AJ^>;fJN5cmPsPfAMqWs#BC&>Peoh-V9BIn*FH zndQdVuHfE0iZMKXY16vpBY*Rr$FrrpUV z<2`$Pap#OLhNdT!8s2ZP04e^L#|HK+;;YZk&$F*yy@n$IZ3V=fj&n9wua5LKB&Dl{ zC(Vw5AQmg{9C|>gz6p^>xO}+Xxv{h(re|KXeXo}Ru?CK=f!t#U%uJ1J~HyJ(zgIl@NMW$mD4YU@@aK2bD=58Su$>uxD8372FH@fRza`-{`& z23Pxs#@a2Cx3xz8Hoe|6KijM_te3QyDi#*!;MtTS|9SAp@X4`@v0)RF@T;hRtN9e! z2SeP8yJ)g_+7{CPpktm+Ql243G=S{L&J{y9x6KK4RIvni-(<^|dg(UmBW@AB!{5}} zXTusJrpK)oMFlQ&4FVBb!cRvB3>NPd&s7(H1;w9)gS(&?oajG{-WvyV)v7V~pP$Uh ze+bDQv@zvV%>g!zSH`Wz_VfhgPQ+s7P(+)bJU=|p{?}vMch@vG2jET&j*nf07X*a6 zXoG(^$@kRJ7ntK~0Nuw&$3NB4Im?^CnOr{P%U*3m4M3 z;STrua&{-SUgzmuRGFG_bcT36@;D6(6o|(PA~AeOF0Nco4RQ}80r?L3myU}$fDbF8QCiQ%2I zpH4s;Oq`sc)xr;xG0c)>DD>gBCWK^yj(D%$)X`L zBym?-EFa?t@H-%Mz`#Hdhgyt}!+B6lxF}5{CFyC?zD=tmM@FqsXgHpDT6_dG94qZR zYG~7Pr@xlq}QFh=NJm*0n~4hJ@*ivw#h?{p(8SseR3indgGZbjQrXf)yLI zrlzKBB9;j_|1O6$1b~on3y6UgqA!o6zEdgm@WLM zP#+M2!{<7e#-Q+9kND&nkhl!b3+J#(HBy$}rlnrfdpQkpShUyxFL|ly5ol*#~ z{dov@SYMC%0X}V-&X)CTcAc-8+dL*OE9Y{#h0})m1h#XqtkK}kT&y(7-=k>>rcaL*4{!DnFptc*bW^G>mdvU>HvLjM(ISDzyo(mYtp5cIeCgrc$z&%T{*o?F2ZQNWRF8 zh#7BaAM#u%*u!gT&{QFX#x zRRR~=4gJZHUse6Vf^S<(53thS%%%g1c=U~fiR&5|KpFekz%~6;)Am;r*`h1}-V_`h4BMiGKjF+v?Jn(eFwSj3 zXNPxC|K0Az7n*mkjH_2~s~>7YU{cTrk{uklo!Po+JMOPM&?pU<1N9Lqf0SydgiyY2 z_-m_vm{We*&lCZ7vpv!1Mo~ZS$zZy9Z>e*RZOpn z5hmD>3a9Cu@k()iBzNzsgS$tV^Fh~$+$zD_!_`Vt$)T$4Q4{MA1`fY#&@jg24tE?Z zI)q|;e|!#1c+eCB8q}W*?O>rzBmaheo;js0l-Jy?)%$~kqU;nEuc-}#XP1d zyH9bQXhg4e{osS1AyeCee~R^=kwqq^d#ZX8yflyrdD-n}#w{iEUY3A@AQAP46;x}v zfag8BZDo502e)ksL5TOEP96ZDN~gbo0|dhXoOHae%kzK~0W^;PR2sUu=2ivZT%gFc z@ru<%K^A#6&A>*ml}C*VBMm$n$YfEVZGprLYzW}{*%}avVIqCsauc{USUeA0!b198 zdwU&joLzqFSn;umhbYdN=ZaVwfY$jfo^5yen!;C4?? zE;9GY^sj~>FZD_)X6C?}kM4^hC3fyb3w81HKeR7r>*5vTT)w4-%@ouKXg=|+;1Gk_YxkxQeEp-o zrB^;&-G^Qr83N@9a7T-KZ&1?(9Z7Ogh#&r%J^Hkl|2axtBsk=I{KC8X*v*!$CU*(F zp04j5qv=RQaLsL*FXt(^tvX^B;n6sFzUsz{wCh}6tm9qlf>aMON9XG90+x=* zju(IgQfi18{Tj&!WEvb9u$s|YPR%dCP<%s)=hk-_t=1;FEq%koq((r!{e+1zpo}=A zgL;Bq0&F?bI+>ZVU@X&Mp%s)Hev}Cd0&sm0NqUb+Q4TmU`D_D0eG!ipMk0tFGa2p+H9fZD^%<jtKZ6?kR2%;u zu82BHSzCc+Iz$YL<%D=ss7qzhED|v@C#Q4T04^5r4NPPphzV@97F!#HC!s&tKUaF? zUo?4ov|19RjW>f%CfT~E&t>CkWoB+!MpU5`Gd4B7jwoH5pd1EXOQ$VR!B=b90VCqQ zxubf=OlhyIY~ZJ>Z;Ug^$YjNO@`VcGV1+`AS%3Th&m&Ve4Hv8)Pb}FI9VUa1_wmH{ zjFVtbp&)pAdcF}S2)%ut3r&QWNaGlzW&3m$n*|VZiN(+oL=uN=Ltp9FH8yRV|HVa7 zAK>@KO`9-b?{dn|7w?Zz7k#F)gN)A5t7K=xGw>1JeJS(YPVrAN+_#=!o?~8Tj~B+W zVnu$s*D^ARVPe2Xa#w0!*X$%3u`;)%20(=%mU^S6Gc`0kpg251*~rgNJbF>@MgA3D zWjZoD4@4U06Zr#d-zVa;zz#ql15BxH&VU&VAOoyYf?E{CZBMk0*C56y?_TIZ&4V6? zPz?dLfn3JNK^!phvU*L=`|B z{ZG4}iCx2U1z7tFyIK~jqc6TLcwY34=maW%?S$0h0H$I|f$hfHYFFt}N_} z2{Dwa#1d@4H8`AzwFvo?LW$Xz13ZObsdgWFl2o~S?=*?~siH;477 z?Ni;@J_~pfZ}X*|1jWrQK@N?<>dbTOCmMkPz{FU_Q?WRs?d#V_nbKn|J+5KepK=Zc zDyyQ-0O^J>Ub1z^fBt-jb68_mhJMFEjx>$~#Iq2|7?=IXkt4htH}=VS2@IP(wA|Eb zdlyY(VuQumg9kxZz@=|)rXoKB-cLbQ4d1ntMnG{WWw>j6%{(Z}3V~SAr{H+%7Wb0V zo5!uKC{i8@0%B3s4e-+*9MrCs^-Sn?kV{~dRxVNqC{xkYac$J&G!W%ac9SW*Gz@1|-t-maP#7}2#_o1=Aw9^x8G6bgj`l>Fkc)-d^r==_{X zJIXs1AY2+L*i@Ywn&yzhn5v+=BbM;Xq=F*ve z73IdMdAm4_F9uR1XJzN7F6XvfPWU~hW*xBDyA+*+{sJz^8Iv^EzU8N8GLvk=l9peb zkp)DF?*1VH!yk*hMz?nvA%)tz$ z9#5l0RTquW)tt{m2MVSG0%4j?)8E*Noo0?2m}-{N=wo8{1?W5;6XqX!5~+@|GX+J% z2Fau8Ty@=?!skz-veUx&6!T-CjjgTQ!jBs6i`mG>hvxBYocu)pAf-e8z8nN(K{n40 zW?0^1n^P8eZUYlRGAaB6rvlwM4x{*&v#Cw#wEpHt)QlE9r^w^qo-~h*OzC2WQFx6R zD2^C8;VABlNgJU2bnR`%X({qH8^xEWa*}B%X%ky+|pTqkB zmEsgRxrL>{e{pfp%4NVZLBu75IxHtAwxFDx*ATFgSnH;y zUb2brKh7x>R$+_+1w4!+3IgO-TOysI=0Lkdy05%zJ=}ESRr1e2 z{{)rxhabI8tg0ItPR)5P{$bVq(K8rP6D2q>YO**E3s(#b62MxI6_83_@j6zkaGk(e z$q;Hc`NYsBpgj#5V+$w)>J$RBT>I0+V>GQOq~VOdm5Y^&mB}_w54EcHBOUXsPrQ75 zUw?ecLa~W%H~(o72r$4=5m*r^mm}+=)P;+W9h!OjJiW8^cRjyg4=?h*#xoI$G#4<2 zP(&!Cw`R|*PghabBXSeJoA9GwpUmRG5?l#TKMJ366rVf3{^J^cb4os{C<j3-bTPNLIms#iqN5B7ZajW4HLfw|4jlVa zY}=pxE92BM0$K)tu3O%Z^?>L`hlV&v&+P1cmig(xu}GRrERK|M=EtDc72{i(UEDpM zIsaB4A!qQH($5sd$!GW7!EDl#z}mjR3S;y@0EG#bDG@agWOo)L!a|FzPel{E4fjO9 z|NRxU8w)cKl#!Q+RO9GY-%nRX(YUDwMW6!1LBo1KV&)b`L4HGW5lG1Do*hkw z6XnRsZ4&r`CYk3@WC-)JgOIDW>~>KwKshkIF;zLWTn)t!m;n&&AVBs1zzLN6%k=(P z>-P5cvp)L}PC?B9OsaK%@z}tl6s&dNxPx3%IRlc?qnDG!Ci|0wd;JpgO3EEI)v0G-hP*97)qb29esg&mUa6 zG16#fod>qjFIGaN)f*W-lYGNE2lGEqr3Umz= zjUb5Vw$DRf3(N^A6qwUfM!=G%;^gu2$YEQ8B>}p4>!#yGz|yt)8=f_KOvHKUCorH} z8+85Y5J~}DTzG-N_YYRTbgq+_XgZ&=6U^vF3u+UULJ$gF71p=3q=g=HH` zxHye7+t(wCYl(;X&ElJYD20Y3ztvMDP+ge=Uj_{ySmu%M;kWRrb`828oK>6>u!^+T za(6l2X1fb0EFS<1IEj!vLsWzY1t$#?H4F=yb?B^+Qe1D4caj<&GjyJLvVPjVD0P*$ zoY%t7gl>LnXuLs~l9TjumEo6d1{#~(n1A}xo{bn55e&sgc~@mS zorQ+VPcw9WV|+G;rrVrsp>a`L)q&xW$hw{&YB%`wKb+i}$7kdi&`0mtv*)0JLHjjF zUYE~zaoEXH$o4vL>=8pQ-m<=W)fU18N9FHh)0r(+7bit$=1G3RiTLu$@s^8%#-r1& zT{Gb1|9P;J_WO$>YnE5kvA-RslC2h8+?9Mw1;(pDoCeLaUQPCeGb78%A^2XB0I8!mj$|Ee-Y+yaiYgoUDoBPc>l zjVvDrjzojS-#h*vU1tJUbKbuHFvc)OhOtFu&txZCT8u3kWN0BOV^K~Pr-)@R^lj%YA-DhJUj_@$N{_PY(lj! zR*~6|xZ;O7(bzA(jcKS!7_!Eu`76xLnFoz^#u{qCc~B63I1U`2l(kDbekA(2Vrpa7 zeJNE0h#iK3g3yh`7asoz6Ns0M!oiSvOB%5I_g8yw@ZoPC7F-J{%m%;&`SMF;HVCHgZ`05Xy@veB(-N9{;sw8 zQ8z|(CXwxSGm9bXM<$DHUbVy8r=Djl!v;!sGF!Isf2NS(N@lhb7l{d7A|;kb=hMJi>Ec= z-xBf)-%J~1o`beYpWiQJn>aQ48LT|I1(hiIK%c&K>hloZMN^fJVep zzr{C*Y31754`CO(FgO{?-<|c>&otB6WRSO9bSMrLz>;MV`*1lk zO4hDlU)aCHF>jyIzJ?r9C2A?;-6mD?IUP>byg3a_s)nBuv0>T8omb+t~N@%z{iyf@LNzzM}r8(M(d30Nq=qpqKo zSL-BJ1N+8z4?5D-JHjV1?8iVsRu+*yI(YK$IS<@&nmZd>s?$Wxjiw}s%RRpOqOXd| zk{=ehF3twZ6s@G9!_6}(0Lfya-+L%&(%Q9Cj0fz{CH}HUUX5cJ0gNvt7^cMZ#DCf&FmTXW$l7^MagAd7 zG@DOM{qoA>E0ae~_A3vL7uivQz6 zW(|yG$8lt$fUi!&=9S)g>1vN|>-lA)LLSyOxkanRd#t=Jx%kRjFTK3DIKFn#8iTkA z%SZKQn<<5eLaCTS=abx+r|k7$A5(5XCoq|WoKZ{GZQ8M)Qvo+GR$ECaWV&x|*bJjv zwxXpgEXcbx?xMz2C9Jb3D(SUU<0vK6(^Ntxv&)DKEWUc?~xVyt8%c0!7#6)#&JC z+!?_t1;&<}*?a2lCEO*0nqvFJ!L2?ve7RLe z4!k1Q9RPh&F1@aTYVELH}EoS%9CiakH zh|61Sp(Kj+|9JfP1W|w$fBdZ{>rU@+V}-mmN%SF8cHjUj?lJbkKV+Qz#=!myGenZ> z?Hyv6pT1E)ud*WKfthA6eidMtOd)heIZzf2NhaeC;XHu>>zhp+w{zbB@RO3%eUkU! z{B{`#iXtbIP-W)1vq!!L@ut352%%}^t*yHVc-wj!L}x&U+pyh$fJI$Ksm40$0EB#N zYr~G0;MT3@sf_wKsuff?V$?ze#rt)yt^i^k})$XpVAX=tILi`8Z$X3y! zcBiNs3L?iYG2)1E@eCccWJLwx`L^uLOaqQ{?}G&~9V7r%AzUGnOPDn zP==|h8&CZTCA=(Hqi?Zw)G2NM+K}5Eb@oh@M2YQiQwe5Bx{Q*?Hn4CPrm z=6IZcM$Zjj|I(f_WU9nrKs_AV?bJuIEDnXB#cgJ<_$thW6Qm(h9Ro|5KNsj0+?=GdD@uHIvAM zq3J_$K8)I<<%Pf@O|SWi&Er@~d)j9N+kmKiT;GdHNdaxWCzNlnS)@ah`A?<3c+|DJ ze{}HOpK&@utDc;T_o6=NJ1X3HJnpLWB`_=Ym@2x&qlD4{FbX-7B8qRfljO|t^t__> zT+g?0nQammpg2WTY)CFC5?a<@6(t36echE6tv=$vHL!|+5p;FXibdsvPoqvI{{bYK4^M^RBS$Fcf8A)o7w-69>dpdVA2)$9-PD_*xzxhVAH!0ZPv>XC5YWB-p zUUTm^4l+sbMlC0kRQI0@;`$ItQfbG#fSrlzz96EOl>GF=*tr+Jg*bvUBpI}leLM~v zEg&WGF*uAD@_{!iz-~rIQgiQVQ<&yhvMqoo9 z6%1B6^wCFNrC*35rjH;;FBtBtG+tLXEF|DXnIq9Ti?C74Fgf=eLuUay7}~(t017*h z+i_rIaZVlY)YjIHpET({$r(ujHD#u0$fnNA%{z-N$lnI!O{CZ+g@2^<%DDdD2$ zmrLu;nn4~7jd<5rFE+7873N$ z3>3PMUfgAE1%m_>a|@B)zW(aUk(E|~prNP>BX%GtOGt5`%}ssii}t&c&%|z?%xY1f zf@c1qitRAg&`J1lZ*C(&0xT+6salg!(kS5DGonDANekJ!o$M@jM9AA zoznVE{E)UBZ(ZF+3;nsF63zRoBg%ki2l-AHnWxHybn@nsE)S;k?P@WXLAYaAeW@nU&XgLXHA~CAY@dc{ljAAlC~Nox*FCYl20Ja?@nWNSa@jUYy-d zK_dOTtTtrt&h^(Dm$VyZ(BX9AmUTqphGUKk%G=5@oo0;dJCrQZQ8!kdS|%kFMRiN^*`ng-^qY&JI=2osD02g-;OB=Z4&e{ox%mZd> zPJNVLh2t^4nw{ zF&Pb<{~fth8aI`c-Y<3Ti0{smXsQlL@)^^v$!4CkqT9Y%3CC(~YZICP$yxLdk(7?q zrTr7lNm>?%98Kwd@IGuXGIux%M5hEl&oVRXAMd0bmkR$55Dg3g!vuwPD3dOuvNlG? zMI6OTktJ(xg00!T+rG%`a&WOWW_uCE1;A;ViQxl2N)(D$52qz!$l)Q zA;7AFf_1Xe2-yLItB3U3D>hk3MY2~n~#0Ud1ztbz{?ssjj?yIWaM?aO&2Q7j!0rv|Yd^cL!G|!J6HR?^R ztH~dSQrDMtG0jvBJ5S7SvUNw8X<3)9dQeO4?2Jr?dPz*w`XBld^pU_1$ic+71sVnx zqOZDoa*4zFP3C4#Z|TXXlq-sNI1!Xttbcb^5>wEz1lCbah9MzI2L{$EI?Z6>5P8OF z64KW7EDG!NaNo>9C_{3<)z;dM>-+o$XUxQ2SIvPkp6jBBi{=rTMhg<324!}r^>IEx zMn>Z17!O8+sf;pT=(zJ~oI&xf8bXEHgYRTzoign4G|GH5hk?iwI6R}ofpP=q3E&EI z>3AMM7bi%mw%fi7p@#3%x7h2nD=^XMS3G|@urduJAdGz11;a1w{!msMn^o^RXGL`O zAv#Ye)jkXfT+;dA{qsG#@^)037mwIWC}c4p@!;(J`IuHP$~Ic&nXkRo`%@t7xOI}s zAq~xLUvov?N{D1jo2CJDMJ^Nfv1`|-I7#2`EQsMzJ&P3-8`MDQ_Cw( zh!LI)8YS`6yn{T#A@mHeX+%CkCkVPrcDd;tDYkcZ(_q3FOa^QiIO2i~4w=s{wa3Bx zNV%9~x+F);AKm+FYIp=WcxOrnZACz8VGtI=!TMLW ztAO!HU;X^EziH@vBO@O}2g%WqLbZZ!`i)vkf{iU3tA3`AmmEvg{P^4pjB(<6Qwo8U z|94S!a9IN}0pAcBGxi19ad=dkC6kV(Tun$w<(RzMTsJ*ttWKiEp}9lo)*<=>Tf=aF zQgxK%Jjt&D9mH8tHxpb8-X9$ad@pTznmW|QiypX7n!AjC z?H0kVZJOQ3M0g-!uiozXrwXI-G<3-#BcnCr`WkP2O0kSJ!ci&-?6Rwu)`_{wO}1Jp z0^go8h1eMY*U`j0t*Bb)q9-NEX9u>JL0Koqpkon)gY*uQKtu}xebf`!#+N;kP5xn0(@S0su^z$`c|*!~8tKywi+fej zvbA*N{r-pM;sry#b93)snpSq_ZBtd{jVIk)=VavPw5!>M&FUpCP;6l*pPpaVSf9t& zQ2e1W9w?2i?lr){80A0wlwsiRj1mED2PmlAOS97;OhsqY*r?%jt-)hz)~3PTzY;`h zI;M#JsSZ$(A-1I$qNT#!p|h`4@cbdMXy#!7LdNt_(hKn6E<7rjnQ^1%^o8zbHET*X zrSxx#{Nl35U{?IM3d-Tafnq|i-k>)`jYE6<)T=sx*gy~%iQp79w)p@_1WT~AbjBx0 ztf>|ef&;9)PW0%{H6(DN^9QJ9_))le!j}tTu8Injm{9LEc$|C6`R}gz9Nm2!#Uj(t zsb_q8s(@m!7N()Ua8*@rd23Moae{Fdvo)_h)l`9MQ77U)M=rZEa^7ghIufPDtTb}4 zZoX7jaKYQMVnDzPlJi>EX;eX(9d|TEbX9@!v74g2qj8gU3|vA@@xmq*+vvEFgWk>= zVA08AfWf9yze+2L9-kCSdfmF!aYjwDH$n^^R*ZexUiW@E_tnb1s-tM2UND?W5ax|i z4Hq1iA|PN|(X5OsMhPEY^z^R{u#45p%$$`p<#}|3zq==~KN&qCfSC8o3^W=Kpl*^^ z;dtp>8FY9aVfG>*UNDotxPi7CT{Uiiv4B9OZ*?8THt)vq=LR2@bA z*-_a<+zs(pD-3&dZM!N{1(SzP=l!z@WXQJ!JDeMA8tS`I>wtpfL*>N(tJ6RRKZLyhEnOytnRgW&@ss}kWavLQ{`9oNveMe7{kvFFQE7x# zyJn4;4OdMg&Nyy1hraabS?8&E6RcAlT8dt?3J0)gq(WT`$}+(a`SJ@wJ*M7 z5Oa1<3}Isu)jhIXtcYeqPTtn7gdWK`$tsRZPJSn+aXBXBX(S>Nu7eb;m0=nW>+!yk9VcH!`zxWcwZ37YQ&4li1ca6P}zy>RNLLbyi2;0;scg;E}G)68>Ezpn7o z%>%21F(J{@{l4E2!721OuweK%Gr$6{I=tHVyXoV{pNq8=qtPf@afo^!?0E40?&SPI zJ@%8)38D_Jmm*b;?ey`D%QCd6dhRqltG6k;bbgPf)+_LE&UXj^%SKC)cq7bo4JIGA zkDd_d%34xc{2i7IFa!Do zotMMDG8D^3wn{uq7{4;!oi{qNeN~pC&r-nwpy0 zN;7}uZOv@`Rc|;9arpMIF+P=VNfzNLmS!!DcrFDrfbB&+r`tXXrW&?#;5;xD%!g6A zW!)8zBKsIB7H@?%C%yeU`Io&6)C4t5@FG(E%8?$XVkU^A+1Za+@4PNJjmk_w+CKO;5fK&pzu1G!l1ygil?G;u`w(kO-{MZzZ<2+MT2bEJ zeIgs85slfFBUO{Iw}8@JKAc=Ix8u68W5xj2JmM%p$THhCy^W$k==e#yj?WafQ@u4^ z{o6xx*{rr4nV5^n!Ouav^;zdHUH>>ri5HC7Fc9@Zx~Df!UO|s;k%b-jh~}Z2=A&6@ z43KMr=UeH3@+@?91nzyl%x{r$`~<&Lcq6@b%Biv0gMAD>{9x(G>ari!PWJnTYfZ3? z^2>pc%3CU%FQmH+zWjbia;;`u`(o9ua}5kgA}PBr(ex$n75(4WuR{p^i@qw&Tdud4 zLM1<*3aW{%#TOCzLbV$FyeG{Y^?^OQ)+kLJVke7Kgi&^d4%&1l2s*fh+$ABmDDlOl zlMA?S;DB_8&^6j|VaD@X)5(6iRTYA0cC=?{?kwe*nS1VUG41cPuQfPxB#eW&oF`;C zhvr@w8EKj{W_!#^7$-a>Z3`i8JJqzj98_!G)X*IX5GHYPz(_0fHY!5SOEWXsP0`*# zgnDU{=Hk{7y0<*A#stjdW5`wcr8=$NFxo>6BN7}O95nn3loR4DE56p~Q=FZZx0Tj< zOhIX`=oh>GO7M5{QOM^UM|E?3nvkgGsxh8ihy$t@5MlG$pl7d|G7&C^N|t1o7k|V= z0LX*J!aR*F!oV0^*{7EVV~WYKozH4lsv3;|4aEyls|~3;3AY;Mi2`qsSFRlmg@R(7 zIHTf2k%P7<+7X>>Q#Bg(t%!G`r|9X)9v4Me#0{^wqcHx$GxYk0-qD}_H0af*F8!oP z+fi9X)A%4)3YS;w8kcwjgp4ytEsh>AAbRYT@%?$QZ1Hgv5Un(o3U?Hnj5jbaV4sNb z5aOg(QD`Txvy0npj^&~*jj05BAxFXkrf(doatMXaS!7#X!kBH zp$U+Kx|r`=pv8WD`?{=u(~`dPnroPMs`cH9`h5CRPA(ukp_&Q^I%8nv>--5T)Xefd z(!V&o0g&1!wBc7l-Qw{X5(~QDMqyq&;$r)2HJNrpdx&YY#+tv-0#H_kf)bpiTNbvU zE+j^QM+Q-UJ!NwD%1JA?eVv3oad&f#^3uJf;xbrCzP#pPrIBB9M&Sjs(>^d6hdO#@ zNK3sUoYj^5wUuWqtiG|MbXO6uspCWKO{b2^a@Sw~8EMHzcj+zWb(AbGyDZHt?)-(( zkM5ERJIi1Fe>*qU!0PYof3J&NMxK_YV_9Rf493)3&uLBa4WpMDnNB^mxbjA1iRuFg z-=*iBpWSf%E86pYahm#>x(54zo84#*2+Kbhzg`5zRjY{(_2rFejrDSwRKMzUT2a20 zC?fy2l#~WiQi^EWy#rV}zz(4t4IlW+WB1B%gFI4eV|*KfJD(QR5ETtw4U zMB`9%3J#I6F%%I7A4X8}58|9K?Y@+i@qds6FPMj2i-}1|^_xQDCJuvId*rLw>0LN4 zd;qzTc^Kyn>%zSQN9UvPEdoDW@;w1FoF0y7Na+PYV@lgl$%yjE@04hWpx0~h9W%QL zx@W)d+Z9I%xeLpNt0teMyLPtZB^(UmVx|S7zQ&cMNHv#EJVQ(+qz*(#lBjLju6{>J$~b7_yarQ8^Bz)b3I>*iYLM%0DA@yh z?1v-PnrzYa|#>VERT|eFDWSzhTT-Ph-@qc}u5QOy2wJRR+L_!%2Sow4un;9R|B#Prl%6VYsm*wXZ@>w-LxTmGI+3Vxq2@iW01pf|?| zbO|biM)$91FCyG7wL5TsFyXXd)GW+!DLUg!+1{r^!4^|w6iR#0qXN78B3vMtI4Gq( zcgK~3aS7*-AO_Gg7?~g`ebGU;QV(WsTG6KG{jII^jOOHCC<{rtNfDdhXzO@nEHFqI zKrz558ZT?EzY?8>gTLh(&>&6q_lPQIyrgsOo4R=SKHuuznc-DVh7a+QgUi7dtupmD z4CC7Ndb`%S+pDy0`+20a$%^ETBlZz9XWVk`wna`D6y5)v2n|K$?Jb;iLR#)JKm#3< z!7~au=UB|P8Z(CDXv$XPy)}J>O6eHE&cY`NN~B=C;uA<-SB+~p6TU|gefi*9fPy{| zai|M%lj#oy8348;TGAYtuvqht1;erqP7C5~tlRVD=FTk)v?8Ys48|W|>jk%qC zSzJPZXVdp2qy$>HK7h+|g)Ir0UN7}bav}p5|cYqJNDU@Tt#EI|3K_CEu z1w%GYp8()3NVyOt5Hbb?K%&yd0uay48TNa8316Gd5shs=$uVAXJ(h&XAKt_I>oK8& zH(`39;mgHHShS}?j{ZWCu(JlQ0Aann=J~2|mHF!OzHk-`iTq9=Xd`d*lBqNFY&W&> zY%8O2+&~&X01!BOE;>)1zUkgq&Ffq12`AM$y+~L!*q}`~N676^*?akN6&w^rGpBJg zIjREo^;OKJ?g3|^t;94-(Q}d?7@d&1-mS%Tn~rA!FBdr;L#iQF0cE9ce6Q`K5QEs4 zx@sK#EC6s?eyJMak~S;x`D_fb?6cpt?GJCKmHaTnGeERPjIoHH4IKl>g!MEj_t?C- zEA{vL2m3T;%A2gJ{iH3wLco^Q*~yRb3v3%G3x0sf_xV-Kke{z$w0aelfvQr%FTiF& zd+}^7vs$X@E5sMcr$u}&F6rrMLJK#?b|ONWAQrF<$bOg?fj8$MkX%+I$@M7hj_q|Z ztdnp5u3z#aww4v8xCE+9_9{!3H>e97VeyFPy~^BfY|hWRtlv_?tX3_rvo9X=;2QvpG+7u(7MP(J%u=c4@!mRVIdd0Vq2f(d|eaCp{I*x&Sw3;xu>ERkqP8er_R$-fuJHp z*$IDX`*&0yBMgYkA)b>@*s;5j9#8W{%yY zNxu*e2c-2pc%KXv>;Ry%_d&s1A>4o_e#j2_iwo)V;&Mc+vvn#Ig~1e9iD*T!ae(FU zaZ=tK95TC{>rc`y8kpD}Xc+v|%-4-POhKIrAa|Jvr3P7qUS+D?I*~GC^0=r3O+sEa zGICeS+V*Olesk7V)mpfQ#$t7OLo#JP;4}Yakmu%GS?#6O2nz$3)H+UW9j#=eRDi9y z-MA_R4L3OCC1)L^21CO=z@gIW?w4BkyqqorkezeT8y+?^xfVC`baal9rhn@7VF`(% zEPBRHzw)kjd5~EYq;Vbt%-f1t4t93bub#VOA^{~q5+>^D!Jsj==v=b} zlyfc}mx>OfAGysa{6>iW5kzRv9>~kTU+%)T zjbX#szi785j^7!NyIX&q&Ywl^KWx?d7ADc|+}zyoE$7Xis;jC}tP&TH&be2cZ`$~k zNY)HF{FTrt>f6=-QgZ2^5+x$B>94Vr}-{xK*p2qBkXwMt4F3KEY z)@gk&($d>NO;a69jm`Q9&d$tu3$imBWNA#xEc3{@EAspQa;e3uiwBI>ZHvY3sdF~x z()?nqNlT)!?>0{MtIZ#}nA$4%w*O>7#DNOH!nmUxG=c^KNJ1O6VvhxFQV~U7O^6#r zYFK+g*T4N*!Ltv!u+v`&INW6MEzeC6W*ah(2Ro}g-E*H3C%(9?26!}uUUsEe?Qe^- zsa1!L>1xp7QSDAA1@rT~v1V{c3(XclEq7MAM{!ycv=)|yY>e*^(3;#S6W6;yN9eCa**47-!?kn))ovjgLhp9xP)t5FU~fXdXh5$ywk;NtO=QE zs%KeHL_}iz1yxmVx`?rYjMT{QrkbQ0x=8t+v-TOvpjS-GdZr9(vY8|8)U=KELr@zm zK=^F`BB%|<&VS&1E~F#&C@N{^AEwuGe5|`t0%iBG`Vp@#G%wcgR=a&BS%z)2^RVu! zs&3i$;cS~%i&vk}> zol1WrIQD3BwMgmuqA1+)4Pw15WnQfj1TOA5cYA*N?W$uF9kkCpDPz>kzh2+=$AcQF_{0y_i(?Ch}rXZ?RQ$C7Wm&3EQLhd z|NgA1-nS6cn;0N{^@`4#Mn~nPpWg7^iUt42atMl#bZRQ1B9`^zKJ%*M?|-)Y|9+si zqr2~)ow)53L1My(pFbzgE(iZ9+W+g@-&-N%z|!1vi>kNPy?^yDc=|Jf_pcZ1b@{pt zAa8hH0-RJjV64tnY4Rea1V&Pj-R;ZlhLWRN3%iYIE2{iDY* z7ihcDD2sP z4&wiK{!d%+u>|@|kox}kBSAl7kQT8RUWzFAwQV12cK_FQ*nRQoFgEyOE%|jE zDXFXwuOYCuj>?)MEB75~cFU<;1w?n_sXDiQc5D_39Al2BpckkR&4z94X!p=AOk#}z zZmV7^lnj2sE(c}o_>Zah?|oy5XMB39#ORt29l`eC;R%^O8hnYh`SLD0CsLE*1|Vty z;7bawL}X6&8{)S@u?TxA-9fuk^~|^&ndKK!LSOV9Dug>eKzoxykz^m0#H>!B8O_HO zBb5Mw26WR!>&r_v)AN?I9ME z{bgk#hm>F6O!0tV07*w`6-qoPL!T(vkO1&q?!d|*Xc1M-i4$ScEp8XoB;6l`3^U~o zawl{J(54wjlCwU&uUfz1eY?tXwP*7=f^$Mxt(Ve2XfUh`bYyss4Yd(o z!UQ%TbOgFb%26dmoggXzE)73gGQDDTrN?>a-znnrJvpf3R^8q>fyisvPqV(NGA}1t zh4Wus2 zTF6wArJBb^`2GmLg+QNGR;c&!-Co<)|LA-J8_bKjt<*?$A%9V>!Guwcp|rEm?ew49 zRwDXBW{7Zl?}N~od+xcQVA3-*r}myZDJ$O{?k{UnR>U{TF1JYjQ|&hkLaP(Ch846{ zm5=n$;dfxB;wx;UX*PK-gQUbZliQP=ci8JsDfpM0F-i=<2{l-H-$QDku_EojIry|$FB zd0}N`;FF-BmYQYLJ0u9KF)O+E+*nwu+xi@q4wpj^fRdKw)%##z*)1q)71Yh>+(K^P`nOyeqL5^vCKz93M2P*6mB`Pb><`@Er)rjB<)a z(TLVshnh(%_~K4kSy`0Ec}C1`+msr}?c%gd_7i2t0{e5DPN~d_lUy<$<9KEbEVImp z&$#wp*f!a%BU%TX^?3!`Zn7WB_oWG6(9}&e43QZ{1pZSQKo?)Y!s;|tQ*dqVG=B$K zn|l;D$EAM`na;^H#%~E=AC;{#wUAGL{af^(?svy1Z0Js@<5XHbg#Fm8g-^Iww?;fI zozGe->uC6)0CP`DBbuaq70hQ!A-g!&KalBGtkKSV3w*e?#|!qFVYh z&FpB;Aqr-ERW={bDWqSHC$$W_zq;z)CWVnGc-RXxAWm&8*pZzbt=11v)P>p^obNbw z0H#iKfq-ps&L4@uBqFJQod_3GevTo7DRtKk`q9ogbhMA6#xLfw-6~oh@h~fzL~62j1A)Pf%)U6$I%9P zE%xR^f(Ur-2k*}u*j>o8g28}^SZd!vJrBOu?b@=;7?0+gt}Bh^BD?@&Tl9RBS}*F_ zeav@V1pSj13aCfH#-;bZI09v+ibzE?$Fxs!1gE2hA$k(Q48cPaA$qOS(t!Qj z>Lh{)`29fY@(BrH-hm(!2(F_?+?VR<+zUcbgGMU>0xA?YOsc_Ugh&yhb<~p(I34Tp zyC90UdCB~pT6lw)!eTYh-s-d%UvajZp$k*%%J5C}GQJ%7!G zq6ptv%mAl9f@TUYvx25*NmQ-od(a1(e0G=lnBLQ~g>6o~246@mk$81P;U|+S_U9+@#%;&g7oqyAnKt@Zbu2y$_4nV;QM)G{vD~ zVVTLz+HLqEg`kCCZ<%!3%WQKD0%TX(B8eFhmRXgBC(u#bz*R+}2{0KJ1hf&2n+Gi$ zy=XO+wiHgGTrH$4wVp=*#CJl*L4+VvrtCuuj`Kx-AYl~-nHZ07nxGZNC1fJ%P}Kc$ zKd8mI#R2hy#~J_f5lLNqC0~%1Q~Z41Rst)3^RU)iZQ4}SFxVWUVTL?63jt{u=TEOJ z(m-9_k-mt2XLC`xr!$0VDZFY{j41^nGHUN8CiUpf?e>G=0ZL{)iYS1Z5Z@Pq(pndn zzF2rC6MUjcehn3OY1HUn%cA)mqr`dYPm= zBSnh{$3EuwOPfNP_sGl{z#}S+jO7J31&dWorKDCo%cLbU?MEtVs*iw(X{7=*13o<< z6IENzJk6kj{2#nD2vy)VIZjDqyH)fHg63X-b+6pXSvAAJ=iJ%}DbJoA(U$pq`8Q2a z@UqdEI+_Bbli`U_yT|~n#kf#Oo>0ORs(m?e27?bH3kiok?#W%(^SekYXfmOdWglGk znTi~O76k%}($|5`?*WB0cg)fMThvK8>Qrq8rm569g!H26tp%q#_om1E4zI*IXS z_#*BsaV3@jOuICmA}E17Ru-l{mx9OL7JRo!aB+Dv&s|KIdLou zLGXg6i9wBT_9*`gYxsPjbs#~Y2utDtP)ndg#S5J~Qa>0xnw7B`C7)VBM0JkKN(_h< z4Gf9wb8rrO&S7Qi%sb|1ZJykI*X^Zyo=urNnGhAkl(b0H!cnM(6~P{t;~>BbbIK!#cZoKR7Qrww9BFbv6d6t$tG_FCrN_)!p3Dbj(G;)V$p zI&Lj%;|FjE5PzdX#t?uMHJG!z?R)P3A~K<`sUM}$(DrqC+s19iFzGBDC3Hgq(wt4s zWR~noA|~hJVu_54nrSMm6QHJ%msR@-lEniMWLZ3O2um3y39*cz`&T{m0?f_~5s5RPrGk)SJn)9v zJO6n@Nwoy-uZSp(eqW&+l2%|Z#8I_W2*Lh*R1kiknd8FYM$3Tc23x|EVU31)rm~0_#xc;DM>&T+i885m)Vv_|PFWFQVdF_|$wAg%N6!KhPk!5= z%ri7;-&rLZDKQMOQ-r)grjK6QkH^Q*r2SN@Jf_E-n({ctW`Jo>xDW0!LV4d^0Ag~5 z%BkmmsCbO4z5HZs8}7lifH*FspL>2@q)7A(@`S)kh4zUUVhKPzhF4P7$;->DDE#I#whBY3{=EPd(&Q~^0Zq)zdDByM_Gc*dw?3bt#J97le&FK9 z^`#ig=uwk_QU0sI5{!L|g+mpb7}~KAXlZUGTpd&V-zd|ZaipP7cZ-XKu@7+n*jfJ3O7-ALPKR`-Yi9JzvbicBxa&0k!2}3Gf)SOceWB> z3z7)m@!aAOl{8;gL&P{CgjL~^0o`CH6)&ViVJ&`uta)gUb4YrcB-Xj;P%{dNEZN@< z7LRCXgBCYs@4TTiy}bU+&Hdxl2gheU2P8JU{Tllt&yd%oHKrJOKBjHr)Fm<6Uhqi7 zzk-N@)=#-pwUEcDGEf(_!mZHo%*%UMZ?D$uN4vNFq8XL^>vH%DF=-j_T<8%Ijk!1E z?Q6N~)u7yuM#sFZP@U4y$bZ89#n`&*MeN?!j%b1}m-k7R#5LhbQbQm4ZXdlSnh|y* zo{Psicpsj;ds9h}d#%KbJPeT!Mkw9rVlNwGqS=eUjDb6x6pl_qL%vVr9c9n6?Qhmn zQ7+*AypG>-RJ2}T4h0k=1XMhmG6LWM8Y?x4nA$cc_C9J`WC4&acT4GdYAg+nXt+Pt z^A0(c*f4d^hC#3~N)O(b*96FVdeu97CBx#NlEuXX!oxb-DyJc9Ecs}g8=oTHRD|I4 z`^$c_$qYx;?UMGKV#aK&g)>cTOppe(2&0|YR63GsL%p23j9Asm3+k=E9qcm}9i#;a zu9LT5J;!=heJz+1g>}Q*AR_YM{ap>#p+9>3I5EEa#}thg9H#v7y#`U8-8|+I46#PV zL?hH$z>_77A2uG}d7&9}X2SI8KTh^5xT~!HP7LPBah3|5UIAp1XG!7QZAGP%njPQ= z%#n*v=Gy-9hf%~z zV%xYdNqIr~t=|il3r#O-7z+fgQDr1KoXp__ZXi$>lHn)|{E(sbWhZrKb7*|)kSb;Y zt2DemYM>b(5}IMq1^_Jz+CeC$*a+Mbd9NSOPW;~AK`+}|b^5l@wSH$89o@Y1Qt_AB?dPw0 zw>qNhdyAjn&#AMoIB-AryLr3jjNNUkvg+H0^P^wHrM!?YR57qAxa#fGW%Q3Xzh8d- za)&twzT5Y1tE|t%1;ff_1kRtXDz_Xvu+N#IyvJ_-cVAxCuPAc6p?5W&Ji^sGF)@)0 zc5Rx!E|X**ViUKQJ`WzC?J&Zv?)`lG?G23pxM}_E;tzAOFU7@8M_s2eLE4-!K-X1^ z@;`$*=!Z3``;g;DtxoPZ#wzjjQ4OO)=ttf;JHNTmTEK71g9ME`X>`a*7GkYC;&lO@8~$mE}_=cQ&t_Y0pTDRr}B5n{9c%+ojaIu>b-mSRPRcn zdiA%?%mquSMng--NT+IcU4=9NN<@?NMpRTmW!_y-jt8Zs)(j0msv{AxWI}2BpeD0$ zsRZ}chke|KlEH77F;xj5*o>aVhb7xQO z4^3IR`jaO4q6$!ivJp^-u}oGW*MmwVmJBfB!vnJw!}4B6u4y!rms+i31^Nw8S+L^V z=q=@2#IJ~bos#P^g!F;Vt;cWQzO{%bV28!GzP+nMonqF52@DS83BHF89U3ro>p=C^VuS59#p&W4B($#23tBdc znsF;0j)8=8wY9Y;OrBgfW7v@uzK;qpOT&PFQc{j;fP9^NFHO?-(XCs?=bk!u?p*eP zeo#K9*}t3T;rp9;9-!PP5PfIWCOaFO9~o<-=G9WABWuX7^)*!K=a>K{{M(%@AHSjuo z|Ni}jyoO0R}=3(7q7Mawuqr zQHCD-)9>ED(-5ZtZ(1BV-k_{Q79-JOZDJrCuKoGvK^2M3w!$p1{3#aJ|+V+8#dCD;^K`Sza{Y_q+A~JS=UK#e*fM3V(anHo-n`} zKcofD%Jh(jEiLX0)w^cWD(UiNR9UEbSfQ{(PUYrAai?T58Rs#OFNhxWV@w4{pme#- ztSsEc4&(+&87Zb6)$n&2jPeC2cd(Un)DMmL=ptP3BXKmU-rlF7vjA1TIRr6k8~>1H z>eg`-gq2$3qkD|=3jQ$_yAWM&Hs&}n{t4)LMO53E1pWX!CJsHATxP|ccZH>cxnR7~ zke)s}J3I3gD}*s)t>5wcF|M`NP9I*MWHH#I!tlFN7V)85MS_qlMhtBb@nk<$)r2j9 zzqti}vm=ddh;UdCX5CF;T9Qj zkI@O(-r7_PX*;J%FFJP?+rFdt+&=qv4-RwfBnM|(SXjX7c-+w70-DZv`hdp9M!43x zoTXYDXO4rob*K$K%gyy?$;BX@~T+a$B!2zI5^te38gqnjKK#kKrVxlIy?N~)|b&sNA`WLR}JrX2O-$@;-}}D z9(IflpG!+7dbKA9OtT-VFRSv|&IAI{X!$D2#;i5E`Gxcao?M)-ty{+#cn?Tr2FFA# zE#o9pTirg<#}b)phcf*iWSxviy6hRjs_N9eT!XFUX3A$fPvz3i$k(BTuyL%sAZhLP z?HO`8S9qs%gyJR#Pkykx|LmNstPpv`C-tu#^witin@3>2 zRRQy-nRN98#WXax$#KgqVLa~moBQ~&a-^=pkV_5^Yxf!@MO>Yv5!H0qTu4Vf%8OS4t=$ACVI31<& zXHxBp6+0>t{EUV~^y

mZx;e^paFB9_A zt$4vb#&a5-#Bqtu+X+E4kJ7wp6vnT z1bdDeQ&)!Ib1>p@yG>9e=N{Le7zz93F(n74kCXnGYU%DmOqy`gLgj7 z30@uPlHK!TTx_guoVWGKv3#3fe))y1mOSbV^MTEa*>W}BRGU#!!UAIq$QHIYepI+5 z3h+t!DxeJvWO;}^#@Rh12axNA_6o30K6~>#D)^-k`8wbGodgcmtEXT$}Ka|)BKSTWrb z*jH>#Q`@@JR6Ya&c+6*<(TZX=zj1UJ6yh zl8Jr68mU%Tjp_|$^AE1```0lOS<%|Ro_0#JVH#0utR*4C!Lf-ac#VUD!^Mb*u^b8R z5wGhR=ydHqyO&Q=s+LjJ4qDDmxDtYqc32A84 zPSBL_h^vcNNRT*~LYeT1O+!6zS#?MwvJ0I{B>4VlW)K~S?FbppeT*&nG@W{6*c|sN{}qfc2mz}5tbDF zJUwcce1$wZKK?XvmHM1;F4M;?MIJbPOsohpc|=UKrT%f{B?uj9!b<>9Hsq&AUa!nK z9~xR_)}(wYx?3%CgT(xRoTkepOm1Pd#;LkaX(<@0N#wLhy*8O$# z=+T=v;Dk}Fx9rTqZ`HPba0-)rh|F>h4YT~?qAV_wGscUs3WeAJvFbFG+BOp*zJAPm zLTYX$7U9HC;u1#}WQ%)P;1@cE1$j|%)GMTH$C@=m2&9GeWX15aWB^uU2Y6Jvyml>D z?%I{Ovdr72&@@U)qgHxNz0^G|PPNGV+n=qiHL=Sn6rw+ZG`ujI#vQ-a==b8)N!AvwH%~Y7WjtO+-!z{Y*7c_q)AB?hgM|~9*Vorq zxKJ*(Ot;fQSX&Zd(pcr%`o^<{fk*gz5#})_ipy(sL}Yl)b>*QE-RK(D{ZpFkUFH1Gg~%+vY!V2ChlteWMH0^j(i($Hzn zA65$6HhJe)Eb*O1JDs>lb|)7HuIJ(=zwl{4+js}Xd&nR%~M#km!d@Te7L2XEE7yzrck zqcpb~|CZ%mZefwIrgL*vtzhLb$5oFr-xXgBO(i!CblD;TQ5oUBI*)|uYh1z zBt?9v>yJ_5!k2Xn$l7)*`{AG{`=w@P8xrnqi>?q+&aPd%-e*UTMS@1-=uoL#R$qP4 zs%0>yYDZaEG{)>0QF=?AU8|E*d9E+oAr3d>ADyMU|2bt=ic==x8Hhky=NCtJt9NpA z#F|eMR=k`EYRr|jr;};BbXW1rbAONoVC2c?o5T0qOY+Y8!){Bj-K?VsB7swc_=A8d z8xjLvVMy3@VW3|AwBXU`jLT8&vZ0Z^<0Ody8I7ycckbK~Aq_E7rbVYMF&PgY%=0LA z;etW#Q)H@b`4^XCExA@>n6xQO#4i^v3?W2!?s_}ZaCnqWiDl7E?*IMO<8#jBWa|aW ztacM>VV)&lYh{b`bNmtit6hx@ynVvOqwoBaMb6q9C~NAO?ucG00xKvue=I(14l*{GJrqSFtM zle|VOZ#>!VSU*UCu5PYn0j++rqHb-rK*{~Yh`yKvGvvZC)&dH_=87bVf8x5`>V5D& zn<9*3K*K$a36mx{O6zUyDjqO?9ztHJQIu@qW&ND^(CIb>QC6O&ZcguC9>#Zyv=$y* zrA5+R{k3t^;vTW6l;Q>?Mzu#=>lK0EXME7YVansxzwsup~Il+Ny8aE|- zBp#JqR71Zp1$-ql;IjUdDaVNkh4C1hP_u->HDWRT0K(F3A4FiE|Mwx{lt507w@FuP z?vQQx>PY+6S@JKc@oFHa?y;o8fWSKDsc zg6yC!u}m7P_Myq4d1&pkr&#V*neE&tlPLsY*YTn5@7JgckE~m7$6l}j5Jg0~G?)2R z7)iVh^)6^APH7Vs1TIOzN?d1;aj4Rl4+jZ5OQHk$qdF?WW3+JG1j=;r1*pp^_lX$h zYk5+iC6;4@u9?vAWR-A_NJ8o~q-KJj7QpK{f_ ztao6KiF~Vp-Y1Yb#a1+Dez~5g?isYhw4nL)=M)K>v0_E2e!m1H#F&nuODMurT_e7W z_Vno)7^FJyx+`{9g18L;Go(hC(3BxQI!A*1T}u-?JQrp;C^<9H3~Rs!T)%nq7-mos zplwO?#8#7vZdko~FqS^)V@9?jups^c+qU}t=k438y*29WY;Bp|v0>f1VFmV$-sPzm zk(K7+q10m}mJ{FzY)=dj#GIRbWcx;t5~g7St^?0dMiQ_e?!R`z9Rfw+t>Kx7XyvA4 zB5_LVvheWmLL}yvCq$kC))FpI{H4aJK1A)&(Hrmp1RdO9=K)es7_tqCS>-LMwQACV ztS(1g3OQw#>BUTYniv~<3fnfy%4*S(CJ%X9<2#wxl@(^Y zaKew@yjdk~Lz;WxM9xZ5kyK(@c)4}ULOe)hb>SrDT?nJ0a8f&y>io<`^{!~nHW~3< zY34@4wAP8oj~%n;N=y$&H+D&qH}Vr~EhDtVbes)Xw%zO5fMNWePcpDEvCD*kw<{%a)W3){u4%tn#jVp8#v4+OdbE_OP7M|79oI$aIjrpeBrG8< zY)=6>J!hddXeqlznkAw$pcC#TsIC#;iQ`Rgrc3s!9jRe^Qll!C2~-IvND@3D~d|G@pX`b`bdJX4#2uhodk{PhukE}fPMD8+ILH_n0KNF+sS?3OQ$ ztJS{T8bJZaSW_gT^AYZW>EsWVS9oYj?t|B_SBeOS59RFS-1NdEV|PT>MnseOXiSt% zgz1=NtSxud!zFTw>EB6t5#{`$CKAlU_J z_(W(dj)IoMlon1sR}` zqudFv4?i}U2{(@~BIGLsTKW+l4RN^e0xR0Rdm(+iA|mPHu&^-sR$J@PFmhtCIwF&z z34_%rExJ)>8J%cUUYj-TaxWX}z%wf<(pE%8N7J7q=JJZO&SHp+pWC~B-MRwrw=V=T zZfdGXe-m6qpbSmoN4~SObG?4k)E9|A0+?WRM6Q1SAowNn0vaNeV4We$iJfFmL@I*j zZ~F1iC%5Hb{n+MjeUy}1u9Imn!&+Ef0ieb=7WyeLwP0f=ii0lh7z%!67+g`fb3K8bsrDZEz*B{iz~?{JPH=fn{< zwR_jnTvT1?XIBV%CN}WhyS1Xg0Wpii(qb+!{?V8rvh(?vagSd6vsP{1vdnYSrtd}U zBrri}ah+UaTa$)H`zMLa9;|fTy1>K)B9>WuX1yApD3(liv6{gAUEx3(0TcX~3FN~3 zfLt3uzA!7wrnc4EOW1ono+zl)UV98QCnAh&)A7ZG%3L;){mn{elWKpi&$?Pe)=({A z9ts1(8G5Z5IZs4xz@%IWu>?fzM1e=^rTQXl&aMM$g37go?G%_U%kunSAAy$j?K_XJ zCVmXOsYv~G^;|ZGTgHg7ph2?d^_@nCV%#aEXi!TK;M%#x52jbd9~HaCqvPU|%B#ic zBDoPsKR<)+IroTwLNQOAG~rsAxoDMSU^lw!^YD}6l>*ONzFc4)BDf*$Y>H_)9^%9m zIV6cl$g+FZSUn|e+q&!a+0y4vssWwHSgw2^(63n6{F^qya`Es2kLF~QcBqZ^e}oAe zyT$@oS?|47oRdwGZj{M)VBr<92WE#|1?)dY@G2@2%t{dWHwVu(*)E;HLL~R}h`s|c zqsWD_|5~=F^q2EqzcUBX>gOO++sg%`6**0cKvxNPFj-jQ-0gF)+=_qRERx%6j8jr- z*kIz;qb1ue>5K^Q8ynNS{5i@B2$7P@xmo*YJom?o-nelC*Et#lsPTb;-Vsqe5kL)G zXZx0hDwexwzlQ(3VB-+|g`VV*puHmTeP~lqQ&YFz-F*~{oun;+J6D{c+A1?HCms{d z81F4IC6Tb!W!ni^iA^>(k7gg!cnjs1JI}t}O5KnO**cH#JgI4dNGA$orF^D{$ zWn^s6&d4(I*I`wkT*Ru2suEus^!6FGxq`AsRP-G@KOPMb*p8?p0*hV;PU7yhWZg|A z5Ca;cd@v7a2ib`uVylTaMB(fCb7O*Yk*~+Me0;Pc@0UJQx&7>8UTe*|9(81Pc?-4r zG4t);+}_o6^gdY|zZ@R1GK~%5zeL(j_5?=cdS})8^}|0E7zE!Gh!Wf$##h%fO(k6D7YvB20gyO`tAojsyIx-L`;#o zPZS-6)elETh{K%p;%rSAK|bfpW1}a)is*v__&k!5vY$$k{ANY%lj#*AOA?@r;FWAh z_Er_vL>z8T3ub|94&(f?Wg-QQaH}A^lE?`}wJMG|35RQ0!VTVua?3mCN zIgt<4{?Lrh#T2{5#`T;tv7KJE5i*l*>Ut~Hosu#u9wdDM7*jIJiHgH;5v~hJR)}bd z%=F`3b=kNTzF`7Qu=vFREsEl z;qoPE1;Q%%>#nd;JC?It!?F+OB7h{_5g_EmG!D{*%%qn>W{f>2&SfAsD^K;7gPM z!Nqb;KnP9oZ)$3N?%v7XmlEM%b^8_zYMC!XaBG~`!9xQYQxBy;ii*5(?E2I7rAi%+ z^M3XeZL2L-nYnxhuY=4x=}!2y(KgObPSZJ`+u6$-O8h~Y+JNOj{$(7yY14ze08gmv zJGUd-9F!0VnqS;0VbSPpKypbGp4AMIJC-K=ZTPY9kO=gfm{|~i&Evh~O-LlH&xV`T zed48AIbRjwVMth@!6Q(CJbWk1?p z9rVvT3{YOE+H}2?qiUlZFwDeDFIf#a!(8x;G=VTxd=t!`Lr%bcZhwxZR zN?q>f<;d)DUQ`EpJKjp-a&J=7XyVL*)@u{VJ=NyiXW-^YiH&#U(&kavo6^oTJllu%z~0bkYKh>U44qF+iemJfq`HC=(? zMbGA5oQ(3k7+okFz}-rvBNPknjF5}$B;__}{P@+321bwVtxEswJ*y@b5d=k1^3Gvz zwko9>%y+BQ%MmW{A9v1Y@7z`47XBenbT_<$PWjB9>wv_=5hPNYDsv_xvqPqvMh`g- zU`+6q;d^+%ugWZF{gnJSYypG}-oKl^>d6e8xHUiT&+{{?Xv;R*sEFWsG$nj13XS2U+lt|1X%uls1;Y`3>+sRD_ara2xCXXFV|sPZCJLv> z70Z|M55YJz-On0}^bPq-fRy~)4(Z&hby zus|U~7Nr&)qQSpdmOK^^!L45?*`Tto#cdO_fh;R2LUi(0Ev%;y_vFc1crtXiQf0N2 z*baalPuZ_i`#9Qp!l;u0bBjBdELK~7?m#Uhj8nK`h zGL~;yX$3rHyN)!+6gVT)+qMma;d)L_occR~S|P~XKL_)?dh&k4dPt<8FhU>h8c$w? z@MbuS6ksj77C|~tig2VlekKHn5E*Qt0CZG?XBLJB6LQ7~uiNdq9ZMO|f0F!$CotYt z0eF8m;eVMu)0sD}paGJ1}R0Od}%*~f0dS=(C$6Cmj&_#v9LxiyaW^*9^BBJ&5 zNC9ixHiBXhNytT*C7bqJ3lLJ|J&LSq-mn6k5c)FZi87oh)RJ=43OW63xwD0XkKOVN zj3^QjCB2TROpayqbVn2ETD5|so%ndBBB>@kB*~sz{lYx@_#7xM#ofR0gT@}E% zto9-!kF|%o-Usj=YJsxfboDJ}(&swEmiQ#y_i8u`YkZ0hwnLBe23I2jbLWD2mG6^-NMUq#~y>5yI z1YjnmLhNbsIN3K=P1piY9vh@>2K3riC}mHfTo+xs07{QH@`QK&HYC@GjH9eh&zRm8 zH*5Ozcu=*_1n4o{`K}S;pQ=wn(UV4kc#>P=$mzZd_Le=D;!Z-#ZYxGY+K*zVoNMNF zU!g7Ar5=e`#o~rOKDekDe!$UP^2|mP6N}Z9jA9ka?==C@q;-*3;AibxmVu!=c79#S zqR_GOy2BlCm7x~l5kF0TgP20vn0wjc#lMSg_fQ0{OQaCNL0JL%jnV zb_eN%)-w5E@PDnGUApR`0%1w9?jYyiFNhvq<~)+1OdwI_Vta?e*H0tN#v~)fm2Nj% z1PBtUTJvBa#Xu096w!-A06vWu?_$(D{+;Kx3BWG7fUqj$Awo^Q{_Ul( zv|~XkAlu~DArDkv(u`3kP!HcEI>9tmuqPyz0HWMhy5eN4>rqHkdb`f z*nR%|-STQDL8*5Ik+`zYoQVgNif&ix{o=+>CM9W!_!juY&&lbOCj=EE)Df^Y->DdJ zNmt9S8-OK|MQLeiF|Yfn)+5i0Vw?j6(7|!Rs%*3n#UBzZ1##9?9BzI6A*G??^ zX;z1n=m4}PWX&Rki^unKy(<7n5q$|B48E#CW$vUL5+v}(s1a`iEQ3})5jmx-mBt(j z8bK70eL;o^wtsLdb()LJQa+3XMJ{knh|aT?cY=q+_qqv)quR3K*f!ujre&V}qET4^ z{YXo-4U%I9T!QQE^)tDCN%UGhWC0J}K@dCN1x*Gg~Qv2MUGVdayKN+|r7D$g%Jx zVvzM`(=>oM%5waPPn?-UTfVt@XaNrv3i!BHv}?NH#@)(zcu@qnLIMecf~is3BIgf& zUp$)XBcyyE>n7c_B=h9SpHlP3!=*bV6~j~e^}Anbrio!Tdh|(qWj?Kak;rPuA&`_c zsfgT8f>=AkN6(3PpM-+EdPhd)-eJH#TJLX)+p;;t5*bgK%^MouoGnqrFB?wPu0cb?yi@ZJNt+&oS>Us0qOBjl5I~M^$ z8l;g*eL-{4~rMtn!+sN#v{2EQ}g;!hRYX zm<)o~6O0+WF{LyS5IN{h5Bqc&%XyKLKrW3!)NxcfwH2@dwuVykLDiKqdn%b2wP~@{@gh;hUbGo8qV*iNPAi+DWx|>n+f=Sn{ z<4R^V9vmvcXLyfu?&^|Gb$12wZWiydZ>8EIUy1t6Y*lSjzLMzycO2?l!?(Q0Q?Ze% zgjY@W5w-K_kK6e zq%-ItkzhV$*Z^E1f#bFsyL(kP|uCE~LOdB;y{RAyIlSkmr*jjPw&$!b zlyLbSiA)x&P~vuy&8sYQ&X2Y0lIO6QNuZfYyZ9CbK}_V+1ru|m>#w(8V5WB3dMACR zh>_QXmmv))K4+?Wd@mMwc%`|(I%n^DKFa*c`tnGrH0UN%woDShthhgfGuB&Ob^Nnr z9yePdO3#tz^ZAjboN72g<&zrcS+Irjf3y}c5Uw(g)l~7;^9Jy$nP=CfEiO1WC!4HI7cJH}^f@14i|*b-2!HUrXWrWme5J(K zYRfV5Wa9v03GIySkpgLUN3?N01YXtQlX43>-L{sFfsA+|F8`G zv7Wrv_xsy2H%zWaC#2GVQd?&Z3!D*kvsjr=UVKa}CU0KCfD57*wl%zW7$RgvV`9NK zE{gv8_n&dgt<1o_`!|W41;B{NYoLjdTZ@e5qQwALY8K4XjiUphUsUWl!^$dH*qEfK zs&8G)X*6yDglRQJB$z4bD*B3i+O zluAG<*#LEvys013(S9hgh(bk{jMkiylc1aPL}=K+zdem6W+P$2+GBam2oQI26*e9B z))<*pBu)6z&gd~=?^C+e;LGSy`54w6v%WVoBE|Tu+t6B!KwbC^K+A9$DDLaIVA1ns z8EK5J9Nqe)JK8#6Z1D2vGd)o?A;Nk8p&N91>u-9C<2AiczO8#;d~%-NHKQ!`1lJW< zd@#T52!XLcPOpQkjC{rvFeE>kGYVf`ehPt$lbJatKyyS%*n}Z4RCSE%XWf8(x$Mpd za6e{hrSoLYZH&0~xbK#?fzO2Wf^XdB%;|9MReJ+g{fmG?3Z$JC42~@*%SI-|9_Bt* zR1ctPTi`NWHt|6JW6vA$q_I&Ll!S=ii0IS$5H`1=-p|TVx^cgo-Q8=rLP2}VloWc@ z!P>#WM$wcFa4_#t2Nr%^S}NDuxFR$4K^2%3r7={`1V`DJjL>i_VI~ipOOqJgdZ*L+ zI{CMzPydt8)84lz7y~3Y%A6_}Ob_PjtVO?>5vw|RROw<#AZP`@GiS6-$m!e9a5sjk zOxnm$he0fx&r1$^atT8mW&S4;+w#gMbpM)ToA1`G4cxk~Eaja%f#}KK@a9bG{)HJe zuI0|sDWYHnvX|I;$;^BNegBxd_Pc;=Uq)6fM`Gt&@NAJ0nMS~st3Zz6tf5%eV0z9= zJHhzVJb|IuYCwR;m~Ipuf23Wg2Fn`7WH#!aw$SFh`lp$%LhoaZ`@_L6e-bkIIfxI8 zlsf@pYbVT_LuSz8&-!z=-psu3x#(XnoBm;-m->>dZ>xWFr%&`93;Puh)3L@5=5W^4>WNc^H=Lv;b9&%d3DtbLQXCF37xn zh`?BZuuR%x4AsR!0ZMC$zK@6u>6x}PNA?6np|@}2GhY6`|I`jr=MJtq<F+-NVI-YFX=kj(mS`fgR8r&m=ZH(HUBGtY@BcQT&F< zToO-6YZ*hiPMb`r)mf0lBII>2?#Sw(@63F|aWA5ZUBqO_m>3appxwD5+_aC-P2S z&^O~P?$C`Z4$uA$5jntudMu$&)7E+iPQnN5Ue-M-UPhfA;Em$-tq{Gnpzo3&b;l?A zkKdBqLDQ?d=JHgKKnc89x-=#hqFWTbWZ=}hgHOTbsN^=Xs(_9PgcPlHQ#jTH!UMVJ z&3ml}dXaPcoxc0HUya=$JK&>j9b`B+Bkl`0YRs*?Ho?f;3OZZ`+r<*dDS@VjOA6h>uEAtu?&a4R zy(-8cpn$54t>%>woCTHevEsauvXzpd0%6;;>y5{+Au&QPBC5uY;= z>>Cgc)?DamfR~MgiKR8Nq?8`(6$G8{IrB@zUZC4l2PhDopsR@yz1@QRG9o0}@zF^5 zgwh?~{8CsO!}%&AxD_hkX!2nVfYmySQ}-r)3C2zL?nO=DpV>dwgrWv9l}2g*6-wI|?WmsXW7B+3nq(~01q-V(C`sKRP&pf!OO4<49Kl7W zgkLm4@@6EPjR-chF?d?%>N~-cpRjZO4~DMWLv7OjhIuBZIUwme3bZdhM`_n!_^7-R zpa`+FG@}NG<)4m=5OGq=X#$VKCvY^R(ay4RRKNQdHH0$5zU)R<2xAv3(_Yzrmm~vB zn?4sBICXl=aFSy>iaPnKj!aJ;7?RZ84)$gV^?b& zZntnrj*%z{6aok_2t<{Jo>>cW2|!QKR;IzoH?di34`1l%;Q>8+*YL|LC%q8>Oefq& zO>6=!v}0(82S^0*3=4G~h(M>UZB0_Qarns__EBG9&R+e*y~;i4lVgA16yzheFuiCh8TnNIdV{=%wMIjP(M2UX#lE#@r}BygAi!C^;|Z6%;YzbkmrTcUULvj z)%oY$o`#Ee?HLG(jiC}Insu7>F-4a-azn2S*sWvKZfpvBhx~g2r4= zZs<5@FQzt*av1;$oGt|FAHHu`d@WJsdgWa=47Xl(3u{PG_#~m_3aZ77C~DBK5G?(& zYlIO+jC3NvkSk;3jOc&p@DVZ*#RcF`o-T+zNDzYaj@5Hvx-1>wm-N-VYVt^Q(pG^FH@n2Mk0>z}gg{ zs@-x*LCbvOJ~zQw3Lgd-Y^Y~1HxxOviIW2cMaLE+D*}-$@44Y~Dr6kX3698f!)Ks8 z&3Q^DL$SU$3>@^FAyYMM>o?K*1{}L(@eyRg>+RK{>6B(^Vd|vRYdR!~@~nhyYvP)J zz5)m9vaQsfwqc|Cn>d@+B~&DZe9fZ8#-hg4g^!Bk>dnMmm(4RNEuhaf9Dw&kNzn2! zAA(F=UXFst5&o9uhI4Swa9Ue}L9~O!f=6gXVv&qzB1QdZR8j zSnGOcxfsDy(V-g zilTA$4Xjf+s434yxgVK{uYejNJuh)o)jAUGs24djfVe3^r2?H#YQ*=Zii75SHrn98 zj2ltRNcFYmDk9Vhv9Dn$gywnogObpm`;4Jhu0ff}ISu9sqnv4YUZ=HTV^M+G=X(#% zQ3103K-^uMbLV8^UMD zZTY2Z9Bs=Cb%P(+HVPjOx;261YQ(Vva{{n+P^pBxlHL%*KzgZU!Ac-60Wd`TIASze zAWP*^h{1?rXiBT`Kke~5r<&LX60+T} zN0O0139K2G3|+U3$afdkN1?>w*HEw4pm`${m1VTRXONEAiX6K(>i1wHOqD(Za+~!Zu&IEzXHfL@y_Lq$Wry94FkbWo;Xr* z3IuZE9Dt;0#1I3vhOM>esTpGABszfzn?bU@W2lW=(ED*C0%p{O8D^1(f^*u%7g~fp z`9b4q49+qS1)bOicu=tDBZOa>5p~Fd5gK5~3?h_iu2{YTC3v)t53$$D#{)gg?7tQ; zcd2>7#V1y=*l$IPH-8;fGxhBs=O6pS(2p;KZJspg#W%ikx&O-0jlRs3;F?#w%wA^A zA2sYx@3?>d;s;G%zjk`m$3MUJmyNS7opzL6D0Y}`eR$)Mk=)cO^|}J9mQC=PO-x?Y z{7wJEG0$D|Pw!s~|C_HY^HzrSU&qsG^l)m1kz~*DKw}@IN%F#FJ^;o)b;K_p^`Ei) z?#z_MeIK7HLX(U(IK2WkHsK6a^@N@m`84dp>9yN(+xtFca?C8QxcKrb4%0`)w1Vz> zX9P^j@xw)Z=a0S!T3~855*XZH{anIm4o8l7tMBYu?Aj9k^booQ?Bfzi=+es+5^9c- z=K>q<+_Qq?l;Vk)XXUgnabe9d@0GB^>U&n$yl8?gDT3WY`w3bTP9r0~A~oQl5^4l{ zjv0OxbPTYPgSKtHC6!@Kt`2(_*W7z(YvQpFZ1FRiBP#pgvT+YYqLK?2KBhW-WV!URJn;yY__`Pqvne?O-0$BQ2P zz4%c?PuDaAifTo9Lm*WkVpke$@uhjjk%OMG!L1$75<+ku1h8!FOiR)E+46`^BR#f}WdNy*POlpxIRhQ(FT@`Wi6`7!0Qta6n@B zv5LmMhLg@#bf(F$(<^wZhQB8Xnk({={h~nqr*FUn^M*3<8KYw2 zEwO1aYPz4vXkt~fbPH(9xCDntwdx3}q`&9lWRdl|{p2*d(sWmy(BS1tZV|oPcf8iW zeCZv%Iez;;|LW?_JN@F`<%#~{0q7=9Me|5tN`ms+$qT>vg6^K*`?nn*SEE@xwqxdq z70Spl_K}ka<*i{H{Lq$egpR1}P}*Jk;a8SOiJ>>QQ$r};UAw}aPSUaR%M0#%ZhHkj zWTPXsM_F9MhT7+>Bgacww=Q)`#G<8RHe@=JS;Y8gAg&u^vST%#xT z8ZoK@zc-xGFCJY^lVM}^p#0AazDHpIsOiPkbB?UQ4kas%OXFXLM^k~y&6_K_m623X ziunb?f2y6He**O%&>g?W)ag(1z=@olkE$r_%;7opt)GMvu#*p=5&Y-aC$3>DZgK74 zCh7 zrw31DR~*R#ia)!XQG1<|s=4s6lz!H$-+2^La~njU6E&pKz1^_r*P=Ifxn(+sk-kj>I+<(2&j5CbSzeeI;Bk})r6~qfn?(Cej|D9Kl5Rv1(z)xHPH-8?u zWzHu5E%?V|`qb&uKAeU>*65@(XZmz>t)Dn`>YS-l&xB|gh`KMZd^d0P2z~w+F245_ RmzIgcCo2}7n(zMAe*mI +# RFC-[number]: [Title] + + + +## Proposers +- @rmahindra +- @vbalaji + +## Approvers + - @vinoth + +## Status + +JIRA: [https://issues.apache.org/jira/browse/HUDI-1290](https://issues.apache.org/jira/browse/HUDI-1290) + +> Please keep the status updated in `rfc/README.md`. + +## Abstract + +We intend to implement a source for ingesting Debezium Change Data Capture (CDC) logs into Deltastreamer/ Hudi. With this capability, we can continuously capture row-level changes that insert, update and delete records that were committed to a database. While debezium support multiple databases, we will focus on postgres for the RFC. At the end, we will explain how it can be extended to support Mysql. + +## Background +The architecture of Debezium is shown in figure below. [Debezium](https://debezium.io/documentation/reference/stable/connectors/postgresql.html) is implemented as a Kafka connect source, that reads change logs from databases ([logical decoding](https://www.postgresql.org/docs/current/logicaldecoding-explanation.html) in PostgreSQL and `binlog` in MySQL) and ingests them into a kafka topic. Debezium uses a single kafka topic per table in the source database. + + + +The first time it connects to a PostgreSQL server or cluster, the connector takes a consistent snapshot of all schemas. After that snapshot is complete, the connector continuously captures row-level changes that insert, update, and deletes. The connector generates data change event records and streams them to Kafka topics. For each table, the default behavior is that the connector streams all generated events to a separate Kafka topic for that table. Applications and services consume data change event records from that topic. In addition, Debezium registers the schema of the change events in kafka to a schema registry, such as Confluent schema registry. + + + +The schema of the events for debezium consists of a before, after, source, op and ts\_ms. The `before` field contains the values of the row before the operation took place. And `after` field contains the values of the original database row after the operation took place. The operation is specified in `op` field, which can be either `r` (initial snapshot), `c` (insert), `u` (update) or `d` (delete). In case of insert, the `before` field will be null which for a delete, the `after` field will be null. In the case of update, the `before` field will be the values of the columns in a row before the update was applied, and `after` will contain the values after the update was applied. The `source` field contains a list of key metadata fields. For instance, debezium version, database name, database schema name etc. In the case of PostgresSQL, an important field is `LSN` that represents the log sequence number of the change log, and determines the relative position of the change log. + + + +There are other ways to deploy Debezium, such as Debezium Server, that can write events to other stream systems, such as pulsar, kenisis, google pub/sub etc. However, this RFC focuses on the debezium source in deltastreamer that will assume Kafka as the source for the change log events. + +![](arch.png) + +## Implementation + +As shown in the figure above, in order to ingest the rows from the database into hudi and maintain the change operations done on database in hudi, we need to perform 2 steps: (I) We have to bootstrap the initial data from the database and (ii) incrementally consume the change logs to insert or update the records into hudi + + + +To bootstrap the initial rows from the database, we can either do a full fetch directly from the database using JDBC, or alike and then incrementally pull the change logs from the appropriate checkpoint. The other option is to let the debezium connector perform an initial _consistent snapshot_ of the database up to a specific checkpoint. Subsequently, the debezium publishes change logs over the initial snapshot, that can be read incrementally by the deltastreamer. + + + +To incrementally ingest the changelogs from the debezium connector, we propose to implement a few classes.`DebeziumAvroSource.java` implements the source class that reads the kafka change log events. We reuse `KafkaOffsetGen.java` that helps reading events from Kafka incrementally. The `DebeziumAvroSource.java` pulls the latest schema from the schema registry, applies the schema to a batch of incoming avro records of the change logs, and transforms the records to extract the actual fields of the rows in the database. In case of insert or update records (identified by the `op` field), the field values are picked from the `after` field in the incoming debezium record. In case of delete records (identified by the `op` field), the values are picked from the `before` field since `after` field is null. In addition, we also add the meta fields from both database and debezium. Meta fields such as `LSN` for Postgres DB help us identify the order of the events. + +Since we change the schema of the incoming record in the source class, we have to provide a schema for the target record. We could implement `DebeziumAvroSource.java` as a `RowSource` and allow spark to infer the schema of the transformed record. The other more reliable option is to implement a `DebeziumSchemaRegistryProvider.java` class that extends the current `SchemaRegistryProvider.java,` and implements the method `getTargetSchema` . It constructs the target schema from the original schema by including only the fields nested within the `after` field of the original record, along with the meta fields that were actually ingested. + + + +To ensure proper de-dup, merging, and hard deletes of the records, we implement a custom AvroPayload class for debeizum: `DebeziumAvroPayload.java.` During writes, we check if the `op` field of the record is `d` , we return an empty payload to ensure the record is deleted in storage. In the case of `preCombine` or `combineAndGetUpdateValue` (merge handling of records), we return the existing stored record if the `LSN` (in case of PostgresSQL) of the existing record is higher than the newly inserted record. Else, the new record is written. In the case of MySQL DB, we will either use the `GTID` field or a combination of `bin file id` and `bin pos` to identify the relative ordering of the events when they actually happened in the database. + +###Handling merges with Postgres Toast Columns + +[TOAST](https://www.postgresql.org/docs/current/storage-toast.html) (The Oversized-Attribute Storage Technique) is a mechanism in Postgres which stores large column values in multiple physical rows, circumventing the page size limit of 8 KB. + + + +Typically, TOAST storage is transparent to the user. There’s an exception, though: if a table row has changed, any _unchanged_ values that were stored using the TOAST mechanism are not included in the message that Debezium receives from the database, unless they are part of the table’s [replica identity](https://debezium.io/documentation/reference/0.10/connectors/postgresql.html#replica-identity). Consequently, such unchanged TOAST column value will not be contained in Debezium data change events sent to Kafka. Instead of the actual value, Debezium uses a placeholder `__debezium_unavailable_value` for representing toast columns that have not changed. + + + +During merging, we check for toast columns in the insert records, and if present, we update their value using the values from the current record on disk. + + + + +### Deltastreamer configuration + +To run the deltastreamer, we need to configure the following: + +1. The `source ordering field` should be set to `_source_lsn.` +2. Configure the schema registry server that is used by the Debezium connector. +3. Record Key(s) should be the primary key(s) of the database and can be obtained from the schema registry, since debezium uses the primary key(s) as the key for the kafka topic. +4. Configure the deltastreamer to use the DebeziumSource and DebeziumAvroPayload classes for the source and payload classes respectively. + +### Current Limitations + +With the current constraints within Hudi, we discuss a few limitations of the current implementation for CDC. Consider a case where we have the following change log events for a single row/ record in the following order from kafka: Insert (LSN=1), Delete (LSN=3), Updated (LSN=2). If all these events are ingested in the same batch, then dedup will only pick the second event, since it has the highest LSN. However, if the second and third event are ingested in different batches, then the second event would have deleted the hudi record from the disk. When we receive the third event, we would insert it since the record has been previously deleted. This limitation holds for both CoW and MoR tables. To resolve this limitation, we have to keep the ordering value of each record even after deletion, to ensure we can apply an insert only if the insert event has a ordering value higher than the delete event. This limitation only happens in the case of an out of ordered delete event. Out-of-ordered insert and update events should be applied correctly. + + +## Rollout/Adoption Plan + +This is a new feature specific to Debezium CDC use case, and should not impact existing jobs or tables. + +## Test Plan + +We plan to test the Debezium source by setting up a AWS RDS instance of PostgresSQL, debezium connector using strimzi operator on k8s and a AWS MSK kafka cluster. We will test for correctness by performing SQL based DDL operations, such as insert, update and deletions on multiple records/ rows in the Postgres DB, and query the hudi table to validate that the operations took effect on the records in the hudi table. \ No newline at end of file From 86ec2004dbca462c2fe2cc7112d910d27f049d9e Mon Sep 17 00:00:00 2001 From: Rajesh Mahindra Date: Fri, 19 Nov 2021 13:56:11 -0800 Subject: [PATCH 3/6] Add RFC for debezium source --- rfc/rfc-39/rfc-39.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rfc/rfc-39/rfc-39.md b/rfc/rfc-39/rfc-39.md index 2762fd40152a2..0007609f02abb 100644 --- a/rfc/rfc-39/rfc-39.md +++ b/rfc/rfc-39/rfc-39.md @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. --> -# RFC-[number]: [Title] +# RFC-[39]: Deltastreamer avro-based source for Debezium CDC From 7f8f01a311c63733f39f7e7a579b0f6e1955d85b Mon Sep 17 00:00:00 2001 From: Rajesh Mahindra Date: Fri, 19 Nov 2021 20:56:09 -0800 Subject: [PATCH 4/6] Add RFC for debezium source --- rfc/rfc-39/rfc-39.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/rfc/rfc-39/rfc-39.md b/rfc/rfc-39/rfc-39.md index 0007609f02abb..2b1467e99c3b1 100644 --- a/rfc/rfc-39/rfc-39.md +++ b/rfc/rfc-39/rfc-39.md @@ -64,8 +64,7 @@ To bootstrap the initial rows from the database, we can either do a full fetch d To incrementally ingest the changelogs from the debezium connector, we propose to implement a few classes.`DebeziumAvroSource.java` implements the source class that reads the kafka change log events. We reuse `KafkaOffsetGen.java` that helps reading events from Kafka incrementally. The `DebeziumAvroSource.java` pulls the latest schema from the schema registry, applies the schema to a batch of incoming avro records of the change logs, and transforms the records to extract the actual fields of the rows in the database. In case of insert or update records (identified by the `op` field), the field values are picked from the `after` field in the incoming debezium record. In case of delete records (identified by the `op` field), the values are picked from the `before` field since `after` field is null. In addition, we also add the meta fields from both database and debezium. Meta fields such as `LSN` for Postgres DB help us identify the order of the events. -Since we change the schema of the incoming record in the source class, we have to provide a schema for the target record. We could implement `DebeziumAvroSource.java` as a `RowSource` and allow spark to infer the schema of the transformed record. The other more reliable option is to implement a `DebeziumSchemaRegistryProvider.java` class that extends the current `SchemaRegistryProvider.java,` and implements the method `getTargetSchema` . It constructs the target schema from the original schema by including only the fields nested within the `after` field of the original record, along with the meta fields that were actually ingested. - +Since we change the schema of the incoming record in the source class, we have to provide a schema for the target record. We propose to implement DebeziumAvroSource.java as a RowSource and allow spark to infer the schema of the transformed record. An alternative approach is to implement a DebeziumSchemaRegistryProvider.java class that extends the current SchemaRegistryProvider.java, and implements the method getTargetSchema . It constructs the target schema from the original schema by including only the fields nested within the after field of the original record, along with the meta fields that were actually ingested. To ensure proper de-dup, merging, and hard deletes of the records, we implement a custom AvroPayload class for debeizum: `DebeziumAvroPayload.java.` During writes, we check if the `op` field of the record is `d` , we return an empty payload to ensure the record is deleted in storage. In the case of `preCombine` or `combineAndGetUpdateValue` (merge handling of records), we return the existing stored record if the `LSN` (in case of PostgresSQL) of the existing record is higher than the newly inserted record. Else, the new record is written. In the case of MySQL DB, we will either use the `GTID` field or a combination of `bin file id` and `bin pos` to identify the relative ordering of the events when they actually happened in the database. From a7c182dfee0fbdd9563ab238db7694f994373d8e Mon Sep 17 00:00:00 2001 From: Rajesh Mahindra Date: Wed, 24 Nov 2021 13:28:59 -0800 Subject: [PATCH 5/6] fix hyperlink issue and rebase --- rfc/README.md | 2 +- rfc/rfc-39/rfc-39.md | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/rfc/README.md b/rfc/README.md index 6fe6827dfd91f..493e07595ea29 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -62,5 +62,5 @@ The list of all RFCs can be found here. | 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server) | `UNDER REVIEW` | | 37 | [Hudi metadata based bloom index] | `UNDER REVIEW` | | 38 | [Spark Datasource V2 Integration] | `UNDER REVIEW` | -| 39 | [Incremental source for Debezium] | `UNDER REVIEW` | +| 39 | [Incremental source for Debezium](./rfc-39/rfc-39.md) | `UNDER REVIEW` | | 40 | [Hudi Connector for Trino] | `UNDER REVIEW` | diff --git a/rfc/rfc-39/rfc-39.md b/rfc/rfc-39/rfc-39.md index 2b1467e99c3b1..9dc6335b2b9a3 100644 --- a/rfc/rfc-39/rfc-39.md +++ b/rfc/rfc-39/rfc-39.md @@ -29,8 +29,6 @@ JIRA: [https://issues.apache.org/jira/browse/HUDI-1290](https://issues.apache.org/jira/browse/HUDI-1290) -> Please keep the status updated in `rfc/README.md`. - ## Abstract We intend to implement a source for ingesting Debezium Change Data Capture (CDC) logs into Deltastreamer/ Hudi. With this capability, we can continuously capture row-level changes that insert, update and delete records that were committed to a database. While debezium support multiple databases, we will focus on postgres for the RFC. At the end, we will explain how it can be extended to support Mysql. From cd8a19cb73a783fb83da08c90bf7cca8402ce6b9 Mon Sep 17 00:00:00 2001 From: Rajesh Mahindra Date: Wed, 24 Nov 2021 14:18:01 -0800 Subject: [PATCH 6/6] Update progress --- rfc/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rfc/README.md b/rfc/README.md index 493e07595ea29..c61dc24d5e2ed 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -62,5 +62,5 @@ The list of all RFCs can be found here. | 36 | [HUDI Metastore Server](https://cwiki.apache.org/confluence/display/HUDI/%5BWIP%5D+RFC-36%3A+HUDI+Metastore+Server) | `UNDER REVIEW` | | 37 | [Hudi metadata based bloom index] | `UNDER REVIEW` | | 38 | [Spark Datasource V2 Integration] | `UNDER REVIEW` | -| 39 | [Incremental source for Debezium](./rfc-39/rfc-39.md) | `UNDER REVIEW` | +| 39 | [Incremental source for Debezium](./rfc-39/rfc-39.md) | `IN PROGRESS` | | 40 | [Hudi Connector for Trino] | `UNDER REVIEW` |