From 74108e558b6d497546f7657971648d06881c7f58 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 28 Jun 2016 07:27:12 -0700 Subject: [PATCH 1/3] Added Structured Streaming Programming Guide --- docs/_layouts/global.html | 1 + .../structured-streaming-example-model.png | Bin 0 -> 125504 bytes docs/img/structured-streaming-late-data.png | Bin 0 -> 124617 bytes docs/img/structured-streaming-model.png | Bin 0 -> 66098 bytes ...structured-streaming-stream-as-a-table.png | Bin 0 -> 82251 bytes docs/img/structured-streaming-window.png | Bin 0 -> 125098 bytes docs/img/structured-streaming.pptx | Bin 0 -> 1105153 bytes .../structured-streaming-programming-guide.md | 888 ++++++++++++++++++ 8 files changed, 889 insertions(+) create mode 100644 docs/img/structured-streaming-example-model.png create mode 100644 docs/img/structured-streaming-late-data.png create mode 100644 docs/img/structured-streaming-model.png create mode 100644 docs/img/structured-streaming-stream-as-a-table.png create mode 100644 docs/img/structured-streaming-window.png create mode 100644 docs/img/structured-streaming.pptx create mode 100644 docs/structured-streaming-programming-guide.md diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index d493f62f0e57..2d0c3fd71293 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -73,6 +73,7 @@
  • Spark Streaming
  • DataFrames, Datasets and SQL
  • +
  • Structured Streaming
  • MLlib (Machine Learning)
  • GraphX (Graph Processing)
  • SparkR (R on Spark)
  • diff --git a/docs/img/structured-streaming-example-model.png b/docs/img/structured-streaming-example-model.png new file mode 100644 index 0000000000000000000000000000000000000000..af98765fd83391b444e4f00e9c8c76ca743206f3 GIT binary patch literal 125504 zcmeFZXE>a1*EOC@q!5V~EuushHKGI&Le$Yhq6DKyCsBhS6VZDQf`}fyjXFqlqIbiH z7G;dl#$XulCBOT5kN0?v-1o=-hyQVZ$OL1q>zva*_g;IgwI@_dL+J*|ZITNYF5FO2 zeyV-p!j=0ME?gSFb_Mte*R5_{;J=Hm+DdX4N(b(&1K$w8RyJ_GaDkTc{NF|E%n{%v zvKLgIK6(E3;^y4d&(FuPtve48v^lCT8bI2&E^*sEe_BCB9bsec{N>p*9*>JYPoCYs z9m-L$?5r6v{4}J3mz(@jFRMbs`x`lJm5qBct7zm78j&)cV&yqKeF`63z3*yLk88qg zB>DVC^8N_A0J_HV{=X0Z-Gl$e!GDjze^bGKGvR+T;eRvX|EHO7%nPcOP7B~7qKyx- zKEB=3?q}_|a)V>S^Et9Hc;c1`>ah~81=VbfvxMFB-nrO-8=ga!4bROS2X7g=TgwA4 zB$xd+4_`@UzbftdT1QRdirmRigpw^@Fw)Q~X)E1L(-NUB=&kM9IC#+CFgOV>$Z1kF zi5^bB@gFWn9&*oAg%m8jyy2I=B__EDnM15?!Saq@ec19y=U^Rii>}3v$A`PsqL_#0 zdg&Vw6Z*J#H&5cos!bM#{}fi_w_dA6@OYaSZi!)aiqUta%|b)(RtO9&iGkV_ogIcx z)#fZ5vDo&9bLhnXxIk{UUvYGM`w=CP6wU~+=8T(GS!aEBTd7#bLdd>B#P!DYH-;ul*XR1uT<$4S-%d4jL79}iK!<0ZEDch`7znagp_J$3g2lk+S*N2SL_o;Mwte+ljD= z`j>_|_y2m~jRN4plE;Co6%?{xet?qNr_^z})TK}cioXUf%Uxj2#P{D&03kz8QO+LZ zcq!cEkoNC^`AzjbyW)!;Z9&krF;CDi&^ znjXg=P!7=f4A&^ zq7NzzO0v0zLT3L=S-Z-2_Zj)=^TKSgRP>d(meIKl944*iBy|h@?{Wf^QkN z&2?ZgTKSh})<$s?@cOMPw=+4M+<(6H-#7p7d;Hg<^8ddJKxeqfZ_D_-OT|>ECZa!| zz3aipG&X;*@@zarO$6l52$VVCjxmB>nXerL;rrSRH47g?-?T{a6JWjJ{BMz4*m;xP z$0Hs&6v)3~ju;S#o`;<6E)b|;*Av`wtk+QJN3{u;&in zD?9SsyCOvC(!`weAa`i4|A(yZZrmD2A`$j0^2!j7o{Ceg0Mk1$TZRsqy&} zsyC^eyMA8E5`+FERmON!uN%y3Xx!}5JySVAh}(lqQ&Q_rG>mLE!Cy|7?sesL^|fzA ztCtB<7eP$!i8W4?Rx~(X?|fxtpM6-I5H@bU+x5HGyJVG)U$>|%N5Gy}r!4g0O$M=_ zy#wEkJ9+Np0m7K?}7qLU1(W^k?4!?2=sm6gIn@P90d3;RNz#41tq|ZAr)5=_7 zDmnqPSes&Rqi zDc7jR=pO7#c6g}XU-Rrg$5HDdnDQ_l^gRk^s?9RBQ#zTfO1Xx$%8RL#1eb%=`*OOP zDl^JngW6_XO}b5szfXN&6Dlqq+{$YVOLM58JMBp{H9!bFm`j`=pV{fU-7);&_27^H zw1LJfmg(zZ@2bkX zYb-Ghpd{rXcz4E^9C;bn!|esbVh2VvuU@GdvxLjCF0Y}sDG>iK@Ox`ju zFsZC=l8+9~PQ<>3Z6UxLB#B0t#L>}F`M|(H-4@z#WM;E zgP5xFvODb@r)wI}LzCF)>FIgG;ZpPi!eN(8DU7JI(yGs;GlJg2$;`dSX0E|KNix;& z;o9x~XJx!2_@*hW^xw}G&s|wcHjZXBqEOk5>qJsBHS*F$W^F-lsOZ$w#BM~>-8aXt z4QESzYYl858y+B(9;fCJUnEnA2O*0*P7FtWAkgny942QilsN|8hra4LQPL&>cwnF*yX9DW1q%w%8eMAE0+zviJA}t z7Wh0`_Rvv6Vp>c|+9z7#15n$8Rz8jnKe$Kv8M#c25A}?Y`-*Fl%<@z{QKbyl)Bty;- zpZNLnX9T8cZ=jyWTsV{^+@LHvBCJCAeCg@3htdXxH5dbTeyu$LnukhKDj9Pf_lF++ zDb;bH7btoDJo^>o#B}tuY*i9Chb#ur7*6()xm(&3 zI}qe8ZAm${9O8)UHEm}C=sIW98=b26*yC=M`#44?m6ux(a-KB7HfP?h22>E0;W9ld z*l(vH?RE6hn;j9bCweBn*#5}dqQ?OmHPpM&t=BX~l(<*wub>44)~|H0`?Q2)4dVN~_7Q?|WmE^3NhUh<}cF3&hMKSxZXGa_$wr zO1^%wUDw@Nc(8X=Wz2N_;-9+nJQM(D_snv&iO64rQmd~`Cur$vSP}(6lB&!*!h&l? z34H$fEpwS_(;UPn#3WKQWe$$*s!3DuodGVYv~e8>-C}h$hE7GOAAi{DXl2?IfRpVm%LyAYOo3Bx(Av7#jl&es<6Px7G0NX- zCyyK1dq~tAHBCH7LJ~C|R;orJWgU`YM>$ z;KyDmH+rCNf5WV&woZL5i|rUXB_?$r9PhjC-hL%ph&1YNjRs3D(TMiFmmd@^Kof42 zcjP;g6u#AAp1!4xnk=iOpo4(gn8qT*l*LrE(uqm!iyGqj2NXiDY=lH~tq!E4IJiY( zpV?73GeEN@^+9a&kFbyC_ilXn>y504)`nD51gW8{F{z-5H_hJIjiK&XHh31C!~Q`+ z%CXIKr4_B6+%G;b$<%Mjjd&H)DfpImQClwzpC41XL~yCzD|M8zHvDQe32Q`}z#2ZR zm?FMc)}W(BUm758SFUFGKp6R<6=rQ${6TD8TYqmPSrUmpY1~U}e%T%-OG09^>aE{) zpa^PLHCk1fWyxlKubB{*uob}5t{2+vScP+KpdA;1x>CINBKEwe|= zlx-3zs`%C$);cjdJEj!*69sVy?Qh>yH@!9G_4lSpTLlQMd%iUx)X-bhLpS$}YwRbJ zQswTkv1KbO-E?4<_I6WJ2Yq&mrs3@&7gUK+)^07`;d;1OV>QX?(<~g%-vdLJPS#zG zrho9tANX4%k=?Y(t#psy)s>`Bj3_jZqP_pkjTPRn=V-6+Dtx|{Q}EuSAiu_GPO_MB z9kEGK{dpAbgJZ3os`h9$$srDkV5g!x@*ebt*T!xc5PSe!U`D}bNJ>EQR&8p+^c!J~ zM8UlHcpjj-udrKyM;ivUCx<;ugrHel(_UU)5SO-rsy_$qe`C)W5>Q0Da$p(deb$&C zpcyYGVp0a;UmQXXBs|HjL$Md@ftn|8r2Rea0_>M&KFmf`|*@(=|>#q%hI_xrcVK+U?0<$f$`vWt1b8z)UXDJwF})f+{HX0MU* zui7nk=Ovh{lLlD*KQX`(c$si?(B-mCyqdpFMK1Xdi0dJaX;&TyFQSlT@w(`&PKuTt zd}D8_PYv^E+g~3|v*q&FwyU%%~=N-cMh+%RZ|)5Qu_1M{UOnp8Z~*Xf6M!SjxXq z5}GYnlze_#r>H>_!{&?v4*x_HF3#C6-Ju#7wxwiQdYQ?MD*H_j8w-5N@{=f06Dv|zF+Sx6j9;jZf zrkeZxdbd*&ahtB8&Mk|nN>tc^C`g@)dc7_>nSJOcp_87T-m%8QFA(X>Ce%dK`RTp| z%*NPEIw^}ds?lxvN$iAI{KU!)i`SVa>K@PKw~t-S_u^=_3^pN%Q^%`)L0dA+C7;#g z+G_X3eCg%x)M(n5LLbRPrTyp5$)c}8sXG(_aj5j z^j2iwti4KjxFV9Xs0IJ*V9r|5p!{w9&(yudvA+JK+0ToaWoKS1&(_DY!6`lxT6m%# zMX$u-=PVP{-Ys>;41CbSXT)&HU4TBn#S$7DW+G~?$P!58w)}J^nAuNXOPG>nnUc=C z$rMN}#2i9K`7ItJ!TYCEf!Aqg$ioc*KoLh2RzD6X}ePsJu;^r=!q)Il9&&};gG zX}*8uO1iqKz7K)o_JNh#V3Gshchsc{FCFE2p4?6Qp>ne}2X#~S0}8(D5kQ`@YijL% zF>e>#hCU|llNj-7@m+3Yp6H!%XIPFx`E$``c)76qp*H;wduUfeTTYC{9FeSZlAZ}o zm&0?-0*-f&A5i7Gav7>BEzd>2>YYxGPOOn}j5#@B9QiW35G(okbTTv7XHp~9rCOY~ z*??^|icavsnYiZh*1Wo|KR#e{>KO|eQcP9&f%#c)z2&sH3*kkrcPN=pRBoSMIEOY< zv#Z)qo)S)(J-vFzMXE~wL4ml~3;|}fGb29Yk>^r&4pjr`Ehh^HHfAoQe_|h+KwfA0 zGhG;as<{%-B2(IP42OS0J6{Dm7`gNtADew#BR<>Px31t|d8Og2d$6H(0=q#Dg=6(@%l%KUhm}=2^B?%%d6$XQNiYgQ^Ix+gQkjZlCMTY$-uRb zu(kO?Y-xPaRqRa@-=EHMkv9zttuxZE0@&|1FF9VY_cntt^zM7vd&D+Zy_e$|fO0eC zxpv=>t&Nfr>D*SzHa}0{oWB2E<1DVS`D3Roh`q#X?>Ci**+OJ@IZ;q)a_oJVWo3rg zyOJ9$t}DHXwJ1mdR%xR9wWEPCtiH0vQ7S`uIx~hlwBIRG62$fvS?cO|)qxk2AKGqAK~7wr9MH*_m6AZMBTx?_fy?_ zZHp`p_1ygJ!?Z_v=j^2ADmb#aju189*RvWG*FC*L#lIJm`A#LTNnD+< zvw4YF-k|aHDp_*)prk@x^C4dOQFwAm!u8Sy^;ai?f_BDY<`%6n;N_XV!9-8UX#fsB zDRAO8k?gV&IQh0G=}mTUVgs&TkH8b3@%Bj`{3xM%t}4-Us}VEVtFjuEaKA;$P=nF! zc_E7DA0!fE2Vz#kmz}JP*IE*|%6gefQk8zxwonN)nih!6!$W8B-49}yw)$H{*LQv+ zn0vmim#Jh2HP*L5C>LWJ%Mwcvu)a@*OmVABaDJC9;KsRFQGvwP^7z$^$I68JSox#T z>t`_wtE8bugK3iNyn29VTOfJ#{aHMu1x_eG4vCN3*O zpuyPgTefraQC_#?vg4lTPlVtqtP3DkYLcLJ21=ItAC`5+L2b#c-6NOVM@0;Fz;2l^$NNS4Ht{cuXxQhgKL~RLr(-7b^P6zBJtL^@`R<|ks3a$2NyPyZ z|1P%OS+-h5N;;3Pff~8k_EYhm`}rmslx(aWX}#n_S($fJSdHNgBr9FL1@V%0?`2ah z*YseqmOZec3kEMQv9b~osS-!PJu8j8cIa~{=?K?1IzGwmB;g5bzCx{;t=sdSeV@!m zj(*)0hM3IVN_b$ss&W>9KNJja!uY>-P7!mxsS**Q%=nIq#6505!ge9^aU;vvVJj`_ z%Z2EO3n0;_01UrzBu9P)RAV43HRSq%m=t9L(4Uc!aomX_ArtG&Y*|^s;YyYTbJ83- z%G1{(_!xa-A#ttbzWoz>wT`oO6|(WH;>ql?tSTSFIEDDO*v+xcO&)N^7eGK*5A)AY zz=u{f_PB(KX8~|cZf{OU!2l}c-yR5lupI7X(l?*kER%Kh&AiVC+I*CV+Ev`N-fzK| zFdaZBx-ga(<9)Q#7@+>v%3g*`s_18Zt<2$Ank&9!m07@b;HLg+Z@v809!9dq)JN@W zwex4(FlHCoGGW~QSfLmUR^VJKDkIi0;nYd7uE0Opr{gW zLl+cDrsA)}Uz?bwwkHZo-^-lz0V@68?|udg_j;nBsP1wKx}?@_+nMS@egYg#z+;%p zMoVUMiHQ)K)q2Tr&lOX?o!;;Il>CjbO>YDxe^+?N#+%iCjZ7d*jaLqmtT=PvzWN^&@znM#=1wr<7ceZR~&SWj8P5SQ7c`)x0LU&R53rpWcDx#bt-c z!pMf87n5)V7xtP*W(m(q994zWR;8X#dg&XL2aggDhx6W|``m-P3|+dUec7u!Q6Qrqj?fyP}W zx_i6pcl0dS8rgY|R+~zBs`*e}< zEMoKILs_zlnOW9UckGqM@nnc}L$@Kuk5yjdQRNt1%>U3SmE3kR?`?X7HWv?z)-$Nv zG+lj7U)OOVv@$!4b=50g5|m~gR$TYopFUopkV+(Wf*N`f_taPSl}orSE%oV%_=`tH zBJ=b59&>M$m^Mp9-E%c=c%2;oJpO{Hc^uT-0>UR>B>}^MlT0( zas0gG=!unlWu;?jHoyLaA3g@@Ga5$Cd$S5v9r-yldm1#f81ex{dA>GPgq?ozkZWZgXj9K`cA)zo07*uUV8b#o4)%7F73Y-DsfK& zR#3UTE9hkA#Gh)9@wdM(s#nM=ODcP>#cv;{v;-0X*Dnyb=y&@?vIv?*EBLZVVGKG= zt`}^huhi)GZ6XakDmj0u@_x5hkzNh7CY@UJ=@7$x-d_bp5z0d$@-thaGZ*QO4xP#a z_#o4Z-;^B)*5Nch+Xm)|RH5i!kA27(UF7DG7G;w66I!yl`FX1>7S_xj^z-I04~)o! z$VGIP3xXIxFZj#1;~=9luZZ<+|p3eC=87cE`J8Hq~=Q7d!EmN>6sGwSmqduF?5h zrvh9`=J@s7Qz!%LlbyT+l3T@^$%0Ye;DdUTnS9pK2MgFT%|aBZ`^Hg=&nvQ{SK5(& zmf+iNHi(yx2@bF*eYJNv*{A6vMLx7C$AtV5c_kx7gf7rJHJ@zQGmcM5xOneWChp?% zA#u}~lT1t0ZQVY-(%1?9wA@#_Upt1XexS-E*mqwy6@HTC{@hyLrd8@kvm&^FeQ?~H zz+s`!m2EyY8+ZW}di~^t)@%{?ngeK4V#?n~Ug_@ZtLn%L7fE>rK#OPDiVBJn`hnEy zlxrmo%N?f{BG84?%3O2;ke?~(Bv}_;s+-Q*s@EUznf;2p@(I02c%YdSnI?uhMdT{%-kolo~qRr`G6@3OTX&|8ePl*yha z{HiK=7R~4$``PzMzka)xOTk#meij~YwY{U1pcw3J`Y zl@|+4i0mF>4@e#wvH<$-kYsit^~s9gaTmP1!KOCp5X}td8xh)ULSH$APOwMuM7o~k zMvxz@Z;>Xo;i7at!d8tF#>^4@CNPidZZ{DT`-?BPhCu)AD$cz_l@UT=Q;)dnxs z=!pV2TO;)T_Vy+CdNGstNuV%p>LT89O=qyBbQF?qvT-to%LBI{rJF2Zii$N{LaGY# z7yn5vy|i`vvpR70vqD-UHp%zWBdy{^E%EUK^gLC2G*6?B=I==TGL>*@z3VciLhB== zG$)aE{H8sUYPr8wCJx_evU1a>%=ZArdFH)pVLXecGQf>qsjzpfUFV%9SJpCpKbCB; zkooFBW7}Kcy!6J=03ELgy*N-dFB7rqC?~RuHUW%*Ce;0>A(y`pF*%=MS}D~$$L0N@ z(}vCtQ70nBuK=do$4%0zf$CsXk^RQb?hR8PDmY(8(HIu%m+()*!pGxi2tMy`3{D zVUhjJ$Qqnu{x3a0W)UPNw^LP{F)=)*Fw)dDyxaGLR&|h&@1yfH!*wQfcL<$r@pxgB zEP67QjtiWbJ$SIS?%yk9JK^|+5Vf8D(+hVQ6F^Q#!|!i!%4klNVBgv= zJ0yHGba*+hl$S7aJM4sUN7sUNB`yh0q&S+V4j;FyfjE=!`1TLikA|D8-=$X5Z@efheD6Lw@=qw_Ewm`!CoG`ihxy?zS*L;+xmge?&1*Qt6x~$`LM`LM6PTL4LIK9 zeUSHvf7QgRX!d9*GY98qyaKQeJ<;});p!Z3!ZfJO40RY^>?A8WCtojXRxhmw(Er+u z%5Q$7drEgkl#ssI`za@zCUDfr+2(jwqh*J2|fk)W4XI8mL zGAW-B5!*4Xfs?s@?G{LBsl~AF?r*_ZLhybP83do5{EORNUgdjULaHd%KmyZixLm5@ zpJcL$c#$Mv!k<>_kf`Dr5hZzJqTR48>GaCyyJttQgHGz4d~;?TjE!a$3qBtdWm}%C zncdGX2%BwX_=m)_D}R4V>VKLNiX01d-K9}!lr~f~G~P!zKESS8SNHp^GQZiOcR31_ zw<3JCs&qQNlAc;oG!=aFtK$JQV0@my()WtcmZsf1F}(WdK!NmXnZ7pM22p%ai#C(>a`V@lx2CMVRj;Ye3?Me!Tq2gx&0(z|#xRuxt7w za-L}I9aYo|oS}(k}NkEt0Nc)uvGiyj~C-`^x&c|1^{AU7E3Zu{m3z5a33-P|IqZNl`fOySlNsHAc=o z%Px!$S=4WQS1c0~z8aw!`72a7ul08dE3W``&ons4=SXL-qd1tVkZZdQ#oFgQj#&r} ztM+BVE|x8apjSQ8PBqEFAyekaVKIiCI=vR1Wxw>Gdnux@uj4PQx+Z3`TRh>cBi{Nu z5UU%{-YkL^a8`R0qjDu=XODM$>L`JH8^YhJ95E&K`N;*SD)~@*U`g=Z??gfThtj^s z`Jy?=sDr>;c+nv$v&yO0rOt9b=Hr#S0vR-;RyWmt##D)u;rldg$0 z@XT9^`5o-7mDK70S6un)$;K&T#6*g9+rTLR(s*)}O{bDS-hjU387f$Ca+!M45`uRS z;}2A)O8ogM`?iDTt1662x>U{Hf#p_}J%hL%OEB{t|9dN|kfUTb%*ZgX2x-aP0Z3Dc ze4})`2Se=ccD$RaBrW70PhJQ|x!cjb=~9(2ePYr^!3PiIWkj^rA}b?akabaT^OA1} zVqM(RBQ%3Q-o}moybtttEak4C4yrbc5#wwmz%;j0ha|P?!yHl~tp{ouN=Ey{CPOj3 zr4~p%zY#^#r)hHE0KY;YfhT7l`BBkSe-37d@E&wIb=w{K{P_-^+qDgNT6E}UW#`PR zz8ef;BIPb^K{wJYt*mOM>fU7QP2kk`Q!~6Vn3|gm9F$mt4}&yCp8-|EXw#3jG11Z0 zFZJp&#~ZXw6tch>6%k#E>)F@fn)uyCInTmaRh|7%aMIqs&f|K+Pd zQ~c~=8dERf-O;4KTaG~lx>Qv}G(6rhc&}NLCw^rDd3|{i|&|Xm9|$@mS#86#Dhr z$;nO)H+8BMJ_W#z%3MA?}YM#lKA(&yZiz66aFs#8}(-Hyj-e zr~lFU3i*a=^*p_HFgjLK(*fBUFKYBBG3BCig#DqZ=YDGDN?&G(HgT8I?gar&&qm(+ z)6<<1W=v96TWS0Q4GG13i$_K?)lONc*AB4&JG@e14N0WwPwh3yOl_T~bl;PSReft` z%q1`9KXSPR-~D5#TWh}gWT=qn#W&e60_S8}xQTlPJ=E^jP}Et>7NVv{o0yc>KEGkI zC_@(1CW)EEZcgi1$Gl_+6Aw%Pq{4B-{1(*Vx%~4M5*^#R9VZCTtJp`CqGlD-hkVfC z9F=%d##6IZh4=>sK=n$?aWr-s%uQpAsy2qyf1U#N(TAp}&!Z)+<&K)^Et;9`8)+Va z>_Iq%h4V6H00j?7zo6YGsNHlW`9kz@vYUh>_q1Zli!zWI|^S2p<&~PE1T9h=+aFz@1Bg`3&P1FgZZeKU)66Z z+llImC_4wsvyCJEdW9j~rNb)Gn29#YtQ9F`4H2L}0Eo~sJ7}N4Y7ve5> z3;SoK|+cn3jyM$;Tw)WKvc*_#m0P zP=`5~`oTn0(rJTdlHk10vyI6?b#yShH_=8xYYrDA!6{uTuDu}J#GlD$l0?LDL>Zvg zf0y3)FoF8eY8IDvAFO0WN-Qe*tEyqU^${CzSTX?PNpDpSCy+YtlJi5&R4%HUw;NNj z#SoRF%7!^hKRfxN~U`>7`>vIJ=sUZ=W3Fy2viyyJwo;lm;u=PuR+j$7sgf9bjfCA9-Z$O}CUanr6XUH{GIw2`qH`#=k?{>7Bu!8 zsuJD8vc##DB<*fCH|Or1JlOI{Pzi_OiszbT>GyEHLyQmk-nNpkgztON=EorM8+qp} zjbe92&!jJGaNaGLwJgc)A*wRXYW=7G(-G#8+escqI3ztVSdNEp#^kZ~GSV zjfziSc4Ssb9n}$bXR~V+N|$bc5?RRw40IkAU;1&~uOQ)GVa}Cnw-VXkh3h|PxbjYv zlI$*fvTE|S-vZ__-o2m$dVu@m{3F)^*l)Gl8bAzMU9(yIVMxXHskJ<(_2g)e3Id#w zTLon+aK7d+c4Bjm%he4#l2+rx*-FuVo`7htaGi#)d(tvA*}k$KliU9w4!0SkHYvh^LcT^wInxlB=-2OI2bd@(n0&6f`B+r2PzPIPuJ#N9( z>vw4&#^B6P<7F)$WZ!#eIJh^#Ge23y+yMcbW4LP;zB@4fR59AO$t0OIL}xxtpo`f{ zIp_0c8n|D6+dU&wxfK{+9V}>r)~)Q?RV^s zEP1MUcTiyK8Y1?29$}&;aP@e4hv}+Fn^vU zOAzl!bpFI@E@g-1o=lQy2NnTqu(wj`4j2 zhP-wB#&@_}HuAL|SAPTJVHl0dD-z^;@nY|lx+PLcF!>1H7WZHh;#c~JFl!&nu2{$b zaKz&vjur&1c-!X8i+*EYtWRdz=x;i2i0|*jnGq(+-nS-`J#})EJ%<*#6KYmK60tuY z7D*N4Ap0O2Nqw!vpo-I?J%r+5XB^&M~XZA|@{K9Uu zK-^qu7(@249hLrBhTqaS5ATTomSrgnW?P_peb!~389w0a4{AGWt&O1POebSvUBbF! zq8zO<2N-kQ!(Z(#u&r{~an%;fgAl4P-H^XA`(~w<> z(c5%cU)Z@5iT`BedL#iTU^`p2^HN_OgBhz^&e?)tjW?vwZ?pyT;N`BCaFN^GBS+KU zoTF*JDVa`VS?A9Ta-Lu|Y@>DmB<^pCRg zz9&3W6o3a45DD<{poxqYqgkj4q2P*MGQLeJrBWv8*C|2&LxsU~DX!Ma>g-xL;3^4q z_Z5OZ)6&YprYY?Y5EX?3(C(fAixL$rqM63lFG@MwmABsI8^Q|Ydc5)UgJdsfRfoWJ zXpnvW`x4%rDfkRAt%ZG~HShXxO?)X&MJw2R$|@j=Cquh8v;EUIHp$Y)#YeVNUz){5 z6ZuIYWsoR6l>g(;cZ2uQ99Yz9rws?eEPQ7{s;j+)yVC%lZgtcvD(p!xKU%9g>%qyC zhQHwm{?lx1d}+ORN{1|ykHo7N1q=?bvAvi5pdcv%d|JRip9KJl$W$W_>jUTvbC8pI zgegcq|5NXXjLm~n#+?Z{g}$9QqNEEzL;E$6cLp|2EqL&@V%nFm7utTigg<#Z1UOx` zc>oCqpmpF9zln}*V`m+=i{_T256Gd|`c9^Nrdiy8M@rF*kPL@#!HQ^IKJC;k`5 z(IBsX%vEP*juysuMooqi!GG>!P4pw)%4{t}Yp=No^^n4;BQY}3P7XF*URhD-9hO`~OHM9qG3T28srS0IXabRfmsR6njWNXG`(9}0D=Y%O z(xHt!vZixeIf}9B()DJ@@6(%B_O{P0?Ey!Tc!R}`aJmLR)N#?mKUVV$pZjh< znrm4+AR2IsZxU@SQl4Wes3dE-ao7*qz}rl(RSKNv8Qqst&PCKI72xej5qG13ciQ|u zs=d3m=-k1G`c{Da?MoJxD5A{CV_I`mR;C=+x)DMh4s2#qi(_MHlZzK|*zQ1AILD_} zZUI^@_097F`zI(1BtQobTJ&%N<|u>qng7uq0E3ZhA@~ZTZ$Z=PcWZ5Z{Q`T_pL=;R zF0@miUmov5IhhD0?wPNjcjU!Y9`ki(;5T#cYj}=#`TFjmqn8(}u7CsPHoD*g{v&?l zE-k@F(R+UkWM6cN-?5pkGuypujj!1#d`kyZc^BXx@6+fPX+wZU3%Kay$L)~WvN5`!nklK$NZg*nRnP~327 z5w7{F$2xN`5sBYo0}OwJQDk>St=k-_;aKBW9hG_#=cWx%Jt|Aa$KImOq5@LuG$%m6 zzWz+J1;6vNpv%*FMBkyB_zzG7Ci@PxU^M&0q8kiS_9phs=~n^%Gacx&Dylxr0zSAE zlBusqfO`NG+U0(pFX^YisoRaavJ(gLC_8({s&euAROYV`-Ka$N5Sk)mR4J~@9DCdc zvIKS4HT`3zyQuAmHG3I-XRp|oMm`9e-J|~anwb>tM3qU&U_b=;Mb(Ydcrs$Kn zQD8($Wa3)OD>b;R9LxXjQMpx~Im#yRSYDvzb-_=QYPE@yc@{iw?Y8LAFaHmKrz#XZ z3_F$_F-PoXDv1-7Rfx45uTiH+NDboQ?S7e|wbMuUlhfI<1*E|=8)KR>#l~g#Jt*wE zTCA+Cdw4aGY<;@L6R{lg%F(6iIe=If3lE-1c z+JMM8CUFt7bmm0DBM!0vyblMHPN^*x^Oh%8KA2uWzFg$EyczALmyG&ZR_xL=wV7gs zn8<+FN<)gD1Y7pPID^1|&++@mpz)6L#$5B)uZvv0GWkvE!emKyJ9(sh&u&QPCPY#@8}4*I`plS!mf1XoZ+?iGXVn%S0 zgH?bBk9t{s&YoiROY$XtM{@$vffs;Xp11}OYN95pOhnLBo-f1^)#nqR0mKnm^1;;a_C%RSiYCFy0!(>^fC;ozS;*1wfXiYF72>k0r%vyMdo{ zdfBJQ3+$jDBH`7cNiT+f9%C6S4`^L;eH+CZYO$&1*w?)XzhhhY&m&>Y8Q8Rrq=$_j za^?8cvhioSxCsdfiPbz4UfIdXsUG*Vk!5@_@_^JTirOi)^ufL(x9jIiHJV=vgp8Rt z3TAMok7BvVkqytzOfrHL87a9>Y{`6QG71mc?>@kx19Y*Otn$tJnxsr3Dm#zn5d6E^uUOyn9n2o^NEgcx-2IdP2c!Q z3xDeB7M~~%Fp9~&B6z~~(Q{zkvLg+T&8fR&NS`#zSFbu9?0QC4TnHR`UZK%QJcbvX zToTiNOgs>bY1GkR*exo+sr&8H*Q2+tTArPHhiF{lW`6g`7;rRK+freUEv_=%_4nEs zF2FSe6w;K1OJz+4wluPe$iCuqEjE$66e$3rsn+cZ@0d>2i)5JAOMiHdtUK z0s(l>>pWF%Rw2VGBV_9`SV3fypAUQ^NnMhz~|xOGY?+>%Uo{EzwN9p&?K?qXdq&pDq=S#M5&>x zoAhY6)wt3frAzO+RI)o8AN_Xq3ZrLr1l;k1c&f0Jl#~nHmu_JwfGk0m->}>8%#TZz za!I=0>}O&u$}2$D2wKjU_;vf2R{W*xGth0^D~TGGC&tJ zP4&Buzr848P2V8oynlffY-E3;u4bxVOp_@Xs+HSA(-LlvfEa@mOdiZ#BW1X(1I#qd z0xFU7I8Nao6LDT^$bIqNdh45sjZW~2u}f|nQxN3oWKksss+1tPMUoud^h7bTt{rS% zXK^rtd5_#h8u}+ft}=4S2Y+UtJ6NZb@I9@Qs(G-Civ?_;Hp5?29_@5LmH3@AN~LO^ z^Rg*6c=ZP83DoGws}#mBc^S-oP>}ztB*DJC6?R#m?hYYtT7o4La{y-m4}8DdHvI z4@Q%1)1f$#9=tux zCpNbW%(T!jL-0u}`#B>oqbGf81`=Ko>uE#so<*8XutP(nl+ zMFa`yk`j0dM+F3F1e7l6j-f+Zx;vB<7+Qvg{~EZx?{m&`{?F^*i{}j= z7-sLi_O-6~Ub(!US+*eRC}?Y}g{G&rUQ=KZc`@77Azlxk9%~&h7*Zd1Rw0`tE^Jj?rW6RX+!6P@8S>~Z?l|@S*XL3eON8Og zwb?YFY(rTl@qK&NR}}8l-D3aE;3H>PuWG2RZXG?<#!9u#yAYvC5S{>0m*b>^MLx2q zi6X-rU-pk4*conn>|1|%5q?p7ka9k~#kjvt98>oWWa~MhlvVN>ELCzBS!MJekv0x$ ziOe(QPjmVS&)^sJ?o~OhZ1$_$+4x1Eiq}VD0c(hhY)<0-2`Z&QpKmTLsQHuThW*(8 z-g2K6xIM2n#SP2{n`8_)t;gB4=cZywfUyoULMatY%2yWt{V`7>-28BK$6@@@t=5fC zMMs5pBtB;8ouIs?Zp#KXDsG>EeKKp=J~S8@*&H5+VyDDBD||?ROUE3>a^%r?c6u5c za8G70-I|`d>i)^Y>O$EGGm333Rs$;;&{`-aq2c`DBu|$QGPwGPxFefJed3c+nk@+{ zzDXXWWvv*8pVMx(n~~gEnB4s$&7%5do1)&tX!FyQMK3ABTEHI^OxnK-;aDQUIBqGw zF{A;?5#9%Vgjs@CS}&62FwOm|ySdnJ=TL%{;X*!;rm1%u(niB4`sP?t z+;$(^X#WuZ!K7RV4O{mt4W|ZvbNcvQN!jb`pMOo77N)gwuX@;BQ-tGS1K`iPn*i;MU&Gbd12H@I3ueVG;M#k;O-BD-w=^f$u76rfkW&_Ay^ z4w$36S!&l$w42ZUiUej~`CwHx$bblrKN&nYR;O-R+wPlPob=i_R6;bT2#0OF&)o;l zc#*J~Y6YkpdZ@iyz10sqH`>y#5BsPy z_^2Ou>#Stc`yQmBww*C9Be&};HOU%*2>me>0LC1N5 z&V74!%1D)vHxUQ{3Q!}LIFn1VQIIn%E{^JBz21Euh7-0I2VT|U1;t_wx9&XE0WGIf zOok5A12<|95TSJy#xQ1$!*ZUc!O=kjk!0YbWvu1(dr2953jYb-5{?4;gyAU8v7*TZ z80P*SumUJZHGS=SSc2;3QdX#0n$gd0`Y9O+r3^GAJndzJL?cj^2_B39z!XVcWHqzDgGUn3@@fp#7$aI&+cp2P zob=WHVV`zWlhD^sUf9UcBp2va;>oO-L5@tNr|j3{Cr$J^H{Cjskt%tT(s3{r5*iTs z5LqF*M$4Pl=f|n4w*4$%eErPx@-L)nXqycU- z{;SIps>{NLI1YD2h^e~?McOEC{@~t3U_3cYU-&AsGB_lsWz^I=;eXF#t3f{-hf#y9 zHI03C2XxjZ9ay6uqP+k`p3l3+jzV{kVXpBJQ217!+kjFQD$GnwcaKovF|uv^_?IS2 zKp2S3W#Mi7#(9jP)2^R)0>T?s?fRfqy(Xgsr-S=otWd!4@(E72j|xNUnQ^qCHyi#02bz_IA|Cw0AAlP}NqyZ0QM ze=*7=RLH+ZtWQnxzp^r+j%+N71Da0>NB*x8&HL2$@W$(o@tgi+P9@`gHWf=NE9)P= zlY&&w?5G7rdsTDz7yuR@HgJRxCfV8P(`fBvUg?mnGI_JkH_bQ>{#1pqo6)j3NIw#( z)nsvjmRGLFYk>-H5Ycg=GzSy(ORyC69*IVvw&62g$ITxSa;p=b7AzRGKQP)OtX)9)(gA1hCl~#(p zY4Kh^e-5qP-nDXE`$8-Y!i^FkH9QD`pW(MQBRt97L?1spG+#UkNl0Q}cUtv<8{FfX zAAF*$@}S?$r)ncHK+o(ufTb;b($@0MGUaagUMvMy9UKPg=+Dj$sjIf*=cJQFLcT0w1VW9WJ^_CZv!!Hiv(@Zdi$R>%qB9@oI*3HN#02*|fEZ zgaBEq`+QjG+f2I4s18Mu3>ao$8B;Vwhs?1=5>3EvireSk+;}r(xL2et5$8i7;%+*W zYmV2rhM3!uM;YI7Gp(thEIvx=nR8`}-t7!%$=I8+RozN8Kv^gol9SmQNb{Go=!_*n za2beDjDV=9J&e<0TBEo2$CK>B1nh)^O*;2IBFT&9JJC29wad)aWE)=)bA_4R$L2Zl zJg3rC>-AE*AN2%GS4#*>+`EXraQ?h=&tpx>)HcrTfVo7!fyyN?Aat?4e4TQo)Fq`~ zK&#@@(9H|!wz;Q#%LJ;fA~~(3?vj7in<_M;!tLJUHq=P6G%TVN55%l{Yi-G*-D5ub zf#Wreoz{vl*FHPmH<_ADl5o9>8J&uJIhvc0k~h1T_wmVHZPt9f+__-ODgU)vMsY*;^&l;dH_Ep=^APGpf55RzvAIqo<=1?G^S z*&vNIF9M8>;F~n-jXz3o37c5n`VguLTPSCT8ur=tEhRF4c`x2CyQRCD^r92*vJ?8g z;OnOtw@+|BU_%Lri!N69w`RUiMS6W|3nq=4y0dra?< z^EbexzpWz#8q$7#M)C4Ih#=NK^X&GmC1%xp43YaxO~q12_lj)9opNM8UygM0-^Vgq zegt3zbvT-Ko)<*A#4*fQ++#PVxU%f+^Zg>|zs?TRf34Q)AEmbg;woC^h@9;*s=I|p zFwn{J{AU4VY1arN5#mx~CIvFd+#K6rz|PS+%y{jPryN!;ajGooODmrkUj?SAI44Y6 z#a*%{dJA6VVyct#^DUgEAJ^Y4LYjN`#7#M^i{h9xVLG$!7sl@U&%%2~^%t+NMQy=#1CJL8oBi{%!x7R*Wk>1k^qUPkbojz;_!-yEm zBL!y)uw1*>BDsmnZL?Sw^SkE@c%R z?)7?BMNg7w5eW3Bn)(36xD%tR)TpoSLd$mAw;al>mgh}OoI86UUizwMIf`kXOv->0 zIV(!S03?yVO->?xgg3wk2H=$vmf4An+y*i_TuJC3hU0qv=4imGNgYaZH){rrx=AFY zg8s~(35$wysd4T0Bm=W0e#$eD$RqQ-faA-}T&8Tdz$y~3qBq>cjEk%1XUJLAh~pY; z^sT3UANJ5-+_p>q2%w*hsU_gXC!i$kT3Y&Q@xs`Dsa0d z6Wh-dsI#JQ5*j)M)QpjCG}2$2qyd`P)l0zrKB{H>o5u{bTylI~P&uc>aO>(;o2 z=4q9$DQtWens`V#kq&Jbn%fLjzw`JygA$Be$1%NDW1+RAwAgaSZk1CBr!t%cWSNDa z#g0pz&S!jXR8R5*06{=~Pf^(cpVVkQONosk7pElt2&KZdvIB>qg(maH0Q#l2T*7>B zqF>n5lk}14X`SPG227XrZiHTqAUfq$at9OXho?v<06$zhIUbZe?Uk2s4L4%0TKCPhd(8K8acb>2t%rPcL7j)sB}{uU0R&qK)Z7W^OKkKs~Z!GM|dg znmP!~ni=DaQ7O?tHqT(9y%_Aw^%|n3gq4o4Kshb6Y94;bME38j1vTFHQaYJ&E8Pqa z-5UFnS(uM=Kbq+HXSn#WTdvhQ)vgYb0dTQbg?tiC7%XgbXH<7{uZQ!x|5Zjd2K7h(bD7SN9yI2@q zC|*lAYnG1+{ls4k%4>fQC)+jUDig=Ql*%ZkyL>NfhZ$`C%wd8O;L&Z8y-az70U8CB zWQ6$#sFKbPz&JLsTARhj<(@bW9WWdBPM&VfXxvvqS0ig&b9tY*SSXCteYvZHW>0a?pk+{2yP-D#D$VZ4aJqZ{Ux`Y*QfkdFCCGwT9 zZ~GW^Oj1A%G5<#j0BN4GAOCQ{%+Ts@0n>7`X@1BUxG76lm6M*5ZGZ{ENsY)kn1{Xm zTk#wY^DU1wH8pQp(w@JIZPdu9aas1f#**?Apg9bm2^zHN@`491@eaGx{4b6Zn6-5a z1bx7msD=kYJx%>yzJe#z!FCsZ*ra!C2U1&`MDaT-rW`0t^IxmoNyv9fy%jh}5 zDjm15iiec@MPJYK&sR@4zEBUXKu(|aZmdjiExn4^n|^z#oxzXAple$#9bZJmL~uJa zp_DY#_p=rRPSLYFAUyB&1j*`eG~IsAwLkW$VdsyDh+MWDB>aQh!FpM%z!7eKE~hVBKhs2NL^I zZI71zNTFbRjz)mnN%b1oHUgY*<)=CM(uapm&Vla_pzC@?3z6Hz2jUC)0!#^im?u`)?Tb+CVp99?j5j^3nP;= zK#gS760|L8jV?Dml=$#xum2r_U1|hH)Vy~><2w&G3jAX6NL*y`L}f5hBT->93K5M~ zRWxjtQ)=3@WmAfPM?VxD9p^>p1Uv{G9aWfW!U!_ZX68+?dB0^6;2aD>$X+R~yU_pT zQKJq8+x$mqg3e#6YaYCS=)B+R_yCC!DN7dyO~Ev1meK~}PtFy~O>fu) z$2JymkMC;r${2`-OgYEd4ZX7&`fUj@5Pi({)@EdP+AfIQ#bQxB^H}>QwHMD1}M6W3ovNels6yzCK>S2A7V&l2BCF!bt z2GfkoV?M9+YBP@;bL}Y{gmvsljw!sLg6Ay7q3WJ_E&gvz#DEsa6#m0lqupz}{5d0dUOAloKr$zG zTmuCD!-RH}y*VqE`j-!r3s6WcTHg_x(ObKDwwhsr5ig6;ZxU(*2=WOc3Fqj%l@ve> z?*xro4ZlLwS)p6;b>X4X^MAR)9}|tKmRyXO=6lLWX;kx;(6Al1Ll!D9VlAWbiT)Mc zV|q1u__ViVamHH!R3_+)vSasCntIjw?PwX6ecOaaS9+08{!e-;grT%fu9MwxNc79c z2Ypy~>ChKHG-g|_NVnDn%0an#2DcRhj{>s;GmuxC#577-_TsfN2HqfGCiGs-h8_@h z5l$WAAN^ey^ADK7)HLwfcqSF};+(j#ekHci!m?<_%P{oqM^)~j+cniE#~N5{WYJ)0bl;eRzTt>ts&bQ@EuEqmGb?}SYuL1J@A>rk3m@)>%t_MA+u17vv840O2C|xDU zTgW#?;Bygr&ILy?GYB0DUD2uvl;KhFgp%g}`#BSXkG8e5B3yY!I9?VvGm%4x|9LWnKPpT~Tb<6W!21pvXKbLbq=CNIdG_H$+^erSY9>?1 zkjhW}$Jq-?3`M1|b6+DP`NT3fv>?imY>hT^9TPyER#iy~c{@EvFLq z#Ld!7t|QV+udH7d(*&8>65&QY#Ede#l41@?s^lWt53QiTLG3?`u~-4xzfJ|~;kt&H z&r)X+3(b{3*YxR8g>j%9WPcqZMEv6M{CVIRI73KqweZ~+Z`!4nS`AHT|MhjRLDMne z8TtF_@fjLn!_1|iBTEw2h%=d7SJAc^zm``7nYgWRn(@Pn`$h6U5C8kO>40wy*HL#R zlB--zBj%mN9M0iHo^OSpqjUD_Jo+ za$l4C(IkxsjvKBagve3l2IaI2%gy8>8!fCO=qLhIq(Uss)SfNRmz)U%>>MxM5K%tGMm21c^&S@Lhe<*isP5SvD#!m<{hyx(y z396#}TPxx+0F17`yr9AZ7=mR%HdNX_oO`b#?YQN%RoUff4w zY?B6IWv=3SA^EQI;3eD|OLeZTTsH-FUZaO;KvfApMq^UkF?@?nQn&<@vDYL5fy>_* z))_f@dqa4{?S6vVze0eTz`nBAfbqv2CFD z_=+dPn4s zx?U+<;X9ba2c173Oq<4n;c4j9x<9OKicRD5^)lQMANR-{qKk9y``=oglHtl^tfmvE zgELV6`!?RC?-#?bW*Ai+M^l?)sI=1#Z{d%tGPPw-$k6D?agkwMIS;q|KAr2t3azCv!~hy1^kDZ;KFQ? zHaf(>j=0B#SAxVOVPq(XO1ytxz2f$jzhAZ5{*Jc4tA+7@_SGZEay=oK6*iQrr=^O?Tobfw8m2|uGjJN zc9=D)9MMH?+C6rr9M2d9`r;1Ej z6i~Z1?|&&MIuc?yN5`HzOb8SY6qUAO{vF>m%BcRj@bmrqFpE%nI1b9cQzwiC8ao?c z=#$%mHTxyJfJ#i@9~$2ymUO{u_TI70s8lRgFzj=&$u2U2Xum|?3eoC(<*v_pc5i%c939!cF%f6E7!d~+jUp4xtr#LlE_fd~?Djp{q?I|2uRt5URx$r0Jym#m z5p|n);26O`;Ebw6L(^vZ_SW2?YsCN&x~!*2QXhkyI_lOjde8DdZ-G>( z9UjnS%aj=lL1l-?)C45_0W5bDX@Z3_Fp;_o8>Kxr@H_;~V&zF15PM@YM|PAT?qD;nt9z@QTo}e0 zv7SHFvyuCap>J7IXEGf@j?|PK*JTU{h}HE-IXr%wt1BAH5B#x)!qsT-wVhdty3q>n3~ zef+C%9Z3>U9*Fjq7>|IE(82II^O>ba{Gu4Pa5wVZlt4tAO0~oqd6ZL!AaqcR^SeIx zsPU~0*eZRXtYvYJP&xc&xrU%R5m}y}mcm^+jqAEQRnq)hXL{~SQR>_ep1BoWaHJc^ zjys?_s{RO|3s#(iOp)UyfkF1FAPmm4S_9s6N-cP74 zxl8uqq_zuP|e%@1~9HH~WcUz0qWc(4tE>6bG|MSj=YMy4H z5y$j}2N4OtfW1e33o$CQo`|>;`F>af9hwKKzrxG?{rny&lnQUl1%YU-BdvIk$u}BF zJyb3du+Q3RANjeRyA04-XPTuFtEABE0zou`l>C9%1{6s6!n`z;u>J@!u?L5sUASsnX z9i+Qkk1uj`8wl$AOa|15M|W%A<$(YHJ`$CnamStU%|51h^>&x7=kErd~@EI*bO5}&BpQt!UdAo+Pg9`r& zMabQjd37pNoI8)vni&`n7!ue+a6&Jq)~z0-nx}&F{X$XSs(bC?4Lsv3f=j1>p&WF> zRDIqnAB7KFXKV4BkG_A5bFp{Vgw&s6nF)c&ECt%UE|+!~1)TFztn*Qc%vS{i&jWJ_ zA@r0VX;^JlrZXs@xY^B5QQRpE)V&zOe{M*DBLFJ#sc=H*YZUqc=VKmkm+n=u+CDn)wlAk*#ighe$hdh!<7)(-%`%Qfo3H|7G6g$k3@!lkzQ=66wi zS(2T2<6HW(dp_1-ospxSBsrmctD)LYWQZW)g`irnUDARRuwZI7+dp>=EAsXSE zs7;{-boBHo&D1w;y>AQF*o|E&GjDmZih19OLq^0 z38VtLyqH24#lu7I|2ks2Te}v?&%S#kS!#zy3DmLh9nP9BNq-6d?uYi%XWei@B)8Zj*=aMZ zwE1t6Iidj_$7$aG=1%0|ZV4A46tFKd^TYnacxZqL13p`0BHxb!*?w-)lal=C?<;LF zk>sj!q>&1meil@A|GGZK=N5D!?YX4yNG)F{d|11L8>;D(XHIds?zbp37%c5Zf364I zD@xd}np9yS&ghvp463$?#C1OaJR8P{@(pXI)A8#-83+S2p#4sG9qKnZ0V!c&`s|N| zywx?7BMPue@({|CtBnWuGEqp@Z}5(8L2|Y?I^b-es>$0w)zMKvjY9=oHLh>4)C68i z3g!ZzANa1O(DF)=Y+IKO5R&i?wC9ZV=x{_5Y3s>#&qq;fTXga}Ya zL^y%kEQ=;?eTY7QUAmc?y=0-k9|`V|esMo!2P1H1gI{yW28bET2~XoDvXbG5P9Z*& zzg6%b43sV}tnR6HSWZYx>{<8t{Ato(YYtO`IyP|EkT#S5m1%A11779T zk~>hY>u%hWGx`)272eYE*7yZJXgGWG5C>ArC&;7lq9m`Z>^Hl>L)%xn6|EkC;k>Wc zM-JJ^1z$cQ8|1meJ&dnTAVE@KS4)^IaVe!;tui74ui&_QJ*}sg@Qtt!4IAS2#>%Z@ z>Hx)Sw-R)_$)P_Wj$l~spM_#w8CxK+2 z))WgpUDROv$F~5TmeIyS9jWh6d_DuhcHF!EkT#0LKhW>8%7dB#FuNjs^i^FS@-813 zl5+){LnEbus;1^km41Q?9mce-{OMQo44jX4GNK^!G$3VZ{xSPQPk6Dh$H69nbGkg1;cn0u<~ z&;N4R9dG|YMpvNPb_!;be`CM>>69T6h#jgm+QwHae)rdodQV~)&`LMryOq~pGJ;1y zB_1^Xa%IO%%dX2}C_~_Zsn=sLU?33n=53Pg^4>X{s5frC5di}^1E<@*D4-riF_P?9 zZY#}2sGo|0JQ#+Z@mI{m21W6rqZ=kZ`0y`w1dQK}e*3{w`)s}^b=_A#&1Rf2!@TKoGIs}HS$vNwuV)H`Rqu78X08{ zHZ416oiv_^(2Wq=@uHu!v9r2D#07(d(_eAdp8)+vPt;$=lmUq14pJq{bvq{X@o3oa zu|mwJD|g%kxNWC|H%1~zv@%6kb~m)n8v>WLd`co7$)k=M-UXL{%H862ITh73f|;m} zL(zaBDyp12wCv6WekZTty}`CK)xjgYud|s8-4O{d|B86gqBiN9^A2d;4?ybkOA8R@ z3dLfnH25;Nn%x;{JyX3;8~SWl64LM}mRRm(mPo6N(R)6DEj6?6@3eaL?(<1O41*A3 zKY@99LYs`*vXJAYmW4*PAAfcLrqF=Ps^N&eCr_+^v~HrDe>%VRjw1)Yg@~Eg?;VyrCewDtwJxDy+po;g$_+w zT3(56@+J~c*tW<1wQ2yY(2)g5*EhgM4nJ5w+nk^=3K0+7dq|hgR_Y|bRjY@Y#N?g%~-v>y<_2%-ck2j zFve*!wrtDfuXqSA@a(9}?hs=CZT_Cn+Bad)GO2Hj-O}opPKYxe=5>tXd)X8krsVT| z6ybK?B^v%v1HoT+?Xz6pHS~ArCRzsIm>a6$o2=}s6=Ew}4e!VjV!aDfJNY?=k9Z}A zBqfi{UsHxXx;_)^P@Z$AzTy?D7r9S?lN(io)O{h@1y;-2DDWm*47f+$2Jz3YIJ5*A zHy$OYybMcT?!SqD<0pD$_<(}YO)cegxxVLX6cYnU89!?X5>6dXhIL8Cc7J_IMsG$p z(fbka)T^0tr#GeCA|U$WWkOQl*ekMV;DimM-zH%*2!vUO`eDWNAK1zJylp#BR9QV3 zaJT-B^M@of3QJu~nObtj{}{YMhGfweX;n8}X2yin8n(o|B|aw5IAC%2aHC>wQ{8`g zT1&-6j5*p=5#YvRVY)|#q5d*i=0`Xu`|AgCFR)7QUU9J|H6ZuLJ2z$j&Zv~=8Vsb0 z;bh~RxTbClmrVPH(-Cz&3VU_M>%9Zoc-d-QUdg?EV|bd;rU0Y~pzzU}{hEFdZf>I$ zUBiagYQ&2q88i@}_JP9||1P#GY zsEWX;g%?@NXj~%&d*{2c(S{(H zy$4p$fl^Is4C@{E2_9=&iIw)4dik=v*CZd9p*#W3e`4Gna@Xrf(^S+=_3;Z`q8C5& zxmZRKhqP%Yx&v#@i)5zE-U;7gtz`FFEGnWqL%0ADB&3by-eWZ465A6}FIia9*!u!D z2?(Mv4Cx3Us6LlxdqT4sKT zAT8ZT*_Lp$3N>d&@nueKS+L=Q#E8MD`a)4KyU`&SSb*ry$V!;%o zf(LHgynRUdG}sWJ3_0jR2>&K=Lg__-wXVB+Ro!6?r6b?M)zTLuz|wX2w92k#>;urz zL3!QNqJr;bL5TTasK@YdGwu)xkedg070gz8_%dU(mG?CZYS;?8p|6HSQSYK-{IU&- zbKbnUl@*&VAuE9O7n@hHD12Jy7yjGry41n#(u6>Vo{^Rt3@I@74inBCm=uT&2MOC^ zsMM|^R6dgaYT&hvV(fBWpF6bLPF`AK4z&2xRGyh4+Bxl!(~xIQejwL2 z`Mec<@pB`-{4#EtVXH#xU!$H0g8B^6@M*In^Ae9Us-`gbfTsLji`S zLyLxnhJ$NoC0jWY(cgL#!Qpom$rqNx+U|1v6H7%3pD$gqTwiY4T;2Zr^`wz~UIEhE zuUMy!I+n4$PYP_(EFzywCSs0>JhsZlhGZ|4PL_V-nB-=Vpa^`^YgTiTI!?&5Cz#5l zlpM(r&v=j+s9fMw^&CoBG{sAiT$6pro@UduFq`m8LtN*iqN0k5b%kM8vjnY@xJ-C` zBr`^p@QTA!5bS;DrSfvrN~X@#mS6az$)HUHlD0jAP&fGICj3_1l*v5M>6%1qc zgb*P^Za{kZfIMe$kTD{$^9U+j@m4S)BjMMPO&U$Udpe|zcmK>0AX-lfWG3{)BC})VYpH|C*ul$(p@?>WHigQaRQle zp%6h?$(r>s$+FEs+GZtsVKXTPmkDl^Rs0`1a!FiVfBRA8K=UgyvJhQ{#k10c_xK9h zE&qZaVTMq#k1>^{EcYUQswp5(egiP-an8=yvN37j%kPW;r(cE25am)&VuC-?uSLmH zhHapX&FzH6J#$fm@o>aVN|;G;lLbGs>58D~ny#4Z%Nm)Dii)a9^8TWo>w@{(uy+aN z;#OF2O&`QDAw*gx-$fx|)P&}eBI(G}6d7nqFY(cbNBvA~&X2(ykr4Eti(PGXMMcHj z(cXjuf9+yN)BI8TZQoTU`fZbU9vbLTD@m(5WB>yvs-eopJIlW=nZEngtvUkFG% z`iu+m@|YrJ>E2g^#l269|LLVu;;fmkt)QZ!vJPG!?dtW{W>)xlnQFuRV9KZ$w$i_2 zq6)P7&WS8?KB!uHve1_FXt>aL9^ah&!Hs#M9f?#Qi?wfD1Qjnia*)zo?XLG{Tho3lfrLR4bAkTpTsuQ za#a$0@R(Vb1*U~Q%M|DV0S<%Q4y~VH!+_a(Bj=R*M}s3(s&!OaHjwk2pcih>gy`gz zn#i_}1rp8@tXP7k7nU;HJiWzg=jh?G9t95NbHh1O8v=YnMrA#1s5;xv_3xV{q?-=F zCn~6{98YjK4c`nqgY2nD%gD?W`Abw)%wBZoYZNPWsZ@H}R*$=^Pkt8=Kwv}qKA27S zHS{{^d-0Ey+fB7$H*5r2#=1sT*vv^Uv;|ME-U!Kpb!Xx~u6i!Axxd+8!pFjw+b{(= z$QgS)EpTda&{A4hDIR>^17hl8UJ-3IS{6X+zU49ki&@5q?<)Sr8)H0W(i4wzA;&TZ zJyfT=zqV)QdsgEmyvFn+q&CdlJNWJ&Ahc(c2lQo;0nRHn1PJ7QC4kX z-;GmTugBFrjSTF!l1Ad?2}F#T*oB8S>UfqF1ta8>LQ$w?NzIeRjpV%T7!pIs^AkZBas6nK_u(ePR?%T25}J5>iam$U!& ziP$N3D^W^c%)Uj#wJFwlD@Hi6Zmw}&6O3$~*tkbpYeHGOWb5Agwx_P)cl?u9J`E-d z5ilm56J4WyIZ9*`LN5RL*Zt^Em2zbKm2tN~+mpFPmE_7}*~CrT>J!C-c73dwCNY`m z*&5rGn`d+Mvopl)iM$_89wdYs*%hOO1=}95$JAfz|A^H?>;ygwXkXZ|&k^v##^=cKw2p`^-9w zatyhed zxU%SZoE0&2d2?9=*8Ga)pZVqQSIKP!b3`Nt3uc~cV}QXCeSXD-Ax;Yo$D_fdP7PM$ zMJ+Q&KO$#gEYB7y8K$Hc&{v+2C0U%V5H&E=M=qSSe6jL57@yle`i5? zs>rTad>n%rjM~Y5zh2eQ$0woSP|DC0ztsv;MrC|XU#(ujx?o(+(5Ky5<*@a&csSk@^&-oc17)lszp@)pgUqj)g zHTa@=>1+reqqSvq4iO-_mc=lIpefI{HUoTCha&FG5c<+o&RLq-@2;)NxPqrsb=%`MdYKEqHJzqKU0!jU_cyMePFpV&*mi%2 zt2+t@r*fCt_`H)No8eE}a$*&5@aLCK_*z(h*1%BXbjtfqx38zVibouKZ~ljutAj-1 zYX`S`H94ohpfS~?to&Vdh~b${Q=9hL=ACUa^;Zl~!vq>ux?V*MvjCbeo^+E8ZS-(6 z&D!3G70x-2m<_W{56pGRaUF-Sd2wI;)82??5&XQ;mQboQ-`^GqGy%38MOxk8Oq`?h za`Uy#q|AJ=Baoa_L(gO{HHx+Ggdy}!#$qW z(blt_l^3KhvveGW>TJh@i9IoHYk$ML8^)9z7Id0SG;7(Nw7!|D|BRWY&_RfEn zT<=#ldtptf9DHHM*YJCv^6|c7U;Ux0n|JK3r!iKWhl@IL3qsQMk?TP=-*26Gwil$k zsVf!~8DXH;uvrs}<$1Hnc#qGiA>MV$8<_XYs*TT?X41vd=Z()P`Y;epdqNPwB9sr);B(@CKnbN^?$J%DDD`s6)yR`q+dqDS%h7H1+u%nmPr%0R&_lIUa7uAffuh@X=vARfv} zb6RKS!NAgS`1aj4p*-KNzT|=Tw%+DDc_v~_rz4|z(EL(7jty_{IGM5?X(lU~EBG08 z788cxpKshQdv`F?YEz;}d@x8F?XMc`vK(rg&z(QL9A~>2Jf-@>bF(%dGE&L&=~0u+C4r zWI|9^T+HW2Yp|?Vl{{a3ObpGgDIFcuz3*)?U75nDocct>*NK<=*@~le+^6oJ{8hbq zh#O0mtuBE{#@43Q8STfGJ`ZwOGt07=^)f5D;Cg_dSgu z0Z$`Q44?}fFzUqZ?Pkz8Yl?totx5LD_E2HG!0vY+xUwu`$Vn2+N8@y(q40f2xc|8C z8`Y1pr3lU9_EA)R0UNEjQ7bnLJW@HQlu~NJrwKGWUP@064$yq6xwI?t9HyF+NY*`8 zckDugJtm?)aoX#S>W}d^SZWPSdFOJ?v%KmWH=a!;^n5wfP0rv`SA1anjQx0I+XxX7 z@^FR+Jv7tdjT5gR_lo1|r1ZWeK%R8hTjEZwYufc|wb_=8(dL0OC9fdVG`ZLn5Ux!xE)JZPVpdQJ{dMX{x zEbD~}ueuB1zo%8m;*(Z=n_OuirLPdfH&(>!wxd0NM0Xk4J#}#!9MO!(YhOd-mx(Cv zIsT<(_Wdr9@KePk?BY#J=xDxHwb;jvrn&JY*onC17a9+%*_VD*$ic_P^T*IlK8Iuy<8ag z-1iYlyaa<##`p}$#ismNm_&ZW<1AuaDNSLwR27_y2B`rPz|x3kI*QCYDR@#+Sh|8+YIr;YS79V?n2tsioAr1|zody%Ce{G-YLVX&HHW-4nc| z>$d2guZL^;W{BksL~Se6(_5m}bvXtC>n@By`@HQ$$x?yEP^YNTqT}dIMxXQY4$T)} z3Xxi#vy=Ci^~v)5+Kw?W%H3nBN)KhE_bixa?Ln#RIk(d9?^QZN8of)$N2=O!TZ@*| z0dl``(9CG6?Sq!!(f#Y7>?xFPQ{i>mp)R|OVoXG!&FEJ$u6*OrPbEg*?_c62aPSH5 zn9+S?*&RW&aJFj&-S--}kjSWB6KH&bqTDT7S!wrdXsx{P+CX-ItQO;qp3K zV(YMsx_L$4i`b@oyDU+ly6Qma{K>OTobj@J?yb$>NtxQY*IUXwt96;W`gNZV_cQ&f zhC*@b$0qd|nUlV+F&$~-d+8}=6!k_V5FnoVx7a!jyv1$hlZmM7`M?nIMA}SBi6qyq z11zia~TiFmiM$Ht0n z!$rgSyqTC& z=&`fYoctb$BUgkdZTHCAT>aUoUs|~_{95n&82sqq#5gjqf|NB-(h6vR60_2_)GCxQ zhh}pzcOb)YAZ7%zwxpq$Hc0LQRG z*Rg`K#EcNR^_LdYS>Y$Ic7L-qcShw-_guI?duDq>fM!RSlz^~OP3#ILt@?JsHHHiupO~I;*zld@U17D~q;Id}xbTYPpKtl-OgwySJK4>l0};V4q%9ml{tc##J}pnQpcz zgD&k1sy&Tamzbd`;{#jvpxFvkEoEeFG``GNNcvc0J9ccw5V4D^e`!H+4wYcpzj~`Tf-1Af!rh@1# z^;SE7SO@bX#w_?;+Cps)ZPY6t%ii~#vYbe*T2JmBPxO4az?VPv&GUEnO^@0UxhY%C zg^$h6!BVr=5@_~I(@WF6y&c8`w}$IC^YBmHwjx(+XZZ?>BZ$2x-NQAn_|_9R zTsOs{&HUyT^R2dWiqZ{>z>K(l0sHl(Sx+u+{cZKbJNb34g*wmLQwMScOh@LRLq0WrRTgC_1r>^%YJ|zNS^D zl5%cR*D9)OVqrUE8;E+*Ws}oaPKXGJnf#=h;=6GxUI<>OKW1~hE^$(Z`mUTqt@y0a z>RuM%T+VuD9h67&tjx?#h(lsVe!Dr2X*W{X9Vt)PnV;?`mi)mj>vMnBI}i>R zRX2ZEn`YK!#L84A&ErI$-)cONS(J*LvSAmM7c?66=dL+1EpFE1BYK&c0(xuhN#0KC zbH~XCrx!Oq2}>ELl%|)IW)&;~w_KIw+dh1TD4ODfDm*?^m8n>9Xy$ITO4$c>qw;T? zIVDbzh2Uzp+$l$90#A5ta18?Xvv(5__|l^#H#2eXX$)`7`jI@{QSM%X4z2RZRBJ&& zrOFPDm@j3?sSJMVzS9I5y4t7`)3#sEd0~UN`h`)MN8-$P(^l_CT4y?fb`cQuci1DY zabTX{#f6G2tVRreoIr`C& z_&pinm(iyEyW5c@#opIp2lgs@M7p8U8X)#uqTHxg4URrR)wW<=-Nz^Swt8MvC;3zh zv@F~8)ox9?*JWYc_PKXbCLt&Gn9cgQW1z6w_&3XK#%PV=5c&0AtB{uKn?tOR-G_z_ z8w|_>Tpo)u1~^BHYO{v-Ms)1IC1VTUvwk+poyNQ_U)qc3*{e`?w#P4XYDy(^2EMS$ zUGLw-Cv%>kOpq-6ak*!D_=`|=;bU2ht+N+1DJ}=wnc&ndo7i4dzmdKwor#@$f<(AC z;=6DuRv!SYH9Dzo9!v(R{}~D^v!+FPyn>YAdD{)Fj4LYV7iM0s3qTdI>k{?W#kohj zB|Njy*~dQQCj6GBozGjrIA6bY5PHbS!4}M+ca_t-_t{wewNBPkc|(j4jaEDYea1T~ zhhARM{j0fdiVk0z>JR7v6@}$64~51ZOLi`u0^wvi6n_imni5(yyanQ(QvXRFWCznV zXc>lec8t#ymaqD+Vh-=(8jel3A5vot2j9-qoxKM&@wDT5BgVPA!y9G(KepaGp349K zAAcpI6cr9dM#(%zlsy~B-h1!8(d_eaDOCfRRI1kzZo>A60|zcL`JpU5-p#HZ)3!RXB&YntcE(OSv! zQ$Tw^$*#4j&}u}oq`uLh?Ig@3W>&sgittFN6$yPBpx*JF5}jT=-(*QBtMpF2Hhn~q zOaGSks$*t9D2tix(ndz((AJqkZYOVrSCQ@rF$|OZHKV~*()A3sE^RW^ajfT=vY$M}8{*AZz)ewh#%~t|XOzsx!Yu+1G7ccw5 zelyNp+=zc@y0PzFW(m0wP|Be*!tuEw*1$7>qFB07Gsv!6-w1s@nwF zXFf<`pjU^x#c2J41H%b=Rr=0}COa96lvl~nBl|@FQ!p~GTIQYz-%Y#y*#n>j2u&>? z&p9=d1iDqdRMN7@;us&W;R>D7{uLL)FKcYCugE0{gu1#cCLlChwYD*FWy%yd|3kB& zTz=U+2ruIm-s+~ark;~aC6EGc6(s>OSoVZo3go^_N3Eur`0i@u%G+wksaF{3YHWpd z;xj|W!Mv*ZdHX82vLU zHVr*W%O_`qv}o}%;ua8qfO&de z#(b+ZR@l9YrG@cdS_mS78D^)hD!eeNdfGXHq9O-6h4=cUxHPiSyryecRU7;gzP0R_?^76r8lT7gF$Rr)36h{mx^-H!)ySih)LA4s>4NWDF0@^;ZnJqshP zZ(Ic`iCoXqt$=C@z-hjdpO|}|(2rs=ir6at)(C-hlDWq|7iy+~fyq)>SGiOum>rb9 zfv@rOV;X+u7B4mLs%|UUrSd=D%xxsO?Vc6HAiT;En$UsLN&P&_1AMH43`G{hZ2kE> z)pM@%{`*5o{*}<5qEwr-fAHW68Z^wjst{^Hh#X<>F$5nnug2$w02=(GevU`k?B-jL zXy3jAAv8Qc`Z+KW@S}E%tg;S5=F8IGV1@@5I=T-uLIZ~EZ_)SS)f*r*dBbuWTZma< ze<-P299!1{*0jl#NXyVnyJjOGEBGmgU(+}uY)pqs)fl3BoRMu`(yBT-mtLtfl62cW zIEdld{i(W=>P!X|Oy5&Q)=Cw-?0scp&a5)`i>CMAhX`E2hI0d<5wce`J?0q6gB4IV zU@|JwoZ!tO_NJt7G7s{uN;r}4u)D`LAAQbVue9@l(gL6du#1orf%7uv!PW?WqtHh3quSLz?I^v`4rF_b_xp*0#onR)L>{-+ zC|7|yl^KN!z<{t7QsJ93`{}6P~iqU z*9->&Cdodo;7B+?*~n$1W;$`aprbkoL(%%kO#dNJ+W6--}35IpFEU*~pQe za;E2OPVt%Y&9+PagqkZT?ZI8phZwC$KX@n0LX7 z)B2qMoVT(5U%tFx)Aw#T?)~%^rM&a&|KA^Ng4hfVl$25BVqKON<=+`Mpb?soO{aV$$?YqHHL6T*AbyBCof~US zF38Wva;#hM8~ndDHK0z$?Vmj6GW^j#i6~1w6$q_QwzrBr4J~yb4-sJ7+(qgyRg%ut zvrZD%@EC&e3TeH96OT^@$}vwESLD#d7JlYK6!47M>&-hbQZh92#S{=1P+t4?%J?^g z9B_>yocN!$ZO@aW`F-Wf!(yX~lb-WM<{nlmU*=Hv&fHhQrUAw8T}@w^fSJd^4tJH! z#hCHIY|bSyvr{l>mL%awgZ+zh_Ptfye#bGZOf$(-mDCDgp*$p%91r>O0h__;zV=`C3BX zTZ;8|uK|N98Fi8I*^jOY#pk+$HTt9UrNJ0c@!e;wQ=02#UA>z^#TB89`%9i~ zbV);Aw7t-gKz$!JYu7w?D|$V1a~Kpe?ZNeXnD8r5w`1P-!};f5)%ESB6^*G^E@M`1 zJsO9IhWfU}!&clD#r+@py!(s^R;)fYz_e_7WMRez_?Noh%ZiP)WV!MMXceO{8l&LzD1KAQrBQQ~6U6!d^YLNOqu9)%(abvEtKlhk$JmzRts z>yEl}h^Yj;K_vl-{)<_)xV3c-;y9NI#WcrMWg(wfO_TT%0@f_d{b3^n=cMuuIGRN} z)-I!XS66}z_}X|JC)sKzR%v}-S|`wV}acdLumwRbome%utwrZ`s zZi%em>9{`0$mtO!@$Su0p+yc2z1un#$)X6>RanOdP#G`_marnEMfFp;UlM1F2|wsg z7pXtEu)3WHv4#)i_Yr1S-*$2c4ktN{dCrKG3q8ct%`pOhCoSlA-S=~FtBVj}`1H+V zWwZqIx{SaJVm!i~^{KCS&Btjy8|Zu;u(p>m$yuK0h;{WW={io)l%6+LZ71j7EZ;)) z2stb{)V54^8hB>)E1wY5s45Ec>J4sQ?sX$rgTtxPz6D0d8u;{6kFJ~Tb;LzZdH@|Z zLS7Bl7=MJLbE0EqNTLv=Ug&V&A2S1W)Z{iLa*Uro<5spTK7Bw$(NSHSTLsBu2fBN@o&8h%4hHS(24kG}Ouo|k)yVT=G|ca;D=W(<)&Q#zS0a_W zYr*hg8`HDrU|FDKtFNtHEvB>Zh}2EPTKq)HMi72X>2jhsX(03~9{I@IBCCPIQ2ymL z3U-&viYv4Z)0Mi83&a9e$c-wI`feJ$;q{;Njs3);Mznj2LhGLG*Qt_ueKK`rAv)SP zO)yk^#gc;%f^U*$DSXxKg@*_gpdHmI-`7Sy<&RxMOqOalMYXoX0IcMHe5)ATdZAZ$ zD6I13N1`8voa*VTb{1luWc!^|;z2h*xh1IPUK`n5E@aLT1!t&ot2@d-U|~z);y!|$ zyrUlUMNQv!c@2W4(K-q}4PX<=91%ra57(&b$Z*56#EfO1(<1Wx$_2wvc8WJ}Br z%EGC|DR|gpJoy~WS19qm;cZq@z?ScskH2o28@h@eZmv^cj3iX!Od4AMY0-1|J+Wln z0O7gbv=QrtM(x!}jqT5!S~9L|qZMW?<(Fa%)j3i+V_ua2HS6A~JECQ(MiWP8u2R&_ zRue6CXERKgwx&~M`#XudlLXPhBLG^E-1D(U`yoBmt5MU#stb$&GgY)OSw5{#>GY%M z4~0dL?L7#7SAf;22Mq5qJ)e$$m^xxlYY=BQbcw%t3|5`T)Oxo@+jZ4)<5isOL+tEk zc9yu05NERAGLI!NyjIqf&fzxnA*V1;m-|qr`3|A1({u2%$D3q<43C2(bzfOud2qit zD2g;RmMsKfg&r!HNNfHc8_#zJ8Lmg^Odc}rinZ9+>=f2KJ)(lUAGtgHfJ`B>XzGf_ zDjxxNE17Qjv@oS}DRh~$d1Tv3Symq}v-i`2`-tc%QU4>0Zg&L6&QC*)U0<+h-Jj0z=lBc_|4fiaBrk6#eg{D={;s&BlF2FwC z0y8r7goGdd+Ia7q7ZTIeS2CuvL(Xzam36w1!1uHy_%#-1!;Q3f&yY>Ori)#-vWsW% z%jdLcd?HGZ4`;_)oR(LKkoj6Ajok;!6vKu3x{g0g&wN^-(ir3nbYWg4oxN$bH%Ad?#uHc*NPUVpiC2_WMlIDXPh)Z}eU{xZL_&y@Ue^46>94yS z931YJDeh_GHmyFj2;NnP+_?ym4!7}a+j$!fK*DUFA2QC5Rog{WCFN-46blJ*DveO5 zPN*~Bo=0RX?|+&q-XAsvrnG(&O4o^}^4-&UIeHj@T=z|-3EAu|cZ(ZPvkW2*sDqfm z?jLsIYVg^V;`8Z?blk*oBjwLjl^5_VnmLCdj6?@!7VJwuf#L{ftYM?{_W}mMy-%rCdj(I zx7ESE!R?ohFzV*Bu7)D)frCzf~(ads$F!Z|yEs>`nWb&rCv{`I9yNi-E>Pl%len5~zPhA5-@AmX zLBv`5^qFa+DbI~>N6Fcq_AzZT{y>9IH9-(+mV9}k4#~OSH2UaQ!D>Y-hx!peMnCz;1xS&*Vs?%77U5_tMd(U z13@~Bfh#$?7hj9d4>!NR|GohW3(IB{S!Cx3XaZnA0q;OK`*4~>$havyX`JiuHBr5V%Kf2kz)fG(H%y9j1K^+`5kB7V*tg}avZJ0pJ4EoG9PKe2zj`|4 z2JYB7QU<%O>4i&}M&z!N*fRD5aiRZH?COqesbm07#@`NPl^-;k5iP_@qbZzUrDA-e zuj2VPA{Yfoglq~7bm=S$X*nk}o;Fl~cPyC4z`$&@u5l73=1Pc$!uM|ovY~jt*Nwg* zmGkWmFeup(E8EP?wOxxNaDMcCgA#*So-{V4sS{C+@mj;p^4XWo4+8C7sRH9i%RLtO zyUS0?-UJJr6c%@)fc7H(5w0qUTiSqh%6BYxG8Iy0>J#{-BR-6E>AKG}N?^W6 zewnEBL*L=?zXb`n+ST|vfSsW zQaSd{!L}7dE~Tl~lZf*1hUl8YJ6|d48PUV}G8^gZx^$#&UPQ8|XTKKo!kJ7dUQAFr zURuP)#(~UAdT)igMX}WqZ4RSeS05E)zf^;~V%#7QT1!lQGilaaqP(<{5mQ)oBL_h{Iiw35ZNRmm8Kq#6M%{MDZ8owy9%aZjj~ zrZ+ygl9&_s;! zGW-rZ5w3Xi&HlTEWITCMJ#*-ztNvrxYPVPCFbA;)u1wd3hpC}I;xXB$mw}wPE`qAC zK7W@=)+r2N!SCfBOB#irI-xcx$?BgnJw#2;ZnM+*v-7RbI3SNM!GQsb4TEvi9mvGi zEA$jx(L;xu08mZuc9(ut+aba4)YUBBe_5aYo-iS|sk}WZhi&sE)J#oxf+UGxr}0WP z8U&{y>c=;~1S;Ad-s4(iA9^XP^szNgOrd`YKo9T$o5Tz8oQ51 zqI~d-^!JX)0sIk;n2ivPuExTsdd4w1`7&^loo~LBWL=u4yJc%U3=HxGPVm$nKovk^ zPY}?*tCW0|iJv~|6S>7rxP|V_F(CnC#SG<=>R(+7Wq8yYyr{d>_@l3w!R>~|v7#~c zjVadr5$+*+@50#^fyi3?xtN5NP=w37czK?>2M>FLs91FHGqsO8aOSQrmb?=85HUe}Hv&aX zVjz!~E=wT$EW7Z~55I66p(JupGa&Y=mpC#*a*lxL;9v{p)u*rJ$&lbx!Kbas#Z(?C|14$hS<0rUT|BZ`mri0U4H#;2KoHh&m_I5o(-a>p zpDL?(=l1I$qkiXbd;?2M!p=1 z=6=pGPY^2!GABt&df^k|5V~+v+^9nQ{u24pFB{IFn{68YK>f?_Rw3SbvXbtS|8t>n zk`b~{?Cs&P8)^1vm3(XHom4tLbNjKnH^&)OT5bEL@@}on2&StTe3O(VulzlG3Q2Hw{@1Mo@QT+`oBbAMu#3nxD6_AwPUQxZjQBz+y4{0kqZNHQ`X zq+{WM)0ts%D{q{HydQ_^Oryyb%=We9;X{!zBZz$Y!WR&fMPIN}PTc|`{aAB^`c4IV zCG8r>eW(1)r0#j8@{|{8P5>}Kc9HV@8!!Ch{~8PWy>W&MED;C5pGLlX&II!+7(v;A z)opQ58=THelSupW?{^3hUm7{50`XVl*J<}jO}%#qwymV!NhD0Nwg~1V*c};XB+kk@ zX1kx2ofK1i8W;7^3spz!IPsUTV5p{p%pdrvYGuBXdKFY znpLHBi!;Rd>P@Kh>Q`7Bw-o8B`Ai8+rQi2&m2(f(4n~)wG^I&|>r7e4CbdKVJG>^w{JMq3m5$PF((5w%z2Y z;QLJYE%rzz=G74q>Npz5hj(^b#znVozU5S!seukVa31zmYoAg0jXq68Y*YC~tSzzQ z`Rm=xR-$)9`qx{8t}cqhn3R+;dVza+Q3pyl+bsM!4CXpcog9t@M<;sZD(&;+G!#(h zL{@VG(ehrzJt#o%D7}&9j_&JO-)2TaS+f!PFCAoSASssorW4iX%P=*vJlp(Pf#wQA zC?p`719lJQLng%qCCszTNDG@42XNYgLF3@G=#T`D&$G_WQ)F<~ zMV9=J%)}~jp=3F>d6IVxMKCuYLN{2iAqN9G9t+Q~YhlJQWdiioBkzUnnP zHTkl_DZ`=s)8`_Ct{Km(|p~Nr6CzYMd?A2S@vv(y*LmAk1m9DD`e<|jcE%7(0O^XFGatko6 z1>){vgp?hUQ=>~m3xNO+gw8TNJ^mIXSr_%r>%@LDMBE-Pc}Ou5w6b6Bm*QS;UhWOw zPyL_~1E1)l{9e+&bfeP#vAtvqrP>UyD#~8J8ZxZak^s!c(NW@~ZvDXmz$&ZzO1V>1 zvgJk1>`CaAGgckCBZGbi4$7qo$Ki36-Mw2E)fix~ifqtRUgI1;G+Vl%5NA^B^U|{_ z7D5C#O)av}B!mFrHokHuSE8hx2w)5%SyqWjkD5%Q??;+I9A2uApqOKx$EOyXDG16u zQ!lMfOKOZF2_>swFQ|CU0NuWOZM}{mvOUylTvoE~du8WiT$iUt^KFriIiI20rn0iK zdzdNY*|@-WC(00U@+)e!U)EU>q+jwJ(&iQ34N95qB;5`T-F>-2ok{mT2}dxP@4LWp z64)W5({Qq@N75o!{n}KYPgktY+B(^q@&WW|X2R3D3;t#|8oqWB`l`@xM6gnGz#qr?6_A76ce>6{NE7*gFa+AX5~a` z7Xzh#D_ieVaNH}{{!_~GPrm{nt;R}QZ3S}@w*S9XtaIJW zd>e=p@m#Y<&($VYPt)<>e6^I0j*WF?lC+5%Wd+cAJhqo(Cv?~S5`GAd@c-3Qz}7k} z#193}(BJXmbAA8o#dHLcyrx+;Cc`Wu#tJdzD@M;SZt>*`U9nE;nYLyjjqyc#-d`9y zL@Y({LH~chpj3W3IK|hEUJ99D-<>9MWj?7PLhLM|)CXwKf)}Q}PGncb@4_N?VK=S- z<5-$|9vb%RhT;e*%c($gA; zh$Q&c^{;pV4sO_nbRn}dRAYZ(_0AJ)FD9yndMq%jnKEu;#|$z`w8vk!*8<>1N!%$;uy*&%yQH$ zD)1eCzE{|tQ1GeFHLW-2OvB}K+gcrZ*P)36T@8$}Tw~&5+uZ1!m_lt8)ln!2k7Q0B zIMpI%Uw^QDne^DRM(F6rXNT>kXA@&*(+MWbm~79Lq9!1g<0uBkePVAPo6PRfNWcrxh|d8%ZosBgx0u^^_lu}< z9~`gNamVWMb%ptC$2>%FBD*0sgik+FZTn@t0Zy{m-aug9cW?7eNTIQG2zsC-0O7e& z?GP5jUaP(;j9kV@CDqYd&ZIFMYL_O#^$|z2BaaBG6_IiC#6qV}*YAq0517`b@RY2W zV0(1{9o0@g?X1#HZVvrxcQ0N0`CUc$A>vhp7$TX*{|O&z2}A7?65CKrSTD1))1t++>;O!DoK!n0dnBOl7$?PrkRdM5O@5B`046@_r9ZS_$}dU1 z(L||_i`ERE8Z2eXPo(Qo?6sy`YudU4I4LVQttpR2b|&%@p{4^5If*ANDIoLh6wqri z%qkp(H1C}J62fGHd)zsN-4m(!7Jd1AhVJrhBCHi6Lbgew6744j!|phj z@l8c#oY!dq;0!F-anjg8;J*q4cr~DE^fkrLGvxKOg`aru-1&mlC|Bc<5V0GdH9gVSn^kcrEEh3&IDI=lxF4 z!{p;7=-r9Zg-%pXD>Y5BZF_40eByzy1H>GU^nNgHZS2wX2hQ`;J=bL)fXpLzj^c&e z0X%4Nnkzpyxn^3WmrHm6ASQy=kNRx}{y?P>s4^-a?8mlgj3t2{`R%tS+Fd0?$c%T) zTbH3SQG6RTH%zY*f#*aM-YbzbtpL>*S;r)gGpf^XTEnst37&QXorUS?{(B0B+j(syFgUC8aT8I^%)QbfAA6U3fn=doUUc;3JA3IHSnI+y-QVg+R0|IBio zV|s#lODtT>7Uhn172XW*@1)(&^i zPBIdgA19i9Ks^wW7F9GvT5unBLAeuAOYuvdH($I+4H(jqm;r?|mV>~*5onrlk_0Ou zK$R`TE9jABFA>ov`)iO<1-w)ndt=bmf&uQPA47@uH(5WGTk8(%I6jVFN)B@`PjDhnBGqD@PzZPTvZbB~;ZJ*$ zn00y{$yN!=cTP7Z-L137bBKnuQMx_27X-7ji!(^d8eR&=0{1=UF!JGlFm%gp+lL2o z^b;ySHPKmtCL$XmZ2IM3zvid*n_vUcq2b0YGU~a4Z``;p^5UkRQ1(ow!M4(V>zWU_Lom_HDal>BZEL~ej&Gva1{YkR}pdTz#FNKP{s9LLQ<0;9u6 z3_V7F#GD8&?C3N*Sq-Ky+3KcVj#2V~p=lZFu+MZf(Oavm?aTS+YBia!rJQ2U34xI{K&@MSP(9(<3<9 z<1_TQdia^X`Y28QwVX#B^#bK~-{Lick}0PQJ+EgGuZ`#MNF`DE{%Xd|yB2L#UgKN> z`FoQVKdBYFBf>b4cHbLA%B1M3Sig}eblf1)SoU^vTj4TwR-2DzO;lD_UvoI#`!b{ctOyXU zkrQrAFKBTSsQF_n<}Avb6S#4KW!Ebsco38}Z1&kP#S3#Ug7JHS>K_w`wsmDZnD@2^_Wy=TI4j}U%&v;ef`J2?3ansi`7O6Hhq2pp=l zK;#+8uicn-7UEJ+IrKqZTdR}0RaPasrfqbN1|E&OM?(Z(D9g)B9cE}whZis(w;Y|o zJD8a<}S6 zRljaWxkMp@=yF) zz-h2$neN`J25$dT@Qw?osCQW+m#c!+KWp{5dT(PyJr~X>yL?Z@_;}+i9~t=0lk+Lg z={}cr_p3kLw|7{~uP#h0bqOTb`79GI#Bn36LRD22UE`*|3qFWREL>Xa$Q1M1Jnhq> znJFx&4g5oh1(uNOT^N9nBHXdX!BLmeg9iWf-D-*L#f2gT#vF`=rsS4h&0(!aHT`?LC+p%#hvcy5G_rcU zE{iVUyf>3d-z^XFKgx{?*yX-GW!Cx?T#QWP)p+0%-o7q!NPd||NO|}tv~6O-L2zJ^ zkm{T8D8?gyP*G6y@|b$3w$N|0|O z211%%JdOT?v-G%hW>w+Oam%4~kJl7=4=2@OR|(PF%YHC1kSYkuYLc;Xh=)e+Ff`w7 zPK34ZIENb~e#kG_&w_cvEJ2lqopZjh%m?EsVYVgW#s}rDNtwm2aKgAYvuJ8DR<;{4 zk}Qqk&z`x?o(-!F??dH%mqo|Syx(=(X;5~mP_`GQhH;<-@arDdmk@$>EYPeTerO3F|>8syK8a`*Yvwd-yMGy1fuDQGyb&#~cDPee=GyXsGvPSKSYrN^2W&x^&alTytgdy> zi+|KpB(1nsL{-eb)rL*`$q_YSUV6z%Q2$f;+O*o5zc@+a7mz&1|zlvY?*XfSX{JIerv!2fUw)EtF zcUjEU(?;7f8`_^?6Qc; zNMhg)Yz9_vd@X?sb$8g}Zl!dh&J1RV&<`)jPf8ITrmpf_cKPu7b0PAr|A~MvY=rm~ zrI1^1JH;8X2IGB$$H|n8vA&!H048K~i&G>waGZtYWOPy*jmg@Ny>Czoi>;U|P(tSC z89>za2HR?eOy=pEizSC-2WLGBa|9*Rj&cTIq_g?L%AJQTTFK$P;_seuQTeCirK#65 z){*l5Hv~WT-&IoiUGUe><6CY+`K; zSlJ|KAdSW97*#?_EKUyBhmkE_CFJDhCFF;T6;je0-VY0NkL5=lliZBOJu6-#jB2`&9qrw$m&wD zQ;Y6saD3j9@)wQrUEMX6lNw?0P_a|B*RQZoni)JK#2LvmR}C)m4py*#kT-cH!tAj; zl6m!0QFo7$UD1p5P9XhL3ga!7c{4>au_0A3{LC1{vJ)Z1SY^IFXy>;}00X6SzO9Gx zLEdKCov5Y1L}cENh#!-x?fu7;+-!x_=`L{SbOQcT>K!lz@T4UhRl4P1lU?1%$j+D&}K9(QK{>Jg+FN_1Ubiu*Tqtx$@?B7DO zh{~D;NDdw3_#V`Fon%M4t;OZ!etQA8rOzVCZY~kMd@Z^&+`oe*{yd9ymblj4RH?Ne6C(%rnOl4mic%%pKTO*yHu;6tBA%)-@ zmxUJdyJ+IRoCjUNCZFirfY%^Ei+j_G#b1ZIO6FEx>WB_ge&)VRWVl(Yh_rF^gOJQ+ z9q07M8j~@h?_sm~mUCR*x%`^9ib$FW#~>Q3IaUrba9h`{*!HzKAACj@dfsVhzF87o zenA|Gd0>m^xH7M?M*UTw6Kw#koSU6(WnG^nwiGH}cqMx_Ew#8$##Eg~SE-@y6PO}q z>qU|H6Dv7>5z6#?^Z0s+lcwh5lS{R?3%?alNnopH*dRpPXw)%Ve)1F1Id6F5XuL&+ ztk?PdsL@RLQVe@%)s8=RinO}6vQ#mI9^#78<$E&BvUc&FZi-luOTl?&z(li#Oz?_!O_M20isk0sI*A1X4DqIt#nt872}J^i9}W& zspbxI?DX&cdNX@Ju;OX)Yg1VfP%BtewYK1G^s2ST_}OLHllo|Un?si7udpwm1z6i- z^+);tO#C8^ukU|{1k|CYCCQ(=wb-~!dxcTC5&Sx*) zQ~KvsB#r`Z4%NDG@+^r2zqa{TT^JwPBQi0tulj0ku?G1tbDH{7EBEa8o;-C!%yvRx zvy^PWd4Rnq0dmsts@H!&^^(|Qot<|+MRA!gwjZUcqaFtO z5c{f*xL4BJo|p~~iz0SC!~W3c$(&l4;o&88qev+06BW)-lG?spRcsSU?Z4AU<>jwL z%w^9-`9;_(^?&SgwDe10_x0|iwkzfKoi%}$z(Xop_e#SemGiD-m2SLQ-?<~Nt0nlv zIl;sy(5-tru#qrxE`yc*Zt94JaWvugDA3rrs1Qs3({u%6#id@g#|(8vVL{CKubWxf zyIKe*^FW(X@t%0hn;P5fwz;;#u^ye?!440Xks6WL=GdR~s@as;WjYE`)ZD7T0aqrK&fj*~Qy+77=Zh4Vz8k(l8y&d$#?zo_PABJ-pbGzg0 zhaF%Ruy_T(qYEgGn#uem&%!%ELz&~{KW!vDV>e-d5>BS&s&%S^A3Mnh4qt0;u-=QY}>XZUV<*a?X`_~+|*)* z{zQQ*xRS40pImljX}f|WPqCqCV#QgWYa_^@dmr#^O%;Uui6}>=>h{K5G0{sjZUnSd z5zvzYh$erFGSer_kgg>=(1WQMY|@KzcOf7a8q&=wEbO6nnGeC#fH_$ zYkzp>x86FzN^FX(nmzaFQC<$=a9MJ`FPsjk?d8-*He382ZxJ>)OP2~+~*>`L;?q;A%$X6@u zwS_Z;*REB^p76`j_dUg#^F5;7 zOB91_?M|}iDFb|mJkfJK{`XyH<6K{*9sob9|E+#6Sl0hK;Xe4mU{#- z{2VAT%@)eh>a_Rmr9q+cM_noNxm}0shQC1JectMFHOM57ce;MAVbPz-Dc5Voeo*rq z8}WGzMpN0fUcb=ouNlQ<-=*r``W5h3M_^^B{1xCSu)pfLcgqUEe$Ner>K^FZ=d)J+5J4lbW^0bFi}I)<{4`*u-NI|1KI}nSLu6} zzlH>gtM+i#?TM{^qIK00joPiLFKXPmf1M4yGt3W`b#MC%~<~&nkt6M@2>TO;1t_xo-@a|KW*i zfKKTvx_R&~B0ctax#<9cuhY3+!HpsV)*`sVr#zy^*WZ5uprGtm7kdq-$KZ}jK^SBN zoC(dk@)_y~$x3$fM~`R0!7ME9rhZ2#5vSqqLO1l#=N-F50}*J|&S_}?8mVHKbtLk5 zsD(VyewJAD;c3Inq<`GS^Gpo{B5SLA9>nLG8GF3*uvf_GN18n$TjF%Ic!1I|HJP7d zdgV%-+YpoJxFYJi#IT3JU&aiUA&y2bwR!7hSUzlcU; zf<6T`V`FQZ^^yGUK&PsR? zCcX5CBz1MM#8M5I$vdTKP)aC$g+T^O(*CFGvh7;`P#2$=|q>*6&aN^0V*c zX&yNUB;a;-+a`o-b^7we(ii0J`fKZp2aLl6WRIsguHL!uhySQro#zcyQ<73yBwr*r zE)hro2+-)lx=h4Q_>rEuc?RSk+a%|4{k>{>VycyH)8CJeju3c!H;V{8|9(}eYy)^d zok7xFpWh|)l_5aVF{K>+dtIDh>RMz=2eY$fGBgLw$q8UH^*WzC=50xW>!I_(tbbDZ zY{vUr3-f_Ss+hQd`QO)@oY8-9) zX=JQ*NF+~e*hNPh)-nrl0teS)Xrh2cQN7?A?BQAIHK1i>@^YUB5Vp6pYyG>XjP5cZ zS!VylKf5sNz&BL0O>^#dEo{wC+-$pispdmRKxh}sU*Bew5%~T}eXd`BSrAtX8=z*v zFMIp6QR0MM@$jSlk)%ei!gg-&&TpYDPTaXA{G0!OZV3B#9PqTGdIn*8O@qb9clQ80 z;AO14*)d&`MzDVV&IC5#S|652{v{>VpaK0lDCV{|U|=Vu>Cb2l^-(BG zyC)+a;Y4cRjLnkZo_UHKjX+{}g)gsp%!Ox$tJq-6akn-SfLGJKnL~Sx+XdOwqlX zDXxyC;*oi@Vv^MQWDuKQ4lXWDX_YhmxkQS;^M~T9xIYzgS#0e7SI5rBz0#!0*-8`2 zxr?a!^p}SB&Dck~*-VfALxcED++)0yCv7)8&!ge}b#ovKY0;+f=l}oUzwNzD>j^M^InGX^+pxkvXTaP&PLiNJWPBMFPwkp zM;Gl!`V02!!v#bl;Z+OczxNBg5-+?8STps{KyL%KNJNW!AD2-fo}4Qfe(^S3Rp5F? z1{5y;TRb+?X+SH;5M+|j_aENHE(yC37P?%(UMXwG{b$+2n^&3}+Pb>3UoA0JZ5oTEYlHbh6w+a!b{{E zVs+Hhc2PX%qHPOE5s{8M52l$C>lkJCbGTwlDs?+uNuNhtvkE5sTa&2-eVv_468Sw7 z5|Kucu_=52O6}mlP}S8BF%i(sQzpZjNfE$FeE8c*th_>IDJzLxZSQIshwY!Ck2gB`A-|Lfsv(lTm~hdT zDSfqjqLgbK{3lk9w0~*=?0{Id#egM5FY-J8YB+JabzI5t!xrkQy>OzQ@kO8gQO(Z> z2!}I@y-6v;GhcxM#7F)6Y=kqd7_I%z9l06PkGb6ES{A| zGPrWeI8VOT?T<5=h%I7f=1KfctpO^;h|q%}gST$H_&iqB&9OlO)}8j}Y`kP5@uZ)e9Lcl}_bP zKWV+qfj@EKMdD|XsW2JwuGnrq^Vjk zQP=wDbyS8)ufMxhCgk=_$WhmLgUXB{)!#n{1pop66P!G!TxpF%*uNooKqkw0@fCP=b4yo z4y{Y!9A%OT|5pd7+yh(%RncP`IQ;p% zf5X+EA@<21i6_+WJ<(03u!uTeuT(-LT z;KFJ;Gxa@>R@`Sy8IA`*h)9DXBAqjIr=)b3ibzTbNOyM(U4nFX z4jsbK$k6be!F#>G|N53|v0O`e-}jsyXYc1cdvA^jQ~5Eav+V5bSV#2wM44UJ^O2mu zQnwNl8D-@a%)Bi-Fuu{4&-LhcQkbGfg;UMma5neB%tvB`Ig)$aJcieIsViEMQ@7cl z5;fPZG3|uPg}{dmb>2}P20C5p@-LtALgm-FrScz#jT?Hwl)7Sh%49~;!d$P8BTPnS zoKnV_+2W_&&t2QR$=&WB@rHl`}>BQZZE-&3%GEoBVZnH65)^1-%R<%06vrV?JUdknUMCr^~e}v8%iUI7Zw6l zg?;5e`t>`vbR25_@|bHmW?->&!W^XrgFWxYo5al6Ad}|Zetj*rqaW=02n z^nO$bu9Ab>4v^HO~7{;BW3n{KspA1J7XwSs5$Y zzVUZqcTcn(gJq@15M}RRcO~STx+tSdQ+k9G)!_VV1&vGDaym;z6%1z9Wi0_QGc+b!hNLWfKn{i)&s5Eqp7ZKojfT*Ppe49-Bw5L`9apsD5)|CNsl!7w7O z>GN9EGV|xp47*Pn1z3Vtn)Aw_RZ#Zt2NKyz{T1jy+Zw9%xko&AsRYpO0}^kS$9_XC z8?~X~x;Iqh5OyC*$}%`Gk?Kuq)!MLS_%mM7q>mL$qSHX9oC=(6e;xBCbNWTem#S%o zl8WIh{MGeh+SGcZV%bg}+r`l@^7ao&GV={ z0574R)GSgLVa*r!@E2F_6=BU`MnJ|vXYQ$V_$4JJ$t2b&t68;CtM7esJ0DWimxhQ3(5|xK?INckumwkD7Qg?p3xqI~%sgg;HOq0;8euVn) zVP4h7YW49()&3!HM=_ob5t`@sti;^#Gyv4&<$ef{&pg~jk-7Bh;C3`_m zdvEToGQ+n=+%`rQ*)|S9XlYq9Za(g@Zl^~>SwZf7VBnBa?f#VWe(jtuL*>fKsj=CB zmgd>IdG&U4;AYLfMr7q&q#lHFRv-SJB(7A^*IY-_{hz{EZ zFA6aUg4Xf|FiQO+Ct8%@;e`A7sk7>$KlBDo79w)Xk@XP^LS)vC=AlL7v+1 zWt-fHwtPNM)tes2ohx72E2RkG-&F@Jv=;lN_+rLaOBmwNB2^8M&!P&V2TkRERnn82 zRATL$7~8dKTmc!U$NZw_Z6Tbm-(Fq!ZKEY9J`u!H^`0+=t92B|JqcBo?MfMsvtk^%t%WS|vrtJ&j+@uix5a;myrUtjvH37V>SADSO`0 zXwk@peTK>`txV!ufD?-OOutPc-u>)QJ%@qX6ug6V%GCX5R6;!pd)}32+|kMkcir)ToW_zNbnnqr#$^;(EF(C4_MGVNR?qLLcH2 z+|qu$zYd!%X_F=lwWU*9_Z$d8(gg2;!|#X(F*UV^;?LP=+Yr;^`KN4xq7`eei|k{J z@D64&^`C9N{r>XiirH24t%ES%P(GCLJ5+9&#?Ib|W?AXWfaKgP_bs8FIQmTyybMDf~%_PhG8Po6FI%A(ZWb+vDeAw#Hv@kJb2HNo%G0@%&dX z<4}Wg|BhQeTr!p8O~B4bN{_jNK()aeL8h$g`00nz`d;xhk!_bn{(v1|p7B(K>Z*Mg z*}s7%CYPaz##Uf0RMOYLae7!SS>mF)FT=s{Q;Z>3!9?0aweHB+tdC>768_%eiqj@= zuM9R2JyaHsNaQ0C>G=Go-7iZ{KP_iM8eB+ztA^3YekInB#;KN7W&IQ}kzLvQ&Sd_o zQFQ>6 zGiwPw6(;Xj@gj)8w;8V_=SvkQqbPa5A@p*#J8phzWUTPmD*MYWY%0M!Q-d*FR8c0KSY*U-=!AzXpo?^N!4H_{d$V zZezxFEYG8RH^epFPxmr%8KzwKk-ZN0XXX1mCd7X~u_qxRr(#p~^6o0$h^LG=1(9lE z)D^_Z+wyYHD9CVp+U21rRgi_TJOO8%gH3Jt9&{GG&MKL zEhfy$BoADu2);=bx9aMz)&$sUtNW%NJ7}LvsBh}^EQ8QbA@+54>cf4i#2)X{Rs@{! zp%IS(^&854&*sw1g5j5|{noh>hF+(!f~IW1ipI<_6#Hr z+|`|F&nYbXT7#)ny_dZ-%}j4XQij2mOx9NQRyL$nHQN`&=`=$_K8O=K78Vv#0XLW8 zxT&)FiO{v|yh>w9PU#gF#48(E!=MU&cp6!HYFAX$ro)6Xv3ew(ZrAUO~#aOE@5 z$VXuhBq@laIX{-Wct`=;;Tn~aQ|f2ZS6o=WKh=KKMq*KY2eB6hWmZ-7%M!Z9^(ikU z#M5ldw#F*Xe^QIdIgj*W@GylmcGO;nd)$4?zvDfDhgbl&-4aFxC}vE=?gM47ehEwn zKOkCm**yw*^-5X0hoXjKs%h1F>24fleol@J$E5?1&yXW#22|v4knxtkTzsS3I!^PA zoM8>-^(K^oB8Tk_QXBRHVUpgw^PplfFdZ*6rDvorUgGlW+3^I?Z)BMJnvM0qsmRgr z^gEUY>Me3W6h_oCNrnFh3Npg5Z&VJj-aGtF`zCOHr=*WKdqM20<&0qJ;JMf@{{oSxVC9$Q?X1d9b0p(+}0;o$6S_U@UxMv)o!zzB~rc~vP# zYfYtDR8(i_ICJDsa^3`JM#gPRRrc#}Io{%h{l;WNcPx*y*VM4&q#sM|s1&l9sdO)C z!z0p@aH}>b?3s!V_Qp=oW6WwiY49AnZbX;!MgQ;Zh2X90riD%S!jP4=*^a=US2L$7 z%+3tzE07?njb!K(9sTNn+V@93GjwuSRT^Q3zt8t%SoN-hYPg>ja_;?f${4f* z*_RnS`>OK?*PVs7E|$xIVSQf?hw+l>mr>J_G@UuG*jywapOz%N^wRR}N1yg7R+SE? z4tQL?VjSwj?|Q^u;mqbf#a_Sff{25j^thi5bPG&x%#Rft5|i7FYTx*^?tT5B~k-?lPwPgYX9#B62fy>nngX%1M{ z%#Tsy8;rnnDSX+{uix;04L+POh&F?cDCq{YgQ!2_qx+b=G~ZC3*+i)=Ci~VocS-xm zln_5VwVWYEKUzU;|7S^0Xl7h1s7pbW$=?N16eM}nq3~=4LU^d-8R85yYUG=K>Y#45 zLs`~-YGN{vv(sb6{EDThn@2N>FF1#sD65aaAx!|&0oiNv7RaXQVCP3Sp}Y-W#-%f; zy=;{saU-KB3@lLshV=|O>7qDF)#kN7H4iwPpYBKO2MmqA=~6QH8Z50KO}-Tu)0#J+ zo#r`YveXl=NtBl(_Jn+_T#7r-9=blPVv?J(>8+sYeTV8zC;2l22@~NAuaGcT)R}2E z^XxsuyMfQ~sLZR<^JgbatX7%Au!n7<&oSZym979C#Ub`#{0nsh35#AvU2Q*FaR&g1 zkk`g=kxt7adw*s6LQF{B$9Z2L{ks-sp!9egw z0+HluGO$P%^Y_Zi$pBE-@vB@`o1U`LycG6?7wV$VRbR~WY)!K$2F~{OSJ#-&XFd|No96gg)SV8iz<;jE836yInUojlTX!($ z2Y)D@9DwK?)%ti-Cwy4dUS>Jge##UQ7h!aJ`whcgo0AX`GRKtpW)cfEx0CG_7E|Ub zs2;R*%E<)9mMxJhDMm6e4NPKq33y@)_V_~dGo#*wr~zpK*SyLeXaz8Qo@SLEf$6I- zbh^#=n`FG4gGEbiwDG1W0r)}*f`V)NC!zD3Zs~nzlgqtez^w{m<6TJzwT4E&WsUs1 zk!;2MO5l|o&_8_$#q(;9R=c;msnt6av`c>HyPu*OXyY7`YG5VGigFJ-t7ue+tybEv z97{%e)Wvvjq{ub52VOX36}kHJ#o>^{nbH5OoQpz6UcOO2p6mUib)vz5&bj>BqpUrU z1y2kgohwqmam%>%g$Jz2Tmsa&K`2QrR#TA&3-%so4&} z)TR0M2J?o7ZaSIo?(720>>hv-8yRjitTUN5{{-Tw*6NPoY64ST9jYQUoHAqiKl=Il zMm0_OPZqp;hFwh_a#Yc7Z}p5~jOZc?<;0S8m%S;4ZoU4$ezWA02i{ZSYzl;+SDkte z&Zo*MALaW7(v~K%cnJ6}PVI(IcTs^^g}m8S&~)aEw2TirZD6-XVEL|tHS*VBF(pSJ zO7_hOLjOwsx?vu6V;B;uaz$cTYnD~jp53mxb&*>mwX!?X;EHN?NK!uD`zjSbY`&YC zDfj47;e|!BpGorc)HO0MxqpVLF1_BMirmk_R?fnUYl}83Rc-?AYuYwDw{+FI@#YJg z6AXNj4gkL2)8}VpYn{(}){;J~omo_GzfnacF<6~vaulP|cH<03Bu4V50VDzkF=X|4 z-R*yS0SNFMRxQW7*V=$^1eM#cw_z2Twek7h%Cn$tF@Jd}F8x|45uO6;u!wpXqL5V) zV>@??RSUE{6_rTdK@R!*P-bDctHbX~q5g9kn{Rns(G#u!ZKa0)*aE3a1ht8xlF~2W zlWOQ!Kb!Rdn;cVSN`x0#lX6+a;Oyc@SAfQ9ni02~v=NAnXPHfqVtIW*a@H-`n^NP<;s1*)#6=1DwMzR=?Z> zg4M8_bKalq^k^@;)HnXy`u9ir5`xohUbFX1oFor>y8L5tul-?@)YHzKQxCm}7X3F; z1bFgY!rp-ic?FsNiD9_FnJfFH?ZKa=5~nKA>IczBHAO8F*Zs)_YL&vSrtU6vF(~+X ziIFH$$s@;pea8V%) zF*S$CD~QCuoB8(G>+f3*exE>*&7EFv*3+I;${* zfGp4ZuE|Ow{G~O!oA~4~olnda)X#gp`%5#U;?cFQk)C|1x!&VszhhkrYD44AkETZN zUMQod7H;ods{13JYX8M%6hZq~!*V~8s3~%s6V&aIhz3ge7IMz_TDg=<9xFZzR4{|# zhe97?M>6+Be-*5rWeMjA<%abKvDp1#3kZhdRI`-PYMWLYpnk zzW;2KPbs0j{3I`+^VFp40VW)9qynkql$x>Bg_p+MPP~owP}B-3r7F<9`0SGFgF{0V z15!M>(ajGsxt6}`-b-Bg8 zJqdr)N~?tn#s@_lxH{anQTh)O`V*B323`jsCEq#*{^ohtpE3u$K;>zMZ1Q-F%TAj$ zxB!VN`Y~Yb&uAjmGJRL*%HT7aH^XXNY!|h0x1~W84dU!XeS*JXTu(EjbgDoA07JxHndzB}dy z?FYfnI2ai0s||tKJ8#(R^5aHtf^*JB<(c; zJgS5KUW|en5=1hOef9@wuQ(gd;ymfV9mEH`9b3^x!bul4FxCF@CQsxsCl3on#l>l0 zL~e?6qQFslU&O`yU7#zp%-9v?i&@p3*4!o(WVqKl9HfkMzPi$V=& z=fW<6jKP=6F}0ul`*crU!Y$(OfRq@g?jrRf9YYllyU%6&jw_P)u8jDN{21>-vz=cJ z)>)W@C=lO4)11pKE{5!X!!+;@q?g@aus1a|`CAaS_zL4tRPx17<|moUShR2g(^&+JU=NbP=CiE~LP&2)xA-UJ+)B`~SR)1dGURso8dcaYqU4 z25;>_45pt-P$RhBS=ee?9Q;=T7dqzo8s`MrO9W_mSx{NdVIi>E%C9w6WR;xg)r#G zZSi_dt-B_VmcT$mIIe@P&fxH{+(9HH!e4?CAOI>l4vW0cpFg){U|xLmXD`O+fgtn^ z(QOwAHRc%}fU-7TY6kdgLCm<0FDbY#Z*B@zhKk(^MBKsk^*H~5)m(bL9rl>~&ypcq zZlY!zfusLUarkG`zzJ3ze*Xq;%+N@1=W^jCKs3g1+kg-BB*hS7y;+#k|DFhXZvn#= ztn-2!-pYYd+jZYy&HuB}fEuik4OP)ex%Rz= zGU)QI-<>$YFl z%3_g+Vt3;xM}ZBQ^j=R2JfIl< zK6ie02*q7IAPSgJ#dCe#(!V^+KXProdD%T13Ru1YChf`#sU9PDKeDp!>>OG5V-!aU zLskoo2n+qooo0j1K)!#Ah}ivY_hyCbK2|2-R@qMwR}5AZ(PFdF*Sp4ungw3?pdfl6 z7&huI%YPK;LjNz;2nhjfb8kpa~W%lWd_;e zZx(t=9Qy1+Y)Sy)zZ#GISLZwpt{=o7nGOez-UnPTnyp?Tg52o(?wUFR(93PUG|Oor zSOhMq<(>mtlP78$J+dWBqlP` zCqj-~DC%JLQw^Noyq8xPexCrq1`k)-=$-$lAO!-;vhGCAnGXHecqxL+u+*#U_!Oko zEWZ1?OoUDVeDYDSuWKOzh(@Kf4z^~=P5f#f790yfKIIYVn&gu_gvqfxe3*a==1{(# z&;Zn+Lw8p64^!~pLp`7mv#%16oqX7sy=8f87TK~5r2;Y6cJZK`t~ckg_MJa&3xoxC zsh&oB@c(O)KM0jdjQVz=3l7RSG>y88-RFbL4PN-kr9WkUER#WrPTxv<)e@Nq05{-S zg0Gf%Y)gn7NVb)>c;Lu>UZbY2Uei#M|}jF0avHAL~dQ#A8`$2fCf2?-dR7hLr1{QG^l z*so?j5lVf}_n%Du5G11T%X0VSwMPoy^qTQXkiz55P?k9UomMJZ+6umG@J3(k&o@cX ziFSLnN$Gemrw2kuX%FnzfxiH^${N(4p}cm3PMXFfCSlJV0gkhORHh5^6gFBJMMcA~ z;OvNR-(2!GNBELRxeUbp&rg;GvVVDgkVc$7R(SI!Socqt87&MAc`fy^?q8#MiVZBp zTwZD^yLY~`?rbmLB5S5tm6FSO#`9VZ!P#;NGEAi|tGfE6@^z{1BOoqNXD6g??mXi9 zZhl}g7tgoKRd>k2E5345DC~aabdN0c5{XKkiECaS*LukS6qW7>dOX~~&v@RfnC0Q1 z?7g1hAiBSGc*c0ZKn3)^ru}X9mP}KN5wZhi6e|j{v)Z1zNB`E6CtVT!1#7XDdXcds4k3IOPYw?UKTg$qm)NH~n)6uM97j|%1o96v_dZbRnbxc13CfqsC1R^yh zjHPM*3d%pe#RFQEqvLsjrC)j8lkGpaZ*k3)wl2YWvN8cL?A&PT6G->j?tLcsdVxcD z&VXyzJ0~W9qx~oQ1a=GS77kJcDnohPpv=Y+%FWu zcxV+LS+(A<`~0*vZU1^*nIf`7QTD+KvGzr#O+kX;4S&vl{r46efC7_A4PSB2axy#? zHt|rZ0^itOJ!z`NZsuwxrq{mnlHF{?$9j3Ex8{NJrH5b|RXqH@rI*qdK?9syJZlBg zwm-{GM274O0+eB^(IY)C%tGB6HL4E;?(91JsY(%?!C1lxHRZ?uu>@d#9Ex1qD!NZX z7$6ywx%>y|G)2rPVB1UBgAeB=VG=Ikt#=Spn?x=&lug{YdTJDz?Qi)M-Y;IKjgg7n zRJm z*{uiS-iMeNS4Gz)CAltI>sB*{Wko<+&Yb-Rfw)Zv&ddK(TXoy?vPnQ9R+l?+1tvE2|n)Kllftwi<3a)Qo8kgZVUG(p^S+?!SC1; z?P}W+m$mb={W_8P+{S-q#eKlk<^<*6a{QaHW6`}|D{SeOEuAZEmho(>d*^kh-cKa_ z_Ltk>If^!QZVy;_e#3X&ooBs>$$U!v9YJ(U3YgC09rKHC_IDsXyt@)+)wch@3W7lx z5$+5$@wfQF`!40Yp79r!RN=TRqysun|DxVLGCewXT z4EhXdVHpQ(Cl&p+eO>AXW4M9GPbK5vCn$rHgAn$wR?jPZx~jvd%kk&vQpRwP;c7H& ze``jtr4uSEk7yQ+g9Qwsua%SF4@(-hlAi{E>#fm*x--+&(?uJBd0-zP;=4i4e{EJ4hZC>tLfz&xg_ps$^Bh^HdEY~$j)cQ> z)P48fN5Vf5IKAhG7;X}&KdrkiK71a-S|7E2NItKIXg^5DlQ|3CzVTiV(~`^Iuxz@h zG@-maYf_Tlw5FsqRa$>HYqrd_?hh-WqXN$ECB|12^H?ncHS#SS!d{`_kIHA#v0zksyoB6Z5m7H= z;kih=)`PiVT|-Df^n?Q$o^M2Dst}%<=T6n{^yU$ZhKFFjZz_&?kTuS(InPW$Z%-i2 znwR8-dh94AaDMHHmH$(Liir!~dU`;gnhz4jivY9f!=u89TcfPQ4zCOe78b|H8!x%l z?C1{LNnyxRtyBzV@QqJAnj0$$VSm2yf`CEJmf9|b{n3*%`XVAvJ?t!X{cJYH;XJo! zNd>L=&Lw(Pd#P=5^QC@%bNKAR51*x%ko$WrETioD54JboR^t4=s z7Bnm-(OY3)_ov~@AMH@a6~*Q7TFYZk3nZz45Th`t<`3sPUBmOm7ylOg{-sXG5`A#; z$RNxMF0NS{!Hu{Y{Dtq4)GLYuEF63kR4rZqaPr-dc3wfkqOWSs&DI&LA?+tY&@AFA zg7brXV?2DM)JFnoEx!fDNJaieSN@gfs5xKO;*ezklwkJxddXxN^yFV72L&YelvLL2 zWM|`EWfsf?(QMKQlXSDhWHmHnlZV68gbtbCT6pGK={AW1tn=5y;^^PF88Q|=D$fu+ zTIR)5qWvTd{2)WdNl}C&#@F$CTW^P-kZN%B;6nUPLk7WE!uA1ee2HK zkAk+M6O~)$$F?d+rtBH~p6eT^q;|)|P6!cE<>lCsx8iS9i6gxBRmtN1sdfXh19vZP z{A~Z4qz5izZg#YWjM75873B&8Y6YJ}JF>dcgX@O;SBB#q4Vy?J1=vAa0Xpu!XGy;us{Rkc zEog`TWOPZON;+inZ;=Jaw`j4BfAN0p2(?~hztgIJd1Fznwi5KU_#=@g)WEmNspb6F z_~Ade<=?+W-UXri|MKa%(2rxLv^BKc>%tz?)U2yrIDQvYru zFfOAkL1(ja5&0JeJRrICzqD_q#Vq`uZRhkm`J@UeZ-re0uJUYeN|n64Ow+j~j3x7Z zY^9A=dYo$#svjY)^0lq|&%>UAQ}lrJzbKs|=orh)`kaWXQ4c$uk=|aOM$$6-mNIo@ zSBQrFMoh&<;xUuAjRT~KF{$uSc(jLxZ$y$_*TkiG(3+4wS&-2PT zU4?lGrgjUW1?+39wz77?{67*y2yOZ!=WyqRifaQJj(7Qc7~PQyhnqPLR2@AZ*DxE+ zLM{2*Z)P{lwF)z4M;$Emk)DueIYn1veqNIyv+4ND+%t z_>JE^61z6H@%XQ;kiL+6qVvt*mYsm@j>3LBets%MXUSiABSxO(ebq)qXw`N%YEodc zdbs9rbzM8cx zN(Z1JrSUxj_|r!6zJO&!ygP#Ag`Laz*JYl1>ZsaRvMCON+SKgQbj&gbp)ifn>M=YL zi#hEkmnI}0s*In_Ea5>sLoIAlNq|o%Kb@YEJRTA6eJ=A;OvLzy2&PFD7`Y|t+q1gN zG;G4DksnXInVNa{v@sh3*S5@@vu`@~jUd7&l4clZc+2!8$mdlLvpP=|%Dk#Qw-CLW zSjKiSd2g!Mvy+8zwQ@a9>XnMZCnO4~)8&Eu6B%~G<& z*PXI;$J!qWZ!9EBS&NWzHXv;~_!9LI2*k!_ddza|@zS;ZvO9*K!`;z+bQV<#Qoq-b z$v-xRmEFx=*rO+}?neGoKAD9>d9Eb3cLv}tu#@#guL}A(Z;Q_{>SEH}+niwTr>ZvX z=^?FVx&4$wkL&TCJ@m;~SH+J9$ps(w1AR>(_z-mnx_JEjQV{HYb;hr>Q3+PKcH5Ld zmVtt`~E{v9R_#{ib=}9wLLX>J%x9w?6nM2T19SQj(4L zwzjr;1qHp2tbWRropw)rw$t)TlIH#W?RYJ@M;!(W8&+Oru2^z;q1Ah7|47`Tl9tSE zyz4v1!0FC>ex7(4^F|g4i}*?X^*m(-g@sDnvI#>+73mUYx=3@v+BbHpv%(Z0WQoQ}uiD^S&;D7l+fzPzdNiQ) zLNDM)B3w%3kxkC=$d$4b1W#tKC)L^U-g4naLGl{c6S-&DYgLnO*G7vU9BavA;;3be zA|Hx(UI)m@C2TaDY>v-LDRWVt2=4G)#nM|R<^u+C#g?NWlzucye55LJ++x}0Jp53Gr*x2z3j|(yy};4Uxu`o&qzurO6srK z>hzn^Xm&ZwMUCfXSIwWB-`rCaI4he*)|sE!XVhDl)!6qHF>izh$5#xT7v}U1PCV@* z9L|Cw=ik4M`GC}%?6^ZGTDzzrb(b-6D3`5&V>+?%))vQvQSr^1chQ_HH;!d1c-O0s zZ*63=)Pk}G#`x&5%uM;Timdjm-ky+7GOp#oVGY9)k?vRsEoLuLlt7F@^sxx|VnK>P z515|I{lZDNTdEUl8hFlwt2dh7`7RTINf;dj{k~Y~sP!jae1)%H0d^toDB-yMq)pi| zI@?0sQnV0-OpoAMo-0M-gN3^`d(Mh1cp$b>UT&Y+YIAoawdGOrK@^T zrAY}16obz@HA{=USFD-GW}=syIkRs?S%lqjhaE6HZOEReo`AGK*On`&8xis9i)b3@ z{wnUCRy_Z$%!&PI+baEO=NI0FJ5lBJxyGKBD+ppIrbl9`5tSba1X?Fv>OKC|o_z}s zIWUFC#J)Y<65sK=ermF#t8N(lc;R2+-DhlDIKf}_Q*A@Dz+pQ?29+7_b|B1@ne(C{ zT)a%pxr%2oCsc*`N`>`8X@xa321A|+;=jzmOLcL`>b;q{@@T2_%`HS+xMmJ#fm_XD z>YV`7?;?KE0~IRzx5Mr`+VpV^4dUsUltD6KMm84cg;GS;mvD}L36ohZff3zQR+aZL z7Ks4@mLD7N)^k5JPG<>O?6TFQ6>|_W=*2}xV0Cf6LwQeChKN2Ne)s@8JD~)o`$hjkrB7&cg^@HESq7R&NSADfXt{g z;>gGt()7BC+*cxnLP6(7zmJ`)OS(}f9U%=Vrmch#1&A1~+%>?Qz)VmrFO6#q8nQ~l zpKTR&q_aZ8@v&!#PZuk+@jMv`TZhdVR6Mves6X1~vPl)fVqBEzw=YfaOtaV2&1p2X z`9>2F&_jt1qjE@u#ZMs$I-lt79$QRJI_3$9pgfwZguV|N9*O5$;Jj8eB_~_##~;-A)euW zL24Uedinmeui{rSYx^z@tz1D3rwy3>e&e|@x;Codqj*U`n67$W!RA0C(eRVS>rIZn z@1eav4}I$xmgvLja;zF^?bo$Gv~Et_Wbb(+Ud9@|cI+C|&=2#iB03|FK~oITdBGY) z#6^tL_5M${+lIjUFhQ6K7@x0VF>&X0h`vZ%t&upXCT~Mc6gqZNv z>v`6NQk`uU^DTbJ6$rsT7(hAc;ym%D>y}kZEEm$oc~T5H3~#{Un8k{5;W?*?w>ZAF z9&5`$$iincoorrM`LXJ>Hd;tmXSC9y4sBj7*N-6#HPOq1l~FY(-ydqdz(MTmtA|h4oMVE-wcqxV-bi2v{d$! zyuD;~mV=CCVZ+(I;*!KUHZRz{PYkQ<53(*5V4W+qwU4Q^sQijPN2KM_chKNn9C(Ke zk`23*nz@uo^oLJ{c+T^Q$Qg2q7Kpv5j~^{sHW}g-tEPZf#+R@>F}>MuM|fTofZfEV zF%IN|=}F@R!-?orx0ygM5{d_&rfa2?k-HtemFQJxhe=N4mgYx27^^y+jOKm;rLy76eToCKF;6AyaXTR{P6;RiN(e+-X^c=fPD&u z>`WZ3h@p=?5O=SaITWl&P@Bj)`OkTMxiY@V!2_*@k>-~xzrh5b${ zT8_m_g8|f_58;WVJo_SnnloGG$A-9sBHCOhXlHXgLQFaGc3 z9a~wv9kef6JNH@6?H_n}Xt__JrxL~Sg-o&ikzXHkbD@Ovjm6{jILO>b`@Abv^XnSb zZH62*gq!wt-QJtz6@O-6WVAsbY|ZE5LnrIQBKC|qma;978D9lJt0q^SUpC$ua>F9jAi^SSYENY;GNR|94KD7xuGa<;s zmUA=Z2d7J_7E2m#XE6J|v53mRSRO7Cg5~-~w0#f%c8vX(NA?pxVt31Sk;4-fE5phb zW@RW19_J3Z=azlbEVhuv#f><3HL@_O?mEko&XNW#KS;hHjbBO_~cAavS49=GcO+ zWBf^0y z;~g(f$5j`y%W2ouyXjj{_NzNtAfNvB`A)u`duq9r2qtA&_Q8$xruJ3Lmmf-1nKU%S z=X^*hbH55e|6Uo2=*nE9@5l9>gM8win-b)F zu9#|td^dNWy7P*Fkgj%T6j#R{c|pj;JA3DPJ7wdM+Su&eY}@5HJ5M~xiHcnoftIju zqb*dnZOt|a^+R6Lk*c8gEm?*eEwQ(0{6=$1&$^^ns3B7 z=AGBOO_gTG6y&X7g-n4WYKs)3Hn)sHI{q)E1#VOKYmUyz9Mw0?T=0*hTcDgLx!k`w zSO!eEO`a7JOiWf2&7=TW@;yd;8zC2^928HqPOHW>DYo4%Pb+oA!$&o_(&^$9I%SoL zu-vo1l)uqNndN>t(VupscZK!AEk89tdBgeS5T?h--)Bxu#lFvpAim*Z6R3K0tiGe0 z{C3ftm6rguJl*ni;5+OcF$&enT-*cU$lQ~$&^Yw9T1W$n9D6vt4{u&4rzBuWI#9_V ztRG!mBY5X*j8tP$OHwW6fS#AibL7DuN{OMe?|KfoMk?1~f$!q5uhBI5#Je+Q(5gFy z29MNo?Wz5!?6qtxT`QxU>)F+~v?!rz$`aGmsl?$2v--0mszdoTh~ZzuH74y0PuK1b zLl>eg?p#bbZ1m<9)4AlpF&M@clT$9b9Yi^9A7QX8jCU^8uW2Yw{YFF0O6umB~v!7sw$BbGj$oM|q(>&Slcoku~)>Bu(su^e* z3&v@mgF1LsVH@C6PN%XS`mFJ^9bDNM^ zU!R-oW~Xu@XZ~(VyLob>Zp`vZMe&{@bT{guoKmpYG})cZeleN*p+t4iTXLV=-Nv27 ziMHkBceRhzRJU`%fL;HcSsTwZRFo~h%#k)he0hk_eCA=X zAg0JY;8~jRv91MauG-~es*B7D0R5Mp=Z2#e&LzQET(~gsDKluFw6f<_zPdOx7>XCsuW?L}!H+ z#}dIux8oY1&g;wDm%I>ZRnm9dLJHpuL_?Vssxys}3mRNa)C~kWSl}n|zzwlAKuO7D zmGqm4yda&`2?g_c`Cr1P`Z}|T2%v%J+yaFGwEC82(-|H^LP0-D!xt)?GdSondbY;% z>y4Ld1YhETaABHGUPE_RWj4j*-hQmfj&1j}(Y;v>^A&0O7HI{oBsgh)!DfYCsH>uY z3OPC`!u7?(w4N%x8*D-Tp^|%%?(Rwomm)y7N(C(oibXbV3(lO6@epv*`<4)lY+#AM zxg&cUVtvc7+mN46y2r*r=aR6nbdyVZx|_}|!=)Sdxdt-7Tk~80WHi8m;Jg$K374Ec zf0PzArEOpb{@6P$Ii8-bZ>;)CTd+RcvxB`%V8mt2gflaYO_bA$anwJ97rpRY>##}6 z&G8^ikqx;S^W2aa`r+oiM-=L$zLT)!C3e=anR{0+<9>T^IXw66VSUeA$MltetLo!* z*-K`H?$PT^-TA4OAxQjaE-&e?9i+r2;sW9v1s1o0^b9_poIX({YYf+S65ol(&tI7| zn4=!|#w(~C-GTX_KVXH24u@r>vdVE^`)uCA7~ynKVzH|CEVE2+aa^!>|L#@KVG2P_ zbFs+U$#2&bQ$^}EXj)6A(@jq2D-XZvvD{K)J{8?Dh>u1`P7cFcopQ38yMky+ZK>$V zv2J^K(0ULyhR)z##uZH1!Zr;;efXraHa+Wwuf%eHw2+g-tAgfnUfrZTZTOM_d2i?T z6=d0EHj+WSXf8ohpRYA3*BV}MdSAk(xa66G`<-fZ3f}oUpedec^Y<;Ke~LYJ^^8<}8nvW_2J zm*K~zkdeYR-C7Fxbk%-fZ<;##sQTl#woBh@f=zA(+NVf#x&=+B=KEMIYJjb+mduS_|7Y7&BsCZJk|Z3g~2n8*5l7S!(> z{N#&y3!N)gPlnGj^)K>D4%&vI5T@c_pWMv1rKByZA+BRxuLmF&IP|9ICVSd<)i#LV zh~MrZLpUG*`Lut&^7eBq#f*j}ho|24SFeX@z8QJ(7S+D^Yuxu65tS|Y$fa{Uhi0XY+H3j z{$1LFq2l1mq#JDw6Cagis75${xaqyq$bKfd(as0jx4d;zG@l&v?N#X2_2o<|d_3<# zO_hcWF_;Eaa)J>TMJ4FtZc>s*DK_CIe2hJYsUK5|pLDdKCyPoopD0f-hdsr}S6H~7 zH~0R9h)b9};}uRELtNkSN*1wnteDoCj}x!g3m9vQbJ_x1f0`u*BI*@gCGTz@?;kI1+q z$AuAt<6|w0@zIA_(r-&QR^w9?45a*XQ>atvGbs^IC#QaLy)Mcej1XOJ$+yMroZ={Y z;dLQAj1F1R-ack@-92tmKh838$r;y5FH>w5n8||foEI0B);1I0xk*=BO4<=Fs4J;2 zqKHU1>~VD!S>R+h`p__-83;A1VSwsboD!GZVEf~;`kw^-KdP=WAgZlvDTt4}VIXjmo#B{cb;lEam+1zB1#Rfbc4Y}DzrT-SD#Fbu2wUM(em^ZuMgwUv_(!IV z<&s<4J;OYBX6KZ@CWgd)+!eqZT5?C#M9>naU_<1>NSiY)v#(s$m5|72Wd391A)XK? zxcOxCXUp-%TqA`;fE4&xN@uwIXgkl|Ox~jX4Bv|I`zrthgFYiHq0$ zN;TuVprz{t6fj`Op~k&JfcoKA@NE7BM6SywaQCrpnyi3>Xn?J_C~Rrj#~) z>*f`-A6li<(;lfzwfj{%qTw|UxRpQd8pqK<)4j1`YmQ$Ln5uD7BO=@5KN(x}DO;3YP)W9`;Okq>>s!N(TN;Qc z;;(s!=M~HLKkK+SNC`z64v`#Plq&|B^-)r2sN#*$wnE zYxc!Lj3+;y7KlBpp#Q?hsqh0l7$}fYNefD9H_@6v4@}5trGdN@WS)fLn<&b7h2PL( zgoEpmAehl>GI?@RGHbrhGBs~x%VJQA*_vB3oM__64jQ;Lv_bsZQ?E;6hzDi550q>S ztlI}2hiv1y1gd@=t<2ZVD7$Kx^bZx(3uTB%r=@EFcn3#yNodsooeB)!=CY>fE; zqAtipJEis*Jrq3F!3RTT90ugIKk}uikCTCBd{Me?Mtyqe5-cSk2Dh0MH%cbQx}+%W0QgCWm={~uM}kU&Ous8^uIO( zkdJ@N2k2ZDL>(+y?IX45C2Gkz`eN?Ln`B*WuK{IS@X2v zsYaoTCVw1@o%I)@2~Wa`bi8}i)Q_vFsN5%WZF#61fb$BW)H7~%K@P-S6eW>!!`kZYZh>qXf!t?cK)h7naGD<_>L%f0L%fWpL%(D?SeeRO2P zO{m?nsO~7xZ6!r2s1`bcd2fTiz-_QgI2RoA+^@x0v1#t?>W|zs-4isN-!^ow~%tXV^Q*W{L3`)aGU1ow`TyNIjkeqfJ{d z53{~&0*<1?HUfZw=By9r^#iLPSGdo5KOtYBlV=_vGHCDnRs(mPz&(S2Vy$$9sQtS} z;I?p@y_wR%+GmL#&ARTB?ZCi2)}7W<5&O&A)d#05rQd<#WX-eU70t0p%r}MUYV|ZE zX7MDwl(n}XKzfeVxV0~sF?MfrrWn7b}1$GJFUeeo9c`#+Nf8v9p8_wm_B$r2i6+%lBVx`l=-_V^{B)?noIBF zKrE9r&j$mNbCZwcro=y-hiUgFvy!S+70%XrpvslMmKiO#SI+c~84LjbBi3J$-Q76dpA#Ze$bQK0aGlUcyzqd^70BG zQdpNDu|*w+^USp*i#-gvXTi9qUZzPzy|bj@ z{$zH(0?vEQV&vUwdbWEeaPi z-^M(xeQvw`&c*3sFov@;r0QUPXH38fCH}=v*}X((%6Qfw%+H-`mkC0mR{U_tH2njOwZqMu&J$P| zw7~L%Pl{2xWKI^B&V|yH>P1pGK+?g?_?akHA78h%y=+C~<&rjJ9p}0B*c5x^KJWMP zoKJjNhVAo_#OrnKZ`yZYOt<(D1I_6ApB}5PG-@b`&yEy={hvEekU1Rvi1iz$jND2V zds0*HG+#-ffIjWCo=IZmXtb)sr}E*(9P4lpe`84Zcpv{V&1RMdkPF3hnK=g z2~E~!W_ku<5a{_&-5&ne!lr(B*Oz!4;7#l1ErmYeZ-owE-$bHyO zfsGx_Wex=a_geUD8q@#El85Bw*T{x}I&huK9}?8% zN1N4e6e|I(H3P7&o1R>ILFvwv)%i{DWh$C&U%Itcwen8y+NVkWIV1}-0(OL=#*><9 zh&4&Af+aBRAanY9Fo>VZF$%JjL!W53)|Q=DfI%85oYnqJUkCH)t@4^U$kE-YuePwE zaL%%#I;6QW#zD2)3)~Jn)SWZQt*u9Ve|s|XV+b>j&Xi4QB0sqE>5eT5JYwELnZV=z z8#CVJ6xvYARze1W83ME?9a)sJryZu0*i%n=q)7Qk5^Sepc1$d>br4O`8cf^q9SwZ* z90OQ$X(46?*j*&v@S(Wv1e-s&MS_eh7U@fY5sB5ijhDiX7Sr3FCn&=0b>q%+6nc^~d8uETH`k3+pN=a+khIC{K4}?7=>0vC6xrn3m!uE;HYy zhDUZv{UXX&42Y>}95&ZX22rg?OUx>ST#sx@O(xY*f~MFSul~5QOfoqdEr>muB_OfR zao84dlsnObdF)R<+v!lZ!DB6;T?9vVJxlUh9MhhrV=9l4sAgkh`dG2mH>YA5hm*cQ zfK%LXG2OS~F=`5y6TiOLRP!PtZS|Nuso`_~{obmUrP7I8e0eoW&-N1QDi36yf$ww1 zB6RKo8?*HaHG)+MP|C^}NLD;%gM7Df6Ij z2q%v1-h|AGB**a1d=XVg1m6Z#L)JS+0Le->&&$mwDR8?K3yEzKm9JTPq zZM}e6;??Cueu>EygGFd)RgaSTR^a?((I4#~7;OMIE?WP19tGYYVopU=4-Z>x#S*rk z-Y&N6A;_;E(B6?}VJvS|nC1yMT-&s+DIfpI5)bYTmOO<8u~lo#d$xOAkB(bS?02qt zi0^)Qb=Ydnyu)ND2OKeCsLj1_FtkuiB*ltO zO>?o=i^L4;kn7iitZrwbZ_S6k-CjngK2fa;AnKqchU5DZDge*sKj#VYyGL)L$vP-+ zHbMr~2F;4OMTE9@vntDRjUJEgUaC2u`+-P7M~x~U2NrQ;$5uwh5NoPsq%VT5?FGh5 zVC#Lesg%X14Q{nR#^)BkiELD9CV(G6+TPt-sWhf|3A7|Q4V>>lds#6_aW(WDBLo(b z{XAdAq5SM6ce1S*3CqDgfpQbvoTVwkkrtV;$f7^WPj7XSCQGBBkNOSTGp=&L)xyu~P&-$%^Xq3)^}JA?%*y^hzw_lA$8TpZtj1>@-!*VI%-dbopUeV7oY-c?F0t&cBXuPm zVpr(Q9yqp@Yw&>(vO-t_JthS;E@inx1@c19mEF-$CcqtNtkq3qb+-nU`*bALz*M%H zLfzVqlkfXlAd%{|3!EKt`lPjsqit#_34zmXp81-*!fKcUFx`xxz@oCS&}nyW=M)64 zG`I~8M6J9!=&cUnnVr}!{~j_P=^eXhsQqy65Anzxc&(%3WBSM z815%QLc(s-S8$}@HSuA+!ZSW87a?ZL6YN7mP)Td@h}o%v<`FFNxi=;me-%@i`!sXC zi)EZMTUN7dX@OUxLs$nU-_lU@FT=XXfKu8;X59&^>V5UoGtTcd5$bx~qvPYXE{7jy z{CB+;6a;r@PKS)JeA6m^Uwx&|s^R&5kpJ<`pz4m#*HTFO5D5a(jsU~C=b6(xmf zJo*I@T-`>W=)qheObYwi!UjwGgG<2z`EtawH&A(B2s){qYl-sKXn@=x(M6J^TDiFR56N= z*6y*7jLu?AJLkqKDixDFF8C__xtV88n}q$E!lF$r1(WN)-OO~lDAIEigC7NAT{QWo zWe7;uyrL8w&0wR}a>;>V>#ZSfZCj}F14F|TKw1icZ{jjmO@XmSSrfv9j)O}M+fxIU z)yI)LC0@FVz*M)!ufM3-zk`e5)1<&*E7YDf4fq)i;BP?!G$5njqwb$5zb{cLrNrP~ zYy6%E`+>uDjU8`)KZ$4IG+jGw(`vXqu+sO?n`*!hxHPBfO?%VAtt7Bj+2r-B%rtQ) z1X>X>H5IU5`sbxWESn`&0>k9nFdmwFu<;ZgSv+W=g_Gtf%=*(Kav_C5Q&@k@yM@R9MC~##PhWkXTHr1 zUb>XG5U3tS0^1}>SSgT#;}Ptqw3S#2!5iP`k^A9mfV`rR48(RnY0ZVto%6Y$=rKhO zTXT9rPOnb&04Ja~M@CpZUe?a|>$o8MXm$HdGq(t@ifelNo37*)$cgIi$v%chn6vzuPw!k&&XHMz#{SWw44q@6DrIr~b=;bn|NB$AAx%-HxZK0|vF zcwI!v6A*#$dCshMz;|xq{LV?;LNn(!*(_4%p%V+Ny2iIs&4rZT$WB0ysw3kpYJ_E- zEA0!7t=~CB1&aXj7!CHOkT@bbESl0%Kwxc|7oaJM&ZQ!9FTiL1L|ote(>vYge(#M*8!I0?eQ ziNG!XGeMYXkaznnkM)iq&+&Lpp><=!&QdQcYNRnUE)7$5LdCc|#Kbl->P5yH>GPa} zl6m5j1GUVGB&tlNUg7UWVVSbj8P90qa=h=O^LWt`Hqj<;xIOp2!md)EoOm_{-eQu_ zdliS+7J~C~;2mrg%J+He$tM(mq6cr2-_D(Z@hZ~NFVpL3<1By1mz5d&_`QO0173s7 zJeEsK8|11nrF8MSXC#)1?U#GYK6nArX(M`d<*PJ7?r>v#M+Ae?1cjSUnT)xmwF~v1 zr>g=-pK4@JK6kt`e?Hb)Q)wD|IrG96!E4UqI~~dAC})W3t3kk(WY!R&>PxPR$Zo`= zru>#s+@AmD51o^FZgxxX!SIv;HVVUp>Bi7dV8SRJ92?XlKa?X)JQ(J;k!czoZ81hsCGH{{~QNo8wlGu=!?O-*~fnQ-@BEKg{0wAv7je%aE9T>YWayL3+Nw|3u zw|Eu5<3my~9`GMo5qo}zC|){U9GehMHm;NN;L`t^jtP%4*J}fKLGZLI-Fi6c*})Pz zLc44-w;ubnQ%}-mO+!`~CQJ|JBm8pFYy%T&C6BD=u+b~6u9wTjg25xzx60D*djtoM zvLT-e9)7(OgTFB7pMVV>rbMGmz(RllXhJ~7v*|JHTqUY4Zh!MVf2{aYAl@mi&##Cm z!pdrkyfBvQ3ia5p6@%)gIn~b?V8+cXa3_@^1LQ8vg*CJRSRYRJa5{Ljq1c+)p)WZA z1I0EHc8mnAPYG8n&A`mpJMp2=ugK?^lI}@QIDZl5{{M52OSCV=-j%LK-aO4s#kg>* zRim%SUnYUY3M)wk84ZM#W~4wZ(-Q(-Y3R|pobgsPj`hNrVAwffD+RCV{iUfhjA*%! z2@v5XP&68dQD0&VP{Pxx^xl^53bHcvGb+>9Gv?EWp_V7nJ>ZK{J_3cP4%vzspa57= z#wN=oU9BDQG?89DQaCg*f?0n$cp%cTryU&73dxXxjvf=4XBt~m6|m8k;^LnbaEVSy z#XcPu;5Ym?eRG!r+rTzcQ~d_TjB0>xIeq0LzLL|f&}ip|QA4kv7rRyTjo6#JnTu~Q zGOAh>*1bL4-)H^Roe1S)-f|l8hw|xU%AUk+i<*bumH0l1B0MB{c^`ErJEKfY^IpnE zNuRWjHT<4JN4rGutTfgi+D3EX0Ev5_ta{Yh#~n=-?aa1G*o`;r2 zm-!o?Pw>Om@T&D(^-Hwy*6h1ST$xhKKbUVQZG7k+E7Fta*LIa7rX%`Ze68_pP-b1` zB*rn<*yQrt-+Z-qD!tO;ix>G&&NX@4ke}brQj1FRtc$;2KjyX^FxHrD2y5({wp|k> z`2`s`>;`ohBYGp?{2tn`{J37zY1g=&02wW1LQ_-mEG@7uOL}s_F|V|4<+7Q7W%7K` z((_s{$r?y8eO^YBt&J}$q*=2H#=F_E#^4t(>BtB4aqBdIkHhZgT2C*x;$=cpnsHzqy{|ugEjmHSu&TXS9vQhSr-$iqQVz2~7^E*Qqg_3(nmK$kT2$ z6S^E(&8x666>i5As%iA^^63lD0tPx|*6`R(1Nd+iegGAH*aY&|@(;2M!_WB1>9ho- zq)#$#IlHIdBt&Nd^f~!)&!h*4RN~1$Q-78s{h3YJ)g_JL_h)kL74RcEI{H9J?lpje z(Ec{Lxx6%{hdqvKQ^1Qj#V9TAEY2f=Egra+;qpz-jJS!-)z5Z=a zFIBCk+N}WHw0#@AN~shH_in33S;K&A`yx-oN#ty)&Yp=*Y00j&5f<*fdGh76p?cnJ z67^)%Ez6P=S!_85o>6Go4?IVMGhc4G6~PT*^C~XT#{q5)U!M%J3;N1<~CDQMF!T({*_w(pwr!x`pvkIP9?8tbp9`d~; ze%Bo67(qsLxK>fmtOL@=U7l(v(JuckPuI<~-qC@iG5&Ji#E_629YI1dIko1!9(bVi zW6iIgC%}0LGxW;7f`nB)VxFmYe;N}s%8Sm=E6pKZj@H+Ez8|h2HxPQMyVO6!6NKS2 zH5Pw*F%q~MuShO{XhC!muo}EY;WLZ`cj|45SEqDgibexpqS6)JMWM@wsHGUy8sQu+ zm*&Y!^^Elt8Kn@1L%wcqhV>^({3zsa*TcPZ1F&$1`O}Bauam`P@9v2yXS6SK}eqJ8) zN_AFz78>TbQ9<)2qs*Egx+5F49e<9Zsw`r@ zJNYHLGuY8#%qE7P{+8n|D)olJgVeG_`U1c?%O*7~eO#KdEyze?**FRe&x^q|K4*Rk zidk!uHa+>6$w#CP<*+r(PjTKbiT~}`=S~`AeQt!!V1a;9xWC=g5jbxlZ4j&XPGMaW zjISU!TJ+O>{0lJNy)5&#C}O#DF#m#mqk-2otHwGlPg4f-OSd-NTF`4Xg%A1392f7n zxEnOTUUlQ%rySrpt9wVG{T(oPX}Uj3sY2Zlan zi<3aIcu`n?U2nl@UbE>rXtH!R9l0)kL)CBlWk)`e`6dV`+?CM1G)yv;Liv=#ur(m? zWHz<3+v#V|k(K7^l=mZ~Ic+@rw5R{ir~(RrZ4Q(osjknm1p#G1LcGWe8I(SeJamBa zaklinh*nMW0SHz4zM1r(&#t&+C*pZOMGeclh=56*%eBacgYyZ!=flx)yB2*o2&>|v zVN3+kkwIoh>!fhW`n$!yzK8`5CTNeO(cqr37!mv)Je}9#*i7r`0BvmPbTjd+89bqUpxFi_Xape1n(XdGepa~ zY)&eoqe^X!jFgO@jlq#p0~27*Vq*EQJ)_K5(3d?joi{&Vk9Rf){CLQmDwV{eS`Mo<9@_SCpzq#y--w_ zJxN{5NPx~AWs?PwU*e_-CUO=4&@5HPAU@Z!SLj_vb^{B0xpkl$L7ktm+)LWhS9s2! z?IXkc6A9TE8u&PFSSCG*&tFa-wsM~|sK~yX7jm%X*)Y3XVd#Ti8T>`@6j|)9(55~j z&RL*?m?JpBqbz_}jm)8Sd*>q0s!y1m5rBoM6gHb((?B%qxuPJl%jI-_9930+dRa>1 zI1%v8vN&*QUkL@y{QuW!5Zz>y>qc+xTL!Kpkdlc!h3h-ECnF-CiKRX@LzTaiDmz3lU56(u zDSbENCrr&e+*nT3k~)I~qd11RH61NI24)IF-!L(+D*LVm0GUpF=fLnjqW!QIp2+8s ziJ{E9P8mtGd52J-<%-*6ds>uZy(bUG)$NHR{{u-zKyC~Z09Oop2+ejXc4k}z)~|Ai z5GjVH5N})t=4Xl4lg6gnRh}aTCW=~G7StYHG0lWte-GY>_tINf>x&`DbJEpK5LjMW zl%Lu=j+(DB!W3$dnFvqHs0_^AFtz+@0cuZq9wmQvhVlaGGptQZ&}R$abicl zx6kd=#bZ^fJK}_sJv1>4pQX-f29}SAGLk*8R_CmPD)6$zya!Rj^Yw+!#{uuf!)qma zbt`_0A^Et)_r0Ky2s;M@26KbRPB$n^L`1+NYUwT@3pM13*wQNlRQ^mZcHd6Fi-uBS zDi>0r%llU(R`<%idgLf@<;A|BypWG#5-jmP>Okzvu+n zUd9FdyiI=iXV?0Y!qmehsDKBp5VYBLQWK-kInOejXKH~j!*|Dir4Be6+HcL;_}5KP ze244P_m|zM2sy}Cm$1zlt!Ifl1fV)8=D-3)ubm9jY75pzVaNI%q|GWt;R`P5n@Rel z+7Ry9rfcryN(%5ndUv7X70V7qC@x_4-=Wl9o?Ke3&pPU)_a!(@cZaxGcrP3F&RgR^ zR5a5sE?hOVloz>YR-ATb&pgnQxc1j4wwMI0|pPq~%U_R{0(O{9YSK*aYrA-*sgko9&Ob z0;%&^W?p}r)|8WDDb>+Gy*?u+@U0i_uxvQa{0X@MIuOe+yiw*AZ9Fbm>!lZg()K2u z`>7GL`|&Fc2YoMdriza}rSV}F=A7Gwvo$r7rE6_1o{7AZ$M3GjD*SidMkX5tHgBx9 z$5>+z?oG;D6u^J|dZJkRuXhv`Mk$V06E=rFLxkc+3tPH^A8=0U#%b*@pNh0wSvHJj z?vNFPS)Xk4EYHa1<^SfMwo^Tgj7(rBIb0)TQT4&4M)J88OAbM19=1T?vHWu+A)CbS z5W7jFpDM$KT%{+cU1z@2KR%}}GDkz;#tEIfUq;A3(EE{T^Xnm`q~)M{N+`rgFjjB2)Lbm<|)QyVn5a(Inxm&0LQsc zFPKe^c1`l$ytMU}oaLwmAeLaiURxp6nQl=PH%c z_A@~!r`BD8Nz@Ku8~OzIzW454t$=3k;-60HRbSD)XOKw+R~|U6wo}iSqrX>lZLdIbI^`e<+Wb^^%%h-s-mkyoDJEAtEAT2ch4L_cl04 zXE(#mVoXiobhc`DwmK?tHCI6WNkplZs*%V=9|mciKK7UxW(aU!NiC3%W=B4xkKEm4 zYfSNMsK)B)-46~Mv^y-@9q7X0X(!DJDP)h_h1N#}U%Q1zz3%$kpcDETAUU1^@|XW1 zc|3p_J`4m#W1FFPlR3@K$=|pH3LGpL0lfu9iwzf7OqCVfmt-~z4e}+kudS{1-P(_L z!Z{VuTX?2F;yR5a8_nXhVxqNd8Lz(6u;bA`C9e#bL$@AhZ4|*FuUdb93LHkS?)V~U zsdN=0u%F%7*>lr~rr;FBGM6SJRP{-;8s~RjJ-D#?_{Hfao2ss^WdaEBa_PuBu4f`a zzTc}h0wr3Q7p9aoYUfREB9HXs5Q-Z14G~7oU21Inn`~l9o3JIFp>qF*5edGRpck@K1Cr8vsyL#Wl9< z@xMVq85v-}J_?+K>i}8*jyJdXQG=k~exlF+n_x;NL;<9ST^Rq*fBhF!k`&QjQe1hs z%w`GUX`26d1phPi34rkr9{gbbM^^yS0XedCK_5U*;t8K4-N%Lh^D8qPz_zlqNE!Zz z;elK>P^x;1ena=~Q~th%LLcxND?v2paQnd^f}oMpy&j=oY7P)JG+5M`kFmeUfKHM_ z`k#IO`+Lzb#EA(33uBtQH+%CwoPKD}&jtTy!iOLQSppwrEv*1t9}%?wR&gIxbfE!` zdF3%RaIW+3X1z=3v6<=H-v*c5f*$<;-9PZI1Bz@vc*2eU(#!0%fue5x1dT5ma`;Pc z1L7}JUvblX$+_hJVxoT@3V2sdh1>{lRnj}@2*>b#{?UBB#GD0##SMX({@sZe!UOoq zmBtXK#s|jaQgw~c8|!C;$t`O7j2heTngz`@Eekwy4f5V)&? z_rHP7aGu?7Ae7IWHAae-4pH_qpi>cTM(H?~qRCS|?#q`_x|cl@<|JyAwNxatEtb}L z&z-!)kSr`n)-Giw9t9S$be7N#O%#-*SzjIM&u=JFL%=o|J26F8OlBUhEJ~fk&?q2x zuVl~y#6b3B7zV)PoF+ssF$%W%J{>XvR@6JZ7E7c zNx@Cz?hHC^MOi%&K^Rx8hgBh!LlkVkq-~Oy{er>M%RHV-riB8V4sd=c~tig=gMwcN!21OuvhOA(h<5Qu$Nkc{bRG+v^ANet&UAwh`}pd9r(| z1zUvjJu{Kz_VNfm;@Nn_7*U@~(wY&KPLeJL<)cXq1(QFq=6-VXw)>B{!qdy)_aEvg z$mm^6zbKS)T)sMX+91p+;(Aw-5j7X40L}Ldh_rCq4yhtfpUvRGTx9V0A6?8^~P zb~6Uc=ke@Z*4+XZJD?3K2Bj9LA>7HNY|qZj$B14^FTv3G}ug-J%-l&wBPI{4C zSIFV|Q(305!jE+th#@l*M(A89cwr9dw3NN~RLmS>gRmQ>7#10Y`6EBRKOKw4n#Hn+ z&t>Ye4@%mfV`vNYpd+S-l+92SydY!KO0e8FfJn|T#*{9KcE|CtTm9J50yAQS*YzLL z^II_yEr-$erm6p-v1GfJyHIh}280wJ0ly&?ytOT3<12cISU9yF^CvR_f4 znEWKaQ~l%V9m!e{6NlR6p%})uZ0_RgCypug%~g%|rN%y9Au1CUqkb5!jl`HE=d@4^ zM{8!k`Gecn<7;(Ub8KJi+o z+vJr3v^2r_tQKJ>wB^woh=Zk0QaCYC&ip}%t*4Nil0vyZU)ojX6Zks052X!Ba2eg% zqPFjr(9E+8yUkhWuc-)3d_Zn2ar-r!0?2{cBAlpuaL{+KqIyY|RFdk4Zt)NI1?<0hy&$UF z$fy--?^tLx)#sg|@}@>%kyCYsB;N*7Jd?EKS=d~m)8Pg)(lQP$Ix$gRp;A`mBW6zJ ze+>A$ z9*uO3B0aJBllT85&y=Z}o@j+95;y?K%v9+nq^)h1>o=yK`MALZUnWacQqzq1{0iuV z82v0UbSJjHt@U;OJnII3ENOC{h~qnWo>0X0~6_I4e?SEsGVx(`Yp(74YoZtS>l~b*~#WiGU z9?quDxRkJNX#3z8I46g#L_S>1Jx{LUyh75AGuu@x)xAiECv*pTLp!OW&F#}cs!0g! zm_k!Q)+g!Q@w4QQCk|gk@le#xW%Ed;!O8%{`O4tE2YwM57Wi8{ed^BKV`Dc3;3O~H zouLy$o)Fbrf~u?~JL?^|F;6U%o4Wx-yv)X0W-o82 z+o-HY*oUbxGwaBAJr?$HIFQ7$I%7<9knko`cRX>kD0BcIln~qyUr4=qOgY5-kdO8A zy#M0EArsuE4*}rAkFfX*wZ?Iq_-H#dj{*PlJqj@v{e)G5PJsi*c%o=1(YUSgPj6UO zb?vQ6*;-7-J!Kd#D<@A%rDw2Om%LsS7}ED^KAWv1A}0_5ar(c!6~XJU++)!QG^=D={Vzmr;4It3__d6c?_L;VIIGTPvVLtfDO$%ERd>Cji#uo6|MNpW#gcH`Wb#UiU!9oBA^aus z8$Z#PqOhT%A;Jh*RYhb`14{3`M8ZFv&8`<{Mm9h!_{Q&3JXDMv=r!du5>a-s8M)7h zL*aIXkpSVYAQ92Tyn2#aR$unR*vjwylqRx!sCvAIbyss%@5fSK-+zj6wK2))|gHD># z59m8y6U4)DI0MZ?kV;QHp@1CsCe<*wX_}OXo1_)#A}JI)$%*PNU87P$n+BasM4gf6 z=b!9HZ?RH|Ps)I#FZBQMwU0ba!flDIbyvgb;|nX?5prmBVn}l@JEdN&bYd9X+c2rA z;wKI5JPhdRjaTUgp4{4!=kck%|NbDMGvCr`7bp?EocmWMf6(&k|EfRAf9R z?>#$}mDQ~P>xO!@+J8t$q6-Ok4FAkCYM;pFrb09cj-^aN zzYY#9z;|r4uPkd=;rl%@S)Z4VM&m{R==p038`qL}HMQqN$G%Neh*wUiwaQ-yJ0wtP z;M6AcPR81V@Xb`BsZ$XBHQuNafKt8wL8}knnmo#c$s}%ic}7&+9BN@-!256C25CVE zj1eA1Bx`S-bzR@Z(s?k#^Q_#{tw>EE!;0Wv>d@`MeIMfw3Xx#Rfbv$PDUryMS8-Yf zJ}0}<+=s?rj$({0tE4&nB@AalQ+lB1y(Zt{>96TL=(U6@!W}~!BJ!V~SIG1X^^`ka z(}aBQHHj5uDxjRS(_oWTFLI(avM9BYlwOwwq)Z%;GCYOgEgp10bSSyp<8wEJ?ViL358we%*>msmSjL|CXU%ba{cQT<20?@EEi)4lBb_?e3qsPHgi z6~Z_0CPcL0#g(2nljXM`S!haMMz33|IQkoCw-a(+>kXDA3wB(-a}i3S6Cv5U<}c(L zu6hhq!m)YZuMPoZIr<)i`r1GlA%cTES8UMW!HB;tQ0AbLpg3bTQ4E$A;&}ltB0WHm zK}OV937ertzIRObnIIw@JKeEY2y9+aoQS;aXvFsQ)H4gPo(s8a!zr3P0_Tie!TYi&|7`>PZe+iGGM@i z?`1w1>c8?alnE~hm~UGG5yb@;11SO@u7P53{O}KKgmwvvt*__;3_*F3BS`lY3?%|m zGBLJP_sQ=&9*GhIQFjCqW+eqS83LT0pqb$u;a31!6omVOsk|36l^%Eij)Mz>cNaF` zVmZDC1_Ims`_&RHfJv7;xJS?tVx9u;ek+56h42Fy14Ag3Jl=Q<_k9P z#;d<#MPUS&Wx(M8?`P+xdY$Zy?E8ejRc;th5hDkvc7!r|+mK_PZomD7`?sR1^cF}G z;r;rR%=+I$QsJeARIG0w#VOv1UPlYDzZT^66Aee{vc>6Z1ByiqA~Jsfm;&ro@&P@P z*OPyL3HN8o%}@h6BoenuLV6<6WO-!Y`7zAK_F1}!6eU3ZBoIgZeYAcoHq55JmHDgV zX4C)+rjP~O|04h}U5%F=sI7eV4tr`r-R~`$gA0h%QX9jul>DV^f~YgymkY=qk?x=E zOpm*_E5+W))TAK1C+n}GPz(_xwmhQQ3K%ACV|;{gPJgGPY9g4Csoh>OD?q}h3;}Rr zfMnpCsUqGfIbOKv9&LGYMkwFi$9rzd;0`^I*^4v<&WpzcD8m%vIwD>^JZWQC^q2e9 z^9@KBmAI{N0XqLlbOYY~Rx=nG`EOp(dyMAJ)C|W?Frp!xQ)7FAK&2d?9}Znbh$;ZT zu)sk+U9_3EDmY;LtC69K@T8zjSlyjo!$m?AkO96N4v#&qT0ngpv05J`XN*ZtJ@U4r zpy-rX_?jOcAs%N=eYfIpRzN#tPbXm<|54#1T$~+#4|%VqYljFv7NhV&T&V)c$=!7j z#mNUO)l!t=cPn7=9$nzUPw@c53TUz2W%?}&|DM~sr?3@VL1z+qw^=0MLPmU)&(@9v zqeBiygCqWU!qtNjb9%Ty!{F}5K8c0pzr0<6BplS~>XLD?-!GvE@gAesMT(jLJIuol zU8Xu?F3inEy>QIKQ=q&TDd6HWKsN?OyYJ7Y!YL9?{V*f~k&_)*3+XL_8H5oVxv`*t z_Q)s0Og6=`>>BiWq){hR|Z7VnI)eE*12 zJVA){z_}6*=j0%K&`-$(1z2JEmO?NNm0prW1U>N1d3Y1xA~&R0@X~Yqw-;;%5Zq4y z9<9O)iA!(^5c`C!0Gxe)nPbW(AT^A1x7K7|pi7z|#a~ACpQZm#@fW)b7jUER&scT@ zSLL|@mM}dHRpp45r_C}Rss#N+1w!FFiRvN&1nl4oH2A+~oIiz|e_h@Fj=9y#EYaRH z`A=9dfGz02ZiA@-e`&BB6kjhBy@bQ@*>p*oO*Uvwg-FCPXu^ZUHEq@e>`Lpf@G zu=LjkC(EO7or;ZRMx@#Ns|9!YqUJ%549|^^PZI-%f%{MPnqujG7(S!oo z?BoSRlfgtG$XKgijmSxDG^KW7T0FxC&Y&%K7_!obpNZCpN$}0y)3zC;xK(lc+6D zw20S@9PtAe#+x;V+@8nQ#^YmSbYv5CtC66{?a{pPJmM?Tq!+mGd{+xLA;0}^^yB&` z0PNF~S(P5S1{hWnQa|b+H3QrSoT!X}HPVPiji#z<2S|M^^-Jfyg`a=Yd{*77+DL=C zm;A{9yG!;h&kIdm$9=zjUP&)01G>JoG0~b{zO3RWtrVo_q7RkIj2DB%YPc zjb(c3ithwCjN+!dH(H}YSFMl_ihxi5xuU4`UI z&9ZPjE{^)u*(9Y7Cq^V-EP#6Ah%w!lrKIG_#-w-dB*uJ9@!x<=Cm$oD6LbC6h;S^3 zF!M>u%P;9MqL%u$g_A3`cd2yh^H2%H@p<8<2A;Ln@$v4f1s7EYj4XifB-p$QpB~M< z^K#deu(rmOgmBTRQU7iYe0F3RlvdT5rzleJE3w8YMfjJN?2CubqwWo;4q~i(D89yM zHpc&mz)x&m;8t$0G11Orb#gw=Z$k^~Xee=rPZZEuIB_9{W;SJ|CRQ^UXHiUA0vTSe z3r*hEcye)o6!rZ0WMZU~*}&U5RoGB%QE4?hH}*DXbZNYKSG+&_!It3 z%tl#AA(G%a$o;-NiGhJ}V))5pjn(3tq{qthHH-9ynyYRuBvmm&*KEYqYlxedP#DL{4o`HiRC`DXkx>Zuw z?Q2o)j#r)*VcOLzp!k(+i9%{WBu1W#blPe0{#yshYI;?3JieU`q5?V}%M0s<^R>uKB};8g$>Y6EZ|Oo)(|e6bG8UQ++1%K#^dafL4U2J(R0Zgeow); z52jDQ+MsxI(GE8!lj)_+>fZN(H!P!7B(LX+f9|5@9x%N#2Dwe1E=Ho$SV&;tfPP+E;Wdb?X@&34@T2h1YMDhE@!NWbvyGC(9za`*31|tJOwtj z&F>Wdo8+)tb)Ya2B zP6WJqA2_{@K||Gp`J7X8_A`1QtXHvq3kygn+;aVLD=gBmS-o~5Te=!bCusY{h17v) z=-0VDT)>6EWf%Q<(B>CQ2z_pE9GG{MFMOrRG9;VPE}qd1L#b|M-0$7fcP|^GdR(O* z#ZC$2TOloH$9u&xBP_}?a^?J=G{`i7YR;dFvW%(9`5Dyxr6td0cgXq{GpZ=m=M=J{ z#`%xd=Zb}d07B4)OC-77qUjX`5><(vq>LI=G_<(+R0GmAg{P7w9>wPf6I_u`$={DSdk_c4y<2n{k|B(&{hq7y!g2j*3|1&$ut) z3IaPvmK-HuDb5yu^cmRch?wX0mXX?)(r$oBf>K#-d?W?j%j@8F;JV1i2TP6#i~bw& z0<6gupMNrNAeTamLA}F=ABq2<-V-eFYdhm$@WUd=&t>QR+qeo28 za+V&CkjBcdVr?)l2BM$m!BuDv@Yq032B?LDL=M*)#L%MD^L}ceargokF;{u*E`mej znI;h==04K^?sy$z44qgR>T@``Q;5rnTh95LkAbBmPr9Uw$7te*@=%Z2gX)aX?7x_! zt$Ita@1s9h3YCo#Tf z_6ptL%xp3Kf^M*Y=W<^Q*gAwocsoBposQk0>T8nz$6fu<>vfMv*e_%6&+GU`0M67k zg1$WJvYB6{E3xF5GCl&@@V`;YjVzY@)-DlYWoCBIRnl1%kHls1VjOv-pFp1IIbFVf zH12Yz*bC+xxDg4Lg6fnQ35R~>Uob%rJWE>!O3Gi-0ho#vg3&jW*@`h$??{9ZSz|Xy zk}QM$fE(dY99DpczSZSy=VE03(fnRf=*THp*CRvDjoxe zDIww3b&QC+aw}^=l3l7hwBo=GX&HM)CXB#uH<&7S zTGJEm%BBTu)?RT$zF;|H#O}W%D-(UVmwia0#3Gxn!O|vlA+t;YfWrNZ?Ef@vfR<}j z;Jy*#=k7j)ZL2Cp(TX@PcyEl;5BBwy9IPHdR?`Q-q?T=BGML%w0d4hv9}Qs_P8z!1fLysadL6V#Nx~=!OD$kV2dsMC90}n4xt^V zgL#_GI!?>|-3)Y94je!;F(z3V-tbN(W}pE{h9=%0^!}j+)YOY*GvE57r9h;%3iR;R z^h1gD7A^Ht;I7fVTgTkOgd^f1#-ydo(rGd#88XPuXKjz(*}o0jJ=f|o+ZX^Y`dN9Z zt6rx5IVhO!mKHW32CUn_nFfF)EDzqIN%=)A-2wMB((Gg0FZUS0c?u4!t(F?(R6q!8 z&a*3_K2~qrTa$gHbFP*zx zRR|26LG|$aVpJED2Z->wfmK}O1G0b^MD7#BKAUm8A*1uR3D3G^Rn8CG^Jrj0M5xa> ziem#$;8=pkl~|9~&h(=CZ_f6j1d{n*J5p)XGuc`j9i;&ibse|0u4|4KzS4ZV2`|*i z!h)wKz6j{BsS=kFE0i4C+0L3~CWwrE`r z31V)DZ(%InMNcrA-n3uoX*=e@H>RO#S_0mkG90#I1Z|WQHhESGlSTr6gk#(C9IJ{L zAK1ZLzz)7sRF(sn6EI8Szh?=Aa{kv@CWjRWwh|I1Loyp#qVzR{Rora!f*4C!SA*Cw|#|TTHK$cW-Vp{F&!pm$0rc8=h2F15HYHHDu67^VJ210|s7O zEZd4=ylE#u7ZqX)pl{0}K*;F1@mld;=^vk>-*P(@@Lv~(-Ozy<0bG8U37(bcMajh@koPwKSYVq)Vnq(_WzFer~6$6oA1|J;cO zNR2Q)+fLbOyYbm5m=x4CL=7Xw)@B2M;)Fa0O#*a!k)8w^5&yFNMF4mCa`OhkKasL8 zIjuCRu?H;RwW_xfVvZPjx=N3YrXfz|dJHqm&5tWM*-D6ise+#m27AFXdKD+vJ9 z01T)BMcQqr=PeI_;k@D&{S^lM@c@vIS}o?kye>)#RlGg=`!1|ElI&_$qjmfKdc4_ztBkedp7!5JyC!K z0@b5a68J?Q05x17EO|#IGS+=98Ej1y8BWe7yxL%q{CGN>B~0ZYaX49X4Z5-1epX6i z-6Y8+w-M3PvVHpKW0MzJS`q(YDxe;#nz1Kx861Fm=rWl7B2}0(+sxEYbN*(@@=CHH z#}Hv0xz1yqPM|T(aBlJ#iH~^2;DEMBiaVSmqyb0H!O1w^Z?fK}*_0@xoq}(X1z&(HI_YS{ zy|pSrdClu4KG?+fH1RaP>$PbedU1Zr42Sx{r0% z;}s5#jW}|%*B80_?u=LK85q33%UtrZqSTdfF@|cn7gd^TUHam?a59+msiWju9Na9; z=fQNS$4b|jNbUf}a}YS>`E%=IG0d~pjhXTfFkwtu(VHtsrz@dBorT|bHCj8SZg3yi zsw&Xe0j*Uu2bykH5d&<=`WyDfJL}{?;x~_|yp>o&Uu_+GfGTPL5h*9wvoo%5z-d3c z)PISTHXuNhjv4d%V?`w#CZNe&{tyR^$dr;eOePlSfKiKs=>d0J5WoRjJ^WudWP3luoXRcyqVWG6`B7L_8uM6WwPJ2SE?;i`z`(%C z0M>x^K=6R|K!}d%s2iu}3)`+Q2@#M-$tnwDebH@X_;fTkXP_q=)zbxSA?M+DSyxwA zqrWrIAQ22SUlU_hQh>upWp}=hj5PNF^C58SMh9%u@-g>&ZruYN>j0Pweh(rItkffpNt3#g-DgT)>ejEEd{65A@xh zPVYMA$6O)4fq+^zAHNgjZxc=Mej39dWuW_3%)ZB%77-%1&|&}lXg7?A-o$4vob7%5 z;T17KPkj656;BKloSZg3Nd+7RYLCH$1(L`=Xm_cM+5Z6QEBzG|XE<3Fm?dS7HlC{3VXn?DtWOzZ*=FPC2V1 zN+PbFB3i0gMMUWF;)xhgK>#$5`+PVPyO(umy`|63OI^LEo&z2;d#FJPB9sZyv!<8u zUeE9qzzeOU#8Y^CbLqdd+h#yi;LEF<4j92@VY7nXNaq_;La0;)>fJfaxjikY5PrGvFo0c5dGmGH4>)$$K}Df8iKxof#=!{uAo+A9(uC<)qRtlN*H#IzoD z86W%7-6F9Abemm9YLy5qk8#KB9^hX98Bi^7ZSml7a>UUE%tf|bx;({f&W7SKerew} zHHh-c^~&jeAD-NcsKV`X8n|`bT@kg%2`s6APDBhoc+^M2!|E;0{pfId{jJ9hasUbC zL7Nc~czAwSG7q9fNrC92x3*S9Rm!5t0ES)q!GM1qFqLTp!ev4MvJhtDkOk zYBMW#io<;p29P5H?01cWsTO;ZW-%6Woo0L#K}SbZ95-Pjieh=u$a*?dd!7ZRT$**`(5wV||P&|gdk)4Z|6Qsjpp*c&RTdgxejE6}6 znyeb118o!WY>NDDzRKFjJ@zd2T5idh4o~>PsJmg};epn}v|jjLAR`j8EwR?7_WPB#1hn)YR@B?s)_ev!Os30uEg6mYXUp{&{SfFIb#cf zy&WI`8)URxgTT^=qBTB{a=9KDx!OE@5uEX_s}Ua`hZe;lo2c zvBdw9JVTyem%S)3?{mjC{amh;QX8xO?Rvynl`WAyf}$a|DphdWXk5$!X!o%$USvLp zBqJ7H`#gRc>X~-$!MKWGXV+B#N1NV=2{0t0c`HLJbJUBqW#G?cTM0%NGpE3rttkJV zz-1DKGbE)FwLP-o*f+mzpr;5w?_h^)?_k86uoF2u%by3`+d zxdUFg$+y^_$HHmA>T@(rPPd}ncJ`&4sqB*kkMZhkx}<_=Vs=!|#@ias8hsP)gc*9- z(Z;#3k989!t825pLvc^?z000=TzT|0F_N>QvO@jUC&ptrARoByq={jHg!v{o=)Lxk za|5g`8hpc!uwxrhi>9_fzu7ag_BP~MtxV{eEp0$DPJ3HLsZZ+oO$p?&+e@HkqAw=< zZ2zMKE)}b6xi(N0rJd0ai*NOyMg>stN!6RawRBuY^hJ239vOlQP>K46Pa;us{%A-Oi>YbD;AJi@n5x_faDruXbL6o zZH#xUq@0nrIeL?OUbo}RdT9Dr>to7tb11y0H{(TBLt;T$kETBxsoi-iSS1m z&9X5aF!Swoy>0oNYuBdtme6lGEkV0sQFlxYpfMo)mrYH0(5I@YdGA0UG z8nWGK)cf;ar0wmvp&*>ko%G|gRA3A08Lx@8$FXSUlA24<6_}JS=B`%Yre;d4I|2nC zORobTN^)BEpY&f*{lrZi3+~x`X0l6DE#5wLB!iTH z&6RU%REdx1m!p@m^JPbMt}+vK&NYnAUEGcnhQ4MWqDm|$zwHnQbOwJ;VLXfuac|DM z>`RZ_U&-FX=f2hA=ubdKb#d9YmdSJF$N}4D&w(MaT$Lo_xgJjv4!O!rM?FNaK{=YU zRwzk%;9C4?i-Z1QM(5>yhu+KaY?{GiYnP7B1i!`oi_t#NnBLH;#;Kaz zMP!}N(Kyf?7*AtR4dZ0=U+Pxd?NeB95ltk(N>Jy;g~(dP0V^R-F>xQbh>%1!qwG>V z4P<4M1o&X2HWRw6T^>(kecCtmNMfZQ&n7EaW;h#_hrr)YkbT?omfyPVuglk{8tB9s z({b7Biq>-!Z0cuAUG@$w?JZB#BmGXqIY3?pN2DHTb=uCw;NEN7;+x01U$B@;-eT5Y z)E6^ZVErAi(CvorBd-G&w}T&oo8?uf-_%r%w#5}xl?y5!2b8M&XAK{HO!7fKbXj4+ zmg3I1kID_Y9LmP;Nwc>E9G9N%FnWfa*=>9Oos;w=^D)F=hEaTXEuk@?7=lPB zTOe_oDWnR{=vEu^O-l?Y%T7{x`?-cIcd*e{jU%k>5w18_2KdcoYwUJQE+O|HUuO4~ zdKr)W*^ov04`$P$#RKF*@R#}H2x_Cgj(aMUR`RCqk=BXAEKd(u9&^Ys-VlP6!=P_Cx-@7C%* zSDV-TfY|ML%?8&*xg?i8(|FsmUAijG*iFzfao7NiC{5-S-6ibT1l#W|B17;l7Dj!#g-2bFA0m^Y zcnbG6>xV`i6Vf>~^X^SJb|$#fa4Tf&VV~^u$&=HdP4a;#G*JwaQw{&6=})CC?`lKO zZ3yvRsBkNa*bGd1$MIhHELR;n{{Aaqcuk^Fbhy~BaedYKvSx@d;&TZba{2A&k7}Kv z7+K1C4(%{9sZ|-fRoe098u4(JN5T>W%4xjN^khNW7*gTvvv8IoZXu=WXFwp+v|%R? zKe^25_vnZ1!sUgU$dgHTwHxNPl-P~8GFs1W&34N5imw&Lc>oNS(VyWCL0fTB*y(K5 zm|fKA)tF=_k$t|2Rmc3rPy#~D;fLw&QCrc#z7}CtQOe7c4*U?<*Kls`hZp;r04P^( z&BsVU>V90E$hetqFsv15r!ZyPZm74!79k_Q4mDS)H@RPU1ulQGC$^vqe~P1;cs4I* zvM;fqwE$x^~`yfZD7;|1oE_PC-qvas-I+$351y)i=&!cf8_X$ZlQp z)tV-!0m9trh12cZhP6Y|#-w-1+nmgJegQ()zOIZ>ljqx_{+YhNc&bJp&4|vUW~8wj zxdAOn>SYL0v{D8Jull|O7Xu8Jz=GK5Dch?*)wK~@q_8$DK?*|5S$FrG{RX}q1^_Ts}C6LWMJ_O=$wPA@NaNqcqfHsnpsxhgIdd-966JBnUr{( z0qj0pO}A~$7cSYA2`UmSsMnPj&Jjcsb4?{dYqkxCk0M@=aHKsHGaWP9*$Pa_w1sFe z>KaTNg~@6XsJ*|pvcM0{WNfhT2WkrDg_O5jw?oEfh1KX@nljn~s5hACSBnTA*%M4^ zuoK!mpCi{0o#ka*yF&Yn|2#hZj3u7tnkE+L5De^>v}iaTHnspTWdXv?+ZbSA($a3R z^n36V7@tHvXT#Rek-T5}X26JH=tVk59#`5WcHO|rPLTxC>$E8Sn8wZFC&%=4jTXmG z_C+@q*^6BRb{IPketbbB$Z%yItz~LxSrn`3YiJW0Bf+Vd+|u%`j6&<6qNJZJ7Go#04TayB?j62Tjzt zjYQqJYoFF^pwbn#91-bfEme)bUk1#}LONid9?W_fP#1?bGR#h5;li)-hie-|o?z6) z4;(lCxjCjZP0-}s2Q_VU=Dod@Sx6-+5xy7!{jmMHHC1J34*GmtVNA*I;c>E{d3e$h zi^6CUn{=Sp*Q$t^F2k%NE@(l52AwV>xa>N%ITv6RUS_7X^ktw^_yT9ILEnqGOig;3 zW6g#{Yf9t3{0YlLmE28GAlvCuIq8j9;b&{6Yck+0TA~66V+<4V<2W1H8A@O;phm95 z7vjf>t@cPy+lCgHfz|d^7WGGj7T2Tcz`W$!F<}KjE(x5;oqf7z-~D#y;~jxv!gzw& zVRK@D1-!HSv_>}VJ^RKm5jOJbbXK^@>8suO4Zs5yNB6|=gJ(5sHBvz%uoOo zx&=qP!d>cJzjAfWtpR{bJ6*LdIMn>zl9}=je_+x379*0wULI@|6QE*@-)?t4Z{fmj z_+VK&!*Bs(`nUj$WpxN&U@ovZ)K{Ox+TK61cJb;x|3)haLZ59ddk2qw#b&e^y))r& z%x-+~qP2~70Nx*sVbk{#1Fop>`WNPy6ncr_J|1V%|7KP#EDEf`!k8;;7?_6PUO?FnNHMpJ&kziHR16m*@$^y=A<6E{{CXbbUCuf|u`Q$f!|<`eMr&E4&L@ zFYWv!`S@FkV@05(4dz}V0`HaHhk1HO>Bd{0TxwzzV5 z7ee3x0TU2T5=~l#T({5~(`{_J>!Q=DI4s&v8ZG>64WR|t?7H!6xAXDNNoIveU&VRc zjvrs+@$T^7QG440p_zT@ir51t9)PBBzl6_y2cU5RL<~RK1e2EXb1nhS(o#}jOZlH% zt|7DOOYQ;l|ktQR0EaMaBr?8-hO` zzvuQ94?cJq!!8UiS-#X>=1L1S94RZgTb(Cpe!TMH@-=TW5bA)Kj~`4b{NL%Q3O4%T zrOUsh30QJs3|S^;(=f{*c%8Y!#V-kV`zE$Tm@Mv9fTy#_#SAfqyZ1_;t{bV_7Tpaz z+FsbNzv1BpdMRiaPlxuIi}mBQ`0(|Wn2GTBuKBLe{3m6zc6PxZm+Zz3UQ<0OPYMH$ zNEUP>1)<2MhG)xtnR^o-5W_#0SB3HZyAfUQn-dkp=fi>kQyT4ai5 za69n5tiX4HLa$ltvX*$f_w2r6P0xT_l?RG>+#37VSxHqD{n7R4Z7=az`TebvKev%( zMl)bF;`MCZ8Y9x8qdIuKHGZ@#ci;CVh|siJPV!c!7tou|;oX{V?VNH9*uT#0wEfK? zq@vp^+i4;%cfV#I-56CJb%Lk?RcqeQ;KKzZ2j=jDiNuHnkX+KsK7S+I@87_=fe@z^ zg2)Qx%RIfwEk-I{RO*QpPqDMV;9gx~085hrD6PmAFLz%S05HFs7oz{uX#W0jUpD^g zY(dy9^nPBFVppKV=dY%V8(IFUr<{Z%G2=IB{vznodu(NC(&*XVBpEj_jH-{Jz&(*? zYN#4ViO+dW@bxl9XCnZX=YzPBDf4?5e!2_)M?dxpTUthN_Sah7%zi%1oYl3~f1Yu% zv9i!#NK|}}o_>)QQh23v);;0hUe?Eljb^a9;67`(!+L`VE*pqqf-;=Yjz7^&d|Cfw zrp0hbfV)sKznjBXvaoPU$?m-%bVk_Az|=HK%Jgwa#jAfGbcLMNOc*_)lo{Y3*!YBk z;B3zZVlIXXY0KlcyR@(vO>N(E$_FutjQP`PvEoHm=V66Y=KaF7)Dxjk|80oCvXfbn zMCD;&cnh>8;;D0%dx86VK?`H`F`XTK{8O$lc952V7JCoDdT|e%(8$BwVm8M@oNLat zd8<;z-0Vi_|M8)5fEq|r0%8F^!A&&KJp;!SlU5`LqXeqiZB@i^W3Abu4;V+y)jXnw3x;rkJ6g)j?$b-u44iBBh&d+Px9R5Y$<^J*h zKY)k%?ny|C^mrIv#OuIDEd!rWhXsxZyvE5XPrGRM)Uv2I`se~k06W^G&<66D0%SMT z1=j^UDXroD)Gms14Y`jVxtlTC0}%*N;avDDK29I@yAW!~7(l4kj)J`40xBjq3Mlh9 zH966%Jkuu;VU|FF%g(No)V85l?UWg;liEFMQzTK)NyHg(bX0A!?FEZ}c$Y$xCWq3p z*sVs;V8?G%O_{SN#ix2{UP9T=>>cz>aYwB7!Pvv&eD_ap3|9A*5qMp|ru@sC{(KZ? zacwwUyMJI0zd3$%W#PUZAM4pRve5IK8x}I4$nF4($Hn8K<~T20|K#gB-b}lrz944D z12eRrC=RsPnWM2FiR6Io#2y+Bm9GMoNY66hDD!K0?y2 zMfbm;hUh2kr~vM8`O;*ki1JQOZvcdUwmfYUWU%wHBLwq5lhWgqf7_`4Wuh6$xM#{p z8IiHfy-Ji5e3fIv7rWiiE+^>ybUg0egu9fQYWJu#wa)Z+Y`ch}xDI?3VK^HuYDP=) z&eI{gVW;;W>`>lB|2VHCSZnx#$WwphE7*qG%Tcn~F=VvJOX;t90I0RIiV=gQUgd@_ z?apc1=aSN{WJo6@2l`mjqzi~%IFii=7P~AqoXa?!yV#|?(QDWWNT^VbtP6T`(jxdD z#Q*13fr+mA#!iNO)$fWP@3Y+aQOH&XXDL2wPPm%Jauf^~=aW|1VS!Nv$LcVAcPDOQ zqpUtZ&Y>)|lP21-$r~+m9YL_EKYCWUP@Z5I7C}(mBnY!KuMBLjYfz4~A9o%d$_m?2 zCe*Z$ENHOnA2-~bD5^hIARfFU@Lp;Qs(p$+VYdXw>EC&3H^8F66lt$p4DS}!6i2v zRgb-N<}W8!j_1ize|*uBLAt%Q{7hOY+B+qB&{FKJL5G9<@JYo6oZF>S8onP^2aHb| z>^81nIBhfTb8nsCmzt0@qp_(lTLa$3CU%JTg1b~bxdxK|K20s9(oDXZ6O^lWl6Vco zBI(#TqHfsc!uexJ-Sph0m?pVO(E6Cl#M`ND_b6-3fBR*bcAz!*wD_6CI(n(DJZ26H zt|cip8-5unQFt$4N{VUqE1xAwC%2+KPo;VE3OJZz`!ofrBvHgj9BuQ5$p5tBCanBL z`qu|BK_vb!xPe~nchu?1@v=dNR;NpPOUkeVHV!+LIPm6euiZm2b;{wcLJHF@8)Hkq zfM>=7la!UNo&yn)c9@YIdQa}K41+kxvtFfr7L}|^_nJowc}z(W?Bo^E|0r?{)6}3g zioz`3kS*`5_iXm?nP18KM&)-C?-wPeR$Eu^%P?AYf+a@58mS14jp~r^>o zdi7s2>(|94S$4+Ke6&9HkQ$Xe>SsAb@jks!;?s07EdF|L;_?LkP-=#mi4`qKU;o_z zylNg*BTIY+(UFecpdi*c;VRoz0d(FF)R0geA9=8``*j(+kc8$Qwo6|ekl>|7xPG=A zly3UCzs91_%xFoQcH6xE^~G=;HF4?xzyG^ns^Yma8f2 zXmA*v8hJYx(7;OXvZT2}kAej66TKBi1XCXY*sF5TDIrdMJy70ht@+yOktfpR>0?uv z6BP&g6c$qEMFo;igHlbX+|Yj9PC5Sc@CfcN?A?!WDX3UI*UtW4m0Boy9w;=E2zSk; zRWV-UT=I@2`p)?B$zr8}U2|(S{o>?q3VVQbNSsTunC1q*H=`+P{7{~ zqp!&rYqkjbBpt-gupON=z1nICsW|^@tJrG~QqhW*sNAGsEnY96OndefsFHuU21*!s zjyQ+B8E)>W*u1eouA=jGb(*gd+-#q_%w+!LoMh5C!&co^&q-oEduMoQrMu&m|9X>5 z7F}wIb3@|TltL72Ltko3I%#UjPh~t~fvwUx8nDTFtB1`gj*C2WwT2t`e+z& z>f`e!Vmbh-kIt>xoR5GWUai$fb|nT?wlY$fz6|aj4%+J6a&%a@W)gQGq3O34Kh6^n z7EUzZU|tmtN^tEC?iuR^EnU}-NGR*g@3Svo z%uFppXh^fYmr9?lc`v5fb5o=VEth`Nr7_*0sfrfaY4ck!e;VuzF5I&fMPhv4mL*%e z$9BfB^w9|*r1m);&PQ?H$zdT`>sIYvKNo)0WxlLn?!2g1KU#Mn0`IX2z!v^G+$q_W zhIKB?o#$+3OJ<*$DAP(Fl@E`xmLTFozYM$bmvud9K8S#lwFh>SAA>ZXJS#NIn8+|INGW8207m<; zRS>sMyCuYom;=F{R#aoS6Y!lj;d%lJV#yi2aU#XO(b%&wk$vQ~gbYiDAlXeh>p3!i zTrcoLp(4&1#7zD4j-$)+%EIT6=1}C?WRNp=@YY$)t~@LV?YBSLmC_E$vUAkme}13} zKZcV=AZz2?W9Q}%F6|aVa&&$eEwMViEiGE#c_lr%ho@rP+Xw0K>I5~PY&FHOCv&3f zAM-8r!mmY4hZRztBW4U>8!1`qnMK((LFpg~^Q@O3^5xB}-J6qp&9rGcfm2RDnsp_Q z_ibRkA?Y012^DiyP-W8_ronZa0W!+DslDdS-_RM7*>t%_m8B7LHs@u6a_>G;aM-2m zI+qjeH8gphi1n`@@9zkCq^2tKO_^tJ1$)59E*t@-x7!D-IO_H zeBy*-nMnsbvErjlLDd~$20c@f$tLA$<6Qc9n$C|n^;`_DJ8s9iG149UZ!_xiVTf;f z-<0wNZkr$j^?D67*{5RIf`oP7z!fX|yci4_A~&I#2;cm1?}jMX^XfVedta?>VMnrd z^?M#uji|#2yfR6v1+XyjJ$3!20^dtp_gJ0C0ZU;Ov zF;?Z8n3fylq%hiSmD;j-rZQPsIT4Rd{hJ@Dq_Ad^sXKIVm!SX@Gd3j@C_>|v_pXq`V{5@-x zquo%4GTC_^h|f;fZsmoqd&*BUX^k0c^1+a5H*84Qzt>94! zeR|k%T3eR)MtC-UWY!t^bxmfCl%09zJ}&` zY8~54C_*;6JVlwoaF&&Gm!CfysIp58l7~oPCF~15myZ^5d@fMPC?5kQ>b?7&6eR!S z=?(hbX|Z08JlpTerUyL5C9@q4l!A!?b50-T2y%;dIg0o9R--r|tM5NQ8Sy}#ibsd( zMCY!~W%cIQb(f!vUGEJSw~)~9)iv+%+qz#I?DRO*W*CzY`-knQ(8_Ql%YId#<$y8U z3)p@kW;;ZIvc!@~$@gYSubfHKOf$^}PoW#83`2iFwh5NJbU3N#W7WqRw&zmy2k)4& zKGXhTgb)btk6bdMPhz8qRuXOMkbmp|K9A>B z4(G5C@CRPYCUHxj*mmsM+uIa`CaA_-QWqV}Hxrj0(r?M@f74CvF#b90yj06X=8Iv= zk^|KHjw8?s?jrMe0j^uPyfM*VEXx!00_8j9mAO=S_i|({-cji*C4c^cX!>-}^q@6& z5M?y_`iD&N&V;i1Ndyh??h>V@mG7mxW&LpHUPA|eCbUGbR++D>9e(ne&|`y6aakxKQMtHg3J{fmj{!9@q=B`?aopIY10T z9lPvTfD`yucKX^;{4^i?S{_|K-kD@vfArk+CpTl~S#<<@pOU(1{DB$Gj}<&C2^I{S{0O&xPh|7TgA zWjV%4D}#`NGrI*ESiE8wd((K%P%iyq<6+7d@pND*B_dIh4hN3)L%s6jia!_4N(;!T z2(W8P;6?or2rV6}f>po=On6rCxByz1#*jCWZ3dj$3&U=!St5o&)uTD|nx0=KQ7BKLM4=)JncMn73ZVvKaL-dhr>oId|L3 z?gg2&Uva%^)E;AyS>jNsy0*0891SD4wvX6Qs&Q6uLjm+;Tc1w8tLS4^Oui*tNSp-3 z6AL8~H#@G_y|Ew(YYv+!%v94&bA>a>h)y2Y$ zE8}BJr~Xb{79<}7CHrwLSH%;BC_y~ER_qKl$j>kkmwd2f9iw{Sr)q8VXjVwQ;s%3W zsS|Rej|>qCja%(p+S+hGN^p;ugTCE;*zp05NZ|L&u2)a;ev91m$ok^JpBncQ6WH0lMyUj0$GHZ98aRR_6vv<>CCLdPc{bR(_` zXc=l1&@z+rRiiX7j0D?k2b&y^e`*gG{BZ6Cl4eDF zSy>NudUfK|`N85%i+R0L5U;;06DMq$WtGbxn&$bLjMOcN#SJ%$C$XjGdDEQIV0LCO zyz1A8n1FgBhwz=( ziQK_U0#atGU+d;<9gAfGcX~{AKSq13EumFa-Z6IamQ9QI3~v8#4HvlNZ9A_Elm&R! zy{jfm;B{(9e1ZkL$B;6y+G#`tCu`u%z?g(6)Hxo=sdr0vxZ2+hSWw_7cF^VX(GnS< zCijj|$0z0xF3ad)??O&$b}n{hn9SjNcytFeaCb|_H@V2kvhd(Bham@fLXP_@Dk31& ztF7Zko5apws{F48|V62vAFM0!7hNv zZtx$L0v6P~<24XF_*Yx;^CQdE!27BwhmAr?5ZUP<{7yvI`-o=Ub1b`pvO*wA7Tx@m z6Z&JQ%am~(`QxPwtijYM)-@m^@mM7SMMNy6_g));K04r|Kf21h(q|mHib=h0@};jbbI!ST@Y)b z*$0RRMe~)3(ATGKS%T_*@$PST69AW`%o}06!DEe`n>V1(ovJg@(WU%!=^gq@d!@gqVr z*{`h)XNy7WuozE;PhT=KPJeFWo-Xu>1j2!BbK3ca#2#BFsIVrD?XEpXnyc??ogCk? zcJdo(DWxHJp_9Y$@T3I((pjwiAV|j?c;PXnW$PT)SjU^{rqbZ~`P-d_c}q7}TcDH@ zSMchO(6>LJm>@3u!XsI`mGyn`KHHthuxEff9%VDia_=m$p1i4k=;rJbrfa`rDL> zQ`L9x)-`HZn{x1O>5mJ9$N4x*iMf5;<%4yuFCEOo17W=fP=Gv_u>uzy9Io=mR3~{h z$NwZ3J!yINQLmeRaxPD2s8Sv*iG*3mUt0U_NIYZ>je!M`E@Ud-B4`rh$zgkexle{CsS$X^Pn)iE}~~kKf<}N z0LM-WwX(N?h58DoS7B4>_gJ%;GP#j=W?HMdgR3>9~C_^Zr~41n;2WXvv5av z8(Jz$NE(Uh`9aG<1y%h_9Q0RM(+zGA9(wx`5gtC4_2}%~%M+8SQuzB>e|}rO4aWPP zRO__a8#+GS1>Zt6OqbOoENIoVCq|Cp-OEjEfuZchoQ-pX1@n26xruXPE#(hlN#1wfKKQxh(3J7bi^A!j56@7<4J}Nk1}bm(*qd&*`!xa=eadEjXD-QlKlboO zw&&Wpi?`PFu=c;Ca)1zw(bp5bix>^&F~0IMfs^#P-C9K21ymrR0fPPx!{tLoecDI$ zpGc2l{pU3Ao<9_09xkM{LDrh8w~$kA=^PqPeeiW>Pw=brcrAFpR_k4XXI-E?{ZZLm zc%c*E7NHEjjbI%&mZ*qX&`#>x$yHsPrTDWXzX&Dx8TkB3Wi#iOF@gH0i{s^4!V}+o z?mBu;t2AkF9;c)f!lcvdS)k4;Lj0D8D@0R$L7J=0!twK<7dJNomulJueh@E!^T#uk6 z9u0yX*}opzhm>lvS93UrDHGoMkzhe%bq#04afwbfZYZqqO*$=FCF^!BbzlUYq7D5y ze;o(}qe^0TAP;kNm~?}`j##rOh}09;pDLoC&NTZ}H1RA$5Cev z8~uHl${dVN?KfOH|FZt`1jvPo$%b9t96_U2wvb#vsNI9usXEj>?!gKp7I99!WbO8c ze3>Mdr)%g@Uq<>4J_=10x|{~nS_@@P-OBBIc3caVa0$pXXKLN;d6kXRA-Dt(CJW}3 zmicFq&{Im_xaz5%t~CH6YStM8%!=_fZwcP-f3&4UBZFa=aEeF#`qrGYr08S;N_ZjzvviwRj>>`^NXc$+iA`wmB?3*!mtphHkJIt? zSUMJ89In}d=b(QT=zQu|0O=-bN9PY*Li~9$@DAA}+=4(Ck0t-EwBJ=e;7xKG2CuK= zq%MY2*%YWcT98n*496z)&A58L$#8!tRKLpbtkU3I5B=!~d1;YTzw144foB9r%b#_D z5#9s0WmU7#a{`%Hmd29>r;G(P$=9k9IITG&RPQD>y|{J7W}6 zPb!_aJ`J=!N!IlWc(r?Lp_wv0tfMS~swn?R=vZwYp3oWGC|0E=E;7}rqRP?))pi16rV-A%5e@0D_Kp))&lL&m#3siy-4JQMo|bnwM1X>)EocC{ zw0-^bCpDdj6huYv(c7s{4Zu71q+weKv_$6eo{2tJl0h?@-X;z0Kf-^%Elh}Z3yu-G zDTtcb%Fk+_I9MyAC{vt)U*c}Tsqo~9qFXbq6tDaI{Tk(nYSjoleVL*J9)^3(I?7K6 zyCg-Iz#>Ve3Ab@TIFN%dyc%@Y%c`ZdCUS=9kEinqMP-==b?B{fPKa4`+N!3`1DTsH zuZ_}@N`s!bWUW$gFmw|leoXpb61R90I8|_!p%Ux4y)gWyAD zPP;>~2iMICYbJ~4mBuxV7LgBJnmvvBK5uAin+8i!a!6ZFRrku9?69-~^;~#nA{-)) zwbQ4&DkV`{Tl`M@6>q+8=9)|$-JCS)`lOaGBi~hRxT~_+`tn z**0Ab6IbNn-~x*AI(_5@s(7@l~r#EY8(@Mc>N{vt;oVXVTCoI>8qE@{8~%$_&OnM zOPiq&8899OiGtWsG#j1@?a;zdAiYv^1A?k)Emq{v#;MRf?J@Nm#SJRQ8FPmdo0KXg zV~R+rcXJ*`q;@!VLa;4Ha*C1k;%B+GkiM95y|1U&$hDJJ(e)IX38zOs22B|+!~55~ zrk#yhE6hE~vdy;$jDzgy{{eZw!B6(ZbG%K42%c# zQE35=rSV)HABS;ziiPT$l+{9=2nT~}7w6V@7252h#tSHul~%cGb6Yaz3RKBz zfEG`&FHgmf#ZWFyFmQdP*jE3py zvy^?=K$BV}uCw4u8&qI{&R10i!&E)Hg6+oBcbKg3Lz z0s`bjeE<5VmHD;geMPeKETMShjE2r|2j7O_0H&2AqelMN6p`@R)-J__S~yYZ#^7f3 z_p6}yh*Js4;x~!)Q#cW8jhY-ffplrlTrJBY5KWdlvdPt7xfr!CBj+m;BJ8e=>b$2h zn&4Cw*~F4(lw)HPXgJS@7eX4`i#co1Q#UBrq+@Q>-!$@yKzpVKvTD=ALcTcn#=9uz z4g%&^kg4GcQf1Yutt^wEtKs7pM_ZAhJx8Pl3}>Pzp^i=9|7-8s|Cw(8f1kVDl1g{Z z$*5>%O{EWVRw^Ne896KMj?5J1(6B_@mgqLqdCq4Vawx}gNalPNW(+wFlf#(V_kG{r z)aU*@?$3X4?}zu}dhGq)^SU2j9?efU57CGm z4GHq()aC;`E)!%!=uJ4pE!5koTcZ^B%p%>tUB&a@*x8;tgTn^DGd^6vm-X)q$T^NS z^SKAR)>SW7OzqU14mXTy=!rQ>yfnXXgwWOz31Bp5bYdXoHbrbs*`&^^Dpx~qKlJFT zn1W#_?-c+Tt>mbkb1NCIy!6qHKUjAW*MG_Ajr^7CFO$Nn98aZq`ObKc=lr_4RcNB` zAVj98XyuLiM`gnuHP|qC^So z`Dwn@O43Y!9Z{0py?f#UjTJmBDc0@78O@}@Zx%g4y(oFcBj3h9BoCf6Sc$ppZq}5u zT-&maT?}Jvp_AoEM?L%)+DEb|C%!wBvZrnwID9PP`|2t}jEFIto!Y#dsiviUUP6Z* zo)+r37w|7{3XD5JV^GK|SJ#-X&(`QyweF0DwCF-gRbHc)3cyE(Rue>>_E{7Si6fVt zZJ|avC+s@o?O4&4hSN(uyW9yMX zVXNSyyhs)|S-2PcGY`kJe$uRRz?vdT5IVv=^sA*+*|SfmsE)!i3_0?U5l-5Y*47;> z6}~YWKlJ)C7QTmjB-^zqJ56{&0Lh{HZfl6KpK9+`@O)&pmjeJq6YS3FB1wzz644ch zGxncMz8SBs={k=OBDx=T>FHju}@B2{sO&6w@hZKq?C;rp%Lk~ z&VDI_H7zH-m{IyPeFfY)w$nU8_0zOnM^Dl#yf`iK(MkZv+cNQ&+z`()b~b zR0ckr{KdVJh3iYAt`NZbToudU%`l4+_R}Dh=P+Ny<_f+^QR$2*FTeERY>y39<4K-V z2_?orDy$FtndzX59`smXBBLL8eCtnQ(;xWFCE*Ke+h2hBKvoo9a)lGiI`fY{+%0n5_%5W18wl@9H_TiKk0M&Wtx?Z*k`|! zz0-98dx`s`x_7B6-2z{U?(RmzT6WQ`Dt@MYfW;+1K^5dw|NASaChXJ133SS9tKsCK zthgXu$t(|@WHz^7`0N|e(n;tSj71<{l!q^}qJnPBR-_caINQGX5{6T%QLtHKnPh_%G?OCk-#nX+i3grV^%s(@iX81#$W38+21obNwb-k^kgZvm;fC9dw zaxJ{^aXZGOrMI(L&>T8ieAPI`{8$PW7s)8$>i6qxsX?3&n~uw2oy>Zopr_TZ3S=MJ zMb?EozcU10J|tN~x2P3DimSW7lxa>Gz3qer=2?o~GpztyPJ_4OFFl#vK=Ftl{JZ{k zLg2Lsn{;<#;pG~btQ~pU{eOHdb26`!m zPN8b>uf>UU|WT2bUevn>k(e$LQ?FSqlPK= zwN+rxdo@xUG+5~H94c^EJKOJ^;ZHm>VZqU6LKL+pewNsyo5?C>K)LC-TTdm&33$(TI8Gey+ zA8&P=S6>r30K5FgVJYn`8QK@pLT1ej)sQ+FcRL#EXC{s-8!BE*?@V}g7oOQ|v(0!7 z85WzW({0DuMGb#13)b@~nIYlMY~gke6hX>~%a-3;;pvbR0xN#RV~P*_rgmEXt{KWp zZrHSwn_uk_Yvp^}X3CltFRhgtKd<3m%+U(gH&8K9w`a*Y_bkx|+KESQLPp(h9~#_% z7rhT1W-47fSWD^BFh2XMlW6Z%oh-hp*in80#dqrCk}Z}hd{r0mI$FcO1jEZdoHi!` zGcL113Ijz zH3L6@J31y#{VDC%I|r#&vsEjq=~;GHlM-mt{RE~DrHiT$!L4vdOA?U7Ibm>KZ{OXO zcw3ZBrNOjaxXgRzcr{HE3z89UTBq)zX2dC?UaBt1F$4*^&l0RakF25$a~|ghz6(Pk z#c0`&epTcsNFm*1Q7`O@HlM~}$rW12&t~eU(@eqQxO|#513*i@DTuDfLu<6JzcK5;An#s#aIoMP)0682i+db zvWW|0aTSFnibLuCQg{brE0!!nSF}%CHs|^W?AG|(!Hu8~8*V+esA(=O?QM)P=Nz#< z^c`k)>ORwwB`M7uD~k&14g#I=Ew0NxX}~SFE_GKbP!0M}n#`Q<&5Jpm!(Ly^ygpkI z8O20SO4d95GPz;XHk&-@VpIGzks%;eEh7S%H|UfN>Ik|kLylr`8dsL`37BGaHQFk73fTypJ2lTQOP zT1e0f!*yhC*x+Ttg8i2~24AjL3@luvTLLjS5F8D(;$oQNxbs%L{{pD~zpHvcK;&7S z0s-lBu$IZpZZ_9i^iPgeHLEJ_H8Iz6UD1@MT(RWqTgfOT{>)dq%9)9#?0nf}V*CEtghpa?Z}&(vVB#M*k7Tq~e>H+yI|C+Hu!mMFXQ;>s0H}&M%M%=uFOW= zBp`#)7y*Y?n>kv3fqgLs-(rK2Us=3IK?YQ`AZHw@c86-|7#rp4cs~g>UN(!mGr95%zr?6DX>!q4n0(xq9Y? z1Xm*4OhO9MOqjc@=@)<`Ef! zXXZJTZ7Xn-%rTb$Wn26M3_S~-%U)+hx;XuQ3c-^5w4XpX9_m#$vW2+M`YHkWB7a>4 zK>JGf@i3QY?J+r)lTFlzR0JVGCIS-dgrj)HdnGVd?fs%+t;4fw&}maRG?*^D6#Zc3 zZ}dW<*y4MdzKVdq9?Y{n`;K*!dk8;5dvVHRjFmwMN(Mg6yYkz_TtnquadaYGdD8^)}PMHY;dmBELk65sb5dUKy#6>02lTLv*T=)8_shHUda^W!vdB;LF}jsWK2EVZK}VAm|9b%Oe*} z$xUZ>4FaqtXXi5je}rbr%82I*I5}zGd1GO1W3X?%s|G_4TB&O+ruxWJ{7#_Pb|yHg+}#pwsD1ZZ5ibt*7`XW&QPh#^jWX z(Hm6mp?-_cS6HxudLy{exKtYC}{>9L%bJqS+pIIo|`&Hd*kGuYX}@Hl&l|IvK$ zrtq2Nodj^u8bSIV7{3|FqVvNQ)D(pjQO0K$3IIPw0TY((XxJva8}DXoT0#CI#(KKD+59E= zRDds^VxP+{J+Zm7b!&98hg3}J65RvrKr~_)l}fLhns7;8_{$G}7)JmBo||-RS?&RT zYOq?eT>{VSc71SP@1)>u`Aa7gv^S!^J({))*bPHOoU~o*_~VZQg~ULWkH@M0sAKg+ zaX5r$fkABcAVhZEMFLxQ++Atz@ch4P|pLf3hf3A!BGXMYp literal 0 HcmV?d00001 diff --git a/docs/img/structured-streaming-late-data.png b/docs/img/structured-streaming-late-data.png new file mode 100644 index 0000000000000000000000000000000000000000..623628b1019d80742017fba3363a44680ba7e059 GIT binary patch literal 124617 zcmeFZWl&sUvo4BEf(3VXx8UyX!Gc3@_uy{9-95o|aCZn6EV#QngFAQf+1{rr`~E+5 zcGa4i#nepE-EZsD-7EZ)q7))L9y|mD1fqNJXL!M;Ac&nOX4=63Ez$Z1Hblk0x zkf>rf1bi>KAkAw=!IV%_L%}_RV^Tvwbwox{{P;ybDRWY90};dYFO z{AYS}mz6ryH7sy(!ks9vL#fI}F6`i;-qsIpN1!UmYPX8CRg zmC@aSci%}pF#ID&Dfe(8BL+RbWkAROjqx~PAj)@lo5*B-8;^@HK&=7^&9eEW`2VCo zvFMsKDp-LEyW;Vx9UEGR-{Tvlbv7Fn&}r3F3w+y$K==|&t*atf)ZX-Hns_gR;aM&E zI~D3z6y5?rbGQX3G-#dKBPOhUfFSn^R!rMr%===@sFs5^G+t%70 zbq5wJ%L~WHyHMd4P*kwx+aH%FQ8ey)%zNfVbyp*5mo7Tpg)B>@XMtqM z8w=GRQ2|sVcb1_+zl%uMlw|lg^?!c}x&gVz`!kUlPr4Ee+<$OEPmtgglLiKEpP_hD zyY}yS$%#L@poiB8&L`h(t3LT**$(D+72@nGw*&pHO}^q^`yuajAl&d}U8idB*K))p zdelYyMcMwNPAlZqYCG_N#eKr;kk+P6w|pWzYcny5^X9^^_pho2c6-|7ydHQ*u}kH3 zOno>?d3yYh$L4x=9bR(;hTS&iIeVrtwAU@34Nhy!|mf(5N)=v@! zCE-lz0%w}T0<9uCxi@Y<#iCyy-dQ|7s!(q^e%nh&a`cVyFgco06#2Zpz!M@tvEc?_ z^+DKiqu&Yq54ZovO?N0?i(l=1YRqze`$|+5t&{ zzGaPHbp%NB@ZwoeIizN+{#$H^UwmWwLi=;SR%b}}3}*D6n8RwWsqqB+5G}HXdA!DU zGcR!?c>q6UH{#VL1{VzZ-!zQ`avzWqCQicM#K>7|9h?$Px^rZ$+D_!)TlBK@jx1#Q z=LX!ZjtQMW;4KGRGB1rL8Ct^JD?W^ZlUmfPz^}eZlS?ZCsqMNfKzlsc;OyJ!ve5r# zaKZs(7nRaqEaP*+HahS{PsZE@V&*xdLDiTfzCBNd?v*KoG(BWDzm-TH)7pPFj9t(f zKa&GvlB1%ZNj_S>dFcvJm%FwM_j#VQ;|+SscTz(dkO_kpQh`|evVU`%;NwHV)h!Ae zf`V;tAnEoEFWE+t=58PM{DI>0wAGy9l3spJ3dr~B6BK{b(d9o6jywgtU@KYUZy)iD zFYR(7k1Tj_#F+ATCipY|G1W+oXlJUw3Ge~|sw+~hd++NA$q^Mfu47#e*&u&MHFx}~ zgeqV32~+Lw=@;s70@1H^Ul7%um)!I(e)H_ja`Wu=T*;7h!EKC?;{8otU{5OSi{gLa zNi9B~fzzgA2RqnWbS%MS7=`XImVM&brnrOg7!x$4##o%a{|-6ZSU);Zm54tVaG42q z`OlqD1i63HrtjTo{vW5I59v5v(I2?lP)JrXHV+0$ZM!vn;$}$E2js9*JxP$kuW2s@n^J6wNRR$U+9Knlc>DJS7NSk^f2}SWmBC_S*dji*2 z41hJAsfqflJ*9Xj*N8g0jPUm`r0g9b1;02>IOaG~Sf&v1Gv5}i^GrQ@`^2`nwV_93 zy6wT1c$+CV9P-x??oJ!vN3yhpsWuk25CZf4RVu9f#^p6^98sTP3RMtt*Lo9Sa!*0Z zy6>CIdj2bGZ=%pl^hQ15V;NF>v*|bI9H}-B*q7OwB2fyveB;6f=4cR5>cMIzAOEf* zQyOj0k!uf6U(*Ef5f@NZ(eX?=1>qwU(HhOoZ!U>ZpTDM@O3lyD5|htSodtLf2iuxo zYlbeu-*m@+9@YO74|Za!xuw;0)z~wNVmq}&0r8TWe{-uB+|SusGHlX?3V%0oy?r3r z#Ad-ZC$%=AE^U!kd@GhwTJgucTP_>Js`^c}HW+FDw#2 z*50tpcVTKG#7`6CQ*Qnit;SE^hPIthzNwucisTc8d5N#rt4?K$C8 z9ap;osN2ju`oXj`H1TK7`sLpfA%%%UKU2(*K(a6Gdz|biUpY~`i zpU~b$vGE*wVz6Aq$>8xjAwA69NL=cBfi(`mgn#)Q?;wAV0F0Ep z%Rm;Lx{9l4t^H*NF$eo4jJpyU_thl(x{dA_xU@pbGr=H-kTs5*xak=D+HU}UIk4|C z_o(IiliXh5L+uq_qG7QO*kk<+i4(R%3rqc^s??H|DOPR&pX~PzHVs=Ab7)X%l`pbV zru!I9*ZzpE%j1QCN2H4sWrvG?Q}mx6mtTQo>EPq{66wvQoU^u#TWr5i6qJ$_&1KSG z#Cvu(km&akjeYN;cwzMYE9fhM1iD_x&F{t4MP}!2$y8BM`sX&_Ci%~OW z9bn{U^nRE(viUg&evnxcKtqIGaXdDg0FFNX#{hQq5W(EMUdM(*VD4bbI2-{XlapzO z^x>04uP3kli1s;ETog0ptxMhLWOHH@!A0`FJ$5$&EZ|r2wU$~ol~EobVVrzB^InWB z^nDm0_;O8%9N(U2Z5djGMmdA#G^t5G{E7w#00lq5+9LZOvHbPM4eXs=L$Q|skoP}b z!jPoGpk{wv3#P<>Q}f?~)PI843iw+KgxWT8|AR03M;-Pq7#vEWoPM;(|COA7edrJh z_Qh@%HH`l=Q4nz8>JK9pVJKveRpxKx{EMz`Lh!d%fAly0=luS1^GO&i1||~hj1h`| zTu^@+?7b@|fjuc@GV}TZ-zosAnN6n#ce4EA+DOxgQOch%O{3 z$aFs^h_%{4$;0J@9PKmz%$Wx+f;z+Ndf&Fm;!*Tei9X}Cqp+u|Epu+iQy>PN$&Fqhw=`nPj;z1=UY?$JQ z!jX*6N*k8-C%*FO-R@voC#StT$u^{AztnCci8}sJ#6N74&_hM>k4K;!|5FM0MZe57 zb^Su$UQ6kx=aN=&N}$-@Tgb`gBc>Iw!)Mw@X{otPRWjWst}~sjV%xd+=HaxpUdOr< z54u$xQ7d{`=RL6$qY$jw<7Yn2_@oDkv6t>?aa?%<{9dHFWLX3wf=Gx zO23`(AMKa!PU#-^%1PiD^Bf7Nv6oqyL{D&{{Cv@zWJ^v!DYCIDXi-4m@nS5FKPX#h z*d#YQ_zf@cyuZ-eM0J8R!Qoc4=;{R1q8?-i5XgEy;iFY)NKo^86rpunzH#!H7hIfr z6A2fgsAY@PZc>l&<6cfHXE893){^>TY<4Ksh;2^W;+&NKQY1In6}IZwl5#}v3!c#8 z96>LAU{_H-|JA`Nr>(4b@Nh*bv+P)-kICr(;S^BaIP)g1b3R zS+Cil+WVo0!g9vcze2Z7f>&?0m*#FJ4R?P-X)yIh3_be4d;vT}Kp0XbMS+!sabdC9 z8^bV5>M4-db3`%x*(*rR!@YXO{S_4epBd#ko9ijui0m!T#H2$XYVaa`)CsCsy{bfD zX3%Fe_tU57&*VxngY|@EKr>PcZL+1x*uNvmi^%ll=zmPQz8jW4Z``lJF8nm?0QZ5B z%27(OVE$n>Q|+g369ZEipgsLhwbaR*h5tt*F@&L_kLJ-_ys%L9b$RP8=`!6;hE@1; zUS6PZXSz~@gE1Np%QS1qoV;?aO>ghI3L^(|@TSeadkcmQgl5^c-O9DFQWm;K(IHWB0kY!$OOMApIHDW5;)i&;^D1ZW&o!v{OfN=6$c*23k}@}jKSaBOEH5vg;6@db zASx1LFNQ>FWOgtF_idx%nA`h0$#J!~lYPl>ZP&8SH!0lShi(jiKAFVZLZ{QLuH;^~ zR=k;|FlR{Mem69+qImG`zd}kJITUV)v79-j-(Z{g=!~8h!=0t?4RCj5|5Z*LjT? z#fQR?A_$2Dn+BprlxaAVPHb>qgRX#vrtWa`P(cq{kI zOGtmY$+;@C9RE)bsUosYdeohm@w;-TY0RH~*gld(T{VRdpyM<45H|-#u1EtB8qG+@ z=hDhAjvH40^rYUw-FV`}{I2VSoOOg8uY_7*7Evh*M{?z6h&#_AG%aZO%r{o*b(ZEl zzc1^CDl4GMifV^`okXY4@;oSaA$WxI-Lxp35qn3D=Dsun3d8H@fI^_&Whdu}ah0O4 zaK=`~AlO3q5rn;VV{}`7vrDgU)t%N7M+$CDoC&V`%q+5n*_Wc6^F8q>rY9f zPF)u}FP!^U1rHeB4%*Sr3+i-*k~>)ZubX!QL%NN+?mRW#Py_Pp0ha#U=JjkW1e@aN2paXOuAowlGxm_c%l{mTGvzA zd>%FMVj90RTd)=Pnzt`ixkf>4=1sk9PfU8Vo=dX(c-PpIpGja9V;zk(?>%d(Ts65h zD3n276DA6T2!Y6mP2nb<=eaV|TwoCufM z2Lkq#kbaKcScXNapYlJNudYp|&Dpl^zVsREa|Cav=HeT&Q7Vkoplnq(bnxDGwR_L) zCgylZn|X6g48i%hwC&BQ+jaR}KX920`QuY;d$V&4BoGTFQ0dur0jfWIG#a&LJ^JxB zJRSR?1_QS$V_%y~&}QyM5!c@$`@M^G!F+7`bLZ#)3{<2;1*U0y?e;WRZtC%cVZfFn zFI^{y3~H}yhw3^01x|HJN^rtsa>m<-uD`~g>;mT=)4<@vK67CY2}Bp=d|K-W>m>=K z0qOV23M<1QEgRc@9Zt@l+rPiLOowdU`>DPm(5SKYwpusoDf0@H%(?dMK?HeZG(I>CISzz#6(`H=OP-YEH#sI-Nu^$NVDbRy$q;+W}iN69Ki&b&q(m1WI#hvsVcyI(6)`yb&I9VJLN*su2~p_17j*6grUR4 zKbV#ss&p9PM|R4>Bmj&}^33U;lQqTsx`9XWp>!RfObylry-|}^j6xuFn_A*$p{P)0ZU;1ZG$$BMzG6y7bbN!q8X!{e_QBkxZVyMSK`L29bC587nLRS~_ z%6&B-T{47FK1QBWy??Ey0+K4h7W|A9KSvMOW|zyGvq?@^NNbtufOc{@(d0KRYzc*S zn9zQB<+B|qyXiHS{OLHUT}so57wd@b}*&ElEQIQcK0M=P--nF-gB+4Z#u+y#G^32mMKqFCtSBiQIT{k zT@pu!!SmQ|yOMG&tWbA49-~xQE05EL>*ucjho=(FB@iY+UIoMO+#r*Dz9~T>bRg~h z2Wcwow?C+A7m!**>`#y+B1I8!^WINN>b4`3LDO8g1#!oIh>KP<8%?L)XA=K4^nq*R zwsndfh886=TT50}){_??b-IE(txW8p-ubSSh?i7_yTaX~@msq$&u7A+_m6dJJOl!Pm?;gsWXet8-S`@Ei&nH85UcCET_$8u6AW#0=8H z5Yl8*Wry!Q3Q<{3%1TRezOu-wZOjfCd3}C-1b%ZTsPpPla3KXG{Um}=-H{u&prv7Z z_YNH-X1EnmV^;2ygASrig&`Km)m~q$$`6tC;zZ#I@TQ;OfgqB;yZ# zF^qpW4bPnHPvUdLRxbnsW9UM#b(%Ro;y#y)8X^kE09)|u?V)t$=NBpywGEE!Hl}LwxaW2s{0uk zFIiK#+B2+5iIj;jPjFWP*!KO*wNjV1i^W7s)xaR!WukLMvU|JZ7A;X+nhN6wEDCEgnjUm^GPl?#sWZvCwvZ;>!|42h$5dB-Ht zTu#Vaf&2^W4T|%|`>TEzaGa`N6=ixA0hiByx>CTvKWD=)`Q6H)+nI`GmMeWL29vJ6 zHbR;__9i{8=bTP8aAavZ$SXH!yFA{Nz6P4dK%ltyMEbMw)Y zim}9f9MLj-;t!3dOVnH+@WNMm1a{a?1Z^4l&jlRV2UX>S%V4E=inCpSy@I~4SsW~R zVK-vA4c#yAi69BVsia!JnMY+N7yI~nG%`tFP8!9zJi%_g(C7a^CHF`HLK443h7O`k zK-p72x6F9G%-xVllL3|pc1ahvm2o4Dm0|%b#TbgduR#Qvk1c>yX7Xr3FGhi4$Ima? z;)apZSv4IJYDi2PvI*61gEBTI6|N4w7=3Jw^V{G$62E_JUn%usPT`aTw8qM1jKi=6 zF}I$5x4aze zlJuYy%14#)zoqx;OgP@KH(u$3F_DhDB?mK#4Rydp=$l(r1`2f|Js`n2FJiz}a}_l^ zQ>^UWKn4?UC*(*noXG+Edu0qRL|@Uf-XiND)|05LAEjeI?7}qKHI*l6=28@nmuC)p zz&UxLsMd}%(sGi@=OPqlVeip9mH&3eUV>TtCYP?k&8G2n1NxxW_olVW>O&DgnLBWEIi>=O9& zgzc6`ifAz?7A0)e`Qy%W zZn>N|a7mZdbM;mQTms>@R7U+a^0J2gm@ZcKIgEC#Y_cEi6kjqK_+pnRnjzb~=KLwm zSqX}#$szruuc#|SHV-t)#kzakz@;vywRve8Jp-G1N05+o4}cr0W)^49=~YE2)s~TW zKPu~ev#ZI|SzT&Uxl>uJ+=$#h%DGTABveP)Apme#go6@Sy?5s=Ej= zuKY_ux_&=bi@0#2pes=rhEAcUYrJ!qe@C-ZM^bN*@0_deBs}ao6>TK-K)vlMS55=4(ZkPo z(n+S<=M$m$=zFrX-qZ?*{@QQ%Ie}T@`)``GgBiu_RTcAlW2>E5 zunI7BSaEXYu6*oPt8qCRkMjtN5cBxv-^YB+Kix5Z=Y(!nvOdRua>I7>0(a!M zhQ2|ERTb(7>4zzZ-ihW51-uE_0FNOCc$37rU#1m^8wEv(_Mqw3=?ji-bdhrG_ArV` z$!S`Y$oqNOG!ik+NbgqpYSU;qTv^)EG5Fq%7=)Rend&j*;7k|w@e3Zzc66xyBu`Lf z*Cs=Vl;Djev3WI|7|)JPI>qHgn{+Ld4JMzsQ1&WbU_5s7aSM7uzQSIvT(5hpCJ+GG z7qt?(C#mJ9tiS9W%e!bhpeU2Gb!JdcXrCrj?nb5n<6SK+71&{%*5C(uq5sq6^=|`= zx@*&T9t%n1UL>Couxo!-KV~}CS*Lp<@{$?%ernSL2M4b{zQ&}o{PScgp)P8TTumy* zRBxsRs_3Mtc{eQB!81j2sZ-_+B9ncAd{j^d8KPhs1!bH5ZhhB~I=<;~CGv17vtOg7 zhVRDM04SbIFa=|!TkKqD~0dhrHFLuHoSD8VmCkK9^dyga_dOVI?mC#zq=>C&~k|8 z{3c78kz2;t4(-j*&h@O=JjchXG(I+o!$2JXB_Vv}MM)s7;PI(h5qjtDSJ$~?!Ud^O zx6gz$|K2ql(Lfdm#RB`ub?9`q1HLG_vB1h`8vZ*@PeD21$$ON=c$&vdRJO zzb|U0O7)yA?USDv`HXopYagyZ0nsLtkQ?~Ly3Jb z#eN>uM28HIt5-=;o)>;oG=1cuwi#Rr;r2?Ysi2-8MlGy|)=jAy6x&Pg^8apB#Bl-C zFrwEkW5p|%Y}|K#Mdv9OjWf{XHIuvxA;pmrW=FD~=GNf)_;j~yuzSMnSENMCqR_o^ zcXEst=7raA*|bVxit>&Cj?{TDPI)Lf(*m~5+`+!kBUbeSkllb;s;YO?#WqgE`@F<1 z2GLP(8JZ*}1ZinGqed_%$l>IQFcq16>Q$-RgMIflo=z{)+*QePi+arTa_f`4={9_L zjumDX*mT(1qf+LE4JZ1=KjNBY`A!1XVSq!q z=Am~>9nHV^jNORuV_%}JVI|&37s{pE>RVLO(X0f#M1gP0ZyKjv;C6CzYs0D;>%&xw3S- zwb51|7~uYb(g_EgNLcXI3kzhwnpt6Z0VM+Tz=(QA_D$Wj?h;5bd)NqN4ZZ zdQf((Q`ne+81144$!kR!F=A)3-}afeUtn^5n4PhixXASA*-t51QjIC$T1Hrx?s7ZH zUhLZkXyyo7ypRaur_GrQA_wWRe6K)qtQSA_3stVxBr~AL;Jg`$q@KKwA~{!0&E+$Z z+IKeSPL4W!H}_euYv3-Y+Hy%vr383_kootS#fO`f`wHyOfdlgDU*6E-S?&Y{m2-L} z>A~urwEBc3H;+_lGK@?-T+e(x(O5ZKrmvNQ?-5%+pwJHA0D}U_4?2X&-plLsa}c5p z7<^T$X8z{wl<~_5+v&ljhrDpa?{xts#a2p2U}^>vAKQo2P3ShSJ&O6gHhU>`Xlzcx z$O6C-o~Wlj6S$Cc0V0|5a>Q|=w;C1x{_ul}O z@Hi}mgh9$dG@ok!Ru?Hw)gqbMNyo-0yrDrvHKziF==3LH1|(a3p_fP#^L>x9Gx(gC zR{tb8Tt=$~UwH+Nk@sKYo6kRzOVwl)$$x4xrwtE3?9VXJcsFL${8kG)s9mbu1a^HE zurubqx->8(Oy!s%7b_e2!)Qz28y!#1-!SMfNh@Eo7~4W)6Qv6-s%?-;JfbrvUKYYg z1V7#~w&CC@`pph$dEehy{^{5p+V-Nbs~{_}(TzlXHvB@&^RVdl?V~L&y)ph6$LdLO zECm3cZQ4n+8zb;5eqD903jHF+OF^j)*S_BeCsvG!8yZM+`~=UhB+RE3%zHQJ-W2?I z=%gOB0GR11^Kic)WS39pRyC4vUC{IMv`krQ(6+5T=G>OYr7b+Tf$F$$Y0S18lAmyY z+z&fmhB%`&xpa`IsYcXyXhTI{$mwbAsO-RYpY=*)M5tw+&6dw9oFQ$4>IabQ$~e^f z8}*(LAzN@Rf9{qOO1$pTZ|FHDNNetS+vH^EV7P9rHCpGAW|du7Pp}VOA&I+K)K|gR z2e@)#54*1o`%U2CZ*r8V_U+D~fbmGw5pZ!YTV8~CVyBkoNH9_(DSgaaMiQ$RU9X_V z;F6-z@j4G~jO2Li*tK*@NlhN?B_+)WY4NT{FT0LxgWFk|z~kVOy<*1V-DmYHnzQM* z+Am`o+io;(WVfo+J$^nh-N_wfZyzw=dfiSMZW<%^P@#&cBE-kgX|Fyyovob4GcC#p zqAe`zQBZ0ZkQ!-=mhEnfbJx=Y(BfZ@rXC3;JGX|^Gf4=}dvD|%J*^xr znBMSxAK&EiHkwFUherW-SI5$f%n$()yit4KH6_93WWK$o%GHC=-$^237;}fi&FvVG zl?JM0PiK>e*f``BfQbd+mWr`IJ}iXRMc0$=p0_WeeAmkoJ!4;9hVD{QzW0JPQ}cKn_!@2ZjLUy5e(iHiTsbVm zQskSY(N&PzjIqS1YUL~1`bx-uS$qbR2*yqwKg}+d%6xgTpQ^S)6yu(ItyIlcld|J; ziSEt;cP-pP&AtBgwJ+$PUaE<^5e0ypoG>?K;f}2DQiP#$9;c#D$vDbzs#4ufMiL~X z$9{#;+0kyE>PUy%*%mq!I%(x>+arpCl9Jh`!%XD9*44Sb>KEtPi(}o@-QpZ{m@BWi zmGRYapL98%yf7D#T{1`&tlDcw-fT*{6gw|OQ{!&OWIN4a9Fd-nfu3H(#y`#l{5SEBUXX5**7<#h#6-Z3lY=e0KzK zRPIm|01ryb7BLc3rOy<_CW^aBeEBsRXhZTONR|Z*MvgnRE@*B)m5X}nok_Patv|{2 z^09N}Maq4(rO!z5mxNjq-nbl}Vdkrt4mHWrV7Pt$guwhIfCiUm=U2_J+Peqs&uPq+ zUPBmju4cff9z_9B|1HNkobn$sl+AYcW;I{h50UK=<_oxBHkLP>H{Wa{BcS=rs)V8o zW?!}Y`Qxfj&nrPq4mlb`#h1hPG8XfOWt+>^hX& z>@|84m01V8KIJLJsaVx{M!m5)^X=RxDC!U2tTlh&b`lH1_~PVDUS}MqG!Slj`nYw! zDLS|RhNlZi8ktORKZ&b(?!Bo7!A^btE!~X`S8q4>sWZJz`cHdhH3^3<$`a{dVvTS= za%4M8rrQ)Mqt>)%q$;j2+NEOK&hPY;S8 z@~>`Rfn9Z*Ul^Zt?N@6Xxov!JxkQgcQROyRpp|igB)1eKyKp9qM;@}@<3;XWm4T@9*G$5~_2!_u;L@Ig-37sxXj=+@s z*6qP066R*Tgmf=}Sy8IlN$e-!gM$Y;#s! zo9I^WR^OZ6&`$ZFVc1FhtmiJNij>>))T^Gg3NwFvgR%n1Mf5}Q#XTER^Cm{@A_mTD zEE9zsV*}j!X3ctu{mW_2hrV{s2?gD4-4nW_Hs2P{>&{v{zgsM0;2RA)m z)W3etg!e-`v$3(2x)OtWG$iqzP$bXQ_*7Vih67xe0xJX;96xt~5a%x)>Qj3hy1w&H z1w3Q1d9hA#2%mb42LeH(5`owo7y3;aUSB3HZZ)j;Zr&X7_L$z9%bn7@+ej9S8wfBk8jiV@*53ypdA(Kz46-ufL^7e~(sQ#U@F zd3&J8M(2YYJRbnNHJ3N_sZ(4nt!|%)i%YS5pN{Q4P9C^+V>;(~@Dp70MTNU8(r#9( zz|hlBN9+l)3BxV zW243MD`s?Vvxm9_;dtYg+;lUCMfS|y>R&UVG*}cXDRlbbzxcrN| z3AIS*c)QoXamje;`Oh~vMBAxPf~=Zje&j;bpU-c!*tQw@!Ina;WtEp;}P-}e1m2i$v`H24&I7kn- z5=6K?JNsU+qz3->^m_5i>4k%EH(lzy(ee`+nY@{TkFr(cDY-S*S)6Mn-tvq&zh<)u zBIj>@DjnMz7q0xdz|KC)PL3 z{R#;*%8zKKdu9x_ep;z#w9HOb@Dd2s$QDzy!B!-C=SK|mDJ22+7DuFSy+e+IzUW^Q)%<%S4BPFTEs|HATYdA}Bw%snS<77fAl z{v1X_WZqMsd(`CWQ(j=>>*&D))-*H0YoJ>wdD2XpDtf!%yDm7O5Z_$U_Q3?m_a3$x zC?#Q`4N3Y%F)TGW=#dcm)ne8%7~l8BE>IGxQ|Y^0=;u5|@K%C@;&;@%tJ%|ZJC(&B z3aX3k#!4w$pp$d#q3^F9th?NE%+_{N1u1rB>9YgbyrFRRK4b#CO*f0{>q8|Li{QLg ztyELdO4#N?GBTzmisvlV$y654pn@mUc%~4?SRk1$DL!MCZrN5Aw7`T_#VjAu$uODe zimTjDhb2|HNVX^bLbJI5ziCcYVXI1RUSsEk9YJHfDg_Sauj0baM5!(M+p{y45Axs9 zUWXXCq=0@47?Wn6f)gQq#i-!bfX7%-b$0blP;jEE8jcS@5H=^-y~3z0qKr<3x0bx5 z8KLt%Q1tulapWx|l@HP@obG%@-XXoii@at=;2KCP|0b(7enFdMs#XqrdWM>2XjTtrMcP!1IF#uS@^S| z^Do1oN4(znBB6{nhx8~9n*%>;?T+K+w(@ISR@tHsP=&hXUQ=jH)<+dv0?jAqJy~Yv z$rfZn9+Aeg6?(?ZxzxS|Y55ifAIo|L-q1on`=r-9K}vJCmjLST&^Dh7QP(s`cgLyi z1V)D$`M6#Q6N8b&Z=1>|>c>sE&iu}2fx6A^GE}j+c8QYW)6w$TuhEW- zMob1hd*K`=^J_tp@ZUY^))II*Zt~-X5|Z&vF2Wq0AZZoZ?Oj0`c(HL-)_$3^i1BPm zdjjAxA#V>{CX^b1j>eoUp||8!RI(I5;uzrJv_z0%gpwnM*ttsm*@p?>r~0G`B@ag7 zIZ2Rf`3LxiJmlBM9WY`~g&jX#3nlTP$K0@Eeu(WYKM=vXMLo)f8%&v=--?!8T|**< zo`3b3?CdYl=*|sXN7m@Mk+}cb9B~%9(PBCBy9K^{yn>UCICLVIIl$bMIlsr&0rN&3 zFdkDmAY;CXsLTCeSMNwcpriPPV6VpixiKc#SZ=xBJ%7vbs2Ea?*5D;WcWh^GGY#IK!uS9-T>vVNOX>pA2+u~B2Rdb)B#>Dq0e02h+R1glZA(h%M z8dp~8>XK;ndJnu#%8A4O`{>L-Dn#37Z}a+NR~KuZPSb#Pt{0tW{)!ZX&fg5)?DnK_ z{)4CaQDdi#IGHPYYn@ypr6DsVHeo+e3AJU#=Q+_-(gWUUi5maeoNOokB3iAl=d5~_ z?$VU;tFb>+Q``5CN%B5$V`m;A*BAFUYaOWe&v7~!V8*hz@VwJ-bVc*#el~2DTo#0X zg9irg3%c7a$T59g!nVDhj)q?XrbRqHI{(OXb{&gCPmX*shqCpGZW5^XGI z^AfZe^l)Z$>6%AXYgeM--I0-<6$Gq+vE?^Z1)bkzrKfjG$g4i(H8nYQ(xpWniG8Ax zh5NkY(lS@nFl4`O*?(4)A^j{_Sl~_v&GJ7yVXr?4nf*+gH<=N8ODrbFHLCkMhTd^q zY&li&l)ZPejQQ_p0kCfDSr>||;(fDNuG8_-s<%;~0^Q&Bm+u;jo4a-0kQxW|b$54P zzot|znutTBGVAbmjAnBo$9l}7#2EP61&>)~)S7H^oi8SV=VegaN@1x4uHqb{+5K*2 zPcxRY`huRRQQS@p#28|NzYw(8bo?Y7C!2#iq>7lUqD2j(L6S0Y6i?+u%es9h`dUSm zY8GdNS>Ies@_N-Sk*R2WH7d5b{qAo3xMW_Ek+zCxLy?fl?~TJH2?hO< zr0UQfzRODCb-ZvRMiI1dHLss9Es*Wt=(of!z$okGdjq_2CmN&bH3+g6zq4qTJYRz( zM(}$F%qyd2RCwpSWJ*3$hE~01S^h@n5<|CF>>Ke|wUX>+Xi<s{#q=w;;=4zpQZ07Mfa%_FSZb{1d8-3g|w+V!1BF3y}9F+UUqVCIp+7 z_+m_`E0KdHEhCNQ?BoIWLH5_QBOzi@Db3gI)*7V`iFp!YHW@dEvl26mMR^YCDyS)a z@>Ln=4Qj&M1D8ZqLypTz?mvlf$4`e-HZLphT6Lg<%gH@QrBXwHc3oe)(y$@ z=kX8Xh{ue~PQj$D?n@QKh>a_{pGr?Y}w_DnC9iz-swT8Q_q46t5_*yt+LrIkT_A{AnVIV5@C-}V1a)Wj( z&U142%{5T*Zy3agcBJB99*ig4CWz|5QzYE(K$^la1dHP9yUagzj$~qDeF+HT@LX?;IeW zSy#~a#*3BYbO>|z_5mtJd6b}gL*wM08~jLGv2lsE5oB4PB=x|H7twHi*UajV_yDpu z;cHOW{jUS7codCSZxiDWISE_<_Kd@4u2SRl`(k$c?@#kI;e*W5Dx(f?oXp9h`qlNg z&U?cx2=T0!uY7e`(R*hK0cpFV?CN3z1YL8d*v}2JcCElfmw2W2^;YNHuU4VD)eAVk z<8RK^L~?u{g07F&R{8K+LOIHo6Y{nvc8_;56g_olob=|)#q;BAe{P#vn=Tzs6y0r( zXEI8cPr}9b8ur(vBJEu6YA$x1$DIgAM2bl%k-d}A7SH2L{IX1d`kQ)y>3jwG2M+#n zsb;%T5Cy400~-Pj8d-nMkL*2$xgl_&I!1|N`ff@7v!9+ExqkSY0pF{mbg$HMWS2iG z(Ewwm7MrEM)CR#B-wi@ML^8GoMVuRW(PN+RI4i1Pa?}b_#o+nOz-P`4$(zL+)%zpC zoAt2UMLRoY> zmpP0b`zUyz>5FEt^)?1DX$k1S5pPx1;)KqgBbo1^B&tPC4 z{9r3fW}~x72E5^n*uBnyaijG)X{pZ019X6P*Ijq~wHihtYT;_Dme_-JKkuQNn458b z7x@H;v7-&|+)I?72y|l2x_Jl_y^UTxK13^?tRKRTvsjZon_0l-543N&sH+Q# z{ab$^?e8c;Tgx92Xr9*%XH!XAwZezrW-qeQ!d5fsh+r_bJGz=)#dXt|6Fz{NKWWFs zHjPhTU5N^tURHu(u5y8O4zI5v5O3y=Au6v;w7Hxi=nq24y4jGTpr~t1e9$8qe^tNn zGiHj5Lo|x(AOmVU?gVa3V5tLU_{~eK!#;Q!PM92030<1kQB8$Of-LNNb^P(d^h=%4 zKbKYR;3!Vs2Dxcz+_M1yUM$1a^Tii7wUYJ+Aob3wOn!mwh$NHC_q=NODDe6R8k!O1 zctLh$Z;d2XDx<9d|C--Hnr>KB0#@{K3wsV1hl=4!Qh(4lJjcqw@`eM?z6Qc*H4H3} z`{6pi>8}Fn=)yy)V5zvSTYj=Gx#XxgAru#H+f-MhZ0JS8_rqb@vsu%?B`|8e^G$?2 zzPKgS`}Pchv-fHZT1xNq+vy5P9D%b0lkHM+YY!n=Wd59ugs@@WG!gvfo#Hzq%t&$) ztWeIX^QCDMMK4pzKMebFelC(WzV zv~x3e>t;cOv{a*GrYH^KGq`wPhR1Y>T!i@~0!IftxE-E=qo*JjoB~s zlVPz+>uvTZ;R@czO15?-H}c@&esy-XGuO)7u&oas~bt zCi7=CcK9Y!)efK920bri*^t^o0!fxC@PI)EXN9qbd*kWwQw*LFdb>-e8*g?RJkZ7J zGY98L9eX=yNpZ@2h-TYi@a#n;->Uz52Ahua?qCHg^QbbJ&;Bqy<-9q6SFOo_BlL2Q z&`Xz>nT18>zmKe>jvJDL4?8C|NeCW8+CIua3qzJEJB@T4g;ef zPg@X+moL)g&s6vZGhlSuB=-RWfCFRJWZGEG(&o* zo}c3Sr6xT071?D|=gQ$19T7QPX;Q#8x?W=?&A2yFC)yfu&WGZ7@FOv#7b1y|cPQW( z^pHv;NA^buxc~Wl41dOeVCYg1L^ufaw^grNeJh*jtu2>SY37p!@ zH=?P8+}Vx0*MRf#ZhN?&+n~bG8lIe@=K>E%FjB4WfU27oNYy_hZXEM-Teal)1M)ZP zqjv^PDwYV`IfDo_!EqKt{aFN2t8n{A4fk-0o7^xY9#-I|Vj1&BxZbckK z{7$;yacHGErnL0Zvu{L%q#vrd$RZ8=Hd!(sk6U#ql@!v*bZQp$l4W-bDiXaXIjNU; zK9RN>J+Pw0b;TD;u9z10`$^PD3!q_`2-!dJh>zGwC4KGCUU=*NC7TQ@rD$GMS1n5B zD9vYNV^f^D%(zbC7+QNz6SUrvcVkr`fhERpHYFM8>@Zfj-}{eUg^fOd;>Pl@FG|e( z!v)^6-4Pg*drP?O!O-b8@15v5T@0@mUHx<(*75$cX-CBxpJybY|A*;%RGyk5ZmlQY z6G`gF;X=Y^i0Ag~D{!lTPj5tnh5$Ji@CHeQJQ33%&u8i7s<_8Pu>l8VCXP>iqVOU} z(QKv%2RnRNRrmif^%X#IHBHxnK=9!15<+lycY+3Yx8UwhaCi3v4;tKqJHg%ET^IQ8 z^1NTwU$sRQvfSC8meXgZyFZ)u@`KB!a`n*}w@`0>hkao(_ifs?@?7#|;R48D=z zB^&_ay+X>EhL5Ja9!gAOzRGy0v3PO%0=Jf~DV=*`p$1rq!_xUTS)%36Y>)Z{a zUrH^zlkIh@@QV$=0kP!Cp~|tEhkFV9$yvu`qSF~b(d8h;ehxvF!(N*s&l2kZinYEW zzsTbmbT>s7meBA&Uz(9WHcAC9vd>Usm90PZ!PM?NS{@gb!-|#0-rtw!tj@6FoV9@x zuUnr{$%Wmny933Br=!UapZxzZfJQE(R+n5u%aT_%uf#&*%T`ac3sEDQc!T8Zi9?;ERWmC(dzKtAc<2J1?XtMK=Ux1 z)mgoo(m7JQ7a+#Uw4_nkTMPbc2=fzNowQo2A#F$}2$m|H14z&^ebq!7%%pCTgUelx8glOyVh(n>Wy5i!VqVGo zC)vH$M?6nMn~%-xiYn&KVRswyorEF7oQI@!%MN^p8w$9b%$P$Gohq~)$Hl0_QY1W3 zXdh3}SuP1-Fpv{+zt;Z~UO0%^13WtL+RpT|(YJ_hBP%59r6~6LbR1eGu$NVM!}vVh z#0*MPn|zY~`Zd{h4=9&dyPv00`gZ{m@Wnk7xIgwxYq4uGzcvplY@^t9D5o$8tc$#T zE~=D##jj)vXX`)9elH;^rWXP5LE9X}9Te}0J#?wr&a8iapWtnobFulHBR`|1-d`Bk zf1<@2{}%wkoi1N-Ag3Pmbyls zvt?SIX`oZHn#VcZ+tObWMfH7bzk#x^sXrthmny&02OfiRh9~+d2c-9FO{P2?K9pdL z>4nDI7X(WIl}%Ah*B(AV&0N#nMVR=uV}S>o$z6N*WSt0&#$~qz^9jPr)xxZWN+s=h zRI7xh9tX}Q%MW8wEqSj;p@vJ-rQ7`#2~&`d_Np@pT@B86Dpjjk3OgrL2uWzgR@u#`hVkTz$QXI)fk->!+=4vS!0?G5o2=AbLC2F4orsrdz9 zUSVHYw8c)$*5T|t3k4#&*Aq4}HkwT(lBb%u4%LR8+wcDB1sv5Ta>r0K@|F1uugssd zASNBzXF~eE<0m9S7&oVm_5E-L-|x(zNiCEBwB^#%N(-PA<)e>hdBGW6?8fR^=_SZjIJ5B9pu^|1A z`^R;$WmT%J=X^v2=~Zcuigc2nkB3f*ZzNam+CM0&?Q^qQxGq(Ai+?nub2LpSe;C?G z`j{ajeEQv4qHylMR5ePh1JSPckZh%AEfg!J8N+Dg8+nGx{ir#Iwr3rIrhu9Y$g#80 zR)L8Z<$L9HAAerMewsC8&n8UBbqY`+`8AN6%^}g0-0|!|D;_O}gh}g8l7hXS#R02w znd-}Zd~X_baL`xvqdvf7!3SOd0SYI`%Kslx*oh0_NKo^K-x%JOXG()+c83At#WvX& zwAMA@+HD)?3#*5;_R$vV3~H{06%&UIQZ-1;%mprK(Uey^Xn1dfy7Q1w&gj1T{NuUL zxxhwcS8dJRL@taMsTYOzo+?^}0UFB&9#U44cld{p_2%~e4cD%Y5Qbplpd*k)*U{gQ z!nV_%GUl)l;9A(o=(jcMM_mLpRCiYG%H9;a(S`!VYjl6zf}?ehsIe}#Qf>&FYH~Lg z)E7ul%%8a^w;6dw>H*iFBRtpcjP4t}qq@8}!DzUBqeM7Q7o2B|QbiHSh>*QTd6)GT zs72Qnp+5erM(s7F*G$ijhefZp94fPYStv9WPC1!azjHJ+<>qn@+0kE3#p(;3BI(da z=n)}U3DMRW8rYbrcc>e^zZ{>E-RpHd4;X9kcnSWR7L{poS{MGUFD$U@SB_M@bC!;7 zB1>UEVJ3@+l0BWZ!emjb;;a?^B`ru|L$*Pvi5E~Uj_dg-O=`AP3t$P7J6NmLW*wWR zEh4T~tsP9w3RK37m@qErhp46Rek7w2a8T|S%OGzHJm=DXzp0rmoXVW<{c!u2*59vN z0uD05{3mL;ZKs(Tr@^Oap6-U9NfM^T4X(uj#o>L6YiMO32HjqfwZ+WUju}`d&CuB3 z?|i>kKJpi23Hv9@haBOIjVL)fvIaXS-%`8C`^g8Hv0_HCEozB*T5ilw%2isTX;0Z1 z8#}|A7;R9O9@WjZn^UZ#LgLwHqIpZn1d04qh*~5%VCmrDa@&W^Z_fWI)W$tO!+&JX z?s?VbuoHf~+$NFEYC%FuN_v|(cbYq>!N@+NJ7Y4(=MG zUPh&@!VYblzaac1?5EAMT1~s(9(^>gq!4HW0+3;nT=w6eUBWc$pxfcjE_!(`d3kEr z9eyblgvSMtguAo+Q%g5TH@bVK`!&0)2A;xiJxY_U)VeoH z3F7Ca0kzl_hSyS!JiCX>%WXchj;TTKomHZ*7pH#^`_YV-@0LN{Q`|qDtJ<}-MEalk z+xlf}0B0@8(}Za}mg)g=HIHfak&X*d^~sL+hrK{o2g7RADSo5iE@ym0a>}zu^h$hI z>-j$_yRWw)GOu3Iye|XqFQ>O2KQq=Ij4oa=w37*pJgboX?fzxAp)Ud=hy*TgRyT3M9%j&p^+pZi}?=Z81B*}L>UV@ znhkVxHEy*%R#RFIdsvEX@uXN1BsHA*mRfKnimMNBbRJa?Tgnwflm1A#vY^c zFq=_+eDq%?E3;c%!4T~bL%uF~42w9inLS<^aOD{i?#YGJMnD5{II*An-_G(GNy>OC z+aAw@Q=zH9kUIZv2}kSg@p9AEr0sf}`1IM3z@AjlWr<~z8)P6o8B38isd-E~_*?0ui%i5Y?P)rJVWF_;B<}lMh_wH=dC0-e;wPI)n*# z6+W+9dqoJMK1Q|6w(z!BK{@mjwkcv4^R^6h*g!iTg;*fvcvg!a-v}cX#fO>uzMdrA zu~)$(renQ65kfF_ah>Kk@Xs@fm*o&OgZJCVpgza7!nILv4sOkYqJ1irJup*E9WCggliA5lthnY`8$#!8{dWZL!ak64gxZN0a~Rvp};*ky^4+g(olnpJ)2MY z{jJ+0-)5YQ?C&p;mf8Fab>mM&<1ddexK5rOGF;IS22({#PiwTAwx}`APuy3JDjZDHbuX(aCN51}ze%gp?>{$$=nRLin7x?HB9 zfz8)RQIXj}a&Fv($U-l}R#jVF9`Y8JLXVeW5c~X{l723f+SIWc_~Lse8spYr{&363 zpM|-CW1A7cWdt+_9G+Ug*uQ9tt^R;&6hx@7exyBIx|1H89d2&@v|a)@JhPLdyNc5a zwSwPw$14oj_GgQ&-^ssdPJ1j-xq2Es@dAA<>4)C|MMhky+ocC1J6~JS&c$D$DOf!*AdFd7UN=~rl_2(6EV=i9&S6r_CbJBGRj1vnx9}*d z<#wR7?#N_*B@nS%tsr)HOpnSanb%@!%wgiLaCLI;w#8&|dX7#^PA?{h^V6JD+v4{O zP${nq=tta;$dy#tfcN1Vi+FRCt8PZ>m&)230%SsdY7t_B@33QZ860ul648SuR?KtG(*>3;mb7Y*)}B?=oNX+Fa7M&SkRA`m~iKPC7LLn)$-q6MDW zGJrU*^tYP}^8RVPd`UyYKD&0L{jt%Lk7OYpRQKJ3Dc><8;+Myt7*@=W_0Ra$p4*pR z^Ar8BKPe8!hu9vtEY(hR`1k~D+g(S;G>g2>OIjHpoJ*9eTN9}@CSfU>>OHa3*mu~z z=rk*Pr096wB(!tf#>bJL$}`lq9;~?Lq9Qt2+3t3#d6Rv+lj3UiZ2Q&Z+CizK8|DeW z(oN-ynAXT>$C2`ZBV$e6XR9c6ZT^#{?$W@4Z?i6srCcO{D8}JZ|7zV0T?ujNCqt3# zMcWb@e^igKMoUm6D`-yryJRzbNr8=@nj)!p{ge#c;2C{ZE2T$I^E@WA$d7 z$bDnsv-Mck^UNLdPWM5L#~|oBbHXgvaC;ZRb#Onf`a{8az8E#qXmtG@hxC1kv)Bx9A+a)`SU=oveI#DMzwt=-ivoY&Kd zwLeOyB8xLy6f2sBJ}8!nAF!_=@n#$lb>LFWUVQ$CcH)LrqosOILd7kEmyZ0 ztrCkewij4Na|mF&uCLQ-73;q4$jCa?wrzcu8^T7B@qO8GeXt)xY&l`hO6`Y^iT05T zM#Ljwkart1-A-pK9gJV|iKVn|SxnUWD9(FP9_eu{B*I&tcu-adsqy#}GkIKUX)M^0 z@k{8r`C!IK*Gidb!I@5( zYNz`({9z#+IdYtW0AGhvgnRbPX5`EK`SgR)osC?cK8L*26Vc~S(c0eYpGldRH36!a z?we)3)bygY!*`bJSYwbJXM++oM!{e>MXN4N&5FC@Y`owLirvjBTT$&*5*P(Oz=*Es zVZVHNYWQ8h(5QKKvF7_^5++tPu)jySPITj4aD?xvL$gKMp8Z8oG$*;G*<0(2UPU?R zrQeWe^FqWiINZcw%CX)ZJ^1YnMpf*uU zS9V^dO}PZkrFRQk&qIW=?J=4{NYDoM!s%x70S&8AKG`#cPN)4Vt4gn{o)^&2tGN$_ z=b`ddyH)%4RDh)86_m7cWieN306UqW=gaXd4!S<5{ZRH%t!~gs&S;x2TH+eWwSX2% zY;z8U;#6-|^?q2gQw4Gc9-2R(o-Jh5O$TkR~sz%QVA$a+Gubc<}Z0Y_MClZp&K9 zSa{}tW@@=wZM{ruYq0IOT0I*)xsyq-tXQshzgw(3GTz7^-|oo^Y$)7|fP$UBl{QrG zU~DD~TFoZaf!)`J37$r~=LzQ_s;8~%tpNvYC$cJ+-TRdZe{9PP#;NuAs@g>~bElQU zsbJgkCb|0QTWJ#PJ^JUT0{J1l#)A9qUmX$YCCvsCvTS6~BOjIzEArf%hZ))xR^4_4 zn4bnahA;XF={{%jznsVxz zC{gP7V>qDGs)v%d<~u|t=R3PPAQ(RHvb;fBL+gtbG=>yNTAAo&-=>r`S-52r6O(5X zZ4aoXh+TzZH!!#=#TH9mrn?-&iQ|w6^J4zC-I8YT!^kVkX;Id=zgF}%hs5EUwmh=y zYOLUeR7oD?fhTC|`lXd~rCAogf?>IJ19IWdONtyo?{&EKReWQKi0z~?nuC>v#s9VS zPf7l-o=#36#VsDAQdkEZpPbeQNBNV0tZ!$@-$e=Fb99D92+ zSXX%`-0%EryODpji|LtY%aDnj48)w}TAITUrqA1u`Q!0znpCngZrs$>Fh(;b@lP?P zi|iY7|6a-q{6-q#H%}Ai@Kxg3v_oA{x7{fv-rhG5=ofb?+rH~)$u+rj@q8ULlm7eY z6s`QF%_gVqHZ+j}rNj6j`BCWY6NUvmTAjU4|1<___H^F@C4 zSAg=#{Cy)*Fh^`1 z)<^1W4e(niHbmba6o@3a)_B-x{)i6 zF2}QKsBw|lK)347^r=Z#y392(T;q$2Hx9e|Ow7B>wL7hPPdwr&}YPkPHN2ij6O5=d2K+y<7^x$Tw{8z0^3wF&}nd@PqF10*1P5yQt}4 zdo4wGMeuDFXBd?ovdR5>;L5Wdz4`lwwr*I`rZ%8?Is1zkp!0qksTGy&_ElT9T2IUV z=5aYD$??5VDMLC<-}_az13%7bee{KJEr-<#))ZsBo1bV8>c{4;&6ymZd#cRS(Hi7I z3hgu5x`(XyCze0{esr(s3zyObPfP-J8y32oq>xKKm>$KG<&cGC0s83CZJ!6L}^EayVI za)Wov^2?5|dhc75T#x6OwoMpXS=BxeIIO$a`!@QJfn;M65PZEWrl+SjfrK65RLBrX zx6HQu#E=^e7OmKOkrzt-W~{fMjk~Oi`eXK998{~!J0CUfy-)%ORJowdF7sh~&JW4g zKTyN#n#v4pH`C!UBWT31QtcorP!TC&sUZnZ#SYkwP?X7mi(?N2 zb1e$F22h?{Q9jqN3+QMCF<9>fj;|f4Dd;)&6nm`gmD#uZ;W(6+l_j8d+(%(-?+vdJ zHGVG52vX3SV9q*Dmj0cE`J?x-?PdX1f7}b#SYD_AtuSLYcGuWipcOu zayuVlu*;T`D}z?w`Foo_)RIQ%wr)FWM5j&C1)lP@Np!4i$N0Jzn<%e8rwq4at*r0t zT2?O8>%yE$Ao0@L$$69XlM@_?z4`tO{%Bl@pyUHgV9!_*f zh1<4Ci)T8gAJ^VxcS%D^VE))sX+>4HE<%?|uPmH`;(oe+z}7=up=H=_H*7(?rzC?g zD)P6C3?R4pX-9ynq$#GzD_6EO*aw`H0M2LSynF9XR?Xlqd->9pguHg*pZY}y|J|&l zT8s{2CE~-QqhI?V`&8w>w=OrGiozC#n6`UlyIQ}T zT3GHX0p#`w7-G-FeTR1aeb|nHy~rlX)2v&4eWTWOcRv1?@iaw;rIpK0Vq06E<`uoN zSRL*Sk9&f4ZpM2gRmq3vs}cuX=hYyHIl`{1u7a`6`>VEw!K}L%jnn2chNTBKpS_j4 zoqHwm0X8*I`wU_X`A1@sgwsX|Wo=cx$st9$+4Z`|zD4>%a)?g~D!LBvk9$h%l&wQG z?9FAh>vUxPuuvo@cU>vZumopWFaka9WCD+3+1)Wc86+6Q2ncnG&bU7|B^YOpg$bU7 zC|2!@2e>$29*pCHP{WNd;C}uP49Q&nlIpyS>d67B73mjf%QzO!!fWYMl4pq4+4`=> zcmjC+1SNw&Uv^%`4@lW_O#7H~(J5v*zEXv9Up{89dMI~{TO4K!2W=*AOee_0YV*AQ zhGw=JV4^E{pCNSQLx>1iNE8$!kv@QnfR`PQ|Ka6ihIcR%?kP3D-c^FLT}Sn9q4Bi) zN#>+TZLb?L45v`y!3Y6gDuQ5W9#cizXP-BDnIkH&!A29!t$kUJJS8tj8L%+e8V15%dlC50P?ia z;!46!qfzu?x~(t+R!-ExcxBk}g`ecz-yb!T5zbgQy=uBZi(AUGHZ_5V?Ws@s_(7l@ zpwlpb=}ku%#8OV{5Sx|H0wO9BgCh8oM4pH-=GBd#gOMCwbrOD=PixR63}fs0dp$?+EVYHFS?57VeAZN6>M!+JET{CTT*DvLQ{LrJIIKQMVG9eFu6n( z$yoo`GMbB$7R4frv1Z&TJeiO;+)NA?ZT7$V1M@=Jn{XE2&U4c>-`WC*^{=1a*1^>hQndP{|Q>&+ErN%lh_+1 z)p;YjHx{QOU#_KwIUib4?-F?2GMb4n%|fj_4RH<(`{Mi%V5i5vcz^tR@jcmkMN2{5 ztL{19>3f7*=5e7YK48M_Mn-BW;r&8b+_%aNR+daIGi_Y)CvrCLDOG6Q&bm9H;@xhs z;bu9$NQC!Gbq&cBpu^T7etA@p5qfL+5X%gy;j^ZP7i9GG<WxVo2tmtKK&ZqG1~U zk`b>_YX~n*Ux?e0E7PuAqbz0>NSBxk6`Ng7$Rr4S%w@i>#7mmlpS8^qK}0mjEv}OG zi>*rRR{QeP!RU)O!1v>&KSeEhH|U6-gv?l0v7VnXVV(h=Zkmd6ND@-|tb-R@fJz!N zilaP(iok5UzmqMBovG-OGcK^I@N;9+3Zrs`Sycc^v=($|Z*%;C4gn2{9)y`_Qc-BR zZBp!K$bQHeCl7{ zz#w(TL%iP-xX9_KV4Ht{1-=aH~+4_C4d^k`2795G4 zXfp5H`Q~^wtG5_aH(a20J8tNqZXlMjEFUOIV2$o9IG`$L=8#6zL+VlY+bk;u*4>wE zP*(YypS)Xcn&;|GhRY;LC?QSAxYWz}-o1pqXNj=?cTp8Ug*~@^^_UFbd#<5fEpF_C zS`QWKfK(vWi2xNz+l_tBWsA?;q`z$~;E^|3P7cO?ZvH?A3WnuRvvC09N)A`<3F@<{ zw_XmE`ei3gGKYb&QqP@6f>_;uSZ;K$k{vhVklJtui^yBv0kYgBz`e`R3vvp}%h;D~ zI`T=on0wOx#|bPPb3#3=5zd@A>gC}>8_IA&C5J4!Cc%3+ikNpU#QW14+pw5`l9uUv z(SD8CHzzunC0X$zc+Vd}ipAikzU)EnzpawNl_2;PwlioV)$h!`5#`a|y8BNe#?V`5 zc@hyo+GDZ!qozQwFt->%Ro%T7g(F=QM?E?mri%^Z`t=ZI^l0xWrCb8rY7TXIidx#; z6Fx}nSC`0E-b%%2B9~xo=2J+neZS|Q?B+ zn<4RNP&pucb&JAKcB;x^dDnhb-3K+YtV2_v1a)f1RY)=yOHmO8%t>spVBbFn8x8pV zp+Ut)3C}}O%lx5nL5hA!0rwNt-w%Pb;~SoNy!Dz4w(xf+#{~=^o~Z#7rjEmxI=_N1 zYINR0m71m5X;F1_C2nyqG14#c&px;kI)FO*ei{iZ8zo|AS_0?G@8PV-W3#g8c3wTy zb$@$_n2x!tzC0fz)Ywl^E?CC+I1C6q4fmh5QvYKoTTTWlKHPt|4}SiOmPz@>=O8lq z#CV<18Lk~Wy^Juhz_n6L;{QEk;;zMk`mj~Uy?z&j9-f5OvsszmV|b$mf(KzTOkDw3 zSqjSIZ1IX(CSV@Z7nNA~@oI;$_xC3b zNPqP@xNKVh=bIz2%n*`t!k06?(u#%h??VGZ+2pBKq zvKTjJQvZ6CRGJ!=A8TFr708dY{3KubC8={Zh_f`x4KJ3J#8gF_g8xy(&O((Ajhx=| zz&E1Ae|X|&)XKl8Q2X#vNX+Rr#K}|Guo}z1o5pI5%sHN6l2DNF)QN{)F!uk~h zx+vc7g_(LP&=m&gHBod!5GA!@8e^2Hf<;ckqsXqtP&DwFndIXIgVD}+?!%d)gCek> zI^4-4^hOjAt=Pc=qDTU|^Z(p5*#&c~D8&AyMW0Wsac*V(%dCzz+bUD*x!MUf<*n}{ zt7X;rxJOuzw#NO{!1^J$i!xBNLtpHzUS+VhWIerI$x^gq^_>?=rZJ<_#sy}xU)4_a zf>C=7-Oz6SU7Q;_*_uidy|}NHb@fksQ09ir zycZZ+Nqzt%Hkdoa3vNxE8wKUHaH%#Tbm%wynk?`)bFRClrrpzhD#i}*>0S~FK z+r#)jPzEcTYaur{o`;_UYhY{L{fOimEZ19`Cb=#Y2Q*D7P=Cf08Nmw}4<%|5HTV|j zWZO+0TW?r5fPXI>%mEKDUjFK^_l}OvLAufEw`(m?nfC^?bBst}S;Y#1fUw;^9{HaD z6k(Oe>D%535XpDO49vK62aB&+OMUsSRk%{q67?6NK-PBEYw{Yvq=Q0IOg2=XYL_Ad8`-8RU7v!Bv=Tcs;7Ov((&` zP0^E|Q~SoZ#arSyxh%2hPBS5n`YEJSrvfc+5`0+BIqLP6TZzofm_|Q*_+z=2cZT*N zWj}a>cWA*(bzb2`qRMc!FAXO> zSKgg?ASuw0YbqwZ6Zb^~QtA_6NF{D*TyKUXp8J-36DJFk>n}BzT}rGL3NR)RSv8i! zf;#JjtrE>7V+5h`a%0M~60Vz#wIPnM^_iF>N%>%>!vy&7{;K{Uw{w0K^mT_#mhi1J ze6T1-GZ>(NbQEhA2JwA0!`dpY4UZ9iswHOzzyKGpe7DsQv z2Ip4>d+$Yf=5N*d{zE6j9yXRhcKFcmEG{1oP%&X}PBngmmBwqOs)^<$B&FGL)YVKC zvS#mK6l0)4eg{j54w&5#C1B zpnnn-s7490>(Ft!Tx9*Ma6naM9C-2%>cs-?a*GmV3~mg-~+*-7H9L7 zp!iz$7YOLq{<0C#zZCm{=qtQD$-Vj zc9gvRFlNDt<`=B&7eF72UinNB04vd!%Qda%Qbsh2q;IJ`EKht085laWgP6d<5gI?# zb&^n#mg$4Lq#i3_0#y0OXFI)ns=l8H{uNv+m3@H$S^u){&iyKBbrySFtjN!6B=u;C zVaKRdIWS<%X`xE(grKQ(n6<;OF*C$;xm1kB8hd!; zGg)25M=C*P$Yrm92=o&CLf~Ys>{c|Kk=O=O-M;)juj zHyU1$>9u9-aFrq-e*TtN7A627M|If>Z)h)fL7YE;9fF|bhWX7i4RD^8>bNzv~-a-w%XkR=QLo@-HKkz4nQ-#_sZLOR^j z-~erkV=W-FTDI5xZXmqRT^~(f2DD6nHG3t$O`Ayj3ba=#Ty?Jp8tLY?!QqG}zY4)N za-rA})8epQ+$gJ<-(((LeVdg`U7675i!XK1_z3nHZc%<~XD#pf5B|w7A@5+zY46uP zntk=&_|Z)66+z&?&;?PAb21x6%LBcH%+#pjcYjSTH`ZUR*lT#K`zRMD`9C+d4i8x; zzr4IuA5{jKK#YfnrvulNZ(0roe#B1!Fh%Pdvmguv4Q#YxdWJTX+GuC?=~9oDjXq^n zrnxa5nD=L6D8Kid2~B*`hH5mh-j)^wxW-BHbyaf=UIN>4%c`%fif1y#dl9K(ks7^$ z%CzF1Fx?-zY6|{5dwT7)=Ls_v#;IkJT%t~HVY~+v6B~9ui+eeopKBb;Q?*O-ij849 z%^5sN#@`7HKXgthmH-F5jtj|V!f_*Nat5nBUte*|{#QF?XsCPcLY;0L%~PCu*hB*y zcGo{aw$nBt+^{ho-JcQ_jM%m*-#L{XA)L!+k~<&*phLma+2z8(oczz$ImAJGFtoOU zKDT|Z6q)qjB2z?QH5qVF+HKH%l`d^`A9P@w9!#rkfdBhl3fMKxRaJIi_GIS_lmqu_ zNE-UexBk5=goxLqp=MP+zVEgpZu}o=qb=E0)y8E^7HA{DO4ll!>!CVm$7{|5H2s{N z{@tJ~YAQ(zar|{2d&o|KO11jFQ~$z^kb2!x7K^+>7~1GKm+j7sY%S#LfFtnSRr2-W zOWEF1E`eoLP*d)Ca__1ogymc7y;Vx!zECAh?-tP>8ezC=Mj>!srhn%cz^=Ub0w60Eb5 zj3uFa6%LGU63DYcYj|>uvdXXzJ>EZsnaa^@sCErZs~;In!8XSN$>u0?jnFq<4gXV> zwv2T&y?^)3$$SQ&`)!Z4;U~a}PWpG&;F`iL?Rg6NB5DO=xY9K{Yfvwlq!%KWUOZ?P zs>r8cmKcxMfTbj|{wFigCt`*aMxXi!J3`HyV}*z9PFz;DF@sZg`(0>z*XT*`re}4- zEPsG{E3W-nsRANGV?GYGV%Z!cSa&%fx^LHJ=DBbh_F`ClCC_GS{wpAo1J5sV*Hfnf z)QOd1qBf%vCvc7LqWiAEbIb{Q#kvKsL)7#sO5&c2@FGmyegrv%?SpxZ~J{wDR zd&8pKwaBcOt%Swj-WrTnoMv(xVD1$~A5$Oo0W%KQkmNj5QmA5pB(ztE z14Ck?CX*_qqVsO!2#iX&WVS_q_DjFHU2}>mpc^i;OzZr6nq4fu4)+c6ZQ(b%VZsLp z%nihz$$-|!l0sRSrf5eZ9Y*`xJ$dW=W^3n5KRKH|;7!ax$mt(HQ(%2)xh;PG-ykYA z^z(QsvFH<;*6AOgBQyJE(|1t8#dlgdLxwylC3OPY(mBL zZYZ$?145dD11YXw?RvnT8FIj#}@QD@ZUVxRzcHFg~Tmk!u6-C;tCJ@IZk(&W8{9VfQQ zQmLw8z(5~`&*eeh;wAQ0eeF>bq|Nrshb7hwCIbcv2||*>ZErhe`>Bxdoip0nq@eb` zBBh0X2E%zjC`EC=H4Q1$k^pOwp_pKr^A+&nbq#U?$-E4N)h-4KS-T%}QQa|uV)a@s zGiDqY3qI1mO(I*C=}a+M3)}gpp-%LyHp2bVGwLksvZ9N%pv2J-xL+oG!8m`aNf}N0 zzFo^nd|kMwOrf0pj3SO$gmPC3EQe6x^&)0RH78Cx4N{jtIww+$AnXG0qDU~oSSo;d z82Fj7Ixfk$h1!!w#f`-aEcW>i{W z>}+QekTNF`S>uS4I)x);;ISt)*ThSeCc?R*i-_Ka@Pzpxz+)5`+S}57+!Ou~+M5x# z%SSMyXD^0iii3>Zqv=D>w~Zz;?pxw_|2O~!>H;O8nwxTSWb;Py3ZQCIRkY#0Jw)Y^WnAOE zX^&le=iXlW^F183y)|@&w^^~I_fG_zH((-%GQbRaWE}4l zC%tpUH}KgJ%^nE^i%OEj9&ichX8(y!p9j*?Ea|p?q@2ci>z7A2-dr}1=+jU80$x~X z-?fT(;K~%RdR#gEitnZdB2$A=ILBdpA!`Tja|q*M7~A29e_dn4djtQ~3*j8^LD@Rv z{BsKm@2!IGQA0$3KsmTbe%m=kiQH1J$%9m9FEs9)I;10x)#T=Q4YdC}6f3Zt0#rj@ z@OM2zovgBY_2a40(lu7y?UhG%jzR<}Jn4UK!!kULPW!&czN?>C?cRzu9UB3Emd-7x z4<8xn11h(?nU35B1CWn9+^e}9#a#R*91um<)<-tG&+!t1$b)wpDM>K8k;Ho8$RfxK zO)0<+qe%0M&hZY0f->q(meLt-@wSeSFunzEmLQ}(K==^A&{Beji20jCRUDjkU<91R z*{NNe^fkNO&dio{Kd3*Xm_+Dg3rvDYHCJ!}%r6L;CJeAyEb@qJbc?!6HI>As>DhL} z=dA3+-_*X7n`~2k&aivVP5!=TL7Eq2emA?e)!U-eytax4+71E z@08)+A=-=ds>A%}quv2%4nzX>%YGtIvx!k>H2hDh&NHSOw?WPy3(b-wvT#l#w<~@x)f}L%j&F=LZqZr*0Xjf*Sc$WFh5|&~aF1JrS(T@| zWQYSiI~&qX>hor_N)=k&DQ3bo@H`biEm$ul7Zr&dgH%o8NFCe7r||C?D~cSm|LIaL zcqX$C+S^u;LN;+VO;ueeLYO{|S=ch=ThzN#96IFMK;E3fU@hx2gBSk}jtXLg`_$+@ zvh12odrGfJA_Zy-Gw%JgUUhhH@svVnr}rP5ydRr|+;gNSobiV@n8}>TnM_C&N-nVrwK$rMjht&-Lp>d!UXR{p8g4~Gx zHLTb&j@b47tvX-hwpVMy-0QrAs3jXUaIho;+ZKxp#l3q^nfEK}!tUfCK=>-^*5XF{ zzE5J|{rL)$daoftO#hROCUNUW+*ljF&DB085-bsOAEVTP!q_nPw~?MehWQBR2&2q& zj)H8Y%9rnH1P&}foWKXR&8CIC9AK-!DchM$4xKMNXtCiW!<8x>q~pnMxPVyue&{u5 zw#&SVc$OSM&|oPQZjYt^5$Kt#p7?xdJ&c$0mC{k{k1Y|b3B`IXPGyXU4!0$t9U`{V zRPh5fRyVO2w6=OHi*gV8OIkNu=-!*tmh9Z$runDhxu1|TNUR%a6aQw3lATe2GjF-5 zsFDK&mQ6OnwA6^%0dno?yvu44__OGp5R8sKXmM0QnmAl;JuQR z29#&&vcBdUAej;p1yz@NfnhTl2@M~rI#*|WgG0n>7bJn?n6=)j9Z{xZ*&9QFg^DDD zjW@+3gFHv}QZsUnT&kwt3K1>41!#h8cY;gYazmzQn2pd^{O(!oF7&AWPeOS({Y~AX z?JNB0XK%83hEpD==EF;15Z5>84VDcyDfEqaOWM%R*#YfYZTBtb1ArV48Ykb~I?Xqb zL8>#z+1goFibvJN@*kJ@X=R?yJ!5SJf#GfQZ&a_hQypBH+``svg2pQ@_~)DOn;Dh6 zI_O?$GEaRxdqJw;^Rn8AyYTp!ASVO^e~n&zF3^s(H<=EN zwOBNJ&)7SRl0b?(kR;;Z59xz&lGm z5p7bhcN80~wD^5EG2)uYG*En=L?}Ln_M-UBAc?uk;)_$w79&+P=qCa50Z`owB%#nF-*g$c zNQ9bdnXZ8&AP?7rtGMiVpT(9pc|q`IxPC1_#^f^htZxmhomWuMOh$j*lk&$_sVowH zrGs=5(h`TN%15nZiRCK$9IeJGB0%10((C>GBaqi&=;!*fwB79$#rt1qvHWWd;y!XB z{?IQ!{ah87O#rjODZlkMtXHz-qy3dV;t1`F_I6f;X#GALm;(e}|KJdh&suQWqwl)9 zTYC}stBAsVpq2jfxsBj+BaUV0S-=N$vOp<^cIMu_^ucR!TobUF1mRP6T>i7<5t+dG zogboNCU6|gUHuXKPbXHP9Im`Lgg&ksNRI~3tv%0JJ6DD#=I2ZUM1bAM3_pvuU`{jq zMP-?~HnOMAwg|e8RpT|7{Q};MlHGa8HMeIJ+)bDMxBeIyxI)>i`XAtiU;rPd1hw>V z|F>~+*=Ot{+^cXbl)C^0Deo%X%Z<5>Ei@gBkJM5l9Tx^?|MnSUo1k>j4&pjIb6DEb zEt6bA^d1#!-3&` zit$CS+e|&{_V8#J*qrsEhn(cjYSW!TGL^Urga4Vu6_Dtq!5COCSN5{kt2@uS?MdrL zgSA8uauA$2FZ@Uo07po;@+|Aq(<`ZJ_McrfJ{{8k+VEE_HPi@ZSB%0;>>VwZ%dvBekvix*SUiPSKceo;MqgvOt%KiJ* z%TYGb68Fu)QN&U}2Cw;I*W4~7vqe2QP{1+QzTv$8k6-M3f~xUu@6eyM|Gbpk-xAGO zVBMHF`QbIMBlCxeUrVly-Zpl6(ighSpf?ugJybUO;0Xyc3cr7_;5&c@;tcI{kE;=H zc9vuz?)Mwt{Dl(DfOcOXVWz`#8tGb9UD7Gvn;78ug+O9+fb%W-Py)D!AL1b=JjrwB z76^65$-S-rrG)XoJZ1aKd1f;}|5!;BScx(%cwe$a-{epNum5>fz4W?5Yyld<5UN~kAc|5te4Y@;QUpUTH-g8~iRH$kCGSyfU&~;x!k=cAw{&hYN^k~XUGii@ z3^E_BRaP|lN2D9@M{WVvW{d$?5Uz8}f;zA!H(28bFIW8!TEt8BGzj7tuYB2-N^$Jc zk$U+)gjU@?)&1v#)PKPAQmkMLIzR|#0V87O(1mh(VG0z#{V~)i6|U9{8~{l2;VtU{ zn}wI*Z$qd1wb-jyanCVsm1f}bNZ@4#OWXE(sa+nO@v|vUey^L?tcv;VlbUk?_+bf& z4*`fxb}asf4p%rLk!uF^qFL%6w7)lxfBLb((nl@QU1Qdm_UQNm~uQ%Zi-Je&g>_J~%Ga7ker9;Z(qh<>g_`rx;5WodH-S_Jnet;0DYZL zpCacxY&kUgUZIVgvD;=P-TC%dIc5A3_2B#RML@b+>FzFRMClZzL_(C18XBZQ zxP`tP-E+xRInegQ_R+!er)xAZimJ-jzI z;yd?M-qc0Me8XX5UO;wsHd6n@$MB2mUpZhh1JfQd1fT6T_MYNMVzmzpDQU48J^9$U zg``kbJ-(hPcG9x z_V+*tkRHS4kW#qm$p>Q3?21B@Nqo?rAX#bjd6OyW3ypf_KHh#7Dc|p?ueGar+YG*$ z!5w>p@}gkkEY{>u+0>eQSKgD$7Cs3aYeFdAHp}uQ6G5AnbjS0=nx$LW%FT zGh#&Rk!CeT#@*gt8or4#`FsDm8l;ptV+gzVjBWUHLCxeirrH~BAf*R)PLBZ7?DI>u zG99pW#ng-Z&cCDH2*D-}+b$pVYdu$g>HCq~zv#DXBv0egm*ph3M}0Lb_36cMOfvN0rAf(iMyyHNkZ?CubkQskW7u_t^&37A0Gw1w$cGO_MFz20{dXfnaz^Y0iL&b zqk(a61%X6RP@ovYIp0Z_p!}75jpT6%F`EhAAK}M_JMN&2b|%1ae+UP%p(92nFrPPe zz42!cdcP^$^vPqABm>?efeoDfa5J0PIR>p`LrQ~1l)Tl+SS0eC8fnBo zTHsVme;7fFfZ??-9+!!5X5Z1}UVEt-73GD>Hec&a2}KEv=8riNH=ZMRfi^Kmacp~z zTKaHPtsSc0$GJArfN3P1Q|1c;V$xl(s|)sw)olJUYO<)&+^W8d%8ZNB{5yD|A=LK9 z1*31tPapfPc1zoJU^)RsvK{V<*KV{6k!OZ&;X??&3WNOoeeF|V$ z5Ev+z->5rzJ&nY1sV7kN{Y--2m+!Jq@N2YA|&tj7H@SHQ@`-^j0+;hBM< zGkvxq*4UVj-q!d+_7!P`^!7(N*b?0S5`?VQi=MmvT>0MIF)&M*RPY@TvWn>nfiAU{H39dAlZrs3{lYaMS;|Yxg zLSxnZjGh93Q|(WU)zV(8Jn_X|R&1*_cW#k)QMjr5uomT3rGy>9=n9d}8B==Ucdq`2>B$y6A`hf;w%H3)Lb}KAt4zv6cKA{ zC>b$L%o;`E&u+reIs{$-;Z)3A^p+&u$yOcZ+>KT%b{CuD!m)&*$sxX>9m@87G-Ak5 zaUXpP@FDGTlx7Hso?cN{+`{D^$^#IUS9iby5dQ$DDBEHWDQK0RhcJom*L$RB+6OEc{?+@h^F@1>^do ziEp05V-b=nFuqeM0_4xsIkEl|GRay~r`}#_H;e+w(j&oVPxH#6R)Q z4Y1^qu?W6{uG=imEI`pt#q?0T0V}585rZUQPgomDALYw}N+j$9>EKDo7!CB-2O%i(Bu9OX( zVFP)_jU8Ssl(vbDFRj(JYT2J02$JFJlab1R(9{i7}`pXzXdKF!6G|5_=${iza|nF(Nze*iCR$nYw7wPW9trswas;z4fVX zdt&{+Z?C_+E0!%)s=aeHL4lXJ`dbSQhBm`)TMSzpSh?N$zl@pz*Hkjo9Wk3Q-=QNKH~NSwP-VPhPcz>q*bnddL*$a@viqxVHV%#^pm zi;z27D@--r%j#B8P@SWV6{ROXoCpg21as{1zt&|0Q+X$ zoQpSZnnd54`Y%aOL{|z~PQeR#=Kn{Wq|y7rCa1==VIDQS*%Erevletn49La>zR1*HCgJJk-8_J>^1 zdcY$(YupL4{QzqxwnqyCWBvRYmV`*K!&*dPW=JD$9%y9z(cTksW)nm$f;Q>#FvX0` zb$EJ-yv*m|pkw~44&T#0S4x{n|MJtS*W>)2==i(kA(sd8pvQE^=9hc08a<+>dP+uO zJOAX7Rvm2@(CagqV#<7a-37lP&p5S=0MJD`_&MDFM2S}s1M1*`Q=odif550M0ip0*Ji=QUqZAo!i%+IG%6*0_ym?F57p#ld-QJXB_A@yL z)Wi@(0Pn0uGsIG$0-pZzmQFhXjcy%wviTt+B?4vplN@1UCv5slpMJSjq@*#B)(qL@ zrhMkqg;)P|J5sMX(pjNiPHT|m2@%qXUxhMoacQ-0X)sjkC2B+ba=Ms#pdsUVyLQ-} z{JDs-l};wu2WgzkHx6GcMr8)k{wlqUelh@nzTFD5g$W{*@Rf5{$A`?}ICk*dj;vUP3yKHCZ}&2|vlqv%Qaw zXYAIM)9QbGDahADW;t`TlKlO&rbe}eo04JiqyGyvhUW{5pPfsZvDUgCp?-)#XhL!O zNI(=!?6H<5Y@zuB%~FCDm3HLeNAH*tx2GSPTB==QNNHOh!3~nR03xA|tGFM^jvTSS z3re!j3Xb~FNB;OWj$$Rg7ST{mSg_Y-U53dP($Ckqy_l@2XUY@s_Vydp< zzw#nUD+BU(^y z^(>b9m?BZOV;-6F17?c!b(7F!*%IEOFTPIJ6NP8?v5&g9T) z&ty14iz9RT`iNwtG`wFn{gWZCpuQCYx|AeXdGy34hhhU|UHLm#UY}>B7NoW=2eX4C z`|3ft$3I*NMd%z^=ZRBd@Pj9iPCXum_(0l6;XEOYdMjR+F$u?GI9b5j?P(S}n$#~Xei4tUP?3+*xuDwKs=cj)OgcZrl{bH^6!5hzy0@*) zmXnuCe52FyfqjbUO#?kz2~{*F&ZROFi>g*4U&e?jttU2w?BTBwNsw4 z-aA2?paHr*oxXf-+fvEkOA&D7JJ(90z#5ZQ9~($^rE2j8*G!{k&UZWl{1#sO!J3s` z#Mh8zCB9njc+$eJ7T1gOo2itzT0m0wW)Itz<|C2^et3_i9om+##<39k3o;s%SYax6 zfgQ9-4DF&t4bi-#wK{sxn2Vx6dL@cvW21BL+G$=pm)4h~*PJhyEu$&+j#qwoS3S3* z4~(~ghUz^KBCcd&NojFi`L-ruMX_zWD%k%t-0PNaAOL;zmE`S{4VW(l%jeMVDKhSD zZ}+HEsmTv?FeVW_Xe2hQ9eM-T`My972?bt2kH$=BnDywFuYcigq>tXd{5AhDYccqy z)%13jJ3;(CI5^DGkI){jC#lqP$LTzDOGW?7wTnyJ-r zxN#${gh)aE(%oPx)E_90io9$zsZzUU*Lv~hopU;jQ-#DIpT^9MGY?b{?$4oa#DiYH zc`tr=X7Uz*)ZRQn>p zGUV#7%h^reC46N;Kz7A%F47f)k)mh`{YWryg-S3@=lZ zqM+(x`jwYJ3Tt>1e_ZxAVYgcB;Ai|r*n=i()9P`S+3kHdf`SjBK1KAMec53i<3Jh7 zau*Af^(R zRmF&ggq2XK));Z*p-NGRg16qIbs$RFdB2Q35Ct+_Dv>^9U1)u>Wu*_jZr_fxNE-Gg zJ%%RCIMLi-5{h&C(+7m^>2w27Q+)ZaIV!av6%v6dBVkR3H7Sq@>W#LjA(iKbqGEju zM5GTrBN^950mkZA*m^ok%*!f6^mGs@ETV zBtNKeE|21@w`J+h5#&vLJDK@ZCB(yin^$Cyip$B!7 z$}{Esq51m|xh3w2@{^ZPg65!F#iB(*vDXcnZ!OCNTffe@6n^^A*N8Dw3n6r<*j|+# z4_oy@&nT~b|5Dn92htw$nbCFwK*y|#V42W;AzmXX<_N`W{C zMCnjuT*DIJc_{KwGgeapD=4n}McAZf#dRxzTEy{s=5_26baPutdJf@s<3_AG$&!#|qO5-zLt07&vC z-gPrJidABXPUDVsusCqIu4kM-+LbBCQs>AbLA)ljtRvO93FSdORl{)*p=ts;y~o%v zpRbwQ13Kw)Ia2Jh5=!P3k)Vu33$aOLj?)E7hm9%Mpflde@Z4=wLcp+PIf+0lTbm+y zv43}F2m{e@HmVP~JF#+yT~{k`e&ynZD!MrYK+$%4LRo~LBg&U!pE5bce$Ois$a6w^ zk!g8=7gTaIl3!tJ{m?AlR;lt1Y~D>5dvj9ZK@_09>>ef}yHzaL zPYt5&4T!jKpBDU?c;~LDEar<)IlOK-D1&=9uBs8>+(&+SFjP)L3a;wyGpUPewa;m4y$4lGodVgBNb@-Y7O*t^|ioFTz zjDBaBZ>_3w^Lr$>lishnyiLD@>9yg2F0yZq62G(QTfSn~6Y$~kA&##d;2pJ=F0TNf zyq|_{GENW-hc=WC*C3))Ou1f=^b4UXe)`pzD09H!b*RG#^YD%`J>Uglk~p2N5T|%8hoiCi z&AH@?H8z?>4Y6&Fr0Gw9eZK2skb-P@PFr1oZaJEK7(5X zVU65WfT{yKg4c=X@d=2A8@_lHQ@JVT}dB*&w=OBN^ zAn_*^P8J_o^(0UEr=W|}2REpngu^*2&hf1azQ<^zqn|NXniCSF;&u+~y3!jVLdi~zE<^zVp%h$Asm6CFpqV$`;AE-T%!ZC>KH$L8s( zR`e#C*GZyRG2@<&4@|l(RsU>V4v6O!cMb-%qg%iN@eLwHaM!d^xptGcZa7ayzb)-z zXA}3l5B@PR?|e*JHkb>3I7hkhIBm+FA$eZl+w4%WFW0eZxXtNS8vi~-@MV-NP;aiA zXph(`?7!vEB{PpUXteL0PBTgG8Qqi+PYKwvbP@U_$Ny1@rwBqG?^$Y-*W0<;{6vF7 z?3IugG_HL675*N5To<+~1SXU(uu&zjsHYPU8;T}hkLt9)NPM7$kq5_teajm;()Y#_ zU|r8$AT`~IxZ6MO_|%kKX4PsvQz`RtcF-*qmTTEh5t-#O0mQ);U?YYmh^A zz(WF}!MNii_osWkXR(+Xt$cUifMQt-3dXJ>`wN&8s984d`_gPcdYmdju4y~9lle~+ z8@JJhaDg(^cPGMP14I(EcLpmDJcK@72tZm4T2VVx(l$y*05nNHNT}*!cU0zLGGq7MMD&dvj?N9HoxatEY>5+FdhlM%8k z&gJ$a$HJq-AAa(>Pu#}S7tU^V!nFDfGltAAemT5(q5XzSl-vtO()_v#z8zpF zCbpbu0>?=l{toxU7Dg_pa>zmx;@)shy`qYvr~FNB<817$@-wi1_q#ZwC{cjneJy%B zq}I04(c_ju8pf`YwX62pjIOMhcwE479eW6;lyvZuop^Ll(Jmw8G|e| z?bgh=;9EPzs>7*C zM(ljI89qu?n-&%pO|Yx%flGz%WKYGuiE221;#3L;VSK(Ffgs4{r;KzrNj7`rNidfP zv44anBPGUY3;(<6d{V`ziuLGFcE^psbt65?#^#5PT`9JC1t zLFVUGs2X!^sS{{I=|_H5R(`;}pVaEv*>+3|evFXxY{ZM+BHbb@bppCr zdD%bqJuS`fHm>k5s5WtF>y(>`3{I&_fwdHo@=U=-26L7!yxx?QR87&t2Ev~Z7Q`S5 z1kms3Xum6H>0377BqqM8?6weoy@T|SVD8j6)L4q~4m_Ipy@~A$zQ)F|z(7<&wi+$Y z8}ZXjg=^EMLx;mcHHQg3{xIeCBw8G@AdSJZoV@qqu!OWpmiE z*@{D|TZ{>!)B~IY#RV_cy^c5zzallN)uu%&O_&UM_*XRs4e6e!VIXlS0UEZ%LmmZE zK!%SJ`abdtEl%We-@Ff#?`_Na!Jtbq!Z#0xWZEfriBC98ol(`taV;F1nqeAs9aoqO zqkw*(-{WRG_$mfWPyDVEzS>=sYt)kxx!=y+mc)3Fq2x41b#o-F-eVTXO&{l zObwxM{Bp;5^tgRNZvC}ytkG*Dv$waz{!cKra3ux8pXp=k1p-akf5x^3MDOq^|N5*z zthkv!qu$@1ddO+-~+k5Og zRbqK)9k|3YJ){E7oTu|W8J}WX`>*8ASp!L!l3!uetX3Px2}Wf163!a+GBt5{i8K3- zCNYrc4MYOFinjteU0LXql0znNH=Ty)ru%PG`$52Erbq}-Mh5L2Ve*b;Q*4;#H&;$u z>&1|8@@J!}ODswfj1iN zq{4BjInwj9&aX62KgH4OUZ+sY87AL&ZZl&cbZrCk8|0_7Kgu6YO|^Eq7hUvRyc_yR zMuVd}-gwAzHy0egY-(#+BWZ5(xqGynrW{+kN%E&?#UtZ$*jeBcEc6WQWnk_sL0f4b zyCjPkrGrn&woBhsppqo+fg8pLfgI-It?ii6BU3(%xiXB1*pOYcdb~5Nm>}>QMOeye zvl4m*YhB?wL|rTgV~;+;+UO^CztSo(@9>pj;FYN-)0NLtD!$+71uJvxCStH)GY!I2 zL0M!l9Nl9D0K+O&QFSA5G>56q)J?9#B3FtpP)|DOQV)o<-SM`^t~8u6ZCQ5`wAo)u zeOIWJ-7e^DYeUYutik2u)(2&JFy%mGK3M&X=kCvX^@BlkVj#_2D^9$~#-sbrb&#iF$SaOZHlP38@HOtC0k(b2zoG!MT3NDvOWb6(b?&X|z zB4G{nEq8^Nnk^gOo>cnpB>}`{n(!SOXB@5x9Wu>|M3jy)5Q2YFCNfsd9NNto(I)c1kg zurzcthRZL?122Y}6U0OrfPO8OIsl6#I_dqTnTqA@{{Wqr8@n+7fgi^+tQmZFL>3Lv ztIJDB0X9us*yehk*U;*ckh=8fCU%Hpm2Bn@6dCgufd?FVR-a9hKBirU;mi4aJGJhj z8jfSGu=UaPUt-nr{6xlWIc%x zwk-DnfQ$fp4q#8I6`Iyb%{J6jA%@}YY4XyxtZ-qx36FgN9unzoUDmR*l z)s3P!PmSKZAm&!J0u}EG3(BgjKgFL}Qv{`LRWG~G#>zj{?0RGv(0#+%YM?hpqiU79 z6{AHM?^cZ<w*dFTvsIrJHcb&`10rjq@z>{DRTK2)jct;!T`6M8cSNpje<( z$zuf-S>BgX`Bo!5y@T+nVOsLp6CUlU4pNO=Ehj!=*Uf$)%jQH`E_GGhUp7qAm&4oG zjNGM{(ecftjjM{krc+s+w8Zmw-xuE$&2l#{BM4Vow&wt|eOoGA zqCz^EOz7Ryw_Dy~q#?6K{@Z)lJ17maG#KMueB(J}t6rC)LOLFOEc1j$eTWYU zaQrnn=GhhR4m@dcpel4DFMciwZ&zG2>xh4U{Mf&l(*VQZ zi^J1?GP@Gk{#3&Dwxj6uazrFo;iA4vtQA>nRF^U=v>x>$OMqBxJbjR5kD?D61Dwe_ z+=rHdo+>iEl%6MM3r0dht&MD#VKR3XWj~ex44=jx$4KTPhlTtM zWk0Nv1(VO$C>%h$L&2bLRsChp)O}vlHF>{UiQ1F#&_HH(@;;*u2!!PM2c>A1e^m@# zZ}jHz<$AGg&p%`lrnPVK)msek4`)Pz%KSA_U08Dhi}kgy4uq^<0!uZvd;|12G36}t z^GS`gM+0JkGTLYAqBcBCXE>rY1~i&D_TE_Io)_QiW4Yd44ZWLwN|`&K&y zEv0xkHn~Viwb>ZRrVlcFdyZw%WPcX=I5!GCbThFJ7`9sAI_-fkxCL&z2-t%lTv^R&p0(vHPZbUAS(YIo@ zzZW47XXH2>F}rzD6EB~5pV z*g`o5v||cARx!g3vMsI^D+0PV58Q4FZrvu^cKT%9=0YS8mo8Tu78JC~y9P#{Ntb6~95SWhqCs|xUENgpJ((}ND{#d69 z`<`sC()WkG9E6T?FxNGQ-rV;GG?TuQ0mc(;{-{2P<6K1-CxXgW?E7fw4APs*7KHh3 z?ni?rIWS4&4U+fV!oMN)LK?tnI-sCb`j~8ipP%7e1_HSfj2AQxZ!hg&ZocgR&hl^f z`D%qvPbHP%=J*NA?p#3`lMk{i^io_>wzZ#qb0CfEcp}Ub)eayHi~12#%cdg>jx}&2kA*)$q8{Gi>7r#?7lg#rNPW|1t-uwDZw&8*$a8zT%1cR_2&;U>PS?6hg zfp`0JbEBN43{5>lXE^{96kDmE5NBi9T56yyCWdV%j8)AZIzE7-Ke19v(51za$e^lx z_=Q2Fc!thurqndcfPB{krGyFS@)r0E!-$r!=-5L52w-hwpbUcnf|g};^yJLAQto}< zj@dexMI8U0NQz8>0Z7}zkll=s2ro}j3M~#_phw@Q)wA3016OlFGL{DVTHKuQGIqOM zAa~-*%If!@!QXu6o=#hfMBV5^FQ~JgI+0Eb6G~^Df2%^*O8*A5HP!-2NglqdXfyG$#+p4vj*D3K#0*{a7vx?|vmds0&L^gqq=qA^O`?%#OAu6VHS2l_Y{ViCAh@ zRG2}BL)J^;yCHU0CU&?H0S;ft8mHJU*ln$S?%(B$;jI5f>>D^?8Q72jpmhsyiab>1`tgNkVp4@C zxLcn>B8w1d_KI;5#nPFbWq-D-;iWqABT`@C{BoNTxd`fVLf-lUwh#$b>RPN7j+;S_ zY|Ahymtbka>>4duQg&nv)x7UpnOrq7;)fBZ+X_)FiiBL zx796T<@F{Gc?2}aOwGQ=c*=8u2|DKu@8N?6>D-E1{%jJ9J*N9{6&0_W02FRP5$gYA zc#TxBv!(mMMULCT5?E8(gf;LNU&nWr#IHTNyW8;ev<9kuwcuRM4F@6OdTybDH{MOHkZ>o)tcb_{d? zkOPj~GsqiQ@Tf}!#sX{vAD;_jVSKARI zQ`q!ez=1Vax`V0=$~P(d#Ov0_1E~UL54be`F`hxN{k;I)JNMHm&JQt9dSuQAe7Vnl zD&L2-KvO7O!D7#q!OUf+o4y9`4u+!I@1=o;K=4@1F!Z=ag%XYC1&?0V!>=wI>Fbjp z;y<(Y>=!+^_6~2cPR4Jo98rttSn}taio7Nz0?rE8k^)<6+63aDu-+Gneo`Ukmsyue z^UoKc`+fs?3Z@=mTJX4%6fvi*Y5kp1guWtD+L@rKj{*z{%t%xJi~f+Tm5!eVdF($8JR-sc*`bDrk9?U;81Bl61Tj za;ry6wm11%q^6g_SLEu>l z1)u2ke|iBJ|LRUW=o_hx3%e~9-a~ygoDM7a&ieT&y=MUvs@XL7Q+w|d*}xTz_&E*- zO)Z&U+$IlO?gsTVpTlx)@CB0&V1B3ZR^Tn$DgbkQeKf_q!FZ z&WxoL=76a4H$|Jxx+KonwM7$N)GcgSTFxX#J_rR^@A(BXYsjira%W>b=#IBe7ONbD z=qVSB?!4C!fl82y@yW)p-nQJKCo@`;BS}zvv$H=&!%01r!MNg!u2*0I)zfK;V-3IG zA=Bv<=OUWhJSDc*<(I0DQrR4SBo*Z_J+EmU@la{#b&#ajdMYzM*mT*#2{tObP-xN~ zz6;SOta!yQRKs~1-tTsF_i#bDv&W7E*}cN0;`#R&$*el7V8B`e>iOZ`* zGOLA_=T;iEq@H~-h+9wFCD&*%>r|E%=cHIBa3IcOu^$DLrKaJw+v!1FBcff?i=)|O6|P-YYq4>V3uB?fJa zB6)V3ob080_%!%A)Sfhccg_AWDD-=r;sS%`cVyk(erM2GrAlHmcBa0$;&APyQ4+eL zx`=y=io6NoUDTHon|^}HtO|L1r7LRDSwoFTQ)tiy>d&$-dtI8S`!S(LZlCzZkdLC8 z@f2#`&$lil+38`K+bKzf-1*&!`J5vzN)LA+-mG4=P&7tCAR|6%(r=m2BA>Sh2STAt zXlLQ_YHH8a!04;^YAuq9?-@r3kP~SbgIzbFtBf99r@wq0_VM_3*^+t<_<90d<=s8p zJ=SV6>iW&y`BFnBVpZ&)`HE3SWl)&#fuW8%x!Qvhg9UHuLijPeXEzR?zO;Oqfp^}B z+hBI0U(i~QfWLS=I_^nstRv?xr3~#b&u(=)=(r)$t8SU`W@w&ixxxuV0Q*7%WzDFd zMD6rLp87Z?opS{|_C2Xx#{!~lYTi3fek2O-tFd&ElJ75Gh|1Y4dZpx-?{lrI(1e!2 z2&KC8e{f|);P5AU+rhTe`_`_1%<*=fKV3ZLQm6B2x+&~1PhVTzsvf(z-ikPU($#&? zC_#2XcKOnXXkcRI*kZnd6#}jopAzmQe=1^Q->G8fJeS}RLiW46GnPPIeWZUPd6)N2 zy5c(5H|TNhzE^`;7uKdlI0B*+nmmQF<*u+n(^1z%PW-Ex9}_PsxWG_^AYatnVF4pw zJ^?Bsmb zlOwYx)k7*|#r|73%5h{ZWxkdm+tK++k5pmyBItI@PSkS_Gh(#=sb{@^ZfV!xgWqqC zx01ZW`bRwVzB~a##9>cCNBPFS=W6k~Tr8MhWzEqkoVuen z`ro*8Gt*NeAVR=z#eL&b3YCC&7qD6$|1zDE8$5FND@8$sh(mKg);t3Y#wMTRj7HIw zrfaVsx9Pn4ariRsT5xaCGxHG$jHa=;ysTxvu7)hen^siIAUtB<+slO+voH8Ek74sR zaqT8^T}c+WCYvrkbzH&bH_;U*>*G%zn@wg%e%pB?{lYC|*&Ia)X!}g;$@>=8QD^q!8OFQ!2af8prsJ=l70+O};UtihBiClH zE(lgdxX)BsJ*AEJ$=p2cte^p-DL|cuYf~=My$P`8(q}3{Q(W3Sj&zhy6mKZu@^(;l%F316Yw!!vHG1P>igYIn6!)g>NE2K-`Np)OyH z3^$V%XY>p*^@!1Zll@LJr~X6#FV*hix*g6myh{N1a(}+ z2&$K^r#{gi+{gO({qv%9S-BVB0Ib??d-u42NRrsmae zxT?oW1s%>5q;bb@TAd-{xd`B)(OQIO=w?aTw_G~Jf-RzeWG3J<&b{hWV>?L*Ezj{0 z;$qsfw3oM&HYpgnAypt6HMu*hzLk`1J^JC%n1shDFe}v~5+32V#N|g!NW!1$<2E}G zyQ1QS(IM~{DP@x;DR0IakGxU1#uq zp#1bi|3`?-Jrj%_gPwm>%sZT1?1D^%09=7vl-%LTgouu7W#D5U;ZoFlqAm%#Z6K8O zsn*XqX()$-g^u`7g>IPId}*CR4k`T&I#$$uJy38-7fFy1@7ar{5%~KA3ZD&}*SU<) z0I^}~(nuo4o?%aNMme<0Q1}wGg9I_#BzRpFaH!Y_Fo9>qhtRr_jy*TfKKmXX9&UJ% zf0ZLe{zDfPI{C`(;juqXuv%b5hBW-OfZ_OOgn$1ziJp5nIgg}%KlndiiASPsP^Eh8 z%?h4~U=1H5bcFwX#o`wP(s*{gMT&pD5vLWzWgGVGLMEC>W~Gu4sr;Xpg_Z)3OPvkn zFaGn%|4GCwoPb8m|7ZVMSkY}&!gH1v0Va~|7+$Ifp7<2&hS%5k6p$Bp%G|CAEYZ|NlEj z|JRiN&Dj4x{|sV+s18Y?kcs%idhlCVrl3owHn;DB-}%(M-ue>Z{bC(4z)p8Y8^Yd` zt0*Wa;74Flcs~x%YoCTsT~Z2RqBzefQ|-qPK_sN#H2>!xh6KUzIntE#-Rdawed25G zKU+jXEVIg4?SK1DMMhS(X^`Doga`xh*a#pm$f6K0Hk1W8JdYlV>^Pt3_K(c{WxXhV zg=2+13k?m(!$z0$0PXLJi+7{t9txOaKS9I#mxTP!6@oPaDl<1@jQVd4lLYzBeEYoa zq^nEnO&Kv%DW~?2+&HoZcA8DFrK11qm7&>zpAU)KUWCrK$i4p0qeS8>l~t%~|8u;5 z=@AaZ3z3I1;s)rF_OBU)?ML#DS%D384J1SN=~9!om=|DeJ{sSpG#n^aOm; z_Ci`(+ITW*NEjLJ5&sZM4j%H{K?qj(=l{QNhV$*e+=NvS{vTrkrRZ^f&#?bK^nW#7 z2$ZQ2_wxnE1Q@y>a#R+7)%1Vz>hCj)vcNjo8*`BTHLia|7s85o zi*(Gz#ig95>HU`IKi>(!J;W5)6t zhrWRUu_%B=Ff21$=qUIewyWO6i0x2_KT-9_uaeak37|1>7}+uF z<;kPa8;M+FWsn;2q)j($2 zb_H<~oZ*tXxkkIRCmpO&DBLNeUBl3ex2yj8WYeW{6w}Epd^YZuuIjUwL|7oa@gwb8&OmpYJdHHhS92hB*LScm8{Y>S}K(k9EFozv(*r z#MR)#J=ZqiTswIbc_Rh$K|B#To5yvej{7TAwN10IReNGw+)ADBoP&1b@lqYlx+Tvn9&(qfwQ;!LZC1t|iAKzk13)!!9ElQzCMShBIKx^$Zo;D4sE6v6vB=`h-5 zCXFS<^^GV{#hp^M-!>)zSQGS&Xt8r;Sa6lS!)^WCcfIqt&g%e4enw74#%}FvEFDF; znBO8Sh`~h{Tj30~?*r?T<(CzWC&$CZurIFv6V*K^)f)2%;5YjMt8Bu;5=+Y)0GYd^@_{#)tK# z?V|IFhj?4NJm?($eWxgf0?TIJLN~4If48}QM_CRds?TL-&rB8!z zs6XXAJeG&hA?g=~j||J|P$#ekX8+exKEQX}7&Gb50!F}4Z;5?4ei-|?_ZMMf+$pIi zHlUo)hlrHdp2!m)x5?l9T-l7(AH-S=UooN}i}TXaefg5fn?g9(1P>U16+XcDp{xxx z6tDs@$A9(RSR9eRQD}DUtwX;5TZaz+1v?N6Kt8~2VZg@eflge<(NAp#58_upc+`*|-#a*%L|}lU zUI%7tvrwNqe_C={+GTisSn56fZ^4dPq>6H67GVgur14Hff29b6F*o47o8i=oVOP1* zRl|sh;Qc~RqMKV-#8&65ct???SmOCnZ3Xr6}FQT4-_p#8Ln)Mv;c>R@Oa#j5Ru@a zFaipq&L2)fYVo`C;jgF13s5@iMN83=_UJ8zE#8FnFh$tT(XJGfMknZ8Bz_LL5`$N>yjtCs`&UWd^ELH#+1 z@K#I%-~l<49O>%42?`1%1QrFUF%Ic0l4b>N7{OO3o5$SPPb+0}*s})!B|N29(w{sW)I9Lr(u@+AaHF`MBI1na8twy1tZGj>zPBb8nCKeCe*6gZ`w>m^R#Ug{g z>IT7As10pVXsjaQ0ZB;UAXr&+29FPX&qZXWgNtBQVJ{Tk6XD8)%~XHV{SYmfw8>A} z#))@{$uzhm1xjgp`9$R35~j9687#^2Txx>oYD9^fA127q`UoE>n?D}Rp+XcfIiWe| z(B(<|7$cfv8EtdrO!GftvGjy`*3kk~aU>;jRXabgMyK_5Ip4m# z!#^4+?uviqTdd6G{O!AX8M--D7*W?qDh(S zgh@WPlbXQG4bWp!1CBvSJwN;Hi^EiPlnXHGy;32%h(<<*TX|o|KC`_0vYr-qY{iXG zLg0r=u%O5CPI)R!G|}aANo|m%vVCQ!lOeOvj?+qgqrdGg2X+!78xHQ`rQF%QV~h#x zFlFiq$b58)ytW-xa%IziZHdTDf&M$X#IpLW*4?#TS-PLWrbU`dnW7h&v#RE)Gc#gC zx`Hkrt;X0L?~RyXU4M^NYR~i5105_FhAxXn6q)$WmJXY&h0iSl3-OnoO4#s$SW#IW zcNW%+l_IuM!$@J*opYmJIyWt6cZdpmKtMT~N>8$pUA&0qmmznG4Pmad$sc)pJZf}4 z!{Y3;M>+L@@#X4S?v}zCC6n`lA5q_s2nI0M@@SDXm8Kk*wAH*FQCYN0fB-QSR4{?@ z!M(PEn_-(3jc?6``eR9bLYgZWso@`sL1@FW-XELDZF;u5i|tU<-|wPp;i5ll7Lh3h zWd~f?fD+_BXn-pqZp8s3dOeL+LIVw45Cg?qAUo08N|v*BMjW-1OhDA|75?LCspw#d zKtT7C?jOp9q%ux%e)z+OG`=9(e} z7&4)PaUfSOiiVKDLeSOK2ige+u5Npdh^mw?R9$K`$>n0oolSQf$H<#?Z-4LnKZLz? zTvXlKKCTFY0)mJj4Fb~LUD7Sxk}`C6hzJNscXxMpiF64=Hw=w*$G~?FKF@RB_dVzQ ze*eK|_UygZUTfWP-Pd)`3&b$+Z)Kus^7i+Fumpt|4UZW}QorW6zo2(^^qg^@B+5wp z>5k+=YxC~vjc1kNa}I-_iuozx&*J39zO*gn#D9DHSn-^6(;1BBZo-AHHJ}=R6MyRW zx$79Q&TfWpEyunDk&+%)MCRrT$9Ub*iH~db<{(y~-rZ9(57D{wjoxLV7XBG#O&SQn znLkeJ%Oj6|s>C|aNGMaqFRUA!csq*r3lYneBgQ?#_^DPmB5Ebp&4j>x)t??nF2uwF z`yS0#bW=;OW`I-xA8RV# z>1_v-4^x;ggfkQKFE9IGe*QA04;ui0xv7EFD$UNo@V{U3MFeKPZ1EMJ|A+7Y<*fK$ z1Br_~R|V65zodf%NR*a2Ln;5sPyYRpX~5b{=Os_SdD!)T4e_r>{JGj^8C?)!YlANy z{WogyU={z@tv<^m0QrZKoCOMOj~_n6=W>X4vbqoWp!A`IURz4)x#iM1M@~+zYc=Y` zDWPTcBaEvbVr*JI23eZ)*+4}9%|kISfAVGFHq=wOnz?J$*VqmL?p01K~w7Vpvu0bB9D!xQw2^oceW8y}zVqPsnHiF$3&YH*j5vNGo0Tvb}5(}9e93+I>BRfu-! zCZ`f^a2g(eB)jFbFq@f5o@djA(r)b(xX3yzr?*$4XH64QDdm=j>cZ7Y#7=z^ZBU@; zCaK>$5lep?s#)6_+8;|}*n3+*C@q2omvCJ0xu47Dx_0>!ol_E1%aX=l!oH zU#o^alQb{F<0d9OPqE`9v|_zOIO;SNy_7#QZT)Z(5Yo?|lYCcs6l3dEYxvV#1$H6i zi=(gjBfc=-P4!GN*do}yR%gD~N@P5@y!Kpp*FQLzNoyZFaJq>xJeuwvFn0CzMxVNwam6ZisPodXurGE`bK8*G*+(Ci|kdQ5k?AzFK?DzesfcKj62=fo;^xB1CS1T2m2?_N;FeYj2W)4nLPqqSc^ z$9fv_Id2nwXYR|r78ylMUXRF_zny22UEj@0!Ov3SFJ=6s2oW3>;0w4^FI7)R-F&Y_ z74Xf6d|+M&|NrxPEP~W~h#}(7x6I7Y71W~yF$vj9YjbYUE=_B2SYd7~5yuV+4V)aw zeJg5EaI%*oVRbNuIB|OIh`ifUW}$q7hr~X~EE#5ZHTn_xp3{aZOiWMG{o_(wwc|;x zHh80kV7K+?xsTwhLzR?FP4WEPi-027*9c)>Bc~t-6D8Gx97`Qjo2GsS2Uf3N2vpnF zEaWg5=z!Pa#+w|Mn)M0B8sP-_1oz3!!SSrj?6{sVjddLy32b?MIi&n!R z3|J0Gso=A3|@a|I@7MjqwJV)MW=KX=SP-npK$dtpRlk7 z)+tPecs%99JMAJ}{>su}-vQ~P7jVPC4s&Afz<=;`Hl)OzDr7n`nT@Es(>e8dde+oh zr6@W|MgHu1-80ml~;YU%OKYnZtCR90y<^*LYF50?_#h;Ihv6cCJtU9V2 zsK|{+65JRb;sbn?txB+t*F(>@q&{L^lMs;&ZpY&arb+L$h z^a>q`>9PDS<^;DeTg%HCz8b(25l=UBA3K(;Tn1wM&%KtnQcPvq6lrpmZ@H{j(xG|g zFTnF1%d18LPd9(2M)vpQv+QTin?l4fk6q7NJ~y4fO%-ON@@)1wHt#4t``Y+wsxU6B zuxX@s9n%F7$k@Wg@Qst0Ih3kK!%Iu7^fD)2M{K`N@KPT zF7zuaek`R%GG54tS(OTSx6b(M1vPu&6)JoZ74Q45^nimpbI#$C=^y@h;1DNtW#G72IV7R(vX7s%9=pzZay zs@<%yS{3)x^&uE8oUh=KzBYcA`eVyDW-Amb6o;BdEuHqJWr6t!+aB2#ih{=mo*cdS z%_r#Y`YqTnAPm95Q!9MaN3CAH4;_yBFJnd`=#ZdLFr}LyIX*w&d9;dgDR2*H2r8HYh z5elu&B>|nrVF~zc&zwsN#+toJ zh^?$xJFv9C6ZJxSV{vCb{yPj=QYcYU-bI=?Ee%E3nfu+QQ)=w8$??ogs;S*Kh^QtD>jfFDwdp&mm6N3W~vf(%O5R$_X)#ZqGZwk55azwFT$cAY-{<1sn)pFa+ zIaktV&w;p2(Ltsf%spm9%O_WYcBUwMP%V@?S}ppjP@$r^NECs61P^aw@bZV4V;c4L zkcNq+jO@1Ks$@9i?s_Y*db^H})n!`H&Fd6gYri%f0XYS%H}4F}m>@AFLUX}iQ%zS~ z(j}Z*h>;f>in?Z}ov&(CJubgV<2$vv@7bEPN^vnQsBmx$89*IDM8l`gx4_zRYPtH3 zY3g5YD}v-=ubV&VSGcXyk-mSbw?Nf-`SrM_kR$D-+9uI!h%LA|^|7?D`S7BbNsPX&PO}axnQOmWaf>v$Pko~7PgMSCud6t5v5^RB zp3|=u{`1H4#o7Fg5p4Ft8)ZrFu9(*~6{TVL{8AHT4csFt0tt5sle?EE)+Qh{l|g$1 zvd@CwkX*F>94wuqtk$8M7x-#cS1D>GVk&z+YN-|!q@=!27umk*5(<1U=RG$f$jUo9 zXE8XnBk{g^b$H&aP=c%iZlZp@BX>WZlHpadv-j!e1lM=h44UOD`Kb)^uaOTBoTSfq zv@$OH$9kvW-EW(OD$L<}7d(rzserScU3V*rEz3yasjH(?lC{f4-I{r&vO&48ED;6<{xPG;hy>x z8=_UyX0+Eq>Vz&hm?@Kg(1?2&B> zEvj^bCxU0qLf7pWKr-=>fME@W#0zF1W#@hRbx0y_hPLjyUNSmc4qwRA3WPkGwU>xe zl9Cl#D&0LgcYDtx*RRtPb=$1;TQB@V%)COZSWYX1SPgf2W~X~xW9VW6n4g8&ctXGj zyBrPLE${nQ9S++`;szLB!THcH2mc8K4j2seU?b{fFPU2uTKBo0*1>%l>ndqY@!ks{#S!G3d% zSC)m#Rs`;)5RaPYrxBtzNcH?J>ZR$SWHw(ML08hO2n$U~WH0;0*qX%ZK;F_kW@F7v zXM0THFQD!XD?)_d>rEFi8rv>DYK`bq4XCAgS~`~|^F;}cpb|+gFJz?<-xgw*$G60Zwm}dR@MUVw@3(=T-GmU zoNvdimBTO?-i_5Q(N;~gMS7iu35-IWUpQT5l^5Hu{d(EtHotUM!m;Ty0bkZZvpfVi za+QlsUmCoN0v!e>y9Fw)7d7T8*8HTUVYC~<#9HlVBcs0_igZ5d80v>z9H;%gN43nx zCMKh3(%#uw`IqK*QmtnW44@7MG6z_@pI8qM!*&A`Q^I#O~O#l^I9jguS)GVW2_ zj@$XNR!iq~lPssd6RODg;>ddAv_T$T8x`33IwxBBPA!_V>LDdoW|EL4oV=XT`z+WN z@>;y<6f;iy!0tVhU*CUq*jGK0?;OXkoNEEvTk&gh+J6W7?y|R6QR^QR;E$H^#Qi=8 z7AAHN;f;17W7qhjCNR{gag@#UgX+*i-U!12%EqPnq@dRfxg}MmUA%b?ckvl+w9p9g z-&HmU%EQmb>k#hiB-UMe6M+1rRuSxfdgg3jO=OZMe?>G=qL+j*oC+aY8wPiV2}lrF zS>~l29@n|7Mjx^6Y%>9V$GL{cK4;(3*3&-m>)v0D@@-$z(xk=`oHn$}5omZRz>1w_; zIBuStW$^Yd{AXsp$ADdcwc#QNPJS;8V7y7QW$5yXwrR2HrLu9&(H5Sm%;J$Xm}Q`*H&Z>@o21+1~dUFu$! zICtj*uvdFgC+ZMT)h&&j8RBs~j#drR8$uI0%l4|(;`;QKlqLj*C?(-e-@Xn)IvMe; z)LtK&Xg;CJGhIBFbPBGYI>U=PxIIE>N{F|iV)PKo8`U?$&C;9BTC|~7sy-w%DqgB9 zx^SI9djK;^kBy>TU4e55o?_S7khylTKR=s=4s|OiyL>l`{XK>U)pXp*#cnY4OSoqB ziU<;WT{X}&`tAdBN6oD5tX@LcTFd~Vdz?WVD7?AS>33EMlU>`S> z++7qB^Z|?&!Z%!C<`%~BJK!tuK$E^L_OX*}&9g#$piT2@HC-VLK2#vKAwwlXLPv7+ zY2Nu0;zmGEDj&it04y=@6kJy=jh}vF_N(5@N-{Y%rbr=_?5!YiJp6GXk+Q%+;dY2c zLR^U1Vu+Qmxyl4-&?)+I6Y}RjrIr7M7CqEx{si-w0Em@^l>1OAfI*-C`sklO`NC{p z{3l%MFBIx?6;LSWN%{!1vi>iRf{|<>w!LX^kN&ymzddQ0769cwy&C)<^nw3#G5Rqq z&%IK5^yoj1^5KrjFcj?Xfc-y?Mp&2l?Ha-(*k{Co@G|df2&%641i%EJfN~w-Gb5uM z(Dn!BD5~+{EE)So)${7&=+BhyW*g*rhGtBDY&MC3IQ;00Y!36ig`W0QyeM&lq4e>p z*qmu@np&W#?GT64AB1P9@V{`enCI+np!;EcKksq4j7*I0b!>VKNdrJKjV~D!3SM$K zNZ+h4AzXLdU|2eNHudfWMUo}Rrny4Ba(5)=W<#N#YW|mBKl3N9 z%@Ly?Y=u=)?IRsAHtj^hn81PoHQ!*rpalF@_Y?_Ubm}f~E@X*zi@r<+zA(6k-}>*Q z1L3&Ig*~cP&Gbf7rTFnz-%+-fAz*+pBnpb~irPn&jZ)?B* zx!+@!i=R5p5K_y9xt$lez2%f;wXY~?=&o->_RTFbw63msW6QuH_+yv*^~O#76(gBl zU(J=>BhurAA+`$mVweE$PF7N_e`I>4&NAr#GUB21CuE@UGV z|0@3U{Y(~V5GNbq3HBLFD?dd!-%Cd0Fq-tv`dsZ-)P%lI(FvX{x>1$i|8^=ls`t26 zd8~Nhq9J2hb2_N31+h&zo;p56=(hUJ73DEsa1=jhmAvD78xMch_-?cDWQQ3wU)Uoi zou2)<8YdiJw198y+0jvQzpb@#`kmK8yT}wm>+$KM>|F#Ce`9u<-O7wr6qtdVgYZ?U zA|DvJSh<5wUFIVJ@{n#^x0x^AvJ#F}_2RrhA&g!3| z=l>EgF8VJJnBo2%u>R}G?VP}&W*@E)l>2YS11J{4XeIt!n7v5<)xtc6z^bO4J#bwA z-F|$01C&MaHWY|S|JV-zR#!iQNHx}o#4S^zNIiz7$n9_+rleAW{|rC%89vbw>z=he z3pw;Q_!ABA6O_b)09!`#&&LhV@oz7Pp$&TsK%CsirelT7pwswXtIi^4e0+R9awfJQ zx2{CJ*BTbBt_zkkB@IO(P4$2eXHg*uj@-x_NF> z<9SpEa*h$y?y7EX+#xt@v56dwu}>ES;#0k^5ef%(d7(e6qFtCq_uN8}vnyOawjt(A zwwTPG<4%Bt(NJB#GBay)&=%%8&!^Ra@5&Wt@}dXy*cTK1``t7&G)~WdN{$!nBR6K5 zJ0~*U$(q@FYV9mQ=x6J!z4IBA$T>bvR)C}}-b_@bBdS#yAafq`jXrKs|Dzm%r%m95e0uJssS z%YM#;x!Q+{)h=!8NA|v|0E*A?7b2bjL&P&%Un4+(?E=8{2nkW0o`@e-$BfULY(}&A zcYla-Gut4Mmt6@3ZvIIPi*)Qc^ z)JZF48a=!IIICNN>b#=+LpS#Ikx?WKb`q^X>w|fwGaC z?KpCEP}`ta(<#pm_(FFIuYErK_2KDB$U||cPjX}XD7-q~TC{`n@maBLsHo6Jw&^Y& zWueq-OYBV(Z?TP^WN}T>SB5k?noXfnVQ<9NdqOLrdvq78RSsLj?o_U4BMv(g&m6ke z>a9L0EkfLTMi~SyXi1&86A|OD?rQidmDs3q99UtcV91+Fmu+pSo#&=JcZAK#Y znRPUadast@61eKHS3_sY6UGpy206Md+fXWNas#z6pUqKgR#Q42xFYu1c|t18+@S1# z6)wz;@Q^Qbr<*=8=rp@T>`au2SKO}C@bdDGyp^)9^yJ-`cO|qg_40L7&X$Pao#Jv` zCEHu7ODj|^5mr-eWga>8i%8ecdP?t18qyb%#G0$a0~Ad{6VTBg>rMPYuf6f<{>pN( z8>ZHmybsd*weMA03oBKt8PRs;khMqlrz@hURT_nh1WuSEn!Y+gtCA{R>WjF~c1+LQ zbaye=?8d7Q3e3M-#(9~R(~)9hGdRw<;7^sJPE0q4W?pg&63N_hQQ96XXhLj{uTif> zt(MXT8t}^IgGXk7;M=y~F?8$qT0!*BmLzZ(eolV!E>Qopa!w9IxU-4;?bfI#X%0`( z7r>TH#4QuYZuO2!+#rcjQ zZ1XJa1p?;{k9isofaj|$jFQ4W4QAr1n0>hCf3g+BRHg`Nlm-1P@1g3G4I?q1$@6CQ zEnPW6Aled$*CjY9WKTVR)M~6^wc`9b^f<1Lr)jARg0xhU@+q>`;^^2x#jjAg%FL4P zuEjH;NCnD$a9IIS&M-JHXYd+Aaw07MS1DrE6#$2wvu5#m+`cz1O-A8PS{9=Ql6J0F zn|jdE-LHrTeYaV{-ul%c;vgQkKVGeG?-tebZO>8Mx=<^NpYF~iP%CkUXk;?gnQ^7& z+3E0HRDsZ*n5L(BiFv>auj#$UJN3_QlSDgr3wef}6>J=~1aly3mN0^<;W6E)sXHme zrn5|V{<;#OZa3LmFD}2X*EyM%np&UcbRkU+lU93e@h2JpW-}K~v~>AHlI=#!u7A~h zl=KNvcFV0ykfF9U!H}!B$EDJHlZ#Gvi6`aXeOvNOo^X4p>2cppN0D`A^g2bV-mk@y z*>;)DPGzzYtTkT43y^Lc@=uL&R-~hnM2CjrzU`+t>${`Fd%_Uv!k+B;4D`G%f^>V^r&)6%>w7g6BiIEcqA6UzkJz zgC8?PZaLlEH}}V_1@#i!w#uu+aBdNQ0l zp)cdIPp0?l|>WI&DMw zeB36N#M{+x72g{ z0jbN+sdle)&K8-uS(5wSWvymw!e?=6T+Y0@Uz`mYyIYie>Cq!v(D8(-P*X(;YR*GG zm@I0-fp#B@2!##^)aTf>;C3zPOQANVy`@pfT+^FU`tdz%8{OF;pB3UsZ9A3X#9E*& zPqY$>n)}U$`eYo5+V&!ajBj~F4*G5EDzqUIZC-w~##~V*Mbl~bb@$f5LvYQYr556>Zi>na#N!9Y# zlKy6IPM4o3H_Ct#`|CWopMorOD?G?8XV04!MAp`L6E4r;rDP>@m~Jt*!c06qYB>_3 zS-+hFbKnpBaKE0H9cY;Pg`MUtba5iu@bU|%Cv=^6-}Rnv=*}Zn-+zkn3)*q| zyJgUyeW5nsxRMPk=y8ZuyROVcXnJy(4N5z86=;@F(_Qc5HC7noHC;^xZZ>Qtk0*7e zxeRhkyT_2^VR_emn_c`RhbLV4jw4mWcV6h2WU6g4z`Osp5^#CWd2*SH9v1-sqf>E2 zGYm_Y0KuqO=b5NxZMhcaTyw#s3#|y6dH_N1ntnN1?`2u(7e}w|zWeH~hu1@SqMWRJ zVu48k+KTaqQ0`|doCV}yYVf<~e_Q3Vn!=6D@!|A67_>ooHcusl8)}F<4nch9C9*;u zWG)%Buox4x=lUFLf0Xf!0kl`lgZ+!S zQ%blT^m-kgqL-vbXll0M{UN6+*Bt~4JNz_58!_FXRx z@wr;7EQ(aGd1J*jYY++=`T83#-4m8DH1E&;{081};cN>N6Gfv7p8J`S)-C2KqIq&D zV|Q43?kS#~No4s-buV&k{q>}YuJ;JGht&%e6=eFN_OIU-L9?Nw+RRCRGg zQMvb|Q@PQvuxzbS_4I5lx89cz*n9cqRk!qaiAZ<^3BD3eF9i! zOG3%GS=A72!``tMr^pw0O3l8>oEo1UgP5yfouwjMwiNeru?To^Q{ULV@2;>N_w9gKR3Inb)38n4mb`b9!8=TG$qFGQtFjO{?I>MDw8a=dcgQF%=ENnF#hB*ARu zgd-VRz9TmG;=GU3R(B8shSt|?CT?Hr3n!GZj|?YJou$lFM_%T5tjAmeVJ8ZjScLQ? zwD*DpuA-t8@oxy3+oM_5f@{M^9O>$Du2pLb{E}C6FFP}0`FUyvyoWsY-WopP5yN55 zAof+0E5td@h}tPF*=BLgPoWLf){i)SbIxjHV|`j|zuAlVs*^8xu1Q-KH=I2C_=&_B zNwb3!nv--tIn-jWB{|M+e-Qw%HuT&0NRY1`pRJXiK0RvFRWH?kS zs={8m?(^O^H4t4dzt&`dxOr63j3tK|JLB!&2`twdP6s(Fm-u?=lHAV#S`>RM3yC3H zzs=)N{Y4Hvp5f!|YM|gd>wB=mXi?+(LF{ z)srXZvb1c=AV-jHaBrNRgIPTc4cM6}axa~uDfSzyC#`sPKjqh^ecL0yxj!GFC!d<> zld;PsR5cr z?VVZbzNfcC%eyAqS6b5{J@+B5F+jfYeq9eocIDPn+N-$SFPTVuP*QTXn6cGRp<5WJ zPg()U&9E%uzyomeSq@)7$n#)Z97547CF?rRDgWhrMEoGCUx7!kD$4zrgE!eL>TZJ3 zR9a$M4^$9lPlAi##xF!>it!yl?fMp=BRZxvS(E+as+>jruGw#_<7n{ohNG@kYf59FTDx=e$C-Srf%h-w+|=z>E6PGM#0l}m z)Q)wCLz37hRhO#V(fE$0U!CsYm2K2=wb6?3?P5EL?6IzRr~OKB&Q*Ztc&fI+<6Ufs zyBX8>?fH5Q*`%03&*`L?u{rl{QQkH|U0xdf50TDjSG#(z*z?6%#_j5MM3!d94+}L6 zDasdn(TlQWc6kn5EHV)@vWBsPGR~5hB*8NSYaK4v3T~H-jrEsW+&7DeMv~HbmvRJD zmP`EHLsNyv>5b1{rFhf+Dm&P{(7>ku@T2-3lmrb+BY!)pGOa{&gTp?d>LD#!YPmxF zY(uF19e}LkAuPhOj?bU71TpTVJeL+cxA)x-GI_50I#Qpy66=oEXKEf$^aSkov1hAA zdu&&o8>q@2P!p-y%0CYDd)*9#9BB*|$`=V**EBS0V{VU#)>ARrj<;rVmqJg~gF`}c zywPq4M3jsK)D_$2uXWviIIb{Nmk3!!?K?;(iBThbI0o?2inycL$K$K6=l^~?HUvs=jlUZ3#!g_*y83q0tx zL18wj{%RVqJ6)bBN|~$pq<^hx7~{&P?|7N7HOF@?;LCGJk0LDt%^-2e9692teUtPy z30)^By8eK$#Fpm^50qGtn89a24cRs4C(u3smf}vOfGBH5!f;Q;gJ!uK6)y%iEC>H=XDWxS$lGqt z#UrvOX|AgdwZ8;BX%o&c{z>kh2-@rwaxU1NHE*Mx>W+fbFgl%h5sXc5in><_iP4SFW`Kyg1K9FnF#;hgH<8%upw+)t9VBdpGsQ zQq6zW-Frqy$G@j4;Po%=z|g=j@6$XlR@2fJ*QUTIYx2 z0XhfUKIiD>bG4vNruH+EqkwfX2tM|KZ2-F#TZ@<0okyA;#Y8h+ha{mt1)5y-VSJn9 zfSQ8GrdD!rhqDf+|2Jl1$}dwfVw=dw*h+zmd<0RyuF9}YPYjY+E47?_h@w4w@QjLR zuM<~iYcgcK9)Z*YF$gcd*3HGEy<6kd?r?5vKn`4D$dkFdA?C~MY^XwmgXewO-u^C? zk^~h1pncF^g(F5RcrQIFK^$AVSQ@g=f2_vqyy!?lm)sUkf7F%4--|ug6&(93#rbS( zWCK)l>O_OR`V}}mRpzWofD5tf{3-6#DXl(0Rwr1dQ#HvAy0|{Imj|!aEoA`|P^XiU zSFgI|vr88mr`e>+)@@OCZLGt$l68f~d}k<8djxFC@|Gs8%T1QH9Y;BW3m$U}W3DhT zA)(t){6n;$KS1DH0yr|RtdX))RytJcAD+|c+?EU&{GMwH3nk>*s4hn3&{Zzb^vm_; z>;xqYh7+xCc%38g9gQP3tqUB5xewX-=LJCmVsJB7-kcRLkoe zZ3MW6zDmB`b>p5c!A?nnHl*l1yNh`LV|!|ej0LUA1a*@?MPwWC#j(>|1nYZEUo*E| zw>!79m?*P5=pGqgh%_orzVS_)ZJAPTCyTnAyjvK3LxU5IfJt`kVgUe6pNa(R1$hDR zZ}fE!JE8Oyb)IfZoR>yk4l2Y=N%(z(otHW1B=cu)VueS5B2D{I{xlhEZ3MA;*D6{8 z0Q>8h%V4Se6}upmu4KQTErH=@Yy1hhXWJ^zoYKl%byJ|(M5$~i5snbi@TH*nVU+*p z3}2$$U9FmItQUsn1-Z^~abJ=TXz(U@afeKn(;u^%Z_wiYx_HcAd!VGazd|h3ZSHw} z^of#6<3izurR2H59#1W4VBLZn&(KE2vm(`^Lb~u1gHvfkQ-lk2jjNNO-#@wT+0Ev^ z>AI~LYt|UlOy(TdAuxuwm&P&LMDfRb_`RvAGA@Nutb`GyT&^mKJ81|-dycfCv^4~`wn=2#jfbl6lE z+ZN(m*?Tsgr6pgIiqC%gaxSbLR-w4Ld66wn=svwoAKopS^lR=F#i(?a_y+_+k;&P) z8SbCa5U=NkxMvpnlsC>qn{Ahg^d40|V*>}lpQt%^mkcIyU1+INmjWAb&iw6Ib?S}v}YrGm|Y_0*-dXS^5LD0A{Ms-rQ=3@YU>m# zQtM98kYctl1TN`8>t@VHj%lY`sP^?)q_v_fdp&_!w@|9q-ICd>EaM=&N>ZtqRj5i< zrlpkw7pA|`2GbL`r!bv|5E=?h0VvoW2bX5g5cDa{W4KoGYPI2;bzfp{3muS${&Yt( z5#YKsw8D%|a$im1)&=cL^!v_t5dSR^Y2bbo=`KM6G&id(A zXJ^6rgk#NO4R!;E?Njdr_oH>}f^N167!0UmbpQn^fu4pTOl1rWDOVVXQmYh_LC>a~ z%)qs%(AznA-dtNy(;4njiI4MPT$4DF+xIo0;A?xuZY#@Q<&XK!246XJhHy-msJ(e* zIailMr0Q7~g2wGM&KapYI!4^;Jfd5otv8q?Et|+j^}YBb(NheCE03%P;Q@7zh#YCw zL>(0IJa5(G=0rqU@7;YX&>jmIu?H zt%X5Papd{~IDSqF(w4y7{Mj)4DUW%0g&{fF{VlRUIIXNZLnktsox^KdobB2frVw1S zkRB#SGXNYm&Nx+QiFBtY+I(sT`YJcmEUHCOxWT4;1p#4p8h+jdY!b!c^WCIZ-2X_ngGLlKHnYJ+tv$d6!o8 z_ACOU8+`&5^V>ONHwOx=X+)tJMs57jj1a>T{n7`NOk&Co_Q_`ZW zlT$SePz!ZNzLmk)EHi5C6byvf_m&-)oB11}V_GNnqm}Pujmo{8Mautzsne&A|vWtJXiVV z?Af{ve%*Z&-FBC|HOsb#8+LZXb`rK!Fn5>T+aDw8zLn&a5qtr_Q5HYKtB&nH3{{sU zR}LnecJCloF9<9+;fjja6Amn>+*Mv?q!no2OIkxDywvORgm&f~O6}K4T#iMAU5*T_ z)cCHBIi=$7m{c?l5kz%#WVHfZJaJx#x(TGETxYOjN(HuF*gZLIvZ^!GIlM0PF{(8J zO>NqxSgnO8dqphj+~j*=(Zy~bEM7qrZW7|3I{@0q7Y|B*Ag*NmEQo)h9gDczfXwW) z>PT1SST_x7H3M{2Z%oB>*$Xw;5<}{)A{IT_%qIqGF6~zCj(bOE6|n4fE8%zdz{k`g zX%eLU%wCTmSgiyT;BJVv^UnH@Rq~*$xQaL@by->BGH%f`7J#8fSLc_+&44#C|E5pO zKQn(gga|{$cU->2s-AypV#Av)%xtnWhYep^7#gmkcJw9k%3i0J3ATxwS}d)E)?Y;h)JPRo>M7}z1H)cS#7^^bRaxu^BHe* zHWg(T6>XwxJpCA}@IHeyWG+PiG5jqVka!7n82;%y1KeVbo|rE%XS1jObzZ|Y%(5y{}Q z1>YBZaF~coJ1)mW10RW}rN6rUe2nDPp3ghYbh74sjHyb!Si}=UbipjC36BI{2yjWB zob{3{{l&~cM>q9UGM_Cs=JV`88%pXiv`bw^)2L0g&X9pUYq>kN?(4Wv{dTsQeUyn;&LVnaS_Xc{!jmo?pNQ;r~J+K zXT@N7!|Z-bwCfJ5Xf%~<#O>#?w^KFgl!PpVj{(*)Al>|j&IgdidJ&{zsU^dBuWiEP zFuGuh`(@;^7BdmWiJqHA5xCyyJu$Mw<}J%)#3Y4dpPJZ)OQGJ`gBA2U=#oqL96 zj=85&R!YH!^zhA?mme3vx}`XrMH;(VICx^mJ^d@SO1`f<4VZW~d*4rSjr>do$dnM< zbRwZOpu61Pqid&t!$KCd^^MR8C)`0oKg^k2(8`DqFTwd}rR8Juk&{Y#k{cHLG=kssbmTSsn{L-f@HJxsZBn32$l29_wmtgm zKnM-C^ri4yI}@R5{oDi-ZOS9SvX9{B|n4p!7>hp2b2{i4F!mt2Q{W zXyVnNjg@s`VO&vde{W!H2C&dFt~~)3TJm2kv?}tb))^*It&8)CSNnR9W2s-MK6Yi*zVuy} zI5Ap_SrFa1lc*wsvFcmvz;PrlFIi~ls^3JUq2_d&ED zkJh<$D75vHW8d^K=1Y^`-4MneG%t3<#$h)X&GYuIp_rCbo2F8T7u|vkeZ-Sl9KUVdh-SgK~ z=twXU>DR2Uyni<~`uFQ7jR49NHZ!NwU)>1*{Xv4n=&&9cNaB6>uEm`iTYXH6_Zmm; zFP{I%B(B|drZoE3m>y8KzlK9x4paP0_mq{}d7~YeKJF=cZXWP-QC>(L^Cvs1%(vqI z+|CL9i(oznVZezHb)+=PSHm|luHT2&)z_SoPX`RXJcBp>pOm-HZSekRu8~+CMENI1v}2$7!sM%-8Ch$;f5A`*G)%k+JJ&>@^OA{}n15a$KsbF&g8mY{ zf%a0Se`MR7z=@Q7ol|w0(r}D_*p{=$4TKMqBxA3d}EtPsf8vDe3u#*WE%{aw;wxR!Bi$e>&wXRma&ncYzu!cfOYE zbM>I_-yLL1+31V1oo7GCvE1rsRmxMv7UslZs>=!qp1q;Az?DQLOSel_bjG?}dL08H z$$PmY_rx>at}@dNB}5Hw4QQ?oKZ|sF9I{FMAa652ByFXsOaWERz$&oyvK+ zIM)rKYP4){yMZ6!9kOjM@{9l1Zv0MMZgf}2DzK?~4^Y$g8xq!;wP0#xAfZ#m#Z^1cHN6qJcCWjV;K~(t1@?k7H!(Z;3-fjjzFwgce z5g-ULhNjK)q~c;2NN4&q%As9p$(L5416)#1jz#ca{zeFzl{a|)&-#2#6&!JI1kZO> z)lBc?##%l8my}!!51{QFVRUVsIqT&N%?DqWmCP-aTI-bWZ=N}9Ep?vaUNFb{8$MGU zRb;j8chSAKzBYuYnt#|+VQX3Fze=VAICZ7}gHxx#N#ifX1DPzLix&?cxR%0X!GK>5A}-OgZ}dB%=Zk|vH)LmXkG57hSD`{eOlpz|04IukkH*(!uiCE z!|wY^d}7Z#v!ml(dxLjvcOACKJ(l=<&&E0%sKjUFxWyX|6lyCXvYLK=TW=(JTuFXX?Z%t9MJ5xO*XzT#FNR2NVyaj)$Kg>1)Vf)D9Wc3vTICPI^d-q^tOy$<&@w4Odu4jfznfn|| zq9<`fHGJ0TmH~J*D;(Eots|j=gzzqx%P*-l3csE+YHlx-pzTiGiE-362?-z{kf1vN zN8@}yTy=e7*k-U~r5>Ikt*e4Zd>gkiE=zlzZ%{E+{g_L<;fM6b7WFtJfymZk=m?{S z4?id3TqqR7Yvqzt93JxNGI=guCSHhA6 z@bkd&rTJ+q0T*NWP!#c_;7!W?04^!RgGZql{41Q%S*%W*ud|QTLpNXH_t-^~@X*`I zESo}O5w7kw#4mVxbi$Yo!NYF?&_AvZiv` z$ClMbmckhHbP<|<{<-Mz&}MFf-TKnEv{X@+r_F-p=qcP|fH?zAhe&KWr(+n=WdPjC1Tal>=$+d>r5(u zGtFnC>Sa;mKh$O#;>ufxG zdlX?xzCJJM3mNN}4>Sfrs@_eN{o#lNe+IdIvgeLq<9F@r4djmNs}WsFt{yPrFaX2H zeu<{8+CZkA;@T-e0&jDi6d*ElkyF`}w>4B48UKF;mw+Xzz;*!ijW;el#}T>l^tislJT3k`*ewqMEmU zGy(#OxT$84TLHF;tr_P&L=4?jsR$8MJt<9q+ z%wxpQg=%`YB+SzirU5;u`lIW0Q@$%!v1U&3c;~actT&Y3Ljk~mQ2FoMV|JhW4$vEW zeu@TAk-VRFDCL;1D4@(=UoM-VmbP$6YATHAak zeH2l`|F-kcoIDRv#nW?XromW%DUbhL?;Q7)@} z_lF2FWfhSy{9_ae1Qqk`M9m zt9~8vR1f~)x|G;1wXq&gAsK<8DCYsT9w@Dn{w@n=nf z?+Z~MfdU_8a+qrxlhQo+K$6E|0pr&Bfrz+9c|r>IBASj*s3)VjscH0+H!4-?a+LVZ zl(yRS(RTX_5_&)rqB>FS!(iU&aqFC=OnAD_YrV7OFg2Hr2@XwcEsqW+#!8PDcBFa_%tB|_Wg^>s$UT)HB{Iz?7_2xn1dRM#v@5-;|@FsFM zhn5gy1{t_=fVnK&F$SE6{1~&``O^}4o{9=7dzpks(p%%&*WNffJ5DV`5q=8+PRYg; zIJ_K@@J1eTJ-$Ps_UNTqH)k*Z)en$jBt(4iv%d?|BP)xO{tOm*`N{Cv6|1jAJX!t` zc3ywnR2y~wL+@@-aP4R1745H)##YFw33vcOShRjEQL zLu9>zwO8g%{j@QlD#t}9K(gOPyDyaFq2*dAR898YVD^_s`a}w+P3*UYPsZh28*C}# zm2~i}6W@Ybw5_sLA?`IeuQjw;&_}0wY0ACDZV$ET?yJvKmwOOU*&Qfww<{$SZ4^WC zkyI(;N5lHvZW@{&eiw1~@+f2}H>}VFvQ*;o^r^aX#gU8zL1|{_BXx0=4s1>3*EWUJ z@4F-qXsMCM^3X9~v@SWG-nF*Mv-i?BKChSKZ3L({uI6Uu=04R)`b&ySl4>-dO-Mkp zWV$B7xuhY|rbrL5TER!;+#v}WHF&iktXhw4$NfpYv!eLjoG6M$A6P??$1Ak8McKJP zec8?3;9}@A{e5rJ`<70$7^xzjiu)8VUwfRjNy8dNjWObIL`Knm5VEUkyMT&?#tAh{ zfBK2cKRmr=JAe&^v%{DM_4xW}-Ao5%AIC+uBX|RVFDH3kwWN~~8eL0$`}5UC+-H@d ztixuPKMM#$CyVl532-$1qP+qeVA5ht#?vNE>MxE!^Ibl;O>l!pJDdBs zX@81wrJ=;t!{t*yKM(@?B1bZoXR+AiXT*0egentSkuJesoxGWc*h1^Zdd7`UyVJVg z=0}Uy;AbB5dj}GKH*E=LFniKA#LJTmFqaQw!qGneCR$~d9!axr=Y_a~aKxXBNcxk&97}>5+_addC ziD!Mfmh82TJEwQ1n*O4$;%u$ebR1Alq7(e_8&~jn=ja4YHsq;6Sqth48#%>u$a;eU zvL-z=_TuaEd2{=E=O8Pi&9!Ya;jBf9)npO9;5oSJ?)E~-b??{1^@+t;ONyuG?x#mQ z($6y4LLGOX4awM#S@IXQpWU&QRBV5tVc~3aWNE?}0jG+#iLI>b5O3KxO@K>2u#k$k zX;To)VRT7c`f(e%Ag}A&YwStR5srv%^2nw| zY1Ynxovp~ziu5$}un9AeH0pPcNNRs{YM`9|bE$n&hBx7a@s%8rLuZV2`%NyVpheRq zEirzPZ!-O9BF>sV*RE|AZ8;ZuONTScb}3)Gm(O&gBZFfLUF7I5ERt`xk|(jRFA_I9 zD*&MXrM%fy|9o$Zb3ke4@_65d`7fOI^Hh^>6wtT}z z#hzeRaag~nY2rK_5U{q$m*1N-Zoc4(h}2>r<1;ca?yKKf(BvK%-dOfY6g;&Oy2+Me zsVWnl3R$21NYyNYgfB<^8U4%N^Ke>jX)VgH?SkW*5BDZkzN8h!DrImUrchb=S2ZEe zPV&-71S~FMr3Ix2pGk;`4=q1{iXVshRrwGLqA2*g#8(DCQ<1rOLvlr$Gh;K(-)BXf z-~Q`{qnxEZ`tZShi%j6D5cOYMGa+#u&WUjsRWbD0xnMO?UOfH2$X6g2h24r;Gdfh>GCZ58O92WVCFsCzWwHuBu#A9Gr#6PvQ&){x+sbA zN#?f#RoOb2VU>b>b_ZO_fE%DtJneMzDo0i(5T;#c(O4MJXtqbH$UmW@5A)JV5?y~7 zRf6Dt{@E~t8?OQnse0r0mBc~gy8mETmMRs-^5W%Fw`b0oWlEP9{B{$^RW2(|R{@7M z;7eVitaR>$amE8*2HU5rGm|G;q#nE7rzyE_Qp{kp(o99qQv;-tM_*HOF==w2_O-Na zepWGv7DggP5_tsXy6N90-t@2!T3DlRmK%Srpv72FOqh$ay7SNggDt=X><@dJZ>sd& zkN`|Ehq`q}&P0hBq|w9QyNr|`F|FbE-Tm2puPWdPOzzlKLiEircF!uMq^8n}c^${L z1Q9A)ZAa+Uf{Q9|W<$7450&I|U-#mTWmysnxbZIiN-B-s`Z-*pW*7%sY<|YP%{sv! zE)zu_ACvd?nI&nvX4-I+@JmTSp5`%hql&%ry|H|67LxOAE^MAW`%67h`A0|$fdP?X zhzYx(a0ORcYr5xMrQnRdo)0;|*;|gC1Ffzdv}3=$ZggLzabJXOZP$HHuubllaucIn z^mKJjDTlV)0~6%e3n+;yMv#a-+jCAXi@j-~6~227P-Pi%9Kr?mLsr zoHlDCtQTxA%SHL!8*0RN?<9jTO?#)~2ZAsOV4ac(fS4u-KM zA$52{8ks;XJ<2+H+8cgu!lr(yq=IPJ2QC--6CxbbKAIaN3qkG&n890^h9V(#om6TC z?LN+zm`x;a{e)-nOe}6pg$aHJ1?^-c-r(YGk)tQC;L+&TfkK$%qXmIgNrETJhIJ-XoPNME~0HC)# zTH?>DjaTjU2F1doKNi6mCt#-*UL*2&9vFd!*gM?YB@QD z1k0Ut?qo;L@mH$U_7c7+Cf}Hsnmsd81;Ks4yj&0`r_HJyXy4%QexNu|o?Li$u<@e& zD65XzG~^q{JJt0mX}UaF{~tO>WpTI}F4Ce)nLBJ0tZaK}VctHf%V=z$(bvMW+fYha z!V0z1`j4qWFO9IkRb@REkBg3Hh>APZX2r+deu#sHilwehHyb0oTV~iu^eWF<7g|-G zElw*X6Dr#?J;JP=ZCl(bMu-o*nAV`SMZCIigZbyffXDPHzCWED?vYIUCR{dlBhpJW zr+wWU#{cBe{gTI)_T>Y`BgaZL)O?%q9l&lYFOgvpXffhn&`thckuObqQ zs&#gTNy)NcZ53}V*!9T>Y+@ce@I&H#Ib>IL(YA%_`apT!S@&Ho|J$>MhYWU-6C2#z zkScuLN%1aFxNQ4|08EN3IlbrkB7k+=hR$ltffUBlGg(yX`r$Hs+IJC6{MaLOen5?+ zX!cHYY;Q$E-X=z+8a9C)a{5h16J}NWw`U+%=&K+ZS;dQcw5vQB~Tc; z7*%9@D!9c`%Ekyz zDTxyz3-E`+XS65!Hpu&qkqS+D*KKAKXLws$is^1FH=l5i$v5s+l{y)iPAhHz%^d&h z^j|+k5{$Yb8#rWYqZ&hF-280h#!C{`F1eaXOg*jfRkkWcFv-5+AkV(=p&CVcXI9IW zoZTJ8XLZ|7gq@Q)_1<$Av9+zQ_BYUTdKWYoq9Zc|7w6>H?ZGLUYAp37y)0ZrS=WVX z)>mkO#%PnIo>XiL`c|K#(JmRm*q8A)A_SVK@|nRG#9L(vCKZkAk zECLxi=+n`yDoc^H{fqpe0hWf}TgkFa9BPsF1U0nU5=-iGitBC5#4R_w;Pr_bN{P4uR0 zLW@cu<)2^y(&p$KimpVSq?A%xZ5|%z3*$ZGt}Q6uPXy*LcWXj?`eg@(z%6*|xBd*0 z_2SzduI5HGhZ3^GjWbz(b&f(c9W-olDcizRBa~5+Fx#1MZqQj9R-S|)`)tyYR!F5_ ze?~~RIE3#(O?o8M@F6e!C9g%Sgv;LYrCaXU9k->{VkjQ6sj(_)S@ z8qA_TafQs{5j%^=F1SZ#&e26p1Y?AyO{!NAr+9<#zY^gw!dI`M$l4VvgBkI*9f)+ zMedMcilxm*`@=99I-|TZz7sJE>WP<9o`=OHtxA$@$zoo$Lk>n-ZD--@e~q3x8~xii z6Rg(Ss~KtLFXaxePXi)r8T?~0wyX)TGj8Nv$*07KbEZ{QL5tE&d%{(QN8M|+d%8!b z`q@4TPPJb!Vay9|O4JN&WuM+7gKEU0?4xrKCp&nNC{EJUJPN*nDR#|}>lYVCYxR55 z>(;uuO`admMzI@4sJ<$pu#Z9kaqWdn~DWij!q4?J&_!)n%`Rqc&!bZuy zbKn0+Tcq+l>Qb0kmlq{?XupX``lFz!TEr}Hni@Ze&((#s`T|Vl8YWsOeD1kZr%LMbzDQN{7UPVB$FeUInAaN8jq`|@8O=Fdd^ zQ|R(L!EuAkp1*2?_#x9KX)~@Jvi12L(-p)%{*t&ZzQJj*b1^kDT}*&pW#bk$Tujv1 zc6$>u{R5{#Fy7&v&%&Yqfx8chw z$Vbnn7;<56Y8)8$X7Ej&2hdfl5tA{>CFk}{g7#x1FT!=3&@ki>Ybj-F`tuE4JpNbF zN`6DDJycR>|CG}mi^%B+wZ{+FaY9u_O%Yxe^))fkKbz~Kmp~v$DMuj|9ALarU2T~4ByoI zU&FzD;P35u-V5Maa$a6u4`d3coc0$4W-ELGo0og9<*hwKdZ+3<7;<@b8u{=PQ=49M91yJqsfKxha$p-@C&YB30rg) zoEJ#G-AT^oKgG1JNum~n-L_LcP9dt-mVNIYZ(q?t`))`&#kUc!8bid}7REKs;{LS_ zpMch^O6GlVDzA^JOacOLR8>_M(t(b|Ib-by4~kT4KYLxR@~LJ?f^!zaRvFjYnyFr% ztVC?tMSdC2x+%MlY*D<(MzoC4eBHYu|ob9>;$+7+ANdFLyxU^|xgA7A7C z1f>eNq24*a;do{Qm+IfxXjHqp8(J$;tu|9&Hfl@V*X_P0k?QA8#;SY{oQpxKgzG{l z+zHI42bYg@QQ1CO{ByTdfW|%TdT;W0-M=qVdV2n9?;OjsCMFTI2~|r1oVY$cwIygi zR!aXRXeZCII$!YuCKOJ@o*>eE-V>t{js>sWfOmKItrgkJ%_r=wBPOkLL1Z`M+=~!O z5r(|C@tBRP+QXDqytk=p{lH$Oi>12m?>0K5FYPpXx7xL6{_(CK|9|E~^q=|IPhN~? zyqfSCk?p`o3NHKknxWz(hTldYx1DCR{phiHXQm7d>A$YT492%fNdFK1I#Zy)u%vE{ z1$zpgT^&8x>*mKXNA7uKYu@{>pA=0$6EM@Z!VZ{|4=mtORI!|<{rn6%cJr*fx2~*s zGc%Vi8z&?>$HyEEjuq1hYlj+Ik6?zU)fP6|O8KvhET7|IpbR8a@w+AS%|&y&S%qSy z@tt;W1C?)DB}b}&+Ke$qQ>CZ63C9Im5d#s(_Yy@Yv{S`E-hxRTZ5uiM)1_oh53K`z z@)aHdvCiPoWZC+c_3agN#C)ku{F&>UsCJS6Zfgrx~ zSi#AiUN}{L$lFSgs#Idj#hl()u_ak!si_cn>mqS|+iv>pj#*-_U8#Eg>}OM!TpxWY zu*-~b)%#n?BK^wG212-fpQMqar8^QY{GH7=3Y&ajCL6!n2r--s&NgyzL=%!VeynYX z^7RKI(i5dYDLsR&_R&&kI0B%v^Fh6r^-Bzn8aq8A0sFB9(h)Jb8dm+`_!DoSmfrT; z=DByE?;MRA#p!2_Y8e9Jbp95!K%(mgWaFmL?(s>oR4wV+eqpB%w+)dbqu{U6r&zPm zn=%&;#B85XH}TT4mFZ;dkw;1u{EW@^4nOStmY2xX6?S_bBrZ|XQvEe5N)ZWNj}v7( zgRycytOh&n&b$hXib~w(DK;l^nkhDH74%W`nh5nT&0?K^dU2NjQd zDtl*5*C$ArzaQ5LmKPuQ=5L3+yM3IzC#AzeR1_xdeIV0$vc++Cp(48QW_8q$4k{Rh z6WIJpK)8JpR{Oq|F}a&HL~5w>ci21Izk?yLgx`BRj@xFUqe$o>>KXhYxKCa`?Sx9k za#zV)IbHPmaGK@ynlSh%G@-=Bkv=C>P(d+WnAvTCS2=-0k47#mESrXX4rIfT(=E(Q z$Wgord(oeJ(Z>Kq)k_=9!4b3Lhh+T0FOYm2;XAdE1d-#sOj1$Ad0S%&@v;gW__Ya? zy_g4*CTi%9<599d`IX)iX%TIUt}8V@_%9_w6#SC1?~sZ&X&5CJt`3y5pJ zM6Vsavb?o1iP?A6S2aXnG1JRFgKcSvFo$b z^^9X<& zTzNT@kcl&@CNP&5&%u*ArmN{YCv;V!1DQ>jJquBwj;!NM8BFm7@C5=ELslq^1jlo( z);Q5dDm47YcRw*lCY!CwP+Z$xnc(S9bD!@Emirxg;Qfj8+uJK5lDi+j`~=JR=(+O` zT?4Un144Ek_n)BC;e2)gL~)4ugMcmZ$8Vx+hE`=U-#o=+*Axz!gz=Qr!ao~#-`h8U zHMRB;nap?o`EUfO^|zu^g3o_ERRXf>58r6!*f#QkP^rXx^lMT%c&{s0#;W94=pa=E z{5{p88T4FY{oDGkSSjE8FRqzaImmps&+vJ*_MiZedjP#iiyXCT5M*x=Nk zlXdU&7TNet;ti)yzZD34-yS zcb#xKuNf=#6cd)1_DJFMRIg@K`nGMC^BSi9jNzE9c!Keo5U^voLex4H^|5O|u@A{k zL7(hoW^kP?wr+A_8-(fc(W%*+*Y?j$C3cRu%HI9Kj%}%vR(nF@1X-_O2^%3@{Q0pK zIxQ&G>wZHx=6Z_?=8Ehswbfo%mfabJ2U=Y+Sa9Zw-3$*&)*j6mq{wGE-eh(|*%L?J z+j3=kLrb0XqusO7%B{VW8~TlgPFEqK(1QylJet1$KlgmiPGK724b*fno4lZqqe$&k;U7&$HA2MW9XrUe7gwKDJ{;R|5oa z_4l~(T3X=)rSiiR+vi}dRJB$D55u$&UU9+tYvj@2%1u)IJJ2s;>^%RjSD}7^$=hyw z^_b+;-~G*JqG*bFH#2ZUx@Pv)dVeCmk>)FF@cifE>kAQIg%LvSVr897n*lo@Nqs=H zz}S2Hda*_BPQ7<2CMG6#dUCoyyu1ZU|61-$rWys~HTFHGIFyz}w(2mPMr`C%d6?=a zc7DTrpXyS#YTtfXS88-#;&r~t*Zt74TEy>>EHyFi&ugs@?3-TF#FJrH=-lWpP}73W zq=?0DXll1c-pi)FfbrsEP*C87yw@|WR;qMl$%XXC)V*#ojHd~C-ucu$%lK1g9p#ni z#N)RC6CVl{H$ADnNxR`Ai_p5|v_=1K@GNavxHo35CG&{friTUIT6od%FO)S%2nYO1 zS9+cgSv90|PTYc49eoZ*Bd|uyn0jxX1v)7B-k4OK@bd8s%%X9fJ&*{Wc4rEh|ST2`|@X2_MUv!!Ck zblcG>BKg#GpvJbz1R-Hz(_IHSa4ba2J@9m^Lk%liF!`dBsA+_UxqhW@@1YPpC+QJu z!l+I@cA;!5kXw3knfF`9poEEgd9|$Y!y+@jiN3PRH8LNaR2PBmvmdvmA*R6hY3uS) z)~9qseB*>gO@B-}#uwYmpcber&U{(ScBn2=D~GfQT^K}_3{YG(pCy92-wI66OhHWb z>rZv71i<>o4NAzG&jAD!u-G=Gxfg(BFOYs3zfOxqn9|T{hL1k=jon>_5yW^XBda;x zJ^v?e4}SW%?ndclVvV4DusD@?ldd{N4+sE%cWjwKw`ardUGhX^5pf&PLH~27?za4* z^uP&0iKb-@xA)7Eb)0q6cG8MdzR5IlX?$9e=Mtuwy2@CE+$Yam9_N-JzH*^F>N+9cCdUd|hs$9~;b&@PRYGBSWtc?|YahE1cwkg{bE%q7 zds$HC-l%T95`^t!p=B0ObG@4{7_tF54f)5Fr7M+ZyoFXx)@>CJ9b_rDAW9v|vPxY0 zD|>`k3HL4{YtOA`Az}frmjD}G11~;$|S@)+RCgFNt{xHm|#IJ8s&qoA9AQp zcw9ts6^XRe7{>)=cT|NQY_%!W#pk!1=GuNMnS0u!*?nf!&xL7ntwvC8Byp)AnI>;9 z0jEw$>*#(wq5&0buGWKP51E4`>jUy7>g-8mD)&l?q$WnerI4|+5mV?!l5O9eRUvb( zw+L1dz0~8Zp{9r9Z6)|_1TI*)#>FKN+k9VoYatwobEz(Bu3dTc?2@+HeHj|K^R~7K zax3cAuai%oY&9FFY;K;^h}5kPHkjFhwm{quJV;2N%);nxNO>RLxdu)6dpxY+0m6>k zR%TNg1o0u4-nS@2U zl($|sZFb_?y8b)}aFWl^TW_{+)q;&V+~3TnQFJ@sJxA z!7XH~?i9J$@0AihMcg9{vjkS-@%@BZLewA=aMeZ67p%pHTc5R*&S*L9f~wg0e!9G# zaxo|aqmm-;1>E{4NU~%CK0(6dE5gS~AJ&!SWGoUx`f51D#BDR-G@#oX^u}suSueB~ z7^eRkFs7Zn8Y561v^Ca)Gq)93+n2w&F^oLdI-12lA3eJ3SI+|8R zVchj&x22wU+0iEJNDPn^9i9%tS zn888!IQ0`oG9xgb_0qcyj*H#~yJ7=XDUY)TMRw~^#U_`{7h3w@AQu^Qd>)FdG7dun8`U@_EvecV`MxJ1C%SFYmL=CD?7+6zYKy0M zeHxGF^kR4UCS537HnTsgHhUN>*z7Rx>AA)2lyA`)G#k)xA-wj_qz^iyL|LtUb*$qi zjRkAAOnx-eRTN*7bao7NM#i4UsoG7+6y#uTIyNgJynR zdJN9>d6HIp6{?jR81(wktqpq69?Q$xzaDT?8yRT&;}nqh>p{xk9>>Yblzq-WL2)NE z|12Yyw7`m|9?lDP=k?xi-V(4HFXNmCi;{15+}>m#MlU7-hGDa_z&9>5pZ^6%Hu0}KMv%0~Ix6~HkzemlbP4XT3@%?G4n;}DVuZ+o$ z(dx)q?qlU&2&VRS@52n~_d(Ex_XBCnMZKIKqG5TR0pWs*(KB8_*OsaYRZVm5I2#Q) zGEv~0QsIg_ze5t85?8j24au9X;WJyI=UAuHUFHs%D;eD!rJyxahIKx8~rZ1rKCfpR>o#7*moC2h0lYNkD zIW1Ank{oNWG`4?JwPvYrN*IZf9VYGy_|l5!e9v!MQE5Ny12%wqxeCS6Qgb**qqcgR zu0QglTqL`(QCVtIjJ2~0!r4emM*?Md0W6md^O5Tg1sbXa$T3w=YWmW>MX>Q)YO*D6 zI=)<>*0S(D?%Fng-DFBcWaY>_^h`oa7;XM~E3wR`5PtlUI`A^J%?YQwc4Mcpn#G)~ ze7Dau(U(2Pk7GH;))eAVIPuqP?X&H5?NCo1#8LgF2mgai{q@2x?SU5Myvj4Opu>Yx zWw$~B04;nbOYvsgLhNzS*!Fi5p)7teMGs_Co)dqFV^=f84;w6Cn{ssi^HM%IO7@EG@uNRtnR26ueTkbe=a2++C3fEh7fUYE z*uSb7EM{5Q1+{&N$^iOsJqV=*=Oj1#J!am;;eQ=2cemi0r0k zPfC(TV*Jcq1okotbHGz&UkjuO)FXmba-Bbc&VR#Gc+~zD&!LQ@LhHsEm^rC1a($%J z;L;6#>Dbq4Z%wI7L`@Xt0m2^sTU+_d$9Jca;#z*)9=$kbn%w%kVo{H zy0Ta8oEr>3jGkSLl?F4pV5^!F3<@DdmKl(a?Ql*I-OqVoLZ|ctiHA525RJ{JRVPtB zTckFK-VMK>?`%6V?&S_r8i(b7w)CGb8Y9H*u=TRURJ7((Z}F;pl}uBMpg88&1g72M z!;;lB_J>;0wgM#(qS{YWQbh^YX{V&Qih{L|v&16VhVKevxrD0cYB!WTt9JAY%Fvq{s_^Tyy`o0#hXE+FAQC!3uh9wCb{J#^#!14lsRd&OUXr}O~at0B@*y>>)L!35bC@br_`ew z@R?Lv_EjjJD81LI?daq-e5}GWqvwtWvRX3c=jrF_Xybok8XwWWKUR_=z6%++xhV^^ z6eA-gLGcl4c&*;pC3>}&7rJZp?%lhbP#brCQVKd9?ZJ6bHgdKVft=Kifym3wo`nz2 zyzNh;DI&w7QJ>B_@XoRLFYF2%KIUv9b$xk4Nr2OsEckXgL{O5HZnD$?c}H0btxz~F zjn100?@3^{L|$ezl#x-9aE&LxOyXR$ z^E)x?k{Nm0GkHlPVb5SuY5*ECH`i>Qy0%nu^?v-^mjrw4JQkLF!wkhFhrFtWDYp+( ziHoI&NK1I+&>p*rx{}*4)o}#n^XEFo-XHz!ga>UuQ$c(BVKa}PE2X!=CdFapKsncM#ntr$5_%!sEeC05days?+a+_PF(-kaFb5yThIM;cM5+!cuN_mG-YMeC+g$YkhpF)FUHy?lk?fmK_JQy{! zZnr-xbG@^t0r~p`QU-FnNLy~cwGebctx6JXDEFIaJVBqpzC;RR_lKvsQ;k#f$2Y&n*4sL+s>!NJw$f+9$m|Q2c#h?_|)5#(MBt7|DFFbg*fI zethh?!X10|g*Y{v^wifhw_4*-_Q@-`Bw9j>Sb%Dd^mHUeXnbl*%Q!k6bRaeJ`4iG4 zJ~NtvfspFh_qe^Me6pVjnC!O9;EZX}QGCKX3*FoolLG#pqvF&?+%B|e-pYtd*bi?4 zRrA!Gy{;iOzyuIXfQXwV3A#;MUgOowWT_h56e#LU*A+VNNN@6Pl5B!{CTqN#tTfW1 zzB-WaT*L(k4wy-8_q1U&J$9-UgN|USqEQTrAuHuHKA=#;u*G}pgUa)%SOP6G z9x4bGo(<_yjt7jrrY1MOnnzLj;Z9*<*VFbf<0crXvu3?&_|^DY;vVO~|G0@fZPJ z|Mc0f{^`Ucj~ol!+!DJ)ajT#)qusmhvfV1+0+3;gJlT4-8A&F?Rm~3fkh&1f2*)pB z4V)>B+R1KhG=%OyuGQv0e(TjhB-T$%eW-!(&?HvRDls73IWt&Sr7U@)r!Vy&^ zS$$BLY#rcM&u*`={0k}n&mU`l2F#xXQf%H}0DKNbfk`elCC?j;L$OKew6*5u2L_sK zKu7WaO^y4Zu)YE4PV!i2WKzh_abF{ohR;v)kJg!ScLewGHi^2MOaT~pD7VfMr-FT4IIx-H;NzzF)Ze3vblQH$bEf=JIfPpi^DfKlN zmMEYz{{Y|y)ZMfhG}UZVVknrB$#s|mpwm^v=SP4d%=YVedh+2xZV6;kLYPVr zYfI|zHbB(xQZhbEWdx%q7MOB@^UJh92TM)-l>aBfs0wt!(@PRusj0yOLe=+Ct#^D|6%6;EFp-{WC9z0;|9vR$c3adC0uR_{cX zBF;mLus)hTm7zBJR{I6(eqH@j@%bZOEDeBpq_NDVOjTpGvHmuR!0C2e5}sc8ho~1K<|5dt=Kg_HkPY1$O|A3yK#2&V4D+% z#`Uulo96ts;J+d)Y7#O!rneCv;y@x=Ab5u3#sbOpFrjaEfOEm=Qn6X9OzAXN9TPdq zrZ6@dDE&`m`?q;%h2eG6Re$uRTj?e?R;|K`tIO6Xib{xGYTrYfz09l9OBmtJtk+hm zvcDFl#(9)gV1jaTIia;kx%1^qObjg!HYNK#n9{Z5^XZ{r`}1_AxDV%|X8dtRK0Dyw zw4KWd5&_@a7Mq1;&7+M$YbBg6>B3oy3@?#_1T01}5wSrLfOa(EjrA@P1-T8vg&6wo z3)?*XvEyz5Aa-xR3YtxMeiIb=DF257K>u=xdN8c?50V(J1`4dbF|lxhve7pX-bNwf z^={71j>T=!EkrE|k3PKrSJ-f0eBvoofs9u^p!+oK$iPjL-GJVAf04u4H`$UO!qYBp zg9KhVb#$^tEFJA|kmUloE@z!5@}^-i^pgAhQ>cN-_xTool8g!(8NcsUx!bwB`u0U*FkUDove_U`s?cCFd8w$I75R_F=Cd9k$2F|G9^wj&2X8NR zMr;}IA$NsX5}yB4M)*UYhDJulJ0nut*Q{7qb+|S{yA##Nr{9RNN3$;qHrr0GYq`ce z5@U7k6Y|*&jBCq`Q`61e`}GF{R8%J^QZU@UeMNC=F2A|uben+0z5l)C@-L^0Lp|?X zAf>I!Bdw-rSzwZ_{`eS{m{ar{3T7T+ryHjEpkVf?kxorR#IakQ|sWUwn z{G5$tIE_jv<-t%2HHRrp@kvkp$I|0dp+EOw-4Q??*jPRqdVj!|H81Tm zyPBeKk~-JRH7MvencT?Z+&gw#gU2uTk&QF-1r`{^@fjuA9M}OV2~XgM2W0g#<2Gb0 z38kEpd8PVIa|tHClauaEFO-HQ==^1YW^Dj!!Q~uiAg0CX?6q74WZ4{LsdQsaCoK;` z%ZY~)aN6CKH0^Y0P5a`b4Yd}SLhdHbGYa(J{qteL{!TpJ)O)wyXu0ZJG2NXO2FF@# zdK8Yw`OW8{*6jeI1wnasJQIyt|C?m+*M;vuM&Z`)A`zZ}DF)0bmNhmmW2q7ud00;r z+vQs8OZ|(5(gRRu@_xJ_CVc}`V-E`@My}gC%q`fQW+XOT#|GD9`HjMfd5t3*J6d9Odax5Fn_N1PWmp zEJ*@w_BxLH>IM5_LH+9?ghui}3$xv{(c%7rbTS%*8Mc#u-2cU-cm&-bhLEClA#IP(IuIVzOncbnsN3FFEqD5E4un`wF)L z>3=jApsAp8;=YPEtf@h6I26x;G^W0R4o&mGEcg9*^9ZybmcfEStwEUpPdXZ40QQV_ z%IuvMI%~IYo;Mxp>Qro?U6@|Iy;$qVc~X!g`u?CJ!*{k{*S7PXma_QSMw8=C=WOpn z?Q(H;+BT8B%qi@oWrY-sI4wH6#bop( zgy_Ey|9OPs{ysuK7K&E9_gm`QV^?P?WzI#sqON}}A_FUsY}X8~yG=Br2q!6tp0BZI zE1R$9wCfWv+iq#HzM^Wlm>YXFTQ$G5M3-R8O=gAxsA9d3&t}m7Ws}G<0^KV(B9-ye zQ-dI>E*h21Gj$MCRkiVmNq8N%L6)HP(i0W598q;8ih+3c#vX^a))8y3qQtCOtQxg^ zXHdr6S*m8+Kn2{kqu-vfD0z=5_=B%{v^o z7L1}uauBLQ|HTD>Jlt+JW&CdY9jP;QIf$3S#G0kllh&HC6!V;jA?(EZL_+ot-Y#}F zi?BRy%PDl4oH{`fhXX{sb*@Tvupz?SSq@Rk>$^lJAcP+U;YEI;_tciI@Af~|9q{5c zD^jHC^-+Ty=S3%*IMJg__xSGH zwuv6oXVpA;xkXgaIwHQkhfvGFj=V&yzD`8@unN0uy|I9y=V9jo&(W>ONe21FK`6J4 z>hccwiyV-u`~Bd4&t2$9AdSZ(UwtFch746NK9*BVHaedViO`Y^H}A~krM0{~BLuJY zhD5B^PY}lL51*wahus%FMfa{ zk#VW*BO(1I&56nn&XnZ_t%}m3$6uli>Ln-yeofN^atQcWFwmP4N*dSLP4_XQR+Zf& zf0Gy}oWP-YvRD{wygva}^VP^XGJ~;JDcsS85yLr-?SBlVrV zyYUJKY@+lo#f80R(qM{ZYReZZ9v)Cw;aEYRr$=VGb=S6q=6DDUTy>wa{x1s2lvbHE z&+PU5fnmu~y?MLEt})zkJVb2K7aq{aBJSQ|Je)&m@a&;k-w^R~e*T zr`oNK1(h*nHvVlgr3;L8Y$t$H;6E7vOaF71@hb2~aG#!^Qql6PTJ?;;_v;I;sT&BvhmD?0s;A|;kY zSWH7xZEBVEB#gO;xhVMfTd4Vyf{+|bR^>>~Qb-yPy8Q|pZ{eyV;IEY6_VoUFlyjaz z8r9(SnrXV(&KZnX2N`!J(-1LbSiV{`B-FFTyKaU0 z3^uANTE>H?j7$r?r-0TUI!%0l^w0Yo7%u+tW6YCA>n{Xsn(Ezc+=BBNGfmcr$!Sm` z*y!f_2`&|z+u4B5#o_!#_64t+d6v)CsmY~_2bS~qhEJ1fv+oOleDMrk-T43Hi_bl& zUlahmjcgnVQkT09){DJERx)8*OUpmS(UE4Eq`G!SIlsnALTPlvRUu~{Xh2JIx6b*`yV|b3Zha1DiQ+HCEXzs z(jAg29YZriBT7h)bc1wvGn61L%}_%KLk}=?4e=XKd_L!J{Qd4)_uh5yKi}nAv)(gn z;rn^^-p}sWe!>{%OPutEpNpM^fYBIn^Hbq5Xcsg7OGC<7^ANAiBh0r-+;9=X6}FuN z5^6NO!TNq!IlfeQwWrbhJj9}mT*MThq|eb7oS2Y9!{h_4!Dm-W{)}- z{_i2*{aebFLe$P36;gdt; zBg?h&Clhv&B!I+vvnT0Uk@n&Cx=)c2_BC;c>{8$V|&U^YSX8yFInt%%a%+=iNySrM*# z^|?sfEbZD8{hY&f`NSPKQuNY>PnZ`?b4zg-ToKxth;f z0p5rj)@4O~$TB;NxzA22?86I%1M_TaT*2GT93(HSfWbl0cJPr7C3glRgGV1kAbSpC0#jn&$^0B~fL)PM?g!YPmi2 zgF9oL6>tzWq9-d)Q!$Ou9t{j=krzx313t!GYV)u3&UX=B>t4vje!Kf{kuQg8&e4U& zIl$OE3t*0}H(p|X?Ef8&`~SsJUr+`8 zo}@e*mUg<${e&4^J;0M5RVdt-Jon+W-e*=iq#`-+llAf@U5M@`uf+`bzAY#S{ub)+ zz41DV+J5`synb=e71MZI;a~9SM3ZPJe+Sy;wX@qQJx&8}CpUX*Z!Et*IdPq;a46)q z9Dalc-A7_wUBgcs4iH`zon&$C?FN{qJc*1w8u%tEb|0v;WSI5fM=$&p<%^nsgpU*W zf5M(|(q9czk7T*gq`5{jdidsLbW|9_mBos`{|EKl5S3X0wbtw#-W86=EE0bTLASQ$ zqMk%vhNoz*D!Kg7LW7;>w=wk%Q%2L1b8p%*;61xRdFKRl_gk0T>%S(D{24O*9ia9T zby0ziCCR4XUE}{dc<)aQEUxm?q!JYs{Szg0?Z2e{iLiC|L(Gu?#dNkTNO1}+-ri4eeNP*sXuC-Cjs^B2cIu|_lmS@lfO|5H+6dJ zcsIgXJTYrJ2?ZZmC!`rStc*@hw6H4$(9qGb_IbYPgBNo7##dF{CVT*ou6s8hhsM#wzegVgl3${&^{xae2YG@}&py*&BhH zL?L2b&M=jjm2q?sJ_OFe|#0!AV7~G$|F?*q7$P3;m6ZA7#xUz_qQeskQ(=WSrt^q3tfYE z6xK!KkK(Gn)cD58@c1r`9n=~(>+jsd&@(!8yBiF+m!qCfuG!QnV>$}U^3V;zVmn&c`+#|D|d15ijTa+Ld_ zq=EmF3Xi0LAO1##|B&Oahwrp#%Uv)0`pMrG_b+_BGTlUo}>?whp0uhgG&WmZ(CNbMaofkC}LXrFb%;#2poY;j)<$IElK`pT=P^;oz|pm`q|^u_P`%5Qs{ zOSiZ`Dn51biwa|WpIJp{RGP%FB?>X6M&~+s3Fr!`|DF{5>v<==M^pSf3U#QGD#$XdHkBV;YmN>a%zNksJes z;l|aIlR62)hpfp+?NhfyQX1ArS0~d-T$7k;GW=7RA3;@P>j;+ztQV$YA7s#M`%!M% ze_f1!@Bg49{8UIz|1$8tgPt)uEt@43vQqIFkChA7d~$ww-OJdHQK#&SI#eo{D8g{! zn5ZAGHUWLi1&apgh#fwGwkH`|ie595MkSyn^9Lnb>Mr0NBfoqR2`#?fbKa1T_Nbbz zy@dntGgemCFIor(^d;IK59)FtFgV1>E5`Gm}&oR1TAJwqKSHrf4bZk#$M=X36GgS#vKUJh0Lx$(Rw zYQRswP#%u-zAn}i^6(tRWI?fwpy9opp@oX|T1BPx=4~<|paQzgk&V86(iSh@u&H4o zbnH>L7BAOi-S}Hm-ea2EM>3;gvA5(``ht1-mJrC6$P|xS`TD)(te1%Jy+N_7>A*tk z+LDS#>9YEYl$dqsdwA-9d>F~#qtXZTWY9XE630R4oYF8eepbCyd6GBL^{4-+gufn^ zlrWO;OBN6E)kn%yZ_@vUhPA~d*)_>7ZhRBSMm1zQygz*5R-@BZ(q-AW{5!I(vH;!P zZ9uMNnyEBy~wM^i3Kz+6Ot;XE?<=(|s1_CP3?_MCmB6zeh!XDkoq4SK{sT!f0ff z+sSZXw*5kz5@yMULrml4k#*czEB3t}{{9A@k6TzVwTB`pBH3PY;Abmvj`#*Gp+)D;3`!d@aNn&C5r>q2 zsDGJzG?Jltcj^Qto$ET?7t0oho+lV0${m58hNXb6o`b@@X#P6ie^=d~Y%!h?^qnNA z8ZTF}o4MX&jd*}z+7}a(DHHBmBthPV2L3)|UF|XR5|uae=4F;YDyrwI%t}ajbNh*% zOZ?Z)@0V5vK39je=&^pd3rq#B#SOZLRu#r5-1?kVsI zroE5u^e6JfXfkWlC`h@78s85a&Sr*_d-#9cg$OX8CC31nGFX!jv<;aV*BGp}Klv_J(c) zt)Y?cZ=wvy&B{2QycgW(iT>1^)bWV9SA9!rx(YypI!5wQWNF`^VsR#RTxyoyS)tyevp8S9D` zoYl<0`^KzRg{_sp?HmWlieon#KoqRXvo%SweRLYpv}NX|8otNdJofhXlLrk54XNx9 zX4!NJe-mWtforEy0qp(}vg*bS zi%~!Wg6V&X8TdKsSCbS}rJst5&e@Yql&I^aKH}lbYZ@gk`U^&LBs}kfYC5Q^34}G; zKTG`DIU$H1iv-iCH6$O7`I@?!_VTJ5-GQ5Jd(ogI#qHcN($V*fmi87$IG=3gisII( zW}#+8^cc~p(|lTm^RQY#QCZZ2^zTCwr)A!U6*Nav46BM?4_!APLCZg2fB%M*RaB}Z2QmD z(w(Ik^)ft3y!N5Bz-KKb^G`|Vk9HQ?kGOpX^06#E^<1>*X=&vzbnj;?mp%!b$gn4A z<~VinXfm@q8U4e2*G-yqo%J)?y_$#F#`Fp4Qvvj9;;c(I7luN{XSMkg z&u9%F_A;0CdY0&K<0B`vRLDE!<5;-b42anDCu41ViUWSmg?kvOcQ#|Ce77q=Y(kNL zy30wLe-%#IV~lvrstW-yM5xlBuzBN_PiW{C*ZJ{#bNeW?C!KW z&|~+6SFUiJH7p6&Fxaezw8j2?R!(;JXJAy8j!izxsg@l7pOHYltfeJVVRL`<&8p#0$ha_iYI--q_tLtimZqk+YG2j(Fq3g{cZ}e zn$%wqrcL(g_~FuOq!NVAAq^8f`(0r%h-`v@kr>@__?S`A#nwF3Bmpn^u01>99UT;} z^fN-!q+c%bPYOT9`I+;$^|71n-+t16e|bOk6E1I)Q3U?Qk{s{*#@~#L1&(J!Jj#6L zXu}}Seu&+;?~qJaeb`zRA5YFp0;I|wZhy@J_#c}8e@6jKa4S*7xls_&-MPm) zTzCB9#8Y=tQ_bmr<>CJHc72BkbXYj;gSx*bCp-AQ(i|(#i|tMzq^4I{HOn%Hh!S+O z89-NRN(8K5I8&ss3qm>!e;K#y`2Ut{#WNU8w!!E)%>;NSn4( z6FMpNl=32la}1G@@~UO9;g>&v{7md0D(jAxsbO3Vhw=x`?5NZouN09?3&yWy#KWAC zn^?b~y~)crqR9q2JX2yx?{duA9f}IMZ-v)jbV;oRK98|A(I5#&uyvb|t~0@v62jq6 z(+1q3`(A)1jyqZ*{FYcDEC||PkTZm!8=SUb+_A)!Zlfg=@%qHlr_wdz_D88m2{+r+ zK2pp!huK?|tFCwk807a>1d0qnGR>vMqLMk);J3^QFMERjW0L;+&^nD-rdxrXY)Dl* zK#h3mt(H2(iP`2dd+YA|k_?&IpsxjDY8%TaK{aV6^R1;`c=9fM_!kt!m&FX0R5P9$ z&!cKZuvrB}lJ@{G>-P9-!}wA1*CUGhB=PUc)X;Lr=L>QXH!r`mw$JUwlH~jAq;&}f zox=yqzcPdz5ixzl8hT5+dzww{(fR4ApdV--bZbpGLI;zjB-|-kD(%>>|qwi~>WIiHqFnFv9KOQnTU_~TPPs25q>5<(i%3gDNsPnCIo zP??^6uDM&bj`UktKZP$MUCgb0lwct>D;*xs*j_A&?FiZwJ}m9Qg;7n7 z)x>lAoH!c z^Cy{S+W4G*^9VysvdCGNTTg zcQ)BEDkChgl$Igs&G-3_Qo^b2AVPBcsy^ZuPTEFK;l%*)d5X^JGHlrF&>Wv??cPW; zc@0mz;{~%;TEQ#6 zi5|(=FlPK*E8Ahz{8mXO-%w61cYg}&*XlqZNTxo$ZqDAQzbSoYl>t;2)qN38TLt*> zsN%T#HJr3N-tTjMzEdgO^;SdJF?z1!f6vQb4rNBv`w|vC>OG?7^AbtCg`2T~3z+%$ zp@FF{+(j&Ck_mTG_@9u`oPOEmx46+u;eR_Nkc$8pLc(xL7!E1=+-*kYk-Z<0;K5Hv zG7!VhZ;oT0V>bH%#4O$SBD-RR)KJQW)9{m1wO_3(YfWF~jQv=tmnBxy$BVbslA=Hn zif~S^SF;a?P(wg64^sDjRfe(wAQ>6+R@ntt^kl_R_=r&5Z6sHP53-@-yQf3NlwRnVX(V zTYe?|Z^jJKA{h9aBZ#)j?_b1su9Sp9T&n)Rs zJK}|(V!9Qol+bh*+&H3?sK>42_4(@dgWlQBSi!20^Uje=u1L3~2UoZcp(n`)Z6?FO zAJw<()(_8E>T23hZ<%ZKUieOe@|;L z2Z4Qn5)Mbh{GvL(C>WWm_QJ;qEZY+9BwP~W^qemJ zO|Zb%gZhclI?g}3m;YkqpRD~2^9PHo>m3sEMVIsVP~xiaH$NT)NzGY!S`IG$cIxAC ze1ly^v)1AKGbVOZNEJRtIw%X9{qqwE0u>b^Vq&9P!xJAE-n@_#7uVP!zDdoCZ-h@& zt|HZa|I5J~hToh#10i<0U*n(?Qh^h+y@~$!eS6NsXSF>oLBwh0yTE+Htmn4R>VW3O z!ZfHRHt^YT&_ols#vq;7&axh;pQg`rqb{gs&>`db>omh-{u!kUPuiyb(rQ2sINMcf zH>ilWAT1Aw+=$;bh`30!M6CyRp#ZNm^3qU5XQjtVlizYuHE9WJWhbESBH5txKy9ms z06TYYQs?$nnDNj|_}E^$mD3mpu4)n=bcKd0b4}kakC(l$pWtzl$3S+CLB6Z3*JiTq z0|RaLLhvXNectTNE9|$`p+F%k7}??O@gCnE^7lFFTrEh*Zgx(8 z`u;Zz?7w|HKf#RN)RC=Bp-iW~FqtY};`lM_SB5P08VK$ooOH zu$=LhP-IgGH#?jlppX%ly$Ni~TwPq9UYsE*R$A*cW>8~icG zV9eMS^$?(koW%!fQ1;$EW^FF1y0}NaDj&S7JFokAMdqw#Lm-@O)PAC@*NBdYQ!6a} z05RZ%upJpk6(-Rvodr5nq{Z)ggAC#b`>a-v2k94`-6Dcl)C|r$ zeRQmo12r0DZ5gvokWatfa`xKFIBj6Oe4&zc5ZH;SkFxFH*}_EI#v(_SB-RBu3^#Yy7qp`%4q{Q{wwRo)ucPC^c?ZZaA?f z#mx<+Y5-LYzqLn;dl$NW42C4R;V!VboRT=quKj{pqExs#`UTk zOq#R%YF=o{xQvgv^^odm8C1e!+Bt&##|9%_U(h4Qq;vBc)!J5Npa@QDI_*Qlmkr6N z`PuEGis{!5^_=aaRh|mQ+n|S@W481FZQII)k+H)}KDHb-sx4(Exmnh81(2~wWetKWtYGh7aO_E6_jPrYJeGP zqnZ99jv^#oae@(dqhyC690$T{V9DIg7eH)`BE*MN<7(M9ziSh34<{ib-3n-K9o_G^~u+~ zmKs*)x!}w5JQstbB!nbwH&3{{^V>D!BD-D6VU3INGe-$NvSXszc-U1RyBM3)*0_pT zob=X);Zj||ta%3_H*a6}n?Qn2Mt%vb#_MQYF#j3Y`c66d4NQFT-~RLQruEJmZ~d(p zmPeU&wR2xV2)xgXZz~dyAnb$9XX#hu)k!&Cx6@ek%)!=pg&~@#kObJwf_we@~)7qpfaIgtV zE1XBrcEwFLJb&7~5)S1NI*39}!4%y-zU7U>Jt z{&n%N!+rGwnNjYpd8-Rg0H~rdQx2^TkdlO5{3Sy3nJQZ2DfqGG^%3u5Zd zxs9DmCJvl#o2!+Sq{^l8V1+f@1@|T(Bz38LTLQ@G-MHxC0MTL(5iFHTjjB)-+O3RR zawshI3=jOL3Rh`pMKr*5WR>Ry_-f`puGdWG?@!d1hF9hQ6LgW~f=w4#9#P1aQGOpj z0l(Sk@2Aa+$X+%t$vY;?u|k>LPR;uR;w=qgE(>#=rJ>3*)d`?0t(qsVjH~6*0_AZ; zFl&H-c*|6e2q1J)j!x5C?zxtWriE)bx=O}*TmJ~28!XV|1{NPN#;rFs)mk+#PiMcJ zlt8iOYhF-Gc*v=)RG#oyT9*`#ReOB~)0Ziwn?u2hiC%Oz66 zUS+Z^tmB*20ex#;7QEOYpRZ1|<$Pnv{MNmEr7nd~4rFO4ww2|}_jB#-AWW;vW;Vgg z-Dha2IYLw0ovxirkiGIiRF4OF%2H?$kwz&Msv0y9a*%2uA*$CP_V}~}j)-}6L}?)( z+jsEprpURgl=gwUJs-tllWVJYdIJ<9P0%W(hGRQlDs|D1ab^llce0bX2K(z_(6f z-YF!TSQX`}9a)`tPH-e9c^RBLyzzWI#9`d;Hp{~`C(w{bIph!>XnTHE0vF7i)-YyV z+Rxm0;b@yYgxWl^Y_RR}co+tt@0eL7@0xY?w+|UN8lu5T)(0kh0RkY1IuLex*?vJi z5m0IRZNdI%&7h%y5Ya#M6`CKn)7}$S$S}Vxi!?ieqW9xPZSjR%Z zp(||U@&X;rv<)=CoN0|FJKyAH3z(h8C|y*6B$9Vg@L*Wo;| zGg(Aq$&#i?Rx)(|Q-W0T!f3E%BC;}pby{QU`{{s?$*hBG?sO`(U;D^}LIa+Gc3&<+ z_p|{`i^GOW`5SN%^;zK}$q8j=y)CP4E(2!+c8I>+qVwQE4Ykq3(^J)Ha(|-SLo;`8A3{3aj)r! zAvNER&Q(ti+L8We3+I@$2fW9#AYID*LqK&hx3HfrV4hB-03>k|9_bg=gp9@ySBHgF zBkeCJju+*~wzH13zu$Q)#)_x$ZgJz_a@Tq`{fOqUGBi?acKamCs-ByqYSa$1FOzA9fQ^ft_0ftUEblxp}r$#FENW z{nP_?y2#b*+togi0^^VB3ge|Gi^kLBk9O0ijmrwysPwlJQ zYJWHIKNZ-cWDGfB1n~J3Fc8X-*#r~srt^I002`1cpf`+RD2B22U3n)8qWqUx*TSDI-G=TlqM6l z(F5-RQiQE3?7TgvKr848b+#F5ivs?3K6)4S#! zO>*D7p4p|DRwFh(myb7WFP`!V&E28W&qRzG`ORcH*~c#4Nr`)WRGQS@VH{L7~<17Hyc3W{l4M-no`en)Wt%@jF7ts0j=VixV3Fdk#lF7S=E~0&iya8Zq24h{Aj*yy$k^u zLa`cxB&AKH*Q&EFN2aFi7NC7=&~5FtT_V9WUQ)(Z$l?9g_Sdm)DCSB7NLS);|)iaLO+S4ev9=oiPS9kAE_pCl6g}uMn+-hVFB%r$H z6#dWFYZ1Z!FJFgLF>(W+U(HMC6lLD-c!MwgC8M=0Sa0%|+!4Qlq3!qFOb$D00%xI` zR>Bo+XCNB2#y1Qe&iQu&7nbmk7#ra?CevAtk_g)E6dtIJ?4+Ylf zzhExC(livI_j`;CD9ar;N@n-F8bP2?JlpSa!T?zJ;Q_dhBPpp^YQ?eHUHo^H=2A=Vy}f&(yVqdV;uq<5%($(!vYFqAny|{{ zD9+&6`r>Vr@>t?{o2pUYI&JC5>{$6obHrVyLBpo zr$rCO4NO6u_e%~~X=DKb|MV>1i?jM}^? zswlGt%s8h~lrOo5ebIaQSnhyA-!6mC?R($LM}=9tMw+!HkpapZ{2^?lZHuI(y#8R9 z_1#-NN9%{^g&k`jAe3B=g`bco9kcd$85`U@MkIFy8BrAF2I8sg$??Sn_%xBXS-}s4 zbvxYI{mMQ?dZwuuQR+VPdvf??)PF2zn{c+e;N00&>H;v54FOj-&)A5-wY==|PLc?d zy`x{4Spu)VE8Q*FG&wfwG)t|RacP!eOl4amx_4r`GGY95$>IC^v{yGCv}XMFEfVt7 zO`r4mEX_3$QwezQy?O8Oqmz(yM?Rh^sc9~?LJI~?Q1;04LU0{W# z8EKX&WaTd zW^4#fH4T?{@S4^m1a-uBwLFv4o1^`klhSp53mIs^eMJGf6nN)%{S+QNh=0@uc_2~8 zhD-Z4=g2zRVAsbAQW;sFYhs7{))V`fq241l@-|A_NoMEh>T&q4` zYXJijrELt-tRsQi5y7NLc~#G56GB3JoIXwW=$`4FLIn8*}U}f zKqPB(vO}Au5`c*iiUdymhwBXumvgYCnH6
  • c3*I#Iq+VYqz-b`egMJW z$K0oAvoFiLHs3`NJ=fB839pbxZXh?5Z~;&|%ZL~Hn-3eVa^i(}apA%$%klpvzm3hF zdb6&RyXO6M_X+gg8$cf_Q-gguX^a0k4~7w(TvaS*EMhDwVxsCh@j|>*;iGttc$o?> z8UHuPmu4JMm-}Jj=~BaeN%^1L59^To9Xenvnkp7$Sfc)@KW%K0dc@1rcuDxbVZJo{ zKkf5&wFUt#^acD+?$`P!eRFExO!$!cqyDtMMlKYr5HB_Ga(WAV*_U~LI*wW1B{8{% zj8U5k@A z#8{-oiqs=sh!^6;A6w}k(fq^a|EcfK2lx_DZZ5^XwnAPl{wG$WVbvN>@c9dP6vv(p z9njzCZ(84wdIQIc;s362zv{g|1y|P^cqrwm`G4Tp;=JF%PgSvrcp-OH;iGttcp+Yj zn5g>hz>DyIm$)BIO!oUz?F}UR2+iUCg^K?jx&|L|-dcrE`yBPh*k_VCZXdTsNtS8TuQ!U*d)~6E7KFc0*^ilkjhWduMuf5DkFH(Mp;_{67NsPtv)+ z8Gp3J{i3T%{zo6k{aoi@Eb_!-`bQRf)fhp{$?8&n)Sr!q(B}YNvi<)cN0lHiI`=Ov zEv+H1Lbi+ns`mRIq29X8{Vo1)llw(SUj7g0I`d)3P<^I6`nnb?Qjd5cUWgZeY^8sc z{G%5CLw`}ZU*rj&r`hIzz_^s8vWycp4FAg*KxC9^Dhg)ki{3f zcS6-zAzp;HO58>LRm>xme5B_88Tiu4#v%{=JrO>)_lywJ{{LW)tXcmr zQHlSRx>n!SAJImY*)afB!D)~PW zUuNuV$=h!6|2ir-xnIhWZ2tF-6S9kVK3LbDb-&|;{~?duPwpRb?5ONJ{e#?!P+W-- z`#kY77rc1+KO$cyj9ids_x&a=*cX<`@AlIsV_| zw|IN{Mv&*dHvs)qrU=*0$+zU4w8j72&mT%s#W(1WTu3fdVMQ=SypRjEv4waMtW1TM zjQ=fuGxx`5S|^V!4>ne(*w-$|OZlJN4}+-UIrK;FC-XgfsEHpHTj&e;pWLs+7i0N!<8sFGL~m2iYp6f!FT;Q~R!O~qjh#+! zfiGuTM}SS#shz$o?-GolQML2Hur)MaVt+Z-sTC6#4>(V+!y^sgWw$&JhivBjGjTQA z7xmdeHSwcj>}&hv;Zi&6*v!;zz;;|y=Q@m*S^11PUN>G)9*FekqAT~A?|3|2|E^~j2|J&q#QIVJb>6`RT;DfQ;qXYT}{e%ADj}LS3k6QeX_C@7>ktckf zW}E+oXNyTH%Q#`f@V|@!L`Jcq=KBnNaq>Upam}Ca;BV4|P2rsQG^e zzI4*D$OC^*gwHMhA3G~2ujP$a(t3vfga1F+BTMs+pYNwqUgCfHI(;3u=(?}7eI6Up zf9byi^mcEO$RqcY`xz@ed_n(Mcy6e~HtpLB`+E65B3~wqT##q@e!Pt_~q%HpEI91F4 z9mIYvF0*|c%WAPK^@tbZr3ZM)_`gBEG~)<4PJ7QCs@A85`;ziMxgQqEaj5S*V;)Qr|;lfu&yiARkg#R1nOT+(t;C|@~_@CVG^TlF0eRFEx?CP3fZp=sf(!NEX zuli2BbP6x0lly^IPjc&Kd6!@qKdGIG{Z3(@Yq-htRvgP}eVP7C|J7ne>Ma0XcFUSP zvYDgLv__BqMtyGpI#j3F*Y?RngLWd5`vW>y0POSGJg#G@{g`+r_t%O03xbzDd?-KU|GRb~_s=a|^AE7zizfkc%vEO;@C-?uw zP9|Q}InutgFY)3ZH_|^A!aw%FPC>2(&Pz#t7kSMeB1dCn+3&ZqO>$oL4=!B4t#M3{ z|IxmLThKnPF)Qbm09eMu)ZaqHj%01uwwv}d?MwS=u_E<|m&Wk28~UxCV2>uae+6NF9r*mBk@AK zXmDccDDYB?{~<3b_lrE?^EBK1k1mvwRF-kVhT(r11Bi@LO+~>BeR1+Xh2;?V*9qZR*9^ZyKd>11P(2mYQ2pIiJtc2-Uv%^$6#^$h<9|9`MYmi9S+ z-_J|@Ppr0rRmMujO0Jt^&lK^!6$EHiOkjNW#%C4(H^lw2hQH15Kd!L?dpg(sac+!y zs}}!Hq)k48t}2hvmmL2?9_Rg>_p9QN#4F;(8;evrr2eQs@gr>}UMletM_Q0D%I|JS8Ws9!rp>PyD|kVoz(_cIoG;xYZBGWG^>L;X>IAh{q; zOr0t4Qpx|27oGdlclNeh{J)O&JGo!Vk!=3=juTL~GDYl5!vBy*?kD#*>6?|Zi9d1! z0$wFX?DND6@nYj4{p{uch+X?>%pWl;qAPfJGfhX@cAxHLwFR}Xq z@jJ)hKJM+qee-~YV2lI)I9B9XF^Cg`CE|r+CxZpeaYnq9_&+vZy5z~)zm%hYl}YL# zFQZZ?xPP-R%e%<^7}J3lazD90h!cY);)UFAu%J0cz)Oz*H~DRB_SD-4JSVyB z{dLgwEKSAx^6Lisa?%$6^Bo+hS{2_SU^RCg%44k5Vo&N3FT@M+Qi|0X|2N2&W*jlo zSOT_Fr-u8I@;|vB2FY=l2JW3LYzu5#jqcyZyvS4O-{jhBS~8|F*H|5L9c!0zhQ zLSMlD=pW?jI&qbFiGr6s*A9 z`vJjZzdzO9K(dd}9PVGJ_@BNsIA2PhcM!Xqcv|O3ybv$Mi+@~8|7gTN_CT*et_3bj zNq!f3%^xC1V`SOyx3W!gUiJ?zT)(YxOpyQ4zJv?VKCXc)=c)i$#+%e%Bfg%jo!WZV zex^NW4=q-t9`Uj;c-akm)lRT)6WqT7xspTtKLYnp(z(BRccCrr7hP5IKe4|s*w4lr zu&Mep?MwR-FMfZae=L-LWc&XKv1toWJZ_`gl= z7ZrK=pV(h0?5jB+Hc#r(zO=8wg62rP5HA{>m^yR7OD+CK`=WBc$P+$Kv(5idNhwKX z87FKQ{+BU;$SBoR6wJ^UC;vkp*ZlduyLwDPypS6Zrid4h4(K1P_=lSRXW&aG8;d;f z_eA*I;{UO;a`I^YXeF&@_&@mngFUje&-wd)UgCdZwH2%~Rx(y{9VUCGi0{E5K&xT` z3DluZ8Ctipb8xQGcFaJm6OI#`;a(gUc z_z_3#h3ZHy_&@mkp0op5_>QB>dknUmE_OdL03FSEm;G0{$oW!yx2-hYlEvrjA7ko~S?SPa9jL z9`Q00UQTa;FZ(j@PscIKy99G}Fly=&t21GhzMky;#<8dIr_ho9Mt=)o#6CYjyzG|y z`{{lDndj-ypZ$*C%<^Cxb&7p$pFA{ZCvt86A0W2)>>JmK)P6$$AXnFktHetbyzC+O zmxmZW?w4}Z&i4P);+0s9f>pkA$#o~yc!K^$f78ZhsW)J}*!(~B{dwWVnCl3zi8{r; zwnAPl{vR;5EWYNNH2_WSa_E5mL9W*NhSVcoqTR?tJUKOsi>5v=6_-} zDpu3+1jiB*2Q``aBmOivF?EQS!Qn;tzf0T?2qydesrCkveT3$4|3bz8^rgZ1Qu4fm z*xkg_I!EG#cp+Z=<68PhBmS`mdJS?da9K+7yU1()5IGtn%YMI=ZIbh{e{kXYZH;4s z{Ezk}T!8j*4O}@_1;8@ir2ZQ5^-K3j{J#c!`nX@72b8(L;s14M6YAGak@}MHKje}7$^DE)o_I|EsEoZq+)#hiA4o2U z6H{jjyj1c(W@zxQ~1LaGyJ1AsFL;KaLeSRt)0AV2OC)*vVi)bDR+`CH{}imo9m-_Alk=UuBXy z$jhkI3GUzQ%knOAKe->6490Zeh1^f>58}jNiFhIR8!Tv!5%7}Z|4n`yn?3dR0nbTp zdw(5tJxf#ZzWlnuzMQnh|9l4rs#e9f2w2Tshw>OJwb+w-#0&94yp&>f#{UiSr5Q)e zG?swv)T!aVr2J3rhe2|j?EB8x%h>C~hO3--Azoa#@RboSQ{yG!|AzU}@c-272(Y_4 zwa^#vKe-fe`2pf(x7^=P@AJ<*Plx{Ocl>6S2ivGq>}&hvp+P&5 zYxDmAvBhWKxK5<@6Z!|ax=vgrUZUV-54pcQ#PD&yl%saG|DP7G#A+0*@|{brJE_JK z^f&sOHa1JW0prEy|EcfK3opi8M}SS#DfYD$@@nz_fU#xqHP@^GXmXcB2lNkewbnPJ z9`O7{aD*mT04bGR6=N-iECZ5(g5--FH@!}uX(mxvUk3G<9kZXa< zQj*_AUh{{@(HL3w`>kw~oR|HB3)gRJ924Y!v@hWTw2y1x%DE~4mhmR_*NCqtYp1rJ zwV!DZ+Cz&KsYkpl3|@A_UbPeK+XVNoK(6Ev|Bt}^lXUKH-d$*m`$bok{7>vJ4ED3} z25hSSO#9Nl#Eaiw=pPH^AKCtYLTs7@`$p&frKP1cM8{~7qw$;KiN z{5=sqxA=eTteiZWKUzua8U7Fc|6q?S?Q{OVpO^TbSZxKXjFpU)T!+b?DdKxD2+*pS z!1(Nq&no_Ji2G#?f1BZdTw?|Hbguj3+!*y%E&iWKn|uUaRUV-)IsS({&igs51juS=Uyzjlh$ zmyG`*kK9l0XDsr>WBNyB>}|LBe;w_2a=(-# z+5GPvC!lU+irANg{~?duPwsEhH!Ecmf8+)Pyh@DN=ZP2M#l}PW*~|YC`4X2(h}<4a z82+EgsicDAkF)_f;eQ;x_Xiw5%@sxN%MQpR_mle>dp-O^{~*^O0MkD_I_QvpIQc&^ zU%H;R6aEiAzbEZL7XBXtPu_7tj_eCxV)q5&caFh*+}nry+yM*07zg}utjMuq5GMvp z#0$qx1`C?wjCd*We{8;V$&!9mdnu_=3*A4dNq%HpEJ2+6aD!xU) zYVJCe$5^Swp41~=h!^6e6st4-Z;&s|IAW%;1Z<~H4fiGGe{w$zlH+9Gcg9}EULQ7G z<-`l|;=+ZmjCh$EFA4uQ%$J7$r(Q>Z-PNguzJUM9{V)i*-=PD>qN!t1f+y;a`qRc1 zsYkrbgqPD>;LE}qhOWqTyot>HJ+fq(ciSOS?UcKFE;;AeScnfG3Gh~Y@$xF zudR?*i~k3VEsL+YW(`1-yBs>8e~_!Sz9IF9m#BC#{NFY1SH1VA;A-_aLMkfdsrjE+ zjf&NDJi)Pq#6e9a{)j&fPD~x*WpH>A{_hg^1A@tZf2zHKWFMh9+`mxqKYeL%zLY%g zAa*zLw9b)uAzp|V|G1X^(TIQSfnI}L3tX0x{4VmEKSYkk$g>pgXep}<1 z$3hvEF&^zpxB%_r8hAigE?rW8jre-9cIwi1sEqcYJwh0<&l4{TgO^vsZnYEa+XVNo zK(6Ev|Bt}^lXUKH-d$*m`$bnLB76Ct*k2gzdu>SjZsD7FCZ4POp?HpXStz`m0{hiU zuy1tkUs_sPLtcgKg#XbX-~Im)>aEM%-{Su^xnFd3JhGEJi2a4azSD+2+e$g@OZ!&g zqj-*ZAzq4@sQNw!yu1kQQ75!7D));#;qx@x{O@X$RF-kVhT(r11BeV)E_cnEzaUWk_vM(p#%3-OZT z;psRA6Y$am_sbgoHpBn8#tItZT=&PhG3u>a{6CR41sm?D5TP%B4zU@RLmub-ocCvV zu(9F#OuR7mx^UquBVLFXA2wX&HSr?vvQ;L?Yl8dtmbH6}|JTq6ANR}ifHLyidGHTx&hHRoVyffMa)# z-77I7ers}1@p5PM&zU0fWg@>I&+z{QCw<(X+^5Iy68^{0dw;;$0jscZZNvZ5aKA{E zXAT)V7(24q08CVVCSJ(>A&l7Pi5KF<#zW}y40yQ(WS0`;MPKtT?$sk}|8kwu#jOH4 zk1YHz&jotN3D!?aA4Nvt+SAVa6WOV}OE%{-oYz1pmE2c65B+iM#IchaGsFwIuoy$C zz7sFC@NzJ8nM*0ZnSFI;ohv(^wU@PjxlZoFjxSONc^Rj=#Qmb1FKqEY_oIh>RPh|P zB=?j1Rag;B5ihl3c}cI-AN5xen}36I!mU{XUYeLI&oY*niua{M4fo|8oBU7ihpJ0) z$5qBy#8~9Qg|CcwAzpmgaFr7;wej*b=rKzT^X1Iz2w6iFA6n?ki^rVDsdyje;yjMe zzNxW7yht3c9m{k2B^OeE;zxn}KAcZd3w$}#J$KVi&9SAvJX(AV?WvvVoAk{(xFKGM z7vjYqXXzhP`$w|(FZ9>7HGd7wJL|O9m%VOT$0to)Vs&b)CgUvXkNPvdq&X5V#ES+e zrq00dayeo z+6?ezu5Rovv9GO^cb?XM@omTF==uDGe*8)Nk!!TEO6n0WgTTu{;D7U6-c0xBm3$eS z_7i!ruU$jl8#F#eUmAojrO#|hU)f)cUzj@d58}mOL31QtX2%QOzyE5-xnK3(Uk$DX zw8E*TyhlS~fs9edyt z5KQ*_Q|%2T`v_V6XnPIgCRP4KUmA=rS>qG#2T|n{_Br|o{lmsX=ri#W6)z7MY6r!RUki{6gD(k^e__+77ThtdDDfBKB_Bt8zC!of7fSnk4wuZ(yhUVPYam3#2=YLs_^ z`=uSi=V`Y2AKfV>sVw7!4a5I32AFeSu8USC@!q5B1)Xz~2+$bBq7w|NOcVs*hIEdWQdl|3BCzNXjDiMcz}m z0P;BR=e(cuerqiZev#>e{*lGrOeeN}LZ7KW>Mw*5`+NmneusMdBG*cg*97;=8vZuJ z|G0)cPv?B^59h|Hw`%eKMA}rV6KUlJ`tl;)@8@&xfR`$^1AiPVa;&JrieQR(A@^%z zivlknLY-Qv0lt)Z|K4df{J(}q`M6)62b8(L;s14MQ>{*e2igIq{1RAEIhMZAy;wXubGaq`g#(2F`X!ty7Wx{2+jyj&9%~@Uf9oG>!cCs;1^_h5~zg6L*c#e1(JYFsXCJsPWCEz9c zn!oklfZ_iM8sp>s!F+;oz zK9;Lvi(ra)Azpw3c~@PPfR`q@KiMll?)}Sksv0Eck%j-|xj^qYAtb{ZcgVG8+N(g? zb`E`c7%t0L?$H5b5n~bk!yg~$AM}p~{o^H&U76;Z|D zvJcw)3*_J!fWA&&*Wkp|Azp|Vg9Xizcp+X2IJq&td()iw(%YU{PRzM4FGbrc`JcW? z-wa{IK2N+5FE$=RpNSXZrGSx-NVn&w0Y`f@nh?d5;^CVjICAH{RT3-MCK zMAdiVg?Mq`<8M&rOda4$XS=2;>`-6ciuRYr0Q61zrVAIoGUA1J@nOSNPP`B=4qTjz zGI@T!lsd$hrMA^A=vZHFjrQmGpT0@oEMlVSJMluiRNKI>^ z+E=HbqkZ`x`jCtP=$rISg9Xizcp+XiI5BmI7vjZWp>X^rzRP~Oxztg<%(W3;RPX(D zxG%RvAGq%p_e9RE0|_RoGGjSoc@;j2=ZF{LrHBdZ`$O?4%!=hxJ}ezWX*^j_}k$9??(>`Klz=YePL1;xFfR9F$r5HI9@6FVHo^H3)HgEc-)|-V%X!&9xX_=@^hMeS?*CJ?4>8?jav^N2if_;#xsY6_ z!ir#ucp(>RW6MDBaxD6prA6viy$s5IA7;iGtt zck-*F3+;J4 za&7)c`^f#A2di;Iy!0}bXZELkXq`z^5h+i|3%JxIrJIw$o=Gg;>F`L^bc}jD2CeS z2ZEO$qt6E-x6N;I$y!PHGM&S3GyIP-d0x@^-XG45QE%1a|B1+`bcRp?BjlU9#H1~ zhX2>`eUZ3d%p&Op$dz%OzS&#fOzckk(!QY>V4o*mh!-0V6`wzVbAZcS=C-)BE?Mo{ zZt?#*e(mIbDMzyT-#bnSwZOWzxi{bTdjnTM8^4b190QkIcwmt`#TF=Xyt!v7DT=6&Kt;&`uPIpYECTPGfT z6Wn;kMdN;HuZH^(?<7sd`^gU- zkqRqV|Z)d`H;tcgURAC^^t@C1YE!V;i}P z^I-KjXNY4R^!a#lzk4LG7~G%j6Zro&`JeM(!|kCXxu4u0!iasIcp+YFJcK?k0ABus z+z%@{(t>fnJcqb1?g5-8m_v7J=LNtj*Q~i_t;RL+Os=jI2M36k4}&kR1l}<3u+95f z_8bet9#PlbbHW2(Vqd#Od9tU$dy&;~QtA#6`y5MfETQ%h`Ul4nbzijkz$&?#T&=>gV4ip;S8HS6K=JZ@aR2qm{jh!^ zDY&)pi|Vg)ly_CM>^$VkxXD<_SQ)~IeV%wBUTi#sJ`*pU!^`&IA#&5^6& zM&UF?`>p}0-$Hhd0qC2Z^G&XEaGk^Ht1cg=|I&Y5xbT$$FPDP%?=T?lN0*cR{(|f5 zs3-qJaPJ=K<}l#lyG2Z(zvLQn4Y`J6;{*eY2fZIl{0j5_JNJ;^KyA(hNRP(Ivfpnd z|I2yVKNz~%Zt17azCyVMh`fTVc|TYAhiKrpk?HH)A3@Fe#EWo($$Y2Q;30HG{NvoB zTXPG{`v=GUE5M=T5dV+B{gZU=Z}`7Y+z)q6(xLeKb^2!S$N7xC-HN@m2koIAb2kv5 zale0jlr^uN_gplkqs;^Q8+EP>siUQ(r8VSL$WHiQ%5oRo`XY7LW$thBe;>I&=`r+i zF#7rBtvL<35Q>?nZ;~6x4aA?v2OHqRq50bnhV-nTj3tE6(`@tq7UOSyGUJ2|!~Ze{ z5E)xl^q=H+;8nAJVD?k!;#lJ9Xy<=#jsdu)2@9*@FJlqMidw8lJ>rFUnKNF#f*5}T zWWzs)4((QP zAG(;UC%?n*k;TVN=2)VrC*ODKkKEwHhO3--Azoa#@Rb2CGQX2~o{9HE;C@-d-)8t9 zI#_{zoa_ELH%7fxi~lDm>-)UFhz0pS$R{CR?!%F*9C{-6bIm$89(3Wvlo2oFE`tTl zk$9OIFQ;K{Hkdtqg|}thKFx;zWe;2*_sjEuGWR$9zb-ORzjl(}VT`p%^M2Pj@Q&#C zPmqH#9sMQully}>F<2sAW{&0bH~O2_-+lwXJE1V=cg1^D)1?LCx9t}HucLQP?w4{T zoBzGzgph_7NH@iKziX^>P4xFHWG~jxU&bQFA}>CQ*dboX{Y6YveJ5UK!ppZuqrGm? zgSjXmqDuw-HT*xpuYKH~+^5Iy68^{0dw(F*g2J^8zjpCiKhrkNI8eq-&qXf9c=VUt zPwp?_!i5znBVJ~T<wp z@CuM0^<&Mys*6YC}`{{guFC3vUgx8k$VU+qj^pUT%WJyUfC^&*(3o9ptlvL7W);5HGd4 zA*WmFuUGo}Cj8<-$Qf_;gg?+FU5edb-BmYh#lCjW`jxUp7BT4FW-mQ{q4ShGP%iHb{1#aq z*Q73fM=m56sxd>nEG!qQY}^z3eiX6op~(9nS0?m#==`DCv%X4)P~m8d9OQq|wd@~U z;Oj--&)=&X>2IO0GwE6(?}}*5amZ&N2gd@CNA4&0t9Su;;#iS;bE&a12fTa|*L)1} z!N{tlh^Hz!rqa;lq2K88XeG@d{vUz+C+XbZ%-6~N20Ve|6xfon?_H5kLH;&!&UYb? z+)wV$VWNl;`#bT%dB2T^(B~HL@)K~$f8zh~-5+f^5V=;GBA!;tADTQUW3;riw1&Kj zzRMUuo>BMRW0AV+GWWOmpWI)GsI8L`uqAf64)Oz$Uk-U+#ytb@zr-|?`G3Hm8x~@K`A!A22=$Cs(t3vf zga1F+r~M)78n|zJoVo)2UlaMZ$ah1Q=V(qs{x{@*M}7tJYml+_HhL@C_YUNDTKOGP zj`MG+n<+2RBlV}m4)L-uczH8o!E2FELw*_Z^O2v3Ec@@sJ9)Q9J{&psPCo3*c{<|1 z=mEN^otn^JI)~q8_#f90zHqMle{W@=_uhZ9cNZG7Kybv$Mi(douk7)j} z-QxdsXxz#DQjTQvzjvJAx7y|z+Af;yI(t89Xya)1m-GHm+%xAH{}}%a7Bt6(@nZOY zg0}d$U&wz6IpKdCz4r$~{b{%_h7Y->6f#75p7vlYQbHxEM!XO&#EV}83+Nxh|H0?? zq+iIw|MFa*cbwqYOtOH$XFodj><8zqoVQl_tbJ|)V;i3c&;AJY_m-a-scCK5L zN9^YS5%@y95HA6U}yjVv7ZB2;0y6WyaXVUJ59V0FF8O2z6=d7 z0d?d~4^21hFJni}0Rmr$7vd!Vk=$wGg?PyUBJd?PUILfPosNBN$|Lr301JE}UWk_f zL~^Hz7vd!ch`^Vj;U!gv>b<|A>4yDf>_fRQlvvhk0 zpVv9>^Evn2&j;qq5<>hY0P_Xr%Wo*MB>A7_3jha@bNBoEqk9A4?jr>5XR&bqUHHCR zOL}MRz(3u0FdjU2tj)NksW4w)zARfhrktUp+PYYwnwi7;PazAU4}n!*1XUw|?YNFn2XpJ+&nDyY^qO|AaV> zti_?#(qL}=`?&?LP2jc3YU6)-JUk!3^TBdXtQCg&@{jR_e3AVp1Ug^Q_T%D#TWtSV zT_>#7Z@4tL@BXpAgYUQSJF=~i4)YJ@-wL!?H45en%$HTku|mTCGrla?e`3&-)olNq z%?9n7v;W|I)$a#ZP{)#}|7W^cu&?m%8kY3p>H)Za;kj~kcKn(G^9AP1udG--2Ik8@ z&KI2h|LyrboZf-Q*?-)*z*W}?tM?py-4FJ|;d}e6MUZ7_@K}fWvWyaI24TLyd|8tq z%MxL}peb;B|2RD?Q_U|yTpr-px*xzVWfn)kx`xN*Vm|y%hWP^X<#!xeoDTB^=F4I} z{7#1X@;~$CFJ1HF`T^GU|E%jJ^o8{Y>u(7meiMNC0`uiJ6j_oC^9AP15<>ig0hn7b zxBeG8!F+-F0`p~c4Z!0A9v`c-q^k>6(F!xty+pj4wUtqrc%8J!v zV7|b7S)Cofru^6V@~Z|`kHP5%ceY^uH{iHu1I_);02jY{snt_pJXn9Lv*Xtkm@hD2 zer3h#F)&|XzO2rUUsL{TeEC%atH;diX70H_^Y<3O-DBvtzcE-15yqRlM-!~S)d;dI z9_9MG{&Rc*dpo%M{r$;#J&^6kbAfQr2ExaMfeQb0 z?XXeyhZ*m-y;}w z0*v=J?y(H(Z;ppB|JIt#mZrhu4IXbx8L@T<=F5MFFTb__fDE`iOZ?jOSvz$(7B0MY ziM3NSs3Q!D3#^|!KK{xlZW-=At~<$SRIR@Td(#=`phQ%$X$Z)N?F z1G7H&`8YBoHC@4fGEMI;vC`pAFi@%d0b8;@Mz%i=gSa- zcT(WVfk({$0c>*@7Y8II1UWi+xasZO!C`D-%0bWy;X$kr2_y{(np?WNDCp>DfKP-L z-f{9zH~RYf+Nf?54(n(9A4ybJ*6x--Gddu*g_XOdBarhT2+eHi;^qNCc&|V_m$!!t zP96p_tsAHy1mWX;IY4XsU1q__7T;x&xjY7XiXe{?Py?yCxtlEnk>l_=yew@24!H}E zMVzcKPC!lt@^(u*b1NXX0(qmOqce~xNP*06u?WX@k!)^m@dMM`-1dk3y)B?5_+TZB zyNiRl&s^Z2d~tN}1p1nD>%l9jo!c%w@Jn(4u*{ z$X`7FN>MG0ckEjQVZjuYA`QbxH;?1 z<0HMC9JFx!z~O}0S*tF{$=2@5`XC1!Qs{ zzjC+N$JGIc^WN7^5f?v~$IVj@S7&YzKWyWsf@>qlgJN`a2htoqO32cD?z0#{yu62< z!7dOF%A$;|-HmbenX7}9wURQBK^;(5)=q}=bwdGm?Z9E;$_BeQ%yAv$M^Ad+9Hft@Us-z{LX{qCOrpSHp1&)E_nK;-Q1n52za&4QW7%kP^fJc>-?%IYS2_ zO!e2ouORe2MFld4+#p-X8rn1;vrxilF2xDF>O;OzFXROBxc?}(V=mPi@&fTQKVrX^ z-2~aprFzYku!lOqCpe;+(c95m(Hh`Y9(XadG+Jh%90$chJ@J+CmGF1rtKe@~h=D9Y ztsQ`OgD@+^se?3Y&^kBp>a@^?pLlqkcr|z*@fz@2@!mqap-*!t^DW(v32pVVoRU&x26HOHbh&c~ArkW0?j!eXISpeGkJu^i=R3I?(Qxzv~EM1`PkqbgD5s47$m zsu1Epm7?yTo}=#JB2;v0dG$IC(h`5NjinxWiizr7_A!-nB5p9TW#2{i4i9(VhX_2f*UZgNm z8mWL(M;ajaB5jebNMB?iG6I=^OhaBr79h)!&yWqsR^%7t2v`V^py*MYs7)wolp;zS zWs0&vxuK4sLQ(OkG*mXK2=sLwsulGWHG#%Q)1W!fn*cv|p$*ZNXjk+xbQn4jeFa^B zeu%C^x1sydvv_29EO>%=TkurzjPY#oyzzqZ;_)uy<>Ng9W2Xae6dxa-9-j|?GZ>pD z_y_S1p2z|X-i!>_?_#~&deAYdX8B#l%STNgJ7JHgpi$3 zj8KKpl+cMVfH0o$8sUAyTEfqSQ$&Ue25~6E)f+Gy(H=&nj)qm<|CFR zHXuGo>`$CPoK5_Q_#N>O2?@!15*ZR*5)8?4k_3{QBu_}%NG3?BNd-t1N%xR?l17kb zl9rLaB^@FoBjY8LCo?7UAd4WoN>)Mko@|_)hFpkTmE4Nlk35OIfV`Ic8wC*s4}}87 zJ_=uoc#7K;FDbrK5>fI}DpHzL9-~a6yi3_gIYLE4B}%1DZV0)ArL*(@D_nraMHJ zM0by_l^#XUORq}rKp#n;OaGdFl7W>$fx(*L6hk&cEyEb&I!1X$YsOH<9L5I5NhS^^ zWhQ&3Xr@A@7G^ZF0JAQ$H*+%cBj&H`XxD9BXSptPUGBOj79@)Riyq5imUNcqEF-LJ ztUFm+)tpo7`PLh*4_KeGzJ-gJOPb4uD~{^{S3fs9w6Z%4~Gnn6|NT6X_;}OlZ(jijdJNy!J2 zGg6zS+@!Llx;JxgKCt=x=0<5+X&vb>=_fLHGCO4aWJ+Wvwn%O9*pj=YZ|kP5_FJ!Q z?b^n-&3fCVZSAt_WzA(T%C>Ij*nVJp>h@MSPC0YAG`WxRT=G`(m*qbzY*09;a9!cs z4v`&hJMwpoDQ;0bs#vCkQrf8$s#L8^t!${AsQg}Ky^5_$mP-Fl$(@IHmhHmZrM@eA zSEDMcs-@~x)qb_jYDd)`s*|egt0$^|)DX~c(eJ}&*T1GeYOuo~(xBOp-_XOMWeoIBmvzA}1FBc-M*A$=>OqGn?}v=NcD5 zmtdDx*DbDRT>IVB-7dQ$-1oZQ@t_4uo#&qXoUHDe3nIO0O%hR^IjQxz`}pK_M? ztmoO*1dW6{=eW;BotsXyOKeEmk(6_u<@~AhV;8J0)F#U(XQ!~HoKBfc#iYKuxbtE` z8gE*BI)1uWdgrCxmnt))Gp=4{zI^)fOr}$2>lM8#53g>%di5I1wa6@FmSI};o;Q1L9k}&6S3UQ^ZRy+Dc|3XN^J(%!3lIf91^tC~g{^l? z?$q2>yZfMMYf*l&P;q7nXGzjMx_goLiSGyA$Cmn*j+D8SeJ!^wZ-21=L34$1Mg2qF zht-eNA3b@j^0=Z>p|bRe?30qJEme1)N zqMB_r_iN>9%j=Zt9@neZKWorxsCjMp`c31W#@07hZ@S((yd7xrYMN?3{toY5XbWXa z{Ck%7X&*Lx$Z3^oz1ODH_WYy%$L4m+_Aj5@KTUT8c9M0*e`f!DwM(q4q+6x?RgYOu z#}}6`Q(uF7sd~?U$fV#MxBNDchmhjpKRk%R zId>azu{i-j>IRl3<5=tvD+Cb~LC}mF7CUtTi=DXy*6nX0sLb{IbssJ_J|zUD3gM(z z9p7+oOPr263Qas8!wD(-mL%&?gE|1u0Aj72Ep^^vPUoXjx}LI+)}ZLx2# ztq`S}imD1Y0ti97z#oKdgN`d7!T=>{YC-~_EVx`pFoOlucXez>totFWAo%Au2#^qT z5;M1nfKwUn0tx4E7dyD{_jgq4{5A$IE)}_u8v&)F7G$v80uI{?N$c1-dpNt>IlFL( z3yDC|J5)8nrOTX~lX>Pu%6itYw~ry<`h-Hr_nVh zSnoM zUVcI0ox4R3Djq(1T=}HxX>DD7!|TR3Z=2dawtwpA{M^;uGcY(bJTf{qJ~4@F7p|QP z?t9CYwTlY03yDUf&;+=4A&}mf`4iIah`xBMoQO+Yubudo9>X$ao$PkXR00iv7)zSIG~Hixv4AAF7B^0{HVu$#fk4{IBn~{ zEcafBQJQC$Oy7yfJ+20q`u7KQxNeLmt|p;uPU0xs!!l48sZdsSmEi%~Iz@yT8Vji? z?|N)}{E8}ysr*yy%YDNLnSq_bWqO`wI_!)ygYL-*pPLHp&z>mXM^~`Xyv{wMC&2|0QlxtM+YrH7>;bSA?9n^8LT@K2c(lW|9<740VgmRSYxpQnqg~(yH6V zFh&9im-qY>T^ZPZ z(W#En+wp0E$eSk|E-LcUvSqUJ*Uw4xq2h%`3B*{AHhy#WKT-DbxoFLO^OurkLk?mM z^*goe?ftm6n3Nc{WqBKS)c011(_tagO!NlJ^K8mrKPe$`+*Ls(X>GbFk0}Yj@IytOn|0A9;=@-pKg+n9$pq*sN1e8cp%V zR8j0AZ#q*}u65wGGw)0?lxdAH)wH_G&`RD9Tl1YPm(_NKKWROV84kM`ceP7Jz$QCU z(JWbfCjY9R1veJ5KkIW>u-K=1+fCQ=!!I!J4;FU&WUqhsgdQ;nRgz3SWO1*=&!-3) z&aVx>C58^}x_#uG{~f+e%>>Ddn}dfZoJY{n7tD0SSUyKE(pR#E=nqRy7;a~L5y$Li zlslz8VO1&>nUm~z!mTK>wK#Kv*VQ!Q?lomH|A=qOLz^0sNPT$TSj`%q!$L&#Iao+8 z7Yltb>_R=5lsb4KXYbU!%RgWCoLG5w*JYkd4zS7xz*N@WrGI2Rf zStdsd-*l&+Z>4qQpC!55S!eHi=E2v*o}D@hZz1sN6*sD)*X(0gJ!o+NI$=M#_mdOL}9PJUq!sVxCx=h+lqomWmUuLf-tY0Ze|R3 zM{AKBu);zmf_?5hvoH0hg$K90lT7cLOph4M>zkO7OY{p;*k*c+>ruDdmND)AtdzHf zenSm%eX+BW=t<94gF^yGM`meeCHtjA>3rC~0qxB0n!IBWG%NSz8*2yV!_cIk1QsF= zFwVn5QtZQZW04-S-J?rjiaZ;W$gQodNM?;sKDaL z0jg>B9{XYB%HR8LW&`wZ7k+!#4@t_wz^+g+& z>E~OlrfBLuBu>?JMC67J+3pY7EajC$HgbATrz%CvLzS+l=>0y%QAB)Uwh^w;`ftnI zM9X z5>hHGgw&m6KBfO0`Jf~81;Y#46IqC|14nnNu)lg8*3a`$h3^RgnyQ}a3D-O3G*S22 zpreD;DPp+;=RG`RBd_$n`ot#Bv@tr0qaj`Q#Ne}O^yG;KiuxI%tFtdW8Kw{Ki79P~ zb7uE%D;2|B{-}PyvL3%{(kJ!ECBFi>lV9q(YtJ}D<4+N3a10`+Yf{FETK&pg<;jQo zVrLw0x_()AVp8hEXZ4wjlvv1w<<^X)<>Zz1u`fzb&p42XjGVgZ*B4(o8)!FMQ@DSm zybTK-Wu$n?IAe!U$;=$Pql|?P_0Ar;<{Mm?)>O8=!JXuk@%o+j#JQ4pT;Yp7t|gnL zEbP#Mg^ad24`BnTCjsHY9cM>|xMv$`{kquv`a-bK<2;GxFC?d9 z>GE@;riiK=n4qw*tJ~9V-+z&Yh1~pJfE+p9(U|BKnuEDSYV4)O>)gZ}Bkp9EaAoyc z`t0-*o>gYFpxeIFTKG$+a|5fo_6fsVN!oWeawQx%^Vs+E9gZ17j$w{aO>N!6T8B%e zcS&m=8&ICUd&-6wV%HOlD;N)Nv%h*I!6vSFV|MC;cDDy4qI)6^?XG+^Y1GKZwC_V( z=E)n@CTTIPVkO!sBJQ8g@ZW8YO1EpZoUr6l?k_!_KjbPD>CRJUvgb^6T-*kMcEK?t z4hZVO$Cv3W_%wc|&CdF;kJUMwFqsXXZb{3lyGgeq!IvW}4Tb|}{XNzdXOS@0zSrmN z4k{72HYmbqVT3sKgh`fS54jy`ze>MU(8WEE4%_5PJ~X?<9eFpRRRP`V`cmtH;H?~4 zIht)^j_*5ZwuRnBjcNEN*brjRct)tYE*e}8!@ zs`~!P84slSOjo2_=&f0$+Hkd(TdiWNoK+EuSSZpWws*76fIGr`X8gYScn;Zw&83LZ z>edN+f%tDHgQpFK_gi_5N; z_w1xD9H``dzil!InzhIskws~g_#f%CJ93hm%O`@)7>VEzAr;Xkh;%yh@^oqLg)=QF zpD-%#dS;&9w}}RGcJG8R7NXKEwtG={EVY7^u+4XLmihD)Z}V(`?lDl&)0ekL9&C2M z`cR~7n6GFY8K9*)22^iAqEXcwwiN{J+1+lqRWQ4rs}m01>Ywep4#U(($8>f)_crv zG~=cCE->SdPM^g>Az4kXbxoi0^Q#i&Pj*PNP>UTxHRa0XI+k|vFnp35FRH1%&QVY# zzSAqV;6bqgd0@mLzS^SxnCm@{)QB7Vuuya9lS-as#_R~@nXKKBGnH2CF;RkYIyGdUTtl)yhdttm z;*aV*V&!<`(;1oW9z4gqTWL%tW%A;8qQj12-m>M>VXi(Z|2ixHOhXvO*Jm${v5@#I z9ZRBSd+$X0I^)gy&HdSt>%$^Vhce>oVs1NwZaD6F{iq^pN^I!0+;c1xq%x`5zD25Q zc89k9JF4!!z34h+XBy3|A?@<&;w%t>o2F_aPh6d^)GQVnhi;9K`~4_sl6y0mBxSo^ z0^zfuY9EfV$_xxs8})U(dbOv)p0HVW&ykI<_{1zadpN}}MNrjSWoABgA^OyQ|EQ*~H7%lCwZBi% zUinsl>W$tayhs3Ryu7l(3{>|^UW$R=37sv^Q~qQyZT(bCIljcWGvV% z^A;0bmw^re;>xs4O}XJ4!w5x~21T@xk_FElpopTEyp~{S`-PRPW!e=ScL5 zd4{0Py>5#_eNxsT96=?!k3}3MUj4EYg?;_2+84O~hwmtC<3E7$7PkpSJ9n}Jf;ge@ z%mj(+V9Uh^Y`al`{RtNr4`CNKYxlXGD4|1+4%cIw555zjRsKLSLy+zr&NCq;)lPK( ziH;p-+%U?(B$+CI6DP)vbNEBnq4Rkd!oa9ZF4s3Z?VVMNPd8&ovx9Ewz2GTZhenFJ zH$0or31By3eHT!+MQY5!;5L8xg`2Gw*oitVh9*Kmo67TN_a&B5Bs!L>xp-(XnA=V0 zYCLy}qrcrp_Eh8f#0{Cid*+*Gk7G!Qc3`AMWlk76zfUu&;J9l?C#T8Qy6s_zE9b49 zS9{wfX5ul}$wR?;W4se~_s`WGt0cB3mrJTaH#%%oX!H!Ero3?@$S(tFa`W_X=8=qnrRqVf#NSgY~zH@&p^?*)I74P;DvvTS6 zGS_=ezCmW=AP(AD0gZPbH4Dh=28)&B`GhvG4`on(`sRp{J%wjN8y$5=_o=(WomUh4ukkz+ z2*gi#D}~hY7ZXtqLF|5UH$%p~N-ylgbVIyXx0%mT_Tl6`+T&fk)*IO$a7-q<(#3=UCNcqFI4mCCU$sCZxIfm`}~e#^J^br zuNmhO7tamb!g3zJ$_u1O4{+8K?~H~93Y}hXS|dD`8`d6Dhl_2J`tgOhvD(rfSC$}R9LWH$f$KF%cF;yvw0C zCLB5$V{UAB63xkq$m|x$(BJy}!OOGz=rGws*4bx=dkh;h>EqwA*4XWCl*r08(st~l zGgA>eI93!N=lZNs@mfZ&y{dxQ=hreN852!xa$d!s9)CIWEHypJxb&gK^-NRtp0>L3 zQHKq_re99=+C z%=8^z5*x5giNU5qyGu_DNBOF{u&70J_pQG-vX3=Yh(S%Sve2hetzTjM` z_MzAHYytLsGfmHL5^8lLwBGshO&ld`+GM8|6H1zTpa%cdIr9V6taOaKlyd0j!Ua<0CzXHx#{H=h9ijPmXGBxNO&FU~w7< zqjr69>!4oQ;Ruz(drkFKrm|1!D{eiEt+tb!+;vW#DlEk7^0C6yqhH#JIE+@FMIPaA zWQ+g?B6pF2@WNPo|1c17`kLAmBBb}Y2H4Zyakdb&TzRdzgHR||zNwW>FNZ`emcL4+ zaDQa;w=BKr$D2lPe>hxlrt11e+P48iwOw(e-?-G|Nn%wmWDZ+SxNW}tkRV$;J1jWv zTUp)P;ZYll*c*W<-aDh@rvfuQSOYW}hb+kO#>O58s5n^fDMtkh*`8~c+qH-Pb!1us z-JXNKUEYnuNeybmhomg@kCT&~LOntyj@e^oqF?N*uwgxP>1wfA{AKC$*~AUD;vbvt z%+Pp~Tz^DITM`%aPH{KGdQ}nfT$$_0fvLF1_XZMj)$K42;oHdDb;)i?`5AsW^SqTM zH0UY$$9?X_5;qkx%5Rz$;+yo6<;Smb_j@>bSsgYA;;hA)%@0} zFuc*ST-D$6i9Jo3@I}%_=^n}28@(6wu_Lvd7uqol@g&nsN(YAczZsooC~m)Pj=A4F z48B47)dc9rvGenT?|s&RBcPNJA>zkZOyIO7ZvN*;LnHW8e|&`ojKqJ_)f<#MeH5X* zmy?G*(6Nt&(pu1JkD)<`;@kC9O}Fi}L-HR-IGo)yHL_Jzjd0Vog5pfs0h`VNF`rl_ zubEs)D}wEIwtT9kXL-@rJ=B_CaGTiPKAja9$bX&su}K2=VZtg&>}1fUdxGUV$^26T z$zm^s++xvZwnCUu`5G4(S6%oJ8glW2&e_kV4{pT1G`ks1UhARM#uH$eD)e->2!^(@ zc)T#}Y1OsT{-|E{`u2_Px(OQ2{`wa!c{5b6tvj8R11n6EdEZ{6WTSn5$IG!9ozyy8a|3s#}GUL`}G)&g=0t>X2+63>Dr&jm210p* z&MPfOsT0vGaTzReFA8t?Ds0$bpmRcHXS%ME_Qrr4Ee-zLWw`8vit)44DqT_wwa5-M z7oz4W{$#n|n%or+W6lZfsZ4>0#8ASJqq%L4DV}_`)vCA<_|NCFV$dxM9NRZ(# zNc`<@4O*l1+k86{Ji?70w7rw7)>D|sne3Ol-`t8N68HQBSY1WDR z(UybB+q6$JRaw$!aH>~~-w)xc)?>T<(&_H$iP)Rl9s1eZ6{#Mb=b8RC@ca<%rtv+R0pcKLPOH6kM z3^?#E{2gO1Ad0KO|A;058j8DM{S{5@FQADLt%fGCDjHwN32C4uAUD>qbfRWzEkP}w)2Lb$n$V^PH*=p5N&-cp zFNYXOS5;L5Cv3EPp=A)~IDdthYs_*8Loqhd=b&qhPc4U_2giMu*J6%F%ORSto-=s} z8ghbjc|+)Ae}$Oqpyd!x`i@v$0Ob9ox4agln!iHK4UOdx4K*-y^R+W3H|ZXwRP@uReROj*P$ePmouiU?g7Lt)>TDt^^ihDW6IA zEkf2bclWS%Tf*n>QGce5O-n|K!~R9ctBw>=kw5CfrO@sN{s?;YS}VRt?0Ez%y9}6V zz~r{b`I@f)JvV87Z^DXGCv{dWmp!oBo55VW6z-Vy+HfmQqj?tSA7FY{1UvQaEk5MQ{sTn$z7P7A>D@ z&E33dzJl6GFfE6>;IjyBfk2D8V#Ucnuv_K>E6`mLFaS$y_vF~xaC5WN^4f7qGCb)8 zme~$8cqtrrz}j$gCSW<-MCy?@FG20H!IESBBG!R(szGbRiLTfePiDb27Qm|pOj(Mz zADG}@X1PVm`#$cL*YD~x4W&5H8T{bX?9%3L311s;ZYEw1=aTmc?;(J@`wJXd%-V2s zW^Os$>hmMz*%jaxt$!Dd-4$2C@s1_xXt0eF&b)j9MxXE<*~8A-(Ha~c2Cs{H0<3lB z94FMmaad^!B_)LOOBY}o$=?wit*tQT!Wc($TY%t!v9bByyq^%uk1YS>$K7no)ebN( z*U|E?oPL6AFQ=Si#F8(dmKR_CwZ_llOcmdYFZ+^XIl%HA&7T1KE58FQ+234VeCeLr y&%)sMuT_h5Fu%jLy!6sNgT +
    + +{% highlight scala %} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.SparkSession + +val spark = SparkSession + .builder + .appName("StructuredNetworkWordCount") + .getOrCreate() +{% endhighlight %} + +
    +
    + +{% highlight java %} +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; + +SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCount") + .getOrCreate(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +spark = SparkSession\ + .builder()\ + .appName("StructuredNetworkWordCount")\ + .getOrCreate() +{% endhighlight %} + +
    + + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + +
    +
    + +{% highlight scala %} +val lines = spark.readStream + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load() + +val words = lines.select( + explode( + split(lines.col("value"), " ") + ).alias("word") +) + +val wordCounts = words.groupBy("word").count() +{% endhighlight %} + +
    +
    + +{% highlight java %} +Dataset lines = spark + .readStream() + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load(); + +Dataset words = lines.select( + functions.explode( + functions.split(lines.col("value"), " ") + ).alias("word") +); + +Dataset wordCounts = words.groupBy("word").count(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +lines = spark\ + .readStream\ + .format('socket')\ + .option('host', 'localhost')\ + .option('port', 9999)\ + .load() + +words = lines.select( + explode( + split(lines.value, ' ') + ).alias('word') +) + +wordCounts = words.groupBy('word').count() +{% endhighlight %} + +
    +
    + +This `lines` DataFrame is like an unbounded table containing the streaming +text data. This table contains one column of string named “value”, and each +line in the streaming text data is like a row in this table. Note, that this +is not currently receiving any data as we are just setting up the +transformation, and have not yet started it. Next, we have used to built-in +SQL functions - split and explode, to split each line into multiple rows with +a word each. In addition, we use the function `alias` to name the new column +as “word”. Finally, we have defined the running counts, by grouping the `words` +DataFrame by the column `word` and count on that grouping. + +We have now set up the query on the streaming data. All that is left is to +actually start receiving data and computing the counts. To do this, we set it +up to output the counts to the console every time they are updated. In +addition we are also going to set up additional details like checkpoint +location. Don’t worry about them for now, they are explained later in the guide. + +
    +
    + +{% highlight java %} +val query = wordCounts + .writeStream + .outputMode("complete") + .format("console") + .option("checkpointLocation", checkpointDir) + .start() + +query.awaitTermination() +{% endhighlight %} + +
    +
    + +{% highlight java %} +StreamingQuery query = wordCounts + .writeStream() + .outputMode("complete") + .format("console") + .option("checkpointLocation", checkpointDir) + .start(); + +query.awaitTermination(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +query = wordCounts\ + .writeStream\ + .outputMode('complete')\ + .format('console')\ + .option('checkpointLocation', checkpointDir)\ + .start() + +query.awaitTermination() +{% endhighlight %} + +
    +
    + +Now the streaming computation has started in the background, and the `query` object is a handle to that active streaming query. Note that we are also waiting for the query to terminate, to prevent the process from finishing while the query is active. +To actually run this code, you can either compile your own Spark application, or simply run the example once you have downloaded Spark. We are showing the latter. You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using + + $ nc -lk 9999 + +Then, in a different terminal, you can start the example by using + +
    +
    + + $ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount + +
    +
    + + $ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount + +
    +
    + + $ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py + +
    +
    + +Then, any lines typed in the terminal running the netcat server will be counted and printed on screen every second. It will look something like the following. + +# Programming Model + +The key idea is in Structured Streaming is to treat a live data stream as a +table that is being continuously appended. This leads to a new stream +processing model that is very similar to a batch processing model. You will +express your streaming computation as standard batch-like query as on a static +table, and Spark runs it as an *incremental* query on the *unbounded* input +table. Let’s understand this model in more details. + +## Basic Concepts +Consider the input data stream as the “Input Table”. Every data items that is +arriving on the stream is like a new row being appended to the Input Table. + +![Stream as a Table](img/structured-streaming-stream-as-a-table.png "Stream as a Table") + +A query on the input will generate the “Result Table”. Every trigger interval (say, every 1 second), new rows gets appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want write the changed result rows to an external sink. + +![Model](img/structured-streaming-model.png) + +The “Output” is defined as what gets written out to the external storage. The output can be defined in different modes + + - *Complete Mode* - The entire updated Result Table will be written to the external storage. + + - *Append Mode* - Only the new rows appended in the Result Table since the last trigger will be written to the external storage. This is applicable only on queries where existing rows in the Result Table is not expected to change. + + - *Update Mode* - Only the rows that were updated in the Result Table since the last trigger wil be written to the external storage (not available yet in Spark 2.0). Note that this is different from the Complete Mode in that this mode does not output the rows that are not changed. + +Note that each mode is applicable on certain types of queries. This is discussed in detail later. + +To illustrate the use of this model, let’s understand the model in context of +the Quick Example above. The first `lines` DataFrame is the input table, and +the final `wordCounts` DataFrame is the result table. Note that the query on +streaming `lines` DataFrame to generate `wordCounts` is *exactly the same* as +it would be a static DataFrame. However, when this query is started, Spark +will continuously check for new data from the socket connection. If there is +new data, Spark will run an “incremental” query that combines the previous +running counts with the new data to compute updated counts, as shown below. + +![Model](img/structured-streaming-example-model.png) + +This model is significantly different from many other stream processing +engines. Many streaming system require the user to maintain running +aggregations themselves, thus having the reason about fault-tolerance, and +data consistency (at-least-once, or at-most-once, or exactly-once). In this +model, Spark is responsible for updating the Result Table when there is new +data, thus relieving the users from reasoning about it. As an example, let’s +see how this model handles event-time based processing and late arriving data. + +## Handling Event-time and Late Data +Event-time is the time embedded in the data itself. For many applications, you may want to do operate using this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time a column value in the row. This allows window-based aggregations (e.g. number of event every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on streaming dataset, making the life of the user much easier. + +Furthermore this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0 yet, event-time watermarking will be used to manage this data. These are explained in more details in the Window Operations section. + +## Fault Tolerance Semantics +Delivering end-to-end exactly-once semantics was one of key goals behind the design of Structured Streaming. To achieve that, we have designed the Structured Streaming sources, the sinks and the execution engine to reliably track the exact progress of the processing so that it can handle any kind of failure by restarting and/or reprocessing. Every streaming source is assumed to have offsets (similar to Kafka offsets, or Kinesis sequence numbers) +for track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. + +# API using Datasets and DataFrames +Since Spark 2.0, DataFrames and can represent of static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point SparkSession to create straming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the Spark SQL Programming Guide. + +## Creating streaming DataFrames and streaming Datasets +Streaming DataFrames can created through the DataStreamReader interface `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source - data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. + + - **File sources** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for more up-to-date list, and supported options for each file format. Note that the files needs to be atomically put in the directory. In most file systems, this means that the files should be moved atomically into the directory. + + - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The connection is only on the server. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. + +Here are some examples. + +
    +
    + +{% highlight scala %} +val spark: SparkSession = … + +// Read text from socket +val socketDF = spark + .readStream + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load() + +socketDF.isStreaming // Returns true for DataFrames that has a streaming source + +socketDF.printSchema + +// Read all the csv files written atomically in a directory +val userSchema = new StructType().add("name", "string").add("age", "integer") +val csvDF = spark + .readStream + .option("sep", ";") + .schema(userSchema) // Specify schema of the parquet files + .csv("/path/to/directory") // Equivalent to format("cv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +{% highlight java %} +SparkSession spark = ... + +// Read text from socket +Dataset[Row] socketDF = spark + .readStream() + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load(); + +socketDF.isStreaming(); // Returns True for DataFrames that has a streaming source + +socketDF.printSchema(); + +// Read all the csv files written atomically in a directory +StructType userSchema = new StructType().add("name", "string").add("age", "integer"); +Dataset[Row] csvDF = spark + .readStream() + .option("sep", ";") + .schema(userSchema) // Specify schema of the parquet files + .csv("/path/to/directory"); // Equivalent to format("cv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +{% highlight python %} +spark = SparkSession. …. + +# Read text from socket +socketDF = spark \ + .readStream() \ + .format("socket") \ + .option("host", "localhost") \ + .option("port", 9999) \ + .load() + +socketDF.isStreaming() # Returns True for DataFrames that has a streaming source + +socketDF.printSchema() + +# Read all the csv files written atomically in a directory +userSchema = StructType().add("name", "string").add("age", "integer") +csvDF = spark \ + .readStream() \ + .option("sep", ";") \ + .schema(userSchema) \ + .csv("/path/to/directory") # Equivalent to format("cv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +These examples generate streaming DataFrames that are untyped, that is, the schema of the DataFrame is not checked at compile time, only checked in runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. + +## Operations on streaming DataFrames/Datasets +You can apply all kinds of streaming DataFrames/Datasets - ranging for untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the SQL programming guide for more details. Let’s take a look at a few example operations that you can use. + +### Basic Operations - Selection, Projection, Aggregation +Most of the common operations on DataFrame/Dataset operations are supported for streaming. The few operations that are not supported are discussed later in this section. + +
    +
    + +{% highlight scala %} +case class DeviceData(device: String, type: String, signal: Double, time: DateTime) + +val df: DataFrame = … // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: string } +val ds: Dataset[DeviceData] = df.as[DeviceData] // streaming Dataset with IOT device data + +// Select the devices which have signal more than 10 +df.select("device").where("signal > 10") // using untyped APIs +ds.filter(_.signal > 10).map(_.device) // using typed APIs + +// Running count of the number of updates for each device type +df.groupBy("type").count() // using untyped API + +// Running average signal for each device type +Import org.apache.spark.sql.expressions.scalalang.typed._ +ds.groupByKey(_.type).agg(typed.avg(_.signal)) // using typed API +{% endhighlight %} + +
    +
    + +
    +
    + +
    +
    + +### Window Operations on Event Time +Aggregations over a sliding window is very easy. Applying a sliding window is simply a special type of grouping - each time window is a group and each row can belong to multiple windows/groups. Hence aggregating over sliding windows is same as aggregating over this special type of grouping. This is illustrated below. Imagine the quick example is modified and the stream contains lines along with the time when the line was generated. Lets say instead of running word counts, we want to count windows within 10 minute windows, updating every 5 minutes. The result tables would look like the following. Note the window notation 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. The counts are indexed by the combined group (window, cat) + +![Window Operations](img/structured-streaming-window.png) + +In code, you can use windows as the following. + +
    +
    + +{% highlight scala %} +// Number of events in every 1 minute time windows +df.groupBy(window(time, "1 minute")) + .count() + + +// Average number of events for each device type in every 1 minute time windows +df.groupBy( + "type", + window(time, "1 minute")) + .avg(df.col("signal")) +{% endhighlight %} + +
    +
    + +
    +
    + +
    +
    + +This windowing based on group automatically handles late arriving data. + +![Handling Late Data](img/structured-streaming-late-data.png) + +### Joins Operations +Streaming DataFrames can be joined with static DataFrames to create new streaming DataFrames. Here are a few examples. + +
    +
    + +{% highlight scala %} +val staticDf = spark.read. … +val streamingDf = spark.readStream. … + +streamingDf.join(staticDf, “type”) // inner equi-join with a static DF +streamingDf.join(staticDf, “type”, “right_join”) // right outer join with a static DF + +{% endhighlight %} + +
    +
    + +
    +
    + +
    +
    + +### Unsupported Operations +However, note that all of the operations applicable on static DataFrames/Datasets are not supported in streaming DataFrames/Datasets yet. While some of these unsupported operations will be supported in future releases of Spark, there are others which are fundamentally hard to implement on streaming data efficiently. As of Spark 2.0, some of the unsupported operations are as follows + +- Multiple aggregations (i.e. a chain of aggregations on a streaming DF) are not yet supported + +- Limit and take first N rows are not supported + +- Distinct and sorting operations are not supported + +- Stream-batch outer joins are conditionally supported + + + Full outer join not allowed + + + Left outer join with a streaming DF on the left is not supported + + + Right outer join with a streaming DF on the right is not supported + +- Stream-stream joins are not yet supported + +In addition, there are some Dataset methods that will not work on streaming Datasets. They are actions that will immediately run queries and return results, which does not makes sense on a streaming Dataset. Rather those functionalities can be done by explicitly starting a streaming query (see the next section regarding that). + +- `count()` - Cannot return a single count from a streaming Dataset. Instead, use `ds.groupBy.count()` which returns a streaming Dataset containing a running count. + +- `foreach()` - Instead use `ds.writeStream.foreach(...)` (see next section). + +- `show()` - Instead use the console sink (see next section). + +If you try any of these operations, you will see an AnalysisException like "operation XYZ is not supported with streaming DataFrames/Datasets". + +## Starting Streaming Queries +Once you have defined the final result DataFrame/Dataset, all that is left is for you start the StreamingQuery. To do that, you have to use the `DataStreamWriter`, through you have to specify the following. + +- *Details of the output sink:* Data format, location, etc. + +- *Output mode:* Specify what gets written to the output sink. + +- *Query name:* Optionally, specify a unique name of the query for identification. + +- *Trigger interval:* Optionally, specify the trigger interval. If its not specified, the system will check for availability of new day as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will attempt to trigger at the next trigger point, not immediately after the processing has completed. + +- *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in a HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. + +#### Output Modes +There are two types of output mode currently implemented. + +- **Append mode (default)** - This is the default mode, where only the new rows added to the result table since the last trigger will be outputted to the sink. This is only applicable to queries that *do not have any aggregations* (e.g. queries with only select, where, map, flatMap, filter, join, etc.). + +- **Complete mode** - The whole result table will be outputted to the sink.This is only applicable to queries that *have aggregations*. + +#### Output Sinks +There are a few types of built-in output sinks. + +- **File sink** - Stores the output to a directory. As of Spark 2.0, this only supports Parquet file format, and Append output mode. + +- **Foreach sink** - Runs arbitrary computation on the records in the output. See later in the section for more details. + +- **Console sink (for debugging)** - Prints the output to the console/stdout every time there is a trigger. Both, Append and Complete output modes, are supported. + +- **Memory sink (for debugging)** - The output is stored in memory as an in-memory table. Both, Append and Complete output modes, are supported. Note that this requires all the data to be stored in the memory of the driver and does not scale well. Hence this should be used only for debugging. + +Here is a table of all the sinks, and the corresponding settings. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    SinkSupported Output ModesUsageFault-tolerantNotes
    File Sink
    (only parquet in Spark 2.0)
    Append
    writeStream
    .format(“parquet”)
    .start()
    YesSupports writes to partitioned tables. Partitioning by time may be useful.
    Foreach SinkAll modes
    writeStream
    .foreach(...)
    .start()
    Depends on ForeachWriter implementationMore details in the next section
    Console SinkAppend, Complete
    writeStream
    .format(“console”)
    .start()
    No
    Memory SinkAppend, Complete
    writeStream
    .format(“memory”)
    .queryName(“table”)
    .start()
    NoSaves the output data as a table, for interactive querying. Table name is the query name.
    + +Finally, you have to call `start()` to actually to start the execution of the query. This returns a StreamingQuery object which is a handle to the continuously running execution. You can use this object to manage the query, which we will discuss in the next subsection. For now, let’s understand all this with a few examples. + + +
    +
    + +{% highlight scala %} +// ========== DF with no aggregations ========== +val noAggDF = deviceDataDf.select("device").where("signal > 10") + +// Print new data to console +noAggDF + .writeStream + .format("console") + .start() + +// Write new data to Parquet files +noAggDF + .writeStream + .parquet("path/to/destination/directory") + .start() + +// ========== DF with aggregation ========== +val aggDF = df.groupBy(“device”).count() + +// Print updated aggregations to console +aggDF + .writeStream + .outputMode("complete") + .format("console") + .start() + +// Have all the aggregates in an in memory table +aggDF + .writeStream + .queryName("aggregates") // this query name will be the table name + .outputMode("complete") + .format("memory") + .start() + +spark.sql("select * from aggregates).show() // interactively query in-memory table +{% endhighlight %} + +
    +
    + +{% highlight java %} +// ========== DF with no aggregations ========== +Dataset[Row] noAggDF = deviceDataDf.select("device").where("signal > 10") + +// Print new data to console +noAggDF + .writeStream() + .format("console") + .start(); + +// Write new data to Parquet files +noAggDF + .writeStream() + .parquet("path/to/destination/directory") + .start(); + +// ========== DF with aggregation ========== +Dataset[Row] aggDF = df.groupBy(“device”).count(); + +// Print updated aggregations to console +aggDF + .writeStream() + .outputMode("complete") + .format("console") + .start(); + +// Have all the aggregates in an in memory table +aggDF + .writeStream() + .queryName("aggregates") // this query name will be the table name + .outputMode("complete") + .format("memory") + .start(); + +spark.sql("select * from aggregates).show() // interactively query in-memory table +{% endhighlight %} + +
    +
    + +{% highlight python %} +# ========== DF with no aggregations ========== +Dataset[Row] noAggDF = deviceDataDf.select("device").where("signal > 10") + +# Print new data to console +noAggDF\ + .writeStream()\ + .format("console")\ + .start(); + +# Write new data to Parquet files +noAggDF\ + .writeStream()\ + .parquet("path/to/destination/directory")\ + .start(); + +# ========== DF with aggregation ========== +Dataset[Row] aggDF = df.groupBy(“device”).count(); + +# Print updated aggregations to console +aggDF\ + .writeStream()\ + .outputMode("complete")\ + .format("console")\ + .start(); + +# Have all the aggregates in an in memory table. The query name will be the table name +aggDF\ + .writeStream()\ + .queryName("aggregates")\ + .outputMode("complete")\ + .format("memory")\ + .start(); + +spark.sql("select * from aggregates).show() # interactively query in-memory table +{% endhighlight %} + +
    +
    + +Using Foreach +The `foreach` operation allows arbitrary operations to be computer on the output data. As of Spark 2.0, this is available only for Scala and Java, not Python. To use this, you will have to implement the interface `ForeachWriter`, which has methods that gets called whenever there is a sequence of rows generated as output after a trigger. It is important note the following + +- The writer must be serializable, as it will be serialized and sent to the executors for execution. + +- All the three methods, `open`, `write` and `close` will be called on the executors. + +- The writer must do all the initialization (e.g. opening connections, starting a transaction, etc.) only when the `open` method is called. Be aware that, if there is any initialization in the class as soon as the object is created, then that initialization will happen in the driver (because that is where the instance is being created), which may not be what you intend. + +- `version` and `partition` are two parameter in the `open` that uniquely represents a set of rows that needs to be pushed out. `version` is monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. + +- `open` can use the `version` and `partition` to choose whether it need to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If the `false` is returned, then `write` will not be called on any row. For example, after a partial failure, so partitions of the failed trigger may have already been committed to a database. Based on metadata stores in the database, the writer can identify partitions that have already been committed and + +- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is the your responsibilty to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. + +## Managing Streaming Queries +The `StreamingQuery` object created when a query is started can be used to monitor and manage the query. + +
    +
    + +{% highlight scala %} +val query = df.writeStream.format("console").start() // get the query object + +query.id // get the unique identifier of the running query + +query.name // get the name of the auto-generated or user-specified name + +query.explain() // print detailed explanations of the query + +query.stop() // stop the query + +query.awaitTermination() // block until query is terminated, with stop() or with error + +query.exception() // the exception if the query has been terminated with error + +query.souceStatus() // progress information about data has been read from the input sources + +query.sinkStatus() // progress information about data written to the output sink +{% endhighlight %} + + +
    +
    + +{% highlight java %} +StreamingQuery query = df.writeStream().format("console").start(); // get the query object + +query.id() // get the unique identifier of the running query + +query.name() // get the name of the auto-generated or user-specified name + +query.explain() // print detailed explanations of the query + +query.stop() // stop the query + +query.awaitTermination() // block until query is terminated, with stop() or with error + +query.exception() // the exception if the query has been terminated with error + +query.souceStatus() // progress information about data has been read from the input sources + +query.sinkStatus() // progress information about data written to the output sink +{% endhighlight %} + +
    +
    + +{% highlight python %} +query = df.writeStream().format("console").start() # get the query object + +query.id() # get the unique identifier of the running query + +query.name() # get the name of the auto-generated or user-specified name + +query.explain() # print detailed explanations of the query + +query.stop() # stop the query + +query.awaitTermination() # block until query is terminated, with stop() or with error + +query.exception() # the exception if the query has been terminated with error + +query.souceStatus() # progress information about data has been read from the input sources + +query.sinkStatus() # progress information about data written to the output sink +{% endhighlight %} + +
    +
    + +You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the StreamingQueryManager that can be used to manage the currently active queries. + +
    +
    + +{% highlight scala %} +val spark: SparkSession = … + +spark.streams.active // get the list of currently active streaming queries + +spark.streams.get(id) // get a query object by its unique id + +spark.streams.awaitAnyTermination() // block until any one of them terminates +{% endhighlight %} + +
    +
    + +{% highlight java %} +SparkSession spark = … + +spark.streams().active() // get the list of currently active streaming queries + +spark.streams().get(id) // get a query object by its unique id + +spark.streams().awaitAnyTermination() // block until any one of them terminates +{% endhighlight %} + +
    +
    + +{% highlight python %} +spark = … # spark session + +spark.streams().active() # get the list of currently active streaming queries + +spark.streams().get(id) # get a query object by its unique id + +spark.streams().awaitAnyTermination() # block until any one of them terminates +{% endhighlight %} + +
    +
    + +Finally, for asynchronous monitoring of streaming queries, you can create and attach a StreamingQueryListener, which will give you regular callback-based updates when queries are started and terminated. + +## Recovering from Failures with Checkpointing +In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger), and the running aggregates (e.g. word counts in the quick example) will be saved the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in a HDFS compatible file system, and can be set as an option in the DataStreamWriter when starting a query (see the previous section). + +
    +
    + +{% highlight scala %} +aggDF + .writeStream + .outputMode("complete") + .option(“checkpointLocation”, “path/to/HDFS/dir”) + .format("memory") + .start() +{% endhighlight %} + +
    +
    + +{% highlight java %} +aggDF + .writeStream() + .outputMode("complete") + .option(“checkpointLocation”, “path/to/HDFS/dir”) + .format("memory") + .start(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +aggDF\ + .writeStream()\ + .outputMode("complete")\ + .option(“checkpointLocation”, “path/to/HDFS/dir”)\ + .format("memory")\ + .start() +{% endhighlight %} + +
    +
    + + + + + + + + + + From 5bf043ab3589ee0376cf6b272052ff6a993aa5f6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 29 Jun 2016 00:26:29 -0700 Subject: [PATCH 2/3] Added links and addressed comments --- docs/img/structured-streaming.pptx | Bin 1105153 -> 1105187 bytes .../structured-streaming-programming-guide.md | 351 ++++++++++++------ 2 files changed, 236 insertions(+), 115 deletions(-) diff --git a/docs/img/structured-streaming.pptx b/docs/img/structured-streaming.pptx index cd15324863ac68f5db7b13f1b364931b437d1126..537e4e32e8aba392c5bf888c52c5b8f1c8030547 100644 GIT binary patch delta 3952 zcmYk9WmHt%9*1W@VnE;^-Q7wIB}glwgyb+Z!Vn^$(mk|-z>pq7Pyv;07#a!b?vz(T zgaIT(Kn3o2S?k_+ul0O*_HX~!UgyL4t+m4%Z|61Mu3*+Bi}DecdO6I z)~N91_5vI#(rtmQ)c+Nw7kWSae9n%VWXzv_u_UjigqYf(L;FoL@{|7qjR6ixbJ1La zXoD76tpKW5M5cEIG3w`LcajveA1goqBYeR-xoY!R874xhYE3Az{s$xCL)P zn+W8U?rMAXkL)to-N1az@>XUK-Q!rhNNQ$S0ulO4S2LU1x3CFy?OVBdN4+}YF1o@E zMb8%;o-zA~uC5o=@T;MG=?ndFs1l~y6h57#)zMhw9VSm!bc$l6+e%ZWD?>xVkc{(& zQ-feXl|fzITAv>$uUl+QSWy3CW%_q$1+(Z(KlXDEeheL0s*q4ASD3Lnd2mkSAuL)RJ=lp&z>TpgmDs6M%FSu$$DrETw;%Jju3? zy|wl8p5qI3+SL0QwYzT}hsWO^%IisvNvoWYz$Hz0M0Q2W4yzABGNY)fmZe$I(3`nB zj%K-uJrlNRVQ`Xn1#8)~>uK8sq8>+70eh&7I%f5UCo$w9u>03UhxAe@+bkb4dq_8B z7~klek=UMOB<_At&s&u$o`bRNvRvkz-G_`=ki`lfr4x(u?O_Xpnjpj-%->vO0`u8e z#}}>g<;2vO57_v}CU{CU=ZVMB%~tt^95ov}L!GTNf45}DXGO=GL9U@n>a8T7A}tHD zobnZ0QHU{rg3Io9q)}bivQr4fsL|VS%3_(o{rDth$99gWG)Dk+_o3fLw6ErYZ?M;8~nrl=4 zIp*ffQ_d=Gi`4JouBEqU404v#xkv2iTDUEqeRPpX!o=(}QX2RK=l8;UgqbC3aLyiC zl{aeh0F4J(TNUJT>Hp}~cJ^Xpc1#0$(y5?{IdgIOM2qr@8+_%cqc!KYR9vCHrM18> z$cvDni9L9eq;Rge5SsIxwe{kW#s^ZlU6fXOutdHmroDeK`%72t60DjjWB9C%@25;- zA&bWo#Q_rPXLM9gNl&o0a7^eA6Xfrjg{fZ0kA_H*L>|&{Z{e-9lDZ>zOvsJ_fAI-T zWSFadq&U5UNHRz^&E9hL4KDKq1~|~isC1DtfIBZnRI)WK(WX{9Ls4Jza*s@elxXGK z+ED9nJ8UpX>^v8u8UZL);_Q~8Z*JW@5{9Q-TeAm)^E2}6ZJUA0V+=i6tv3xrjvn4z z?W%(6_Suitc)2&%UOXAxo_ZD!VJ#5ZV*8TeZIzAJwHY_EjCG?Xolf2|f; zkoP&Dh$_TQrQLkb_(>rt!^O(UXyJT%d&lnG^o~zUGGEoT#wOcR z*9Ns0yu%ldx07XxsuVh3l5>g(e+&(DElynGt5BPrHSSM$iVgf}3@5sGrS?kEwY=dX zezUZirsk}574*zN?Bbf^nKnJ*nDtYFwdX_-3v@iPyHB0jeEl}1h*2z(E@bLRU#zeQ z&UbIg#N;RT)^2%Gz?}^zW0SOFo5G^xSvcth|0K?^(W*d}&pTV+_xN^pGtp$FCiU&s z=*H``CKI(TKV5s;TZ=386fr-M$_^!~0UVQ*)0O5aXK3W{Py6gv*tF1rHa(NM6ZQm)@$Pe4V)B{sOg1`(QX$%z#v&q+hW{ zMN1rsLLmg<|bqbF(Acy~>;w)hz-r!=LMDB8Gd((e1_cPr#zB_*d5K64{j@-m8OJ)YsV(}GI(6Dmi0%0g zCDr{&C7w^A;|0gU>qe>EE05>^#QiqZif$^K8>F))*opp@p58?4pHrv^WO{T9W z=X&vM^-8_dr%Y@0t+-a_4|5heJEeCU0)377E6A{h@tb zf>x1vS~+v{%ZS(&&fbXNcqB*A-LvrO`)>z1uLNJTtKKAY>~>j&Uyf#fgCEh6GV)>S z9M9CbKGWLfDx(4cYI$Ts@BAt3Tx=s8k{Uu&;x(fnoiuQqC7i? zCR=Glup*wZBA|E(@DSp;f`$ncQkxr&DZ4<#NdEGq&iM#7p;NSy7j zK_K?n(K0}s67AR|8}l}Y@U1FcmpaFwnMC4m&=Kan%2M5av=&5*U1}{BQPtglr_&w?|qA>Cg;3rO$AV@T67iI}ULrXz%0Q>w4PDAp;|vEOH=M{pQ zN7glcP~3O?x)1kHVe&aMWUeCOsHslLU)H7}K*+z_aZE0$p8I5?rbdf^tzOqjHYM$o zzW5b&qow$xCPmnrNQ#vF=atMx%qZwPQpM{QC2PYVx7|0{$@PYpPjG9VFqv`H%ml&a zI~8P;>QgZO(iJ}z^aa9L&g__hy=sQ+=;nYWE&l}HUh$Z$#xYl*mo<@+&a=3>#3kdj zF~d7vkzzGJMC%ojqY~#e7|5!;l-tGFG~(Dj#;%dhs?iK(Ql91^fP9jQxE#UicYAiL z>Lyo|v^0C?dR!g0ln=EvmVw{9+~@gmb)iGGhPKN^=}NJ4Xgdk%Mi;VMy$D0b^jv8L z27bsrUziMu0VY_eM)Rvo0+uuHkoTL-%@YH)oI9cOqj#(5tjSqYUQQEJ$e29O8F>{S zRzfJK(S#zTYIugE!X3M|o7okoeVKv09^FN&HC19z^30?cpF?P*l(?0A>Vvz26C^PL zSHB-qu|3XoMxVL=ZfGY-?>v{6qSH|J4=&T#{qg>}0F%h^HeX^|Mc6--XRwX9`QYkT zYn{#VH8%Vn^&A(H$IfftqZPaz__jsp<-I=aU@FSsz9QV)(+4;E5K((?_PV_vE%42< zwoyHHIj;eH2wi$kkp=e`ZU^H0FRH-z5vraYJ`KM#cCI9dUr?lolh6`TDM(s5VdDn@ zK`1&xoHqKqZV<_MpkRfZQztTapv+Q@TVvJAXc);;ldqVtvvZ@qQrGv3!y$TO^@mtr zkg4fs`nWXnAuZa-E3BpS8?=to{MV}t z92^IXpg53o0x%=kM}S-t05AA@0+1p&HocV7zjB8BucU~Yk%3l|fG`1}`DI!WYT>&3Y0AvJImY0c&B^mf=3O`3@bvf5G1xP_zz`ZH_wShL5dyl69 zX@YRuOX^z zgJfXfC;)&*^8hoH4+ItfRj37MxBv*U{IBBiHO~O~_v!`H7Vzz73(hY9O3+&%(;}b( x4Fk;=0bZ8>wj%)q!a@vz0RJ9>D1jM^0MF&00=6##bkHnt20ub9H}d!3e*o6?P3Qms delta 3942 zcmY+H2T&91*T$1jq!SQy-HC)P-#-6swANWX#xT&O=_e`S0qU2C5ZIilz@OV z=?I}lK#ExCL@q6Xf7Ea8e0OJlv*&rvIq&YivwLPXzezK+NwezgEvAo~LH1b;0Kl>t zrdp62EhqiN{lS`*iziX%^zku>!XEKnVv!$))yevhRNhQCs#Jv$sJbW?Bc;u0aOnE! zWNtd{HB5Z_Ok+bTV5PoiVB=DL+Sgk0b|KMc#YzOHA4h%?G51bbq&Hv4i1c;)d;hnC z)P&5E75{-+A>-$|etL?BKPc=cN7ZU2HB+H8Xn&v17rLcQ(l0Jz#E}==Y+5AxU{TI- zH_H53vG=aT(6xr%ot&n{S7;ze3Ib_|i|V^^T|x;-V?s?x5&v>N4!v%-#AalHqobw7 z<`_~ei7DN*URf)EJzbjzA^GCY0Rk4I)?cX+y@Mme2urq=8jK`V@s~v~q0O`Z6U9hj-cfRW2 zq+DgM(Q+(dUV^(v|g-NUZMedR;Vn)1k%S zGU%aJ)?lom;EQjcN&xoHUGsWWOMfyWOs$iDvJ<{c%Wrb&@9dxEAw#nrteUc50_bW8*d8sMw^uz0cYNH@1JdDzrHJ&`6 zOH^!K!I!V%2`iv$j-;Z0hKy9cBKLz=uz;(t#8c~BYq{s|AJ}tHOt1I@^-`ejtb^Ie zv+s5ozMEfvui{mq6`*#wn!I=#nrYNCW**k<-RESlV>I$)BPVNcB^7NJ_Gw;TL+jUl z>rM^g8c&6v{E*Mt7@RowF1`q!LGUr-R*n&kyX{!{QBkZa-H_|v;^BMUWZN9k>NQhm z;k&+=TOJY(j=o%@qHAUtUqSZM%`U#Yk$=sI(j_RxdhxdewbAg5Qd6Mm?nWG{Jn1})-`8NDEB zLmjOb{!uC^=Bs%$)_>|{H!@0ro~$~8xQ6<52YFOQY;q)`q%zOM3$wi5bont3cPGyB zAI;bs8tsa(&_8YmFWx3(8QPqMH}0y0mAFZ(-{{k%0YPM}GX2mp>fzqN?O-3ddLA8i zsnuG2GIE)GEG)r(7EPl=A5X^fLK3%CmZ3r2UoLILaGZ-OvKNtGwaK_}GB|YHE);~J1_4%*W9oBC8Yy^|4Fw`Y6=2vWZv7wIXpbj>{Qbl@Kn6# zk!sl;+)!8ExS_5Yc0(URVFO-bybnP~wKrinMTV9V(hS?kY`_Fon z@SKx+YXh-r#<(Uhn=8cq)}6$ByX4+)eeG_B5 zJBU~_jnY1fZx^MkB=B|!ZE4AS3R9{6y?!BbhKXm9H{Qfk%%{aLlsmcfPW*^SLw=?2y&B)2Hggeag@)4jL2T6nLRbm`^Ndo>gH&k6|I3 z5o8(S`X#hg8P0Acs)nK@R?n5~tlw*~-dgQaA-?e8n+>up!wU-LO|Es3s*x3|sH{jX zPSfz+L=$w|p=I~saWFuBL@`bd@Z9CmYU0Gp{>A5pa7mOvgYeU^jL7@8IIvNwP2l&Z ziD=Z;N5k;MeE=e(LCsYcpwY+7$SIW-*~X@NOfR3dWNNeAThx(k5pJ1nH9ivIg1~ML zQwZ+Wn1k|FM9S7@CDBI7Z%jqZ{Ot_3Toyw5natT1Ln5Y^1-{Z-n4r6~UlGs+ortGHU|9Gi$CrURKub;`PI-0B6W@;)tN>!n$N44U0%xVObno0J$PX?ha&a5AzZ ztMuZcnM`O@bg9Uq>)Z_mj>(mf;~rhBi2YE5U1m9bIV+DxbvGG`o1KNEAAVqHX?bcu zX)B^@R7Ty-3YuT|@_0uV2_MddMxe_h@dCr=WBp0rE1hLOu75R49oW%?()amVGTYl- zQ#-&JA$U7@m91VCrl(q0xVvxI^cUG47(GMSS#MuHnCSC3(Ots80TetQlUi))m@YGzfoX`oeeNd(vt^+hE^#+X;x3R1 zhzboAS}Jr@=&78d!a#+Q3KJD(DlAl3sjyLDZ8{2Q+7!Z~_1(jq?>?SvGWz zb+Cu({hS-IR|+^owjBUr8GLULSLPdA4JkSTGh}<-)C!CT(LN%VEw`?MJ%emTcX^d~ zdA7^H3X4py@jo(!W@|X4SSF}XUXDKy=PFHlZn+}wGg}4S0$iS* zz>O8`$4loXk&NrK{5m_&(3#GAxzBVqd}>p377>5Q_^VG8fQ$VPG>tngU?el0d9r}0 zXrpN_SPO{me*OR}b+oIhr_Uf&(tt0#y6v>GE$@$k_)=;V9Zsp=cNWoW#a^T>^VTO0 z!Vu22aitLvqGg=jh*Vf0L$Bh=VWd=xM7u6)Xl?{6=5=&6)cmXi(2^qn-T?2-nvadn zn>WFJLw+d73|m&cWE=I?TDTB)Cs(@fuB;1J4<5bWi0kVuY28X(Hy|uYdYPsgdWPC2 zyq61Lna(GPZ`r9h)P=6SkxWPt!U+fE75KG(eJL41P--$sEI48e`5}9+u3y{WJc`*z zp;UGG1@d|#f17;3n}L)MaS1$*9Q)_b34Y_PeN}usAOi{3Uc<-jW8`N5N|XAb!R7Z3 zVNU3{F2eU3Y3O-NJv&tF#FkYmKy>!v6jV#`pMnwoH2mec(D)%tE_&gME~7suP+U`_ z*XW7V1H-e((}byH!2oqHCHfk+TwZsK;@V?8oV|4KL0eR(jzmW#P(5$QfN8y-oT=CG3 z=s;#)oGtdN8%;BoETsm%#fTS2u0{8TFPeGM*B&m@T5~ z;8L4teqMw2Ed{~^aok4XPdWzq6+r%XSoZN>|y$ElOj4H34XWyo_3d8 z&j^P%HuA|{d$(0{&`MhWwOP~KP!YxoFO28M-0!>zK&G4h7*B0JQ@a&P%N8lE?Whpn z%LYwbZd6Wt@K%AFdqM$a8p_=ucqufX*@ z%xyGcMG+hFm42AEqO~+uWGX3lGz;(*utYV93%ZeKmYK)Gta%IM;P8hb_Mz|B;@Px% z^qD~kgUq3B)jN)~<_iHUHmTkT79SQqYYdrPKd3umJ)vuYF!PPKcw|@cLx!gFN+~?K zEaYp9L-6{cU2~$G{4p z?-qYzJ_1+)xPdhxfDxcyh(Fda)dC$N|HNYB)EHp(JH{AWF<}$N!Qvo#YpRz1vo1!= z2FwEZ_h7NXDBJwa{JtU!)_MZW26|xoryp<2jLn>&_IsiJti3h?Rs;puVcRFc{MhT0 zU>RTt_R%ER2q=neqn<#E?tfF#Q`D3()@F+8W%>X02B;qBW5{2@H%%3=u)o6fw=jwP zE8b0mWkI$ve+AdMY;ZKJdY<9meg6Y7_e+fc diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 10690013e3fe..6636d16a8781 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -8,23 +8,16 @@ title: Structured Streaming Programming Guide {:toc} # Overview -Structured Streaming is a scalable and fault-tolerant stream processing engine -built on the Spark SQL engine. You can express your streaming computation by -thinking you are running a batch computation on a static dataset, and the -Spark SQL engine takes care of running it incrementally and continuously -updating the final result as streaming data keeps arriving. You can use the -[Dataset/DataFrame API](sql-programming-guide.html) in Scala, Java or Python to express streaming -aggregations, event-time windows, stream-to-batch joins, etc. The computation -is executed on the same optimized Spark SQL engine. Finally, the system -ensures end-to-end exactly-once fault-tolerance guarantees through -checkpointing and Write Ahead Logs. In short, *Stuctured Streaming provides -fast, scalable, fault-tolerant, end-to-end exactly-once stream processing -without the user having to reason about streaming.* - -**Spark 2.0 is the ALPHA RELEASE of Structured Streaming** and the APIs are still experimental. In this guide, we are going to walk you through the programming model and the APIs. First, lets start with a simple example - a streaming word count. +Structured Streaming is a scalable and fault-tolerant stream processing engine built on the Spark SQL engine. You can express your streaming computation the same way you would express a batch computation on static data.The Spark SQL engine will take care of running it incrementally and continuously and updating the final result as streaming data continues to arrive. You can use the [Dataset/DataFrame API](sql-programming-guide.html) in Scala, Java or Python to express streaming aggregations, event-time windows, stream-to-batch joins, etc. The computation is executed on the same optimized Spark SQL engine. Finally, the system ensures end-to-end exactly-once fault-tolerance guarantees through checkpointing and Write Ahead Logs. In short, *Structured Streaming provides fast, scalable, fault-tolerant, end-to-end exactly-once stream processing without the user having to reason about streaming.* + +**Spark 2.0 is the ALPHA RELEASE of Structured Streaming** and the APIs are still experimental. In this guide, we are going to walk you through the programming model and the APIs. First, let's start with a simple example - a streaming word count. # Quick Example -Let’s say you want maintain a running word count of text data received from a data server listening on a TCP socket. Let’s see how you can express this using Structured Streaming. You can see the full code in Scala/Java/Python. And if you download Spark, you can directly run the example. In any case, let’s walk through the example step-by-step and understand how it is works. First, we have to import the names of the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark. +Let’s say you want to maintain a running word count of text data received from a data server listening on a TCP socket. Let’s see how you can express this using Structured Streaming. You can see the full code in +[Scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/ +[Java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/ +[Python]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/sql/streaming/structured_network_wordcount.py). And if you +[download Spark](http://spark.apache.org/downloads.html), you can directly run the example. In any case, let’s walk through the example step-by-step and understand how it works. First, we have to import the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark.
    @@ -43,15 +36,21 @@ val spark = SparkSession
    {% highlight java %} +import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.sql.*; import org.apache.spark.sql.streaming.StreamingQuery; +import java.util.Arrays; +import java.util.Iterator; + SparkSession spark = SparkSession .builder() .appName("JavaStructuredNetworkWordCount") .getOrCreate(); {% endhighlight %} +import spark.implicits._ +
    @@ -75,45 +74,49 @@ Next, let’s create a streaming DataFrame that represents text data received fr
    {% highlight scala %} +// Create DataFrame representing the stream of input lines from connection to localhost:9999 val lines = spark.readStream .format("socket") .option("host", "localhost") .option("port", 9999) - .load() + .load().as[String] -val words = lines.select( - explode( - split(lines.col("value"), " ") - ).alias("word") -) +// Split the lines into words +val words = lines.flatMap(_.split(" ")) -val wordCounts = words.groupBy("word").count() +// Generate running word count +val wordCounts = words.groupBy("value").count() {% endhighlight %}
    {% highlight java %} -Dataset lines = spark - .readStream() - .format("socket") - .option("host", "localhost") - .option("port", 9999) - .load(); - -Dataset words = lines.select( - functions.explode( - functions.split(lines.col("value"), " ") - ).alias("word") -); - -Dataset wordCounts = words.groupBy("word").count(); +// Create DataFrame representing the stream of input lines from connection to localhost:9999 +Dataset lines = spark + .readStream() + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load().as(Encoders.STRING()); + +// Split the lines into words +Dataset words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterator call(String x) { + return Arrays.asList(x.split(" ")).iterator(); + } +}, Encoders.STRING()); + +// Generate running word count +Dataset wordCounts = words.groupBy("value").count(); {% endhighlight %}
    {% highlight python %} +# Create DataFrame representing the stream of input lines from connection to localhost:9999 lines = spark\ .readStream\ .format('socket')\ @@ -121,43 +124,32 @@ lines = spark\ .option('port', 9999)\ .load() +# Split the lines into words words = lines.select( explode( split(lines.value, ' ') ).alias('word') ) +# Generate running word count wordCounts = words.groupBy('word').count() {% endhighlight %}
    -This `lines` DataFrame is like an unbounded table containing the streaming -text data. This table contains one column of string named “value”, and each -line in the streaming text data is like a row in this table. Note, that this -is not currently receiving any data as we are just setting up the -transformation, and have not yet started it. Next, we have used to built-in -SQL functions - split and explode, to split each line into multiple rows with -a word each. In addition, we use the function `alias` to name the new column -as “word”. Finally, we have defined the running counts, by grouping the `words` -DataFrame by the column `word` and count on that grouping. +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used to built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function `alias` to name the new column as “word”. Finally, we have defined the running counts, by grouping the `words` DataFrame by the column `word` and count on that grouping. -We have now set up the query on the streaming data. All that is left is to -actually start receiving data and computing the counts. To do this, we set it -up to output the counts to the console every time they are updated. In -addition we are also going to set up additional details like checkpoint -location. Don’t worry about them for now, they are explained later in the guide. +We have now set up the query on the streaming data. All that is left is to actually start receiving data and computing the counts. To do this, we set it up to print the complete set of counts (specified by `outputMode(“complete”)`) to the console every time they are updated. And then start the streaming computation using `start()`.
    {% highlight java %} -val query = wordCounts - .writeStream +// Start running the query that prints the running counts to the console +val query = wordCounts.writeStream .outputMode("complete") .format("console") - .option("checkpointLocation", checkpointDir) .start() query.awaitTermination() @@ -167,12 +159,11 @@ query.awaitTermination()
    {% highlight java %} -StreamingQuery query = wordCounts - .writeStream() - .outputMode("complete") - .format("console") - .option("checkpointLocation", checkpointDir) - .start(); +// Start running the query that prints the running counts to the console +StreamingQuery query = wordCounts.writeStream() + .outputMode("complete") + .format("console") + .start(); query.awaitTermination(); {% endhighlight %} @@ -181,11 +172,11 @@ query.awaitTermination();
    {% highlight python %} + # Start running the query that prints the running counts to the console query = wordCounts\ .writeStream\ .outputMode('complete')\ .format('console')\ - .option('checkpointLocation', checkpointDir)\ .start() query.awaitTermination() @@ -194,8 +185,11 @@ query.awaitTermination()
    -Now the streaming computation has started in the background, and the `query` object is a handle to that active streaming query. Note that we are also waiting for the query to terminate, to prevent the process from finishing while the query is active. -To actually run this code, you can either compile your own Spark application, or simply run the example once you have downloaded Spark. We are showing the latter. You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using +After this code is executed, the streaming computation will have started in the background. The `query` object is a handle to that active streaming query, and we have decided to wait for the termination of the query using `query.awaitTermination()` to prevent the process from exiting while the query is active. + +To actually execute this example code, you can either compile the code in your own +[Spark application](quick-start.html#self-contained-applications), or simply +[run the example](index.html#running-the-examples-and-shell) once you have downloaded Spark. We are showing the latter. You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using $ nc -lk 9999 @@ -223,7 +217,7 @@ Then, any lines typed in the terminal running the netcat server will be counted # Programming Model -The key idea is in Structured Streaming is to treat a live data stream as a +The key idea in Structured Streaming is to treat a live data stream as a table that is being continuously appended. This leads to a new stream processing model that is very similar to a batch processing model. You will express your streaming computation as standard batch-like query as on a static @@ -236,7 +230,7 @@ arriving on the stream is like a new row being appended to the Input Table. ![Stream as a Table](img/structured-streaming-stream-as-a-table.png "Stream as a Table") -A query on the input will generate the “Result Table”. Every trigger interval (say, every 1 second), new rows gets appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want write the changed result rows to an external sink. +A query on the input will generate the “Result Table”. Every trigger interval (say, every 1 second), new rows get appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want to write the changed result rows to an external sink. ![Model](img/structured-streaming-model.png) @@ -244,11 +238,11 @@ The “Output” is defined as what gets written out to the external storage. Th - *Complete Mode* - The entire updated Result Table will be written to the external storage. - - *Append Mode* - Only the new rows appended in the Result Table since the last trigger will be written to the external storage. This is applicable only on queries where existing rows in the Result Table is not expected to change. + - *Append Mode* - Only the new rows appended in the Result Table since the last trigger will be written to the external storage. This is applicable only on the queries where existing rows in the Result Table are not expected to change. - - *Update Mode* - Only the rows that were updated in the Result Table since the last trigger wil be written to the external storage (not available yet in Spark 2.0). Note that this is different from the Complete Mode in that this mode does not output the rows that are not changed. + - *Update Mode* - Only the rows that were updated in the Result Table since the last trigger will be written to the external storage (not available yet in Spark 2.0). Note that this is different from the Complete Mode in that this mode does not output the rows that are not changed. -Note that each mode is applicable on certain types of queries. This is discussed in detail later. +Note that each mode is applicable on certain types of queries. This is discussed in detail [later](#output-modes). To illustrate the use of this model, let’s understand the model in context of the Quick Example above. The first `lines` DataFrame is the input table, and @@ -262,29 +256,36 @@ running counts with the new data to compute updated counts, as shown below. ![Model](img/structured-streaming-example-model.png) This model is significantly different from many other stream processing -engines. Many streaming system require the user to maintain running -aggregations themselves, thus having the reason about fault-tolerance, and +engines. Many streaming systems require the user to maintain running +aggregations themselves, thus having to reason about fault-tolerance, and data consistency (at-least-once, or at-most-once, or exactly-once). In this model, Spark is responsible for updating the Result Table when there is new data, thus relieving the users from reasoning about it. As an example, let’s see how this model handles event-time based processing and late arriving data. ## Handling Event-time and Late Data -Event-time is the time embedded in the data itself. For many applications, you may want to do operate using this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time a column value in the row. This allows window-based aggregations (e.g. number of event every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on streaming dataset, making the life of the user much easier. +Event-time is the time embedded in the data itself. For many applications, you may want to operate on this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time is a column value in the row. This allows window-based aggregations (e.g. number of event every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on streaming dataset, making the life of the user much easier. -Furthermore this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0 yet, event-time watermarking will be used to manage this data. These are explained in more details in the Window Operations section. +Furthermore this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0 yet, event-time watermarking will be used to manage this data. These are explained later in more details in the [Window Operations](#window-operations-on-event-time) section. ## Fault Tolerance Semantics Delivering end-to-end exactly-once semantics was one of key goals behind the design of Structured Streaming. To achieve that, we have designed the Structured Streaming sources, the sinks and the execution engine to reliably track the exact progress of the processing so that it can handle any kind of failure by restarting and/or reprocessing. Every streaming source is assumed to have offsets (similar to Kafka offsets, or Kinesis sequence numbers) -for track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. +to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. # API using Datasets and DataFrames -Since Spark 2.0, DataFrames and can represent of static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point SparkSession to create straming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the Spark SQL Programming Guide. +Since Spark 2.0, DataFrames and can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` ( +[Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/ +[Java](api/java/org/apache/spark/sql/SparkSession.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.SparkSession) docs) to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the +[DataFrame/Dataset Programming Guide](sql-programming-guide.html). ## Creating streaming DataFrames and streaming Datasets -Streaming DataFrames can created through the DataStreamReader interface `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source - data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. +Streaming DataFrames can be created through the `DataStreamReader` interface +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/ +[Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source - data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. - - **File sources** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for more up-to-date list, and supported options for each file format. Note that the files needs to be atomically put in the directory. In most file systems, this means that the files should be moved atomically into the directory. + - **File sources** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by move operations. - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The connection is only on the server. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. @@ -374,7 +375,7 @@ csvDF = spark \
    -These examples generate streaming DataFrames that are untyped, that is, the schema of the DataFrame is not checked at compile time, only checked in runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. +These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked in runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. ## Operations on streaming DataFrames/Datasets You can apply all kinds of streaming DataFrames/Datasets - ranging for untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the SQL programming guide for more details. Let’s take a look at a few example operations that you can use. @@ -388,7 +389,7 @@ Most of the common operations on DataFrame/Dataset operations are supported for {% highlight scala %} case class DeviceData(device: String, type: String, signal: Double, time: DateTime) -val df: DataFrame = … // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: string } +val df: DataFrame = ... // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: string } val ds: Dataset[DeviceData] = df.as[DeviceData] // streaming Dataset with IOT device data // Select the devices which have signal more than 10 @@ -406,40 +407,128 @@ ds.groupByKey(_.type).agg(typed.avg(_.signal)) // using typed API
    +{% highlight java %} +import org.apache.spark.api.java.function.*; +import org.apache.spark.sql.*; +import org.apache.spark.sql.expressions.javalang.typed; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; + +public class DeviceData { + private String device; + private String type; + private Double signal; + private java.sql.Date time; + ... + // Getter and setter methods for each field +} + +Dataset df = ...; // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType } +Dataset ds = df.as(ExpressionEncoder.javaBean(DeviceData.class)); // streaming Dataset with IOT device data + +// Select the devices which have signal more than 10 +df.select("device").where("signal > 10"); // using untyped APIs +ds.filter(new FilterFunction() { // using typed APIs + @Override + public boolean call(DeviceData value) throws Exception { + return value.getSignal() > 10; + } +}).map(new MapFunction() { + @Override + public String call(DeviceData value) throws Exception { + return value.getDevice(); + } +}, Encoders.STRING()); + +// Running count of the number of updates for each device type +df.groupBy("type").count(); // using untyped API + +// Running average signal for each device type +ds.groupByKey(new MapFunction() { // using typed API + @Override + public String call(DeviceData value) throws Exception { + return value.getType(); + } +}, Encoders.STRING()).agg(typed.avg(new MapFunction() { + @Override + public Double call(DeviceData value) throws Exception { + return value.getSignal(); + } +})); +{% endhighlight %} + +
    +{% highlight python %} + +df = ... # streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType } + +# Select the devices which have signal more than 11 +df.select("device").where("signal > 10") + +# Running count of the number of updates for each device type +df.groupBy("type").count() +{% endhighlight %}
    ### Window Operations on Event Time -Aggregations over a sliding window is very easy. Applying a sliding window is simply a special type of grouping - each time window is a group and each row can belong to multiple windows/groups. Hence aggregating over sliding windows is same as aggregating over this special type of grouping. This is illustrated below. Imagine the quick example is modified and the stream contains lines along with the time when the line was generated. Lets say instead of running word counts, we want to count windows within 10 minute windows, updating every 5 minutes. The result tables would look like the following. Note the window notation 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. The counts are indexed by the combined group (window, cat) +Aggregations over a sliding event-time window are straightforward with Structured Streaming. The key idea to understand about window-based aggregations are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of, window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration. + +Imagine the quick example is modified and the stream contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). + +The result tables would look something like the following. ![Window Operations](img/structured-streaming-window.png) -In code, you can use windows as the following. +Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window` operations to express windows.
    {% highlight scala %} // Number of events in every 1 minute time windows -df.groupBy(window(time, "1 minute")) +df.groupBy(window(df.col("time"), "1 minute")) .count() // Average number of events for each device type in every 1 minute time windows df.groupBy( - "type", - window(time, "1 minute")) - .avg(df.col("signal")) + df.col("type"), + window(df.col("time"), "1 minute")) + .avg("signal") {% endhighlight %}
    +{% highlight java %} +import static org.apache.spark.sql.functions.window; + +// Number of events in every 1 minute time windows +df.groupBy(window(df.col("time"), "1 minute")) + .count(); + +// Average number of events for each device type in every 1 minute time windows +df.groupBy( + df.col("type"), + window(df.col("time"), "1 minute")) + .avg("signal"); + +{% endhighlight %} +
    +{% highlight python %} +from pyspark.sql.functions import window + +# Number of events in every 1 minute time windows +df.groupBy(window("time", "1 minute")).count() + +# Average number of events for each device type in every 1 minute time windows +df.groupBy("type", window("time", "1 minute")).avg("signal") +{% endhighlight %}
    @@ -455,10 +544,10 @@ Streaming DataFrames can be joined with static DataFrames to create new streamin
    {% highlight scala %} -val staticDf = spark.read. … -val streamingDf = spark.readStream. … +val staticDf = spark.read. ... +val streamingDf = spark.readStream. ... -streamingDf.join(staticDf, “type”) // inner equi-join with a static DF +streamingDf.join(staticDf, “type”) // inner equi-join with a static DF streamingDf.join(staticDf, “type”, “right_join”) // right outer join with a static DF {% endhighlight %} @@ -466,9 +555,24 @@ streamingDf.join(staticDf, “type”, “right_join”) // right outer join wi
    +{% highlight java %} +Dataset staticDf = spark.read. ...; +Dataset streamingDf = spark.readStream. ...; +streamingDf.join(staticDf, "type"); // inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join"); // right outer join with a static DF +{% endhighlight %} + +
    +{% highlight python %} +staticDf = spark.read. … +streamingDf = spark.readStream. … +streamingDf.join(staticDf, "type") # inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join") # right outer join with a static DF +{% endhighlight %} +
    @@ -502,7 +606,11 @@ In addition, there are some Dataset methods that will not work on streaming Data If you try any of these operations, you will see an AnalysisException like "operation XYZ is not supported with streaming DataFrames/Datasets". ## Starting Streaming Queries -Once you have defined the final result DataFrame/Dataset, all that is left is for you start the StreamingQuery. To do that, you have to use the `DataStreamWriter`, through you have to specify the following. +Once you have defined the final result DataFrame/Dataset, all that is left is for you start the StreamingQuery. To do that, you have to use the +`DataStreamWriter` ( +[Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/ +[Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) returned through `Dataset.writeSteram()`. You will have to specify one or more of the following in this interface. - *Details of the output sink:* Data format, location, etc. @@ -510,7 +618,7 @@ Once you have defined the final result DataFrame/Dataset, all that is left is fo - *Query name:* Optionally, specify a unique name of the query for identification. -- *Trigger interval:* Optionally, specify the trigger interval. If its not specified, the system will check for availability of new day as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will attempt to trigger at the next trigger point, not immediately after the processing has completed. +- *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will attempt to trigger at the next trigger point, not immediately after the processing has completed. - *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in a HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. @@ -554,7 +662,7 @@ Here is a table of all the sinks, and the corresponding settings. All modes
    writeStream
    .foreach(...)
    .start()
    Depends on ForeachWriter implementation - More details in the next section + More details in the
    next section Console Sink @@ -620,7 +728,7 @@ spark.sql("select * from aggregates).show() // interactively query in-memory t {% highlight java %} // ========== DF with no aggregations ========== -Dataset[Row] noAggDF = deviceDataDf.select("device").where("signal > 10") +Dataset noAggDF = deviceDataDf.select("device").where("signal > 10") // Print new data to console noAggDF @@ -635,7 +743,7 @@ noAggDF .start(); // ========== DF with aggregation ========== -Dataset[Row] aggDF = df.groupBy(“device”).count(); +Dataset aggDF = df.groupBy(“device”).count(); // Print updated aggregations to console aggDF @@ -652,7 +760,7 @@ aggDF .format("memory") .start(); -spark.sql("select * from aggregates).show() // interactively query in-memory table +spark.sql("select * from aggregates).show(); // interactively query in-memory table {% endhighlight %} @@ -660,29 +768,29 @@ spark.sql("select * from aggregates).show() // interactively query in-memory t {% highlight python %} # ========== DF with no aggregations ========== -Dataset[Row] noAggDF = deviceDataDf.select("device").where("signal > 10") +noAggDF = deviceDataDf.select("device").where("signal > 10") # Print new data to console noAggDF\ .writeStream()\ .format("console")\ - .start(); + .start() # Write new data to Parquet files noAggDF\ .writeStream()\ .parquet("path/to/destination/directory")\ - .start(); + .start() # ========== DF with aggregation ========== -Dataset[Row] aggDF = df.groupBy(“device”).count(); +aggDF = df.groupBy(“device”).count() # Print updated aggregations to console aggDF\ .writeStream()\ .outputMode("complete")\ .format("console")\ - .start(); + .start() # Have all the aggregates in an in memory table. The query name will be the table name aggDF\ @@ -690,7 +798,7 @@ aggDF\ .queryName("aggregates")\ .outputMode("complete")\ .format("memory")\ - .start(); + .start() spark.sql("select * from aggregates).show() # interactively query in-memory table {% endhighlight %} @@ -698,8 +806,9 @@ spark.sql("select * from aggregates).show() # interactively query in-memory ta -Using Foreach -The `foreach` operation allows arbitrary operations to be computer on the output data. As of Spark 2.0, this is available only for Scala and Java, not Python. To use this, you will have to implement the interface `ForeachWriter`, which has methods that gets called whenever there is a sequence of rows generated as output after a trigger. It is important note the following +### Using Foreach +The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.0, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/ +[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), which has methods that gets called whenever there is a sequence of rows generated as output after a trigger. It is important note the following - The writer must be serializable, as it will be serialized and sent to the executors for execution. @@ -709,9 +818,9 @@ The `foreach` operation allows arbitrary operations to be computer on the output - `version` and `partition` are two parameter in the `open` that uniquely represents a set of rows that needs to be pushed out. `version` is monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. -- `open` can use the `version` and `partition` to choose whether it need to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If the `false` is returned, then `write` will not be called on any row. For example, after a partial failure, so partitions of the failed trigger may have already been committed to a database. Based on metadata stores in the database, the writer can identify partitions that have already been committed and +- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If the `false` is returned, then `write` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. -- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is the your responsibilty to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. +- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is the your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. ## Managing Streaming Queries The `StreamingQuery` object created when a query is started can be used to monitor and manage the query. @@ -746,21 +855,22 @@ query.sinkStatus() // progress information about data written to the output si {% highlight java %} StreamingQuery query = df.writeStream().format("console").start(); // get the query object -query.id() // get the unique identifier of the running query +query.id(); // get the unique identifier of the running query -query.name() // get the name of the auto-generated or user-specified name +query.name(); // get the name of the auto-generated or user-specified name -query.explain() // print detailed explanations of the query +query.explain(); // print detailed explanations of the query -query.stop() // stop the query +query.stop(); // stop the query -query.awaitTermination() // block until query is terminated, with stop() or with error +query.awaitTermination(); // block until query is terminated, with stop() or with error -query.exception() // the exception if the query has been terminated with error +query.exception(); // the exception if the query has been terminated with error -query.souceStatus() // progress information about data has been read from the input sources +query.souceStatus(); // progress information about data has been read from the input sources + +query.sinkStatus(); // progress information about data written to the output sink -query.sinkStatus() // progress information about data written to the output sink {% endhighlight %} @@ -784,12 +894,16 @@ query.exception() # the exception if the query has been terminated with error query.souceStatus() # progress information about data has been read from the input sources query.sinkStatus() # progress information about data written to the output sink + {% endhighlight %} -You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the StreamingQueryManager that can be used to manage the currently active queries. +You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the `StreamingQueryManager` ( +[Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryManager)/ +[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryManager.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.StreamingQueryManager) docs) that can be used to manage the currently active queries.
    @@ -808,7 +922,7 @@ spark.streams.awaitAnyTermination() // block until any one of them terminates
    {% highlight java %} -SparkSession spark = … +SparkSession spark = ... spark.streams().active() // get the list of currently active streaming queries @@ -821,9 +935,9 @@ spark.streams().awaitAnyTermination() // block until any one of them terminate
    {% highlight python %} -spark = … # spark session +spark = ... # spark session -spark.streams().active() # get the list of currently active streaming queries +spark.streams().active # get the list of currently active streaming queries spark.streams().get(id) # get a query object by its unique id @@ -833,7 +947,9 @@ spark.streams().awaitAnyTermination() # block until any one of them terminates
    -Finally, for asynchronous monitoring of streaming queries, you can create and attach a StreamingQueryListener, which will give you regular callback-based updates when queries are started and terminated. +Finally, for asynchronous monitoring of streaming queries, you can create and attach a `StreamingQueryListener` ( +[Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryListener)/ +[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html) docs), which will give you regular callback-based updates when queries are started and terminated. ## Recovering from Failures with Checkpointing In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger), and the running aggregates (e.g. word counts in the quick example) will be saved the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in a HDFS compatible file system, and can be set as an option in the DataStreamWriter when starting a query (see the previous section). @@ -877,6 +993,11 @@ aggDF\
    +# Where to go from here +- Examples: See and run the +[Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/sql/streaming) +examples. +- Spark Summit 2016 Talk - A Deep Dive into Structured Streaming [link](https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/) From 78223a4cdb55e035536ca4bf810cd49a7fdd4f49 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 29 Jun 2016 01:39:16 -0700 Subject: [PATCH 3/3] Addressed more comments --- docs/img/structured-streaming-late-data.png | Bin 124617 -> 138931 bytes docs/img/structured-streaming-window.png | Bin 125098 -> 128930 bytes docs/img/structured-streaming.pptx | Bin 1105187 -> 1105315 bytes .../structured-streaming-programming-guide.md | 313 +++++++++++++----- 4 files changed, 230 insertions(+), 83 deletions(-) diff --git a/docs/img/structured-streaming-late-data.png b/docs/img/structured-streaming-late-data.png index 623628b1019d80742017fba3363a44680ba7e059..5276b4786826236eb9ce043c9373870350aea7ff 100644 GIT binary patch literal 138931 zcmeFZcR1T`-#^@{R#B81t=XkE9YPy|+Iz*GEw%TE5kyr>QPi$Y39*veTdJzs*du}% zMa_u4WBh!t>wDezb6@TK`|mlP<9Lqi4{`VqpYuH5>-BoQ-=9QiYbw!Du~MBocaBCy zSzh9 zkvXR#|5VTK{K^DHAfv)*$A%>XREh8QZ3TyqZzD&N&hh)SqiF*ChdL&inBFz-VXWOo z0+jBS=e;D0R2CsqrEtjQK3#M6oc0`T9G_d&70&j5GC8rjveFf>7t`6*{qDy3^W;}$ z&i&`*KWFeCKlo1={3jLsClmfB6aJkE-|FAQujIwqUa$rU>koF$uCP~%gR5cX!%?I= z1`h`AXg?v}7c61N$JdI4F=&R{nNS}$cGU5f6uicdh|JM@cpFH zf84ema%8CwuQCdQ)0h4Ccb+FNe*KxnR%qyPW%s2@G)W-Y zz<#g9O@oiw>$;;sWvFoq#MX;W+?3ht5}bX#z*zCoT((oqx^|_)(;woOKq5O1{{CN~ zSp!s@osj+=0Ut^mLqi{qDvt@A?W2mWO;riY^%ElkgPL(@FSx(gy38)8DLV~|w619m zG_*_WrgDzmdLV>%mM}*gwKUjb)Jn_W-W|oy3tY4t z4#sw0$25cN(}SK3e<~j(jSXv)ux?gRhrw5ayxA1cbPz{VH|P{osugCOoVBZ0TC!*o zX2&OW96f4o{6K8AaW=F~x$=T&5Jm&z=9`oof-hXCWSTNBB1U&1`*x`VO04wYfG z&CwJ5Uv)0-H3{Olc@|H;i{VudCX4*y-#M~mSx0kbr#bsf_}earZ_$+x!v$@2q$^W< zn%}HWtqqN?8*xpbrU&LeOrS_Vwc(9=nG9`r@&n|Sm~GO`8Po0!LoacOSo-!(@!*fQ z5%{4-@f5<2a?%ns$-JxlO>B4C@;$Vc5=Y_U#2VDBXtQ_k$C5C1slAVA6Qorvg0b!ZLB+mv`;MTBeNu3}5&WatOdt^l&F-VHQ&v+G#k)mJgg_daP866Uib~mF`d` zLSB$bb&~mS7&TvPqDx=akRkpS2IoRN>D|%fXcZbb7<+$IX@eVJ#)<3@i%gAlE85ts zVP8{PNbk(kq{gkE)XZ!}2J?5Twky5LV|X_j_!`$1pH=2nR5_A=3edMJ+&NGPt$o;lqpv7 z)eptmXh4EblFi*wmuUHRHmECi(`J|H=K9tAPo}6Kfyf6=9|I;K&==8^(kcE7HkCif#3qBu>5jNeo%Qpc@AS!D+1_F6ENTbtOz&Ol!UT?4V~)TV22> zhWyCxeo^bEjgcVi$>!!;f@C`taqBh@hVapQ^<-biqkbkr$~d*gj?a8DW%)H_r8KgE zcP%#(J}C!Vpmhe{aMR}`Y+t`VM?W$LDqd-{KXq*HWGW0`3pyzzZSfL&PgMuK-mDr- z+8_&Kj=?&oZ-TOk*Pt!B7LBy8GuSIBm+$HZD{bWTd^jAF*4XHu2@pdMwA4)|Jh`~n zb9oLtAJCC>kA9B#C?gHo@hJwVm5Rz1u+vQ3b0Awzc;EZb(3b4q>&icq?_KSig^9E& z1Dfz}ZI^`U+8d7t97)>kZ=>fh)&lSY!&qbHTZr;)X1FoZA-Lx4`_ivSnG+z)C?fL~ zTjK6y#$^uSTR+sAhBiC`62A}G$vG@+Jjov^$lygrNysdXSDPbfQO)B{=$s1CPQFD! zwwYRVl?xoJ;7_Oj&W}#PBLO1Z)apF$+U?f#MGcs%W zssmEe-0o*8e2$(7n;?+tBq0+I+c@$z*@**-9wmLEuIan&J6UIIn39O#58oU`EGkkb zJz3nZo5F7l^SC!5-3FvPF<06{^fhlbW0F3Q-T5CC_8%v~$DcDiu3tPB8`(lL)URhM zXXo03tI~3!9;ToSP_}5@R*J6TRGRt}5$MCaX39udmXr(hVZrB%0yReEZcBcA65yX% z_(?E!SD@}&ivrk~xzo2`Wo-W*uZ^|;(413cw*EVXND2~@(wwxB$NtlcgA3ir8&VTb zRSTu!N95|D!&t^Zj8}qH)QH!Pv@8vxkslpR=!UIp<#Il(bQ_m~dx2e;s2h?apjzJ` zYj;1f4|;E8^BMfOZuz42`RCr0$#6twwRdmHxfbg$!LYD&?<^ z$unt&tSp2>T|Q5H>ExX<03Hi#3qHsM@8>lgw3(f7m9%=B%LGyL9UK5fbu6IhXw(6O$hJu7lowYGWlpmaa$ zgGCa;u0YX+j{@i5Z(L)X^1+ij!N_LZ#KxR;2?VDS1~-@iYJApz6x7zVA7|;E9HNG< zE=?|#<;@C*Z~77aAdX2U@dMWq>tJv$T|ou6W`3`;jgD{39T*o3((9V%C>9&OKfLw3 zJ(KirHa_boTHLUm(EG-LOu-vc4mrvS!w3bhhLl7?8FCfxLSeI zdVMEBvUNdycnhPQLbI&gulJmdNbPWZl(rijNLCc-Iv(&tMDm}Xt7Xy2j4)9~>2ih_AoRF#D5|Wf91f;W+ zy)L0h((2pYaq9kdX-#W4;NiYmT$Z8y6AbkDHulaZk;%imvz3C{Vw@$lFQuq2|ArmU zTwv~2j^fXGM{*zV1DC>YQ-**v>$MhJGQ-R=|@=F|Lj_`6n%lc+d#;`~`@sAPxjl zMVSLpZS8=sgp-v#Ke?Oci`Gp$dP+TL%+|Nqu2uVIJ{eh-1$z@t&?e(Ib?R%4y|TSG zRw2?FXz_s+;fYgU$Ge@rP|&T%WxK>&T&RAwskcRKjfRf$Za~GNJY14*>!(v1(Ba zNidR7soBA^U0GVSS%E&hd}i<lNM)wIRf*b z7gL4Tu8Cr9>MZnsdYO7?Q*Wwlh4pwg~wjDtpKaYDC`5T>!?X&;MaU|L6>64TeQ|b%DK| zTJK7-zu2wa^q~%|jTYb9Tz9t-L~VT~(06Ynl?g9zzf^RR!f3I+KEaT(f)=Mgh6=+(G4rPoQcB9net$b#lpuoy8 z1AJlTzIJh{U&z(&_VUu^1~q?FI_;W6J8cNTb=){)q|W0}eU_RjCUR?S(vJ!&>qz>@eZj5Ud~PdjWE7Gcg& zJ^9ATPkPKPc)G@OqpzO9BkMAD-6siZuG)DMa~y$C``CFHfoZ`d+p;b`cR~wX@18N_ z@Ri=~#2uH_X$o`(&!!;UO;ZV}E2)H9WX8#BCn*H$^L zY3A0m-W4m)8O1!bMCU|y4Q;&$9&XO@x;s-Th!~M9TO~WpxcNUdhQG0(@=vYm`9mMs zUj*Dxq-z|}7khoYJ+n8O>bgTs?2ORZNx}?^s#kDb1!8*7g=kr3IWJu_OfV46+;l9A zNivs6Q+b^?qkApRPtc6$~THWIs(by{3~Qp8|1g zo!KCGi%b*{q$CBp5OF(Wg^pr6tiD=-kfv0zgYd_MQL}|Qizl%^3_2pMQUp<`ujz!+ zW(&H(7WL?u#~r6D+_jF~@KpXm5@rw1UYU!Tm7Q3)a6t#9)fez7w(%5F6@L(x-^>%# z%M7y~=)6_p>F>s3P~NrkL%q+=*w1lSPwD)bele?G*z;Qrde3+sAPUQQ*Nfr1Zlk-x z9@KSj!NJ1??f_>VRdLKWAOGT<%4*l5KdwaDU32jXrA;$k`JBT-kyh5X!DA2Z1=?sjx~J^i6h6H(&b5;7*HO^PjkzFXN-t-D~Bj}vam+YdKsq9EsDA#TC=D>2hR z|9kw|DU7GOlLA=!?eHu*U31r7Q3(}?`PZG`IRTbf`3xR24l;aRvvA0^O^~|0IfMD| zR+rc!kj^mE2MfXF@SNsoM!iuLH`wcNRi*aZ+mbFYYH20uQ^rcL$z4xl2uk^dUIZpv*RhOVr~B`5=F*O zn3}%t zA1;Vm%|-VG!BBJnlHrnFNk2plee5iEE@X%~r;WQBB~mT-@3`?F!ePimDtrF&-Lx1H zxlG>9k3VDwW2?0fdsNe}X+Q^VzLar1gPrjvxP-AkuU+O9`H=rZi%g-CD&avsxXseA zpbkZMl&Z8{-e@=+ZYTX`8v4(FrtLyr(cRPgpwqBvN(i%R=;6|<*RYmoyL2W@9uIt? zIZv)@DU(_Crwa7XwY!rot3Q#WDGIGdN6*-|*GG0T;T`0o(q)|mh+Jt-K8C@S=$*8) z!^3Ky>7b5&{Y4M|yq>T3-rFR@UK(=&5I)`Djwnt7f4IZ4dulPxnv^Q^R!B+r4;1pZ zbnw4g>HojK;1hoDvfHm*8Nz3wQOpwlay72ft84Pz9co&Ib0LgZm9;{E+82K-w0}vM z@%P@_4jY!s4QR@I6Z=v+NMTD;!vAJ9R+B$qt=I&Hi9SabL63<0;QamK9~ADKNUHpx z=fh3OqXX)d(fz~I0+kkr+-tO|Q;(-swA!agY28t~`vU_->MnW(EttJ2fbD2_x{v_z zgE9%N_eA`|99QwtZMe1N0rjE<@OIlu+c#S;QThC~4cl}Knla)~8x^uI;lj%8HF3uL!3iVUE(Y7Mp2L_l2T zCX=iY?~A@|C_&T>;hAN12IUDrOGo>$vIXoDPaDb1J8j*-;=H+_x%yC`>HM-J4eJMh zu77oBOsA+RyyL?w%=l9qPSk!JZRr3wl>1QXE#*U%2#EP|T`8p5smM^HmVraqCRg)U zvyQPeSB9E#C-eR2y?q%e$=g49eoB4*W717B%PA-j&|PQ~+F;%SSB^bljriD9z%P}* z2(gn%BCw z#+J2p8lr(6I=~Jq-*=qi%g^&SrF7BkCyEDCC>%>eP(OLn+$FeEp`^KzW)fYolI&_I z=2Cx$v$Kr=SYJTv_G&cd=skDP&orcl zoCO2;ZkIlh=aNF$P=)`XnOxg5^WW7qz9i5)yS>i^Dv(wF9`v5->QL@$nejbE1McHM zcR+NW^XhtC?B2ua;LC}(bGR;D{ZpqFV=6OfSM8p8B|E)3a(;YS@NMG*K-$)8wLIDw z{mADMreqyO8OTS4{xzwWQslXYZh1!PilkfJV?NDknlb>EmAQtK3P+msTXq1@4h`;I zB!)Qrnr~3lE`%N8iqp|&G&}jSh=jU=tIFB45bP@TbXLt=LB(nJqKGyFt#+2{1@?m% zxe`7t$aSBQn}kd$F(;E`^{F97$a4Nl!BSCLI6L~mFUYuvb@RO+4dNe*-cDMp90$HF zxRU=g=i%l1U*HeTE@FO-U)xrv8rjpg_^t@ zh1G7RMpowJ?)qvwXSj&*eVPLC|J@5IyPaKPXcba2c(1Z-N82ht5(+li(+{fF!j7`* z#U@luw<$aM)zOO~h&xsk^-X!Ab_h=C1Goo%D92Jw=wLQ!)d0>mU&Bq1 zdb3r?=t(<>ziXAJb}PHF|`RX8e@(*lUX%tJps z&ybvkD!Q?!KPPKm7eklfaEcJcc zEqbO?_2@B>UN%Hcf3k1jMY?&xv6q)DP;K1)0@gtQek!(oJsUU0cZ!t4M~M}A9Pt96 znL@LJhv`MU79DzK#AiUemd$31Iqytvr=-0n%X*$pZ7YQ?bahHDTxFt-^;>T)5m9B{ z=H3}jC3I}82!wE^RsZF8cgHDg8n{BK%Qhjy)y_qStGPr}39L=87PM2B5(f3-P!J9h zT=HH?Ze8f}+DS07bMfVHrk9%rc^5E5Q)zy%NzN+U%fcMOH&Z|Ql;O$A#$r2E1e(sS(k`X}lJR4!a*YCkQg`=s zGEmzyi5@vqQ%E3OIrZEx_#QsHEQSpbDP7jIx>nN90&~DrX!uW-&NA~)g&uwg;!5mr z2Ayi+h#Y`>bcUmo;2XdfX)ae(ui?ORNsAL8l(2hZAWmj0T|tMcrAUr^ zmg2o}b`9%7uR3~xz1ORBuriC%*=?W(Q-F{k>AbCBeo>yf&0|fQKymvi1`t;vK};4$ zbb{BLq1TfhGIOA2y)J89mc6%^EtP<`{O2w?@3XTVg#~KRZJ8 z+tI;Qjh3?vQE_>3EAsd3aMpNsJ`Z?jf!u%9Gs|kAMh@7Q%89N&gshBm`xwO0xOP)A zLAdnq<5Az2L+Z@&Q-ddJp?idCT*$|;Vue!l~Pjv~%*Ulolx_do*{GB|-DUH%i@6G!T<}rSSnCoYHu(DNy>$R#=opu3far{U7 z@T|Joo%=?a0MyR#N$E_c(3t+GtBpPW=q#yum$$RQ7#_Iv9n9guRSLfF+uCOWfZ*-l zyy$EqK(Fd0tEpkp71!2-7$)=NiY!ZT!9rZ@nUSZ`&5*Oo zd-rNW>F*UT|!%oGcaw2CIkZhDD>88#m504n?J;tn)nT(USBo& zV6~ol)MQVd#(b|Roh|Jy4E5Xh5(+It6>Q!imBiR!a`GA=``sBNXBSm+Ub(qzw@Ts$_n_xslf-?d4(K zGi1|K?g#R^L3_ewQZFvL^F89A_`{@2Ii)>k**jZX3PQ@39pb^BGN=bT%M0j>HPqx< zI(PzgW-~=Hr325fNYw|70KbAj5<11Agp02xXL$c2GES9tCW1Xtu`U#5 z=MQ~Js(tsz#PZ_&@l1=qxrR7g@h5TUEvXpFy}MdMf2*}nziquAIw9s~Y7^Xo9yRBm zoi5fom9oizs@$VC2Vs8_fnZf?B-qx&;CV+(*DiJg_kEk9ouCvZ-HoHmms>HDto`f~uKhaCzqz=vKA6Mu zM@pKpiUd6$E8E_Wh=FAC;#pj`0^DiV4}d8t-^{7e@{}XsPx)UhOGtnGJsE9^y^?tPHoO zjS$e#Z@LH9e=PAdn+~2CetcQ)-dPY8-B0`jb5vG14;XJyoTe?ti!D1J@GvbS;%-xv zgtnF8H{}aPXE%maI#9Z^E7=$=$>}W&zx}hdLQ#dNJc1E!e--eV@WA=|1Cj;qbF|}) z>tZ@&J_y>h>8^qoZ`8{Q0ItK0E7<94*Zrq043rU(dwH(!(4JJl9Zlvm-)6e|ggmD) z*~~kr*Rq_ByQtM7|FN}U#mFb@`9;&-fk>nw{sWzG+j;$sH2GrN+tOUOGsA-}>xrEY zvi&1;kLSy|^a1AhmaF93$wS{7#PnlJol(6&Z8>i@>=G#HS2KHM)HR*k`pzGgeRDQl zuL?_RXjP>QRCj^L+#UEm8^iXi3^>dR`w@Mfr4Q0A>$;=AQ?iM!C{73K< z_e|Q|nMvOGs96;s1}Ux?UfQz)gd~HCT>CvapJ@PmIbqth$Z*?h)Lda{4o|55(MBk} z^ixL8>-CEt6Yy_CAV^rOULlId>G)mg`#5$RP$Q5vtO%Co-)-9QO`MG=ts`-Aa__~o z_taOWPdYR%xT1IIFT9)ouw_)Xm98$7UCpy=;dE08dpOYQA!GCf#ff3zM)(0^T>R`(N@SCr1c7^ zaz?P0xT7;259wpwRz(%r~S)IyEUrGpo-_9$My=UGk9qhHx|{Fl->B5ICOhAchmk z+zl%5OPhL2&0#6muhBnj;ox~3BQh=qj@_7~h^XVb!}#DyV~n_|0Nci>N`=&yOq<+M z(sWNj{lY!_DQne$=GA8FFK{oslKrp5-V>H8moQdsuD;v%i9x&RNaM=+2Dj=pd-U-G zXSJ^Ym9Ne8a&b{S8uJTs8;>@qLNEf_d7I49@qg&l7oBx&XO#4>;clKHS?znfhoIb! z03>`eWukhb?2Ts;Cl?q3#yja%FX5;(tzW)ko3 zZR?O=+If4gKKlU7rjU}FxLt5n+1Iz{up9E&2TCL<-AThD-1FBx`dSM>*$%1V*)W^5 z3Fhx8NiN9h0Y52ybXj!t#qQfCMZ!@%v?VGlyEG`JxU`qBfpni6I0;DfdC9t1nbdor zx<|S#MEHm!5$a60p~wPG38O7b4Z0Mi+p8C_+^eJqW(V=j<~G4kH`p=@b7xGE*9sE7 zCK200zg8y$Vwa%JRYT5AF{UV7cNHpSg+d?}7}-d^kmOL~@p`hhfoG~;LrQfC)pvrb z4nh_H2*r|gL*<72v0RMdiwE4T9UYrzGJcT8OtZKiT9tHgRcmw{@^)|E2+8g6S36zg zk802l#zkwf0|7u)$>2{907g@H_6&wMK)B7i4CIbZR3BO3%{!vp9``x7D_36g9xrjc zk(HgN(3+ZD4`wNq8?k zU64>OitH(zO|Hw^Mu@G&w_=}k)J5{%k?&FtPak-Tyb2?l}$A&su0(ZyZ-wrAH z@j;%4x^siU$tAcQy_{e@+#rO;GE%Ri#a7zRDe)a`pw1FZ-)u_uj;Z$%YkPRh@M5-L z^rTMO-1Mf(*PtBFZ$Z*w0+toEo$KKluV{-v*j9H9hlNhj!DoVO2y3Z$(nNy=%a3tL z%~1&s8T!C5wr%*$(sx4PUdq5~v8YHrV*AV@s=elRd5x>$KemGJ06bOj75Z|K0Cnw_oqv*CSLpTi(BIB>@;lnxQn4L%YwMU2Dfe zwAiM`tHGFU?tupAx^*nB;a2%C9UX@W1n17kw!R2P>$uIkfgDr7m$6*p5g6QTk=t;R zQPh7aB#7yFKMrU-bfYRAgO$ZSdj~vk+o5~!$q*Ki>4BV<Pf)#h>YNw~_jS*>p@g zQausBZ|6_6;y>4Z!%qJ>lK(f4J@uM`ct^t^J=+ zhqqA%7Bi8XN09`gh>_1>r7W$y1&e#)sjx6qM7ZEFM7XJh@#f2}^6}zHCA(=uh*=1Rmejs)-sJn*W zi8=Wojq^eiJT7etiiBt^KTc1(@i=`C=l1hr?B9JcZA$Wo&Kf#=H5`t~ir0+dlok^p zkGgF+-I6C?uAStHCC%+5!Vag0sl+3QQ%P{;#0%%ik1h@zi&y?%A8C-eX>!(&|Cn@} zDb$ihpc+{>`mFGM%+XP51`>08;+DxA&ZU(Vg-_o9%Dl?*B1||Nr%p zLVi)MKo(PXm)@==_>y`4_okKkLih|Po?FuavB^OaK=`tP%~hG3)n{8$DI=)3{L@9w z{^&;K!kEalld5VbuTC=*z{%5P8+A=c+}B+1tYT#7WFpdd#y)xE&CSi}dtEF(^OAp@ zPtom9&zC$+5AfMFDpR0Y%7a5#6?8xpUuR__Ihy~g+^L|qM*k51Y&J3&TmjQNlQeYY zaugl)LQVe8hcY(IBZzN14`z8rJKE6IcHu$xDcRzE$ofUmsoN+^bgOFkDiPromV8U!qW6U0Ml#Mc=-?O?(Jz(<{fm!;d?F(-3)U!b!5p@-U~R)97LJXu z-B;MP@m%@CBaL+DUAf-Tnb?*JY_O)t*i_YC*6_xvQ<*E?C9)=I_v;*h_>JV!Dv>=} z5NwzVhZ~1x0!>+he9NXFqygI6rCLd=ElWSVTJ4@&wyEMV6wWC}_LnP#e5Jnsg?ppm z*S{XkxkV<@d=O#&;MY^OgNf_-I)^-$0j@;Y^Nskq;k7zy=W^Js{RySskr5HgaxN+V z9^2@HzWL+I@0#exook;M@Mb5czCG0z9hfGMY~!8)IhR#b_E<62X^8UC$mZ5oRE)~d zlcL2nl&)OkEN$wuy)Fik=?Smaf*6 zl}%JHn0C(6R%3(Jt35ih+)6}exOI&-w*1mbzXU9dn>Q6zyQKbc7Wq_+r~Y=jS_leR#-g3L#`RQ3dgGAMul+OgW9i!S{mCBufrT+9XIaJ>J3;$+vW4W3L9AU6p4^qsd{lL%Xd7DX zWdGlDQCOLrACry7nJ%@vj)!{!Yxn~Fk|3vM&vALvljG}7UeiUxC*Ps~H@UmjaI0xs zE43jhV-=|23S(bso6Ob5$-Vub(P~O3_ED2ut~3EF&m;)pvNs?EzecvbvC;7c?nk0f zILRrNoJ)5*Fri>FGjOTfz$@xoa(_tN%8uiC^U#OWN$5a!Yv$CJvshlR?5@Wb?kX&3b4@>7L!vi>k1z?AO_xfKuf-Qok z@M}OPvP>aYxe{sCk{jX!=ZcG7tz( zwResFU?P-mY#tP>NqvYk$v3!9&B}pR$lUfgVr}_9E^>*!(sXYH2hI7 z(PuU|5qAE2_%=k9SG&;1h1%sP8*xw(eEm^t&yxfDwUjrm4WH;*_Qx(cs8XQaw{JBQ z5re3zT%Dt_K8`hCO?03@TTT5I%g7VMaa-wkS@GS6J4tQRu!)Vy z)mk_JoL--@0>S7m$*l_0CQa6}XFj}6c9En!Uo#FLj?`R{W~W-I^>;fX z7y$RYQep6$MI7z0*)mGX-_(7>FXmrd;4Amlu~CNa&eTC`?zem=Uwyu}=8@WB+U>t+ z>;(H2QnTZ5p~)p(bdG3M=CwF%=1{S=+CI?8(tXJF%qts;H3b)RUo2$Cqh<4I7C>7K zfWAGrK_R<~7pH&A$Vu_ORUYiI=o|*`(obQ@EO26Qh+Gq|HoX}~5cRkdGG67}nmcUJ zT~;~VIy&7dY4^p_i zbxmfQ^U}pd;8zh>&OpzXQ#jkfz&|(wb^#V0eHwVn>J_*jSbD{@j9WcPi}H zlHmx}@x$ZY^_KR5EwX<#9+V3{rxjH;t+2fo^pPe_=FS6E#=l3GX-a-!^DkdrdsA?c zw#=f?r{&HA*2Rvy!oN;-YWJ@1PZa>_cMA&`)Jlxo8YJUw91eRDQC#2k=H3@_&A*e_ zl%5;|R0F>F6CR!PF4y3{892UfBq)zo4Fp4L0}rBcdp${qFk@d*P0h|yN16nzJr+IE zv@lsQrCImAK@t{KExG&NO0r}p2wOHftH@oy<7;A%W=j&?D|%jIkY{dE1TKXw5KAri zq#yDpJ3##7Y$=Rk-Ten@N*oC_?OE4~)vAkl$okJFc^IM@9R3Lc5VmCLV%3h%3_2h3 z>of-+6Scg+VB&DIQ@QT3Ves4Pfxc4?>cZPzP8Fg9N7#^R(@5@N>RL|qu#=nSIhPJx zf(mRdYjUk-v^wy2Zc$TA5Fs_={%s2Fk-rMmNwOTAt<$Tm=?gdLCSiEUY4w5ysn2xQ zb_xU~UyD3A;1-{>E946P@FuxK!!-GL6tpSb*97w zc#0Mt^aN>E%aNC=-Ai4Q!ZVp*VTxAG?OS5IqejE(Ms@PJAYm6v_u2#1iW?1hmGYQ^ieLSU;X4~2> zD42JspFR7eyl^F>q(U<0$fxR9%%LG@edbt)iSHA_$$sX_ zaf2Ib!l&ZLpslGShwziB*qs9U0K!tEtk!h1&jM=pQG@+gap}>-8(u40IL;kVPnnX} z;YK)~NC&b*NqUr=wMLX!_9mPCLc{R&c`*G3P4spK{K;oLlQ4{s{(y}hasB+>^ZCPS zb3vnVgcT;P*v_RivR+yFVo0Ds3$H2vPy{eB>Mxp?1nxGt68yv99jlw^_C$wCc2J;h zZ?i$3|AYzu(qT_ScRdf21kOj?d^y^=sMfr7yUexF)a&aTR*hpc`V!D;_zYdDEu7Sj zN00X{TAb-}(J(|j|MQ+ZB2aw?6xaESKvOb^*Bt}LTGu7~^g~kX0#!`o$*Ye&$)7#C zMKj$Kj-jKQZoxHZz{N7Q9}JfTI9gV_neEW!JaTakmmUD}-O%_AU8`x{5sQ$q)0q+!>DFYEdKHGWOvkU9S6led@YjSij;pZNRVNVR1X%JbPM zqLz(M#&<4+1=&ZX2-kMXp4jgQyi9+^uf-u6Cu?D0(XXdJJ<#si;J0tDO zIA3YaJ-YD4lnC6^fqZ`t^bFo*+Pk`ZNK4bb^SDm)Q7!1ckny@y)*bC>vIDVB0l6^N zkum3%LV+eT{pNE~;f~Hxmt;k;A92yK#mm*XN0n!zuQ(Yvjlm`laq;HQJL^T4k`Uqn zz8cC6r{c##;ON$5bU$}s#m#DFSuNY}DAMYKxnc<+twDTE<8ENdM6p7I?l4_yR`Fu^ z{pl7oGpq~(4y*rJz>r>Ay=oS&`Ha7ufv4yd+b}^1%qBWl)>3EuC|W&2tS$x--D&WF z!02(u=t#CwF5nE8mE6-1sA(*Ky{`6YP_I5wv;RI=Q^k72;>tva6IaM~RSEIuZsV_0 z(qM!`^sdoN!Yhr-dTb#x!AS<`j$oa?iNyM~5Bs*s%m5Wbr=)^VWTy2JVEmmR6B{gb zz0f=WpSQbILKmKSPq6N_5FlyjWBatk6*IdiQ{_oGHH#Icabyh|o5>DfYkos@0{JwC3ZNtpvcywD<>olLD_qWvuyITDj zW9g!XwQO;N+Jha$(xp36y(g`gULSd!M8l3pWirL%6dxXBhPqrni;#x0y`F>u=^xA9 zf+=8wsDu}cw2no&4)*S4hbfX87e0?ms{n@PWMw`p$#4jE?>veId8e^N1MA}$qeZRiy5AHkBSMAxa=l7;rfCOT3W&8)7C)EFGnI3y2q z3kmI^(yGx;*26+IR~Ry&SW0xq;=w6#*BXunyoY zJ9r80lli>i^p$v$->PCsU0D22&0LZwsH2F5rrgEU%?1>(@{{9{N^n)SSG&t@UzuU@ z_fVRDaYMjG#Shhba>FMkM~>((w8@#~8n=T25P3Db_edPB(R*VWPF~wocwR2l_am06 zax`a9zoQFwya5!f)z2yfpJQ88rt~g-T|npkb;z+E?lJ@o+sY0aRC_p^taL`Q?mI5X zoqZaD6l|DvZ)G(lf73N;!&f?lHtr}|3grBz`pOD*K8;?>nUTjnYAKG5^H|gz?GT#z z7KZH|Cr0S7}kEel<~jaAu9pW5@qTA>C|J28)Us3bI~ zQQ3jctz6Ol8MO`xio>5^Ag?@bAL1nZ>#Z}+@Vv~#J}AwE+pv(2@I);mKSSJmv?cTS zkn{`Dvs^HOVRvr<`R_q|j&vv#A?l?Oz(vE?;A&>I(&rRHx`$h+Yj-&3fveS#L&EjbQut5(>Ed4v5Km{2Y$6rbbZcx6(63h{7fG?3hC?ys{x@&g-AgDI z{Ir}BCblzPY<2+?BXdCM9Wa-A#AGgZ`VM{6)0N6^^o;oH1|<32e)-ZS7i_|6t{3f6 z51VFio78c;9A!B=Ft|x@#yje_O*9BOHQ;Ck(p*m0H31U?yqRv=DD};>&&wo)w9%lNk+k6QHT(i>RQKX4by7qrh4ZYNga?It~V{}pA;S%t*M4Pc)OhuOiFf> z8ILoa20EBpExB(D5X)A4H5wHLOIcIasV{0YZKqJ0&>k4)HKu97lg~-B1wE zlzU20=O3={#QqQx{@GGqS64klrXU`%%-bI6GOB<6GCldg%FoLIqe-{P2!-|1^P6Sg zhFt`kmu57XVFt*2oL||zy41{2rwfmU2ZWSEZ3WCuSN-`xE&v~pZAy_}aw-!N{N$>A; zcnT57sLx>UY%W62+4YUXu|A82!te`NTMXynSS)KSr2l6T!+9xJMJIoLyw zP_^4U$#V&1{c`7`I92Y`y`EveZ1d8M$;Pv1Xk8rutix^yYu2%Q8qpFb)5_sIFQ($@ zr|?`0W`$uV-h|!p=HXB7RW4PjcJ_yZLUdA}nVs_6E7lb8$w3oq*<#_`ZD@TvowVw{ z?=Mnn27_>JlQ%Y3DMp<}q0sG?{K(Y+qws;A=PrFi~E!-s?mJVOlPk(kNzx877{FGt2y;hmxQnn<(>4C-waF=5BoB&nl5SB(wFY1q(D?T99 zYHbF;nTXrOOxmwnhi{NZngOL7 zX%&zT=|;MS?nY$jM!G?|r6h-L7?1`5=^nbfd*&Ixzvq8l`_;UebJmXaiM7{hBJ(V| z^Stsk1t81ns3ehlJ8RS`opedQ-( z?bJrATOZtgSWc_>buaMylZjVhF0_f3*dDk|^k1q_R($)9N=cPuwc5>f#&m#R{qo3f z!tXBNt>bfLN@wJNr6_*~A@k%zRi6wr@F`^AZxes>?b_8FX&U ze7q?<(iC_I?rB=15^(_Mw^oiKK(b}ouM;QYVRH{7bM!oyT0X)ZyD?eHH+mm+Ah$wi zheF5M$?`~MY;5yNf}LnXysY1RFP4i_ifsj_wQb)rNp6OQt9h*u7~^33Fv%^fB0 zi}qz^9n9;B$l!W1;wPQsJBRB$@0ZThf-@dY2hfV?9I#M1wc-MPHw~Uyq9*ScPY8W< z+L`-0JS}&!{-xPm=;-nh5UR1#h6_k5B`^4(q5)~*e-t+; zq;7xaK7LhEaMpZ+ZcLGFIU?%(t;e=$y-L-wk)#>Elds`8Dv4V|_V^+8XhHvG*`lJt z)^h$)$zWMstV#b_zKLTFHt%<*YG+PQ-OP}O$VtW`0;Bs11eK?e2Tt^V04u#!ogTsqN& z8l6f4YH1k{MFiD$7bNTylUn zykbpRR>yc_%Km{RPxreoaIc&X{S)fvBg$-_-0w%-2a5TYa}Jf3ElbBg*^jR^d5`J) zNKKpHD~H~8bEFyE{UJK3aF$(8vSf>@tE!TD&3C!iLjR~gr?scwVmlIryZ--7Z3ZQi zt&!O#*W;x2afMKyTP|!Yws-RCTCEDr>_X1n)iPxs3$(0bGd)rZ3TC?t9bi|^UO56m z-$F~i+V34dLp8KZZ?p^AJt>pL^(hgdNtz2^9rM@o($crG+Qth>zhwG=oT4bs?Mj6I zs|i`QV)u;2s3iS4(y)T0K5K7>yMOarQ=bt!AK4DA^xeH2ad@*%rc(bNU6HcpB3*W{ zVFB_3JkIpP{+A!bUY)iFW}}p1OFRtShqp`fQ%7@LzZNek#@5_Y^j1%;5H9yjBM5tS zVrm9SDC|LUMw$N8M!c<2siNPQk70pn++3j^{aKYWx4ovT|AAD>7{?p2jrcMn>yRV% z3v8#c%(a`N&wS;5AUlhD_V;AoE6)KWi<&RxHTHwIYO2dSGQV3AKm5}vR@*!pb!Bo| z68WpC9C^4ro_>dFN{(($AM^ikGzstKnB%q%}gxpT= zXV^>PaXs1daXo9^sisQAQ{PaPw&4JBXaAn5H+dMhV_*{8KWWzk zs55yXU#@6#6BF2ni;z)Eu0M<#*fxduqidHsqLO^qkfpErtdR^8dh~F9N!by5M#wl~ zZZDT~RQeTKSHCCOP$}7O+T9}3MN4v^WSpJYxCfO171MToZk~07;Hmh)Ll7%;gV48H zI5u6c?2HT4WNX=Y)0^mud1RG2CTz53&GWl(I>J@J(w2Gg*I9kVwMb2_e}hIvD6A^x zqo+O6LHnL}6?|I!h+@u$*nh0yZL_G^Rj$vpJK1f-?Cd)ykC?D%pXs(%wNC*e4^e!@ z%i7G82cRV>z369bA`uj62)(no0L9ODurW8U?E(*5r1fYC=sN= z+^YOS&nY149Ww|6Y3d-O7wN1@Jj%i~>HvDwT)O{@&~*bc#U z`!lU>>z1kvQoWVR8Ex_>I}(G*9|+$_6R{YUpM9fbR91KpJ+fZ!@|l~yQ$f5Ru!{M9 zev4~)PM@zte&cRz9k+TQGi)KnG)d;q9_+@WsB(d{UVI5j8(cjHdaxDOPZq%Pz%sOT4Mt zkf2$I>FTzRU(Q%L&$rSsp*+@!KhWKY$kD{%`aB)`bKYxjXMs+Huh8o>1bKRX| z=S`R3V`i_P?Qn zE8*9lyt%aRuqo?#acGa*R4p!dHB;*dI8r#&TfqHAmR9dnbnh4L?Vx4Vz1e z;ab<&r=_x-ozdig05ysVew7bP>-v4-jfwXNpMc}A6}+*UdP-3PdTkhDoYs}yP#OR7;0|+BtPM210VN=aQ6qVS0|a# zDpw-@oh22m7uC!sIq|ak$pbi*?p6AS?=wtap;I_`*qiRk;H>I2Pf1pjbc^o%roFw1 zH*chxaW^U*?diOmdOBgM(EE0kmsw!%6*~G*hOl=tGMhrbw4z!ar9quu8nzm``Dj6? z9kv-I<8%8d#vQ!6sg`gD@)b8l0*OrM9gh3qHmI5h>O*Q@x8_7p`YkE}fDpFY#0ztU z^xFIx-m4yU&P-%zce^P1-Mr#?#tPsKL^%nY5vvsWnk^Ziq~6UHCB z`LZ(IrS&Ir$;(S$HLCCijjaW1R9VfS2lZiO7dxP_w5gDg;;Q+oOoAHNHQnHEvVZLd z+sAm|EOW0!zwg=c2>I!A);c#fN?cD&&e|peFOJImvy9D#ofiPS7hLHjTH2TsT@(Zc z=bxN{_ne9I)SgxUy5G(4_MyX9hGu8`*VhH@^iiGP?X({0@{3Qt>>LZCjF64!f7b3?gka=TynRqIv-cIzyiI47C6BwGCkje=N%uo3wY{<5 zQ~e2OA;Z1r?cidZzGkCZ@8c(pqsN#Tvx+m{k5i|Vu8}vZ6E7;kf#Qc)9oM_ew|^?A zJpn5%^7OPkjzz}0mAoWq*`P*8aO)xDY5%SLY+XhgqFqXw3WQ+r#ocu0hexp?-S~kz+lm_(sb<=z}RITA}V-_;o6pwD6-%Y2X_tuO1<|H~m z2%U?yH1H_f=b%b;-=!=&t@Kavh)(HAdF1p=>&I#n;!(ar$1T@8X50IO>H~T5)2YV% z5b5X@J~ifO`)($hF8~xfr^!Bd-vfW=ks20+d#?d*UpC-oR)wcl8Q17?{C6<7`Ri3x zNSAC|!*>R(qGCgf9EZJU)WSGL`_{c+hTO z5=hSnp8O_>s{XvfYZOqjp!GrZ190eXTql59bEWM3LPG>rv@fI9687i^SQS_?9^*Xo zAx@@T8G)Q~_cV6k>OAXSSoN>tAAxalXM0HBApm(h;%KV-?m4n6I121E(iZ+Q3DWJT zZjqS;`B!SZ#ud5QRF!Gr*|SS)1IG`*Q69>@TUYMSNWuH>~c0J{%CsSz*|o+WFY$ zICm1bb<&M&`YILY%STzN1CdOQ?ayVJplq_dHi+b0fzh} zO9iZFF`qr8H`0xr7u@YE;;fX?Qsvh_X(H77gHwJkVBbRV0RUa`n!uf)$C*3g+_Jq?HPgM7s+|+Yh6<@T~ZXP;G}V;rqQw{zX0_E`*v^#9_a@FkJoES{ z@yD#mEkk+fwzC9u8uQ^`OAT2EYUwT%y#OK9`73^xVZSmtIX}jP#7RNe|(N zt}Ql=cJbjBvsdYIH20X-C!jVPD>@{?bCrA%?I2tCw9PHEBAq*xI;*WzLe#W7u_uLU zO83vT=^U&sMv;8MYw+U$QA|!Eqv!b7$KPcB!k(0s6cTT;n&p#;$Q8Ji{-T;G$cvzn z2c(7F9O;@h@R27cX;fYCh!%A5ID028mlFe5%rNhB7Fh-vEN`srZ=$f+<;AK1+=|qU z!lf?HR%R3X7X2oZ;&+p}V?V!6BA)UGhRy4=N76%~+{}#fba8W_1>v7Qz}=gR)wOan zg9Q=H9~u9lrhP2j3Yv}S9)1H>nLE4r=ma1-mFyyEW$2Mc#Q@N`nXPFog$9!+1W3#c ze{x7NKsY&(jb8aPa~fS5gPm}wATwQsr<$fiPxbRPj_8<7 z7xshnw#X3bw7c(Hb(iq034w)Dv&wfmAIRA2*$G=qC?jK?- zw*In#WdZoD$VH<`J^q|x2|^4C839{UHh<*^yAq`Ul8{$OJ|sP|NLta<4v}ztsmqMPO|!S@dq$)H z^RCD99f=Bk8CCCo!tqaBFEi!y@MEVs>l2;J5rg`3A6-u4j#8?m4CGL8CLk@N9@6Q& ze$Z*W{{jWRLmzIt#sP1=M3wz#r zMp)7Xt<#G4FnW#yII8rK4@BL;0VvNRK`)$3nn6E>Tx+m6-y7c2R^(ssT&@n?(Hxb> zJtc}sXeXMGyGj=gi}DD*0lCSDOVb!9VVOR*e<>)b9PSGm>FL3+mGpFysYwty^x*N1|S^1Li==);5(J7xAEIw!4 z`bItf$m*0f$%JX0-KC7T6Y_!LPjAu5Y;<=f4kWVehhV41hR5gAwgXpz0#lKLguN)G zt?oV<*_}Zr76tjOo`u>>b1|ks)!X+$S*fX<&H>l0&-c>jFR`s2*H<4etB=%CRic-1z+Bbz| zG}zgw(67SRO%WFpkXo$651v_LqM7#Xk+MMO6Z)>#%(p_8_Bx8tZFikUIPZ888QNys zGtZxhqX&)IrQcBbZVzN)ps#g*Cc`>5l3~@lGx{ZD4|MEIJD7H%DV5V}<3@LRHPq&6 zLtIx)*ox)#`m%nARqCF5XN{? zd@FltnO*G<1Rb@zyO1xT)VVtlaf>KBsC^)GSD#Ko&q}!?8rJUFxk);zmpQ;F@(tgY zA5tp#-2DeL_NxnVib8ir28WdjeQmgp7y8(QG16wrnbnZ{; zSmsb{FITEO7bJF;1+hcnRl5C`1MT0~o2ourIynWX+n7K32+tw)75t;2gbN597w#-{c<>`dCC!u51~R_fWi&aG zBCpW5bYih9YRm1yB#ne-Ol~K0^!HF(&gd>W+vRl=s+g_>#f`DXkb_4z;P`n%iC;HBr_PoDRAl!>~^`E}_0$u>_R{ zoXBk-9~Mg1Xsc=dH%~eWpEl1t6`L}78PGnT@S;Umn?xSXN*^!iR3)peAL`?s-CvwF z?&V2-qH>J)N0M=Q{Guw7$&E@8B-wJNNibP5jjhBWjvS;E7_3N>N4X?}+=}8h8&g8$ z%13)=eA9bLdr7wqP6W#if}*7Z}7pBacH#(q1J{sgdn>~J2e(H=&b0uKRz zhg$`Z@%4XOYMtTXwT0e=?&)`do~nKMOmAw) zej(bPT&_+TTkK1}swChNUrD76dDP3B2w!(SBbpffK6u*Db++muertLoq1y6}O>yb_ zJx8&jz>oAFba{W^GFJZ2p+!`x$#E_sb4Wrx$sKN?*QnjLsQn$8~Qf+e)gPbnM9sI)f_gmH(C1rkUUJxC+GD@fsl}P^9cRbk!=mWp^_R9 zpS`@gA^p0om`H)ax#7h%fLfjdYsTiH-2*!H-RYjfOBU%cHD4pac&p!3JEWjI%jc^Z znftsf+WglPs?)_WZ&p!584p|x#y$mZlUf)?s%LKNv#4LEx7V5r)?u+ddg0dHl`?6X z&>;m74ME!tvWFu*^*E-YRyWUX7peXaSsY+mz+hRRFS>nPX7xTKoC<5M2l0Q0Ulg4^ zbzwKxaspcox1zlH5>YB6rKQ$#v1F)BJXdkrwbDA>JiqpgTps)e^IRD|$hW!k&z8cF z%aWqxWiYXqK$h`Q#HgWB(c>bcu0*?pz|`j!g%XjM2;PQiq2?6Bh{3OK*kTUnBGa5} z+ICjDM6mkZSa4pkQN?go=v|Q+LcFqoZYLoTLfl`hp6!9OwN^*BtHlSIAU}qLWN%=S zj`KKdLOp{U0-@#d(We{mrO%^Pg~r5EBmBo2@Oxa>N9uxv1WgAevwqM z`7*y{m$Ab(uUP6{wGzb}YJk<@e4@{nY&_?1X_p>FT$P!}Zi)&!HS%?GZap4Iocl^1 z@M4&m<7?Js@DX5xb}Jy+753NlCgJL8vpIXG5zjf4jL2$ifCE~-on~oisR%Fd7=h&Z z0K<&konZd{=UhRRy^%>BuJkA*Ef~G5soN>{IG;PNQt8NM>$R(YwfsEo`MK0eh22Tn z|M7ou1R(E{*NwGI?9Z*XlTotv?(j=Q{by|?ap>myo>rk7WL?F^xWZ^?s`fl9;lUeX zo)^V=7*)Z0*`~mwhf0c~mGC%}j>___Jd{390*2)E#ufJ?5vC?)(4VS%y*-QQP9w?h zX11D)pK3Jz{T)rvZ`>Ag8r&i{@Z@D%Vl#R4Nw*$zd4?Vn zu%Cukz$EPe;O>jFeAf}XM-}X0-rv^MOwvOd2?EB}-s!l14FCCT=N%DW>oV+mFlFRy z372Pmzq8G6`7|z)TI1Ri$PH)dV}9u}nf%ylckWfUAD(h~*$4bu@`aHaVq-iFd#W7- z$f0JbT~sh>v!uv4^c-+Cc$b>^JZCG?5OLn|+#={E**0t;Y=zXMTdWH_QY`G}oOb!rzhKnJOMc=!~*tpbYBQko{{2!kV zqSD)`eY9oP#3bxyA7yzE9GfZ6|03tl;{-QV*7H-hDFV3g`~5vq__3=@J_>tvkX8oA zFcu(Aci@6)joLi_-yJ3!qn|8h{KtDfoue5mFE(A8n(Grzf*t-nRL2eL|E*CoWS@e4 zywHV9Mf~`nlH@=@U&b2*_d`zWwtQd3hM;DWL#I>FjvQu__EE!72y@-(qXtzC=r?@@k1QIuNnMphE{b_wRKdULfI#p~+pE$CM>h5qKODOmi&3%R@A%QF;GWOKFQSdv?ySB#bjzW!(n) z+`TziEsV}yRCY#PEqy_cmmAf{Y~gx+-pQ)c`FLFT(PY8Uv$TgPe#OtP%cK#xdLRt? z{41v8O3*ew|4KT!4e)e^epRW(Nx9s_u33p39mSww>|@L+JlH8zYt!;|bm-k;eN@{! zL~)?(&NJItz4p(Twmt7jzsy?`&?IFc(`eN^?f`2sY^qaHNFJff;O(*3tYjSA^Qera zt+A&}CL1+sLx?{T8@G8lI>&xCzi4ns^`YW5Qr7$4{?ie?*Mri2smU;@f#P3k)Vqwf zfSnwuad=1_&heI;v$5Sq0YY|;oe0D7@BRT3#CSkSRnWf$$n@rV+9yP!1ZRosx@4&y z$t4-=sp z3<#-Y6LFHrjXzfOCQi}h_nwrr3|QDjd_EyIy$#t1@L*~&0k-tEqxFCs`DRCfN4hoC zzaAJ_0Mg^U%KKgzzK|ei1UL{8T^Klgs3n|hIX5V?w2dG9N6i1j{E;Vq3tcrUp(HJ| z`thCDE(XIHk_MUVAb~!bbgjJH*FT(CTJH^2PCx*IGkjHkf@%Tx&1aPp;IVt9T`nbj z!(tbSQf!7B-As$U?QQC#$MeO2ycAQ?JafOYWJqII{rwCHQv z=TvNtSaMgVJgUoax6aU;Y`4(Sj8#GJYW#J^Lv4rI`LkhVyqGj=8X3@uXZi6;r)nIf zYO-xI)y00U{eMi0;J}6Aulg&+*pGQE$uM5YRQTHRX_TeO8F%HkrTg!XMutrXJtC@S z@7PGX=;^7_X_6;*ZF5voUb%b&-ODA#P*K#4(;)!w&=wj@ zRN8$o!u`07pc$AC8h?<`-UV@)%)4#Z4zV~nI7Im6(R;{f*IH87l@^_XuQ3gn&(IRZ@D!7LA4M&ZNIgt{5(lhOQE{FuW-Qh!zc}>?f;+k`v}> zj?zVnFPC=93|GOo&|u-WdN06Vl#4^B>fFc2Vb8XSzXA8BQeCeeIJSAVf67Z2za4r> zh^MJ|-!Jv=6i!ZJSoCq%ca+XvDp=}pJS6pW`2IgNDk3mai!r0U$)HNM!GT@|YWFGt z<1;(rgF8iXs;M!8%b)#o zxta@6eXu$v#@4l!1`8@A=bWKd{Y8_0(4+xhfvzj`c&%I+cFM8bq};9CQV)E!j;YY{nIEI>>f~l zrKXo!gQX@rZB9~~egp85$asvBykWC6A;AhXs^_wo^{Yhsb?R~LB;j&Z2q1*9T1O{R zq%e7*<}mty@|#>YITAM|)%DrO&GGbuDw0t-ngxt=T$+XPHzzm%WG-_WK3e0TXXovm zA@YP}`a799X7ZdL^C={(+MbHT?`8yXEpmfWdT zG&(LRe|$a1ijYe1)|k|s79V3V{O5063(7UgAwHUbunr6_zPhD(jB`hZK3V zy3%dDW#PsuiQ(2E){Yp7xo;Opb&q&jXph8Id;7u`BC_ELpc@sYI<}MF;gLgI36SCC z>fhSd)S7*V;g#mkfL83fTyqgV1Tu6&KHQ}HHY!t*bIvr@M@6J_`e{-8oRGgBF8y)$ zB7FPz)r~9FJ(t*H$lc6|{o6yVaow39a*H7?zR_`GkauX= zZ;bTaJTpN8aB+B-=BN}htuzgLkxXw-g7`W9-&vftC<*3W5J0+Ten!dHI9{0I!P>%j z%2RVHg-|1Ljmxzg?;eWo0TC9|`ti%IGlD*A(@4 zT?!}l&{^w9o0O5Yq-iS41+5?q`2V5o_MjD_2n`ef>%q1WV1Sw`^DXFR_|?#(LVg!n z^D8~$LOBUeM|IP65eoqS5-l@*g-bnVtI6S)^XRk*xWwY%kYR!W4LtIhzHZ4dv#3I= zWM?P2b~XdekmYB6u0f(!_XBn-&#=z2f)5O9>7296W^Peigl{adnVxMT=~!!W-*#Ik zQB`)1xlVJNV~8zL*+NIXN4dUKGxGzx3_asZiQx+co3!ElhZVHtD!5^IK*<%nS zs3-l^k!lB;tx>CYj_(7+dArI(^hfw zC>>rt^S5YKvvnUCt~1>NPnAguWx#$j*sdyYh5QdQeDOxs3s`qYPv*NtV~(I?1OD?t zSkuJOke8U@b)wFwqh#`i;O1?pMyv*z0kWd7c1_=%kfa@t?e@!msbLgBjVqf?Sw&s>4K$#xuJ40w`D-%Q*h<(1d1Q1n#|%2;e@=ruO7@^S!_!%( zC*|*i2&*cE@^M!LoE5{DTf1ro_(-pYFv417rkYN!p{y{!Il;!aD5(gh^FOusMGzWq zYCA;q>)1yJ_0axbC?E2SBXIFA^p#pyBaUE35x5DxbV0rdIrmcS{MG*yi2pEAn1hZ( z0u;w-D+dP3QzNt;U%BG*t+_PX%(;z4vMtB)giqyDK->zf9$8d}er zE7kZ!&K$w*==b{h6(+eDc7_GKM5`wgb^4cuM-UoU_+W_=z8l-K4P|H*Zu|I@hx z`&j=E0yE(MLQsgqk(JvDOCzc1IWx)UR1=kxR2duQQ;mJ8UA?f^%BNRH!3&$Lh*rW2@ltI~r z;f#A2BqK{Fgp-;g$sZFD5I8L`5>Pq#$$0{V7xso-mPlLflUW@?o+L{|NHJB2MWa*u z>#rFYuyUse9T=?}VO$6M%unf49wSalN^tezV%?30K$OosXU>vhl|E|kdTJBz9M}Rm zdMEugpUQCwR5JY^^vhpY=TPBBm;Xl(b{I?^XB&Y6u^Stxm=;cp3j=@Tob95cR9=hK z4o`&Ap#Um%mnIe?aa`o)r>L|iZ+Q1#YRw0+LG`v*Bzm`3SgkiD_8dV{YwkJl9f`M{ z_b`bYpfQ^9M>)>+OC~S#N8))-@Un={-LSyR_^p0vITn1Se%`gd$c;e^F*mTo z5g>cW~dEXnn^COM2ihEpkt-ThBnqyi26#!l@{9B_o-Y;ItI|*TQTBJ{E|}Jp{Cf#l3@)XKGI>21XXtumh&LRi zWxu*q%x3zCN5&K{;tOVfJmgkuCeC!L?8O(ezH&COSRslh7QB6l7frT>`zxk$!9I!& zj3KoIw7cEod_3%Fw_I3{#-8=KV_Y}}@u9^(b%`DknzvYZvs-^|q)!kGcy6R?y6WeF z3wD_K%oC}$ch#uBrrV`EVE^_OGnu@vVJf%9=iE{F3@@K^X1a|84)(sk1m#)J*=+JucsSu*Mc8k!V(B~S`QAO^^%EY&kL0Xu z%?4f9^=gX=C}dkC;`LRRF3mSl-(|;6Gj#3CxF=Z=7Cs{(U~a~_ML}*BpFL|doX$1v0*~Q~0&

    z&UH@ho0`QDy0v7kska}ksR8|0tp71 zXFNG{0#HM`l&SBBhN~H+euC4(ycqtNdXX$i9gjNlkG7o*FC9in^Cq|-(BVuSrGBi{ z`6E*f4ikXI^-e)W`FuN!d+#%Sqod#*Xk)|HGn&K@oa0&B-xvbi+%&twi;sv%&@uLw znK}=i{l(0@PIaSIalDB=o>EIxO6*gnpktZmxBGVLm+33L^D#QU;88*^vzb>9Cs&dJ znCW#I*}M3Kw{LyRin8B%d7l59qmI3})a3t38JgzEq*CtQZ+dmojD?c!gqR=6rB%j< zzlNPe@RIJIHH}W?wqa@%nEctoBfVPmeG5dlac2amACkIotuLJ|k%iHMj^^CiT4M5q zwQ~JWEr-MQr*MONFQttY94ftEFe@7vOf!Cw*Q%~P#T)dx#qh55T|Ky~1v|X>wX>(+ zB`$C2H`!Q4@+X%(v%xn&a99XQg^@@*sZhzQ%@&Jw4oQ%_yTbI!!iRoVQ`wruBd6CZ zvdCS8He^l^JWy8e=t>FdBKg{#XSTAwzNLVyR1?m%ZY-)LgUdY!!I%<~G|g6x+$B?D z5opL$iveUXygSQ}1bY3jdvARC;d|=YI<=k7uxu?F<~HHbxE-zO#xfhFtnI1cs5!7^ zr9IU$W-HGNi3?*@L(YL^Vf?;SM!rI&vo9!R(Hn1hcSd_oQCBhGSq}|&KiODIr!(6_ zMa*^~zf1UO)9vj;x8Z}OY&_FQm@)*BP`vlfw!&QzQigNCi;?BbX;6g-LI^iTepCSv zA&NbA2t3IRZ=|7WUWRqd2+jBly^GszvhoGGZ+0BEbHnP%#mRx|YhiA`zm~Qvg$!(s zdII_vYX#1br$mb2iW>B@)~Y_4S?N@?x$D&PIj=6Zs^FW`c0}51+Q-g#np~c`JLb0R z4ox#p8HHnMV@6C=%vJVgd5c_bmEPO^Xd^mWttgd}_QK^U=L9t5vdxxjO-IEhr=;vG zr9ftfQC6Mn589t|H4ro4#lTLBbcPc5$D4he$D(1*HFI}pqgQO~!XHIZo~8k_sQ2Ix z%vq|bi>7W9Eio@F(t(Pt&Z7yv7BxV{ju#-Xp`O^J8ksGC&i4KyPCWk3;_$!3W|pS! zw`cupg^FUZ{q>kFKKaMmikDTl$di;^Xk*_%hbBg z-?9ewYO|5>K5(>3w@0!I<`nt*c+Fs3u)e)gzuqX0-l>4iui2vP`fzdVsI#_$vzZa! zWkgf2Web@;)l3nyNEe8M;S#FI%RfwECQh(`YR|)s05=KZ8JORf>5cn))Y6Y(@11^` zf6gGkbjfdH`zA>KnIfz?D}oi|-*y zVAr{$-4KTsIKT4Zme`Z_Itg+|oxuHM{N3dGX{UPGiB-RIq{|uRSTXp97aB9QrQhsu zS{0?$1zM&~wdZYL<^13>G1qGD(_*5_F3HvY>u)Wl7<=0OuU~|Q)u>sWp=iTg3}3L2 z2?*3wHubL}g+Le7k&s8h z$=Ztou1(ysk0u@g+nZfN*fWEJ4mRw%XLO4FoQl1L7V>v2m+|qPOZTJzoK;8`vCa#2 zbFVYQmxyk&aC-J-o&l3d@Ir4h32VhZS8=BYN4!Xn>zV%22Ux2$m_o>%f=x%|!rSIZ zdyF8!r9ynPF{uqe{_tJ01mlMxgQKjD-#plO;t@*3_pr7F1O4?%a8Bg$9}V@&tD<~X zJ-|#yMjh?wk2o98*DEd^q?h=CD9OVYsih$KKmXE(P%eWHrl2GQM5$$*VMAV z#5m%?&VX8g20#pd)QKtxP!e7}ThMnUC}lLmliLq=0C6HC^Q z`=u7uIae{H108Gzz&ie7JmIX8$2z~_>miQGnW3viNI+%Ot-dB&zEE^J(7|X(Tg0Zr z(>^sbY`*hoa28TCrQ4P>*T3|_M&;z(XywWn8Q%YJb8kH*v#H&^(T@xP znelfXps3R=23IvY|EKX3X@ALs)&jls!D{*`DCZPDWpb!wP*#TPE5 z<>ImhBRWC7@FHuDw_Yv&rgTA<1pClge>{Dgg1dUt4f~OKU6we`(N24oOs-nHd}w>B zjb|cOwU4&(sR4Gi!`bqa(&%he7duFzFl^2)?F#*N)Y(wS^+-_)6*VpVteqZ`O|2l` z&Ev7oQPm@plj}k_EQQRlPAA^eFCA>~FNgn2lTxxE6U(;XJZ z2oBjr*v&xFJZIms%E|_}lSZ8r$g5i=syjz+p!&n%^VO7GA12l2mYom?S@ZZmKkQQ! z3>Gq%=`=U1c&Ah$Za+yCLsAq8JAVa@#io~d^{gpeCsKdC1HUO#ME#N+TV*gN_S4>{ zT8(h`2umW4JF!b*U<9qy2X%GVI$@y=`M00$HwjNL5b>tKXG;hqN8T`PvUWMC+#tSs zxgVuqTh-j)B*d^o#@D6GFL7(!iDE<4!u1E50YiR!c*ryU$V*HUq^+hP)Wa2FoHmp` zrAG=N;OB==zFA=tD}xM2kYQicVfy=}B#d~U9Xd}VUm%jRxp3>Syn4MA&V%36xY_4Z z?55(1I&@>j2EbTw`m7ph~?~Bn`8S9iH$}O~Mz<1xo z;CU@Xz+J!A|Eg;_uUk9&iWq9mXJrjr$gqFRUo=&`4!ZmQ&UFY3p9^%@EOswu$1wO8j^K`vcUiN3kX;#i_D2!mM0+i z854r$>RG})C@|ibXF!Nyg?t`K8mdFoX1=N4p8|a|y11aPC@T`dCb$wZB({-`0?;Onb(zWwvWWQPXt{T?Fg@1-n)nq#Jbkdqwvm2cE zU|!o`zktz!F-U+qc6McOKEo< z)BleChP$l_Ik~;$Io#^cf3A(dr?-tAOWlOy`r3Tp=GF)Ug6jJQ`i2bP{bu|V-`(C>~Kp$CJadkCZR+EkDp%@Hhm*i{!u1g_oetrD*YqEr zh9x0C&(ZKegrq0r*0!F~xBTPd?O><9?eXgY1f!3c(DFFm58ufkEm3r>d$LaYs;Zhu zn}(Y-SEb*ATie9?lU-ids_Jkq4UAR2G>6qTV%ftNz1_~Jcx4z3unRQqAbhWK>^>B6 z0^{!|0jKNgjv?HgvR$$NaP$7h5^2scZT_let=^WX>}G^rEvAnSI+ee#P;Ql9#*JPG z6G$6TU0huJ`24ucNY5=P>U9@uh9`H*?oEPc=Gd0YL6+5@GLs_i(=tK!buYvvD3pbU z!S78MNv*DT-mK)zY%trbRCXGhbBm-=CM@h1WH0z2B;I=T-r6W30DiO>++OiqA0rMPYld6g8jUgHfBwj3sxwpEr-?d^Ffq@T~gTYRdHxwfsC$!yXOSh&kp!P=$~u-cwFFmBS}!7+CK|55do zQBi-<+AHbI5?=L7VF$Gb*^-f#XT%jSxj%2xOa-$CI z^-qE4pzfRAUSIC0R%v8rWlF4Lz#yP-QB9;3@}S2-4=|{Mgd@EeNjml&Vx?)1nK|+g zZ@wwq*hNN^uD z#Lya7WBGndT~%)-5zBc1W8h1jW%S|edL=|DzGYb=8N715 z!n-dAEv&~&#y3phF)6&j31WFmggPxM8?^q?Uw-1 z+t^uw&+$#O`J7tB>(7$=HW~Adl}gF4nNL}ktyMMg zeGUR3q3L3xby_97USR+1(wZT-TUXF&ziz^EOzryokQ!z4i^{&sL8V!eRjm2?ebT9C z`EUC9@@FOEcE{ccFD*mverv_dwuj1kf#qI^w&zGlz}CATepIC)wl!I<@rx~QmM3sH zwidSAL#-M56Qf;m@?Ig-V_*=CXRh}!OQqTWeO{*$-CW|N8a-RTr?m64CB$4kqqqS} ztJSu01D}z;zfr5KtLp^=;pb1gwtT=r@&Kd|TK|{JDE>MOVPQ=Ko+SuD$i|AaOO{gC=F6S~JQpzdM4nl2 zOCp2HgU>6*!P_zy`r$d-miNq_DWS=A={m9t*B{ z-N=F+#kyYHPBJ`-tMz?j1fu)wpsn%7B-o*C?v1+F%^hn=%YbMAQ#Upk{{vei@;o@2 zw(ZTYu~W%Vyuu%3FEGIXRLDR4t{na-<3%9H=eLj~ZJ1yzF#e?@b;&SqhS?7{44(?U z6=g4f;7#((Qv+qJCjhj2-;>&bAFb`V9K`YNkNL?;K;4r~tUd+zY)XGuJKw)=6>26r zwdTytF~}Qxv`GcMKMYg zn;T!GbKgN+S*L`3@7#wk3rTBZj6mca+wpc`m_o+KJ(CO_DqXn~NA@fgDXjB$3_|j^ z38VJZhR2Z;w2Ok&CpF)cI*#G2X%G|D; z%;lOj1$3j>5K&VZ_>0(=QT7@K@QJC{LKFwT1?~G=B2Ykq$~#uccqDG#x@EYyu|mXY zKeF##E^N$(9?%Z7kZ^DzQqhTCI$5@#Jryu~_Y$Sa(4|Y#n%~xTJ(X$W0R`l$G00^% zA~RyXRx#v5k)Aqp;@l+@Yu=c7&ju`F75TQk6=Y4UmC}#`y%K$LNM>E8Z`R(tzM921 z?$c(NDD&Xhm>pocI^W~Ijh~OU=N%`i;V^-#vHS$|1e6}T75AZRz`N&XmzlJ<_zh|qCTW$JRdW0LZvb+qOh zvPz6ERO9nT4W)CWC?Zzbg1}q|(p0#@DN=(oYDUyR#>1E_90r|b(}o&9LTA3r&|iG$ zb$W40|}Z z%JUb>-`Z*^(VoYijo~rU?T3sID-UkUYL7mHlu3<`$N$1Bhb+P;F zf4ugy?+-z_)h>SvuS&|WP_NuWeJcm-!ltJZ%}GA!N0|x9^w_P_*!`&w4{HSb%Zmy& zEqY=wY2af1VOR3A#XwcgJzc9->m!{^K8?3!vm}h#X0)+{GgMrPqAw&)5{OTO6lBsF z!T7<+;PPyqyQxO~vMPoQGL@L+V~P*$nV^9+6p4!%8`NM~NH)@oA-jq8R;q#KtY7h2 z+e>WON<0YCFgpHsL-&Ek7s_Oa$)tMjq;Nmx^{V?~Tb(I9N-}$s1x-24|ygX#wZ{1d`rT+76lo4oF}UY4Q<;rjw(Fl&XM&CEttDm3QGOU}A`Fy`FgD z;gqDzLt?b2s{P6<-Ip_hnjo0eQ7b_}Ha!u$Ome1YRT%&9a`0XfgY5D(S9hDFtF!MJKy zfDA^zjDOzSA$poK*GY4LyjlM9B8Y0mi-A?5$~w&7a(XxBZo60*j2nj3A;*ubUeM;< zg!?5NXN7VFPqI-qB)Wriz9R$>OQ6-* zkBDf%78Fa~oUir|{jfE_RwVw9h8P%8M#{z%k2J!Z?^ivJRfq4u6NM&HPW#ZU>rCd= z_}}=`=%qraM`pbl!0{Zv(5q@0cc!AFoq5TcWzFn>j{^#jgW3i0%hsV1F(GYODvCGB==>%d9BvYX#H-|#w| z-*&Jp-H!yBerNC#h%r+h4m z7wDH=nUUiSdS#4^21?M!2!Yz*rXmV1%r@vadG95r+&EEmeUixCmC!Ah5d=q(H+aKq zeP|wgx{h(#^W^*jNF|maSOB9WQKtBq0L$+3ZTQBEJR||Gf5aB`LV-SSpdFcTx@h{1 zOpXFI&QAh4m!psDdEbk%K2$GOIs+Qvt!*{OmCTu zkLxh#^OMAe$W#6rzEc37_f9CiSDPwVLdM%A$Ybo1_bNVqN#t65sQ()U8fu!u{pSw3 zD^#21650`bFjwyQ80cGX8`6- z3~fn%+Zf)g8yC4&Es~>i<{)($qhH%^|M{Q=UoZCX5hD*{MUhc3+c>Q$qB=9r;B*_YU?zpV{PxYlKDH+JRtr|?mF0)JyHyn(4DWkq3kYuKE;e5%^QMb+ey}{9{Kq?8aV=C zHSP7*H@V%Y#fPNz6H7TRjMXjLsBZjRh+T`mHAS<$Ue_@{KXI(@@D{9+zn|fvqk` zbqDsFRXK~(si*F_*R_r##W{l%%NTFv?FUH?QyYf6ns9*d0kn@^ z0wBNU+^0APwskt5L_&t7_+qT^Byb4Y(DuuZ?nSz=j>hAnmylR@kL{I4pXe1mo<{PI zW9pi!G#Q;L{V@M>*mfug>Gtm-)f>MdLUYH}Xz9kqliY?D{l@_$1nNZv91vYmL_&M~ zARiqZDPmv6hFZQ9%WePbqTj=mQ(+$Qa|nSo!mtA`X}tWuzoe6cwIY1C%uAt#IXbjy zbnl0cpt4T}q&l?%9NrM<{dt#X)p4zlGCF84Muzv!Qw<95h%<)Sx*`_9oEoN!1T@D+ zH~~SyuBdxIiC33*l-R_32MhM^W+@(6e}eP}$icy-*&1R03i4a&Hr9@PYtgd+uk915 z{jzeA>r%5LCbHyvQ`AqWlHB%o$ww}wLvLXSVSWpE-)Y`Hp#1Az)zxx*y8(~NM{KB& z9eA<;snvS19Jw1?LzfA^N`@(J5>q*dT49E~XLw_VSrxjG{v&9WlCPWI4gsK06jSE! zaiCwWnV?}aYjjl9-X6-^b#H0M_OF{ZA(a5lWnxg+lwQ&g8AoJp=z01 zP98F2*>@g0rC-O`jJGIX?BfdRyf@^Ko=Qw*2|KX?&;s}=T%}q_p6%;U?(Sq9fr%4w z))@to$71+93(OZT7wmsM5(<0rYij#?`+~2~eEdrE;CX0QKj+hOEh*D4ifkq-#;yCo zFb=!}y_gho+jO25YU$()43^WzTVQZ$5l}A4`O2^aP&a3UZ_~$)vr26vKr<^vZCQ`w^`TegaQ$hxyMX(%F$DCOn9& zw+v_2{L*&m^Q8M1uXN2a;0~Y1 z<_DMEVJar>#tQ~8>>@BCJQPzp$OL-wW!{no6|8{%dx7|>T(WLDkuT}~S8<(~)cnysIOZM5qZJS}Ov4yy7|UPHE7#`r=&^8H)~ z9E%!V{MHtJ)(&S%>4j>T9uNT#EkS-}X_4->tf?VHyS$ilSsUAriWB<^$V-_oiWKk* z7_udblmBs^8y({t6RT#G=VW4X9R2)^KR?M<)<};{L?x5c%nSuXi-MwUY4CyZg zaAT|T3-JNi(&Bdv=tTW`X;1ShQ61gB{Orf8%i&3XPU**Lc*PnBwYRUM4<3aUE{X+u znfBW2-qj#GK6gA}y(z1;GvY(G+}AJFvldiUIOAr6`NzXEa1RoG4vm+(c*A-FRyh2N z&0$Eot{t;VI*eVQsh-`~qYE|7q#T(2Q7vb>i`W&jL0`5K51#litCT%vgv0#WfNX;)vDaFNq-G$m|vAXcP(znum!I1d5GBh;7^y$aB`g zmT)Gwz}l-sO}@H?97rLkLxNEh_kATQA?BncEy=_lX0MoMK>p<=7BwX*C^1){lKx6S zLxJc)jZN}$3W+! zMz5Mzqb}6GqU}e);sq$tYlxeCdY2GvoJznklTpYRzkuo0_O{heh6PU}Jv{%19r5?o z!YyC7#&8^E*{2J%sF<^B_1y9>T?#;{D#?R6Uauo*M|JOTN~2NSy=+2x08L@mbquVv zVGC{r#Xg2gFcI`=C20u6PC>15wSTXrVVWpoC0Y$l~Qhh_9Pb-yFM7@{Re#4_jrd_jE*=Q&R91d;5-9W10tNozK@cyPPp(9Xk3V_?aPVeG=7v*&w_Dumirsj7 zhEJ6(al#dcKXZAU^~&+=NCacJ6q#g(_QAZ^;k4qf1huDq+Z~7YG!`rP!YzwU=UZ?d zqD6Fm-5A9c?g0)^&VA838#+a!Q@52ojf7JAFRE=S!lDx$v@Jq2G|ILeD!|5OuaM#1 zi|tY#|B!VctT4g`PsIxV5tb}Ews7LZ3g-y#XUoXZ>8hlP*30?J*gSJE8sni0`@XN+ z`XHOtX_V{^$)YK^9=a`m7bV;N(@Ke&+Oxm9{58Ho+Av2;?i|pGcLCOr% zhd)_x-N}g|t5s;zc%wbo7I4nT#rK1{-?UB8mFI1Ed13!DrPkw66aww11<=di36;{1 z5f+p7c$y^iy;G;l5OMCf$M<2I-ox*Nzx#Qt{Amfr*K1`$Wu5g^ZP8&u6V=6G-Dj6% z;_P;5xhw0t0nN|-)~wZ#Nc-C$N@oik!ACg(7qT{&=<@mT!;qp$2Knb*jC-_^CB3ld z3YQT4G}EIa9rrvsjWxF)%Z)m2S_zW|0`!0PGS%}@&C09*VuFsTgB+mEO!0t252 zz_G-~my;hp_^w(*OZ53r=IIQrcj$JR#J}e7a>MnOY|};_96X7AqzuuQ$O*3v4J@3j z<{n^>Co*S9@*iP%Fv8#+FCLaTUtm^%Lx&T75?MbHNpFv^n3d}`By|6X*ZF~du6#BVFNVV=OwtSYDrqn6RBPfMVcx2#Xmk70#1otKD z6rl(iMrp{5Kal8@wpl9B&rf0=uYm~@9cv0C(03dV)%FP8B$ap1{q1I~^pCKWFRP;=Esevpv6MlhzU!M-9oHm1oVy6kC+)1L-I-$HaG`P$cj#8^>%J@0V zeNV{5HjwvYGXlEHvFFp<478 zxJcJFkfGtgD>)6>)9JvsuA-Z|Om{i*j?!J*XyHNH5KNc4N;c4RsbbxirF!ej_65QK zs4k)9f0oY<;9zg1Yy`hO5ak6%ejE_$bEK&u=qTp7TKlYqPIzy`{GH|eMU&#efo33O zbUE}`T!bS{pw&7=z z3ulD%rwKPrRhLhtio6vD#)zh4{(%7+mVT?s?lwpK4U>PzIY8aqbN}HVM$~{`7X5O0 zyhY~;5-tRj2#b%bJoF&Pd3cp=-W<|#INus#RKLFwSKhMWjJ4)U=P z#A+_NTDZt>l9UIWcfb+fvH}X!C@YVUK!-BYAVK4m_{7GttCNqW(aV)JH-HDOmOR$l- z9-IE>)gR1%=0~;Q@hxuF+$}B9U~Qa$U&Y6=YA}8P$G~-Pc4P5i3XD6<6`=8PnD7aV zJq~eKTU`6y^T;&-P$>b}#c`T=DDX|NFGEH$WqgwKvQ&e^F!xI0Ath$;RT%hXJ-cp| zvt+d!OlUBAY^K`A!uW9J#mWewe9ac)J02D635@uo=Jq^;Gypn$d}T3%ox`oW_tI=- z8>lMU)HOg0!NuFO*tjRgf@lFZFcWW+Y|A$yDJ|@@eSOEiSz%kJ@V`0I6qX~2w<5A0 ztuCJu#MK#+2Tb_E7{LrW8e2ubMbpmLL!Rpso_taoRPODe@940C{ZGgWBB9f%>kkqq zM8eW8bVrL~nWtEUG|xWCDTTCyUP*rNJ(mbQrYzYmT>O$X9Oeod7_*n0S`%n_oN1b3 z$s}%=B9(7&pC6bS53FZ!@rU|7ilzYvu=~X?aHJxSN=-m|F`Q!4phtMQ+o>PB)3ms1 z(-z`6Ni%Nz?I)7ZxY=<2d-)lGha8Cx{pE_|CT>wrSkm>jdq<^K-}7~_9Yq4 zo*-!eA@$h*VR{Dqwh@HYdT`3PaHaT5dUJ?`@$#&0L%Z|e89D0*0ENsKQ-Ov zEV7!7Nla#6VhM?@j6kINEj4&Kd;IKD5K%O6!y?rGfhZZwhvlxi_xsSj^A>|XS(Z2> z<>esGUOPM>cxq%8YP^sA8H#hx4bdu8ACqWlRsN-m)LZsYFo>mk>5bsE0de@#-}cZ} zG0!$q$fK^R0U#h}F2xT0v9L&Q0y|J6kz@Cg#M_EP)*DG+9gyq%+4u>jwr&qzqm%MH(q#(R6!zR!TozEH z0hXQ*y!?!PMZXjq>bLz{c439>qpB!DjupF6%TsG@5~gm>%cdqpR`nl`-upB6O;qEn z40xJL&ky0%wz&e7!IB-2SO>o{eI#IHGa;!#R=oF0do*+XhX@r5(Ef-2s92@c_|ftB z=PRMGWj@r)k*H#J@t(=n{VQDO^Sb{K3GWo3CDIk~sd`+)NPmm8o#g7G(FE3)f9s|= zCmu6bA2*CT&Mjv0>^0l6EdZX}8^8hsbocej@lB|PHcpy1A(LV!DVN8eJdNmpm6w!U zWjoIzBtQ6^_iR-@uaPiMpoPIEZ)!-j%;5))C>ahh`^F4l%@FV-NAt6|@<6f6$_x$8 zW1zo!_&(MHYz}MAU^2CVjbf6wLew7dc@;Rgl%`IfwW@`Zl(0z(Zr6`^@o$O-T9-e+ zK#uZ?+mp!P7*EzL0y#hYG1m)BB1;l9q#PZ!{u2uI50?s?lo3dEio3fpsWVc$T;uOL zxEw|Lzo$nU%$hQ#MpFU|*E3vZT(HmIDEs{UPS7|SGRiMfJ-!uSD9!N25sl>!m51AH|4$S!@ zZ_&6j2>+?oUCKx|s-p!e?mYIE@Fd6@0K2>mb`m!8p!x&{55xpfmMLdzc8RP7>D<4M z`5A}P(f@2hGf1`-EYbMUF5^c#HgKo>gH?D@AWm=|C{c;3F*Ol#GFZj0(~5BR(LCCd z3?v=aOKgveuRneOg0}6XSYn7JQ>xrJ3iPZXMMmY>^Eh z0uc-JOxw4*8ygRpPkRzh2QKv(&!CzV$A(y9IU1;uYu@d-fRQ!aN@OBy4S-oFEDQog z|LQu(F(N=qlf78*KUxCXM(ac0S#T5O(QNRhICBx* zy?vh^zh%Q~;WuNY?5@zsN2R=n#^G5ABlP+<*|yS$2gv;8cw_(JLIA2?HlLSnmbBxbXp-EH>^*4wND?R% z@OCc&YR-U%Tn7n9^qsdT)pL|*8jr?U^XHXAZqP2s6`9d|)*&elJXR)TTxMXF<1{=B zW1tNK+=3lg^n48Mj`&8D^9^*v0+DufDN(9fVnQ>t7|+Luq$^ecd{S*wuGsu{4NspC zM4=yLP?|Bb7D6=u`w!`qNV<*#PpZDasRhs`nQe~4CcyuJ38KJ0Ds}kqGfSY%f@y;e z^zxKe`|DNXtM~~4`eD*KJd`3s9t$eQU{rgW8rG?o9;A`?ZK~hV* z_-wCzKrT6YCen|&4Jr!QG9(#(?lI?L+JIN&7|C%wSPJxPd|2t7^4CQnL}s%lKXe7V z`8lZ9Q$#C5YG6^vE4U3*_Y{SIPhP!DH|<&7k)L@X*{s$z&R6;8WiKFia-g-++uh=d zSj)tX055Avv_aI*3s;7a`Y()qP7jAQMDu~tMBtA9+YaYK-l1&2pd6gwgd%J2qnoY7 z061i+i^k*3UHJo&@2thh>gye;Tr=hnGWNp*(RECkp2@OOKePDdiDrvPMa9R0CkDrB zxf|!iE5oW?n@xkiBOm(=Z#H)=Xp=O88dJk5&I$6m^WZWg?w!rh((#(xY4aHQ$n~Qz z?w*%f&V7#1_$u^9tCK4QRQIK<0`D>ML0~!y0Nw`!HZNepBn9=Ik~X8VS#@f;CaiH{ z5<`v`9ErFMe8*UmH^22~bm%;o0WY*06hSp2M5u!Ip?mDROR<>y1m27kp1QKgbMap{ zr;`2E-pcB?uRJ_K)0TgTIHdVV(GPqcN1Ex#h@2L0@8PsVlh*iK?{cb#FALkiPLJKD zC`c7H^^RFH+4`qQDcuc{T7MG1gV!@gE3Zyw<1-B89E{%08ptGUnIfDvtiL|&5)a&5 z=s!&R3q({EEl>{+rRDLH)u7ffj2DWGdRKDzB{_wvy02>V%XH!8BSnz>NNQ-iS$EU_ zO4|DLGM$;sTRcF|GQeGivX!Kc$8UeGy8*qMKdA0#TjK#`l7aZbB0)w}-@}Rl>}rv= z0<qK5eT#S?ftMJMVuLpx7j ziPz3-aOGqyjR41sti-}%oBlCo>H&p_C*XjXH z*%(kl3}Pldqr}VGNygHc8YKo2h%U`>jEo@~xS9oX<<8<8Z4VcXLMEO(rkX=_za5_X zN-u+6(hrv7&Un=#efR^!;HuXhT#e7P1Dh#C2Fn|~8}uq4>7&M!Nj_KMV8m!8xvu3H za1d4#bHsnlX5pegVCk(QFZStU=|sl_*NAn+A6fvNVd60GN#ZCR@Yy6A_ zqO*R|x*xA+!8hSq*zCg;TZ0iBtv%ZxnQ`W#oz8vj8Z^{x&hIrw@n#ttwwUZFdVlUF z0gp%NBu|j^a$5JRyj18oU+({-O&GIGT0+=o&ldc><(+$(K{1AZ8S+CO0_uPlKppVh z^*wNI958+ebPQW2MPQX*@C7>OYd#Bd=VwJ)#GRbd5H&c37@{09sy%=w6>AWR#I8ip z830;OW8!Hj`p@VKVXQRH`!={M=borz+pW-8r2jCk#Vgb`i66QURI8nVL@JdEb(L2l zE9k~ON(>UtyF7~ag~89jQQplj;T2qREAk@^pI!s@UoHHXRPZrbV*h-SmOJF~hL&_S zc5}7PoYWr_q0nD6dj7mx+t+{Af-m(>2Qnv<1o$P${|Pu2r%Zce9|V$@)bpgV-RxGW{3#IbfHMt>v{ko^gJP7mW2rUVYVfUGE}z+nnYI zLq))h0MOnfrUs*35YP_fkCCWVsmV7XGVUT|E!JVsRvuNvl<`%gp z4huHDSbql}i(cg`02l51~>p*99l0f*Yn!P4bcySjIu)7%5RpCO?;Rm8BbF%wB! z5s$Nr`i1#rbWd;jBMrcsE#QeSQMkw-13e}-n0v?$QkHjshT6lL!v4}J-@rT_ozeEt zcbqL^4hbg*q4se3#wdMGwh_=ByCJifH(>jx%URCFA?19$B(UmhIMvVBQ z-$CHgfWqp82#?`Wh`z+oD5`@jHl*2SC0(A2DH~M__|ni+o4aD0vw=t-_q14Lq=IL?7 z6u1O$S_*^wQm-u3#|ah*M9Xmg+$>6GLTY%6Yzc=)n?%qqIOHDC1uyWeBrs)dm8l?m zbb)lraKYYD;;6_hJx-neD1kW)XYhXZuNt#U$&SH7E^MEOjbtp9bTdXET`K__2@+p} z&s3oB(=Qycrf{a|x|%K%(K8Rj=*xg~Ee2Zn$~Up68<3l=8Ok5<yZ5MS(?FYRnO2%ST`( z-v;D2boY!uuxmH=z6YJWv8qFWQ5pf5t+%3DV@|!oDa>MVE@F{>Cm0RD9ioVBEV>7D z5tWeGvtb;KhSf?(>(JtPUpMy%rb}JE8Fbv){Tls%!z#(S9#N;z-eY_T3^IRA?tg(^ z^%dm2la_dm{2Gf5RrN|*^}hiK%`^1$?)c9%bj2LFKR09o3y(*J@@*%?{;0gkdbEL69@U#aa+e8#L&LuXxQxuzA%j2Hoe`_%oBIDfBWwNMFHTvZW zf3>J#>3F`$M_kFJXsZ#Ons_%tT>I0M+~LkgD#}0AKQ4iI5jDxo*oV_9=Z$nhBn6jn z@riv%pLfB!XCJoY&=;*`cPHMKe7nut11(CD7*14-T+5-B_fIMh_S`kr;-pxxV_)Xl zcT4;|E?H{-Eq$Ba%~PFSv@g{h*`VNG;#y^YpYf3(O>}f(hULJ*(8uttPLwpw0dTyi zi8A~h)My00wu14-;e%Q62NL$qQ|#X?;j&uM0|pZTD*PK<12*4?v3}vffd!dF{e$y( zFaQ|?qOj3|{4Du}F0?*7iGLG<{@LHH1^@4!+V?MwP7Rw442@DgobuG3j!1dB!LJr_YhPzIe8lS~R+U(y%;9N?#r48vdAAAx2d#Uuq zF~m5(>Bp(3+wKhmr%v{$Lz{LU+nen<*3RS!Prf&q`8$%cNWAZ}vn$b&auLREzIO`~ zzeWgN5}MZ6D6kjdN?~w3Kk*TljIFH$b992J)#}8@#%gX#NRbN$ zqsKn0IrLXI1BeetgWIZ0cn;3<8#=X`QLzS>pb3MqWI3pN#>ws*aU;Po5nu-lfE}3d zpuvl$Z^Mt(5Ak{%D>C0{X4ND242WM!^qcDp1x~*r9w!3*U{sQ!WVKf*3n42bXQh=8 z*ALsCIJ%~YVvM|;9q9QsBmV_>^htbs(9`m~cJl@Q^IFiy-Q7%q%`?)*g$R8tH?szH zuEfGC^sztCo*7ngTGGrD+YnkaLAiB#;&CMRJ+ZJ)$weo;A-M+B*v7jg@4~EzODF$C z^+=Gp6y?t+rAfIs8zf7H;!Z7C-uZX|?AtHNepvrzje7rq#Kp~ms zcb}-=M|rU+kBhOB&akSkU4!`w8^@vFNlAISK_l?D6T`&23jHLrDpu9g5x0Q3n1HUG ze=bB6@{qo-jZL#zr{L3KV+g|gkxrv#FNn?Q(&Ij4vl&R2U!zk>DQ zKy2L3vuTOBR%*L8aKwrLrS;yP+~;bophV-Ap7O-awOUEHq()ue@CZ>>LezL}fd%wX z3RgX;gz~bF;HHJ)9u9}MOFw@9CnZ{t8{^-t>-DjS$}M5fgsVl#<4~`t8oKG;{FCV9 zXr2ne-h3=19W2qdQ_#Zd+SO|bI4nA1C{A^5m?>j!hWlU-*ZHsmA|whU10LFM+irP= zywhrqJ&!(Xa$t8MUZ^E=7^qoEfm$=p(17sX%}lT}&{aQ*m#K^W!wSnwW=`qx=E4)v zkj)p(JDg6~{aod)iUUc+Uo7?7@%WW);JQ2rY9>wj0}aFt>-=2HIgqbT9h0}{v65{OqJO*Ppd^E0Zh3!xRwFFdEWCO4;v&+EVqS4w zJ-RnvXU>)m1%#N*{PT}E`?;h{AP`avB_UDs-`vdtZ+RUs#N`Zud*ovaE~=yBO-!P9 z`bH7xuQC3XX(D4Nnl4k>B&VA0r>8JesGMOaUg~1KjBErMlTc*X;)ICs4mPCaPTzru zqB`toLUwHUWY<+~$E5wt>9*11kSNW~7cZx{+Xa6uq2d6q3{JH~;ZxX&bUKuo&K+}T zI3(AlDY%I_fjvn++s=zZSQmbk2yLia3b2|Uu24e414U^LENJ#Zbf$(b`1wCBQH85Q ziyKP6qzb>adH30Az`1lvfjnk1CZc{C-1O6lI4YA@>U^#zPY@9jp8AB0d8fV{7l@uL z$0^2U`Y(#|tPcY`WPbC%+iESejqfpI%AT}0T1AjeqG)^-;`YY=eY+q^`!C0DJ>Plx z;$3OY_ls=1M6CWo2a4|jJrP83<)$LJwby6;%E_9bq@ zXa`4Q7O-f zKhbwkE5SXc<$p0L@TOx#_s#yXvNe`Nv9-m|7R7|jQ`+4;gMMmG)#Fw#bQ5S zwLwP`4ba6-5M443IOpJ9zr7c1#Nv`nuLEz}yev6*ZHYyv0@Yr^KbTU+wAy%=KR*%U zrYhv7dC~!`5WTv@D zT}C9yN423MD6*pB9T%0*xM|4F)a8_%Iy*HE3rf{^$3eDjP|sIe7|RG zJf~C?Z(X>^I5v^3apHQjdDac2o$KP8h87{4pQZ|k*JG4uw^+c{6q4>LPCq$*=h{<~ zR&y>Y1fJP1q?@VlG%1}qW-gtx7pVREE1sm)^&vKp57TtfDL)imGvhBKi{mzd2Q7lR zA>LP(a5I6u*zo`O$3|u(bq?8i!Rfp|@2FPyR8a4|F>bb(`kbQAUZcT;_X1Tu;1Jm_ zME&LE6kY%?V1jT0l6^R6N!Gu9cF?c!Kh%>6botG!sKsyfCnMHqM||J_ukhffE3w;*==x+Yq?&kS=?bjR8Ml-jqvc9MARnFAnQQ4 zuz#1fkm+xBTJ^zjMG~`b=*F{+TuZs-{BBYuOAK!{3CJo&QL&8qH5{5ZT{Z9`{DfZd z(O27VJWb(|bq}Rs#=dgVg`y?JrZ2T5H90uga17K8PkB$zR@>zq(ThTx>0#WfH7{6{ zF4E&%LBf^OtE=i?nbeH7W+*xHbN|uo@ExR6fh(;_>6t$5&Hvhfqs#(SmdrqXVb2jO z`ULJC#n4Wxs=j|9B*T57zxQ5?3WW<%#A3GM5W~cBpltZBw}^=zmu6qZr~TunXB8gy zk?zL+2L8>!#U$Ycap1X6dJ0?J6vRhz%4NhUcU$07{++WaVyYB6q)4s=QrV>Z&hFbf zZnBa4QB%c|cmeAM^97yt6XMcoUXAr9As@{LKmA0Aa8s2J*yDUaH{we~!7me$N91f}g&u(eL@!^Z1 zYwi4=ObbwP<3YPe!nSkb?vwL<_@^c&C$t~Qg>M>jlKQ3q)YzeKsNmOuiW@TkC^M4~v+vdmh=)$|ll`pFP~vN7CNYWd2OToI zp4poWkQ4h5o0xw?XW91G;?&7u%Ri9~5isW@6D#?OM494nzbxO1u!^u?-qUDmF*%0J z*}Q|8mOT*vyn}`u@;k+Ist+uJ#7M@Y@7&CB!n^3epr)Cj=SeSMbY3Nzdr+hmW8c{9 zRH*&SM^qaCy{`Ppl@P?9-?S(`Elnfh!NNh&Y-6xA>Eb==&5!Ut4g&Jv3miZK37Dr? zqUk@u?9EJb4$&S*PpHTE3}S=H8+wa9NO5y`Bh^Ak%A0k_J{JhrQmdHw4SA-F zcIlpq$%%ZPSU*EbXfLbaQ2WRhui*^@Zg1bB_3tpj@yuz~e!?}gU;G640l^jIrvgNf zvs43NZ_!VFe=>t4ky=+Q1I4A_;G4$QEg@HhK2Q%mfy)Cq)bZj`34sv*XhO#4GZ%@G zJBmGSo0uxzUg)wCmND%;7t+-FPHS*oumee%>|=6s^`c>H{KQ1}FEn zUoMcm&V4T609{s`VRC1#i@EEV?ZkJVjhqbbtqT%2*1=&Fw~t2{PAZfh^4Z})%U$o1 z^$(UU{7e*@3cLgQ?Hvh?RblB-(&Z~MWD5}?*{l?r&Ec3I92y-XL2Run>+r7|<+eV5 zFJ0X#HQgFSxCvJ2mLI6*MonYN`r_Yl=^UMrTZun%^ zF}62ba^f_bhhH&())&8Rq5XFs7G?5Jv$mHtarI#zr*!qqA)9Ld^q+yae?si#(rgL z3X!6;#hxKYPvnq|zUJldIx_lC$E3o`)=3w@G~)tER!N2oMM9dnJX2qEBB!(_^~lPO zb*B_JOB_@oxH})1&R!BkjGx!An=#qFTj)Z=sW#|#IBd!*-9(R-<>#Iun3Ih%Gf|Q+ zG)xTx&pGeAB;Y$+E{X{6{cdg2w{R9}U)i7YV$O^*URAu`GEAQ3XTE+elRb2&5}dF}7M z=KS6%(~?l;@~8KlDR^!%XVo9Vnpb(k@Yu|3(Jh-(gfqpaMpqqj)y^oc^~T_3R-~aq z)jwHGlZ#K3LpGT)YQ5`0=b(^HSNcAF?a&EMBjO=;Ss&q*mMo`|{T$%>Wr_HDpKb@8 zT6PpGi@QLav;A-N-)$mW6vvEcE(-3;2V|3Y)w+cXb_)vKO{K4l=3Jx|5Ag($&PNm= zNe33j)LNAeQ*(PamJ}Uk#tm^ek*X9$P4^uT1E->M z0|XY1;CO&y+g3b65L2|R5R_-y-oy`UzUr!`tM-)H#00ds$0+>MTD3W@gySlP%(&s^ zi7oiQS)nl_dr#2Op%dNKqDb#_(BRkF^dDhNxyCdgG>}sx_p1XsmTAQ}TfDzM zM{qvCrNy&WCLDC{qa;6=mnQw{M*y?qYVh;dCESsTNMlHyzX`}-U>^+a{!6tIS2G^C zZJg+W5wb~{9UcWQ8w`V~sDZ*tEAp4m6Oazhk15MhD>V33HDMk55i?i(FpBahCpGQ- zr{oipQ(uRPd5<-h(cV?xeB_by# zu`suE(xs`#1gFSAI>oc9=iW^F)7Ul{Bgt<2itut!a%;&Ce11U<0*>ady5?K`&Tg~j zd6$w?lc`DTXXjV=V;m^J&A9_TVGD1M%=yGyiL0F~x4+i~#s`Rg0@Z&qdB^bjw~ATy zbYCSkJZJ|6+z;Z}=@xMt`g+dN+$f2Gf+z_A7(QiR#NRXJB!gDV)&$=bA!95T`d7)a zh{;9bRl-uu8GPi2@_5f-Mp%3OO9_%xE15*mM-$J`7xgvP@6eVZ&CrN$=Bizn^=E$# zIiP6j=kqO%<6bKSo<&7?rtX>OdGB&*$v*CJuk`SF7Ckjmq$|rA9u-C#?M?mhhWNF# zIe$h7^V#{Q?gy-{w`36?&j$Nht_RML z{&qJ@PxGcQ&DFmxJmL6of!-`pc7#Gna*lQ7lKsOiu{Xt`ri=ZQPZe&U#(E0JJ!j76 zQl{Isy`X?unjfqWwwoR9DdIl}ds#i5*jUf|wv`qPu3#v0P_V&hog1ghmMm`DniXce}hEXD)02`Gz(j#{2NX!kn@UXtsrE zt~dj-(4QAxlake--C(fdabPhk`GxeZA=56w`U~m%%+5fwvTouHM6a${eBKZ0Wln=~ zgHN|!&&aNGtvZ|EY35oVeK>sspqI6r7eD{R_gHcs=8oPHS`?3t#nUxqax4WWx>mJ_ zRIQ%EUw{8gI@Dp2!Y-3lKi`G=(W&)fEu>M~@Nc;Af)0vj*(nj<|rYJpPIziX^7J%;#nzDq(vbaZ53=(7h_TGpSjw9 z$8cq=BH^%S{e-`r8a95v^y9)}N@wp|IdS!Di~(0_j;(kI!KYglDmZ8nGFVRjPESqO z|5vEhUvK047cN?m@tXWRHI}&E{Djy`6{b@1)S#6R{~H?Zk$R6s#Wq`SLe)1h>3as!)|j!k#owNcM`-}8Up``i!r zez{+Gc-Ee4tvSXV`5R--718cWLer6xrQ&b+TAY#+2?|q&;iA}Q+Sc7MjDLpCOW$HO zT&^sM|L#B>k9W@|wZLplOv40AWe!Dw97m|ge6^*X)3ZutE^V4^h10pT% zeL)58D-to=()Y%OW44;IdT{k24IEBOjj5d!q24QwUQvQk1Vgsey_xxbgaRALx5J!XF~`XG%1)-~P^1G~ z|5*5(eEA`F+BYUtOc$-@J%=@hN2n9`V=^b!6%UEg8|HXD{Olh`9G#gIy|P~_tI=Ad z4F1rICL7SRO+o?qoC6ucUX<}Vl0Kjg!PQ=eszXnIH;WUT&@RWFl*gebWO9A_Dt)bf zQYL+ZBJDkKZHG>{*-$LS@TVeq2PO(#aa(jjO5f`5H8J+g99}y?f`ysEqWIkDje980=y8?hTPDfuRe}!vFcNK0LaJ?AyA6MFF`&wH>{Jh#r;R(B ztK67tGs#twvpYtPO~J-SI-RIBR{YOju`D)xs-;q-M7_hPMoId`{25-grb?B=VLvzD z?8Qk_ouAJ2$5_)XBX(!C7#t#dRg!P9&1-Q}V`9f#X#wam}m@s^m{I$ z;$7q5WeIt{(^Z2u{HS2EiLtVbVwR&aRW-AZ`WAZL5jNXODp?emB49@*%jjm$J zRcVP#7#3WKOy}!y&ZB1X1C<0O8=nwTB=qY~@y|cX4z`|gfHX20g{(eG;YZx^eT^pM zzEhxvS90(t^Ia#ihlwn?IpK5xsLKW<`1WK%Ll5F4MSyAye8Nsm%(UkK+IU;=>u_Xw z*$cc*$H^`Zb1H=32)ZxogMrx@G$2L3A5)wx7Z6CS)+UuR(8>?uTG;BS(;Ao3Yl}L< zE-WdP;f|4K8BV>AwG3J1r>1z+c%?f$OIZ22M#aRuWo6^w@GQr*)Vgn)X}O~d!ok^_ z_&nOogU7@zE!1i$g5mmItd&2XnMJgBHg2r`EJSR@d6j=N*{e`z@rkwI#);!;shINt zk{L&6diZNa4UF#b{SWXR<(5P65!N7%3xYKr4l)+_H-n-mI?6dL0K{&AmKQX8!ruxV zma~aX9%KDk6F7^n?f-((N2C-h#&Dg->GTf5W{tc|?2_pWhm*+j{!LX9h)K{s&)&6#irJ{e-26Q0wb%uVyIs5JmO-jUMaZJ$pfhI(Jkzj2?ftH-zNmviveYR;VUr zH#WGs$S=n!H7s8+*d!XjzNn7()MP~_PuMRgyzs2Bx(ISYyOB;nKG2`LacJ~R;> zj;W(>8XDW#c>_e5qGz&ECRnjPBIJOzkhZA1yozFJXuDm&6b06Xa92W#tUR(&z9p|K zY)SD>d(Y;t3R`-R2X;O|27YL--&~!{)w(6T-D)4hvEgt&$N$ZAaUclR0nUm7Qk+}9 zo0y-;Jw2qPJ{^W>m^`CBhwLr9%TnpBN_9->>(YwG6Wo2)_~XMI>a5y_ciGJMMgW^3 zTY@-5lQX(KN)dOL1BO15^jo zGZY)(J@W>Ey2v6IUdh5tt-5L_n!c$=z30o`_o7&sF3Kr(Q=JPH zZ$ekSCQCC>*Y5>R#-H;2Iis9kVPq=Mz&?FHhm2_@)sK%u4(=71sl1NqzH=V}zN-xZ{?x9%~F_)JPj@wpHfMI z$HOGDGn=o$Q&)F-p6nb$zt22Xr`w}z+=7>-v^!*kP1U-g*By*tKa*>+Rk6exjR|#rmCptjgurPqHl4eua91E0YQ{x>AOvY9W=3t)VC z#l~{HF6GjFgNzXetsO5BsyK5d+L;vK#cB5u8+-C`2`;fJKXSd+vgeY#Ic&1D_-U{m)E`D}W1*(J;`&#kaWt0|R!y4NeoZh&bwDVgW^xM>rptuEuREBNP? z(y&GPc$Qyvb0=o!e$WI^DpB#-w}vWGvT+w!Kvlhp=a7tP-n7I_%4x)G3&)bN7YGFC z)=|bN5;x{ox zQ8YW|0o}2xruLW4VIIZ%ve_N;Bvqp>VeL?^c5EhxA7Uf{ATBb-<)TQJd96e<`H)8` z97?7^np<~+4Y{u57RX4v0gY{b;7hBZH&K(}xZ!-DRo9nblGAfAo7Zu+U=ng0XZ88B z>h86@33+{PuU3}Yke+l_Mp@&?4fKNMeboj3S>+k$@yJ|e#gCwwkvg#ho^1R|D|d@2 zWqx|N1$~>H=cTCrEY2qZ$2TF?Qe!iSyWOy<27@gFDVS)Ec;xnBvdv0h2tKb;kAO_} z{d)`yzGZt~uz^Z8S46JYhF>C@Bagdss|1=O!1B{sx_S z>3MZAUqm?qy1>r3vIDPx!59<6Opm#isE@bKJ@7Fqsp}FmY)d5 zr_c?*+XR2&)BcWikX%JN6T`3b1Kr;4+yF>h6H+)dtEKQ)+otC6Q@VzpQuuyFmxfol zKTGV7YJadAGi>3AbCJDUE-}MQ%KziFpnFat3+PWK8*xu^5bO4ETZKyc3P1IN%;EeF zyv8oIeTJl`l8^7g<;0QRv*9(rq0pHB#sfsm#A9t$PhBH>-HtV_q>cke6ARCxsG@VM zVABo0ct^-MwIhk}SbqxtVP$gr*0}|SD+QteHkQ3+sl@$4^!BQvfoZiW$7NdbmYN%nup} z2?e5=rFblA30k>?$D)b>sKfp7WL(Ajvx6&IYa5SPdK`2+Xlhp}J*PGFuataU zak?K!O|pD@YNGLU*mfU;B>d1L?IO>1c%1kqrORI&s)!EO9C(5i!w(#S=hW8Qhmgo` zt(6=b=NPVBkDMW+y;n4*8oRu7xI}NZ@=5jjUYnYHcV16SYI{4#v~SQ2L#i8h5_TB5 zZr5=5B*OF1QNPcWP1ky*v-Z#)Xs(FbXvKA$d5JIr>MNd8*^M-{np$*}ip;2L3>AqG zl_IsVC*_c_JEqlfjIqd`D|A*xl7Y9DEKK`$ts1P)vY6@?O_JdPD`2ci1&X}z6;Cf7z`;7F41n@V z#UzizQtc_43NGRAtvKo1_6gs8OYvGM_*V3wxvV#II-h&}^y>Sa;0%myPU>pCq| zG6O7EacB5G!xP^y-TqJ)8 z(s1~?%H7+Z88q4`c3M>1=!{p$Ni@thDGwQTD@6RLg$=)LgCGWkPYQ!v-JURc@&pny z)mzZ{4arA@z}@YNmTlj6tr&*~La5#2`X3~krcD6F<3KEa4K9R^9Pgk*$;cnk~6lDx55lBy{d;we?+bE~PL{GtJsEZdE5tH`*kLZ`dJ zb&zp&1=^~MR8X*AJ;U5g7aJQa$GOKaEJn(Iv zrw0b1iKaUB>PBk2<}?A>6#KdfZpB<<`h+ZW~|l6w1*-6+i-jK#D@ zt0CP#gd~#~$Vic!09eQPMkSHV4v;YWNl zXkIfoVc^jFhX!7vd5Pl;8&IZZT6S!F1^7NAI(FA(o!r9wh#@@Z#;$yejm041M{!0F zRSgnA+vO&L?qNS;70Kb}#~{ei-#=TQ>QVBmprmDle4_6e#V-5WlgBiOhc z%?cSdc;hTZ$eu{OMz5PzHD3oh4D#=Zs?+x;5k%3kb-T%{92df}QkA=c-KsxL%*U@$}3af1=>I&&F;DLmk8BSc(?DnXPc2?xI6@#+r8$>GAU& zspx&NcXxGI7iIglQG%C!d6{(jCxa$-#b80&nJ3)`bJs$4g)K`-w%bG$l-x%Z&9)t6 zV;VwyIwXuP3mm@I-cHvKPXaWt1=VB%7kE)ZR4yTH9;!1$l@+4F`X_lK1dtr`Jz;Vg zrF|7wG&{PTilVNBwgx@J`4IK5obqamuYoL2_7a>l+=iER4eF;pwUxgDs+RJ}d6A=7 zJ!=l_f$JovdmR(aKr# zVE#u>->cO7k}`AG4vhAy&A}wrqvJzi?*tva(pHo_XDd0Fj8%w$F4w8IZoaMaIHP`Na=eHQH@=N+G~pe9@e6j|XbwsW14-!yY z5ay1HNOJ}`xT8__eNikx-AE$SQ6jpX$OtbCgZQ#q_`RTaqIg&!&5s*Rv^nLyItDQo zi~&L1LRNfnF5fTPK1L@!U5sOwHSytvlRl&?1hu$_``O6AStU!BL!A@<-cE)O=shjLm0Yya>A}fAuY~)f5o8A_$VpUPaz1@qjFSWRvT!5{sF=IWhU0S|(>AJ+GS7OcY4WMjy*zmG^*ZxG_uHjGv&kE^WI+X^j1x9nk@t{teOurc)S)l_@vJkqG-ufG?iRBKk!-D z1YC^?CQF}rWN7X!MkdNkR(ZdY@DXYqJOjHn)aGrS6w#w|k`sO(bld;x@~%_PZO1kf zJHN@$J6uM$vx8St;BZ6*4t-4_|7;#DWS;tAy_@u#36*b^8Nm+WU+^p(ji(a~K)^SK z?=#54QQ`gTl_-ZTTk^?54Us|}nLJ2L*!K|@0J(F#4$Und?qI6mGCGI@%4rJ_S{nXG z?$bNm_^tsWCa90HO_tSf$rKkmer(vGQ>v7ekWpM9`UDsRlqPJOfS}nb$|VU(PL%zN z_hZ~~D|Pj$;r)c-lt%g~KgKuoa&q%0<=H&FURpb8j(#VJ-XLDa$Y8CgiRRaqJ?-1a zv^vk*+6e>eXC7(|y-56Cevr4FdA1qd%)WjtV!5RrgJF^Q4U z?xfmtuNY>0InG=d#5ud1HkJTr_MyZ^of`at{<`cSOT=7V#HD=m>utaRi!^tj>SCaI zkNrrl0oo^{E1xh4Yw%OJNj0zM6&JQbyhb_6BchA+4&XpbfzDdA`FdnYY6?zQwxe50 zf$Ax-+u+Q~~q!&de4mlh^$x+jXUu&-(MZqoP-s;nxsLZq7 z8vlpf9ZlAx>kf6W=9jU;S9>`5E0BCnDF2Ri;S`fAMT6K9INbAqXVR2=458Ae$*+~Q zKF)fHOL!_YYK&|zK~5|W)y?{c`U-PBiA9#f|q@xhT#2SKr2v&suWa4ipi^ z9vD;}h>z~BMD`iV@b%v8=K#=UU#pH8t+f|C=#gzG)WffWHjApNb&5qmTyk^E&0G0+ny$3w|4*|lPeK|n<)eAgZ<+iy-O|NPv z3UWB*80wjbt>>orWiSiy-P}2xXdUZYup2hbsrzbb*?4*}0s}w$;3Ui6M0fx7Pv=V@ zDv>?VApAwf4J|-xDka*%R#ZDx8zGxsFBvO(scJ47a@sf0Z3(YxOle$Gyj0W~r*(xIB5yFG*a*NlBcaFAk2#gmT=EB=1= zy`3x{W6faaYM1g4KXvft=jII{xpMKuRucNxvB)?Wdz^%t2{fE_*ZKc8fv`et`~N|p z;<_~Jzt9(^ibA$#;-aQ5vW%3VaBMSv_j0@nm>HW^MNVz!^Bf_7HDP79I&v9R!tebX zp?@vXzYzcb^|cfc^a9jEqS3z={I>L57k%=bS5F*%e4^)R43)iULKOjg2-uWu_z~Qq zA}=q`O7MKMT-_*&Mh0mN+o+mt&224B&4I3pT&%BIQJ-J`7N}dOrvxR;78)aG@0GS) zZ)9@roe!iu+Ja|VRt@;H(OGumz`OxFCUT+hfT%m-76&Pi&PY=#zOKwH`};9J|7*Qp z!u^siIx6ZZ$xF2N>VJtHfK090NW}(z_~#8jW&M}I_5jl4A9d`=+R4AHY05`52g!tX zex1Dh{F;``j}SrCzcg|R4+w);q;8V!f4OI;T*NQ{TP*O@l~dX2_roZ?0c}`>CUO7r z5C5)izoy`eIWVt`tkM4K#DI|*Y;P<+GWU_O-fV>Vt$Yc<=pQeGD*izgI-8=3N(-mX zZ7(Jj(ciCc_5`S#;-!`Nec{(blCdwXf13;eq*0LIL2LULMj&tK%>PN* zk|TeK-R~zgR{@lrI1MY={J!#gg8#~sKp^MUo51_}$G@|e(i#=w-`OoAE$!kj^MA5t zeoyXQ-ieNjBLOfCM==$k8T{uQDT9Dn7~PL9_?wD5(C2DCfY~jzsQ6nIxyRCE0?_B-^*yk060p2ALfwX7k-)1d#FyLBk!dD zUe852z=+ommK*$@_%CJ^V*(2?KCcWcU6_I11pD8N+6f0(Cj{3Z_=jZv8<*=DffeJe z!p{6lQ-4z5nd(zWRmPVzj~71o_x;VEfJe$#NdDyoe$6%q;32_!RHoiU)wP}v-W0rxuq;u{R}0dT zzD~ZrFinhF=EMai$3Ots2W(>*W!!(8Xo@6SlFN@z!$}@{UHH>EMO_jK%s5UoYfMx0+}>lrXrnIRPRp4cVFiFhJrjh)wgt|fPn6VE}VPxja&^%a?OK`^+k zf5V`#jJ6VV+-|_OB{@Gj6JO($^fjg1=`oS1x0t(R$!lTAg$jF5t$Y*BBPD4tni%DzN z+T6n89EXI<%p}i|H<4>;Y3X{Xf46D1#LyYZl<16F<~8K10LehY`Z0^-DG*ji1H0_L z$beyfza{zcy@7m;zR{s_F7g!meIfVF(heu-u!=6Di2e-T9#BH2bJaz@6sph-Z$6+O zutuv_He5&wZz^hOoy8K9Fh7V>N`D0|NMzXtnhWHb%CfSSVXvcT?jjk)0OTly-BHLG zBMcibQFf`le81(y_c5xN|D9YMzZbl*MsCX1G@ftv&zD`9Sy)V#;~nTc#cE0AgjiTw z1b_uki9xSuG1d^ftTUTD4Vr<$AR0nIibF)r{^Y{9qwc0q>+Nmdq zhfTCRWXXWx@in+N_pI_E^&F8coFcJ9P41Gn#N zd-YMB0{h+5EQ2KV5`dm)-5W5isZRy3opVS*Mxn_-#FXa*sVAD8NZZd-FVt$AX5ldN zczMhB2OdCLHH5cP9O$M*t8Sqcg?urW+!)j#&u?kbiYh4W^u<1m2- zz5&h*E$?b26f(o;^K^6WL1%gV7HNqU36LVvW|H*=?G6q~SW=m-WQ`aYRFB}3W>IzY{WKo%-MY#=EC7-EMJ zjX!@Iw$8NXFe94kq=;Hun0B{5b}Sg= znwtKC_Gtp3PzOoj*ijY)Fa)wdGmNQT2C@uu!y7y5uZB`pxA5%*km4l%05_UGFN3H2;CRJ_{oTvw!3W0~>RI@Q5}Xs9^l`zPC_tF&|eW zA0SWyW&qMR0{><#`h=Y#95NoNTAnl4N~h|nU-EhjRSvMf;Pr3xNZNon=eyU>5D8Ed z7Y)i8Bu9SC-+eU*{2Gl4FKQR+F&;?~eeZQ>Y|OxgR8AUTrR~xB%z**ws8PkzGH-uQ zgF)FhsQuAiqMB#<>iVQXU*J{m`S%AU2PCnH=6@E}30e165_O^_1Y;Ax(Pru25sMACyd>y{sTTzPHji39OkP zFS{W2?RK?i0?A!f@1Y^RVy`%E?WTAkGvzdkxt#`ZvFWI~n|w=YUdtW3m?_8~S};cg z*h!b}A9}O=CL|R18TfpM((*YT{FJ!n08|Lr5%vdMZBTs7a5p$zc)u|W{l(M#Mbe3C zC*1wK4(rt54ey{#%@u!rySRXfY#w<|A__K>*m7Sw%STsZ``27wA*l%{JzI;X)aNBW z5Ek8ojJH7Fcz&su)%}-^uAqFgfvKUQ z-9WV*c_Wr#WWL0i__VE(k`v!(0o%f$fs93*hLt%0J0k|DBiDci+Koc{CSYrn2{9&J8I7e+mNI!r8Tat)kq#>(F>q$Nl z_XLTPWL})?+9cKK(; z-7vY4C?<`2oU4}bvZ{{#jAr}3q~@D%dzw$3x*Obn6zc?ng|#XSiawtnv4T|K{X~w} z;4Us#$FrOmu!T1M^p={vUrw z4hmEQOYVCFB#@O(p zK$gr1ms4o_%V7V{$2GGc^T;ocDSrNRTXC*R-z$2X&YLY?tw_4giAlZaYnQ$!(`bI4 zN_PwB?CaCM7u39#34}dtJn|djz%Hl4zD>=w(bz%Bi)%+MlXJYQgdKmI9w}s zug`g}KOTSS)U>@Tg;V~^FjSD+D%HYVtw->>T4BcJlx0nEP_(- zYT5KEG~|Jg4wo4X;<^4fRoa_kQs_}xporiV^3YuFOQcuV_0XMKGN6Zzj&_Ng>eFQw zN*jMSE$%Ud3Q^+=dCCD<3Gvp4QgB@twfr!z=Cc;u8VE<5oMR##SP0yEwojIHay&fm z$u_$rS>Z)L`BaHqmW|o>^)VKFa`T$ysk>Ut;*EE0C67d0-Ti<<}mPbyYEKNHjwMt8(JH zq4WztQ6^Zuj&7U}!5?-lTWNHEtYbcZAzrqP;vTU5VhxtAZCD$U_+&Awt|%IMU=?ZI zm@xHBa4`OO_EH2mQY3#W#ku@z6i~ms(Y!fF$QDimojHQmklG;&AuTrP+El-%D8?-UE|(HSS9(0Frr9tMUNH%tdq8D8!Bs9#@URYKcLgHqnrXlZV`8i4 z5g19-+`8hYu>7)B|BcAp%hRa8;Q3Rt=C!hoD0lyb2VzB5UqWy(3GzgqfCGnlzJuI~ z;aLVZHYJvS1^4+zhQ0B%E>3uzbpzW}(pKHD@6+YAQn@@G6%Tp6{k z(0(9@9_7O(vkev;m`*bhT!%k2xOctg9O{spezgc0z^**@zfG+#H>$q(X>fwa8I}dz zYMr*kOpzFt2g9UgRitHI3s0HRE}mwkd8B3iTD#nkxR7OiSGz$wArJlgUUozIVTrw6 zxCOw%o=u|od38NDipJy8i}NPjF11NZ;-V~K==uElRUs73MYO+m?(LJ`N&0m=2k_11p$JGej( zW|;I(8Jfu-$sNCuAz6%0N(9&QD>Q_N>j60^r@NGA*KQMz?Da z$z^8W3{IOmwj=KHXf=f|^v7=v%EtvQwP8U6vJs6dMQ5KY$CkR3pdSLzpPru4woc++ zC_8n>3i`U7qj%&T&H%5Qe(Gh$FPHdCO}E^50fAw$yoZB=HU$)V+U|EkwLf^C++N4j{0+JUo$h<6J#pNE;GbZ-JlGxFH0#tKGzsx#5rIMeG z0e+>1wDskimQwS}tlnj>u%Z4J)c;JpeI3vm<1ZSfycVYCZRdKrwP^dk-(jkBdHVx@ zC5zK!?MvoRrS_POo;*;%Zw`W9E9gDkS1#8q6)WJ937ba)5-ar;AU<;BlM*&P($NI0 zG|yH|ctqP`JR5z__#oSdWTKL*KR3FTwZTQsovXS^a1=2l4we96l%K@RURwZT!za(0 zbaySwp7%M(T9<(oB=R*N70P<9yWGUt2t?omCL?UH1@G8=M>kV*Z2}Ma7X>VLowOad9(0(oti-7g?qq~^boJyjbaMWg;j_4o`E4# z-Yf5Ojl@XMZYxtJ^0a|S=eaxu%;1yU(7eEvh(b_S!`-hLFj(AE*>kVj3-AEC-S}X@ zYDsd7aYP@?0d{dQTs-l_L~UPgh5ySaYIqGj+Ry}B0>lqkiub9AUj0dszXtQN;iy9+ z&F<0LEC38yM`Alx;?h6E-azR$r+W|Vw&tkpHhq9Ez+FjX=o|w9NQUmY(_T7`Zdu=v zHm~+p$C_}L*7Vqy&grK>EY+h(Ur_a}cK)HY9#2-L#X<$M)%Y{r?1+Pnq9*y13Bjhz zvpUP#Q#vUk&x6$K58GC*2Ad2Z{TU&U8y>RFb?x`Mo(|8@H4SqnX=e?)SwU=1Qpa)^ z%iRvmi3N6KiQRyWwZLH#QM+z&4ox(A#Y9WMCvc}Pzk{?_xjgH&2A!6(5zI#bX-$8V zma9)4;g*zf(`*2w7RlHJS~b%B{Em_?C*jz|&t{dty{bkSWKKTyr|Xlp(e4|SSav(} z7+n4KagDa$1RA;k3lQF5W!PPH8WrD#fiEnZYmPq~_qgM-vMIfvupFle$1?)&_NPES z%7|a9)yOMP_frX3s%CVAt`~cDrqjFDj{qn1VVj)GxhjLCehRj1CoDM{JACPXfJsmb6g2!A{cs{y9OJwQZuu_u z>Ur;rkagz|^#WI_nl*OD5#oLOLVE6N!8;eL#DMW*i|;uG)`O*Z4BX#1=iqwM=pB%} z^rgTm-yx>v%g_n0hWEv{Y+Iu3k(zWse%NeRIPL?udG+8=GY^d}B}LL>9N%y}D^q*C zN~dsfzLqh1yQ-3QcM{$fMA&nXE6mlC-dK29IdR~j(AnV?w#`?5wTSvV-=QxScTz@X zJVts|jO~V_9-A2Q$D~6vik-*=dk>ar^)py}Mvzlj7%6k*R5S8=tP^oOOPz@&A0lCT zz9W3iA_45HielD-KRZ=8-(-ykm*(o&cd>Jb#mBtCxnOIWGhb^s?=Lgl)(Zypk!G*O zGbg0uA|t_~F)+@viqOr9o)A0brf+*az1;L0FH*xkyC0!HUpnC=*;eFmJl*zw#XF%+ z$g1&J{Z7p*J=gisYA*iGiU!%Z9j^s}EhkXMTrd=U%3LJ@avw_=VAi!t+@-(UAsw;w2oy*-kOKR z4{Ob=8;pCG&0UI@H|lpy3X1jF)ve%j{c?M|-6#y!mCx45dOW-9Jr7!TWulPv;=QKP6wj|d;u{rO=^7&(-8x6Mvpzldb#6hlkSc5?axTgOZYZ{FTOOl3r0(F0u}rj| z!k=+#Cadwfkn!>fb7;+uEHZY@-2T=fBi<3X^K@ax1)^E&A!}31vaj?C7OVYl8K}ap zoBO223YpQtbn~u)q%A}t0vjK01lzNxKh#ywrfSyR!f zX}4$x{azMg`cR>+GiMODXO%w5;;3!hX5sGQCY5PtGmXNXs_Cr#rFT93_6K*WOkz!* zQ6+9m`sW+%3Sd1uEy^m_w<&--w9xT~IF|{^_Ul>m%Mk~yX>ccQf8?MB+`Bt!l)Gj) zaG2W-I%gy^_njjkWK0;|@p8p`*@{=fyLZxkpS^vesV}Op3eIdr+{y(&vW=44vK>?tOMwrVF9zb>NUA=Yp_Q%;{QYY-6&$krWt z6SJd1Me{*yAx&5g7N_Qt(c&Qg;@cF#d8 z7KWzoTl>~BSzY2W5a;*A8EuOdSt%*0dLy~4zIcANsSxY3Q@StG=Tk=W{Uz*>o`u;stkgyY6i+^2hZK)wx?Wo>z|*n?DW5wMF;;{M<-Gq~Og#uh%3_ zcqP1;RFJWFJE9WzC)QHAC#9CuOV{&#Z)Xp^LR{t^M@?C=G<%g4#$)1RMWvu&`Uc3;@oR#_sZu3i;2t^d>k3 z{$wCKsrk&Ak^72nMEF1bd5JYJkLUMI`+s6-f5UZ@WWd{olnjagw}YgCpo`(o%&*}1 zKfiMgsFi#PA?f)~Gx_V965vo?zTNppv)bR&LH;Kxk;0Ye_g8hyj_J)O%LjGs(T&~= zPI$@pC8gdt>N-A~?uIWZTb?U{bLe!A*J~pibI$YN5BeavV04HzPdFAUrUrkj$sX); zcE;mog)9vsA$}r_GIKAN)ZBgH@Y!g2=Oo|Is1n){4Y+#Nr8e^=v?mo(bhZ6Lme{?` zW7!ZDh3Rp_<7MR;Db5bb{4X%9s5ehhbFs$_crXX)^`)=kS`d+dc^@K{x{b-#EAGJ9F4SPhIz3_SW{&eX0ss zD-E!m0IyW+?ex5;OdZ|(;fxy)W(&qhq*>Zv5^n}TNr=sq~ea3qV`@_Qb|K4YF%yz{IS zA7nn`EdiGGFu{rK{Gv^OOehZNSsFsBJZ`?t++3a8soHYdke5uWdy-y~h&g$>phX<& z%wvK3l!I~AxNsS~LT3SHvv3~INy(R6vZGNX$^&oOOyr#x4b@hFt<(~-D?cairHs2_ zoDsq~W>Xzk1R|*`M_G$CVf8KI&tA#Ng}(M-hW>465)#?KkM{*me)!M&4$R`8VB&uq`u}oDepEZ{8`j-# zG02r3My3)XrxbuLt8D?-`Prjuj#|zJH1JPtzsYrTRyM~DL-uL^qbz1{|HO3}x_78P z+u0q-G+eX?B&Ul#(5A0Ir%&eB0Kicg$N9ZP2Ix@~mOxxO0xNRT;-1Y;=5!D7IP1y= zFdWp`F~8f363xNCQVcJnms2IST28{_F%VcqO&gw>X^^o@wi*;J9%QsM7D1cQ5gUu9 z-6WRZc&@h|#Vp4N1jH)cTtT0v?Ci8&7W1iIvx}*%S;s2|PELXvq<}NsF_By8#&Lf- z4hB?Y<>7R57`oUV&f5ivx28Xyan%2=H8wILxl=n|CR04Uc%ae`AeD8}kU?YCx)Zxx z9kp1SnsMT+_r^P2*s2E78{9+37pW#(fozv&(3i|PM&LC{QfJ+W5sPMP~|IKaM}* zIrk3gm5*9KVh4e&gWox$A_|-#;yf~Y8&dBx_u8W4i>Uh(wH;-myR6?szb;uf&=m5( z6>Aeb{m~Adm9Ga^eINE(x6iRWT60&17itgJ|L}O~tbb9|kq#VcAHl;6)T7|s`LHsYbja@qZf zi&q%i5i<==kx=1t9k=bFx~_|HTUeH-Px~@wEP$pUYT_#Dsr%!ngv$N#Bn6bN{68^O7c%CX#%~@^!B8wo( z!BEh31it*4#l@W%trFB+uO^Un`E^z2rctgHZbZ}(jKPI|px@0@+o=38;&?`;n+MKN zAJ`UpDe)pw)mFV~FNn>!t@Su_!*4|@bbgHc3W4xCT-xig^^RP!ZerRmY7&PvoI|R6 zBoqqV_7P|OQHjo9liUVhrhq4RHX-htboK|t6yA5ps7Qldvuy9*2*$1)Ah5f?1vZ3G zYy@4OC5tH~mWqtT4L0j9DHY9%m%|V}%IVFR0#|x~_K&VgN;Xt&a6@FwLpF!a5hbTd zuzGC4f-(Jujq6quw9Ci$kMHW&5-X#Pp6+Xt*BTuA>HIngS)u)$>f6(0K!Uq(!&fu9(C6u;ls)ZDF(oaT_0a!C*jGTsl`UOQ2th&ufdC0ka3?r4?ry=I;Eh}3 zt|7tQ-Q8V6aDq3k3EpUMhyOB@dGp?9|5|tTYP;R%+*7A&SJl2%Ue$A^htf%eh$4tU zh2OOM>n+>h@5g9-*WTwNPx(G)zg)BNAv;}$FPd<-nNSvaqA0SLO$^8Wx$qL%Z?=+l z7E7d3KP!alvwGoYxaTA~Zc1(_0h5QW9CK?2_RdXh+V<{ETN@XZEi5%PEDuX5i^DA} zYxo(|imN<`~fQmAA2B5_sBflBhHF$KN6q_R6~F&{5pCO~zU zW!QG_m$_qdDK@1$PR?7_VRrI(yHC%$97#ZJ(Uc2cP$T%-tiLu>ao~<7<LybE-=zYZ*F2*oDQNbnlGYF!ZiE8sGqDhTa+ibvElYF za_PB>vRDnW8s61!Ww`3*_ z7;=DD_*nX9d+9YOxZQOr#g8mg@-5U-ubq6?skJ4ZL(FXuxa&omomujAKuPx+Rmsay zib&3Wodf1a``y$yE20;tL}Xf3y~}*HTJF$`3p7s!vYwk*%^lWph-dwzIaP~IqwF~v z&O4Aw+NQ&Gfv9d&s^(?PEws9APejY}4-4Icc9_#ufePYPfh%_~D&L9U%t;Lk32M!A z{2u4STUlI5WKK+_s$PmPpgGbFvdcqI{N2nZ)?YoDfX8p4UyI{PjK_HWwWWBSPS=iE zmf3jQ=hedrd1*|K{PV^M(xVe;&uXDw|#^2iWe3NOxh0nKmrysx8(eEnx7zk#kif#W|xK$ z;Y`>m4^J%86L%57xqPG?XHBBkF!!28_lK+S_R)NAhM1K!x|26vYl+n({X-NFuv+(y zhxQM;$-lB?rMe9QNS}DmW0qI}!soQH35cF?11X+o*3TX5wp^Qz1*6RbJ&ATl#P1^n z+%FL?GQ3NV)MX(;7tK4GI3SNtLsxF~+mqVn^Xz9W`*L%w;F?g6(~k)brXGC4%kHMt z9v596eaXtfs07gq)~z}RL{1r}t(!Cnr&)aZ+;}_O`W3qw3$0UB3YKjryX9tIJC7Lq zaU3}_Ixl8c8RUj&(7EJglqP%6t+jHtsX4(PMf=}k)_<52F6U$%myma zOiwJ>8Ow8BTp$btM|qMT?l3Y<6-(Xf?XGXHqmgZ_Q%9&s_pnqwmD_QtEn&KRsbv3^ z07IuLyPN(-G+Zv3q~+JDe2=7!)DhlwV=7UElN*{Y!so#RH7R~hna?Y;;*PmfQ5?%r z@S#Htc3T7(S_Sp%6Q+8_`s@9;LwDui)O-5r+YI{ z8t$OE+m^+G@roY1>URrq_71uS6%$V;CDqbhqc1u@r<7=%iW!> z$#0L2I&)#GVe?_XWF`n-unl;&B;r{gzrxg8HgKX8jn7XV@*(6uYJTMw* zLvi+qaD1y;XTx6J%@1kCQ^kV`D??GJR-mSU6vT%vy!m-+R8Rd+I7XHb&{)%o2Xz`_ zdw1e;4K^rM78>>0Cg-lq<)bKz#GwAs5wcAy$&6ua76Cqkr71x}5VtkFx z&J(;JI8>iIH_05w(f#DS`$b!UlOs0Rahe*7Kf8jYW|}VN+fb*W#f2Cw^Gs#~>$!A_ zlx0C|SF-oQ4v*BeZ1;08P-~A&g}m7&gT`9~CCYmXnj3nnpizMx#*a zbCPAKS5P}X(vjWs8T2jHnXogv;iNs4a-m`07QW)jp-g+RJ#B5)wP2}d--vv3qu#&m z*WQ;8RWjt@>B_*NbT0ADp%mbDMo6XbxO&igGX1>Y>&fAvI;&3KvNTJl*Q`B#+ayNl zV%FGKyuC7Uo45&G!hUFdyQ^cB(0S1uF2)8`=_pKIbou`Bcu@NF5HXj_ugs%{Miq;h z5p!9E23;MlV2*5OL;3&(+=V%XiH3{1*Y8(%`%@03FsJ=#A&2@2-{QU9QngPPFpAEX zw8a}?5(B%F;Db-ya_x@dG84Y6!C$6MM_lK?D>RgV7D6cyu4uK!Y}+%80HxBK=wvgk zif#;`w*4}nca4Z};?x>)6s_1{2n0FD8p(LO@A;z*>l?ab%i-1uB{)robvbHs==6!s zL`ouevg+Eo2uc-T(9Y(^#p2pW|-^=wa`70%nY(kEBxZD0$ zyYkd4=g%QcLodIuH8Ul{xM6kVgT0k(jLEItNGe^3|&xQ+m!sm82i4abe2$HD|r+{f*!?g+H^aOdS+(f zQ%WW#rY^3X^@(2!4Gos2EQ`1ZK0bUIH%!A|n*3^d{{&Drt#+iZw3o~lW*i@IyFrNQ z(tAc}SF`k>Jo%nrwBWi^a1rl)hTkDW^K?jyBdBDr)(6;93a_{{+G>;dkfFpB!e*5s z;`x^rA#cxOJ@yhsHbx`R=@Q&{WYZYTk9Ij4ySM1$*gw#m?cpTY@{VnN$ly)bViYd^ zfw-+~aC3kMBmrpK)7azGe^q?L2RcK0p#O+9Xy1Mx{`o^WFl8zD(g6OJ z$`nVfB^Hj)w}-!8T`>}Gc&V5Jn*~?5&flj^X2HbKi^C>_N!}sO>NF65Ninawr>q3jJOmQ=8Xi+RKh7qL}r`Y1ShEs3I^7dI+n+ov#vn5xZ8N@%!q-S#n9 zNl`=gkXoK)uYcF(yP!vfT?0)Gf5Nu}h!Yy>yd80BsV+|<@US4`jk=$q=@=)~+e~ia zVzS_X$BrXuP^`?JI+{KqZrJ^Kz9}q4CmY3Z{rlXC>Y$yBzZRj=Oj16n48~ELhIrz7 zZP!MJoxJpFxXy=zqCRp)FPkF&(T_0Ho>*u>SSHWgq;iXvF#zNE0TkARAIC=4e|2Cb zi!4FHpY;~7ZYin=^}Yv#gIp``n9@#oQPXyh-tuJ-*?ewF0coAewoDjH;VG&@Teb@N z%8syYpV3(+#b6`hm299O;FZjr#__#-x4*x&ooJ>vz}#IFBfFBWdc}O#Spb%8d0~Si z_stP0hY*Kep6?{!=PxL#P+IyPXL?Rb)gR>qW%!#Y-o~ra%r0Mf9UEtl5aw!Sd~2yy z7m!L|yjceBDd!E}U#I8?f!!_Y%BFTx_RA`&-=a{i9agT%dXagp!tCr1GjG;~LfI`R zN<-|fQmNWHyo_(a_cG;|++UlKHl$z5T(A{d!#k*_pN3x-UGcq=^pQM zNBQMK^SgEMx(Qst5&L8r$1(-SvWNd^Q&q%@3cYT2BPNY@zsvV}yO>(WlM%|@g@$=W zgYb9&Uvn`!bTBVVC~d?(8|*c|^6Tu~Ve`!EsQe&mm^+}`TBSxB!%>)aGR~2Mf=NZp zU4Qts_9G-UNBjINZ1+I@bw6v`xBOi!9pjx?ust}K;lpz%``%oQ8c_b*U|NlZ9GBl{ z-<h2B=oB%*2VdxH%SxM6jTJ<$f zO|QB1!edT4VxFS$n#&|9yI$A6uz+0BL@{AJ9ldXoO(A~*+l@)>%w>>knOAa9m zmxHL@PN%#GR^V%;5(j$UBJug2tW^4Ca5((g)AbQLA}PU{9*JtF(KoA?%WWC;IARK; z$&}<;sJqxVdfc2fOmu+`wi0&(cpevPOZV8!^+&;nnTqYJa*q~ zJ9#n|`rk#yvGQ&WVQcr8MY&wqWskkAI?$sIIw;H&t z=07mluDv60c59cMZp!ShC&65J5o}DB%jkVh(;L`gL!Hzz)Xf*?EunHpgGAoM&nCo; zu26{4^80D$@1ApAO>tl}7$sqLCX01(ca*W!FAt5M_MWX?Pai| zG#QC{ZEK)>b&NvoAxS9-a#1+OWIVX|M4K?=XPbGE1t+HAUZb?a5J`6HITpp?dgO=y zi09prY_r?peLDuSMGq?TUB?;K)Llp@I8RJ%9mzOJ4*OLC(Cw8*WXKHRgE%ijzz@Jpl^k@a|SSmuJn4RqEJ%Sr7yzh(yHEv%l zFm4US1#OSfsaNU;aUw+eND{xb`aH${&D^b~l&kF}pl6NiU#AfAy7$kqm>MrHR`9B> zWt2U9MQ&ph)~0m4R4LLFU3Pc>5XAXXLHztQf-eq{5PuV+0}1tL6Q&f5&G2Csdjz|_ zhN0mTBC+jtpJ?^g^nEW?z~`8i5yz#NM&pkDk@T5?Dlk#YFIv>T?D;HHy;w{cUyJ!F zH2$c7eeS43^6a#Wd<|KfbRfx62>=Z5LN3^+_~@rMcs=)r1E%ByRC); zu&%XR?kq@;3(#rNDfU|5zaYDhU{lA-w+|X8HH~NBnAj9-DT=SKoQver3WfW{4{Q{z zS3i*6Xe}aO`~AihdxFHs4sXyJ4Kq`_@Vki&J|k?w#9HHQNxWKt1cB-C{x_S@z%DkO z0%?%cJ_us8-PX0fFKL65wm$F5ep&9GYt_O&F=17%Ujl23Bj@h&@98zi+3(GN@bNnC zTt~}Yv2tErZl@0XMUj?V&VMxaD|o{(75XhzOlICY!Kq0^Sxv{aPNqwHSEWRSjpVe+ z<+jSM`jrjIeWlAs&DGNfJPt5j1TRR{OZzfr99{htl)Vr}GFq&VTrSUAxgyYHGwW%; z51Em=I_3IVC&O}-pHt4l;(josfE_hc&*iIM_m`m)_GMDHERA21vz*kp=@=(9=A2fb z)^4oScDl+FgTxN2>5asw>3U45?=825gFb9sC1fO6xSzAn58MPT@4DEnj!t8{I1v>YFinQmSq|4hoiC( zv9V6mbPmw$Xz8`^=XXR7X>TBfn4;;FcKd0(Zqik|<(CZh7bth`H~{mTHZ)F?6iAaf-$-iP?Su>yP5GCPhT4wY26u zJzu?0iXv3Q9gR?=;`9dG!H`{*7@;(fOKLuy@J}28II>8K$Bep#H94)&?H+oSTo1Q`AEs9HtyzGqDT(d(>75_5=*tHlK`F6HcuOzGcBf%S;+{LAQkIQyL zxeNX=-<*?9B9LSMPuubE7>7VMWc>Md1FkO9zbKgAIFSvsA+Zaw^RojTH2G zlBI9Bk*rByLiZG?K?Vki4qx!oN8TLb&_If3Sj6m7y9?t)9DPD@xuwh}TP1>)I2H}P zY}#*vWWFs};dtF1B1>2V(k%JERPx#zU zLmL{CbyT?Q=2U2Ww?5j7KMfL2>{l~S^3pb1h&tG&xm^r~VtTjnAS6~x)l-Et1uXf} z&`nu~Y^v6vFeLRWkD9jO`wmgngJ3=}Ckq>!L+=*Mu1QvBwAQ}2^aZ=+mYvP7_vag8 zTvtW{Sxf@@3iQNh?vN1AU)4fi>n54i#<^FrcP2HmFy9B7=Bgdm7DcDZdzs(HOcd#u zYE+vQjV~wL?bs?J4o$VW4fcx@or7ePlK2te7z5_LV$;U($qPNhDAv`GvyYP925VJs9joB?>c7>&!ysMC=sQ<&mei-e8rR8 z^x;zJ{w||LSawvY=6KdQhSX4Zav;KLz0ILy)S_cpX@abhXPRr#4p1gl+J@U9T9OP zRpdry!LFef11w1%kXT7?1oSWe!3jhWG+9vVDJE zAyIXkm*y|joS~6o;6*XFY;wo0VVpFFwN;iu=1rhZlQ7vRIIq%1Q?t%+1kzbM-{)T6 zjv`gZE2~n+fn@K;+`laUg#H9Hl}HKQ+S%me_&$a%ym4HvBceAn=A~%GJ+Q~N1fDbYRc`-Y2DF!v-_Yu_{Zj`?B2;; z4udbxFA4qa((;I{s6xAusHd9zutiDxmfw3IUzm0JY@tqP#P1f_LFb3r5Q5$-Hv)$5(+7f^TzpTP&&o+>Mcv93CaI*HdIiN$ zMRxIJmxaXq@=Gg@E-*`{1)oWG4gYyU>?G~s<@MjS0F2gdn!My2YI?|1W09o~oaqhB zcidlJd+^V)MT(QXa)syoMt*$2Wg+uhvk8(byd^c-SuJC-$#>{%u!at5IR@F5z*vm* zb9{yXAXB_F3$ID&`K48LUz4w-g5Rkx+-ot>M5Z84k%b!vQUaX-_pRMlbi_?1(sb(q zRb{>)ywxamVG1`vjwFz;5hJc)@7Xb(Fn6=a8=jj?s;bh&<*phBe`62|&KfKR-%K8? z*7M1yTwT2=QYonaX*nJ+0u52^O@3C~mAE4*P8BpbI}-HS<5I8T4>6A`<}{Pi%mw9! zPMJemM(!Tt-T`xsGW)vZ4u05_;s+!p94$602gC8@Vh@egWlwvf)s)_5rof2>4buXh ztokx?n<|59=dVlAjiwb9rEkRnUH=_veZctaBjooM{1f@vKk;2suVj5<* z@#J$&m#ZNQlh8+yj|W7f0*1>B1b?H;$KSC>+Wv&)+4T$H^tqLt#zgQNx5kz2Vm&(J zzyU`(mJ5Dex__QWrf*IzU+>T-=X$mg!n}RZ!XwHO`uG)+XaQeUda~y^l@W6H0^y#l zONnAb@KGLnjrbn-#$2D+SRhuJ(U$zOtc%eCBCmk347y#4hUD*n_2aM@;XwPdH&BWJ zA8v%fI#0I&EH&MCSTwrC1@am3!zuOslhn{$8Q28#{u~R+5ah8@g)%?Mts5uT7bsTD-vjx;mUE6~~%i=XwrU6W1@BA61C3;~%fl zo`gVw>FqWYT_ZTml%83liTMS~)eCXj59Ey0trz6s&iC-_@3Th;@gC}u8lNs@go zkkh%o+fqzd_JR1sVcbF){hrg1@IS)^1mMpD?~!waz~L+U_Z$33lg{!ikd4;|f;<#Y zq~S-%(xW9j!&hTgzttZox@IS9Hjm5vtwa79Y$WNQwS$IQk$%S|fb02q_@p6=h)UvJ z9Lwr5i5-jBVVgKJ@YKuw(6(>2o2Vvm*l5sM(Htd~8Few7ds@%vDrIgT#GK7*mhvw0 zNjh?_6!(SVY(}XUww0;wC=5pm?g@sA#g*`rbR52dr2X++tem4N`;_IcCQ}l!vIFYb zRNyqG$ajz0Zre7b)qMmv=C5J={n&^uAshGBvAQ`iJRlorS&vLck?~q3H!(c-h5VP# z$fNdK9!2BUVl8@Xe6UdWR}qrL-{HSC@K))b)r1t3Wf(=U%sS1l@l|5JqX5{TpKShM zl>g5ad*ueFs901YlAX+%kgHYp?LWfrulElPtCFcy0I(-}x{`POA_nFJ!21&K$ zq@*MxoA8!dKQENr-$irK2z$4d726*#Drz)BlBJV?0#Rzs@V~y&`cf&XDkgAJHtuC> z<+=D)j|eVO`uZT{=28r#o>ky&>FONnhK#(%Q{ek)KT+C2nORsg(RMCC{mSZ){ARN1 zdN|)%XKZyNZi?Xv53w&2(D+X}VA(}A)?UueQo`QM?h8Q98q3qAH{6AanW3(PJ4D*9osK!Jbnct zzV^|K5p4tvu8aJaU`|evuF*ac(d>~%`BrsS?;`Do)SHmtfoch2F7kmWVPwuDgX2wS z{H@+Js#qCW^~j zMfdLv&R=`-pU1Cya0iSem+pknsK15f_u2o`%UqxhabN`8f6Ds*^hKk`dR`jv{m_4O z|DVd>pFjTRCF&c#r~mAz|3wdwMT!FE#0Z^Op3)bn%kl=Y_Yy0_r`bfcXHP$hz|}+* zt@ppk{J&2FJ_#6(Aj06D1o0b;Mr26xGnJ7*NR4LRpC@aUy@CE8C;w|MUC9EvvF({? zfsl=FmA{^OETZUL3~(`cznxAIrTDE_{!%Xs=x;%`0Ylf0;roLC^^gZ(F9OkD8E30362=~ODIXg#FX+_ji)Sj|Y;#zel}k#!X}KjjxI=G1sL z|4s4CiR+J+w^e^!5)0uk(Gz@HQ)4Xh07ya!#? z2}*JbNqec!aY~8t@b;+!@8=OqGMr0b4){{nK&z=<^SLe{#O8LD!uQO|#K0iCV;~7g zV*p6KKS6r8$2`gh3+{S;8l&O$X^kQ_4H^s=uPZBS%(9RuH|>E{4X(?!@y!IR53Xv< zHo5|0AVzx6$~rE-`Ed2g5c3~7G2-^k%_Y-#m_g7u?ZQa9n@h02(PgOc9ug7~=b? zN`vIhh}8xxAtMriD;MiQ!o5=0FL2=BzIx8CT3$3&J;*G8$ukW1q~u~uXj;ac6i)Sk zWnpCRp~YasYum899x6tZf10h{7JOdvL|O#y=VLP9SDQ1P>n|iaG1Yd^(V0|O7Lx)I zua8bf{m%zE_{n#9;UxkD3>DlP8{ms4@PvN7?w^g`{xi#2->+&c9q~yp=zDTD?D z>OKPkZa0}AR1bw>c#Y;rE`(@Mmw7Cao(Tq3Vhr#*A3q_cFrm6>7cT%GmQ35Wn0W|c}6W|)9l2&3T zG^1<&Q3Tzv{QJI$~m^@Tm%z5OT^5q($Q--$XNz!Yrt zK(Zt?wZ}izB&X->A~~5o*7Ee4brimIQFxT=Ju5!arF+JllgYyhmZ$D*8&Pt+Rl$eK z42`DXb6VcVwRtQk_^aKU&T|A~Z00p$zFxJ(h%Sh(Tr5~xeUSni02P1fldq`ojyn~; z7fY}zD$y@kQ53^?Ul`g`l7>@w^~#&;TLA77L{hU-{|U?qqL##QgO>n6e;v7zEPs6f zs4v>;I~;dYci}WRU+|So8M-|(9Q>yE)bCogWQm^7b{aeqiA-~Li!B$Lb?&w6#2mwL za39y-hyx(aR;Ro_8MT5x&lEz;%5MUIZ=rlA@j5z#MsAR_poZMvD8%;{*(*FeTGNq6 zrx&n!I+py=_{quI3X+%!)C0T5d{Q0EdWWi*4xLgMzGr|txX+!Jwe731cc5~JZxyk4HJKj=e; zA}^QMUSoG(pY+t1;^tfyv|`l?b4MCjG6QHhaq85B&uQVhtJgeff5-$HG(7PTrgVC2Ye`7{M2|h6LBMheqp^G-5F2Cdb&E~r{$IVz8iS5HVY-l_=^`)eihWT zaFo7HXZ5tq#`EUpY9`$ddFu&s03N_j`bz=t4-OFlJQ!#w^#Tn?FryFiHmC>i7t#q&mDk-ms(&zeLN_jcoU8#G23(yAWtLhxJ9?(|q>l z(eQ+Ce_9tjBA`VKv3%h-886iJ4$|JZpoLV3pkf>G%eHjHk9^FiKb2uvn7KmGn^X*- zA9Cy!rpl&qz2QU*(AOgnL=zsYK+0Wo7TzI-Ar{F7A^=YYhbDh!Uz|MIMwTd0;zlL_ zNE=Z~up2!?JUWyO2dq-id!m2s2N@(Ft6m_V9`6>m)12E{ba?h5-x6G;(GrASZh`Zd z(5rI#B5i=b?e>12%lq;ddXd5r9FWoD<>$t~`QB@;u|Kq#)R!LP!u=C zZ~~UD+olRPI5(-N;6;mIhSW*M+XvzaSG@E1@e;!7NTimg9g zv=?ymA0>d~*sly9r?K4<^j*)?2{^kQq$ap#w25&d5J?t=GKL#`GtLD1E8Hz>@>lPP zCiG?-gm*B(&voFzsp(qJlAmn^3xN}UtkSC z4>K+UGJtOBP@aWS_pR5QdVg?pCXd!=>s)7IX1~m$#!IDGCElX^kovH#{7^nmqu`XI zZrE@sozv@sTpABK+wNqfIa~4P@Nhf!{nR~|qV`=;Qb1Me=Xq9MyEY^*Tf;vxRh(L? z$BW(^HaBE^sp)C5zTkC?sy&qRH;pbF_jE`AmMhPyk3ER=#zRK4 zr5OU{pUC&L*wX(?YaIo#GXZ-0QvS!$Y3`2tFQXm&ak+CU3GRMK?SNu`EkHhcvV#B? zf1v7L5+luI&K-%5ASQTcn4JEAlRJ{g5Luzf{%OQX%b&Jz&CKdL4z+;DB9WSffmbrz zhW+Pfe!-WS@5ZG1lZwLmc2GQ@!4{tnva@LU!0+stqw*r$!I2A^KM3x;ut2o|(UQ$}Q+VmR?znX?O@jlOEdYRaO>a;Zo+s2Z&fN&N7Txy7npfOvU1E14u z18}8X_XhzzML*H1!a}WH=`%wEb7W5*Vdb5s?wRZ~nE!0tk)@=t!u>5pXq#se){|=0 z#6*uuz%U}yNTds2T8pO2f2ayw2usC!R7Ps-DdQCNhoy(2ZuKZMu?e_!4AH}QqYY_k z47*g{-y8JI)mm9DWYw`+jEk`AyD_JEjn`s*_l}{U$Gx%47_MCeA^;-Tz8+C3I@9s+ za1@!Sv7M1lv%9}Wq<3`z)a>7+2tl9SE^`5n^bRj@bX|hSALm2#_0Pr4PHqg(w{62%Deiqg*6aKX|A^~XbrTSooD7r1>71e{dCXoT^-!??J8 zw-NB)!hEKFs}=d%S*QFvg5z$A*^h|l4*(QMg_JI=(tBHRH(1RvOz_f+ZZQz72Nvi? z1rxagr6D~G3=Ns0l__b^OTSD>sFtaRv1v`DLOZ^4bw&9i;NAoC{~-q4YQ;g!L-vFc z&bU$!+5CB!&yL`3R!SyW|B^Yezyq^mHwo_5x|$VMg5gY5!nhNO7cA)u zd+#zUs`34HuA|z3hHzY3Lv&tU;o)%2=x&%&com@pD7{;QaC8*ELsGuBgEBXy3F}O6uRV}LdiYuY1)m213!jk8W|_5K^2 zvEkYLpX9Ln+>XLc(cWz=J+QO&3Vn0Dbi2paO2Oo_%!H5oPuO$}GY#@Shl6cuV-ySe z$9vMJ8oj>w`aT4{^A%Y53xV)v87<)KBTLr@=l~XVAR^(F)8XP8i?b z44(9hHNOm|Z$CeXrvmpIycVvF0W{ecOEq@L&P+7F>OJ+~w-i|R2=K}&q+F*VK9{WE zEkStiHQhFqBm2I{sP~2*I3LlKD&t{rtIW zjsF8-Y+1^KAZY)9%Omb_APNYJsM5U%lK^6W~5@7OUBUs!nDb|7~xE zO2Dn;Ov|@O_AQij2^U~mz+V}jMVg@~BTk4;+>P7iI(U;C(0*eGb`q)(od3v=JyAVY zEtg4L-S~3irljTee`;E8WQfCvr{dcb7( zpq}mqJCZ5jKT^&sDc@%}+49<|j;zT4!c9Pl(ymK|duXW&<<3ZnxdDW!uGPsXN6Ax1 z{YvP{?6VhS*v#E&A4}(z``F!6W)Y{>Qh%U6g_I%u_ntIu0)~B1JzGA;LO@{Ncf%ty zog6?AtsZCH^>wEmkgs*{ypmIx*2Kz#TrSP$_cj^XfA%G8 z3S%KEti9-NmL*~^wxK`>syz)JNA2Xkei7;AZv)uVCB)$`j=i-8(q5W0JF!J74f!aj zkjmH|+-FKFX`5&CR%zvO8S=QuvmpYNzdZ%vPP#w&5aE`SOxZgNH*z^IS$A{2iTWZu zbsu3Vp_fd;4@ezzVvth7>g~G!XfE%xr5y*s%fhX4ox6lu^OEJ}pk z_avrOj2&@09i9h$ED9NPyur1tX!c_L`{_*op@NNG2v0Y;D-x9WrCOg)@ypy5glhpc z0cm2-Fj?`AF=k3z#^_d5s{luDc1P?%WZDd`D9>+_RA7OVZ#7{i+iKvBdJeEYy_XU>=%;+Qx5*n&xXODY-g*?9BI5lxg-- z3w6nS!U#_g0{p__EX?bKdMH?OAxt{6p;sA$h3JOH6Dfz%OV<#imE}A8^C1nKvvpE? zE zF5b6%@4BrC_KCY0s%N=*C|7SHx)zzO+xKw#revGy0TlY}&=`zo{L+!qBl^Hs?7#gz zP){Nxnw*DkEbTq$SDD&M9C}WW@h6)=C&=ZG;9RU}W?F?zfgl$GCZ$XPucn*xcK~*o ze};}*;kg5sx+m{1Ux{y^qZTr}w-!c_wMN zgebYtPgcu~Q?|{Rbct1@{dZ&~u%*!ksGSKw^BXmg)VafJ`radyn1- z+kxNfJab6BAO<&;qtxnTv72vY_mDx80zPPD=h}rZ1a~WSZ~j_1&+*+u2F{j4jUj){ zU$E?YuTZJ-cnF{t?82wW+Dk>tXq+V~hH@uI#3Z&}m0=I4I~(`ar`aEOORgQ}P>P3a z$atpOp6_XBbU695T1-vaid{)0zcW7`RD3&-NYcQHLmp`G+2m7$($~3e=Uys7=cZAj7Sjz=971jLG5R*H7`Gzp%r!Yq>2q{%?orj z{1Abd6~5j!-cC9-w8W1OXjAV~G&;l-IXhdJW8MZ);S7RV+(jH6Wr2KaS^BBg47z9zfs;D~7gzhR ztc8%hB7sY)`^z&3E}Zw$VP|3YGG2}C54UuL9}#6M;f`Z=cb>|MiP?8NpAKpQ0zvRX z#PYM9aX9VU;QA`A*Ex>}`DI)76p0oYww)@$v5|xCNTx)U zN;FhXxV!_pf4q7Bg5n7W9NG`{g6!l!T=961saGA{7zmVL z^e?mhOe~^eXE{h_OnX-0udC{Ge$^3@_1q5Nq5R}&c5?x#$JS++(q zB{N({&aZH@8%@tEFM0mTb^nu?_&bsWWIp|YSbdaThQ{~O*R$btx_%6PVr~O+1?W&% zNwFYlYwi){Y{o!hPNbrzggE~;rol)(+g)aZLuT-uG&=i~unV!^PACX94sCtilBdr| zkvqnMYV@7LuFOQzc-$}nkrsrH_eok1AU_Eju{!A`LxTe@f3M+xBo-du{A31zE|JrH z(+zqI7i^fbaI;HxINg!aA@MIXGoDkpb-rtOK~$x_p|kLuT7vykHAde zVf;@Av??gN6I?UVas8#cUbiRm5^|4V`<}L2E{NP?jz*;3fAC5ww#$yzt_{=d{kqj@ zSUCIycuhCJF(8_s9WBK1W!g7g3|fCt1w)C9z^9QMC&3j2>9jT*rkwvo!v8(_^5=6P zr~R$(F&S)&AW0NZyQVv%aAY*!RAJFL7oQGQ;G48CjQqIeZoR*%xe|BTHCTSyaeqfjnBCDS>x z;~l-;>urmEBIr$y9KO!jz2Y`T;fqNe+^2V_Fl7VKgd_o^eTb;t$)((5%Gd5UkpCM$ z{PXhF5xhs6C()5Dk`Fx3tHhAX3}T3Cp`j@E5pcT}oVAa2aC(4l@Z$b^xOD$XrsyA- zMLAl{f-^!91t@rLzH~*&>Ex5rwbH{sz^~TgGYS#`>iY#EK^7a#H;GF-o^+{W41U{0 zB8@NIk^wGQ*ljo?4Z{qFLy(^{aAUn3r0qCg7ZvG_<3ub&KxQLc6uF!*By&{~BT67M zD!fK)T9C9^gFRvT)iGaxrsV4T4C-jyjMV$Txuw6s_P>cgN!p{--@}VI-F1kZ<-iGK z=8)Z1p+!lk;F=?-1lI%J%6dVc> z9C7Dgl=)2x8YGw3ty;9j^`TdMSU_Atc__bljjT^DqBj zl$eHQt+sF>ygcsq>Uod~ue!QB1}A#Px$bCIV_WrAq?gW6TYPHvAt-MyzN>`XMp^e{ zEPjCn`I=>5DSNOb_}46pZ{g(Ru^+KvD|s2i5A+!yQ^c7 zl;9@&l;q@T-m3k6&5+@Qm1&5K#B})!=PwhmDKHrz2igxbVmzpDfAb4|@8(Y;a9t)& zcPunCQhK<5YP+u$ugkSEca;{HVj$1`ikKPYe^nvBN9Ei9n9es^@dl9Ke_#Ipc!wAX{`lED zG?Tw+82^&_pK?EJl)oCkOdy0HN^~Y>|I}rSUqOIZJC z$X$m|I=M*cedfl|s&_o^5w#n0ByQZf3APsaFw^wx*BHu^Qt-XAGNGMD*T%7CgVYh6 zaa(GqJ`g4qkTS0xy54153($Re5Vzf0b)SJBL~UB0MpYC|qBey`^(-4@p&8*Q#MEdx z$a(2|<1(*LAK9yrM?yq*%LwSd8h@tbA~&t$pM+W%)6+aJY0#WS&aH}1MP1Ep&Nz^Nm0nc zB+-8qV1DUe7?Juf!~1XQ6B#}R*o&RH1FiqDD*v`w|Mfobqd*Te*98sm|9AQIADfpX zvL?OxKNR<$9|?#&ev`z8n^yKec_05>{9efb)(Jj_;^a3p{BP##UpwNH#G@Tsw=&K9 z|B?j!B!T>&{)Cb4zr^nMlUINErg1wB&a-(B%+;7a<>cOvWicJ;&nS6_y}Z2iIjLL= z0o>a@w<;On1P#hpq36C{O* z&QP}gWa|hBfXCt3L7_VP1#YmZ(7e7`lpowiC%gL9yChp?ue2(-Ck9Ot-d5S(0Dr*LB%f(~0 zfoE=R_35S?geDe(iLUh+mG-|gqqG;K(F<~S3PYD3LAMtxotAeV@X<|i?r$aTd4hq! z6}PX)a?V@f|2h>i{lKqV8D8&*LeZJ`VcNil5>dMfi?4Q@G6yebH%YhNbjtJJ(=9YuC5U$1S^^a~_}JN1my>Se zYi*VkRd26TYOM$j&Ik{{j7d4=ln-QbCl?lk0$4Nu*Lm91V= ziqKl#f1B5rf6c4G;s(4yXwGW}h*~G5&34oxX5_ zIN1NQiKq z=0vkj2#40eN()8M3?O*QNZQ@ga}|&YZm^!C>3FE0Fx9Nj;A3Kr1{j#X^$X0BwXbPa#(rh&zH^M@g>FYmOhcR^$!RW7&`kxl~y^j)7n$ ze&*?q4V+|`Fl)P?E7f6{mDog7rt|vgkRMx14TU-12-p{W@?1XaPS*h@6_f{J8 zthYULSPP*Nz5HCdkGE(}mW&JV-M_cO<*tbal`W=Jw2@mSH^h9=a*j+$_tq$HJ~jt0 ztQyP(p)2OoYgNnOPJ`)4#d%2h>24K^RSP%AOk`NZNPTTdf5_i_tv|WdfjqR>^YuQo z*_v_Ju@lMOsGF;lJeHr3On2XO1%K#BT6A0*w%31nk}!DRX3aZ4y^tq&=4L;dd1Ew` zW0>bu*L=L{FfVGmY~5B>f!^*sstcRj#}!{FvMBtMKAG~U?e}g*=MfXW-t}Aao+c~R zgTNQ)1#63yT7SfC8)cDz(lq@7PqXASk)DmI38Dzg5<^nQ_hfx8hL$ga^ZpFE0%_gf zl{r94`>uH0*LGJ|eWDYMcBJ6N+A4frH|3FZURpNuY0iv(aD`YHZsN|yr>gufaxD2y zn5YTNa-`a)w{Xu+cXcik`7Pm!!j|R=pNgbWG+gY`LOU)+jyI%lxGD(p4aLIZXj}B& zAC^+mezk|bsf(Q!9LV!svw2~1zhMo$D=O44;qW9--PcQFXh)UUoBl6S>DlQ zcpO!c(mP8K<0D@}`6~B$yiJ!L%2OkbyhjhL3Z~aPf1-Vl7X4y8(mFSh`O!1Sz8PUY zAc1nw6w3eq*n97wrq(TPSSbP`pdz4jL_kG~ARxUdAV?9A-lTVs5~PcONbexhyGU=L z6QxS;1QLq$8W2KFNb+rt_nzyy=Y03i_m6kp8OPa^+3fx7^*n2>-|ERkjI@qH3=%#! z?Xt)cVxuJQ1E&#B9u>a4h$dgD`fbg7+$RXP4pUEj%m^|(;UNa#|gZO|+PYNd1%>e9xG zk_Cyip?r4@@+!;_)X5!>hYwBmmwQ-0?X)S<(2l@H-Yj;IG$4;IR=M@?J$tTjHQld7 zb}fnzG-IZoZAy4-Z|}a%lPY!D!9$Rh=`3@>{#GD**g=ey z{LMmEM6RhPay5}ie)GZ4HRbe4NSR(nHl{?Y<;xWYiH@6WW+DcInVsV2^2V7B5Jv`{ zKr$^Jn;IMD3AFNahU-}3i}HPXD(dX~Q6*t&IJMupwa)ma4~s?x{O0DBS;FAzX@qWH z+5U2YH}&8UUEi+;L01XzL?#Asj3_3N0LFxOgrtqjQ>F4ko)`p_-f70R@YTlXsy2Ue zURMC9KKX&}sB`_tN~-&sF=q17J<4@+ckCu{ETTsa$)}`{SnG(S;i5vHQsk@EYhOyK z1OpOA2JMr$jqcMyoe~yU4={=S1M$^QhepY}s7v*}OX}l##3&1x19*Y%1HLA3Pi$8s z&QO?X373%$S1AovVG=Z-!f+2#8`mNy{lGTR_YQ?T&Jh4pIiJ_r5nbMikxC-Y8yI1} zzAMiyf_d>cwV`)YQPb4@v37_2wCZig%IFj4&?CnHldNg`hbnC^X7`_rGri1$j@n|iWB$Wv&xTc)^Jv@4mwAH z2U>yex&a%spDab|*Ho7)jNA^;9MJjcxwTqab!{vfF;}Y}ML+0J%}5j!%&~GCPYXr_ zPXE@j2XYjxE%3*+7-jhGO~1Wg8YO0!Ye8Ayc7}k?1ETy%EJv)=@-+apofkjQd*Mw_ zZ=_My@-uKRJW|$Cajskv*gh(mE)@ecTm4D{kC7sv)By@fJYBXmx*F+#CJZTYcQwFQ zMEu>@HHQkw6U@4!LJN@Hj9$wuHnSy>mD~|^?uM2?U!2yH2f4CczKWdwWmasFF9o&s zk2p#t?az8Rh99L0IVOFowbc>9bnH>?uWa+IjHyZp(Q_YeLa6|TxSJ*j3sqnZn34Hv z3kYHUwH=?&@Ihbi3uA+1^ChQ7)7e zZ9H~|OTy{VGb{nl99I>7ePbg7OeOvHb9sSNGnA0)IBV`ovIV4fnCLxFrZ2H^g8q_= zZ|t+f=3n)S77%H~9Or4vk{3!^1K+BUfNqUn`jWu-Wz3@VomL)`H*9_D9f}m}=attg zIKtvBrVVs zKKZ0te()1g;Y&1XlfuW3r11uc8i*w}5L|}R*k0bS^AtHfaI@wBs?J&sDGaaNLqoDx zHrl!hH)=U$dygWL6Eb=W%X5r~YTxzB!kH~ZX+(%l`3!~RycPMpFFm3tO`U`IXP3rQ z2cvXdxx;se^cYore8sD=GyH2+8<>_Gws;$^wbXh>fe}TLhd;*ossCa#+W@X3b(@Pt!W=xQB9RSyzr%u|>)jXvYvPoDF zBjW+k;|af>oT}jbOk~s%mjA{40Z{r**vme(f(%ivDBnoS6nQ?0$v1w+FP>XUZ*Kpp*Zgo6Y zho|p3-#vO+;2V~%&y3jSalB*-=Q320mRma|9K5K4_Szv*U)jKR0`h9kRutC@-qej+j3b7>DJzIC?l!B+ez7s{uUoE)x%Gl)-S9y zZ@W?E9TB;S*BDYXPw5kvU$SZ8<1Zb$Zq4^-nCtcDOo6&Zg0pQ-W6q%x$>ZmJ1?BI_ zLoyWSp=j?bn-@3M=HfB$zX~$mewUJLX-wbSGt?jwHXi_!-PD=EqYADO*VMm!kJH!B z=9j@W{p1JGMB#mPK6e4732s6d#;zQLm{{mJfw>!Z| zYhv7Be}<+mzfVZnN3}o)`B0xa5i0mU(!_R}yyU?J(jxbz)430s-8vi~*I| za=5as(=SJF2#jaPP*O*R-w4VGGz!jm=k`d(T_K&^FpG6qDP}q~i=l%w`!r@!DSvI< z*N}`XpiAl-FIT3>+wq6Ibn0L)abG{~poG#SWqAr8(v>Hvb|F?|FWq&tdMQh+CSTm# ze*W$tW4&e6N_Tg_Vgp@gT0BVOKtn5gIF_Huqe|W1toH*?_*57=6yztjDN`ql^0fUU z*KD%k@*~n-ihq^!R0T-M$h@0*Z6|q8J@3PFAJ=!*g^@3}TxFS3#9hH@!nv(TWP$7C z5N4lq!?oQ#>T9OoiS7ZHu~UIY{^vk;V+l;}O+BDZ3a3DYXlh@h!TlAUyA*}ttxijD zptF~EONe|FeJoIvY56_tNze7RqAos@S8tU`2yy|9ek(22?ij3LC%l-wz_+&oji@!? zE3Pd+a-AMsVN|t3Z(JkAf5x0cvixMt_zAw{ii5HVrQ$K=_=QoGlf;-P#)l4^Ae4-; zbPEgOqni-oqamS1G_~6e zKEz{?(c4C8AhU71R>6Jh^HKr+BSJp^6sB*jkOCC~Dl1d7Mg=4ziLkN8?nL{r zUCdmjks2t~Fhig^BOs|;O!DSXNlLPWJSICi`&-QEgICasjS~oGo=V78HrWO&ptOmQ zzr5|cSfS)v?(2cxVZ^X}j4RxOzB^MIwpygY&zSQorb3V+rLG=#<-B!(XfktrVe49L z;LK?irLm_n?2!{kchn~OqRr$+T$Fl~3d1e0UEYzRyzv@-7ofwT0vzO!XTj-P$1!lU zJI^!Av&&N5`jM?MtI?+MQok9Fj`h5eTZZ?m=3Det4i!^6MQ>-P*HQRGEe9pd`=iDC zFrd+lqGDLC7y1~HoQj^gJ>QS+BDRB-gr+@y-Bv%e`k6`@H+#0b4D{#|6FK@MXm?)} zw4H!}5Beh>4!IkA5WQo9|I0=#n0>D5uU#@@*;JRV2aFkSd{!*TmETPzZU}qqKqHxo zZ7>*XNj3}0l%Qd1mYi7G-OiQ`Si70a=Sa)jPT;H8sNx!*pPx`_un2qnln$hltP3IAEguZVciMwV+X06Et2e99|NtIL66dRMNr7|5& z!*>-jDV+fJd^a%1Wa;qYA*N*TSW7x+Ywaog*{^u}+e`up>`%%fHIz)qYClRn!xtso z{d(X_ErYOf%2~k3#n#stxgwxcKY>SAy7vZWvNUql5Ak>{dcvPJ_i7EY`w`-Uc8Id6 zY22fdSsm0tq6HHPpW(+88Y=Ht_3Q8D$vj zE7}nEvD%08SpwCPpB*V1urRtjo50omiDcw*pOa-@;%s@pbC2yraqou^rWo#Yp5(2v z2BS|xwlDT;M$B4GHh+BaHn$4!-5%%EGL-04z8f!n!9>1kmuCha9Br~<+VEOg+3(yQ5^ts7`|+K~Vr7>z zvc^lL5R^=?H+D@&OKazqiS`d&KxwrSuj?bvy)^R#2d?`}2OMs}$$5xttv6`Nl@H?N z0w+APJe7k{vaKbTZbN0qvinDEC6rmx;{Z8c5Kr9nUOWI-8+byr?!tQKCvJRNoj5G; zVBifRYtqGbmR1HTE5Z;D8aE;;4aR#|-9PZUYO*g`{KNO;tZT!?t=DgtvDV!5A{f2K zpV8k8!hd2rE0&T{KzLi*Tk!`qn6ex8ZMK(dMr?eq2Z;GwzKt#w!HbfP&kise4c^p6 zs?#eCwXVL}KRiwm81rtjI*lomXc95qy!1WUFz`DMi~AhcK@9Il&)`khvdlixESDJM zIyJ?F#np3U^W;*%EB+>xjszo=OlbR#6E5sVpE2w;{U(DUv)9nhY|d888@_Zodkl)~ z+@`;$sQ@YStP3QfpdAa(&&yL3U)Ba{dv-(Hnc4XTwZ6nK*suPz{JmH4wOE0;=VIRL z^1fsqo=qU4+1S-tK$aR2)$u-(P7&}einF#bvAeU{nc*+iToAcgPm%r4Yp_IJurl$*^9WQS|)N2k~kB+Cp^?mxxud%ud^HD8lo z-te+`d@taw zz3Iwjid5^|V%Q1l%lfxSc|BCz(WPTl+#>{ZTPYt7|J;Ieba{dq@AH&F9EvXA(=J>2H8ok<4{$&$7#& zzWOT$&U}S<{WF!*AT^gC;QY;ZPu239$g`iW%IW1wx<$u%c!fw1sICzxh(Nw9$0NL~ z05r+)W$U?GxYhC5`c6zLvH2w6^*Pw?s6$-|l7$!L%GiyUTs$o^%hF@tPVcDPR-^cuBp z?Y-|NSC@C}A0n081~rD^F_U(5`*{EE1t2XyXd!0 zt}BA{vq<8uKH=XWIor_X8>8Qgv*AVdk{@Aty(!(t+{Ut5)E+(gPl$ldcP_ zvna@q`ChQWoO_D@@gFsyv`nwvj9*j4D^)8F7$%GHWA_7s>aW>qu8rRN?1o}6q`S?b zENbd*9$BYi>|`;$z;!#Med4s5K%GPDwL-Mb(VeS1L!TryCl(H`0Zms8FKw*N#8W-_ zfj?|yU0O&$OloPAZH}Ya%t62|C^U#&6er(!WV;!Z@^2LQXSC_hi?cgGdPZi}#|O~s z^)v$B4C>usJ6n2}tXM=pmU-`s=8=7$qz0&PvMIg&ScWxrTe!8UDZE z=%4TZ^MVh^6H#J+do#V5BtjaI5wt2JB)u1tZjXOCcPtc9t&#m8sq=xiXZYGf&%x{8 z$p9*JsIbP0P`WMmW)2fr-^|ezvd2%=4-RHX2?A9xdj*8mq?eK*Zq(v$aHC)O)KeaW&3$-S14Kaz-F?2w@_$ zHkmQ#i&XFX|G#&~oX_OoyV$XApp=DgON~De5a*z$H1_OugL2=RY9%e}$Spva*tC+Y z>q)b*_;S0#&fiPH?_RNkJwO?&qRowY!MY18O* zxfHCj6m+qsZwiUc6uHF5-z09Q)f=xnix&wY<=Fgi-H`4X7nQoaHuL!oV1=O@&_sq9#(JJ+W^ z8EwF(xDPH$K}0qi$xE{%{FQyu*4Y#bxyOw4u_ZP+kR5kewNUo8;cB(J2o?i_^o!9_ zDelep>yDW*f{$o(EL6VwuzRy^6idVzht(HJEDqfLgD7T`EK~ZJE|hT$ADNa>HY!b< zfM~2|jV#nJYD-q+GDr1LbDh}DKisdcWG|!>M9!kFSPafP1uLKG4*z7Xu*BVd@ifdA zyqaH8VymgC8DH_>&E3lkY`1g<<{z=$U|SGy6%OE;+`Mr1y<&Q6j<_#O?&-7KiU*HL zmGPv{69i%c_?jwMDz39lyr2opKF+R=X)1DE!2isjXD2~m_oeXV_hL#-^4=2UF+<+1 z3y~p;Qn}1d44D&Gp?IGzT7(EZDX5d>%BIW4kKy2q_!>IQ7URdNcV&c+gO}kMkO%Ya z_?p{Im%3JBHKMdKh8*(J{(i@yPl@~HT6I#3$v^I3#5O)e?&9}+-AFT8O$0ncET6lP z!#WD--fPBm?E&#*?S542hK0O4e>9&cV|%Oe?(uOiRX7)$=l-($uaj{#*;)O1+!C9A zYnwW=UUGbYOM#O$Q#8~=@k!Vk59f*fG%OBK`OCtXhBfn|TX6hBogEQMH7&>EdIftD zc(lDs8Qw8MNs7paK3dZHM#UQ}pS9`?qad1EJS&btG*t^~da9(9@mcKRN~{J)eqO=% z(32~()VErdRx~}|06hgL0nXfZ6VgdayHi0*<9*qSe)Jl5qaoRbrg3`K;PBzFoWlIn z$TfD6ede15<@enJsHNIj(kDyqxDh_05+O?|P|5CNLB#mp+`Qn6w~?UWmC>4C=9Fbe zv~L)LtxY9mlllhs`|ix2mtK>aHIzL)3{rqljla{A>HQ*-w?a=ga>nX>y{880O`k_D z%`khwkvI7D(y*_Zs6u8tQx;9u?=XBI~P5W~UM%N|3KlOZe!cQcyOTseKzNc+(Ueb-c9xBdtJ7Q zDDKOAxV7I^g~+&<>v)h}VVh)H-+Z_1%l~@ZXIZ!2`xu)CF3K1*Z@v!p?LJ9zJbO7k zTamN*!DoJj%bhoB-7AOKGm@flKCs$|qJ96P9ekB9o7+n#)d%;fD`=ibUG${d$+F#$du!Oxmu$ag3H&yrd5B94x@Vi0i5`0I8q5s|d+++P`Vr|9*Qf-5N~_@qv;iO9s@9*3-7dU~x&X5O_$88`s$ApX z+Y}R3;AIJxMM|#Zdy^eEHUY3kc`MXm@8W{*4gbP7WsBCG9X}ArLRO!ztykl@j50+} zn30$;iPP%x#a$}O_HI~ZU=D>XWfkRP`5uf9x^^!!sAR~_Tfei8_Gn1qafUs=JyZ{j zcs5z+b9DNoSe2W;Z0hA}$xiaHqPB7o=awAfT|Z3sd*;4(eqWWBEefFYWnDPDAto6XMqBexckIKTut}yJ49*(eGPL%Od zA!*U2mt!l>-1~jWwZ-&a<9ANn&{j+PvL1hNp!VrAzRS}!4RUOjn}7%y$REhtc|O_! z8YimbCy~fLa_T(10ex}6@nG)70g|3>GICa-+GyL3OIhP~2%m#0xHTp3xMza(24*8_ zFN{$H%0r9W^5>gS8o{-l-sz;I!qA_XTiFv=l}w$}!C#Ha+d%d;YiHsOt9z)9-8)Zl zrC0xHf*#)u>*QA=aV*1egnn9UXGL)x97>mG*LGh@bE^iNefCkMX?%SvK~mMm$mYYX zI~>ob@vsNIu4E6x?eE3EN9gs{ir*9c_$?qkVfFe`<0$Qj_YEV7@AI9|;tR5!-8UrF z#GX6^wXuD#(sUbD>z_Yevkj2{W6acl&Bu~r#Bbd^+Qy`VqPx#UK9+FZ4`Nv=$zUx1oNG(vasn!8ocAaL6TbBYKaO%-0{q6Wj zT?pW#F=@;1S7jcMKR;gI6--LhPnUcAZsnKxM^51qKwlPxNj%>@;LNrb{P*Da-xtvP z5;%Y^;;uDx=f5)31@I@HkahZV8fDdgGOem80c8}PfK7>>`uk2DhgGtCuQKhkVEFw3 z|Ju*2i1RO4ikBbz^##-)ixNcdV&(UqJoR^3VR-5vwxZR^aYIt`peNCcs$6}$@bS}! z4+(U&OsDPtpx5~bvh1smSC2WSFdpJJ!@$GDz=E>q3D{eNrsb~OdP%OF9|^Q7k-Erk z02qHGf6P+LzJBteRQSnArsLkTL@VHwm1Um!D`)^-O*16btO(n-5uqgXdc|3{lb2LFQeFON~H5RL)5 zrfM*FGD>v+&K9h+c%m6w!|niszdI$}4$8$|PYG%`gDI8d9NGfx8z>9rgwjw2U{M*1pVEQAba}Cq6VIN{ZVJGv-Ojpiu2=8KY{W ze20n*DwMAw2-v3t7~^d47MI*_ayKC6@^HEEj5(Yf4)enm$}Wu_ z=y=_0TJJ?Vpi4nj9&$$9IR`lV1hp672yBr;LK;wJ~PIl;H?UA zM5Ok7jk4f;{lip>jsu-W`4JBN=8K)+o&ptBeW2JgGsb{VB~@hB-=s$ea}+iC%4 zxM#;chFq{|@Mx5IBcle1{dw>qiCN7L&!_$GOXj2bsqN3E0)Em|qZz#eUSOyD=aeWcS_6-?*T?kURoSNYDQ8B%}C6=$x z2wJ?)=^db=(&Nm zvG0aE+@&RZ6+a^MTjodF8kig?vn<;SIKV6X4w<)HAp(|8n}p1;tX|N z?jUP<=S6X@ed)2@i+lc$r?Gx}*0VzLQ@ejN}DPIZ{C zXK3Pkp&A=?lkYxAdmS2&k&~IXru`t5LIQ|#OOA-u-v_5k_0hZm9$=3lgGO$x=dT$$ zPWTHCNxbn;H^)U}T1^4X4@!>pT9!ti3V9uGFx=AIcWQhRNWsTNhYad#Zb0UQ^jfBb zDs!ik6b~#@6im(K9@RN~0aww?r<>HB9m{!d26u07>X+4o51pnOx^^B$&d$E6CS#4~ z+HJx3Ewus%bICZOxoanwgy*S;05_(*6;lprA;-g8c@Wcs3GiNv<4-%;?TuqiDGxrJ z^MvhLJ;f?4WOVP=$al>-)y*HPdM~8!O}d_DfYBiIRLyRo0I164L~yly{rMF@2G;RN zphuH*K{m*qqK==7)jWx_zWmcpcPfnm5iNCoIN2m6UH+8Te@Nhf%O}MhmRn1}h-69L zvKfqDQ{CaH)C)*0TJ8w#4Qm2z!A=lK^&POEyWeVT#=88@UN@FD_bSG=^ZQ|+GCj_) z=w|Y)YS~{b1_Sf4w?=Ej$)>$382mRY!HbHVcc>}9Af7w8*tqF9a6eu$6+QVdKfNH& za2?8pYaMbpp0W3in1O<(Eb>Yx%A;$6Q6pg5#{5g<#=Fv0I#F*9EWMWes?~OEDvwoHpL!ubo8SxiU z6CKNz^%h0G({O0T>MT@|VjYvumfF5oX0w88N0N=Lutb-!3^mqoFo{C_997;&n>Is{ z{@>2r4J*Ix*2t@Bk+|pD(BoO${YuGg@n+PT?{@%E^ZcU!#K0-739h|T zUI)ktT@n#~UQXQfBg~bKDA4CN+w9i2`}gnLZ&-Lu9oUS~_3e>RG10?)GYuX3IaOiB z@x~RRj3{mMEI;TgZ&-4s|HMZ@d(P>p<n+a>_im&Gmj9E_^i{Nn#h7z#0f=!bg}SSv=;meMZ^Mt{yBH|X)84Gw=2bu3bo^W zi&+H+7CYTz3k`D6XU{%(XV#DTsI>xOZ#AB;xU5yt{B}3wR2>6F=020H>+fPu?g|;P z+$Um!u?Gr8r4?)X?nB>tHd_C5@X!A!*>i)8M-r#Fe2;y8PBAb9Tcc$EhGk!mp54Gy zS%-%wzS?R)dsjt8^b>uT31jT+XEySTv9 zGlFw&vMqy1d?b`duDy?^*_6()Q5uGJedKU6Y5GjAZP*C*R!W%kr(&gxgm=wf5R_iT zTmRIRebVx(EhmV`id~zjXj58i=RWe_gaX&ac--^R*z(M@?@*5$!wT*7#1cU1By;>SHdoj3l{BP zR%nt_PNHu)ZDBu*3fu;oc-JC8^o%`QWe^VQy;)n?)GhYNTE zmw6XPA6z5(i-i!-cKIahIFnp_Qbm02hOf8TY{Y&2vNe}jX*S$vr*0L7)aAV%YX4vb zw~Eit-UTrfmsWmpR~OchJng?!Cd#~EjaltVb8+boj5(rf*TS$40`{bMU5Trd3{1pXimx> z()C~e4f7HDVceq_+`m5r!!w$uxV;~p6**ioL{uKA$5PCpHLh-z2vs~I$Mwx-^d0xF z+uR!3j^}4QYu_qryI)*t&lpbSQcKe4`|Cv1X+dhYXY1ya^bf2u{a#_YlBbZXOm8Jn z!%MeM&*<=rOTdatc1}JqBgU~yOx|-g#|)9Xb$Kf*CLai0TK@RhXTZn0=D*YmJKOt8 zlj%FX%z9iYe!OdTTKp&%2oN6~9?P}J*&nxfL^gXpJUYzK*LJk=nDNrS)3A2fG1+%D;pMxIy!RroBe#dFl_kNV`qQ<^;>)NCN8_PQw}At+r3(Gs4F0 zpvf@obqJJkYMvU(Ey&~?;gk6kKe+N6#No&M>%DKo+6SAJZu24^2D(M?US0dXhW*-7 z{l=>D#<7Yd#IC#=v^x(N0+tLXU!mt{~|`WnbZ6B&eb`X=AhFyjsMuYw^9wnf7m&<`lrdaTE-v@u45B*Q`@@L@t}iE&a&vUZ!(KQ zRp49T+EdP(n4UKGNrs!p%ULwbstsc0PNpqqZJd$i)Jl5Np&pm^=?tk1@)yvY z$J$MAaHKf^(|#kij{}l4us)w0s!qGk5%8k|3>LnN_&Q5b)y?SB24LIb*ARdF)x}N7 z^j3NE$Vqy1S0I}XNiTb9LC8cGP8}UVs#2?8)}*{=99iy*Y(0JZL)x3cl8EWFDX)B> zXxy$dgW)*sH19bs0JHem=beS8Z|)B;9X4^lyr#(walJuei4Vxt2_<~j2yj4&WNZ)C zuxIr~$l=@uB`c$UQ+IZ(w@HmEc6TY*%FvXuOTvmH^R&o`Xp$e$6sRNi{PxpzHQO7U1cy= z9fK(i7BllPuK#EThr2!R)u0g1S;**7aJW%txtzNZKc8u;zY!V&=-}$gnfi!}_6cq5 zsUk`?WAwXHE!^9496MEGDv$dd8&@+0orf==QN>R7J82EEBvlNcTPZak2I^+|qj2p# zg0^)Gag4{`zSGI$#&&=v-Or+(Di2pdk!7<~T6z-u%NnCk2Z3Bt@-EG#r+%=l3dYkWA`DV74$Z+O~*g}n2Dm_yP zdh@k>hIwG!)VPSp&dhSXYu&Q#w??^r#@y zF6%E4er<#4$)n(yfLyYln9vRt)%p+_UFee&+BOK@|H5fyaMlJp1;IP2SvM%A55(-C z1-@OLO0mveGfdla9_1g3_2$71Ctq4JBm^8e=$;WqHTW-2JCx*z9R0wNiFA2^2F^Z( z4(NY4Ts_A*8n~vofz!JkE!CjC+f6Hn{I>@lTgTgO4TCiVmBaE5cgvn*mifh}h57e) z&8kA;6OXy6y=GAs>6A0#SI<`kwkIfF54 z1IVSV(&DDQUSK_ZejI>5SK?xZvyJas8sqQAgw(CeZWEMzMeO>`qJ~ZGwUI(_-rS7C z?bBvEKq6bCAMG|s#>R?AeB(KDQ_xY5u>!mA~GV<#X;7n|GD(`ycOPz5}TIpyemSSN_89e_hpGW5Dgz zdptAZ@;`1O4wY((sR^|FtCq>Xp~F+T|4DTR40eEV>{!wTlo0x5pndNQCD@ct3-L?# zi1-Uze6H?{9Pr(2hnKXqVLURzsd=pJo!s(sa>{xz%w7=|%~DSdXHi-&uGJw=hZw8K z9YnhjX7~XH1e+yg58?bw7}0~{_^z(idUb7We;oaBkKrWv_KYkTaH9!qM#HoHRcYO-s-4ve3W(F>^pzW3bopNnCO#n!DO_&pSqj zSw7&hU3qwMX<6aZ#CrTtx3Z;Cl|z$STWO6ehbu5$!PhEpP}-)l`11 zDcqgF<@!&GYGbjTVTHLRa0^TXyp^UQyb`yOQyDU~*r7!BVKFDwK3uXUM0ZS9kvxwf2GhLv{0kQ%T&hn&-raJq>6etj=)5%2I;+B z7gGhD=9+URQk0%bPnqk>HB2opraa4N>6vnCY`py7ohXuL|Td%{}Yl4_+c zzg2#J@*Cz<5%F@>3i^GxtV6DJ>XXfRlOJ+NA@fWY0RDRZKx7zU54#nl%x#g-%dWOg zt)(LfRVLjMnU&63#zQF1;nDtelYgCwVOv71H&py6(|`8b1}^7mt(Lxi>bNQDU<33s zAY|&3?_^DPhS#{3!Rug!##@p@6_S{Rq1ZR*^g6bI)O!9g5%J8rwc&Y2%h^VcYLCKZ z2L=(hic{noe1VARmH&8NA5i0DO+`f&`jVh3-9|L9q9;E>9Y%mRBSYGA&}frATc}T* zp!t@=K(SYK;T#SL;Q~Rk-n&&!dNfgs!my;Oca4iEr03jG}Xt`a(!C+1I`>a z$ZYR&FE>Fj_^0DN3rnxrngr}91wq@LuR%&52QNAN&cOd>Wrh2^COY15 zQP7FK;rG^N;-(OpIi5$`5BVO|sHBT@Qq9)dX~TV^XZxd*MF?KSE8Tikr(3f7#pm&g ztoLCev>)eG#V{g(A5iEyh| zk=liw?`P!@j!p-HCtS4*JnFv;nfk;c@H9n*8S2QRxpJ@G&3%wD4( zhfsEyT&g`iG`d=#};1!2;>cpd2k9nbtvRKiG8rl{? z?uY%ULPG_9A`?maj=IRurVd`enKyK;vp+j#&wZVo=RvhZ;|p&8K)Jgf0UihU&)Eic ziLQ(ZnkY*yI&~y!VoX;t__iLC;&|er48&|KN1W>JtRi>2&%K^vhB~rW1$%mliz8Ko z#E|1Rz0Zo|9R?&?=v2cq+m(1AKuYFK%L%KDv3mbya`<%k5X3w*-jb)~XO)A5IZ^@j z+N?{UAi8Wkg29OuOh-Ly9tud6CqS5HLFTtT{Zl2})xYRpJ7KJfDo%gX8#)QV(0ZUn z^H`K1s?q<6gbR}gb30k4=K8d2LqZWJ;8~ouNG3xBDP0vT0+u*1&)wP|C5#;cYewhC zWz<2^Kw;vj;oPGQ4h`0B^x>`*XG#6Lmb1ma*Bh9kRF z(o=c&SxW?WfdP~mibaZ^B3iocRCK(~4oJ5VUKcdtQPFjrUL~#gJ^mV7eL6AoiWbZy z{PjGM7pTn@JiVH#3oSFueOhtuRk)3ZaQ^Mp|5;8JJ>lGBo$jFcmRJ+IRaB$(2xz}{ zsMngTjN9u@czR;qHzbL(&)|Mpe+)T>iF?e$mw&cU%#E2F$B4eGc&06PRZ#i}+J{0F zmm{FYYwIsf?FzWON?Yxw6M8i4lV?Fuc}Qqjy1Q3xx~reBAWxxLVU$;6k(GIOL8_xE zPwUS(BNi>(0JXov^oLj<2pbn4(KYLQKyTEHQmWSd9W(~GP@f#7Iw6G6m`U#DWgqyP zu}IRIQ(C1PWkdA*OyK|fj{mA=RG-A6Ew@}S&r z4%{vMqm#Az+Tvt;uSR8jahEwCWFBpNntri`l z?^d}6(g|{&YY^}RP?y)APW>JP{I@hDF8vCK6d=BW54~kwnCY%34c|~z+^#|1^r>mLEs=tHPt0WN^ z(sOVqZJpkaa%h}pkL*U5)D}!|m(>nQqG#A6Qon9>&|2t7|L-;Be+8<1x3lmmz&o=| z@Rv}7>Fwg&ck3NS=xqy%nPaIL`LULn>1s=OJB7HZH=^3=>cp}ka1N)FuF0jwV_o75 z-A^U$C;N4uDf2Y^REOInznIjXe;rE3Di>8jli~ku?&Hb(Qph*2$uh&U4d1Q^^m9V4@l>QbHC6V!eN)wCizt?gohN_ZPm_><90+kOpdzL6E zH81Q6-HUwfw=bs$MYnAMOjHs(J2YKXm}WU7b{GLLjr$UIJq$wY=-HX-bYeBL)sfTf z+t!IZ9}WdR6x4s}A;gWDa3!T4H)e_y)dgs)#R2HNV$l`1oB!VC|8PEW$X9xiGx?rWb`7>X z+*3yGx<5}9EPZV(=PEr-;S@I=;8Y3n_-xyD*IucFE?ZryIy zI)7bxJ^le@mn5Ahm2NV(VNMfWvfV~#%AlbL9P7<6Twi{JbJ{>C-s2M(>ERw3Wy6xUZ9j zPBHEXn81hIN?LML4z@HrRpH$Xw0puu`5*P4MFdK*0q3VQZ<5hw27o;U4J=Pspc zYI`gtdW-v-pQ(VaTKnSd@vz$aVhX$ej?gaPacO?dlp(RL>I^+ss;LyWW^UK$1x88+ zjQJz5PeqL-va46L^ly|(+?Tl!@RI}xu>4G?)C>9t(EV?KYQYYikd$=ZPvQULrUf8- zfX!zuh??c!fA;sq{d1W3zaA=e0%&QHdExA*e}BNAx2jp5yZxt0avT1~O;;rXKnr%) zxn%xi$lp$OlT`3@Oo!myVORbnn5Qz0?ahJTRDF#s44FjtPMXJ2k)6LP>|AJ!?$e*KI=n zeePU?%fOcMtc;GIfV6bt(b7J6z-}_&Eo5*zFs!{b`bX99i<ud~NmS%;pX|f4Eb{f|Ab-x-*5kq zVE;!B2=mPVx_%g-=fyvc-UR@e-aDz(q+t@&Z|w7@a5?|&5r>)?XzES1_a~*~os*l@ z68Ft!V3yV19|zP@c5UZD8~e^B6?oEV&^B<)sD8Lq)8yz2)VYoE->j+TbCK-HF>YcT zKHG0e1p(PUVK&N=f#x1*cI^)MDB+{i6HwD4hC|^4ASlW$v7r`RXl|r9GN z!&(I~20aBJMnxJ4Ea30Yui*`-v}~#o@P49t(I;E}GHTmQ3*OeK;GviKRho9j_$lQB z{E$>-$8~IRX=uQ_yPXL9KtTVZ(Y>V7n;ZWixcF|ok;CsmZJh;%PrE6DPsT13eA^(G zOnpfH^7&cJ_{*@Zvag2o!Kg%U zOF$9Tcjii}B^~#Ub6{t}E?F?$4vdYpB#poGN}H7CZl}Apf;#5CmiAdejC)Yawjz+6 zA^EhXJP5FeN`g)_Bme&KHFi#f zWS3LJhHBTMSH2){**s;X`w$HU}#dAGz%mMGtR50Cp zFweA+$H>td+^Dv;)Q=T^H3h?5ZZqOiYjC65DB-1(lGJf@L|I>s+x-wlmj=J<#Uz`} zm)@vcPyKid0>+|v3T5Cu$5~$gyHE~D;>-%z{N~l!O}M#WNlUyjGCbz=NcN$0;I~3@ zA+*@b-R_SsH_BfQy=aU~6xvTl>EZNVr(JSUrKggNq0^4oFUuwQgibs%Y4KZ2JU2^d zPqv2pwjJ_CRX27bgm37UKJ?b;RvE3Vq)js}%AF1tser(8Clo&B;*2Cax69?(eK=NL zRqt&6logpXv^W?UMdK<>&D4aoH9yVXn9W8BPM|DckjNwlAISdT@X_Ox^a9Z5Fa3eN z-Wr}xC6Ai8b-5+W_FXrPtM*-l=jSdRBAytMF4eRK!n`AUK|H(YqL-F5-IMtu?>A<& z>@xM;kIWR0Qx=tn?mM`Lk?B-vN(#Xd;U3K8fu|f4ipR$Jsah5wH~Y&@sr2DTP2`)+t1c>G z*0k#_LPtK8Mq(Fd{fYnkKr_FU>ZAM#F=2E96;(xhEYMinzg}Uhq&+)Nt>^DeYS*^RY>O^Yc^iru|Xbz9UnnYfsX?+xXk%3uM+H z-Kx5cPjW$=V(E)~%F%Lii@qj)Y!kH&mO4gbsLP5TalOZ__PK9z!Kym&ZOPD6f%R?(W_%7%Yu6Ty048&*;B{oGwwq&b5>pKZylm%X{*U{H~4kG zl+`BsH)z^m>P=NJ;avQgNQ8qmr?!w*ibrva$7VCxhzsehM=I}L+Tkf;B=v1FM=B|( zJREpo3R}4=u3q_F&53e_wKgA)UJkxgM5n3uYl(Yr6(wSOA$4F;8=QZp&BvScPDk=9 z4YJ)vnvII-#fP=N&7))!wWnV;N2a{?)fm+DXY+=k>Br3TM}o_evJz%1wK{!fHQO9Q z5Tl{wt*aPDiDA^$PJ_0k`$%ewGxfe9Dl>n1BS*7g&mJ^(A-E_CV9Bnq+N!{eHVw!c zXSuOzkxcL?{pUb&cdp%}`y-T=O7~_N?-%#nDXvrQzKm@uD}zKKCCUXn)8b7(Zf0CP zX%U+6w2=I}?g12;E~)ifM8_U#3B<+o#qo_gEBGoIJ7~PJT3~p>d{_-Vq&BC=r5ZLs zX?QFqA}D9`|FQSp@oc_t-}tAcI(4ZIiVmZwYSpMsRn?4Iu|ivWrbeg{s#=PcR%{|_ z$4F`gLDHpa&j?YpwnXe0iNr7c-p}WIfA3HGJpccm*Yn@y^^)s4kNrN6^Ei(4w5#F; zta<=abM3W%W7mG7!=@N2L|Fyiwrx6bV|=he(Jqx;CC*GN=w-PV*pw;dg+s@M)8of; zwzOd!HsRrRKQg#m_|@g9eG- z$^xaP=X68_S47#rjHZ}+u4a($jU1}{UGFn+B3*!UFi?Se|PHM%@-v-v%R66(t#;C0)ItEi`d$e zVQqi&L+f#qDjCyBQ*Cmullu}zS9Q;sdp3lqXZC?(wHYd2s`yw%Q&gaN8&0WpX>`-v zP3G?WPOZuNge%(UcB)+nQ#}hI>=j7PSAlZH-txJ#?3&^qA|Zm3GgD zcbd5Jq*=9TuZfxmFqW8K z@GWC=@h^Jn+IGC#4v2AQM)A?c7qo@@nr^s^L|EXC#b;BA@lt?Cvk-FJ9fek6l^1zl zgC~ZTt~FJrSj!$vCYdf{=h--bXW6-1WGZO1l*4G1P7l#N>q7 zMcxu4?2{!xY$`**gJQ`?{#f4lYrB+_#koE^Bp~-;P-i95a|D!^@W$hfkKuG?maZLl zl9UNHA;|@9(0Arm=T%z0el}zQy~CR78mX+ax-q}$wpwtm;u#A}+u2+Itm)i>G0y6o@Y=z103u7Ud1$@a(?hFCw_D#cv#*>Du0VNkzOr$B zAmB!Pkj?stl__Lj(C)n;P~n-Yr>p*+1wet3exg5+XL-ab=2QA$z*V{(Po&URSixFM zsvby(b*S=d4!7!G4t_J*Y9jr04aX&Q(TJQbk2kaV9||AhHcwA9)(**x)C^_!-uU;q z_1~LC*W|r?SJs!_R1cm}6w@olwP<^K3D13sFD&S<2KdHD2BP~r6cbW|3J%<5gP>`id|9vBKA_Bt=2y)HwGQtS+ zIS$xr33qf~T2dkKcBX1$9*YwGq8QXHywag@XL>t_crb6(e=nF;= zn&NmmMh7O=yA}fPqS>6AVJLm}j}VnC!UVUhd7)PW!GC25TyZ@5;>A3x0zND~NxPFm zn&P=y!E=q(goNddmiAG~+&kSyO zxwd3B5w=~YsQ+1h`7gmlyy6Taz@##+EaQ6?Q}0Km-3bnLFkxI>E6HZH$%q%Govy&@ zGoKUi1L@(fZ;t44|$>TfB+G4dnh;m0&vDal@e*E}J zslqnT+`*!G^LuI+Cm%lVOJ0~yp^I8Rm^FtUT*ylLWHu|-u{jpi-+W0TXPo~}R9WrK z_;O1p(zCP}sN*tnbGqCjn|Iy1QoX3X^p2P8d*63`jip%Gqe4P_*^Oy=vTlf2V^!jp z*{ef?3GOZlp?bs|%*@*kQs(@zrq4X9Y-8ED4aq8Cq018Dgb;4oCIyGBY%-e`SG~|( z?XI8^j;|T>E~!d$IwjExmEo&WkQl>+Vrc>?bCa3p*2;!-Vlq=V4WLUoCnadHQsEiw zDow|Qek_n`>@aAUlIg(+kBNzGDwpT<{Ux-5>&e6KZK$GYKjJ#H?EPC8$Q zGFER~=cUwTEXtQvw57})ARknomiq1qybpzxr6*~B{A@<(0Rf8E^(|l|-!~KIV(D7! z*Jf?-Hzy2?FC`XU>FxSuc7}uY7<^te@bJ&-FCL;5M?gSH;YG=rfN!LX4Xj`4U!;)C z*ljZoIB;&|wOmujmmCV#1x;^_8s$mYS1&NdyqpuDit@!+b{QB})prxsI}^)X5iif( zFKRv4yxrfCZJ1L0jp1zy4MGt2X2CQJAzmrerrK{rEi1E@=lWJzqI|#Sjua1^FH%9f zE>GU)b_JDbE_d#r8*9dNDI>0m*mA4a@!(|3jZ0N{_Xp6dSpjfBM4xwYqZch6ULZT9 zW-{M0kSTtC4fz2 zC!Rc?M3Jq;CbH90>lWei-o;g^B)^P7dm?N6%F*KZQ1;zWM|0(BphfZKZqMeIqt+kV zOtGzF?%O$tbHW@pNJB6$kC-$|3;-Z<`d38+02`+?PZKy3Wlwo^{=mis$j@$AxP3CE zoiC%*>gYi8?#%wsXf|*9;nKlHZakmoz#^3iQP%TKsz2S9BBW?6vYo_I9-Yw<3WCkH z|ClGA0u)L0+#u9e_tkX|LHTf+ym$D0rSc>`tnp1Lia&L$9}n?UZI+Tjuy(or(6HRI z$|-BP>EOh4{HjYH{YmYxBYbHw;J(r|K^R-75H`|M#JA$Qwu z?^XH%HGwlq>I`WQGr?ayY^doaEZcc=vvg2lYf^+YJf8^^H!3k}a6cvYl8M9+@3VfA zP1_ENN*9@FQoFZ9T}aM^8zE`Kd-TGessyZg=*{}2yef1XnPkQaxh@AM7__s!Xvv%E zntIAL(r)O8@}~tw5JD|z3)l6L8f#B(?C9g1+;6RJj;9?qYhmfFX;*jMmWsrNbN1pc zcb+ckR!gs}IFi9WLbq<&a8|!EddJEu7rGm_C{2@LM~W%MgCsMelht^LKJ@kT@-Sx9 zA7jOt9(ybrwZ~WJ4ZIv55Z1Z>hYYob%R9v5x-Sp6B`g2qdQ66Jdb*n(ps2`N$)nQc zP1_v$W13K9xq2oxL!a?HyRWz9ynK)SPgLPXXqk?O=%Pbmeaco(kIQBT+8>7KpE7{N z;^Py#&&8MjD?Faf14hq>V(*LX!4+9m0bo|n)^->Bld9I8P4BXO4dMy!(2%fm91R z+mMe@Rq9%JdL2_f=l;VdOT z0;2Rwc3=iKlCEQ{z4%c9O((yyOG^za=g>-$aUf~hMdPkt=a_55dDK7T5}GucO`V&R z(wJsWm@B4970!_~It-G8i{|NDTg4ab3iq zFgun5nWMTZuH5dM1cUYm<@mv@2)kJ`!k}mM>Jc+iH}9;(260MXxyfkW!Z1E7jXHJD z6As9v_1|gNBGtxok?4`CX6!DJw{Z(5#JRC zyu{ zE5RoVJ}q2smgI#_h~(uQOuj7pqcr;L!uAVi4+}DbFE+9;9YXl96(-?9L^-3$pSj`5 zqLiLx|K~IQo>6Th#d?5x*Crsl6y;qd5g2L-+sn#8=3gA&(LK!V>XJ6P=9am3stq(7 z0%->CRodrdZxGUyt1md40@c|&_t)t?7fkNV{W`WD6@RDs2T2C+X-aT=m+d^P@1viP zF?34}8c)y5nA?ivibR>xc}l8r@ZmGPp^P-C9^5W@$e2nQ?O*7atME(_kb9TA9%t6& zEwLokT{?ZK$;dqm_p_o@KH4k|w8((H2wfACyjcu=7|>ngbwBfVJw<6Qr|yC^z{9@n z_F6jUnGeq>IU>Ir1y6QMK>xr#DPu3pljpd<0qOpoO6!jwADE0{;B~y%|>Wnck@QEnH*4oM4 zx`!^HBpd(8eNL^;Y_9x<$*z7qb7AkE!_|lDRAT>B)N{pePZOQOz>u}dhuf@WgG{tJ zI!fnwAF@D4UkgbeL95tQDufvV>KoCT%i?+=PvgIz36cF}m&udCsKxD# zn&XQ~Et@5vCf@zQjOx*?Qz@vo{SX0_9}I^`(YqzGmO=6H;4*n_QPHF&53htwmw<^Y zRc;mYJjCP;!*khlXmmgQsKg{+!Tcmg4oR)1ZDHbzO1&pzDpu{beaTmOH!iKdzChnGfujm3@V3aN(2lB z0ve?e$XA)!=HdPtpf0Ib{S=OoGH)|8 zA)R;O>4%ehxR4tWdwPA)Fl`ZpIz)2oa{T=BgxmyjnCaS4vVyzo%66!L? zg;Z0(ELX+p#*Svyan|V`rVr>?Z86l|eY<`_{n5n!3hJO7!D!`@;j34@>|0jZ#}OH6 zJ(IkV$%``F54;J(3n0Veor|)Mwr#Z$E(so9XZMmg7hId=Do>TprQSW0*1N zRkWte1hbjz%_HWlE^AbwWx*3P=`ricP1QG*!dK1J_sqlMY9)*;GOTW#K5?eg7>5(q zaf~W3d`&h9&%R$TB(Yecwm#5jKdfLaS_I5cwaSZ0912Y@17-#dyA*Rfm=8XZO6aXb z&@11qBG8r5#hqdCQTgI%%hQSkSmEgT+c`OV)jsO~5ozSy*N8;hxJ`CK6T=rmJpjeB zHuBCrv4T35A8X}Y8CnZ0M@kzbk_&QqKCfl1Q%I^PE499Ec4Yi^JurQA#;f$kh!(Si zumN!+(Sh7_M6Q4MeQf)qK0^ZC!=Zy!9uvJw&ImG_E=S86itfz|@@LXav1>Ko-O`2+ z_cHfL5$@PnuejFyw6~@Q$c7dquS|A?nJ@|NYtxry?@#Pc4#+|O|it;4QJn=#lrTG*Om}=-P1fkA?1AHMnrZQj;#%aRR zbT+y;8k89I$ZEI`C<0~pO8~iF56}W>u|i!q$COstT;F%cqq$6 zvY(&yAcA$aR{^RJS&&zvc(&uEiGwgn#vf81iM61iTvja%aI&-+?IW z^AmQ2B^(2odF`$8#!=e|Uf`eVo_J}lHRYa{`{YV&t(z7>mb+y45EV}BT6kZmb92Nr zPpp}b?sNYW`vrSsf^@jgmOq(*Ik(FTx;i(vTO{6;d+P@*ShZ*YVD9)b)MzWdUbX_m zZzt6COdlUcV=ZaYW6S!M@~oy0Fru z0C6b5`3cs)qQz{nJ{F$5R5rwH9ZbG!J|$SP_x-p0jYkBSuFbtjcEgG-$3K_;tL)*# z9%xR8P5F+|ivMfZ-6ZZfE*IMH{w8G!n&eBqI8#2Mq>bvgD!6QLm~@`4mKxg!)(qqR_HbZ&#Wo7}Mzt*{bG;Jo=0Y_%R=XrE8AinTGa5fI<^#RV&~(<-zLMb}yU zTBPh*TO_qbR(Hi`fwdPk>+#mDKh_*jNWBP>sXyt+5Aq0aIAz@$ko3#Lm3t_n|xJ9*wcdpa(hy)=sDlm<{Rk#}>n+$e;Uyk2t;$Q)b_<03n>QN1><1+(>DwJv`0=$wV>ZF2nRI ztPit#<6>dJ)pEWdg?uJ>_0%@id^^HA-`%i+tZS~$IC1Y+6{eCv{2!|4$~AIGWv|r= zSA`%D5$^+qTR3y5)pLd-qv5*&648%N>osmXz$S9qHTVbO~qPZCVgd z&+isCyeRdEB5k2s0jZ4FxVD$Ibm0$)ER7uA!*i;S&Xh8gJyM+Z8){6gmt5pNx%=Ze z-wv%cY33QAZO2mJW(5q)GUIiByseqcw5XgU?b=l`n@MM$vni1JMS;mEoWHLC3bD9t zwdQY>w|(*`ZWObUYzqxt|MrJ+$E5nFT+zq+f2_3f zeOI-HNjdmYOT5ZQX)&qPskzOPaB-+jO7M!Zy| zOHDQ=8hjN>1Co9vgkh_ZMr-*fs*y9#x($+Wq?e1n*paG9ya9kxOzh`g<)A@900NI} z;RZSz+z|Khr`J^r)$QOnLJ`GJ7m!odQnsrjR<_A812vMENN2|%`nbf_EzSO$e#+_ z%kSlUEi>WJI#$wtEM9@(cN|1?Y=`spj2bxw>YI;YoNca*9*RqrzhQ{m=4n4X8Ni;C zO?43;HQ(gN9t{j1DEs*_exRxM4#7wdUyZ7$13oF}IYU3?c=n3>Vf?y4p?dsul`Xn4|LZ6fF0*hBxD>a}G5JPgkuy(ru3M9UnxfCn39%7|eL-u-v{LJmlN~tzCEqX6RUaCf6(;Vy%B5-fAWgh zi$6GspQn2?C|})w?#j;5<8fI{v+SZOHd0(gn>YkuCJJM=*H@MSvKsXKHk@7yLsSs_ zNeWbU4;v40RvsEsP=SXWF; zN--Zo_>6tT>Pj!JA z*^a%Y(jdsgqrDKnjE!TxkU(E2=9f76%%~M-zogCE1V~taUOM&uITp53L9+9xe#g7< zZj$@wJK6|grQ0-vLD&TWSw>}gpEo){uymTBG^p7w;2+xbMTRd6Kgdd@6?cGNta`n+ z0yFQX&{~UJgQ;ud)2}7^@%(-@q8Z-1tqW9W$LTps0F}QiAPcz;H|xIJ8=8JqPPgiH zSl*^&Ieu(`Q%bD(6?y;@;YHKaAOz)ZFrbrAf{#JL@TzsfVYa5_X4gekMQB9qTOYaf z&oahes6Hv~DVH;`H)oEgl~nq%R}xZo!?kA};315q-39;6Of~%c~#(|y9s9{wE zM%zhmei2eC&_GCIY7#r@V~gCLhsSShQ|2Z7D-6uOH|p^m?cF%-jtmyGU=k*24afl= zxeD5mo0+*{MjJiPzsTv`)TP2v1=m*YmGuSTzFW$EraSg!?LKA0|X_KC`$-6acfuvN~xhMCpJ}VY;+dmYe5y5G~bUtJz(EO)@ z?xd=xyV`hX&nRix5bba{!oN?1q~26nT*?&-J-(eP?~Jx^+LkB|UDFxqJ()863(%W1 zcr4>1A;yCDFNId^f8fk|43YaAJ9!!w=#vKsP7dT z5>h1%D=@49+)UG-zmdkh-e76y6k2B05yP0%%`uyoi>ZIc<;#RTg(uB5oeP$-#I7UT zbiYeeyd}VSQ30VT)TL_6)0zXYp+L;ru|+y_+JO3DPlQdn zyreGmCu%?)RU7dL*j$}38{NP!;sz2W^5R4k<;-G#$HxDqkj4ih0A|H@+xqt|_2@9x z4-rL#3JemXK&|^^>QT#v-Ys^2f_*BS>jJ`m`<49TL$(Z{u1}fBqPG{*D`xhM+HXEp zBZi&K5kH|qh`RejzGf<{Iku2u$1`T4#>X4W?0HXp?c;(kNY#s=&T ztXQNW=)jcddzl=Ug!Q=QIP8UR%r`2!4tcQS??+dFE7WJ|48c7{*CG@ zg&GZyEbfBX4N5Q#VwUe)arj?_^ZvI^^PVkDhUD>=KNu#_M-CZ@gML58espO0x&DnB z?{m;Z&WS^Mo{r1vdzs)gc4-n`p0jR^J zc_LDj6pd|e@20q0-|p)%1GP~kWDm%4{^1ijTfgQVZOse}1#>DDmwrB_U}kIB8x96v zF)|Ca@p<>Cq{m`ER{UB)orzZNn01$@Wdg{c_v^%vfbJlWfRo~_b-gu1ePZg9_~;^6 zG0?bP>y!d;;oTfG%y5OH*%dlZM?30jSsJXl>GZ`r)m7b|#{m6_W3l5`aHU&~6TsNv zs0;L7yJS?sEkmDR1CD3jrDs6J6p6xNpVD3&v!u!Xt3b^7%Rkk%=oi?T*%+6reRw&7x`K9g`9a0cP#!&@NjD`K<$p0 zNi|8B-BaAUDl$TB(p?l$4$Yk&POA*9{OnMj1*{&kxl#NB%2#?;O5JBWCf zRGL3|y8i;Hw})AMSjaN4LF|#q>jAnBH-iV{;udh0uJpXSV+sy zQ6xw+mg8wN3Ml7Hlra~Gkc*eR8{Xhoy$Dl|A!<%+jOxJ*59o(ix!rX8vo~Om<*U91VMUJf2PZ6XbPpwdWeL%Q&2X5|$ zJ>B$B>!&N!o%=p-C#GXRpMv^+-4CucX*BLlV3yUv9^pQUapHjMnBegQv-8Dg%ezPD z&vJ!!YbPAFYxT~2dc^0b+(%OgzUuRXAuO=$C#TtqzKFPPb@xWFU>?Uxw5vVV_+{9g zkzgyWW-*U*BZFh4u7YyzW~+;+QrFDL?8oBIKl(n~vg{34*fqBW_9EE#cE?WGpPCh? zuedjVcF5hGsX91}J(Y_eu1>A~&0hbN@H&1e;)0khw%8saiOZcs_v`UW`2Rq}s%Zp+ zO77fXEP33cpij2RE|fi2n_qh|A@3&hx~D>j7ux4EFFT*1e5$;AKMUX{x`IgbjZ)VmTvBW1*r`5}zW@}jTg@n({4GGI1x_iU?${qDwFIvDD@~oLn)3Fy= zRIb<2;7<)2VQotQ!$s1__?6Mu{CgPzftYJA=D;K_-{XHUGWJ2pC&}Ye z&*L_IA=(IIw`MM!* z%XGE->oOZ%@Bad)0#;=eeoi#M_*iXBUG{~&nEOzh8CF{7U3upqBfB3{G_LwEe8RaW z`-Qv5Mf891{vbV+I`r5Gp{Ej~i%DR!32=6ruM%<)i?7FIhW^fMjVqj_=efMHdm~aV zFTcP}M84_Vhj6X4??v1yHajAdRs3ntLW4X3|4zunp3%30dI!tM`{jZz+f)<6y1q`SF}|{}*JP9xthN2;qpSQFNsIG7%Y6 zR~`0t1D^*O5VHz2%9I$2)ZA1U8obZ&KP4m~f&lbTBAIts# z+NQ6t^9qXDa-{DjoIApU$0zi%l(qT*SFDnDM5@@B;JTC;vKiR z3c2#!?HW`iULg`i|afYx6k(H0{eRn_8sE(LE0EiJy=i|8`+ZA%ADH&YH_D z4lWF1oo+B3i2glrb4v7F>*>3~jvjpnx0bweCP=X?ZkQCW>X!Z5e%>hgQ&Zz>-$b`G z%J+x!TFF0zM}1+XE*2bUzTTN{|Ik%uo!4vI7SK%A@$-_RL6Tb)yA=%mxa@EMld`fo zn5(^{{299BN0nxDhQ8&q$@KMF&dyxD-@>8BgXQXzjioZ^XC3@!E&bp zJmOzCD;wLCMXZ=)d^L$G5UB8Kb5dPkUKxZW$ANCFZ7ppbi*=DyL(Pr5>GF2Bc zs_3t9H`m9q42!jR6HK#EPu@)I)&3j#{igsl4*o)Tdz8#EmJy``HxZsl3l*J;Qy+If zjYg_BFF$cCf4{q3{cekJ*v#;(L6gTrHyvEZbYH@4d*HUx>6>`GsOP)f@HMxa%`Uu5 z!XsA6r)#l-i}%^r2V{W%II>^wGTRW{z9Va<4_+<0MlGA;$YUMKG1(XXsrhphE$-Pa z5pid^2OoA~F8uQ&zLbEeI2$A8l_^Lv+Tq3;1Ll5A5>-@J&W0mli20%+i(Y%1IY2`w z>fWDn|&uD-a3^NjcUgj zSFZ6|9bQy_ctgrj`B}d^F5j1|!*qgZU`C~ zFHj{rpC?0hteR&;vpzkt{uUss#XI-Ym!k6yRjP6SusV)_NY?&JOZ>=ZS-&e_5K45%G^Gm~{OJ`23?0p%L^KNdNxtaaZN719)YAL&B zyq%?n=uVLHm`Qh;*+yAWn9C|2gGLr6?At;JMI)j`BAZ5q$$6FUL?dyC(1+Fq&P?kY zpqZeKi)SHg!IYLtRO#&|>{~buca}H&#p|Y`VYHg*QM-ry0hcwN{=YiuF{(cy` zyI0vOZ}QfWh?;B5gZlu>e#qU07-)R!8bBBiTR=35A{!2oXPY(*!^DvXqy1nEF}a_{ zyBn>It4Oor2qMCWT=c>&d0+*cta58RGoP!dp@1co9+`+Y%FZ~S-FQ>+GV)-8jqrZq z&3l5G3mKsbZeoFzb;d#d?)F&&ce-0!M_|W9Q(wB9x_niJPGd^reuct+%m@a|iG{iZ;C zcBm8*N^YXT?_B(w$bVDIe)u5IQ>-~CRCb}rNbxgA6$@laVwemPg*$052}tztW@5(E zG@y6Jp{0AG-8GA+g1)|jdsI&D)8)PnShVS;prSKgB4gg%(Bpn%pSCVQDJu)VGkc-M zrJgHxHo4{zwrp*VvO8vF6(vUV%Bu9Pbw;=mof}4mn3gl_OC%d#QOGr~fr`66AvrNE z@1XqF=#iZsBBg71esyZCRZ+Leg)wG73K_o#7ZZz1dB;3!jSks)$D}Sgr2j*37v7^d zQ4OfB!Pb+O>U-siL4j{P^{(lX`PU&WI)-n`ss|V}qt;PaQ{o!?9zwLRR+}Kw-`F>G z%0}1l)c{NGmU2M7Az!15NlS^cJ=tc{McBr4Fm_(1%fZ9e0=pEv(2_H>5bcxPQTAupz2CkqxarvIUg|MAbi^mL^BLsj2ehHOyBAx`f!D0%h}W!(S% z|LOlVM&Tyvnort!4-7nS4@?B0AJ{kix32&F`ZD93uu0JW(vHCtjnPq)UGM&Q$y?>0 zE4#Z|8-VZ8p#S|>`0xHW)(6Y*5Bl^>m842j&kR-AM(*1*hyPPR|JJ_u?>ILAVAMU7 z!O?R=RZXT#>-nPYVXFTpN}4`IUNYW2-A-sd*!r*)#b$Jh*d|~sZ2%+%6afHE{SO=d zo34LR8gchWjtWi@shf)REAX}QweWRYYaAqe>5znti2g2|4-y9k)W4PfFD7}|{0LBN z!cID&9lP^ckbK|Kb0Z+JBS%fB(aS$wTMgphg`g1HhdduNWCnH~FcU z`Z4*b_7cQHdmZ*hvqjF#bW#$X`?E+!}W2gMH@!;{H_j_61-b1MtE&tHi2-=lp9mV-t z(LkH!sf3YOCQFpDc+00zy=zTKm?#_wAHt*{=ras*qf)y+-T}M5F!i~WpbJX%LVa7s zizXNa=q6FM9;6dTxNmRO>Ozl>yEA?aw7=yXq>OjI#iGysQw2Yec((epOt?kWp%JEx z7-s}&ss8Xn;MxE)Y;`3YItGVOb2fNFFld5#`}vokQmEbG*_tDp*vRQ_7(J#Adm5h_ zD!d&%C7oC+;vx#+CT?_#MT@q5J!AQmHdw;+Jx%Jyje7%wg|BNoXkv|??~ybA2g!T( zQcQmoX9aL?z~6tES4R!cuBza`5lq3pg-vWWG)#%22G2_SUgpBE2Gw*#4AqckYaZJ< zG>j|4=_V#iFAl@T4ah}b=z4~?aKgx#RFA}Qy`-4UHUB-@`3Gv5ROeOTt0K5>weFXk^SS@)ycE_CG;|{L2q3^E%wCDxMnQM_tKk zA_*b#?b1T;?ajA?eOEqUDFviTu17@uT_8oJ#W&ng$(YERFD0C40iF5R*WggF#Q?}e zS)N}lwWV9OT@a%lp{|>1SmzmzZV=@8+nCq}^WgKLE!{M3Mjv_Td<4CgqvXi81@)&* ziQVj!${;?-N3##rJ~7}M?OE^SOB&##_ zOtc#v*0s_zm7njmD^wMlI<**6$vik95cm(*H*}p1n9{Q64n#C_8hMo*gFi+po0Gr~ z@#^gfM&#LV?#${wj4D;=1n!d;{JZuuC)n9Zy8q42o*;9WfNyuNKZf{AfN*qKE%g#MaUmWFm z;+s7{^VeII9_7A#4bWh>l@OM@;J%#k~^dRs*JZpwhi5ReRkW6H^3uaJIJg7!Fv18&h~q)4}g-VthSGD8cp zA%;PZ4vxPH7I1+UE~QZw^t=6h0@&B2^geFv8+s&BB1{X_tWBw;Xuem< zif$trp;yOSM5^?-t%+fBRyXsrQuWMt5z%Hw@Mr~^yo!B*O<0nm9{fFe$f-IG95 zD}@Tm<#~^^IAki(&Y83n$G|ZOFebCqb41Z9zigA@Rf(2cSr^hJzFX)5kV$3J+HEML zEuxC1c6Br3dQ+X1w|z9Og~y}_{fb~00xerty*@)xvp`s*l^uW9<=eHWjaUcXH9>gw zw~19fB;jBbuuAVe59OsL=fK(<>ZYVg;_MxpudI;o7|Pq<91Mzc3|>|K5zchUqH$y> z;fIk`;HpJ#I0PD3z z$Dn&b_atgKn#G*!>A|!4T@9oi_*A5>Rg*X}-0$JsbA9-@LLzB%YpSAEO_w{Dz}Z#N znhGP?Mdm2z)7#dnD#<&WxYsC*&ojd>mGMUVLfd?$(6fo?FIE5LUx!}OSm36;qUF|Eo zg4YsN=y-w)!9Xt}nPZ9P(-zXqhA#(Mtitm#LYe1Bb(;dk+S90Bogoi8h|mr-0?ig} z3f&+&K4O+Xv8@DG>YD^63M4m5iLV`o`OZlubhXhNOtznx&~~eX#!z4(G=etTN9tlU z3@@6_uD{NY_X(}+!)p#h{M-`oW%w4VkIKpa@RR@Y&x7y}LesB2ju&!0C`_Jp(IV7O z-gzNCv(2s@zyi>zVsb;f6^m^#(*Zx6$3EP*L3728W!b*nA>FYVEjrc>Q<|-U31Iny zQ-o3=a!)K^vU1l|N>wiLT8WiFMtsKgDS_U+mV2pIem}iBoM;ds-W1W@vqarKlH5scbiuGF$M7dXtjj#jr$hK%j`^ z^N9wMayK@|@=0)7de7pxoe*U^*n`ohCYbqxyxUzPsg$tkjcLMFd5=vT>{Q$Jy4&2~ zJW`vuDMEGZs3tpLh;~A8xcy(lMq3v8xjBGD#(XQlP^*JZ5e45P*Mxy&&^sVDp^UE5 zAy2RufX_Jm973GBlDe|_L?hMgw;g90C?a}hz&)pE(L2q8oJo=UCTm67QvelHknLVd zZ*gbTm|}?JzgbRV==naNsuT?b0htKIeekTOauIT~{U#ZrK=@*6DB6+vcvudC8yMi1 zd1EoVipuZWPIIgrxL`+XiVuOczUJu$bgbP>v=Np|UypUPsMnGo^zp#sOlB>{^LGPe z6jug;y9e2p(Nqj}7?kRVwp~;}tI6+{uLY0HxU@3>Ax(oI0iYO^V*#ck#9#*m`;F)N zL4!E~+;_VrX;?(cfSOK6apQedZR@o#HA(kR0ju#ZFk|&rb;8J2a?rGk$-DErL3Yia z;nt>`={L#oqhoqw91i$JhfUv)l&`=o%_HO>;aKqG7p%bAPRSIMhtdFO@dk@5MO4^tRxk!=e3hZL5(E#{VLu6J zk55ikY&G);xlfmy8J|=%kpE8@^k3#3{h`P=Lk?M~-#ssw4v~8>u(<}R?V@8tCdzf~ zx7Dqz7>oUEhe$m7osAgNf#{H8t=0Va-7}-f@aasadQr}jV}VO$LY23`p9)0J;a~#- z7ekJ?y*HiZwnD9iX}W+KEI9&es5R8s?3VoskfUChn#j7^CFBUoeg6)bmUgwlNm~(~ zx*%Kc^L1_3$p%iDdb%t#>XOwpW7wiFyPhA!Tk9JV0wOX>y8SD+oj}r%EGIzS)z$2> zwfeBtntar2W@2mQTU4$0IO*f=WvrxFO1MmS`*bG{g^`_|m2AG50_6`GgujK*)KIOd zIA33j_~EhKe=&?aIRq_{KO>%?KP|(gDk|3?@SAW`^yqg|SMZY5+D)f#Dg)4kWg*~hOyz30;a`6VZ0X+7G~<&k zZ_U^*3SG)$I5LT_j4aF7tAv@Heo+(h^)}!&ic6LADl4OApOSX3%M` zw^~=xR`(;fj={h81lf(aKcEdbgQ%uXBn>e4hr#@Bc(GYgSOF%vzkzKP&aR3LNXtmS zQFee@^r5GW9L(EY7`aukSQx}QO)bMhEt?ww5X>G3@wO0eC{2uWF(-R69;K9aswz{kaSpXa zj4B%xY*6f4>@5%^TxUpQ8S;HgW32uVZcxp1hN)_|H$wE_e{N16)Ns(%V28$#UNm1M2$Nv{vh8WDJSt1&Ttadz`1m zq5LjorF+IC7@k3O8+i7Hb>o8RX_CjpQBzDHaQr$TJ{!iwDz9IyMQe5!3Rw|wgAVr6 z>b`H})~DJ#>lX-qM*Q=^9u1oxuCig}!J!~n3RoA8-3U`lH+TQ5fy^z?f%d3b#5P=K zgRoknpu=3qg9Tz!Jibh=2AUcAU0iZ8W5;B5zb-V)EPX@4wS91UC$Jn^SdC~ZeOPh+ zVfp`vwjazQ=t7P!K5Sfl?B2iKdN9?-g&EQ6;rydfY$QxUQDysHksbGhy~4J!%P7Wf zhn2;F;n+km zo);6Naj()FGtV!VV@k5Br+>DvMvlf3r~J9kACrPG&4#PD@#;GOkE$b^LV^Y1NaojV z0lLEOHxhbk_?mj1T z&B|yAB96@)j*@1#oa>i9W28I*DOZ0!V|GD@G28wp4mRc_6vGWEQ3>tmwZ)p-3Gj;)K5-0?!403B9|K#kGsrtP@v5%d<-Db&C>N!sQCsg@R zayad=6)p9{!LmvXZc?}^G9u4gu`b92Ceh!ulw7^w>C!a!K9F6V%h(INKcZm-Eb?*^mYY9$fxCo~dS7tLqtNa$^ zS&F50KNv;LK4t4u=^{1jJYjteozZgf7!B8`-FK7(STtQ~5fkYK7#WaJb#Bv&!;hP$ z5QXO7=oKTGKE-2(>}Uvi<3vN+^@|D9oFbu#^sKGgDL~>eF@(`Iy`zTiO!p)z^`?{D zD*@R5K-`4~(jKSpb)W@^Ja!MZQGZw!VzS$%k(}`u2`&xVz@f$~i^)8*?QYn$$VQ?cv0D!tmnN-}8ZmKhy!FbLT(u3|&;_UeGI zcH_7%Dh21t`1RhnMSMA>Hc*{5nqI;7Uqa0=AWMS3wUJmaaj*Rb;V|lGOnfUD{Vg^b1M-3+ z2JZB5!*Fb5U$$|K>RGm@GUNLbhzlsNqAO@s4ylWApVb|W!Tj|*bI$yJbKaTfo#%PpcV?)6)$v%W3Az}rSSpDBE>S)R z)WZk6ALUH5=c|s=(=w9?==wKNNtac%%?lhD&;U7gwa5b`x@p~|78p{C^0br~U>1xI z!uy-~qsn5)*`B*nmmMY13rk<<(?ck(6h}Slpj{CM*;k=Y1e}B+u-XKIu(i|pFy?C@~gu=)k@Lp7{$P)9d#`kw-D}0i-EI{9{`RuCqSpZLU$hrjLQMD}jFU^<9>s;L zG5g=<5~%SFLWHboiBZbGXbex7@GYJo;D>_jhg`?11sgP9fC_n@&)h@{3B){D*g6?5 zyDFh%a0M4r3L6$Un%mLAiSyK`-oG_$U4>rXz%^@N@CP!PD#AA=jHV&9KbC9ulma?| zY!nYu3KO(?ve6$CIy(q6jE9>}in0hVXS8RURIvVddtgyjr zG*wnK*vdZ%@he7Fda*7c3ACuVrK>Gx!%S~;a0|laGw|FaZbvilVGkj(O6Tm9`Ruue zkeISk!GWGwS97b;@kkdxH)9TRlB|`mr_u;Y3 zCf2(c)Bj;!2{n1EZdjP{gbJcq>6ulwEQ~Xg6Z`7^&_~M6pHgRwx>I<*$gTN-s5-PT zf0+X@m6r(=&-d}Jz=AZ>C4vNP7danNFO!I~al2Z$SaZw9RT^;3qCTXhk~>%HVIEsl z1H8_(A3K7s)k<}1lJsg~8oJy;ce}xKNtpZL=t@HBrAb}uHDchE6o_+w1i@^1uh`>$ zL{+fRGcox2L!h~LdWeNIC=Y#d7H!ee_8CfkBDTDg++Cj=S~F6OZ>LOCY(r(m-RY=# zO~-p8JxneCiXOT>rXKP#xj+pRv&nXxzEq0@+1t@uVT!tPNSfwe5)M(J?@!bF+rZ^2 zbJd}P?KCRJ*cBqq7*M6(7X_HU5H+nZ4=P4DOV4|~@|BX|nrK!&E-WNzWCiT2{M-5c z$Q0x21;BoL&Yn3IV0c#0?JZ&%S`SKgB0ce0YF(c`DWxwBbobz7m`A^ ze{0;oD+=@N%uezWe5sXp(Oru%_K%+hJ;LT2=qMfBMX*=vOIn)Lj=~BvOdbLAC;DJY z{^homkp#d8M3!&J_|)l|&Gnhmp6rGmCnn{vE7g37e9rm!yhp%BI(AA7GqpEr3gVIq zRJC3KRt4EMy6J*8eiV{Mb9TJIDW7SmrkPjI$?Ud98{|*HWOEtBk?MWiBHhcoT9BaW zcUoZ7EoC7lCZjFC*3CG1hkfmK{MNq`$6s*e(tONxL3Hf9oN6v4s;p#-9bl@#Zv$Fw zjh!79*K)(~-*Gj6B4GJ+2&tkl>#Q$T>&L5zM>QdB1g?B6ZTAyeIKKNuam@SNQ1*Pc zhJNnrP@HkOwOfn=m@w2`787a8=iBtXdVw*bVA6eQ8L5VN6#e*%d`LqI(keQ-=$+TX zn-zy0)!yk^{v)zI6SJ6;ZK>}*$q<2xM5ws}=G&CFZCNYMi4m5uuu1L^&I;?JcmPfU zYokDgnv%)Wvfqj%VI^%t>Fn2YH>V>}hn0yBaK^iy5_lbEdT~S*C%c@ijpcke<+j0U1}kt)`T_5w z;g=gZFa_pVxuNN?{_wE)hP&Pi`?k#&)Vo6}OZ?P^J~{YNDT1Li%Jh#RhrU7k=4{u+ z@x;8=&6LyPY%NciQ3InWpw3foA~a=%3hQ)BUubm|b?X6IAd1E!IK`jGBog`6kin|M zMDK>l{FKHNxbR{eH4gj@Hnx9uNW7vPe;FSIVvQ9S%rK7C{JbK-BD{Y}C&$C-qIcuv zi@LL++rP;{TZ^y_Y2Ve|R6Vw6WO@u#0JDx_xXCl1`KSxsi(edr)kgx#QX1{em+ua_ zzU*#?RV8!#7X2p+fm|-N$P7;GNqmfEPDj!#6u?*KjtzzDV^5lC#BEbmPbFApz8@q; zS=yB??a~Z-Pb_!0LYgWIJ?lqjNLUi*KsZr$xd&O2>3rW?mF5gOOnymQdoAOL6}G9% zlLbMItZkNSM9jqvdFp_-rQdTlU1#_!B~Nb@7)!bvQYP-?d6-!=?gFds!w*5*uLBq< z=j*%2dwD5PXI-js9L9SG!!dung(Ph!@7r!qXX_+JT2Gvs`1RbQ@jt;DV!P-bLCQv8 z=IQdOy6agvS#6u*4+pFeoXTVFos|14vv2RJ^R47l>z419FB$^p>c&6g<-!o2jt6^| zbkwT(1Rg8ZkQ52VdMubh%!bDl3v@%5eS>@q$#d1bKCtwwO!s!9i+)xLJEm%glsGWx z#cyKFx`|zFsN_-?iFOfzU~InEN%jScStFu_bOeKDaCf5TdfD8Ek9f7Y?kFn}IA@=q zqwoe`z!-%TVVwXKqJ-QaCsvI}g`<7{WGR|v9v$sI?1lq5!SR| zP04iGX=w%+uT*OBmfB`3|6J>)=+Kz~$^mU)Gy0U|sdw1tp1D+gG7OjzWIp-JKAjktf&->?g7J(5`?Nuc42@ zQ<0hv{kcw!t<%K8tW~OjUY&M0rO3=!`C+^oCJEL)U7F{Q zd-3jo9ByA1)1BjwD$aI%xA?eanVaNcO`n(UyB`iu<>DpLFM!ngJ#TR^PKY|ybeomX zE$@@BfUALxrGjZrf@O$*+u*c>jm%H4-Ba>vwV-fFHJh0goO#NSJCYBao!t?F8_7)X zV-WbwJ>)*9;s791HEZoe@z@}rg+fP(WJKE$$<*J=v8gv97Ymg2hR`8z4?1t$CY>u% zytuQOQc@xFAo1{6AN4L6r{)crZ7}AJ@h4aiJNpnKADv&ia2$pRx5E6j3Sy1G=G-f4%3a@qCs zd;|mgh%w8wu7oO}aRs+7aBBr<+H~YhLmGKMDbpz2Kt$F{>%z*W%aTrFUca9zR~mfg z7R&8ctqG48V<%&^=tVjNW#;cRVWJ8OQXHC46bk2vA8a4prk#?SPu;u9o5Ed<3xWR+f?A66F_sst73G(@84x@F~i<*C3z60~SR;JtgyPuXl z)6k7|<(hMD59?=!)#=5khUd42E5kd#KeI*$e9-2RWfzH^=nVFJl+1g_FU>6XBbvl0 zrsy3^s+(28I5(vzj#?NcyPgwe7tm$>uO{0ES2xf4cc8legXUs|Re`%4#t2p>7j}}6 zpsLDVS^#6o!)b8?Io`9f3Px&xbo}HFYKR$RSj)e;gAEa z&QwhP{-b1yiz@jxm*h!a;zZf|^ONUzNA2f!-;$!t+LdVVo)Mn<@4Y`CPIi6p0ZHqF y4|rG~tm9%G4(ljbN5MJ@)=}{P2?f}Yw?>on8Td2ig-XZNJXL!M;Ac&nOX4=63Ez$Z1Hblk0x zkf>rf1bi>KAkAw=!IV%_L%}_RV^Tvwbwox{{P;ybDRWY90};dYFO z{AYS}mz6ryH7sy(!ks9vL#fI}F6`i;-qsIpN1!UmYPX8CRg zmC@aSci%}pF#ID&Dfe(8BL+RbWkAROjqx~PAj)@lo5*B-8;^@HK&=7^&9eEW`2VCo zvFMsKDp-LEyW;Vx9UEGR-{Tvlbv7Fn&}r3F3w+y$K==|&t*atf)ZX-Hns_gR;aM&E zI~D3z6y5?rbGQX3G-#dKBPOhUfFSn^R!rMr%===@sFs5^G+t%70 zbq5wJ%L~WHyHMd4P*kwx+aH%FQ8ey)%zNfVbyp*5mo7Tpg)B>@XMtqM z8w=GRQ2|sVcb1_+zl%uMlw|lg^?!c}x&gVz`!kUlPr4Ee+<$OEPmtgglLiKEpP_hD zyY}yS$%#L@poiB8&L`h(t3LT**$(D+72@nGw*&pHO}^q^`yuajAl&d}U8idB*K))p zdelYyMcMwNPAlZqYCG_N#eKr;kk+P6w|pWzYcny5^X9^^_pho2c6-|7ydHQ*u}kH3 zOno>?d3yYh$L4x=9bR(;hTS&iIeVrtwAU@34Nhy!|mf(5N)=v@! zCE-lz0%w}T0<9uCxi@Y<#iCyy-dQ|7s!(q^e%nh&a`cVyFgco06#2Zpz!M@tvEc?_ z^+DKiqu&Yq54ZovO?N0?i(l=1YRqze`$|+5t&{ zzGaPHbp%NB@ZwoeIizN+{#$H^UwmWwLi=;SR%b}}3}*D6n8RwWsqqB+5G}HXdA!DU zGcR!?c>q6UH{#VL1{VzZ-!zQ`avzWqCQicM#K>7|9h?$Px^rZ$+D_!)TlBK@jx1#Q z=LX!ZjtQMW;4KGRGB1rL8Ct^JD?W^ZlUmfPz^}eZlS?ZCsqMNfKzlsc;OyJ!ve5r# zaKZs(7nRaqEaP*+HahS{PsZE@V&*xdLDiTfzCBNd?v*KoG(BWDzm-TH)7pPFj9t(f zKa&GvlB1%ZNj_S>dFcvJm%FwM_j#VQ;|+SscTz(dkO_kpQh`|evVU`%;NwHV)h!Ae zf`V;tAnEoEFWE+t=58PM{DI>0wAGy9l3spJ3dr~B6BK{b(d9o6jywgtU@KYUZy)iD zFYR(7k1Tj_#F+ATCipY|G1W+oXlJUw3Ge~|sw+~hd++NA$q^Mfu47#e*&u&MHFx}~ zgeqV32~+Lw=@;s70@1H^Ul7%um)!I(e)H_ja`Wu=T*;7h!EKC?;{8otU{5OSi{gLa zNi9B~fzzgA2RqnWbS%MS7=`XImVM&brnrOg7!x$4##o%a{|-6ZSU);Zm54tVaG42q z`OlqD1i63HrtjTo{vW5I59v5v(I2?lP)JrXHV+0$ZM!vn;$}$E2js9*JxP$kuW2s@n^J6wNRR$U+9Knlc>DJS7NSk^f2}SWmBC_S*dji*2 z41hJAsfqflJ*9Xj*N8g0jPUm`r0g9b1;02>IOaG~Sf&v1Gv5}i^GrQ@`^2`nwV_93 zy6wT1c$+CV9P-x??oJ!vN3yhpsWuk25CZf4RVu9f#^p6^98sTP3RMtt*Lo9Sa!*0Z zy6>CIdj2bGZ=%pl^hQ15V;NF>v*|bI9H}-B*q7OwB2fyveB;6f=4cR5>cMIzAOEf* zQyOj0k!uf6U(*Ef5f@NZ(eX?=1>qwU(HhOoZ!U>ZpTDM@O3lyD5|htSodtLf2iuxo zYlbeu-*m@+9@YO74|Za!xuw;0)z~wNVmq}&0r8TWe{-uB+|SusGHlX?3V%0oy?r3r z#Ad-ZC$%=AE^U!kd@GhwTJgucTP_>Js`^c}HW+FDw#2 z*50tpcVTKG#7`6CQ*Qnit;SE^hPIthzNwucisTc8d5N#rt4?K$C8 z9ap;osN2ju`oXj`H1TK7`sLpfA%%%UKU2(*K(a6Gdz|biUpY~`i zpU~b$vGE*wVz6Aq$>8xjAwA69NL=cBfi(`mgn#)Q?;wAV0F0Ep z%Rm;Lx{9l4t^H*NF$eo4jJpyU_thl(x{dA_xU@pbGr=H-kTs5*xak=D+HU}UIk4|C z_o(IiliXh5L+uq_qG7QO*kk<+i4(R%3rqc^s??H|DOPR&pX~PzHVs=Ab7)X%l`pbV zru!I9*ZzpE%j1QCN2H4sWrvG?Q}mx6mtTQo>EPq{66wvQoU^u#TWr5i6qJ$_&1KSG z#Cvu(km&akjeYN;cwzMYE9fhM1iD_x&F{t4MP}!2$y8BM`sX&_Ci%~OW z9bn{U^nRE(viUg&evnxcKtqIGaXdDg0FFNX#{hQq5W(EMUdM(*VD4bbI2-{XlapzO z^x>04uP3kli1s;ETog0ptxMhLWOHH@!A0`FJ$5$&EZ|r2wU$~ol~EobVVrzB^InWB z^nDm0_;O8%9N(U2Z5djGMmdA#G^t5G{E7w#00lq5+9LZOvHbPM4eXs=L$Q|skoP}b z!jPoGpk{wv3#P<>Q}f?~)PI843iw+KgxWT8|AR03M;-Pq7#vEWoPM;(|COA7edrJh z_Qh@%HH`l=Q4nz8>JK9pVJKveRpxKx{EMz`Lh!d%fAly0=luS1^GO&i1||~hj1h`| zTu^@+?7b@|fjuc@GV}TZ-zosAnN6n#ce4EA+DOxgQOch%O{3 z$aFs^h_%{4$;0J@9PKmz%$Wx+f;z+Ndf&Fm;!*Tei9X}Cqp+u|Epu+iQy>PN$&Fqhw=`nPj;z1=UY?$JQ z!jX*6N*k8-C%*FO-R@voC#StT$u^{AztnCci8}sJ#6N74&_hM>k4K;!|5FM0MZe57 zb^Su$UQ6kx=aN=&N}$-@Tgb`gBc>Iw!)Mw@X{otPRWjWst}~sjV%xd+=HaxpUdOr< z54u$xQ7d{`=RL6$qY$jw<7Yn2_@oDkv6t>?aa?%<{9dHFWLX3wf=Gx zO23`(AMKa!PU#-^%1PiD^Bf7Nv6oqyL{D&{{Cv@zWJ^v!DYCIDXi-4m@nS5FKPX#h z*d#YQ_zf@cyuZ-eM0J8R!Qoc4=;{R1q8?-i5XgEy;iFY)NKo^86rpunzH#!H7hIfr z6A2fgsAY@PZc>l&<6cfHXE893){^>TY<4Ksh;2^W;+&NKQY1In6}IZwl5#}v3!c#8 z96>LAU{_H-|JA`Nr>(4b@Nh*bv+P)-kICr(;S^BaIP)g1b3R zS+Cil+WVo0!g9vcze2Z7f>&?0m*#FJ4R?P-X)yIh3_be4d;vT}Kp0XbMS+!sabdC9 z8^bV5>M4-db3`%x*(*rR!@YXO{S_4epBd#ko9ijui0m!T#H2$XYVaa`)CsCsy{bfD zX3%Fe_tU57&*VxngY|@EKr>PcZL+1x*uNvmi^%ll=zmPQz8jW4Z``lJF8nm?0QZ5B z%27(OVE$n>Q|+g369ZEipgsLhwbaR*h5tt*F@&L_kLJ-_ys%L9b$RP8=`!6;hE@1; zUS6PZXSz~@gE1Np%QS1qoV;?aO>ghI3L^(|@TSeadkcmQgl5^c-O9DFQWm;K(IHWB0kY!$OOMApIHDW5;)i&;^D1ZW&o!v{OfN=6$c*23k}@}jKSaBOEH5vg;6@db zASx1LFNQ>FWOgtF_idx%nA`h0$#J!~lYPl>ZP&8SH!0lShi(jiKAFVZLZ{QLuH;^~ zR=k;|FlR{Mem69+qImG`zd}kJITUV)v79-j-(Z{g=!~8h!=0t?4RCj5|5Z*LjT? z#fQR?A_$2Dn+BprlxaAVPHb>qgRX#vrtWa`P(cq{kI zOGtmY$+;@C9RE)bsUosYdeohm@w;-TY0RH~*gld(T{VRdpyM<45H|-#u1EtB8qG+@ z=hDhAjvH40^rYUw-FV`}{I2VSoOOg8uY_7*7Evh*M{?z6h&#_AG%aZO%r{o*b(ZEl zzc1^CDl4GMifV^`okXY4@;oSaA$WxI-Lxp35qn3D=Dsun3d8H@fI^_&Whdu}ah0O4 zaK=`~AlO3q5rn;VV{}`7vrDgU)t%N7M+$CDoC&V`%q+5n*_Wc6^F8q>rY9f zPF)u}FP!^U1rHeB4%*Sr3+i-*k~>)ZubX!QL%NN+?mRW#Py_Pp0ha#U=JjkW1e@aN2paXOuAowlGxm_c%l{mTGvzA zd>%FMVj90RTd)=Pnzt`ixkf>4=1sk9PfU8Vo=dX(c-PpIpGja9V;zk(?>%d(Ts65h zD3n276DA6T2!Y6mP2nb<=eaV|TwoCufM z2Lkq#kbaKcScXNapYlJNudYp|&Dpl^zVsREa|Cav=HeT&Q7Vkoplnq(bnxDGwR_L) zCgylZn|X6g48i%hwC&BQ+jaR}KX920`QuY;d$V&4BoGTFQ0dur0jfWIG#a&LJ^JxB zJRSR?1_QS$V_%y~&}QyM5!c@$`@M^G!F+7`bLZ#)3{<2;1*U0y?e;WRZtC%cVZfFn zFI^{y3~H}yhw3^01x|HJN^rtsa>m<-uD`~g>;mT=)4<@vK67CY2}Bp=d|K-W>m>=K z0qOV23M<1QEgRc@9Zt@l+rPiLOowdU`>DPm(5SKYwpusoDf0@H%(?dMK?HeZG(I>CISzz#6(`H=OP-YEH#sI-Nu^$NVDbRy$q;+W}iN69Ki&b&q(m1WI#hvsVcyI(6)`yb&I9VJLN*su2~p_17j*6grUR4 zKbV#ss&p9PM|R4>Bmj&}^33U;lQqTsx`9XWp>!RfObylry-|}^j6xuFn_A*$p{P)0ZU;1ZG$$BMzG6y7bbN!q8X!{e_QBkxZVyMSK`L29bC587nLRS~_ z%6&B-T{47FK1QBWy??Ey0+K4h7W|A9KSvMOW|zyGvq?@^NNbtufOc{@(d0KRYzc*S zn9zQB<+B|qyXiHS{OLHUT}so57wd@b}*&ElEQIQcK0M=P--nF-gB+4Z#u+y#G^32mMKqFCtSBiQIT{k zT@pu!!SmQ|yOMG&tWbA49-~xQE05EL>*ucjho=(FB@iY+UIoMO+#r*Dz9~T>bRg~h z2Wcwow?C+A7m!**>`#y+B1I8!^WINN>b4`3LDO8g1#!oIh>KP<8%?L)XA=K4^nq*R zwsndfh886=TT50}){_??b-IE(txW8p-ubSSh?i7_yTaX~@msq$&u7A+_m6dJJOl!Pm?;gsWXet8-S`@Ei&nH85UcCET_$8u6AW#0=8H z5Yl8*Wry!Q3Q<{3%1TRezOu-wZOjfCd3}C-1b%ZTsPpPla3KXG{Um}=-H{u&prv7Z z_YNH-X1EnmV^;2ygASrig&`Km)m~q$$`6tC;zZ#I@TQ;OfgqB;yZ# zF^qpW4bPnHPvUdLRxbnsW9UM#b(%Ro;y#y)8X^kE09)|u?V)t$=NBpywGEE!Hl}LwxaW2s{0uk zFIiK#+B2+5iIj;jPjFWP*!KO*wNjV1i^W7s)xaR!WukLMvU|JZ7A;X+nhN6wEDCEgnjUm^GPl?#sWZvCwvZ;>!|42h$5dB-Ht zTu#Vaf&2^W4T|%|`>TEzaGa`N6=ixA0hiByx>CTvKWD=)`Q6H)+nI`GmMeWL29vJ6 zHbR;__9i{8=bTP8aAavZ$SXH!yFA{Nz6P4dK%ltyMEbMw)Y zim}9f9MLj-;t!3dOVnH+@WNMm1a{a?1Z^4l&jlRV2UX>S%V4E=inCpSy@I~4SsW~R zVK-vA4c#yAi69BVsia!JnMY+N7yI~nG%`tFP8!9zJi%_g(C7a^CHF`HLK443h7O`k zK-p72x6F9G%-xVllL3|pc1ahvm2o4Dm0|%b#TbgduR#Qvk1c>yX7Xr3FGhi4$Ima? z;)apZSv4IJYDi2PvI*61gEBTI6|N4w7=3Jw^V{G$62E_JUn%usPT`aTw8qM1jKi=6 zF}I$5x4aze zlJuYy%14#)zoqx;OgP@KH(u$3F_DhDB?mK#4Rydp=$l(r1`2f|Js`n2FJiz}a}_l^ zQ>^UWKn4?UC*(*noXG+Edu0qRL|@Uf-XiND)|05LAEjeI?7}qKHI*l6=28@nmuC)p zz&UxLsMd}%(sGi@=OPqlVeip9mH&3eUV>TtCYP?k&8G2n1NxxW_olVW>O&DgnLBWEIi>=O9& zgzc6`ifAz?7A0)e`Qy%W zZn>N|a7mZdbM;mQTms>@R7U+a^0J2gm@ZcKIgEC#Y_cEi6kjqK_+pnRnjzb~=KLwm zSqX}#$szruuc#|SHV-t)#kzakz@;vywRve8Jp-G1N05+o4}cr0W)^49=~YE2)s~TW zKPu~ev#ZI|SzT&Uxl>uJ+=$#h%DGTABveP)Apme#go6@Sy?5s=Ej= zuKY_ux_&=bi@0#2pes=rhEAcUYrJ!qe@C-ZM^bN*@0_deBs}ao6>TK-K)vlMS55=4(ZkPo z(n+S<=M$m$=zFrX-qZ?*{@QQ%Ie}T@`)``GgBiu_RTcAlW2>E5 zunI7BSaEXYu6*oPt8qCRkMjtN5cBxv-^YB+Kix5Z=Y(!nvOdRua>I7>0(a!M zhQ2|ERTb(7>4zzZ-ihW51-uE_0FNOCc$37rU#1m^8wEv(_Mqw3=?ji-bdhrG_ArV` z$!S`Y$oqNOG!ik+NbgqpYSU;qTv^)EG5Fq%7=)Rend&j*;7k|w@e3Zzc66xyBu`Lf z*Cs=Vl;Djev3WI|7|)JPI>qHgn{+Ld4JMzsQ1&WbU_5s7aSM7uzQSIvT(5hpCJ+GG z7qt?(C#mJ9tiS9W%e!bhpeU2Gb!JdcXrCrj?nb5n<6SK+71&{%*5C(uq5sq6^=|`= zx@*&T9t%n1UL>Couxo!-KV~}CS*Lp<@{$?%ernSL2M4b{zQ&}o{PScgp)P8TTumy* zRBxsRs_3Mtc{eQB!81j2sZ-_+B9ncAd{j^d8KPhs1!bH5ZhhB~I=<;~CGv17vtOg7 zhVRDM04SbIFa=|!TkKqD~0dhrHFLuHoSD8VmCkK9^dyga_dOVI?mC#zq=>C&~k|8 z{3c78kz2;t4(-j*&h@O=JjchXG(I+o!$2JXB_Vv}MM)s7;PI(h5qjtDSJ$~?!Ud^O zx6gz$|K2ql(Lfdm#RB`ub?9`q1HLG_vB1h`8vZ*@PeD21$$ON=c$&vdRJO zzb|U0O7)yA?USDv`HXopYagyZ0nsLtkQ?~Ly3Jb z#eN>uM28HIt5-=;o)>;oG=1cuwi#Rr;r2?Ysi2-8MlGy|)=jAy6x&Pg^8apB#Bl-C zFrwEkW5p|%Y}|K#Mdv9OjWf{XHIuvxA;pmrW=FD~=GNf)_;j~yuzSMnSENMCqR_o^ zcXEst=7raA*|bVxit>&Cj?{TDPI)Lf(*m~5+`+!kBUbeSkllb;s;YO?#WqgE`@F<1 z2GLP(8JZ*}1ZinGqed_%$l>IQFcq16>Q$-RgMIflo=z{)+*QePi+arTa_f`4={9_L zjumDX*mT(1qf+LE4JZ1=KjNBY`A!1XVSq!q z=Am~>9nHV^jNORuV_%}JVI|&37s{pE>RVLO(X0f#M1gP0ZyKjv;C6CzYs0D;>%&xw3S- zwb51|7~uYb(g_EgNLcXI3kzhwnpt6Z0VM+Tz=(QA_D$Wj?h;5bd)NqN4ZZ zdQf((Q`ne+81144$!kR!F=A)3-}afeUtn^5n4PhixXASA*-t51QjIC$T1Hrx?s7ZH zUhLZkXyyo7ypRaur_GrQA_wWRe6K)qtQSA_3stVxBr~AL;Jg`$q@KKwA~{!0&E+$Z z+IKeSPL4W!H}_euYv3-Y+Hy%vr383_kootS#fO`f`wHyOfdlgDU*6E-S?&Y{m2-L} z>A~urwEBc3H;+_lGK@?-T+e(x(O5ZKrmvNQ?-5%+pwJHA0D}U_4?2X&-plLsa}c5p z7<^T$X8z{wl<~_5+v&ljhrDpa?{xts#a2p2U}^>vAKQo2P3ShSJ&O6gHhU>`Xlzcx z$O6C-o~Wlj6S$Cc0V0|5a>Q|=w;C1x{_ul}O z@Hi}mgh9$dG@ok!Ru?Hw)gqbMNyo-0yrDrvHKziF==3LH1|(a3p_fP#^L>x9Gx(gC zR{tb8Tt=$~UwH+Nk@sKYo6kRzOVwl)$$x4xrwtE3?9VXJcsFL${8kG)s9mbu1a^HE zurubqx->8(Oy!s%7b_e2!)Qz28y!#1-!SMfNh@Eo7~4W)6Qv6-s%?-;JfbrvUKYYg z1V7#~w&CC@`pph$dEehy{^{5p+V-Nbs~{_}(TzlXHvB@&^RVdl?V~L&y)ph6$LdLO zECm3cZQ4n+8zb;5eqD903jHF+OF^j)*S_BeCsvG!8yZM+`~=UhB+RE3%zHQJ-W2?I z=%gOB0GR11^Kic)WS39pRyC4vUC{IMv`krQ(6+5T=G>OYr7b+Tf$F$$Y0S18lAmyY z+z&fmhB%`&xpa`IsYcXyXhTI{$mwbAsO-RYpY=*)M5tw+&6dw9oFQ$4>IabQ$~e^f z8}*(LAzN@Rf9{qOO1$pTZ|FHDNNetS+vH^EV7P9rHCpGAW|du7Pp}VOA&I+K)K|gR z2e@)#54*1o`%U2CZ*r8V_U+D~fbmGw5pZ!YTV8~CVyBkoNH9_(DSgaaMiQ$RU9X_V z;F6-z@j4G~jO2Li*tK*@NlhN?B_+)WY4NT{FT0LxgWFk|z~kVOy<*1V-DmYHnzQM* z+Am`o+io;(WVfo+J$^nh-N_wfZyzw=dfiSMZW<%^P@#&cBE-kgX|Fyyovob4GcC#p zqAe`zQBZ0ZkQ!-=mhEnfbJx=Y(BfZ@rXC3;JGX|^Gf4=}dvD|%J*^xr znBMSxAK&EiHkwFUherW-SI5$f%n$()yit4KH6_93WWK$o%GHC=-$^237;}fi&FvVG zl?JM0PiK>e*f``BfQbd+mWr`IJ}iXRMc0$=p0_WeeAmkoJ!4;9hVD{QzW0JPQ}cKn_!@2ZjLUy5e(iHiTsbVm zQskSY(N&PzjIqS1YUL~1`bx-uS$qbR2*yqwKg}+d%6xgTpQ^S)6yu(ItyIlcld|J; ziSEt;cP-pP&AtBgwJ+$PUaE<^5e0ypoG>?K;f}2DQiP#$9;c#D$vDbzs#4ufMiL~X z$9{#;+0kyE>PUy%*%mq!I%(x>+arpCl9Jh`!%XD9*44Sb>KEtPi(}o@-QpZ{m@BWi zmGRYapL98%yf7D#T{1`&tlDcw-fT*{6gw|OQ{!&OWIN4a9Fd-nfu3H(#y`#l{5SEBUXX5**7<#h#6-Z3lY=e0KzK zRPIm|01ryb7BLc3rOy<_CW^aBeEBsRXhZTONR|Z*MvgnRE@*B)m5X}nok_Patv|{2 z^09N}Maq4(rO!z5mxNjq-nbl}Vdkrt4mHWrV7Pt$guwhIfCiUm=U2_J+Peqs&uPq+ zUPBmju4cff9z_9B|1HNkobn$sl+AYcW;I{h50UK=<_oxBHkLP>H{Wa{BcS=rs)V8o zW?!}Y`Qxfj&nrPq4mlb`#h1hPG8XfOWt+>^hX& z>@|84m01V8KIJLJsaVx{M!m5)^X=RxDC!U2tTlh&b`lH1_~PVDUS}MqG!Slj`nYw! zDLS|RhNlZi8ktORKZ&b(?!Bo7!A^btE!~X`S8q4>sWZJz`cHdhH3^3<$`a{dVvTS= za%4M8rrQ)Mqt>)%q$;j2+NEOK&hPY;S8 z@~>`Rfn9Z*Ul^Zt?N@6Xxov!JxkQgcQROyRpp|igB)1eKyKp9qM;@}@<3;XWm4T@9*G$5~_2!_u;L@Ig-37sxXj=+@s z*6qP066R*Tgmf=}Sy8IlN$e-!gM$Y;#s! zo9I^WR^OZ6&`$ZFVc1FhtmiJNij>>))T^Gg3NwFvgR%n1Mf5}Q#XTER^Cm{@A_mTD zEE9zsV*}j!X3ctu{mW_2hrV{s2?gD4-4nW_Hs2P{>&{v{zgsM0;2RA)m z)W3etg!e-`v$3(2x)OtWG$iqzP$bXQ_*7Vih67xe0xJX;96xt~5a%x)>Qj3hy1w&H z1w3Q1d9hA#2%mb42LeH(5`owo7y3;aUSB3HZZ)j;Zr&X7_L$z9%bn7@+ej9S8wfBk8jiV@*53ypdA(Kz46-ufL^7e~(sQ#U@F zd3&J8M(2YYJRbnNHJ3N_sZ(4nt!|%)i%YS5pN{Q4P9C^+V>;(~@Dp70MTNU8(r#9( zz|hlBN9+l)3BxV zW243MD`s?Vvxm9_;dtYg+;lUCMfS|y>R&UVG*}cXDRlbbzxcrN| z3AIS*c)QoXamje;`Oh~vMBAxPf~=Zje&j;bpU-c!*tQw@!Ina;WtEp;}P-}e1m2i$v`H24&I7kn- z5=6K?JNsU+qz3->^m_5i>4k%EH(lzy(ee`+nY@{TkFr(cDY-S*S)6Mn-tvq&zh<)u zBIj>@DjnMz7q0xdz|KC)PL3 z{R#;*%8zKKdu9x_ep;z#w9HOb@Dd2s$QDzy!B!-C=SK|mDJ22+7DuFSy+e+IzUW^Q)%<%S4BPFTEs|HATYdA}Bw%snS<77fAl z{v1X_WZqMsd(`CWQ(j=>>*&D))-*H0YoJ>wdD2XpDtf!%yDm7O5Z_$U_Q3?m_a3$x zC?#Q`4N3Y%F)TGW=#dcm)ne8%7~l8BE>IGxQ|Y^0=;u5|@K%C@;&;@%tJ%|ZJC(&B z3aX3k#!4w$pp$d#q3^F9th?NE%+_{N1u1rB>9YgbyrFRRK4b#CO*f0{>q8|Li{QLg ztyELdO4#N?GBTzmisvlV$y654pn@mUc%~4?SRk1$DL!MCZrN5Aw7`T_#VjAu$uODe zimTjDhb2|HNVX^bLbJI5ziCcYVXI1RUSsEk9YJHfDg_Sauj0baM5!(M+p{y45Axs9 zUWXXCq=0@47?Wn6f)gQq#i-!bfX7%-b$0blP;jEE8jcS@5H=^-y~3z0qKr<3x0bx5 z8KLt%Q1tulapWx|l@HP@obG%@-XXoii@at=;2KCP|0b(7enFdMs#XqrdWM>2XjTtrMcP!1IF#uS@^S| z^Do1oN4(znBB6{nhx8~9n*%>;?T+K+w(@ISR@tHsP=&hXUQ=jH)<+dv0?jAqJy~Yv z$rfZn9+Aeg6?(?ZxzxS|Y55ifAIo|L-q1on`=r-9K}vJCmjLST&^Dh7QP(s`cgLyi z1V)D$`M6#Q6N8b&Z=1>|>c>sE&iu}2fx6A^GE}j+c8QYW)6w$TuhEW- zMob1hd*K`=^J_tp@ZUY^))II*Zt~-X5|Z&vF2Wq0AZZoZ?Oj0`c(HL-)_$3^i1BPm zdjjAxA#V>{CX^b1j>eoUp||8!RI(I5;uzrJv_z0%gpwnM*ttsm*@p?>r~0G`B@ag7 zIZ2Rf`3LxiJmlBM9WY`~g&jX#3nlTP$K0@Eeu(WYKM=vXMLo)f8%&v=--?!8T|**< zo`3b3?CdYl=*|sXN7m@Mk+}cb9B~%9(PBCBy9K^{yn>UCICLVIIl$bMIlsr&0rN&3 zFdkDmAY;CXsLTCeSMNwcpriPPV6VpixiKc#SZ=xBJ%7vbs2Ea?*5D;WcWh^GGY#IK!uS9-T>vVNOX>pA2+u~B2Rdb)B#>Dq0e02h+R1glZA(h%M z8dp~8>XK;ndJnu#%8A4O`{>L-Dn#37Z}a+NR~KuZPSb#Pt{0tW{)!ZX&fg5)?DnK_ z{)4CaQDdi#IGHPYYn@ypr6DsVHeo+e3AJU#=Q+_-(gWUUi5maeoNOokB3iAl=d5~_ z?$VU;tFb>+Q``5CN%B5$V`m;A*BAFUYaOWe&v7~!V8*hz@VwJ-bVc*#el~2DTo#0X zg9irg3%c7a$T59g!nVDhj)q?XrbRqHI{(OXb{&gCPmX*shqCpGZW5^XGI z^AfZe^l)Z$>6%AXYgeM--I0-<6$Gq+vE?^Z1)bkzrKfjG$g4i(H8nYQ(xpWniG8Ax zh5NkY(lS@nFl4`O*?(4)A^j{_Sl~_v&GJ7yVXr?4nf*+gH<=N8ODrbFHLCkMhTd^q zY&li&l)ZPejQQ_p0kCfDSr>||;(fDNuG8_-s<%;~0^Q&Bm+u;jo4a-0kQxW|b$54P zzot|znutTBGVAbmjAnBo$9l}7#2EP61&>)~)S7H^oi8SV=VegaN@1x4uHqb{+5K*2 zPcxRY`huRRQQS@p#28|NzYw(8bo?Y7C!2#iq>7lUqD2j(L6S0Y6i?+u%es9h`dUSm zY8GdNS>Ies@_N-Sk*R2WH7d5b{qAo3xMW_Ek+zCxLy?fl?~TJH2?hO< zr0UQfzRODCb-ZvRMiI1dHLss9Es*Wt=(of!z$okGdjq_2CmN&bH3+g6zq4qTJYRz( zM(}$F%qyd2RCwpSWJ*3$hE~01S^h@n5<|CF>>Ke|wUX>+Xi<s{#q=w;;=4zpQZ07Mfa%_FSZb{1d8-3g|w+V!1BF3y}9F+UUqVCIp+7 z_+m_`E0KdHEhCNQ?BoIWLH5_QBOzi@Db3gI)*7V`iFp!YHW@dEvl26mMR^YCDyS)a z@>Ln=4Qj&M1D8ZqLypTz?mvlf$4`e-HZLphT6Lg<%gH@QrBXwHc3oe)(y$@ z=kX8Xh{ue~PQj$D?n@QKh>a_{pGr?Y}w_DnC9iz-swT8Q_q46t5_*yt+LrIkT_A{AnVIV5@C-}V1a)Wj( z&U142%{5T*Zy3agcBJB99*ig4CWz|5QzYE(K$^la1dHP9yUagzj$~qDeF+HT@LX?;IeW zSy#~a#*3BYbO>|z_5mtJd6b}gL*wM08~jLGv2lsE5oB4PB=x|H7twHi*UajV_yDpu z;cHOW{jUS7codCSZxiDWISE_<_Kd@4u2SRl`(k$c?@#kI;e*W5Dx(f?oXp9h`qlNg z&U?cx2=T0!uY7e`(R*hK0cpFV?CN3z1YL8d*v}2JcCElfmw2W2^;YNHuU4VD)eAVk z<8RK^L~?u{g07F&R{8K+LOIHo6Y{nvc8_;56g_olob=|)#q;BAe{P#vn=Tzs6y0r( zXEI8cPr}9b8ur(vBJEu6YA$x1$DIgAM2bl%k-d}A7SH2L{IX1d`kQ)y>3jwG2M+#n zsb;%T5Cy400~-Pj8d-nMkL*2$xgl_&I!1|N`ff@7v!9+ExqkSY0pF{mbg$HMWS2iG z(Ewwm7MrEM)CR#B-wi@ML^8GoMVuRW(PN+RI4i1Pa?}b_#o+nOz-P`4$(zL+)%zpC zoAt2UMLRoY> zmpP0b`zUyz>5FEt^)?1DX$k1S5pPx1;)KqgBbo1^B&tPC4 z{9r3fW}~x72E5^n*uBnyaijG)X{pZ019X6P*Ijq~wHihtYT;_Dme_-JKkuQNn458b z7x@H;v7-&|+)I?72y|l2x_Jl_y^UTxK13^?tRKRTvsjZon_0l-543N&sH+Q# z{ab$^?e8c;Tgx92Xr9*%XH!XAwZezrW-qeQ!d5fsh+r_bJGz=)#dXt|6Fz{NKWWFs zHjPhTU5N^tURHu(u5y8O4zI5v5O3y=Au6v;w7Hxi=nq24y4jGTpr~t1e9$8qe^tNn zGiHj5Lo|x(AOmVU?gVa3V5tLU_{~eK!#;Q!PM92030<1kQB8$Of-LNNb^P(d^h=%4 zKbKYR;3!Vs2Dxcz+_M1yUM$1a^Tii7wUYJ+Aob3wOn!mwh$NHC_q=NODDe6R8k!O1 zctLh$Z;d2XDx<9d|C--Hnr>KB0#@{K3wsV1hl=4!Qh(4lJjcqw@`eM?z6Qc*H4H3} z`{6pi>8}Fn=)yy)V5zvSTYj=Gx#XxgAru#H+f-MhZ0JS8_rqb@vsu%?B`|8e^G$?2 zzPKgS`}Pchv-fHZT1xNq+vy5P9D%b0lkHM+YY!n=Wd59ugs@@WG!gvfo#Hzq%t&$) ztWeIX^QCDMMK4pzKMebFelC(WzV zv~x3e>t;cOv{a*GrYH^KGq`wPhR1Y>T!i@~0!IftxE-E=qo*JjoB~s zlVPz+>uvTZ;R@czO15?-H}c@&esy-XGuO)7u&oas~bt zCi7=CcK9Y!)efK920bri*^t^o0!fxC@PI)EXN9qbd*kWwQw*LFdb>-e8*g?RJkZ7J zGY98L9eX=yNpZ@2h-TYi@a#n;->Uz52Ahua?qCHg^QbbJ&;Bqy<-9q6SFOo_BlL2Q z&`Xz>nT18>zmKe>jvJDL4?8C|NeCW8+CIua3qzJEJB@T4g;ef zPg@X+moL)g&s6vZGhlSuB=-RWfCFRJWZGEG(&o* zo}c3Sr6xT071?D|=gQ$19T7QPX;Q#8x?W=?&A2yFC)yfu&WGZ7@FOv#7b1y|cPQW( z^pHv;NA^buxc~Wl41dOeVCYg1L^ufaw^grNeJh*jtu2>SY37p!@ zH=?P8+}Vx0*MRf#ZhN?&+n~bG8lIe@=K>E%FjB4WfU27oNYy_hZXEM-Teal)1M)ZP zqjv^PDwYV`IfDo_!EqKt{aFN2t8n{A4fk-0o7^xY9#-I|Vj1&BxZbckK z{7$;yacHGErnL0Zvu{L%q#vrd$RZ8=Hd!(sk6U#ql@!v*bZQp$l4W-bDiXaXIjNU; zK9RN>J+Pw0b;TD;u9z10`$^PD3!q_`2-!dJh>zGwC4KGCUU=*NC7TQ@rD$GMS1n5B zD9vYNV^f^D%(zbC7+QNz6SUrvcVkr`fhERpHYFM8>@Zfj-}{eUg^fOd;>Pl@FG|e( z!v)^6-4Pg*drP?O!O-b8@15v5T@0@mUHx<(*75$cX-CBxpJybY|A*;%RGyk5ZmlQY z6G`gF;X=Y^i0Ag~D{!lTPj5tnh5$Ji@CHeQJQ33%&u8i7s<_8Pu>l8VCXP>iqVOU} z(QKv%2RnRNRrmif^%X#IHBHxnK=9!15<+lycY+3Yx8UwhaCi3v4;tKqJHg%ET^IQ8 z^1NTwU$sRQvfSC8meXgZyFZ)u@`KB!a`n*}w@`0>hkao(_ifs?@?7#|;R48D=z zB^&_ay+X>EhL5Ja9!gAOzRGy0v3PO%0=Jf~DV=*`p$1rq!_xUTS)%36Y>)Z{a zUrH^zlkIh@@QV$=0kP!Cp~|tEhkFV9$yvu`qSF~b(d8h;ehxvF!(N*s&l2kZinYEW zzsTbmbT>s7meBA&Uz(9WHcAC9vd>Usm90PZ!PM?NS{@gb!-|#0-rtw!tj@6FoV9@x zuUnr{$%Wmny933Br=!UapZxzZfJQE(R+n5u%aT_%uf#&*%T`ac3sEDQc!T8Zi9?;ERWmC(dzKtAc<2J1?XtMK=Ux1 z)mgoo(m7JQ7a+#Uw4_nkTMPbc2=fzNowQo2A#F$}2$m|H14z&^ebq!7%%pCTgUelx8glOyVh(n>Wy5i!VqVGo zC)vH$M?6nMn~%-xiYn&KVRswyorEF7oQI@!%MN^p8w$9b%$P$Gohq~)$Hl0_QY1W3 zXdh3}SuP1-Fpv{+zt;Z~UO0%^13WtL+RpT|(YJ_hBP%59r6~6LbR1eGu$NVM!}vVh z#0*MPn|zY~`Zd{h4=9&dyPv00`gZ{m@Wnk7xIgwxYq4uGzcvplY@^t9D5o$8tc$#T zE~=D##jj)vXX`)9elH;^rWXP5LE9X}9Te}0J#?wr&a8iapWtnobFulHBR`|1-d`Bk zf1<@2{}%wkoi1N-Ag3Pmbyls zvt?SIX`oZHn#VcZ+tObWMfH7bzk#x^sXrthmny&02OfiRh9~+d2c-9FO{P2?K9pdL z>4nDI7X(WIl}%Ah*B(AV&0N#nMVR=uV}S>o$z6N*WSt0&#$~qz^9jPr)xxZWN+s=h zRI7xh9tX}Q%MW8wEqSj;p@vJ-rQ7`#2~&`d_Np@pT@B86Dpjjk3OgrL2uWzgR@u#`hVkTz$QXI)fk->!+=4vS!0?G5o2=AbLC2F4orsrdz9 zUSVHYw8c)$*5T|t3k4#&*Aq4}HkwT(lBb%u4%LR8+wcDB1sv5Ta>r0K@|F1uugssd zASNBzXF~eE<0m9S7&oVm_5E-L-|x(zNiCEBwB^#%N(-PA<)e>hdBGW6?8fR^=_SZjIJ5B9pu^|1A z`^R;$WmT%J=X^v2=~Zcuigc2nkB3f*ZzNam+CM0&?Q^qQxGq(Ai+?nub2LpSe;C?G z`j{ajeEQv4qHylMR5ePh1JSPckZh%AEfg!J8N+Dg8+nGx{ir#Iwr3rIrhu9Y$g#80 zR)L8Z<$L9HAAerMewsC8&n8UBbqY`+`8AN6%^}g0-0|!|D;_O}gh}g8l7hXS#R02w znd-}Zd~X_baL`xvqdvf7!3SOd0SYI`%Kslx*oh0_NKo^K-x%JOXG()+c83At#WvX& zwAMA@+HD)?3#*5;_R$vV3~H{06%&UIQZ-1;%mprK(Uey^Xn1dfy7Q1w&gj1T{NuUL zxxhwcS8dJRL@taMsTYOzo+?^}0UFB&9#U44cld{p_2%~e4cD%Y5Qbplpd*k)*U{gQ z!nV_%GUl)l;9A(o=(jcMM_mLpRCiYG%H9;a(S`!VYjl6zf}?ehsIe}#Qf>&FYH~Lg z)E7ul%%8a^w;6dw>H*iFBRtpcjP4t}qq@8}!DzUBqeM7Q7o2B|QbiHSh>*QTd6)GT zs72Qnp+5erM(s7F*G$ijhefZp94fPYStv9WPC1!azjHJ+<>qn@+0kE3#p(;3BI(da z=n)}U3DMRW8rYbrcc>e^zZ{>E-RpHd4;X9kcnSWR7L{poS{MGUFD$U@SB_M@bC!;7 zB1>UEVJ3@+l0BWZ!emjb;;a?^B`ru|L$*Pvi5E~Uj_dg-O=`AP3t$P7J6NmLW*wWR zEh4T~tsP9w3RK37m@qErhp46Rek7w2a8T|S%OGzHJm=DXzp0rmoXVW<{c!u2*59vN z0uD05{3mL;ZKs(Tr@^Oap6-U9NfM^T4X(uj#o>L6YiMO32HjqfwZ+WUju}`d&CuB3 z?|i>kKJpi23Hv9@haBOIjVL)fvIaXS-%`8C`^g8Hv0_HCEozB*T5ilw%2isTX;0Z1 z8#}|A7;R9O9@WjZn^UZ#LgLwHqIpZn1d04qh*~5%VCmrDa@&W^Z_fWI)W$tO!+&JX z?s?VbuoHf~+$NFEYC%FuN_v|(cbYq>!N@+NJ7Y4(=MG zUPh&@!VYblzaac1?5EAMT1~s(9(^>gq!4HW0+3;nT=w6eUBWc$pxfcjE_!(`d3kEr z9eyblgvSMtguAo+Q%g5TH@bVK`!&0)2A;xiJxY_U)VeoH z3F7Ca0kzl_hSyS!JiCX>%WXchj;TTKomHZ*7pH#^`_YV-@0LN{Q`|qDtJ<}-MEalk z+xlf}0B0@8(}Za}mg)g=HIHfak&X*d^~sL+hrK{o2g7RADSo5iE@ym0a>}zu^h$hI z>-j$_yRWw)GOu3Iye|XqFQ>O2KQq=Ij4oa=w37*pJgboX?fzxAp)Ud=hy*TgRyT3M9%j&p^+pZi}?=Z81B*}L>UV@ znhkVxHEy*%R#RFIdsvEX@uXN1BsHA*mRfKnimMNBbRJa?Tgnwflm1A#vY^c zFq=_+eDq%?E3;c%!4T~bL%uF~42w9inLS<^aOD{i?#YGJMnD5{II*An-_G(GNy>OC z+aAw@Q=zH9kUIZv2}kSg@p9AEr0sf}`1IM3z@AjlWr<~z8)P6o8B38isd-E~_*?0ui%i5Y?P)rJVWF_;B<}lMh_wH=dC0-e;wPI)n*# z6+W+9dqoJMK1Q|6w(z!BK{@mjwkcv4^R^6h*g!iTg;*fvcvg!a-v}cX#fO>uzMdrA zu~)$(renQ65kfF_ah>Kk@Xs@fm*o&OgZJCVpgza7!nILv4sOkYqJ1irJup*E9WCggliA5lthnY`8$#!8{dWZL!ak64gxZN0a~Rvp};*ky^4+g(olnpJ)2MY z{jJ+0-)5YQ?C&p;mf8Fab>mM&<1ddexK5rOGF;IS22({#PiwTAwx}`APuy3JDjZDHbuX(aCN51}ze%gp?>{$$=nRLin7x?HB9 zfz8)RQIXj}a&Fv($U-l}R#jVF9`Y8JLXVeW5c~X{l723f+SIWc_~Lse8spYr{&363 zpM|-CW1A7cWdt+_9G+Ug*uQ9tt^R;&6hx@7exyBIx|1H89d2&@v|a)@JhPLdyNc5a zwSwPw$14oj_GgQ&-^ssdPJ1j-xq2Es@dAA<>4)C|MMhky+ocC1J6~JS&c$D$DOf!*AdFd7UN=~rl_2(6EV=i9&S6r_CbJBGRj1vnx9}*d z<#wR7?#N_*B@nS%tsr)HOpnSanb%@!%wgiLaCLI;w#8&|dX7#^PA?{h^V6JD+v4{O zP${nq=tta;$dy#tfcN1Vi+FRCt8PZ>m&)230%SsdY7t_B@33QZ860ul648SuR?KtG(*>3;mb7Y*)}B?=oNX+Fa7M&SkRA`m~iKPC7LLn)$-q6MDW zGJrU*^tYP}^8RVPd`UyYKD&0L{jt%Lk7OYpRQKJ3Dc><8;+Myt7*@=W_0Ra$p4*pR z^Ar8BKPe8!hu9vtEY(hR`1k~D+g(S;G>g2>OIjHpoJ*9eTN9}@CSfU>>OHa3*mu~z z=rk*Pr096wB(!tf#>bJL$}`lq9;~?Lq9Qt2+3t3#d6Rv+lj3UiZ2Q&Z+CizK8|DeW z(oN-ynAXT>$C2`ZBV$e6XR9c6ZT^#{?$W@4Z?i6srCcO{D8}JZ|7zV0T?ujNCqt3# zMcWb@e^igKMoUm6D`-yryJRzbNr8=@nj)!p{ge#c;2C{ZE2T$I^E@WA$d7 z$bDnsv-Mck^UNLdPWM5L#~|oBbHXgvaC;ZRb#Onf`a{8az8E#qXmtG@hxC1kv)Bx9A+a)`SU=oveI#DMzwt=-ivoY&Kd zwLeOyB8xLy6f2sBJ}8!nAF!_=@n#$lb>LFWUVQ$CcH)LrqosOILd7kEmyZ0 ztrCkewij4Na|mF&uCLQ-73;q4$jCa?wrzcu8^T7B@qO8GeXt)xY&l`hO6`Y^iT05T zM#Ljwkart1-A-pK9gJV|iKVn|SxnUWD9(FP9_eu{B*I&tcu-adsqy#}GkIKUX)M^0 z@k{8r`C!IK*Gidb!I@5( zYNz`({9z#+IdYtW0AGhvgnRbPX5`EK`SgR)osC?cK8L*26Vc~S(c0eYpGldRH36!a z?we)3)bygY!*`bJSYwbJXM++oM!{e>MXN4N&5FC@Y`owLirvjBTT$&*5*P(Oz=*Es zVZVHNYWQ8h(5QKKvF7_^5++tPu)jySPITj4aD?xvL$gKMp8Z8oG$*;G*<0(2UPU?R zrQeWe^FqWiINZcw%CX)ZJ^1YnMpf*uU zS9V^dO}PZkrFRQk&qIW=?J=4{NYDoM!s%x70S&8AKG`#cPN)4Vt4gn{o)^&2tGN$_ z=b`ddyH)%4RDh)86_m7cWieN306UqW=gaXd4!S<5{ZRH%t!~gs&S;x2TH+eWwSX2% zY;z8U;#6-|^?q2gQw4Gc9-2R(o-Jh5O$TkR~sz%QVA$a+Gubc<}Z0Y_MClZp&K9 zSa{}tW@@=wZM{ruYq0IOT0I*)xsyq-tXQshzgw(3GTz7^-|oo^Y$)7|fP$UBl{QrG zU~DD~TFoZaf!)`J37$r~=LzQ_s;8~%tpNvYC$cJ+-TRdZe{9PP#;NuAs@g>~bElQU zsbJgkCb|0QTWJ#PJ^JUT0{J1l#)A9qUmX$YCCvsCvTS6~BOjIzEArf%hZ))xR^4_4 zn4bnahA;XF={{%jznsVxz zC{gP7V>qDGs)v%d<~u|t=R3PPAQ(RHvb;fBL+gtbG=>yNTAAo&-=>r`S-52r6O(5X zZ4aoXh+TzZH!!#=#TH9mrn?-&iQ|w6^J4zC-I8YT!^kVkX;Id=zgF}%hs5EUwmh=y zYOLUeR7oD?fhTC|`lXd~rCAogf?>IJ19IWdONtyo?{&EKReWQKi0z~?nuC>v#s9VS zPf7l-o=#36#VsDAQdkEZpPbeQNBNV0tZ!$@-$e=Fb99D92+ zSXX%`-0%EryODpji|LtY%aDnj48)w}TAITUrqA1u`Q!0znpCngZrs$>Fh(;b@lP?P zi|iY7|6a-q{6-q#H%}Ai@Kxg3v_oA{x7{fv-rhG5=ofb?+rH~)$u+rj@q8ULlm7eY z6s`QF%_gVqHZ+j}rNj6j`BCWY6NUvmTAjU4|1<___H^F@C4 zSAg=#{Cy)*Fh^`1 z)<^1W4e(niHbmba6o@3a)_B-x{)i6 zF2}QKsBw|lK)347^r=Z#y392(T;q$2Hx9e|Ow7B>wL7hPPdwr&}YPkPHN2ij6O5=d2K+y<7^x$Tw{8z0^3wF&}nd@PqF10*1P5yQt}4 zdo4wGMeuDFXBd?ovdR5>;L5Wdz4`lwwr*I`rZ%8?Is1zkp!0qksTGy&_ElT9T2IUV z=5aYD$??5VDMLC<-}_az13%7bee{KJEr-<#))ZsBo1bV8>c{4;&6ymZd#cRS(Hi7I z3hgu5x`(XyCze0{esr(s3zyObPfP-J8y32oq>xKKm>$KG<&cGC0s83CZJ!6L}^EayVI za)Wov^2?5|dhc75T#x6OwoMpXS=BxeIIO$a`!@QJfn;M65PZEWrl+SjfrK65RLBrX zx6HQu#E=^e7OmKOkrzt-W~{fMjk~Oi`eXK998{~!J0CUfy-)%ORJowdF7sh~&JW4g zKTyN#n#v4pH`C!UBWT31QtcorP!TC&sUZnZ#SYkwP?X7mi(?N2 zb1e$F22h?{Q9jqN3+QMCF<9>fj;|f4Dd;)&6nm`gmD#uZ;W(6+l_j8d+(%(-?+vdJ zHGVG52vX3SV9q*Dmj0cE`J?x-?PdX1f7}b#SYD_AtuSLYcGuWipcOu zayuVlu*;T`D}z?w`Foo_)RIQ%wr)FWM5j&C1)lP@Np!4i$N0Jzn<%e8rwq4at*r0t zT2?O8>%yE$Ao0@L$$69XlM@_?z4`tO{%Bl@pyUHgV9!_*f zh1<4Ci)T8gAJ^VxcS%D^VE))sX+>4HE<%?|uPmH`;(oe+z}7=up=H=_H*7(?rzC?g zD)P6C3?R4pX-9ynq$#GzD_6EO*aw`H0M2LSynF9XR?Xlqd->9pguHg*pZY}y|J|&l zT8s{2CE~-QqhI?V`&8w>w=OrGiozC#n6`UlyIQ}T zT3GHX0p#`w7-G-FeTR1aeb|nHy~rlX)2v&4eWTWOcRv1?@iaw;rIpK0Vq06E<`uoN zSRL*Sk9&f4ZpM2gRmq3vs}cuX=hYyHIl`{1u7a`6`>VEw!K}L%jnn2chNTBKpS_j4 zoqHwm0X8*I`wU_X`A1@sgwsX|Wo=cx$st9$+4Z`|zD4>%a)?g~D!LBvk9$h%l&wQG z?9FAh>vUxPuuvo@cU>vZumopWFaka9WCD+3+1)Wc86+6Q2ncnG&bU7|B^YOpg$bU7 zC|2!@2e>$29*pCHP{WNd;C}uP49Q&nlIpyS>d67B73mjf%QzO!!fWYMl4pq4+4`=> zcmjC+1SNw&Uv^%`4@lW_O#7H~(J5v*zEXv9Up{89dMI~{TO4K!2W=*AOee_0YV*AQ zhGw=JV4^E{pCNSQLx>1iNE8$!kv@QnfR`PQ|Ka6ihIcR%?kP3D-c^FLT}Sn9q4Bi) zN#>+TZLb?L45v`y!3Y6gDuQ5W9#cizXP-BDnIkH&!A29!t$kUJJS8tj8L%+e8V15%dlC50P?ia z;!46!qfzu?x~(t+R!-ExcxBk}g`ecz-yb!T5zbgQy=uBZi(AUGHZ_5V?Ws@s_(7l@ zpwlpb=}ku%#8OV{5Sx|H0wO9BgCh8oM4pH-=GBd#gOMCwbrOD=PixR63}fs0dp$?+EVYHFS?57VeAZN6>M!+JET{CTT*DvLQ{LrJIIKQMVG9eFu6n( z$yoo`GMbB$7R4frv1Z&TJeiO;+)NA?ZT7$V1M@=Jn{XE2&U4c>-`WC*^{=1a*1^>hQndP{|Q>&+ErN%lh_+1 z)p;YjHx{QOU#_KwIUib4?-F?2GMb4n%|fj_4RH<(`{Mi%V5i5vcz^tR@jcmkMN2{5 ztL{19>3f7*=5e7YK48M_Mn-BW;r&8b+_%aNR+daIGi_Y)CvrCLDOG6Q&bm9H;@xhs z;bu9$NQC!Gbq&cBpu^T7etA@p5qfL+5X%gy;j^ZP7i9GG<WxVo2tmtKK&ZqG1~U zk`b>_YX~n*Ux?e0E7PuAqbz0>NSBxk6`Ng7$Rr4S%w@i>#7mmlpS8^qK}0mjEv}OG zi>*rRR{QeP!RU)O!1v>&KSeEhH|U6-gv?l0v7VnXVV(h=Zkmd6ND@-|tb-R@fJz!N zilaP(iok5UzmqMBovG-OGcK^I@N;9+3Zrs`Sycc^v=($|Z*%;C4gn2{9)y`_Qc-BR zZBp!K$bQHeCl7{ zz#w(TL%iP-xX9_KV4Ht{1-=aH~+4_C4d^k`2795G4 zXfp5H`Q~^wtG5_aH(a20J8tNqZXlMjEFUOIV2$o9IG`$L=8#6zL+VlY+bk;u*4>wE zP*(YypS)Xcn&;|GhRY;LC?QSAxYWz}-o1pqXNj=?cTp8Ug*~@^^_UFbd#<5fEpF_C zS`QWKfK(vWi2xNz+l_tBWsA?;q`z$~;E^|3P7cO?ZvH?A3WnuRvvC09N)A`<3F@<{ zw_XmE`ei3gGKYb&QqP@6f>_;uSZ;K$k{vhVklJtui^yBv0kYgBz`e`R3vvp}%h;D~ zI`T=on0wOx#|bPPb3#3=5zd@A>gC}>8_IA&C5J4!Cc%3+ikNpU#QW14+pw5`l9uUv z(SD8CHzzunC0X$zc+Vd}ipAikzU)EnzpawNl_2;PwlioV)$h!`5#`a|y8BNe#?V`5 zc@hyo+GDZ!qozQwFt->%Ro%T7g(F=QM?E?mri%^Z`t=ZI^l0xWrCb8rY7TXIidx#; z6Fx}nSC`0E-b%%2B9~xo=2J+neZS|Q?B+ zn<4RNP&pucb&JAKcB;x^dDnhb-3K+YtV2_v1a)f1RY)=yOHmO8%t>spVBbFn8x8pV zp+Ut)3C}}O%lx5nL5hA!0rwNt-w%Pb;~SoNy!Dz4w(xf+#{~=^o~Z#7rjEmxI=_N1 zYINR0m71m5X;F1_C2nyqG14#c&px;kI)FO*ei{iZ8zo|AS_0?G@8PV-W3#g8c3wTy zb$@$_n2x!tzC0fz)Ywl^E?CC+I1C6q4fmh5QvYKoTTTWlKHPt|4}SiOmPz@>=O8lq z#CV<18Lk~Wy^Juhz_n6L;{QEk;;zMk`mj~Uy?z&j9-f5OvsszmV|b$mf(KzTOkDw3 zSqjSIZ1IX(CSV@Z7nNA~@oI;$_xC3b zNPqP@xNKVh=bIz2%n*`t!k06?(u#%h??VGZ+2pBKq zvKTjJQvZ6CRGJ!=A8TFr708dY{3KubC8={Zh_f`x4KJ3J#8gF_g8xy(&O((Ajhx=| zz&E1Ae|X|&)XKl8Q2X#vNX+Rr#K}|Guo}z1o5pI5%sHN6l2DNF)QN{)F!uk~h zx+vc7g_(LP&=m&gHBod!5GA!@8e^2Hf<;ckqsXqtP&DwFndIXIgVD}+?!%d)gCek> zI^4-4^hOjAt=Pc=qDTU|^Z(p5*#&c~D8&AyMW0Wsac*V(%dCzz+bUD*x!MUf<*n}{ zt7X;rxJOuzw#NO{!1^J$i!xBNLtpHzUS+VhWIerI$x^gq^_>?=rZJ<_#sy}xU)4_a zf>C=7-Oz6SU7Q;_*_uidy|}NHb@fksQ09ir zycZZ+Nqzt%Hkdoa3vNxE8wKUHaH%#Tbm%wynk?`)bFRClrrpzhD#i}*>0S~FK z+r#)jPzEcTYaur{o`;_UYhY{L{fOimEZ19`Cb=#Y2Q*D7P=Cf08Nmw}4<%|5HTV|j zWZO+0TW?r5fPXI>%mEKDUjFK^_l}OvLAufEw`(m?nfC^?bBst}S;Y#1fUw;^9{HaD z6k(Oe>D%535XpDO49vK62aB&+OMUsSRk%{q67?6NK-PBEYw{Yvq=Q0IOg2=XYL_Ad8`-8RU7v!Bv=Tcs;7Ov((&` zP0^E|Q~SoZ#arSyxh%2hPBS5n`YEJSrvfc+5`0+BIqLP6TZzofm_|Q*_+z=2cZT*N zWj}a>cWA*(bzb2`qRMc!FAXO> zSKgg?ASuw0YbqwZ6Zb^~QtA_6NF{D*TyKUXp8J-36DJFk>n}BzT}rGL3NR)RSv8i! zf;#JjtrE>7V+5h`a%0M~60Vz#wIPnM^_iF>N%>%>!vy&7{;K{Uw{w0K^mT_#mhi1J ze6T1-GZ>(NbQEhA2JwA0!`dpY4UZ9iswHOzzyKGpe7DsQv z2Ip4>d+$Yf=5N*d{zE6j9yXRhcKFcmEG{1oP%&X}PBngmmBwqOs)^<$B&FGL)YVKC zvS#mK6l0)4eg{j54w&5#C1B zpnnn-s7490>(Ft!Tx9*Ma6naM9C-2%>cs-?a*GmV3~mg-~+*-7H9L7 zp!iz$7YOLq{<0C#zZCm{=qtQD$-Vj zc9gvRFlNDt<`=B&7eF72UinNB04vd!%Qda%Qbsh2q;IJ`EKht085laWgP6d<5gI?# zb&^n#mg$4Lq#i3_0#y0OXFI)ns=l8H{uNv+m3@H$S^u){&iyKBbrySFtjN!6B=u;C zVaKRdIWS<%X`xE(grKQ(n6<;OF*C$;xm1kB8hd!; zGg)25M=C*P$Yrm92=o&CLf~Ys>{c|Kk=O=O-M;)juj zHyU1$>9u9-aFrq-e*TtN7A627M|If>Z)h)fL7YE;9fF|bhWX7i4RD^8>bNzv~-a-w%XkR=QLo@-HKkz4nQ-#_sZLOR^j z-~erkV=W-FTDI5xZXmqRT^~(f2DD6nHG3t$O`Ayj3ba=#Ty?Jp8tLY?!QqG}zY4)N za-rA})8epQ+$gJ<-(((LeVdg`U7675i!XK1_z3nHZc%<~XD#pf5B|w7A@5+zY46uP zntk=&_|Z)66+z&?&;?PAb21x6%LBcH%+#pjcYjSTH`ZUR*lT#K`zRMD`9C+d4i8x; zzr4IuA5{jKK#YfnrvulNZ(0roe#B1!Fh%Pdvmguv4Q#YxdWJTX+GuC?=~9oDjXq^n zrnxa5nD=L6D8Kid2~B*`hH5mh-j)^wxW-BHbyaf=UIN>4%c`%fif1y#dl9K(ks7^$ z%CzF1Fx?-zY6|{5dwT7)=Ls_v#;IkJT%t~HVY~+v6B~9ui+eeopKBb;Q?*O-ij849 z%^5sN#@`7HKXgthmH-F5jtj|V!f_*Nat5nBUte*|{#QF?XsCPcLY;0L%~PCu*hB*y zcGo{aw$nBt+^{ho-JcQ_jM%m*-#L{XA)L!+k~<&*phLma+2z8(oczz$ImAJGFtoOU zKDT|Z6q)qjB2z?QH5qVF+HKH%l`d^`A9P@w9!#rkfdBhl3fMKxRaJIi_GIS_lmqu_ zNE-UexBk5=goxLqp=MP+zVEgpZu}o=qb=E0)y8E^7HA{DO4ll!>!CVm$7{|5H2s{N z{@tJ~YAQ(zar|{2d&o|KO11jFQ~$z^kb2!x7K^+>7~1GKm+j7sY%S#LfFtnSRr2-W zOWEF1E`eoLP*d)Ca__1ogymc7y;Vx!zECAh?-tP>8ezC=Mj>!srhn%cz^=Ub0w60Eb5 zj3uFa6%LGU63DYcYj|>uvdXXzJ>EZsnaa^@sCErZs~;In!8XSN$>u0?jnFq<4gXV> zwv2T&y?^)3$$SQ&`)!Z4;U~a}PWpG&;F`iL?Rg6NB5DO=xY9K{Yfvwlq!%KWUOZ?P zs>r8cmKcxMfTbj|{wFigCt`*aMxXi!J3`HyV}*z9PFz;DF@sZg`(0>z*XT*`re}4- zEPsG{E3W-nsRANGV?GYGV%Z!cSa&%fx^LHJ=DBbh_F`ClCC_GS{wpAo1J5sV*Hfnf z)QOd1qBf%vCvc7LqWiAEbIb{Q#kvKsL)7#sO5&c2@FGmyegrv%?SpxZ~J{wDR zd&8pKwaBcOt%Swj-WrTnoMv(xVD1$~A5$Oo0W%KQkmNj5QmA5pB(ztE z14Ck?CX*_qqVsO!2#iX&WVS_q_DjFHU2}>mpc^i;OzZr6nq4fu4)+c6ZQ(b%VZsLp z%nihz$$-|!l0sRSrf5eZ9Y*`xJ$dW=W^3n5KRKH|;7!ax$mt(HQ(%2)xh;PG-ykYA z^z(QsvFH<;*6AOgBQyJE(|1t8#dlgdLxwylC3OPY(mBL zZYZ$?145dD11YXw?RvnT8FIj#}@QD@ZUVxRzcHFg~Tmk!u6-C;tCJ@IZk(&W8{9VfQQ zQmLw8z(5~`&*eeh;wAQ0eeF>bq|Nrshb7hwCIbcv2||*>ZErhe`>Bxdoip0nq@eb` zBBh0X2E%zjC`EC=H4Q1$k^pOwp_pKr^A+&nbq#U?$-E4N)h-4KS-T%}QQa|uV)a@s zGiDqY3qI1mO(I*C=}a+M3)}gpp-%LyHp2bVGwLksvZ9N%pv2J-xL+oG!8m`aNf}N0 zzFo^nd|kMwOrf0pj3SO$gmPC3EQe6x^&)0RH78Cx4N{jtIww+$AnXG0qDU~oSSo;d z82Fj7Ixfk$h1!!w#f`-aEcW>i{W z>}+QekTNF`S>uS4I)x);;ISt)*ThSeCc?R*i-_Ka@Pzpxz+)5`+S}57+!Ou~+M5x# z%SSMyXD^0iii3>Zqv=D>w~Zz;?pxw_|2O~!>H;O8nwxTSWb;Py3ZQCIRkY#0Jw)Y^WnAOE zX^&le=iXlW^F183y)|@&w^^~I_fG_zH((-%GQbRaWE}4l zC%tpUH}KgJ%^nE^i%OEj9&ichX8(y!p9j*?Ea|p?q@2ci>z7A2-dr}1=+jU80$x~X z-?fT(;K~%RdR#gEitnZdB2$A=ILBdpA!`Tja|q*M7~A29e_dn4djtQ~3*j8^LD@Rv z{BsKm@2!IGQA0$3KsmTbe%m=kiQH1J$%9m9FEs9)I;10x)#T=Q4YdC}6f3Zt0#rj@ z@OM2zovgBY_2a40(lu7y?UhG%jzR<}Jn4UK!!kULPW!&czN?>C?cRzu9UB3Emd-7x z4<8xn11h(?nU35B1CWn9+^e}9#a#R*91um<)<-tG&+!t1$b)wpDM>K8k;Ho8$RfxK zO)0<+qe%0M&hZY0f->q(meLt-@wSeSFunzEmLQ}(K==^A&{Beji20jCRUDjkU<91R z*{NNe^fkNO&dio{Kd3*Xm_+Dg3rvDYHCJ!}%r6L;CJeAyEb@qJbc?!6HI>As>DhL} z=dA3+-_*X7n`~2k&aivVP5!=TL7Eq2emA?e)!U-eytax4+71E z@08)+A=-=ds>A%}quv2%4nzX>%YGtIvx!k>H2hDh&NHSOw?WPy3(b-wvT#l#w<~@x)f}L%j&F=LZqZr*0Xjf*Sc$WFh5|&~aF1JrS(T@| zWQYSiI~&qX>hor_N)=k&DQ3bo@H`biEm$ul7Zr&dgH%o8NFCe7r||C?D~cSm|LIaL zcqX$C+S^u;LN;+VO;ueeLYO{|S=ch=ThzN#96IFMK;E3fU@hx2gBSk}jtXLg`_$+@ zvh12odrGfJA_Zy-Gw%JgUUhhH@svVnr}rP5ydRr|+;gNSobiV@n8}>TnM_C&N-nVrwK$rMjht&-Lp>d!UXR{p8g4~Gx zHLTb&j@b47tvX-hwpVMy-0QrAs3jXUaIho;+ZKxp#l3q^nfEK}!tUfCK=>-^*5XF{ zzE5J|{rL)$daoftO#hROCUNUW+*ljF&DB085-bsOAEVTP!q_nPw~?MehWQBR2&2q& zj)H8Y%9rnH1P&}foWKXR&8CIC9AK-!DchM$4xKMNXtCiW!<8x>q~pnMxPVyue&{u5 zw#&SVc$OSM&|oPQZjYt^5$Kt#p7?xdJ&c$0mC{k{k1Y|b3B`IXPGyXU4!0$t9U`{V zRPh5fRyVO2w6=OHi*gV8OIkNu=-!*tmh9Z$runDhxu1|TNUR%a6aQw3lATe2GjF-5 zsFDK&mQ6OnwA6^%0dno?yvu44__OGp5R8sKXmM0QnmAl;JuQR z29#&&vcBdUAej;p1yz@NfnhTl2@M~rI#*|WgG0n>7bJn?n6=)j9Z{xZ*&9QFg^DDD zjW@+3gFHv}QZsUnT&kwt3K1>41!#h8cY;gYazmzQn2pd^{O(!oF7&AWPeOS({Y~AX z?JNB0XK%83hEpD==EF;15Z5>84VDcyDfEqaOWM%R*#YfYZTBtb1ArV48Ykb~I?Xqb zL8>#z+1goFibvJN@*kJ@X=R?yJ!5SJf#GfQZ&a_hQypBH+``svg2pQ@_~)DOn;Dh6 zI_O?$GEaRxdqJw;^Rn8AyYTp!ASVO^e~n&zF3^s(H<=EN zwOBNJ&)7SRl0b?(kR;;Z59xz&lGm z5p7bhcN80~wD^5EG2)uYG*En=L?}Ln_M-UBAc?uk;)_$w79&+P=qCa50Z`owB%#nF-*g$c zNQ9bdnXZ8&AP?7rtGMiVpT(9pc|q`IxPC1_#^f^htZxmhomWuMOh$j*lk&$_sVowH zrGs=5(h`TN%15nZiRCK$9IeJGB0%10((C>GBaqi&=;!*fwB79$#rt1qvHWWd;y!XB z{?IQ!{ah87O#rjODZlkMtXHz-qy3dV;t1`F_I6f;X#GALm;(e}|KJdh&suQWqwl)9 zTYC}stBAsVpq2jfxsBj+BaUV0S-=N$vOp<^cIMu_^ucR!TobUF1mRP6T>i7<5t+dG zogboNCU6|gUHuXKPbXHP9Im`Lgg&ksNRI~3tv%0JJ6DD#=I2ZUM1bAM3_pvuU`{jq zMP-?~HnOMAwg|e8RpT|7{Q};MlHGa8HMeIJ+)bDMxBeIyxI)>i`XAtiU;rPd1hw>V z|F>~+*=Ot{+^cXbl)C^0Deo%X%Z<5>Ei@gBkJM5l9Tx^?|MnSUo1k>j4&pjIb6DEb zEt6bA^d1#!-3&` zit$CS+e|&{_V8#J*qrsEhn(cjYSW!TGL^Urga4Vu6_Dtq!5COCSN5{kt2@uS?MdrL zgSA8uauA$2FZ@Uo07po;@+|Aq(<`ZJ_McrfJ{{8k+VEE_HPi@ZSB%0;>>VwZ%dvBekvix*SUiPSKceo;MqgvOt%KiJ* z%TYGb68Fu)QN&U}2Cw;I*W4~7vqe2QP{1+QzTv$8k6-M3f~xUu@6eyM|Gbpk-xAGO zVBMHF`QbIMBlCxeUrVly-Zpl6(ighSpf?ugJybUO;0Xyc3cr7_;5&c@;tcI{kE;=H zc9vuz?)Mwt{Dl(DfOcOXVWz`#8tGb9UD7Gvn;78ug+O9+fb%W-Py)D!AL1b=JjrwB z76^65$-S-rrG)XoJZ1aKd1f;}|5!;BScx(%cwe$a-{epNum5>fz4W?5Yyld<5UN~kAc|5te4Y@;QUpUTH-g8~iRH$kCGSyfU&~;x!k=cAw{&hYN^k~XUGii@ z3^E_BRaP|lN2D9@M{WVvW{d$?5Uz8}f;zA!H(28bFIW8!TEt8BGzj7tuYB2-N^$Jc zk$U+)gjU@?)&1v#)PKPAQmkMLIzR|#0V87O(1mh(VG0z#{V~)i6|U9{8~{l2;VtU{ zn}wI*Z$qd1wb-jyanCVsm1f}bNZ@4#OWXE(sa+nO@v|vUey^L?tcv;VlbUk?_+bf& z4*`fxb}asf4p%rLk!uF^qFL%6w7)lxfBLb((nl@QU1Qdm_UQNm~uQ%Zi-Je&g>_J~%Ga7ker9;Z(qh<>g_`rx;5WodH-S_Jnet;0DYZL zpCacxY&kUgUZIVgvD;=P-TC%dIc5A3_2B#RML@b+>FzFRMClZzL_(C18XBZQ zxP`tP-E+xRInegQ_R+!er)xAZimJ-jzI z;yd?M-qc0Me8XX5UO;wsHd6n@$MB2mUpZhh1JfQd1fT6T_MYNMVzmzpDQU48J^9$U zg``kbJ-(hPcG9x z_V+*tkRHS4kW#qm$p>Q3?21B@Nqo?rAX#bjd6OyW3ypf_KHh#7Dc|p?ueGar+YG*$ z!5w>p@}gkkEY{>u+0>eQSKgD$7Cs3aYeFdAHp}uQ6G5AnbjS0=nx$LW%FT zGh#&Rk!CeT#@*gt8or4#`FsDm8l;ptV+gzVjBWUHLCxeirrH~BAf*R)PLBZ7?DI>u zG99pW#ng-Z&cCDH2*D-}+b$pVYdu$g>HCq~zv#DXBv0egm*ph3M}0Lb_36cMOfvN0rAf(iMyyHNkZ?CubkQskW7u_t^&37A0Gw1w$cGO_MFz20{dXfnaz^Y0iL&b zqk(a61%X6RP@ovYIp0Z_p!}75jpT6%F`EhAAK}M_JMN&2b|%1ae+UP%p(92nFrPPe zz42!cdcP^$^vPqABm>?efeoDfa5J0PIR>p`LrQ~1l)Tl+SS0eC8fnBo zTHsVme;7fFfZ??-9+!!5X5Z1}UVEt-73GD>Hec&a2}KEv=8riNH=ZMRfi^Kmacp~z zTKaHPtsSc0$GJArfN3P1Q|1c;V$xl(s|)sw)olJUYO<)&+^W8d%8ZNB{5yD|A=LK9 z1*31tPapfPc1zoJU^)RsvK{V<*KV{6k!OZ&;X??&3WNOoeeF|V$ z5Ev+z->5rzJ&nY1sV7kN{Y--2m+!Jq@N2YA|&tj7H@SHQ@`-^j0+;hBM< zGkvxq*4UVj-q!d+_7!P`^!7(N*b?0S5`?VQi=MmvT>0MIF)&M*RPY@TvWn>nfiAU{H39dAlZrs3{lYaMS;|Yxg zLSxnZjGh93Q|(WU)zV(8Jn_X|R&1*_cW#k)QMjr5uomT3rGy>9=n9d}8B==Ucdq`2>B$y6A`hf;w%H3)Lb}KAt4zv6cKA{ zC>b$L%o;`E&u+reIs{$-;Z)3A^p+&u$yOcZ+>KT%b{CuD!m)&*$sxX>9m@87G-Ak5 zaUXpP@FDGTlx7Hso?cN{+`{D^$^#IUS9iby5dQ$DDBEHWDQK0RhcJom*L$RB+6OEc{?+@h^F@1>^do ziEp05V-b=nFuqeM0_4xsIkEl|GRay~r`}#_H;e+w(j&oVPxH#6R)Q z4Y1^qu?W6{uG=imEI`pt#q?0T0V}585rZUQPgomDALYw}N+j$9>EKDo7!CB-2O%i(Bu9OX( zVFP)_jU8Ssl(vbDFRj(JYT2J02$JFJlab1R(9{i7}`pXzXdKF!6G|5_=${iza|nF(Nze*iCR$nYw7wPW9trswas;z4fVX zdt&{+Z?C_+E0!%)s=aeHL4lXJ`dbSQhBm`)TMSzpSh?N$zl@pz*Hkjo9Wk3Q-=QNKH~NSwP-VPhPcz>q*bnddL*$a@viqxVHV%#^pm zi;z27D@--r%j#B8P@SWV6{ROXoCpg21as{1zt&|0Q+X$ zoQpSZnnd54`Y%aOL{|z~PQeR#=Kn{Wq|y7rCa1==VIDQS*%Erevletn49La>zR1*HCgJJk-8_J>^1 zdcY$(YupL4{QzqxwnqyCWBvRYmV`*K!&*dPW=JD$9%y9z(cTksW)nm$f;Q>#FvX0` zb$EJ-yv*m|pkw~44&T#0S4x{n|MJtS*W>)2==i(kA(sd8pvQE^=9hc08a<+>dP+uO zJOAX7Rvm2@(CagqV#<7a-37lP&p5S=0MJD`_&MDFM2S}s1M1*`Q=odif550M0ip0*Ji=QUqZAo!i%+IG%6*0_ym?F57p#ld-QJXB_A@yL z)Wi@(0Pn0uGsIG$0-pZzmQFhXjcy%wviTt+B?4vplN@1UCv5slpMJSjq@*#B)(qL@ zrhMkqg;)P|J5sMX(pjNiPHT|m2@%qXUxhMoacQ-0X)sjkC2B+ba=Ms#pdsUVyLQ-} z{JDs-l};wu2WgzkHx6GcMr8)k{wlqUelh@nzTFD5g$W{*@Rf5{$A`?}ICk*dj;vUP3yKHCZ}&2|vlqv%Qaw zXYAIM)9QbGDahADW;t`TlKlO&rbe}eo04JiqyGyvhUW{5pPfsZvDUgCp?-)#XhL!O zNI(=!?6H<5Y@zuB%~FCDm3HLeNAH*tx2GSPTB==QNNHOh!3~nR03xA|tGFM^jvTSS z3re!j3Xb~FNB;OWj$$Rg7ST{mSg_Y-U53dP($Ckqy_l@2XUY@s_Vydp< zzw#nUD+BU(^y z^(>b9m?BZOV;-6F17?c!b(7F!*%IEOFTPIJ6NP8?v5&g9T) z&ty14iz9RT`iNwtG`wFn{gWZCpuQCYx|AeXdGy34hhhU|UHLm#UY}>B7NoW=2eX4C z`|3ft$3I*NMd%z^=ZRBd@Pj9iPCXum_(0l6;XEOYdMjR+F$u?GI9b5j?P(S}n$#~Xei4tUP?3+*xuDwKs=cj)OgcZrl{bH^6!5hzy0@*) zmXnuCe52FyfqjbUO#?kz2~{*F&ZROFi>g*4U&e?jttU2w?BTBwNsw4 z-aA2?paHr*oxXf-+fvEkOA&D7JJ(90z#5ZQ9~($^rE2j8*G!{k&UZWl{1#sO!J3s` z#Mh8zCB9njc+$eJ7T1gOo2itzT0m0wW)Itz<|C2^et3_i9om+##<39k3o;s%SYax6 zfgQ9-4DF&t4bi-#wK{sxn2Vx6dL@cvW21BL+G$=pm)4h~*PJhyEu$&+j#qwoS3S3* z4~(~ghUz^KBCcd&NojFi`L-ruMX_zWD%k%t-0PNaAOL;zmE`S{4VW(l%jeMVDKhSD zZ}+HEsmTv?FeVW_Xe2hQ9eM-T`My972?bt2kH$=BnDywFuYcigq>tXd{5AhDYccqy z)%13jJ3;(CI5^DGkI){jC#lqP$LTzDOGW?7wTnyJ-r zxN#${gh)aE(%oPx)E_90io9$zsZzUU*Lv~hopU;jQ-#DIpT^9MGY?b{?$4oa#DiYH zc`tr=X7Uz*)ZRQn>p zGUV#7%h^reC46N;Kz7A%F47f)k)mh`{YWryg-S3@=lZ zqM+(x`jwYJ3Tt>1e_ZxAVYgcB;Ai|r*n=i()9P`S+3kHdf`SjBK1KAMec53i<3Jh7 zau*Af^(R zRmF&ggq2XK));Z*p-NGRg16qIbs$RFdB2Q35Ct+_Dv>^9U1)u>Wu*_jZr_fxNE-Gg zJ%%RCIMLi-5{h&C(+7m^>2w27Q+)ZaIV!av6%v6dBVkR3H7Sq@>W#LjA(iKbqGEju zM5GTrBN^950mkZA*m^ok%*!f6^mGs@ETV zBtNKeE|21@w`J+h5#&vLJDK@ZCB(yin^$Cyip$B!7 z$}{Esq51m|xh3w2@{^ZPg65!F#iB(*vDXcnZ!OCNTffe@6n^^A*N8Dw3n6r<*j|+# z4_oy@&nT~b|5Dn92htw$nbCFwK*y|#V42W;AzmXX<_N`W{C zMCnjuT*DIJc_{KwGgeapD=4n}McAZf#dRxzTEy{s=5_26baPutdJf@s<3_AG$&!#|qO5-zLt07&vC z-gPrJidABXPUDVsusCqIu4kM-+LbBCQs>AbLA)ljtRvO93FSdORl{)*p=ts;y~o%v zpRbwQ13Kw)Ia2Jh5=!P3k)Vu33$aOLj?)E7hm9%Mpflde@Z4=wLcp+PIf+0lTbm+y zv43}F2m{e@HmVP~JF#+yT~{k`e&ynZD!MrYK+$%4LRo~LBg&U!pE5bce$Ois$a6w^ zk!g8=7gTaIl3!tJ{m?AlR;lt1Y~D>5dvj9ZK@_09>>ef}yHzaL zPYt5&4T!jKpBDU?c;~LDEar<)IlOK-D1&=9uBs8>+(&+SFjP)L3a;wyGpUPewa;m4y$4lGodVgBNb@-Y7O*t^|ioFTz zjDBaBZ>_3w^Lr$>lishnyiLD@>9yg2F0yZq62G(QTfSn~6Y$~kA&##d;2pJ=F0TNf zyq|_{GENW-hc=WC*C3))Ou1f=^b4UXe)`pzD09H!b*RG#^YD%`J>Uglk~p2N5T|%8hoiCi z&AH@?H8z?>4Y6&Fr0Gw9eZK2skb-P@PFr1oZaJEK7(5X zVU65WfT{yKg4c=X@d=2A8@_lHQ@JVT}dB*&w=OBN^ zAn_*^P8J_o^(0UEr=W|}2REpngu^*2&hf1azQ<^zqn|NXniCSF;&u+~y3!jVLdi~zE<^zVp%h$Asm6CFpqV$`;AE-T%!ZC>KH$L8s( zR`e#C*GZyRG2@<&4@|l(RsU>V4v6O!cMb-%qg%iN@eLwHaM!d^xptGcZa7ayzb)-z zXA}3l5B@PR?|e*JHkb>3I7hkhIBm+FA$eZl+w4%WFW0eZxXtNS8vi~-@MV-NP;aiA zXph(`?7!vEB{PpUXteL0PBTgG8Qqi+PYKwvbP@U_$Ny1@rwBqG?^$Y-*W0<;{6vF7 z?3IugG_HL675*N5To<+~1SXU(uu&zjsHYPU8;T}hkLt9)NPM7$kq5_teajm;()Y#_ zU|r8$AT`~IxZ6MO_|%kKX4PsvQz`RtcF-*qmTTEh5t-#O0mQ);U?YYmh^A zz(WF}!MNii_osWkXR(+Xt$cUifMQt-3dXJ>`wN&8s984d`_gPcdYmdju4y~9lle~+ z8@JJhaDg(^cPGMP14I(EcLpmDJcK@72tZm4T2VVx(l$y*05nNHNT}*!cU0zLGGq7MMD&dvj?N9HoxatEY>5+FdhlM%8k z&gJ$a$HJq-AAa(>Pu#}S7tU^V!nFDfGltAAemT5(q5XzSl-vtO()_v#z8zpF zCbpbu0>?=l{toxU7Dg_pa>zmx;@)shy`qYvr~FNB<817$@-wi1_q#ZwC{cjneJy%B zq}I04(c_ju8pf`YwX62pjIOMhcwE479eW6;lyvZuop^Ll(Jmw8G|e| z?bgh=;9EPzs>7*C zM(ljI89qu?n-&%pO|Yx%flGz%WKYGuiE221;#3L;VSK(Ffgs4{r;KzrNj7`rNidfP zv44anBPGUY3;(<6d{V`ziuLGFcE^psbt65?#^#5PT`9JC1t zLFVUGs2X!^sS{{I=|_H5R(`;}pVaEv*>+3|evFXxY{ZM+BHbb@bppCr zdD%bqJuS`fHm>k5s5WtF>y(>`3{I&_fwdHo@=U=-26L7!yxx?QR87&t2Ev~Z7Q`S5 z1kms3Xum6H>0377BqqM8?6weoy@T|SVD8j6)L4q~4m_Ipy@~A$zQ)F|z(7<&wi+$Y z8}ZXjg=^EMLx;mcHHQg3{xIeCBw8G@AdSJZoV@qqu!OWpmiE z*@{D|TZ{>!)B~IY#RV_cy^c5zzallN)uu%&O_&UM_*XRs4e6e!VIXlS0UEZ%LmmZE zK!%SJ`abdtEl%We-@Ff#?`_Na!Jtbq!Z#0xWZEfriBC98ol(`taV;F1nqeAs9aoqO zqkw*(-{WRG_$mfWPyDVEzS>=sYt)kxx!=y+mc)3Fq2x41b#o-F-eVTXO&{l zObwxM{Bp;5^tgRNZvC}ytkG*Dv$waz{!cKra3ux8pXp=k1p-akf5x^3MDOq^|N5*z zthkv!qu$@1ddO+-~+k5Og zRbqK)9k|3YJ){E7oTu|W8J}WX`>*8ASp!L!l3!uetX3Px2}Wf163!a+GBt5{i8K3- zCNYrc4MYOFinjteU0LXql0znNH=Ty)ru%PG`$52Erbq}-Mh5L2Ve*b;Q*4;#H&;$u z>&1|8@@J!}ODswfj1iN zq{4BjInwj9&aX62KgH4OUZ+sY87AL&ZZl&cbZrCk8|0_7Kgu6YO|^Eq7hUvRyc_yR zMuVd}-gwAzHy0egY-(#+BWZ5(xqGynrW{+kN%E&?#UtZ$*jeBcEc6WQWnk_sL0f4b zyCjPkrGrn&woBhsppqo+fg8pLfgI-It?ii6BU3(%xiXB1*pOYcdb~5Nm>}>QMOeye zvl4m*YhB?wL|rTgV~;+;+UO^CztSo(@9>pj;FYN-)0NLtD!$+71uJvxCStH)GY!I2 zL0M!l9Nl9D0K+O&QFSA5G>56q)J?9#B3FtpP)|DOQV)o<-SM`^t~8u6ZCQ5`wAo)u zeOIWJ-7e^DYeUYutik2u)(2&JFy%mGK3M&X=kCvX^@BlkVj#_2D^9$~#-sbrb&#iF$SaOZHlP38@HOtC0k(b2zoG!MT3NDvOWb6(b?&X|z zB4G{nEq8^Nnk^gOo>cnpB>}`{n(!SOXB@5x9Wu>|M3jy)5Q2YFCNfsd9NNto(I)c1kg zurzcthRZL?122Y}6U0OrfPO8OIsl6#I_dqTnTqA@{{Wqr8@n+7fgi^+tQmZFL>3Lv ztIJDB0X9us*yehk*U;*ckh=8fCU%Hpm2Bn@6dCgufd?FVR-a9hKBirU;mi4aJGJhj z8jfSGu=UaPUt-nr{6xlWIc%x zwk-DnfQ$fp4q#8I6`Iyb%{J6jA%@}YY4XyxtZ-qx36FgN9unzoUDmR*l z)s3P!PmSKZAm&!J0u}EG3(BgjKgFL}Qv{`LRWG~G#>zj{?0RGv(0#+%YM?hpqiU79 z6{AHM?^cZ<w*dFTvsIrJHcb&`10rjq@z>{DRTK2)jct;!T`6M8cSNpje<( z$zuf-S>BgX`Bo!5y@T+nVOsLp6CUlU4pNO=Ehj!=*Uf$)%jQH`E_GGhUp7qAm&4oG zjNGM{(ecftjjM{krc+s+w8Zmw-xuE$&2l#{BM4Vow&wt|eOoGA zqCz^EOz7Ryw_Dy~q#?6K{@Z)lJ17maG#KMueB(J}t6rC)LOLFOEc1j$eTWYU zaQrnn=GhhR4m@dcpel4DFMciwZ&zG2>xh4U{Mf&l(*VQZ zi^J1?GP@Gk{#3&Dwxj6uazrFo;iA4vtQA>nRF^U=v>x>$OMqBxJbjR5kD?D61Dwe_ z+=rHdo+>iEl%6MM3r0dht&MD#VKR3XWj~ex44=jx$4KTPhlTtM zWk0Nv1(VO$C>%h$L&2bLRsChp)O}vlHF>{UiQ1F#&_HH(@;;*u2!!PM2c>A1e^m@# zZ}jHz<$AGg&p%`lrnPVK)msek4`)Pz%KSA_U08Dhi}kgy4uq^<0!uZvd;|12G36}t z^GS`gM+0JkGTLYAqBcBCXE>rY1~i&D_TE_Io)_QiW4Yd44ZWLwN|`&K&y zEv0xkHn~Viwb>ZRrVlcFdyZw%WPcX=I5!GCbThFJ7`9sAI_-fkxCL&z2-t%lTv^R&p0(vHPZbUAS(YIo@ zzZW47XXH2>F}rzD6EB~5pV z*g`o5v||cARx!g3vMsI^D+0PV58Q4FZrvu^cKT%9=0YS8mo8Tu78JC~y9P#{Ntb6~95SWhqCs|xUENgpJ((}ND{#d69 z`<`sC()WkG9E6T?FxNGQ-rV;GG?TuQ0mc(;{-{2P<6K1-CxXgW?E7fw4APs*7KHh3 z?ni?rIWS4&4U+fV!oMN)LK?tnI-sCb`j~8ipP%7e1_HSfj2AQxZ!hg&ZocgR&hl^f z`D%qvPbHP%=J*NA?p#3`lMk{i^io_>wzZ#qb0CfEcp}Ub)eayHi~12#%cdg>jx}&2kA*)$q8{Gi>7r#?7lg#rNPW|1t-uwDZw&8*$a8zT%1cR_2&;U>PS?6hg zfp`0JbEBN43{5>lXE^{96kDmE5NBi9T56yyCWdV%j8)AZIzE7-Ke19v(51za$e^lx z_=Q2Fc!thurqndcfPB{krGyFS@)r0E!-$r!=-5L52w-hwpbUcnf|g};^yJLAQto}< zj@dexMI8U0NQz8>0Z7}zkll=s2ro}j3M~#_phw@Q)wA3016OlFGL{DVTHKuQGIqOM zAa~-*%If!@!QXu6o=#hfMBV5^FQ~JgI+0Eb6G~^Df2%^*O8*A5HP!-2NglqdXfyG$#+p4vj*D3K#0*{a7vx?|vmds0&L^gqq=qA^O`?%#OAu6VHS2l_Y{ViCAh@ zRG2}BL)J^;yCHU0CU&?H0S;ft8mHJU*ln$S?%(B$;jI5f>>D^?8Q72jpmhsyiab>1`tgNkVp4@C zxLcn>B8w1d_KI;5#nPFbWq-D-;iWqABT`@C{BoNTxd`fVLf-lUwh#$b>RPN7j+;S_ zY|Ahymtbka>>4duQg&nv)x7UpnOrq7;)fBZ+X_)FiiBL zx796T<@F{Gc?2}aOwGQ=c*=8u2|DKu@8N?6>D-E1{%jJ9J*N9{6&0_W02FRP5$gYA zc#TxBv!(mMMULCT5?E8(gf;LNU&nWr#IHTNyW8;ev<9kuwcuRM4F@6OdTybDH{MOHkZ>o)tcb_{d? zkOPj~GsqiQ@Tf}!#sX{vAD;_jVSKARI zQ`q!ez=1Vax`V0=$~P(d#Ov0_1E~UL54be`F`hxN{k;I)JNMHm&JQt9dSuQAe7Vnl zD&L2-KvO7O!D7#q!OUf+o4y9`4u+!I@1=o;K=4@1F!Z=ag%XYC1&?0V!>=wI>Fbjp z;y<(Y>=!+^_6~2cPR4Jo98rttSn}taio7Nz0?rE8k^)<6+63aDu-+Gneo`Ukmsyue z^UoKc`+fs?3Z@=mTJX4%6fvi*Y5kp1guWtD+L@rKj{*z{%t%xJi~f+Tm5!eVdF($8JR-sc*`bDrk9?U;81Bl61Tj za;ry6wm11%q^6g_SLEu>l z1)u2ke|iBJ|LRUW=o_hx3%e~9-a~ygoDM7a&ieT&y=MUvs@XL7Q+w|d*}xTz_&E*- zO)Z&U+$IlO?gsTVpTlx)@CB0&V1B3ZR^Tn$DgbkQeKf_q!FZ z&WxoL=76a4H$|Jxx+KonwM7$N)GcgSTFxX#J_rR^@A(BXYsjira%W>b=#IBe7ONbD z=qVSB?!4C!fl82y@yW)p-nQJKCo@`;BS}zvv$H=&!%01r!MNg!u2*0I)zfK;V-3IG zA=Bv<=OUWhJSDc*<(I0DQrR4SBo*Z_J+EmU@la{#b&#ajdMYzM*mT*#2{tObP-xN~ zz6;SOta!yQRKs~1-tTsF_i#bDv&W7E*}cN0;`#R&$*el7V8B`e>iOZ`* zGOLA_=T;iEq@H~-h+9wFCD&*%>r|E%=cHIBa3IcOu^$DLrKaJw+v!1FBcff?i=)|O6|P-YYq4>V3uB?fJa zB6)V3ob080_%!%A)Sfhccg_AWDD-=r;sS%`cVyk(erM2GrAlHmcBa0$;&APyQ4+eL zx`=y=io6NoUDTHon|^}HtO|L1r7LRDSwoFTQ)tiy>d&$-dtI8S`!S(LZlCzZkdLC8 z@f2#`&$lil+38`K+bKzf-1*&!`J5vzN)LA+-mG4=P&7tCAR|6%(r=m2BA>Sh2STAt zXlLQ_YHH8a!04;^YAuq9?-@r3kP~SbgIzbFtBf99r@wq0_VM_3*^+t<_<90d<=s8p zJ=SV6>iW&y`BFnBVpZ&)`HE3SWl)&#fuW8%x!Qvhg9UHuLijPeXEzR?zO;Oqfp^}B z+hBI0U(i~QfWLS=I_^nstRv?xr3~#b&u(=)=(r)$t8SU`W@w&ixxxuV0Q*7%WzDFd zMD6rLp87Z?opS{|_C2Xx#{!~lYTi3fek2O-tFd&ElJ75Gh|1Y4dZpx-?{lrI(1e!2 z2&KC8e{f|);P5AU+rhTe`_`_1%<*=fKV3ZLQm6B2x+&~1PhVTzsvf(z-ikPU($#&? zC_#2XcKOnXXkcRI*kZnd6#}jopAzmQe=1^Q->G8fJeS}RLiW46GnPPIeWZUPd6)N2 zy5c(5H|TNhzE^`;7uKdlI0B*+nmmQF<*u+n(^1z%PW-Ex9}_PsxWG_^AYatnVF4pw zJ^?Bsmb zlOwYx)k7*|#r|73%5h{ZWxkdm+tK++k5pmyBItI@PSkS_Gh(#=sb{@^ZfV!xgWqqC zx01ZW`bRwVzB~a##9>cCNBPFS=W6k~Tr8MhWzEqkoVuen z`ro*8Gt*NeAVR=z#eL&b3YCC&7qD6$|1zDE8$5FND@8$sh(mKg);t3Y#wMTRj7HIw zrfaVsx9Pn4ariRsT5xaCGxHG$jHa=;ysTxvu7)hen^siIAUtB<+slO+voH8Ek74sR zaqT8^T}c+WCYvrkbzH&bH_;U*>*G%zn@wg%e%pB?{lYC|*&Ia)X!}g;$@>=8QD^q!8OFQ!2af8prsJ=l70+O};UtihBiClH zE(lgdxX)BsJ*AEJ$=p2cte^p-DL|cuYf~=My$P`8(q}3{Q(W3Sj&zhy6mKZu@^(;l%F316Yw!!vHG1P>igYIn6!)g>NE2K-`Np)OyH z3^$V%XY>p*^@!1Zll@LJr~X6#FV*hix*g6myh{N1a(}+ z2&$K^r#{gi+{gO({qv%9S-BVB0Ib??d-u42NRrsmae zxT?oW1s%>5q;bb@TAd-{xd`B)(OQIO=w?aTw_G~Jf-RzeWG3J<&b{hWV>?L*Ezj{0 z;$qsfw3oM&HYpgnAypt6HMu*hzLk`1J^JC%n1shDFe}v~5+32V#N|g!NW!1$<2E}G zyQ1QS(IM~{DP@x;DR0IakGxU1#uq zp#1bi|3`?-Jrj%_gPwm>%sZT1?1D^%09=7vl-%LTgouu7W#D5U;ZoFlqAm%#Z6K8O zsn*XqX()$-g^u`7g>IPId}*CR4k`T&I#$$uJy38-7fFy1@7ar{5%~KA3ZD&}*SU<) z0I^}~(nuo4o?%aNMme<0Q1}wGg9I_#BzRpFaH!Y_Fo9>qhtRr_jy*TfKKmXX9&UJ% zf0ZLe{zDfPI{C`(;juqXuv%b5hBW-OfZ_OOgn$1ziJp5nIgg}%KlndiiASPsP^Eh8 z%?h4~U=1H5bcFwX#o`wP(s*{gMT&pD5vLWzWgGVGLMEC>W~Gu4sr;Xpg_Z)3OPvkn zFaGn%|4GCwoPb8m|7ZVMSkY}&!gH1v0Va~|7+$Ifp7<2&hS%5k6p$Bp%G|CAEYZ|NlEj z|JRiN&Dj4x{|sV+s18Y?kcs%idhlCVrl3owHn;DB-}%(M-ue>Z{bC(4z)p8Y8^Yd` zt0*Wa;74Flcs~x%YoCTsT~Z2RqBzefQ|-qPK_sN#H2>!xh6KUzIntE#-Rdawed25G zKU+jXEVIg4?SK1DMMhS(X^`Doga`xh*a#pm$f6K0Hk1W8JdYlV>^Pt3_K(c{WxXhV zg=2+13k?m(!$z0$0PXLJi+7{t9txOaKS9I#mxTP!6@oPaDl<1@jQVd4lLYzBeEYoa zq^nEnO&Kv%DW~?2+&HoZcA8DFrK11qm7&>zpAU)KUWCrK$i4p0qeS8>l~t%~|8u;5 z=@AaZ3z3I1;s)rF_OBU)?ML#DS%D384J1SN=~9!om=|DeJ{sSpG#n^aOm; z_Ci`(+ITW*NEjLJ5&sZM4j%H{K?qj(=l{QNhV$*e+=NvS{vTrkrRZ^f&#?bK^nW#7 z2$ZQ2_wxnE1Q@y>a#R+7)%1Vz>hCj)vcNjo8*`BTHLia|7s85o zi*(Gz#ig95>HU`IKi>(!J;W5)6t zhrWRUu_%B=Ff21$=qUIewyWO6i0x2_KT-9_uaeak37|1>7}+uF z<;kPa8;M+FWsn;2q)j($2 zb_H<~oZ*tXxkkIRCmpO&DBLNeUBl3ex2yj8WYeW{6w}Epd^YZuuIjUwL|7oa@gwb8&OmpYJdHHhS92hB*LScm8{Y>S}K(k9EFozv(*r z#MR)#J=ZqiTswIbc_Rh$K|B#To5yvej{7TAwN10IReNGw+)ADBoP&1b@lqYlx+Tvn9&(qfwQ;!LZC1t|iAKzk13)!!9ElQzCMShBIKx^$Zo;D4sE6v6vB=`h-5 zCXFS<^^GV{#hp^M-!>)zSQGS&Xt8r;Sa6lS!)^WCcfIqt&g%e4enw74#%}FvEFDF; znBO8Sh`~h{Tj30~?*r?T<(CzWC&$CZurIFv6V*K^)f)2%;5YjMt8Bu;5=+Y)0GYd^@_{#)tK# z?V|IFhj?4NJm?($eWxgf0?TIJLN~4If48}QM_CRds?TL-&rB8!z zs6XXAJeG&hA?g=~j||J|P$#ekX8+exKEQX}7&Gb50!F}4Z;5?4ei-|?_ZMMf+$pIi zHlUo)hlrHdp2!m)x5?l9T-l7(AH-S=UooN}i}TXaefg5fn?g9(1P>U16+XcDp{xxx z6tDs@$A9(RSR9eRQD}DUtwX;5TZaz+1v?N6Kt8~2VZg@eflge<(NAp#58_upc+`*|-#a*%L|}lU zUI%7tvrwNqe_C={+GTisSn56fZ^4dPq>6H67GVgur14Hff29b6F*o47o8i=oVOP1* zRl|sh;Qc~RqMKV-#8&65ct???SmOCnZ3Xr6}FQT4-_p#8Ln)Mv;c>R@Oa#j5Ru@a zFaipq&L2)fYVo`C;jgF13s5@iMN83=_UJ8zE#8FnFh$tT(XJGfMknZ8Bz_LL5`$N>yjtCs`&UWd^ELH#+1 z@K#I%-~l<49O>%42?`1%1QrFUF%Ic0l4b>N7{OO3o5$SPPb+0}*s})!B|N29(w{sW)I9Lr(u@+AaHF`MBI1na8twy1tZGj>zPBb8nCKeCe*6gZ`w>m^R#Ug{g z>IT7As10pVXsjaQ0ZB;UAXr&+29FPX&qZXWgNtBQVJ{Tk6XD8)%~XHV{SYmfw8>A} z#))@{$uzhm1xjgp`9$R35~j9687#^2Txx>oYD9^fA127q`UoE>n?D}Rp+XcfIiWe| z(B(<|7$cfv8EtdrO!GftvGjy`*3kk~aU>;jRXabgMyK_5Ip4m# z!#^4+?uviqTdd6G{O!AX8M--D7*W?qDh(S zgh@WPlbXQG4bWp!1CBvSJwN;Hi^EiPlnXHGy;32%h(<<*TX|o|KC`_0vYr-qY{iXG zLg0r=u%O5CPI)R!G|}aANo|m%vVCQ!lOeOvj?+qgqrdGg2X+!78xHQ`rQF%QV~h#x zFlFiq$b58)ytW-xa%IziZHdTDf&M$X#IpLW*4?#TS-PLWrbU`dnW7h&v#RE)Gc#gC zx`Hkrt;X0L?~RyXU4M^NYR~i5105_FhAxXn6q)$WmJXY&h0iSl3-OnoO4#s$SW#IW zcNW%+l_IuM!$@J*opYmJIyWt6cZdpmKtMT~N>8$pUA&0qmmznG4Pmad$sc)pJZf}4 z!{Y3;M>+L@@#X4S?v}zCC6n`lA5q_s2nI0M@@SDXm8Kk*wAH*FQCYN0fB-QSR4{?@ z!M(PEn_-(3jc?6``eR9bLYgZWso@`sL1@FW-XELDZF;u5i|tU<-|wPp;i5ll7Lh3h zWd~f?fD+_BXn-pqZp8s3dOeL+LIVw45Cg?qAUo08N|v*BMjW-1OhDA|75?LCspw#d zKtT7C?jOp9q%ux%e)z+OG`=9(e} z7&4)PaUfSOiiVKDLeSOK2ige+u5Npdh^mw?R9$K`$>n0oolSQf$H<#?Z-4LnKZLz? zTvXlKKCTFY0)mJj4Fb~LUD7Sxk}`C6hzJNscXxMpiF64=Hw=w*$G~?FKF@RB_dVzQ ze*eK|_UygZUTfWP-Pd)`3&b$+Z)Kus^7i+Fumpt|4UZW}QorW6zo2(^^qg^@B+5wp z>5k+=YxC~vjc1kNa}I-_iuozx&*J39zO*gn#D9DHSn-^6(;1BBZo-AHHJ}=R6MyRW zx$79Q&TfWpEyunDk&+%)MCRrT$9Ub*iH~db<{(y~-rZ9(57D{wjoxLV7XBG#O&SQn znLkeJ%Oj6|s>C|aNGMaqFRUA!csq*r3lYneBgQ?#_^DPmB5Ebp&4j>x)t??nF2uwF z`yS0#bW=;OW`I-xA8RV# z>1_v-4^x;ggfkQKFE9IGe*QA04;ui0xv7EFD$UNo@V{U3MFeKPZ1EMJ|A+7Y<*fK$ z1Br_~R|V65zodf%NR*a2Ln;5sPyYRpX~5b{=Os_SdD!)T4e_r>{JGj^8C?)!YlANy z{WogyU={z@tv<^m0QrZKoCOMOj~_n6=W>X4vbqoWp!A`IURz4)x#iM1M@~+zYc=Y` zDWPTcBaEvbVr*JI23eZ)*+4}9%|kISfAVGFHq=wOnz?J$*VqmL?p01K~w7Vpvu0bB9D!xQw2^oceW8y}zVqPsnHiF$3&YH*j5vNGo0Tvb}5(}9e93+I>BRfu-! zCZ`f^a2g(eB)jFbFq@f5o@djA(r)b(xX3yzr?*$4XH64QDdm=j>cZ7Y#7=z^ZBU@; zCaK>$5lep?s#)6_+8;|}*n3+*C@q2omvCJ0xu47Dx_0>!ol_E1%aX=l!oH zU#o^alQb{F<0d9OPqE`9v|_zOIO;SNy_7#QZT)Z(5Yo?|lYCcs6l3dEYxvV#1$H6i zi=(gjBfc=-P4!GN*do}yR%gD~N@P5@y!Kpp*FQLzNoyZFaJq>xJeuwvFn0CzMxVNwam6ZisPodXurGE`bK8*G*+(Ci|kdQ5k?AzFK?DzesfcKj62=fo;^xB1CS1T2m2?_N;FeYj2W)4nLPqqSc^ z$9fv_Id2nwXYR|r78ylMUXRF_zny22UEj@0!Ov3SFJ=6s2oW3>;0w4^FI7)R-F&Y_ z74Xf6d|+M&|NrxPEP~W~h#}(7x6I7Y71W~yF$vj9YjbYUE=_B2SYd7~5yuV+4V)aw zeJg5EaI%*oVRbNuIB|OIh`ifUW}$q7hr~X~EE#5ZHTn_xp3{aZOiWMG{o_(wwc|;x zHh80kV7K+?xsTwhLzR?FP4WEPi-027*9c)>Bc~t-6D8Gx97`Qjo2GsS2Uf3N2vpnF zEaWg5=z!Pa#+w|Mn)M0B8sP-_1oz3!!SSrj?6{sVjddLy32b?MIi&n!R z3|J0Gso=A3|@a|I@7MjqwJV)MW=KX=SP-npK$dtpRlk7 z)+tPecs%99JMAJ}{>su}-vQ~P7jVPC4s&Afz<=;`Hl)OzDr7n`nT@Es(>e8dde+oh zr6@W|MgHu1-80ml~;YU%OKYnZtCR90y<^*LYF50?_#h;Ihv6cCJtU9V2 zsK|{+65JRb;sbn?txB+t*F(>@q&{L^lMs;&ZpY&arb+L$h z^a>q`>9PDS<^;DeTg%HCz8b(25l=UBA3K(;Tn1wM&%KtnQcPvq6lrpmZ@H{j(xG|g zFTnF1%d18LPd9(2M)vpQv+QTin?l4fk6q7NJ~y4fO%-ON@@)1wHt#4t``Y+wsxU6B zuxX@s9n%F7$k@Wg@Qst0Ih3kK!%Iu7^fD)2M{K`N@KPT zF7zuaek`R%GG54tS(OTSx6b(M1vPu&6)JoZ74Q45^nimpbI#$C=^y@h;1DNtW#G72IV7R(vX7s%9=pzZay zs@<%yS{3)x^&uE8oUh=KzBYcA`eVyDW-Amb6o;BdEuHqJWr6t!+aB2#ih{=mo*cdS z%_r#Y`YqTnAPm95Q!9MaN3CAH4;_yBFJnd`=#ZdLFr}LyIX*w&d9;dgDR2*H2r8HYh z5elu&B>|nrVF~zc&zwsN#+toJ zh^?$xJFv9C6ZJxSV{vCb{yPj=QYcYU-bI=?Ee%E3nfu+QQ)=w8$??ogs;S*Kh^QtD>jfFDwdp&mm6N3W~vf(%O5R$_X)#ZqGZwk55azwFT$cAY-{<1sn)pFa+ zIaktV&w;p2(Ltsf%spm9%O_WYcBUwMP%V@?S}ppjP@$r^NECs61P^aw@bZV4V;c4L zkcNq+jO@1Ks$@9i?s_Y*db^H})n!`H&Fd6gYri%f0XYS%H}4F}m>@AFLUX}iQ%zS~ z(j}Z*h>;f>in?Z}ov&(CJubgV<2$vv@7bEPN^vnQsBmx$89*IDM8l`gx4_zRYPtH3 zY3g5YD}v-=ubV&VSGcXyk-mSbw?Nf-`SrM_kR$D-+9uI!h%LA|^|7?D`S7BbNsPX&PO}axnQOmWaf>v$Pko~7PgMSCud6t5v5^RB zp3|=u{`1H4#o7Fg5p4Ft8)ZrFu9(*~6{TVL{8AHT4csFt0tt5sle?EE)+Qh{l|g$1 zvd@CwkX*F>94wuqtk$8M7x-#cS1D>GVk&z+YN-|!q@=!27umk*5(<1U=RG$f$jUo9 zXE8XnBk{g^b$H&aP=c%iZlZp@BX>WZlHpadv-j!e1lM=h44UOD`Kb)^uaOTBoTSfq zv@$OH$9kvW-EW(OD$L<}7d(rzserScU3V*rEz3yasjH(?lC{f4-I{r&vO&48ED;6<{xPG;hy>x z8=_UyX0+Eq>Vz&hm?@Kg(1?2&B> zEvj^bCxU0qLf7pWKr-=>fME@W#0zF1W#@hRbx0y_hPLjyUNSmc4qwRA3WPkGwU>xe zl9Cl#D&0LgcYDtx*RRtPb=$1;TQB@V%)COZSWYX1SPgf2W~X~xW9VW6n4g8&ctXGj zyBrPLE${nQ9S++`;szLB!THcH2mc8K4j2seU?b{fFPU2uTKBo0*1>%l>ndqY@!ks{#S!G3d% zSC)m#Rs`;)5RaPYrxBtzNcH?J>ZR$SWHw(ML08hO2n$U~WH0;0*qX%ZK;F_kW@F7v zXM0THFQD!XD?)_d>rEFi8rv>DYK`bq4XCAgS~`~|^F;}cpb|+gFJz?<-xgw*$G60Zwm}dR@MUVw@3(=T-GmU zoNvdimBTO?-i_5Q(N;~gMS7iu35-IWUpQT5l^5Hu{d(EtHotUM!m;Ty0bkZZvpfVi za+QlsUmCoN0v!e>y9Fw)7d7T8*8HTUVYC~<#9HlVBcs0_igZ5d80v>z9H;%gN43nx zCMKh3(%#uw`IqK*QmtnW44@7MG6z_@pI8qM!*&A`Q^I#O~O#l^I9jguS)GVW2_ zj@$XNR!iq~lPssd6RODg;>ddAv_T$T8x`33IwxBBPA!_V>LDdoW|EL4oV=XT`z+WN z@>;y<6f;iy!0tVhU*CUq*jGK0?;OXkoNEEvTk&gh+J6W7?y|R6QR^QR;E$H^#Qi=8 z7AAHN;f;17W7qhjCNR{gag@#UgX+*i-U!12%EqPnq@dRfxg}MmUA%b?ckvl+w9p9g z-&HmU%EQmb>k#hiB-UMe6M+1rRuSxfdgg3jO=OZMe?>G=qL+j*oC+aY8wPiV2}lrF zS>~l29@n|7Mjx^6Y%>9V$GL{cK4;(3*3&-m>)v0D@@-$z(xk=`oHn$}5omZRz>1w_; zIBuStW$^Yd{AXsp$ADdcwc#QNPJS;8V7y7QW$5yXwrR2HrLu9&(H5Sm%;J$Xm}Q`*H&Z>@o21+1~dUFu$! zICtj*uvdFgC+ZMT)h&&j8RBs~j#drR8$uI0%l4|(;`;QKlqLj*C?(-e-@Xn)IvMe; z)LtK&Xg;CJGhIBFbPBGYI>U=PxIIE>N{F|iV)PKo8`U?$&C;9BTC|~7sy-w%DqgB9 zx^SI9djK;^kBy>TU4e55o?_S7khylTKR=s=4s|OiyL>l`{XK>U)pXp*#cnY4OSoqB ziU<;WT{X}&`tAdBN6oD5tX@LcTFd~Vdz?WVD7?AS>33EMlU>`S> z++7qB^Z|?&!Z%!C<`%~BJK!tuK$E^L_OX*}&9g#$piT2@HC-VLK2#vKAwwlXLPv7+ zY2Nu0;zmGEDj&it04y=@6kJy=jh}vF_N(5@N-{Y%rbr=_?5!YiJp6GXk+Q%+;dY2c zLR^U1Vu+Qmxyl4-&?)+I6Y}RjrIr7M7CqEx{si-w0Em@^l>1OAfI*-C`sklO`NC{p z{3l%MFBIx?6;LSWN%{!1vi>iRf{|<>w!LX^kN&ymzddQ0769cwy&C)<^nw3#G5Rqq z&%IK5^yoj1^5KrjFcj?Xfc-y?Mp&2l?Ha-(*k{Co@G|df2&%641i%EJfN~w-Gb5uM z(Dn!BD5~+{EE)So)${7&=+BhyW*g*rhGtBDY&MC3IQ;00Y!36ig`W0QyeM&lq4e>p z*qmu@np&W#?GT64AB1P9@V{`enCI+np!;EcKksq4j7*I0b!>VKNdrJKjV~D!3SM$K zNZ+h4AzXLdU|2eNHudfWMUo}Rrny4Ba(5)=W<#N#YW|mBKl3N9 z%@Ly?Y=u=)?IRsAHtj^hn81PoHQ!*rpalF@_Y?_Ubm}f~E@X*zi@r<+zA(6k-}>*Q z1L3&Ig*~cP&Gbf7rTFnz-%+-fAz*+pBnpb~irPn&jZ)?B* zx!+@!i=R5p5K_y9xt$lez2%f;wXY~?=&o->_RTFbw63msW6QuH_+yv*^~O#76(gBl zU(J=>BhurAA+`$mVweE$PF7N_e`I>4&NAr#GUB21CuE@UGV z|0@3U{Y(~V5GNbq3HBLFD?dd!-%Cd0Fq-tv`dsZ-)P%lI(FvX{x>1$i|8^=ls`t26 zd8~Nhq9J2hb2_N31+h&zo;p56=(hUJ73DEsa1=jhmAvD78xMch_-?cDWQQ3wU)Uoi zou2)<8YdiJw198y+0jvQzpb@#`kmK8yT}wm>+$KM>|F#Ce`9u<-O7wr6qtdVgYZ?U zA|DvJSh<5wUFIVJ@{n#^x0x^AvJ#F}_2RrhA&g!3| z=l>EgF8VJJnBo2%u>R}G?VP}&W*@E)l>2YS11J{4XeIt!n7v5<)xtc6z^bO4J#bwA z-F|$01C&MaHWY|S|JV-zR#!iQNHx}o#4S^zNIiz7$n9_+rleAW{|rC%89vbw>z=he z3pw;Q_!ABA6O_b)09!`#&&LhV@oz7Pp$&TsK%CsirelT7pwswXtIi^4e0+R9awfJQ zx2{CJ*BTbBt_zkkB@IO(P4$2eXHg*uj@-x_NF> z<9SpEa*h$y?y7EX+#xt@v56dwu}>ES;#0k^5ef%(d7(e6qFtCq_uN8}vnyOawjt(A zwwTPG<4%Bt(NJB#GBay)&=%%8&!^Ra@5&Wt@}dXy*cTK1``t7&G)~WdN{$!nBR6K5 zJ0~*U$(q@FYV9mQ=x6J!z4IBA$T>bvR)C}}-b_@bBdS#yAafq`jXrKs|Dzm%r%m95e0uJssS z%YM#;x!Q+{)h=!8NA|v|0E*A?7b2bjL&P&%Un4+(?E=8{2nkW0o`@e-$BfULY(}&A zcYla-Gut4Mmt6@3ZvIIPi*)Qc^ z)JZF48a=!IIICNN>b#=+LpS#Ikx?WKb`q^X>w|fwGaC z?KpCEP}`ta(<#pm_(FFIuYErK_2KDB$U||cPjX}XD7-q~TC{`n@maBLsHo6Jw&^Y& zWueq-OYBV(Z?TP^WN}T>SB5k?noXfnVQ<9NdqOLrdvq78RSsLj?o_U4BMv(g&m6ke z>a9L0EkfLTMi~SyXi1&86A|OD?rQidmDs3q99UtcV91+Fmu+pSo#&=JcZAK#Y znRPUadast@61eKHS3_sY6UGpy206Md+fXWNas#z6pUqKgR#Q42xFYu1c|t18+@S1# z6)wz;@Q^Qbr<*=8=rp@T>`au2SKO}C@bdDGyp^)9^yJ-`cO|qg_40L7&X$Pao#Jv` zCEHu7ODj|^5mr-eWga>8i%8ecdP?t18qyb%#G0$a0~Ad{6VTBg>rMPYuf6f<{>pN( z8>ZHmybsd*weMA03oBKt8PRs;khMqlrz@hURT_nh1WuSEn!Y+gtCA{R>WjF~c1+LQ zbaye=?8d7Q3e3M-#(9~R(~)9hGdRw<;7^sJPE0q4W?pg&63N_hQQ96XXhLj{uTif> zt(MXT8t}^IgGXk7;M=y~F?8$qT0!*BmLzZ(eolV!E>Qopa!w9IxU-4;?bfI#X%0`( z7r>TH#4QuYZuO2!+#rcjQ zZ1XJa1p?;{k9isofaj|$jFQ4W4QAr1n0>hCf3g+BRHg`Nlm-1P@1g3G4I?q1$@6CQ zEnPW6Aled$*CjY9WKTVR)M~6^wc`9b^f<1Lr)jARg0xhU@+q>`;^^2x#jjAg%FL4P zuEjH;NCnD$a9IIS&M-JHXYd+Aaw07MS1DrE6#$2wvu5#m+`cz1O-A8PS{9=Ql6J0F zn|jdE-LHrTeYaV{-ul%c;vgQkKVGeG?-tebZO>8Mx=<^NpYF~iP%CkUXk;?gnQ^7& z+3E0HRDsZ*n5L(BiFv>auj#$UJN3_QlSDgr3wef}6>J=~1aly3mN0^<;W6E)sXHme zrn5|V{<;#OZa3LmFD}2X*EyM%np&UcbRkU+lU93e@h2JpW-}K~v~>AHlI=#!u7A~h zl=KNvcFV0ykfF9U!H}!B$EDJHlZ#Gvi6`aXeOvNOo^X4p>2cppN0D`A^g2bV-mk@y z*>;)DPGzzYtTkT43y^Lc@=uL&R-~hnM2CjrzU`+t>${`Fd%_Uv!k+B;4D`G%f^>V^r&)6%>w7g6BiIEcqA6UzkJz zgC8?PZaLlEH}}V_1@#i!w#uu+aBdNQ0l zp)cdIPp0?l|>WI&DMw zeB36N#M{+x72g{ z0jbN+sdle)&K8-uS(5wSWvymw!e?=6T+Y0@Uz`mYyIYie>Cq!v(D8(-P*X(;YR*GG zm@I0-fp#B@2!##^)aTf>;C3zPOQANVy`@pfT+^FU`tdz%8{OF;pB3UsZ9A3X#9E*& zPqY$>n)}U$`eYo5+V&!ajBj~F4*G5EDzqUIZC-w~##~V*Mbl~bb@$f5LvYQYr556>Zi>na#N!9Y# zlKy6IPM4o3H_Ct#`|CWopMorOD?G?8XV04!MAp`L6E4r;rDP>@m~Jt*!c06qYB>_3 zS-+hFbKnpBaKE0H9cY;Pg`MUtba5iu@bU|%Cv=^6-}Rnv=*}Zn-+zkn3)*q| zyJgUyeW5nsxRMPk=y8ZuyROVcXnJy(4N5z86=;@F(_Qc5HC7noHC;^xZZ>Qtk0*7e zxeRhkyT_2^VR_emn_c`RhbLV4jw4mWcV6h2WU6g4z`Osp5^#CWd2*SH9v1-sqf>E2 zGYm_Y0KuqO=b5NxZMhcaTyw#s3#|y6dH_N1ntnN1?`2u(7e}w|zWeH~hu1@SqMWRJ zVu48k+KTaqQ0`|doCV}yYVf<~e_Q3Vn!=6D@!|A67_>ooHcusl8)}F<4nch9C9*;u zWG)%Buox4x=lUFLf0Xf!0kl`lgZ+!S zQ%blT^m-kgqL-vbXll0M{UN6+*Bt~4JNz_58!_FXRx z@wr;7EQ(aGd1J*jYY++=`T83#-4m8DH1E&;{081};cN>N6Gfv7p8J`S)-C2KqIq&D zV|Q43?kS#~No4s-buV&k{q>}YuJ;JGht&%e6=eFN_OIU-L9?Nw+RRCRGg zQMvb|Q@PQvuxzbS_4I5lx89cz*n9cqRk!qaiAZ<^3BD3eF9i! zOG3%GS=A72!``tMr^pw0O3l8>oEo1UgP5yfouwjMwiNeru?To^Q{ULV@2;>N_w9gKR3Inb)38n4mb`b9!8=TG$qFGQtFjO{?I>MDw8a=dcgQF%=ENnF#hB*ARu zgd-VRz9TmG;=GU3R(B8shSt|?CT?Hr3n!GZj|?YJou$lFM_%T5tjAmeVJ8ZjScLQ? zwD*DpuA-t8@oxy3+oM_5f@{M^9O>$Du2pLb{E}C6FFP}0`FUyvyoWsY-WopP5yN55 zAof+0E5td@h}tPF*=BLgPoWLf){i)SbIxjHV|`j|zuAlVs*^8xu1Q-KH=I2C_=&_B zNwb3!nv--tIn-jWB{|M+e-Qw%HuT&0NRY1`pRJXiK0RvFRWH?kS zs={8m?(^O^H4t4dzt&`dxOr63j3tK|JLB!&2`twdP6s(Fm-u?=lHAV#S`>RM3yC3H zzs=)N{Y4Hvp5f!|YM|gd>wB=mXi?+(LF{ z)srXZvb1c=AV-jHaBrNRgIPTc4cM6}axa~uDfSzyC#`sPKjqh^ecL0yxj!GFC!d<> zld;PsR5cr z?VVZbzNfcC%eyAqS6b5{J@+B5F+jfYeq9eocIDPn+N-$SFPTVuP*QTXn6cGRp<5WJ zPg()U&9E%uzyomeSq@)7$n#)Z97547CF?rRDgWhrMEoGCUx7!kD$4zrgE!eL>TZJ3 zR9a$M4^$9lPlAi##xF!>it!yl?fMp=BRZxvS(E+as+>jruGw#_<7n{ohNG@kYf59FTDx=e$C-Srf%h-w+|=z>E6PGM#0l}m z)Q)wCLz37hRhO#V(fE$0U!CsYm2K2=wb6?3?P5EL?6IzRr~OKB&Q*Ztc&fI+<6Ufs zyBX8>?fH5Q*`%03&*`L?u{rl{QQkH|U0xdf50TDjSG#(z*z?6%#_j5MM3!d94+}L6 zDasdn(TlQWc6kn5EHV)@vWBsPGR~5hB*8NSYaK4v3T~H-jrEsW+&7DeMv~HbmvRJD zmP`EHLsNyv>5b1{rFhf+Dm&P{(7>ku@T2-3lmrb+BY!)pGOa{&gTp?d>LD#!YPmxF zY(uF19e}LkAuPhOj?bU71TpTVJeL+cxA)x-GI_50I#Qpy66=oEXKEf$^aSkov1hAA zdu&&o8>q@2P!p-y%0CYDd)*9#9BB*|$`=V**EBS0V{VU#)>ARrj<;rVmqJg~gF`}c zywPq4M3jsK)D_$2uXWviIIb{Nmk3!!?K?;(iBThbI0o?2inycL$K$K6=l^~?HUvs=jlUZ3#!g_*y83q0tx zL18wj{%RVqJ6)bBN|~$pq<^hx7~{&P?|7N7HOF@?;LCGJk0LDt%^-2e9692teUtPy z30)^By8eK$#Fpm^50qGtn89a24cRs4C(u3smf}vOfGBH5!f;Q;gJ!uK6)y%iEC>H=XDWxS$lGqt z#UrvOX|AgdwZ8;BX%o&c{z>kh2-@rwaxU1NHE*Mx>W+fbFgl%h5sXc5in><_iP4SFW`Kyg1K9FnF#;hgH<8%upw+)t9VBdpGsQ zQq6zW-Frqy$G@j4;Po%=z|g=j@6$XlR@2fJ*QUTIYx2 z0XhfUKIiD>bG4vNruH+EqkwfX2tM|KZ2-F#TZ@<0okyA;#Y8h+ha{mt1)5y-VSJn9 zfSQ8GrdD!rhqDf+|2Jl1$}dwfVw=dw*h+zmd<0RyuF9}YPYjY+E47?_h@w4w@QjLR zuM<~iYcgcK9)Z*YF$gcd*3HGEy<6kd?r?5vKn`4D$dkFdA?C~MY^XwmgXewO-u^C? zk^~h1pncF^g(F5RcrQIFK^$AVSQ@g=f2_vqyy!?lm)sUkf7F%4--|ug6&(93#rbS( zWCK)l>O_OR`V}}mRpzWofD5tf{3-6#DXl(0Rwr1dQ#HvAy0|{Imj|!aEoA`|P^XiU zSFgI|vr88mr`e>+)@@OCZLGt$l68f~d}k<8djxFC@|Gs8%T1QH9Y;BW3m$U}W3DhT zA)(t){6n;$KS1DH0yr|RtdX))RytJcAD+|c+?EU&{GMwH3nk>*s4hn3&{Zzb^vm_; z>;xqYh7+xCc%38g9gQP3tqUB5xewX-=LJCmVsJB7-kcRLkoe zZ3MW6zDmB`b>p5c!A?nnHl*l1yNh`LV|!|ej0LUA1a*@?MPwWC#j(>|1nYZEUo*E| zw>!79m?*P5=pGqgh%_orzVS_)ZJAPTCyTnAyjvK3LxU5IfJt`kVgUe6pNa(R1$hDR zZ}fE!JE8Oyb)IfZoR>yk4l2Y=N%(z(otHW1B=cu)VueS5B2D{I{xlhEZ3MA;*D6{8 z0Q>8h%V4Se6}upmu4KQTErH=@Yy1hhXWJ^zoYKl%byJ|(M5$~i5snbi@TH*nVU+*p z3}2$$U9FmItQUsn1-Z^~abJ=TXz(U@afeKn(;u^%Z_wiYx_HcAd!VGazd|h3ZSHw} z^of#6<3izurR2H59#1W4VBLZn&(KE2vm(`^Lb~u1gHvfkQ-lk2jjNNO-#@wT+0Ev^ z>AI~LYt|UlOy(TdAuxuwm&P&LMDfRb_`RvAGA@Nutb`GyT&^mKJ81|-dycfCv^4~`wn=2#jfbl6lE z+ZN(m*?Tsgr6pgIiqC%gaxSbLR-w4Ld66wn=svwoAKopS^lR=F#i(?a_y+_+k;&P) z8SbCa5U=NkxMvpnlsC>qn{Ahg^d40|V*>}lpQt%^mkcIyU1+INmjWAb&iw6Ib?S}v}YrGm|Y_0*-dXS^5LD0A{Ms-rQ=3@YU>m# zQtM98kYctl1TN`8>t@VHj%lY`sP^?)q_v_fdp&_!w@|9q-ICd>EaM=&N>ZtqRj5i< zrlpkw7pA|`2GbL`r!bv|5E=?h0VvoW2bX5g5cDa{W4KoGYPI2;bzfp{3muS${&Yt( z5#YKsw8D%|a$im1)&=cL^!v_t5dSR^Y2bbo=`KM6G&id(A zXJ^6rgk#NO4R!;E?Njdr_oH>}f^N167!0UmbpQn^fu4pTOl1rWDOVVXQmYh_LC>a~ z%)qs%(AznA-dtNy(;4njiI4MPT$4DF+xIo0;A?xuZY#@Q<&XK!246XJhHy-msJ(e* zIailMr0Q7~g2wGM&KapYI!4^;Jfd5otv8q?Et|+j^}YBb(NheCE03%P;Q@7zh#YCw zL>(0IJa5(G=0rqU@7;YX&>jmIu?H zt%X5Papd{~IDSqF(w4y7{Mj)4DUW%0g&{fF{VlRUIIXNZLnktsox^KdobB2frVw1S zkRB#SGXNYm&Nx+QiFBtY+I(sT`YJcmEUHCOxWT4;1p#4p8h+jdY!b!c^WCIZ-2X_ngGLlKHnYJ+tv$d6!o8 z_ACOU8+`&5^V>ONHwOx=X+)tJMs57jj1a>T{n7`NOk&Co_Q_`ZW zlT$SePz!ZNzLmk)EHi5C6byvf_m&-)oB11}V_GNnqm}Pujmo{8Mautzsne&A|vWtJXiVV z?Af{ve%*Z&-FBC|HOsb#8+LZXb`rK!Fn5>T+aDw8zLn&a5qtr_Q5HYKtB&nH3{{sU zR}LnecJCloF9<9+;fjja6Amn>+*Mv?q!no2OIkxDywvORgm&f~O6}K4T#iMAU5*T_ z)cCHBIi=$7m{c?l5kz%#WVHfZJaJx#x(TGETxYOjN(HuF*gZLIvZ^!GIlM0PF{(8J zO>NqxSgnO8dqphj+~j*=(Zy~bEM7qrZW7|3I{@0q7Y|B*Ag*NmEQo)h9gDczfXwW) z>PT1SST_x7H3M{2Z%oB>*$Xw;5<}{)A{IT_%qIqGF6~zCj(bOE6|n4fE8%zdz{k`g zX%eLU%wCTmSgiyT;BJVv^UnH@Rq~*$xQaL@by->BGH%f`7J#8fSLc_+&44#C|E5pO zKQn(gga|{$cU->2s-AypV#Av)%xtnWhYep^7#gmkcJw9k%3i0J3ATxwS}d)E)?Y;h)JPRo>M7}z1H)cS#7^^bRaxu^BHe* zHWg(T6>XwxJpCA}@IHeyWG+PiG5jqVka!7n82;%y1KeVbo|rE%XS1jObzZ|Y%(5y{}Q z1>YBZaF~coJ1)mW10RW}rN6rUe2nDPp3ghYbh74sjHyb!Si}=UbipjC36BI{2yjWB zob{3{{l&~cM>q9UGM_Cs=JV`88%pXiv`bw^)2L0g&X9pUYq>kN?(4Wv{dTsQeUyn;&LVnaS_Xc{!jmo?pNQ;r~J+K zXT@N7!|Z-bwCfJ5Xf%~<#O>#?w^KFgl!PpVj{(*)Al>|j&IgdidJ&{zsU^dBuWiEP zFuGuh`(@;^7BdmWiJqHA5xCyyJu$Mw<}J%)#3Y4dpPJZ)OQGJ`gBA2U=#oqL96 zj=85&R!YH!^zhA?mme3vx}`XrMH;(VICx^mJ^d@SO1`f<4VZW~d*4rSjr>do$dnM< zbRwZOpu61Pqid&t!$KCd^^MR8C)`0oKg^k2(8`DqFTwd}rR8Juk&{Y#k{cHLG=kssbmTSsn{L-f@HJxsZBn32$l29_wmtgm zKnM-C^ri4yI}@R5{oDi-ZOS9SvX9{B|n4p!7>hp2b2{i4F!mt2Q{W zXyVnNjg@s`VO&vde{W!H2C&dFt~~)3TJm2kv?}tb))^*It&8)CSNnR9W2s-MK6Yi*zVuy} zI5Ap_SrFa1lc*wsvFcmvz;PrlFIi~ls^3JUq2_d&ED zkJh<$D75vHW8d^K=1Y^`-4MneG%t3<#$h)X&GYuIp_rCbo2F8T7u|vkeZ-Sl9KUVdh-SgK~ z=twXU>DR2Uyni<~`uFQ7jR49NHZ!NwU)>1*{Xv4n=&&9cNaB6>uEm`iTYXH6_Zmm; zFP{I%B(B|drZoE3m>y8KzlK9x4paP0_mq{}d7~YeKJF=cZXWP-QC>(L^Cvs1%(vqI z+|CL9i(oznVZezHb)+=PSHm|luHT2&)z_SoPX`RXJcBp>pOm-HZSekRu8~+CMENI1v}2$7!sM%-8Ch$;f5A`*G)%k+JJ&>@^OA{}n15a$KsbF&g8mY{ zf%a0Se`MR7z=@Q7ol|w0(r}D_*p{=$4TKMqBxA3d}EtPsf8vDe3u#*WE%{aw;wxR!Bi$e>&wXRma&ncYzu!cfOYE zbM>I_-yLL1+31V1oo7GCvE1rsRmxMv7UslZs>=!qp1q;Az?DQLOSel_bjG?}dL08H z$$PmY_rx>at}@dNB}5Hw4QQ?oKZ|sF9I{FMAa652ByFXsOaWERz$&oyvK+ zIM)rKYP4){yMZ6!9kOjM@{9l1Zv0MMZgf}2DzK?~4^Y$g8xq!;wP0#xAfZ#m#Z^1cHN6qJcCWjV;K~(t1@?k7H!(Z;3-fjjzFwgce z5g-ULhNjK)q~c;2NN4&q%As9p$(L5416)#1jz#ca{zeFzl{a|)&-#2#6&!JI1kZO> z)lBc?##%l8my}!!51{QFVRUVsIqT&N%?DqWmCP-aTI-bWZ=N}9Ep?vaUNFb{8$MGU zRb;j8chSAKzBYuYnt#|+VQX3Fze=VAICZ7}gHxx#N#ifX1DPzLix&?cxR%0X!GK>5A}-OgZ}dB%=Zk|vH)LmXkG57hSD`{eOlpz|04IukkH*(!uiCE z!|wY^d}7Z#v!ml(dxLjvcOACKJ(l=<&&E0%sKjUFxWyX|6lyCXvYLK=TW=(JTuFXX?Z%t9MJ5xO*XzT#FNR2NVyaj)$Kg>1)Vf)D9Wc3vTICPI^d-q^tOy$<&@w4Odu4jfznfn|| zq9<`fHGJ0TmH~J*D;(Eots|j=gzzqx%P*-l3csE+YHlx-pzTiGiE-362?-z{kf1vN zN8@}yTy=e7*k-U~r5>Ikt*e4Zd>gkiE=zlzZ%{E+{g_L<;fM6b7WFtJfymZk=m?{S z4?id3TqqR7Yvqzt93JxNGI=guCSHhA6 z@bkd&rTJ+q0T*NWP!#c_;7!W?04^!RgGZql{41Q%S*%W*ud|QTLpNXH_t-^~@X*`I zESo}O5w7kw#4mVxbi$Yo!NYF?&_AvZiv` z$ClMbmckhHbP<|<{<-Mz&}MFf-TKnEv{X@+r_F-p=qcP|fH?zAhe&KWr(+n=WdPjC1Tal>=$+d>r5(u zGtFnC>Sa;mKh$O#;>ufxG zdlX?xzCJJM3mNN}4>Sfrs@_eN{o#lNe+IdIvgeLq<9F@r4djmNs}WsFt{yPrFaX2H zeu<{8+CZkA;@T-e0&jDi6d*ElkyF`}w>4B48UKF;mw+Xzz;*!ijW;el#}T>l^tislJT3k`*ewqMEmU zGy(#OxT$84TLHF;tr_P&L=4?jsR$8MJt<9q+ z%wxpQg=%`YB+SzirU5;u`lIW0Q@$%!v1U&3c;~actT&Y3Ljk~mQ2FoMV|JhW4$vEW zeu@TAk-VRFDCL;1D4@(=UoM-VmbP$6YATHAak zeH2l`|F-kcoIDRv#nW?XromW%DUbhL?;Q7)@} z_lF2FWfhSy{9_ae1Qqk`M9m zt9~8vR1f~)x|G;1wXq&gAsK<8DCYsT9w@Dn{w@n=nf z?+Z~MfdU_8a+qrxlhQo+K$6E|0pr&Bfrz+9c|r>IBASj*s3)VjscH0+H!4-?a+LVZ zl(yRS(RTX_5_&)rqB>FS!(iU&aqFC=OnAD_YrV7OFg2Hr2@XwcEsqW+#!8PDcBFa_%tB|_Wg^>s$UT)HB{Iz?7_2xn1dRM#v@5-;|@FsFM zhn5gy1{t_=fVnK&F$SE6{1~&``O^}4o{9=7dzpks(p%%&*WNffJ5DV`5q=8+PRYg; zIJ_K@@J1eTJ-$Ps_UNTqH)k*Z)en$jBt(4iv%d?|BP)xO{tOm*`N{Cv6|1jAJX!t` zc3ywnR2y~wL+@@-aP4R1745H)##YFw33vcOShRjEQL zLu9>zwO8g%{j@QlD#t}9K(gOPyDyaFq2*dAR898YVD^_s`a}w+P3*UYPsZh28*C}# zm2~i}6W@Ybw5_sLA?`IeuQjw;&_}0wY0ACDZV$ET?yJvKmwOOU*&Qfww<{$SZ4^WC zkyI(;N5lHvZW@{&eiw1~@+f2}H>}VFvQ*;o^r^aX#gU8zL1|{_BXx0=4s1>3*EWUJ z@4F-qXsMCM^3X9~v@SWG-nF*Mv-i?BKChSKZ3L({uI6Uu=04R)`b&ySl4>-dO-Mkp zWV$B7xuhY|rbrL5TER!;+#v}WHF&iktXhw4$NfpYv!eLjoG6M$A6P??$1Ak8McKJP zec8?3;9}@A{e5rJ`<70$7^xzjiu)8VUwfRjNy8dNjWObIL`Knm5VEUkyMT&?#tAh{ zfBK2cKRmr=JAe&^v%{DM_4xW}-Ao5%AIC+uBX|RVFDH3kwWN~~8eL0$`}5UC+-H@d ztixuPKMM#$CyVl532-$1qP+qeVA5ht#?vNE>MxE!^Ibl;O>l!pJDdBs zX@81wrJ=;t!{t*yKM(@?B1bZoXR+AiXT*0egentSkuJesoxGWc*h1^Zdd7`UyVJVg z=0}Uy;AbB5dj}GKH*E=LFniKA#LJTmFqaQw!qGneCR$~d9!axr=Y_a~aKxXBNcxk&97}>5+_addC ziD!Mfmh82TJEwQ1n*O4$;%u$ebR1Alq7(e_8&~jn=ja4YHsq;6Sqth48#%>u$a;eU zvL-z=_TuaEd2{=E=O8Pi&9!Ya;jBf9)npO9;5oSJ?)E~-b??{1^@+t;ONyuG?x#mQ z($6y4LLGOX4awM#S@IXQpWU&QRBV5tVc~3aWNE?}0jG+#iLI>b5O3KxO@K>2u#k$k zX;To)VRT7c`f(e%Ag}A&YwStR5srv%^2nw| zY1Ynxovp~ziu5$}un9AeH0pPcNNRs{YM`9|bE$n&hBx7a@s%8rLuZV2`%NyVpheRq zEirzPZ!-O9BF>sV*RE|AZ8;ZuONTScb}3)Gm(O&gBZFfLUF7I5ERt`xk|(jRFA_I9 zD*&MXrM%fy|9o$Zb3ke4@_65d`7fOI^Hh^>6wtT}z z#hzeRaag~nY2rK_5U{q$m*1N-Zoc4(h}2>r<1;ca?yKKf(BvK%-dOfY6g;&Oy2+Me zsVWnl3R$21NYyNYgfB<^8U4%N^Ke>jX)VgH?SkW*5BDZkzN8h!DrImUrchb=S2ZEe zPV&-71S~FMr3Ix2pGk;`4=q1{iXVshRrwGLqA2*g#8(DCQ<1rOLvlr$Gh;K(-)BXf z-~Q`{qnxEZ`tZShi%j6D5cOYMGa+#u&WUjsRWbD0xnMO?UOfH2$X6g2h24r;Gdfh>GCZ58O92WVCFsCzWwHuBu#A9Gr#6PvQ&){x+sbA zN#?f#RoOb2VU>b>b_ZO_fE%DtJneMzDo0i(5T;#c(O4MJXtqbH$UmW@5A)JV5?y~7 zRf6Dt{@E~t8?OQnse0r0mBc~gy8mETmMRs-^5W%Fw`b0oWlEP9{B{$^RW2(|R{@7M z;7eVitaR>$amE8*2HU5rGm|G;q#nE7rzyE_Qp{kp(o99qQv;-tM_*HOF==w2_O-Na zepWGv7DggP5_tsXy6N90-t@2!T3DlRmK%Srpv72FOqh$ay7SNggDt=X><@dJZ>sd& zkN`|Ehq`q}&P0hBq|w9QyNr|`F|FbE-Tm2puPWdPOzzlKLiEircF!uMq^8n}c^${L z1Q9A)ZAa+Uf{Q9|W<$7450&I|U-#mTWmysnxbZIiN-B-s`Z-*pW*7%sY<|YP%{sv! zE)zu_ACvd?nI&nvX4-I+@JmTSp5`%hql&%ry|H|67LxOAE^MAW`%67h`A0|$fdP?X zhzYx(a0ORcYr5xMrQnRdo)0;|*;|gC1Ffzdv}3=$ZggLzabJXOZP$HHuubllaucIn z^mKJjDTlV)0~6%e3n+;yMv#a-+jCAXi@j-~6~227P-Pi%9Kr?mLsr zoHlDCtQTxA%SHL!8*0RN?<9jTO?#)~2ZAsOV4ac(fS4u-KM zA$52{8ks;XJ<2+H+8cgu!lr(yq=IPJ2QC--6CxbbKAIaN3qkG&n890^h9V(#om6TC z?LN+zm`x;a{e)-nOe}6pg$aHJ1?^-c-r(YGk)tQC;L+&TfkK$%qXmIgNrETJhIJ-XoPNME~0HC)# zTH?>DjaTjU2F1doKNi6mCt#-*UL*2&9vFd!*gM?YB@QD z1k0Ut?qo;L@mH$U_7c7+Cf}Hsnmsd81;Ks4yj&0`r_HJyXy4%QexNu|o?Li$u<@e& zD65XzG~^q{JJt0mX}UaF{~tO>WpTI}F4Ce)nLBJ0tZaK}VctHf%V=z$(bvMW+fYha z!V0z1`j4qWFO9IkRb@REkBg3Hh>APZX2r+deu#sHilwehHyb0oTV~iu^eWF<7g|-G zElw*X6Dr#?J;JP=ZCl(bMu-o*nAV`SMZCIigZbyffXDPHzCWED?vYIUCR{dlBhpJW zr+wWU#{cBe{gTI)_T>Y`BgaZL)O?%q9l&lYFOgvpXffhn&`thckuObqQ zs&#gTNy)NcZ53}V*!9T>Y+@ce@I&H#Ib>IL(YA%_`apT!S@&Ho|J$>MhYWU-6C2#z zkScuLN%1aFxNQ4|08EN3IlbrkB7k+=hR$ltffUBlGg(yX`r$Hs+IJC6{MaLOen5?+ zX!cHYY;Q$E-X=z+8a9C)a{5h16J}NWw`U+%=&K+ZS;dQcw5vQB~Tc; z7*%9@D!9c`%Ekyz zDTxyz3-E`+XS65!Hpu&qkqS+D*KKAKXLws$is^1FH=l5i$v5s+l{y)iPAhHz%^d&h z^j|+k5{$Yb8#rWYqZ&hF-280h#!C{`F1eaXOg*jfRkkWcFv-5+AkV(=p&CVcXI9IW zoZTJ8XLZ|7gq@Q)_1<$Av9+zQ_BYUTdKWYoq9Zc|7w6>H?ZGLUYAp37y)0ZrS=WVX z)>mkO#%PnIo>XiL`c|K#(JmRm*q8A)A_SVK@|nRG#9L(vCKZkAk zECLxi=+n`yDoc^H{fqpe0hWf}TgkFa9BPsF1U0nU5=-iGitBC5#4R_w;Pr_bN{P4uR0 zLW@cu<)2^y(&p$KimpVSq?A%xZ5|%z3*$ZGt}Q6uPXy*LcWXj?`eg@(z%6*|xBd*0 z_2SzduI5HGhZ3^GjWbz(b&f(c9W-olDcizRBa~5+Fx#1MZqQj9R-S|)`)tyYR!F5_ ze?~~RIE3#(O?o8M@F6e!C9g%Sgv;LYrCaXU9k->{VkjQ6sj(_)S@ z8qA_TafQs{5j%^=F1SZ#&e26p1Y?AyO{!NAr+9<#zY^gw!dI`M$l4VvgBkI*9f)+ zMedMcilxm*`@=99I-|TZz7sJE>WP<9o`=OHtxA$@$zoo$Lk>n-ZD--@e~q3x8~xii z6Rg(Ss~KtLFXaxePXi)r8T?~0wyX)TGj8Nv$*07KbEZ{QL5tE&d%{(QN8M|+d%8!b z`q@4TPPJb!Vay9|O4JN&WuM+7gKEU0?4xrKCp&nNC{EJUJPN*nDR#|}>lYVCYxR55 z>(;uuO`admMzI@4sJ<$pu#Z9kaqWdn~DWij!q4?J&_!)n%`Rqc&!bZuy zbKn0+Tcq+l>Qb0kmlq{?XupX``lFz!TEr}Hni@Ze&((#s`T|Vl8YWsOeD1kZr%LMbzDQN{7UPVB$FeUInAaN8jq`|@8O=Fdd^ zQ|R(L!EuAkp1*2?_#x9KX)~@Jvi12L(-p)%{*t&ZzQJj*b1^kDT}*&pW#bk$Tujv1 zc6$>u{R5{#Fy7&v&%&Yqfx8chw z$Vbnn7;<56Y8)8$X7Ej&2hdfl5tA{>CFk}{g7#x1FT!=3&@ki>Ybj-F`tuE4JpNbF zN`6DDJycR>|CG}mi^%B+wZ{+FaY9u_O%Yxe^))fkKbz~Kmp~v$DMuj|9ALarU2T~4ByoI zU&FzD;P35u-V5Maa$a6u4`d3coc0$4W-ELGo0og9<*hwKdZ+3<7;<@b8u{=PQ=49M91yJqsfKxha$p-@C&YB30rg) zoEJ#G-AT^oKgG1JNum~n-L_LcP9dt-mVNIYZ(q?t`))`&#kUc!8bid}7REKs;{LS_ zpMch^O6GlVDzA^JOacOLR8>_M(t(b|Ib-by4~kT4KYLxR@~LJ?f^!zaRvFjYnyFr% ztVC?tMSdC2x+%MlY*D<(MzoC4eBHYu|ob9>;$+7+ANdFLyxU^|xgA7A7C z1f>eNq24*a;do{Qm+IfxXjHqp8(J$;tu|9&Hfl@V*X_P0k?QA8#;SY{oQpxKgzG{l z+zHI42bYg@QQ1CO{ByTdfW|%TdT;W0-M=qVdV2n9?;OjsCMFTI2~|r1oVY$cwIygi zR!aXRXeZCII$!YuCKOJ@o*>eE-V>t{js>sWfOmKItrgkJ%_r=wBPOkLL1Z`M+=~!O z5r(|C@tBRP+QXDqytk=p{lH$Oi>12m?>0K5FYPpXx7xL6{_(CK|9|E~^q=|IPhN~? zyqfSCk?p`o3NHKknxWz(hTldYx1DCR{phiHXQm7d>A$YT492%fNdFK1I#Zy)u%vE{ z1$zpgT^&8x>*mKXNA7uKYu@{>pA=0$6EM@Z!VZ{|4=mtORI!|<{rn6%cJr*fx2~*s zGc%Vi8z&?>$HyEEjuq1hYlj+Ik6?zU)fP6|O8KvhET7|IpbR8a@w+AS%|&y&S%qSy z@tt;W1C?)DB}b}&+Ke$qQ>CZ63C9Im5d#s(_Yy@Yv{S`E-hxRTZ5uiM)1_oh53K`z z@)aHdvCiPoWZC+c_3agN#C)ku{F&>UsCJS6Zfgrx~ zSi#AiUN}{L$lFSgs#Idj#hl()u_ak!si_cn>mqS|+iv>pj#*-_U8#Eg>}OM!TpxWY zu*-~b)%#n?BK^wG212-fpQMqar8^QY{GH7=3Y&ajCL6!n2r--s&NgyzL=%!VeynYX z^7RKI(i5dYDLsR&_R&&kI0B%v^Fh6r^-Bzn8aq8A0sFB9(h)Jb8dm+`_!DoSmfrT; z=DByE?;MRA#p!2_Y8e9Jbp95!K%(mgWaFmL?(s>oR4wV+eqpB%w+)dbqu{U6r&zPm zn=%&;#B85XH}TT4mFZ;dkw;1u{EW@^4nOStmY2xX6?S_bBrZ|XQvEe5N)ZWNj}v7( zgRycytOh&n&b$hXib~w(DK;l^nkhDH74%W`nh5nT&0?K^dU2NjQd zDtl*5*C$ArzaQ5LmKPuQ=5L3+yM3IzC#AzeR1_xdeIV0$vc++Cp(48QW_8q$4k{Rh z6WIJpK)8JpR{Oq|F}a&HL~5w>ci21Izk?yLgx`BRj@xFUqe$o>>KXhYxKCa`?Sx9k za#zV)IbHPmaGK@ynlSh%G@-=Bkv=C>P(d+WnAvTCS2=-0k47#mESrXX4rIfT(=E(Q z$Wgord(oeJ(Z>Kq)k_=9!4b3Lhh+T0FOYm2;XAdE1d-#sOj1$Ad0S%&@v;gW__Ya? zy_g4*CTi%9<599d`IX)iX%TIUt}8V@_%9_w6#SC1?~sZ&X&5CJt`3y5pJ zM6Vsavb?o1iP?A6S2aXnG1JRFgKcSvFo$b z^^9X<& zTzNT@kcl&@CNP&5&%u*ArmN{YCv;V!1DQ>jJquBwj;!NM8BFm7@C5=ELslq^1jlo( z);Q5dDm47YcRw*lCY!CwP+Z$xnc(S9bD!@Emirxg;Qfj8+uJK5lDi+j`~=JR=(+O` zT?4Un144Ek_n)BC;e2)gL~)4ugMcmZ$8Vx+hE`=U-#o=+*Axz!gz=Qr!ao~#-`h8U zHMRB;nap?o`EUfO^|zu^g3o_ERRXf>58r6!*f#QkP^rXx^lMT%c&{s0#;W94=pa=E z{5{p88T4FY{oDGkSSjE8FRqzaImmps&+vJ*_MiZedjP#iiyXCT5M*x=Nk zlXdU&7TNet;ti)yzZD34-yS zcb#xKuNf=#6cd)1_DJFMRIg@K`nGMC^BSi9jNzE9c!Keo5U^voLex4H^|5O|u@A{k zL7(hoW^kP?wr+A_8-(fc(W%*+*Y?j$C3cRu%HI9Kj%}%vR(nF@1X-_O2^%3@{Q0pK zIxQ&G>wZHx=6Z_?=8Ehswbfo%mfabJ2U=Y+Sa9Zw-3$*&)*j6mq{wGE-eh(|*%L?J z+j3=kLrb0XqusO7%B{VW8~TlgPFEqK(1QylJet1$KlgmiPGK724b*fno4lZqqe$&k;U7&$HA2MW9XrUe7gwKDJ{;R|5oa z_4l~(T3X=)rSiiR+vi}dRJB$D55u$&UU9+tYvj@2%1u)IJJ2s;>^%RjSD}7^$=hyw z^_b+;-~G*JqG*bFH#2ZUx@Pv)dVeCmk>)FF@cifE>kAQIg%LvSVr897n*lo@Nqs=H zz}S2Hda*_BPQ7<2CMG6#dUCoyyu1ZU|61-$rWys~HTFHGIFyz}w(2mPMr`C%d6?=a zc7DTrpXyS#YTtfXS88-#;&r~t*Zt74TEy>>EHyFi&ugs@?3-TF#FJrH=-lWpP}73W zq=?0DXll1c-pi)FfbrsEP*C87yw@|WR;qMl$%XXC)V*#ojHd~C-ucu$%lK1g9p#ni z#N)RC6CVl{H$ADnNxR`Ai_p5|v_=1K@GNavxHo35CG&{friTUIT6od%FO)S%2nYO1 zS9+cgSv90|PTYc49eoZ*Bd|uyn0jxX1v)7B-k4OK@bd8s%%X9fJ&*{Wc4rEh|ST2`|@X2_MUv!!Ck zblcG>BKg#GpvJbz1R-Hz(_IHSa4ba2J@9m^Lk%liF!`dBsA+_UxqhW@@1YPpC+QJu z!l+I@cA;!5kXw3knfF`9poEEgd9|$Y!y+@jiN3PRH8LNaR2PBmvmdvmA*R6hY3uS) z)~9qseB*>gO@B-}#uwYmpcber&U{(ScBn2=D~GfQT^K}_3{YG(pCy92-wI66OhHWb z>rZv71i<>o4NAzG&jAD!u-G=Gxfg(BFOYs3zfOxqn9|T{hL1k=jon>_5yW^XBda;x zJ^v?e4}SW%?ndclVvV4DusD@?ldd{N4+sE%cWjwKw`ardUGhX^5pf&PLH~27?za4* z^uP&0iKb-@xA)7Eb)0q6cG8MdzR5IlX?$9e=Mtuwy2@CE+$Yam9_N-JzH*^F>N+9cCdUd|hs$9~;b&@PRYGBSWtc?|YahE1cwkg{bE%q7 zds$HC-l%T95`^t!p=B0ObG@4{7_tF54f)5Fr7M+ZyoFXx)@>CJ9b_rDAW9v|vPxY0 zD|>`k3HL4{YtOA`Az}frmjD}G11~;$|S@)+RCgFNt{xHm|#IJ8s&qoA9AQp zcw9ts6^XRe7{>)=cT|NQY_%!W#pk!1=GuNMnS0u!*?nf!&xL7ntwvC8Byp)AnI>;9 z0jEw$>*#(wq5&0buGWKP51E4`>jUy7>g-8mD)&l?q$WnerI4|+5mV?!l5O9eRUvb( zw+L1dz0~8Zp{9r9Z6)|_1TI*)#>FKN+k9VoYatwobEz(Bu3dTc?2@+HeHj|K^R~7K zax3cAuai%oY&9FFY;K;^h}5kPHkjFhwm{quJV;2N%);nxNO>RLxdu)6dpxY+0m6>k zR%TNg1o0u4-nS@2U zl($|sZFb_?y8b)}aFWl^TW_{+)q;&V+~3TnQFJ@sJxA z!7XH~?i9J$@0AihMcg9{vjkS-@%@BZLewA=aMeZ67p%pHTc5R*&S*L9f~wg0e!9G# zaxo|aqmm-;1>E{4NU~%CK0(6dE5gS~AJ&!SWGoUx`f51D#BDR-G@#oX^u}suSueB~ z7^eRkFs7Zn8Y561v^Ca)Gq)93+n2w&F^oLdI-12lA3eJ3SI+|8R zVchj&x22wU+0iEJNDPn^9i9%tS zn888!IQ0`oG9xgb_0qcyj*H#~yJ7=XDUY)TMRw~^#U_`{7h3w@AQu^Qd>)FdG7dun8`U@_EvecV`MxJ1C%SFYmL=CD?7+6zYKy0M zeHxGF^kR4UCS537HnTsgHhUN>*z7Rx>AA)2lyA`)G#k)xA-wj_qz^iyL|LtUb*$qi zjRkAAOnx-eRTN*7bao7NM#i4UsoG7+6y#uTIyNgJynR zdJN9>d6HIp6{?jR81(wktqpq69?Q$xzaDT?8yRT&;}nqh>p{xk9>>Yblzq-WL2)NE z|12Yyw7`m|9?lDP=k?xi-V(4HFXNmCi;{15+}>m#MlU7-hGDa_z&9>5pZ^6%Hu0}KMv%0~Ix6~HkzemlbP4XT3@%?G4n;}DVuZ+o$ z(dx)q?qlU&2&VRS@52n~_d(Ex_XBCnMZKIKqG5TR0pWs*(KB8_*OsaYRZVm5I2#Q) zGEv~0QsIg_ze5t85?8j24au9X;WJyI=UAuHUFHs%D;eD!rJyxahIKx8~rZ1rKCfpR>o#7*moC2h0lYNkD zIW1Ank{oNWG`4?JwPvYrN*IZf9VYGy_|l5!e9v!MQE5Ny12%wqxeCS6Qgb**qqcgR zu0QglTqL`(QCVtIjJ2~0!r4emM*?Md0W6md^O5Tg1sbXa$T3w=YWmW>MX>Q)YO*D6 zI=)<>*0S(D?%Fng-DFBcWaY>_^h`oa7;XM~E3wR`5PtlUI`A^J%?YQwc4Mcpn#G)~ ze7Dau(U(2Pk7GH;))eAVIPuqP?X&H5?NCo1#8LgF2mgai{q@2x?SU5Myvj4Opu>Yx zWw$~B04;nbOYvsgLhNzS*!Fi5p)7teMGs_Co)dqFV^=f84;w6Cn{ssi^HM%IO7@EG@uNRtnR26ueTkbe=a2++C3fEh7fUYE z*uSb7EM{5Q1+{&N$^iOsJqV=*=Oj1#J!am;;eQ=2cemi0r0k zPfC(TV*Jcq1okotbHGz&UkjuO)FXmba-Bbc&VR#Gc+~zD&!LQ@LhHsEm^rC1a($%J z;L;6#>Dbq4Z%wI7L`@Xt0m2^sTU+_d$9Jca;#z*)9=$kbn%w%kVo{H zy0Ta8oEr>3jGkSLl?F4pV5^!F3<@DdmKl(a?Ql*I-OqVoLZ|ctiHA525RJ{JRVPtB zTckFK-VMK>?`%6V?&S_r8i(b7w)CGb8Y9H*u=TRURJ7((Z}F;pl}uBMpg88&1g72M z!;;lB_J>;0wgM#(qS{YWQbh^YX{V&Qih{L|v&16VhVKevxrD0cYB!WTt9JAY%Fvq{s_^Tyy`o0#hXE+FAQC!3uh9wCb{J#^#!14lsRd&OUXr}O~at0B@*y>>)L!35bC@br_`ew z@R?Lv_EjjJD81LI?daq-e5}GWqvwtWvRX3c=jrF_Xybok8XwWWKUR_=z6%++xhV^^ z6eA-gLGcl4c&*;pC3>}&7rJZp?%lhbP#brCQVKd9?ZJ6bHgdKVft=Kifym3wo`nz2 zyzNh;DI&w7QJ>B_@XoRLFYF2%KIUv9b$xk4Nr2OsEckXgL{O5HZnD$?c}H0btxz~F zjn100?@3^{L|$ezl#x-9aE&LxOyXR$ z^E)x?k{Nm0GkHlPVb5SuY5*ECH`i>Qy0%nu^?v-^mjrw4JQkLF!wkhFhrFtWDYp+( ziHoI&NK1I+&>p*rx{}*4)o}#n^XEFo-XHz!ga>UuQ$c(BVKa}PE2X!=CdFapKsncM#ntr$5_%!sEeC05days?+a+_PF(-kaFb5yThIM;cM5+!cuN_mG-YMeC+g$YkhpF)FUHy?lk?fmK_JQy{! zZnr-xbG@^t0r~p`QU-FnNLy~cwGebctx6JXDEFIaJVBqpzC;RR_lKvsQ;k#f$2Y&n*4sL+s>!NJw$f+9$m|Q2c#h?_|)5#(MBt7|DFFbg*fI zethh?!X10|g*Y{v^wifhw_4*-_Q@-`Bw9j>Sb%Dd^mHUeXnbl*%Q!k6bRaeJ`4iG4 zJ~NtvfspFh_qe^Me6pVjnC!O9;EZX}QGCKX3*FoolLG#pqvF&?+%B|e-pYtd*bi?4 zRrA!Gy{;iOzyuIXfQXwV3A#;MUgOowWT_h56e#LU*A+VNNN@6Pl5B!{CTqN#tTfW1 zzB-WaT*L(k4wy-8_q1U&J$9-UgN|USqEQTrAuHuHKA=#;u*G}pgUa)%SOP6G z9x4bGo(<_yjt7jrrY1MOnnzLj;Z9*<*VFbf<0crXvu3?&_|^DY;vVO~|G0@fZPJ z|Mc0f{^`Ucj~ol!+!DJ)ajT#)qusmhvfV1+0+3;gJlT4-8A&F?Rm~3fkh&1f2*)pB z4V)>B+R1KhG=%OyuGQv0e(TjhB-T$%eW-!(&?HvRDls73IWt&Sr7U@)r!Vy&^ zS$$BLY#rcM&u*`={0k}n&mU`l2F#xXQf%H}0DKNbfk`elCC?j;L$OKew6*5u2L_sK zKu7WaO^y4Zu)YE4PV!i2WKzh_abF{ohR;v)kJg!ScLewGHi^2MOaT~pD7VfMr-FT4IIx-H;NzzF)Ze3vblQH$bEf=JIfPpi^DfKlN zmMEYz{{Y|y)ZMfhG}UZVVknrB$#s|mpwm^v=SP4d%=YVedh+2xZV6;kLYPVr zYfI|zHbB(xQZhbEWdx%q7MOB@^UJh92TM)-l>aBfs0wt!(@PRusj0yOLe=+Ct#^D|6%6;EFp-{WC9z0;|9vR$c3adC0uR_{cX zBF;mLus)hTm7zBJR{I6(eqH@j@%bZOEDeBpq_NDVOjTpGvHmuR!0C2e5}sc8ho~1K<|5dt=Kg_HkPY1$O|A3yK#2&V4D+% z#`Uulo96ts;J+d)Y7#O!rneCv;y@x=Ab5u3#sbOpFrjaEfOEm=Qn6X9OzAXN9TPdq zrZ6@dDE&`m`?q;%h2eG6Re$uRTj?e?R;|K`tIO6Xib{xGYTrYfz09l9OBmtJtk+hm zvcDFl#(9)gV1jaTIia;kx%1^qObjg!HYNK#n9{Z5^XZ{r`}1_AxDV%|X8dtRK0Dyw zw4KWd5&_@a7Mq1;&7+M$YbBg6>B3oy3@?#_1T01}5wSrLfOa(EjrA@P1-T8vg&6wo z3)?*XvEyz5Aa-xR3YtxMeiIb=DF257K>u=xdN8c?50V(J1`4dbF|lxhve7pX-bNwf z^={71j>T=!EkrE|k3PKrSJ-f0eBvoofs9u^p!+oK$iPjL-GJVAf04u4H`$UO!qYBp zg9KhVb#$^tEFJA|kmUloE@z!5@}^-i^pgAhQ>cN-_xTool8g!(8NcsUx!bwB`u0U*FkUDove_U`s?cCFd8w$I75R_F=Cd9k$2F|G9^wj&2X8NR zMr;}IA$NsX5}yB4M)*UYhDJulJ0nut*Q{7qb+|S{yA##Nr{9RNN3$;qHrr0GYq`ce z5@U7k6Y|*&jBCq`Q`61e`}GF{R8%J^QZU@UeMNC=F2A|uben+0z5l)C@-L^0Lp|?X zAf>I!Bdw-rSzwZ_{`eS{m{ar{3T7T+ryHjEpkVf?kxorR#IakQ|sWUwn z{G5$tIE_jv<-t%2HHRrp@kvkp$I|0dp+EOw-4Q??*jPRqdVj!|H81Tm zyPBeKk~-JRH7MvencT?Z+&gw#gU2uTk&QF-1r`{^@fjuA9M}OV2~XgM2W0g#<2Gb0 z38kEpd8PVIa|tHClauaEFO-HQ==^1YW^Dj!!Q~uiAg0CX?6q74WZ4{LsdQsaCoK;` z%ZY~)aN6CKH0^Y0P5a`b4Yd}SLhdHbGYa(J{qteL{!TpJ)O)wyXu0ZJG2NXO2FF@# zdK8Yw`OW8{*6jeI1wnasJQIyt|C?m+*M;vuM&Z`)A`zZ}DF)0bmNhmmW2q7ud00;r z+vQs8OZ|(5(gRRu@_xJ_CVc}`V-E`@My}gC%q`fQW+XOT#|GD9`HjMfd5t3*J6d9Odax5Fn_N1PWmp zEJ*@w_BxLH>IM5_LH+9?ghui}3$xv{(c%7rbTS%*8Mc#u-2cU-cm&-bhLEClA#IP(IuIVzOncbnsN3FFEqD5E4un`wF)L z>3=jApsAp8;=YPEtf@h6I26x;G^W0R4o&mGEcg9*^9ZybmcfEStwEUpPdXZ40QQV_ z%IuvMI%~IYo;Mxp>Qro?U6@|Iy;$qVc~X!g`u?CJ!*{k{*S7PXma_QSMw8=C=WOpn z?Q(H;+BT8B%qi@oWrY-sI4wH6#bop( zgy_Ey|9OPs{ysuK7K&E9_gm`QV^?P?WzI#sqON}}A_FUsY}X8~yG=Br2q!6tp0BZI zE1R$9wCfWv+iq#HzM^Wlm>YXFTQ$G5M3-R8O=gAxsA9d3&t}m7Ws}G<0^KV(B9-ye zQ-dI>E*h21Gj$MCRkiVmNq8N%L6)HP(i0W598q;8ih+3c#vX^a))8y3qQtCOtQxg^ zXHdr6S*m8+Kn2{kqu-vfD0z=5_=B%{v^o z7L1}uauBLQ|HTD>Jlt+JW&CdY9jP;QIf$3S#G0kllh&HC6!V;jA?(EZL_+ot-Y#}F zi?BRy%PDl4oH{`fhXX{sb*@Tvupz?SSq@Rk>$^lJAcP+U;YEI;_tciI@Af~|9q{5c zD^jHC^-+Ty=S3%*IMJg__xSGH zwuv6oXVpA;xkXgaIwHQkhfvGFj=V&yzD`8@unN0uy|I9y=V9jo&(W>ONe21FK`6J4 z>hccwiyV-u`~Bd4&t2$9AdSZ(UwtFch746NK9*BVHaedViO`Y^H}A~krM0{~BLuJY zhD5B^PY}lL51*wahus%FMfa{ zk#VW*BO(1I&56nn&XnZ_t%}m3$6uli>Ln-yeofN^atQcWFwmP4N*dSLP4_XQR+Zf& zf0Gy}oWP-YvRD{wygva}^VP^XGJ~;JDcsS85yLr-?SBlVrV zyYUJKY@+lo#f80R(qM{ZYReZZ9v)Cw;aEYRr$=VGb=S6q=6DDUTy>wa{x1s2lvbHE z&+PU5fnmu~y?MLEt})zkJVb2K7aq{aBJSQ|Je)&m@a&;k-w^R~e*T zr`oNK1(h*nHvVlgr3;L8Y$t$H;6E7vOaF71@hb2~aG#!^Qql6PTJ?;;_v;I;sT&BvhmD?0s;A|;kY zSWH7xZEBVEB#gO;xhVMfTd4Vyf{+|bR^>>~Qb-yPy8Q|pZ{eyV;IEY6_VoUFlyjaz z8r9(SnrXV(&KZnX2N`!J(-1LbSiV{`B-FFTyKaU0 z3^uANTE>H?j7$r?r-0TUI!%0l^w0Yo7%u+tW6YCA>n{Xsn(Ezc+=BBNGfmcr$!Sm` z*y!f_2`&|z+u4B5#o_!#_64t+d6v)CsmY~_2bS~qhEJ1fv+oOleDMrk-T43Hi_bl& zUlahmjcgnVQkT09){DJERx)8*OUpmS(UE4Eq`G!SIlsnALTPlvRUu~{Xh2JIx6b*`yV|b3Zha1DiQ+HCEXzs z(jAg29YZriBT7h)bc1wvGn61L%}_%KLk}=?4e=XKd_L!J{Qd4)_uh5yKi}nAv)(gn z;rn^^-p}sWe!>{%OPutEpNpM^fYBIn^Hbq5Xcsg7OGC<7^ANAiBh0r-+;9=X6}FuN z5^6NO!TNq!IlfeQwWrbhJj9}mT*MThq|eb7oS2Y9!{h_4!Dm-W{)}- z{_i2*{aebFLe$P36;gdt; zBg?h&Clhv&B!I+vvnT0Uk@n&Cx=)c2_BC;c>{8$V|&U^YSX8yFInt%%a%+=iNySrM*# z^|?sfEbZD8{hY&f`NSPKQuNY>PnZ`?b4zg-ToKxth;f z0p5rj)@4O~$TB;NxzA22?86I%1M_TaT*2GT93(HSfWbl0cJPr7C3glRgGV1kAbSpC0#jn&$^0B~fL)PM?g!YPmi2 zgF9oL6>tzWq9-d)Q!$Ou9t{j=krzx313t!GYV)u3&UX=B>t4vje!Kf{kuQg8&e4U& zIl$OE3t*0}H(p|X?Ef8&`~SsJUr+`8 zo}@e*mUg<${e&4^J;0M5RVdt-Jon+W-e*=iq#`-+llAf@U5M@`uf+`bzAY#S{ub)+ zz41DV+J5`synb=e71MZI;a~9SM3ZPJe+Sy;wX@qQJx&8}CpUX*Z!Et*IdPq;a46)q z9Dalc-A7_wUBgcs4iH`zon&$C?FN{qJc*1w8u%tEb|0v;WSI5fM=$&p<%^nsgpU*W zf5M(|(q9czk7T*gq`5{jdidsLbW|9_mBos`{|EKl5S3X0wbtw#-W86=EE0bTLASQ$ zqMk%vhNoz*D!Kg7LW7;>w=wk%Q%2L1b8p%*;61xRdFKRl_gk0T>%S(D{24O*9ia9T zby0ziCCR4XUE}{dc<)aQEUxm?q!JYs{Szg0?Z2e{iLiC|L(Gu?#dNkTNO1}+-ri4eeNP*sXuC-Cjs^B2cIu|_lmS@lfO|5H+6dJ zcsIgXJTYrJ2?ZZmC!`rStc*@hw6H4$(9qGb_IbYPgBNo7##dF{CVT*ou6s8hhsM#wzegVgl3${&^{xae2YG@}&py*&BhH zL?L2b&M=jjm2q?sJ_OFe|#0!AV7~G$|F?*q7$P3;m6ZA7#xUz_qQeskQ(=WSrt^q3tfYE z6xK!KkK(Gn)cD58@c1r`9n=~(>+jsd&@(!8yBiF+m!qCfuG!QnV>$}U^3V;zVmn&c`+#|D|d15ijTa+Ld_ zq=EmF3Xi0LAO1##|B&Oahwrp#%Uv)0`pMrG_b+_BGTlUo}>?whp0uhgG&WmZ(CNbMaofkC}LXrFb%;#2poY;j)<$IElK`pT=P^;oz|pm`q|^u_P`%5Qs{ zOSiZ`Dn51biwa|WpIJp{RGP%FB?>X6M&~+s3Fr!`|DF{5>v<==M^pSf3U#QGD#$XdHkBV;YmN>a%zNksJes z;l|aIlR62)hpfp+?NhfyQX1ArS0~d-T$7k;GW=7RA3;@P>j;+ztQV$YA7s#M`%!M% ze_f1!@Bg49{8UIz|1$8tgPt)uEt@43vQqIFkChA7d~$ww-OJdHQK#&SI#eo{D8g{! zn5ZAGHUWLi1&apgh#fwGwkH`|ie595MkSyn^9Lnb>Mr0NBfoqR2`#?fbKa1T_Nbbz zy@dntGgemCFIor(^d;IK59)FtFgV1>E5`Gm}&oR1TAJwqKSHrf4bZk#$M=X36GgS#vKUJh0Lx$(Rw zYQRswP#%u-zAn}i^6(tRWI?fwpy9opp@oX|T1BPx=4~<|paQzgk&V86(iSh@u&H4o zbnH>L7BAOi-S}Hm-ea2EM>3;gvA5(``ht1-mJrC6$P|xS`TD)(te1%Jy+N_7>A*tk z+LDS#>9YEYl$dqsdwA-9d>F~#qtXZTWY9XE630R4oYF8eepbCyd6GBL^{4-+gufn^ zlrWO;OBN6E)kn%yZ_@vUhPA~d*)_>7ZhRBSMm1zQygz*5R-@BZ(q-AW{5!I(vH;!P zZ9uMNnyEBy~wM^i3Kz+6Ot;XE?<=(|s1_CP3?_MCmB6zeh!XDkoq4SK{sT!f0ff z+sSZXw*5kz5@yMULrml4k#*czEB3t}{{9A@k6TzVwTB`pBH3PY;Abmvj`#*Gp+)D;3`!d@aNn&C5r>q2 zsDGJzG?Jltcj^Qto$ET?7t0oho+lV0${m58hNXb6o`b@@X#P6ie^=d~Y%!h?^qnNA z8ZTF}o4MX&jd*}z+7}a(DHHBmBthPV2L3)|UF|XR5|uae=4F;YDyrwI%t}ajbNh*% zOZ?Z)@0V5vK39je=&^pd3rq#B#SOZLRu#r5-1?kVsI zroE5u^e6JfXfkWlC`h@78s85a&Sr*_d-#9cg$OX8CC31nGFX!jv<;aV*BGp}Klv_J(c) zt)Y?cZ=wvy&B{2QycgW(iT>1^)bWV9SA9!rx(YypI!5wQWNF`^VsR#RTxyoyS)tyevp8S9D` zoYl<0`^KzRg{_sp?HmWlieon#KoqRXvo%SweRLYpv}NX|8otNdJofhXlLrk54XNx9 zX4!NJe-mWtforEy0qp(}vg*bS zi%~!Wg6V&X8TdKsSCbS}rJst5&e@Yql&I^aKH}lbYZ@gk`U^&LBs}kfYC5Q^34}G; zKTG`DIU$H1iv-iCH6$O7`I@?!_VTJ5-GQ5Jd(ogI#qHcN($V*fmi87$IG=3gisII( zW}#+8^cc~p(|lTm^RQY#QCZZ2^zTCwr)A!U6*Nav46BM?4_!APLCZg2fB%M*RaB}Z2QmD z(w(Ik^)ft3y!N5Bz-KKb^G`|Vk9HQ?kGOpX^06#E^<1>*X=&vzbnj;?mp%!b$gn4A z<~VinXfm@q8U4e2*G-yqo%J)?y_$#F#`Fp4Qvvj9;;c(I7luN{XSMkg z&u9%F_A;0CdY0&K<0B`vRLDE!<5;-b42anDCu41ViUWSmg?kvOcQ#|Ce77q=Y(kNL zy30wLe-%#IV~lvrstW-yM5xlBuzBN_PiW{C*ZJ{#bNeW?C!KW z&|~+6SFUiJH7p6&Fxaezw8j2?R!(;JXJAy8j!izxsg@l7pOHYltfeJVVRL`<&8p#0$ha_iYI--q_tLtimZqk+YG2j(Fq3g{cZ}e zn$%wqrcL(g_~FuOq!NVAAq^8f`(0r%h-`v@kr>@__?S`A#nwF3Bmpn^u01>99UT;} z^fN-!q+c%bPYOT9`I+;$^|71n-+t16e|bOk6E1I)Q3U?Qk{s{*#@~#L1&(J!Jj#6L zXu}}Seu&+;?~qJaeb`zRA5YFp0;I|wZhy@J_#c}8e@6jKa4S*7xls_&-MPm) zTzCB9#8Y=tQ_bmr<>CJHc72BkbXYj;gSx*bCp-AQ(i|(#i|tMzq^4I{HOn%Hh!S+O z89-NRN(8K5I8&ss3qm>!e;K#y`2Ut{#WNU8w!!E)%>;NSn4( z6FMpNl=32la}1G@@~UO9;g>&v{7md0D(jAxsbO3Vhw=x`?5NZouN09?3&yWy#KWAC zn^?b~y~)crqR9q2JX2yx?{duA9f}IMZ-v)jbV;oRK98|A(I5#&uyvb|t~0@v62jq6 z(+1q3`(A)1jyqZ*{FYcDEC||PkTZm!8=SUb+_A)!Zlfg=@%qHlr_wdz_D88m2{+r+ zK2pp!huK?|tFCwk807a>1d0qnGR>vMqLMk);J3^QFMERjW0L;+&^nD-rdxrXY)Dl* zK#h3mt(H2(iP`2dd+YA|k_?&IpsxjDY8%TaK{aV6^R1;`c=9fM_!kt!m&FX0R5P9$ z&!cKZuvrB}lJ@{G>-P9-!}wA1*CUGhB=PUc)X;Lr=L>QXH!r`mw$JUwlH~jAq;&}f zox=yqzcPdz5ixzl8hT5+dzww{(fR4ApdV--bZbpGLI;zjB-|-kD(%>>|qwi~>WIiHqFnFv9KOQnTU_~TPPs25q>5<(i%3gDNsPnCIo zP??^6uDM&bj`UktKZP$MUCgb0lwct>D;*xs*j_A&?FiZwJ}m9Qg;7n7 z)x>lAoH!c z^Cy{S+W4G*^9VysvdCGNTTg zcQ)BEDkChgl$Igs&G-3_Qo^b2AVPBcsy^ZuPTEFK;l%*)d5X^JGHlrF&>Wv??cPW; zc@0mz;{~%;TEQ#6 zi5|(=FlPK*E8Ahz{8mXO-%w61cYg}&*XlqZNTxo$ZqDAQzbSoYl>t;2)qN38TLt*> zsN%T#HJr3N-tTjMzEdgO^;SdJF?z1!f6vQb4rNBv`w|vC>OG?7^AbtCg`2T~3z+%$ zp@FF{+(j&Ck_mTG_@9u`oPOEmx46+u;eR_Nkc$8pLc(xL7!E1=+-*kYk-Z<0;K5Hv zG7!VhZ;oT0V>bH%#4O$SBD-RR)KJQW)9{m1wO_3(YfWF~jQv=tmnBxy$BVbslA=Hn zif~S^SF;a?P(wg64^sDjRfe(wAQ>6+R@ntt^kl_R_=r&5Z6sHP53-@-yQf3NlwRnVX(V zTYe?|Z^jJKA{h9aBZ#)j?_b1su9Sp9T&n)Rs zJK}|(V!9Qol+bh*+&H3?sK>42_4(@dgWlQBSi!20^Uje=u1L3~2UoZcp(n`)Z6?FO zAJw<()(_8E>T23hZ<%ZKUieOe@|;L z2Z4Qn5)Mbh{GvL(C>WWm_QJ;qEZY+9BwP~W^qemJ zO|Zb%gZhclI?g}3m;YkqpRD~2^9PHo>m3sEMVIsVP~xiaH$NT)NzGY!S`IG$cIxAC ze1ly^v)1AKGbVOZNEJRtIw%X9{qqwE0u>b^Vq&9P!xJAE-n@_#7uVP!zDdoCZ-h@& zt|HZa|I5J~hToh#10i<0U*n(?Qh^h+y@~$!eS6NsXSF>oLBwh0yTE+Htmn4R>VW3O z!ZfHRHt^YT&_ols#vq;7&axh;pQg`rqb{gs&>`db>omh-{u!kUPuiyb(rQ2sINMcf zH>ilWAT1Aw+=$;bh`30!M6CyRp#ZNm^3qU5XQjtVlizYuHE9WJWhbESBH5txKy9ms z06TYYQs?$nnDNj|_}E^$mD3mpu4)n=bcKd0b4}kakC(l$pWtzl$3S+CLB6Z3*JiTq z0|RaLLhvXNectTNE9|$`p+F%k7}??O@gCnE^7lFFTrEh*Zgx(8 z`u;Zz?7w|HKf#RN)RC=Bp-iW~FqtY};`lM_SB5P08VK$ooOH zu$=LhP-IgGH#?jlppX%ly$Ni~TwPq9UYsE*R$A*cW>8~icG zV9eMS^$?(koW%!fQ1;$EW^FF1y0}NaDj&S7JFokAMdqw#Lm-@O)PAC@*NBdYQ!6a} z05RZ%upJpk6(-Rvodr5nq{Z)ggAC#b`>a-v2k94`-6Dcl)C|r$ zeRQmo12r0DZ5gvokWatfa`xKFIBj6Oe4&zc5ZH;SkFxFH*}_EI#v(_SB-RBu3^#Yy7qp`%4q{Q{wwRo)ucPC^c?ZZaA?f z#mx<+Y5-LYzqLn;dl$NW42C4R;V!VboRT=quKj{pqExs#`UTk zOq#R%YF=o{xQvgv^^odm8C1e!+Bt&##|9%_U(h4Qq;vBc)!J5Npa@QDI_*Qlmkr6N z`PuEGis{!5^_=aaRh|mQ+n|S@W481FZQII)k+H)}KDHb-sx4(Exmnh81(2~wWetKWtYGh7aO_E6_jPrYJeGP zqnZ99jv^#oae@(dqhyC690$T{V9DIg7eH)`BE*MN<7(M9ziSh34<{ib-3n-K9o_G^~u+~ zmKs*)x!}w5JQstbB!nbwH&3{{^V>D!BD-D6VU3INGe-$NvSXszc-U1RyBM3)*0_pT zob=X);Zj||ta%3_H*a6}n?Qn2Mt%vb#_MQYF#j3Y`c66d4NQFT-~RLQruEJmZ~d(p zmPeU&wR2xV2)xgXZz~dyAnb$9XX#hu)k!&Cx6@ek%)!=pg&~@#kObJwf_we@~)7qpfaIgtV zE1XBrcEwFLJb&7~5)S1NI*39}!4%y-zU7U>Jt z{&n%N!+rGwnNjYpd8-Rg0H~rdQx2^TkdlO5{3Sy3nJQZ2DfqGG^%3u5Zd zxs9DmCJvl#o2!+Sq{^l8V1+f@1@|T(Bz38LTLQ@G-MHxC0MTL(5iFHTjjB)-+O3RR zawshI3=jOL3Rh`pMKr*5WR>Ry_-f`puGdWG?@!d1hF9hQ6LgW~f=w4#9#P1aQGOpj z0l(Sk@2Aa+$X+%t$vY;?u|k>LPR;uR;w=qgE(>#=rJ>3*)d`?0t(qsVjH~6*0_AZ; zFl&H-c*|6e2q1J)j!x5C?zxtWriE)bx=O}*TmJ~28!XV|1{NPN#;rFs)mk+#PiMcJ zlt8iOYhF-Gc*v=)RG#oyT9*`#ReOB~)0Ziwn?u2hiC%Oz66 zUS+Z^tmB*20ex#;7QEOYpRZ1|<$Pnv{MNmEr7nd~4rFO4ww2|}_jB#-AWW;vW;Vgg z-Dha2IYLw0ovxirkiGIiRF4OF%2H?$kwz&Msv0y9a*%2uA*$CP_V}~}j)-}6L}?)( z+jsEprpURgl=gwUJs-tllWVJYdIJ<9P0%W(hGRQlDs|D1ab^llce0bX2K(z_(6f z-YF!TSQX`}9a)`tPH-e9c^RBLyzzWI#9`d;Hp{~`C(w{bIph!>XnTHE0vF7i)-YyV z+Rxm0;b@yYgxWl^Y_RR}co+tt@0eL7@0xY?w+|UN8lu5T)(0kh0RkY1IuLex*?vJi z5m0IRZNdI%&7h%y5Ya#M6`CKn)7}$S$S}Vxi!?ieqW9xPZSjR%Z zp(||U@&X;rv<)=CoN0|FJKyAH3z(h8C|y*6B$9Vg@L*Wo;| zGg(Aq$&#i?Rx)(|Q-W0T!f3E%BC;}pby{QU`{{s?$*hBG?sO`(U;D^}LIa+Gc3&<+ z_p|{`i^GOW`5SN%^;zK}$q8j=y)CP4E(2!+c8I>+qVwQE4Ykq3(^J)Ha(|-SLo;`8A3{3aj)r! zAvNER&Q(ti+L8We3+I@$2fW9#AYID*LqK&hx3HfrV4hB-03>k|9_bg=gp9@ySBHgF zBkeCJju+*~wzH13zu$Q)#)_x$ZgJz_a@Tq`{fOqUGBi?acKamCs-ByqYSa$1FOzA9fQ^ft_0ftUEblxp}r$#FENW z{nP_?y2#b*+togi0^^VB3ge|Gi^kLBk9O0ijmrwysPwlJQ zYJWHIKNZ-cWDGfB1n~J3Fc8X-*#r~srt^I002`1cpf`+RD2B22U3n)8qWqUx*TSDI-G=TlqM6l z(F5-RQiQE3?7TgvKr848b+#F5ivs?3K6)4S#! zO>*D7p4p|DRwFh(myb7WFP`!V&E28W&qRzG`ORcH*~c#4Nr`)WRGQS@VH{L7~<17Hyc3W{l4M-no`en)Wt%@jF7ts0j=VixV3Fdk#lF7S=E~0&iya8Zq24h{Aj*yy$k^u zLa`cxB&AKH*Q&EFN2aFi7NC7=&~5FtT_V9WUQ)(Z$l?9g_Sdm)DCSB7NLS);|)iaLO+S4ev9=oiPS9kAE_pCl6g}uMn+-hVFB%r$H z6#dWFYZ1Z!FJFgLF>(W+U(HMC6lLD-c!MwgC8M=0Sa0%|+!4Qlq3!qFOb$D00%xI` zR>Bo+XCNB2#y1Qe&iQu&7nbmk7#ra?CevAtk_g)E6dtIJ?4+Ylf zzhExC(livI_j`;CD9ar;N@n-F8bP2?JlpSa!T?zJ;Q_dhBPpp^YQ?eHUHo^H=2A=Vy}f&(yVqdV;uq<5%($(!vYFqAny|{{ zD9+&6`r>Vr@>t?{o2pUYI&JC5>{$6obHrVyLBpo zr$rCO4NO6u_e%~~X=DKb|MV>1i?jM}^? zswlGt%s8h~lrOo5ebIaQSnhyA-!6mC?R($LM}=9tMw+!HkpapZ{2^?lZHuI(y#8R9 z_1#-NN9%{^g&k`jAe3B=g`bco9kcd$85`U@MkIFy8BrAF2I8sg$??Sn_%xBXS-}s4 zbvxYI{mMQ?dZwuuQR+VPdvf??)PF2zn{c+e;N00&>H;v54FOj-&)A5-wY==|PLc?d zy`x{4Spu)VE8Q*FG&wfwG)t|RacP!eOl4amx_4r`GGY95$>IC^v{yGCv}XMFEfVt7 zO`r4mEX_3$QwezQy?O8Oqmz(yM?Rh^sc9~?LJI~?Q1;04LU0{W# z8EKX&WaTd zW^4#fH4T?{@S4^m1a-uBwLFv4o1^`klhSp53mIs^eMJGf6nN)%{S+QNh=0@uc_2~8 zhD-Z4=g2zRVAsbAQW;sFYhs7{))V`fq241l@-|A_NoMEh>T&q4` zYXJijrELt-tRsQi5y7NLc~#G56GB3JoIXwW=$`4FLIn8*}U}f zKqPB(vO}Au5`c*iiUdymhwBXumvgYCnH6

    %3A9#UphGkjOS%M5)<`qz#Ao2`%s1 znH|Gh^z~2sh{s=_=o7W61#RQJuCAppuS})zs?aZJ2dR}7HrvEVHFf0+TJUNSKT>sJ zenRlKi2nD8<;^RcwOzu&b-WG#DXOA8sTQG6F)Vtr;Pf3Qo0z*v+R=XJSxlFgQX*Q$ zhJbmVSuam8q5E;+mguyjx1)6>iXKJ!2@%%7Cp5u1;}U+A zb|%;Vab}_#Su5XDkXh}e9+pqvQ*e_zkUdUNbplNDbtjuFpR&1;0@Oujc{nWP@7sG71jlBp zdKsvW1`6-OU6J%<2?X?D1RfpCjxr&XN3D&V~N!qfSVI#Xn z`JRE4C{*?Y^h%dg-*BsDti}mCD1h8}4AX5k@=A%aoo}_C@9~UI#S>BV3fE4~z64D; z5=+G`DwDRCt4j=pwe9yVHW}rdghX-?9q0Fl`bmL9ngR80a}**|i*IKIV^{W-4^i(I zUnaI!=$@57k22Y7a-h;5FV3=aY!OC0cxgW$Xnjw+pcGUstkPGW*w;CIrE}O6)4AzX zr5SUnYdSpSP!O7==z@I`@(|}AdD!2T#*mD7P!>SL0X8?~v$okXqXVc>+RlPW#vNPJ zeo0jxH!rrHUyb>)VM%Fc2fV6F^KG>G)--h*yF5qSmO!Bg!loQ`y> z2F`0i9z58h7(P?f_S^BGRUI7VaY{dfsQ}Kr@(GZc2)m+b^1@|X_{$TJzR{4{%X{)n z%>^9iO7bn?eNnmzoC1Dv&gVg-`?3$B4-TcPQ?jK7&vOsxEM&q8w-;J;Py0_L_!ag~ z;#VRXvN_=nJ%A4924(rNFsqp{AC|tA~KAv&QLE7F}gd zot}UKFv+K*gpy@7_wfx32g;4o{GSrEQK5>KAeim3Pr!4zJ3~k-vZ!biMY`}+B}G{s zdIhh*(w6vmZqUol;4?wBE$=Kp>){}HYNloAyh?KZLqUbB4EU8~b)~dlnbB48sa1mu zmJc_WGK;XwN$7J=3Tf-Jj<0e4Wd}3&d9-F~6UoxE4k$Y<=YzZ+bcuLCl$?ZR(U+v> z=k179GtRaJ+Cm+e6r5gZM+sAmv{3YkY8))2;x9K$1)?FYB3(|du1(H$Km-hc|o~6!94Z}X(dbP4CSrB1SWqdo{9X<3GcXqHv8m{F8^TTXMQort1vsz0 zD`L88Sdhz1n|KgEmMT`Ey*@Fs3d|Mzwu+f__0(uk)X)y-bIhS4&$Zn@P(QmZXL3fs zxQa|hI?$Wv6+0G)pfnQCC=cq04Rislml&qmbkfh2P=+;y&EN9J1Ciaa^G?Cg!k-J} zv`;oP%JM^e>^UQxNUH7@s?s8-N4nmY5}JLXd_BE{*4+|A#t%py&!Nc1^-6ITwC~3s z%s3QD#`^eQ)?1;N-_}RhGPh4uDv#AnT@;d67}pa2nA@D0UYR3rdu4F9a6PfdCAW+) z6{#>xSvV?+&`NNYf6$fK9@!uQy4wUAAB`WM&~Kbk=?g3fEnm!$m(Wo`2|o5z=#ci} z6V5p@KGQO7*@y*=9W`!}^g|1-Bv_fkax}a0ozK%Xl=oa2QUEK#*Dd2M-yo`cqwRrL z%1PQPj+S?uN>CHfm~xMSpxd-ZvCs@F)h8n5)QyGnTWLwcw!kYnERBryhtsjA@i+Sq zi*^OPy;{`f`^S}8TAQXgqL)+6_4kbAC)%E~dR&(24|q`wi7E}%6tB6lw!1_ig<1Tr z#wW$HmZNkHOC4s@Ul`;aUR3K>N8H%(vWQyPfY;fIB=O5oG9ymF%NDEb*5hLyV_x%V zxn8Nfrn=9-^9Ar{;2wAEONHFORh9m5fm-n&4plNK;eXxkeT2*~v3l7uP~tbSZ>B-r z7qp<;ac5=y!2{V1v*S`{aoXO?jApyR>5?-@<%XW7wx36(F>9a5V4d@(e+E#Ka`5wp zb?ks;qmh;RDZ7jLj-%iSEh%b)v}?+J(5C4nb}b!vhP*A+y~(PA4cRa!wmYMwew8o* zkTy~_9Xl*kn-7tSDLsTuic3wA?0(jDNj`C9S-taFIMw_s>2*E)QcXOlUxTTCXtrKWM8DhB0mTL+gG>%Qbqd zbx~y6LV913dDKAyb_&AZMzvA)m#bqB=FPvB!!DU-QDkNj?$jdBeGUHFAl3u_>TZ_s z@H)YGFdJSz%Kg@AG`7jEE^(q8wo=c_?BE$5sV-E2ltP!=4X}~gkmY$zm~+xkzkedA zw9noSn7ZXWQEmrysay4@lH{AwN)3zLhU8yFHgui|(`D>Ybl48_)32S>I{WX~WQtgC zIuUK;Sk+I_`6M4>hZKY%Ci!Ksyv4h;!y8r}6#!vOA=KgOF4Q(fOiz9OZCmg0pd*=@ zk>rcOrjiCRKV@YFVUN1UcQ0O)I7KIq*c}}U@AFhL1Frmdg`HfmNu~dEI2*N1>-mI6 zfenbs2RxU{hk%z*_UB_3@r19t3bzaz0;076Uc)F;2Xam2qC9;iysyfOI`xhr&kf*4 z8NI-(w-6eaBI3h0J9Zk@6WzRrLb6>(C^->$Y;m#er_t5h++zxQEi$^SR4pKN zL6(D9X?HsQu2U}&bR4>2rD<*jZtN@e9^GyFm+#9iyR~OCzf!e37&sc#a&rv+xw)@w9wPOm4v7NsE@Ftw=RQu5?}t zrp!Dyj*zO>e{LIPXa%d5GwZ*A181p{^Swk1&#)kzgm13K3}R>`;xoLZ3;L-lCF&%M zbzK+mMe`h#)Mn?a=bNWXpqFSniaOR&Py}H?__%6xa#=uOiUxsGf~ycT#F*U+J*34W zcEgKr^G?I_%YzTw&&OxGL)O}6AF2S0saZjF& zcJQ}4K=S2h6L-8OpYf4LwyOLKUhl)5(zb^T#L@6wD9}7?AEBF?0k~^XA5vrYd7H-3 zTkdcIL9#N4y_~R(FO3kXsCuyO-~m1LI+Z_MUu8+l$U1$^?r+~UN7a}O=qs?j8wSU& zrp$-i5F+Eu^ZoW~+`*Tx0TsBS%8JY8xWxRZe`ft%?%-Slwn9St-gf*heT{o?jgNMD zsF6iJnc7wcd@D!G37mUjcwnf@6bEhD%oF>N6An@xSva$JxYe_3Y)!tjyofvQ8z3OZ34u?^omGW1s83RTy?JXE4KCmD2c z#HO6^Hn(;}r${Ng)!RD`o13KjwunTs<$d0(F{2cQ;ih9pf$m~nnsiEJEd>+J`);wZ zs|+{1YzA>%&h0*gXM6KPFU2}*@EL8F1Sy!_JJoLvob=+R zY`$`yCK5dylh-}|G#d9Jh4hJ&^g%Xo!~XHW>-8Aa#Wk^;nG>1&~s zl2y{ubY8iWk`jl3p-0F`0YCbRX|00MT83Si<^!t4nHy7NE3ZZUp$B&%;x zs67y&4vD0CF74vJG~6imDq=XGz{Xd2g1C7KICeeLgLW#MZXMjS`CWsE6Z^7O1G1T5 zSg@RQ-ubV<%Xqx&Z}CSPc-#dA%gfj4Vmh)veZwW(sYaA@jZ;o*ZK`&cE-FuzSW3q3D0GSsU8S@kSL{lQ(Ys%gZ#rO>*U$IBSMGYI4u6gKshjZg z62F_=3Sw8|ElNs^(!Y(qBg67w7aPJvs_JK)jJ`=LTH!;#6;p{(UGf^4zc~03gduxV zS&4MA{JTh*0^6MPlGj&O+_Hg2E%j`-$L3xFns06_J=eMZW?k0Sl&)eDyG_?;f=`?Z zYaZr6ohPs7@_rup=S#O3#Pw8Bh-Z{9??6M(FvN@M);JW z?^&rb#+^mu!i@1BVP7XR_S2!F-v)(kwaU&e?^^7;c60`75*;;bI)p6ZcEEz=OzU`A z3Uu_#CB`hOVoRkFJslVByS5e7#$KjSK(VV&RX@B=kwa5ex&q*K2#IO)S+cw1zv>ya zXei~`Xtl3h`F(^jr$3a&>74-csyZZv7)W01s_PK9XcSCOd>Bi;!qI};(6D%`BP8es zk%$(^VB96<5ikwvYFk=(6eOy5*8B+;19&bScvydI-s?dmx;l$vg!Q^3?O{xfX*GCt zm~p?Kl*Z+HaDz@?^kF7%CT7}a6y{YV3JMzRQu?C?aOlE@y(TT12%08;mKO^X{Drde;R4o$luLrH<;lYZ)!3#ZR>%enuw^YUf5^H-EC27#uFF!Ams0q|ucJ2@gjv|5+D{%`E zREI6P`qwrUO)3uCI5_SL_;+p8`>2+OQ;Vi{o!R-p()xrr;qacQJg#di`ZLKxXDuE1P(PV2^HxE80~J*8*e89&u0C%2&xo_NA$q` zc}BX3_!!Q+1LRCa0ViQ?J*lHiJaS`cl+}l&g{!QP!t@7|VCI_Qicw5JQi(l=TMyHg z6(AVA`fR1viu(`bX9^Y>Em|Q(7yq&MYt8!r-Tu9{bu3loa0D9OE%`sna z(}d>V>mHK($tQ|q2LGt`>fUp!A6dm|tQ!!{!kGQ);hxOK9{4TjIEbk^UFW50;b!jb z?RgM2pPFZ5AkjA5SU_l}F1f5EMi>Z6d80edKVyf!?#7kZOS;@!D^p{Z$9?O_BWGij zJ>-bRy2#WJEl?p!6c4?cMe$QceX^9_bs|1y)HLryhSf^thxY-_Gn?1S#%PsGi+ztd zeK37)8IEBuk*(!0axjQ9e@9SB)kHuirMQG&df}O4v12T$>kJmtIrH9OtesULM{o$S zPj7u?*P+@#4TT;=QDh7l+8$&)FW_B^rILBhJKd}r1~l!@Tq}?DGJw9ZvTz!{?QIg+ z2X=p`c_k*Rci}(0t*J*0DPKD!5fXI2)~zG{n{N(g08cfa5FfKWVC-veKQMaI$Fi6* z$5mt3SSoc<^2mL=Pm57p{UY$dL~3dafiyeX|U-MT`7)9X)4 z_DWK#J+Jgct#}2E_UnrmpUq!5S@#-P23n4X3YwDjls6@LMmRMY?C0@}YZ++^)Z+lO z`dYCds^$4@;EcQ&VM86jxYh~ZZyTq@H+k%2(G-cj=lAhn2?udewB0^U-T+j80Zu9} zh@C78xy@@hGkbrr=j`@5-?a2c5(2#|0N?`}VgtgtQHM8w<;3bDFsBrK@)mfOChJ#p7j| zOA4hgbnDYvYy?#uJ}vThfQX%l);V7g2QqD|CV!smw*G~JCqVBtEKhUJyo%R_+4C4F zi$m{JK`2tdRyc{3nl@RMtLpCS4n3TI$81lBs?hbyoI5CI z`wBw=2Vqr*stocD5Jkhu_DG1wc<;r0yXtYKoS*`Iyt`EjXSi7cRh+A8aC~=S4$v0qEP?+Fg$tU+lVnCEBGd?{jM29xFFe_0%Yr{tEBj z80d1E(kpiz)OEN*5RGznsU~Y}rQRM9?rkiwl6O(MOCNmkC{Z>zKlpL57(=)!S@ucD zhi~Qh|1OTGG>WI1nC!}sCyR9Y;M=*9kLt0R!&WnZ^2-*;rI7`ks{$0BQRh%7@v`4n| zYgJgO)|gx>DA+Oqp7%m?ZN--tTv1Xep(?+7>4D{$}=lC7YH)*D|%tyk>QUJ~Em6A0+tjs@$So3?3gAzpD9P?R|Gx zlKcO6<21CLkeR8ci9l*uj+_m()ZBuKt2EQx8^v|3ti(ZSYVL9?jx@KTWfGYwIWY$$ zYN#ZL3lx!u&iQ?>>(n{V|IhEazWjM#T%ViI`}3Z!*X!mU=H?i$Z&z)SA0}plM-+#4 zH2ou0ZrZX(Xnx1yw@O|x*|ofe+duotymRZ)+Fm>f=ms`~aXgJQuQ-%abA>@hQgHV& z?}dJ%a5;=&x)t;^(3nFaMHI}lD7ix>tZPvs9vxI?Ij<6qCO#4^gw_Wtfis$I;gwkq zBKQ`xHekas!Y2oT4M;L7Ufv}PDl3%UD5y6azgO zO*|M*?&2cIBW=t=QD7#V(ep=3&U}h$i#7i)ege&xPQWwAXh0&k z8AGvt4frQKD5#95f6F+V)_2Lqdi*&ZyHO>~Q-6s9Wr!NrbGn&GfMLNPM8D{*HhRf2 zYzOl#EHIF|Vi}d_G!w=fM&_y%f6wzgfbVDl0j&sWC2JVaUf0dGW)DZ18&Xq5F)c^bHkADjlyl&a=war#P--V!LDr5VKljy%x zf@^Px?RkIR)IaMFrpk(6_amjV)jq?{m+`ef7!teFD6X!P%~rg`L}fc7xjK~|1S5EhRj^=EnkoTRXg|%So2|A zU}oYc5EO3w4;9&uj&R#L;rU-_HdF3Z&ckWdr`N!66V8|S7-)gEPhoN&@ZaL)pXKIP z>i<7`{}q`1-~PSG>Nlv8p_8^9^xkdWa+%M!p8TzzeDHATzrZd#Cq;s;caghE&*0mH zzuNtN+I9f&6@}H++WMa99%^SLG&g^x+w+pmjF8XtIWBqGR<$qq9wXOA*}HiYS1O>jgkL;(EAn>!drO!3M>3teE?4v|2N;AyZa-Q{BFdZ{%NC|wqzcEm9m}Mj&BIqJ^i{zqrx>9Cc&Kp%pno)HB^fj zo_IX^)cIPZD=bwv-!Rh;6pM9X2V5uy0m9|fia9xL80-hg#=_(yf!N6zx^eBY*yt8* zdoSIx)@eKs9RV%Z_q-YQLFniUG4A;fn4Ax0+h<$TgTLuGU4~uWwCVH_U9HRJ;y)i< zuKT9mpU4QEx||jJyCQzowR-DW$TAldEEDD zZ~dbxy6mM7!6v~?MNip_V-afS)iFbFJj^DSlg;NvoxC97rOI04Zr-u?sbC!rur10&w}DmR3mUoCntOJc6H<{Ol#zyS`wK z9y!VStj7Oz6di}nCSnI*h0c?v+Xa6fle>67I1carnD zvTb3fz@f(ylpWs0oQ;&upo z{HN;?=Stmj?@t;?a`#d>jCyR%1~^J8czjLo6r-tE>Oq>lYI%8(?RcJl{=vFMiBEE; zk8~9-X96|4UGRPACIH`5t=J{+Ohsm`X+7~2pibJ;bQJqIQQgZ8_mtW`#Ra58*B^cH zgpHQYKX+ShNFEMq**Ia{qF)BL(%zHtEXaywQ}%wi<&QR^ZD#R&%GMeIqcl2*W?$Q* zo-zV316G<+%!cg)1e4P< z&4zojQzw9mY+6}GSj65n%I<7}Yxc~$nlKYQ6%7Q)<^rvlN)g{$v5)oLt?87m`Y>Os zc4xiNb@+Nn#!*%JFEK82XntSCd4S=xGT5q^v%`UZDa0=4?PdgW%VJ`5W9a}tZaBcn z!#p$MIZ`vld`QxQFc%*St@iI7iUu>Y_Y$##L;ZP1tbOwqklrwSg;Z~?M6SALUlqdP zcUMK^czYW_d(;ATmqLV;06|}yvqgx(A5XYWuJAwfS$eEI9_E+ogn<0YdQlhVIvU4`XW8$J=>@eqiJHb0aCCd@j;_Nzd4~_bGsg>O87jTfLiO#z}Ckf*WFQL?( z4k)b9N-2*5*(sWGT69+M2SgTD(JH@agG$fsVPASm!xde`+hOlc`9~2QlgsFyrkvMh zrVVD|kh%17`!fT&e_D6aQ#-t%oxrAP*`PPr%8Zm#e-A4GIGet${4}4=Fexrs8Uzlc>9mmsh*o{N6s`*uz(P-4m8;`sFVPuYJH?t zXBB?W)A8x0JCuc#)1}os1%etuXIQ(2Ya>c`rK_!Q>`a3wuYp_$+(P--*V{{4cr#%y z#sjQusoT;{w zvS(wxUgy@xE})cY^R;n$_aZO(xM8ww;W8Vt@GLSMmrpy~D#sH`kky1o@oM$GiWX2} z9wEl#ydMzi;c{;)UBOI$CaTWJ{CoD!R%BoPhmEx0AMPHLkyzoJ4T@Od#vI^V3x<44 z{Pig2JVoCpvhAA@+<)euF=1=lEnOx;U>F}Z;+sZh>SuLQVF;i}D z9a%L`>W&saBo_`Vh!cCVx3;ttLTb&-AsY#|s@iyP?v2N6Oa&$~6o>bVP7$f0u11_K zzkWsuQ_DMUV&S$t>XKanzgr(!iz}$tI{-?CD#%r}a@LC2>c&gKa=wPUc6F1ud(dT& zVl&>g2GM3C~=9nHU5Ra%R9;{^!{y zLnx49GA;GI@6^V4kUQ$?66(Cn7@G)nM+U54bt#j&HxbBSis9GD4LYp)v2Y?}?ofa2 zqntn~Y_DuXhps za~o#bHC1VfYxQI~Y><1)JJufUHT?&rE8OB4Xl{qL$G)=O+UboW0#6Pj$7Ch0>VVg+Y_Oc8Yc?(A^Imv! zsM9G-=h}^EVp&D2GeDacV$=25 zi%zFU(ZY19vFezp(lDd?ajU<9R|c(xmsaa1Bebs z&<$sBXdy!{Smj3bYTTeY!%-xFRg$=$ys}t-+_Ay2{|?v1I^zjKA=kqO2$Boc=ykda z8=5(Qf4uY_Qi17L#3*ErLi%tYy-|vp5^*#7HAfU1)LY7VEg$6#koJSi)QM-AYUMQb z8P=H>L5AaQjt1I|y9?+t7weRtHs}lN(^6?SGL`YeK;RPpk*O-hJN;vMDy^8z>)jAD zbF14C%Pl+58q6%m9zWmgfGLKMXL)? z8W^-Ty*TJ4BDcMkP|eV=(Ct5E5yOyUs4_!dOfNDfQWOe4OywJlJFTZs2mGfx3k$ z{hqgK$)#1E<}(irdYm$g-i3%y59yA6LDyq$iy8_WnJ6D?dUzbGtgEKY3an$_OODRc28| zqfrzmRqb(AZYZuW(Sp%Al|a^RSl{h z@@V0T81BfI!aL-c`!%T1<`jbp{Ani}IfH%W5B6rouf*9xI*sUcfy&fK(_s#ZD7&&aTigqD8 zemsI6TF6n3iU@H&78S!3&L+Q3X72m0ZLrI_RD8J~oDF6kQarEnvkC)oMxcjN+PvC4 z%Dd`a*`AzqRVB!*^OdEm{zCM3riK3`;&rk{paK1HE~ikuN>ZQ-qE-GZAPYO2%1Jq z?U5(8F*}`rK3n8*7z)PT^k{y)nx)3-5wV}kz_iLC9-}-d!@A%uz#{znZq25Ds9k)-k(;27v zHYx{yZ;C3Ah9r?{#!+}h^rUTBi+HzCtv}TaCSevzdaf^q)WSz+jN={^arvxpAI*-v zl7lhDXxS){D;@T`kPDn+`1|PF(TN>RMGI-|5Aih&)owYMx!$wx$HUBmsfiwxdJKn@ zG!gdEAA>Qu8*E#B%xj_ER8yNopk35pF0V*2McHP|jWSOuN>Ww;;rq=-^7lrpgAg8w z=B)L(4p50kO!hhaT_8e5H#mp7AMb-^LcU4&I^Bcpg^lJ4o>)1@YMkCUXiG!LtTl6A zIDNXSz8q5S5Qd*a8$mY;f(&|ePi>retSLS#Vcl#$YJ}tVN&b{~FWr11q}=ee{eEJM zPXi1ZoX~o$B$GXMoNUs04N!N|&H}MKRzt4qJt0EnE!4)0gfvvQg{hv)f`#YmdGUh= z-yaFscQ-`-Zo21&8SKJv9T#Ojp10D5E?sbhHS*^~#Ch*pL~068&L&_b{nj!#ySB0y z^KZZFN2!T|Ffr0jjie*laT$44bC8-N7GE}GJYJ8`pw-1}R9u{R!y)wt*MLdog6X%* zk%vC+6;0_2_rH>F$~>CLTfZTvRuGn$)Pr(Aceq^1XvRpGI3P6M?Bt=aq(SN*dTT$_ z5H}JCt%aS*Ib2|A&`ycJ7R^1 zI(2nFdh^G{+(7GYY)NA>L2>2G#Q9T1R-Z`GGSjC~z2fg~wJ9?G!2G^1&w_;pyH3R7 zYetQIoc122n9$_QX?z?~4l{@%H+|0;L$m}bt5Ay$#)o?K)6(`Vshe_@D_4sg3XWrK z05C?z&~*DdQ*^`+GqrGsh?Aj4IlhH1yk0F^)By5Kqp}!G3*3hruU(Y_DumqfRCi9>}0rV z`Los3UY|s#OF#8~mr}R1K%;xpEray;TWcL=l6sMMzv)VHE$-XH+$%Uw9n7Ni3qcsE z8HqbCC*O+CA$$(-zqXj*4<#wNzc4Ud!iU|iS+9fGo7^xydPu?h_!pYD?~)46dx6s_ z42tE;4cF%;YjivI&`Y(|;^ST`DjHi9gk+7c+r)(`k;Bmw03+zIt;2+Td5UOm$Hkce zA$&(O2hphMc&So*^mMMSEo`ottda9{t@DK+{fLj(#A5aQ^1Lp7UkG+>onrsTy1_Ux zW)5Hk$taS<&O5W`(AK~K^2o4S9bk+y8t+#w=!(OJj?=1R*IrjdOiXHN7p4rojwL=b z%vPQFMM@#9`xUXwb?;|aB3gH9MD83klQf-h#gutRFxV3KGyxuvJuoy;65#w@s{cJt?xnX+RSTjj z!o-7pF;0YlbsZxVC+)Pl;b8|?K8Jgl^To6vgj_jR`Y@}BdD1;ER9`iiPqY|nAk%9; z(ecsRnB^4U`!rbCm1?5KkA4$HJB(b8Dq=?9fyGmTgd>aj*68UPA8hx&1!?ZG{$d&30*h@lGezXAG=p`FkP~VOR=lWbMT8| z%9fkk^=$`z4>aY(Hwy`;AUY)Y!t3M0lyhkTJMv2C5T{0(#D@*n>B&}NQ`d+9 zKApm@JYlF9fjhb1{^}!ehBWt1l;iGXLEz+10Fhuu*;$h7z{dJg%29DhRd=S;-EsK_ zWX}>mhEYM@3>0|9`f=JM(MUAn+c%kD<_fztMOJ@VR(Lf+(BVeyeq#6w#j!o>kHlod zS&rvbP#ng1Ql{m#tl>`eoaNSUqvx0sH-bk_?Q8PCa2kY%CUsT6od`#qLzrMoi?+9j zeC8^zT9(Cr)&c2JU_-TVE1f-_ru9s9H-*8#y6iGe@o^2L;RP7%NN_mU0ZXheKwPYbf>_s5(!8cDgDxhRW*}V^9d4XQ{iMQ$6)J5(bAG0+=!>x zUmBVa(=PLqAog_37yxU~t*?}I$HOD+b6Gd*$$MkJ&C*MId|w4tlUQOB?YsziN9z+X zZMLYrR832UUVP2;qTOVMT5YO;u2;)W%{EiztIjSW8*>oR5!SR9u5kxP7KW3TQxn*n z=wqbyt+gGsrN1aZ{EOa%D9G|(i1YRMg8}h!G91Bc$NXvmQSQ?-h!r}V*peGnp<;Y~1qUqELVT_U&b&R*J?^8$> zMRWF=mBG)@-C{7~a@*BXTBR3~Wd=TXh%lK=IoMOi$)puCKob~}BtHg~IBzRpXw{^$ z^)*Xw`DoWr7cXd!huN7Z3uVEMG}w(1SI~m2qbxK-*xSiS`B3+uYh4tjAnPxIxFqGJf| zFJmsB#HfEe{$tW?G@f|f(E18%4)1xm2z(8sfk#{53qkjbGFzXQt~8W%spm_M%+*}$ z$p$YU)jT4A%n#rDIp)rO-X_`m+xk1k?kFarq%b4@J(*N#+byxUu|KWOcHvcUB4Ohe zk0y-a2H-EEE5I4o$q3dd{2l=?mk^job~EoWQcV}voJ4l_`HYcb~m^*-85I+R=}9U4E{t()aatn`>f!BW(n z%y_NSbfar=+;=4+P$P&M-(Pur#Yfwwdr*%&S;@B2@=0q$Vhgxg zWsM^MtBzwZ3mI`2x6y2oy}{_j5S9j%_T|{Xop786oKOXTEcr@D8Yt4`gt#m3Bf@{voeBq^L(;F{}9bZ8-E?Y zD6F}agTi;B;mLw{hAC{0nE9GF9D@!4W?AF#LMK6Ic-n8vqccVO8!LqDCx~EAW3>|L zoZau~WhN;DHMtbGeb7J?4v%&_zv9bnv6rHq(mK%fZ9^Nvb=+4}FLXU%%zo7CK&K8d zB-PLNV6Zyg1@Dg!6Q-=r-S*1GGui(UVQ`LW=1P}5W2Kt*4^8j*xj+AtH^2F0=LKK( z`0uHxk;3sNb1B$pj^zy#qt&V-Tn3z?E}oLZjp2IGe9v|lXJ#E0|CbduMUDytmVRbH z^8CnbkI+}*h0lv^!vu`!yzRITxUsdK5-7W4B4c=_qw?qe_89c5JthBtgc`=bHD36< zH1=*ecC5ouGXW$kr~&Me?x~v@W~%BJ6@Q`@&ffkZlK)~xY!@sTxf%Smy)xTYwK=i} z)2FcuQhxorwSPpk;6DIW4%7M-Z|waUJlpLgkmX%n{eB)PIRBepg=~O(g6mWsF*Gxf zIc-SE?DV0!A7f9QM}T0<9?y>+{?{#|ehd2MqmIDW+Uk#W88?>?$vTrX?3JWLuQ4K^ z3Hj3-31M|^VS*KGlaGr2>k5qm1r`Yp3z}ff`Xp0jqL{}e3oW&tp3$e{2hryNUV=o> z^6~>jo1GF63n%S)f3arx$v-8gKX5_gx;KaC#+xkVibIS7rS)!_1^CCe3Ej@#Md%dB z+(dIp<$ejPxADSgSHbTpk-)-6woharoCrR@bQ6MC1*mBK$-);I-F%RyDrlq}P+?8I zjW;8K1KG?fiF`}#ac6PO^5=)}nTSEZcr?T4kynUVte>kYWb6NSwZ*?3(tof0_f&tX v692zSjsENC|8?~LI{N?P(JgK9mA{$%>5rn*YHv+92|l{ohFaxUZr=Mp`xuIy diff --git a/docs/img/structured-streaming-window.png b/docs/img/structured-streaming-window.png index fa48303e19e779941cd0c489cf320898869c8e91..be9d3fbf8ba813b9883b2e38ee3b42536ddd1646 100644 GIT binary patch literal 128930 zcmeFZRaBeZ7B&hL+Tv{~R)Pn&mg3&x?rz0hgFA&1iWA)3t++#h;vU>BcyafWezO1n zT`kJkIj5nkQ2v5dxwUAfPg6}A)O!Y>g*kQDi(;)eKp>4lf_6#k3jm_sF6 z&%x!l1N7R)cE$l{6kmh=UcRgn$;6oReL)+7Mo-Ub9P*pr_()f5uFsavss1M8Y(6k| z*m{CP()a3#`-$3a`A(LJ<*3Dz^zeDi9o8fFZZSntCOh7h_qGI9mJoP_F1dO9_lt~Z!Y-iuyB_WFsC`Y{1UUSUN{{2>7{dybc zu8i8!RPE=tCv-(+zbakIcfI&imeqG(p8Z?Dp|V1){(N-kvMO?g`CYp%hP`~z`1&cg zXY7`YPke`ovi7H1B}F{?-c2Vo=e5OzORlHW+3v6G%@Kh*fZGk>P4d4_fCXjdvgpKh zpV7qD4i`Ogstj33Rk0QnoyH0``Ptuv0)EYfDSanhLJHQ>OBE8New6vbcjG;<&`M;a zS3mF(IX#PB?jinDRQz;N1L(J?Fhi0P(t2ChGUr5|i&?nM1@J36k>DvZ{Vj|)b4!z}jY2f=?i;2QLPfuVUyvsnM-aRjg7YyA=fPxwD)_un)4 z|HjtRC{gy6DrgIz{>$Sikt7-j{5uT)&$IZ~;1n+8W82Ku2_DG#QT|^q#Fgf!BRJp; z{0ql>;=~Q`oHhE$z_9sSkOC~f`UT1^fd{Iq_|vaPG28wXoNv1NXsc(rcF`C znz6gL*US-xjr8BE%-;NTHlFuQc6^|niKhLtn!i;MQ1Lw^(Rl{*H?%%?d+!|RxNc!I=a)1~t)opWs;dd5~Os|yj1KH5(@dEi|x?)XTmOMK1S z_sl8M5aNB;-^-xX%GAnl&e#15`1IAZ1VmjHP~Fn*uuoE^Q9rDi9N_YQ0C@}&p~HdY zFn8YRw5?rLhX(3jHwFieef3Wj#ilT+WF0>No|E29lVI%ejP)%Zvfu)=m5 zWs%c&ikhRNjKVLEd;>1>!s&`f*&AMp>Y|O)6T4Bv2G4z`8L;%Lm$Z)=+|x^~NN%4h-Ol1)w2JmQmKx9o7#D3JjvVLs@Ca znHz@5?{))0j{pH5X4SLDL_Sotf9e|9ml>q25+v=$BAu&OMQF_pdNiC@)fQ^{W-Spi zA4}f?G|JQ_zGZ$fF*h7ejfmF~9^hlCT4@)TiyqoGmu$RS5uUGoB3Np9$e>Cwo41*C zjoeZcNKbR$oQcDU1z{#KzDX{q^1 zB1w3=gpRUrY53Zt5U6pBks?ztfqC#}s<6}1U1lpa{i!iWF zQb=SuoQRXo=h^x9)`GVapU%+U^_~fCji#i42}fkjZj#%*2ZMk!PWc0iqzTYpbLfTk z9bU0170={T8%2hRrt?v%1@rhCLBIvoH;fNK@>!nPB+4c}*9<%JvOGQ5$Hpg@Ksy!| zle+`o;R1RelbC}PI{d;;cbzRpqn!jvX0t|T=!o4>yK~gz9{n8JXOh&Lh-l<#&%L>7wQxldMsEh_nm_I)Wd+p z#)%(5-DBLt2QjbrN6+{eFnLdfgeY!bl`F)v`>^}jW-JP=FzSr-yx2KfX`8gQI{cPeZ%}}} z^H9$VjW<~_-v=G^*zECIqY-nvFRzjV`=JeI2=DkCI1e;#h#MP#e0(&*{m$-ou@Hjf zYf0Z$?>zB|#f*5Uzb~F*bSFxzGd77p)@3VrqVH2{p>%V;$HE5YztWBzO{PbiDb(q3 ziZwZg=@<5@%;4Tf3%vHj^k^pbzjuyu->hPrp{`%hHEvdUe_|6=jlQ2ynH38kb97aeII zSX}$1&l^29S+}0rL)-W9Ex<-bCa5piqDYVTfO!~~?NH14H7M@J?*R)Uws1UxsC ze_)SLwvdiaN2v(Osd?s7ERGp71VdJ}tNp89+Vi#r)thyd?->#?6rieW(~A$cn-W_A z4x1_0S2sakZNT7yPQWxfxs&0I%a&O*X^T z1-9FqdnUZdGNN5_!_H&($=|2=9HHJEzENbkIR5j?oQ$sra}Ql5;6O=C#mG(u3P2^$0h0F1{04v&N7T-8YMiP?4 z^MkPQx339BDQUcA+eb69HS;h&rY3l{?oewRn;db@kFRK7LPw>a^0Ak9OP5GkATJA5 zsQRZ4{CHsY`l=CHgOJj0?U%+{a1h3NARZa?4@wC`7ZG}530c`dj|>3m!J@B<051@HDPis|1D+Mqte+u$sJD2n-hOD z>lrwr(;#K#`;Z~3`s$f*cN}=VSUZPRx5+U;DaJnfq{4)t5?65Kz0*6Yxeb#{*YP@j zaI67<&&rM4{W}9lHOy&yGX$Pqy)6{YL|4{Y6QDJ7@zh2Mg z&~N@0CAE|k!^HcsDLucOGDff2)JgZ?;2y`6Nx`FLpIX~#Ke1c;soBqTDVJN2mOs}y zhAG+k6a?*iRLQqBLxW{F%F=D_6rD}}u*KYThCW$o)bkLp8iYpMOVY={T0^R$226^0 zU=o<~Z}`-vT0MKh*Aj9q-O&BR-UxM4<-i>D{;qL;X=s6Em|jc*%e5aOJ!Q+&S!lRN z`}UZ%sv=1EQL_vlR#>$$3BS^(NiKqH{#YyBi zb+kB(#4}FamXj&;o2dNX;^0Uh9^NL@IaUkDr`(mY_jby^%q{UtE@#*VZ6Z5FC$vxO zYP~Syo6A92tpCRQ8Riu*!9<)|`7NnLmh<*xR$)<*F6hv>JSe9&{w3cX!;hAMDd%T) z+FvOX*Hvw*ozv|*Al?%mMB+^mGFIqTk-n*5=)I-cEw&+f*dWc<3E#uUheoZ3SI}y7mFwQ15XF1ZS(Zq3In&-UJn4X;^Vzg>Q@HbacwXupeHWtkG0oOuiF&Lh^<}=ZR&rMc;)gRuA8?ba1>FgRl zpMIwgylY+E*OaB`o{cwgXBKZ<{tapHKer|9#Xj{Ca8am&4FnzB?G8sUg%P6-VxktR z3`-n%s63q8)2NPU(O}wqHF?#CfrMHeWf~Np9sVy?PQ<0%TVs_IEk31Jg)zB~dYjhE z?_(Y=dI}w^5X$Mx6Mw2LzSDH#k<67qB8T)^aHs(1DEPmeZhlC$m+a@pOrzmXpz8fdDIKH5D?h~riPI-H!%;Oq{S=heXOMt3K&yu;-_TlPD!FZ983S@r5_MIi?FcV>>9 z?ofp{pPMAJxN*SrF~c~hds0iE*w-KOunOpvox5>xSeNIAGcPh|m2miZD9O8|-+GtjkzCnzB(Pnupm)Tcy1e#% z#?!^Zrc!ML&=xyP6MA~C<@|Kof5}Vg6ppsFabtl2L{{<`F{Y8+M%k3cFTgQl(R5NH zG%Ru0Mm%2{`Y_ol3<|xhbjg8qqfKmDah5HG)5kM?0?VgRx-R&)e;x)bi} zSB9~s&PO?bXsJ2YLOph>x7jwTJ0|mVm1BCV`v>sup47k|`Q_SCvh-gAxMF{vZOrXB z`htsD>Cyc$aE|XDXq?U6NwhJjs?kMs1@!4JN5Y}&tMTo}&ZB5?UR97mOK0;@7+a|rHEt0E&4G-YmKA*LL8g`)j5FYFNS z*GvJiHYc<^;f4D=yPePqi=O0EO1rGP<7OsN{gGlTg718tp-L#bEnHAyu--1saj)J1}0x^bo zmAd~91Ae2Wr6Yq%ppK&ey>yTGrqmg=h=_<|)uvJLkg$#V&tz=($xMcp=W^sF> z9WXQO<682zIh)V>mK4EV$Bh}Wu7gxgmjdciTbl9!$eMpmA z(E8zus@9I;bmSsBKGTRD5Cb27-$Rl>AlO8j_=X2sg_l37#!9MxLV;rOw5F^^r31Td zm+G#nF}WhoK}~3W;#b*XC|y?#oh&w1y;#awQvHTRVv+2Z@ygx>5rb~)cgFkq7k?wbA3|hXhfE>Z?w?_Y zpGc?%HGhcD+fXds@Lmu~wrN&FJ6U=ex}Irxa_qi7+uj>~OF~jqU$Pjce{$t=_+diI zu90b^$n~d_4`mnmTl%Pk80NvDbndTUEf8qq!wF`rc%<;o&^LzuY1l|oTCUmmygX0} zb;!fO1uw>-0P15cq>L>boKh1!po-ab4bCv5J!c#`2EZXNQ>NTvvuOQx7@*|MRdBR%=-XX7 zN=eeUDQIleq&)dp+*(Wd;@ur&UwYxP(d-F0p!gr61{D?}CYBvn#|g?gWmNW8QHw~j z9^~mcwO3eLAr)5CDKHU~nIPgAm~}TDdiXXlb2PBQ2)5*mCc5(3lv2zvh>-{ygHY%P zd1Aej$Fx+>YvPmf%g=?SCVL-j7MxSunBLO?=h1(_D8a^HS=$hs6z2&Ngn)J z!2|ikPnLKw8Z4V48VOfoXFu!&<~Nh^Po385KZ=dRvFn62))EM?meupl<=#&W!;Nmk0AELYD6;+xQes|(Du9B&mA6&)3iw#j=u=8g-w z-Xi~2L$J_pelS<4bb1xl>uItWXC4vQ?67MOmGMC%v~ zUr6kwh;HLgUlb!Zg33icyt?}Wi-oG=$XYh@A&i4hMn)UvH%&Hgxwc=JXsV^#ZJu-= z4D8;QI7k@1wr%v(E#K zf_K;amMZN*%RX)~ZxG|~vA@+NZiCA{^>zs;vfGB;+0z;P62Kz=N$Hp)L^gru1dFfO ziV_F3jnts+P8?f%B}W4&xvB`b@_>4<)m^@AzzlbyVC702j z4b=7WYKbY6pF!bt7f80hWM+%&<2=Y0gP?B6XSA0BiGZ4hg?$E^p~#Go;=!Sle^n+%KFH zNEr!uB+RpwRGepjkCeZTp}OFAxkd8kTa)%^6@7Nd|8aixgHp}=2yjWmBA3BT@r&zr z^1+3;>@x$JZJd|b6iu@MX5J&hDjqp!wJbSWPLV7$!ZWWoAEIclnh90WQ)kqWKFJ2J zcLmdz15c~GkZiQmNP7FZ{AXWW(e8CxJl=2gv#aZ!o|j0`b24NQ#`~7YvrCzaa6i^- znQT5t%aullQN)znv+=brwc}>ht9`r_Ja5 znBF}rhy&5*Z5%4O+ZQZ@(-X=1TeMvv7M?vIhi3~fBzU7^#kr5Ly`^)C?3ChFD{N(7MgStxCCwMor}^0obW3FaA?Ek7h_;|( z&J>k|buVN>d9iZ2I!DH>hR&0~95ivJ2oJypw-%_Sn|wbUS9^MRmiq?)De1G&Fu!>A z*AY4!XBNFJldA3xEf#*`Fe>d_I@SX52d&d>>a*D2ZJK|CVrn8rdF%lNgXy=sOcVyP z(FUCZv5dum?05Zj+*M{dGkeJ-hc{R497^3)RI$dPzc=zhAM@dK8A#6Wy~U(MU84A> z=yH_`nWe9k+ElCWwx$yU>M#qrkQrYGx4DNKz5zEoZ!~K04XChj!HTGQt9osQberdm z(moc;(X~b-EFa=t-;Sw7gTIO)`-qA+n$;#GVEBBVe-QvfBq#i;L4v1M?Z8RQsg#gJE`8_z4kgPOFh#Hrm^T5_Dpl&b+R>E~X zE^9eB{&mAOA^>o&K~uIGmCka(?co@^$u*)F+@(>ex)erCI!GR6foxc=Pwp(jRRjm2 zp3&<``Ag=7AXEUCM=}jBlTDSP6tp1lL8jcQs!9XZ-w7|OF)|{ftuQPh1F8%Li$jL0 zfgswf+%zGK(EwchNJX~^IRfz!b42zlso*9zu@q|u7S#y8!CP+CDyp1thsOjYsIWi? zn2sS)-&UP~`S8$o(UV7x@ZHQYo<_1EVqEdd$f_Ue55JF_=>(p;1j4yJ+%PL3#VAJc z$G|(T9KX3cbAY|xm*?V0<0C zRSH}|lprKT@wYDvm7?Rzma@b4Z3-V9KWc8(5S~5~1*sVkBBzQ8QQHc*{l@U-@D{)= zT?{exrc4XL#3y{AzhzIaT0-f(&l&VQGEEf4EyF)2f8rwv%?gwPHNN~PJl_!Uz#K?g zA#qW6ewlf~xAeJmncTP>ag@S&xM$S&RN)J)3yb~6G{05{s4`3FXFQT+y8QdS^A91d zz0a7Aor}!!EMbZ@xlHo@MxTV_g@RsgM9kJW-1)R_Hre&aq>1AW#wlzOOzK9=)V*1C z*X`RhbKZ$|-^&sIAW%`J;gE`fZ$EhaE)!0h{P@vk>5Itu)s4x#H&(7Mv{_tzaSjpo zS!TS=N}6^_a4N{7ijZ^fye6qgq{4jRaHM<=WLkWgQ#nHQKGPL{zU}J$l+6P8dFw*B zXLBsYinYMoA$g|5S=T_d$Lau6d4kWfM13#gXE3i(67l47 zRr}}u*L^QT_`G8$S-95r4?pQWviD7Kfv`Y+U<@cdspmB{m|- z*1MN~7(2#Gg5=rdpQfhx{Xg&SgWr5hT!NlrTB~vnjs`Q4uNos&y4sz1h005AuKj`P zJevDssYBFc!u+K-7rr%jD@Ia|CQDoVaE-BOQRcEjBWUDMOH4q3wUY`BIaNE;E)ok! zp4aQ=AV%c*|Iz<4?`^p{KhG50lBa49TC!NliZR`WK#T_Ljo}VuzNmgUO_DZR_alm4 zp!lpWm7*~Cbuc=*sz+U6PL4zzRSCq};iN}h0DOZ$+>#xxkC`cbrw)bJ@t7fWx9#>9 zu{7#f^j+9gLuowF-|gS9mc2Yne4d@U@bTVQAyAka3w;ybMXAki zTs-;~gIe?+Qz`~eNPA7NmM9L+H~YHCjnFr@&3|TTXWxWVZIAz^?2lLHu5#265cp<# zlpv_A#VNcDeT)D&%G@lrZM*dt6vl3Rt7;7l3nxDt1Z;5i8(2ioT_rnCuF*moVgSMd zD*@k+WFOv7KHrTP3(a{4PToN4eG{<{kPP+OTH@LR^=*5`3*X8R9dQmbD5k0lwGD~s zH|MgA#0V!0AdlkUhu& z+0l3-D&40$YsO66>U;L_7E@PAuitCe-#5I6JAY+3QVM`aSS?Bd zZKFkGKP{n{>PZ`Bk&Am`Nxsaj$*r+n=MNHwkQlc43CDLk1UfoxT_;moyas1*^EgU; z-e3iL={@6ii&ei(GeYP^$1)OqqejVn3*Q@tg=JV&?tC*q$!G1_Pxy^yPspVDmFqFT z&l4_~6AX|SaJA4U1J_)jW68euyzyr0cSsQy@RgjDNa{U71s8z^U1;T~U*S5XPk<|7 z$qX!wn=yhWf7$$G2>uC3dgz7C7u7pryjaG4~84r>hgftT(yjPn9}^M<3Dcc zgD0#;srN->`dCcC=+0SXzD)vKYJc2h&@SCyrbx>h(PSq<3PDJ-#Vz#z}!@pGl?GA6(qKNiRXA>5MGP)Gg zj3~j4Pvtu9DIS0qT=|ZD;YaG+FfIsJG;gLZcPH1hW7xVxBs5W8DJDmBy?>gfEx{z& zJg#8`N_r~t8QALzc#JN1)Mhaa}hy)9B;m|y8+~IIY}e5bB~NeHiRSi z0GnI`_ONw=;S)UYxk9vakd(F_!LJX28|o2S0EkmjA9*{c3HKw+jq`-&>Afb*RpEAO zboClygeESBb)&{08EXoNatRE7Ni&gkr*1NdBmELG1Yw7Dpb8TOv~*?4?zv#ah;VieFKgR>p;LUwHm z+$jf1Pjx;xQm&{ZJq_wl=76LJ4jB?YMkoXOm2yN`ESGXg_}d!N>Ygr$b@^L@rcaN9 zNae4;QV-vJT1Zs4QqO{?2Y4JQ)_SU3P}*l(_7Q4r)G|1ZVk=N?MT4{!d|_{EZRz!i zN?Equ)8pu6JrrsOHh22-4r;6}ka*f%QD&S4nDp!8NCZ5J`FF09Rwgv%ifuGbahC)d z{p-rF;}28kC6a$7nF@pRMK3MFncJNnKmW2_;EK;V>l|hkBksVVWHo zDw1y?TM+>la9|OGUU=<4Q?-}2_X500LWqV8kP@=}oDEL~Y@xs^c^h|`BOQn(t8Q?7 z1R=tP+>j!Btj8T0qQ%7u#G|-($U{!B;qg^^AWyW$5%Rm5QaR zHZl?ylz8=hO;9otkGiGjsX&ix%eM1Fr^~5+@9Ft zNQ#I6nQlSE2TuzpkKu%TE_mSt*Bs%Gh4p$9Cd)*U)+!dKcz?(>eJs-_TU*;nxK1-) zb@!#pV4mZEOGwb3C6ikBg%aS2_h4_Yn14Avpf@g^8s(JGvrW9Yr~{ugB2Ds-fxo)} zb38JMUW0M4+-+?l`LKiT#id56TEE?!?j`yUm;ujDTC|!ip;K_kG zc$7CgzsC>2;@hL9dMn+)*E3=!2XWoGm{UsoM zokaJjubR=K!dEfa!Cd^?8Ot<;Iwx9=93C)J_6jo>;d%LX6Vx|+%S%1No-pJ6Ydoe) zr&ILJD@gEIBNEK@cJq0=H(2;lF*;@2QAeEdtPvW?+eh+DpNVZ-Rov(`1FM|>EouG(AoAcG(HEFPvsUp-zU{N04KR<+ZslYQ> zL=M4uSrj%4zs8fhB6A3P6|sb=Qt=;ueu&Ot>Q(x44uVnJWcK=AI-c~4*;WwvP-sSy zQGZx_R-!`NDjg#Zm%Y~65IjpzauM`F|oZi0Fpm{|~EWOu&1lyw?l{5B=E*LR6 zy*=r=Q;cWS^T6F3D<+pT4*@EjZjEHQr~EE}JJgGfRiMtz%JIO#j1o@eDJhd8ucY<< z%ld7Ug-bworWL2fM8+A0S~Z(|&0@3XUq0^notOJaFiTwn{@~Z{lDFE!a`NsP4w?B|>;WVe4Jj3}mC^JF1O5ji)V47Fv@$`t_uPGm3Akkt}^Ok!K!F zlUSVs8f30ojLFm8oZxKq^!4aI#nlG|vM*PE#MlS%K(zVqKNobcmHY9Ewp1fh0HUz) zBnnp+=kmueTkqhRRG3?U5hEC|@_Nv*{9F%yCq8fkB|m&OA_oJDzj8yXes2Xxvv#V` z4KoOCU4Jv80XMRyaqjexe}6BTnwtw)O(Sts`SE4j)t$(wA8+_@;YQR7tCBt5dfd7> zKwkzHWlRwggZ@hpJ122rCQi*Gw!(NW3e%PwM`iK!^ovtjstzr23V!F!eBzT^REPTa z7OC8g?+)}S=zYr7kqK0Q_zbA1zh5oVhx!bK-)Em-S<9k_(}$_g7g+u;KiShMDHQmZ zpNx?c=QQ$W>44MvDM8O}T*x(AxJAJEkd?hqHykc4g-@#oy}NU|^*Xqnmfd3j6-gYC z4|3HKAU-rp%7d?&|#cCs}l3Yao$&`|$fiiN@N2rq5U)*bnPEsU+(M6qI+ zptB*s!5ik}VY@FZTSih}Ket75TXudck-OpoW_^e=desXCKue5X_GSPez=ok{Jde+Li)dcl_0tfP7pvmoLwGk2F^2thj(T+Qz?V36zn*w= z{aEEL^QyfZ%GEsT8g93KloF?Z!>K!C6jQN|H@W-5aE26}S`Zf~t@jAckbeA~$yMry zv7IT~@Fn;S_yO>-aQ$R$lXDiSjQjdnO=L~{EWUzU}q2W;Jtx+1E+#eTu6=g zk-^?#-FN7^ET5fp!E|`+3={Vn>-B^toBKJ!=0ln%_RjKyxC$6dD=oD?Z!{kLq!(v-%yEi;yC<}5q5EKjFe?e05#DvFy0HA2oRJ9TRgL2*QjneP3L0pVuCzt}RmMbt@Ex2-ePiOs zs0$E+p4mQY`KGt@Y=%iATXrE_Nu z((j8n=9Pl}>-4ov+WS@OHYMPvH5t*itidQlK~PKo32rZ21RCJTKG(IVj|SH3QP_e@ z6K~fYABGzHt$u%KJSc}6477kMxohr)2DVD@g#%`bK!nram3!T zK4pz}YI7lc=S+`t^(ojdaN+oqwM=^iU!*Dg9D0}%MPynn^b-*OpLMhyW zI>L6y6e5#F&JGx1=;CbH>X)~5N6+o&wpaKoBlJU#?AG%05#H5jZuf|o?db#St};pr z3huen52{-RIRd1-eZ`c6vJ-MP@5c8qc`*;`O*fbi#x#RC8l0~LaOq-K#^{^%)#5xi zfAczp{2JAmg9CKc3x<+DIj&|*p5){9f0i3r~~y$`&X#CdAr7c=`d z)?4AJR!1E;VxDBXhs^yfc5+^tG1hC5`W06ZhbPDHkoFq5LU>$R$>%!T8A&$jqs5CH zCF)kypTS2dRLDY_`!AJmrU98~FNLbQmDsJ#)c3~!0y<7})JnSGTFt=VMYnV4tJTE?LPhn}oj62snXlQkIgq%M?hRVu~9SvED`PJPH zVmE)2JRb?Fdu()s(chFfY8WHq357PXi5x+0pVCs8BM3$Vl7{qS6wB~s5gs$l| zJ-^|jPHR$U*98vtpNMlvdp#&JNlk3wY9*jImb2a{;u8zMqH$tbg|Cj~5APY<#e8_$ zaAU3E7O1^ar|CUkc@zu{$TNWiN`(eF$7cPKI|;lJMcXd$qTXtkbXf(W1KpJih4hl| zCaQo|koNFZgSEv)otTPMDtNYijPNi#rsj`>4JK_)a{90=?6-Xuk>3{shBG+dT97j( z%M*eliwZL99~F-gx1o#hVMc}9av-1>u3u{WnRmI&5$^sN?PscV8Q4pn*1QJo`evpA zBph4dO-mWjMJxn@fpw**Z8>SBVcS$Ij`zdl0idUj*B$;y{!XtI&ba2rKcRM|xg8&4 zGVAR=WGFYW;n4hHjH?Q8jrWd{hsz;RoKUNm-KZ_3yQfF#>Z(`a?FxbJWfxnkK}DGV zNa7uOT*XRUIv?hgEyed2?T&ecqdz2pT}qPCB66djHi|wS6svFw@vsmvUy%wp%m#*@ zAKtk1&?a9T7$y&A@yZXrCb29v=*2ew`bxqTak&0NlFG6| zv|U34Cqi9TJdiOaH}u{HN(%0{zO2)tM8&@mi{gxXG1PhI+fx~tgART)%4k7AL9$?a z<$b_fubxcJqTR{zcFlJ?^=k=;62hsA8dE<4VU?r#@L)RAOX zRLK;a7gPpt^|4b`TRHTQlwPpf^dw(@JL#~r^xVEbMi@j3zN703AznBoDqCra{8eW$ zpxHJt0 zPvJ8GKm$p*HIo9|*MfNe`&j@6vYPk9z$TwFX4O^H4d+l9kM_OoIbc_Cnt-opQvI*q zUayLQB`cpX{xIP-=8moqN#0!7zQ5ve7RE_M>jwv-YcrckfG!jI(L73n(mpet0x>E` zI&*w%RbI&W;{`h1`5Y)O0vaF&p^}td*s4%Ej;?jOTvy#}%7VM_d=y0`Rdd&C4-nh( zmJB`*1k9bKZ_N75lDGr05>v0tpUeAw!=ik3w?Z7!)Hga*GzCkhA$3k%3~Ik1gO3SjLx(AG?wNkKlbs%Ri!f5?}tGoL2BqeD#% zy09dhGsK|9VT^3^`IgO`ss>?#mdsZxIRmv>JGKvh%knwwVc=FMOw6Suog65a*qtOM zV5+o9)EVD{{ol=M!N;+*L)YD2{gBBlL`jkCGv=g9%-u9_iK0QA;IQXqP6(3(} z1c3Kbbo43BFRw<>6$vqyQ!I%9Tl_YMFNWvr=b)SKkqKB3IJqrri^vMmnhy}-!#*L% z*>%?#SlsmIpCoYp6Se%JCA5_b5s{X$?^Gv^WX0{TwiZ1vHRaIOX-=M&Z=2RE*Nv%X zG>Y%zQyb?SC6v-aD|A&Baq|1)|5~`?0f>+YsowVnN7oNnf+0wvPx21OxZ2%UoA>14B3KH-nC6(wKjt#C{Z^va8F5^Td zqwa8ZjMeK@VF+9P8=0aqBGP5^oAepL_Ny~ZfD#K_kN>!YQOegjXlx3_<_f!b1IdMo zMa%)@x#1+o@{cW-9>#qlUE#In%zm+-vNxgWjgXXkv$|$`Un$N~xxRj-prcDL52l|-_dVclr+j=*S*}~gO;fH*eAAvSqvy&# zoM$weQiL|?8`V!W=0)>ovh=&44%=`!O56IppD}JS_(c42;Q@GvXR?@alDl&LI+i3$8Q_Z6f5Y14<=>N>>T` zD-(q0bwY(9G*FYwXJ9ue-A#~8vP4tIV7|S|`s~1XEaeL@P;(h}oemGUaj9);jKX1y z3c85Cz-aaJ?r5uJBt9KkDNtTfdC>SXM!l-9L6J!kye>yM8s!CEl58p@{2nsH^Uek0 zkr`ucHv2oXk4)!B5f@Gm_pc!ZR~}cShUWFggRpBji*j|%;Z2YqE0-a{39d3g2VGi zQH@Sn$+!?cslL3oe~otb;s!^)EC}9sK`Q`5E#l+rFY&Me1E|L#@??L;_T=Kzac0Wn zy}r)&JR-Zb;nm&}t6utL759W29aQ)fzDBmD`Jx}$K3}fO^#=L3?y6aN=6QA$(HS&_ z5*?i1r~7jP07>M^)imGQ2I@Tp8e#PwN!Bg(9_epzB{lE?`#ps1W==Z|0FEtbJVbrO zN|J0EJaj)@mg`u-=a+80Cj~8XT8Ae{nr%^C$GWo#p0mZaERlszQh@G9;)s6085bm8 zyyeStFt8~*rW|&4+Bksni#8%xt)s=74-8C%RmU4;36ZHHRboHoG>G5(y}GhIp;abV z?UgD?yK6hcla$yUH_IHKSoA#(p0BZ%G7|I3t(P51# z;o%{h^E$zNgUO3Bt$P34b4V)PU+X@Il3mefg5#RB*@*c^PimvA`!g8*1)jyuB7f_? zmB>mOH$cPSe)A$KG=VQ>E0qeq9irn<(E9uox9@gpld5Htq@^m2Y6JqIA~Fh+91C3X zo$&o-o{8$u-XdpW*jk#LNMOn_QGS@~ZSweSarP<7h!(!6QO#h-k9rjk|@* zEew;&pR!fIlrDaHcz#sA9xL=FT!aMc##+izXY-Xj=prt6Tm@?vRXHGs4cJ2T^+wV4jmOWZ^CR;Ia(b}`TP1Q&Abyl zCXo%oXI8Yf2B=;#$S*I`HIJ{GU$B>bt$p>L@zvG&hi?A0ZF(Ku{+vV|vCO}wHDwUp z8))|KivrhPgEqNH}E8+)|l0kdIl6d*(KVUX8t!kbmi;B7r(w75f;w%;1Sb znb$kEbR^I|SK3S#S7CSh_@<7wt~{lZr(d)K&xm1|(RW}!4=fGUV>WfE?wYbpc_f)N zct%_=HJ&gMuP!c%2RiA0x2{*%N8akBXPJyk1c`L5h))*I?QVkzxt~zz9F1*{=x6(4 z6KDAxU8;B#_SqX)&DiBv@`88ZmfOV1#q{J_xT{4+%UrT3weNBpC^DJDV+zJWSbI8s zz&r5X3SPjJ-UwekdS(tOo=ug~@uem=k6WVXw@Z3I$54i_O_iwS?{nIYhaTG}bDF(# z-oJ0^c)(-Skx@4FD)Q=g-bi8eyd?c=HZ)FouFa%cObVMKfR~2ZspdA>GW4E9IIaV?T;y%hc#pS85@GU;V zG%1Q_t`Hf|wNhmwQO7-7nAT=|!A~Y^!6$V3xDuoL5`0q1tCr;a%sGyQZitn7N*d#d z(52TwGNf)K?J0(}-Ei$Q3ax9o&On#?a0**43dl?`osad_{d2TC0dJj^XxHq#E{D(I{nD5oinjqKgvv>IP+KAY_Bi&CgMOC9iJ3^Mnk12je1FY?#8IN z!OIV!H9w+pP|4QTMM$ZW|9m0?SsAih>Xe#=vo(No6N%jF*|Wt-W^jYPvbC3lxv=V` z7PMrTTC_EW=hQ)c0rrK1udk!4*L)|_$3qyNOwrXAT@L-B`xhRba%r5 zL)gnTniLk0_`6$%d zMTb1k{)qC?$>rr>i}enaV7+zVSkCE@?pa_+fZgmRHKm|E5oG#3fY4zF?Q%uj&^CzLk2D`jhz%6oRj#9XWOvNoDxrk`u=&e05TPSP8tQDK*U z5Vyq{Xqr|YQ&^|Yn0#guK!6j^wYxGoITAnv&2Z{&C|DfZC^tNw%`lnY@hDI7X0upL zl3mVfzsDA@-!NI>dZSH`W@Nx~YGJRi$en40Tsd;>2M>V}cG{@)cURRZ&kse(*`r?s zSTv0HTOYy=x`{cze*NmctoLHMJ1u<4o5#lKoC66}-y~O5Q)~KySXJ9aX-33-m3Xyu zvEk!$5Ti^^GdQWCDznG?2YqpQM5qonp~gx{UsJD#>yK7_!i zZ=h$A_m5>axzDF>S?v-TqKn_dS`5kVHqWNulmC{9V~y92?-hB`jHs|(bsRz_&i?j& z0ZTQny`G~d<2aBT9wViokg3_7%AFPH{;{&FSY_M&^aGL+`7({--ebW?Z`7932y@hk zuv<5AhxKY_oJO72v!>-_{tqDma?gVAsStA}TO6nd+aRlL$TSN_V(X!f%49tWV=x+F z7s|`#Jt~I^DXpd6gb@Gp69U_ztNY=^oRL~a4vYIBkQRO_m}V`DY0UibBDa{~rA-zg z=o=$AR^AR!fAP7o zm+?4zI%jWPaOqi;1_RW}tE*HT zr+C_6#p>ducb)Vnp-#jT{-iQh3kIibkJ99kk1UnnL^T+jDvV0O;rL5~GAZz#7tziL zsf@_juQRIlPc!p2Ny~P=_vrTSjTrF>;ynOip)omU^K07C|5kArt7OLIUppJpOZwxy zZlHbLF(+N9^N2#5r=e##)RpT6Sx7ko_DUg<0lzcqak_oq$^#MiK+NS*#iB? zy~r4ZQ~Nw$>(QL7T2cOhj^osnkX`T;(t9%{lm(y$hoIOvRQb|1}-0PBt`hPtA z1d5^{mdz9h2}}-iku(>38_PrPS>>y5!lj%L;A#7tHFd|kATsAca~d{ss`E9VZN~SU!{y0M}j%2>Edg&*EO?R2z}LU)8(! zlnmemfFaMiS?{X<^6IW{c2#1AHI@0% z`npE+TN(Hc(L=r%)AWRQ9jf**-Q;PU(aW#!R(9Sm@Sje0 z-)gtPU-cs3wLrs~G|H3$yY0;S;iopzbm=R9R_3%=jTpdv?@S;}MB;=A3VZZ9p*NQ# z=k!54u=H8iaNko)RLoY|ZGJ>O-zcjJ*Zo;J|44Um=SNq$08)m81$UWnMw_xifdYI@6YqlX-aTxpM9oe>k*CDF@U1yT?SI8>E}16<${K zS0nv>X>u$+u5tpnCW9=fv#egK*Q~MfX}bf7%W_;cDSVG}z}94-$7lClwCQLmKIF4I z31{2ubMXcmt`bs};tyvmrIAJYkW_y$RW^$sUQ7VN&iL&blzFz!`Q!6oZQ3(|>A^1~ zz(SV?Ty&CS1;3Z&ZE$DtX_HF3s?at=&*O^g2?-hBIMQ^DEbNif9-bXi9C=Sb*V1UF z3szH-SI7Hy)48o4^u&+JcukgiYhIfR|5kU#5&Zg;#8FReWBh?iF!wj+CG$0q&vW3W zSx&EZnXUm)T5tWD&&hKV!ZxUrwy34}l7j5*eS0lO=Ea#(!n5>k&=a>@XSp$JFn#Dt z_hNy3mxtx%JWy&N>FjS|-3IMAa*muiKsu zFfr?ieGMyQ@t1=~l91Hj`Kzf|THM!Gol~)2eIVe~-ri=%&CfMcfBU=hW~^aD-k!TO zw6e=iJ135}BHQ|__HQII9H!rtZXe?_ZSY;&qsybTI(+H8S$A>bUGT%r>BIYsb=krU zF2ySmHZv%Kxa|*|Mw-DLadk#2YIK=-9B*Gb6}pZscCTS3bPa+MX1_xc7I=Zas}_j= zw3YaV+bSVK$TL+_|DvSV#?h|Jlv$^|GACz|jMn zD!!*r-y80ZhqIzJGY4VUOeozn3Tw52&W zeh}5<=2?636Y*JTs+3nqsdOiyX{3F2tGao7v=_Y19a)xiSTwqScWR1jtlDxm)-fg7 zZXz~)qJAQBs!$a-iwxB*nmM zAYNS~pFD~zRY&fSmuE0Y_UDAt@Hn+kfnx8+tPY;d{SgorgnYoN$Z9y@+P4V*o)u9B3piOy~*1i*Ak`C0OwyoteK)4Eg? zo%KYRSx&!BvDv-)Ag%nvcUeeDTqB!Z_lq)*llv-<9z9-;vr=-2{$bQlZhicE3H~}6 zrQzX{Zaa;Nw=d7VuasrAS$4~CX6cfajz^S(H%b$@8mey>vGtlOQP{s=Vn1s z7)%l;24Yqa8B6%Yn;k5*a34h@^{)1@-f2C#4A?t2~Y={DV?8qrK=9Gn;85 zut9^Ky0T28M|GZGvICgUPN7tQvT7|;|ATFOFYj8ka_7+N6!=i8O}B7dj|d;}EB8bO z?U;D2{*>ov8ld%0*`md@Zrii5DEaD|4%0;oiUsa1#i$8x{dcMR9di(+rh)W+#6izR z?rBdB^M;KLWHR55S$mPIK=Ukb-b}lOgD)k(c)m>jH@B6l13TJ|BO*gY9W4g^ia*KJ ze?3p3*DIRzCaFZ=hm@3T+6 z8ad?Kvnu|U%XJ3+mnJX|VXvV^8%d$sP1?z(-CJZF+8^XJi%RXe%2s_ZsV^%Jphdbl z$unXndwBt~hDRecM$W|RlDRXANRQm77R4jU$5b|hE4)1~?gekKG(jS_krF`Xgt0u~ z@_T<7)FTI)b)`F8MjmlxANEQiDeKTC zUQo}E_qbCTH}T=z<2+3fD8b-Nb{MG$BAwqaF#Bc{YiGOsm53=;(>d<4 zux3X(O|*tT)}p7%I6X`5jysEXl8@RGfVhYG)R?zd7xS*Vj{+FyJJoU}kTBzue|OBY z8b`ATq2eX*TzM$W9X%i_E|9yMgL^BG7r{^3|)KemL{op^UQh$x%###bSj*LY+Q(%xf_ z;VwRy+kz7sc=|a5H#LP9{V~mVU~H*P^bPApnz=mMn~fyau*D^u^(>BG=jCJRts}%? z&ktiUvAhr*p!uiCloN0zlG+o;VR0m%`jbSrVo!~qX2PCUwR`uoE2PStuoAhq+CdU_ zX9P%n3#54iV!9X6HX^1;taR~b#hK~Xs+*8*bPDLFDRhNSqepF!DE zA$J#WXnxZxs(tQZ)V$_!?eXJF`W}lWetX@K^SMecQ*j|^!sy|fw^EK8oOM}Mxek#u zvX$J|yWcGBl2`tkc|a{u+*~vsmBNQ2Evus4?j8Y$A|tQ!PhS;J!75{D4=TMBFGx*- zA{ir$hH%@F$ICNzz1kg6q*q>p*sp1NcWk2P#zB$sdEcm!jVL5F$gRRw3_nOR%Dum5zCBqz{wzw&#BVEyc6`*GzE11%<#aI< z?BQkeu|Jxq*qAu?%?|&D1RXnw(FE_K0Fn4DENmiiWBH(%#KW|RZu09$dnRzYjR$0w z{JQr%+V*&Id^^lMpCbovmCbUr$GMxFq`YqR_s_e2&kN_6l~Rd%hS;CsIOX9XAql9X zxUb_>8=@n#%D(nXKHW2CBR(aBTg_mjijL8j&NN%l-Kq~3Jr&DrJPvJyA-1OB6WQdu zx+XF6jGfm0}tcb%Hnzgl@sust>juC$Go zpJy)yPYouShbPSp9gGhWxD84RLlqfMo~JkD4Y#z{Gp+CYr$w#UzQju#8u;o)MzBp6 zD1L1B&T;RK}xbD@wG-ZU5^<051!yCF#L%xP4Guh`oCxbgM@?1xS*Qt7$ z%L4kYW%oE#o&k@EP70L0r;K#jAUxhZa_K2M+DJWcX9TZUY1uumfy>+H@6zZsJM^1* zcd6_=doKa#B21c&VR0d>GbQqA;4RE=h;7~YM4@=#hrUx;0UH&rso3ecp2=h=q*McU z!!MHj9CWYVkHXEjB`5@(ul-6I$ZK?0?&L~xTzjpPZHC!7!3QonCqlf1SIeY0W>Rkt z8PkDQ9+m#Iph)iFf%N0RHXT!~xS}Tuu>m{yt)JP1NGmb#-BqZNT8&BDuSI?8`K18Y zNuJqfm`<>K&E2#p%(hr#4wJbL$PhT+cdl04grn`h#272O6;y&3tMR2pE6!;fe08bG zdt75xO0U^if*6}-+HO0dhkR%m2gmlq3xvHv`M#G7o2UTaR-v`%oDy}bH2dwdpy^Up z#gUU?wlckd+Uvpbg>fUDiGku8F~8qkl0$U`9aQ){@89XtX}yFQ=74yInvOT|#_qSs z)`9pic3km1)4Z2csm`s;0nxD0lQyw!xceKgt5(^vyt`3zn!m}c&vApG6~$%Z-PO2_ zPbrPD{WS`@`Cnxqj7{7E(UYv{Ech$X`)Sff`}iW$)3*0IV>_C&ZCp%~EwigP1fWuI zWX_c}UTl24UMY0iar(Mda(n@9r1u)V>{J5ToB%g8T3Olciq1NTUZT*Y`-@$l^3+ws z`FWhNW2#zZIx_@y$K0tPus^Td|$G3wbI~4CC8MTTd)stDr1MJhbsdX?k$9X!q_J-teqsyFiP2Ta<&^C#gyWh`m8PBqSWJmIc^ zm!d&^#mQo;2z#0D>+Q|iE9`@oLgn^gnYt@w|BIY>7FAtf5>Z1xbfiMgba_P#Yx(@? zp@-Wbn*v6SKXBdQeEzEHpD*9-Cz_3NO`LcdP1X-=da{&&J2+CSFctP+~*FE-eDD+?zh85QpqHla3) zT63esR=7C^m>#ENL=VO8lr)lQcJMZ>>#8UlJ`X+nny-nG3KljWh4IJU6&hd}GDfZ9GuV^6 zNe#sgXbKe0oKOCi_X@`};z6g!R??-zl*{MM6UzqC4H@8b+3YF{;0ast%sY3^`IVAi z?#4YwefkT)&VaY934Hq{qOc(^&SQY6vD4UFQg=T|2^8CQ~Z~O8=*f2V}NsUfv7nY znKxORB#)(LTwgbPEb5QyMw+Xwe}7z$@A`hyx3k}eaO>!g>br!dlhm-8ZHt@&1@nV@#!%(~9B!2W5m7MZ2; zIvEJ}t(37vWrLXw;G2=MK8b}UJ{-FQBz99W*~9@%E^sc#^UQO**^tCvYL7soS|u-pB)Eb}1gtWe7nf&}>sOX*3=LpIpV=|SbcL-|Q+GFB zXJ8PrBG6xRVdd@95XGKn^B^sz`Op%K5^41BxraJ7LA}*~W=uTmT}n*Ry9Qc$Pdsob z1MS(RBs+6RAAcxr%oEi9S~RD$3;j<&bkU4hU5|^s;M=7~S^KECvHJv-qkWcRFg@RO zB{}#L)3fpV`&8po)rgdpyMyNuoS-SqK_Lwqey(!tr#}*GlNe?Lx}jclylS%k*uIRq z_M3CfoygMTGrad&ZuG`^M(*}VUWfJY{%}{9fKvUVa6Z5al0nq+l=;H(P48AC7ky{X zTkeb6MJDM(AwE1l-@HeuIoygN574sIQQG!vx=85HVchDHQ6JP%)2=Sz+LZk52HjxN zy^`{Sv&RwhI7rJ$;&<<})mZFcFSpZ?bXlzw%Yv~aF^z=o!{ehqy z%%SW`r0Bb+Th>vy|4l##l7j(_ih1d@UxF($jDws$m$A34X8DXGd)JOx;y$OOL`5Lr zW$VAG&@mZULLHrR+Ly1v=;e;Knnk(_l|R9G!;LY9zc_udQDOTR%>tskAW>I{ z`(G&KKD|mHh-t{#6EVX-35NvN=Ld0IrS>GB_Y(Xle&(^hH!Ny}fM5D!HJ)xEou>E} zdKyFj;Z!s*aqpChl!vCZ_o z-CoUxst>@2-zv67MgWby^vuO&E>>Ggf7qOf8?m_~j}FmI94`-j6i&8IOA zUGZNoKN(IN;5)&CLOIE#J)dv>QA#vhti?uqUx|(R0iwq_)fSb)k6fUT} zNdp#ALeG|!zyerp_er<_@qjant8KU#ANdy=Ze^-+3~m|*$SgUZBUI)g9QEc@vo(V| z4aU!DW(E4q3Z61N!n`{c_7&UQ1&U?x=4-%NC_vPuibf-A@ycuXO*KB3r&4v++(XiJ zM*C8tv}JDxFG;LwuVottFnM1fumoz7g5OU?U2NbFHF*5Ey{YrMxs4kGy=c&OX++2w z{5X^No*Z0<@O&R~m&~rB!y3KVsSh)A{Y0!R9|rlTd=ju}jdk<>&#{O(-^i*FVt(;0 zkSwk{pOJ_r>UM3tljf|jomxbLP-s60EQNARvSB@B4|ro7r}hw|OJk5yD5SLB+|IrC zSlzO=j6w&s1_0zooy^Ky8?8QgK$aVvl@aV;aTI5fQCQS;kVX`eI`-#0Z+=@i;Mm8n z*V4PI#8zDp^IUNvkOB+&$ET2+w2v;jL>%!q#r@NadFj<=&qi=V%ryJwjq3YvF#O~d3?%ywMBcmWuPnA zG_S^6(kQ>rsTAs#CVk${ll!++bIdKB!w}Cq^2RyI09mLa)Z1@dhR7=j#u$+?7P7Y3 zN6OnII-S2#FIi4}U0iPrrq}5oQL1-Tgdy`qF=F2W_4jvm+IKZ)b1BV2^Hb+MK8ji{ zNZtxsBWSE@rLG)X`b)jhT`r_NHK#kXRkRVDO&c+VSn-!wg~`}fnItTGDGtq!9AP|D zJo6FYsibX&dH*oYi{b_CcfYPPSRP5G2+}rD2YM!7cdrLxAd?hNqL^aHH4?f!oDK&L zAp{CZ*gk_qNC11#K+QM)PoDNWloT*sR^z;s>Yh}-v}ZKm&6FFqeYlG-emr)C{kUek z3pQ3Qp|@4cYBz21W?a$W#t?ALC(j>+1vKa{hYtyDBBdS)MZNt@uJy9oBUm$Y&e&S`jr4pH+anythCP zN!P4X&|=iiJ7cGdC~Naxws^AF+1Z_!R@e>1p1%z^B8v{sRIub~ebB{e={ zl%v7}QW`pn{ol%kIcUHq`Q%TW*Q#Fgu2<7OBBa*Q2`1_C#468b1hti&k^Xezq$=bV zXo#pFm0Z}!Xvd+L3w5QXJJRY0%PvQLCJy?I@6Ks+{{4>AMJ^Y@^emue9ENtb;O!=_ zl+3X4==JSto)SUXUsr5fHrVyM-K0){DCq#I0^cVT_yZ5XYi6?>gBed8&WX!LYf5(& z66eKIHwQ`#J%sA;c+L3Kc6g$*=Ad3!>?$emTl%iW7cP_TJ<`&dfB%&<|o`OJN_Fo54Z1_JOe(4 zOIir5#eO=y?@o2&#{8f(j)?pkY1;nq7+o; z%I*OP#^VB)1tNsBw3E5S*zy|!ByQ@ncyX)VDZzNNYn@dK3%Tc0?w5-*_oSUKw0$fN zKau#(Hn%q}cwL=AAkeOBH0OWNh1+rq<&1YN_Eh4=Og`hY+j-=#w;ho&`6@0$I%7$u zB7NOEfNYq`Y%fWPomM!I@1Z!|C}@k&(eRmp5jAu~K)t$hk;B;!??h~8#Tbr*zxM$6M zY=`Yx$5AB0Bv0$+-PsAEox5MX82hK5d~qN7#d#pPDxN7@Uq4amO?baUb*R~q6;(Q( zZ7|rFD{SQQhZE2Rku=cVQM`+Q8u18HDr^|d-U1=W=4dz($GILir+P~2 z>DDaFX^EU&->hFX;;S3*V?3<{2^11t5SP@D$>PP`ZDf!l3f9-*LDJJX2cyh_(URre3qeb}|k3nozAzb}iZk zE4#%RH=lsEp9YHS;0S>!t=8oKeBf;wtT@(My>Alntrn0-GAOOoEh@CbG~mUm>Bi48 zNb1?&*vC7YLtBa`Ih^HYOjwSaogw_nMJ`4I{^3;^Hvxi+2Y{#MU2%Fh=Lpj<9*fEI zd(tb`J9V#m;>2=nY`Qe%#j2I+6K=)c^UUkEi?`y%n2!nI0A(35W9O>Fmw~bjq08g} zXK6VNKwnZ=(~`R)5#!e5vVV09PG4Qm>@Y_y2ykDd+dHNCd~6gm+Y$WRU^yMvR@NPXMu9%H9|LF~(%+ouEw=syM2;HG_X)g!yiz2|Z-`oAJAz3i*yj+Qg|O{c{zNSh z9FpAygmDXGLMq^F1M!*bjG)|bl@YPMPsSM+}(;N*RI?_2{20$zfGYUHm9EjCbMv*@epm% zNrHpNcO5VuWGgZ-hfQdEjMC4XzKUN66qE4 zmQZXn$^pb8eSxWmsgWLv#X`^~&k1g)^#KR$s^EWy@_z+G`kafcMT*}DHvBn;@$3pz z*3@wsZHj2~oxAQ@oN1D(6<&JZZ+W=yE_T3FqK72wseGt{Nao!L{{yy>_aKr|xt2AX{ zDgfYAOK&oB9Cn84Da0dS@jF{PJ7U~~^!DT5&4ju83F@QzqLNsxxGDVs6+ESQ1YYSQ z`{~K}eYdd4gO=6xL4DLM&}00n8eW>hGw`)J;>1IU@WQOoZd*nc&JZ$L{(D-^?u4-! zGGI${xdb=o_~*C3H~_z$LWiu~c(DYh@ea=1>39wL*U_as9Ec&(Vv_zgLBi780p0D! zU6;uIX9O6d4zz%9CZYW!uGA$acgNC7xH@iv%G=D!>iw$3-KzEM6#R%kwTolvrzTT- zqF$;#sUz0XA)AR8;XUj3QH(8&cK@1^M2=W_qy(rddZ8Nwi6;2MQOz~s0QUG3($Qjc z{V@SW<)@TpJ+a2x8_Ja4qla3LK72?693h$;OL)a7N5fqqK_#G}Xxxv}>i!535Gi8= zH|qxk(E<~!L#~nQiNv)LalH2(-#}jVrr9`oKvgT21Nx>S{mUS#@hLgp|D@o`3P>S; zis!~ORNjG#!O0K)glv!WhjiR;=9i~!g+z^Ba5kF_j`n9B&(9h~&%U#!1|l&o&{47* z>58)#lv59d=%!ock(pr$#RIl{5d(Y6|G`4bZTfF9!%q)maGC#L_JV8Eq3?7nxze($#AE800WM{Sr5 zYMA1-D{EoIobgO;Cy7ez;lDK^fIRS_r};-~tTfsNwUi z8$JDEQgMu%Eh;2pjdicHlQ>TGX(KYSNRkc?k{qGvM2M8*S-ALuG- zrD})GrgPU_qn4SyqwTZEkP{#=(gRUDC>o8;%^R%?ym~Y8dAbQ8bF3|<<9c((!kypx zWNgM;SIv|84gJ+>8(p7p4}SIvxF7;ZSe#A=<2K{pCU4vpU%yp9DW5RhNTnjz&(C)n zVX_X}7UqU5=UVpd2Q1aYDB^2B-v&PSJE2t=TCXX@WSD1OwT!q)@9b2bVq7{pHgFm- zJohnhE#3FOV{`?|Cti zZRyE7ZT6J#1t{#jO^k=KAm)adCg!;IXp$^%Mo`~qkHVQnZ+7GfW5ypG*-|( zTTZ@XT|&I{I!$HqisOJf`!=8S7BRIXsyA5Zi^?(m5 ztm{AGKC;P(WjNQ6vUB4+?^@s76#jswq{(0KpM4P=aG;!C(1Ak@EK1Qwzrjm8(_e@hywSc9sq3xP{N-# z$NKnYF3>O4?dXjQt)I%pVA0N)TN{!3)8IsdN&fndgUf~*m|FvJ*&(JDBrOi44mYqP zUTWT+6)Imjpa>re@_!a6?h3+Gj~=Gw*Od--VFUU8Gh;JXp+Bbbx$0;>LQ4TCxn^U+ z+JX2a1oNvsmHg*60)Vmp)K$Bwio~%1p1@^xTR`i>dh*KtRp$c;Ka4=0-(Uh_-z(VN zzKEn53H@ZzMRXU4%!-+nrTav@S6+<2g)ew#8XIvW+JN)}FDP<6PoBL?x4^g;GIe?a ziSik{=@l@%j}nhgA)aeSQSWd+3Zj<&d?SY-46`Br=!PN_&s4#Yqh1yE8=*TL|Mg>< z=QstelQATU@3H5%+0kQ*3X+(p$Sc>eBf-~HHdHh6x~)4@)7ky+0`u*=?%X`>|9U3k z;U5iC63Ex1h&+$k!-29QNPiZIgh zHFH;An^jB_+pYeWat4ZX;&`w|q-B}+HbDCPH-ecjX;&*8CV>>)8n+5VKd75t>w6&_ zc&Il=sJt7WF;CQ$Nk2PZdwX`#Zxw4tdZJI_GF9;|P{wNFAaQ3u7@YFLHI}x7EFGDwyb?Rx}(j z4>Gu{4RJaf~ zoo=Z13a@&b^Gzo}x2;W?x32VlGj!g0b6@D6ZF~Z#x7!7qRH{8nKIPfyrN&f?^e5jh3<${5qih(b9^nB)p-eikdnuOv3(6tJ+!wu(uq2s zOQT^iou66b=fis#m^1cDX_(fTJ^N&VMBB6<;LoD+v^5(DmpL+@+6ZWA`R7J}N9yxrj)M zI@aP%FQjq@ap``@X12S)hStbsWDmehW3^mP-XE9&YpMbnplW@<&4==7Yz}4N>gnNV zQ5l7jx{Ho%oNm=FZGbV`n5r81!nASHARD;)JI(Uup}$z>K60~X|Hnx}K`VDMn$2d7 z^+v0BRyC+sVZRMFyK6=fzWW-vdVQmhn#L!OUn}{`k7GhJzI%I8roex zB0JqGIt>XkbkXtRXkyf%zwFk}EA!slhEY(W?)a+b>NC$*fHtZ$P^G($#BO+%bwy8z=rn}~hhMx;`_T$F6@Nh9B8jP@5?0cT7=9hS>+2Nwy zn085SgNf3(89?cx)CVu)`-*7xJNqaEwK&aYcnYL#x#CuKi(82#=udaBv^RVmhl)jK ztOWR$ED7DK`264OWW360_wgRrk4w?A!7$x7J`H%0C+hy!n}E9{o>|#{`7v2gHy4R& zRCoPjKB;&Lhg1+tT9Q5(3C%729SlS>NRZnJTv?;Yf;QsB=eTqtK0tWmlHMOUz`NER zTZcxyN`e!`yPG~&%W=HN3N~>MZ4RjT`Dv*D690-Je3qwfXX9k4#uC$N!-!QK2>B*E z+TNcql8OG<%H0oh+*EbEZn?KJn zAFv8aAg?uh(v{Tk^4@jKO%rX9enVQ zpVVXfX$xU}onL5^+9pp!!A0?1B*?f;90Ji${E(*3=s#n3$pJ^pEE$4yyb> z8%;aYEwrI}Ti`WN;I*w7D#fq%|M|onfa}!$i2^Y!JU~@u9*&&gMm8iKj{HJ)9brl& zaXL{T6Irh-2MBEdX4-Lpq^f}h;aXmw6@E~H@ENTWHQ63MyDmHUkCra{P270U=2jq&HWZI{g@^gn}f9oO{v)g~*jOjQkB$k$VjiYb1Wa zzmEd5R#VD|-13mBxEPM{S@-KELhbPZdPEGRS*INoLzFStSM`jeK4Ei!Jis@A zi8v`$KgP+Zp?8$7`vJ(b9IvW|FU=e|;KbAE5aVJklN6)75YV`*wEGx!Ysok@%D!rD9o7RA7}~N%zCZz*x{w$#gaAKeb3px1lq~<(>v|mBcBP{dwO7hnkai>o$LJ694<*7 z?mn0&6Fr4!| z_sYw%!fC^n3GR`@iX|huZ)U2TgQ_HI>+N>%`6s;ezi(K@oA3to)gG?druoQdBlTM< zOV?{mLT$ba2wi|-<)cgTT)m&E+qJq!n6`-by zLxUKSG{LF*GW085Jff)G7t3P7SE_>n(&wh`sPWhVA;ZQnuDs&1ImEi;hy997XL{H@ zF!F<;Hh82Ss>d}=KXU$8%pIgEJYY%bWO~=!yHROW{ZG^&PlE+e8NcRRl@JQq)LdAm z!kM1Ob%#|}7nWX69SJ|gt%;WGJJN~WbQ@JzTz*m=9gwn41;SwhCdfd@EO$&ZMBLaD zNIyE2?Y{pGJs0vanA#0(zw3cKQRXvq5g7!(@E!DImIrke&+Ha|k*)MU?AROBn8`PV z-Qq<7f9pt=eWhn*x2R7Yp}edUr$A0pc6QLlK+><_6^g!0WTW8HILnhu=y?)1AJ1~> zG`hJzGp|Ltciw|F=oKceg>4GAkY=0|@|fkx^`5cQwfQJ_$Vk+&XwP)b$&mU%z>s-^ z^>5c00ar398&uD}_v(8dY~#SU#S!RSmf^$O9g=z)rsC!eI3M?;Go)HWh>CHv%Q==Q; z8z<~7Hm3c{T$Wi$;>BA!Q|>>+MtAa9+-ENOgY%!ge9Ej6c%glgDDfyUG39%P)4Nyf zZe%lMntfxw8LD3IEcOoG8=j{2P_r@@Jjp#Eh;t`*Yiu-FW{ES?P%vdIZ~V@jN2{*M zseTsKdX%nv1&iq;HTPpXe2rbf_-AI^^Lw+sXgjJD%6C*!#|qw7ahk!-c2DJf zm(-pj{$Rs1t)PsKRvX~`Nx}tD8@sdmReRE9R@8!p@Sk6ZExL089YOZQbCd*jM(Qq8 zWKOFuV@;ie0K0|_s`fYcFwC?!uKjwV^6I$&xK;Q7vv9 zT^M$jZCzynE;S}*i24Fwz}@8<^$}$ij{Gqo(1fZ8JNqL%qho6JpX#Q%6EAWW*jy3YhBym{R@8sT^z>QMwA6aj48u8TUJ9)!(# zKW=)ku74hWiC*f@g{E5N<(T3{iL-WP>u2OV$c>e24BZx#QGB-tRK-~oe{k?`c<<-f zk=mq|MQL|6`ZNT-X{zb=jfxT$jTR&p`oL9RYVZ0=bliJAquDVks%-I9E-yTk|0N{K z#)>&kRAVN`H1GZ^byT0`V+XvV^r1Fuob=Qc(njN-}jFb)ih0Mi5pcW(<|0nzSc0LHVLox9-Jbu zHX?Itq_mY)nD}8|!0&ZGNFP77GD>s=j=thF^v)G_%$qfPW1K0@8C7{3*}vo!-L~K$T#;C&wjB(F7oV&g>=}2vhaF6uhj6d z#Fiy#RGQ^$=SaQJEfGg}K_D`KX~-0gccdqJgXPvyopfMn86e2~Ni6YW*Q`cg+_dFJ zx}Ao?f#ux6d;b^esHsBNFFmTP>zp6*-+Q`>OL48&t#=W}^=c>t1cTofS=F9<`Q-$< za+IINJ0z@ETwa$Jm0rt}8h1J^KFQ&tWQ%k6D!??M@n4ddNrNnD6N(eRI~0Y>@?w2H;JY)(!_gt|M4ct}bNO@}k6CZ@h`Zsr;dH}tM$3+u z?VI$rtg|WnkMneI`Wr<-Yqm(qfq?Q!`l*`4RcRt{-d z_TaK@_UK@f)iNcE{LsXs8sEr~Ft^Q4kR(xu525e5&q<9gBM?680;> zfjeUex~Pda9kD2H3_+(z>zT6nSDUSE!w$E@D{f>pRZkmQb5;|Ov0=z=g*-6D8kY{i0+b0Ae8dOY`4`2!rcD+S{mqS5qz9hBxfs%V@m)YJv6b1z)cn6)hjSMKJiH@H#H-c&6%R9tGd7&yUEgCc+tk_LlCy$}#|cuv0fb;V>0ycn z#1Jaz4E!G6N?SQfcYmo5^JHZDnG za+4(`x_ezpBj5p5x0*prWA7EI@kjaKKzm>a<}0bElIp6@22Eb<@1JFnuO0go>w)c( zNw*taz1yY*tzShmnOayd7eDnIvF1w0kfP$OxiK#YzNkB+3lrP7ge9@-(SZ{lRC^-( zqf?c(UJD@%G>WE{z5$OL>yv1GjDK~EI>d0_EoN957*{6;rm9uM;D&B*v{(iY!h0xD zQM&e_Md31ZBa8MbYHw)DK7_G29XF16uRpLL8~DJ4$PVJ@UGTq9X8<3-g@f(?Zar%< zk={bPG;m>wEqOQU`~aBYf++n(E%BbNdN-3P^!=0MHmK)EP!HqZ%!%|d6C{7#scyl6CLJAjFFLy<= zyQqS7x_pBY`qtvx^Fo)tARPBhVKiH6FO6W0B^4UW{&9CT6{wgVyA_7X>Q&Ma)dKyD zhVX4tJo?cwM4K&(_=5VStrm0~H^?|W{9JWQJxt0Q^$;k5 zzOzJk|5(e+dyiX`&Vz^VQ`wxtfjohpd#{7hf?BZw8cUA)w)`>97K$yZkiW%;9oL*SdfkO@ zfBR?oGy` z!qyxNWc5lWO4Q&hf1|zFBtuI?dyQV4aW*n#)*>9saRSIwB6dHl{hSEGEc__r_Y}2n zHFru}E5hn%HH&&jukQDiGa(pXIwZ+v-!ts{9UoOt%j#5gPv1X}7N%9F)1j$bP|*@5 ziwod$xwwzrNC>1F{nFBW!5cZ^l8O@~j{Qn>;12OeR=jgS$@+q&`IQzskCVB_Qd0zz z4~|c)X8rxfgDRbn(Ia%$>@2|T7)a?rKK0;Xi)i`IK|;*r=Qw;>I`*!Ck1VSNczFLq z-djgS)qM}+FoF)T^hhIv=7j+WJ?Gb@I8P?Wn8rPJx8|f)sD_t z0uu;36F+H1C#MDE-w#xN|9rG$EVs$YdB5%yAJOh@cX+2{nld)EC5y5HGO1eVc45eF zOUU*hkkxp*<5hFl7l>>2l5G$2pf79ksuMh+3GMQgk-!(KyH2-W@R6H{e0af; zWod@HL&JV2vO>Hg`s*6tig&oU^S}t@UZ}CW0X(6C*Z8~y}K_x7X_2U-S*LN(23gqrvdq( z{jVMkTpo@JLN4(5ao7s`w_>W|C2jGRT<;@k0p*CRtX=J!TSqUaH6ZU6WjQq%qq!0p z^fe1J#0sqQOrpO%0B5rf%y?P%gimT6Xv}+tspT0BC%Twg2e(T3(?wKDBrb0N)Ol_( zT3IxiqFEibcb`!`V(jI+*i?isEVJH6MK`uwMG>uUvP0mhuej%iYNXbjhw}-mfJKbd z=ezh{pPNH@KW61ZzMcYhqFP{6wmvNY$Ws!6-}Z+OMXIjcFIoy=pVXw+D|uM#+3YGc z(p{#4Z{OmCr{vexKL<`FI|qTwm=@JlqXGOto$q&chsWVb*;pFZYX6K<>do7=$VL1 z?mEC|pe98?=!8Gg1|IXF3$Q0-gCuZ5a4jLh>FXHGu0EeBh$tN^ZS;to4u14Qp_ID2 zL{CnO1#|j(nv1>9N~qUu-o7oCp?Ok@zedjisSCGrNTe&R+*&&AA zwjAXF$ruf5`*S&K;KDo5;2gY z5VLCkD7K+@MjdPI{>_N;N(Kmc)*m7%GciQF7o&KYOZZ~Ca7UUYm+v7VGrQ* z0R3V9r9Wg?gR&w7V4nDxXaWxe^9$&JUG-+>1f!s-ct^(jnQLe?s{!C>Svz@m_d>JR zNifbX?WH9$cF+6UPdoQ~Np3BKSn;dI_qR<6+jg@diP^8j31c-ZQt%2AY&Z>izDZOH zSMK;<(MQ&|9%B&o4U<(2dv$x%R^<8*dKVB?v@e4u1^aC0WKYGGHr!Ht+AORm2ZxS% zSj^ysQC1qVu$gKTjq_oRj=BQ*oMfp&>*#`CCYK#{&V%Ef_nQLzx*XPCTELgomH>73bnG8n7j+iXf`3Vew)0cr5U}y*L}2!ZoK$ z0}Z{acaG6+A?h#hroZrqc1_5RAECNYM-=}_+RfnnIEp{8Q&;?WpowtpG;@xnwRl;nR{ zwn&T@&$XSD)g;g~l)im)r0uKig5G0faTyBw;yW{a@uS9(R$`e|uz8j*B>^#AV{S;v z^G3NfRJcwnM{cPS5MyX{!bM$|SVumSNkE%byBUOX1SgZLAo-3F`|1iOLUgI!H&KIupON!jlkZK&r6-Qmx{% zY!1LcK>pS^AwH_!Sb*{kY%)~mOG)j~v8uG^)8=bgUX6U)z0kr2p@OEj)Q5jh0V+6F;LAk0pWJc+O ztV;Kr?f|6#fcXLjnyhm@sm*@{?D~VqzAcx%+`BXRpi}>;=+KCW#Aqc)(%thGz}R14 z2hYU|;^;qW;c<9-_LBIvwfW}Gsl8q=|Nj@moS^i8uo%C82m2Q{6Q&HSyFBdpB zpw2ON2f_!PDgFFfybA=&rQlSZuFyQ5M5i{8i%)h8{Ng*kXkePUsm?$0-p7Jrv z3|2Zx0mQt}MfPZfx`RXRA+5w!=oKOlPMHNQpu#UKrZ@i#y6k6&j9H<3`9>2 zc-0zBPZCVx1tqa~=1$5ro3b;_G?8?sdMl_XQ4-hHEi~#Js@*<6f@h4|a@ZZR3vKKi zL8kN1WtMJ0mdy_CJyR($Da5Whm$eh-pYLT~J$ZH^mX^^WhUdnDy|Ezag+W2MxePy4?Oy0_l%81ZQNHqu3$5FSAyh2jC(|vWbCJ# z9`Dfo2hjyd`KXK7kD%4dNzNjMi>rENQvR5axA>DS8sE((tnCP@D9zCd=_ZSo`1a_= zS|?iPx)@9^Qdc{tG;kyQ3B+YVicdFnEBz5VeDcK(y@>mAwdkB8q+3PXOS@D&!){8N zjUTh#p1DT<3dF--Z5VRZ>GlNd%l^EOfN*}O@KqZov?cTjmfGc>sdS(9o1i2jX($Gg%>I`}dMK=9vMM)Px{>6y19HD)0o z!r&?3FyM9zQ7<1c9IZvwgC$Q$WpsYmN59!p6Hz8FMbmti)`Q*rkf*uT&_=DL z7qJ=Z94%sImULVS1D_zvN5GVGw>h*TlMi}=)-p=&jdyz^ZmC2R6y}q9>x`R}gpY;A zk!fj;(-MvL71F|0#rj5K1Z&ha`vth57j)Yn!JEaJJtXgzFIk$NRvi-*@A<^4#*(RD zW=Hq9=oZ^?bbR(!cTwiZaxzrq=nyRNA_2*rBIEeEtUKzYbsB5NojY6yM}L|TlVX9M z;maKGQ>ynQA`iv%YaW8rNo>Pl1IL9(sV8pm`b&%}Dush}Mgvrqx03Usz$ba^jaDpD zLYkFUM)KA^PKAb>O3-Qe>(dGW2192&q%QhH-;cTdm7-$69sF8Zj!>tAz8nv@*YVXD zxV%|pRfH@0Q)tmc3^GR&4mnGZ(eQdZx3iLDhnItUbQfW)Q!a|^h?;r?GL4VN`AKyh z^M=Q?5EnEB&Y#&XI@bobI>XAG!u#F_B&qi$MQ`qwX>v2pnl|yCTlqe%!G%3C=JJBA zG=DSEq*X%~9_qodx80#O1DnwDRY$oYjUBf9ELv9xS9{G`q~%#_e}S-w5sML@@eMB3 zW%z|y|KQ@%jXBEmkx)YpTE#>1{kOLh9mBh30}N@mJCH`O%h&F+^3D&(N{{nwIa``U zSJDgtcbb*DP<(Bd-uPQl6^ZBPEt2!S0AkxIl`3skrezbZ&Z0URn{XXw)ujv= zr?EEx!yG9ZH_mz+;eD945giqDecY}2XbDIcN;A)3hW7aT3FOf%YBi?6=^mSkubzr$ z6?hp36o&&mtN2Wqs8Y4W?Y6le1PE7^h0J7G;zrO9mApUn!fI8-5((Q(Y`>Z34nCEi z@NY35FK0etRS8O$`fyG=M~GE zUC6_)g|)3No5isxS6rQO!P}2NUnj+bi-E(!z%~U4P(p<((j(6MRtqa%qMN8XMB5KC;5=3|`k>k7qGGHM~m&Zzz&CY(NLeo01 zdz8Cdz8fKvyxtsVX<{{xLbvmlvA&WNQwamU@mBjbvuY^`lRkQM#h~@Yd!vfW}!%ER=O{LC$rR@OSke(G#TQ=m{L%a#*2K?J0?VHmN zreEz3{;YrhT>k@c$foLcsDO(1yS0!G7<6%lBx(!n4u3wWH=YR~mH3l3wLbxx%aG@C^W7_D z9ibi%vlQ2Y1mdl6gNK@03h>UBMenma zKRBQuDwRGMbSX-_=OWhUWTG+~)CPpy#*I!29+3-p8M((|(7Ug89XIuyHQQBBGnQ-6 z7k7ICEWAX=YGWAFuYsx z5o2|a0ElMACe$uTgKKgeEz6j4T$PO!CTykNHhWar*A6kgK=>*|@D+q=xNDdS0a_sr zWEt)FW^CK;Q<0O5E>Cqstper5e3Ljm^**&V4zbXR1RK(P^}+D`y!$vb)%}WOf$x<8rc56TpT^l1US6AWmdj-e} z2;UNs(H|Cqafy3)j@yy@na|HGr+34P^Ifjk*TS>`LkbXX4#eo2t!Gd0ao4~LV5SI8 zb#&S6#{9jg{-W%V4chU-dMyM`qt4`k*ie<9-J0g zJ5Lx8%@>8%yYju}{p9#tTtge1V&^0f@>^gGd! z^X02(L+6p|PKr+C{^{sFV#3k|36I~MnDFid7A8(@96^Gh2}^_|Ttn>)=JLDk`hhVr zu+@SC_E@F-#++C6vN>J$A-4&Jb4{92c;At6&2U9o52{&Jzkm0HwybDYU4HrjV1z2m zJQ;EgD}|mmG@~V?SXbn*gQp>$Kz#gA>kZ7Z&tb!M`v#qR?ixbY746&w%r3YTRvMRu zS=HtDTJiHSkrW8M4jTu8JlQW^gkUHUg-rJ00XaYqhuq~f&m@+>#87dq!m~u}cNgMA zouoIOO$Q0|)%vdb$R|N3MD^+K#(6qyC`Yp{T(NR=4DKB|PW7y=y@+O|bgj`?h!49+ z`*{=&22P0ILDDRjzWE7=4DMn^EKoZ=5bfr6$RmOi-#Ui5q6RTz=u{4Qnw?sNSG{kc-z&ww|#(0J1W}obaD)@23oGYv=G#xbzzqgVw$eA ze@<;N-U`Hg`H3r{S$AJ~DC4YQo|mF)kR8w|Z-Rhm#-}~yzDz+ow{<4Qh>+>%o%y)D zeeYUvbNXWIgV!RfC@Z%2SV0f6W^~!J=*n~_5)bwZl4Br9PE0Ke8_($L5GjZ)gCBH-1n`DbJRh8 zh45%VegnhOv%R9)q@H>BRGvDwM1JF$Cvw;70XY+n@Vr4*1TNI)gP)z9i@-P-Gsfe+ zE;31>bc!M)n)&r5+lHX70asTlV;czRW>}o?K04`PlI^JIUvIcFmZ~m8R&I)n2GAW- zt1a-SMnIi5T3pV+U|YRf(4-k&lPw{iFYWThCw8lKvki^dBV3<=Gy)kQYceJI5Ej5& z^*sD`Im!F>@JaZH#uO1J&exG=LJMnJxfe)CnS@cN_YSe^wOkULv<942X+iBw#zLC* zS^W=I>P@&qRnz3CHb9K6Tvwvf&Tg>v>kOAy?Bd@UkCv+v=>2=H{QFeWxdX%rF!yr@ zR@&;n0EP&0XOX;$hAz>kaih9+=79X+B-Nhu${-}XH0LZ~Qt`K=g#kqxIFmW)RkUtK zlx(cq^`$0CkR;n#PTyZgH%RY#4aEJtQEIK+nvZB`<`M7`VZ4i$$ynCdJL(;~iFcEv zT56W2s=P1VGCP$Hz2^d(hp^yYzMu^p>T|iwrP_i}Qbtt6bLdT;P$fzfp?c6IvhJ=ra^|q&9GnlEfHuEt$Ohd4_;%*r`DdA}D zsoPv?T*)CUFQFZfENb*oXC1khwT&{Br#QD9kOT#AsP2#Vr z~$6Hy_D;jO3vHL4d-7w zZAj~KZF!mYoUG->IWG2r(9gP%7T+L4JSPFD&nJ+^w-eCKbKDm{h3uzTZCWsZV)C&m zmnm^+do{QFo)2Hmc=ZQJAP|#$Y(5DW)O{DdKF?Yn|8SkJMv0JPFa-2V=s6h8wc5ew zJF?P{jN*KnZm<0PL#bDVm876gn4v7}I55dkO^WU-o*P6gpoUue4?rZ6nbX9ZPe7D6 zIJQ-doweFy(V>um6pNt*rwItLBID=5K>Hy3Ubpq!;AYeQ*QWHs>}EeSs{6-|^BhZF zJj4Ur85BTY#a0GG`IAK1tgb9T<9jUW=c_?qo(v}R4@H;%y!0HR1|*2-Q<4BmX6ulK z0kbW~>3U(qSVXgG-1w?Ckr%eeDJUAc}t zxsC`Mq<(IAp%JU`=hyeG&)t?|Yi^WsjbdI^I+;R~L+dZQ<+Q@;aaoA-P-tI-IJN{8 zkMH@O*c)TSC+l5N!L|vMqOhnOdpMHOGGpvdI`-_emr3Q`y%@|@%sy!wIcN*9%c=W2gAs_aXPwv=Zmp+ei0RK22@Y zY?PD1<(7*w%&5iMiN)@$Q~KlnZ6e(bQOJ#8a$+|%tJcsU?u zR6e(+PpR##PF&QAW!LH3oQd4}lzPJEtbpv+LBr9u#RoQ=FNpEeR7216 zgo;+FurOBkv!_RI=M>DItObxtGH5`6HuC10q0ZFe?stjzpYOV!w+SY2>V9bUC9eKb+dChN@((~c#t zs!7y#7?nAD7<&XueRx@K!j;{kBLKrWuMX)c{{Rnc^&xX1KgNn&a3o~C4XSXzLW*@9Ugt&_+u?N|f#+`<{CNr`UL(;Rug z%`T1xOWrC8^PIcVmSxE{$beOgAdZVSX66yN$!5wQEH~1A+IadTCEHEO8%28oE(LFo zVJu-}){f+ZhtBn9X23@

    ?DMrPNil?mG+b@}n(hk-4DUzS`HxkKVY!NgZF_kggTp z>)~3)++t$&i5R^gxlqUzS%FT%#v^D;6(cVBs2LMMp`%m*tfdGp1F5FrlNG+R^{#wU zUgXJ~laOvOnt+xvNCgP@CO^XDxDzSj9}At5C-_McYolSSa#Hvn`0SZ`tR~4`^X<&$ zAlV2^ERa8je7?nTY_q+P8cFV@L^GmW5K;6#=`H*gsj@f;tG6kTZ88wBrkQ6$>r>q! zBs9AHF224c5}Or*c%Q{@&OkhLgAFORV&QNBwL ziXuw@TtGbCA-=MCI?V9MM=RzmG1ezNKqBqZw@1WJZ$Yfq?YKI|of&{ zfbrZ(eB0x6i(;Cyvcjb=`o&m(hhQO@A$P%x^W!v2(wY98k!HIKSObgCCQv3LG%rx5 zz(6znV{7JUx5PUikr+zj+F&um!>MbEBO6ubGSS*>m2>(h_3vsU82qj(VXHg znP7;%$v!KHS64G0LaV7FvhpyRHNZV{&(YM`Tdi^YG~dVjic@z)cg5cje)X_3ZZO*o zC?8fq*BTu6AYIjzAN{T;;cZ35U-=?38pg<)C$ilT(L%h7GwpDZClNVub}djl;Z#d_ z*)E^Z$ZRd!sNh~x^O~BDMfgeyYPh*g%ks|q)8WMpr-)Ba1y7u~Pfu~+l{A%GZhEHG zXZ=xD^9E=`MDxNtA~m2X%WML+(3i~V)CyYBL^8JDEM7U6lLGV%2mxj+`I3fu1IpQ9 zBh@+4P|E$IHIH_)_tlMCue)&MI@)_La6gl?b6jVO0Qvp*UR3ZbSX}Ln>DiI=w7(PT zcj|rlz2i(SGenp)y|wLjNn&P@g$JfR9Ed=^%Osx3_*y6|GJpWk{X^-if91kFENVq2 zE#scuv-|bHPka^ao>Hp*L%<-oKnUdO1AOrgR^&_OA5C)bXrMGiG%*p&ZvcMBr-O-W zTXPB6(C3uufzl+%4w0Z`xDp|&#Cuh(Y=HAW{uK}P`vt$6;DhQjYHdBs9gO8nO&RI+ zUyJA9UR(1_@ZBDzj9kb8$r7z#{)hkW^yhv5JbQ5uaGrCC!LF^Czmg-aO|K}ZVYZV% zH7Te@A^CzT%2^EC-#&Guz#Oa`rhX-xB3obg1E=^Usy-kXIo?@(TFdaz?K&O!`ad$r ztk@y-qU)MRz#?6*)PMiM21+!Tz9chz@^`@NzD#WX3iSG4!>PN$@!zM8_DTP~4p|_F zE>s<<>;DA}NKIk_rKF?)0dl71LwLU-6Q}#D3;e4k0LTJynT_W#INn}L1}K8^2eN0# z%<4mt$6o(Dy1vpO`&rFEahuBwn_i!gT=-XI2V^G(WOWG(h~_^t{s)nNuigS!kP(h&VFWe#@RzT7dZ7C|JP-2lefKS-c z7dJKTGHLB-mFQ0z{Gf6Ic0N9U>AjKc0;ILF-vn)u&Hl=h18|2ai{wtW znU6YXea~}O-i`mF&TsPK9g}#V#_eyGB*h-D<@|+dIN(j;1iNEL+`e_uQQ1^ z0+!<^jPt}AcGpD%|4`aBSv>+D>k_iZ$HY9Od<;TH1OA}^-?N3W4Pv5}eQzxK>y`BvB#z!mP23NHNf zuQy1BAmf7e?7txWq48^CyY;j$ss4Z%Tm%69ZCen8tYZ6bsQ+~fM63W>YB8?bOdrNcPtM^t}C5q|^NB_6*tCxHBW;D6%;c;g#n;w1kO7Eo^CKi>aCxph4H zWw_`68{Oms^pEM>Kg9lb5u}mlJ~V~j#fJXtrN3^KYmvQN;X_=1di`21oOhx%7+XW{~#54_5SgE zpAh1F{c|8z1YTxCE{%6NKUZ*lK5~OG5u-B7h!#HIs*~84KJZfPn3N014-rI;R)%Gt zO7Tl>;qwBX(>FE|_`geFf{9aFyJA_$EFsB-u@CANCZ0!x<=ilaaZOuI7it!1moudZ zUe}>!QuilP65eqQU!*F0+?jc@w-O;&lQwnkHCpM`o5TYLzO4#F%Kc1+R(H#oL(*~} z|C<~wA)Xi8)1Uk|eWtIXI~~khoDw3YUmpi_kwpk?*OGK*u40LEY4jNYw*HT~HZ31E zVMRz?hd9Jfs?JN4G~`0|!n>lTrsg#JgQ5rXWOFiu!jmWMcrjF0(Ye$OJI9L=V)rL) zbnpNi1lI$W2^Dxi8-VCP{%~XgiEAv9ONndYB^Bntl?Bv|mPK~6XfIy2Jky-rG_6e4 zfm`S8zLlyuH?AZ(zOEROt|XqN?^C?E^yTy!B04%+x@W|=bKrq?+5D_5?0k|zuj;Zs zH8s^LdNSPkzgEm&A7Brift)sfNh}Ne(S12_Hg(0oLUTCYLul=7uP^V>^V=3&d(Ari zMevq6lHAlN*aEV=W_kj8jJ?Gw;_3ypkZ%sbYqGE(4gmd7>1t{o29M2-7v*i)B?~w% z5B+IBfX1)HS|uXu3vkwCfP5?-{-j#;F zM4`58LG#GTk}J-?AyPapj=ty1f!H9o6vZ9Va$<&JA7SWz13=^u)SYXXUzs5ly)(ntLnElJUra$0ZQNzw0c|e zhw+|bg6QViQG4un;;DB1QNVt`e&Q%#w?>iGe2MAtCvo_Tw|v4cbg3+M-%YZXaNA>p z=ukkS;Gc$USd(#VN6H1=D^?SIhcC>#(^C6L73CCVWj}9{;=SZYN5u!TgKCvHTiAT{=8s!$(2={R5MuvfJT#6i39@y1$O^Jeep25sIUr0lyh=SxOVf-Mc1H zdf5>~E(4*9ldMXo$`XV~7~i0y&g8vqf0*?F0Hux&7`DzYs0g_J{lf=?4CiR}HAfe^ zPv?dR@UEZFOZcUXw3HJqDMh@r%j>kTu&{HNdn|lC9uac9;g?yZzyO!{eqY*YEx+CP zGa3pAmEqybf%F%t)w{V-kjF~e#M&lXReX=(Lx(Vf+tzsIf?Vp0FY>jjD^-6KI(yfrq)8$c)3c9{Uq5^F~fnf-x0>VmO|5@`lr2bTMG;N!40~Cr?q-f%;uhHsV z5zTLs88gXLqd$DWVMXM}-*!L_6q$xGFUlhp!FGBRxS<5u*Zz1*v-@`>zyOAE?BR)D zsRshnJP80+i_%|fU$DpJuzRk_lXgT75Ru3=6hh+sjKuE}XyQf>;D?Vh_|_l7KtaXA z|G9feXI7(aY(0R6kK%tWG5_fnt$vqW5Ua};PFh~Y7tLri33QQs@5h?08}jw{q{I^l-7GyiY-SEDOJ-|6Y?W;YgzvGVQnt_!Kg`*P9Xym& zDS*AH!r>Ukn`Hiiz@6Sad|9EQBcvi2wUq!z4>bGAL^;{Lb0m#IQ|&=^Y$g)-&2XwZ z<+jW83axYU)f-?Qj!J%8n|AtLI5dpZi&lXx_rVL|=`QQ6OYhBMSiT*OHZyySE&K?^ zvM@VH-m=AjH3UvkYQ)vVHGGktNYs>Xhb=1sq;PZO6+EE_d zj#IRoN(~nEy=D!wtfPt@TfBNuJu1arR@6#_nt_C=XjPR~PiQ6e0>TWOKs-*vL^Hib zVw0F6>TzPIQSK+KUNcQ!Tdd_aQfuZkj$SPM-5wM4Oqvd|=@E(xtR0ePh=U2srfnFH z-||jXuX(W>b{Vul&674Yhbtq^i7vX%$BG%l^U z%)DsENbpKaD&|}nbqd>KN>&toS9`Aasi|&i*SpuU5X;N+X*?0tYw-!f)@YMwLi?-y z*KfgX6La0$XhU7An8~x!>Cdzc+p43YxW3XJwyipm2Q8!APqBc$G+|TPDZx>3w(e z1TaJ;|6{YNXqZB`%wlueJY)Oa{1@PT5k`?jHik48N*vFd22LTZ?*!1SVb=N~WK42V z1X0T|5vre)THAkA#(YebLC*?N&RG3?k`$OSz9N*8P$#0@`FiPH{CcI6@9>H1!h2Q2G0*fJzRt(WZ$exf#TQ)G!jKu}Im! z9ntby!F^x+#92#hyPA0$aFo(%hBrfO4+1@tM8S=Jc>ARj5fZK9bU!exmuS?Tx7ue9 zZZ@QTCr863;&)$r*gGQ_M|)0^Yu^yAc%Le3O;1YAKVS0B;8B3N>p)|Qkw<8;H*j`T z?4>iVxoTH>JIm{_rC!NL65bvsU^b6)#2L4pA}gd+lIO8hl6RA2YT1gaV3R4e%ID2k zhtP(o?a3{KIZ?#$NH1+|#JiNS2I=`8O4L->hSYIr%Pdcm_vozxCw=sBCaM5GBeb?V zr|tD*)s3zrw`Dk@(9a?F)j1XPg+ezszpd(5vbVPrS36Y_sj(07Q5w2hW%Y)8s`JtD zW9WeDqwim= z4~flx|5_ga+&bY2;SzGw{(t`7umVK)*=2Rm;eUqiUyxP;4A+G8Y0SR_=Kl;bJ`8Q% z9%DK~y5kqu{twXp4JHa&&s{(u=rE|@{s!~MBhalQ+Dx0|tBmY6h8EBGganynKHH2R z*)JZM=C4yixKU)y%2aGL^UGAW$o7HpgL}%Dp}dkBV%m!@Zc>iIf-osZeF7_d9+m_s zFO#s@@GaIWUo;o4-?DnpW1Ka0sQfHzs#JLl@2fL@SL1*S{=)$AnA?Hr*ng*QMI<99 zmsRX*m&F)znSLAIpGo6Qn=2Y|Fk<5*D+}@7-{#llKjMR^R)O2w+6J~$JXgoqQdTFQ z1r*pveXc(drf>hWd!D{f&C41YQ{f+yU@cG2 z9|HtF0wyf8UegM>KD4j&5_I5EGq4*cHfb;7c4Tp|TMc}?NOA4_Nh<99b;tr$2*?~|Jrzapn6gs!|&ErgoX#LzO8JEC1WvDN~R}&K-6;@uATenWv~)Rs-@7( zfD8(E)$mSB(U9-4;vEvsa(-|7a8l;t6Ru{17SDzdp>;_UI z!T#vNfV<-zzmhJ@c7bQSfGDrP`Al=_K@8_o7yrR~5_Va;aG#X19qZT@K5Lh+o-aM8{lMu)0m^B`SG+;GCd*@iQtsWRIvrP+^N4kKYo_+TE8 z*(9+BWd@WMyTq(q#m1AV%wd0Rs9kP-gqo!byAeHs#euUCpW{{0VrW3b>5C!^af0%|3b*6N?xW zM{{oL1Lvd^0~Z*lh*gIgTRS)`Aa(VExca@F5<9k!ir%{7ZrIgNF21G?Pn_l5slmZ7 z(go7vq9aviatf)m+&v0-_vs>yFv!cVnF}0h9R_e6&-^k0mwmDItDnzDU{9qf_WJmH zQ(?Al^a@wDxq9(x(Q2#RO81A0ZR!W8`CM{#vc&SX4LpZ!Y!D{*dwue@*Ga4j42Hw^ zr+%tbS?|2cD<(;1=?>KqST(q%Y0Ewp$A`9Gf;CfUR>AjK$~i`|-9@&=zI6RUP?_3w z70|$r@8K^A-7fft=|(;=9)8hMQc9i zOXORXOVu_OpKBJYt>i*l#ek3HpoLCc(~ORX#E@7d8?t4rS_w$9tfID+cX7rIiv z$oUQibT(|HT$CU%{Vf#5$?+p|+SpQCWSrN>T{gBZW)T^jrgCZrNuyP6I(+;k_8%P) zq&a|XgKPV2K zWxC!N$>TvS1p<1hG=krXJuAF=-q?vbgm}74s7+cIGhtKg^7kG;i-YYyUo^v+-F-9k zu4aQS=F|_Cq|`d??C+^D6oyNydq-i)k8R_Z2 zk0&>Ss~l-@1Td-nu8?!M3| z;L2+h<0muksjFPcS5H*c)q$RR;3RM@lAT(d_D@C&$7^@yMs|ZWc%h?-pQNf4NiMRa z!v}Lh;`3FPfbTsI!YO%&duV-!4zX%?q3_k3G@VoLe5uTwKn3cLFUX6J=H^dG`nzG1x)xu;eO-pKL{hW$Y!TnOeZsRL zqZJ={&JJn7>RdN;OT^$&e!Sp7z1MzXPh3PczFsc)QmG6@wErS&<{{ zBVRE9b@(F*_ir7St_ZAOA0r-dK=Ihsr76X>rJ-`6nT*9S2D64-9X^NGVd3TNZ ziXp@R#w}phoxg6*5=jp+)Q})*suFKI>6WTy1ytK&wI?l~W3^s-9#IL~2SYVWqo0pG zP4g}8wLLLgjwjaQXOs43@)NJc|2h6^Ek!QNY6SZCM;7CzYr94v7RiQ`%=dYCVm)T3 zBMfPH^c`!-JD)pK`?k<)BFLuVg+Q0TSFlv8Eag*q%7RbnI>PlafnC$XsGB6wyPT z9WVo|S;gq2%o%jNNQ&iB+alk}_Z${lFJfO4UHN1kJep&NE#a>^LG&%`7k~If^AXvn zVRUk&$RK5`bTYk{rnkT@dF-mVmnLSCRVpCX#>UX_mXx~$`!U)pM}vGHxW++%!hJq# z^=VsY-l1muCavD?4e^6|e^KxcSEQZlh(PM<_o+iXN*vJIan_GQVH^l6MA$%FsOp2) zNR_BJMI?fdh-kK=%ZGN)*Dyb);a>iF!*Xx;b1pzsjjvnZ6rK#a7kVZ+-OZTS%(%np z7^QMbRHRj^U6|bD_Rgfr4&gJ){rI(bKGDN3*K2WM%)uKFW;tElldCbkQD9xix!+3$ zUz%X;9g|w0p5n^uNAkal*eDAtycZ#OE{}mjit*9ygBAJK&<)9hvl3{UBReST8E}q? z(@9a_pDVn%D5%_In~n#V4FOg5#NBC-@ApAPnISSRPO4v6@Qo^N2e)(uoOiiJ z#9s1WoDK^ca34?_%5{^nyXHQFg%F|RN`Cay;e9eHmA`)LMx_pNjZAqfu3AG%Tg21C zQe=oae4m;;r(T#Cyn0E87_V3z?@7!#_`Wd|xJS0SlIWOnsI~8FRQ85uZA|Ov31+9G z{gnit5RJ9xM!)mC@$3F`xz@A|&=GX5^p*(kT=jukk+o1x`+`g9dcW&q-xolt5Yp0p zF>ZLyigWV!0O*XDdsq{u=Aq*HYfc(A>q-2tZcO%~0J62$M}2sxdBCS%8dx$?k8?^%NA7$?1;C z+|`Q7+M0HW6mIogyol-x^1N~DAgih2GG`3OF8ij!_qFr7y{Rwa@8Wapl15%87Ek#H zoT*#|(Ppvn%|3b(LNdDBqsz~eZ20i5dIu-D&EphzIq{_=zb2e>yQnU=Z?PuMo|N!> z(Tn5mnyGH=w1^8as9Gs$im3D3lnL{-!MxMqu$ro2Hn$U2AM;HnkF`3j)eDuZYBIXE zL*Awm4DXXgw9}nri^*nkq~3B9dtL5m?ljV@q5Wj{hhqfG)^B+) z5CpE_i&S3g@vIe(b|8y)81-gdWGTO_8IGnQf*%4tTl|L+e!Y#E%GaiSqyi?M4#rOP z*|WYJZ2nSx?$bN>N(5)B-1@8Ak9h*9VPTRj<%`?p)K{BBAK3yiNk&E;!qi7yd#$P$ zKO*<%1K#`W)VYaz9jf|HDjsDE4@b@1M&tSNP2;)~ht0jikD?w)ug?2_nR?U;NE{E= z{wTW2ti#QjtfoPON=2MUT%}a*B4SFv4NRZ%KkwUTk%@d@b@lnViJMfjCs*S#$;Ejy zyv}^V+%}58lgIzluweK?_1vvG(tYuBEOmZZt(CfRgk!}o`$K+8f5z^|i64<7dVsq3 z#{`wDH7AJ!sp{2CvwYKIJlWY&Z0eTC;SjfquG8W>9zL5i+`ubjewUv9U}WYyaH=ku z+<}Z1MiQKD#+~dwwe;Xr&=r#u+8F)8^Zq@!&ur4eW;jGB+3jPl?PV+LJM*JPp4{oz z8325Pj1OOWbPy1{@NDPVDX9<0N1Cwm(`i;-&|V)XCAPe9V^`Yl1#M17_nLi3jgm4d z>TWdh%-&%kCeoqPOMwt*wrHi)x{$Yo#z^4tl1n*oJkh9+Td99nK(>~QQ`#_YRpTpK zKm8hYJBt5?|4*csXMt~dmE*VRCP9`i-29uHBTVLrL*Qb*%J9v%8|tHc`u6z5jsSaM zD6EX)qhir2Zu{O~PEt}Nepl*|ygYa37vQ^NU{{d9ud}6{>7@X0wgiL$F|+Ptz06^) z7EIy?;^VyeD{_o_x|6%6Pk~DByF~sUHv`s>3^d?f50Hbq6jpv}g02TkB9%m1!1VTC zj-7vwOn=Xq*B5Mny5ivP|LCvr=YPHU83gd(OXWv2|7(lCmNNT0fK3Yvo9Z7=$^T}7 z1606y`<)<)-QUNN|E(SH+z$(|)9D+}3s*?2>Rdf^Uw2nJbxx%&O3O2%!%T2gk=PyyG1)IzM@B=p9R3=cO#Qhi`E!(4Ke|{ z?GUGGRrCG4`)?n^KMWC8gcZz^I@rKu9w&&mYQifc;Vw!U-@xacKRVG@*3tUzr3`S| zh?T#^O_?q*6bCPrslBbNZrMJ5ZGD~=`$*Sg8Sz@0a#5tuVLi}~HQ;;aqQ;!*m15rN zH*IzSd>lX%o9NtDPi=7|)FX-r=}k-$!`(cTKdXCrR6}eLQstC4#5Gh|({J`g#L@*4 zjPVF`ym1Hn7PSG=9R&E@L_jV=B-QT1rTiYx>{gx8SYgEe*q(xc8_vXC-VsPo#kyn{ zUl>8j_S?hboEs;(uB*X3V0{@;g}V!c;ReTi3=Ni*lg8#@;reK%&z(6J$_8f!ZlHca z3pbcB*XA1`i7=X>%`|8gu?)tgO0wYu7E=D^d!RV;^-|K&p;=cYce5gN4-F1z97WnW zz~uMrd3VrQ=eS}qAGfRI=|+(E#nTe^Gp6LiU450xqtFW+w+!PDL%w9V1Me@y8fu0W zuWA7!HWkQa+0trWcdVDEUM(ZxZix#2P1F`}k=VWvHGf0OdW@9>PdE2&VW@IvOk9+# zBTQi+$aU(v1%tQX(MTozmUi2m*pM9HgbY zyHT1$_aUVlq#J(w827o)d*Azh|Nf0}hU3Ul&fa^?wbq=Um}?|cXQ^R%y>6$jLG*hk z15DFGYxr~JeeGwnDlG}2c#Qjm9}F>ngb6(!zWm&Bro?n;?R>51^yZV^dZ1*khL*rG z55G&}hr?miWADaz5!T^vX-(cSo;drQ^I45JUo}2s`*l!BBa$rqe(pT55Z&MsrB=~i zXV@pTkNpidFU-*swtDeV>UOs-yF`*YsOVXFDX&S0RA`esK9Jg^R(Hq8Eor_3YoGMx zXib7Svkee_jEo>4qu)`MZ%&vDE-RMyPQG~OdApYjVxnT|+qNsvDJ5WGGTukb1~!Yn z4ZG#dRLGvN&F8h<=Gwj5D#>lj0TVVHR1=s%O-TupO1eT5efB({BME?({xT8#5E<>cL!Yt4!(f$p- zb1HNYQ1C$~=S$f~7lqv#!T@2^E(m1ympxA!{!AQ3q`ezgQtR1EE-Gs3>xHGjK=|2m zA2=#1Dgt^Dl_mKM7KJ+nMj5X(p~UW>SzqFIbN~k%J#n_$(m5U3qC5mPCL8uB>71f zw7u`@*2`UvVd3XbUG&|Av(Rp%TX46;^YEN#n%HQpbsE;W7&}<-{DuH`y)7)Xe)Hrf zq8@(~l@Zu-R{tfy3x$tp4W_qaN2Rr>JheR4-OW76NTodW1&@`PlK{ZSNC^;1J0bjz z(Ppg6RuII+ax-=$4pyg zH;C;%UC^(LzQ#I5?wyOiSo9XW>#C~rS=sd*44P!Oo{FrtpL=XEQVz4QbOyugd(iA- z(z#&?kO52Ob^Z@bF{p!EZXwKjCGzF6eyKG>>T9A>0G$4g?;YRCiU-;{uRfoM`{~H} zEtG=1WsTYTL)2c9{gA&v~DUbV**02qL_ceZd3T9j(3 zqeqxJGg=w?sgL+9Q!9GIK=^m1-(0@AY;>LB_dB<{1RZZ>>|cF=mW9TYqZq&Ql9&EM zC9^eVKiC4Cr07+f@i{D~;y7(*i<1usdG>?NC(ag2mA?qVOq+#1Kx;KiV@_5j^Ua9VSqJ9`Zyz8pb==ErcBi?TD%O>%oT`hq+uQm}v(<{{Ij>u9 zh)3GiL}VMtic_n$ky6&A!D^|NUyG1js9m_7@fQH|e18xuR|U)&NWYa)_k8F&!}fRo zfFT=E{PMcZp|tVi{iV94#NlaS2Zzd9*Ezjghro`~n=uS0!ab`m$icc-CJ@BQ!=^%? zm%T|{Vm14!COD-Rh2^P*YJZ&yw`)VuTrf_~@1<9$l^RovjB^g|c>lpFuZ-GzUz=L` zzWnMe!QE+hKh;^oskH>>j@7i3O-5xYant}2fTTHr3v2!SgG!|-_wkuqIZexXKheY& zSqj42OC_hHZ=&f7x3}ZS$}NgAI}2GXE%*Jqv-d*$qz9svZ$fl25vA&wFOFMjFU+SC zw?K5N5v9y}A1#y%bNWDoT4$AyISzW4DU|O+i!}^{t6e zMujsN<9RM~-Ga>7F8dePMK3k2{&MpG6L#N4*wBd3<5?N|@_9%DDo5c`;G*{~|BlBd zzjB@?e-i%*O_@8OZPS=*+u!F}%*9wD6`7t2o!%?ym_pyW$)9fi`qnS?$z$i;Y4c8j zpO8t3WFLU|*J>p`IZ!zh$Ej6HM>P;`mxf(D92qD^F#gy6wT242J5`!@A9?P&3kG+H zkE|?#OGG?gT$-h%6hNtci}NlicS92hn7wBALL8$()v z_#X4jT6&_0DsSyWrr-Gs6Dx9H{|1c~BMKJw`OyMY=ff*Nh@m`Tj0kS^1+t8TC%)Db z`?=g1P4g~zPds)clCDnBV%Ci*L#%hM>}+Q%yb+%eTnpAUEBStwO!wuD-e5}F)uuk` z{D^bc}T+%(98Hj$QpYtx4!LK??HODXz3IX*$z9_t=K%OfYMyMdu z=LGhM0qW4>KYLd?_GNMcc9Y%xt1hnm!>JBkZx6)=hxL5bR87t1;u4+ux9`d@z8MSR zyI)dxsOdWAoEKyssy2wqegZg$QhKS$SQ`=VRsTfbE6eJdT<9D#E5u|EY-#)|}jEbm>gkOX4 zlk8;~M75|^;4BQUCI_JP#z=CjJU)yRe<58=K*!@QOyK-`0~QLwiGrXWKiY)67*@=9 z{bAl?e0xfk{X90Z8bTjK=Ia)}5xlne{?%s|jmqrY&B)d|hk$2gK)i$(GOBDTQt1X`ZZhbV0NOT$;cCH8zfCnd3!S)`QMbH1*HnGE-qMD z84VYP6LF4>5A)~DJ-cFJY6p9H1OdC`SlWW8dAs33mBMoml|t3r88r%B zYT3dNB813Tfc!Ehp;`g&cR3eG>u=kZ_Bu~4_5Au#PmLCoNa;8rjNe!1lyPOW&n!$Y zDJK&LG#A!|&mdN_r(DiEUSSq$X1k!0iB=K5va7X{9nBg`3lH=h@>fcdpl9Q#l}{5_ z3CE;{!Sz0H*JEYP&5$+Qwro&IX;za>Tk^(zN{yU?ZY_fI2LYI8zd0~5r4P`V(`_=| zEi;iR>PW9p8K|h%IaO;0Nm*H0Y2Mhi8xOCZG(IePjer`>ZKtTDRJ9(BSc6S!<%!Cd zY;A_QEG*RZYq8@`!aY?2c%1MWXt0=slUteMq7nna0u_C^OAtB#*)x!^qxiXWy{|Z~ zeP*AsR_ci*RvK?j^u_9W(hq^yOoyVjB;+Tj8PuoJ5^_>rbxb4*dWa=@l=h3&H~_ie z)V7o!r){g#5w`nWdXJVa(^3xnVqH!p*E6P8i<-jOMc9ZOOU>gwkV08R<}wQg_`KY9 z`Ms!<>ol_f7+mzmos51g3Yc_Srq_pUs}luqz9hi;nAhh6MJ8X;yoCPnHT>D-b%|{{ z9##XN<|@4NDof(8snTZ@^@PVR_7+7d2c(URvTM=I7A$wi4#RNAtzGALKq{aOx`81g z4OsTiFfN>*PfPT;tnx1YQZ!y|p}c#_=A(8hbz2zh5(k$>%E4{2XUrib*Sp8?v@-xj+M15T}xp&#F_=`<;4>-Uf{`&d(l$uuORR3Xl2*oF_yJsH7535d8!<<$l`Tn>q zjyo-`Sf@elE*_+?zB6ttGJ;*7mY&_k+=X?u6x!O9r?y^^ zJ^N3Y5*sX@J9UT11;6wKza5KnQ(5xz;xiV7loga7b6ZF^SAIHEdb2kJ*G&Y3weT8( zvKCTQ)}Z-MqGvqeK3+{_!X$}!C6Rg3!7X66hS;60QUB}v)5AqycSyNrsf10>aksPS zB2F)sQDSHWmut3!o^DTrPkl&3CYzZIjF!0dM>7V|Qrs^3FF3~8WzN@)sNx$lxQZ3N@ zkeOLYf;ds46&(7M?zKoTTHZvp^yc*k$Ia~=HC^(aVv+O@k6&Q9a3YN`sZs&)1ap)S zet~_aG;pai$^vOqpD)@Ncv8z|DmL=Se;&KHv7_HwqW70D8$jHmY>_CbsV#yt94B5({<;WfQAa-fY3usZfwSap82KjTmvO;WLpBW50p zY&_cm@{t5L=Xzp3k59F>bF!7i;S|t9Q0Dgb!E8IU>nRbDRPYQ$vDAx4icgc+TG00( zWg{gfhZt~(;`z`2(jy!eEx1R6hyd+N6(JF1UKiBwv~V);Hhrh!$(_+W4dq_vXbB$f zL+$6C$+Arl`|St38t++>10W@Dwu{D&#gi%R_lg}pBvO&`^BQ`lZJPz zO>27vs)hlclY5411aNcP67ZX3`U3t)Ct+_s{tHpdkQU;k@LkJDn@uh?6d*}mWtI2v z-gmL(O8U$CF%>0cJr!6a1ybdcX>;M?9Hutu3Qh^8flA*>CudG~=E6ufybJl^PxrzY z8mt1 z22;fu)QZRU!`=)f@g$c5sTF{*9~TC||En6-503$u*Et(1&bmb(1}bXGu@s2lu8ZMC z>B3&1KnhcdOcSS*OpfK&i5#_&R4kIck=)fapT{-itIf95%!|d#rDrc>& zJx$`?cC1?$Yj(C6KqOTN>t@;>*k+TB=iJv+8q};Y8!-%^K$G(EI6Iv4AZUiyq z)3Ma8{iC+w;B2`*UAuF~F4*j7aW6}g8agoB4P2T|t+j&ed~)PBTgrI;aoz>|d=^en zJV_f52>&dAm5SMW@ttsmsB_#F)pvSrFR_ZUk-Rcesytv%ZlcUo@Oo`kZhNNUVLpis zD6nP)*baSuL}5+FLxeTe@{~mk0ywoULj{)UK59qsh*EQK=!~#9CMS>}Ojwn3 zi@geHw!S~Vu^pfJYt?N2bL7IgIRrp67DgLT`Onv~ASOhdW)o4+ihoc5Ok#!ZxvSx$ zo7OS3LA5_p0A~F4XfYluK#MIPsrYuM`>a;!3?)ba>9gD;bi9+q%s0PF#PX;9wyoD7 z`nc9HI)IYQQkPL&cUPrDqcjEeR_oR4i7pYF1m8KA@~VaJOE$?k(NfL z$=AaZjy&-adguHTnL5RyjgkUziR)-f^U`pBU5jn$?UCu~Rt5>iSh%&#*zx@6wrrt7 z1MYJkuQ*Vt_UrXB+Cx=quMW4ylTyB|kN$FsC56uU28l)N(YuPJb4`a^V&DWOS8%Kv zjrCkzj(gjkB*VS7#FHKNZ0e)I>fKjqk%AZKC2B1^W(nwa(ox`hb*Hq z;_#fyQo|tRKNlgKIaI)&bC}-Op=))4yFN?MGao9eaxdsEroz{0gJo?xsLMe$|Jsa@ zR1)W9kVRsUcUffnDSI>ciM`5ngdWiu{Vq#(#J){NQNPY=b)A7L(X_23g`SU!RxYDf z_am|Pp~sk|U7@O1zlwRMtDqYIN-~vILmCq%`vGs|Uk1zZdjo!iv<7PbzRfkfLAuYU z4JKGzW|$ravdi2QL=58ZIq4R=wZb5*xkhKAzQ$vFd;3P-(rBovuf!zvR0Y?uF9H7CY?8+_09sNME!dJ=AdvvTSHeXBWIs}mga{Ejv{gKlM3i96b zP)mdKjWUHR^67dpgfslwalZSM8Monrbc5P`>EX9obv}nBD(BjB!?c`|zP9vtQ&F`v z@iXo64Qk#eb-Hoyk}g<(eRS~_sAMj&LQvY6 zvwHruNtj?qX)r9NFeyedx{o!BvUR8w?hr9^BZWdqvY5%4%Q+JFux7DALV+e0417Hj zf%mvaTCjP$8e9O-emH8vQ!6bYI1(nK6OnQ2&`oO zb`*Y|lXgV*r+Dz6Z~!M4H-p)~6K-o*FdF32i#xj2Ke);`T6tGE_DifFWq_OWC&(-a z`-G75iw2vCf}~`q4b08`f;efbV9+(jUQu$y+n=&fm4~}qM;)`Y3B7-+v%q#a)RRwW#M0@ zthF|kV7G9gk1QXjy`_+sZ4J0}fZ&s=|HOWc{P*>|5c{ICt6FVwhjMMxf59GqfzREa z%#}c|qNp{sG3dJZrCWwDSUqw?8+vc$BrqCjN4tmAPO!@b4 z_;q3Gen5qTix1EKD+(ULqhaRn7Op4aUz8O;$Cltrpz{)lbA zTpzwlWOJ(I*&7X1>A137uW0t~YJ68`YGfN7o@5&N1`R2Y6hM}~*$#{v{@&ex@g|R8 zFaTDw>xYPH9T{5&koii(7Hs!XVlxYyCowQC`ab@va%1BE6fD*WCNkUt* zIA^6axw^{3M&ZQesZ;of)#Savy&@k!`r`4=-|zn&ryR1k8)EfDTWhR9jFkSV(gX)e z57~D-f9K$h@%yT|eBFlK0+U1&pLi_#M1T)AJFktb*>EYifft>vrqjZeLjeDTo`b3!-%`zW+@Ad(AKuF}#eJ5B(a&&KsOP z&X46jzF()M|2B)BfHchFLyayy`qE9qD2nN;<1%mpw&eQF?mQVu{(6j%hgecoAogBo z*hj|iuZANLIcvrDei#QE6NFY>qLO-4*|}xDv@-Q+emP2G3?$366e5p*k4mp6tl@#~ zOUgq_(eQu7>c8n7#E{)uL-g56PyZ3L{~o~q_MRD7QxOt=&EWseJ_6HfEe4oV4e%j* z|Ltr4?dQR$r9WA{KQa2xkoxmcFBOzfL&k&%lc6C^YzQn4DpI0dCn@2^<-k8A1 z{y$jJ&(c7$8|$CR4fGX*KiCwIF>T-_J(sDG^4G@-VO6<7@c!Sw6naVi+R%tecofZ_ zjr_)87xgMC>OdQbv-L}4#SaeIQgejY*@xF!7VGzTOHLTJ(y(yU&Q7W|IZKN|8pq%i)Yj$*it;S({)?W>yyeBuDV4P@h`Z z+fPW3iQ;*4vO!N0L7V^e4FWZlBC)w7h7vK;3hKj z00^$UcXDlSL^Qf(<0Ph9MgBOY7>LEgrkV8fWY}{k^G4Vr`OA3TEs{QPLaaDVUld*s zeWE5_;~+Ax9OV^dm1dRw(+hy>vsJbayf-YpVZI)$fr!7~sD02r+U;SeHj7w~h|@d1 z5AV<3_v34pota@F1#-eL(&5go22X81_m|P0^kzKK=-g(FLin00lyXBi^!Fk_c8fIc z?TZIkyDF)fwCaAKe4qF@KGEpSV2vBv0U_@PGR7P(@iy6Ax7vO{UhX+~`1apTQJoO- z7u1|Viv)b3C$<@FQ&x8Q7U};xv77%X!v{I!2?DHG4?!*4k2VH}7X0TJqaXI4qD-OE z8=AFDWr>nu&O&#b{Uedx6OdnX+RULt{PUCAov8dL=m}By)a=-QHeFyKjJC^sA}^+_ zB}WZJ^MzO4Vg_9uQ5;PMa^?~%7L-w+(Z5YS*m~<|a=U_C_P5$~US8kvejZ_v zz;&zr-Q?+=-Vqwqm0dIv-Q56;n{&7W3C7kvca5z_)ye#s zKL0TJ{zzt(<mt_Bit1p;n~{R{rv;(T= z-fC%|!0)K*-Ze*yZoJcTZonppbY?7KPA9s0>#Pv7VAWK~ZbH{ZZs}WW9L+jh0Z$IU z+KBQ-P@ekFQtXEZo6LDDvvBij3v@kXd3Z9$#84pHqN!Zx^BmAYGXjzjg`C3p`BP$= z_(J6|68gS0=E{N3g{@h{Mc#zKBR65Fg0xyAo;B8yp1Bv}d0%yGKE;Z@TFmB_e|+H! zaygtAi4X7L(S*S2)O^`y=X z$UpM2UjE?(jZJ=Xf&(Tq<1FJ-gL7LwdbuN$&3Z|JadFBXo@<^HK$~Lo*?GF5yYbmY zw}4u9&B#`ah8J%Ej|AKey5h%}E(u+aD2+Fz@484Cwu4-zo5YrXrpAD8j^h!m4`zO+ zyKLp9L+#VeQx=WJ@?M6JCiOLzEO8B^=`maNwX#epMvn|;ai9LNP%3|8nlNt8>#Dcv z-N#SExbNlQWpV-NZYFCNY8J;&!{$Ul_6zDHQN z9!X0;@>~s2tAP@z5pgj&Yv=NUGgh4Rx|#TVnzH67h1b_Pk6Ba!{*Ixf-(u?b ztV^lMgQuoPolCULBKC^oq?;NoZ)_|^3mYuie=cJ}o+3o}KZgO|UPFZ#)k8Ri;y7xd zn;fIpxVFz6SDYJ+cv$Xa5RG`(q5i7FE}}k3G=)j8suztwJnD9-DugW;F04$0Gllp^ zd9)>F&uA`~BemI!*&f|KTSv9;BywMS^^oyQJ4vj`}IJY8r7&>#}w$sC!0z1L$uhS8M5BZ6$R zGW#>PI`O;7^rEYbFU_uXc?!-TsLIQCU+44~NB=U+vMcQRQtWMX_dU5+VD? z3-k2njhFC08`&rOH8YWn8Gl`?Y64)d??NswbkxpL%s|F6+C}1v-^R%2b~9 z)}u1ZMsCBQRDSWC{)2`B+m#ZE7P z-9Nstcg^JxpjuBDKcAjD#Tb7B@Q$yZw-Y%~L!(Tp%>-+|BKLEfXhDUU<*^N3kJX&I zt-ei?@u3I6`zA{wC~*{CIyrQNffPsxqw))=pI$|rOT6OC?&;pE2Ml})uU}_6F5%j) zhwaZ-IkG5hN{81zxB23#d*gS1u`;e17>7ed1ZI;k&cI&ruGkrGRM~YO21#$8qPPs{ zn|uj+`~ValZqSBCa7ePk?H_~)1X@FLUVzycoz-hA=Z9muR|pF218-U73^WE}l-y#U z9lVQ1JUT94!ZP9v;C6eXZEC5zTgrf-F8?P&<*CRpT1bDH`@Uf~8yVOPWhiee&Wcb$Cb(*atZLB!dN{|37$Z#i ztm;`WR(KL;HsE_1{|7l7zBLCi7i(ruDZ|8iw`#v3NW7K~&KL|Yc}wz`mzWvw`DMK; z6DZ4>D$xs$Am){Q&rMNSTwz~pl-%RlTxSUYJAXx;a}Km!d}>6 zujE~ytQ2FJ%E?x-YM*@PzJOn2naE;P+i0b&!){%vdJD?6spyuo(;+ZLkogm;60ix( zR3nBhk4IB=Z1d(0TWDll`)VM{Cj!5XC&lBTq6#rwuV>sHd7yim9 zj>^HP$`pB~O|-+$_jQzC+qp2*bXFFOUCXA4Atkg@n(v12?0W(x`X1j_l^29m3iQk| z@bl*_>A7BTh(uHlo=yUlrUsLJ!no+P41T+b zbJJzS52eB+QsP`&Q0t;(!b!ZcbRe_>^>W1&DiSE;<20T&EWXh8vgG=A5C1(-OX z9e!TI2Ho9y}r_t@>#zS)y_AHasG@S=;M2`_rTq^G!~B&tW=o+)$lZfP zb@D*hf&~+FN#O_$<|0s;3o}!$77RI%I(gJ9rx5K{A|A0pjGdQ!FxRTi|o${wcc z4X}rpJ7m3QH`GboULm;OK+GYVBABObIvC;g5Swc`!)R7*w|L0c(A;r14@hF{k|23^Qp(n z^jfc}PyGv06ziW;#jicLsVnA_a2! z7mMn4B*S@NzmwU3uF<-ovJjt#M2q*QVX~PLQF)d7Xya2<~?-(@g#Ht|<}P z_+rRnd~D2oF$IA=Z7DmR45M7lO1JR2@55wdwcA&0sARdueBK|MnaV=yv)M=Hhd=g9 zCNYw7aN|*)Y}(4;P?))WC226X)&0eqyv6jIiM4rwZ>?1xs6=ZdA4AWXj_yQWMNkXI zM-2X_|FT5Cq7T{JT9?^sXa3M^hK*%BF!CYNV!A{WP~T>B2EHLY-#Kvt?>~1AGXt7S z@bmxhowyp(avEQMx3+2c5ScrWR-EjkQ?zmVmtNC)De!d07xIg6w%$(uh34cb@1u{D z|B21Y^t-Q`a<@n+Qqk-XfR}Vf#2#*?4PnRwrnM{AD(=3+Ebb5!rrT&Bd&uwox0-G z0QpT+BDrF4wFp&6mUJ5DfJs9-)yFO^^>4_EKISYaV(M-t-cb~-#suxh4W{Jjo65N= zr3|UL6ez#pnPZG!en|*?fVSr^zoe>7LNS(6z!n8!BsA-HBBBI`>|dHDY&g*=)7}J- z4NC(?hpQ5&G=Pe%O&T=WUM#3SS@W>)SZA8}@>*+hgZpeay-A4gp-=nqUVo8sjEjE9 z!E?_U1@$O;ZSys`uz}j_T$=IhGm7e1T^Zf1{E)47PS%8+beUJON?hQ~#~S_@weC=2 z_t+KhH9GubnL}>!_f}=~g$pXKnDG&8K7;iePde>oe{GATAp(giCbB^LfxnVvJj4jS zD-J|U2qM{iJ{XXr=5tESxvnqq-)upBx`$b5S#KKTNeQ43?QX91M81Z|Ai=r_ht^nU zO@|w`;bZ<8eSDx-*dWLzC!}*C>~!$4yEueuW*?+*)Wd++ z^WF`7H?cVkQoK^Sh&?!1i8`xt<^9g+<`6FQ0*@POT9y1BKEg!nz9=j|aY8N}`Naz7gX#aqH|M8|7 zn(dvm*d{x%D3(J9Fw!4( zShn`q!faNqz!f}YIvL6Q`qAOc35;y--(wxDNfK2`oA7L#!U11?1@Zc`c>N39{?{A- zA7lUn>-LGwBheGqvmxX{y|5DPNHK+xf*P!*V&dK%77=Ym5%$+y{g*~Y)*E3ErsI<) z=Jfe}!UD(nv{mZu&Kn%6q5LYw^}9ae_E+e*9FF-SZUTK{uMqjgk0&GGNYhum2>TqO zjXz_wq{IN1vpymF-&qeJbO7F>IG8k5jq(TeE~$R>Ada!gN8QUfiU$LNu4DAD%{`n5 zhsX8<^Q*7GucCtQS18>gs94z~FP6R%Ggw~=(>SB_q#C-v@0!~am^Pxm{xa zu^`BpaplsioBYZM&RN4`#}Ks7SouZ+EsSDdH6RC|CG=7d%F5@v71MIKrBFeAjEca? ztV%s>y~nzwmCEm;SxF2EIusMvU2qFTb`l6)uB1dK(zSlu0e|E=Hci3kvQNPsWKGXu zlA;6cFpl}!Mj*?gn9U65v~?>+8ovlT73L0&UH1XWY1Ed#y&){W9RLu{%McuK&T7*B zv4|WrDJ@Wf`c!$Zq5ULh-nOd`ym^MXc$eO_P0W6KyQ6Jc|@46`dWsstS~XZUX?bOoK1VE>UNO)tSQK& zl!muQf|zE1>sw4=>;WN1kX);&dAnl#I$0R^(@u@3Ir;8k*v{n>UT_a%xJF@xwh`oO zWrQd(WdxaL$>prAF??RwTke75mr|slOYkv^Z+~&7BcoUViXI)?%yj&yk1)pXWd-Q} zItcRCnBdRjQh>IYoWx+CN`fDvGH#uATp4>YYya8?@5J`)aFlB*YsXORF5MXJHh0aM z>Fdzj#5Dq|b$&y;Qn_}Mx&#`|FQxAu5=)%uh3~Zi_-Y=jw;XJMk@#-9uZ@E>Ot;`^3$Mb1lF2S-2DPr9^9|{bwcls*qe~qnJWljk3N4hz%xqL zktRF-aQ_xOlCua11_&%)IU-re&ansW#R=C|{rm}%6m(?P%d^J+$H@HWUrhi8ze_ra_TL(C|GKVQvhu?v z?`rear8of35I-1T1e#n`zq@-3w8%SdLSryt4+wvD^MMnvg{-Cd5uC%+vR$nDVgLHi zl~4k|3s(GpeqR&=EYM!O+Gd}Z@N70&xz2;15J3EabFby)*%|VXdd%97PJy zH{B1mUpOt_B9vXd!s+}vR^(}Gp zT>HvI^k)(C)yNRn&=<_(_ML1n?JBkn0+%Fbj0QK0`IJN+mVOm&j^J|XS^N=_=^UM2 zXn~+~&ci()$;e@_;D@458G`E+K<5TOS6GF`*eKZl80{~wzXu=sWZP2v78y$b-Ejbv zb73z}HZ?O1<1DjZg?@baWK;g?c%8ka%5-RK6)t^Zt%xqx>7-Fi@~WljBIBDV%J|0f zWLj$KAfSbKf$i}9DJ`E4wbvC7l+gyAYz|qn=|`mTN=HXW4_!8M_X5^fnm^XquV{01 zhhIpqW?KMt?|TT^LZeEzeBGlrLGUnHzi+ES|5rQE{kVJI`|US{3R#ci8Prr|nQ)Z5 zM_9wfM1;uFfpm~69gVW_9RcjL0@+kv87j%RUPsfgiL_}e<(*%4<^Teql!dSI zNOvY`vGKxv{|1cnl6R;))&D^!I;;z@EZcC;$!@o)?C164OA100-T?|fs;&XUk1;}D zxGB`T$xl4jSyu5urs_fE(1QK%mtQQ@>RZX9MGO^nW|<5ojcv#zKUdHd%)5IT-ncjy zkMflEdoB(hp196IYb=2Br0_Tgtjr(0( z-LP7zx@hGz0G4^G19b^4=1Z_ZyGs(+Y=#k^@fBM|@-geT9H|7a*O&`Be6FKOla}5) z-+Zn+L*8KiJyT)Ge-J0kr5L}IdJaNoq*5m_5JqBx<#KdfGAwweIf%YHWiar~<8ss; zIySw&@CK-zF`FVOzm2;k`yJ%`pkU#)d)7#smX<1Jzo9zkOudZ(XQ^1O=#m3Fd&}~n zk64_%5Qk4F2%O^Rq#2%vt&uyEg>qTC&qXnomQr_=i^P|_;u}7tX2n((&-ajq=i!EH zjhzkJez{omNw;ZQM{g|cJF5;heZ%uD-S<`kaL?ww7=9%ehI=m-Pp|CKkFINi2AsJV z?uT{rXfBbXFs+S2gRVRB^;Z{3+t#$(CzJWwVF!mUf-Ps=a}!@|C>q;#;hau;73zj; zAkFdCmKcR9Iq3Ye5B47BTVUlX2WyWDCigc{&qx}}ns9x$zQ1T}8Bdj+qw~xM4ut`L z80ZSWxEi^bEcG3k*@giu&fL|Xxsazlbkgxo_}~rEhlpMW3p$kRud3mzk9&*0e9#~) z4_QU~It%~OWQv5e8vSQd(~j}mz1uHYqp`fcOks9pD@7~ND$vFCDGx9PDH#}$s}$)n zN%gJg1ODCS2{4_t9G-@W^fz;t`Z)W5oEEM%W)#879oUN#^ZEPR>|iU zl$pu;kdlCy%ksoC(PrC=EuJ}*$8&pddMPuOMt1CCJXb!aKR$(9*K_^s!SwLe_Y~UM zm-i&=5sS%W?@t$-*kPg#U_^^)kwTyC`53?;U@;i5goK8oz^cY&0-sU0NQleBY!hqW zrMWf)QYNf%Hpo;qykcn!-jkh6eAqi!7)(x#_Q8Lv-MptpQEYuvTj?@xLGW;ZSYNl% zZyGD;oVyN#{DuOC?GUk*Z}_Zub2#4duZ_sP`-2J4E4QVNO8rHz#QO#I0=2f%QSJq} z{mF!$KD^l0xEs>OVI)4cA7tl3O)@0zd+)BkuU>>x@p2ddI$j6SNQUtAx|Xs76l`>z@BxzB+jWQ`yVXE+wEXYfq z$4&_4_@I3;XoTBqgUOC`8hfMsoh%zYfj_!bnzPKd&XY;`tWUxlG{|h~f=vd4P3E5> zr2hnj&F8cCncZrly@Utu@p(^fvvH1UYE`VAGfTkh52!pBHuh4Va27DAfm6{kw};By zTCqyt!o?#=;H8!kGQp8GyED3JghzpKW%7$v_*$vS#yHYB`$HSih!3XR zT4`?t8a*}>~Fw75qI%*W!9{Jn=6;fETzW$6%SpKj|jx)YUBO`O9$aTrC{8n?^rZ(F&$5} zfKlY2Y5FtOs$G2wxQhaMmv%JmR9*A#KR&^~r;?|m$hy&1KFrx3%1>~Q3=JKLA*k^} zNgtHnYRfm9Y>>=dE`KqBMF=li1=-g{Rq=LEOcy=5>u!vFgdFtW8A-Te{0K7m%@n_WG(0XTt$CHR4laeBcm z^|rz%XYbOoY2fo+z*?6TvW`dpk^pH0EBP4#QnLPznJFhgRmX+za`k&-Y+a~Ta@r2z z4(7mp8q{Qu6|B2YbbcW%46E>lEb8au;hp;Pczj$f7@V%kJ%bllH>{4PdG!v5RjoCT z7Pc{nSKs=|^Cz3vU1QTAV-_$TRGW5{MEQY$PoIVSz*=#v6`H=8N^tW*u46GNAj2XP83}(C~zy zN}Cv_rL<8|xm2~lfvO`q%iI|fU$7)?A6`3XPiFPLpgI*&tkWdxR9*(K1Yg+)XMVWv zZj@AFfg|@9YqZdx511E`Ab$9e=z!@jw20y5syTP$0^F0-bcq(raLc8(e7LH3+QDcg zvJ@|~PG#KT&=*=(9OO*e&g|<}X?gF> zm=@zehb~G84M+X1htPeaAiBGi=;-K3BZgKdm;JG?6P5B|E_z%?Fh*D|n;{1ObyT(0 zSpN)O_jD_>Qfmr%Y+LVM)RFdB|G*in1P*PoP(Nqv6dDoRI()LpN%x>)i&%|cB`Lpy za%UloI;PlA7owQJa zwKxTuReW;IJFs>Z&OyzVO`TOlxk*1-Aixx(ruP3#B@`?*;KENtA`E+ADe|VATv&Iu z!Uo|vvoI{0A!pNqrAv_yqyD^QV=xGdt;(3a zYBjmZ{p9+dO`&bO2qggl+`V(1_|EW*LkFxB+&9Z{T_iaLN=adCVQ!Mt^;-WR4el{c zO{RNG=79u%^K-PkiML%%yHM!~Z_@T>8>B2IildD?mi|BDzA`Mzb^BXsq+38_Xpod< zXi!ob>6GpqY6t-Z6(ps*LApBx$)P)?yQCZ52i@nK|Mu+j`Ms|BFwCVh>sj}@*ZQsY zUbfLJ^~}0vJK|?htx=5LjC+d!D3S)8pEn5=)uZTCzKN0|WaL=l)#C5dA_r}AsYl6` zuTV(PH$^gY!Rs;EFe6<)v~H_sQV=N* zPuvD{{Ok+)G)7clg&-BkW`;4W4ZL6gr;0d@wrMA`m8%3L^FTRN_h~#;Q4};6Tp(+^!&WK3c231B4at*o&9vIqDT#HG~L`7LZ2}PFEW(060FZGUc?(^%`v=gT+S6;OTpRETPJ#87|^-uQLORC8=)qMw^OW=S273m?Sdk)y-a4QhKhG@39o0-uIXc`>_FI@M2L^TU-PePSvrwGz}!L>IG> zT3I>{a9>r1`zqnk|vGl1HM~!;de&m+42bLfk9xR zX>lS^60=H%J6LLVSk(z{W;EOlI%F6xwn~?bTtPHOd^!hQ-)?vrlQr%n?2M)QxU98F zf&yGG1@lI5Y|K6#Ya6r*J5^w%A2p4E_6_wLY%h1&wF_?3%kF199E1%TbUCmt4wgut zH#26+2U?|K5KsFmD3q5Kh|vqo4qPnPtRW8e4bp{e*YBhBH@`hj%8~CI`RnLf-oP<1 zFtGoOo>}RQqb9x^f(jvb>9X?gMNMSB9^rO87ryas(cHFofaYoDN2AwT#zdR`+!Vs01v0uNPfO2FcbH~y9i;p55T;Z z+I0ZBWY*$}NsiImI}I5*Te|lm7@^0G{Xa*pmfEm{9IyDG3oQ?8kE-fTE=M=@hB(`s ztei~mmV%e-NA4jXb~(a^{l}>yvo=u!(os_7oR6%sFTWoVhl52(ZAXaNSJSj9$gWRJ z%5k0()Zq=s5j5?*d@=jL z@2W28U(FoijUru=XTq;>P}sRV^4%A0Pfh-Ti-42uKTaeHyIy%67=&p)zd`RYZROct%@Sf3yM1R%4_9LY(R&c{RLI`gSl_tS-v z+|`OSOnSuzzmW%SiZp<&BO+BYz54=&bJX>h%N5i74mqb*7iT{So1){)FdD^WCi#6KiyIGz&yg zZ;|r77hQ;7Y{4YL@H@ql3)`EkGSfS;+8Ql=3zds9k*~{KyQLYO!A&gcMTxoP-xD^U z5+!h3vm@bd^AxbwI<22Z_cHJ7-}u0)Y-UO)L?H~ibrsr)Q<0fpMh=%6WImo^ zu7&UYvY_GQG9uYFY+RJ4-D%E!~<`SuF4$; z_5?cyy5cLX;ysAAdWYpmd1S#o#X%|GzOnRO%nra)GYh^Bs34_#kN3&!YjYBuJS|zI zI??R1Ue@vPncgmkRS6U{AO$)_*+|9tEm35_vsg*{!f~smjf&~FkVD_@h&z$Gkuc1K z%d(*+V8`=>?+3L(6m#+wOK@yq|I?dgV}xVkQ3M%3ze)~T_AMq-E)I)I*0E>FH(jT9 zJ+r|UWUOqFY;YE04#k>d=!|=sg|c$=52sV4Zy#}-m{8defkO7w^Lazv%XPWFl<8L z^Nh+jE}TX(UR>x=Wm$n@2~XROT!q|-oZsI2L$Tjj^e+eT7cTw%-q#v?0X@dL#c2GA zWaF zZUqqJ!n+MQ@h;1FdDUndu=$kw5{vzaOF@z(iEFJ7V7&59b-H1UNMzbXY9J^{?u;UH zH{<@U{nHD;Uzjlue4o3AP!GZt)?sX+ZE_rNWWtBSQa^_QmFTejjUU*HoMa!y;<%cS zyu8Lt0*gPe1?C@_J_9a}s>&4igl0*?94OcAl&ge!=&c9|cx z0t7|uOr$bj>XD)_#N?zchOuO1%f}M`1t5MoIQJm%->(*GAinU*?@ieg*#(x09f%Ye zqelUhm|Dz*ZSHKvRf>$zH`WMA667ux<)R|-tn4?ICm1cP9=Jy`xKfPo? zM9;%Sf^q5Jx@_}|`n)d+*jcj{je3FnMpo2i&RlmpHfpn_aQd>G{CXbiVUk7I0-ewo zCB8*H4%oUHwJLHD3#l7FC%m#`TlwJAT*Tui#vKLpu#lbV)PuUSVShy$sj0#Lu>Ifs z|MjK}1cX2c-#$(Ks}gN4bD1vGVC0DCy!&2k_|qJNge%lCrY*Wq+%J{}x6OLWG)3 zjo-hyaVQFGT=0IkduK$f-eb4)DTS&J#T=iL;IKLcA>GDoOff@k@DY8VOJ%wKduIdf zT>m)(B^l%rB^W_kv&wYMhVXr9C-bDVXHRu5zOYZQ|!pG8AFL9ejvSg8H4(w~+XA z(Z_))?*|*jA$7&#vh+s^c*4Y9FE)owE+?Yn1m%v4p}MC#avKjf_~;`}u~~}x-%6$x zAn8TTK*Mgik9l9f;@>VOH$0N=yp;Zd>Og++(e=fBU^P=Hgj?e#uetQrNGVM%iXd;j z0gjh9(X%OR!%2r|Sd&*%@TjTBfcIWkyuf&fj>~!cnB&Xl_K#2Tm*we0q6?v3M=uB% zPus~^Uz(^aFD;S0G-;<&+T`v2ZrPEI8lH<<9M^{WO0N*_r%_vWp%>LNQW7FMjP$al z0!~zoS8p+if_%$raw$sCxyCe!{L`>fUv!O(*8T)yzr%v$1zKAcd_7waqTZL0Tzer~ zAc9g)XV|Ib$rG&Y^~;T20CkbSPv! zEe8bIOS+nVEU#ltAHpcQU-#}H>|P%^G65V&Jhs*udKVGFsBSx8kQ^z}f^ctdGe<`v zSQRL`Q`D;?duy}j2ZbE9npG&3ne>Ujm77?}%E^WsnH>)Z63Gr+3)z2MO=Em$vO3fW znCuyE(?}~kH*0-3@ZxgUrqAa`gfMlIeZ0_wzNXh$Urht?2C6E}o0KCMS=1Fx-Z83Z zdBi^vOtP4y9NAZ1{EAg94(CEC5*U=^;SWKj^VO}Hlrrbd5~zO5hE9+0|n-(Y~V^E88RyQAeze*yU~W z3_>2bV`mEQHu*+?@(YC@&7yCDc4;?c%}FZAhUXtyiu6&Kyha@M5W*bl#o-Mw2A>PU zDF>$Z+iD@^ag;oD<{3pAzOu(WhWirrTh=rynPbm~x-thMg`lz&jVUL^R@Wr^oU1oH z6l9HeTtfubM86v4oE66{$acGrNr zsk^T?gVjL(?ZNt&%1gmgr)ZYVODIksg6}b}W(dVkm`sebFYR<27bdW^VjRGpj?nM89vrUS>y*d5oiBp(R1{Wg`K@#icaNrNTk38@! zAHDW>cKPVN?ZvkOZULjOKhr;Dr2N_bbc7p|q4_O|=<77WK}soPU9YJZFLGr>X< z%P_RCHmeVA;)+FI`?!0yGo*+D7EwHxQk8J}VIBu4zYg?~0WGuNNhw|P795uNT8m?X zJF=rP4P0Z>F`9OYs!IM9eiJ>2E9|s9f6Phn!0E{zv`^u6Fn#$6>X+iU<1L*=vf(MR zBT(hFnQ-FWwtBXV_$9`AU@9`jE33?RI`80mm2DV#q0P7UkB@gbHl<{)Xrmhvyzi9@d6ShD@B|QMCV#dGKry$^5~}mVId% z6aMa4>qX+e7>4R!2k_&=H_?C10{Abw0MOVK^|_%}<}ai$B({L0>deDQ_4X0D$TW?j ziIiCrZ*Nc86O+Tp{{yA|uT2N_pbjqLu99_a{Z6XIQVx>kIgcx*X2WbYlj*=2FhjNI zRr<)3f5aldhf4AQ9HQSBQA_#t3oZB$=d`RcMrnJAR+a|N0K|`@~Ge>62}_ z-(hJSz44nFhH$eX5g~NL3 z*6+v7MtvgQV>_5L=pP24H~3AaXsk5iK|-k{M)5QUMTj}aiUq(OP3{5dfAh435Fudp z^zEw~&pPI7&p8rw)`i!myauRY+12dMAD9+0t6yo06x{(sgl7eJOA4~D0h zw^ZXW$O4~<-ieWm^N2;I<)l)Du>$v0f8w|K1rldCljT;$)*AIn@^s0Vbz>htek?l; zg*dJq$ECtbj3gK*Crfo*idAGBKV@cTO*O_B=m+DKECC*pS@8S|LT+J<{oa)oV>H=b zZzVMWs;Oc`S&;7}Wf)ymJZu~w?VjMbPagR5FfSRHXcE5u*#6r?co6~YRQpt*UoN`V zrwJNqI@hv&>KuvQkl#G<4Cu9oS+O=zl$%{M0rMZD0gvm#3C6uSm0vHwYn_Y#OsC6A zXg~x~A^=`-&^(=al&4ny^}5zQp<;JZOQP|*Vx=!(g4x}}qmDCwNCFt~JKIV|7mDd4 zRn)wy)tQ%-5^jLk#YNTKp*+Y}P$F{47RV)-9gim^d^AUVnZoFMj69!o3RNxikl2}E zAOC=m?hmkcXh@-e=I`!}2tM}tY5x9dey~X_P_Ok|95iNCh;`?x0i0N>>Q@~lp`_Ea zFD%;AQm)5c&xu{`A`c%QU!Irh9n~QL^HX}f{K+Hy-aqP@T`gaz=w_5Qi&*ikf6A4} z#@c++@V&jAuN-I)6YEdp1GbMcMb6eb@wc@(VB2y})0R(+R)ia8Oys!JVM~#Fu==JA ziL_Y%MZZUL>@JoZ%r!^3BvkVxcrxc@0V?VXa=_n7FCNe^`a|p!aBH^e^wO1~=gFO( z>f#Z&BF2ZycO8LOY<)nPsJ>e3_8a*0#UpltKfbVJuFbHvs@+Au52X{oZI?U4pNHt4+D zK;NJ_g+W!y#UA?2070W!?`dH6YP|}>WM=`u^byMOcL?dB56n z-?H4SZdyBV=1C2;4(D2k5B0g0m+fcwg~2?|!rpnJ+DfD0cd^bDZ#WX`+3TI5l z4uxNwZFg^)j&Vec@RfQ|W6J;+`dtJI?9UtT3qql$rtZp9*@G9avPnFG5wiml7CM)L{xL4x?N=W!U6%4)PVDw1Li@m?kD^h%I1RqJp_)m3d$*nRd= zN7574nC97I;%H9U9Uq;KVT$2C;s1ci<`Lf?`&GNN)bl;GD`C!K&LHk|mFGwLV+lAV z@fKk2)PBFy+~$STs!s3)6@N3oa1vX$8{H3|&NgQ-l(;_{l0U)U_5``PU)!3=sM`D%S2FBKrTL_A{&w7IPzn@}8tX3nbI!deP|&9+ zG3dFE@r3Jo$yEw_QzqL*p!DJHfQreII(b(**HI?07j<|$qRDX~1-@UxWoK62`J=o7 zztd&#&W~D2Abmr-m3Uauju!o54mvW!u{6b!4A?W{dy(vZD!lgR=|7Z3-B5l*S5oVh zgj6}#(^xdmrAz|ikRDNG(p*tmT_5Jat&s87$M$tptzTq1$p?Cqk zqAia?=lX_x)k3|Ohb!GwL=ba--!_`K(>tX=*vNn|1AI}VE}#SFP7gG1XJqn}cg2=( z9qUmPq(GlgV!eJbOyyq;>vYWlivj(dir$we+6+3GCFRc#4%}C}&B!A;&dXY52STFO z^+|2MLpFx>^n5OChFIx0gv>Fuo1C+9sp^5g?K1lxltZg^I&$-s)${y=TwUXxP02Xg z>hV!jzVy^B#4Jja76t5o3KMmz>|Ai`gDWy|yO`^Xqk66~yl52YX_Q%_zv!0KLA8K} zj!q)b)2xa~#=F8KN%joPJD$H!z;3pwaCnvd@^ET`oD}{<-MLN+7`3zg%BtF5WD-He zmA_20>?hAutyj!3sdL5z?d!UqaJ7f=u+_$iPX1+nw~Or4A*xPvNo}B$Y^BvQ>SYX( z6Bk*c^?Y8T6?UjJ=qn1sA&^Hu(wXqM)!`#Dy6pdK z;te>m+pzhjZZx(%bDhZWr=nSX%`~Nv<`b}vPTWu4y5EmGGTl_UT<^s3Xh7PsByeOx@j&dEL#HNQGihoouY7ciJ7#!>&4- ziw?Ge`8-x~zcA?nY+FA_I-6q=oYrTkE0%4#zG%q4vYo~{+#TKLXm7lmnu%*IKu7uV zIcj3Othc1}GG6NkpuDpjqh!2;#?N(Q5tlV+3xEwg%GIK}fXMDUN}u&h=;=Pnk*${R zICx2>1uItw35Sx`!Fcb??wTEc&ifCSNP9I;7*eB7Mv0`}=Wp94BE#KW_vX*46TwmR zM5;v_)oGV|vsEVbQXR=>Wc3x1L=sJuSi%m?;R>)Jj5XOWOooX(=CdNvcU5rEsziu9 zYk(82mMId_W$-y-rxf6BG66_Af`*dzPw&8Y5O+{?zuu1QS*YL*%IS|>v#mEYvQeB! zQO+v!fOU&I%q@QzgYAG$Xd<(KqgXn85rMV3Gg+l#Ig!7L13j&fKI$->U#EG@t;d-1 zS%ky7OsQd@VADR{kR)Qj?Q>TYBjyd0gg!s6%f_slGuz;zy=IkhEjn~fiK@=lA6y1Z zU14H&MUd?$h-}xWDuC3h*+T1hdlu-^9Mi)1Zhh!! zx33ST)Mi`kAxl$bdc+?G(TI;ucD3Ds?fkOYJ(;rD&=_ZzT!Jhs0Rjc zuybikaO68l$nof%nGigSCVE}Nq`|E=F#r@e73ZzK28J4p-<7`9v2s(y?ieS*H-AM_dtDjC(wQH}1||1Vb3b41>goam z?iHHd`p%AeYhxGfU5FWqQ3c>24;TpM(Z}V+b`7QsjUYEv?b&K`BxR0M&y3=D^lh{_ zeM*5j1tZLLz+{4m=y+Q2g89yXpRbtuNF#s`n~d}t@N0TJBPy}2ZA z(hL4xuX0swniStrKcFsqcXOpbRd=C~nZFpFNpd zzw^tu&u3+ zn);g;JTHt}sf+uj0Ka8JqN{qIgH6Trs(;sbbIGIwJvs==4VQ0fcQ-w+@8`HKn zRF{MJ86o}szbY|o>YMl>`>|*9u^_sTl%LasJbLb(=dFJ0Pc?VJl?1weN@j%PdFnGZ zYpH~pZ(Gz-+5gf8cDXvGA4Rm=D)aNA-pEicdA3Q1iJhX&pw=S-!M zfc}cRojXa}9Y~O;uvY-ErC^r02-*@%e2~thUpHJk5Xiv}hpbkLnz`j-k}*H(y)@Nn z+&V)pO3okSHBzshbo#t`zK6HDd*Ma>9INcT>mWeQ~L9bxlk`} z*?N(0a2+?B(cyH|*oOZqQ3(H4qR{T*<@(j=#n}E=?a!{M8FH6L4#z43u2?9dXIJzi=pf|vx?GQGxj1JX|y%skH%i^Hn?3M z5jrd)kjUgXCPTv~}tn2*uCmr~v7#+!E=59(P49D#y(YZtvyl#v_WQNyF3jiKG` zeJotj(bIstTwyUUBN6J=kwqPzU%YR#b8`43kuh}cEK@blJFV2>GW#T9m9p0@mfa|# zT&FsiF!9Cjl7NJQpW_{lhAO6>ENxHSWPbP@!RP5#M_*UErcfu#a6U1fW7*%T{19yu z54&*{U=l3aS>w_}rkaP|z{ef;b?Pl^SKBbyE&X7I>Jb~QMNYh+z(mi)64xOhBo~A8 zJ?5BCO1lH{wmy>=a~HGuRG$Tb$_IzWFAVA(=k7Venj3{8kX^3O25rL-z+X@Ml>0Ddut>EZeSo$IV;aRbM}$*Y4!6qGe$RLMp+4BQVi za1N!T8dXt$-pvM_UwQFZIkxXydzS%skz%OrI{uU%1qt(eGvx|jC%&##KCKI$T>Gqz zkt6F17l-Vp!$Brsj{*zYZTC3doD|NEFD;0(@~j{;Yo`r6edpJAy{lpj`$jN)eQi6O z>)8q9#iP~CXl4KSaTN$i9SUN^Vs=kur+@XZf@9Iic}fsl4J}Z-9+XjWpH;~ zlx}%{ z*&P-eOTVpd7e>oW66DQ5iA~yU{n(T}yBp(kfHV+<^)<$TPJds)JKX%Lv^OEA;Ils_ zRE|`Sd>D-ICiH<>&3C<&^7cO>SxQo=Ak!{jjGnpEeO;$qJP-R=8L&P2)3Y26HK4jE zC+U7Ku||BQC5e<%SyYu)-`A<0fnI6@WH`rui3_ZX|No_7owO$gTz_U2K@kXBP(9DRDpqROqK4#e0(8D_QJ=_PcXpLuw6IUPT5Gpl zWH5-=W%A|oJHaGz=}vEx&1AG9%9}#`Yif>P?ZZnx5uQW+o|VHBR>ND}qVMX>d{B!w z+v+9;_YGSr#@VcDN?W+}=IXoV6bK7gw$&u;Gj&?K5>kE^W@D)(ldYCsy=C&q_yOv7 z&_k$jpxwc7by5=4O{bcE*?k+i3nvc7bfGx(?1ae>8&dl;g$|dPT892=w!RD z`$pE6>oTE%9xA{MT?rjnOOLPnkk5xhC`q&5?sEYN!GJO2C@Ihi!p_a>$S|H1qW^VR z|M}c^2QRga<_cXC?(go`zi`*@xZJM*V=&%A#GB{d@Ff?2$O#BLCONpWbAXw!Sd=nn zPF(5}f*&W}U!W2;V? z&1}4<(3*`Ocn(huT?)Y4_+$vm!~H@b?r#hfxFoe9cf1XO);61)n-TGQx6A1JBdWih zM>ZUQ_*bd!#D56=74Bo#lO4j-LNcpgRo17}VNZ#wvEkd2bgd?u+wB?DYa)I*bM$a^ z&#z+jZMhDAOx$8fcSQ>4I;PW%I0<8gRUE6CVA1U}tOKbz+`wf|F^oxHk3iuDb*ctaIXDGz1 z{124#Cz)>%3a9`L0%6j`{onHSRt~i{nM<+`vS&ClU8)8A44*H+F}^gSDWrd<`NMNK ze%GBZV?G7pz@pTW%IncQl`_lrXF$V)^JEcm@B0D+Jagiq7FuRzIc4Ptf6*R+d*Tf~ z0biTV_8jMb&>}*Ql9`#=b!Y-JIqq7|v_0`Syv(;dZ+HcKN3h$Wg~fQud-MjAuch0p~j^$LmRllnVhWxZ3Qs*T}H9-qkY8b#g#)V!@ooUd=9xv^?hSf*0td!cUIP zeK-3uy}7oO4iF$;f=o%+B^; zUK>mkAHM5&=x7*)wt5veFJ0?s^4S?^Rt&G-ubCM9E=}&g3HfTB?Ga!iISb6{&OHr7Ujb>ff{;`VMrE zMppmu5(nyOnHZysf5y#oweE0n4X_FiMD5O0kxsle3(^u3^K0O^W!ACFx~?+L!y){1 z*E>G)i2K_Y9BJ;fg(Q_8PU!r~m(wybY6Chz?kDqjG>HCLGC&WD5iG}wXvE1MFL4zC zvF2f5-=Ai^N8!Zz4ubc|s=6a&KA%Rv$3~@>3+_8NsYai)EcL$|B=i5 zKmdn&fy<5&W;b0UP3wL9abp3>mVM!Uy-2s|?v7PF1=ShX^**-^nBe6{)cV#@j?P~m z_wzPTxlkFS_9Ebx-SK<*aIQQavB#O0)Ao({@!k%yBj(%f4bI*rCy|ZzFObo z;0Zxp=eXzf@dVZauoA`uq@$C?Hvu{tTvvw*5inpEoX9xE^YPXa5lT5syZ^{Dw zm~L^;uaJ$J$;}Y30P5fP-NUUHJmum4QBD7jnE`LYoKOqCDZl9qC5r^QXll3|TUu@% z=P)K@3c;1Lt$oQGO|tj_XEuPwD$JTx=Ny5fAha9p`Y1YrQS)mi zFbWB8<3{bk@6+K$PD*{&I_j^IQ@~(xU&a;w;qm{aihZ9*igBH6KU9Do@e#hg zmC;%9-0DfSKAOkrM=*hmq2gT3rik}PTpH_b`%=Hp#q2v>YZqoIJf?h6{KbhAEHHgg z^Tbb}-HKX#uUQ?aXBIT7JL8|ad$5n1LuyHl5z%WtcSLi>x+7CV1qRFc z=sJz!X|Ve z8xGjWGyBpI8rgsKwf%X$QK%lyIW5QKNc0ft%dtoMlwA&I!M;R`BV%1MsU+dLTF2T=Dlk->%7@7FO-HeH6z0+DxEin9dZ2+fDOmdo)?EEbpc>)=k7uP{F{7o zLa4AFQ|?jEUEsxlOPwtzUdlu>Q9w_3qSlxbV54@;5e@F}PS?CWt8@U#&i631?kcpB?w?7Hr3Ab( zb6#!Bi98#ob^M-=?SjDnYO4|y*L?rBr^@g16#Gtx1lbcJ)vxr;cc)MhOd=b}IS+{O z9_x`OSm*Y~UA6UeiS*WZ@%Bz=63~`$ekK%l5wtD?*rA>Ga**M7}O(9-AkKqvxv#xIxfURg{YVHyyZg3 z`S-YPFNRp924K({K1ae{ZBXU*D0?DvytG-5y;_jECpxunEKrms`wGHCD*MkidPo{X zL-P{phhst_UnKRiu!jPW67pD#b|`M3EsU=UZ@VMItQ#U@v1R458jGNC#fVy2DZhG` zYxL%4Dt{IvK@Hdtc9aUMdX_<8?Z?n&YmJ1y6MEx2FhSX}P#z5-DoYy|JRiuzZjbnS0`LvUta<$@EdvaTkDtag11f@|0hJMI?y(z>3 zSWe%``^3l2si5|M!wdrjxR;8|b8lPjF#Asn_^n>TJ;XV8UNTc|8oiL0OS!BR;IRxo zYVcT%??0pszEkv$KZvx}X;#`A&t+Ye`4Thy5@tH=I`6sPTkhEYY^ae3^9k<)I@(Hn zy!6?YBC>*`|dBBDAW1v#D_ADK5)G)naAF=ls z{i-bKp);&9U1iPM=y4j|G`Vkfe3ap762&v`C2rerfwVchh)XpKZ2aKVMUg&E+j@>1Hr=jBi zM71X@Pd9Kqq6Vi(j%3m?^9!R+Hy3?+XWhkB@?3rCv5JlCRvPR>p?vHZ7>|N*gl52584CGRxB42O1dC?Nruh$^8$2?2ii_eW;U&o2he2MaITc|W5xQJW@{n4G@omy3^j;3c*ufv;X zfCWv!9i(Z!hc5j)DPH40;})s1Bnsn|n?(ix^GgA;{o5*4AO7ZWI#flZYsUI?cYCr%Yx6QF)af2ixS;~zR7<6AvT+jw;xAzQEe zY?U#Q4(MtQ>r1HWlEf2=9?p{K?o9DE=QVwUgiZ=nl3|a=mwo@;0R4XeTR0{596!F~ zC#@abm^0~}6FzMKm59I@(hOF4XuIss}#dBMc zj9>nI1N_@l5Rgtj-9R4q``ZH*P=*m9ZcY;frX>MwsdIW<1Hyw(If$9$RUWOr4!nI| zCB7NP%E2F2WHZbT;SRP1CnLNchO_Xu67<_-?NcEFqu!vpFJ~ok_Q^}tBE5+UtAVS) zn?`omf`i)0HV2L?D`UQkT^UM)xVbZ-Zc z2s@{15!AUpGGzMOO?ZLktChRKI_O~jw1W7Olmu-xut8HCfyFOGv>c?{BXtz)ckRWu z&G22p|9Xn?L*fF>Hv$wVrm@#gJ4YzUdJ|FejzmK1#f#MN=t$EV6(4RmZM+R)X)A4$ zSy&+3K*&VPAL!=2eVWqKRbU}FcCp;&XW?(g=X)HZr*I{?`$me7*}o#R5=&QZAVx$o zH7^X4b3edC^ha+cGgp|w!^|_^)9|&VhqYi<+s@kPGsX(c;cfJ{Wo0{~jQD3CGn#zI zVc`m7U(hJLT)vF4NbL`#DL%vhuqyrr^2o9|kNN$RGgp18?>EAQPHY1U$3d^abHK zhU$;Xe*kZ61a1|-b8e~T%S^B%dS4U>7EKXLb|HmxHc&_2gV-x0pK{KNNWn%P&XtAn zsN@7j?+-ZJSAK{=Kt;(%sSI##@jx7XJX~?fd|FHa3E6m%-R8`8rL}Pw{MPV%XNQtm zk{R<7*Al@Zp+`i^6e5*`zb|Ggx2-uWxnwZCprx96lQl6yOt*#Yj#h`|c>K-s3I8kD zYss!qMLsD$qJXD?$4}j`4>={Z!Js9&4aN;msUuy4vG)=pm(|M}Qe+RRMcP4)$a(>z zr7jFBFhKScXX!#bkW^Uvpm>>>0XAV^%Hye_EA{=Me2PXN_)==>CQL`(qA$hEw`yh~ z`6lbNTu|~_Jv$D0dC_EfMM)AHA6u!E>Ygy3kp}HFJeF{+7)3$-sofNV$ANy4QTrT3 z#{&O$T2N>n-aM55qlguV0KZl|IT=LRaPJy<0-tcDL+QNW&Fsv2oGe?jtjO~u1-{VN zS#f%#10+Jm52oLNRp^j?*jo-?AW7U_xSJ1q3cWJD2EV&|H21JJ9&;yKL@h~-eL@%gn3$RRIzYq{9jw6ve;=DS+#7aa@Iw{g;F5DrES{(R1tLyq5Fcxv2sw$`?Jfe=Sh$bIfu}B7 zE8p3P0lQd8n*=By5&_EWuS@Ncm89AZcWGR8l6MHqk7gO~hRG{8m5XOz;-+Z(c$F8^ zF-hX>z7~#r1+g)N*Rw& zEm75L+n4;}a%)zadK#CZg#li;}=)7mhNnL zn1H{VJlqs*iO0O3bqo$OV!;(usX}$EFxibfas972t^1pbIcx5}>CP1dxH<$4g$;Ho zWct2XmgNlijAuUFMB&Xq>#E63#qcQV3T!HUV#xRIkq35a9$wyqQKC!AmIhy1tZa;L z$1xdt9*qtmc%cJtc=r)24^YkoZipY&9Y^L^OaWVU$W|JX7XB}8hbL(KE^T^0gO3aTu|ah z*+7qZeRW(Tyc_|nq`iec$cxuWx#)3tZJ)C#r%Y9$sZ<)>_sO=BpzZ{ zzs$C?R``o8OW+4y!wyRanndd#71g}H!czze9LDqAum-&daogok50uT_Uj6Jq_?j!- zzc6NbO7qPcC1H+(4u8JTIP4N?dow55^tIqDIt1=(H?Ypq(zSP7$M~+{s#ME;EvJTb zq{-hrd=WR)jR#GbF61Z+W%28Z5={kVo5RxJHk$YyV%1ai4MTHyPQ2kpO6OGG*>YTa zX&$uf=GjL-9nx37_Ho>v-K2Q?H^1FCPkFv6WYjNl{s{}xSe6?-uwk&B6+!}ewsWHC zxItObAMR-jsb%uY)+r>FKfX{Nfr^B3^^Pq%qV@-hEtc$@h7#OA@cNl1@8y??e}#;4 zfdISgD1?~t7gqXrAWR+(l~=|wIgWpj#>d$m#E~rvZ*ghPq3iSFHPCMz50`gjf_==7 z&j3)`JIeL+V~E&Uf|Td<3zGbZl?snll%1<~p~4N>J}& z{2z%H$?_403*D-8`eXWSX%TdKW;!5DwD@M9nI`9Vxyj;fvgxjE&;lJs@NJhE*J^{N z^tgnteB$zQPL+tIX7XVqQ|*pgjV=!g`f;D3Y`%)Ik1wlOs>P`42j`n{Wy(iagCN-} z0pp^W37`EHK!xkVit6S#U&f$F(oChlcq!i9ZM)AQ3;z86uPm2{JXIKD4%dfWU`o+{ zk|Vy4DO2IkEiwANV=sM-B%X1&lTa37ig`>sAmE>%R~*!OQ#JEs%CgU4fAc^Z(Kr2? z@0IB@W8pK^4Uku5)>8QNVHcq3rV6WAVj$8I6kBc=-5DZYO===r=1L<2A4r((bfP&p4?7p>}@j{6sW~A>>BeeuERzx*^1JiD{12 z_qNq!YL`6!aiD)B*$=bDEp@%XzuD^zo(@s4{l+fomKC-hrX_}ZJZ6PA`s=gck_{c?mkC$&`DCq-ZaTYcpDRwk zacB^_k;Xm!9>Cm@!MTZ7OKjO6RU5D8=%%j?wzB+ZGtzDqnY$8RzgR%9!y$ZUa#q`C zO8}&4=I-(coN+Q`Ke)$(bw-Itkih7L{;eiz$M;ui2_Iv>-2Q|a=x_UdeU!w3G+gfH zGqo>&b0QjWHG&n%5XR%^6~wYA0>{38dtuI^I*apR;PoKhpt1R5RXF95W~|Y;G!BD` zL@0$__63p1zUXTX+?6&n1u32`wVaM^Mfx35d=EZV+KitFo|ZQd1!9{Kqz%)qD1UI+ zkGDNycB0`G2A8l`UK5l<;y_b~Q<&A&(SlBrBN$uf>7$&NU-g>y@FZva3n$$0Q_q@D zmrWD}GPBPXK8FeMm!~RfxTMVNj5A;$=E9#^NRcr}f)qxeujZ*P^$ykwPF~raxVv0+ zKI%}j{}g=|H+Iu`_%BJ@Lfv5fvvC|~eoGBOg18+2@{o{Ig+XN2#aLtkIyUI?bz(Pw zt4k;~!t~CWt@e2Z+Mx-!#uNP#ZyKJ=k5+_SWI*f(k`Yd#%}CjaRy_rzXakb$H=JfW zm@#NHI3G5yPfQ=_p-3bZ6ZxqE5x6y50+=7C#lTSMnY@G#`9_P>$ImjoGR|JsFyo#a zLQ-m~1YADA6!q(n8=E8_UOw4;MQE4ZkCb$gz8_8e z1NR+o@27j!or9`BpbC7OSL}2Ld5(gYQhXzp* zX&6#q7(j+@hE74np&hzGfnn(G_5TL1v)@Xr?Dm9yJO){)^)TWO_vi+fv!W=v~42@8D_qik7}FX{a*>?F*AIZ-Dx= z;k>Cb@-15Q9ntFnBThdFJUeqETXLp6I-4)|2+qC{e6m1V`tj^kBWl-z5+8}4E947* zF#YlqgilX+5i%#~sL7MyIV)q~I)k@)fEaeVPn<<`IQFhAJgP2^?AV;*YlRQ;`iBu$ zckUpn>5=dCi#MXwiP~!~1}wRE7uKI)j%O9yn0OPD-L%TjpP%kKZwlt9<<&6GC|o`! zdQ z2_?Ii{EEb1DX!GL;+d0El%z)%-kbIP^7pY0mE(VJ85aqB)it+P@u9qtJon?5L7{=; z+1^jyi0O3VEYy$Z3r|(|d8v<#r#P_BrUAd{!vgK&bdwIoYtD%d1MM9?KVJIsY^%3s za6cwvySh^yrs>n-W5jXL${qR<(1b3uSs593-}GR?ksTMyg|i&hUz4C^YH);CKG&F-QPY+)5Jp>3nHxq z^6n|eS5OOA@zDehg(NCCz-WroU?T)DoUtY$^GtJL-Kp}}KIX9L}BtT>y33oDF`Iw)0YD*D7R1Dj%wXWlaRX3V;;Z*)m*><*~*4|Y?6)2qbSm{@D!plfNI&;ghsg6FC?)EGeML8Z6 zohF>Mu%$SB88cNE`SlQ$_wf|Qv|MME^x7v4;KYhPt@{N+Wxr$E@UU&PvvQEdYEp9V z^NP2GOVV`lKDo__|E(PPTS)>iExINq9vFBLRr*J(PFQ6KzIojI_BUHOi_y?_%5d;OMKlR>?4||K6 z!O=OBk5?Lhno)|)xk4waLau#*yVSTO=8Zdb(tQv<;TxpGtxx&7DP+yXi*_5`)Hb!y zh}V`d`~{^wB6t#b)t=x_Uc`U+2KeBg{oh6zG2=fq%5aS>VWL`OXSVRY>eW_|B0YAU z9Gh!Xiz(bNL5?B!H~^wT@txWYSa_? z?HO&wj2W-uxSTTG?sF!p1ctmJk^-K#3n=uNy?>SFFJAed`06R;d7}}lJ`(ep@U%*W zlcW7{tH~=0z7mIRd|#P&?QS!x^hK$szi>q60FwhUbl20J=3hxrb88o`YrQiXt`pcX zqIA*mJL_#3^sW7K1BZ946$KdD;jOoR2(H`;nFGZME*mH;?iBD`un00313E=sEM`if zv_|9Xrp13L3QrV97C-uY{LG1qg`WO=SrYgz?)p8U z$XQ)j{`eQ$u+1+HUFcPpogst$Gbs4I5fFlXwwo-Q{ZBCSmk+nf2~UPki^hBwRXvvQ zvKfzmdeTE>G3rcV=e@@tNzj`-$+SLeh8T0Y)?o>L^$KVcQ#k9H3L(}@_yc0SAQyN+ zG2TOCpRapZq*GrYx^ZVI`oih`NMI>qd%8!+Z2FA?gbcd&?qOAj(UMCQr~X>fn3I>3 zrTE#!$~8THr4TG4xAw0M&+M0z#XZ0)_jsNb)*U!iALWKZiRrUFfE7p?Ky`JgEr*4) z`$Ghi8u6ju{JW)w{3l;Fuyw9wiW{+#wbRjHal#Xh5AK|owRW?37tnk9obhB1& zB1YXP+k-_2%1>SPd5oSZF6#&zj=<`G4VXU#FB`l_ww^Go{_%9zH|G6sLY}6Q*>uF2$qMRcCAwOP zQnO4POiebP#ug-Vq8%Qfz$74`HDKNy_fBa+>?enjyMIZ1_r$m#=6Q$R zw9|8CesLhIS_9_0gj~9n@|Te2mn?rK^|yMtY{6Q$c>J04xnLYnJQf)fb7oE-5tDH< zGML;;nZ>GU8}9$7OCYguHbvY~-NIMOfPdnK1yRzYcA$7eY_-;C!q5D5vaoNnb!FXp zo7g)NrCmqIa@bHb1X!Yc*2}D(Wo(OyZuMcg*vc6{NzUcU%{7;ZY(3+&R3L7h2Ie>Y zh3kMXZ?zM;Xk0u_EIa(B&q-o7D&LfOwhM8_UyL>?gHL{A!=T&p-mP8j<=x3-Itj5Y zH~?{ah6s{nDky2?scF&pnN~5fuuysQJzW1*-;x1sXFlsw)tG(D8}$oV4Y%ogn7-iF zUr=f}Xv_jzF8i(`0owF4W;<5|q;REVDuhBO`Z)&VLIgX- zKZc=jaL4O_QG8@WG6jk$89%j=_)E@fk0Dj8Yhe&J`UG&z8`*>3U+XOjo;Z(Zm0_D# zc`3qAE_#E=ygW&We*ceI08u%hJ|ghjxkpIp6ADX;D}uDaH-d)!QG%R3o-b)?OZr5I zI|*j30xg+n>27orWE7_Hi-quO%Z~b^D%c`DhhkCvJ|yssOkYW(XOfa5dN81X?PzP9 z)8lH{tv;92m@n2is#X)lWKL(x&gk?n!QjA5qnSR;Q}cZm2JCEO$!Y_uk=K00^@kQ5 z^C?t;H)!BMk-|MQe?twv`HPDSQyzO0tusF?;=u(q0uZ3=4x8!2K`F6A`LRRwh(hjn zujzR-*(%PM`bsMX=cE*|h*GTPoj-j344Lz0esMS&1+;(}hs^g>BhHL~hv`s{;yTv| zfBfM^DMqehS{1;Q@O@bf{Qxu|VyJ~CIaU?DB{aON2ned}o3&Vgu=jb;s}q91hYxh; zbqER^ov8XKYv>r#?@OLygm-ovnFv^( zzEaAjMpJEAe45+T)`qn#HS;ETz8qct%v|RS2 zV2&>FhJ_T}nt}ao`pack+f5hDgyQJuX;x)!{1HIh8?9Zr#imtUXRZ1)!gX`k$7@F( zXl~@p3FKy|@c60Sf(Hhh&pmci;J%ajWT}f;fJ=s%)2}(zQK9!XkbL$@^49}Aud`;d zaB2yQtq(w%yeZRx3gW+I@>JoXr#a*1tsp99JRa&K{n1RFr{TRTLt5!Dx-x59 z&8PaEr{}lmW_To!1fBk9@oxVgrkiQmkqOkgRxJbtfpaBP3I);QoT`N|Pi?S8Q{ShD z-8j}e$;(aZqtq`Y_#?)3tq?CM35lP-3k9HIu2R5{!DdvS>;?IT5DAL@+C68s$mu%I zZ{5#tGx#$AQ2I-Ymvi z{%@hzA3Z?Wt`*Dva7muLw*1y*pu;jUk0`0T{2E8Z895|>kh5gr?^NDfmfn7Xc)K)Z z^WC@n0`Qn=9!p=<7lF+Kzq^v}&-okPu3`qs|5ZChmR;t2?k<8bcrg}GyEJ0 zQi#4YUNtcuE&Aw4AOkASHRPQ#`*cB^l=F?PLbjgvvo9l$M5CEid(Yb$9ptc+QarLS zipZpNNl{24jsK}i`!3z@%`yulyW0Y-&#HT|{lSRd#ZkMt|B$jq7uo(ZIqSb#a?Nk2 z>%v%guFWXp+ledn?FSAD<;qMikdjH=&ey%=`}AoE9cyrQ6N^@p3M%i>%QX3@ zi7U78_0pI3kt`nqqA?vQrIs+UT`!~V4jyd}EbIBo=+B+M{3!Jb0Vua;Z3N-I z@B6)xqU-0JTRA5Sg6gK?(xE)1lop>GXPs4p?ronEgLjbr8v0A*l_hkA65V9z)=a4BQHr9>0Ye!&8LT}gB7vE!)~!#r8A|5ICp$I_GwOhe#c$~ zpRyz^as?GG)Dy?cIhy0kT5J<|)8iL0=Ln?Vox2fF&)4TdN^tAdIYPQK63-DBF-)=C zI(o91bcf4fuOF)m{i!>K9ofwbUkM3da|AHPh?So;n4FS9p z8&rX9N)dV!(-m!+a_LZ>O8$~OGX94};9G=wtLp;>pCPIB;07Ru>Vwd zB&~O^Yr|A)GvKJcp`f|c3R&|ByJ`9iFdW z;{{%{*PigR3iYw#Ml{!_rL_ZE{9S0>mTb;sJ~2R$@HbB9Dn)p-ofR$qZIHdv?v9xK z+zi7#f_*~fhD9%q%%M>dfvhD@TGJ$cDEQL&Amzk*tJ&(v%(@)ENN~L=5-%PR_I)wY+^<|q% z9$mh)Dn%t|v#}^~SOhFcClDZ6qcFVoXVJU$>d`o%cl+ikxIto@is`B!81IAgO$pN^ z$hxjAVDb$)mh$mj;{s>S1^5*zNxMKue`bFt??ia4%Nhel1(nR=^VEqJ1Yh76b6jFw z)3{e}vm9z4{d1PFgS@z!Fw;}C4y#D?ELaxiA4L0HV850V^U$d+O|65oYG&Y*(Fz*U@r80Y4}pt84K z+;X4H?@W1)utv`GDaB%@)o)Dt`+J)5T=7;PKS)G}Z8-#ROXi3-lmTyr3O!PIHmyvR zZ`BYfRekKmn|if%8BjBbz4J?g$l|v`c_mXFq@X{(SQKVTkZltz;{Kt;I5VQs_crR1OM){uL(mS(gW~G-!=y%Ql z55Dc%+|J5{3}skniq4(M`kzj5_R$}Dc`w#f|3ZS@C+PFc>qFqj_Hw?#X+~l)M0-N? zr5%lV8}%{Ya`)APb0xA3IHRfAM#4q)4*G3UWJj5)NW^GRK3RH_RKdyz zDpYopHRi59!zd#;4IbU@mYeJb_c*wJ7tt4Le^I$H^8F@fw9k*b|9f@6G06Nae-Ev0 z^@l!khsL4CLgaIMdy7HTIF?(!ES)T2@VX`hGic`PGB!z^XiTmudTlb4pw&8L-S_6C1dP{IYSq_vDS*1e!%Jd5Z-bf;ES) zR_(q%eypCN)fum2_hHZ2Ob>#esgNiW{P zK6@)H#FWWsDbXMu!d{jeVS^Z~UqfOax;lFXL8e;B|Lq)ipA#OvZ4njm+3Dq{lDC$J zVM)t#_k-gmt?|Ld`6F`x-wGIyD!^cSLBYx&s4u5TkAj_DaHo#a{mbFD82!zx`mxxj_<|H0`moo3vqXRSzkrloC)o>VlS+S1O{>{pnIV*M z1xw4AZ%nWps#4FD#(%j_(bT{lp++D+&HMCHikPK z{q}2|?8x>U#shXtJ(5i%gjxBaJeSwrwws1B)c>O(BItMwSlH9T8(5me@rU#o7XYV| zj6m2hSC!%BKV9oL=HkfofmjQSGOtp{4`MypM32OTxeG@ZV`4M5nNC&Rfhoa}4aLMG zs=N`I1&XwA+Zq9256 zE6O{UH&+BT@AmIJ#B4Xv?*}e)N5aG7%}IAsw}NMUtCvza(6p5Z7YF1FZ>#)DQuQ90 z*^0B8vzQbzstQc|Lnf9QG+0^w0*8@@w+f7bs%UOuc`dGOZ3mDfA_$aPdRU+sFUFr@zF|gK$oZK#aPs zsv(O%;)Q{_sNx2J9oE0}RdZqVS9vkHGKC7V409$Bjh`~O>;u{SmpvuygO9~B>HK+_ z8PytQNVqRYSQIRJE~ZYd1_hY)T+&K4QC%a^$2ANiyCt(kE2c@-t>N!4WURdiL=Fw9 z)hLW8vs>BW9#aP8ZPAlzLFx=AX|P48(^&Fo8)xNV#gj#+q8Phq!N`u@iWrxmtzv$4 zaANxj+NEYo2xC4~cS0S51cT(SD&)!U%9+HAS1zSea*cBsaX|FMmaVK(stUA5XT8lC zk@hfNLfL^IBd579aP7i5p)Fs$CE`z+jqWZVy34THEhkrUXMGb79>g8{Fp3RYY&{M} z6cvx6qBVZppRo?wbofineb5=D-)S53l>shmg=iUZo)M(Hf!NFv+UE&4Ny_(0%O(fD7`ky}W$CS~ zZIH)f_gVun$me?X&A;hS{oA@KQ6x+tD{=u)q}O=?k5&)f)4N1R43ybFMelR_sx!1N zz4cJo&|s5qcdpvI68a|k13L{4I4Az>oSWlumZ&`vt{XQ6^~LPLdGB7u6i|FIiMLSr zT)kuhbb|z%4ER&C=Y76Fa`@B5+w?(4;BrMCvH4(L^k zP?rKpWp?km98+Ir+H;`d_fM|PBQe4q^|>Ub#(}JCAH^&6a-Mq17eV^=TwDp!h2Sk8 zF5Z11os<}I+bCv^wnn|OEPiY4*Y{fTw$j?%1DL(eAIv&L4*9CL>d3h`cb=-(F6TB2qig#stZF3-o~*`o*=o5|o-lNjeX99|ssB%Q^HXT-XxARPy~<7*QI|q?_YQpP!uk8$$2?&4 z?fDtjrT6A)bK4V4NOp<{D4!&v2ZPDJO=!a@V#JOFpM7|JL+ytLQ{{Nb!nB(q zdel*25qK@)>wc0;88*jT7%jHxbvh}5tr)bTcyW_&o~JY{OP;ETd33g>_imtI`G~e{ zeCw(xM&-z5WI@zEB-FCh9yVV{&z11Lm}Cylxk5Lv`cCN*rJdT6H$Fp zby0ayH7CQ$PR{mlohZRtLy%pg6w7*YaHVu^D48So3ly}}2yzgqXi#nh>)Q(ow@Vlu zE7;tz=s(Uthq>gw*d%!wqA+;)2v*_}%3r$$e504EQo)xXmtN40zwNi=~jRDD%FmpX&SqHJJt6eZ0W&=SALov z)o|*nJkylnm@c;xF`vZi3RjoXiiP^)NHQiByyROIR(em=EA!jj$)I_y-xh6zNae>q z@xK}179%}kR1@mp3`t(}z}fa!j81P*VQ}@-S5NBWLQqxvLq%j|Es!d7IXbV8&Dza- z!E}r?ONgjsu+S25PiKMcaT5#8KkHM~uqKJN3QtVF8%Mog1bz3Dd!Q#@J5~DXxH`#h zE=~JP`uhcMF+okB7C8pmkh)-Wzd2ev>MgZ0-^*oBxudvMo)%Rjz_(z{TG&Ch%}?Uj zx0!`CxaCPL0z;@DR+KY|oiAE{9GE}8p7X}BH`x>KIgn

    m&FU(VZ ztn>W>j7|FO_7;c1Gdkk4VbG}31;yS1*lcHX?sAQx11$1rMklLae6wPF=fF+xtDFHi z#qvZczBQxDDD8p?#Svq8k-}-@65V`ExtOFY&kAYQ!`CCri+WjKV?DvQnx@qW$mRdZ zC4cn#m0wv)Na9I}RU??Sb53iDpycxDe5IdvtyYQ=Y2(K&8MaP4jj2NDZQ{{KqJj*C z<5Mh}uPB^*levAJn8u8d+TAaSlp|MELY7o@W7pZS==PXz%5P1;&g7V$H2!g5gO|sT zLLJ+|)tFs2yaU@}tf!x?L0ZTsQX9c)_QcUGvrlrO0r~0s zQguW5*g;}(W=LJ)$be`&l-H6s&?>M`hT1+&T{XO+R_d z*4vWc4e^SAf9TM&GE-&p(ko*|C*8F-H&iEThbH6c$#;i{4|<;uSNF|WhQ6z36;Lf( z38*|d%|o|4x8k~5Q5vW(j7bO(Y+huE1k0ojr~{8tyQD6i^GRUZNbG%X?#by|1uB`t z5tojIyGYTygP(Ft3N5Lr4B7uB8hVsTvB)gU;xl2KHsA30IUCqqB#m z=QGbIqTs~GfVx#5wYL=Dw?%`0e0aEuvtsyXA3)5%UD4QMzHMMF~e3% zgIejZuUXVz(|lH1tkuQN%x?g!gxY~%=j8bx?$lp1Fx$7*p0BoHAHXTZkue(`9@^1M zXa^cl4~0NuLz{A}2P23lJX+Mu_l~6!MrCe{wVw4$r?qms9E@jZ+NtQprKd%DXWV?GWv>oWkK(9TM~=EED)4r6NulWja`~q$|!f9fR!S~^Yg~n@nm0*og^0L zbTx$ji8KE(+WKem|L?oML5=?(y@xF-f0OJ_vReipp=W5GKcoL`%(?J?QmDUqz%G$Z z4{{g3+!&D6Iy=!l`d2Ne+xeu#{}dWn(H9qk_cL6Iw&IiT1ah26V_P$3GX^unP<$B* zD0abm-s-vnR6lAF6@t>X9hP4Q$^e!|gOmH%A75}Bx1g-i%%rYw_)7|&)BV7Vy17xO}4j+UEXh%x}>`##6W7M?4$eo4}#lKR) zbG-_jVh@cWi`mWPLCz?!*w|n$PFGJ^1`GJ|xMe(}i(@OXL$%6hu#4ivwH)kQ3 z2ad5t27QZz568KU9(1O_?Q8?;+4kjQc`anS`LNg^#l^+F>-e&uHHST+(&4D zdVhPO?5I~rRp_k4hsWzC@Vd@YB>Rtw{BJR9dSpAnGc26gC2Qh35Z4nkbEekabk{eK zdx3SZx8n63yy%i}TdhEa(iQPSrDP6_;GS?1wQ_Hl$lQ^VfidlZ#jvtReOD>IB>jly zfoV6>zdS}GA#*mqRer)^cs5v9js3u@!Ix>M7Q3&Wh{$PzFi9=L(fg!pn!Lp-;Yp^$ z%7aNS@Pk4H^aUK9hNV#{vK;a>e(@S#<5Yiq$A*?Du72K08e2xK+v4t9!_~f*E}l_w z+*QizjNLvD`2RjUa{T|`|DV5%mY}Ke41Tb3; zOH)!@ET%4|68Pr)SR7>d#%pF>yTXh6+s@tcg9;}FsXB~O5cm#Qlt{&`93AHN;t+hI zg!EG|&xi+@duB|%$7_z3PfF3#&}jnLS+V&5>Vdb-(aG2NIoc&Mz>&W(q9)XyzwwTs zU3}p4?qz^f3Eo@$7YKZwnv#{f7D!sHrt*`myy1l(f4@0r;1^0#@LzNeaykFwXXANa z`Y1e!=uR)l5BwHX@H2_T4AWvjTCDh>=jtMC;@Yp$0>bp>K&4{w$K^;4BW7F1Vfi2H zBf9zq8}?r_KJ@Mh3How)blqp&$5V;Dtbhx~ml}Hs75!N18#WPrR7!%;*FMN{DL{Yd z;~2UI9}1h?iv1ScD_t$se1qyr)Z4y*UA ztr9m>$Z;Zkz(4+OEv}LK@8*aNqFnG{teiH|aH8@}Md5Elq7y|K!ATT@f$|`EFL~q7 zCXYN-Yve^$5*fG4Vj+6vMZf{n!uI2~ywlZ$=G|HF%9x_NwQArIN8+`-nTog{Md!*k zzlO*3qTZm^)yKlI{-=$)D)(53ycETxVo%ujp}EE-%3=2X&iOjRWKaMxD{G@7<_uY5 zR=RoX?!*P#)GN2{wh<2K;3%lu&dqyu-(Hh?)}R0-!kDZ)Jd@@6r?0v@jqhC_`3*5W ze%hqMLw%e*G`FK?sFT@hbAEni^F0iK_1I^dZ+-V2vPIQ$>-P# zqF2$|&8&N(a2+ga@Puj=+f{(Apr5C_&8=;Y0-OKU2{o{P*8BXn;@iH+ry*Aku}b4|21DmpIJ(AI8K zbg}Or%`eNMg4By!%AMcOfEca8l?rzWYaW~GY1L$*LuRV)`g!8Ds7+W9oCuD3HnUqV z1DiP8jvC>o>nE@IzZE5_?iSy>vR19fpcwqiZS$v>opRsa^Tw}IN;C@HUP&rC^5L>J zlVo(OJ5>oE{V2NU?^5F8D*ogUa64-D{6fBC4b&Tfy*kTVMO2*2_uFe8ZZ(i2N=W7~ zF?C4op>cR?9qsS3aP$87*BU*GSRqo~o%V!Y<=aiu&HQ(R3FlO^++vTutX8}}N3J0{ z`NeE=oJMAI`1{5%tIybQdMaiWyggv@Tqx3^D&gXQJi&?j^GoTgqMJB>TsCgk9Z!jI z)8JyWWwPa>X7`XZp*Opg4Q?u88VCv-AC}&H-=)h7~pBe)j$89&(35nuDmfA zt(|xl=5VMj*>Gtn?H}Km%pj{AbX^;Fe`Is|czKm=oXJP#AgxbDI{+w&SR_;^GT6)1 zY*)FJ&tk-+RYDO1U3vO7)^#aq?_gTM32}cC)>T6IBSkzv36i&Q5dl%T!4uZTJmshR z@bWbQg2AWMJJ#~Si4@7UKn>n>b@qOJR!fGp2Jc=7YBJ0-%>@P)e|+mD{MG63LFJYd zRs)XUzAC>cS1Sk0Ut%xd;-Wv+)z_|3+|amkmoWFvUIOnymvH0r=EI>LrYS1QqX@U} z-R6=)fh&e-xL=Z}ig-oQ?Z>nX3X2X-MB4c2B}%cpLE5dKl0a;d?;x3M_33&A!Oj}* zEVv^ZyBM|f28Q3g(ECZM8Uqn88s6u+A1`-PK`L^|Vf~)W29$-U90L2%XJlb;z3tl= ze%%Aj~{_ebJ*d-t2Jv!1C%^|qu2dt#75}v@f zjZnAUva_*?-L;cB;bHjnHgt{ob-ZfYaHdV=wRf- z!~1>32pLG7nzV=#`S*hLg2VfZ0cR(aY0pc7L~n+bUP=aP%g)o9*2IIS90S6mpUGd9 zQ&I4+2yBnV3c!(Cz(nV;7v2oOwDL|szyu!Z@?$q_0T&f3G!hBFyDEbu`s}B8nX>tdh?Woihc@oS5PW0!7`8a(-U=vD&cx+ z9+{3=DUNXy#)nY4p7R8-Qcjj}x>(1Gx0w>T)Q$S*-T>f=Z$QFo#-;#}{@BbTaxX}r zyQbFiC(P*4PeKdr>wZyn2`9I67CO~^iGN&BkE^CRwO?|9*a|?Z%55^aGm1hW(Mq5e zY*BKe^ug=CE0!UL$C2ndH_x6Li3X(v!6fBdE9g70(4KbUtD$w79FU?wC~;Bkj0>$l^5FbAjzWRL-8)e)&(GBngkTk{R}c zUN56yc5$Lc`|;7*u~-`_q5~m?DFy(}*4TcJrQ~Yq)Eo^EhW-j1ViQm;|hE@SOkc>oM|Cx8^Sqn16gm?ph^Yu;up5 z<0zxu7)tPhhd&S96&paFz4WS?B1Fso!Ml}xEu~et1VSmCKg^jqv;&X>y zx1s%KcyVw$xLwQ&yqN&n53)FMDi22&@*`c!W7E-OW=?~shM!#}`N0d*9N zjTNlj7Mib7s6i&Nz73m)Oha-bJmC3;J++_VPxbCccKjpH6`EJeK@28*L!^c4T zN9*Ds{YJR5`LXCEoO*pcwgu&Ee%o*0WDr%K;n&*SzH~6QEnuyL8HAXe9;qN&c{;@l z^&FKa?L720KQ4)=;;Je~vlwMf_$p&M#s&?#eM~MLf0%Ymrh^P`1t%CKYUm4t`N5;; z&CbE2J?d(I+XP`P4^Fw@fsva6@eBEcp~)2aofe!d+jh zd=LeGw7d?m%kRW~X|1jM=S`YSqUtU$1RQa00ujG|S1{9%%HhfDwePyRBP-n>T?{g0 z*gLM8a^spq_io+C`hW;u)*@UQXO85nO+0Kh;5i*3#_?cGY4iG<;8Er_$B+t=^y3Fq zps!(3PxLSa`@Uc+h%p)_DP|o!)B>K_iYJxN=RbLl+iIQ5*d>^cRnyPfrO<3Y9N5OP z=41J&r(E3Kk6X&4RsqR7-mnl--Wc2RrA$%^@R(5k>aIrmUsWn<5T3BiEiX6h8`qK>(ycz-hY zr#)k{cU9h2+>FT5YHfuX_9bR>M-;P-WQ{FhH$-HuoQ021q~ZsZKR9#Fuy$BhC=JF|LOqeSdaNGxprYVTK){cy=+%Zo#_DnI&31Sr zPi==3cvp@jOM>>&7Vpub+5gXM$>6_b=a}(AxT?QXMRL8;kr#Ip8k~C75oU2*sb*JwGzPRZOerjYA1Fys!A(|%rm;2<)ZWAA+%$#l; z#~(?}2xGP*Vzr}1A-~GtI39(@hku$Asx>|Gxmx|KE-g-?^@qpeND=ld?AI`!>bc4(Z7}o%;GDZo zKuF-Zbkh^m^kO(I*dY-@a)svmdeu|iqaG9Tx*I>Q0rENsOTd2TO1|OOjqax5qS3Cqb4h8D3_Kb5#F!Hy9*$QVNHp z)iGCD_E*P0=x8}*6@V(_ISg(p;AOGa*stmU_GR+KDCMeM2=0m6rqsKxUaPKmk_Vzw z`#zQEoEQkK!-%_S6dmp5o zM)_&gJR$B%$e(eIqnMK&c1~S!EGkneBTmL0ZeuU%-6E4zJgA>pKPr#0#IgGK79!($ zf^XV@I3pW^tijohLES$Ij2>M1-RF8paB^+eZKvAFN>^@uRK8)l=>5Lf5Ic4UeyWqG za?gdMt&|gZimr@{dtOx(RTQkF34*<#>?0s)QLd*fR4OFUkOqQumqZrPRu(U{s(tJ{ zbH|-7?_HRT6~Pb@5p_IP>-y?uxx&tPpGzMGa^Zj;wX)rt1#rO<~o#@-@H_ExZ;UZe8wCO z>RAtiH+GDEa0%^ei&M{7bZy{w9`$B9rc%eQ$Hn=Y5Y>oJBBCwML>KgyI+MC`v70_M z%{HOPH@C&D3B^^+l2_=}@OC2!R+W%NQC@!+VM(*Q$cS5)O4bu(1%$NmkkmAAITZ7L zevjT38M~C<#Wxvc86H2wq$QOqd4Afpz(q~S|# zRpE`X)#xAoLB}7{`Nf!;dvEyBpvKv`cy9~VhnvI~$^r1%B6m@nh$=8!b39A9=)oZi zYgFrzv`A2rQx(6f(iQW=ssk+?&hZ9w`ak7oANk#?GU`NYK;~lRRK$-ZKcyeF!ID@zEz}HkKO6Bf*Pr)m(V2J7*;Dsk zRH%*Pk;Iqk0u0eKb9Lp*u9@uKbU`0UkXh2YnekyA*gTQ5A!q-GXCkaBMucTNg%Xv; z*c~>D%D9g84%y}6Jh7(*x`T>x%$-FfZl}P6pZP0``RTs^4}}qLdf4C;62sI?5wo!p zmlkN;c1u9d#988 z5QaBt%MCxNHc}&)mq*t~)7q7N8pFAjz*@ihUS&jzbV}xE8GgIOyw$_Z)rNFVydpoq z7}PYv=2Wy58N=WDp;wAY%15JbCE|{!Be$KBQ^a6&1?~N3_9z+V<~i(okcg+r1C@;) z71E4{N}1MH4UCKU0P}8>M$ztyv6xOxze~`Dfbji;c^0pfXm1+Ga6xdD%81^E$Y5K? ziYxwucF=Rjh?kQPZknqhs{=;HthMuJd{ldqc|dv)SLs6!E`Dj;eDR0Zaw2@!W9%G3 z@PE^ZzoqB=?QY8%u&!tuWK$4~)7>JU>s=`v3`y1 z(CnIedL}WkxzVoohg>*~=$=Yn9FzJ8zuYF5i~ZGQPR`-;NjS8)0#tk#7N!4zq8L#) z|GWX0i>#pnq4dUQ14M&UF1SY2F52g+q}gMZFB9c619j@0`v+$22iFqsjCM>K*Ji%8 zwinXnmrhXsyHL?ZzdC*Vm5wLoN@$oU66h+=UeiLEl)rM$`O)yemg|@!t}A2@NY$8| zAtro;vYLz6>@UfVleg`RFF-m~&JF3AWwqVMa;T4S_k5~=Q?uUu%z$o8KnHIn=~f}< z?lSD(a57niUv1!ShfT+H^82-+PfvDLOnV7;PY2))6f43e>=K^2 zJuEK#o0{~e5_`EYzqQGhid}+e<~fDYFzeJ;tsi+?#)5TjH(xYs2=Mjc9HGp!ofxZ? z1TTtb$s9tHs7PPR?vAKCn4(b5L;2WXIC4fI<&-$l#xYs|>L*XLfQ=#d_d z1;_;-#msZUJ5^{U#60ZVEX~lPC7o?*O7%`k9+l=2Sr7gWa?FbYIcy_Zf_ps>0%NqF z{zkLfm|)P2B~45z2kEp}f>3-o{B;4zvP5CVxqBvbr-FpVu%6hsrKTNua!9#7a1WR z-m6PiAr9HWh)dE+fvDMy==O7}I1dXSSVVTUgHzXP+O5^JtTe}eu7DF4qFY@|FhW$E z4Ev&v)mzf|i})LOM<^d=db~Idri8y}zVJVibPa&?d7EyDXIu`|vOgwHiaLYhf#Qo* zFf{{Y*gCjMc|l%d#vKOUdCNj z*@2xA*7WQaro7~umKNbaN05QzMC~8o zy}jfbyX5cz$4EJoXkR)Xg+xXsTWcb3qZ$oSc1lXI!rD6B?*Xty8aJJhr$}wJYD;h) zxDreof5KkRz6}qseq1FOV{7p1?;yAS$ocjKFWQ#q2g*U1!%b#$~#7R#iL3EnFB zWViZAwb4LS((_imTHoLnk1}IRvJEmN3apn{(M;*GBDidlqi3B~Ii%x?m+HEstSs;T zL*i`xiuQXf!?*eKVk~4idS>m(L$(M$AgCHc0UvyPZYE&}Gv_2ptCBEu)2E~bb;BC& zWqjE2?P{$>t-J9L^7Tp8!EoaGD2p_F#;wxRFg*!WOA|toJWev+e%A) zLgStDZ;WNCe7*4Txas+=xujZ`ikXIyAoV_N%_q2Oc9u7YG3B4=c!eanGWiyiRt6GG zg33ZY*dEj~9^O~mt;%bUF6fdBQxCF$PQYM9xDuF#t#g3r@|B#D)9d0a;K~!fH~K4N zW>oHij!JCmex(;Ykgb@l`+u*lend;M)=YTOKTVlVm}wyFQAf`50v1KowAW=jIv0Ja z>;blj*$J6&QhTI(?wsJJ{L{ypoz#%Uy}7jPuT1x{_Y1Q-W;tRc$~J~5d6Yekz-5~s zo0gwP8RsX-)-aNjPi|{)pM?1nJ@+4dc1ZE18D87KH!(*vCYmq5;<3E>W0MF@0lwL# zF!pJW85cs~ZCk&kSGg3!tmLcCG2&y-CkNzNz9SeL^?C41-bF2JV!NzMlW^pH8&@A< zl`4rmmj;*#TRr1&j(gyYjtAwVZCDJ$T2`>IdG_c>pS)o?K zQps4uv#^Z+@_&${-?_*$?+QIMNeL0f z1^h7P0lf;WcCiP)Aqa_NW|GfIwd<-h>Q@(2?dI(N!4}kw{kl2*5gU<7sI)z(+|mec zFt%Y2i(eYkQ{E^))W%!n^j8qGbcUwhvf)l}DY`-orxMU#*IsMRxhvlQyU;K? z1a@rcHLwx+=0rM}&CoA4w6pNyx>CEtOg|d?#b@`zo~6iN;dLANe@*br5AM?x2M$zE zRg>{|_a0ck-*He>*x4>!4aO8=;xQ1_<^VDikO6wcj^-7UrDT{Zv2-bc0 zm$k*yjU-z3KHtWU@vE7R4t7*4S^M=O5&Xxdl4;H6+O%6?Y*DN`L=5{wJIhCgA?)mub8(Cg9 zvp!a|)|sUA(Lr3?cIfsJPU(pR24BG*iPQ7Op}gT#hIK%Fbf@vfL$sy4FUy{L+bk8g z-+&kvaB;Nn-2geI<~DS&(a!d~;Pdktn}T?wM{5SLb{LVt&Zg9EbiP0{C&rfNIqRiz zo4jjI7Bwc$RtW=wKGY!Ee2eUd5BrDW{~L7s-c?uO;BGdNiC{2a>f+*51%6!iS?%0g zlg~jvZX)ErXO~=Eu5~G#Wg*Lsm>ef?(s?LRQauxSqA6&tFh@sCXZBS5T(*~w3bbl)nV_qSgEo3BXfm*4ssguFE?F4njrfD z(T&#{DHHKBEsP)6wp1JfK_#72$_4??dBrr7NUz%?g4MH(O*+%dZ?8yvuH4!@+!_}1 zWr}J{jJ!4D#1Id9vPbrk)tzh+q@j%Q z`G%r7qV0nvuXc+K73@`pMT&nNGsms}OpkvSnjDw~tV5b~00v2;d&ieO=?a=d7EkB3 zposz?>Qw9V%R|0tKmr3qMlC0orOCbq{lXG;#ya>*4ZrSBSgf2WS;Py}oECswdl(Fua#OjSm57Cu_;d zDP;6}i&d>Igk;Z>Ez`PQJ`}I_?$0_hZn&?<*td+I9j*)XY9FrdqX-eFv<#>x)pK|a z92xS4sq`WdiIE$Ok~@UX7w6yAN7k+{-5qimzPgpo^f;iIijWh*sJ3ZsOX$|BAACz`9T2 zWSGTtEGz%62WkTb^Okzn{ExNWA^$y8F#tckD>0)#e+>M_wAa)k9zp9AXE;!;juR^- zq!)RlV3%#)$iY-vA5if|cJJ{*0#sRBQ9Y5aBN|$|;g+m3?`xNP^hx2C$AQEuc`FCF zAnFF?xYXD5&0WoeMo_?=cyI}@h7;2V-$!IPa_`tDY)CEMi3zQ}wqG@g1Wdcr$r(qd z5hbizKK{imze{%PDbzYCqVm8K+EU|u+;-$Qfw8 zRoS@GY5mego-n|wpC*NoN|H_yWRyulqnHGh+D=GPP)@Y#QWaen^yB$BykbVf_`vdSl@ z_EAOC2IEXUY4>fJ3UnSIsPG1e7^so1C870KSr=dN{r1gY9s+zkEdqc5h09qI7zLy+ zQ0gNlXo|6y&0UwknZhTr{XjLMHf!YaqLo^h<)6W%x##vZJM8~Ay@sLV%BFIjjsj)C zM~^4YVUn1^rd2a|FqxfxcvlB=Zgscy-Ev6x`?}aBcc)RTdYf8oWi`2 zyQeyUvSUSFV~ys+$obP(r62rhKS2EiF&>d4{j8n;9N8xmCc7)CE7vNIpH1DBK%#G} zE2Y&gQ?KiXz(P2-`&6D;|GgLdt1bU+k*?$4)SNQiv+Ss>pzjH*nK&q9Qu6CmaMQGe zww7FL$uf1)GsGtvc#vFL-|rePzfaZC699QZXp~9c@0GG2$PXOl;28*2mGY&{yPl

    {o3jSl7c=al7Mh&c|LXW3E1 zcq6>edhTcDvB-$ZbUzIBCpPy}Uj{{EMsRXU2Eplt^Nqg1ESBnF_x%rV`M(-0`}_R- zZy1Hj@n?jB0LcSDyHG3URR4Vy%n*j$IDW?Aj~HRFEB9K2B84an&ehiZDl=$4YZl!t z=y3GEL3M@B=`g3+x`-R|vYVslSedtyTkhRly&rXJlfy5t%t2xtT>gX;OptDc$77r9Reo6KmxFTrJb zpz2s`0z@zCdAk0rSubqv-bCIbj=7V-FC@Jv_^m#*svR@I7-tFpBZeSSZqes}zQea_ z)6t;<_ZQO!HRx@@>R(%7zuFtBfj7YEM!W#OG6X;(-t*A&^hKRgdIqp!hE0#p#f1&A z!REO9<|%Sa62sh*`MweH#|!OO$P z{xrcHG5189qdHfPDqIK4B=@>p6ZPK@g&^ib_$C2jB2cU>t9#6gO|?3&P*C0`R;Vr{ z_FU*tP>N2noo;YdWM7VH?#SS$PtvnyLT_q#5APTzhUf9}8D#xBWWaA%nf5gg90KA( zcuLEyrlHrJw)7BNo=^U{_2+ zbi(RbM`!P69Bg(s>^O+lX;ycNXe&l*91bs-9FSU->AnBUm-^bz;1w5?y3! zp1k+7Aazpp9(!8y&K(J;w)l!*<_13|>aDvQ zQ56(EqLp+APnr-j)rZKrzqfCTmqViifjmT9ouhcFBl6OSFiD3ZVczz*)wmH1Q}1Fork z<-a^b!}49_vBnB`_te&7vM_vN&0F%aD6>yd+%y^bK=ME;zC5gITB1IJFq=Bm4(>wd!46R}36M>t9`gONbT%V`s*60G zCf>D^k;a{#rX|;?7oE8g73 z;;0adO+$D3!dGXNqK&*0EGvRD>J3+(ml=i;>_p5iLcj6!?41$|Rh3pFE?$U|x!@rie^ni}mOgYSD(-s_#}6HSS;i5fWBxO@-`6V2j{ zq@58>;WUWm8H7JZW&2EgLG+@L4)$to#HB6Fbllvh+1l&d?0jd#6bZ&YwMQpLrNh~0 zv51Z8Wx7>OiAfG$)vFZkba9XXc@XQJ$2iAJs_~XKv8mFV0!g738uS?KjgZq~VlX9^ z3%dy(S7N4m^uwCRL}2~!R>YC1a3i1x&|^$$M2-lo_TPwpUE?ViVW?hu%K*FKoj7v) zb{yWOl3CyfUDH|{J<>$5nJ^Rh56=Mv6oi@LrBiv25UhmO{sszJ8_TT%c&NpaJ?m1? zcaxU~bjKj=v>BsXgP9#!9-c{jjzxlF6=14?8;!xnyO*~Tv1hU6I@Pu7S>@8~ z>4L9>1+tt$${!j2&&N|ga+;m4V_b=Ohs_tOQg0>6Ju%8imeR%P9!V{V#|Dbwze9^^?7rZ^ZDn6Ae zT^^48RFnrhuMs4|*_e90ij3}Q^AT;^vuH5P6yjDyXA!_-VP+}#mo<+)3ZL^|h&SF# zKC_}fUk@};tz%4k6a54BoHwROS|lX0=Lub`YP({3p|GczA1nSkN%el1_3D{UFz?pT z`yHu~0r67av;A)liauf3Bg+8XRd3D9RTGK{@khKd`VkboWLLqDqSzA$@o12I8yR$g z_emdN3i1>g*cC_>rZ^J=agUv`@1Hj#yghCfI{y%PS0{jN$-p4QHrl$AO)wV7GPWvk zE!e9u&-#O@-F5Ov`PBl+C5Xk>Na^Vkq}ccGJN)wea$#q>(^S(Rw37q*2h}G26&-&` zS2?a3wuVT)4_y}R(hvHgQ3dafOWos79fsn$>3P#Xo_~^dvlcx$g9C>up+u(u5Fn0p`)E1a!^T7Q< z%$H=DFzA2)#(F+ZP;OqF8`+wii{$PF}xo>=C z>;5#$XZ667o4)IOSh}b}-I=r1l~^tc3kMloRUT^Y@nZQ5_qWP+4X5ljhNDfp(s=d4 zYU8xnYmSvIXZWdjM6X5<6{zm@p6Q;7o0I1G@ZR&HtXziGM>6*c?wApQ_NBYdRyAFu z>DwF*QEPN^I9W|iSEKd?=hWyiGT7>>*NLfvVpU)DqyeFXWZrPv8Tl%yE_aIt5pvzg zWaHRa8Uup~_Q__q)EgDoENt`)t`_dh&m+4+8pO$FLp1}DO`**PMm!;wuAh!QCPg*r z48IojqWCQHA$uw=j@TU@C7k6v%s92wL)GM*C}C%uR!m=egt`)3EyKnCVNI0A(N9T3N&cTC3y>E$>N z(--$P=}v=datHZ8|1*2|S7siwerU&Zh;b}J#_%4mCH~oQ9&uk1!G*i89nUczP{lI= zM5wR2ISVi6Z-+UBA!Lo}lbXb8H_Qe5ZUIVTYQ;x=pX5mDcd>OrMx!t{9$aZSCq7%_ zAtJx@iERdQC{VLGElo&l5hclBm+}{u{QGLnIYWkMyJTLUZ2Weqn;d^xx~@7!yk{Z+ zL#kUG=aTIu;#=rRtB+Ceq#*ZfKvrV(i5%8Ezd4ww7A>G}-rBa>U3OyUCHS(C@>k(U}Qslh$nvTd5nwWJ73O`g!u zU3vT5+{-%=TJ{XpUOBc^wCHap)%h1iX70eg@nq--yj^^4rGPcZX6p{u_A!{+R!=zmgQ(e+Rp-}F+GA+NM)7oIh)IUkEJ5S946x)l10g|w$74=y&p{LB=Qn= zEz94Y6F~4@brH3fXsdVWPS?ojEV;ayG)L!~jVoA`^Aq4-!;aI6rRWWc zrf$Av8Kmc?ENi-a$VgbguIPevfD^fXVk@@lt=aYI4Bqh-)Ii!|gp?0W`W`>u6eb@l z;uTL`5mRW9AkPw6t0E)X3ueYYCpa5d=D!`>t?M5v4IT2`d-3D$V+Lp5_sH{;Q*Wvk zT1@w%!Eh0qigQt3;pMoO%FWX&;0~jxo1w=jQj{tr**P=t+8b==%#r>cG6YXZJ$>=S zuT4AGmjH|9e4jz~?g6H~Pu(LVxUi7t!i8~uP1S@zfsu79)!v;o-S@VnCX!z7`*p9D z6pJeS4HDaITt>Q>LO^;&Ix1A0xKg-s|B^)|@@Z5Bl0Famx_sf-Bpuo2arDi`^%DM4 zNbc)iTRXSsWl@aoXb?}tRaDd%Dq`$pVYI>0W9ZZsePN*s(sSI?V%s)(Y zcl2tXZE7xq_L#=#W8}u!a^T+KEPx=XI{dC^$uhjPG13g9dB0=`g!u6&EVkyXR-O_M(|epbxlwBJ|c2O~nU3hc7KW88>*b|nSTys~~x z25ot#bHJ>DtJ)1TXQzD0Sy}HWrClq=N;UGRNp{ggjSpd;y3;R;yS*7@Pil}uMY_br zGCZF1-eq7~&PdS#6Gb%*O^}>Oepr9fhL|Q!(-T-sp)T)$259u;nsNmiuu?Z)UIHVt z{9=~PVOv0)0PpHDkoNHWrse{V7#B4$f!d}fVR%sR34m9u?fdbTB#6cNgi1Dd!^JYq z=RUD>XeO^ilZU(SFJX(ZhK`tGrFmm+c&PQa?qHs5E8F2~)$Nw5k;Roo@43~-!3V(|}0kEiSn8=3w$C8j2;=kShA)`1sTWB|nQhj{TU|YToAz)YV{v-py9@LOPsyg-|Dqs(#**MQZkb+}J~1yA}cH2()%6H0&jucrITe1Vbf5dEWv|^~jwlV@K0N|C_VL#uO^mQEKtbU+b)z8z^i4SKs~kPaUg1D%7km0aF@=*K ze9$!os`5uJQkoDjuo*_-%meMT_&jiMK1YzFf$Zzju+I*rj zt=9aKSU>J>K$DC5YHbunmFYl|F6k?&8d+9G1`UE2j7=OBaBm`S1?(z!l^fC3BtM&B zK9i1NKDhw6aL0wY1yT;l!m5=^yIk*0O+Lb#Vp-~h_c8P+p=l1pPye37z zq;ol?DAel`mL=#u>9xY{bzc5 zE*DTFIWpHmDtXH+Zw^wZd%wb(18IYZvUYPX5xgN6zsGu!CMvOS4U8D88j32lWZ4TQ zv7A_={d&VsYmEd*)@RS5x_QSLzK#Ly1q00?`dB%;3hVKiMV6^M=cyrgvLRNDrfI5a zc%kTe5AP0bY%s?3TJ6ZE=^YAlvq*3R_|Z;6ksm1C_$R(&*O$Vwjs~S0DHx%VU(<@N zU2s|KtIv-W=y|ys&Am*!QHBcDbn{AIL=h_shEr`^ioa)ZL6@ZJ$Nc$T?T;=5NM%kJ z^Y(8G(0(9CzVTU=haU>s-m*1QkYxEa814rFQ+=0Rc1-%J5I(BS4984Mk&ZG+@Na_PmG7eRzC6*nEP{~WH{i-Dw+O4X|K zvhs;3kaW`P^bZssmD^yVnOdi?Gk~U&v^LKlNK{o;k3Jn|lXv^c6f;pfmrxpnT}1+w z>P6rdn4lyNG9qW6%~H3d`=XL^T_5EElQfNW8j z??)h4c8_?G5i9Mh{Eg6`a4FnTQ=Y;h1FFdUS*;dn&O@Er)_M1zu2WBh895}jm}>xF z1E1nat_-Bow%_f#*U34eN?t0hn^l?s%A=gOjcBG9dO0czx`Z$QYw{elJR3C?lV7ziNWyzG>BcO-A>Qgz9;%?0^wPolPm^T3P1a_X0e#P&)Ee<54` zfo6r`3I0PX)_j@S+Wajj$5vC&1mDr5l!&o)l`m|&@VYb73^Z`L)*&xwTiY^H+h+k6 zX#J&_Wau8Zl`SZobAk;#0;_a}uFb^N3l}+ z7E0Gy4c5Mrp8370^enT*+~H$R1NqG5Z_Ui|pGtwkkrENXxd4K{^~m5=!GD3ab)JLz z5P5b#=LGcaj(^V*rQuwxT~fbqzpRF=0_U9i1;y`gpSi&9shH*3^`tl4iHc_pa#^}H zRCZ+oKDyO!U$W?_FCb2DphdoVSyFhv5%k2U`6IEfV`_RgWF*s9FNIb(vgi~Yy5t@z zT|rbG%Ah$z4^RwM>{c_|t9n~aTUzrfH%RM;DJ!4-2MUOA&;AVFC|#86%8l)Pu&$-m zxw`oeo}19r^Rqr=H;V@i&4Q`WHuI|^t7|aH^Vpu>l9!+4LpN*DGpefa6-wlWCMM(3i?JGAY+5yyAuVRa{^satY(cYdq{(ouzM3s?A0;3c`izJ2 zuqa)p?770)dH@JYuo_?IhN40OZXIuRk)jVj^=^}TlS2|9m4-dy*0!Y9yJ?zJ)k*Cd zil*=S?kX5hP+Dx4{72yyv3yD=w!#~%4V-LmkHJ0`6@o$mQ-JQwchW+wpGBL!o znJg5IDBbM$3?3#*chct0BFSiUd-=V$Xt}Tfk4EaeDWcgXW)OS_d{%V4?t3Rm30hD@n=QZy)mmq258+*Y}ampU%_Fx5kTuFP4 zrbc9Iiqidd`S8M#7|11;%sGaMbYDobf-F5XcmZTzlO=)|xPux;-E0T!t;_N^ElS^2PJ?Ve@x*gd0z#*m$3>?fB7 zylRU9*_0B1sM*SvwPD0&G-4Eu#ODseXu8EKc_zc77A^C`yNah?xhrX25*xitNQuVb zo=i7RG`UXu|A?KFCoKbYt9LsnwgG!4_<{9dI*DuF_-B|zF7KT-BY5bS%RUD?k08ml zysZN%_)lTEL8Z!|_9_{a>wY5njT_07q>ZTn_~%oqNG-{sePMMT2i+}v!m~{+^t@6= zOPh_|F;!xk*3X~LB6mcGy|HrGA0!E^u_Xzf$kf@_sbD7l@j_|UQL`d;7x%?$BiE>F z>*}xgeFAoIy`^y4*H=sg$*>;`5XGNjK@pSk%h24)S z5sOY5l^hf4k|+@=5es|XC0-weK3LL8`%IfV2SrU+?$90kGH=>_-b$f#TekFIUBF79 zLh#bCn6#@NWdqqZ7Qx{6c)a>-n)ZHiF!>^xH?1^nJsFZxev^3~Pq6XXSO04k^7lGF z_u#ls7cLk1F$_;E7CV%x2n2j1cL$`STNy~3Ml_1?(a%-p?z^+n1 zY~|+hzmE7(!#%cVAG3uIWn2Z(6{HtApmlwvHWNG_JceRZSMq{p1Dp`6`Xd&~&iEpY z59$y|S&l-E>!kksAC|*A1jh-%q_&B%A!2FGS3l|}*?zck?&K7mnHjvFet8$s}uH5{* z&JaQl_Zn4z!68GhoRZit?c8MwYYA11qa9wyN@cYUler)0L5+`bILJs z))T{|c4STr!Dr9l6+5jSfXV!g!tCJw%(n8t^iD=W<$?Wa7DKydsB58jh=F~J5lu-W zhxi|(42m|^-o`p&V?nD+h>!S|+sr~kkVk;mIP3f0b$V7imDap5pX>l4pB{3%2 zC7i0LJvQ08iRkbUfkBy?&DjIj6HlM~Ejs+(Z91aJ6vgxw@m`a?B5K1ir1?~0uY|O6 zgzZe2VR(@)q>}6Gx55O0bYF$EZ0p}<;dkW2IV0vYJ8yyHoj&G)5$2tsgy?4XUDhr@ zr_O|u#2QAleB2@c@@kvmMTkjo|4*;O8gsNS89>XdmG|Wa-MqW)pS(58!ibYz^^cca z0R*9uV(7bQ--=$U=c~nquT-!E%m47SRpprI;@}O?&etnu$9nH1YF@houUp)P7**sX zgW&1Pmn&0_7Nc8)rAh9z>McaUmEX3W-*b_;~VTQTOu`d}wc{8ud1zhZ&T6V5M|OLwfo9W`iswIT76tPdP}^~=3YH_3I+8VXJdc1CbOUX_TFwRulnaFF1;c1STcot z9hvrO`DQ&9dLG6+YhPtryqYq9u9&_00g*#U=k6c-n3xa!=ZC#3-(=8%Df6GOgvGL` zUM*+A!8Hf|PFa%S`6cEW`!#Y@C(VAo#?g=Sw{PP`%(XAO{`+Zok?$F5{rn13%AhqD zxx%1OT*s7H#yZoz*Nd3I?S9o0|F_@&^Bev3pZ_{hf0^Ykv;5^me|f=QUhtO}{N)Az fKfE9&gk^c3(<`OAuCf0IMQWe@ literal 125098 zcmeEug;Si*wk;5X2G`)hU4py2ySux)#&CCcXMhauB)D5(a2;HOJG@DL=iFQG)vfyn zygF5b4D?KW-`=~s*V=3KM5ri9qaYC=K|w*G$jV5lK|#HZgMxY^g#ZWnOC!H4E93{X zyPC8ZRLum@0ptmyi;S*26cirjpD$?Zycx(#M4)6PL^XV%k8|Jyl6J5L&rYlwkLmo+ zIz?}!O1Ge3-(p}xi>4wZcWR`P-{EOUNW{~9ey3X(cHDUz%G{RqbhgGtWt+rd`-|E( zXm4-qQRs}l+gU&N*6oZK0~S^U>OYVFMBqOW_)i4>6M_Fk;6D-gPXzuGf&Z@~K!yQL zrHY93FWeV`XydJ7=~l#j&zj!gL89UymCr%9EltqYlQ-!AV&IeWhjDr$EG!{7{1IdO z&bhL|Rv0lF0fr3f@53ZA#0$$;U5cRAXnw|2{aU34^Y><_E!A3))a72V$yNw#gGP*Z z9!WshF8%mH!J7$mb-6NF69J_%Zro8AXmqGg$(U28g%vA<`^h(7Zpbz;VaaLhXCjSS z`t*P@=unUm(C99hFiT7t@#BNd*%oc-)Cf9q`PDP&T>5(NuSLj8UDr;H6I6QM9XDxg1yR@~Q|Q^m>|#?;yuj_qTVi@)+)a62wEew^V^ zka^>u7?|)vBWiv)D+}Lj`E543Tu|x@(=0@eBq)|#PnzJ&aOvqf2Sn`qod%VbwZB__ zy6s7G2mAM?{@fQ7%qcd3aFke7ICi?omM^71c#D!pCt-rV)$ub{oon)?{P-dwl?qJB zs&_J-DM^j!k8#e_2+hxb?-g>p|KmeP9BRPa-ll;0fByGBuW3Q}a}(QA<{AG=fq?&1GBC<;B~d_mHQBNg%A1@`}3 z(ch<_Kcfir%8~!KLCj*1FyOUL>rnqI4m<>(+8o!56qj##ml{|}|10;9U*M_99;Fwh z#UN8+CKM`q(6;w|(LZsbO#0E7<-Omnjz`PLD3BqD{w*~0R|43YPC%fiP{@I;RPs%$ zPf7W{&5R8Pvl8?u{J(Q6qYV2!PT{$o?tgsakWsAgU6i$-=4A&_w#c-`{UDCZtP<)B z2`Q=R@QBNDpK7HmsdZHjmn(&lQAfDwbScrFAE=k9I>-jL<{zVrdpMVVxt!LTG#>(AtSvoQ;YCONt=-Qd zVG(ZF|4f~b5Jf~r!+X5v-$#Cpll$p5kr5GWORq^~(6c+wn;L~SqDzl^Xvr+fN{wrm zA!LV~LwSCITgvvq4>LCU&teC!*Ig8J^1BKiw|!O@7Z=6h;WB>p>o$ky=Q?^o8+y9P zLN4H3`UAME8C;K7J3XTrJWWWPS=L-)D)?X3b&xC~u}|oWb0)pLQ@?zOQ+mCoNo{kG z-qx?9Z#<8h_xGaz@0uK@{{6yFEUU4d^J1=WbNTyR-EJ8m6hkn(oUhab=xXqIfu+}~ zq9Eq;3boU&HE|K#98S%FKe%XI_HY|?n{#2D@9gZ{pMRTF5m4mRa65%)r&CtBG^S-v zoh|Oi7mPuWnw$Ei@xKCD><%+I|DYW~-5i5m@V!@r)>p4}tzoTw(A~1_z?0P2Q(#S__xkJY0#*2DV$4$ny=>TKG&&qW~S`lWkj`+C*_U4EJ&=jH&K(4 zI)CBI_-L8W&EJ%78SD_t(ma=bPR#d1)MW^H$@`Adur-K(co7!v+f+AvTvFdYI~5%# z$tb=*>XS6)ix|!6sjRa&49u4~uX*RSIZpORy0sllipSr<*6P?a%g1Cb|FvdsOu@p; z-!T)FbC&ZKijD!%R>%Znij73Kwb@<=M9+#a(b#X4h#P47%?g#}RT z@YwrQP)-m&w#b@@OJYmRn(oH#sS?%kC1NCn88_JJc;jXj-mWE~h-N}PhTihvrI*HT z?0`2ZG@&f)z%d*@YMt<7f>Lg0aSeE&`tdL@PuR0%c!|JM2e!JzX{Mj0l`F{k>kQhwqGEyUVgv8il79|LSHDh@ zBA2_}t3@moLctn*3OOj_($*0K{geC+4_L~BwGFhjkLkCpue=SG^P-hnz;~Lp2C2D1 z$i-9N5$Fv&zKaU3MkEXGE{vFoq^RJ^MgZB8?9Ahi_IQ|Gdt*sex^r`E!F7w<7}nV- z@W};W>r1Og&5;!LEWSn{J8os@j^gN^3b3cPPucq_*;%9ap*2{$!cvi7HlLq{YnnSs z`y=?x)gDVd_=wt*49s=-^dWGn2+Q&LIqU8|JnZsOoO(I|YcJaC100yz@Zs0cr%s!} z!?U4rCXl#pPm0G0yBMci+$$hYRR5eK@3t3-@Ao_=y$)He-^n!CdbVI*h2TY1`EaQ# zLo_h^zvnGKjNDKg;;)5eA4C7c71T=*p~QiZeb)lZ=}XC&);liki5#7sBN2AHHUEHC7!B0u^Oe)t@laTAUH znK97vh<$zAqYB$p%!H^EgUZHah`}nIYPw%|QTCtP+7JOha$(3Zdl%BMc4QL;%*oG_ zRu{?c<71+}8AWh%5LD4vmJV=euIKKM#^n|e#FoXpw_!!`&2d8}biTG6SWw==f;&u8 zCgBW8eQFYoL0rxE!g-e0B1L=Hl0TfQCZ@`c!>{tL(Urj=Bn(2#Cp6jE&){$nK_j@A zWid>6pmVM9t~QiN$?8&EJ2unIMD`}Dnc~ba|J%W9CQtjJcG>K!aPEYzX3bt|}Yr!|7 zX=E<`J7w?d{45{8adR8W%BvF9ZM<1$1_y(jiWcVMjwOE13VqyMn^XGbzF7s#`VhP$5{w!^J&}LUKF}ppdQ@zFBt=pdPqf6)@N>y?=FI)$p=c%b59@m6iIG=(ND;I@05O zL{To{TxY9awmeevHOt8Xf6nE^yzj{WF<1Ps>T~cb*SbwRrX+HmXq%v7Mm=Q^n8g@3 z98{&ttD1#$T!Hy>jhfqOwTS%N1x;Hl4ElF==zDYO_34P+`b&cu@NYAYx~ffDL9xc= z%m$0}z8_9lKYzqZk@>^~JN1t{zHw%@yIf-m!FI&sB06ih6&U|#ItXe@69%u0Ngif} zK7^V-^IDCW+%e|2b8U{E&2i6Pp$xX!U+|jx2-E8}Bi~<^9n(q1<_bv19BOIDVZGF< zuG)Eoh=YInWXHdoFpPmVkQf$g-Xo%3PsIy<3#{+Bs{3wrjK|Wm z|Fm;t$Jl!0U=#Rs)yg@JH7DqGC_tFUFgQr4r14YY=Wu~qlf#2ojHWybMD@#LPvXW^$Ydr%YJNHa-NUBpgxb-s4@U8KSLfHY)4A?{&7dXkr zUvA}}q&P_Y3yfrFugLxA_5|%k7Dqn%v8)Z3IL+8`iuA=2+xu4WT_Np}=M3#hoyU9c zHH;rTzl3Hmd=4f>6G94U4d z#0ez7h~ZI~+D;exAflm%G+JG%mjO&{#VJ=eE6l-*7PB zN-eIsKgo&~#^CeR*Kn@>ZZJ_o=`aJ+evo5MSuZ7T$7j$|4i{+0BTIez_~3Yj)g6n6 zNl;3SP@&S)@{pN_Z>Ubb$J5?ek^XD#;hwmcbS?vC^;7xZD2~!_&u`c05NQ+D?1XONSf+KdNxa(jh0hA=LPy%gv=Q>VaLJV2FNZ?U4j$6Uo0){pQAUOD&h zn@7^?azP3dx(iG~lwmy3)|SRhN{{5Nt};JXfA{!wX??5~kl}FL-%{kC z5`xId?_=07Sd_u7@(SacMbnkeg&9cWrS^LR&Q-%f>NM3^AWdR9IQJUV#qP+VE{lQ z;?c{EVcS1`w|qvw;q$59Q}Ihns_{Hq8rZ6#`e2l*@Lsd_vt7yvrJeWb4ChE%7MPb% zwmfZx4GSf4;H{+`HUteY>v2?mb+wr(5f0=w$M!s0=qcm^X(hf}NUL3~YkHport}Gm z%i~-=ote|usMGmy@!O|0rb45pn4r#lDxHOnp6{kWJdr`^6=wKitfx=%Zz3Tf`Uvy$ zh&mShd1{;&9 zt6-GW#@&4&l(gYfn6Ubj-MFVO(E-tq(XtwkE)_arzP2e=)ohdGdb)vkq71BbkG=_T(s%5!PB8uOUV^aKK*2kOSq#D1h1mS6`pL4M3G7@Hc$6r_@aqi>^g z{s})NJ|;7@qpIMbsu!?`4+Key3;AR{c$~@n{J_!@2)o0Dl`7HA6#LK80pTJ2y^Mdtkwn@p3Jg=$vlK{u4&HGqu{k;w zr;Ztf;C-!2e+_2ik3qxpPHP3uses}n{?sIH8uu>GJ?Nd$BrGYgkJv6M&uMDm*fbnT zg{Y|NbMoiwp0T+B&OIs!3!3mhV|D*qZFHaV8?hdK=RzhDj1M*zCf`VDj{ags$4Ah8 z=jL`6FFRANpjP19@eyVX<(+kqQ^C}f%a>9!9fRvV;egIY(N?Abj(ye`JM2k;66W@1o`wW-3G#6{vS}0VF)m+X;Q09s# zH~V*ML2A8LW6$f6U(;sY7dKe$9p8k_`%G>;fDGX)ybN@(B0jzA1wxbj8$E`oV<8wE zhNPF`@Zm+!dQI~bxBMQDZC;xo(x>t zdt~&+QC>loBT9#Cny3Nptek&BC&K_EI{NHRAG!R_Q{Wq%^@-4`M^mdtt_DXK(f-X( zTF#^Mvvp-o1Vt0O{e#P(!jIR+do329&NcfGbh|uU{hWtj6IoHn>wsr3(>iuK?MQPw z+S4A1*=(MU#5?L8{@oc|pG(Jd#n^XW9lkGixu1P#JQC8m?n}J~0n}nCb=NL9g9rVw zru&0s4+QzW`CAaK7sBYyPkrvzw! zSM9dXOCZvtR1au973q-b$`SeufojQCz{5D#uf4h)hgTm^j@Bs>q-DqHcWh^l`zZ%3 zBRrXI^{`?7-QU?*frSADsjiHVoE!jK)@8Nw8ea;H`>a5wKP+}pK+pleMbu>gAR=CfNfh>Us z!V=nmAgx-BRIalH={n;;5_IhDu;i9ObOr=kIZd})p-L2<+1W^~@P!zFz#($9)o_an=RJUi;T~Rj4z&ofgE;z0cg|HYz=O`UKg(y{*8?6gET>Y zo21u1KjVU@9+>=P1Pm`V?G!(Bm_~Ee`96Zf4rUtz*&Ac}hB}8L)=7kdq!wFq!tvGn z@176m>M6gZWwU_^`KLs44si~KQu7Wve`47lu8{It zNZa~dZSx8S;>v7A6hxg-M+X_QW;_J&MMx=R4@%vbdAwYp*csIFofve+<35u|jG~nI zBxu`=#oty)jpjN8Sc!imw55qTcILwM?9Mf$&C08_`#zb>_GXZdS!R#fs7&g*ayr*@ zy(-kQSJiBi(xV?(-K}!=6KlbLo2F6^$g$euX*`}U=saRA)$14_c&`43B0cP_Q(S4B z$-_Fw+B1 zmaZtCR4kdF*^Q^y1=$~!ak1;)usk;Q2EPJM-@2zbi)#8h;}5O%nZc&Ic$LUwm)?dc zaPVzUUOw9jbw_A-N-ERGvot>_xISG;Vh$#%^J}Y25b2^U{b9dto;ISLaQ9V+7gu@w zb?ziM?;oR-A638)b0|xjeYBslpU|69k`7{20_+1E8{m*~5}~A7zy8B6TRwh@Z^X(x z7rt2b6(_;1jww%Xyxgwp5pPf8OAxcCd*i>4YjivnPRTnQqkmg2wumXUM^`|vl+#{F z5YKKoV$z_LiLo8o18y1RwW-4oRv8@Xg;T;B)`1WAFPZ|~004Oqf5zt6Z`u!fO_0@6x zy+V=Dfudv9ymn_66}TyU>~^8z^6H>v@k(zr^mtBSMh9=0A!<{boP|5W@iDw3#R)s3 zBX(*;^Izfs)^J<*wkX^d?wlAXXN2I&G#L!}xGZ zyXn#dLQx6Er;SFecHrinHFs9O^EH7?7#$lj}=oPebAX}f37P;f)`7p4Ks zNlW!m^8;hE4N$G-BCa^EB56=pMullkrPUsT5iw?L4*>>?k*d9fm})YkmW-zTlI$uV zNOym(ZElQ9|7OsNUc0i~jwdyegoLG;TM}u9M)mwhZa2??&y*E>&tFLNo&6JYHd?;n zC7hj`3cvE(-Y+7Rm(@6&dlm&)^0ySv3JBf8VXqCr4PsrYLSu={*r#f<1Av#>)kqkqA$&0-g0&c@mdQsTbe@h7b|ln%;GVX;x6Q)L|6Hy}yu5S1z4|!Zmj+Qs`2M(yfGRY@SAiF7 zc(RRhSX3gxlF%7eZsEBQ!-n`)db)Oy+bgG|ibG*avgcP2vn1J72fX3F?Jbo&;CILm z^q``AnL6=y8MhfoZ&^(;=#9iJl+*3El3qj!*Ofbb#WrA`u6G#yOP4gE1`(<-NY%Y- z>6nSY^;`k85AV%>UZtv6OHKtN7?r$#2iJH~jv_a(EIuuYlp}_qwf870ll6WyD++@M ziNcs9=*`eqnF|5LR+`OuBB_z;g%T8c)AjB!JxAOic}}_tg-F6dQK5^}#KzsGVtQ~J z7}LcQ3HAkxR?7iW%A2 zzrEJ4FrJ){5;qGsRWK8I5-F*KuJtLkU1weQ(8nb`D>v1wX$|0 z=wXwe!)&8nsy)6X!C2unR?B(iC&^33@bYL>O!xeL4s*2)q=z)b9o4~@+930JgoHG% zXca!TLRPLnWTn#8@FkGYIlfxR)ImvIkR_WpbIrf~OT{RyH#!Wr^E%Bi_iw4NU0ZI7 zehTF!l+c7@Q4vC7NepWxm9a~j#E#D$qncU*ZadfD)^KWNp3}FmunHLQtk_SVi5y?7 zNgHjo)@>S9llIhGXC!6a+<2mSde+6Ets`jVyn=#F*uz@!tM5SBmaZcF zuF3BsZZ}Cl$pmriE~NH4J5?QCfGnNs9n3sHB!f?waJTt$oTixmia95BQ)U*hOBD`= zSyaSmIMocTqN?r?RIn*^Jswl7tN=a?1aa>U^Mkrud`|C&h>mG%gFjo1}sBs z=R{)vxT(W#uPK1*mvUY4`x;Cbn1i$*GbzE-XxaH1lzgFr8ZnfFDfY^_a4KoHVd@cu zZ*9-=g!}vrWK(@*@n?^)g7rp$q)C{6b(6U%5pQJ&t?rJ9@bF^p#~7rdR==yq`E^}v*l9*O5Y963~0G0b&-*6B*GS+t?n8iPs*Aqv)7uH7$idD z<^&-O9eq6tdWrNp6D}o)$0#Yi<32T?;Owh=}a? z6&yXBAr$bGyi^x9S1#I#a%cetT*&V!=bJV1%@mkhmwR0!!OdgkIxPCeW7({gLiov` z|Fgfp|8y-fwfeo(s?Avemh6C0nh@1?|BvZW_~n-shtt6!XK>vBh@4yd&!)>Hj`(phOhdRtQhyRTBBzn{@{h>cowmKyueqQlos_}*7KuR)k|%A z-jg$D5fj!NKJN^q-AQptWPU5ytc`d)fkB338f~xZC{FWRBW+xK?82HdkuC!bz;A{& zvoFf{aVRbuly6}R^8|{+>uGr1-QTrZjw=b_R3>aiblM`t7utSAu$bJH7Ju|smXha? zh5HnFJP@cs=(F$GUwe~k(IAb*k1hg5#xLTX_`&g7)0xxgj2Hw(ShK9M6UGZR9wK2c$x|JJ$yS->4(8mUcYb-!Rd-z~!x%`?(<{9!&pM_BYKtxPc) zO(A?8{2Dn$efc0fMDB~7&&K5n&+`>eRjW2z+QKTmsH7^3aJBW`_tkfm5CyA5yRxAq zPHKDuzu=Kh9;gkzCJL(y34uu8g{LdU;lGH0@hF2NzR2;)JDIFR`ft!ZMStib(k91X z-zqMJ6cuHABGzy@`;gEC4kEkbxPDlu)t-xrYp;|0EdlH>uoUd4Z~u;<KwiCiVQahktT+BNi%rB+~q=R=)bfZ4s7Xrda2#ax<< z5YU3ikdTdt+hlxf1R9XYG@O&gwpRC|1ehsXY5oQf1q?_OZpjBu+i0z=PO*Wo0I8AO z+71JHVVY@$nh)`h{_&i2Y)zpKE9Erynp#XpIG`N>w+>e1$|bkXQT@oXU~e0X(*2t) zIY{jYm8K6q9ZjBhcF3i;mz0=S|C3!@Odg+O4wRP-rY}pqPkU3T5YnyJ|52mUP1o;% zF*E_e)>(oaiQz^E|%IqyX!>FWM|L~Gi2dwWrSwpc!xfcHDziOw_e zYdyhH62-tqO`udc@Z(@GNZC;3%TkJF?|A}v57XD(dC@auiL+K&--{%_Yt_>KL5(td zu#1AmcJXrr3vx9NrnwVD)+!&C;AYNJBq0ID1zkw~m^!bo5UuQFi0urW{-bMMZ(pDOOV+xW?1g@~Ok1?$eSwDop~ti2k=JII*>I6W z`QXxt8cE9Q{d)Vy{lFs(&E3SO2$t}hqZ8N`r1DNmUWm%h3;j|{<^{jGk<@T`wYGOI zU4Q7+>3HXZScM-$3X}pKdd-{ZU{mC@)~=RxvjKXHyfVafo^|WLM?zn|ucC~_LbOT& z$GDkE;r?zc1l*6$P8%lX+>?Mvb%dvI1_C2h*{X~(0Gb-cef!zU-DDBwY>8ZECqL*1 z#<%6%_NY6gfsxSK=`3a`b@*CrW8S7YqnRpSd*c9gQF!2EORZMpH##^VOTe@4_**|b z@S}^CuUq^hgQqia1P``TS2D=#IbJjRqEuqH>Xm&VY;y0rl}XCWcGu{`ldfg8as8ph z!sw(u8;Bp^_S(PC_%ZLAUCZ5EUQmYbL;g;mR6^QBWz{*oUSwi46&_vimQEpwn^#Fc zaPTg81W{r&>V(3F=JjT{ma5Md(T3be$*{`TH<|+7#SG@JVy6FBG1GhpQOq)RlrhyJ zWmYWK$1=9ei3v4!u_E%0C*bBFO-I!NZHwh>ot~Z^TSWi4)5l-;FEQ_V{HRd~r@lx9 z&5W9uV$%4{FG*^&9xTsRizCzPLr{2w<6JSb<7yyl!a1a;0l)vqB3xEli>87_)YjfU zc-y?U5vA&=;g`Di=E7{5@9Zwhp$`QU@8(eKwApOF zK$_;L&`dO!$*eeR%D#COEeYO>k=qG_iGvH<(F2YgesO%tcR-o(<&?3m8UOnIJs`tu zsQKAh`K{h*j7$UFdh^2Q#C7nF=f$vr7GXlZ zA1$`Faq~IBYwJC8w$q3YPV803P2+-+BQ@Vissc2P*-q6Hhl$N_&a~M0xJ**mK=pF# z?7Hgu+nwU~8R&Rls{w?%G~W-S@JFjLdTtwfZT)2H+k+cO?q%$nKeIRWL{eVzALivH8&3qkuvI$~stb{LDHN^upXxI%#zz zmUDHH$rdlg4|qrDpbWsrmv@i? zBEPt#V-zyQdGny76^@J^-WHV6kUhM_q>g*xQ{pni5bKAnPxJ8MHJL@BO~_71n(U7B z1;y%HF&fNL`V|lrK&}WWp(0trq_h=5i%08V3}+Ep18y%Hb8LinV@-dq8#ZP)`62p)e%hq*A2`Hib_0E9$d%C|FS&WLR2#e6ZS-A@EU$ zh5toRtZdS_uxN(Kap&0IpX2)n4|e6SnZd~@7baT=k5erDpg5xq;_NSlb7h4w z^Z2YIWkRgYF#bS)lmM&P_nnrc)U24OZz;kPrHXfP6i>NM%UUompIqP0hO-Kvus@t3 zw3`W_Cd|1bVX)T<7V4*^cbAFC1`qV=bX%r$^^q~R+py{BQPAaN;eXNxf~@EEo;iL} ze!RT*(^|9)ch%XgWq}SLhg$>#g6o4O_>wsb3^qO!11NM^s5*xBwuV=dII-;UwAl6! zC!>_HhT)hozBE0J$@u3CVU_9RXVyjPSBzDVA0dF|b#|UMZqpwM?{m1cgCt7tsuqixI;`N4Iw8}#Fh;{Mf&|xdRwYJg0<~UJ2El0pVg8)<`mxw==++eF0C$xv(%V+zm zLFk&8Pxtx|^lg8kYmb%FUPrxprp}cZlf~i<`21A7!6wz=2#|(Fq*e66AL-F5p}P9| zVoMS~r#m2L!X-TFsQJ_tA#o4n{TY`mUs~j*@DI9AMH!L{jowzTfNQl(mM+l1l{~Ap zpE#{-cBhZq?^u<-EIPdu&cMjxbn80kIFgxU5AJzXx)PNk#hWz;F{dH ztMMYIzUPzn0y_N4@cDejry$n) z+aGieIMLDmE;d64eNGwGG8FszL`O1s+Vyl+N3~^0)|ATaPluO!hEK-z0sTb^aL$AG zDm%G0Ol??*#g7rM4F`@HJoU8eXS#m-~{6pJEQh@-(J3HU^PwNbeX;b%$s@0@|S$w`zJAB^|b zA1HUw=Xrk88$V~PmryUJ7%~91r$@g}n2wKCvbRj^f*V{qm&d{3rgPcJ8&P?6FMVxE zoOGM3VVUKuRLX+zEpdYY%Ory6+FL75OCAvCE6u41OmOB$qNBspaDJEFJcd=h-?pJ5 zSF8HPh)5fXZQL{zbe5&m%%A?T?BaYHV!)d%5a>Wn%4)ZzaL^l*ewRYLMl9z}gI1(B zH=@r`jQ_3w2R;djX+sx&Y(G1%w-R^&%rNi#t3E`eQm>$4`X<(ZHq=#8@sp`v>Xs`L zoOWQ5eLL>;oCe9aJ5igud4C9D;=bSH8EraMU7Mv{X#`Mz_;s^VN)J z@>5@SlLn+KtAIn;T|&?ioXwxBbfzadil zrE2Mq;PjW!-E?xv>+>3QJ_=B=O+*6Sh^l?>L-xm#KejY0 ziRfH2Ksiny(pmp_Sc)lN*q|qQb+LoJk4xjrnghBWg%N!*p}(#(fn7$G&~PzRzuHlJB~kb)rTiQnp5~tAH1eN0l^9)j?K_x@At4ZEZ~i`tvM8 z#Ni^_RIy#;xg@-o@zOONs2uDHZ~q9t25BPOb2_!=lTV90)GpzxB459KtKYm+uN*2s zERB8{s`Qol+&JM#_-jd#Kc-2)*;hNj2hJSFY29Du?d2Yu*YM_O<@e^RHE|l*yNtv? zGH%L6D5|;{V_d}a)EuF=28HWUsV7S2=>0nXP1V{6{|!kANby}Qp@!0y#K6PthHFyp z=EHb@eBP&+yH&oXDHbk&#{285&dPDE0m(Fqe9OsO2x+H1+|>-KQy;4GczZ!_l!>Me zW}DpkZT6FFgAW$_!d!R+W$8yRc;W7|Fqv1!B`e7?Nf3v(?Gfprzw0k-b9+Bbw%N!u zXI?Ef5@B;3w9P!-lMf%!T&Ed(^%liaypf~osHxt!ilIUHpBZDC*Pu?Y+4!Vg)w<;U zeZ7Wv_fVR+evvwgFQO?d$ZC(riV>W53x8w)2#H%~MY!4d=B zY<8<69|U>Yu+~?7Q;)zu@-$Vq<3>E%8CBa_jc?!*@HYHVcZL3wYl~$OALS;!l3QK- z=eHLsPVDD}Sf1JWBJZZ}FLw)lg`aXa|jBfY*&Ds~g@`!VROR%K0xE#+Lh^Of@lysz9mWZe{$S#irn?@j< z7~zMF1^oUnzFJ3y*ikEEGKb)R2mBTbdhc4=eyc?)^f*P`zl_O6EKB3x8% z&%pppax;voK7ZUVtu0>I;rwxd!U@V`WF`zSdN*9L9UfXI6ID8NZkNL*RW;LSe~s-T z9}#VMwVbxETU5lJnsr^9C$$9l_%ho9StH38hAtm()ltM4()E_Ee1%mr9%|h&JAT~J zn@KPkxAAmxTfoTfzrkRW^M18qc-VJd7lA229mA4sq%9GvVINC<_4{SaX;#srruu5^G9KJ}cz2^^NQVZ@) zU7am=7cMepoCW9^8Gh-j0YwPG8@?AZj0&N?di#x-A3e*ycRB2%My>eiqI(-IEC3g> zFr*;R5)c3CHHrd?}?|Ck_cbdS2|Rd+Xky>0m2spF+9h%hDx$oe86ondwU zh3W1bO|RP$%|F?wBp^jE$@&vt@pg5FjJ9*~i5!5)s%XEzid#0iix1;X34UI`i$T5| zq21z%`8B)yz&nW8d#TSmcgbU|WvH!mOIN@U>-hYQ%V}A@r)lx%&7YG3c271D!~&*$ zVN&z64WcNVr{`6kRDQZ<2`V_dG+Uu)-stCbL;m9V(}IA{i3WvIL9#Cjl<)d=m z;q?%NqNV8y<08S3#gD)?ETLfItTLoXi{4Tr_@vWH1I;m}TB-WrhTH+Ii+)^Nz5&^q zYRXYn7Wnphi11;W;#vzmGQz2knL};4*GyWyx4f-TFfbL8WYj$Ig=Z@&?P>V#;Vfx- z#nUW*O`A$0qibCWn6F9wRTA%m-u!A3A_{Etrc{-;n1XjsDCexZ%0l!2g9PbEhqoY# z@LKh~D=x%OlD7v;8(q#NdQlFR3<8?(u1xl1VyM~c)5 zoVQ3BCi2&tO&$`gQSXF_fUl!ctMoBlYIbFN3-_(u9edJ>?TI?si-}|51g zSAkvs4AJ$7VKtYR&HC55xyuNONA*`DL&%W?eMlo=8vENKD<2e64g@fLX~%Z`L<;7S z&GnyH0WvR>HT?Ut03yz}f`YQGFw$uul~{y1b+Mj`xUKD82(!oEzDluts&~>9?IHYq z!iN!YCpw~oWzY->|cNs;UXw zyJ;~33oTm?l+$?Mk@;s3QL<3^hJRn2%Yr27bh{d4S#Q`ZEIazwGMGRw4xgylu zcX_;Bj1zSj>#_d}G9#(HLK$pqo`9m{2uPu<%DvFiHp1Mp-$UiePAOz_q3af9!2I!| z+2cydK(&W3{UeFI!0Iu;2yh_k*2lD^M8HDLCF0;9u$BC7HJ)&^Q)iSQDKT~mvY=3r zkql;Tkg4%ZWLvAQ-YS7^wsfv*4ZCGub9nl7Pm*5A#cXV7iFv7Y_n01;_f9(>JvOFP z{hsb3%zu7&PqH6AHdKVaSpCSJmb9LsG8(6KdQuP+gv z@FBme!GxTEIuAzQe(tRV2-j!UneF#5>TG?UTKY@{-#;h&kRyJXBb%B`zFS4vHeyVx z>Zs*gdudx(B>|DW5^4^(nFA*HGB~MnSLU|(@X+iK5Q^VJEBfHmm_;uC7~RcbH{ef{ z`CR%XEN&VPt5m(qEX8K`*jZ6YnaiGHK%saG$L<%QpiurNapAI#6{{Z`3E0@!4t0h- z$YzW|o&Lu3W4Uj72NeoGmsF|836Z)7Oq74sq@$(|rJhVz>>rsg9H?%ex}&QvS34=4 zt?bqJIB+|z+KnJ6_-l4$Bt!Q+!93^Y-OlAFZLQy9`}-Lc*%=|Jx2l^oaq(ZP$hf^P zYIJf0jOmWYkcJSqgUL->8c&PcBh$zt~h>7Z^9r+t0zFu@Q zmX5fNFt@reNKaz6UxB!FHu;){S4OOllO=@H8S{F~F!denvQAumB=m~f`=2=d2+Y9G zF%e9=Rsdr<|1tVa>c~1&OU0V~Q)#stNi*8lubR<2=;$(RpN2+>e7~XOpSE6HFiiWZ zt>jhn>$C;npb&HBdL6IHPYWn7{w7GGQO;Gnyjk5J{$<_Z{Rqd`)2 zoUU3YGatuHk%dr^o2#q*_O!TPHQn=5Ir6Wn2!lJ=Z|Z`ye9oC|Ru3Hc0HGmho_PK)G=!(>}W!u-du62gu? z?a-7`BrBy_!%RKAO|jlPvY z=ag;xE3#)3U9XQrH~$R{nqbzP29h4|>Da7#j4$O{sEST^0*?px9g0Otdgnkf4PK3T zVBYmNAQIw8?1ir&Xj?D8?@l086GQ_J8mBKHQiy1gxv|dap@*|}AGk=Kh-v4hL%tsg z2J{3_SMkm(vVH9SnW+-%@+tIMTuXgz*^F#`j4$e%mjMVFo+^5M-6IXK05M27X7CW5f z_Ufv57%ZKTSIbhp_?@nc2>$PZG`hb8wh|M0mdTu2KC6J_&-TMZ^eO!Z6cWEzNJ9aO z%y4o>K;^vR{5|K4-x)pk$_yLO+q1 zk)KJ8Zx;0D`{!y#1GaFoqZ-+cbU?iT%o;HPlQLi?mTMT6xCfgSn}L@#(;%dgKdDOWsMuDd9_*q zIt}-|JEr|xkRxDRr@k)o=N%L18^BuUn>h|B1^}5AT+mEmU&1>WgXQkRfIf6?V4uMW z#+}iWQq|r*I$@y2_@VT*-8`ABSwb?SJ}|Sbd+}R9z&l@ytzuksTY&+)*-s<?4p07|#M4u!^TAS77#Ns8N8VOGYWC*~kduqWh%y;1 z(M5=#I9zguK&s930B(JvIZE}G0Ur1vgAc&tw*$6 zguu?nThU zvU|5ld%SL$GsKt2S@1iFOwk&rDA1xODuDTxGt(2+UQBDg)q$}Je{{!4-uO@WKMmRA z$&L=EuLL{~Q3MxQ+=3ZV`thH(B+Gw#Oy;)BJfaa7bal2@!I>b+0)xP~1fA~jkdt4s zW%ElagWUeV3h%d`@)}lJ{qr(8o#H(t`Da@K_i6}6!27{8niRLs@Nu*06Yf?@Tk*sf z#QClm=I8#;VN;N`twJv)(gVEf==YQG{N7lpgu89%Ha5a?daH!P_Q9c2twI1|*qzD^ zgqMw;T&%ZNXK1!t)44uYTDh={s2{)G8FPi0Ju%|D#S^qyDq{qY@wAQ4nrh%@;r?W8 z5VAKs^7VI1CUlc^?+mnZ-c>V&Tf#Cfw(EhCb{dI%&+P8$jX>GW8}OW@vY(!O>7;Lt zD=B27N@HxMJ2H~l%kEnHBe$6jX1h31-Y+O?W4|tz{^GA>A96%BPgB{B=_3h*n~AyH zih`Hbhgq7#CYjM6UZtdT9PGCNp+lTJ=X|04B(Iv8hrYC2}cnL7v`X!fUR%( zF`Dn9F7O!Q=kzcr_E-RB0GvrMnHHPaWit<)+Ru%#t6_t#hR=;i_O761z@v`k8pI!t zF&U)}bV#vc`ayphlBUy-9~d`B0Q7JItqDK#GVWkaX9L`vBY{SiIJvUp+V3`!bo<3y z`Roz3?#*eH6Qp$;0elH8XcCGMoc0UDIh5Dz)r5QBa6CO7e;=4Y3=J@J52s9BM@YVD zH72v}pr57LTL&^Toj*yhIrps>Y=W0ACYk3D^g8V<{YK6c#d~|oJ?QR#rRd0|F)GFJ zHr4q|%h4jclYjWw$J*(#_Rf9o*iA8dH%8C96m7wVnJu$FpMhMrYn}8r9Wk$0&wfXr z$x5-JHUX~(*~!8c{BTtqc^#9QkdPQ{kU1!X3ab5;KxMFd4V8_Yj^_c z>RQRBxKLqD&Jy7)x7QYA;x@r0e+nsqH6vN0P;|++{takb8*(V>IUh%8(4 zd@yuw=Vm_~SBUI6-=*ljR2QgC4hB+Ace$PL?TjVlSP0XvZ9UXo=dI3`TB<*j6O{=; zqtIZzhz;x6>m8zf!%&kWyP;z*>C6w5Lktf)1o9}_8bh!A>~_tZ=_-B}2QfA6)7GFZ z`u3d)sjXF1?^)SNZN2Z9o9(9@$-DGC7f{*{%$8s`fcsR8SWfs5Wj!1D2WabCtrrvW z`l29eW8h;;%vEW9)EN#z^E)qPFT>DtVK_q)z{Gv45^^ySU}Ng)->&!TU7( zs<+u#eH7JM&qBbf>3)EP1fI-<3Yw`~kI?tIZb>eh2`Q|am0nNL0T?hqIq3Yurj~YA zV&LP{*Dv;dceW-hL7E=jEC;5FO5foSqi|hsd>rnbi?knoxCbiW;(U76lXCJw>CeXQn3%a_ai5G0=OK?fNm!*$ zM7p8)i`)ppvv+rPBV zl~_%vqne+{xW2UKJJ)*NHtF2Zu{=t-zx-pz%II4; zQmUcwFD_i?Z!^NCdKZ{d{72?aPqVVaegLtnTP} z3LO6Pq4r`)l~OAbzrXEH-TOKa%UH-8d)st~%>k1hWmxQGNa4uiZ*UwtE#I7|`2Bdb z$^D-Hn_NaqJH^hM?t~R`RrJ)2iweH#74CiC_sY(i+f9z*mFPLP{Td`lt~8VQEhrL4 z5msK%;t@w@J6Fu`1R*96mRGU7f``9KfMp>ae=jD_P23e7B_!dXiW4LfW08m(DMmzA zCepS@b9Xa(PvSd1w-m3C{nWY891fekajW*h59Sthxux_}n1n=eS#9S~8h!5YaF)P9 zO~!GzmfzW?H0*wEArV_pK&z{Go6k=m*^rWQ6~Z&jxpesixKytMX!t?0dQ&P#MD?ip z>pyW*(7(*&1^Dx0+z@oktiRbesvVUv6u8C(da#BUJyB%JpUiBZ`5w6>E7zGECdc&g z3@mc+SjOI&?QY1*axz>>q0s3=Yks%ZdZ)ay*m`y+ru{Jcc_@WF=i1qFyb_WB9OvM! zdRMATMh+Vn&$3;#}gx~B(mQ1{`j+>d@z3Ekh*FIaqsJp!Puog-?({DICP!_ zm)V^-90zjWpJrzWt}5WCGWjjF#boW4gHYd#HBas7rrTe||HcVKT;D`|0{e-I2q3e_=XoMfToi zu?=cIDJZygWAxaqaZWa!s zC&Y3x;7*8sweg9+ZXE-xnC8XfOnkO?gH9BRp%7`II=On%tax~H#GKr5LH*tydPQBE z-U+L{BuXL(8;G+jcvc7O7Cs64J>KM2*z=scvdQNo9dig3Wz4bTiSMe?qL7D#HYhAK zamAc3+(}4lvUE4;ZMp6#!uJ+GYTjLxr~Am!!O`Y?4A`qZ9u#aEcUKa;dBoln+&9}a z=imTobggxXY!oP_x>391!X)A-0ZuF6_Fzfo&`{88ao7?IH8+2y!F+k$jmXUKBJf>! zk6ANz{G2vj@Eo_46XVcT?`ak%2X5&(VQ`^aIB>K|MpdK%fs1Tw9KJL`17V5?AC>;# zE>0939IT~2vRaoAVk#fp?2?qlB{|E~E_*7J^VDzo3|E~#$LZ+rH4{f;UHwRO^y|~P z-NW&w!)B3%rSU6V$^{*qRpY)?$+M3IYPm{TPgm2Tqay#rSaskOYBvSFVHM?ny>`Ye z9&PaX*~EG=tR!8E$EWu2ui^=oLPD8NcFbPd-pY|fibCN>9>`s4(o=6>du?IJLb25G zpY7C=a?AX4$>EOKjg+NjLNi9*zBt zm2$|!gKGKk@bFv{zmCufd@ZAvNi{^7(MZ!C$WH69-C>f+Sgx`fj1b3(7J{wbnRocZkwD%$~PizVt-iSvu0lx8WMqAc|b zE>+P=pP8m5^HnX8E;;{I+YJ^Ti z@TQ~-&wFdc%MZJZA3j9rd1lk7e$bE4yNrUCDC#gADQw+pX=^ooWg}!GEr(xn1?!tx z+Gn8n`K=8v*7IDhK@zR z;t>DtVj+%e-;c{M>D^gBr`J~FtM0Kkico{Y%12bZ0fXGfnu=9K$%I+wKH9a@`FWz%;PIG9nO>ngf>3r^au>tk7Ss6g&5pA#A<6Y>F1=H@KrU{W zQ97eq4fBxhmwEHMxw8~XbUi`faG;3tY(1<@QI_E+LGLsHB=<-45bVnlHwub0np^$Q znvhymU3x-oaobt>6AjjSCSOL;%TR=sa8pd`RB$!UfDId8kkd!w&X+FE6aAf zR8S0L60aLk(?-PQ-YZaYNhRfZPVf?(8s9GG>x;$ZT6fsRoTL%uY&VQaZ2=G?Me$0=lwnnI^-SfBTO= z2$Q>|`SSE{IYtXOS{}izU^R*?XH`z;OL4!FC(^Io+~wD6jhE+N(0j3ZIew^pjqHnX zoB(Di=GdK>vYu*2uQjorsmg9dor)VEbs9A_+$wLYn@CXZ*$qPoQ5{ql1mt6j+J7`( z$SRyWZ_mu0T$}PhXDaUdMWv70X#tYwu9Tt2biaz5Vi;@B1Wp$0jn<+51cs@P{4_vQ z#r2V2vh_bNF6+A=%kXj3*7NHNRnpl)iz`sp3Y&VSv!QQHk46Ui__QDP`40L1FF5hIB33!4s1U>zZ@ylh>eafR_c=X@uw%8bY5rwXJ{xFOe@S?mbH2D0H zC*(+T_2d<;W}KFboATY@U}d!Ly-#{s^7U+%<~!c{T_%nbhs^P*Lbt#<3vZr#hXdDB z*p9JEb)#|L^>bdi^`Ab67dd3DcgZhS$$k#jYBs?NzZ6>3*dR>is*TgGi!v(ThrB+k zY|%CcS|XHtxoIU=wOot2<#|^V0ZfzuNts>(-XPeLTzbhQFcqvM5O8gGsN@#cVK4gB0X`9wQpH{wlHO0$;!3rmWxEDgqTUdTOyo_dx0-Rn2<8$=8Z?-7;5Z_ zxwgdMrW?YeU*y&@;e%5cglZfmy} z(ekOQ^6M$VzE5Svl6OVwrAGE|g|{`P>8&VVei?fk7y+U3g1c@J<@WaTXm6>B+>hJ3eEHqW zwv!g`O#(|Y>O2&w$fiAX-x{a5fI0g}}`s#@1tlNe5TQ;_akUU*1v znxmJBfawt4cuv`#0M!S$<6@JP4W^-(zP21CJ9{kT-ZkdUWbx37JaOmYF?Wv(#GUas z#sII_AAaM!ew|`nvHv-jZ{?0Pnt;F-{=?tX@H!D4S%48xEw3Cv$*1i~bd!^V5Y_Z| zJj0de){Az^H6MkJjGhstML&KO##Q;gqf$oQ;B>lNdmt;xbZBQYTCaP9nkkc3DxNm4 zsKKMVtj|N=`io%HU=_ZvLIZ2AN=M~sCK*?m#L9Qd>#ZYP9}a~-*E9@3jg_Mw@TN>^ z`M6oW`O44Xd!x^1Hr^J(uR3R~1`FiTr0C%C6mGwVI}#epiRcY_&XRshu|2O;AIdMK zzS1nj(r6Up0rK{1asQNbFkBC)%Db`Apq*jhW~>H>MUJLcT&}o*`+$4S?4F)A6+tv& zKhI-e0wsThsAQnc!up4RR3M678mCQJ=AN-h!R+3O5hZhiTKGr31CmSU*wkhe?jNNS z^@zE&VMBg7S;noDt@cbqD}{1{SAiDT8+%g_a=<0TzXb6B+XT`-HU$5@unwHPcGxLK zq^c7CJG+i4CfvH#uT2ZXxp%!Lf#Qpwq&tukL0hM0v-WvZEf8B<9fRCW1Ye|M@mc&pMGRt8)!k@$$F>xMAM5O$^bZL@NB-m(M_EZp9Q=4Ld^9-g0? z@^fwfe4qaye=_wxJMKQ!{G)jn(E+V>fr+~XB(OCtO8BAma9pVAKDSYJZbJOdLpjjF zxHm!6Pp8d292lL6$28kb$J$SpNOn9_yHfwPf3;5LyX5M4Dlr~D?hyVhD-}cht`|R!G5oa6c6qo|N|7#?}wRrHh9Y zN^Ph)WMMC5;bP%#I5>F0dnw5@{c6~R_okU`Bp}OPv6>5Sl=-bSsItnQW3Du_J$hOI zm(KD~RvYCB`6Cpxh_@&_5LaiZv?l9;McHC(y}G8BchtoDf*fJ08#2!FXqkz+ONQSL z^*^CyraC{t7kvbBmwL2mpYHNu==&pZmh3O*!W)*_<>KT$*K4SNX^l2p9_2Z{bqmCt z-cY%#@}Ov}4C`*}t%8n@6~}^o@mae*X;nP?L-+N%?LjT4)x6pM^+8%`%Y|3Q*`N?v zMG#*Ia|#0_xnz5-dxI_s+>hGq1VvWea0wbY)4Pslf$=xXd?%wggt zi#7>+d_;SHzta$a{VbXJ8mV)QS1eIiuY@0gu+a$pNTgFS*tdP#)KG-#Zl~hZBcbl4 z(z{qH-iH$wk?T3VLi0md0m0W4}T4-|5J+!3p+|_+5ll>_O`t#;1gFO%RVrM>t$4fhx}< z!}7!0pz7|OxetyDQSiECbsR8KgU@At&mzn8!8EGL=g390p)d0f zAKvFW12P7n)o@N{Y~^YSZ>2X7PLRu+4Bi9kuS$XxeE(O8^4#(pf7fOVL)yemFjjTs7l`tn1h$=T|n` ztlF!W^-biBe*ixDEm)1Kp6^uxudAiZTt@)eF0`3Rro@3PG~|jfX|^io-rwS4%;KUO zbEKF=CCNJJGhxg2y7HOS+AtWxpog$-NoaQ2xYuq+?|U-AMLj#Ma@Te!{&@ud9@S5g z>f4E%%25_E(sCAHtpq*pE|(g7Ey*jg+Ee$^M-giIIH=UeKFUe38Lq0vZdtU=o6 zOfJ5?9k_}!flPQ0_(GeS5FsC*|J>(UW}BgyaMu{b zrEsg-$bc9JC_}5da-=^J=N%{KH@!c9loJAXD9w9`kQGa zeg@V~`mEMj9UvSF42&6z(sljnVu>Z$vZ+951nFZZn0fAiH7?O>I2uub8+Nj0T{fLP zG{dnJRoFKu$0|Ae^>|0IFmJGTQ5Lv90s#Az#_n#puKp(zm^xlYP#fzRRNa$w;&E$` zm-*{`Ab7LHf0Y*AKj)DvpTHYtsr1RvKDvI7I;a_qC;r~UTB>I?D9F%&S4WE?;M4Dm z1;34ozUw{FQs-+gnPZMcQSv`vn}~}2MAEN76uz5xPsBt)`;^g+!Ii34)bQXnfiqIQ zOEc)8Kd4kZ7@>`6NpiR5sU1|(8}m}-G15$XU>OaHSvQ-gv~<^WJOwU&I1h zAXgv&@6Bka%fqS7ns1#oRrkrBWlz=M00@igw_caV)>=}t`nqacLNb`RhbC^MfF%Qi zjt~=S2s>q;s~)D$!G}GW8e@M2{67gwYER}a&T~Nb&jN*abq@}-1jehwhN84-OdUa1 zH^Y023qI4OWxF1Me^OnLkRuV?K1X%Xgb&{|;(-ityDRb<_x*EXG!T7)PR-CvR%Kg! z*t73i%_yCPoYR&iM?PzcxElI0m~*`R}PGOe@RJC z^W@ZQ%3lwu#>X$sY>QwNP3N{wwy33?J|d=u#OYnwrjL_it1b7#nUXhd&NT(X$ySdR z6+ZG*2ZbRmf%gg}zam-M68Uq0zp$n99js>O5?71oNUe^njXXsy?%G&i#T&6+16Q_`#^+YzP7&;`SZb{P@EufxcCzwA$s=y2zP(h^6A1OcuQD`+BI^3#9Y~?%V zF^6e9h5foILmTg>kF1n4Os_DiHJO}604s#xpyc;nm+jTo?p%5*n>3b-_qkXF{$nVm zmG^6j)(={a&}Tdl-_~1SdO>AH0RP5prg}>c(a)yYe9xM^2dD!^dli)cpZV4vf`$z> z_%uTc{x3sO8J5HPL;J?<Yg;+?OpFwA5C&sNNxj81~^Il4GXokOWNvQ;03K8^6c-?3SO(t%V zW#ADz>0}%^-cV7Qes#-O2{pOEO>QF1)Ft1K-<%R}%Qr_s%ChwmutJMqhXG)PEbw3y zXvyBEAR0g3RJ#_}RQK9}Gi9~0>|Oa0dY)W)N;Lkgk-RRj0w*+LwW>{6E#KPRT;|cZ zqb-9rHpxIG97Iqul%z4Igs#%`|_zy(UW6&zSR5c z;dB_{K~wGaZx!8zJ`V>SWn3NCw6~_?CRXS`inXn=?*TE+^F$P+BW!}XnVB|qzo7t9 zQec}k75iY7CSF3;Z_;5T87J>y`@=3xBt1#iJEsCo%XPvgDFSD}vd7331O;5mi1FdQ zibqk$MOv$W#IxZQIG`U?qtKfBpIRz6b)-F&>kkuJZ7Ut?aoiBuU_tnD{3XP@(EV`s zo(eu}4@wY@{E zy`HO!HT%)21d`#AM}PWdUD#Bc+P%b*;Kk=-GtRJxL@u;cVNb)mBdFm*{?#mH*NL@- z?g5F8%ig-}6FtRG_y;OGsV1pAb1WBo0wnSe>{&zYR>)PjivR_Hp3}QX;L(3%ew9`p z;E^a?DDv1WZCRZRhR$lrSK1>W%DK5=?p{l3)r{Kl&W{VQ|MO06xwP&b2Sn%ljFo$y zy$u60fJ5totRqi0v>wZ|HmVe|ZJ7+e%3{*S=l$-jJL*;84)m1r81V@E{Vx|_zUPcIS&9C(c@ zDzV<GHRJJA^^$A%_Xng9Uf8PTK6e@#Qvlu6WrKISUD{Y1kjL{RvBv3K)M200e`M z5wHOAXkNeg?#7NaZb31sX=fL={5(C@nbffATh-aj_w}`Ib!;|!Xli_O|6n>rcz68z zu{FGGlDxsk(96)fM%(`k&>{hPiFLivriOXnLUPo;&7VYP`1FsTCX<4sV%m<@%J!0j zdox4`&HH`aVhhrtT7gNR_5f57|9Q}`^$BuCVt(yJCkys0@OzOci;*^?;|~{8iUlSb zaZ!Up4I0JUGR=njk(5SV745yqG((KSXW|WGpR3y|&L3{R9s2%Q*q`DNKwC$vI`1S( zYEBP%Z>?STL7qkL4!qsQBi<1AJXX-3*g7blIqBJnXLff)?fpabUs#|3*5LUj7YB%O zk4u|k1EZflYN}~mJ8Ei5Z)H(u@ENJuhfLBX6t>z5bnHaxoPo}O0QgGv>#?k9f2LzJ zG`f7otX7vujjs{|FfO$pG|yeU$DmRuxgk-@E~+~zN;YY`m2_x2qIK%C%OL+?wX;I+ zi=RdF>SAqT7J%#k;HaQ2^UDR|xFeOEBl2Zv{Mws%KyZRMVBFf1X7f0}Y?cL{SVqPm zB$%v=swWgA~OrN1iLiM_J?`(5&;1v!Kuj49lX_oaUrBw;(>PTA? zqVOG|>C>L2arQ&Iqw5Hs>}=14Ao`~xeI);BfD^@l)HH<~i8$`E?asRSGnKCketd4- z*B7!6x{)Y*Gv9idWq$qJAu)AExAnDtEInVrYh+9kpc7Y=3y4=jmWow-Mud(79uUlP zA+o?Lr)Y_&%BXupO_GikJXMPq{13NbMQ=#@CErAROA_LPP9d0~-j<7Fe{C&;HsbvvTtMvP&l;H?g}e`MKW)kzVzbyrSx=`M6T8M5IozJotjhYl$y2tH`DA zk!C~er{z8Gz1cacy1I>#^5(0Z{zW#?n1d65$N}$kkOpwd=1Lhz2hrkSH4dHTwDrq` zWb&6<#tw7S)$`{%oFjFmKiOZ_dBOD(rTGeokOIu0MKHJw!!EgHwfo453d*c8Q@m#Y zw)-in9WFh;FI_TYYjm|p4QVvh6qf}c!ShrrMmJpFlH0Px5MEZO>IlPd>ei4Ulg13}nl@x#rJ+tNx`r zPmq=~|ANb2wTvi!^8I|XEi*E6iKVqCZ(L*H-2gYTjX{%9?`ali-7h;e6I%Iq@Ha)C z;7ZeYi(@cBVMPe>0eZ`WUVrK_@IS~F|MvLgxJ=p{8_)#?sC4KAqO>O|8%!MEFG&xo}dAePV4+GlV~(+MJA9koq#bPK`H(opw-zq7s@9oKOU zFbKHQM76Uic3;kD6gU(n?kQxk*4?woO%=d8VmIX|=VgjQ@+Tn7-T_EuCGcGU(C!*q z;!G06c45-6{!qVWs$?L3z3L?slT%N@Jrg0(UcGOzI~7s^-@X~wGNfGy?)Q&&aI|UX zYv?}5MLsJD+sv|vk7I*xNK05DFRr>y;9}Y`BSmGsZ0M6IMOzsD|@HE z*kXa-OlvxMbbR@MuIGqL=9at--bikVuMD)Fkcc1)IOAL|E%2l8G6~3%X!in!6+AK6 zSvLTj&4@xzi;97-@d?nV?FL7*G1tW_{39`cvCXWD9_opE#OZWCH_35uWYYi3=;PE# za-(y_Zu(=xH%`lXzK~}bw#MBmVEy@tUORB%{q$i@5-R>1%Ft2F4;7?p~C9~t!MAMZ3Z6P%l0#U{+W>C zmxlGyer;E)QWL9MKo{d4;MrNCzr2q=Pm~-Dm!FdF_CUPZJvCMgxQu^S~m!Ss14r&DU}su?K*Y% z6@RwvhbEtS?Vfb2HJKtyhc}~fMOcZ<;H`=(bTE+%}p^$v&$byJBSwqcIMo+RsQgUskk1Lwj+VPG_ zl}jfr4HR$6v%~P3C=~3#b40s>3_VBs_$DF{!v|Zn&FjbXCWBq}{NHUqY8x)dbnyyG z5UX3&z{?5!o`nHI11pF}!X&F*5%A<=f8jz^k7_79uFub_C!oKdg}#h&E?8S1N~ae5y{Bw<%Pu(@Y)uFu_bbpMZ;|0tAvUd1yPlt_lAQF4LGl z$3XR+qv=qia?0m_4yyF_OC85F%<{IrbFiGfqp{ITZ0N~`tW<7a%@e=J=k{`AtKk=0 zUl$#t_*WLHK0jsrPc5{7S}=|4zTg4!sDZj)W=bBuXgPP^luvWNO8Hvhd}X~E0b@$a z9<6@X&D?DRfsQ3Pb&^bfc~~@i4w9?==T-9*{)+dcfvK*qj7MUVUq;3Q()DiYI-{w0?0zn{fmCf-s}M0uQ(H4K^3Qk{Xa?{}Ct?$*)uv{8#{M z&7jA|@~E3l9YLuVb%7f2d&LZ2S(cZYEy0TiCR~#Srl9{Kf;^fYRQ!fG|4_NEX0@ikL=X))VF=O||zDAN{>+m@x9(ZCg84t!y z(vUcE;ZG#L$VhJI&RGbmVcqz!WcJ^Qi+ubw28f{r);dV8OCdQqoc5A8mCz{PK11Vy zWXMUQC)EL2c6LGSLz}gY+XDX}9EANE>23k3k5D%PoBf%A3K9(!#j#y{(EgR#RHfnU zA_tJa*;XnBtEM^is>0PAymZb>;4|Axgs++^9wS}646TO2`9fuZCOO#KzU?GNXI7*I z__GsQ4WqNr2A-zx#N#Vn<1<5|RU$zhHBEm6_cuo3-5$m%xoCT>wNd7;c{g^C=oJ66 zh9)Iojqr*>0GC4;{q8R8ho^H~$z(kqn(_?yo0)4VOKBuiE_%hy+AxqT$ zi?k=%#+5z`)=4ww;i{jbJy3!JX!l130PsXlR5)M>1#FFpt=0ki5-r1s)kSZ|U`?1c zTHsgKKhC;6&{W(=f!2KhI6jd0lK=t_0}TwAw^W?39@dT4o%5nG4_g4xXz`OkVtZS! zL+WQ%s0yN67RJA#DkpT5?EHgS5F}>fk+|t8w5$U`b0VNd7Hb6G9P2FQn3NY%OEAbgqL=?VHj z`LiSd`SCnoqyS4Vd1+=@_7Fqd6TEHI%n|=ueJji^H*xQ~DUuZ$aM8g>hI;=jS#620)r{*oDaQ(7GmWu?@xD~xQ@Lswq za1yAaxn@@JA66Cs#2Qo8^~-Yr6sSSZ)L9h6VP$NbSLv=cpKF`mI-Sp|cy%0;7(DI7&rUDfi2b}18D#U!c5QMAwpwIPO z{%wYFwGD8G8aPYjm$Bp>YTDZyb&qa#S&Fvw?VVFdBL9Fl$D3*Kp8nT1+$}0ihz-JU zM0{6A^Oh4y{69yC24eC>NFeYvbHbUpX4Vcu7TV`rHCkB{$9p`hFB_w=p@t)zcSfFq zsO$LGuj9`}f3>40roHuucKPvpS@XVuS7wslj*=ai=xmAqsEVz3ua?StIAot`Y~DO|L#~ehxZd?L7O^n>SDRD-L;KBqpVqRSUnT!RmwV#t7RFAX zA@vBPjZJRV^-7{1Kq)+4zY96|3V*58Y)9Sg>-e_D<74LIzt%U!Rbx5&d;IW)nVa|r zWPL)*5BT0ZN$sC~_S*R<*Pamueozeo5L^+$q8%)i9a87_t=%EYapK|{8n0DVrgT^| ze9BDa+TS~GSU`9sJgE}3wkGA!`k&Bf!p|B z)g9-@x&e&Fsu+5RKN`5ViuZ<9IAfm!@7APJP}$MRWATjmqdG5WpLe67MxW0L_*M;A zjlJF&A(g6cCgmD%+f7uvdP$k~)n44}#qi^fX-=K0YUsZAz~y$6*WoR{8ICNn$asVT zG|C2H|KbJ5FH!f}_nKR*5@O3x?(&f6M9*$}@gTfs*8p`M4Sy&{S3MB=Xh;x*l%bau zrHgkFc2sJ(PP`N1xj6qpT^>0GlRz)Aa;YQPhjTFk;e+VT6+IMa6)NVZ?8j6F$ z{LeQLKAS7J#f;vC`kaDKWxiNfdw$!5S(io9FD{ z!4=Y0Lw(EV-&|oM(1ZXh?ty9VXW>(v&!b5S8p`hug9U=oC#A`B2)g;j^3B9r% zfVA9}4%r-|Ad$ik+fEl2I(x2~*8sKE=lvSr)F_-R#|8kd2#`ajhC+EN^(`IM8a5om z_4iK&J7$>CO1?4GaZY)Q%wusjUGVb3bO|dwCjS*s!Vq+9=-VC=VPtRT02nwmWkloC zNb)*8zj)tJXJ&`t1?QENv#I7yj>h*lT*U+ebQ3(h2p1HwL3r0-Af~wVwcVA|xe_*j z$BE)rz*Bb0l)x$J(T1&?`dWIbqB8-RNl#qrGFK84UHndYBJQsHs>>|UWq87nsaKdB z4_R$w5ix@RmEilC66uiatPHB(K@T+(R8^WrHv5$VSmposE`a!z9bLFaTN`nY=S_U= z@l{>?ok#ucJaW7d9R=3Y+k)5=o9fxZZ^K=X#kH3ejWvLvK{S80Iy4Pw;$^Or{-_L2 zmX$D$T*#Ld6g45-PRYb^iqaMFKPLc;>Aw6cxy0q|P~ z;V(VoGl34h2W9(_4?eX6yR)FQ$81VPO4>E17bd5#NtjJV)TIJ@|IGtN&}5SE_>@;X zU81wMLaX}cy4Nw%>ppi|EaI|*`n+M?ovjvaIqLl@8tq5m_$8NNHy+3W zsVdZ!;$-5{uPi_@--OYSjQ4B)7l5utlK|Tq7A|`2y)rE(4A3oyNvr3F_a;+)7?|em zJPHSIt9XJt5023^e+Yh*0zAzf*lS}`G?B>2Y?w*>d9zURO{*mE#LSmwEdvV+kgfx@ zCfDf?2lxZkSWEy6=$a8bEG!=3Qm4Knf+~{Twlr6r59`21@BU+&yg+x`)~P-VU^^j? zP~f3ScKEqkrN@Y{{^YiR*|lvxnY7F}lAp#1Lu2tK`M_BYIJ&w)arE)X-uF4vG&8LJ3bjOtb`XQ?8O>~m@ z==>VdJ-z)^FX`1+_CW8KVgzpgl@PSJ%A;A>6U9l2;}o@f%nSDf8DF9vZgM2ZNDcgcP>Q@FcC^ZV;AYI3#GxV#p`Tlw7imTqP^lsQN~zqNIYP^CB~d8I{MqarIeiDD-3Y)~q|xJEf!;)vYFQ zc%p36RCG^_I#c|w{BcA2xl_zUsL$V!Bt^i`iJq<2mR1hORaZ^3PoLacmV1x$!JNw{ zvecEJ=%)cMD(}h*7j4?$Y!u594K&ex8nHt>7T+f&_Bf0av#HUJkwqIaL$q%JbHQX+ zZO;?_w$g_C*0EwL?xE7i6UMz-S3OZXt+CjremLtKn22IZTWPO8kYtg0Gb>|EClwJ@ z_x72oyaEkAZCJ952aDSimW!AgUbd%e+&q`p`i z-rc`JZ8_h#eqqa{+bo)QO-sfLQY12EtK}L)NrJ%81>Enht^Bpe=X4&}*j|qlmubtb z*={)dzZs)!pLe=YFt)qkZ!Fc= z%DXLKOTkPBHh>ztgm0Db?_R%;OK9Pwi~Gf{4$k5pHaR~0_}vj3;rvm3v9(Os>35Ds z;Y0!AyHMd@wEgtMkz@tc8+fy|9r(@koXUfJ=_8^3)8G~#LtU`UusSY={hkpdxqQiF z4&MW4mxwNXeq#M=Hr_z(riG#pw!hLxG6?N5dXxS4em(GjV5!pAOuEkW>I|76&GCxR zRg8CIuKn3Xe50V|ZT7Y>wjUbrNoc>#(|hVrQwkX+=8VAd%;Ggk?4vqMsxBkeC?p?F zvlzYP98%WY`hLXdON+$}y15sxnfHz1ASmT>63)7742#xq`h2M{s2=B*UNhb+mlp;! z5J;fH$gv6h;x`)Kt#Mz+9%|dVI+r!EGI!BF!|ft&dCi~ zHf?~f$waLfXcX8Dk({bExxjoM40bY$aZk(_kUF0-?$Ntf!<2o0TEs>eUL_#T;T;#+ z65474NhbeIk3N>vX7-`CNO)H$SV^yD#jdfW=ml5t6{=-45DQN*7D(nq= z7jow9nG{;GNw>7o1lMxp{w*uZ^tvopO!mX0^b@k6gX#B5-r6~KBQ&UWm=|q1IHeN4H z_MlRFN*SaqNo^}9IjBncFu6O2;Osh&bPg}78*~@@DH%vFhxyq!ya3Ho+MNrYTfq;m zn%u5@N$hb~1Hpbqpjl$mpSweA-q8qWYL@`0mA2yoi`?LKwo91oRLTCqS=v#QEyr$^ zaVAH-@bg_=x0S>tMH5v0A>=Guom_UL$3)D2kNvKln1ie>vgSPwKqbVpQl527RO1X= z6{j!InC6c;Ae!%jJ4@=>;2q`h1maE(9*Sgx=YQc;;?j`V2==ojn@g3Ltc0`08pFah zoKW+TFYjpu{I@qfm{6`F4Nn;0jAIV{55)y#SK4^(5g8lA4Ed4`#JyNZaw7iQ#A-0) z+crWe)csMM4J+1Aw`6>d0`rf}OOa!0D0r2h6pWovExL@{oyH4MHno^gZo{}pTu+WW zqeL_MNdD8F^QXtpfu}rrG`mCL7p0MBo3rtb!5>qj#!~l#U0nseC#S1*s@Z~L4M9z| zEq8^<2?tpkJr9lfC)-y)MT96AqG>7@uRNUSVWI`kQ|PF9{+uuEzzee zT-efa0k)rRM|*!<263sT6Ts=K+?(xISUP?!_UB8)VC=WnJOT3x>tAHBg>s4Ra})U9 zgUOV=&Uo|el%LS8Xo7x1BI{!+HWq#Q--=w=BxS~t?VT}w8x+SbZ@^^l%S&WU<)S%= z3TYj{tc0fi7FK`*C~&UpjNFxJ=U{VS5cjl7$%lI}SPSHPb0Jy0v(M-3D*>NuaZnc#+=Le>q3*p#m>ByRF7l0k({nCD>!`bL`a6U1 z^df({du5!C*sS(Z@kK@2Aj||MVnR6b>nMS(!G7bF9AR846dA~=_PkJ}fouUIF(?UA zE%M`;Lo;T7Uh1}c)AlNfRDUXJBtioNqTYFT`I{T!I8E|c&FWf$Yfw%~$mW87kcgS= z^+~+2Mz`GKL>`MppCL1i86@xZ#QM5q_OjP#S5^W3!hf7iE8=Z=9dfS_9P0cfsjU?l zjYKN=*KEyjuAr~+oxc+6wUHs=@f#^Fn0+zH91^B~w&3}x9EKU)!~IT4(>awk*^WKM z;qIsA)EFa+5T5>z4+;F_?g|Dw%#ihPstpukbdVpenYo>7t+yvcjlBDgarb*Wz3|se zoL;BY7h&4}dF!VUAW!+^PA9fmmO&GUdmawXTd?E{93Fn)8WuPyfOg)I#K3UaYHi!^b2bFA=_TL8rHlno8D`J1 zJoe5?=Ok4R=oKqlS}G5)OWS89&s$5T#(l%W7G`)1K-57;95Hp2Fs!UxWmp?`Z%b&S z4RHf{@L%#-&Zg8}%T$7gg*~~$&iPw~%lb7_oDGvWd%%F5`p#w<$}<4rX2O zFid4Y?~$g9WTijrzUXVr0f}r&D)qs^ev(wUmbFr=3(0$eMtUUM+ zs|$4^tGH}5QdVl90X@)O$cP74TJSir(d3&s|LdDS788)&@bFCNga=K)JW+{26DX4} z#*1nXw@^c0U$pRKI4m3#)ZAK=ym{ASU%PkK)KYIx`tvyDTd2NQ+1l4W>vvW=L9GrZ z&JsoDGWy-0X4f2q#!%n+Q=MF}%Nul8C{uC7q~p$k|f%rgA+2@}i4Qw}{;V|BID`m$XwQ?eT^c;)aO&-CwV3t%=m*hZ6sjkAUR zU^1IA#TZ@dlaN1(p96Y_oma``mktyO;BsUq2-reE$UU{5D$J{GD@bcO$lBA-Cl>E2-rI4~U^6blwUx5xLZg(oRY)q6^AZFJ`9>Hu*suGu zl}fQLiZ+R$lRBx?t1z!~r>;(kfmv7(!OU5lW{PDOC5qs}*rv4qYu!OiMEGjXOEUU08kjp%bLJ&g?qqj<}DuA-Cu zg4s%tjGT2%r?sqsPn`Z9} z4~bL62DN~{6Px-QBo_m|cq5);`q`ha#US~_RDhUQs#rtav-M%N3b=~=osw8)2{yMU z+2*G^l>O44xd+dasW>i^TqQbm3_oT_+IH1fNHQbKvFH~9peP7N>D+IAi`_Y$@(W7z zG?%8Fmv*ANM8{!_Q_J3~VGMD#xUJ|YShKi(P&PX!?loYmJ5g&Qna58{W&tv}$>*^e zF4?)#d=&qN#*TN`@F(mi<)591$$*Db{CUHiXSAp&qRSOA0&Oe+gC`E+<))AS_h);+9KkoiIs>)~k1IGbTN(BK$8l*u=T3V!AO1c}QI}b=o zcZW!KHwUC!;?SjZ9=hZE9MpT?_pZ1s&wZ3beLQ?3le@dr$UW+Ok9{ zSiPnh3{__Y{PH@&hJ)IpAKtAMVQ%o+H&F4!=0FxY8&hrcMU|gqRA^u!Z%-vM;H>3s zVv_MA`jY@ab!5)DKHz##%Vrd_MqN4A!-D8HmIDb>3HTPXNf}D?;G(=D%K=60nttxi zp`En19#>(6Mw;=?tr3)XL#>JyKtF#p9oWya|1~8Qu%m$NbVHq*72d{(f~7CdOK-_9~SJE(&=f zxch9bBfxyOk-=tX7ctKxn5hscv1M;-Qf_7o54-z}#w*_OP&G$XO0@Gh=Aij;nKg9u zpo2((cm)x{)7nFSfr!5fplS1)6#gg-Hj#oZ+HW(rVCs+(m)~L$&!JvJzsT1H+@)3V zYWBQRb&<4Sps2uI#5Sd)==RAgUJ8n6@~hm8qIWKQV%a#D`B?_Pvn5$?9Bx zMp^HvXPuvHi(FNz-f0jcCZ%AfVp3b)S*s4YRw(J(j#TIIi7F0t(5}=wou84HBkevQ zES%`z*R&~0mZ__GHZBnw$toiWeyOV)+uo5Bgb7CYdUnWrm~If`1RGbR%pcn#?E^G8Q- zsjct<7=f&?!{DMw)>HenlvYv0Hbx37v8S8f zXWMP_X{4w~%xh64PG#ukaHw)V6xUp&Ef1@#yVESBUHit-$))qrVFHFc{r%t(@&0HkcH3PSA9Kh2Z2`In|n6E9UUp$$ds5Rpu>pkQrdCte4DZ&+-pK!nPlnI%Pd5_sB+dq zauND1Tbr-)^2jaftLG%rrO9`A`iK|_1|sSQlK|WaG_mFaRZ6)m#ixp%lj>@F@6#%V zVql&^nD24ZZ}?X0Y%>06m~yU7Ia<%_$Ko;0U>jI@j=$+^!D%-a$KwYdD3@Y5RZN&g z%4-hj#|xH?KQM@OQ3I~9sa^uMzQs2d!)So;JNk-m+5sSe|hz27S}C>a*M(?V~*QlZPiMo{K*s#HRx<>2uiL` zAu2H-lok1nnbtmhceF(AH}? z)x`7Ww5{z3NU74_5a*Sp1HH9O`NuCDz3gSvK<9s;9xP_WYECsvND(IM9xUxTET z+^93j0YQhZ3pLtu3g7I-tu9}24sgaYA%ICS@ApjCd9ODC0+wbE1YQ--vf%F$6<4> zag99d{<;8&A?53olQjHdWl5^qY7&%+W)ATU;TK2cqY5t3!SRJ?14g@XYcHzurwA_M ztwnJ7@tGmG-XRydUZDnO@Ow5i9tgAffzxZ2H6po50}8!LQEXAo3iPZ#?njDUQ|xo6 z`%n5R4Ulhu8T}f3-E?w_>EzZ4F}JizTL&U7xZP^xV@V=T+GCHZ4I!0JK8cg#M}kH8 zb8-^FM~bTuPwF9wEQ`Ch1E)zyp(7g(!WW7U~mMW1OE;#F1W zUWvy@dwK{?Vue{WQwWlH0I7s0=qDi_gGXj#jM70ouP~}@*cIHaoHT)1@$yi;nOjiw zD!L8hI33d8N?uHWCM~Lb3w-v2r=g$}eNk=NUCX6ofKkzog%IX81E%6VKVYc;HeQ*= z3vyp4M?tpY@@p(viVjd>Ip~HlO`8@Ha(9e`6J3JOa8YPj5V~Bh8&*W4q7~!}Z<$>L z^eB1LqaRnjk&O?KUTYJGHo)l>j{+ShK*Og!KM}nrAZK0-STU#xNRIWOOux`&`a#RD zxmQp8nJB9;BZ>e$)CiLFR8ftN$&L|uTuefg?WTsuCaM8{pcSx8EbG73V%EO+E`v~B1foi%M5^I*{ zvZ*kEJg`7$fl{!M0}l#>8^TSLDdbQ{_%CG*j zqY0+Nv0CkRIKTp0_}tX{x$r%1wCd<*M_-65*|rG0hP*0aP0=sRn|gvZEp`SY$V7@+K#` zTw-U4R85~6Oq;j?v>r6mh3+ng+|1X2nF@AW_llk|p(llWOCnMhAtRHw+qT|xoKCL|{*js(eKLd*u zA1xCs*wuyH?5S*2s)kzpQtY~#Uta!!NHn&)5<4ffzUS}*nh>-3x*5#AY4_G@SZU2X z3N_*Rz}@B*k4kAg<^1;DL=i1JRx$78JQH2caNOK9^iqcyIK?O2Hr0Qs84Eq>zs-KI zRx@c8H`YgIZWU&>j9Cb+GFtDgz-7A<0oMNsEKH~b%b?8#qBYS(w>RsN-ObH>s|}k) z{<770hd|^pOF$sAhVlhhImIQsOW=6vkvp{id2BTp&#yo(@gahd3~v{soBYsnF(e+c z)!+46QSvva4MPyYui?9!97M4C1X@%g5+2sixve$ohdw59#%FA`@eEwUV+1k@u3uE>E34h1M+wc^r zLEASswUhF?(n>|kNOT3}$?bD5U@^M|(jTK6o;~HM+KJB#g1=nZ=R#~cViMz@0w;y{=c7SDYQWPKlGnA8Dqtj|IBV2O_dS~}BlS*CkzKtQmo zjd)sdD}+nXGN?CbRKupIk+$MVn>DgP#u7@AiFIx`eJG^ona$U(>Vv43ZycAUdDM=@ zPmw&-%D0@DIhyu3!-5fCX^~>sB3MeUD{WhyIo{BRLxPvD-p)4b=MZ?Qn)eKLHR&RU zRO_Aey%{3l(UjLbs@1KqgUurVRZaoyI2|w?w|Ht$`&FL74M~A?+AcYzl;A7r(PS)g z%C%B=@rf@{olz>}H_@Aw`s7IBQX#9x(Y#S!a~21n94v{@d~36 z0A2N)nI}0_an8LWKKPvJY|{BlYpn&UPVXphF(~I!4@4~yWiyIsfkhlNQdb+QgZm@LC{69%*U*<(^2HzQH^v~I8Y=}g7K}G89C*ny~+n0Z1GZQ zXcQD{zN3@mlagH$TCo;%G_m&L1N>`~5}2FOu18qCjTEJledJgal!*#q!;$t$=mlFCzU^|$ zT;>j+V%|<~;Nv3-CuYx_X*O02MHRc4>5_B|nx+ul2siV(`IyYGerA68#ciUtNybhm z=Jq;nasJ@=w9;t47d{!DS@a1I6X!+n2v|(K2uS4qmcm+7T1^j4sE{~2qrrJa2K>r? zpVXEk6YQaIpxQ?CPLwyVo>zc`2yC0bda@E$-47xOFn>K z-TY|=KLG?#pHY@ITgZN9z06*i6t{jQS|=1OEo)wpuE8|C7+!NU#hK78Ij~(z9xpM_ zrfnJZD#!*CA8C)Z7l#}i>8@^DZP3yowV)NBaLm^`Ol~W4vXNLgnK=sIFin!kz&y03 zEv76f^#&-)LlG|-B6P8gSu=))K;w*HWXs*6_x+3&*+cVFz(-m(ldY>SK#DzMZWKqn z%M8rz^lk3CnTQtGK1Ipxwr1oLzx`H7;ou@448JE}fLz5o$jxJP0e(Y=DQZ-;NO*Rg zMX3n6bu?I|0dnmg3%90V{p{FDKXYT^cYd}Ao;+eFs4Z`-($~n(T0}k7J;b+)@jd7` z#I1IEsujVlOG45OK1%i2wW3|P=8@(ZESVsUZtArJ(9fEE#+tgGXHhV*fhb5a?UM2% z0Erlzq`A$WNxl>t}>C%re@`1`?d%Qs;sEVoxIz>?1ph1lYrjmc`g*1F*vM zo2zf>FFvnG_nJo8_UiN&DSnE3X&h(sDw+NvN_Y8!nmI++3+WfdXJS(GfWJ>hN2$GY zHncSFJlFHbfH=fvPXDuIEDsO{{MGsdRZbWBVmOVeC_R2gkgz*P!g86* z04FW*92QoTr`SK;{p3f`&5u{VJe82U^t|Rs->zpN5ATaD45xHTdB#%y>WYzuJM0PB zEZ?|I7jMD&Dr$f+aZYRjU2~lSa@g{|7ifUfbqFcNsIrj$dAE&~le(d2@08VNu){}W z00x}WDqnFYA9Nr@V~j&%PXA+MjC|eE@ zf>gQOI|OhqOEkYuT2g4oUKc`V1Q@|%ofJ`45(8son1~2uWsV~cqa`FI@5sCJRoT*& zN8|dp=5s9<4Qh0Al$azFXOp)}KfooJhbWMT($TVA^k)iLWT`$YFk}?~0OE`kg`<}ehLOu5^8r%m zpT7Vsb-v7@*l=Up-C1{wI``8kLlsSaN!!8|&tB_IR70kx4a@%h0EFP15DB~IT==6e zjqy=LohGwpQTp$yKhsvjiZldWxsb8tkh=Cvo>47Y-E} z9brFT<$6=+tu|CdQ zx1+8geh$1yAGVO*Q3Nre~n3alVIHBp7@MX_iR* zdzTX{`_M7(U+D)U&SL`B#s!(SZR0&I~a)szc{u>6JFYkk5`{Rs8hoJ@=#Aj#c9rUM>vNP)cRgr5R1txUNCpB4yQ`u_~H9j z8=v$C;WV9W7;v_ld>kFd?%P}F#L zkm4{7qQp~P+YFa&GO4y=O3o4k(i}irTV|<(6K=->F9xmRgvqo8(<58# z?9BX9%fV3-(UnQgmyQX40l$_%vQ-+WYEICj9WfAjO|4h6?pJ^Bpi(gFsPw4kE1c}4 zi6y~8L?GY)RiOlLwykusbGLx1VcdXv*XZuhAnZSQ*LTR)zD6pfi>dQLE8&KyMtS&d zupI<|!AvgAr;-AAAIdCkWQ8xhCy^Lh7S;Y}lo+vs(tVaA60#pUaRIcOS|j-vl(byHTM8&)M# z)ieBeY+)34C&eQF^T&sUwLxN{?-Wcv+7$OQ%igvV$#C4lP~kpmq$v*!7$d0XwoO_QiYkxNq+lWEkQ^}kx=lfW2^borWxE7U9&<-6y1l}0_55hf-4<-rhi!fN zag9EUNEej2jMna6kiBlnPw?EoDQlYXFdBQf07D!iGlTM3<~U~J+^+gZ4A!2sbI%<| zNXrB#v)j88C%rwg^~v|IvqGD$T8Zr*=DR9V<-*BcaZ)4H9$8!oTy{;YnD@NqWOF?S zZ&FA03aJmiv~b}CZ@w{0T}xlw8;Om4feO>9J%MmC%K$@o>H^|(nA3u-C{EA_e@2CT zR_s&EwWgj=QZtEZRKk@J{x}h4m;esq2Wv!_1*|;e%kqu+hM0mk3<)|=Q82@gXe`f^ z6nAIUscQDamkg3qfexP8Ol2EAjDg`)fdtyMOdXfixv^^ouTmna3ajRc;infAMe#ht z;EBrJGa&bri&qN$?sB?M6#0ExYfN9iR%O$9Tar;WVXN{{rIIaymk|u>z9NV%_5cXa zzDPK5_%?XL3%yb+Z?XQiN&_4d1xSPtI73+w6N-PyyCtR3Gx#YEE1T_jMmb4pMe)$N z2>m{RS*AI~d_`R_nw|KpNc%>8t7^^J(dA?=JAY+ZIIa2%LxLi!=QSfX+m@Xv1gd`K zP}LotXxo&0FwtJMMPavs{93E&<1nu4Sa~;5Klmrg%O?L|fo(k}I`a^=>EOW$H#pB8 zPrP1cb)*P?5(nD;`Kq;V+4@A3p-44b0vesry9y=e40{M-y+oun8n`hu=4r`|`!@d- z!i*n~XOr4w$Y1qq;ghZ-0sfWr^^kd5Adtd!>t{AOLpn2VCs}x)hD~=d=f(GoA7Q=E zx~dK^Y~K3`z{=ABkv*5tqhaP6;q1vW7D0hKPb*ik+Clc&_ot-u-xA|*N-etP+JD74 zeIsh+FbBum1fR(gCI!sR?{=co7pAyThb`Yae?6*is>tD6u=G&WUIafeu0FiZo;~9a zS$5|90K>40fZ*stMd8iZ6-6B1#XR902fQc-0j*p9M^xDHF&@hNI;Z6XZoyh#=odYe zzzb=gHq|_!Pb*7iBM2pZ3OlRvzt!z*t3hmHQkJi1{Phw?O4y4PudpYlw12tbd+9z+ zMJ-WgUFC*XwqJ2W9|O`5NG1_q(5sE-dsTi3CA74(qyQ8kZqH_tIm=kd9e@~32*N6! z*O##n^`qUW;BkX&+dzAHjp{$=NhTZBp%rQV+QXp>_Gml3xpAQECYRW|ER2x^c zE2xcZ67#UUOvBUfxc5>|9@^Mo5wkMliILr>3II@cF_MecCXPjS7ONLCK*A1NV^i8j zG|vuKQVaXxVELJO2FdTg zX6z#;c=6ldt($_Ed4|TqvQ=T7{_6n&SHuC33##UbLS9o-N^JM$Pcn}CU$GfLhGq>z zSEQ?C@KnV3!pZ-7;=V13EHXK9Laosto-)z8ICvDuHUmT( z%s7+*9B@6;@~{Nve|OeHhCpc{{FQ4N2;I7*c;5qL@2PqSD}K=C$omAALh)}x|M``P z6TlSuR{jK1KmO4`?js_5F%J>9<7>Kbv_hyUkmmLK*9Wi|FdR0+eRj#eI`xyzC@y}o zZlmk<(1iDIQy_IVp%hj_=wA~Ie0?Uy$AnJukHK?>Up)Oo+~1ZPq!I2q3kl5nA6*8{ zVEGZS_ket;0Ab*BhYI(5CjK`5$254i#sP~*wEO=$pHj<^L1 zVdq`gE5%;P#T=*A{%!m@!MjFb(r=1K3MSt%ToCY#OBKl^C~mhPh1{dR7J%Y;0z$^Jer3U(1Jn8c zn0(|A@6s4KWktmfhC#%~T7Q?C`Xj(*Xf@lWod2~S|1uUZ?lu67NR;x#=|6TB2;o(R zR{&TbV18JTIy#H>Q3^Z%vLg)KW{=+DH%gr4GaLeDx z0Jz2bsBQ_20Qc`lVS|1QT6u6p0xO^Tr=^Z6z<@9Q{_hWCY*4gq#H4y@s0Hxwg z&ai@Zf8X%mUz-8eJNlD__^(zZ2)#&aoqEiAL`TJc2Li+bmUtM>OX1-D)&D)i2$68D zF$2V={%r_{(806D{bxx3{^LJZ`vHc|>H(Zw-R}w`e}BLx0dEw*fgb?6IpWuHZvHG2m=c33qn&!pb-i{GmNZ$`9rsW1fp`O zR@3&0U8kdc9sV*MQjY)7#ElFI{U zIGx{XPE0rNyneLOV7|ejHbfC=6*f?ya?H^<^~sBxcO=dT9S5jsGl@(5=$oy?}Vts6R1!EM;9(S z6c+V+xj_ObCQ1RSoi$Rnfdp9L*T=d1Hv+WJ6Wl4mG{${ErenAQjp%)nqo0UCVK{We z`J!y5*-2i0gpBg~J?-Nc3>xPQTw!-+1l@h$%*x86Eycdb`%vP)mnZ^OXKs@v3-R|B z)=@-GvFXAkOk3B~jp-LGkA7-?Ss0|vqiDF+0^DeJw%Rzifm`tS3^rrHUgQclznbA! z)nB)℘eebl!j)6_SvUAd+A3kmifkW^y~-Sp`!4iC|>9leO}&ExN{>3Xs6+Gb$DOVWJGECDTppb?spbcQ-gU z`O?f2U6T~)FcmN)?NsmHs3l%o7R$BBUgRXW(bmeNUzBX6hY!PM5fub1>$?rZ8DtPW z3dWJMB&Eh5gK#BA;Dc1X0XKW6;2zE*Cl`8oShVRr5y0H7B#78o=as=8?%Lql;XW>n z?2lcH)KZ-^GqjA?cz5V{nmtMtr0SVb05*5%EZ>(Ocv(dGzUC zzyRO^)*;Uso0LT>N1H%&TcCcuQg<$Tq@Vizl=uKs;7U zDF|zOtT7SqRd%IuZ%_ioNRC@~(QT3Wr=W*A#M}*yjim%Tl#~BOd7R%-g4Msr6(w6m zG{ggdg}<*5GZ5d)4gnoIgahX$$0NE2UQy8(ai#97bp4N7`jue2z)u)BV(6Z56#K!% z<%2)_KLS(CMhOYX0-f{O@=|vQzuo3zl_BY=F2h^=m**pcR5m zEp=fUcIrlU`<{16IuJjoLfQ(JF`P(p0RD_5$*{5Sfppsv+or60&2zcXuE;C{x*?Y>P8WGZjR}@d@Kg%xcWSqeB8t0P@A_>vs-tvCN9~SbZ@No57}27? zobLyh-Ibn~hJ1W;%hf>?Md?)>$FE1TE!q~-6B8|lo%H=$e0stouRj{7-?HA^y!5v6 zZ$?QfidBmPD^;YXSuxiA+;EeXZ0WihKruaOU{yR{BPOG zn7)zMk_?`;fK%sZH#n}=3QTEx^c)wR47S;C%&xmnSfE!Q!RGEEoS?6Pv*8!_eB|#9 zosB*Yx3jVJWh-@7IW%5aXMrD9d*RmSs&XbeP}|vgZ=F`Rm&DA!>#58Z|F+ z52$T6nK-?*oJE>BgN8IJ7(_(!$R=+Xo0fJ30yu_Q!|@l zGL9c~BZP(QoYo+0nA{@GZRuJooW@iC)8d|c>S#fFp^FmK7X3^0H+w(mJ2{f zNwu1i?Ib!sdDbLhW(^4~qJ_DpmqKpHHnCoT*HbVn za#b3eKUI5Ve^=(5SV698W?9g!b|_jX;&2}}Tiih)_94}oRku#%G%}}K>+l%~kMp7Y zu~0s-6=+q~^&(zz*mJ@`*V@;jTWvY+(-owraVvhJC053n3SW>Nai(jcxfw~wSiUKvd4G{__=qvHk zo8@If-W2|55iOi#tc~NbYi7$a(IB6>otarLa!b>mDz&=lzS3OM$3S$yV|J-$=U!PY zJM)Cl(ouBSV>qK}c0agl9&`1CWKE=vJ*_yMKV53-8p#gOs^wFg9qR(nv3Rajzh<7$ z6>+DxI3DpG6pM0zTXWlxP58&Nf&=QOK`sG1wnf#D^A;8?B)T34pBRdGO+gGdC@_TL?v2R&_T+2&-)*FuA z!7~)@`RzAY3%06%&FO!AMF8s8bIPT_im&}qM8D<#zZ#H45W^zqqWq(%|HtEho&mwC zX;aI@{MV@oW=H;cYY#eLcN}FA-K@T6(qo7x)=$+ z8(whn7MJI-Yzj+Z@uB7-M6xx&h$Q!l$#|vr0P22)0|hh*-o6PpGhM+Mu6Qv^uIpt5^!z$ z>A&9VlfWU6V0a>i5%UT*)`;#A%#UCfEE4bkT}rDEmtIO~2}CnW{MMdkYs z@s(zU{6fvZ?^O9;t_xBh`RooMR&W}VBHfRUgaCz{f?{OIveBe-6l^jfYnP``Yn;E0 zmg>-$l;9lQa&GXNUUeU7bud1@!mwWi>6Gvr`>xtZq3@M^T9b?YHkXk^+4)I}kV*TD zv|^^M%iFRu5;ikejwi$y42FYA5P3~BhQbYQwk1Omrl~vvg@)|DQAzBaB6h0@tMLqZ z4_a6CX9_d9Z{u=y5JL`W;A8C&ZYS;*g=v>bsos6NoK>KbIY4jb-n^~3JK8*n80m6% z2sIdgI)26vA8QU#`AsO4&a6!*%T{XBxNPb?*hXZAdAYfuv?MabT5x`!)H|{JCrjMM z{l!Cd6#LsD7sn%wWTx{4Phw(XqK zdT774$ilj@n%Qtdz!DuDo$oaaA46gOLaoJySvrv~D9-BRd%*XhCcwqU1}OZbTwjK`^qr%fkmrO>>CmC%Ru6UxhGWe*3fJS}L!6`GSzY1p_QVF>I`@#w=#!ggG1DdG(Ovw-kmdd6nd31%!`W@$_@5*Ho!?O;;Fag?EQ6U2<=v zpCs=V6l~2Z4K2xJOb6DBgwr&%8IX0OI|hS^h%K<@h_nq~&$EC2UPh{OoZ<~WxpRt4 zQ55MGvL~Bf?9x8-bfMK<4~J1jT$3rih{fbue7V&m*Sw%~;vr7`!=l46s~dY>ry_D0 zi92yu3!*Y9R#yM(e2vt5sobRtA@2T0KN{qoL(F+?cdf}d_lu(sp)=)GZrw!lwzC5+ zl`rIGMb#8e38pP8X;ey5#Bx8-WqZENA~BaruOKlTq}Ep5@U7t{nG;7g|eD(M{I$FDKrn~YlD zI@*dYtTLVFpJ|vWu^-ql_jMqu)BEghhp`WF$#iWl+(Pn=E1k-lAwsoy?GgOh4Wg5Z z^rGwhF+Tz_QNUutS5X+pFR35^_s~0`9`~wsjpYu#>i>X!SZ^_cYIY!Cr>VZz2*#zq z@u;tsQd(3SF~Ap@%1a`UurS)^vX(u`87~}@WIY|T8UK3wV7zi!@_2ER#c_V2(wwpR z*xkzvdp2cZ<1D8pC3qbF>;jRy8K?1h`7~){GdeeaEK4*0)2e?`(}Gi3Dv_fAi3ZBc z1DXQ9cS?&z7NNi8$hF%g|JK0{+IW?WB;L)z$H8>=5*OmZ6!zJY{vZy_&DjHmJn3l1 z;~yU3qjmDy3f-dVxi!+nKfRCS3-=lz_<5@HajX_S$ZGZOv5?8&;n?ig11ZNt4_Ic= zW)Xx0piQmKW_WI77K)-Yitf3`XlDf_!w(=f0iFiWTG5VPnKzB7!)~7HVFDE=*UER7 z$j(+D9h600*UjJs=f99vD(wD|{Ux57loZq#!>FHYJWZoe^0-$`<1nE3wILuZO}j19 zfeD)-ptV@=(B~|^a3l0MI-0~LR8ZZ{De~g5+2LsNUn?EYlI5U>e!oCFHiJ%XcB;6J z7;AeJg@UmOxNTF?+=}R8SM72jLTBa;+Me8MR~F7H3Q6b7oV#lZa~>3tm1#Z&~s_>SwjOcuimG+1_w1;i;tAm_OM8nv zK#e~_@`*Y7sE;*tj;w6uK;V8z6!@oFa= z%SVkA&K9}egcyNL2qGTA1>tfK4l>6q3q?!xhwSYE?imlmw28(iOWY0v4?Bp#75#1w zK-z~k9jXUXi&&*+S{6N{VxyLHpU9@D_0;vg)UmXHQ!zQ4EW*40q^S)8o#o^k(B! zVOqhT@BGkaHmCN)mdXU$t_fKAN(~FQW@`-nEuXU&oB{+hH@z30To|5y7&d=0#%ygd zQmhGnLCEFYQSn6ib!NHX?B)Wa4Md8xd4N@iV;1rq{#(hlo1c zfmHbiN8*w;rgm||EqjW7#l&FOy3)|(&+O>Se}Fj6T4SIp0dfEb)#7w-MpVeZxnzb- zTKkipp|rjW_Eq|c`UV1{u@53bq>MmkZr&xbl1F8zA1a{x2ZxM?hk(+Ap3;5M9N}h@ ztR60;3z;_oCj5*xHiuKa5vR4cJm^h8&U+3?hV5@%lLM!x{}~lO+036^2=ru6Y0Q5# zeI1k;{ZN8b60U7Ig7EFjr?5E(k(HTWeZu3?tE%~=S7V{dOY||0q0V{10YeU6D}p9xA&Gy z<`$aL$w|ZGMO=c#AYfrCJc$`!sg0FhXrA(6a#c8lb(r_4i)uXAQXd` zE3+u(T1S4r7B%CU7poJTwli=lAOqAgev+p3bYpIQ1zvS&7lFJ4 zEHvJnU{49)u7I4Jfw#<226Oz*SNbz$MpzDRP8rNV55RD1nI93Z}q5r)@V8FJMl> z!G|OM`c*Q9rg=Hq9?mC*USqW@zxb7ev7#!mB=QgJ8}nFnAyB^mS!| z(g}Tc`At)n0SYFH;$d89fk%r@maEkB8%(E#l41vs>nRjk?ediFf@tuqRHJ_7B}V zZx^)4jG#C>{V(CIhS8L?^9o8Q>CT6i&WFUnDkrCZmo4PZp0GaThaF4^28tn|9H1QQ zt+&jOmE34~i3fqM8y0_o)~b%r3>!{vKUSsOP8R%DvCLxlxVl+p71Ps!9?$7iG9X^X z%;VMI+mGUeX=bI^oN@X+A%^4_$SMf9zqKX*)BwwO5r|`DWxhuCD@!%7s#`&~!eY&* znH*jSX=PX_CCk-}KpK*Wiy-4U;+sEA)s_m z*3H~0VBotA4&@l9m#_lLZDyuB?J}-aGSOh@>)p~Appoqhai|m0!wB9MsQXJ{AUhzm z?BDBr{*NIZZ2%Q4m8RpzL=D9sqzx^U_*Uj)*NnX$mv{xhtmqVWt`HAagbRvra%`8-ctRxP8tjVXGi z2U_`jFDwtL^F($(7M57#0oLc!A7OxHm1OS7J_xlEVcQ5T%VXN2$2ylymcq)(EZM*3C@iwqO|EQCR!7*tDi>0h_V z6i<9X(<*?X;$Xp7Ej=3iY4zhXbF|55ot3zI5ycJ69O~T0XVgqM?U1dWDzHE4Q%d4@ z13~VBZ!0?KHtL_w+k0Whz3mPssy$k6JFQ(vL3%-X{#YAfAqA&S#leGbqui#rSX<9% zk>-WyOmhLpWZ-SjD;&g8#S8x!|W#W^r)z)%`=xTrgO&2)jX={ z&L^k*)w}5}SAKCv#PJtsE4h@_>OW5EoF!yrY9^{J-1&ZDnN2qU*(M(SPc4l$F6`FD zjTpqE_-!f9uhr58$BTw;ur;eIYLxZcyPlYItV2!T*4*W9%}xyVINqPgE?>N6r`1?> zPvH6K=2~%xuO4-a(KF&XCzhFj+WWW2Xa zgn*8TIn^&YUubZe%(U`K6a!x~wQqi>R>~ni*_jO`vN{S`Yov&nGj9DN0p+naWA(*d z&>+L*Aum7*^6snN8U*Yor7O^hJ#pz9qS1JIO2JMU zMzg0KA`mU@axNUFC?hC=P<`cER<8L|Ww!S0mH~_C>2BbJY0JHkE0gkdxm(5zKJ>dV z57kDdvzcxAg%Z$hmLJ*g&15F@40wZoXVR| zL(?q6Ut{2-N$L1TDq}GuF3ZKTqow8v@AIoitG!yS>~wjvY3}#-7rNOxFTorZJ~|{d z*kf>QSZK_2v0hs7YzQe&Bq&FwTVXIFvGeM@v6`CEkQAidOb*$7xGO1ai?k=)Rnt8&d%8lbSn8zY!J9k~ zwH_2&klJDtof7qa(L$9s{2Vo?W&h$7BxNHGZ0;|+Fw=&RSn{nj0hZwp;uRqYDL0Ce*YF;_K&rf2&TFh z+0qN{F4cw^b0>77Lj?9Jp=NuEDSe;lJn@4_Kmwf4_WzK>Qh5yQ0zfacH5Y6Jz9U9F z-$W3&-&DW1*8lmhC~z|5#%$w*$^ZZTF(O(f`2XFLH6`HZ7?O&3023(x>+?(&kj;Vo z58(jY=6`=uP)MA2{$*MJ-+f^m!0e=R1(+lFfA`gz2zkL`%_}l z(nsl%=+|P2S#OdX6ae4R(5bNSPa;U7M9=z?4ChJ!1|dL2AT~mNehg#ZPpi@~6?9>V zvk_VTXJX>gIbyuw@9Lq6Uq7IoZ^i`f8~GJBXYpb=z5bqu zhUDc@pQ~n(#RsK)Ol|ZWHL=_PdFvwFY4PfP$9_o5{Q-`SC!g>}Y) z4DEMDiGpb+ylHWYSC#NHKhV&uQV!0t$+MibB@o4Qu2i0tF4T!hp2=3W_g4}+O&jWT zQ-H?D$2AEotc(hO6kRVXk{CameQn|Et%CT$nR0I~b-B91QolidLTb9nj7pJqaQfi+ zaJ5V}()le9)naZk*ZflM@{65B3oAV{>^=G2pwFE|KHb2z@EeH8z(uea)=4?o3QdBa z?Fr|P*+5>U1S3A(sw~D?V%*_F8r+!=!z&Gj@Gw1w(p_}91X5Oz^Oc~uS% zEp!x{iYZrzL2!XjFfClgmzFMe)%NQ?E_@*U>DIK9qfF0C(}u@W@}Y=`QCnwEXSlvj zL8eVVY@YMNH-rggyFq@arBWlZG0%BnRCnIeo`DH5N+GJKZFysM>%-}5t_OBYB!q^0 zLntwQvX7L15#t^iF}@_H17buB2f=px3D)6eAzCJ;erI~jnQSl0Ui@!KJEVxneZR}!2iXct*ihGvk%I(V*HCYdT&;bi|kmW@g;{RL2LPkfBe8HHF zv}aC8lq$%0@8ax0>;Sw=!T>TJm;ojkxU+&TRY5Go&w#k$ME@<3<}f*J&yg+XD?)!Q zFVAn)(=~bCytQ1YO2ov(oCCZviYLX-A2j_$V=tf#Ku87-+!7KE`7_lKll&fAtd<|a zAiy<(bLb(6wj0aHvu;%g{A`)Hspu^c2AVlt3d&fu1mkRS$q z>=D1!F_Bd=?2yIr=<(N=B`iy;AehA&e?ovp{&6mQ`AD(S%(A$7ik2;339wIRmwHa*2SzTgL=n$xTu6ylJkh7IFh0Bk6t?&96V1`Up49?$|7#70t-3%!$sVv zb|bZ{XFTk?KU$e!Q@&3RO3%G{Cxp~2O?P*lZ0tuFJtw!3eY8G2mWA$+fws>0<68J= zBODwz`aK&wvbLKn=2`7ou0q^wCgU=CdqmjyVWLRY^lG7))v1kOrp6epQ}avK0)c>x zv~+c0R)LiYmoGQSh-?jyH)rEfbSMF_c)It1noHA7^fCHy!CmYy^dxz5B!mn6BTh3p zUt|#PcqL1AB(`7Wn2wC22=JW$CXJjbY2*My^ISbsCk>I`Jgpr0GlfX0(z(p}kU&a! zru*abAvq#myP2@1a<1-&%0R^|Z|`9gp0NW{dgRxjrJTX!I*YrV#LDrnhG>_1b0)M} zO$kb+0&NJapT2)oYc^xEoT=+gf|}`16ZdcaXcJQ7o~|$)8?P`PmRY6>UB0~gF~hdK z^M85IMMtS3XcQ=2ubOK&0@7Quxc8dc3&{Bs9uvL&f0%pgsHnQOe_W7M5d{Hhq!buL z=?;+`I)tGGX6P1>4h00HVUU*Y?k>rpyFt3U^LtS5`|){y?|T3Kt;NEF19Q&obM3va z>l4@cvcH4?AFN!#y4yq-AH8yPX`Vg5sTS|yPF{FcwOjtCkr&f-gjeq|2KcL1_HNv6 zoChbl@Y&PdxI|vZ66sEM}NScs#9>MVc`77|YoRbx>s z2FGI}QIDS;jpR=X7O}^N9PX^S84Y(Tf0#eTq*_Pn&m?mq03>}}q~Lx1HHd^O>=AkuO>hA<>Ir?UZQ?o2!NNSJ8QH$%k@?JxnDrm77T_9)Y7|gSq85Urym0x z2q|g!?LEAM!TGp7o~l~0z@m4X?{Io?Cl@T=EzdAvU1pcXM3_5SmNdJ}SOCvl zS7(SeZN?y04(sk(j`5vU@EVB77#xf<8sphf4EUxtw9|6^VQZbOeEQPy@@JKMK@l|A&O}Xc?b!P^J*M+pce!hRih4kzxOi zKb;92^s<_+ao$J0JXvjcqwhJnt7H)W<4u|e z?+xx3zc-dII>^Kvrb%SAPGgThnzT=rTn6DiAy$vmHEA92x=&CtE{R9~u3)fuQz=7E z=>C(=OnD_ar+{a-D?@iFb@L{ zrJ5Mfzsq?l()QNmtO#EdU{*EIj`Xke&xGF{Zi5`sk28}j>@CAKpF)$3^XS3D$F=>D zZD|JA%%4Wg)tXsM{MBQEQ;DjMKgQo9pp()Q> z2fcX4?=eGo{xTqevqz0RE|MeU*z|6j!s+6mY+sU>zu^S-;A-VcLh(Afjj&b;4GWj+ znGttyjkdU=>*+h5`cFok+SaLg_;Sc8y1`)_+%u1k0#wlP`#ssk+eoc6qtH( zvkcv4K^|YIDT*bRg8Eg^>m)0!mx}cIvZNV^4u%OPK)Q6nG3}HjyEYNj?W!4a&!l^a zT~xNv7__|tvI4R|%9AIe+GGas@s%f0JB_MMrr~XAG{NtQIm__(TQ@$T57clS zoQnASv6XC3nzPoVKLb*Qes%`K$^92+q#|I2-0Jnd9KvFw;~K~Zi!i58 zFzkmU6DUf5qLNqwARNkK0s?aO>YxYsjt+uu(|PUoMh~?lJ6~T_UdnQRoO^WS+;~m* z%wMZ=AG+q7PM+S&PYUfw0%~%@*^Z<2(b8oId)5!7MB>ZO?S;v0Ac)Y}=LD>K$HCkg zer~^L)owHy;FE2^^a0929T+PIf7G&?+Xy__l*7GE<0$x!F#RV>fA8>5Kw9ovn1o zaKvSa_et!3+Oqg*5a*3X_z*?u_ky;gqt<5+5F|t1z`IDNGUycS)mf&dSws_uEhm17cL|l6YP0wN z3eAaZ>>v=xl4q=h8E+H0FHe!vW#a(&cvGB;pGc;3eTaF;0z&mV@U9moCg!EM*~4^h z98P$Yl!zaGf2Yz>9MR&}b6%e8AEfrHcxQ~g#OztEr=9`pucz0Z)24MxBBB|ydDzmK z&l&~E3h}KXlS&{PbnhZA)L7Gf7oHEyjr#OC?bK&qeSFsV5Q4mblOQGwd`{(2DdHBt zY0-p$>u^WAt$?}0D74navX2XjN-m$X>d2T+Crfr-^0C)-;ksq7n=GB;yKVH~@0ITJ zfpjHy9=iExt?p4ycgT7SWhe_HX3Fi!Yo}JAMAttp{iBWee;P=$JQ3nRsT(|c^s%|s z*5sU~?o_MaV!T>oD!Xb0ZFVglTk;41J#ug{h zT3;do_SEb+YyokU@;C~QhT|n1(v@}Y=p9b4>Gu{nq6S!<6M*`FY%uORj%KG1b|+zu zupN)L%e2@`OzZ16fR^NTUVX(R9dfKIGHHz{>-uccZDOb%%6o5o{~FGE7DG35IfJiM zVfl>P^%7-x$(GCDn)_1`O!>a9`UKVwgE&W9i5bC{EIOvzf_Pu_7M>=GHK_lI9Py@E z*kJSgK@Bq$d_c786F!+sq8W8Pei$_AIB$Z8U=o8AO`Y2m7!N<~jNxGF0H2%Fre4W? zJle-2I`5HFtO3d93-1;?Dp4|E0|WKZ3DIin|Mx&SE&~G<^_X=NbE-))9qPlp#$ihvUUr;+bRr5bNFC-RE-k~ zso;V0(v1kuwt=9mVO_(sBD|NM+HT^nBab&Hg!flVXh^x7KhwHgxgD;oFHPJhl(n3z zP~Fo_dq|{iT5k<(k%p0Ns3C$4q#j>Se-VN^J==oeP~ZeEzzLYj!j`_7&OX*jksTiM zab-JZz_Hs;j?+@RZnMsMR~xO_AIvc`G-I-Bnr^TLZA;Srz>>FWZr>aOGkw@K#t z-ppV4`wcZ|2vBZ@U ziYM2TELmkD@x3J_+Yu~=btP?#=1<}w$Wv@qE+t9H+=gCiE6?Ux-?0bC&tZ+?!nJ|i z`{=Z-CP-Lqb?sDrZj3jG*XonxyXSX^`1bEtIWA?HpNUcNrvX`O*?q~I(s`yV9Nb*g zz|_z#9d?B2!Ysk1jLEzF;Vpmto+!S^?fE&WJdH|?_3w93oPYkBJ+1|!9R#C7F$$_Cj*ulqeH|e(a8op3E<3JOlVjnQa~E?3}Q3L-ZJtfRR9heTpVZtPjFG6 z7j0V=gypPE*V@>huOs#jcj|A(qgWPJ2q&t#5}c0bMgNj+#?j4$T=IE9re=)Ts+AVGx>nvQy$jcS5Z~aM zQB{7&O$j%4spB{Lh=4T3=dyzdChz3-CjBRs19Az$*aG{HYKOA1MIwU|dHmo*Z+pO8 z7Lx))9`ABBpeoPhc%)-EQfF^e#j|g;gxPeB72?JmCmhHSliGoVVuNY2snMk-Xq_Y^ zZPX}Ac^1E|&pU@7&PC5_>7CQlSx}a;P^RklHA>l5 zZ|~3@jMD(Y`lyp$X!=#~7Ls?#X^$NTkR84WTxbj2+ecRZU>2N8Z?`=qQuI)7d#cjb z)|5pVD7mA}CUF5O*QI57mO*UwW~5%kp_m}0?j#W4kg_ZaS9*AM5egCvI^xe+L})8HnvEuP zbhQmGRe<;*m>*XA-d9;(^VWkYM$VfcJTwG6^9hSQr-3|@h6u=NW{1O( zl23ZZ4ui{@8>4z~m(~*jMv0N+S)e+sC`TiQ&gSR$Ow zrd6AHRaU_1<@OV4)dO$M@^5E8$vhx3{YYO@O=RFX$j&^tBDH$zW2M)x=;O;v%-+eB z|K!nJIQh|PD}9BdebKWRPJM1UelXF9#u_I$Uc8#WlxKlee~0&KCs?jh@r7S z&D(7<{tqSqYXYut!u<9MabNzk_8ej84#=QlugCQjw(T0P zclF)x__y!y`5Dqww)fBPN#H6JbH1Py^{-NYfNLPR#dwGWxOH#Rq*4G(HC>FmB>958 zB0+M>ibu2BF5q~3Tr8|U>Wzs1lSDcJ=WNudPkJG6j=dsK-+#Ag21gd;c2=yITDt%i2R$kj^mXOmZnSh~RJsY{k-y4_-v&~MR^7>YjT z_nmay z^OHM{#|B5fLQ1?Cri9hA$&(BzdA0~NB0$*x-~8qrD!@?Gx_i_A{lZT%3VJ^x&ST?& ziNtWadknQ)){S-C9>g7(YN_?uy{Wch1X&!9lGs|WuZq?fWhJETibAd$r)8qr`<@3& zVv!~UURVvHZ}PBgY)}!P)KI@NAHKt6;w{7GFA!Z}hoSxMVNhIj4sm8CT@Sv9yvvHz z;%Dw#DUS=vdx1ZZ{VOD7F;dbL*dIO}r-@1oL_w~jwq!X&^L;BB`ds4;iwN|-boE^e zwu|nZKrw0Ys_k!HF1Om(zN6>snnCpr(zT&n#trt-dL+)qsGe>7ExxV4C)w}sZz2Sh zl-5O)GKk8K%d>_etf2^l7%#I&yM7I=xc;lFshUc;V|+@LhJ5)G7!i^ZbiSYSwLar!BhYeM?q1NMJdI)yO9 z}Ud-*cjhoQ7pGly0R8G0~E_4EE`5H4Z!s60^<6J*ybW)c!K<9&}kdP^#scIDBS0) zUVK&9HOpzDEivdIc2@K>_YhD$P452w$rI`C3HA4JqV%oAi>zJ{azjmSNDIysB<=gr zwugS-t;9(8^DC_XIm!P7EyWXzw~o73~5gDLdEUrA6cW=?TiUC5|upUs($|pqk`(uTH^vbp-e2X#9Wau zhylWHK-cTJD1eM!VPwDXKE*WPrb`&{pDUjeKyC|)|E@V{ulS~B%F{6tUGw`aZuw#j{qSF7&DZtx zg5Da3gEV>DRS!X+A2ANjyYpG`Gp;KppJt4hNmak%_UMu;$azth?phb6hZMCF^3}%^ z6n~J_PwDeuhDvo62aHH1hKhJVNq8D$XD383w9_nNT-7K7rT?ydtWPhw%khKh)2wm z?nNL5J!;W26GLb^YI)0i(Ee;;>R($={@;=*L$^<(Q1L z+$YGiI8n*`5EXjZoMVv*xK*NOmcS~8y9i;UM~H6{`@1n0;jyt&S56PV1G=8U{Q!Ti zJ8=;>7Ssa_Of`9^;bMUoUy_oNasn~{R_9!Y3UHU`|Kl#9TEmqi6u!a_CccOAHIT(Q z{ia@J(fKtuSC-&a_E494N558+oTEp;j;;hGPJRu8!Q66k9LIf37oRIniGHrQB?0yQnTz zXE-5yZS|+#A%8D3yrD3~biBB~O;6PFiuADeyc=J6OYs&MK(j~g7VTnll%Gqpxe&Zz4L-30SjC<4QECGGcZNLX&#swz(u&8iazz5m<0fk)S zw}w<7Ngg+UmHAI9;OP!x>c47Pv5f%~dQ8I^07W-XDr+>8gF18^G7`=wX2&zEQE9D^ z?V~nO1F0Wh0tE8^XfBEGnF5l(tg8ac`7w#%Z6R`$CQtYLKeOS%*lkr@4JfL(f1sa` zCv_8-2j_HrMM0+cE(+}JY+6Y!u}crY(&Iu!RKW;yrW$*N=4z)^(tmbI!2(SE-uSpY z?eZHcl@aE6ep`HCFPipiUnvq{$Y{CSZQEVDa}5KOyGtX?$}T1%VP;Q%>vhEoPAl80 z?Fe@IyEXwzSgne)S-YufWaw z@gGHAE#jesJ*BECac}V*w&f;Nd*|s}tUbocdLlff>tgTf&enr2%=QD!XBz#^RQ(Ft zgC4i2ehx6hsSX&^Rd3UtWTsi+$9EV>*FP81=VdR-k)Gw)w;B|cn9)CY0XBm-f%cu( z(D028nEeG5UiC1z9o_|=yZvXBD<}$V57^hoj*O}Yf_wrNfsUtgQl*AjYJA==HEVvQ zmub#~JbVb5C~IhmKrj?b|G)XXE>i%S18b7)^e|~{zL~wgWP^5wv!jc9HeNZ_WYy~L z0VWzF%{=9DZ+y;~Lfhxc`3{k!-t{UuBXG$On~dhct*uUd_V}*Oi&g)Vt^LBg$cds2 zU;fY)om~L?I@#R?+eKLP_iRncB{S`KCkmgJDJXA ztnbg9WU<3o6mf40BuV%5x@X9g95Fs!W^|-&vN{7h8H2qiMrrzmb7L71=Gg_dXwFsK ziF88@1d612ZyJPD*UW(STSLiJ7auYOiZCfJb?t!yRP%~A^+G7ju<<1rAm_#66yvW0 z7QPmNXh`mVIHuV>BS28&BSJ~Ijb9eqS^6FhhpO#)e=!SfINT~D6dHqnKQ;k1F){gD z3bax-&h&snX^Ri_Lv+g1sT=hOdm|w;&=4A)yaElK!pjJIHWw#;w{v>bgPMJ1^1g@v zOa^y!f4vg`FCe0fSF&x0%KTc0{%uK=tKBjotBhM5QzI8|j1nfQA2J zsi}mf1#VO&>Tj+Xmb#xmb1$KujTD|Gf6H;|Gb%fK6YKVhI1!K4sQ+8**GoyKpE;Lf z#uCJ%SHklS{+UT4u0P60lcI2g5avwvXecc(-E`7m z&bw70#=(GK!OYFc&bw>S#&g4FboiW;Hk=N0q`EjsPSw!Nr8<_ajolf= zlCDV}eZc%=$L%GRNBn*8FNX;+s zF^@YybN4~;5564yFryYqAw(l6& zV-|u7KC_=x;*Xq7&O>@qcfj!?Am|!B(#=@2m`1BYKq@u@4PQNtnW2UXkq_Fxj4wC} z$1n6m*)}Pz=_rb(@f8_x1L2HWv|GC`QQ<=*sj29&RRU8E=&5q#{wW;aL^tFlEE|HQ zEAHV>*vY`~Hge5Ovcp0{jWw4Am1oOHSxmD5{lRF??Rq6vx-m@YfRvO?9VF!+Pt#Ec zQh(N{3s$>0;eKe@PI52)sKMhAq^XHLcYv%`baxDnStjiN?5elEu#&qVWOSrhs-V5;YXPPqaCd3naYKD$>()gLvu-o)H(x7D2&x9OsRHQ zcz{3#d-tr^_V8qc1%V8Zo7Nt$m{!N?&xBFz2!oh=t{_u15QRl$_*)K1u}Ue|JE{og z;3YzQ3)~(Ci;l#9(v;7SJb+Ypu>Ene(WSkf=1Yy42!#!de+KLw^43h;Y~Q-7(-%{jlA| z<(F$4x$)^c3W}7J;PCHd#W7=!XRyxaaO{+O=P=W)l%Pj#gv)NY?zHFA#ly#?GJ}Q_ z^IGR0#|#c9Vwg+_2LfyK3*_EAxW|4ZJzFoa+M?A0T$w&)!z6~SxWq(6mjuf|!D%PZ zAW~UD4VH$hsOqZP74de2{+RzpZ|VIpssn_8@TOv`{(Oy(NiR7!5ULN)M9CI0HE37s z+*>0@-ha>ufzzW!N6r;iHx#D&+SWWPE>6%P{9*l7gzaaI0u7qWpZWa~5mw=i<%mvg5AX~S zp*_EcRUpFc4uzKM56#YSI={B48_VFE0mZNjq7W2zP~!p?;KQ6d(HfOpg8BpAff23m zs^4{x)VjfXQ!W;F8Fx)HgJd6J;33ca(H{npIvc3nc^4FrfAGVivq+k9J9HsujT>vd zuVW#7b<@qDlwd7){yFPn8#_2Oo%`KOZ8I#g{BsjVzR~35)Tap)kIDc^g~&voYZtlI zL+ei`y#=PaQ9me=rXPb!AsXE;%-f*js@Fk4D)U_z*&hYnEDR-{#&XN3EXR!0)F9S< z$}9m^$+93lUb;)V1BGRpM4C9GuS8hA>qGfnhk684KO(S?(^?&fBy&9!ax8E;fAZok zUkJhcQfHjMzFn49D50{*DXJ5kU>$G{mO6-a;3%DJbIT;9Z}{_CjAxKd{vNn`=a&76 z99{b=v4^BokPB5UxPip}etNw_u8c$x?FuQ)P659fBNDE|gUKS-=Oa92a?J*S!F%LS z+>!j`DPDiS2cFrw^c*7rQrSYPw&^majSj>B9-MkMBHCgxm7cm|pT7*O&1yUmeDKZ) zs-WUp5mp)s7!jucd@q*034FG27o9MI|8(ArzAzj{*hDD=Rmr3$5Ul1bSwPyE)y)}bGzOap!lx^js!uNA;&T!~8W&V^>3kN62qwEvQ?b<$QD z)0`VL!*KBPLTmcys*Iu4^PS4{@__jepSOR6W%;|0S*ZIL6jA-MeQvFpCU!yHY#*~n zuNVLx^_PMb@U}7EdfQlv9>>tPDben^#U8RFTN%hyvvi%~BzXF7}Uh9O^5p1zxC z!|A4E5@(@|RPEhM&$@~|5r9*Hw1I7Rj}gBRm2LsW+6uR~sQvVw3E|zS_xw?*DE|5R z{)$s+)MrRdF`o}QiH$Sb~3$J^z+tWC`HsOq+!Dbd;9v18hM zTw7scRbqIJ6KXw~l4YUJ5j=ToSj#6IZ@0#=hgXHw-H4iN#trTzKNr{X9$rYz1J?YV z>d2AYnFgUiw#(I0m)(kI^{le+>|lZ|ycGhmj_5pbibwPSx%!JEzS1fDg#>iBr`g|- z-8%4Ym4egTU$0DSju)~QqW4eOE#4M&^&T_wyhu~StA5U%@(i)5=KzU^Zs|?m1zOEF zeR0qKr2qc2=tif>W&KAYhr~D(5uxYR1K8LWm_8ND>Ws77BbS3=eK;LJcY-Y!Dj#ZQ+VsjO zB^E6-e|;Laz>gbL=LbO%BLoYO&c)&vOh?oRzN^zM*`Y6^oOV+C;od{j zRYt_aCaP_Ps_K&KwB&a&QJQ=kazzT8GF9p}{JWk;5O$CkJHm3U4jYQ z`=AqSnSu5P$nc1liNiYGvk$OX3A`BM@G{MsnL);|OD<`il`YJwgNK9pyM2Lx2_PJV48o^h;hFyOV*|}2*d}evjZ)<$eKkj z)eO8Qh5P}m9`U1`cN=m=8#o-7ZuKh`TFAw3T371_QZxB*w^(@QKh+c(rGRAV?rdMF z-QiMsxguap%8+by6=XCrdHx9@OkzhEu* zhl$Xw-%j_e`hLe+6H@at1|%AjGE4L zK$c>E3f8AH%&;epRjoRmaf}Iyl~K&|7Fx;V>ee++w96tY?{@Hc(`w^qBplDf~e49O%mMFX;JS2aleiFpG!_xb$~NRz8Mo z>pbz#hSUyLNWtBM9WL@dRJ0oyF_cpyj5vEJBEJ`C5e{wcPDdG-g;usQj|If1Qx?an#BaX+hv~iL!~Xsf zdgHrYqY#jP4rIpKM=jCdt1!84v_oxF;|+ZEtRl@*2qfKyH7DA=upoYwiuWwX(fGCQ zfo1Vkij$z;WgzK&O#dztj?m@8&>XI~Hm{;rs*N}Z%9SP~e3cFiZI)P-yiGCw|Gv(D zuUYUZ&^1BTOfJ9pJR5}bxx|fhC)bWzj?RQk*1#bEO5=}WwNWU}=im?xg6`5F$nYU( zRK)%?S3SgTMdB2S$O$_IB~ZE^1@}}W#1nk3x>X7HiPs%CI0?n!dLiub?|t9XA@(W! zNrVHLSO9nj#Ns`jYP;rjvS(n<;ip##Vn9&B(?DJj`XccgO;ku_d;Z#sa{jNWLIFj%=|dP@W^p2U87co@Kid49Q+7Wq~X;i7)ee#MMLG> zZK+kcakumD(D?hf>8pD+r11TSa)PY^T6xJ=5#)RvYPfrfUR48OQ1J&{hal(!e?9It zLh>T zB1BqS6mhO`Ucq(F#6UX*W7l^mwoIWRHVrsWm!%^+UfkZkmpmTGe=%HtlV2XYl(&2l z7jLY*)`QG0Ew<1)J(>_oP^oHXA&LHYhqYgeW^}g~Whz7;ExRkcwv$q5Y#M~qa2QT) zc*Td%1Ti0+-b1hhH_j{zR-IfN*WjQl#siPxUMhnmJFGm16M>c>fyLsMO2`R>(jv8{ zf}W$zkUIU$ZUv7eB7_z^0EDwyN)3q_n?C}fB@EPC^a1KC9oc+IS6s_f{8XCW zIkj6SK}GZemG7TAlMY#4z6WeUT2l}K_&4zsuG|ZksV%vlnFThDSKyYM_6kN_*&o7jQHw1`=S|9A^3 z6!3s___8|r)MQ6zTxl}N!@}z4)+L)z)k^IB?uHF*JtPEt#O|{OWmNRI86Kv9Ly>}pEed9S@g}=3Z{j{6U9ww<5SFuP9qA93q#I&S<5}v8ZX4nguW*Q4 z-in?xT##vdu<65pt0-`L(0}*uh@O2^F*hbP-lR8`MPxg8z`$;@0_#EuOQ4iTlmjkA9%pQXc!*Q-A9 zn)rg^yA%Z%O@~_gn-JB`c$#!u0TPbWmf9cJKAm`ZqJR>O)s|oZC!cc?eRi#j`hhh7 z5G3MvA2S|JN;^GXZ|* zx@EKfe{fk$7$ksvxy*cA6lmF^H6)X5eRXa-QDekHLrwj0+MAH!+%tgi0yPGRLuQ!< zf@`EYmW2g(eBj5XNML9F$y2sU_}3OB_z2^+M`J4%vC)qL>?Gj*l;|4kt{_4-j^3Bw_Wo*(ekeo;6)>vj{TR);mC|pnE*?J}U z7a1I~$+ML7?fT%u`A$8HYb}3`kp}cZtr58`rH>%IeU0AnmyGJ)TYCL`_n<_3)rU^N z7#-*?p0>ECKm93QzztCG4tfFRFYp0Dck+CTzmX%*48O9PIy206V*)05(QCgc z>9D_?xaCx?P-Z&5+8RJSlxu^%-i@^AJL@VfFE2kr543*jhtVeRyM+P{3Da)uAEMvK z#S^}gDSr6WAsR^DRbtSqb?((W?L+SH;u~z0Oda~bmRJxb{^%8V&u-lHjaCS5>*~iZ z=F}H5Yd50{&*f7kBQjJAC-p~J0g6>xkM+T88elHE2!J4!9b_OZ)qBM`250_I!uvRX z->7f6AJY~FsVOrJMzvl#IURRFjQfjFkDOBT<77NMQ4Oa)7b@|Ty#zd?zQ&`OW-8ki z#hXD(A6Dm{sTDd_;&a0HH^++yPbC&IRmEUcrto) zK<8@_j&G)mr;Sph3wh`VyXHY=hc#A{826-Q|0RiK@NY?C;5Mb+z>X5=^FA|t8X z`lT3{XO;-&j2R${ggBLm-q!tl0N?RP4)$lADZl}2qgzin%bqMV1t$OxRsm0zo0scX zUdHYnJa9bR6QdIgB(dYJv{^~B+ajl>E0-o<()vy*;y(t9GQV|>^5Eb0KdiDlBFa^- zj(6kaH0`iHqbo)p&CvoEvFBt{o77Qy2)=l<8{y`@b2x426S@;l!wx`}*x6QAR_4#X zY+t)v7+qCincMypnAUy!{^@Ol%kod0?lIZD13@(2siXv&YaAqb=L|n-&gOJ&t`Ui%C$u*F`x8v zjVW))lY)JT;`?Z7P|y?jn4No=7v_Z=0|=P=7q$|<8}|WFAmwFz*lceE zPF3Zm14lZawQw5CRX>|nywN&8Ob!`3_V8KV=052YiqABcaJxC;n8qUo-_>@A1Eh_g zI>Uh{Xx7$*qvBwECIV<@q`HHV3~tx_Df-uSc=zfQ9S^R;$uktj4IPV`!w!F5kjn>T zm9BP(B_s`|%S_tZwH?Q8s`JQysC>FUy&yuwy~>!SQFhN|Kk+k$d8$GDYQ|Cmr>iHK z=~2P3eJ2lkupmV&hR*oC*}!G+)RYXI$uv4bWV>f_)2A6Gv;7v=E3iG3)c4+b)XDZW1EczS~^ruVWB_a1IqvQ9ZE2R|iiJX0GZ#VIkp7`Vhm{ck=6rpPMz{!Bq zcJX`rI+mM>LjVy+(w0-%Knmtf+scUb`CcS>P8C2@%2Dh55NNQUA%e8+M5xoWmUnIQ zq(09-b_=*x+k4&}&*7gnt*pEeTf?F=)ZTwMIFzuqgmhI}W3$q;$8)!8ut6xl|3*40 zgYhm6st-14qK}$Dj>|0nz9EnFNn1kwC_h7lZ`xY1+elqx>4Qsj|^?z0Bhcdr;UuCOiu5LwAOPZ-82c&bOMYrLUf@1N!I^G`_@p)GIgAH zL@`4t`>Tr=+N%h*VOETP?$C%U7hJviDECyQ-M|dn0e0d~S*`U$9$)+{0 z$2LvT>dpaPo~(00KW6duHSjrr=l_<^Iw7`2eBn)v!sz`&JLyJ2rW_U3bu`2P3xE$; zJkd{k)SI_XYpt_?oWjRlRN^T4UzSF?WRxd;{rr;Nc-=E<)B_v&;?S(ld-)$Ovd9j=<8JzXhWQva zhR(iV=+%u_!kWtUH!-48O`tceDeUhueRo|MYPT~beWe{TJJ0u=(5aTXF_imgz0czc z{m>84Ej%5c*T{4oi>=N0Cu4 z5)RE%Lj|%z>?M7N=1#QjA8B&iUx+93EyokgJB3r*O;@4?wlE`K-;c1lZ}(fSaq4?( z&K!orSI<1JLh&4Z!XPa`D@aF?ks6>&h1qc-jj&eIbKMxpNV;6qR1p=jtvD|GDzieP zGUc(4dL>cgrn7aQs4YV{w6!d%(rP!2Zf34ZVj?Rz?GB&D2@sewY&dwKMgXw0-|gbN z)etjoFDYxkIVM5wyzikB2Pdb|mtNqjG411a?I3ebS9vN=_XFTNNMex=4el^L;wu7@ zQX@J)LCe<*ueqPodg0trojx8eY7JX}O9U#&q}8c>(=Gr(1icvHH1$ zXCtsrQ1`QhqSf%bua6pk&rRMmCS6Cuyo}LV{7BE8WvVp6o@DYlAjGa_ zc)Z$)NjXPJDDxa(D~)<`xLCK^5=!T+q=xCK!5WJTcS=y+0?VBSM#gQ(HUu~qxU6*G zK3@ZjT}pLtxKd>-CMqNj;)?K{GY3*%(5Cqv7eP*CXuirl#08Q{bOVa51c3*EScF_z zOXK1wK`d`xzq0MV{aEo0Uy%cm|I{65810iY2`)uc0jGwX!&!<5L979?#sL3n89=2tv|w9jt5G3FUriEy2F0?)={}$KPopHdC|9vJyYyj->FEP;Cz}H0{ea!=qv{)HPxHx z5HeG31MaE%j%A>Zf)J?LdEJbU)IxAnh4SG2C7>t$VENJl2tX_{JmAAzBH`xeQ7{w2~|F3#HAU4IV|?O$tUtAAhT6iH^7&;~+)JLhgza5KLIwQILiZjlSHevEO`S}F$h7TOazb7g zmrA#mbAhq&x$?`3ObsM-tfW-9ny1;85oL>G9(6tH`>$yvpS9r$lthU*BOL!EI39N% zRoF2G#yiYI9OpCJGMjNpACSxcHg--_8zcvvHZDdp7oRE9l3rh9}bqKZ<{u_?2os}fWtrLVPQSOAO$*H{)$Db0vCf01)9NUfbb)1J=@xO~fQ6oHc+W9v0F7MFP{hl}QrBfEXWrtTyxRE*rXn zr;>u{{ZSs!U`)q|%y{qJ1vB9nYBTZmMp+c-8-!RTG*CKpfFdCi#b05sL#aPH`-%kl zgq(}o{-_fI;RHtvML?f`W#D=s8r2kmodWS$e!apJ+L}EqFc%ybf)h-Q+gmD{WcMzwlgRsMLRz9aE0j>n#z5pks+Zwn0)xlBxn05wVWX#YuB1s?zQpDLe*j6Ji!ddKb8m1aCx#>V)qUXbHs zjpg+lf6s2(G_?oPBm(_QAe+f+bJcqJ7`l~; zveMkZg|b*MoFYh%5NJX66$LPa&hgykNNPNeV;<>$-CyE7eowe@p~WAr(6$nyFS-QLG|u*{0czQBM{5~PeDEC$ z{qwp--vvRX;2}iNmHgE_@#Q;O$ejqPZEdvKX5tsxx zDw|qJ6_doRm(oIFt&`G&67ySxer2OBb;oOJPNFa25!j*rOTzG8h<9N-sNWidBWTuW z9lxZAkI)vy*;H#Pau+$Lp|sW4;3zzN53phW)W;I9)IDR1RX^vF$o3@gKtAV=(H3(P z6rs#Vvou$>YL=#b9b3q@88LofA*CPkI;1ev1_C_X>2S;7R1vvjof(G28V3>;&u&OjJENXu z7+|zMY}hp2HtCP`o7IZF+s5pX*x8E#vG!TXy7%VI&qtd z$5BU&X4v|Ld}&{YO#nW<{j@}H&z>1k30q4qKUQ78n2#+jp;?h9fQJAK7ia~?!q|xZ zW;Rp=IU}fWQxX6m*SvFKzmu^x z6re}xE#h#KP8Xa_&c<>|#!)dwtZMaLJ0)~=}x5? zx*L%Y5s+58yBR`CIt6Kv?(SCU?heU;p_>`vyV?7F_I5wN{l3R{9P`J02srCL*R`%V zuUHWKhvq~Y4If4n#wy;Tn$0V2F_KgV#;k@d&^N`&fN~Bj3 z1kJsUn9ozU{J!n8JPUD8RT8|Z7lfZW9DjFU4Qa&~t!DPNj_!x+Q7(I)g-1_RzIzOe z>W^d}+O+)0e7BcvvvKVJn-k79oh{5M)~eH;D&@OfCaWrSBk_vyAxCp{@?L2SDBXI#qLne4i5x+hOnrPobpK~x=VdRxiH&PQ3i)|N>L&~ye>v8 zMyshCXAPV0yqkGV6!gfty6|O1BxVr}n^7?RP6g48bZa+29`pnJ-wfPK6dz5BWkb)t zUC0rkM?FA~@{vG1i11b_ROe^cN>{klCL=$SJojqR#H`dJ;yGug{#y5O_K}%NTY+kk zvJO*1Ns@#f_{hmQRS%zq!#HS%yfW8J`HkOjL4ny=wV1>9@&(RXm1;rE@mLXSQfJt( zw+=YR^KcvWct~q4=avyOM?T29i+nH~n#!bAXZ^*jDmP>(BP6iXPl2~PjGj2JdA{0; z`mOP6?xAFU`pMYGu+!M_Ij88fMfcuBlT^l*3jg4wh+jnpchuOJ#4LkEHBq5(9V%ue zD`q{f+b^ECgOYqr7qfg=Zm;$>N!&h->LqMk=>aar_87`St5eWM0-ggQ0`sRuRX zekYW0k47W)ngKoIAD6YelR)aG%|wbC(wNXsh82J<9_e6gtKWhk zj!&zf+Z*H#m%Moo|B}2IFp!rikhx|5;&XH9$ZLqOtO>__qTBA<=mKyRXVq(BLze0n zPvUCH;6%k11aa2;J1+W)zJ({kh}I zdwP1}x=hO)6UQpU%3s(d=!ViOOr8lw# zYA{0!745o{Mv+5cV$;PdUL=BW4-Xn2m916_r9V4LmH4t48Moeg-u3bz)7i6*qMJVOkLN+aEVc|#9B9hVqlJ9~-K0+?V`<$y z+rW|_3VURfC1K0Y%UY2TJ;!*68ZqcRUe1mK!_Eee5Dt&l+QN+D!_Qo;QT;W@#~Ck7Q!ErCQ`Pw# znvaJJIMTT6U31y7b9}b+UE)q zjSOxMB_oTrV#%3Dhe7nOc``>tqGGpsj0sZ*ZA<{XS!K1@cy`#`^9|O&5i+m4u6i{Vl3h_ZkyIV*1Rm=HrapvqH?X zKzl8T`J&p@LEQs_GNS?{VUX<-IsfCaAQ2*sTcmH%fGPWbKI@$Kw)y%=7yJ$6F@yuc zwob2wW#ZTyB672?`nC0ItNP2ob_R(TO&c zw1rqMzt?e{K+r@px*QU?{5e}6Ri*QUptoutSRY&9g;=Ro;l@~mC}~R zqnnK^g}Y70FPAy!tY#6YqK(ZS{qe4{nuv^4Mt%YpTl*r0F421(y$kfd<>#Or3N7?uLEZl2h#=$i&Rf}&Q)j8lRT}(`{2fcNM+I*u) zDv8`ApClK~=P*(C&7%e{VwaK_s)wP{QyIe+CdNDI`K})D{pB3VUt_K3=e9e(B)1(N zO9Bqca@WY1DC_EQp`ta+Lq1vpG?<0*w3rq4Ml*+f!E-A4gzk|$KK}Pe8h0wAIUwD@ zeSfl`RiNQ@g=nT3yYI#4>;>JR)1li3lKsg(Zi>2}zYTAwG@~9b4bd(S9M3o4M~E1c zH{VVyw|cuxS*yL>6smeFf>Diy~;-N9tO*Cc{V4~nTXe?<)mP7Mb{vCxlV?qKO)l=z&= z6%Jib5@zat6&Bw?2cs@xZI2N?%c#nz%e;v)BU%y09a|X_QYL^1$+kI{ojIOo3}H;@ z&rrygEwE-}MNM-*z%2d5wNi}(LrX#{2{^@4CpUl0STq*}yLuq$Ag#|DVn$(uoQ=)n zICVuOOKzF+Xc5ur)2U#}BcX>IMqUEb-;e0aS{r)F!(X`zZP#4jW^Yy9=J?#j7$zSt zp;-BzE@iBJyf-&c=(Uz(KF_HQ-#^!-WaBce>O zYBM%o+yYrVbZ2~5odOD;r&$SUAE-(1*&%(LV^%XX_RZZz-HPT_&crG8@wxcT$*b(; zg1Rf=&-KI|NlNiMDG6)wjES&2sw5=DM8?VoznUFNq&%itUNLMz5#Jz;@|k z9uWzbVllOU9ACn)S(qwWz~y|))+?!8tMsEorBaqquL;cH)YjZ&s+tj*cr34vcoE|( znrR4;LL#MNfK0EsxE9Cqhi=C+EX!Gm78;Qo ziQMtrpFZL~Q!*(rKV&EJH4sX(AMu$*mPMLP5TBOlI7h`93gi-%c^uhJ%}}~lvenfj z_RhQAOC2&@e7Y!h<=k|S^BHh8T6Ffd1T|(H)~8qE31U(i%FYi6%f#JVp;tB_*7-4K z1Gm(>h_zItGrvF+baI4Tu=f*plraz7|HZ67?K$W$!2BHcNjFea!ih5Vtf=ZKf#{U} zDBEt4_$+o$JCb@gHwvwYzow{DhQbJmERt-dr67mumkhP2p?w`BW{p zkl7VAA+Zq6GTvVRT6N9jNfeS2}1R!#&X(!|2_FYYj4ofjXk#qWoDA_*#w{=idw-zui=2x&U&J`lcCqY_`)z3`W&58HExOByN!5KElK>(`@;o-fBPgFBi&(6%C`| zRx$Cg!=&{vK=BX!sh(&Y&A-~7&2*uve?>0W&JKJBY1T(E09+c1awPC`wC!R8%qL2| z=VCCG=C(|ij2D@jBR@v08RFb#)$$GgG?uayMRo3 zo#J0#A38L+kF>2fCZoj? zt*-4il+Xj>W1oRLyUYnIWtm%z8fku)1BR?LH_1l?EF{W;oSbjxCJtz}x6Kf?H;2GK z4&wXE8YULbPC3aezw;Rj&bmJ0?wB05@3ETkc zEtDm5MA5!tCzyZo^>-1GlV9mqYKvE@YTZeXkB>-9-mV2No(;1Q+TZBd+m|FA@BggJ zwx8^iZV+m^=qnc>i-BhnbPJnkO(%U#OJk{9wyleM`A#L5^=sGz#}9n&>-h_ACtzvO zt3>rQ;F{OCT_*VXegeQN_N+RcCdgaMG;%voeRRuSR{Her^mE$JAyur6a{G7uJ85U6f8PG zZz$ol4semN8X1#@+jtA4zBp!eX)9nk72T+>m2fhp=L6)FKf zTy1-)1qhgQ#{9nbG*37Yr9$z)_@N@*^-R`;iMV-xgTP|UVOKXUoY`Bm zLY6oDYsKS6J9Xno$IdrZT1=`h$)n;;1r%rMw`0?1P0KHnLQ&<)6k9gFZ0W{AS=z)J zYXv1*;hM{L9t&P+b)x*BJvyS7=-jEjOWDTqS9n`dsX89Bm>fr0th<$8r4i93+Iw$& zFNW5B5MFz1xJH~Xi!2K)lU}siDK)Qkd8{|k>;42+C0V5am=Px zCwvC-q=l)e_lX(%6H3{}Tn^@#gRsftHJ!{K#&NosPnH^8`xaYJBnoOhaDyY?4Oe25V{RB6;&f#_G<`f)zqJu8bIZ&s0ylSd(W&4}<_EvG~h%|`$B32r7D;DUkA!qgM>v$a6XMydzF)wD#P8O^-tGD~lK7!N!2Zmj>OO zcN=Vc*G=e~-8utGJj~@Y#RJx>ty7;MCngxQSYnci9Is1o9FBE-W7Vu~WwYyuY?Iw< z%zM72-Y!-zfru{IzWs-&_!0d_RF(ul`2s*2*wuLuYYxCA&ZdqoCtwRXw~=)CP3Mt& zD5iOSsE(d4F;RCrO1yi{nlgET^0js)PfwD(?_Hs`obm8i>P8y>$x~f{%UWo$T3M=37Fh5x81mLJrdXoJUB6 zH)Mq#0@R8ys<#&JF0^tskz!&&$zfntKE2%A(~|3E`-tX?D*@?|H+toDWiz_)GrY><_@U%Mczb- zQF9o36CyX)$TwEo?C4O9H~*Oh5H0!*2W{)N-DB(M4c2Xoy~(9!l%7WO@R$=2jATbgt!8nNsvRy!^G!Js4JEBo)$e8)?!2O@l9 z&9BiC!V|iuYE#OOU|;8!l9s1mmCVs^%~fyOwC@@^00SkT^~uK?E39|(iS#FKrF@u} zEYnJ6RTC`Hl5yM3q+hRXvf)0yorBfwUU~#^*-Uaxud&4tBi#=owA zVnIUgF~UvYyD53qC0#!)Te|W!@c@m@HRT{Mg_Q0$Kn|%9Fjp4s_mms%qCG0qtX4vg z#jms~RHW9h4Z)d8qaURl@W$IKyc8y{fAGloX80%3r!#bp3^jtFIim%be)f@jwdV62 zn^3Bz4+c!8U?Bk7dhZYbLa}}DUQ5h`n+xuzx-U=IOy=8H ztq?-2_^YEuc&ylUv(+2SjS~HAOd&tG$4|9LtMY~ivB-uK&$6^RFPS*!>(Z@DpDsHG zMsx-M{)S!0kYCJS9xU+CZik49MG}{ciWU8sg%v%2eldEW3ep>u}c+#ZQvlkBiT^4tL?!PWDSln>CtZifN6jnnl zH7#5lk=%N{J!N6Tv$LI)0y3ZKxizmrd3|az1N)XI3?BaF-rX=}l$(xT-*TUwXnGUg z-ExB5Orb{ZZ>`^{yP!+$wT}x96+!?KsUDkxRR3UF4RJ?ksjLavBzJg<=_L-DKmv=sfkTC3 zIcT|F@D{5hvt@8@sNVC{-fEQ5ulxR$vv@ps5uGG8w%mi3cpn3M-H5$`#&&iJoo>CHmy zap=<=zg@d5z-t$0fe-{DABD2)t`l)x>dLsPILertps3WRb@ojdVbS!2V+@P5DpPocw|=| zci$0Q*@XPg2ef`> zWFkqvnCQB3bd%>h(=55VrJQb05m-#__YO^!&Fm1bk=HxaXXoY;ZNp^NTIw)kr}nhu zLZ=3+`fqE_SQ{!2K*wh$v4#g};Mwg)<6~UvTjs}JFj^0th1~&OOTaZHnZD_hHW?AR zoVnrH7qO8zKrXf@?g+Pl>Cq!u*Uz07%`R3rb`~F)gra#!;`UBm$F;-`n+u-ZMlK_Aojs7kEdcLy zHBR$V>rzo_Tp0JT0W+jlblH_}*Kt+vuLj6x>*R5F%yy}$6!1z54i18Fl)T+6?{Lvt zz9!y*^Md|i)mL61;tF5BmKJ8?pUOFpB%v)f_X1za_*mWMXCKA6LYa%;q zZhtZ>u^7vYyS&DXhH=3~mY3(Znkq=JUX@L}Py1S{M< zftoJ@*3B~RdbYJztte``-bV^otLjnyz%E(WlN}<6-RbqmKJD znb!r2IElKAbie36`m2V{NQV%xrcvveZ~#@NudSP5+-^?Sp?t>%(JPxo!2x=L(yKVK za@y0|eE(1{t}^6LvKDB7zxYw<-rhJpNX3(p?C0g}l)&Oov?41{ZA0tnUTWC1ebEgclKL zkyg_HA6Y)`9Wz0i`=L3a6t*pLb?YO!KZEm}BXjx?F^1!N(rj7plLFPTD?Wn~;_$uZ zee7nJz}x$PTXJnFF)f;I4PVf--cAAfR3%%#!d9E3Z->7VI#t}wV|`B4M!xr{^#f0m ze1Jp{R&Qlzf6lw6c-I15SG`Me0q^InA7UugtDn=+hQ{@7FdPl@$2ICL>2NWPL=Oy1 zryq!11-47>)AAz>2`ZjG&A$6xl&=Cs~RyO<{C@1^wyFwlMyp+TjFdk7<7$k!U8vSJJgj z%Oze~DnFMU;>S25jNum6z=+Mwdh)Ms*BCTWF^RLURa6jhyfy9`@EA3?D8dD3RsuzH z4UN^4kKx{6Z;@W-sSRo4%?kL|g@tg`C&pW>Ss`K~tOPkelWAo#ya^y7Xje7FS5rsxPIs|VKM64>#mx}*@;CC^*e_pvK16zLe zsQo*|`ahT#pxMrbWD_Ym1BB;aBx*azH*P?+c+$nwPKwL_ogg-9ZoPHtvl9VdJjSrt zB8s6rt`v1Xf{@pA!Vn6W=If?7wCN|F3LR}e58Z#cm~&vtslva7R-=@-68mC~C@%aDMsJiZDlvwGSge$@xSMdqx?c4Kxbn~JhfizVUhqK~XlH5ZjFwR| zzrM&BCueMJ9Oad?apQvh+_*|(ep>>zu&ra6pa^v8K{aC{2M>j>fSj05zZy*^TP5y) zqxkzat}x#ty3srJH&TB=_SXZz^#D?g_oL?z_VjnZ^!I0M9SwZ{t=8`rq_NI?h)P zx55;#v`xCZOr31I_x)=ue;uLz*FQi-o+{H~4fWQc0;pbl<+)@kRBW<1smBDpzkUR@ zn6bSoaqXa6;KuP)kd_Vv+#u;-{$yI;b!y&~D(WNMunHBH{^eD1qgjp1J zQkhtV?^I;(BFTi9sL9+V=eTU>r*$}GZx8n2IqW+N;O>@6n}gP}HhWDP%wg}{v>ecUVYpq`sgf=Gl+S+sGX@cxhPC*EtMEGA z1c07hk#Ma%m{4M!sW+oevlK)~$qwfgvWcmN@5Toq^PbE|e6tej z2Sg!)Osn<}D*nHZ(F&z+U^r3Vh(ORxtwyc(d$Z}%l9xKo6Wk&(Kb1sasw|D`RE%oH z8ls%E^6_!41iBPStE>Sgf9(omgp{SLvPSJP5c?jma|4%B!lmwlF{6okf?U68Cys%eO7q()|0;9p32py$_1Nf8!4D}!=(^oD zQP?Vw%Vz0yQk^AP)Ry5^y7%qV)Edn?U7h(=C+HLiF|-k-lEvtk`pG8+TVgLGhv~hG zp0ERx9`!r8I1!JTB}OnxBPhf3g7iYRM3a6VYrI5*$K$9>-f$<3QA52c{*AOA#pTKk zwp?ArqR~!q0YZNc7 zSr@(;#vri}BfZE=b5n6Y-K7Ln0Zqm}y+&#F$D9xZUjychEv>;=^FGuAuP4fB0yI&d z0clnI`6g@lUspSe?5MVVw|L`iKi%RY2eivojytWT2ygX--nn(073*V0sfqir#hEEr z0xfl|m}DLubgIN&h9~O=N<>w|MXAbz&OIRs4A#lF2lQ$2lr9T&Mt$)WgKai5Eyt5v zdcQh@p_M1zI@KDA&;>@1<;KzNYDJE_t%p-I*0G}E_tZFLP`KjXdAZ+A$5-Sf_yPQQ zc*uJ_Fc~SKnC*o6YT|t-Az?#VLNTkxxV;w_5crz^h_u9K)uq}x6&Bdd)Ht8#7+?_= zGHfBDpNaww;^%nI5-#Ix=-4yo+I4w%EbG`kSzDkGgh^RsH71!xfoo^fBv-IR(Du5A`i`Z2XhV#lKC7|n;_m*I? z&bZDli}2hrqMnnZH3&Q98xWphk^JQE&huybm_1#fB?DY`M7yP9TUxPZ%bTdKB0DcH?^}Y;%r`Q;y>4y=_Iyk|h6E57GX~OrufJHMFwG@H93n8)$>%J8qH9 z*KVjJ5!F9B{PjbnSfheO!F)v{!6{$0K_xA*vm4lK0jexc}VkWtmF z24Ea$&}6ajgOLpx)rR*W?2m=0n|cT}P_M$(m!m#h2vI}|pMO3&q!JR+aYPSWsBT#I$U!(T@#dJES0B5! zcSu;*#{JCQCkx-%r=EF^_0k73sck+arnhqK=#r=LDq&Z38PE`&>lf7Cxo}o<*~tP*5niug zDboTJ3_=!NFU=T|89&mgse)f02hI2#i_}*#hSjGnlLi-bCIR&BG(o&T7HqRYn9d{?h=4vgVjaFq$ ze>f$;st+3xG}2*F7?;yE8bvTsZe4?$P5H8zWiPGxRO3a+=!xT-RFCtkZ=4|t<94lg zOCLBkni!9J@}C-49?dpOIV5s#ct|w4?BO#PjX6aER_Eot0*sF~6!|Y_HYQk<&{3LT zci?7Ep?AXIoOT(<)#chlrDZI57H0W<>-aLCzVZS(X&yCbyf9O@J}Y>u>y})Txr)a% z%vDRxWQXO2Y*+=`+6EGZ=ra^wI)CT0`S=SvGI|F2l05n+nTWg6KYdQ%S`4ROV?4_0 zkTSN#@rs+?*;jvI{S)qtKJw|Au~?h@XJ(%KWf%*YEaUen*J?Lx%xVqd6>H-X{J_u3u6SBk}6zEkdMyNdw0x}*2o1(hGK>=O@rCEaYU z;T$fv*eWO4NwroQxa8^F${ZZyMzRU)DnN0aMf^c6Hp$|P;o(NmNZCJl3g70ousxE7AmpCKhdS=Lsyt&@;x0 zz>Ia5caT;7@6y8D&M+i9DOx}%X?D80zN2gC0Wg7-pgfI%Z$ zHhNb{r%y7W=Bqye6N~m?Xx54yB#S)=c~z!;(A#)*i#hV2?qmh^Au^T|el-#ianp}u z*{hWH2d8%rO?B{t^s0no_xz^SE5jLpDDf(+9T*C;!7#N{s#ap&lLqZS8LLoq+2gN~ zi2s0sewhkbx^p<%sK{?|i?2SS2(_4B<7lmk?E@WQU=&yeEt#)ID!p#Z=mA55-^Vu5 z+=Ld}Df}{4lN1fW=WfqKb|kz{CMNP5Y06@glM1Ws@K{nPB&QNpVsOf~AKbg=2h@^x zzaOzr=&I7<9Vc9q1EnKVD*e4cb$r6t8ADN3x0h}@&yQqtAye&g3SCwo;)lPyPxQ{VdAAj7HfS!9T|}sss>&_7T`*Jkq}a5VXUWL`wkMld z#rgB!F+oh4=*PBHhV7tk9@2HU)#S_!>U*6Pm4eLgviDMiCFKSAN{(B)=V`FzVxt0Z z+qVA8w?~(6BjupnC9zd&yYE`50EfDj$l5vnKmRF3`V)pQubm5O7cnR$eXfoBD8IK@ za*FC<{cGB4b!-^1=3^uA%mFiHWq7vL&PyZkubDRtN%)H7#a{J^f{10vrW{*@9 zHI0R`wSQE^zR}Q=ZzG#Se5mz?hAS{av?!CSr$A~|oj2@^_t#ggwgHPRB@#=EDTE)t zo<>A=7!4XUwOFU7$aHjvO?&Yg*qn_Tt~+}Uz^n?N_p#34Zwzd&3l%1D>d66smQ@zQt1y1YTqeDx*V? zOq-;(KF(J^nsI$LhrKQC|F){Ott*ei-h3wyrE+gTf&9}vx=Hh?E5ERLo4(P4Xy5;+F6BMJV=|Ic)%)fKmf{!w#Swu|U$$?8f4cCWpkxdA zltAcrih<$?T|0o~8qw^qkU5)miRrNp)|t;zz^6TT8q=s(eo;iGqZ9#CO;}^+T)fYA zo~;tKPo6t)V&Z+NU{uC417$`4Sr~kAW=iIws>dQhTq8vt&&RpgTo@g7)xNWNrhSO6 z&5B0^BiHy;E4`zzsv^rqn4YG{%k@&jcr&9wUaSxxpzAJ}ieB8YsGc!T>41T6FwbSB z?$>zdS18ukz;*f{d#hbXxjYfP9T;Ylsg#LJS>i%8flP$3y^J+1Ml{fj!i2ibcZiXV zjXv~A{5_4A)S4jsjpJIv)D1y)#~3II<=xrb=!yF9Xefn{_m7G!22VmmlwV)U+`KIG z=ggzE>lBBnp+{;TwkYtfMY~^S-LRrOAv`Ru@A&#C4Y!N8*tOx&v zWx~^cU|E4a5eV%jkzr%$|J5ePVtUt0yW8SuxHLXXgED1HZ%Oa*%UUW zkFf@>{jAz@m$L>q=k&P&n6+Zo(>@W4&9+ zxrb3{1Zo101&Ah+c_WQ8wMM>5Z~oe5bGF zVE?HwGjzAX?W9)NkCIi3BzY3;;T?b1eMnCQ8 z8Hm!W8(! z7+qhuwQ+4`tHTKtwbye(=frS~`;S`lL8hWQL^QW3KPz}rLLvr>&k!uh5<>^(-QxoV zd!{ z0=-dQr3s}vFCo9iqRcmQpToELa@R6C49W&Wq1NQh%-+;|?cf-p%uS!K3=H3(C`28vah0>{H1|HHm~%US$uz{)2-4iHvWP_{@$t zc&y!Kev>Ao2*GRB%T`YdapWEvIg-o_jOrsasyEKAN*(Tno4dU;*|H0sMBz-OKHY z;`^tU>$&RFR@Pf@^8KJKq9(}~KCR_=W9&BQgZ`%oleYP6agP$GaEY}tJB>g6P~w?Z z&tAi21jMNG*lW{PV>iU8RHXLf$WgQOmwAWHBRi6X5Fei*LEUPpbOQsSLfl6A4#WT| z)4{uU1h)lkxtu`xJPSJeiED3dyN__7h2I_H_zzojnu~V?EST|sXH9&6QjhTVZ>0aQ zZF!^i{)Wj-H`j=Z7tdlo0Lk2kRa-0Q5m7r4b}Ck4)sqTBhbkx_6da_}`f~^l^pGm_ zb8IoU@PgYn(*pUWv5Vfv42GM~ukZc3N6-5s?8r)qX02~H%2M4Fu~zK}PiEZ0E6BL( zpxkvQ3~~ML)@G6tdAi@J8-zwwO)N0#_FN&+X~>!78xoH8g)ntdBueLMof^Wf;Smu>W24}a zSxN^EiA|Dzv@hgVZO67R3~RO8xFE#{V}q1h5xRjel3}P?g5iiopSg*FOc^Uk2-Q(* zytreHvqA(fBmgj3x@_l3vFQgU70=(2`TcR8DTeMD$&&|%hkr+CPbhuA<$H4taGzRi zz@9nM=jw?}9|s%5NY&rHg$P4%Ws!RQGSV|$6O)hZS1q36%^@xR$~gRhYShdWW9jt_ z^{d8lRfI%y*}TolK#?+OkM{TVmzGeXj$CkPxJTxP_vrkwbxd3!3z0eU=!i&yu&j>W zvWqIyNc#Cdh2{_eNFjjP&>Ve#Q#lmnI8H@kITTQPga%N7m?|a&azS~qsrJO}jFQ>L zq+?mN_66<%*A3H6TgI;_lm!2m~{i9fftXy`Zv zQ)ngI7Nh@Xe_8VoZk<;NN|Q)t2fLvuSzjo0%Cdhi4@Ig;4Ca!J5;%AkQITjxq&(#m zc}6z!ED5=@55TPKdH1JS@_myEB6YH$KP4{WUgD@d$&&uwT|p`~Cmwa~~nz&Y%A*9t(&h311)KP06I=yRU z&MUpHU(?Fo+VHyz0%IcH@eC`xl}V;uEo9^iX$u#9tO3|&P&&hnHP$|wj!aEGsnDt9uK>FCqJeA zr2OyG05D|!JTyJ zO#{B)RhDI%A;3ij!yB{Er>%b0NVtOB=Q7y4h!+ z?UM-B)*i0O^3jgnyzG zLo-zwSjPz)G-Z$?__IdjNbxw0pHFp}NI=LT1Q0ens~4YOF^_dvX7VHgU>;g9xd1MO$b_Mo~R9;Ot_F?ysZ;QWd@uMw{Z` zK>XJ5RFT&loSf?+_&>4%tM-fM$q%9+m5ctSQ0a$c?3^cOTX3aTO15Z^U`0`r%#q*2 z^_Wnfp~@PM=y%pF6B%~-iT(|fwj;t|Pn06me(>4E zWOeUKRRc^q33H`o!rz+e@7HdB>dPu}qbYo|!kJ}xpVl;iRHdiXLeq6P)?rQS>zbgA z#b6t`jY9q-J;UMs7LMJBpmi@uXT|VFv1#n;=~qnq^>S?{w}&yh`;}=PMcUc1DLXSe0h?!UOW!#iII}edEcLl zaOEtZ7+K)7HIVXxc*-hnn9kc0IRaE7J4$st5t_AeO-9hmD#98wwda&WaWGf_^{0NB1QfrO4S>)29qiN&;tD{*5GmfbI%3jfCDejqD{bYH4a#TeHB%ILh{l zf@9YgyjmQJ?XmE=3v}qPLoY`}`&oz6Q#70L`T6pQb2*tlbe#!2W`oeLc{wXwC=*H6cY-T z73&61-hLyn{4{?kknGak!kU`o9D84Vmgcc4wLMn}G;CDTHwp$#mNvLXg^|P4K#U&n zz#{8tGOOXfr8?`EKf_MaNZqw!fxd))MP_(vN>cK>$XLwbjF zAq+NT6nkEhdf|6SKY(L}yj!Qs_R+?A}jcSrq6Uk0kZ3 z_*j>wV{{w&I6Q&k|6}j1;-cQ#KTt&k0R<%mDd|Q5si8w*knRR4ht83q5fw$GLt0>l zkQlm45Re*Rh@nGbsG&R0xc7eF9q&Fj=kk31cfSkbS>N?^tz?@6k8kSDN(EKCP>-V_ z>%(xdQ64K_#B;nz4;f220a--2Jpgg(A9%mdKV$rA->?P_*wnShjilA zBuZ#v0+t6@?=;p)VVvPFY0>{s`@;^GE#SDe)bqP;Nb!CHTD-u(ye=Wd#@d>uJko=k z3@-5pHF~_}PQ8@G?dNaSMxf3Y;$`;jYa2|{h!rkr+?(&w7I+?=d>EK+3Z=oJHKc{; zC?It&w&LZG$v*6Z5UG+a&fAz;kBtZaa*w}eK?f?sGpHq>n$cZp{eZDDJ=7@Gx;b7c zxP8-M^iZgz!S0AY-=H?@iI~M%!H3x&iDXU0N<(QPGiT}%Li}V67O}<;%u063nkPm_ zc_+gY*^CkkN&ru>{rx@3BMypXGp|sMc2p4i#d7!lkXQHHE9v}YE#8JvgkRY5reH_9 zoCcTn5gnH%xXJa0qE${PsUams!(TN?9_FrqKO>_5mTP6Cn72`*%}WUmq1_jeQ`~(< zcugGr+~gG8xfPw9D-5Bw?$x|TP&82$XV0!v*%M}jt$**@w-E`O z-T0yK$UKo$bIJBwdCsKgOXfi5i5Cq@aO}*VfL(`)Sdnq(2HSqS_uu;Dzi#+BN6D0t z&XrMgiI;6q{R^HfkArXE6Lo%BDvcd&`uT~!5aGoL)z5eiL%GrO2*adqf|Z8S+E*R`4^VZwuGl@d2X-C<)Z=@PE`|zQ&ZMa&W{p` zA9;A5dww7^{VIpS$BjHtgyeX2s2m3T@ByWd@@g=)8*q>JqS*;pxPS4Sm-RURzwn%r z77ljhaDp-3?da1x>sCVYOXq)%-Kz zHEnD=dRa&d*lPQzs(I;1H}H$eMbO5k!IAgOMg|MkG9_4kT39Yhwnb<7bt`vFpi)FG zIXxb)e^$}6&Ypi7^DDyb!v8$|kKU{wu`;G{1~=Oe0ZWj&_Fr`(Fm43BYWzdgEGG<@ z7!$2d((eJmsTXcl1yq~6B<;6XU*~^5B>ZQaI?3|Q6xT^?OT(`r6qg1ewZpAR7XS~x zzpoM*dKJDT~!^U$*osM1RI^3>5`@97xjb81vvLcJMQ|274Ah9~M zA=m{%3PD$5z??yA^ZyzUr50bD(<k7%s>-)47 zB<@z9G*(IT*gbD06Z&6UYWke{e$wZqIM*_M!bGX-1$G~=Rld|oU+dCZr~~XiBXR!? zXn=Ykklv;Nad<-oqb(Y691lZTk;Naf`E4hK`&oE`CqbaqU5tM05=Aeu?$E(IG+!FX zF;Miw5Hsz~YXhfZY^oYS7Dr++J9e#6~A_=%FYlVP{9(Q8AS z8%}~5s2JTODsEv1@(G{3DSr1==7RAKOQ=a(}gZYGw!oxm( zp6TOxkp#{A^k;0L&sUx_1VO_~mJb~S6n@(+@q2BwCx=%HZfUB1{?`_A!Jw5c*%!m_ zXUCz~RUtJuNoa(ML6UM|596aXeaVkB01g|Ksob#3!Ac?qN5J-TnNdQGs29V-O&7 z^;c&rOHSsBX6D+b5TP%^x4QeBpXFTMbYtCr3!8g27H|{!Wf!19^kel%%pt3AG?1*0 zNI>;SIygse!pwD`$4OF%f?^NLa0WF2FX@^~bu!Lv=iLv6DzzRHYm_|dh8lrtFE zH@6+guOEMcohtM&X^;vgHWJc0#Qxq8et=T{X&hjYEtvhQ!Mpq$a?Ue_xX21DLH`~% zOBL%Fza+Cg`!ht3`i$#Y1qT+<#p!P!Q^kpqCF5gL2_K(WR1u4lB;OJ%4d#HB%WM&F zrGB_5$UKNb*anc4B~SW{3O7vsk_`2K5SpdZLmPB{S9H|v;D(CGABTAP$N#(gI2h+% zbqn5~vWs=gWvnR~$8JWb3I|$RiFtlanw#4*=h(X!XQ{0hVwxdu#5OiHO61ILb~b3S zTE#zTYe*E~_1g2DM@`N*aZ|G|#BrogZ7vWV4_0th(u8`z6(ZS>o18n9sn;r#@+j{l zI2*Evt5jdu{vNjnv#@J0cAa%@{#E%MhqgG^ya=eAT#M%m&zF@cSUn0}mnAFwAUi1@ zVrU0QEQi(?*@2iI3b$FVR>=>(FvKHXC1ro`Fzr)uPpV^3m7G5#0Ko8JS!HtHZ}0&A zWF@~R`8>MPpIFL6(BJj(g9###u}^Q`fCdNMZtPpZrs(ikv&g`+`~}TzR@#JUS}Ez0 z@HZsuzfYmDa9|A@ndjV!@Si4^<^J_F>aRrC_-^kS4+RD-%jFvG+FKbnt2Ix=P&U-w zBDnd`Uo#%$yqjFw$J4IsLw7^}o62yr% zENGds^6)oz{zK`bU^#*ffa#z4?CYdTv^c7B_F^lYjEP==;uG`+ia-IBd*5uDJ8v zMPxrX&XLBAtx5J}vz!U#cFSk1TPH;w`Oe;a%av#xeHPX*+hmko0d3 ztiv*;`i@0DTK1k&Bwa&l>piH`8@VJzOwql#{2uN7xBKTqjHR=EeaBA))ZjSp5qN== zNb1fU(DtR?3go5WzB$KW@0}i{)6cy;dvh@#@QH)_eYJ)ne+*W%L58h_2^LiuCFL{7 z7|OF(MRWHbzaTZn4loegn-k@(!sV}$AJfZje)ILermQ0HtEFi?s0{Gu3FZWG+UNUw z1|CEeYi`dW)F^QF+H^0%)0A~oO@))b0~HuXJ$>BW6yl4Ar%P~<;Mw8K#9&(w_@CV zX@|ci@~)G<$Ge^uCQOe6$sfIg1T|}z+376XP-x2ZBrjK`8%iJVX|t@j(|n^LHF?y1 z7G2`Go{Zut3J_{V#6}?~FA3{zBc(1$c9BAiMY8H?lIwOKt^Kma)M)ZKNkFc%A5P|d zKD>d&O4ad_@u^gITK$G1cq{l-9v!rtx$QCg^~Y&8M)#3r zIas?oN^fRv!8;gm`_0LzVHlz)hIRFEAmA!L;F!QTSS;04^` zj^*irq=CAHu88WYOen&ehX_NUxxQHF`G-``^>=+Ea2tz^gBd3_+U5tW5REy9s)bU@ z3<^sgamm%;hM6atVpH_lkJU!q#y_TMjlMCdTJUdF=oUyUhQv@1tEdzIc={jN)WUMp zGho!gUybW)g8iqj5i$+*$rP!puI8vb>`sd_nXS+~4|=2*UI}7Fu+K@dylGad5o;{e z2Q46XQXzO@igN~Mje3DCo~BFYpLhc=wqN`rfsNHaD4YM`J~LqG+1?OQg01o3@pG00 z%R_0KbZgtu+j()XJ=lh-NgQ!ZmLi%7tY2ti;(80cJhj6;c-v;_pDml_DCLs#EW{ym1HsK`a^>Lvkh*pCE-mCBV3}$7+OVaxiYZM#dHqDe(mCyZEs2|iv z{AU+Hva{Lkx5>fIl$ej7LYCD=?e6(GzcV8MlFW?QQ192!aD4in>ONw1Ja5!h07rcZ zDmHYk_Ea(RV(yEi>al2U+hMLL_L5G=a<6O3za{d5a3xshguLBpBEFWtWw^7qLi3rB zn9`oCZn6EI(&U{gm8dVWMRnh($6gut=WzDN{%jYhVOG1?PF_+IbvG^%G&~dc*zx_* zGgm6?FX(yKJ`W(A(>ZeBXGJX$aa5Vr^Q|{jD2T56E29-aNI)l;BNrg~tkITws;J>y z2{+-J|4*quZQx(S;~F-N>rd26^BAh0l#GtXF6$Z3Z_Z+&HZKdVLj~GuR#Gc|PK)h**8yJpKUd z3eEUaqyLwgf7#S+T+d}tiiA&Qc`87b_u(B;z?15O)T*s&cQu2$gK?FFI-T$1QFzW? zISz%-Lo$7c4_*U1*S^rxcXl1uQRjAVYr^^Ow1~W|_+%}@&NKb!MkGPn3FZGg z(C~jB|A)Z;-w{CL5{mP8|9z9w-{g+}7W7MEWv1tfQq#&~z;8P>|6M|5a880+?(e(B z^#r6x`b=iZYj6n>XF*gLV|I2%$*K~#gK=t?1!DhQ`Tvxcv7)S&mt64hrhoE{KqvG{ z*K~g#clhM44<*+f(1Ywu>vC^%bLUR>@s8lXZB+T|1-Z(8xv^#~mts0>LqU3sduzru zSeC(HMSw!lbXXP2wbRrz6`{`bSdGwJY)_Q)U3A)CfBBE^;LrC~J~H>c|29ZDf1h8L zq+`xN;>{863$Ju<%n9d9*N1T%>$H2PyQjU%$U{O_Q1ck?zv+qpV{UQj=A*ulz#I5 zT0a@X9X!*A*ME&;f>k08{ubH(UUOm;3by}YkKHD2%JNK26XH8Wvi@M5MQDJK>xJ-}er)tBB*Oe#TkGKdENF5*+}|}NIzk;99&+JU z&(L$0WXxa~`EOVKs}uS4=#^|r=IxjGafBDRUhg)Z-w1*lx#AzrK$32@F-V0M}o86(qGm)p9`#iYU!Zl zpqn6|3%s{aWqpR9b`ItB&yL?fHA^{mMr9(UImyHgR4Ma^mYDcx>arj znMVV@D~)g27!~V+UJFr_XyCq8C#PtCy`9NK$n`oaPw5yJ_c{?B>+W<0{{Zn zpxM1e#J8?T<=SA|S6kCpXJ?YHY}KT%JZ!<}v~mxT#NFlSAv}OG!0t^*9HxoT!}pci z2O?QC2<w50Qf6C$6JbO zGv)ds&&Y~+o#81&hybbnsyxTnZtvPAQ>eMh6+17}xGlrj^OJRVc+6&$6h~676TvF! zswT*p6U2Asfm}t_#&otoE@tL3fcz>%zyhTsWqg%%W3X^!%w}S7Y*}F%_{kIXsnM?( z_ol}&{UU(pA=&|&m_XGsW=N_|2gI={y$21XdU_^hW)@})Ko|9J3Pk=qfCTb)kr3t_ z$%duDO8~GzPB-G+%62pqtvfZ9kb{6f=Bi&i*0Gtw%~c(atSh$TBK&t8#|X}XtkJs3 zAv$s*ME=(ci;(kQv1-%;}t0g6j?(`NID6{h5KBkkX0X(F$hh@HdnSq?GY zXYx!uw8TV!o7E6R(O3UG0W;^140ltJSr~RMgqlgDI&$i$6gk*+i>}`BXbBxd*Q)H1 zgi<0=S!ZMRYDnK_Se3_;ZC`Dw+8Ya=SUU8M-&UfEd7;9+5s)9gCrZAsBgsn(b$rHM zxCUAO8P?03`*9LS+*|+DRpQY^hODu)$@Yyrc38#9LVS6;xiZ=#7Of9h5HxZsYONtq z^D^rn=DqzV)BJ~jf&-XKBE-)ltnpPf%gg%_UlxHa)|Zb7kL?6BI=w_A8((90{;hMrc`2BNo8@#?>M& z^fm`wB#sfl4^z)AQ7S_~e&qsXWq{iCKvlf8F()w&)+yPD0)azJgnE7W`mzrCfaivn z3x1;iPt;^%rq_SP;=dDce<-_sJ z&GiqFu~je}zUO$9qQwG-8QQg&@ZZid+_i2P{BYHeG_^C$PVLh_kMgMER$<p?#RyA#&%&$5#rUzbsyf6Z(?Zfl z3WHh^TVKZe68V!Dj*W~7HoD%AoQ`6vSsbcqDHWwIuly!#Ohj_c>e5HVz;AnY`!yc@ zO*j31317)d$`>1A#?#O)H+LHfh5D7$q@U8s0jx>I#Cn6>*WDu4Td9g+ahzwOuQ2_wc%D|nu#2u<`KJs^cF+QWm zH9n-4|71a|>Cs_#%_u825;$uFZXG2q8!B0a0;d{~C%m8tZwrzwUz@dtAg}ZRz-@k!_uOd!j4(j z>6U^xTs)PLH||y>Ljp)$!hGg(9jGw^l6(repm_`XgS64|xF7I)KOPJ**^DIR7-PCq z*mWnp-p)BPLlgsYi>CLgrVmdgJFK9pto?$`H%t0#83IliTq?WEL$J%C zu%o6M#5XrTQ)ZTs06T5IS3Wq05|&tT^#yNGv|El?8A)c7iyHbo;w_W6)a@ux8sgJu z0w@G>_L;HBhax=-OpBI3)SFeBHtWyi!kD3jsH)@CakHvk{G`F!JcT)a4$frRqS9y^{Dkzn8U>obCBeEE2ij>lt zYAOtYK-Zy_P}fMQ>ceK{@q3WGeSXfq4euGb(>fuiI17+dGwVrXDwwvBd&G$b5_OeCB~^oZv{Sv4xcN;-7JnU}2YtO? z=oU{A1IA(Kaevnix~#U{lSx`+%B2 z3cg(pB|z4^43u(m`luWjV-{nU=$z1@0tbdFeH$#AHgOn1=(%Oi>G^X2%oNu=8ZW8q z>Ar8mJKHagfQEYli4;)iQhZ~n0QO02CEIYiE7P;)NILdcDK8V(U$)nbNDyEOkbG!U z<2rSL#7>wUj2)H@U4=5kW|eI)2)WcG!t%1_?hh9NKrGw63#w3-dBfJ*JhRy z2E}&Mv<_?U{@308A1&}tVf()y_!lPs7e)Tp7~=msW*jPWRNdPlWl2gT*KY}_Sa_GW zYm5KC5a3S0110BQ%3;FV-vLb=(MUj8O85U>AB=z}k2WKEGx0JZ*7x&IX+8`aFSBag zX2K}85 zHi``q-;(tlkeb^qM!9BhZ=6HoRqmMEU}il&eBTRdLV}0)Uvu(bN@QH_ZC{*P#?UoI zDnC;;RxWZBR!nE77Y{{`V1t`X^W_q-lrVU_%fwQG{+ET3ufN7!?M3`0ve(Cj4H)?9z?2IwB) z18%V0)r=KRuRo^-x7}sPl;xXD~V)s--qzBx+ z?v^iQS8MPpa1o`}z@5eq8>c2TG1}!?g!W|z z<;DhwEY=VtU_7x;R|Y7y*dr&9;j9lzxxYaUx@elr}Rvo>(S~NF{vukC^%zj_Jeg82f|nCTMnlQ?M$u7FvV(8bSmW zSqLt@*qLb7oM5G-1fxe0g7X%)oO{X~@ny+)L^MOr!(!2_S2q-zjn8zu3Id1Be6!|A zOqiYK>BeVbW`q6fg`WB-?n>J{NATWkw7Yy}8>&Lag1*RI6z7jgh@Xk^X%~Oegh{n( zY~_N}d6oBa!5IveOPyKh(QeEJp?;a%`JoQCPs|R@@aJcItpwtXzxzGt&Q_WI5AUw% z<2d9mfF~M1U)u)J`ReXwkj<;C9VaLl>~8DbQ%r> z2zE)HX`VFpi)S>wdI>foO~T)HU)lFoL#aqV)oFH4D!)<*_|&edg^bV-GS@GwB@gU- zLpKN}%;2MA`jq$+97pUISQ;UuMtJxJ9Kdy9y32KBR;u>NWyGUUUz7s$Y?`OMdmTSP z(HoU+Yk*bt7n zsbHDs9#6eU3Zk!Nq&#Vba{LzAtpIS?^E%NgS3g!xEe%QrFJLTyD1Vp7Kp$-yN9=t8F4pl>0H@`YwUPr_f?y8m_?ROP!g~GP&m?&3 z!^c$~5qIz+oUQ)21=9z60bca|Y(D|O%e(d2j2DV=2z8WW0HMB6Mp{QkB(jsPlQ4PL z`Ju~2@VP7jx)fxQ#xh4#$nehlNe~=AJv)7N0cOW4y<}!G@qEHW{Q0h5lU6IEMto2p zVYNpbB)qEg*_k6_)H90W#mq&Iyj**-^|QwejdEzbXN#y( zzdOYcH$5PEkC86CFopR-;XvsayjWe!akVS)Z^!^5{*v;BG7q{{`~0xRyyqTl0>yb& z{7N*p;asbWzfQ+`_h`{_n#VX^WY`ew739A+5(#!123_q}=8Z#a_uf>H73G~s$_^=^ zVAk8A&{s0h@tzGEo%k+I3wi(puZyan?crLw|F)OhhD(kv^4kliOr*z z`?}_IJDV~%tQO*qpM$b-lBzV4Qv`*qg3rC!^1I6r!Yn;Bxj|Vx3y?lUb3~JZkIreP zHM-QFyi0e5(7tCQs<^(liY6|)c4%SpJ5~jtRi=6Nx*cBIbO^0B&&^+;mN+GeBc`%M z3J}(zJKD%`WI#+tlUm#^&+xsq@Z9F-_)Vf#cUOdhM|9_5$SLFpBnq}@e;unC7b=fb zR|!{jKUJb3Y@+=4CYH?}@EPCASv0 zaCRJ47YSv~{dxN#8{WCx4zD=n4WJ`jTa97ononPMQO~R5^5@7+T8=BE$R#NnzT0Q6 z+#^|qba^Z+AXzxa>L!w;XopMElx-T! zhy?-~m>W_FB(+M-*}oyt1gz8B1@?V{euPTwDWVdVu^n~u{J7#OEr?`k6)^3U!fB4q zFrQwlCQ=1iuP7KzjPn==;4o{h&>w$3?~LBbWr2IPoJPtRqTR0UWp)1Uu?4T;+Ukz( zL5*P{+>{G9b(lSN`HFv?ik~}V61z<3hdbCT>)pb-#ke)u$1=|cehi{|bs7`uy9Gok z!f?l`8zuThDGSC&VxyQ}2e9eJ9srI&MN}7`V6RX}!J)os$6j0AfFW z?g>=$DN%ZXl=*T4cjJ-S(>hJA1yN#EUlYLP2@PH_7w&m7A>4KQ$r!wNyC3E4^_Foz ze_mp9C^&cWH<|}TgQ<1fW?2Au*nPgH$MYlk4Dn^X2gfsGtKAb^-LA&t&rOxgse0H_ z%-+Gw9Ui6ZE9R3$SRI|+0~v1ZfARA5OOfal|Ae*DHa!o3JOD{B@A|qOx>Wz}WNY_a zaQ1%lH%*DBgX&7)^tX>NwiBdbJL=xW{m&nZGK(Kv7ZhC;Lam9{oJlVSNZJ3)?0XU) z6M8LJeA$?^O^$HPV3#XYqe-6;&QZi>dGYKT7VA%9lD~x+!OH~CbGZ-+4M-1%{Is_W zx!hL*$ZoA4IsLLIfR1ubaE{rG zeqEls78)+iF&ZWf3|hFVefTT#9e|)fw$-*Rt_AG!^G}OEK|2JTch? z-jWVRl!2F3I6U|;pNr5zXt{fPQx$$vHvWT%wQP9)W_EKFBtnAY5mJsUL6M zPX+wF!0eqe`a9EQ)i0V~k6+Ai^^_WG;2HJYlva1*h7ctq!+a^+mlYSZ&#`x_$k#5rBD(O3$ z3J*)O*Y146Iz0QKe<7cy|BtlZ7sl&>h)ZQ7)zhfqwE*2g?eT7LA0;1V%C)U^D;R_F zq{>eJQj(|N=fy2>QzYRb1?JedKR`=D;=At|9q7%zHp712NM9LHB*o$+kLKsr%%=H7 zV?n2r-Kt4G-&rG}xPrLc#4})huC&a2x>c@HT8Y_x^P}=^u~7mko3*?Yn~~jjI6VSW z%9Gb|(!=3E@X8oxl1wQo1?cwy1 zTu_y6_PY_76vx9&D_vN_n?+k~4-lK+FDtdO3@7gfI`3jNUzh6V2M7D4z7+PeQ!d%^B53e?+04mE~~QRP)#1u zHT%q<+aE06*;p)f(@f%3>^XVhBvx$|bpAl22?#ee)ir9=ji_A^viJ zOIj&@-0xMF*j-|$Z-wm0n}VtW6EDp*=B^v^i31}?jBpQMYA`Avu^`>0Gj#1(G9 zP$cM14C1Qv=j%hsSQ1_K?iVpb&fy7&1hvGNWknf$)*ED<)vL!3K0IRBHC?@ugfQX}OyoFd#rqI&oMn$%ALHz}Fb|TZ}2Ow?dKN6?tdLE4l^t zb&?^fr}AYRo@VmJ3()!X4Zq6)2b%!7v#Q0EBcVMgSd1!1v7=S4&MNB>9O+T34+JPP z!)jTo50qBvx|TLMDN1?cIYb-z2Aa<5UN3^{d-+u^w_ha%#%;#3sVMeh_7bVk2ch5X zpjf{Pw>)U08Po?zy;IozCUC z7UJZ2oN9d-AlH^QR#m>O>$%?&WQQgJ$&}NGy3N;+4Rj4^ zoW0*JG8TRj@aSiMSU<2wNf5ZF0#FW&?)7W1CI(g2<6P9fI9YC84Wy(C-7ZNpn|C8} z+>;WKr>|bU+2xhWrh9Q)Ym}36&1G#*bhzygUt#WyDKKDZb@4veY{()kva_GQ=WLu~ zOWMkEXIj_Gwca{+YdWTx>=;l&Hw+Rs37(DhgNrEdN-m0a9Yn_KMT^@sww3__y?F!) z44So(D)c0gXl|q=@)9W&vt{{VDQDkxRnc-Q>A+UG!$yBa*;X2V{cId@(u^843G< z#M8nSfz0Jh(}|ovqUALA5>3#cCwB4-{&d^tP5Ki`-2&3qH74%0Ma2XU13xo_K$_k* zy}`^+xLrxp;Jh#n{NOp|^1>2QZ>Ipc%vF?|jcZ^t<-FPHuD2`RL$sH@A*OMJWrp0O zn~1#&ywwoHyA)ENJYyDj&MV{W?6`x6ic=CFg^@V&^F%d$Z~*2tV{HPCDHFl(xu`FX zaV~Eky*dOa4F;+Hf(D{B&=g10j=)$gyRg#|$E92?G-hp7=^-FqZyJ8fqoZz zsoG_eF&A>1NF);?^6g<-kFS=ISaySV$s}eU@cjna*=32hfqR`PsheRaIBW8;xEQxA zxzxuhb%oV1(VJ5VA6cty%sD|#c{N+buMVxPe;(t|&9a<){6M$R2`q9XP^>dUs|Z&2 zanqV7#*o}YBDacU64ALipkKmzDmRA3v;LA4hi2m4$=XPjSyI$8&vQM~BdlH*!AJdpr{E#GfQ=%-$f}Zp&Sby*IeSJR!w?^36CM0(&E77L%y4WBvh2 zKgHxUy`(FKzeaylcV_MPQMri{ozyvCC8pV3U+UNmnHqk=8eW8+jny*E==|T(daxPu zT<%x16VNlO?N6}5xklME z&$NgQ|5<-in0qb0HOmQ@+{H|>HzKwZih|Z^3kY>d!s?D*N1)QrAp5=#`!?P{Fm;y( zC1GsK9}Ao(ARD1VHXCF>M?NbfYZ|US-kJ7FuepE>T5Ro_fdV$pcP|}$yvKpD1x0@- z0eML%*9qLCmoBVt;6lqxFPOc`_}LJ)k&&lBzZZ#SasC4q4DLYzqJM$*xq|#Za;!4d zx2E#LFpTMsVb>?QdPSN+@b}yOJPltDXgT}TG{5Ze#ZywheVa0ng;;3t-1#bu3|_x7 zm3UMooqLFx@?_Ba_?BwC&x3x_^7P$ed>s)L=c9yW7EJxL^!vAOO#u&SU&&#M@y_>P zyqfY-z@-?kv*(E*NKR2$GEu6Gz z(R6x|Uxr&ha}e3c&qMN{i|9Jam9CJBPmzHwyI-Enh>@oh-2xR^T+ z5aboOCmb3Pn3zq%y30b+h!p$;5-`=?V>MLgr}e!&J>aRb!4uZLpu=M0*^7DxuL&^Q zg~VYCLqD&Be7n#l+=BJW$nql6_Mh}aO+6+;l$5hY^$i-}L60w+Zm%v>4>-PcDOTcj1ttP&*10uKL*{nUWTas&`njgI3 ze5Pu3Quk)M+NiR+rxJvW9}`xzuV1 zA+fNAr;{icPiT4Cu&rlwob7{W!TPz1RkzB^yQ_vq<;c1W96Mnpp#P2T* z@prFql#uTZxE~Uim;bue03~Uj^A#FOd`VYnQyM3H2X;bPcVvm-;W)9>nz`{=FuudX z1wTK2oEq(^NEe%4aQWoBH)r1(AOEoCK5d@wlxd*0KcT;hb11h31;}poZWoHgc+_q{ z#!)cP7n4j{+&FD9c-s>An@9LxL{8rYxg4HOcRalogIGABXe!$;pbX;Ep3;Z)!2;Wn z;}6BKj~Vh54%Kfh>)cQM3?3bI%5;P?@i>?zfjSH5zLzV*fo}CO_{Kmmo-}#nz$FtB zh{P^zHM0Rc-R+j@KZ5Rc+l=$kjx}t#xNe`weoV7a?Az-UcJH3Ox+HK+p1YhiLLRjO zTUe3VxrF{&&ji&dc4tH2prg<$Nk?D)N1zu?71K3%grD!mQ!nc^gxwV;Up8JrIS(QX zoc5My?r0jJ#ds&wC#Ujcq$`ZI2@%H?p2R7JMrOPGJTd6z*Et5$OK;sjI}e7VZIeR9UWk?AW@Nb?5t4$GHzl4zr+@PI}?MwRnf_`pSdw*Z;`||Is4m`S}`R z;X3}J;LpvJxf-YYbYrrqHu_wx+&#k45izfeC@`Kw5ZZ~D$~z(`Rnf<7U*Y)WT>S(` zcdlZvEny<8$aMIExf1yFlIxybXc=zQ`i4MKCwhSj+6ETe5&++Fh*0 z8IG=OWJT2jpbZXzwNOoDS`y>?VnNPl)0X9yC zXCSIS?yrZ(fwkzF{Y_6Z_jy;PwUEO4E#HvSNX@PvV{X$+?O}Ws##l6vY^N15-IDS~ zhIOhF@~8Ft$-^p?#W&|CG^(7il;=7IApdE3TJADipkLFGfcfz16#UVttgj%P2RQTvOJnoR7;hCBDgs9U*{naKx2@k+s@bTUaE(DYh$8S7xNl zbX2rU4U+^D0vHAJ1uhssT@EpM7dAXRP3yw!o!?1rn;@uu5|Eu`@`LG7ma6Xk4jI(r z$~ILWmsy?fUEOIBsYNU+d&vb4+=D0x0o;Ko36_mwhbr z^4n9=S$J`0*iT(&xs0MaT@$h~M$4`mgWNdha4j%^(vf&5Cw_1JS#RdQ@yZ=RMKmQa z(ab@aFm({7n3AUHWe_{HH(~c|;$}>mROlk>riEYZUOR%@nBR0gf&L{jj$(gs(>Dbx zJ$`i4YP;(WF(l42>#p4hf5LjxU4C&R(~MSrl5}lx^LluUS;BeHB3~C=x5NsGcE%#Y zr#?biRo5x0V`7&PkST|bCr)hQd+VW6FKMT0rtMQ)E_CbfO@cD9xD8wc4_tW~SnsYY zJQ^JuEfD%B8hJ4^rw{!E%TUT@F@^&o8R%Xv<9`+P&Mr(!ab0LKvJKmKuS;(8%$$Rn zcd3%lPkF-R6dczQVI4WYH0b7Th2FW0q3g(f8g2Y?-G}~T#E@_%T|us*-Yu-fjb2Xu zqG0x^x_Vz8^m^EdC_LpefB&px~$&H066I#j?q{LCj4{wbaxhu5a`_I>qULAEGNTywE@ zx6U=Okn22_tO-P9E z9uos@fR~EGFJh53=VB}=X$M8A7v)?tRy4GoA01{77RBUjwUen>0Xc~Y6P0YWR8Eu4 zaRdHGz*erqJ2vi@jRFFRX>e5p6@YSjbk?@w3GZRpx;L%hH0EhWcL zq~F=IVH5?KKaczUHXOSrmnJ+ z3ahE;@2UF%HFKWpOIQa@aZDJZ%BdCj(qU}bpFY&n@gIfKecE74NmSP6lOJlMC$M(t z9aaFdd1BaSb5l$99DSl(7cyD`^>@PtD=4A}`D|3( z`6r_M;|36ck@ZQhCX-Y7uOh_t_bZvf!hy998Pnxw6E--`?4Q!f+ccdR)#ZEUda6G) zH@CW8kl$=egK#jt*V`PaiIHbQV_ zZNghifn2P`Kn<*yaT0yx$~7}u*V%}Ws&$Hm>cf~&y^`^WjBNU5GMFA^K+#^j-iH@% zisQmws+>lFgym9iVi?~9_ST#5j=IFpv`+r?6S>}JdyyeMz3}zzkn2ZFa4k(^eFBVxTL3)Fc}XV8e5X0CD!!~xdDGB~ ztu0a`=xm{MzktRz(I}}C6uv0Ik4IAaR<$4DnX|YLK`iKIc?LWhU#t7N z!m0NT)6uPY^aS?&LNnHl9{=t?CF?dAhos;})40Fx?pK(=VIXj$Ve~1My&4I4lljb? zG|{Clqv}ypn;)=E^@%Ki7*t{t<20k#jM@Y>j0nnvuwqRq@*dAU|>|(m^Z7Bth?_aZ_L94TL|V##eJq8W(8GO>H*pwK)QG? z$Xs$yYU6B%xU;aR9DJ8Sr8xa{#FchyBBLdqs@but7aWc0CBeDe z!#*Kr>(D)pT!N{SNA|Ob#Xi_4*ah@aPV;D!Id5svV!$Swjp53|63{hoQS7{M-xtKt zbunfruI8iR4pFNQ(iz6q5sZO!o`L?;FHyz1*ZF$T`p855p#H$lv-v?UWIMt5JS4^w zT2a5auAeYsh)pxwq;)zQr!AP-Wl-u&rbqXlF?FGKY@ie{3``mr3$2HG!D?A^u^}^e z7yh|LCkP%5j@OI#i?}o}(hf5=tnxC~?jGH$ij10SE|qHerwOpwCf)9XyKM924ZQ7F zfXpJnIj>JX&b*;i#jXd~==(~?y})`T%?X&2dUz9{3cyx+5P~1~|DX20JRIuweY=#j zQCU)i!en2HvJ5G*%uMzzWRP_#B#fP;l0CzaJXM%J>EaxZIq&N}?`t`)YwqJP#&Pxy*+mv;S<=fV7Q>?9%u_?- zdLQRTj`^GxL0pJ+P~_RW?h8IT^|5lkTOF@xaO%e$xS8g_OLih(68W0Au}cVlXa2}9 z6jYGk<{ybb;xiIY z47hrRMDXfAw!S5{Fsn^_d?33dFbUzw_^gg!`Mk0?Z6v~7m#`ip05s~&?^+&KW}4wz zSKP955QD&ba*6{~$Hx-3tvjY^^RpVKNK5CyUiI1R=>#82#5Ie*lm_?vuIAVjyW6E-p#Y|1kaq2AGAjE z(O6qDj~4v)=8gAPi#ueT#@w6Sw_)T%uvDW^=4P4+vfZHVyF29+J;_JPNJFNVx^WkV zRHI`Ne2*yt3Zt#)2FQDQ2pVtES}{{4HzWj%HAQHEyFu$9D9zc?)tE}FuPsY1dnK#0 zd`3m2-(LHU;mkemsee*y?LU0{6oat|A4_bZR~MYh_v>K1{bv?k{Bq~8)MT$w`19L#Z9M!Ecr zuLB_T8t52{GS3eIgtYLHLAN=t0Ue9X)fD0G<9Kv&E&U&h)!culR;+6o0&4!-E^j8$ zBg;VEfK3+q8(A-&s=9n{xdMKl<^&T1><_iy)ISrZlJq7R+zw07& zJt1T~PkHH$=n^wo0*84a7wZr^aaEOR79s?RLxb=)43gawvGNxCk0Qk4%q==5Y|BN}(9hCUi%#U}nZ>L~qV8 zXHfsdZ!srb-}K_~9{J>d(xZL*gtkC8Zy5cv$^HNFG+2`LRYioS zq36!QH>R-|P^p$wpOq3obsq!ck$jX#gqAMHE!~o@=X=Bk^rm%PJDAM(|F0K6V{kCR z$aPo1QXb{2?s#5$up!!X}#-)r}q8L{(VCli@rK(jFsQDT=gEW zpS{8(srtBjDG)5}qr#?Iqx+h3+Fq(Gb4%dHvGYA=8B$%-#&1ddy2f`u2F(|bO_>@5 z3JheTpk`N0-7DX)SC;L%h&W3~$N}uW@4z0p95*o!K^_s)WFke1`7%5ELQf~DXjj@i zj6PrR{JP9R`LX12aPBY5hCFU8y3x~;`Q_tkHcQeDQ>Z!V?v9z!mL|vx)DZP7adZ@V zmOhG;E+~~m%*V~g!V{k(R@+B}WgMhM)FR5#_y`6pBi{5W6o#&xy9w(7%G^h@Gs&r{3lPSz!H6^E6@ zmIbr#UYKMX#@^k-+!~Af8ZVj9755cU5m)h22iY+V!Rr-WcDEH?snp`LHh9VkQ`G)1 zc)cUJSh>UbnGWJYqvk~lyCb}FEwrnHnlIOWWfSWctnykr%VLt->2>@SC{|A@-?Vn* zj2HI=y>+c*+PGPA5&&zC?}!k7@PsJK^yVYm!C)6w;pO3Amx`F0-Xk7cTwLbO+eZWk z*Z6~jF#5V{cRx6>{dT1FRQ5M0?w_?}C|z;sHL#doXO%kF`zAMc`2fiP!G@|tTLSiE zufird7B7)K$xb*d?xz#sl+ZXLx|`&HdF}BV|GqXo{7q>d8LQ>mYVFSJ2SAt}U{X%` zS}@WhCE7fP??wleY3)cg^N4Co`5>?UDV4qMP8S7zmeJCcGi3z)3c@~yx|?)5mx^lH zB3vF!ZCzNnQT{R@`$3TC%bi>3C5+e_!aLM!zqKIwaIlx6!7=1bvT}&HkZ^)bF1=5}+7)t#4oxB}kAW*<@nT;=uLyfC{Nj|JfKl-ByEsiS^sIl5M-*{DWd$LD zZ4sAL=I5X}D!J@MA+{VW#gOIF7oj3M{^Hb*;r076CW%VBCm^<{S+qD+Ws#t2uIg)4 zqsB%59)~G2C(SP-hiPgxc77>bBq?Yt|e=jS5{jk z33O*es8a%AI=JQEDOQ>?iJI2Flp&M8M;GoQx`i@Y6R@UfMLJ~{QbT-5_soN4lU39S zC-GMT>J}Z-yN_5&k>dQF>s6OY+Q~E-2{B2D0P|O40S)zKPIT7SwgpbAFH=wH;hTyrM*2VAFelZG<(b|1bZNVi#18N}akDJz`(F5pK z)a5TQlH2K+z#aejkE;b8{^uorHM{5H`+>f5vK1k3UJM=izLLle%9kZ*l4BsC1?~rl zo;KjrkhupQ{--*pk6(g_`9Ydhgwe?Lye1WncB}RIHvKwIq_Fp%aQA1x^0{+UG-#O8 z5-Qk2Y3~|%@s5!TV$|oTZi$tKO`<(2$Y@-B+WZHwdY25vQ4y7NKQH+YaQOuddB1q4 zCfk*~ZtLc<9oQ) zIb#Fi0ngA)9(ahX0eHCO=)-L!?gc~Zmby>FD8F+*ygB0dq^}dZQ_d<5{|-UmT`6}8 z%CE4O@YhtRf1rV0cZ?!D)OuEHTyl_iiAKO?bW@;Xd5c27-O!r}jW#pYu7UN2j~h$$ zhdIkvMbnaw5^2WqjXV2taa}p;K#J6U{{VkTGmUZx~ zh`qC;hw!lmRYPA5x3q6_kd|WlPsVhz0Jw;|^4$v%Vwe!~NVNVpX*w4?=t5kL)zE^1 zszcdRZQ(C;1%}}9wIv&YFjrh^o#K6WayNqDpgEjpAa;!MDVh6%s}^Z(cupxMhN{Ig zpwJbXtF)fU+0A+7e)yG6J4pX>nH>wK7F@94{E*u;jYM5C$1VM-1tGmLJ)yV&L+~7g zfC8`EIIfK28vw0@@vAxU4xu7Fu$oIc^9{4<=P3XQN!s=+)64HNwCV#lYlltlHO)Aw zz0y?geqZa6nW5d0QR3`8;d`l4apNViSjBMp%2-2sv4nVlp|9Gb?Q#6IOU`*aq7_k> zm4#@@>xQ_gjXOt36S2cyBM}CV3Vn30@z;s@L?;wukb_jO&ylw%WnxgfGXwNawh)hE z;_l$*cbT#V%bqqSWF6dt#Kg8iGI-8^7Kvj=k1DslGyTs7rZ;>*WdmY9nBgxPH27Z| za0_6Q{JoJ`7;G&8Sc&R2X1sjvt!%x%!u3b#vY|y!a!QyyWkJbqc|E})4+V@qk{?!- zRCRe8wtNcJ(CMGtLW6@u-u9_c7a}3Ek8}BkC z-0cHc{up=2r9N{vVkMH8LY1Y?+@L<O%2#Tcp_%4F?`dcB}etxzTSOf!WG9bicqHAk)2zLNy|WmkMK zz{BsS&Q&dch|NLK42C(LC~2tBXG~psI#67IVYcDrP-ddyt6jA_|FbP_&o@ccyDIZe z&kM2^2T)>Yc8kaqV!JXd>9gKv`2dK0%Nu+OX3}& z+EXh)A|U_d!lotsEUCCxvAcw>LKd|k6&n_WKfC01HU=C)Vqa!#22#`zrcY__RmD=& zV&UoXlq?C>=Wt?Y)XOgC?;qt)Jg1^XF3P-E`hYN7;O8htBR}b2lt@eBVCv2I!cPkG z72X_Kgc59gnM+jfje9Q$L?-9alA7U@)B2y?rKK$6esIrhJ=FhZ2<5Vri1ha#3Wq!A zolRIvSFWsk;z+mCDFp{z#$v;s?nbUT(niA;O+o{WpS5p#fU|1;0M9p+;Va@A7r#Np z?t>&@WdO5onC`J*v*7s%KB7^vKs=5%5IfhWWL|T9dGHlk0nrXM%=^G_Vu1Fx%v&c@ zZRrnKSroH=%cTYj29Jp~$_W&tzNr^ki88Cg)tGJVD)@?(8n#Y3D6oHs(o64s?`iZy zBk`4AZlKUs7e`UZv$F<_eU9#Exu(nD)wvGT7Ys5c-gZ%+NTvbUx`;T0G{^t-j#&D zex12NQ!?MV+N*tRB}4ER@Au@}w8o-g$T=lOUv#8%d$9J*SQNWr=}L_7TE?)XP-N7or;0qS#op^v`8J4V9gU+-KW$Gy#J z;K-MbY|UcqBdZf zkA2?Rg?e)wn{<2=s&x!P1@G3E#S^XEDg4S30np`Ih+hx-o5A#>%~fhOC)O2xDCXlY zh=rtFR{_TC5X*i5RL0ujqqe@}#>> z_dafo0ghd}3T}Xd>s=yy(fYvpJPJa@dFG~`RM~~0?u#VdlERu8Y8UfBWhAlxWB2Ck z$3sgbYHr&B*`Pv2Y%9$<_pS07z^yI_@X5~5E=Xj-uZLIvY@kzISq*_hY7flAPUCe8 z!P7SFauyj3#I|1X@0|7V#oSbIV-nu&MFfp{l*!%*PpM8`0OOkx4Hi%DC%}Ps$Eg)g z_7`^e)n)opB^UguwUwS!E?^JCEnGaBa$)%;U{+^0tCwMhUy5r1*N2_ z;A&>Bbphq>Wbi;`AiV4R75S_*F6DZyiJQq$*^?Nyai385Z9)RqVC}kB*4o^+J6gjl zbv^3~w=l1D!IG_ABdGq-LT}o*pQe)d{V_~TKv%#;x(03ci1Uv1T4bt6K^CP}#n`vd zn7_GBm?A>6DIy?O>p=SD78|t^<`1>TFd5x--T`852_An!L(hn*J^o=r%Kn@;ZC}9+ zCpmQbZ74Tv>0=nv7x9Fr*foYk$tq$!TnQZ#X)MV`jNrK#nv)qrdP zjU{eBEHwOb%=yLol}I->st{C{ZOe!Be75#d-XJ#XQ)v>?+J`KO>uN2jxE=p~%vgoFHL6cq;r=o+Rmqt;{Kpm==Yc;>bu0qLZgsRneXoE*cb-veq?CKSf%yt_p#sPYUUk z>*AM>-zjac@-beT!}~5ZI*mzR#T$ z9DRuY{H&`O_0B6DMXYhwX*a5dTVXFX_w6qGlpk=oMQ%4W&}exiRfP8xVN76Hh2F(D z=A31@v>Y+tkoPnP(hiz+v>U!L-V4zg03934+?M9OAM?1o+{Mb4FuT9{mn-WKAUM#A zS0ziI-opjHI5ybQ*)(n^M0VS|%S8 z(UX_{W~?g_#&5J>qM4A?i%lzZ} zs^?LXoAOh?sZA4IS7F9qHdOEAcP~;g6PW=g7qir_*gG>QSNmQ+e{{V=P8HrInAVyi z`k+eo71S(qdM%Za>EXk}oG@!G?&Z@swd8vTH_84>c{d3Y2U5)UzOo@JZZ#X!ZFY2F zw=OSx`K4aWv&=>_3j#T@FX!azjm#l4Pd(o;YjIQC*fP8$nz+ z4L`kWRCCi;e=~n2!Kk^3c9&_@l0)`FHh#~rsylnsaDt+jZd?+g(4KtH)Fhyw$<+i^ zjv>NZon3e~fSoER(&Q{@3o^iXCklQNqPA=DP;TvWSu62J?M{1Hx6izrI-A2 z*n2lsZkQ+41BiREXJHu|tsSrE7s%9Uf$->411DyC`I=z(|fYB!?H<%WBR+c0*M zuqNf`IKUpYtQ?I&B?7RY%H!5C_#p%j4*tNT{9SgWCl*>BvIoYU2qA^d9OiB)nP4V& zUpQ}1FnVW@C(2IjZ-t**Kxk1o`U0y0eEDnHLJPh2n!vY*x-LtZtiA>I8j^b)Ts(ae3HI?(7TcU5)e^Pum-wuaM-EQ&N$IO< z`KgB~2^MI)&Y&0H6CN0!t39Alx_5}QbZ-cB+15lq5<8k1S@yI}UI&W4N1CX`C&QxV zuiq9T(BF`}FK4Dt@ zLe0VE#%(e{@h?dK5B>qryAyOw-@s4HxQ+DG;mcCG zYL#7*Z`w+3di$QQ?VdMN6R`g|{d_V>Pni^_8Ej;dmk?M&{UL%6jI?Bfw?fzOzDY)> zRRJNp`31dQCN@}5MmWcedtj9ZSX|t>5(h{3Cvr{)re372Uf4Ze-YMcNO7B?@$k8{l zt$-R`rjPD9scZZ&r_j~Vxb2q_K8-yWdvtYhCIUXe{5$EBQGhY00*Rp;3HJ3jN+(-% zreUZ~fW-#>+QYuxu|<}#>S3vSddP9Dr1mINVm(9Kq}l#2wUu_JUNO#g{;Hbt}i zN@Z{MWFB|oxba8(j)ATZ*~dMDR788D18)&lRtxx&@Qy!EWl!ZFDz-Y);|*dyp3(N9 zT>yIXkGl^Prj6^e9lHq?=!ez&lp`&Z&5U@Zy;}1xT_!;lM6)y(W_Z&Y=AqrFK3HIIPi}e9aNHdUmS_6K!cPjT(oK9HVC=S~(2!d#nwT;H zM433WMp6V2RLhGr8(l+uVa3ir0BAkev zthOw5@S2pfXjI}gfgMfoaJ2s8q+}Wg?%>&0&x&!8{_*=A!}vo8Y-Kn78#3s}G7Vmy zvhKq1@Ufldgfadg*^O*3;u7b%7)hkfDP5EM`Z8~jxsBNU%n*G5-3bVa9uXWB?Gz{w zEV$s4-pOAbn88%gLH$hq{*d15RI*L8=%Zh#pEu*kT}F-FXe&rBBnLtn28dcKlLc{; zF7>-CmoxsW9)5j%qSpRiL(OC9#Kl2+7{Mz%BWlbTg6S7R6HQ@T&S)szV_BBRM@K4 zS$#doHxZ`Bt@5qyv*lDY z+pW2iXQj}c3w|8CVP{_VmbvB_>zBs0NAq^i+R+y1Be=wYvF+wLEwF1><|+-$w?_GZ zb}KirFE6{@5UuK*_W|eLnwS9O~Lm;oCvWBGl`6|0M=fb_uv&~1_R>D^v zuM`ohxd!c>t@M(2(D05EX9CX;4P}_b5Zq$RqK*0z^qQpi>hH)J@>}vy@-9Qe9BwJB zI7SUANovagQ0Xckj5^Zs{RZEV=R!#Al~^OqI%DtS&E2G$LRYTSzh(ZW4)-lJ*WZak z6{N9y_L#fhsuT)fwo zf>P`tZC<9&3{k@W&>@7_7ZQA?`@`THui{Kohl>Ma;_t-fy|IRl+AaISo7TQ)y5%_g|#}Ww=SS z(Y=|JO^}8@dN+rff8+Vil7OKG#clUYb2A6uHk#Vp|D>`Wsw_NuQCV~-GqPXz_2{g{ zjzreY-z?|95oyl_hTc40;J&@S%+0rNxPlepol`wZWmAk=YG($cneKR~>PIzQwq%XZuJc;?cTujC5RHT?uhxO=d6*zbvlV=)o+ z3!nTsdGwEBxhvaklTN=|%lAz3E01{h7maxVU7tA&`)oh=pl5LHL^P~90aUeD2f_hD z96z0}Usu?7&^NJ>slUt-fYq?f@=5`<^D}V5?vscN94{WQO(|EXLQ9Bji+Q#xkC#*~ zi*JErxACC=UBCTf$zT??D$CVTsinX#EFsw2qZis7NlRY=)Ga;|Ekj5rVY_i3CZjP^ z6mLsi-HdqikHFGTr$>6@a2_5j(|Cv}c&e|%l-}K@!I)Bu!jGOGDyAtM!6Y`yiq8Ad zWyYR(>3^EZ`sW_GF{I?R$e}7xGAM178LBF(=lv!gM-PZ~5iE!z-^1HR&8ebM3B0j$ zuPQ?VZT?S7U17*JqLL9jpWt`QrNHapZp~Jf(l{C?{t2rIUvP?%(YM`~Yca77Z?WFs zzwOmx@QDq_=rcdje+#z0V7~$;Hlx%54TKFh@_Q{Wr>)ed{kqmr4uyghqaAfa{-2-z zeXo8^1^-$o{9pC-fB)=XNPc0Oe}Db=wEUG#`75jX&tkfNpZ?{&|4XF*66yaFd%o6S b*gNrbXaCy-Z11Ig^dFGA&dowq>nHyMdZZ36 diff --git a/docs/img/structured-streaming.pptx b/docs/img/structured-streaming.pptx index 537e4e32e8aba392c5bf888c52c5b8f1c8030547..c278323554da8ca3cfca0e3aa7eef3f6225997a6 100644 GIT binary patch delta 12129 zcmZ9y1ymeM)Gf>aK?4jBT!II8cL@;OU4y$z(8dA;_rWE&y99>>*WeC826wmcxc9#G zz3=~ft=)6>IaM{)wN{@#wY&3MCGy)OYVt5q5maVKXE0!3SQtQwEbKS{C-WD3yCVZV z$1UDEf1`~j#GZ)(`*WIz9t+&s#4GN}`sq(~3zm`z_F3`N?`0LII8t9a+lNvdYqP!- zVHaP34di7h?lnq5`;7ZkA`iyO7Y~b=j9p#qS4ddjn4sMi}b_5b83Zi{B7aLA}|{`vg&I zlo%JoqPSVrZ{|qKQb~gEx85xG>3(}Y|4ZB7_37QI;-$#rp)bzdq%U=Oc7mNt%B}M& zx$~{14#|Ek*HHKh>j?#J!uhqYNjv#?lU|41-1g3^(b2uI&V){vEl7_+Dorch0WQN& z-NL-M!duYL%i>sIyr`~BqHBd(J?R%hA>T@y4i4FsY3ZSt(s%+zmp7wf zw70PRafhWI1nuewh=NYH?pjAoCl%$I=GoNqI@uX;Xq9epa!MMST9Kw-X>77T1#(TsMwr?qEhvzv)~$Bd0INgQDSA&X68ldS7hw@xH_gM>cX> zk1C(&T8`8jkW9s01=g2(Gzjz04o;`t@cU4tns?;;OFfBjl5Z7K*0iD(-t0J2l{Xk7 zxw?JHYT<8)Ia#~U*LzB@vhf}??hn6Eb3Sb77Co(^hh5qWTz2ME5DNME1I1})zCrYm zfB7BgMgpoQnR`RG!w?aWAE#UyJjb<=j$UFzu+NMSBuVC-smr1YQT}K$(R17FogRFL zPB70CF*9OP(yxB3SXA~$gRKrpKf#@^H78g_?nK;Ux$g>%CEvzwBrV78(TtA)tDZ+O zf4l6{XiK&y%C*$-Uh$AL35ybHz3K3*N(#&HEV~-*C{K#PEJDZJhJ)4>s!;XAxc;(F zR>B7g0Eri^G64fwBB_GbixeyHcrrm09OSw8<=S{zx}orCuHZ%GK#a@J7QxY*t$g;Z z&2~ZD#}-yiwQ3J^y0(`x`lQ6JL)79uTR^&PtNf5bV#vXy~Z)Pr~JAiS_8qbM2ssk&JlfI(rpFV9J0dETf#^;@~HFHM6Dg@~;+ z@+2AQ9M@U5>K&88oh`wOu9sw~RZuj z0`fZzqi;ndJ1Kw6ZDa86f+RG<$JDs=8re4a!+oM2$n4kUt{DD_2-yggAo_ zcju!70d7phdPLr&D<0bLt@-rr%3=bWIsuR>xT%iPyt)M{uXqjH09szL;sT` z>r8Rhgox3-VoC7Zw$@h(EUa06cc*FL7Byp3coS&uIx(-d?j8sR17u7*%Wx);$c%>r z@HAQe;W|HnD-+yD!%v!C7)D3 zg(&&`Ofi}u`5~Gb>QLuF>3Q|$hV40SCB81;oBIhqSgJOV*lEC2sepWP(D>bxXgqlmj_*o=%0CY!T8c2m5ObDpTS8D4P(6@1;oi) z^jan8Zql;$fxNSdUu~z_LUlAo_mJ=Wr$`@bRrTE;MVl~AmJ!GJBU=|qW?X7rBvJRA zj6h@WkEYOcGAeP-TO9GCN;(|fcPP#?$e!-2Z|ecN%8k~aYg=iaZY4&FAIMgDthq*P zC*nG6WQHs8pZffKV1)?L4%NL!fef?03dN3R(@4cPs*R&k`RbY@3P8N3fu3vw(!Ef& zreu%_b5!5&O=!3>{l|O_9bTh#MBVF!1i<93YlLRs!N=M(UnHcI(?|pwlFpnMNr}wW z8&p4w%p+FH(yb1gC#+r4^=^6*%o z{?s;LnJt zS8GwaAZIw|WjcqNJ@A(qa=`;i`s6M*t{iRWNoG~&9N#tUP&RZx;+Oqe^n5iG6{+~+ z(SEL^cN*LyFR5-`eRZ28i9LqBD3}cnhEz9ICm`)jstybWQ%h901ggy&!3HviuyCRR zq_DI%XOlr;xL@9#(Qc?X=12$jz-7w~&ckO5)-)@vul>b7R$e~66rP7+z?wzAd2GMT zzD#iYg15T<*8*1-$mmOR(5Z(=8AWzJ&H6g%0ie5H^#}|1Qo{WBeSb~K0h9$Dwg+y8 z&DQgr?Y`p~!3IN2lKJuGnZHrCZNCi5i)YvRCVgGK<@jz7pXH;URulK*9v6=tFIR16 zqwWLszS-+@dqjJ?6?t#Gw#XZW$w-l3y=#3V1h^2!A{7V2&nU-3uGG? zTW|DXN#5{3Q0~*Z65H`X*lRG6rh-cTIgv4ed5kH^?7#(Y#B{r*0-|AvKi3r16n{tAa2;sc+c!8*hhvTT6vRz*b8gxVdUKz|M`pAA=)qCht9w1xBf%4uzqKO z2`a|FN`U*i2^V%08hD+TZBDBE0Ih_-tz)a)JD5%Ju^xJQB=g>{#F1B;@YYy%6HS!_jg<<+nvgx42m zN6F2vc|r|pAhUrVrTm|(0lKBgKA|S4yj}`7tR1{BC94i_&2a^ zUyHe|#~lROnhuB#aD=C^{TzsVCndVl%>&zu@K1;T?*OcEuYpYjJNN~pJE=AO?BH?NgKA+%q?=44q{im>gqS_5|Sd1xakq7PNQ7pT^_lc5l063ucrSLCc{emS>; zeaWDcxehlETOa>@OnOg}tfVC(=;Bolnz?gY6+s?Itk zTRir1v_iJ6P*l#e%jP}_S4>slWy<2RBu0vdRh}CTtE>pjv204T?5VOBK~LelC`mE>- zRodUnh^q4m4Y!@K$+ux;mKddi)t^r)!vC~iP%+4U=o`T+$$@${?bsjMmvcQ44xaw< zuOWJ*@lhh0f}Rzo(g6OSU_+RS*m_b!M^e$Bl<#^?be%+u5iSF)yRWiSX{;{hdU*7i97zTVe{eVny>{6LAM{+wQXtTzM}skI0Rs!SN)KV+ zi8Cf)Cfh;}vyFhur~9~CN};pmlHaDtc1Cw15kpo$gY-4sCoyq%;O(sS@~w)dxAE|B zQ8q$w*X(r`PUDFcP!TgRds6EeeS3dJ;7Wyml^;oDV+tXg%S7OMYxBiuTKov15w!YS zbS#Yp-x+#A155`!XtNeq@R0|>7J;T8({SB_t;LM2b?=bk1BhDa8BuJyT<7XSbii2A zmZthvf16E*H@%j>b8XmG_eqx)4~*I+Ue6eg+L(K`yq0GJsma`(2cBG*ZQ99QfQi)f zu$rs;SVZeZ9Yu<4T?D^Ed*0MWtWJDV>?{K05_97m7{~VFUSvqlOWW&UD#!C|VQVto z&uV4kUKp4tS^Z(YqS}V4c1LY9KgrvN(zsL@vHT4@!(3vZ+*aK5v0bH24>n))Sh(2Z z{t2~Rmf>dPLoDIMuKvFFgGn^TDcN--zBIhRH4)NwzJ`cFzz^_BEoV;H=PftCh3Jj{bl8ziXwq%`S6_t)U#^u~_&KQyq83G|LXE(>wB5mTnbUj5A$Ghvu+E}w4K$dd9BTC*&=jobGT z0+js0NkPZTSkfV?qVeLfSzN0~)FCaAq%fQv1e|XoTY8=+3-Ys@x{+IOCu9IqQnu?{2!~LS2!=UPLU#HsgfvQYc;eX2Gf?}umK5NG^+L+7~ z9hKXf*r$JYzIx~bevDwT&ac5 zf47PkCDZz1DzYE|-XC*IRgL;c!|EuSAx_41oO&uqS2PM68iN;Ie}{I6Qr)ce+@(I` zy6AJdr=4G|(z6MtZ2op@ICPq-TAq?(s8{x;J+q*`&cVJ($(AENsgA(Fr&La4Bwo@} zry^$O$dKwA|H%9G$i>y8#7U8thdUWwYsURq=L;wLp{adf@x2|JUz*54;+pKWvmc?M zVC*k-lAVRj-H{lozn0fs#V6Dg$=GQbzc(E?@9iZPtz7E!+^YRWtFM*0=%;^N|6$dQ z-Q^}xqTiJb-yHf}K(N}M$F#J{OB+j>!)oM-=fzwc!i*7znWGFzi}6`0?DRLDempaw z89wk+l_zlrToy6QTw5JUu5It`YAa@jc$+#fwqAzQ2oyMwM+aD4@e2o4%MAMU=(pH% z7I&9$eIh@Umzl>Gp0kdz`{m^(zh(b3I9%6An_xxJDzfPNAcvXph^1BKSJz%yq_~cL zu-u_32Q7P!WAx0P(J#AfSN2p$W}GkJfrktnxTd567tKST#<%Omp?Kap%@lIws+5aP zYQL8aey7a4@)A<3xDTpj5Izh^c9sY#T#DP-ko|?5RpQON2>l^T7inMV~X@|B`JxyvjMD@11P`FUKv zDhdAwpyroNozI%U+R6Z>kj-pkWyw=#9smN-) z`nRrrvIMXxUipl{`o;vN<017jglJ#4V}K=f-4z}4{9KFt$1}0IxOMi-6o#*IugUJa zi0lWJ0)oN@`JX04HU#ME(nFHE-{vjH^*=qlxGS_XOmv-iKF;I2%ZT8+!TEk|mGE%o z!1oR@#(1t|6#d9tB>9xrUEVc)DMN!Zkd=$WIG+H}KD{R&u4?6Va%?r* zLv%3=ZSd*S_rPfZonYLrrw=U6UIA#cEq=p3@% zKN{3yjeoQWS#U1|l{i{`uaF&j9t#`8s(Dgd?mCev!&u9%b3(*iD^a%#E{xbtEzY~6 zbqWftu#9($Wfw(R#`19e^=n}uDa;udM(T4TY?lPm88}5Ig?=4stqlD97Rlzc z^}9udH&VvTwCdO|!jc8&_L=j;GF@pw8#P^HZ zF@wc}J4lOXGg`|r!qTM371bX3$9pal*MmAsF(K^3^Y+rAbmyGgYaz!iLcVMZgq;iI zW9W&dVKOxaHgJ5|qyCRcd_+LsW{9Y-dLta!TdC#SDA$1B2vW-!aK`6Jpr^H>o;k)C zPJ|4dKu=QK4EA*1{9laTk2aCf2vyPsCJ<#4rF2K5@Efnx`_KjOaj?~Tv(wfCS%Mhl zT_9m#J?S|d(qGz27cg;@X_IH&HU$maTwk7F}<-?I~9; zQH-;?o_Bd)UVCs!Y>mqy!<@IO0z&X(z$*aWVa*9OEYElbm3QBztLh? zd;5{oM?t!lvYW3~MuH722gjrue*7iBOyHtTCVzLnO&3Zhq53{0{c}@g{YWp~GN)|H z0jcEKeykef(x_0qW}6k5xa$?*S7s^3-Z)t<$h1t(kM3~!HiAavYq^vSBl%M$87Yi^ znlnf1Tm1`Rv09)y_0%~p=FY{W5tYWg2Pcyz>3gLz3dIBJPNrqz&*Hq*h4g6Y_Mv1N zv19iiLSG);(vRjK+~^OC>V+0W8Rj;1cKXEMO5?X-o4(V_D1KlD5H*lfAB>Cs#!WyG zE3|}@Kc5zNX1+?A^zk9n9evzW|COxo|=#4$5Ze%G@S@Rss0;Kohf9nU>IrZcOYlVj(B6}ubL2$z5<=~9Gc8;`&|4WZI6^_I zvkx(-%DbOCs2O2`+t+K>uolp6Dna!<-XoWKj>u5j)O?XC;=(xuz43y#b4ZL+K^l}b zpHOkFL2ZFtu%if(4aV`f+LXr80bH=Pm`-!zcJS}0!lPmSiOwPYl3M%1j51*`BnjkU z!(|SH6~Za<_I3^toL>*o`r@`!Jt+Sp6NK+BvHZ!DW=FwL`QCL{pBWY85;HHti}orA zAk)}=57K~ucSE{SfANuzBOf6gy$S{0^)SxkiHcIS@}{qkPFg`LL{><|vGLr`g)3GB zzc(p!$SOy7O$*B((6wzpq+e`?0CzVfGHe0n>+(xc*g>oym|%BMDUY!``7b}R6Sz{I z?uvk&!kRU-6iDxdsxgnYA&MB;>*+N>{KoYicUw;ar#2;=+?1#otOuM2oGp9>rgj3t zYnTx9j~yNz;0<5<`UBI}gFsLtMaKQQ8Z}bggkEV}LHefAUx1Or>2AWng+a|Ol#;+) zgrM0sY-C^tGEK6%H*?g#T1CHfj=`t=uIH3f*c=SxH;~iWpQU3srN^Wvb*~Dc&CV2y z-<46v={K~#rKxS*q;&WnJ?(9l z-^RVe4#k`&AOkRqm|f3Z5I%_$pu7W{1;W^R84qgj0_xLu!gNJ3{PCAohzn7pg3P|0 z&^j#PrXWy8I`#JbNl|4RHp8%G!Y2X1RegGxPJ=|AjFC^{N& zABgl;1``Rn)7wM8UzDsBY5t|o_qJ*BSRPGhs+5i1z63HNJkE^DtVRw?38JaYF-liv zCY6{kBzLUS@lqloL?zi7A2Kd1t}JFy{JFM0O>&O=JJ%T2LK`}L>56KJj6(&3tucH^ zkyF^lRsz~qjME>P^9ld2deHx>9z9A>P?rn2#h`lzW0 z^1x~2{;JE8nUFAP5nO+ow-7#`0-tUVr!#^0XnhcWoJofwy^FlEE2L8p1EKpu?`Ohk zXkqBsZ{vM6h??DOn8Npz7SiGh+{RRF<@Yet_VN(i$wMoe=(vt#C5%hw6kaoO4IUFu zY^P->_rl*6OuRv~9}r@g#sbK&JGbfHmUFqDLAK#^HlN4-fL?wEe*fsvo=yLT=WYkjL_=9yjLQ?K7~s zNWL_`)UH50t9{HRxYrT%`tXw;_4&&F@kxU#+)H@3b2eCC1j&ZyG;)pMkECIVHOK1m zc?8`H4~FFTc@i{~j_lp;+c6KRn|Dcy$jFP$a1}j-0nZNmxok=gJ zl59CcTeDvn6KaqAW0Qk0HUC#HwrM`HSLCc27d=2S2@^#FCf`j4Jh;YkT57a<74hX9?5LmeZ zCOe2pMlkp6D#mZ$idFMUXlBuvCx_^8Ee<%kgmcJyWuXUQEu&NWf23ujn3Xti48s!d zj8=e2H$+QMQraWqEKNPYeMAw^RJ<$*NlaX%H!2NfQtf4=n4qYWh_W%0D4kiz8l zCP&6cZa7}Lu$&6GPxTc6fFdsLp8Wf0H<@Dw{>gH$qJk-?FBdYyE*&FA;i8U;P3HDD zq~vNXEIA#9ZDp6nv6Om>)t~JTgt}RDwxQw$v1lukVv!>~=YAz3vVFT+v*_*G;*VGz z!4w51#k%{yNY5L2eDXY;PT1}(XVD{&z6+>NbUd$0o!SU=-$qns04*;mw9S95vHHLL zsGJ!e^WKG?J(4;!vRl9xUOeG-Cfiu6=M~!CECp zCgwZ78BIeSS_G~iGrfXy*N-XLj}mN8JgY;aJi^{#v+p5l!TDY`6FF-sU7it!Grln zfF#As@0WXjW+ zH^|Pl`$F)(1z1MvS9!gGxbS_W?h00!UxaU~64AL-kCF%OQ~%zZ#7hl%k*-!?tIP%u zQf=WV8&*wyZ^*}qC8&&(LH@b7Mnu(tIR3qyQFEZ~jxq6L*0*Jhx)yq2ndiH%=A44W zezNL#$~(ee?CGfhVo_S>yXFV8x>-MKrTd|0@v2$OFRmv_(qvDF*TDA^>f2=OS)TkG zhlYpBV*7^HJ!_#_YQ zybN20NPC%){Un-VeJ9L`8ZHyXQ)a`cj06t0-KcQv(;MOe?_o!glw^LL{?IR^FBo&? zwc{`;&4RAJ2-q`;9?KgJM3QOAX#~zw;}trU1RSdNbh z5GdYl-)yYNCO+1(5#ov{KO7}C{$2eeoaGV_D5q#mM%ZLIe-yEFi-_CJiAHv1>31V# z)lU1tBYbJ_67?fWe)o?NiVxB16&hDd+D@?iR(cpG!jk~2j;QDZe}&)DvA3CkF?ANs zASV3257)J>=CXIQbk_}SqS{|juP8IEc;;{!_7KCH!x%%!N%yfGX(TTzQL@diks~KS zpJgsHS!Xp%(M|7}%Jf6oyDl95t}e6Ly#Doocn~GPB86bqj>eCe!L|@3ZBlR}aaFsO z=3cEZ3{2npe`kr|@pCSCKG<^V|F(-IRa5Q7WbudYAJ3YEN03}0?!qinabK z?5R&Zrlf*UiR;viVJ1`I<+@4*8#IQ7Y^8V}Y;A>)HXwO&TmS-Ra-PMhy zO5XmreAdarT~UI)2c=`;LfVcvl}fT4Vk3^df>v%r5USq1ExofHTZG?t_SvHGqT1)I z#kM}#tmRd2YH28r+ij2T51X&JI=W8-OVp<78eXC0xS|^B0U(S8P4YG+4*NTg4iAx=U6$i3sQGAJV=tm=9|7d_y7+J5}@xvt$k#uKoMnIRm0 zs&f0HLbdb*f5zzCXeX8f2CXtZ9buuBI_az+Kgl8V|IUHn; znX_<%ZSU;?Xix1!;1=L?hlV+R$yp&yE1hTi=VV?v zoVn2ZW>*>iTq1c{-8ust7C5)6`BNJotkmH(OW4oauWk-|HiLF9Z~X=ITFgiVGiZh}Xo6k=`wF;Mz`x?nD-d3R_zI*~Aio0T6{xRx`wFyI zpuYm6nL!gw?uuN!tA$7h3j;&ktT6;;0s)zmxEg~kWhLOSI!THK%lS00Er~##LDSLv zDHR=gLy3*2lF#PY=$82f4ad<#;fxvTID;*X3VS#utkjZx+8C8?aeaMx_#K%%O#>I} zH;;{c+rQ65di;8Us3?MB$Xg#&FvYowW=3FT{~&5}YY$a2NOn)!myEem|Ct-!#RBAW zfAW&5d_gv59~>`JWVOK=SUzk1o|fzm{3T!2C>hYP37#ya5BzX`kCPvkDx?eX@{iBt zSQ7V8)6kW{oW$x>p3-r=6vY*%$F5)*ocasf+z2vscmlBe!k9vQSSjlay?I(>5 z?lI2eB>Jr&cE&l&0|(z<3u@m_c8#h}#2ctfLwF5DkDtnd5MEA?dBlJ%%1zU- zW$Be&-b<(6Gpc=oHO}7;t2Z1tx_r+kNojsYWMB`F3@?1wcuopT^>fn<_#UKj%q!4nJKT(l<6!bo(>KCfaErArg@O zaIkNK&)B;C<2IIq+iOR*?7BvPcQAc3N$QiWkG}bZ*G_RkSZlrH83zjdTR*><2viNT zXs#Lse*ggvL9FjWPa#JnpH6GvVDA!kn{gP9{W4?J^O#Sf9`W6%^6xRXYNG z86Vg;N?5F_3S!bOj#8B2G@*iFQpj^lOKA4RRFv6J-V;}sl90XG7-Vegk~8~RSX)z9 z1d2q_-%wa`UMw&RUT~7lS`lSh8N^-+vo*C1{A&fsEg{`g%jU9~V(Y?1i09^JvDzU} z^3kL6(PlX6T?dJ)mgFKt$4KReXY0W0wPnEhxroi7+@h^~RQe-&nn~E*WG{W(BK`aq z^I5yI{YDEd`J^>Lw5}kI7)LX%QXbXAJgMKze#{{?# z7WscZm4^KJ4OR!GL%65F;vhVT*A$qR_}M zs~E`3>u9hxnu%-%REe*N0&*}32183-ZuZrV(f`LzRMJO*M9+fhVUrAAIrF~*C(;>$-~PW@rW+>m82%Ue z=P%zv4CcV-un$K6_HQ#ng=EgX_6L3bckH7%Fc&P638ZHpObQX32eZJwwfK+3y%JCp zqUTzK^{c>m`cef7a=0{;=B#aB`n^dGVLhpc`5kNj8!v%(&K z`;Q;JBei~s}k`cnLV9UYLi*9OG@-gO`t an139sdsk4LOPc delta 11982 zcmZX)1ymft)-{T2fPvsH!QI_8IKf?lJHd6Z;1(o<2N>Ml-6ex-uq3#<2Y8d4?|tvS z|DUyXP49E+^mJEs*Xq5e!5z}zcIn!DY;?Cv-K;q*7#J32*c1*fJRlF7kCWrEvA+8j zA5PF_`7;)aXpP+9`+huntFhZ-Efqe^=9b$GI>u;{nEBAullM%ofo>WFY)Y35M24pX z7XVDS`}P%zcG#Emxw&_D|@TuoS$`?{}w;T<4i&X9yotWkn@jiZH z7oANkx4Ll3-T%njm10*u6%(2G$xry18nXH7cTe5LqVc?N5d*X>XfK*`2ew-7*86bK zccmSJcX@>TiN~YG%A0-F_Q)0=j|_?>0+iva|0Y{%=UTQ?xeXH$gnxu2ZAG~w5!zQqWvEk9%_g)e!bJ}4T$=b$Q@y% z)-?CCh$kI+ZQ)4qnALt!zk+Kj`G{o-@;O3BXtbz=t77)IUVe1u-k43ww~4_G-PTKE z$YC?Nkh`#>^PK_hN`)7#?=R8Ay7@iXva=s7zUJ*4Uadf(MboSK;-2@B_V2#&aDDR~ zFUCzXC?PHn-6Og;8lt{>$E?^^7QC%HKkqoVy11&NHg#AM zlo7D-aJrx&slWJ;_dT9PMplF^%`e*FKrX-B#Z%6mlUJNcJ~c5$r=U@xTZXG$<8}Ls zPPpC7^Ia8?%&3SHt>#5YB~qi(%)@ZfGTX03L&Mk9@*r?=SxcQ7SzL3wfH8|yd*al95)^f%WB<nJDKG0LzL>-1&$-Hn`Enj6n$^BGP;=UB@hDcu48|S# zmZ-bTCj?~%%wJqKa?sNmlUY4b8?$nM?hp9*L)&TE0prw5wrBdP{7iXeqlM9zS^&$| zHZ4E_NLXMgJBCb!y^P8FD-_TvJyLwQ#E>^!PVl&PB?MW4? zvSM2Sa;HDokn1YTvYyZNru@C==v=8S&#i+l-t4XPGflsZ*qryV@YLFR0USU7vj^@L zck@j9DB;bG#g|_}y@c|VTrOIcNm$e$50rVxfHoIRTC+3iMMF8-oMXg0Eaxyv43@04QLU0qksZrM=}XM+Ux zBevl@YaSLBeiCacay&|NOUO*oSzVU>YB}_a*Ho?7`>HKiDL%L}_+@URF+h2z@%tNb zSztF!+c3FA9F7XmaWAvPKH;k)PvYtRZ~G1%(^ez12v$@XlrQR)wpihuST`j46zf9{`xZ?P@@ z6evl(mMhhSsbhTSJu-3qW*Q!(@VN9Fiw02Ke`dR~h+l_}>LPFm27)oiSYiJO_)@EnsMA9Pcl@pJIrvU1LRjcmeEATc z-l1NTFV5H$TI9QT<(`MXiv!(CdWR7$b_+I*1oRB{VPUXf%}5uR&xH%&Pr*zOZqXDm z0t_Hlv&}Ew3rGzrTyOZ&;2m7vY-}FM>zj33h@HD5XY_5;l_V?o&_FuPC8!MSBR`M* z5I5cHlKVo^_K^}$oS#*>K1voC5~#dM>s=VfT{lT4XpT!V3hOO$D?-!SA92wl?59Y>-O|0|bo2ssXIecR34XxUX1zyXIU3N@ubZneL z-?vhwKUO~`1J>JU+TVUzi1H3QZyWS!XKCpDrM@D`0=+7V9A?f1!IoBmSl0P~5JO;# zG~;asYK(u`c_l)}nVwKRo#{hKfzTO>Ox*_~&nKy7_S01Y2z6|`Aldw000u)o2RLD`5cYUEmUt z90U%|KfI%47!2_pA~hGqAAq&`&gQ%qa{jCxmdeQ%s@p}5exc+BdC&$K-^hX@s7@pP zv~PBiUe};@P{V(cHQi=G`+Rt04WBUNg!2Kn)!BZ17#3Vmx7Ziyp+zx!n7QEHrCNMM zRv$+YK)f12ycqD(>0lcF_5K9dlA{WSPiUMNey6gwNc8A9AV#vPNz8Y?(tLF~zv{bZ zq=iZQlTCCyMn;{PU6Yy8nbNsKg#^$HMp=cFp|3)ow3z^zSR!aXxaXxy{ou!vjY%;= zuUetAg;G2KKcFS&Hxe^uh&)+8cmw?5K?!sjjfZ?)D^6~+sec9+w*KG2A|~ZAYeBfpbFTUTfoxcn_0k;HvZ&fcIJ*zORbS()&mhf z6J&oM={@s&;$ofCs1M9K(BbvByKdW`Fol#f-z)*ym5Xh4K;@jo8K*939+km4cwqJh z>S~hL_a78P{x+eF5^KTn!L^b}_y~5eDKVrZz8K=*D;Gd7b$(-auu)(_nViQ>!98{x^1SRI& zXefQ*vRVL7)F4&Y&fsrv|78E2+FByS?<`Qp2ZkX>{5z?Ra2T@eH$DFi}j(@Psg5=DYw1g8=fth5{lx6i=x8cN9;MGSj|650qo`az6k zlv-?**-!{Wa6AkNEFWAqc6ue^Cd$2>#G2U{3JOUr_W6Gjm+i|am#idx6Bl;9a=rX; zDNVJ#2mv&d&Ob?D&EfwR{J%Q@{@Vkg0yN_W&j;rS?N;LqKD~dRapx6w93p1Cnn}WM z=)SqgHM@MnR>(rpC8>{VX9SuZu-xjtTixBr@nTc*Dq+vHF2C?FIPHqb~ackk8^=;vmOBuaKTojcqP=H+HGIlw(NVXuXMnZ! zpQkyQc(B9IPOIkj{k~aD)>O7I#4K{!aQMwb(7JlR4&lGYvpoz5R|`<%gsa6aHV~p@ zApeWYRWWS=hMq!jtuzIRuh`!yu^b#UA<&hCnU%ltxa>7=WS?VSdic%8BOb^GR&t53 z7SIy)2P*9PVrn)}M_heRsbt8(WhYPh&U&J z0-wNA4Bt=4MRE;a3_FK8o(6x7#M6KT;V<_Z0-{{b(|BNR#xJiUtl=AB7TjQ2mc+0nnwgRbL zfp-D&I$p{`j%qWi-5f+U#28MDe6sZ4n7juzB-}w2Mu1XyvHNmdWNRV-ua{NBtQE5iJ zsWa=WV~xrSw(GqPYDQc)Gn{s5U`n`5rlGP|UGKHRHoK{FyFTc$X!& zh1CGQ`_>>J)i3C~GQYm=$xx+^S0wWQ9Z+bvvC&PZrz~T6sQ%Bhw>3|d2(RfJrffpAAkvC`O}5o`n8DZ1T5v7$j-xpyfiFKXtd~PX~tT+5v}x z9=0{M9KvFPU10-Lh>G#X68cgY0gA_(c9E!4W34g z>DO~6*{KGL?INtQKZl&pHalc_Q9vEPJy8YCN|{RCqo zs`Rcv-pw#^0o4*(VCucH`qQkgN6c(*plR0^@y!@=42A)D&wIAEx$l1}&w&}$PMRXi z)%vSXUkvGHa_bgU8`m`(CL}E$6uzAi(Kd4hb_hWX7wl4(wgRpFurj)ruC|dSqg^7C z1*lU6WYI*u7CmTE0cb||FnI?HdC}}}$am5ynEpTD=qq{u)cwT$fY0S*5n}B}rpMKC zYH7cAXAjWYY~yyJTZnzW3j&(Dil98Lixn&x`!R-|4)NNoL)Qti>w#F}}|o=@&f&0K?!;YGFea{%KZZM@orz4_i> zWLk`Op2So7;_*OPvj9LtE0rS&`f6E*{vcHEhf)1U?}OCl0Ej@S-gnuO!wE7qdxg*U z9y(5Oq~S(aYPLakO%6FR`(~Fc#D`nv^r4wv^5yAPHoCoXAd|?joZv#8;`p#8M%#E{ z=-ca!$av?7*4F{**J&y72Ud2-7B$iCGgj5+S5>b%Xc+lijb-yvKNdlLC0M!G(j_$v25*nYZJ-ceq z;Q4M?-Mv;EG;@TpbzL7t59X9L;cr?+ zi+$bl&nSLm0seV6;=RQ_`{(&+i{Jp`t!&L#kUWj-*F|8)(*Pbf=m9J!Xjr;YqXxd( z%OyPk6KCE1$uEj})qc%A@IP?h8M?I==^()hGSuKBH>zR?)bVy*+}BDR7hAh@6{r|- z7e2;ff(riMV_P0V?P=0kO&y=q|4Z;gu>MQ%AGkhV&phaTT_|ZJ^aVa{ODi2la!)PJ zzRD%?TNSB2{uBI}85<0my#aAtP{B`HlxcNtMD!j0^wE928b(TY{S!|x}x-GPg?V!Q*!mJ?6wA$u>!Mh=at1AEW6KBJ(R81ZwvJL_N&uJ1#V6Y zIX-%Cm5#Yuf}<~w_TCf|otiy7l0G=7Em z4ztbnLy6SH{yEP6K15FV2m$t1de=}gZ;}xMA3-u$N~-CN!B;aSuDnXi!uQjxmqx$J zlJNZ^+?{|!bVsTgCA*BI^%CB=WguBKNSP2>{6gJD&o0n@R0^XOQjAVf>F-wUiq}ldrqGgQ*n-=6uY$WBF3%&Gp9x zJOU2Nr+$*+DOWLG#ZG6QQn@IT@WcjG6gUwK`T*diqyj>-rcx^yL1}yCE=4KSj zZp6khu!!k|LC()icUbKHDb|DXoa9kGv>7o5%v|`6vEF*L zlR`@8^=3e(?Abuxg7Cti&2bGgAOcRBZdJy(A$dchEPU`T4bj$_uz)3I?BpbJfx7n3 z9AWQ^#vzuEl6h+g%9$1mU*Crz`|n_d4F=NWq|%wP*wx*kE;tf`sEk}fxB+%14=Psu3LSr8Wb#6Q1O4bgaB*S@9oqo^dx zn=F9E?#4G(RP$mWNxw3-{+lj-ly9HnN>;gqS-p112H|9`%e$>HzHuw!^AVflqhYsV z$fR0)ThVJ0&9I?53>qN+O8YV{wpTP;h&`m}strM3>GZKRnePxSe{3`yTX`XZz_KSZ^+v?A9t;#ZqT)QF#qa|5IKVN)vek-1_Y8QEI9x-@D#oCpgA5 zwd}V;3NqxopKKI`i;?SMN+Z9L8wmJNv~i2AIs#Xwi23xa_4P`6Q>mQI$u<*_oIyxz5a0#JCEFZeoo4iqOX%DO2^3ll@twrh`taPsExvtbU-E4aa$ zs;A+WXaY@x%Mp~jxHW;3BPnDBX^B?Lt7Yu+9Mc_&#WS`Jbt;Kkj%WMKX!rrLa`Hl` zrr0?BSb>DZ)Z09%zPm)Z% ze}L8}dCheTm8$nmCJWRNEHhCpy|Th_5fhZ-SAha%TJt}I?vpSNr480Z{AWqLer&Kk zMfa7|O_rE1>}9E-iV+{T7@K@EKq6&WA7M~2!o&(}rv5eZm^@6}TDR3He(Sa&Opqpx z+Ny+A&xIH>yK3v~}nS2NtqSw4(9av~YY!Z{fD{_gBCEtB!PZ6oMe|`UT zGahOUkj=R51SmnmArKtHlUyAe0@16TBi^009;~;#S9enplvpVpK=G22C40g-^p>}) zt&Yd!H~U5!TZ^Zen{;Uyw&!_$6oPCLIg!CiQ7~bHn*dkTV-I|yUBK)m>F>n6qpIwlDdh$s_{kTKzSGF7*S{xxS#cjrv zY+pF#glTq!5iTGE)xVT-A7?H5r#CD=eXC%|dhj|W0U`@aaMo3+yT;~{7yf_GyHpWfeuIam!>18 zJfu)!6di*5V2R3LJy=r$JvKqMP zanM|~3bqXY6dFDTgo#GEYE#?gmiRENQ5uqm7Ctx%uQV+53eFu?v*WzHj?R?l?>!iG z^qv9pf|}0V@K~0wS*S?wE1C4K40}R zYH7n~SDv|oRqdAvFAZX#9zvGT1L*d!O#f-wYthKgxV!}T_)W(n3|Lmc>{493q?x*2 zblDhj&^Bt5CO^>WY9z8LaU*iF^0fur0$x0?pz6VX3gEtlm|V~mLUy|U+d}aDzbph= zsD%(^$4rm)&qB!lYazh?mxU1R06P~9v_syRh4n+s)Q{gaTmLaB==Dpoo+gmuvkn!O4uw`-Hw{XwF^XwFSaV~iLNl~QbiU8spY8^BxHf# zMVHGS?obrY6T^>N`T>iL%VW}e?ezmT2W=(8OTy-ZC&h`xuka`5nGkBa($62IGqlk{ zN{blK?EsyM!a!T)``Or&_lJ0F4Q7^M2yAp|v!q5O&C^u)(`_c=FVz`i8`I>~Cv*3N zT+g+Z&Mu=nf@2l4B)fH5$Jx4ezi;Lpw(QdBhcw%$2jMu{)cneXN|LFn{jTSg3YdNj zu{M&|j41ZuRTtuwKcSzcm_6Rm(VhwqHOR;Nn?3SfcmO}%NZmQ>&}-_@7Tqge&*S-_S0v-H3jR*_dA&F1c@g6UJh-%2@N7bL%h=@gEABO*rGQjD0_zYa!_XwR zs;?>`ak&xca@zaXDdYyLtF+TM|D%{K_0iE4D;P+Jk1vf+pfb)BUvFOvX&+;YHTPSv zqi49dQO7|J$0<=1we0g_pu!dKCeAy6Yg^m>Zu39$a?MEDW%_fF^JmM)X06Y+d6#KnZ9#Bx#IETYMpYte!w)lGk6+{ z14HWR%f-UCwS=?!L4`!+7o2#YKaZ_?j&|8`rs0H|$( zQIY7$j?E>}>l4Q#V`*^SA3S5dK5>dOCr^iN*trdlglX&V0&Tz=*p&sm>>F0Tsr?PI zbc>zgN_C{4LpS?Mk1WHTR>`V-voSq@7wIo67TCx$zW8gE;KoWMY>j@UyiUC#Uvs$$ z3T^|o0=Ni`E`9|+tZYQx_Y8@$FZwcHM3h{_qy8|xzMJ1NbrDBp#*{RHXWnrDOqdl? z>{}mlb!pRsfmNqU?&DLZ1Qx7`y;+{(g#VZJNgb=5}NxC3&1y{ z${hA8Ol{>+t5%W3UTj-C<&AVwmfrYfzDJgoL`^BQ%3rpoWAtQ*+)05cqTx=zJ2D*! z+7cjqwtF^h4C7cD&CT0mK9ujAVj4pN#$Ckmi$wGzfSEqzFPKO5dh{?8hPirdVW{Qr zSDjot;tRizDkloqTkBNnyy>qCP@opnOE!^1Ps*~C`cm){AA)1vm&73tfaxzK*KYb+ zh3}R;Pj=BFe0Zy99V?5Qs+Q4onr}ThTYM#?%00s?agWT!t$V_J%3OWba2b^O8LNJi zmlRf@0H|!OUyw91XO!eJV$8qS>I)Cgt)4%#RG5`mGqKE?z$}Xlm%WFwP9qhJ?EFi=#zF#4X?n$QLYgt zV$Q}R+fP|_tW98moO#w^ExKU(PnD*D(U#~FUB8iZEZC&nmAFF0H6hVG+8WE7ZQGvI zqcU|P6Z%J(*dWN$#0TfX)(l25Lp6Uf++cvx;I_bp~ziNV*}dQXk33!~P(AM}`$Xi#Q7nGVw`n{S*G zgHAN)D(>Hghd8Q-bKqMsCxg%y$-Ca<9c9KQ0xs1PB?i$6Kqs+N5;^ily2i>t$md;f z!HqVf7_LxHkL6B&ziHwZa++5*IX_EMq?|6uzSA%b=Z6QX!{y8GKbC}cdJg=>X9z~J zx&=N2-Pj3i4c4L3EbE9}9SLnMQ!QUzA4RF!kbYDa;dVcZyPNb;yP(QctG5zMllvC& zFlZ(mcu1L|;iGbOAiP`^5V{cn%8K2y|daNKAF> zAPn`q z$7$~{t8#3UihVcv>T;aSS6(kN@D-hsnW;C}-?1WTowinLd0A^b-6qEWR*MT!5upj8 z90xpEPN$#N*wLA_Be66Q16HrSEIWY&z)!wXut zQD-kJ=b}|y*Vevm5k9Rc_mkPP(b7)4Gc7AmUgko1rJFzc&~9AHPwSbZ?tLen(}_4= zCyy=N9nsE$t3B6bf9p8X*IiMkDw=q!CvH`_i4QH|)x^Q#NNkxD;wnRgUljQ73@taE+}zgO)dm>Oa- zUg{URVniq)J`&4!PQ3Xz{P+?GGdr^|MhG*))6t^van#CP35f-%R&5nqqVIcX3%dpi zedV}lql5y7o~2Gm60j9U|828~gFKA55;tmcj1YWC&f&dql;v<3j}ln2;)4k)|F^YN zj&Sle`%SW^vQ-QEOG8k@yWeF*>dyMAc9{laa{1dA46(Gk&XBSstSrV7f?7Tvk zvV7@LXlBy0P(Pu$7X5jGq`du4((2}b_caY!JP3&rJ2n3C2h-`r48yj{n)p{-@vvlO z%X0@ryZ0}u@no%Hql*tj+pXP4-L@qwhKI@{)|FswFi(4f)WZPHn6d0)1i5yeuw)rS zw8tOZZI%)mi#pNv!4{zioaN^HX}w?AX2%5C>8EG3GA|In`0#mvcLrdo!ET356Ro?h zL)#2_lB*AMcGIfXDLi4yP-i1#{405gSW_}De&lBCJkT#U!V_iZod;y|`4UybN^|Pv z8kMG+{VJVHcI)G&X8Wow;g#s`(UwZZ0m9){HXgsSm<%I-uzTWo2ylmnUP!e2xA}Ed zV}N(&AMA&l7S$Ab$oJO;tkTl32P7-qh2I{B9X`Xw1B?Y%RHAoXzbsFM2HK<#ul9u@~04pqz6kx?{X{4G! zwbEuj$asXaH(s-~iX?wd`XNVvR_`I+$4({}NB(IB2W466EhH230jvWkM6Qj?WoRfm zx7d=^Mi5hwA6*%7v^o^OQj*&Q0-jE(Ukc-zgCR{H%r?u@#bQ=0ICYPr+v;wX9|=DTCsq_oQJ_HFHH-r{;!Gs~dDfqyR?ucZuVA9uNQzo+%UE+#kEXFuG~wigwY5&g`bf+8$Jk6>p`V0NK^AL`weG`S{JdVF_dT}jP~mKmJS=^?GFiS4|pV6 zy{U{;tAdO#7#0HzO&0LHEg>|i=u1E5N!Mq9sg1vWr+_53II^dSlL$x!Pr`$^3CAS$ zqa;@XGRg^XL!}UT85()esQoOkm7QuZug8e=mDc!O@kArwyBa>htTto(&`4~5rg`l4 z3p%JdR9f()4K937MW|Rw3GqOuoBNOtBC2)%m{5Hjjwv&O22a}b-aUcDq@Al1u|?F~ zOu2oS+1YJ}+=eyZXYDoO6>dE)r0Up&pG;PiD?3G6SKKtX-`(3%Jf1YAWoC}rG?@nz z98ID6=Uyz4247W=G{|GZ`1`l;8}qPNLW;}tgdt5n@^3_mp^cmOm*Pr2uq_Q!fFd~5 z|9vqTWIhYfg3SOm&jO@i@j(x>00xr(`WF)h59Yr=cp#BEXovq#@xc5a9o~R|a{wv0 zBW=*kJm6glbd3Pn&j0`*tOWou+=b4+Xss?9NNFCx1b3(Vp8zOp9>4?ltoN7b!ND8+ z3#u(ZgQ$l8NZDV4W%M7CGHHYc!dQgX;~D?U8ZSb#Bqk}%CT#!vb8kTFi_rI_{Z0SX z2$`XSNSB~Bp=SST9G3vRaM9)<$z=dJsBa0t4i|6rj{ued?6CYGiDiH)+=bJ>SoPnS z325VQtkU;iOnC)bv=sCol9B{O1m&$j_s<>rFMIemYx?;g5&Qw*fOC%iN233d_woP8 z$`5Ea(&T@HVHHX!K>v<-1N1**2U>-W9c_9_Z#q6`Y!$!+YY4htg%0v)HAr9$APs8< z3RnY(!3u*q)<4AtRvYH2@9C|1Q8tFff4so

    -{% highlight java %} -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.sql.*; -import org.apache.spark.sql.streaming.StreamingQuery; - -import java.util.Arrays; -import java.util.Iterator; - -SparkSession spark = SparkSession - .builder() - .appName("JavaStructuredNetworkWordCount") - .getOrCreate(); -{% endhighlight %} - -import spark.implicits._
    -{% highlight python %} -from pyspark.sql import SparkSession -from pyspark.sql.functions import explode -from pyspark.sql.functions import split - -spark = SparkSession\ - .builder()\ - .appName("StructuredNetworkWordCount")\ - .getOrCreate() -{% endhighlight %} -
    @@ -73,24 +38,56 @@ Next, let’s create a streaming DataFrame that represents text data received fr
    +{% highlight scala %} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.SparkSession + +val spark = SparkSession + .builder + .appName("StructuredNetworkWordCount") + .getOrCreate() +{% endhighlight %} + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + {% highlight scala %} // Create DataFrame representing the stream of input lines from connection to localhost:9999 val lines = spark.readStream .format("socket") .option("host", "localhost") .option("port", 9999) - .load().as[String] + .load() // Split the lines into words -val words = lines.flatMap(_.split(" ")) +val words = lines.as[String].flatMap(_.split(" ")) // Generate running word count val wordCounts = words.groupBy("value").count() {% endhighlight %} +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. +
    +{% highlight java %} +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.util.Arrays; +import java.util.Iterator; + +SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCount") + .getOrCreate(); + +import spark.implicits._ +{% endhighlight %} + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + {% highlight java %} // Create DataFrame representing the stream of input lines from connection to localhost:9999 Dataset lines = spark @@ -98,23 +95,41 @@ Dataset lines = spark .format("socket") .option("host", "localhost") .option("port", 9999) - .load().as(Encoders.STRING()); + .load(); // Split the lines into words -Dataset words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterator call(String x) { - return Arrays.asList(x.split(" ")).iterator(); - } -}, Encoders.STRING()); +Dataset words = lines + .as(Encoders.STRING()) + .flatMap( + new FlatMapFunction() { + @Override + public Iterator call(String x) { + return Arrays.asList(x.split(" ")).iterator(); + } + }, Encoders.STRING()); // Generate running word count Dataset wordCounts = words.groupBy("value").count(); {% endhighlight %} +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. +
    +{% highlight python %} +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +spark = SparkSession\ + .builder()\ + .appName("StructuredNetworkWordCount")\ + .getOrCreate() +{% endhighlight %} + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + {% highlight python %} # Create DataFrame representing the stream of input lines from connection to localhost:9999 lines = spark\ @@ -135,17 +150,17 @@ words = lines.select( wordCounts = words.groupBy('word').count() {% endhighlight %} +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used two built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function `alias` to name the new column as “word”. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. +
    -This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used to built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function `alias` to name the new column as “word”. Finally, we have defined the running counts, by grouping the `words` DataFrame by the column `word` and count on that grouping. - We have now set up the query on the streaming data. All that is left is to actually start receiving data and computing the counts. To do this, we set it up to print the complete set of counts (specified by `outputMode(“complete”)`) to the console every time they are updated. And then start the streaming computation using `start()`.
    -{% highlight java %} +{% highlight scala %} // Start running the query that prints the running counts to the console val query = wordCounts.writeStream .outputMode("complete") @@ -191,29 +206,158 @@ To actually execute this example code, you can either compile the code in your o [Spark application](quick-start.html#self-contained-applications), or simply [run the example](index.html#running-the-examples-and-shell) once you have downloaded Spark. We are showing the latter. You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using + $ nc -lk 9999 Then, in a different terminal, you can start the example by using
    - - $ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount - +{% highlight bash %} +$ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999 +{% endhighlight %}
    +{% highlight bash %} +$ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999 +{% endhighlight %} +
    +
    + {% highlight bash %} +$ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999 +{% endhighlight %} +
    +
    + +Then, any lines typed in the terminal running the netcat server will be counted and printed on screen every second. It will look something like the following. + + + + + +
    +{% highlight bash %} +# TERMINAL 1: +# Running Netcat + +$ nc -lk 9999 +apache spark +apache hadoop + + + + + + + + + + + + - $ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount + + + + + +... +{% endhighlight %} + +
    + +
    +{% highlight bash %} +# TERMINAL 2: RUNNING StructuredNetworkWordCount + +$ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %}
    -
    - - $ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py +
    +{% highlight bash %} +# TERMINAL 2: RUNNING JavaStructuredNetworkWordCount + +$ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    +
    +{% highlight bash %} +# TERMINAL 2: RUNNING structured_network_wordcount.py + +$ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %}
    +
    -Then, any lines typed in the terminal running the netcat server will be counted and printed on screen every second. It will look something like the following. # Programming Model @@ -222,10 +366,10 @@ table that is being continuously appended. This leads to a new stream processing model that is very similar to a batch processing model. You will express your streaming computation as standard batch-like query as on a static table, and Spark runs it as an *incremental* query on the *unbounded* input -table. Let’s understand this model in more details. +table. Let’s understand this model in more detail. ## Basic Concepts -Consider the input data stream as the “Input Table”. Every data items that is +Consider the input data stream as the “Input Table”. Every data item that is arriving on the stream is like a new row being appended to the Input Table. ![Stream as a Table](img/structured-streaming-stream-as-a-table.png "Stream as a Table") @@ -236,7 +380,7 @@ A query on the input will generate the “Result Table”. Every trigger interva The “Output” is defined as what gets written out to the external storage. The output can be defined in different modes - - *Complete Mode* - The entire updated Result Table will be written to the external storage. + - *Complete Mode* - The entire updated Result Table will be written to the external storage. It is up to the storage connector to decide how to handle writing of the entire table. - *Append Mode* - Only the new rows appended in the Result Table since the last trigger will be written to the external storage. This is applicable only on the queries where existing rows in the Result Table are not expected to change. @@ -264,16 +408,16 @@ data, thus relieving the users from reasoning about it. As an example, let’s see how this model handles event-time based processing and late arriving data. ## Handling Event-time and Late Data -Event-time is the time embedded in the data itself. For many applications, you may want to operate on this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time is a column value in the row. This allows window-based aggregations (e.g. number of event every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on streaming dataset, making the life of the user much easier. +Event-time is the time embedded in the data itself. For many applications, you may want to operate on this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time is a column value in the row. This allows window-based aggregations (e.g. number of event every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on a data stream, making the life of the user much easier. -Furthermore this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0 yet, event-time watermarking will be used to manage this data. These are explained later in more details in the [Window Operations](#window-operations-on-event-time) section. +Furthermore this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0, event-time watermarking will be used to manage this data. These are explained later in more details in the [Window Operations](#window-operations-on-event-time) section. ## Fault Tolerance Semantics Delivering end-to-end exactly-once semantics was one of key goals behind the design of Structured Streaming. To achieve that, we have designed the Structured Streaming sources, the sinks and the execution engine to reliably track the exact progress of the processing so that it can handle any kind of failure by restarting and/or reprocessing. Every streaming source is assumed to have offsets (similar to Kafka offsets, or Kinesis sequence numbers) -to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. +to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, using replayable sources and idempotant sinks, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. # API using Datasets and DataFrames -Since Spark 2.0, DataFrames and can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` ( +Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` ( [Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/ [Java](api/java/org/apache/spark/sql/SparkSession.html)/ [Python](api/python/pyspark.sql.html#pyspark.sql.SparkSession) docs) to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the @@ -285,9 +429,9 @@ Streaming DataFrames can be created through the `DataStreamReader` interface [Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ [Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source - data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. - - **File sources** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by move operations. + - **File sources** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. - - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The connection is only on the server. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. + - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. Here are some examples. @@ -305,7 +449,7 @@ val socketDF = spark .option("port", 9999) .load() -socketDF.isStreaming // Returns true for DataFrames that has a streaming source +socketDF.isStreaming // Returns True for DataFrames that have streaming sources socketDF.printSchema @@ -332,7 +476,7 @@ Dataset[Row] socketDF = spark .option("port", 9999) .load(); -socketDF.isStreaming(); // Returns True for DataFrames that has a streaming source +socketDF.isStreaming(); // Returns True for DataFrames that have streaming sources socketDF.printSchema(); @@ -359,7 +503,7 @@ socketDF = spark \ .option("port", 9999) \ .load() -socketDF.isStreaming() # Returns True for DataFrames that has a streaming source +socketDF.isStreaming() # Returns True for DataFrames that have streaming sources socketDF.printSchema() @@ -375,13 +519,13 @@ csvDF = spark \
    -These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked in runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. +These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. ## Operations on streaming DataFrames/Datasets -You can apply all kinds of streaming DataFrames/Datasets - ranging for untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the SQL programming guide for more details. Let’s take a look at a few example operations that you can use. +You can apply all kinds of operations on streaming DataFrames/Datasets - ranging from untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the SQL programming guide for more details. Let’s take a look at a few example operations that you can use. ### Basic Operations - Selection, Projection, Aggregation -Most of the common operations on DataFrame/Dataset operations are supported for streaming. The few operations that are not supported are discussed later in this section. +Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are discussed later in this section.
    @@ -482,7 +626,7 @@ The result tables would look something like the following. ![Window Operations](img/structured-streaming-window.png) -Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window` operations to express windows. +Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window()` operations to express windowed aggregations.
    @@ -533,11 +677,14 @@ df.groupBy("type", window("time", "1 minute")).avg("signal")
    -This windowing based on group automatically handles late arriving data. + +Now consider what happens if one of the events arrives late to the application. +For example, a word that was generated at 12:04 but it was received at 12:11. +Since this windowing is based on the time in the data, the time 12:04 should considered for windowing. This occurs naturally in our window-based grouping --the late data is automatically placed in the proper windows and the correct aggregates updated as illustrated below. ![Handling Late Data](img/structured-streaming-late-data.png) -### Joins Operations +### Join Operations Streaming DataFrames can be joined with static DataFrames to create new streaming DataFrames. Here are a few examples.
    @@ -636,9 +783,9 @@ There are a few types of built-in output sinks. - **Foreach sink** - Runs arbitrary computation on the records in the output. See later in the section for more details. -- **Console sink (for debugging)** - Prints the output to the console/stdout every time there is a trigger. Both, Append and Complete output modes, are supported. +- **Console sink (for debugging)** - Prints the output to the console/stdout every time there is a trigger. Both, Append and Complete output modes, are supported. This should be used for debugging purposes on low data volumes as the entire output is collected and stored in the driver's memory after every trigger. -- **Memory sink (for debugging)** - The output is stored in memory as an in-memory table. Both, Append and Complete output modes, are supported. Note that this requires all the data to be stored in the memory of the driver and does not scale well. Hence this should be used only for debugging. +- **Memory sink (for debugging)** - The output is stored in memory as an in-memory table. Both, Append and Complete output modes, are supported. This should be used for debugging purposes on low data volumes as the entire output is collected and stored in the driver's memory after every trigger. Here is a table of all the sinks, and the corresponding settings. @@ -806,21 +953,21 @@ spark.sql("select * from aggregates).show() # interactively query in-memory ta
    -### Using Foreach +#### Using Foreach The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.0, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/ -[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), which has methods that gets called whenever there is a sequence of rows generated as output after a trigger. It is important note the following +[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), which has methods that gets called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. - The writer must be serializable, as it will be serialized and sent to the executors for execution. -- All the three methods, `open`, `write` and `close` will be called on the executors. +- All the three methods, `open`, `process` and `close` will be called on the executors. - The writer must do all the initialization (e.g. opening connections, starting a transaction, etc.) only when the `open` method is called. Be aware that, if there is any initialization in the class as soon as the object is created, then that initialization will happen in the driver (because that is where the instance is being created), which may not be what you intend. -- `version` and `partition` are two parameter in the `open` that uniquely represents a set of rows that needs to be pushed out. `version` is monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. +- `version` and `partition` are two parameters in `open` that uniquely represent a set of rows that needs to be pushed out. `version` is a monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. -- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If the `false` is returned, then `write` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. +- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If `false` is returned, then `process` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. -- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is the your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. +- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. ## Managing Streaming Queries The `StreamingQuery` object created when a query is started can be used to monitor and manage the query. @@ -952,7 +1099,7 @@ Finally, for asynchronous monitoring of streaming queries, you can create and at [Java](api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html) docs), which will give you regular callback-based updates when queries are started and terminated. ## Recovering from Failures with Checkpointing -In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger), and the running aggregates (e.g. word counts in the quick example) will be saved the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in a HDFS compatible file system, and can be set as an option in the DataStreamWriter when starting a query (see the previous section). +In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure a query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger), and the running aggregates (e.g. word counts in the quick example) will be saved the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in a HDFS compatible file system, and can be set as an option in the DataStreamWriter when [starting a query](#starting-streaming-queries).
    @@ -997,7 +1144,7 @@ aggDF\ - Examples: See and run the [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/sql/streaming) examples. -- Spark Summit 2016 Talk - A Deep Dive into Structured Streaming [link](https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/) +- Spark Summit 2016 Talk - [A Deep Dive into Structured Streaming(https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/)
  • %3A9#UphGkjOS%M5)<`qz#Ao2`%s1 znH|Gh^z~2sh{s=_=o7W61#RQJuCAppuS})zs?aZJ2dR}7HrvEVHFf0+TJUNSKT>sJ zenRlKi2nD8<;^RcwOzu&b-WG#DXOA8sTQG6F)Vtr;Pf3Qo0z*v+R=XJSxlFgQX*Q$ zhJbmVSuam8q5E;+mguyjx1)6>iXKJ!2@%%7Cp5u1;}U+A zb|%;Vab}_#Su5XDkXh}e9+pqvQ*e_zkUdUNbplNDbtjuFpR&1;0@Oujc{nWP@7sG71jlBp zdKsvW1`6-OU6J%<2?X?D1RfpCjxr&XN3D&V~N!qfSVI#Xn z`JRE4C{*?Y^h%dg-*BsDti}mCD1h8}4AX5k@=A%aoo}_C@9~UI#S>BV3fE4~z64D; z5=+G`DwDRCt4j=pwe9yVHW}rdghX-?9q0Fl`bmL9ngR80a}**|i*IKIV^{W-4^i(I zUnaI!=$@57k22Y7a-h;5FV3=aY!OC0cxgW$Xnjw+pcGUstkPGW*w;CIrE}O6)4AzX zr5SUnYdSpSP!O7==z@I`@(|}AdD!2T#*mD7P!>SL0X8?~v$okXqXVc>+RlPW#vNPJ zeo0jxH!rrHUyb>)VM%Fc2fV6F^KG>G)--h*yF5qSmO!Bg!loQ`y> z2F`0i9z58h7(P?f_S^BGRUI7VaY{dfsQ}Kr@(GZc2)m+b^1@|X_{$TJzR{4{%X{)n z%>^9iO7bn?eNnmzoC1Dv&gVg-`?3$B4-TcPQ?jK7&vOsxEM&q8w-;J;Py0_L_!ag~ z;#VRXvN_=nJ%A4924(rNFsqp{AC|tA~KAv&QLE7F}gd zot}UKFv+K*gpy@7_wfx32g;4o{GSrEQK5>KAeim3Pr!4zJ3~k-vZ!biMY`}+B}G{s zdIhh*(w6vmZqUol;4?wBE$=Kp>){}HYNloAyh?KZLqUbB4EU8~b)~dlnbB48sa1mu zmJc_WGK;XwN$7J=3Tf-Jj<0e4Wd}3&d9-F~6UoxE4k$Y<=YzZ+bcuLCl$?ZR(U+v> z=k179GtRaJ+Cm+e6r5gZM+sAmv{3YkY8))2;x9K$1)?FYB3(|du1(H$Km-hc|o~6!94Z}X(dbP4CSrB1SWqdo{9X<3GcXqHv8m{F8^TTXMQort1vsz0 zD`L88Sdhz1n|KgEmMT`Ey*@Fs3d|Mzwu+f__0(uk)X)y-bIhS4&$Zn@P(QmZXL3fs zxQa|hI?$Wv6+0G)pfnQCC=cq04Rislml&qmbkfh2P=+;y&EN9J1Ciaa^G?Cg!k-J} zv`;oP%JM^e>^UQxNUH7@s?s8-N4nmY5}JLXd_BE{*4+|A#t%py&!Nc1^-6ITwC~3s z%s3QD#`^eQ)?1;N-_}RhGPh4uDv#AnT@;d67}pa2nA@D0UYR3rdu4F9a6PfdCAW+) z6{#>xSvV?+&`NNYf6$fK9@!uQy4wUAAB`WM&~Kbk=?g3fEnm!$m(Wo`2|o5z=#ci} z6V5p@KGQO7*@y*=9W`!}^g|1-Bv_fkax}a0ozK%Xl=oa2QUEK#*Dd2M-yo`cqwRrL z%1PQPj+S?uN>CHfm~xMSpxd-ZvCs@F)h8n5)QyGnTWLwcw!kYnERBryhtsjA@i+Sq zi*^OPy;{`f`^S}8TAQXgqL)+6_4kbAC)%E~dR&(24|q`wi7E}%6tB6lw!1_ig<1Tr z#wW$HmZNkHOC4s@Ul`;aUR3K>N8H%(vWQyPfY;fIB=O5oG9ymF%NDEb*5hLyV_x%V zxn8Nfrn=9-^9Ar{;2wAEONHFORh9m5fm-n&4plNK;eXxkeT2*~v3l7uP~tbSZ>B-r z7qp<;ac5=y!2{V1v*S`{aoXO?jApyR>5?-@<%XW7wx36(F>9a5V4d@(e+E#Ka`5wp zb?ks;qmh;RDZ7jLj-%iSEh%b)v}?+J(5C4nb}b!vhP*A+y~(PA4cRa!wmYMwew8o* zkTy~_9Xl*kn-7tSDLsTuic3wA?0(jDNj`C9S-taFIMw_s>2*E)QcXOlUxTTCXtrKWM8DhB0mTL+gG>%Qbqd zbx~y6LV913dDKAyb_&AZMzvA)m#bqB=FPvB!!DU-QDkNj?$jdBeGUHFAl3u_>TZ_s z@H)YGFdJSz%Kg@AG`7jEE^(q8wo=c_?BE$5sV-E2ltP!=4X}~gkmY$zm~+xkzkedA zw9noSn7ZXWQEmrysay4@lH{AwN)3zLhU8yFHgui|(`D>Ybl48_)32S>I{WX~WQtgC zIuUK;Sk+I_`6M4>hZKY%Ci!Ksyv4h;!y8r}6#!vOA=KgOF4Q(fOiz9OZCmg0pd*=@ zk>rcOrjiCRKV@YFVUN1UcQ0O)I7KIq*c}}U@AFhL1Frmdg`HfmNu~dEI2*N1>-mI6 zfenbs2RxU{hk%z*_UB_3@r19t3bzaz0;076Uc)F;2Xam2qC9;iysyfOI`xhr&kf*4 z8NI-(w-6eaBI3h0J9Zk@6WzRrLb6>(C^->$Y;m#er_t5h++zxQEi$^SR4pKN zL6(D9X?HsQu2U}&bR4>2rD<*jZtN@e9^GyFm+#9iyR~OCzf!e37&sc#a&rv+xw)@w9wPOm4v7NsE@Ftw=RQu5?}t zrp!Dyj*zO>e{LIPXa%d5GwZ*A181p{^Swk1&#)kzgm13K3}R>`;xoLZ3;L-lCF&%M zbzK+mMe`h#)Mn?a=bNWXpqFSniaOR&Py}H?__%6xa#=uOiUxsGf~ycT#F*U+J*34W zcEgKr^G?I_%YzTw&&OxGL)O}6AF2S0saZjF& zcJQ}4K=S2h6L-8OpYf4LwyOLKUhl)5(zb^T#L@6wD9}7?AEBF?0k~^XA5vrYd7H-3 zTkdcIL9#N4y_~R(FO3kXsCuyO-~m1LI+Z_MUu8+l$U1$^?r+~UN7a}O=qs?j8wSU& zrp$-i5F+Eu^ZoW~+`*Tx0TsBS%8JY8xWxRZe`ft%?%-Slwn9St-gf*heT{o?jgNMD zsF6iJnc7wcd@D!G37mUjcwnf@6bEhD%oF>N6An@xSva$JxYe_3Y)!tjyofvQ8z3OZ34u?^omGW1s83RTy?JXE4KCmD2c z#HO6^Hn(;}r${Ng)!RD`o13KjwunTs<$d0(F{2cQ;ih9pf$m~nnsiEJEd>+J`);wZ zs|+{1YzA>%&h0*gXM6KPFU2}*@EL8F1Sy!_JJoLvob=+R zY`$`yCK5dylh-}|G#d9Jh4hJ&^g%Xo!~XHW>-8Aa#Wk^;nG>1&~s zl2y{ubY8iWk`jl3p-0F`0YCbRX|00MT83Si<^!t4nHy7NE3ZZUp$B&%;x zs67y&4vD0CF74vJG~6imDq=XGz{Xd2g1C7KICeeLgLW#MZXMjS`CWsE6Z^7O1G1T5 zSg@RQ-ubV<%Xqx&Z}CSPc-#dA%gfj4Vmh)veZwW(sYaA@jZ;o*ZK`&cE-FuzSW3q3D0GSsU8S@kSL{lQ(Ys%gZ#rO>*U$IBSMGYI4u6gKshjZg z62F_=3Sw8|ElNs^(!Y(qBg67w7aPJvs_JK)jJ`=LTH!;#6;p{(UGf^4zc~03gduxV zS&4MA{JTh*0^6MPlGj&O+_Hg2E%j`-$L3xFns06_J=eMZW?k0Sl&)eDyG_?;f=`?Z zYaZr6ohPs7@_rup=S#O3#Pw8Bh-Z{9??6M(FvN@M);JW z?^&rb#+^mu!i@1BVP7XR_S2!F-v)(kwaU&e?^^7;c60`75*;;bI)p6ZcEEz=OzU`A z3Uu_#CB`hOVoRkFJslVByS5e7#$KjSK(VV&RX@B=kwa5ex&q*K2#IO)S+cw1zv>ya zXei~`Xtl3h`F(^jr$3a&>74-csyZZv7)W01s_PK9XcSCOd>Bi;!qI};(6D%`BP8es zk%$(^VB96<5ikwvYFk=(6eOy5*8B+;19&bScvydI-s?dmx;l$vg!Q^3?O{xfX*GCt zm~p?Kl*Z+HaDz@?^kF7%CT7}a6y{YV3JMzRQu?C?aOlE@y(TT12%08;mKO^X{Drde;R4o$luLrH<;lYZ)!3#ZR>%enuw^YUf5^H-EC27#uFF!Ams0q|ucJ2@gjv|5+D{%`E zREI6P`qwrUO)3uCI5_SL_;+p8`>2+OQ;Vi{o!R-p()xrr;qacQJg#di`ZLKxXDuE1P(PV2^HxE80~J*8*e89&u0C%2&xo_NA$q` zc}BX3_!!Q+1LRCa0ViQ?J*lHiJaS`cl+}l&g{!QP!t@7|VCI_Qicw5JQi(l=TMyHg z6(AVA`fR1viu(`bX9^Y>Em|Q(7yq&MYt8!r-Tu9{bu3loa0D9OE%`sna z(}d>V>mHK($tQ|q2LGt`>fUp!A6dm|tQ!!{!kGQ);hxOK9{4TjIEbk^UFW50;b!jb z?RgM2pPFZ5AkjA5SU_l}F1f5EMi>Z6d80edKVyf!?#7kZOS;@!D^p{Z$9?O_BWGij zJ>-bRy2#WJEl?p!6c4?cMe$QceX^9_bs|1y)HLryhSf^thxY-_Gn?1S#%PsGi+ztd zeK37)8IEBuk*(!0axjQ9e@9SB)kHuirMQG&df}O4v12T$>kJmtIrH9OtesULM{o$S zPj7u?*P+@#4TT;=QDh7l+8$&)FW_B^rILBhJKd}r1~l!@Tq}?DGJw9ZvTz!{?QIg+ z2X=p`c_k*Rci}(0t*J*0DPKD!5fXI2)~zG{n{N(g08cfa5FfKWVC-veKQMaI$Fi6* z$5mt3SSoc<^2mL=Pm57p{UY$dL~3dafiyeX|U-MT`7)9X)4 z_DWK#J+Jgct#}2E_UnrmpUq!5S@#-P23n4X3YwDjls6@LMmRMY?C0@}YZ++^)Z+lO z`dYCds^$4@;EcQ&VM86jxYh~ZZyTq@H+k%2(G-cj=lAhn2?udewB0^U-T+j80Zu9} zh@C78xy@@hGkbrr=j`@5-?a2c5(2#|0N?`}VgtgtQHM8w<;3bDFsBrK@)mfOChJ#p7j| zOA4hgbnDYvYy?#uJ}vThfQX%l);V7g2QqD|CV!smw*G~JCqVBtEKhUJyo%R_+4C4F zi$m{JK`2tdRyc{3nl@RMtLpCS4n3TI$81lBs?hbyoI5CI z`wBw=2Vqr*stocD5Jkhu_DG1wc<;r0yXtYKoS*`Iyt`EjXSi7cRh+A8aC~=S4$v0qEP?+Fg$tU+lVnCEBGd?{jM29xFFe_0%Yr{tEBj z80d1E(kpiz)OEN*5RGznsU~Y}rQRM9?rkiwl6O(MOCNmkC{Z>zKlpL57(=)!S@ucD zhi~Qh|1OTGG>WI1nC!}sCyR9Y;M=*9kLt0R!&WnZ^2-*;rI7`ks{$0BQRh%7@v`4n| zYgJgO)|gx>DA+Oqp7%m?ZN--tTv1Xep(?+7>4D{$}=lC7YH)*D|%tyk>QUJ~Em6A0+tjs@$So3?3gAzpD9P?R|Gx zlKcO6<21CLkeR8ci9l*uj+_m()ZBuKt2EQx8^v|3ti(ZSYVL9?jx@KTWfGYwIWY$$ zYN#ZL3lx!u&iQ?>>(n{V|IhEazWjM#T%ViI`}3Z!*X!mU=H?i$Z&z)SA0}plM-+#4 zH2ou0ZrZX(Xnx1yw@O|x*|ofe+duotymRZ)+Fm>f=ms`~aXgJQuQ-%abA>@hQgHV& z?}dJ%a5;=&x)t;^(3nFaMHI}lD7ix>tZPvs9vxI?Ij<6qCO#4^gw_Wtfis$I;gwkq zBKQ`xHekas!Y2oT4M;L7Ufv}PDl3%UD5y6azgO zO*|M*?&2cIBW=t=QD7#V(ep=3&U}h$i#7i)ege&xPQWwAXh0&k z8AGvt4frQKD5#95f6F+V)_2Lqdi*&ZyHO>~Q-6s9Wr!NrbGn&GfMLNPM8D{*HhRf2 zYzOl#EHIF|Vi}d_G!w=fM&_y%f6wzgfbVDl0j&sWC2JVaUf0dGW)DZ18&Xq5F)c^bHkADjlyl&a=war#P--V!LDr5VKljy%x zf@^Px?RkIR)IaMFrpk(6_amjV)jq?{m+`ef7!teFD6X!P%~rg`L}fc7xjK~|1S5EhRj^=EnkoTRXg|%So2|A zU}oYc5EO3w4;9&uj&R#L;rU-_HdF3Z&ckWdr`N!66V8|S7-)gEPhoN&@ZaL)pXKIP z>i<7`{}q`1-~PSG>Nlv8p_8^9^xkdWa+%M!p8TzzeDHATzrZd#Cq;s;caghE&*0mH zzuNtN+I9f&6@}H++WMa99%^SLG&g^x+w+pmjF8XtIWBqGR<$qq9wXOA*}HiYS1O>jgkL;(EAn>!drO!3M>3teE?4v|2N;AyZa-Q{BFdZ{%NC|wqzcEm9m}Mj&BIqJ^i{zqrx>9Cc&Kp%pno)HB^fj zo_IX^)cIPZD=bwv-!Rh;6pM9X2V5uy0m9|fia9xL80-hg#=_(yf!N6zx^eBY*yt8* zdoSIx)@eKs9RV%Z_q-YQLFniUG4A;fn4Ax0+h<$TgTLuGU4~uWwCVH_U9HRJ;y)i< zuKT9mpU4QEx||jJyCQzowR-DW$TAldEEDD zZ~dbxy6mM7!6v~?MNip_V-afS)iFbFJj^DSlg;NvoxC97rOI04Zr-u?sbC!rur10&w}DmR3mUoCntOJc6H<{Ol#zyS`wK z9y!VStj7Oz6di}nCSnI*h0c?v+Xa6fle>67I1carnD zvTb3fz@f(ylpWs0oQ;&upo z{HN;?=Stmj?@t;?a`#d>jCyR%1~^J8czjLo6r-tE>Oq>lYI%8(?RcJl{=vFMiBEE; zk8~9-X96|4UGRPACIH`5t=J{+Ohsm`X+7~2pibJ;bQJqIQQgZ8_mtW`#Ra58*B^cH zgpHQYKX+ShNFEMq**Ia{qF)BL(%zHtEXaywQ}%wi<&QR^ZD#R&%GMeIqcl2*W?$Q* zo-zV316G<+%!cg)1e4P< z&4zojQzw9mY+6}GSj65n%I<7}Yxc~$nlKYQ6%7Q)<^rvlN)g{$v5)oLt?87m`Y>Os zc4xiNb@+Nn#!*%JFEK82XntSCd4S=xGT5q^v%`UZDa0=4?PdgW%VJ`5W9a}tZaBcn z!#p$MIZ`vld`QxQFc%*St@iI7iUu>Y_Y$##L;ZP1tbOwqklrwSg;Z~?M6SALUlqdP zcUMK^czYW_d(;ATmqLV;06|}yvqgx(A5XYWuJAwfS$eEI9_E+ogn<0YdQlhVIvU4`XW8$J=>@eqiJHb0aCCd@j;_Nzd4~_bGsg>O87jTfLiO#z}Ckf*WFQL?( z4k)b9N-2*5*(sWGT69+M2SgTD(JH@agG$fsVPASm!xde`+hOlc`9~2QlgsFyrkvMh zrVVD|kh%17`!fT&e_D6aQ#-t%oxrAP*`PPr%8Zm#e-A4GIGet${4}4=Fexrs8Uzlc>9mmsh*o{N6s`*uz(P-4m8;`sFVPuYJH?t zXBB?W)A8x0JCuc#)1}os1%etuXIQ(2Ya>c`rK_!Q>`a3wuYp_$+(P--*V{{4cr#%y z#sjQusoT;{w zvS(wxUgy@xE})cY^R;n$_aZO(xM8ww;W8Vt@GLSMmrpy~D#sH`kky1o@oM$GiWX2} z9wEl#ydMzi;c{;)UBOI$CaTWJ{CoD!R%BoPhmEx0AMPHLkyzoJ4T@Od#vI^V3x<44 z{Pig2JVoCpvhAA@+<)euF=1=lEnOx;U>F}Z;+sZh>SuLQVF;i}D z9a%L`>W&saBo_`Vh!cCVx3;ttLTb&-AsY#|s@iyP?v2N6Oa&$~6o>bVP7$f0u11_K zzkWsuQ_DMUV&S$t>XKanzgr(!iz}$tI{-?CD#%r}a@LC2>c&gKa=wPUc6F1ud(dT& zVl&>g2GM3C~=9nHU5Ra%R9;{^!{y zLnx49GA;GI@6^V4kUQ$?66(Cn7@G)nM+U54bt#j&HxbBSis9GD4LYp)v2Y?}?ofa2 zqntn~Y_DuXhps za~o#bHC1VfYxQI~Y><1)JJufUHT?&rE8OB4Xl{qL$G)=O+UboW0#6Pj$7Ch0>VVg+Y_Oc8Yc?(A^Imv! zsM9G-=h}^EVp&D2GeDacV$=25 zi%zFU(ZY19vFezp(lDd?ajU<9R|c(xmsaa1Bebs z&<$sBXdy!{Smj3bYTTeY!%-xFRg$=$ys}t-+_Ay2{|?v1I^zjKA=kqO2$Boc=ykda z8=5(Qf4uY_Qi17L#3*ErLi%tYy-|vp5^*#7HAfU1)LY7VEg$6#koJSi)QM-AYUMQb z8P=H>L5AaQjt1I|y9?+t7weRtHs}lN(^6?SGL`YeK;RPpk*O-hJN;vMDy^8z>)jAD zbF14C%Pl+58q6%m9zWmgfGLKMXL)? z8W^-Ty*TJ4BDcMkP|eV=(Ct5E5yOyUs4_!dOfNDfQWOe4OywJlJFTZs2mGfx3k$ z{hqgK$)#1E<}(irdYm$g-i3%y59yA6LDyq$iy8_WnJ6D?dUzbGtgEKY3an$_OODRc28| zqfrzmRqb(AZYZuW(Sp%Al|a^RSl{h z@@V0T81BfI!aL-c`!%T1<`jbp{Ani}IfH%W5B6rouf*9xI*sUcfy&fK(_s#ZD7&&aTigqD8 zemsI6TF6n3iU@H&78S!3&L+Q3X72m0ZLrI_RD8J~oDF6kQarEnvkC)oMxcjN+PvC4 z%Dd`a*`AzqRVB!*^OdEm{zCM3riK3`;&rk{paK1HE~ikuN>ZQ-qE-GZAPYO2%1Jq z?U5(8F*}`rK3n8*7z)PT^k{y)nx)3-5wV}kz_iLC9-}-d!@A%uz#{znZq25Ds9k)-k(;27v zHYx{yZ;C3Ah9r?{#!+}h^rUTBi+HzCtv}TaCSevzdaf^q)WSz+jN={^arvxpAI*-v zl7lhDXxS){D;@T`kPDn+`1|PF(TN>RMGI-|5Aih&)owYMx!$wx$HUBmsfiwxdJKn@ zG!gdEAA>Qu8*E#B%xj_ER8yNopk35pF0V*2McHP|jWSOuN>Ww;;rq=-^7lrpgAg8w z=B)L(4p50kO!hhaT_8e5H#mp7AMb-^LcU4&I^Bcpg^lJ4o>)1@YMkCUXiG!LtTl6A zIDNXSz8q5S5Qd*a8$mY;f(&|ePi>retSLS#Vcl#$YJ}tVN&b{~FWr11q}=ee{eEJM zPXi1ZoX~o$B$GXMoNUs04N!N|&H}MKRzt4qJt0EnE!4)0gfvvQg{hv)f`#YmdGUh= z-yaFscQ-`-Zo21&8SKJv9T#Ojp10D5E?sbhHS*^~#Ch*pL~068&L&_b{nj!#ySB0y z^KZZFN2!T|Ffr0jjie*laT$44bC8-N7GE}GJYJ8`pw-1}R9u{R!y)wt*MLdog6X%* zk%vC+6;0_2_rH>F$~>CLTfZTvRuGn$)Pr(Aceq^1XvRpGI3P6M?Bt=aq(SN*dTT$_ z5H}JCt%aS*Ib2|A&`ycJ7R^1 zI(2nFdh^G{+(7GYY)NA>L2>2G#Q9T1R-Z`GGSjC~z2fg~wJ9?G!2G^1&w_;pyH3R7 zYetQIoc122n9$_QX?z?~4l{@%H+|0;L$m}bt5Ay$#)o?K)6(`Vshe_@D_4sg3XWrK z05C?z&~*DdQ*^`+GqrGsh?Aj4IlhH1yk0F^)By5Kqp}!G3*3hruU(Y_DumqfRCi9>}0rV z`Los3UY|s#OF#8~mr}R1K%;xpEray;TWcL=l6sMMzv)VHE$-XH+$%Uw9n7Ni3qcsE z8HqbCC*O+CA$$(-zqXj*4<#wNzc4Ud!iU|iS+9fGo7^xydPu?h_!pYD?~)46dx6s_ z42tE;4cF%;YjivI&`Y(|;^ST`DjHi9gk+7c+r)(`k;Bmw03+zIt;2+Td5UOm$Hkce zA$&(O2hphMc&So*^mMMSEo`ottda9{t@DK+{fLj(#A5aQ^1Lp7UkG+>onrsTy1_Ux zW)5Hk$taS<&O5W`(AK~K^2o4S9bk+y8t+#w=!(OJj?=1R*IrjdOiXHN7p4rojwL=b z%vPQFMM@#9`xUXwb?;|aB3gH9MD83klQf-h#gutRFxV3KGyxuvJuoy;65#w@s{cJt?xnX+RSTjj z!o-7pF;0YlbsZxVC+)Pl;b8|?K8Jgl^To6vgj_jR`Y@}BdD1;ER9`iiPqY|nAk%9; z(ecsRnB^4U`!rbCm1?5KkA4$HJB(b8Dq=?9fyGmTgd>aj*68UPA8hx&1!?ZG{$d&30*h@lGezXAG=p`FkP~VOR=lWbMT8| z%9fkk^=$`z4>aY(Hwy`;AUY)Y!t3M0lyhkTJMv2C5T{0(#D@*n>B&}NQ`d+9 zKApm@JYlF9fjhb1{^}!ehBWt1l;iGXLEz+10Fhuu*;$h7z{dJg%29DhRd=S;-EsK_ zWX}>mhEYM@3>0|9`f=JM(MUAn+c%kD<_fztMOJ@VR(Lf+(BVeyeq#6w#j!o>kHlod zS&rvbP#ng1Ql{m#tl>`eoaNSUqvx0sH-bk_?Q8PCa2kY%CUsT6od`#qLzrMoi?+9j zeC8^zT9(Cr)&c2JU_-TVE1f-_ru9s9H-*8#y6iGe@o^2L;RP7%NN_mU0ZXheKwPYbf>_s5(!8cDgDxhRW*}V^9d4XQ{iMQ$6)J5(bAG0+=!>x zUmBVa(=PLqAog_37yxU~t*?}I$HOD+b6Gd*$$MkJ&C*MId|w4tlUQOB?YsziN9z+X zZMLYrR832UUVP2;qTOVMT5YO;u2;)W%{EiztIjSW8*>oR5!SR9u5kxP7KW3TQxn*n z=wqbyt+gGsrN1aZ{EOa%D9G|(i1YRMg8}h!G91Bc$NXvmQSQ?-h!r}V*peGnp<;Y~1qUqELVT_U&b&R*J?^8$> zMRWF=mBG)@-C{7~a@*BXTBR3~Wd=TXh%lK=IoMOi$)puCKob~}BtHg~IBzRpXw{^$ z^)*Xw`DoWr7cXd!huN7Z3uVEMG}w(1SI~m2qbxK-*xSiS`B3+uYh4tjAnPxIxFqGJf| zFJmsB#HfEe{$tW?G@f|f(E18%4)1xm2z(8sfk#{53qkjbGFzXQt~8W%spm_M%+*}$ z$p$YU)jT4A%n#rDIp)rO-X_`m+xk1k?kFarq%b4@J(*N#+byxUu|KWOcHvcUB4Ohe zk0y-a2H-EEE5I4o$q3dd{2l=?mk^job~EoWQcV}voJ4l_`HYcb~m^*-85I+R=}9U4E{t()aatn`>f!BW(n z%y_NSbfar=+;=4+P$P&M-(Pur#Yfwwdr*%&S;@B2@=0q$Vhgxg zWsM^MtBzwZ3mI`2x6y2oy}{_j5S9j%_T|{Xop786oKOXTEcr@D8Yt4`gt#m3Bf@{voeBq^L(;F{}9bZ8-E?Y zD6F}agTi;B;mLw{hAC{0nE9GF9D@!4W?AF#LMK6Ic-n8vqccVO8!LqDCx~EAW3>|L zoZau~WhN;DHMtbGeb7J?4v%&_zv9bnv6rHq(mK%fZ9^Nvb=+4}FLXU%%zo7CK&K8d zB-PLNV6Zyg1@Dg!6Q-=r-S*1GGui(UVQ`LW=1P}5W2Kt*4^8j*xj+AtH^2F0=LKK( z`0uHxk;3sNb1B$pj^zy#qt&V-Tn3z?E}oLZjp2IGe9v|lXJ#E0|CbduMUDytmVRbH z^8CnbkI+}*h0lv^!vu`!yzRITxUsdK5-7W4B4c=_qw?qe_89c5JthBtgc`=bHD36< zH1=*ecC5ouGXW$kr~&Me?x~v@W~%BJ6@Q`@&ffkZlK)~xY!@sTxf%Smy)xTYwK=i} z)2FcuQhxorwSPpk;6DIW4%7M-Z|waUJlpLgkmX%n{eB)PIRBepg=~O(g6mWsF*Gxf zIc-SE?DV0!A7f9QM}T0<9?y>+{?{#|ehd2MqmIDW+Uk#W88?>?$vTrX?3JWLuQ4K^ z3Hj3-31M|^VS*KGlaGr2>k5qm1r`Yp3z}ff`Xp0jqL{}e3oW&tp3$e{2hryNUV=o> z^6~>jo1GF63n%S)f3arx$v-8gKX5_gx;KaC#+xkVibIS7rS)!_1^CCe3Ej@#Md%dB z+(dIp<$ejPxADSgSHbTpk-)-6woharoCrR@bQ6MC1*mBK$-);I-F%RyDrlq}P+?8I zjW;8K1KG?fiF`}#ac6PO^5=)}nTSEZcr?T4kynUVte>kYWb6NSwZ*?3(tof0_f&tX v692zSjsENC|8?~LI{N?P(JgK9mA{$%>5rn*YHv+92|l{ohFaxUZr=Mp`xuIy literal 0 HcmV?d00001 diff --git a/docs/img/structured-streaming-model.png b/docs/img/structured-streaming-model.png new file mode 100644 index 0000000000000000000000000000000000000000..2061aae1ada413bd1f2059ff3cd470eee13f3196 GIT binary patch literal 66098 zcmeFZc{r5q8vx3L$kvX^Qei4Ps$PKfLzO9+V}dy}v2qtr}xgR+i2`##erG+AS; zV~{lllV$9~c}J1@zRq=>>zs3)KhE!uu3j_m^W6LL-1q&=vuir4%=?b*qoSf>R#&^C zPeny{o{DNu2R$9|3Hb0wBj7)34}H~3R5{HjCV)Q}?y8x1P*JfRr2M0HNN59=C{d|j zQNDhkdJ4aH-D})m@!R;bkNppg4;&D9cj&ay!((r}o~IsG$vW4eZ+7@(Mv-Q=Vu!v& z=2gegLbr-ncL!#uog-vZ(dt(p7)FQz>oJG zzj%g*em{@W11f6J{~rFwg#T&6|E%DDR`5S7prLOxwI+C46HUA?t?f1m<0H`LnfXbJ z@NF%zo}l51mG}Ml(hp)&>^HI>7s!X3ik0K{nct7Y{e>>2E!KS$%jfXC!iIEUhp$n4 zp-tm&4z2!-JJ$42gZw7RiaXxpjRU8BG$TigrbX>Ci=M}H-JHy0hU(E4tJ9WfbG?!i zT_KLdKfCnb5sQ)MnzwqK95iqyO2!RQn&bm2%{ZYpn490Zg5eWd`KV-O`fMBX2WZf~ z(nGhXS!09`ZDZD`mB`gIUmIPNa}#v@VkMY<69<~Di*SVhJ(d_3CB+^bfvB)2yQ zX0FJwIIuB%H$8AcAZf9{ij0p4>zyg|FDC^ZocGpDk<@&=(!<9(P@yv>H6pU3?;Z3Gpc% z*TtrYl94{?SS|Y{GJ3;Sf6p=er$iP~?O26ZrY+}uS9?oCO1}uN2uI$=al5LyepJmW z!LbA~AE>Yi54Y+pR}PzSwlyIU6S)XX`RQ79vj}#GRstd}{(_=7slY72-$s7}|0yqC zJ3zwP-B{nHqfCE#lYd2juJp>upGvTcq52G2-us6qde(^iq(^veHK)5@0@W?(6-6C2 zvb}H4r#2A2LqW(|z^#KU6dP?Nw9dF6xB31^aWiwfK@nA}j&XRBHZkFY}=w9h?##avl%iEeys9HJ6DasBFS z*6O1A3!TaFxknfaSSH_+nq@e4Wsw_`=Y7!5;L(oa?S7c&`!?n|J_7MX3hN}imQzG3 zhq|(9tufp5N#i=f7*-vb@YU-pCYX4{QTu*CP_(nXL|xtr-s#bM(X*)|3%2!(k9-uL z)-V@X&a0E_z0z;JG^tti*eoh0cBQ?N2~!lbwb~niR+V5tV2DHX7m4#=nbW~Hh=VN8tK?&2EKaRdBE zEEQJ57XOy5%%u*Uyo~2)v%CtqotsN0r<46v&t;MQt^I19i6x&1+-|ZuzG-aRpQtVf0u4Bkpl{Wp(2+Jy0Fm-is*_h@)dNH>(IOTu**UXVebjpL1M-^9-&_<;$_x z%Q}YGG%VHCa87uj_ltF}+VdOdUj$amlUE{bmqt`w0y5t}UtL)-!L7R&;*9LNO57S@ zC=C`3g}&m>%X8H;bt_u9brV#8p3Jp@(OK9?wsY?2dTKIVeQ!g0d{ID$>?ib1zl2Kg zHnzr>sDs%x?_Zlf_(Sg2?`bDN-z=Tiz9B0uWs$mfL@r6oA+kcSq^EK9U^aejq;4@v z0jWEODvj=0l=-x1mX>n}hH}j;=)xWH6N+oicVC#_Ghr6+q~<+F(`?n}krwxB?2>+A zB!IA`L&=iFx5bH(7TBV4=&`7^5?d~je(vZ%Jb#({J+-;y5u|!Ry;=abTC}9y^%fU> zy)E=yPY6sEjr~9qKY4LM=qh%itof) zxz&H3@%5i{SXv*^=vX!_)3YQv+(Oycv0I{E$|#6E_LVVoA8pK1e%l+?!fVE-%>fws zYV+-L^E>LPlo*S)#~+Q--PTtn>rWs_Ex+75j`Im(Nqun@ z8-PdXg-K-)4=ii7WryaIOAIDlF~azB3GkKS$#Go1;;0Wig6w-cmEe%tj*uO8DoH7e zhjV{y6NgV47;wFQHVn(XAU?KEa+nwphpF=K<;}KARmd3k)*sNpoTF8RtTX3CzAfB7Q*CDU4Wr(QA2u~!HM=C6rJbEr|CQR67y=yg3ozSFzu z%0Hbln z_7_8YTT<~3c0(VMh4Z^SnVlBSB^6g#Cafw%9y3PPt9G<_lKrhDIUJ(vc&?I`{u*I+ z;pFa^A4cZVsTR9rcA5RmfKZ3NNg)UHqPFv^{ZX%!LMojGt-HlE6wPq! zSNi=G%FgsuL6jpW^m;izq*e9owY^FX?Rs$4Vu3rQ@6>T_Y{aekjRu!_Cxifh@auB3 z5k|}krU53)*MyLhP$Si2&Z^@jK06LNA8xt6f+I4WO>hG*>Bb~<$%~QmM*J`g7==G5 zBa-1xcm$A}*xXMMu6sdvCsuIm!$;ve%U#LBtDQ=h3f!i1QJ8#hK3VXz9dGJ2yTyrW^EOMmGBqo!59Z|D@=2SOGj`p9#l=2%`>$Im^vD#nEaXkI z47K+!jEWDBjJ!=QQyQplP9!S6I^t}#7bCwn`vRe3@IwOMfR?b9)62;;byC?2(ihC4 zdWl$;gMk%uvcC+n*m#VX(^c`G{4f)ZUOPbICUldvOnO z{a%Ip^+Kciy(6x?0ugO(*DE7T}erP*z?4Kz8nJ){V7(xpD{W^xiw2P6kKQsk6!p z20t6P-thQI&o0-?Bl;IQz9*IIP&<=NrQcJTF?V>7TH1y;CugQ^Y~=f*&SD^?f?ROv zIJ-|y&ZWa!VZrxA`{9bDvpYOAST?@_iHU`j52V_D5w&J*%Y?!Ry%M+VhJ`ycIg$2m(O902 z3fSu0;$GfABPr2VvUT zu@S)eT+eL0Hdy3XvBIHvv3+l2o=6P;4wiX`L5Pjv!o#xm;2kQ)u3FD8jEG}gx_qw> z>~Sv`VRI%%&W<|uAXhI0GWTN_95L<~Z}_psbVhs?aKu{5z~y5Yu$#F#h&V)?Oqo4a zYbSpF@-TexzWO;Q1w^27s+)Mu-yN5K`F6L`up1>-A0HY@=J=2)b-A6uPvStZohwch0xuZ@h8|+PzZyZ4Z4W%N)OD}L{*n2ZV?ut z|1WZOmLEE>Z0>}AquWGE!E5dKg6L)Q)ca{t&7%*pE!1<9{)s zk86$&vT5skG^-fhFQ@l^LdTK&UQTXwXS@ilZF@~465=QDvRSu|{-^7s>|#9yBI-PE z;;Bn0?ATxvYH&i7J74}!zqBO`EtNr>V+zNI`_7snk}V5vrV*so=Fo^Ht%JJ-|BKwN zvUQ6Y76#s2V(rT4JDCeO*7who#Hwn0#YBE*(GHf-XN-Lpgf8qk(nNrW5iNw~TRtuP zu}}l-5*354Y10?K|G6@-*rh)>wevXh&(!{R%sOc{gmaq)Rnod7U;f5>_`z;}{Ue!? z+XPsjrE$gH5)p3BXM}h0n}KmzRh^!MHLJ0J?D^*8&KrN0!)eC2*T-b=(DNTF+Y6?} z;nP=V9_{<{>q4LkTi=)O|6YUyzM77Q-x<)4RR8IPf8!Inn}$VAdwS2|s{Vv%QbNL% zR54fj(`}spW#X;MgX+;;&aH1vyo@^iZsr?2k!k#m?p**vEe4+>-*KdWUGUY3;EurF zN;~{#M6fi~FC%1_E#qQWo`5TKvcbpxym1!8oknN8<7b(t?w0*1xiXu#e>N`rQ+Klq zdjZdJu02;RM-Co;`}Lp6J9U?+wJGBRxfdR>!oL%!{>5wYBc(p3s9r6~~ z+h&(IV5IWP%J({=m#*xk-_Ncx7}0Z|Xj?xbi#d-`2l7C)|GqonuAEU*Wt~;TQ4#?& z;Y_UhH|#6c1-6?Ja(wMGALt)mvR0;4Ym_d2A&`t`61g2Dvk6k4j<;QI)XV%{jltT@r+kRHfA5cLJ z52x#T-atCc3eeCCK2}2Hf<><0Ai=g4XjuX^=8@+`yEPbU!q9ht87P`IXUQa5fM`1T z^87k;h>gob<;D48TVB}MhEj3xzdLD~bi~AvAIt#O z)g0tadifokDge$iu4s3-hdMHvpJndksY^@4=PlCjH;zPmnXCbIzVk(9_qs+R|4Zon zcS|aBML4TeaQ;B@X)FgVNoz>ss}Qv8B1NIw7_r2jTt3K)P9pwT;eDNagKb|T1e*D9 z`@N|8E++`(J5YLgaab4b07e7^TOHZHvkEIr9o~sp4muQY^ zCejHhyu{+!{Dh3UND)*}`aU_Ns%BL%-7dy{S4(s%i0=GFiStj;rx+>)5mh{8^QR!D zRt)<~L(wOFZVbTn_8j{ck`I(Z0F5}x1gANwHK;e?X0hx)^pIrc4XnySbNQH!&m9t$ z#mWBLcWe+_i}1&h&K6B|b{C2W3@3-Mbz5UOfDKYKmt=r&9!?cwgT0sW}#fwi0} zl(wa}3lND?H%msVfEYJ+d9ZV$E{I>1pH*8}2k=v4x`T5QZF^ur#}l2G%+1CH_GCF9 z+1_fKXXv(lSS1yC55JcuiVF96g`$V{VIw!JWW!QkzvQD9yln(iom4Xpyg$Y-DfV;B3w>6gFg^5rv(f2kWXW=f*GwhEl zMME9m%zQ|n|7M7J0ylu4s{pfV#3#hX^@&M~|B{cj>RA&Svvn`LfaxUUWr*PYeeRKW zIulF+-$Z)uzN|<$?JBF!d*1Vc3i^)0TtcEsT3Qx4KVO+VQ+Yep>gB|$;?rL&C>{j1 zb0h?SQfsoI@(0rrwb5*TCD8l8 zt56AOsU*3z6ZRVxzfxr$h;;uf5=f?e)3B_{-W5f+#8z0mM$w=oga}hU?Quw z3(f*#Y#bzP9d*6S(Vty`>a4r+gwbINWs#4;}e*+VJ0;&lF*D>ac zNYKC24GYMCG`pE>q-!O>u{{5-888=XTOY2j(b-$1W{v z0h(M~A+_Rqudmm$$xk)kW!qo<8<+>b=mb!7J=4wa<9N?&ZozgfSp)CrK0Y#%OvgSG zeceYxf6wt;^_=(%qjb^B8v2mxyy%``2ud*XJ*?bdgaIS0Zs2q`X;pvExqAKz{y!6> z3TkKnAa!cZEbL*qP9!Ih^tRcKm@Xl%TxmpS^!6kB|MR2@&9WWE)7#&h|0CsJ0hS!w?j)~@BnhTP$4|%o{LXF*j1oK za$^PC`wzY%tMJCyrrlieNV_^}4wjW?I0q&h6@HJG6gBy&QKEo@eXLi&3`jZk^tV{* z7->5fgmPy#u{HhPb4{y@EA!RS23G6j2<-^Zcy~|xuE@}?!6(01q8`np@O&eNCGI<hQGV)R zj$03}Q#g6l5CHd+Vl#|46iCXglD^Ab$L;>4}2((U@eGREl z^#w4b2G#7t^3>(1mxyd|65b??zkk4DT1Xnr?bZJ{iaIdKUC?c65wG)(8_Hp zz#1*RL1ssS9_A*d8B+jbgzKcx$u4U4w>y4$*erX29Jujt(V}FRCUob$;b>gGC(5F& zK({5mqh@FXdq}|gRxlktS}Jv<)7u7%{?HC<;R}k#2GJDjENaZaD}`5d=04em?1Yx@ zE|cLxFh@0&fBcKTtAP)!>47Zna$?V40Fj!)-$JGaf?-d*qQP{ai z{x#n5Up?QLULr^1eUhA)P69>hi70g4My7Evg=a?Q)?HJZO>lYmFx-t#Q9I(@i5SG3 z6w*#Ee{p9X!rG8l&dA9ytP;=(YeyP?a9!I84OzgBpwAfMB4QNJzw0&JU9y7CGO|~) zqmajTF@_QW8fk>p8?@~^8JM@mDxgP>+z#B^MaYO~FBr~dsnJqX)LP>@-g$A2LQWPq^UQh`6`Ibc@sr^N!>&;CD!M6`Q z?jHz?Q8=_*1w7R}2AT($Xymzg{L>a*|aBcd+eaI^#G)} zrT6m}o^-PamjQ@iSo82^zSQG)HI7c^ zFYu9cU%>vFL6xP1SGZCt@^KKk#C==>p_# zfP~{0B=hN@K1c7ew9J40ED0jh#?_#>KZxG{yg6Qb>?b=ZkgP{(%;ikVF1tR7)Iv*F z+U~{9WW{YUG;A!LyYQZBKMO^mN>_6z=$q@`u}?TtUcupR{*OxAiB71gg2+}T4kfkMK3 z>DMDWP=9?e_5qAqVDmK9PrQF6vM~*Cdn?qpsCGbJ1Bsx2{L!d| zCVq+j>`ZafG-DRl!1 z5^Y?jvR8lod}nLG?Rnb>F3z_5?dHqt2jdfK(txV`O5wJF@2viR=K)OxI==(M)NkY8q@(Z(eFi4 z=vn$@c{y5JZ3+=+n|*hRqwt;1yNTXSopleOE`nZ8s%$9x*qT&@fVGHFi;aLZ4&aE z0U7H-T?z)$cw464ZARLfz+E#t7QtY{1WKik;lj9E6`=Huk-*w{h}^>C-xfz*hNh+I zLfq%8FkYOiT?EkWY%5%cOc4I2SpiV=s#41JxW8D{z6V6J&b281FXR1s@>&8&UINL8 zsORBjwooO2Y z{3t9rlw^4R7mERMZYEH60w|ji!N%8OihH-nm?(jYketUqdPg_<2+c$@*+;6nDE^>} z7kZwNY1QxktzMGf{l?{tpEE_@DLr~Qkz|nAM5PiiP}=~%++`U{MBR?)%KT2;8zqmK z&XqEYCxM0Wxzi!e;tL>w)pF9F(?!x&W?i#>Qc}Mk;qul& z(+MCK;g{GY%RV8;>dm*&R=G3(7;hvn`blkqvd3K!X`}#&>=aU*3&M&ZF<#_RW>|Lt z+CyI$-;h0?Jy261&1JefncedB*Am;)l*~ku3w75P#!qlvu`pGKc6xLz^=6HPJwR9< z1pX)^d%LDB1+t1@pe4tDbwdx()*!qBZSTZC&vac`?VW@`?fY*ldI#h-UBDJ8TLUPM z-%A@nHOZl#)Dv^B{^stiscGoQKoK)Y82uhw8-DPs=ZtUsh6b6wyNB<zum`e+wU}xDjxT86!rB(d zLFJCo@cm!CXNoOpvK`0gG}eu~F|>!jdJD^XvwcgP{9X|K=>SC)pq2?B(nQOwbmlW) zD%lrNF};VqdDNYd83w@gMKm`ZyCZV}1*`P`-n=6=)~h6Er=qO@Gp$zpxFB1%)s_1syZ@ni2l2Tm;mocLz7% zxJWZ;kT{?*X?Rk}+MUK!@jaf&cA0l+=!Bv?Un>wN`_%6LuBh~$2TJuLSL_e*hz@|7 zz@@RuObH|L#NiZgnsUHB_G#Do|E67A$-_~jM3O6$_f5g6o7g`!fPm9{6P_I~rRHJh z++Q)>;%?)W!Fk6bADYlEO(6wI_~|!#AOhRn4X@3$2A|4}1hAQw6;^q&+CTug<7C6K4ddrm-#-!EOG!lsdA{ys=)b>_hXhhh zZSBTi+8MjI8&d-Ez7r>CEPmS0k9Z*X4qe?-;wLz}(IY_-SQO>3&M=e_vkk*~hK4JW z%ffxeH{~)=pHM~Dwvuj}M56ua`{eQW|MB-!5uq^F`ZEoXA@v?;x7rpvRFB3I{ybM& zgSMDY$$AVV{J432|JR*ueF36U9kMA0{9lxj3i_FrEBqV_SJg2I$`|NU^z0vu|4K>@ z+j(*ROrD_(>6On^l)?d?JYj0K%U!j0b|89E49-sS%ONKMS04kHVsGHN@h6*uF_0ztHoB8aVB52m)BYPVW5Obnos61r8 z6JuDXAFBcEl6MNA)@Vctr~PA7^uB#l;Q`eu`F0J}tiA{ATJCsc+7cpMaNsTv>Xc&6 zzD?LqwlKyS9lgqUj>ulHi(MP%dO1 zyIEDqg)P>RCnvAo2tdjxyjmCZ@tCVgzB7Z^x=7fnYF(hvAheyS0;H&cj<@%uffaSD zVU_a8kh4VXoQSvtPXQ9`iST_p@&w0VK)cZrdG4ghCEXB)>)b{b?aQuy4uzO z?pLd_AA$emVM(S*#7%wQzCQX3qWlId)VH4lU4c?N9&k&|85rrj=>x>RHRvPG|N>Hyaa9=FtX>ZYXT07Qv`%_tTIxE;bYTehRykBx& ztn6kLr1K(NP&Q>Q7EIK263y2jk)LDW@EVv73MIZZ2M0YU`t-~A&N3}H8%eE8lOhU3F;kk`0Pyzt4wXX0L87?q8_Z?4aFZSs_Mc==s@HqBIOsq7YEV zpeZMVS$=v^sLH|a^9V9-q!y!F;kmxt=tg(3_lC5Hc#QJ1MY~M|2ZZedq?c{MNAa1= z+Jzp-Y+C+Euth{CkFNaC&{>O!SZ(VVz-cXZDqIM$V48NZomilqf$K?4m3J|AqP_8 zN%Akmp2?_Abk*n2Ho$z3`jjvxCt=;pK*9S;*=5qPsf03iN^ELB&@J0kY0sqHVF6vz zX3?5__ql$2%hfIUG3SMBb<008FGg1e4yr+Fb)O) zi({1`Z<1Dee90gZZO08Y9rMjp6wZdA#ThRrg(Lz!wfBLB_sr*7ZKGsq!FmqmVz2bW z-ev_ke0Liv=;7pYbd>hCD&<`R`K6YCUvv+zx;_=*=hhI`m;`-tfjRSn!8NbdvC>+K zt1j*SpVLFmkdp-ylF_BYkE>-6>o@OhCUdpVMI|>6JMh=ME@zjSk$Eyv&w~RndRK?#2UKx>%?q?(;`Byv#&F;S*)G>#MZTq`oz$$*)yk0`T%FG3jN^cS>k%-{DX-5{}v05h(G{kWwm}d{qL}cegwtboAgdYD&k#ymWeh1tj>6 z$7phOpRasM+2FkumgCH34o@-yB{PNJURSILcV)X#o;@J5`Oaq^)5(#_(5Qy4y~cLg zVIe84=Ku{$OEGErgPAw-R!d`U#Ua`7V@kO|uRlD~b}Yfm^ra7E@0fPE;PK*S;SQH& zpQCZnSp2E!6t3VUE6n0FqiKH^r#?ah-veCTNPw?;hi-L zjJn~abu|hh)wGlFdTy=waBLc%Y#lIW#_FO|LP2lz0j6(F4*9}ZmiNeeVc7k zw`f#6;~1#hv6;jL!&)4u2ZsHF?8o-NdUMLI8dipUddK7%n`H)t17_YE^9kY=!oG4SJ9l6@ zrwd~MvqWXOMvV@rnTi~AiC==eN{?UKoEiac{fRmQiI@cwg4B|@_~eijZ)}w)a-amI z_CmvnYUqGEK4q4uo~FNtMMSMX61!@p75KVOR+`7hr44yqmPDUQ_-xdPjPUx#$ z`ELz9Ht66a$ddfzGVlRfjvlM>sGR* z!U~J!$A1cPc;5iOmrl-NJmUv;;58fv`hVz3`V+7h*p}NC$(1}Sf%mFLg5W^z7No!D z?sN+2pV2blW*F0w#-za@K>LXykC(=8mvd(s4+-vze|oU3(R+(8y&f! z9)1Ysz2W85u4oA}*?ir1Mp^_pP{YP&IJtQo5s2y-U?#0)HmArq1OX`oIhD!4p}-}r zYgKmiy*jx+o4jUp?_S&MiVKH0Eu9nMBiCK-!EGCo7iYV^PYLY-Sp0Qsn(eFb-2uT> zrMcH5hWHz17`}RkT*X+qdwfg%I+TN|1^vKXOpg_Z)pJJk_Bzf*XJ5rhl!3Rp3gk_q zLX%056|;UuQvfWJUlp7N+>e;fZ+i|eKclxdpLA*{50V;_W0xt24D2LC0zEv-Q5Aq~ z<{P(NQp)|3R@bVM{a=5fjYDxx)!pZ^67J|~ab2rzWy!X*H$rS8gdbblr?9w{TQFrC zKFKb)ERKrpn6~*!G`bz#E!PhDT)aq#O75>tDPH}} zKxxl94n$(h_}+GYe4yJa>iHdDavOS@B1;aw+O;%@2xgN;Z7HRbUjO9Dp7t-RA!8es`VgJOqKN?hDozIY6Jp zx|y~(dVXW8Df{b-z^KsuIP61iB|KQkT9_s$+R|LeG2r#bqdrzvIrkfEOOm;?zdavO zgbnakX{4=A<~+%8LyhdiNWzm~zKg3S>L~pdE!TKNzZ-rPf6Xe&N8_G3?o&~MEN=zB z#Qwn1kwtZ9q)0)wL*Xxl1!F^~Z}P|zJaBW<%#eEx$ONEJAd>tzS$Q5=E)evw*Wbft z$fD67gB?_n_pYq|YxbmicrG8SR^TXWp(H13W_s7YlMwc!6YFTYvdq<8yUxV}uhU z0St9~nN-Ki&(4jEVd`OlL7SGsgg*Y2OLK2WBrwykj5NB_P7g-HV8HZ7g=~DhR#M;j z=Y6Nj2&T6r?&}lHJeG>=EVm@9Kx6sJRnAeN{4E;|J=57%b0!y*rq{>GJKHT8;}gL7 zQ?U76n*q@Ju^J4X7$U)Bc}eMMo%-Ioaze-oiH%cbZu-HTRiDNSzHPz+;J0u_>n8mu z5oEo3z$t0DmHYYx-TV_(Ml0c+mbwkjB<2683y69_0~2Vt{DRL>=sw6e6(3OH(x&j> zb9qB2|m~NN77Qr>Nb5VOcPuZntKs{GE^x5Z(C@ScY z!}o75t|Tk01^7p7oG@zrP01Pi(gDIdg>N((!6@~Suk``e!*T{ z_b2uE{?3WD3fQs2(pz(ckq09mN3_++_&58FFGAfDRLTBz`2M4g-;MwUgfxEA@_;-{ z#(`j7ef2x+P#OUC>HAF-U)Tr&ZYR96n9L@A>CSHuD*#L?PY{H2P@Oqni-^AN=iQ=U zu84$AtkRTRvhJX9jptN={4Fcryg2zeo?AmlD<>*0@wu4Mw$u3o>Fpd79k82ThhmN3ub zAEcxkW?o{!q0PFKFeI3SE;i^#suwK{MUd|YH596YGEa#jJu+hlq;M5H&nqQYDn^Ei z>4Kt`bk!B&PDHGW>Cp>@pru-pl(hgJ9BR8QA5%iE%$7xR#m2$e^=Je!w$33LT(7+5{^Br$La_D>eRmGR&g(+4B#Qm(Lhjm5mvu1oX z8y3a=i(|Fwiu8>YXy4{+Q}4~>nS1QTS!q>)E?878i*B#TUl(x!GKY@C%W)bf^Rz$& z>9Dcs%Ix85VXr2=`a2Of71f>gU#6HZl!G4ViPL?Jp_uCfP=!O48_A?ZxUz~zS`s;) z<}Fa4EmRt0NifO9e?sIBy5Nr%H|FZ{BBG}{#aeg)6R_vI8**VCYFl5#QW#ji;t|~? zBW1=tv=W=l#i}->8hB>!ltytc?PatAr^QLmXrapJu zec!g%GGPRNR}E%RrF5Cy9C^eE;|62n&()q}=?F}*@~cmKD6o)e>*8ZEvfe{Q=sc|u z6n+5+^m)$==^KlbQ1iDrnsE)|^ybpY3f$IoWE2!|9UO_ylr!s|ed`XLe%f=DBXKM^ ztYWg_>pos2b*SZLdd{YK<%N~`5l-ON-bJp6m0ZmYL7!V~x&C|tCthAxq+8MTL#+x@ zCb;rIRxOW76CaP6FJIAT6OYwrV*>u2REWUxxLfScM%7PVa}{VIxz? zbZ6WWuqcn2xIU2y$(NsVt5%mf5_p4O2Q_Ns9<#1JZi|?P<+9}E7}5(q1jtVBd7`4= z#vP835?Zk-5Vzb4O68O)C?T6tdd*TD3^!RhfY}4@0$Zt7M^C}AL2cX!?C_iT7;Mmx zU-)WEK(@zieKy*7I98?9iceB2KB|wuEa={qx%^}+q^|Ua^4x54JkdM=$5&=HSDZ|; zx!h0bnny(`F<-~xZ#F$})Al~6`>9bLMuv3)cU+c&USVnfmgFNX-3fSGVaAbMMS$SC zfx>_Rni5@8qJJ$GB^1&;^Kr^SkQ7n%;ao@1=!n(V<((`^Z#*bOhaEGftjg?Cn`w;DbNX!S$?KYPAgH6HZO4= z;6sMEOKMMYcM;B#p4AF~C9L%l&d5lW-P8|0sS4Ic`h+?5FWeW3%mjPuW+W`>FxLljixsj$lkd9Ai^2p%pc1~rjsi-EV+CX3)a~y#M=CK zO|iu7e%UP6%Z>Szki~7woMuy0i6OCKLOjH(w=l zxszHfI?u-f1EhjOKDa=+$(azDC!1pDpMK66mIpj=Di4sY@B0`-$qlx0lBhCtNqaDX zr8Q*rgJ-s^2JlNUL7XWd+(DjJ!^Gwm8C}Ir$a77UU)Ib&>?HFXRlVnPQW-A!0nE2M zmr~_V1A#sReF{;6f1xK2%b&q!5xeHT1iNaCQF4V#pjhp6%MOa1e*h~+(>gO-R$s2M zYN!-~twD@C8#AW4BCsC0d(T$&9=G)aiao6+iOtzERf)a&>PrTYK`1Qy6!%}U3nfz_sDGb zG0*Js@$VRUMM}NgBRy~VQchC3q-IX1z_+N&ksLTbNqn`+66*UJTLn|*Ru4Bk0%8FXYcyJ8DA|P! zO@9o8RZYjZ{-P13Aq&;yp8dA0B*!l;EG>hCK0Zd1u0LSt{v=sgAL}vWY$9);A)M29 z7Ri{n8O~%n`RX7b=fe0-QidCeFI{`mppT^TwrS0`%K7bnV|!?M$QJ{3`SQ^ab-X%h zB>60{=?e2bIbP&^N%Ob?tcBN2eI~;EHIwOTk>qXpqW<^l%*iv+adFsN*%_OP#FxkI z;x7UHf>ng-ynlb^W`!#pJ0P8)9T?@WfMmVuuejo_zZtk}_#>^QnLH%4Gg1L0NDMC& zNiP*7D=Hf%n;0dL3k*cEsMbmMQP&Dmd|4C{NEVx3TmA7Wwvr?;Gl;is-1$YB9qat@ zV1Et}cIx`EKfX}T!2aPY|1E_d*lp|bPo*{hAc&84N&V@CR&C%)lq3gmndW!c-yI?} znld@L>rNs-*Dr5ET>_6rHX+8k>wHP2@bQPocx!+*_Kvnz*a9%SA1P=OtcmYo-62s5$x@Yq%zW_-pheppi z|Htix@807pf1XbX?N1D}bghAL0PWoKLcflzLTL}&1=l@1aR&dx@kAI621R(aJRW%0 z!iqu){UQeIXyCy0PD{0&GBi%1Eac4H3h9XjJ|S{2K?bu{0F`7R`?08uZry$k)} z^>;I1>#eA0il~2#+Wr@oU-^LF(u#ax_XiRv_)@Gs-vVH+3Do(Nb|Ea=GmF9n{@ISBcqhR+xRrtT73U?F= zBDURoRx+9|(tK_>yl+-mw#t3D&e_w%_Sb>;{lKjV@Bi=bRRB@x(>-u4A zIy+W3>6vm8UC))1=adgd(p+)BdB9fhXb;DIH*1n-PD zTzAIXxTXqz{!;MuYu1Sm1KNHJBw8P2WMXE<<0on=*5?}15&{-`3Q+S1V&gNKx74y)oLW zme!VczfMUhj^R_V~L4w_$!TY5K!4o>mbMjGcThmD53h zmiPRz+Zr|(l-_OqCQbP{o@bv8f273FjYD6(4kqz)k)!U-*0QxMk4d7W$NdJ@`lnXr z$92taJ(n=58ak4Df9mP!7smGnp?if97hO!VeQ{iT=XK+yKP`Wc)H<6WY0D#N4llRsjkfFNG;n5p8=F+3}iW{?y9?jAYi;vhWOJ&}?>-QyO zH%4>#T>i#ZIY`^^H_2=Tpd&nHQ6(`K*kyHfL3Y`FpPJ2u2k;NKAJ`nAq(7d90N~8+ z6A~GGt(N!dtYDUDkrRpOxm(a0&ZAjzbG>J2AZW9&=_2kUC6k zw{ayw(?;uh>mty7V^p(Q*5eWNq6W~nhp{tXqYP0{L303@p5Z@^afzEW8Sx*|5wGkT zBD68l97LZMzs`Vf@AoE@=3M7JA5k^!;^w!Ip7g@B?e8@QH5=>)`GY*7%D|Qkpx#Om ziWuU&cC#nds|d1lGC9Y87Z2%T<_cP$CEUr!@VA6ssr8L@EuZ)#5;z;e7`TQT=wIq- z3G`C#2MqlOxpP%F z5vLw1ORV(hMUio+rf1P?&V7(Bi{JwLCQ}F?zVWP(+O?^Imi%{due5AGTs+?AKi?EQ zKi;xAY+Rh(5&j`v{XVaf^+Uj*4$&0)SW)y)YI?Vll!CABfTQ=kWIfTAV$du%-`GPD zU00Z&$({9Qy8bqZ(z8pgXVc>H2LYEs9yW166j^66gibAC{}1-QGpfm@`}YVaA}R)u zBGp7drGxYmREjhaP{{P~hA1 z>Zz;>4@?L%iD*dSm9>uNU$RYuL{?0smF|C}c@N5qeq}-qAH04N1L7BQ`}O)R*Aqfl zQH3kLX80qvipp3PE8zL?PQTjsikH{OrX|uE$3sl>@E9%OC;Rdxhgy&ua~>nJ%dgB`ejCP zClL8N*Wr|psBwRM-dciVBalYvOy_@ww_ESt9PN|1-`xa=o%7F76GL`$b$jHhOeU|3 zv4k(w!EpV^*&QB)OMB+i(EmP$BslH$PAW9}<*x5E=-C*SyWiYPKXDtrz0`B-R{~)L z%peIEq0H`e?LOh95#;6bPZ|UH(a37LtzrBwA6eOex#m8GKBa))mvhdf49MFl`5A0~ z0PyS7XO$09UgYZyxvy#$w}@_W^OC5W^pV&WPf%x#@ z!I#83Dw(~Nzub?SaokfG)UBONf_(F21?}< zYU1iaqbz_2%*)rmF#~u3re1B|K_(}N*b`F#MFa>G^xHNLTNDiNvg!b!b>;Qeq-sTUK|Nzk zhUvpf*+{k*Tz20+Nv8kdOXy_*%ZNeD`~XAeCdw;Ioflu1Z_ili8gI@N46P-H-qslJ z8`@ck1zI?$9k9OFJx9Pb!@lR4FE}iG;%v~ioe(nvT32K0fz|%;dV-f(i1z(%^F5UZO{_#j z_ptT4Y;Aj9WckvoAa=EPdc7j6{Uh$}PJdY(>CDMUKOBT181C6`dzYGDITooaD&oSw z6zEuLfN^pyZoe(W+G3Ga^ewtS@@Q(x!#e<~cy&kb*W?N^TQ7f6ZRs{mf2-2`Ldemi zl+_B$#ddv~mPLlvrJL`LQiVelPz6)e z*opK%8esNb27E55RkH7uDrSBMr<-C z3oWA99J$}5qra{gvB8!@{=NhBD%-n}K%q?ji3BfNp|Agtbd~|#Q0u1Pw_3hhHl=0% zn;#BbWE9yJ&^vP;&QklQ|&~VErW4DTK$J)qZ{Z>#A zd}l4}mR80xKreqcsg_ONlMNJE>1Kd^DfqqT#L06eK|q|kYR!cIm~$$!vcJgAyBr`7 z5emL_TK?fjxV-#Z^HuRSZ4%^7b>b54MY}tjDLb1LyPMN9`F)lnLGIHAQ^Ko7E!sPI zS?(6s-9@sFs@dj~{+o=EFTTg^cXUP6Gjd$J_Rx_p@O!WnE#=z0H@vhH%Z+(lFF6fr z{9nE;7<8#V!`IK@9;xl1?odnPD5v>O??di%pWswR&Lq(OZfk2JLAt{Brmhu>oE#kQ z)cJ%jwa({rlMF!yE#FPQ%#6D!#RE}@JjX2i51WYSf26R- zc;d~Y-DkgI^#K|Ej%2G+Ow9uf1o^@;=SziKp^Z^I)1!et}>IGx?W=~`MV(fSZ zKTwI0=DBT>G)2J$mn3IC{cqsX@0hyy1u8YV@RH=0X5`Mf33b5eaU%ADgA(B3#M-YY zD|dC{q_)VlL&SVUmmb3@V*z~HXsj@FgzN`QSb2%;8R|KTsq5wMuK9z%|Ao*2Uzixn zNtrt$M-KQHYYW~+8`zob%aeG*e(S0;UQF?B3mR|r__vqZ$#eDLE7y-Oei->P)Mcwz zj6ViyHNf#)tD<`yXH!FI=3mg4W89*X*DN@7S|a1ZoG7+*>)Xv(={Ut|<=oh>tRS-dlCNUDDJURo*;ez+>C@&a znMQqPXU=O-j7Q^-{g9R^VPso1YuI6!ItyQT8x@ z>Bp&*!$I8Ex&7w6)*i0*TxyfT$Lbk;jqHjI_rln_Wymf-FXaUOsZT@BBjUO0Lj>Sc zFXdTVrJoiZQ1?I4<2`cz*}@UpGGN&o>)18%uvMuqhrar~w*$v_yi1&n16S@e}-bVgDgr4omTl#W=ixZdkiOPHO zQw*Qgq%_lu+~%e4?cNSOhUC)zycB&5zj4z!3TYrZFVRfx|E!!ut@z>hus+KYi{5&s z_|k8*KxcmiYLI&kYqEUB3c-!DRmYAXV>ZIm4n^)w2cPybT7Q2P+3YkhrBk5Ob0UIO zllPtV*N;r)quHGrW^Y|~fdw%SFj{%F;p$6lv#6h23{njQ!BOv2@7>4DcQ_I1IYwtJ zuo|(!ftihp#}F`?MiCbJ^jnt9s8HK?D9UzfJ8E-tG~m2Ba$Gc6I9k!$JqHR@PkisR zNM*DdGvl8)!JA%pkvx=V5T-JkJaGcSd6J^api(Kk^4*VPAVYr~ft8G93%{I!dDIwj zaWCP&iygd4z#o%hr9Tr2_M!0s7JPG&-bXQDF#)9c9(?rsgV+neFEq#_Tl`ZFi1FZ^ zNN+n+cHqX$l^O;vAL_!T&$7llSX}6_yS=L53Xb7xHtphvdO1zJnnhn!G~ad`#aO7+ zI`ml-*aq-)ULK?`33SX@%rF))tH%0zs^ytAj>;^#TY7poH2e1Y&Vbo+ zNFr|i)h|*bUvqD}cq)@>s}f)I9oBurFXX^6=^BUKG8*U6;2@8AFFLn}eysUw0vbud zmED&~E=FqE)^D{<2IhL`>4E-nosQPpoCxkWs`#6)uHfJ`5KvT&#dbjNkgADTJ|)vu z*Xmqt!S5U_GaB9&lm7_Aa?8Vem7cc_k@N%{;lRWRSRwxb-A1c-=T2uqIgjy|H}`Nk z`}EhbpSv}(7H9=?Ghs1N_#P~9oh>rf=%6XxUg^W11Fnt-K%e$}+MF%1>cyUxvqIDF zs_Ay-z3F-YwPF~#R{)+}Hi)Rrr2<-T$0ru}wR|ReK)cY#V5_i-l>zI|5tIJa>0qmL zI7Rwa83aIt7Au;A3lT+)k?eHcHnlX7)pGt)gKIT>qP+H#?N+$?jcg?u$o$C8-XK{y z`Sp#i3eoYL`0Z&8<5-)TXsn}Ty72TD-0b*_uGG`wLIDw)M+30Aw}!uawiTGvIvjwl ziR4Ow@mTJu=LUgMLRbi4*COJ$v)rW%p93yDljoHt9a&M_#5|^I#%NWiX#%; zSsTG+cmFQiRWX%Wa-m$eA&Q!_BQ`ye zO`j>O7W0q5JiJrD)Hr?`C@d`JBL+#4F*aFcK(M66Z*-)Pm2Z5>qIJYca$1TyxxSPF zFXNbX=G+cbzO9~9ILoZN-f)3gC4-{ds$yH?vx?2o5$paKj>0M7G0i6mY%2#L;Z|CN z%22m!&wE2^U?oDaJmXl#vGxCIz<@CT`)zE#g74?f*$f1bvV=s65 zRz(CCUY@Ta(&cJ%=jq+S`VqH=y@+m?e2X@FJp`;PWw0XE%-u>{BDw7eDfq^`I0;2S zZ_dHw4}^}BlHPoMO56d&aZwmv>9dNDC>sPmugHR=RUqGMC%5AsewH@do;HTZShlbBgY`+Ge(4~T0 z$LVWq;rG>6EPC|@l73aBH!y$t@Gil9VdnG(7$eFerTdnUiyZx&e6V&L+|xsIunYAn zS!He&VCR_-O<*;aW32<$$hVW64g*J;PS8M18tFD^z86Uz2-1u1*1yGM#a)5$^;k9< za%ENQY!# z$XBex+aK$^w;P;)%EzJu-0Gh`nlNO#AMDbPWgv)ga2Y0356nlCsLIyX)&bVOp7}s@ zh=LbzkWNVgtDkbCn%#rb3hO@QY~#%YAAi8Q&8g0wK7Cr-V|{E9YZmAC^9!$3^oVcv zCeHXtUzmev{$t9ACwN)!wBtEV?E{Bm{#jTwGPtS+RlS;d| z#n|&$VM`_gY#?P_jIU~jXP364{Deqz6W@^V%oy<2ZZt-)#w={ld3t)35L3M6ci22o zSo>-(R9)|e=dgkiMeV*ETZ*XkVTxO;D**%PS$f}s}Ah`+4OIYxT7>K>zV2ia?b1<$`slOG?AfK^IfpM zFkf{_vs#ANMshK{78CYat^>e_x%wV!VU26ap_e50G$9EoFkt8tnl^#`oAlv9iT$okx-x}(jJPkdyS29v53;oLgCf3o)V@Gro0t6G@*+Ao*cqg80 zrlR<++veo?KXWBp1%S|MRyX_2|1<(AWZ*5nrtgO_*Xm|0XxZ-9Q2Tb}D4F+o|0+8N zsYr&zm{7VG-o~9_IQlsqvOB9jyQ{`Zl=o`Uz!q}x!rNSbkQPAb&-=7w@H)*O66c|E zr8m<+M;9;XUFG0=H65xVUYc@!Ik?8Uvt`D1vp?v@BQfLS`7CYyw+sqB7Fj3BoemqL z9O$nXz7v8=*Y8=qFrsH_h(o~4*Z$i!{eDY>0a-SXV zflb$fI?`UIJ23%@R9UK0%9^M7R5c@l)NP$Jvsn2EVfRRJ1|SsvMvH^I#*(C=LDhpD zHKrD)$-BlD``G|ZM|bwBihd;r$Uxv13jFpJsmd$xrcG419xQ)Lr2wpd70)+^)3*Pc zJKKT>c{Qh2$*zP=lsd(Lkg*2iGaM2Xsq0e{hR!SyxzR|3DH7cwH3?aI_CBq&>cH+}v>NHaal;WVWSabu6m(&qoz zyWo})uI1?=en9tPW-vq&uVjo!^pm!M#aH$?Jcy(bX^U8+16<2eg3i#RTYbWW*lPxiBoiCTXyo=u9;}X8%y==sSLZj{~Q=1 zhx__`31=C+&ntNAwS#D9;!d7AMd(iL$73rAVo^O5V1~m!U&f@_TTOT+XLg5-z?|ha z=m_zTM5IoV9-lNe0C$}x=L=7PP62ln^m4JkzB?ED9fBa`7ff97G>@uHvV|QDU1QUl z0*t^<&sDSdd#tBf$Ch2@_-uYUhrg7R1XqdS>$GVDdb-()q) z-KF$*1Q13!oS#0X_NRTmh@$}E7u4a|#JLW*NBIym96Y!)-7(^oZ2nm10;*KLoHx0K zCD~*gy9n$K$z^Wrf1)Gyf+&Yn0(gi0xMU0%Oj378jemlcG&LUkAEG#M?O9pC9;tpg zY?JZhE`Khs00-7o!$Fv2<;gn-HZP_-k((-d$$J!@33`J2oAVf)3tyZj zBk)DVy9=>m%UI@K0QVUsIN}8cW_Gqz)^!zWVGG`VJ3vciVo7`N+YUPZh?cajheE(% z@yD+&?9ITjIX>=Z&Df8%(VY9S5d2o2>j`IYtW2v!XF`BvBBZ8~2U8t2`(r_O2oSPK zV#^L_1yWYm*f| z7_`@(z0MOo7M&BgSE8rItq_B-m2JR)9Q{$8<=}c zdpPzV4{>btKA49jvIm@T z|LYhjAX7(p07ZcG z877#Ifdy}_!oBhEtV|GzJAb}z)0;-Et^7NpA0c4D7yr_Ln?A8DT-3rF$;haA3iqe0oK# z*`V*LegTq8QeF3A>nU*Z&imF_nUT(b9J)Ac?opBQYf!FoS4+=uX|DULXn8K$eMWNi z&AhiOU!Z0T@VwuqH@+}Fw#DU~t$<>Mj7x1n^_!6TJ#0E#!stO{iuaef&(^5Ot%@pq}$~VvfOe9m&Z&2$T3|_LoFO zxWO`sVd-`%rBMjuDkcdjJewuiPif9`BQ1ce*#WmyV^(Y?r))9@CX6+XzaDsx>=k|%jv2K9MnHjDiC@7#i% zJQPq_Pb0t7*of}vIrZEk8kv#0-~rW_L`g#mq_!@hA0s6oLxF0{H!ilC9vfMeRs{cm z^A@lh8)C~qZm(dt^VF!f(7Lzv4kAPNo?f%EF1DdRBLC#*Tr>?#lpM2xj1_rz9R-8r zEqr!b@Ix*_;~>Fw9GT?7K99Go=L{j$A&o=S*{-Ow5NCo9{j<%N)q&~M zP`XcnDucx5^ZnMOQN}P{Ruucl`Yl7~cHNHWMJOEHIq{g!$LY4X2ks6Xv^jvE5lr#kbVmUt2`OrN z3c(jIy#$Rs-}uRiv#c;-)|pY{0)x!t8~GwZvilWhX>}B-8wI=^bGZgRlD`tdGJWf8 zVr!$Gt_yocTF?gjg!8~iao&=0c*J*H^N7!#$70M8;yOO9y(-RJo#EHrh8{P0cT?Tn z!UnA9rsCn$%xKQtx-EN@6|^BNQu38H%og4VT?iWOuN{Ll$#|?qxR+$8Kl!+@RRP^} z)J4q^Ha3bj!9OqF%vy#m;O1>vC+zf9^~`a(`k{Epfk|XnOJHcFbb6wKEH?LS~ss`$$sNX45X+!fxmj3)7S~ z%dUVY>${DQKQ>HPsX`Z4*O_3=z7#XNMX8DBC<-dm^&92*NkfgQFr4 zcR7`f0jdk8LmEurv!%yM0x=fPfV5?dQuW45Y;mBgAT54Q(x?i_`!|MDAnRY1lNMg7 zJ-3LwvffnM-!&AT)}PS4oIULbDGFj?52_olPTfrsoVo(k#&e|Dj!qf_$K>kbpxQG! z^G+rX)8z%n1tW-nP?E~RD#nSpC~w9UgMqbu*9YO1(<#+{ACOgkgd$9>ClGR1cicm_ zGie&RDKRTZ@OKrFNcW5A-3~I8L3vgY6EyVMwi25!uEmOunN5Ze4|>hxgLN#;c@7Vz zW{a)c{jIcM26#BiOdmM;@qi||D^JgE|BTLK+u@(U8QQ5#Unh0cZa|&-DS=S^K4cMO zy>^2Nmg!$-zs#^pzm|oB2k~=3cj;TK=q+i2r)>L5?~3W1xMZyLFlU-7U>qM2l~BL1 zjdauY$F;@w2ttsZ8uj-z8e_~zRThQ!ZjLFT_WND#18+lDV-Y?7Na^M&pYH9|$aMRV32>(XdZeZPN<~ps?#a0<0 zPs5`FRt>j!9YB|8gft@ue|=!>SL|M%!eKh&2!&@@-R1iFjPB`J0j^^ZSXA|U4xL!y zU;xpGW{4z9nG)4*BgxmV_X1A(bQ!eeg_mx&AdMwQ@#P0tDkv@8s=n54(W4tI9d7jj zmHwpi4J6RkTP?D}YnAfl?VZ{F_wROSFk5C2J}EaLbUUO=V#%(E@2rkeO1vkIpxrbq zV2mb-ut3uPTEybElt*@wilq+|zLXmS)Y#gM7fl5j@Tefwp3tbzj%BE|%jr=~XI}HJ zq`<1L#{o&j7@@P*VgNL91^(7D$L1vv*P1+aTD)uU0;UI_*X@bBGl(;^P`?B44NHlJLnfw^W}NOJ)*5$YGe z!b0*k!4@ivjklT2~fFlgiKH*z80 zF?{4T1A@N}wZ&_$X^+t+@n%x|Oy4EZLDlmN)q+9_IMuXX(hhHz2 zdL%Y>oEJ%JNtP`uzH{uw(CoH__>N*JG_l;Agm93?MnhcZsX=ZCpTEd8D4c(@_QqIj zQ?PWX|I3fIdof78`v@uhu6)Q$KFod1@=+SxAKhl}St{}E(UPQi(x>W?{ZQdlGp$n#So26A` zZbaATA}u}tP94~;+dS2;CS($XS4`jPn;kn)&Y?etjQu-2N7s*`Rb*i2?0YtV9j}u-ITgU@EOzuT!qcU+d{WGEjdNu{UuvuisjDzQ1m}Y zIm>Iz>o3(aD-zY86pslsqBD_iry_P`OPlc;dVuWeSvvUxpb=;he;K6aypy?s%=4f$ zTO@w$sh%26hxe_VkA;OulwzVzwv5n*VxK4Dw4+ZUNXdjD*iMd; zOB+d+79Y3B@^>GNl=C$ZaEuu-Npky~e2r>c{*3})bi<6f0gZtm_%d2yx15J|r=YOX z4u|F(&}ZVbQ**&g1zYW7{sBXY$Z^oSLIwbKX=e=@(qlLl=hhUb7f0-ahbo&ceKdjy zNrYVy5`Pr~i=)Z5t5>5~&iB0>F;wffwK7Ko(M1+Api#(azdfiAZl9)3=kwur7&TSM zctP0`z3tINM7!X)QKrxl$2Ep>E*hyiKx1pRTlx(eB&9K_#_;)i=njuvawPtEAk0fixVqeF&~&vsJ?>OrB% zvU`DDlsLO-{-fJanWOklpeSo%oA)lWf%D4Swq2%Os>XP_><8iz)5bYO463RWJI2D> zsTQ6Z!u3&}Q(ZvecsyUwg=k8;aR4x6foPGy=fYSUqB8AY)MFOHk>h9B>HKRKVII`M zbG5eAUu2qDHl~oCZ&)cjKHb;e>5spw7FXX`5x_5s-X)75j~69nzYs9ys_*RalEJg3 zLng^3&pxjfvjQtE=Y_I_&>cYk8;I!G!D14bENA->(VRTCk#rlVPN$;U#9s>H_$P7Cj)DDDP zQj0iS{)ZW)K)%+Zmg|ZthXCvqxb|g!akbij@r>zM4CWGi%hdo?ill-qIGTG{7aO9k zLu8@8W1$!z7t~h^AgdBvb|@xjIa#npXQZ6kE(f&F_4+`A-H5$T@s-uc?N~ZYCbF%; z2^)Y7*QhTcJz#&*zwB@K|N(b8ang_IU`Fg$+!IlQ34uMVXb_=RP? z{6P!SjeR8vc^m={sFVG;t$^}|mD4J+METeKQzZno?KkA2ul>7xwZS3AeJ#G?CIBzn zS0i3J*=V7%Aw@G*&{?175KbjZE4XbuJdmFYos8^nF@!iz$NSb9Y{~(6Lt-ZHFg!R%iQG68G-f093ygsR@e?kFWMM zmNIY6xIq|UMZUQvhH8U2N=~20cq8@KolCo~>W88~GPZODjIDdms=^kCW+lgxvgc9A zaq&@a`v4gaGfub~KP$Qk0x+g0aUD_a>;7x(o&lHpte)!z>S&)ZRS=vnRLKn<`q(UP zH*>SasOj6>*7T?#Wc|9)_FR|RQrLnq=Bnr1_*@Ma-NtyN|4QJRJIWoh%)tI6#{%^N z5~X%ov*vTqhwcX+@4{-G-NnlX1@w(j_aH%1mpzF&JgBxVJ$yiItgx_J!wYHg1rlTh zc@F^Oa-xP+w)LG$p$}o(<}JR<>!R#*+yw8S`ne&SEu1lPyOWL4*?K8RYE@N9VPeJC z{&{qCK`%sw(9gcU7~jk`U;TzGc^bLA=*8JTJni`4G*yKHoV;ePU7iJKM}^}HL_hAS zWMq6s=VBN#QUT>^!}ubN7~s@>0M1bmT}!|AVQgLMdkk!Wq^(8LO@GZ^SXgm^Xl^S> zea|w2yF0D5hn!m%+U;VPGRkv(nl#P0jxb9i5;p5IPIODY@BCJ0dqizhv0rqzg`$y@ zykI$2jOq9S?svJrjKZ!&{X@+PQSVb=Bj5hwA={=MtNCooC7quWu) zhdzGB^jiU9T=J92yXL+8n*(3VzH1`<98h1V2d$vhB%@Dd$q4^KpG|y_32)=qm~1xo zrZ8o%m@hu}Sed&}arwy9c<0Fa0G5`m_A2BqEeE=Hfq5~yL)~ci`-ci}-jBQ*PZH!4 zQ}ono0Kw=lF`S>e6@@Ha!3^>AaIjxoeu;bw9c)j&9aS9@ks8(&-ekr%J{?p_dMmqw zEyvp0=5Md@&Fc5<7g*?866NMu{JWimsrrsy{REKaF33RHn2QwxanxRBYuo9aOk`=c zL-Qk(ySD?LI2&{*NX8N?!eyakB8pr+HWMUSJ4>T4Fwdls6#>szQY6mox~sA@iMpkK z=*=61x<=*mbJ`d$CwFO73M>=3 zb_>M}wS8Ju4DqDI1R%}fQ-+Dj<)g1&0LwcA*EdHwW zLvv%~dO)4IC;+$FkMgWsX}x5t-TXA^65#X#rC|K*f%Az-xxeneN7q7@$6l(`)<-go7(g3H91t2 z{du;4E_VO5rul`o5DIvZ%c_K%C^`TVb?d?1=Ab9FQukxX$*H3t3Jw`N-@oRPi0D@jQ2Go3F*{N?p{gxNE!D;ZkUuS%nJr1IFww={M zaZWt>OpcPv{+Ng-*?;SNzG?tvF!pmR89h52j zfj5&J!}CWJ?cgY01L)Mm(%_(<@)0*Uux+q~25e}v!)tDie!~QKecSqLE?~jU!ApOA zNbtk)(l*y@|K-pAZ0~O0?O{NtK++|%GEV?!Bgu1`ZD)Dl_>qNqkOM!L6c#MKVST2% z<9-_@3h^YvY$EBzw>WR?iPN(LAW8kANz>sl59NlRn|z-S%;Pnb5$e@s-tZI$n19x% zkY8UC=*)OLqv{Vj-5+o*rCiy4c81Plf{{G1PCqA4lof0x;Lb@o6@-I-N^nMR+n}zv zS5)qr4r&(C2-Bglu|t*oz-<%H9z{{mJnsB?!BH)qLqNng`;5FR%HVfKZM(QngT69h z)_lK?)1F)k6=PgHJ3V}?ptx)#agHvCp9NiOUyCWgY`~#$G;Q@ms`dl(A!2=a8X|F$ zQzq$sn~f&(6|g}{8zb!zcKRd->_P*c?ONgxt9$U$%%I~dH=-ZfIf ztHm!>LGM1?;{xUjGS(*zOG`0)ASwkPWdKqCV=obbr~Jpf{4+MUkgSZYRoS_G2qSZu zggLw#!TkKQFH5&%2fg%#4Kro}I&1h6Kjy*iHjLKXhGi7^im?*{RDI84>+JEJ7qHkz z>H6A(%s=|Go;?YQ4)CAx95QVy1d)>z9x47Qpw3KXR~umBS5Y1+u8<&Sg2*khu{cP9 zxcP-(zW!}*q-S6q5xXRFQl6vPG<}w*?(5O9`?KuIiZnRx^zWpoM;u|Dl%k#0PYjxE zv`RWO*qYhDZM%!>#9e8`1%RQ>N@h16pz*yI+?m$%ZgWsl^VI&jeoKKaAUw~2IxWr~ z&qZ1<7jsR$#%gf1uc^%SJCm|5C)@yaEUF z*b3E#2sx(=7Da*3)b$dXiyTpSl2i(L1~X%rm!F0BwwAt^jx0}_xXP0uLNBcclPDigZK-OP=w}J-xteK*28Ro8tl2KvrDDju4 zueA;wl6uN#b}hp0-k`N(BXR)dYftOj(TIsx>ZHH7K>)7|21=F`4Lu}PV{S5OY=;AHB=N4E@?7=4j0y;>_ zjZ6PFfcs)R$KxOaf)Dt83HhUeCk#Sq@-Aaes~OVfx$*2)`@qZ4#W)cbVx{(^WS4-=znT_^$|pMc+_2&83y zkek>+ew7RicnccKe1V@sOQ%24Ks-fdMA88Z#fb%uZHaNRALmo5$LPGuj*Wb{yN#7Es z&EOJhA9pWt@YcPB@(a#6U%AHMiAsa!AfA~JTdAV>c3N!^I(ZxL-`BQ`v8{n%?!tx$ zKJjWJ3-^ni78ID?_d?1+AmTLf1rw8^k9C#`lVZk@eo))jPxm&*0%XaVz%sQD3RzCx zmC;jN0$B_4uuPg)#_B1b@RMOPjD2;DJz-b)!qSq#twIVXN7wo{eN>y`N7c&t64jMn zu16?j*7v9nxh{;QwmgmQa8hM~U(ydkO2u?!KNo|aTtctCg|iR&9*+uhvqAMNGw)qS zi|7QM1adpu2M_hi8`k-fiNO`fuE-6(MUjOZ==)jmhV$7rV6I|Q$8Fs#RrL!52AKpS zxP5F?eDWpVgmANqLV|O0W~tK>?eR9m5xq!t*L$? zV0=AWxYn-OQVo` zID(nMx)N6*d1IJ|4VI8dseV#S7R0de)=c<-RQ6!5Q?|-QQ@U2N$T57TQ;2#tYXxdO zoy@V;j|ei)6*x(wo!l;$zIU-;RDaz+qnfY?($0XThq)_~z4>j$TOwgTenjH2B-xgz zjWhZ)YOl{dH^)f|NVys`6dSPKj*P9gmqSe>HR@MMtpe9H4MfXStVxx;v04@N`dk~l~Y%~GW zS9m+9TN5hrB-OpqPUr8Q>bwS80y%v|>Gc&GNUh|SD4Gwl{$au&$s!BWp#ae;>e?bU zcID6!mkOku+zx%BIpy23_nT>7gt+4}C^6rWh7dw1t;xBRTCOLR_em%$gzvd;dGOw@ zGlO#d9lX1%LlR8T?hU<-TuVo`gFqh8wiA!UjGmo8)Vs3;qjc-ameG#dt1fq5MidOq z7v&9t*yChpc8&#X*5AaTX>wVAU=g}=&PEya28oH_nh9_m%1?j&2-7!Qn(IVuOc-^t z`wFn0!vV7Gh_lQ64dVA z0~xpU<^M#8{};Lh00xNiANvJh9s1eGbj6ubU8!Zi>s zH(@jo_)oLLj@Nn*5b6IhBj_dcocg(qf>4f*U=^1v$egn4T=$mma*XV$r@^~JD_nqPF<+<@hgW0QL;8<-vq^xbS$VkK1 z25)ImRDtIo)(BzU8%GK)yMw)?J$KoI`^#oEE&t!ew-RnwQe@&{La;gNC3`vQ*bNqB z2DYDZS>DldwL_|g0lj;f;MIkHB__5GyxGOLE(oHPGbHHIoUzCsoIPS%A z=f93{2U|jMDa`agu*x2dK$<476@E12Kb-cj4++VDVWcmAIiTp#7a^Ll&7(Rs&5r)z zV5(x+9Q%+vZ+@=j*MnjbZ=%(VOOQn#G6K-E|CI;;2>w?h{Ff2- zEQ0?s!vB|yz#)yNLs84wnrKkJU78@V`JU>R9uPdo3<^w*#XYeQCczkBewS}|zhBg-eFE;$@H*Dr$LgF&I+S~Mfav10~~I8MAr$KQ}jSg~l;_=tw1Tg_JMQ&}5p zSwMXVd1TC!V6Ib$Sh_4eH(POYz@NoIJG=QtdsDt~xN4zVM6xI>b<%WJ}T)pWHam4S6qcic9I!}KvN z!9&$birLJHP6B_`%5Ivh*qL8{gtaHj_kPIg+&10D;$OZAv8g-{>=9;<#}9+B6|4mZ z6|BD)gNQ`2Br7joSv^q35VjZ#I-dA{HI2nDqvucol*!PaJZgKoGv8V1O8_M&GeJn) ztU|CB3`w#pUNj(tydKNXqOg67lk|9l2USNIhp#CJBm)aa>xJD*CcGM`-gKaZtoH9z z69Y|WB{z9ZGF3r3M<)30fL)R?o+L4#(&y}>eZ%NESl;Oh7n47w!^bIO2FSBf`NxAj&Sv$LWDaLP zVba+KS&*;}Na&*f(690`PR;CPF_ohf!865K2E5h<+;?R#Fp=gWaB(DHu!7XDe;BO^ zK9wHXrTkPdU67NvgthpD#$^-;NRRsuBt8XsPruDY>=hncz#u6JejNNeCP(JQl@}LV zslchS{5BWt25w84paFw8HisXn7eSO0Kg)IEdeByvnB_r1ZLV+f0I!xq& zkVOVBU3$RIE-M>%1k=nrYdtV(YI&vDa!IQ9{sa)x?9b?k<% z{Qfam92s%2g7(+*2sT2BbOWJdH4ELP9ax@0bP~#0Y_rYmH`P3{BiLzi*Z`}|hpKnK ziReCi{X(zS6W|NKhO&+br2l{3fcU^W-OdIiVk;4jS?%>%l5BMpHz*MR*+!p{W7sU4 zBa8+pb1Jej+o==125vdK5 zvOyugfo-gm)b&SVo5w{d<=w98+n^euXF+B?xvE4z1e1%z zKvE>kA#LB^OQujwi@Anko(opKJ`7eKkV|Y17~M-i%zzG@f-sJAO>R*A2PohV4!o?q zV5=-8{t4tth^I8(->UTFqE$}uWXnC$BnuqbD}YJk1hoZaABmCUoeki1^jUL3l^K}t zK2G(tN5w#Q-2D4)=WfLM%qBeUQ>Za`L!ZO;ha2Gyj6>~3<^;Cl=PDBitIHA*2rBQ` z#FjO>K8U5lV30<9y@LXsO}>RJAXTZNV383G64c5w7hYprW{}G;F6lBw@xi1dwTNoJ z4Tl9=)uI{pQ|IkEZ>~75BA20=euYqU1JWBN$vhzXx^2T)0l zWq!9B9WNdWvAK_Ol3^sV^zUYbh8`n?PByj}d{BYJbYz*#y9}sBY=Zb-w*8cyb0%Hd z{e=;bfFXJzO95GjC%v+|h71zVIG=qP{SwIvdoN6EiD$Umj6ZooV*x?EkAe(Gp0N5q{j71^!m^y)YB&`8uME{aL>2 znn!O$5eAs?=@efj)1}R}5FaEJo(pfwd~Y?)@0kUMp`~^`T^(`Yy7#)yY#md_6NjK^ z)=P#1Q^hw-t`>!T!%;VV^7{VzT_e+|2xhh@mDg#@-J!3qm_|oOxU!`t;6T{uuhJvv zp~#abed$}q3GECijN)BAE24(x+;y^uxb5bO#^V;_T)o$axVbBm884VVPOR7}+`T@u zz0oNhuo_-JSk_1MEx*R&fzr_ATG8m2`puAyo8z8Q54FiCmAl;A|GafdtQLonQ}9Hm z@G%@_*7=;WK~IAZim{*j%6q45iAM;_w)@{Xy56>&8jpX?>Yain58aqpB*3M55nelY z`%PIiy|Xj8)O^xWos32~(!Kr2T@pYHgZJ;ebF8h526;xWj$8ImxpNb6&gn(Z#H@U? zNST`1VG}Bp;^IfNRSYRFCagbrYrK+WeYE8d_C0-Ip1ovWKodGDX}zw~uRZuy!Oa49 z7v%#@t)d#IJ{QTpp3vJy-F3f-=B#IZotpI@#1x_kPfq_+wGj*lkJqR2Hs$z}dLnXa ztL+Tsa8s+KFg|1YWyfnPeV1{$&i*0d5Mv}9KG|+9KC}?}cfvSB3aSpbGfVU$Ee8Ru z`Ysm+6Kfsx0vJmk6mugnND@w}Im{(E&^Kx~j)hXlt|dv)v^3H=X5e$*y9ZW-4iOJZ za$H{&)pOa)=>C*l2l|aS;*qp}z_Qx~kWVvn%B{SsKQWX7JL=4et-D5vja7khM$7C$ z6V+eTrylpo=cw+}E$=x&OPQ=nNZA=Wek#Q)=z zgWI^DqtwdB#e+_sQ=>%~d=776tO%RtHZC~Z?6CayNf#4&fXT>bmaO`~dsmw-)eyi0 z%X|$kaFY7{A=vRcWh*b$xYYNt9|IQy&-d#{?V1Ok)n)G6lD4&32;uCDkFqae>@8?#OQ!mfhx(r04lgX9IC#H8C9%*!Pr5gwH>2w zojy7jTAT`hbIns#KN=pgJ**h8#0sx)^z<$_v=|hj4;WxwA0Lf{oXP5aE_u4@M;DZX z?Zr|Ku}lg$C2nIy{!!(dlN|+|`7)xx{RR!U;TM;8E*8-{t8-<>c5ANJ*@VBJ-KmdZ z>`40pmur@dhL>wMzP?iMJW)yBzm%hh!+GWTMkCj2PVL;ohAfBc(CY!*z_Q7m40>L& zcvX(K`=ngrlbej83}vL!2@|xEo1DAR(ZUw6k$*!)TX8UA*8{%pHF`porRbX{ZKTAU zya)682Od>j1LqySo}F zrNw)`srS-kkYd7TaFnx*5*A*!R@%&}4~?cywNjV6nkE0lA~fjICQ}K6D+7O7bV%%M)W3QlyDi>cRIkhAeb)S;6YUV+`VFh~YoQc# zvI8j$Uv%7EldnXMHLHeRsMSle3c5R_8{45EG4uS|*h2rh(!#`Oozd5S`V1-B|6$Yk zGnMV#Cmc;yyYVG#G(?u=o%@E0C!Imi{jSlz`YZLE(%SR!1=AsCR&P(nP%K+KjppYU zjVkC+5F6krvR`N#MJ6-KZpg~>3d4)+K2Mv)^k>Bf6uT;oys<)dDCjgPcN)koF17RG zbt(`0b_K|`q(+{51?m1~w8`sOW^8e(pDq27ol_n62sQ0v{Wcdue${cO6S0kU2|fQ{ z)v(6-RKL4>GV&AKe3JRX_a45O!fNLs$Q`MsOq57x2|V64L{c~uAO zgRuYQO{+|!bd)$uQTQo5-`&M2_sPDi;N5(%(mw=q01(TPACH>N|3~_;=081tgze+FV_2=rmcM3=l zeaT<482BcW?E{Xs6Ts_K2QL+bA9Db?4T$sSy9Ks02`3shPMOCK4Dt3DxtIL?MyzQbEV4+ZI?N?)v!%4U*tBsiyb3SrUiSHhra zG|`l&ylNaGV$^N)ZZb~YI&1W`7qd&yRFJT0#q9G>i{5clROcKOB=h3$*@Z_xU+Wot zs%$PYb*YBMLLqx|_sc6xVy#%tw*1Bk+XqA2(vE%k(Yf%cKhzo_AsGAl9bCMV=1@eR zaYpH zzC@Smmqkxg zB^3pEHd)4S-VDZ>s-w=4M`BD^4EgsL;P0clGi*cI?#X12ajF>Is9z4zRhW&Zjy@eR zRR0(j3T_=gS7xTvC&HPw5_rop>@@T8Ti=b&n(YuGra2kQxart_p(175TS{3YS#q>5 zghhK8(mcwb>nWq;p?|;X&8iQED~N`;38z`v^wXz_im(e@S|7h3r!*oYyne@aB)LY( z-Sf7Rif8P#S8NS%MrS9h4>@+5kIFlaZ|!i&_HFl_8Jz1aTq&sIyweBgO@ljl*eVT$ zq!~;UOFM7NEbQFZdeCUvFPib2>j8W8A2PnXXUuAbZK_KpHLOC}fT_P&-x&HqiM+~D zw){&$BIC*OSYd{BPxr={l4bmishD%RGLttJ^_=gLbyV+ghv0^ordM=~ifdQ?ulBw> zs>!VD_tmkCW1AUA0YR}qGNOR=4vuvKgc53`iS!yNp@$4(VU(r`y+tLI&}%{u>IjB9 zNJ0w`loCw{(O`%qhTI2qzHjDz@3-zhcddK>7}jDf0wm8l`|PvN-oO3ZXRmw0kgV%# z#Sk=sayLM6|B~Byn;VC$p?8KL#ue_#%sod~l~0e0yXF-n5#mASJNlFxW=}()@7g18 zcpWuSSR2DTld+n`!mA!pzA%QSjQwIztfP~QtZK}jbW6QX&FJC#TLKHbv|g>eo?nC< z*(olDR;))%b1%>G{8wLUq`ocT9}Gg~<#FgArea4gKjqYaqnns$s4@8J_>YIGNWX+3 zQ+>Kx41s=E8%E@zT|(=>K6m_$N^tN&0;i>0YT0Vh8Odl@>tspiZ8d-nRbE(Ns2m;0 z3!~J+BrjUs0Ds;ZX_A|d%g>U_r#_cF;jNzenDgW|DowRtyi?&B=xq69DMq;V*oSsg zrk8K+2C4*RXnbY$hZ9C5S zY^8^f)^9XHtySB`V&YhNTlkSO)gR|ae|R>jjAZ3~pAoWvg-1t)T5%(4gQ2N$VK>~l zA^c+FXf*FPep53D4E)a+#Q*8_TI!{Dz0IM%_MOKG5H18+QQ(643L8tisaD)MVQo^U zW%l+lr#e~6Xh+0ebG^}_ThKY2R@K|$5F0+Qb3oYo&D}Z=0$< zVjgPGjP{+IeR(|t!k^|A^$O;#IC~6s~!h}b}wO~XWB#$<KmVyeFvP4 zwOy++bVZHSn$DK9O9 zQzezVdke*h&K^(B{Cl{iZV-k7i=3I$a|^#ZrJXstZmK<;?48 z1diBtoQY(Ww)vBv^_~dYA5hchm2~tgJ_$DXbD^dW7j5DuhoayJ(QNpm1 zpdpJ0XgQS~?1HaB_}dy-LPjfW-DV!NtF_p5s-q_RS;_m80$}b~oC|YfB!0Mo4t_D%x z3u-;v@TSpn>2WG+zUt+b7+RUypDlm>%b^PlO=$h4c;jW4XF~4%C|};Kc>{=e;MWGM zsshqIo;x24K}W-nemW2QA%C6?p%HLS<-12B%`Bj!70&5L7%FSQq0TVIuOGcbLqm(^ zmnmS3dok$vprysn!5aAUsJ?k7MyTBjhln4sTFcgN&f8IBF+ABb=)&zZ+z6&dfvGrT zDeP+x5|HZb1yXuDR3Ko1E&Z#GtU)vH4cYFvR67 z&NjmQwn&km@=I=@v|R+{UhcPRkkB~wK;DTY^wsK&SdwLMfhJbo#6|Ch&nR)Hw%bTz zlH6DPPg8mXL{qw8JEhVqqtg2b{c;&PdhcuCe46;$!!KwLeJYSqqRfTM?jrRm&B`6+ z^#h00L&wIMt%EeZ!YxdmoUisK%guiKr<=8Q7Ab&EUGv(u1FU9WGN*5P?dCVlW$N>H zk;9%}-@EjG{Dgeliu}X>aq9oM$dli{gC-LdngAC%fwu88EvEBB#N2M@_mn?UO!`}{ zyfI==g=;&X+`@bpy{z!`kSq-yZhu=Z$W2XF?*hcS8o-EwE+UDx&{2tnNf=J+7!uZH7q{(OSP z_Tc`nO{9Xd4Bi&wTW%6>HCaS33IE}lq~Fk)lB+N zqk7IJ0pT~)5fflpV>0P|VyH+MEn7d`zcsDz5Pu+52ZW}P_zzG5ou|_~J-@ZIWR=GH zKP_2<(07MP<{;4dPXVFu$KN_4xAch0O>P#6$Z!TC`B2^8?x;Ajo2u}9fwKP&KEAMc zl+T(GWBz$lX2(PlH+xS~#pR z@#}RyXa>K_=_YDMUsQItPQTM1^Zxglx!fOP5uwg6xIA+mA6mE?G0(KSbr7xJU;!Zy$Qe0A(T6Rn?iuN1h#5MTLBX}lpv%Axa<+1 ztM981_unhDX1>e9=&QOJFxOd5Ek25F=H*AQ(m!QevGNXh(YhJG`>Oml9sbnVA%u?{ zd0V6C#Ya$P0PIRC-Yv_vgR7Wk(VYSI@}gDY!Y7t)KYxAYoD)_$Rqz~_WasPN(8+MT z&AcrCmjRAH+&TwRyR$c%edZGfuX2<|QfEEIp8uswxQnXysSqC@^JdAj5Vs5UGqEB} z2=4$sEg{=#GXHm7W08_1y0C56w6lQ5hbA4SGb|>sU)j{8zV*Na^#t0OnAvf1e{bf2 zTOE`297`N&qY8KCP~QZ8X;HUj>&iFO?O!fnFh6N!vK?l!h4&iYS^Rz!*bLWD+pDqD z^0`yT)}HINhFTF@&JqyYehg6kqg{#|$cdbOdz6Ide}xg=p~{0t(p$fd%bNd=lGYPg zJCcpe3p$vz5Y%ccqzqEtTo`Z@Oj%yqydU;E9wJ)4Y8{codwnluDYs7}<~T-Ad7 z()9rVn%<}4;}@$y3f`xXTN;hUu2N}<;>xKrK?JjhZV%UFm9_NjdlnlPl5H!KPpeG1 z6+e5el*DgmM!8v{L|4@q(YgA)~TmX|YPy`SI2 z#JE~OSo<%@XU3W2*k)GAoBW(e)+-q)5;`%?m*K9Ldw7_^hN6%7B!ZF2{Z|he4SG+w zr4V`_K?>Z+`R$51C5GA=gQlzzo(Q~>sWM1o^`&6+joHyHt}y0{VTuE0@#&l<)2!UB zXv)xRIYSfousZDB`1~!rpFyO-DP+5g4nd8^{wc#ifsaK&TaHBz`W{Vg4tj>7>Z!5W z{A*H{9OdS@*?W5OKn|X^&81};Kh~W&H5%~Y+S=7aADjj?y4j^E`s=JuA6?a<1c{{` zi|^q`zT8w7Yo8t|mMm*Spr%K&+Ery%_1k~D`pqx-DwvY)7ZiKm4puT;dh3(|^P z7>i-nHDR(G)R1=^d00;NZ;Y?nlC4@#tQh5GyZsw7$%8wZIM~EMN2wbD(H@kB z2x(2o)QR?8-hVpQ^OZZq2Jw!lvd_pkr;OWB)K9kksMi*Wt`)EMNHF?huLpJ0 z>U+f(U{b|G!7y6#rZPa~kA>wIFF(w!p|s)+1Va!ZJz!aKC6hblKw4hTh%@wKeP9$~ z7PO)@31^&4jDZU;o(Y|>QecpH4zWCTd)^KAhqr`9Ol?@#?wPI$`^fJ2Mk~QiEWW{g zkI$1WIS>68HQIXN2_!|N#lP13@_EbEw(78J)H*oIP%qjFwlMCQc{jQBtl%v^I#sn# zY`XIC#S{aRe0m^6^KFzYty{MK@dTrcjy}>9lF5A_mzg{nkvaCl`^=D8terQiqcgN3 zV4VcQm80FviJgnAoUFK0PRWP4v%Du`thnU zm(;LqiTR0q=}hb4F~e?|NlPGPz%ac;sc5nf+oA^|&a>grs)B@Cr&L=Y#RhMM@R z`?2wp0ED?42k;C)B(tM=wz0J3L`)S%Db3tpeIbBy0NeV974BI4HQpeeCHb+%3?Ldf znUr*tSq0m%wz?n5!iEPd0&{^t`3_QrHsjIo zgH?XLs>Lt$ATk{%TF|f>WhkAWKi0}s+z-8+vDl0YZ%>)*9BepzrC5eNBFv=mSnS%D z#8{2YXxExVehu8dMhsTu8;UPA+(@9`tPk>7y**VEAt^z?8>)jzX2L#0k5A7utwd3w z=nDsyhk(Vo--z^viOM8b)?Mo@0f&l=^-fb4?!a#-@;Qqoor*q(pK8&zV9aY}=5- z5w;Tsr&Y{Z*CRBZs8uc-deGJO7Uat1OocfXb$Ia&+m91E*BLYQ);ds=nappvkWMUB zVW<}73I@@FU8dFs@P%y-U^?iTsf%2t5Ft;jM@-c7JtmXBK@3RuF8?}ThnkaVk3DxN z{lZ$+LWuTwnspjmuG*vG6mpL(ntgr&&s($AeN+R}7#;hdEQ=&ngz?=c++E(4&#mMX zMa(#@XHOMoQKQ)BC-5&DQCF2_e9RYeujzu995e231{1S z1JSJxm{NS_s4%9Zf?3k-P~$fSnf#G!dXlziR1z3is1|{&BeyzB-K&sjx!WHg6q$-8 ze0i+?;AJt|&;IkGq=?)h1uLbBNrhQ{uS+?|_lIGew9^%F?PJZ8aVpx+M1_zV$$(y+ z^$a4}V72P6Am_Q#tt%GS4NrBH?>^g;{Kf^?mqJRC8748UY>eEjG8(5r*zG#voLz2d zG+OFVAy30uFC2$kBaPL|&og*Jg>c%DNNc~sI+{xG1M{#4^`L`;5dYVuo!^};*A$nn z7h;z5%C95VxQj&AD7c9IbOMVFS%`tdvHEtAX=(?vs^mvpZGTM4{jF^!yH2aZ7;9iB z>GFPZpH56`INm1R>U-n+B9-v?(|Z2CoQgv;pvVC8p$`AygXkaHGoiV^zhOC7g$d{J z!gpn!xv;r1${h$+nig?HBn567!1I2$EsJjtzxe#~ei{GdpJe%>XmSicmRLQNSJ%&r zK{ahLk9x<3yJ2r@0nN}$6$})T(j&d*{1Lel2WtGHjE94isHR|DvnLwrZEnMAKWgXQ z^v2(LMVf?ijaWOZ-iyAmR(_NlWFoatKg3qN5a$ zN7WyA-mpH%$=YdR&N?9~L*M61o^h1LvS}@DQoAcuYh-PFZqPUl4^`@M1<9?mlHSUS z2>JWwO2|iV9?#o&C=F^5)du#u@b|;7qShB41=Ki?e<)IML(*<>q+Cm%S(%glXimD%AcOb#1sn3cb@EB zX=rCrSH{x63N8I2T5-e7I)f4J+e>dfX6X0*+(ZX4Kz<}W*W6HxaJPN>PT(JM0&FVj zc%G}ra3faQrA5bsz>J?9V+ZWH{`6P6m)IS`DGIgVF)2`NbZ@29GILX&ZuhrjJuzU) ze7qm@cVXU&pMPv;I75$Dx%@iWvF;u>7Syq>!~FenG+FI6q2bSdwwrP0^zMH3yLFMR zD|hh56lbYx`%D3|C?A}KKR=DLWu0(`Cfm|=@@%88q4)&h3MOG;;Mud#(oycrnd}>} z;iF@>`*Qz~wg-|9wb~%%*f=Fy98SID`bZi?^4l`vu}ZIf+lXsB+O=j~@)qGayI#)s zC|sW*G2f+q<-tWqIo4P>^}x~ex#clE=b->}i_=l+O5D$qrwJmrzPXb-GNBrGhiNTW zP3X4FX-f=hhfwS7D?B9GW#n8F%Cvp}-z^-~Ibie*bdIT)D$MPioc05$wT9NDYI@|X z@`e5dxc5)QF!I_Cmm@_@Ll>}ZpoVm~>eP@%w~ET@82f@4UEOWOVZT$zf-AR5JMcs7 z=ZHgnfK>(=?JOFt>3u)58m89kF~#fg@#+8t;sr(ndTM>5Mz46X+VWb%=LAsY-LBDq zdMSg=9E#VWGRxmbOXEZ!7~gZ&+M@=Aix~jmEmp+diw)xW!-_ z$>NA@;UkNY<;FZsK7F~TQ6!0|dELw`s+bF$G+rGu(B7YFxsT;G9!Va|po2#8pP}+x zLI5k<1X5wOi+o`AHAMDCv@w%F3zXMVzJ$J&aaRohreuK?ns?dPb_`txuL797&X)g zs#uNpm<%C&rrlrtE_>mCU)7S;$&DO9ZTQxr`0K^S2VLB5nkW-YOvNR5m2DPI@rGW{ z4L-0?UcEvwjdfrlOA#t7X0zhOH{X>|1~3{JNU_TDDCxWf&MIU>IE(jae93E;go6== zFLz*+_>`C0eUUzal?1Z2NKLt?rQ_b5T&wB8<5J|owoooNB(KI#%-Pq5c0P18 z-_Ri1pC_W6#3q8!_uGx%KCWDlPUMq~y&vhhh-e7etZ1+;mD>vhFRtKaRkdlsLYPDzeD%$52g@;j7|#QTL> zg?hR%YcfRSOsb_&6KjjA0MBq(&Owt^5DKwRIc9Hg0Ud{X^j?t+ftVX{{|G zAI`}l`sWx>*ZXnD2#BC$rJ;d)L0E~jo%@pox{nA9LZ?$p+tBH8>IBPv`3!2Q7CTQ> zVCLUaufG*x?vS;hd%9@^V+l@!u%ZG$3#&uoR$a-cXW>g)%jClL7t^s{RWW@FUyctn z1Ub>e7iR1whk{DuO&2LZ9vKZS#I>ugh1NeX4LhT_9V})9iE#yG1&ZJC+{U(^(^#ec+C^-k)r2Xn2^aUMa*}Ps!_iigGSl9Lsovi`f zLv-OMpwF1P0_ng73M3asHX!gB#gs|j6{%Di$;K7 z{EWvaXR?wJr4`>moDLjo)L^d$*h!`*hAS|oU{#BK=GrhnS8m=M>SRjLXk8=9RlCHc z{Z79=(T}a#x1r6$$qZ59<;T->FCd6q{`$-HCYJ$byYv|uT2fZLHecc}A}Mznos8s2!pNz|fCu+){<^BVE-uFUrk@kP5lFSElUPZ%yD z?h*_OkeFcgj&Sz`zVYCWw=)^rYoIJ}vH$lT^!W#rSd9KE^OQ?LMJZmx1h%-gxU&hPY^m_dY}y$Ic8Y9<7Hll^;h$z8?G9ZTHG-~UEQOGYiA zB|>}rEPM}b8&H?qaH6y#xUnTh zTNf)xfGRG^-mdzriphW~dT+BS*npp|O8Kh%!O^(V7wePP`hwc1HhvM+yMlty+7O{J zyjr}<37TbP+;IZJD?c~fI>M}|iJW_Jjja;#aG>~GNA6tU59gtdpZr75@n+}y=MPZY zi#Khl4%0a_W^F8xA@VrUmtE&>8SDbo4Mzrta!i9-*PP=34+I7FOE(?Tz(S; z@@_VbKSEU|b;Zd06zy;^IcX=QjMgFSp5^PP$743vqGDPp&0<}bFm{t@+Jth2VfBVA zEllmJ!WTM|_vx6HvYe#BAZF^D+2W1`7mHBpt;h^nw`dI-IuEg$QvT_7W%G~Go-heQ zR}bFU??3#!c~Ry9o$1x&P@FV&|FqNcBDwEIJ4x&u#TXf=d)e3uptQN`EuS&=f?eTj zi{u82O@8Dy?ewlSnE`=KTW*D(P{2>ov!gNW`lXIAi(I<;FhNflNF9 zz>!RClQjd5G0n;RT1Q%gbw>@E3p2@xzyb6UC+sCt2PP}zEp;b`HHYro zl+9ZLz-ha6{m4z^na&&Mr~Mp;Hs)Ev2|#yz_Me0zqowt7`3+s15$rKAopd4s)0~uj zl=)c`zXq)R$|H0ci?G2Ty$d)>g93mq}QrpPPof%>`-`wBdXZ~hLW~heR1!wfQ?1V+fjJM z2g{hvy)j389F|cZ(I$D%>EKA2&X~nwz%usW!N`463TtsNso*OaZUC1ky$si_?`K=w zxqv*n?g~3_4*_N6Infn?K=}8*0BmAv*$oI{ty%kWpuXUB^%QgYkKgvH4)GEdaFAte z{(ZyK6N)l+EXK(+GvoZFbVDNW@;)6@YMIA)2EkRR!nHzTyv2l zY%bqc#o~79s*>0#*eyL-C>Y-dcfjgj0iiSZoNl-pu2t=h%4@9Ttkn07-M99b_^~H8 zZ|G7J|NZZxLhCC7*Zca2z3QEe=6av?&n?dNzaK9tiqQJ}!u#MyTkU2a9&!HuygLOA z!a)a3{asK>tKUIf(yNk8u<`M%k5mHXLF+NNRBmTRJ^=r|RW%s|pzbEzd4!*78k|eK3UWwc&+$Ug4@p}r;yHET)SelD@;;T5kfU)@g>!eBM!8gqZ9+oWKXSn8UPy~ z9wkHi{QChL+*ETDQ|)@8Tu1e_yNd zeeE7?VTXk`)C1iZ{X8EqUD1wc!5XPYK8Uc*DV671dNfH|eRZLFXEVyz1G;Tpo8tRN z7%s8DmGBUC01z#Th5qYns%ry?3_04L)`x5N1U)GKeEcj!Ws6^tX|mpOp_8q zHk|~lYxE=xdt4Rbf8u&AC&syEzr3mcV^8NbQDqgk*jv4TmfNw&m4j}1*HXiH^;JZVtQ$GentS>rZsju ztVPwRHSwJTR|Q5Z`3L$w#e0mu50=@m72i7AB*537G?pA`(Aao64)h?ElyzW)jVl{T zlT@gRd5JQ~;`xvQp}a;m^fp-$ zXFTiL$!|6s_{gU zUZBdBo{7bn&Xml}0ahmdB9LW23dU^?<4dv48^>A85g;(;-qlzxUGIt%50A=&1^FRP zG#%V8bAc;H%**DR|2lfLOO(=dh-9)klrPkR%7iXXecEn}yn>)06iqViIEl^u6m|ab zx7iA_x_JtoUy96Y%YY{*eKT43MNpgXHdDGG`M^#DDpturZJ~jp*lWzN*@4%M??i=I z^FCZWezpo2z|~Xy&mvBU-rmm0b{LHK>*yUr6WU7A2s8f^MV;=}Yp$W^dDAmL_-{#xt8nALT!ZZ7`oF#e%${v(aQs`fw9`2S8Cm@B&s z|MjmerI3q1+5WS6znJeo$@G6c#QvZEDZzcKL|)FxIr#OP-mrfGUyw@{7pu=-`}KbT D)j8rz literal 0 HcmV?d00001 diff --git a/docs/img/structured-streaming-stream-as-a-table.png b/docs/img/structured-streaming-stream-as-a-table.png new file mode 100644 index 0000000000000000000000000000000000000000..81812161178fa9ce89e94d9dbf326de8da385be8 GIT binary patch literal 82251 zcmeFZXIN8d7d31dyMP^~>Zl+BA}Sq%Wh`{*gc6#72q7Z9B&aATRYek{1(hPbcSvv$ zA_N2}2{kAI0tqdF5E4ju4>->AJg;+ozu)V-=7-0SoRjQx@4eSvYwtUGep~-0-)^DZ zn>KCY)4KJC(WXt?8a8d(dUWS@;F)RuKN5leHhUS}yt=92i^vr4$FBRgz+Rg+2_9Jg zx7h{V1zd7@lhz+sO#C-b5AUjzxoN{%1Nn8-D+O(_*Xv`2>pj}pf6HR*BW~yKXG&U5 zs=qt%V)*HA2@yZzL=has=^UZ*^Hl2Wwmv)F;-m#;33k&=d(>R6Z_m(!Cy zl{)pN`(n4(mn=wOVor8J0jIZCO~s44lwx)0)NlJPZ`%Ct;omj*HxB+ygMXpmUnKk& z3I9dHf06KCB>Wc%{~tlZG^;IsrT4`g+21^#mgl^$cDtmR9-2G-z@7X+Z&VEZDS=!N zyYSDLm3HHs69IFVEvA5T^|1Y>D{M5ux)(BeIJR%Rui0kk)GS>qN@bR#Ebueo6CxVI zvfKVV)qTVQcF?0BVew)4u*p~bgIjhUf4B)aq;)(^l9{Wh3;82v8!hC;c)fhAzv}S* zgqnx5=EIfJTEP{N=VFxKH_inP#s@5vG6E8uYqntRx@Yn}D>YZ>-whP6vLz z>HMLSos@9Ns4JdX{YG7{#5yJd zwJrF{fyEoS;!ySxq(ND5;<&jLB63`_Xmu;TTi>Pc3Ca?IE0av^49kMC7>8F> zZSl~u8NRSgy}fKeG0J=}Ff%~zyx`w!wC<^&F38_qw3&34Vl9w4x0z`Ea#FlAc>a+2 zJ3UtKYqP;Xv+iQW0O`8Cz)bVqyOeEVR?xGcWqh zjl*Ec@^sx)IvBY!n(1?QLmHjkRC^hrG9waVyvOnA%GEc^n4GG31u+Nq89e7&LXs8e zaNke>!_;Ib;qxaA>7JF(kw z#XK*#ew-$2e!3}>*SaDj4PuA%KjjTkDW~Er3HJanZMQPx&$BzHy+$<%?G z)-((Cy4>bz^*$$5bXB{B$tKO&m&aO}xvR<21VP*jv&B>?_6k_PyNVX@hHn%*L_?93 zED7oZZJ1WZIXtIH${-xvSSyoTnCi0ukB96D9hiF5ZNAG*^ zE|Mwvg$NQ&9UfM7iNJ&iw-3yVyuOsm5pxMh8K-~=E(gk8OIJoMh(U1YfU@z-sr$6$ zLZnXtqZiptU-U8;CO5v~-Dcq$Rh!5hc0u(Qu}&=X8d^JP1y(sti$@bC(XxwdiU$(y z28Pc%1z6l8FGNfv{Ndi{K%`5{&;%xMT2RE7S1eEDU=5Co6^2iUA_r>18YjLh|5rks z+>`9#GTF#Db`lLf)DcVJX}4Fe4G%DNUMgN%Yb`;rt%|0n^vwcto!ZygDo&AZHE)JB=7RvnRBTX)&Hj&_M6rD(sS}J#vw;xjt#`Qt ztB540_7!0y*UcoREHHKKeY6uaMibB2kJONA7PKTxEGWr{J5`@8pPF>L-4=2WEW9`3 zyB%KJaxfcN{_(BtJB7+FFPRk$Z(3(p&{sJkx>5$EIi*04SWoctrysexKcubA?EoW+ z$Ax9F0}nV}moifPeXvIcn9V^ICpm7O*r0wjOMX42=@|O7VAbH}*{=%ZkwP z!G7ul;d?lc5{tTczsfp0@YE4xNP9_Y7onqA5G?juqIfeY^Q{9GJw9his0wjq^N>U< zvo4p&a4TM(6Zb9mDY6&Vx7}YDZSJ1<{8Of9Qo3|rH!;z>jWb#LMO#yP@ji#YSrNK;HYVp1HE>mMpN*sjSn;#=0<<#E z_M2Qqo^3Pj70>Ppc5H1Vksqqc5dt}QO`$%i^|ob~yKc1>-aN3_87OXFeYRy8hb%0f z7M?QY-o3VN*In~(Q;Nw5evoIBJ+t~Uh4=BH7uThEM}Eg?~Bl{0IagJ=kxjIVHQ1=LIutc-DA|1*n5j zw5Yv~$S6v0pECzz0IUFJdb)7GmEMs5&W$G-v$As5%j)RtXw&8^&Im{p6U+ir7w> z(z>KEywtoM-g_PUZR8FZ`zb2vhG&m$iWu@-q-f$|Z~gn8JLLDoIP-N9^eA9|XGfNV zmZs7GPlWC)>{UOWQ)nCDt-gdaQ`8G|uonK#QPF7c344oy+(66nK9yh`-beQkDDhgK z2+f=U5l6sJ@7vTS#b6iofy8dHUcB;?=w$w0WPi5we2e<7p+6t@ibt*9Z%Y#>UMQQ& zk^Ed~l{(?4Tw|SWlNvCHggqBp(3_bJ(r3yZF3NI zV4{5wQ1ffpiFw@}Oblo1t_|AQFmR}I+r1X4(mcf*IYe{NmS6D7$bC2AUfmhn^lDzQL{-uZ=lBP);xpkqxrAH(2a}f&1(@J5XOZd*bhV=-2|! zynB(W%ol*R$)W>pvFDJy=^(nK4SSprl=w`0u zl(L;A{|0sOHn9tdaw8w5Pvu^E9UMifovo)nzk8Y+U8z1L@CKWzr08F$dD*<{;26|H zBK(Tv@;iU58~bvkR~a+F=|FLLRojhg?Nl#9T2P(VH>gdcgxA*{e`k0EKy!`lQ**UE z1mBC6ZRW|)-iJg(goksm-Hd2t*2_%!%2Xt0FhTrI29MriTd}$mFK9uSs6yB$J9lZ$ zGF9+_kMF)tFE`0>RW@ta5niwRm;JUaDuuu1ryV|7&ru0{T!WuVIBMOi`DZq|9P?7M zRcKWtMbp6b>tma{D&HzVg}T32AW^FKtH6jaLf)|gdsW5U|9Odrbl=%NPTlvOTbTG^ z?eoA|A0tR6cTNwP74Iz1a&h;JtqgX-%N4Jh;Khqqy^ta90hU?@=~FFbIktT_qwSBx zyl=i0vXPEcd#pvXL?|P+RbX`@E&;LF?p)_Mz0@8t%K-40l;sjsf% z+nDv#=#<^OHmJuEJ}JXJ`9j+i!26pubZSy4$I3mRt78C)gvE;=Wk7_#1v0oL5eQC_ zMUwPDZD%@mCacg~t**&#;D^Q5&NH`dmVa2R{tp)O&G&QsTzxgTTqFDr8Cpwxj`w2) zkCN4M_?28$-6AN`D+|xTF5NJ&mriw+2>E@@UOKrt4PbQmQ2!%dyfA zk2;S5U0bair^(Ppx+Wi}4nOWDy7fRGN-r5A2&5s-$Y6;gU+ycO-s_Vxv;|}0tJD&+ zE~bZc`Hwg)Z@xnI{=Ock!MCRodHHUWWW&FO?L|Smt-%whw2PbTKL1{g8V@P6X6Ynr z?qBLfzQcfGyl#im2U2T4zAm@nb;nFf&GUg{4GD26s*5v&dEu$&f~v>k)wKwxac?6t zx30%%L(evn+r@heD@`ZcR{x*{iR0Jo!67e5y^E1Y5-_7<;xe}2{bOq|Cqynp`!##p>2#?KgefXm`DIT9D@Uxj7 zw9=9t1cJsaneXgiO<3r}x1;Ne{1?VcADPE;{F@)XBn%cC%Z3ax-i?b}6679)zDf-w z!N3L&X@V^|>5|DHJ8;rR%?fMglXwM`7S?~JI1hVO{fwT&tTzbSH&?P6j4#R78QAv% z#kUWCS>m#1W7Xj==`On zQx3V2)j8bVZawwqwN(kKbCp8`7bh%fgPpN7cE|u3p=`JNM@{&#wnY~)~x4dtsV2gs&$0ep@Xz2@!cVZB1wKj&VjOIrMP1em?wA-080v4;- z%!?%4B*Xg|Ey%;T6q&V{@fkpYBI{kfX1(3p**s;QV7@w@@m^Nk5$x6@XTN3##_kJg zdoV1)K0~A1-b>^iaO)16+uxg6Bm#nh^#zLZG+$J`2V|TQAozJN$Muj4-bjVAaljAy z&lgHIr`ql(t)s^90gIyh5PL9sf7u(>cDzAlUi*EnI?ztSR^*`-1`>ku$7|4$z8`eg z-J!*)dq=0eip|0{RWvdLWU!wYQpnhf;F598l2tg~67MCeq{Zo9+l=Rc*(dNFWx)pc z%8EQqYl4eZQwCeVL|rDuRrC^}35mYTo<+i3*n%={Vo(Iy89Zf#?;hGjOpee(D1i~t zPH<$^)A1c(B%1faF>74X3h~M*->GwZm8i$~LPRq_W`ruWgaGkGPIUs`Ko`|@b+iI< z>u}ttS|dXd?=!Vit*EOb*~hVa1HylVe?!DJ{mT7|9~C<|rh7rpEwTE@*7 z6m1xCpL-gwhZ$|GiFf^YucJ7)dfZ#Rm#NfziTE9vlsQr!6W}gHBO{F)3;S0}Wr-OT zd~h-N5q6zZKnAUM#Y>h3xgA*MBr0C9xd5YWP!arm+*`L$V5QC>>*=NJzTnjwBl=We zRN??Hxpzqe({yC`^9qp#<&L6W;W7dQH;Ni~jpaWaTJ10??6)VhpC;0g_ zprrN2g$$TP8xd6 zxiqiWqE+beK`YGX65f3>4w;nhs;u}t^arIIKGHezrADVMVsTL3EwsqZwRD^JT!LH$ z<0_sAhG^B~r>tH?3Y4#I1L7?=eD@8#@cvx`$^_>Uu~W+tPSL#?>4olJDdJ5ZXt~(K zEDgN0o;15|)XU(kqBwTuff2fNpbDqd?*Ih-j@yen^z@8`0Xnb#A3Fa}-N=8R`$<^; z9TWZRssEIJ|Hj7utFe)Jyy*N>m4q!n?X&RO5=l8SCq+0MTk>?SESQ_P)q=YR^e3{a zpP_(+S#t3VFsegI<@)lvc|D-~~ z!v@Z1r-OVjLAB?GxkSGHqxa^N5)0EqRfiAcnte+%t-WyMr{Kl|YSp&+qwvo#N<00i zX>^ZbmsZnK_cXXMtL)O5M(-QnO3eGbMn;Oi@a$KWG2{ z^euD^?dWx*QB8XkLxx&qjgyhtzke64G<-e-!tcmjBrqUPVqb|F4>^PlCfp@c}s z^fOMZ=+C#i={_9)TX6ok3iy<$_4dww_3rgoav_&(AC=sd+e0^;KZ)>iD`tDJq${8S zSdCp6*WN<(wqGxA0~QF)K7D)>kZ^YKd+r`oKQ*{hbM#onE=1OjBT6Hykd<(Gr94J_ zsrt{9{2SR5ih=ITMa(AMSm56C0yk@HDoB4Y%z3BGsB}NL$}b>AfFde2c6spq!=_TZ z3+ox!9B;UhJ&zx1a7D#7#b!|V4E(kxI{2IB(w(3KtF}iv&Mt#L4L|y(cS1g*y%W8PgFw|iz$BW=reP#X>W`)uBeOGyFV-VtJO~ctK)7v>8`)>=u8?P z?6h1Q=P`WSvSGz83_andpun(q40B#iXP3+Tub;d#&jj#P{O$*!^+*J!&%9WNum|tq zqOD$)hiM+Xgwa(!*iq>5z`wcSPgiZ)61pCZ(~X83pIp3c`i%$l0R>EEOH@&5m4}+X zb_|;3X>#b1g6@cv#9RNl?&qV+!dsSD9a;+#Ma#vb51Z6m3U^yrD+erwye^3J^pTes zhS@~Bk$6_eNh5S^sApU@n4-JYL8_M+#ck z27UjeAtz*iqvjVAHf@0evB7%T2EK+VJmmYbEUbvugH0&OtWOz3z&z zDfE&J5jO1z+XCCN9YxFgv}C^2k(_<#<*&y)I~EKHwVSYN(3k z%qv`+0LURYfQ9+K5Vraxn5Tv9Ah%F*qhW4UFmH?Og0^tmko&8uugx`F=&LlCrLyY? zK4|Ow-<`Yvl!+*b2F=GRiRONRl(i4ZSOHaN6bCLcdXO6_4>>&r9 z?a05|Q}fy+liB$J<&No`g*s)Wzv6#Ia1_*32-EPcx-$k*5PrxMj(qc)l}@TL6o-Ud z3w+)c*x%i38S)JJ7<=EU*kdJ9P_A$OPsYB(zhvHJyWaqKyxBlCF^38PU46F1YnX0! zELCr%(EFtx<&CqoO_bIbo9NsYjkLaaDxp)&axHNri#i20bz#fLyAH2pLS2Vr3ok*$ zarpQcRC59WzFMcdlI}Bys>txs#B}+?MdZv?AwrIpEPB0l$Opit&zOTwgF1js%hs^W zKmNw9zu8IR`hEzL95?P_QAWut6rS{n|9e9Hr%HUy6D3(}&zFN+id!7Q=LwL}!RO7U zYBzT}D9XEjw29J5dTD>{g4DG%u_A4qsZ1x!IZ;ubDs_8Kh(hd|-nRB{(6?fRF}O+tuU^Az2rwhMQkwLk(O%O`&9AQ6-(0xz()1W zm&(EIHq|Pq9#hj=r2u&LiPS8F_V*_iNXe%s$@R-e!N)R-1!1Z8tU<13)O_4a0_yR! zsN9LGS$z|Z@jGxIex9*(6eZ-a7v!XTq9R_B{qd{zh#ld44yqA+-3PWase^EPDb@$N z-5k0z3;MR8SRu_ETy;N3D5Cb}xUeMR@3uA>$WM3foTCf}l!MrnCv6{j<^KV=L< zpwstCEOH8*<*o?^N$KQ1aV*xQVP3we$Y9oxsd3IhgOOVC5?UXno*oZ+N~J@qkD#yZ zFk$gSSY%ZLA5G0q`7A=3NKDg({Meu&3bQMd+LWaomwq|k#F8Hd+}tyHWKM1<%5|7x z&|wHaLL2eFV<^JBx^h!NX>ELzO+&x?p!LgrJZw1sh_6xgejRy>{X)BMwuxEBVvCum zK$5lA8+0KLnjSn9+y?#xAm> z2R*SjSN)&KexWcgk(5m*Q)7v4)`9t|AyapKps(pVN&c@2W;)2f`=J+M7~kLI?oY=_k;)PZ5$LD?`p-=V=S%(#m0 z`JP@W?;_OVZ?HW(;v9>yXe+f;w7D_IvU}}Zoow>x9w)A;NWG#`i63H$dM?XWQqJeG zZR3N-%AdoM`iXDVVQ!wiZlo&zk9t9Z`ZA>bHS>=z3vN3Sp%$(wP)px){jIKL$`&Hm z%J58_Syn@QNU18ZDd&{kTIbH6`?3g+0S>Sb7%wRB>2tYZdf*V4q4!=wSWD>RO2Dz3 zqP3F_02E2}`7H{C^#waB^?$N%(HNx!zWHJpP|B`@gdIE^v?NrIMg@exSQj)?`DQ!= zkJV&RoVnrkOwT$I)zpr0CuREv8Bzzn4qF-YM!X|zNx6RSaJN?}uaU%2hVYAaP#71I zi!R}B1btZgJW{GRzqT;Z<7c<2~cSqf}+}}rk z^RCaY+-7j)P*bK=#f4L$-nYJ41~IHc9{YTjddd|(w0@qMOwruHyFR)UpBFg-=de!g z)^RUk4}iXu(F=ULyOf^QnT(WyCapyy4NN)PIkzbMLW7R;M(}P7xv~`IYQw;^Hok!j*%hT)XF6QAI*TEZq!4*g%%P6TuMxcmv zO5;iZxHqJ{Afz!ra&+DWCadVqK{dr560q8m@l(A17aT0E({h&DN2Pibv+?=+MWV)l zt%&jFaKASM&IRn0poPd*~4tYLs75v7#G_(2EaiTDCKCOk zKqcQTul3OO(vyeDQ@N*OmQ(VZn)T)?^yZDv8f5*~Zj--y286tgJen$Z?YZN4))u`~ z``RQd&(p;GSZPdbQt;jxa(XE%wE97)J;|#KgWt z63VUPsMOpsN|45-fyghWXL0j2tz{mttLUItL;N5r*-LsrlQ~Y{B+&nIS@eji9I6(gDNINm$%yxy5z1m9S!8Wc|1H=w<&0dWRADD2)qGxw? z*U{j-GT9j#CD&F+$C4$Vhy*#S8mMAp<0>-X++d~-P^du7XEI`wzQi|wbIgFc+G^Tz zy8zA=D}V1zhihej*A0(k18AeeJUJlSze`yGX_Sy<&d0Q{@%-);AR^Q`n6ur5t8rN6 zi8~B`VZdjG1H-HfjI{yhq;GQoRFHGeNkjiD*nZT|qNVkiBf8BB8%Nms7+|s~!tj7> z%suIjSYd0-LPE^AV~i1!+^n)X1YYOvV_i*E>Jsq7F+lvy#V)O!vL=pUr2TSxkweBH zQc>5O7C^Z=ba1*fgt~@gK0ts?R_`s0`OM)slm!l~Iug(nj(I$3uIPHrfUE7L7pVM;J zG%!+1JwEDFFJiO2>hzfE>}pUdC#RsRW5I|r!R4yUb%SzNs(r}gxbj2g*^5pA*>ncG zTd{Y6-US?9#|lnV>@v-CWCsL<*jA#b6u5~2?iNio^L0gbL3!X5ZOMoWpZ275A9ZcT z?eW4fVHh_K-0g@v{!1J`M67rj>T2NU z8l8}*WE4s!3zI%ss%zFQ$TKsk91b^&-6M%aB%)Nyrc)VPAz}Rek>UVi^$xG3Gy8~h zNx{QBQ1XRu!J@W`Aez9b@1S|wY=+AARtwc!6keuxa!1{Cw6!}McHMI_PT-s(bhQeI z*I`HLZNJ9EdXtO7eC$?uW?Ha6GZuw8hC(9p3b2Z?J(}*n2f(+T@mWZh&jz(u0XXqY z?HFog&-X2-uLL%nm7OYjRd|JEaCc$B=eCalBo@a_U0As~eAP;a%SD{SpP)%OGMB>_ z1>DLlOv7qYDB~r?mX;kKAmcT{1uoz=)tiy@3o>nj16H>O#%N0q`Yq~bby!!UTzZ&M z10zK2IB8FlJ=V7fFkMoxLw0)4pT82kqV?VonkZescftuSSb+K-3o7d?qF;=p z9LBL-U;heL9H7r&>qE4ty{&yVzFcc8gBnng*_rv^RrW@0v(l5w}T%+6N00nRxv7+C(*R-wYkMqM>iMI%7tll{zz3JcCr*K7nyGCv4i zCYz7KK}YXn;&)$zw;ILw5ptt>ImPj(}Qm)=?4OC$aoK_DqF4XL>X|!Yc6MKVUQD2R8Rkym-}G zWcd2E{hE<*7&T20|29OX9?M^u@~67u2W>P?Cl-fyZCbAvq^DJkqg_=d9mW{@Y%(b` zaKyi*K8q&GX*TjxnhN zU(Q3U9WDNd`YM_;+B>gmu&%muvgE@MzMc3SdY2dQ_Mp_5AS;sj-1;5O(I`24nMcub zXObPEXXAm0y&X&GfBRrKCuq0hfN=a&wJ8~@8E~zb`|_q>5>VpnMvsa=KJas>@wd)5 z%i5qSHgLzA^xWANpr#G))dr4o_~Fyy_dj*vx?OXE7&YPsfZPP0H9vrg&9E?GjrnM% zHoqir{UHUXB$~dF1L~zcGw8476O*X+?H-dE3V0nYrw@bjaz6T#LipR22ne-K{b_Yx z@zg4uORfeLbYxuUnQ6jbc+uIZ_8z9rBzksGk&8fiuMDW>4T{yg?eiUIkMsfPGiG!W zGGF)K?XXmIZPm};0@%p!hI@yA83L*B-pGv<+0TQCdHFDML5mh2$A2%t`9h7ap|n25 z1LnFMi<{5~ZlpBEt$&rh?dwG0z-ZY0XKE1I?Mg=olKYB>AE+DFiOvSnPBjzgAcZ$? zmjfYN%ShjPI!~t<`$T`w=mm##{K1bHu9rzUQ0N3D%{uimzsjGEch$J0=&yx~V$KMR zSh%aP6C7414CD~$$|WCLJua|-Q?(|HOhRu_$ zo2qvY>B)4Ug~Dvy2UfLKR9`o~I%)D>yU%|Fie3CpAe~u)7e*$(u3U0^M9OM$D5n!| zFjUriW)t%uy$QFI=g@LMH?8CMOz$t3kY%Y$Z(f>yt4uKAWXzXn-3kyN$y?;pjm`kF z6h52DU0{b}P*|fnetXRRQCf=iS}P|=(WZ(wxQ3e;P1_-6o(B***5U9T+eo zRYmR|c5>BYHz#re0mP1$Oc9R3(N|p^nWEA@>j-~psGvEcuY}J|As(93Oh$-DTb}D< ztKM8sucN4l6u>$%AWnnp9^^-yjLvWdiDvQH1qE)`rECh8&ui$4``JXyE!S<#B>-P#S9D>;(wwjE!flY3KgIs zBC>GZ>rDJYGe>4!n0bsjRVmsO8C^ztAG=n8Dk+O7c#ji;y=S!CPeD~Yb@PyOIwp|G zvOz#Eg;i#Q9vBsApO`#q5T&MX+uTR1T2)Z1_Y-$hR{KU;v4)1EXs!oBvr?(Gs}E!gu-NQ;*NE@&MCB+q z$(7|3=Yr_VT?yUQ_1Fp1;W#4KKTuy487s~KWSdt~Ocp|@(v@+WHX$&xG>Yp^=q_L8 zTq?w+09f?*dBGxb-E@A$D$=1_W6n^dWc+<3Opd?n$A04f>9Dm`ZonV6tmS>4=qWi< zfh5)9y-6UjMzRCciBY4Cx6fmIn~eX_)E`_I2E%GOD;nwkjrAE38PgS0%yM5%TQ~rS zKzEJpui7yf997na6FyR?2^CHKR1tSg1TxU@9d5)g+~H`PowrbLV$*^zn_WnU=1|BP zz@Y`VL=S7ibAlc>7Obgm)mSv2(VDL%)havV!>0kEOj)e^m3Olb0O)}!AR3ho%o`ET zC85S+lICJ$eQb01`d#-p$}+{O2~F!n(XRHW`7z_D4%jAahKZR|O4br17^~@nJq?&^ zze9P+RKvZm`9gk>W;B@`R--rTB=_Rz-7dKq@x+!&WyO^%elB<9s)<?=oJrx4yT4TN}u`gk81(p#uG_LgvP=DIO(lk9=Phl#q6B z43g9_LmzWe9>(PfN${P`H)obttkZ(5YnjZ%?_jKLZXyk9rzs==Z@to#sit03#hwVs zbsGDhAlbklZ;0JB%E?m=LYqg%ySB$rF>mAiSdKp2QZj~)6zE=t5392`_1B$1z>IAR zadfjo<|6N5G2rGgH=0YtAZai@V9D>!X>tVar!|Gg>dh!PcOPPyL zaU2v6eLT4hUs#T!z9KLYtLZJjPP@)$t*f1>r&gHnGf8(Xbuoq$xVj$5m(6%g7I~(4 z(V0)dj6wH)03m)OAyIKrcnZ}>*R=S9o)-Pf=T;>?cwqdAalU{k0OlmP-Lz3kw3Rdc z4ov6E1C4(5Yb#1%16AiZUY_P*E3iiG!~VgSg;<_=05{3+aIGB2n%+H(Eh_0#7XF1u zI7zV*@Q!?W9C3dyFP&5E|twAkY%OOffBlh#OA zD|ILEOQv`rJclFM9dBIW0j;7H0cx~ES)bGX^I=&br#q=%S8%D4v+ARirrWz z1rZs2aefqyUsP9H?0WipX>vr&k_(ev)h{^-sNZv+9JckZvb zR{9(;*Xpeb-eLG?%x4=QoMR<5&9=|%2+U9lI;L1UqsRUfkQW}*t4~Kj?!7J}3Hwa- z#s<7$i7VPx7kI-1Y(lf3I^A;IoAKLUh_IaH!kft4x;A3mUeTQOmgrg$yVWOyhNlUs zLv=#u9bGrIc*2$*Sg5a;B#oBNw^ZQl6@jD<;$cPXh_7QYCCOIm+z9vl;3_G_peg7V zWtVGd3eWgEU0Q&qBSP$?euA1 z@d51KW`i>~sis#&eA~!|Zk8QuwO`2HMWhKzU=j(ngPaDcK;>y3Xae>f60zJ&hYoWr zWT=W_GPsJ7zs>k{pzI|J#M2N2Q{{PZ)PRb4XK^n-j ztI6Jw!WDV6<{Vm_7(g;-+H)7ks*-x_W&2^@1GG7*Dbmu~jt{3(moP;VQ%)Z?Dv9qA=|Gff$rod@$9z zc(2AC5#2Lt8~g4;9Wk<(yNcn}H;&U8BBp0wb&QjKxfDDM^A_Wzkhpjum)E@hsd4eg@vH>C&m6rL5nmo| zb~LlY#%OwE;lKeeRU4+;fO@@R{%eS*1| z#i!QW-ClQOf6Y;ILB|6Dg5;c9y;Y=CtXnCOlyIHR%%C#F=r99aW@Doxaz;L0FD@5Fb0q@_bbJ7^h}JEC!%+90kU+`+bZ;l25h zTIj;c_a>+MJ#Erk4j!GI+*yx-bG?Q4t2Om&sI7jDaG*6pxA}d z^9lY}w+OD~@VOi89+_ZXFaG{{m)@U67x(wU4GmY9n-V!$7vG?@{?cf6RDcc`q}1i_ zLd|@NKL6p%!tnK6XISIg{WoZH0~;K0O!cUFO?DDtVb4tE3+;?m5F;Z1EK;OT%`Wjy0Ov7F)JvlgiOZ_8}n(;-8W*;?nTCE-i zLK6Yb=YKVc&KzM*ahtvfSlbMhu^#P-J%dQLHq68<#dsU z4s3$&rWf4BhfhKUwk+as(7ndT(7ld*%Ee*Yb>j6q6X(VMN)=smpk+4| zXkf26ViF~+2VX%{?8=C~3IAXoaUHy5DLMb+?j~#ZnU6#}dIlur6hk}?&@kFDZpU01 zw!d>R#v`4XUNW-O8)boKuIcy-yK%2iLr9eXO+U#ms=rhn%!4@k^T|SK^Vu z`EN}^%Y%|g6H;xkXi$il98D>Cm+hJjx*UN0s!Zj5irA0@+ut|74e>v^se!sK=RFGD zYHCq(zpET!(U6L}$rFD597W9z(#m$5rnk;|Ekkh0Q@Bha^9wK+2czRQQPR;;*|fl^ zd|Rcb=**(GZhYfm_XBUw(CblwwRwf&&z$}w=_(5PvBRIKavpy8!)ql#$Uytl$cL9XI(0q_J zRVFoQfDbFbBOI_^eM2S-V5%ZUd)VHS(*q zExr5rn~mlgqYvxUiz68ey(1Kc*8+<~O53|8YN4w~ym&7J&`uug*nV&4jvIR3$E8P`SmLXk9yaqE_D#&k-prBkZ%}!jA1*nD?Iwn4VfD9r@BhAmb93 z=JuQsP~LMvGcUkmZLDA|71lq&(j2)x-S1ZdMT98S3e{czWdNjbYTXpuX?x*NcC*!p z-!Y&K6i`w*wViyvaziwa^<`HDJ6b%-@ToV56JIdGbp;L(sR-ME?4Wb*TWF+iPqlB* z@9?`es&dfC@a=+PIyP_KQ@EFEDoxwRaCIVGBgC1ELWR{z7fK0lsbFn(4)B$gZ(rO< z*1!Y{l~W-04--G%^~#Q=_Vy;cqM$rIE4#h28tZl)QmkLE(dU60r^v*dueEK1AV^U;9y{u$z}%ti4=us zKTT!sOjy~rICh#f7}I()1U3WIj*9!KfI%5S z3czRA=R$5hAFF~-67j2MyMA}}7jx_Xy6%DlV(eJ;41iJR{ z02pGg-edLE8(4B`>UCqY#hN2CNI{dN5#j;uIoJ;P7sK^*Z+e~CTgR(MEh%DIvl_#S zSWV=)B-LNI;d&Esh_sbAmUrZ~DN8XPqb67@0b5CJWAdr;shKeXd>)t%4`v=ib@XQ> zP#v@eHD;wO*-Qb}v-aEzPtt4ZrDAb`)gRqb{!{@*c|a@9ZU*5oIXY9^Q+yv0Jm(cp zcERC;a_vLJt(;7$Nh7KONu2PSlsuzsIZ!z`CLDFvsj;hC6+3-2cE!4!Y^7;?t(Og_ z-{Eqt<4v<+2-+BwDo*u%01YNcw!QtLjq#A{eciJ_2BBETD#SEH=3=MSuxkkOV%VJA z-~=nYv8V2r9`2?s&l_se^?k&)1vlPu`XHGBI}o2C+o{xG5>~@x8F5#awPiOLePJcKf?ZEB7u1&}*?Jnf{^u90fUVBXTl~sBVvB=smCi0w(;9|lBy1W( z)s70EIC0(!0Dc$5u4cZ=wn_FX?pvC>lzQs&1}xUM^7T(kD}iv)T9l0t?lF`eR{Nf+ zt^G6+h_>;J+4qK}&5J?uQR0%FaoDMDtPhzp1ng&F84PySha5c8}FT-y;)vPhzRV_{1TrZ}y>=BUV<%i?`XLDeb)V#mioG3C@bRk$j zslMT$%x^Z0gI_?0e!rZyJ`DGpON=`5684TQVOXPrBG6*oDympVB=l+Tub0&MZGeIi z?T$4@CAaH=zpzekWBp+rtV>}V{mnCbVOWxGhbN8MSLz7G&apnI**H?hWku7Zp2`<5c54K2<;O+rJ z@C1TuaCdhC!8HVTcXxO95E$Iu-3Ixa`|a-C-TN<3Kb^xcneKB=$y;w#*H~1!K7Srg zTwEcYOTsk&;}?U*t?Hxg9KEBaPJpv0FPdqC>>^Y9qdW6R{&g3C@RR%mf%ld@_L7Yc zh)tCeUMvG}0rzu~sO001Y(Zgyxz4Di5GzR+e2%z-u(ULL)G7PqdzI*G0n4~GfCH#J z(A5Ab5NOvQoqE!7N{Q$_3*dS9Ex2RN2~)mG9a@9Jc7AB-8bKfIKchbN+jC1 zCZts;r^X1m^V=ky!a3eDa45@Bil{NU7buO(N+mJ8iarc0n|ii@nyOEaMF%9!_f48T zYfH;&&jp0#C|ZBRIX*Fn=2eqVQ)iAt*A?!G&z;+f$)QpK(-%^JT2{Vmkffqz+EvIn z5BBooh=~F#Pi3dDSMHiqmR=-0$(t-Sw?@*FGm&W;j-l%rX zo@Y^*r{LGv;@ohM&ZB--;la1KUI=+86DV{Wgg{)h3e&C;w{IGNq!(9xvqXSHX&m6+ zczX5$$qRpz^ZWnQDT7opvn{Eq05duqI4-|^IQSNQ)y6IP@vlnS?v9lF<%?h~MaHtR zjRwBn?H6%fc|1*)y;sN?q)d%3e!ZF6mMI7Q@@&0Y7IX^+@0d(7dZCx=`W)p^3f=WW7~5y$J! z1?Ru=q_rXpu4)frm73*dW7*0dXSJR@z!w~C=m(yUodfVoDq7^UfAumMyd+u2 z0##^BvDZ^>X?;W0C53Gfo0E`|?bXYkB=qm-k;KNbvB$yT_HSkx zK|QycNd3CIiJ$tM5#W8c07d7l+f$NzDz&uua64HU{1KudsdZP;_!z@$)I!$lT9VWU zta;*dYJC+bbQwf0L*7oxCUbT>7~;Pdg$sc zo*F_`D}y+@aGT$2VM~>{c!h0S>}@*4;0%BV?ZEWmXPixOvk>6sd4{y_hM^{IcyP$asF+vN%-keL%^-<-k}~x@zG%ysuR#qDJ!CYiW7x(st$+h&;Zm?>eUz zj>^oxKTJ#IrS}X=xdNyy$44K+gMnkt1NzraEw-!K9$vBHU4LU%jb+-Of?dpDIPi?k zdj zsm?K-ofus%`;Ey32^j%8dOVzC4C?P3ScMl`#~S53|3Ub;+3B{l=1}IAa!4!lHVgyo zKytHnTIVrUq~)-fok>8ETO@E(0>6A4Q|7*@7#%FJ9!>z3(smupIfmSwqPt!eX?yH; zuiYtC6x8QfsC|EBEV8n)h8xl%=iGCujsscN01uK0p!hy=0bF0O8d%p#CB zBD#9^9&Sjj?K{Wr=DGclT7<64b03qHxpGgRO(PY7nMxv0o!bvXay7;F;hT-jZH9dt z8xmj{Zkf2*B=v06iq@YEbp9IC*=~C|vBxI2ZL!3vuB-(ROB4T0m7q^N;TphVixR*) z!19m*DoOpetJ`;`!eYMmkYh-cvt1l;Zfk1}gdFDM=o>*6-Vqp+iu>R)!Oqd79k`o( z8`UQ%PRj*nvlzYP`JEr%jQr7q?$STJ)9HDBMD|XMlZ&I0J4nha8+a|@Rx=Hq!&^e8WDkDK^nZNh%q zXz5#xR`Lx!QT=qzrvR|xLqtIJ-AhOFrDjzxo+3esL;1!fjt=x1Z_k;G^wQe+3*mKM z;$VpkjmGkFplur_^f_btanW^|@Pz5-Wq>;Yx0o1JJT-DMCO2O3+T{UyD2ZNh{%kjP*`$GX*QerIWjcC%ZsoCYlTFjiO3MOh7kUOY z)Mj;iP4D*eeJMvf$rl%(Cr0@G{bj}Rc}cNU-mH&9Y>kVc=}%YeXwN#2!c8wgS4+eR z$4P4Fv#RLPrIxGFwt?o0dv$a?`%XL5s!N8q%(0j7Xv^ivxr}|BoFO@nOhQRJUGju> z=t+7%){kpPEf$;T7>xpNKR6WFH)FN8ni=xMFBrWf%xDwpU zF__tTsWkTU+<)IJ9I+BqUT>job;;$d{WxlSRCqGqcwE@4Ws-Eo3NnA5YOy_Wa)eM3 z;FjwKXGb~?g0=|hD@QH7)uqAn;IJLUlealV;1cybLCRyb#{RWNaqw4i5UfAaOxNP- zH7es}Qvg_`0o;1EF~S=HquiFoWpm8xA1)66m$)?+)^o8?P|a0cb#iMLoqjp@%=pv7 z#9YlM_wdbgV9Uy?(bVY$nRkul?ET7EUD9J5+Sd1iG*a?VEevvm3Ag56JpI_*|7GL1 zv45MIV&~X(UTw3ebXSdP|IL6OO5l*Kq+!1Pn95-Cy}2iPik@&dWVt(gy00D~loAL@ zRm|<2^ySQhJ`MFEp9h6b7-BN3-ajp7z3uR-;%aXR3UWWI_==vWduRe*SqG zYN=)eI3?5%@0ud#+R1sh-ET6s&9o+NibJlC1v6MxV804s`JDWybGVS-HfD970I@VY zGF+uB3qAe?arAfyqMnkgDwa9t+ugOwv1xsR9JMHfoa=FyyhE>B96MX5-?vH!yUb@yPttFL zs^f*P7t_ykZOlKKTRdcc%0}Q>_ z9HCwTsx?1lDRu9&Dy2d{KIT)0&v+GGf+x2kD(|ZR;{naGWT1+!epJm@z&6w%t~!|x zvHW2}HR1HpyfI+z#!45{hLs9h@0lxpJ@1~PpOniv^WJWoqn#ml|In>uzm>#-s{hLG z(rfPeyPySCGOvutBgsRXL_VGO;B$5-=(9Ai+8w7{-+MAT zZ7tc2-HzHF`E9fqwIY`p6*#`EjCn~*zTydeMQ8mW9|CT97&l0M%Fq7MggR`422hoD-&boskzU_SVx5hrfeO8K(UAb+XV9gb(tYhd8$p@3dR z`~^3e0$Z1K+6W$5dX@jU7uNoDP+>B>(5=wpMxjO-R#fP}2c}4Ru;o_fVPb(ur=JmA zeauIs=(MGWYT5b?XdoM7U;$T?w07 zj{IfCCL{4zmr3^pL6iv|eR_PBmXiZxeo1`=GxCAnO5Sus>DcS`%KP%LlfSmnR5jTNDUWE|Y3H#FhEyb_k{4&8IOHdr`j1zt#G1|Gqs{ z@yljkK>z4|ulVjgcIso8kI-fCxP&27VW2lgd<6Z&0Qf5R+a~d7-4p5XuT*^VA@?}F z@%RAVh2oLMNNS6T%DKRn>4$h;T6d<-Q0TI0vPL*cKHeMUj`VBuDd3td~R8v!Dr!@Br#+4hH4h6_8rrWybFUScIc{ek;!*dT&ECL9I=CR5e1wLZFq2L1x>e(&RUn(wG^>|nRj zPf+NAs~ZwK{S`?+d%sfXG9jOFSwj zCCUxOdnp!3_|wk9tZ#H0qEqEH5ewW1yfX zxkuN_+Nx`0IInQ9@bf(SNQsV-Yro*)y`aEmG5&?GBH z2Z03fhJ@?PH9}~1`btemGpdoVRd2RW^^f+x4mk78%S8r@&~u;>3T6LmMb%L&byUmT z3*BGk^g4Yk_T(%u?sWQXY#SLWL)#r#=mrqy;xIO@V}zxoj4`k-rm3eo9-x4Cay@Go zM&vm7ro(S3NMRNXurMR6>0crpHjchX<8j!NRAL2hY;RvDvKS1WIfwGca;zHAwIl0t zb7E{`zkc1y?0J9V)MUSVrhM1LgW*w#juDt3bW1dR{0$izM^XHX=RW#Kyzri=9HueO zP4?R(S#db%mON)$1f)g`Xk@RxIX;w=68O`e%Spn5e38`O%I`2LXDY3$u79H^0eeLK zjQ*gBWzRjveLwKxX@Z^kO{6|neZUK(as7)Irn}@6QkuGSZ4vJcz2}3EdRuAr`((wg zZk&1Za9p8#)DF;=tnDdn2Vc}-XDA8Wn@p~Gf#KAR3B{yjuZhLtO7njby_bbQxG?J6 zz4?`jz?S6wprd%o4K=)Q{E?9pEjnhmRQ=^>7Zq+t;d%b2tj4COgr3ieiUxGhXU?R# z{Hy9>unQ78lY7%X!7rtt1(h!4c*#Btt;1o_~KY1rDdYR82=8hM@?(B+bA+&w=Fy~Dl6a41?%!%5g5YuX-xV)Bfr3t zE?{R7J6I9!oq^)K*1UkG<6=L!}o7<7l$Z)%VAz!eOR% zVD;ggrIftvTs84=7t{o7rN}S~Hmuu+&vIWEHH)Q;?66?(9?PlQ8&!-6O>BU?l3cGi zozQ=&1c@()U2f;ZAh;gEOjf3MMa&c&9XUAJ_jqPaU z=2X}7t>GSlnf+2qM!qYR&E#hr)1KhLOvQSQ1gdCCh!7sMiTKM>8D8BH8vYX&6rw{+ zD2!Ks9M*(TtWFH@H|A{6s(YAH+%B(Jy1=i0r1)D3rIA7i>9NFUBkbD4CY5wcke48Ak8#-#{ZmNsSG>LV{=h0^@`&YzpHD>mnVlKQg_y8a=t zFAs&;v4+0ku6REG9Q-V*ckbZj=YAo%4553)%SXjCTNG&0d#V#%^RTSX_d1X^!iZJr&ZAhfj$XeT zgBz>3Iuh=q;B|zpwDF2aq(xWwg2xhD`)+Ae)vr_xKI^V$w{^ena0@*01v|oq>oy@! z+}%_GHj^sy_5V!`lz8*v{cZ~uwQTZXu1}(=;7xv0G@YpF2#r+BT9|3H%8B7`hd7fIsQq1CmE19YgY2-d)Miy};_ zzv|XyToDZKX{gIKog8m=$>QBwvYy}YdTp|bAJ(ozV*0AG8VZU^O$?SCbEh}zVLW_C z;er{Efsh;u;p33IjF=tG3;f`mndKu0g}P21h(XjUOATA90-FYso-!U6_@n`2*z8XS zn2=I*&LcClEa%?iyfINe?g9&3%yA@ZTvf&|q)cw}4cBBjJMhyO7@`Pm~+h!}XN#q>v&^2z5 zpa<33fP_U;q&Ld%_olD)t|?Y-$l+!5Y>dgr4~%C_6~vBHipsN^vdXwhSIeVB+R>@0 zkLI49t&7A;7=LKaANTc#F8TbRCWYY>d+U=9b<~Kn;-5AnU^!J7v-x1F_~z}~x76v} zv$M0Y+as6XVphBYHrp&HZ~CJsG~?uZFJIuK|HVZ91!V?$A0za?xL*9>Px7iapH3ki zHp?R*W*U{m7Q>SiacM||=eRTZN#tu&SOc%p*ZD5tRI0K&ED?Vc6B0WSo=^Wpd;ib~ zcOfCTdUPu0L^8l>dHN14yrHJf(`38EQq9ZET*F7X(?dc+(s1H$JsYHx!wA3hK2&;hZnrym(Z&YkawT1wNMA+H;9Re zT`@&jsaSaEe4i0Ls4XrohGV_W1^#IcjL~rRj(>Wma$I9FxrM`zH~9@KJMtg5{^zfl z?*JqJj;SdNOUm~v^Bxb(!p^HW{>xifr^q>$;2JZq7JL({ip-+h5XF&4pXuF5wg`SL zpiJs6Oj~Apec_AXmI+#xKX&hbb~O+kz&xj`d(%pBuj1)Yt(Ck4#Y!!d305d6DT`J( zLKIgh7I`Y7*48@W!JSt~O&v^@3-!_DQLL0|5`Mg zE_O+{xw1DK-8(xwGh>PxBTMP8m3WCGvK>t==lKfj{IDSTtI>3d)@|Ilu2i$BmDz9@PuY{O zosbPD=ig)p_gU&Y>AA7>K7E?tQSxCtT55qjqOtE9vlf5!OuW%B^Oj3sF?b2nO2NzP z=9dK{r=wR~tL7Rmw~P=uTq4p?=l* zb_OkPpIyNM7y>o=g)l*X8lf=RA1Law$zFt)y%b?DY zFydMKS#Wm5<~^3Wjg8H1cMuk1_7Mu`9y*U!g)jKfIxL~7;2(Da}2Jt zT6P*heKhOyE+G3?W%z&ztd0b9+J>cO5`1z1o6>jP4bbprNc3?2#L?3=x+&+_t>!PPXrc*|+3VP42Z9mm5XBa6LZ=jrkP)agHi zby7?y?F7dtxPk|D%yymBvDDJr+gtuxzO_`R_=_jgslqABqaYgh-^!a&VVW_sZWve1 zoZT1;v{K(PGA{dmqBon&9f2=)Bw=;GIjLO&#bo}Qet*D$W@%S2$gLwXVzAQg(h)Ep zT0C+Qca3Udj+G z<}})7y*CASbGMM{zo_|l?TEyKRvsTw4cet8Yq_!DL-fg&1?KH3Aq02Bn2_BKQlf7Q zPS~AeqnXT=Q6Hf_p=M!a^*X*nB*Ph`?&e|#!pld1l{9SRUs73LCrglG?i?#9D7b~; zq%Q4-^+KI|N>`bmbor+$l0t`L-Vm0gtQm(4SPWaOOT3l2d<*?X8rZvFSpm5EqI2*0 z>R0P+Olq2Ga6P(|+K!0+4`2_-6+`W{=UcqhG9AC9QxD|Z4ic0t_~p?tz3KnOm@u$& zE~LF%emixiY(@u#bkO80q=4|DbzaCTn1M?@LYyI+8yodkg4kzrf&m51I4*hDSJ3l= zgM%SK=lV+T(J$ULGlW*Wo_xAOCFHosNMW3-wSWk$w@#0^Y^rlwK%(Wfs z!EV=`b_WXQ1$iS>{{TpcSj>x^+h^0S?{Jv}V2-{PXD*|o=|hRvko<|i(#!o zs39)ywKT^}`rvNyovHMh{6L{S%nwqF8D<;tG4qj)Yh9Z|ao5T0W=b{cLFJh@0>gI= z*$ee^CCnV5A@RrLTVdVzeg1v&tNiGF3#s=jS@`?M^#isdVq!h)weJLQdP|vu0c0o? z2ta)u=(4_-wNxTX$8Rzg=kC6>cP_t#-oHt*?Ms68DZQ$z2`)ou_J#ixB3^~NB)-Mb zN}x7~o=e1N)*<2lMRm^GJ3)q)d|U(Ckp2-XVZyivW$Jibwx2IHZo*(k)E;#X*&~n~ zhqYKdy(1r{@|*V1xRSA%eh#*3k|BtbzG!x$VRaPlxNXy;3=VQ&Aq7F3+<4vOxSf(P z8jyu*jKSjRc-|gdLEyhs!`T6>=fQPpRKK(_>sKq`xOAER$JaGU*oIWfnRh)ii~h3W zhD?m_9Vy7z(w??1)1pSCve&T13A<>1;2-SB*MiAI7E2K!=mWh*k6~d`rDY#?$3ncw z^dLB<5$5JfhM7dzM|TzL0CVdGeU+gk@d+Bhm&;5Sa;{egF4ajGCrrZw-A*`kh4b~? zqjwcCadGpEi~--1bS-(B`TZ+Szv35YnSQVYbuG57( z7?QMqWX;%-@(4I@RIIqo<9<&#MGe97;54s#bYo50PH*jhe9Ix?sIA{yNz98rE0YiZ z@uBTfj~IRA7k(e0p+T}DhNMoSj;tL@Q=DvV7LVov7E5o%7eA6MY2ZjM{GH>zo4PHw)NUWhh^e*pXmu@J7MQ{oE zao98&4#o#=Xp=Xbx~R^`&=lg~_iaE3!?`_{);0nV*_;+O<6>54=jYd5(w^~LPUNF2 z+P}R;VtPk|cipx*&QPo?_|BhkxnXc%HW9w;5tZr+Sr4C+-OyTLy#(Q=TK!q}_+-+r zdW7&FjlJa1-*x;a()U9Am5qupLyU%l5g6-T($AiCXKSyAO)Zl>s`4uy3M9trpoq4) ztQ?h4Zg}EPH^Ls;RfA!v&E^rnJGQone~IA46znlVEZ5Ih%lpf*s$F`&SJGG@BKIe9 znBvCFL;qv#@=K3zx^E_|QNlVb<5e`4Ie$(Ng{SyUr|V73)5yPB@m|DcioVy%CWn3_ z4tUQ&8Q|+fLnR;_a29BA7PE{IjYjA;0{&m_@ceg2j)DqzKLkSLid{!3iTg7Y@?KDDERd?YYbajI=^-UUyjw_)MuznL0 zd1F>wa$|aGsC10G#$#6P-`u)8TpjF%@R=#g;)B6B?Z<>^!L>q- zr%+`In_zGJhT661;jzWYgkl*SRwCp`EVz0X&e}#dP^CTbs-zw2y0wMhJg&_0>Zn^< zN=qdEO3f-QTSZyJI$}cK*v}&ynQG$1v8Dni8!U&zlhz$gHcT1uiXX;Y{=Mab#4284 zF-2U)MF+sQQ|3sT6cY_};6^RBzcDS&3=i_wM$Cpd=nyDZA@H@sAdjtuRvM9zs6Y@EqlvsKi8;Q{4H9fV%E_1N5+M)Fxdjn+}XZ{ZIhaQm~4lr;i^{ast;ao`< zGz$F~THQprWm+wFO9`w-O1T7BVL(Es-3#`oC*e)vrvxA?&@TN7=LvKC{?5tLLSOsF z&6z-x2q9S_DZv;sxcA;pi1DWdrRFrC@cemH85UHT5s)~BMq1EIm1rG0n6pV?`~QNRznyF zY3Opv8iXTb^)Tp)l+T5ReWM8_CR@%=x`VnC1+7=wr)9v7%BW{?PYE-em&eObBuBx+ z#1q!hEBN-h1_qm7;#XC7fvm2cpWrYITLkqx<>h1>cwsTJzXP*Z)X)#cV5Jv~VG588 z!zvdW0hUQ;<~xiQU&#p$np9Buq=))_Uk2m&auk2y1W10@U$LJfEhI9hYL-`WR^uLt zT`_qJub?gQ1AONJ`c&BVC04FFK-ylv8t*wyZTMjN8(uiN&ZrszwsKr&XQy)D`B19| z=l4Zmetse72` zwkC{n^gfZYUXh41X2xzzZC%7m;uwvH&9f`K_NxDm5WadhC4lkDhX&DkR4CL|qi|Xp zdgIGuq=G46z8Tu$i-tA^qSK6UZqV^ow;OrJ5jr$IE~l2UAVHyIBos~`{Y>XHBVK^F z_kll0m$jlB-WZj;1rj1yWetgk6V2vmDYjt$lTXP*beHwzJ)IX87V6=3iAtYzXc399=u&AvNQRyiWQ$d3 zE4|O?s;sOO=t5`%+kSI9Rfif_$rlj&GvxfwAW>|E)l^*#lNxtfe{u=$0;#gCi=ods zQ<4Qz%dGZz7&4*QZ%L1xDf*hV-6F?BiUcXaJUmE0w^g)#@|liuy-P*~DclMcm)0~@ zoZgwn=|NSI-zXB$0pU=}L+8qXE;>mMEQxmlFA@wmHMj(m5-Rj)1aI|fe0aG`wQ5T_ z`AT&W-0cVwi;H_S?i3cDuCwbsDaGufZX>QO(ZYG`w7UXTt5+s^S_feaBI9!8}5pIq<-a) z#!yfXgs&Q-PZXy|KfgPEcX^BMeqWrKnJLJu;e%-D)TWpyu`*{?_5iRExrkU)L@EF$ z#KLJa_r<>rZj8k~O1X`4TpFeDAK}_7L1=Wiq}HDuZ{O{T3)<(8)O0w`y`M;(^bnsg z=%zik>sIo~wUPqIDiNMWpcnU7k+Vm@?Pn; zc^D!MZ6Ja>WuWuS({+4YyzJn>T)EhtjEll*7To;^7mY^Q7FEde3)bh2Law_XzUSuI z=Fs66DsV>CO>DbzF;W{A>wPv93>@2iPGQh%2bI48Uf^6_EK=Wo10i>7TR*me0Qrbj{2Sp7aRVx0<)+B|NCAI1D!zbCmJb8^M?JRY-Dg z`c80VVGH3njggrS*C4|nqPQ-l0K)T6&qN}7;jjF9-Jakk{M}W+Z!9e#Q+VEP#*N$` zg*CV&;;w~Z>wWpM3I@s+cg98n1a+om13$|_vM87U($qBcBw)N?b92+9{?i#1Ctfd+ z%SP04`!$s%+l=zmK^(*X*o#2__IjNhZgWDZ*{H5lqmM4-p4lVnL^GjpMWU1}uGeO; zjNvMAhGb&u3Uo}=T-iP`gFL{;KT+IR1H#1i@8AiVTfbh{-a%U7AR<;-#B%wbcD0-e zx0-A(aWZ;eOC@{~c*RbCx;0ZS*a8qNfl+{p;&t3!vW0e`;(c?<$;)$F>j*V4FkmlE zt$osvfqOEK>e*7w=@r_mw`QPpKiX^nvtEt<5dBb?1Q@1%;Y{D=2tcw1J85C+M9t*RumjFdvD$1K`z0FZGW+|X@2c9*Cb;ZZe^Nn1LoWY;H zU-A|Z29poXYPEGI={>YwsQpK%+9jZ~-iOVh>RI2ebod?$$^!AlFl5464t8ENvMT`n zcu-~O{S}G18rGBp*L!GJAsgJJ>LzI=w&t}2eKdEd&RJ6l*{d@s6BLO}WxL?aDEa&d1Zn1q4(x@?sX4gZwH-mH zJ59p4KCCmRph6O&p zwzKfkiH?pAeWT;MHrIlCXA+RR-2V|1b4f$&)nq%o1m_6N!XNsz0%==C8V#GqV{4Ctc3C}xqIocx`lpr`F+PP# z;umi!+u+-TCw_=phf<+U0a-2t5dh!g(aiNY)%1q(B2%1LWryt?0?cH!r>59z=Uj)^?lIu8#P%stAC^3O%zjaQtk z_a$78CAt6bG=FaG8jzTBCQ84{7y#^2_Vw|y*PGtuQ!Uk(X8tzdQcc0m`g=M$FvE5N zBT~^RHB;m{bBC8NV3Y?HF-5(TXdy}XvQCD9bzQH#`iD=ZOVry!Ra?1*zvL7C6FmHp z(7=1nt7B-;Cil{wK=;m~vfh?y;UC8{>&f;e)2lW)9tr^Y9>Nw(Cj}ebZmqyuHPM#y z_z2v@!m{;Itd+SvGefR(9se809So`HN}Z0c+W?2Jx#=EZ`ftg%^c=u7*8ND27^7x9 ztKATL)Y%-2;azZE4~q>L#`=ac>GVbV(fiXqryMMiL1g^O1GDzAo;ihTj5fihN#Sew z|BE$;hoz;Wbm_XT_071?N#+5Mg~e((e8YjJSt#+ZQw5SL5m0KhDPI%rU-T|b#t+mv z8!UABqwX&N+jc#$d7n2BO)YpBQ=b(St=R9O^6j3>f1iJYhoq0YLI9gG@#r%(611|W zgJPZI;ap00a&o89CYP-0SJ3vITXT8&39*)0c;-qz?f^paHwAl2cv4vR~zVheIzVWC+N;f8{;$V2T zEI+Z_BbO@iWH1lK!Tef2wIV2%vroIJZYY%nf+~QI6~2Vrap# zS!(K2zYLK8Lm>3Iv_R=2mpa)2pqjZ%O>c38*n{~cmpOiy{n3PT#O&6=47s}5&>tNz!{(tAa4_w&k0lR@hNQoeV{QDxa%9xY5 z=!CfN0Y&};VQ6m_G7=JpxL2-|sqV$~V1Iu-E=7&|Vxz-0gyFcs$&kGE1B27emjGf% zCjWYV(R1f58z+=gW!2dy6#tt^g0*#GaP5wba+mz@9UQ-o)KB-vq;)A8dQ z!{luxA3fK)&JXS&1sgP%;gC#FbsueFkn2X2xGdR075=_Z?_F%4f4a3VERcTX0o*fj z7^0BuVngtrM7<}Fr9hrqnuNT`_&im-94x&3u!@?@ss&LUpn9`SS2jWz=A&1s9%NS2 z(BIj?!D03}W~={SNB>6=k|h~Xn#RDo>cjP}sJ*DL@J=p;=MoFuuURZ`V^AQCjmD)& zsf4Hf_%?=F+6l+USqs{SMPQ`YOQ3LftL}jK5~CO=S?gSpKu?lfotRME(EAACP_n zgh_67Ai`irOZz2gxa6KHcZCgQ^`{D+^C6LZriSP+>sP(Et2Ln?%Vcy}*K2;f$-r4l zcIDa|zHb}7Qmid2(Qfm6i!Ql1Gi<1+`8W9Rr-9%DNUZwpT>4umz5FeVo~7c6Umi84f6%#!eg&?!8ip5z0Amqg+^W?Or-EDcgieP~t(M>0i#2;;w zfC~l7U9OiU4ea301@10-W}=-P+=Kr^=KXoaKL_L(00IsQy%PUTr>MQjVZRm@=9WgD zjy_Ojq6ePIPp${0xivR;s~rIE#nv|Y;op=#ejGOi(qi24soZ$!-0@FNT+{UgXV_3u z5mkq^^B^7ndi-@*FB-K7t#!KxTJV4w@ZB_z)>j9!Nqp=~FM)*AAM*3p0V=SsRxf~} z9tz+#nB}j@aBo~DQdV2lHdsjlZk-l@m?0wdIcotS50w)whFhWoa&XJlt@!}?!vZqC z6LEKyFVZ4!=cI6?jJWyYUW&d67nHCzf9eV4-A;ZrKCI*#3}o*p0I`92Qb~n>TNtxQoRE4o9v8dsA@LaCv?v|wVN&rR($sg z7Z1oRUl%2fXMB2*C>XfEK`MO$`%%tFCN$J3E&ZJVriiyOoHX+vDE6W+wcuzGT zjy)-ZwP8Ak4tjznP(ghUN6Ju+D*~v<-1)}~3%s@wD^hFNG;N6bF6vhOpjVgFE=G-! z;k9pjgb56EUp5;es@CtCBkud?%Tit-8zIb*2gxvy4WQ8>TIoX+hUmcuYD%cNqboIdlbg?aalveG_fz z)0WxNFT4>R=fik);05cupAQuBe&nxsedcgs4DgH|^sI!9{!encql8f5oZl18D^LEp zk8ht%Atl!86cZdeD0#6+x$I?e>LZNdG@TZ4K#z>T$%MIQ6@bg2s*J{D2Vw;N19vL) z`pAa|qnT))IDnGufM&kD;6--WpYN@mLYt)`z?!xiST^)U83?X@b{AU- zn8zPAoW4mTMyR&jF3QNc*C}j6hn*HM#Z%Jrc9L-cLI0tWZ=QiKlC*7eKtJtCl`1!% z>Erz^yz@eRWq|IeKbSUjx#M(eYwKzN)syS}CcUo4=TDzb!2l~!wm?*-drswZe}50U zB7(xA$Y<@Dwe%HkeWiCVQcF^ycHafiKoRz+u6E%H{Cjhi?XY;E$o3#Sgo|Mr2ck#9 zCzjN-mQ@Q+QyDZ^p464L)114&*B}CfUrO-My!@{b^e;X*5Ihn*y=UQk+tS}#jRNBi z|IEPuSqO*loCx5eodcUgulGfGK~?Q}hOQ?idQ^tU;<4y|fLJazw!Ht~#1f1D7Kvds z0|lr)sj^ybb-|1B>heqWBlGkS7KW0__$*55345SU$?tkGlLDfTy8)SB>oif%bL>z= zC#>yIfEMneY(qpudujjx-Q;MRj3^CP&q}m77mQ+EH!3}}{vB$`*56e4cX8!=4_Nj- zs^OB=%I|g&p`oGMDh0C<@VbvH(_C3WAyZ@X-`j3*P#&PAPv*0@`^Fr;q9eAv1RZ{G zV2vl*J-&5mCP?B%v?jLfk%{L4p6Re*?P;KLGE$fACOMr`3Z3tA#hVdhYEG%OnXt(UbbE=4GJ7^`r4w|ue<j)oE-oz%x3%ylpNEN# z#ig3yrh*U2v5IB1?PD5|)#EF3 z^Wc;19;012N=Qfmk58{!<9Ch#4G?n4p^qq-fU1(x!}+>n-_w?&pnV72Og1}rqw3dF z3G0TzNkA^YIuPF4Gqw=D{ab(}(Q^y}^O=+V4a|Z})v7x^OM8mZUhHcF?L9X*8`Joubx~@V+-JSjOA%nTwmfKl)rmaQCe6Etf+sn z&SZ49F%a_dJuKflQU*>=Cw0q>U-;{W;aE;7&Mo|gH{j;wPNMDeHsiaCUG2)YX$rm1 z@xL1nj4~!*u|kS@=VG9a!d(Kzv7ZAKMT9l)ah@f%n zi?`mMBHJZ1JBUfuFC7}nEf(6SbC`p@d!}Ui5+RR+WmISdN=Ocbu z+?PMH*;46l1L&Y+tEcj^a!IJ$#FL6)>?&L+k-o=Z4?NL)t&qo0ukpi~OW_^Fd z^fU9vLuKq$p!X9(B%v2wsfRjyD}~^7tY& zP^2m)eHQjek%ckv;`){y@BzzM^e_u;=XAiyt|$tqg?WY-|DjANyeE!gG1^fMqoSft zKPf6seJPHwb#H8?iC?MH)xC7x&54s`prb1~?MWvB!MaS2We(uuxu1UsVbPtdHmRjm z5`&eGZI2`qnLGDhS-O`lX6=FcREL_d(Y%aSBHqR{aO+YPIyMoP=Jj}M3mR_-;-LkR z19v16jU$!M(yo`Wm0}05;YOv2gas%kTbK$S7{+gWzOY%_OA#X4J?dYLpjK%&J>$X^ zTtFY6po1K_#9mO?bHRi)b!o>eq+!*JNuqQB#Mw6;mRoozjEP1)Gib zyS9|)UX%e(5(I&70MNPL`FZLsKms7B&}INR>80#f!TR5wvgYdY-?jvoP;uHNkEtv= zKHOc_25S^3VcN4ty{2>O3C00)URsdiuOKEy>N`-t@TqQKO_rgYbX&{5XJl!Q7S{C0 znS-YtS_AvX~V3ghI>v*}Aw;Y1k%(SX~$o~}79=GcPm@mg;qOm0qA zEqEOdZrDPXdlyKK;HE9Ze5{}^bf@;Q*LYK_Y4n({C1XeyN~&_|7o@$3Ht;o@92aB9 zK8i*YiSYR{yZgRr{SKycc}Qv5s=7WmJNsTHx?GWY@22!axxr=FGCa%*#;4x#CG+dI zD&;G&ALs!);_{j}PzJ6Z=9T7xA5h{gCyOi+4Yl;nn@1CPE`(IyoA(xVPdYOaYU!Y0 z9FaV-5m@rPqXlaH4In~?1AxO^joI~yLijxJw4DQ0P6w-yyBL-8WZci4i7a$d@w2I4 z2mU5({!{0*!CGQK0#+Ehp`n48so(8Xb$@ibm{-uP?)~C!=jUsl`sJm0qlvR{Fd!*f zCWZ|yoLzhub0~IhEyfd>F;UW5qY?$WoD{B?dza@UFf$A<+hIdbcy+495($6>ZDZ5- zhPZt-@P;^fKA$YvS?Vyk0@|-4M67eFxdqV<|HJfS< z{NuSeJDc7a$!>>1`Xr+jUxf(1vqopv8tD@%mAb$(iZe&}Vu1;e^?Hvm6V`lnI1k3( zGz_f+Y6(Yawz21xujj7dwZZsXhW>W|c>wsEwVtiU zK+0sh(f<~&ha?hd6B=c#@-^FLqr!m0Vw470P8X>q0#U&4g@@5qHtxV=& z*)T#MlPfFQL;f?z1&`>5a-ROC_5aaz6<|@eOB;~x?nb1gySuwX+NDdnK?J0`L6Gil zq+7bATe_wDfAM^AzH`p+<+az!;_kll&NI)vq5RjqccECcHUtyXai2Tvao=1j?=f!^IojCndLwq@C=`A+xyjs zcq&1CyB>{bFF0)clNs2ihWidAfpzN*jn)1K;s8jZ5KloAQ*c$Nu;mY}>-Q(^t6}Q? z2PPKXWjl}~(cC@PaBpr5>sdN;l1tZKiNh#6^LZqxA(-ULrTkLHEGT}_kaEK2o}E|m ze{~rc9zOJ=v8K`Cv7ah=we7O(DR+kKvQ+Q#t%~7tl7Mwm6q@&PpJ1PWyCCSPc;3#P z>S&Q_2I>0jyx_R;h^nu7Ga(^i2KS5|pf5S!w>@4h;>$X8!Jn4%!EC@RSQdY~S`>() zU-3h#47Y62Xu2=2c4n$f(y~fz=#os75h? zy7txRIco(Zw=tAmbdtyO2OdSz?{`c1t{8@2LeR4`d(0TCwd~7Sz%pyH!e)Vd+LiVN zLt#D;cxh29VdzM_PCK0?n(+2PxspbOi?mLxqetLh_sGs_-sC(9c#`m7FOb-hIO|lc z{_@h*RZbvk@xtdih)xx1Au10oZliVo42U#PemE=Fa>x(RL zv`S4@9=ILxPhgJgV_xhJMQ@5dkC{*;hK}cJb|@g9kWfTh8rQqW`8cy6Vfai3qOWvq z>Shv3r)$0TujB3vS?IcwsJIZFsb2Qi2sEk9uWhaFhQ#i-{gZUaMGI>DO#?V z%G)*kB*BLr)d8ol{ho?d3HZYLFSYl0YhO~|{#9Nppqg)qQCNyXW|JwYsnteS#2Lse z`)a+NHy=Gy$a#IIYtgRuZ@w{K>1zC|} z=nJy&nwR5}@#TvNL&)bhZoH|;Q@zD`UOnQt@$(fJuX%L+`1@=BVUT|n6VYP88jFJ! z#A^A%=l<(F{J#CC;$sU*zX7n9JM2y76)@M2Ca+!3X`xa7M!zwG+@4su zn0iA&VIm-Lvw92N>2h|cY4bum5@LhAnK#{2@lj@KV_3bBi#+KWtCYQcwmoB{wfIf5 z{#PlVOYC2Tcq<=#$#X>X#eF-ga0BUK3F)I@DPSn*1k%OP*G%k+6j zV!ZS|2CTd#0BHYt3fMaCU6!3^9kq#U=^)fog|h%m*{b+C0&6X>6uP|_m}YVkC;22i zoM(Y6o5_(wqmK2`SMP^CWylO_=Pk=SBLFINB^5or(yf)q7!A?CtRg^GiLl89#@z~5 zQ=)idjtjg1N4j=_k_Gvx^zAo=4wXPdV`O7O&Uel;>dW5*zSy6QrE}MO04gvuhf*xi zv-RXx)1Z<9Z+Q$-lpA>4_@97|;fIgIT~P2!t3R604xom-WB0DN{B4FB$bG!E)pOF( zDj<2E_s3ezG4018W{LNA^}D;=0cR0PQaRQYvTS-84a(ou8X%gXfHqdv-cqZx70roMNo#e z*m+x(kzWJEXq6Zt9a>}^6Biip$xo&+Z+^bh7+*I~Xr3if<|Yvr78a%(jzxFEq+3;l z&%d&7%fYmYNkf=l(!0SZVRpp5gVB?WYl@`DN&hXDZ~XY1tLpz}x#=t}2j3+OG4#|$ zk+zBqS)W1(>%`aD88j152TRY#hFWR1aD1Uf8vS+0p~YAs9p0GY%lu?mY2AAOis0p% zDP8!h5}=(j5T_F%RzlBlJ{ zpg(oML7SFHB}Y%PI3>gI9(H!(T`8=u4}iW(Qn4UkQvPpfjC@TWSV_ygoHS^7bS|r% zjo&DFdGASlNSI6*`cB6-56;I-K3-qnyJ4^k$g&5Z`3%W2w${=~xY2l3^uzk%^zGEh z@KGo`6q5XD^1p2V$CK-qe7TKVy6JNwqk#b4<_BHGp-vx?Z+RX@r8~xxD^b#(k*ob_ z17|OhzT+Bs*k0F25Wf#6hJ4GBD+*u0tMLo!mw`bggTGw;A5YfMU%)07v?4<2BKl%& zJa1yUPAZd+CkL+BxV4OGnwlitTxLuwbdLfnufr>_`xK*Hwz+IM(#4)zoOZ{J25-(b z?QmaO0n(o(-~Cw){U`eVIt17hU;Aq|=%ObB@kZoIZ?t&%8Igda^gL1W@!Ix#JPnDu zni}DEH^#nXhANo%cXsiEBxrP9f<1`sx%v6|jG!A7psr-BA%8&o^$A7F$0_um&ECHt zQ!&yNNYm!tw9193R4|DxYq{@|fWX3umnu)(J6rO%G&T^+&IVvl#wJ_&Qg`AO|9yP2+ zc0is)GN%wkT`m9E1}GM@!o2ZeFubbmQ z=Asf(R{ngQS^@|8bW$!v>TM;NrePxeMyZMk^=7}qU&v;yq&5d)k%}AN`S*_ca^s`H zfCYj9rv0u*;&;D#dvi1W5uDEH+}Nr&%`67NYX?x{J{xxc!^cm*Ccf!Sqd0S!2J-zy zfFNqQ&ONBf`GdsEX_sX}A7Ls4^6AB7A zDx&!-5+uDO?vo+PT7qaVOH2dMU7SRxZrJ^__IaJqh=|9M>wSH^Y(wcb2B<;O*-iSw z?NluuZ1E5Lz+X+G0AP=*ZiDS3j@&7KNLjqqFYNNcv1QBxW5CDkaRA>vPDK%?cnNEN z;`@KxEr6{6)D_MkZ??B8UmsF|qcGGoG>BOMlxEC}t=2rqc#o^E65Y#QUVcP7Ffh9B?$ zpRq7*V&4xX66pYc&S~rR!?_#K5X16<+aT1c^r*Hq9)_~w;$}Ml53*W9%{2bUqWmjR`>Rm`aErRx zO28)ac8?E&qXRymb0a1c2@@p-s1N->Ca?k(fDDuuuL=}0nyY&wih+y-z&IEy-5g_(Qd0jQrx&&(3 zHGLpj5E%ENYlHeaVKpXAro(x*)8+om^+-CmqfPSu!vDK}q5__X?61ZeLg{zwiN}v@ zCVoWrBJ9eTTUnmc1z;fJ+*!A10tm)Zpzha_mL8P27Aug#N3X=B0SEy6%#E}UT9ISx z5~D{5IB&gdUs_rCKW07uT15YR;-3iEZiU5%hc^V$jd~lgSz@(8+FhY0CUWBTXOla9M%}yvcz}98e`#$J6I*Z`c7};Nv8BdSKY#WN-tS)4$qn|>bU|HluaKfb^cYq|Fp1EtwELjlkRJxpHV2{jmx zZ1q}wA7d6^C@uxUrvtIDumTEzeDF=8i<81H*tr0GRu-0FSWeL;ABj~0J-qUS$H6r0 zOZxwn;H)VF!H=KE0sh_>av)&&Rre{9T-QYMn(On4lw}+KAH<&WK3@JJpbyEXfi@;D zkG?{}UUGNckg%S(5H#^HaJ2$V*zg?md``KREI|9;3svX=z)zzDO)i0GiyPQtf+-NS z?ui{#v8AMN+hut=W%PlSwEPXF472W5!qu^QOdq=sH7EP132eJb%k;4QWbV?~9V09*AZRp=;^`UA{OqF`n>NdS69D~)~^+fn{*QT=7x1c0!X_3CnfZk!EJ z>`(LvN+H5RJ39Pd>0a+m?GyqXzBD`^9*G->dK93X=Fd%7S0+=V{HK5YD=7G50^lBB z;2U17pzpNvg4O%(Jpg2cLi5sggEz6Y3eV1M`f>_L; zgJw#~?-NUejR74pz~;(C%#o&tlM4}=BJ`4@t7H>)5iS)Z=j>l< zRQN9<^1m)H>SZsQ2vMDjK$ju_TKbREg~?1tT~HA}AkbcS>E#}~5|959!=SAEZhE@#TfDW{l=bM25Q(?FA&7J2q+N_Cv z)U>zP^7I|BSY(*RIDbq}>#1mIHL__}6|VN6Roa0wog^|Y4WCjlbsI=J7MGOl?5RBT68SC?kz!Bnu#M`G0@Y6cC#FI{2m$mZI(CYxe*bA4+Z1lI35bk zXsz?x1UK$iZZ*6S%Tr&m!s2Ct*U@Qdi~b4g{bzXGa2OA-TkyJ=c9O~30BRu4OL^?Y zWsM~LUzeD*1=%+x2L7VdTT)vhOiSa+lLLAws-R+3*R13t$EQu~hbw1@=x_)c;y> z{$d#RHG_xSbLjkv`!5tSXS`fQaJf@argLJD_j_OA>7(Ccv>dw?{_wk;uo7KP1krh#YK_LF)?-|(|C~khF zs@IHT+A;@-Xq=&vFppA(ON|cgMtTj#3q-+H2lX)QjqvU-?GqqfZ0O!UPfbl70>H)U zE_HC)!2YQ24%wD5cAgk5-LS3>p0OM`|xR9-bMNiX_|CPT9slYbQo4Pn!OQ)2o zov1;(6&rxl@cDdrheDot2C2b8B8|M6M-xt7{59^?9oS_51A^%J0U(tg`i(KFQ5CDm zVEk?_1Nj&6U>ycak81~N1L4Ei*}P|@9hO)8q#v8xfBMqt8~>_L{AfpkOIJr8oc-G) z^H*-uz!oUr58wVCQVY(9n*p)^GcN>%t^1J9`SX0|(poojr{f@!{;xr*Ow5|phVRc5 zv#=TDhPT}{05flUTQ!4Bq2(xsU+G*t{7myEsMAY-mi!;Yb63TnH?s>Deeu6@t;j_CI zmo0e|O!?)zoxn|#O>QDQ&dm?3)ydsh2iF~0Jk@no?Gf8wrF^X@PV{HYi{Tj z(rTIt^C-ih>B}}c&F>9O$cQ-@XA>%}6!DMx2=m7mu5B38`1y;BDHKuVyED^%NVv_U^*02!52Xw;?j7 zQTK`z>IPazf2Q0G7|B9@ml1PuZta9A;Gi+sJKCFtl{9UOJ~g&H)&yb^mlaSgDg$y` zy|*hlD!Pnv89uT3X&d878PQY~Qe>c~Ro@C{VG{yWgyO)3;?8B+z#SXw>c*1a z@SKfSbLFNWHRdL*zOVjCYZa3TpNceF@G{)b_oiow6s*DwK}rvdK$G!ID|Fe7TEZo+ zV?j5{+9>>;!PhIhO=neKf;l~0S2`DOy$}(b)cHql_&T$DMPc~_w_3b?)*oX+yIx@o z#F_FfXpVyLRV*%-f@vJHSGV}Yr8QKG`7*No_?CjiRe=kD3=K%fALPNNY|w)iTX0)4 z9{5$vBsl>U$COb6rb}M4{~GSu-2bsPV zLc*ba%<%p(R-tUtJKU78>uI`eY0|3!Z6^ZK=WO_NlYHxqQvLK+czz(XZn=_hG?FnU z!hvr;crrRcfA@M<*K`|146lhk_!R-1UYw&IL5W&JAKJQYRNUpkaT=Pv;34QcsB%He zjGXoHprIY%J=qx1s{&F_MSR)4m zzo><%e?hpuM(9M({asT(W?5OCND*i2fo#NnM)N_dLo0WMipr7Zn`h zS=^A8LlE^WLr>BAoten05s3mi@UA!f$;^RF-#=!OND^ohpZY*6iE5S4v3fG;ukjw>S_^RQ} zyzi$xfZUS_sF>*Q7g9oDd%}e9>3Uo@Q+Xj&A5eL?VK4g-eKU{r>K%cZ96P_X>jen` za@1EyO9RAsqx4Hf^Md_<@rp=wAlPx&@Fp}A&OFHCj=;FSVL*j37M>GTuMXj3vnmTp zfVlLRP7RZ<@2gK8F1$>h|pA<>=*ci{kJC5%5r;3;&1FrstfN z%pfPdIf#(}qse5sZ1jYPGC70+UiMw;#6i!&q3UtZn_TbDSpdKC(TTt}T@jXuEoaM@ zXMm3M9j@btT|lBq5_aq~gYFN)DBQZ!>Cn#CcB#*kx0uS0ctm$Y&A$;67-E&mn1p6s z`Y=7R{LTGbuWgI@dcUNiK0%b098l-gIUAH5k(iIQrj!@-SCG`D6GG74#HilV{1|fQ z%eKk<*^{NeDQT!JU}FalS#OlKLgVvTxYdJE*t4VO(5mew_@>Qq>{<(L3$`s!{{+_g zNU%rF$?DT4l52ZE^1&=(R4;FGU0pKUDy>_{`8Ip##=PmunRPu} z23pl@u6xY=T8Vl?>IGxEKF62{*AD@4=)LPucddY2yX(vrRYPwj^{kC>UXNS*IIGGk zI`0b3%&&h^)5M1760eXX2_tia)nZ&AEl!DT+x78@qSeK+^z1W(%J9O8Dv&>W^Hkw$ z48a<1Y#V<}GBY!4XY;t&HF=ryWUC!k=0c>a2eVou~ zh~f#3Fd-)In8#S#V!A{+K7a{Ksu2n>V7b2ma1D(8yYg+rWmotJX;C}$<80$>7_UZc-RTgZeSnt2SXMlE_ovM6gJ&_Y?2OD1# zJ%e~!+Z?=k3(|T^Fd(xrFDN3{Qqzt;-}T8nH08?`XO99B?bv!NQB6X}I->UCGZ%b9 zfx81Z%XtdQHTKrcS_@VE+~Nc01bsS7&GPp&L7VJ5*H?AJ!-8P*wB`h^g|9=qk(|ui zhnFHGveIHy-_Xdj>+pP-w)LB1A18JeQb?ri>?2>U4auwMTX5QEMU~Zy@aACXjZDCK z5t&w~nnXxiMjwh7fgwkk34tRNlGbPyfLltH8=kQdizHR|wLJ;3CuWw~V_ z{5a@6j7##wLxTs*7x*^8&qoKBSBtW&9ql3@mj0Hx8yhqU$Mz8O;X4Xei==ENY0ThW zC-c2`$CAF1+`Af11b67%+SL}r8V_H>k;On>iY6p+sJu7QYSwphm|B_82u!ZQya) z8*baS9My9csa*9!t_CM;>}2yxW&U~6i3Q?+8_e6T-99U%ykdF*?gNB7_*r+d&PUTjvC0%pGFj=2=`p0AN z#x;iL?97$kga#~=&b@jUS}ueUedY_DLL&fEmrK|)=qqAIabEmI6jZ&~07bw2HRX|! zG^}&bxFoym;?81opgp2)@26QujA1~;T-|%RSx(-$!r$J5jI1i}{(@gK4#yhH(fUir zy;%-7()|KZ)c8O*(%*JG<7h`Hg>>L`)g1(A-qS<%)-e%WOmD6PDVlzr`6lLWUE~BLE z-%7ELM99PIvZeuz8;*jx?=wM^rVqqp{z1@$@jTq8-@VK#^%73rj4YFhJz2%|~^4b@QLm zl|7(&9axSmfar9KXCJG`5HT;~Nfq|n2nvy^Htv3a5m4k}MdSp)HRx7Dj9S2J4rmaU zR0rW~+iQz@gh&UDm2BUnBPecKqEHU2hNj>e?#GLvRwiuJZ}hS8PLr*|&WENM-y3?R zG5cW7{<2qjJRH8XZa-ZNebb_Rh1PU}6D_hOtG~!-^KZ(u=zYlTR~;I{yha>tqP(l* ztOg@|b;iW~%;w>(7D``!dNc2bYCuJ4>qJ>zSf6@-5ipGwnH7@ z9xx^dgeKc`qrQU{*X>hrAosw{2%}XZ9~$i(6>zAW0%FH6)-}Tc|1k?|Nh*R|hJrr) zs5wI7;#5ao02sH;BRpy(da~vlBg4K;o^ZQVfL0r0PxY? zSf%t?+x)h(-{kYEj*`-v>2Vb<#k2)ED!;$sU7UNm(Or{axNGjk=8l9x`H>IqS^)Q4 z1;Di751{Ki!V$d>^b|l@)~j?o+Ys+`q8DuVit{%RvxWgd0@H6qS8?KED3|!OiGE>q z!BRc521=Z7xW>o6_qM|0K}!K|!>XQ{bEj7B^*O0A-ej!9QI02`o+oHslzPW$UzvuW znN5KoQnOB0k_L<-7-VW^_%ao>xe`Yjo`D7>M#q-iw^tBaY{5xKDctReN{e)n{vqO; z<+3>mqEge)%nxuKrgA>&y=X$JW;AqW13Ubx7^W$j+O{Nk4LM2?$fl1{i<46PH^+ko0?}?~%EoE41%(7oaryRtD;;jWt ztPJ?{>ja6EsJY`^8;9Y+N0%i&S=kT0F|O)G$37|99sOMABeRaKh&+*NU6lskLa~V# zack2J_n0voJUccQ3ER8Cq2DgCg|6p;*2^@RfWo(b1V^ zp}C_yA%m_=C8MlzFENFG%JW2&`2@J@9I)3t}Ohn z>F4&g%Ua)}!kh4u4Cc;jin8xQ`X{C|>Nllc(p!|E=$bOt0f6CaTGB6wf`2U7koRv3 zDEc)7=hR7_IsE>|VXtu0pw z@?>SZ%?6kc^`ddg43=7vM*ZFc8bt@_VGIH^y1F6)+U;6E7RrV3FP71RDioAga9g4jt*Wm6wZAu2P~lLCz)lI#M%p2tGc4eJH%Ah3 z#cDw{e3mhSYu18s|AaD6f>{6E0`LAPX@*f-_=&Q=y@&KU_JWpfZg!ePLrwaq3u(;J zs6I9G`pU6=LPb}jq1aq6>t5Wy<0=lntR#T=b>*(!kMP#Mf`dDutb-zALO&7P_>ONG zw;@_su!o%xjl=p88Z*LCXw<%$KaIEJpcC_u3Rw^MIQ()y@%rl@ouuxt*enj(*I&>uzh&Kv*EU$r_KjE&2T{EB7qS?%< znYKhL-Y3p&yBl_G6-yg;fAPS=L@&n3w7QyO4Yq(rX~SfA}SBCJC)?` z7`?S1J-I_Jg>EqTtl;l>7K%kzZkBflz2m78M!qHp!KYTQ?(AIm$lS!AnR9fDXcHx^ z-_Sd}{N=HAem3NY6ZHAKc@!4~?9=+RXg7LCT$DHl%>KY{$)AVyth9!2+R##jE&A+D zdJ$DvjYr1mElbJ!C9;bO(QN|1E6d6_-hR4c5!{A1ZP&CWTEJVfHN7Xr57mIV7Ox4R z3TB1vcp_fpl_Tx5hiBQGZ{#Sb&97!25GTSnjheo}1@5qi~EvN;<0Bx4fhv&B~hL zc!*M~3IXCPzP^~OyX(jJJ|Kkw;+kj=fh>Vbj+Ei>$5Kz-WXU*9q*KK`Y0Ws zFe2u|o9>L|Hk_r&w>>ER{j9rbdBHoQqd!Yem-JCfm>5qdz|}#9}NM%6_tr$W*LNFm)mw zeadxPwpQqfgrM|<@)=1W01#_W7k_D(6xHMro_JTKe3sgUev$||mgn>%Kv9h_e=b+6 z*iFMoTC;&tTVyV_HL3E4+^w@#$%WiC2lY zTqEDw0~o;&pBg8+>=9mwM`dm4=R$!aZ&oEMTD>oMmS*c^{7w!IYs!q%D0}PPe)GsvjU0Hzcco?i|g)ggbwD ze8~lcTJiEt2cTqv=BREkGa=()1>eCn4py(|CgGMjyPr&PV1}?|N^vnj_@m+i6=DG} zokH8#OEX&!$x0^v;Bq0JewKOBW#+MdK?mKd^~&2?_sD8t!F##Bz0hL8tRchuPQfn< zpdhzA{m`|myCgbSxp)%Loqcww4lA;`T5+t1y)&uR8@77g5l16#P6Brdiv)@-j|Nlx zh#vFqbq57Yw9_m9TnL}nC68QYlYKQ9;E)d(q~OPy3}$aXfKy7pMeQ6(J-cZQ!Jrb* z`bpU4YOu1iM;>=5QcvhVOmg;^tp^4U$%j_i{$&Ek!DV`o=ipVD0x;R=imEarUeR{J z(qM|iAv1P7#Mw9zbJjsN+iqgBet|=M_r|tA{l30#{IY((!mZw_^#Yo*hT!6hN`>|b z62K0#8E0XxhYT&_Pbohf@8oTB-LU&XGQGY`az>fFS195Y4(aw8W=woa+OI_j-U17K zjf{vZHNuY$TrhUVCw)rYtnd>U#0p$A*!5^ED_Lc*ABZZmRO~xR0$)mQ%9`VKVF&UU zRIS8Kpp1^Z68b@RP*6~ePSCdW7-JV4nh>cHgr=T@cnP3OsHoAuK1zVK+x&z-FLBaY zW|M~N!guBqx|qOYkg~sk!+lltTNcpSxxc z!x>Ge}U*d*Vk+Y?uYJzG!SJNVhHos_;3>*q3^x_v3i zc9L4{n`++|6+I$~!Wi)XvKpbjvpTg)obc}Kz8X_ac+gM$`ZgHt-%i&Ayu=F|@GDlP z0?7_Ypf!7Uw$U3lHdET!3^cj~C7y>K@=lY9qV0Pfl@vPi~ZUStDa{9sZ zOcMkIteCRU$!7iQ@5JKLC&6{LH+wQ(nPrTd-qum@hyVO2RO-(_82MP~IAM&LEa>Y~ zhE9=Ge8PUmQs)btm-A3*>_jb=Iie3&l$J!V{UNv9x5hXzc5RIMFib`pz8E#2iyaNN z$PazF1%0jAX3NK0c{{`->bAJeajhuYB@9II=P8)Vs5fOi6OvVv-J>@S3MjzYVW>hK z5kmQbo6e5N`=ow;p8gV)oh6rB$#lS|3v5fzuORB@p~vo`lrTT^btI36jL=n@+edjD z$tYl8U^co|TmoS`l@DK%?5P$c>gPtk5wKZLiGX_7MbF@sZi5K7q&?z4Pbfsi_HUS0 zLSPId*}16pPDcQT)PrSuwFHH(j(Vyi1Q%yO=kOu0Oup>(&3paK%=7n{=F`?A_gGXG zgh7Q@&%m&ldv?m+_EkW+5E>qp511FT%x{I2xU{T{Uxozall0SdLI7!(>(7sSx#Uk$ zeu9rE6NVGhyKv?Vrz-%n`?|y|ie9VI5FlO3dtZ3qMH24?d}d|OsR3r(mfh?DeH&kX zKub`zZq8pO2Ifx+&-SCIujHMHv8MrX7!eoQXZc+DCfIL2()9JxB`X&X?({#l2%6GE z`k|HA+{@HvwFtt9&YkEr=J`J}^?*I#&fUMy02oAB#+;1dS}U+m?O@O3mp zXg9aM&_oF(Lx8s()?l;f658w{1yC7@30O=A1kesa)Xljm=?Lr;4?{>uNI1ovX08BL zJTP`cRDB^h`xP5H_`jWN`NuKphA80ZYo5`Sulyh2<$!4r7>_iq@(wq9xn zLUfdY>$3AQ{1ke5Dm9bi_OP=K{`;<2N?8xf=XZC}#>E1;ZsNelz08W^se1drO+W^i z0C!V}4s}l65FoP>Bv?N(9z9qAbWUt_O-)}P;MgRZ(y%EE>8D2l_OUBD4Fdo|ggv$0JCG5Mtj1leR0awPZd zsNZo8NI~jT7ST{i!HNBaa}i>&%xm%bWvZVl0fHTPng!wfm!Mh`{LFr=)$wX}GxKyF zpq=1Rg}&YpLCF2FBm+>q}b zM%9kYe5C*@20N{pyPUyNci!R1R$2~5_{)^9Ojj7Rf8W^i=JF_hZzyLfEYLVsRu(*i~J!U=ucUWAR6!hy%VcoM7&9k2mV}`-*@{T z4?>NQ-@9k@bi#zr;DlHI8F=w_z|0ss7J3DrX?$OyG_d8*z`nQg64c3q3pW1lDJ9u z==N^Wp!I6O>UuuO5}O?{Lz1)$A_Bpf-NW@5WBu(UJK#d3!w{E1TzTE;O47O2eE+HI zsa*yxzr&5)DtrSsP~dCMPhrE-a%P1yyxyTu(geK)jl$U$k})H}QM_D-djyElo|=CJ>fYC>doTL~7Y{;)2F9aMnnbS-L|XXC2@7 znO4v1HszWwyBXl*XSVz+5>$9#!;{WpzKUs9EvRf?(7W8EgvHmJ zn-{_zZTuJBGcL1n4_qpoPa1lmV!F@W-fARcQ%urY8<+7)R_jO6# zw~?AVp-ntpId+1OeJ_b1|_{&~mluBYZ}?m&LUnnw+%ipA~T zgXNOhtjsYZWixvH2<1KmW$>5V)#2%nK0-`f+OubAb0jDn9}SO)&sJF1clsfaA?nu^ zAQJC4*JykU4*dA{4&bK=L2@9L_)`{#B)3V+jHrck{p{)yhYsVIamq$Ed`E|#Sx~e} z*vO{tWLB$Gi>Tf1-d0)b19#FdMgAM_B&yw950~Y%sp$sB9a*_7AMxvvDPl7lUrwLJ zW8II_=wJ3iGsgC>)aGEY1F%3w_w@6vEWk@?GqYH>VVWj*PNMzY0)k8a4kJA4Z~ zDPP^#CGg&M8~xrRR-xjl`GL*xeW-yB`OG!K2XF%@iAcg*yofDEOcGkWC&^x@$NTGw z)6491!hhdZA~mTLRnki;bC)wWuWd3h`uAL_KLok7$QO2VSHdvwut+>qB>^~gRE{!s~`D(9BE{uCX^Es z!wrnu1d=2l7+J4=(1YG$e_|QqN{I)#zlbu#+G3_mT8*AZ(JPy_W6O97aU`e&d8%+CRM59?Wo|5=TDg~=RrP;I8hqZOYv33 z@o&M8)6cOvsZ=vCHg*Vpl(sn%GcNu(L~l6;oqMcalHzbKeS>cGLDsEd$(X7daSqo* zRRVU!y&sA2mT49^5U{2+Ou%fbJQDAViJi+zK+~W;W|d#LoG&J3*!2nS`K2zN!QXtu zCm`^Br=s#f8c5Jq)I}b!y>BKomusItEB%oSzNCexB5MacAq*%+tR~CYY45FTI+llV zT|Gy)FX^Q%446`^K{n^jw{3Pq=8ZHZYJ&$?&zYkZn+evp30aZvS?CxMZQSwFxCMoc zh6db}54ml29wLU~TC+TU;K+V|gDtp_==S8bbalXaQhylqSl&9Y zwRza|T;KJjBBVE}p7P7;*rHE6^b~*a^2JaI^9M(UeMmI`&=P_P0+FpH`GEqSr46EQ z^?=dlc8frX)f`XMt83@0%#*LCp-WT0dbymK z^JS6N-}lJr`+we}Xc+&#gNpVi9AGlFuhzlG)okrf9833jI(E^eHm3k|VgK9n9nL~s z6y4nlum!6n#&L9-M;_PX2S>_ot6JYDVBYFeE{YV#J_poan;)3JDMa0Zg%z=xZhHUR zW(~vaF1RCR;%@!oHBCJ&bqZ8-Z^^f$TuZV(7HQUB^(D%hzdV9k@dSXKabg}bt>}x^ zgSqa;ts^ipY}T-Or!vjQon6AVTs*@MJ{k;Vw_}_h@)u>+dnS!t9dCAohyGNkCp$`! zei~*0&HFMk%*+IM7;p^i*3x)0hy7t zxAewd_Dy;0T9gIIO;IaUv?Eb)BAB=#ibU|~0hlayJ#O%7$!;tFKxhF(u*>_-{%5RJ zc~jq0Y1F@GS(I@Lv{z%XUFd7|E;q&W#Iq94k~Ck`^m0#mZ*6Kmlwc~;D*PG;jd3#i zgujnQviX(ts6W-jErxtOQdwerLO5!Ka+zbp%~7%mI$G-Y*iuz>?tkXV;6*X!Hiq z1pdv}f5CKUW+`7QDcAd7x8ul-PH?{I?B{BW6ZZX3P)MlWlSfBXecrwyN5!>QJ9JtD z_mpF!`=jY~D}IjY4#+?XK!t-%v&`Q>++vR>8NF5JRu4Ia(jj;dOuhAd-?rFs`H=!u zF=#dW(~V}G+GwYTK3^${RNHKsZo~EEQlsYJ=@0-dS^M5e{vA(YGyNDCs`V#E#HVcF z;v7Q1gj!*xgqB5F`)UJ6!yl#4a`Lv(`#cYT?$cK57&!>DZuHJyX|K4JyBY=cdkgp5 zjVSexb;C4qHF>Ww=^L%?v9gGJ3-+-L1Sx)Bw}U4TtD{ z77$;H7BWT-mhluT3?1RA_F1U$}Mx^Z^5B%5EfgR*QU=K++Ix0DDJ?VTp$5)LI=~C-MH6 z0V|-43*Z+qCT^%Ekz;5mn#r)Vx|~Be$ep65C075UF?uwf$uFx!BAd3H_|h81JVkX( zsaJ`OQhjuM6_VwaRL7+k4XW zC~L72rb@B-w%wSdwwV*eyNCx5^1nXbw@lvhV8&L72EHl9NG>w0663(J%j{LhG1Q^)yXR*TthHp z`eVr8p~4;q;9aI2bcy|9R_i%3bU&YQ^@7m9+#4A834eg@ar>qokGIRUKKbpX=Mgn)DrJ|3v`TK2(== zkM$$cQ@~E+aE*jl&L?-UcydGQkKRwSCG8*lHH-~9wg>x`j>L`WeYAFJ8aFLu19bIS z|Lh%W z^p1`|T~vy|(dR{|4f`t8s($BdupOn#tQqUN#2HDqA;7NdkL@DrxiDE#Pnbh2-E#CG0(x9q#o}Cc$ zr_lU?34hFjVpQzVHV$VP|+vbuAYIx*)UPMp$cW*iXW`KteXcREdmHc?>A6V?MR}*ku1f-*l z0N$BGWXZFvEo(Gb#^hi-)^IQgY6S*(U#OB+)lP=w6!8*UMb^~RpyU(oe675gnnd{fYzK(k0Qk3* zv(_vH-dnOu!amzqK}d&Eve{I21whI`>)A8PYh7M+yK6ABNC_L7FxKdOE$R+4|7%K=nn2n#%wuJiA8&{J)18Y{tbCr2CpmR4nr-*wsX{w2mDLKX zMBxBLcZ9q{Wzy;1sf$85uv~2r84BLm*icjeUB+BjEkCLLV>+;EaSI5DASpnEIq`HF z(!~J>{wDKP&*vt&=8wmo$Dek2o=!0{gml=M%}Mo(CZDum?x1M2g1LGvvG=$c+u1FO z`I42VGda!N{RJR7etk)-h&=rxvG^JpKCZ9KHyItoA1#S?W@I>@fG|JSAJ} z3*MbNeM^7+yokwD=u)Ioj&+lx0N@ok1IH)#>&`WR^7Fj9Vanh3SX4UgvbK z?XwNdRaYSqM?GE}^5sp2-iwt-ptBGPA<{locAo4Lo*;KA=$X`V67UhFh}4shQ@*vi z$PR+y`A<)StfIXdl|O||0gXqzgJd02vcI0Y{9)m^XRnx!YB(OKYqT57S7+4~sAIlb zEU_J0bC4Z96}BE2fiA~tvlA!~7z+CwB5ZL{3D2j_IwI6{jw;Oc%BvmR9-{a+Lq`|F z>^wn2#KDBEZ=%kzGP%$EqEJz6t7xQ8Gg7i#Qw z)PenobJ!^va7R`6uvt)p(=p+fxxNYT&r;|<5i7R~0Vk_3+QB&+cgQUB0(?uqS>+_V zh4+;p^HLb+J*{;UAq`rstCE8pT{@Ft;N*3Lgj|m0AVXF^R$2EigXnnh)B`-hnkhN( zc$&dV#JBYQ$zsqOuuZ+4+A-=$@#~M$_1!2{7I%FQols|a$%3EC#6if0^nG{ zNMKhOXFa{0t*r7OP3(N5edH5>c5(+QAG!+q3HU3;h)DKWfydF?f~U|K{YvOLo=_1I z@G{Is{Si1=OC{u00U@~_ts;A<6R5i0E_C@1#-Vdue znMnyT5J}y5@?3#IiROgj^7tITWr>%7v`k|nfYJCpv02!#jIH&uOZW?6Je`7TAFprh zABOMF-$2{d#xD)U+cgw=$ZQz`qQ+VqINe51^9+aa;&tMclE|bsw--3>#FSmuH{OWz zJ94~|$2+DY9B+?LnPMljZkrZM4HBuQ+6BTHoF7UE;i&}5iF`~>WWt7s)Zd-7|? z;2nqg&fzKZ^5y7YBjWE5V~ED=Wt`-m+90+IU>J}#JKP15-bVpZd&4h}CunqP($xiP zOsEUjp5u`Ad3X#^1NYI|nd8K~iD5;S;+@2GK|{k5Af)xAs?qRab`JrawGQ1G*c2FaSNz5eS|3KkQ}B_jcB-z zVa(2>S93IC4Bzoi5uRJ<0ru?r-G{@`6Y?@n^*l*mw$I7IT3$@ShwOT_5q<}|W+1^n zK<1ew==KcXgRbjr{WVXgveVUc2qt@mO#Edu96uF z?2te5cE)6C>ody5omSVSbVx&xccsqcbJJL(9~7r zqoZW7tGf&D&SY$TYfse7*^@K`EeZEZ;u0@skPcnUddGm9qQHpP0<47*S@)wrd=KiR zXY|)_>AvG;?wp95DVE~k%|#c8gNJ+ZlTNF8GQaCCJ!Ja{v**^!XtjsOHsFnXwiOZn z{WpCWq-m7_G>!t4a*JN9@l8XFP|UB_cu8%Il@=|bGB7Uz>#e~fr>xA`wFfh$c_iD5 z`D0_`1O3Ltn;A8F#B-dnP>KP6nE;UTOymbD_J6fpAn3>FxyaXIv%zWPo=^?NrS9;R zc}DqH>jqWclJn=8k^&%pu?vgE7in@au`ei~pQ5K^i3El^YK{US@6?y=5y~)Ba`InP2v@%$hjQfYq*Dbl0Ej% zaiRvFcF~^*$Q^)ybxKPR;FA%RzX6300<04}YQ*^kF+sKRhIseP0C+s#ILtU0DWXKc z#^*Bw&Ywu*%&ELQg%y*fb6xZBS;XdI&P?VWz0Hx19pO;jXh|%mZ#CUI3!zjAU1!w^ULX@M#1z7p+Y>_ z=y+PUeq;Tf!}`i|JHoHz8;}O^-t(^2$i?y^modbkg)1-Q<_A~*h{&SeK9hR#Z&zGX zA^;WVO6Ay`{cxFCTeJI^uX{I^@1fpOeg|vO0nn$D`YmXv=ZT>;Vn3#JTE1Fiqq&KH zqZ`!1cMArIzor00jt|BzBbMaj#d%_y&+$EGo`nB#!qIwG0IZH4fu|rKFUf*4e`};X zJ$LSL?-t0awD*IP$2u+AUsZCeC8w1VQN)Y1fF7{B@wo9L*_XVK3J zq05_I`bcEaNYq$gwo?WKZC7w0PwZ@wIuf-!=Bl^HW@P|x>5umt{J4~@lwPk8x``M( zO4~GJN%UH*|5ANA(rdg>LmwrIC-_3+Qq&6^(y82P2QFgAEe+!W89YJT>-W7+Qr_>i zfr0J_N&-$&BhxLwipy9J_Px-v*&WmKssf5AhVW@UsE%xpq$5~X;RmSP+(fM+Ztk0P_AOxtI(&dt?S&?8lrow-5DMM5(#i^jW5udywWOqBW}ZU`Ha$ zJ9w|jF9EU%F2JS`oI4M6*Zs=&drqZl`*ggHN--S$dd)l_r< z7d*=G6S(;JQPspT-!>iTz^+Raq-z8MyH0)ogAK#p!ME{NZquYO6<(g8i{BW+74~j~ zOD75k&yUEGm?H6fO4*Fi7j<-&TpSzU_qGt|cOTE$$QW0?zmbw3B8(ClG@RT=32I`u zqCc;B)~jdFWpw~ zAmyIZ=Ad; z<>EQoVu?^z;9TBv>>)TDx6Wo=C1Q9+6$qBOxEM4B5M4gY`Q_`MdiX!{1lUgujNqnx ztE)!%WK0)v*(1s%eCr>Ct~_r0@Zkx<24kNGh*Yz{aj^bl#v45;0zJ><7I>MQu{-qQ3sfZJpCq%57Kc?gDaK;uAnvkv5{1 z5&vt>etn-;K*HA~u&b`akoJ{SSm~ONBZ$tQ`0$))P5QAuU8$V{~I`Y%~#2tT% z&-h~;Vn=!X>DtFp@UKcYwxhL<3W zc+AI9L4u1f_J~%NodKLryz$!YKLweYgu|B(b@CXqvein|{kg1KD73Z<8fj7$q43!-cdU$*j3 z?9iVs{+V_DYkJX;H)mERsU5t)Pzfkrg5rrQHTwY|^$?L?Sh(NNN1O_m*!I?;Gu!OS zl3Pv%JYI3!#Fng|+@QaP6ZoF^)+s?}aoG?ce=~TXOXHp?NEGw38Psjxf}k8eeP@Es zjnru3aQq8m>U?+7yL=ChGo^3r7L{SUo5z<`xt>@ zc#Ro3u4QY0gHm|nQp~d9Np^lyq=abPs|BCW>T|<^9qZbY7 zuedKPtThvqYI(Ly&iluB>+OwpjB+>YNL44UpZy`K!jw!q>o<|d!y#dJ1bxI^yQwEI zxwHqB2HXB1&23?Gb`?cgqpwGgi(f`%sYg~2%H6zcdL)hYtbkS$2JS{Tl)?n{Lgapo zNF)&J1A5?8ft`=x*#mHb3$8i{%?=}@$4|M|>B(gcR{_ju=y?0n_63t6vVu=-W3pxf zDGiIkH!SEw=g|BCPp-&CU~Qy->xxW) z4$~){>)x5oZjZ^w36U!_g2$``pmbsY>=2ZAEU5~j#d`!5Jlg|tL>Su@)yjJPwv2t9 zwhT6Ag|l8*v;WV9AgZDQT=7xZEAYLDC;_n3u8!K;1Ef7qS!2$C(%5!zzDr%a_5=a# zMUImw>epM~0-`zH_}kZ3*sJdZf&=E`Qk|MF8*;LJXQv9w14S4j)CC759pIPXLLycP8>W$lc(VX8cPCf0~^@ z8eny%16ss^qz0xDtmf*5?Lu0;d#^=sqNOqmxc8%r!n#D87((@+o?3KG>!W}ca zT`6aZ|1o*kSl1|mun0T*c-KDxmT#Fsy;t%>xy)Q;Lrp=m))n;v4WV10rz3ly71P=PS||Rck961r+!5>mguNFahIgUR*uFSA@76(1V235^lh_?po9Co*LOPZ!{{KK8>iu~^Fb zbPUl^+&RPCT=n6cpeo2aJ9CeWKmW(B`$Zi**`T`m5-hj$cNzmb0#%v0hm#=T4A`6hX+1=-)b1eVw6j?K4}C!){-&krOFXv4lc`o=8~hCF6rWd$h(*{Adc2@|9RM_Ud9 zWnW(dzr~j0=IPzPz__39ad;Y39*wmUH-U$iz8-kjg zR}ZGg|4}n?JOdPPpb(R^-d`8YomD{vev4C&#e&tk{ZN`*a$jP#$8RP;ur8U9^@_mHS@5!bTh@BlBEjyp_K^9xIl2Vk(j;pA z%^mz}NFwW4gw;QM_9|Au*YN;{svn#77(Q7U!N5XIv3(~<6BRbK6AJ3IQHJZ%IRBJ! zA_lS(+*GXwc<*3~*GQ9S%c9Bos);B2L>rsj|aDLlsrQGxb++eS=goDiVaE zn2sm#?S))Tyt->1?dg&~CUm?KNCsSzj~*)-z;VO63%TaOTcr?C(z%>5#Iv?WtShcJ zM1T#R3fZ~vuk(JdcFkQ~?eEc&vDGSU3og~#O8w$?hmKY-Fcu0!BLu0!7NHy$=S@Mi z&vSB*4O`b7j*9}54O)dJsnV`RGXmvipkV1c%U<4rLF+*2DcI;KoRhhQiKNvL?MhSq zh~{NPxEkW;@h_q(FMxWMtrF8J@(G0AhcrN)bL+BCq&+kh7k`&tX@e04RXKI+8u2Pk zcn}@xce2`hT7`3I*PcAJ#K_qZ5X8cY!zvbnmcx3?>o3HY}KD(BRkJOzuh_h@-yX2&@o_dZbX0e z=?3s{zCXPlxgBmEnwox3eRZn5gTFuuR9+eMu&qkMENlxBzBr(t8 zh)gBGIN*`u#2@FQpQap_`}7uo;`g4nfGU-(50Y;RDtGox(f-W1!dM`HcG5NFvzsz- zqPSi#DjufI;_g4j4*l=V2m6T+XnJSDvkNERKGQZ@8m>-ME4WD~Px_ZR=F^^knhGd3 zy`UHZrJ*w17hx~^bjS^4k~^^n|Gvm3Tp|7awnU~7u(7dnu$%x)KZQ1pzgf2dS< zs1%UA5_51*0Y=tl4Rv)|c;+@Wq}8zRKr@EP0oGffi7MtCEVS_=JNoC#f_E2D!K#FI zf9UZ@M2zt(IJRZ=2PW-p0{bp-3VeTmom5q2|I$e1OpfWchwtZaqN|*PAkS-a#Jbzq zuDK3LVr>DLLGi7BrAaC3!33xKli(cB51A=$5#wmuCVNB+eG=l_rCVwFz?9a5%wnXn z*f9B&rgpTR%u2CrIOpes?53(El%B5cp6q@Ye{x7U))n|=#Lf<(}x8=ncUutb6A!v1h2{$-=ez@38Rb>Xn*;Iz1bA zp_f;FTRI|%2Ve(o`XgrPc$O%T0p?x~F>g>)rKcZ)CXf2q`e5SKhmej#?tDy;9ga9joP_YjDpz_hG{Q?j3o zDh-W?O=si~VwxUeJh3Ed<+fa!{K|Kvy&d8)`-g>yqt4A_*FNo$z4h6(l%?f38sv?g z-HrKpn~S?o^ch*AujqImFz$XD*YXv)r7>SQL`e8su00S3yZfw+f4X}zi~AQ}0&W^( zO|#2t|DrR`P&GhZgkor*5?sxs>dwQ_Us91ngZ@v+-wxR!s@iEVhm^4F%U^iBeZbL} z1=sjhYhriz^)KrcR`&UqgWMZ&A-rLz`-#9mW>X{$ED@pgJ4+0jpc|&8?7O?sf`Iny&czp`LdO7-+5?UU^Q9Oz@4oX-3Au-jL#L~{ZBD*+zWDO? z!)EH_rp7)R2Lss{hB4jwNOxe+YojMSHi<@-GVq8%hU2LlS3N-ua@8AaTwi~0TBdGI0!|hY7nOU+4F<(Pi_|UiSEId2P@|T zK4(ZN_EFEG!qMMu6toEcV^(ki1+l>Yb)NVietV!0!6bMWvAs&6D8CvDK#nVly)h=& zR{Th|;x_k+-J*VgxDT zS2S;GB+ra^RD4N-fP9=KeBcX6IWPtmYk!k(bxJN#X`Ls5e6Wi{7OjFhTel%|zk(FI z+Un{OI+t;-<&?T*1G~Ovg~68g>$$&~=J61)TSL|O7=Nr0zH%BiOMa#2R+zg^mk9g2 z?^lG1HEUW}WDZ|Pc2;lp@Y(SrcTFwX+-z&+Xnxx}R+Ix!A2!a%tDdYZs13xoz3K6a zt{y%x^%!Y8&^5I?52uHJ*+SfPK6gay1UvWB(rquSC~s@cHkn(>?*P>#+&Mx+yN+ zq*NWs`N*cK0fSe$yd*fai)uEhbny0d>(I0eY<`l=uH|jRrqS2%sc2%?fTX9`xudV6 z`t8LZ@0!kTgo;!CIV8c0f1!9DJ~|4=dE4;_5nq`xtzh1qNwyzS0QPEgEP=rC$`1hu z(o7THG%aL9!1-4i$R}7EhtBJysHNR-3*U5Csmor~kFPi<@Qzi9TtGeo`Sn`r=k2~H z$mje6#Pozm{D2e6E^Qq-uq_4@InkWG1~-0X-p;MsOO`*xKf%0@Uz= zUy6V#>fXN-lVHX~J`>ZV)A6#(PNqU$W3E&F-rn?f@1V<2Id@m~TQ0%25-G*{4}9A%FvlOKx}>ka=}f+R zkLDys04;k!bpDR1Tk65<2BIa$D;xcT6s@cFo7mi{Tu-@B)aF! zV3X&A6Ae`s+qh9__Y9oU8vTlebGD-z`>5cmnJi}fMuvR@qMwBf@^TPYCXp9-5qfQr zO|qhh5Gtr9QNxX zplt~?=<5iDQ60Ds_C`;?SnIb%8dM1MR>kJto@TbaoRtFGqU2;v0nKM@s^dEN-!{N{ zVNv=ig#~lkTDsBo@P4cPCB?NZk=jvBN=D`dZYx8vI_cHIMmv<6Ym~{@SEFe`ZDU^q z2ASO5oaT9@g4K$+z{$j7fzM=_a8qz0wtkhk#_oCTbmOBX{3ScY$gcNwMi6XPU)I zI6p2=B6myYz|>eKWC^AQDxO;D;%yE0C>U4MX_*z0A8nZp+O!JhL=0W9A8McuhS(>@ z>+O0yRWP5XqlH?r+Bqoq6XGcd*zR#lmYJZ2p|2h1bjaT~Rv%T^Z>d!=8+ZvEP`_}T zpr6^~5AH@ZEv8nYcGkun7roge4`K%$T17a+;t!B+mFqh(cByr??qyEpTVZCfrd4Nq zhn_L zK<4U?lxgy+)FympR!Zy>t7A`7ZsLAaL2psU<(4MEr9GXj7+SY;tv)IxyZCoIi_mN6`DL9ACVV2y}}p^@Foj}-P!pIa;4 z`bbDI&AO@Fzuj-+k=rPvaVUqZWgezjJCrtVaA)CA?XO(mwqGWmqF82L*j}bgrkIuO z4^M9B-&?j0-stwbsb4gccOuiQvaLBC7X?SJ&#_ zMPY*0yj&7-w!HLcU4B$f{B8I>$+nd+C5GSSbUcdao9U0%D54xP4jhGakK(&MTmZtCCe;h+mO`m8nryK!+OmQT;QalT_cP7sMNo~Cw zTb}*2@BQcYXBy25mzt8c_`IPdq~u+(4zuH4!P-4p1(%@TlJ@LIIyZ!R6lRk;qY%xB zNzN|tNG;Uq9dIlfCdkE}m)OyQrj8mDGxIy%wIaoU!QYPWW!r=Bjc`6EoUKa4uZ7&i zFthDE@nMm1@>+`7y@b)mT!<~+axbOl@}-%c+4xnM<=1|QUiq=|gX*boNujP~o<9sAXs=XH zhmkmSQr*r#W*OKef{biF!PK4^;>!K0BlbF{F)a1D;;f*?x&CVAepEH$ojINkpWk#g z!;wy7vpimD*xUf;jsuHdzN6~GFiKe~dLT^LeCpuOVyn*D2k67-@y8xnQ1g5m=u&th z71tYmgR(rG>_dUY?~VM4Zf5z3c_X1wNGF%w=T*=FJGpy0d`8nZpB?S?((7KkndppS zfh(^o(>(tYxQW}V2>M-Ih^A4joad&klK-rJ`<^-^b<5joSjMhMqra1N$V9Ya%gOKe~+ulC4a}?}yGrSmP0o+&mr_eW8b;k zyX|dp)o5mIKc1`~i8O8C5-|As<)iv(gX=F=ovdHicM9cue4gs_I-r}g&=p$p3R9fM zc!zK^b%XI_{5BpYQVCYeAKk0ol9eQs#qIm4eolLj=aldO3@h$!2JBG`&3J<%0E5o0FFo?3ro_9KF zQ5Gmd+)wK8m!rQxF>^V@k*m8ue~`^7QQjjk9X^R4yk_R0`^Y=F5k^(s>Yl>9gW9nS zVZUNzV@j(2Q8iqBn4A=8kiX>Z6m7mK+B~4GXE)|ClEOjvf^^xm-SPo~tG&T?K8LT@ zKBR8O|7wb&L4`LjQM|ibRUuWJwcpnC!M&h09fe0u(RpyjrAMAa=3Nb0C8+O~@b_-a zBtkbFATLU%jAVS_VN*-J-nprE8#T*$9h$jY=W>K*4K64oKhTA@aCdL~h?@SHwFG2V zuVy&vnz+D%;tC25+RE@e^r)S5LQ|l*V44)0r)}ti7q<$&)Y3leeQwBlpSj_}n1+pR z#&-3ju;lBu(bYeOyBeQlCibQ{EMIhciwyA`Ir0%o1$&Ldo1`ikBfDW^)D60h1v`t@ z^c(73OJz;fkT+2(Y_hss{e%Y(SiFxV>VPgQt zJkbq&i8PBkTF{G3s-R<;vW01(vy)R@Fs)?Z^A20Eb&J5L?=(^>vHO zW|13CK-*0_b$FI}Sf=FeY9#AA1IPn6+SbxKZGe!kqLO2zg_fT!RrviV({ z-)&O4pdO3URL3jQXMHScT2**(L`9V@UT3^y z`!yz7&B|V1$eaP;B2IpR~ zM{!EcT2=Zd=zKCymbd&p*E@E=+&;ie{UDVEl9yDD{KZWvaiLwFA~5DsIf};na(@w= zhtR_Ogo&Cmm`T(sq(fYP4$+{uR1HR=>)efj*^R23)%ZV%d_=_}xAz(Ot9C1nna~WEdF2jq8 zW=HB4vS@k4>lAKhfB>vBCq5tE78L_zAu*CnSl>kib+viWJ*~ z#!stW9ZGJkfV4ovq3;cJl{q%wCaewWzsk($nBS|q(Q;>zUxRYNoqn38eLkes!>n$V zeTN~<6O}mT?vvzX=_fd}=ZD}PLR96^S8PQ-fD`ljmhl@z6%5U*kRIo!0!2XhOxkqo z6VY)1jCWcuZ9X-vgj2d!{dplxe)6Wvnicw+DW!A&w9jgKVrrWo;(7?jg4c}rs;wE?&+oHtu_QB0-go+L%K7E7GA|G z14qj4i6*km{gE|CX)W!JhEiUkXb-DzushNDhX;pRh^66ex%@ox6z z!%`RT9js{|+2l`56%h4at`bXru=UE(W*o#L^&H1!X@Cp_1tc0)ocgTv1GEWH#N%mO z#H_kR_3`w^XYd(1YsX0n40rVD(LyaOsHQi|X*oESYd$=o;4YuFfvIMW)GJF#G28YH zS5$D1(4GlH){dGQvN9bgZurRdW~5aJsPC1H92xB6+Q)D5$!+*2ElQH;jTw}KO@pFU zQKbT%LO4aSDJ&>Dfk;Y+QL&7w^nbtDD*H0rP?_*Bz2uAgk!i9_DLi9y=#KgwHj|NmqkZdzz3&{nX!B0r%#xOUjps{F zagGD&C5G2nDHA@#=9j1&Pd6}&n7Hxc#jCwoga@_j-TaVOpa=rcLdv`c{XR!4P$KCbqofyoD^1FbX$&Z~06XV=iH3+EU?q_)Ok= zwN2GvD$31Ex*u5eOEFQJ%dg#Np+m2V={rho{g$>5MkiWq8a?aHcKLwb@i*B1Gr^7j zooW0g?x@V-4#Fg`ksLQpVkfVW{8qw;RQ0!XTmtfvd(oBNlsqr??Sp$eV{o862A$Ct z?4#%XD#hc+GJF)h3rRQS_IA8!MtoZoA7SQ%)1|*A*cpx-nxMSpOiDYgLVS&k7ps+@ zceonT9#v1;z)aDkV9pB2h#l8f%8m@6snHSf<(+MA0@JpwRi4jrkn^M({cAhRB#AqT zh5JaiK|+T$%6R5NzpEF<-45(v&tYnEWfgs;-V_?{u_peK*EOzC zkp+|tvLnt-QwIxe8VCC_f17}f!cJ4T$;;IYE=cS_TsVV!+hcGvW3vtQik4o$!YEUf zlQLdr-&20BwQnmm>AK*SU=jeDfa>hgY3gO%lXoha^gu&kEfjA1v}j@2Beb84`w%AX z22HXJeb$xEX1Liy2SbBhcJz7BCsiF*4U;Fa>12~En+$Q6+f~oSuZlm|n%ar6{b-vS zzvx!{IofHB1$WdSLaVrBdX5}z6kIR0uB=z&m@<#*StJSeO?&8y%8*vypNn!C9h+_3 z#Eq$6*1_8}2LXPn`aTrhNzq2pr{Ep6lA81LO`OES}1M>-Hk6D}S zg1Om;33H?8@%>CwH^lw#Z18NleZTFQqNk_;@%=FCGpl_^TZ$bszTBSaH={WqZN2hU zl%0)8om$l~`uWk@ecVn^JNuS{&AS1iuHy8V&=K7mA^|wKc=M23^J4^< zf><4K?63HJifMi{7=~T*??QRJ=M5RTjYg0?V_&_v=53wt4#*nZFnpnH7Vh>jzn~8j z_3gCGSw~uhxrl9Qqn((ZA&a&+VPAc>v2tPe8&~FvK2wM9CHDj<2yF&S% z_at_PM7ztJ^~1Q1A{Qj5Y8d0~My3!iyk_=zF6WvQ*JImR!0_E3SWG z11FCz!0smJ3(eLtv-eXqw#mX$$KxX+V>li+{(m39YX3J0|4qVw>hPZy{HF!~X~F*+ vS|C)Nk%EJRQz0WEs`me-&FCW0KEcJVSC!e-UrWII4wZcWR`P-{EOUNW{~9ey3X(cHDUz%G{RqbhgGtWt+rd`-|E( zXm4-qQRs}l+gU&N*6oZK0~S^U>OYVFMBqOW_)i4>6M_Fk;6D-gPXzuGf&Z@~K!yQL zrHY93FWeV`XydJ7=~l#j&zj!gL89UymCr%9EltqYlQ-!AV&IeWhjDr$EG!{7{1IdO z&bhL|Rv0lF0fr3f@53ZA#0$$;U5cRAXnw|2{aU34^Y><_E!A3))a72V$yNw#gGP*Z z9!WshF8%mH!J7$mb-6NF69J_%Zro8AXmqGg$(U28g%vA<`^h(7Zpbz;VaaLhXCjSS z`t*P@=unUm(C99hFiT7t@#BNd*%oc-)Cf9q`PDP&T>5(NuSLj8UDr;H6I6QM9XDxg1yR@~Q|Q^m>|#?;yuj_qTVi@)+)a62wEew^V^ zka^>u7?|)vBWiv)D+}Lj`E543Tu|x@(=0@eBq)|#PnzJ&aOvqf2Sn`qod%VbwZB__ zy6s7G2mAM?{@fQ7%qcd3aFke7ICi?omM^71c#D!pCt-rV)$ub{oon)?{P-dwl?qJB zs&_J-DM^j!k8#e_2+hxb?-g>p|KmeP9BRPa-ll;0fByGBuW3Q}a}(QA<{AG=fq?&1GBC<;B~d_mHQBNg%A1@`}3 z(ch<_Kcfir%8~!KLCj*1FyOUL>rnqI4m<>(+8o!56qj##ml{|}|10;9U*M_99;Fwh z#UN8+CKM`q(6;w|(LZsbO#0E7<-Omnjz`PLD3BqD{w*~0R|43YPC%fiP{@I;RPs%$ zPf7W{&5R8Pvl8?u{J(Q6qYV2!PT{$o?tgsakWsAgU6i$-=4A&_w#c-`{UDCZtP<)B z2`Q=R@QBNDpK7HmsdZHjmn(&lQAfDwbScrFAE=k9I>-jL<{zVrdpMVVxt!LTG#>(AtSvoQ;YCONt=-Qd zVG(ZF|4f~b5Jf~r!+X5v-$#Cpll$p5kr5GWORq^~(6c+wn;L~SqDzl^Xvr+fN{wrm zA!LV~LwSCITgvvq4>LCU&teC!*Ig8J^1BKiw|!O@7Z=6h;WB>p>o$ky=Q?^o8+y9P zLN4H3`UAME8C;K7J3XTrJWWWPS=L-)D)?X3b&xC~u}|oWb0)pLQ@?zOQ+mCoNo{kG z-qx?9Z#<8h_xGaz@0uK@{{6yFEUU4d^J1=WbNTyR-EJ8m6hkn(oUhab=xXqIfu+}~ zq9Eq;3boU&HE|K#98S%FKe%XI_HY|?n{#2D@9gZ{pMRTF5m4mRa65%)r&CtBG^S-v zoh|Oi7mPuWnw$Ei@xKCD><%+I|DYW~-5i5m@V!@r)>p4}tzoTw(A~1_z?0P2Q(#S__xkJY0#*2DV$4$ny=>TKG&&qW~S`lWkj`+C*_U4EJ&=jH&K(4 zI)CBI_-L8W&EJ%78SD_t(ma=bPR#d1)MW^H$@`Adur-K(co7!v+f+AvTvFdYI~5%# z$tb=*>XS6)ix|!6sjRa&49u4~uX*RSIZpORy0sllipSr<*6P?a%g1Cb|FvdsOu@p; z-!T)FbC&ZKijD!%R>%Znij73Kwb@<=M9+#a(b#X4h#P47%?g#}RT z@YwrQP)-m&w#b@@OJYmRn(oH#sS?%kC1NCn88_JJc;jXj-mWE~h-N}PhTihvrI*HT z?0`2ZG@&f)z%d*@YMt<7f>Lg0aSeE&`tdL@PuR0%c!|JM2e!JzX{Mj0l`F{k>kQhwqGEyUVgv8il79|LSHDh@ zBA2_}t3@moLctn*3OOj_($*0K{geC+4_L~BwGFhjkLkCpue=SG^P-hnz;~Lp2C2D1 z$i-9N5$Fv&zKaU3MkEXGE{vFoq^RJ^MgZB8?9Ahi_IQ|Gdt*sex^r`E!F7w<7}nV- z@W};W>r1Og&5;!LEWSn{J8os@j^gN^3b3cPPucq_*;%9ap*2{$!cvi7HlLq{YnnSs z`y=?x)gDVd_=wt*49s=-^dWGn2+Q&LIqU8|JnZsOoO(I|YcJaC100yz@Zs0cr%s!} z!?U4rCXl#pPm0G0yBMci+$$hYRR5eK@3t3-@Ao_=y$)He-^n!CdbVI*h2TY1`EaQ# zLo_h^zvnGKjNDKg;;)5eA4C7c71T=*p~QiZeb)lZ=}XC&);liki5#7sBN2AHHUEHC7!B0u^Oe)t@laTAUH znK97vh<$zAqYB$p%!H^EgUZHah`}nIYPw%|QTCtP+7JOha$(3Zdl%BMc4QL;%*oG_ zRu{?c<71+}8AWh%5LD4vmJV=euIKKM#^n|e#FoXpw_!!`&2d8}biTG6SWw==f;&u8 zCgBW8eQFYoL0rxE!g-e0B1L=Hl0TfQCZ@`c!>{tL(Urj=Bn(2#Cp6jE&){$nK_j@A zWid>6pmVM9t~QiN$?8&EJ2unIMD`}Dnc~ba|J%W9CQtjJcG>K!aPEYzX3bt|}Yr!|7 zX=E<`J7w?d{45{8adR8W%BvF9ZM<1$1_y(jiWcVMjwOE13VqyMn^XGbzF7s#`VhP$5{w!^J&}LUKF}ppdQ@zFBt=pdPqf6)@N>y?=FI)$p=c%b59@m6iIG=(ND;I@05O zL{To{TxY9awmeevHOt8Xf6nE^yzj{WF<1Ps>T~cb*SbwRrX+HmXq%v7Mm=Q^n8g@3 z98{&ttD1#$T!Hy>jhfqOwTS%N1x;Hl4ElF==zDYO_34P+`b&cu@NYAYx~ffDL9xc= z%m$0}z8_9lKYzqZk@>^~JN1t{zHw%@yIf-m!FI&sB06ih6&U|#ItXe@69%u0Ngif} zK7^V-^IDCW+%e|2b8U{E&2i6Pp$xX!U+|jx2-E8}Bi~<^9n(q1<_bv19BOIDVZGF< zuG)Eoh=YInWXHdoFpPmVkQf$g-Xo%3PsIy<3#{+Bs{3wrjK|Wm z|Fm;t$Jl!0U=#Rs)yg@JH7DqGC_tFUFgQr4r14YY=Wu~qlf#2ojHWybMD@#LPvXW^$Ydr%YJNHa-NUBpgxb-s4@U8KSLfHY)4A?{&7dXkr zUvA}}q&P_Y3yfrFugLxA_5|%k7Dqn%v8)Z3IL+8`iuA=2+xu4WT_Np}=M3#hoyU9c zHH;rTzl3Hmd=4f>6G94U4d z#0ez7h~ZI~+D;exAflm%G+JG%mjO&{#VJ=eE6l-*7PB zN-eIsKgo&~#^CeR*Kn@>ZZJ_o=`aJ+evo5MSuZ7T$7j$|4i{+0BTIez_~3Yj)g6n6 zNl;3SP@&S)@{pN_Z>Ubb$J5?ek^XD#;hwmcbS?vC^;7xZD2~!_&u`c05NQ+D?1XONSf+KdNxa(jh0hA=LPy%gv=Q>VaLJV2FNZ?U4j$6Uo0){pQAUOD&h zn@7^?azP3dx(iG~lwmy3)|SRhN{{5Nt};JXfA{!wX??5~kl}FL-%{kC z5`xId?_=07Sd_u7@(SacMbnkeg&9cWrS^LR&Q-%f>NM3^AWdR9IQJUV#qP+VE{lQ z;?c{EVcS1`w|qvw;q$59Q}Ihns_{Hq8rZ6#`e2l*@Lsd_vt7yvrJeWb4ChE%7MPb% zwmfZx4GSf4;H{+`HUteY>v2?mb+wr(5f0=w$M!s0=qcm^X(hf}NUL3~YkHport}Gm z%i~-=ote|usMGmy@!O|0rb45pn4r#lDxHOnp6{kWJdr`^6=wKitfx=%Zz3Tf`Uvy$ zh&mShd1{;&9 zt6-GW#@&4&l(gYfn6Ubj-MFVO(E-tq(XtwkE)_arzP2e=)ohdGdb)vkq71BbkG=_T(s%5!PB8uOUV^aKK*2kOSq#D1h1mS6`pL4M3G7@Hc$6r_@aqi>^g z{s})NJ|;7@qpIMbsu!?`4+Key3;AR{c$~@n{J_!@2)o0Dl`7HA6#LK80pTJ2y^Mdtkwn@p3Jg=$vlK{u4&HGqu{k;w zr;Ztf;C-!2e+_2ik3qxpPHP3uses}n{?sIH8uu>GJ?Nd$BrGYgkJv6M&uMDm*fbnT zg{Y|NbMoiwp0T+B&OIs!3!3mhV|D*qZFHaV8?hdK=RzhDj1M*zCf`VDj{ags$4Ah8 z=jL`6FFRANpjP19@eyVX<(+kqQ^C}f%a>9!9fRvV;egIY(N?Abj(ye`JM2k;66W@1o`wW-3G#6{vS}0VF)m+X;Q09s# zH~V*ML2A8LW6$f6U(;sY7dKe$9p8k_`%G>;fDGX)ybN@(B0jzA1wxbj8$E`oV<8wE zhNPF`@Zm+!dQI~bxBMQDZC;xo(x>t zdt~&+QC>loBT9#Cny3Nptek&BC&K_EI{NHRAG!R_Q{Wq%^@-4`M^mdtt_DXK(f-X( zTF#^Mvvp-o1Vt0O{e#P(!jIR+do329&NcfGbh|uU{hWtj6IoHn>wsr3(>iuK?MQPw z+S4A1*=(MU#5?L8{@oc|pG(Jd#n^XW9lkGixu1P#JQC8m?n}J~0n}nCb=NL9g9rVw zru&0s4+QzW`CAaK7sBYyPkrvzw! zSM9dXOCZvtR1au973q-b$`SeufojQCz{5D#uf4h)hgTm^j@Bs>q-DqHcWh^l`zZ%3 zBRrXI^{`?7-QU?*frSADsjiHVoE!jK)@8Nw8ea;H`>a5wKP+}pK+pleMbu>gAR=CfNfh>Us z!V=nmAgx-BRIalH={n;;5_IhDu;i9ObOr=kIZd})p-L2<+1W^~@P!zFz#($9)o_an=RJUi;T~Rj4z&ofgE;z0cg|HYz=O`UKg(y{*8?6gET>Y zo21u1KjVU@9+>=P1Pm`V?G!(Bm_~Ee`96Zf4rUtz*&Ac}hB}8L)=7kdq!wFq!tvGn z@176m>M6gZWwU_^`KLs44si~KQu7Wve`47lu8{It zNZa~dZSx8S;>v7A6hxg-M+X_QW;_J&MMx=R4@%vbdAwYp*csIFofve+<35u|jG~nI zBxu`=#oty)jpjN8Sc!imw55qTcILwM?9Mf$&C08_`#zb>_GXZdS!R#fs7&g*ayr*@ zy(-kQSJiBi(xV?(-K}!=6KlbLo2F6^$g$euX*`}U=saRA)$14_c&`43B0cP_Q(S4B z$-_Fw+B1 zmaZtCR4kdF*^Q^y1=$~!ak1;)usk;Q2EPJM-@2zbi)#8h;}5O%nZc&Ic$LUwm)?dc zaPVzUUOw9jbw_A-N-ERGvot>_xISG;Vh$#%^J}Y25b2^U{b9dto;ISLaQ9V+7gu@w zb?ziM?;oR-A638)b0|xjeYBslpU|69k`7{20_+1E8{m*~5}~A7zy8B6TRwh@Z^X(x z7rt2b6(_;1jww%Xyxgwp5pPf8OAxcCd*i>4YjivnPRTnQqkmg2wumXUM^`|vl+#{F z5YKKoV$z_LiLo8o18y1RwW-4oRv8@Xg;T;B)`1WAFPZ|~004Oqf5zt6Z`u!fO_0@6x zy+V=Dfudv9ymn_66}TyU>~^8z^6H>v@k(zr^mtBSMh9=0A!<{boP|5W@iDw3#R)s3 zBX(*;^Izfs)^J<*wkX^d?wlAXXN2I&G#L!}xGZ zyXn#dLQx6Er;SFecHrinHFs9O^EH7?7#$lj}=oPebAX}f37P;f)`7p4Ks zNlW!m^8;hE4N$G-BCa^EB56=pMullkrPUsT5iw?L4*>>?k*d9fm})YkmW-zTlI$uV zNOym(ZElQ9|7OsNUc0i~jwdyegoLG;TM}u9M)mwhZa2??&y*E>&tFLNo&6JYHd?;n zC7hj`3cvE(-Y+7Rm(@6&dlm&)^0ySv3JBf8VXqCr4PsrYLSu={*r#f<1Av#>)kqkqA$&0-g0&c@mdQsTbe@h7b|ln%;GVX;x6Q)L|6Hy}yu5S1z4|!Zmj+Qs`2M(yfGRY@SAiF7 zc(RRhSX3gxlF%7eZsEBQ!-n`)db)Oy+bgG|ibG*avgcP2vn1J72fX3F?Jbo&;CILm z^q``AnL6=y8MhfoZ&^(;=#9iJl+*3El3qj!*Ofbb#WrA`u6G#yOP4gE1`(<-NY%Y- z>6nSY^;`k85AV%>UZtv6OHKtN7?r$#2iJH~jv_a(EIuuYlp}_qwf870ll6WyD++@M ziNcs9=*`eqnF|5LR+`OuBB_z;g%T8c)AjB!JxAOic}}_tg-F6dQK5^}#KzsGVtQ~J z7}LcQ3HAkxR?7iW%A2 zzrEJ4FrJ){5;qGsRWK8I5-F*KuJtLkU1weQ(8nb`D>v1wX$|0 z=wXwe!)&8nsy)6X!C2unR?B(iC&^33@bYL>O!xeL4s*2)q=z)b9o4~@+930JgoHG% zXca!TLRPLnWTn#8@FkGYIlfxR)ImvIkR_WpbIrf~OT{RyH#!Wr^E%Bi_iw4NU0ZI7 zehTF!l+c7@Q4vC7NepWxm9a~j#E#D$qncU*ZadfD)^KWNp3}FmunHLQtk_SVi5y?7 zNgHjo)@>S9llIhGXC!6a+<2mSde+6Ets`jVyn=#F*uz@!tM5SBmaZcF zuF3BsZZ}Cl$pmriE~NH4J5?QCfGnNs9n3sHB!f?waJTt$oTixmia95BQ)U*hOBD`= zSyaSmIMocTqN?r?RIn*^Jswl7tN=a?1aa>U^Mkrud`|C&h>mG%gFjo1}sBs z=R{)vxT(W#uPK1*mvUY4`x;Cbn1i$*GbzE-XxaH1lzgFr8ZnfFDfY^_a4KoHVd@cu zZ*9-=g!}vrWK(@*@n?^)g7rp$q)C{6b(6U%5pQJ&t?rJ9@bF^p#~7rdR==yq`E^}v*l9*O5Y963~0G0b&-*6B*GS+t?n8iPs*Aqv)7uH7$idD z<^&-O9eq6tdWrNp6D}o)$0#Yi<32T?;Owh=}a? z6&yXBAr$bGyi^x9S1#I#a%cetT*&V!=bJV1%@mkhmwR0!!OdgkIxPCeW7({gLiov` z|Fgfp|8y-fwfeo(s?Avemh6C0nh@1?|BvZW_~n-shtt6!XK>vBh@4yd&!)>Hj`(phOhdRtQhyRTBBzn{@{h>cowmKyueqQlos_}*7KuR)k|%A z-jg$D5fj!NKJN^q-AQptWPU5ytc`d)fkB338f~xZC{FWRBW+xK?82HdkuC!bz;A{& zvoFf{aVRbuly6}R^8|{+>uGr1-QTrZjw=b_R3>aiblM`t7utSAu$bJH7Ju|smXha? zh5HnFJP@cs=(F$GUwe~k(IAb*k1hg5#xLTX_`&g7)0xxgj2Hw(ShK9M6UGZR9wK2c$x|JJ$yS->4(8mUcYb-!Rd-z~!x%`?(<{9!&pM_BYKtxPc) zO(A?8{2Dn$efc0fMDB~7&&K5n&+`>eRjW2z+QKTmsH7^3aJBW`_tkfm5CyA5yRxAq zPHKDuzu=Kh9;gkzCJL(y34uu8g{LdU;lGH0@hF2NzR2;)JDIFR`ft!ZMStib(k91X z-zqMJ6cuHABGzy@`;gEC4kEkbxPDlu)t-xrYp;|0EdlH>uoUd4Z~u;<KwiCiVQahktT+BNi%rB+~q=R=)bfZ4s7Xrda2#ax<< z5YU3ikdTdt+hlxf1R9XYG@O&gwpRC|1ehsXY5oQf1q?_OZpjBu+i0z=PO*Wo0I8AO z+71JHVVY@$nh)`h{_&i2Y)zpKE9Erynp#XpIG`N>w+>e1$|bkXQT@oXU~e0X(*2t) zIY{jYm8K6q9ZjBhcF3i;mz0=S|C3!@Odg+O4wRP-rY}pqPkU3T5YnyJ|52mUP1o;% zF*E_e)>(oaiQz^E|%IqyX!>FWM|L~Gi2dwWrSwpc!xfcHDziOw_e zYdyhH62-tqO`udc@Z(@GNZC;3%TkJF?|A}v57XD(dC@auiL+K&--{%_Yt_>KL5(td zu#1AmcJXrr3vx9NrnwVD)+!&C;AYNJBq0ID1zkw~m^!bo5UuQFi0urW{-bMMZ(pDOOV+xW?1g@~Ok1?$eSwDop~ti2k=JII*>I6W z`QXxt8cE9Q{d)Vy{lFs(&E3SO2$t}hqZ8N`r1DNmUWm%h3;j|{<^{jGk<@T`wYGOI zU4Q7+>3HXZScM-$3X}pKdd-{ZU{mC@)~=RxvjKXHyfVafo^|WLM?zn|ucC~_LbOT& z$GDkE;r?zc1l*6$P8%lX+>?Mvb%dvI1_C2h*{X~(0Gb-cef!zU-DDBwY>8ZECqL*1 z#<%6%_NY6gfsxSK=`3a`b@*CrW8S7YqnRpSd*c9gQF!2EORZMpH##^VOTe@4_**|b z@S}^CuUq^hgQqia1P``TS2D=#IbJjRqEuqH>Xm&VY;y0rl}XCWcGu{`ldfg8as8ph z!sw(u8;Bp^_S(PC_%ZLAUCZ5EUQmYbL;g;mR6^QBWz{*oUSwi46&_vimQEpwn^#Fc zaPTg81W{r&>V(3F=JjT{ma5Md(T3be$*{`TH<|+7#SG@JVy6FBG1GhpQOq)RlrhyJ zWmYWK$1=9ei3v4!u_E%0C*bBFO-I!NZHwh>ot~Z^TSWi4)5l-;FEQ_V{HRd~r@lx9 z&5W9uV$%4{FG*^&9xTsRizCzPLr{2w<6JSb<7yyl!a1a;0l)vqB3xEli>87_)YjfU zc-y?U5vA&=;g`Di=E7{5@9Zwhp$`QU@8(eKwApOF zK$_;L&`dO!$*eeR%D#COEeYO>k=qG_iGvH<(F2YgesO%tcR-o(<&?3m8UOnIJs`tu zsQKAh`K{h*j7$UFdh^2Q#C7nF=f$vr7GXlZ zA1$`Faq~IBYwJC8w$q3YPV803P2+-+BQ@Vissc2P*-q6Hhl$N_&a~M0xJ**mK=pF# z?7Hgu+nwU~8R&Rls{w?%G~W-S@JFjLdTtwfZT)2H+k+cO?q%$nKeIRWL{eVzALivH8&3qkuvI$~stb{LDHN^upXxI%#zz zmUDHH$rdlg4|qrDpbWsrmv@i? zBEPt#V-zyQdGny76^@J^-WHV6kUhM_q>g*xQ{pni5bKAnPxJ8MHJL@BO~_71n(U7B z1;y%HF&fNL`V|lrK&}WWp(0trq_h=5i%08V3}+Ep18y%Hb8LinV@-dq8#ZP)`62p)e%hq*A2`Hib_0E9$d%C|FS&WLR2#e6ZS-A@EU$ zh5toRtZdS_uxN(Kap&0IpX2)n4|e6SnZd~@7baT=k5erDpg5xq;_NSlb7h4w z^Z2YIWkRgYF#bS)lmM&P_nnrc)U24OZz;kPrHXfP6i>NM%UUompIqP0hO-Kvus@t3 zw3`W_Cd|1bVX)T<7V4*^cbAFC1`qV=bX%r$^^q~R+py{BQPAaN;eXNxf~@EEo;iL} ze!RT*(^|9)ch%XgWq}SLhg$>#g6o4O_>wsb3^qO!11NM^s5*xBwuV=dII-;UwAl6! zC!>_HhT)hozBE0J$@u3CVU_9RXVyjPSBzDVA0dF|b#|UMZqpwM?{m1cgCt7tsuqixI;`N4Iw8}#Fh;{Mf&|xdRwYJg0<~UJ2El0pVg8)<`mxw==++eF0C$xv(%V+zm zLFk&8Pxtx|^lg8kYmb%FUPrxprp}cZlf~i<`21A7!6wz=2#|(Fq*e66AL-F5p}P9| zVoMS~r#m2L!X-TFsQJ_tA#o4n{TY`mUs~j*@DI9AMH!L{jowzTfNQl(mM+l1l{~Ap zpE#{-cBhZq?^u<-EIPdu&cMjxbn80kIFgxU5AJzXx)PNk#hWz;F{dH ztMMYIzUPzn0y_N4@cDejry$n) z+aGieIMLDmE;d64eNGwGG8FszL`O1s+Vyl+N3~^0)|ATaPluO!hEK-z0sTb^aL$AG zDm%G0Ol??*#g7rM4F`@HJoU8eXS#m-~{6pJEQh@-(J3HU^PwNbeX;b%$s@0@|S$w`zJAB^|b zA1HUw=Xrk88$V~PmryUJ7%~91r$@g}n2wKCvbRj^f*V{qm&d{3rgPcJ8&P?6FMVxE zoOGM3VVUKuRLX+zEpdYY%Ory6+FL75OCAvCE6u41OmOB$qNBspaDJEFJcd=h-?pJ5 zSF8HPh)5fXZQL{zbe5&m%%A?T?BaYHV!)d%5a>Wn%4)ZzaL^l*ewRYLMl9z}gI1(B zH=@r`jQ_3w2R;djX+sx&Y(G1%w-R^&%rNi#t3E`eQm>$4`X<(ZHq=#8@sp`v>Xs`L zoOWQ5eLL>;oCe9aJ5igud4C9D;=bSH8EraMU7Mv{X#`Mz_;s^VN)J z@>5@SlLn+KtAIn;T|&?ioXwxBbfzadil zrE2Mq;PjW!-E?xv>+>3QJ_=B=O+*6Sh^l?>L-xm#KejY0 ziRfH2Ksiny(pmp_Sc)lN*q|qQb+LoJk4xjrnghBWg%N!*p}(#(fn7$G&~PzRzuHlJB~kb)rTiQnp5~tAH1eN0l^9)j?K_x@At4ZEZ~i`tvM8 z#Ni^_RIy#;xg@-o@zOONs2uDHZ~q9t25BPOb2_!=lTV90)GpzxB459KtKYm+uN*2s zERB8{s`Qol+&JM#_-jd#Kc-2)*;hNj2hJSFY29Du?d2Yu*YM_O<@e^RHE|l*yNtv? zGH%L6D5|;{V_d}a)EuF=28HWUsV7S2=>0nXP1V{6{|!kANby}Qp@!0y#K6PthHFyp z=EHb@eBP&+yH&oXDHbk&#{285&dPDE0m(Fqe9OsO2x+H1+|>-KQy;4GczZ!_l!>Me zW}DpkZT6FFgAW$_!d!R+W$8yRc;W7|Fqv1!B`e7?Nf3v(?Gfprzw0k-b9+Bbw%N!u zXI?Ef5@B;3w9P!-lMf%!T&Ed(^%liaypf~osHxt!ilIUHpBZDC*Pu?Y+4!Vg)w<;U zeZ7Wv_fVR+evvwgFQO?d$ZC(riV>W53x8w)2#H%~MY!4d=B zY<8<69|U>Yu+~?7Q;)zu@-$Vq<3>E%8CBa_jc?!*@HYHVcZL3wYl~$OALS;!l3QK- z=eHLsPVDD}Sf1JWBJZZ}FLw)lg`aXa|jBfY*&Ds~g@`!VROR%K0xE#+Lh^Of@lysz9mWZe{$S#irn?@j< z7~zMF1^oUnzFJ3y*ikEEGKb)R2mBTbdhc4=eyc?)^f*P`zl_O6EKB3x8% z&%pppax;voK7ZUVtu0>I;rwxd!U@V`WF`zSdN*9L9UfXI6ID8NZkNL*RW;LSe~s-T z9}#VMwVbxETU5lJnsr^9C$$9l_%ho9StH38hAtm()ltM4()E_Ee1%mr9%|h&JAT~J zn@KPkxAAmxTfoTfzrkRW^M18qc-VJd7lA229mA4sq%9GvVINC<_4{SaX;#srruu5^G9KJ}cz2^^NQVZ@) zU7am=7cMepoCW9^8Gh-j0YwPG8@?AZj0&N?di#x-A3e*ycRB2%My>eiqI(-IEC3g> zFr*;R5)c3CHHrd?}?|Ck_cbdS2|Rd+Xky>0m2spF+9h%hDx$oe86ondwU zh3W1bO|RP$%|F?wBp^jE$@&vt@pg5FjJ9*~i5!5)s%XEzid#0iix1;X34UI`i$T5| zq21z%`8B)yz&nW8d#TSmcgbU|WvH!mOIN@U>-hYQ%V}A@r)lx%&7YG3c271D!~&*$ zVN&z64WcNVr{`6kRDQZ<2`V_dG+Uu)-stCbL;m9V(}IA{i3WvIL9#Cjl<)d=m z;q?%NqNV8y<08S3#gD)?ETLfItTLoXi{4Tr_@vWH1I;m}TB-WrhTH+Ii+)^Nz5&^q zYRXYn7Wnphi11;W;#vzmGQz2knL};4*GyWyx4f-TFfbL8WYj$Ig=Z@&?P>V#;Vfx- z#nUW*O`A$0qibCWn6F9wRTA%m-u!A3A_{Etrc{-;n1XjsDCexZ%0l!2g9PbEhqoY# z@LKh~D=x%OlD7v;8(q#NdQlFR3<8?(u1xl1VyM~c)5 zoVQ3BCi2&tO&$`gQSXF_fUl!ctMoBlYIbFN3-_(u9edJ>?TI?si-}|51g zSAkvs4AJ$7VKtYR&HC55xyuNONA*`DL&%W?eMlo=8vENKD<2e64g@fLX~%Z`L<;7S z&GnyH0WvR>HT?Ut03yz}f`YQGFw$uul~{y1b+Mj`xUKD82(!oEzDluts&~>9?IHYq z!iN!YCpw~oWzY->|cNs;UXw zyJ;~33oTm?l+$?Mk@;s3QL<3^hJRn2%Yr27bh{d4S#Q`ZEIazwGMGRw4xgylu zcX_;Bj1zSj>#_d}G9#(HLK$pqo`9m{2uPu<%DvFiHp1Mp-$UiePAOz_q3af9!2I!| z+2cydK(&W3{UeFI!0Iu;2yh_k*2lD^M8HDLCF0;9u$BC7HJ)&^Q)iSQDKT~mvY=3r zkql;Tkg4%ZWLvAQ-YS7^wsfv*4ZCGub9nl7Pm*5A#cXV7iFv7Y_n01;_f9(>JvOFP z{hsb3%zu7&PqH6AHdKVaSpCSJmb9LsG8(6KdQuP+gv z@FBme!GxTEIuAzQe(tRV2-j!UneF#5>TG?UTKY@{-#;h&kRyJXBb%B`zFS4vHeyVx z>Zs*gdudx(B>|DW5^4^(nFA*HGB~MnSLU|(@X+iK5Q^VJEBfHmm_;uC7~RcbH{ef{ z`CR%XEN&VPt5m(qEX8K`*jZ6YnaiGHK%saG$L<%QpiurNapAI#6{{Z`3E0@!4t0h- z$YzW|o&Lu3W4Uj72NeoGmsF|836Z)7Oq74sq@$(|rJhVz>>rsg9H?%ex}&QvS34=4 zt?bqJIB+|z+KnJ6_-l4$Bt!Q+!93^Y-OlAFZLQy9`}-Lc*%=|Jx2l^oaq(ZP$hf^P zYIJf0jOmWYkcJSqgUL->8c&PcBh$zt~h>7Z^9r+t0zFu@Q zmX5fNFt@reNKaz6UxB!FHu;){S4OOllO=@H8S{F~F!denvQAumB=m~f`=2=d2+Y9G zF%e9=Rsdr<|1tVa>c~1&OU0V~Q)#stNi*8lubR<2=;$(RpN2+>e7~XOpSE6HFiiWZ zt>jhn>$C;npb&HBdL6IHPYWn7{w7GGQO;Gnyjk5J{$<_Z{Rqd`)2 zoUU3YGatuHk%dr^o2#q*_O!TPHQn=5Ir6Wn2!lJ=Z|Z`ye9oC|Ru3Hc0HGmho_PK)G=!(>}W!u-du62gu? z?a-7`BrBy_!%RKAO|jlPvY z=ag;xE3#)3U9XQrH~$R{nqbzP29h4|>Da7#j4$O{sEST^0*?px9g0Otdgnkf4PK3T zVBYmNAQIw8?1ir&Xj?D8?@l086GQ_J8mBKHQiy1gxv|dap@*|}AGk=Kh-v4hL%tsg z2J{3_SMkm(vVH9SnW+-%@+tIMTuXgz*^F#`j4$e%mjMVFo+^5M-6IXK05M27X7CW5f z_Ufv57%ZKTSIbhp_?@nc2>$PZG`hb8wh|M0mdTu2KC6J_&-TMZ^eO!Z6cWEzNJ9aO z%y4o>K;^vR{5|K4-x)pk$_yLO+q1 zk)KJ8Zx;0D`{!y#1GaFoqZ-+cbU?iT%o;HPlQLi?mTMT6xCfgSn}L@#(;%dgKdDOWsMuDd9_*q zIt}-|JEr|xkRxDRr@k)o=N%L18^BuUn>h|B1^}5AT+mEmU&1>WgXQkRfIf6?V4uMW z#+}iWQq|r*I$@y2_@VT*-8`ABSwb?SJ}|Sbd+}R9z&l@ytzuksTY&+)*-s<?4p07|#M4u!^TAS77#Ns8N8VOGYWC*~kduqWh%y;1 z(M5=#I9zguK&s930B(JvIZE}G0Ur1vgAc&tw*$6 zguu?nThU zvU|5ld%SL$GsKt2S@1iFOwk&rDA1xODuDTxGt(2+UQBDg)q$}Je{{!4-uO@WKMmRA z$&L=EuLL{~Q3MxQ+=3ZV`thH(B+Gw#Oy;)BJfaa7bal2@!I>b+0)xP~1fA~jkdt4s zW%ElagWUeV3h%d`@)}lJ{qr(8o#H(t`Da@K_i6}6!27{8niRLs@Nu*06Yf?@Tk*sf z#QClm=I8#;VN;N`twJv)(gVEf==YQG{N7lpgu89%Ha5a?daH!P_Q9c2twI1|*qzD^ zgqMw;T&%ZNXK1!t)44uYTDh={s2{)G8FPi0Ju%|D#S^qyDq{qY@wAQ4nrh%@;r?W8 z5VAKs^7VI1CUlc^?+mnZ-c>V&Tf#Cfw(EhCb{dI%&+P8$jX>GW8}OW@vY(!O>7;Lt zD=B27N@HxMJ2H~l%kEnHBe$6jX1h31-Y+O?W4|tz{^GA>A96%BPgB{B=_3h*n~AyH zih`Hbhgq7#CYjM6UZtdT9PGCNp+lTJ=X|04B(Iv8hrYC2}cnL7v`X!fUR%( zF`Dn9F7O!Q=kzcr_E-RB0GvrMnHHPaWit<)+Ru%#t6_t#hR=;i_O761z@v`k8pI!t zF&U)}bV#vc`ayphlBUy-9~d`B0Q7JItqDK#GVWkaX9L`vBY{SiIJvUp+V3`!bo<3y z`Roz3?#*eH6Qp$;0elH8XcCGMoc0UDIh5Dz)r5QBa6CO7e;=4Y3=J@J52s9BM@YVD zH72v}pr57LTL&^Toj*yhIrps>Y=W0ACYk3D^g8V<{YK6c#d~|oJ?QR#rRd0|F)GFJ zHr4q|%h4jclYjWw$J*(#_Rf9o*iA8dH%8C96m7wVnJu$FpMhMrYn}8r9Wk$0&wfXr z$x5-JHUX~(*~!8c{BTtqc^#9QkdPQ{kU1!X3ab5;KxMFd4V8_Yj^_c z>RQRBxKLqD&Jy7)x7QYA;x@r0e+nsqH6vN0P;|++{takb8*(V>IUh%8(4 zd@yuw=Vm_~SBUI6-=*ljR2QgC4hB+Ace$PL?TjVlSP0XvZ9UXo=dI3`TB<*j6O{=; zqtIZzhz;x6>m8zf!%&kWyP;z*>C6w5Lktf)1o9}_8bh!A>~_tZ=_-B}2QfA6)7GFZ z`u3d)sjXF1?^)SNZN2Z9o9(9@$-DGC7f{*{%$8s`fcsR8SWfs5Wj!1D2WabCtrrvW z`l29eW8h;;%vEW9)EN#z^E)qPFT>DtVK_q)z{Gv45^^ySU}Ng)->&!TU7( zs<+u#eH7JM&qBbf>3)EP1fI-<3Yw`~kI?tIZb>eh2`Q|am0nNL0T?hqIq3Yurj~YA zV&LP{*Dv;dceW-hL7E=jEC;5FO5foSqi|hsd>rnbi?knoxCbiW;(U76lXCJw>CeXQn3%a_ai5G0=OK?fNm!*$ zM7p8)i`)ppvv+rPBV zl~_%vqne+{xW2UKJJ)*NHtF2Zu{=t-zx-pz%II4; zQmUcwFD_i?Z!^NCdKZ{d{72?aPqVVaegLtnTP} z3LO6Pq4r`)l~OAbzrXEH-TOKa%UH-8d)st~%>k1hWmxQGNa4uiZ*UwtE#I7|`2Bdb z$^D-Hn_NaqJH^hM?t~R`RrJ)2iweH#74CiC_sY(i+f9z*mFPLP{Td`lt~8VQEhrL4 z5msK%;t@w@J6Fu`1R*96mRGU7f``9KfMp>ae=jD_P23e7B_!dXiW4LfW08m(DMmzA zCepS@b9Xa(PvSd1w-m3C{nWY891fekajW*h59Sthxux_}n1n=eS#9S~8h!5YaF)P9 zO~!GzmfzW?H0*wEArV_pK&z{Go6k=m*^rWQ6~Z&jxpesixKytMX!t?0dQ&P#MD?ip z>pyW*(7(*&1^Dx0+z@oktiRbesvVUv6u8C(da#BUJyB%JpUiBZ`5w6>E7zGECdc&g z3@mc+SjOI&?QY1*axz>>q0s3=Yks%ZdZ)ay*m`y+ru{Jcc_@WF=i1qFyb_WB9OvM! zdRMATMh+Vn&$3;#}gx~B(mQ1{`j+>d@z3Ekh*FIaqsJp!Puog-?({DICP!_ zm)V^-90zjWpJrzWt}5WCGWjjF#boW4gHYd#HBas7rrTe||HcVKT;D`|0{e-I2q3e_=XoMfToi zu?=cIDJZygWAxaqaZWa!s zC&Y3x;7*8sweg9+ZXE-xnC8XfOnkO?gH9BRp%7`II=On%tax~H#GKr5LH*tydPQBE z-U+L{BuXL(8;G+jcvc7O7Cs64J>KM2*z=scvdQNo9dig3Wz4bTiSMe?qL7D#HYhAK zamAc3+(}4lvUE4;ZMp6#!uJ+GYTjLxr~Am!!O`Y?4A`qZ9u#aEcUKa;dBoln+&9}a z=imTobggxXY!oP_x>391!X)A-0ZuF6_Fzfo&`{88ao7?IH8+2y!F+k$jmXUKBJf>! zk6ANz{G2vj@Eo_46XVcT?`ak%2X5&(VQ`^aIB>K|MpdK%fs1Tw9KJL`17V5?AC>;# zE>0939IT~2vRaoAVk#fp?2?qlB{|E~E_*7J^VDzo3|E~#$LZ+rH4{f;UHwRO^y|~P z-NW&w!)B3%rSU6V$^{*qRpY)?$+M3IYPm{TPgm2Tqay#rSaskOYBvSFVHM?ny>`Ye z9&PaX*~EG=tR!8E$EWu2ui^=oLPD8NcFbPd-pY|fibCN>9>`s4(o=6>du?IJLb25G zpY7C=a?AX4$>EOKjg+NjLNi9*zBt zm2$|!gKGKk@bFv{zmCufd@ZAvNi{^7(MZ!C$WH69-C>f+Sgx`fj1b3(7J{wbnRocZkwD%$~PizVt-iSvu0lx8WMqAc|b zE>+P=pP8m5^HnX8E;;{I+YJ^Ti z@TQ~-&wFdc%MZJZA3j9rd1lk7e$bE4yNrUCDC#gADQw+pX=^ooWg}!GEr(xn1?!tx z+Gn8n`K=8v*7IDhK@zR z;t>DtVj+%e-;c{M>D^gBr`J~FtM0Kkico{Y%12bZ0fXGfnu=9K$%I+wKH9a@`FWz%;PIG9nO>ngf>3r^au>tk7Ss6g&5pA#A<6Y>F1=H@KrU{W zQ97eq4fBxhmwEHMxw8~XbUi`faG;3tY(1<@QI_E+LGLsHB=<-45bVnlHwub0np^$Q znvhymU3x-oaobt>6AjjSCSOL;%TR=sa8pd`RB$!UfDId8kkd!w&X+FE6aAf zR8S0L60aLk(?-PQ-YZaYNhRfZPVf?(8s9GG>x;$ZT6fsRoTL%uY&VQaZ2=G?Me$0=lwnnI^-SfBTO= z2$Q>|`SSE{IYtXOS{}izU^R*?XH`z;OL4!FC(^Io+~wD6jhE+N(0j3ZIew^pjqHnX zoB(Di=GdK>vYu*2uQjorsmg9dor)VEbs9A_+$wLYn@CXZ*$qPoQ5{ql1mt6j+J7`( z$SRyWZ_mu0T$}PhXDaUdMWv70X#tYwu9Tt2biaz5Vi;@B1Wp$0jn<+51cs@P{4_vQ z#r2V2vh_bNF6+A=%kXj3*7NHNRnpl)iz`sp3Y&VSv!QQHk46Ui__QDP`40L1FF5hIB33!4s1U>zZ@ylh>eafR_c=X@uw%8bY5rwXJ{xFOe@S?mbH2D0H zC*(+T_2d<;W}KFboATY@U}d!Ly-#{s^7U+%<~!c{T_%nbhs^P*Lbt#<3vZr#hXdDB z*p9JEb)#|L^>bdi^`Ab67dd3DcgZhS$$k#jYBs?NzZ6>3*dR>is*TgGi!v(ThrB+k zY|%CcS|XHtxoIU=wOot2<#|^V0ZfzuNts>(-XPeLTzbhQFcqvM5O8gGsN@#cVK4gB0X`9wQpH{wlHO0$;!3rmWxEDgqTUdTOyo_dx0-Rn2<8$=8Z?-7;5Z_ zxwgdMrW?YeU*y&@;e%5cglZfmy} z(ekOQ^6M$VzE5Svl6OVwrAGE|g|{`P>8&VVei?fk7y+U3g1c@J<@WaTXm6>B+>hJ3eEHqW zwv!g`O#(|Y>O2&w$fiAX-x{a5fI0g}}`s#@1tlNe5TQ;_akUU*1v znxmJBfawt4cuv`#0M!S$<6@JP4W^-(zP21CJ9{kT-ZkdUWbx37JaOmYF?Wv(#GUas z#sII_AAaM!ew|`nvHv-jZ{?0Pnt;F-{=?tX@H!D4S%48xEw3Cv$*1i~bd!^V5Y_Z| zJj0de){Az^H6MkJjGhstML&KO##Q;gqf$oQ;B>lNdmt;xbZBQYTCaP9nkkc3DxNm4 zsKKMVtj|N=`io%HU=_ZvLIZ2AN=M~sCK*?m#L9Qd>#ZYP9}a~-*E9@3jg_Mw@TN>^ z`M6oW`O44Xd!x^1Hr^J(uR3R~1`FiTr0C%C6mGwVI}#epiRcY_&XRshu|2O;AIdMK zzS1nj(r6Up0rK{1asQNbFkBC)%Db`Apq*jhW~>H>MUJLcT&}o*`+$4S?4F)A6+tv& zKhI-e0wsThsAQnc!up4RR3M678mCQJ=AN-h!R+3O5hZhiTKGr31CmSU*wkhe?jNNS z^@zE&VMBg7S;noDt@cbqD}{1{SAiDT8+%g_a=<0TzXb6B+XT`-HU$5@unwHPcGxLK zq^c7CJG+i4CfvH#uT2ZXxp%!Lf#Qpwq&tukL0hM0v-WvZEf8B<9fRCW1Ye|M@mc&pMGRt8)!k@$$F>xMAM5O$^bZL@NB-m(M_EZp9Q=4Ld^9-g0? z@^fwfe4qaye=_wxJMKQ!{G)jn(E+V>fr+~XB(OCtO8BAma9pVAKDSYJZbJOdLpjjF zxHm!6Pp8d292lL6$28kb$J$SpNOn9_yHfwPf3;5LyX5M4Dlr~D?hyVhD-}cht`|R!G5oa6c6qo|N|7#?}wRrHh9Y zN^Ph)WMMC5;bP%#I5>F0dnw5@{c6~R_okU`Bp}OPv6>5Sl=-bSsItnQW3Du_J$hOI zm(KD~RvYCB`6Cpxh_@&_5LaiZv?l9;McHC(y}G8BchtoDf*fJ08#2!FXqkz+ONQSL z^*^CyraC{t7kvbBmwL2mpYHNu==&pZmh3O*!W)*_<>KT$*K4SNX^l2p9_2Z{bqmCt z-cY%#@}Ov}4C`*}t%8n@6~}^o@mae*X;nP?L-+N%?LjT4)x6pM^+8%`%Y|3Q*`N?v zMG#*Ia|#0_xnz5-dxI_s+>hGq1VvWea0wbY)4Pslf$=xXd?%wggt zi#7>+d_;SHzta$a{VbXJ8mV)QS1eIiuY@0gu+a$pNTgFS*tdP#)KG-#Zl~hZBcbl4 z(z{qH-iH$wk?T3VLi0md0m0W4}T4-|5J+!3p+|_+5ll>_O`t#;1gFO%RVrM>t$4fhx}< z!}7!0pz7|OxetyDQSiECbsR8KgU@At&mzn8!8EGL=g390p)d0f zAKvFW12P7n)o@N{Y~^YSZ>2X7PLRu+4Bi9kuS$XxeE(O8^4#(pf7fOVL)yemFjjTs7l`tn1h$=T|n` ztlF!W^-biBe*ixDEm)1Kp6^uxudAiZTt@)eF0`3Rro@3PG~|jfX|^io-rwS4%;KUO zbEKF=CCNJJGhxg2y7HOS+AtWxpog$-NoaQ2xYuq+?|U-AMLj#Ma@Te!{&@ud9@S5g z>f4E%%25_E(sCAHtpq*pE|(g7Ey*jg+Ee$^M-giIIH=UeKFUe38Lq0vZdtU=o6 zOfJ5?9k_}!flPQ0_(GeS5FsC*|J>(UW}BgyaMu{b zrEsg-$bc9JC_}5da-=^J=N%{KH@!c9loJAXD9w9`kQGa zeg@V~`mEMj9UvSF42&6z(sljnVu>Z$vZ+951nFZZn0fAiH7?O>I2uub8+Nj0T{fLP zG{dnJRoFKu$0|Ae^>|0IFmJGTQ5Lv90s#Az#_n#puKp(zm^xlYP#fzRRNa$w;&E$` zm-*{`Ab7LHf0Y*AKj)DvpTHYtsr1RvKDvI7I;a_qC;r~UTB>I?D9F%&S4WE?;M4Dm z1;34ozUw{FQs-+gnPZMcQSv`vn}~}2MAEN76uz5xPsBt)`;^g+!Ii34)bQXnfiqIQ zOEc)8Kd4kZ7@>`6NpiR5sU1|(8}m}-G15$XU>OaHSvQ-gv~<^WJOwU&I1h zAXgv&@6Bka%fqS7ns1#oRrkrBWlz=M00@igw_caV)>=}t`nqacLNb`RhbC^MfF%Qi zjt~=S2s>q;s~)D$!G}GW8e@M2{67gwYER}a&T~Nb&jN*abq@}-1jehwhN84-OdUa1 zH^Y023qI4OWxF1Me^OnLkRuV?K1X%Xgb&{|;(-ityDRb<_x*EXG!T7)PR-CvR%Kg! z*t73i%_yCPoYR&iM?PzcxElI0m~*`R}PGOe@RJC z^W@ZQ%3lwu#>X$sY>QwNP3N{wwy33?J|d=u#OYnwrjL_it1b7#nUXhd&NT(X$ySdR z6+ZG*2ZbRmf%gg}zam-M68Uq0zp$n99js>O5?71oNUe^njXXsy?%G&i#T&6+16Q_`#^+YzP7&;`SZb{P@EufxcCzwA$s=y2zP(h^6A1OcuQD`+BI^3#9Y~?%V zF^6e9h5foILmTg>kF1n4Os_DiHJO}604s#xpyc;nm+jTo?p%5*n>3b-_qkXF{$nVm zmG^6j)(={a&}Tdl-_~1SdO>AH0RP5prg}>c(a)yYe9xM^2dD!^dli)cpZV4vf`$z> z_%uTc{x3sO8J5HPL;J?<Yg;+?OpFwA5C&sNNxj81~^Il4GXokOWNvQ;03K8^6c-?3SO(t%V zW#ADz>0}%^-cV7Qes#-O2{pOEO>QF1)Ft1K-<%R}%Qr_s%ChwmutJMqhXG)PEbw3y zXvyBEAR0g3RJ#_}RQK9}Gi9~0>|Oa0dY)W)N;Lkgk-RRj0w*+LwW>{6E#KPRT;|cZ zqb-9rHpxIG97Iqul%z4Igs#%`|_zy(UW6&zSR5c z;dB_{K~wGaZx!8zJ`V>SWn3NCw6~_?CRXS`inXn=?*TE+^F$P+BW!}XnVB|qzo7t9 zQec}k75iY7CSF3;Z_;5T87J>y`@=3xBt1#iJEsCo%XPvgDFSD}vd7331O;5mi1FdQ zibqk$MOv$W#IxZQIG`U?qtKfBpIRz6b)-F&>kkuJZ7Ut?aoiBuU_tnD{3XP@(EV`s zo(eu}4@wY@{E zy`HO!HT%)21d`#AM}PWdUD#Bc+P%b*;Kk=-GtRJxL@u;cVNb)mBdFm*{?#mH*NL@- z?g5F8%ig-}6FtRG_y;OGsV1pAb1WBo0wnSe>{&zYR>)PjivR_Hp3}QX;L(3%ew9`p z;E^a?DDv1WZCRZRhR$lrSK1>W%DK5=?p{l3)r{Kl&W{VQ|MO06xwP&b2Sn%ljFo$y zy$u60fJ5totRqi0v>wZ|HmVe|ZJ7+e%3{*S=l$-jJL*;84)m1r81V@E{Vx|_zUPcIS&9C(c@ zDzV<GHRJJA^^$A%_Xng9Uf8PTK6e@#Qvlu6WrKISUD{Y1kjL{RvBv3K)M200e`M z5wHOAXkNeg?#7NaZb31sX=fL={5(C@nbffATh-aj_w}`Ib!;|!Xli_O|6n>rcz68z zu{FGGlDxsk(96)fM%(`k&>{hPiFLivriOXnLUPo;&7VYP`1FsTCX<4sV%m<@%J!0j zdox4`&HH`aVhhrtT7gNR_5f57|9Q}`^$BuCVt(yJCkys0@OzOci;*^?;|~{8iUlSb zaZ!Up4I0JUGR=njk(5SV745yqG((KSXW|WGpR3y|&L3{R9s2%Q*q`DNKwC$vI`1S( zYEBP%Z>?STL7qkL4!qsQBi<1AJXX-3*g7blIqBJnXLff)?fpabUs#|3*5LUj7YB%O zk4u|k1EZflYN}~mJ8Ei5Z)H(u@ENJuhfLBX6t>z5bnHaxoPo}O0QgGv>#?k9f2LzJ zG`f7otX7vujjs{|FfO$pG|yeU$DmRuxgk-@E~+~zN;YY`m2_x2qIK%C%OL+?wX;I+ zi=RdF>SAqT7J%#k;HaQ2^UDR|xFeOEBl2Zv{Mws%KyZRMVBFf1X7f0}Y?cL{SVqPm zB$%v=swWgA~OrN1iLiM_J?`(5&;1v!Kuj49lX_oaUrBw;(>PTA? zqVOG|>C>L2arQ&Iqw5Hs>}=14Ao`~xeI);BfD^@l)HH<~i8$`E?asRSGnKCketd4- z*B7!6x{)Y*Gv9idWq$qJAu)AExAnDtEInVrYh+9kpc7Y=3y4=jmWow-Mud(79uUlP zA+o?Lr)Y_&%BXupO_GikJXMPq{13NbMQ=#@CErAROA_LPP9d0~-j<7Fe{C&;HsbvvTtMvP&l;H?g}e`MKW)kzVzbyrSx=`M6T8M5IozJotjhYl$y2tH`DA zk!C~er{z8Gz1cacy1I>#^5(0Z{zW#?n1d65$N}$kkOpwd=1Lhz2hrkSH4dHTwDrq` zWb&6<#tw7S)$`{%oFjFmKiOZ_dBOD(rTGeokOIu0MKHJw!!EgHwfo453d*c8Q@m#Y zw)-in9WFh;FI_TYYjm|p4QVvh6qf}c!ShrrMmJpFlH0Px5MEZO>IlPd>ei4Ulg13}nl@x#rJ+tNx`r zPmq=~|ANb2wTvi!^8I|XEi*E6iKVqCZ(L*H-2gYTjX{%9?`ali-7h;e6I%Iq@Ha)C z;7ZeYi(@cBVMPe>0eZ`WUVrK_@IS~F|MvLgxJ=p{8_)#?sC4KAqO>O|8%!MEFG&xo}dAePV4+GlV~(+MJA9koq#bPK`H(opw-zq7s@9oKOU zFbKHQM76Uic3;kD6gU(n?kQxk*4?woO%=d8VmIX|=VgjQ@+Tn7-T_EuCGcGU(C!*q z;!G06c45-6{!qVWs$?L3z3L?slT%N@Jrg0(UcGOzI~7s^-@X~wGNfGy?)Q&&aI|UX zYv?}5MLsJD+sv|vk7I*xNK05DFRr>y;9}Y`BSmGsZ0M6IMOzsD|@HE z*kXa-OlvxMbbR@MuIGqL=9at--bikVuMD)Fkcc1)IOAL|E%2l8G6~3%X!in!6+AK6 zSvLTj&4@xzi;97-@d?nV?FL7*G1tW_{39`cvCXWD9_opE#OZWCH_35uWYYi3=;PE# za-(y_Zu(=xH%`lXzK~}bw#MBmVEy@tUORB%{q$i@5-R>1%Ft2F4;7?p~C9~t!MAMZ3Z6P%l0#U{+W>C zmxlGyer;E)QWL9MKo{d4;MrNCzr2q=Pm~-Dm!FdF_CUPZJvCMgxQu^S~m!Ss14r&DU}su?K*Y% z6@RwvhbEtS?Vfb2HJKtyhc}~fMOcZ<;H`=(bTE+%}p^$v&$byJBSwqcIMo+RsQgUskk1Lwj+VPG_ zl}jfr4HR$6v%~P3C=~3#b40s>3_VBs_$DF{!v|Zn&FjbXCWBq}{NHUqY8x)dbnyyG z5UX3&z{?5!o`nHI11pF}!X&F*5%A<=f8jz^k7_79uFub_C!oKdg}#h&E?8S1N~ae5y{Bw<%Pu(@Y)uFu_bbpMZ;|0tAvUd1yPlt_lAQF4LGl z$3XR+qv=qia?0m_4yyF_OC85F%<{IrbFiGfqp{ITZ0N~`tW<7a%@e=J=k{`AtKk=0 zUl$#t_*WLHK0jsrPc5{7S}=|4zTg4!sDZj)W=bBuXgPP^luvWNO8Hvhd}X~E0b@$a z9<6@X&D?DRfsQ3Pb&^bfc~~@i4w9?==T-9*{)+dcfvK*qj7MUVUq;3Q()DiYI-{w0?0zn{fmCf-s}M0uQ(H4K^3Qk{Xa?{}Ct?$*)uv{8#{M z&7jA|@~E3l9YLuVb%7f2d&LZ2S(cZYEy0TiCR~#Srl9{Kf;^fYRQ!fG|4_NEX0@ikL=X))VF=O||zDAN{>+m@x9(ZCg84t!y z(vUcE;ZG#L$VhJI&RGbmVcqz!WcJ^Qi+ubw28f{r);dV8OCdQqoc5A8mCz{PK11Vy zWXMUQC)EL2c6LGSLz}gY+XDX}9EANE>23k3k5D%PoBf%A3K9(!#j#y{(EgR#RHfnU zA_tJa*;XnBtEM^is>0PAymZb>;4|Axgs++^9wS}646TO2`9fuZCOO#KzU?GNXI7*I z__GsQ4WqNr2A-zx#N#Vn<1<5|RU$zhHBEm6_cuo3-5$m%xoCT>wNd7;c{g^C=oJ66 zh9)Iojqr*>0GC4;{q8R8ho^H~$z(kqn(_?yo0)4VOKBuiE_%hy+AxqT$ zi?k=%#+5z`)=4ww;i{jbJy3!JX!l130PsXlR5)M>1#FFpt=0ki5-r1s)kSZ|U`?1c zTHsgKKhC;6&{W(=f!2KhI6jd0lK=t_0}TwAw^W?39@dT4o%5nG4_g4xXz`OkVtZS! zL+WQ%s0yN67RJA#DkpT5?EHgS5F}>fk+|t8w5$U`b0VNd7Hb6G9P2FQn3NY%OEAbgqL=?VHj z`LiSd`SCnoqyS4Vd1+=@_7Fqd6TEHI%n|=ueJji^H*xQ~DUuZ$aM8g>hI;=jS#620)r{*oDaQ(7GmWu?@xD~xQ@Lswq za1yAaxn@@JA66Cs#2Qo8^~-Yr6sSSZ)L9h6VP$NbSLv=cpKF`mI-Sp|cy%0;7(DI7&rUDfi2b}18D#U!c5QMAwpwIPO z{%wYFwGD8G8aPYjm$Bp>YTDZyb&qa#S&Fvw?VVFdBL9Fl$D3*Kp8nT1+$}0ihz-JU zM0{6A^Oh4y{69yC24eC>NFeYvbHbUpX4Vcu7TV`rHCkB{$9p`hFB_w=p@t)zcSfFq zsO$LGuj9`}f3>40roHuucKPvpS@XVuS7wslj*=ai=xmAqsEVz3ua?StIAot`Y~DO|L#~ehxZd?L7O^n>SDRD-L;KBqpVqRSUnT!RmwV#t7RFAX zA@vBPjZJRV^-7{1Kq)+4zY96|3V*58Y)9Sg>-e_D<74LIzt%U!Rbx5&d;IW)nVa|r zWPL)*5BT0ZN$sC~_S*R<*Pamueozeo5L^+$q8%)i9a87_t=%EYapK|{8n0DVrgT^| ze9BDa+TS~GSU`9sJgE}3wkGA!`k&Bf!p|B z)g9-@x&e&Fsu+5RKN`5ViuZ<9IAfm!@7APJP}$MRWATjmqdG5WpLe67MxW0L_*M;A zjlJF&A(g6cCgmD%+f7uvdP$k~)n44}#qi^fX-=K0YUsZAz~y$6*WoR{8ICNn$asVT zG|C2H|KbJ5FH!f}_nKR*5@O3x?(&f6M9*$}@gTfs*8p`M4Sy&{S3MB=Xh;x*l%bau zrHgkFc2sJ(PP`N1xj6qpT^>0GlRz)Aa;YQPhjTFk;e+VT6+IMa6)NVZ?8j6F$ z{LeQLKAS7J#f;vC`kaDKWxiNfdw$!5S(io9FD{ z!4=Y0Lw(EV-&|oM(1ZXh?ty9VXW>(v&!b5S8p`hug9U=oC#A`B2)g;j^3B9r% zfVA9}4%r-|Ad$ik+fEl2I(x2~*8sKE=lvSr)F_-R#|8kd2#`ajhC+EN^(`IM8a5om z_4iK&J7$>CO1?4GaZY)Q%wusjUGVb3bO|dwCjS*s!Vq+9=-VC=VPtRT02nwmWkloC zNb)*8zj)tJXJ&`t1?QENv#I7yj>h*lT*U+ebQ3(h2p1HwL3r0-Af~wVwcVA|xe_*j z$BE)rz*Bb0l)x$J(T1&?`dWIbqB8-RNl#qrGFK84UHndYBJQsHs>>|UWq87nsaKdB z4_R$w5ix@RmEilC66uiatPHB(K@T+(R8^WrHv5$VSmposE`a!z9bLFaTN`nY=S_U= z@l{>?ok#ucJaW7d9R=3Y+k)5=o9fxZZ^K=X#kH3ejWvLvK{S80Iy4Pw;$^Or{-_L2 zmX$D$T*#Ld6g45-PRYb^iqaMFKPLc;>Aw6cxy0q|P~ z;V(VoGl34h2W9(_4?eX6yR)FQ$81VPO4>E17bd5#NtjJV)TIJ@|IGtN&}5SE_>@;X zU81wMLaX}cy4Nw%>ppi|EaI|*`n+M?ovjvaIqLl@8tq5m_$8NNHy+3W zsVdZ!;$-5{uPi_@--OYSjQ4B)7l5utlK|Tq7A|`2y)rE(4A3oyNvr3F_a;+)7?|em zJPHSIt9XJt5023^e+Yh*0zAzf*lS}`G?B>2Y?w*>d9zURO{*mE#LSmwEdvV+kgfx@ zCfDf?2lxZkSWEy6=$a8bEG!=3Qm4Knf+~{Twlr6r59`21@BU+&yg+x`)~P-VU^^j? zP~f3ScKEqkrN@Y{{^YiR*|lvxnY7F}lAp#1Lu2tK`M_BYIJ&w)arE)X-uF4vG&8LJ3bjOtb`XQ?8O>~m@ z==>VdJ-z)^FX`1+_CW8KVgzpgl@PSJ%A;A>6U9l2;}o@f%nSDf8DF9vZgM2ZNDcgcP>Q@FcC^ZV;AYI3#GxV#p`Tlw7imTqP^lsQN~zqNIYP^CB~d8I{MqarIeiDD-3Y)~q|xJEf!;)vYFQ zc%p36RCG^_I#c|w{BcA2xl_zUsL$V!Bt^i`iJq<2mR1hORaZ^3PoLacmV1x$!JNw{ zvecEJ=%)cMD(}h*7j4?$Y!u594K&ex8nHt>7T+f&_Bf0av#HUJkwqIaL$q%JbHQX+ zZO;?_w$g_C*0EwL?xE7i6UMz-S3OZXt+CjremLtKn22IZTWPO8kYtg0Gb>|EClwJ@ z_x72oyaEkAZCJ952aDSimW!AgUbd%e+&q`p`i z-rc`JZ8_h#eqqa{+bo)QO-sfLQY12EtK}L)NrJ%81>Enht^Bpe=X4&}*j|qlmubtb z*={)dzZs)!pLe=YFt)qkZ!Fc= z%DXLKOTkPBHh>ztgm0Db?_R%;OK9Pwi~Gf{4$k5pHaR~0_}vj3;rvm3v9(Os>35Ds z;Y0!AyHMd@wEgtMkz@tc8+fy|9r(@koXUfJ=_8^3)8G~#LtU`UusSY={hkpdxqQiF z4&MW4mxwNXeq#M=Hr_z(riG#pw!hLxG6?N5dXxS4em(GjV5!pAOuEkW>I|76&GCxR zRg8CIuKn3Xe50V|ZT7Y>wjUbrNoc>#(|hVrQwkX+=8VAd%;Ggk?4vqMsxBkeC?p?F zvlzYP98%WY`hLXdON+$}y15sxnfHz1ASmT>63)7742#xq`h2M{s2=B*UNhb+mlp;! z5J;fH$gv6h;x`)Kt#Mz+9%|dVI+r!EGI!BF!|ft&dCi~ zHf?~f$waLfXcX8Dk({bExxjoM40bY$aZk(_kUF0-?$Ntf!<2o0TEs>eUL_#T;T;#+ z65474NhbeIk3N>vX7-`CNO)H$SV^yD#jdfW=ml5t6{=-45DQN*7D(nq= z7jow9nG{;GNw>7o1lMxp{w*uZ^tvopO!mX0^b@k6gX#B5-r6~KBQ&UWm=|q1IHeN4H z_MlRFN*SaqNo^}9IjBncFu6O2;Osh&bPg}78*~@@DH%vFhxyq!ya3Ho+MNrYTfq;m zn%u5@N$hb~1Hpbqpjl$mpSweA-q8qWYL@`0mA2yoi`?LKwo91oRLTCqS=v#QEyr$^ zaVAH-@bg_=x0S>tMH5v0A>=Guom_UL$3)D2kNvKln1ie>vgSPwKqbVpQl527RO1X= z6{j!InC6c;Ae!%jJ4@=>;2q`h1maE(9*Sgx=YQc;;?j`V2==ojn@g3Ltc0`08pFah zoKW+TFYjpu{I@qfm{6`F4Nn;0jAIV{55)y#SK4^(5g8lA4Ed4`#JyNZaw7iQ#A-0) z+crWe)csMM4J+1Aw`6>d0`rf}OOa!0D0r2h6pWovExL@{oyH4MHno^gZo{}pTu+WW zqeL_MNdD8F^QXtpfu}rrG`mCL7p0MBo3rtb!5>qj#!~l#U0nseC#S1*s@Z~L4M9z| zEq8^<2?tpkJr9lfC)-y)MT96AqG>7@uRNUSVWI`kQ|PF9{+uuEzzee zT-efa0k)rRM|*!<263sT6Ts=K+?(xISUP?!_UB8)VC=WnJOT3x>tAHBg>s4Ra})U9 zgUOV=&Uo|el%LS8Xo7x1BI{!+HWq#Q--=w=BxS~t?VT}w8x+SbZ@^^l%S&WU<)S%= z3TYj{tc0fi7FK`*C~&UpjNFxJ=U{VS5cjl7$%lI}SPSHPb0Jy0v(M-3D*>NuaZnc#+=Le>q3*p#m>ByRF7l0k({nCD>!`bL`a6U1 z^df({du5!C*sS(Z@kK@2Aj||MVnR6b>nMS(!G7bF9AR846dA~=_PkJ}fouUIF(?UA zE%M`;Lo;T7Uh1}c)AlNfRDUXJBtioNqTYFT`I{T!I8E|c&FWf$Yfw%~$mW87kcgS= z^+~+2Mz`GKL>`MppCL1i86@xZ#QM5q_OjP#S5^W3!hf7iE8=Z=9dfS_9P0cfsjU?l zjYKN=*KEyjuAr~+oxc+6wUHs=@f#^Fn0+zH91^B~w&3}x9EKU)!~IT4(>awk*^WKM z;qIsA)EFa+5T5>z4+;F_?g|Dw%#ihPstpukbdVpenYo>7t+yvcjlBDgarb*Wz3|se zoL;BY7h&4}dF!VUAW!+^PA9fmmO&GUdmawXTd?E{93Fn)8WuPyfOg)I#K3UaYHi!^b2bFA=_TL8rHlno8D`J1 zJoe5?=Ok4R=oKqlS}G5)OWS89&s$5T#(l%W7G`)1K-57;95Hp2Fs!UxWmp?`Z%b&S z4RHf{@L%#-&Zg8}%T$7gg*~~$&iPw~%lb7_oDGvWd%%F5`p#w<$}<4rX2O zFid4Y?~$g9WTijrzUXVr0f}r&D)qs^ev(wUmbFr=3(0$eMtUUM+ zs|$4^tGH}5QdVl90X@)O$cP74TJSir(d3&s|LdDS788)&@bFCNga=K)JW+{26DX4} z#*1nXw@^c0U$pRKI4m3#)ZAK=ym{ASU%PkK)KYIx`tvyDTd2NQ+1l4W>vvW=L9GrZ z&JsoDGWy-0X4f2q#!%n+Q=MF}%Nul8C{uC7q~p$k|f%rgA+2@}i4Qw}{;V|BID`m$XwQ?eT^c;)aO&-CwV3t%=m*hZ6sjkAUR zU^1IA#TZ@dlaN1(p96Y_oma``mktyO;BsUq2-reE$UU{5D$J{GD@bcO$lBA-Cl>E2-rI4~U^6blwUx5xLZg(oRY)q6^AZFJ`9>Hu*suGu zl}fQLiZ+R$lRBx?t1z!~r>;(kfmv7(!OU5lW{PDOC5qs}*rv4qYu!OiMEGjXOEUU08kjp%bLJ&g?qqj<}DuA-Cu zg4s%tjGT2%r?sqsPn`Z9} z4~bL62DN~{6Px-QBo_m|cq5);`q`ha#US~_RDhUQs#rtav-M%N3b=~=osw8)2{yMU z+2*G^l>O44xd+dasW>i^TqQbm3_oT_+IH1fNHQbKvFH~9peP7N>D+IAi`_Y$@(W7z zG?%8Fmv*ANM8{!_Q_J3~VGMD#xUJ|YShKi(P&PX!?loYmJ5g&Qna58{W&tv}$>*^e zF4?)#d=&qN#*TN`@F(mi<)591$$*Db{CUHiXSAp&qRSOA0&Oe+gC`E+<))AS_h);+9KkoiIs>)~k1IGbTN(BK$8l*u=T3V!AO1c}QI}b=o zcZW!KHwUC!;?SjZ9=hZE9MpT?_pZ1s&wZ3beLQ?3le@dr$UW+Ok9{ zSiPnh3{__Y{PH@&hJ)IpAKtAMVQ%o+H&F4!=0FxY8&hrcMU|gqRA^u!Z%-vM;H>3s zVv_MA`jY@ab!5)DKHz##%Vrd_MqN4A!-D8HmIDb>3HTPXNf}D?;G(=D%K=60nttxi zp`En19#>(6Mw;=?tr3)XL#>JyKtF#p9oWya|1~8Qu%m$NbVHq*72d{(f~7CdOK-_9~SJE(&=f zxch9bBfxyOk-=tX7ctKxn5hscv1M;-Qf_7o54-z}#w*_OP&G$XO0@Gh=Aij;nKg9u zpo2((cm)x{)7nFSfr!5fplS1)6#gg-Hj#oZ+HW(rVCs+(m)~L$&!JvJzsT1H+@)3V zYWBQRb&<4Sps2uI#5Sd)==RAgUJ8n6@~hm8qIWKQV%a#D`B?_Pvn5$?9Bx zMp^HvXPuvHi(FNz-f0jcCZ%AfVp3b)S*s4YRw(J(j#TIIi7F0t(5}=wou84HBkevQ zES%`z*R&~0mZ__GHZBnw$toiWeyOV)+uo5Bgb7CYdUnWrm~If`1RGbR%pcn#?E^G8Q- zsjct<7=f&?!{DMw)>HenlvYv0Hbx37v8S8f zXWMP_X{4w~%xh64PG#ukaHw)V6xUp&Ef1@#yVESBUHit-$))qrVFHFc{r%t(@&0HkcH3PSA9Kh2Z2`In|n6E9UUp$$ds5Rpu>pkQrdCte4DZ&+-pK!nPlnI%Pd5_sB+dq zauND1Tbr-)^2jaftLG%rrO9`A`iK|_1|sSQlK|WaG_mFaRZ6)m#ixp%lj>@F@6#%V zVql&^nD24ZZ}?X0Y%>06m~yU7Ia<%_$Ko;0U>jI@j=$+^!D%-a$KwYdD3@Y5RZN&g z%4-hj#|xH?KQM@OQ3I~9sa^uMzQs2d!)So;JNk-m+5sSe|hz27S}C>a*M(?V~*QlZPiMo{K*s#HRx<>2uiL` zAu2H-lok1nnbtmhceF(AH}? z)x`7Ww5{z3NU74_5a*Sp1HH9O`NuCDz3gSvK<9s;9xP_WYECsvND(IM9xUxTET z+^93j0YQhZ3pLtu3g7I-tu9}24sgaYA%ICS@ApjCd9ODC0+wbE1YQ--vf%F$6<4> zag99d{<;8&A?53olQjHdWl5^qY7&%+W)ATU;TK2cqY5t3!SRJ?14g@XYcHzurwA_M ztwnJ7@tGmG-XRydUZDnO@Ow5i9tgAffzxZ2H6po50}8!LQEXAo3iPZ#?njDUQ|xo6 z`%n5R4Ulhu8T}f3-E?w_>EzZ4F}JizTL&U7xZP^xV@V=T+GCHZ4I!0JK8cg#M}kH8 zb8-^FM~bTuPwF9wEQ`Ch1E)zyp(7g(!WW7U~mMW1OE;#F1W zUWvy@dwK{?Vue{WQwWlH0I7s0=qDi_gGXj#jM70ouP~}@*cIHaoHT)1@$yi;nOjiw zD!L8hI33d8N?uHWCM~Lb3w-v2r=g$}eNk=NUCX6ofKkzog%IX81E%6VKVYc;HeQ*= z3vyp4M?tpY@@p(viVjd>Ip~HlO`8@Ha(9e`6J3JOa8YPj5V~Bh8&*W4q7~!}Z<$>L z^eB1LqaRnjk&O?KUTYJGHo)l>j{+ShK*Og!KM}nrAZK0-STU#xNRIWOOux`&`a#RD zxmQp8nJB9;BZ>e$)CiLFR8ftN$&L|uTuefg?WTsuCaM8{pcSx8EbG73V%EO+E`v~B1foi%M5^I*{ zvZ*kEJg`7$fl{!M0}l#>8^TSLDdbQ{_%CG*j zqY0+Nv0CkRIKTp0_}tX{x$r%1wCd<*M_-65*|rG0hP*0aP0=sRn|gvZEp`SY$V7@+K#` zTw-U4R85~6Oq;j?v>r6mh3+ng+|1X2nF@AW_llk|p(llWOCnMhAtRHw+qT|xoKCL|{*js(eKLd*u zA1xCs*wuyH?5S*2s)kzpQtY~#Uta!!NHn&)5<4ffzUS}*nh>-3x*5#AY4_G@SZU2X z3N_*Rz}@B*k4kAg<^1;DL=i1JRx$78JQH2caNOK9^iqcyIK?O2Hr0Qs84Eq>zs-KI zRx@c8H`YgIZWU&>j9Cb+GFtDgz-7A<0oMNsEKH~b%b?8#qBYS(w>RsN-ObH>s|}k) z{<770hd|^pOF$sAhVlhhImIQsOW=6vkvp{id2BTp&#yo(@gahd3~v{soBYsnF(e+c z)!+46QSvva4MPyYui?9!97M4C1X@%g5+2sixve$ohdw59#%FA`@eEwUV+1k@u3uE>E34h1M+wc^r zLEASswUhF?(n>|kNOT3}$?bD5U@^M|(jTK6o;~HM+KJB#g1=nZ=R#~cViMz@0w;y{=c7SDYQWPKlGnA8Dqtj|IBV2O_dS~}BlS*CkzKtQmo zjd)sdD}+nXGN?CbRKupIk+$MVn>DgP#u7@AiFIx`eJG^ona$U(>Vv43ZycAUdDM=@ zPmw&-%D0@DIhyu3!-5fCX^~>sB3MeUD{WhyIo{BRLxPvD-p)4b=MZ?Qn)eKLHR&RU zRO_Aey%{3l(UjLbs@1KqgUurVRZaoyI2|w?w|Ht$`&FL74M~A?+AcYzl;A7r(PS)g z%C%B=@rf@{olz>}H_@Aw`s7IBQX#9x(Y#S!a~21n94v{@d~36 z0A2N)nI}0_an8LWKKPvJY|{BlYpn&UPVXphF(~I!4@4~yWiyIsfkhlNQdb+QgZm@LC{69%*U*<(^2HzQH^v~I8Y=}g7K}G89C*ny~+n0Z1GZQ zXcQD{zN3@mlagH$TCo;%G_m&L1N>`~5}2FOu18qCjTEJledJgal!*#q!;$t$=mlFCzU^|$ zT;>j+V%|<~;Nv3-CuYx_X*O02MHRc4>5_B|nx+ul2siV(`IyYGerA68#ciUtNybhm z=Jq;nasJ@=w9;t47d{!DS@a1I6X!+n2v|(K2uS4qmcm+7T1^j4sE{~2qrrJa2K>r? zpVXEk6YQaIpxQ?CPLwyVo>zc`2yC0bda@E$-47xOFn>K z-TY|=KLG?#pHY@ITgZN9z06*i6t{jQS|=1OEo)wpuE8|C7+!NU#hK78Ij~(z9xpM_ zrfnJZD#!*CA8C)Z7l#}i>8@^DZP3yowV)NBaLm^`Ol~W4vXNLgnK=sIFin!kz&y03 zEv76f^#&-)LlG|-B6P8gSu=))K;w*HWXs*6_x+3&*+cVFz(-m(ldY>SK#DzMZWKqn z%M8rz^lk3CnTQtGK1Ipxwr1oLzx`H7;ou@448JE}fLz5o$jxJP0e(Y=DQZ-;NO*Rg zMX3n6bu?I|0dnmg3%90V{p{FDKXYT^cYd}Ao;+eFs4Z`-($~n(T0}k7J;b+)@jd7` z#I1IEsujVlOG45OK1%i2wW3|P=8@(ZESVsUZtArJ(9fEE#+tgGXHhV*fhb5a?UM2% z0Erlzq`A$WNxl>t}>C%re@`1`?d%Qs;sEVoxIz>?1ph1lYrjmc`g*1F*vM zo2zf>FFvnG_nJo8_UiN&DSnE3X&h(sDw+NvN_Y8!nmI++3+WfdXJS(GfWJ>hN2$GY zHncSFJlFHbfH=fvPXDuIEDsO{{MGsdRZbWBVmOVeC_R2gkgz*P!g86* z04FW*92QoTr`SK;{p3f`&5u{VJe82U^t|Rs->zpN5ATaD45xHTdB#%y>WYzuJM0PB zEZ?|I7jMD&Dr$f+aZYRjU2~lSa@g{|7ifUfbqFcNsIrj$dAE&~le(d2@08VNu){}W z00x}WDqnFYA9Nr@V~j&%PXA+MjC|eE@ zf>gQOI|OhqOEkYuT2g4oUKc`V1Q@|%ofJ`45(8son1~2uWsV~cqa`FI@5sCJRoT*& zN8|dp=5s9<4Qh0Al$azFXOp)}KfooJhbWMT($TVA^k)iLWT`$YFk}?~0OE`kg`<}ehLOu5^8r%m zpT7Vsb-v7@*l=Up-C1{wI``8kLlsSaN!!8|&tB_IR70kx4a@%h0EFP15DB~IT==6e zjqy=LohGwpQTp$yKhsvjiZldWxsb8tkh=Cvo>47Y-E} z9brFT<$6=+tu|CdQ zx1+8geh$1yAGVO*Q3Nre~n3alVIHBp7@MX_iR* zdzTX{`_M7(U+D)U&SL`B#s!(SZR0&I~a)szc{u>6JFYkk5`{Rs8hoJ@=#Aj#c9rUM>vNP)cRgr5R1txUNCpB4yQ`u_~H9j z8=v$C;WV9W7;v_ld>kFd?%P}F#L zkm4{7qQp~P+YFa&GO4y=O3o4k(i}irTV|<(6K=->F9xmRgvqo8(<58# z?9BX9%fV3-(UnQgmyQX40l$_%vQ-+WYEICj9WfAjO|4h6?pJ^Bpi(gFsPw4kE1c}4 zi6y~8L?GY)RiOlLwykusbGLx1VcdXv*XZuhAnZSQ*LTR)zD6pfi>dQLE8&KyMtS&d zupI<|!AvgAr;-AAAIdCkWQ8xhCy^Lh7S;Y}lo+vs(tVaA60#pUaRIcOS|j-vl(byHTM8&)M# z)ieBeY+)34C&eQF^T&sUwLxN{?-Wcv+7$OQ%igvV$#C4lP~kpmq$v*!7$d0XwoO_QiYkxNq+lWEkQ^}kx=lfW2^borWxE7U9&<-6y1l}0_55hf-4<-rhi!fN zag9EUNEej2jMna6kiBlnPw?EoDQlYXFdBQf07D!iGlTM3<~U~J+^+gZ4A!2sbI%<| zNXrB#v)j88C%rwg^~v|IvqGD$T8Zr*=DR9V<-*BcaZ)4H9$8!oTy{;YnD@NqWOF?S zZ&FA03aJmiv~b}CZ@w{0T}xlw8;Om4feO>9J%MmC%K$@o>H^|(nA3u-C{EA_e@2CT zR_s&EwWgj=QZtEZRKk@J{x}h4m;esq2Wv!_1*|;e%kqu+hM0mk3<)|=Q82@gXe`f^ z6nAIUscQDamkg3qfexP8Ol2EAjDg`)fdtyMOdXfixv^^ouTmna3ajRc;infAMe#ht z;EBrJGa&bri&qN$?sB?M6#0ExYfN9iR%O$9Tar;WVXN{{rIIaymk|u>z9NV%_5cXa zzDPK5_%?XL3%yb+Z?XQiN&_4d1xSPtI73+w6N-PyyCtR3Gx#YEE1T_jMmb4pMe)$N z2>m{RS*AI~d_`R_nw|KpNc%>8t7^^J(dA?=JAY+ZIIa2%LxLi!=QSfX+m@Xv1gd`K zP}LotXxo&0FwtJMMPavs{93E&<1nu4Sa~;5Klmrg%O?L|fo(k}I`a^=>EOW$H#pB8 zPrP1cb)*P?5(nD;`Kq;V+4@A3p-44b0vesry9y=e40{M-y+oun8n`hu=4r`|`!@d- z!i*n~XOr4w$Y1qq;ghZ-0sfWr^^kd5Adtd!>t{AOLpn2VCs}x)hD~=d=f(GoA7Q=E zx~dK^Y~K3`z{=ABkv*5tqhaP6;q1vW7D0hKPb*ik+Clc&_ot-u-xA|*N-etP+JD74 zeIsh+FbBum1fR(gCI!sR?{=co7pAyThb`Yae?6*is>tD6u=G&WUIafeu0FiZo;~9a zS$5|90K>40fZ*stMd8iZ6-6B1#XR902fQc-0j*p9M^xDHF&@hNI;Z6XZoyh#=odYe zzzb=gHq|_!Pb*7iBM2pZ3OlRvzt!z*t3hmHQkJi1{Phw?O4y4PudpYlw12tbd+9z+ zMJ-WgUFC*XwqJ2W9|O`5NG1_q(5sE-dsTi3CA74(qyQ8kZqH_tIm=kd9e@~32*N6! z*O##n^`qUW;BkX&+dzAHjp{$=NhTZBp%rQV+QXp>_Gml3xpAQECYRW|ER2x^c zE2xcZ67#UUOvBUfxc5>|9@^Mo5wkMliILr>3II@cF_MecCXPjS7ONLCK*A1NV^i8j zG|vuKQVaXxVELJO2FdTg zX6z#;c=6ldt($_Ed4|TqvQ=T7{_6n&SHuC33##UbLS9o-N^JM$Pcn}CU$GfLhGq>z zSEQ?C@KnV3!pZ-7;=V13EHXK9Laosto-)z8ICvDuHUmT( z%s7+*9B@6;@~{Nve|OeHhCpc{{FQ4N2;I7*c;5qL@2PqSD}K=C$omAALh)}x|M``P z6TlSuR{jK1KmO4`?js_5F%J>9<7>Kbv_hyUkmmLK*9Wi|FdR0+eRj#eI`xyzC@y}o zZlmk<(1iDIQy_IVp%hj_=wA~Ie0?Uy$AnJukHK?>Up)Oo+~1ZPq!I2q3kl5nA6*8{ zVEGZS_ket;0Ab*BhYI(5CjK`5$254i#sP~*wEO=$pHj<^L1 zVdq`gE5%;P#T=*A{%!m@!MjFb(r=1K3MSt%ToCY#OBKl^C~mhPh1{dR7J%Y;0z$^Jer3U(1Jn8c zn0(|A@6s4KWktmfhC#%~T7Q?C`Xj(*Xf@lWod2~S|1uUZ?lu67NR;x#=|6TB2;o(R zR{&TbV18JTIy#H>Q3^Z%vLg)KW{=+DH%gr4GaLeDx z0Jz2bsBQ_20Qc`lVS|1QT6u6p0xO^Tr=^Z6z<@9Q{_hWCY*4gq#H4y@s0Hxwg z&ai@Zf8X%mUz-8eJNlD__^(zZ2)#&aoqEiAL`TJc2Li+bmUtM>OX1-D)&D)i2$68D zF$2V={%r_{(806D{bxx3{^LJZ`vHc|>H(Zw-R}w`e}BLx0dEw*fgb?6IpWuHZvHG2m=c33qn&!pb-i{GmNZ$`9rsW1fp`O zR@3&0U8kdc9sV*MQjY)7#ElFI{U zIGx{XPE0rNyneLOV7|ejHbfC=6*f?ya?H^<^~sBxcO=dT9S5jsGl@(5=$oy?}Vts6R1!EM;9(S z6c+V+xj_ObCQ1RSoi$Rnfdp9L*T=d1Hv+WJ6Wl4mG{${ErenAQjp%)nqo0UCVK{We z`J!y5*-2i0gpBg~J?-Nc3>xPQTw!-+1l@h$%*x86Eycdb`%vP)mnZ^OXKs@v3-R|B z)=@-GvFXAkOk3B~jp-LGkA7-?Ss0|vqiDF+0^DeJw%Rzifm`tS3^rrHUgQclznbA! z)nB)℘eebl!j)6_SvUAd+A3kmifkW^y~-Sp`!4iC|>9leO}&ExN{>3Xs6+Gb$DOVWJGECDTppb?spbcQ-gU z`O?f2U6T~)FcmN)?NsmHs3l%o7R$BBUgRXW(bmeNUzBX6hY!PM5fub1>$?rZ8DtPW z3dWJMB&Eh5gK#BA;Dc1X0XKW6;2zE*Cl`8oShVRr5y0H7B#78o=as=8?%Lql;XW>n z?2lcH)KZ-^GqjA?cz5V{nmtMtr0SVb05*5%EZ>(Ocv(dGzUC zzyRO^)*;Uso0LT>N1H%&TcCcuQg<$Tq@Vizl=uKs;7U zDF|zOtT7SqRd%IuZ%_ioNRC@~(QT3Wr=W*A#M}*yjim%Tl#~BOd7R%-g4Msr6(w6m zG{ggdg}<*5GZ5d)4gnoIgahX$$0NE2UQy8(ai#97bp4N7`jue2z)u)BV(6Z56#K!% z<%2)_KLS(CMhOYX0-f{O@=|vQzuo3zl_BY=F2h^=m**pcR5m zEp=fUcIrlU`<{16IuJjoLfQ(JF`P(p0RD_5$*{5Sfppsv+or60&2zcXuE;C{x*?Y>P8WGZjR}@d@Kg%xcWSqeB8t0P@A_>vs-tvCN9~SbZ@No57}27? zobLyh-Ibn~hJ1W;%hf>?Md?)>$FE1TE!q~-6B8|lo%H=$e0stouRj{7-?HA^y!5v6 zZ$?QfidBmPD^;YXSuxiA+;EeXZ0WihKruaOU{yR{BPOG zn7)zMk_?`;fK%sZH#n}=3QTEx^c)wR47S;C%&xmnSfE!Q!RGEEoS?6Pv*8!_eB|#9 zosB*Yx3jVJWh-@7IW%5aXMrD9d*RmSs&XbeP}|vgZ=F`Rm&DA!>#58Z|F+ z52$T6nK-?*oJE>BgN8IJ7(_(!$R=+Xo0fJ30yu_Q!|@l zGL9c~BZP(QoYo+0nA{@GZRuJooW@iC)8d|c>S#fFp^FmK7X3^0H+w(mJ2{f zNwu1i?Ib!sdDbLhW(^4~qJ_DpmqKpHHnCoT*HbVn za#b3eKUI5Ve^=(5SV698W?9g!b|_jX;&2}}Tiih)_94}oRku#%G%}}K>+l%~kMp7Y zu~0s-6=+q~^&(zz*mJ@`*V@;jTWvY+(-owraVvhJC053n3SW>Nai(jcxfw~wSiUKvd4G{__=qvHk zo8@If-W2|55iOi#tc~NbYi7$a(IB6>otarLa!b>mDz&=lzS3OM$3S$yV|J-$=U!PY zJM)Cl(ouBSV>qK}c0agl9&`1CWKE=vJ*_yMKV53-8p#gOs^wFg9qR(nv3Rajzh<7$ z6>+DxI3DpG6pM0zTXWlxP58&Nf&=QOK`sG1wnf#D^A;8?B)T34pBRdGO+gGdC@_TL?v2R&_T+2&-)*FuA z!7~)@`RzAY3%06%&FO!AMF8s8bIPT_im&}qM8D<#zZ#H45W^zqqWq(%|HtEho&mwC zX;aI@{MV@oW=H;cYY#eLcN}FA-K@T6(qo7x)=$+ z8(whn7MJI-Yzj+Z@uB7-M6xx&h$Q!l$#|vr0P22)0|hh*-o6PpGhM+Mu6Qv^uIpt5^!z$ z>A&9VlfWU6V0a>i5%UT*)`;#A%#UCfEE4bkT}rDEmtIO~2}CnW{MMdkYs z@s(zU{6fvZ?^O9;t_xBh`RooMR&W}VBHfRUgaCz{f?{OIveBe-6l^jfYnP``Yn;E0 zmg>-$l;9lQa&GXNUUeU7bud1@!mwWi>6Gvr`>xtZq3@M^T9b?YHkXk^+4)I}kV*TD zv|^^M%iFRu5;ikejwi$y42FYA5P3~BhQbYQwk1Omrl~vvg@)|DQAzBaB6h0@tMLqZ z4_a6CX9_d9Z{u=y5JL`W;A8C&ZYS;*g=v>bsos6NoK>KbIY4jb-n^~3JK8*n80m6% z2sIdgI)26vA8QU#`AsO4&a6!*%T{XBxNPb?*hXZAdAYfuv?MabT5x`!)H|{JCrjMM z{l!Cd6#LsD7sn%wWTx{4Phw(XqK zdT774$ilj@n%Qtdz!DuDo$oaaA46gOLaoJySvrv~D9-BRd%*XhCcwqU1}OZbTwjK`^qr%fkmrO>>CmC%Ru6UxhGWe*3fJS}L!6`GSzY1p_QVF>I`@#w=#!ggG1DdG(Ovw-kmdd6nd31%!`W@$_@5*Ho!?O;;Fag?EQ6U2<=v zpCs=V6l~2Z4K2xJOb6DBgwr&%8IX0OI|hS^h%K<@h_nq~&$EC2UPh{OoZ<~WxpRt4 zQ55MGvL~Bf?9x8-bfMK<4~J1jT$3rih{fbue7V&m*Sw%~;vr7`!=l46s~dY>ry_D0 zi92yu3!*Y9R#yM(e2vt5sobRtA@2T0KN{qoL(F+?cdf}d_lu(sp)=)GZrw!lwzC5+ zl`rIGMb#8e38pP8X;ey5#Bx8-WqZENA~BaruOKlTq}Ep5@U7t{nG;7g|eD(M{I$FDKrn~YlD zI@*dYtTLVFpJ|vWu^-ql_jMqu)BEghhp`WF$#iWl+(Pn=E1k-lAwsoy?GgOh4Wg5Z z^rGwhF+Tz_QNUutS5X+pFR35^_s~0`9`~wsjpYu#>i>X!SZ^_cYIY!Cr>VZz2*#zq z@u;tsQd(3SF~Ap@%1a`UurS)^vX(u`87~}@WIY|T8UK3wV7zi!@_2ER#c_V2(wwpR z*xkzvdp2cZ<1D8pC3qbF>;jRy8K?1h`7~){GdeeaEK4*0)2e?`(}Gi3Dv_fAi3ZBc z1DXQ9cS?&z7NNi8$hF%g|JK0{+IW?WB;L)z$H8>=5*OmZ6!zJY{vZy_&DjHmJn3l1 z;~yU3qjmDy3f-dVxi!+nKfRCS3-=lz_<5@HajX_S$ZGZOv5?8&;n?ig11ZNt4_Ic= zW)Xx0piQmKW_WI77K)-Yitf3`XlDf_!w(=f0iFiWTG5VPnKzB7!)~7HVFDE=*UER7 z$j(+D9h600*UjJs=f99vD(wD|{Ux57loZq#!>FHYJWZoe^0-$`<1nE3wILuZO}j19 zfeD)-ptV@=(B~|^a3l0MI-0~LR8ZZ{De~g5+2LsNUn?EYlI5U>e!oCFHiJ%XcB;6J z7;AeJg@UmOxNTF?+=}R8SM72jLTBa;+Me8MR~F7H3Q6b7oV#lZa~>3tm1#Z&~s_>SwjOcuimG+1_w1;i;tAm_OM8nv zK#e~_@`*Y7sE;*tj;w6uK;V8z6!@oFa= z%SVkA&K9}egcyNL2qGTA1>tfK4l>6q3q?!xhwSYE?imlmw28(iOWY0v4?Bp#75#1w zK-z~k9jXUXi&&*+S{6N{VxyLHpU9@D_0;vg)UmXHQ!zQ4EW*40q^S)8o#o^k(B! zVOqhT@BGkaHmCN)mdXU$t_fKAN(~FQW@`-nEuXU&oB{+hH@z30To|5y7&d=0#%ygd zQmhGnLCEFYQSn6ib!NHX?B)Wa4Md8xd4N@iV;1rq{#(hlo1c zfmHbiN8*w;rgm||EqjW7#l&FOy3)|(&+O>Se}Fj6T4SIp0dfEb)#7w-MpVeZxnzb- zTKkipp|rjW_Eq|c`UV1{u@53bq>MmkZr&xbl1F8zA1a{x2ZxM?hk(+Ap3;5M9N}h@ ztR60;3z;_oCj5*xHiuKa5vR4cJm^h8&U+3?hV5@%lLM!x{}~lO+036^2=ru6Y0Q5# zeI1k;{ZN8b60U7Ig7EFjr?5E(k(HTWeZu3?tE%~=S7V{dOY||0q0V{10YeU6D}p9xA&Gy z<`$aL$w|ZGMO=c#AYfrCJc$`!sg0FhXrA(6a#c8lb(r_4i)uXAQXd` zE3+u(T1S4r7B%CU7poJTwli=lAOqAgev+p3bYpIQ1zvS&7lFJ4 zEHvJnU{49)u7I4Jfw#<226Oz*SNbz$MpzDRP8rNV55RD1nI93Z}q5r)@V8FJMl> z!G|OM`c*Q9rg=Hq9?mC*USqW@zxb7ev7#!mB=QgJ8}nFnAyB^mS!| z(g}Tc`At)n0SYFH;$d89fk%r@maEkB8%(E#l41vs>nRjk?ediFf@tuqRHJ_7B}V zZx^)4jG#C>{V(CIhS8L?^9o8Q>CT6i&WFUnDkrCZmo4PZp0GaThaF4^28tn|9H1QQ zt+&jOmE34~i3fqM8y0_o)~b%r3>!{vKUSsOP8R%DvCLxlxVl+p71Ps!9?$7iG9X^X z%;VMI+mGUeX=bI^oN@X+A%^4_$SMf9zqKX*)BwwO5r|`DWxhuCD@!%7s#`&~!eY&* znH*jSX=PX_CCk-}KpK*Wiy-4U;+sEA)s_m z*3H~0VBotA4&@l9m#_lLZDyuB?J}-aGSOh@>)p~Appoqhai|m0!wB9MsQXJ{AUhzm z?BDBr{*NIZZ2%Q4m8RpzL=D9sqzx^U_*Uj)*NnX$mv{xhtmqVWt`HAagbRvra%`8-ctRxP8tjVXGi z2U_`jFDwtL^F($(7M57#0oLc!A7OxHm1OS7J_xlEVcQ5T%VXN2$2ylymcq)(EZM*3C@iwqO|EQCR!7*tDi>0h_V z6i<9X(<*?X;$Xp7Ej=3iY4zhXbF|55ot3zI5ycJ69O~T0XVgqM?U1dWDzHE4Q%d4@ z13~VBZ!0?KHtL_w+k0Whz3mPssy$k6JFQ(vL3%-X{#YAfAqA&S#leGbqui#rSX<9% zk>-WyOmhLpWZ-SjD;&g8#S8x!|W#W^r)z)%`=xTrgO&2)jX={ z&L^k*)w}5}SAKCv#PJtsE4h@_>OW5EoF!yrY9^{J-1&ZDnN2qU*(M(SPc4l$F6`FD zjTpqE_-!f9uhr58$BTw;ur;eIYLxZcyPlYItV2!T*4*W9%}xyVINqPgE?>N6r`1?> zPvH6K=2~%xuO4-a(KF&XCzhFj+WWW2Xa zgn*8TIn^&YUubZe%(U`K6a!x~wQqi>R>~ni*_jO`vN{S`Yov&nGj9DN0p+naWA(*d z&>+L*Aum7*^6snN8U*Yor7O^hJ#pz9qS1JIO2JMU zMzg0KA`mU@axNUFC?hC=P<`cER<8L|Ww!S0mH~_C>2BbJY0JHkE0gkdxm(5zKJ>dV z57kDdvzcxAg%Z$hmLJ*g&15F@40wZoXVR| zL(?q6Ut{2-N$L1TDq}GuF3ZKTqow8v@AIoitG!yS>~wjvY3}#-7rNOxFTorZJ~|{d z*kf>QSZK_2v0hs7YzQe&Bq&FwTVXIFvGeM@v6`CEkQAidOb*$7xGO1ai?k=)Rnt8&d%8lbSn8zY!J9k~ zwH_2&klJDtof7qa(L$9s{2Vo?W&h$7BxNHGZ0;|+Fw=&RSn{nj0hZwp;uRqYDL0Ce*YF;_K&rf2&TFh z+0qN{F4cw^b0>77Lj?9Jp=NuEDSe;lJn@4_Kmwf4_WzK>Qh5yQ0zfacH5Y6Jz9U9F z-$W3&-&DW1*8lmhC~z|5#%$w*$^ZZTF(O(f`2XFLH6`HZ7?O&3023(x>+?(&kj;Vo z58(jY=6`=uP)MA2{$*MJ-+f^m!0e=R1(+lFfA`gz2zkL`%_}l z(nsl%=+|P2S#OdX6ae4R(5bNSPa;U7M9=z?4ChJ!1|dL2AT~mNehg#ZPpi@~6?9>V zvk_VTXJX>gIbyuw@9Lq6Uq7IoZ^i`f8~GJBXYpb=z5bqu zhUDc@pQ~n(#RsK)Ol|ZWHL=_PdFvwFY4PfP$9_o5{Q-`SC!g>}Y) z4DEMDiGpb+ylHWYSC#NHKhV&uQV!0t$+MibB@o4Qu2i0tF4T!hp2=3W_g4}+O&jWT zQ-H?D$2AEotc(hO6kRVXk{CameQn|Et%CT$nR0I~b-B91QolidLTb9nj7pJqaQfi+ zaJ5V}()le9)naZk*ZflM@{65B3oAV{>^=G2pwFE|KHb2z@EeH8z(uea)=4?o3QdBa z?Fr|P*+5>U1S3A(sw~D?V%*_F8r+!=!z&Gj@Gw1w(p_}91X5Oz^Oc~uS% zEp!x{iYZrzL2!XjFfClgmzFMe)%NQ?E_@*U>DIK9qfF0C(}u@W@}Y=`QCnwEXSlvj zL8eVVY@YMNH-rggyFq@arBWlZG0%BnRCnIeo`DH5N+GJKZFysM>%-}5t_OBYB!q^0 zLntwQvX7L15#t^iF}@_H17buB2f=px3D)6eAzCJ;erI~jnQSl0Ui@!KJEVxneZR}!2iXct*ihGvk%I(V*HCYdT&;bi|kmW@g;{RL2LPkfBe8HHF zv}aC8lq$%0@8ax0>;Sw=!T>TJm;ojkxU+&TRY5Go&w#k$ME@<3<}f*J&yg+XD?)!Q zFVAn)(=~bCytQ1YO2ov(oCCZviYLX-A2j_$V=tf#Ku87-+!7KE`7_lKll&fAtd<|a zAiy<(bLb(6wj0aHvu;%g{A`)Hspu^c2AVlt3d&fu1mkRS$q z>=D1!F_Bd=?2yIr=<(N=B`iy;AehA&e?ovp{&6mQ`AD(S%(A$7ik2;339wIRmwHa*2SzTgL=n$xTu6ylJkh7IFh0Bk6t?&96V1`Up49?$|7#70t-3%!$sVv zb|bZ{XFTk?KU$e!Q@&3RO3%G{Cxp~2O?P*lZ0tuFJtw!3eY8G2mWA$+fws>0<68J= zBODwz`aK&wvbLKn=2`7ou0q^wCgU=CdqmjyVWLRY^lG7))v1kOrp6epQ}avK0)c>x zv~+c0R)LiYmoGQSh-?jyH)rEfbSMF_c)It1noHA7^fCHy!CmYy^dxz5B!mn6BTh3p zUt|#PcqL1AB(`7Wn2wC22=JW$CXJjbY2*My^ISbsCk>I`Jgpr0GlfX0(z(p}kU&a! zru*abAvq#myP2@1a<1-&%0R^|Z|`9gp0NW{dgRxjrJTX!I*YrV#LDrnhG>_1b0)M} zO$kb+0&NJapT2)oYc^xEoT=+gf|}`16ZdcaXcJQ7o~|$)8?P`PmRY6>UB0~gF~hdK z^M85IMMtS3XcQ=2ubOK&0@7Quxc8dc3&{Bs9uvL&f0%pgsHnQOe_W7M5d{Hhq!buL z=?;+`I)tGGX6P1>4h00HVUU*Y?k>rpyFt3U^LtS5`|){y?|T3Kt;NEF19Q&obM3va z>l4@cvcH4?AFN!#y4yq-AH8yPX`Vg5sTS|yPF{FcwOjtCkr&f-gjeq|2KcL1_HNv6 zoChbl@Y&PdxI|vZ66sEM}NScs#9>MVc`77|YoRbx>s z2FGI}QIDS;jpR=X7O}^N9PX^S84Y(Tf0#eTq*_Pn&m?mq03>}}q~Lx1HHd^O>=AkuO>hA<>Ir?UZQ?o2!NNSJ8QH$%k@?JxnDrm77T_9)Y7|gSq85Urym0x z2q|g!?LEAM!TGp7o~l~0z@m4X?{Io?Cl@T=EzdAvU1pcXM3_5SmNdJ}SOCvl zS7(SeZN?y04(sk(j`5vU@EVB77#xf<8sphf4EUxtw9|6^VQZbOeEQPy@@JKMK@l|A&O}Xc?b!P^J*M+pce!hRih4kzxOi zKb;92^s<_+ao$J0JXvjcqwhJnt7H)W<4u|e z?+xx3zc-dII>^Kvrb%SAPGgThnzT=rTn6DiAy$vmHEA92x=&CtE{R9~u3)fuQz=7E z=>C(=OnD_ar+{a-D?@iFb@L{ zrJ5Mfzsq?l()QNmtO#EdU{*EIj`Xke&xGF{Zi5`sk28}j>@CAKpF)$3^XS3D$F=>D zZD|JA%%4Wg)tXsM{MBQEQ;DjMKgQo9pp()Q> z2fcX4?=eGo{xTqevqz0RE|MeU*z|6j!s+6mY+sU>zu^S-;A-VcLh(Afjj&b;4GWj+ znGttyjkdU=>*+h5`cFok+SaLg_;Sc8y1`)_+%u1k0#wlP`#ssk+eoc6qtH( zvkcv4K^|YIDT*bRg8Eg^>m)0!mx}cIvZNV^4u%OPK)Q6nG3}HjyEYNj?W!4a&!l^a zT~xNv7__|tvI4R|%9AIe+GGas@s%f0JB_MMrr~XAG{NtQIm__(TQ@$T57clS zoQnASv6XC3nzPoVKLb*Qes%`K$^92+q#|I2-0Jnd9KvFw;~K~Zi!i58 zFzkmU6DUf5qLNqwARNkK0s?aO>YxYsjt+uu(|PUoMh~?lJ6~T_UdnQRoO^WS+;~m* z%wMZ=AG+q7PM+S&PYUfw0%~%@*^Z<2(b8oId)5!7MB>ZO?S;v0Ac)Y}=LD>K$HCkg zer~^L)owHy;FE2^^a0929T+PIf7G&?+Xy__l*7GE<0$x!F#RV>fA8>5Kw9ovn1o zaKvSa_et!3+Oqg*5a*3X_z*?u_ky;gqt<5+5F|t1z`IDNGUycS)mf&dSws_uEhm17cL|l6YP0wN z3eAaZ>>v=xl4q=h8E+H0FHe!vW#a(&cvGB;pGc;3eTaF;0z&mV@U9moCg!EM*~4^h z98P$Yl!zaGf2Yz>9MR&}b6%e8AEfrHcxQ~g#OztEr=9`pucz0Z)24MxBBB|ydDzmK z&l&~E3h}KXlS&{PbnhZA)L7Gf7oHEyjr#OC?bK&qeSFsV5Q4mblOQGwd`{(2DdHBt zY0-p$>u^WAt$?}0D74navX2XjN-m$X>d2T+Crfr-^0C)-;ksq7n=GB;yKVH~@0ITJ zfpjHy9=iExt?p4ycgT7SWhe_HX3Fi!Yo}JAMAttp{iBWee;P=$JQ3nRsT(|c^s%|s z*5sU~?o_MaV!T>oD!Xb0ZFVglTk;41J#ug{h zT3;do_SEb+YyokU@;C~QhT|n1(v@}Y=p9b4>Gu{nq6S!<6M*`FY%uORj%KG1b|+zu zupN)L%e2@`OzZ16fR^NTUVX(R9dfKIGHHz{>-uccZDOb%%6o5o{~FGE7DG35IfJiM zVfl>P^%7-x$(GCDn)_1`O!>a9`UKVwgE&W9i5bC{EIOvzf_Pu_7M>=GHK_lI9Py@E z*kJSgK@Bq$d_c786F!+sq8W8Pei$_AIB$Z8U=o8AO`Y2m7!N<~jNxGF0H2%Fre4W? zJle-2I`5HFtO3d93-1;?Dp4|E0|WKZ3DIin|Mx&SE&~G<^_X=NbE-))9qPlp#$ihvUUr;+bRr5bNFC-RE-k~ zso;V0(v1kuwt=9mVO_(sBD|NM+HT^nBab&Hg!flVXh^x7KhwHgxgD;oFHPJhl(n3z zP~Fo_dq|{iT5k<(k%p0Ns3C$4q#j>Se-VN^J==oeP~ZeEzzLYj!j`_7&OX*jksTiM zab-JZz_Hs;j?+@RZnMsMR~xO_AIvc`G-I-Bnr^TLZA;Srz>>FWZr>aOGkw@K#t z-ppV4`wcZ|2vBZ@U ziYM2TELmkD@x3J_+Yu~=btP?#=1<}w$Wv@qE+t9H+=gCiE6?Ux-?0bC&tZ+?!nJ|i z`{=Z-CP-Lqb?sDrZj3jG*XonxyXSX^`1bEtIWA?HpNUcNrvX`O*?q~I(s`yV9Nb*g zz|_z#9d?B2!Ysk1jLEzF;Vpmto+!S^?fE&WJdH|?_3w93oPYkBJ+1|!9R#C7F$$_Cj*ulqeH|e(a8op3E<3JOlVjnQa~E?3}Q3L-ZJtfRR9heTpVZtPjFG6 z7j0V=gypPE*V@>huOs#jcj|A(qgWPJ2q&t#5}c0bMgNj+#?j4$T=IE9re=)Ts+AVGx>nvQy$jcS5Z~aM zQB{7&O$j%4spB{Lh=4T3=dyzdChz3-CjBRs19Az$*aG{HYKOA1MIwU|dHmo*Z+pO8 z7Lx))9`ABBpeoPhc%)-EQfF^e#j|g;gxPeB72?JmCmhHSliGoVVuNY2snMk-Xq_Y^ zZPX}Ac^1E|&pU@7&PC5_>7CQlSx}a;P^RklHA>l5 zZ|~3@jMD(Y`lyp$X!=#~7Ls?#X^$NTkR84WTxbj2+ecRZU>2N8Z?`=qQuI)7d#cjb z)|5pVD7mA}CUF5O*QI57mO*UwW~5%kp_m}0?j#W4kg_ZaS9*AM5egCvI^xe+L})8HnvEuP zbhQmGRe<;*m>*XA-d9;(^VWkYM$VfcJTwG6^9hSQr-3|@h6u=NW{1O( zl23ZZ4ui{@8>4z~m(~*jMv0N+S)e+sC`TiQ&gSR$Ow zrd6AHRaU_1<@OV4)dO$M@^5E8$vhx3{YYO@O=RFX$j&^tBDH$zW2M)x=;O;v%-+eB z|K!nJIQh|PD}9BdebKWRPJM1UelXF9#u_I$Uc8#WlxKlee~0&KCs?jh@r7S z&D(7<{tqSqYXYut!u<9MabNzk_8ej84#=QlugCQjw(T0P zclF)x__y!y`5Dqww)fBPN#H6JbH1Py^{-NYfNLPR#dwGWxOH#Rq*4G(HC>FmB>958 zB0+M>ibu2BF5q~3Tr8|U>Wzs1lSDcJ=WNudPkJG6j=dsK-+#Ag21gd;c2=yITDt%i2R$kj^mXOmZnSh~RJsY{k-y4_-v&~MR^7>YjT z_nmay z^OHM{#|B5fLQ1?Cri9hA$&(BzdA0~NB0$*x-~8qrD!@?Gx_i_A{lZT%3VJ^x&ST?& ziNtWadknQ)){S-C9>g7(YN_?uy{Wch1X&!9lGs|WuZq?fWhJETibAd$r)8qr`<@3& zVv!~UURVvHZ}PBgY)}!P)KI@NAHKt6;w{7GFA!Z}hoSxMVNhIj4sm8CT@Sv9yvvHz z;%Dw#DUS=vdx1ZZ{VOD7F;dbL*dIO}r-@1oL_w~jwq!X&^L;BB`ds4;iwN|-boE^e zwu|nZKrw0Ys_k!HF1Om(zN6>snnCpr(zT&n#trt-dL+)qsGe>7ExxV4C)w}sZz2Sh zl-5O)GKk8K%d>_etf2^l7%#I&yM7I=xc;lFshUc;V|+@LhJ5)G7!i^ZbiSYSwLar!BhYeM?q1NMJdI)yO9 z}Ud-*cjhoQ7pGly0R8G0~E_4EE`5H4Z!s60^<6J*ybW)c!K<9&}kdP^#scIDBS0) zUVK&9HOpzDEivdIc2@K>_YhD$P452w$rI`C3HA4JqV%oAi>zJ{azjmSNDIysB<=gr zwugS-t;9(8^DC_XIm!P7EyWXzw~o73~5gDLdEUrA6cW=?TiUC5|upUs($|pqk`(uTH^vbp-e2X#9Wau zhylWHK-cTJD1eM!VPwDXKE*WPrb`&{pDUjeKyC|)|E@V{ulS~B%F{6tUGw`aZuw#j{qSF7&DZtx zg5Da3gEV>DRS!X+A2ANjyYpG`Gp;KppJt4hNmak%_UMu;$azth?phb6hZMCF^3}%^ z6n~J_PwDeuhDvo62aHH1hKhJVNq8D$XD383w9_nNT-7K7rT?ydtWPhw%khKh)2wm z?nNL5J!;W26GLb^YI)0i(Ee;;>R($={@;=*L$^<(Q1L z+$YGiI8n*`5EXjZoMVv*xK*NOmcS~8y9i;UM~H6{`@1n0;jyt&S56PV1G=8U{Q!Ti zJ8=;>7Ssa_Of`9^;bMUoUy_oNasn~{R_9!Y3UHU`|Kl#9TEmqi6u!a_CccOAHIT(Q z{ia@J(fKtuSC-&a_E494N558+oTEp;j;;hGPJRu8!Q66k9LIf37oRIniGHrQB?0yQnTz zXE-5yZS|+#A%8D3yrD3~biBB~O;6PFiuADeyc=J6OYs&MK(j~g7VTnll%Gqpxe&Zz4L-30SjC<4QECGGcZNLX&#swz(u&8iazz5m<0fk)S zw}w<7Ngg+UmHAI9;OP!x>c47Pv5f%~dQ8I^07W-XDr+>8gF18^G7`=wX2&zEQE9D^ z?V~nO1F0Wh0tE8^XfBEGnF5l(tg8ac`7w#%Z6R`$CQtYLKeOS%*lkr@4JfL(f1sa` zCv_8-2j_HrMM0+cE(+}JY+6Y!u}crY(&Iu!RKW;yrW$*N=4z)^(tmbI!2(SE-uSpY z?eZHcl@aE6ep`HCFPipiUnvq{$Y{CSZQEVDa}5KOyGtX?$}T1%VP;Q%>vhEoPAl80 z?Fe@IyEXwzSgne)S-YufWaw z@gGHAE#jesJ*BECac}V*w&f;Nd*|s}tUbocdLlff>tgTf&enr2%=QD!XBz#^RQ(Ft zgC4i2ehx6hsSX&^Rd3UtWTsi+$9EV>*FP81=VdR-k)Gw)w;B|cn9)CY0XBm-f%cu( z(D028nEeG5UiC1z9o_|=yZvXBD<}$V57^hoj*O}Yf_wrNfsUtgQl*AjYJA==HEVvQ zmub#~JbVb5C~IhmKrj?b|G)XXE>i%S18b7)^e|~{zL~wgWP^5wv!jc9HeNZ_WYy~L z0VWzF%{=9DZ+y;~Lfhxc`3{k!-t{UuBXG$On~dhct*uUd_V}*Oi&g)Vt^LBg$cds2 zU;fY)om~L?I@#R?+eKLP_iRncB{S`KCkmgJDJXA ztnbg9WU<3o6mf40BuV%5x@X9g95Fs!W^|-&vN{7h8H2qiMrrzmb7L71=Gg_dXwFsK ziF88@1d612ZyJPD*UW(STSLiJ7auYOiZCfJb?t!yRP%~A^+G7ju<<1rAm_#66yvW0 z7QPmNXh`mVIHuV>BS28&BSJ~Ijb9eqS^6FhhpO#)e=!SfINT~D6dHqnKQ;k1F){gD z3bax-&h&snX^Ri_Lv+g1sT=hOdm|w;&=4A)yaElK!pjJIHWw#;w{v>bgPMJ1^1g@v zOa^y!f4vg`FCe0fSF&x0%KTc0{%uK=tKBjotBhM5QzI8|j1nfQA2J zsi}mf1#VO&>Tj+Xmb#xmb1$KujTD|Gf6H;|Gb%fK6YKVhI1!K4sQ+8**GoyKpE;Lf z#uCJ%SHklS{+UT4u0P60lcI2g5avwvXecc(-E`7m z&bw70#=(GK!OYFc&bw>S#&g4FboiW;Hk=N0q`EjsPSw!Nr8<_ajolf= zlCDV}eZc%=$L%GRNBn*8FNX;+s zF^@YybN4~;5564yFryYqAw(l6& zV-|u7KC_=x;*Xq7&O>@qcfj!?Am|!B(#=@2m`1BYKq@u@4PQNtnW2UXkq_Fxj4wC} z$1n6m*)}Pz=_rb(@f8_x1L2HWv|GC`QQ<=*sj29&RRU8E=&5q#{wW;aL^tFlEE|HQ zEAHV>*vY`~Hge5Ovcp0{jWw4Am1oOHSxmD5{lRF??Rq6vx-m@YfRvO?9VF!+Pt#Ec zQh(N{3s$>0;eKe@PI52)sKMhAq^XHLcYv%`baxDnStjiN?5elEu#&qVWOSrhs-V5;YXPPqaCd3naYKD$>()gLvu-o)H(x7D2&x9OsRHQ zcz{3#d-tr^_V8qc1%V8Zo7Nt$m{!N?&xBFz2!oh=t{_u15QRl$_*)K1u}Ue|JE{og z;3YzQ3)~(Ci;l#9(v;7SJb+Ypu>Ene(WSkf=1Yy42!#!de+KLw^43h;Y~Q-7(-%{jlA| z<(F$4x$)^c3W}7J;PCHd#W7=!XRyxaaO{+O=P=W)l%Pj#gv)NY?zHFA#ly#?GJ}Q_ z^IGR0#|#c9Vwg+_2LfyK3*_EAxW|4ZJzFoa+M?A0T$w&)!z6~SxWq(6mjuf|!D%PZ zAW~UD4VH$hsOqZP74de2{+RzpZ|VIpssn_8@TOv`{(Oy(NiR7!5ULN)M9CI0HE37s z+*>0@-ha>ufzzW!N6r;iHx#D&+SWWPE>6%P{9*l7gzaaI0u7qWpZWa~5mw=i<%mvg5AX~S zp*_EcRUpFc4uzKM56#YSI={B48_VFE0mZNjq7W2zP~!p?;KQ6d(HfOpg8BpAff23m zs^4{x)VjfXQ!W;F8Fx)HgJd6J;33ca(H{npIvc3nc^4FrfAGVivq+k9J9HsujT>vd zuVW#7b<@qDlwd7){yFPn8#_2Oo%`KOZ8I#g{BsjVzR~35)Tap)kIDc^g~&voYZtlI zL+ei`y#=PaQ9me=rXPb!AsXE;%-f*js@Fk4D)U_z*&hYnEDR-{#&XN3EXR!0)F9S< z$}9m^$+93lUb;)V1BGRpM4C9GuS8hA>qGfnhk684KO(S?(^?&fBy&9!ax8E;fAZok zUkJhcQfHjMzFn49D50{*DXJ5kU>$G{mO6-a;3%DJbIT;9Z}{_CjAxKd{vNn`=a&76 z99{b=v4^BokPB5UxPip}etNw_u8c$x?FuQ)P659fBNDE|gUKS-=Oa92a?J*S!F%LS z+>!j`DPDiS2cFrw^c*7rQrSYPw&^majSj>B9-MkMBHCgxm7cm|pT7*O&1yUmeDKZ) zs-WUp5mp)s7!jucd@q*034FG27o9MI|8(ArzAzj{*hDD=Rmr3$5Ul1bSwPyE)y)}bGzOap!lx^js!uNA;&T!~8W&V^>3kN62qwEvQ?b<$QD z)0`VL!*KBPLTmcys*Iu4^PS4{@__jepSOR6W%;|0S*ZIL6jA-MeQvFpCU!yHY#*~n zuNVLx^_PMb@U}7EdfQlv9>>tPDben^#U8RFTN%hyvvi%~BzXF7}Uh9O^5p1zxC z!|A4E5@(@|RPEhM&$@~|5r9*Hw1I7Rj}gBRm2LsW+6uR~sQvVw3E|zS_xw?*DE|5R z{)$s+)MrRdF`o}QiH$Sb~3$J^z+tWC`HsOq+!Dbd;9v18hM zTw7scRbqIJ6KXw~l4YUJ5j=ToSj#6IZ@0#=hgXHw-H4iN#trTzKNr{X9$rYz1J?YV z>d2AYnFgUiw#(I0m)(kI^{le+>|lZ|ycGhmj_5pbibwPSx%!JEzS1fDg#>iBr`g|- z-8%4Ym4egTU$0DSju)~QqW4eOE#4M&^&T_wyhu~StA5U%@(i)5=KzU^Zs|?m1zOEF zeR0qKr2qc2=tif>W&KAYhr~D(5uxYR1K8LWm_8ND>Ws77BbS3=eK;LJcY-Y!Dj#ZQ+VsjO zB^E6-e|;Laz>gbL=LbO%BLoYO&c)&vOh?oRzN^zM*`Y6^oOV+C;od{j zRYt_aCaP_Ps_K&KwB&a&QJQ=kazzT8GF9p}{JWk;5O$CkJHm3U4jYQ z`=AqSnSu5P$nc1liNiYGvk$OX3A`BM@G{MsnL);|OD<`il`YJwgNK9pyM2Lx2_PJV48o^h;hFyOV*|}2*d}evjZ)<$eKkj z)eO8Qh5P}m9`U1`cN=m=8#o-7ZuKh`TFAw3T371_QZxB*w^(@QKh+c(rGRAV?rdMF z-QiMsxguap%8+by6=XCrdHx9@OkzhEu* zhl$Xw-%j_e`hLe+6H@at1|%AjGE4L zK$c>E3f8AH%&;epRjoRmaf}Iyl~K&|7Fx;V>ee++w96tY?{@Hc(`w^qBplDf~e49O%mMFX;JS2aleiFpG!_xb$~NRz8Mo z>pbz#hSUyLNWtBM9WL@dRJ0oyF_cpyj5vEJBEJ`C5e{wcPDdG-g;usQj|If1Qx?an#BaX+hv~iL!~Xsf zdgHrYqY#jP4rIpKM=jCdt1!84v_oxF;|+ZEtRl@*2qfKyH7DA=upoYwiuWwX(fGCQ zfo1Vkij$z;WgzK&O#dztj?m@8&>XI~Hm{;rs*N}Z%9SP~e3cFiZI)P-yiGCw|Gv(D zuUYUZ&^1BTOfJ9pJR5}bxx|fhC)bWzj?RQk*1#bEO5=}WwNWU}=im?xg6`5F$nYU( zRK)%?S3SgTMdB2S$O$_IB~ZE^1@}}W#1nk3x>X7HiPs%CI0?n!dLiub?|t9XA@(W! zNrVHLSO9nj#Ns`jYP;rjvS(n<;ip##Vn9&B(?DJj`XccgO;ku_d;Z#sa{jNWLIFj%=|dP@W^p2U87co@Kid49Q+7Wq~X;i7)ee#MMLG> zZK+kcakumD(D?hf>8pD+r11TSa)PY^T6xJ=5#)RvYPfrfUR48OQ1J&{hal(!e?9It zLh>T zB1BqS6mhO`Ucq(F#6UX*W7l^mwoIWRHVrsWm!%^+UfkZkmpmTGe=%HtlV2XYl(&2l z7jLY*)`QG0Ew<1)J(>_oP^oHXA&LHYhqYgeW^}g~Whz7;ExRkcwv$q5Y#M~qa2QT) zc*Td%1Ti0+-b1hhH_j{zR-IfN*WjQl#siPxUMhnmJFGm16M>c>fyLsMO2`R>(jv8{ zf}W$zkUIU$ZUv7eB7_z^0EDwyN)3q_n?C}fB@EPC^a1KC9oc+IS6s_f{8XCW zIkj6SK}GZemG7TAlMY#4z6WeUT2l}K_&4zsuG|ZksV%vlnFThDSKyYM_6kN_*&o7jQHw1`=S|9A^3 z6!3s___8|r)MQ6zTxl}N!@}z4)+L)z)k^IB?uHF*JtPEt#O|{OWmNRI86Kv9Ly>}pEed9S@g}=3Z{j{6U9ww<5SFuP9qA93q#I&S<5}v8ZX4nguW*Q4 z-in?xT##vdu<65pt0-`L(0}*uh@O2^F*hbP-lR8`MPxg8z`$;@0_#EuOQ4iTlmjkA9%pQXc!*Q-A9 zn)rg^yA%Z%O@~_gn-JB`c$#!u0TPbWmf9cJKAm`ZqJR>O)s|oZC!cc?eRi#j`hhh7 z5G3MvA2S|JN;^GXZ|* zx@EKfe{fk$7$ksvxy*cA6lmF^H6)X5eRXa-QDekHLrwj0+MAH!+%tgi0yPGRLuQ!< zf@`EYmW2g(eBj5XNML9F$y2sU_}3OB_z2^+M`J4%vC)qL>?Gj*l;|4kt{_4-j^3Bw_Wo*(ekeo;6)>vj{TR);mC|pnE*?J}U z7a1I~$+ML7?fT%u`A$8HYb}3`kp}cZtr58`rH>%IeU0AnmyGJ)TYCL`_n<_3)rU^N z7#-*?p0>ECKm93QzztCG4tfFRFYp0Dck+CTzmX%*48O9PIy206V*)05(QCgc z>9D_?xaCx?P-Z&5+8RJSlxu^%-i@^AJL@VfFE2kr543*jhtVeRyM+P{3Da)uAEMvK z#S^}gDSr6WAsR^DRbtSqb?((W?L+SH;u~z0Oda~bmRJxb{^%8V&u-lHjaCS5>*~iZ z=F}H5Yd50{&*f7kBQjJAC-p~J0g6>xkM+T88elHE2!J4!9b_OZ)qBM`250_I!uvRX z->7f6AJY~FsVOrJMzvl#IURRFjQfjFkDOBT<77NMQ4Oa)7b@|Ty#zd?zQ&`OW-8ki z#hXD(A6Dm{sTDd_;&a0HH^++yPbC&IRmEUcrto) zK<8@_j&G)mr;Sph3wh`VyXHY=hc#A{826-Q|0RiK@NY?C;5Mb+z>X5=^FA|t8X z`lT3{XO;-&j2R${ggBLm-q!tl0N?RP4)$lADZl}2qgzin%bqMV1t$OxRsm0zo0scX zUdHYnJa9bR6QdIgB(dYJv{^~B+ajl>E0-o<()vy*;y(t9GQV|>^5Eb0KdiDlBFa^- zj(6kaH0`iHqbo)p&CvoEvFBt{o77Qy2)=l<8{y`@b2x426S@;l!wx`}*x6QAR_4#X zY+t)v7+qCincMypnAUy!{^@Ol%kod0?lIZD13@(2siXv&YaAqb=L|n-&gOJ&t`Ui%C$u*F`x8v zjVW))lY)JT;`?Z7P|y?jn4No=7v_Z=0|=P=7q$|<8}|WFAmwFz*lceE zPF3Zm14lZawQw5CRX>|nywN&8Ob!`3_V8KV=052YiqABcaJxC;n8qUo-_>@A1Eh_g zI>Uh{Xx7$*qvBwECIV<@q`HHV3~tx_Df-uSc=zfQ9S^R;$uktj4IPV`!w!F5kjn>T zm9BP(B_s`|%S_tZwH?Q8s`JQysC>FUy&yuwy~>!SQFhN|Kk+k$d8$GDYQ|Cmr>iHK z=~2P3eJ2lkupmV&hR*oC*}!G+)RYXI$uv4bWV>f_)2A6Gv;7v=E3iG3)c4+b)XDZW1EczS~^ruVWB_a1IqvQ9ZE2R|iiJX0GZ#VIkp7`Vhm{ck=6rpPMz{!Bq zcJX`rI+mM>LjVy+(w0-%Knmtf+scUb`CcS>P8C2@%2Dh55NNQUA%e8+M5xoWmUnIQ zq(09-b_=*x+k4&}&*7gnt*pEeTf?F=)ZTwMIFzuqgmhI}W3$q;$8)!8ut6xl|3*40 zgYhm6st-14qK}$Dj>|0nz9EnFNn1kwC_h7lZ`xY1+elqx>4Qsj|^?z0Bhcdr;UuCOiu5LwAOPZ-82c&bOMYrLUf@1N!I^G`_@p)GIgAH zL@`4t`>Tr=+N%h*VOETP?$C%U7hJviDECyQ-M|dn0e0d~S*`U$9$)+{0 z$2LvT>dpaPo~(00KW6duHSjrr=l_<^Iw7`2eBn)v!sz`&JLyJ2rW_U3bu`2P3xE$; zJkd{k)SI_XYpt_?oWjRlRN^T4UzSF?WRxd;{rr;Nc-=E<)B_v&;?S(ld-)$Ovd9j=<8JzXhWQva zhR(iV=+%u_!kWtUH!-48O`tceDeUhueRo|MYPT~beWe{TJJ0u=(5aTXF_imgz0czc z{m>84Ej%5c*T{4oi>=N0Cu4 z5)RE%Lj|%z>?M7N=1#QjA8B&iUx+93EyokgJB3r*O;@4?wlE`K-;c1lZ}(fSaq4?( z&K!orSI<1JLh&4Z!XPa`D@aF?ks6>&h1qc-jj&eIbKMxpNV;6qR1p=jtvD|GDzieP zGUc(4dL>cgrn7aQs4YV{w6!d%(rP!2Zf34ZVj?Rz?GB&D2@sewY&dwKMgXw0-|gbN z)etjoFDYxkIVM5wyzikB2Pdb|mtNqjG411a?I3ebS9vN=_XFTNNMex=4el^L;wu7@ zQX@J)LCe<*ueqPodg0trojx8eY7JX}O9U#&q}8c>(=Gr(1icvHH1$ zXCtsrQ1`QhqSf%bua6pk&rRMmCS6Cuyo}LV{7BE8WvVp6o@DYlAjGa_ zc)Z$)NjXPJDDxa(D~)<`xLCK^5=!T+q=xCK!5WJTcS=y+0?VBSM#gQ(HUu~qxU6*G zK3@ZjT}pLtxKd>-CMqNj;)?K{GY3*%(5Cqv7eP*CXuirl#08Q{bOVa51c3*EScF_z zOXK1wK`d`xzq0MV{aEo0Uy%cm|I{65810iY2`)uc0jGwX!&!<5L979?#sL3n89=2tv|w9jt5G3FUriEy2F0?)={}$KPopHdC|9vJyYyj->FEP;Cz}H0{ea!=qv{)HPxHx z5HeG31MaE%j%A>Zf)J?LdEJbU)IxAnh4SG2C7>t$VENJl2tX_{JmAAzBH`xeQ7{w2~|F3#HAU4IV|?O$tUtAAhT6iH^7&;~+)JLhgza5KLIwQILiZjlSHevEO`S}F$h7TOazb7g zmrA#mbAhq&x$?`3ObsM-tfW-9ny1;85oL>G9(6tH`>$yvpS9r$lthU*BOL!EI39N% zRoF2G#yiYI9OpCJGMjNpACSxcHg--_8zcvvHZDdp7oRE9l3rh9}bqKZ<{u_?2os}fWtrLVPQSOAO$*H{)$Db0vCf01)9NUfbb)1J=@xO~fQ6oHc+W9v0F7MFP{hl}QrBfEXWrtTyxRE*rXn zr;>u{{ZSs!U`)q|%y{qJ1vB9nYBTZmMp+c-8-!RTG*CKpfFdCi#b05sL#aPH`-%kl zgq(}o{-_fI;RHtvML?f`W#D=s8r2kmodWS$e!apJ+L}EqFc%ybf)h-Q+gmD{WcMzwlgRsMLRz9aE0j>n#z5pks+Zwn0)xlBxn05wVWX#YuB1s?zQpDLe*j6Ji!ddKb8m1aCx#>V)qUXbHs zjpg+lf6s2(G_?oPBm(_QAe+f+bJcqJ7`l~; zveMkZg|b*MoFYh%5NJX66$LPa&hgykNNPNeV;<>$-CyE7eowe@p~WAr(6$nyFS-QLG|u*{0czQBM{5~PeDEC$ z{qwp--vvRX;2}iNmHgE_@#Q;O$ejqPZEdvKX5tsxx zDw|qJ6_doRm(oIFt&`G&67ySxer2OBb;oOJPNFa25!j*rOTzG8h<9N-sNWidBWTuW z9lxZAkI)vy*;H#Pau+$Lp|sW4;3zzN53phW)W;I9)IDR1RX^vF$o3@gKtAV=(H3(P z6rs#Vvou$>YL=#b9b3q@88LofA*CPkI;1ev1_C_X>2S;7R1vvjof(G28V3>;&u&OjJENXu z7+|zMY}hp2HtCP`o7IZF+s5pX*x8E#vG!TXy7%VI&qtd z$5BU&X4v|Ld}&{YO#nW<{j@}H&z>1k30q4qKUQ78n2#+jp;?h9fQJAK7ia~?!q|xZ zW;Rp=IU}fWQxX6m*SvFKzmu^x z6re}xE#h#KP8Xa_&c<>|#!)dwtZMaLJ0)~=}x5? zx*L%Y5s+58yBR`CIt6Kv?(SCU?heU;p_>`vyV?7F_I5wN{l3R{9P`J02srCL*R`%V zuUHWKhvq~Y4If4n#wy;Tn$0V2F_KgV#;k@d&^N`&fN~Bj3 z1kJsUn9ozU{J!n8JPUD8RT8|Z7lfZW9DjFU4Qa&~t!DPNj_!x+Q7(I)g-1_RzIzOe z>W^d}+O+)0e7BcvvvKVJn-k79oh{5M)~eH;D&@OfCaWrSBk_vyAxCp{@?L2SDBXI#qLne4i5x+hOnrPobpK~x=VdRxiH&PQ3i)|N>L&~ye>v8 zMyshCXAPV0yqkGV6!gfty6|O1BxVr}n^7?RP6g48bZa+29`pnJ-wfPK6dz5BWkb)t zUC0rkM?FA~@{vG1i11b_ROe^cN>{klCL=$SJojqR#H`dJ;yGug{#y5O_K}%NTY+kk zvJO*1Ns@#f_{hmQRS%zq!#HS%yfW8J`HkOjL4ny=wV1>9@&(RXm1;rE@mLXSQfJt( zw+=YR^KcvWct~q4=avyOM?T29i+nH~n#!bAXZ^*jDmP>(BP6iXPl2~PjGj2JdA{0; z`mOP6?xAFU`pMYGu+!M_Ij88fMfcuBlT^l*3jg4wh+jnpchuOJ#4LkEHBq5(9V%ue zD`q{f+b^ECgOYqr7qfg=Zm;$>N!&h->LqMk=>aar_87`St5eWM0-ggQ0`sRuRX zekYW0k47W)ngKoIAD6YelR)aG%|wbC(wNXsh82J<9_e6gtKWhk zj!&zf+Z*H#m%Moo|B}2IFp!rikhx|5;&XH9$ZLqOtO>__qTBA<=mKyRXVq(BLze0n zPvUCH;6%k11aa2;J1+W)zJ({kh}I zdwP1}x=hO)6UQpU%3s(d=!ViOOr8lw# zYA{0!745o{Mv+5cV$;PdUL=BW4-Xn2m916_r9V4LmH4t48Moeg-u3bz)7i6*qMJVOkLN+aEVc|#9B9hVqlJ9~-K0+?V`<$y z+rW|_3VURfC1K0Y%UY2TJ;!*68ZqcRUe1mK!_Eee5Dt&l+QN+D!_Qo;QT;W@#~Ck7Q!ErCQ`Pw# znvaJJIMTT6U31y7b9}b+UE)q zjSOxMB_oTrV#%3Dhe7nOc``>tqGGpsj0sZ*ZA<{XS!K1@cy`#`^9|O&5i+m4u6i{Vl3h_ZkyIV*1Rm=HrapvqH?X zKzl8T`J&p@LEQs_GNS?{VUX<-IsfCaAQ2*sTcmH%fGPWbKI@$Kw)y%=7yJ$6F@yuc zwob2wW#ZTyB672?`nC0ItNP2ob_R(TO&c zw1rqMzt?e{K+r@px*QU?{5e}6Ri*QUptoutSRY&9g;=Ro;l@~mC}~R zqnnK^g}Y70FPAy!tY#6YqK(ZS{qe4{nuv^4Mt%YpTl*r0F421(y$kfd<>#Or3N7?uLEZl2h#=$i&Rf}&Q)j8lRT}(`{2fcNM+I*u) zDv8`ApClK~=P*(C&7%e{VwaK_s)wP{QyIe+CdNDI`K})D{pB3VUt_K3=e9e(B)1(N zO9Bqca@WY1DC_EQp`ta+Lq1vpG?<0*w3rq4Ml*+f!E-A4gzk|$KK}Pe8h0wAIUwD@ zeSfl`RiNQ@g=nT3yYI#4>;>JR)1li3lKsg(Zi>2}zYTAwG@~9b4bd(S9M3o4M~E1c zH{VVyw|cuxS*yL>6smeFf>Diy~;-N9tO*Cc{V4~nTXe?<)mP7Mb{vCxlV?qKO)l=z&= z6%Jib5@zat6&Bw?2cs@xZI2N?%c#nz%e;v)BU%y09a|X_QYL^1$+kI{ojIOo3}H;@ z&rrygEwE-}MNM-*z%2d5wNi}(LrX#{2{^@4CpUl0STq*}yLuq$Ag#|DVn$(uoQ=)n zICVuOOKzF+Xc5ur)2U#}BcX>IMqUEb-;e0aS{r)F!(X`zZP#4jW^Yy9=J?#j7$zSt zp;-BzE@iBJyf-&c=(Uz(KF_HQ-#^!-WaBce>O zYBM%o+yYrVbZ2~5odOD;r&$SUAE-(1*&%(LV^%XX_RZZz-HPT_&crG8@wxcT$*b(; zg1Rf=&-KI|NlNiMDG6)wjES&2sw5=DM8?VoznUFNq&%itUNLMz5#Jz;@|k z9uWzbVllOU9ACn)S(qwWz~y|))+?!8tMsEorBaqquL;cH)YjZ&s+tj*cr34vcoE|( znrR4;LL#MNfK0EsxE9Cqhi=C+EX!Gm78;Qo ziQMtrpFZL~Q!*(rKV&EJH4sX(AMu$*mPMLP5TBOlI7h`93gi-%c^uhJ%}}~lvenfj z_RhQAOC2&@e7Y!h<=k|S^BHh8T6Ffd1T|(H)~8qE31U(i%FYi6%f#JVp;tB_*7-4K z1Gm(>h_zItGrvF+baI4Tu=f*plraz7|HZ67?K$W$!2BHcNjFea!ih5Vtf=ZKf#{U} zDBEt4_$+o$JCb@gHwvwYzow{DhQbJmERt-dr67mumkhP2p?w`BW{p zkl7VAA+Zq6GTvVRT6N9jNfeS2}1R!#&X(!|2_FYYj4ofjXk#qWoDA_*#w{=idw-zui=2x&U&J`lcCqY_`)z3`W&58HExOByN!5KElK>(`@;o-fBPgFBi&(6%C`| zRx$Cg!=&{vK=BX!sh(&Y&A-~7&2*uve?>0W&JKJBY1T(E09+c1awPC`wC!R8%qL2| z=VCCG=C(|ij2D@jBR@v08RFb#)$$GgG?uayMRo3 zo#J0#A38L+kF>2fCZoj? zt*-4il+Xj>W1oRLyUYnIWtm%z8fku)1BR?LH_1l?EF{W;oSbjxCJtz}x6Kf?H;2GK z4&wXE8YULbPC3aezw;Rj&bmJ0?wB05@3ETkc zEtDm5MA5!tCzyZo^>-1GlV9mqYKvE@YTZeXkB>-9-mV2No(;1Q+TZBd+m|FA@BggJ zwx8^iZV+m^=qnc>i-BhnbPJnkO(%U#OJk{9wyleM`A#L5^=sGz#}9n&>-h_ACtzvO zt3>rQ;F{OCT_*VXegeQN_N+RcCdgaMG;%voeRRuSR{Her^mE$JAyur6a{G7uJ85U6f8PG zZz$ol4semN8X1#@+jtA4zBp!eX)9nk72T+>m2fhp=L6)FKf zTy1-)1qhgQ#{9nbG*37Yr9$z)_@N@*^-R`;iMV-xgTP|UVOKXUoY`Bm zLY6oDYsKS6J9Xno$IdrZT1=`h$)n;;1r%rMw`0?1P0KHnLQ&<)6k9gFZ0W{AS=z)J zYXv1*;hM{L9t&P+b)x*BJvyS7=-jEjOWDTqS9n`dsX89Bm>fr0th<$8r4i93+Iw$& zFNW5B5MFz1xJH~Xi!2K)lU}siDK)Qkd8{|k>;42+C0V5am=Px zCwvC-q=l)e_lX(%6H3{}Tn^@#gRsftHJ!{K#&NosPnH^8`xaYJBnoOhaDyY?4Oe25V{RB6;&f#_G<`f)zqJu8bIZ&s0ylSd(W&4}<_EvG~h%|`$B32r7D;DUkA!qgM>v$a6XMydzF)wD#P8O^-tGD~lK7!N!2Zmj>OO zcN=Vc*G=e~-8utGJj~@Y#RJx>ty7;MCngxQSYnci9Is1o9FBE-W7Vu~WwYyuY?Iw< z%zM72-Y!-zfru{IzWs-&_!0d_RF(ul`2s*2*wuLuYYxCA&ZdqoCtwRXw~=)CP3Mt& zD5iOSsE(d4F;RCrO1yi{nlgET^0js)PfwD(?_Hs`obm8i>P8y>$x~f{%UWo$T3M=37Fh5x81mLJrdXoJUB6 zH)Mq#0@R8ys<#&JF0^tskz!&&$zfntKE2%A(~|3E`-tX?D*@?|H+toDWiz_)GrY><_@U%Mczb- zQF9o36CyX)$TwEo?C4O9H~*Oh5H0!*2W{)N-DB(M4c2Xoy~(9!l%7WO@R$=2jATbgt!8nNsvRy!^G!Js4JEBo)$e8)?!2O@l9 z&9BiC!V|iuYE#OOU|;8!l9s1mmCVs^%~fyOwC@@^00SkT^~uK?E39|(iS#FKrF@u} zEYnJ6RTC`Hl5yM3q+hRXvf)0yorBfwUU~#^*-Uaxud&4tBi#=owA zVnIUgF~UvYyD53qC0#!)Te|W!@c@m@HRT{Mg_Q0$Kn|%9Fjp4s_mms%qCG0qtX4vg z#jms~RHW9h4Z)d8qaURl@W$IKyc8y{fAGloX80%3r!#bp3^jtFIim%be)f@jwdV62 zn^3Bz4+c!8U?Bk7dhZYbLa}}DUQ5h`n+xuzx-U=IOy=8H ztq?-2_^YEuc&ylUv(+2SjS~HAOd&tG$4|9LtMY~ivB-uK&$6^RFPS*!>(Z@DpDsHG zMsx-M{)S!0kYCJS9xU+CZik49MG}{ciWU8sg%v%2eldEW3ep>u}c+#ZQvlkBiT^4tL?!PWDSln>CtZifN6jnnl zH7#5lk=%N{J!N6Tv$LI)0y3ZKxizmrd3|az1N)XI3?BaF-rX=}l$(xT-*TUwXnGUg z-ExB5Orb{ZZ>`^{yP!+$wT}x96+!?KsUDkxRR3UF4RJ?ksjLavBzJg<=_L-DKmv=sfkTC3 zIcT|F@D{5hvt@8@sNVC{-fEQ5ulxR$vv@ps5uGG8w%mi3cpn3M-H5$`#&&iJoo>CHmy zap=<=zg@d5z-t$0fe-{DABD2)t`l)x>dLsPILertps3WRb@ojdVbS!2V+@P5DpPocw|=| zci$0Q*@XPg2ef`> zWFkqvnCQB3bd%>h(=55VrJQb05m-#__YO^!&Fm1bk=HxaXXoY;ZNp^NTIw)kr}nhu zLZ=3+`fqE_SQ{!2K*wh$v4#g};Mwg)<6~UvTjs}JFj^0th1~&OOTaZHnZD_hHW?AR zoVnrH7qO8zKrXf@?g+Pl>Cq!u*Uz07%`R3rb`~F)gra#!;`UBm$F;-`n+u-ZMlK_Aojs7kEdcLy zHBR$V>rzo_Tp0JT0W+jlblH_}*Kt+vuLj6x>*R5F%yy}$6!1z54i18Fl)T+6?{Lvt zz9!y*^Md|i)mL61;tF5BmKJ8?pUOFpB%v)f_X1za_*mWMXCKA6LYa%;q zZhtZ>u^7vYyS&DXhH=3~mY3(Znkq=JUX@L}Py1S{M< zftoJ@*3B~RdbYJztte``-bV^otLjnyz%E(WlN}<6-RbqmKJD znb!r2IElKAbie36`m2V{NQV%xrcvveZ~#@NudSP5+-^?Sp?t>%(JPxo!2x=L(yKVK za@y0|eE(1{t}^6LvKDB7zxYw<-rhJpNX3(p?C0g}l)&Oov?41{ZA0tnUTWC1ebEgclKL zkyg_HA6Y)`9Wz0i`=L3a6t*pLb?YO!KZEm}BXjx?F^1!N(rj7plLFPTD?Wn~;_$uZ zee7nJz}x$PTXJnFF)f;I4PVf--cAAfR3%%#!d9E3Z->7VI#t}wV|`B4M!xr{^#f0m ze1Jp{R&Qlzf6lw6c-I15SG`Me0q^InA7UugtDn=+hQ{@7FdPl@$2ICL>2NWPL=Oy1 zryq!11-47>)AAz>2`ZjG&A$6xl&=Cs~RyO<{C@1^wyFwlMyp+TjFdk7<7$k!U8vSJJgj z%Oze~DnFMU;>S25jNum6z=+Mwdh)Ms*BCTWF^RLURa6jhyfy9`@EA3?D8dD3RsuzH z4UN^4kKx{6Z;@W-sSRo4%?kL|g@tg`C&pW>Ss`K~tOPkelWAo#ya^y7Xje7FS5rsxPIs|VKM64>#mx}*@;CC^*e_pvK16zLe zsQo*|`ahT#pxMrbWD_Ym1BB;aBx*azH*P?+c+$nwPKwL_ogg-9ZoPHtvl9VdJjSrt zB8s6rt`v1Xf{@pA!Vn6W=If?7wCN|F3LR}e58Z#cm~&vtslva7R-=@-68mC~C@%aDMsJiZDlvwGSge$@xSMdqx?c4Kxbn~JhfizVUhqK~XlH5ZjFwR| zzrM&BCueMJ9Oad?apQvh+_*|(ep>>zu&ra6pa^v8K{aC{2M>j>fSj05zZy*^TP5y) zqxkzat}x#ty3srJH&TB=_SXZz^#D?g_oL?z_VjnZ^!I0M9SwZ{t=8`rq_NI?h)P zx55;#v`xCZOr31I_x)=ue;uLz*FQi-o+{H~4fWQc0;pbl<+)@kRBW<1smBDpzkUR@ zn6bSoaqXa6;KuP)kd_Vv+#u;-{$yI;b!y&~D(WNMunHBH{^eD1qgjp1J zQkhtV?^I;(BFTi9sL9+V=eTU>r*$}GZx8n2IqW+N;O>@6n}gP}HhWDP%wg}{v>ecUVYpq`sgf=Gl+S+sGX@cxhPC*EtMEGA z1c07hk#Ma%m{4M!sW+oevlK)~$qwfgvWcmN@5Toq^PbE|e6tej z2Sg!)Osn<}D*nHZ(F&z+U^r3Vh(ORxtwyc(d$Z}%l9xKo6Wk&(Kb1sasw|D`RE%oH z8ls%E^6_!41iBPStE>Sgf9(omgp{SLvPSJP5c?jma|4%B!lmwlF{6okf?U68Cys%eO7q()|0;9p32py$_1Nf8!4D}!=(^oD zQP?Vw%Vz0yQk^AP)Ry5^y7%qV)Edn?U7h(=C+HLiF|-k-lEvtk`pG8+TVgLGhv~hG zp0ERx9`!r8I1!JTB}OnxBPhf3g7iYRM3a6VYrI5*$K$9>-f$<3QA52c{*AOA#pTKk zwp?ArqR~!q0YZNc7 zSr@(;#vri}BfZE=b5n6Y-K7Ln0Zqm}y+&#F$D9xZUjychEv>;=^FGuAuP4fB0yI&d z0clnI`6g@lUspSe?5MVVw|L`iKi%RY2eivojytWT2ygX--nn(073*V0sfqir#hEEr z0xfl|m}DLubgIN&h9~O=N<>w|MXAbz&OIRs4A#lF2lQ$2lr9T&Mt$)WgKai5Eyt5v zdcQh@p_M1zI@KDA&;>@1<;KzNYDJE_t%p-I*0G}E_tZFLP`KjXdAZ+A$5-Sf_yPQQ zc*uJ_Fc~SKnC*o6YT|t-Az?#VLNTkxxV;w_5crz^h_u9K)uq}x6&Bdd)Ht8#7+?_= zGHfBDpNaww;^%nI5-#Ix=-4yo+I4w%EbG`kSzDkGgh^RsH71!xfoo^fBv-IR(Du5A`i`Z2XhV#lKC7|n;_m*I? z&bZDli}2hrqMnnZH3&Q98xWphk^JQE&huybm_1#fB?DY`M7yP9TUxPZ%bTdKB0DcH?^}Y;%r`Q;y>4y=_Iyk|h6E57GX~OrufJHMFwG@H93n8)$>%J8qH9 z*KVjJ5!F9B{PjbnSfheO!F)v{!6{$0K_xA*vm4lK0jexc}VkWtmF z24Ea$&}6ajgOLpx)rR*W?2m=0n|cT}P_M$(m!m#h2vI}|pMO3&q!JR+aYPSWsBT#I$U!(T@#dJES0B5! zcSu;*#{JCQCkx-%r=EF^_0k73sck+arnhqK=#r=LDq&Z38PE`&>lf7Cxo}o<*~tP*5niug zDboTJ3_=!NFU=T|89&mgse)f02hI2#i_}*#hSjGnlLi-bCIR&BG(o&T7HqRYn9d{?h=4vgVjaFq$ ze>f$;st+3xG}2*F7?;yE8bvTsZe4?$P5H8zWiPGxRO3a+=!xT-RFCtkZ=4|t<94lg zOCLBkni!9J@}C-49?dpOIV5s#ct|w4?BO#PjX6aER_Eot0*sF~6!|Y_HYQk<&{3LT zci?7Ep?AXIoOT(<)#chlrDZI57H0W<>-aLCzVZS(X&yCbyf9O@J}Y>u>y})Txr)a% z%vDRxWQXO2Y*+=`+6EGZ=ra^wI)CT0`S=SvGI|F2l05n+nTWg6KYdQ%S`4ROV?4_0 zkTSN#@rs+?*;jvI{S)qtKJw|Au~?h@XJ(%KWf%*YEaUen*J?Lx%xVqd6>H-X{J_u3u6SBk}6zEkdMyNdw0x}*2o1(hGK>=O@rCEaYU z;T$fv*eWO4NwroQxa8^F${ZZyMzRU)DnN0aMf^c6Hp$|P;o(NmNZCJl3g70ousxE7AmpCKhdS=Lsyt&@;x0 zz>Ia5caT;7@6y8D&M+i9DOx}%X?D80zN2gC0Wg7-pgfI%Z$ zHhNb{r%y7W=Bqye6N~m?Xx54yB#S)=c~z!;(A#)*i#hV2?qmh^Au^T|el-#ianp}u z*{hWH2d8%rO?B{t^s0no_xz^SE5jLpDDf(+9T*C;!7#N{s#ap&lLqZS8LLoq+2gN~ zi2s0sewhkbx^p<%sK{?|i?2SS2(_4B<7lmk?E@WQU=&yeEt#)ID!p#Z=mA55-^Vu5 z+=Ld}Df}{4lN1fW=WfqKb|kz{CMNP5Y06@glM1Ws@K{nPB&QNpVsOf~AKbg=2h@^x zzaOzr=&I7<9Vc9q1EnKVD*e4cb$r6t8ADN3x0h}@&yQqtAye&g3SCwo;)lPyPxQ{VdAAj7HfS!9T|}sss>&_7T`*Jkq}a5VXUWL`wkMld z#rgB!F+oh4=*PBHhV7tk9@2HU)#S_!>U*6Pm4eLgviDMiCFKSAN{(B)=V`FzVxt0Z z+qVA8w?~(6BjupnC9zd&yYE`50EfDj$l5vnKmRF3`V)pQubm5O7cnR$eXfoBD8IK@ za*FC<{cGB4b!-^1=3^uA%mFiHWq7vL&PyZkubDRtN%)H7#a{J^f{10vrW{*@9 zHI0R`wSQE^zR}Q=ZzG#Se5mz?hAS{av?!CSr$A~|oj2@^_t#ggwgHPRB@#=EDTE)t zo<>A=7!4XUwOFU7$aHjvO?&Yg*qn_Tt~+}Uz^n?N_p#34Zwzd&3l%1D>d66smQ@zQt1y1YTqeDx*V? zOq-;(KF(J^nsI$LhrKQC|F){Ott*ei-h3wyrE+gTf&9}vx=Hh?E5ERLo4(P4Xy5;+F6BMJV=|Ic)%)fKmf{!w#Swu|U$$?8f4cCWpkxdA zltAcrih<$?T|0o~8qw^qkU5)miRrNp)|t;zz^6TT8q=s(eo;iGqZ9#CO;}^+T)fYA zo~;tKPo6t)V&Z+NU{uC417$`4Sr~kAW=iIws>dQhTq8vt&&RpgTo@g7)xNWNrhSO6 z&5B0^BiHy;E4`zzsv^rqn4YG{%k@&jcr&9wUaSxxpzAJ}ieB8YsGc!T>41T6FwbSB z?$>zdS18ukz;*f{d#hbXxjYfP9T;Ylsg#LJS>i%8flP$3y^J+1Ml{fj!i2ibcZiXV zjXv~A{5_4A)S4jsjpJIv)D1y)#~3II<=xrb=!yF9Xefn{_m7G!22VmmlwV)U+`KIG z=ggzE>lBBnp+{;TwkYtfMY~^S-LRrOAv`Ru@A&#C4Y!N8*tOx&v zWx~^cU|E4a5eV%jkzr%$|J5ePVtUt0yW8SuxHLXXgED1HZ%Oa*%UUW zkFf@>{jAz@m$L>q=k&P&n6+Zo(>@W4&9+ zxrb3{1Zo101&Ah+c_WQ8wMM>5Z~oe5bGF zVE?HwGjzAX?W9)NkCIi3BzY3;;T?b1eMnCQ8 z8Hm!W8(! z7+qhuwQ+4`tHTKtwbye(=frS~`;S`lL8hWQL^QW3KPz}rLLvr>&k!uh5<>^(-QxoV zd!{ z0=-dQr3s}vFCo9iqRcmQpToELa@R6C49W&Wq1NQh%-+;|?cf-p%uS!K3=H3(C`28vah0>{H1|HHm~%US$uz{)2-4iHvWP_{@$t zc&y!Kev>Ao2*GRB%T`YdapWEvIg-o_jOrsasyEKAN*(Tno4dU;*|H0sMBz-OKHY z;`^tU>$&RFR@Pf@^8KJKq9(}~KCR_=W9&BQgZ`%oleYP6agP$GaEY}tJB>g6P~w?Z z&tAi21jMNG*lW{PV>iU8RHXLf$WgQOmwAWHBRi6X5Fei*LEUPpbOQsSLfl6A4#WT| z)4{uU1h)lkxtu`xJPSJeiED3dyN__7h2I_H_zzojnu~V?EST|sXH9&6QjhTVZ>0aQ zZF!^i{)Wj-H`j=Z7tdlo0Lk2kRa-0Q5m7r4b}Ck4)sqTBhbkx_6da_}`f~^l^pGm_ zb8IoU@PgYn(*pUWv5Vfv42GM~ukZc3N6-5s?8r)qX02~H%2M4Fu~zK}PiEZ0E6BL( zpxkvQ3~~ML)@G6tdAi@J8-zwwO)N0#_FN&+X~>!78xoH8g)ntdBueLMof^Wf;Smu>W24}a zSxN^EiA|Dzv@hgVZO67R3~RO8xFE#{V}q1h5xRjel3}P?g5iiopSg*FOc^Uk2-Q(* zytreHvqA(fBmgj3x@_l3vFQgU70=(2`TcR8DTeMD$&&|%hkr+CPbhuA<$H4taGzRi zz@9nM=jw?}9|s%5NY&rHg$P4%Ws!RQGSV|$6O)hZS1q36%^@xR$~gRhYShdWW9jt_ z^{d8lRfI%y*}TolK#?+OkM{TVmzGeXj$CkPxJTxP_vrkwbxd3!3z0eU=!i&yu&j>W zvWqIyNc#Cdh2{_eNFjjP&>Ve#Q#lmnI8H@kITTQPga%N7m?|a&azS~qsrJO}jFQ>L zq+?mN_66<%*A3H6TgI;_lm!2m~{i9fftXy`Zv zQ)ngI7Nh@Xe_8VoZk<;NN|Q)t2fLvuSzjo0%Cdhi4@Ig;4Ca!J5;%AkQITjxq&(#m zc}6z!ED5=@55TPKdH1JS@_myEB6YH$KP4{WUgD@d$&&uwT|p`~Cmwa~~nz&Y%A*9t(&h311)KP06I=yRU z&MUpHU(?Fo+VHyz0%IcH@eC`xl}V;uEo9^iX$u#9tO3|&P&&hnHP$|wj!aEGsnDt9uK>FCqJeA zr2OyG05D|!JTyJ zO#{B)RhDI%A;3ij!yB{Er>%b0NVtOB=Q7y4h!+ z?UM-B)*i0O^3jgnyzG zLo-zwSjPz)G-Z$?__IdjNbxw0pHFp}NI=LT1Q0ens~4YOF^_dvX7VHgU>;g9xd1MO$b_Mo~R9;Ot_F?ysZ;QWd@uMw{Z` zK>XJ5RFT&loSf?+_&>4%tM-fM$q%9+m5ctSQ0a$c?3^cOTX3aTO15Z^U`0`r%#q*2 z^_Wnfp~@PM=y%pF6B%~-iT(|fwj;t|Pn06me(>4E zWOeUKRRc^q33H`o!rz+e@7HdB>dPu}qbYo|!kJ}xpVl;iRHdiXLeq6P)?rQS>zbgA z#b6t`jY9q-J;UMs7LMJBpmi@uXT|VFv1#n;=~qnq^>S?{w}&yh`;}=PMcUc1DLXSe0h?!UOW!#iII}edEcLl zaOEtZ7+K)7HIVXxc*-hnn9kc0IRaE7J4$st5t_AeO-9hmD#98wwda&WaWGf_^{0NB1QfrO4S>)29qiN&;tD{*5GmfbI%3jfCDejqD{bYH4a#TeHB%ILh{l zf@9YgyjmQJ?XmE=3v}qPLoY`}`&oz6Q#70L`T6pQb2*tlbe#!2W`oeLc{wXwC=*H6cY-T z73&61-hLyn{4{?kknGak!kU`o9D84Vmgcc4wLMn}G;CDTHwp$#mNvLXg^|P4K#U&n zz#{8tGOOXfr8?`EKf_MaNZqw!fxd))MP_(vN>cK>$XLwbjF zAq+NT6nkEhdf|6SKY(L}yj!Qs_R+?A}jcSrq6Uk0kZ3 z_*j>wV{{w&I6Q&k|6}j1;-cQ#KTt&k0R<%mDd|Q5si8w*knRR4ht83q5fw$GLt0>l zkQlm45Re*Rh@nGbsG&R0xc7eF9q&Fj=kk31cfSkbS>N?^tz?@6k8kSDN(EKCP>-V_ z>%(xdQ64K_#B;nz4;f220a--2Jpgg(A9%mdKV$rA->?P_*wnShjilA zBuZ#v0+t6@?=;p)VVvPFY0>{s`@;^GE#SDe)bqP;Nb!CHTD-u(ye=Wd#@d>uJko=k z3@-5pHF~_}PQ8@G?dNaSMxf3Y;$`;jYa2|{h!rkr+?(&w7I+?=d>EK+3Z=oJHKc{; zC?It&w&LZG$v*6Z5UG+a&fAz;kBtZaa*w}eK?f?sGpHq>n$cZp{eZDDJ=7@Gx;b7c zxP8-M^iZgz!S0AY-=H?@iI~M%!H3x&iDXU0N<(QPGiT}%Li}V67O}<;%u063nkPm_ zc_+gY*^CkkN&ru>{rx@3BMypXGp|sMc2p4i#d7!lkXQHHE9v}YE#8JvgkRY5reH_9 zoCcTn5gnH%xXJa0qE${PsUams!(TN?9_FrqKO>_5mTP6Cn72`*%}WUmq1_jeQ`~(< zcugGr+~gG8xfPw9D-5Bw?$x|TP&82$XV0!v*%M}jt$**@w-E`O z-T0yK$UKo$bIJBwdCsKgOXfi5i5Cq@aO}*VfL(`)Sdnq(2HSqS_uu;Dzi#+BN6D0t z&XrMgiI;6q{R^HfkArXE6Lo%BDvcd&`uT~!5aGoL)z5eiL%GrO2*adqf|Z8S+E*R`4^VZwuGl@d2X-C<)Z=@PE`|zQ&ZMa&W{p` zA9;A5dww7^{VIpS$BjHtgyeX2s2m3T@ByWd@@g=)8*q>JqS*;pxPS4Sm-RURzwn%r z77ljhaDp-3?da1x>sCVYOXq)%-Kz zHEnD=dRa&d*lPQzs(I;1H}H$eMbO5k!IAgOMg|MkG9_4kT39Yhwnb<7bt`vFpi)FG zIXxb)e^$}6&Ypi7^DDyb!v8$|kKU{wu`;G{1~=Oe0ZWj&_Fr`(Fm43BYWzdgEGG<@ z7!$2d((eJmsTXcl1yq~6B<;6XU*~^5B>ZQaI?3|Q6xT^?OT(`r6qg1ewZpAR7XS~x zzpoM*dKJDT~!^U$*osM1RI^3>5`@97xjb81vvLcJMQ|274Ah9~M zA=m{%3PD$5z??yA^ZyzUr50bD(<k7%s>-)47 zB<@z9G*(IT*gbD06Z&6UYWke{e$wZqIM*_M!bGX-1$G~=Rld|oU+dCZr~~XiBXR!? zXn=Ykklv;Nad<-oqb(Y691lZTk;Naf`E4hK`&oE`CqbaqU5tM05=Aeu?$E(IG+!FX zF;Miw5Hsz~YXhfZY^oYS7Dr++J9e#6~A_=%FYlVP{9(Q8AS z8%}~5s2JTODsEv1@(G{3DSr1==7RAKOQ=a(}gZYGw!oxm( zp6TOxkp#{A^k;0L&sUx_1VO_~mJb~S6n@(+@q2BwCx=%HZfUB1{?`_A!Jw5c*%!m_ zXUCz~RUtJuNoa(ML6UM|596aXeaVkB01g|Ksob#3!Ac?qN5J-TnNdQGs29V-O&7 z^;c&rOHSsBX6D+b5TP%^x4QeBpXFTMbYtCr3!8g27H|{!Wf!19^kel%%pt3AG?1*0 zNI>;SIygse!pwD`$4OF%f?^NLa0WF2FX@^~bu!Lv=iLv6DzzRHYm_|dh8lrtFE zH@6+guOEMcohtM&X^;vgHWJc0#Qxq8et=T{X&hjYEtvhQ!Mpq$a?Ue_xX21DLH`~% zOBL%Fza+Cg`!ht3`i$#Y1qT+<#p!P!Q^kpqCF5gL2_K(WR1u4lB;OJ%4d#HB%WM&F zrGB_5$UKNb*anc4B~SW{3O7vsk_`2K5SpdZLmPB{S9H|v;D(CGABTAP$N#(gI2h+% zbqn5~vWs=gWvnR~$8JWb3I|$RiFtlanw#4*=h(X!XQ{0hVwxdu#5OiHO61ILb~b3S zTE#zTYe*E~_1g2DM@`N*aZ|G|#BrogZ7vWV4_0th(u8`z6(ZS>o18n9sn;r#@+j{l zI2*Evt5jdu{vNjnv#@J0cAa%@{#E%MhqgG^ya=eAT#M%m&zF@cSUn0}mnAFwAUi1@ zVrU0QEQi(?*@2iI3b$FVR>=>(FvKHXC1ro`Fzr)uPpV^3m7G5#0Ko8JS!HtHZ}0&A zWF@~R`8>MPpIFL6(BJj(g9###u}^Q`fCdNMZtPpZrs(ikv&g`+`~}TzR@#JUS}Ez0 z@HZsuzfYmDa9|A@ndjV!@Si4^<^J_F>aRrC_-^kS4+RD-%jFvG+FKbnt2Ix=P&U-w zBDnd`Uo#%$yqjFw$J4IsLw7^}o62yr% zENGds^6)oz{zK`bU^#*ffa#z4?CYdTv^c7B_F^lYjEP==;uG`+ia-IBd*5uDJ8v zMPxrX&XLBAtx5J}vz!U#cFSk1TPH;w`Oe;a%av#xeHPX*+hmko0d3 ztiv*;`i@0DTK1k&Bwa&l>piH`8@VJzOwql#{2uN7xBKTqjHR=EeaBA))ZjSp5qN== zNb1fU(DtR?3go5WzB$KW@0}i{)6cy;dvh@#@QH)_eYJ)ne+*W%L58h_2^LiuCFL{7 z7|OF(MRWHbzaTZn4loegn-k@(!sV}$AJfZje)ILermQ0HtEFi?s0{Gu3FZWG+UNUw z1|CEeYi`dW)F^QF+H^0%)0A~oO@))b0~HuXJ$>BW6yl4Ar%P~<;Mw8K#9&(w_@CV zX@|ci@~)G<$Ge^uCQOe6$sfIg1T|}z+376XP-x2ZBrjK`8%iJVX|t@j(|n^LHF?y1 z7G2`Go{Zut3J_{V#6}?~FA3{zBc(1$c9BAiMY8H?lIwOKt^Kma)M)ZKNkFc%A5P|d zKD>d&O4ad_@u^gITK$G1cq{l-9v!rtx$QCg^~Y&8M)#3r zIas?oN^fRv!8;gm`_0LzVHlz)hIRFEAmA!L;F!QTSS;04^` zj^*irq=CAHu88WYOen&ehX_NUxxQHF`G-``^>=+Ea2tz^gBd3_+U5tW5REy9s)bU@ z3<^sgamm%;hM6atVpH_lkJU!q#y_TMjlMCdTJUdF=oUyUhQv@1tEdzIc={jN)WUMp zGho!gUybW)g8iqj5i$+*$rP!puI8vb>`sd_nXS+~4|=2*UI}7Fu+K@dylGad5o;{e z2Q46XQXzO@igN~Mje3DCo~BFYpLhc=wqN`rfsNHaD4YM`J~LqG+1?OQg01o3@pG00 z%R_0KbZgtu+j()XJ=lh-NgQ!ZmLi%7tY2ti;(80cJhj6;c-v;_pDml_DCLs#EW{ym1HsK`a^>Lvkh*pCE-mCBV3}$7+OVaxiYZM#dHqDe(mCyZEs2|iv z{AU+Hva{Lkx5>fIl$ej7LYCD=?e6(GzcV8MlFW?QQ192!aD4in>ONw1Ja5!h07rcZ zDmHYk_Ea(RV(yEi>al2U+hMLL_L5G=a<6O3za{d5a3xshguLBpBEFWtWw^7qLi3rB zn9`oCZn6EI(&U{gm8dVWMRnh($6gut=WzDN{%jYhVOG1?PF_+IbvG^%G&~dc*zx_* zGgm6?FX(yKJ`W(A(>ZeBXGJX$aa5Vr^Q|{jD2T56E29-aNI)l;BNrg~tkITws;J>y z2{+-J|4*quZQx(S;~F-N>rd26^BAh0l#GtXF6$Z3Z_Z+&HZKdVLj~GuR#Gc|PK)h**8yJpKUd z3eEUaqyLwgf7#S+T+d}tiiA&Qc`87b_u(B;z?15O)T*s&cQu2$gK?FFI-T$1QFzW? zISz%-Lo$7c4_*U1*S^rxcXl1uQRjAVYr^^Ow1~W|_+%}@&NKb!MkGPn3FZGg z(C~jB|A)Z;-w{CL5{mP8|9z9w-{g+}7W7MEWv1tfQq#&~z;8P>|6M|5a880+?(e(B z^#r6x`b=iZYj6n>XF*gLV|I2%$*K~#gK=t?1!DhQ`Tvxcv7)S&mt64hrhoE{KqvG{ z*K~g#clhM44<*+f(1Ywu>vC^%bLUR>@s8lXZB+T|1-Z(8xv^#~mts0>LqU3sduzru zSeC(HMSw!lbXXP2wbRrz6`{`bSdGwJY)_Q)U3A)CfBBE^;LrC~J~H>c|29ZDf1h8L zq+`xN;>{863$Ju<%n9d9*N1T%>$H2PyQjU%$U{O_Q1ck?zv+qpV{UQj=A*ulz#I5 zT0a@X9X!*A*ME&;f>k08{ubH(UUOm;3by}YkKHD2%JNK26XH8Wvi@M5MQDJK>xJ-}er)tBB*Oe#TkGKdENF5*+}|}NIzk;99&+JU z&(L$0WXxa~`EOVKs}uS4=#^|r=IxjGafBDRUhg)Z-w1*lx#AzrK$32@F-V0M}o86(qGm)p9`#iYU!Zl zpqn6|3%s{aWqpR9b`ItB&yL?fHA^{mMr9(UImyHgR4Ma^mYDcx>arj znMVV@D~)g27!~V+UJFr_XyCq8C#PtCy`9NK$n`oaPw5yJ_c{?B>+W<0{{Zn zpxM1e#J8?T<=SA|S6kCpXJ?YHY}KT%JZ!<}v~mxT#NFlSAv}OG!0t^*9HxoT!}pci z2O?QC2<w50Qf6C$6JbO zGv)ds&&Y~+o#81&hybbnsyxTnZtvPAQ>eMh6+17}xGlrj^OJRVc+6&$6h~676TvF! zswT*p6U2Asfm}t_#&otoE@tL3fcz>%zyhTsWqg%%W3X^!%w}S7Y*}F%_{kIXsnM?( z_ol}&{UU(pA=&|&m_XGsW=N_|2gI={y$21XdU_^hW)@})Ko|9J3Pk=qfCTb)kr3t_ z$%duDO8~GzPB-G+%62pqtvfZ9kb{6f=Bi&i*0Gtw%~c(atSh$TBK&t8#|X}XtkJs3 zAv$s*ME=(ci;(kQv1-%;}t0g6j?(`NID6{h5KBkkX0X(F$hh@HdnSq?GY zXYx!uw8TV!o7E6R(O3UG0W;^140ltJSr~RMgqlgDI&$i$6gk*+i>}`BXbBxd*Q)H1 zgi<0=S!ZMRYDnK_Se3_;ZC`Dw+8Ya=SUU8M-&UfEd7;9+5s)9gCrZAsBgsn(b$rHM zxCUAO8P?03`*9LS+*|+DRpQY^hODu)$@Yyrc38#9LVS6;xiZ=#7Of9h5HxZsYONtq z^D^rn=DqzV)BJ~jf&-XKBE-)ltnpPf%gg%_UlxHa)|Zb7kL?6BI=w_A8((90{;hMrc`2BNo8@#?>M& z^fm`wB#sfl4^z)AQ7S_~e&qsXWq{iCKvlf8F()w&)+yPD0)azJgnE7W`mzrCfaivn z3x1;iPt;^%rq_SP;=dDce<-_sJ z&GiqFu~je}zUO$9qQwG-8QQg&@ZZid+_i2P{BYHeG_^C$PVLh_kMgMER$<p?#RyA#&%&$5#rUzbsyf6Z(?Zfl z3WHh^TVKZe68V!Dj*W~7HoD%AoQ`6vSsbcqDHWwIuly!#Ohj_c>e5HVz;AnY`!yc@ zO*j31317)d$`>1A#?#O)H+LHfh5D7$q@U8s0jx>I#Cn6>*WDu4Td9g+ahzwOuQ2_wc%D|nu#2u<`KJs^cF+QWm zH9n-4|71a|>Cs_#%_u825;$uFZXG2q8!B0a0;d{~C%m8tZwrzwUz@dtAg}ZRz-@k!_uOd!j4(j z>6U^xTs)PLH||y>Ljp)$!hGg(9jGw^l6(repm_`XgS64|xF7I)KOPJ**^DIR7-PCq z*mWnp-p)BPLlgsYi>CLgrVmdgJFK9pto?$`H%t0#83IliTq?WEL$J%C zu%o6M#5XrTQ)ZTs06T5IS3Wq05|&tT^#yNGv|El?8A)c7iyHbo;w_W6)a@ux8sgJu z0w@G>_L;HBhax=-OpBI3)SFeBHtWyi!kD3jsH)@CakHvk{G`F!JcT)a4$frRqS9y^{Dkzn8U>obCBeEE2ij>lt zYAOtYK-Zy_P}fMQ>ceK{@q3WGeSXfq4euGb(>fuiI17+dGwVrXDwwvBd&G$b5_OeCB~^oZv{Sv4xcN;-7JnU}2YtO? z=oU{A1IA(Kaevnix~#U{lSx`+%B2 z3cg(pB|z4^43u(m`luWjV-{nU=$z1@0tbdFeH$#AHgOn1=(%Oi>G^X2%oNu=8ZW8q z>Ar8mJKHagfQEYli4;)iQhZ~n0QO02CEIYiE7P;)NILdcDK8V(U$)nbNDyEOkbG!U z<2rSL#7>wUj2)H@U4=5kW|eI)2)WcG!t%1_?hh9NKrGw63#w3-dBfJ*JhRy z2E}&Mv<_?U{@308A1&}tVf()y_!lPs7e)Tp7~=msW*jPWRNdPlWl2gT*KY}_Sa_GW zYm5KC5a3S0110BQ%3;FV-vLb=(MUj8O85U>AB=z}k2WKEGx0JZ*7x&IX+8`aFSBag zX2K}85 zHi``q-;(tlkeb^qM!9BhZ=6HoRqmMEU}il&eBTRdLV}0)Uvu(bN@QH_ZC{*P#?UoI zDnC;;RxWZBR!nE77Y{{`V1t`X^W_q-lrVU_%fwQG{+ET3ufN7!?M3`0ve(Cj4H)?9z?2IwB) z18%V0)r=KRuRo^-x7}sPl;xXD~V)s--qzBx+ z?v^iQS8MPpa1o`}z@5eq8>c2TG1}!?g!W|z z<;DhwEY=VtU_7x;R|Y7y*dr&9;j9lzxxYaUx@elr}Rvo>(S~NF{vukC^%zj_Jeg82f|nCTMnlQ?M$u7FvV(8bSmW zSqLt@*qLb7oM5G-1fxe0g7X%)oO{X~@ny+)L^MOr!(!2_S2q-zjn8zu3Id1Be6!|A zOqiYK>BeVbW`q6fg`WB-?n>J{NATWkw7Yy}8>&Lag1*RI6z7jgh@Xk^X%~Oegh{n( zY~_N}d6oBa!5IveOPyKh(QeEJp?;a%`JoQCPs|R@@aJcItpwtXzxzGt&Q_WI5AUw% z<2d9mfF~M1U)u)J`ReXwkj<;C9VaLl>~8DbQ%r> z2zE)HX`VFpi)S>wdI>foO~T)HU)lFoL#aqV)oFH4D!)<*_|&edg^bV-GS@GwB@gU- zLpKN}%;2MA`jq$+97pUISQ;UuMtJxJ9Kdy9y32KBR;u>NWyGUUUz7s$Y?`OMdmTSP z(HoU+Yk*bt7n zsbHDs9#6eU3Zk!Nq&#Vba{LzAtpIS?^E%NgS3g!xEe%QrFJLTyD1Vp7Kp$-yN9=t8F4pl>0H@`YwUPr_f?y8m_?ROP!g~GP&m?&3 z!^c$~5qIz+oUQ)21=9z60bca|Y(D|O%e(d2j2DV=2z8WW0HMB6Mp{QkB(jsPlQ4PL z`Ju~2@VP7jx)fxQ#xh4#$nehlNe~=AJv)7N0cOW4y<}!G@qEHW{Q0h5lU6IEMto2p zVYNpbB)qEg*_k6_)H90W#mq&Iyj**-^|QwejdEzbXN#y( zzdOYcH$5PEkC86CFopR-;XvsayjWe!akVS)Z^!^5{*v;BG7q{{`~0xRyyqTl0>yb& z{7N*p;asbWzfQ+`_h`{_n#VX^WY`ew739A+5(#!123_q}=8Z#a_uf>H73G~s$_^=^ zVAk8A&{s0h@tzGEo%k+I3wi(puZyan?crLw|F)OhhD(kv^4kliOr*z z`?}_IJDV~%tQO*qpM$b-lBzV4Qv`*qg3rC!^1I6r!Yn;Bxj|Vx3y?lUb3~JZkIreP zHM-QFyi0e5(7tCQs<^(liY6|)c4%SpJ5~jtRi=6Nx*cBIbO^0B&&^+;mN+GeBc`%M z3J}(zJKD%`WI#+tlUm#^&+xsq@Z9F-_)Vf#cUOdhM|9_5$SLFpBnq}@e;unC7b=fb zR|!{jKUJb3Y@+=4CYH?}@EPCASv0 zaCRJ47YSv~{dxN#8{WCx4zD=n4WJ`jTa97ononPMQO~R5^5@7+T8=BE$R#NnzT0Q6 z+#^|qba^Z+AXzxa>L!w;XopMElx-T! zhy?-~m>W_FB(+M-*}oyt1gz8B1@?V{euPTwDWVdVu^n~u{J7#OEr?`k6)^3U!fB4q zFrQwlCQ=1iuP7KzjPn==;4o{h&>w$3?~LBbWr2IPoJPtRqTR0UWp)1Uu?4T;+Ukz( zL5*P{+>{G9b(lSN`HFv?ik~}V61z<3hdbCT>)pb-#ke)u$1=|cehi{|bs7`uy9Gok z!f?l`8zuThDGSC&VxyQ}2e9eJ9srI&MN}7`V6RX}!J)os$6j0AfFW z?g>=$DN%ZXl=*T4cjJ-S(>hJA1yN#EUlYLP2@PH_7w&m7A>4KQ$r!wNyC3E4^_Foz ze_mp9C^&cWH<|}TgQ<1fW?2Au*nPgH$MYlk4Dn^X2gfsGtKAb^-LA&t&rOxgse0H_ z%-+Gw9Ui6ZE9R3$SRI|+0~v1ZfARA5OOfal|Ae*DHa!o3JOD{B@A|qOx>Wz}WNY_a zaQ1%lH%*DBgX&7)^tX>NwiBdbJL=xW{m&nZGK(Kv7ZhC;Lam9{oJlVSNZJ3)?0XU) z6M8LJeA$?^O^$HPV3#XYqe-6;&QZi>dGYKT7VA%9lD~x+!OH~CbGZ-+4M-1%{Is_W zx!hL*$ZoA4IsLLIfR1ubaE{rG zeqEls78)+iF&ZWf3|hFVefTT#9e|)fw$-*Rt_AG!^G}OEK|2JTch? z-jWVRl!2F3I6U|;pNr5zXt{fPQx$$vHvWT%wQP9)W_EKFBtnAY5mJsUL6M zPX+wF!0eqe`a9EQ)i0V~k6+Ai^^_WG;2HJYlva1*h7ctq!+a^+mlYSZ&#`x_$k#5rBD(O3$ z3J*)O*Y146Iz0QKe<7cy|BtlZ7sl&>h)ZQ7)zhfqwE*2g?eT7LA0;1V%C)U^D;R_F zq{>eJQj(|N=fy2>QzYRb1?JedKR`=D;=At|9q7%zHp712NM9LHB*o$+kLKsr%%=H7 zV?n2r-Kt4G-&rG}xPrLc#4})huC&a2x>c@HT8Y_x^P}=^u~7mko3*?Yn~~jjI6VSW z%9Gb|(!=3E@X8oxl1wQo1?cwy1 zTu_y6_PY_76vx9&D_vN_n?+k~4-lK+FDtdO3@7gfI`3jNUzh6V2M7D4z7+PeQ!d%^B53e?+04mE~~QRP)#1u zHT%q<+aE06*;p)f(@f%3>^XVhBvx$|bpAl22?#ee)ir9=ji_A^viJ zOIj&@-0xMF*j-|$Z-wm0n}VtW6EDp*=B^v^i31}?jBpQMYA`Avu^`>0Gj#1(G9 zP$cM14C1Qv=j%hsSQ1_K?iVpb&fy7&1hvGNWknf$)*ED<)vL!3K0IRBHC?@ugfQX}OyoFd#rqI&oMn$%ALHz}Fb|TZ}2Ow?dKN6?tdLE4l^t zb&?^fr}AYRo@VmJ3()!X4Zq6)2b%!7v#Q0EBcVMgSd1!1v7=S4&MNB>9O+T34+JPP z!)jTo50qBvx|TLMDN1?cIYb-z2Aa<5UN3^{d-+u^w_ha%#%;#3sVMeh_7bVk2ch5X zpjf{Pw>)U08Po?zy;IozCUC z7UJZ2oN9d-AlH^QR#m>O>$%?&WQQgJ$&}NGy3N;+4Rj4^ zoW0*JG8TRj@aSiMSU<2wNf5ZF0#FW&?)7W1CI(g2<6P9fI9YC84Wy(C-7ZNpn|C8} z+>;WKr>|bU+2xhWrh9Q)Ym}36&1G#*bhzygUt#WyDKKDZb@4veY{()kva_GQ=WLu~ zOWMkEXIj_Gwca{+YdWTx>=;l&Hw+Rs37(DhgNrEdN-m0a9Yn_KMT^@sww3__y?F!) z44So(D)c0gXl|q=@)9W&vt{{VDQDkxRnc-Q>A+UG!$yBa*;X2V{cId@(u^843G< z#M8nSfz0Jh(}|ovqUALA5>3#cCwB4-{&d^tP5Ki`-2&3qH74%0Ma2XU13xo_K$_k* zy}`^+xLrxp;Jh#n{NOp|^1>2QZ>Ipc%vF?|jcZ^t<-FPHuD2`RL$sH@A*OMJWrp0O zn~1#&ywwoHyA)ENJYyDj&MV{W?6`x6ic=CFg^@V&^F%d$Z~*2tV{HPCDHFl(xu`FX zaV~Eky*dOa4F;+Hf(D{B&=g10j=)$gyRg#|$E92?G-hp7=^-FqZyJ8fqoZz zsoG_eF&A>1NF);?^6g<-kFS=ISaySV$s}eU@cjna*=32hfqR`PsheRaIBW8;xEQxA zxzxuhb%oV1(VJ5VA6cty%sD|#c{N+buMVxPe;(t|&9a<){6M$R2`q9XP^>dUs|Z&2 zanqV7#*o}YBDacU64ALipkKmzDmRA3v;LA4hi2m4$=XPjSyI$8&vQM~BdlH*!AJdpr{E#GfQ=%-$f}Zp&Sby*IeSJR!w?^36CM0(&E77L%y4WBvh2 zKgHxUy`(FKzeaylcV_MPQMri{ozyvCC8pV3U+UNmnHqk=8eW8+jny*E==|T(daxPu zT<%x16VNlO?N6}5xklME z&$NgQ|5<-in0qb0HOmQ@+{H|>HzKwZih|Z^3kY>d!s?D*N1)QrAp5=#`!?P{Fm;y( zC1GsK9}Ao(ARD1VHXCF>M?NbfYZ|US-kJ7FuepE>T5Ro_fdV$pcP|}$yvKpD1x0@- z0eML%*9qLCmoBVt;6lqxFPOc`_}LJ)k&&lBzZZ#SasC4q4DLYzqJM$*xq|#Za;!4d zx2E#LFpTMsVb>?QdPSN+@b}yOJPltDXgT}TG{5Ze#ZywheVa0ng;;3t-1#bu3|_x7 zm3UMooqLFx@?_Ba_?BwC&x3x_^7P$ed>s)L=c9yW7EJxL^!vAOO#u&SU&&#M@y_>P zyqfY-z@-?kv*(E*NKR2$GEu6Gz z(R6x|Uxr&ha}e3c&qMN{i|9Jam9CJBPmzHwyI-Enh>@oh-2xR^T+ z5aboOCmb3Pn3zq%y30b+h!p$;5-`=?V>MLgr}e!&J>aRb!4uZLpu=M0*^7DxuL&^Q zg~VYCLqD&Be7n#l+=BJW$nql6_Mh}aO+6+;l$5hY^$i-}L60w+Zm%v>4>-PcDOTcj1ttP&*10uKL*{nUWTas&`njgI3 ze5Pu3Quk)M+NiR+rxJvW9}`xzuV1 zA+fNAr;{icPiT4Cu&rlwob7{W!TPz1RkzB^yQ_vq<;c1W96Mnpp#P2T* z@prFql#uTZxE~Uim;bue03~Uj^A#FOd`VYnQyM3H2X;bPcVvm-;W)9>nz`{=FuudX z1wTK2oEq(^NEe%4aQWoBH)r1(AOEoCK5d@wlxd*0KcT;hb11h31;}poZWoHgc+_q{ z#!)cP7n4j{+&FD9c-s>An@9LxL{8rYxg4HOcRalogIGABXe!$;pbX;Ep3;Z)!2;Wn z;}6BKj~Vh54%Kfh>)cQM3?3bI%5;P?@i>?zfjSH5zLzV*fo}CO_{Kmmo-}#nz$FtB zh{P^zHM0Rc-R+j@KZ5Rc+l=$kjx}t#xNe`weoV7a?Az-UcJH3Ox+HK+p1YhiLLRjO zTUe3VxrF{&&ji&dc4tH2prg<$Nk?D)N1zu?71K3%grD!mQ!nc^gxwV;Up8JrIS(QX zoc5My?r0jJ#ds&wC#Ujcq$`ZI2@%H?p2R7JMrOPGJTd6z*Et5$OK;sjI}e7VZIeR9UWk?AW@Nb?5t4$GHzl4zr+@PI}?MwRnf_`pSdw*Z;`||Is4m`S}`R z;X3}J;LpvJxf-YYbYrrqHu_wx+&#k45izfeC@`Kw5ZZ~D$~z(`Rnf<7U*Y)WT>S(` zcdlZvEny<8$aMIExf1yFlIxybXc=zQ`i4MKCwhSj+6ETe5&++Fh*0 z8IG=OWJT2jpbZXzwNOoDS`y>?VnNPl)0X9yC zXCSIS?yrZ(fwkzF{Y_6Z_jy;PwUEO4E#HvSNX@PvV{X$+?O}Ws##l6vY^N15-IDS~ zhIOhF@~8Ft$-^p?#W&|CG^(7il;=7IApdE3TJADipkLFGfcfz16#UVttgj%P2RQTvOJnoR7;hCBDgs9U*{naKx2@k+s@bTUaE(DYh$8S7xNl zbX2rU4U+^D0vHAJ1uhssT@EpM7dAXRP3yw!o!?1rn;@uu5|Eu`@`LG7ma6Xk4jI(r z$~ILWmsy?fUEOIBsYNU+d&vb4+=D0x0o;Ko36_mwhbr z^4n9=S$J`0*iT(&xs0MaT@$h~M$4`mgWNdha4j%^(vf&5Cw_1JS#RdQ@yZ=RMKmQa z(ab@aFm({7n3AUHWe_{HH(~c|;$}>mROlk>riEYZUOR%@nBR0gf&L{jj$(gs(>Dbx zJ$`i4YP;(WF(l42>#p4hf5LjxU4C&R(~MSrl5}lx^LluUS;BeHB3~C=x5NsGcE%#Y zr#?biRo5x0V`7&PkST|bCr)hQd+VW6FKMT0rtMQ)E_CbfO@cD9xD8wc4_tW~SnsYY zJQ^JuEfD%B8hJ4^rw{!E%TUT@F@^&o8R%Xv<9`+P&Mr(!ab0LKvJKmKuS;(8%$$Rn zcd3%lPkF-R6dczQVI4WYH0b7Th2FW0q3g(f8g2Y?-G}~T#E@_%T|us*-Yu-fjb2Xu zqG0x^x_Vz8^m^EdC_LpefB&px~$&H066I#j?q{LCj4{wbaxhu5a`_I>qULAEGNTywE@ zx6U=Okn22_tO-P9E z9uos@fR~EGFJh53=VB}=X$M8A7v)?tRy4GoA01{77RBUjwUen>0Xc~Y6P0YWR8Eu4 zaRdHGz*erqJ2vi@jRFFRX>e5p6@YSjbk?@w3GZRpx;L%hH0EhWcL zq~F=IVH5?KKaczUHXOSrmnJ+ z3ahE;@2UF%HFKWpOIQa@aZDJZ%BdCj(qU}bpFY&n@gIfKecE74NmSP6lOJlMC$M(t z9aaFdd1BaSb5l$99DSl(7cyD`^>@PtD=4A}`D|3( z`6r_M;|36ck@ZQhCX-Y7uOh_t_bZvf!hy998Pnxw6E--`?4Q!f+ccdR)#ZEUda6G) zH@CW8kl$=egK#jt*V`PaiIHbQV_ zZNghifn2P`Kn<*yaT0yx$~7}u*V%}Ws&$Hm>cf~&y^`^WjBNU5GMFA^K+#^j-iH@% zisQmws+>lFgym9iVi?~9_ST#5j=IFpv`+r?6S>}JdyyeMz3}zzkn2ZFa4k(^eFBVxTL3)Fc}XV8e5X0CD!!~xdDGB~ ztu0a`=xm{MzktRz(I}}C6uv0Ik4IAaR<$4DnX|YLK`iKIc?LWhU#t7N z!m0NT)6uPY^aS?&LNnHl9{=t?CF?dAhos;})40Fx?pK(=VIXj$Ve~1My&4I4lljb? zG|{Clqv}ypn;)=E^@%Ki7*t{t<20k#jM@Y>j0nnvuwqRq@*dAU|>|(m^Z7Bth?_aZ_L94TL|V##eJq8W(8GO>H*pwK)QG? z$Xs$yYU6B%xU;aR9DJ8Sr8xa{#FchyBBLdqs@but7aWc0CBeDe z!#*Kr>(D)pT!N{SNA|Ob#Xi_4*ah@aPV;D!Id5svV!$Swjp53|63{hoQS7{M-xtKt zbunfruI8iR4pFNQ(iz6q5sZO!o`L?;FHyz1*ZF$T`p855p#H$lv-v?UWIMt5JS4^w zT2a5auAeYsh)pxwq;)zQr!AP-Wl-u&rbqXlF?FGKY@ie{3``mr3$2HG!D?A^u^}^e z7yh|LCkP%5j@OI#i?}o}(hf5=tnxC~?jGH$ij10SE|qHerwOpwCf)9XyKM924ZQ7F zfXpJnIj>JX&b*;i#jXd~==(~?y})`T%?X&2dUz9{3cyx+5P~1~|DX20JRIuweY=#j zQCU)i!en2HvJ5G*%uMzzWRP_#B#fP;l0CzaJXM%J>EaxZIq&N}?`t`)YwqJP#&Pxy*+mv;S<=fV7Q>?9%u_?- zdLQRTj`^GxL0pJ+P~_RW?h8IT^|5lkTOF@xaO%e$xS8g_OLih(68W0Au}cVlXa2}9 z6jYGk<{ybb;xiIY z47hrRMDXfAw!S5{Fsn^_d?33dFbUzw_^gg!`Mk0?Z6v~7m#`ip05s~&?^+&KW}4wz zSKP955QD&ba*6{~$Hx-3tvjY^^RpVKNK5CyUiI1R=>#82#5Ie*lm_?vuIAVjyW6E-p#Y|1kaq2AGAjE z(O6qDj~4v)=8gAPi#ueT#@w6Sw_)T%uvDW^=4P4+vfZHVyF29+J;_JPNJFNVx^WkV zRHI`Ne2*yt3Zt#)2FQDQ2pVtES}{{4HzWj%HAQHEyFu$9D9zc?)tE}FuPsY1dnK#0 zd`3m2-(LHU;mkemsee*y?LU0{6oat|A4_bZR~MYh_v>K1{bv?k{Bq~8)MT$w`19L#Z9M!Ecr zuLB_T8t52{GS3eIgtYLHLAN=t0Ue9X)fD0G<9Kv&E&U&h)!culR;+6o0&4!-E^j8$ zBg;VEfK3+q8(A-&s=9n{xdMKl<^&T1><_iy)ISrZlJq7R+zw07& zJt1T~PkHH$=n^wo0*84a7wZr^aaEOR79s?RLxb=)43gawvGNxCk0Qk4%q==5Y|BN}(9hCUi%#U}nZ>L~qV8 zXHfsdZ!srb-}K_~9{J>d(xZL*gtkC8Zy5cv$^HNFG+2`LRYioS zq36!QH>R-|P^p$wpOq3obsq!ck$jX#gqAMHE!~o@=X=Bk^rm%PJDAM(|F0K6V{kCR z$aPo1QXb{2?s#5$up!!X}#-)r}q8L{(VCli@rK(jFsQDT=gEW zpS{8(srtBjDG)5}qr#?Iqx+h3+Fq(Gb4%dHvGYA=8B$%-#&1ddy2f`u2F(|bO_>@5 z3JheTpk`N0-7DX)SC;L%h&W3~$N}uW@4z0p95*o!K^_s)WFke1`7%5ELQf~DXjj@i zj6PrR{JP9R`LX12aPBY5hCFU8y3x~;`Q_tkHcQeDQ>Z!V?v9z!mL|vx)DZP7adZ@V zmOhG;E+~~m%*V~g!V{k(R@+B}WgMhM)FR5#_y`6pBi{5W6o#&xy9w(7%G^h@Gs&r{3lPSz!H6^E6@ zmIbr#UYKMX#@^k-+!~Af8ZVj9755cU5m)h22iY+V!Rr-WcDEH?snp`LHh9VkQ`G)1 zc)cUJSh>UbnGWJYqvk~lyCb}FEwrnHnlIOWWfSWctnykr%VLt->2>@SC{|A@-?Vn* zj2HI=y>+c*+PGPA5&&zC?}!k7@PsJK^yVYm!C)6w;pO3Amx`F0-Xk7cTwLbO+eZWk z*Z6~jF#5V{cRx6>{dT1FRQ5M0?w_?}C|z;sHL#doXO%kF`zAMc`2fiP!G@|tTLSiE zufird7B7)K$xb*d?xz#sl+ZXLx|`&HdF}BV|GqXo{7q>d8LQ>mYVFSJ2SAt}U{X%` zS}@WhCE7fP??wleY3)cg^N4Co`5>?UDV4qMP8S7zmeJCcGi3z)3c@~yx|?)5mx^lH zB3vF!ZCzNnQT{R@`$3TC%bi>3C5+e_!aLM!zqKIwaIlx6!7=1bvT}&HkZ^)bF1=5}+7)t#4oxB}kAW*<@nT;=uLyfC{Nj|JfKl-ByEsiS^sIl5M-*{DWd$LD zZ4sAL=I5X}D!J@MA+{VW#gOIF7oj3M{^Hb*;r076CW%VBCm^<{S+qD+Ws#t2uIg)4 zqsB%59)~G2C(SP-hiPgxc77>bBq?Yt|e=jS5{jk z33O*es8a%AI=JQEDOQ>?iJI2Flp&M8M;GoQx`i@Y6R@UfMLJ~{QbT-5_soN4lU39S zC-GMT>J}Z-yN_5&k>dQF>s6OY+Q~E-2{B2D0P|O40S)zKPIT7SwgpbAFH=wH;hTyrM*2VAFelZG<(b|1bZNVi#18N}akDJz`(F5pK z)a5TQlH2K+z#aejkE;b8{^uorHM{5H`+>f5vK1k3UJM=izLLle%9kZ*l4BsC1?~rl zo;KjrkhupQ{--*pk6(g_`9Ydhgwe?Lye1WncB}RIHvKwIq_Fp%aQA1x^0{+UG-#O8 z5-Qk2Y3~|%@s5!TV$|oTZi$tKO`<(2$Y@-B+WZHwdY25vQ4y7NKQH+YaQOuddB1q4 zCfk*~ZtLc<9oQ) zIb#Fi0ngA)9(ahX0eHCO=)-L!?gc~Zmby>FD8F+*ygB0dq^}dZQ_d<5{|-UmT`6}8 z%CE4O@YhtRf1rV0cZ?!D)OuEHTyl_iiAKO?bW@;Xd5c27-O!r}jW#pYu7UN2j~h$$ zhdIkvMbnaw5^2WqjXV2taa}p;K#J6U{{VkTGmUZx~ zh`qC;hw!lmRYPA5x3q6_kd|WlPsVhz0Jw;|^4$v%Vwe!~NVNVpX*w4?=t5kL)zE^1 zszcdRZQ(C;1%}}9wIv&YFjrh^o#K6WayNqDpgEjpAa;!MDVh6%s}^Z(cupxMhN{Ig zpwJbXtF)fU+0A+7e)yG6J4pX>nH>wK7F@94{E*u;jYM5C$1VM-1tGmLJ)yV&L+~7g zfC8`EIIfK28vw0@@vAxU4xu7Fu$oIc^9{4<=P3XQN!s=+)64HNwCV#lYlltlHO)Aw zz0y?geqZa6nW5d0QR3`8;d`l4apNViSjBMp%2-2sv4nVlp|9Gb?Q#6IOU`*aq7_k> zm4#@@>xQ_gjXOt36S2cyBM}CV3Vn30@z;s@L?;wukb_jO&ylw%WnxgfGXwNawh)hE z;_l$*cbT#V%bqqSWF6dt#Kg8iGI-8^7Kvj=k1DslGyTs7rZ;>*WdmY9nBgxPH27Z| za0_6Q{JoJ`7;G&8Sc&R2X1sjvt!%x%!u3b#vY|y!a!QyyWkJbqc|E})4+V@qk{?!- zRCRe8wtNcJ(CMGtLW6@u-u9_c7a}3Ek8}BkC z-0cHc{up=2r9N{vVkMH8LY1Y?+@L<O%2#Tcp_%4F?`dcB}etxzTSOf!WG9bicqHAk)2zLNy|WmkMK zz{BsS&Q&dch|NLK42C(LC~2tBXG~psI#67IVYcDrP-ddyt6jA_|FbP_&o@ccyDIZe z&kM2^2T)>Yc8kaqV!JXd>9gKv`2dK0%Nu+OX3}& z+EXh)A|U_d!lotsEUCCxvAcw>LKd|k6&n_WKfC01HU=C)Vqa!#22#`zrcY__RmD=& zV&UoXlq?C>=Wt?Y)XOgC?;qt)Jg1^XF3P-E`hYN7;O8htBR}b2lt@eBVCv2I!cPkG z72X_Kgc59gnM+jfje9Q$L?-9alA7U@)B2y?rKK$6esIrhJ=FhZ2<5Vri1ha#3Wq!A zolRIvSFWsk;z+mCDFp{z#$v;s?nbUT(niA;O+o{WpS5p#fU|1;0M9p+;Va@A7r#Np z?t>&@WdO5onC`J*v*7s%KB7^vKs=5%5IfhWWL|T9dGHlk0nrXM%=^G_Vu1Fx%v&c@ zZRrnKSroH=%cTYj29Jp~$_W&tzNr^ki88Cg)tGJVD)@?(8n#Y3D6oHs(o64s?`iZy zBk`4AZlKUs7e`UZv$F<_eU9#Exu(nD)wvGT7Ys5c-gZ%+NTvbUx`;T0G{^t-j#&D zex12NQ!?MV+N*tRB}4ER@Au@}w8o-g$T=lOUv#8%d$9J*SQNWr=}L_7TE?)XP-N7or;0qS#op^v`8J4V9gU+-KW$Gy#J z;K-MbY|UcqBdZf zkA2?Rg?e)wn{<2=s&x!P1@G3E#S^XEDg4S30np`Ih+hx-o5A#>%~fhOC)O2xDCXlY zh=rtFR{_TC5X*i5RL0ujqqe@}#>> z_dafo0ghd}3T}Xd>s=yy(fYvpJPJa@dFG~`RM~~0?u#VdlERu8Y8UfBWhAlxWB2Ck z$3sgbYHr&B*`Pv2Y%9$<_pS07z^yI_@X5~5E=Xj-uZLIvY@kzISq*_hY7flAPUCe8 z!P7SFauyj3#I|1X@0|7V#oSbIV-nu&MFfp{l*!%*PpM8`0OOkx4Hi%DC%}Ps$Eg)g z_7`^e)n)opB^UguwUwS!E?^JCEnGaBa$)%;U{+^0tCwMhUy5r1*N2_ z;A&>Bbphq>Wbi;`AiV4R75S_*F6DZyiJQq$*^?Nyai385Z9)RqVC}kB*4o^+J6gjl zbv^3~w=l1D!IG_ABdGq-LT}o*pQe)d{V_~TKv%#;x(03ci1Uv1T4bt6K^CP}#n`vd zn7_GBm?A>6DIy?O>p=SD78|t^<`1>TFd5x--T`852_An!L(hn*J^o=r%Kn@;ZC}9+ zCpmQbZ74Tv>0=nv7x9Fr*foYk$tq$!TnQZ#X)MV`jNrK#nv)qrdP zjU{eBEHwOb%=yLol}I->st{C{ZOe!Be75#d-XJ#XQ)v>?+J`KO>uN2jxE=p~%vgoFHL6cq;r=o+Rmqt;{Kpm==Yc;>bu0qLZgsRneXoE*cb-veq?CKSf%yt_p#sPYUUk z>*AM>-zjac@-beT!}~5ZI*mzR#T$ z9DRuY{H&`O_0B6DMXYhwX*a5dTVXFX_w6qGlpk=oMQ%4W&}exiRfP8xVN76Hh2F(D z=A31@v>Y+tkoPnP(hiz+v>U!L-V4zg03934+?M9OAM?1o+{Mb4FuT9{mn-WKAUM#A zS0ziI-opjHI5ybQ*)(n^M0VS|%S8 z(UX_{W~?g_#&5J>qM4A?i%lzZ} zs^?LXoAOh?sZA4IS7F9qHdOEAcP~;g6PW=g7qir_*gG>QSNmQ+e{{V=P8HrInAVyi z`k+eo71S(qdM%Za>EXk}oG@!G?&Z@swd8vTH_84>c{d3Y2U5)UzOo@JZZ#X!ZFY2F zw=OSx`K4aWv&=>_3j#T@FX!azjm#l4Pd(o;YjIQC*fP8$nz+ z4L`kWRCCi;e=~n2!Kk^3c9&_@l0)`FHh#~rsylnsaDt+jZd?+g(4KtH)Fhyw$<+i^ zjv>NZon3e~fSoER(&Q{@3o^iXCklQNqPA=DP;TvWSu62J?M{1Hx6izrI-A2 z*n2lsZkQ+41BiREXJHu|tsSrE7s%9Uf$->411DyC`I=z(|fYB!?H<%WBR+c0*M zuqNf`IKUpYtQ?I&B?7RY%H!5C_#p%j4*tNT{9SgWCl*>BvIoYU2qA^d9OiB)nP4V& zUpQ}1FnVW@C(2IjZ-t**Kxk1o`U0y0eEDnHLJPh2n!vY*x-LtZtiA>I8j^b)Ts(ae3HI?(7TcU5)e^Pum-wuaM-EQ&N$IO< z`KgB~2^MI)&Y&0H6CN0!t39Alx_5}QbZ-cB+15lq5<8k1S@yI}UI&W4N1CX`C&QxV zuiq9T(BF`}FK4Dt@ zLe0VE#%(e{@h?dK5B>qryAyOw-@s4HxQ+DG;mcCG zYL#7*Z`w+3di$QQ?VdMN6R`g|{d_V>Pni^_8Ej;dmk?M&{UL%6jI?Bfw?fzOzDY)> zRRJNp`31dQCN@}5MmWcedtj9ZSX|t>5(h{3Cvr{)re372Uf4Ze-YMcNO7B?@$k8{l zt$-R`rjPD9scZZ&r_j~Vxb2q_K8-yWdvtYhCIUXe{5$EBQGhY00*Rp;3HJ3jN+(-% zreUZ~fW-#>+QYuxu|<}#>S3vSddP9Dr1mINVm(9Kq}l#2wUu_JUNO#g{;Hbt}i zN@Z{MWFB|oxba8(j)ATZ*~dMDR788D18)&lRtxx&@Qy!EWl!ZFDz-Y);|*dyp3(N9 zT>yIXkGl^Prj6^e9lHq?=!ez&lp`&Z&5U@Zy;}1xT_!;lM6)y(W_Z&Y=AqrFK3HIIPi}e9aNHdUmS_6K!cPjT(oK9HVC=S~(2!d#nwT;H zM433WMp6V2RLhGr8(l+uVa3ir0BAkev zthOw5@S2pfXjI}gfgMfoaJ2s8q+}Wg?%>&0&x&!8{_*=A!}vo8Y-Kn78#3s}G7Vmy zvhKq1@Ufldgfadg*^O*3;u7b%7)hkfDP5EM`Z8~jxsBNU%n*G5-3bVa9uXWB?Gz{w zEV$s4-pOAbn88%gLH$hq{*d15RI*L8=%Zh#pEu*kT}F-FXe&rBBnLtn28dcKlLc{; zF7>-CmoxsW9)5j%qSpRiL(OC9#Kl2+7{Mz%BWlbTg6S7R6HQ@T&S)szV_BBRM@K4 zS$#doHxZ`Bt@5qyv*lDY z+pW2iXQj}c3w|8CVP{_VmbvB_>zBs0NAq^i+R+y1Be=wYvF+wLEwF1><|+-$w?_GZ zb}KirFE6{@5UuK*_W|eLnwS9O~Lm;oCvWBGl`6|0M=fb_uv&~1_R>D^v zuM`ohxd!c>t@M(2(D05EX9CX;4P}_b5Zq$RqK*0z^qQpi>hH)J@>}vy@-9Qe9BwJB zI7SUANovagQ0Xckj5^Zs{RZEV=R!#Al~^OqI%DtS&E2G$LRYTSzh(ZW4)-lJ*WZak z6{N9y_L#fhsuT)fwo zf>P`tZC<9&3{k@W&>@7_7ZQA?`@`THui{Kohl>Ma;_t-fy|IRl+AaISo7TQ)y5%_g|#}Ww=SS z(Y=|JO^}8@dN+rff8+Vil7OKG#clUYb2A6uHk#Vp|D>`Wsw_NuQCV~-GqPXz_2{g{ zjzreY-z?|95oyl_hTc40;J&@S%+0rNxPlepol`wZWmAk=YG($cneKR~>PIzQwq%XZuJc;?cTujC5RHT?uhxO=d6*zbvlV=)o+ z3!nTsdGwEBxhvaklTN=|%lAz3E01{h7maxVU7tA&`)oh=pl5LHL^P~90aUeD2f_hD z96z0}Usu?7&^NJ>slUt-fYq?f@=5`<^D}V5?vscN94{WQO(|EXLQ9Bji+Q#xkC#*~ zi*JErxACC=UBCTf$zT??D$CVTsinX#EFsw2qZis7NlRY=)Ga;|Ekj5rVY_i3CZjP^ z6mLsi-HdqikHFGTr$>6@a2_5j(|Cv}c&e|%l-}K@!I)Bu!jGOGDyAtM!6Y`yiq8Ad zWyYR(>3^EZ`sW_GF{I?R$e}7xGAM178LBF(=lv!gM-PZ~5iE!z-^1HR&8ebM3B0j$ zuPQ?VZT?S7U17*JqLL9jpWt`QrNHapZp~Jf(l{C?{t2rIUvP?%(YM`~Yca77Z?WFs zzwOmx@QDq_=rcdje+#z0V7~$;Hlx%54TKFh@_Q{Wr>)ed{kqmr4uyghqaAfa{-2-z zeXo8^1^-$o{9pC-fB)=XNPc0Oe}Db=wEUG#`75jX&tkfNpZ?{&|4XF*66yaFd%o6S b*gNrbXaCy-Z11Ig^dFGA&dowq>nHyMdZZ36 literal 0 HcmV?d00001 diff --git a/docs/img/structured-streaming.pptx b/docs/img/structured-streaming.pptx new file mode 100644 index 0000000000000000000000000000000000000000..cd15324863ac68f5db7b13f1b364931b437d1126 GIT binary patch literal 1105153 zcmeFYV~{3Ix2WB=ZBN^_ZQIkfZF}0bZQIkfIc?iMJgQ7z70X3;+TE0DusntvUx}3=jYy01f~E82|!EOUTy7$=Jq8Pubnh z*inbp&Dx3}9|VXz7Xaw{{QtN85AK19lyTVsdIXUt@pqwp8jFovFc#y!pF>hWyuBm| zOMh09d1&sh(nvs2Q8WpJM@i#(x?QJ8rquDWJ54B4{qN?E!(jV)q4|H4#(Ch>g}y~@ z5khlAWkDD-3tPcuXGO8Hg*-Vz^Aa>YB4!Y*{hadV_LqL1QJ+DPh#m)-n~82iy=lMz z-oAW#a<^g*7{)uqvts5BRF*~sJE~#UiI2?$EjeiCRv8Rxk{kx2n7AXaS>VyY@I@S> zmeU3)XMs!9i?b?DphJB?TMR0sUV091^f~jSOf=V&PyrEl6w2ux1qsmFxz;pl^wWym z;JhZGh!3jRmH;i5P{I^4+%*@Cw_BxF$_X(bE%c%L!-Y{}b}Q*tcu~)d@y@my|MV*r zjd{}UJw(65IUs8Z(nX7G#WI*+d4uz(H}aRcL1&CU=}r-10Q{aai1-KonCVAOU9Coj z0ykyrKwE%sjmPM#SjUy(#3Ty3XDtr%e4OM}4$9%Ww>ySJ;@D} z4#hR%JSCXV-4-cr=HwWSOh~zuhVz-e_E8vJ!u*~VA0fwM)x zORj>|=xZzzjGXCLUZHk2%SC!3q|IXE1d@S@jn$f0*DTR76Q$&FucV6PJy+`M$G|I^ z@Wc*i2a9(*wa4N(a1`r`E?li7XfHlwN%vsRNT(WFq7+uO0nXwPfBd4sWxz5@9|o|!JnlnXD96_|qF{L~?L%38d~xZjF9;-KTCcNaSBL>hLRGIN(G zFy9p;6-{PPpdC_r&73){w;dkX>T++~n^^RED%UCeloFTqF}Z{McP_&OhX>O8-9K#K zZbR^0+SxhLIa-+;89UPb%X|K*F#H`o|M8=!=`M@!&L(mdT1S zPvF-;3a|=7_-0W1O{9zk-Pwo3mN=td!%$TiBnmJfD%H?=lwkNBM;)jLXRVa23s9I&W(EnH zTb0}L4o7UOnvKcNXvUZeQ%%1T*E)wo*2P;&-;)$GLb!UEQLWuSho}n8ap=l0UHDz* zB$`2i8)!`UU34OsozCh}ICxLqI|L4BxdSvQpzeb=l#elRlE2KcURZNe5bwef8W0Q_ z5DoDJsmTPX1;gGL5UT{7a3{ofU2 zk@F*XR=%C4=l^$zVfs()D`UGXzK7_bH~uXFEBuow5yJpyn}EJpN_O~K@d~cEd3ep$ z{ViT7bFV){!Q)w*A10}rpIj5Wg2CPqSRSkXxn}VX_6yhUoZQv|c0*?NvT@ARgSGg0 z_%8U&Kdt4sUncSuh%8erCTk??(#Q$M6DgXbrGDsJ|HM=gy$dETlUe4eaG3Nn7(x-N z$iuOQ-)%H{h3Dcpg6;a@2#0b83q91(avvkbxg0XYsJKXQ(A3T85jP36aAB0;NqU#p zR6m9?;_*K3snO-w!?95Qxbtsq!hgx0rT#ll9fbt|K>6;Szv1Wq8+guBHf=KKQGQ>M zUEs#rl1V<3Du^#O{!CwxP?+cSgO&}nurkVEUTt*Ut^@WbNnZf}1GWC;81u^)V~?q4 zQ*B`lkx*(%aeqrXRz!U&GQO+m_<*AU*+?Q-!=J*GL37^qvaOQ z`bNtMus7r%q#rG+;1a11)qho~=3Dbr1&P#cl?JWSStd-*1YMSD1V#{rP+95mts znp4)~SAZY6iywIh73EL=ZovbH0QTiBko#_sfES;~gg-$}LYr{{Acxw!iRD#bU|T+; zGT{ky@k|L$UJPRt@$EKsU7fr0_C$I~!*v^U=2fcZa#ntv!5edMy!1f1U zMgsw2ZjsQ&O(k5}n6`Bv+d1K`oW?ve-4{XOZcdq{+O<+B#?@-1$TR&s^X}F12L9j0 zJ^Nd=jQQ^tYAQeg)c>&Y|41(XDgH?<&UG{#_gIm9w#z@@sd;Kd2POJZ$&5!I&y{6Z zm6}@~(?}`8u_G1&5-%huzr5`g@@b`R$uYGq(QgpofS%h|@!J7Eujl7Up70bkjE(KL z1n}QuQqb(vjE0Uo@2)z|Y*uc{IVt1G=_2nt?^voh`97YXdOqlRDsBgr+9FS>Ij^G5 zJUo`pRBqU_gcm~{($P+P3@FGHs%Q^NQ$~&zk~l{_SDxH$k&H!tAEdnP(}fOD%hHXz zu}Va)p%KkcIwJI!g9q0K z(d71*$1_nEwZg~ZO@vu~d(ph3Z>!N^y&FAQSJHXGAv32Z)>t@id9%CR5$@uP-h-{& zwp5M(5Bn_cu3>TvxU&9u985FN|{ZPck9EU-BRNbDR#8wmEhRz?4HwBd$f#J zVn>~VdamDXFE?#}w|*iGTkO0aW6nDG?|xJNGP}?F*;QrLhu@8vZm;{D#18{$rf0{# z1) zhp2Ln6qPVm%w@}<85o$q_!_1iDK30s9q*;Ph9&m(ta~Vb3g*mW8AAKfdk#$7R zk}^beM?xjpR_3>_wy4J)mm5ZXy3&O-xcF|~YtRGFodQWVW!s-9Nt$&ub-RAt(!IF1dk-!E}#VvN~cg>dd`F-gY6zC*5n|=C+ZZj8q5L_=QM^hAFt;>*tz^ z^^7I#rHe?<%w74F^y*d%g*y>DyqQ^&mpDegk;o`GA>s-g%}xM9r_k>T@O9pNnu8Y7 zk6()LvUi-xjP7@)8j+9NNMAV6t$8#5_1DW4rNzck%N%FGg*8LPsqM~r{S<0qkD5^rU? zGnKu5JxN8-+1agLJ<>s*&iSw(hwKA6*Sst){=rg`q<s;wVF z11SsAH)FvfUQvhkA)DCk)9E+{nac>SHM}X<4m7b^I(Uiy!Lv00`OWVU(eOQAO7N>(swPk#c7o+h*Dgmd$VdEjc^978=)&qSgmz=` zR>36PS}2*kL2q3ghIk7jrE1-OT3)|>6m53DvWQKUl<>xT2w)O+o}$9S!S}OP15~vK z6X-mr@Wby(IfRRIC%SxyRsB@G$sK8Q8Y+~8q}=&*x7gdhq<0w9@GyT{n_XS=7!Uv+ zP#YA$%__BagRoK+!OweXh(<06zT5x&^u*^&lQM9e}l76M#Op3FMOnNHhxd zdA3&p6bw0YGi`3&X{2>Czsvd2Tg7X4rKek0=y&`t%%)18IreI~ zHUNV3q)62A$iwbDh_e`N9n!(6KqDd*x+4pguC4!!N7!s5Zbyz=408>W{vstn zWtt`h3X~NG#XaH`)m6lrgKv0p6+{B|1W`k?yN6-&l)?on8ti%-;F=FB@xF%7J>#S& zHlF4`3$}yHnDw|1gVJdeB8eec`ib-_kZl4KaRuOuksu!Y1wYDCr{Exvc#}-7OuIAr zg={U;L=(CeS$PziymL#))o)NRu}{1Q7hXsR$Orx1hj0vd&;|gFCunVs{I5S$0uvc% z1Jt_kpl_t?Ctp-jbRI-fLI^KI-rKjkaPDg$JBWZCMD+Hlp@L7(On_sCtdSH^T5T*g zhXD)n>z4XLos{wCPSY1%yJU;caOY^mEnH;q2(Ei5SFK+Q=%fUA+YI?!A58L9URO6_9+RH`tIC5?&F1W}&06}axfn+x`Fa#t; z(>DafY5c9-PY5`pXQ8o6Lgzkpyq|h1Yto>%?hoyIaQ<(cbPNgpUpXifU4GFO7*ZO@q?6kd^ zp>9$T0D(F^gj-01uuE0`wE~?7Z;>q^oKQ~po^uL^>?^mbW4)pT;?VhjML;9}hF`Q_ ztw1aqh&doOAS;j+h!-)v26Q(feQYlq&@Px2$O^=>=+?6?%8K46H7_q3!8?Ky zqCR%Q`(%a!2=~4BN805FYr&t_U0iEXf5d{r0p9Tp1$x4fSM(ACCMbx$AQ5E*XGkAe zyH39xZ_)cB3_A=PL>ovZ&`)Y53h0=e`YV8dCJYPk4kDoOTcttR6^L3yh`E4Ca=&PR zzbbVjrdEP(e`~?Np+5=%fnAGu{ZGSf`wGM@4R+u7jd`!=x3URMwY}fk6WIL01~dTv zd%^#H0}S!+27BOt^-+yY!R50=+LKG*xj(PYdO9|nmgDvk zl^6JovvVb2}j7NwHz zr}c5`W6^h!6+^Ctn^%zi3N?=q@6F6E=0>psoo9ytj-f6F?#E!nz6UM_oQE4r0=@@h zZUCEPD|Z=!xLqK2g4>Q>*#y}@(?&!?dCBvqB8$>ASI=eH2PEQcwlf|%(mtQ*Q{-85 zgrJd(Mv1I8JQ?;$t##XErDP#)O2OK))?9&)$B}`(^F!=QWuo!wAB?~f4GWfrPOPH| zB3Q}>$UG4`rCChBM>y-oIvh;u$8|`Z-9n=Y-2JHOmFVF!A|`>Jmyd zK*~AX*l4fXQ$ zMK*2nCTVO&`f4?PJA~^Kb)oK2;1dP=c*A7jYJ=IbaM_RI6TSv3&mW#2ta+g*84e+E z5wHf@zydLUiszTQphF=;ykBQ9+%#8{&U)Pi((+!6EMet?srwkva2e`Rmsvbu? zC^q_qujD&cD^jzfqd+;B|m%4F7J7pc)+K1FW>Eh3xwN-#ct@`UXq<961vK!r-$yQ+RO6|y3_X(%HF9) zb<`CO>b_y)5^_64k@H*13OF1z4f&S__%B@&iKw_Oc(m7b{2(uAaEGhu1@4-gsq5Sf z*HZ`6v-c_Bz&t7%S!4cfEGmV)d9HE7EjkBpeBbZ*|9w^Vp+G`H5&-}}?gHT7&I0^1 z^LMVZ>afR(=ABdi0-n5ui_A5H>N=KuA+6>dkGK8;ke*L1OCB$z*zBqJ^3*ph(numz z*D2l~DKZBVcJOC+D=;z=-`69Pbjw>g(LOmFrWgLRH*$_1d+20y^v_eqI5F?_7^S$F zKzz~c=pRpIIG@+|Y2Wu$U&@3yGNqD(l0QcDW##YlH};sa`R4MUu?Gs2Uu;tm=JKQK zyO(atAn=UK<_<94Ie2n`?6@oUL!2iKH|%f)chs3O1&xepC<;i*_4@}TQ{$X1URfKr zM3ly?oDbS@yyNM?%cJ5S$uBa9BBzvHUu`2bcecTwPc={5P;_{sk63Qz^huM9?9E@b zH7D!$x_UTI@!N|jeI>u`8B}`eZMODC`B^_*AD+~CBJrr`nBfN=&Dme9`#N^^4g1!A zKY+(@;j5$w@P-eJ3nq6~YrPffi3pr=Zh^E&>_g6#oq5&k)1K96Jd2$arB$|Bkk2DP zE2g<9_S6sSjgRTNHxTh+zPrclP#7JVikTik;OsE*a_=6a;ONXnM(+F=#1ObnIc4_a zlWCYu!HU2AgoG)!S2r?dt@eRvC>xr$&*l*p$sWQ|>}_f7gpW@EG~vySg|dRClT!IQ zbvTVt5ozJss8OqNjmu5c&wGPqh@hCSntNmx^?O-yBG?tTGvm`XHOgplbjMTnH!q6@ z*Dq>!%)KqQL4G5CdsmHls5lpJPV*_kS?8$D#pk zC0MeBWxl1&UlBNwGEpTPY)FmcJr{n5e^cAp)`o=BAKtHz zA~EC+5q`uGIQN+b`(Z?6@bG|MD+1G_LSiT!?#I1H_XEF#QIgsWOwp`xZX_n5bjsS@ zOGM&$+0T#;*TqjW$A+-Q32s3LwUJ4BO>u}ALYh}LV# zD{=~~PRk+_1o9+Hpi2M|H!QbARXvR&@uNP8wQRX%0uuVXW8D%F^g9~g;4(eI6-~(V zNW~UCAIB-p(|+6>Gs-eooT`{sW6Im|Dwbi@Aj&6&;C(V=@@5Q(K5`qM@s0$ zo9C_fwBv>oyhy5Nt$0jlYM$YSRiaTx9yp87@_zAF{8+s7f{b{Ms=_#dM8grloM|K? zc3S--Jq4l!H;LC8+I+%l3hmBBD$O9W?Q_MTc1YW9guM>v`QWXJ-4DT#1_rcbrRCg= z1{(8Ex3WzW`5?@w#7;rnkdPM`KwgPh*c-aobG-d3UCRD(;_H3l8%~x^M2`sCdO4jz z@B#I%)XRH&1iBx%0JQBKK#Kt8kGeJE4*aSe>+dx?{z0FUd=UHn<#sdA0w)DZ0COh= z?}i0d*MfUcqoTk=Oa42VrFo8Xb8!zmE*Mxy1Huco`le`fdGRiAOh-iVdBppPq3Q?_ zMC0LlN6L8QZaM?eu1VGDtZQ$XosP_cejggM8ZrSqcZbx;IHm0JAW|aMqO#ar?dRSR z+WmlG#d_5rl^+<qS=Pe^2^V+?D+!la4 zcibPSWPnynlR6nJ9WAV9s;8Mn4av?fV~@EnW1ws^2VZ(DB8Ot&VKDMty!gl1nQpq) zZU9>~!;}kvpQZIRsobSP#57R-Ed~veiyaD$f?w7+14TT+pmR#i?EM=%RJ=v~@nSc-ypCppNpPW6vYF+dEBWuH??1Vw7E zwY#6DnNsVEl9EvXEaKc|5-*awbu3JAj7+X$jaEj+b)V$}UNo*LVqs$%itY^?6^u2U z?$w{3sxK_f0Tot@uP0;IjFsD>Bb7*E=SF@#Va#)L%h{iv!2&qJM$(<+4wYA))*!l$^rcD|w zXt-NMf+Q@rO()t(B@fUWvZtT{`qjfU-H^6gAvkwlBr|)v#izdAu!^Vjh^mV(@Kpm( z_&(G{>U|YNZulO_qkyXtl&rr+Y6OV!{7HiE@J$`=0My{5Wzvja7DFRrmi*ENn^&j} zqGf+5V+K_PQA(Q>dRu2mjeB`AVm|H6aewO6Tydj5EK3JcKpKArSd38Qk9mT0vkUVxeUCR-B3TR`RMHn zvLK5QN#jYYYq3J-WJH+w8lg&0+MW12zr`AWig-2MIhMzCi39noN#^}Cf_yER70O#0`_Bx&^!;=p`h7%`72dkur4|j zkOLIrA!bz&W@Tk`>;na-cWmOV1D_8ZZ%r>-;NoRHGReokxi$TFVO$cx=KsaY2mHv9U;lJ#5)JMid_c?8Ae{Zkn`zdhqLcM9*aT=V_W&sh4>HT!VSG zu*$xKe@nOJOJ21!rCtKZszi=KYvIL8n!{07Twi2YTuQy3ID@6+Fws780h-e)-2q{Z z2)+KyHI5CZkKUuy1@;Pzx)*3|vs9D;B}SuNKw@gaA*UVCN_?3*CBP#HH+acYF!C0P ztP^NvZ^Z~jB?>ZXp%^sL1jPv;G?C!}3L?-mocfgoYiSnkoyovEBpe+S(r!foO5+Fl zqLr`*HYwo)sPc`Glas9Es-wEwl$Cl3Qy@tW$fG`J`(YoH?=;X2x+Xp)(mv6+keK{h zR<3l1!=OL`0$=4cjm3b4>s|s{0gL-W7ObK#y$**|1<%#^*Hr%QTc54>7QLU_PvsSW z-HV9VPjG}#^(;i=(qkH&kt(1hihdpZIw0#6W!J%oNc>L$*>vZ+fz31`{Oevh@Nsyj zY;k1tp>kmB&ZsUma1df`!C9~!w-aOsj} zE`7YlL8n}&+>L`^3GgvB5_X3N?TkbFl@cbuZQ<~^uPPIfeT{9KyJQK(Qf>>NpX_m6 z=vm3`D5{Rqq6yy1>#go?8H4T@85PxCyGqPm(MFgNbi}1^?#$!m)X`|IfZzrgMW~FQ zwF&wtX}>8)GA>#fOR;Qi{m?o*v0L^?=R+m5MINgT=f`x^lBb^c&=pW+7Y@|X-X)io zO*FptTD7W*@D}}$%e8YFTaX5W9?ak<$F)4ER?pC%m0WR0t*7Z+iNQ}Be=l=b#oLX= zLb0C1WjKY(v=5MO|EV4AIyOdMwoYpFaY&wPE^LBqFlzQCq+DcJiL5#+Y)V*?)wCt3 z(izM|4UoQ|^U5kE6GsqeUB%+}a^CRv?cp(m>GK%xiuAY*;evkehK`O%UOk%iX+w5D{4(^3y=0}446#Mz~G zb>ANqm%SCi{+MLhx}-Jv;4+%D2w~QL+zeqRH*E*80t5t6jA5|%{)LD*^Bma(VHV9A z>PJK1CmB3#8e;B+4^lbjq<3NDNB~htc^y0F1PR{gyOyq8URN3j*cLdsP2agaW#MVg zCqA%aeX7TjjQuJzO6eWdTmt6i%5BRX!4No-S^+_C(oEbOs4gGj=jZ(A2i&6alSx6V zGVo}pJ8JYR@;oe-A)ab8>xYZZBuZBPYpM#JZ7B%N& z5X~pe;y4{#NXc;=%3=V#+=He;xj4HD_mATVqC!pIox*4>oJpvOR?#)uRkA>Uf;a%@ zEHtVUgTCK@E6gNrq+-8ZY7l?$sfrBA^Nkf@=|9vxqLxt&&jobOtCdx`<6#6}4};kx z>t{$<(#wj?vdf~foT4gdfCiuMFt`7LDx;BncoPMOsV-L2P+fI6%}f-us8>~cbS$4W z_Wz>$o_qK=lMgD4cnlLz008RY|95&xZ7< z`JDK0XGUWt_8)nb>V!GbQoNDDo%;y}V|!Qi3mUvIUVJ!HyOkKsS9ji9bieng4b+So zbLs?B1=+`c683avo2Yb=SFM#aMl@)m6>ML6&55nn@3YD++UjiD4=b0uJx8mxEW$4T z`toYs(M^{0fbAs8mZ*mbnG7V&=%&VVGBoSr)gals`^}J3rnky^bz?{n{#E!1zsU1Q z-Z{7-t?HU^vJC59O78(f^QYD$OpDYR*DY_}U7vT;d_R$Czi~+Xd<$A1y1G*U! z06~Z&TnxAZPB4SovKA#DWaj4#$sEStxbzO72qG^85@ioDgC@4C=;ySt#y9pp6)+{>tk!B092y=vzYhGhaj%hVY; zDo8L-4fDs##<`^yxn2Y0F2y*W@OHH$GCnPNs}p2Y+)$U566E+8t@jT)#hoq2cJCeI zJa)_C3u#FvU>2756Z}&e1j(Fu5lvqx^=Q;HBTtGeg%$}W;RgZ?BIlh4uAEJXGgkFS zt&&!ef29<~D>}AJ-pRmRjqnfvYNV|FNC`v;f@US=821H=9TsV%DT#m{xi4-PEKqzY=co|VaL6S?^Ez)~ zPnGRZ+|w}krThRdnQ6rFB7Dn)sJ%pWPebFR;xIZab1J$JeNO{o`*QE|S%Oj!!dT3C zC%1?|H9uTw4iG*Md{`JaS85i z#)WuOga84?4+MMif&o0S2tuLEMG>VE7AG!+^;ZgGQwmE^1g=e)uuZt1Np_<7#Usuc zN#h+EkC2=J*x04;;T>&me;|>pZ z+2@So5e6Y5x>PVi#&0>RnSvXOVAw7+rIT|J#;ax2(6Vmo+4d7l9|=^0 z2H~Wdl>kiy!9r}u4#6{M=c^&>Yl`1>AFW7IlZ#;C}J&2bn`)HHn4= zEyOegV^FAU(Zl9mtqAeg#T;h|p9@?|j~RC}Ts8k0kY!yTFsg-TJwreapkiGQ9FE*E zBp1-e4n>ZkKR{0U6cCc`#lvCrKO_Dr9St;3e!Gm)iu;pjBs7v-{)(aa?CeaR*@1x+ z{U_&ab}8;AoA^r0U8jVEPShyJG&tw9UWCQvu9Q~e8r}kfK~^9vZ84(!=IIfvji1*> zDdP{6GyxJ*yR1I%957e4Uu4(TpgL)r1D@Q3*LheyEIirtg(kft4{o|;zZ(+9T~hI@ zz^-HV>rXdJQ+WwN-ZI*73|0oEHYUQ=PL{x2{$M(b*~2x|n-mwMa{_u<(g>}OwU;4J zjkVicG?cfY*>lRWKn^0wPPSJZ;r=nPAmvwg@p_QA)M5Bi(t~sZ!mACKsE%yJ%HWG1ACbljK~Tcbh|EDd;~^G~z4OzX1`c_I5fQrB zo3PSX8HVTpd;d?cf)5h4M4O{+TyuiDhNeZj~Y zEKT8(j=|QBv^{7|_jbSq2h{^0#dgGhBMT%$$9MSDn9r$_BuN1 zYyvOlkxvT@*0b3{+NCmL{4WS*B5xjWys%Z+|gsHCuiQ6`lqfr2Rax#T~@#!7xkd>;Ab4{gPA)29zBDpVd}gZxpt8=whL+b z>9)*zcYKCGC`b>}Zx^~_{W$9ovW&Tfj;%|4<;#Scb>ldE{o z0E9%lgw?LV$tHOudZO!|cv{g@)gKZP`q(vb;AF<0>1P+pvjT5MOZue zZFXD2D=zjxgR29VVJJxyk;KHGhEzOD)^<`TH&3+Wh3Cwhbh9 zL47a|lxOwRQ1)USn~Bd$gGJF4)*#{RVc*MhB`_b{L0gnPj~> z&sLTFw8D`NJN!86^md`}nIkFpMwZ6sF0C#~h((D?m4t%+8_X>Qh{DXd5~6~0^7Ohp zY?C5h!W}lY6cV`;J>@bFE0rvgDnx@@Y4$J$Q;WT4o!HSawNCCBPqH)JxDC{bb~8os zXpK2D-gq;uo=vU(`uI4T)1+~ohsK!CC>v^QjRorqtUrtaYwiM68fOJROxKftfvIw} z!9HpDBA;vK34DA2@#iu9|69~1!@`6}gQva{Mg*I(as{eNG3TQ5t~82l!10}1~dA{GjTr8) z`$AHup-j_Db6=vr*o=$hUjK#Qrl)O@EA@IsF@7VsI7R8k*V>p9z?ZMiG})6LODf|; z*rdw?)J*7Dn&xZ?+2xg*kw1e;EwafnqI@WjJoR?QTzM+iJx(VsYP-Fc%tm!txJDhW zB&Lgar=h)SDjk>QBovF$jX&04X<*0TiQFYvzLc4Ua z)H?1ts|-A7Z#52P>FM^%!Q%PkHydbd#caiRcx$q`L~d1|tfjWvuniI_SO4sLSg zcYjnDW(3!_&!bKbt-P0FDJ)H62hVZ8AdMcYek6!!A{4v7J#lfhNgD3!J~(GS=BxJ| zSEr7!KV0T_-gl4mcg+4paE0roD=?YFI+nx7bby+ zbx=SWlYw0W%`!b8LX{BzjL}K>i*lWT06q#GY6;{{QNW^s2>1yKHo<5n>q{}_RvHyb z%+TCw@07)&d4xDx50Re9rj;C0;MDWGqLihG$W4}U;zlp_j(6H3vj0A-|Dp#iINZRz zVvRh3qbqRGA5ZH!e>P$mMRsHrFW9G)o9Og0J2;JAo0Cg;(9%slJRD4{$Twi6QX9<^ zG-$3r&u3=0l)m&3;7hoRa4ELFau&>nm;%PqB*aGwA=f~-6%Yq$R+4HZ0I`j|Iqh4T zHWPZLvHwa~NU$)oB(?QoC|kJiV71Dt`r%JOfxhqeq;E7n^G+(Hf!XSDPs{!`X%ea&o zJ{)M^MT6?PP&{`&MT8y9g1spQ9}yf7vcs%Lk#6~19QH-t)hTb1xWq^kSygJxA&-xW zj``yo!9^}R{#yPqgwEc2-pG{#(qv9{-Yn$czqb5-TJ9N5@dM+disqZM6cf2^Yqni{ zh?n8O8m=dg)O}FWj>@$6V5QAgikXiH+VC zWYu(D!@*7#zFT|AJ){}hq!!zs&WPDHk`@yb>gRy90KQBr{s+U=s?ag#dVp9gSpxItiT9$(+^(=`*$QOK{P{Jteg$NY)Z+93EW-txF{d3uG9c;U zF`xz>8Bbj^um;}Cd)p9^4Imr_st;ZnqSM$HpZ{8n0`!dd5J49%P3X|O+jT-aESBIQ z1HtfcD~ATspcuPf8c_qLiwz>jA*y7Zk&DLV0Cx~b_`bZ`1U83B5{(th01XPU6;?RW zFlSZ0i6_mJ@YH7jFZsvk81GThTp{9+q@Btn^lGP@neIXdPx))3*68Xd=45V{H72uX zBeuAaaZK{iUd(25{#3Ve(K@_5U5m_@cj|k2VE}RH#a^(hQ>mV;Snp~EklNw~KU<>Wh|ZK-lvYqNAvJbc;me*oR# z+1g3OJmoOW@_M`WafHItJdUrCYhON`qKt>4q(ELs74!DUNd}fIHd?sB1#x>oJ^3V* z5+yyqauDe*CMDmg-wC++NxzI&*30O`=w~`4i)YO&>tuc@TU{MfL7{nxh>CGS>LDWa z!+RUMuP2o++rs>0&y8BudE`du?kSCB92U7QW(y(op{VBQv1q7>F`K6!yol9vT}#=V zVlhxFl&U_~oywO7F-nIta>)on0gaK4UT4==URdLNj)glF(%j(ooWpim+oF97?<=XB zDlu3*%8-HN#X4`%(aq5W!y(`NO)jGi2k+fXwDbElei*B@Zm*N?(SC~yJxK_%RSLeI z1!-{1#}IS{cE!5E0(im&!K8!7<@=|T2h{hXFXoR6>aw6!SHSt3cM6W&C3FNA!0>?J zU*{8d^koax7P!ER=@*K&VvX(+D17jJ98dFl zCgA%opsXm^ZpQ!xgL9|E)mpm#6rJbp;Ed&9qX!^ip(pM#b^E7VUa3?OiD8^I3p91MHeZy_sr3A-F8<~Gx0olk}Is7;MHVM!{sj2-k?n)gMCg#;2@rzL|SwB)7hgj*) zY^>U`1G@pG&ceuD+_cU=kijdPh$r3bg}ppyGBJAVuShM25d~6#{QBDcCa@8-A$;(! z-}nz<>8S|XQTjK(MZT2(-y)MAK?f?0t@m4bl*e`fajd}+esL^v@m`x=8~tcM79c|N zSa;z6)Vfsc8^PU%sKvI51+@HTb-!ztAeC_+m5BRVmLSu=OTPP}JFe-BG8ZfW3qlqB z*^j@{PaRMXT|MQ`8PLuP@15H4^cZ!MJ8{RW6UEOT)l8~eDvWd7r#j*q-0qu1)AOyA zg`qN%VoK&~K}70Bw9?3&QhW5X6qu&&RM_4pmA@W8H;5CfakDCC&x{9ddMuUJjs1** z90Gzy0m32f+T;$I_!Wm0O8s-8f(A!`rG`1b3bKdg%o%R)!_&psdQV2ITxSbdQkt&q zQK?t%sR?nOE1L7G1CW{pO&MQ{#LvBT>EKh>d&=z7)*IWLn{3Q6gYSANlvRK;E3fPd zWGz-OZZ#vmsof0Dmz2lTGL5$tmnCg;o$`{`995@4-?Drv+v4J7rNqsm+s2TSwh_}L zv4KuIZSHOU2kwj2CoaASF8xs{7xRS7bI*%2oK2eET!BMItFbis{mv2=zp!OuN!WqKs`cl>bAUAC zp%W(KW)3UB<=P4QY%l!QO@3s!zE9N>+A_+LEw1>fU5QCaD|~0NJXmJIo1Fp2p|45w zrwTObOaMkPuUHtDex$rP5w=}d3Jx0YI-fJN_?&GOZrEfkb1er3HSi&{Xy$N-Bv&pk zTGa9yAR-F0ZtuOz8ekUGYMKkbsiHIf1^-Uao6W305ou}gPoXpwDF4zTa_IlX-aAHD z{%zZ$F)B&LwvCFNitUPR+h)bK?G@Xq*tTt(FaQ5}_ny7)*{|*U;e9&ywwAWCesir) zbImzsAH9#!rx`-K1edAQ(@gle_z5aqovvX37#&&S42~vt;}jvM`h~p#uQ%V z`Ro%0<9oHCwaw5z$1q?9`=I96S%!+)&i$Olu2~9szhVnf9|%K>ta~YEaU5Z#``w&m z9@R=eZ9QJqphOR9WdUm0E6ioQ?#n$H=>=DRooJ+o?hEU+qrttvqqQ=*qB3dGLq{)r zUQZrYpB;hF)*S&K)SnfmYjuFbEp^e`CP`c=bfQDjT-0UyeMAIHLR1g2hhkk$)OV>` zy$uzjDGmAy8DEmmA4EFCpR%e_p zKQgk&94g_jUO0jcu2;`q5V)@wB@>zIXgVx(8VAl;leZtm0bUGNYosF8Jg4Zb^QYh$ z-~?V{ujjMZqAqLu#G<7>kRp9Nm`&v}l}gXjGNRxfX^{D4 zi?*lmuT`(V{ans8&1_ahkUu^^-r)l~sdw;5l0`Cm<0{(0XIQJ_@vFc3ZfS*)Bwx&@ zKR1@c3rMK0QJWf5fDRf4$1a%iQ}^rMu8UWl4m8R0B*NcZ12lj^E)wUu(F_YM1<0+?@KNMFFjJ(PZ&d>ZBIh9n>Qv!6xAX ze3dtJ- z#ETV66Va?yaLwmAWRn3a9Ef?%c4GM&Rr*$d4l2Dc?r3v8M5D3WUg>)TXYnKK8m~PM zy49FW{KUL6K9c^GSn@$)LL2(p$faS2Bs++$ zQ091;6k+t?bQzFAlzHwT7vKa&Os z8;zV%W+=`S;++ZMk=xCjF@Dq830k>C{Eh)i&9ag-_T?%rZa;7@$ICchwC6<)I~^Uv z(Vd}p!^=izTM+t13Er2z`{*-%ghDvUhlzZU6?x39%fRGdw~SU#X45D{-}7??5FoO?K)16xcNp@_J_B?jb-r`qYHpW{ofel_eH5&f}Y0C-{y z@+?V=p31P)Y&MKRydHjCZP7c3T?_-HrL16GPpARNP@Vi&5{(N*wL86LS=dgd0=!!4 zZkxL z@Cy11NMOEy2WEm6h;aH7@Q4F)Kp2Qq{~iVG;Fzw1VLdICb@B8RG|bwDv?zWy7RD`` z*NX7n9Mei#DW*OMX7rRr3P)0;Ms`o4VSAamvuoBPxqPl=eV>{muOzn>eq$t&seWkY z=5v7_vzY1_voZr{xC96iubP&qO;$Ik`o1)wraR8#fgzQM1#BG3G)$fzAYL2iTPSOo0zD^6uRK zUS8($%Nq<(*7^PoAfZ1>^p-AXQ{vbrXG7`KA@}$wnAr|CXZI6PFxRh@B&_U$GlOUG z*7gXye3l>wbO9dow0Rk_A;78pz>set;9a$t-rp$DnTHwuz`z5bgxqCk8jFwjV}nZLWgbA3QCrtZpX?+usyNl*#U$kLpj3N%wozMOoYv#N=It~gao zx>{qUXrej%Rsw_91EqtaLL`v^lch32F3*m?51}IG(=VJjckZqGam6WrKgEe$=v!Oz zXg*wACfvJ7MK~whm9)oNdTudQr?g||VI(#hQ_DJIUyx-?JonfmHRO%07r|hTcD3Oj z$=vPMv2a7$MN@f$K?Ks^&7&$6-CtjTVex~O_P`8nH~1?0SO>MQondWzNpJ5Nc!mtL zIQNpztaw>+MQYNxoXtYS&3?D9;^Kk#$DpWY;FF)^J~BQXRVQ2rkPlr zmZDW@Wp$xFTB7qT6Qri6b*BMOP00D!Fe<=lRLD*E%86;7!T+6)0(Dzno+K{ zvBL7)78v$XT!l-p%!H!|!^z_4Gl4A$r7DQyxE>0pWh*6jRzJCz`iITM$Tb3*JBi;7 zf-4<@zV~pG-8_p1?o6w35}Qc;?w?a!AlP^DY5X;c1YBk{ceeiU|I;k6|KC~oS|0yo zto48FT44RZYvF4xaQxp{_?m@(^7H&Zc1Qe^ujl`UN-!cYs^)Lju4OsNcV=(&Q`?eR>Ft$3IX}_Nl2g0w&q<|Cl_6)pBDuM|Z(*6=u z`;_UP+|G_=zFi7W_Pv)Lb|i&_-6;>73rZ*H=y=;y){!6%1^K zJa+-gfP`ySC>B+K>LKXYx$wa56&IIa$QISM5>D?tjUAge6n*Hq$w4#;SUgShg@94} z!D;sv?gz>+t|mVn9D3{h!vPsm?6;{ORKDwuWwoQW&d1vQviA4OzRGIV;&7U0FJC$XW0)W z+K_HAN~QZ&)mJRdbBCmr`-_KMhSMvFjV37eHLH%hLsp zm_ry!SOea1o`ejnJS%ThPZ^ugZ97&DwCkGAqRwgD=qDkt-^+xJr2+4-fJR|32MwTKF);kNciujJ%JQH7A%NO~YMnLYv<)EnwsvQZ&~l6*qZd<M(e6$Ck~SWlrDWxl2t&54ptV`$AG^P$C&EnR}4V-vC%=O=&3la=yS@J?g8YRX=Q} zIBlzW-!?uVX(UB0h3R#b9(+iWcs)O!pl_W~8&a8zYvhSi*d$iSoY7n8kKD!rNT z9<+^VMHg*zNyv<#-65L4=S(B@f&OIpT3ABE`f<46-Lt;@f#b-nlCZ@*7GeTo;q|D| z^3aC5Lr8;&T6!bbr z4n*X>b-2F+Ob^Bz7$lNUE%Xw>F^&_gnqH?uQ}sGnaUcC&v^Gie#|3^(ncJwQ*Hi>F z{?pQBRS0A0;OI+eov}rL+-hAyMs5 zN`aG|G#D`ZYm$agB;jg2b9?spx-J-|=G(N`3_6Zkahq)eA)E}$)87PAlhB?BWM&A@ zvfy~Xvx4ISKtDT7HBb@JjV}|Nawidwuqx&Yd1)dSv>CU$E%+oSM)Z{#jUj9~r=gJ- z(ik!yu-SDT5k}P;nC#d|re&IO;hfjJ3twA|TU~!*1^$4*#{ol6@}g|^tqC}Vw^c;( zo&4?}9)}(XgTd9@1BCwkgOSo1K|98ivK^@Q5B9~jv@M9XhnFj`VK5ukm3~GF=>7dR zcnt1_E4&TP6QoTOdOr*SSNpIpdN*F@rZy72@7Yz6hig8AcjhbHPhRa~;10B<#5)sZ zZwi~Dt5J`{&(B|vIaL>mc}E>5))L1K0FPWtNtE4*g zm__8pl;-=({-sW*_wylCi+S3I6<{(8rnEJy0jZEI&d4l~T>QY0B88)eIWn_O4TZCA z3f^BlDMR}0cJfL@q=y21j=SB5{)L?DTc=ySiqV(fFpN{&ZJEb=8{DUm@E^%rNT8cm z@-~htMZTXedC_ARws7^GEV_gj4mnjCWE-&u=2rFvjF2aNQY=PZi-ki3Aak#L+%o_A z&IfS!Z;&2eXeAlu-$$yyGsJS`RqIuD1TUSO&!1nBN)3G!eU6fdLz^asl|R!Ph*%y` zBWy3O-lB9OaS0Etm)( zegCA&#j?~bxd>3$p7dR>CcuiE$>~Y(Sa)|CahF%BCy@wUj4*eV)d zoFgvG+}%76uf^yAZQ1`-=*dCc@LTUF6ZBGlT%yko6uD`R!T%?20*p1#Fs7*+0IG(k z^IdYdp6IluOhZ z0`=U`E`#>ul-x4>B5CmUbZ-qn7jJqef~xQC7XE%W4v)k9?M=SsALpkEj4Y9HC!XLV4Y zwbTUE0VMyJ+l1*gG^U@JIGlTU1K~FglPwIO0;NIrNF0*weK_T(cj7%|;p2>pFAFy4 zPo1G=^x9D}$3M3G<~L#cE`EPot#<-Bh5Xh;#k(;x?K+TT zp#a66A7Kg21-wD|4*Rs4RyI4mYm!&hCzASN__Wt@Rqu}Eb+jh)vZk#sh)2?hyjS=r zZ05U;-hBH5&`!6!x@s9?S+zCNMAKPh#u`>DzNcd-espMu^jc@j_#nD#mm#W`r71(U z`Er;KRu;r+D16BlrfG9_Ypecih9C_>&piT)oxxLTz)MP5U`g@G;+G~~2QyWoyp`(5 zvMxB+T`cAm9+q-!y^7(usrXczw8NuV@-;&1&ttGDS&3b`Y`=I`seB3vlhUpXI#F+C zX(GNh3{r1a@nkdRapO@}XEAlFREMjJInDR$_f0+@&rcg)O($s`uAXq)Lg__8Y+*mK zr12SwY-4XUGHO1x39^rVD%@9qT?UC8Ysb*;s7)gTOuSoe2;lUC?lp`-${PTUtxl0E zY&7w0+Ogk=px-2~Ag8huR#Z#4x&L85Rb-c4n*6!|2Ki~BX+i`0sbL?UR?()WeWR7c zY>g|qjS?E%Hknuyd9;JRWYXu$82`UEGt>cu!05g(hU(W1Z~rdO_&d+4PoB0w#+7+&an37~ZAc0{ATu*px{fW0 zHdE{{;p6zzX2V*-yaE~>91z6Xd}SH_3v)Ew{6{?>w^YVbV_=_B6eVD2Y6D5HtK;px ziS7R9)Ek7&vV?k=d?+y9iX`Q!-)WeoxC9NIsx;NT<%DZ!;matx^~5!KKV2yd4}1$L z9aWpwWVG7k!7UN7K~jZ;!Z1WHbicwfeUdOOr@9wx#eon-gqoLW+}gs3hMJg!<*3xI z1q;xb(`u2 z;}2zvSv!1M9Y-zhr7^cs4OA!#rusx}jrY@&AGNW{3$VuTrj6|$#@3DP-o}lqR$CWV zWgX@b9w*<=*33YFowc1F!d`-m@P8})*pGJZva?~!zgBCyuZ?skYsqa;Og}7|UA?tr z_z49%X|V?i(nN^0u6AYW9>W8Zm@7W^KW(X(<@R;hAJIKXoFSKI^^8LB;1 zQ$or|n!mbiZurQO&FOT;*d~Dg+Rc=>!=g@}tLgaihxxlf`;Iwc{|ivu$TNcyWOV|u zglz+|tag-XtXDlrotU=RDK1)^?PvjjvofsYc;@U2NRM3p7+lHlvNw>odb@aIA#_!9 z>_YZxO=I!sbvU9^t`l)-2vxo%dH34mpN!U-!p~udh^cwQc<-Wi4}Fg05Ef1Z=h{&zJs^9UqnrVkWcsL@5tK{>yM=!La;og~UiT=zk zSE+m5T>{Z?ZXS%%E0<=Etnj17zPE%jaFCNVu+SZidRUU*`iw{jwuG}>wwqko%AUFL z{rV2qvO)gTui@DxtS4%!tfOyJRrK!sDt7~_NT*pl5j2?NiHHZJ4(8XRg;qO_eU2Nb z-DJL-6~4xc=%WeG2EE)WIy5WcJ_r&#DI!xX5};cVUqAr~;rCxa0m@(A7FSrO^)g$t zg(^HwDy%;&zAcg2A3q<>UhfQKjyeyPy^Bn*csTG!=J(e!utPbcO8d2;o5ybs4T8mE z@Eg)q@(1mR{B0_AA0!RdD*I>hn!R=HWoDIAlR-spxmj#x!2Sq%U>v)yXB-UQbKc?VHmE<9KIk+gbp5$sUm+1U`*tw| zJAVSPcp%0n{JOCYMgRFP@8@9XB`}2l`ox&we+f4KgF^h9)c6PT@JwCHdY2W!n|_K5 z>cn8X;e!B#O8+XIqnUZRcZR=#&v*2dhN-XjE*G{Ez-l zpp%lVFbejI?eusz-Yl6(96Y5r`~LQ&iBKaUUk$KsYHZ3H?+8{6IdCOCO-)jq{?Ge| z<#Ln5Qa&G2#WHG|e#zXEkYLC8#+>@SYKW}16AcpVp5)n=v1=`R9uW$K>y9VL^e zOn$`e8kx{j&{SH%@!(G?l-=J!|5SPYj=YFmG-(unQBkasleLWoMFR<6JgEa1-brw_ zpdDbF@qQkEx8(J4fNZsZJZgkcs+$>D&yvwCc9544$z&3-=d4l%uJNZ6C~xQ}l?A-j z+PQ)|C{|7>Y7yVP&DUjd-Lyh9D3r3WuZKoP>bba+!Z7J#6l0Us;P7%fshMvtq;QBP55fQ0Yn`vOEY z{}U-#Ffjq$L?|ZP#*~##U{f_aC;cM2*cMkdtlXI$d=gP+guin5yjUnP8{E#`NA+(W zrSVhm{z$A0b)MQ2{kLoc%|0(|xiNLEo!Ac-Wl-8k0HF=4}I;h zV2eSlTd4D0bIQH9o|F8+GWuc{bDxWhC`*}agq9AW9Ko!#+24W79l@Femge@@W^Yw3 znSs?zeTkVUhioz3qmD3E6 z8n{>n?j>&E564N;iW^MUX@s(<`m7!uuf$fH!A_qi`C+UpD(fP=$C3!rV-)cpx?((M z%47V_%*r0XrH}d}zZIRwX7*BSVE{rTodm;4XCle(qi#-x7l$IwHq`1}D;`q1g3aq9 z(Z@h2CqZ*2%7i!l|p)d0zGf(Uwz|aMj4eqqtER|{o>vjhkiQf7}veD@b zDDvQFzt`(OI_fJs%-!Qxhg%7kJAeIcPJVU8!`XbUTk>E9)Je*E6dX?1ZdRn;h0&3$i^7o^aD@^`Rw~Ox8jhmt&#`b&lb|W?WrQ|av zn=qR9;{0*K_6rqznf@2C?=~4`&f{m+I)T;%tA~%1y?2tg*XyE&?d5z=CWmTsoZDO3 zxF($pN`svN0nWIUZ!|+wFJJF6^lfQ**-anqdOi+3Y^S2UI8r<)W*)74P zV|X+1GxL%6*I$idv0-`v5!0Q}bU3pF%z9=b_^$_p!$aSs^N~`D*i91d#!*Yh9+UY# z?<&uf+}s0HEa_l}bClp8$5RoECK~~UBHuK%s^jFItwvn^etYgkJC9t)8=|+s;X*Eh zPn0Z=9cmra9WzS~#gN|m%gGC}57)%psMM90tE(^nR<>6VkI3qvj#Euk(75g+8y!Y& z^LYBhe5czPQ|0GIc`rNQW-g~Wgbu2lkYa+G|01a*!%kQxir#f&uessCv+Zm*8A+q$ zYn*4shM%VQ5=-YexRL&`FAH z2O1#bC|Kd$^ha~OP?XXbg1HyYXbNQ?0eMQLG%X#ZTaqUu7?W5U^e$*6*mjVzqDYDm z6=5KnFfp|B7eep4jJ42{q&1_?MSEGLvtOiBXl$Oe6ELQcEd;y5+4yq5-WM1dC}xMt zLGWE6#jX<12It3^lYz;_2BvjD&&Et4V`dyf3>8&SlFSyBlx zW@9ukTc^ONWL@$Rr?IZsc9^T=AZ_Mr3wrkkR92H>~4r!1BvQFw(b}^hf}F`|n2^ zL2zpzM^(q(lh`)Ze)H-NG}ge}q;1u5eHPXF9svv@t7?OquzB48TFb4LOde+_fGQm8 znl5Tx+vs;)R6vi2hUy+9d5cud$%nGkQA811Ea*b%X&-EHuPS-7Ir$&l`2Kd*a5xN9 zh0{!-qPl_WP^o3Jpj)%}ffE!5L#%+#U9O&u?Yj5@O%(S|9Iut-TVkGuu7bt`>Li@_ z!0EZ6VgEhctCf$cZdqf@OBoxPl!-gUbcY|O5s%v8lSPonhm|02=l~$vYqv5WR5724 zAITbJp@pq|?$R`b2tPB!bt<6+&`V4`0MHxe{!M9HKAOlmHuGRFP`njnOVUQ1vUr&z z+9fcJ3d9cOeS@kEBG=)J@AUF z&^KG}?7**!ry@!z)*d8mzgJ0t^ZhFKyVTrjA7DCc5Svps%rK$+VSU*DCrit5gL|G% zDNEvxf7bIbx;LMd_9ub>3ZVtYcU+hty!svL!MYtFI7HZ8(<6~nX${Kp(uB%W?JsUJ z`!8Os_QnA-UUDdV8k)qS#e9WWHV z)*Xq(CceC)ED6BjQ+^p`J;b+lYnn zxT+gE3Q}+1B^J0mZF)F!%^y|w$@tfY;lfHWl=R9hdackz(EV6f03*_ba57#bX|X(l zi2D!pxQ%bR0BY%B(tHhgHN}>o{wXeyhyIMDkZmlq%1G2&V`I@!3-OTDntK(>gB=O= z;PCrIAtf`A`Y7!w>w4pd#ENqg*M0B9BG-1cp_Pan@D(X;vE>DB&^i%TEYxxR8XelC z-r$Z5bS~-8rMdnJ|UzOKxkl z8PCc*=&IM;k?rm;;Ml|DYxNnF-D%oT*3kEI)4)QzQnETpr}!`hw^lmIKpS)M-^t4W zcJR!i#MI0jqY}xG)EI1G2nysyH6ks=dPk^ft{+umu~C16pCdrX^nSu%auF7*B0CD( z@h4>@`4XH6%qoEHXWP;;Xt~-F?D1GKZiEiNG{Xn=S22&f%d>{O#nV8)bIa)apnQMv zuG);gr5KB!?b(`MuSP04DI=>pu_hhfm+-)sR>BhcLm>5t2@JG;NMgs4YG-Y}ZKS+I zS6e$(D?S>Hnq=!g8gYA0z_d;^;}SMzriA2JQ)Z)c9Osw|Zn<-hb1h} zDe-=6yK5)`zo1iL3bo1pk$( za1~R1_*;nKGDYN#-hqi8<8A!0=~njGOef#zpm?0gpLlLwFJ><)se#`ljrU&ETC=e6 zY*uxZ--c0jjspA_tv=zvdc|vY-#0e*qNJuf+b}?XABnh|87! z=iUNysyMP(X4HpF!bJz4=_`rxy-_@w+}ngF~?e?J-I-FFS{I-fx79b$g&e5c=I zM@?xSJgY{3R<~@01O&Mt$GfGnrGELzqt?!kwBX0|XHvlf&KE@~bg4Pm97>BJ-~u}o zPD+UzhQ+WqAc#%cg@$p8yY7wVIgH->q-~AfTPlgW+TZN`nCrk)C8zCKp*od%==1)SF89I}@BA zT$@Qje>)J;x6yH!Ue%5qcLXTSI8WI9%5?iQv5Mm)gs{30-7_Z-bgMLw)2y)rZ9 zAWMpFDA7@Hq@&evlnR11-}(?wz`!R*)NIuTF*@aU~*L>%QM6T z4m#pcqc7bZi7Pq%-~pWe`iHBZ8dT|K!N8UGUJ=|ZMc53@y>WormT&;c&5d}Bs~cZt z;WWXdv7NK2dTFd;$wf!U%hT2C?Czgz;=3}nGu1#$Cqktr2i80Y0=KNfX~!53x0JCC z!=(F;VeWIcTkQ;zSx<)4fx}1l_MqWvgeBM$`a;}my&Ll(HYdUq!$eL>)!B74UK76R zt#Vd$5e!FS@9>bMV*Ft73~)7aP{xBEIqfU_4IA zivI4~Uo|AC$|{;MSR1s><<&u=&s&K|K452IZv0iruBb%~vlGe1V~643v_paDfUvn% z;MhPtxDNm+YMCH%aSpUNO83%owAPFU@HEH|cNrGq{MHN`9K-W{tc|A$7CE^w)lhTI zp6*Ru37D~%2GMSewD`zIV~H_R!XlL_9q}K?k24eAb)Cr}Ne~sncI^+s7RLCe4dPSwYB- z+c+^hm%XPrDmzK&m#AeNV8*VdJ8NppJ?Y9c8q)QC_6W%+2^J*gCM-pbb&uB+{AyT2 z@7l=jDSh4ud{2ZBy6F@;p`y8dSBAt_m~pk>bB5( zi_9>7e->BwGISYpfoD_JU$35&<3X>7xHT6TokozF6l_&X|EMK919PRm$Go~FO)Sn2 zHp!EHmN?lTzM5{iar6IV%fp z`HZ*J>Ap6}d-~F<=!a&zara{{qjHDVI9y=^|&QU2W0_+hX3DLFp^$$MX>VSE<0q5RfCzV_aM8KWP} z&yDQBcdZ#JWH-2t%rwAem-p-L>!J@`450DYI>wf0jcWa2NhkSCC{V=pR>dq+@@TwKACh{jy zBJkpemGj4y@Eff3cyj42^x#}+(8-sDh6N4cx3NBW+W4xPW{p*n(lZXWMnRBHm=&Dm z^pvHxugh?KfdAFeIrOm#mi`qEzD)H0eK`2L*z-#_kNT?c@uH9MAxJQ(C=cI<*@08O zX{D$hrZX?Tj1Z~_kt6y-rkMmky#enTo?p7TsR=#sS*1>2cu=s(1njWet@A2FJ)wg9 z#zgy-{ubd3T6@9y``Bx>`jY`XW#lRn(?(os6Y+?L$MZ=On|_(9TY!sa1)^@fd6uxu2T=V*|tuk6>4fW<&Hvsu$q zO?sLX9m)*rjdO+tCEu?49g!IWZ~txmhh|J#9a4$VA;O~AJ|P_?jz8Th&ytUas%O~F z0TwWdKisj*K=BI<@l??HZ*M*B2r5>b6Mk;aaeN!IOisN~QcCs=3lY{pTwohwdyu!X zq8rz{C!srV9Z4;Gxa^E?vH+9mNiQTM7o-E#cu2_ZZ#Ofbass^o`m0~M$}Mx37oBx$ ztf<_`1L0L>X>fU#D9DvHy`iTV+LXYE!QbBuy5# zDLCkf%pn8(DR@L6U=#T9x#7e94i4XZqQ3TavPj`fEzIbq1&tmgdg`YG^*a&cp%RHx z7!4*Zw0oB|FYkvyf39griF!kXmC)c?@c$xHe0Cn3F=WxjNEED~>4i9<8y39?() zT3K}?ZX3V#nwFQ{SH7w*UHuWZ#T;A3>`PbAcia@T5buelNiUG+op_^>Li1+Kj(7tI zbKarSy2nLh&BiK@^*XWdEb()F?Qp24@9E;? z>i;9-EFF~+O86q@D3Ja=D*n5Frak5Kuln7=F8C!bs1*}oA}6*qlvrGN&>51FSx8+H zPPDRBij^8Rlp_nP=vne(LSZ$9UFIy}vvA?zV7@GvXY_uT%OS4tK|TTAidwtIpJStH5hfP~g@J{xzoOOtLkWn#oNbJm{?L4>Kd z=hHn=*+~)*m8%r7;2YZ!On0pXKCNu#*)`8%Ho$rXbc~MR);@$(LBPS?ENReb{L4e) zSt317BBnrEYn@3#7_jsBR9@kkVE!Ogt;lc z(y5J>R`T_sI<-WIXy9gGQk%p|zPGKORoF~*`# zk14K~4Z%uBRN_^0Y4!@8>kMr(ZZwd-gtnGY_Qp~q!oNVXu+^7g8BoUje& z<m!WPrYffxPhKTn>$7{xuBCnUjO7lK?Cyhr)0M%(KW>L*Z|)8b2`q z-Q?3B;k1Sx=K-UaFsYR8k4CIzDYKBoOesCQWlN_zJbgRQ84w|Q`KJn8Y`BdO9De}I zr!yQsXe2hj;M5WpBUn@pq2LzOKfdWKCj^0qc!e9oe&o={9I=w&Svt(f6SW42GW7;y zW17J7NIWsI2uKfTClGU-$O;Q?47Jm3!w3moHysM2MR{6kihO4*YBazmywUn&ujLUo zJ=b#cPJjL=1(JKiTzq?n=xozuK}YQ=#lejXNp{m(9ss)AMcm#bf1x65J-?xz zNNOUj_MwRkL0q5cW>ojAL&an*^2SOWP9t67TsML5JrvF+qP?YoHwCG9JHmBWARf*g zQG@P2K$D^d@g2ufC^^YMsFUs17U4G&*P85#x{$W7%izSkjn3!u`Imq%ngY0)@g59# z&|K7PsArKRjG_3(!wpA7EHJTVYn*GNIl=Q)IM}!=mA_?guH%Jx2t1V97cmkL#25Dv z0>Ux*o$`j+*hq`+Sxxicr2bF&2>D1EVAGC%Q@a78OP8r3U za3|PefH&sEnV4Lh4%LYYU&-vc!D0zFkyA&pszg;qdkc-;M zoZS3_Py79CHrRbC<_wMYI0D;0-yGs-W{jf>vT?nDgHGOo-$FC=vnU}kBwXXc(gfBE zb&a$pcNfe#G;HK-rH=Rf*ipT}RcKZ%lzJnMK?(tV*XAbAnptB#xWc#@k-6;tA?B$T zZF#1c4-dU4SJ#pb59n6HFiS?v_*V%Ed{F(4f6tTp)fz^gIwPf&A|GRf(XVkIqx%AH z-Lm8LH$O~avQo}(vlkynx+jfkfbI>mk8(j~RVg*fsyw2Bof(u~+Wjip>}_zinFqaA z@3e!*FJaHS4$|uM`&p!nk@ESLb`hu(XEZqZ5)h#L&NcldsYnMaKaU7M)jNIT+w=iV zmD5IIwq)7&#g9_|d))_RBKt1|^jBCG{x|6+i=6?wKDqc+fflzDA0kGfy+Q$dkVygO3nR7x=^Z{aHe^mOJSGDLlCrQ%bL21 zI8UXPSupZy6{|&CoAm$>R>)pVf*jy`D6VZ7&Qj2|!W*JFHH<5o3;Ax<0*W9|>KC^f zF2H7r=(WcyXn@+Atcqc!&+;?y0ucA9mM}|ALFOF8hlUNR6Pk+l5T&H^mKCMW>u1iTWEY59GPV+cF#^{pad`KJ z#;6>hF_?wyb)GmcDHAD67q8vzNLNEesM>@k>QCBr%EET;3 zH(izq?B-;%LUS~Z9~?0@*lAffaQi17qN(&if{b=5we2eQgeEuESD4XMm`~Skcc?9{qB5$mEl1mhSuian6B&X>En`XOv!1Q9H!U98hb-{AOe*rmLwFEJpzdu zcw^~C#LiSJ8gvuuwek=y{>c4k!!w>7r5n2d7LXsk$>?PY(vChzW)#b~GDX=XX11K! zwZ!rsrX~&J1O}RMF=ZIF2PgvtMG*CVO)Ly5sxq%ucnp=&(S&{tJb7(NBQ9i|%XR;S zKJUOLynv9}%4^^H^3`|sUqj$O_UQdn59I%lTy^Y?EFJ#QM&a+?EV{o&|NHc&C0Vsg zf1+LG>vX(0eQUVdd zIg>Zos|-=!PKowmu4T~MSKcADkuIMkKnS3z6ou6m@1iVa(`wR^C7i&C>G{>_#hkv4 z;1}Lo=%C4gkNg}Dt<4-5nEC&(_a)%A6xG$7d(WND!^fb4Xh3m*MDsg=agN4c37VJ~ zah5otL{!x1dolhGqc|lFG0sLYh{{u+#(BmeY8-Hiphi?gZ~$io6=(2YyKh(D>h7wp z-RD&I>C?UH`|9>N)pd4P?Y;J1Rn^^f4}8<}9{9?)oc;d4Irdl2Uwi7we{uA0p7X=Q zUv~nZ{F+nKm6v$|Ksm>eeHg~J!Y?~JpD1JU+2Jsf3VN4|9;+oedJ-M zz3riw-1_^^zRwkw-~8)G{pfitgs)Pr`do&Wc1*Lw4D7hL%d553tbKRNAh|LeTT>(74Ep-0^5kb8aO*5{wH<7Lu}`MdRp zTyoA;j{p6aPCn}jU%&j~i~jBW^WT5jCFkDeiBCB37r|{J<+dc88_6KJrFqK4|+9_xkd^&N}X(+Z=q^qc=9*alzBieD=4GIBfSv zR^R`M|GMG&EAapQzPxm+JKlBqACLN@gYJ6#!C$!J*)KWdX}5aTkN)HB$A040Ctv*e zFFo!pFZ#~kfAG1FJ>=YDS2u3_&_6xw#uwi3sDt)D?LB|~o%MTv=G5aJ@zG27{^w^u z<`!4H;O49U_g~(4=zBkO%yEBm;S;#IL^pmN$Lv z^j&}b<0Jm^%=g}F{~e!t(Yqdd^Y=gWnpeB~9S(Zq)vt8GGyeEZXB_e98$IRqdtc?s z_kZB?UViwa4}ISWuRG*HAAHQgkA2H&$NqTx>%a2-`~K+qkN*YEqjd%p8txBlaMjwk2-^3YfAJpTC)z4y^OUjOl{ z9sRoJ9sJZgeC{K!y6=0=|Mczt_8#|s-VgTq&^I4-h4Zg>&?kSry7A0s-s(laxcfis zbL#h=@Zlr>`Mh`R{pNc<`qqE>nk(%8)g4FN5gYBEyx&(J`KOP5)2>rqc-5aj^DaO9 z{C9Ue>RRVK?Oz^o!ty6SbJh1e|KV@C@Gow9=&?WlkC)!+z)wBsjOSkdkFPxCxgWjv z`5Pa*?peRT!@Uo9_Otf;_8m8ldH1I-dF{QfvAX_O7vev2-}Y@+c=u6nKjzt&Tr9RG~lz3u1=9`@&Ff9(BFeZx;* z_MAW2{;?gq?sohWZu0LZ9DD8WJ>_xdo${~my8UB6w|dtfzU&_^d(Q{%e81z~dG-^Y za`!(v@7(*G^M-dk^8D*x@{OxK`KS}W@X|kf;{zV?%6C2LCpWv`k+;6dFMqV( zKY#rS@A|tF-}+~F`_7&2e(vLM{%5bf!M8ti^-q55VShEg!7G0M<_nLw-CZtw@kGl9_SAO`- zfBOCVJnf#B+~RWU;a~ilup}M2*Lm02b^BZFyYF6-ubi^Ri4S@32@kx%2`4`A?;dia z>+d+>_~Y+<=e@4G*S>q)BrU=4NwEMH84V2G8wRX_ApqCD!)0}W>} z@j|>ff#E46UWgYDG@Qj<$BVPk?m~2BG)7*rGM-qPdhSno&?oih2AiXlcp+XKu<#ZU zFT{%%9F7x1!;3=&-XhV>%F@!(s>$*lwWnS8r#$M<3pU3I;)Qr|z`|QZybv#5a5zp3 z4KEHAc#B5h{59ltE8`)=>A9csmZ(23+)5{iKlmU0t%M6-0r5h-_^{zB9~xdGhd_xHB) zHYqpyt8=8hM32;;5;VjM@uGxEQjK^aUWk`U4ZuZ`XZPjZy-E%@zBC?>x0@_ufY9%J zS5M{bZDh%j@_gDVms5YlOBogZ65@q;@uQ=BhIr{WUSM&UID2O=#Q!63v#j|WzTQXp zS};;ZPPjyVm-4>IF`4qhs!M86e+g!Y7vjaK0s061!wC#e;n45`HOTV~^6dU9`Y!M7 z%icP^`~S(a5a9k~-9P1R#$FGWip4|27W>QCTl9gd@5Br7QiYG=IpSq@yrj?C%k%p- z{})v)-_1B-!|=bX0f-G;l|yB;-R!pGx(U~vLVjk>)8FWC1`C=a@zQ_1fWIf;eR*%s z@W1Re=SqM>vbRt2o?puUGd-(D?jm=&faWS=Eb4zOVmx3xaQVKgjPYRhcwji(@W1do zbnRUC$GNer52WJ+ng7r9jv9TPzV7mCR~dbMc3-Ez(cfIY zd*{4g%8_jT_l^@n_M`2h+0MKp!S^mgptH^sf5e{!hu}xzg?I^q#Hz4hcrpAxK^J`7 zFXX?3obW%6-unZV8k`>&OgDyK`Cfz5n&m>;m$9;p3V#XlLcI9VQ9eVwM8}KpfAIM| z(L1v6|J473mbQy-JI?#dA;Mci{1Ja%a5zp7FT{%j7T%&o#Eabjmv(zQm7b6~$W{9R zREk(c-N^lGjc!~WXMY(xT)6O+5ii7x4;!v>;)Qr|;lfuI0WaQr1HNmT@(9;9qLL%! z5&Jnn1ilb2#7h7oxzof8@sa~X;LFhP5>Q9(^w4y}{xWvt93b$8cp+W_5Xqe;UWk_* zAOc@v<0Wvp-09fYraWRl2e7~w;)Qq#KqPmXcp+YLfCzjU8eRhG$ekXVZrESOj+_Go zz7Q|OO8_Fd)5Ht$k^@BGOKiLZE|)tU``VO8?B@U$_(Hr8F9C?;P7^Q0OAZi$FGIsi zKpnZ$L(>iW%h-`~fWQ~xg?I@-BzKy4AzpHT2z-f+m%!z6r(<86@`(K$zye>07vd!V zk=$wGg?PyUBJgEscuCcvdhc&&x?z7AI|AV%cba%1UUGm4d?8+lmjFa^r(@#K@XTvJ>NBvbgSm7k`LcA0pQTcUfc!4rT_#eP)$gAkP zi~*|l`=P@F(JhcnPCqs|N8hJ6?p((`@tqmXfzV znQ_8~;eQzehz+)?=s(eRv)hjM5ct2Ev>r!)qrYjfBK3%u{^JGwJpuP`So|;l=hu}0 zhtWz}&+vcn{|CDSNg1)#e{6AV!m)|Z|BK~}2aE?rOjLa*US`KjI)~q8_+R)PI(NSJ zhjU}pTebLqA~vY9k?$OBH@oe~t>jjpe-+E=Z}hh!CaS&@Fa5`htlOvA@c$Yx;p2XJ z9#H1~hX2v z?;R&p*-g&TcF}Cd{T4!CbDSssh(8A`yhX$d@!|!C<+(ub zIKgi@&ik<*KJUGK7M#-e%)&-vgAjf3XwnRhgs? za@BqS!QX=u%pO6US66DQe6ofwiH(Hc?~eRL68ZJW&qh8TcE1rYBKMAKCH#(KiTREtrkeAMk!ice z=KaF;*Fe56@{5r_jr>dGgyVfj#@->sUXJYq=RO~(DyM%m?jOH}{=a~H8uCMsWzE1l z2PJRAB=Sde{+Q_=w6TlkNZUoT-3`I{CnJ9vYs{BR42Aw{XU1N}-a5D;UWk`D;^lvV zzkfl@yA|@j$h>E)AziBu^cVG>i6uJtG4{0wDDPlAn{g6w|2gFJ9zE|b19+-&5Ac%Q zkKfWAW4Ts$Qjhv0Ub=&q<*&hypM|;4k-`ILSM4-F{vh{zP|{&6$o>4kMEl+btY1d% z;yMTL5FGCSFC6Re*~=hK43-ugFCRfa{uX%yS(Q2*@00d-y?rVrr@QnkGih) zte-sJ`Z8p}F6T9TzCj(si5K%M4ClcH3!39V#)@x-F0PNvcg|MfnHyL{A=Va<`2X-+23rOL+-NiANoxHAYMWkvCji9*l%e0 z2=dW%&q_jKLYnp(z(Ck zf9iS*NpkN*_QgIAx#Y9ZFJn1lc?lOTtVkL0!g;U{8?JKVMes8I5p;6`vb}TbS@AOT^avwHa<-`l|;=+Zm zjCd*G)_+S3duPdSYXeF&@_&@mngI$88tY^BuD!OzAvNz77zl`Nv^Y`MTh#lgE+*QOx z)pz1Wg%|nXL(a8;p79a7S)HW6=^TEW;eV8^;8)Idf1Dem-m1m_6OmDU757>BU5}ov zko){UME1sZ^q1UE?)T!Oh#lfZ70bQ)r2eSCqEA`he}&H{B4<3;Gd@Dss+06r*6q`5 z_+QpWGM(5GP+y)0l)1m*|8v?;R(Eu5Mj>(f12D7=8UDa%Egc zf64vi{z{D47$RP#j^&AtsXyw^_NUNi;N|O>i^~5QTy#D{SHlVYwddZ3|0npRkNcDR z^!Q!E|2TT@4}`8>xVF*vBEC0pTlDq&$dz#&{U!I4`ztYGV~BVmciDIdeI{O_;pLa$ z|A(eyfJNjZKqZ_U{44w)e11=Uhb;Ur&jotN2~{m{TzgUXQaBF%%=YfZXJ>pzf64vi zekV3um?2)G#d7$a$f7(K&U|IWUxJr^g&us*zC(5Xgx}fo>V=rw%G$rQ+uy6rGg1dx z_D~oy_eh~bopp|> zcwZFQ6Wz!?kJFKZF`d5IsBZ@KU$hbJL3|pHB;KbA+UWgZi1W})<#1&JAc$q6+eaPMzZqCs+=jxj!`_LY= zhw)9#k$53q6gc?`>KusNm*?4}E74O=xL=<2Gv;;-%Agc{})jf8;)MKOmUw_ovz$NcIu3#zc#{Ch}nVW_j$xSk&oQ#C`4*<8|Aw ziD%;3#zW|{3oqci7oa@IjgVbR@;ezjbd@jBU^H zcgU4sN{rYTB3{TfHXcHsiI;`K%R^z$T1oh_z?GevE2A@L;%Ft!A^snM z`zPt#-;8G#k^2upzkh&S8P}l`azDAh5+gQ-h?j+mq%TV)JTwcq~;_10zXZ}I;ka=*Oi_bKE`|EI4H-q$PfQN$1NLcA0)QT3g8 zSy;SW2)icl1>)#Uuy3ciU)m>po@Sf>U44?uGEUeq{4Zkwk>RQwDx0%zUW)cr#&)!i z^M20zD=}hYhMJ%4Gi==X-xRH%7fxi~lDgqbOJNU4A#mXCY)i#~&kmV>{YM?kD$q@lnJM z@zR@E?zB1WOZygm-uf=jj^7kHOLOM8tR21&(mq+YPqX3wHPrWUzdR2pbAQAC>mtJ^ zzp8xBx{)@&AKB^WQjYeK`^o)Yd=#-mypS7;n5gQvxg!~YZf%E$f5cm}^q=KnZ)?+;X8xfA0eS@-`ba>fVtXZkvQ-Nr-cGx0*a zgfL>CCtiAwm&?KZcR*g0{|_MiAAEifxLraP{+H(hz2gL&^Cpoo=W&s=`|Zf&e%|B2 z{hi*&BHqK|JuGLebHzZ!XW}IeUcQXjazHu;==3}Bvi2{$>?Q5=gw#Q<+7F;og#3Qa z+%N6(KJSnA5&LnlpYkfX6?kLp_2_{9M*rx!e>_$APIymw8H3)o%um93rdyDrR?Yq#6=gAVm&7urYe z=Q>BS&Lc6|g@;g?{BDeO5{&Jwyqvr*68f#{Jmn3M8;BQ|FSyDU+&@l7pK2xam)!5s z*bKV4Dn$MqIT!O699y}*6#M4-&i;~XT)6O+5ii7x4;!v>;>C%VKK*|Hm|d5Eg)WU_ zUV>KV_>x$4Vl}6uqR#B^)F1U{<015!cp+Xw7_ray2rnN-zk>J9(cdodC7L{a>R4Y6 zMNj1aG-ce@BWz`Tvin#3%>FX=7BNxvop>Q$s_;=fN4x~_a(n2^o1i~(zehU}baN_1 zXFN%)2CM!JD_W1$e<%8&NmO9FpS^Mw-O-%Ot3+`VaeUW{H@J>{h z_B^~I`fw>S|CbNM!`Crb<{=u@>k;VUA*!L)T-yiK`EN3idEcc9e=pW=R1bzC4M+cGpriZ4&+metq}U6*N4yX(GvMVL;JN*T?>fYnau0tZd~Wf-{GVS}LbFFJ zX+6XL!T%rZ5+r4!gUH-}D7at#2Q&9)SMndskntq!!F2?#BV>56vElkmypRiBxbT$` zF9}{`eYz`i<>dK2na3OcN89B7xpUng=f;8T$mbwmf}D&EU=PlNIS*Ei69qHG3%TFK4$YBxapL6| z=uVXym>{GPM}S@>U`3-pc?syfs{U(S~2{&5{Op3vT`uKcbC?+v^ieY!=J9n86c zJq`pO9*O*of$J~BgN+T>=bqq&{k39%ZD-n-Ty5ha^tmQpCs$ z-bSP+qz>{jzIK)S4;wi5UjW-2m6|JfsKC}X{P;=8+5b(#=O9NwULVFz!G0QDu)pL6 zg9Xizcp+XKIJq6loT(x{2f5lMFoL|PcwgEQ;o7+_x#caNfO2*GhQO?@{BFVL{cl5m z_RT4&;)`2t&ZV01Irm%Fm)P=L*YK6azIKH?xi9g3yqt-5u4Df1yQuFBYKcMWK{rvaT|?g4bF$q$#u9&~ zwMh;AsYGtdlmB16{54<}eP`^@;Lg;6{^o%fHUG1H#IwPw=19Cm!OL$DN3VpO?Z+c= z7W%K9}%JMCwnOT2DviML0%tX?;EwCsG&cV$b!5-+8hH6ey^oU z%-NXi3I5n$Pn@BBqi|t5rx00#l6SL||2y+Pff2g^>14mZ;8+CplK&%QjbV~~1*Kd? zE`>_+yBgz|>)UbG^%?T|!2J??k1VLl`L(A0oHFb}x4h-UQSN*%0Kcu|i5}UbvY$o| z?C+pr`OvtF{SCxB69>%E5f6?NzC(E%e2HcR6Phv_Bg=ljm2Hyq^1lNMeBEYWjzw?0 zaTx9E3-^CIXf;$2dlYQZ7Nyqw8{OVucsqBV*&cm7w(t-?Uc?|CG@vV z?l(=jDn6*z0eWT+C+47wtepKN*VuRneI{N8ix>HyEwxkZc#j5*mX?;*kXIpF#sF3O z{Y%|-nfqJ(-!}J)&R&b0#a#2bhrV=U>8vls#zIYb!7)G&?NP!UY?GC-zk`kC83t@T zggz55gT{;euhv?rW_%9H7J0(wX}0;_^;T$uE2}@FF@6E_Ndp$ zJzntl1l+%2@xPC6(4EmrTF>x*@c#!(yIl#gW?i%XF63r5QRq6WYyZyoxg+vqechQN z=r1RCMtP+%Ku7fFwK?0zSPlp9;-h$ucp+Yjn5g<*120#CoM3{yS@|+|4Uw15;kOz7 zN7)K0ImaV7H%7fxi~lDgBPUe(MSf>JH!#ink88M8=DU#Bw|T$mo2+q^QUv|wWJZ#g zjRAUW4`Y|yk^Ln%z@c)OsQOO45HA)!j>rXL$CpU)Ig>2%WZgc^hW~|AecUh41Ipar z@c+8V$OP8-5UDTn{CVPUkk?o4m;QaIsM<*S%fv>OcU8#aK7rgLm|W-?5A-}4vz2&~*&u1VgnnDKcD%Nogo&aBU0DC8|42D^U+6}>UhcMiWB!q_g;3;RoM z@cDwPoOtOKUY-ma=MwDOczl+%f4NTX!j3Oe2f1oL0LQPkikj)m@39K~+K!3^oTsO(aM!-+yXX2$-Tv)R80POqhEw*fHZkEMo`_jJ8Q_cBB7;M+n zIOdUC{NLMof3kLtaoY3I*Cy8Jjm)O^Zc>i{2H@+t_{RQ{yAURGn5g~$Z|+a;aa@SL?w^hUn&xVqaqD9>I0j&B>t$@qxQ+Iu zeGxV?JcK?E2rn;{pxZEC8vdV}KctP?)IwkW-@u+NyC3*}YTLD756!sJI|dk#uP5=3 z{UsM7mXZrSIvALLoYs-}tflwLdhPUOd6!_Mjt`l-i-;}U1Nb7^y;`Yw&&hV|p&px* z#sG_GkAaOnljkwSPR57o9BE&Ub%+=LSP%ZOTjqA@`tHp0bjXW(&)q{S&f?hD_Q}J= z<$vPUjz^HU$j_FY=u~<|p-+EOUhy;UsYvp&F~EY`V`%tee*-axc;;9K!Bg!gDPH!F z`^%IQ;F=wE^elCI;G4fdKdxKWl)EILKlhI+N+ZjYeFDESpm#X?fIoFy1B}?;! zJfDO)mGC~fUvRI@79ZLC-h=iFs(!P-%>0zU(VmBs)y^yjsKeWD$C z%h$nmYV8q(HYV|)$M$u`JNB1cXt1w25-$sZmp!mkkZbL@Y(C5HlzeG^5;{WUXpAiT z{Z_U~&ddJ6h3mI9jtTNV+Sga^H{6gL1Bg8ewrFcC7Q$3=KG42{P^s1!U;y?tcIH@z z_lR;>fUg9<(?1r1`{iEVZdCv+R0_n?LT%Moh#ak?ImG`XaQ_5(Z;Nx<;(qB&CI6!j zedm7T|H&8t{as}4H{eu{0S07WazDA>_`3Q?|5zCA7n`1L0-*L7JKp2Eqot*#HRM&u zmN7upe*aQ;UFQB4|F_Bgq9!l@4-osG!99*cFb-?`9*hdEHL)5T0}RlX{IbKw80 zr(=M&$6{VHPibH87+}Em1+HaIvrzMzA;$7reI}lXX9Q0*|4$iDE8{cxdjjs?u=rp8 z&#x=t@}rfsp5g!C{||NvlCl~5QsRHe8!Go3Zon8|`4G{oPv=$TGuophdPaGrF#zq$ zSP77+`IUGfUKRu|D*m4_o;u@mI)~q8_#f9;K}F|#e>gWry;Y0pU`0WMzU25H z@`ld+N&lgT+%tg+QzTU)*duqTNb+P1@MUDa@6LI@n)itp;eB$y;6R(_%pXDij}%X# zj?EGTWF~G&-vGCkr{MmV=eS2YFFaI|a zPqY1X4F69MFMQmej2F<3g#U5$-XEwWez^eF3lPIKd;5e$$@v|SNA7oXzljx`Srxg6r($ny`usoqGkxwnsbil6?Y`=K)}UZ(MYpL;E;Z$Lw3; z|2E>Oto_S%e4K5{r4Dk{egLMb^Vm#ZmUnHf2NSDMyc6eXd2S5Qj6I^Q>lEkxW={k2 zPGj~=V4%;LKG0usKe<1{LkKs-3*%`BBlh_Sc**g92jlZpbAIS36w9Sbtz_xuq%Hne z&4Zo(AHjAxxgD05i~%hD8D5p6;nrHVkjitCryc`z*d7UP*gkR%+$a}As=gC1y~In# z|8t6`63U~ehWnE8Ke-=}|vXqxM^w-64YPJqQ`;r>~ zWc8eYcp+XofR{b#daQXSt257IsYoHM)G{|@`~PX}M_(`bdaW@)%l54UJ*g}E1fIip zLU~V&@xVR)Vf-t_KlM0KFx8;`Z2l+rL)i+hsm1>d+HTSBFUuHUf8@%fut(+Zizl0r zC;J4Bzp{C+zq7u+=y*}ZO!^z~2cK7AMd}S4FNXiS#{H`I{+fy9!TTX<{vSBDG|#?O zjsfOm-{O$L`@PUU>=P(s0M3K8@t6KVuGV5j>Jcx|@gn@+CGLlbll}fwdjrWnLTJCa zchS~!m>6Xbui5BgoU)?4YD;&T-_ zHwF;<7Hr$r*iPjEf%aw31XlDnrwihR{-MQ+)FWPqmq>Wo4LP+_ApX^shaW)XXeG@d z{vUz+C+XbZ@PAv}FMX@zfAoReKiwL5G6sPD7Mc4E_OF5us&xS3g|Qr1$&ClAz7sFR zOBFtf=K^@i_W#50R4I0R#zjXwp;wa4&6GrU&@he{`Za(P`5Hg>`TJ`kVo!sCzk&Y4BMA| z00Wv(twECvqyO;P@kLKu-7{ zNALXs$4_%bk^8a(^2q(|aK9OwSB?Q9+t(hans*Rl?muwY<`{tc1j0#;eJhXjH&rb6 z>X7=Q{ye_sH?e~roa%5ll61y)Dzmt3W%!Rw0V}RK9ZAaen zwQ-#<0ZZPP?K_9xaqQXd*t60`v=c}&rj^U-m|7f2*PdD?QsPAFP+P_=}`l(D( z2f1oLfZ*>j*D`wqMe57)F33a-HhC^w=!{9wRo}TGHwF-V#J6v3ym;lwGl8|n00XdZ z4%h52xfMQ=!$j40;-xlTa{S-HcskXbUj_vY_T?mUzuqT}zee=UWDL-tea$#A=9<5e zSsX{J#{d!am&8-bs~szIJVgCbeFfd9$; zF28Y=HRwxcga7-qPoVQ&K!f_@JuKz;)y6gL!LfvmhtTJaP{6t7+JJ-Zh`yflWF+T(WEYZ1rB$vqyyk+i>VCjJJ2 zzdc>OhguOMOQ$5pwm6Q-_W#rR6n&|2U#c|*Xc;dhyaoznpTIBi9aG+8V|;OsTNqEH z#Z#OAi_azp!qnF3*zt5ra`;p1Yb)f{;(uZ_8dgL1tN$~MmPX$TiTEty@{t(NUWT0Z znOq3ls^+ltx7PfP_HbZw!SHAJziZsDdhbuc)f!63>L}%@`G3K%u*h8VH5l>W%#fZB zO8a_#T`q1$-tr;f{m+s6b$k!2WIRNFBlp{Q2z@4A#=`$y;(nMU+3zoS&lL5N|09I< zN1S)5o?TG!Ke5$XEF$+?9B5)5@z)yuvhjz}z3WJO&>k*a_{v(s%N|@O$hBx=Dar3- z>=288aRG=Njge))-^w=0dD%a>aQ(K%F+u)E`;upG(7w=`HDkwY+(qu6O@G86^~Yx$ zlQ=!ddu+Q)#I7h1{w;9tO#82-+`j_3l0*DI0{2g34`HhfZE?S7r;`84{jH7j$^GOm za@TOVOEq>yyw25Qe9pz^Z2vzYHcf*5qI3V!($X68Dr6`8k9PR(|Bp~_UFQB4|F_Bg zq9QN<6I(fK<@^qM3x2PmKXSiApEdLsw1wD<@>;_)?Lj=x!{2K0KjJ}D?iYE&=V`Y2 zUwF2dq_T_?HVpsE7(iqcD{8*a&=)8FLmt=s+u0Y9+|O7!k65YVXvChR9^(PWP6i8_ z zC-;;4x#q8qeZsvxihrtp*XXZ`{~<5>yg!}8Z!`RlYplRl&UJsB8>8N;#s3p&laJr4 z$|Lk8$N!MWc|YgM#E4=8hg!~g3dBNtSaUn2D-u8gc`=uPo=6~-v0d*@=#J(i_4|(K%a(^Y}?30`L zBlnBnKrgj3{f*pJ2RFov3NK#%-;7sr=l%BkdMW=;U{fFWC->>mj)ebl^xhw^i*(UzT{o*%JQ0+{AQ^j(xUa3Fo&*OJa{@;vOjd6dnR{;1w`23!< z16lZgjP`rS3C^IK7RK%i#O~z(W@%-w3_k1U;7Z_`W6yTRo?bph49L#W9^_UV524Q; z!%K<(WAmj;p3MEF9J;DZQU`e%l{&%wn|)c{g>?+XV3S+n!YtlE7v9fkUthR^-0!e+ zkL>Hku#Io_m)vUOA@rGesg0K$|3iOG-S3}vkN&B6A2^xy9X}C~OO|d&goLaeM$MB^L|(- z7ZZKoYha(;@6c%t{l&Hi<GOx!=e2#qt{5 zMeZlpkZbzOHOW}AMm*ryv-x;%dJB9x>;DE)(|7cWF7lk_JF)LwGwd7p{}1mG z>DGp9zoCD4{X-RRs6Xn@_NUNi;AJ=BRJ8x$3wcrRA)x*06#Lpfd1%m1WR*YK^&qb= z&*rK6UZcO{eg~dv=r6WCD3A8wI?OQEVfKLk&HDUI^K>6yO1YZEr>q$J+5vgl{(m;V zSAB*&a=-hY3AvwKL#`PH*I4gNah_hq7sYc8@`ug;Q{SHt@Flv_T#9{dg}hq)-=OU* ze_F`r5y<`IesVv#U$x!@21)8Mo<yfrnC_n*WJaFILI@+i}~yzE&!3EF|zFUTiGT#FZ%}P&NC*j`$_s;a}AQ}LXqm?v=_c=6&T8_yVBxsKEy_2y|pr1cE{2mgPt zN7k(W8>qzpXdAho+^>#D#0z6NViWP=(E%NfhN5X*x&WV~u^ULCXrD60$i04w=F z5?^NQY{}bh@&7t1IOqLRj%4${cbt%2#Ph+r_N@CIC;Sh2KQdpsp0^YJ4?e#q?LZd(A6JbNDyZLe?OFFK*f|FG zZ9b1GIOq))oc<2oYsWf$ddKfz*p>go*Te7dHz!u4yarypGHm&j$M|C7A@rGesfCvk z|3iP#pY=m5HT(1=b8lJum+Qcml}YL#SM3K7{C)A-@?CRnmtgrFxPKR9Sp2_`|9dBP z>?QX*I1zZ!j-6uLgYsw(j>R}P3&nvP|8MeJo~vi)%bDbP?+v*4*)D6aFDGsBKe-<| zC-;+U$Tj4e5ZC0+b8K8|o}SYkVs0#d%lN-RzBKpSXI*!<>AmuE!+lBlpWILGcdvsp zwsj)5WigZXpgn{$wRu2zN%+5EzBK$l^}2f{*%wZ>&=>GOxgTohyx$R11{6Dzc*_2g zTeZ9@^@x{F;pOxe__8na{&XC(yh|`#2lJ+GkFZbfC-*a!bIre29PCm2Q}ufR`ny~1 z?<1Qz`b=Dn{ziRv5WJ~Qv9IluhX(CLR@vXKw*XksKC4FVci6v2_Ellq>hCWRKQ1*H zFYe>>K>K2v|4n)jQr`yTijxgUVF6R=t}j>-1_ z!*A-w%0-VI=b_IB+q!an4(QMIlghGYj+J>&v8X5Ccj_b4 z-i$nQzu*$e4rQ6s-rY7zGxlhmtL-jTex9PgqOYGLJ9R61C---n3lsY&{i9^#1-0)U z!Fi=4cuvaiWb6=&e{mUz9F37>zu(F>$$8m7xN!Zp#xX(uNBjEZ>+-yx{NE*S5_=SE z(W&Q1H`_<<7hJ-Kq5M76xsh$`#jtO@STsX_MQ`6P>R9wn?kD#bapC*kB3^c*4Ykt@ z{^7e8M2=R{9OC~GxPOw){muBJE$)~0SMopl&^KSdzt3n=j{cJS`?DX!#W3mhZF?_< zeLKPZMs1=$(PJ{s>R~*n96z$Z3v~}6+yBpiH*}004{*KF($dlz@+xG@7@%suf2q4J zbAOBf+vI*xkC*>@h<%yMtV*X{DK-|Gk;i$z;1c!=WqH2;DBI4>*rQWiZFe{3SoUbq zx9FYoe$M*^3z|F!ywu|VCgOqcd75qh&-rEg#f%d+4FAg*K-%Q094eckFHZiSgY8t9 zf*z83Q?~DgK0VH$Kc~oMJOF=B!2KH*|Bvk$WPQfvM=NPP!~enmA8h?T_i@I)l=vU==3%=GJLp{H=NZ~} zr<~%_FEi?|B(fQK#@LUXH!$PeIYcZ46 z6P(R~yRz$tFwXXQ@u%YdCgMRlhu>!SAJ9+m6j&#kLTIlap!2cK5v`1z)`5n2iH7?Bb4zM^s_s|}d{NJE| zY`6GdVvm#ir5wrTfA2WK>EKpj#J(i_4|#p}&CgisTM%wYxc{?Ub9Kir*wYo)ULzz_@g?i4=!gCFuF>c8Al- z`=Qq?p2BSE&-h)kR}_ikEP?iUx@uckeQFt^C?DYzX-JWcxCvHq6$AMI;$ z-k-!6S^Jmkh;vjWse@d#A3)X58{)3zU0dtRa~rp1{sRSMpP{RmzI2Z0AAL1@EE#_2 z*!^2_zu4Wzf%Y?g_s#Cxx9j**weQ;+n?&|suBr3Ai%?w1@qY*7{Zw;)_*z5UpZNMo zTm0YaXH$NKe(r~Cc)x@5LUR8M+%IFrGrMhH6}O7J2h*Hm+nH!!Wh^H5_j)eX)v;K{ z|8w?_p#RpX&~RT;{_i#Sn{|X+Sp2|we;|gN7{5TV`~%&!Z*RC?Lf88YHeAOA!VBVp z@HDyFWk+9GhwYp2f5Uuf=8seJ2lT2=E%XKaKM3v@KYv`x4^!K&8GCTvFEIs9Q9H}; ztoyC+>85>q#r+by-s7`X!5-w+nwX~kW*Uo~4*hld9#`a|Qn1=}>pDs2TfqhO0~ckI>&7|5wleYZ)Kb0yzC!bxPDvX zm>~b7eF-2;K+rN*tk`rSlQn_cXB^ zQWM;NO{^vS&(_)rV~5L@uCgiZ`%jSNOwHJLINZOfzvX9cVVP^z09q+7`O2omvsXUl zmGD=K|C@*h!sltW`QH_f%s64g@V|@!q)o2Mp|TnJ;^hAUV(T=_YYsqcGy6uwSo1#; zGQYMJFUkGB*jOx|0e|ugK~5f4=S-$Xn} z=kVJM|Kl3+98zf>B5?`l#;CVy@&82HBr7 zoyvc+cvw?^LH=*Te`Vc1&4&NiO#I0O4=LPyA7I7e|8;3ov7Y98q`qYQ4|zl9{-pn+ z2k2k+3HmSO6Oa!_c8=vxF1EfE^3(9U^DuTu=lh)ZLnoOZYd%ND-~AwOQRdZV4I#ok zr$*-e(ym94`{#;h?za%$Z=sU^8;l3rE&g9etn+aNnvlE?X!zedPJoQc6tOP}|3e@)jepaJ|{jIr4N0cR}&x0*I~^&Y|vp~s)e z*jDT$c(0oqurK2k?NP*`m;ZskCg%c%|0nPZANMEs>Cp~(A24}0u&OT3c~j)R?0`J_ z5`C%YOWyAbgqJUZw+>Fn0A1zQu5-WW56{Jr`w@`6K4G6*Apfw|DtlpHC;vz0OV{&u z!vDeN_oN-j!vAC7#XC;$62JT?@;C+}cXG|T7ua{kCg5f0V>|rM=jA^Tzp3&o-?`ZS zwqWvv`_Clz>v$g+;yU1b*8xlX--f@*+P@~gNFC&={Q!JnC@F8IFUz~&<9!ZJdohoWvY-{+vei_xz8A*+0_L4PwmEyph2lh_ySdMNTGlyyo-$I;?Xy1hYyY9chxjMDb7w|v19|j5frPPUt)#Q4V zH#e^N670DP`AB5BCzqra{H;s#et8xNdsL2}g8b!8-k+O2oc>4upgrK2PHZ^JPw%?_ zZblj1-*)=4yi0t{q03+yeUrXf2R9wY%dgOnFQ9!N00+Ju`K>17*`Bwc{EayOpLMhR zPJVA>$*}|S-h_U<2l;HY5BpPczY6Ff80(;~QXTmn_1CGn<8Js^XZPg zYz-0(+KF7kcBqWlC-;XiVxK2oh!-0Vq0ht%@e;y_eLls@9`L{6wl1x+p{e?WzO=&q zaviiP+y4)rRpS8qCVf+j6{$zO5HG}wKep08qWOo-|DAch2H008;pCRrFlzBXu^J7l z>HBGv2j@_YJEb1+N4yX(et#Lbe;EGn8uxpjHMR!^bKrityqf<9j;)lhJ9WQ{_aFJ} zD#uQq@gw8a;Kqu=|6SsKKrq?wFSz#@>MhAWLUXu(q2hn~(%^h48C!_m>&&-IUE+m! zF<8(Xi5KEUgA-H7gO@!h4{|MJl#=`|=DIsB0Fk3HvbLsU8OqFO21gf5eMN2lS7|{9|{C*cAoBzXk4{>Dj@Q`;-6i1KA<|AA$QP>D=GE zyU-T*i*_pcpIB`StLpd&8)bEA58A`VL+CT{LcD}9VxM1dykz_Tp--xW{zm8irKP1c zi|7~)==*Y|e3yy`1Gto!YXZDw|Qi~O-N4yX(#EU<+&cQ!w z@ju!ZmHS1W@OheT{s&A-Nh-@YVZ-phi~&SOsivY}hQ2uYAM&{7&ozJ5xC!ti7jn;B zHCBih88>n4BskD!B_FBze+Irplgmlufxjoh=NA8u%Qce{R3ELR^$h<9|9`MY)~x>< zsKozh8@Zp{AKN!6&lB(HZ{$M6cKU}$2Q~ad#s82OecqqW;kOz7$2C@9Q|EhsI5$SU zRg3>8(k36jSCvQTOOF2`kK9l0XDpgM7O8Yde-K3j z{J(}uKJJ(20cGxQ_8SRr1x<}Vnilc(UL zO8$?;m*~P~vgB>I_>`yM+I7^xhxvl74!kaBai?)4n&b1M