From 76b6ad649135ec0ed425c3c173d91f1e0ce67c5d Mon Sep 17 00:00:00 2001 From: leesf <490081539@qq.com> Date: Mon, 21 Feb 2022 20:14:07 +0800 Subject: [PATCH] [HUDI-2732][RFC-38] Spark Datasource V2 Integration (#3964) --- rfc/README.md | 2 +- rfc/rfc-38/1.png | Bin 0 -> 207639 bytes rfc/rfc-38/rfc-38.md | 283 +++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 284 insertions(+), 1 deletion(-) create mode 100644 rfc/rfc-38/1.png create mode 100644 rfc/rfc-38/rfc-38.md diff --git a/rfc/README.md b/rfc/README.md index ca9e8782a..a9587d1d7 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -61,7 +61,7 @@ The list of all RFCs can be found here. | 35 | [Make Flink MOR table writing streaming friendly](https://cwiki.apache.org/confluence/display/HUDI/RFC-35%3A+Make+Flink+MOR+table+writing+streaming+friendly) | `UNDER REVIEW` | | 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](rfc-37/rfc-37.md) | `IN PROGRESS` | -| 38 | [Spark Datasource V2 Integration] | `UNDER REVIEW` | +| 38 | [Spark Datasource V2 Integration](./rfc-38/rfc-38.md) | `IN PROGRESS` | | 39 | [Incremental source for Debezium](./rfc-39/rfc-39.md) | `IN PROGRESS` | | 40 | [Hudi Connector for Trino](./rfc-40/rfc-40.md) | `IN PROGRESS` | | 41 | [Hudi Snowflake Integration] | `UNDER REVIEW` | diff --git a/rfc/rfc-38/1.png b/rfc/rfc-38/1.png new file mode 100644 index 0000000000000000000000000000000000000000..44238888c9b81ddaba34a5a7c9a6cac1112f9e4b GIT binary patch literal 207639 zcmeFZX*ktz#}~2x3ZATUApB#F%IeA?A4u zI-sNkG0&wDNf1Pk5MI6S=L~(GUf28Sz1}b9!%qI$|2?d|*4}IV*4ld~+QeA*80TqD z78aIcdUtM`valT7U}0fvKYVCEr(IU8Xa9@U&s6t1OG%F~asS6nr+a$NMn)`B`_G42 zSfkun4*n>zpHA;57M259tSkrizgd6eWwHJF_TWa=fj^(w+J6*0J(;(_!lKQhcT>ka zh;?!DNaF8SVT5g{SG9j_18~yjxY1PY$D))_{?J#w*T|-$n?kn0M&1L@8Ah-7S{jsafj{T zM*iXdQ|m`d8Bh1$8Gnqs{rcpuYMv?675P=Z_RR0RXUYOkc`3e({RKUbSt2w@zszTo zc*hcfe)}l`BW(GocOOepzV;L`m6lMt$G)=9Vov~ zxbh1AU&ieB$QCm2%lw1J*Z1E^e7l|f<(G^*F_O*wYgwG)JTannMCtUc+rJQnc9x1< zgiVdubhE5vKq5|7tUBM ze93dPZk&p%Y4@jn=x8_sGwmpQpUfRkEc*@Ay)Xw)@YFQ!Kl>9C(l3_rCL2VkCw+EY zlg_M4tPEPI^Gli2EGj=l?uH?8awmOSKd=oPAhFUMcBUhrH@h+pSgOwFQfxu?=wNz> z>^5w*sjbuzl}K+))m(Z0lSm(yXnVf9$caGKU9}rca=GFaJM->b=4v9r$_+lnALP|( z(#}Lx6>bix9``lS24LgFo>R7OJ`^Wf@3!x(Os9y^ydA>_TH@%LV>It;BG4{vJ4zp> zqha8VoET*wN8@<%-Bw?(aoqnJlBCJzkyuARO*f*soc+u1--Q35s!Kl!K5f;$gSv36 z-tChVbM0yVbSU081ElG5xJySnQLEPQ`k+Nl#kV4>k6;O}MK=^*z z&Bt^TZ-cyyVN}6t#D3Bu`T8~7g{9;0p|_U^`|1C!iK=EG0Y8(m9ex$ef}*pe9Jh9R z9?ZBGFBSQ&Iy9@l=;x)|>U=~I*BNbW>w5;f$?r=DL|_#|Ro|xL={IllbWu~+c#$&UI49y;FT-#P1Eg;!* zPFVmyz72gvts~**93keHB7^$|4dKZ0K1TYL`C~s#_1~ zF_lL9vt`TE4|Y)Vh@Q{BXDD`G^<_mlLxt`j4oGD?m-q1~Bx|8UQ@Quz=y&(7&FeD9 zQ>OLo_V`hbz~JD-=V6biQutOlx3}iC`Resz@%{rj>K{wzZ<7<&vTppe9!o!&4` z713RqOW3=8@OE*aiU`eHoRPlA^v-W{MECj>7g~vvs-4}R!erZMX}MW&@gM`%rnXP_ zRquE`r0K=|JcZ4(EUOCnL%@ipE%nUv<(DuqE#1-`T$edL@bcZm`V;O2$Pe+(s=?+rG`m&aLxNA?zonhpPSWz|8Eiae*9F8y zkKS(h=us6BkN0K)=&k;9nJv<=tZKDm($4S8!_sXQjTZ9r9G`H5CO<+4Y&JOL#QITZ z5QKOCOG?13ql>vbi6zGONqsA2bN=+4^OM380AcfV_BgqD@cT+AG?W=ec~KuiiGAh_ zk@V`;QI{xYgV6m>!>Iax!h4qf!(OYs>DX1tLV%EZ`K1PF$8uPmr>Uv@mRcC@(<8O$L$t5Ii^m4V5&tjU30UrI^sm>+*L?NtbJfE;ye4 z$p)^JaVq+;V3$sz{9wETWVCh-&XX+qtzq-kd;$H9ZS_dV0@Xg2KJFFm=#p_V?)Hs2 zD=k*PEUc0kWjaab90OqQ-8B~Rfo4kJ7bIhcy}6U80nTNs5BEM~FuHFs$r1`9L8NI1 zW9GbSJ$bq(wNp80QTuDPv*S1(fYUM2Rnc<2*HL8iN%!V(LN0i{_d%RBwJuMP_>&=v zh|+N=uQ$|u)=7|xk&xru90ZG<6TFh*H-h zvuRp}RVf`TRthVwf+rg^O@%V^DtE%KM8WxI%K2j1E+73ZB-vB}&~Z+TRciOPhz#I!Uy(T~CLW}@rmeSrSV1-I_n z6o~^rQ}O9y*~kMqIfd%$y{R#jYFLlw%Xk}^^{RGibb{&C^^sQRY$1j5z|ZS7o-bLX z@DHRl2ziG5*q6S`OopAi;BDdJX}M{gj>?_BiOTV;9_BoT9y>;XQL^TA_jQNTf-j@{ zhkE}{q5r7!Ui^Bx96`ao#lAT*UTohSOlDz-O7=f&LQ5!z$bW2r4h-kroPo_}hSRw( z+sUlYBY_QUR+wedZv%apsQ2(2c3tO1j)Lz1)r%k(RDS|B4pjE8R=Ba{+Id|~pH*;e z^C?Mg7B$)Wf z668wmS1Ez1b?i*MK`;7_Qq39vi__ANCam>TAwDcvIYjLiUcx?~FOpITD=7n40bn!y z;YP}kaJSyR);8l$Dsr3K0V>`3%x1yk%J+X5(F3x1l!rg8XqbZNqmMr<=OJ6e=}}+J z?2T<_^_izX%;vGWl;+AGwsW&PsP)Ky%`H1mMbll(&>N^`6d zu|I5dYukFJ%nylpw?`V86t2;|-i?1IX_oKaNMs9bjw^y6uMXPMKdUY#%+&9wTHD|a zca|0TNg0%lvURVZpYdjRFc>z0nk}eyy5q^Wst6yG(%z(M)yh6S6aTBY>*8silR#|3 z34iJ4(AjKEfQGjh+d^LfhM+z8!byy3`)N}^;Bsg`&zmQB#EJIT3Ll_f(xW#=_R(YKwcpxq{bViwN0qexK=K?v$@0&t|34u4)`kf5xtB?W z7OoFzHY%^>e@WMw%U%)S10i#=Ec9O@_s6X(`-6I}B+mYm82`kv>9X1m-Z&*z_FuXo z4ng}M{ZsmDwx58_{}hcd5;#*kr4o9|^uIM$4xhMq&OP|pfY`0JS z9PNDIPj&v3d;7WlkzXqMWaC5I<(f zs!UqOFR9daFy9}4cGqj}KiT?E8AllHdziJ+TbEA%^6lLp)k$9@+oT+5Hz0mtXBql!%m+BY*bwkE$#Y#pm~1 z_V7He|4XV{wb{srHeOz|bT%ad0zyAM!!vX*L|NqAS{u#E} z59RzD|NH+e{+FfV{&B9K@7B{@LdgojMh2Rb@E_IOI>gg^AN3pFBq<>h&vO+`2%#TXw(XMq+mmu?x$Nf&91*F=ydh4 zExuY%L}DNVDFaCZ7Bc;sV`2lj_f!)hE-BP{%-z9MOx|9-OTl6J#=>W z4WG-RrdLr!$I0yXxtm^TEhf-4W%L6dmwWLW1RF%xyxXm=)w)a18NK(MAZWp|(47Kk zA!V`lANBr-uLv%M^iFG7xhtA z^R;Db9*|7hB#%YZj;Th}uIqlEVv`ZNz$8grO?AUD8}M!}Xw;!e8}7wHQ;k$g4WD~K zTh5b@aqs#3s`H)%QROYP_}_LL`yxrf$Wl7`Zc_d@AJkZOzYbA_NIv^G(*s@yfGxeP$|#sXz7GNzOjeBS>g89Qj77i z;gcgmJu&rv$Cf8P>PytgXJ3#FoD#9$MeOmK=_cHk4>SHM?P?VmqrvXce|!N|ZLZs5 zrsnR6_bKLGBxCEQqGci)WklCu+Y=@piaJ4LtJKYyh#ZS7H4$ipe>RLp~1OI<6chmKzq? zRw4+3{c*@76Ntdi?^tW@77k);weq1JT(R5RwSM0+QgGfmTmkmNI3tdY21a&7f|IH$ zG2syLUNSIs+l-=97#K9_eqsc8=%emmZcs!u`_6d9L;U-*nd>zhA}?Zk)YUdZI|6S% z96ml+QX=OgUBvr`^~)+=_s%!Q@dB1wXYJHi?{qsRmc;H9EqbIn&C{)xE5bi%y+Kbn z%@tyw**qJdxYda1xv*$2w)ew|}?Oo7Ipr3*i_?DS~p``LXKa8e*4LQ$T zH~#Y?p(Hw_In@4ZPmgXg&DG)P$tK)QJW0nc(-Xc195z%z7P~?gtg7-wW@v z&W>KQuEKDrwoWrg_GF3`Mgw#KX$=B+j71HqO+kr%G`=Go+rgTIRVE`$zgjsMt@ZE{ z7KWb#Fo6tB*}d>fdre!n$O@M?DAR@-H3G=C>tU85nI+3CSz1yH_T9Ay?=88r?Vvko zOs`tIOOm#&@F&XM(@99609iDYd)_S#xLOBKf^E@{bKzyBj+tBZG)e`MX_LW_rIN0E ztf5yoNmdTYsJIl~Gq+8%4i_6~>)VnuExMx393R;%MK8I2<%_DeO2vtt;vd*tz61M4 zn!M`0G*;VnsNnX=5wiCu-oNLN#X;c=53=25Nqr>$rDDBFD{uYdD#`|%BW5w^a|7;x zOmBnI#vI~Nb1fB&eW_chm#0PeJ^W-udbMipkiN@knp<@<#kgTA3nh58FmmO_MNKILgY|b z@WW>$TOct6cn3`TV2z4!Bats0#Jj{9%vljP^8glhF<+Tr;7obwP6&yMM)}Uz?N*8> zCcLZUBoeRet$+Hm`Ets>Z~enKtjvw;55qpNV@-QT zcf}HQ-5Z-q;p296|DhHm0q7G|?Y9)cxg$be7P*FN$7fzMJa&CIIE!%QJ8^VmKsxHo z6nC?3>UN33V1OR%yHu8Gn4!#$Igx~N>}JKzW{HmAD3okP$Aw9S@;;S5$M%7-siNC89dt6zUn!Q z#E03>jt_n@=SXt$Zn*@H@{Ec}ZY8rSPDHrIuImXH|Z7c~1azIBT8)a}0)PmyF+kX-|=iBdiEFn8$EvD$on z*3@>AmNqlJYKLjF>kRT^Jz!Y0)&=376mvM~(erW50De+9o~y)ys39BZG%?KAF0xd>$_e8#pi6;QZpihGGxKAkrIewtWlQof2sW5R zVQ$xdU#<+LNGCUhFGF_RmyI)RK(mN51yH9j65$njylh&lcH!70$4G~`{S0C+D+xB& zN$kAq*6No;WpML*fcLlV`g=Sf+`C*iX*QbC*uS?H6CSaGxUF~R2jsfF%wUOIrlvgzYbnv{nnvH3%y}MxR-`k3b89`e5o$Wec-`T4dGKPw zMblnmq;|ulieij^`tIUU)N&p2H(9s{zF__2y$i3TQag%^E)`@Z24#Tal#p&(E zVjSrmA?w$_0hr_8DfR{}hz9B{L&F6W2Q$FkRjVpUu!QX1aP?Urn}Lo7)TYUNnp)If zEHnZ7SBz3+7F{ zRx8+_kaSdEOOl_F4M_c|v3c}ZBnr@cuy0dIs0Krm8?L*LQ`i$XJY)63y*hNXu+iUX z>p>Zy`aFnhIh8Mu`luoEu}9zGq+YIji^+TrK08?lEE59vn=nn# zVAgggL$a>9`2%cZKFov>nRH;2n6fa=TF1BQAQFN8kdeam*QV_mF2%_jeJv%mhG}cZ z3uE0X)n2KC%5ebLon*i(L#3wyr#qb2fmfL;1CRQ?N*v0j!IrNo*f2t|X}LV>-`4Ka zKS5*y1AbGehOE)6u{OMy3nYUGt1+1!NY`?ABXQ7nah9D1nji3l-=*ZxA!U2)d3$&w z7Mb#xqz{=3wmOY&O6b?~&mJ-En(5cvYOI>v-lDwAvJw~7e^3={_CjeAm7VfxqeBvZS zNL^0WK=Gq5?qvuQaIjUcX{W-;+*;sN-teu-*{!v?{t$pcf%{OdTO_x|vN%%AU*&5p zjhzv?7do3E%Y~(N7rXaAs@GS^vZd6w%w3cocUe8)k|aAW6ttYAP_dJA;Zy7f=x@bI zRQMWAj$$vB_PsB}L0`NpIhZamc*gnQhUL)J_J3%ZvnC66!Gmpy?vwAi^i{!Vb$u-v z+xRl={tRt5{Bp3^D67|VcGKHeQ2s;S-tO_x9{{!)}!>L{V&*bdyZ1YU_RjSwI zwp_fl{G0HA^rljb8j6))&%vAJcW0fyc6;xgx(FSSHYK#Elkp+BNiY=ctJyKti*I|f zanb(q%QD>^y^^x8jD5{-*hZloVzA!*S<@&Vcu7>^JfR|E*0pFi`otp>IgLx=)sh?0 z{`5|oT$rgnwDJ!0?M1}!Az%U2MAZUl;e;X=?ngL8#`nYJ#%+RK5^JShX0iVEt@!1@;RvzXN;gbiDVC)A2_xY1W@KG@1G zoXq1fJup?eR+6D&-AK7QCz&2G)n&$zdYsRDBTSPODkkU-!C*SpIL6vNx}QSp)kHhL z{ay+!Mmd!MISN6K7)j>CVs)mE0FC05*NdjRdS{O%w!E9@Ds@+}jn~-CY@J#a4gLEB z$UQ)7iz?N+#5i1lqLSmo{qd3!Mv#s~Nhp7t0YESXM6HU0zJoq7?87gf1D6Ve<%h)6aM*~#;ge_+6MXA7oHiTx*YB*Jq1e#YV*!C+Z3F7k?dTeerulU_y z=mIX7?;CQ6F)+HxR&1Ptd$nmGVv(MViXN>(8)j?h<4J_Wp=s5hUj})*Hszom%XPp6bo=UU?_TBxT`OI>i*o^cssuXWX#n!%Aj3OLni5lYC2}Y0Z7$7z z(k+`jeYhAp7O;!EDL(IxEVyh#QO@IUJB`b{^NgH_gdj<;NkbW^>$>C4@BPdzZXbF! zZ#KQ@H=teN!(7qu5Yf&U#Lc0x)ZHGMd%C9LRkH1@U-89Ztjrt`RVX84U)g5p7ovIr zkWcjc1n!i>_x0fsC4P8zLXQ==D+V0rZ;uI(0p=@JcUL7JqeL>%B-B^C_9w}u4B((! zcawYV_ZHIkwe4y~4!Av;Os*&yEUw)-q7qQGymVWF=IO|xb?|Ex?|7pu#1=#wxgbhJ zRtTUvj;R#wmkaeiE`}q?=c+0!p+A@+of@g8OYp`jkt$ICp&oG<4WrRr@jCS8U&`vg zVKRrpBj0)MBMzw9nj5gHhkcRQku0+~#A>^d`!1IsBC6$12>AnyyCVF@a(ByDd3JWi zlqc+io%NC7jF3B=!a1$}vV{hOR?LH8g~f}JjZ+YeJaA@Ghl4#@cx&B+^ZEA^f1Iw~ zO0Ibud0RB-rvH?)od|oEX=}gEclc7x0}+hdRK|H7!9;^kF*@hZuLW3w9yxTDR`gu- zjd)%=R*Yfc?oSV-!nHVf z{6r-jwKSTpv)B4m6j{EXkAikyD{dJY3=4#WC}*}blG*t&=EwKcBG6IO8R!2Ih550K z5PQZ%Q*Bh|@Ofj2X$@qR=Mg3?&s@IQ+x~SuQ<&5hGYiM_mXmrlH0x3yXi=yioV$A`Yj{x zO$&q<*+K`>aOXEez1T?-VM6Dv^lC5o5iGNc z>=J}(179o z3W+hNG+)EfU@UF-8^w6>iki*0D|J?wh1yOg~>11j*oh^+c4UcIF#(k2fX%2|pvqmH@%|C(zXEmk?T| zeJW{hVA-QxEV2EBCX`#uyU)B$ojA1^1K&rvN57h5^gkRtu6Frtq=2lIN`}pbMWt7+ zUA7tfEA#oO?e5&iRyM~u6sx(3r_7RUHP7F*>nN_eCZ2dJLY_kj7bZ{*zsgSOVzEiJ zvexPfp4E0?6e)pNY;1&Ui!I{#$;pu}Y3J)LA{=t5xYi}2fsTm7T5e_dmsjM@2c}lH z?Z?Pt^E+mF^-ome=FL_7(17tSL}7*Uo~;+hyUYm~GdKn!gltWI65PDg)bxiHk8yvA z8N#X)Mk-0_4@Mzu;0@kBbmdn>u&iSxoAubeG!GB(ua7S>51Kjvc>K&ixy%!DL+Wy~ zqQ=J)H;1Kx;xy-~4>NDr&Js_3ZM)?6xb*`s`$67BpQ0R-n8x@62V@_gyHF4ZKO|yO zKo$*Kw*6jpsvs8@dCBeRl;W)q*$r%*%f2gLH_N%u*VJlyA5u&T21t_ZcZ3g0Vp8Rt z{8>Rmw#&Y4O*L@r(#2{~ArcBN6HLPx<`o9y?x5ZtI$eM#IC%&u4Ul?dT7&&)Z_v91 zIea%85;AYxb_c-@c;eHPwK!5S4T%RDo&mqrmmot^_qvod(AUE}XVNp2SAmH;pzWG?TGT0bG$P9Ck=B}+ibAHQKm%2;5p z$7cdt_&J~EVde|9^ICun=y%I!^Fx*j;)lR;x74aYM@Jj*xxfjp!F?Cja`H;z-%h}P zwA@Tkg#Y{EyWGmkD`i0?pyw9`hTMGq;4Vr0aG66pLpJKNqPn@2>u?f%z37$m9Tf|vtryIZ+VEVq zWl3u5+uHX1bNH9}0|6$4)$lW)O}Q=us7+H239&t7OWbp#;*F+b=>F3L&!y^1a$z!B zNXXSmnaQ0p0gjnuQdVZyt5~vbiGrnXYgKmCJ%8pa0Vk4nxoE^|S>ndPoEitMYn}GM z&J9Jhc%ejl__9fTcyAP9W~;6PJL($>aUx`9X8N{(Uzo`3a=P$fHOm-vRWp zg-hGsKxMnpJ=(YNyS6K5mj}a_UOa1(98j3}-1l!R{@3X^`>c4_7$cCi`W*Zzv=d^G&+roWLhWzS#{Q z;9x-~6&|)un_mK5d~8*7jdnATo76d!UHHWm5?!K+WFNQ27~&tbVyU?qS7%Nj+JoK( z?K~hnwn$ZHuTadW)w51WdUiXM{;8W&57%qV!ElXlW3uRJb@I)F?L4KNzFo(Goi$B_7HIi3DSoBuJZ_s;eX9SMNw zFKb6WpOBKgb*al5=T>_g>7aW$iK1tD=xWB0#JYH(YIz{3g56AT0tmO_i;>iAe7KBi z{&YD;rjl3DS_)}^$M)cKo?aP!@KNTJ$;HzOQYrc%_6z7E6Z%)G?G1D!Y8|CbV=i*g z`q5*-LoWAy&})p$hhfzO(-kjCbL*izFV2ek_#|M}(OJm7aW!l)WWu-fC}*Bc;`2;| z+Hm?*!&`Sing$2vy*KE`T?aQ8 z>yozq#N z+IMkW(Z5mkY#*X~QZ5nE5i(NhYzAOoq@#&G%l+PfxWI-9TF4c?mX;U~4^L;C8&wqokuCI`z+S80fi&RO8Fas8^(i{ON)&-3 zKzv)2S_a!`vY#M{5Q0h+rF32;R5nLBmAyf}faNLWJ<`EcRcCnlPUDjun$=ZS0gO$iLEVBL(8$&2 zN}96e@{`xl5Ui|Jxa(c@e)4XM%DYCEp7uXHRGkR}h>ZB2P>H4ColL)&Gxmq_8^hdm zWaE~DEH@<^Wjh8`8;YJqUK`xR;nxDsLFE`>0BY#+uyE_WT%Zi%p^xqETV2!CNqz74 zR|TW0;oW)5;>tqho)1*{tF$(subQd@(Bqad7n4*{10CPpopOAXbS_rWovu2L^q1Ov z2A^t#ntwZFW89(SEEf9?b&u90cI<9-ls&Xym*VYu2Qk-{7B+pF_7(1QSRS8j8 z670b*3ew7smI#f6+|&PX{U6aWK~{u6UW~9z$eLOcTZc3O4y{F}60lks)UxgHdIJd} zJ!VN<&Vw*+{mO)GbpW`tXwP38Hc;Z%U>lrprcCQP(cdT6l-{V`On;C<{(|xCO-zvU; zS(iTwG@|Ahvp$QD^lJ7_6@O%*p8YNhm|9kt1#HQ>k$Q{1Y-z>al?c1fc+<(7p;uhu zFLo6)OyCFX-uS#z#P+h_5oe9z)d$VR4t8NuV}6`(7WyaNFFDl8wtAA*w31wOSf4#- z(^)osj52X}HwW%(BDOQb&v^*e=_^=x+uaVQhosu0sX-&uAp7K~kd{yI6I&wll(;Or}0%`-Y|_W8ZooN zSGlwj8B|wGC6tivy7%RAn&*oSu2-ZV$m0&>YrF10&-=US5cb2#yk=D1dq1Jp4HHYHi<}7Sh zbi97t_3os(hPSce^;;(BK$Fr-fMR@E-SsH$1vi^6eYddbn@~x2RPQ)>JX^Vl(SO%2 zZJy-;Iu|@(Ya|4~ zpyPBuJuLpp}WJ! z`cHFKY^lwJSsC%QWFIW9kLI25lP+Rs=gm89rjMWbczvuTcPO}L_}Q9Mq*IfwQ|U_! zb}4G06tqJTEbmJAHnj>F2}5EK8W{(|jNc;h^tGxc@6~YU;N=WgG0+R#0~lM<%jG4R z*%!NmHn$9oh`Ubi9|%1siY~7Z8IrDwAYo`1= ze#dYdkJ0+4SCudZeLC#Ie6c8*i&{zV@)+Y5C)8_u4RwI$FS;uf>^qZAoF>v|ma-gl zF@YZD-lpE;QC^7XaqlLL8c#%Qhx)g>eG`Rcwf)I_@-PP{tzkg`5nPG1m)_h%Ps`Qk z$KxGMG?!H~JMdU(nCe!Fmw^D)qw%W6iCS53D|(6fV%MY$4Yu)5Ady(6kDJkkEZ2b`%!lgPj*gv`^M2)|jJ99SkR_yRA&(3K?T}MG=uc?!7<4BXIfRxJ zyuFX>&#`Y|Fujx4yI9M;8cQ%7gR@JnpW{KCDGP5@Ig%#QyvuKyyDwy~#*4c)sab4Z zQ8o(;xL5+U8XC`afY1BCl4EBX#94ibc#|i~=jESv*NVlvq54L|Df?>^(e!7jDPCSN zo$R3S!YkV)%;p@kO%ecR)Ns}Ou0~mw$x6@kCtg>72=&+|%83?k&&!BL?~0+8PH@=W4=Ri_6xrqSEyIT{ZUGulDkQ<7rrv!)^@22BHRx<5F& z5Iu8LO{}H?+;FWTrmu&7@%+w-V;m#AGlM=~$HU5T8~zw|j?w;mPJJdZ4I4!^Fe&)D z*52CmIj^|7#ao+gG0^$Os)XSy5~!tIPJ$}(=-_a`FeYyKLWA1->~%9W4^Dlej*Pp` z*ud(2ooS)4UN%&&bWl>im5dP=J71&JW}+-lf;i7=9Cn(QvfGGlAAp?H+O3p(>4%*t zSQ;66_QfM`v^Y>U0Yj@GpBby@!Btj9Q9UG$5=JYkK7}VH7g@ls3s8}*Epq-=+g%$Q zbrg^7xn`lww&oBq^Dxkh4*QL*x|wLgEl*vw_yO+r&jFyWCSEI%fIi1eMC;T&NT={$ z@G%Ra%{`=27dQ+G%pKi{oSK`YFz-^{UQM7z<2RPZ9kFl?VkHlruD%Ef-lqR?Hcoi9 zo4C_OS0TF3F`S6tl^t1Nk0T_!Ft?jpa;pUF-#+jq!Dyy3$)*^jMKWn2 zY8Po7kx~;WZ@dUo1q@B;(Xa9K_VvatBLlUIYn*F3zI_@)NS+wz&!s-ISpW;Uzs|*+ z_jq}&rsgW&(~S54(54BME$x=Dg+tMlZ|f~~vorm$qe$j$K^1*@Jr2wSo~cLKyar9s z=+4}{vAjlDf5(_e*Cc%NfwW{$Z8Vqs_QD;7!A-mQOYHhzhH~u01i^UX` zF|Q}B85kyXu)4%g+mNfkBP2P~CP>m`z|?^^iG@ z^_3;mGyCAy=QyP5cQ&a@Z);L*3f(?!``R)g22i4Y%u7M`q6Fde1dxj{YFDz(%FE7Z zz{gaEm+am{9Ll=Ycv4r?wWC{Uf?vR{Mkz0#&>~G%D{#)zVug2xnp+?9Vc9$($@tbv zVbz*^p`b1|8Iz0ear5ixc7MN&Q!1S)zTn=z3-U&<+14A6t_ERpkG-xjNOFQ>=hjxc zKZT1RnxTS2Y5n(lubIv?C9{Qa265Kak=L+b*c2Z&Z1nu9 zj2v1$jbQj$KEkl0rn+Lf%I)CBwKzEYUwbgfyvG1xzgI4n-48VxeF^Q9y?uqT8bs=}1gdoG$Jrx$R2U*EQ@Jd^v3*d-#`c@SHlx0a@Nm%e`k zQ)(yF!|Q!y4dHOD#+EcE_%fZkKJmUx$w)SaVD_4LVx;Wx0spJD0cOU!U_f~SbnU~I z{az4YaU?_$PW>p+PDSjc1WnAH6RlR7_0NpimR?kbd70%k*sx#mmE@iILQ_gmEotG^ z(}T$*!xp6wZ7eaQmg`rV6Mc3UY@0S(S|qqbZB!OCoeHR-C7a_k-?H}d0^M7o1dQwp zZ^$xV`CKe9Ma5sShB9%It3ekCd?wQp$!IobeLazOIcZWlyN8-cP1=4kMX`o2`%D48 zhrQ7X5nHq}PQQXnCv?vFscud}#K>XPLg+V_2g|x+s}U``LAHDKLJxLFifkggZ;=%_ z!g+6+_Vu5KOD0*`TzCoYysh&I)pkEkF;9td+-%e$zNzPvENb|CX$y5_%d$H{1k_`r zmgkTX89bHZ6z59nC)O+%1}(MNi&>#*%R$SVV(|oKFJJJ9XOyXYet%U>BZg+E?~2CK z@8Y8N!6@5-H`otXC(oM5Y^!q4ba&w3A$GQ#(T%f5rg!R;nHeCToG@(|P4#FI|Ds!J z$rnyj$pg>2uVXPUHqo?Ct4W2+3DSGiwpCATh`#kSnmK4%g6jjPf+csd- z$!!Z&Sn^T8+P3R+fHol+Gv zRI4uA%p{%^(2~7QAe1}0Yt}58oVWq|UG@1Z(dKu9{*$zpTw@-yGhK7R9cmXV)}?r} zd`tjJN(R(zQ>=IMj{NE?+3ChrYG6kn_IAeJj%WGijwk0xcY9xBgFG_Ia3aiGNptsj zZJSK(1b-qf#Muy@_wfr_r2sq}hH-ay ztDB)T>|Py^l4;X6Rk+MiDtsbsFt=_BNixMA^M{BqoW9Z48gei|3sDL#<3g!-i$5gP zj<8c-OL3!CR`_2wmeZ>GSS&|&KL)e0ZqOuNSxm9Ok0Cwt_|uz7!IrGm*N+RNW~9Ar zj}`060-EIOtF#WgEx2_`RZ>O?Rv;NuH5+4>?Y^K77=)-CgkGx)gb_q76)G5s2Xbv* zGz^f8(>rxkI1<%f!uh#qDJV+3@+0V1(V=GB;ycXNl%9`G@MG_M3!J7iC|oJmN=Yd; z=se@%VUj0@um#j#t3Dd!SH7utvNXIq$!&Tv1s>OK%@bsbMVMt3Cx$xDPIcsdN~0XhCX;)F(Zw;nCz z=9RF~N5<&~T)3Tsr*=r)5`;`7UY)8AUk;tb^N4b2pnL4(IJT0Joo>UV#)mb2b{yQo zcOdoq!47Y(FH`DG8Fp3xKf14Af~ZCsTN%9L(5Md57d?78w)GKPgGGQk zUAB|$vZ~!8@|9&bBBGbdFRPVe1Y8{FTMZ|C=UnDAC?f(eO`o%H#(>|Pla+BtwwPbD z>@cmmf%~cC!uJJult>nNSmg^0RT3$U-+@?ixcYNkO_F{ z5yqeiu;%Z#d^&42Ev+`YKbJTXIzEVV%*EL6nKmsOQKHAE#eC-cnok14+s`+cn;zFS zY5gEmA9~lBAqyTG+aqola9D1Mdfd8E+Ef-U?7oIUQD*kd> zgEjvHs`pt`>vGSAvdsR?PQ&GhoU7x>t_T$<|hs;ucV5I=lPRdL&)s)np1smAx4 zIeOW(zLNSDwZW}WI#~~=<5dYg;6a*5#ygI4->j`a0n?n*^I3YP)UGkDe5w|T z?+N-?LRC5qn{r*<&ZA7C!dz?upGf zYiBCvy!C{Z?=+eeU)f>z^I)Nc3R*4)13SKpZE2pRCO{GXblos+Q+iY=#eR_~q`;9D z+CL*k-+jGo8Q!vkY02nUWm#A+4aG~(NrcZz^sh|sbWeX4n%-(x)IiwoOi`G2p__rw zK-CW`YiQxxL!nHvy%;5wpoWW%#wv?`dgoc-go|QmJD25 z=G2rVaIaYj*luZQyXi8UU2GSF}oj-w1In{H~YpY~E!f_H*XL{Y!l)zAq5NYPtp@2D3iO3X{VuZLbK*&Y^}ou1llTM#X5bClCiYu@9qLxnVTcj!(t`|4;7 z!0%du)qa6(flc3mb5971(L~p^Mk78 zLVE=0cBk?czl)+<6R(2^LQ?|Asz0YwdfGp`(dyQ;G_-n{A7UPao9Ya2^YY@*qa+S6 zw4;>F7##PG*O(p=qLF+0gS}#Kybc~%vW}cu62-Xsm6~pc2>W5OudAdg=;Ci<6Sd+~ zdua)umk`ZXVEvFQ>`NqETo5Cs^o6vYBz01uufHa(ulu}9joV&vST@SlP^1_CA#f=! zIwa3=xO?*^^%{GMcf-YiQ2G&hV94bfa!f_mTv*{%!Q*hK?&3f-(XZ`W%EUY-@_3th zx%kF{TgPzocKg-{xin^jIhM>FM6r2+lk;waZpq2T&psT)O0wsTzx6Q(L}eoWAF|#u ztjPxq`yQ>5f^=GRcZUe5q|#ji(%lROs5DcNkPhkY?if8_^eAZ<8_j?bFaPH`o_OE) z-E5>BWb3<`5BSp`mmI;0a+^@6U zNbEeS@jFWxkH+wJn94ofwlY?=X{n?VyBq1a9Hf^kKb-TDz8cbur@o2U>4(S$=$c@1 z`1S$|Ax68;U$=JJ2bx^f^2N$fXE8y1R#M{yf}6g9E#sZ9{kHV{P)h;TjR+kc^HZBO z7yCV&H)FSo*GTuBil!a9`b}VT1AL>7xbT7ln#YOOk^C)vyO7-9dVjp%w9z27F|>M( zo3AS$h?5aQ7v*E4&qzJ2g%}<}#7PK*wx1{Uc>SL;1kbApdimmDTtq~H%WXT2-SD9=hn>ArpVN_&Ul2Z+N+K>i69BUy7Et_3@l*WU5N}A{y*XaJ3XL8rmY( zzNfzT^+F|hQ)Mdr`Hfp7ME0WEz6HmdBvyjP8}8hg&yoPU{$#4@*);@{84J1fEv;6O zyPhl-2~FNuOHB5JF#7f#-k}Z$*yOH;ynd~XgfmWiNOs;`xZaE6bp|8nf_W;M)}w!q z^T;nr@Naun`j0vGD9AJ(c><=`e(}!L&zaFP->ea6rxr?Yad2e&4!ut9`}tDk)0Io} zR+7$a>tRP=wY2QVtEC;;Wb7CW`jAw!^;84z*|dK~`r(YeIpf1yB_w4S#gY{s)3_QO zXcBUJZZBKex#OvI7!kZ1@hRIAjaue_wxPz+#IAmdZ7AH+=vZ=kRt+E%T`c3w-kH^Uu?zC`ewiK&cQ+V3LSEfF8&s4eA{dydIo&9{KB(c z1L^!p#ywViCb?v4zHhlZuhrVsu63B5PnBcOuNdUdwo-|aU>`NkT@SbCR2O3FtE>LrLF-vAAGR{Jj*i<1pRs>JvG*W;!v5kXdc&-zJE5<7(Db z^P?_BG(-R|OZJ#o23N6rgu3IVxg}k%TSg=>eL_13AyhB!b?;i9@sFa=88%$(L^`@ejbv1kcC+ZjPBU5oJ|$faW|6&G(ClMd4XM@9 zeKuC2sNzMb7L18lW$rv;zV~DhKhX-hOXVAY+$A?8X7W8P5sXF-5Xjy{`5m5Tu^anu zbw=!Y93|h(Cv4QHnE#3wsYnyS6NfvSuShYQhpfdPTu9hkZPs;}Zw!L=v(S!sI)pO_&aSJ>kZX&!9SUadWTYTR3NrAM?9P1) zf5(<_rBWChNS9+0%QT6>=_^~`@Wxf!UW##dN#8KA;hB*5p9`KF_4?3g$+4~P_ zk`^!2TcT7pzi@b5u8u4LQm|gjd>s_DcG#K3Q25MFdS^0VzHjnM{IUz!(kH#ncL%3m z_t!V00*B1xS!3ddcGcLkORi879{LPPJ3Y5T?qecZQLXUC0rf`NY=HN*!>jPXN;w*p zm9y>3<1EZEyHE3NVaLdR%XP-byY(Y9^n8g@qt|07BP9B4wl|5>6aG*P)9RS`N`Pkm zW!bk<`^;jc*n`8u!1qUx0)_-=-hia~e$sSKrTam~&$`sdxbd^GZmh@uCX}>S+ z8q8Jq+G3}-{?0(QWC)^{_D9RxmZDM4u7So+l$-05`QW#$otRdyKNaB~BwD8aTZwy2 zKo_2ic;B{{<1%QIW%2u~y-y;VA$zLksbP*J*|$s3TN4MM@3_Q{4jb0bq^)dN;L2&{ zbnd}_B2+t=#6m)R2b5@B+swd(-U+$eabNM@$+waMY)1rNPFWJHGst7AnK9LVi?Gb3 zZSU;Ea#x_EL-~$;3QnTzl^g$LImJ@R*wzKk+ok5$nK<9g(CGF<;J@q@%%mD{#c9oHqG z{u*uTl$GxkdMbm@OF7v0YZe`QCcizET=xF3wclromJ@9;pW8~7?A8nap(8Wn2Hm+? zG_$GV(-nz3eQz2su#?bwX%AGUZ%pmIh@e)qh6#O*|~aN&KbM<8HbFGI?OukiG>wZLAbOy81BQ+WkDxH4@skReIbp2Yl_Lb`VdP z6&RoE`=sW}{TizRgf3z|t)rD{C)3BprvW9BbNMzNeaNyD}5Oi)mWNFjH#YCaKP>qetfpS;j;TJ48Fq ziUV^Ck{m@$XBM@xt6!T}1};brL#UC~3?D_lerJ5--HtA5Fbc>Y5e5zw&+4o5Z4CDO zoWCmyXuH&f#&LDt*7?EA+f;3&o-t!isMp){WosP?pj77xyjx4(~dAP>{M7an7l^i$~dtQ|zcE{nERW3?DG-F0ps#_yO^T8AhEuH5a2YTcOs%Je12Ml*}aeNyOb^8yo{ zGHJV9xweQBnlZYLazHk#N^#;vM>n9{8&yLGYJo#dsQMxc)L$c_Byoq^_SVSzTX3-O zlloT(hdXG{dduZ}T1T!R=TR#^z0|Why?h-w<1tW>7YP`4i4aGNR-1~-{^dsdrS#l9<2rg|QHzrPcdP$BwI7x7tx-ssTOnc` zfr3e?2D`7iO`k~W^xD6@9`QZrk{nyp$~xrw`_%<2Vf!5HnANrSwma)q(KsW-D_QZw zYMNc@;{BMWpEv)I0!t?YjR&C^LEX~TKROgc_HZ{BL1R#HA&yJO*+qRN(7DNbaXdo` zmS6Akb7$5k_*X@^Ey?`V=hciuI68+E$d0!de76%KdwzPG!a&ou7Dl+99K4r2)yu)q zh8a|m{l$Ck9guveAH1c%K5Oi%p1glI>v}u8?!nmR*_=^0f&%@;nEALMllj2jW)fO| zFuaAv@U0Hb@19!Yw=sMj;%2w|`h`^PqL_@;tBkrf^ZRpiP4vs*@{^^o$?|V)PRTY2lc3IH`p&1&;EP(=iS$^;$hK;#%(CY5y9Yz&ej$+5O|@cw+&9DP6q!E#p7B+^ zNBp~YZ$MFWVPpKZ($ZDT)(4RJ@VMKtSh)~arYvVCa3>l+(8<~w%1>k6J4lme5k{tf z`~DT47D@SdGv;D!FCC3RN@6Z1byc&eXoyW)F-s+j_1fi&^}fiE+c~2U#$;0;!$C>m zM>LBhv_H(ib8d+8M%LL(mNJW+6L+o&JBh0nZ7j{>!m%svmX8nUdXY||zWa;9`$(Dk zfLPUkmj6T^S80UYkhCcSa0xK=zJK>B2Y7ECyAiy@SAlUwRboD>z$cX!)f6p7FL#Ei_&tX6`x+Fx%kpHy?B*%RiV_P8wSX9k7) zsy=hfVUW9HS?6cNGhRILH|n*mli|PyRMWAp*uWbDKcTujybU)7*TP^Evd^m_h|fVA zdIu56Mfx<+iX3E+a7uB14rE|sBm~Xwvs{GX;fo)~AqO2Zz8wDsoB;Elv0P1op<8zi)^-IpOYb8Iz! zE)x?iF75xS_|nIFZew8XE+&OZNv3YR2?7)sY{;?|*%&2+#^RLXuZO+__vIiJt5H4m zeoD^$pKYXVTeRGjDE-}FQ(w9#4zNLW#Bel2tbXTK$-KuhojlKUV)E#KJ-tR8F?FTw zWjCRRO$wM%#{Ft$2eMG;!5k5=4i{UcZj|<8MxWpKt#mfi#6h38;9Ri$(L@mD^yFZ3 z<$Y34IrtD-P{J+(Y;K8lpcc)Qc+3@#+2fz-3U^fJtMFkJT&S!&)SrC+Sg`r1CT6{j zXGhsl^L59auWvuKoUgCR9d_G#ZjJN+uyn1Y2}`xy*mi}Zw^&-ig*I1vUf>6oSXA;V zr!D!l#$L?bw<;De@1w!ANzL0P+4ZL&-9cCEwFkn=8=e*dZv<){2jI8E1E+X&QkR1uVw4(%ro9>&*tgg9?bXQhMTM3X^!(d&rqE-_%%8 z<G z$`|}i&i-2KL-ucZj?hoo`M4h*r>l{Z-LVH(8Ju;bP+5HFnzN*~>bL5laehBmT<@1k z4dhm>4z^Jd#gF-t^DHTW!8)O`-(doemv8(pOG=a$L)C^=C8OPqd)UQhod_Fy;j;L` z7i`He?J(mo#V~XH4IzuyWUt{GEUGLY0zGTcvbTGhY*Oz3t?}4xxD_T#M)CHzBsK&8 z`M8=sn>vh0?p3Ami#y)Vo0H3N-wulZ>;vrMQ9eSgf~CE}ET+yQ!pB+$(ll4DLjmOC`)A&?C($*`cYV#i+s?K|6QJfZKO+ z{;18&?G)bnm3HL2G1KZ+zxeqs9a9Uo>pF(C>cc#zq0k<{&v za5n3D1DqZj6^*UCIL+RxklAlDCd`Q$^*E#n=QIh1SUC1iD0?JtpY6u4sA3ju13wgx zEHdwTKK^21X_|hjhKhkYammf#1$d;mVZ#@^cx-OjIPX6|EmJf>&O)jdWRnD~)qD}A zBFEC*U|JB5P1%(e0aGSBN}@e?oF-ppf@5C!%+!njBkZe!O3NN?+hoJ{+&~MeY)cyN ziDmbhe7pW@Vr%JSsk{igv%y_Iq6@y*$0zxuFN5aY;Mp&_+j;3MvL-Nl^G8}<-NT4) z@2Rx|*Jr_UWMh023``@OoSEu6mc({#ZVGMt)MnVQ$JCl?LVf3J3w8ATjt(?4L&up0 zT82#E)pj-?=^CMi!}+Al9V{oD3R52=9aecE+B=nxkn{l;17iGzGaW9U;B+E-@B1BmtTkfZ*2^rZqo1J%3O|E)wA-H2Tz)`6q9fn1IK*BjB) z7-qd}AA5j8f){|1{xkq8(_j{%d7HR-sqj?c@n`XZqM|>XPdtd?Plp>#>FnA+Q_M+F z;lxl|i5;_p{&K4r_@33v0&*sQz7#NEj?H(Z>N~JtZrDo^`-4vh^1>vzPg`kU+LG35*M$LA31+l7%I+I&VM{8(7;0`ZgI zu9i%SpO)0VNLs^6TpfeM}`)gnP5n}fjo`qUkN5*O@sk)687 zeD{x;)eZUI_I^phGP0He`fcM7qG{~Aop4;(`vk4N<3B=4BlPII;dhIa9Ua=)p>Oth z{kYS(dxAe$Q&?-?0Ni@N6TdGP3HQUcOv+FYZA@}D8yPVUt=32~b&1*IkqftBg0nW_ zH)aX}7ob_^7V&k;FRd)dUyCA9Vh%SDT9o#bj>wV?Or7B;$Fw%}i zz}ya>%efc*f%%<8C{`vaN_*gZC z<^5;v*W9T? zTmy>eM@}@5cEF3A7+)9=!Rmf8k;)KdMue#}h0`00@&z|uNuezNxX3rC8885q)0&vl0i3FaMOdQQv&>*ClGh{j9rM38i)Wb_eU zuvBjLVP{*1T?C&eRp!Ugnmmyd?z;R);8(1^4zQ#m+vW!mGYC1A@EFL_D7IP*NsNO4 z#MocdM`Mo-{<7hA&uiIO_xkHu%3(EcEQJBPDM4?M8QxfCWBf@Pd8(_%Dq9qVY~79V zi+@ZmJi2m%89sMK_nP9C6vF)c@xIdSM#1>z)sD^klpS!`G*J zeVYJoF&dkfemelM*Sdt+KRcA0)fbWCGI$j3RNHQexUe(6((hgc zvFqc~vg|!Z8Wo4y={O^h0&r^&w{i;21kz@LTSV8Az!3qomEZ%u7J6! z6C+Tt-*Dt4mCq@ApfXF1fA<&dtA1j zeU9e5^c4Bswc>d`{u{~p{}xX-KVEe?mK|1e(=Ba`Wh5_IVQsrt6D54pFJoFbvNh+9 z0LE?0NSFU!plGsw{ta9I+J=P>Cp3w}&x%PA>qX0a!+0S{m{^!4{*24HizAuzVtb%( zjSPC7x>LLuM925#GnoPT;IAz{Z3_sJ%TFhzw$GRGq#g>&Ch%ek`aRA=rZvEk< zfeo4Xv&#`s1|YMfXnSm{b)Nc05NzX(ZEdC!KQi}O3a{ftPaX<{JGJeZ_H7{qG`hwd zhX97-ht+#F*iQi4(k{2OPKD3Ief(F;=PYSmJf-5$biZ2kG1qZvtN0sEgNL=vK}LY z?V@;Zh%S`oLRPIpb5otO zju^Usc|~?i>xNxabews_oJ?!}DIiLfG0-a?T%Ct9I^dB)q0V^GXf|EOUjgHyWvQhM zcL|aZt&h^;@CdrSlgqU%pfCyG%}OvrY@1gMiPk19j=zEYo9%z z?dU^Xe%E+~8OZX;+_3e zrCL!x!cq)HgInrkY3)vAm8V6J6AnlrGy(!%20gM|V$Xb+&3Blv_Tx01l)ck1%hKRi z$zVLcnR=ey(j0+yih#=#i}!Vo zcnVw!{2?fF;Q353AhZDP(#mtzR1zL>j+iS<_J=*<^f#kg{vY>*J~r8>otH74DF6v1 z@^--WM)|~ui(bo(1ILh8P3P+T5|7VMp{Rn(k1f(hvYSJpoyeituSynbnB~xmZ76v) zA>`y!Vo9fHbm_KIhi$hl);?#OzI83MQrCQwK387$21^-g=QeRkRAQL9mb)X!zAF*( zoxv_`Nz7nndn^ssQKqQ-6OYZ4uJNu6mcIbuDaJ<|_y5xT55d_DIA=a@gwy`QGY;`0-OC_va$J2jc9dvEp>0W7-m z5%_EBIH2NRI_`xVQ=a?tg{{K89b^*-FA%*O*>}*i()V?FBA>&r_)X#TUXPBjengSd zihvRWO1TOwXmwjnmS9khIW@uQn5|i3O0NODo8P+Pal({WmNyVQzs_t%C_yTamaG@8$8&L)lE4SV3_Fe+yk~IGi~^nk1;1 zHo4tTyY0+-?ya>98h;Hc&>#S#Y>#np-<*Ee#YER#BzkIzJ7bSkgbEAiB5$vK-J~x$D*Si3z%n3f@7)$7J?NZRb4(uEH4rtW8D6d z`W=76#oL5BR|2g+Z*lU#lcEcZ2iU^p?&Um;j#^9H@>va+xZWaJ{|rjY-)QRnXg2%q zY#i>Gf=#5e=Jm8|)TK*#VF_dhh)gKxj?e8E^|PQeXr^ylDoS3vyk&gW|$`_ z^3*<^ncO(-1#t~~AYD5saz!tHxLSg~mahauGYxyUTk6+MFJCS6wQqv{nE0#}92U`m(2OLl|rw6d`sqnb2pJ;c`HNle7XiR%1F}^7EyLyZAr699e*rs5a7R>@A0@jX>+>BydS$vOwcXPl9Ees=^ ztAgRB8;^9q+u#KsSTn1}b2}w(HGdhJtFwm0U#2}|5--;XWQ z-{q*CY=ShN>q5=zE%aRqZ+HM|M|@$$nHq+d^#^;}Zk&Qxk6G1aTtOdY}s zxk;0QMh5un9qbqpSo1TU{jI-W%%o&_S{CFc%WaHTqFglV9tP7}#MqUG1Y674R{HM6 zTSUePq(?UHjf z0j>KIli!x%Gn);!Fi8*mizVNk1BFr^7=o6=n>w@-unYapL_n589}!ZeOXtO27~t3~E*cGe)W$`v17VI>C|LE3fYtUzKW7 z?c_21Un~HY@lJAdrF1+P=o@(84~9g07_{$tw4M!X5bW0@A|jlg9t{FD!_5JS!6w@ zQkG1+iZoe`RGH7h>|WR=mOCmBZghN1x`Zu5M7fs_tR~W|AA6tqzFo#`WVXzNvL0*u zmx~Uk<>FqMVz&3aSp~+Yf_}o$<11LKil7h|L4!Cj$F@+{akB{uAz#oKP74fe%C^;;E{>*!pHwuL5olVL50d)YwY3z|X~ZSuOLFqnbbm?t%qR3b3XNXr>|Wgt}!ID&t^T zU%1dkIWYQ);%I5dfF1SB(5pcld4u|hx1K@X*`S(fMwenESyv}|+4w{XSJHjoPP*qWv{5$~Y{ebI zczo1L3Tb_qbzVV>_JxK)EoKW&*kO`G{25H zUsX*s!1b_N!9klZi-{%i{I5pLGZ(k(Q&%tcQ`Z0UrpTp^!w7lbjfx`)_p8O=JyLFK z^1ptX*_wJS_=L}~!!>?j(F$SmW>yOQAikWMU_<7h@|y`#a&EU7*y$ z*J6;}Swz2uN{UpCCb{t6l@)cM@TB<^cEv-Qy8;MJ?*R%%Y6PW?NK86Etb)Ho-09oT zV;KReOGXE0ILZguYZC)-9d7dlX2&_SwV11-iMxWPNq~R_G(cJ8-KX1SkD$e_G98W( z+zh9|Kb$mqdh3;su(9NA0fUevD0y={-ls=lWCG#kjZbMlEMtt_KTA18fhad33a!w2 z5mpxJuCbPym9GDC#Stw+>#b-RGR$F0tg%bWOWq!Z-ZcjROINE+X>y;ETr`C1@^p(V)7RGp&S-2|*SukWaLRF&4Ax459tH zgF>MkgR^cUzz+=>vaY`f9 z{kZ0Tl}Sh-W6bS%g|t=wG~$}>5}}$*rhyj3d>;)5#nHjL(K zv82`r;t17|0V{!V79;fvCQ%bYi;|(GxvrmJCcRRC*T`J9zWT~TBae@_BTK+Lk(K%A zohofBYLS-IHRRRBKZ`9TIe(Fr*kqh%y{WzbAQNWShmS_z0ZQGs8}{16`bIqo3at;@ zdnem}F+F*=L<1TV19%2K)yfg4ZW$2)EYcBh&8=1A)GW-oF4HEPw7s0oXnsLL!W{RU zp-;L6=otHP+4{v z@5tAe*acVHJvFU{5l7a_^xq!6i=Y##ts#%O`XQD6hE1Cf>vx~1173@AZs(Jt4J@9`i!2fX|gqmQ}>I_K$7eqipuR#Y~=B51*gOs=)DuclN0`bFjKx8qI*#&oleTE)_rFf08cX!g9r!qvY3c@;o>j0^`+q&fHK1!Ue8uQQ*ny&t z?yn-;9%nH=Fo-RwHNU@}+o-(Tul&_gxV9|Tc{PTcWV7~xjI$Q%W%4{_M*@qdyWY1Q zVR}6H68s9F#_zP;;_Z%J_GOD8-3IRFL|DNaoQq?jA$O;b-`)ka%`$}*DcdqO{)v+SV{BFx!14%~HcD0?hBMnlT2S2O{f_2gEuAu-x9uf+}Z^DddhR-nY+t-5<-{K{i2eCeJ;6X@6t^gKpprljP7_=nz)K6Z+^E+6%gK6N&f^N_-7-oYfE z3B$(l94O%UZYlkj_b1<4t*d;6D+0S0+`F$zH+obpHvfs|WCebOsnh5c8+A!wNJ2&b zW?`acdTH?GE$UccON03zil1+7SPu||zbdX{7E_dS?OCMg`?dEvyhLO%k}u?}Vg7Dr zS>T$T*e2!XRasl6RGRyhDsP-O-#62*Y_NPgzF1ZBfR8sYoJPw-ROmcrEziB5i#K*m z%qbr40nkt~XHM3t(lC$-727B%H;>ZRo3R!ESx~w*MkvP*bVTI|R3{XGHcU=hbj3~O zml;`2h||2%pa!(QWu0Q*J7|v^LjpX@q;1^2h^MWg9k=dYatB4)T8b+*FPNlj_&V-> zy7u`Vwg_dSU3nwbcU-&l{{8doeK?TFp4|s?`rt?gAE4kHzeU2Uzs&T^cX+CR0tHCA zhOy^q35Z&+V2%9^#m*1J;Wk11Ic`?D-Pjk(sKmZHB_o!=@`k6QtR!^l^yHpTq76$Q zW1fP_LQ`GwmNm0%8J)eH0l1Pt)1a*aq&uA2ci_#i$wVu^&p|(Jzh%ihx%&pxjFM}7 zl7d_GNbAtF&BeW}Glve|A^JYfHALoE^NHL;(3Cym)_Gq}G9l*KTD74eZzN8eMlf=I_O8 zu>4|R-IH3O-eu|CsDJUUjPY<;A78aY3!{v%Vgi`k0YFUzbv*lLIYpHw>~U0pleKV5 zlFmj;R)?UdDYgY6J4cbY4Dnz@#ELp~B{)9~u+j1KxLfY}ZJ9wNQ3{k){U8o#ytgbY z#9||A;S&$;##KT3UT6t*Tn!OeOJ#3xI~VuvGhnF^E=u-`0N25;@%z{Y`)6?X%xc2Q zE!x)O1KeDzJKl=mTah}%MH8aZw;!?^s3Lu~P<|%nL~`K`aoO&2Hh%hQc|8^}J0MbN zNZ1Ru)(L$eU|ue=bujt{pM;9H5D)`$<*rxKHi&d!mHxeWgV}Uqt7mad_erJ#I4|8? z5ntL4p`8qsL1RfxM^mT%Mpp#+)#Li_w~CT24}U%nRMsS53~4CvSN9M9>Z{FZeABxn< z2dH84ZXxo21~dMzLHq{0qimc2UE?GjA*wEYdFds6Gnn&bd0ZdWx$~_ZOKUDADp7Uc z{XEgwYJ^tR>Sav04L2QiH~!ND!~p(}6^h9>J)v0B$y97Az7|akX{tqd#7DBok#qEu z2KBXfTB?+vumouXTYV2T%-3~E6Wf!`C!A|BNl4AAcgFrn8?Fxu@39)in&66uj@0>l zP?#}xAhu}@M4C#z&dktzU?dgOMIcLd=noBC{}2A~rXa2BHz@twxDb%rl}sWt>Uczl zkB|Rrzf!IBAr!az2ZyE{F1pKcq3#->jSVOt@$lsDw0Zn%D7Dtb(06!iR0tZ|*p75A zCJs2)wQVp4jf&FNT$ehN`N8Y#wH?AoCBjcW0`+Dyoab1x{keMs+AIEB(KXg`ONJks z2h9c%&LJ51DwAEhyK;*L-0@Fla(6V9v*MEu? z`QMMpTXInDBLw#*k}7T18*+Fz>u4)b@BtW^#~cO3_U4~yY5gdn{7n?!vHB$T+wpc| zC$H>X2Xr7F6aRop>aDhxvwiYoZ${4BrR~UHtPittiF*{z$VH z7h1Vgt&EqFMOy@IF&} zw_RXw)^WG!TLQ^j^NF{z7Rc>2VK zy4&6LJPAqO(g{dQezn`V^IM>=01H9qsm$*No!wUzy?k)I*LTHrPL#Bo~CP^CQ>!THksrk4iaZaf|&Q8M~JC9=lg%eKJj!0S!Dk z+@O@hP_5{=0-D##yRSyy4=26}6w~yu*WhBSdX; zy}djtSm%1V9{1sQfU^w1mr|!Wz;eBw)4HmKvwEwn-u|+@& zOM38U-=HhaV7~EY3+wdbk$bk7H+sB+wUSg2&&sINKGwm6&=qnfgGH`&fz~`Zta9L7CK52)i#3- z)158e@R+s;n`UjL%{dMl8wO)1C)r$??c^%Oe3If8;`=)4$R8&+^23y!#li3nmm2TQ zazI=|y^ukKLt~Qt!Z06C)gojV8no~|=RAdXk5zKM#!c1IrTd5KLl3)5<|cVJm;ZNx z>}5w2f?1Xhv2uRj&j}ikH!W3ajZ}K8cs#mn3{ZS^QLrU(6x+lzwK+pR)i|& zHEaG#bKXbjpKGnPU z6E`i;es-o-AsE15!8Lvw@GJubN#iJM)XI4riOPu5#fc{bP zn;Y1$C-qr2%Jv--kk8Ab*lV7hRJg}wav-1qVNE6R(}2~x*517K9BpuDjP(iA1w>}w zNqhuPh6{<~i(^lSGOAKpVw~QG5OJG;w7*8t$VrA}NDsUMbgxjF;Pk?YxLZ-f7Qzws z)K|W!`Gf&W+NhCg8#bv5YBplnd{NqW4hg)Cimb<$_z;#jECy13TSB*4+EOCyeC)tl z=K-cKX4qd@0b-1H3=~H5pi&%FVT+&8gT3hc<5gMQR-Sd|&Wjd<;A;y8rsZUdN9`%R zMLk9b&yiLqmK)9mk^}RK7)SBX6KOmoBxj!`J4i83Dn&6&{7prC`G;np<+}IMCnT1R zIu9f#_-xPQu2PcvXPLN{KaQWGD5!|I>WU>^(wET#{(z+i+q-FGm9Ldz6sUhj_^{R^ z^RJlrmE#cg#y;B+C3B4wnfwI3esO7to)!Zfxwpf`iT zp>Lw&BS5KJJ5!7uY-VTueC0P*4aAaWUp_C!XDrsr*zGr>-~ThE8C6a9^=Xj?;gbo?Vo!Tl4@4N}eyX(##~f^u72$pgX; zCxppF1Km%|w%vaV5@KAEWT(M09YZ+MP4gD%c+;5*Tdj^y9To`(D;F==$8(MhMiPE! z)QA@9vNX?Ke^;8dQW@U> z8$Ch1?a>DYMwpCP7)$TWM>eo7YMeCwjm`fj^!uA!{QWDIN44Z4Wh=o{f^9dNVuql> zT=}hjv0>V{E}Sm48c@Gi5CeYp;>5fGP}70f(7y457qL(B7}>G<|ZG{UV+xK8+OU(4bnj03QDo$>Hzo`RM3S> zx9;(B-kySm9=%V8=?oF3xTcx|pC~K>$#+fseX}GL5k&$ZNZao3StY73r>F*>wzadB7LeIv}Dh-*8Ac9ZCQ_5x^?KB zfeld#p5LEtRuh{XhoM?=2^-#jp9v@Ii5hdg<)4RA(*7J4!r^41ogpmRlqvbgqEJZr zIDMx{VWDP~PDID#xvf8~h+NPk4_+J->mue}S|y4?c3r18u~tv4@Z~G%(_D#l1eRU9 z4bq$dC*^(k9A%LHB4svXrz&`=pHk+xpW--*<#qzUvIPdz4C$e zd{Y5XOR0ur0Bu_9`lD0cE^5&HD^Z-E?9?MEGeV4nvpJa`&NtNmtovCMEa8`Rec$BR z-lno}5(BwX@Vk@^i@YXnUTl4xP^K+IGO~CU3{b8x5O_oudCmR1k=6OnlnFk%^1E38 z_)NVdvxnZZ9vmDsCi_DQMe~x=JuJoRC5p7+aF-f33U8MQTKg!L9LV!d7Aumbj&yK} z{?UX{6qdm~PFnKBDxgB({~_xyqoNGEzkhfbx}-}|N$HYKQ97l&ySrOJ8l)Sfq`QZb z&Ozypp&N$I|9IWM>yGDnKkHm;&SRguj=ewMDa-DRV$thcO;ZH-jniuN1tuXhcB!Pg z!e1TBOR~LxfI*e$+>jm%q{aENnh-lAI9l5)sLb|X-m-RHe_aBz)3mkF3cXuq=K(3w zJl|1&VCx>2Y&MwJ$RSC@uyB1cM4HE&^<787Um^U}v5pI3Os|tB_`i>J4)vt6j0L5z z0Ux0?m%OZRomY0H;i-ZDKi$A$gaAL9b2YK33puTGH>io|7}3q!n@hd)JCo=NL4z%0 z_MWw@SmS{5;I8o>#ks?V7Tc}ho`^@Eq_fM7?C$ zC!bV)7)X&IW5y*0Ae8lPXH;27V>M&%Xhy;_NoODe>FEKB#94H3(YPD%*dNDQ7pEh5 zzh*P3ohL9dDU$lL2vZvY?1A%TcnuKlVggfulX_4VqJOkBy7DYpcrXJcwy#~xY+-iA zCcnD+ytW<49qyc5Sbz-POG1CD2^)zLOtxlFF~b31=R}NkHX|D%_ZugnMD7>2z?_8E zs*0KlL7tP7nLwRBRd_+}1c0UwZc@Gst{C&3o%{H_^qF=lP-hC3PY1L-}LS@SWOrAIFEI+A)iZS{=mJKYi_Qun&{hgIxHVDD$tdO(EDlD-14z zJqy2wetrllT10H{YXs;WWFh@}yW=AR?^JZFKU5Vk`1}Eoh{@xd^G;`S3#s|SE5=&Xnl(vrRKnj={ zvyR@@Jx!&7cMJyfe7q~rBS-fE(AHg-mKp_DGa6}eO{jQPo`ebWbKv96R5Zjg-XIWo z!}E_jBw#fNHV2x>y$1eU zjW&G1oO8+WhVcPTyaPcW?)O9TpU=9>KRC46dI&JjIc{T=pn6WL)_y-eNMAQhPeOtM z0O|UYC{NDz)13E>g3{|1Q{BYZ8T5XR0ANuk!NVT@^DiuQO?xaJrX&s|hE=iD2EG9C zqby4z;>o;8wuF1WTbxY8uVp9iU%A1FSSez8`c`g88~nlBLPNsrhSqUn>-hE9!*02< zuJW0x(Or_wSJ}$7zE8iRDM&!>>C}A z-jRk=_t+%`T7k5~9Sr3vdzuiQhm41$tB2b*G;W*| z@UiekT8XzdQi~y#ZOWd%sx6LpNLensSC_S zPSt!&lI;sxp{59X4VjOz&N)4)FzK7p77PeNwTsVKmSOW=?AQf#47?{OPJfG7h45y= z@@>36-}Zw-5?qbq`RFCr#;mrc<)dqtA~p-WPwtUjg*ox%{nq}1;j+c>vZA(DskwWz za7*9aO5Eg)$65q)1aqODQgz=Qi0zl<@X$e!Xo%0Cc51gJ(bBLJWB->W)o&Oy+tbwm z!O*?Hgk58qbBx+N2-fUGeLA_pgvC#K> zGS~OZxd(%(aE>AP=N5VUg{az5+y%7OB}8og@aI3QR{pBLLRt4qWAz@nIcsh&fVCCPN z3a7j!rflW;K}p$Sq=mLSn`qyz)Tr8K<4+YEnXg{Ylz|}=s?U=i3K>P&Xe`NOy6ukg zD{0G9f25oJXr7Q*GT1a(gk@d}fV)LNApeoA%b%iVVAK#6Y0 zW&f5OnFcXP;fCTvgJQQ!V|soKVP3~q6>ym-ZS8G#JGr^(e`EYco&vCIjQ>@1S{Xzn zG_7a;;EkK$ZNx`6dG$8ca7bWlnPhLh!jxngRgabUvYCOcQrS^xA%!PMhFTQ z)W!|P0th9@EHsV)2qt#%z^%A7L2s6~>J=m&6?Xw9<^Gt_hMoAqP3ep+v<%7=Dd0n^R_XvSA+$?e+sj!L6qMqstUuTs2h*s$0~j6xNgX^WUp{62cI zPK`a-Eh#o6b|wAb!?9`cS3t;DLBHm2YgRwRP&`hF-wAw;8Q|vG_zI}({jJ^Mh$j7; zQMDm4$!Y6X^aI&ZR?)=;6-s(OpAK1L&Gh~I90w&T0x{y8s75<(_*$xBP2GsuVgYmX zT6#n}G(N@S2i4j7Uu4YnC?_neZ-9OnR53lE9{I1?Z10IdQ4h3wZS)D|iK<&HCq}Hg zCP|ifwqKnvk%(d`0cuC^Lc&0avx`+hQ$u$KA8--PZe_+As@Ufz{*6ada3AtdmjK5r zmZG0CGg~hlu>?airJNQ7bVYBNk1;s6_%?EvfW&Lu`6=6mvAG`b)5|F9wBKN~OO+UW zlAGd3*6+!mNH@sYuua*rkOXW9E{{KDEPZH0ouc*7Zgr5E^-ZWIFG3+uBDG^^E(qc#aaDz}_f?*7d}B1vgpH zo&0v?0xgqh{`5W6WUTN1jmJG$D!}`&@h@t7wlQa#V4>|aGgpxTCKXmb$2|i$K^wdo z=+lav{a%}CVgOFH8A2TMfqtdF!`F)~CSSB{h7BIYOGC&;mwZb%Z7E`?49l2%f^lk^ zb_PxeHOmqtgaxObAsTCxnOpj^4!1>=5dJ>U%!Uj*Ue#&{;jQ*TL_8OWQ*moY$+02A zlyY!d%INd2^2W!ha=xF_ZQ2L!AiEBqe#O{Bv5|d#!y$T(VTxZJMX+S0CT^(P#x?M& zjzIttBMUm*VT&Wu51n5rM26f%h^E8pHYY{6L0cO`A_AwWRCM3})7VH-1>yyL#asw- z*+qycUwAtsY4rN@rS*4Y*+~uP!xy9r zzVoB9psWoYn;*+Oc&ueb>F?&1M7jqp!u8|W4r#U9nTaU_a3m*+em=jiFLhZqycd?f zUOe{DCpap67vl3p<2D+);q*b#2ZoSRG(mhbs@SZuy5Kgk#R3Esrf9K=w$|%VZAz6hi50_|2EK#2-;xa!3XRb$n3kf za~eu4ik?7k{?K7fL6jLBPUyt5-j_PVZDfzjgm3mKBEcVz-fv&YOoZ|k#tJqU$9d@3 zkD#32D{174vnx}noH_Z(;Ah}%B&ld35Hj&3aT^5`S?0OQ*uyV#4K4<>q}PSQ&0DM+ zmueF%!NcH;uf2URYH8v^vnSo@{f#GNGlF6$9CHodmk?6rpu$Y7NBHcIUh6+YK9saQ%>6lVdNQ zob|unPIe=%6UAIO6&)l!s!p?1go!eTTaJTMxIa6jp2JlIzPf% znQVC)ebwD4IvcH%9%{0q))0Xl$^UD}%CzSQQ*oKNC|7PD342z_42pD0k@>g%#Tz}e z9qRpgD^F>pYZ!_r4zem--*}G|7gv3;ahBPi#J9&a6wq`G^`SsHEyi7JbnZwV6m3E{ z7p^N?q8kdfl8u4qnK7;Njle7@1bB7ann+pkVt|AjI0TRcjooPIIR19Np64z!*^(ON ziRrMUUaOfxGF7*Jaw!d&8gdb`?#F(#)$A`n{$#4Q-wSJFK?j+98qO8tZlzAs_YA~?j(V@ zi7WLeU#GKkJm=4=TC7>qWz(%KHQ{hf@bBS1HRKlkU}Eb??opKsDlNtMKNn2@0VxDp zppGNoO&+}1OGoM)y zp6>7yeME>3?gFlEJfBj9Y&6$z;=PtVqyD)eDTv^$5PrOHGq0rN)$b+D1YSI;xR`+{ zY%5CpW6J$t=N2BGyk?$U&w;i&yDx@U@C)|9R^6`mM*DTcn_3ay1ty^a1Gq`TB4VJK zfL`8eDpI%RDxf1e^802x1KjwNxABnzb8(qdK+9XnK(N}CVr;)s*X3WYdnOy_Mf0Mt z1KtsL>oX<6z=Ip_I+>WaRdk%IWW>J5g!vT9ZTScUqk5m)RR8NO=?f6r88LTdMczkl za`3vVI|=j>)zDE8wx4SVL`$tRdgx!f95%k&r|)p&Q81Wv`OhI3bs@+aId**>h5Wxx zffdgG!e8{=rX)G)Kv+zLJ0SFn9$csS#sqWMOUq^vznJwKZ2V^dgW;Z}Wl+Gdvt%1s z$q;XZFxT8E#xUkeHB3X$3;SdXyZP)51xmJt(W`?m(2)3~WwPES!7)9(@s{z| z0FMbE%d=|E3gaej&^*m9N{&^4Yp&)bj$o;_ueY+YV`YW)#+|;@waekgJ`O`{PR6yj zRdUGrOFwu675L^6MKvf`-+41^Yzqog;ThDHHojn(%jU!-%AFi2Grg+L-}i)OH!~ zNU$<+eO>pOi@8zW;J>Hyzf9Fj0A``bacMVRQ_CDe-_u_3;|B1l{yNLr!0V7KWG-hm za``HzMQ`wB#M*{>0IkXkWZ(16Sgc_}_-@XP;Xs88U8Io`Qj+|+--lXGVxN&EP44#b zsdNLcZeV` za*Eve;h!2h?Ph(N>~kwx5L48{nx@l!xQZBCo!D}B_Cnu2;$a9~&wO7SoFmq2WuD}B zlmKnfS`I{4W7-|UF_xd0>5v2lg{GEq#a0)w4u~`6xazm|*9x;XGxDwfy7Kt0u3QHo zT%KSzCHddA4NZ!eegv_+p8l%v$jCH4MZsHuUuL!R8YDx(EPX!zyI_sAV-0Cvb(JhR zp%BhE*MDquBNF#6w+B|7??z_$QOhG0PG`N_TPYzDHKY`td8+_>x)3jo(|E&X9qkuR zr^(TA)V};ZvD=2sAA8W3rwYEcwn;u~$gCOpv(}`xM(DtH(Xxm1UN+c!C6hi6e7)=9 zA|S^#3FV)tdz*a~CmcAaTl&t>)RKSD_n57P_RN}e+&%?u@aL>h$wPGa| zHrVduBG=>GpU{Fzd)bN!cR;`s&vd^^9lKOGVGdtcj`n}vb)1EtK-vbZGLNzf!>+3dMq?t2oXPTZb7)0IG_X~XJee*|3 z5^^zQS(n6%BO&$Xch7Q|X>^WPnJ}8z9Dy9s?chOaO|&~vbsTN9kLf_vmpWrerdC`1 zTXdBi^6eG`02^Xv$a+kh|MGh#~qyGc`z+ht>* zgjZ$a*46a;>NdtLbGB}V>h^lkx!g~{?R?Bm<}|e*C+}qm%4d-5DQF*k-lAuJR_UPvR zk`ET1KguiTbYu^XEp0+eA~*c?10IvE)5ZjHB%ZUFc3-%YiDu-O%BEV?tQ~nqTeJ6Y z9awWCx(wa9-Xlt|WE#O6doBF&dW&$Qc zxnmcndzY&)!{GJGUN-E<59~BpPFqe`&Wv|87@M5WgcF<|uc6Q5GGwLRn=mK(m=-7p zo73f6q)iz54q{nRVMy6RtR&W1-J@O8a@QrPxibKZK;A#PA6U~^bDlPPh5i9}s-F=9 zR0s$W2ohNbMLnWS5U-=QezfQ)HI>IAY|O6i(Wmpq!K15N(E`Z^$5*O_0p7@iTa&&N zF}|nXG&iTDy|$J4#*`=f?gw7G`9$*>v5;c)JsjK4EZc9=$u;4 zG*!Ur0BfA=ndxeGVOs&Ca% zb0luwg;}{ex=Bs>9?eIdb;sa;?Xf9Rpjxr9k=Q6C>32C;zIomugnoAMq#c%e+c`lR z6|Y~yB=PTuP+H|K^@%iT?ng7y!8M${yug5`rJ84*Y~Ry1>vw6(slgn*GDpK=Sv?uO zP?8m>pXFjM*dDAq?bbXCx^IPMKejoHjfky2aaJkad!#yMPTEdf@^<}CO0l0TwiEeX z$$9jmwTg|=AX$ux*dxSeKxEH&Ssm_9hjYklaR@{TiL*9O2%c)gLUiz$izklhGa~_3 z9s#WcVe@a$hg5Vc=mh0?kPM=Tv>TLI3tgLwB5WkRruGTplaJ6I60Qyzw zw+JcrMGnAHu%U3+>ZzA83*n~PCxB9X!_rT=Z=aTEKd|eEJ~jp`JjM&^VeU+Sy5Y+p ztA!0f>BFjdg%#99K&3JeJ%_p6S^_VSiBj8;EXn$QoxL_`~=j zZB&}4Vgk5ff zE?rYybpM-eh@Rpab+NNsqRi}|TAUVYIHz`d54xuam z-yQOtG4a5$5?SWhw309!#c(O;j5keZ&6`f=Mh#r5D)Q6L)fj?J>O@o0z@=gN(c%I8 z+n3O~lG=u+$tQ2oQqQ9%GdiQLR6qZmoo%7i0?!@zLCiwq*JQtu)*RHV zD1G%Lud4Z6+<-iWT#|8b>c}DRqsomIl7-zhqpcl0)!i;Cl?JZP%sRDHK9A>O9U$IR zNSFpzV|mdi=qMtx^rl8Ecg@OP$A`ZnPm_fC z<;?()&?Quhjn_bV;)L9DB~h%Ix9cMI;vZ}HgMubeG~Vwd!XkSdR1AxJIVXi$y8!HC z&hAPO22JK}6jSy0*X!;V%_!YA7jwoTHz=IY*zF#Fph42R(y^XQSP%Asg7orUacaWF zPR3Y zqSfMKz2dY=Q0Hw*RNjaqcX8A1CDRXh!(7>hMVMI9(DY z!M-Q&D8j)powe}P?20>6r#s(FMsV1CTtaR3*I;ryML$~r>9i>xTEddEHT7G6v54gn z+EO-cov`p5f>wyt|L%6{}-jUR8D zek)6dn_l~vc}R5V;v?^@|L; zKPe?UeK77q7T%XOD1;#x!==pRvD&kfsk&`9(`$KoEj38+Wcm8~n$~@RLyschJd`r# zA4%e6I`BcMz%<(Mq1ey|_pUf*mrWV5k#k{(pQu4#fS8!r3pl2bP|J1piMjF-&j}A@ zj7nS4wC!}yuuJqlQ)m)>4>Yl%k2L4vXQ=}|(14L59A+(^wKc2fg(!3-S=rK(YIs9K zcP}gj%vTpBp~?RP!_WrCeRxq5n`p)DLw2q`2b`)En1BC0ZP(i)p-VXBz~ga75fagxdPM5=Ev6tfPZkMmq532s1`k%n_FzYsa+~`TGx48h}vF#AXAS2-* zAAtp{wzu!r>UeT)rgHF?tc)H}*4*nXGs2S<*{jI&+!CTybGft`r|pL>{McD&?JSR0 zw2mup&`DZsV_suL3j)@@+pZ5g` zMZWBhcUi|!Z>{kEeY9<`DzQHBlT-M4PLI;(Nzs8P4$HbwHXBz}(}$_6Z{s`S|FRtE ze1w}(hjUXlI5>JDbxx>8l}?Hny*Q7$QEVCuz3CZDNGJXu*c%mqzuj@#ezY~pRD&_y zj^cmyN9rQZ3!JSFDwa$8r@{E7HrN}A3~9)FwlhK0nX5M++xkb`b%{yrJAetD~6_7 z&QxaYABStT#u8Btm;y+PF!es!M;M%u>tu}mO2BOc)|5yaXm>yQF(C>k99qBJ=4!d( zb5s@S%y2Yi>pLQ6@cUE!kI_3A%sjh##Fa^-@;j*`h(liq^^_Lp-xLe1sWuFeCKRLS zzKjU^lgJ1ewIc%-3DySEc$`wikuOI8E+ipSn}p@fozCY@c-U+vW|c_MJ|AN`Tp^hi z;8?`k6QpyIj*FQdCYHw_&oL*HKVU*pqPp*{TQc=$ZZ>}gOYVaHM)_!RbhYuY@6-1g zDyn)NnxZCZ0f$6t&m%(-rRo|J+%>H$N)Kb_L0`%Syn9759GcYF&07i;7|U3U^&=nX z8>)iZaIaF-4VDl17je*nDxqq>C#4f1D}*Qe!#sg4IbLsPwU_A`w5JH7zl}hhLN=p* zMD1w6T@|Opiq9XxQXbKSZ|5Dph}?dY9D!HsZ1qD__xo=H7nY{nAY{{T-E5}>3GK#W z?BAoZPsCR8FY!l_aDgT0^EO7ASZU9baqoIT>h~F)5IE>gCnV`gRl^_`M$iMlgA-?w zSVOZgC(!MsC&~y3c&YBZSwiS~{zLh!@*hDCV55*@?hu79a{x?riWlYx|FE`g|3DZm zTY7|XJh_@3*K=q5PF`89%m?gOS~=i6=BPJr!`S@Zy})D#-oK#~$rck>z5wkBvrY0r zV*1a=Fj_%W&Bs{|?H@$$N0hcYbLC|W{>uWu2}d(uo}8`F94ZxkKA7>CjefmSdd2q` z>*A-U3UPGwKhlL*2JBkm9!<5Mb-%uB`+LF!M-JY`sM3W*m(!yAyiA@wDeAe;s=Hp~ z5+OX*z@kk!l_?U|^@gC3Q1+I(u?F3!v=mj#ii=#kqPancuWSJZSR3_jKx^5^6{8SC z4qthY%!mV7)(xZ~u;)%zV%y04>809FKBtI*w>y;(JMu$x$X=9EO~nb%HvdOQh1|Y6 z41T(a-UfkGqvH|Lp=$0^ZAnZ1<}A+4CTpJB(N^r1FZn4&RT@$Ap$MOvlvMQKf)b~m zby%mN8>;0_Xt=>Sv?eNb`$1dpI%fg44n+dFYv$aW4 z@WphtfZ`#<%!+90BXZ@R7kd2=A>p8rd)myF9MP(wWB8TmgDnl^H_s)b9M7eR56JXO zF~V5Um9anDi<$+yLoA(+J9eiLbHgZY7>)vcE&{Ob!9^FE9&viJJcTJIY9?+PeWis! zU$@kN-?nJR-{35mJ8v5^5vh5u86rj92L%qZ>f`_#zP3(QImAUWDMABWtl0Fwjtlf= zIJ5h4^ONvhk^F~Vc*m@yUen7a`psgv@?s^6Eg4HrJl*MjX*cSyHw%n__!THm@^sFb z!)^;t14XwUbi>LDh1|zcrz4V-Ty-ViOO?<@4r6w?%e+w z4Xh79iRMU(`3QRDq~83^b#1qvm2-*)%5pA^a-DEP3Iz+nA7queE~}#a^l}Z1N-}r* z5KQTb+Ily(krZu|;UMtmKRl5IPtFyyx;DPKx1>C`Zjp!?$8O|p?{2~ zqJRg-*OTs8g)E+8=+dNam!J37YxQu>bA(Q_3)o;}7-|u06IK_9X1;(LwMZDs_SV}F zDVwp+cCJ!}D)Zzq#qd6*bu*mwt9SGLbxoi@5Xl}2;|Q#6CqP;Pj#{?1qS+L!5w^{) zwI^G;pHJ>0^X?*RtFm|<74vgX?#5suYj^)BQI4?PDwq~_fTzjX;u~}5Z=x>tls4P> z>i;lSXtI_ayM;y0gRy9+{v*aFk*~CI?wa?g=W}qXXH*)^_zH6|&dFkXvk^pr#<3Ag7{t_dxg!WUPyAp9#_Zo37%QxB#N25%842lHs5Mr%TdaS_ z2As9f^vNS!XXPf2 z;}V^q%-f};EWNxsbizcyJV3Yu1}s^k-|dVaheT57VL?s6v5cT)lO zn+=z}f9$&LWGg%BI#6(oY5FE$6nF z&ENWI@AWAUPf#hpYTP<@MWxmyUft4fK$$$nk4T7hxrL?pdL}{E^zijpr57b;p8xa?~ zo5M74R{Adhp~SuFT2Onhs&p^4#C`zWHUeT0q}>GD=^UV)SyzpRp+xHprpi8MmJnmL zv%l9QD8mbuvqI~r)y=F_wrNNnA~su1C9AFM=X5GexJ)mRrDn>vX0>XRps(c(=GC#^ z5DxhEe1T%WVIL^8)f0R`OleE|=qYvXbg^KHWz6N6%z^%KvAENDabU03Fur&UDT#gw z$|?|M@5v18C+!LT!iJxN#KvDfiAJ+zr_DPzLFb*Wa48>are|OC7wUCDyX^#O^{ZZa zF``kP{bg`S>;~-OJ^EEN;z!7ZS8FI{rkdBl`p!flzXgso50x^9U!cfyp53uxq8mTg zCTqHMO>WeC`7A|LGz)q(F|^JUsL0#B=J(WjnYVpSuW-dI@fa$B-%>kK8w)G^srJv>; zusxXcIYgxx+wMUwI`ZzX;UzF~IiEKz)vppp8>BUaLbHK327Ei!RkeJ@2<#B2!45}u zgYdPQ*M}PVS}>wJQUj+~+Std=$J9-*5&xXMh{WC}?otke4aMCx(zWw&gwv7s>A@WP z7LmaEQ@5vgfYK_^m+EBHxw5G9e`;$8+%J!TuV>R5J<4=>*UN9FD}0^?*aA3Um=vV} z-L=b9n2(SUs3F5>9`Oqa4oMe%g^E(b?A#y<#u$Vt!3)Sk+}#bN7>k}%WjESbgN3w5 zdu-v&)|HZ6o9T-t&~HK4DqvJ&!1i_E(#VA?VmhDYc;bLoA{W{=>XG?EpIQ!BPv59O zznYd-^o>y7E?%we-|$A(-J*DY37Ov}B^E{>8X->1SzCkJePjXvuNKj*7i5pYRaoYA zqsSV$+1CeChx$V-z!x%|kGkMhf5y{a5=&|%+D%1>ImfPz?L1l@^gRBPd_(ckD>HYR z)WDZ-Jl#}a@+bCZw*t|Kc(33FswllsB8>|(k^nc{Uw^H$yDj!j!=NZqgaE#MJ+(1?j1R2k!uda*o^&?SH@afRPSXvwKU0xj97m zb?>?O#CMBx-n-P)ls~=>~hGb%c?av7i+BkK)24?m*oV zuGsrkUi&v~el5GaIX!xN?}bWFR(2ah9DS~4Ga`6y*PeEH>9Hpo1qKpiN|$PTa&iMO z+d4y%1bWzCx?gW!MM{5X)yQEhE(erA>0OnU14#0rdR-RFM|Foir8yo;dVXCR)7rU! z{6||#^-uI~PWglP~W_D zX`{&%f3A}eL^%C?VqyHX$*2qEl9mU(Rc`5rM$)HgiiV~gBexhit@__Z*d=_^XPZ** zPyLO2g`Unm>>wxRdKI77jVP(!Q4(&gY1Ox~RkvD6==qGKF1QRDG4EypY1WAoYQGe~ z`u?el@RQWF3ksm}&ZRB6N%KxGiLUT12GlR-Y+;J;qz{<-Tt5L-uU1$SwcSgcsgJYk z9(yAS&pOZIoiUhh{z@GOPcM}IeKBcxF4LaUQua)x*r;kkPhL(^dgHzx)_UQ0S%jSL zGFC^^DI+EXAr110^pANsVP*)#(}e}vbw*vvcy;#R3eHlOy#ufJad_%(wQ=0gh^#o( zozs!`aDr%}q7JCL{5Z|&f+Tf+K_3b_Lnfv)D#9m}*I#9ZHa_+t$7#aISJfS;r>s#$6hOLe2LQ#&D^ietT`O z^@NN$|3GviR{8SArK`sV(Z>d}JTRzyDPUiMQr6?<>aT5En>@K0%F_@!jF2WpSRf_N6wU|;fa?J8r|QgBpRc4J5hIKY)tU18eNItkLl zr=zCH2Uj77JqzFd)!wRaXN^~pvfBG-L6MNQ>ko1DIr8jYeM9jDzy&8n$ng1SimL4I zkHKeaHLH)lNQ<;3Q@JacL3^Gt&-B&g_c1C11lyDVsyf)7y+ZD zSMZvDkGG4OC$I*!gQhP{_6Mu86l<=s0tTr`G)>?UiAQ~!$BGttflJP%KgVr~0b2PJn+uj|9- z3d7sS>x%N(idzQH&Sj3BHxcAoacxjI^NGx_aSPpQRBURhIiCb6T7;+^g)s6*VsnNZ zu2C=yY*0k#;E%O&!aKrF`()8xpv7OvlA=woeSOSm;zp(fuM5T9!m+*_Qo~Y8fzmKt zB_P}Yf)^<`{La4w60ydj#|7&JRzE#G*h2ayN9#R&{9`We6ihjM@^d+~OB)AJjT}uC zo3cyMpVxh>-$(lk*<7nqPSAF?mJ;6IH5cyi5(@mrPABsNXmbE=c)pn=JJ?S-XVvz% z=W+LJUxn9;49*x-e4-=|up(Jy>Y`F+zvw=kFUSh55U&RboF_+O@2OwAX`VR343h$y zT1Mf`VXze|Dz>vadWh+p@;ut>+~o^`3?e@gmr`@^%qtAUDPPpt%$fmwbN}Q*SsYxf zAo;7vx&_D%Er(Ms8&RQSqYHGS(DE+42#WgJN%k4b<@0jind>}DF73VsgA-#NL2KLv za~eQ8P|Um6{S;PYa~)o(8>K5SJ+QR^n0wMgHM4v!i@C&R!LC$VK%2iLKgDV=dYS?{ zGFXvP3z!tU&aGoX@&3oAJ4ty@X5Ncq1{%F0s+K1UzJF!tXUlqd4y0=_mAnR$-0uwT zM>PqLpaqEMc-f_il?pw%JUP)9-xV+8JuZv80&KX<;pF8hW*iG-j22~$c$`wjR#G{I z&dzf=et84?Z37XelDoM8hKM&)!^qc0n)=0p(~W|pFB3OElpGvC%^)An9$R5d0c)dm zJ1S!rGQH-*dDm#uQeD;%>!?o_lj=sm#W1zZ`Lw*3E0D|VjwZrB5h8pM|9+2uAM{jl zXDZZD^JdYP*`rAKr@VaSch{rsl=@5-Jo|4x3{37AXA4edhVSkDdX~c%-g^oWjxG}i z<&axX%peS3+;N=*T_}DsRSBRGqWgWpRPD9jj~t<8#}*}}{9fY)y{`+H^7ie#5qe7r z!r-dt^hM}^BL{94$0~$3Pel@<-QeKYZDiXbpwF2b=l-20ukFQDK@Z2&A()*UoE!Gr z1WW`j3j>~e6-+Iy_g-4|Op&NNf64CV1^<$+=*Id6j*&XUH7&s88BDKRpeMH+!7;=^ z3Nq+Q`#k#dU6x*>5tCC^O3$Dd0Q#fOT1{l|Hw!l?!82Y-r+CG5_G0HvMO@UT<+;3K&%Mv zoxTWXMeme{2os>F?HT7aWx6Rlc(siNEZ0ET)j%RmuFVad@-s+qf zOcoOz75NmhW!L(ZM+#4Ltku4;cgV>uQ{u%dB6d5o&T?4-9hT#@W*wC%&3r@rL&pBM zJ-XfT-fPii=tj9z)jr1T{;cDrI)U6`qPst8ImATT!Ed^Ruw)9LP_zw($)RzQxCGov zqEOMYz^|S+(53q~KZ2yRpA1x0>sSUUT9^tsQpXz%KH`R=P;n8r^3xn*8T9u2qIb00 z9N0^H(4A})RC*bIgmHD>}~5BDWJRtDcuqI!J(9c6@Cao3dSV z>Wezuz>z8l>eC&ctXM+s!0sA98^iy~jCSmfHKgqhG4Gdi>yieK+NUW1$7F$nAK-)O z7v1OeHG3KMG!6aTuk%gqM!R9U?-lgfv2=|71w*oDM$#ACd*i1 z+iBWXOZdo23GALO1bF>I)2OqL^LnbCyXPiD+Q`b7{ylHcM0zsvvBW;wz6js8fvW^B zb!@@yzMyN0=Y6TPN0_2<6M2w9$oE5X{J95eii__{E!{9-TGkrh)gwlQz~pHxvEr_U zvRfDmk?EhkAJLJHo*z#;f;QE){aZPw;IE6}pgU%58bM0G==&VYoG*=D|Qk-k5(c{-TM`3hH)j0l#^ZHjf2*EJqvt4B4>XsnFcvU^Q5QgI>pVG*F zqVmN@a*`ccytbZ9@se!}>clhhPYJ>XowkRx@9YQ%-3^Gh)eU~`9{+@Qb~i^3%$i*M zOxChLAdT@PLszp(eWYtn)kBzzRWtrz3LHLC%A%HWp^?`$ED6)s1m>*mU)jdsRQEV| zsJgm>S||I?wGSjr_MY{~(^8^6nFXY2)NsjzS&eTDA7p>oyOo}tRnrF(bNo3VcEvd? zrN$DLphm)W{6lb2!^(>-lRH!`1n#R5`g(V);(?^XyQQN7Xr@$iI_(c@GZk0&07t`h$Jf zRAZfc2oesD3}1Z^o8`5en2rk~D0HCi;I}y^ zuASq3_4v0FBp5eRRJGs&Uti8%4aE62h!ajP$JlBBBvdE2Mn7t< z>s3ePSOxlwr0mp>RWlG}2U-IHra9Da5?s#3Bhn%xBV8W-Z{r?a;$2!gsu_mX+LGZ- zgoWh3Y9q6$$+j@rgQUKh>NOSGLRv8)OEX1iER5;G(beod5e2zNhU@}b)VAqF^!h5- zP`N@q$>c^B40sqN)16bg0zaHYSGtB^xeJ%-^^X6!JYWaybhgM#CAO86X!VXb20zlJ z{llzlOjhRN%P}r4;1O*wKDaR??Gr{k8I#Gpy8a;I^K@*31Apl1-)XkscG=QrSf0d* zIxZC14z}vk$%%@>$)Kx`cO`g;7PpEiN8LEO(jzZT{JW*t!OPFh-x{{Wr{zUf8P#-c zm0NCks98=D%RTc&QE1)fg{};0P&Aj>A~t4)yRb;x1I4Vhcwjt_i^##`gt4tXXR0r> z;Pc_Ga_`42HKvYFJ-dB{Kz*t-7|tAT&jMc`nQDiQumH&HPq4zY803oIz9z7zmORqC$~S+Kt&<$UMe>R zx9yr$H4D zS$Av%dd;GJRf14XR5yj zz5d;}ko{WWuHF$Kb&7o@S5wsk>UD+jz0!Fl3dW!pT0 z8j;<};I&4tFM;)42|R`=Z#yJ0O$!G)vdKO*&0#ay58gSU0Fo!u$_*Y=2`oyPV&Clu zxU;vuBD>W;RgO8Fo0qg!tSDLUb>t(sfb8e4*8qo-S|L6v^@nU_iF8p%0&%)4+eRym z&QpWEgmIx4I3hhHTg;fzxi2Ftetka!Y;Lm6+y7EePuKlYh<8gnj;*kbYp9d5`>5l` zniZw=lxCo&7&w~;`=0h1I@c~w>ji{j!f7g9;k1Z_>LTnRlEgIQ?Z=n@Vmdih@Rf() z>l1X0KvPd;h@NVa4^a(nK1ju6+KQYc8P11a-dmj_ftF`yc2m#ayi(uuycb#X?R9Qs z{o>>oB7L;a*i^1;?sy3oqscyKJ5ZA>cCw z-;~N2@yl(rSB$oC6hBAgJ-c~dx#nq(hEn1>*w=|@g(&reqfq#h9F`%c_d%yXYL0h_ zn;KOuMw(R`bx*1*bk`a7ODxP&oGM2^&@y&VmitCcCoAZJHJx$K(8-4;mNuF^=ZLk~ytsVy%pB!{3NT-W{nbLI-}-JCcX)m{g`YdXB(C0d}A6 zI=4P~l8zc~F*c;v>JJeloi@}t7#v11kz}GDsguprSy?fDiNDAfSAupp9^}fpcUYQ! zjA;Oq6E2c3N@5&1Kf4J1e`LL7RGUq=KMaIITeS41xU>{2#jQA$7AfxTUK|o2IHf>a zB)AjY-3tUO?hqV`6i9Ik9_$VGd7g9rYrShe@FiK~nrmjyp1pr!U1m!qk@mFAi2n)S z6Nh?iq|iGajPo~BFTB57m%Opga_@R8)L+bPyNB^do!TbOejB8u{lfI=$c<6wBSAUC&K5GPu=4(R0yl}u@d*S$0v zSN85K#sj*Kv_#9&n{t5F#5-YoX>&oiXUgyXeBH@lQHoCV-ID#{EtEjE;(V`rX z_Vd`jAFiq zPv~d3dYQ=K&N*Bp@lD40=1hX5=+&F(&6CQNA>y(|>5E)~l#@8b&mxSvn<6a3gM;!H zhdl1V@|cC6I6n;Je?Ud^iXkirWpd`9A|$4&feBPw;W{ zZ-0)1+9kBDv2)Uc$Y1tFesL$O=vl~Q3VIli7$pNvX+4*Gao_a%xQb(|eS1Gxa8T^3 z!psWCBbeKLlAE#Y!8neahoOPLj9qmq+{qc35si`N4i0>?I&Fx_g8R=0-fa&Ykf4&5 z-)f5V>Jk$HT_87xE|&eR010_q5+(tNhJyJvk|j=1>ql)4hoFIetS2K>9v)x5<1cPauk zH~i|nlU1-1kNB`goulcp#O;E|8YsL{n0`5K z^7eYK^;JP#(%hfs_*)Ax(H-kU7usd9g2{@<(`N2?lc1mw+M^aVysJIBBNj2IzCn}C zRx(mRUMWhBiRP=Fy+#Ok7ehO*?q~C!3LlVLMC6C;d5LoueB#jD>ulw z))HWB`TWbFp!AlZ&K3XfQAwlKM}7@1A!MGZqdrz`AnV;nt`)+jU1_N zApDCV55~n6BPA>$93Bs65i-xg+&DQy&yT_@e^Qe{%_Iglf& z4co)~$!o5Jjv0rlx0pY%Pf@c^l&xsY=B;QD&-c1C2IWP(`p-A& z(cANeF`8%sv#@RL0;{9+i?LdS1t;-gs27X7cWqNof=;~K0XX0Huw@c~TbM~UY403C zV_yl`zR*dsuV{TOc?hyHH@|mt@LHwuD_kF>fxP1rXkkGR7EB({NE%~!i5`n>26@+? zWjVg7B&fw1b?47rKCE_G29~Mer~>JlIhTSN408mYWGCS)9$?+_nI1d)$(yn}%~RI+ zks)cYRA890&luoJ+3b@5qIS7w@Zi#xooR}3x_{KU(83m+Eo&!t)5c^d%jS6Y;>>yj z*b-w04gX!?^IgmI*|S#+Q9Z29oz%gD@7yw=euKqiN3M7+RoI~#;U&u4hHL&&u4Z2N zTGFrQ4Mbi%FWNr1bVac|oxrakdkU2K9Nja>@u3j0gu}R+tzXyy%7aI51;UDl7Gdl*TS{)O4X{&%&@d7a={KLMUO0l2=mD=Cpb*V6*y7nwWv@-kP7ha|fZdxo!d4x{ zFR|1y_dr&WjpQGT%U)GR)eH5)spSDB?4L=JO|XEPf-}gb$^5Sg#6rO*F1tzQW<|Z%XTbT3BsI3`I0cN$_*JbhN))^L{1KBi`)69- zSh}%sGL`+ru$@W3w-G}3X{7~<(#gU}2^DYmgpoRbs#>s|K1M5kN}?-#K({F`h2F~8 zhQ=y=6V#A4`w&8P8%XBf8EZS?6Fd09svtPS0!__^WNG@p>pb{oD)Aw(xv4eeQMbq@ zOr0bAve{lI;bhE2tD@P-Z%@-lem>w7u1n+$5<&*@BWOK;>z<3a)KR+wS^H>Pa;Sa# zEz43GIbaBZ-UTlPQeQf%GnPzlE1$7br@n=ttU&Mg88us$s#fW{#snVQc!AC?{5sgH zX@ruDhdoalWTm#>w=NI2R9X{!>b?r>9BC&e1RB;%YK6B4pYG}QcHM|lxQILZ>kwAW zZ>Jwf%bfT0g{6uYt=vbCONw&FP3yir1r3EjsQ|zZ0kaeeDmg-o+rHV=L6{WX{RTPpC}M z#7C}aXDtCC%UymhyJ=|Bc2&+h6sZI05e5q;x^v!u=ZQH$RKF|^4?HCr6w8{V!Q+4f zode%XU35h{S=mRI^?ILb_s=7NegFiJ5mv?NoXhYhLEuz04NTsZ^7j&2CH;B{w-B`{ zl2^Si7cYhXK|xw*?3#}5FHa&bJHh9-j0Bs=iBu)oX5o1!Xeqis6sDbNoV0i+fX@E* zK9gkeew}&({4na>Pn0v5=cJZMSm2XN_%ae*4fvnYkc3TH)!6!aev1$Njfb&kNHS+% zV%Tb4={7nx8;@3=4MVKVW}0v;vU_A0gEYvE|B$}mm2qyI9Yp>j6BYKzUuvZWZ^%75 z%8|?XB=X{T(CbGk$Iul)R#KV)I`fQ}o%c%{KqO+E!Ce$p^c6S2e;s@0cs^}Wjm)`4 zI=g=Ht-X%1LMA*9rf@ECx9t0;?Zf1=`cr=nxJ z`ZGhi#Gu!uzk&_3nf?rODMm_vC*p0~?3F9yP{Ysv{9yD?_L0lN z^jVbt5h;WI^uy>FWe%c}8M>=}(xZlJq_6Bf^&N2zX$@!+4}>Lf=cCPcNnHB0!$a7a z08@)%lC9g{cGbcBLIgro+JR@n9hC=G+=eil-lN@u4V<5XjzV4`!K;LXLNsM7!-jUZ zHwZTD^(83~^Thtz?_g2FCpCQtSU7(XVkygpt z5^1P=E^#`jey(S5ZrEy2c13s18`Ui=_u?LMPX(AZ5Ndms)%^7^XEBH|KMY#?bhmR4 zXIqQwsDSV=jgk2~kEerNSFQ7hzu>?7x`c@`Vgv~6Me^R;6Ta9UzbdB+ZC;687HOPq zF$M`pUSWB>#$T&z_xcI=!&>*Cd6MU!L#3{g=kL0tvRhc11oEV@tA{@9lz-Q~?Grvz zXIX`x6VCsJ1a#Del5oj-d^T!Y+Q+iR<80*tX_O^2x>aesxquM>AQie0cEgJTflw&{ zagsFKZso@6D%y9>7~5KsD$D?j)F;%aRZq~M6SVtd`>^8nzJITmB@nC9sqO0TrvS8z zMWvlcuvv;JrLhv-3{fxgq2{e!X>1f}VHfcidmSD)s#`emCnZcd(V*SmPmC33dib==`j%ntOxp>Vp8ntuH79QcfqjpL6n0#IgXrwr)~wk4^)GX46` zTcGs7m1sG17w{q0J&ZX*65Ebt4`OqMiT`dP-cR&_sY}}NyUxuoUkn>iVHnao-9aLC zM4j@=9tX2`RAUO`PVt6&eB4XsrU_K(@NHEN3(~1>8GWvD*YCv{K-n*Vao7AxgnCs0 zKtg=>P&GxO9!m9;*(tNT9W^3W|L~?i3a#CxyWLwGd;w9rceKeki^Nx*6 zp~8-{35L7i|L(J>=#HEYV{ZlrxEJiTkDA96y@#+DW)jq6i0N$M<9|Md-j8lE9Y2cQ z-04A+7Jn)P8+5YxO9y*tb!W>H$R@dQZ=x{&nzxE&hx7J@l9PyCHS^VaSv+cdmyQpO zaW&42JOk@#kS_ckzj$wJG0o+S58*<8$EY4|ZTjByb= zTyhXoj6UbnlUzEn#rM!#_=5{Y)af;k-k=39>09ROD&8()VOKStHKDt zl2onux92=czny1=mU$90dE7Yi-qt7@dp14vsnJg!C%Hq_jN^4t4e=|3p{Je&n7nFS zu)76S@kho<)8yUkT?bUOoq;6pML{3n{qOAZ2uFZp7_`Cf3#|0&^8TEF?IV6pZiB(~l7J6PhSt}(VTr@#g4oay zHJ|?4Z7gqNxRPPJ>Wss{hYcAWs7HyTws!2?NBCzoQ3F)|QNlY#ZkK<;k{*EQRuHlC zqANWUABu5hC`IS{LB_=%pMn8dS=wL`KRtmqWNXvU~xI9OncmVmkzN7?&N)C(CkrSN2S zp8mrQn3TQ*m?e=A`_z*>=Pvqy%XL?U{3AXW_xg9U9xiiRJ%$5x2iVQbl&6FzquT1z zoH^zQ-}$(PL?kZ=A%dcPIiS#|89lqHPn3%>zV~C_-vrR@c#pZ*S2quIMTzB3?te_|#}UA!f*SKe%WsH^$d8GnNj_ z>S?!Cs5qfzvLs%Vd?laT^YFk!8d#?TXe%?5yEBvfdeH+{2wVGwPdk}?!?M%kvH?$t zLPFPN-0l}US~DZ>hTu}goIC|_>KDWLkZaU;@tmY_rKKN*ier5HVD0g^!IHS_72JJm z8r0h}hRQ9v`^!E=Hj$sL(&WX)vy7<>d9uji!^trLgWvy+r&ZX^jfjw;y8maY7ophZ zOD#Jwf+WlKldvW)!(&T#Y}(d2g8q&$@L=u-xEH43pQqGK$#KGf7Ot|A>V@#g0!5}y}0Sl!66 zz;!@Q(SkHXRCYE-=2dEhROsh|-1<$Zj}z98-E#sdqy5K&gU{_{T_VX4ZU90%d|f_u zfe4M;n&QwCKJCe9ZDugZx<>!I>w|@+^H_0z?CsGBAP393Xr;aR^}{`=YVY;xGRVXP zIPnp|?hICVRJBEf8WdA%AJ;PXra!!nAloHgD7H{vDGF}bopLWmYp{QBtc^RUgG+&6FPCwho zP2>B|zkbNXt3Hz0()t*@F3>LePYSgvz#P=EXr8-B66Irtz~Gv+?+X}mc;Q&r;XbRu8hkvgqkuMQ>1F3V0q7@&GRpwS-5x7;0}3LboV`KvZC3SH7P zfm?iN`EpKPA<%)*T)f+VJ}xJ`lM^C{c7m=z?&Q0sJATF`qDTjvO({U*)rgE7`)8ds zX4J(~LdnkZ*w|P~=_2m-T6hN66!fh0AC79>WfGT4{fiScGm<6XOea9A@M7^>SQ6#! z2ic!VCs5_v2L9F@vMpv|T$)+JW`;AnMETnw{~apOmf~bc{(j?%e?1taJ|^dB8)lr- zKJK|hWXmcWkZ(NPW~Fx}*&1+_6u_$D_3QcP7TBG761-;+UFF?)@Tsb&px-jwKL^|X zZ}@OmKqsi~XrZA0=1jk57>Vb;Tb&hvR|#C==d53P(-;Gu&AtwxcFC7Y;m}4a4rA}V zDxM)5`&yRxng_~S?Py~+%pFAz1he~ZE12l}UpEG{IB~h6whL!{Lz6^1!IU7pdMJi& zA{p;S+583c0#-;F1SkJo(EC6ZGI!LV@U=TcyBZeBEcZ#=9e*B9`HR{`x9LF{BE8u> zqN3z|Cf5tIJE1QE&TRs;+b(-J69rqhmC*I492nZ3izAH2Ll;d>QuuLqB;&g<3DutM z9BKnxd$IVh0_a$9X@IJT6OolGW$mGdq_lkJbJO@wsEJr$we!fc>S3Rx{#(r zse;$3R&;lH$|&Fulq35U@}Zrz%$U!GMnH;z>wxzwj$uLBeg&}asPVQc2?p3OAgK`z~`tx-D>%eg}Pse)Gf1b87I zcbl`tKj0*7K&S- z*P5x%vxodJI@cfgBRO>FgoJ1(wLrkbZLj69&ER#W!C{g!yWqch_FI)F5v^;a0p~v` z>={$`(rAtmUZx{cUc&gvRF#}b0cUJhkh4EdcU#G8&p-L7zFt!jBZgRS7gPJF~7olULn!5#`<7(SNf+B?d~>fA4mcC$LnHSC7`)N%_`#!{an)~)F4tc z&HCtFbxWlOc_<23)Z+EY<>jDGn>v~cpQNR#WY9+!;Cq$|UfMO*0k@<16dX|==d&MG+r8!XT&S2v68w+2g3s5>?-zC;H5QeUBsnsvoC zpvd^bsf^Ab4`=_Aw9@6ASupj^IvV_7Iq3|6YNFpFH%y;&j-S z{&Qrz7{o|oc>VfDqz&?RFuce({LDersEf-I*fLR*<9%rrqr}WQ{j2ekP9_P!(p~z6 znP!zBuzMU!Gv9_HSOG2nmHSyinlfp@%vArMTl997D8UUC7)Rvy@?z5s=0b>;9Rn`I z0(1!^>>h4DjxQcFLt?<_ldhgJ)!Z~qJPYj=0L$|p;f|;1gWoL%TstEk*jaD5aJ{Ay z1vuMpkiuuWlq%1&Uiwc+Ap05ca*yRzyw&-CoY5(nrZ-=je@HX#jc!G6TccJDuRM(N zx)>hSBpp0yU8)5O{FAAd_AO@*EQWGe$@K&D#hd&gJXW+Equ804m`d%^0{?-) zgNnR*6?zR&RP}BTc~m0|Te0yhs58dcQM~ME^h~V_m)JWptb&LQsI;@u^D`f?#q9l~ z5XokI{G)_UtX6ERjrt=5mZy2Ie5rq${bP)6XH+YXXp2-yebK1~7#Qzyp2$DIZ;B|;?Z`g(aVdaL<2OUrOku})yflgw8b3!xPllhW z9&1`qmoR5a^;Ky>Jn<|JU>MzK;TUJFtYJPL8K_54ScJ+li~X-IqDbsEaz6~NKuY7I zJVMElcmoV7XGr{a(a>u9mQ2NdDW&U!9eF&lm?AW$hJal_6D}JeyB)wlkR4E>6GjJ z!rt?vaYV9rI9dtX>fIX9!mH}bkVQq`mrmzKRr*pI3jJ#l?K@CYFf|#sae*{lVK4jS zI=6CYroqk_wWk{e0gae-R*z(={scQ4u_tguka8n}9MUUu7Jnc<1Qe37Qw-U6x>76w zH6kF#X83SUU>F4J$zG<{+IIx!J(6>`&XBfE^vkvwD4ujOX!QERCrStyCvIdq_~y_Y z6+$>zeJK5%uWo;^Gz!8F`^L<~w4-UJaku>el`zY5Do zxiz{PS-BZO#e8wXrAAf^)tLKr;H|3p&7P8{jDt=K&(*`QDB4f+IZ2HJL@(kenP@UM zJ7ha@w_9vC)u@)C0dCY;@|-=FKgKMxC*Knd@n&_0rUdb~WjmxtL9`iyX%J*>VmZHl znH@D(yJSCd+wC>M+u%%`UNBL7kGCm!#rZa~$IbKX*yfe$G&fMe?`ufTcPV}$QfI9g zrj(~8sIZDBw+!7Hxh44tXP-|b7IEhm)14IluNs4{$8?g&<;M1uxk7w zH+-Vc%9$~d&}&ASJn;F8SH8m#284yBD*zudx{#`p@; z#IT8wwVHmOZZp2ZCz@ZHST=Yg`u;amP;F=SrY*KXC_-#%H zn`F_By^zwU?ArPaa0Dj0e|z?iB%aXB>=?>vkg!1*p_wNuxI466MPyX1-wY)bwGO+g zRmSPD$#?1-N_ok}JTY^ZZa&}k{i`A73v2v#+w5z$QW5ud8RNGhTyI?8s^=<8ij#hu z-}B@-`uo_ua8%6&FYb5Y7oWs77n4alAWiEz8nD^u|FbG4hVhYC*M!8N&{QsSjlapJ zrSOuaqX^wwFA5b!Ma@Hj{WmIyn7dxf*%`tBrA_g#<04+9rP9o4GvT!3F^%apNNiMW z(#r+g9f%dKFW-BefGq{oNNNC(^p;uaKFH17uV36M6TC67Iq;2c+bg%UeqH*R%M|ag zzNj)CipAvsH_0g{s;`8(C3xLx5=hVhsBBh3eXJM{LJOzYwOSUbp6 z9j^CUuxWWCXV$5p@q9DJ=Z;qCPDt}_1Y^T-;{bc>cTs_+xzOv%Hm;CBX-C3Lckb8H zbRMbwU;_WI#x5yiJ;ul*Ah?~MaNdFWznA)lzqO3fk_pACuwJkCT(RVOgjvr_zXxso zyj%-s`_^r1{*!x)++NGWy)pf>P#jQ--$z)G!jcsknGCq2E zc-&~~Sp9nRUoC(%M#oe%SYDI1w(>>HiaJ7NGNQ+sYh8+LxpBhU*x=<$r^%;3xU53= z{|vQ$TainDI+X{wYPKJ^RSmy*-Z0tZGy4asES4Y~e^;eV-P$2Nf}03y+Z9A0M(Ni{Z0ej3I>pGPSrw}|3RTtggQ)-1HlW( zr=s;4xtXSBgD*j^#kaX&#hgpE{U^9|9>i#)-Xh;M{@vYL;hj&OdqrGgB*J*ds6O(i zU~Re$j>vl!da$Lae-e4T_F{cog$bSk>{|28ew1|fJuOvCh$Vi6Z6Vn&-KEz5f6{QOGcw zDmW~>a3+zQ{LT#3^~rCm!56}zuy|v4(UKAR5@Y`u65yMHemp8YjB~F|Zq#0{{X$9u zc|5+%X%MJ_!m)zFEn<|<0fE_xy)6Pl=?GC=Hv#t;)Sbg&+Y?9HdOA9RVwWE-Y~O+ei{S<$h`?a<6>> zrW5r}00Y5P?=vH(Uk~AqJHkG!r||O7)~{bZxA+Ob5QJESGcmT@2$@T+y+1p1&%S9! zI@`Ao1VCM|(rMk@_c3DjdqOhp5j|p#R#{G+B=W(VFZfBX!o_+ht7Mhg z|EE>*Z@a{j1Owm7^fl%pxokuZUo81Y4~aU~N75sj%%PMntdY#z>u(8E_$HJAoMV$6 z=2d$4j|2OB#`wzbZ`OH>pD7vJi{eb1t;$c*S`hkyXKEBw6=*HuxqlPw z6f6t}HNgSK%hMXhZJXSmndm?g z&=A!&(1q+xRthFpu~9E48n{?d9 z4SN&hyHO=>ne4=9EV~yeI4{#LRz67V{tCsrHp5Zl65I)bh1 z1IjBtyjr_}0n-*{zT#pQzMaEe1$UkDideMQmdgpYyb5NrA^p9hApG*|$F<<$(v%H+ zOJ9MEr4YsguXWmy5q`3jAaymRar%$^0X(rT6}LD|+kfSh5Gzhnd6-^TN_8!}1>QeCHY=i2M@#OC{%#8TcP4}`S&Us~}&KBZSb%~8bre-9}U2sWIhuIMV5?q>D# z2cC3v=~zrAwYbVuM6&^ zDhed5^4n0B^(;`-qFJ^2Xb``~;BgN)z*ux209(Z?x+21Z+>_95!MdX$ zJDhWq;&fs(`uX$cAokLY3{2scN>WVDP7OlOazIJ0A9<+^pH^u}I; zE->Y?!RIyLx3G^#k1x6~q}!?0%b^c*?uQHe73{8&-nI?L%_7KkbO}3nNNr|~G%Bvq z{WwJPnbrP5wXQ!5Sn*V~x5Z{jlR~2%-WT|O`nk*E+Q3{{ zcKspQynxVKt-IY{q@P85uRlSd+_VBX=iSEK9y05mkFw{AY#A>?Ox>4sC21zBDWoxI zOLGn)`^1a(ssk|0aslmZ7WkOiul&FhjQii(fw+DObnxfk!{+<#_Zlj6sPV8we)}2x zLr3rzpA4wN?ZzG1mI$yL?)w3=dqS-;;C44`cg@OYa6vwO_Ph(-3epdA>P{2ge@A8? zUQy5Y1$F2>+fvfU?ZOG-rRiN?u%_^+(}K21524D7HF|7c-;dU(3BQ#RJH-ea%-&>( z%LrCMGjyIFJzSQ7 zeY{q$VZ^dQ{wFIEgPifs!<_%zSmQmizJGseaIJKQ7YPx#d(qtT;y$@*>e^YlDSJ=6 zRYi}t#?v+-d4goDx`bzKtd1w&0ks1nYz$AZV ztc}F=JmP`pZ7hCw3`qhIiB{dfId9b+>1gnN@lvzHomwSAUjt%YnN`~1Rh_}^ur+B4 z#93dxC|>h!X>*dXw_OI-<^o3q#)lgucBh2*7C?NuUK`4HTHC(E(`+Q7vaaP9-oH*0 z`TLPhCbltGgf@L?4|0EMDEN4~(jS!YXdXo{j)RYNs~A4R46{|qJY@%Ovv9Vz4-W>u z>Kcz+L3{}xB3illQ-kWQx4PZGKy%g%41U@0Vx%RbRhdhv0NL^)9wBW#J-u7@7xQ{i zvoq!4yS+3`2g3s08p(5^9&uwc{{7BW8^&s|BF~c_8@sumZL(dhp1&H!*@jclN?6}lCKDn2NinBKoHlS!#=ZUxX- zUSUc~WDWP>Hz!498En13xSB*WGoLyjTKC11M(rbM!N|Bw!ypM^jVwH#0=#iiAxSR8 zwr$!Av4=_p>Jbec&O?=A1_JOC`x3Unc=o*^q54l~7b?rytBb0H5UUyd0%NuePY$9l zs7W_blMYag3afFa<9Z7Ul=X)APveed!~-CE8`2atSw!1y5Bjb39IDSUH|*kw_fg|P zv!W%E4JUN!6)KH)tKUEu0==&89o{y}rx|VsXEurTxwG-xT2#N#!cqtu2cC$d%SMht z_ER&nZb{?Su3!V3Y{?3|N3=S46VChSiy)u)!ll910a9+h`eMacaHV2T!vBMi={>&{ z=*f}_-IUI&!!Om`zlnd+9myv-NTGZ1FCG_@{@+bd_kiT+6h!q z)E1B!+*9ud((SAmPksWkF=fgQZ$-SUEI5=-8m{>U?bbqXt)B$?Oa2q4hyf1#rSXex zjE@5&bIGN?Z_Dr3bEqm=lB|XZzH}XVV240pX0$KX9Qk)b4fs(u)K?=WB%GtU2zE8% zToKACITo)6{juYZU^6KOm4ELG{`!pVHdcY7*pXMJHglt6J(T+B6;}>L$wO@Uobfh4QfEu!KTso1DQAS>?+m~=7Alx(6cgq^eP$6#u-pPsSlc3LJ2>w6joXh%k-QID>}_28#=!`k;h- z3^zeN6hR>34##l9wOC^N8U4&6x;j=On@}H*St;6_DE@p);LMi=y!Jd(cU0p32HDWq zS`NEjI^_7bE6?3W%{vvs8E`c;6eCNkHRRzZ?sc<{APdU&M`k%3^}Kk9aSstO54Hx7fsa<_wye@O$#R3 z&-5~>uIu1bkH99Bbv%qd`rXM- z>z`7kX?8m?TOG)caRZKUr)@3fFslPWl4AR*KZZ0SvlF4@fr2vGb}bO!4F)#n2q7w$ zbbr+Lw64`#Gv;&) zpR^zP9hRmUW{oK@?k?PzkEHTX17gCS0-hfh^dY0nigxa0(d?B4Lg|Hg>rfsr)U_9F zdI~uvh{wh#@O;{)=*XK9@pk#LS5TWP&XYhI+PJP@jEo&5idIX(zVN_N9ZRODtV}gm%@_; zpTuo1kKSKgn;xS}_lswMz6Zh~suN36K6GY0S7iI0qk+?(fV;7Ky=yt0fISPk(jn&Y zi2_Q%p6}6WF@9ZFSU#C3k(XeaI6hxDGuTUJJn4b%w%bS0vKNb}ycw+2bCpKr!K;CI}Y7Cjo`QKYatR}9Gu;F z)v;_2?DM73dvCix`E-#s^=9#M@qFSrq$WWyboMYZ+n?T%*8JI4j@Q`1uZGC6xvv&H zMI#lWsvc7F71Xp1XF=PKKEI(Y`(I*JMSH{ES)yms^sy;r?6*paK_-gi+uMvEwkSRM}N=+a4vkMfMo zg7P^A;KT8203Y|>SL_(CFz)U^r3k((J&sdC3=2cnG?`jHT2P<>D1-OT%ll}?G>did zg78C9qr7)VN}~M^Wv)gC-C=z?NjPmFK?mbZn+G|}_K!RjlA;LTW^*u^X*l?6=P*4f zc?||X#wHp)_1OUKF3ib5a)`pQl*Z+}94AEWAcrlYkqwg)h)2GEzN?j&-EEmv=zUZ! zC_BcFmtDT>Xt{f0o1H6mJF=2*%`q#nj2m+HM+o#+4`Cm0w~1y-t<+PW!pV{brdz^- zsO?F5NB(Ko{H>DV`QmNavcZ}zuVW`MQkTVxLK~~t@wAwWrdj;vsLTHTsIa18%fMSF4fs;r%e4i^YfY+yNNk4z;=fgT9+(JKDOR!nLvv~K z4!g*eEXr%xnR-dJf+Z=@IiWJ1&hTR2?ZoTt2^S^$d!v)piXY~>CUa*wzl)53IH~O1 zosA3&*EoWLXZ+&=r~;OE8xPuskP5B$=emu~LYAn0WKpYsEwyeG3tKBQqAt|i#B+?j z%@>L>leYAF_~b@^%EI~itNon+w~A93jS>7krn|!uC}dM~CQTWM-<>|1dIrZ(WSFc+ zN^*oIM_Xm^GF^ujA>rgdq8toT{Kaqz8G_vgAMfDdtK%~0_VXa?gK44E_^WMF<)TEf z`1sDZa?u*X!nQn76onDvd^Nl#DGpsil$SsbZSh4=6f+p#V41U{E#40K3X0b~w32V) zG@k)5{5pWa`78X9txiBAx8!h1Dk;g2BzPy4-9=olmLLv*u-xkJwX{E8D5E#*EzVxu zxo6eQ`sSg&=_?BAjC>M?;=4U$0hSVXg2BN%OU43cNS|Lb59~ur@pw%2ixFj~yuK+~uNoT4c>OWq<%eS&p94paW*MhhZ|HoIu0}wX&Q5HVU zcbq0WowQ=I+NHg4Z;?0n5noiW=^=G~7kE@*dX1pvtlkEH=K>C7j z9Y1L8_XRh>f1`dPzg#%tm){Z_KD_5f_-&&rpS6%C&SR3a7WU zN?s-@K7gt!Dc1G@E%qNAu%jZDoo{|;VsDPqpgX&@9JlBsar}xP)^#W~17~D)S-A4b zJB_w4qK`i&52-2yld@a&V4)~#jBv7}LGhkYM{6P1FZO!IsSMX~Ae7(MNu{3^MdtfK zd5TS2MuIDOgO(lEI8ZI;Kq)LQ=B%5d#!!O&B+)3nmusxl_fMH368cmx#Oyc47>X?9 z7LqhM5QM9R_3q@%o||H7n=RpQpszqJ|FLhRthapt+Bt!-fN@BoEUuWGK0%p=o#xV7 zmqO}>v`lO&$Ulfq7A$fcrPymNfJr}HP1+P3C)^MrQlQeBPn=5n;$Wiemx0T+zmiyz zw=};UxN@_8E;*LNa_NSiUYk=w(b7M0%wwy3iVYm)RG_tXEDPq_m{2me+u0Bj3u)s6 z%S)&7e1T1Q78`84TTRX4+?E;cw3cOhn?QL|pstQt7qUX4ujyLL_5)Jar3HNefZ^XX zqVg>#6S|)}aGx6UPF4qnPq~m9k*F+P5|9*&Gzu8M(p*}gyYV$(yLh{@vC?}_f|k*t z6Mc%a!wmiEszE5>MQNjEU6fkVhJ;ZC`kd5>ot%rR1SS3#2KFyf=|rBMfHzc!@>4{g zXws;u1_wdSo)xA0V4hS8*xb&xc+;?{>HW;%Ne*mx-fnVG~rXkx3IysQzfroC?lj3;x15-wRJ&>)N=Gligj%{FJ&zj^(nZ5fc6D zG_>w?mF86%A6^H86}@pPn>pB3c1pLcw0ZUuxp;1FU;!cU`v&_{ntJv(%~~Pqo*S*u zi&`H&uw1N;})aLa>M-)7GX~9v>{ddl}il!TSMeG$ZLfs<+pz@=9stzt{RUUl>j!n ztf!V+7%EZ>)z|TexG#c52PtIVgGZM0t)ps!Z@$@t;p~kh%C$KC-w9427dSB`*a7_h zr+e#@g^E6<*C=mM(l&(+)LxbjUsF61eP4I%rDt$zFbhi*9z5U93zJ2o*>rsTDB<}} z>|P7Ys(2&gzylCbJBmW;SZ&IAzbfLTvP@A+x)b4Rj!^+#87eVRt5o}!U^SxhJRRPo zYdViAPbC~{Y7NYvQ?u{^FRgw5RHYTkY_{NWk&(QSwfJpo3VDW&ruZsyqH;fpgBvr; z??{#1?n6@+Eepmw-BAN#tzc&OXZWf=XRe3-LEyc9)r+X(*#yc|iR`DsIbH7;FS`Ad zh#muVqsT6r#>X=K%FBFlX`;P3*Yje$87imUp|kP~l^X3Vn=TpPfn0mA?WAV~&4BYx z!`mb<#csHJ(A{3xl!;$kY*bD0E@EBEV5eVWw4MRHJ-N)iNpA&+IN0%5*x$<7{p{7p z1DL@hD1PKzuqi`3rUp3IMn^pZwaW=$TtM7?>}xK9FMyr=bXiNyT;&Au(miH|tk5my~zs7>y47N1Ba**UrxJ4RfXQw*j(v z*X6?fg-+ce*1FLISIg_Id`#xdXQ{x+{vq#$!J4%Vjpjh@bJj7VLR{m&tc!;_1@?m1 zh4{vGaiZ2f1DUWgJXrkULJTmrpsr0S?I!#YBDTWbFEL4`xeh@-(?hKLCXG2LG7EjoGZ7x z&_WxY)_?%!$SKyH%~lSN4DZ*IwNb#+T6{_`Ik6N+w#}`RFqxUSBs%WchM!~Z5;Peb zIux_EmZ>${6G@Jii-Y-53U`LghdY0pga?NY8hIAQre2a+FvM$}G6B^EmDgs=#C> zukMsH@&hPi$Rb^crwGkZ^3QCz;hUKnaIdSm6>pul183|r?#}}ghD5D0?AfGOyfT(J zt!g0yCNm1De%|;xqHJlf+?1@040?7-Na6g=1>(2Fn8*0giTFKQjwB+bOYt}NV@2{} zh7NM)qT_adHvz-7@_#%Syo2+t42Zzg*?aIC%E_p_v0g=l6RPY8RQOy%3VSDapHKZpq8s)KnC zn58$%fB!6QT+^S)EX@|^==@H1--&uD)dKolk{CmU|NYwOa z(jn-&WircMq4ocd)g&5q^cFH$!;}BPPr!nsN@GLKtY z>l9byiZk!xAo*=xitoTKRmIk86Z6v`@A*o9k)7tz@k0wF<4DXI3KUDmvvuL;k>bDP zU@zO-g1F@+t&FXfJ~-WQdKotN((|pcW~&jQsJeqqbnBs0MnBzK{MLW&5`C#SVme4l z!+aCr*Ni$QJVA^#bQwe^c@?|@Tp!ckzAp!9dAm}W4MPXjzOP2!n)?JLXdZpM9~6BCi` zF?QZ7T7fl~2gl;HY5F<^+ROh)A{WTbkKaaA^B-6!|D;plsQGrAbNk~wk;G~Ld^W1( z&>Z&s$J{M-cA6Z^!9kdspiAo*qWP-XyE>=*t0Q5f$9>Ym9n0alaq_-Y$_k%whfRU$ zS)k{3Q;Vdo!wqKDqsfs+w3bF&m#Gm(1|b_mhq?+M%6iqW={WHd?i%j&ysugBeNFM- zrR>X#P`3yoRamkq5%plZC-aFlB&1GW|Tjy9O)z526&v zB3ZGL^N-NzTNkMByRo)+WT^HO0r=b~3jS66kpJb~>B(v9VR^EZ{zE3`GW4lKEaT;& z+U$YaMf(k`z1G3lrZuA0f2YddNLWgB_N5BgktGuhuW55{o=4k5k|l<7w*5@^{QszW z%dn{9H+_4CLAnI#773+8Iz{PHq`SL2h7cvByF(i3?vn0qn4!Cyq5i{fpWS`-n3ufa znD~C;zRvT4cx)|<`MAz_?OqNQwCztx)|XFdn>9|u2Sz)CH5E>>R3g7s^$UMcvmbxx z=K+YqIszW3MGezz(P9plW9HeH3%m=EA;T-C<}T z6z-Cd2SPZ-XT`^zgQT}&CYu$`UBpAkaFwwgGvHZPRhKFh6UViFdgR5aP;cCH6 zYV0CWZ3NYY7=DX+_O_p47lN2ffoZOO=7YKLwS@6RoAf-L6VmOGUc))cJt->x8$IWwsf}1J+3p;TlO?tQg2nS!x zYYCK3r$Jl(*xvfD3~SRR&m-pAFsen)t0{N+=mn_nU7-=!~bRq2*4!`*-I1oa18 zdp$IdOgg+tWfAEED6}Z2%6hv|!SiCTtAsOEqP>MQJ<1QG_D;GSSg=LZ}0t4fee|70T zyxLvjyKil6P*7~R+jF7P{Us)@noed0aoR4gH{laktk6Ak=Um=210LHf8fwo#Mj- zy1PQgD8wOt=h0ar-Fa0RsaKuy@T(fR&5#q@^`yQm4bu{Q17~Q1viOztE-JypLXg5^ zGkR;z=V)%Q{As^D;RP_dx}EP6R(*os6qkBG9_X>YM2D>)FB0|hO?r{W|3q`ubX;bl zusq$<8O?d_1|M3EFLF;%d*27~Q0XSbDKk5=sG+^)jqOysGWpX*yegI^ zKWTjk{EuVj!IFa6PZ_&KL6Mt!^(9HO{GoY5ij2l^6j?eQn&kJk-?w6=}QG`f>)@Tqnx9Ti#qW1+g))x ztQhusqnFgKg!1EV_GS?@za}^NA@nRw%wwxNIig+gbljJ~q?O=A8<&mT-W{=@N0oA- z_iR`lQX_eXK?v;?`y(&VCa<=a%!3DltwTu2mOeYX=AGu2w5xWLkMlH(>h?Wq`?L7z zBF~#DgeE2;Za3d25GIDF{_l?KNP4_45zUijL zWDxhGqA^CS(>{T(UMJo=0z)tT5C8h##C`3H7W{LdnL(A#zG#PC%H?DAr;~DORF3B! z{%cm=D#6f1Xglp}K7H|LrXyjRe;Kj^Ex#!9^;lXjCl~4X3TfId(oeKsz?ScJgzZ{` z@;Z{b8+^X0512J0^CONh>dUS4$3zyWzv-k3&&9s(uReC=kp zcqhA1&W4ltJjU+x+`J6O7397+X`QG_jv~EHvwLYpE$Jy(7Q%{@+nrK$lceW9ad0nPJA+2r@ur2#Pn#w)yT2B2nvs(nFqHJN@Ck|RJeUS;P3!Kst7lhg|A zj%DfAt4pKph^1V$|9deeDdXoKi$t6gM2&Q4$Uv(eXF{YIMr|^wj?2gxF4j|lJ3d@> z4wFa{6w?XRnw;}N+Ts*4@>m>noWx5r)NkSrKVa0mleycK6wz$On)xcc$4+!cV@^7D z?w=Gy<|uWwsD%fi$q$6r7U+FY)AcZ6&hZ`;CwDo^mIs0#-{x$xPMsko;f*WldA;=2 zd@mT`SEDzft4mn(N2X906S^X5H6H33TlAv5?LfBFt4#&6wme>nuf$1j7c5st2dO$9gFK-0^v7F*%Y7c$j0b`z8`G4>UQM5L z_eHsMa>O%H_j7hMnOKLi)->c2veXxs@BXLK>sC?;c>1n z?y(xPP9D2QI23ih<*r|T+l063oT=G?|N9x&Fhkq1>%g{5q+k)u-asU5!S68Go!Txd=NIdKLrQF_Fu8O^||T=2=#3uTnQ9%_wzY zm-314WBId6K@L0szOj{WeJ#JxVv2$`Ko80Lvo)?t5(f)C*#znd0%FUowyq>8`@d)U zdETlJmhZN~QMLCkp@zYiSwoyvS!4bnlgGEQM#sD628p?vHR$6Cl=&r}pI0PTS00{g z7mM?FEr7D0?Jwe;6T40@rEWb?1yj;a=c4>PV7yS)#aWqe`;A4_HO2M zL0R?DH1i;T4fQQU^j4ho%JPnuhxf>~B-q0J^|&+hDzV*VI=6cc{mFnb^Ze7HGwv=( z&h=oJ?IY0*`#Q4px!l$8xWCj{U<_)r`SO4@5{5jx^Z8y!ZJAsPrfLN<JGbd1a4YM7z3SOf zbN#!_jo8==bJ=}}%u!MH>YOO7Uq(aw+azk*4qvY9jjZ7)6!gOj{bTtE-x2t@g!AYc znX~;ptn-4Ci>jbWezz9@k=d?Z(?)Y?tY#`Fg8wgxADo|69D1Q=7DwTuQ{^S#F>rqq zydwZDa<*{0?xP>&O+%kb7aoI|n^r0~%HTRu@M+ubDgRj48rkTP zM%ikg-0QHi_K^}UkllLOvHb0{wycD09-Io?9rX)_+dZCpX$_q;Y(x|X3TgEB1<3PG z1K&YjVAnZ8Bw&pwKsnF@1;Mm7I*?-lu33U_G!JB`T$;S5z*rqQnY*a`=bN#LWmX}R zwC9;{7HyWicCQMZr6Fsd9YBn2dhyF=pV z@t;|+B9fcPsz`#XDOo-EZT;Ng>nfr}F+D_kRrjAS@qI1H&l`feXOfg-iRZVGO~hBq5-y+q9Zp7rt?3nE}X!{g9pB)|87|E%6E z{mXG;Kb3i&#`klC!Uw*8CkCQgULYGFBhsH05`PEoyjwV6e5E-DbDymSc>~|XPpNG^ zZRtJb)j2(!^{KJ`d!fkiqq<^8_}pPd1vXXoy!3ihg~Dv!@F+|{BA%jPySDDQP<-A1sv)6FVtM<$O8XzYW8LOoZtrM@V6!+eVjdB zP)ac!4kVJr&p+91II&xRZ))iyD={9t;QXKV*3W{ta`xD{cz=&422M3mGucnfL4h>; z7zZQbZnEZe9P$WSZ~gCjj}xX_^9BLU-=K{JVEYP0099nf4D!v#M~MO$j$!(!EK6hV zzSW#t^U?}ht9WS<7{8c_~G&FRZ{< z(_*s~)qFz*bNaqZxJ0#nXZuY-x+x!DP<1+p|h5Fru)=PHKvd4L63+XnT#oQHoV?b^3J#I(jE!7bMB!Z z3^eXv?;Y-5D_bY>}I`Mo@EVEB!Ds@(q2{q-Bd>Qd4 z`@;_`=@?CsZ|~YtD68R&Lpl%k?1jn24t_Bbe zy`Y0?<=BB6FC^CV`wlBaQ_zSed2qY!~!%MKBZ>s;##Ucig|=DrjP zE8YL{LlTKC2aoMV5cwaF)Ds0^*u9Of|L5znvBgEZ_Q$_sgP}-D1LH`y?4!!UVX7UZ zQPb!RK5M_2h-<@a!Lx)EKDt-~n5=-$qVLQ5UpUO#a7m&4ulTt_kC#&iCwK6(OPwb) zY6C=Z>eu9foRblR!GAZY_e&tkE@O;v$BtdkNA^YDGB1pq)xN8{O8kh(JGi!5d2l>0JdJjJ1k5031Ngu+Zpu_{*gF>Klj*zb`W*U8iA`K_DWfq zxkVTCzW4Q8@TJLa#=)6>1r(JNgg5+W30!l^9&&SL1b23zjZRoOKfD~$WFjGX)!?wo z1HLGntb#~aj~%`vBSa-d0Sh84Gq0Zup2Jtx;0B?^g&FhVCopayPVCv@)xrKPv0h^~ z=&aWeHFOUkB+<9;kq|Q&1-?RED-g; z$`lv5jzV9TWV-jUQp6>rbYL<25syg((gWfQiYN!S(NiuiKXjLaq7om!>2?G;?(ao9 zJ-Qr?iy%1;%E+1sxRx`9Pui5wKEGNYE{Y)9 z)>O6N1_pGZUgCMxjqP;a#J0BXj+Oy!&W-bNFmj>Mi zw|WCQs?rEQ?_lusCIUn^zIKLoaWPX^F20hxTIwfE&W&aj3X|ShbY|#dkfW%frW68y zh46a{j`*pc69otg(EUR}(vdzYj;sc(qlH?LHwQwt#V`Wx=y#n07JV$>@jG4sTVwS^Icw^ zJ9E)F3fXg!eyJoAEY9dl_^{D+Wk_ujy)|v~AC~!EI;&h<$>pT!#LxXQ`F^3+D^jh)K6$bf6;oc~>W}qdJ1gt0DH2oPYqQpEWF7|b zE-glEx@mlSuZGaS(>?EeLw1p?QTf(Y0}H#YhzOZ<)^)7NFK0VWE$n)Uh>6RLC&)2H zfG1<&qMv>UG_}keOPY^{qe6tv+8-SrJ!(|m9B&EJ_C0q?VN}#ye)}=$?Mf^2a0_SB zZvlem{8~J!bz4T&gz!rxcPxZky1kMzn8H2JmAJ3906hz8uAF}G$!m^SMi+l54v zHlMZki<98LUWY$$PtO742F4zZr*GeyV#r(|!~{8;kYiMMpY`)k0u@mC^LV#XT~jIS zTiCU8#u(yG`m@N9Vdt^fe(BtZTh(CJPoq8OoIWq4j}Y?K8Y+2KqBr&4K4m~cJqp{H zrec@|6>%3Xbbl2U8|$%8oFvXXS_8>LTI6n+cPRmi?e1nS`G`n)CW(ar;ww)_$1u2? z^jJ1&A`+@`KU!5u-7Lr4asSra=<`0cJ>&_%Hz5s~nTp6dG!)Zt*+v6vI&Ib0ynL5? zXrpdSA(6*U7MBT0WifQFV;W|0wtQ#Zc*SOAUU69V#N{Am- zZ!p(KOwpIT{-^CkCP4y=|HEUt)7vNQ%$KLoZmNi z@kpucyj#Yz@*?4G!Pgf*F}T5fyDgLhg?LzYjq!{K+GzTA&Q%w=D>h7!=yiyuifS*e z+e+V;s+9zs6zadPqP)s~b7m`_T5=P)qF?AV}To6G=NPD3FlL<(0Q-2jSE77m@O6 zovqwIiM{S5d1X`P@w192b2QjBXS?xxTi zO6W(M1_C(Hf0Z;V!Oem2om9mY?k+UI1A7Yf*kJAwM``SlX?w=%Mq|oqrSH$Ms7(S-ZxT+u+a6TiMq zkNUE>!%&2L52vrqaw@4|a%}vW0lRCC&g+}Gz{+3#tv!pB96PG?XZ*2?=KUhHZ%`+i%L7a^lwIG%O{$1 za#pTT{F%BcZs}J{`kjr6{&(rGV*#=fTTsETYD~+BKkIUwf9W1V>iU4m%gTg$%j?z; zT4ryV#+w|kItzSQ9v1U3iF0GQTsr@^tCd#%nA`|%~>tPOLuRy?*Y_s)CY=86=%2ccUmZbiC2&WTn>H!heNdorGy>;({yv*SuzP}Ft()+AL!}DZ}j*whsie~XMk+iz+(CF0 znRhZ=bdv)Mezhc$tYcwka*D1EH@`{&25m|}0?@~s& z9*sCe;dfPQUp{*{w0ZYC!7aoZ<|tvsFXNK3X5-)SwW&mbKlso69Rx)YYY=PFchH4> z;_$8L{fpXO7d||jod~TS3IFwi)(-7`X?4?zF=i-uXzamzjO(JWX4&xirL~jKd(QCl!Iuq9 zf(ufCkgK*D6Bq~ccDhKi&jl9|J@ORit(<0F7uVyhkHQ%5U7H87_qNy5wNU$g#FDiO zg`KD7;n8yA13I0&baF?qrKMB78lmkJ~Q(826~@{XLp5%sZcYEZtHmCiE8+NBR|0n_+_= zW%syv<#14t8CniDkUyO7UDH)JRt1th=ODZ2B-oYj<81LC;PhEGy&bpI5Zb9Osh>`i zw7k9X*H#%fa(4c}D@TI8K}NLG!HfK#_mU<7MB_)b%Kr;ZRae~Ft;aWHMWgg)y>oZB zS_$XD0MLzQx_|IQ-pu1B-XM_2mXi4NUHi=SG*>z0c$A_}m_0avC37Zze@b+0U`Z;E zGLt^4*C+ff7na%r)-S!NHEK~oq<|h&Kd~9Kwus8?LgaUUuj!RW#56u23{@iExOMZi zxylYFf8@~x&sa?sSzT?xlK4F|hco!-Gkh%NNh%brGvnwAA6r3Zn;XU71TA6}6fLWv zc7;}Qpw0+TdZq7)f0D3^M3bH+AUGfcSm*SvDMs1!w;7)UQaWMSkhPGsfwSP6ph!h zYxuBec`~{5PT^)TJ}gxnFLXD{vNkepk7l?PmdW>3Z8}a5ZoyBNv1SmrRhx_=33w^m zmbm}XTcbXck8k@PXpf&BUCw@$lZ6pG(e+ji(c~+JnVTtW-w&Kld~LHxiJQaiU>w$m zWeQ7?;JtPqLiD43D2!{)ZlCUQ11MN9>{xtah9+{QPEcYclq%A1D8On}amnoCm+yVj za?L9K#d-HN_-wYHT|9p8%NgU$TI1ZBirgb6Yf0Un=S?QqNRFU}UdVH(!lQ?yiK@Ih7ys=`}&F$gk(;Z#9eCRJG7jS(CSXNE}3nvgv z4W4+3h{(oD(J=;1#b1nG-gzCmO>u-*c6#xVXPW5l9LzTi&SDQ*gwJ)+?6TDf;D3)A z@t9@e(c6~LLVL5Htj>|#zm)SbA z-@c+x&l1+E_OyYOMJTB;959Y~e<7>%JS3~&O)Htcl5NKaEOG&^?rR&1K!_)rvOiR%PPUG#b`pCQYeZ%j=NCzC zYc5AMt?Co@C$HBBf7}W8X^`zR!JQ-@yl}J z-**?=pKmr-zST)ffW%4MKoM!CPjl`yV zXZLQgYmfZ_aNhyfg!rc-xJVvT>Du5{SAzTCJ|)}W#QDMUIQ0BeFOM>CHeMdPonZ#{ zJKA%QcM`Wpcyr#n)y zUp_uc>sEkc*x%p^bAo*uU;cZ?I8}_OXO8$b7m~~)PL)1GSay#2pUdX!Ab$Ex1P&u9 zs0AYJB&mL=6Y9!$(f4m|INNlC(0_Q<_oXyvL<+&%*r)z|YS{oYW;<9IVbs`HkU1*7 z?@-P?F+(k)RECmKSgy+n%`f_#iz*ZUpT$V;6Be5}1F&rR<2XJC0|pY)FM|5d{8ail ztNNL2X-H^t`stYSGuxv{_ozyhG}n-M#uNSq=MqcgCr34j1ynx=rJZ7s+_J8_0VfDk zA*qM@>NFkH%J6ml9liG znb6k8^X34a(Owm;!^z4`cV_Fe;)ab~D;?&i_Qw&so-hqgpdr$Fnb&bx5Ty^B0^iYe z5|_ib;S^3ht7;g$Ben{@$#P@0a^p$+GvqcDyG5j|X?>8$c)h}G?s)g`IJhig^cKdf zC%dK7bV$UsURU54y1(u#VpIS{rQ$W;9RKKRJ^NS3 z4EXN#er&?exJyuk7Cy)mCh|R%^i6^YTsN3FTxRFQXgibf66v+=$*LcuZ2~&D3N!D#id)m;_40@ zi~SAX`LmF_!r!k;$)I%^#eAc~`@0owVT&<9+T~`Ians1yN!!lH3##@FC+LL+7P08a zm}5$g_2`7pY6_q=aT#2sg!<{7m>g=Smyc@lW^vh7z&mn_bFP%YiE4`Lw%C!}Hzl?i z3%0$wK@-G$PeVGfPuF=}5ZQLgB^`t-cir4fVnf_^qrN)C?4knKV+c^aZyf{1n(m4^ zl)$V-(tgq0PVRg6)EnW(C{cja8Y4toLmWWvAtd)dcq+_-D8J z36>b>{?sUcwtPW~Cq&9!)_2u;v;HHL#n|dpdbg)|BY^U@_;zj#f&QSsGHdM z7@&tzwMv&45sr*K4L%iDU+7N;JU*(M@qqkLG_$-oMS9QgOztGX54=x_QrIr!tBUXW zn%;^2jJ};zYs2$>{NvASxfa)OUNyB2&#;ZxUt0KimojOq>qy7kzIBf#U~KuLg!2KA z7GP5QE{h{%$@d!MJpLW;QGywy>Y-lyCk6IIUEhA6^Vl*)LVsV%I;eyCguqHII1Y}>D2VJs5iiVJ(QNaqgj{0g9-@l8$~~)T-sDC|c-5iyN~7?bn#q*D_jDh^VJQz37kjym zkLA8J&jChD4`7%~m8&}kL(+wfo-3w40`T7HjJW%j$Pdv~b{sb46YC*yhPj;RD zndDAZBWL8`WC?owYUGSf7RK>(jc6)~RH(NV+@`mxu}~C5S<6m-9)K-zPi8V%4Aax) z(`D%Br#`^Q;CZ8&9qAa1&Ui8SM76PM+UDl!{AK880^)DW9+Uw51aUDRGBh;Y^Y{ls ze%&?yb2apS3ELeQ6Qa5^(?oH`CkYCl{fUMnJ05x*C)`!6mRn$1&|HXW#{Pz3j zqZDY-qu?bEy@2ONMP^WTxG(Yl910z<9<=$C_Ckb;GsDskX1m!7(uvr8Rm0c@D$x}K z%)3usrxG<5hlXbLwgu8zK&AWH1TUvv|Lnp5#-!#(qx@K(QKZPg$#s&D z8x~-C8nJ^fbiUe6q%ElpSenPXQNn3>(w2Vqhu!EO!^kr)J1{$A~B=J7a z*u8RJLawN1sk4I;FxxOYT+NirddH9UWw>LAIw*MhF4h#uc^~AE`w-drbso7 z)1K&|`Ji5*=N7z;oo_zpeK*W=(?!a-=}#G$QjK0WW!*tSUakcX@BlW*&w>`75O(Vo zq}w|ed-8hzWy4zW(aqCSu}waeJsj0Zvc$s_Q@FHA+55n2w1p2)Z2d=KcC+C8YoMEm z+!Jo*T95yzq4<_-r_Tn>SdcW9oo^!|TR2T*XS#aHJPDUqxPwS0a_f^N(jrB1K9> z<7@OS+xg~rMl$NQ@4ev;dA-pJ2ph{dGf`S5EG?)1E{AVkC8u!_A-Z5dhE`*5 z5R*B^jUjisPsjH@igu^Nn)b_ywuK5ZbZ|s?)rk1O@R*(vdpuMCT4)<{ zx=M~Uz8dKbAIo1a%$yeA*SOLW7a!ptJouMGkbl3=bJdOK=Y)(+5lq|16au$zy5-E7 zu{HL-Ma~j?%zwgLg>UfWJdgjX>GeYV0Ybkic+3BLT8UbZDXCJ zP@^?{-+jB>jlvv)o;$*fOh~4HudR$5yOgh0aqK=dK}KEolvSpL>;1obxTGT$uXUgK z3JSYA$ZoC|xI*7w3E9olZcBmO89cx`??n=x-~X`D)evi&Q1-eeSRaDU6MFIQbVq=W z9OlGpf&}XOILEmERFeWxq7?l=^stgd>Z#evP?_cF^^kTIQOrZAk(Yt2_0bBMkoqzH zY9-(s>bR&?6+eKu=PT~tQx0IyzW=-W8hp{vXf$MAm#Z&Rh?|55Ic?!Im8&(V?CBS= znh`}3_YW}hlpiJw)ijk){^=9-)r<;4Pt!z}-35dM*Il>ljFIC!s=PNW=)&r+E^M2E(K)w+GA& z`yivdTa#bdJro#`93C%M*m#Td%TA@TOx`H^3-_umQ)F8kXSz~jh^ZmaO=#gWVb&^; zksXF`Yuklk{PAUn`}coYC>4KZYTf^hpqq|5S`>52W-2H7r27GE5EA?o_&hQ;M1i^ZD(X@o7k`_cYRM7E>)dKsM!| zP<#7vU2+`NO6j;WpmM1ZqrV|R+9=#G>)9h@3C~eXkGx`-&Gb_ex*|{D(3p>m3Pc~D zP3_QZY=6%!SRqtr*Y(!dz=}y*P(%iEJeb0+W76I@stvWK5lC!b-%n+f{H-wsFym?5 zQpiW+!uCl>Aq7GpnJGx0kQq*qBtcqO3^aW78wl^|H766pMx6xyoF$CfB7~qb(L@rvg zB${-OGoHC`Gs14F>=kW{W3Dfnrl;x@dTsOaxzF#;U*i}J3qgV!G<9wPjzR{#_GwO| z(v=7|NUVo?lq=&dgN^7p4g??Petu*f$a0B2%unzmF*fQV2JRK1zz zPw!^&J@a=6`jJPB8Pv8(T6?g|e<*7g@l5jnUtT%vMFHF02c)n^{1@f81@W`S-%*U^k#gnKl`X5^wqfsUP4Fpb6-OZ@A{uZ;r7F|*WMC0Ky zWN*dZ@vga81<&fKAZflRv^>k~cppWWwNZkR^$Ycc72U45)Gi!C<3YR|El1>bkw%eP z8Ro+Kg*k?^!y1*T0~Vykw{VG!uRTgul#hR~DP7pMK!UG=C|PSR*8<&jnOke9|14k8 z@HXoyjpA~?E41*gk#?8AFrc^NsSv~aPxo5b~n|MdZ-e(c1;G|`t+Eh}}PSNwEB)>W) zFV35t`GOn(h|hGv+g~Rsp%{a^qPy=`n%(H1G&0oMuRqp;#2D(G2P0kSx?ji)<>YwK zl%rDFOz2aMa}_H#bB+(PvM`2wo^lhXh*RTZ`c^6TuWeTAG7NSp7Iu#hqakHJC8B@o z?<-}NLumE1tnA3(;E#fkt7prj+IDq1Lw2~%!D^btG~Z#or;OJ0%6gu-P2DPkeH{U1 z>$~Y-qE;19I0wqR$SZdDVfMb#X4)_=2`(1|3@Xcm@M`}CRmL%Wg;UU7IgmL56HpXaOI@Do! zsK&d@+hdGtWbPiqdAm$?)YwX@#vg75sl(6IkPzS2Vkzd5i46LX)P8ao!A0DXQf>kN z%2|zenIS&SS(YmsLhCr&BJC;3L{($yxT7F%w5PnyNznQ#v%mqVnZ zI)O-jpLlX-Sf%UO&a&LIuM)s2GA3CyW?Zu-F31<8%;wvT&aH5WbM^!MvxXKBc@55P$ zUjB0@ywGys=UqyBf?8L2*WTqaJfsVVI{x*JMmB*-dQUdhr666j} zff@pqXJL@HUWLBcWcofUZ{~W@^GxebhOaKHl+2U07H{xxk70~BtXb}P%#j(ug*Mz! zTlu~FHvvae#gBudu5ZP>%&-ra;eG}JH%B30G1 z07;FqLI-LpKu04c49IZUl|86;)sVDHxxGr?T|z-IX$Yi3d;@Uk=S=LhX+TgN_=X}6 z;mUi@(g=GicSEuf!77a7CQ|XCtY!VLy5rsEuFX8*V=I|pl`n4w*j{VdaYN^9KC!+x zdFL?Vy;=29N)K`A=r?FMts!bf*^ezE)5(eN<}$}L^Ob0Jse5<;`gv2$Im73#)enkI zNIv!4b)Y~oxA*{NPX<{o&?11sfcWF5_U}jy#7OR!-@lgvNRRG;)*sXjy!I_=X1xy` zUY%M+_3ak(wc+pN6mp9_i_-$gGrH0kVOxX;H!=gqETZR19&o zLHhkNv?$I`OQn53DzH~=aI!dn%GQ0>0Q18Ye-~;K`($%_kpiJPqf$4~&?*Vz6k&{x zTJ$%C2w4)2e$24nDt6Q-D4!b>(T={DIsUb793SSqTlM? zbX|xHxvJw~8xe(6P~5b^iiJn~W0R+G$UQ|J`()57J;H>HFREMDoyQ7wOKS8LDx^T< zhuVn1tjbV!-RdPHAU0TUXbx>v1B~B61eohNdc9xpSs;a-81k_xUgJ5w21~RXsnc-L zUq97WeZSJ0e6?8%rj;Y~f&4p%OV&xpr&mfTJFhlvP{8wouXRzLK6Q17VxMdG@N9G! zelo)MumW5%gG31yKIxdIEmea;n`WiMI_E!(+*Z#6PNGE`CSDis*cAqLNe7C-{){uJ0B}+yH z{5)+a1Fa3Y#gnGhQY-bFd?gqvZJMwml$Q1I{6ZX@|f&I($yxOkj`}JIOQc$aQ zZ1Cgzw(sPA`O;Y8I1;T++rzT49Q0lu>tfA_@YU?>m} zDZQtGsA`BY3GOS+qD9J}=?bv;HOWjX$tp~paA9Dq3Zf>8+ls1lWQ%gaU-X|S(NIqr z$w;N!Rv=6FR>d+HDwTArT&*X{9jT{a?i5TjgEYq7|xgj1rl zH3s@&l6&)rA%E>%({^=5@7Npayknk|Gcns5K_WxktrQn>kPB7!t|6)7A6o@7eU8g{ zPEe(#yqfn@evlEQ#IWT^O(B$1A)MFpYj@4SI~w#$EfCK4tc>4RhlGY~5tS&z+eV^K z-@IgwWN_-aFNaMyes9I;Ww@vL=U^asjP3B<3IvD`Mv964vnk|1ueV96+k-dBln29E zTi$`jAgpDlF`>QWliLS)Vq8E~M)HpH74$;sM-YW4gKg7Z{w>@MoT%{d{aUR=J=$-R zbCfs0m7TFYW`9yJPmE1*{Q9N%#~tvaAVd@E!hOU_Hx^5>;bK$(4-?LISXaK&VaA6` zhMzxbED_(x+j%M@nCJF%;WABOJ#6zi5K#HTor%eVCQE9guhtMRTU=dS@~-tsIdztP zZN>RcGC&0x`nc&^A!CAEV6lgnW)thL&jhn-2g+n(v=&Vd=R;7{0>`Rp%}M#-&QGsP|@LL`c;B?EDYY)|2G3k%pV6$}`Edmuv!Atht&(9{que3*^BAed?{`Z03HZ zFw;FrgEKlt--3Lyhk=@0=XNM%5HFQKmfpXiD6{e2O z`E6@8pzL}C!P5%*b@aDkgE3Trf6&Icc$a^v(7s*--kgn?6k2t?EyZF0>Us7g2TULq zI)CcQp1(a@RIl}L6TCdlaajA!nZpK*GR_TRuC@W7ZtX(xCMxqtA|coAf)6X0>|voF zzvTN!bicCR5W36W$TpY&l>N{E03LRyeAPqFfKL}B#h8_DE5b|-b-i8nA{6PQ9%>w~ z3zF$)dXnwbU(@TjGd;~;|>4DK_h6;_(_a%%JsYi`z zvNJ{``+vD-TjC$`ik7{Nh@a6HqM|7lQ2`|5hDlSQKlle2%lJw9kWw5gpmjdJJG)-5C8cBz^YVX)K4yKQ%DC@+z6|6cag@w}^I|C5KYv7dhxQ zid5gLu>bE9iVM+vzv{e@6?Sj$J$KKBTS9KPQ zwKCU@_a3)cfV&mUe`1ze{}S0~iCy0uiU`ebJgRhq%^FiLe(&D*%py668vBTiM@b)d zVLBQ77n0)tj79+3XQtOy4e`d3ABb#X|18RXOoEP`-2=0%#e&_+m~ z_TOoe>X7qYhADH5`a58SIq9c?Hpp(D13|+GNW@%flIq0dhW206{;fu#3*{8MT^?CH zXysL#J#+wSB@)tGJ9!}&H$yt)YAx`Z0pFjfaawZDZQ9$bSbd93WX+OEYZ`P4 zJB-I1P>f8Et8X#PC;Me;1*l}@usRKh!Z;098@1#y$}Y7c#KcHdMUdwsIDfj#vIU&} zU~I|lGPJK+A0Oij-NNX7k8a_$>pvH>TI=8W!EJ(CS7aBrMhuR?Z4<;wfu0+%3&DH9usw+vvE1P+Trq9uuN{&ZE(;I3E8nIZ%JnoGxWl$N%sY zXUuix4Rv%3L}t7Na4lD4ltSr#p5W*WL%0|?oumgxasxT4*+w^mv12{)VFa z+IAdo1&3(F=ex~T-iN=5M&56J8yh@MRfzih7l-rEl}xoNsF3J!umtMJ_OwkoyRKKn zGpgZXSi)P$8IdByr=SVg6>Ok#RW6`PbaqM7HOd~Pc_Q*#R3RG_1 z&NmgO_b*Tt*C~NM|Mk347oI?^G%mU1l5mT!L94PYTeid}Kl#aY3a9|$i6@?T2xK??KmOxCX8ZM7 z@!P-s+X;Njz?xu&z?Q(_-~avJC#ysA&K6VxEPwmAf19=!8_zri%e1!iw?N_(Pdt$( zXy)m2({|D#xF}#JXejMkR0&vS9zF8NBXQ%6HzrF^<5&j4NdYv2lNEU+h$Amv{pwdI za4+*s-U=YZ{ZH(qcb>pZLTl5^X=~PmMmr64^}?HQH+qPGoxq zrpDBMCZ^nG9?!(fvLdoXGA2K5uCx!@Cs!wZSs#1f_LqOZ+*S19`U#_g#-1B?bmi#$AmiRG4ze;+3(o?aif~rH8$MI%8t2y@{6TDwh-b}(&Z+2+qci_ z8r?F8e&K!f#1M)?y=nVyYyfN_o_a;c`W8CrInrq4Fd)GaR6OZ1A?Gm$O-YYgt`V52 zUY0K3)1pbMzr;+QtFNUC6-s14Bz_C-ct5{aVzI#A(pkpJMmpqM9i5YU?nm+O$ML_Y zg)lq;KT2h{{-t5iehwZeKSk<5U={l!GGze;i?<5QE^|uIzOE^8^LO^dZTGB=qZZc1 zN`R^rC(j0iNB_>Bz?l%!p?x+A--Ut>Z~zKH$67>bTW1OvIq)wp{BiG4;6CvxY;omO zh6LtOrO<^%ms?5cz&eKxMq{BhX5jK-&<+?;*8f=rpJfFtrs%YKJMl5r(G+)m^}cxE ztIv{v3o6BEnXR+Hy6EZajazSgJZ}5$<8j5c$Hnpsmjdny7*ZePT}Ie?Fk}4mC%zs7 zYpHYUeb&(&>OtKOGho=Ux+`wp^FUO(`{Vda=11c+w@PAFhR_uHXx|6xr4PXoXl`baY7mRP0WI$+aLuAh0W#D8P>PU3 z3-VM1+Rap5jXtvccS|dKqcOpHJcp-0qNsQvi{&d%!$rlKxcv)j0|UvJ z)Lx2%)a`+d!||#AbyNJs|M}0cx0|i=53K?CTOS3$*x@}03LxzF?|3Ah+4VSb-~t+b zyC^innR72cH5RU%jb#{Ms-Plr+eZV@Qf|}ctt76saRmc-=R#c|cANX6G@YW^t zqs!!J)(w?!Ox>N3D-%H=!A&j|SzNq5MF z73k^OJrGaby*lo^`MC(*n>Gc@!MQ9X(J*^^`{VX+J{b=_@?>0e$=R{!S0^R+FLn6+ z8=_r2w~np+k8vp4_K5WjJ7e3k+hWby9f7eR+hFORsqY8A_;l>+*&WyXm$$?)2Y4vQ zF9sH{DPIlf8*QMJDT96Lw_2!4pj($KSf>iyImYl%6)hm$DBRDOs)BVQO)=?g8+=*o zSO%OUlw}Z$S5VNL;1PAT{~HhE92n=oI0wc#FwTKrJ`OM=>A}xM)GdsTJyWC1AZ|a4 z&-CfFQ0xB_AeEVuCL*I_2@C=lHZY!HOykw)UL8PqbwV@tetw>>K6J3gLMz)CYVQlk znZFY(j5cBq<9*X(2S%E_7*D{+!AP1w@!Por1{7F-aG$t>R=B!L@dY2Z?IFR2S(3H&)RWb9o_hd8)(Ne_*0{?cjuG;!oAy-u#A z*K{qP6GyM$X(=7t&ZVp6%=aya>7;=d-{oiK2d^IDqv^}9oW57{Grsgnx8-nxo9U4L zTs|k#rYGIz?-<^z?|I+%35pYlBfAe zpPBfc`3m6XdUm36V1KQg^2z*gsr3?mr264q(=lD+Wxi!P{4L+)hjj5jl`CJR({lP; zdiRmX0NaJ!ic-YNMd#xc2m|h{)dwg6!&pF5PN#$URV+dGGkNfo$c;B`jT^tdJ7!L- zkCn&Hj8kWhMDO5<(Ok#j8)jm8oy8|AMukDT2#`?=Ouz({9EhiZwa@sjHGvjQ4$?g( z*Xs^IwHt1@A$`~COHj|P-TL429n*2Uw%f|DzyA98@|VAywvAtK!37D>GD88a2OfAJ zfmqjGdu_b+t#3_-$fSt?&$YDs-~ao6C*a9EJbXecJ+0~l<^&A|q_m(D2>aN_K9=4W z)a&W#Nubr={oUUsK99NkqFPmDP|vI7@ZMNn zo0n;Q-2YWdfhspC0Iz=_E9#pPV zvoww}NXYFoCo?LBsVyj~xqTddwVnK^=RWlL;qh`i^Q!){oo79(-ZUTS^gY|L?N@!^ zwVR1J{_GPFq5e@{b^|Q5U3^_h`;Uwwlc zyri$yXVNDj_W9v_&%Y1nYrdba(rnty$J4@&;Nvdk0ZXEiodsbCB-Dj77dWc;NGk)DRVmyM zhdPa5om8%y!4U!ztVGG|p~EUZ2APZ!^B{ zNCQ-)W@%XyFkp?eprm4xwnS9&vFG6cv_TeqTzE=45dZ^^t;zpha}hf6h1R*F8r|Dr zAiO(j(SmfK)KC#X1z@d!AJiwl{8T(}{j*e9#5r$YlB}kh(7Sa5jj?}wSKRx}N8^bv zY>MxE>gi~&n;eTSae=Y}0Ew-P=-?rIzD-*U5Vc@JoV4u3n1_{H8HBIDcO>rp)`RiD z=bw%HzW78eJYi0hr?90NA94c2BfE>S`i?d6y?an1=ta4yF6JEv7&9;8v=dIj@~=H6O`V42|8U%U|HJXfN7uz&pI;k`SImwX z>_X^gm9-A}7#iq|Q&*l8%NLvwBikC|8y~}J`^k>D=W|cQlN>NJbt(05(djX}etK-( zu|3v4`%KJSG@X7CKXve2{nSDmaBK9z9sEEp+`^je$X3&82wjufLkDzAVT9(mi?S8o=F+F>nC=2k6HOa25a-3qq~!-A9}JL;|o0 z-j*nT1>byKT`Z0vH!ZDzXIKjk>&uR&fg3y8zgyX{G-r#lK;j68SM-BqHF7A%nU~VX zp28s}oS3_3RY!dOhHpnN{wej}2wt7Q@;-bix)9h6!rD-6jOKv}apzZVi%0%>E1eNM zdgC#1^7$vmgz31Ef!40ho_O%vkH({4c_F_35BJ5zlNQ9;m%aiYj`Uf?zwqia;)KOr z@y!q46MLQ-j@5lTV&&WC#EMrO2Y}feTQ?kx8~^FfSpTg(v2OlTG4sDbH(WrH_y-t@ zLa?vke+n)V8`M);a~m_Y&(R2#QZI*%ap+hT8Ki+WxVR4ghPCun0is-8OcgsEW>vb6 zZRnmrUBLHX9agbvg~AMuut0T>hj9*!b6}hU;~W_0z%L;Ou-vLJP#h!s{$O31sH0%Dqq`r!%qgJ!$k&O45o_CCJ9ci$A%0PjJ@}A-Fm@A_4!UgAfif zdFzbdc+dYG6Pp(@*lUm8A$%_-iw0L9J#c{)U||_?FH39GiysjB8q^ArA$Ea=iSG`IxS> z_`V_E^SAZl`?(C6U#6GK?ECWDd(t#E)>GMoPKrPu9(8ie^;e58cshdPCRExv@uvG zVr`&pmbGkaBX5dl`Q>?v1x?bI>(%dqK>AS3R&83L=?_eKv}`AxbW)0MdE}=-3%Ni1 z!#_-xdIEe}-)UXvcEnq6y)|vu6`Z>A$}1BHI98Sgq6FWxnET7W{LAFWP=1@2@9W#p zv;?|5+~dCc?u*yG?scgQ#u4Zf{QI?E`?dJ5|MkCe!tu;RyXAEIs}_P<=ebSSe62Hs zSA9gzn>UYBrXNn}S;wX+KNc;*DLv1%cFT1>_FXLqOH70O|MLWR{>h*GNot!} zPUWfPk)Jtz>DaJgL;TfW{Z*pn!yo=ICt9DEKs%rLj%C00+G|r<0+;et-kYcLY1!nz zSK}&sTL1mt@BLolbH1-N;RP35kjhZwyJ*J`59a*?fx5Q7{rutku4nSx9Q*O-RbbmB zP%t`8|A_Fi`M*s3+@A8I{GRo$ZnWKJep{zm zCmKJuQ`3-_x)o5q)u*|xb9$y>9MkoxY~*zPo^^|PDlj-^KU7zQxWswHo^SjS1XVzUlGyNYIa=Cw4yp#tbQb8AaFjN7n=z+>i ztXijGd4p9GsHvyk3+jzPi+Zg@+#%4%Iw$SO$$;jA_z=VF2!WWv;#YsHdZs&8Z`m9} z4?h*9nNP*s6L!Q|mvKnZv4B1mw!B+cpd}u9;OM{uH3SU|$vRZA;!_4780-fyKm*o` zo&sua+pr^s@blMK52ypZJ-Y|v?yo$=7FTrbGOWVRUP>Bl1+3r{0l;O)^LyjYFFXn8 z6Y(qWy*L)1JwJw~fc!S0ud!C0x?o~l{bm5512@N0-`O0`+_fp@oIWv1SX2pCh)>wo zuDRyT(F};_$*+U;_>jYT(!(ZRdFjP*;K8AI=JDsFZ+|7)`x%b_T$FT8vZEB={3M_r zK-62`ePt{;XEy%$ICP>3`q{2{=9HE3jk@o~qhH;U;s^-YFEj#}>MzgM>&R(@!#&Vu zsqJ|)-`5hfti>vKv{V$JDpN02`k)3(?51F0$L0d+Ct}e>)8mcrxr}X~^cj<}YQu5h zv8ODK<4@QV-~PLM;>p{$#@`xaFQ#QZ^&3fLw#KRhS$eH2PeO(%dohtu)Vo; zQhS_+8J4W_4#{D-v758;N8fTqk+&x#>-TU7w7(v|i%l%tl<+l&6GIMaQI4R`HjgZ7&08S70sI7N#9N2M z-CgSs#G1PS^{B6_f8~N$cKL!BnT9{W#)HTdV;`*dF8YR0Nh(+hljJBDKXE0A*&^{BuabY}o)AO-+!@ih_ ze@FF;?OO1z_FefBe6u{MXz35Jk_^_i$|IHZ4R#2feH|=%;Lp#oG{Ap}!la!vltF+m zr;+rFpe@S)%BKgV(1X92!4dTBMJ@hAh@$&UjqoLpVWbT7Gakk{a3pbHtW5s6LBWwk z_mNJE1GJ>j)s}?Z{_TLq!#D^2BXht>hw|*CBnN*x$r;g{A@zU^X3{@@)+RIF&pIYw z^ZWVw!|soLZobBoj@-_CE)Ds8zZ;BW$nEY$I$s2mOru6S12`s>J(za*o)%6{dWSKW zaB@~;0&Y6SH`FHDHLxX5@SJ=(85=tZ%kg|i`iyHnPWq(ha9Z>IT$Wr0<;-Co{sl(p3y&_f0FO?fqO|-ZSKKX|H5vCvEPYCW^@SFRtw3qT248Bbt9Kuq2#Z<)tAjVyx#OmpVUNdTN_eg5;GPw#smfmUhy zwR_d8UX`r*d@g_eDC2@+`la&_8m-Z^K=Z!Or6nImp%vcKPd^>E-F92DR zm!4Z}r9pru*PC<;dTE)b_1}aE6Vf(f)3#nr!!mdkz|FGdd&Ut=vCNh?%Z@zCd~i#$ zG`bB}+RfW~FwY!vUjEK}7Hrcup$92=XpEMMzHhrXWF4S2pdhCH_XITc9Vj16)3oHf zdd2soPhj)SZ+>$EPfbTYSZ2#QmVTf8o5NvK4%0N4W-1R`F7t2i6`;%D(Cviy4A-&qcfb9gX~%4zew?)~1rH4L~Jk_Q|rSg{^_Zq#-I+hhCdA zXBKS>UKLp!vuufIbR^bQLdQU4Nw&PLOp!Am4&1@dT{D)0gNzkJ+h5opJGbp;J0@0J zw3P-*FfpdZ#L@&D_(2ETL+kK6R)&vd4xD)UUO<{15%XU)ITlXZV=e^;SXgrF(W(uc2LftBUV4LK2E!QMbx)&XqDToD_lG9HM*rQ)^FQ_H5#Ba;7p;1!yWMD*N96DU4e9W zk_QXCnkAO-71v+Ef{f^j0~{iv8St4`A04N^YI(Fx?f3E^S>Jnv`zP*IipwRRnmWoC6jedSxJWPW2l1xx>52D|>zUF=o&VUaj=b;%T z>aK*0z$tT2esu>shGPBwn}S26**lGey-aAT}kcwDl;gc#yrMd2H{-*74LJr5avQP)JrVe~lw^p-{fY3}44MKvky3{srKurQmp;{KxO z9?v(88Gj-2qkJpdTiJ^H$Nz&ZMh`P_DFQZPt>DzEHhBkRVn4TAr3C2;Og4&aBV+Rv zEB084VnR~K9;`D?SR5;tpBl%s^+d;GJ7W296XWQk7}+O0iA-_{N5PaNnB2AWPR4Uz zlKU3>a3}13JYeCv>#j@S(2gBDl8?69WW5HAI)Tsqd}o}~Ow|MX7@022($eXI1Fz5r$h5ns;2DS;VIs!2QZ&iDN7)%2t%gOKLw zGd~%u$@hHUcl3`ZaHz$g{`!abgO7t~h(ZzPa66nNGFAH-AJ5vSi+_^9Q<#V42cIKw7mJ|gmC~zB+)q-jO|9716 zfesQ#>55tPv^^aga+t#Q)qCO_pZi|yco+h}-b%~C6#=Zo8hci8Zz~`4buU(P0zGwD z-1Q!;#N+opLxSKmGx1Fapf=3b(gJ?PD&2r)^aQ(?_!(IDoMVKwD!AvH=iUL`GQlN>*lNK-NYUw=~YW@UqjR4y&`CYNsS(#m1bI zeowy!+=1Lr_?5A%< z-{beV47gdssWOt8{MA+5rz^3t0iB&dJpjon!mUxMr!wo0u@0IA-G=ZNScPW85V{|Q z&bSZWK-U0rZM}L3OCJ?Kh>W`}yQuIWV?_o6CF-DGb3K_)gX)nF^j+#k_Hm#?^?wIV ziZ97m(C@&%&;?MNXu*0-#mhG0$^p_vUQWJrc9bU5XHr)4Zp8v}2%4>edVn}~i^qgT zjj`v6{^;z0b_77*itg1Hbn}$tsdHKUP_JVJ=|Ma7Y;A9(URql90ZewjMw^x{56@Ak zkkAA?x+Qmn@k0S%F3B4CN<6j2KMTrt^1)dJ3*E%kY8OYNup2;My#bx{jeC2kFVc5c ze~~nc$YTrjrtI|rel`L&9*N<|r0PiI)HqE?BnKRHIzY?=R0oVXRBi4EhyD9Y`hNbAYW-+OKf&n-h}Jkbxk9 zq_F^#fSJDE?0*Cv1<~BLs@0f>R2bKON`TCMOJK`S?ym%{Hf`FJEbB}=TM(I_!O5s` zP2;rFPD>zH4d9ep=m666w}6&@<4jKg&maINJzB|S(8_y;8i4hpRjc$_7W2*J&h#2T z=aJvfag6V~(wEEXcWE|G2GnvIrelx?rj^r4H%I^YuH`leeEN>%(7Ix5-)}yqpYt-U z%v))aHfi#?!RPY9_k5P~$z{rMkjxbS-~2~;L2+o!B2L`au#JNR=A;QAXCfq^LQzuc z6gl;_wVU+w=)J6W4&CR=FH(PJdd9Cmc|Bkdr5Z_xXXuX*7h*AeD)nC3jSNo4ivT*v z)h$=E1Yg)f=k{Bih@q7L;&2<#&dHaPCw0g2V>$Ku#Kp1vnD$tV3Ts|~HBn`CY^*#U zy^ULAsEz|1ILQ`q#;OfK1_3K-U4<^II>A@pADmj>FoTR_8NLhqmljll9w^iGL`C5EXKvAWtHt|^Q*On|W3DZ>ISK#HuG zmQ$lepA2CQiNziGR0@6Qr?%1*EaIwIT(TvaMJUjgntp2oy+YlxGd2c0dN^#OJ`U{Y zh`ayw`>}mhB^F=Y7RRn!9IZ34q@!OO=_|#)o%>_MC-=naP57-F#u5<#4jV$od~6wS zYRCU0U>1#XK&QSQEJYd0I`PB`v)D?VBTsdJntk*KRX{fDVOYcoFn0u=R4B(VeTIFo z3aJio438;T19H$54&{ab!|FMKW0(U+qS_3*<$DeMYS3y9eOtzCu)+yvE%<1h&LvMdKehuL>ZEK((CL=s00LBr^0iAUK zdN_~(O5_SZ>#)j$M9PcX7yxLMx8oC$;bERZV5+wvEmo<~+SYjc87pFJ(CQGG^A#2~ z^}VUn&mnb%pfM*uHzmJ>)|vTHUXMWLFcIx>>#=4`764VK7MhUm}IdHsErBmIc$x2 z_2u}_TaE<<6X?lp4S_2s2D?lkI`_n+p-nNlidSxz?VX?{R9PY8d1^siQczNB!k7D! z3>sz7O5cuJfq7r>$Z)|07o>jO`*+`ccRI+y11|KJmqP}ZylPqYwXc0G-u13`CD6)k zxt>_-c38LO3S`-*8CNSg0Y3q|bI(0DF1qNV^xJ3lgI@hepT7mZq}}*}V{W~5GA8h+ zuR8%UpBdkffiD3x^YI<;jRnDsm&ed)3+-Zzf-1l~+zESP3| zY4o|zEQh?+iqLp__UuXN8^>qnA)S7g4*dgqz=*%Ki1fMX7&0CCo^%@g?I+iv>G|9A zUMdqQ`j7r2S~nU@qqPw;rN(JkVF3t0M;!nIxgAo_P5@I7#WtkZnb6gKy%cL0y*GaO zkHdk2W;@7YJ#yiRy`GXP#?;!(C2YSqJ`3}OHUz?A~uOPqG;%GT@&d*Z5B zdALh4j$Oj3pKKeXX#$97)U}V@vMIYAY-1k))G0=#FOc*)L$u`5M3YJH|~tBxAn!!SDz5C{M9p~JR1-Zzl2(xHJ00A(yaFA zn^zyJcWgv=su8rABOml&Kie&f@U;MdI-v!Bg{m^rcl&D-R(L&FBD$U510xz4NEPwr zCm1#i*e1AFWvo@Ape5>{005?sLIEhZbJnvj7p+dI=vQ=c0TMRI3TuHa6!lo74S_VJ z{@E?d0B9xjo9BJ#YUgkgZp7N}?~5U}K=(HS{51nU0?xU8cM$m~53*p#7V`@2)9vWZ6Pf@t zyUBw73c9qiQ~p)b%{0bAz%)Tt+gXwAvXw=FzDps|t6jU&X(opk?5`!h=J zkw(@yJ>wksw{gJ1oxy>v!3J&cx6ktPbaVXAPt5^4jy!!ZQPtDHp7mpMUFd25m zn6*Vm4fN9tAQ)IVIAYj?%-0C5j_MhW{nJ2olzs=LKhHp``;o~r#Cgf{my4Z9=Rq-z zhDM{aB3TPB4tida(CCjJpa0KyU^AZo%hl+MHI+b&7k~Z)-%OVerx>!{%?BWd(O{t- zju$(b2<(qZ6M8xPp%h2YZ;EqP92=Kj*bs{r5r^?O^A#o`43POSk72;e1e-}{i3X!{ zdXpZ(;^Rf>P)=X`Rr}_Ir1s5zJNMnW5Q1WU1gZ>zNx2Ua9Mb=t{_q6Q?2C*ixMe!} zySwS8o8qtk`mf{KYp;#p`mNte1AQ&(KK}8Kr}2UQS;xRkEr1%W-mKJ)z2m%fw^ z&R9f&oggIZ39cE86Sxz=^O*pav}uKBe7Cjg%kRDKeQ!!Dk4>ae+62c09DV2f^UqK3 z> zix(#=J$>h0am5vJ`st^qyiM0hlYBSdC!c&W9g5-MA|4Eq>9uS=H@$p#Mg|nk*F620 zP6qh=uG4cgQzn1(>R4{P^RmhPz|9}a|J0{ZyVR*5t_C}9#~lWIsxrY( zi&@A*36N}LaCc1O(2Dspim`lgdt7nJl32N78J26z+ci2uRA_F=+DxgzlN}%d-<;$# zv#+}4kxYOyk{gQ=0dl|?LA|sRm?R9K1UhOC`gxZI|XS0g(T-k^x*H7V4tgdJs zXo=2UgR%LsJ-m!g2dEkWY!Xl!ZWxNw&RrU7Zrv7Jp4k(d*X)m}XRuJw4*1tyiB|j* z7Pn7|XCHY!x_3Ya&9JG6`!3+a0kH8!J)YGOAlaH1rRj7o)e5Y zp`7=-dPo_vCbvz(FCe<2+ye+l?LkYiXD|+K-4R>XJkOq7vIek09bqv>UVyQlU|V07 z7syaIK$)m8bV(iv_Bqy~OncdK+E4^c z1bwbjSta|&Zv5|I{W;j#A1(NvYvDkPu0Bp^rC)Bux@`cf#4_7cwWuosT*E@ZC<;o_ zWx%*0w+sukp=trc4Xc~t>shmp@ZOzc#PgWWvu;V=x-fXlA&7VuqA>J+^6L|FVS z0(R!#VPk|E>+b8NSg@{Un8H4@em2?@!Lp@i&ez4lwB*aP`p% z0)ryHj1uSo0U7rfe}pKs*>fV}zZ{%+_$!SQUgpZsAr1B%&^nkn{7NJGk?))Nk4isf zoQWag;dGckW46>*v-OGtCl3WtXPWkpO#jE^e|Ua}|NeQG|M2_{|NYCJf96N#gYrh@ zBNa8d)KHH2{Y4{G({SOfRWD) zf<*@o97yr~Zd^eD&zI^!N7rz+a z`qsA+fM(hC>E~e-0%itzAdS*%S>2}m{qKK29hxH$DDCp~_Sv+(`m%sew1cv_Z@BVHqUApwp)?)MXfQ=7+@PmoZ9uDI3JRT5ql%Il$`W+O!d+xdC zQkj#UL14V+0UrkGtI@46{IH+|y(mQ~7BZLQcXSMr<_V-#6UZcX>A612)aB_pFV?K7 zh&Gy?j@47_pHcrNQLd%*vU)VPyTkQ<{w>|(`nOhZFUJR!%FEr)d{?Jbsq324e5xEJ zrjc-Ar)>pf68fH^8JO@q`qo84HySNx#afSJo!(GttPX>SOmHyx6%PvQDIx>zG`%A#hj4n zyQ!z%g744>ziZ7^vSWI z)DcfT`E(rp#N3#_Y-aQcdU80xhR0UNV|P6r>z_FQa)H0S?VNbKcQ~47VF`z&;N0V< z$K*32_N~VH`Lnmj8GXxR#^R~b3b3`St0T5OvMuiT>fO-~kdwAx?xtVq#&cOK2dN57 zqLULBN86oNc&QEYJMqzW$V!phKF0=uQR9}dc1zeT&HjOJJsg$hjyUnc#nCVkpocBL zT|0Z@?wjw4Ef4iZ9r)+UOHU7WU`G!;Zo+EN?ZK;`dOT*YYl&$~+VP)9yVh2Fo9=2mntJQ83C(rJCp(9Fc*&W`+w$k2O0R|5MEN!6-Y?-Y?U7O%fLDV6c+QE`cY;F>_KNg{bB{*YhM$jTpWmDS(q^^^msu6tw{K7Eeqnn$ut_Vk z^DjLsjy`z`^@c?sTac9zeWmmb;1&abxfePF^!hl3zHc+&ChnQgKxqW70#-TF;&`;B)-1uS{-TPDX$Q&xQ-7e&N&*tjM#9BP+L$JUOgSujCxC_0DcF>CO^zQ`N zq}&){a<8Aout_o4i+{aVPCK8%fGG7(T#m3%(_KdpQMn^Nf=C)ChErWUT(%>Lx9MC)Bt8JXXe=`S4tZ+8rfSN&Ih3=3KQ&9em4m6!r z5RyfcUtBUB6S5@<E;jn84(?*j?lN4Xizf)2UFb;d~-}*ZPBq^PoE`#O#evvQi zreMIQRaPxmVU_~E)DDH8A^)7u54--n%m2gj&G&!V^Ur)>_(NJw*9(n$Ndv1I7O z5}}__ce_Ul8X4I$q$ueRKav>MCQx+K38=D9v2S_ovB%=`=5Yw{Ge%?u< z_q=*=g&)B{0VCfPIJ@D78xjB}fY#B`ku3Vwu3ej~@9e__wmdafpw2D28#Zi+_r33Z zaq-0$r*u86!)?01^E`;Fw4FY|? z`m4X1Kt;Fw3gUU#grJ*c6O5Av!8%Xg)w)i)OiS=hpl|l<*$JHVR9R1`)v`{Y(f1}z znw035Hp}RtAcj2Nkr(>MvmBmU`+*O9AQmlJlupNe@WBTY-O?@}waSy`42lYJ%A0%c zxhDaCr=EIhisP1OPXK=Cp@-5oTu&5!>s#NNKt;#vf_~B=*eSTTW5-ih zHv)JkP&dR!J{JK|^rtrfs%pB5Ea$T!x< zGf%F^f{r>O{(g28SHZum)7QW2l9+cK)`Eot(E<=O70bPeDO-4)mW*|%cZvGcKh zE5ydFo1>F5gWSZDWz%BAx;=64zdRjJ-oGX;`~7oc{&BNoKmKBG`}{qz>52W((1YL6 z?G%(eKJ({aNhcP!dICH&G_D+pD_{G{n7EKJB!1%h@JU+%a4KT8*ihnNq{hLRbrgIA zble9p^r;VgE{?1p3$F|!TiUKgPow{jyAufE!ia6?&xd|Mr0$e2vL}YcrNwcF!-3)Gt z_~OUD5z}W)VtZ>#tbcL~hq7Q9C+}uPoOC6?Ho97uj{ShM4}BJ}_rWcIXVgU>HgJ8& zIr7U+@A%3`zs+gJoKj2M8JaW{C!fDGj(^pXXacZoW?}EbUpX^6_I)#+ySox!{rm67 z$*1qY=i?-r35QXh4u>&u7=gfwkyp ziv5`n+=Mmj@p0B0meFTb;{I;}L?Z)z&vK$}+rhZuli!ZZUbP~Yoq9}^a0FS!2c+dM zI$oe5>&I|xa2%wZ)edt(()zLRryQOzEqw(}rg`R$4RP<69_Kp`@aNcV{UEkq|3~~( zDyP${vHXg8alxxsMuh{zj>J%tm?M$0ahk?C@NeLNtHW-&aqxNh<(J3t#~+{Ga=__e zcRc)T9B@GH1WNuoKzGvMb|$~eW3S_3oCE)HIl#aLbVJ`CZkyq#Jhm`6FtDR{1vcvt zMgz5}VH%j`uMSdE_=R1a;1bd@cy5c(JeD2?)^1zNH}zq1=y`hi(4!ackA2@b~3Ouy0H?0{k7dqKSI+EOfDRI_4Xp2;ftXq_WU`0+ke_7cFIEU;hjL_AxE zTVMo;_=6L1fI3IP^u11;GrLTH1@oGi`Xh@fUj?J=OZ*6OxxH3^NQ*F^xo~ItZh!Tf ze+z`U{nhQW2EnQ|Yu2Qz;F$LX()0-@0BB#OrJ7@gvEY){W~SwHw~@Z~wXaP+=L8r9 z7H_%bmPD7}qQP|a*{5}%R(Zag0Y)w8Jc(CO%x%DeXkY*O*AtLv+MYQ3&Ud~u-uT8h zCXg*=M%^#uxEL+qBGuK{Ka1+pi|330m{rfpBV)I{@Z{1 zf7!baK;4e2Uf}!OmYW6%Nhkv0UV0Uf5~W@cq!+=*?btuxOcwuoz2sZ_a1LP`v3yvcmrPPN>^%i$pygi4t(V+U)kA*@J(-eQ}g%(2GUV~ z;|r{_pCLdp{bnh&z5%S#MW4`_Zr&Z_;Q;`oqx`hHDZJtzBWN&KAUTVIdtJ)wo?S`Z z)NXf+=oX($$7K>@+#Bep%nohHm(E)YIIv#$HS$KJKOQh_N~xkBML$(fOoBZO>ck|y|*M01?GaDc_S0CH}a;+)AV#AG&@1FdOM|G#2`VZY%pFX?$ z%D;ZIJL?nQtgmtwxKfwIk@fM2o8RvG-OV=Ls5|z0M|2mg`*wMS6)0APS1f?-S{BW20wWHfx9*Pj*xG&Q1AkX~e5X64XpUXCxx3zNuGii69(U@FyvF+O zM@M|60I&KYZEd`FrK=s+{q|!YP+;kYx^I8!g6?Z){;&X@0?ICTM0fRT9Mj$Vf%ol> z`q5nX*Drl_wRcT@Rk)DJw)b4^UsnK|FTAe1xPI_Ge@)qUZQHkT_2xtCE5P;BFMzT< z$j<%Vxz&yW#7bAMkd`pd)l$_}+e+^(wS=|C&6?V?_9@SPX!o8|PwPJauV3#zbNUy$ z_2tpGzJ6T2#`O;EPI%x6-PLb=e0S*2)faT<&S@T_TS}kvzg1)Rym}Cy!~1SccWfbv zb64WIR<`@0chvf^qLvG9sXcO6U0D15)t-l2{ZdyK#atoJtII3!h|3?|J^rZ=?%sXs z`@3)c*Z1m6(ch~NaD1;j>X4(mYhB^^ZsT3A++FL0t92J0|D#$yeQ|e4d3c`xSR$t!~!<{=H`Wx`<^HJv7MrkVRW(OzKd#Yc#|crSU8q?>bP zUXa?&ztUb@oTtw0%gm53-gz;Uo}~?srDAp?--}t?ozm)CJ?@b_2z%%l^NSoXke{3^cjw8=YwU_wl zntIU@zhQql4^86Gn)L8Gu%w60?7wrr``xe6r=Nzp;g!150-k6>FE6@DqZiN!%H++y z{D*JqCXUW&Q}V?^_+}gp%TpE}DH~n~jmh}uXVU`bozs2gEpP5V|Nggk_c-x?)-ULH zthcFpxbpP@1-9J71 zY29^ic%$yPr<`1F95pGKTtBXz{%%}ZVcf2-{#?C>Kb7JCKYlnyGESw;93)gS%RWXa zDnu07Wbf?5F^-gz%*<>;RvhCX;~a|2?0xLb;TXqpI9A`&`~CU+e&65kFSs9%`@XO1 zdR@;+!`lEr5Ed8?=gl^#?*gU#SN)X%wEj5%Y(mpd_M_F(dMH)OUFU|H@o$FP=BCAP(5jjH zW;r2)-PiCI;D_>GBO2N#S6`VN%{1!~lS76F?bFsEL zNtWrLLlCd^B`ED+?@9gUe8q9Y9eS6aI@A*xg)P`rxMc) z_IRXtbFV6;QLC|*Mda$RkJ-70Hj5E%yGXDI7j!r^=X^NsQwJzM37)Qfm zVXd|?a-LF%^|`6gKwQX)$sC@7Tq@x#Hk>+YPhwAsVYL4Bywzu0=rT^LU3=G1Y3AwD zpy^W^R9UiqxoDh=8Q$G@{WqdcoQ&DFbuWS~#B6)$G=%uIg{w8}xR8Ah!^mEZ`#rCA z2+qg#EZkoNj`?eMPB6UCorb?ZucG=>?9Wse{O40&DDKp((VE~`51%wmJZP3rW>Nav zDmK;MINGc9BgMixf2&V0i+QUWIxD|67OORfESNaBHerk{`wMD5y{w%D^1Xm7YA(RM z)FiD&{vf_OUk0aoj8X(@=yf8y`PqIaf@gP~%;CP>j2$QCpIOyA7+3;38hVH-x5}!K zmIo=k^)+W-scnA~*$8S+|NdS}9$jO%EXO0p^j;H8Oq{Q~mz^C(jq{$)Yq*+b!1`j- z!R_IAju-{py1I%Xp9^*?705{#9SG*BagPXFxa+*z zv-Xq8UxMbU!~`>kCTr6ljz?~{=xB_2ikHr$oB%y)JF0nq;C|)hB+Z@_gE>A%6-VGR zPV+^Y`^kO)^MB>>x5d4Q-aiHN#U^d!b)KYMZ@925w>)^8kTjTX-uj@_+@TfZ`Vjgk z7%RxO`%)(=_H>u?T|jOmkLkLYY9*ug%!np;c*BrDZA#w*deTi1N8e9ki@D74&$Gi> znwO>3PmbuDY4AfxnKe>r1O2snthG2V9_1Hnu^Fp!e&lFTqA{0SHrgfECLj|DA$BME zcm4gvju}W^gz{071`5UPrbM#8w=I< z)ND2PY>78@A=B%i)gpi?k5-*AitWg#T6G)kI4&5}@6~**(92KyQd`{7r< zwUjKwY@NV4uXhyS)<)+KmOB(%0Q_ROt%;u8ANV-<#pTO7p!LeoSVp9Pi9)~Tjw`pU zn@bK)dJ%j5`Yw1ZL6&K2UbkOE?7fv@fj!WejN1Ux$39&n-cXSmWbx^~N#CEF)u)zT zW#yl34$XdiFMN<1<{X;YF*fUo4LLC7K7)L3PN~_zRFJ zx^-kpNybq1ZyGgI&B+&QXA6cbc-C#&v?EnxlK`VN#PvmbR$sbZVt_Mpt!)L1#bcDT zTfVB)+=XH0=t>T36B;G#)p2TE_*s)#-xNjC7rN`z9mG_S*;Vh?3X|uL(dU>FQOLDP zpBWEE6%27c%XW_O*rg1FSy)-0ibm|c2~5(){SBllNKwLd`s!pi2HY;d@J`>E#njTX zXa+X+SBLI3VBi|3WAiWkf9cJeFU!7j`q09LNQ8lv0d?>wCGEZ3m$avpm_?Y^mlRHo zBGm1D`Pzp_rvBOBf{>p-hp3J3_vy>pMq5UxaLVEsT(*B`QAH50W7RO18agNRVEbn` z!_B9{ic+5*8a5w~)>>6s(74RQDD<=+>2VkH8ehq{*t1NBmcdcWgUH>4?9*jBUz+>7 zS1x!zVDynW+V(-hYHZnqU-^W|S`r90@EmCe8uCLzxa_CNp9q$LkJ)Fs<`ySq6;p2o z>;ly3vYwVE#6=2u(7E;ru%qdiiOfu-xK#F@rtd}mu`(ksZ-yl* zc5Hr=MLSNhB6{6nQ&TcVxOl5mrUNEk$+UGYrCl8bTSu=qfq7ogfe&|W*QMO{o+(Kk z3NcaHHh;981LF_)>_D|O>~S0|juCMTzX!%Dfth*wMDMKgq!_JZ*eyF`Ha%pXs;oRP zaEN_DI5r~}e?wNr6WsVnxlw*X1u~HH82Dq~x0Jn~*rWM` zD|5r%NoP)e}O-tH2-^65`PRFJUgLt?OovdDu*H3-vg9+ zcElNGQQcM(mSf+2640rcx)ya@%-vv>xvPziZ+s8PZE3-GqHbV&MMO? zAld$?s5HyV<5z_#`26Ux-TIZDY|+R+15tYpp=Y+Cla;m2N88HT=gLY;{bazhcon7)!1oS!gKQvul`hX@#+5+ak4Nk%Q$B9ZdBil zZW=^^0&^WZXqH71k@t(58Vp~V9bWH9RxlVHdSRL6?D;CxzjRszieZzmtM>>2-3mBN zQE(p*2vx%EZ!6dYsq}k;r-Y+r4jJ)g*8>#IUS}C?e(KY0 zY@X?(64Bl)TSM3FotmvzUZ3d*m9M()%x*P%zy6{ohDwN@Xl@pEn>5!%X}+wER=%y2 z1|`&uU&qKGUws1+jd+hS0I?xqc$%-l&gdL%(AKrWbEl;ALc_;pu1fa;4lJ??AL;+O zg!6yNIsZK5gRptD#bm`D;(nP7X~ascvT}m{hedlo^}M2^`v#7&R#}T)7w_smO|+ag z?E>&Q>lY6^+$xff`O{UG!b2OBx^@{mJ_HN=~#ZzVhZr^E0Lw#E7oTgX!xLqWgxH;jZVTu)m`w=kq-C7jVRnDvGPb=h zo2tI^@=5y4dorJ^buw~o{k^)`#`?n$Z+Iu_vv8W1&u}n4$v`Kl>S0Ks;*4VekWb`z zu=g^oNI+El?3r*AmF*T!fXC5CFYr_m0ugwIKpry;imt)9hqasI?2%dj*Z`c2%ftYD zC4FYguD9`LeClD-niBK;C=1DphpKh`Rz-ST?SPTdaRi!n%_hFE%wd=6rEt#A4rl_& znMW}$7aTyw^>l;h-(e2UkhH$tLEajpBIKqf4NFu=U8qRvP8Zzmla`xCq8OpkXq#)Tc=VBbnV($Y{(%TEHPJ$&OWJET;b}KrZ(#I z8RE{*{!0Yw4%H|Wm5UPrc?QVl-i5Ie;`o~+VB;NUm|7A}`Q_o&oIbYMpz_~$uDVz2 z7~v6!@ic0lP>IqYvmCw{S6Y-m#YjwjwXHu$wKM1X#YxdW5!gf_8P}1 zCfH;azUeIfi;SJZt2FOg%-H|k5Sj67esw%*V8;vni)irqYw9Yu{B`@zIL7l(Gd;Mt zF-Mu@2GK9U+?`Jw875zJVrOyjdhc10XAd3irQ|K1nm=FYEy7evXK)fvLA9yu#DmEk z9bvKg65Pcy5V3VxH~VhDdiv~suhKY2(b~fvlPNP@UKJ~MtE2B@f84Eb&g-ch5Y3QgOoR02zwY3oN}}pb6^N0TZp@u@<465234(HgFY9 zp_(N}9T+837j7X&L%r+0&1etca+HFSJz#w;ZLO+$V)F@7aD=LO1B;J3FS;O!*@6U6 zl}W6fj%Ql=7puViBwkPwiVz0lc8%{(m`yabuHVoeGqTSdC0XZ0`o3nnlkM7uTj*6w zFeN9O?mO844{Dw&X*?v_(Vx)fJe`}j3ZbD2InH5K9GA&qPbH!(w7qK|)Jip>czCNu zbOyF3bb?SMWW-G3LcLXM`0ZP%4JTKIL+|!rbd(N8=qi*>Ci}N3d(3^;Mvuij{;j>j zkmEw;w@X#uXRx0+#7P}JVC6}b4uUbaBe%+NlG%i6`i*QB!U_Xv_MY2B^mk{|B`$4Q zf>isB6!Wc}j5D((Ty>uEcn1Wy$9{uVuDsl9i8gLs;Xz@{%C@T33CBjr(!dp3$i_zJ zkKY|X&M28o(^xz>PkX$+&&RkqH!t?|{YWpFBbJThzX$GlH5>-Zy8UAngKoSsu(V(< z`lAq$9rz%Fc=zw57ZJ+T8s|qCTqa*S|18213junSpX;XVa<{8x68aK6rA9m4JMUGg z39BQNkKdbPnv7D^N`Zq5a1SXQ89mNkdyZnZ<`cZ1B*O(ebaw4szA zknV@@Xw$_U+wukDlIezqDZLTjnX929Hqgx7Kc8gNpkUoXFjnE>kV9o+g`F!}KSip1 zrvB3uI>(1SkC!s|H9@$M{5x6YkgbVp@D02*ZUBG zX*Y&d(B|?VEJkH-m7J|izu=Y1LBy(M?9X^LhzDL7De76Km2MgUW_h z!YYFBvJKqRIm(YbQ_sOef6J<*D({oO8ya_&hs2dR8iH8ZFXH0(-Ky zCbnL}U(cPt8F;rweMZ0c@~zH-`zB&Bublh{)KV3q17xAzO-=Z&e3S3GE37^a^{f5( zMDYaJZM@R0HT`;YB9&HQ+{SrnoTo*i|jy^H0WBM{NwM< z@GzFG)cku_tf?6?HZ!i%PdU&TD@dAadwc#I;^Y```%~6hr-P0x_bGGsi47RZkw|(n2gE#AfOQ$F)B-NVc-vOL(P7<|P z?!oB276b|?YBD=dC;MfaRjs^s9yCqfMW{?;TEb&%EBvvKNCM!!k92>iYi0;lKgalp@zh=nTX z8b+m-_1ilB&rU9K&veGSTRf1d=D(xZP0vJ*pN@9(&MPi=um;tVBb$#R$L#PkC`bKj zaXuz^A_<59Q|bc!gfrg$@<#IFErC_}DumCmD8Wh@+iRWdyFa?L-G-D_C1V|15+tuQ zO20#&hBZf{LO%lD&R;WP@r^-bh~xJ-q)DjEK8JNx_aHYFbQ`$wP!5lv>PS(h%8j`bHoU|1>dR`A#0i5m@FPgPJL_2@!bRF zCroh2Mt%bJXn@TeZJ*_DS6#E&ZvIp!aFNx-1T}e7Asp`fCpe6%V1>K-c=E?j&Bnw# z4=`o_T>qsR@Om4y>dE{o1eHAEk@v->H-Dyq-xF^95sxKAQw16lhl zJ7U}ESBz3n!{q)n(Os4QxVEcg`#MX(VK84S*qfo(B}%42BW0xD|C>v-PscvxDr_AZ zg&dLYW3l5fzc*;RaEv?|YioRJo;}wun7?1HFk*3XY+LWi|K_X)f{rxwO3!%Oh&Ewt zDqUrSEVzyUewK_wnm(0x(Kcz}ZVEMbiNscsu>jEe81!>!pyXY81a?r~`?T8h&fr zN2@nhz5SMhg|zF&)N?Wj)N$*`YqmdwIoS{v~P6Ecu!&x!0`x4rnorxo#L^h7RRC>wYSo9t~!YYua%3ubykbPNZ7byw!@qm%kc^K_2#M4x|&@r|;Wy=ZOvd5(T`e=w=Kck}YP{cQ&^OUDLkDBM-qhB_mGZ9NCQ zb+>a|79a_a2WbI{;HHF7^`=?Efnm;qSN6#?-KP>>U66&Z|LlI&IAjW1Jae|~=WfZS zBNaGDF$4KfAVK={LLpsgYtC#mgUiN>iOKrH@qAgQiFOu=L-W`~g_CAfcG&J+` zHU1!M)paC4{tH;NLYVExa>vLopSexWx&mdfC*RM~OPwWpL{@Ymy@SOj>lJ;P&syWZQP4)}27PE#enPr6f_piQ`D$RbX zIA>SY^XTO|hV{3h?;cY92zQ=7Q@uOaA$u{z(bFq4;pup%+e2*jge|{@QwM3k=U$0H ze z1QsQUd!q$d0p^cS!2z{{U`KMPGucB#94-!FcWyemyK0`%xEnaBXYP?Du%#AL>$>uw zu(_mp|MXL4*H!4jWJ2ZrG~1aJpovILNts0-v?Qv875#g44m$xOX=l;0UaFhSOU8fV zp;pyIZ_%fo8ps`~)mqcLHS$gA zoe3d#*f+BLKx5P;tY4jxAr@I{Fb6wp7+@%qo)3i1JwMvGCn{PZR7dG86vCIWWZ05f zJusF@Vfiu7bpRP(0wb5u$yH%I+|DN`z%W~~fodxZ{=IFjJko`gq{bn{+ zlsYsS+-D7;6#{T!KDsPoj=-fDmxV-X*a&U+tvp$Y_~$Kg_R9OIGu2rK-`h<%6&m)2 zTGwQQmt-44PCG(ea^hBUx&wPBTIf%yY~Yej#hqXC3i9hP2W2Kqor$Z(^jg$RGbgi$ zZaF+T3B7m8G{*i6zu8-WTQ;y22%{)}L9@*2n{UX-1kv~ODn~!H>tchptb3>WJG9p+ z9nInGhv$dQA;U{Mf6sT)4>{zA8GqpRkE)u|I(XBMp3v?_LaeFa^XAgoTE8HILVN= zC8MwXQpNRo#y~qph3Qq9I>)T+c2)@k3M3Y4@V{9AwYmF}SIob)JPWY>&Y|ccL4m0M z=dv%swwv)*ob8H6{7IdC+cPU@Y`e3}GAEn7T5y*>{fusKclZIqJGXb=_m*F$b0>YY z7yrRj*K&VF*4-vr1rhEs?K3NL0r2SyHf|RDeLlx7BY8C=++(`MPG*oTA>8*>vO+bn z&wl@xbCd?->32%#*9uqFdjo|fmGN&R79@R3tY81~T0{52$VHsaHiLd_Gqvh2{V4-p zjZ;t`fRzRnQQzvUYR(FQy3LOD8|^tu3AJ2O@ALP0%!}XMo?d>*`NJ9m`1PlHC49Ff}pbvohrE$|;N$O$=CJe$9Kd?j&s#Bpdtq&rWU9X>WR+H=<0b z%2(E;Y0cN#tW(S7ady77#{DjRu$lgL6TK39mS*&hR7hI%g!h|fsh&jP^TE()t{qBl zdsl;EUBG8F8t{jX|Jv&*Qm)9&VIHjw`uVwZ9!kxWHnuf13=z87NmKd7jp}nlT-Hy`1chO1c5|#L2qy90ElKg0pscK1YM|adIHShdJdt!sv_pLLKwTVbK`8W37ET`*zJRY#rhvG03q-^5mWT!UnTZrr;LBL1<6Cu1N!nC8 zkfVuh>^O$>5-IfKd_X*OI*HmiXc1tVTQ|aKd2xNzj|h+;NX=schO=6|99oyQpYW>oyMH9pmFDb-Nk%h(&uPlcctS@ z=wd{#n!eKMgdvxz<83kI1M6)uzk1Cj-l>l(RI(d++3OcNX^uEG+Ao}l_?6Z7?S6|S z8Lcv+Y#8zFtbjhnk>Gb*sm`iK*a0rNbI$)>-yK3T?+$DaX4%IyxYv5U?DG@Q+DOUq z5uZz_xGqlCOcG6E_iwFQ!R5^)#i%K-+xM@CvfKHKzw#??+SRYs^Ix}H>Es>b{-!Uy z=V{xsQaF45vBREtUa{C|EzdYII$R!B(4M+CvxYo9k16j>6?$lpq%pE6u}h-|&)oF& zJehta0cNJpGyN2E34KD6aJri~xMeBIr{~)>2>Mh#1Ev>!F{beu6Um>GEJ3W zoZE9n8Uoi(t&l#Qj%;V!8zo=g;yVD$a-WFTiAVyfvLFAquRCQbwsiK1QtUZ>sZ5iQ zbwH=UlEhwxOQHA1d$(tbgO|-#WO@#Kjvu53CyZ}O6%DY89V@Oh-a2f_UM_z-FIdL+ z6CgEdN5e|cYVxTPpx~^k1Dq1_hP>K~Oe~Vv-KWY5{DFqLuGod;+`??RoR5JTaGIte zx~fn{;!W1Ke5>oJN^l zH^w3j@NO(-XhY3WriD>QLGEk142xsei!6*^IoK>TZ|kcTI0#T#tS0M8V|f`V3rGsE zIKhq^bJhdy1l=v0L$%IEKo*iO4%!x1O+o@U=sK&sWnjOlczLkglTETgBfmWj%mcAy zTD-vWZOV*|0-*uFzJ?vQr-rdlQ}zIExO!Ftv;ra7whh4nY=^pOWT@yAWx?wraSM#& zEK3}^Fkif1VR?)BF>A>2Vrx0T9nz^Y6~%Zk!jDjHv0Qq6Mrw}8Nn}>HyF|qRcmKU<)Us#%x zJ*6eY;aOh}!8>hf#mqEx%3L;hFDd+ty*jLu&(nTM$>v^sEZZ%{(?@^I@1^#=$G)F_ z&s%gw*IsT=-CatV=2AZJY7xRvi0m->^ox?txt^pkedha4I+i#p7mnn90zfZk>Dtv@ ziY1TEHDG!PJPr^%-Fu-Pa+(R0fND^gqclE>IjjJ?JD(_X1t|CdkZriU)?ww~#5W!Gs?j&@N)5Vf&3kV@P3_}bh-dzW_Se&|x>CpvLJj>3`r8%0&KA@7&pnBSa^ zRDT zQa*n`ucywBSgx z^d-04w-F;l__@?eUl6q!_YFp+H1S@E)ZbjCf$Mu10@p}5-FAdCzZj46aCy6@eUNbY zmx~#r&EgZ2V#62jMPv#RX%u0WV&xN4{GPrfxQQ|HH^e|LU1Hj(amM-Y#f+5OX@h>X z4-&$^p#z*&2Bz$7(=;*O5B_SJlg{Iyl*@R+TUN&er=KCm&AdwjD9=P=F_S>-UGzTM}Q|@Qx*{?5CS+2LU0dDm0Qt3J*_1KFC-k`St zJ2?y)Lc2S3czk#$lmRETGK;OtCHcM;XXJdwj9&R4dH+hJ6!{%WwN_1elK9j};L9lk zGy1x>hIo!n^Pu%DdY*ul1x6cNS=CGR*bm`ZeWRD*I>{%C8xvDGVi;v9EC08o(L7Ma z1VRE}K!_HG2^@cXFIWiKkBW}cQE-afU(e6pNW(-!j=x5u+W^WdSdR1|QEQ=og zG-kQdUPT>sZ03KI=f|-L*@q6^j@=$&2^=WMPSBt;u9Vnnl|^r8_bQTN{F_6s3 zJDB^|^qZ~#A5OU-At$TPn!ywI)*e~p4CFARd~C>z`_0z*gY2mBi>(eZwU&|cFtTQB1L2u@3V;P|goh#Fq$(6Jw zgCE?wS=R+~Qh4Ly+@M2U2+{ah8Nz@=f*e%{)vbqf0MR2&#jhs=&x>_tsocl+{cFunvoh&WzG)B<+P*S)P5n^W# z!025)VFe+bAV2{D1$@0$+Sgu!zB8%8WpN+6AJC?Tf$$$|sP|m|`wyf1&yM}SUond9 z^&mCl&YH;QUR$;Lq>psA)ux-SbhV@ z%3?~c(T+&(G!e$CDV>9UNL|Hwb4J%LH~E`e71!%lYhDdBk$jQ4tU_hj`=VA6J404u zY%gpieJTcuBuD+8jqMl5Zq;6D7)g&8O}G#QluXG_9rb$p)p7NEa8F$`vWhRH`;Cw! zXGmoGAXC?;!k&$0)8_A;F0qX1i1qjyF;24Pr&&;HkCBf`#cU9p(7;0>vd}~e4g3=% zAt=G)XZ}>EdMXD`+!iHu#`W;MFeDfDPhF2Oc9Q>U`*C54V(=6mycocDfXzeeE`c~I6;0udTNOYfETVAb6X^?ZeV$#h7@^J-#|Lr{ zhzShrT&h3#EgoDssSk$#HhdDAn>lIYN*n%MM_`;Bph`UWI^|h24{?Oav zoRDIx-I0!J;MD2s`2YP||9-Z4;SOrmk4jWG9$M_(uoNMu*z`!z12;#l7(7Cu2dfs( z9xy2s|4r=={5DC$M{F2J zmy7b&j0~n&bOrB4T1+of1;L}cCm7%62ia$MsPCU2pHUjiNDpXT=`jvSxXPvW4i$hA z@u9%#ZH5sm6@U0`z$f4(nz#8Wf@xId6VayTX33wcgiQdV2k2nb^&|~0kW^skSMP6# z2I^F(6R7`*%(aocYOfYMh~h>`NEcNwr+$m>(rt@_72D$ez^hCDT3~?Alf8(WL-r)E z)}=r9`dyZOcH9c|_bZO8tav#op<_ZUJiiPi{i{8zQ?_9Z6`dF!x8HcXrZ`u>@i<)V z1kjVbaePE)GDBgaoLw!2&+H*hxK~Ig*ZR>ix6u~)`Q%opCF2?0maXjBw;5cv&zO;G z)R3TW`If*#FN_1+)(Zn4#Yrfl4!%%Z=aOanLmOyYc#lHQS%f!UE~~XctxOq2gwaCdXNy zS2R(Zt&4;atSzLj;0eGt;kckRCas!CF%kSV#Ur)nNYV>wn&kv#a}}bqpFg>5_QfT- zMRol>t&hILTPSJHNEgX2Qyau3G4{fl>iB>Mdz7_2fygR1mXOzJy8id+`J?{Hy#aWC zkmKLyZ|tWkzgP;BTlea|cyClkxEAWYNzN=Pyc*P{clJ`S(Gi{bV@JVS{X8l^5iCb; z9D6JHi9#t-xhYdf1V4(u6gSe^^=R(>U!C)1oo_vlb|~1OA1LIgmAl1W4^#ozdXoMp z9X`cjf#v@)4WzyPE~24oBuN;p$JZ*EF%M+6*!yJYU6|=>46F@}F>8tFecYmhl z&Pg+26il8ZsK=*IJ%HQxYliLXQ3veCQU9Uhr)go#3G0{*XXb8zQ6_EIzxOS};$$>U z*p8i>NzT0{|B?Ox1pyKOaLz|Gv2*cflFaMilcNq546OZL*5KKS#kuW?@jAulHrEaO zhy&&%HzZ$VT&7kD^m%2nv9??Dq9v{+EBTQd7F7)UV+|foep975&lU%zC;?@;gNS*& zuBw-Fo&v@Is}sN2eM9Y<7^x(13sy{Vw*L3t(iP>quXo!W-#la1ty;?fnGNuU9L!Sa z0NI(K`wvcnRB1XtsF(z9S@SvrxRf@&Gy63RXQsJs_0uiC?4aj_HdkJ@d=k{c+A<)Q zF9MGWuC{g|HE~*|s^B*La=z74saO!=Q&4+UKd8StnE<4F%t9U7l~0O*rWnfGpTZXG zOk{`xULOHpQ~hjF0K-7g79dx}Go6;(poxF+4$))mX*DmvnehH{eamlqw5l?`lksQK zw5oZVi~sl>Ys7WS>yzzw1};_Dkx#?G#2mT{jpC5~)P9%1!3yb|{;Wi--CgQkbK+Na zX+TO@C=$mh-rkHMc%%s{??q&yNkmXO=MvV1TxHwa;JM415Ca- zw{QI~9Ql6@0rcb4%+nW_E28dbQs119D5+CCNU>@>oQ&fkz<97%Mj4KVC9;Q#?Z^ct zRPf#eWXcE-CM|9#>WZ~E-Vs+um(ZV6qAdP~jzbgQ!bK}W;OMj*9jXnziTCJ%s2M#T ziJxkb8lM^bsKKfc@E5X83(VZAx!33CFSLAMgL+V^9cAWsD2h&-lL;F5yKxQ$BGu`D z;#}q>BQ;~z^F~Ty96s+#Q_-#;z;B%qI`JAD&=dw;a9omjFMRe~ZsGmv2V0}>pqumb zfpY)2+;cxjBH$8)?StLa( z8sEKF!DB4(W=@-+}!D^th={teV2BcSj$~0Yd_*(s2TAx`feCo#{N}c#q3V- zp>sp-xOza9;@Nr`Mk?oOMMTstfSND?Tqa~KF@GyRr+QbFuCN^cVH`tq5L&dgz1d27 zOe2IL_qw}Tc-v|me+E?mAI4|OJi=_Y>b)V2thICJXmUqCuw>)rt|oX!I{>p(Rq;1L z61}x9xZ|O1ZQ5lVEBScj@+2YZHZOc(??UIAqlWt_RlV~4U!<0Y4O|)kXFP4jkj7*< zPJD?@m7z(04w_r7Bpn?zKTEM74ks6C1LpiF7MaXV;H8Z;sw3*zjpBZIce@efeob^R zRxv!=ig9G6zVRgUZq-BB=J$-C#wPa4>pJR;rzoQw%*!L^9Lc=Eg76*rD~J4RH_@&7~*wI zsHHeRH~-*^cxz_7&^n`@N~|Y5(xp2}h?00e)Tnw=9L$NZdYd5!9-)BMW!LJ_Vrc1sm>6Z|G8 z^*_(6XS4Q9%wT=^7o8uZ%TT6gW*dX!rJNIq!?S`y2Wuy1O)zi*=519Fi7b&jCz(H| z*yU6<+XaSoIu{<_BRR*CS-qjf8VeW%;AraXJ(>A~}=j zX!CI+MqNJOcv#yU%R%<`?$n4?k7N5q5AxYn)1eDmmM;Wa-cs13e%VnkC{+S31ZWm4^4ADV0q{1~LXoCZv`;D2*PM>w0%+Ag%wf&v+sl!+Y&8!!l z`>*m#lciB1<_AkvW}L0`69sKAt?y|t{xQxmf6Vy3)UPg5Y^&k#sp;XMefG538lCJ` zOc3GT=J2i$k3agmeQRyNibvO%kEjm@_|ov9vpesNF!Fvo%5nVtDJwQn1W?7Gu#m zRUDQ4mF3%@Aj08I@EGU*QtrO^kvsnMXJKBg#o+`oe`M6r2TkJ?mP_#0ReK~p^!sl zm|N4pZXPn2Epaf?wXIL!p6zGunJ8&jkXbjYFfSTdrhGxA_KK9y`n%_<3Bh(DYY)4- zeBaA*@mU;8d#fosa&x(AETuht!V^flhI2;T=k)$+e=Om1&n~HOW`Q5PTG03Z=m}N* zf7NBDYzA4u(9QTF@yL|5-?j2Po(Ebb>;t2;wNRj_MvC&sP0aM{^J^${f{?A|Bl^?x z#z-VNns6{0%DB&ZGS1rgp5ywNB5t`XSN#P#jN{>l3*-Iq2S^8}(~)q6nTDT3O=iv$ zUQB<@hGVa$&@ujz9hvyCbatD#G=uW@*`F-sMKKafbxuZ=zAN8~ zHOlqB@PUq~rOH$A>bA@oYu|Lhg8amh|M}>GeJM%Au)x3M(C_9bKevGLtRK z!P6O(FySju`C<1|*Pm=FPNQOO`Mv}pkhUtCzjQ?49-_%=Dnmoq$G#tN>MN^Vx2_>e z05}8O0~9C*w%lkpHJ*n8;)n({&Oyz3K9;|o5Bm)k*ull=uQ8W0 zFBjl*Uxde67ZrM;2v?-qGeG-;z2S)fY~H7OGlvI0^heim2SFiezu z_$|fywL2XkHk)kHzkPpOh^fN?)iOdY?dH^Wuw*OM-;J?-T%FhCa4hI~4M})F?^n(h zIICC)X>a>4zC-GxPDu4E-Ev#T3~fIe9%qGVr|pmuo5)UV;F8SuVtlrW*UY3ig$BXf z@=s_l*m!0#1oS24^|pnuz0^&93fblN#tG(3&RjCj74Cl^Zd*+ld~E2ktf}GFJ=oo9 z{bbV=W6cItmCa`c`40+uQb*-pPGp3JH}5YOW>H!MBV>-{K-#$cARLn#krT%oydXKE zlPUnejQJexj{BGp!_qyfeIBQ7^y_MQ=b{iof<%gwMk#A$K z;oDps`rKa+l1e&FJ_a`H?Kcr*R*S1&Zuzjq4OjdEUK$UxdRjkdpBQ3^pZl(o`dy1J zN#;4RK1MP~(%ebuDG0q)L|vXU6^X^bUSS%kdL${Bo3UlNvIq{NpT2e=o15$#vRG(n z!ZaFRS5|81!*008wuFLctck3OE@gUZH^YYFRw&w@|3E2j#w}<0F%5Y~zbb8MU7A3US!woD1naCj9_!Y>? zM)g32KCx2E;7d}{{`#jBg@sfOB(Y5HKricf?`2{v9ddo%^hu@_7lmQ>v-xv7y@nq9 z?~VTV^;~3gsGAM|?OtZyjf7PQI;+Yf^L5h*-j4Uy13u{gQK2g`lwCMCM{Ak7n?%or ze$pV*Pn{n`iB^=$oRm`;Xq>srWsg52U-RFTHBfEs)#$$0yh=YcHUb{IAe6vgTuanr7vTPjMyu!Ggs0o$Zbby7lV%(ZxKpyQBR_*BVQ&JOOY9~y`%^g;`rrClpn=t$_i<$B-;_=O=grA)@$6z~wsuo_nY z<3cv{$=n5;4t6W>wOBTW81;N1y&uiE9^HPN-mjhda6-K5s z5*aYGr>d``%XXHTpa|i`YS&-l`HE~pea`yKS-;r%Tjd~LTpSO|9^oPAvrT^9fYip< z)lr5#7>U}ZXP1r9G2MIptCvGkfzlDn$s>$_&w#q`Q1>wX!Un{e@5JV7toD#jb(5#^ za^9>SmG*Lck~dar#0Z?~K(X0#XWlRY2Oz5Wv#q#h{$Ea^o!a~Zgm2%UVjIAUJ-6EBg%GV!a`9+?B44phM=ja--t{ z9S_trg@V2oR=r`2Iev|*VE&KA5&2I z`_(wwr2pMURDFoNPqV8LtEcAFCpDg)HvDJIL$c>V{G55(Vea{@kd^a@%#=vKo^wjy zq{Npr;fIf@^O2$(<@d!uP)9fN)cjgpjBJ!n7hhkkHHGX=3_oSOmoa9`h?JE3$4^VK zNs#E4G^%3KyWu`_eCp-*=E!*S|Dm__A8E}Az)p6Yzu?FS-Sv3DqqSP2r=g;9KK2$CCX*T5&Eyz|wrm1AdOlRnZ>Zb3d$RFqs z*fh?!4Q6sJpty046yBxTqNPUYAo(0?!ttHrL3(b*r?shh|7TCv}mBz=b=aU zBoDIj#09ywYqZt!QC*yCs z%v|)5ejg)|^t!C}UISGv`MP(5)$Cq7Pp0R{K6tBc|D9=ux){lwV*>lC)L-*#H|<#S z&Cz}8d~A|%Zfya4Vko+AWC#;;;XdfKFfW4fB|q_KoC5jHtHtkxh??87i&jNG^|jJ?lvC`OUf6qh=Hb zCiM-P=De)XcGWE%sF*6J%k1`_wi#UUP4uajPhL*^6noUeCgL08p7F5l;2X(CZctq2 zgs`l*z;I9kagVD;;sLv$Q0Pdo2z-PK1r8Ia-dyS2BFLm*=kt9*n;lVqyA-Chm1(_r zppM+2Z|Y4X4+$QOOkJUBjS>0ptK0M)`A_eG!XNXKops2EQIdIAYq%6{i|5B}ZWNwt zfDc<0;6Pi%0{L9WJN@-mvuA612BrqMk|P?Q<~%+Gp;24A4t~qZ+1Jl@yfb#&a)U|A ze!86~i{R8giwMxLkMZUZ3J%4}V8H}9kh|cy7IgcC0j&2%mNWm`p}S3t zkk?V2y�a&-1KDNviVdHwJ)qszrL9#l+TpC2pG_&||mYG@33@$%o0^$O}x2P@5Mt zV9@xbEiLk(0we?@f8m#mh>fYW&Z3tI;>I}BJ}3{-F9%MFL4k_BR~QQNbea#k7kSop zchj|v;`4{%&&!M85d9-zPo6!aoib+AGbJV^XJs1;YLwhI82Z@AA#plw*_7C_V@*6g z2A>8>8g-LL%)k~8Qd{Mn6b;Thb2AAYIGUe$*$)dwhQ`nD79KDuNx!G#4E(*=_^8;H$?kYkv+cz(YYbJRA`-q zT2x~g_eBQ;$No6JYpi9hyn_UC5yP|iSZqs~oxw!rutztN7j&R0Lq@+Lu&_Z+r>&_L zS^L~YREfrScq8yYO&l$^4%$sp4{^sxYwxd~<%`@}d|B=x1}%H#4igh)3&>R_(oNlo zrV$0mcB^jGh1VRm;2uqp3y@-FeVCx6wy$vWUan0g{xd@hgRvw`u_`zo5|F zlT3IADrUWdYWt(%ikQa;C&SMok9EV_mQoo+i4vVoU;!sIx7fK|kbg7547y-Q)B(?W zg7E`tk^jUP_7V=gJFL#fWeOM!Jc_qTOne*m47dF=B|Z!c#Zznsxy8zFE1{^vj38~2 zXRU6k{yuzNC3RaYXPl)XV=aY=ud}a5Zm9^p$uQ>|!qnd=sDowye`RmiXx3~JgyJBJ(MQ;@rUGnXwPwW9@xs=0iN~Ou#(*YaAHEF zHa9|b88wyDFsjr(r2H1@YS+=<`>dVRuN>N4!2Bwa>F1-?h>TlNO^MZkgb)WxNSQps z$gt(WSp&cE=CbV0%L8L%b;I=Lo0`fn=2~^>6S=0i=4xUSEO8K>JR?rYQ6_zbw-pie zi^nCgv@3~MKdBQLOvT_*r1?}h@_IaqE<4bf>4vWHJ+0)#VjoY_2$j3 z?VT@`^vbzJ`PAat-si@$0}`oDp8UBtLk&OhYHf_XOVu3|d5YfDi=CVmw;J~jE@K)^ zHtD|ZXL|l8diFEB+4Ltr_w@WwdemCZdkyRE{+QLd(1PIx+rQW^Nm)3~8w>SjE0+)f z->rMT>9LxBb>+QyWqKB>>63L;=YLhV*L*a%?RUMx;Gx$z8vny4IaF-+Xh^yZdV<7E zdiFIyhn;y6Fg`xI!{JPOwq+n!0FI~CXq2M2wn)V_OlWi~`MM~??}5NlKCR5B#jAz7 zt1Z2@*_OMT{q$N)zv{tP{y(bXmuo(`p0(-0u%OCJ+S3b$yA^-eyA!j!36LoCpduEg zVd$#_c(kPG`7ymVzieEayh$`_w^N056O~D7-zTRURjooRHNyPKvYg@e)onR(j4A~_ zU3H2&tWPXI>_u*=Yx61A*Ef!$eyRC((#PZX+zwDrXz%qPt%s#WY)( zNUv*j7+&W~+J>A2H|9Bx@;dur#;k7liqyQRUB2sId_tdGU(*CTKkM`8>6U-6-{S)J z`kSMzmo^?B$qlA&L;U_S?&kY3g;Db+HAL@tuwE&wj{gs#FBj#%yM_Kn8K$q0blVT# ziJ-(MCkQ!7Jpkd}qg$`BPpIR6j^6B6Pv!7fYwyA9Ud^`#H@@=Q=)P2)<2k(uturGI zre+)1t;(XGj{V8VVCbPtx0|d_RrJS)C__bZpZ42T>ZMWsx1e6Z3hCbSts%Cf+}>VF z6GI*?N0{4G0QJKtUrdbA$m))b5hp`^tu_nJ55Q@qsv5wVQ@bZC4X+|bEdOLdIqyg z(@1V`j$@5&s%LK?3jHYMM59a9ELDSfFch$Ckd*y$(k9^3UIt(L?bzfpA z6GW`u|3Ree6-b&LwHzHNv3Kf@o@Y^5;2&>slI7r5jNs^aRh#azX@Fi2T zb%Hez@`g~`;R5D7n@e7ICwPTo9!&TIwXB+Oy+I{4h*EOg{4%WqXZGspPH)M=ZwbjT zUH!hIY^FjAU1nm5qPP7e;uZA4bAdC`|Gy+3<aeaerCKuqvIvM-T|+R z^b*<4uut*i;wLh@ei7Qj8~T6Wc)yCeUWz|DFex=f%S+Qlop=*=YV`lEa__%rof)Dm zCzu-s%kD($S>ou=glqNUjjpQT3fuCRap^S}7xyZ!AYAt#y zi5A?NKbKVprtzt&jPx3!WnL3SwZ7v%h<2qXYNqNLbc2pADRZtEKa)T}?^j*zO%qIW>wst!E;uNf?fsj}bfjhNWf_={ z>E1*-`^1tlko3#6yw=<>`nc*ksXGe<$Y*V-2(9f>l!6^%p1Vy~A^ztfwIWP&faZ-~ zSG&@fBBjX)xuuK5pqY>y!| zU5OySyLm6Nn~U!|tw6*5=JwtK%4S5>G!moHP3K`ugMIworS@-07vb9`XoBgs;nYXr z#Gw@RVJX$j!_f}`YM{}`p4;(!sfpDpY> zXXhpKNaLe~a-~uq(~9$YVb2t-ocP#>T5avb4#USUZzJd`yhiQJeQcu;{kk+Y>TTG6 zH~U*RJ-|$$a1r5rt{Km+H3NqzxCYhsBE~p7mw>f#SYwrr&zEwugfDG-6Ozt)WB1Ew z=A5GEch~^5)bC#0;x#sr)N=2scXhv+W*&Q~VdF-7<<%n0+9my8eQ*7Ctka$&g2qOk zhT>Oge}*P}t8Z&OM&Fz8yzmuDqY96K=P%SDbg%)u;tK35yOXPx(7`PG^Tj( zd2r2ix+5>!ElL4(!dilp388u~_eUQ_S&Vq@${LqBxA^rRI1;f>X+XM(kk8xd&f&9? z$JsaChjrs_HAUOJsHVnS-7D&M#I?I_=>{gfbP~c-l3jwOy0h_{xwHCzd^4CzZ;&dJ z&Y9^gyn0>_S5sgql-T?*fJM9fycO*^fCc!YhfHA8fjtqCuJ*lGWSK3!6o{`ju0E?? z55$@MqT6(DrEtY8gS%;%3heVK@H0kOb(^C;e(3DrC<{Z}3n%ydZ_4f_`i8MdcVz1^ za(xbS8TiB7#oOIQcMO2Qe`a+gNKFP-*2jteJk`06gBW3%B;Q+$w~eVm-M|$jWMFM3`4DMx@Wc# z^K0JONisn(jUeR)^mn zfBF}^2FY+Ug1K^B^yxJ&UbQSjR%F{M82Il-qWoOXzAAyTPI#zhl}`yVc7}qTa3#ys zJ^w=r{}F)fieXdlCFVQ3bHgO9-Oakzxzc~Nod@>!j=5L3QxVr*SpJkR^H<8IzvQlBFJ zP{6O7`VGM)j=?$Qk1K^fRGgM4k*&>e7;C24^HBBe$Z!!V!OV5ZSTVS%7mxXA^Q)x` za4HY|>pk0r67I#4ieN?Lv%kIO@cHz2Z!>>6bT$2LD?%gVt7ws>=lg{|Vy#UnZCtla zoQo!6{Uf{I?n?U-SDKnG{=eS#yloHnf{9CJDY7U*1%3$L=z57ZFZsz8|TfJU$y#0U;0P-~x`sxQR zAfAe6tJA%Z0zRF4+WX1N`;`txyeZ z{0n^OW54tUZwFVgfuvV>A)JujANKzA^gTf{J?3)?1HKJx>~)C9a4@zAAIJ!{0-8Xk z(9{MEKsg8$bgO8 zwoUdwt{Bopym7lv-4fx~mALn~3K;<+W^|;17l4FwN^j8T&2GQ@#FQ#AFUu&9r2=<3 z(6F0Qw;WOHwP#nxx;Xc-tz#Ee(AcuKZ(Vt*?>$rbefLt(ca?VEaIb8gPNI*isjpwe z?N@Wcw&UC^U1b+MVb93-^mO*ApPRhoC41FCvEe}-ghH2VX4?#qMKhU z&Fdm{mZ3_br`nGP7OmeRk3F5F3ym6k`yGNFO2;Nk9s8to%U5=tzqJYI$m1#f^peUj z_-k6eJ3$~fa|^?6;GZr4y03#J@KUT-3yZ@$8imB%7BGXuvte-T(Fcca3 zWz&TJCdTuZWB#G2`4nZiKW`d&5*0oqf9Ob{yPP}ff*K5A&$joeX>=t?u9(QiJx#&& zlCA|d-71dd5%to6_MNqVNbSoBl3_M+u+Lp83401~-3gE2xgbYP=izTTE%_s{PtrCyN-lv|?$Hn(5e#BIq!%(jklm;~2d%U_Luq4&g!1Gx6o z?c6^T%($@KhmB~av2300t`^$@$eTwGu>a}2`bHEB1w7!BRh5R;s}wn+hgOC{6ag<< zyZX1o7r=yw&-7Lf^Ys zpW#-fh}!^H4Op{b&cT`S?hc?hgP(*AU|DgmYcj#Uf`KFRrA`WvCVS3_lVz9DNci=hVU}#YEG@h+{B00>{em?fWviJ^r{v9 zN|5nX%q`9qa1iEnNvL1cHvNo;#+HvxATL6}i@FFPd$T2FHNWaL;E8REl{)j)_n6Rh zhv*B=XoD$K{gQDQ^Rr)&(=g1+`QKQEbBt*ObTjx?%bQVUmtqnlG-WEJvSUvRl|g(v*TKJD5pQjlJ zsY&;n8C5&q056_*XA{GPt=J@|{BVe@S!n>?WO(~DB$Ej;W?;*uy)d<_LVl;-rtvrR zpI^>Bg*voPoOaS5+DBD04X9mI=6h?~7-0VOe~$(K_vHTmg)8%bK&&0&lb4T)0p^vw zNjNFzM@fH?cmVzeiVy6?VnbhO)z7W|^6~*q*akbqInpEavfd;-q?Uxen*Gafus-u! zBBJhlZYJUpu%wrfUjsluP=RZXU@?=}pr!&Rf)F9o^xg;&>tiM(-;BtcJ{XG(!SN~< z-hoDwr`h4pzcnlwsLVWGM&jpKYE)P}U=PpgZ~fP@@^#a={M|*5>QC=w?Zk01^1L@N zCIq`L13txMs2_e5v`4Vh?`ebW4D{HdE(4gDI?KF+9@SF&l%_FK`4d1NZxP~Un-FWbi6k2ME*!@@~>E{OYpdB~g#*<96mrQ9OGcp7l_#E7qyFhIA zS)DUo1}-y0B;Eq?w>3SeJ7C6`uSLm}l(nzO^0WhYtClsGwfW)WY?BAL9-10hIJ|NB zIf)HgHfj<77L@$Yb?S7$JRwE4lN8L;9Lr_(}%h!B*tv_#H_H(FP zsDC&V7*XI8j(L_71ORq8yREX5Ao&waF$oYX^ynCOeS&TXzOpO@<5!d)+tH!(Eavyb zmaWF-Ba;dd27m$2`!;x`U)=TqpZ6VlkRY%O%gHoZWe6@1Pn`oE!KhyVvefw@A6^R^ zg$2B8_QQG?tx)O=4q!*}6YcWB^ep7F0_E& z*mkg2Bk8>7W6S^Is`*yNhLHEL;{D1QN~jvxTOFfAd}BCJpX+@(>QEa$xqt6%NU=$J z@nASBCD6AT`>WMv(W4a9tN?uaLAi~lxD1mNOvpYHyG-aB=VZEbIyb-KFCev#qUTk5 z(k|@1%)1rA=FSSo*?1SmO{xz5tw zGAdEyds_0F;H>xD(JPo(II{kX}eq>BTSx)5SPs*^6mZO}u9yHfD zolt;XAy(Osj!dg;&o;sHV8MA~{u}f30L2x zen<&hf0k&I$EH2A9qX55_OLF1vKMaW{hAkkt2lY!%vi9u5skk zFNM3XMSn)X1Z3z_ilP_*PkTZl)dQfjp^46oKxWnwGX_C5@1$!2ws|qDJJeZh9CgqpVO; zEKv)ypXyr8_!wCVOB_Dvv1@50dODN@oociNX=wDtfs$5A!wkh{c1X2+ z?qN;|+u!ry*y^*Ii!Zn>1MIgOvujP~O+MX_HZ@7@NFfuW8Z^!d80>caJQoydJWxSM zs=IfpKl!eG>Y&}rC)|x6J#x8BOC*lVCy*6-hI1xIhLEQ1ag=)ldPgFwmR~Z9x8LpL zr=eaVsu1gB>l@E}k>B%~H;j+~Fr@Fqu{GH6#_sLiN3)5yo6&SFXExQ4?8@6e)+@K1 z`t^P6?{r8$LKZob%3gLz1OYyUhM6PE*x3uZKI~EmWfX#J+)RYC(pdO1)tLXIhnSMb z=@E|;B0!StPyA=+r2tg&>8U{I&D{F_w1G7^HEhf0u|-`7W+!prn7~+qqHNbGq|sd zk$csbavS%dG?zTk5yr+Swh#4NPo~nIfq6}KTl5IHk8OOw$u%{nKQ+KB(N^!0sI9ZpQ%4TB!%|76ixJ&CF?K$$)aDHu`r4Pa4Aa5nen6$J z?{>7GK_Q1iI?qT+Dt}9u9^{3CTLzC>`@B`mj)7l)re5|@a_x?yY`VMxWnLm^TPuCt zdRZ^9Kx;{_^%x|5tdOcIpLoq1w>100B9jo9-&WN2PLt1x zr5uU+(}`?e zZ$~XJCwm|xa*f^GXqREo3!U6l=&9$$W@y{<#k=;pts|RWbUVS{l3DP3g9>LAKrLZk z%f5@g*m1NZwkxmokmRpyxAz;)KL_MK5uF9AkN6prl6D`JwkUg4K3frf%xFGQv%tfD z5@y}j!xOljd0SAcqwXE=(q~^1{5HEI)0&vcO_1Yl&gbK67rYoV-w3q*eAsY|IXf^2Q@@P72$FDCB)FZrny~@hDjyWfIBLe*MCZP(*VlBr-mD z1Tg>X7~%@IBXVC-`3ywxEQMzHrI4;)Dx30ojf8zPi+}t?hChOe@%?N8F;@?hHLmUB zgKyFcIx`fAAR+tyAwR4a)m)mX>+!?f{s%#Ljhns?eFRP>8o{&V2W(5}-L#G^s}C8h z@Zy*m7F0n`m6VjWU&4nILo^X<_lj~`cWKTMv;^jANIU-@w!%c?N~?nHS9A46I1Q=O*fK?y7W*!DxyFbm2Y>z zl3uFXf7nM!jM5$~jjs#9@!S2F+O(b(#DF@~;2o_lXp`GYa~+~QOSo;kJJxKiG%#)2 ztv&3CJ;HEoWzx1viDmfU!{fsAZ+Y_^kKp|_GN-t70wZvl+ZhtW&mdQvlbn?q!_Yn4 z6q}5QgvJfA^%mP$8i|mj95u!#m~_#tU$js3DIw#;zo~NxcfNhnu15w@n24V962BA| z`XCy$#{e@H)s)_O|6&0wI)UfEgPi@*()W4-K_36&H<4ACW$4nOs5OwW|Gd3=MBHUgx-rke+m00pi zfTI21;()o27TZGuElhBSoE4HZZmM}b>Wz%zUE5N)4aeqSO2SHx=}WK8j{<7XlGL^| z{=+(TeNunJQoh^Mp?hFTyaoS+IO^oh1LaYc-Oqd2WQkCe<9Tc7@tC1<^%ao_>tO!! z-y|_0k5z2&umb7xPD;%{>i*rMnZ{u=-gbaJ7Hc&ZfbvPTxjqEL#X67?Yktl(=t3i> zzZhYYa7wx9E{gl-@HsX**((Y5+5O(OsOv-kvkcLPVIkzGc2`J;1MpoBw~WQjWt>Ph zlrvsa&sa;fPPkMDy84{<=b5k#)xVE(D;M`D@)e};H(Jg1+HaliFR(8P%AbLj85)&zc-_p_43Vd0$2p5yF@Uk3`6;BpmRUNb3OsS1{ z%r82pkjT5lj*8;5RM^{iUbLv9Ee1v-SA=;#MIFs&w|#DnHt3U*H5ICC6|U1c`)Jv= z=a2WR%KaZuQuj)-DCJPYA!fK`XsU~Yu;FO5K199;|8b8P_yn2ekP%z(#NQRs-}4v> z?EBG8`>sFD!<{41)VPmI+RTi%?KrqnI1g>=dHW2to)3Rw*}!}3{)viere&w)qrwdM z_utBoQKE}B&>2fddS7*kuxvVQ=r$hR;g!DsKiC09h?MPbUAQ!ViCPv%n(ev`j17Nz z4OTUJJ`vfm{z7my{?W1BOP@e*#ft{AYqyB>EK2nGPKRz$#J zHm{EMKEVN4<)0r9e5I%kaPNBgiQd05EeEK7ep=UN(gId{h9iKXRX?1tB$&47d(O-t z$~b;|cDa(;K5&4?=8GXo^4{x)xOyr+4E!L_1?Uaq8v~>PDS!Z+g5mKz4$po6yQ}T- z5z8paS}cK$l?vjG*M>*G6vQE*y{uG5f=i7%E6}V{6JSjbPJq79{(z zX}K^Pko!BUFVH8Mvu`dc99aZ%sCjt(kM`xOsA23ztj9o|OSW2Gh6YF#T@DS6?@BMk z{?!tdIr&!=GZ0xg3R=NN&o?=rCW;Dc1y6SorrnpgZfAzq4HTX=jSQgsv=Y5{=A`MX z@{4i)V<`*oWSc`U9L0EYq*|GQPllt7F=Hx>K`0#l*xE`j1MogRpxZ96ZmM3-Ho;dUPfD7~Ydx7rCZizxF$IhVt{}m5xYQ zM0#dG`#JLGa1tp)W_nB;e^zeuz6CS!@&UHRA8d+no=^D$luq2~&Bd&UMws_MTRe*`JJrrpl^@e5Dn2Ya`5+A5?zx|t$emFSwWe8+p?(9; ziIt32WD`RcJw-i9f*q3jXkI^w`0c0yh^5RjB3r%{p}h9C8ewu;kWs+?WNXsKs5Ryd zulQ!QAw2|cR82w|!>kwR+~-Syyxa^B(i>orT~t!ZI=(nagwwF64iNXM?AR zesww1x8~PC&7CU@UsB;2d~%wF;f_Tbvy%@gmJjY^rVoW zI-1xCz=+jTnNkdnn#FAeK-Co9;g^4S{d1TyMmvQ>?Zoa<;eD%on*`@O_?JD10u6|Xm zrgJRv(PxNoh|rFxny}SRxcQ%9oFAilmWre>H2?=9A%f&UmVykC$!?X8EIeyB+v0CU zHwz1NF!&tKdSf8;CIPQ2p_q|kbCAO_X?N+%@wT>Fnb?QO&SVww1j zTHSMwl(QH2n1JdRU8ws30Cdm=uI<5vmy{J@Rt*DpY1^@XTw-v1HMK=}sqSHRIhr_$ zeNGrI#5UW79Zjcf_oFJqw-KQ_bQ2zOKf`q^6hsWvF9)`Qw;P6h9ZqsyX}4ZY<}t`X zQ9WjhfT}&;>KbCtJpcP^n-6D5FpO^K#}ph_h5Og;Z*kdtU3_`f1)2idZ+c z{l`k+8qLbj|KbIC+5W^^+Zw5tpYULp4}nwllFUCCR6 zVT|!#MD7-{WWZ#{f4-YX{TQs5hdu$R3S9FFfMH)UeuGk8(hO%IBc_t@Mb~A4>kwTSmSp)1qX_RPC4uD|`oSr|5K|8nhBKe{1hv+dp zt(ay{OssUkyYor<%;pmUYkb++F{2)3Boew93=}<=-HliT5QE$lB4!e_L zB@xe-RJwuhF0G9{T~K%gFePi$!>+hupi%R!=MNyfxJ5bSP)#@Q94mg#x&LxT_y9TU zc*&`ZkAu}UKBb3_b5QFEG8Ok=$xx_0f5L_R4zIIMPmhtWnZxFf)J3nRF7hQD(VQJ6 zinofVbDGH?1_DuY8 zw&Zl}b9q7ct=efGg2nbFe8m-He1{kEJ@XruVxb3sNYOZ;RdhEbk!K z^jF)6eEaWzRmlH-vix_+?7;re&codCpK$%IHLbK!vS92vCM*f;9-F*JD`~Qwlpj*_ zKJgF(zIQp7Q2JB0j*bjq%D-{91%~el$3lK{;2YFcSyl0Lz>u1%*RHr{9O=&O{}e$Q%9HEM|ORwgIdbF8O=4&%^vfw=aKwqkMVTuWv% zs~nqOYhr~2B%xc5%tyI|TOSP3(Y`WOcgAJ{09^ost(9`}s(uC!%i^HRpR_u8bWC%< zs@J)KU;w$$mmQCmB$Q^8y3h&s;)Vu0c21@Jdh-K}6pVFZ8H#re3E8}y-BU#~Dh8@O z0e#0LuW-lr%FnE_qjKyoW;0gg)@tUI>ErCr3@$d&IqVF{=_|=oR9tgwl5^3#$7W|^ z;G^W`e6`E%+J52j{M+xuCd!iijkC-&H?mU=&bF+J2KX;gsC;<-pM1luK?kOQ8odM; zuJSW!)SDr!9i%i&@+{a}`b`r~i&ute%2%Z~QgZ(|z}3|4G0TXo(tw!{4waQy+WOH1 z1_{@Pe%OIS`bxl!+rNtTe}%o~-qq^cw2ADF!=zO|(=*f7AdMO?DsAC(sslt(kW7+_ zDizZc%6KI@FSQq8pU-iCWIBw7g%+GHg(B+EWd7IJja?AmRUPX4_6Q`85wWY zkHZOq6|fk<(}i68@2D!}@;LUD0}bg=qqh6_4wa^(rJZXRN^=_mStBDP;)5vbR1i~< z9W`?10~Y0so#{3cTaS)OTCjNJ)$@zS;S zuhJNmHG*dHxUA2{DFttcDY1iV-B^w|GYxK|UC^)(cqcU>exGHM^Jx95*IWCpK6}s_ z90@!13i8NSstuv-sfrRUe>R`KB{48d z9-j(FaU>3rRSsS0N!a#LR7wvD5~q!K*V-KvhF|WwR_A7?*|W9e%s8-be+Y*Qn{bsC zLwszKb5y0QlAK5K&=#oQGgEOqrj7i8I9-3CV)PlDd@rS6jQlyLZ=rz8Qtm`EcQ?;vxMAyc>;Y`S=e5iSm5;i6w{Pt_&one zSZ9^k}NhQ*ETHcH?wdN3X_&PvaermO)?^Czv8S<+=v?5u1C9D{N?0Y@2Q0 z-gI{TysvS$a)ttoCadqMJny8C>RgS6mR7Y2fod*Br5Z1}2i@aUZeAO5Q21meyAJm> zhaZ~Tb&Bc!J@qWekH|@^RG^^CG^u0zgJ!=_TZr|a;$S#_E1k>1ydiRt=DpW|Nt^*A zOVv4ayCp4dq_rJ^J~QM}BfWm?ixwfZZ#Djcl*l4>+NNVHV=g@91Doe8nOo?bcSe+N zv=+wo5jnMAR)Mw!3|&e;E(=6yK*E*_q%Xf}vyQd!+JD;i0e z<}5RBf5UYZklbe9@3WoD-0#ia&MEQhd(*Ni33fZ=ezK9Ny&{}R8GhyBTow9LUXr$N z^>Myjw%Fhgm4d+N$_JqF?uhuSf4U$(#fM$t?f7B8MHQ(rK1hHI*NA9DVa*Jq_+!Wk zkN(r4>ggOy^LBlM2C}k-x9-GF4$u3DyQw`>92csJo4`X!8vdSQ?4aw?=B&6X|2KWT z>~QTt(UWWEncU?d&pljJ(F>&e(Hf-hn|>`C5$GOPu!oO3T<5>jLnD!^WH>L!`lCR` zEUlI1n$LKtBOcz_{lz@?8`fd@>_(>O=s~eQ#eeRlAz90j0WB)o)t??GjZiBy*Vz{f zxKG$T5;RT|G|pA+7YG00j76wQPs@#}6y5pMzCn;a`e*y2QFLhwPEtbA^d9YW{WM8mD$w%WJ$8n#C7o}Mv*2cE7- zTPFdOQcTy_Z|npE!jm^@uQJt4vlBtzPE<|vtQ(qjrLHh)(`GxP#)`*phsI+kC+mts z;6A<|w;`jk?7p?IAzCt2?0$}lRF_%fHIYi+utyCH`#Uv#JjbH`S%S}*3-T&4TG#!M7A zz?d8)xrBjcUbrHbd`jP1W>U2H)91QspGs_>J>vSNTRh0$P}(%Kc7y7OTetBMf{pyH zXY2c&%>xG9Y`IQ;Ww%%;>@7vBLnCsiv%JE!StnQksn@uHLY&MzYiy1};^52N=#6P9x7nYOg1x((K# zBo^-+47@w0`hF>>_HW4IPep)9797DdzEP|d8djk%Wjm@I6Y&m=C@IhQg_qMF>%I6g z7+U072>Ve)zhW}}A4}|AmVjN7of_ilkAONNL*GH4uAfKE8qj(NnvQTgFgQBDB2Cc~ zi;rr^YQC2E$PUsdx_wrcY<(DOzoH<9=YcLZre55w>}_j&DDyL2NPvL{Jfrv-K$V>i zsvVjD=dZsMCuwFu+R6JiTNznc&)RdiK;%y4=_(%&yTL0z*4fNgCtvij6W{cLt_khM znjFS78vAm>;~xup(k@+Yfrq_)?S zavW-|VqJS9{$P@1Y$gVHjuj-WQHRr2*=6VgE_7V~ty;HUZM@r);l9IO@eMY<(M#xn zvjM5&`RfDjs~;=r%{I`J!|&qRK3g>|sOt2Sc=ZNrBNZ|pR85J7bXfs@AF4sPlFq)` zjoX@qL9#l?PP$A`+R_?ryrVUl8L<=QEVrfT@{)C>dGD_n%OiSc%YebdxT2pEjL&(F zIU{>F^G+(BLGrrVK0KB5rnV}@f^v2~mVahRw=UVtHO@1L?MljD_SwIM?&X#ub&L)T z0iY!F=}?2{a(oCPZ#w6r==`P}_OBum*+$x-WPORUDdQFUd{3AoYpV!%M2ADOM7QgL zuLWNM1`i9XbVmZ+kgck`JgDZQj;8%SeHTIDopQnb3yM;%{Og09;SzQ0i@xOp0$Og> zgi@}i9gr!Bq}7@8d8T)~uPz^#YNA@_kJOF|6{Q_&dLrG7vMoUk>QM9S|Hr&Z>Pfvl zOp-jp+I^mW0zpyTS{ck3 z_y=dj_S!Q`Z9^6tM2B>u6M{Wb7`i+)_k`-%_2dLlO?Rw zlU^LlwP~4IEo_5I+oY)UyXCPm%%lt78GOhS-R{#XLr&$yy2cMg`>fYcmTerLNpDv7 zpQdv$v!o|WYkRE?JsYaK_3=zCDls_frua?m{i23`cCF5ayLqd)0l#e24Q57M@kxcW z6HO)@7u88#I&lTwjvu!gzY0nS!jmweK2L8!aJG?l2?Sws*kZVI<;VCo_ zXd9W*Pe>F2?ygy>$aYFAr2Qv1Y^z;-MM7>q`^`Ut+OtXYQ7@<><^DEaBp}6~{&Q7v zPS&&CIUM|mgQHRNNA}jow7L17oKf^TIZ#VNCl5?Fbm2Za7z=0e{KngJ7#&n6EtkP@dN;vOJEYulb z4fqZhLk#2dGL@4BzP$ZC#zM|-`=kUQz9ST8#|>vLklodJTKUms@udg6y0_9GBh16; z;jpw}yEYsy#BiDlwL06sT(C9&7-KVE8x*`SIkqvo=rvO?)9XLCI<>i2+gsL(*R|3a z>^YEn+2Ye*d`d5z>N+boxU@Bt}V& z5-CYV8U{*t$H;*aN=w%OX({P0!2v_Mq+^U2J$lqV{>ODcxvnQ1JmWav`(@|(`Mlq+ z*Wz*QMp8QbIbcwnpvm!XlSV9im;>VoY2#pNyyJ_Wh+fT>-8J$4!qE)N@3(7Am9{Rb z0h?yOUZFZ2@|AoLWO^ng(eLzK7Ed;03fTeP6EkJF+?D^75c7VDsKR^ePl5k!xa-w&Yqirg4GOzm2{gURW*Xp>hA1?-E}*Ka zdcV8%Jj_Fl$Wy)K@B_ZQWc#H#o&bqCF?_V1mndK6P>}t%46+eKPY1{EV4HQ<@>wWKJt@n@YGg{78dixFxjUt5jt<5V}uxO>6y5UiS z!2I*29eXLb$w)*>iBTBq%A)GDMd0(Z+3mIc>v=c7;)&Wixhmf|B7|UG}t! zU$gIS?b@7DC^=svd!Y^=&~LX+ug!W6ReyHjafUc`I-1VDNpzh%_8i#ON*hUmJ@$1b zWSz@ZJo)9Z{GW#TJ4fw&AUUvA%A9dO-kTdm1rvN<(Nc(a$=B{;-gNcde&89MsZK@H zDmHb|&;4TW_NLh1YW&;zJ5bY3(;Ta%#~QR+yYXhg!%zpPK!9a83oE)HW=piXJ1 zshGT;Cevbo3oQ45Co|%r7s7U*0{)1Yo;rlCQV1yp$n*yD(ghr{jO(d&^boAv6Jv$> z|$TD%7u;t-bEmNHN;U;6}tAw%bLQ&Lt5Hgjgr`)M@y6mu0~pDQYyJ5_mm$m zYQKZc@XYaB@WzM5R|zP2Z$aJxbvnI(48AH~t|D;BO=*6XIl51qm){?ICJBQ0>`71*?);@x|44t6JM;0Id zTES8-g&+JmdgatmPduYo&s{;J7;DBj$d}b1N3k*X=!0w4^7}Danc*U&YklhthAYKL zyC_@n_LU)=1Ml9&OpLHxV2nJ1qqmN>k>ys6b7@DjItiHo))^}*O+n4Mp0;G2mU_U< zLpja282b{DwQUe}@$ceUE^I}u{S~Hq?g;lq?(3_%OQgRMYH2Aj$gc!BR#Vn)!G;|; zGu;k{YHlo@!$r}G&+jCp+wf;CJC9mu{yv--3@ilc*k%vEd7ZY?eaA1r+>zX`;LcZy zkej1y)6;pOxD@daY)xolznW!XX|}EVqTSLO1jdpaf0K!)RB!*op4&#e+eXWd@hRDe z3u$co!34oEFIl(rgydedjST%;C7DJpcz`L2?~- zoIS5YoKIeth&{jZ%9UeZO>eSndnCrO?UI`g@DAE?4){ z6}3k5KLE)`Hr^$tInVnd_Y~Po3Wb#e-DcO4e{a)Sd0%hyv0>%dX3ykK#m9=AT0xE; zZ*?Ur!NW(IEhP}$t#8-OZ`tuAugMkEq#SrMk}DjGZk86C|J8M55qZHZBrk@wwdP9x z?egAYnWoI~&Gbw&>No9gg^F)|BY!8*HAwynr?$V&bRQDONiQzmE_98*l0U`a4L(6; z$6>mAINE^*V-ez3FuKXhC@NC;7aj+GURv(3O~>i1J-5ww_AnC(yhM=4&RSZI0wWiq zVASO60zyWGv$PH}Sljf;-&#r_{WYBQr3Dy17F-32^I;>!9CL0r!?-PGZrsvfn7!fb zeNT++RDZU9hdV`R1Lxk4Yiksypwm2U%(cAaMEOC3yx9K&lRln!!a45?RgcqP%mc$Ei zlP;gH;Kp|U>o2N^=F}re*14UES-&&4;mdg`zJ^)*<%?8=CX>m?j=#40g5dYJj~tB`XMDkOcH?_`QAYyRK5|`%l&0RC53S#5SCw2{GX%XU6?!C;oq*mb4D7 zHuz9lc7gp{2>hl3u(-OOHm+OhotTZ=o1pqQV#^e{M>g;91_O5_Aq+yVy&8mm7yA>= zj}(hG+?d0UI_dv3H6Sw5`EQ!>XBeN&4QeP_QI_Foq= zA1vT_>GDKV>H?acpK^SOOv6~e-LI6%-X`2UIKCr!p&3|fg2a>tg)nB2U;VmT+`k5+bS^tbn?l^y zG;WR^&KA6ZrMl%mtE_vBga7)>+>HTIcS|-$qpVYR7d}P{wKMNU$i{5;_1RK~bc)q? z$BacaHyBr&32ucjo|MH`-r?lkw)=?WaG3se8`I|$1s3ePh2qRo>NN8VQMe*=)rz{q z2Lb-&vzTy4`f-7Bk?v0$@%7>^wF{qSckwQh+{UrF80h7`Q49prj;iJc1QcAY?z|ro zKSQ`(BuytWO)Y*1yvn1-DST*#Ki^Bygg?^(UMDLOgG)~uyxKFx0g;k= zLF)pZoSKUQI8W~-qT_n5BkAsmMrn9AYd7Iq-^{?BXm6?;hJ`sv{s`)G6?5Uad#eE} z<4xtq--X#yPtuO|NyDFbHn@OCLNNyuE%}QwuBK;>!)-%_(vgQ5XWxj;p~Ls z0x1ow?LH>~xVMH!=DdpvebgNuOK~(=Ik9~g9TDJl^Tb&g_TG2>xxiGm1*SDvCSIx0 zS&N^{JsF>~9nZ3EnVA+DP7^U{<8yqA&n3_0coF(Dd~}Qe8J=Oi_JxJ-)1U!#1Kf%fnP8i0+ua_-H}5Z!q< z^~Y65ZAof>-V?PN$;VA+;`*j(b9G?gYmte5`@v+w>!sd-G0T>u$;&;{>0sk}hvKqX zcZfK;eE=bMIBvI&<-=p2Ri_v@6f1$6w(sO?P<+^LZA>X$nY`qsVEt*$CTI5YcOkmG z7c_pWNL0aGm(kY#Qj#QbU^^m9*ahW?foJ2jwB)BAoqECG(4w-&_>>vH}I; zAoKo6lArW_!vb7gJad0~L<%`gpFAwQiZS{zsmbArlnRvLQO}@HRp6`ze6o$?nyD@yK z1Yf83Qry4OkYMii&NQR%oX;FN*US%2r=S*z>1v;&YEeD%ptqj-hoH%5S%jI>Wu6MIiuBRy%ghbZ3+oTE`XxKKSnufxifh!O(wkqG zGpNO+mg3f&jZGSX?_S%2Z?>P(MSTg#(=gO=S`rUB7*+7=&Y zFjGfe1|mK@HnE#>p@Z4CPkF|=@?f3Zms=O^)Qb0l@{f?y$NloK8vS1kQE!;zecT*H zr|W_=pg!6_W$cTHjK%2z!Kvmx?K(6>*DaoA&Q<=?H?!p7V@bQ66Lze_-*9eayFCv= zdwP7MpS@s@WMjLgMe?mm)mwY(78xUMyz$!5=A&0x#VI#E@7kQ?ZS!MK_B16Pr$kpM zFi1Sl)!j*Qox$y>kKzF8d9PNn2H_Q`hYW{Pbhl51pOpPL-*Bg&Q{oNAf9bw$4nrAf z1LwN?>nHYSmqac+F}N?go7je``Nz)iLnqJ!AUxqoHn-C*(frY#LC^x`q4vP~xPA5N zT7PCI>mLRC$Opg+TYbm)b{aYgA?@)oeT5V}PW_f>&eXbG`U%I+`2xb}T)EIx+~Ygt zadD(#&onVq7pkYt80-uE+R%#PwH!=;HR&?-oWO8q1GFJNDUa}Yvt|1QP43(g32bi+ zfu8^K+ixpnroU>Tt38mAO?KITCtNCvqODzN9pdZT}!8U~eZohGLQJtBYlQ9p; z@GWmp{_OP#tXj`Bz!NW3!i}uTje~;&6(>(w!m8lobznFqvgq8n5{8hk2qlS<8))Q; z6ct>(moiQ>sk$7?Golqvp{6rMSN%%i=%Aab62q?`z?9s%@T>Vfcb9Q>}MOzR0`$&7^ zT9tK5;r_p3dAUm#Pt*g<%UC?-seSITlTSH6mC^ov&pZGJIIV-D8L3X9l2S7d%a_nL z;XDUaPY2|eY672uVr6!#I$PSmOPXm)VneB};)6vF-Ef|p${5mElf{-+vVQ*0ZqQ>@ zU|TKvg689OUbu`f73iRotY_@iHO<63INvW1wWY)S)+%?tg~xi>Sa091(y5~erySt( zAb;}XsbJ|QTY0vq$ohg@l$N3-oH8fPv+oZ*qw@x03~$4x>GUIj*AW@Kr!2IHAeq&j zN=4OzJ!`pgRCHlf*zYLF8+`gu2jI5tKq^9#s!S5H81x>J8BC$(RRk60M)+X-)uM&h1X&sGKWhM!F5mVH@tb7? z=bW$xMPr{LN0r!hKunw>_YNMgg|CipjmXaDMVtMn86j8)mN)-zv6J!Q2UD&>g6NtL zb2M0FCffR=dtmrg#&wH$W5dzG8}_=x9h)!zgaYq75q)}R4O)A~`o54HcVF35-25#& zU#tUE{%>qEv^-(K9_i|UX<LqJJcNWJ#J`MWSNXJ8Mg|R5{H<45ontvw_jw+BuTc~n zbPKvn-`qd`|5*TjzVmc*{b;r)C+3P(1NsW?_F=q{;~7=CGxcW)`?FP zu5D4wLz>+dOe$m!iU)D04-=p8R^Pbx_U;?_7Ak$63CZrClj83iqgS^w=Is0=t2QhC&VE|ixnbQko^#!eJXaeX7j?Ay;g-hir8$j8SH za?6H$ZvE_5YDZs$3lbk@VFue%XL~;*!k)>pLWyqEDdz6%t#T;2Vw*jV@F+*+t93Dm!|E97b`%KN0eo?o(1xF)d8dbObOF?@0Gc}v zQzNtAPE{RSlI+zxzROX1%`}`I>0)Y{wJB~NcjezCtKFRYUnE$}A3L61VHPR4X1*)o zMgX=Y7`~%HvXBGV+1D(Aeg;XUC!{!d7*KD$A3pIfTx#c|=Qw>%IJ8hu)}k4Y0*mw0 zfwxAg?!=m?yYOog*TI0;TW!<7@(PC;C#>0!)(oW~uFDQ~h%NqU5Y9 z7P|p>twPsS*8D%U^hJfu>q36v#@yUe7g~JQxsDq0)=4a?UmR9 zJnrRd%Ons>Tmx~47*9iKt(d)KmJVF#Hj+Dgjol>sUd|Nmk9Dgnjpqkf%K>hLQm%+X zrm~fAxE0He>(f&9=F<}zv#`PvOtPRXwdQfRL@>_rmGM$6^DH=(qHjX;gweT#hwHN29{=VWoEvr3WO&-#dc-80P{hx)Bo zlkB*+ET;a@IvD$Y?D=$J3ceNr>o@hKHn@n3$-krNB`riKU5Xnvtsv98Yu4=)Wxwq_jFWeoqGlUA5a)%}a}@*foTt3R zF+1SXIq!|?+rXCO(fH6qCTccvWcQ0-!@^)~X4+_RTm4PX@=t`(y&oSR2tFeta1=X^ zRM`-z`>0&5hZpi9NBeN;Ia`v;Jh^$skG&rydOX;fzSyQoVTq(UOhamI%x)mh~NKX;5%;rk&u!`<5S>)Td z74T0+SuYr-fAGEb6uIxjGo3gvFlnBm7hqvW^<~4R*kT4`){i&rs_EjpKI1i@pS3ks z<9}qZn|xA;v96>t&dj>mIcYg9Vr$U2i8fvzThh3#-@TbLo=TyimP|pdR5kQ{E|3QP z_f3oLhs)(uc!C$99;_geW6{0VqO5*!vXn?J17(y`=!+a;L~bf zTPb}&35A=JKCql#`AjKMjDh>y-~`^fqU6i>%_V9vXa;Wfhk=JA`VTcC1|QNduMOIh zJ?Jha+5|j(pR#kz!mFLiaJ#wnxo&b|9LOpmtBx~1z95cLUaU(o&&I^nm*WMM$cPWw zJpYH+BOtzL;m894gB_^R9`3{PZeB^2Ebz>M98>=AobXWJCE0`}pLB8ZBWhJ8ctbfatf z@safl=Cf)NSoY=1*qhDR=c4rj>$9{IyRky*%^O7g3^`jj`9UzC&y>wmo<|09+WH{lD#7Gm77j!2n<;2roei3k7^~uAd7QdMk9$#Yp17yB>gI|ms^uc4g zTRKd|0@hTV*GiVwP1y=n6Bh41ve|1pwf{W$)<#rB%Yvv=ZwubgLgraYw-ISxnXXXfw^vd6R{S{Pk+z3k z;t7)bHQgzf9@w=(zhxhL{?E(2Y{6T{=fdPe`^jL;h4$lnS08zRC^R36;%hp z87Wmq)<7_Eir{PpyhRw$(N_^8A8F>*wb>)KfAmk8paV~ELQI2Ot4@54y0gm%04e0o zfH^q+&Rq%hGy^D10i0f}N5 zUQuVCz%sdNRF|GS<^RhxaVh`i@S zVA_W-Z7VLV|6mso(&T5nrCx?QwD*NG1mKl0)9Q0Nk85Sp6I2akBh#>@aq&t$aA|3d z<)8V4kTQd1lra&KY4IS;?yul)NR>fRWEaU7W%I$NxapzsPWvL2rDSM>a6Lm^g?fO+ zCP5~YbW860u8nD2;6WF03a-~>sOcNRBr7$3=yMD>-kCzgEjTtTAaGN*z$nPl!2^OHw!p4?P-Hd^yr|O|-9Q2O>Q8@ZfjbPCv6?g6qyne{0Zcad zGB*cQx4%xcgQAj7wxFC)cDOhP)4%U92qHfIaLn2B%Z{WO@>L~n=>`PF?tF;~3ZOlx z2w;CYTyi)2@JugMxE=M#Nikn??$i6v=&(q=C;$dx@UA8f!RPYX!ZX@@-Wf9A-%a$u zgMj{1I(H6&9+O{NDs*sPpP(mi93IXZcVG6i_cI8=?Iw{66+a<;Aa0UM!wex;b*{lS!+Dtb|k52|dKK$3PN+p3WF`L@~ z*OV|x+{>JY>OvKo9m-H&4^P%bJDM;^SY9?dhxi|%7Qs1mxv;D>k7_o4*gUBLH?uH5 zz@c2*ul8^j*h!=D_oMH^uK;2A*Qi%^{tM2sa?AgX*x|0IH1oexT)UlHV28OyTzVP5 zM($rL5Nz_{>O`Xi8`l# z2?jWGJG^Lfhfco>a`8-*;qeu$cb|FO8+QK^est#KTAAY3Um%N)Mip^p;kvQI+_>?< z{)}~&Ui*2$!THy~-S+b^x9JvU-MxS%$=>}HM77cUjb!gm`=4MAGp=8ujVb2xaSESN z>#4Sk4Ihm46z%bH-khfqQPY1P;(VrdAWp|3+ixL#kEG?Xarb#|$4{u+u79c3~b zxLr^N7=mkR!*JJxoEbTV(Yv%9m=AIFVNMtxKY(9uM^|a z7s>ySHCr^iqKDe7riEL2IaZYIkdsI>$ua}^c_>W=I1LY93T8seKl`Q7)@a~p6*qjp zA?K23zfH^$U+V7>-(uA>t&=bkG<8$F!e;OKdH&NMF*=kf+d{&ut+{~-S>Qa`8s21edZ{Ls+CTW*e7OBA^w3 zn%IBN9TZbiVu}h}j(P~pr|9E@ zm==M)(NavuP(GRKYoF{XKTkM5_<8HW`r_r3^?D_;z6tUvT?D|(Y+Wbs)72u zfbO{5?sn;A2ytFX6HLnHZ3-}-BD!HCp&k^w+2|(w9yv&^4Q_FQ6}57A?G5uhvWZGs zS9o0odop^&lkt-tNK*|6bW{~D>{AZ|Px@wU5a6AYoHUUx}fH#K)oh6wFs5 zec{wYSTC!sDfXrdU9EEW7zqK6%EKX245*j(-e!Ya_It&%$kwG7s1>E&FH2uJRz>$B z+)${3Ouy4G%sBNh#c(_KK*6?d5?iL5=f&@78B@3@RdN!y7x1!jv549C057+Q8v*vi z)og6IB=1EuT4w^OIx9uk937{eHunZsa~pxbXfX!#)l4Kg&hBdDtc-pxNR74sU*o{P`@KgTUucJezr~=7R;=$cb%WNA zOR+S2{=Kt~?0DLY@+Aogxj?ypin|i)IMp2X4N5vvU)9?Kw4a50q=JU+!>A>xbg0j=L)(9Jo%2(Gbg8I|O& z4nMaea5>j>Nz5!G6_t=MyiAgKrn{v0s5M_rte{`Br?Y!Z|W0pvxz{IgOohSaH$U2k#W~o#5)}{#v#%`Z)M?=6cs!Lg-Q=aZwm4?~O|SVl;Mh+)FD(R9T1MQOVmWFa}h>z<>ep3GL> zXHCf1#B}eDHM+qULmj8BndTouzm)$qY@{N??H$;t|Fb^i7-MW=IieZZpT77NFQd~g zr@6U51G@>>hE5dn`#mRJt1Od10Ji>|_qR8*;{J0u`t6aN_xa()+1LSH;)?{dQWnY9 zsCbTB3Dq_No&ixm12!PMmE<)lIS`mgB`Kt$^{Ach8}H)N|C}SYXJ+G4YC|e~U3;?l z4f;#fg*BS~vBW=wln%viZq3-7^ z#6Ui(Kv|y=B3jRnVpqmKc)x%!bv^ha8rOjCI?=@OVTmIDjmwl!lBpCFH*I2o-o7p^ z4>Pc3r@oOnuqW8EVL>mEfa)6+=Za0Owi=a)`oF*jQzvAcdCQ!#mt9u(`NK-^5czN{jH9oP(pmT`W=*J|jsI>em`zH4Kv>N%9x%=_Huz4Y1bgvI5Js8_GuD6bz z=k_|H$)}U6;F*uiiqTw1 zpvAlhUyWMf*YQ59r$;yAtlgr~o*va~vyXj#0#cwdP)g5{p&RbcUSli4y#&euR|M+i z14LpvlsSE>-$p5HRxT_IpXM2vGWQG3GRz8Uh+8ft^&hQeZ_vQ{1|Lgl??vUc+ZplL zs5T4dn4VJE#0lCA6zR@t1i8&q^N&fxzROJar`!|-q9yrB**`k;=@@K5$?>9@Ij0R+ z^-;`H#Nu%o&FH5QMKQQ?@)|*GEoY;~;so_t0_O!o%8qGrQnIB`rS3dJ23sUmTbpYl zh~F0hf#M()ROMLWveC1|NQv|7gK!z+HVufcq2^G!J(^1?9H5GiMaQ`tnf4FW)n~=R z-4&WBW?=THK0rCe46(;Oh{_TqB+TXqieGW|^#Fauz3M&!sjZ4=1GI2@iF3cwDd~M% zTflx4d8mGTj@u)0Z0s~2AM4BZ9gGAnS;7%6y}e60zy@9vliYE30Gx-_EWvnh`Q zQmMS!(s;MeAd7zU(Y~mEU>eEw-XQOmM!ug5ZO6^<)4=oN$bVRdmqLis#&nN4cV|UI z3r7l%7-{k-c8NE7`nDdOJW}-?X+K;IK9RQS69f5+^pm;2C1)W8T+SpFP)lTnqQIBi zpR}j9r}rLqRQV-xSAc{?(2a$eo~I~W*zE?_-N~cUZ%Ibs&JiB7n1|h#79IaWubjer zj!F{)e&+$fCg=|zTMZgwi6f#V!pzH?8Xc=>a;LL>1C?&6r7+3jkB9$BI0}N*h*!lK z?5Q;ff7t+SA*)(!>3x=njXb0KR+G7W{iuF zPRxx}4$SRbovr+B-9mmi&Ta<2)*SQV8Yr2GJrLXY{jF!_^9my0mfd!D?E(2FzmkS1 zBweL_N8khWnoJg}7nZJJNcha;+V$)4+~o}u(0(B7>&eOm`P9*0Ql{by;e~^RGf;7g zrxwy0_+fDv>5+RvD`j#DdJJm@K{51r!#`{$EN0bLBhg$>_mIar zFkxJqx7`;@Y*S~IdpTMo&w%mXHI^clfXVbn3ZI;CUG$42y86H|JIC+B8wAq#Bw>iD z_}U)mpo5PO0Ypk^DbF+E6QQg^#K25rZhV55xA~8RruTNj zzAT*OLVCX+E1VR*o%-A76xjXK`pwfx5x+x=1^#t9c!=NOOb!J6#e2!jN4v3b;J)%d ztwJ&#mj9;Q{#obtR#AKHFLna}PNS<|tKCw9tIyCr?J*Y-jhYVlRhp0+@_q1BZ&_*0 z5FcudTJ_?FiW7l#VUXC^aYvMJ-oBXHYx(J*?+r1i_b_SNkWxgvJL}M%@vBs@w6NHy zc7uSYKjG>Q%>io2PE8im5f zdKOF~FXgq<_eX?fH&8g@L{9Yt44PYZ=?4L>0Trj{23#_u`ILI~KzOUa zgvJ|_4d%4~ zddKQRG%N74_BhYD%VQw~{TB+Q;FT9`j&WE#O=R>6@l_4-*@G|BY{RVD{S)fu(gS(@ zfH&^-Inlz^HB!es?anW`VjtO;3-B=1oezl6|Dg=ylKss2?aHg=+%2nC$SLI`04ZmJ zQlAul6`#w!S?cdeo%l`(BIp6T?#YNI0(|}-?PV*r8L=H+cLL=dn-v~aZcm&n6viaO z!d|gXCKLvM_0Pv}?K>X|!|muQ=j&g>Xk2?Mx~SR4Mm25CTH?T}P~Zb+uY6joqMpR; zS_0%G|3nF5*~0*RgqD1iqkNGU+Ck{(27=09jF0#U?8d@(=w`7F351dq_T#>Gn`CQc z?XCh@G%{SG5>^yY*8ZZ?K5bdmxWX8EHcY>4Iy43ZOK*B^IGBpmte;7CZ~5k7*+(8A znCD(`!hM-~Rl7Z=KzuNNtajplHA(ztThydqpzpMLX)~ACbUwMMbuF)a>3r~}I8VJw zAa1%)-3iA7+!sczac_*kMDp$BSuZ4qt3N$~^m$B+>mGO4#a(>bjqHDZncT{qajnsg zdd6}ge$!uiA}Cb}GA+xt_VpR-^F`nnL#`QB2RJQQ0}p=;zec=e@qk{AF{3To)@Cq( zshye?ihW&h@Qf+afwA=&qkrPB(;TYDv+vo|UB`Y>{5|D#+9oWq;`WMcWeC)Ae&^J} zxfVI2D>AXIts^bwsY)l(=Y0Bp>UGX%iu>iuS;F7Y#uj)$@&B@6<+FcrN!Fns@50)d*rdZ5Zx@QbK-b#-h6(LG6d#2T;-&ZObxe^H?qSIsqJKBLn7uN~yUHj2z&|lRCO71~ zK8PyJ{qqEt6Qm8w_|LgN%!ksE*LH0h2cQhq&v{+T>syg4fTta|if z4MN07G#sW6-WQ2?IL#{5fPa>S|+_|N4$Tz|>WOu=EIHuq+Tu%1prL^t`* zk25bv7_Zt**r1lp$aI_|S(~3V5u?dvO{^y8t;VfYBW?}U>Ug#kqFyCD_2z1dH^N#S zT3$fa&v6dSCAX!~vNn z{e*A^Q(?DIg`1(1eb4>VXSJD-wv|nr)^L=cC5y`Kw@IOT+(q1} z&MvPVsUA>U+*Z#_;fy*N5NC`bwC-ad0Zfe4-_T1U5y}lmjvBy#b)js7Y2<$m2^tb5 zr}9B3CR!;JP;pu9&g?8@_cCo=#B=s#|H2$r@Z`Wv&c5MNKgGz?%{-0)WT{EVf~e(M zZc+}h?^w^K4e+wC6-wl+?CVHPyl911#i62{dJ6Sr?l#e2)w=gr?}#?ZK>uajZNTex z*N~D`@U$)Txhwqu^YjiCBv(#pG&kErZq9;TWcQ)|C-a0%t2EHx$J_2fBi?wxA?;xk zNzp~|=ZJW1KZi@(F&$!9AMJessp_s6%-FHq2-}5?T_mW-V%a#;JfDX_{NPCrr)gQ) zWpsEtF0vrc(Iwri=>#9{{X$qhNquZEB7MW`*w*Qd)~YXR^__JwwT&%GUx-gq%7EcP z)Roq|o>OHTq|6__%qa~JXy&4xCjHG7OT}d51OF#?A%5F&9L2w zmJyZZ`Zz#9p1nYe@bMt_yXq>eHcd^fV?kzboc~57mWBT8gBmjW83&S67`B3IFF?ND zm#Dxogf8E_7s!%nbIwv90!5OG{)6IJN3lkR(%4hUWPxBu=t!E0Z>Tgub zGh!qd-;1G8g_|>e0gu>Uj@BPF;dAJlgcY$*!PM`AuU6^wTo51R0Nr;>4YFk}f?9kv zJR@jSEab2GOSUK*#d4e;!jR6?-*tBmCwbH=nzW<7Z*MGM7Yytt2G=^0RV+MTI;rs1 z>Aq3X^nHI%ihjP8x#Q-fa=OG$mRx(X4^KAdf7tyOIEv73^s*SX?Bvx$SQZzyO0O4hX`+z;cFa}I-jF> z^M}#grKZrg*MK2|jmWd7-~+;*d*9--4~eA#W&JD>$;*W4UTp4!O1bjY@c{4r2 zt;2yO$cora2Bl?wpprRI-&uoy3Y5CpA{$GHeEC$rfC{vi&a%Y)r}0%oi_|k@s+-yD-icwecZq zl8rUHFtH^T`T@bnr0v$PxzrbZ#vvPH>^}Q~A5>ImpF%mlpQw#i11CteJ(Gc1?8$(* zahlkG%0=sA@I^eJGtm>aTz_d~Vov&ng8-Jc$BR|eIRB&M#2fAcZWWl4#fS){nB9)^CFXBC*-FReA((lk(fO?^LqzH`!In$1xpC&^A{==dpZ(i zf9vuZF5^M9>p%-Sb9Ab{;Y)`sjGR&?xDpbF~>GF`FJs3>jVOU`T$ z4e|X%?g35CDwg51bI#BtNo@20A74@H-lS!0oxsTs+wH5ul%A{ag+8rSuPnDS zoF{OlV@k<)gy5m;3N2}~ z8=0dB0HG70nf^zou>Pgx zuNrD3Qq@VIh3|~m;*i)R@IO!_+T13PmzFB>Dda!CSB?JyQ(Q%v<`wKa`-T9Bw$;8A zkA8Ov`t8P8RZZ%)j2oS%3qT1EdiYIitkmzNk$P~(=C8YtJ6gz|ukMI2h zjNn-C=5|X^dMvpa1SfnlZ9gsm>O7DQI*b<0!Dob1DC124n)?P^wRPi?H(q@VlMxT} zAT-N0#lH0hu&jL(oo;>5uaUhA|9eTMDN6!Qhf|8>R-RXrex>KPqJESURwd6i@*mr| zSu2h(6RVSc%%JX{lYxvatRD7(3n4zU9_Jtu_*PVxb1!f~`4)eie6iH+16UI9P&OeD zKXMa)2Ji1yoS;Q0C5wPb7G25Z7d2EQUX(|dgxA{QVs4Y53-?W2FoCiv(WbqTL+eR) z8Qw+EnRk#Lq|f#J^UEDpyw?P=0Cl)zuksS$28*puX)BB2h(n0C5gZfDxvOEo?wV`# zTVv_;_UuwxymK_1YzW4VhK}o9nt$C1mm`fU#AcQhYNRNYv|%Rp{Ydcf4+%@1xnso~ zQ}n2~#~5F^EMZ%qE_D#xHqP$t?FUzZ6cdj!ZTW1#WEG1r51$ zd!ptwZ9 zU2=6*dxU7Nlh#s8tP?e68IG`D1-X0nlfW_Clt0C`4 zm6uU;I)8-PlBZwGhosolq8&AjkP_e~v=knVgVp5wJ(9zWUm|-m+nc#v-U zNlI^u{Jz^p^_giaZpP5~pIi4haFLDpxm^j_mzOXPbingma>-)C)xy62kFBqailf`M z?ZzRvySuwv0>LFng1ZKHmqr34xI=IV?(R(!+}$<7T^iTNckemly>svT**~gBRqa)C zueJ7EbC$NyVhZn84O!V|572ZDxwve(Tg=FcYX{uitET$DK2QuZQ*fsCD`Pex#v71@FyQ3k%Owqo6xUecbY3tDPxx+^BwjU z(I-UBv6!9kzs>s6rs@kmyExs zKYY>!M+vMFrutHXigMH9LV;#0sXMcuUg`#( z!|Od?%C$fcUcV25#kItH0p3Oi1EFX5G$0=+5>U*$e2pPF5Q1J4fv|)eWsq(CkGL$$ z;-E%!$HhD8F{U(8JFlAq4spVHd!gLE5hO}L1FuJ`6bI~Tn|?q3V3LbgI9DurEU(#! zVvKC^%K<@1)~c5QQ?Wx#`@@G|O(@suidod_Dp1>?VW{lT%+YR23lSpYknZ>dLhFd0 zbl${^y`!ffk1&@<+>VWk($v{n5G0}4db266e`AIo%<0L$Chzg}xgrZDQr!!jE(35c zTncP20kQwII=Hv)r9Do8esm?N!jWM&-r$Rb?E}P`ZjvnCm}RZlsxb!Iz?oz^k=wAF z!C2CLBB2(9)jeEy6I$|9TGBbY?sw6MFtIkQ0H!;Rpt}`yG>O_R<_*kLYZO~(HfW?RSnJyAuy|S zv~C*@AQ8zWC$IB&;UWiB|MNcQT-6NxAp(xN;Xcdbo9WOu0y8+c!HdrE1eT_6iC*{; zG&yZhAQd2v(1NkLa&RBvxpTEp|GNe04sW}CVD>XQ+t+>+`v*i+eH>xMm*s8|;Zu@@ z)8}H*9j3XE$vdbEoV87lAesNw)b&!lh_`L(Xk(&_V!*b0ULxd)u4X5)%{QK0HB4qK zW8`5TFXCrQHVer_q@f9i@fOb5yPr(;+@?&)+_<)fC&Z9UCt8{dHLiEY(5&Icg=7~C zK){|+O;x7z?)?STUfcO88HdC}H{MQSjv!w)9{u=7s)vGhVQCCYIbvXFzm>^-Vo;h)^EDuVfBd}M}cl;>Kf2;YC=aP+dlT@@f+=7# zs@w5ZQ#NBa3(I>zm674;S&6@zJ&?#50gUpma0N7cZ_>_An~Fj_K(!Lxk}@VWUqRLP zvosL>rKbCC;H{`Me)YD&tDL`4y_4{yv8q74M`Nxa)#pz zJ?fRYonkDfWq2N|VkqZ1Zt)D07A$x3LKf_r*%+Qbs*1Ta6mL`|ZYVImloYe3i)eiz z+o`Vz7#Uz7UhZwb`PLy(II>{m-g}sP;Btl!1UwP+yOZh#R1FI zY~3&d=hVr8Np}C#%g>vP_nJ3>EN?i`#58|uFE42KStd*#jpVOK^nh1w@wdMV`g*_1 zMMR(ewlLpdi9BYe@%s)F?fi~p|L$_p4!&vwEd9O5Ia;nnZEov1im3Qmr?@sEO@dZ-Jco>1vmtmf4e@mhJE4-ZtG;vEdrbtt)``WS8KE$N* z5g-l82v2OMqxU{6;s|qtAN2dp@Fy?I*l(U(=@l476L&iI{TYUz(PJ>)>a@$Y#XVyk z_#5-^vnt0)tS9o16ZzGAcK3->ipTk6vM_mMSp(}aJgZ$cV%K&tfp0(N(Up#5?TNAP zV}z8yC?ARbvwD&6 zJp(cd_jNYA08%KFx(6tVPUhZ1G*Gi4L zBxnmBuAaTgN<}!rDU2+pYsVY4l4s{rZAp{M11qzZ9|Pd?kx13j-E#TKSL4JxzBe5u zNtAJ9%?%fDUzdtE$PkaDmV*sZ+ zb3w3-Bb}Fmj_bnj(tmcig8Q1vd)Clbf9Oc^=$ANM>ciE2SwVo!fL8Xy`OzPLoHB!x z#p(Di@T|-=Us!d4DBX1-7i|cItiOD?uvV*t7Dbs}`$mXiaBB$$M|I3gr+NJ8bL+Ak z`gJ9jCi~KrdUis;=5*^-QR-E`pV98#-rlc*;V02q@Qwe;9rHVlk|2#a$T`2*eDtd! zyRTMkTB#dhjZ4mI$HJ*)dmAN=nq(~!rLodxnq#%FCN>Lu&p?MHn0qt3-94^4AncAU zT=BO;pN>b1D}K;`lSN(Q2)VyG-R|XzeXf8R;*EX%Z0 zIY)-c8Q3{#Ed66Ue$@yzAQx^%F)&REc{|RLF4NV3TkMwULx2tzEmJ&l2ED_ST zpS|#H+*R>+tTDSj*p<{ZMMl_tWo@Jz=r#99^>6ah*B; z^oI+xJMm6C;*F8NvEZ18bDs<7ffQA@|&figH<+GZQovWUGs)hBii4? zGbaYij&{m%``!kR86aW6v&XOE35TvNhMQK7ZuQ8->&BgJ~-#JpJT$of-Q>&^0!GapyFFS@G> zG>M#7wl}s7*|(QFEvHA!dn9Y>xsd*>B@jL5TlhZ#`yUakF$00Z_&x+by!-Cx;*&c) zLPvI;!tqOCSmMPD%n~q9u@NpYFCH}qzC9+{Sd_S$o4+`dC6{fQYJFqyW$kP0O>{}D zFE*a|i|yRs`Vd|ZjqFYMlLcx9ocLxva$SS@?|okK4EMyVn+#9xhO7OZZtea3jTUaV zCO)4Y`FKCoHpNUpPyD+Y2Xx8qn%<&y4qSaQ5L+$WcrP3X))d)WZaW;f-HC17F~kqN za(_~DeYyW_g z4%?0LsxzrG*_63mZ*Q6T3$v0c)kgmZQ8Isnu2|cnxi@li{NAue*%$>=X=d$8j}D6#un%6kuYtNqxs*?t7`rrC8FhP{3P-(m zB%G`Sh8pXM*v%}H!CggYH#vi^(P=ALQu_$Vxq~1d-|sP7_yux*Qu}Knt{2p&RfoRP zEd5}9yzim!Pvdy zo40{Q_x=qdqh>*1if>G_kdZz|s$kiN)}j za~nO*^jg=GFFzV}s!$~f!!ORnXTpF=L;JhEt<>8-_d^(Od*y$AkC0MZO1K*pgK!g! zw|%7qSo?ij@$?5vl~ceC+#&KRlrnqqW7*ai#$5b53kTetO38f9zqX++7c}XzAQCu+ z2&v^O6+4mGb$HY+K#2x?76t5)d5ms`Og7ocwC5i89oArMP;14H-f)dzTiNtGXWHLe z%$QZ{<>`L6b3tp3JvAb(Wx=y)S&f5sJ^e{1RK$)f5VUyGci(7r^b}wBL6G|YAs2bEr55`&DZfNz;~^AqKhM)b1P~P-uG|R zIxnA<1Qkce0TvbFX-I$Z!~Ax6+0(?`rhsx2P#OWt?OWQH!DP9`eq2ERpyfLcW*ke8 z2I@xRKWI6!NV8aq7zKVd9~ym}nq3Gi+hNzfu<>b=^XMufx%a3#Cp!yX;v{{}ef}z3 z%c%CZ0PvmbDMscyM2+gV-`5{xrT8a-4m!5B{TPeC{O0R;GVEo5P=kl!Hj8Qox`^rySY@B)xc327 z@xNcl`Se#xK)<12_$aJ<&j~e5agtVuw!C0^1~b!Tr6Ty{{NS8Qov4aon-zuxUDfk_ zAh*f2*~J&@G?K_IwR7}Y)QnYK&a<>XKrd?Jy;L+MDO}0x`~s zi|QB0(YNd*gH4aHTAL3SeVJ&1vu0(5>Oigo8u|3J9ptC;sj6!3<@wXf#iDf?7O||F z7(Cdf^ZW`r_~RiADm7=iCZ&iS501Ec%YZWcqW=uzx5OxMUJ~gztKsl>SRO*Q5iI#= zqE019H0x-mAnAczlihfN!Dt~TeBj^L36Ak@LIl^E;q%G%j*`;6e@~IhX}xPW5VH_h zbkE%CFsV6Olno-D8E|^ujOuzXCeCjykXbqr`*Q*_wcIs8P+gsN9qGBG&1WR_N0WW{ zQqVZntvEIt>^yR4NFL&fszmKhxJs3u-f75}s!{%&g`vQgF z^=aAu#nJC%^Q+pjVrXUYjJWO?21v92B((p&OC-LJ#;Vr;)&lq^1%xP82W|P`_#Yr5 z`=aulAC&DcvjAZ5C9X`X{ z$u!k86?6jZ$o#84(*{iEiD}86eb;>;dn7k>oucfANilcd`HeW}Sq; zLSVlF&9%f(*3kAaXtc;Dp`b6EutA2*Y6csspWd_>CBT_}8`=YZ0zz(tk-2e2w-{Sw z#jM;#^f&m5MSwvrd8rhvOY2fh+HTH@zcl9Ozc}IyEUauhTXr<^2iyHZYT7dpiIhvnJrQnK^=)(;ymH7RtkqVzGLo6-c zB*FiF#MBS49B4HRt{AtKT$u8p(8rrQbveQtESpWdDxhP`k^fP&`R=`|(S+*4TQYo3 z_+ODw0SqDq(qH}+zRB@qn7QeT#+?t);XAjJJo$#fS7+c9kbiO>2}n7NJ>K#vSn&G@ z&3Y_20%k1!ysB{a6N{16@zTl}# zllN_fLN`k$*az;)4@Q8Y4{FT)eVB-YkrdpDyPVy0s}8G1lw1&zXPgWt!Wj*U%<}Z} zLEUDYmjTwmiu9kpN6_b33mOTUci(~&KVQz@h}7i}_?!-uS5}MCk;w*F##Zbd%@6B5 zf}383z#JnBe`;DA9|W^U84e%4F0)EbhqfhBS>LC}St1+wvdw~T3(y_$5fL>E=y#-T z&;@IPBd*nDeQJl?1s&AI=p@1Bs$8)0u8k8aO5^k-yiQ{!&}@dSsY0#I0x|N zS-^J{x|m>+AbD7R_8$oGxvbZX(dJHX%KHs!uTI&?Nuv0C$gA)@M`Rj{B}p+Q7N0Fa zE)kFK6cWlhaQ&XyCS>aUf@6L{gY-9F%g~4&IW2Ye1<2eL6u}%;AG}*`~d+QPXrJC7cBL{UuK>Jh0?=jbj4UKbaC%aex*5fB=n=% zW`fK3lZb27J4_G!mZmR~@DQO-f=LZvISzJWb%Z}F*lG{Is`?XI)%scb-Z_XXB|4|E zY_-{@x8e@5T>QXxXChRsB0(v^d8T#3dOzYN6qLE2%J*K~Tq;0bZUgIq1xoWdlrNI! zl^Z&SbzXXma3+O+pSHtOS#eLhR-1NQ6tMgkKA`VM|66g5XXah3!T*k({+w;3c5^n8 zvr+fm@(0u6RP3kej{&s_!S~sw(T`DBY{E%3a4Y$QX9bVJ zbS@_)dA(`JieEHb2KMVmt7b%Tw!hrv9@kPu4n4y|+0fCn{VUf0R;UFT7MdM=0@gK1 z#IDaYp041cIMb06!qM15!w$8!6D>5v`I{habsNV(wMveuMH z+X4n?Dq?=gfIkYEsb7b-1<%7L<_}a0Hf4XV64)k^tf@fhS!?S4q^>ibtgX5|W?8A_ zW2{_%pE*}s+_0cPd3@qG!_Pu(m?$3HC{J}BcDhcL^1rfvS;io5mu|_rZ=Vku)r;+z0 zqNk*5WH1P8FQgZQ(uOs4z?#tl>X4|Ct#SVVFVrjO@MK7e=y<0hpjs~=2zwY-jHv&P zbP?tha43mMdCI47PhEwmDf{M#C?;sUyC|jt-J=8O_(l{+B6MYFa==jj0Whz_3YF^p zaEHq!1ZQaywFGl$oPZ-hY>*CI$w-LkclFUd;5ne5aRyC*-V+6YG$iOBhgV$H_SXdn znC?Ws3wJ(X1@Pbf`S?ouX^7DwTdrUwU2qByNWqKm1-JT9?D8Oxkl5TOs)rUXza@Yd zu?;#i<;NR<{HZ7uT_2Mc;La%qPzVwPeTCnKFRK7tes*Gum%PzLG5#S;{H`LjwL~MK zU4>~9%Ue;2TN^9rL(f9VDcWdbaPSE`wJY14ux@`!Md#RlhS>zyAFzr4-$ah zVDC5M7RVI+A!)oJPJxuKIRC*@j1v)}1B`V+Ehi@2-=4L;vM8O2Z?9(EC%)1Sqs0N$)H5}z#WEs84sHkQu>Oyo z;64pNR75Ibatx;>HpTBD>&|u=tWXLe9R04rF+SOgsfo2jisv0$qPGGq$Dt=%>OXs6 zUEFA1p9=BireLc?C<5lEcp?;iN;VbC4HZs>7E|GMI1V;+jYlNh7SMA=Z{xPjr+UK7 zw?-O$1(D#OP<64l4nrFPU!dRGPuANT>3im@2dM1M{lkm4p6Z!@nf(^jCf3GiC7k0e z>N#d=oAjf^vL1zOl@6%KL%y#s%vYXld7b$i1d1Ax<#^_UV;63`yumq%;+MJlJAYK) zOg&sGxmu{Ta4Kd#V3Ein-COC1eV>C*H*`mtDxb8RQ;zB5gmQpF83#cujSJfEV8s|eIY z>-PH5q+6BLEKK<6BQSlmnmK6guUyIIGVTJC+_IHgn%5{9CKGZ9d$*T4PyM_#h{t*zbz)4pyDsPJ#K3SPH-vA*m zp#_vp7PN;$vS*r{Mm>>uxi-|^n>F;I zJDfJ0u5nT`Ea)}S*v4V8QJlHi|58k~F7U0K;eX5%)JDj}7Wph}9FRanRu~aL2hlWk z|5ghpqY!9{EEC?nnk&_SKq>A->FCRU*4!Uf5#7_}&?=5WCTx97DU=;m=xQ`gZYc@Z#V(T8w#t-1kh>OJ@`qd|d zrty|hhecxl_#c&{S1WHWBEbLFV~9d%st0c)&%kA3@Kn{w@bR_~U@PjhXHG;ed)D`% zoR6y#zEURU2q-pJ3ZLN@P&w*jwDOk24f=pb7j6gQuVu0q=Ar&k6|NnwbDzuX13P>g zy$xk;%2UVT_e(KyZcQJ`Paf~rC5*ZZ5w3aaPHS7mHyD{0dK||PD!*cbn7~RI{F8YS zC5Pu(>b&rrF-<`wy z!sM!K;z*8szXjc7sltd$G8tFFUxyeJuI-+)$U18v-=kBhi2yoaHi^84yu_mT)31vv zP#;UF849_>;dvP`KH6!zhU^%7xD(Wx>?}u{UNTr}e#WKlWdw8x%HYBF1LfhogmPn{ zCH=5E-<@z<)IUir1Fp;$>8kZAJMK!hfqLyS{9(PbS8LRybCa?FyjQ*@j z&oHa*&}%de0gFCxkacdR4M-)+0VMCLRL5gvzrCtF4L14SHE2)`R(Uk^i@ltgK< zS=_()+MZ9u^FXlusb4WuH&k1Zok{(YStvSD%LK>VEczIbzG>QlGN!IH#tqR_+U=GC zmF<6c>^mS9^C3f$oKez%sc}(Tsd33zyRIy*FsoQ1aqwy6%YvUy{$RVDqcHmZQBmHO zz`~!5Gis@QhUOkduV8tgY#X#=66CHz!WNq5qbtLO{k{!$Ns~3J4kJ@vWy3VLtXLzY zrorZbcC$fTq;J00d@0P>i5y{;L}8^#?2#42mTM)hVLk?bH@QKH+Xgn-L2=#|BPaV{ zw!y_`oGFR<-$yO~>X6u02gT+C{UUz)G?kGS@bVMj6W;iQZ?~8;xk*@{c}Go+F#k;Js}#1B}Q&hj4n2 z+B$cqc*Q}2kWfN7g<9+R^y!mjlcq!Y`U9)qZp34exe~}uMZzX5=(*T|{}}kfRvV!s z1%Y<}3C2bMCjh`y6d8gEFo9u%bBEjfmWxBEdzJ5s_)Ab)L6X=Q&R1oW?nIfZ!DX!` zBP_Rh2O9)~#@PhWO3*LialA}MPSSXOH-Ln&gI;DLeOHgbm;YgVDT-$YthIS z>75XoH0FY$+)u@|qI>pXOK_nI63Kpsd8*2 z#G1u%ziWZ*z@4R)No~=3UgqSsY7=BoAtPB zH)#jK6@TE?6xs3@tTi9uB_e&PFamU#_K1}qgzQLV^4wlhKX5!4({mTXQr}$94;P|V z)CAN>x}TJd65Aw9^9DvEi#To8%<%!31afOxAF<=OQ_sS`rPSJ(d}>y)$1g50j{aqK zViXDksr+kP^U6su$zKm^+~%ET#|1i|emPg*Buj2E3|46rS8E0@w;&gjxNl(}Vv>Ba ztRmJtiZeH|eR)rkR=<7Jc46#d>|H4)OTXLVkb}e-#)SMC@1p>!XMjsMBlz_P;51#j z^P|h1=BP6p>5s)_3de>=a5HhDfBi#+&W6bY;z-BAYy0YWi&OAiBc#LpjJkIS?TNb? zC8#r1r`zQA-;JDq9eU2Fz|w$I$whV|8x*^zV0k&hpUAg~h>7cxS+thlxR>@KF53~X zjd0w^vpCv?KIMy^eaEkkq|P%{GCKpVmYY2rN~U0+2;B8E}qu?uQaK zxCvEFwQ8e7?D;#A-KgV+&P0RIO#8pP7I!kxXZmFW4S<@H;h#|G()p)T2|ty2-Z{mJ zdEBC`G?OPzir0L950T7-L8r3_lk-QghCd%Nz4W~Ko6djH;h`9l5)uFsAN^+0!~~K9 z#X|;LNKhvuef2)>whV4*n?0$?0n>|Zj#1vcH5GDWf5E?o9)x44j|Fkp&ckI7YHB`5 z3hjc#g1CR8Bw`}r!;3nLFAY}1n9DO@JsVSFa%0s?JV3@goMfBV{Eu8#YLDUy0q{o} z*C1QTqQP))4_7pgQaR~fkRB*amu!TmSq55laYvU zA)@Exh*8Mq`P#OKa%FtI>S^HFSaBRcoAFrd0ZQTrj#V-;%o0kxsw^dUIvnB3U}0o{ zW*KD&Y`t+fiZQl!kq(LvjDZ#Fxmp=Y-VCQjs;71CLhKbFZehP_mV_5}&)gJLwvoS(LBdW$e8 zHf4o+#NxB(n&U`7$Bf})OuhZk4z$wBx{L0}nBArG3gb#wm~Nivfe>Mrw@OmXGFOKh zZ&aBu>;z}oZZ;u$Uib&alyNmYeI4AM@dl$YLIDq0qbeiv2K>NAlR;05KwS@$rlEXU z=zeL6@*zYe%OO2v1}Mc6frO_6e^nwjkv&5i;^Jc*m}ky5#Z<_=dwuE5l*<>S<9XrK z2YwnlTkQ57{aKu*oUw+W=qy_C7NyQ>XFtN*~E2Kz8Af$in(kxg!Uhj2pf2-B^3nz$Szs+E2j9B?J=c3H{~krK|L z=jOm@l`dRqcFcJ`e)GJoH5*v|+ZSm7mSpKfsIABO)Zlq$hzMusy^}1v$7q>0e3fvV zt+wmFxpRZP0HXqq;UMw8^7$lYw6HffUrE-p0Nr_6ed|I9~T|F3dg2&|XW)G<3jO)muB0t<~jQgFeg@>3ai~ zKa1nw9QJ{q=o5Sx!z+7X%PBE8s>u@?o1fVwexAw9kzddiyNG^cLLI@<4}1~!s~s@f zy6^^jMvR@?BaLWLkN(AA5%{5%^?a92^*AJ!angiL2JmI$w=(R^3KYjU#u+0vzAQvS zpwuv2T_aP>bVVc*Cotyk5GY%`Ib;K+*HgJmra~~9C~|+c$J$6t8CtgEEy`Ul;oCY8 zntfABbw539f*l$0*xezUpF$^`{nRu*{{Co5z*;1Sw9Yb;FWG9)YhI{~zWJU{sT|M+-HWeV{HM8jsk_RLy^IY8WSM~k;A`YK*6n72F zzr!+iNSH;Brms*L-pI5Dw~!A@$tN;?@V%e`X>(9b=(Md!$tw;*8qGGNMKq@Hfxznn zl=Vj|JSEN@%&OtwB6Xl?GP|hTUp5U#`ez!cp-vQ(AART3gJui` z{u<}Be$)Kku2crY>&`=N=LB%kIqEaiz+Iz7gKKG z5?lIp{}EufMO3S)=&@G>m)>{3})Ti=Wi|uk#Ab zHPpqfjeKW)uM64&l*q&|i@E16T=q+TIm7QB>>n+plqwrhs3y=RE26);r1u0S?d?^t zzEN^sFS$!4q6^ys)@fGcRSmTa0~)7B(og304gBQ-$c+4^HLh!ag)dt%WF38<{s%7h zZQF~Xe{hUExMer`Zy!x#i@CBEm^)GO&Mz!yJsm7-gQwMtKPT2q8BesDy*&y;T>mB zfMft7D-ip0FDESOcRa2oI-knXpmG2=j5cq*bz{r}T)G?@mdR)PQjC2VA`B;G_q1~D z&vb`Oxq#frr&3B>XJV}>6Ygy4kw zJ}GimFNH^nUkX=aAL%(s2V;#R3%PkTn+vLG@JTmvS3RYYS9kAEKyb`C4W38 zyvM34C%)7ao6EZai_S>tKznixS>i7+LY4>T}GfXD(}(i6TI7^5r{UZcZ4 zP9c@;dp{)ce3Qj8j+>|(_RV6#=w-87T%SE?!6r+aPRE_Z4>!F}p>}GXbB-FL^hViD zksKX^{aI5s$@9RQo4C2?s^*XJ--^lQ#VT1!aL&y8eots`9Rl#!w^++9oB?>@BW0-Rq2JR709%y<7Y=weheBVuBTgnLu{f_?9GtZIn%z~choTt*8+PM%^|fSo+qU`0+UirBLEC6|NqEN4ULcN(#pv3S@`WG- zeIXTq8h($)Q@PDD8Z_2c7003z?kKvP1lTu?mO~m(P`%Y|ph07t3Tg4y_LtY=en4Gm z+e7w!xaqmsHq$$qt{+Eu0$3yd!D>nk3;Mjzyxg7j+~hp>Wu@^Ch)ihFO6UopsoCP> zSf}3Z^nQko`#UB67%yOQcnh`~4l^a{xA5V(?g9@d!gsretDn*H-un?d#)*x9Y~@V8 z@gjZBaZ5waH3aG>O3HIIKbES9t`gOFse(24`9>DU$0pK-Aev{mal!_A>RE#;kbHbn+_Om5_LW_=;X#U!njPo(GCQ{%UqVl8iS;pll?1Bh>}c49AEEi{p@ zwD}b$`vS%QxrZ`^_J10oSsQN(z}_ou9;uc7raasj*xGq>?5z&9(GrNn){?6t zY!k;x9X0TkYLI#8wfFW7^kMIsXbzoP85|5U5bZ|F*g$c}w!~|8iMu%_M+=U$ju~Xj z{)>l4BzC(UL;Po#2mC%TKr7SmwY}6=(&?MaoSqlgCHq#Kr*3%a_)HT@bW`2&J=3j* zq|OVBY6K?)NjQkPfKY`tR!QPPi*lwge&VYC{6M2o(aXV46{8^)t>|wq|EAdei|PEL z^%WxFlqH^c2}*<%udf?EPQ?cFN@iRl-{7I!>pjy9;zVoOc)FB4=*? z$Q3wgUgB>fwUF5s&o%c{WJ;Wtn}2*3(4}WP;2r+)esIlA0|~?T$DMU}Lw8%l3C9$3 z)3?Rk_TymjJkQoouC6g&&oN|oa3UDI;S>!0ujknJBfjXG*^;K82i&%AxUp`&5>=ZYciYzQ?{_)3?{S)KI!ecEyLw@q46`^kf;2+5hv#M0FT3~7-1_@!F6LPJF z)VsOUo`OoX7r+jn_&Pd0Kqsen<&_z!cdjW*K;81`^B+r{EWLMv%8w=ZBybUj>mtSy zKF7|!e)l#adyTGkr*!o|I{*dFm8y68qZID$oB-G>^rwU$+3Pfm8lBTd%NRtw8dTG922MqRzcMFsVQ^jS4s z+}|s%$q5_I{O-|@^t%^X{e9a<3yvxTwVU`b9BT&xoBGzz5%Dry$KaNM^8mxbzwR}d zGff`1!g?2X^gO{KS2jvTg6bhJ3Q~PC!eEEICb5{=OAI**-{tUqNB@RDO>x27%M>Pl44qpjq1MZ&T)D1;^@*08_>NLjs??@?1zH3=L7&M zWEac6@z%3^--&>ZNxDNy!#F0W*jN()_>qpwOZvkkBf^xAyi#rl}_9KC9eJ(5K zR6_WH?&+)$!{6wquq}rEi`dQ8jSF2b)V98gzw7LJr4!I#^IH!}qnjS`PUaQcQQNXQ ztn0-(UuyWABda4V>=xR@R~jGWSHQmIYj-gM>B>?J#c>JFFsF=^QE4$xgSg*-jO*4! zVa1Da5Ap0+V}cq7RXv?e{%7JhBnumhU#LkCZ{suwUo32i-%hnNYnu zbNYl|f1_5q`LrUcRFI?THL^Y~QfWtV_U~Ifc}T$n)+d2?9S#MF~enuh&Lo zHTQ2wKYO5DAqTX+v|@C+ur+%8S*cUsFNSOAI>r}cUjl~zTWS=PZ}cXba`HBzTe{|R zRC!bB=s&`R`TCdRB{=-hW2MPHqf)!}H?T5+itbT1kEkoo>jyn@jM}B%Cs!G!QQ2I{ zIiJg|3Umr}Jo2XPX_U``VTs3FR4_s$9|uZYIwv*BqUB)1;ZU-!;$LVhEuGZ|oSqWr zyHe@^`|)rMSl)^Ds&Kj~V$%RQs#jNTqP`?N9C0f{7RxoxNm`sPQH-;D)09q4E7|PP zr9)-J6A_7JM~wl2#Z0=fdFOT7@&+^=8(bijU48OQ`}=CIO$IsmN*!p@iokLyHxn9m zM(i|-_*wqq&2FSvO%P{gpQZ*txi!}3`RffYNDBrO&R8%PB6PIz?t^sl!4n}C>?F>c zmdvxJIClM*#Jb6CfOGwdaq80zk;m@6Mg_PM-E~fQrQIT@vcs{`zGW|B&2?M;$U5D; zV6TT1=Up^}u!qmjK@`v8Oc#yQdE4ee6$6 zLdMt9=gQX%cGqG@Co!*v7M}rC^&5*0I94$*a6rzAZTmG?gQnw@v6M|>r5>56n}LXO zpc6tyIpEW_ADS>l+I}$8P=`jq7Klv8bMoG2s@5Y;&h=?RvAqr0%?7qf@mG@7E?YAo|cE39;ZAsVy9qAA?uYdo} z@P8qi=Q4@f$>H+derIdHx9qUu^D6#0C02f3ZRycX!cnv9QQw0YyNJV*-Na-SO-0#n zl7dE=?7IRUlZIGu;NR4dArXr@8(26d`NO%kHT3-~6g~pWJz~7U;WJmCm_>Z{$*iCUY+I2AZv@V|5-$5B0ABqrw6| z#gqk?p&P?c$?rEUc~02!WVBE5FI#-4UvNeP2!cuM4NjyG z2b!6dmv&iz%$Of5Hy+p!7fU-KTtI!8=hLp&fPpWMgRRdH0%fj%&OqAd)&x!U#$Wz|C7 zk}8x%5HOjDM~i5X{r}&z1f?6Kl|i;E39W z6o0I#gg;m=_91MOXjnx^&S3H*1{+=*m*-e<$q8jTc6OGTKoC4FW%lb`z|rFhy4o3 z!6a4s`)^sH>*K4T{8MiwFW@kqTGzLDa!%?4Vd@F8X`Dy9j2(qDW>1{h6?f3#(BIQ+ z!zXo_WX7`+Z3$;U6v-*S>DLA+F!2Dwtg^Q&&bi7gY4d29?h}ATfsx#5QaVy}L^L=U z)n{H9Bla(6SK3UvuHjQ43K_c==LSo`Q)@2HSGCS`e zGhT_Kb!>w>&jX14gjh&!4l~9;M1_2sg18hc{eeS9_^Z*GR?j+j`F0GQ?Vd$!4)YZt znuEshBfZ1m_~yscB>tz!RGWM={JvYB*JB3mFeO87pSXwf=id&iVO{^Rk)`ek6kOvy@ew6Vqq+zy=ELm`dApwA)j!owgVuna4pXZG;N3F<6`7M)7!teRx$Cc4pJ%=@{w{OXt zFDC`AzFUME`!C4wd(3S#nIG#cfqklfX9c|8o*fu%8#-vi)8nTw>(|gv0SW(^@3m{JcPrP=ygQ#YzL6vnkmulS5lUImbe2Di) z1%4gB>O6JVGQ&rbA)Cj~dZ}k#Skg*4M8En`7C6D{M8Z!Z9v%$#!<~>m>c>6S_5+Tc zr5|&dU!+oaqk36vOo+^!;Pawm75s9B`k4J2HIFA!dhdFu{$;~kw$+0Z-XOxv$x-nB zwNbz@AjTNdc0S<+x=Km(n_i+&XToQ#CS4Inz>$!d?XU~4oUQT1; z@DMkt1X~s#`rX$2uGP8nDH-R__86}ay}Icaj$A7ln26M>H2wB3YYXTgiV^*JiMD8( zM*@sb=+GPv!LC`c@jL}ZRe=1#NFsm?rGSYC@#sb^$uVPTk~>~=V2Ztr?kI>ELMbw*$Q1J zLiNj7hMD>))IL?V`IFgoMS^8h&CKbtHuW5ks3AmiZF zs;|P)y>8>yvoRoZ#1B3`NUdr!V(B=kgMeLL)JZP`F5<|u#F{MLwgu`%3=n3F!G3;jL@wMCj6Sk;eL4^Z=1!;f-= z)%~7DTdN6lS!M!Q4kmLQH3(FaG3D@M%-)w?{rlKijDQ$Z1}O!~1aypNwx?LQ{ctV4!; z-Blyj(W+6S)E{3J8_k*}uXyAZF7X_x@X%R}qMQZ|eowVT8MaY$b+(`xvN94^diN3gU22g^kS(ghZle2Hw`OH@4i%3? zgI(NsffcswyQ_~*Tdop^bP_w#0eu>8cu9PkN*!W(D<^o(wmbp)MKe7d@#c8s(sdyKjuqAg3n9yvGqFNYN;PYH^O96+_6pHq6X}MA6Q>YXc z!nDWr##Oeb?Jmm@$$AcWArn(CC~r*3t*@TZ~V38x7l)A>O|>`Qw1#^yk|l! z%0umoe#uS$R>8a{M_Kp_CcCm&d0kr=ruyeT8?F>5$|JZfmKRXkhWtm*=WNSc5q}<* zQ4meYp59-7qin^9hD-FjxSN+SOybQ3pm*K)(c^L`H`(ZRXW=AM=8|1&Nw7i#ay`tC zCofY2wwmye*!F@l{QnOd_-_Nw+x9~a*Z!I0RfMYlUlgr7?(@XfGDt96sSzMP+SkWS zRU$?2E9(5?{rJ;zYTuBDj_K`xOtrbxRh(&P>#>ws;)rr@B=OIpTC1^geB7#WV05Xaaww?>?(Rd$186!j|opV-ORqpKT=sbRqudqGOznco{(YT-{jeLr*OP(~ap z3*r0a`23za?Z_&|3Dk4q`LwQ7YdQgP?g_v+E5O_0gWJ^_DK{kuzYm4Hu!{k&^=o2D zdq(77kzZJgW*j=e?gx#?*mrim%3ceOU?=pfk0R5~pWkBb&>M5F>>p|?k}ZY8^&CS- zz{T&sR9e1R>BcwfzYQG|SW6JM{)VbPsq2A9^cBLMX>P!xrhTQg_bCvY)R7;PR58xl z9Z*MZr}=reWC8zaNBK{hR@X%X8Lh%EvqV{l|Mr{UYIg7jz_TI)iSOEp`C&g)a8F58 zQwGtm50lWia?Sf`Nt4OoB2y|Ii}_LvHg0iGPWQS-$^Az`(t>?g_NMueLbhJG2BE9` ztF9J>?Lb->dz#FRg#m%92Ic76p)C0#s&zs_=t-;r!)CjSqHKBpfE;v8T9(eN4mJy$h4Q>_^%Gx zz01_PZ`6kxtoYF2yxl+Y{G^-{=#ThVcaz$h(-g)0iy_4XOJmA@2P1zO`biS!gk-*P zp4+ATGXo?@otgwUQ*G~|H<$(TlFsc&{y<3GZF=uX-_Rq3VpHuK-e$+_w&R$UejP*o zTPdK}l@xlp74(ps_^ zzvz>r7=T-$30Y*h3CG|1Yy-4MlpF1&C6l|e>JrHp`R_^8dADj8xf9;oitG)=Y#_=j zZ#9gcGz@2Sfk|xhKQ5kuH;Elt9aQ^AQDSFtGCY1o%P)SgiU!DAYOi&d#Opip1Z`k*Z z#p>wd7a%}X6r$8aqC1RzZrBa$T4pfD3?|WeN)OXIWpcp7&evBJ$4ugr|Hk`-+~R z*DDgy^G#U844>PYlv0-#&&ycP%2g={WqjA9%Z69`m^>bwDbs#DZ11pB=0w>(aj<+f zR$Ih!jV}#(@k&n-pcWGVigDC?YLIz<{fU8AfkC~I{qPQj5rLpEN89)u*Tu*pvzhw@ zAzCdw5=+sEO5OQZQ`1Q?p)8Kq{rzpcT4Prl9|JdW5?thAKiG)JNkiRa?_GCR3y-b% z_OLO5hl(4QqDu^?yXhz~N|A!Jju)2=4P#0A_cwYBT`xGVE6CJpgW;dpZT0B`wI;dV z>@Hw@MiEKUFIBQ!yT}cqaG^;=y5hVpI;R`hQ6zd(#^{*}SVLu3uF59?n17>Wu1!qM zpQXX1T(!yYJ$fH*j=Agf(xr}ozZ6yZC+h%j%g;K-OZx@b_faOLw4a^b(x#1a4t>U-&;%>jhu44N`Q>pW& zRV=a56IqU)eebMaX4ybr{b!O0g_CfniOdQ`P)t!bHciJujd|@!y^9%g6uD+J#T6(N z9YtcdL%4odi=lsFWr2sf4kg>xoQZfBU&Tv)ekd0QY9C1s83d1Z>YA37`%-Ae`CzEA z@>X3ios3--LcsSH_)c&LtT7HQ8sfV@jQAw^(e6&$kWrMpEXzvMSp0DS>mldz{7IX} zVB>?0<7<1b7Rg8++IV#1eMy6scZ3l~TkQ;RI-VgyD4Qu7Pi}MUIQN@>Z(jmr#dAbe z{tpYlPg`Fq7#$_&ugWTz9~0wKzzq%B^oCt+t))R_5)Iyp=LSRjHR_0Yt5v$0%d3>> zw(GfAnDb25p|oP}K}_%~L77?%t)X=<F4}Clqw>X|0-?wv?z#N^G4dShGza5t|A{ zf1;4#7f)?7`o-qy&k50aYe(r34ixgq{P*B!ly9z_C}#NY^>Uq}KZjLO^%9(L{8e1l z&0NM;tNsf%lKk)%PfhlTG;ro;z0TK}J;owXw^hquBvFD2N-v2+{y!PbaGHLcTOu{Q z4jjW+rkq*zWeNuKvm`)}@f0skg{9>mfECPFL*LI-m@n)N$Wl~EKswN22DAO%Q4!E} z-BHm`1vJKSL5EvDwp8x!N$c)rxrh9{rGoBEt-x+GqOocEy97FFFicWB>TS(IMFw&7bQ?Ai(oDK@`*%JRajG`EH#vEip93in6^6sVtvAgTCF&MKfLuS0G z_sw@#jfi|})OAL7HGN2=g-Lep*PHhaJz@Bl1N1TTn}Z3)6WD|0;jd>BeuWbN-PGOr zil1c%Bfr-|-ir$$*Upz$?_LI$lf&;|B}M_CF@IOP!^5$V_mks;z?gQK zZ%9I&%&Q*IH#1U;&zoIeFIJ0D$5Pvs9{v?Y;B=hMhX}1==sv@T2(PwO5?>YRHedp1 z&}WkXAD%(u$9k`*7+*U!$>*Zj#E(p!(qX=gi@r3Js#ub3zMTK-zR|RKZ+p2P6(Nr= zETGNR85@idOowU^iB351@9ZD|fE^axk5i-lcAZU$xo<=l%~Q7nFJ=9m`n2T%Iv(5M zfDzG8=?WG=n{??Ig`1;}Zr7=vMJMvZB z_-Vb8|3hFhsYZ&Rj;y{hr5U!_N+@D(p|$m9cEE%ydapKQmwFgL`zN)1UKYnq&FlM} zZpfW!74#)(k?vHJ16u2+7M7g`!v}PUUziSFl3nG1cb~ugx&Ttuz*L>x=9FxHMoWwl z^vIV-M=a@Ar&=P8#Lap^DXvxI4Mn<)l~{dxz&W2cE+qj=R7(wZ;mG4pKC#;d=Z>?9 z&W$UwsLooauHgjnd7-Wk*xWC|P+@ zfONDs!q=Ozhn-J9GeLb;h~Hiq?B;D8+RTQ5s{eEX-03}(C7=XWfUYQQUL6s6Ox{tk z;?ebu6LH=CltK0DMkpa3pqWu`2$ACzxu);<9XUwIr!OI*!yTUL`U;@v^&6coIjU}} zNSOdu1V~;7h>%L;gjwVxU$<9&Q|;#Lfzxe|BWs0*cdjSH#(T(%;J=YXiRR=2Kt~T0 z?a1IbGuzjrzXeV#LQ`z?JeNX7=w^xdbI#pgAc%lp03$nYiiZiPjvMXerc* zgPgx?gIlQp7U(2(=uobqHX9UTMn97fm001$q&l7eV#nZzv~hxNiP5#>1~?NiWh~R9 zD-tB716U?XVeYDX28jQfX1&Y!+=4WJl?m^)vi>mlk1n{<#HH1C)!jzrY#QRd`Co&-vQ;5r@*vn*9m-JajVlEG6Qlk zKKIcv9WsD~H}*bRP!peie|fSa|L8XyQMS9{woL6oyyh$Dv=AXwK1dh>rm+6*{6FUX z&tWVO|AiHi{B{E51)D1EpZH)pjKY^{1-xK%n+QRsRR!93|9{xr8W4bZb;YL_s=O^NA=-c*$R|E3^>A~88Juf+ z^Nhe*W%#g49n90Pg<`13d32`aibkYPeT!mK#yg17YU>!ikcW|Q7$0S`w5%^n^2Prw zwBGvE?@wQLW)3swo965vBYSDz+Alg#wSf`! zoWW}EMUXFmox7FY22hh$--LcXpV{u*3>)YWgr;Pn|vpfFU zSQ#O?RO1=N;cXMmhi0~kB1!Vfn7}Yo;zM^MwH; za4~vMlU*d~on4K@Rb7f{bT=k>-%iL-!P3V7`QM*D9(4p19+o=PD^|A;q;dOj0J4B@ zb{nz-j(3{2YQ<1(QZl{vP@~onDh|y`#1#q>E8#RomI>q84u|g-c>*x*LtIFQxPztTH-0b8um#P?rh+;X6Zj_(Y z&s!&ho#Gm~j?9j@<6!dxf3P+~M|I!+RDe+SF-NlOn2pdSH0ZyTkR_R5uay)YBV9=5nJ-^ilXlY8V|{Rk*{QSt+g8^e5|cU`(*`-&tWP!2mUgs*l~#4rAL zc4n=pT$s_tg+RxHTY5_laGU;@@JJZ0bZUUj5_LOL5nc;HOG$>u)|A!@0p3R?;_EYI zn`fcQ5c8+d7YV9+M2T`UFGai_nZ}VzLDJF+2flqDY#$IHIy)`>(~9vj>W)hKz0B8V zK~t|-KviPDsU^vKBObES2t%qM@rHNk7TM~2-xgWmI zzWL?nU4n6mjx%VQb%s8)hYJQkwuY0J<{+1R?TD)O<<=2%jFM6#LWEs$a#xYxF?bF| z6?Gk5qcr*(=*4_|=~2+*PEgys4rlKkN~(8-j~Kg^My*pJ-^!|gw!}-;;R4#X)9f>h zP+!l)d{cjsqe;Z=*$?z;xlKD4UO6S0t6pULTvcAxHWU({mX7+Pr1I^Of{x4qX)N=q z22&sE(8{v~Nf7pvrTC`QG!-4esHVy*9~kn>E(jiQB`Co?Kz+tYpZO}9cXS97TW`;u_)zbz_D zcASd@e+zCL5fX&@=d*otmgme=#Q-gsP)?n%hGdN**13WQ0+$UzG^m!$gQ`ufsss5P zjJOGY`pmwV&sS8Dkfh>IRP9O575*KKjvmb$=zqAX_liDNl-+$Q1+A8Q0E0AYBVEbt zkN>=At3&;v_X*un`|Ar_Dzh6enr044P0zhXMfM<-7|SbA2Hb9kI>X+}+0b76aUEx= zaqmt0^{ouQ9p~eWS*|hYoYZqTgl4IGM*D8D-?^+OL<#-le@ zNtOL(KX8zY-D~d3A+p2n$I!GS)SMEqckEPu&LuRypPqIYXC1Q`a+orf-#i=r6NIua z>H$|+nr-YE13OjWYN-o|IVN|y#=$Q4lA5+2#g^)hs43r9joFN5I}&mjVhn7ubK#Q( zp{oX0fR00Fle28yP2Ur%Df+JVlV5aYVoxX;$;WeZt-MGnn+Q>ow|qq&9GoK|SW{|d z+q14Y|C2Rv?!|j^eu7R%pmNjY7^k7%@nPL6`UUE4E?4v8h1bJV>pv!T3P+fFUgNBF zrL|l63>Uz`$ocXn=6-^+9nUN(=jH9B>?vi#KVSk%?B#36Lo`ZmG+@n->TDddv<7e8 zKAn4y-;REEluE-)h!wQ>K5sTiL%f8j&wBh%4&B)(# zS;9EGcHc_+xk|dCU+$Y+D8mVZK&`!X6(mK=XpS&`uSxcF*CRcDTHf&4l!xv1>RQ#~ zdkApFpl&?ip1S>v??5}{&qHem(!*;J2KVk7wfy_H!WcOwqa(Mu?BAG7%l>Am`L)?y z6JL`bE}r`AWxMG^Yvxtj4SyrG6-g|pe*??mhEPX84IJ3wo(&7UJM>)GB3nv)lfWa% z^dbVH_>$^h5R*8(Uy*JGt(~mimOM7XqCQ|lywe`-_WIK}<*~sRRSrW-nWHz0)v?T( zT%YzF`WPTy7hfv5i0=WPqoAxmx>jr(sdBmh$JO}L1I5eTjW>Q4Q!GM=I_qI0twour z*MctlzF zW#TCQyj-L;%x22J1ZpqQ7bEnUb56Yicf6=xFbZO%M8KH%m+%Fa%`9L(W-412n5+~B zdhaArQTSHw;7@^Af3b}cjLu-iO=tGZUBk^}>B^61|JjXN<-2h3@&-CAP|G?A19qhd z{*e%?JBWrPKM&?tp+WYk?saBXD@!4o4aN!E}*=)dNnqLqmSeWIzGsEy&>;@apde~$ZrpJ5s zwGtm@()fa5$eZ>G`!Cdg7B{4MR~gMP)!r>w@jqr@8v5_O6}Fs2kNQRZZ-*Ar1rY03 z3jKUv3va0Q-}l3;9X|qZ@d&^N^irbU<3D{9Sd*%TJKfAWHdR-{2`UlI(XII0k2YqU zMvXsovuxko?}=VZk-MvIpqh`;R%`!Bi!5CiboJ|LbZ%(xe0V92bI{0{mt3{fFa^{hs>6Gm{|Wsx%XowJ;s>@%oP0wiaoWLXr&lHH0hzs! zg?}G=UTDop`m{*Moxm}lU*0r-JzOUZ2*a1B{q1PjR7XIY ze4nE^he|v0xX|cVoc8>eYY42ZzrSLcO z*a5V+FGA4OrM+e3BmODIddteMq0ljSZ_%ye%Y=l#Lw|++DN$My@6+4w^3jJM|8>KT zT9z!m;A_k8r)zE=9__2kgxs}Z_0CCm{$5_Nvc-+EaTC4+xV~TYaha3o^#$`(CfQMG z)?SRh)qsTXZHW=;<-M-sn>%20BwXlN=xGx?N%}|8+ zh23pjwl{si=KJO;lggaWpW(@n`~JQ2&Zoi;Q;%mST2^IgpWCHX?)EC1L6H?x=>ag$ znfIs6L7S9zpM4bJg*X_<)FYi(Acj%;6+aaIa2Le*u*mi3c?)gME-<~-?R-^p4)j=h>wsK{DK?TQ8)g2F`4fzk1VWE4&rasJ$ zfv#+#*UO5h;cf88%qJpI)8lu$uvwZg$R*as#U-IX2ly;aD1S<*&v)bIaPOGU^o!c9 z@3xn9RgkZX!-?(MU8rd0k-uu?rIz$z0_@ILu{0qmw{3|UL9#}l`N;?YBD|N()pjf| zJdWamL%rtLYC)AyV$jI?Soy=>FIe=Ou3}fPkEdu|G4K)1 z=6>W0RPpI#2Z^gEM>u-d?_8qUd9(hI6!cyArQ7_5rcS`22$qxBz~w4DfpPkLl@Foeq;t<6^6-p|i9pYN4WcZfP~fG~oxjsRtNoZ&kU z&pY;1Ygxk^tItlkJEMwm^K{{ucXUa^TBr))#sn;I|MSrcW}&U&?%W>!WITo@vBqpq4aWNLj358-NH?+^$6{3mpIe0;JE9nOnjm>}akYCt`p@a-eV*G^ zJ!+T=F#1;uMudF(vj{e6Lv5i)g+HrBbg{B2_H(Bc{AGA_6Rw~F&?oXIo!d~kr}u}B zkYr3_MnSu6(rj@dPET;OiXaKoJC%s70E&FJjJchc2$v^`%nv1B#^vl@c@$(hAbIwp zDf$Emp`u8WyX(T-GAHq>xmhSnjNfq?*`%CZW6{;bq2GSQWwgrHOzB^tL2h6vqjsU@ z8~9B^b@6e>&F_m{aQL3uoYdVs8&}a?H8|sTU&cd@qu;j~RM~yn(~!#mEsnjM8L`ux zqx64Lh3hJIHDBD^av)LLQ%(F%eA~`%i&QirWI%?FyQB8gN7#JTROa2j=w%U9#cMqm zYEFZkdTgGKM?@VLRkL}!mYs)HG5elFH!jwHCqomw2xs>G(PyUJnzTK*{&?HAve*<~ zlO^ISgq)ePk9z&f?alBgKecRIbgpprb4zH;s@FEIACWl!m@6cAeXDcZ+<8*I2Nnkw zX5V3nCzxDI3k2MSHbX;HetU%DfSzO*!R1*VrdnSJ+hj9^;|H<_Z#>503{1dRNV&Zw z=YUwq#K;891oZ8}sXRMfk~9tzVMUY{ckE7jkDowDtXa(xZ_ zA;=o}$xOSbM`e+^w6!=aG|Sgxrmaoj%A*az_WUJ~PKUxF?}C+Y{Ww5@i<{lq*v$TY zz4f(cT-B1_-k~!jmAaHZ{zzSyV2cK?|g_lC_oW-?p zA-^xYpKK}0Gk}eD%O4kK8|eco1siRs_7hEw*;HXP_MKNGA8(KjW%byZIu&XlFNDn$ z+Y=y zBre**9lo|(@?}2{`XRHMAu4p${%P%yrGUe3LoUa{r7PsZ(hTT!GsxbsaAc;~y!e(jM!llZ*??mn?u#$gs6_>CPQTO|_hs#e zK;AdtfK`w7m|IN{A=?3}%(Fp;#m43R)>jt(E|xjs{@_GU=fijY&}faw1H?|N!x3#i z3Isl~m3ZPkZf^kT%knB!eeQpiO~3j@0momN(adCpLqSXrV&W5ikMbPEu(*-SBW1yo zehcHMld=Aph<;<_vufu|CRCiBlY&3SbE+Ikx#zrc{buW&#`*DN)%wC@!uXtjF#Gdp z$>Gcv+N>F=?70v9m|a#%(|j8O#F`Xyn^Wt{ zW-sXB47Y&@nz@~929Jhl+=Lt=+qVoBijGCIvA8xsekL?jZPr=N)_2x^)-aPMk;@NQ z5d`%{1n|2=ZFt~*abOv{@BRvcGdHV8D%^ws1voD#f4-_Vg1G3QrsDK{C^WKe-)i)w zrfhRKl9{jE5gCEZp3T!`%x%pbrq4Zt$_NTP+A5Nf+;Rt@wihYKXXv8^ zX!k|Bjs@)lv>(A)oz)oaQpnhHGTsxut;F;r!mBBuapY=Ydvn7tm29o3 ziig_At{hso&O2$k=&EErnA%BUYJgC6wfNO1X4nk!TI}pL^J1^_LPFN=w9*au&L2_6 zm+5x}_EOB$!qfIV9JzvetzR63V-Z4`#qQGn=J(&!Pu|()NGJnyujvnRRun1Qaj3&% zNS#_iS>>8llA<6zUN!w?mY;^-kqx7ePwmMjI$t=mN5C=Va13ymlW$x8RwMps&TI4A z3|(B=W`;uuw@NiEfJIF6nAL~Uxx_j@zV;~h`ADvk5~s3n5o>a|a=$C*T)sQcwArst zJGNxKt#LQH3TwVI-)*n)aWxdH%_JL9~>+9 z7wd!%F1I}3t1ujtOqkBTnoUKP%H@UWmyZ0|pt4tseSJm#rcj#IqY@EbaI+6H@I`z1 zOZNgH-}7F|vSYcu*#OY|By?}0EqML%x@g(|wn=3<`pWzf#y!<^58Hs2sp4-DekOS8 z6Ic(>C~sUF6~@6rnk=Yg3e9VB4f=BUZ9GitYA#6ysqXh!mmsX2oM5wjudkr=;Bo%x zn_(dWbU#Fk?mOcDort}lJ)t8@!QZE1Pjab5*Jb>EM(71snAkcQ7#Ho7Vhne+Ho%To zDE~<88;E5^J=7**x}dU9Rh(Qi8Y7yBymi(0@-Zs%5f!YkGeDL&FnXMSfZs1Kxu)oB zb6jzz)3iH4xUAVWM615LtTgSgA+97x4(vIsg$1*-$ppkN5f{EuA;+TL##!Ug$kcv6 zBbMaj;d2d>wRKMragKb)Q zWl;D>R6#eR$<~he-CnNY(eI=Vkf^iBc<~7`?G}-@$rI?Yvij%!&BMAH>nG9Q>ewGX zEKSvmW^V7*Ky+d+)g;xv^mGr8UUX|_dX{o`Rv)Pe%=~bao7;*C?xl-DrNg37oiRXc z<2O1??ImO3G)=>QZ0jo0VaTN`91zU#3|uN_4wVCJN}y(hq+bO^7A8QC%OT(4?skbK zD;hJ3m4Blkz5$?cl>I=tIeATlL2oF*H>#%AW;lIg*5Wh2QJ{GyAW_Mv8?h- zipmR-D8tB`Tc;sPuUnx9J-W%=QZezYGJijV zk>4~#b=o`&?3+L{!Y3grGZ&Cj5(j+0BkP5b4GGZ^){Ds|*!plF!3yT0rz+iQkzd`+ zC9*6}JKgU^m7VdOiy}wHOx-9-{EoQR@DOEP)r{gvlbv&=kf7RdgK}fjGR=U2is2bK zu+%zVXlqxfLaX={*5Vux#GdE;0mFg1`j{9a0xJeQu9fYv@1a`_%gsfsW@bkd@qf&FY5S? z)T)O)ee;6Hj#--HmsVD$_>j|Vb*O7`Rr$NI?aa+m&6n;HMQu~+KlSjbnh+1=k)pG{ z(U>??pnZ^B^_QTMnu)$g1Ydp$ zL)+&7*^595dRGpbvUe=MfpsEb=33ZuB0<>DA(5J0qCymDM&A8?e;*@&mp86Q+_YT22fv3y;PO zyK@N$gA-v;umcNp5OmwRgY^#RJx z;2inleD0{sMJ=mBEyzvJE2T@Jm}Qw(qs>3VC-HW0D%DQ&-2i#F6j}q19NMr9!SMWg zEP?oG#9-xHjs$UWs(sthv$MTer}E^dt~F2pzy~0xr8f6W>~^f@B3k9{VdLph$M;J4 zAw;q`@v*P^L1+J=DsQ=xdGAi_fi~b$PCQ|Mi8aQc?WWC*AUcbhwdMRv(HoMSeT=XE zx;l1~#a;a&iD=D^KcZK@h+=d)v49;7N6Ra?P$AKwGQ#{p11_0m!1Z@0?#U*kd|4z46XO_>^R!4J8j~R)6E{Cv4Qz6}2_5jcy z$HTL8k{5vogA&L5Npo>2vm86Q)ae!lpqqfkVx1z`D66QLZ=$D_utvH2L$z(IwS}rA zEEi44xU2ejuG~azIo+dj>?_qRQd=xran!$Bq3_4C&Nb+SbY841BO5tMLHoeMw!PZ2*u^o4+}*I1W&-Ms z#{Gaf_3*dG;5iFqzi#X6aZdcpi&~eqoKdm{xCGDj3ZZc~pSOP?iRum3|4FYUR;3tT zcA`AAk2i07nRj$-H?P)rOb;Fp+b}YU2>F<9vu%to+F-*(D!*%Qt!xYl-M8RT3ka*~ z2VoBmxsQvt-w*C(@e2odK74te6CU*!;?*yu15LND0#b{YvzI;J9Q%AJPIAlWxbmmA zNglQ=+8E-c7H=QZIN}XTJ>IEbbRMxm+lJ8>~ht?D?F%-S^{ye>GIJn>ER&KVdwxe7wk@b&qXC8@MQ*@m+tyZjE~=`si+;b|=iS%VmYLfkXP{%PAxTx_ zP<6jGR0JYhpn6(4op2P#nyylyy+-o3Z&>}hWUfqPf>Oh0`kP8|!|lTqD7fSX##YO} z&9{quC+(ivXoB@~1x~&Etf#NM+Vq?EDk(cFdu;14(d}@fyZdffeI*}}RV})Z~becq1L@Sfi;UHhTfftVEGW>i0HnD z@1+*ikKgQ0vu4!&TXp>CFgX5}`$P&uoHOK@>YsOA&&zOio^nHEZ|u(SE6cznf`auA zI^ps-v=)Is2&p}Y${&$@J$0c^$R^QF-k7tYx z<`yCTFgIoTFuadvRk8x~z^M}WcvKpiRiz>T7F6;&K4d*VN6vp3+;#1s=%dz2KIG`Dk;-)?ZsCh8gB#{3F;w#{4e z+Rol5MrdNFe1@N|vs!sa6p=G^by`jB2jpxu&Bs#pB1d$uRSxg96+LpMb%PeRPX2N_ zn`E07O6hiXv|#4;K;M-!wLG%?Yy7P`2CCVA*zS%I(OM~`$opX;GSy-OJ8kGN?NA<{ z8`%){Gvfb;xT>qo*`C`tpnodDvnwm(^))WkTGXITzS_4o^m&A0UuJa*8cP4)2E_kr zMd*E^rRfl6Ptn)I+7Xm+7tYvTU4aI;RM}=xh0=_0&i%@gB}#a;BQ_8c-JHu|Yy67y zE0_RF)+vHS@pihG3g@F+B=WA!0%z-fB((UB#3-B+Qaw#En5!Z14WG!h4hZVl#=KZ)l82;xOh%5R5OW!VJ+DH5odG;TQIj}wIFCrq_ zqTSpNoxGJ_<0%9)G>6OZuUPlp5TswF3!|~_?ao$xM3k4Neb2L8Y^gr^5Hwy>ASgm8 z4P9|+C=6{ARl>FG1+rhZ-bAX=gw(kcb0#! z{qRS>aMnlY2XqNh_QUv}0gjZh36{KPne|JH+8Tb5Rvm*tEINI8Y6vu>s{bsncS=OtVzLs4h zyRZ%+qXakPqJ7eev>5EJ4#zwRSb^&&7W^+xTB0zqNdLd~zB{U^ZrfKWMnM4)L6M>& zAYBEd7ez$rNbewm^d4F$0xC+?PnrP|5Fzv;y+%YjA@tBYp@ben2@q~Po}+jN@BQ(< zamTsijl&;dkG-mD$=>axH^)12*P6goSGd&M$-qo{)FjXR@ z7SU_>*(|`pakSB*H{Igj*Pk&q(b;V(Luy$dXsk^UN8zxFNUljVsU$8 zZ6ebfi3S}=VUyb`)vea|&{ylPH7bk#70jQjDV2p5t7$NbZ+7}buA`q_7K2J+6}Ieu zX3C>_Udu?yFrevh1(rB=A6HIB*;Kh08MHf>P_a?4$ zYV$E)pNrWn>~82)L}3EUkRdfwy!QkCkA~&5FL>^YAQ&uB)l^V7qX2`-2ntsy$HUhm~pPDy5ejaum5tbqZ$7P*e-4$(|#C(dN#U`H-#qbUYQqBg{1yv(V?W9zM(KkqTtR| z6CV4Zzbpjh)u(tIg^T9Djz`rrl(T%SKQF*25+dc3*DsTFI;?juqmg>lRX`wb-Q>x= zkEG{HB(YYbldshnA`0r=vdA?zuMUxMQoXK@Hq7p;gcy)r^^sm#{N%3kkuBp5y#jyO zvh^JCVmD~opHNr&VMqVKgm;HgiK zcv`w;hx^VU8@jtu(qlhmJ~FD^bWXuWA&GrQJR>#GBt+s~*V%+C+`%rzt0UX%wL$R6 z{03OZWp3d8NfW0%s-R+4zli&Qm*)*^l#6{fHKihD9cZI%U!F1*KumsA!-e;IO5yHg zA50~8PLF0G0;51}h`?yW9h;X1;bP5p*K6mS>m&xGHXD=mKlR1UMbcYx*K%yq-ESnE z7D16ir?Oc>jp^g+$s_!dFqSLf@<6t4>`sGiAj&*am8;oJiyrrEx>LrgrEQXY3Hst-GNVssgX-@o66w)OW^R;wCDS5B=} z;`@LLT4P2!0`qm!g|Wo9goh+0J?2pJu2s^u?{`qcI0P)Siv3ymSyGt|o=*P0_^l9tA&!UTV;IR0bV@t+?B1QN%X^x*YNsw|cg>gsx*>ODunX5X=Q19eU2XW%Bf zZVv)nyO-SZ<%)JcSC4QgyS#ybl&EmgY75>}dQ9}nuf3MO>}Wk!+rC!*?PdJ6d^VC- zdi~62M6GO0f@2XVCe51{_@>+~rOM*5UdDiSDh&uN#5%1D--e0o(YyGZ>!z#no!Btg zZt)0lb>Q)v$*Wx7qyXBbp>dh)U{tcJSnt{dE*ceUNHO&3E!{UM`lyTLFF)LRpvr!W z@#c3-*4eJG%=@Y#AUR;-ttZG-7Lo4EGK*JcH7ZG2iya!pIb~00@`7czpJgBVe#?b@ z&rIY0==rk#?#J?_^R$Jn%rU<^2cYfz{Bt zrOwGPv8N_d*wUJ)5=%g;aSatJ@>XXpg~%&SWAbvb{a&G0deVJDrKvXjn$esl;L_|f zg=eIYX>S-V5BIjFzDg<65P#i#@6%k1+>!;~mAXVUwq*6rqsWe>MU{&DsrOs*?c=<| zKtFqyI0<;{X%&ahP2PuiU}a)NrAaWSk3o()HnuMHwehyBp+So`l#bd7k<{ijI#zF@ z#$VMoTf@l4!PDagALo=PS1#YF)tfn(0^$!QP%GPSL#mT9+madobf)XAYQ9N4a5dkP!BMRNVUBS2Gc%i zsDtaolTrbs49f0KPc-EYk32F zVy){!`9jb{{rhwPUTI)wZaR~T4L{&9l?79%ka=h-h48yjv*$NyO}F{;E~c;Z2iJ!~ z;xxRY_!t7;-WNtN7)Hff@&~rk3v^5yJ71|HPrUod0e$NRzCyHjkU>Nn6hO zViW6FJl{I(@d8tVrn+T>(z3W9{fM|~*?V#N^-5tcZ)$Htu^e~(eMZR#F9syH+~QJU2Xm-po52=bKc=m1 zW?RPuer~GWP>pzk%bk+a)k<0zzeIa4Aqvk7e2D3kZV6t5l}`Jtrete2j(%*Me7)uU z!aBy0Lu{+$lyLCa6-VF#4xPgv5}D^@qBiOhrUl=_;w>(IL11iK$L!mNC7cGBGCa-s zo^S6MS&spz_MbvC#zd1NlAla}>)un(A8d>1$wnF;W;LqGEbC~Kl!pC@9xuK!k*Mr( zG|*6#JoWdCOzuqP#k{x{L=oXk(-j^?FB_V!xTJ0c-ybvNva9fN1kh3e(j5YrXvOF{ zWfTQQ-oGXd;;49&oVtScx&siSqH>j`zxGj%HiVHv0$fpiJ;c={={glTwaN4EL_yWT zI>f^_GCxtVE7H8L|4^MchZR17<@)F|g@G)*6@)b}KdqGe!cpAg2K}^{K$>?tT!^1~ z%!ev|e`L0Gn@~o^f14ZdnDo`X%9Uh^>UsOkp1^M{DM6A@+hNV@!pW|yIJ$0a$B$y& zKH8;uYNj5$U}yGKlZJ|wrRxTUnZEb8Vh<`^y_eARE@Py&y4EN7E)g{H(0Jd%Xok&h z?)>)68w3<-E>Z*_2^r_0sCvqx5)N7=6-Wi;*C+JIurF=$uH0m2;7{Y54XuF8Xg={< zECDq2nuMUxhio}L@fi}3YdRCS@i$F{{>bN?)MJiZaV+TAaESUT$_a-Tq}t3O`z4mJ&#_8#NaFxBD*6s|MYx91mnC#8*?%)--7 zBMpb~;czhVXQ5f*m@`m1d)__4t@vs!#h{Oe(CmzdS2#h_!o{Dq$Ln46Tf$5vl#ORt$@>evDBW^9~rtQ@Ux^3s99p> zLSg&ZYcvCETp3NT(eJhfA|1Jd8)LdcUl&I^)e<0TiitK0FeE3cWJj;3&>oiR{K)(- znY#V_IG{;ipYU?@;`ja#3P~oe#$*jg7oe~6n~a3#QSwZW?3>KLV4gptKQFF=nw;&k zQb|c`dosQ@XfN9b`;Kq+4{8x;%~)aqkcMD`Ebmx024%jf^!onI zl(J!9Tj@w8;sVpBOf?;u_P{r$T1I7RDTxZ=$zASc>TPTrm2UYJKIDutwA~p)>m%U4 ze4czbs>hXEyISa#&+SW(+RoxtE{}duA{6SiPb{;hr(WnY#}9Y#efP0xGu1%Q@Qt-f zYwiMuzJEn!@dWF9T-FAFP25r~!C`a~cT72NTjiH+WDJOKThR4L+=%m>Nq@ut34-SH z+P^84#Fu|R4N)hQ(K`FK1Oxq0lW#l9fsu^3{LLuJb!T>&h4oYL`{~rF!AE<%akp0U zTX+pTd)sy(kZ>IPsf25&1V@GOZN>|G*V0wWk|cz=uvOJ!ky|7m=rdCnFeOdN+n;%p zLF5@jtFy$z!rqz|13-=yRmqhDBSbQ`gn85wMsKP=)vtk?%ItrgAF+!R;oiniTu&+X`) z5`+oT{;D~i2%IJ%Ft4&!tp}s{u7X?e8jBXJGTW;?3(4mpoCNR~@7<*4R6CA|%1Q2- zJtN@0y}0Oux7_H|E6mS4tPIS&dRrz3x_e?Ei4k+Bp9StPwO(W=L?Ag-T$PfnnxQro z9zHB(bZ$+jyXpe36h1hI+t2e08=cFrSf8aI?eNAH9W-pAyd&Epn+|>UFr;j+cOrSz zrUV}>xDfQf;Yl5t3g<*axzNUEO*T&KVV4p+n=&xz{_-y^>~?LT2N;aitR;p z3WW}pN4X1IN?xNIpllP2^d-vMh(K3K2h|vdM)8rb%-bq;5HNp53^X4T8{+z|W}pVt z;ukxMwFxz;cg2|RwN+en+J_=mhfp##PRuKK``50kn$d0BaWank2im9X=6V)j*l3%g zZ8A_1qvXS1<&1KzuPMZ>$1HqC?oh+Bo7(FF>LEOp6>-gpMMC#f1f_LY0IhM3uP;k~ zCdm_UVNTDu+H*QD@tg6YdM`}$TT6(T3x0(O*i8i3Aj-W z4{ogzB@+<)9e5>SdvzvCpVL;~ADnwIC{*DwS-6|7c6uwFUh@7g2IA#pb-V4*8`?Iu zC7++Dc@PdVDPt5q-*L;GZ8qP}E`I6nGhpQ09Cax#<%L~D_8Wl_C_oqD8w7@`=v9a_;+d%dVE*t?@1!+A^WQNPXDN^&v=PYavFi-ST}K54#% zwD4pY9{&LgK23HL#cEmMZ6u|@n z3xmpbSD_}usLc~H5l#l2lfHn5t0Xc%<^{Iex}>W&CPUC?$!Aid=~R@i9XGU;4#pqT zI=zQ>A@hULOP0oZJ)xJ9dUYG6`;}FZ5sVkiWb3*vw*7sK4wvWR6+AuNBu z1naFYEizXV(D*EHSzy6ri<+=2RWt)f)euw$_Xnp>BbQ&`23VyDMsN07OlyRm)>1W{ z-g?mR%ObAcg3C5Bh+}W*2mlu+^F)TU=0aCAx~@T~eR1k#t$-#L$*keG9{!Nx%=&3x z1FCIQS6ztU?eaU{-(lWTJ96|$j=fz!r3E^1f@}TpgL_&@d3I=3$`N)OH53O%El({r zE^%K%Kdy1UFnHDi18#2)#_&&A%nhb!F_+z7UMR8|x%~Ow_N{ZEy~SCOY0~svs6|%D zib|P?I9EbsdwZK^>EfA{-*p`Hg`059fs$rfZAX2Ch3O$hglPsM(3#B3 zBuxRrq}oxB<#m^0`t)#~P66QIvU%qWD8YwTK?o8nA+)9mJvqoD$(h|FhOA*CC21nt z-xb!pZIqS{Rj1e*p)BI(QQzw(acL8blSJ_C=!mt``LQ>c?eUmH?dcBBG$^u2R>2KV z?13Sb;cra>Wk5_@Dh7}}(Cze{+1;Xa-!bcb@+)wm0qiL?zI?Tk$_j6{v%hmk)T3nS z-h7Rc4$Ptfs83^P34@wyo_;9K-_EG1Kz49rNmPFM4SX;JqA zhxf~`T31zz5NgKt=_TxCpETkw+k$e2gf68K9tuQ6KFmZ7cgFiOGI6c-%#9SCB|P$t zP1!fL90V2|Sk7*^?#QH$3+Ep?o|nvv+T`=P(OlT^OkK(SmG?TfIfe&}7Je>2Z-Xs* z43!Z)IexinqT6gW-15?^-&T}Fj`vwFp?193c~eVFyj4csZSHcU@sy$#cm?_yL9KOY z=HMUK$od%)!li8+fbtafm|JBk^1b&#FY~ zLJ4Dpq>U;rTp>c`eEA*?rRacW-@<(GquWWgdYOV^oo=MDD|&z-{IvXdkM=6}tJakV zsd1(5BL_l0EjX@ME0eBWAgJpcU*#4*5I#dJC=C++TCoh6glpBn2(!pANNuf;B5Hal zX?=H^@`QVLvuOxkeg5Hn*vD4D_s?a8Xh5`;gx%Ev38hyqzXdx-QSR9tX`rc1rG>vD zUeI@vSdL5DFa=NdDwd4&bh|V!?J8b4AGJB5qkR^oD+|0HRGx7P-?Z??u6H6uxLny zwe5Qh@atkPO5b(%{j3@+*_y479;0kD{Hi^Q(O(;rdM8jc`oyFZo(LLL+tECP%pQnD z>cAaKqo6`LZxq^; zOqH@6BBtEIsskSDl?bp}fUCk_0l@eiolhT>WX0Dslfrz^NEYg}4L&c{g#J4kh`4f7 zk19*C(tKn1OVFTcgUcXjKfmnIswT;!+tEnp8<5+dM}FAcZ)_ga4ir2U(7v<2q2y} zS+2)tHgF+T`kpufk-KAi5Hw2=7Z)991AZ;Z^>|LsD6Uc<()+4jfkMNmsggqfy7l0& zQpn*d?vuC=#zISyO461Ytx55a)KJ2V#OhY(;ewU$b z-<=)EAKIqO`Apiy0ef$>B2&pDAbZ6tipIPOA(w2=8!QwX0nL-^q2@#W_@v${5Zmb3 zm`=kJdE3|T`zLoRS+u*u?z{()bweU6zd4dhwk zTtPDa_CO3ov$-VjO4#=jIa7D0?YSw<$5Q;+dD;Z!E4c26G0KG=37NlE9O1N3ZO1O% zh!9(`MLDMuRh>7Id8Mw##$Or=51YGYf|W)=DbH;d+PBtWgatvfmhH)++C!GwwVR_V ztSTaysQGX}!1}_4jBeI*n`fqFShN-={TL?}_6l~-cA%!Qp0mpHC}k{j-PqYv)JJ?R z&6<E1N~E2YLd?zRx(?M`bXl;#PagzSU*)}Hb5d%h}-vW8>t1- z){2$8Nv&&GorO&Je9~m`J7^SDa+Twg^=tlw?;Gw>0mw3=1WTNpS1T4PvK+>Ug*CmLmqs9)NyueU789 z^2RIBSmf&Wof)w~aU|4$9c|YtF9$=diud_j1Dl zj?Ovw61DXD-3Ap;mN^)shc?pJ;>WT?p25VcO;?jNc=mo3*QtOL!mP@O`0^gfbwM7; z=5tu0k8fi)3~f&KN7eb4uI79^Q8xx&rU?!HukSay=H((yQYG;!z5Q(D29@}w&6-nGFkam_T zLq+!hn*IV#kBtNC%}9DIji-w_ooxzGW9j;W$G7wH5ShqfYP)uX?O#(-aQ|db&BduA zn_tNGJJmf&7JKpPfT2$0?7vX&Xd8`Q3Xy$^^+BOyhi#*vS>oeqay`QHZ(eiMUzj<) z+OcQ+qr-nLK?Pcp=Wz^VW1@ZSUi+Y3E;mX29KeoUeIZ+^>=htjR0Y04Z<0v<2U&sH zwB=y4)hxw&pFi&Z8p4d9$ZAD={Fz^OekEo^HOA2rS%ZgolC1`YwO+iDPBO%TH9F^2$21ubPbVPfBu6cRtO(sG}45i>M>C z$W{>FDWm9cwnr6h@3o+L_|UL5da$hRVGb5%KMFHb-)zqXTwAzNXcg zfD)__NALb-zaWQbpFh z&wGv?_HW@0(S-B!G&gcDFMe#YWZM2YKu(NoT0Z|MJdwRWC@6==~A(DJQDWlofB_aXo)+ z{h>a!lsAuS{F}qdI<3p=k^IAA0;vOKlb)mvf0HX&qvicg+mEZ_>#|}0H%p>fD#5f3 z9jo5f@-NK(*MSDnjq9c@x}V{WaB#e>5heWe4it9#SjoJQ1^od0We2eE{{is75G}N= zwUFspkl~>t0{%7Wgx{srBlcvbPsAt7Sv{&>h;_uK0xuHXdM*uA<=A2W7T)|Y;Y!t( zGOLW|EO9RXOg0L1NMdR$-3`6gP@|hXPLmD#M~ER3Sk4`oeP$~!ODN~}w%|X$@%NnX zzX>&n#i4vv+``2^#^DiFwP$CK72@e)z8`U)$4NbzS50iDy?XEAl3h-#TU}}C)oYMf#owl z0Kd$pKm5-*`PbL~R^WYlD>lYL>{vIcbCW37`@VN0jvJPEmf-(8-uy7(MRE3Va0e-WD{fgoFpLjHpMMlgLsM|0Mc9ef)nC{r}Gr zea?ttbt%<)p+^cfIZZy&PsU&&wZKL>DVawxgfN?!m8xl>dFJ$*l@D6F9jAo(fx-b0 zPBe3b<}5I#u4tyE5PSy=`l%W^*E^+@gFCMFC7tt=aag2uV(Y`VhFz=8-zU#}Jgu6w zSr97Fm{Pm_+D293Q21e1EXvI2L+LLAp89+*F^=3H=oIBBYIxNwgwz)<|ekfNW8 ztIynJ;JfKg0V-d28m%Wa1`2W|vDka+727`G-Hk6kK4a=}IH;RxYxBl)$_Y*eUD`$D zN2=<84*&7!!s-C&kyU8dO(0Goqps_H?O&V5X>&kWKj*_3+W^lbEMcwe+MlKU0xA-U zE!SprDWE8XyJJGqG3StJJ~QI}(B@e==7srE@7~A5AsytxafUdcP;EaV5F$u?vt~o% zJ>)@i+^X9IgW!zK^p{z;70$YpIGcVE;`y8pTJ#CZfc=g<5%&31!Wv-`D+ina`}?y* z^dPbl@C3rpQd;k7x+-d1kbd1$m7s;N*nGCp#1&A4@V98p+5fZpB&8C+RtmvrN$#LxAO!DLh|IzuKinG<>lH~>gEP-V zgJBdqrBki69r&p#wGW%icLjIUjN^Qr5bcP6EJXd1XtGKmq(_|9Fv%}|svfL(61tjD z$44)IP@JzrWup$0IGhWB^=c{s0HCJVV!<`qtOJDy7NF(c3qfx+Z!|qr4~TS0^tpc^ zChuxbhMT+8l*#FUf>JLEdF@r%RCTjv`BXMS;HYB6P+VfCuCWa@OT_o*6uOKlP;7&k zAm@>_Kw>Z(=s|vc0Lzgj%8sH=zrt${4VEhDYP(QLvF@%_1h4wQ!qvXln!g{)xUdWM zspQshduOos^%3SHvu^+5dXZc%=8Cb9*C@leY0;SNeK9>2IMy!2U`9QFf9mb_mb~CZ z(;P}3XE7=hU@AJIxGF=qx{RZn-%affNPeOz72wG=)Z_g05VAuE)!+HEK%$o-D-dbu zUeMjO9!_oUdu!Uaqe*o4hLP~IM~k;RgE_fTN>zpnYys;?7#_g#IXKTi%sT!Y&I$^Z zT$6E$kM<$+ob()%;XF|7C|dz^L?1qjkf{X>29P0}?7pu*uMQaUJ9NhZavgw6LL*vj znu*}2I*dCm|unr!(L7$K@0DXS)hED zW8zZg+1u@_VET{(m&lpL=@k&wfC@Y8x)pwgYTq?v!<+Z;&$WW=3(k90n!!;vbhERh zUxrd3-9o}@YlD^kqTPXNy183Vq&jnL5T3St;Xf;EW=Z6mY4qcE33wAX(k<2QzAf;z zLzk@L_O`h9Z@vo{U}sre+p3SOwEqYPvd=fib@Z$&T~?ef+aI~$epNFMj%uf4pwmT` zOHX97k=UcL=l`0UV*xClTqTtT>@ewJk%8O<>zG5mS#{*NS=a5?@Wv5wBRQe|9*d}r zqD#x~SOS^KA*-2MafA~*V1Z88*jHi z4DjY4$p53pW@b!&3SWz5%}eax)u~qt{Vk*t?xm-%>bcUFHSTU@n7u+!<{6<{X65{g6&MQNg|Zf zKMTR-wt*VsY$a*?vhEq(6-o$I z0r^AJqBz%+rPur?4!+LGf}o^V7H$Q|3!^??q5vhnem~d%5~vJ9nksAi&dt*MSA>ih zi3sdPT8RIVH+QDX3g-cKFIkQV$*gsSQ``9pWg1pEmqUFbvsfBa4FF=&-^HA)`#r{< zHU2E^7CZ^$Ugi?k0;uDxqAfC%l)^U=Ityybeg}pT!U$7$0!DWhP>Y+QR<|sEc3I31pnFS6pI@Z0<3)_U9fh{5ZBaPqG*7m`=DV7vWg&L7p@e<4WVoB*-) ef9Tn@!;|Jzk!gC + +# RFC-38: Spark Datasource V2 Integration + +## Proposers + +- @leesf + +## Approvers +- @vinothchandar +- @xishiyan +- @YannByron + +## Status + +JIRA: https://issues.apache.org/jira/browse/HUDI-1297 + +## Abstract + +Today, Hudi still uses V1 api and relies heavily on RDD api to index, repartition and so on given the flexibility of RDD api, +it works fine in v1 api, using datasource V1 api, Hudi provides complete read/write, update, +and small file auto handling capabilities, all things work well. +However, with the continuous development and evolving of datasource V2 api, +the datasource v2 api has stabilized.Taking into account the datasource v1 api is too old and the spark community +no longer spends more resources to maintain v1 api, so consider migrating to DataSource V2 api, +and use more pushdown filters provided by V2 api and +integrate with [RFC-27](https://cwiki.apache.org/confluence/display/HUDI/RFC-27+Data+skipping+index+to+improve+query+performance) +to provide more powerful query capabilities. Also we could leverage it after V2 api get evolved or optimized again. + + +## Background + +The current Hudi read and write paths use DataSource V1 api, and the implementation class is `DefaultSource` + +```scala +/** +* Hoodie Spark Datasource, for reading and writing hoodie tables +* +*/ +class DefaultSource extends RelationProvider +with SchemaRelationProvider +with CreatableRelationProvider +with DataSourceRegister +with StreamSinkProvider +with StreamSourceProvider +with Serializable { +... +} +``` + +As for writing(batch write), the following method will be called. +```scala +override def createRelation(sqlContext: SQLContext, +mode: SaveMode, +optParams: Map[String, String], +df: DataFrame): BaseRelation = { +val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams) +val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters) +val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*) + + if (translatedOptions(OPERATION.key).equals(BOOTSTRAP_OPERATION_OPT_VAL)) { + HoodieSparkSqlWriter.bootstrap(sqlContext, mode, translatedOptions, dfWithoutMetaCols) + } else { + HoodieSparkSqlWriter.write(sqlContext, mode, translatedOptions, dfWithoutMetaCols) + } + new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema) +} +``` + +Regarding querying, the following method will return a `BaseRelation`(if not provide schema) + +```scala +override def createRelation(sqlContext: SQLContext, +parameters: Map[String, String]): BaseRelation = { +createRelation(sqlContext, parameters, null) +} +``` + +For streaming writing and reading, DefaultSource#createSink and DefaultSource#createSink are called respectively. +In 0.9.0 version , the bulk_insert row mode was introduced to speed up bulk_insert, which implements the `SupportsWrite` v2 api and uses `HoodieDataSourceInternalTable` for writing, +right now only bulk_insert operation is supported. + +## Implementation + +Spark provides a complete V2 api, such as `CatalogPlugin`, `SupportsWrite`, `SupportsRead`, and various pushdown filters, +such as `SupportsPushDownFilters`, `SupportsPushDownAggregates`, `SupportsPushDownRequiredColumns` + +We would define the key abstraction of call `HoodieInternalV2Table`, which inherits the `Table`, `SupportsWrite`, `SupportsRead` +interfaces to provide writing and reading capabilities. + +### Writing Path + +Hudi relies heavily on some RDD APIs on write path, such as the indexing to determine where the record is update or insert, +this refactoring work is relatively large or impossible to migrate to v2 write path under datasource v2 api. +So we can fallback to write to v1 since Spark provides the `V1Write` interface to bridge the V1 and V2 api in 3.2.0 + +The writing path code snippet is below + +```scala +class HoodieInternalV2Table extends Table with SupportsWrite with V2TableWithV1Fallback { + + override def name(): String = { + // + } + + override def schema(): StructType = { + // get hudi table schema + } + + override def partitioning(): Array[Transform] = { + // get partitioning of hudi table. + } + + override def capabilities(): Set[TableCapability] = { + // Set(BATCH_WRITE, BATCH_READ,TRUNCATE,...) + } + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + // HoodieV1WriteBuilder + } +} +``` + +The definition of `HoodieV1WriteBuilder` shows below. + +```scala +private class HoodieV1WriteBuilder(writeOptions: CaseInsensitiveStringMap, + hoodieCatalogTable: HoodieCatalogTable, + spark: SparkSession) + extends SupportsTruncate with SupportsOverwrite with ProvidesHoodieConfig { + + override def truncate(): HoodieV1WriteBuilder = { + this + } + + override def overwrite(filters: Array[Filter]): WriteBuilder = { + this + } + + override def build(): V1Write = new V1Write { + override def toInsertableRelation: InsertableRelation = { + //IntertableRelation + } + } +} +``` + +### Querying path + +For v2 querying, Spark provides various pushdown filters, such as `SupportsPushDownFilters`, `SupportsPushDownAggregates`, +`SupportsPushDownRequiredColumns`, `SupportsRuntimeFiltering` and so on, which is more clear and flexible than v1 interface. +Also, v2 interface provides the capability to read the columnar format file such as parquet and orc format file, one more thing +is that v2 interface provides the capability to split and define the number of partitions for users, which provides the possibility +to split more accurate splits and accelerate query speed on Hudi side. +However, for querying, in first stage we also fallback to v1 read path, which means we need convert +`DataSourceV2Relation` to `DefaultSource` in analysis stage to make the changes well controlled. +The code snippet shows below, the `HoodieSpark3Analysis` should be injected if spark version is equal or larger than 3.2.0. + +```scala + +case class HoodieSpark3Analysis(sparkSession: SparkSession) extends Rule[LogicalPlan] + with SparkAdapterSupport with ProvidesHoodieConfig { + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown { + case dsv2@DataSourceV2Relation(d: HoodieInternalV2Table, _, _, _, _) => + val output = dsv2.output + val catalogTable = if (d.catalogTable.isDefined) { + Some(d.v1Table) + } else { + None + } + val relation = new DefaultSource().createRelation(new SQLContext(sparkSession), + buildHoodieConfig(d.hoodieCatalogTable)) + LogicalRelation(relation, output, catalogTable, isStreaming = false) + } +} + +``` +In the second stage, we would make use of v2 reading interface and define `HoodieBatchScanBuilder` to provide querying +capability. The workflow of querying process is shown in below figure. +`PartitionReaderFactory` located in the Driver and the `PartitionReader` located in the Executor. + +![](./1.png) + +The querying path code sample is below + +```scala +class HoodieBatchScanBuilder extends ScanBuilder with SupportsPushDownFilters with SupportsPushDownRequiredColumns { +override def build(): Scan = { +// HoodieScan +} + +override def pushFilters(filters: Array[Filter]): Array[Filter] = { +// record the filters +} + +override def pushedFilters(): Array[Filter] = { +// pushed filters +} + +override def pruneColumns(requiredSchema: StructType): Unit = { +// record the pruned columns +} +} +``` + +### Table Meta Management + +Implementing the `CatalogPlugin` interface to manage the metadata of the Hudi table and +define the core abstraction called `HoodieCatalog`, +and the code sample is below. + +```scala +class HoodieCatalog extends DelegatingCatalogExtension +with StagingTableCatalog { + override def loadTable(ident: Identifier): Table = { + // HoodieDatasouceTable + } + + override def createTable(ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + // create hudi table + } + + override def dropTable(Identifier ident): Boolean = { + // drop hudi table + } + + override def alterTable(Identifier ident, TableChange... changes): Table = { + // check schema compability + // HoodieDatasouceTable + } + + override def stageReplace(ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + // StagedHoodieTable + } + + override def stageCreateOrReplace(ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + // StagedHoodieTable + } +} +``` + +Users would set the spark session config spark.sql.catalog.spark_catalog to org.apache.hudi.catalog.HoodieCatalog to load the HoodieCatalogto manage hudi tables. + +## Rollout/Adoption Plan + +- What impact (if any) will there be on existing users? + +there is no impact on existing users, but users would specify the new catalog to manager hudi tables or other tables. + +- If we are changing behavior how will we phase out the older behavior? + +we should keep compatibility of v1 version and make it transparent for users to migrate to v2 api. + +## Test Plan + +[ ] PoC for catalog plugin +[ ] PoC for writing path with UTs +[ ] Poc for querying path with UTs +[ ] E2E tests +[ ] Benchmark for v1 and v2 writing and querying \ No newline at end of file