From 620bfe40bedc30bfcb08d7dcefd20862b8183973 Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Sat, 10 Oct 2009 00:03:45 +0000 Subject: [PATCH] HBASE-1887 Update hbase trunk to latests on hadoop 0.21 branch so we can all test sync/append git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@823747 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + ...adoop-hdfs-0.21.0-dev-hdfs127-r823721.jar} | Bin 832411 -> 832430 bytes ...-hdfs-test-0.21.0-dev-hdfs127-r823721.jar} | Bin 535832 -> 535832 bytes .../transactional/THLogRecoveryManager.java | 3 +- .../hadoop/hbase/regionserver/HLog.java | 83 +++++++++++++++++- .../hadoop/hbase/regionserver/Store.java | 4 +- .../hadoop/hbase/regionserver/TestHLog.java | 38 ++------ 7 files changed, 92 insertions(+), 38 deletions(-) rename lib/{hadoop-hdfs-0.21.0-dev.jar => hadoop-hdfs-0.21.0-dev-hdfs127-r823721.jar} (89%) rename lib/{hadoop-hdfs-test-0.21.0-dev.jar => hadoop-hdfs-test-0.21.0-dev-hdfs127-r823721.jar} (94%) diff --git a/CHANGES.txt b/CHANGES.txt index 003bc27e37e..2d2b641f83b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -105,6 +105,8 @@ Release 0.21.0 - Unreleased HBASE-1722 Add support for exporting HBase metrics via JMX (Gary Helming via Stack) HBASE-1899 Use scanner caching in shell count + HBASE-1887 Update hbase trunk to latests on hadoop 0.21 branch so we can + all test sync/append OPTIMIZATIONS diff --git a/lib/hadoop-hdfs-0.21.0-dev.jar b/lib/hadoop-hdfs-0.21.0-dev-hdfs127-r823721.jar similarity index 89% rename from lib/hadoop-hdfs-0.21.0-dev.jar rename to lib/hadoop-hdfs-0.21.0-dev-hdfs127-r823721.jar index 5ecf0284d02f38badc5a697bb4aa40471efbe897..52adc7556c866826b5d0bc834bf5aff9d2087412 100644 GIT binary patch delta 59382 zcmY(qb8zP|*Z1G8ZQJJ7wr$(Cw)kvq+jhJ4mfQAQ+qT^;x5jhb*Zur{Gf)0`Wpdt0 zPBO_%&df>9$@gC;-;q@0At2$w{->Du%O)YwLKyxBJX8OJ6xeSh|0Vr5>i<&x8xzET zR4F0fzW_Qoko&&_iT#%pkUamT5v2V8u(=j;5AlCoXih|f0RL|# z9YgegE9+RX(Es($mhkMq7EPjdbhQ6y`IDhn7VyErz)m5-z<5)VzTu}Nm?CB95Fuiv z=&>RJ+SNk{MszrhOyfTWaAYv(k~wT2io%~v`CCdR6q@<%- zK{kv{|Ksm0_8_-j}9iejo`@Wlhw>22{5**Ey} z((y~n&CNh4xbNv|cVa_jlepRQ1maNAF>~~&EZHsSEPLd1m6y&1 z>SQ8FaP3ZhTBt=o(!FY8TI$K}R%FfjSAAklvnT|J z|GkIJYtuI39C#I9BTLu4P?l6kpa_PGjCLgas8WzR96@8DA&oaWGb-;Pibj*MywT71W_rWP@1&}$6VNB9j-Po7>* zub~20#B8Gv>ipflQuo_yuDZ-il`ip%_vw!=I<(AdfMdY*bbzI(hzlA<00OGNK{;&p?JjV$ENteHp5x~^Y5^&>;CLIje9V`-@LcCCfILm z8_cW91}@P0EV>AP0J(&S(_7YnK#2yTxV6&+ zbK}Cy(V@UXIO9^*?h#Wkznr(Ah~=$ns+r@~UH1~_G9&pzuIa?~9U_^i^f!x&{=a9= zx(`W^QSzlFPVR(J(xs-}w0-hN_;zpFCF(z)JYr{=N9sR2#UmRW6Z{W88PKmpO#cKy z+uk7Z-}(6dq-zmCa=>A*aq~{s>d14S6_9wDkQ^X3B+P3{@y0IrDQ5ps6B95lR5HI~ zaOwUC<@u5v{>6WF#wz$Jnouv3`Bkd=MYY;X^_E~^ys+I<#P*+gBQ3tlcBIdq*zm7; z$V8{MmA2NQV8*fzH-{WInL7rcwB3>3T!N-LQa9;3YAwkKEql-^Ig5j52BWL0;4fA2kO_b}+aeK0tKCBiE6L)2M?u3PQ*I^x zzJ*-qwv5I``Zx>)LM1=+Xl+6dLgqUUVtEljWZUij#+DAbZAZ(u481s*aDCxaoTL3^u2j0rGxQ^+ zL>UDbgfMl`$NLqN7+U++&L3J_dBd1|ZH*+k65k&CyN~)YIr0;Fho$<^qW^VjG?ww-pR zsl1bq0$-ifO|pM@@wlC>wJQEwuMK@juxKcdZXpgiWFA^Wtf+Zh=YkPR;Sx7K7eb78SzGLpG)DAAR)Tr38Bj`ZU<{Z)%d zE?$Nzo|8j#$rA22iOb3b^B!}Y@1yIib>IjXA$unN=pnZ>hJf7$O}4|4+Iqyr<`O3$ zRgz4zPd{&?6S5(9S`!&+$8ca#E4TnA#u?-j>Yx%Dr)dSDKMH#wHTJX9jNmLi16Nwn zU{y76Dh>`$X{I4~VW4q90J7oo+^5hAN=z4t;n_F11n#KVWl9Rp{2H8=FV-*`c}7cp zV|2H6!HdH)xo=UR1|MfcCG@umr1u5DRCjv_$;`oZ3H^GWN zNc*auD)VLnEkz}7y=VhlsEr7$V7S^wPuJDR&nDUp=rodwc^NJ_wp&5j*~RKOh+06s zv@67uS?xN2eB57BHRRQR**;Cz9~Nb2L`Z!aKP-8O8|_*u(Twq6_qNMtvefhm@^D~_ zwTZUFz>*x4A~~&#@S#7{Jbs~*P?wb`w~Nj9u3_^pF-k|{aM{K=&G7)OP^C@5Giu17 zA|KLymTHclpI}?%Ox3RpCsq4gKkkxa%C)rSrH#xju(6qiMc3yI2?maJ8)ej6hBMJ@ z$@9(irGj>_$(WA!H0F+N>7c_kjQXT`$fk+2C8I^rhO@<7HN)8A#r%++kTMisIoiBs zy|;dFW9=#;v{V{W;5!0#gP{4RGmhKpVj`6NY(mqN2p$ERN$UesuqvhTzflmnCiSPs zuQk;wM2D<|rs}SGi^V(fxaTBcmh3W%-Pmb@FuI+)?ob60TAFHUDy2|ccFh6lO#@J9 z3+R?5N54i_C`nZ!f(A+<8nHl93)Y%h*t`jqwRWr%4nM7;>em6%vsGKA()CsQr1!lf z%JNGs$8tT>0YYRHijLz7-r?#&7;SOV&i=~0Us_U|LyNpg2+gHbQr^YM!DJ~q7V7oq z<-@}=98Q_D!*f>}^6qqJys~!9!APQ*KOsz|8Zt2K#jqjVvlt40+BOMb;8VaN@O2~f zF1$eb_aKuDkaz&kgvb{+i~|Vx!wB0JNWQTb4;xTiIY{i^a0N%U z?Z$Wpk{K5d?6_9>)Ah=n;M~i51k&{?+)Q{9o2;;%E@c6#N{;a1m`w)QS@-!@OE2Gz z#1r)L2)dO{{t-ebi9$Ptq+08d62=eykr4Y6n&?spx+(7VLAA}Et~U{6n?3c1LLswW z@P+77Ix&sc(pV-+nf0ZZz)x_&JIjW11_yY@beJ3)h-<*3{_gm-0HM=~j zesK?%aODM{56{t?%JB@u>eaik_XuP%4qNGdcvuq~J!)gxVdnXPYEA*CwQAa;1 zb}yNOr3Y>8v0fq#ckh*}M3zRu({c|aco%z@K4k;;-OKf|+sLK$r7e@MZd|rE{5G+c z+wp{w0qw+j?pbXE9y$QT*RN^kLwj58Bx+0~$697wDLgugCu%|%r%&on!MvTxl{0_A zzX>~HIgCS!clxs8P;~9eQBn|75{w>!1Ud1{B8t|?j^VLE>NkViBFq^cqN|Tw`$>Z_ zO2`0riAh;=l?!fO-ZZ=}3uNEE4W)tdgldgw*CT!nL0mRZwBI-V%KMEua?~C6g$Zow zy76q%{S`gSd#w7Q&Y~eW%p)BCkT^|poN($urC-*(A2zc=vY8X<0Rp3D_Ho~J&jB}^TU@>bUfv@V7KOppD#g( z=@u!vMfuy}qbs+Kg7AqjUq^;I7c0Pz8?zU`aJbHI>2>xwSTAs%LfgWNR_`V`rjqF< z6Syf=KlyPhpHKHwq+J(DByJ?(hCVgf6hn+@MN%k84Z#ZL?<YZZsa3P3)$^=aUI@i3;)0ZM_`;xXrwWvnfpazoY+m09t5Q9rAs^cw{5buH%& z2;dVzx>U9>sT~u)L^c(98p|IB=|jt+_C?L5<1JX*5qPEd-=mbQ)qslXUmoMLkVvKR z&yI!-J#_dD@v0%6^T=budMCB8vMmjVgOoO!ulUjn_&Dzg_LHf5@)MU-$CZV=*-Aj< z1&2s@nm1Y8+{=#vF8jXr10THG--L3!Z;JpSF6tS@~hN8lVaDx6;(mSX27IE*R-63hg@lz7yrG1 zK?t&dlf^V};OEI6qTl68z%ICeqz>&orx`Mpm^>r0(KV0pvLK5bjObsnV|uOx^0te} z^SFWU|B$OzvgH4eOV2-VPKme1)}tL!i}Es3mX+hv)e5(#i<5mpm4k!QwEX^Ju*UmG zp{B9wa-a5c4lvw3qMTQBuiI?TXC=P(S__9k7l&Xte3MrV=E>@2u6|$jr z_sG7~K;dE}1|$H5w)GFW{mlfot~1DqR{cq7j4Gj+YG*4<(~DiAykm}TLgn++^fZC9 zGr&A8<8RXBMqv^SjTKE7S@zbHa4gS~4P;m8Gl03kJQn(lh!~(6agS!ERMn4$fG1Vb zneiQ5^&Y>{OO94T_19gh=*=Kgl|oC{D`Oh8TrLNOHM9d;+<8jUvYnOhdp->xqvwg3 zcJV`mN-8~q$*91NjTSd6Qc{}Dow$;CK+2-3_8u!b(jvu?EFqF6#=jNy_DMYo>{$HP zF@TbltOHIMGY|T1H_X~Z80-+Df*+32wKHAi+8A_y9sq`1{e*DS^rP1-0KhU*zuwW= z0T3b<-#mT9|LnA!?=)>Y5AdIVFuxl_6)IF2wo-JRyF4k|a&1MQ^!& zq&af`38l*ld{^5?h-nMs1p}0W{l<)FHz6jicOd@wKUN$PpI*5ipd{1_FQr}u0YW{d zjrkw6IVf0qoC>qp zeHM;nlZ)6OMn^ivW#;ghR{0{Snvg;*6jMju!4}fZO2cp5_6(^u{w5WQG?L9qwmCKE zX9MMOrMCS3QI`DCQxP^V{Di+DfhRm0N}lp8bozd65*o)cB<0Km(OV(hQ4kR!Sm)D zBy5$|glH>xtx($F7aE}jCz@HSv7}Ix8QS_Yr>^QNW;nSxwLi0LF_7h}0qrT>Mm*^3 zGS}t_WRaqJ2bNsUR>TNKr?A_eeMn?42E4KsFRF*dVwKXNzIsj&r6ZMHDITXA7Kfeo zfaW@C7;-jIDeixe1@^XL0y5ddPqu_Ie7yG-me#GZK*YT{Awq8!? zcjr*~U6LCtyFL2KaKLN~2%m9y#?i|{5(+cAaehXzO}0(<@-xe+pVPwgH^&R(9c~+A z>lwYgjuCU`-m}QKiiVytg6ridNUa1AEiK zXKpxda^vLgcu)mjQWS9GxRMU%9zncZ?Q_$5>)fUGK5p=irMd>6CY7OU?Ql728c^^^ zEl7p-1asYiupzBoCb7Ta6m8kF=LJ>D1Yw(7nOL>ENYd@}=lla%o}ngzFm^m^u37@b3^x8B1TdI=^v5EWE||`_aFzzd?a8u?NoY>W_r6#)SPzwnnL)<)8a%4juG&Z~=Yt_iJbEIajywlhZityd) zH)%wxQ9F}=>=%{%WUeH+-*jpUJBa$4r$>r(m?xUUnMb$*P712yOSw*3_uMVp1D;w} z+zZehyT|m^W(SkFH4B4@plj8lFXQq&cg9TAU)OV^%w^xu4_k`%S~N9qHP`SCG?lR0 zjIhm&hKjz|g9OmS)akFZ_NIbR_l(1tRRs1Y95mv8SoHTb4YR#i z(=!E@KhOmMs8(|M)76#aFP`2%LmuX<-dU%Qx1(NAslmvZYd3$rn z{a|`pW{(}qmv+Tj43q3YYd=(=-d0+k8gWlG^?c_c#rmgXu&_a z?vj9S*Y?K6Gq3#qWkgcO|I+3Md}-IPOw9kFjeE81HBN@~NQW%B@Lq%@R+#t+z`Y#Q zvfv}Dmj&WFI;@&SE@c$zAU0_$`PLt>qRe`H`aw2rsE&AnUN^E#Mv20nw!fw@gessbwkL-HUO)8}`e;AFwkvVyHaJ5rd8S7{QFrEe3mh*Hr)N|=<#$sd(u z=sdAY#nHK`W8$q)Dfu_0*5d6dqvPuI*=hk~<#a0NSDA!!k)j3M3Y=)Mtv;1g#zlyO zTy8tuDZ2$vH##W-)R3YKTyP^*a3NIy{-_@Ys~B-OW{*`h_NQ$zp;crOt8r|w{Gg2u zs`=gzSL#hNdXo?y1h44Z2JPp)!c4IG=%|}=9)fMOlwCpB_<38R@vk4JMJAOc>Z^r_ zYzUsKAul&oDcvbQK3S_yP?!V{OoNfse>{RPY7tHAKt8c{1DX!Gup6tejJ#L?{HS5h z6-MWEOnD$8bm93Lo6G8zv}=Kb$lpRCX%inQ*3{;t^2QB=MEMwV0(55n97bNC%Cfyu z;%u1IHAAK zl1R`EO*Qppm}lz5^)hVYW1a#i_}NGJgZZ96fI84lugEJQIvhD+q4xWFVSJ!|uU=6; zH2#(MoPCGzgmu8>0jNZ5Hbt_Y0iN?m9ql6G`#i$et;!P(=^`{z`_x`nL)Qf_taI?K z(;GH!&Tu*TV0rFfIeyilT*Dm$@AoP%<2j?q-+c`&$c+{e?PKYJzXE0f3kZ_nKuhVM z-#-WHr)A^Q50neT|621h;`lot=}$N*&DWEVNrJOsj|B2l9Lp9{*uduGLZ%;8tu$QC)Ouv3+48rqJirxHz zZFWF2?;xN45lPJ@Qfsj=1bDA0xKiXWIGL$7#@C^@HE`~8{npw9IbpXo|J#STL~c~Y z>(`7d7kA<6|3c!3@-7OFGbBEQWZ2xHodlU(I7dVv%TXp1d83b-ej%$5ySdqwnn1zaeT`BQ78t{P-=!a&5e{!M z)y0LFmg2XG*bq@$EunvpyLN}niB(NHll~T?9zH!lej$QXL9JxpfVE>$eNq^oc(CrB zBci{=dC_NnYsWWgSJlUzji|H8}{^4nz62X^V1 zRRKMrbgspnuR$^A0XX#5dE(&O((`6oo~!Zz!xM2P7~VOK3b=*<>1!xlI1(&Ctgl?k zFw2*L4VENQd6kucLF+77ihkqADRWSBkY)S*^OHp5?A8-H$F^~hLPwhMmc5JIR(If) zY^TdEg0GNravy&I>oD2Wz&_nw&lD+-zMdim#Ck`xz<(KX0q`a+jzw3+tUW*{9Jg)I zvzi(2u589nu~IlbT$jfMny&|OaxToix7uZMwU2SgbH?#fBe%ifk@{Bux%mTM2em2G zydqa{*5g!O*pA1M5FpG{^oQi{X81@kVY}X+ipU(3!$0|=fnw~;-K@kS7+KhOUOZh_w+Bu{YhB%%aVmllLGxVRH+ z29pvj&%~)7Xw8ocCSL0YlWeW8gWWx2+ox~T>bg8`pokRdA3W8}opEzHhVgH%Ba?f- zeIU1O0X;iA=pI_5e9sV!s;!=Qr9}NI*)*P1&UKz9m}j2(wB>)PyN| zPJQLD7G+u}84PiiH7z1a>?w-$!n_6K?pI|CtZj9yT886I2rn-0N(%^p z?){KmZMSHLu^q|lI@alIx+8cIzDz{F-)P)i(0EE9{#m5lZzz9iF9z~X3BIz@Cp7&u zT5ajQ1RHI#<`P%MN9C$$Rc8~v+7PU|l>7XnL#!0`HoyKD%H-IP7A)W^xeUKS>v&f3 z;xL+Ph34S5&LYjVG@q{+JRl+rS#}55Tckzoena4KF9Z{8fFW#RIh+Wb)kgkF)&v_a z$1<5X)F6T!y5t6Hz!K3P(1aPt6ft1mM2I|wb`wqrcRi+hlSGJAFwSv4z-!-xeuF-3PGuGn@Vzweeb%EitVplbbjiK$MBd^V@fF-M_J>57{)kSo@e;V&ck>veHt(P%cs9Q)Tz zr~6924C=>PzC%Lz2!AXyiq;{}alhH|ByRf(_dHVR@!Bc)MWfO0)40Wl=-0GSI7G3r zThaBwYrE=8%Oqu(5nd7j=gezGs{x8Z7P6oJlZb^=1rg9i#wm_)`EcUnh8dT)=j9Yc zXNw^uW_~7%pOd0wo_8g*09|YwLe$&AUZMHNyOha)9uDCe(_kSz{rVb6dObU9+>wqk zUUTy&CAZ?~hIK9;801zHE%>o8G@4TzyX_!guXUZXD*7)l4ufi00oz%3& zOTVm^U^Y}Q>*1;IGr6Iw2|AgjvALXyHsDX=tcP($d?Bl`yt+ZW)3RLf;(zg~*679; zR>q!0twr7vR2i2Q*5t=R>j}rl!my$3L*PlQ&-exegB zr;UNrXFX8{{)w#prd+?~w4z6^g=1AR^0L-$P$11RIsA1DZ7}S@EmbYuC~3;9^z;4% zn!4-i89^}dPuWMs`EKsDmtzw;Nms2xnXGQIg(Q6wxwi)}=`!j`diY~BsZGS$mjE%b zZGhDmhWrDDRnF;WlpBRYiuZs+2OM)^PLb+1#MGUU4$ZYVy{j;JAcFuU$`@76(7_R+L4M|8NeA?f?Oalqb z_L^8^D2ZOY1a3OkT+<-T!Q+X8$^;Sk!Kfz5m(Oe}hNKyyX%jecO&k5H`@>5rc~nmO z_;gQcxv1ZwW5DZfbmo*CyjggX?pl9tWpyH}=qhLcVbtEmB`SwoY-pS7xGem_K5yr|M?KeUrDHN!gL*&wXBRue>%wmz9gYsVguOZl=#7mMSv z5pCm#wEA8#82eys#8U2&_syu$p&N9xE&ZV!p7cG2#WMg`<)7IAMz(#kO*#mGkq%*}*=UnzTVQMc==9sRDuf2px=5XWoS( zQCFYY~PY2IbOFYH*fUxU12F{<7O*J7DYarq7&uGn;pnX zbFmi{@6G`{BEDJeG*xYiE3WoFa#g5}$VRd9zil(;tw zA}{dQDf6?qc4P@=k%=427py7XB)k3e5-1P|i=B$f-KBfRyx{6rzho=~dKbvd{c(Ig zQU2d_DKU>cU2ixrFc(ZPFwOs)E=8R(p-lsjQTFM@Lxmg%hr)s=N0a^&JEOxvLcl9| z(TpWdO@8m1nA7vCEc@L$l&%Gej!v^dVd8)jr6+A!npbzUZM7?_U=sYiAi-1woV~c$ zC=J-AX!x(6w%`8j+RPOVwVH==k8dRQWI+8)LTiMp2L18?rt9$#sFQ|=V$~!gXe03iyKZHVx^*6w4%Uz zgf|19#jyIWTinO4oA#V|fxY1r9#Y^?=xCVxrX@C{zun1o9}g$r^C0Ir9rev|Z^YX| zfabtAJSrUr3}{9>`6Q_8pR!Sg$11pkAIRuT|D;VFtjbN=}q*B8L`i4l7cUM}YPke z<5j|!wLNDt@@S#j6RmPl^!^wD3Z7o2q!3#XKHV%U_(-RykZ=OGJiet?pcv+zytyI) zKB_G)u&&sl8$Q0!7^K=wT4@g7lYoOoI06SnbDJEj&ck#710P7+@&U^-;Ufu3aFqfn z->8SLg*^H)O;wz2Z~(m-guk&?K@C{%3d1v4@8G=}H}pp8H+)7Zw+v9d!@A)^B;Q%# zxlAZHAvT=17%OdYA@fZGH&0A9Pt=^3;tP)vFut=*psSD;{;5lXB!L`;1UZMU^bQd~ z?kASm7q2DmQedTOw{k!^?|Y}Bc)T*zYTt0_(L#)C_O3+hiUYa+nd?#8ToadfujWUW zX}9sgI|q{C`%+7@0cG2uzi0n00_#nY@y|5lL@PC&XNOBuHF{&%zfaRWLwr{5D)Y#n zmG92jM2m9@NMpT20h%}YGDSlaKOf_j2$=$z(FE}T^&9V8?Hd%deE{AN>Fb2KpVho& zP(}x*PV@qu3}7Lwkviu;_^vkGi#t9%*MU19PCh@|8P?>klep}qbNpyCphTd%wYa`$ zqQZXa$;aa7p*@b*ifIqCn9_cZk!gj+M9*k(ihDgle^NcQ5h$7w$I5YjjfLbHHr3vY zw^o8DX^m{V_oFEo|E5gDCVmGVn-w!|ZK~+=wc@ybL40qJUWZh| z+8{b65h!A?YjuKi^MM|AC}q$vc~IogMA3xrtwvPs4kXa*yyJtMM6_>`hB?EJJ{{*K z?XWeJl+?uFG-|WbE!PxxjwM2TG4GI0PJQ&@RV0(!Y@F~8Zw>#=Mc|S5B3TLF6 zmBj3t|C$?eClR_(to27RhAdl*h5e7?Nwf$l8Gs)n$+4W!hJ4QKO4oofzEl)m4Jo2A zpW#~|k>W#?wBL`@WNl&9e?(5hrVo?zK&6KVHQZ>}dMXp`6iUaMMEdgQW*{`XXCr+U zhHAI)g4pmQGI{W7hInE~fJxB+**0IISFaXaoJiHNg+%Zlt1!8%uA=;=9+^;D^sVh17gcSd%mzM( zq$l4rC<+t(+3GJFWgB5o?%#89IVThy;mv^E0F_tPpkvv$(!j>ekVRsJOVM?sGrVm6 zCs}2w4w`>t0WYQs@e^aDyv?tPvgq+<@PKVyL0V9i-y>Ov-!5wsaZV@}#(F|RODdld z(scQg>9Fi%?GulZcSr`G?E{bU8>N1Q+9vQ0@$S`I59)GX-PDK-*~GC{;sBWE30(BF0v}Vl#Uvn zj|*S9=E=x3=0to;SxHEy#g|rbvdGLmRc@5qy`s9HLu22 zjF_iJg$+v7id0=^5@aSS{D7DjWmD6P{=pwYUfp(4*;8H;>!&(pEq>&_3mmMJq*#M? z-5XP^&2eTx6aqLfzQp+_FFqMK`+_~FCokmh5jf2pf$TXW@;Y!V=4t+L8{R70`q?i(!B1q&e;B@K#;QGIgzRBVwrg-y!cfGwK| zCa+mM?0dDS2v=HtVUaj2lKEc}Z3ZbGe)%rPPPHQ-SHYu*N-|ujtwjFACix%^w;q18 z@ff}7f=PrT=U_{!s%OKy!XeEV#c?f(jgLuOm8?xJQttg(jo@N62@n?MpAibop$T(< z>Bi!#Fq=9bk5cMqvs7YmV+(2wFf=s*nx)YZ0=RF(9uDAkchvWs=U-_vX;<2`v|kB^A$ zI?~!P=D)!H`$9wh&)1Lt&1kQNksZ)XaWpjs;vqBKc~cWZAyGwx3CSQ~!+%mHLZcWY zBh^xNnR*Wt)oF{~{{9Ea)M}(^FNi9J3dMj`_pe&o{$p87|GJB~RWfnM?|yB|!35&l z+X%`n`+b)0{qa=Sj(@s!FW^A~7R#0zs_nulynJ(DfFd}3Fp}_}{}OQZ`!AbaKS^-z zfCKVCgUN(~JM%giA&}>0SEyfxoIyf{Y&$H=$u^dNYuRAUjU8P59w)Vt{r(IkVZ=$- z?a8Lm$iYh2Ep%Xqsu)Rkl(f_F2y~#s!|VYz%64>bVmll#X5;LBP>=iR#l5{Z%)jM> zg8{TfF`-NC3%<8!RCOd|e5}Vk=|2+y{y$BL4@blW-dV*Rk4Gq|o%Z(Xx(#av9aoPb zKSrKK_PzX5BfK)7GYLqn+w zcq$+4-P=+0pNMmoe~0-@N%}MJ^ZM?=Z_}>;XW<(VlC$+3L?CRDnc(yFpWTBK7V zc6vAH?|2R)_&OS9*?Pt@e_oH zA+u4!g`_8R+Dq-HQqW`EgrpS!5J|WC&2X`-o$LQO%`t6XFC1dy5$7^>cClSzXGw&f zl1VDhO)&kCPoiqVb0wjR$1uvjn5h3~Un#JLkzrFo{4N6KGIN3*(m^Dqh9-{Zkx3WN zJp_lyze(XX2n+WVX<(K#$}TB{l0c~y1rApbDMlw1H5cm_i%ApRm^{h{$VR)G31Ij% z(NVWcCWSisr06Sn#Hg2MQc~ZWN9!xZ6~NRVrC{fnWz)$iux(+Mm1mkpXQAg~HY=!7 zn{o$&Sem!?HQH#sV|blZ(TtITy$kH4eam^KFz?Yil?zmP+jy5332R~{!9S%SHL>BL z$f=M%Z8%WNR?a>%ck&Sc3Dh5xC=xmSMK&fzlm-NmQUe_6XmQFJ9HAvae2@mAy9V>9 zzYOD9y9Ee!^-i7dR+temneSJ^1{A|4&d*E26q8&S4G5fQbBBLopfLhW=C$xFD50os z^R+S1b~zNF)$IO#FBzlCHo4iIws*oWIlz?FU^KNNKTd?5^Yk4MQ6I>L>~4eb*QLeU zB#J7)-fNkj7DZ&Vg@qPtG=*t&)1f~JC+g=GyG9JV$Q%i)AW4#qjVYQC`)5k+kR_~# zAI17)`E#R5#n6P)wI{Yg%KWqJo4N*>h5dH=Y)_+%yCWkmvgRFOe=f754%Tf%@xP$a z?&sZxhto*nnPg6&QKTigU4&y&%4H}Ws$@WlB9_6aUr9|wvx!xs|@L^>0pzo+$C1JAPI0GHEN$RbIWbJW+TnHgNdLZI+9#g zV-KNN>qS{dTuXlSjuM_uj?3%+!<$qCXJ;&uBZWaUxpC(A9W-ELYpP2t#kH_05us#O zx{&F)Hy#d%6vC0o-zQ9Swe<+$N2Z0LpQdxHEY!x)OAv>cmUFJG*q_ztm@l+=v5(5F zHQMP?c1NQ~m`L$RJhUIru|Ys`?VYwx8}TL0Bghdes4CvBV+t43G}fmZHVB!F?uhh= z^ron|W3$=k{@asb`eSu!xUa!|{#v zlqY~nzZ7AsJ6G^!w4jA_l8$CWQkh{5l`Mw@I9vI=E`byBkbd zj(cj?jL~3lA8D5fU_ocE^VQx1ZC3gcq+^-cTG9S1-b`PkhA9cxDW2(pn**cD4oh-X zBu#L5eOwvsFiR$Pj2-`-GB-A~(!tvVejQk`d_top()XABfvrgrn+W%MF&e@`vslh! zNTc`AHoiOnebs6eN`T*Aqr>K4n=qXvxNk$0ziVD0uO6(}8g>s6x2Hq8q$so9A&RgP z-@2L2xI)TMLNhKD>RavJVp|$Z8OGyF;TEr87|dv-WyjXkEmyqo2_zGz6Erp zq|&D_W(j)T?ZS>6mHvaj)Y{M5Xkep>!w+k?JS{O95n1;nj#?^F^CB7PVtL)`^3F%0 zZo{9^*Zi8TRc zZ`qzjPHaw|A_I=YdVu8*<@a*i+7Iv{(munytAmX96rsL?S!}_Xpq%@@=zL(dHE)Q! z>vtgnKTw6#Zg~BR2D5Imf)mcjrh{i_6;*w(nYGLsdGzj9L0yGQf!pS^7tIAY8>D$E)E2tPe>%% z;GEDl8gS6g(Td51N~WMQNDN}CQkmT((PgLU&}q>%;(uUTVu$kZ-VXH(Z6wqou69WV ztWUWOA1&}#p!UD1Q@Tf7IR2aMTY_gaEn{RY_un;H()bxb$ymGZz7YeRX z=o!)fQUN;GtagJahQT!0g9Nsml1Qyh6ZRaLa)lvk-8~+{dX?|CSzcU2YSGOY(yVOS zQO&Y*tz??eGp$$$cJObt7;{p9I0PKU?@5xoB{ZA>Rd71etcUxxF`zEtaHUlC21?v@ zFtz0^IwO=KJH5ThP~0DVCE8Q&Jy!SK-8Bz=C59obJ?NRv$pU&il(%$;D3vsbB}HvO zg{UOvx@?!pC#Ok^$_bkwb(MCR!4*N+39DDd3?oLGZQ3|`L*bTz%IPG7U!soguf{&J zt8afIU-^m!GtqD5aiA{J0%${L+9#&^YYBxr`25$r`I+;6x#nKSm^^NAJg0E$ME?k7 zy|!>phTPVQ7g7tx9BMH4Eeoa>4TbiW^dKOTldP9=Qz07%e#9R$`d~HrSYx+YFlB0+ zZ19$>CrP#(2Q!gOn2|OH<3&Ogx+*+9U&+UobmAl-G3xaK9^hkMV(1qEQwkpdoYT!+G z8~R*_t{lB(;SQ-|Vm%6#tq8>j!xVX_Z;EuP+C4~8jS!*3 z0S9yMcgxuS6F6`0lacr*%7vE)|Mn~+S^IYNIX&IIffS$l48y2QC*hw0d*QzCtKMBW z7{ilUs)~*r2HqNlq;Q*z4q&n{+(!E>Vb;^#(%f288Q}n``$(cv_u9AR=l}*L?6%sz z<1Gr+C*7y04gFeXRq-OVY)<#UxiQVQ=Dy%9P&$?e7@8;9(v)AWv#VtA)eXMwL#X># z4oVYN^q`a6AEZdc@^aLx?yKM84fy{W#Mw74e^!gCAL5pK*Y~%N{-V7#Zq$yt9u!i) z)ZX`V6~XIc#P`m0Xku)c-$AXFm!($^-lAh4qrOy##+7jw3J5Eks#!i}Qe=6mgC9*GfrmVhwG270fohR( z6)(Tjl-xs2Ip8ZN$-@;AJa5gEwLdI6Ve@)bKwW5+o0eU8nl0 zD3EbhQa~eLVWuf(n8$`PGUsopXkk=Vpj!AK%9fp@j7!P*MEr7&hgyqB-i^vBiO24> zRT1I)2lxnYKt&W}$59~iiSa4jb{j^VF!M$rvY{pLZ%2EAwP5InMhWlQffYHQWZ=O9 z;_(JlDmS^>T=gy8wZ$H;mbOd5zA^I08Q9us#C%2q&PKLF9`H~Gk=)$pHG%;PLr)tV zTGN`I#pHx_@;m08^;RB%dFy1c3f|v5GxZw|A3GoOTp2eix}x1j`r&Nt?~RuJ>iCo&ZpWy?sb|{HFZ6ChD1Tab#&c-X z#fBgDtM`Y}G%al0$LX!+*Tzq#5BXm(6B1k`hMW}QR z7C{*j3`E^ZmcbX9`y6qyy6JfLeh_9e5qtEK{bS=soY%ixcSzd%$fgIs3WAdvdcDa* zA(?FCU$oC!fxkaZM4R$JXA_`hY+)G9W%;X%_D5%QSLLmOYkqzlKa^i9j!g1_gEGVc z93+AiXAg4YCtArk4>yjMo4!wgtcv?+8#03!$kCEisr9VB#{ABPGmRhg%BfM0dT!#u zhftwGvvIhY`zbTn8=fGz@6m(fSp2#<%_9&xP0~VjxL7VKD)izj(ReZP=~N=iyAD^u)y0`qI}4~&ztgx=`1c2bx0hViY| z2yJ9D{I#)gWX;6*O-!08l58rmR8!@okYx|1B>fa?$muRTW~ZBiG~G6dBkK(O_^uqe ztOISds`S_urUNNv{JN(-@lBaNcpxD(?d7{4G)XLuf3kFwAGjSk8m-FcldJB8pySP5?xEz*ZVz?`{S2z`@b@T*3tjEvD)bMYdQ{FSE-zm^ zZNZEgrSlh8mMtpf(4HQ@KDM`$N$t+Hr%E9{=TPEP0(FNErKul_Lmkm(657N)d;EH0 zZWlg_e;?iN(T?$sAz=R+#+B%(a8osBqqd_(v;9`uQ?9@7~prXEA^>`;0g zWu?~DUk7xktH=J&Q*ov#@q4nvq~fVyM2{{SW>KoGujCFRC;jt4J13vt5lky;7ZRky z|8m8f<`y_ySQS`JzDDxCiHnut>32_)&G#&we?Ncj{KdsOCNHj1abBw3P1& zT!9tMehzeLOJGXh-IV0VGllcgC1l5%q8)WAnbjOx74bJ6av?q<=kydJ^qqL86#(CZ zXoy4V57NW3V-GinY&7;*)Bt?&3cLyc3a0T-2lyA){EGtJ{tf=F+qd9t-TnjqsoQtp ze_d*)@ed5Boyxy4(Cvrtk)Hn;KGW^z@C86-CjU5rf7nhVU%}Ua@D07K0sR|wD9p{< z3a;F3VCEWI!JUg+!NU*9uU4+u3M$vL6}shahK#My{b7yx-}F2IJOG~0!Q*)gGCc1> zFVA~$oacQQ;Q0WCcs_#Ro-d%l^Dp>Te;*EcEBFumSFisH_R^Ec5DFY10kB8c$;c!! za`RiE$5wFW=57IJEA-?U8tSzbdXH>{th_eJUNPRuO|~%IkMb! z<2+gBI-b6MBb=}U`qb=z<7&2nw`bp0=ttxIsp&a@9<4BN8w}c&<=zAXvpk!?f177y zdD_62$Iroejy4!l-9j7YZGoX%U|5#f2FGX7sE^4xf$7(zZvnZ}aXXAm-D)_Yx5>}x zIih{r_>RnILo+%A{fN=u9fr~Sh|&9DoacKe_8fpR;%Akl+FCjxC|%GbC7dTM*d)8d zJ+e1ED6?Rz%!Zw^KfELd!|QSgf4nJ&!h3QUX37y-VDrJvLWv8Ed*8u6))(sShwq6p z?j0a~b3he5PSX7XoNyLAf=FW~1mPYO)N?}<{0a@~dC)Q)Cg8$cZpZo&s~_q!jw* z#py%rmY~n|0G*0-z<7x^e|2v(p60!^dXC#*JW4Ot1{lm7*#muT#*DX{leN{gfTJAC zjY&3OcWTrvWRTmSXp()1i8U2ZYRoW_TZr8nGV^Z8O_5t^8o6zQ8P&Xo)N1Cm!8WSw>o-Ahs`buF3N^bG z%6iUbJv@~jgZZueV2QIulZ%+M0nSe_vB(+c;sl!oW9*3v&4S}oS$bSY4j%YJk*npv z9WH*+|$}LdsrI{b`qlO=gTH$maZG|(Kz2jZ<(k_5t=Dr{_U+q2%5bK= z{;WOPBDl9579-gGvpv_x^*pg%xA;UG_*{KtV45 z3Fxy8j9dptJTg0R8`&MbxLfv}OeEAl3_V~7oCGJs1h@pQgBz(`1W)T^;aN%^xP6mS z1#0mbd<|cRe+hUSzJ-)D;6L#LZpC0RUCg9*rYP6#JH{Ym2>FU3#&B+7B0O!Zhb8c= zaB(X<;zixQY?N|q%re&Lc^#T?8u=fP)1kMV2}5KV`O(=hN}dWQ$vJScEGM5jkNj5^ zOp*)WRJjo5$r@NB&wxwhS#X^^8*Y?~;b!TFU&vZ`e@fQDvvMhgqh;`pTn?Yf2KbL` zL{o+^Lx%A<*@WZd3Y;h-I7K$&bh#4e$yIo@JRfh67vgR5Vthuf!`I}c_`19V-=OwQ zc^Q5zFBdL(h49F$M5g?i$d=cM!SV)?FK-m%@=X7F*=K;zfC%cv(Im{vsa|@5s%Dl8+euUoc*jFKS<2PyTuW z@iPOvlMlZ+-gDp^@tzAN8M82x(ma9FMKOime=oog*%-#2*bAKEPIw!8)0oS+45rYE zEQ%u&jjJ#lkK-s|yktiS9fGfgd{||om*RM~iG6V&N8^6Ojkz?2j^U2^CidS8t`Vd5 zL4{)<^q@%wca-XJv#)R+fN|i`5xg%tOdLQEfcEU@eK5w+SJ=zOCzlvm2S{^;y@oPZ ze;VHhBTXFmU+5v|OoNiK^=_h^k_!rqi2%t4z}7@b)za#O;g`3&uF5}_5`jc!^?NUDc=K+e4jS|P;ZuPuSz6Y zkby%;!l<{U45J${W@wTzZE?mF+L1dqZRGA<9`9Ilbcb2upj;x060ZrmJwYd719f`> z4A-qqWiICo@)I!SXOJO3hb;Le^p#)10Qm*wAz#8!>YqSuf&3;({lKKoCbN4@f4v39 znWQtbh*>6`Vp!r7IGh$Sz+8?2dtj&?3W7{*b%Olr1o_qZ?3W!jn4=ZW;Xy|Nj5=0@ z?u?{Uy6JQoL{lH#4$}1X(d{5o32qBWtdnSr+Zpse2`1}y2F%p0&4n4{V?q9hxUiSF zupfHK?@3XzXv4joU& zpPcN`@l$x!8(a%bJVEnnL83!U8;&`4jf86n4?{j`i#{hMnk-KVz72_x(nvhMU)#>X z)?y^%W2idW4YwN*r#`3Hv9KaFoE6)-)7-n^#01Vc?H1Tn-ENHejH@DlfA>hnb!H2h z0wq%Y?kD<-pa#yMQ=9>db$d131@};U5ByrUHveanwScu6Olu9~TIWH&^%EFnT}XVs z2oU07x*RrGSHK9udDS<&lf0#1nGCI^IBNDPPCTqR?N%M1an#)ome<)c#$#(;*mB@Sd;AVN^GwZcf13%$t95E+GJS1> zRn>6cyC&G3lT z29H_W;T>xSd}uueA6q+dxb<5+(Rvc6S-bH}>nW_Y_TVz>e`#D{J%cN)=kNyWdE8{Z zfGyS^@e%7~Y_(p+N3B2M%hv1ojr9h8XZTNgV5z2^V@$>rQLO8E ziK`VZ9mgu4MjedFG~Nd5$D6s?<~F!&PV%0Js=Lfjj!)n$US z$TNV-fVIv8Sl0PnCqvRX6xfbw^N{C{<6bO=o}yJ||I~Olg3J zC~L(DBz<;?-IBLQ-LOdOnm|kI5XC3-;6&0F159-3drSv-;cGmF&gX>w@Gq@RF0Dhy z17)23!PH)Ql8OA^WWF)l2TwaFT6*^aS{WTQe`yL%wG$3U4aIXhl~}nMt}Al(yuQfQ zUd4_z#+g}W9y!g_diDmcVYAFsAyFl})+CSlnl1GkDHpjZWyQ_B!Z>JwJ~*xAXS`DIw&<%0>TE4?e|eoGzq_D>+t#~b&<@y^j8<)IRjx#!+P*6_ zl0DjCZqTk=FFDw1$NfcS3fFcpkeOoYyyCG9P(UKcaZOfLt&me0h-isI8WukwQ2;JVjeuH^5F$F65db+@VzR;p=umXQxmXSor1Thso0{5 z@qRTOx2jUysb=6FH4|S@v+yHTE?nv~q11elp{hitS}5{WjToyIiOK48F-M&tD%F`{ zA-ykFeo?1tMNrj=unLHns@LYde-^rN&Wy$STK7D>4yR!W^|s(5oX!`#I21m{QZjiD zPJ`!g2D}ZH7!Swm6>gCU^%O#mXNMqGCL99w*yNFWcCgUBSsZAnnq7fqSWa%hBN}lI zIS$(mTmzZkLmv|>$O+(H7`YFYn>cqW)vt{>m6EoFAhm~>>_8icX1P3_e+Tq9&EdcU zB!bRBoHmGw^S^_}eg}wLS4T)r1W(uxIVM)(Knh{PL2jlB7sLY=CGk}B0QO9YR6EH~ zFP+>Qc8zy%NOE-uARU7T+sAhJtpgq^a(mt8$rz&fly~ie!tXqve1i z4)ud6FcqGJ=aJZ%j9zDPaio?*57hwKDg^!13Ye-QFik}vpjJbJe_8`!^%J;IT>xv< zg>b357%o>og`3n`xKpiz=hfx#t-2ESsjJYTu0c~>jUH;1`Z;!2HzdAea9|)6>`nLVzMzmyLH6`!;;9L9=U!Ab@k`b#X*3cM24@6u$>z%yk=xx`*iF=M-nQpAdrCs2Tf9LP1G~zPGM~j1aPMXDz zt}R}U4e`ZA@dLVYsU9;PPP^JZipyTDdyRaQOn|SjDUN4){H8|Z5$W;xY=|UF7~_|k zGRC+9gvJ=Ly$faRPS#JVfb#N3a_xfYu}-$3l2~ODtHeh8`yKV7_V+vL#q{?(?lse% z&JHQmN-h3XPw;2pFU@Xv4R3`%QU4Wq4PK{qB=!D67(Wq}@aa@96cYrmn=LBw)Tv$! z5j^o%j1raNEAae3P)h>@3IG5A2moDtNjpvyhy-i~008L>lW_eWlaOHwhc{p!)C+F zCn~-XP*L&ynsI#1;7CFR$JaPI2ytA;ZgEOvagHHB1E2 zjGaMPu!G29Hxq|Ro{7t(z+{hxy+O2LXC3x2-_PVg9UiR1Lo7aAhev|s@=;zqrs1H5 z#{~kzg^>}{5ok&7GWHlTD`Usv`&~1x)T;!1WFSzpBQ8*zH+NgMe}LF5&|r^@8q57QX&X{w zW@!ecCpcgioRqmS!%$~ca};G$9m}y=#~M7PBZfsfZbFZaC>H8? z8qessk)>x@e+r{ZM+DtEZf28ZOrFE@8eY(G7%%Er!kVQ_7NbWXbS`KWaNi!&8Ohpv zZ{3$Nb8g0__>Sl}ienU{j+gMVj#qG8$E$cv$Llzu;|;v2<1HMg98QlaUdUwAoZW_; zDr`ti-bx*B;~ggNQg4OMo4}OaEhQjI{ZT5qIMW}aq$@uEcnOD|(Mxt*qXSw3a8gne$m z9G@3?q~K<;H zZV>Q?E)m7)Zc#kyak_sLrz=SD#U5Yc@ueQWf7#>9m^U@?4MwW~eb-2BcccyY5%6Kh+w2?ei8MN$SKqpp)1}& zMhzv*8XGF2ku{A)G(|d!nEfT4QG<0vS7H_*4?xNXF-ty#Ir3r5myh5Y`6#ZJ2hlAb ze@Bme63gWwtdmb+i+mb`@){p-Vs}*09r4gb34yNAHC_yyUIC{wIDI;} zSI{7jqeZ@k1@d()lqaTui&nu!tKgzlaMAO@wGg`#ePwXM_P|XN0{JuQn4RQQG0gO8 z#>Lf7ukobDcQM}sT<`J5_H;Wz*V=~Kht@_aO14iowR8+kqJSgrQ#>6Y1)X{l%@}%$&aARPpFQcA|yYj zraOuC@)S17)6{5R(=X~X$ja|fkl*8=`~ipMkMz6zCwxZXeMPWt<`F+RO=Eo5Mne@Q+L~HwY9ZYTWedjT9+=i zCdz-#eQ#zSWJm~AKkoMKJ@?$RpEG&tllu<>K$X@Ax7b&PK;nw#+VD{V21-d82x$t`YJ|l9218;QT z4;*-t18?Sc{?LWD;E#B8E06kQ{4ry58#lKzHa}qicR27)2j1ntyMG<5ae)B;Fd5AH2h(A8e(?_^D?7*Ko@KFao z#c+CdlKfJ<)f5O*|S?;9{{Idi9!l3>t;~O%* zDdSrRnXBuX*T-Ys@mO;#qWQWIl=@Iei_`{v(Wn-s>1tohr+?ET2=)y={|+sNkkM3q z*0~$1Yd^DQb29?gBdE3EP&DQX#kTl@am_-PZlDhYBD?WCUw@CUGNi>So5Os1Il=J< z!%?k`Hf%+(L?eEL9DQ5R7wV{_3j?7J+UMLJq$ACN-5PCeKydgwHUExioDiMW7>;yQ z`nr9zRN3ik3x~TaIzql! zJVKi`rroq6&3|>BZSB!YqF`+>poL;HN6>PnPOIgs2)=`c1`et}9*NNQ1SOY_8!&;5 zCOLEJ$%8SomVhY+V$It5p)ldOeOFA2*2W_>%mM^gx42d3MW(+q9@-JzphXhTP)TB9 zYj*mz?pPokLdesp7zkI^uTL)1EjhkOEYKeC2YkWiV1Kx`E!<0+Gx=~`Fdpq(-`*Y- z;JUR)l*leFCm|{`F~uJa8K_qJV&SfUzp`3%HQ{hj^YJAazV2=<)K<$#67)bQ5TpC* z0zpkj-_acjcSN*k^f+R)!Pw&49@JQ3W1WHM%y|gYPe^EHOdwioNVuh>bPIyDHr&Q^ zBZ1P^#eciDYmv`O9% zfM#@Pu?>N4ElAiP%r7Zz9qkq(R4nWd>sl3}H`IsP!*l}%QUbG%mI-L(NdvL8^$27Cf*}Htv9W09# zCx<5qVSCWmfsmU9W9C1JTTI*FV<8}+ifG-zfZwOQLK8^DQ0|zv@FVEfR@$6A@CnN1-RFL2o1w(+p?iENQK&uWzm4TW~w6a%&A;7dG6U zC8&k?i91AZjqVCjWaL_{bw@NoUW9^8@lY(#rELjB1H}32P$(Sp8D>4-gih)V8-IY# zc6P1=D(dUebRR3cf-2Sp>LFi{ZuV^tM`Enn?7cR35?$H|u(f2XU_pC0;ukF7ujwt& zaH&X$gTmYmYNR;elO-gki?q2%=_naV^17vr%DMj|b;pvN==agf>3Lhm{XA1bU79AvHeEx{ zS~LD*4QtA9BhC-0&=2zT;)1@BhiCb(IF2jD>k24TMfKZIKpxF3G3 zz(eq`0(Zk53jBndgY*Iai8B@W7yMg+AHl5(^zpvi;C2P}0=eVdH20HV9f7AO5cTLq z3I%?RvlX}(?o;qz@GL_4$)q_7JOYOiW{##7@rc$y_|!)H3##ZQcz=|y`!~XbbPB7$ z9=Jz=etysZ+@QcdZVvFPO2;gMd_kT9hv3JQ_9*xv{s&>=NUK^C4+Pt|u24m>uqPA_ z%?-taK~ExY_K^P#ZYQtl35Pr*DW~9n@goI~;87VrR`3%XqR=D|t`Lb=1v%h~L=~rm zwn{)bS*gbtYV#zEJ%3^y#d=$qZjX7od@+A#g@*toH+iC+;dqcvX&xVanAQ~!#sU-t zc*51 zQwBx>MT)$lvPL=g34?&za3~fD2Tf2aM~$VcgB+;sTCZ-J#qGM zro)DaYz}upU!;RpW1YT`CmQeWrchmL^LR!UG-&(WVbBr^kc1}=lWFtB!mK_B*>q#f z)6U|NTw6TO1d2H@lO#O3&yzyNLuP{$MY1p+3Y`sddViu^awLMzVoPQ(H&~KY?ip5t zqz$vBIf_)mO(|=y7jB>|B<)28NQ>KvBA-bN#;{TvCQH43c8M{Y?5)zVThH|JVlx>Q2>x*azi@cWsJ!J z1t;*W*?*0iFRFPsffNi=@R0P9*CtEg{X2 zmePa;varra`Y$*e?Tp7bA@U?D$MiVHXPLBI!2+aLuaH*B(y5Ac8qSubYDKDH-Kv#V zE7BUcg~G8jLTm^bmT-?APQOTC{JuC%NIv{JiGMu3zNjaj+Dq2V=Z}%>YM!-3NJ2i+ zqeY?>9(_M$wILP?QZf%q*=FOtid08RCY`QGYjGOFq*`I}*&`$}N*w`Wt|y|!BD)Ak zMMZ^zUMxeHcd{akFeAN@P3($yiE4Xs+Fi|*uIn`|&xtN3=^p8F)82hd?3_0I7`?JZ za({A6X=@A-jJcXal7E6C&alU<-+rFvk0rt^v9!4}67Dt1-SiH&zPdp&Y9bEGl8qvA zafvCG-W}+!Oy-h$ie_nCe>&G)Oc;?MbcI9F%3!#ogUCp@b53a2RhvvtszFgn{qZm;Drq?0=h39ZoOzQiBJ(tPRh%4L z{;^RMZ4oZ`lWjrh8ci4RLlS{p}%h@<35vQ&=%Q7y-S#QoJ>CNKSl^P_k+L@WGb|;gL_&9p+)ch%Cs!GLJB?y9 ze-1=fvktEj4ov?QZ!3cAYiny1B}gZ^pg{Ys>R^z|v;we|E5s>95!HlQV^I`BGnYv{ zB8;OLdt8;Q2Azk-uee4?lmID<@_(}^?~5<2_xu{e7Lx^b5+M@fqLQu0vqYAz06Q2C zadcBrA2K{fh6rS8L=;OPxe6L;Yt~DwqNHVO*K6Bz(y3(HQtRAw#`HMIxl;9yG4i3_ zRNfGk#52e7Bf_vB)zjCZ_~+pB!hvL7m@2U7uF#-iY1&fpDE0LqM~NcNXMch#B515l zOe;_P4D$hkH}|9tkZp*NN$b&uM>V^mF^xnn+bBkGNY@ky2KiTQhd)`cUpFQR80+r^ zTlpVYm?KJ}bs*Fp9)_9zy9O&sP}8~*rk~iDfCsLIYsm0^7xG~;d>?M0@r?i=!4K%~ zCb*f#HC&exYd?Zpsn6lklYi)MgWE;_6S#xU-6{6mMf1Cd%_T1Hi1EE}pXm3B^AcBl zL_a|Ha=%aXHCzl6^ZSkO2aNk3fCt6*g9bi_jPDP@!vI-XT$7^t99p4Vl-kE=76S{7 z0H%8Pg7sRMOr52%>@kqb`an4VSuHu)`yl5KBWQug#Whv{#VN!uvTQF8}+b1qR;PFdcpg&l76J z^tKmhB#F1=g4KBxR2hEtG04>7N3>Pq;*w5LJP0&nWSK7Vc7>gV&T7;cex&f_Dpn|E1eI2FG~BDkzB& zE1^YAjVO4XBH5X28Zn5RUY0WM6T&Qktk#nJ@c>rn;5qnp+cyS-B>4#ih0qX7{eDdkJ zi(v)f#(!jxn6R=KF=3rz^jD&#ON^Mr z8Sn;;6=4P#+M5K8i9QpgA$mealISwLCBxqk1%L4oP=>clP4w3-b$t1`hfk zpq2PgKq7j2fi+KNR`6sk=>g0AbW(uqI^RnaWX^q>?32EihZP!V1+6bz}mXRVQp!U4qDK1yNd)q-{--wpEbFB1TLys~Eit z?|%u>w!-`Hfgo+GAf7BpHHAsF0vt(F5p7 zdjLMOZB;Cd8kq-hb{D)<*z;tyMl&W)}T%ApPm;>|FQh!*edSR(r25Z!E@j?%sn@jzZC<$q;OTM+v^wtSr zJIao4s7*$TrAfTgZj#5DXeA{TFSeBH$||v}#d-Eypv3HZ zS`)H%cESl6&V~*eQhg&-nopdg3nG8vbDyOt?M)q|Z$*zoFRA*uEka^hf`93+cps4X z7p261vq}7YI?$rMu$|qX-=qyD!5^j&ZTgsypO;VwhEQO&>Pk#5={oN z54!XiguY!1ZauPK2DtbnLx0jPWj>PW9E8v!*_$Wl+1Z4qkX)?OIIF;tA}@os_Ct91 z9U{ch!Pq;$S_2oWwQ#+<2Kv-G z_>a0Ct?CAJs~fRU-GsB%Em*6bh3nK7yiwhTeX0-dP}}i7wF3v$PJcY8o`)}}JMd-o ze0)QV;1M+@S=6|cq4r2wYOj>1?v#qu&k2Rt1P*+G4Eu+|ERz@~6hbOSI~imYBZn|m zJaX!0p7z~FU#*ZOspz6L8x%-ZQ}B@o^2e|mamf+beFPegKmQN-FlwDw10B(o$5tkRWBB}>fY97V3-7j?C$|NafO9=r!y3PP&SW`7T|hIUuGGYkzmyA+|~xa5;RNsHO{1 z1I6;7UQR-E1qsnLBt+Lio_am3Rd0gL>Md}#+6R91cIZ_1K$qGNJ4q=nS08|HtAlW* zx*zUQ?}q{P01T=RCaq!?T!>i|_8>W#S;-7%|F8^)tfJp){Rn2)$e2CL#LVgZ+mGP_ zJ~VvP*~95w^nVy+8TxTnLI)i!jfRJqF~URKVDb-tXzCk!nW2;k_f{s{n*~lKlU+}a zVKy{V3-s3_dYwee$en`vQzFr$M4~4lM|}z=t51`j{hY#^U%;vAa}?e@5B2IVp@I4) z^;e{OzlK)z#iT22FdPLt#j{O>$OHS-qa0TcZD;?m)_?GWri2f3ljQg!3nlw4J^8fW z!x5O7?Vr;nj7ar=*_JngY-K;6Og{$U0wDz{1ETxySLk7{fVJ#@$|frXD~gE?K;z}2H#6of!u@X)f4oic$2$~ren8&)A23J#CwcFGL8baG zdGGgN8Lgj2>uYGeQT=yPR_BpvNHC@;>A{;YpF(-vgDYSy-bl=8Al_sTvpN%#hG9PW zZN~OOoimwwS>SPEQQ09P|3$jcW+aU|X*y?-Q-6-<0LTpJWW2c1OX=%w4rwmYwf@Vx z)B1`*$BZU1C>NnxE(L)-B9rJ9y-uo46o`QO39)(z6rx&|n;zq~K(1SciEbxMal2r) zTZKw@Ce*mIp}{=?*17Xxi+dupx+lRlcL9Xlg%EWYK@as8xQmk{RLI)vO1_0cyLWMt z5PzKuO`?+n_M?z3iW!(QV+RpZ9wtczIN2a1S!fa5KQjoIR$CTX^GKl|Io9ft_57+8 z#OI+`^|h~ED;R*uf(I&vRuDStcn}3Y`m8K}-szk|vj`t0%US{hXI+cy?WpJ3@ zx5eN_4QI>l>0ouw0NFj0I5LZ|P9y zxfj6`)IaTBCaAxPXtf(AU?HVWHeSIZ)Pv?{lA`~$TtbD2pQiTzr_zJ3(!=YJ4!`RAk(EHz+Nf@H)Pdtug{Nl?N#yVs|QFMxk( zHx5@B0g6Lc?(ggwaej;GyhA^q<-l^Bo96P&5tlnRnlHx+<8Wp2@FEal80is((V6GC zD2zTX3?&I;9@)4A0+>X=y=P1aRHemU9UJxwaABI8m9gD?3NA7ZuQ9||_J0NJP#V>?`vRw!&^6N=hVh-W=n;;qIt z#{eoLK$i56NC#-Y091#kr&^(fG&WkS#rm{Z;MlRaSzytCpGkwoC}2in(TGiH4v#{K zVTZ|wuM_QhxDL3KVuPzOLdiG4YPcG{NBz0fUk{R$N$Y&sOBj(>0{lNvO9KQ700008 z09|}ZvrwnP0RvrpNwXEHB?}Qi9+b=RV;wt^Cqv$Qz@&6Dl4SORAENuMph7 z&Vf<bQ(f>iegjT_S(H&SogglEi3(APOeCsYVuq+z#Y|P4M(?wn zVzxNl1p{J^3l537s+dQ`XSm>TG2bC-9Abe}EEJ0zVzEn{DV9)DOIYHo*{^%NCFp5Jp1$?nUU3Z{kM*v?%Q_FIH2HvJfw5!_J z>+9$aRCh#rdLrRibtux+6%2P(H$}P@FqqiZ(7K_abz6PIvfA}c>)N($sBLPjZ(O%& z+nR>f#+G^pY$b!-I&2yDh2tB1p}qivF{^9O+P1W*rEb->+Pbyt8(SN;Evs#8THo5x zh7B4~#Ndj8{B?C}wry;GY_4zFxUH>mbAtd*SaF1XJ%Mnj^!0^*;z+eqZ8}E?m&Mvr zDq?q|Q=2yVV)6C8eqS8nWJr^urtQ9+zUsbsFjU>@3;QEI*v6%UKr=FHgnURIq&&ie zHM9^>wj)8=>P#p@meG+l#mWlUfzlBPcQTk}(~!PYBW*i_U4796Rto(31SG$;1+Zz$LyE8?7gkk+{|tP_YY^L4}{(LD@2at6Dr!-05poAuKIv+lZJPax74XHbHu z@evs}BAo(easDoPUE`y=`si86dhwVlZa{RkVKx|x2Ro25l^G}#3&6Vfti+7W_35H471j8nr#MLAOnsJ&08iR2|L5-|~ zgzVU{+Se<8r;N55gF>Ait!wI%Bzf?^F!>V_B`1Kfc02YNvdsw$qUlIl6TwxBtJ7}C zV5B-4h)4HSw^FLt#Fk^Cfu2Y_V3KTIME`^|AjV8eGAzLAB>~%tT2CO}9q}i|ck8c2 zu#{@0scgNcH^JKNsRJ#v$VaJF7gktGlI*qEj*7N_6J3n3ugW}#A%u^ow@Et6byCzD z5-vd!eC|d(Tsu7N6D=F-|I3Ru9VF602|*n6+Y8l_Md+M~S^57z-k+)P}>6xX+M*r4|Y%+h7k3 z{|(`Ph)|#RzmYQ;br+3!_wkN7q-glxz?dEP#db9M(I`guSrzyf-QtfJIHmQ{(i=dl zL6pgw2sPRhC=&Ep;c{whvFeAcj!-0q5`YH-WXC26+8B-pqS3xyq-e4}dX}iSy8`k0 zL8c3ihX$;M-zH@FX(lm}V!0ts&~|g9%ndC!@FH94%ct+SHPc{Z zu3S01JhB2ySmV)vy|o?e?LaY`qK*j3v1o^G8%B|jvO|GzSG-&0_vqwHpVR^yiX)JJ zb|^&{Q(BXh)9nkdO>J6_WXCK;o11EO*M$)rXDB)J2tRJ19G`MV=A;<@52b`C)n znNIjME9#ScA=*Pb7|b|c=HS68sJAV;uMroP4{h}YLpT{3iRTDM6I`w#JQmDP9U+cI z(7#(1H>vDFRotxd4^(!$%I-vrMx@+-Xs}zd!}~(1jx-Jvv+1XC^KnVK6ByOi-q+a~ zh{mM!$wO%o?A#+gv{)ay``Nn9z@WkgxXsf|5orw`V5iOqb|k$e25anE+5jpkjSIJn z)CL(@3o>@^=ryV#4M)@ygDN{H+E!*9A}4AUOV0MBgwDY=oHXob*$UrbjNev&dtgou zDQzdwX=8?7RdI_{nIEa*R#og}aD&Yt_8i(9i1q|yF+?Bgp*7~q0?V>)2>1201>y@% z5=i9XQg|A+u5WIYKyRNwbA!_1iR?EAj& z*~-2zp~zD95JD6x6pD+x+;h)8cQIr7?LJd2QJiPE_G^ZvGF+BYicSyiyQ3VqKVy{ z&0kmq9~|W;ao8f?ZE@?16L%uSr*U$Y#HCvobcII611_@*8-^d6^V8$%aD2S(+SZZi zYF>-o^`&LoK3ytv(0m=P$D=8~_K2|K!DoCl*$vGy=DcB2FV3%vZ6@+dJwEt+!$bUW z$$}As^dTkHlS4Z5wMyb%ALcH}JWKjfMl+bZF=!Sn`Esym&WjrI#92>Wa85V%SJDfX z?FuJ^2K)Th3+qJ_B3NQVHHG)ZTo`w7Ev(FFc8O?weyS)9I-e2e5}`hOlzH5(V@u4# zO@+y1_L*xz?)f@LC$rbdj;ykOU#`x7`2C4{BUT-|jL+E3AMy*cNq(ci#wD|U_ch^f zwAmdWKXg2c@3S>tTX!1A9`kO`h0L$VuW5?+J(b})N9ib-)6V+z__CTbleyBwK8i`` z$qoV6=Qdu|N5`(1MyZwZf(s z#uEx-#xFHkKHV5B{_5wZ0=ttrvum1!i=U=V)6(@u8=XHE+^!`TCQ#n*;Jp(0M|Esh zS9MJ$VQ;JA5?NTy_UFqN)tOWqN6C+&UuOGDh0=B7Ha>XwGyjzkv}`eLdim3A#!^p& z=$kv&gxMOtOZ^c%L%|;G`-x}%^rVkaGfugfh(6?S7=Q? zQ%(r&`D+eqF8^-|syd~v$Ag7$Rb71ZCqwU9waw&i?uf^o;RlS3O`ff8$gvZ;wQk2F zVpFWws6hI%;E{TcIjNMlhj?0oUG7&UcNuQ zG_vuZ0KIIHA0Pet_ZOeb>b&=sn7MjuNTc{0M~~$0;{?*E<1M2n_J`guIqf`m{ABXd z<1Oce5`v5 zo--LGNlMaQ8L!bFs6Lekk9=LdSN6P)yhxGBxV1`7)}+(|W%DwC7hf|MST{(9*Lxe320{{Dvpj3;p?vUUn};QrWb zea`Gs?>`>iz&u)4D-#%Yrumulfx)Bgt8bKl*>EA4lb0rJm}5WW!6e^usxhQG?@4Hw z?&$ft+h6^qyh`gH)PK;{$u2ru_oKh6F6irP7q{}dchwvCmbK;+W1Rw2Jgz9WR;Sk- zPSa{A4lg;81>LpYpPFV~cklkC?j|n5`PxP0jB{_isw)>y%Zqa@yGEGtWM|xQiE!fi zHGWK)q!r;2PZPV8CM(tR>ZFYgKfCU_aF!zXKgDJET>bsmm2d2-&X761m>Mg3W8d25 zqo-q&!yn$t8CSB8cQz`^sL&o#Z$0!(&!yOpi)_V-&~ghd|6Ul=Fa$DetVEEG(0=&~cL@^5@3K+~RQn&(crp%6Wma%|c1 zob$mGTWs$?@{p^t;L!V(c&^(aK+(PO=!YJug`NQ6g09KcbAH!^mbmT+EzP$ncl~g` zEub|oGav9(_@YB+_?jK}j*V_!({I*sJuczE(0r7^juZJKI$8I& zRfLWUjO%~p4`KGPe^@&XkGoIraXK3$VHcncxi2f_$*`VvD(Mrb1fS9K%@59#JIGq0 zo2V0iy-#E~D^l+29T5t-<+wx5x7b2~?XRzi#(itxSrgZF;H(qP-uAYY-Wl7_dsb^R ze_W4TcluW8+}FD)dT%QKe;Zq?l0r8VyrE%Xq7dTF{@12f?cMaU~y%h&3XWyYd9FE-Vaz5uuwSA#V_+e|Akf;gWR72@O#mGe6 zb$Er+yx8?TG(MVLwN>W2!CSA^`?WlHV3omkfZd$ukL0OGs=qI$a~!5I!9hW!ywp$+ZUS_f1n7e9E$c~n!@`uv)z4FI(qGTmK&gVh46Q@fRefjnmyNadC z9c@=qrpDYpTh};ZsHn@QkV0FNdfqpxU=Jnz03o4ISkU~@5@b)|hPxB)B-GTI)|&2^ zab0LCe_LxBI^(lYeDl4KXy0SrERQcia_-KbbK_l~pH(8-o-OB9x?OqQ%h{GUCi!XD zx#sJUJZHZCCM0|awpVhszIZppe*W0CflEiu28LPGv9Xgv@9G5^JvjyHk6gkC_?GlR zf|XLqhR3<?t(md_%Zz)aS1JSFV4b?0}Zp7aH?^Q@Vj|=UMTNK!Rr@-H0V&vXIzEuOEHc2)}3NWLY12?DXlsANGm8Wd1%ka zO=nE?>&pur_WON)Xk41KYG#s6X4tq@uXQ=JW%j0+Vg~EY*#i+Za=0vSlQf@m8F3*) zQQASpk)+FCJS4B3$bFq>ru{BH$RxaUj5D&p{0wJGn77Gq{Y=Ffi8A~(+o(KRo$6xK z-qeW0=fakxb9-&sQZ(P5HHNliVcr&_@WLm(_eQae$`h{?z5Ww-UElQmpk~#K ztGJYHaTewc)4icT&PI$OwQ!%=^0jN+#Gu%WgMOJR8*`>_^XR?_BS`a5&5AH z$=e1^%nmS(uw038*PYj}PrIs;vTsexmW!%~zqU#$C9mBpc2LkwQv8W;ts94hK8@+( zjZL0qO7^9yS5d0sm9gi(b>gwk2#pWI9Rk|gd7gYU_)iPzrI<7KL&k}zDQr#KjNHMv7yhI#81 znvZ0#Y$LgdxAx+fZ-biOY(x7W2PP38h1;ze{gkx0|E=4z%#XGuU2i9cxb7?bI@*71 z-qHNtyo`)bRpO^HSuT4yvuJj+oFm|IA_dx4iq#yR-77b1*?U*h8r@ z^fNKVuAD>7BCG!L-bW32lTE4NAKbLG`ZC+5Q+vY&Ss7wO<`M$TFN5~5(HjSJj-;>R}#N9V3udh7nS8X!z^63*!9kqV9k=?^i zyOquN-R}>78pf28>9B=4Yi`I}zT>$4g^D%u(ZTcJ%Q)m_3?Hf56h%*}zTtXxqAS5l zU_|nf^)2N+x8y~KFIn2g5AmK=)ica@e<<&2uEF)Wif!55dR>%3!2G(JIuqq^xJEVg zujruTORyzZMQwcM(B^IU#BYxHJ;&tG3_SLsIt!Ro>eY(?v33U{)8 z;LaUr6FxwrHSGzBIsbO;N7Wn4oz3T3Uz(rTwSOcJYKooP+t_)m%=wL?bCT?qO3gFF z>#m7sMlstrw`xr^PgrC>o6XdgaUXD%|FQ1H3GK4{JVEES=BUjh^AeGFH3JKN=a4G7 zlFf#d1P*`7SbOi-hrpuU8Q}j2A%q$bMU5bu8~7lW^ItA<%jN}?!9b64_0|O_^uf{?r=k?fH}XgmgttS z?qy@o<iuupzFV}EUkB8eXVY5b1%F?cyi6&el+j0Z$tH2n(J>fjug$HgowHA7t6lkJ_Ts+b*#RfkCjG(_EsyO9xjz)5<=e`hYTT4d(OTLY z-eAPZK5fA@a;3uECb`xA#?c0~0j1iB8&_5BJ!3j3X&nTAp&R`UfuCh=Hu3pyx$aSX z>r;vIAC=i#Csh+amkJSSh5+Xm4l4=?=L(SBY`^E%7fb z)V`6ontv_m4G#)6zaH+aJa9W?3mLa=gesNi(c+;(##kpp;I(z@WZPJpTY*JYMbW z6}v@m+dFn|qvRJdY11<&s&xyB)Xqc=dyhOdicXEc4}a4udV2i9p|IoeYTdMmcl*i# z=R0ZrovKyv1Y7s=>izsj$^*m~UqTKI$=c)JPcG_+Ox>D$w@M<>=qLQ=a9C4DLW^_4 z^&)Nm%6BLC92?&8qw2IG9O!<}+Fig;=-0WIrJpmXK{zU@WH&BD-17jlt&SKWCdf3` zgebT8ef@~1+FNr?7JZlEuvm4P$)~jV#S@|rGlPDx9ANdbY`c^8*t2i*1jkN?6Mh;) z3y3w&@s!{3E+M6IU!Li=;v5uwC7$BqkMZOa3aBAD{4qIzniup2j<%s3F~sqZ{I)*gnj3$vaOv`@RA*)Ou0Xs<);Zc_<<|U2 zT)7*2yVftJRu{JRi`p3*&&&7axVlVk{93~jb3-8%S1z*q!{Gvw!v={IQN4pbMi9>! zOP(?NmoX&|f}^EOyvXsnBZW`5iW#J+M)}DYW`=iq@>;Tv_BTkLsOYeW)|q5(Fg_H1 zyN0?j=0D}<$v?Vn415pKWzRp}v{jEnf7H5h>#`5sH-IeS`RZlebVZXQ3%#uNnM#_v z@ZD=?{_^CP#uW(%kwnibj`o?Z-3BRM{)BB=@2#S0#dY#CPqKFz9n~n@Ncf_nLh|cy z-fhh-VfFdx7T?76bJ8xI9De#g2P_`3^*McFJ0$B@^StYqTce*Sc4)#*`9pnw za@|gTW~=*Km3q#kGP~=u7rIBV@PE|R?~!Y3N&dQcO|;IXrnljWSmau`K!=TCTa09$1?W= z-#w|qNzSHqW2-F0Xy&kf8Yq<9lXEh6t*h0gUjx=EvqJe!#bXV7d%|6+s3WtI)vw4e za~&tL&3#P62KMo{y=fP*8>>zjj_33|Um3z`9?@m4*E}wySN7R+=6dq8YOADJ&d5H^ zg3RQ3$&h6~@E7S$n|?sNxv=}z58j@!@40BhO-h<$`g(rE$b`X$x7>QnQzkbgHa=%E z^v_mW8`BxP;m@I-Ho}d$9kT7$dPM%DUYwJ1X{R~K$20HU4V@NBxwpYA@!~+2cWur@ zr+6NdfCigM*XP&rF0{rvi|OFoa{-g7uj)3%7G$Mrl67gRnF$M%gI^c#*)DLLr?U?)N~j3SJm0taSqxoSeP8Y*%QO^tU$T= z`59#;C!eYr7sa*LGd4CT424TA>@jIJTOFHjLgS&pIJYpNJ={h1+N7KO1%oAzLI%9R z9~+YTmd>8{=B(&zp-Cv(8_XJhx2rQGQTR35^@&e~Hxobz^aNbtD%210>`$2wVcp=T zt-*IKfnapic35HL5O>RV{yC?o;jSZA$BOF5)VINYwi<3CO(v{ocN-U0E_)PNJqQ+h zObeLV6n#pwqbg!SHdwi?F14~cv#U$7x|t_lSQdPq9`p{8J3cl4j%~V-L}q;UF!Ret8WNuvkf|9d+KGy2p{QSGh4c+4(_ZR4_AM& z8Rg)v7N+M{-(KxX9QtXlR}(*OKHZW0<-}&%d6&;K2DzU<92d}$?9}ZHZ0MD~(+9=! z=67BWd?{;W<6nP*@Br+{nTick^Ca*v_0S<;~K3$iuRkHT`-Z2T- zbmCO5(eha>+of>Dcjrd=N#19=_A8qx!0xejgH4Bnx@5FRif5|tNVTr#we%g8FB6Pz zpndDQEQ_!3P}Ob_dp(N3u}U_rGbnS&TWMKzxkXD;UW8v`VyEPI@AIs~I%-kA?S)&V zrtOukw48deb^1t=-cGKBeU9KYh8T{FUs|Pi>rL%47suTZ<0#k040UaG&8O>$QxewZ zByF|q4;t~w{fC)ymt`$eraMm;GVLDRq8LFleDVIpu{dV6mo9sHrLKQn@91Fa?RI`2 zc%+cl7aExrTs+?|5XJhOxERUtcweN_lv@51jx#&Hygj9UW~U*+J?vbvr*$v*Hr>k8 z7LF!vN;7FyqiX7Y%SXlL^`76CsTB#`A8GtU>!WtAcO1**(B|jq+HU(r*LOcZ5|6)r zE^35!JA~q!OqverDTL2!E%MgCt?-vy-Inb`c`>8>_G6X5ZZk>Ey7fk?54lmY@%aN2 zR-56eF+-09qh|b;VBHR{nQgIYFQk=XICx7srnffM-&Fjnmcloq^OcWUBBB=L8`vnr z^R4s8412V~!x-JQBR=_kDVhF;%_31hLQ0?CrhWAPE?wl1)-_`iZTgw|u>|wS=?eGO8gHK7A_3Z6dKC@ zIF;boq_kWwzjE24x$~#gqIdl+pKV5op;LX6O)_ojgS4ddS;B(g_|d-S=3kc%_OEv> zSlyg(b=9}Q0jlxM3!)2p24fbJ#mA)k4W@Hu zltf-vmvMY2dufk8Xx(lmYNMR9Tl|vcOmD;I&8E+fy?jOHf4cK3<#!0r&q4Xc^Y(K# zAIADb{nZU=Y;levwJnFYm0G&$V_q?X>?r>J82vR zZO>oWsA*a`Gji3_Bn|gy)h@FmIXpgxkGh>4ceA}>R7o2h|)Ulx>x;PO}b6k*WEf-?_i~&8GAoTOGWwnsa$X%!GZu zQQY>2r$*YYjghgjMs1<>0?9w_0g^4J2TUpRpqf?#*Rjs-YH}~ z^xN-w`1%~mGZ~Q&99=gFJL(5&&n$n*Ya=nyS9`>@lY8# zqY>QMB%k9rYDi5yW|S(Ze^~HcV@~d<4E{jQ^-*V9$of0&qccTPiA#5TlJ<)yEtwF$ zNfW-MeRm&zdNT+^2UwH%*CK zdwg`CN-nKbM<&KAVoEfX$-8N2+UR@N`L~8SDn;)3YsPxx&c*$_5!D;VAI16td?Coq zj~1(1(q5%5eW^Pw?Z~y%cj&DVNBgRn-#F2)yf?lqpZ&uk`fLv~NoQXMX>1O^xFn?) z$oI)S{3-kM^C1 zPOE%Rvi7-Hc7F0A^Z0`uMvDqn5?ic$ay*7NoGkZpn*Xx4(WQ8(Y`aEx;%|Y^~u9mkH+xJm(kNN8yU7v1^U8Y$-IkH~m%aMJbKU*|> zIv;gxz5h8r-?qkD$p77E(S}dcA_vn

~PT$fNCNbZ%U+b9KBg75k0L+YemAIK{%X zhCgn8EsZ&$^VPvI;aHo>pQoCfg~d%9d*s_Ae=Zojo!9l}rLlI;nRdng)`|ULsBx-| zd}YzsU;K}V*4+2OWZDmfurY~}x?k#ZU;dQHMe&~dDRZ5d?WjuKgPRuZFQi%7S;#Gh4`1|ylsJqn_Sn)JF zZ`fwa_h>}rT;Z=RF15OA`Ia;DJ6>$nkx+1$I=PkojMC(nedkiu`Dv{CE_w79+&UG1 z$)htS-JtN3?e9d7+Yh zweLh?bYW`jFSg*q^Sy7U)T26f+Ng5AR^LthFda zaiKTx=)$`j{euoS6O9V5-4#^lDJw;g&WZwKr6~bL;w|jbCnm~)0+@b6)mimnXSf3{+~~fayN!L~g2QMDxBF)4 z;L~*;AYFlkYKt^*bY+tt!h}d`(&W z<}D@?PMb~nh!X5Y4|)%Byt$P~F}v)q=G;7=$F?DG-DjgOa#|xEb&j2D?(Fe>Sn=cI z7z3nQ0_VRevhF6m}Gin0U!xOaj*x5L4 zU!NWS@YArMOI*9z#*^EXdzHlQbYayjf%Hf(Fl^Vw?+ukt0Qb5|uFTuQPn zyH{v&aiY}Xr2Zl0D5`5qZ@vYMZ;6twB>D7`-IbGizpj?mrGLfMQ&ej|X9WA5^Dt^1 zPml9+3A!Eo)!NiTU3g8lx7u)Pz^=Pp>m+hiDCuH&vFR&Z)0cXprzG1)2Zml!x0H2x zOHTh>7o4Kk{G7R|u*UrR!QSVa9V!HKBI1>|ugN=-DeJ7G;-Go?+inxzv|if&){f!k zFpbA{c&lgRvEi}llH6CeuvG`IhnC)H)_ry%SHfgW`=&n|1aL%Ge%4E|KUaCl+I;zS zr~NpK{7LZ%PBp1#T4oie=Sx@Tyl!Z@(s8npuY0P9$@z1GXM;h-X}$crxS!l;dRAtcGcF7#S-DOUAKS$&j zugE2qAMr|Zc(lh`W_0le>FOwCx##G<)V!eYYg{d+2eRcZ!|D#df>bl+woX1wqB%If z-{;SYpPC-v)w`bEI2yKfs{Bk&Yfn@9J;Nj238UTb49A3o$*~poZwk|_A79Um$sRlG zmD+9I`eIw)R8RX$|KiHcncohtDVpeBFSpEoBqvY2CQMo~+OLnzsW3e0d)(r|l|Souv2aN|L$I2y^sj9}>jYt-1DE z%el{-EX$Ki^SrLR)S{#;9V%OF?ygq}Y!0vMvPms6Zpa#85k9}xK;qv1ZH23XxgXZu zUR7~sdmLNcvZD7vBGanx8>by|iYND?&&v6y9NE!E$Tt!b+K5=}AfOHHWmT*Z^ve-z zH2#qhMo^6}gz(uYsBxDlKAMEWrls*u(5LE+?Xvi%80@qP-kcT1zSO}RKxj+E640>& zDTWTd4<6pf<8U=ha1wzk2CK&TV)h__hKh+$g>ke+MZ;VUuh0T}N|L}4R^coB^5dkH=s!$w!&t1vf&yd&m?g5FjC_Ra=NR^wUV5><%1 zG3p|IFV^vgDts&lDsWpfzL|g;zvq_c`XIop3z&c6#d5~*6xdGgLtU1y%*n1Mte*c#7knXJo^}5 zjXBV12tUq>YN2!X{o4+p#cyWRfru83+s5(lFda|1i~eQ-*keqv#xDXYzAV0OaPx3PB|#iDpl>xK)kvDguf_VYe~15y>JG1+!T(0pv6}tg$a2i%Z(`;= z_=Yc~`}QY(AJ+Bhmt6@5h&UXL10EA1$--tjz@uya;PbJ8xz9}agL!3uOsK-_OJF5r zbD^DZiV( zUW_1EGokKoUSsgl8HgYgF{=;}jXHY?p%^#o3w{S2GYXVY1BPWcuhcCBIShj%xEjPlAwY| z=h0P7EomGN4%eg%zfQx8!A?dHJN(K7VrhI-MUccKEVL3DnNYskPC@_=>Pr40!Ue4N zjuAozrlrG6LI*SY2QPJ3#<@*^dmlqMoCC}?M^J|~EFkH|h&P0Fm>n(CghZ@m`jOCq z4N8_|s$3~>!YK+8lCtpJ2!XxPY>v>*jJj|s8Hy+XQ3qmDYKWi$f3k$Q8g0K3%J68P z!)&RrC_tYD|J0E_l^T`46Vx$hc`Xv|U`opUBJ|Ro^@lKqVISj}ZeV5|<73(_jXLIB zWI%k001mgT4of8wWZ?m8kexQ@F;xnn_Wy8XI)@1?c46Yd#9Vb{dV!6$?~bsU9@F%r#_jxIzbbECrLz(I~u+$xIei%%_NHH`dF*S*EL)rD>&1Iao`qg6SjH zAA23sCKuGwXs2@I4&G+qXLX?BgmeVWTL&aCYb90i1~IlqI9DvnYf4zqXC8J zi*;AXO8kMr^4W;om{%`x5$U3Pe0BDr>`Pq6y}xEN$X%5(+zNMAX1~?=T}qVUE(Z zBDS-j*d==+6WVHUBc_p2P=*h207Lq0BR<4pGFflBgpUP>bK`+Ew~#T9_4`09aM3m* zOXL0>#8sG57bA&Im{~ugh=F8uY)?&n-eACs!_}{b9cRGUN(z#w@NQp-*myRM=)jB8 za7ho$Jtp9Ah9qRk_)EjqsCSGwfyIqv4sn2hQrlDo-f7{-;cl%)f=(4K*#ZH+V?;7M z=L?Y|YlA^bXxD)fBK`Sv5t4wO{5>4$R(f8~6bP!8fia4A;91%xUhJ6AdYUA7WqyjE9 zVq7?GXdMK;OXNYLEvKD{(%9rhDrZ5lHJ47^*8_ttOh$6MDtsmgq>5{SBnzy|rC^dH z*4hwC8pc`*yGY}hwon9VwJ5i>xZV zWeb@FzPuaaZ}e~{gVGgo$9qq*K4!Z77IHae<^z9nI@a0$@8d^Oa47t zBp~#9P6-cP$HYAlD{LACksE6}$efs4j&CC+$CSesMYV2FJR_B8z9q*ozqXqOISq$Pm;G| z^O@-bIp-Ye;!{~HSE#6wj`=JXusB#MX5l8HFpg4|G;C6yDq{)7(kHowB?rS6U1gCa zp`jRVyJeFa2-Gl6bXqn3Y+|v);>Wg&r3-UD=WUiLOv|Mn7B4K>dJMB%!dfiTELX7M zYMo_KfY9Ejf3V!fYRXzXg@Ey1W2UHK8ch-@xtQbwWC{gqrL$1NuznXgDPJ&-kGzyP zZd6{F0tLbbG){@Ko*jkV(WQW59U0`^1{8HeG$dlH1H$XTl0w0w!(!G>5roJ0QtqH+ z)zOsIn8RKtQmmv!EMt^MSQYhhl5z=~LPzH*{scB8TU2D-k1z%yr3WYe#LFS604_NUCXmrriUVdq z)&gY`lO9c=ZW2MY=o6#zVeWb(PMybM_^Aw)o)_EXsgbOx$(i;K`sG-0xEy{oi=vq( zY&CT)wk5k_OjW@G=e0RC3R}qC?Wj~NrzRBlYOV*GIWQwdq$>Q%lS+mqvLJ!RCVMJ< z4d-;C=3?Hc@T59p^X8W~b(#g$ayp1=hGA89QMY2=J+_bffPw~BLZjkt8$1ptLx7!T z@v=ZSF4+DMm94QRh3bWkJKG^DH@3w)*?m)-11RIcjLni`$05$f9of`X*t9rxoT`G! zzf(Znh^<;8=c)c!i&jZ(!n|cvP30k>o@q5B@R?jtMxUvt#$nWA zSE(~t3u>Sq#jGp3L8aHu)veTZnD@SSQhhP3XEzo6wKfuV;yu(47!2zFpOoc$)CD{R z8FgjfKZ5lz$3vvqZg)!32L&7cnjQ!TY1TV4|_(VMz+B>Pb7+ z_h?em!?WJPX#WscPhm0(NUSlKK716`n^@UHy^uYZ!UVQ2SpIYZL(|B{%6b;_XB|5$ zNP&otBY0RJVl8Vy);E}xRU)jG*q(kLub{>l*fnTD=*Af4ECZE5i6ko<=8{@@R&hmC z%%B5n02VyDo~&%xCdDR<^&1w%rh8dyFwyfdtUOqYyr0z@C;%mDux;Ow1ky_!Je11a$vBJ z7g>FgroGk_R2E}jHeh~82W<@SL&hjE zrvPMuHe3ZDI}~+Z09ubWJ_$gZP}EvM$O&x#hzMLS2kGq*oLdCVYod@3 z3v%Lcg7oweEG!23qN{rL?o5joSsX4LP8EaPBoP9OJw=`|6dZ0hs8U@)B>(THG%3YB z$DAOPlkG1L8;V1+D0L-yeV3~Y4ma(H!&&~16#QSB6z)%_UIo9CgCLap?{|CPaK;SO zGKc8Y!mx}4B#2U@pRq22!!4NNa5@as?=w+qjw2GVlPFzw?xsJ6(I5zq5^y+825Pqo zx@=bXjRYiu`U5Wsi4a7<=UJf!QFClAa{pD6;Zr6UL|I;?b08l&L4wzn(>oXX+1Ws4 z$KkfA;Bd+e9HLVwhshKV{6`W(Yi*ca3Ut2n8;UsgO%(Q%q9b-8h?qq>f}B&bNaAp^ z!xMO!s6Z$S4hfGAgvM|grSVzjg;}NP%)-(D5&8>3z)sT8Y6)aCcMnwv4GDp)X@|oh zpPTW&pVFk8@L6fd93>J@Za8KI+?ESchWh^~!2hL5Z(zI(WP*;yU`Ii3tN>u)X zDwlP4hKBE@>81hR%eBt9X6JSxKUA}azK=nC5A#yR^d&Scv1KO zh(i>A1i}AU`yV_wc@>2BhtP*p_wITMo(+dH0@1f>r8mSOGBztrQmASrU(_ENfw21u z{#h`P?sP#&rQuU9cxl*SD_!*6AMe%N0IeDDw<-fY)eWP^D?mc1mSPG(%PMya>a73? zpdJ1_@x|e683gqDW2CBz5GQg5ERRb0h)DSuw1SdGr99uNpqvNP$O38@F-Uo^3nN|H z1w6)~ME6*WB9PJ+fuWWa!H^+|OI!&+RbmjR402|yg!Y($Km+z8P+j0m?0!nFoAw4;BfeXGF`8i%Al|3X&6fM zFrFPYQ=y|ARRC(}7=}8p0tuqZwIa|I7=v=JqC-_z0qCnj1gZlku7bGHdBJ=?FH;kQ zyb%a_4F;8e!F8)3Q&eMe_sEnt@T52Jq!t5Fd<7y;9y`u9QH6L>p?^=jaX1qOdSy65 z6+-vmaEB_;XTEBMpB;V?NjFPM4WPoB5tKA?+O2_V@%JbkgchUJ))i9pU>uulcy%CU zwjD#6sY5ELb9N%oj;<93R`|L)eM;R!P|xpRDCB$`7Sw=vQ7&~2fJ*QE4+f8A_2j!6b*^x}W#<==Dqe~)aWCtfl*CU z7z&)(tD?|l1nT?_gC@MgOZ@FXM#N~L8fg0q{L^L7+I9x6(}q^VjcXwE>=eGY21v1+ z#h{{EkTfdWTnj+?<}v6oEl2{jQ(*_b(h|g|KL-wH%^>#4H;j~X0VLoaW4gzk6I+MH zfwy0RNoK}CAODlC#!oya?2$uv_TNhc9L}79ei5$KhMZ6(n$3j;%Yo3(KTS zgCvo=hgGN*yL5mRL*x}GH{8Zb;Dqn%(5J_Q#$5?DE|4a`Ld0m#K^U(KxuQZmb%79m zvHu7umLQPe%es&}s?Sx+H&H>HINU>!ycx4wfCM8Ggn9<^uO1L8t+9gVg0uDLi9n;; zdvF&B(}Tc9MtNu8c0GFW$F2|fL$)FOIxry+;zm_DF!?@<9k2~D;cx~F{`(RNzczrh z;U0bZu)jnEc<#iY;8H{!RcoUGfQCjOP#w53g207pj825VS0Vp@Y{m@eWApby1c$R? zP@pmf>uNfN!2LHs;&|S)RDs~o1tkaL0IeBI7$S6;!z^Kcz`OTQT}IZNS;c?3F;|m(+19Mr0VBW$hFO25U!=sjGlF>y+zOH5-y;Mzc(*xy ztZC-JQ!X$6g9n#UBzVA_K2I2Ks~Cg|!gve%CdJDGbX~eM>a4jxiQ0KJ}dL|BsAyEHC*xw2S zME)Xy7w%h27xN53sr*7vy0DrR#Dh&a;*kA85DZ5^05b-}ci7L0KIOj7LM<*}l$@0Q zEzzFg;n(XRC0No5Tn-8onD}5(YkGew)}X&Uz7;$hoL~)d2r}w_Z_vPD8G~CXdMorq z*i4TJg5Up7-Va3HAIJaTb)1+4U@sebe>-ddzSIT58^HBJ^eT-_T;p~Oh@Cc&$-$Ok z<)<{M$rU42TL(#?9(7s=1f2FnpvEw`l4FI3P3en9{2@oO7Yk4pBy#5!8Axzt!(;29 zt*9%f6r9_VL7BG>cxnR!kr(X09_(?E-A>zj(1{Jmeu$PC5lrlV6V>0_y?+y%;vS4X zbPu>V>x-eUe1EU|{w*zsV-b3N_@^yB1i9=$uK`5}R26QwgNU%79o<3O-NFO5g269B zQY3@cLuXg$u~Kssk&;!q0wu!A8z3wiZ8rdvOvQgtTPm2i;i?Vv!D&M9ymc#hX4s*Q zi3GFT13ZWbSj-;aw-e#IO(2+RZ6IbCTr+aOM@;B>f>uM){|&-)=l@%ERN=Gs;E)0l zGK&aFVPzD;0-HJjQ<1Qfy`mfwCyK*OfXu`gcAV`0WpV)f6+~XM1CaMx`9COdX-S5~ zRhUWW-Wny6as)(5j*LWnaH1pKvTOuz=ER6+ho>AN1(cu53E*>`8S!jzh!Yq(M2Q0k ze%AXx_&ZJ@U=h?i1m#7;Q0f~&&5J;NHUj8q$_kVdej`U;yUSQksM&zA<^f~P=m6Oq zgnkX|mjeux*hC*SwM{@&$4Ly8u!){BsB8zE_kbP5E8t4T;vo*M-9#@Qej}{wD-c#~ z*vXl`EeUl7P^lX$P!inb4Dq28V-!J2b|R?Nu(At1a7|qRbeRN4c!9V8x47IexQ!Kr zYh36Wwjy{l3L~Bbi@MT_4P#e;KgaqX{3I(;7(HY`du>JV#cV5hLAcg~E?UJ6pssLX zs0l72Km5*(KJ%D2Wo?lH(Fg%wGH#LX^P%*4{6sd49E^w||DRRmII1F~${Hek8oar#p6_cHz8 zYETc>@S#_O3;R1Sdz6DeWNMN-g_{D5SJR5w-1QhKEUdb2W<#QPE$|$M` zLH#(%h=P!7d2DMC*eQ1iIStW}`nOJFyR^c@4mHU%uxjQNXv%s4{@^|=?KNN}z@!1OM?NK4{eoK&K1UGJ_r{;bH+$By2QzS@i=mDw} za!!e}F>o3HGTFhzVI1LZZNQ*U(u9u(KoY19>?BTIB@o*2{QvsxJNzI3vO)!lS``@# zfe>&5L2bsM=pJ)$x5A`|yiAZoi)w~f3k;+e;Io0?)Ey~AW&(lohrKXX@M1v_ooXn` zGYI%>dm!w$4Kji+2a?EeTqS)V+80vJP64wX1GDTIMB_pbdUMz|gan?6f&B(*nnttL zaumn|e}LwUsc#ePyAAS0`)l3?`cvJ7@e2h*qG&;HW~i(+3{FJ!Kw@ItN^nOYq=xXZ zU{Izb3HIfI+g59VMtUIl7_*HH+z|}9qtPo80t}u%PJ%aPk__QJA@o5^N5Gc3C>Sg; z$Zv>yJ|c~vLK3XmMpB213&Eu+ax#w&g}V}|r2!aa1(F+M*8Exgmw^j53Z?h)emfAx zcb<-OI8Ty?eRhDXhs?LI9RSB${ujp$`<9bP*b;>hj3ETa>*xfkp^y~Hw=ooe?}IaY zOufB175$IzBTvT4WKOU81&L^NCr*spGscSo`V6q17?W@gIDG}F;dA0ND?It5p%Jz z0i?eGDWlq}wqT^kB0wl0vx4cC;9*rjdK=6PMqfXH2O}USl<3X4=&P>HU~e?=ZzH&dbNRnjCEslJJRVY_(3FCe$ccM2@KrP0c%Es(;e`Z2HR!+eMJNA zTNn)Ng(ITCQj4&*MgdmUI~Z%&9kL*-w1@8ZzYlBvO(RNoF?#UI20K-(-2rDC~@^- zAUSk;`^5kOuK5^LG8VilF3e})L^ppZ{uzSDoms)_pis|P0Oc!Pf%3vvV(EMCRZ+t3 zyuffvzJJS|Gi59E%XR*JN5(9oVpN{I()c`g#*43OJ8KZ zX-XbK-WV8wIl(w0{P5g<`iR)YAr9+B_|@RNIC{lgg+R9q!1MR}geHw2FDku4?TvDB?utefRo7Aia%PtWx4<3V2^e_$x>17QD$ zM7{3;0Byj-nlv!{v&@(2jCG?1$p&s&?BSD1Lh6zC4 z3=u)8!Kn%KSy6>RQ^*KZ2mYNvpOzBP^|jm}j6Z^!nlZ6v!x}XDVv$4xtiL%iR%0%p z;K~G!bgJx||G-G!WE%1T~fTACzbcNCAl2`~opw z_JR=KFY<4_p$>0Mq1PK#DS)3^v%)U{k7R>Hgzz7}=X}Zy%*AWK?~I31;_WN+>~P#c zurMR46dnYkUiAJ4_4Ob$i(B-0@&babBj-?qOF>=g%gGp?qCuwOc~5hc2n@%3IR$l|#m9Xl~N z=N&wd4lZwzaf)3#Fr5MFgDqeMHDe%pA%k>k1ZQT@OT3Z{p!IEe1gZ=#%2PSed5?DW z_u<~ZNznk7%A}u{?8*fEH;fT}CHR9eRSB)3Q4h`{s44Rm6d$Z+K_$TsN9cp&a|Gah z>@a+`9aR#$qeBe%`|j`GN_B@lM*qK;fHJ76LDw_i=zzfd_5U{6{IGu($a{#;fTJ($ z7l2nLfL9p(d&zS}sL0ASG`f2JP8L<%8|E&`W^74;BBN z6Co)WJ$Q}Cg1%jZ!P`YDY=4aIW$$A^D9<5AA|dz^*!QEW!QWSn|CT6<*(*$(FnH&P z9TuQNYkM3BJ#%~ouY`J`1%cixz@UE$s8X2zi0`|v_!@J7Wo|P#2w*&|l{$~n$DRQD zCxrAM*O$ydFa;vOP%+kAhbrNYJP>B;T~s36QVC+UxDZ=C@^V1W7S;bj+vPw)sJOq6 zEdSlmdemd|$csxDytpKz;!viqeCH+-fDWQyDR5>OdYP*$OtLUZfnFYa=7RpY8&*&{ zs1(TKP+cvUtEOQ1%^9Sezp+Bf3s0Y;FW+q2n$|x9J=%ewW9+rQ6{821^S~u1>01F? zr`r#h0e$NKo94edSLlV{8z(8sqMp*MiD^60D7gwo9;CxGY) zFlLNtOBQa=1CJw+sON+ctNUTi(_n$y0)9cE@dy+T3Ryao1%3!aJpXxe>YopmwWSCD zApq}Bg;8Fwe89``@IR=!|Do<9DCx&5DDq#w*?z5JZv!@!0A;~`Z{?>nscs0P|1d;l zN8iw(A^-OU>%Sq9(vJQC6;&vRTgD;Gfi+IkPvqQxxsE*revJkW zTes3Xa?3LR8j+_A-#tygb^mc1SfTb78y&y5RFVHH>sn)@Dxz>NrH$!KpS#7T3tbng z6vPs#rlz%ZTO|ZqphW%{3C|hyjTKG;%_U?RHt{Yu_t1RjHCDlE(fZ zm5M1TEHM%ix=rai_?F!;>&{K+p0Y<{R5b8?mru8T}GzZ)CJLNS-L1###YrS>WLU+=(`-M^B@O|=ZFFY zDb-(&1m1OFdif9wU%hGAwUX15$QMROA7XAdwTqUKg0!?!h_5lZQiFp8kF?WIUL*6JUpW66=vG7Z^%=x<6Ly_jP5QF;CxsrS z*Y=A{it`~O3!OhwRAGk^^^1&XKEy(m9GnpQjR68(G=%%!+QOT{jyBCG}R9+&tK9@Ke~iLiU!td zUDW22HCh`D`ed03&;m7}>L#wdw0PZ}lYaU1lKe5M^kYtBiuy}_tXj@7LFHG_Gbj8| z*h)Y8<#L5yW7Grm=D1i!xdEW#R27gmJCoTHn)!~r)3~^u2lReG)@e`DKmg40UB3zX z_=d2NB?twls5U5{)qbJ2psdpVp@E<*RgFys(OBsfWr~BGQ3Q`Id_08WtK-Us6{?ut z3&|4gS2_`rFQmUXA?TF}v7C`SS{jzc+C8#|<)aQZF?Z~EJ%85Dn*m3{LfpLvT{o0b zD|L;3#e@~xS^e@JoQ?^ajfoZ1AHqlclVJ>>mOll>GUYO|Mr1|$@0pT+71J9LxYS6; kB68D`Ba)6sq?M*3@+p)#qH?qLJ-r>3Ycr<`vFl9xAA!D0eEQ6axPdSiP6(FGy!2WFt;Zn)Sbdd7@fOi@)JxnSU4xIEqGJw9n-+$bJWX(i_hJa7CgFyIC;0+=AAO9WL(}aZozjpg~o#g*D zXYPJOf=d;F{GUnxipfsl5s?0W&;Io#8B+d#mlFSvlu*3?(FjW6U$)di?IZo`L`xDn zBn0w59Ra-$OSnsLFtGm2U=S=x<{lvuRm%b9AMpQF8`yC$|M_f7bdK_`nlAUz&K?5{ ztbjVRcmkX_Rka!+6OI@Oy(Np-6XQQcubw&<0R$M>8C2$g9L)bHY>1KYQuW!8fveiF zxV`Go_6A{pAXJiQ)TwNnL6HPIk=JtxECSQN>+WV3hHVZ)@76^oeXL@s(=Y~T1Xgc8 zX4Y?7KR-{$RKQr9^d-R)iPn4^%&VjD1=%I0kyIR)907Ruxg06oBdQxa=UyeIH9h<| zA4#BV$>gX_autkZwz}m&>}Mfi;1NjF2XaTl?hM%vHd-e5S~W(NHDA&J@Py~)xQvNp zX)JBVBeaEkC(uW_AUC1|j{5AwH2rnPRtcNprUlnJ%_X8X`xbt$66VOJ%VFl@dVYW( z<17qqR#hH@L8w$y!|EXrZ;p}6r1oSsdl}y{H>7KeT{ZV!`CAe;rt{bwIHO$^f{GN6 zD9S-E)r0lPkJR#`DMozvA#qEg4N1tDCI_Ib^F&#{n1!&#PrW9{lW7}|^HXgyqp75J zffr3+0Zt`%z)dlKOODzJvDuSYkp z3#godu*zqcZ0+Y#-zQ+er&gia1=C7H$KU2OCy9^fR^}{ym49HYGtPRA*g9qBcdpOV zkMNXfrexw=DU8m_m#kA1(^NVV(G}Q8%v((b24)N%a*Pc*QUq0sf*jYf*Up9vP*qKbi!^d9-JiD~=+oD+e0qD>9 z!|hjBIR@jmrM$R9zMjw0p1gfLnA1~zLgKe|IXo7qcNO&X-2}CJkIpOI;>Tl96bb6$ zA-Y|VW}{MyqgQaE(1}@M3vKfn&XWa>e~ge);tya*TbBvb=84(0Lytv&$`$^_6O&Mo zlKt3wcZ4eLCBBZHhTfsI8HzW|j) z(x_~ag}T33SH>xy2F-;w&VP&ffEf-A7Jtic0v~@Co(PD_s@8hXHKKsT+-Uf`T?soB ze!GBJJ0r|H2YwsM|4!SY_RWaB6eJwGBk}_t$rs4n1mLpn{j6FtaN~)m^SBf4Z8#!% zJBwU;_zFRF0mTOt9fejk%epiijj(j4qh@^2O+?BHYu=mU(6wxQ3*co%4}cnseNU1X zR=-!r0wbkRzq<-G~PPyh-NJX{KLfqYubEs_3--tbKgQ*dlxCmpFdBhn=UTDoW^ z3PBi_v7(Z}MN*2_LDD)5qA4plHnAQUBS|<3p%lDNKy1oekW`^Ga9zA@Qy22sn5TvT-Naet+$4#jx zNRDQ}&NCOb$ZsElye~?0YU-lL6RGrjy~2ORp8?}of5*_+S<)OqvR-yAYX^g;f@hU; zuTs%^&K|CqngCULuqhj)YvK!xDykmQfijmEWy1@QL_&9h@bpBaDriJnbhz2}Uh;JQJficqztL@O9ojsVkP?IrL^WvO6{kCDl6fPTTA?XB z4LZ6uP0H%bjMOF9BvycWrAe#7HXd;LT^fFNQCD;fMB||^bIiHbaj_;jd+B&lWVr-! zFIOhxvW<{;-wBxpHt;LNva-v}6(^z!)RS^^#+oB2yV>O+nD`S4a1f}c0>Dv}#7Y_P zD5(pBRFD6D$HrwT>#HEI7)#e=5#Wgb=Hh5TGC0kLz722Zeh>__G24jg=BgS^etXLb zMmz%W6uZQ~ChI1JDJ5jL?K`FGD5aRPR#Qz+DNi+lX%U3a^^`SpKoYhduQdDOZ-{3& zB>n{+r|r>}1X&gB0rz~1A&hv@#8Fi<`y@`?9--LpzmBLE2o%Sve9q$t;=x;E_F)hV zb4njBE-_7I<4u8ehIr&Z>#x`iTOiunq?y{;no}GMX@<}^UD`}~%N_JKr2dE20oETJ3JVL-(PqQgLAO*yb@@Ej!r&3K(rjaSFk zBkD@UhN>cIBYO)goRmVijRhsbq2GEQh7ICCr@a=y(zL z5W=I-u76aq;bHXf>`_Lvu-|yyuK7B-sB({Noo-qSa{iDY4CJ2JsC7#NHnzf2%WoEK z543y*NF;$dM5df5u36Gqvm--b5&ZN&EAhr5{C!c+8|oxolRz$EN+4S`#t*My)vs}; zT(Pi{1fq-#GfdkB7(q_;k2Ea=_NXcti^p(s#S*?Zf#lu z!}#Dc`BJBfk~lM5jT0=xENW)<9?Wnor0ifR?)Vy8C%YKQ!w7iCylv&S){KLK)J|<| z2P6teA~?>FpLvFa%@k~za^QT5oRgfz>^Y4^L!|pE{36k;bNd=Eu_(qvB*X=wyJxos zFZsY3`xkLyFAySsFI>aLEkSSn5aHu1O>gml>QbcAbLjSM8Wd~Meg|53q8L`dSS)G z;wyegVf6!8&v&uHdi3MR6BAkTUzX7BJM=)KeAd&p$Ew9H(52_j!2JH>B-*CQI430< zip+r#g80v{B$uN3TZ(=kv>`x#;fP;p>HMoa$zYb!Y5WyLek7lrnyrea_^DEz=xvK9 z$*~2wrX^f{dWSH^9}1#gMhJH2YlMZ4ZnOL`+ia^o`N&Tf(bHOdCp%>4`B%OV+z?=4 z?Zd;%D_;1twBWMuAv&wLL-0Hn!U_xytU2g78g0^mG#=oGIZs%hTE3#ea6{ckrVxuk zq)Q^hYSk8Ax4))mz0M?KJ>t-aQaEsf_oy^sqJLlrU?hkyLfgtm8QL+KB6LCtkIJR{ z&awFEL+d~0Gb^-eAE@-l3*k*(It{4LtRWkMTf&jyCoo1=vrhq-Dc@24W%@btfst=K zxw=qKF+EPOp&Q5+C#Wv}TL}+k>%;QHE#DZ_u=J|cDH6ox=mzDKVl07iksk8iY6F-P z4-TJMZOXQRSaLgAs`!N`h^%rIZ{a3~6XxiQNg~&4bL7H_pv|9);y<@`UI$G2BL>D4 zEj@@VnNR}rXT!eEmc#CF{CBuR@=Sx2(9vp>9DjeoU-er;k08o^xl^781E_wj(~i48 z{JR4z3$NE7DEYf`0&B8~J!h^w=UYM_My*?s(F6>GXK$gNwjhZdBPM+er0`yOp`YNa z7a6NaGA^NO^MnKW^t-wnT|q#u5?-5`%Wpp+80WOZTn(zKhY4EUjYRIh^!voErI7l! zp-fYqwRKvI(UlD=Yf^n-+UUFnCvuQ;4k3uY5DL#fgoFBLtQz@j%DPXV@qN--o@w-Z zVfI%Sq^6e$1k=twz!OLYetVY=x!>Tx$U1E7a)x9$ka- z&Q0@021%FK8QS%&?bqx)DauOQRuw(Y>*C_k15#&r9>zSlbDubD)YihF+^ko63EVec z5Zs9=YaYg4aX^I&uZzPnxEnxLUm3F-?VXAT$KHITmoE@YnqFR{>CD)+8W8wANzS=7#8s zKvX7az!JlphC91ZOv!EWuL$;Hxi;CnKYVzPhZliBz8G*(HC(Gb8tn%Jd)xUue=Gxb zI}x0e^p2!{bADiQf7wH(uMkYpnlDR)h$X{<8d_BO z0p(5dccvFcejqU2EUcNNqGK+|8bJ7spI9D~!z$(uZMhV{78(+Ysk9x%zAzhiro~Sh zhW{w=sE5Q=W6u;`|3iU6xKwJ|<4TcNk!y1hdw$%GbE=IT5}(K*#Y;51EJHQ5=AXY%8O!a$zy@l~ejMk7g*2tfjw$%J3tPSiB^- zHSt_&If%77*3Bswlp5=W#oLn}6dLg@xsb(c9Qtzi($1z|9R;~ z{y@%cg+;FkBdg*+4Ngu}4S1Z3^;+zK4z}XW0>F*!4Dk^xYykN(fSb%#P`WGRWH}G#_w_FbFBnfLbX4DW5@Hzb-e% ziO8I0UM(u*Ttr2k&%`+BnKoJ$09PIsQp1rQP=A(VL}TZ`(8G%(^N8ic7wxeq$jDPq z3rrj9{nox843$`VZmk@rJyaGdb{upKo(`_0yA5yQS?Tesc~N+!>~(uRSbymuLr!~; zQUY`Pib0o_$g?2~{LCY{py+tbsb@B3CSvVEDpi~CPHNmvzFZGvKABX(ywOIH{H{gM zEF^6@K4?;kUOez!cZWXa*0L2hQr4qK9jHTa!aPqz>5w%U>E_@dJ`5<^#K_DJs2Rls zXUXp*k%MyJA+cl3Co=J_uC1%9)6JNm|8x5c0}rlz)eeT6yC2!5@O~kh)@-O23UNN{ z>fP>}I=fegWK^ANRdU*~ieKVT-hI-I(~4kr!Kk~N;#EVWma_h0;Wo@y-Q5N535;ID z8>T0s*4AG?I9+2OR>eJ_O30wpja8WaOB%|?&#*U9Q9+HvLl9YJl}GjBBC6yI65sfLe}3fogXJnZf^zVsYxM^>`fE%Jyg;PqM3fTy8-1z-=>C4A~;FjW&I~ zk4lNcLkMD-5XQF6v=9%7fD3f31U{gfDMt^YBjQV^c4u-xs67x=IqB0$8r2^(D&Oug zS1Y!LzcHo5$manttzoX%LoZSbRxYgcK6YRUnLJO$bxIy3=;YI(=PV0sTB``NM#mX6 zUrlA;0#l1i>$gm)2y6zDXk7uC-ii%iHhrWjd&JMd1E^U<(hS1BVunR7IRZnRXWkq!Sz< z`z&G!KD^%oX}N5{U2K2t1KmGZ06l{udkPO7UbomO z7aUbPv9o6Tu-{&;W+#8lt)7meCOVTXr#XcO^(9}=^_zgP?AwI~16#!jk>nk%**jeY zV0haQ#^)J|Gr4hL?3dXLb5YATf8i6&G*@?=~p1N|_@V9PydvmBTgK7K^oO zBa2qROdX+zTgeP-@}YHX7_sdGOe&RV_1w!-xOJ#gB9+o*7rb6@etB8sqR!tFUp3)9 zQ`-#lR%8<~4C;u`0v*eVRkBpX@4WD)OoWgI4>(=0+{|S`bKH0l6ofl4)SISIk;$y8 zxz4sF#G4WGfR{m*%_0xlp!1%hWFtp5JApXqQUHfq4a+u)W$Wmcnzkv;N|B{6%4j>N zQ_c36o7YcbZ1dbcUgd|Di{eHIW$J;JeTMUQkd^J=*1|Vc;2GmK%$PFk2z~6PNl625 za42((41IMz^3OIpEL-+l`)M{UgK!L6vFJQVr480SqHDuWX#rG{`I*S=jgi|7FZkZd zZwQ5U_WOv}952#6L_tgS-~Za_l>F2kBid+_x6?>IRdOK+Kou#%nh7R&L08}ouo@Ck z`{i&8{TK`c0sG}r6kEzR3Ew}$x+nKUGSsu&Kl1Vpt%;x5?(xK^3iY%4=EpBk05k+ds4SyNpAaV?_Bc+$736Tv zEaD&WKuAB3-HE8Fi-mI@e7PYBZ>}6uGKO%H<{)eqrqoy(9nAN+b#NE>2=|*%_7F=Y zT*Y0)Dk9ZB{x#@3}hjhG5INP)RSE~x28aHB{5}K-t z+K2~Ql!vQzS$?%tunh5dQj(ttCIi$?1PvBWfUdd^Y#Fl%97r?Q{GCEWCajkuew5V! zAHfy%++U}X54?edQ)m1`Gp+cBD@gP(^dduBa8=?fW8%LVdDTrmgK14qU;wKz-_~ab z>dK*8IzV~(QeiSbqgI9n=g+J7!PT^uV6dUnb@A3NGbB~SevF&(C$znYfSjGV zfYW#h`3ApT1b%-f=qM$w`9*B|)H2>&SOzJ(rbcoNjVEbAtzBsSqaG)xbgFc7e}0al z9Tf0Ti>!RV30p7Iq9tiGVeT9X{a6LxiFB``NfN32%N$4nAGw~C7_fyL*xF%|^~BAz zvJ-i{0ei+L9L%**=~D278}@hC_Yd($(Y2VtpU{*c z`0@306Z1gyNw1~-yQ$E2nNThln#;%}T5EqY_*cj}5e5{Ea_}=phc)vou8bmGq-GUm zy+TMfQdQ5d9^7UD)ln}H%tp2e9wCx4<{y*paex$3L)LG29O^7#&pl!?Sc-uKOlEup{rgFU03tlb*Eqqa_-2TREM64erf#}b1AFxA!K?K z(4QB9{5De3jko-BT#?hawm%J&=A6UuC+8o{XF7nzyUjHmDot_RI>? zVL7~rF&KvAk|wVSzQc4Bg#IBV77s5B9-$kh4w6wn2XuJBc5V^S=W7d!VF8s~QV!SQ zIyiwD2Ao~4_dmxa7bxQ&>3*fQA-W;t7cgm-E70uZy&LK(_I~6f{;9La5&nR#`T6vyx@Dcw+jw+s$L4s_c z124n_w;Y5$F2SOSQDf7KivZ%6AX&w-H;TrEFAnqCU>)vLxFQ+SPVU3FGCAO2n&#dP zA(~y=sTAB%fzwR8FfH3xy( zj;tN;F;|;=fPM-1TbmTO9i>Z9rqg#7p7=l#-|w9KgohL{$~btKfWomR!~g<_ zPnf|=(~`|T^wCOab##QXJSk0bw6|XCME(?okx*4-Q zmQG_;XG~j^J#a1jK;J7`cWfo*a$V~HCj77a_aJ&6Xtlp6c7p<{Go#I}Nh2#1tqG%5 zEV1V+Ds;D7vxp-}nynLp>V{l}s?_sU5xxP0iA%`An@)10sHyr>2i8 znjx-X#UVSgrB9n{Xc5U0?7`|ZgoJz^?KICls))UK?*YqFdWp66dHNDDuY5cr6n9~D z^SA?qar7#qT>K7_V7w1=0gt?Cs1fZ?%rxWL-YWFBfZm9i2K~w1cuFBmLeZ$rXAC6b=)!Ra{#4BT0`(e6wy6an${w>OoXp*qjR`2F zp>Q98j=(c71DfZ4Oeklq`nrbevw$;RphgOl=RIzsScZ?_^^wT9J`18>lis)`XLH7p zzW=;&p1gN$%Q_Jw?^diQ7EzF`PLtlsI5^%(ev4eb0tSO?y)BHJ9Q_REFFr>XVPYCq zGDoo+*J4}z_8}>vsLL`CVED?fN#p?}APur# z^(LaMqGqVh2gTvQcO1QkOr;kr{eb$Bg;hed7sB(v(FUzST=goV(FeIbs>7t#2e$40 z8}Oy=1+$k)_;yQdCgePrOuxdF!U)!PHmn0H2 zed{a7xnrM&)@>8Se5M1-WJ>J{ux5@POs7U8llWI*iye`@Um82vvpM#KaPBi^UY%wI zgjNM4BbO!v=!8?V%$E03_-y)K3KyUs1>E~|6&4>ZNP8;c$QC}4|21|*KkE*AH$f#yZ0=G>~Km z^F{RirX7=wNWegV=0U0RuX-mYHwDXCpfLun>EjAnpE&0&*TDhd8Dn24xfv=F?yf}j z3wi?f-(lIi-A3IHDC?L)5Sl+Z%hS+n31t09)T`HoX=y8HdjaKH4pxwi%SRKzM~{DB z={!Wm0_Q{G;x>IM*x51&gEe*^3h0%>FPRtJ?j9V&_FMX|3f`dJ90O(q5uRfIfL-0g zp8~3gCo1+<+UQ~-oCr8pna~|?(xXQ3cl3}cz9L9}UJPDS3-{A$O)`RrBG((}Rimq< z48r2w*b2)rxPiag>u4#O^p;GU`G53IDi$ z(o`?sLt#oR8O*m0^dB1a>dYk))RS6)c~);Zw<8-p}S^!1pl0J+3iQtBg$naQVP1+v^#r2Tvk5oHlQ9=bQ;d`kR z_C5Isc2Y|Bi1lEA8RmQDd^mkchkG)Xv=@ zcBq1}?)_VS0eo_CveyD;EnECFbqtEXR=-l{vrMZVi8~Q9hHw7N&sKE8CM&!CH%PZcN7i$Sz(Zoag# zL?TgfB!ZHjW{SSmeOdg95(r$}PV&DggD;GLFWc&L(1OO)_uKJ7pXVJZZ z#}c2&HkckuGK}ixNHw;;X^Y5MYH7*py3?t2Rz~|?DIl{T&d7bCRL44UHVg=GY=+eF zK(<^yQ?dMLh)K*M$pW}TOKinV z_hNnR(weEr#UqGL{Q{(piE@`rLzvlLE%Oo5`^dihHBP((AhT=7*Nd zjG3Me5dd9PV7&=BzJ;rUGw3PSBI*<87puvu*3~?eO=e&9fv(?zh4(nM5&Ok7ikeik6t6Yq z1F(xW#jK6T_)TCVYUR_fy*SP_s|U}S;;NuIp=xCqQPWHoP*+v>LJEttyFwVSRi=dQ zqayME5+Js2T~jICS6O%A_5iQ(GYHC9_?$_(?id8F0KONRj#5LME@FE~-6nLyaQ$Rg zsjf$CVAt`6Biqo8$1k_cu`Xh6<jBr&x26{K=0fV^#{K0Z9Ay&9;mfuJCmmN0Qv6 z%ZIb@^%rNS0n0eI(ojDlf2@-Oh|VCVJ)aKZ{<*g`0|63s>tl!x`=9A2K7*4AImz-I z(#Z_c$+`TeoJYrGN7zY}4Pkxvc9L&gZ0hx)K(%{*JjRB|iTg}8i5&>{;~y?5y~sI7 zem2R3d-A>l(rDg;pKg?bOgAC4$J>osJE7zKxcS3@x7C1_$4rU}eG>{q3cKcgc|6d} zpgIcx+1WY4?Fn#(_r&Xc>Z?` zEA&g%)^!AlU$J!dTv(<}mfeF9rB%GDX9N6tXLQ^N&)B%o8+#Ka(`z>-}P5Rb^43{vWLfRFhN^_0j zw}7lK4#KPV;&G?H_!cX7Ps&XmfaFXmVlK#1N2?mEQs~D@xHE3~=@aa24_d=-W#Xp- z`)29kVn?Q`-)AeyOP2op#@U*K(uJa>mDoSGc@!0fRDNZ8Oc5Ah_BI1N2*xG%n1lqNGaM_ZQW{4{ z-6Qkr^&$K^a`I8_Lrxs`-gA9rX`OgZfnT;o>nWTr@!V>}BPR#i% z%Ul!H^tTk%+Nw4-_wJiZN7b0dc!LW$L9iTY6vWc5Ht!gIPL+bIPuwrdFlAGRsut_& zGXA6~F!u^`$2xNcLGiMmFfqp+XBhpF9Iuo_w144Sj`~d!KIkW8sHo(8fZpx4 z2YMo{;iqI1UUbr@4~U*})(4rS12mRxEpHwh=Uq%=8@Ac$YQh)trZhPCwqO26KQHyowEnYNDXE# zVJyXZ-?DW?;Pw-GWgBj`**y5mSy|R#K<_ zFeOag(V+#hhnX{nK~o-uhz`tAjVB|9C}3d1!7%Wy=Tk{=bN;$wk+Hun&e&MNG0=x* zWY8`VYMrBks{THwt)S9YzFd=D(Ej++090lB{JFkuk?N9jlA8bQfAhI>^YWG8>;CqM zn*dHX+P~O{N6zCkf+MDPfR2+G6<0-Ay>>9D&F%zrH-~OVw2Sl!SW*%sRVZ!i`6u_1 zSh-IUAl?|T5U;YoXy(^|_zb`D&Ls#B=}BNiZUiMeuh8rAp@= z)e|DHOQq`$S&;egJ6pX$p3i8w_4W%*AN6pM!OI;n{_KQLfoX4fxI_i|T-{_5{t5(= zgHR}u+7B;g3@f^gJ78p8`qEHLFOY+=xL zymjH0L}l3ITF{f8xKxD7D|7X&J7=R$C%~oz^Swh%-N^jY^=- zkzKJwCz*SZOh&_8Zcx?l(`h6j#YZoz&fXOK#LBZ)A4@a-iQ-@(-m5tOQBiTDYUlT= zIQllg#+hjS?8fj%gu`An(RIyP@;oE3G*PsOp!qL7X?~Cz3DR#GPP^AOf^Sd^fpfiG z;=v*EOANZ_(@Z@z3negFF>3(fBZX@W(`G!3>BeW$o z=-shcgoORh5KJFAkX&BM9VD~})<#fRre$x*-WczXJ=z!gp*XSk-i(vg#~=E@0R@yz zge#LZgdm#3Q-q*kV+J4W26`XjhKZN-5aUCnp%PR)Jm|}&;GE#=4kw)D6;y~>HlBTV zx_$TT4nKd{2N}5RPD?pDw7H!Xn-o+C^N>BB!OPtPWC+*s`B7bNs#jU)V=ZPSQ0@!y zF=T!a;w4sZJkI2I1g^cdgB3I&Eop^UZ^tx8Pj9dfpTA{q>E5$Xa&eg$mj^K0yV2hd z(r>^%D^%J~%N)&anXs9io^tu-pf#=q;p@8y!wcq&YB&80XAXuc@~j#(5M+$qORH`( z4_EZF6jMeaPeEE`bQkM`_!R3yBJ8D5T<7y|S1g+%Au~I{yp;W1x8ykR^jm}Ug@8GS z`e;Ai%XEkE;_SnsnT{#JL@)?*D^F(MXu8;RP- zwN>3M$MQlT{oKySzMw|!SZl}V{J=|ju+q-PNaM$}hvbyJV;eA{&|OkanL-UyhLN;1e_$GEO24Xr#p+fEpALsUeMU}I!!k#)@JPB zZQ>&|oRY|?Z(tq+qUl`6>etZcFE%%-rVjeH1C&OTZ(}CoyrLhLPKnZJDe`dIE`I0} z%>(?h&m9FaU_<4A^SV?eO-^;5dp*PD1Ilzdkp%TPU}|#4CxI9W2W^ruh{kIL$=Pz5 zvAstQ#10FYHkMd;W8BLnl(yaf@UVRkz@UsSKO$cViiLX^Cgi|8?woum68pSCb<85NjJ@UdaZK-ajKT2gC`H=V) zmm^z0NE(AW-v{1{9!-g+HJ;=@&AMREh46MmBcmSIsOmcBf&!|_r(E4!xZ6+{mC1T$4tbZ6jZVtu z4+VuQVHYcJc0$8t_8sMHa@eQLK;m|s)+py9ak&Yni+(7k#KEvwgXEG2N3}G~=(u zHQr&N1lh!)2P?&6=qOpmuYzgoHkev~A{NXKUfp zvjXg}tQP&nH6W&Xr$RMwJIbvfo2NgI~MDjz&ZeY5zhcAtP%A^rQfS-kNjGr>C0++FvuRsZ0vdE@DeeeQO1~M450SU zUv2K_O3Xj5U5DPG>v|=yMKETZKA2vB=u_68*(_%WiqxDopdvM9<+?*&kPfO`8s!1X zKi?}Yumoj3;^~xI`_?`(td?;O|Akq-{%XYFU>t$e4fT|@#GH#2m=pgjiulwCXjF^5 zZSlTEccA(^+hpZ1_{~Lj*$0k6-!`*by0_!y{FzN~k);GgPPs=SiV`(9ELqVvYSCI4 zX~Bi z4^?Yh0a}Q*s_nd==H4#L8_xwDqI8*jxicce$bQr}=wR*scjP9@f^ zJ4>Fg;I7A*VVyg!px&885qX;tk-J0Wos-$POC=!U-w@#OjseE*=wHq|XQu#}gWbDt z((RGpyNBn`UC-dG6_8&Dq^^gNP9LyC0mJXWsYI2RyJ|ua_rp^RmI2nSmvX|t=c5gH zL#s|7nS_5G4}er(xc$k)jX$T_oSC;G-}U-neG)?}!2Hn1QN`Q`$yE1U5G%1*Oc%`d zX+(khreNndI73yu{{9W_u_SjvyH*|Jw|hkMi}9u%<5U!qtn|&2&K+lzQtjiW!-3_z zVm`k0r2S8)J75LGbR%T!Yhd65LWKrM`(AW%Qb^`=`xSVmF_`#XWim%GY8c$Hf@aC% zJW}AOi8R{L;_!u06!fD}O>zEtShUf1DdRwXy%=~i3QV6EdYUe&rZ7jJSOZ0`SdH>5 zii&&l90SF~LfD4W)Qqf>B`nK|EYlb_jVZtg6TC|7dPP-IBVzwvCZCgQj)7+Ph+#)n zOalz4XO?p$KqFul3lYt4CQDDKfqG>Nqc%Yb;)@1K3mpNP!ZQBL27p$+ivN|hTL46q zMDt^Ylgu3|wlN8!GQ^FN9s*>b$E##=MU)2nK^cba87|<}8zitba}eq2A2~m*G9zL! zKdeR$O-uc{AS{uZPIh53#C2j~upF3x%ji#FUkS^E6^HAxU6F$HWKK45@Klq7rD9z& zpgz0N@ht#`_tW9Am`!TTN*84lcmTzUx?DXWb~fSrDAEvqCk(5^-u|qg5kq3KM}iS= z+K+5>^IeH#$QIEjiDrAY{Jt5Z8fb#;+85F& zZT?jbuCYXRWV4eo-`6Co@Qn!%MeBiRaLdV27xON%1UY#8<7Mym@hpmDHU+rcB-)zO zDGHbpa~a8iE*%=9h!e1CN9SIx2C1@fWA7h%=)0sHm(GZ~q@mZ!b}@z)i)tw>rihk} z_Ln6~H!}=*evNW|B$M@z!4fMn4YuCUn3nioyC(T-im{C}iT-q?wk?A9vT9@(O^Irg zq}92-6SC-SQs4YSNOXiefB{yc%DIlKb(q zXe1#4K&yqLo@2JPEz%>@iV}nwN^1uIY|d+SHQa3@@hvK^bX%A(j3!_y*eHo8{0tw? zv%n#8J+AbvS@RXop{Y?Ts>*#FV{R5x*L22MbV=EcoT-e9cIB$NVzanrB3#N6j5r?~ zuTw78e;D|Y?PIj0vH+XRj9??_CdAJc9o`$lL}CQU4+LKWNZD)4eS;eL=42yyIEm1;S=#fPtl2|??&A5r~L7BZ50}ujNCVg5$fW4 zRPf`x9LNYV*HT@xv*&S$nbtO8v}bB61d~zl5bB!Sn%u7#e*uN^8r8BR$*)^B|`;cLMo~=T*3;R&JMB{}GVqO8dHG%Sy64U22G{32j?BOsiy!rL>d6VZJr) zt?KI1DdKs(YF#pwEMhpUwawT84!@fiO<|Z><&>)K7l6OFrR3A)GNwobd|kpY-4^|z zy%_JN9A7$9$Ku3vy{^<6-N~+n6NhWq>3dZ6ws8ZWw|iz_Qw@}ke%SS2YJcOIXb%kkgQt4y?V|%S-e-n39u%>KV1{-cG{(9Le#YuWC8c{wGX4urO8-LCr+VP< zq6iv$L^~a@POG33gvDiS(#@-NfnTBbA^b{I$EE%NP-H~$hwa!}D5Dp}ipp5&v@q=* z1`Jm!;F3zY7()_|RZ9MZqnjri$Mp{YRW%fsHvyYeqb(_9TVo&6bXf?}Z7|5G2TSy# zAEbt{)TqsFlj-%-bjh_Dnh-v*ER|7R-40^I1KUXr&>OvD-&%8z<9=0oE0c#kSIVBC z@T@(mwk6kW5r?ICE3$NnswNdai`19BbmB4?%HCt(Hn9bfu~ZO+)~X$pv2N*-?(Sk` z(g9toI~DiYLPqv`_adp(9$jzTTG3#L`IIzyG2GL2DOYk15UA_1F6sWNA27JKVXQ`x zk}C?%I3!T$%au+$P-Jsp=URkgx|E1dlv!@9V=2(jC=-ng8#2yNORYv|688>h$B}BR zS?5P_zY-G4;Wb%)B{dXup2cFwl24TN&jOX50;&3~ltNhXVkj%hwso4Rh~`^|)35mt zQag?6v<_&e4hmxk(>m2YWTSJV3JEEKfP;_m;c#f{cj9B9PN#lOr8VFQ( z8lieul5dTfmjB4ZHeow_ht3^g&F(PDAA7^FEz{nCx)$D*2hLeR$4g_#asDE2WWZyj zI@y{Pd`b5ajfTa)k*I@*w%o^A=vC!CDODvJ%K%i07hY#OJfTo0p#F(h=J?r|g!F1E zrk6)YhQJ}q)%xf$r~tVx3=C6TG6ov2`A$$CsMUb$CPg2`S?WzCP5c3WfDYY^-5kDy1Wnm=78Z+KO33S`?5}QhPh__=1*g$7rDpMC=snFlGu1Rsrd=K}E zFsxeFta6_l^ij5q*`>QPXflYl_UR!I{1$Dk2kDp2bQgIsM2@302>MVKj0CbZa@wbAppe_9g-aBRelvsA z=&c(j9cGm8Wia9#xs?A<`;d#~87@rs(MX#A(CRH8-VA{N?WXz2e0+=c9@&Lg;v(eK zO532aO%gaCtUO4P5Ji6^pnmA;$wgJ&@U-e#P<_tRa zQVg2KJk-pRUvL;8qyI#(A%&%O&y9E-ON)!8@3ZCEDv>Eci#@IFcRn&Rp4*FaRKUMh(R9i4%I4SD6PtGGX*Sz^N?`IWeLR~=q8Ef{C##Kn)=THng|5ld> zLvdsnD}I!-juCrS768N~T#DWs6lH==%njWAzM&kn8e!pjFJMW6r6G%a5gk0#EqT0n zj7Lcx8*BZ~pFk)AIw*V+3H9o?{AP5Fo&8G#GqI;)DxZUsEb|d(0<0wEcha3%uc7@) zt3$ZZ#FPXL!JD~#UxgV;3B}>VnGUmju5{mR#9(v>~m`3ISG-Tc43&j=P=FMb>8GT#qteLJcMA6J&KnbaE4{YI`Lh!VoU zz`xHIey7MwBA5@HQzYR0{W5p<4tRHmLFag^f{WtmD+Q?{&8dwjfAZPVo-ToAy1v#p z`sk5Ug81B{NwMK#uvwA?NlIwjp-I$K6ek~RKLW6Qzr268@{Z=xWr&Z)rR1cHEOS}d zxHHyR&95_@P9F%oVkvs=)3{~MmyF&nluPpaQfaq{$wUCjzLsD!TA73-NimW5B#fgT zl5twY#P#!uPs2m$tpuEqYFG0PI|#2Y_)g(ePtYyTpXGRF3bk6ZCIXWH#D;3OKR$>) zK_M0bsf(EsKbV71YU(mKR$YmAsSXPKz;?a8n4gRc8#eVbKBAHZ-*kojq!-T9wPtE* zSkLx1Cp&=;zjKRD8l8!?!$LM=MDy$xjg4oI=It1~Ly&8$jmVckXGl28UCQkfU3{O( zQ9v$I|E25z)(yo!UHPs)aQPxO3ggvEp-CqNdUo#+))M9l9(oAJhgtWBU3d!KEmhP~ zrdITkNiRG(rrThLki*6rvI5a^zpga@4*+IBnZKF_{CxJP{Xl6cO@z{zZONTZUBZ?X z)o;Yel&ox2ZrTq@X}vS)S0q)-$FukI1QQn;1P~TJDhVo7lmo>K+}$P&L&nzV*#p#JX)#h!p*Dsiu-WmsYo-VwMurl0EC*OQ{Ihh zX&W$+vhcS8+xytODm=F3Wc4l0^@*BgW;l}gu7(iolG;OV$kQbg$M|(5i)1}35DG;6 z@qDt0v{)uSX;hGEyK?8u(S9t$rniI6>(hTE3Eic9jAl$umHNru^!sTUMpZ`x z8A+%ixU|LQ3o&YVci`;Iv4T^8oHWeQp}rSYb`jc|zroE7Ha7Ba6Rc=z4yye2#Yfzu zR~^4Pq)+0^>eI{0%B!nW?0XXPYw>rClf;ks*^2zJa8t0Zs6;oV;V^j^X5y%B15oUZ zm2+rMk6$0#+r^}I=h{=H5TAQE@hO42(}&X3kHw+RXfp|I;+{QzJu$BvpT&QV?e}QM zdB+g2e+}bGbX2&xma|dY(W2RLs>6O;IA>0m-zbmk3=~t3BY1Wyy^gU`>+Y`uI@Q(V z|L3VV)0Fr<*%4ClR4}4P7Y(y1)z(*XN05{Ld7zz>FX#-W6}1})(vg3;;!Sf494@K} ztRi0{dEdmv%E;Va4%> zo5MC5e>`dcK6n*g0{{i3{L=yc#Wnw;K(}wh|LOK`@Q!Z(4)5xA54?XzZ7Ki2fZFN& z8w1_`13uF8AH!$5{T#jk$jsy)C-4v3Y2+*T8W8?PZ>vH7MjeXs^0$I3ZyT6-##V6W z;a2eQL-MPYC$@sh^K6A4d5=KGR_OV#M*MGjo(LWQ&!^z=ybT$iccHgu51in64+eVP zhoPPiVWj6XDD->||IvSkL*5Gh3*YGVU%_5_5*b2)10(?U>OL8nL`Ggg8}!-=&b+)W z;B12|o}r=MTcOYBHptFzhn({#I=RVl2G&h5c`)0t_WX4d+}Vz7cl`uUwz-z4Z`=SU zJ`Q~sJ`N`=+y>sPer?d7#s^T7HIN={FlZYL-kI&*2!pac8^M2@Z)AJg!I#g^A^DDW z7+Tv(8|H6;VOwB$w%HCNvT4-E20w=lu84@x}yp~7(wjj{)9mRazi>DMxC7EdVzQB`z@TeGB_oU#Pbq4iIJBJ4pKGfEsv$ zr27Rp;T(7rk;Y62!e$iIb3-%Sg$DIJ=paM`!Eq2ag2zOsi7pdO6W!=QS_Ol9O+(Ts zLWr!Y{0oM-s{W8rv;NG^ORz4_W}Pc8sQeC268GAmfB=6-FJ#@2+n|s&X$yGTNYx*K zF`aZchcJU24WnccOp;?@x*Q9$Ypf)&lJ(vU5>pgg&4@_x;X|3Qab{JDy4?cM~ zio~kS4tyKHJ$hH`!AZ&K=}$sQB@KZrc8=cVb?$$pnV!VHO)!k4kyUDQJ}=JhZBWMS z-3BvSiQO79a}(sH$gMn$+_u5YT3$nHHLG-pjVkBHjZl(my|a@-&1r**tU0WQr_o~w zzf~S8akgl35p&nW#R(=BJL6oOWV2wLJyE4uFd~(uCv@iE!RL!ztp|5|ox}yJ*re8L z?zDeV^|euTVxX&a|6V%rJW?<^(EMU&k^tA!Nw$8kHoiN7EJ%)4dtG`DH?>~(PGZ}3 zs6klYiG!!7xUnk4Yb~4f&7rXHz z9b~M{!xhORxZ^w=Z^MsH<)JT?hrJKwq1S({kLID)>p3F(_B~F@aJIevoZZ?YxVIe^ zBiQ|OJvYYnJh5Gm_(VJSTcK=j3cKovMFs6pZ|lW&2nZ;63i__6AXo4t^xXzVo`WME znVqCjzD215wfHQ)j&Fa! zB)k*Gs!RO)t*G1wSNzGA2`l3SPz&lu}q2|OoU+zO9)Nw=>U z<=h&xjkS7ShbAl~{{ylV`p9w^Drb=&oeg7T1)MBTgHz-j@~KtizZSq0Sq-Pj8kjF@ zVUb(}SID#A26+zLEYF5p>%E=N-~VuozO6J!V{ z$S_Wp%{Wa)afXcHeA$9OmKWho@?yMGUW(7k%kg!&7T=I-@J(vpl2_u#a-DF=tAs~h zBQoW6B1is243Rg80(qmDC~p!a@@6ql-YOQ!_2M$QQQRVbCN|2u#l7+#@qmB4S8S23 z;w8CRydv)xf0hr3J@O$#$%l;r@=+sCZZXQ`R%5noH!9_0#uE92aiQE{td-9h>*SNh zRq`pLRqiq#kiRk7?>X?z zc+Uk>jM47hr#g91LR?_6Dc87v8}>H0Cm{f@!oOo8ri1<66wY6F5p3 zFWXT09Kmlr8u5rVn5u+(YXIeV;+s6W29q&i39e6Yt)#1Q03SMy=aob9p!r5 z>?d3YVFI{x1n-9q69*CmpglWwAB=PK6ZW!+$t6biLDF1duc6G9#`k~0XcGs01HA;D zX>c;OZX(JlxuC$92#{<5Y)zC@Ev-!$er1R2s?H}vcF)bj2^_8 zVM)fc#~D*(NAA3|k-JZ2ykpJP9cGDxa*1q8ye8@PM4f~U((Q>bQnxmhd7LvyO5Wrr zkRd;XZ21NBlV8F>`5EOQU%)WxpGa+?{3=QPprp2w)HQ(xT< z*7Wt&?O;*~ZVO4QQ)rCane;vdrs{Sk%+jsRg_-1ILH?V#@L%G>x6oVeBSrZJ`ci)Y z^#@ZsQXYUpi`0L_GN9OUz!VE%D)q~#Ka<)j%cVKwBb1)?y^%W2iRS4R;t2 zr#`3Jv9KyNoR!$Q)4WY^QUd4P4hw9q?J&jy##L3YYczl3I=hujffA|y_Y?iaun^9o zQ=A2hb^Bwu4>nV~8Gfl-oBwmjTEJQXrnM6CtkqCpT?k{Wi;2&dz+CH6sI@KwzjZk@ zSZm<|>k3$7T?s$7*1>w~D!9kG8h&M616!@_j9qaTgPp=V*^Z022%VJX3Sq7J`5AYMokj8EgG);CxW~5xx+wHzJGQ~4r?wurjhyw2$KkSt_8EQ{zfJIM zgUf%59i4E;@iKsyXXJKRtRyiN8mo|QBvWD;8Cj$er;`oJ=S*khxHhI zZ2cNXT08M1>q#uNp2D-OU07%R2A5gC#q)oy-MGSf4sWua$Bott*lN9mk6JHdoAnAl zX1#{5SbxNSS#RLC)|<&>qYwsQ9_EueUjpN?fa-G>Tm*Y@G!}vrPl7Tmf)8k{5oh5T zG8hNm1cPua?4#LNU=)s{B$I>S{?*-??y>RtjnD#Bq=wwwBaAN#cXq;WaQcXM8n2IT)Setc)s|~K4z$%|c9gL|o z-VW;~nt3_qHn?hT@}7v~Msf1$#3d15&JbF!!()&pu=#9&wJCai%^_uBlh)irRE=yp zRpWl-01ih^$5f&q#|-p=YpG})o`Qe%I<^ppk!lizxDQ;f%LHe!XCRdUYn%tOowpwx z&Ls6q8h|eFQ_SkI8|4 z0!7xRFxL7E##>*&B5>E!bi#jpDH(et}OUg^*~S=C{?D`rnA5UpOdIgp)|lll(pd`l0Lh{9?9FI z9$2h(O`xT9h~g7^a5CwO0VcZiJ*ES^@HL)F=X1gU_)=?=OY6`Gpp0_>nA%HEF_Hh9 z$~Q*);28%+OYdGlE2EPpO~ZfbcEaIUNby{eMy+@RZYXwU-B|4EsA9((6U=NgpPXiD zJ$n<^u-Rs+kf@SfW0J>w-In^zl#ASwvf@@=VH~o+9%zU4#U`&xYR`r?xUC&-uT2iz zu@Ub)+1$Fh#0b6de-OmN%VZS?wp8{6RC zHn=aP5Vuip+6X5cvfHL}*V1;LiJr%yl|sPgg>eiyo_6^8Bk+rD@XKv*f3ZcO>Q{-m zc6eYLJV?2h*XratFUjAL?Xb(=o^9|D$A9Yi;(nED{SJ%4eCqdUhlg#Pk9f)FJ<5-> zINfW9E&8g0I$MifUMGLa?>;Ewwrvv(ejK(Xqg6Xwl`9dbw(m@hWRG>48@w~mYf)b5 zxWCv;;o9R2WTu!pulV(PC?pYNyIDVmhOry434T1S;I5evf zaG^R8u2&<;6i30+Di40I^5IQY00-1)9Hz!&shWtjsu*umQ?XS|!~0b!ZdGNtL(Rb5 zsvLi>X5vR`j&P}YLaF&8LoE=Qszwy3S}|UIUrbdC#ay*WRI4+^8T7tbEf)34FM?`` z2&*~~Q}x=s*Fb*{&Y7`9U+Z3oH()82QEv+_!Wn$Qi^Je!EGLuqU@5$SGvOVu#6%dO zSGYwcG*Ac`!45&JNH_%QvB@L%>|mjLvpLXEHG4i*U?sT$k7&ZVH*wx{s$ZLMIwfsqfYcshssn8xn&t9zKG1*TdkzO4BoTB4;1Dx;jI0B6#9{$ThJV2T=$U4stU!czQfwQ4&u@4`5bGq}oA-dgatUuydk=Lz1gg z0O=e&*gm$yQx15j*zI+jPr$B)MsN5{^+HE)_-*whi^u$dm&3`_6gnOY|6+@Gq3s<>PP6IR;lZ;r@A5W9fJb{sbEh^C<5>cJcE?c1EoYu z2G&yY-XH#h-{-3nmR>SqW#K{^L+FKwi%5|TDkS#dnKb6m$~YW6`ymG%`@rF-H}S0f zP-uT5mAut`zlHMEd|mPl=|@LM0{&;d@k+jkS}p>0Qu0vp&fmp@YJJKzgfmIbx#&-` zSRT6gjbUF3;7ba1A4s(c zdaKPTHCUhIJx3qA4o=*2^v-l+11{|bcRYV@rP7GY7#}ST;(2KnJG!@cIX1=@7sn6i z%AEMbiQ)RZyC^Fe5g5!<^^ z#_nYOqzWj%U^Le*m>%m?8!CxaCb3FvvcKO^FKU0kqh3sZzvEsD?dj~4Laor^UkOit z0{+tMhS%|S_#^dSh1cN?YDZJ=&xG+4Q4OC?_d*Fl@VYso8c&<<#Sp<0f5jM4ExrQJ z{{v7<0|W{H000O8m4rn*hjk0oZ3X}U=?s-I3mubCsS1;MG~ErG4N!c7;2ZIU zZ$%krbaZs^k{YK2n*s=JeYsQs&jew5~1ZsB21#0u=KFfa=5L*Nq?2(bY=_agr)=rHJ z%*`5kx8F#OPt%D(0cjN`ddCXZc%H@y8c!GpD`S~9Tw}mB99KYNx!)#jLu$-Sjpqv! z-b{UEtY!o5SSH_ji-w~DjT<)&^cCD(!5wfN)0hy@6P9Hu zObnaOHe)zTn&zaPGP2tZC&Ob;>E|=i#^vw{th(%dJIhN6y_Oqy2Qr7qtDaI}?l+zO zadIjTI2qU6NKq!E@a}jb!9cZ)WjKy8NqMG(63u^_7NvkfpBlg0$aw%8)Y@Jz59Djq zbmLy#Tovw`rE*kClkBgWLNy8yL=sd59usIKXGtm^UfZXN$}1mVxQhy{&b8@`8EaHv zer2nyVU5PhTG}T#U>BT}xhca?=Tu`9Wm6q1u~x@AJgFmwB|2_FkB%r7>v#%J>$sVv zXIOs17T(tJ4o*@GXPzov$Yj$T-G-b>Y)Ez9 zY8~(5Jtn89wn7(l;A(++p^JJ@t$1Hxftr6hAK}V$NXLEHu44yw3aq%osuO5BkH2qt zubFZ+d?2voa{bW}E>TIU5U5n^ja16aQ|~WbvP59{6*MQ8(=;zDr;V1e&s|sI^Abx6 zZYCS!POEeb73re?VMa-%UZ{-9T})?BToB-a$-HZF530UR**WUnNH{q&18WkOP^o{G z_aAVKoIqRo4w$8P{S$^mEjaIj=B`Wz==ceX^1dKEs{@T}k~Vkkg7B8vjC^Fo2JzVp zY8^Ar1r{)JInzoDEDm3M16+J5cA1d7mz)$NRkB4nifySgHRk*0-3)JGy;qrM1T@eVR-C}Gan zP!Wx+X)K~C(ow|RFKCY%+)i{g<^b{_q@2VYc?k35Ls%#u#&z-$+$bMIw|svbJ@N=v z$|rEUJc@heF$~Hl@t}MP8&p*Mj2CfiRHqxl9dy1LC2bmwB<+ozjaI)K6 z3o5cUP4>MB&HoYnzL!c^sFZD$RV7@l_?c>$HeoJ>=>ur;G<5k9{kEyZk47Du2dT@|S60o$|ukiqq()5K^<#TlpRp z*3H<5d#hpXf?xj=%Y=XU8-ws`7!ZO!Wi@R3nhxaV%g!hXrroXZ>H#8Qx4{}&7rbf%kp()*;m5# z&5Kwrp_B9_+;C>bmcLKia*elT4R$FyNE*;TjvD$$@&t7~=!O|vgLNWm;ajofB*a6{oeM;4<0xO0CiG}6RyDx+-&6LOm5n_*~HC9f4SMr%~{->?V_fG zn=L%;7=QpXb|Luo?q026>{%*yVzW*zJO^V~>m>7fi-({&E2~ z+qnsI(`y?t{^MCh#;6NsU>{E|bYcwSGA3Lw2fH1ZL7d!BiF1!Rkf5m8D$`CGdQFA$uJ}u*CT(AVM$idIz=Vbi6j9-xPi!xp*>i0RY z-+=?XG|0_Se=j@m6$k!~sqv}}#DDl@7ycgqV9W}yaNr*u_$LPSXBl6U@pT#hf{?qq zseOGi(U(lLCt{kv7eQ%`M6_6A*dLE;ahk67C;a+1Erwv<;16uq5(qhM^=EJ0P~Z5G zHJjQIuo*#Zj7H)KeWFkl?2~GLjwmjkc`D>dy0}v#toRj#?zcxb?RWu ztR-NIfmpkCVI)d;Zrh#E;*H5z1G50Z)hBM%d666FNk+EEH)yfcGgOk8#F||JtuGOZ zMi7d0Du$vp&Fj<4bW4FhmI!r)0wI66JsjN`e~j*=&AEKIDV&V=tnccI3vhi}EKXz> zmy-}xnwS!ZMhsMI{E299C{R-`x`t>ptoiwp9DiS*76~>ok_0^z2_@*hrchYZ(RcL4 zqTMkq9zTH?Z7{a@w}my9*hEh#KC>2K`bi1Rj0wdX4GFh+D>fro8>2y{8wr%QF4?@`VU>r_5KnUTsehTiX160vEutfB07!|0d%b2se-G{w!zs-CC?S6ps_lYnu2* zW3tR7%WERZ-geUEE$w4`f_Q_(c4|(R1EIb^&6!$nG@+Sw&fe>fZ)aJoJ~cc^2)n}m zZiK=t7&HG#+!ERbKMMf~RZQy(hXQ_G|B>8+Yg@9bi}iSqu)V_seRp3<*cToTe>|J9 zB|IC&eNBWC8C4nIl<_T!A`rgg{qQ+578Fj6X?KlFvYN+q!$~!diJ#Dd!kmsm8(|(3 zy>&}kCLK*)q<3Gqw@AQWf^^xF4^DQ0| zznUWnEf!1m5fL)FN1-RF!OmDHp&8D|>FsQ2ZtiU0TW}kxa%Tfw7d70TC9FmGiQ7eQ zjqi?7WaL_{^~E$nUWAe}laWNISKAzlhlum_kw`S*H_Upm37yOtHUK?cf9zZdRNTKq z(|xS)3aVHas7L%^y4k-i8cVQhv-jH6Lv#reV5@hFU_n5|S{1Y8QcQG{95bK~JmC5@J}rEjnYh zSSU(#EQlLYW)5PkULk6^fAnpWMiB`?j!<+&P_tMV-YG}4o%#-yTxzo(XS6c!$n8_S z>ZaBDK!DVAQ*=8o`@9udo*vi8m4*_#YfO7Z7$c@L=gqIBz~i)27wKh;gS4x57^uaj z*#ZXKF&@BUjK0%1omOkcVj<6(#&vI&EFlec5rbVc4zPw>-8GM!e>xMy#SC%bcpx&s zG?*o|b>joGrD974OkFKw(=vXt9iRB5LC-Vur>^cpmx~eMbo!?G!|q*BYa)K#VxEm5 zAUqC>thqcrmx+RRt*8h{bV@Xc*`{?ELyv^f&o++WO$dY24uAyhG8UM4gLah^u z(L8>r$FafH9!-*Wf6*O;JAJ;IcPnrc+^oP4;2{O}!vO`p1Ggyf0Nk#?58+`2z7KaQ z@I7u0(g*w}&Q#!E@NWffh1(R^%lqzt?<%kl$Q>7Exu5*%2s}NZxJNHiDDZQft-yV7 zzk>gQXA!DSCCyRb5jc!6b2PO`#-RvLLJ4zo=7w@HrI9eA&LS#(JoKc+hj>n zB#UHaGXFx6Y(zWBF3F1IfZMWih-Adb2F@by%$AFeN={jFDN>H4Aj}$521WrzhPn z8ER6DI`<{D`&uC>&FyY0`8>njuXj^fF0;x5A?*&K}Nm*btG;;V$ft zb<=91#~<;;lYM;@s%t@yXJkQxw$B{~Eu{cyc;YabHcukT>VuF^Hzqt?EFS5#MdM7M zm;*CO!qfXa8DuWcDh9C3#hzVKqqFFk70VNM3F# zSbKeN17#suFESu^vDMPS3m&Jf2Tk2o9#frg1>2lTg{%(orHaL7F8h*P=4~|=AxzEO zBeEtUED29KpCXFo4#!u}bwp*8jAH>qtdVLJX`Vy@K!M2(>429qCi4}Xz_;eNYW}$9 zf8mrbN-QGVrbr8<(-<@vjA$en_h`HNLNP5^tw;-{MG8(N>n<%O&5)MRgaxv+$xr$( zI2-RtCO9GTq$F2JN0Q%@|q(Y}k#Ogn(5|aCnZj(T`*BM~yu+pYL^Q>Z3Er&f z=9Cj(j$jLF+mhWQss=h;MAe|of7^T_49dJMC;B{^l$bMbvPEQGhp$v{A#IFld#-%ZAMF zuXdV>($*ax;@#sdP>=O3x!oCSqd&vqtsoM*GvsdbCwjQb=-*`&oB4AnzM6G-jc{Q4 zuXtM!WPdQ&DoT(}azUZ4-Sy!xmuUrHD_4j!iXy5Bv&Nz*gk~X=dPEpUG4{ABSuHvb zjbCw%kSGOG7UgGA-WOk3fA9G?A^@#%10uC$dDA-Vi$&4smo-(Ht>6Mve$% z8blOJAh`+}3AXDcR#DQjb?LS3IoVV)YpHc^He-4M8pn1Bbafv=O{{RR}nWcU``K;s($K!R`6-%W5cjT^WwCDv|* z+o&(#(v#@#fbWX_e|vBzox4lyxtr$SA2ye`yd%c!er_! zt(8YWuG|aCgOJxzkUt0ohZsQ%JSMKO0w{NatlR~PayLv-f9`?DX}y@hDD?FSLJk=q zJPA)x|1|yOg5?-Ykm1K`>F+&S_zC@4xdcQMeg>YU5#zF-)(AMvCnO3V0~_AMg8`V( zS}A6g2Ool>Hp|NL%7ZsUZdG|@d0qQu7Ti1ZPSrhW{Cwad{00cWL2xPuAYXX^is&U1 zm4~2QIS8|qe?w47{XFGifwzyYoJx5Al!(j!erdjecRoyqpTTp4S~ZV-O}4+2uhf@mc!8N=uVzsk9#) znW5~%ekk)f`yrQ`a^5+Go2mT4!fQ6)1n~63bUHMnf5S2eGy6f}9Zvo@%U9J8if>v! zsJ;?gKiErbM_|f*F#AE6LpOK}Dh9w;MSlZOc>t;;D6(5`0(DxE{qhT)S6>T@nAfSj zb7^0ZeE_NpYG^cV0BQ^7(V(D?oB28rmI|M>$i5#I^n*JiA6-yF^nVm=$`L4Zw!f8n!odG!8sX@@$1tDiQM3l!Mt~?IA zlqcXK|yd5RMMpTOP9Gw`bN0{l(+h2Y%+;D71%hQTo&u?k9J#7bxpBNl&s-!F;B zcEQCGunh4;hF{6>q71*b%J342ZSH)NYyxrhe>Vh#x!MCxP-$g{ud=FGE^!v+48m!? z%6+hqM>&=IVNumwnl642mUNU@2VrTEt;k+1AAn^}!e#DKr9^Q&1S>i$g|M={!&(Td z+B2_H~i ze<2lo6^fNV!UE+DSfuJ%ViTi3!k>tAJYvGiV#I`XiqW5mmM$@366e5cG**NeU}&!s zG$#5?kcQ|9IccKH@D~~WiYSPWfHM58f5Jq6-9o-0`0%EoT{dvg{{WrDhY}LeGfJ#Q zGP8mwYe^4S?x&Lq3_vrhksxM?t;Aj=6G>Z$qob1F-$+Ax5|3a?v9!ye( zpj<_mtxB+5wL+6>gLX0lTU3W2@>)TZ&NOK|)1>Vb&N``V%m>OyhxKst)f2s-|)eUpiTv)8;!A`Zvq`kY+6x#*w3J1%h|IkPx zk&3{71?}8$0lY^eU65qJ_hBbVjKl?1&Y^TBdf52+V_)KDM zm4ZcZ>jN2vNR*NAwF3X1am;YFFH+AhZpt8Ny^ajGz;6?EGIi(YZMX&U^|g=g9s2FM zy+beh2H`9p8Qin^4X&lus>3B#dWD5NR0UMQW_rysSV@pqlJj3V+!Lx(={3_}x;g_Y z)LBra&Llp}f?9PBEKt3$e?+Z-HL6d%&_n0uQvU=>LRy>BZ*4NYbpqIqvEv&G%4o5) ziFevf@;DQ%q@?1-mMUFYC6=|~{m{V=DX|`0Xw$`W%R+m3k^L6%nte}aO4iOvIU&Q@ z&_P40Z-g52iF0*9le|f5g8mBmSFA;_ugi zmhFRW?EV5KZ7>P`Fog)}V?w?*r4S6E#A+!bYozhGvW&iV@u*`TbX)2AZXb=x%P;q{ zu@d)n>s`-{6bbATi|W(xfstFDIssBLIf*P&Zo zkEQAcoULxcMs+i;Q_sR1)$?$#x)twKgLuEH;eNFX52`)*f4myPm((b}ruN}cH6~fq zxRj$Nq&zh#6{$O=~&y!()SD0lI3S-OSUzJLszw z@+1{qv}S`6$!ZEdia>rJRwFJx3VV)1%Tef@aum)r>CiOHp#a?~tj2s=VVh{9psP>M z6PQ*mzEiyff2``K1g^TbwHX+uK_UBlK#pAOfaqq!#lLPC-t=rb!K!}1q@c{Y(|y(f zxX>hLF%p`oEIo0PR3-HB^x*b$l0HMaF!$uF@P3Q;4;v@(^34e;oJ{e3huC3sDQj@}Pc^gy>2V zqOXwtL;VBWzM{hI7{lO5Tg5!M81pFXL2@#)(izTyVHplt#emiN9?Wi#F@KhcnbY~VAHxN_Yxt;hhSR(3 z2;>?1e{pt72OS-)hKHCj!b99(@(-`H?HziFp;QX@Rw>+@1y+&CZYIYt8``M_`s)zA zPNJpcPC_Nm7>t{&RP{$Z`*2Tchde-@_6aS010`z<~BwBE}Rn3?S#(jG+ zj9658h{%7bF0?smqfVO6S>%)xIRG*Pe>xd2YxPn3x`#uW%XO{)wC=P%W6&|DO$@3; zs8&cp;0}>V>=eCDsxv7N0hJ;V^?gv(4@k|2Am5Ep=(fQ`w+vI<4w&uEff~084enfM zaTmZk_XODNE`m;XF`Vz72oZM)#NCr&2lW@ZOVcD&$lB{lzL`S1w{VgWor_JPe^UVV zV~{V38CWo5I}uVICP^hY*&yVlw8p{x)BE9yddos<5h?T|$6GzJo}aaX_&oHozV^9m zMWn*Ae*>K9F@~4>3DDKoLw!m`K5wW99UzZh@P&-V=n>FT8=3JE+X46@7mD&aN^C5h z6X44I@MQw!(I&uGuD$_gn~eQcf2Jtn>Qu>qj_(~>z0|gMXcZgXE8#G`Z?nOT2F{k< z9fAY1V$ ztX;8yjIqUReE>Z;%}5wl67nqLc6Lg(5w6v}3~cV@rpVu(M!=5Kg)i{n44i4e3UaJZ zy;nqor4%4>mX%jx1iGnoES+;@ZgVpZ?o~vx(_xOg(R8CXeWMqi7nzX)5-ECRo3*cYcjl*?Dfa1`V z`)l`%IKRns-l3n*b6^$D&2o9}h|8Vp&6i`fakwUZcp-=|jP!`Y=-hK%6h@yAhLVO+ zOExZr0HzUe-!moz>at?5jt%?yxFE~T%Gho`4Hp`R*BIg}`%1|Kf9Kqr^|(jhq8}Hd zMMjP-<6;h63FH!7YJgmtmMQXiuIvyEc}hdH>ktXfGF)!JS!NWK3AHVhMcjD;K*LeH zMF3iXE3**DF*X9N%6eEH+u_r(KFeYI*bX;j6$;zfgrad2;#p6Xc&l;EaeyibkR|&g zvH?0E05#zmS!f}RSB(~Hu{kRiICd;<7Fe|4N3vis3Yd{tv|?M9!=q4Q*kSVF>qNUA zt^+Qm*x(wBQSuG28m@tFQooh@>p_xoX`L^75o6K{fd2tdO9KQ7000080F{JAvty^i z0RxqUMYAZWB?}Righe}Vqkm+TG5`PohX4R3lX0gblYz7+9smi8fQXaKBpH~@#F+_; zw(hNaYpZp^x@*x^TR};a#qP2<~6!z$pG3YH`59f9vAE zR!~DOW@viYY0fuWm|8 z9byztP>fcE$05eJ;0`es@gc@hwedJ7F+mj*RpF&Clj!{vN+we>g_5b1jHlYuvMxHN z|53%MF8CV10jGZ~$|#vm5SP0|g{VX(5>+lSLsY9`rYcUO_gPLcTb%BK0Wrq~{bH^v z=27t(E_htbcZeE?Sl|>3#Uh7T>=I{+C6v@smrGruPSm^D-$a8GlVy}FcZd}Z(dZB> z9b%PBG>O%eG&{r^>Taz=taFGK%C@@1da;3$jSjKNAF+xK(sF8i^T#l z2Kn{NTG#Z(dwS!o@o2!;t%}bwD8j;eU)#h%W2O_<31|^6ZACYkr(kWmT=kKQ1 zwLYq=kDi6B7mump21HjYW`nVKupKE=nSnCFP@r{hEFS2dlB|v6kLvdAk#nkTKd(0! z4J`8oL%mU2m^p+%6M*6=v|gh$$yk4VFy?Cu1^mbt?YI)-fh6mlRW>q&PCtGgSL);l z`Axxapt-lZEf8H#P!NMe+I^vozG#r%jY3z$p7ubG1eZa)Oy?mYKqDqP5lBu1VzCAP zGl&_&QEt%QTzS6Uco&JAV3>qWFl@p}Tunlt8K+60F&H-#)W|wW$o8G9eLa72%4n-G zDAeiEvbHWsk_Z0_lRqI*asn7@vtyqj+nm54nvSHk5nQ#nI_-uGMyjKMcyw=d3#EEZ zY&j+x=#In#Cdt-E^iN0wV$7r@!vd^c60ohPbqC^I5r1NQxBf~5OQ}|x%GP^(60F^p zI?zIke3V*sVTGk6$zF@?sAzvX(ZvY+s?38JLil)ko1~*$Cq=y>;Swal=N`nvwZqds z(Xz4rzr1MEK_VTL5X3>hy->{v8j30(t}a#F$RK-RFdU38W-xuEf^j1@k9P%8YMnBy zQX??y13l3IY9A8E9#q`wSUaYN24V4ffFR-w=O}2=#gY8#$9v zchQ)4AMcn$iiZCUjM;HtY-gh%jbe14Re^ueE&hmsQ(7-;dID%Qh%#9dp+`)-w8Shg0Jv#Z)C$+$a;s}4F9ZC_#l-4BWbo;_< zQ=8T!*)dDe=BAombzua@8A=X40=k(Z6>SP){;o)jc&>P>okNgCrW1b6iuxp9h_=v9 z1~ZPAIe2gi>TQeeYs7`+LtA~p5Kcx$;yJ?61ea?Fj|KBnM~Gt)^zTu{O)7g(6*sH= z1C`ybvO5u@5h;H+8tj(r@V-#0BaOqvZ2D>3a$J(`1V(kW^>%awqA@9b@=#g?JN8Ns zE!KpF z!x8nwpvn%4)|DBD$cb9TlCv!-p>uEzCk^{qw!(K96?!wfU?*!O+kvt|#G zr6NoAU9yzO64|05qEJXYkxC^Zi4;i-$(lsTlFCw|QqdwM^}o;a%&Xt@_xZe^?t8xH zoO|xM=bpRHj9TTE>}>Za^H`YwNn~-(;Z5j15t*K-@#26}`)*B{l3nZJXC}i2Nm=HFfhpHI6CKWe?ZS7mSg4)2)YQM?GKUx>FST@7sM2N4SW3YL7TdyWgLAj9 z4=T#tV%;Itk#=b7tHbuqQ#onDu`-7asFdyFA%!=#NTn6$p4@C8!4pT6g_YR+ovrRlLuEM;-z57QEjF_(H4% z+u=YcT#tNi-HyVG|!FDj@n`1aYm_bk;N>{U0!XHLak+C98? zu&7h_?%*4T>44?zn)I1(lGFsLzQqs6mV-3ra237>ZVI(mKc`ZbJT>0GFEcr%miap+ zJ&WsZtoHkJ_Xb1{Eqf$2YX)0RJro;^GFbm6mZ|Ly$Il$$1cUR&dqg^yA}?E}?!Mi% z-qP=$hkZNDp>tk6<@$@xjO)jTiWnJx-l?r)^xduX*fq`8`TpbiXUbn%WT-;1KU24w zyRB@_VhwhFT|`{i^=|2|)X|37JpB@Lk?Rp5;jxW54cQ-FI}kOFJ&R6JeL3{aa>|%^ zevUD*OCo-f>64)Imt4gd56VU1XTL9$dgv<3C;t3eah|mZ3>nFoBN7QGuTH2aoo`?r z7k$%x?^o7}yYAl8PW?P?`sj~9-z&>I z>jdggSM0nHu{OT-T(-co%biQyg{6^Y$*o7ZHSG2}PA)MVKe;pDLaFh$9fux3550az z?B?OWA!Oy|6C?GSX;DRF;m}I~+5Mx@%_a>GhF=~%5Z4pDc88|hnXRAYzBAZKuu!|c z_ika+3Mros)VaMywENMeir1axr)p(o-_ zsD@H{h86!8`4Sj1Nyow*Rcn zRe3zJWAvNQ$Ky%C@DV~iFPT^Tq%w-F}zsETkX#4J9OXmzM0Tii;hFL z95GPZsI9GZBxuIP-^vVt!4_wK_*@*B_ zeK)?B+|2Shc6uQ1U>mE0@P?VEcVmMY0*{K-SbY?}G<8Vsj-=SeFtLG}^Kz%Rozz|^ zi?iY$=dr7t3O@4faQImrTR6;mU#euh?U%9QCARgh{&#sEm7eiDe|SfG)YnDtBjKHU z7K|2db#&B|s^Uz!$7gPA>li)rWd5;fmc`&#s)fxjlUw(B=P!p?kvyQvW#P2 zv|PHY`WbE4mtL%UBG6M%^~zum_%QW}jyJVUkfGb|^!@iE)A#?n(i8F9d4XW_Ytt`* z+m==N5o6gNYhO1EL2K>RRY&a#emOmrW@m93I9l9MSlhkH?19^hJ4S2Ej89HJVPAeY zt8TkoNu81}UaDLw>gaJNfN#_HwbPn7V=B)d!~|UI{;T9@u}n~dw}1WU+v25zacs8< zLZ+_yTGdMzmUs*Zcf3u`a%FNme~ysA5()lkCsboTX{5bhZ(;d28l&%AydnOxkYl_@g8>d-=}l=XRh z-?x37D`Y^*L2DtFOW@O&sJ!iFXUzKVuix}0`Y?6Z6eCrt^}=ybJloG^8I;?kXEvUD z`s|f6jB{IwPdl3~+Zg>8CP*yH$=Y3-aO}VB?C^6X?+x3}B#&PuTJ~QvHgi01|3ck(-FV0x`&^FDzJl8|iu!Yp zg!aL=cU(yzc4}%{NDn2tcRxLDDC^4`yVEMg;i962Rh`nuq2pZbrzW&d&Z^ctzb}9L zZu0m~cSoy&tyP*Hj)nm$(Te^Gf+y`mQ&U@uHEjj$_R2|6c`ukOc1ZhNGucX3;MNX2 zwudaFU2rqbtt_)yD^T-uDROSmqvfpc>{zZmUF>kyX_po6K3~V6+V6scsyX*>-waMO z`Us`n#(5~;FutX5YoFBQ36bE393mt(W#775E^xAyC+!>EotD)*#(5-q zA2*q3-SKcfj7m=5;1i$xMEDa~V6XJ#VhiM4Idr_JC%9PC+BC!BYkw9{uCj$f1O zvAHM0y0kd@j^-I1hX|IYhoyFhOm$Sn|CCtpEap5;_PC%=eK6vZ_(G>m&wQ&6_(jHm z3+b%_pY2I0r?jd{zPi-g1Gj*#4D z-|mB*!b_S&f8jQ2bK=?3@#d_LHTPe8Jvv|T?y{*!(B|o?h3UkD9j*(!RSs7aS;-A2 zpNlM{A85w=9JW*Y=@%)!M{<|+Gxp)U3dIKnywdg*7rW&0mlqdoet~t=g{z+%-lv47 zjW#!D#hoZOdsfhTOVH=aAfI`HZ!VANxtnYoAZ3o6cWQ2zgwIk-QU<*$s~;bid(8K2 z=(Q|&K>nOnc=!&EwV}d{pCY(i9bLX0sd`+k7;^l&;;vwuF}6W3Na?6_M3MT`)^me- zqUXfFGz|0wZx=UuJ}N!Hni}R1C}nfCVA{j^O_Li@L(RLg$ACC3{oOh8TcC04gKa&V z694oz+*s*0BwD1hFHt4KbGFGF<@0oLjN3#$%64SznrI8a7Y#af`eTWo6}bbX;X;^1_Z+c@u$xuJ!|rPmGd0)r(>cPObp#mMr(Fe9b0(o5I))6t_9f zyuaZ8u3+dyWjCHuK2{jiFwVHZ+I9IF|BE7%jP8dj(j|63MT;a~?K=%c=3L*q^Nu9c zaj@W1A?v%n(~&+KH*FteJK3qB-F$c?FZWGd-qGjhHhyR{PwSMwfM@f&_+{6#-=-d~ z9EM$Vb+ok`bOtX6*MwV8NkduzNprQmE;?o7Y~;r^4+Gq9p52iYa6R0vwz<*sqenQo zqq&gqo$1%*vnHh(tyNb$3fi)Ft@B$x>D8pzcv9ey82Ms}--U}c-TaVhk?{={?Vg{7 z566bJhP6X|@18sB?(@q+cMIF?=x~qw*(u@o;l9VJ1)`lilE$lDXXxuwYiPj$*d2&Z*11x4P_`xPyIYgw& zB4vN}+9s2Mhkb@S<=jIH+;nuiw~iA1zoj(1>`SkW^lRuzZ%aSmNa~3S3TqA$7W9h# zEVVPO@r@?wXy^u`QyO*as9D1bihkooVfyc`tqm8{r|$9#(En8i?36yG3X++S3g#P5b?nI=lSjf?2b$ zto)wv?ia!@MZbN6jwL>M&=Az$o_eL3H+{`1p^ZgknFWUgPbDmnJ`jF$Zo4yJpyO(u!4GP}~f zHYVB@+D7JDn4kO5Dgzo>IUXB)TsK}+3mcdk$KtKc(C&9D^ZmPb{>aXjpWT={87tU3cq&u6E%>! z|J1H4;-lO3UuGq77^pZK46dCXL3<))6O7z=as*+X-qFV0Jw zj+cAcdGgQljC}TbR#;x$JdxJ6Eu}WwD@b1RN1oV=Om#J{HFi_6{grW<{+y^cK%s-k=&-8``ZSC1MFR|s$C zc~G&n|6rG@XQijp!H%VFJvpnOu_un^q$D%h2Bliv)Zs?i_}8Ij8J!N)<({w(u44(} zLhrA=(P5{)^c*I21{38u0!u;^Z0j2Kx|-ZI9uS)QNFqd2ng+!W5eC9Fb4W59o)2wH zKSKSoJDsKe!nZKLA)ZJ6v$mQ-7rJjl#ELUrTp63!WK83B{HSKIOwBmD4mS}c^#$*r zul`6f`%^`2_u*I_LPu?7;rC#B^NbO7U-Oxfd3O`$xcR2EKGn%6MNzf9M@1uQzt7fN zUws)Gb2G8eFDlUY%ZT7-ecz#1(LNiE`L9}hrZx$5UYIHR>~2z#_4>y3YM(+~flm)+ z-PMba#!u{9t2as>IyCM&H5OXJ-BwYb$nbgI_qX)bw@N=3Y=2C~45d|;yFNDEerfv3 zu5(_;ytReBy^{mKW+mkJC@USuDF`Ir!zm;LeB)FYea`-|$5u6*MUtpJyGPgRdIYMsOES|0C zR(QnsnTV|VM~N*0$%kz=wq!+}xpwoafNiB7lurnyYGxK+%Jl0oJ>j9x@Z>A|l7luw z#gKyK5%5(wE?bK>4&0S`K7G0xcOtCivUHe`o<y@+G#i45bNPD=h|@2thh-Rt_jwbxj0 zH$%{6W0rdMwzp8&LDC8BD*lJ92_xH7T@0WvdiNqMNxfI91(FGNt%92qdAfb~CE?^+ zv+POhlc(p+C^ov9>nWRx!?bo9klxg?m92$)Uo*r|c?vI8KhdOg+;rpM(a*XdK^5Zi zn%0Z(u~@&xaNC|->;3VBw+kWq+bezYKWtk!eMmJm_~B{yO{%}cPA$8g@R_}KKBcap zW7nSJu1;6?yB#-ve5v$aNMVzHZ=T4psG{D16NbBWjNVLj9UIwG=vv!TwcoA8_`WZP zLXO20%r{Z&GX$`y1JHxE;$O#+S2p92C{j}<>&esY`Z%lHOcrQRD4nNk!c$w}V zI%{Ic)HHqKp+##P%e9r-+9`L1%0wc$=YewF-kLRr-&jJ&N(AObL~8N+cX#gFL;Bj* zWG80A6WgXb5<(>pc&m#&p-O(fVxoFWh!mmnx#O;%QX&iQy!dgFxT&Z}YEZ{E3OC0# zrRcT%bK9G0;prB_mefbJi8n}3$9GqA{B+a!b)so9wwmW<(D})vH-6biOuy+B9@#Lh zN8o)$)^XSDBERmSl)91MuaSy6;HqN?DWJo!q?wZ!3)vAlanut$NOZ&7CR=MWKG$wiypt@4O7Ee#~Y5^;lA_U4Yu> zZ)Tp3?fJ|OvV^LTf>Y#!EUp?oi}!D;+&|s7gMSlIZD5DgBv(tON~-O-`pS;d;q7aM zqJ>x!l;5>f!?!#gB-YBC)6 zieIXq{jnxBA=|r0b*)^ScHimOTV!>H>Kp3@h(} zX8YmeW!sVnuS;>~{O%r#a!e1NXb<+57;p>DEVhc#|9r4=&b4yRQSr&LR`b2&h#X0c z2SkZ$0a6D>MHN!!kHjXQEG&r;=r3N+G*e(_$n?T3!p9-xxYcRz9EbgkpYK_kFUg(g z)$x2CSA@&KJB}1kdrzj#C~R?Ym}tuM-reH3qt^UrN}WQ67AdRhNqb_U-=5y!nc9$% z>n?j!e!L&qvBgR=|JaW8$1OMCWtH`g9=n(PKFZCmedhINa=4-ge_d!?ncP_Zsc(As zb+fk5?tW3t(^fNNJaS5Ag6GnOju?r=b&~Kw@FTRL1I5aLXZ+7lVY!ZgHhlSWR&&Fm zX3l|*z>ky@j%%21D+U(o)_X#y>L{Hm_tZXZW|FFAoRBa%L&%5|Su~t7)o+fidFQfl zp+oYGLFL4+HifRzj_F9zbh})^$lN_=0(NqEyX8l5s6Jzs4Xg0HExaeErTt`<>e~$K zGhbhSZ4J~e-nefn@&@&m+;H3*!AFT2jV<2EYTr#x$-{d-9&iW0!|7so`p4(DL(&oL zK}shAk9&=>yFTSf_*`e@C+_tkAs#nIojcSy86%cr0eRoCVm9wv(?Y~j;ZIAqr z+uKMP!6?dS!bE0YApd#UM3PV`EE(l<^znpa*ReC{1zo?4V`QW2E>UxvucRBz3EdF* z@kS^{^!b73d)6f z^}8x0tKK-Cl=tH3mO;ju`gJ3xHjNu`1jhQQva6OF-<y&L!Yp8JF69wW?Y z+!uwL%x1rlit?*;0%MYPy1zTGE4>(~B2th>u4E}0oq4iDXPbuphmB$_3vtgBGLCHR zySMc?XZGik_ID!Nd5V?|)*aivm-;wvT-^reY*i+5FR*gE?x32=#cP@4_wDuH*z3lH=ztd=aaD5^huH5})vVgq&O%Bn^O6`}A2gN%zo@WoU zIg`J!uhQaNS$~P+S^bI)i{6Vy$UjuWeWrbt_IA`apMrVnwr&ei}tK_~K{%(tMXu zcHyz5qMiJzd2#PdFC=pm$363#3eGrOAu4w|rZzu$R6wCl+)qu_YpQAIr9T400bE(@ zpfy4|%{t&~zfGfmA7AM;nb8}xNb9?t-!dv{EVj^yv-;t#o)MlNv|p#Bz0p0Q@#*~+ zF5I!6qYsE58;Wm??V1X^bh!Dl9QHpGNB6AnETJ6yz3usKn|$GC(!#=QZMX5euJ&J% zePyFv-Ru3h$J}%6C4rA2FG}*=pDFSSJBbnbKc#HgFScc>*^iuaPfIM7H<3NR=$R-k zKfm#r0ZuLd^D`GgcJk8hP33CEwR|4iuK!FSxhapHQQvXzrcS=!*gEo|T*Gt${WO8M z_4(yv(zuoUo-s$NTmI3M#?f_V4Bra!7{3|fza=rfvtWA1EKlI6jUDxcN?rT~ukYoy zm(kMxRWWsU4i~5J5I1qxoMV6B+3Ycqext6|h{rE%l5*!(Snj*Ye14O;X9vkJyH6|O z!L_k@?ZMqSg8_$|M*HHGZVn{YSoc1RoarBG!x6t9l4|_oqCnle+wx<>RG&V?+!TFz zApT?O#_m`-SA#3+J6iI;n-4EV%^dr_<_+IX%HqPl7YE(q*zdA_oAKASI4$)RrgX=P zCA)w8h0}YjxM$|O={V=?cp?)~XJ009V%~HqVZF%PjPOAo@sv*R;lh>q02|V~K>v!; z4z7%Z%q^nlJspO}srO$?7}{_Sh7VMDWxZSEdGfwd@dsP$N-u-qdqu(InO)DtzMC$6 zT$$%Ey8B(0r8(+(&2s5l{Wnq%PCRI?nlmTA>fS%atF zs_xw}PI_{AWko>vhV}BcpaDAt_;P10_Xny$_;J&yY2*Hj%GaX@4GT2L0|D$=Sp@jNp2`1dd=?N55T za@pkmbu7wa-~EV)Z1(Bv>x`x5&s0&y*0+_y z2Sf2|I>WD+iBY*N7aF%gf7WKtgr~xH8S{VaQ_uRt`)1*;DI--}h>kn8DOx)mFelE}ST;=LSZA44`hE(yxUT=5^p|=!*z?Sq9PgsLV_Vx$& zcgEBSJW}Q79^u_C)$9?+IaDzBWjk-PyH?%0_JBo4jZ?R!dPYB45cZil)!jWL_*^CE zOXO6vkK&v8qV2|K;-R{c3`|Xr~pk@$6cz4b&yy*`r_E3`sPKXbR2&@6i0B<*es zZ#A5A;HxxBZib>OLvON7VeRc zJ*y;?D)6B7%BZ?nzat3$we2`nCBU-r zI;&{YBlXMWo~)0CndX@7q6)m{ekS|*^A;&3LPbmGSTBpNf1RbOz=c!b6SbE+P=47t zySJeBdilxXy*GnHY=&>U9**V(pZ3}tOTPc=2NUDe3QMF*Ri)Qn0Uy`-wh^KB^8v)0 z46U0821iXk9l%E#<8KK+-_GuIh(hh{FBm&?N4aQAka6CYprS9^ZDm@*E#@~~NM3*V z%=YB*%o@MZqUGP%WRVy<9)`K{qW<_KKC+1hn5HJ#739pZvy8iz(2HzNvag|Hog2 znJpWevrl}FOZxapJ>bPud8U)~kL3LvEmH@3RimBR_nFj7;5{GXe37! z24AA_>1wOJ8i6NRJlQ|>e5%8KJ5^t+oPBuJ=GU6@58h`Rja2r$f^@Q~ zhJ*+oGAO!P(kGk}j_>!{a4k65R{PcFK)JTv@t!ROjl3N*r&t|7H@G(#R21u-=+PBU zJK6Jb@}}m=p2i4+_IC&Beudu8#lLE8ZPeh(yu!V6MxgXXp#AQc_&{;)<+J%g)8bSg z;ehGA2c)lvmc;KrQ7{~Mtp9Kg0yM_3}x1 zd$~_G3LjZ%IPxU6Wvv$5Jln|*efA?4Bju@|gjzWpp(Ck@84tVq@=|U;F?7Dp_MUTx z!Gk0DR*Rwn^1*eCD{05`GoYXA;u|GnOALD*j`B~KaK7|sykMZI@TlG*Y_Tg>cv{0h zv%5o~j2#-+3&cmh3}H;wyh@szYGxG>%Jx6BE8Ze-J?*^X3NP4-U$3_VI16xE8GRvjsPRmHOIn0yK*{ zaaEX>!fxCWGb)=mfRn&lb8Qe;jiLTSxGBuDo2GC=7#cW@^TqUa&fp3$zHM)DUr`$D zI*0p>s&o5*>tsY5eWmODF<&MI246Nf;wo7N<{;p?8vPe=iA*RgzYAAo27Kbj4kxG5 zV3BjUXMg|5} z7T7n7C{cm@V$0um9JENq{MXKUQN7axMjHOAt%Vf`kQh=;I-DyHDOFx$Q! z!0R!h@u-%Le~d>CoxpD9KV?fr6pA1m2qh4;w8Kf#z(q~ zZ)9Wx@x89Ju9m>Z&A>obfzNj1MB)BRc;1FKyc{>$e#IgDMXU{yR#!IbfDUFQ!!4UZ z8**O)^Esd4ZLmuGUgBG^lI$yd1vYdZ&Ek^@sQWirJ{$-F?$5x(rgp%2sBq3ld>Ym| zzIpr#)^@&c_&Lm0!gqWSRtj6fcVepbR`A_;)aqu!kWCs;n2pCgI^o1ff+JS&Z15!u z2)gdVcodVJ!7(;*W9|GX;v_7^z%aT6=|2^OnYD42mr+>?bq${$;}?Fk1lcp5!>X%u zV&ulubUHJ>z}#Qq&iENaOMDo+Sx||cI~d=R&`QqP+vX&Jj^0JU!W@_-)OaeAu>jAC z`1*dLQnn57VKdwsiFn%PV!jg524o+w( zhQQQ{HOGL&|@H^xhdz$~@pCv;*Z0YSnEEQ&vU zaa26R!ocu>2bP@2%fS7H1P<8P4dQ6TixMia+H1fGHKyNFo`9pE?ef(m7-Ow2)h7JL zM%`IMf;yJnl0t-YB|yI>lVC|bMitmzkiZPT@PNpTqNW5T%pX=3gcdAZSZxW6ENG8^ zy%TSw&dk6d&kl3sf=)!ESkw#5d)r(H2Fz&9r@RUMm|R2<;W1`|#V$fLX6vmz1beKF z?9qe(Od@6 z*M%+>HmER@0C5s? z->e{U8f|<3J_Eh^8ULtMaeIvbHgR}pkkA|EXI4KK6G z0^X?-nHnwhiTYU6Q%s2(m}9PQCcePZyc2tqRvM5^=7#+$K%S{JC6XKaJc$=Dk(dDD zajZ40JBfB!$v1>Jf|U%yiBp*2;*ms6A=HCTS;Pvgw}0mm!D0tl5!fFminE|=oS(xJ zkQivqB$$FcVbgqq65JmQ3BbO=kVxap5~2bYx}-BiJ|<2i0a$%)@I3+qdO@%_tqQQh z_PfALlyRO2V>L*Y6BDqwJ6%UK!1{db8qtM}`Y}1yvs3Qk9y@L~jTY@2eI5!mZG~7y@Kn;Qu?h`+veHGG9+-i)L?lLk{Fl!$Y zm`5--Tx4echS5sN%nz|CDT0IfEH)tuyj$z#WXiyBAs_80xHbZ!G}>}8hnS*Xy5-31 z!iZ))vuWzngTSB^I3f&WJ-{d57~;yDgSBb-WNEe0({y|M6$aI z%(e#-YSh}!tc$rKERdPY6Lkge@And?!G|30G9UvF^mhaV!FO7i8_+hzwlaHR_7Al& z`_VeNllccm)9GO*VZ*oJ#A1U5FtCjYSw*P8S6Z1_8#DTtX*2qVe&*X)mbv_t`6sqQ zsCvU3Oq;RZGv}W}tE)c3Qcp&^*|D&S#T2Y|uCg;QY=QGyn3Wq(o?r>b0*Unu3l{+$ z{tsm~q$-1sbU-IDl}3*;7CUS(S6^TW!BVTx6_$K#Hm$wJB7-fIhHtW@Vgc>l#?prM zm2^AH3@x;~Sv;}C8#2Oj87m3TveaY!H}HW)9zr{4`3FloGwMZ2+2i78LHplf$)S)y zVr^u>lWt>v>S7|PVpjbikd9-u@Ds{MD@fzBo2;K?>- zkW~Q;<6=n7sG7SmBuz5PSKYC8aY2HCA;BEWM}QA%)Jq}ZBvD4gbEFZh#r#)DoSZ04 zxrKBK^Y6QE(p}8I)sILf*r5CLj6~Zm{yFl{8T2gVO zFVYBjP>%`T+DcSuH2O-~hPAtNk@N=BdKgc35Jqc%DN5$WJh~`GUciF?oiuqG)1~vt zZ%~55z_5=Wnb}nu$?{|aOv6S^aw4`+?=T`OW8wGJlpKX^UUpfN$ylASHsnf7SKy7T zS2`pHh83Q_>Gp~ZnXfU@o@|LZ;HEp-5gS$<-sD* z#f!{UDve|DWEiV>Zz}mB77P*x$y`{DD(JW^#tt0f20r@M!>rP{JBMtDg;RANS(%Kw z;n_*@R&2&nJx}&yM*A|2^Gj9$7!r|qq|d>!CLiL3e;tEZ8edkC!!W1WRg<}iBqW{Q zF=pJi3w}9D)`VN~LCS5qN{+^O)UT0g>qzki@=>gPwM}H&y87-N@+QoEK z!DpzE{I1bOevi?_d;inI)<<5%Q09Iz7v^S>$7Bwyq&7&-#!7dF$oyEqjEs`oF|U-4 zkv(a1$RybS<6HNdycH|uydk&X(6+DBt!=#qd^Sq}>pvA(tpx0jN@vM;Fu(b|C*Q#m zZ_U<<=rZ6VK1OWSko%r2&=~xIe2vx#pUB>r3CW-TGhx$LvMd(85kJW{Fh1p9mrjHtr#h^ELuaP#A4u42=5lAkQ$fSDCt5!!C}5^lcR_!p#AmQmg0{^l$AS$72DrD(T6B2SdNJ<_)k?POfkYt?Kwpe#RBSGDFu(wK9y6zHW3LoTouI| z8*F~{6gv)-R^LV`#!&lC$^|mxmV*}X(g4K)4tYZ1fIS}nSGx6>Vgs)Ov_`=v6p&M_ z4}$O=q|{=i*H0-YNhs~WI3-30DZ$IUkRGEIDCqD(>yc>-j^l&W&_Wd-q{D=)AmExB zNC^JT2T38uz|#EC2DGq~AF@U{8~CBkXn{=#5{5+uAYBx96##oDL;@wZ|3|J91WZDL zpt@BoRuG6Fl>^MoA!Twx6fNE-Z}h6rpc2FZvc>}dyg*11Z9(P_uPVD=v`__r*@1I`nJ98jHqf0ToP z!HABD>)?MnIba=eNC0I=zZhMZfnjky1A`77`?IY7*xBJyafk~w;kd(}bul0ikAiU4 zpkp_K2gD&0l<7w*gWfQifgy>PfnhBj6AP>^0c}8;=)N6-j!7NPlYq2QS9VK4!g$gQ zVu)zI^9(yYk^rISZm^^zKs>)8hzRT<3CW_ZsrdS0{{di{92ok*)vJGHDFI6;YyOob zva-E34$s2~3}j$oK>Vtd2fn%tG-?H88|76Q#IFkQyd-3RnxY~FDs)>xvnPSpi4m;isO&@p{?h8I*IDcD2-BEU8-I2PFZIIU83c~I#I*Z;vk zy5flN^Z)R}2;ShgigUmt@{kPkzh@3E1 z28OM4b^Sum`aeR?lpr3onF~sw>h6sH zfz%G+*w6?<_5D3t2KzrcBieHRM@TUb2f-_rwDt@r1Emr0|KPG~AOW;KlQjV9Em?&) z;2kF+F0?A9zQQaGu!%7Q2l(oARlO@k_?6*#aHfs2GIxy5c!6&40^L9#*TEGCt14V4 z0`Z`%JKqutLy(vOo;0S@%3q1GMpxpv&{Tu=umFGS4P~&6>I12let5rx=T#t6lu2Cd z#`SA`tWnxpT#Zw>@Zx`4@qj*0&cyj2w#8GQ$gpp(M1!K$mQT@wrV z4FEsL&#QlBDbHYaHOLnAUy2&=Urie#uL3_%qm9c6gxcMSv_}dSSBE4}19me#^uGj7 zdnP&mZAwh{Dk}sZQKxmlDchNg$XUrCFhF~?dL&&%4*U;)-vEvTH;K@!&Q;VAK)PZZ z5TqA64ohl4=BOoY^+FlPL5c|gd6RyQFdO1hya!KeLC$Ec z`r1H$@GnIFTKK^)96NkMn>LRB-bnmA?xufZ_AF}CCZuLlA%Sw>jL)E}=_9$G~0xGD&oFfeh^Hz1O-%E}45>e50mR2Q^` zQ51t}#qrGWE>XM?Jf=(Q&36cHFTRSi!d(Wm-aHoe#8(LnF+q-SYoBD;iO%11qhA1W{JX z3qD&0ke`2nLqmFfa4c5;axATWv@K2RfN4MXVUSNEo*Nc3q_suG5J0391k!@{8`1{x zucqlu$Z6bpFo-wO89aCZVU>rKQt?vgP?7-4<8>tQiL@xdzaJPypUT+NG5#$^V2~gy z=>#KC;p;;f^^Oswh*IAn)U!DlRn-_mj~w7Fpnz98k5O~~qt+l)wsMS`d>+pNhnC|x zVaj^YB%}tR^#J#%!tmNEya=4|A9NUj4qaM>xZwFqcnE&~AN~`;-`-ip*MA$vwstbUET-QwN61FyLK%un^Ou>k-qJ2&)$S$`o`S!dh29#bpdSZV&>SSvpq2 zD}>b;cH97oi6g8YvxBfSU>yNzbpsvi6*zYTv=z-lf3Ly*oq{wA%^*88B9BO<%a#H2 ziolA2K7A~Hf^lY$0qi>uoN~pC7It^cKqD8wVvQu3Lwu;^BdqDL2pFDc!GH8i-RVVy zRRK2qjt4hp=Cq!sngcDiOGrA9z^);{{TKnhw1nq@U;fAR0bvqYp=aWS$2NlgL45G{ zQtsbz_xcaL5PD4~fC_#60Nru|_mB39;B1M`ah*8W)EZL6uJyo+1727SS*(emT5(<@(ZJ*9pzuWxcq-Cxzn{d!47(%CAs|#}3)DqWR$+G7-ImsaGWANa`$QNR zropO_J_1)*F@AS8CRX?^;72X%Lsa!CA&?dMAaYX5TS{%9)Q)72%c> zOk`Nufi}ULIsmAq;C~>FLYi;U97twOa*PJsj&a~c`o+c06RSeJ@I^h~L2!lub2;uJM>OUQ?s`oc?`cf} zxVl7k=3RXb{ME<(O$LK6WO@>tc~20e4z zfdhCr0pq+!JDk4_g3lD}gNlYH+W(#-nnuruPMjc>|rnA8h_g$)mh~ zFZTZp;4jWJUU1DXiMGqs3mBQ?O(SP`L9!_M3PQFIqLIhFAr`nch#(Bx=F#jqpf_8} z3tHg<2DCYyJ+k2W3EGPPq*&l2Z%7bzNs%{DTe9ab1A2xIlYIdBQxuJC?*pON-|%jP z?6QxLe-`+pW*>Mfv7ADyo>QuSeh~EEI*@wk$LMqp#*e-+fQ`Y)2GV_ozCaas{{KMW z`2m)&YZ2&8!79X!F8R=i3b-eG5IJyBm;86h_p=(~xA%icaB&)K+mpBrsJeA?6%v9~ zn+f7@gdc6lko^Fza%UA^gF1zZP|x*XROC4V44yNvqK9Q@J__@C$*~KJ|5Kp7UUaU0 z4KC?IKz;*4Jg~`j+NL&QJIE(pf~$}a+`pX`spAM#E4B*pz#`&ARy5V1OT@nqB>s)f zdN|UbcFg?uiNwENd9AmqnSef+fO!LGu2BjAHT#)jctil$S0ckO+4t+_!yuxMD1a3) zUDFCSud^95I`LEv=Idnb_%-EE*=R&Tagjsm&q4;WNp{J(dAzC_|ybtlmBGH;d24QK46 zCE5}MQa_GChmI3D;Fqy9U*LiPhQL5A`!MdL!=-NLln@6%S7*ssy#xnM8wLFHQbE0^_-h)h``NJM~z7bpq1430>84tr#nK zgMv+tb)ldE9`_NdBK$oRGz*#Ngmwd}bvs7gwi~SQ5Na|)J;Mm+Mu781)i96*`*+h0 z9gB>thZ4b{>;?@srK|BWVU-`dyCLcNq@4t862Lh6qr5|KWEfSo zo9Xl=lUMbzz+B;w3~HKwI55qgR{%eK7Yq7e{P8}_;4lVk*-`y}U)}uc(0hIuKi?im3}t;aB{JUT!N6eM|L=jX zZ6Ly`4yWy*rJbTZpvC?z2z3qeG=~kI2Z!URoBkk>Yb!hv4US}x`xCSu@`CV66kv7^ z2;y~g&1`7HSev70>&Ss9pg^b-qf$GW1z^Kynr`c80P*!;(CKK}S>fN8KL3t;DR?{@ za>Y7!FA#m$kBMgQrQQF1HsJkM!~t%%r2pMA9f2S1g`82_9XoTj5Wo({AI!`2F_ifR z?%xXz48Ff%1`qFIXjYz%0aY!&L#on-=VNFWFh7@jd)&Y>${Z}COz1q24NJyCcIW`6 zdt*e$v;oeErCptVhz0sToQ88_LDe2-Sa`7vg=Cn}H~`_x5XcA)D`O$Ty>YZ49Ydh( z3kakF&t72RM5WM$ja2=Oqb49qEkWX>U*Y|%VS)1^z=+O{2l*GN;Zi)9RdcVgpo@pu zYqZMIS#GzLkAo`kQ7CXI{f?^<)<~dTE+-`bd80NKc=7{_3S8XA0Dq51h%!ah{AUJXrA`j2b6v2qG6T$Xwi8GAzP2o$fQI_7*)?}s?6yG znkfOANk4R!XK1``vn&E|W+F{}5s+l8`+#D)@JEDza|TFx^e(%|^q&kjyna8eX50Os4@beu zmkQ3liuQx6d?Yd5L8uYT2vr9?$3V>yhi;g2f!_EC<^uZkn8<>$CQu=MwBtVL=jFh_ z?_9v|^hx(U*c3wcsJ`*_PtVG8fo0*Ef7cX~JcvH+zf?DO#g}^@y=Ft#vl|nyO&%~^H{`2-Whs;vdLdRgu7w0%8>G=9=t>&U+|y0Zq=r_i zb`U_>kNyWr0u&TFi9nB@VUYY7NgUh2A;HI<4xotV7&Mv=4jN~l(<(&6=1_m2tvg8i zeBipzchykj#AA7URmdETV~Gr)YTyknY9{BDuNkM2uQ{5^sP(2V{C5GTf8 zNgVLE6k1;n`?cHL0BsNhqjdvaTke26nN=Y*aWXeH4w1gPpZWD+Ixzb<$o=+oLPKKo zLJD-xC=l5=^ZwaP&}<*zKodII9@sCFc3kxLY2m-)0m)P z)0=ip_V@YazgsC@7?%gRqg8n40Ug!>3BUs!(b1<{HrVeN?Hsr27!co6g@`M{^T)s)IMO342jH%|U}D_~5}WH z6@nYj(t0ByA84_-ib3FwCw5x#9-(qvTcz^CN!Q5y*gF$Y3mkeJ;D4^uUP_q*e zDEtiu{a@Ku8`IPkhI=LHu;*9`#9BYrYBXA_qC$*f6e;nSY)ljXpbnXlOpS>cgF-?u zF`LmWsm=x8EGrD8qflsROH7O(KqlFSk{E#*MFmMTQ6oYKKa85W=bU@J?bZG~&--=W z^ZoX@VA$4ljYl+MKz>G4B#2FMA1z9BK4 zJauR_O^DXDR63EIo!XVx^407pr4*~vN;}T7g0ViN0 z`wp>HAU0zeZae*4uTd{X>k$P5vB8fEeRn+yX65fKW?JCI%TElHU2#_5!l7Y>AtOu4 zY@)Fi@LkKNE$Ov;*=$LVkvdm4Uzq+L@`8F*U^sAAfUggoi^CQhfijFTZG%SsNwA7P zJ*1|<_fFE8tNytl@_N1jvHQ#k_+A5d$-Ts}v1qXvmMTPGj5>9u8uj*>Y^hu>Qr^;X zZc@A%-hcM6{Kulo+ZevPyV0nvjQ&P=4p;@`lKf=?e|e6?rM(M%k)stjeyVZ3GG<_Z zW|C6%CGM8;z5V^EB3o$~UgN_y*O4TD-hnjhLD-rQgloQzI}=cqfXWu?5GSyslj18a z{7`s*86H`?KP3`2E!CA$X+=QW$?^lR*IT4Q)d4<7M&0!rJJyHG{1RANZWw1zu_{_w z%s7n%w5*K_D;p0BD4RC@!Iw!ZJvs~9WctQnCA&fu{m9E!FaM{!Tc7@pvAvD)DmKVJ zJ&`28r;=sR?q<#Pp=P*#TczV)lII8xL~-t1KI_PFA@>CsFQflrQi8lh2B{Oh;n-_% z`s}nYx}Lg!U}m}=)Dm_#2!A`zBtboA*etrbP%EhXIzIBa6}A?k6Go>yYlYGlYHHEw z1AC54{56PqF%S2Dqv#=e(!xvR#^r5=zfhe}&7*JIwJ2hu^;Uyu&7qZzZ0^N2Eg#E6 zK<+dPB%9U*nVozg4XP8+SgQ_!{uyFsT{Eo>u(wl%OWIHZ5lQTN^Lii$eaXTC`pS@> zU#PK7D>-}+s%!pMRg2pB7R5)tcD`9TK>h99?E;lOdGdn={PAZqjeGA?W|*&=@!wtA z7RIW}iePuH3Uh19ssTpcAvTW!VeV1()15G=#}a-g;B$bx2bhE09iZ{%MYvV@lfI6C zcpefaD$$Vg3Nq!xSX}iHm~c{G1eElmK}I!$_(}naZsj!X=-``_PV#ktc{~ElI>a2a z5{zX*p`Yx)TPJ-;#wFd}F$LD5xkRY~m`7vke@66171(*E_F8i4i`%AK| zsrBEv+40NF!O~gd=yt31$nz80!x<|C!Z|_c4CgAL3!G7{az8i+$=%>wEcb_V zmCzyYmFxi6L3szxxnhUB@ronh*-BT3)>M1kZ+34dnoKQ{mb+K*$m?ZxwA@s}#8xGX z3txm_o$V|HKIYLJt`@l3qhy8qf;J%W4&5OGC=buz;5$8OOZ`2w$#lU}OOcp|mRrk$ zE#F5rnL+KV4`Lua%dx?uC6IJZUStp8m$VkVI`Bm=^!YmTbBr>1(FO-_{+3h07pDc2 zX_urWZc==-IoDW_C9U9+03KortKo?(-v7!*4{MKEL@=yewqZ~gW-?_s=A9kvqb2qg zWJ}KR0SxTUo{6vrE${hC!9sQ0qH`Z zA7q*Ny|5O&x~~*c(ATU2K{o?KzT&>^1(L~RW}5#tAygl~*;I?P?Z^J0)zdjbyk*)C8f@TML$8E$Jm}~6(*!T_hCR2{LmJ`B!v_>1zOJ8U& z?m^O{JjG8jrL|sSF|?qb6e21ZWJIV~g{XU&c+v@6^JuZi6VToyaf3Uc57WhO6ewqu zSD($R6q6~&P8(?v0(7Sg@d)0;lpS1_?P3%fqRl-i`D?jPm|3@KDUf4)!HNM2j&m@liD<(@lu)AC{q*{E%<{8pYNbA~JQZ@Gc z?60K+%>48UsV7=E6-X%{?)O}dA-o;xKwu-y{$vxKIhrBXbG$UG^P*+N<0 zd~~AtqOoo+d*wOO#A=5H-d3MiA=P0II%zj<%rj>HjAQGz^Vf54N`=_5o$IAhSjURT zQYrLS@o+f^qb+ohhq?mAbsgmHScZ4FY(|2@qT~f$kU`5&;o5YgCmM}$*=D%Enj|m7 zo+vDzyl#xK9u}K1@cLnL7B+!@n%oho>^o6Dik;^;M_!J!{g@?>z-2nF+;ingI&2r^Y$XohyxfnYwaI<_3k?npt3xj-DBA-URVWn~c2HE+WoQOy$ zGh|ST;#c1>&DeieWi2H{^wDaBc-oJA)qwYXz^(-fMp6B$Yb>b?9*S%+wCzVahB!Uvla+P*!u zDm}51tZmA9%x&;4Wj9)=N0d4E_+OVQ7L3v3yiy`T#myzz*D8!24R>m#rnjD0p)_HS z#?>mTu#b^IDch&pUD@QPt?IR`M5q^mbI`p$&zR{5vjKw%m zOWCFLDsEJFV^5wxxn;){Nyd7!q@lef_&rryD~Yj9RS!5Qf9r$%E* z-vy{u9zfgJD76B|wpV}kAnw_gWVHxwx2LMj?V+UPe^#u%Vx)V^MmslxbL5s2rs+$az zPsw&17HBNeZN}8~(dL$_;kxf7_3s#BQnlKM^YG~{)dgjlch$qVN4q^&>8qS1F+v&k zEE6pnT$vBPp4pp?#+RQK!b-7gVxpNFel(Jk*hDO?bOfu!^wy7JD%uu}XJ^pgtPHjt zi4B>~q7aRp!6dYpl*KH#0EVP0kyDHXu&xb0S7R{u)LKG#dwu>wHVY#OxhxQSJ9HTv zjJ|U6SiBcJSNTpxbb9~7YOohyl(9P0yHd{H=92NFS986{;K!tvc#-vNwbYhhWDx+4 zUS#lFQhRlgMS~=|g1rY|ZUuwSw05+D4Frj6CF>8M>N4}vvMN~+{I#``#RGVB$q+kU zVv+Dy{3Vw7msq!o4W)m8IIfC)3}90g>kHsP75fB$k%N~u_A(2Ezj80LH~>c2SOC7& ztT%vB)ociW1J!H{fQ~inQveHVC{9fcO9M&l6*d?^!4)RR901ETva-Le8tIj{Hs|0Ky?D{y{BiH(0o(N8@iHiDj0tohGLa^*RhcYi^#@?X%q3onU; zR4XWXNojO?MP0Yx6}9@qR}{4GYkG_Iuc->9g_>e~E^wU0cSVj9R48&Bw?>hy-j@gzNE|!%sl;*Ohsqo$XqU`!+_*vx zM=Io}*ebDC$$WwpoKVTC1EYL$7{~doW*mnrmUC>~J)Ec5wEi|<7JfMJm6l_}U9~rD zcpvOp8;+A#X9e%u@)X{xetOcg@3m|j?xvO4lD657r-G|(!R+}1T&un;FOt<2hiPtvMX}n6F}nb$fp8W;z-JNP8)MXv_FdDCGIa2w#HM?3+*0*k9Ct~YH(ix` zQ(e7$$ZC%j1o#rq`Mw-C{R>}WBdtBhuR&>hivErt)xOb>V!8VhSnN-JTKp;2)BvLQ zdI0GpcOcJ)JCO2Q9Vwqp9VvuMCrV>RCu06~C#oVLkUWD?zV!?p+s*{D4z#9;cxKi0JgqC6QG0bs7?`tvJ~?AU zegDR)^9<#~zs}*p9A7P+-0UfxB@kg`+HcOO$MM%<2IWx4``(fXd)h3fw7ids^=s4TdYuN}@oV zl$w`}sh zsNxKUgMt#fOLAA1-5`F&$$3{fV2H@C_XXhgpA2N!cBYsI!`k&b#`YryL#lPorE%^` z>bvY+@FZ5BDg|Ciz7K}L~Dj|-$Pug{|N30q^tHC&27Qcld;?jJUuavvqIU0 zIBq`r7@EMnMvM28xf_gS8uusGDrs5B#d6VL$g)tp+_fUdPUGy9m^R|3E}p?%Lc7r^ zoUbQDnf4vG4OtyGaf!&aB#%>#P$3Wg=)OF`U@#KJtB4C!XYMv-;!wm{A(!6)E>Hrl z@x@#)T&0?IgnJuCLH*?kmx9#_e$6Qspm)}cUxcwnu8fS{VQDaAx+$3f#9e7O<6YHN zmi%EXdYCi+DTcJol`nx7RO14837sSa@^y#?2Jz=@fSQK$h0cJE#PK^E0ezUle=R{e zQ(TAd-Ygjm5f;kiV$NH&N##%BNkqko^%-_rq5;bC^MaR>{gfD0%fNop&)(0mK@$3dK7W3<1M%2Xr z@`bh#obi8rE%wme2L6&Y$d0x0l_*>Nly~X@p>KD8IMzkG)n*C-s+WimiuE_Q6egnG z3|HX@7RfU}m<{VqUH7q&k9t9qg)Mj!=RXrD^4H8224Q#ErwUDYi=B&vAF-_4mI(FO z^Gm-HVleWmjly8Gu*nk=(QoP=;V9OmZ;@~xPn{147GS%_?VF<4os^+dZiE=3ys^*C`o z_C$W=+=6M^dRSvb$J-~0>DUBb$zpFT<%rqhY3w}fW#R@b+Yjkt0xr`TmCoCKrUt_% zTi{kiLT%LdR*7X8HOUsIVBJzSh$Yx*zPrU6h|G_P>FDI!GO-HvCYFma=;Xk!Vk{!A zLX$x%l4oo0WNrUl7nQ^S-d#ClMs1Zx=9H=%&x!Mq!Tf^BZc>$r>)=Ik3~(L&P3(!4 za;z7Vux*w$izPU$FUFP4eOsds6BXN2Qg8hY=cZ=d6el2aZHt)i3~{)*Y~7pJ(_nZN zq%6NDgQA&|X+>Ss^4H={Si5HtwKw2rcAO(lWgKugaxj^fBnNR3<>ion;v}YP22=w$V#7RE@2X zItOo zF^*9Z4@yHy+T}yolb6r$+25#r>a28DVv;#`CDWezDm$HN4^<%YUA((CTlo{5fl;1v zI9|@T-g2E2lx=#LT!UjfWR!dY_iT5(T!^;&lH|@_kW&1=Yqm6M;ode=eoG+z)axr` z9;f&ADtRq7@Pcf4Cyu+zM%e;S{lAw-BiFxn$(@Mi9gw?m7tb%33$QTuRq|o9sJJ3K zh>-cb4C{%$+9EB|rf#^hyj%`eJ+8|CL>F`F<#wEhPjAb1C`-F1m*O5B@LXnZz*BAhrl1B#C$vninR1X3#j+CGKIGN8hldIAQ33}qP zi7yANf$PaQiy~VGik5^^rC_^qh4brc^g<1`)V>kzh5P`nhXZ8 zp_+sMI9*NPTT*#dO~OGEUPImku)K!AYg##7L&ku_zLvnps8V;0xGL$j#1F)KYe_VK z$5%D6^;HrIzoM^_*uTW8F4jU-G^Gy!MAeax0qm+HBLF$ zQ6SdbMgs7tCqsdHN&5 z*`TxacV^zBl`ICyx>k}3KyD-R04#1}q1Lpqc!D1?$u|#Ks4Wj!p~4?Au*n1-Jz_-| z-_CA%Uptf7JZ9H4?=ic$y2mVw!GEyw=KsM6JN{sGj_+WBo$nx%A;P{-Sf<$~&|v~` zPnoadPg%^4o$TIIJK0fPC-e1>E=IVii|KT9nVkN~Bu7lZ_8ALo<}>E~!ZTJ4zvs-? z%I8ed_?(#!e8GI>zF_yxy<|m5GJ&#}EQ~&{Sl4B}Vy*t@6>~b`HM@)LuUQsSH*1pQ zZszotZq&)Or!YggGwEk~y>hGRSg2b7N3jXBIEoW=lcP9^@A4EUD4(Y|ZtXm?dS755 zPoUVbT>{0Knjlh~po1dCaibC=oGdY(5|hMIX6Ca@;GE2?dj?A#5GhmPYZUXT=>F1za z(UYbF7-`R(p0%gTKr+ICfwK4+9X0`P4_4<~4~m=qg$JufaxaQ+gYsU? z`#YXm?)nzr>B)RKdNHuZi}7@OF<l`wH ozJXeKbxC0$T>&8Y4Z0A(-Z!)f(2t}b29kob;m{?=gXpCH14!d+ commitedTransactions = new HashSet(); Set abortedTransactions = new HashSet(); - SequenceFile.Reader logReader = new SequenceFile.Reader(fileSystem, + SequenceFile.Reader logReader = HLog.getReader(fileSystem, reconstructionLog, conf); try { diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java index 782c5a572da..3844d473851 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java @@ -46,6 +46,8 @@ import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -55,27 +57,29 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.Metadata; import org.apache.hadoop.io.SequenceFile.Reader; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.util.Progressable; -import org.apache.hadoop.fs.FSDataOutputStream; /** - * HLog stores all the edits to the HStore. + * HLog stores all the edits to the HStore. Its the hbase write-ahead-log + * implementation. * * It performs logfile-rolling, so external callers are not aware that the * underlying file is being rolled. * *

- * A single HLog is used by several HRegions simultaneously. + * There is one HLog per RegionServer. All edits for all Regions carried by + * a particular RegionServer are entered first in the HLog. * *

* Each HRegion is identified by a unique long int. HRegions do @@ -100,6 +104,8 @@ import org.apache.hadoop.fs.FSDataOutputStream; * start of a cache flush and the completion point, appends are allowed but log * rolling is not. To prevent log rolling taking place during this period, a * separate reentrant lock is used. + * + *

To read an HLog, call {@link #getReader(Path)}. * */ public class HLog implements HConstants, Syncable { @@ -352,6 +358,73 @@ public class HLog implements HConstants, Syncable { return createWriter(path, HLogKey.class, KeyValue.class); } + /** + * Hack just to set the correct file length up in SequenceFile.Reader. + * See HADOOP-6307. The below is all about setting the right length on the + * file we are reading. fs.getFileStatus(file).getLen() is passed down to + * a private SequenceFile.Reader constructor. This won't work. Need to do + * the available on the stream. The below is ugly. It makes getPos, the + * first time its called, return length of the file -- i.e. tell a lie -- just + * so this line up in SF.Reader's constructor ends up with right answer: + * + * this.end = in.getPos() + length; + */ + private static class WALReader extends SequenceFile.Reader { + WALReader(final FileSystem fs, final Path p, final Configuration c) + throws IOException { + super(fs, p, c); + } + + @Override + protected FSDataInputStream openFile(FileSystem fs, Path file, + int bufferSize, long length) + throws IOException { + return new WALReaderFSDataInputStream(super.openFile(fs, file, bufferSize, length)); + } + + /** + * Override just so can intercept first call to getPos. + */ + static class WALReaderFSDataInputStream extends FSDataInputStream { + private boolean firstGetPosInvocation = true; + + WALReaderFSDataInputStream(final FSDataInputStream is) + throws IOException { + super(is); + } + + @Override + public long getPos() throws IOException { + if (this.firstGetPosInvocation) { + this.firstGetPosInvocation = false; + // Tell a lie. We're doing this just so that this line up in + // SequenceFile.Reader constructor comes out with the correct length + // on the file: + // this.end = in.getPos() + length; + return this.in.available(); + } + return super.getPos(); + } + } + } + + /** + * Get a Reader for WAL. + * Reader is a subclass of SequenceFile.Reader. The subclass has amendments + * to make it so we see edits up to the last sync (HDFS-265). Of note, we + * can only see up to the sync that happened before this file was opened. + * Will require us doing up our own WAL Reader if we want to keep up with + * a syncing Writer. + * @param path + * @return A WAL Reader. Close when done with it. + * @throws IOException + */ + public static SequenceFile.Reader getReader(final FileSystem fs, + final Path p, final Configuration c) + throws IOException { + return new WALReader(fs, p, c); + } + protected SequenceFile.Writer createWriter(Path path, Class keyClass, Class valueClass) throws IOException { @@ -636,6 +709,8 @@ public class HLog implements HConstants, Syncable { } } else { this.writer.sync(); + // Above is sequencefile.writer sync. It doesn't actually synce the + // backing stream. Need to do the below to do that. if (this.writer_out != null) this.writer_out.sync(); } this.unflushedEntries.set(0); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/java/org/apache/hadoop/hbase/regionserver/Store.java index 95c0bbec8cd..15336d0d444 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -315,8 +315,8 @@ public class Store implements HConstants, HeapSize { // general memory usage accounting. long maxSeqIdInLog = -1; long firstSeqIdInLog = -1; - SequenceFile.Reader logReader = new SequenceFile.Reader(this.fs, - reconstructionLog, this.conf); + SequenceFile.Reader logReader = HLog.getReader(this.fs, reconstructionLog, + this.conf); try { HLogKey key = HLog.newKey(conf); KeyValue val = new KeyValue(); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java index 11af9b0440c..0854914a7de 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java @@ -25,9 +25,7 @@ import java.util.List; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; @@ -35,8 +33,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.Reader; -import org.apache.hadoop.security.UnixUserGroupInformation; -import org.apache.hadoop.security.UserGroupInformation; /** JUnit test case for HLog */ @@ -96,31 +92,12 @@ public class TestHLog extends HBaseTestCase implements HConstants { kvs.add(new KeyValue(Bytes.toBytes(i), bytes, bytes)); wal.append(bytes, bytes, kvs, false, System.currentTimeMillis()); } - // Assert I cannot read back my edits because a flush has not been called - // -- it happens automatically at 100 edits ... see top of this method for - // where we set it. - Path walPath = wal.computeFilename(wal.getFilenum()); - /**SequenceFile.Reader reader = - new SequenceFile.Reader(this.fs, walPath, this.conf); - int count = 0; - HLogKey key = new HLogKey(); - while(reader.next(key)) count++; - assertFalse(count < total); - reader.close(); - */ - // Now call sync and retry read. + // Now call sync and try reading. Opening a Reader before you sync just + // gives you EOFE. wal.sync(); - Thread.sleep(70*1000); - // Open as another user - final HBaseConfiguration conf2 = new HBaseConfiguration(conf); - final String username = UserGroupInformation.getCurrentUGI().getUserName() + - "_" + 1; - UnixUserGroupInformation.saveToConf(conf2, - UnixUserGroupInformation.UGI_PROPERTY_NAME, - new UnixUserGroupInformation(username, new String[]{"supergroup"})); - final FileSystem fs2 = FileSystem.get(conf2); - SequenceFile.Reader reader = - new SequenceFile.Reader(fs2, walPath, conf2); + // Open a Reader. + Path walPath = wal.computeFilename(wal.getFilenum()); + SequenceFile.Reader reader = HLog.getReader(this.fs, walPath, this.conf); int count = 0; HLogKey key = new HLogKey(); while(reader.next(key)) count++; @@ -173,8 +150,7 @@ public class TestHLog extends HBaseTestCase implements HConstants { throws IOException { assertEquals(howmany, splits.size()); for (int i = 0; i < splits.size(); i++) { - SequenceFile.Reader r = - new SequenceFile.Reader(this.fs, splits.get(i), this.conf); + SequenceFile.Reader r = HLog.getReader(this.fs, splits.get(i), this.conf); try { HLogKey key = new HLogKey(); KeyValue kv = new KeyValue(); @@ -228,7 +204,7 @@ public class TestHLog extends HBaseTestCase implements HConstants { Path filename = log.computeFilename(log.getFilenum()); log = null; // Now open a reader on the log and assert append worked. - reader = new SequenceFile.Reader(fs, filename, conf); + reader = HLog.getReader(fs, filename, conf); HLogKey key = new HLogKey(); KeyValue val = new KeyValue(); for (int i = 0; i < COL_COUNT; i++) {