From ed3ba92c994b178df5b7c4ec943ff3eedb657546 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 22 Mar 2022 14:04:30 +0800 Subject: [PATCH 01/14] RFC:TiKV RawKV MVCC GC Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 44 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 text/0090-tikv-gc.md diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md new file mode 100644 index 00000000..85bdc82a --- /dev/null +++ b/text/0090-tikv-gc.md @@ -0,0 +1,44 @@ +# RFC: TiKV RawKV MVCC GC + + +## Summary +Move TiKV MVCC GC worker from TiDB into a group of independent GC worker node role and implement a new GC process in TiKV for RawKV. + +## Motivation +GC worker is an important component for TiKV that deletes outdated MVCC data so as to not explode the storage. But currently, the GC worker is implemented in TiDB, which makes TiKV not usable without TiDB.And current GC process is just for transaction of TiDB,it's not usable for RawKV. + +## Background +According to the documentation for the current GC worker in a TiDB cluster, the GC process is as follows: + +In TiDB GC worker leader: +1. Regularly calculates a new timestamp called "safe point", and push the safe point to PD. +2. Get the minimal Service safe point among all services from the response of step 2, which is GC safe point . +3. Txn GC process: resolve locks and record delete ranges information. + +In PD leader: +1. Receive update safe point requests from TiDB or other tools (e.g. CDC, BR). +2. Calculate the minimal timestamp = min(all service safe point, now - gc_life_time). + +In every TiKV nodes: +1. Get GC safe point from PD regularly. +2. Deletion will be triggered in CompactionFilter and GcTask thread; + +## New GC worker architecture +In a TiKV cluster without TiDB nodes , there are a few different points as follows: +1. We need to move GC worker into another node role. +2. For [API V2](https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md) .It need gc the earlier version in default cf.But Txn GC worker process will be triggered by WriteCompactionFilter of write cf. +3. RawKV encoded code of RawValue is different with Txn in TiDB. + +So we designed a new GC architecture and process for TiKV cluster. + +## Detailed design +For support TiKV cluster deploy without TiDB nodes. +1. Add a new node role instead of GC worker in TiDB nodes. + The code of new GC worker,It will be added into [tikv/migration](https://github.com/tikv/migration) +2. And for API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. +3. GC conditions in RawGCcompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). + 1. If the newest version is earlier than GC safe point and it's delete marked or expired ttl,those keys and earlier versions of the same userkey will be sent to a gc scheduler thread to gc asynchronous. + +## Reference +https://docs.google.com/document/d/1jA3lK9QbYlwsvn67wGsSuusD1Dzx7ANq_vya384RBIg/edit#heading=h.rr3hcmc7ejb8 +https://docs.pingcap.com/tidb/stable/garbage-collection-overview From 04c39e33363cb7d520a19ead9f09ad718f54cd8e Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 22 Mar 2022 14:17:46 +0800 Subject: [PATCH 02/14] RFC:TiKV RawKV MVCC GC Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index 85bdc82a..ed58e3b0 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -26,7 +26,7 @@ In every TiKV nodes: ## New GC worker architecture In a TiKV cluster without TiDB nodes , there are a few different points as follows: 1. We need to move GC worker into another node role. -2. For [API V2](https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md) .It need gc the earlier version in default cf.But Txn GC worker process will be triggered by WriteCompactionFilter of write cf. +2. For [API V2](https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md) .It need gc the earlier version in default cf. But Txn GC worker process will be triggered by WriteCompactionFilter of write cf. 3. RawKV encoded code of RawValue is different with Txn in TiDB. So we designed a new GC architecture and process for TiKV cluster. From 66a5cb853ca939bac706b5db7b91fa9455e8424d Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 29 Mar 2022 16:40:39 +0800 Subject: [PATCH 03/14] RFC:TiKV RawKV MVCC GC. add design details and flow chart Signed-off-by: ystaticy --- media/tikv-rawkv-gc-compactionfilter.png | Bin 0 -> 138244 bytes text/0090-tikv-gc.md | 54 ++++++++++++++++++++--- 2 files changed, 48 insertions(+), 6 deletions(-) create mode 100644 media/tikv-rawkv-gc-compactionfilter.png diff --git a/media/tikv-rawkv-gc-compactionfilter.png b/media/tikv-rawkv-gc-compactionfilter.png new file mode 100644 index 0000000000000000000000000000000000000000..c194dac3bc7827a18a6a0f4fa679d789b6a574f6 GIT binary patch literal 138244 zcmafbcUTi!_ccUN1ihkC>|!qnl2BuT5PA&|2oQ==0_hIIz{p0sNkAxUz&dlty&n|1NljByFMhhiZNQ#JvEHp9Jw-FH$ zHxdzsvLbcB{R=J zGR2ZF2$&j#?Z%oO#0re11ow&zqDk_1_tY@aFtk!b`?4Jf)@Zb@fIy}5$rQ2%g{0yp z@WeRqG|UW4?CfdQ1b?(28*Qp)|ZS@V16a1NAID8=6MqO3U!p6i6 zPj^@G_jL4PGb}w+SsXX`qrlg|6n+sc9c(pJ@Ekod+mx+~HS&ee%zO;Yjco(1EKQ7U z4Fxn~BU38|7zI!Cp{oj6ek5z5RiK)mmmkdmycF-Q&eVWQ+Q4HP()gaLhN|$!ONFCB z@#itn0((t=J({SW{cPF4c=b@G!Bp zu`n_8b0m6MI_i@(*+ilsknU*irm0S`aa8xSP_^_naEFQ0)Aw`_nBnr1<*~HH{6xpET%reRYO0 z&5AGIvkA?}Q=e!{Gw`E(5Hw5-En&&%`%-Pmo(A52MwS+I4^@(i zA(}^|+gSS<63HYLb2kgNo}-VMjV(4lB&i%N?Tu{%nZ$ zLTgd<^)RYJ4^;yL8)FKF2NA*0oJg|b3TV~>b8mt^%MW8}3U13KdUL57zLr)NbR$)o zfu%Q*Mbc#nw5;I-HagJ3+nA_k;--S}5(upfDE1_CJu{*Y5-n}fbW4bkcw=KNGde*} z=!tPxQ#H0XaU^gBwphHEs;M1=jWuP{y!_1gEH_7M28nG#F^1r62$r^T8z^5gRCctWf$Lz7Ri_OW3aS!0++ z>gsrwr3J^(4#GTM%Zi7!HVgFeqto1&`V?=rv09+26^v=jgljt*x$}{*Zfb*8btL*| z33UTBRV=MYdWJq`b}TD*x`nQ~J^Ve8ZDJ#|_Ov(G^x%b}FWy*)j(7_dO|*chrlCfq_+SKR zTe5*4#zMu?THr49-)V(}UTV|@=Y3zo12i@-GC za|9kNM=Z^RsHKMyqW!&%C^mj-^gwHluRhP6Z=vtSVmf*;bxl329Lzl(RNeLMxsC#N zOB<%A0S|46wzV{@MMkWxQ4Q+jyeh{_2Y~Adv z_1FwMfu65rbfq+5vWCjGFH8~^?uC1|&3D4WumhJ6nWX?3_TRK>B%LsErm zpq({=3vc)oj=3J+o1mv^rba{SSrZvnWIHb-OJq#6ZVJ9@xav*7_xSRU;)BH&kQ@$!o-`ZHylW8wB^;0*rb#Pa) z;nH|MMsOL93EG}+NTKL)RrKxgDyE)pSPO4UZ!{&qOOM4fQWpjov+<5*`uZw<$aMPX zS};850fwGtFo&Mr7?P#2i6M<=XGu`;azm@|Rf#Gpni@h&fv<``lEzj1G}RcYeu370 znj}~nhJM}#n#eqo?K~Z=d|N|5H=2*PpMkG|57EZL-oV3ytwlkr8(K3N zx;&wk71dA8-zw19-&l`=$8)$^R;rM4w7hhMWJ@f@mu=(b&tTiJ?Hx=sNgNLk0oL9) zz>dUrQ)OrgiF9=$h9jWb+j?91Pze-*fUd$-H8eC=v-9@h(yh2Oe~P6anQF?Xm}~f} z2nl#CH3PK0yPcT>fp1~T0WZU2`9`W#i$HgZV*s*L1GEg`)zIBkfQBIMr=rRt`q^RC z)wvvZ0}6@bE70Qdjr5or0u0q1>&3RxRYm(+FbOtr5=A4x%Z6a766j&7Zf)+aLiaE= zB>D>oMl?4+4#Q9#>Q_9;QlM^XU_iHY_u_f;@D#MSm6?gTo|Ubpn!wzR>;Op|?MUJ4 znhOH`&A3_)_NpwjeV~;sSJ%fD%27iL8+C>%iH`T63AieJte%&eF51F`LDvvsNapUk z?tHSYu?3e%_pmhb!}_TB!b-xMI}o5w!D|LE;MG^nf#d)KF}a3nzP5f2T!tD`Q{Bhi zP|Mehinli-3u%zF4E^~GvMt};jYZbiB>DIVRJ8~;M1sAaW&qaGjDaL9st4QN$I_Hz zA=LEbcyk#hstk@d-_Mb+Pp1e~d=ZDR^5t0g*kLr?@MvpBpkp9AQ0Pq)*kegHDj2Mn zkf~~IMu&&OtGjVk;BZwNcZ{mQ5?sj80Yi2ps;JV`%mc`ldOj9f=4d1-8yXvVu+7js zv?akofP{WN7wv9o?L)$wVhEn5bPf2-(4OLpb<{Mp5TY&k9Fo2z3!`VxRWb17JNQG^ zHgxy56w=-G{A>a!Y^H`M+Q&>&plN1_wt-?6aRB(|gD3k@nMiyvGFJ1Y1dtt=rnY9j zEIgOz;HS=F5e?YBW^4^QRRc+(e(K&xxd;E?3`2Vpb6fZ&geqSj3VnF{r`G>n_{0CF zTPan89cw~6L_}~RCi=Rz{-S+t^I9uMe*O9qU6S|oT64-%%k@!eQJC{fnr}v4ym2E# z&DQIL{x<973u$rd&qrC{y$nZ-PlZ;GWmJ#oC@ZIP(|J{bs&~Ai)z^92@AeK{e;(!9 zH4;)ryz*11sv#zg6A=}YHqI_D4W391{WUo{-kY%{KUIzP>XK^`@pf)Dt+ud4o70^6 z*fd(XU;DOMbd^n_IW5U1A+sZ7qOzwh>0NtomTRH?#OalqxTK6v z*XTeSBVg#G;KMEBzaRXyR-1|SeSfTUwCSG5#cjmRwb&BNYkO$jwYP{=t=Rei^?u^%JYkRyYj1&_I@Ag%DZ!CtFWrb^{<}1Q^mPOmYZtU zF5PT?S5Aq%(YjmaqzFnvA?)`H?wi!DZ)0l*#Ec&P!q(~`<_zB`XXS%7ySO9$;uS2f z<=i^GzH31PTwMe46~X%?uRT7nt?5pAO!`p!#DMIfTJQdmS2-<;ai97}+Q$<1Ur#4? zR!OyY7q>HNgIsP#PhvY=ZdOaO1Fs7#uM3=GwH3d01vj7vv(jr5OXL;mf(EmMRl5@v z(luOCqR%Mi_Xe3i@mNzEylbF8xI^($szsN3X>sp?vx*XFfx%}3M7=l}y!IgY+mk)F zzZM4;GlIw3M@pXzyY`P(1&I1~FoyTYd~?a6N2NL3Tfa*p^tn=A({OjK%k}Sbz)gPp z3i9%H3(iL^#MSxqZzGfkpd^=Uzms60OXyfoF{J!a1QjB)Op|>jhgSQK+zyBR^(ppa z9?xVB&#;@P#H~}O*ssi{20d@N|J&y`={^*nCCe;!d7EnQi?d514RjK8e)1SCM3CC<2>KPEMGH~IOvJa${_k1H!sl}B(2;>D*ZmGhn-`R__1s4wygS}Q_T2d3`6 zbC2jkj+dUoOH_Mt208NN!dcI?6P|0*t2O-h`MB&b(yqpaY#K4IYmpdgTj#>t88!)~ z3MmQ7Gi>p|Fszf>w8wN;1agw}f^hs{N%5QH9{uw9o|Q56{byJ2J|gk2!@w~cac~0a zWxn{-l?mmsLp2jG)`a6E`(Ec&+l@V08!DDb?!hAqbLyTIm|_xVj#S>fwc)ql-Ivx1 z9Hy!yCs>MIDWCi~(H8P^eB{_nBjHeR!e)`vPE!MX7W4g3@u7rr7FRAVbERy&C#8R| z?AKrZZMMh$GZaosLVDH0qqmLM{yr&UbNSuzxEkV@GL*Ho^+mLY;p;5#MQi>zK62hU z5z&a7^KtLwrtWi(sCZ)bRqf!G-H+CUhs}q0>M_au@jk^EInbl^6rdyZ)Yc-K@6*-vse-=fj|Go<`%l(b%F5xLV@C{FG&Uo`2A?D}&o9}jEw}}Nd%NwO&P=0sytR$%|D6?uu<~CpRUfv9PT5UIVq-mhM?UI|G{=LD45w)$UBuE$?e|hIV?cTPcJlY3eRt~*BGuk!qnM=kgGYY?5 z*b#hDljB!FkAsFdExv9e!|&6yMkC+?#IMT3PCBS#}*cV%P_mQJu=7YAhR z7RHZmiC4YXyGS^g zdr`@$<)70lU0g~VhX*?#6ZLBb{Tyjz{rL7pd4KY_S=9CquG<@>Zny^Z=Ope@B&AYy zCcoX}pc5C=gT*?Oi~2Ca5O6qU#o9bh;W~|2(K&OQRIB%R2wS;F)j91yR$bt2|N8Fe zg5{W|nwv%zytA4Ad9p$n?gs>+e>zU>o?2g8g#8~XqPkWOJqefJcc4c5*AHIMub*RS z7O@%)S>6Qst#x1KFWsD9{$}@Jv4_V$r?~fZ>yyIXetz??EeFXqUX?MIkNjM;dRN`m zm3srTw_VyDI(eYBL~^_K=khs;Zaoz*>wZMF6$-KvEn;Ib)zq&iN2IkV*Ijci@NISx zwoxQ~w7)*xkiI}%v7~orgQd8}#ArVwugx#^HK(nxzOCzLd|X;fM#!GYZ}0p%%S9(f z8uyjdvO}x)jyZgJ``Bf+z{m(y9D4(}J2YUGrisoRKO3A;iIZRGw9 zdbsxzKRT8a{aw2+@Wk(c6V@RfSrQE;Kv{GO3Kni}M1)0*8oEU0ed}GfgHlJdN1Hqv z3V7c38Kc(^d_BJR%cG?OS((dZ%_}b|#Rd;mYL0%4<&ghgZ@_HHrsanG3>2Q8T&6}n zr_OF|DoG?3J_kozar&n02E%fMc8Yah;N5 z<}F!e=O)8b%if@3CC|{EN)Hsk13VLFk9<$Mea%;wQZm*iv#dceXtZy%g^^LPz^cCY zwI<~*UEU&8pMWcsfdNa*sKPl}iEsmw*n#;)x+l9ot!+4E$^?OnMSI5R& ztL;U3m3u4xuf)4bl(K8wYxgxCX7O5?r?%&KuX9;72~^jYkm@g^F0c5fpT%fM% zUJveyGyi-2Vd1ypk|Rk$u}9u1o=bN70Z}bCc%sBvE9N=hMBQk?0fM9%%Brk0!g z`yhAUl-K^+65so?#xkS0X*7AClm#tiUs-8yb$dyW#m>rKSbE)^CiZFV_ISN{nx3u? zht>vuTGtr8!RSo><1@vjE-yA0#{Fu4^5#{W`;q@c`hufAb$vQYf|_;B|-xh;X;-W+w=Z+rPN`LL^E zD#I>-RoZ+qaB$*#nfpy46t&!;1o~f!0qzci+cJP}bs9F#nyd+r7kod*{c>xH!S>eQ24jZ*GURD~y= z->1Isj$E8gknF1Z2Sj=b>I%25?x?IH>C&|%jz%JyeSKN3uvP0 zkeXZ88raPJHyDbb7H@(Xxv@s`ssVEM#*(5G)$Pab6ewMI{&D51dU_x884(xmEq)NP z&)Y<2<7*{x_5V$$gczZ-b(6!v#=q!>QxD%PX`5{OY~e#o%hfM^`k;?1j30WI)RsN+ z07X7@;{W#REre@(yp6}PTl9txW-go<>6O2ccGHn7`}yPkdOnId zEQcbjbzRzI)e&5rxCXOH`qTELMN?%Ia);q}m%#C)vlps>w_=|xqM*p?hdV^|~M@ZU;zDbM8??Zp9}8(s#@Fx--L=ic^}IjtY) zB29g|p1i*@MxB()bS!#zW;e4UavRe5Obpi&Hnbe7nrEm0u96!x#-8Z99=gNm%&PFD zNb-zDD4`l&=hm>c@ZfKY;c((GfJf-BpLu`_REZ~PPxPspK}bKn4E+G^t*gbir*2zq zFZ7mO{UshyhZl5WgO4tkmHjYKpHo;hMlNaOmhjVhzMP>A2AN;?2{^betwo$ zWigSR;T;#PQLjmRwXq0{*08wpKPHUeuG@NJIi>fU*ae%$ zdC9g(<~y&BjmCsRX<9gtbtWmKzc#_ln{46Q{1_HKf3U=Z{M4)q6gMxN!> zT?NsDf-k`t>n0@qwyPv6#G+`Da|b%gI>4K|BRH$_E>PhlVW&QzraE9EjzRAJF8wR54(!$ zu5JWuBrbl?TU=(OS#!wx$#C=TkHHP?QszA+gWiMr6{b&f4}AAcu}wM;{bUm*X7jC6 z#)U>YIv8)g-!XFf<(I_g48|`l8_9ozQrMkS1eT;1?Wx-9@2C|a85VkdDWtT(N%2)! zqpv+zTbRIRdH=vy{rOAO4qlGqX?>)z+L{%Np}%f6Hm z*<>1dcW=<>#@wv~zmxI5W4~2`quV`O|FmV3OI{WylMnxdF6xW@-;q;m>Q5+o>EQ80DXzfLK8;L)s((mW6M6K{fxp9!vrSm%r%lK6|JdQYsRC505bbEY1HO9N z*y$F`2q@6mq@u)(JQCiBO^3^kIblcY6jhSnjLb>~VG!{o&deoW8J`XT3S>=p%62?d|UVD32IP)`(-~oy2uHNlbdJ1%$*r7r6nRE z`r%B7i>7b&%9gm!J0Dlt&E2+gMyv-&h6-JExZG{+>F9S%?lWM@lnR;PQ<}4%Jq}Ql zxu!9J)6WD@5#VA#GajhxzXnGb* zkX$pLyKTkvy}#WI|7Tk|&j7T0Q#+rnAlQC%T4vu7`6j@7Wr2d@8tVNjU@l z@TPU|_^f(+-F7%kf9-eumX?1qk~A^wMa`x42d2uBDSr#c zA)wMBj9{v02rD-bhhTD`X6TCkABX(?l;`o_(>8Z(Hc!voBa_ZY$CnNiLR<8xJ5YH> zTLh>5*oO0|yB-ty$An*&e2%yYsqaizA3y#v$M3fG^nZ(vX`@obfa%m&S=cs1n&I28E)X6ZAp|M>nF zlsQV%vd|ieq@dev`yGVbJzU& z^Y7SV94Yd!XX)c7mD7GMy*32H_um_g`d(aCO&nxEE!bBz_Niur z_G4eASu5zAf+)sl>7#!vriVFa6^@|rZ-&UHo(b1&7B?aSB{0%cpM~Xp+?ehEKcVpL z4&29C@oL>&pEr2>7m69hzLs1_R)XsQ9dN6-+S5+*XXR{y^K72kDF5+^?JwhW=ftWr zF0x+dR(|cO=2WZ)ge6Vz>E(UHGpm;O24DF%p#Gj2@RN^sbkw)~1{}eVZ#RC1YNn`n ze6+uJ^x)iuz7uaA9(~ds^1tr24ms^Vr+KSn6^9djT^d~U^Ly8IQn5d`x1+3V@6Zd? zFUm(({#m5I?G711dX>eURb$hhg}Wi9cA%7a+p>m2)!jU+K*lMYgG{*SmWJTZvDvuZ zhP0aM$uw_T{bf~{T6XwCnE?fN!5K8fMO+(yE! zW)#=6{6Yc?6RJ1x_e+H-k6mxJf3&ap?|Qeu8Qg{lYTX8i0-kHc7wZ0YLxoH+4OWXM zeb@*$XUa6kc7)HF_m=|zrKtb{UR1PKp7;>GudigR zJqy~#y9FVt3X~*;Hb&mY^q6oz+s8>0Qm%`SJN6;4hC7Ew9qm z84YUU=JOUT+dtMOye_-bwH1h*73;S1m(G0J2nZKym&WU+$O_yciBnLKzPe@N0ok?S z^je+vTxPSoU9#=*f#4G|+xh@Z&4oINd~HuaUxi{u_Q%TD*8^=uH#6QWoV6%AIgnN+ zgsDizC!LZuzzZfsnxpC7F-AaA^%TjQnj=RTsenY^PnPnCugMw z(}#C51kg&--Tn86(o5>XAvL!E)Wm|G;i7hMU{QbWYv%%}r%UV7uKZ8gvJot6Y3_Vz zdd12w)#86TCm3jLR@*TN=u58K!z0bW#q||}zcjJ9yX;d)N}BStvXGzV%aZ^y&APTX zsPM%rE!lE|>CiGAVRuY5^uA{v{`nj-hnoDSV~aPWLVGq)cxLtP7U&KEA!o{$?2O+6 z$UXzYfet_*0i22= z1Tq}|pHs^{TnhQ1bYG#VZi9vKeU4GOudWv*8X3<$o(8+m~VD|d1+r!pPP#50=3phT% zsN_+!Y2(=rU>gudqPHa{B>@QR>|?e-a3h`Zx@OlEf2xn)R=vM?zeBK3Ym)4gCC)8* zNgq19cl1)_b)603gA1N(Ct5sG?Uebpf~ubLYM4S?wFhQPL3%copCv?H)rNfCk& z{=~aAGph)R4G^gxesX&yJN4;X#k&A^Gg`8f+I*iY<$71IDe6g1G1uz`7HgSYxOISS z8o55r6R4q}-OwM?q9y971BN^Nozs@nj^-0xQt=EFlp-9M?cyn}ZD#Xm({r6EpUmnY+!6J3j zIWXybJU=)Fq6@+*0sugRMlTBrux22T(tv#ADu$)ZvVQaYbJ&=^HOm)HmExUmQDlMZ z$wl=^#3z#S%FK#$Fn!q&lB%-Zi{!Sr@7~u59EISdT3bNZ)x`KwOwr_c@6`{0PFI(O z&mu<8^&uH*!pL~u)b+mUAQ)RyobEwqD-QH%7uV^{t-WD};KWY=bCImyBqn1#P@fVF z{NDpWH9nx8OGeVu8-P+PiXV&K8c{oC;;9&T?v&Uv()CkoCBxCIn7dtJ)n4TcWDNh$}c+RuurPo>Ub zSA0EtZ0;2ADP2_l_VLLy0Cavciz3(LqZ9haiqtOK9ok2=e- zk1yHOml(^=grFIdxO4x^7A^ce(y#{T=#YQogwpXSi0Rq$)*g5%-s9KT^adpyYY)nZ zwT~2!{xPBVH-=)GXH^U6SDU*9!aOh1`SB7#$sQhAm{oh5Fp-C!)d2nvhknjqv{G^W zmDx_m0^bj<4@_e=5cP`XmB^lR<}SDk9=@@3iD`aZ80lVP&q$VcbuN_GuYBJ>bwl{Z z!O$OHo=CMn)ScyrEGS^&e;%N&p2~|Eu!N3>UEPbjC3?*IMFn!0_f`fFz4BDSN|j} zm{p{QcS0mzn68sE70skm%Av|IA3D1Io8Mzu`%51Sf$b?w+gH|3&SUo+`wq`hZ=` zT)o@R0rGL1YJctRvq5WH8?1NFtW3X2p2PN}+@Ho9BR0kzJ`khHVUP4S3RB`3#=+2R zvVhDIGdwtKzJ$zznl}})-YCdUnzE|3drw@}2`xp)!uHT#!S8~)4^{ZCoiY3*RQElh zMANDMU7-p^&Nq_!7O7}oRq2?tZYZzS_oUOO+m2aYT;-!udIVhc*TeFzcagLak3Zb^ zspe)w9%o;Q_01#O9A>kH_mQ^Yhc|9Y*t6btKa^q0TgRasFJwMIlvF@|_Z{DI?Jjf; z26s%q37m%h2AxM`)9^Y_s^v*SKMrDc2epwn?n+<+$XEq zEn-T=Q5r0AtcqKN)Oqknb-dW8Kkx_Qit$ME#lI%5^t<_z@CXMXo#SCi+17L4ZIf&q zVac&zrTX=zbd#AV+N)dlAf`1pq&c;%%0i&`ZB+1o7bm#$|IMC10G_O?j+2=} z+8}u%MR4%MDCxH4>S(2=9ELrTg7<=|AW>wwlcyBbR9xw4LXzA-*{=}I5`2BKU4QAs z0581uX+!+j4By^Zf;*A*i*2%Rq59Nvk`6$v4fz%?<(pu6wME=Wz0vdg*AE*OMXY(^ zxE~rCgx!dh>wVR5o3J*T1+)B7EBgN783QUv3&FPBytk&)_0Pt*@A|K^X&z;h<0**P zNGir9HO`&-nD&=s4Hw2Y$IYmJqiCjO+)=2IAlrZzNV0fTg#?K z1i-pGa}v@hrR>MolM9!rl=DxOZ`E1(MfMJG2(dwYXchfHlTw7ELDkm(np1X~W9>YP z7?NMCz#Cq#b!8fNOGin`?YIl_l-Q5Upo=P4xMok2c+T5EoyK+2vE`Rlz0HYgd`RVI zR$<*?QggsS_o1ecf337?W`7l~1q*qF(N<+Wk?^O|#?i`G+x*|1?&6cK8=YFltwL)n zbpGbV5b+aq!r71mnjRdM>c4qbd%a4JsKL`oSZyW^#Dxi<0?Cmo?iKz`%57}T0k6q| zzG4o3CArQ*cFr@{)gFX>fCz(~}m#wYfq2!Vw3?c)g%PUiTH{ zr)jySk<~lQeoRh2-&uCbzO#NTuyP+3Amy^WZ3VJO@#s_%n6ob?ZT~l=yjFLh53*9y z3o8&Fs4Oxl9$zY+Tr+WAtSi&-V&ejA;a!FUv!x4OF$eZ|%ITdf~1|y(Ab?U0R+7Aoe#Ps%p zF+^(+nKoYb>t}U)$OP+olw$PWkvigl!+DbBn;EWp=rXQHrFT&qvR;=c#l1)csI7S^ zVb7b0zg)w~4o3IS+V2alUl>0C?Szx=ctO`S-u-(ZG)0<#pzo!P=ro6ct3#u^KCT{X z-$&0Vn74D-=I)k{+?^utDJ8D-LGe)_NRob)UN0E~`7LJQSiRX`Np)#Z$5=%m=tv=% z1wSi6;SgEf&W^G6A&G`bq`d*2@wNC(L?M+z&Y|OCGp99H{ZR9%Q<8hS20_T<*L6oy+7n-ZwOv>{@@vwp7p(6Q9H zIKzEdJR-!8(Ll5$$8Ilf?t%Ue${L0w+-J)Tg_=ZVw)FX&fsF8Y4$5 zyR>c}B(EA0&&zN^`BpccDumDbDQ z9w=ZZf-drNh?p6P@XfmN2=g~zR&|S`p4s4Z@#2d|Y^)v#2PfKhgAz=efxH1u4U?zL zw`_dm*9LtORq}xnIs2*j4>f(#$!o*Cjn$O-+dIF$P_1q{zsc-M-VqzcbA=#4l3(^E zB4oRssGhh;NpwU+dk`nXLvj*y!(-flZdaD8)vMPj3kFN_8gnEs^<2p{>la+_J8&?$ zRJ2}db#y|{=ZHe&ly@NXE%AQ4cj9Efz;Y;{XH7@fHr1UD$t7d14_}YCif#}`<%|VC z-h)W7H8*O5b}?}*@bQ5RObH8;DErJ4h)o5ZZ$U4&meu33^jOJ}XFA_H59Tr&l^f4M z%Uy^o{fPbfW2m}qRb+b>DX6$tc4PnRybMI2rQ!BKOjE2iuS8PqL$?W>%s5a=gmBUgVAB;>(j=PiTqOO^8MOMOY zLiU_(=Mb3hbJ_!OtDH|SS8|HqoMD8gz>>~aNF4=t<(^!;ZEH$_tKe#;e^fuxnHBE8 zjXJbqSB63YDs9CHL_H&Kz5{e!KPM*!+2adx_Ju}6yDh_5F8Os5K;cnX1BIh^EX8lj z-Wt{Tc?D{{%7L+yM5}gl{L4U8-M0|sSI^QJ7}yoOTJZb`uM}Ol9}7? z+V zJo@>omp_UI3ZF+hCMzVDDp++|h1y46O}hT<$M+#8C`*`Q18vIZw1flBW5Nvm<4b@R zB?-?)97C(Rcr<*NaICi9sbqYMX=}lY|0?Tj=ar0VtYDV>9DUtZ01>NTfkRZhLTdDz zEhQnN_&kHGD#Vp`OtS z{@LXR{vu7>k^iqS3%8Qd`t|D?{$5n}d{MFUSIgt|jylJxJKM#goFq*Hb_S0|*g^DE zxH!n|7~8Jb7(j7luD#AnkJk>R=J#*8e!v~tD8JFVP*0oqX4D`@#GOch~PNz0P++MqBz`YnPnu&lkGHRR`&0 zt-zKF4b8RQMVHeY3i=GtYIqp3TD)fMPGjHBcuj0OD;?+gJ?BS9ztI zfG`Xchm;Oe4r1g+#_-E;xpuyun25ur9LXbG#yoaO8AC*KdY_}xeu7NaIa~SaaOp9t z&vwEC@d0yPx#3b#BjWSwLC3Q0pa6Y;#I8zl>2P5`Mm_7;TYR<|f4v6@RB@>vVq)W%u@TcjDi=#;GhT zX|&>2mLy@d)sNo6%#ErHNb2-@kqdgL?`j}N>cSZ3d@VSV@gz4uG@xUs!aU6;;l#?2 z^mmd}x}+qDnyJIa|;@7*{I&MPr0eW)`_AC17uqQnGm+ zXngZ`?kF}$RO2K!vKJ;bzC6G=+3ZERy8UTdfVuj{ef^DcPRmzZOV>zYRcst-8=kP# z>rrpjz7!i#RS{D!BZ;Z`kueKf_2Gv9GSe*lrhWxMEZ!=0iW@~)U%aA`WAEtSo@hXc zbV-!$%Ly1#q;E|>^3s+2ShhW(Zvze|sv#~dYyJYntj`tds>3ps58k7-%&W{xDu1K5 zXz%;XX!UAuW^Cpz^_{n)>$_cUdsKulTl4QpdD8E1V7caI3ELizBK_Qu|K!)_JC!{g z?|W9tChw9}s4tEtKQi8@@u>w^-B=6;inT!3dy)h6rzIF`b)#g>>WoD5mwO>0N`^rg ztx@YrJ~F82K|HCNq;0V}E>NviC)dL3=fs$Bj1iiIk&(8W7&)T}ULVIl{ z=Z{>Ne54$A_+;tNQK#rlNpG6Sp2j@ZT>4Oi4>`%1=MBq1>e#kp881FwiWQc8d{=ki zI$CuVh!p$f29lsZwy8Q6FP)kI3{O?$^2$wTxNC?)cT%FyID z!xfqJkuu3f&A0AVY9#iUFA4m#S5N@ndJHzpR$*&l$H0T#+pvuCnB1YtRa>MWVWBLd9akK%mLDZJepXMh7(r~9LfEosF{ z)Eniv_-@GT!jxNz9@dvH#N589^j(U2X%c#&epJUwO?L7A{aC9KTR%F;OqsAVBfSFc z@lVsiMcT{m^!&?NLgoy=yA*KproA@14&kU0TT>KSE!oNwJJiVfiqghpQKe6+AkKn* z^A&Jm1Of4q&lMCbxL1{07@&^IUi63 zITJCr#zvsp$N-zWgsCmN8jJQgLc5m$2_puTgTQl8sA5iOFI#kQ*9>0=keK@KvuGth zwtSn_BA{kpAt`jO6{Vb}fACj2&!6bei;YNx*#4RDCUVJQ6fhrIn=N9RpcwUzd2S~~ z2NOICJ|iKrYf9Yl8xX>Y!0pc!xx8Kan11-XCufW7l5gI)vHsq~@N&^D=U^|+O6>#r zG=m8sJqqg^WH;7F&tpnF+8jqSSra^LU4(4#fxO7x`pf%zu-ChPPG`YRfyaI;pZs_8 zAm0!LjDBg@KS~k`J~UBi&3_Dm8ZWx^lA^Q(FMP=+(;(6?Ez1qW&V(PO4cgFF)K0lA?f6Sdo#%v0Q@A*jCa5z<2v`j@Hq;#0MLH{ zANVE4F8%JMxo8j^p9)WZ4>n9ed-(jbZh7gi$)FVHg3NAjy^}gH0RSU41Gu~cq(u@l z2gfY|;=*FR!wCb0SK>0OETXo{N?{Df7omRY5OAS8p?~p!T_PGv!^8wrSr9)K^mml~ zDrR7%CsB;X^c>LkJ#NY4a5UTa7h-RZg0d3o!JqX%nTN_m_RbKdDC8qdbT6S_;!qmG zU^x3kzNony{D{BfZxs11LIfdt{JgLJ(8)8Oo?CZ(N7U;JC(EIW^*|aJP%$HL^bUZ- z5qa9!P~3MzS^Zg(wD4Pidc_R?Qx}HQm%Oqk;8o>?C-jlU-@kS9_lRZVYGI#?dCF@= zhl_iI{G9IY*WF{<`Re_~ID?*c|91-&0mZ+n#eI-wm=R!jJ!G7QxUK9qn~hk((cBqZ zd}g5b5*9EOvcqlwDdf*bcFt7RPn6GDlyBg#Ru?$fUcxQ9oO1qfIqdpO1%g3?_t89P zwAs6Vs9Glvf#87J1gdFP4Xp4PY&3^sd0ImMP&tTjoOPb9I(_Vk#g?BN&ndg?inhD+ z*#9rk#jgn5m*Kki-QeHX!w%Y>I`UkT`m+35V9&Oo4^j5!dOff`$72FE2r)l*-e(&J zpNlcfnbBJ77emQ$?AKxuk?>oKt$frBPRPLb7bL8g{)mEorgxD}9yW9KLm%loL_n~# z7eT8-+FsxDIEpY#eh(LFIsh)425?@(JrAQGP*}9>$%@);Cs}4$yI}?BZo`FC$pU;p2#MlmFEG>?m5R)JU#vkMLsw1Sbksj z30cJ!O0{I!;HJ9a7x@Y{CbMXx3K;aJJSQUg15{E$hX#~PW_f*o_ z(&|gs#0_!Fn^v4D0b)C*8(AL! z`LG@xTg(orvA$_ofHi#QtRO#Q^22cFrC_c2x^KrRV8mrMLj3jA4hp=gh5{NP6X~(j zJTnf*AWQ*(?P{Emj$27lHt)872HFK0{CZ~7T{pIN4Zz!l0fw*x!w&*l2ck*fDxK(^ z*$gR!A&zj&!cIc}e3;z1b+{EKge#?UP2<2!pi+?gYHdgZs`>PH<2ebTA$x(5I2ubB z1_S58R)-?20SJ4%D*EbWBw*xc&G7u7vJRrs`lsFV0mT8D=$Ag_l zjLi$y?sSpN44VuY>1(cy)D46tzv#QMdRYR=z+39nZ(!3t!PIj62g?F{eCX$#z`b`gq?~z((a|tv*r#u)NG0zF~SrUxAAPSIY*u z#iL3_5-y3P0~@0g`de}ox#B!wDzZ^>w84qv*!?P@4fUG^TYiIly3{F;t-)#EgTODT zW4u|KZ@d}g5bEAVGd_u zKeng@kMqY8svGx(_RBeKCuiSrfs}?Il&=c~y{Vwid6Cnn!#16vnNv_ersje9CJFsS zaPX%G28MTOp~ms}hs9C4$=`Qg4;sJ%kP~ZhMGLL9Q1T`@TFJ?4;Yy8cXo6lH`D=C0 z$HgU}2{%8fg7Dc&r=q|LDm`eIDOu1Ir&oQT3DM$R#2PeA6v<(NmHEqXIru@Qtm zp8ouct~00^Albx>HVogdWr>|9<8*642(fsT1)W-9q#zfLAdHoVv7qE|D)3DJl=}HV zVXrbk*}NS>pQfc+etnk^fOlbhHGp_ZE`@c$%|wVw(Yo8KW7&nzimv<-hy1?aMh}3V zpM85%!xzIlDgZhE0=uGO(Xm{ELt&-2sIU=^Mj26dDQrTLtii9VTZj)I!VVyr#dE)O z-%3Q)TuCD{P4v2f!aTzrG=dS_@?u>o(!~Ul&|Ar>~6J zoehl0DD1Up0!F9rNDXAR5e;a|5!F>I*TS)U7P|Uo)48`DJC0|)soxX`4PCs#%UdQ( zCwfvc`VhPiGT0&f1?n7>#E&g;eNQ2>Isz+WAvK@&cl=6r{<`(Sh<1vw=gT1{XByD^ zq-5J|(W{f=M-W8@a34muq^{18gDTjFJzaG=a*Kt|Vu`z97(;cnDA4HO32_D|yiF^pkKArTAB0LO+B8(+f#b2!0M++uI|NMg|6l*$ys>tkQPyG!foNLa5167A_iL;|ZL$}I z|5}<4n>X8}gVNX;sT`$-Tjfa+iO+(eF^u#9!L^OoVP1~8ULHZslh8HP*dHv{@+Ak`L?~rqnr8brV_;zuF z(O)1XyNoD`TmwGHBHDQ4Q5#kriU`ec9_(yx%oZU6E06}ak?Z@xvxbgpz zU<5+hX${n!z%@zTRUkI$xbIq~3r%VET?nR47MgAE10lH$o{K%p2f6|&MSkyAY2%3equ% zQl38~M85a};ph;xD)Q|M5F_3nX`7c)5Ah)v+8Ht=eRJL^`kREVYe7|rOiS)kU{BI* zq{6;WUo$f6dhi_!s}_Elr#~cZoQ|qkRe4!;%l(@7<03Ro)IABAtybj-Lkv}E`16V% zup2N9>i7oOBmNo)-W|eCm2aTlcY-2|2|FZ}ttOyA%6~JtLR-hD6F0VdD~+CLJHvF+ z6390HH(xh)&qJ--49NPrQ5^LzREV6F)z_nIL+&E`rvr>(H!EH83Ve}BYh>l+$k)Jh z_CxV5Sf09~Cu9E|Og$opK-x9fH?@f}5;>*#oTk{_kf3mC&1M%}g-9`(RrL|(FTTi| z5)S7obcTjQM|SB*3*%cM<{v~(3e1floEX}eXb@)`0{I`28T!3_G#?>F5yTr|5fmZw zmZjYKBmewerAYALA>1azy4RrYFHzvBIee`w{nqR{%86jz&ngmi5 z(R4;{8;5LA_}y@+&E~P07ZE{S6S;3qh=36H%?@|jv;BF7IB^+#c|&nn9`*mx_1^JR zzi<3Frz6M8u_f6n*)p?7wkTxJ>=8-!II-mSk`Kvq$&o$%XsXNXs8^ObP^E5y zTnd4-Q!=?o-_`Ju-W!N4U$?)BZ%qcHjO?EfM8SK$oLy8Qw` zs-TWK;1tes-~{e!-M$P~4UG7Ll3gwvixn&QxER_xz!{TDnx-nw zLq+Iw$xtGuc# zh<8HNfoNegW?R`B4%a?Ul*s>LDEtYBMyh-4G0=hJun&02+hhXevew@ILgq*1O! z4L{I+^(aHj1b(|Zt)$ni3WslYyLC@*w)Dz5GooRab&Tk{MyB zcj~X71VL6QJT7BrL?;zXu_B`O>af;zwkcre*=ToWC04@3(XDOK;eG~voQ$UzYvp6q0lUddgrpyBlVe1m-w$r^OueYtE>YG2 zF?nct17xr0o~Xk~h_ozb^cX9|;Ulo6DbllP-mh zA7YOIO+j0PgbwJY=WFQBJl)dU>IX;;}+DDb~Rt`oTS9VoTH4L`abB-y2-0smR#7DPBs>hoEJ z8Gj^2z{X*P+8&mE|09L^U+TDB z569@@KAy82)Ulp$dJyqCLi0u3s|H;Ld+$j2mg@Ne$r6lPZ2Z?gcpbej*sv1rKHVrw zov8y-E|wS+?D=QT`~4t`drGQx5!1(_aK;XynN~`V8XVF~kE14r?XC;)K9Z21CmcVCw^% z{OCXG4PmI&iUlYQg(;r|PYG6{9uSbwUa_i*XhQ$pnTEUp9B&3b|Iex{8N@u*rGQ{O zyr`iQJhPS^yqSS8G zizQ0c5VcrQfOEZS0SSH1nz&GI{s}=@TUm2|i=g2H?(r)u{IliUF}`mp;>zQXaTIJuEE<$pq=FkzA4gYA4GY8GAhx- zEs0A9v9p3US;I%WhW|$E|7k;QSaSIKpbtNOu)ifgdIxBb^*Q!PQKo*2O=F-9w)x;w z-(?T5AAEisny;rmJb~8RCJgFKk=paVF0m>eFo%NxrxxNBFE9)zGw!zD2jW@b97Hfu zkGZ1%5SUvnk@*hPa4`y#h`oa~LS*co{ce;tTC$E*8Ndtbe9j_oUQA$;tY0FfDRxmF z9_$E&U{pa!MF9q_q@xhp%b>7H7JI{-3S_x-j~!1v!Rl(1HO~j(E>1ptXxtBj_=eE9w;KN z#y{g2Ebn(iw^~|5Bgn&Y4vOY+#F}!Q5vj{fYn*JlNdGg!0cr_I?48biJ|~V+#E#WC zCy~VP-nk6GHWdi}=T}w`&N9q#Oo3z*{-&hau(;z+&G_>|(rutAnRl4(m?|&RVEeM= zL*Fwf>%cHh5kJ_=d-1OC)5tUgIH2+nHX2MP$AO7;35esq0PJnV2U99Xp2wqpiQ(L>ik=8?!x z0i@jdk*B=nq8ezQ?81uJ#%1}+{#J?nhTpgdz@a)Z3|lN%6fy>dE*n(!WAM%RAl5wb zUAAgLt+HxSm31GEnMB`mfy&1|p|`q$ubZK8hnqqqiQkFn#}oi37}+$@)PZ z-+t&hT*#@agYL?=Dn;P!FfD3+f2MP^pbIm)Ea#3uC6hHMTo-*UvN-GiI4Ar{9kz%l z>_n?0TFF>J@eMdm4ng2NM91=GFBeRZH15ZjxL7yJSQ z=xSs(d&aQb{H=Ymxa-K5EW>K${EiwF?gM)TbY~Txh{MmJ{X3hhKY^+z>1%%e z%nYyR<`)yQI?r6#?WbL!o#hef|L|j(b&(}9RV@HuVNMi2@(AkdaVS{bf$g^mF!kBh zp@(Lp(7dp%oep_+XI;qYM!qdtNn7D)76K%Z5zb?6+)GGF^=174C2VPA95ZNZ53-$KBjI>iLGFEtPmo=%(O4xHmxe;^elruGQ{+I*%Vbc*TfhL;C{_Y|pl z-uZ9VIR=f`cHCsf8&)e1Gdedaog zpRdD&@R^{DfilcLEPkZKp3NCLSV;a$-llLF5kzF&qjlHWL@{i_Fh`rt>Aar7+t`kH zx>{uHgqR9;sSn-U0sljeAB|D2gw#C4)q`(-ApH}cYEWf)e0+{7gsv{>X!+nH4CR|; zUi)h~#55}DIGkm4Sb%WXF&87LWx~B#O#aGbldmr9?W?FoX^BK(ZEDjeNJwN!EpQt@ZR>69c zk>FH8Vo20;odf>aCN>u1t`N2|4aeN5E%bJ4V>lvwu)2wngQ-yZ{W*=bgsn3 zqemVN$Di1B9ti{zw1orhx-Qo%J9Y6$ z7cc(ax7#p#s#B;(82~EvQD)qz&Q-Mz0guItgUk$>LFgL4bdlln_tj@;{m;{0gnIZZ z45+;o<%28puj@aFCow29?hjrq5EMPN5mx20K@eT2+Mi~f`5;B%QwN#Uu^=4mEK@|S z2!KQKfpAxp2c%O(sj%5bE{~9TCSU)MqFeoK^a>LyZ~O&DvUNi3tv&V%|8<8g`R2^? z=Q46lvsBt1Y{8|oCUGJ=H43AklXe5JbObg$f)s%(%uW?)$!%&4 z+z}6&t3Mh2ClCSzB7$ps@!akUm0*R-Fz=RXMtINo_sEO|`jJrb^s~~djmhA3+bdO) z^PutOV4fXu_$D-!d!^v)>IW`lIJ&*M%~`Ibb0SyaGsq=>j^h1qnyq9G>TqnR!YM&N z6k=Ym9`!kt^42k%Q;4CVGZ_Ycnqv%wrDb*mnqmak zL^Q?ZAEtd$JL6mx+<@=C4LI7|b1v>=c?D$aXL!$0d0m%fz@N~0m%R7Es|7239c3Ya zJ86-nhRZ1_4+JF}{0lSkhnn6cr~2Xl?BVsU0TUDOshcO>gU*P)o%J#TDlCmChS_ZZZO>TZ zeIG@wxHQ)79`yOVj0!`457da~b(XUOI@PDGBoR7TfR7^WlZyM8y09~JPM<+D-9GVs zA9nne_|yKKW9(QSd_ix6L8mOB2|)<9MYzK3bl(G&4@OKtRzU+p9Hx+M8;MK1u8*G6 z7A@%iF;il}UIF3xwSA0}LIpuG{x97rh~Kp{E@YlGzyT@!`DjhtI+Wwur426WCie-? zh%LSY$y5F{;Y!)lz4cFRJBVvt0^LTpnJ(x_H@@Y4x(~|T#*T=NK>NF&9M6t`4mkHZ zD9-BZ$x`%7Se{EbUNpXvuG?;64VJV>4K+A>31-c0xgaquOb} zyohfKk-{y^@X>-%TC?LH*7G`$!Xx zn1=VmLi0$I@4x1eICaFLPvh*J#xo>-`1%j~O0LHqbiVTs)6lnf-!6%XL(>Syrwb8M zA9839i}Sxhuh&x^M>LpOUAK_solq236cb0ha9xU(-iU&_+T-V z{N_Oq`7Z@bfzE1{{EFjUAEST`gLUX3ypqLR!>V&wwig`iEy|yt*Lmdk;g%W$M+Ds+ zsXSr_c~uv$Kqh;I z?A}TnZoL5tsg(Xt#GeU9vj8xST3zkNRoI3xkz0)b5F(QWeec}glf9GwY)56Y_kdT znHrCgI~^LGp>XS29f=C}+2)p9-0b zqeCe44r!h0yYNJK-k7D8Cq)o{?=@Ta76dec*E-^F+X?!`M$HGKoe$vb3&gNzvZwQ= z5aNgA1Mshme`_4EZW}pNdudu|sjT|xBO9WEc`v?`k5ktBPCuGH=2gxc8-|Jb=cVwN z;>~fr$H!NB!R0vzlSg{2jdIrNU^mSy{9f?pTc+5QnQ3EYwkVtV^Pf+_AS2Y;!H+K% zPloi>2UdHqaamX3HLkz9G}3!Wp<|QvI>LnY63)6h?mSbFW=RYcr)6{NI?yXNY%UuI z2!N8;!#}-yZ>xtjnsIhx{?Q}%opiedA~^cKV-PKZ)R~O#eL4G#Xj3Wtgi+o03B($D zu`Q2Gt9G2}NrhiKZ(7&|V|B_HOxj_UATCxG#-NoJn_H^1MCe=elAbw_9wZtMvI>EZ*WWux#C^A2>H_lbCsNClYGq$s!NiNzc;Y!34q{P11wqW;L zf7TLyyi_?vzRu8ZHkP|p_w?5fO%EHU9veHL>)yH7&!>jmZQV`NC%#{!7ck{BDmKQi zk>O+2OY>)sBAXTWg`$Gcm6M`~2gT*b<#^BE!B+X1@ayldp=X_D4j3xdgzJf)Rd0Z> zFsp#DB1h#+Wh{|NtSk#RtpTU74TmtkNTtnCZ0yxoqJmfxBCflqeRcb!78x%+BWird zO{MG|jMR0MREkvIBsVL9(=4uPxV`kTZJVSzc6*7Bhx_i;>ENw)rhz1w-N*^&ia|)$ z4Dj}N$M=?!N+PM{8sVsMXMh}+?QX!hrUjV}7n)nNM@ZRL`6ngSLn^A!Eak~*l{Xi2 zBWX2zc25as$MwD7v!kMlr=qG(g2G5t*wye-VDypWx8nt#@(3SfX+Tv;waJ+=e0MSb z(u%mgm@q~Ek|;^{#o~fgO(P4R;ftWZiQP%_VWPdyR+-xsOVn){wPWl3jw_0bc8{yl zrqVJ5o>CT0(EH)dF}+vqXoE|&M6Izz?qXvOIT?o25#XuO^?K@t`1s4jq~{H*pBP-r zZlx0h+6o>oxu^daU#Qgj;NfMRxC4vL0r)@#K9<>~H*W2oM6t2_@72K)HXnW)lPrtB z&YF>4U}f7VfbH$5wo-n%+9x;f&#(nwJEOgkLj-= zD8REv8aPt2k(6-TUe&=FU zuS@cuF!L+eB%j15E3M@=C|KiYzUfEB&xmXull%C>l{2vG|Z3 z`b|WK@=UNt^83;3yJ!_5S54-q#xGUfyEvOM>3y-I(mE`xQ6bglRwCUdTO^uzhpQ7v zxo(dzRM_A1Um)sa3M?ZR?4;w?PbNyZh;l0Ti*gm^1K{7^=mEqpatDh~*??hXBqHw4 zETb=Wo#pa8O!Z;y0%26)otRH;v8!U52cGz0k5r>FMHug`9d+pO^xMlCNu_513EpDh zG3N&+VGB{aN!c%#E1zeHlGtA9r11mkyR$GVwB!(u=5_E!qO9u)ycT|CZmee{LdU3;bHf^&u8 z)0ww13V0Wo>qPuP)Vw$D`-(Gd+lOqFc$?>jhZuE_#$x<`*7#I*SbP71r~whb3Lf)q zG(ju#1C!GM5&nDY88k-m+vM(~hw};t7cv|&PhjceRH*H$!@nz>*=A{HJ`r+wHB#e> z3tBpyE`#6!Nyx^qnjRffN$;%4m%6zG0TH%_kyJOyrXR04T=;Pz)^UEcRI?&^Ge9i) zRJ!8pOq;qdtW}rZ%Ra*vu^?YIS51X4DnH}BT$G>C_3*_gZ-@*9Q?IEw_w{pxw)}bS z?nEBjMYh~@`Sy_+g*)k3Oe6y(0Dms_wD5W(%`TCdFmdl-3!q{rlYK39mi5H$q2E9 zyshx0+r$(mZM3%uBYK*r$A&-7hfj%VQi@LteTWDMZm57RL1l& zZqHbdm5YsSTrV{746`U(jm^BXob;3?#SBGdQqgCaJzDjo-WkXCTQ`1};W2PS5C}77 zGva1+OmtYMj1O`Fi@>`D;_bl09t&wh1O_Swd4qQ@z1ed+&)@9TA%d`aa~sns8Dq5pPjY@bETSdbWA3aVXTBQ@nH| z3nJ|4g2wXEiE_J6wO21$-WY#oN@V@@thJ)0JsV`%`DoTSqUn6J0~>{U z^yVE^XIt`?R{*4JpR~6tPSXav&-x}bB&)Uh@^quwp8}Mj3W~{m^Koz-#YutpLZv$b zPi|0iTT#qLT)2Pe&`<1_=$v!n)bcH79kpSwQaZDhBA))(b<*6W**il~lrMLS=rk`hLpH-SRq$W`Ni$&m?j@@(YcUX*L4Yt zIPu~m|3kBY6nN-RgZ+BXZW&SH;#iw2Jc1n0#A{2xSW)J4rSH07=#ZL36exG)w#^4) zR7ZplR=K5;wxnBvHAJd1=W(<0H^+f+>H0@)SEMRs;zi_BBAq6UMw4ys)Ep`um0HZI z9&sb>b>wMKy+&?aex`fWRCGdf?}kXvu_ddh<_A=bB9$y1*(ZoxZFUnj?4e?#j!))(a}DC^~fYhK}>Xjj7rxsB?oD z&ybc;OF2oqNn0U&QcG9hG}8qE9Jz zJV^1(#-(`J^*QdHo_kFndM_obZXc0T<4cHc7jt=ejFYsz0XM;(145&a3gyIEOM&A}(BPt6f0Z$IbM>-w)wl^8$~ZE^ z3|)J3p0SfSwV0q*Q=3skyMbR{i?^16VOV{&Gb`YP2%pl(3LTxwnh0jjiAn%5?%WM<($xl8DJrf+lvpm%Y=D zwt~)|r^sEs`x2cCY(-x`VqUT@+vmS&S#AY|pUyofZ1&1bY5#*ROE_=3Q%MJk=u_er zuq^3VIiAYRR$B~nnlZ7*<6$EvQlh3b73W0StG&ggV?Ur}HT}>g-XgK2GFgjap9M~bz)>rsuo z#7;hvZ7g=FvV*oh%w?OraeFp?icZLX0AT?89y~-B&INdQs7G zLw=uQ65b-8k)v^N#vQc$rB%V0vm;P4O13=JFZM@kuV`!cc*{TCuCQ<^F39qRUKW-2 z;H`E0i3U@KCW(;9)m5#k+{IUCj=3jidA1TYx+(8cQ7ud^7n9xHT%sEZH-9}KceeJe z!?ERS%NMO^S0{saGw%@BmUpHJrhYQM0JjZQdE8)Je{o{*=!O_aOJDm=an(I2LShND z*EhI^{4KcDM_&5E%_iKmH_Km0(~i+;>F&{VUJNh(F@EuFIJTbL_*mw0XJ5PV>&@lR zgNdKtY&YV?#5TOLR-g3{TD4V@-EpgWWe^hgl~bh(>;MByxIS{h3y9L|()5uERdk+2 zKWNj&(0zC zER43|;r$uR=R#wxM}T589bH4GnKfM z@FWe(3gs8btbYf+5*#2I7s;;AcGrklk|}Yr?)q_$DQLB5CyQF=Dgt_~o>f)GufX(gYHndW z6?a=)MaIOlg}|GQYm{gAT9FS@@VJ)O7b+{E+s`_Nh}5#Z0*z&mSfluAuiH0t!vR=r z_?PQ9a>dDP= zH>sAzQ6!i|H!~y=-1a^>6y%k>Yuh@0j3P7HfvMKfIqtDAZEe4FFwzpcHD#`Ou{gu! zzfhVwIV>n)JSpKeMg_&CKoEG7jJAv|y~DV<_IQ7i`zTJ2ki?`g2jtaoa}>pri!7yh9u65V2P7hA4P zHgU;G#~LLZoZWLDN$UemvwWxab#jN}M_1}yT5c7NcbH4n2^x_o%{NNwq?2-84!FOQ z{+5qlmhEYMz>1-{#H#Nr!tA=MbaP8O6+WSFyqYA%aD$1e<_YJ^Uvb$+O?W)(4ZblT zPa;DLikn3_^_{PJjYNSh5jM+bCnaPHzCt&yv6jNaeA7f=8l17$4pm&h|7Qr)84JYs2wTPz3ARGNoIIm*=-m8nud zOgZ_!(YJb%@UhN+HZp^j!mN{B?1|7}?0Z~lGoyI&;5CzU=7u-W%q6F8JtR#&E6#B$ zo%6hqR!zxFUIJ|to-JdGC6>A8bw0_d7z&O)(fE?Dts^jeqBP_`LT<>HrM# zJbT4P(bYzzF_vhiw3%})2WBX2wpBrsjh%Vc&st99nQTat1C+GPC+yAa?^%m(yN%Umhhpx6`hXm& zgePhlTQ@>OGY)vSjiWBMX|B(!WHVR@AP#x?0h6$cqVFo`4wkNP)fiD-w5Zt zes-DLM^1mKu$GhhIqs~OWFEvWFPO!Lm%6AC`u66PriHcBxlJ)8jAs+>?il+lojpdo zyk0{SZ@rtD{r=49I)@}bqNrha@Sj<)9x;x}tvB(C!VE36iKRDD+bOZmQ)MV86|NH7V_^dv}(+u>8UD-pETW+c-ln z>Q}GSm@J(-zJAbnWGA~b_IgszuQ5IC#1el+x@fGR(cDw>g(gRyG&=TVwQ%;>kaqD7 zA7jPNtM^B5W!sOv@fbf=D>}a)8OXS>qoiQ^or&6%&84A5mQk;qNEz93Yy|gNyTfViS0@_s}WL- zUTcY7j3A=YoVa;-DC)I-Uq(8HNzIz;w#lV{AFCNI%I}g`m806-Vq>p0-e^{U!@R5M z&k+<|r7gtPrwrwv-iKf1-`nF&We%zze~fO~+a?fhIKv3hHaspg)iS_&>(lk@zVwyV zxM_p&mT2*G7CMR1^h|}c^s!GnE1j+=&8oGolib^^oQMzJw+Cq=D~@rO&Q-6Rb(3r$ zyc5T`^6F_T@ku{VZYsN=eNfJ>iHJ{K8O#dRv%0lqKkKcOjWNBLmq2LGMk1%Cv$8gc zE@u$3R`58RK(VCmywJ#IXN6#5Tcfr9@%KxQ7&WLL zI~BwO2lZ4^n5xQED?ei+n^+tt)zXtOTasE!+c<)#bMsWq=ekhi+jNP&#WEMueWnOi?~j-d78&4-uQy8g*}_~N5@9K!F^8Afyv@bo7Q{Jh?=*!X|DCr#`mcXB7PdSafY&2iOLwv0eWvP@rh@-^zX*f{gaRC}Z@baJQbjz3>EOHZ6w zaFlk)IB_^_Wi^)Eh)=JQSeu(~s-m?m`^2kukLA;}mp-Z7HP5am2sg8%9QBR*d0y>& z`6s9ci)-oqZT4Z*f+j&NE{2v}J1D(&M4AT-(7GR@<^ps;4Tbi!uYn68w=dNg_>?gH zDS2_9Ia$1F0urAGSgN2%7diA^f!X<&$1K_n&?}VNyKQsN%>A+9T_r_|I|X_1>UjEO zp^4j=`7x@C#qm2qDn&LQSH&mX=2}AN&lDwFpwMiyxd_FokIbIaX%KEknrR8Gnpn%^ zxAT*28(Tg0>Wvd8MGTt0FJS^7<>TeysiU-FUsP*y_SxyE8uGVw*@bL+ny9+3&0A00 zBsDcGe13;ig$>G^lBUl)EoBAq@jX66N5dphM7Noo??KF__KjHUp%KROCsLU4EuG%eQFTkBIR{)yA1;K*Wh|JnI}}I9 zBEcnb7mfeT23W6U}VvcB8?iH+|2(8AU?%UY;TzLPTV=Uf&Vz2wH_I|$O z)Q?qvr%~Q?K!U__b)QY2V607mdE`kqfO*lTr;FmnYDebC7$@7>IE83c4|_f?Jd7VT zmjFE8YrHc`_^v0Tj?D|oFQ$GmZ^N94^lTi^JZWG23ZT^ojAw$#?wQnlDUM8vKh+iw z8-8rQYD8(CYK{QPm>y&TW5LO`{xLV7fHT4~8n(O>{b3ZE{CXM|s!qYNMArB{H~sVO z%q`xuSvRH<`M_WkctdRDWeulIte4>xpY4R_^QmSsdD8nb8r!$0przFY^%aLrLtbLY zlhe_HMwX2>$9G{9L;>HMFlfLOTVk!48%|KXcEAw3^Hwgw#bV_LOVCv<#^JYrA5BU% zEkzTb7zJ50VkT0n)BupY6*$*(_(Bz5Dlt9Z1citTXmy_5J6BT-4xn_N(P7JH%9M!& zt`&I>>8sVVxZJ9W*MHc>$Bsf1&N9|R$Y}!-`tx}(9y|ifuuCi-Q&q(U^WPN}&#U9f z*T>`<$54*BKs{pvLlI&i1a76dH8@Q`>rxr6lIp4qbn#B1e4oWx!J6-JiZWmGEXwCk zvrSVS5EGG2DKAAjEq+;Gg}O42>M6w3m$1uPavwVGS=BuDmc^22Okw1KUXZvhTrql* zwTMi)s1nP$TYDQbbS7+l9@tE4FQ*YAyQ9rZS)~(-V*|H7xK3CtspPPtEAG#AkcIOY z<;CP)mjOaJxjEy!`2@H@PQ{t*Cmd5U@UK$GYw9ys^FJXqs4Z$=8S9etr%A0qKmo@^ zk?_1lF&_#5L2CRJj1TGD8J?fe>~Dj1pAvAdEqFS>wcdF>@ifjCF@_8y*H>|-WaE*0 zjXjDnh23_8HEiiRNI1t-uUdNCq@Qhg#dW7{ezmN+Z#;cAxweZ)C=-v450e*Ep>7GY z>sBc6zOI8qOThuRwL7=#Jg>Y7@BJUf@=dV#Oz-tZyNRWWK1cojj2Dq3L(0r3rh{e) zOTqS6RdUTM()S<9{f;0k#DPCPACSehct7HWp%w3_Xn_bvbU(27b`#LtpMW4tXvmgX zeH%j^-VCiTntMPZq0>AXewGb`tkI%=DnOhnIuf#~@f2Ced5t<``mgW&L9j=w~y-h|@ z8>T>e)a=sp3K%v;f+c#Ofh`|DRiFWHkQ+zhcjT?+`HHC%V)!oC?cVVbl1}( zx#Y6nZ#M0<)(Ur%^1lN%!Q^woZww!zh1&?v>9~sHq!!ZKBKv{m&RPRnlDG54cz@zM zK1ER?EcEV2r#qO7>CI3eekAWP&XYaZTepele}aKgwwE8UQ-9_LfdBTDT22{ z!%YMwKjU9eyn_t?eOtmJc}6d_U*2Q1W7TXO70z>l(y{f*pw0zF*$Wp`ay(ECq%1M? z2NRTPOAGh;|4r19tdNt07!BhvH9Yt#G(isF-uiV0+;J~=_X9t6ivK2)|A$)&)`$2s z{hN^ix>?P`f+A4&%98^@)VuD(ZLQ1UJQVFPu{&E^;|`pr6PdZvDmnUS;JMI&;IrHl zC_6uv>8;w3nfOPz6o|y(2CsI8s7_d#8E_4)>p*1P3M~eTR3NPjEjO(1iUmSHq1Pm) zyrPvj<6ATXHwuQAi53z1QG--P=gQUM6iw{o{tW!@6R9)m48Cc=7~tXaaUQ+;dJ&3D zFa6}1O>}e%Z*1=N>2G!U68{sipeY%=#Q;z9$+JbAKD2!@^O5g@eSTs z1u?30xWnA6Z%#Tq?JmQNey`)TU4h>?r~p~sQ(1=bm>LXj)H1O5p++UoJS(IA74Yod zn#lyp^yQP9dyFECO^h9FaOaIk*5$4f0(-U=N~Ug=9YOTLKpLJ}yRX_@dX9pJ1;32+ zVB=A!zup7H+Dq#lWY)=&UplgV(r<+)>Q#YzC~ zuE?W+m->(}8@a3kRjyW%cY*Rx(!%0wk({snl-CR`qJnqev&}s}2scaD>SfxenoM5$ z1gU{)MJG=0X`l4c#qmeS1<)1Pjzk_>&inApIwJEPkC=9MOq1GNIIohEgT`v^ka_Q~ zud#E1I7GSjc3F$vbeB5=K|kWM8{K0^6KC~H3TCBl!`f9@a+r>L{_2;ZcWzjE`H&!E zpJBr`@CIU}C(EDK6i=j2G0JLTYjE#&#W+1wksVY-xAaEVn~iAISUx+@hg{Nv$cZrE*$ zV6aERz1;X4@oED*Ctl-%eZ0a8B1BH`ZTSdp9%|?<^|#P@j5U*5z4`i@%*cIX2Z4r_ zR?e?v;UB~B54FOD>T|StD$ZPzL~PDRDigO+`u50qS3!qM&E$WFL)ben1IJ=gGYBXN zT@MGfZ7?d#dB$r;fad=*su6Op4aAG9T0!W3pc-BHCQE$HtiXT*2gabUt$1LlATa}N z+?(@GG{*yd41dEY(>MrKsceq|RAG<>-VTDbJevh8KpQ>j%7iP4gNci4|^8^vqw0AQCu?bg5;Rkw)Nj2%l_Id9bXVS-M z|EASBj5?z`qu@L(u7u?#hlU@FxGgw3^1ee@#h@iyr@0%oj(&^Mdy1q5usD#2e+5!N z3#{yH40}TsB$cS2ThsC&9+6`*FE)vEb)+b|j54;gIT|u-cp*~aogcf}9LTW9fbsm? z%x=-hrO*hqEz2n5S3y{7$3T_(W!nV_{;lADcl?4YFN*DrHfsC)H%Nv*j;XSM7%o^; zYv|j*VU$fz9&q2rdcP551NJZ)!n|grD<6~kd~dCrFgyW!3dg#IDFBg*lhDb{=dc;u zfKK$;h}Aexp#H<{8R&?TFzioQU+>~U-Cz>}V_@chP{PhNA>ObSk_tk1ZK95%l$1d9*s4f)oEj)JzRcm-(;GZ7NCIedF0Wr8hABxHVN%AW z#+|ilR}-{KP7E5mi{Kfg^;CDZ$#_^_?Klm5muD5QD?1GhMLU%C`_-7qAoJS$!HH>; zS?`%^B~_e8XlUV1;8al(6)nw+;#Yl-u&cl*C8z27{JnhO$ng(T5C6+x(c=VEVDt)S zB%zu^kB$jSjO>IS_#yl(TNe6u-H3sys}sHakMbgULUt;!2xOjo;4D0TUqn>_&7c6u=f>Zj2Cb>>=9xKwPq4W>$MFx5AwkkiaR zJCBH6_#GXyyv}TuY724hj=`-rCM7AO&ZQ$p^f>5L$}wIYv)Nzs0m(=2rn{w$-_Y?t zAOy#Tg-DJr;ys5}VTnLYa)YC6#Ns586rg8`t{`^&_!GH_hL1z2A7QHqN+ZD%v?k|_ zj<;6*ye`^@K~j176r4*-7WzpjH>j~1DR9b7@8RK?y;gDRaTE7Iu=zMBe+c#OyT)+| z1*>l9-i4`?lQ0Sl(`?gWMT1_t#EU7>#6t@p1207FHNc&)f_ac-FO-Wk?yYHAMXPHn zuvvy!+hh)ZZy{u^8$Q`~`GTo4C*2t$wk^Pt&mMn-fO8beUQwca6WK7A3&WJ#u>TvD zx5fUGzNp|!yS4IKp-YnS6v2=+M!XdH!tc-{#&$Q5@kAt8|DBkG`yQBPLO?n=6G5em zL0_1#zD73b$UR{u9Sp`eGtk~V;@u;x_7O3kZxk^P0hdnvmU(a+IRetm+hV3#%sj(_ zC?D+m4&E*DJ9mF(;movvBUSXbs#dr|9iOU|M`+MtW=@=-Oh+HBP zn;}pIu(0PnuYPV2nXv6ym4%fChIY|C2E`I4`2O9Ia6cUkhX3DW5}Mk#~G4 zj*r$4sjXt@-Sj%Q^cy?p+{S!7k$XF#aF-2rlr0R?4`w-tJ)J|5p-UVlMj;o&VvcYz zXFy@^(-a0LvcT-->|j#v()J3FN)8>Oco2W9;#@%=97Jijc;SRO&=>?J;VOb^4~OJ+ zK1#Ir)cG;s08UAq=q~`QvsM8;)@X$9dSz*ZzXu@A%=E#&Z}rcl%d+)grDnd;uIbbC zbCU%8fXES~q?nBPUWca#OTc4=YB0AqevT4OSr#_vhI5WAL(U{54gOm8!p%5^46M^& z;SCZt`vG&Y^YBj!XotH#BNco-EdLRp5aW(g*4Uomr&+}FnORmlgRWn^@H(kU2u_egK4z~%@YucL?f3I z8d-v1#p_lI>l4!HY`|TuaF3rkw!?yut@Jjm=sbl2V~3{s7I})V+c5j@llPJrVvmWe zh)WtKSqcZakbe)bdDR}z`draMJUsL(U&e%4qQFadf~kls^uF;;fvT5m9wu=IZxDSK zqh~X|oEs3MF8F45Trl^FBy_BYkE0BrlVq6j_hTb{u*ew~HSG8uuh``Gch$q8O%1KH zhmagjidA3)@u5}lw(H@{d@i8jUW~zOTN14QKzVQ-uAT`_Is$HTkB+LOGyx)MOEvQ- z4&ys2?Et$nISwy1zit5v1S?I$g3tdpQ6NJrs$5v5p!Vk zdo&pzHeV|hh*hA#?16HAP1qP+4#BqT#y|gcG9hdHPuiMCW<5JhryrZ6MRcq|v9YOM zy-ju=f!#P~iss7(g69iWH6~PZ)`;FGI6`wIz~tOcw5crhKe!y%1ED-KLpe7pM!}vU z9q^M(>vu4hG%`7U{t|5WI3>JpeP}i*mMW>{lwnvBbU{?OCS_`*rgP&r9~ZaA1zWNq zG22uSmBAS%+4ZIltOOmO+;B;U#kz{ImWu1BM>dxal>(#A?RK?n0|}xH%ikDL1v;4= z38`^4pDA#>>|rlg;dDx{-2Ss_)>6D)S4@*(-FUMV8ZM;+fZ%Pa{+#>onsdQv>}|Kv zV^_&p!>o9&1A#mek%zBIKVX7pBXw3-)G!@mF)=l+iYsiR2tLQ3qAocc(CIlnqvLFP zk?6eTX1E=xjyUj<0Fp^k;eSb_rmw{-*i*&O-cOm*+R**;fmul413S#{6p?XqHuoa_ ze&+eQsYs|{SgLs=gIdikg!W?ZN%cVZO$G$O-h33-R;C-La1Ham+B`L%OsPQo-{VYx z9A`YNp~?;gy;-HiufcrgkIcw1OXHE+pN5Ly2TKYACD@`Kd4!!fBD zN_aY2i!1#|uWs1sJ)8$7#J|_GN4PTrtmY!1LY6WIg}B(7Q?3mw+MlCUoDJo5V1?}{P*$1C}?16cs2M-RL~-4tf7)62W6*^v2#I6 zE4b6%bDe^`!-QNPjljGmUCEh)L$&&>z_hx*md0YK(5aWt_K|k{(;kKo2P=DTl693D3S;A=MC@Eu<6aC zk-3Ku+~a8|Mc`wFR>4CxG~a`s5?cAsZc;#ND!~cX5Ta|meVN`xdzJ0{YKUmW3I1jtKDFr0q=)zOK(#Tj!Jl#r$ayIE+C1m&VIC*!0}d+t1t6c!x)PH!eUr=EC1$E`O!0 z_3vrxK!ziALGkCLXK7BvS!q)^xR~`>F|XTJMmN)^4gL^0fK5=BVzwWUid@W1=Yx%P zImygg;l0>*-ktpb-uEcI!(G~Ro^`0*Nnv2N9=uiyO34#tP;I>%sk?E4HWHKiCEGGl z?q!Cv@pN<@1f%`@ke)20?uw4_dXbgsUh7MN`$D z_aq_r-mLnITn@qtbzVO@=WYjm*kC$P-=(>7FJa$Q2;65JKVuSvy{zGNe6TKIJY_iY6&g;;IkWHIo;obrFM37r$jS`Hd7eeVr4m+;3lcNr=$4puD zh@2`pg=&4ua$XV(ab z=`h+YsH+h+SI8H*v-2guI&#U^ethZYJj@r2JLYKoJDp{8`Mk{sY4dm1SL&fXXWqka zaA25#Nj{D9<3G6-QuGs^NG~^b*)`MZw&uUmBS=_O>3u432gTHbeWi@RCoYPz&pKi; zcm&frjCc>_sj{ag8An(Kd@8^x=pb~uI^bAMw$dmjz1%b=MQvE`C+won3$iiq&o1b9AX4h@Y~?=kuIwdPmUU z6h|>)>iST~=}pdMr}A+g0QoAY$+XqWYj9Q6Nn3ti)Bl~`eKm#De0x7G>-yrME2tgi zyZ-c@3_@Heq2u7IJOj}5d(nn53Uo%eVIwnsxF3t@TVS%y3J2dcFRPVysavyP4Y(peH;XEU4DwS%dI_|c@vK<4bpZ*FI$<0zoHkNT^ z%_Ml|RSif1S<-m(3{y}ziR163(hUguS0B5NLU8u47jGsc;DF%y4~yY854<2r9bU3>7ouc6$Uy z3fkl=uwSB&I%WyK7s1s)Nf99z{Qoia)lpS$-QKX>=%zOz-O`;B(%lVGl2RfqN|$tl zw1iTEAdQ4{ch>=wE|C@qiElmU-uHg*KWCgVMA@;PHRt@*yb1xYgqO?26Z$i3 zCghE8(Rn&}FR)zgPxMNi09+&fpXvDDvnE0SJ{mgIZmkx{7=Hf$^F%Q}M!lrqc)UuN z3=R7aoXDVYdhi`e*H0$ZbY$M)#L2HvBMw+7L*S-eslK0{hcH0^c!Og)!RyQ4JB`b@GZnXz4LffIE?I9~ONR2jKH>%Z8&)%q{?Mm{?^K{ofZL`=Jre@BgI{u2L)E zp9Yo6I>+h(Mhk#+j?IXI%U zKpAirp!TE7HqVY&x;B*R;xWU6z@6}91?R1X@4y5|Ypw*C*5Bwju>6uJk1T~2X03)I z6_SVtJrOes<^UP^tv>fBAVLn>Slv z23bB`h^`HSGyC)OxTsojrqlp@;WkLmm7oW{&SeT-15yj1iR}VaefQM6ah2pZ1MDNH z^m(Ay)hvVq|9X=PK%~FnNs@IyUg2mEl;Y!ExAng`00ae?Lp&muwJWBPMd(Afhgq*C zmBa%;KO72rs2|&2@1grTK@A2^jNlT2{IapWppTUS5oRzvuQ@~8?3a&CzWndI2*Sq$ zP_bBZ5y+dUl8=QXTn%{{9BGD4?m0E6IV;s(I1hXBNv2g+@zd|1r? z|KC#oBQWaYDhji&StWW*YH%=@-i6#5S>m+?=BAsk7c2miL5v>0q(IUw=!VCbf-!#{;mr+I>I9y3cLk z6cn!QKel3hk5bkAw+#Dy8H%L;V5+MrUFTe2_bqeiq^=*P*k2LU_p!7MrKfrLP7b*) zH*f^_m}m$ecQpx(CI2Z%v*#PSoLZO_VANR3y?Q@b!_xO5^Uo!C9k{x@e&$BY+$gXbLcxa;*+Mz7akjy~(g;Tt0ly~Wr5 z0nh*n8LmHn+51@m9-F9kFojS1FO8E}80(T+!HgLDzw zyCd@frhtqG3bXMfJan%Pe~6vakWw?R-?Q=Exd!@zPO#$B9k=F~Hr4*BBh?E&2j%Nf zWqC2na}P;J!6>D1(K|vtd{=mYKYgp4a5gc0GT@4EuIDTw7*J3?oMuUhgk7 z24)f!PFkyF@G^kAyDAJxH1vpiTML-B{CvFq*x!0oMmUyn;PxedEv)bHD%y2q|5RVN zM=Iw1$^eK(Oh}p`eGXuClGl^kaBnif)BWBHejD({-Vqvyt ze`+CDJR!~ZxIQaKUMqV4*yQGpNYD>w>pDk>(^sf9nOg3wD9yxu3?QQXAo-|ZYY@gh z@#o^>&2Sg_BW~6O z$HH5AH$_Cj2rj(c5NDsIv8{GE$;FZVFWHi6Uv?>QtXk^=u^K%7uT4g z?rE7s?nx2J-8-2+kMy+4X&6rSKv1fl66{T}aPrAGv?J9a{F8YG4^@G!|Q=8|ESoVF5x>l{!x7RVC%P!48g#lWlvy zMcdXFQVsjUm-TO*#miMHASSL0D9O@>hFS~KnZ(9KwAqhVuw1jN@uCe>6iPo9dJ?bP8|c>NJa)ge*CYo6cT3q;y2~^|M)Bd<|J%{ zE*5Di$?#jwuUHjEz@*M@*HIa48%iK?qO<=MuTADpEO4xo+AV@ z&^LoBv>JF12Z8k|KEIF-)wDGmOj)bBch|P_fWf{&+tN37+~m?GA^q@Q=z1s(jddQ9 zP%jL@ABeKXoT?S7j1|w6QYvJ`jKit`SC&GVV*vXL4S7%F z3opQc??8R2Kx z$EUMIlx9Vm{La59%4^*_b^WzzW%n70PLE@mtvHzhkM*-F0tMNsxJ2!V%16D95yxcd zyir{0`6`UFtC{F;Q$h9aZPc2Dcv8Q!uwBr@(eOc-=9XqLZHhY0RSJi$ME<5-Gaid> zBY4-c<}Y9nis)T9OPDO_o4{iCkyZ8Izw-kSE#qwgccAm+YB1XdA%Y=5x|FZAZ|v}; z&ZM#tc+t8b`2lb&l0yB|_*mHH9@!*w@}S9mxXZv9K;^>k;fjt2u!T~E>fcAOHbsJL z4(ZzDbedv-4?4y(Dwa*4#CLXv=9?&$70{V6JE5_D=a3b@ z+#pBP8>)Ibq9-4y zSPJ)G(M{U4`>6Pt0x$h>5Fpl{$q0UmEin`(sh$PlRW>Ofvq6VhEwcikC@c_Di>1O* zr<)G0(=JdRx&UfR1rkVg6Xe~VwSv{noly6>33}Ws&Q7{sAQWoEYk+i?!I+#R5G{EG zx`L2LaYA(F1|8wF+sz0FKpp&I%<eWEZhY6)I`Z;2ztd1|uiw;^fs?n$mr>qf zC=3gi{AFd3am%CE?!M?EeRX(mXhGZGRU0+&8a!!qkRXx1l2spmf({XD+`f65EK&K* z-*rJh%ghPF&DRB$fkWSO=U*l*00x1(xUL}Qq35P)-~? zIbxbU19r;rZx}2l*#CB-AK}00CpEPPnP4OI5JgEeefgSDfpqO7G=Q4GmTK{tCBf@M zDtL&oQJJN%!+IFOT?WyiP={OE>Gle&kz=SqEL((A!n*X)Cx0Jx;eHGK2?0p@Z+Z^C z4H5?xAxWORetB>ntnmX}*Afu)pvI#7H(3}JX13aiPE3DcBMxfyV6(!DRzxy2n5#G6 z3{S7|GPEb8OB?0LMQiykK*$MU8p@g-Z9rR(qoY^q=y3V~9rSb=7419M}$a8Nym^FjDe1 zi@~gcrzcFODTwOdR!#Bcy$0>qmG$WM`_Q@Glv;se(Ig_^1Rbyg;0Qeq_X?}4e`Sfd z0~hf6K5g*`z@GPr%D@UDr`@6we>?;tABsuDZwOe~1XQvt4A6r$xE0Gz-}cv7)xfD& zmIRLqI!!4CULcB03kFLZzF{K0>om`f3LKzKsrhm}d!eT;%&xT;S$b~~{AnntvLsh3 z35B5K%u2xJI_te>CxHm}XKEQF@CTiy+dNROb+VS0;R7%uyRPW7n4UDsIgb{}jF@i< z#c#$}r$9=93-V<#AteOCpIJ`u#?&A^9m{lWJc2*y-#GFhSZ1Zemv8-oW?u|ywE-s< zxO@n<&*>&t>+RejXe8I&)j+>a^B!LLL1H68rW1@|c;cYa9)bJK=ZGZF!B%V^9B`@+ zyVE)vXAWLKHEF`^2WFU>AFt{vIuS^_$PkE{nwCQ$rUh~eZy{ji`#G3*23`yrM6R_< z>ScjH4J1DXndozMjT3tS$C~J`1)o6%3f^(2mf^mFL5BOUSpEG++Y1V*-Kp!$_Whj?(*=X$g-q~vgxYcDD0 zeVNpJG?|iMS@TS|KZs-rZt6+53DrwS&CSQ?*A0DO`fQXv2nw!zOFkm9SGEf7CajAN z8D?ucZ{bgkTbV<0rtC9byB6W9#x3k=%^g;)}(7-O# zw+g4hZD=ahqWx5s<{MPM&gYA*AIhV+)&fEXZBGx-tQE!Pih2o%X{8mQ7Cv3g_plic zvmwxwEV6l|i%O@7gf$nmXX2|2L9?0(fR(9svk`+S$q_iWAW*T%2c>M5t7#`M&?Ty( z%`;w4=|<1YC$)ezox>9u9YA~!zD7N`ih6ZV-Iyld2wx7i0J|N+Uh_HtY`6lTnlB3bz@8Jm1 zs7m;4iu0~etpMyFo7mLzBW5S)mNNMwWQ;I7^sOUM;6V9O#0|)*a1e`&h(Pd1wBspI zOnmSi1ozvxFD%99)0FRHRw+1!GRmDaR8%_#WAw3QJ@x;+pMxN!$_j-UbOW04m$iEO z>QzGr439bDar1|EOTxQ6z-Ov}bSQJF@2E3+0YtOzT%p##0I5fc12#FLyaO*xj2--p z1(2#5+T|^1Zoxe-B^SUbKiJIl!k-1*d{sSa%&)M_*dr3SmqE2~R4}=5Dt9Kw2AJ}>Y+<;g(1*@vvAatZRnd9g0Jb4>(HurHjO$6$jrHFL9%wJB)cqx^g`Y1 z=d-F|6>j!A2+5M>$@hN@N)iAb$AaT-S=#vIIGi|9ebe6YV-z;hmpa@59@-~-6SuJ` z?wj`_FE}D!vEIxWg~FkSvt8}=M{Pnp0N_Lw58?$~p(C?o!N(yEdNppq0IcbKNCG#K z8~mtl;HC@6&Gt}JBUa*^C!nG1ME4X+Ciek086glLZxa;)BQh4WEYTFOp#B2+TFdVh zQHUQF*rCsNTh^B2YvnB}uNS`~ipuDQ(Uq zWzjz8T9q50ON?Tn1aF0t@%?RoL;s59I=_Ur5x?QV|^!nAS$;NNN&-Ke?T~ zW_Zmo(&%4A)&zoghMpEi1JbuOEU)e3(xA;S5Hs+t*gwUyg8^vS5mqbzt1~+1V)7H- zHg9{=?1LV(plgM6bvbA`Pq)hOGJfE3B2L6TljVfkN{POk~LUkJS~du%QIj9527E9Z^uQC*5o=UR-CBlmL}^vZ6;v>YT%8AE zPG7_YK-9ZH|9?aE9LPYwP$A@g?Selb`T?3Am(@_>CY4}7s4mG(g0Vq*u8Pi$Yda5Y z27Kz=db>S%$(-7{e@s>TTcA`PuUH&>Ev1L)rF|T-=8*iB?X9#P*AT$}bU{s)2=cUI zb*=SRANj6adj;a4@*8{X0w8i7uD<|&7Zlt_7=}&0b;}EKZrI`*OlQoa#=T6{&-WMS zZoXQ6?>T|O3-yl-QsvJT24 zt*K`d&Nt0wQvM4x4XYA&ml{G~A#95^06O2yp@A&(0Zo;e`B54fvNF ztl>v+?L-?#D}&J-AoyC@59l=MU=isP3pGf+-qqju3Ybzl))A85;d$o7Z&{|Zb&NA* zP>v08V@GI<{Yn#2{IS4J#BRxiG$=S8TP!vtJoM?kYHjABUm^xSKF{zsqBc<;fdYj8GWnLs%{9m0_QT=#J8CRliVnPIunF6(?j*DXGO4%D=YF$A2x;K>P_-|!cX?oRxMjQfw-5aWou3ZkAcfNJfXP#sMWkm@y=-+;Tdkzp4w-Xyaif*s0efyUZe^C4Xi75x6s zLJOZXsNbLmO`uUP1>T(I-n~-Ux$->5-vW09whmYY!%BX@s1C-Vl38Mh9%!G6{^=C! z9mpR>K!ZYUwe^c{2U2_0ezmhsM{$Gm%rHO?TNnT@5?Nh5Z#1~!&4D}#+&xD1PWzqYi94jD zfcLiz2f&gZd7SF`n>gVu1gPG?oX0(1QDPMR!Dr9 zmso2HplfUZ_96gxf(t-+vj(8I?F28%#QC^kUs4tWacAtRjsQ+uz^v&&Sg9M zje5;GIl@WEy~rqfH^Pu+%`8c;f~n&Yb}CAr8Z(rmEQ}0!64bZ77OBU5J(fkkA$&T@(BI zPi0IG)2aQZY$Aw0cLAVQD44&5AxdWzn9*KG>wt(NKT-L{U_Ne{C(v-M3_bhyLK=EQ z0BK(#cDN;ktij815_ynk=PnD9U>^YkQ3yzK2m$=mRJhft{vjBvu^t zp#o7~N=m>>Sb(FC1s-l+^Z7jOA!K_eUle<&ofM&*wXhK)lT`-(lM(qvbj`W^AZuZg z0tOdg*$}7}cARLXOJIO4{pA3XLJOc5e(kyfx;`9eErmkCnD&#I)GKuE;2psDuDEc6F`J*RK=A^69(q?zdsArT?uX>tKcmZ9LK|ES2X zwFUo=4dM!4K(o4u>gF3Is*tHNh8F(O1w1Vu)(pUU5Q#Khgmy|`IG6uf4hWzgY|lkH zcSrCJ%28RkM>3=$Kz#cWJ!}h55ic&jJkO|qQ*h0D`G^#F3;yaiu0^peJ$Tq!oNN?^ zBG?uBTi{_RS!6zkyQrSS8D7Nf!pz+1eGc&_dDQ>?a#0s)Q0;T%@`VLaX7O5Y{&fr1X(HA=Lt@%vBi$UA9T-GDuh+Qk*9 ziuOD;>w%~5z8Jl1|H3r|6tU*i66wJ>R#^bPeNh@Gy0{bHp~`E>r{kARXG#~Clzt+O zVAIQq1TtJ>T#<7qSm%3`>Yf~Hm`oq#8;JiW}F1Uf$w zNFfVulJqw_;~nAj`~_dCR8K4;-NA3WpzdnBJ*#MsO$Ub*>LXye(hQOLp>!o~vL7r! z3;8%kN5`F~OneA8UTr_^4?$2b{skBoe2aeqUl$39_MjgnS7$!S#QaIJ#C!OWOgsr6 zM703*M<`$mmUGbn8D~(RJY-V})>Yw0bmi9wmHC6u69I^~@_ahV zv0hzh^kMKbmOEx`i*_B++7UcrP>RTJM(GUaU1J`iXII5jNHeq{4T$S>%lR96h0jG} zDSb3jzp!aAuHl=~8$v?9h(Bj-p*$`I!K0H%7gv59Qg*eJ&kXqH2$*=dB9Ka2v=if{ z@?9IUgDjs`ejwj`G6>trd2b075oNMOx-Up!OIVxK6&sL^JYli){-^#E6BuqKd_A$dLGXZ|X8I>}Mzd(Wfx8Wbmd5Im+#!0}L34`@dh6f3 zB@fcvVqR}_opMRDqqBL*s>y5e)>3z*cGs}UTzW-T0`w>h;AKm>8GcrjW|w2~myV1d zz5~gDkxYXe#R0H%QZjs9`YA2``O5IF~i;(lob*$jd{J)?Vzf zqkH`}Z`XjZK&MG{cKc4a;PgdnExd9qR!}~)SonuYk3H=m`)ksyZ=ho^T^VtuH6;A@ z9e4t05(4}lrgY*13zUV|=h_q6KyvyJl;B_*`jv28DM($@bdWq|@$$6_PfYtJ(E*`L zT_}jj6agE*8+o1V9)C~eq8UofKbx>Xh{!BQs>jdRsveWpe6>5wOYpSMdv?O`+ol zXFzcMfE^>e+}LXMWRQ>>gzAs4t2dT4JXHg#AZ=8y(b2lyhzA=CemB3Bs@6Sx^QC z*wAgCaD!i}2j}Y6$VfuyVxX3v0VF5P7@fEI3aG1oI|ytGbLpmheusjp8)+Ax>wP@} zS&jacBcz$QMVhjw9kbwZpkTT#n<`>Wc^?n85S@3&s9yu<=EmeXwLDXK)6dUT*5YEX za=uoH{UMO}cwIr@iC$`VP>7VcM9xRHEAkvC?SR49b%MYA2}tf1!@mIvrG3t=h#5Qu z(_T?kD~a^p%ipiJ@Eq*#(H#P_!+Iz4{)b=C$P{qBPJQ{U_M#eG_!96Jk|}tKswmw1 z(k01ou-*B%J0yWjZ=pxL%AJen)9e5M?B4-4th;`(%hD1zD)I)E;m<4&57R$6V_^eK zjOFx<{+_MZyx^H$eGc|l49!^NsaV3~oecR38O&Qd@d?T?^h?lAPfkA|(Tt{hpO1f# z3T6I0%gekNGSMwN-+jA5K=6lvoao(rWomcWz;u&_N_nkC%;;-EoHo#U)IoQ6tlJ6_UbkRu^GLI3ZUv&HgkX^B2unj?H8J= zRa8l$;P+lKa&62J>txo?vxek3Df4PI&V0){N?yuQwFq0=0r*U*T1YiItJPGq^{51oQ;gva>{rH z>bbVmT=!6hc9;(4e<|+c4I~06X;BpzUOxqSG)1b%vP)X^S9r#ZMgsz-TaPJ z_R>pdV5;agf%eu=R-&m=S`(?LOc}y&Yoic1~GXYM; z3cu!LxNhplJR=d!a2np0V}qHcOdI~`gJhbE>Spk7G@>YETt<51-5_@-(LXMiStS7x z_Qg+Ns+|hOtAVQm`APP?v5lE?mjiyZTZTccQpMXp^W-lBSj z!FWU~kd}ze@Y_A5x?%R8@uCe-&dn#@8KTqM_^nu03y+*PaRl2hWu-?aK7a4{^;o=P z1)S1h$NZ6I6W*b~=Kw|<%qF(MdyfUywJbWJ9^=>6-l?;WBp$5eXbsdIa5KC6+^AA$Mo@{VtN6yTm&^PoAqtKpr0B`|=1Wb(J) z=1z_GKXrkG`t`AgkEdJzVCglz2F6MMAG-GEGma2ygnL%>G5GcS#Kh-B`9T`~o!gos zBLG<0l^aPzMGfmv2#^W2bD}8Merky2+7Ta*e!nB6!#wrAcyh=pn~l~labJ?8m;maq z(~J*Qqv19m&E4Z z=6(w`w=)pod5&y*K_B)Td{m>TX3|0~eTILbR4b3wYs&3kvbFS|H=uZUnVtcwPDO5> zxzp9+?@lzGU=2MU4j?7WrEMX>MT=_KXu926SH(aVw|fKf<_%ae?aqlc!4oud)1t-L z8gb;)&7B8DTW`#V)lSLUZ#(9_|JK57G51%oVAxlXnk-ok1$64EAA)3 z?&D^34qMa&73_!6h`wnHq7NUVg}?QE!f*X0L_7+vU2|B|x8LR&ACL{G+?aFbdEWEj zg}XaGw-cB7r3S~ra$hq@UpbH|%p5|9Grsm7y40`~$6@Fs=4*X{u5Y>K((ZvW@qWFM z#s6v6Hwx#4(Ic)=rmH|U%q@$kJJwN+FkK$snGh#j-3L#v1>MVboNpV+0v!GvW+~`L zhtda*O?XOliq3KQ{DTK|pMBlkhvr;lzFzZ{7XG7dqQH?@=mG!(9~v3hyvrB!3)2Nb zBihB1YS|Aga0Is{SEBD6_T%~XJ3&_-Kzw5}K<-zcX(s}u8-HssA-_52U0UX$5WZ!4 zTr0ZC#t~G8Q_95=dgZV#9O&~HM0g~`37p(H2s|Z5<%>huI3A86MGrG4lq=Je4ssIy zUNng(r?qkXc*m>eylyn)3L>*|l5+(E#Jtx0c>+x45?DMwo!e4to7}Y(mb7`LMmk2G z9!orG8E$G2SD)aX7j%jqlXK0me|1hgm&i2AXD!^q*+MbbS3>%u;XP&!+I)TTK+lA> zesL3|D{Kag)&bL3%^T+kiRzuDToGL14J2_`-k5k^-r5wd6rxS)vFtxLf$F7x5vmcE zJpUwk5ho+f@$QgSlWTT_%}0VpPn$r@!V+-_Hs5b{n~%gtjiKBs(i8>iw=PI(JV87y zmUrQA(~;z^OX_))TSlEdT30ds!Dgy-cWdj|AF*k%1C$gfJ(3T*{%6L0K|xfH0=Q#xYs82iY8;o-yrV01P)m>{KtjU8tQBC@chlXhb&Az z*{t755;o>VB#h{PH7RXSXH`?9$r03Xp2*}!J(%zc>h`-1BX`;f6NmpZ=q8Otf9yNN z#~kTF;zzTSbfh+ViZdoiQer+Y>wof(;uAAQGjH)0iril!f0o$5M0czP^oyqx-m{fSSxIlu&7%|P(Q~E>(dOz$ zIX79gzRD=$%dKyXjOL$>G(agLn2n|H77OMne24wsPlkS7(zIHE7H%C@p%i#y7CLj&*ON_ZXf{-#Ef`vXji+wLl|f}34&+Hr!F zIFzTpc_^c|1_&#!u13;-TpxAD$~dO})uK?#b+~k#b~{CP(a~Q#3tJ~PYj8POaPbbX z(uhUeTf%a8Tn)t_Dm0L@l6zR;G2)z|j11}}F#ml&NQNY~;|^#0*gL>D?r|&sJ}sE; zR~7_cOfyq!b=D>9QE|n4S@cKi7QV!b7+cgr4(RV?&XLiLy~ku0tc^Ig9AVS)QGhM^ z$gi@OPt`J_9;}i+>~p6K6M;8v-sapr%W1Q~&Ouuf)xpoYjP1qUGfZm}Z;A6d zaXlnE9~z^c<0~{h>1{G3-m~?4Nj#OF`?YY}%KyZL!-720R(7`MvVkI(^72`AwzW8W zpVwf?p^WL*Q&GZmYf9=-wXZzZ8KSO!Sw(X^8m8i+eSLlg)TdexP|+{r`OaN$vuw*@ zEzUi+tUHz7RAr4qbk7S?b}ipXZ$?WBBUfMvcPqQZ`o7#L4&cW2#Q(*-RJ zYk$T7(U(T^c=d4Q>oR4lHCVd(<3Ll>Ld#f5m z9xFhp4`w}~HRXhNi22c@y>w<=saXG$N*7?B_9}3|EF&Z`vZEdA)%1q1r;`Ma+~bg; za!`qNv~6;%OPNdXpb*5DS+n{^s;%*SB)!N3tMX{WSBioXz%&{ohsB4Z!b-_5-3;a5 zBVu@HB`rVV6KrF4(y#H9AsOOCy;)!fb1h?G0CB6d85%=->|`y873Ju$9LcsQ z{Rz8ZyK2=!Zy{{R0F^CLQU$ppF4UyAdN<}9vsJccoG13V<^=sHxA%_woG&JM^)AT= ze%nPlk(Z1O3-!IQDXQ&RaYp{C3YOMpkjR~t z*2Y^jk^w_n!w->J+{xu_s{&{ew3cEKwAWQMSLQ_Dr<1`{Y0aOe^=z~EEQ<+<_lr9c zBn*E1=I@#0OnqO+VrfR}9D}?idpFA${qS1pWTiJ;0@wE{@uEiZIi1C7XDoW~BycDE z?96bs>m|OGs=E3HAl{`nO#aFo$k1p*3fnLdtH;iGV=Y8wRzkt_V<&H5v}o)M-?Mr}tUdU( zxcn!5JEo?rc8)8MGos%~^m^dT4k>eJe4v%b=2#vKs}KBZ?OVJ4gj5EeqtHmaof+ITDU+?n4?O6U`He>MRzKN1O#R1}h}mY;0&Nj7be@kJ0f6e3-}&>0 zjPYP7N-rCI4wp&_J^{h6%*op4bh4JX1ypL8h^{~DY7tn&B6L=^JQ>!SoFcVi6ihaS zvl}+}p$lPSj5Y(>>oWXaH7}G@&5x=%XjCm>p<|!+oK<^lu;0z+s!4B{F1$>2p=w#| zj_W1VdF0#_EAY60D07HePxniWEa}&GVpSG%KLoE7UO5ubH@DRt=!UUH@~TTa({#hemP)GzRh9YE%TQ1U0XY&KT_&{_+BaTW^Q4y- z_isi#S-if6m#m;|%l9%biAxT_qU_2~`}E}mj4oHcx5QpAFTyL-Pu1p}xpbRzYbW4p z@t>)Nln%Q@$bTN}$7QLRoJB1E*%Cfj+m`!PgZ*fXlCON5!hIBPuntA7Rii1^hIYk~ z=Hwf?ZY26aPWaByh{f~P$t=Fm48)$_ib=JFk@%PZE#sACs{(<~#i`y_`0QzWxZNhB z@`X168~(R}KaST-84hUiubpw3{l(kE&KM&Yw`67>74(Tr>YQ{%JCp+!68Vv66kF+< z^RhybsfJAP*L}l@_fwkuX0Kq3J4{5^Lu1cI&D;f&=8u^cGL047ZT;0M*kdT?#7eqUepA5bMZARL4M|?&k#+*^YkhD`TIX~d~b3o`CO=N zV2UQHK%;{bZ=~UmnUs+pYVSnu;Z`wr^Qk4~;Yqj;d2jmB2My`#6_^d!{z4btzyp#g0|J3P){CH<5TDo(j?Gysn(&A> z^rB!cfL5ndwxt$7JvNnJ>$E3j_sR;Y~$9cgPR{ZviV=6lV?AFveRY2&oJC$I3_9~K1#!a$tG(uEL$>>vX3;n359}+ z38s|Q`f}lawRq8jV_qS<^1h_^tNPF_P-ezSje>ofglq>KgWFA#B-ubfkkV{qxJrex z*bL{*a9EC!WR)t5-da}u6cmoV(@uRt>zrt4&A?UtL)db%eC+E?e#VJcua}Lij*Fzi z;0cZMgwT{)`4_vV3$%XfJDP=PmgSjPML9DkLkq-dxUbg&tThk!8Ys$9FQsUDJKdQx zTQX#6?RSf%^ZLeKW0u}zvUIY72__NfgU_&Oc<7tm*5e>Q%dwhdt!wTc8G}G8voM<4ceNr0JB=} zKi+;jGA`KOawNIRF4M~HBu_rjTT5I;{N~g8YF;*!uIAgqGhTR( z49wq~{^mYzbC%99SXPiJt5V1o97$dje`2XCV4zAI$s2viDBi`5-m|3DuX*+~_)tVQ z&$TGpp5nH9p6RWNz?&4D;`@RYEL<`Ii=Vi!qgQ2c-BpKWcorG1DzPGFNQI@exEUA+ z(|nWpqaH~Kv1f90s6Tfxsn1CcGJ29oBln~Pf9b6VVnUneBd-oCUD)bh*wlzY1RZHw zyav%QP#JgQ*T698Xg7j0y$9ds*r&?rIvlNtAcQ|$;FZB@o(~F!|4NU=e`W+G>3fX0 z=A?6pD{LkNbl+Unc+8UfR31ISz5nO{b&$th=>TnbO!Y|T5ud+zHqQ`EH5;l*FapYSS~~ZvT@U za5qdT-ESFjk8oC@JuKY{Rcb??rg-bB>Dn@lfl%O>cS-ejGCw(YBhYqz^>D02Mq3hy zhW{y5qB<)}b@nzh5uaauD}ed!cjMXDf$(e3*OnbIr0$zwNWHLZRb61;C6w21RrM_4 znyEDuJr`}2i*$4DJRN%y{$~4n4YxkkjQvTtbA8^sTXfVpT?`SmC$q z(h>5FkwA-cqKfQ>+dpQTq<%FHeNH__iFBLSX%yciksBvW#=Kc|Io6e|Q^4lBUqkVq zc9H~ficazCJ+~3knooa&Gm8rXM89qQC@E00^Ql{nLsq)&iordJX^yn_iOvq5qz4}f z;&x-7n7*uXwAQH@s$rgCC8HC`i7{dX9pd*gu35 zNb*a%des3dIN<|r)(}cY8Z)(`2x>wF)}#`bk+pKBWbn!(0mguzGw;&y&q6$dHr@*H zuZCh(kz#h62&^(tr)dz{omArrQsayNBH|BM$MTnQK!;;!RmBQ*$J1xOzQ%GD?@(Tg zC-CAygnGt>j*3b6_Ck7IWn?c`hCUCBkAmU7q%@xM9i z^TVD{yZxom5FpeS+`2Q$cuTW!LjSaXY%>5ct`(KLdq;UJpkPh|^Ofn|c(F2i|7%3p z?xZ;Ug7Y7Gqrb|>S(HJuc;+X`w`eyijD)*2n*tUy-e0a&6+ZC5(<%DynEec0pB7*| zwDfq`&8z%Shis@Yy(E2l^BWY+5x&|L?!c`XA(Skx`IRNO+v+pjv~8uw$`6Z|c677J zB7dY;WtVh!oq99jKlv@L^;FW!Y<|@N(V&BAvn|U53*IOm9t?H(HfnmaqO?+%NVoEf1x(AZ z&)cU-8)y#Tbg&o|{W-GB+o>=WqvfZ)ME09&{jJD~eN0C0biMxQd&IcZYLB|lrPS!L z`Fo^CeX1l%FhhfsKp?a&yM(42?Me;F=oAQJD53{OtBtwxp_{7tSW@*K;676{7KPbS5p1_=_@v|3&?P;h45$aRc){97gB7 zI3~5AM75-Yd@<~ge-!o?@x_+TBEL~tVX_=MgymeTw6i6Rsc~&%r=6`MQ13bdXKB|s zPVymqr`D=tyli^%wcTSb4GBJ2^E!BB>F6ZQVp|_f=%gQ>Fw;e=Hc5<~Hwr9z(u!&R0YRc8K41AY48m6Cy!-ko*jp{Dx*vvwcMf zHE4mQna^8^#EJ*!ZJ@lTAg@!@Vq_s11~XVQNMzpgr$?nstm^7*w{1-HWHkCYTWTKM zbh9as)*A$??OBx5>X*dSRTAzqSTvKz_xY4Kdew&>`?A4$edpGF>Qa0MJ#kD_iHQ4o zY8$@+tNcWyYkE@tTYGLN`v&4e7*r!MTQ7Q0-L3Y3)x9rpVSN+UkjT3bh)5n zUd!a2Ia%$;?iQw_vLiLbk0%^24QzspDsuM8YCjf6>Ob0Ff6!r1?RYfM`PK?C(P&Sw z!R%F4+ib@-(Z<7Kzp)Nqjh^MmMq8o|{F3orNB;?TCnd@z{(geg{ki0h2iXF(9+&MW zd23cL_Jc(WpW(w*>cV;EBe)r zx~LAzyT8#Q{Nz$vnCZuT?&f^d$G7@(U$VdC9BvE4;w0S>-G^OUqrazPX;uzv>s1%Z0vfV%|+POC){9ZRwBLK35} z7!L0G&={Gn#8wG<<%{DkK|{a7iiijCR~m ztSb=IijqpKf&h8SrQ}3T6%yxbD5jjqG<)hq-{-DU?L0O?JzmYD z*K5>~I23%H3AuKh@2Qm@$*^?Y=kYa}U}O-pc6DC)iW;C&WT+yXPAEn5zVU%kA6D?i z&f4FH9DIs<-p>WBZIym7F3VrrrfmPEL{*@XfLBuN)CH;e?`*u&@vi32H4y_f*bAN> zQ;0iMz;WO;e1rRLgfmXEf)<5TayhH@1QMyyw-^5SYj>EUrfC^i*0nagX&46oA@q-g&9;}n{{ zZ8|#nWB9Yow{s|H`DYU>;OF;L^%rrIv$6)S8>zlT4|m)DmN{8tx~CWmcNUHs{3ie5 zVUlt_LsZ6Fs-zX+NMw2{dm^z3UG~J_SEMePS60N)Nz%09rloS(cK-w8sezaz`G8NV zwDn2Q8X7#Ezp}ddbvV*tjM6widGeNMd3B1<2xqQ0z;3=Oc@f)y{Ih0|JYB0gTGdpQ zL$tsYWco@l@z3k;9#5ae4-w;YC!z=#Df}F0O2-_2h4ZA_F>rR#g~&}KKYwsYa_Urp zl;YF($r24Lrpf*X`EIJi$^yzBCVhYo*RRO-U3$e*Q<`~b&O?VGQ#Nhev0>7COizhj11L~+G&s55cWCd)3TvxrTj-2h82nZp(rzs2P|}? zgP1YTS!F+_#ebI9t@Y#WDkIUvy`}bh2mZuKAi@_^vdIo-)Mbv{XCf&<#!z zM2b+f6~)6Q?8}cuLT; zh+eT@&J5xlWH;w5A=^_(yQ*x2skp}Meiy?|rJn!J9U$Um+9=h5Vt*^e` z_&y>^Y8AC1bG66U`lbAQk|q(IMeP0qdS$~Vb!yq<2KmQg8% zt(@f;y=7wA4UMacu+BfT@){MaE0}Jeb_|{Fts#V9 z`wD^RMCd2ux4~Bp@G9=(H%8QEcTSJ9{nNh(STbgE2fDh#uP9M_Ut%5XI^>1WWy%%? z(yPS*TiRp1w%;NLRxuI0u?-ABr|q3|HzCusQh(Wt;BMGB|+-2K#v@6&H< zqvT%CS@MUBGIHO%+Bb0j{sN~jgr1%T!aV<2N{bBg8U2D3fnDwJh^TgOF91lXZg z#v=@58NLG}9`>AU?GnNJ1RghC{tANR>2wrDObqMhXwvA2Eg!m8ud^0*qM1lL7l$9o zu@Qfj8MLT9S5O&rwbJvSMHoCS%_G4GL)L9%Feoe{;r*L;7kQI!}+-V zXJq-gO83iu-BVDdd7<=)J8CbGRGv*-lpWXCOYu|RV{)T41>4H=mnI3#s_P?C72C&4 zI8FFDL663n5Jyhk(?f-Ri0W{~uXr85LC*XPC_BK& zJ{Oy>ObUn4j{3EDjnFk25hYf+-aq(zg#K<1S7qM?zYG6$Ow)a$iXTezUSY4XjqX?8 zDq_BoQXy42mK#t;f3;2zO(W`+;$FSw4Y=`&*_2snvJF<2bW>&R{q!ZJ^PH1@J|EK2 z3X%XP(_BREE+JW3bow{KejMHOe97WhkxU_F(ndr@_tfJ4y8H(UTS}r@6{B z@|wU%jQ+xnWYGL8i7sZ-*T6W22#QMXOk&=$ZJ!3i$!(h!Y;yZMgw{oXs|)OVwB%B+ z(w7rmEBn@iSjjPDyuQ(yuFk1XYIA@J(HB8zwltc=+M-)Lb3CK_7rB;l&99_IR2a!? zp0>}V zkB@x#7&#U9a62EZ(>FPt8xhx9+`kO#7&()4elr3mqL>_@_mk)D$1=+cf6j zOnzz*87@vD>)mWLr?wO2QzWMJ_gH!nP=Q)$)fPa7>q}M(mHPB`1!|3Zik!# z3cL&aUv-0bl*j=Y*rTda!JTc&?QV%W=Y}v|MqajH#F4xX{4^FRz;gYye(BvaHmCZH zu{;I6Z}Vj-q1rm-_FRxBP@ODE)&g$3;D*nU+13jc>@j=4dh5zl`ekHsb`tv-{q{(AdsCc%Bf$oex#406o`i*DK1hE&X#es7>sLW+^ROKI%(Pvs9gL z*9TxHRBMQs&xDcdqpdyHk(o1(GM{b@|8c}Ws~CI#*_Ywk zOdO+c>1z4sIVKrm$=Jo|8~o#&UpeU7DF6ULSiVF(Ew`nt3^uP4A%=On$aANF6-|dg z9o#ZoGA(6m%M{DV7RYf&F@+=nyZiul4#Tm8E^$_Eec6Or4%}bA0b}u%Ly6BSA?8 zxF#g(zu-7{5UY}TI`1FljRh}qiWx?g;SnM3XS|cj4v!2B{ql=#bPJ#ChI9xh%3_)2 zQIO+M9MS0LBtj)#;YicMQTwS=@(HGxwqG!sy}`gk*$x+afXX@7o;eqWMx+p52+K{t z%kFT=m8W^C)bXo>IE3aA!R4-km@$SoQ6E;J8Dxa%R0^Z*T?$M^z3FEk+MwPTk+3dS zDaPccTfwuwe>me{hFi>kCb2`7O!v<@7@wJdUHqivwTezGMx@l}iBLRb5s&kL>-%-C zvcxuOHe#LK6Y*=RDO|!CH6Q4H6eFecMX-veqJX*=hCUN>eSaJ+Q488%Zy~H-)Y#`? zxt4q$c{$z_O7y2-{z=8m?X2`O?JUuVV_OTcI8lm+;e6gx&0~KuRr>_1qLaLa8tV*v z7C+Jg`fT{)3{&$&!sXPW+Y-!;xJbzfXoLS*D&n0Am2|Ip@Ng&{hP=C%6*D#lTtwlHH5S^!o7$swPtYX66B)) zJMQyXvNb`3EB9b=m6S+=Am{YDgX(B^F0#pcV@mP(>5H+Z&CNAb=tT}kkZV5Zk1iNG zWL{&wj6~nrQvb_K1yKMKW4(RP=mXp`h5HDfgm(gpPV$~eZS%O8M-?3{#e+zMn~yru zVVb@nwJn=KPilcF9ktxroNHE4#!Ixc93pTz! zI1=iUlt&aP=SoLt{WK|~8q<3k3s7510R&$32`kts4u%O3Gx|)PFHbFu=sm!$_rAdh z3XFk#6PK!$#?v$0@t?l{S zJ&Z)CUSA?J^!qu)dh0sd^WVYTpUTHlWx&T0%E25r*0M$D--seTdP2_shN=5|qvZ#; z$Yykzi3sRo6H)N_g6F!vqzDIth!WZU+|ZIW***E*I0t${A&Py2MM_4nWR`P4O0RH| z7)dLpz^qEZjhr@bz4i*nCFhZS(@OZ;Uy;vlOmyZdp7V@};Ij=a^M+-&8(SrvyJkd= z1^Df054uEx3|S=$Y1P~lu$MZhwln_+EKQEV*Nw|0NwHH0zI^9pNW!H!es8Db{yK*# zqvVs3Z#!{oiy`trhWn>_Z*S}F&Z3DKbMCWndD;4hgcXw08B z59Q=#7H|`?Kgo<@Sb^6=f@c58KbSKN_%phlkiESM?l*o!r$tJx@-?FRH7M>Ly}yP) zZak4xl5kO6uOq;(vfa@7G3X<8#3qj*&mp}!Bw>YWir0^i26stNSe4p@rklR74Q|Aw zNPR`;9DJ!T_y}`Cn$LYykzZ=PHU}P5Xmy>w{=w1}?1_~$+WF-mSSdKtuQIA*@%ag{ z@4sBwv1HgR9O-iB9e(mRTZipe*k)3gHdS75eN0iE-(XX3A3KE#;BZELkz8pebX91G z9u1Unqf}KMH;s*)uJt!j4avBb4c(e!f)i~!J8Ik;a;L*&o9a2Ib65nsT3Qns*4m05 z9x@snJnoylkE`-|_H>hR~0G!!=vePJ~9HTGRht;fJ}4o433ktK1eMPSi1en-}AB$HJjnoVMkbY$vtg#jN7hw?c4l`CDt5N`zwU0N>UG z6)!|{DS^_VKEW1uKA@7#QEpUNc5e)}FWZjYB!tXg_GD`pUASTGk!21%y&dL#`S^t( zM$V#U)5|X0ut=5Edl@#Gs`5WZRGS>@S++_>TYF=0mL>c=|ER79NG&AUVF*+NY}Xw0 zo^xepO~GR17W?^ZU3dV%1TO*_S5*2+EMFgQy3x3zRsK8(f96N~!m+oWgY!OO5%=gP zKV3R))7Ojn8u>I+secRy)_gvWbvs_1bl#>qRln5}^Tx1^Ie8@xz-=@KFz8(BHQUnz zW1<>otyDp~@D+*vk8NA)_+5H1XZ^ntTX(@)_KTN1L;55{#cgb|AA z>)b~{_MN>;^kY2Ed_f!4Qc3&f zWO^?>ZKPqwirD?!d%UaYj=-IgT6kbunc(^;V@NM3@E&2(?0Y0tNtxVDSc$57vgOOK z{Ky#6L7gG?x=Ldvjkxla-JoT#-deJidhxp_nvfL#mNuRb5uUyx-Dv(ODx%T+WmeoC zd=X;$9cKq)44g(zulzH-x?rS_a}M+17Q(P-QHUHic{m3!Cs|Fw&n zz%JG>A38G9R6fIb;`ijwV@cY1*4!)l{I;S{%0ik#+J#nS9ev9b2rPxU@))!0EhR~g zekJ)iXOaV=C*Jud!O|Y+>0oW;#BBH9IKF#ZmDo8MENsd6#;(Yd#4aRc)V6vE#h0A%cQGch^s@B!i8FTJf{UUjPuZjP-V6` zJe~c6a|>P?$GF4!EeHh1Qxmb*6~Ba%6`YkIkd|c3O{=-Q`C64USYlr zmnVneZ{Shro$fR@BD8I0r2Z7%%eQUWZ^X8dU5`uo(mU_6bNX}XZJrrD8EDOx>)EjM zvUV2|X4aSnkqBjWF(d3-&Bvkg7mT6xvc4oH4e@EM5-Qi(Zlhdsw31_!ydUhp%g_#a z^<^BIR8kXCU1Tv*ThhLFN|O1)aJ7D?h`qrYOwz1Ml<>N|IY+#haAC2`h%~raQJ)HOoGP?Q5KzFoB38fd7iq@(pNotBO^nJu*0~z9j0ujh&@ecB@ zr)Q3I%v9^`?dX3TT4g76IB(2c?$r5taBxp>U|0JQyi$3I!tYRQUTwVy@BxtI4e#}P5<$*N3 zGti7@dw#PZIK zNY%ry|Kv&7tL4@CneDI8yML-T*L2b(EYgcp!{y2f&BfjaM^S@KP>Ensd?SgakJ9$K zw%=O9wsDbAp5|ZqQIGn(JZ;>yPAtJ2p`NMSqB_DElN4fgp6j zf7alL^Sv(k#cVTsUG+Qel9b28W%hgdT^SK^GXCL&?%^v2xAd@|1Tk_5J*psBk4gDJ9kwqUd%2qRZOChgPNXGhg*0lM^`Jdqu z;vzog-=;-S65$SGK)N@uV2T{B$(?E&m@b3rDl18V-K#M`&TJ|x^CIL)47G^P0!-+> zi_$f`8up>FpFWDUMp-O|YEy`KU%7%%_OaA%uZrQw)6d083;D=Wk3)+LxrcjTlYQDc z>M6HxW%^>-U)}AH_V%=i*-YEs|@h7Ly2$VNd#qn}11^1?G=FC2y|5PuZZ3I?GkMOA9}- z;k{c*ZKFB#jdTLUY-YEuYMZiXywWPzSb{-2d_@5S8G1uPgLC5)%qv5u99%+7 zWU{=8+IU5&1PunNsdk&;`pgorH_qw0_k%R{i1=eZI#gUmtETmwu>>kK&;(qnki76( z{p7L!Gnx3bd)P7~Ogmbt!rkSCr2$KSRJ^h6qnIi~mECnTT}xYKHjmBb?)qF`k}jHR z1ep}P2{KoM|NhHEKp%XbM!X~J{Kg5e6LuvC2z`Dec|Z7(Ei2rX@myKf3%4$dK_yG}D&sMky z+M<9nUQpFhiwfmYf>Jfv-;<>-^&;8w8EYYdITk0-^V2j+KI&nqv+4Sz3=f8TXqLSG z{jDI&W5<2YNF=8MENp2Si}k~jza&3>Bhy5KFtV8>$dPKtZ~CwrE}`SMGX8{C(UeU( z=Z1nsMmT5&Fl(&Kc%GqZAxvNjv4nF{zOYNTXkT^oy8q zNAh`&(l2%!`!Zs!mG~ zL$I%V)G9l<;Yldl?KN7wX&n|2e|Ax`9I)eXfUO)}MT>!R!B#`lcR42@uB`Xj{{ovn z0{!~fuBU2V_}adMV+dc3;fVY}OkqV@(UcwO_8?kc4oBs~^LUQqx{zhMMu%K68NsPO z1>Eb>a508*Lmv^VYt}S>GXS+>{agGd8DIXlg8e{1rC5N?np*JwXZJ?NxC^o|=5^K& zg>`l>l{uW{GaBQVXqcl+`@N5EUv0<7HoD-`U+5R~L_4#K4~NI&`-A_xEW5ASrFrKM zdBh5O)b-q-RVroc?RkN|UKPbEhxfzN0}N_|0Riqxv?RaD1_dkjKcCv4*gMAsMeSSb z>pXP(85Ju;{qe)v@(B0itCUAhw_Y@>#j!!c!RoT~o1XOZwofz&_w^kZ{HaJ4XgOg= zSjuR8V@sOdxMrUSLuOicbFIbmn@bJ}{T!_ZfXA-^UsK9zPE4vkaAx zn>cjYj}z~G7 z>xSYG=JM>T4{AHEN0>O55u7@^L%bXY2f|M~d80UW$L{OarZ)z1ON)7lcP4J9n7?PK zLAH$Flo1$AQ68r2lW1Adzy4c)WKmj45_936~;B}U%(fi5 zxgP7KtJpW!-zO#1)6bil$Comb*d6ZaF{NJpI1snq_`)HJGAH9PEWc}pvI8qtFsv|KX{(?*Cgi6qoj(u2Qt;n7dJEI4Wz7eZeZX0>$LJ`eRLVPJBHf@pnOp zs?J?BWW{d5GumMZzcJVHN+F9@cn%DRcr~1P(eO@~z13(wY{sJ+xzfKYT=)NaD{au5 z`UUB*5HL7#;Gk^1K$Q#1oBPwg%L1Lj=g4liNavjFtQXsYhbC|%k#FNUuMAm?TjO5S zN&NjPG5>0NHu0(&k2xM*`1!T1pJT&0D~&jO>Vj(gBbN7I-0e=iXZbY2;;&@4fQH-@ z{e80Q@xHk&5o|jS=Hf7fafEd~m zE56i0BnVSiZIxn=BRnM9oPtSRuAHBxz@*<5kl`s}EWObuz%976!)Q(C=geJeQKCq1X@KpS2$#k*`(1p_`9lj)F zbML&^W9%0SJ*EzV(#dKVK=we6rru1XDE6&O1uYV*DYoGkn`EkV>mi_juL6eTr$svz zQ5;GEl@TUI@?Pd3Osu#4O(~KIYirDPxl$`q5emr`2t_SEgHl352qFVUrJ}4O2{Iw#zATmxMSFxR|We58XEV>1}`wjkWR%jLTY#N=~{-aYCaNPHOryv zaAo1M#d6QO*?`nm$*1O=&r^@Te2hG=aip;~ROb}RS+ZW33|YUiZ=0{3Z%C~SJu$BBw3YH3g=yYe96%1))P+Ey@-nK#pMHEH%BpyV!_g7tpa;i z-u~)OpFnL#L3EZ1?vgz@{_nPRI_q z1k2O1^-bSvLdZAjba5U@E9baoD~U^@p}kM*PY~UmUy(K#5cV_LJ)>G=HMcgyMhP-) zFkgEj_@FUbaUN|1?gD$gJzzHNhMph=jn< zv4Om+{XjwF1o#DYb%`@+&oBp`=ECv`f@G5=wNkX=-^XVW^r}>F?n~+jQT3fyHhcBh z5~}LM?%;2_Y`)dftnsDFks8<6KE381>8pS!*PC&7DP-~eCATHCM5M1s>epjQ&-FU< z4;e3*IAIM$xl2dpw|u!hi$7Y|MplzErhET)Sy-Td={T+T+ooeB@8Egb>uYd#bcpbo zji6$o6@Ntw=!~I5jIM9^DV8 zF;)3`IC!##+v-~Bq5SYqTeb@d?`c+o_-8KVVe%t;#qI({JOmXw3mFj#Zpk#2-PN>o z@L7PaD%fDL+E4G#;`$Ih;8fzJOjG2%6ScOjA@wdtU%kSa(;PBZcK7T|no6(pOQ z=m$pzaBzJ8m};QJ|77em`lF7JerDMH)L6XOdIZXj%>nG?+mE3srVU{>g!1ZlM0x=*u2nI zpbZNY>nr`)z5_qyfZnO@Ehw9$zvaEin~0NOpT_d*%!8;t#}Eg@TOG8W7La+lCS7Ue zh;v@?Z6gDPJPPkoAIHnwNDT5j_#my#B>x)z)O;`rWdI4*{V?~k3RzZ^8mj>>=8pS? zqTOG9AIFZuI@E1tLqXX8ykGSIxBVLN(MFa49UV$UGnk%L0QoM!%>WP<4pXgs|MG0I z)<#lKs^#hQsKo_$L}+bZyZ#)q$nm#9H)kH<4Hpu=djd$7gLK4;RW9#oykCBgRX62R zIu7z#i)69=;LH}si=M2>a zxevX&(Sdw@eZ~*{(bOUcm6Jrn<;1{-$)G@Ne6}1R%ou`>B!;1E8y+^K&~$wQDolE` zTsNFiZXu7)EFlW5IAj9SS~w3PyS638vUU^-u$n+C!-mg=t&QEG;p&Qq60#>T&7i=c zqa@7IuNsV?LnI^O(i3DhR=**Jp5CyVE2*zAE6V*?rsGG}`OO27QxD+yl}nA#^3svg zsSZS41v5!6T>~l@hbX z8X$#Y&EkHkM;S)IR-$lT*tdV2L7-|WIRMpcUbVB?}An_ z8vN*{9dtMG-hzIBEWBT(MVxZ%v55!lOKf1eBs1t`yr4Hwip23DIR2OtfniKPPiS0~ zrQ3<@#|}dIqZ7J5wTCc~DbU}Y{P{kAk`WXLHg;qU%a|WX$1_TkFiMLvqCj!<7GXuC z%l?r~mx`elPf>7F+4>&@l^H7Yzs7lpm#ZUfDw(7u83F&-i+-%;>?c#98ohx*rb_(^ zA$Oc=X!+eNzm?2_%i|x^g9BFI7hyH# z@xD^Ak_YgMlPrB*8MO@ntE4v&4jkDrwCf zUawtao7w1|3(RLT+Y@eH1LEZ~w+oaf2qi}PR3UHpUok=jhaFcrxSw_vhQ0jDnxig`adDWX?5&o^L3C3$n64fxzjqHBlv-Sdjv5&Qw^U8u#8mq4tgRp&}?Dw8f!)hak!!I0yh7gppgK%th6$mfU2)P7yYI6&`z4yU4m(oCAeWScU76z;%=P7E{rZb@huS`(mn^jBJEFB3 zg)#mjJ1-|J*zYhqgYvkj9x+8&fyw4A^n}c8Fv9Zk>U4)fl~P($Wmr3k+AKcs@fwEI zfJUS05xGq;2^ox);wTxN``yZu1W7r0vP@=j4>?MF9_R-(k}-E$rLiI`;BQ@SX=&AkGmb2-yk`{a_hc0g=)o2E$TA}M5+QYde5|T&eEzYU z#52ACsyR`+PiFAm#_a=LWZ_Ak(J&p4hrRHKH_Vc|CV=}fB${gT=XVRD%z6~6AH{_Q zUYVNgeC3E+b{q^w6^FvTSK*WnY&+${@0Ht-KX%3}>Kh3@EFNt&z3Qr2D)FknnpZ@! z!n^!~e~L7W92HnfOWw5Gd59cS2?DL4h`GG+{)ae|ekS|3x^u{1cCg<<2X=_oW01D2 zz>}#UH#Se>W1N95^OKX1ia6aBirF2wEOdhg2K|1@N9Dv&1 zy$@Ng2+^?VR+55>@@ww5D@S=gPFO{@x5@uKr$&g)nl3q@bn%Ti`zM}JNd^1zO}Li} z*wf~p;3yT6~O zSRNLlNvSXrl$&qD-H9N;a?};kq(!0h;u3WhVCY}_IVKbh zS3UCggBU#D;IXHSo=gJV*d4$PoBV#$OSmGthoIv}J*3XHe-hA;Yt0-%Cf!ozEwQy7 z-%FtS;si>l$a$ma`rzF?SbWRy~?7?Jv5hrP~z=Av&*^h*eP2!3ywn%BU+Tf zmDr3z_nt~B_bd5qEK@n(QPv9PoiBwG&#c`b@#0$x|6~Fq(Su{3CVTq_R5ou>%+TM` zh<-Gb@y)TOcJN`=0GQqvZcyebi~#8uJTwaD18fQ-#ryD;TuboN12?d{NJ5E9kKsJ}WQQ*ttx;8)h zy#}wxhSb~BIs!ZNPBqH)k)$SEy0%_zZ0XZ}hBRG+sVZXEYVG5+zk#VqV<-17z+QIK zutHYh`Cs{&$OtU5uy%$f+s0mFTU|x7U{2u6nG?9VL@de|gsy*I1vE9ST25-L_vSV? ztA>0eyYIFb3te=ov*o!v7u$&U>&hGR!)`&?fd@D3@sKw31@hSExs&x+i9|DK70EI8 zppg0>_7dN|&0x}ZGD1QHMj#`26s;F!7FmZp?;f&y4>x%pi;IvPDi@cfe2aC#^VSnj z#q8U+?wf5q^<-bN`q`+Th_UJ?V&z@EJT3HAkK$XlXXS%*7k1_?{m%~AXjYYy_5-~X zdw(%BY1wZ%)G+1ZhgTt`oFPx4J(Ay>h|nSMARqH5{a6$5kJB!+G&dB6lOA@+ zyC*;`K_BD1%*y?}8G#I;lOI5@bH)DM>&XyYRC>v*TD+~t|DjUlz134*vU+K?=Z-pt zbfM!3ZCDl{z_2cr=MeXIYFd5E1;n(e5gOm$B0jxdjvr?FRS0>EH=6$B)!zpVCryor z64BQPKYPN9=o`JyGy#DbFn%5c!F%7}_1__<+hRY2C9WVUjmho z_)^%EX}A&8>wEpR^q$fgEt!>vnPD-OCqCcl|dP)BlK5@$~*Nw)r3w-HX z&)q>~8}g=PvPwG$r>^ErnfK|0cM+4Y%bF|`!I-^OBBbJZzhvs!G~J7C}`XHe4T0`4QL&)M86SG*}xFI|r;6= zrA~1jsiI2D0(K=dV55p_?KQuC+Hid}2#?uX-w_T$o459k`-vg*#iM9ypYvk`S)Osk zn`Keh6Il8|3tuc+Xu-Yf+B(|RBQZn1)$GoZ3+iv-`Vza2eKy$h1@VXVX*9C0+APQB zb1U&>Up@KgCj9BAy=7dAL!r_V&Fq7afvj~n5e4KLPVy!3g^)sC2;3rL(xdwRi)4mT z9T=CUjITS8Wm5F7gTd#I7&6>6e(B2v-e7ofQaaFg1JIkF?NbB?zTEAf?Tj2VDDn75 zMqwaF)$*ZKZiU6q(hBSBTNTUUC(#BI?!g8Z2c5Z?-z>Q=h=Z@c9uWI9WZ%v|<`XjK zP*ybZ!&%RPBn2?5?$GcKe8-<~Kfq*=(zlk_Q5kPbB<7#r;?og7P%J|BfxOgsVZ=>H0?=4V@4eu}zlm83 z|7;)pttO8AyXVxriOB&L)T4PL3KWDc*X9xy?cWZiQa_x;M%CAikVmmyyocV9nh61-#eg767XtR(MAQu;}`@0wF>??c_*$SsD~m z{^sHs!jodkd-3G!2<^^EyjqO)&g<8o%0-=PM7<74qq{E0Bjx_G>Nc~K(erS z+VoyPoj3k2<$t1hLH-KXgwK^Lt9c53aB867WzXA>Ntd2jh`^2S6O8^`s^Gm>%%3Uj zD4uE(;LYFs_1`fH`YkYP8iF#_{Sv`6BkE}U-NT2n5{$06BFex>QC8`F*-B^p=) zG}#A>N0F#mziJRee7ZaYHjf@4=hYEP^QElN3g*ctclZOK`ELQK{5l9tB)$grfVUa& z82N+6D5?MexpW^d{DDu28trT>aL&nwOwTrQ)cq-HlSNy0?!k3CA-A<``~tIn{dxS6 zPn-9HuA=yziQ;oB^p@F2ED9~~fHJwPZy6khLwzG|;A1|ZTbY;q*|Vhlm>CMQ=vGfm zD)`ze{-W$JeV_zaJ^I);v&(gwe}ag6=Kw4^*>(H6{e+so%gKpD^gN#yjeA&(+vn7V zsXYDJX1nu4PH248mUz)Zh1Z_}7Jg~IP^Rwm@qF2Wwss)vD>JcZ?~Bi_0V`xozCa}z z_qxn#2WSktf%Az570NIq* z#1OHf9kU{M{mQJwy}2oHGpWC9a4@Nf6>WPCCq=;r#7!P%ECR?-krz1_r-;ae0d ziG}Qp#@~lODdSxIdh%%dEn3lM@XpYr%@k`_praf$GXQc@KCcO&b}3FcoJjC;D+{!q z)c#MOR1I1{nDybuU|n>{SA=_fiY?J(6~^ZSXqqR3&IqBl<8PEoItKA1eDbeBYNQm4 zszHTCJMrn!FFOUixjV^CX>A2Jx&C1h65DNho=P+)v_jWV5k#bvNj*YkUia93_- z_rZ|cerIJSQ)r%(38tR%v%tQPP2q=Ap}PUp!lSRvKmPI10lwkdAFV{7Z z;^1{hS%7)eXugx*I;CyV9oU_)^Jttk!)9WlV~%mw2ZzYnEMTmQYNnu^SkAzDi#v;- z5vdMf*J8R2OUJ7BBTv8?V#DYVp#kIdzG0WSqSL|4Y>@qBWrjA?gh~cgFTF8n*^w%H z+z(M6ERU=XVjaB?wK*?Ji%JdIoQC60)V6109C}JAEqyb-$_8%3KmpGBoqiX=%=dvY zf)}C;+D2~mp`;^azu`Y5J9AAsh^c~t;5WC2Z}8bF>Ytl}V@D85!+;WFm1VGm&@v-{ z3?vRM0diao3Z^B^kC21L-y_i=&9qUYINuKG+tcENcT2kCZ8l{a(i`4xe3GB`C}@-y zFgW$h|F*vDH7vBP^*m8ns`Cff09c&?W`Mv)($CC>@8(7VJUau*#ldJM@f68s``u}= z1T`>SqYb59Q{bv^0NvOGt$LBr{fC((OTHI`w|gf4K1mEWgTy1H$=K#Y7&Nnr`GS}K zWiZ`=3CfY+SyEe`@+49vtI<_DO}ABjaufSm=UT^67^h8zz1fFAZFYy0_f#CKA9>x3 z7@jK6!m4T+M$h51Zj0xVf@rawI509+B>xPQSBMK_tw$0tfF^!C)kq(Y)%6@q=qpaq z8-v1`RumD9U(vn49d&*cCGb2C;Cnl{7wobs09pI>y5lcxln1ghTioK>!5X-*P+c#9a_FJ|p)dNQ zph2`?k=s@LH>0n|@!wp?FbSj?d+PH?DU=cAAA*lVJHIYfego{mpa*EscyA!5a>REE zlj@BkwEW`6b@STOkJghWVf$J+9QA749dAId;P*32eYQiTd}u}bH;1548`?WOP?C>1 z+hgE(SlqVx?|5L)UdyP7CHfsE2!IEU@QV$3$7eqBV!>!d?l4I%hr|c3QlIo7`-y|@ zk?7GPO358GepG?ud)e;PrautaWggo%Z>B=t1jUy={wjl38*XZdSiDFP1E#PZ-S$5z zYub}`$s*8({?0{gJHY7WT&+Zp!mVe*s;>tY)Y8V{SL&{DYiv`C-tqe&^RGxan2 zerxnkoKDs-(JnSee*tMKkN1WPH4Q(U)6gc#*$G;-u5y8R6m?5-Cw9qp45;_+Q&_C{ z)h{m3-+I)d)F7-@&DkHI4!V;?tHn(u($xJ?10wn{=bevfk7~HAexwItl`5YjSH9$F zhv#K0tdRfMF}}it7@4z!L*j#t%prJ%m1PJACDy`?pv=<0%sow<*6wL~)~iJFIfXLn zW?((z6Qg@fnF7>|Y${m-s_0y+;X)*S;kEexN^5S;JN`{YoT4;2t-eKP2HqpdO;IAF zfz&|OvU-s9SbF?i4&19lSJbQie?Yn8T;?hqjlVcDZ)3p4gXWDcE%1Vd$w0-|oGX>gH_#B59Ntm*IEq;N!2K z+_CJ|#TAsj#(oDmyMFvXhnZxoW8IJdqyit+9<{-P++0ibO2rdw-hsp$2SooBSBpKZ z$?)+zd;ahp)9334XsQ}F94M8VZKle1uo2&#r4z6|N9VzZuT?pEn)u=@h-Al-#;1cNU( zcmB#aML_))-fiubg|D0yonk>{s}-_Br3mI3VS+6{w9H>}f2;)6v3ocDG*@zb%!)u3 zgC_Puc*FlMFkXfMf6Ehg;P0WX2q)DMh1IG|N9k%-X(S*n=4aY<3U|GXzktR=Dbi?r zRQKEV*@ zg-`E1#6N>`q%j?$dde&Wbx=S%CyFls&bijBPr#c!q~&`IRu*wlVh9YC=Do_6`4@2k zkI>#E;^=XUG1EU0mq+OZQLU$d;I0J&bjS=<1tgZ{yDZv}TiTs`V4%eXVi^$Wp*lJ+ zE7=Ggb%f^4sZ6qub4S-Qi&`w+gPz#!z;7ksv#0Xp90g|#ghjzEY=UyD~*06+-* zSCAD}fa0+R#>+3@fxuKbQi!D@7x?L}86%r*64{nZXwYm5pToHn1^M#SB!H;e96nhx zoF#z%b+`uh8w25G^<8z?5km~u-Rm6_m{o=)32vnqP`_asZO{VjKR!_Zbzqph2ey-I z()Uz`2OBoArk;=3Uc{zPwTz6Z(H(3GLNDn%Pxi9k`*EKwevd=j_ywZQ!ws8DPJmF& z1G8X}(*vNFgAhM&9ti`F6uYTw+(SW)!$|k2zXi$yP}B3ToB$KRk>`ANU==os2V!3j{_1g)e^)ckgB+h(6Iy}k!M)Dlg(nu$ z3wUId^z5o!KO-*oo!*qmtv?_6*ntEuSfR;z^nHyB+H#5bd!$Cy9YF&%XCRY;TNT%? zy>!kk$d+ChbV1FM@GG43O}ep5KxO~P`QiqD`)}4ht`illIHMeF%eM!wQC*8waQt8c z?NUo(0Jx6Moio&353cl`UGXo$OfMg>j1$_1PZ*{=I21b3zpK zP_u|Hb8_Mg0!0+4j3S#KWC(lM($s%(Qw5!OH@$5Gf(>y0Z{07o-h#X1K>9V_1c+dK zCe4h)sfz?u~vMq;4&bRyu| zDhT~fk1p~J2%5?yEb3pWY8F4KZ2g>12}cPWf3i*wvOk2Xn7-5pPgYopME$pAsy@(v=171yTMU2@sO-p~*HIOc^T>|Zy+YmtIpbGj!*NNGcyZp!aNtKH)GLVXif~8CC zzv`zieC72iV7$X&z5qkb0EI}wr)9N_{8Ija&d25#-7bM-m1iHp)DmoBR|IT`b%PpT zQAmXV(YF5Ihfq`~jzUX#j5?0(T@&4R@e)Sar2l!rnlL48>a6`x(|W-8yBUI3lrV&dY^j2 zqV}E)h({}&LAItF3o&}1>XzfR7KvJ*wxMXe)E73q0}<#m2d&2aCJu*XkfRh=TF%B| zX-z+k`u9hUL|-<}MQqMVc^XyDs0P97FY)ASAh{<>1+nT+=AWQZkbiaq68thqpGLAC zP^-G%0$DP)$?dLg4=VTTJjR!}iMQqBFe(6ZK#xG*l-?e|FEZaWfDRGyUAj`G=j1st z0mRp4Sa3bM8l;${wXrWAf|q)~Jldb_wJ1){lUDnP8$}xWKu!sIYmi`^q^Rl9bg!s- z!tRHo-4264;W_`(sGkUE{`G*7M2Y}#{QNSlawc}nmj)(BAHiRFA>E)XQemA9Y860K z?)j`DnYs43Fji6qjwsEE?EKFG=y%kGo4p=+MF?3^MJOR&fGcLA2Sn_?c<{yMBUNrG znZGDrQRA7i$yK7Q{Ga_G$*4E-?fflqC$w|FMI(f#9muQdIPbJYtS-q)tzWTsT=;!g zE3P|jWNiCb=g&T|=RyH+qfdnglh*A|qGq~s_~3ox%70z(UGdC%OF8B+U)t@M1y4^4 zsrjVdsGv_ykDXd%Gf7O`SwwCXbLem1zEYl!)a=CHyr0^%)eYSTr!p?cwVM*hFTC49 zNXQ>ge}RXvUjMjwJv0V!PBst8Ivg~)s}*0|EUB3p2SKhG>_T!DlXdBE z53mmyOxxuz+nZ4dfeS*?Zb!G{D#i9S*X+e_YYEE#@*_voG+c_HTwQ_6A5bO!Wr0J% z){n9|fs*(DXJbAy?|G#jS1*g2Zd2LD>p%;%#9g>YciG}VxgKYe5?F-b(fyrRrtUD{Yg9tUVy0u z012d)$$HQM=|>?sOzmID7vc=m$03Fl)?RLKG#vCsKGVy}_n(1ok)WiVOL6-B_q483 zRdhUM`A$TkvVR=bTYYme?@%m226cTrjtjsqA1fkO%zVlAk(iI!_cU|qUDXSg68QCI zM|bir`J2+QB775sw@VDg-Tvu@e*b_V3fi!CirQ$gGLoBV)1>l_(YS8FCm#m3`ge5F zK1D-|sm7F?V15b%IwPId`@$UmpQ!p>t%5oO=4Ieq<|Nns3VYqw zKiZ-9v|RQDw5@3inCLJh!?DzPyfIz{=3T?@EAYbacWPDa8l%zfVEbW)7^%jk{~PhY z;2~?`S#G-|Ld@b!;>BxbE_An>hW`S*h?x_tezZRQ8W~C{C*_S6ieK*9H&sTtZ}LiA zK?9Gv@BKL)&f?Y&$QLG{bo|8}=>iVa9q-9dy1#OPq}90I0GXORG)2!`QD+xG0zI?n z9L;Bmv3!62^aQZ@A>FXsYZ!J!V+cfPmjTILlc#~1j5yBGzi9_#8A{WP@&}OF@$TQg zr6<+k$*Ri${Ai5|LkN<%dXF1<@x;IDPtU9Un(+zEzhE6rpFeE{_xtzYh@TKtO$VeK z6TB00hxp5G%IQ-e-)nF(3SI$+9cKJD6fu{fjh1FIX3SoWRRAyY84bY;L^(0dPl{fk zJQ#ylJzcQ8b`4+yZw}ty^KVsAD7U>HacK_$Y7T+B(3!1);6aXqaa|s2vH9j{q2@ev zixKIH&nYL5EbK1^3ndaq(^2qL)Uv5C_Az!05*~vzCc2>@Qa8*9h=j1f=2aoR(i2V% z4IdqYbtW4rEyRO8O1W0zIP9s{Etpk#??AQeGq|o4FK?lN(qIz-#ZfLuj!U9u3tX3# za=s!)`sS76G+b|H!uzp{gG?x8xz0C4=2(IY&Okice_fIl-2}R4^3_dbbkc#BN0|J7 zzqB8N)*b#yayMrHYE z?@S_v+#Q9By^AS)!`?^>$7AGg;05GA*v*B;ri`M0z8G* zt8Nb7V2@jyKM%NDYh<7$cm;$~=c2Y+1w!H)Tpm?3fN;XWS$Eq!#_AvwM;WMn4{S|&cJjp z?02!&uGzjP{lw3h;!@!t*i*o~mY`mXze#qZk&lc3WHmk+qAcg#a<0DLq`GDi#ZoUt zfFANO03ny&ik-s~hk%O6*Ylw9A8Nyw_pFv;e@l<=YnH$fOrdYlA7}LHZq;qx;!SXJ zcdM(;N!^`RmEA_wy2bg|9Gli$!w~oBI1gM?MwBghEB8~D5}byDrq-kJXGHRj1AsRO zi^O9}Od2mU1Th>!yI{%54s8IqFXy zflU7})7?jsEXd%s5x{cQPZI&GQ>mG;zVC>lBOS;%1}d&J8$gC_C+Jkooil^qJh3X$ zWF6sC*P?i!5lvdXWLoY}Wb6B2y4sV^l7pO)!98eIpcV-&`0p3Lf2)vWvLJXl;7WoF zxk^rRLlGy%f|-M6>-1yh(w({I{rU8NQc_v~g@~2nb-_ac5?8Lo_oRX#oBloe;^lRf zFUobrY9k*YqW)O`H&hSU`Wilwa13w)FchqW;?}jnzfgW84AzRkjN!eSaix3A(E3Ga zS`Y%XeDp-(*RqY_tq!F;%R|wBkKZK@rT7@|{!qY$W1J(xr=f69PHh3P6_X&@3q8jm zT=aago$q-aX$!b0?vXknykPVk?kBfNQe=fEUY+Z^4pc*Q;R+~7solg%@ne859P zp`HMM?tn8Y3^LK4PCJw6tz=-hPZ_Su0n_Gr;;j=H=okr+TyZ00y#Pt@yu{pqOy-La zIyQ&0xRan$gE zR=<`bfVzqJ-p(&!0XKku+Dp5+z^Ns$J@2UD8{_Br;xP7-x_nGxM1FgMDf6WX@E+p;IlfLbLqPak=e2{A&e_;0m*h8&Up;*CQ9?tF|i(kR7x z+lAEQV^50PHIurbbJ;?Z9J07i6LVudR<;ihXO_|1&{$ubKjCN_AS>1m~%&?mZJZ6=+h{92b; zhk4fPSufH|&i4|80?wv%n7vozS~@4MXB@gHdk0+?Vx3~YndM}S{Oku_T;4e7eQI}JbzF9h4{{@a=&7I!HE0hoj$m$Z6EoD8aN5r$!^4YKDGqfS` z(w&=kXcfVs5-P*kB}JBcG4`4+I#+odXYFXGZVg!Aw62*E2%*8I=6PWtX?FpwY5R{=; zpX`jtRe=1xA5?9pr+d{Ss7P`4&F51)x@SNwGSQFAUt9Vi?a&3q8yWGoXTFMGXQLDW zF_{>rjO`u|Xe9a|p5g`6hFKqfiX62ezlR4h0B~wrHjAj=w;pKy@bM89(9nBoMnrG% zYd&;;p91~b$y?rFZm5+2NL?A~%9jwcb(LB=)3>$vF$W>09W}pY8FZanO6SKeoUqQT z?rrAtYVjaQBTkt-44W zj;-dcbnAO3G<`A4t%#y)Kw59u#9_csJu~+0lNI-DtPA<>@pT=ClF7e$F5S7tZM%7g zH$vnbYi?sIAHLPoS+SDQis2qsyXp3L&3L&Frf&P%1~k?9YW3W%xgCsyhIZk<06Ev; zc%YOY#rg|^aanGCyP>i)ZCxtDa*3h`^MEk=BLsn$@)476gOn4<(p3+?cj5cR#3KN~ zg#)rulfgm%bjcxtnnt{Jj!!1qH;jA79NBFlo24i{h||c}{GOC|@6`Nz zQJz8Ojqj5zJ*MOC(;t^$l(KKo_B+=Nj6DX3QvQP$+mX_Fa?3RM{eh@9_?>I&hflLm zXd77OWkJV+?>@g+1LKuGtSy2Es02KrbMJVAsG+}l?3jD1-B6t2*DpPzJzM&C<}$A- z?ZIYV8A@gF=?qDqmewzhg}@ly{~JGGfgY5r2{t~bH=CnB07}_*7$+QfCQKTHOKFtP z*v_f%je*Etd|0|WyITs>!}*$5V3zb4s!lWzDzBpfoXhMbyjr&e4LlGBdocel@Pg%I z;8me{=Z_sPTHfxsdv45!w*kbLLiF)^q`n+sWET9)(~!V~ygF8e?k4IG={8=O1!Ma^ zZ@p2;#M|;FJrLI2@q@sR2i72KkBmbt#gNJbGGZ#PsRxx-WZ*mF9;89{OO;{!#q=Y{ zCVU3ac=ub36I}RT?LubKrc(9G<25PyQ69h?Y;E9g=Mpl(V@(63Cw0)j?q>Vu=lc>< zr*0^MA%Gt;U%P`t#n=+rY1?-0)?)Cf)eIK+{oL?&ZhAn=t=uIXM+@-Xop1;hD`m4A zz9{*jfv;$ca_}q{nx5`6%@6MQsn&>Gd6!*NFPoqLC(WVM&^s-mdHX+s!qGo|k||~X zlQjXMyP^LMW(R5n0YPw}o|&&TsW(yvGHw0?P`2L5i30ph%s_BT!XI*gw4+6-bYggL zCyUhXiHKO?IGp^j>T+Sk9#ZjtNbE;c80mm+AtA-*9rzT!JOVYWO8Y1x&dG@F0OERK z;Cn1Zfq&M=cWyo*vkcTRm}-Ser8$4p!0q$d_l19!O9uW9o;g$qOg8{69JqZg;<><8 z%y6}8lsgH)f=4WHIaXEvI$pGFlDuXz+~WYo9+KW4oJ@`F3#|2=pE_9ph#rG5Qm$-2 z@u75HrNjY{rbplxTiof;oRTlE!9iSC0jQ-9RFWS;!REoQkjD8Qr?KvE0*>ge zfDsMdoSYBjGlarPfej?M-e!p_JM7A1CG5oqKvji_T+B$GKEUsCki{hS*3sZXfNfBM zg1Swf_vvP-=%BeHWNqGtJh>)i%Nr=H7@`uX1`_gE5scGjk&j+8w@y#pZf4Wg9c z2ONf308|s)`VPZDqr{6!YlFgI$|Hbyae?}i%qK-Lnw?;Ef^ZrjbB5Lb^5Pi{`yHwQ z5xi4^zqy?c770=W=-@tX2-i}eN!8Th#Yp#JDGMLN()=!FcQrP)3v;me|2j%8xH<*9 z1`v6e#R0zkbN+!h%f{4DO%RH%L$li}Pz+8YJKI7F79{q3-M_tc`FW}fB+{Td%H?%S zKdoPOI&M8g7NKyC;GAX7I7&@fgpc(9lvXQ1cJT~;ZJV(!F6}7Dv?tC6|7QXhyCtgt zFuWyK_yWwMCVt-j7n|=tp(U$E@fJh4zu5EP`XAcejK^{i+6VA>s-4o4k?pu%#}L?N z9fJc~Ka!wq8}|(>6ij^ej^G}{mocQUy;DnZg1z78KHvcU?F!H=Z3P8p6Z@-nY=0ae z1z_JtaR4yqKCAa1ugCN908E@gFeeir83LrF6M&l8ZZnrc;BmpG0`Ci3fAF0erOfD4 z`>^^eAo}^?R}aW5X7H5-fRbQGiC1uBa|B%F-Z6F04EpTBMI~WL%A$=$J?gt0x`qv- zA9a}ON1JTERf`6c^6ranu3P?s+N(V%mDQkYjZ8te5RgiTzqo&~4r5vcGWn5a_V@$< zgKEtHnN=rP0LA(9Ks`>F7PILAh_S7fwOAo`VIT=v0oNT2xUJ|FFn6K3Dq#t%$2$U5 zt<^m|hdL0I0!EdQ%u-SjY%kg72jH{`;c?8bNfP=1b87`d8Ohd1!G{;yXZ!FHWILI4 z_*LccGT^ar?i&EZl{|q{!1WQ_+=~9?`pFSZgb!0ADzx11JA{LoTX&r6m(s$tmI>zh zVFUcacY5Dd!ihq>G4=s2sTx^11R|PuF|gpTd*HHxbNNy9&UXsMjURmhG#m80V{|WW~B5jb5DfC1Nk7gFyJt-0eQUpssxiA@`C+F+Gd}j*~Uv6BtkjgF~p*Xzv&sJvbd7$UUh zo5$o?)OUs(;aCb13j17X{E54vEs%|XAqwkhae2#6YwItOa|3}NxUJDMFzvbZF{s3P zcveMa?nVDQL;F-}P=1MUEmBx|)TQ_yrsLS9e#uJ(jYfqRGNXjw0%QZ8AUOkx&V;n- z`DoQm1DtZZu-N7wx5z8cHhYpXSwBRKm*0zJmAtL1wy64s?c4oZA$vdAZs)iR3@+GO zLz+9uqqvv7rpuXq%M1ybU;ls4C4?xYuXnP&Ph7mkIr3^{@YUSS zL6-X*6O{v1BX0V0MfQf}V3=$nRCyBGeW_tzHW}t3b+S=%ia!5-IzJ&PXo`H%OVWC~aEMDa zinpTxuf=eesaJt+#hq(o??edt_gw%Jh*QJL>h=MHQ)W&V`h#2u5Egh;VzquR z^j#CklZMr)$-b`x%iSLz71=77U{)nTAF;rv0XCLImm(F!Y93pO0+6 z380Nh(mPCKJ?j=?x7tV=%h7N7smPK5RMl{j7Yo@(77BH_+Whw05P&Io+woRpVMYpF zn3B|4xiDT3p=)ekHhm6ofh2njs~^|1IXH1Y4SAv5hsZ?vbP`Go01GagdL7XNtKs3p zuo0VT2 z_XhlJbA?s@f#Iq??o6qE70;{WUE6pEW$kEVxCSB+|2fjhwq&`^F zMR@4C4Kbz)wJfbt$b}OON(IFnlP9mevDEg{JK(ulI(w~MXvmhYPp7O~^xvZ9w2$n? zN`3Lh2`|sjM&*_4R?_%LBD%4nL~EK)j{gfk*v+Riv+bOLF7K042S1i{ZnLP1=55WW zH)ZRfx5as47H%S3& zdmtCiY%6vBWcvDOpH|pwX5=%WB4cH!SrW}`@}cB7qXFa3Mbq8MZ=Eaje zYZdP)$G=KYI&>vVO;-y_kcKvedzqpqp%4#S*1R`^dV*Yreo>SILiX& zeiq^|o?+RVQ%}H=GjRVdwxW!N&Y=r{LZ94j?fy5u=u9hvSUdC&8hEk_w_t+y&Zr|6 zApZ0SkQ7V(kBsZIqqyqaBYmS{cPrQ#ngjyOJZl-@SG)>U2grgsf1_o79c=u6SpYz6 zld&nKBdaTzJmp`;N2Vt$oqn$j%T((>Oe6iK#;ntxffkU=~>^E=9qDftKBh>a7$DxGkop4GUhrx}@3vwm|Q?EHys$(nVKw z7_i|Py8hWJhg5C%1GnAq+lV&>BE88r#vgvCZ8ns*vR^o$c$qPE@&3>K-rle&q>Xw6 z{z+a|nA7av$um25+RluOY*pe^Lpu<1TH#i6vp%=8i9uhjl>E$;wC0o}!y3)J&WJ*B zuRSNf^!yvj?)bR-jWgta894Igr?*=Vfxj)q6>T3e##vB!=z~?egHJt!Ipi#q{4DK3y4h=9b{09#PaTR#^n3Jr0o%!Q=HJ6q z=v1tqoO;bg-XhMilXbK;hg`R~5=_2}w28*R4&_Z#TS?AA(7fZ!sW)lxY*WbjIeV|L z!1eFb>zIPyr*YiI3MYf}r#qM^4RKuE|3&jG$^`mj+_$vDcn|%O2WBO1aIYo{DaJm| z@FP$hH(Vt|T4BD7)MfP@Un6ENj!T{F{pNe8Jx(Fzm=0^PDw?j3887Tb0a#ft?C*^R zP|`1=ypRN^;hQ!#{PW1$qtp*{cF!@vTf10ckMh^2FUWUa6!Jw`|NPPO!(pgFD+Ft$`~n=8GQLY2xq?p&^}>sHyRR+&o&ORgPTir#B)j! zWPAT6-v0nbUvMwV@Ea;oGs;(TADD_IOz)ZH<|%4nA__cRL-q%Pu)$=me_dqmr?pn- zjoFp?^=l?2r=#IGG_T&Wy5fTho@vVJHTkz=>6}!uc(*YqrD1Vy?_TlKzwieaA(pSG zk+K#^6ut21eHB%x>QHEG4`ABuGI`jjR5Ob#^y<^OZ5+|2G25||^o0adjKaI7;db1l zS&wtGuCFDx;tJ`!TB+A^bcD|z5{#=zke)BqxZWL$yg^gcd-6^`wB$o_tdN)9?*^04 zEvYd^C<0Mps=kqB02Rr|Ny97RP>ZlFqnaK*`YOqmfkeC)eX&}RV{&z2Cg!h9C1b*! zwTqRSi>_j&Y`{{}n6R6y>6NW40Ii6Obe>wY*$10!40T&t>Y>@+Y3nmRWldWY$`jbH za;rsSpaYUe82j{k3t2MoIIlH(QYNo8WwMtp$2BL~&kCF_rTp}=|GHn^i;pIkkalSP zKtD!={ZhL(T~*G}LY{;T9)zTQLc1FcVM>~dk*|G;O)uu7M1-xK!&fY(kb@^yJVo)1 zL$P&y6)FEV7?8(`I4lj~V)tMunWQ&A9~WRO8=i1r<)UuWE;%^+tnr8%yQMSKJ+jM) z_XjJKkmuTXj$HIKeT1=5Ci`!arT7@sES01xb=G!Fw|9WEQgB^e8ED}_ zHK_adt8jHixeI0Baj%31C~Ftm$PjZsAL<8p)@`>gC*edlie=O6KxNDM+}CnA^kryi z>C5Zcv7{X(0{Qg|ovEk_-4sP2&Og!hCW6&a`H%GnPpPyG2BR0%{~$2>bDr1H!MAwf zkCu2})CZaejPP*Dhg5mBM)3ppy1hi_uvX`paK1<1MhUk^)H%nk)foE1wz*gs%IQYx z-M*g=jGuxQ%T+8Gil(vi)E(yB* znBqs=|2W@2(N8TmGX;p5oScnNMF)xbw0=IYLcDx1Z-2KynXpMn_pX%M)RLbhG%#fJ z7?3Hn8PrILH=@anb?%rN7B^Ka9+Ah6;?2MCdHIHUruEiIBR=X)YvctR07WKmC*t zhXjh(?6`(#eUjMVK%p*|riiBJMtzVk8HaMo&>VJHTXp>+dgb^9o1MHl|K4dGajRU0 zJlFsm>6;AsCsBG8nDuR%Ru@w3lXPm_f!>2kiAXUbRAvSO+GEBfTAkUyy$_KFe_P7> zSZZ(+G(>IA5-*NlHxfW~{6{GWlqKqbbR4zvN(-fLHKXbZ>l(W1T?J8wabCT|MIiG{ zuq2Ku&-96WB41M8hdkQLY1YoPv(b{Gp;Lq#^Y+#!!UrbkBx7a4o4acQf%eugc?XO9 zr0M!%++IveH3ImZ1CFq(59fFcQSP}Wrnc%Y9Ffonrnsq;7=O+ZBm{~Q3P+QrPk3Wd zBiBVyREpR^*=z#VziMyK*O!lklx-0bm5OSxV7O2v6k5_zE3#w>x}8+!L`1^cd_wC! zI6asA2$@qBk%cYZzC2=;TEu@yd!iu+?AGQOt_<2$tE7CZfgq^eaas)9>1d zSLe~f(8tIppIc$QU7^`g8ErL%<%+;B>NThKck!YdSPd8eL!A)5*IMbl{(z9p>|XLUEncOIDcAD!^0z#tcjUqU@TtfZ^- zo7G}W^~~*>7Y3J_<}3%d4;S2*;Y)N~THUE7>f~Hor3NhH7p&f0mpDs`GbdX(m}R$8 z*@=7a4?63_;ehAPxzd^ zAByBzw%7Xj5DR@SEJ+KN-#P}O@I~57Xl%nq@e{j3wsG!=!v17vL3&Lsr)r-pZ+XYn zJA#5UoJTYc{w5;o+@ZL%DK&;Ux%+1CbNP?^5&V?d{t6V zs)%=#W)r?{H$NjXDyp-E0C25ia3=Kx;bFCtKm0$l^s9h`&zo(^W6{j{l>Il zL8hlB0M+xr>W!)IJe%9nn$+uOtoPU@LN#Gb@k9ip9mbQ6L}6BG#MCIEZKec@g?S=a`mY7Y{SZZZS1)1yv`EJ zO1)&0Z3p6!9bWzO^vaM0yUN$si*0oqB7J-)K3NXW_xPJwuIbzA;vh1IujQXjaq4MW za`c{8oYtIB8`r#UPF=by zhEnYfbg%8!Gv4XVCRG9ikt$vcCe(xwLFsB$XM0d;(NR zk3j~fUtz7&CFMHnm9Lhv7W;U)Bu#jGJ*^zIbiXFHNYYMpVc2by|FwA z#v@b?HUw92tkFlXR}kjZI7?<%C3HJya`i0aMUHbgTW05UICs`o(B^(BrR0bKO&g$u zsowgcP-c2^Xt8##);9>VndkQ*Wfvf&n&(YcIjLbp>&sT#+4$jwE@4%! zlER!4yX-h51k*!b5}8_U^}UAB(Vj$P*}kQ!`BYkRC0Z(c5_aBm_~xd7Qkl(%km1AB zD{k7^ZJe5f>F?NF;%PxskP#+o=Ndjn_oL?cd|WC@l#_6A-mJD>ZsF5Gp3_2K5}P}q zLAWONeE&u#9HEkPKKPGamc~z^XvlJ@V;5AWCezwiYero5m->$==P-m3NHqk$?tx2I zR>l7`NN$qP&Vb6C)%evPM!;;Su9jW-T^ zMBlY#2c+iYgsJ4_Y1$s^9Ia^^s()B90;HpyVjI|!Sy3cVV6h)8x3nPmB#1OJ8{pmh ze{iKP^RuxCg_c)~*lKtOHhlF;Etp&IvNuUQ5T;XJzBOvgY*#}_94 z46y*Q<}Amf9sElO#CUzr&p^cnmvFl~Tgu!X zfyiXd9bmaS+8J*9;6mskv{xZ{o(;5QMpar`a8ZY@|Mf<(mn$X8DnvY%dr0Gd-%o|r z`AVD4v95kBfrKX>%>(EsbVzK*mkH~3*9wf#${dy7wFlz03*>`i-{Vu>Uo6=i!WJr8 z#(f;sgWT_{#CqNwXL&59vo@FF=04vOF&|=UWpeju8hl~k*&d+$+Nifaf9Cx7cJ1GO zOvvc=gTGFi3KdadW<=DNXeEL7021!u1IcTk0I&0z%3Mm2w@3TX{ z2Jk)JvJxV6as341QR2THkJ3kG#s41VcCa;rZQoicDUpNi_ow1(tCIfsDS#9=ha`8A zBqv-UW4&q{4Uo26Ll^%{`B7i^*(}hFz4QRM!?fZE5L9>D0Yv=(sEQmEHp~s-iCD#^ zex;e?@Q>86c(2Arao^^~6m)FKvJ}j2U(8pNXc~B#$xwWx5L-<2{o}7#T;B0D1O-K_ z@e*(mN)5TR;4zK*fPg|ALPaeTJyktiZawUQ9j%(-BB`c(4OEz(^ZNj0K8t?E{ap=4jKO~N>FA{?or!^G$rxcy9Z9R0e=I*$UMC?69HBqwRtvZQ z>0-S|d?ky1fXcYBflJ5?M{fa!?-E~Sb^YVXy;mDGKU^`a>vuj$#?pqzyI{};=d(!3 zUAuDhy|=#=ckKf1M1zl#e_E7UE=`xrXQ(6=JI<}Sm^~@?SyKJ@5vgb{DNG^ z(?lR~$h)N88~YYbal8atTK@tHV4G4D$ELHWFndR{hn9j64$b>Nx%O-nMl%YO;oR?J zfy#}4dhiCB{bYGPa9!ufEx?w40_yB4bqYJhuq&u>vONp7Rl-_JOKC?}Vg?!A9LreM!GUm~}v6W)E3 zwjh_~+iK)PHRvm8XuAdS2WqTaK?e0hy9q%D?z~Nw+thJ!RT9O#P|bd)PlmC_8dX1Q z6OqN|(E#MHFJ`I&Sx7E8jQ`Ed%PmQ*uiRo}5|Lof`wgaX(elUR6r;W{X0Uo`onbe$ zPvLtMqzgRA&G&Mdi7ixXBN7>9{GdJ$nXDfEMus|GypuXr* zT(1fAk0>bYO(3p7DM>%_S+@LokcQ$T;1X3LBTD7h{WKyAvfn)bJF;fEl-fGWX!6a| zN0z)7?tBYa)DGS9rX}jst7W)EPf`|}eKd2K#2%bs1?uJ?UqODoSt}BfCtq*B;41*w zJOeuFHD#S#{2UkGoFh$JD!^UpzYUj}C6OPw+ zV`RAudq4=6N@;jV-_;LT2to?D1gnqBRkKk~_zE6YTFQIyDVw0dX-K*g?o!x#Ln{Zh z8%x4*)*MB`%Dc!W9v0}R_SdDnhQb!*JgcRY2(5EJqeE54<=M6LFOThKKonF2qp(Yf z$S-28?7~g4{He47G~pI>+_Wr!kSy!vuo%Gid8K8J>+Sk~E^fvk-Z7?n?xT^yTjqB+KN)h`3Tb9F=dmv3R*8r06ERRBU% zZvn*L0MIe$tt{YqmFPU56!C5XX#@t?Eo{`>lZJ1qH{d3Xk!dZ&ncRfVuyP)1dX~{P zKRV~Y?+QFayeYsS)KR4lbcE~Zc5bnwlY3Lvn_FLs-1%E84Vt~Loh#3;%OXe#en=fF z24@jrzPqZaISc|$m(h9uGaHz(k8b867A+=A0N0AL$xG%NT0fa%%qI*z{6_dc`gh5~ ziml&D-f7EL*EYs^;}0@OP?b}UR5x9GK4TX`iW10pK!Js-tx7fVr9VV-a`$~d-1ua8 zK)u!u4>3IeqJwktut@%1T%t&N_XI_<_9PzeK|glkf7E%MQa$0Fx;Pxa`r=P$l0JTv z&QGRLEKT9i$f8g*GbMcg!{sd3U4emto;f=sx%)e3r8o`uC>Q26AVN>CkK?-~6BA6N zmxwlc=AV~Kzs$tC2cC_qC-z53QiLMrHGKH1Ke$!zfm}vVdL`1Mzq-%T?($&4uYn-? z`K;Eh7MU}U&3Zt+c8~18DSCRRFHim!Vy1TV!&Unc`NRh4!IMzB$C58jzqr=oCmxAi z`+RbOm#~uc_V4ff*==1V9WJS(&Nt-3$8%;sKCuymF<8L@eV$vYnMQaY^0|H+VMKO7 z%;s-WY2?S39$xcP%n2P)N)r=Sve8xjDXW9}(FfkCw<=a8u-Ylb>;Jm4_UCNY0@+BcfB zIEE7qI=kQczJgD2RZH7@EIVGO)^;M_B8=Rcz5D+3{jU#W_&(_Wp4hrVo_r<21DT_%MTz^wt>!k!u$Rq!7cMok$Y!yxFk!ZtNzO-47 z|7P`whV+rM_sLj1wi(GJEvHak1biZ4_(Q0;(LFaIei+XiFG%0z#|yf3#n(Sl6Qi3= z+bvD_uE*6!w)CDtsr+l$U|rw2xm+S1`?uV9pUlODVo`|xB?%=&@=T%rt87d6>jg3< zsRHFg2K85d!?f1Q3qB1|44WVxK^o!HQCI)gpi*>cOqivw6M2&Xm-kxm>xf>O?9W{` zQp$b9J!iGfZ84+votUdw*ap!{azD|J3puq$q1>Xh)|6h7stX5lJ2tb!a zgLaOho(sj_d9LtTJ}6LFCVDjPSVNq)!0(W^+rU_3@--USJ2nDb>_e5$&nnhPs>6!; zQPE~xzYYb9gbxXN9xYR9o>1{LMSN3!rew>sBa>+BHKzS6neg1!Gm*NhsY^FSTbNb! zsnfuv0$Zd>N^(7uJ3k5~ZSGKN?q^2dcRLEyWrefTo<_LH1P&^s?-+%imUgij{P!@o zaFOsW`e!qlpS=H+$<{A=rl4^?%KaS0sk3BJaM$OBiLYBq z3U}~Cq{rW_5ci}gj%;>^7cv^;p{UejRwXBF6hL|i7Ne3oWw}g?(aA$q?ueB)Ia9*1 zU3X+W1?HpXYfrSbYcD>|t;R&CNHltC;TmNZ{e{pnqT+o(>mrlJhS6l_K|+~&z;H%_ z!K$5xkVCbB?e<=r_u|3Q?)JNA6!u1;WR}aYz+zxKLbs!Y1D-5+bPLvTJjoY0eUA?m zbz+j44i%c)8KYJ&ZrnLVeihvWlN86V`n6S~B4YMie|XVDipqjQ!BQf_^-(x|;LJz5 zk1N~L{Z9(zkz&fxtlTe#Qah__x$?>EigK!aDlRIV%xnyL9m7jzJ2Iw71r!W z(ir2r+wHT7uop}Q37+j!l3-dswSOYnw;_|qDXa4Lafez#P7uCb`z(`1S0mM(0JF-f zQziGRRzfU4)6z>PtH>C6PtGQ@yoSpMk~MH`Eb%Yz{*9dz#;bb^_wKIMZM{xfe}+h4 zO8MR4%qf9q=vx_pZ7Fe}PyF$SlW1^a-rfYxtfVluSegNO*|{8abn}IF)e}Ef!#l>#bB)gVI|E0#D=0ygr)=j#zv} zrlPm@_-GSLl8z@j*Cpf^tP0iOQm&aqJ@Ph38tL^2O`ImhGZAj`;^ z$bHVvk^tR|yT&8pOJOum#P%j{KD&2rp2a2E7gfCyAxW~!1?x-K-?#q^5 zIElSJQKXJrlwVJhJrVh3FcwE|Tbkcp zCKN=)?v3rDCs-HLG@Ubtc3g!zH_{)zV6WRh8AY3r#PKFr<6h8?kNo%1l`}>dpBd_P z;1Df+?k5EKtZZ26iGuWjRVi#@XZL9KIQ{VsZ!u@8jx%74Cu<6Slcy^Fq6;vIiz_M--jub?PRzFcD&!*9OxG{ zq@+EH@N-1tH-S?1uL{LHPW={zy|0$uo^G#cZAWysWkLI+3RR zT0bfvt+kxk4XgarhUz0sP5IbBc-KuH-S@lS_PkFe*SZ*O_WnIMH3*oc820pb#5U7R zi1G0fK9OfywsjOvSCxAyo8VQq;kg^Z-rf3)RTv*L6J*7_F?ne{=Yfy0Fow0WmdQpv zSp3pYOXi9-e|C=bE?WO4;4dMWtaB(r9`?s)X}XTF?>_wi8xITu>k7J@i;|^Yfl3Cd zNHq-}QFFeZ6KWqnX8!mQ{f+&3cF~q68+VKdesB_xs&!B^y7TZBZMH+#ude8n8Lirr zu#HJUa~-V3+++%^9Id#{f^lHw7wrlxCHu-#^)zgw1NopJo|R#n?k%d>dZ zjpS46-1obYG409hNm@hJe-|;+qD<&nu3EQF_aXu_A{54Oao!$O-1y{`M z!dVzw3(-!ByWa1XSRvf}l-GjMzv;sUS9RXly@Qz7_0akfB-+|V3otv^aMAL3W(6SG zQfqS02&l?Ico7i|5N_`2kyADk!vCdvUC}KSy`%aghyu@Lqv-tM_HD~v5<%HP_%X4m zxse`I7N3^}@!_`{RbYq~M2i)8} zm{c1r;&X8pD?lw%J;gKhM<&~12;q%(nD_@A9D$5QB7kVWxslR`B1wF~f#8H-tdWhN zgUL_=!TFt2K~lU*1`Z=RlJ9s`g&~{ruhrV0S8N%^QB1x$J(6b6D>u4S4c~`7Cq1t6 z3V6T7i8Gj+4AeQ%4&1NcDIE3y4?6%rxf33P(iw(k;w7;D`sf>Zfz(XDCi^DUd5Hrc zZ0;vm{y2RKtcs0r+5YjmLJmney)*ClLUP&Xz7$tmS1E`o8Dcl7;^9|tybK$HmO6FU&8j79PcIJJF9F%lj?u;$TX@s@S9 zm;e2_WS}RTfATbt)J6KhA@F^YlQHw`U0~+B!W8;sb%|4m!QNmLvWkjHypBGG8adjJ zr!2fOEOMzQCUp_5K1*(xk2FGM9f>tYA2b{=qv$O!NjPX7c*_uR359q2)8Xa)ND z10WnfS^o#JH~tZ}k%IO#21SmBWhCWQXSl$~4PI$=nXv#6`c=h&Jprzx`^T??){Nc6 zQsV3Kc{ofVIDNE8@eI0KarmieD7~VHedyneim^Ruj zD!kcEAXZ;f<65Sh?KtwNg+c4!wd%vdqfqEDevmiF$N+Qyji&$SIhX!@e@=k)Hx)Fkgm5-nmpkF6 zsbrI1ORijl-|;w*E^2-?^Y_Ub2Yi+Ev7hMwvDP)e+PMP!1uk&XEyukFY|8mm2m%Au z%th-e?*(UPz3x5D^{seN7eq}G^%#{bC7&5^V)bd3*G7#SSo~s)sL~v zqC~U5&1i{upE=&&UV1G2^0@rD+%B;8k>@9z1DekP)i>3hYLf&M)UKEY#rnV8-GFHO zlP*Zj5#ElpV1aFy{W{p;GYwda;rDjteV5+S3{vC#nr9{lA83Y|Z-hZ!r#;+eQ*Ck6 z;BmP45D+gc!?U2cJ#ZY9G8w?Vt+NZ?Rw%zLIqd&40#Y!4=HDRuXM^)4Cx}pzRw8=c zl^w$*hNJ#oki%y<{RT!u)i`2{+OAjo*>E{*-*2^{;d)Q#Qof2=uMs&^-Qh9j8I-&y z^i3r=*cUyBDI`%D;`;?|)|}XpE+^WHM`=DCuEUKSgC@PlzJxdH79+r74OHlVfe~XX zZf!$cLc8UNxOc^?`6jqX2bc=Xo1@eOo=62^i-_L*A0xh~-9$0_28dm{NM54W=@UG0 zkcsFabDwF)3&e?_M@GXsgXL#U&sCcvg?{OWgRYT^!njmAP>AfFZ$a}2@3=zhBv;HM zKtSE$?&J_xVbOG{dJyR5L+uV?_)o^~uS=VN|G*xYFizdX1L;d=#q}qY{ks}yGIhj& zH|@&dc^N!^sU^n}xr-xOD*a3Sf{E3hlU7i?R6&-z4!_md|6}UAVI_O-z4v<0OV96p-#_ZpQ;+Muuj~4L zzvp=#=W!e-t9Me9`18=_cG|5J&vIS1sRBZE9drye?pajxIYlA7mjmQ!2haiBTqR`U zxETEC5f`h~LeG}mTElnNj%a?C$ew;D;cq6dO!!;M9PhBy(iLT^wkN#0y8DjwV=vEf z8GA@o?Bk|VHj;=}%8Axm1k-5O1znF;_h)YJ!YviuThf&L4_1G@%qgB1Y1*P6GoT2o zw~VD`!0b$E$r9j4^}8m|i@25BV1jx1E(yFWuC^+Lo(%x4Z~Sxt}AAGk=LD79Y|2 z{l=kgx{r~3?=Il!u*nB^e|Oxz_O>UjX9*Ldq~8?#)md$>%1OQi{982J3erzoLIay_WIkY z$`4nh^k!%x(*+mQgu>zi4HyL%L~=O5Ess%UhwhFAiv-Sk2Kz;<`zOUYKb%(M4!K#m zl*{_lf~DJccSIa$4eoSwd?+G4C2b8NoDG(Fsw`*Gfu?Z5vQT%7@DqkuiocoTTaq|Z zG0MQq@WYWI)VnhY0+E%6vU1`pO-Q_b!Fh@mYaS+&#&>OQk)^K5fhg6yg#)sCF+rK(P8(lC%`jFz~tsidlZ5P&}h{L<5(5$R5 zc%7%{3@-&?u&Qz5Sk9S;N;6bz{zSq7n~`zlZ;%E$o0Lw*U(|9tLe;r#MDIEgnSv7_ zW7^h=ii+E&v6ErCVxCGkl-+PWlVpQi-Kl1GYCWacUzR!RipHWcn&GiAk+5A@+|_G6 zEDaWPprR9(OBK1bA&JMrEH8R?cfTGjAbtj^UUsE@+RK1)bHn?^9TTRuZ|DiZcy}7< z^sxYm9L$Jci`XCCj;Bke=MAKate}hA#T>NBFJFvq1mWRw=Z=}b!g)d600jc6v_Uc- z5kK0JyP}I9e6CvEK&{aJS}WeuB^LPlUMpqi=8!1%0smzvFdVHy2_F(_XNp~q_!W_S z85@vASUA)!+VJ$i@=S4 z8KV9p6r|g3wXgOC@6HF{r10f4$x9Y^A0G`l#Fz(6;WjW+jGlA5b72fwq_lX6TElid zWlq1a^ip1BTgR??^E#z#1f=u}1gC%eT`JysL8~l%S$=j8`%~?Sq9zlDvCb^js}`40 z=p3Fmv(dym9ORV_pbLCZonv~}8idB{4}h4lDO=cvGQANR5y1K3^W4=k07gqosRxmB z_8L@7eP>p9y{=7MIkwcJkQN$O{+a2fU{{$&YTjET6@t*if^Q2+qwTcCPc1~2) zc9~bL8^*tG0v$w~OiPi8_T^=+Gd|B%vtu0c+Gcf8+yB|@CfJv4E*lt^7dD^jB-42p ztQj193`Mq7+BmGYOO&-%<6-lDrMa$s=GmLFKOBnlF_b@ytqP(ryG&;R6qALp)V!q!qz|qZQEYGddvc&daUqYC!D%U zB7X3+R`(DXcImSzoKiXhguZ)BJ}v_IlxG~&#FdKFqaxGSXn)-fR|scC1v!KT(JD)^ zv6(a9x9?C>*FZO5+KTt67=_Lw&oc#H+-xM2<6yv*k?+w@sjxBI*;J?qTWkgk%f?hf zCQO`e_`>F20O_mAG+nKIaYh+1oXvkIXG-l7szhWy6!SD+H9!|L;q4ukGL>Hy>PAxu zCeF(#xFTCO6lYlxP4N$L@EQKBmB;m1oT3KhTN{D)eb3DOi?0*gwGs2UB+z0eK%SVz za_xN%Zq#)`-?J=r@~Lu7HwGlt^Xrip_^>U{7%q6S_357ZG`ExZR_k9)e)#;NhbSzm z>reBdC}CDdc5^!=UQc`3_x1Pb?h>0szOgKj7XB#gOm38eJ1E@0pHG3OEDJaLuZ)-Y zsfK_BA7$Y;!;rtZz`WSM98CMmuC>L1uPK?W>rSYT*xv2ILkkaLloiwY>R}6?j5|_k zD74v`$Am&U%E+o@Y;7MCJA;U=Jpc#)Sk9^WW9_8`G?%OQnB^V%LZHi96fT^ov6)$l zxpEV4jh54CJ8$(S1_+15PaPG((Z~0?{^yu?B^J4uD@cTc6%gNwGyGrm-K2hU3wHO`((KI4nVg>8Hmo-hvb2NHh? zS4`b#Hdq9ZMBP*8Kz(+apQ71-QjRk3p@s4r%q^t8b(aa7D%X>R-_|qSa>$G}?$r*f zi{n{%oWYX|o7;}Wi--yUfW`=;hM;7E$Stq#??U#%;lvrj;*Ah7WalqUlp?^ymE;w43&UQ()hMGESFm%p&X8Xc2>s+!Q^4;iC#gf zb!jheW?2lf^ET1kjJaKNdf_IqcXnpd!v~PqVtPJPFPIEW2A{T>gO*N}P$jlo-=zZlm+MtZF2Ln+U=+!lQwe9< zB5x)>g~Vdk+lAP}F~@{#2N-eHNcb&f(7h+jMH+dH^O*{4OC`p-<@-GHH`>puGn=4S zd|8n%>482=#a)Mtk2pbDJTgXjT8tFCKj>Gp%_77(=RR|`GO%}zC8aF^CMYegavm3! zK>k-H)a-*WQ&Iikd2T?3%m`%BPC)avhb7kIsU+&7t)^5;>zs0r5|0dzdZ5h&IiPno zyA7kc08>EwO0hge3O*2~b@!LT*=60P6+}U4IoqBdNNHi=^HRlMf(JI5?n^Hv%SNCo zj{b&Bzxc=P3wU&4bwF2K{1!$x4;1J?7ug@@VME$4IA&vJ6j1P2@^3$WE_w;8u4q8G zLm6F6hX>@}aIPwd0nwWFU^B4KhrV}m21vsmT;1cdYKJ3xQ5C2}zqoapq!@3YlVFsC z!G*~5gjsTSWTog}>KyLDAK(a%pMK}X5^a)xjtRyyLhL0QZa^qyeSZ+f%BYin_e>BO zOYW=?fPdhg6kil>97dTg9<9p*S)Jfx6yj5;^!Ys>Q`;M~2YeS2y>T$pik&tK+>#-> zRu(Sjo={vsHFz05lz<6XojmfwXb;WT=0KxJoL!2vmr7G6qZ0Ed=kkWIp7+;;p%a>0 z?l+YQAe@~ez+I;{EpZFG)F?9oCgRA%gd%rf>xJ}6lw2)C(?X_Tunp`e*$FdxcQWHf zbce_sS8{+vFWJa;-j~o%;-@UU$>BgymXbA>N)thkeDOW0cE9=)F2!i5*uh4r8gB7> zrh66Df*7R0(czp5WZf1Uu!Gn*{(;!j8VhLU{~*R=@Y;D(3Cm^x{2HdgWP?94y<)aP zLQUa3f9Qu(X)|}x6UYV^oa4yp`v7sx`hK})0;ZW3ujsc>UcG~3PG+nBI~MK>k>>A%iuq6~Vg{hMIJZ|Nx}NlI z*Cpj6uMwtQQAAIma5JbMuWMvIZH>D?wUuNd9z2dA|`FwdM^Djz2S zTh-UQIq`pjKM-*i-!1WKkyBy_69-h|Nr>?3he=(JeQ4Z07_RYTyTc|fVQ7_Off?n0GXI;BfJbM$kTkEBsjUGP68!FzuqN zYdlW^OG#!fq=LA3d&C4r@YW83eO$xNM0l{^RBXZM2v>Ehv4-HVSx11|($c-XcPB5@ zDkt*gw2uP>)CI!E2VMK+!~89UCR;HVQUHanw*li!q@0WAM5#4X}A_>_y_{@8`3 ztRq@_kSy%7Wlk!(5us<9U*bXkI0DHZUtbj3hX}Z{UlAVqX>kYa3G9km#oRYV zsvf(t`#DyA69rJo1;Zm^8D+%+EArU#t8appCFi2<3!GZTqo$Qw&sUmSc3`dgpyVAq zf=;sSLe*5UBkwj?qlVR4epXBoTO_59#U;Zx%oRF@7I60se~B%|^Cr3_AG5tVfH1Ly zK!k`VsxVl4A7p|TY1<&!gmmpGoD-yen`MIJ9umTgb@HqXCqbARiTcm66RQhwHadY_ z%}aM+;^xg2;h8z0#xK0Ze@hguz#{qbpumCUZV$~D-mZHZu6Wh}W^Nfgk7cX6ke=YE z@G2r>l*;(j$FD1&CvU7ZFO7;=v#9pnycX#EHpF^MvW@nAOxZZuh+lYkoi2^*%8MeA zDIKfnh5F(wS_e|oC;{`Ly`FJ>5%8GRT5nn|a@ab1o1>r(FW9x-gLYkb^qAj|BB|+P z`u%tn4}sGvW4m7+_I*Ccw<7*+Hm#!Dj!@e@5q+U?(b@-;7snL4$M;)-EUc;dh>pyt zE#~Mk&L-ILfO)w~5`(`9qHO>uxkO3uB+w0Hb@8$r1|clb;SNEv zXPqYyPgpSCDcR#?&}ZY+^Tj#-xKmIZ{zL0rrk-9f⁡n?vl4B0!}pH zVExjitMl2$mI%dYva={5A=#(U5+DJb+yPty+L1Es51r8?1cI(LKYf?V|I2TqyWbv% z7X&f*z3U18X>8OY)_M-V_@~IISjsh#(*z{6+H?xfa01TWWP58S@;@&CsmLc!0l_pl zbrg7(UZjU%T|9s|oAR{ImDiz-nJ=*uw6;#o+SkI}-@O*?Rt(H2d6LYR;!5$MU(kK2 zXxKm5for$t#)$0Ugo*T6lmGLFkAs4VCE`P^Cwcu)hk^o3q|>7*Zi^yS-*{e^gnE|S zg^IBdGYg6BSe5(x^*CGM4=-4cQ#XkSFC<*T414X25{O#{Y)RL(0$xb(ZE4tdbj*x7 zKEoRmduKL5Y6Jij`#@UzAcn3H`6hfeNNyO#(cDsQ5M_nsju6!HAOPn%hsIN$qo3IQ z`P{HiVRc~qHjT^~scpk+LXug333=u^7jh<1st7N@f2#ys<>%5rv&rLx*GT%Eu0Y_I zlqw&-D1>27gKPDu9b3X<@m;gff>SczZ+SGZE-?L{exQy>xpvfJE4kQ44?=fe=}xKX zEY?xtC0IIqQ)w{MW6u8sOl`BPcvyCwpm6TehZ@WGYf9yDc>jp}ene9yI+QC30nZUK z>MWD!-D}i5%^nvNC7B|Ed_=~`AF92@taQFbNsEzDL1W?m0nlB3KEcl@fs;Bw24^g^ z(J&~GLp5)~sW3AAQ)p4QK!*Z4h8T&rOLoX_A~^BLx}-xqh>uK5nCiYL8^DZ5+Lf-l~B^tZV4t?OSt6(I4D4BCJ7 zA8xAu*%GvR2Dx6QoX_wFuyIfbTl2iu+OFgh7ej#;fYekfm2npcjrWuIr7ehj)tMRXi(>{q$O+)$ zYEe#&D@1Xjes%_)4@4lUladc6>%7m^(yf-Vzqe&)&VYaO0s0Z1uS!Cj)NOdjj3_{N zUe=O3l_vH*LcOG#+-CZI0dIqDt@ik^RrzMuyV8lU#9&336TjK14VD{iVo0cev`X&9<1>BGJWWl*MJ1k8PPeFAs7-T}b!XSzh6)$IdL+%m7lRVpuP|Ot?$m<;j#@&1;{*^c*WhbB#6W*I` zh5BnkKZKyr@q(VM&V9{2Lil3pD6ga4C*3*Gmz5vq@7KH3LR$P+xI-QtLxe1c1>4BC zf(v|rqvD}9Xh^{91hYedD;BZC-7zA0L{F$EA)S)GJ1=a`fr3FSDe2Jv?e4D zS?ssLu>uRz-OI2>rwYnk?Rp3@IG;t-KUOeBHWC7gX`+Jkz6a)IiE1%@|Aos$76C;P zzkpC#+1rTYsfYbTPSg2fDcNqlWf(3>3IW?&eDcF?#($IC)?-{CtH{JsuKi(~X<6<( zEjuZgST+=0gY*VrwU~Ri)crPZQnY5mF!_*~rD)+HOHLd!QhnK&FICkqQWb8uwME=OQv8O;grE5x^oV!Ngf)Ry{C-6M_{Ic$42RVOqam z&tiQ5vovhL!%eY5_jn>}PTTfpCwHDV(YhWppbrY$2X7-pe-#EY$&$H-J^tvO5sA4% zaq@cl|L^N*vn~5G`0-mdBaWa&5wrTID}w1zLCkQ~9z>}T3(Mu-_2xVV`c+T6WP@H;I2;mOb*08R;ds_Z;>13`b8w~R+-F40DZnfymaTR(A)-)W3=v_ zh^vtNUFk?3eU91mgm>`g0=(bNMvct#__(jNT9>6^>09xKG`H98Au|h{;6iVq zq1ZV9vvnr9JfNL!gCs3Tw9|uj<{m6ns0vuJr2^D%^UInF+n*oe@~j54FO`s3vII2gIP6sy2-uG`RT{WPFgPD>U=q> z@D685j4=G`AY${oP6u91j;gnl@^=m65wtrJkJfxNd2roUBy!fc2W{L7bs zp=y3FyC4ZX)qb1crog9%cMIom+R=e8(n+WMGf)kFY%UKE)4@!_E@NVc6PeuQmur~&7I%%Kb2WE}FL)RTnvdvi!0 zhh*MdecS{tYi;z+^N??s`;iDWPv>MBYH$!(FMUv@ia9f;6hO5Xu@5ws`QTxMj1rw51L%uwIVfJg zTSHa91kKW|X{4Y+=GTbiSZ(%*g|RJCgR=u$@OQt!>)5u_3bS_$UbowQp3PdHSOH)A zslvdQDjkH&ZRKqw3fhmr!|8T4*Mh?y1()s|GV`iRU=jElbJdUP*LD{X=lvmk$_~op z2)@I_7dqO3VaaS|oT#9cW)i$N(^quuu0!$!@Hh$qk?=OP{(irMZ``QRp+%Y9k7O-R zP<#dLzkTvPKVZaRuXBfYaQ$asrOSZtJ7fZU15@yh#7>7P^Q2Dd5 zvObE^)|**}mRVLG<(N>Sr-h%7Qhp$b!RCz+_C^|(CSTPe|P_@0LHBr?HJ^)?J_D<0D8YC&&faz#kd zyFc-T^HXAtDgWwxGdXR`@l*O@e`*(h;KaLy@%LL%8|1(dgY+V7zgMAyRzNcMB)J&> zc#>SeP7ixwWQlrr*5Enq;-4IlK+El&ektY4$k(w1p2iz8m%DW$#X=by@-{%Fxmw5( z+}Yc6h)GhKp04&|Gv{P_%tjyIz@8a~@KD^n$OUd(SFF`=yvS!3k(#3FRkabctXQS| zK`q7ARLYolk-j>pdJkgRp2$K|D9weBHlG}t?`6GxJrdAgVtHQxyJB=FcZo?#nK2ri z_<}nfKUIy=iz>t2d+jeyBfzON>xigQRVOPCub>y_$z5~RzMRJFLTYBT>{UG0TPpd= z%ADOAqWBejL`79^Pr`t9h3 z;_868gY{rTSeSJ*+6eZil_vx_jY(I=-g~;Lhj4rfO11HV znVW7=8;}AkC2P@)RluO7-GC6Zq7P>L+yJA(_*Y{MB|WwpHX)utx+o6i?485ab=_^S zNX{$=#G=RXHm?WP$@1v<`yc9HOF;KR#ZZBQ>8>-R zIzW?*v=t^6q5l}h1m}!#;B3O_(eHjq0l)cOL19LTzf=XwJY`_f`p6A^7QLjYXXMxAYEK ztoZX7{##6@%X_%rPLw@yV`hR z5a|-$uO`X))u}n;{`o4Z1W2>dm5!(zC9S-)bq2}A}Y{@^{tk!&AOthTx zOs#8HUo<^=VmK z&|5f~8GoQl`v(b=+l6Flgt7t(xtglfk1>D1ja~2R#i6uk_zGS2*J; z_cS_w8@3+ z*cY?^9lzqW%!+ZXl<{T4M;v-4e({|bE{<$Maky$hFJg~sq*z^3PSbW1|ltKqMZM9o~_O$1BgL$RWk+dNDWEGT3G zJFU4MiRR*=M<1u9TiX}R*+P*6 zoB}z(>}=Pcb-%qXxFnGS8PvbxY)8Vl%iOaHinA_U8O`?Z?MO9}4~a0z*eA{jNEn?E ztgRBNCr{D$awIyRVSc!qI)BYqcfFamBwlX-WK^|3fVQV*8LYj3Ye5a7+@CH_TC3~y zj-b%07Dy+#1f0j$nrS|YS7jHPwDbVtJTMx_u!JP&E+BBNgz-oX)j$eK!CEgIWy}-M zK9$gH%9zh6p-eBf=R8E1vmuAGq3@FM%PkJq&+$1l#M{++U&-5z+&| z+nsDh2&gL~CXG);*yD69*u8&WU3t%{l3?e%ARXtqJ=mQOJq;4Qk55`tmi#GHo}QfJ zb}8EnR3acj4xUfX^KFk)FOnICDjh$V}k7@jCBbY_S516DfN5KK{>@&GB<2H|KN z;Ax^AhkWZsl*Jq&Qw3zg`)+MIDhwlC2cbTgpog&q+W>Cw@5B5gMS(}nm{fwONZBUL zX*@eXXu57t!clIz2JiVZ zk{AM~M9+(~@TU`t+-xe+wkR`)V%%IpCVd?Q_p@%>ON(IMQZ(mK2x;_Zq#lqbmTlIst?D<$PSI z8RX!EV~&uw-&k)IcY$qM1u*(BG9P04I?ydt6_$Xwh%#8njSb`gW9o8zVd$rYky1$4<>Z9958#;hv@ z9WV9}Oj_KOx+83uB%>QDktM=HWSY_vv}hc&lk8ux87J_P=)4%YNI3c#3WZ99#mZxP z48d}}yPUL=JjxdAW;Z*3GF8Lu&+}znT6(VP=a^T-k9Wf>A@=8*N;#{DA8x=}YNf-vhYbfshE`0rIy>txKdn4KxCEw?=l_!M3S#0q3wfqUi?+^rg zhRK(s3uuL(qXW1DSdZXT9|S{|1;V9kAFDc)MX|CqfMTS)dma%e6Zk$E%ziw)C|SmO zXZuR>y6NoWrU~CL+7F!1FJ6|k(IVQd8o4g}9&_M%Hee!r>G5TB}Wniqrc3b#j(@7V+t_d5!xVb2+{vQZh-aA`i_wL zvo{Q}qXA+>2Jf-y)aK>is4aJdb~z;_$fX_NtXeV1iojmRAc^9eQW(hcED zIwcCpN?v8gzGsfeH9B+g&QI-Tllq<~K}W@>S+23v1fD%5UM1TeC!39RMr?nUO)gMd2yQA$Px+hR8kUi~1fW$Ya+S>CUDh zw8)5ib#bZQMWy@-vBn6~AI4Lwe5T`jAdXP0PM+T|eSEn1X}Ii^TtvE3NZ`v$9C-A_{#rie@)sPlmftI56UVH7ILNpY8rI$XDjY#F zOC(4gHM=3)D%QEae!1aArZps|b<4v?E~{2ut~Lw3!I9$Mp^k-xM&J^?a%r zh@9QH065ztjp;sl_3Jch>X}U5FPN2XTuxyP*)On$YSGK$=Hhs-Kdbc0`$xBLJ(`48 z-14~n9#Q@dK6^;_FaV0} zXTlI_Y>Ny9@v>vj3!FYpYB*L&PVOTc%}6J0&Mp^G@K+$h)1R?`{ScFY%yulH&T3xw z_!Lh$^gd!lB=5k0X_V>d>a$B)+#$x*kcOowT)KSy%_YVOc8ZrP@j%g=ss!5Smdnb|yD`6%fzp`^Eor4uzC63~*WHxyFRsPS&cESDWefHt*FJ zyDJ%bG#sQnM)iBaIfT!ZY@77R$>RbNFqKU$6G`Q7tJacDNg7A`>p1RhZTzRjHBG{?R8=Fyl1djNUL>u* zlsGr7hW#)LDpO2>uUau6C^U@!e=QvmyZA#V{D#F#wU(fp8V{+6V_}8E@wyW~=@-pz zh4d+pByeo`s|pzdIlR&ez7ybKV~MG_)3_N*Jl>B3^YdrS+Ffl|T4TbfQ}&UXzg~Zm z(9o7MX^JrT(>f-oSV*^ex}g(T%~O{Ay+ue4Xx<(1QR(>k0CB*l;NUYCgV@+NB~G(o zb)`t8Za;sP{U}P_z;fg458H)X?->?$7&@|5Q)L#YuU+$^Bh{N&GgWguTBvB>rYk3p8}c`sFw z`n`=`TBIpZg)JdT*TVkx@-l(P&M;E)G1B1@vu2odCSLn)yZ$%kojV{e2(wBJ^(aZ; zOeAm2S=@DcMHaH#r{zB+zIJJCztw|n&h$(Q#P5hO-n=Z=9?;b|9>N1bUC0q z(d}gui{68;_Otg{Gm`AaarBCDqrYLSUc7EOa^4qxJ+j_|at-Vli+SKu%B=75= zUTN7RVO3hX4XAVmBfeSLjV6$qgxDvuZ$@ZFOj@~K=!@!>9bXIc_PBe0-WyQFmkRxuJISxmlitJ3^^%)K>`n4J)AbTT9JMrEhk8 z5=0l=J|vT;ZBNqUO}_A{7R_p#C28Yp&m{PQ4qBO~80RSA?av{M)6aE1H32;#^;?U2 zl@>^|mLA=_lSlC4YPte&`}lwZvNPKtYjOW#AGt~^r?mU}z&DVlakS3e*>;wx8!IU_ zd1bA8epQ!{j1giHlVBp}FK94g@Jta_T(8@M#q%e5F?@AN6cx8oTevcGkiKfa-0gU9 z((ic{p!HLO-`?UO$aB!%zFphsj8e!cM zIM2Y9%@QZ3{f=yiV4_-lW-NSWzQo91J=;wlPCYe$8*6-&lzlPri~Er$dHH3Qv^oRj z=U0U-baGJ=}I1~#q(@I8}hZah}1iHR$Pn>k1Ai_|}e zAk+rXj_Z!X>pkTWw7=4iI&=Vp8)JmUbIUUaIY{8vAdHIQouyY;3#)xRe-;j%v;101P&V)#rt65%p{*5G)U~c2{khXLSkSB7Tyl0oHiIPOy ze_#;~u4~gNge4|}?y;3xF7ps0@!Z>3)tEyDP$5;Jz|#>47w7;?7V(_$0l>KO_|gQ_ zP!84Z+t+1YzxW#zL7QoPynK9E`}U^VB60$-p|2F&R}s4UAKs1!+tnWHy99H-KpGey|MKX-`j# z`{v(WKMuskFRH!7owrC@D;TQH43Yf(<`0UceEOoz^2g?}t$k0rzpiJ9fa&MjLJGEq zdoZjbZaaR{Z_uQBabV5gsWE)qb-}HiExaV$v0<*DdUAJsj=_78`pMbPR4ETrR^oZT zx_`B#)sX9OT9V#jc%BlJ*_*=mq5m-U_fVL%(G{g4ldaYH(VN#^i#+^1cpUB>f z05^m0)Y4BJBgZ4iL+(Y1qsC+|PFq3OeeF>}a#3L2nbz9fDHe7gz#wX2wZK=755S2IAR)pt4|^W zsz+?89vz?-wLgk4zFU3-?ABc&j^yGM{bT5t3LCFS*%)^#R>BnHK|s^gtJzY*eFG7y z^swqa+0v6rXc!D2=yJF;bAn}(G^K8D?6_J0dX}(w_rYm+0>e0$CegtKv=dV`2La&_dTOVk7>sWbmC}<-6 z_wycf4mac!^k^vPI(fT*DC=7XxXFk(rlnoCL>NJ&Kb-cF zPEHP)3|pzzS8SR_#aOGpZWNDWy;=gQUQCBiTmG1@J=)9LE@=F5VT{w@zQb?xRw&#% z%r4Sc7poc5FJ;$2$)A7wauns6?)%yg1Gv8rQ(Tt#%{&oGWR5VaBWPQnbaN()jU8X^ z|McU!o@goGnIe;_^2MI5QKvgZ+s>W{yK=!`t=EYqcYNqgMpNfP)!&k=3B^hVr6JF4 z{q8bbLjjPwyVGs5^`mF1Qx}gSo&OPcI5lkZ@?jAHX9g+8dihF}uCQM62Va1K@TIdk zteXB3k1^b9S8tptiDX9T$o*jKcpxQYeJ|hzCnH{u7Q%eR)zccd8~{AZ;g<(oHX&WS za7KN4;e+S#sN_jT96Z_pIUJL_2>sd+-6J0f6b9~g;96Y6i+G^)1v%30S|NvX52xll zaY<`P4%$@{=3Td2hF!8FG=V#|Y&4nRBR5}+gK;kkI0KVHq zs!OHCoE*Ys>!WC%mLD=2qzE4gPEh37ye!Q~yYqVCEje84%6a&ZJ6(eoR>APIx?{iZ zcOvB)T&9r3JXeVFiXlC}vaHgszeFjftxgWXCms>tQjR04G4RCO5%`<6Q$jrl ztaBMA^kxXpy-Po^TOO(&gwbPj1Q!s)O4axr zUiuk!3|>Qr7n~!WN{e_m?J$ zXZJ1~$8KDfTg_l1Hc@`vTofL?Uih2aCp4FnH=`Sj!yu2~!eUXn!Ra;^@MI+vnT=Pxl19f|@P)#!)`I1A)6LR#U`gIJxH~ssADQlSx>)o@eWe_W)uM0l z+`sGkZb|uUCyFTa5m|S!pgx5ndV)dm4i)qtBHZfj`GAZTb|E}$;1ZNJHn}-jYs#jz z0BHTNEDodNvM&xZD!s2(B|~Kz4PP)q6~$K`+3FSj*E$3T-@B@RW8X|KN%J$N86R$C zz3oni8oHPs$z&vR=q{Ai{=~;_wYJqt3j5 z<@?O}ec~m^Dw-YmP6w$3ND!lbx;sMrkL@-iPSSN#%S`?l{NDk&iRA!Lm1UNaQ1^T?D> z#DB{v;&kuFEf@~zh!8*K|K`801bBC_;9>m$Vv!*Cl4|t%*^6ULrj zq)%>2iOJFjzd`g*tf0$cX|3v|FqRps+d@(K@bfF%edM_RsBSxr`7+8gjkx@J?r-Hs zHONt>b2XJB*3yHpe;zog6j(wp3cPOc)YY zv#~f5zX&((0JEB1W`3Q1s5m@+jt8XZE{I120v97~F&o&OHt;~Vcgs(ci9q*c&ts$q zS|_RDRiTQA@du#zlLp?0PKZhNiL=CT2jFwt@B!F0J`QPcQIz`~R!<=CB=?T~|8_7j zW+eGJZUEDX><`H%IbDwPHYX(^pLg98vYJJ}%hEw-y;MhVrv9wV-`ju?o}6YoG7}fU zSjXP#O?;f0iTQ)@>BgFd{TRmu;WkjCSEG;^qN-sQ>w1@z}8E;DbM2&azL&XXM zyz7X*zWTgc+@Mmn2p~J?odxt?MFbcno5t#Rh%N1cBxV316jeL9rb<8U5c^>F+eySP z6z4R4*eZ1jWP2i7B1C_LL4b9_2r~rCdAn+wEkrH!;*D>dfc_l`xni2zkQ@#h_H_!- z?AEX+wuq4ck)GsFF5vlfaWKp0-JMwWVURqKPp#EbIHA-d+syl=%DPJZtrLo`4Qj(X zZQ$_Z-#7x?nkP&TFwIoN5tNn7OkWTt5kZ1;GgDwW^ohkvzQ{KQskzII3GhuPD85#q znlN5Az=0_lLFXc;>&weQEjCuAbBc{u19YBZH4whAvrt zPP_(n2Wa*0AWck|I&fNcu`<>~@aSAw>nbhlY1XfN)42Gj&y^Kg=uD9n9E)PB#=ieZ zd#{z(>QScrfsxMKAbpEBK_pK7e+R|{0d6xUYjCDIHdQP-ODZiNLCf`+^6Ud3Frh`| zi!u|6bJ*~}TgMS_q<36vK(af>qpSKPV4sZt%U_8Qf!KOTV{M-!Eyu2*e>8#SGgaAk8XE zNVinUm2mXRpf55m)QijhPkov=T>GykcbmpO_610| zGl9TiOX~CZ+{2P}qywUiv@~FyEZvg5SAiJNT62EnUuI8 z_?nc&#Jd#uD-z`53tlOp7c|j51m(&sL}0CjZ@EqtGHH5cHa6g9;0O`;b91}M7UR&( zDbd{7K-UGy*$2+3jY`gainJ{@wk7+SFY@8^I+mOYh&DoqA-N^*Zs1?H=JA~S&S!6Xl!^?XUO5$p++RT z)FNgT;zG*)%rQNkQ|7KpM_kAVq4lOwzqag>u`Ro9u_#Bi#DA&;KoFxHm4mWMLO)qz6MpGF5iI z;{mK`H|DfcB6Go6E#!E;(h~k=TYDLte2KU0jP=gt@dilfV+wSxGRx27J!v}kmWq?k zuhQZ?UcRhE=X1S~v?VTPJZ!iUjER=DO6+5Wvx0~WW7i=y;`kcVOo%tSZG}yu*vx!P zL1xqltEl-Zk_q?SP;R@ks-1`v_36KT_5m{k9JTN}nO$=ZUDoPa=wf#GAN$W4oWG6| zqQxb9OE+|C@hw6@|8sp4JP+E)5hM#$@^Ya5j{iuPTf(CZ=?rlGLJomqYH-vfC+o1F z(uAz|9Nu2)-FHj(OppOOGw#m33TF~lKwNqLlfK10M9-Nx?o|w*I)M5g3qcBA*OXu% zqZ%7r42)Tla8HiD3%(Jw#Ni?RWA>$1kX>DUQr|WAfYAuQ5V&X|OSiSuxgs7th`IVN zG1`@`6Oq4}FXrvZ78)R-2go$7wPsBV;8$rTWNq74O=H6bCD*3R+?n#-bf48B`Km32 zU?2I0WB9+Dfah}eKFG^o#cAs^X+D%<+(yR>$S={nJq`Z?l8fec8SOK#dMX^AWXGom z%yoRdITdH{x;zKp^&iEp`acZPzP(mj^{duG$%ipY~iGVLzOn{%P&f&uO5rci!$3H2#Z zLLtb_UJ5^>cRoYQVgr~`6^PUDtIQc^rZ#7^QX=2?(NPZj;ze$oqNvqeo@KQy?IB1Vm$}$xmo?aWJDMMJtuATsiFJ*KYS!WMd2P7`k=~S8JhokFibwAo4e#4H<~*? z1-P9gl`(P8UpCS@C;hKgeh-zEWZ&|~@-yr(2|;91jo`k1{&iA75p`3v;_@=#2(f#? zp#MRvRnIa(YdBk#Au`={zP!BV#V<;EndyI7#4PC6k*KtP#cnir8~KfOl|E@ z5}9EK+TC)F_LzJ;vwc78jpjtnh2!wpRXQ(%@!(Ngsb?^Mrom9Ub$E%z$umd$Zpr$D zpI8Y>;lNer1i@2K4t7`Je%xP#F24kdCQ)!k96^AN9~lHWYY*k^mmok0-t3y7}6t4yKjPA+rT8Pu!zqYU4o{Fm{f7@ska6 zK6|j4qsbMqH}dYkxXUPvRFNl~LAV{D#t75pKbu1|+r(t(+9&nn{O{Fe%WmMk$NS*> z!TGd)*G(c{L+`^br}@9|yo>Js(0*Jk!TXmGVNtm>3l5P*k+*6PG+9yF3<#@TyLp(K%w=i*2&=Y9;ar zY`UQDN>g%Hc<>Tx?U}^P0tNlFtoLm1ExZGUCpVv-932g``al|y4SQVTXEECt;ukJo zEn)E{!%~W|61~!o2vxWR@eroX>Ep|Rvg}CdymAcjn!VB>6!^o)Vf26*eP;L%Oi^zc zglHFFn@t^Wi1IokK`I_cv?#(m!vQX;7mln)@a*_m* zy=`0`NBAK5QMRoczfSktHAtY;B)g$mjS!L=JqYTD$cp~9~qo?H52|V zur@WcV(k}By44uJv+whI%`!FNGW3X@*iT~O}cOtl3{hztBOOy4dxeUA7R0R_e}RBILC z;y|Wb(eCl%J2OD&7_%FHDZy?|g8{S$j{KZ{=_C#heWBCcF8T`#AuHCwpoT;WG zb-u_f--F^_Sm@bh8=}8G5t~V6yu(UXXO} znxuqGp?6=~w*oPb?z8dE=4;sEEHDNY=%_<7=t$g6F|pV zL|92kAMak#nC}iuJ|6^NuM?ofpD@P2`lt`emu1D@h<_gxM74>jeyih6?#qoV;Ub?T z_YkKdq=;30JL(!VQt={t9Hv@K{55d5-iB&+XCqu}uQ}Ql?B-S`Z%p=0AcYhZh7O31 zo$cV|BiU^jk5ew@xmS*ec-vE05z^I(^ab%I`~XYZg0kuYC^}UUy4S7{smO|i_abn0 z91uzN_O}6G0ebeq&yK_bx6JT@KjO-u#o0unrDuIpUTosikE7$Q->3(_Wc0Y`=0n{bFe1jELF|wI&{l8`1Afc;%FNg;bJw9*>c`3^d*h z3wptPopA}w2|wveQ2tucJb3M}@4UUZ)PE;g!&X#vxaeMTZtYO*{+6KOv8GeW1*$ks zy{T|O4Yk?YS7Fu=;?3#!s$lgz>yg3(&D2X#xdwz4W9)@pQEyEGN$C3KGA&ya$CA9Q zk;)d~O(B)O;hy+g{zdKez^kwNKTtkW_c?7hKZ#{6;N8RHxs=QmU6NLVf6+*8+{dee*ezVw^{DJiB}NXPvXPw0C_6F&qe}I!IA` z8KZ43$wDsX(rbDl`<=v{;lm{tPgI&rmFV{WA?&TAqHMeHQ5Z)ZKy(;N5u{r}1tbK9 zZj>$s#Gs{7no&SPK}w{%1?f^k1r?-2N&Oh&By}~@t+_B@@ z*S_}GKA#n@aTi<}4WIEO98G@C$Tf(qOD9af_80t2Xgq)hmDc3JPK2u%s8q?DuBVcb zn?Wy7H_LL_WA*kH%$4t~&Sf6w;yIZVeQdFWytFe7wz!nj{z?CKCJq1)8TO17Zgli zt3-3uLih0qA_{8yL=ud;&mnXy2no<3G8Lu)d_w1aV7^c*Uw|9fEAfzioUw3Y&d5Oj zF=T%CDOIkxr^CiJg^VIwKo!-OTVQuqOVel|C8-OXk6mhAhx-LVc=EJ_S@Wpw)5 zZ5vK*e@+Pw3RkFtOL?v%BB&+`AxJ|_tp$!4Y14Q%==vm3k@9eO-KKHq!^47vyLL;ru8sy}d>u1FtbEzdIG$+5oAY;Qf~j9cKz|ZY{AZG`1Z+ zr^+v=)&8skE4$SW{SWrdJmPRL<;E)AgxnKE$lG#au%JljqRK7+F+NkW}wuGb$SoEoV3f!rjJV18g0rLhT zq`RYawRNX3eS_}ay*Z%1u0AYbP{UUCms?)F4>U#zjWgZV4&xc)estPWZu7cZFd%9? ztO9pOc$G(b2|yXdPi;jO)m7_%!#%x|xH=Nn%iqk=_|)Q~MV`kEJ{4unljYTGQ+Cu0 zZ-C7fqf?;~?>t)RMIJOzE%PFUQhqS&qk$nQF$PbBTB0&<>@96EYL$+6H%cBifSEr=i!W?!~TYCv7YM`wD-+y4Q5H0RXY#=L46hj6E8I-GB&?(XwQ zQ64jvaFISTVJnHlR9W(ZEK=mVWBu_uu+<_*C7#MMDWlGRqRKK6Bt`zW0$Rx1RIZMU zUEvYWMcuXgJfTAIY}afeS@_Pgz17Lc(k^A}pjzUV`2~zOcbGaCnA#;|l)AZ5&+@xi zH|kUV&EEw|ky&(6#oHG?0t$Dqo-4imW4-PG@l4QTmJi_}?i?^UmRpqeOE_h;U1E*d z@+)&al=a3TR9;}!5|++qSBMV{{F1ZlCqDLHEr4$9A?OA34;l=xp2In?r|@X&zF%>B z8njMJmJp5M>A5n%;cmCRLgvNk?p}*L#?`m(KVuLIn1Odf!>t{fzGHt~VzOd(|)T+i!8qSC4FJh&_!uvLZfx3Li%p|>BiO9LGNe_-nZE*c=GjTF)u*# zYuxVd2){+4Cua7I;H*sp^imcNI`23946KaS6mM0!-KNh+$JTREdPmzO)aC*gNdr!l zn%l;m*I74AkegJm@QW?AXNOXOm7YQo&xG#a=VKvNxJNC@VT|=~hLKiJ;WZZ%N|}M< z?DoR61+WJdi#C|j?8>USM2@X&IAo~K-ER7`)+GA z?@IItPw1}@&J?Rl)wef284MzX;&N$DX$x5jt7rv6&3@Acxq8s%qFED=HC+`Y-Y4@e99Bj9s&&I8Ty=|$wFK89Bl2GZXviywsG=a!#g+%1~ zg{MX|P9OhkMzFz;{XulQ?6%4khMZiuL0Tv}R9@^875LGeM0v6lQ#$GE?nQ6bMI&Ik zsL){Jm2y2;Wh;b%Lh;Tc7OBrFI5$WkK8h#^3*FF)3XFnB7uILpoC%ByxLSYSC?GdH zdS)YDgnX&MR?R)(c)Kr^JH6d;$0w}Hui#%}{oy8DI|Wuk_f$bY3BeWDz0WWGJge`%AvwVY z^ULE^knx|v?}NX5s9p!+lr}76a;O?8*Lmxb9*ZW(&fL6+)~95`1yvKHNh!MnR*jdU zzyaa$6M4(`kwv0MyZW`S*^BtxP?0a8akTkR9yOvnnzfFb4L-#_!5W^^zvbM4YmSSA)$!K?Xw~N~v+X!@-&C zq;+XlYcwYg$&o*(psPb;pfb`KV;Bh`|_-Nr+cFB^fR3~v2ezHU3R z9)X>ZZ{I1hn;ma#MEW1Th6yyh;4Su0xszHi#5fO;yiaamkxe9mzJW;Cg~k^)y^mYw zQ>1)6J86DaK_ap{uo!mZ*37^wq`USc**)IhH_K4_D>C#NxSNb%ZT|*3Ik;}sNcm9V zfulkv3Fa7O`hg>4B=F515lvXR?_al>a_1XD%4&5;=5d^9{cyi*1%aE%G^`aoicA|p zIk^tW9$x9EE+`+Bs5>mm8s{k-SROb)iWt5dg=X6d8uv%-FK`Xn#~6qvDlv zL?~tob3-5Hu_hW!e=1Q|^@iowS(?R^;{K8yB;_gJ9QX6vf)c==ed+zVtB9n{FEX+H zy&37PnH0}??yZeie&6=NpSS^4{;BkRP)OO)8?7wcL()TN^5*k9U_AWIXhLEQX=}Jn zR(2G+w)ECV@uJgyAP`oA=+{d^>%G*~o=1>s*U>Umvg{+e>|L}weGqtCCHETCX6lr*S*6(MjC2&NcteZc$Ff2sFmHpf;eyuDw==*P z0blVWVR!4{&A$SJXnA*|$ie6Hp!A(KDuGoV&-E9#!hgH%|2XCMOG@_;VL?^f4PW$Y zz%g<_MZkt?!V0H6vPq%TH+^9dadt!mZ8ViwK_VRvekKkYSUcyBzmCQ#mCvk9eU97J zjP2!PyUqhaoQGycd{nbmNX+=bX7xA778f#EY5$YUAcaOxr{9n@-$z>dA0~II1xq57 zR_rjK6LSdL%Pj7IErLC3?&F`-{YA@Vh z&Tf+{%I3Nr^+MP8;u`m|Y4_MJJ}=XFI05O&d~xkG-Bz{VZuRnU=|6;B2n}5ZY*h&k z-apTpqLC7ZWLzK_Z5`a*Z%YtdEK7`UYxZFJI+G<{LknK+K(}Yei%^>VEx*J2oTa5^ zjlP}D>bAOaqwI8r_90JjT!7VjrWP$wpW(S+u)o(6;XC=^$(f&3Fb-pEB>0_{h@swM%idEF%>bH5Vw6I zg^jUniC#Z~V&FvTQ)je+u*mt0N74T?X7(c_$4;F7{&;&g%-}FGEq{U9n~7LgDw61Bu%;R=jGB7K z?{Len7zFL_AzdtT8+w{Od62V~XP9F4AskAQ_@deS7%J7_eONaqeRN)cVF$G*qAd+V z61xj*cRvrRJwN~Z{le6xkPdPE{z7?&ZDrJAg>|3zc;z7oCF}$nle8s#`2MoIb9ICJ z;bxIQMiB4s^u`ApjP=rYc*!oFiHxxagx7eT_EC)6Q5R+`1M#=L$iKP2Yv%3Cz4k6* zM~E@a^7vD{-ew;rop$IXh?{#B>yIp^Xg<_qS9#wbOLZbg!B2?lhDu~MPBE4(Plc(R zP3EdfDsme|H~d&uA~INK+T(wkUAo);D~wsC zKNe%uF493>U<=jq(d+i1FNB@6P#Jc~l3wooW@`a9%CBj>=|t>ac!Ad79_m^x%0t4& zB%_qQV}_@66=~nZ@l*ToL<&oDztee`gVtVBJKta8s}Ro7(vy{!VD>ni`a(gL^XhBU zgj+&$v?1ALDi*nIY5{uVExz;p`D2{O(DBz0Y_G1eOeaueMVQi(Ek7FF!DAVer z*4^Z-De~(aBf{Wm3UltD?Z_r&im$Ap@8c> zSOFB z`YCY_a;??pfk%6dY_A zJX0c?ZHQ~)s!7pFekXMkZ4r!>uR2MpvHvWm^tMW?rpnJ}T>949u6aQp)YPLt-BEe> zvRN*~@_EId$Ywc(yQ;zc^Ro$=+7@?&?M>=is#@&&E`2vCoaxXFuoZjXbf-FlcS9`l z#aSEGoonwWsQGwgjCD@MBsMD!a);_4^jm1-ti3iD=bB|y!W6l=ZFKk4Uu<#~{>l8* z2RQWef%!sn+IIG-yzg&MhcQDc5l)n0b@HeL)=B|?S?oliLtwxWjJz6NizDR{ihrp{6gZ zmi9!}BEp7qwrW{~K5O8P*!Sc(zp_@WWm5`?4?R{a6!KKz!~DqPLnDq*<75#A0)xTp z0k0__PTe*1sU(uC2^WaKzVRpjmqhWRfD>D!r`Hd@BrC~(+2xFWsxzmB6ncc2Nv40= zTb|zjxISsQFi!cMj@X11YhCV2EQC$Nr>G5d7Z5y{Rpo4th<$(>e&s4(hTehm^&gw; z#-FiDnQoE{kN`40Q$f**$bp4nVnWfaT4j*P!);moYkni-D3ufpdy*jd_1rY?%Pb(V z`qMC02w9!*yXhNNlpBB(bseubK?uiot9rxUr|}V<&1l)!@>#-Xc;dQ!ikoq8c(BJ2 z$G2@H?a0j|Am$^x>8TRYtBIeOeQ8vVRqC5pCch+0uK|={33pV&t)MQ4#TC?!Summj zPL@1P4~j?-Mwk9`&RG{7CH-)A?Vb!?OXicKCHsS}_)ED1vMOhm`ERIk6YhPh zs|D#pW=I14#_Yf|c%2%(#gTUQXvR$o-{aj|77(6>MgF*go?uN=W}T*g+O@JSYKoEv z5OWKLz{t~gC`ZCj(*+rRakV_RiL%oLLz zyn@aY{}W(OkXSw7%RE0H`l5{)O9hLR_EqcBEw=Eo?{_;KwY3;9v-`t9qigOFFR@pS zg^2@`L8Z)D)@W2EXH$pSQoQyp5d?mHAF+-?RWhXoUkn-;e8}d8XSxnwC}<(k*zhhZ z5p;eW)tJwtB@W5_94DYeUqt&|vA@ty1orDMtf!BB#DC&L@I5rQ=S7ul{PD(^+2Q9$ z<9jEHR+j*A-+sw8Opo$BFs!tpp0INHb^oQ_ZXmf+#Vz_;I#dTsS|VDiu_NQ^Yi2k( zRZEfZD8%dw&&&sEEGCdBb^O{D0s}Ke z$tRRA9@EM@gd$91A-!~|>El5s$oQ}9-@W$AFHvML@7wr5qk^$NehLk1mt^E<@awa& zBmY|_Obl2F|KQ9!=^58M z)6C_ibIk{y*cZ}%zLEvmn($G9Xk^+!wM9!Pm2r40n7dl|MtpI);-8Hs^X&VE$zFDf z6Of_y{6Gz{xXN0bHRSQ)oG>wqQl`6fvVaT}D{+pW9^!AKT~j~XY($DBFDt6yQ{8f& zd`{T~o&S`p52MEqs$^;q2|!t z+uOUzhL{sBk{4FEJ2P`aXcpF{a(fX5%}Fbf{e((W3b6#@2zY;D&fY*S5v zir{2;-5E@?DtRFEu&xvS)q zEWKXE2zh33ruBM;PspzYW4ijs9aud5lq3#JJ?@x5Y8EM z?zGz!B3F-$Y#TK7DoPZ#)i>pTjI}yPK+2*WFD|@iaB8Nw2UEdh>FrZ^#m6p2S~6HF zlVh!p5i5M;Gp@)xqtV9!Yf$1mrE1BWc+8&b0|XmxoN!M0`enfvm#O)SDU~n4bQiel z-voSrlleUW$hTInP~V8g1S9mc>5a$QS7>MV*`+mokUs8ioggf9U6}aW`H{kT)fy}c zYeh%6`97%DcYjB?$cHWne3Jf4!?#J9>Eg#`QDKV?g^G$zmI zbgqc(KRfd2P{TeUJ3w|}(PQZC}tWY7q_mFscMg-*W_GTJF)Mf#3(<*x7s`53#zbDvA8 zP*^PSZXi7!98X)6PL8@05dJL|qUqs_{M{{xX*j}Rt3bdCDuT=3_A4xIeR$#&&3V-w zK>|mi(uxSp?AvS>k-88O=qMk65=fGw!%2G4?!i4o-0;yUsTw_ABp_tHaXdJ4Zk6_n zY!k-uh!xwAY??3unHy(qB?(8|-#>)o8Njk-YE9GhdZ*6l#omxM>DpPJ`w94()O{El zSqtBK6h!7*8`q_-3FQa$K>{&6=ZnnvK~C*K<%8cA)s8=8bC;sn)vl|lg^h!_WItIU z9Y;NYZ`I!0t6$Z&qUj=U3v}EpashIJIrY?Bt-3(=f5`X5jJMyS3O&oCmU9m z-fYIEHV#yvc=~{{l$yIIyUP>Tu5-p~c>DI5&*eq2Jx-vw{f2MkYD(95wX=BYp{l8^ zkM7|U!qVuMq1e~LAEurQ=xMHnfZ$=_y^#v*zZnZMBFxWhHUZJpfHt$b?^D5&4UiRRxv_@Hn`u4YM8*=6GkHC|jjO8~66e=IUh zQdxV3v_2s6G%mnM$@9}wQ(9pkDB4!{LTd0^>YsE{)}zH-rfZ;kZAKn?>gz(=KvMhh zy+`CCzqSMIBD-_;_O^;byl}>l`q(4gnr4bKWcB^w)AhO@Q;GP(qU7j5uMu^Xa#cWp&7*xLGDtG}xxZ1UQ)N;ev- zRG3clNqAF{yqUiJZ*hPa;~MNgO}{i+?eKX1y9I2|S#aLAr__gMDeB=jR!N3tuA@Q% z$hF^~sg`PCq{9?Z66n$Trcx+~u`Tw;L$H-hJecW8sX$Gq38tu5smYgE{XgLR=?fMC zCLqVTFp5Fo`M;ZtMH7=`(8vUd(gmmqqM&Z~eI}l&ZVbYL1SIAFld&h>IZA{IB$+LL zd&)0x*pUWRN>LJAHngMR$Bd-}^G_*?jUJriMZaV77p2QFYbzs-tT?Yu^$_oft|30r z$$pv&L+m4RT8cXOlHd^(F}C{GqHW;^fD8W)u+I-avH89n&HPt0{}xdvQ9w56MGbxCrsw?=!-nH6Tt(`=&D@3O#mf z%(SR<+z=vV1#C+>;ZtC3g-{ZUC6A!P_jcb%?oId*dK4*&3Cl$9XHJ5im%*nx?PK1R z?09w~L0cLQf(?CH-jcGfg2$$@?F4*0+LiKC#-%&&G7NRmf9&L^?unBTqR{8Of5}`1 zvleyd6YyVpHNe;T9In|V-ZQh1ME7bEISI9#V{;a2>3qMQM>0F;^bL&!nqScC=FRs_ zY&qy%})qY;(6>t|6~*b;Lj^mDwN9DwgepVnur@KQbd@s(}%_A@fVHIiQo z5Assp7T=}p-b#?<5+0=)`CCB5kH_vk8@kR2qqFBV@Y2D>*K2z(zu)q!x7Lw{Fg_I0 zr0iLAf@MaY{Sjfxa(D+EgjLV|<@77iIU0%6`bEan_J?z#|F(yT13(;VVMH;mrQd!B zh!<#O_$v6^3d^Vvo)eIXQEg)}(L{2O->sAB3#TG=->(LZI|I{aA0GXC9^uONC;LMT zbas$Im>hDPB_zt1@Rx0`Vpimc3bJAq0sGWKBKWdF^sMs)N7I(^*Mr=xe`lE%lCYXR z4F_iIMfmAieP6gegnmbY?J#Inc#Ufx&Gq|8tQsy%)7#da6(mAIwqfqi^yDoDHjW?i zx`l_as?0}TmiHnBg3~xWRPqaU^1eYb20#9G`gf{ypbJ~iq@1L-bCLarALAX|3oN2KlWg)3(h?uE03Udyq}O9eI57 z|NgGw{wbD~rzy8SRNhk60FUSfuTl8(gC6n=1sqMUe(GiZeD{PBQQk+!xm+4kt2_R$ zG~n-#F`VR@;95BjZ$AthK5lo)HO(jJoIBCEIN7ui^f~Blz6@7^!x-NOi|)^&{Nf^q z`nWkYBy@Obwt0oWp$e?a>xOJgtnyhnJ$=)q0rgk#9Oflt1S$%o^D+@&=+N^YbTL9f zM13tnL6?gD=AA_FRpEPQ7RdfOCW2?W2d6$~=n(nO-xTB*f3ih2q&p5iA6-!WJ*x+Jx>%fX+@SB`| zaDqmi$ag@wCIsLgkywc*L&V?i(N+)Hg*Lu zOn`#_@rIH;g76P)_ZHoRmFPhe4kHu|H@qY3D#*H@fIIWSF^3vf$Ory(=@Mxby0Q4^ z0^Ny6comfA_n&JfIEr`*%I;s{V84loeU&F6+WreRHAKuYqkCoHEb*nN4p~!VR%zTW zB?HXMpJTI4j9Ps zJ+lqILe@<2f|x)`@~g>!Suj*d?mJ8uaRv@Zcuu1$Ts$sK|CP20yH5oC{{WCVPJiGZ zFd8>=O<0j8KW@buYJ3MR*!wWq26Y|`*A>)`=`BN_b5tjpvD}bmA<@q$y3TcOLcjQW ztIfu+`=$Cvc;xcuyweDLDo@YX9$1dWBsUF}-ns$W1hSVYcbU2!k^Z&YQe#0nxibI3 zU|cY+%U_52jLMoRxX0)ZD&YIQgsrO!I_ia7V>(%Uq?3HJ5L)_@A#wSTw{ z>wFT8RhuV@VijN!w1B^>38)xU;>oDpZ)UIu2Kl*y_1>2gnq~ZRz#-u$So9$Ahbvg z_V(hQDZD@tV}!!|o3%(dk!5&|c-5I3YY~5ACZRB*LqdW=!?C52ck04yB!e@ln=-gSBfN>+9N*a#lZLr+@aN|;WR~P@98Vx8O_1}~qUrddS%|e_gFjO2 z&EbOuMqoK@S5%iAJ^5Fe7sm-K=m+0`n>{6{GOGNoXX_G%~^QcFsHqt~BRI8U{GxU3!qX4?@h z)b`Np+xM>zWw3eN@X&;btc&pcBVf)Yh+YlHu@no99;rA*n8#T4dPp@6l>RKRqbzxx zWw#YI1`cfsYaXK@7Q=Of;7VDt;**o4TFqE`4rRmyN7_Pw<^K)sm_v$kqEO)NiLTl|BgcW-416MX2P%a@I0Iz8kD*WWmxWHVsb z!WZz%0b%d3m>>uW1&f=WHDT;Po%9|wG!Tpe#o*DG!0L{in6DrK025Ddf8N3P5fM5R zP))v0jz9E1@d)X4odO)n%w-?n_Y*smC4pz=$8mgc0@5iVlI)fY@WIZCf_KsbpUg0$ z0zf`|SMbS}A7lUeWR|9DZwpGCw9xQBSsgFlfL28G`i=ZUlK%NBkUiEgng`Ma}S!zo%tCH%5&(}?6pw(pnP&xLD$)9gO#>}L6u5!@SZhjgk6s4 zpg58@Bd6~@yraaEXQc3@>5%w^;?rH-N234MCmh_WjkL4t8xequ-7!{}q6{TCL_~k* z@<8_^5gyj$3`ZKH{6_uNNvoh_^+Bd*IAjY$GD+h@M2&+~i9>M(K*6 zB!nLCo0t4N&61kZWFCcDMjqSz|E?s zY?>xmt1E;o(YA8?pWDd#lmrpIpC2qy{&vocDc|MHF}Wr<7D1O$K$tTGV&Q+G#qRre zay64P7A`Xc;niU)Z>@^>1mcI_r}vvzRTi*HYRLM$#hIV0G{5oarUr+gG6Fh)k zcrG}UDw9@RV@)S9{g2_g_|$0gLO?GDwy7=tsLRp67YK{dl?Qhb>Zm2k0$UsLX&4Of z688J+G6wu@Ii@3sTZmx(dgOLfx+^h?n*(vMGx1zDqg3w1(acz3V*SY{Bm1r})E2=*7ro(|(}q+?VFpjb_%y9M^>~#F$Aq!5v4Ua^yJjuSQpw zP@`-C(WyavtTN6Jh&j=Z7a>lkp#1*&qQy+#h84!j^anMsK2x~jPbYCwf+LA5$)M1A z+X$lQ{%WOQZ%lv+JYI1Bp{Z-=@43{H*CqHc36mBgvOOpae5nk}}kjimX;l(Pec1`Cu3hSmdy@9CEf*DEZL ziy0Idx0LJzdutjnV|gIJm)lT3!Yj!5mACN%1;wnUIK8kOjegIbGr<;HA(-dr zMWnp|3FixE=I+~}g%U`eghXETnL$(npLiv;!Zxy_HAr-XpNA9=Japq{As+n?UC9-N zE9rKE(5L|CMqjD ze)EFFb!AL)db#gDI1d>lpEZvnddH8xsEXH_hbCaHnO;6jRxn(F(4<%go(U@{ig4y< zCy|Wh{Z&KgLR_2a+Il*5s?e;$O6%v7GuMCMxJ zM*wBUO&KLf8-&3>HX;Z2(?w#1;H}x7_rv{8M$5cD;H%ytt5@tWQhCCu;)XnxKmHjM zeaL(HP7kpC#WEbWL0QTLgo&%hSBA@T6R&ubjY|VjD&h8-4uC^C*qZL&xir@&f~Hf# zLuH1_f!@Iw4hSjY0^8ZCEulE_gX2j6{_Nj?e)Jd_CkuNCAZs;%H*b$cFiMX(_-%hS ztAnOF1V$aQFKgw9j7lHH1}FNnD8;)7EpHLkM?6CA_zz1@5<)jz?4adz+$M8zjK3TW z8U;3juEa&Kh#QUc!HqA2p3X85DM>He^mAQ?7LWTdj^T34IXZ|O6RxIZXSdWJ>@0L# zMw5Y)s>79h0?2(EliWcbev?@Mv#B791rKf;S4JK~fTd5PV=A4(5??YLJRp~f4M6UV$dC#~fIFhE$s| z9iQw=Fgby#Cjf~^68)uScnykgm-gDdjvteOHw?~v`#P_N00C%c5IrV9-Jkk_+@_|z zJu>^pn`#nx0ZI1`K!AR!g^!$8L#OvEGPYOsz53mhx9{?xMU3$>v_fnFWvd2+a{3rt z+Al1F?$7z_FUql>5F)B&QwVsQC2aS850Do^MMq9@mce=00SizhFkbg1T-4a$jqXW8 z-QPg2`rd;_`44_iv}kI*d%ai|#>kI|hQCH|`=@Z6m6SH2yr$&EQ>Lqx+G_7?=Hc08 z7{7OgVYK#kt?ZPZrpnY}!sJG)ODK~b+vE9K=$}R=O-Gf#o&TaQ5K>>|;5KWx#6{g! z?tP=M>rwbX&W6m#R@*oMvt>lQ`m@p5h9=q#UcaKsFKq*v7q_j4%kC6IJ6i3f%pP_{N*~Z@_9`{Z4e1)2IVrh*ZL7nPjh#wyJD~ffAU3P3cKzW2Gy3=z^cALO5=KwA5O+UP6a;IWj9?&6SYoE@o)2SJ(WAJWU|a0u>44auWgxO|y$n zjl)Fpj-b-9*Vu%??R*j;ej3wVe*u~?H<9dp7J?(Wt!pH^e^A`SkR74b!2tLt>#+6- z31mf0)7@~W->}i;9l!&A@|pj2J3_BshR(U7QCP`uz_*$f;^Zbl8L>2-3Nc#c@F768 z7pEk=flb|d=Pe3|1}x^!va|1N`HO%AK>Jdclu$?ceeic3>phzzmn=Kimuxka1A21m zSk|5*^V06M#c&rR<^|@3wt4%_nR_G1Y--n>lQ)`*8MHMMDaGhg7v?TNnhu6DMuO*3 z(5IfZ2U>LS6(LPIV#YO=SI>Y9LwM!-=hu(QS4nit&zRDz5|N?y0$QN;P+0m*<-v3YAfe#1RgqYhX5q)$C{eeDm286#u$lKB zifLG-aImsD_G>BCBH3YOMTPv3S6z}D3P?1cJZa|`(HWdfPBBO0GyKjH5ur2yW*Kp% z)pa16P~BLp!681UWan=0D~D0FIQ|XtCkzr*4dhx9=mw^=ZN*sA2S1KBPMt!R$>Yf) zkmF}Zc`cx(SqoSQRjG3fKxT&TUH4dZ=&`6V>ZB1Clhgz{h)1fiYX!{aa(JXy#)f}} z4x_8_9r(ttc8vK0pd7mBXOp*ilNK^lCvfEMNzB4vj~O_dY&IAyhbri>N{K-!K_G9A z0uk*yj~CJ&{=yjcKhme*lmRv0aEWEn%YFxYio#039RuX$FY|2_z0QisEGS#L<$0VS zgoUV!+?2B&yZ7|Bf7(sxahCHzU z0)r5lQ^g~q+BY^RWc2XBpffXCmNkvJO4ZvSX!^_|0l!2dIAaBDBi8?~=pVTrp^@1W zD5(Cv(}=&MO>z0n*2S4Gc@kM$6}SnjeupFCj3f}qdsIDb{^3fAB7hNw$4~2`<8@ z<8+wh95@Uja9>B6RJuqS@V!5I&j;eAhsmpL5Y0*^p zhs*#oMGRTNkc<-~9XWn-O9M#Xg2-3L_t!e!^FzvF-V#bpZT21tHn9Eh8JdkZh0$dI z!9&eT;rib?oF{6pCK5hFA$-+5RRhGndA({*3@h3_0us|7Mq_^NHW{XU^x|%q*U$Cc zsB?Ixj=yVJ4|ghi$W#3>{ww;A6YJ=KWw1-t{#oM{SLtp*h`QU~odZJv_6@2cvY4!J zFmLnMgWh64A&?){>sYfv?SESl-D$bTXD)5b49vjB^i9-aInR(-J~zi1Csu?9(WXazCX30 z*7h>=s&ZKaAJ61U^_XX&PQb17L8M=8^C8X>8qsKx)|%&B6b4_iRHDCvkL}6nLwFxE z`-ta%N-tKXA;rlAS)D?-F`I9mhy_@~YiHqcl|PjEU~4{NT<3D!c zgj0N98@=G$QD@AOqMrqY_oZ(&PCvhbSzQ5HjN~M-X#)^rF zDv`@T!u06)se5k4{cnNg@J19Fq7=tNqc;ZOzzV1u#4N0V+AfScU9S0B>W*DB`SG2} z5fw@={Fcp8=K@qHW|@e0UsC?*lX-UiBvRvseS7>%c+0{I<4{ap0sl)^4h)e^p73Ep zR`7kWg|CrGM8Kl*EWHQhj!p$}l_u6o4kjm8DCM{!I z?;KbQU^!%Sv_80Dzoa*BIB3;|-Y2^GYVW7^OjzuIu&T(C0$)`t#_G4grgp7`hj?o0 ztCeFU=#dfL!=cW&*8>~h>fF{23oA616S=gywq%9f7V4QK5Zb{e$y%?`?F@x={0 zHscSQrF!X5xXcuf=`)PXHcw)BZ_Xl}@$?X5h*iVPtiN-5Zh)B5GDt$y((bT@9;`_p zmP>#h+8guDzSM*MB_M00Y$ec4jP)HxgW%%;!g3PB;-0h1ho<(@nAk;rzQyT{+Oae2 zoypbT^5!_fxu98-Rtk$xBrXCrL5cd?|5{P%$F!FLw zGE-a{KE)i^W>_DZF-YIx?vrg&XtET$@H%U0k(Tq>o}cy`DnNsg+62HAQrFLF7qT3) zNnX#+iktsJ9&=CKEz@GAa=_zrwY}GN9iiK8gk#H1bC8F9t?0V*XwM$otK0S4yJhT$ z*H>$F8|>P+EiU+BrFZd{Jf4d(YsofVHQ zbsm}Cm5Geo{FPz5kU9Y4sXi^eQ1DBF|Ehm(e|a!zAx|S*Q1F(y3vwSUta&n)?ES_hSaqi%b*h(vfWH z;%=(F$H;sVOTeANO}?%X?kYa2IlDX6qxU36JBFhrw0?x|RDdcs%^Ovp>X@$r?);!+ zWVm1b$vv6&I6JS7ok@Xu>6-?h-JPl^H;%2Py2;u}-`efaD_1vF+}IZKTS^bi<{p2L z-7wytx);6Q)~GnHd(wr|{*sA(}e=c%pnbkiq&Ahe z#Pg$k=AJhL?{H<6!0uxD`%d=iR-eL4MsT=oq=ygpoTumupRg~4h2Qa89crT*HTDae z;_NE4E8TIElAf!Ns%U9Defe{r{>q$%++Ng_o|12XLN741bdlke{2MGc0bVZ1-j?zKvd<=j-F{o**WoW9Tv}J-bhH?fC0(;!EzI`2+UMl=#RDxVKKhr15yFYHO+hR zs~0A$Y@c|p%royH6>69i9^chYt4Ky-S7e*N$hGZpb-9p+fwRlnEY)P+taNWuf45~d zY9`ukkhv3~ALlsiS*8Yk&Gdo#UTPF1OEj=AN4IvcsnE#)I-1O=`I7EB&kU1 z)u9tkTX<#`^ao}Y8j!&RaT3tu?v3VoXBCy<@|fN^IKuIZQO9V3)`?Jh;Pc+_si^5J zSAM|-<<0+nmG0*dW|?q+f{6}^a#FN%E-e5im@xk!ep}57WESR9jppEDQ zRlM|FtI}*l`|SpB1dB@cz8{zVr9&A!3Vvf@?QUn%T+0+prfel=l=`|qZ_v0(Cdh4J zFrzMSuctM692c|ZH>96GWCtKFP>a|mB#jez7LZ* z1mjQ9CjWLHf1Jdshld^JQ(&S#^RhUCN>Ak(eKW12R6)q={%2wF&G0v@?ov09_A4U7 z2(IMjM(fRqN_)ZxQ}=U(D>9lgdE$S+55IMdhS11gXrix0iiCm4cfS@2nNgXs54Y;% zz$1Sekc#@xKm2=2ETHv8NoNfHb0|px7ViPHfTn$h4PhL<%hP|!`Tz45e|qB%b}B=! z>0wWCF36_=D^ip93uv4D?|2Xq+7M)5s=WsFpO=aP6q^0_GoxE5fZA$q@jRxJ%NGCN z%^+AouBrIbMjZGp@Mce0%qg;-+C;cdaQv!#< z{O1Wzpn?VcTi*2;rcZ-m=Qp5Y@$W1E&YTRV^^2tX;oo=!hF|b}IR^#b4m8 z12EEd3BS0!-2d+F1&kaLjJhYb0M~C@rvL}{kP3ki2cYpx5h7p>j2u&(7XVMAe=4$= zhb7A?0eCdKqs&@PPc7iO6IhUTOa%HZ`HNL8II#eyGHw71nO9NvQ=Jt&9l8yH!^2T0 zgF!QtFb@MQO9c+5=n9J*g_`kHNW!WlW2dVodn2E4z!D>1C&@5~5pF|+!xB)^E48@` zvErBD4d8Koufr`(ofY~U!dAb(wl?}+Bp-O28qAu;B94+@r(cEmIi7-fE@0LH;84vw zl@Q(NZD)XeqrD}GMT!B+OrT>_tKPo^tqO$s37Ay1K#M?9_h>*4V-e8sdH^hfKUeY; z>UDMWSP5L%P*wMO9dpDg@F8xH5bS6I#-vEzvdLhdUf^H?j=4_DUj|Hqy%~LG3J<`; zA$i~LmbbGZ+oTY{3e3n_3ko?Qq6-8X=h;^Ox(u2-SeSFpaltbd&{dA{&w&nz{=91? zaGM0U7y(;q2;BS8d+_uZL9n2dQv$G3`1ICirqNQxw--I`umG2-KHUdAE~AnII4GNb z#{jri6UmdSR1P%xgf=|}u9*M@9B>fo?C%@EW4O1MybOBQaHjomp)co)y=FB(3V=&? zdCGxDf$Uzre&4QTUxOu)jY(1jhLz5|W%I!f3n*urm0j}f&CPwXZ?msf39)4PCj9*L z6nq#z@T{V8;L6?d^m&zOYiBP24ivy547hj!*k>`33cnl(7R+F21RVhfx<}G{sig-f zFt!7i9l4qKN(&17_&opr88@ggO+}j1H17*IWBSR@)Ji+1p(r~!HMHJ_pb(==uzLkTZ?xx?P7obx$T`vObsvCcS`*KykQQ3VLXeZ9?yxnK#0{4J(EM}Xm zi|j)yhX?EZMPyerc*8rR7s|GcDOV%8n z3pyQ&)%*1HikWxjo9EBdPgh#8O-OCBXVdIkuP!cb7dHc*PBBZEO_j+@Vako=wNF>p z1ZkYH50kvQ{PWScRi|05*F?XGNcHJ4mAj?&GHQ2R^y%=*x3BLUYUPdv9(-xT|9X4u z^2fllTaM>$ILKxh2fTvf$v0rYf34k6c6QshZI6`mJj-r{&wuk3xYMQTzRKw+$=zW( zp>ff`M(It^x+3r$UBCnPcj*I%LHz=kNayWHtbR6;AElI>;`rb{W01hYNkJZ!Eet^5 M>FVdQ&MBb@09SS>zW@LL literal 0 HcmV?d00001 diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index ed58e3b0..e3e9e9dd 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -5,13 +5,14 @@ Move TiKV MVCC GC worker from TiDB into a group of independent GC worker node role and implement a new GC process in TiKV for RawKV. ## Motivation -GC worker is an important component for TiKV that deletes outdated MVCC data so as to not explode the storage. But currently, the GC worker is implemented in TiDB, which makes TiKV not usable without TiDB.And current GC process is just for transaction of TiDB,it's not usable for RawKV. +1.GC worker is an important component for TiKV that deletes outdated MVCC data so as to not explode the storage. But currently, the GC worker is implemented in TiDB, which makes TiKV not usable without TiDB.And current GC process is just for transaction of TiDB,it's not usable for RawKV. +2.Standardize the API used to set and obtain GC status in PD to improve the developer's experience. ## Background According to the documentation for the current GC worker in a TiDB cluster, the GC process is as follows: In TiDB GC worker leader: -1. Regularly calculates a new timestamp called "safe point", and push the safe point to PD. +1. Regularly calculates a new timestamp called "GC safe point"(The default interval is 10min), and push the safe point to PD. 2. Get the minimal Service safe point among all services from the response of step 2, which is GC safe point . 3. Txn GC process: resolve locks and record delete ranges information. @@ -19,6 +20,16 @@ In PD leader: 1. Receive update safe point requests from TiDB or other tools (e.g. CDC, BR). 2. Calculate the minimal timestamp = min(all service safe point, now - gc_life_time). +the GC safe point data in etcd of PD as follows: +- safe point generated by TiDB: + ```shell + /gc/safe_point/service + ``` +- service safe point generated by CDC,BR or Lighting: + ```shell + /gc/safe_point/service/$serviceID + ``` + In every TiKV nodes: 1. Get GC safe point from PD regularly. 2. Deletion will be triggered in CompactionFilter and GcTask thread; @@ -34,10 +45,41 @@ So we designed a new GC architecture and process for TiKV cluster. ## Detailed design For support TiKV cluster deploy without TiDB nodes. 1. Add a new node role instead of GC worker in TiDB nodes. - The code of new GC worker,It will be added into [tikv/migration](https://github.com/tikv/migration) -2. And for API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. -3. GC conditions in RawGCcompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). - 1. If the newest version is earlier than GC safe point and it's delete marked or expired ttl,those keys and earlier versions of the same userkey will be sent to a gc scheduler thread to gc asynchronous. +- Why we choose to create a new node role: + - IF we add GC Worker in PD: It will cause the problem of client-go circular dependency. + - IF we add GC Worker in TiKV: Because the logic required by GC worker is well implemented in client-go, but it is missing in client-rust, adding the implementation of GC worker in TiKV will increase more development work. + + So after discussion, we decided to add a new role for GC Worker. + - It's mainly to regularly calculates a new timestamp called "GC safe point", and push the safe point to PD. + - It is implemented in golang, which is convenient to call the interface of client-go. + + - The code of new GC worker, will be added into [tikv/migration](https://github.com/tikv/migration) + +2. Changes on PD: +- A new concept is 'service group': + - Due to TiDB, TxnKV and RawKV are allowed to coexist. Because the data of the three scenarios are independent, Because the data of the three scenarios are independent, separate safepoints are used in the GC, which helps to reduce the interference between businesses and speed up the GC. + - If multi tenancy is supported in the future, 'service group' can also support it. + - Need to design new interfaces for update service safepoint with 'service group'. + - Add UpdateServiceGCSafepointByServiceGroup and getGCSafepointByServiceGroup. + - the safepoint data path in etcd of PD,will be changed. The new safe point path in etcd as follows: + - gc_worker safe point + ```shell + /gc_servicegroup/$service_group_id/service + ``` + - CDC,BR service safepoint + ```shell + /gc_servicegroup/$service_group_id/service/$serviceId + ``` + - the GC Worker configuration in config/gc_worker_conf.toml file. + - The default interval for generating GC safepoint is still '10m0s'. + - And TiKV will get the GC safe point from PD. GC safe point = min(all service safe point, gc worker safe point). + + +3. For API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. +4. GC conditions in RawGCcompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). + - If the newest version is earlier than GC safe point and it's delete marked or expired ttl,those keys and earlier versions of the same userkey will be sent to a gc scheduler thread to gc asynchronous. + ![raw gc copaction filter ](../media/tikv-rawkv-gc-compactionfilter.png) + ## Reference https://docs.google.com/document/d/1jA3lK9QbYlwsvn67wGsSuusD1Dzx7ANq_vya384RBIg/edit#heading=h.rr3hcmc7ejb8 From 92b6b54f415db3c7f732d38c14534d119eeb51dd Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 29 Mar 2022 17:07:27 +0800 Subject: [PATCH 04/14] RFC:TiKV RawKV MVCC GC. add some details. Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index e3e9e9dd..7ff52950 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -38,12 +38,12 @@ In every TiKV nodes: In a TiKV cluster without TiDB nodes , there are a few different points as follows: 1. We need to move GC worker into another node role. 2. For [API V2](https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md) .It need gc the earlier version in default cf. But Txn GC worker process will be triggered by WriteCompactionFilter of write cf. -3. RawKV encoded code of RawValue is different with Txn in TiDB. +3. RawKV encoded code is different with Txn data in TiDB. -So we designed a new GC architecture and process for TiKV cluster. +So we designed a new GC architecture and process for TiKV cluster.It will be extended on the original interface to support the RawKV MVCC GC. For the original TiDB scenario, the old GC implementation can be used first. ## Detailed design -For support TiKV cluster deploy without TiDB nodes. +For support RawKV GC in TiKV cluster deploy without TiDB nodes. 1. Add a new node role instead of GC worker in TiDB nodes. - Why we choose to create a new node role: - IF we add GC Worker in PD: It will cause the problem of client-go circular dependency. @@ -60,7 +60,7 @@ For support TiKV cluster deploy without TiDB nodes. - Due to TiDB, TxnKV and RawKV are allowed to coexist. Because the data of the three scenarios are independent, Because the data of the three scenarios are independent, separate safepoints are used in the GC, which helps to reduce the interference between businesses and speed up the GC. - If multi tenancy is supported in the future, 'service group' can also support it. - Need to design new interfaces for update service safepoint with 'service group'. - - Add UpdateServiceGCSafepointByServiceGroup and getGCSafepointByServiceGroup. + - Add UpdateServiceGCSafepointByServiceGroup and getGCSafepointByServiceGroup to standardize the API. - the safepoint data path in etcd of PD,will be changed. The new safe point path in etcd as follows: - gc_worker safe point ```shell @@ -75,8 +75,10 @@ For support TiKV cluster deploy without TiDB nodes. - And TiKV will get the GC safe point from PD. GC safe point = min(all service safe point, gc worker safe point). -3. For API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. -4. GC conditions in RawGCcompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). +3.Changes on TiKV: +- Get GC safe point from PD by getGCSafepointByServiceGroup interface. +- For API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. +- GC conditions in RawGCcompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). - If the newest version is earlier than GC safe point and it's delete marked or expired ttl,those keys and earlier versions of the same userkey will be sent to a gc scheduler thread to gc asynchronous. ![raw gc copaction filter ](../media/tikv-rawkv-gc-compactionfilter.png) From 2464d0caeda80fa2a0c9c7348c866d962d7d0904 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Thu, 7 Apr 2022 08:57:01 +0800 Subject: [PATCH 05/14] RFC:TiKV RawKV MVCC GC. add PD interface and pb design Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 110 +++++++++++++++++++++++++++++++++---------- 1 file changed, 86 insertions(+), 24 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index 7ff52950..cf7083e0 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -45,35 +45,97 @@ So we designed a new GC architecture and process for TiKV cluster.It will be ext ## Detailed design For support RawKV GC in TiKV cluster deploy without TiDB nodes. 1. Add a new node role instead of GC worker in TiDB nodes. -- Why we choose to create a new node role: - - IF we add GC Worker in PD: It will cause the problem of client-go circular dependency. - - IF we add GC Worker in TiKV: Because the logic required by GC worker is well implemented in client-go, but it is missing in client-rust, adding the implementation of GC worker in TiKV will increase more development work. + - Why we choose to create a new node role: + - IF we add GC Worker in PD: It will cause the problem of client-go circular dependency. + - IF we add GC Worker in TiKV: Because the logic required by GC worker is well implemented in client-go, but it is missing in client-rust, adding the implementation of GC worker in TiKV will increase more development work. - So after discussion, we decided to add a new role for GC Worker. - - It's mainly to regularly calculates a new timestamp called "GC safe point", and push the safe point to PD. - - It is implemented in golang, which is convenient to call the interface of client-go. + So after discussion, we decided to add a new role for GC Worker. + - It's mainly to regularly calculates a new timestamp called "GC safe point", and push the safe point to PD. + - It is implemented in golang, which is convenient to call the interface of client-go. + + - The code of new GC worker, will be added into [tikv/migration](https://github.com/tikv/migration) + - the GC Worker configuration in config/gc_worker_conf.toml file. + - The default interval for generating GC safepoint is still '10m0s'. + - And TiKV will get the GC safe point from PD. GC safe point = min(all service safe point, gc worker safe point). - - The code of new GC worker, will be added into [tikv/migration](https://github.com/tikv/migration) 2. Changes on PD: -- A new concept is 'service group': - - Due to TiDB, TxnKV and RawKV are allowed to coexist. Because the data of the three scenarios are independent, Because the data of the three scenarios are independent, separate safepoints are used in the GC, which helps to reduce the interference between businesses and speed up the GC. - - If multi tenancy is supported in the future, 'service group' can also support it. - - Need to design new interfaces for update service safepoint with 'service group'. - - Add UpdateServiceGCSafepointByServiceGroup and getGCSafepointByServiceGroup to standardize the API. - - the safepoint data path in etcd of PD,will be changed. The new safe point path in etcd as follows: - - gc_worker safe point - ```shell - /gc_servicegroup/$service_group_id/service - ``` - - CDC,BR service safepoint - ```shell - /gc_servicegroup/$service_group_id/service/$serviceId - ``` - - the GC Worker configuration in config/gc_worker_conf.toml file. - - The default interval for generating GC safepoint is still '10m0s'. - - And TiKV will get the GC safe point from PD. GC safe point = min(all service safe point, gc worker safe point). + 1. A new concept is 'service group': + - Due to TiDB, TxnKV and RawKV are allowed to coexist. Because the data of the three scenarios are independent, Because the data of the three scenarios are independent, separate safepoints are used in the GC, which helps to reduce the interference between businesses and speed up the GC. + - If multi tenancy is supported in the future, 'service group' can also support it. + - Need to design new interfaces for update service safepoint with 'service group'. + 2. design the etcd path to save service safepoint of service group: + - the safepoint data path in etcd of PD,will be changed. The new safe point path in etcd as follows: + - gc_worker safe point + ```shell + /gc_servicegroup/$service_group_id/safe_point + ``` + - CDC,BR service safepoint + ```shell + /gc_servicegroup/$service_group_id/service/$serviceId + ``` + 3. design interface to standardize the interface: + the interface as follows: + 1. Gc worker will call pbclient.UpdateGCSafepointByServiceGroup to update the gc worker safepoint as follows: + 1. interface : + ```shell + func (s *GrpcServer) UpdateGCSafepointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupGCSafePointRequest) (*pdpb.UpdateServiceGroupGCSafePointResponse, error) + ``` + + 2. added related pb info in pdpb.proto + ```proto + message UpdateServiceGCSafepointByServiceGroupRequest { + RequestHeader header = 1; + bytes service_group_id = 2; + uint64 safe_point = 3; + } + + message UpdateServiceGCSafepointByServiceGroupResponse { + ResponseHeader header = 1; + bytes service_group_id = 2; + uint64 new_safe_point = 3; + } + ``` + 2. used to get GC safepoint for TiKV: + 1. interface: + ```shell + func (s *GrpcServer) GetGcSafePointByServiceGroup(ctx context.Context, request *pdpb.GetServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGCSafePointResponse, error) + ``` + 2. pb info + ```proto + message GetGcSafePointByServiceGroupRequest { + RequestHeader header = 1; + bytes service_group_id = 2; + } + message GetGcSafePointByServiceGroupResponse { + ResponseHeader header = 1; + bytes service_group_id = 2; + uint64 safe_point = 3; + } + ``` + 3. Used to update service safepoint for CDC/BR/Lightning: + 1. interface: + ```shell + func (s *GrpcServer) UpdateServiceGcSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGroupServiceGcSafeResponse, error) + ``` + 2. pb info + ```proto + message UpdateServiceGcSafePointByServiceGroupRequest { + RequestHeader header = 1; + bytes service_group_id = 2; + bytes service_id = 3; + int64 TTL = 4; + uint64 safe_point = 5; + } + message UpdateServiceGcSafePointByServiceGroupResponse { + ResponseHeader header = 1; + bytes service_group_id = 2; + bytes service_id = 3; + int64 TTL = 4; + uint64 min_safe_point = 5; + } + ``` 3.Changes on TiKV: - Get GC safe point from PD by getGCSafepointByServiceGroup interface. From 2186c228ca616eb5a1100eb79877701458f8854c Mon Sep 17 00:00:00 2001 From: ystaticy Date: Fri, 15 Apr 2022 10:29:36 +0800 Subject: [PATCH 06/14] feature RawKV GC,modify pd interface design Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 38 ++++++++++++++++++++++---------------- 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index cf7083e0..b885bf4b 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -2,19 +2,25 @@ ## Summary -Move TiKV MVCC GC worker from TiDB into a group of independent GC worker node role and implement a new GC process in TiKV for RawKV. +Move TiKV MVCC GC worker from TiDB into a group of independent GC worker component and implement a new GC process in TiKV for RawKV. ## Motivation 1.GC worker is an important component for TiKV that deletes outdated MVCC data so as to not explode the storage. But currently, the GC worker is implemented in TiDB, which makes TiKV not usable without TiDB.And current GC process is just for transaction of TiDB,it's not usable for RawKV. 2.Standardize the API used to set and obtain GC status in PD to improve the developer's experience. +We change RawKV encoding to MVCC, so the GC is necessary. +No GC for TxnKV scenario when TiDB is not deployed. +The GC safe point & Txn safe point is easy to be misunderstand. + ## Background According to the documentation for the current GC worker in a TiDB cluster, the GC process is as follows: In TiDB GC worker leader: 1. Regularly calculates a new timestamp called "GC safe point"(The default interval is 10min), and push the safe point to PD. -2. Get the minimal Service safe point among all services from the response of step 2, which is GC safe point . +2. Get the minimal Service safe point among all services from the response of step 1, which is GC safe point . 3. Txn GC process: resolve locks and record delete ranges information. +4. Save Txn safe point. +5. Upload GC safe point to PD. In PD leader: 1. Receive update safe point requests from TiDB or other tools (e.g. CDC, BR). @@ -34,25 +40,24 @@ In every TiKV nodes: 1. Get GC safe point from PD regularly. 2. Deletion will be triggered in CompactionFilter and GcTask thread; -## New GC worker architecture +## New GC worker implementation In a TiKV cluster without TiDB nodes , there are a few different points as follows: -1. We need to move GC worker into another node role. -2. For [API V2](https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md) .It need gc the earlier version in default cf. But Txn GC worker process will be triggered by WriteCompactionFilter of write cf. +1. We need to move GC worker into another component. +2. For [API V2](https://github.com/tikv/rfcs/blob/master/text/0069-api-v2.md) , it need gc the earlier version in default cf. But Txn GC worker process will be triggered by WriteCompactionFilter of write cf. 3. RawKV encoded code is different with Txn data in TiDB. So we designed a new GC architecture and process for TiKV cluster.It will be extended on the original interface to support the RawKV MVCC GC. For the original TiDB scenario, the old GC implementation can be used first. ## Detailed design For support RawKV GC in TiKV cluster deploy without TiDB nodes. -1. Add a new node role instead of GC worker in TiDB nodes. - - Why we choose to create a new node role: +1. Add a new component instead of GC worker in TiDB nodes. + - Why we choose to create a new component: - IF we add GC Worker in PD: It will cause the problem of client-go circular dependency. - IF we add GC Worker in TiKV: Because the logic required by GC worker is well implemented in client-go, but it is missing in client-rust, adding the implementation of GC worker in TiKV will increase more development work. So after discussion, we decided to add a new role for GC Worker. - It's mainly to regularly calculates a new timestamp called "GC safe point", and push the safe point to PD. - It is implemented in golang, which is convenient to call the interface of client-go. - - The code of new GC worker, will be added into [tikv/migration](https://github.com/tikv/migration) - the GC Worker configuration in config/gc_worker_conf.toml file. - The default interval for generating GC safepoint is still '10m0s'. @@ -64,6 +69,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. - Due to TiDB, TxnKV and RawKV are allowed to coexist. Because the data of the three scenarios are independent, Because the data of the three scenarios are independent, separate safepoints are used in the GC, which helps to reduce the interference between businesses and speed up the GC. - If multi tenancy is supported in the future, 'service group' can also support it. - Need to design new interfaces for update service safepoint with 'service group'. + - How to allocate $service_group_id ? 2. design the etcd path to save service safepoint of service group: - the safepoint data path in etcd of PD,will be changed. The new safe point path in etcd as follows: - gc_worker safe point @@ -84,13 +90,13 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 2. added related pb info in pdpb.proto ```proto - message UpdateServiceGCSafepointByServiceGroupRequest { + message UpdateGCSafepointByServiceGroupRequest { RequestHeader header = 1; bytes service_group_id = 2; uint64 safe_point = 3; } - message UpdateServiceGCSafepointByServiceGroupResponse { + message UpdateGCSafepointByServiceGroupResponse { ResponseHeader header = 1; bytes service_group_id = 2; uint64 new_safe_point = 3; @@ -99,16 +105,16 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 2. used to get GC safepoint for TiKV: 1. interface: ```shell - func (s *GrpcServer) GetGcSafePointByServiceGroup(ctx context.Context, request *pdpb.GetServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGCSafePointResponse, error) + func (s *GrpcServer) GetAllServiceGroupGcSafePoint(ctx context.Context, request *pdpb.GetServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGCSafePointResponse, error) ``` 2. pb info ```proto - message GetGcSafePointByServiceGroupRequest { + message GetAllServiceGroupGcSafePointRequest { RequestHeader header = 1; bytes service_group_id = 2; } - message GetGcSafePointByServiceGroupResponse { + message GetAllServiceGroupGcSafePointResponse { ResponseHeader header = 1; bytes service_group_id = 2; uint64 safe_point = 3; @@ -117,18 +123,18 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 3. Used to update service safepoint for CDC/BR/Lightning: 1. interface: ```shell - func (s *GrpcServer) UpdateServiceGcSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGroupServiceGcSafeResponse, error) + func (s *GrpcServer) UpdateServiceSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGroupServiceGcSafeResponse, error) ``` 2. pb info ```proto - message UpdateServiceGcSafePointByServiceGroupRequest { + message UpdateServiceSafePointByServiceGroupRequest { RequestHeader header = 1; bytes service_group_id = 2; bytes service_id = 3; int64 TTL = 4; uint64 safe_point = 5; } - message UpdateServiceGcSafePointByServiceGroupResponse { + message UpdateServiceSafePointByServiceGroupResponse { ResponseHeader header = 1; bytes service_group_id = 2; bytes service_id = 3; From 15573ff0572aa9fc9c35ca0fe47bc6cb50b27790 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 19 Apr 2022 11:41:55 +0800 Subject: [PATCH 07/14] feature RawKV GC,modify interface response Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index b885bf4b..5a35651e 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -85,7 +85,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 1. Gc worker will call pbclient.UpdateGCSafepointByServiceGroup to update the gc worker safepoint as follows: 1. interface : ```shell - func (s *GrpcServer) UpdateGCSafepointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupGCSafePointRequest) (*pdpb.UpdateServiceGroupGCSafePointResponse, error) + func (s *GrpcServer) UpdateGCSafepointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupGCSafePointRequest) (*pdpb.UpdateGCSafepointByServiceGroupResponse, error) ``` 2. added related pb info in pdpb.proto @@ -105,7 +105,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 2. used to get GC safepoint for TiKV: 1. interface: ```shell - func (s *GrpcServer) GetAllServiceGroupGcSafePoint(ctx context.Context, request *pdpb.GetServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGCSafePointResponse, error) + func (s *GrpcServer) GetAllServiceGroupGcSafePoint(ctx context.Context, request *pdpb.GetServiceGroupServiceGcSafeRequest) (*pdpb.GetAllServiceGroupGcSafePointResponse, error) ``` 2. pb info ```proto @@ -123,7 +123,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 3. Used to update service safepoint for CDC/BR/Lightning: 1. interface: ```shell - func (s *GrpcServer) UpdateServiceSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceGroupServiceGcSafeResponse, error) + func (s *GrpcServer) UpdateServiceSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceSafePointByServiceGroupResponse, error) ``` 2. pb info ```proto From a86c475bb5b2b591752ccdbfc289eef3dc26cff6 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 19 Apr 2022 11:46:54 +0800 Subject: [PATCH 08/14] feature RawKV GC,modify interface response. Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index 5a35651e..4fef07f7 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -85,7 +85,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 1. Gc worker will call pbclient.UpdateGCSafepointByServiceGroup to update the gc worker safepoint as follows: 1. interface : ```shell - func (s *GrpcServer) UpdateGCSafepointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupGCSafePointRequest) (*pdpb.UpdateGCSafepointByServiceGroupResponse, error) + func (s *GrpcServer) UpdateGCSafepointByServiceGroup(ctx context.Context, request *pdpb.UpdateGCSafepointByServiceGroupRequest) (*pdpb.UpdateGCSafepointByServiceGroupResponse, error) ``` 2. added related pb info in pdpb.proto @@ -105,7 +105,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 2. used to get GC safepoint for TiKV: 1. interface: ```shell - func (s *GrpcServer) GetAllServiceGroupGcSafePoint(ctx context.Context, request *pdpb.GetServiceGroupServiceGcSafeRequest) (*pdpb.GetAllServiceGroupGcSafePointResponse, error) + func (s *GrpcServer) GetAllServiceGroupGcSafePoint(ctx context.Context, request *pdpb.GetAllServiceGroupGcSafePointRequest) (*pdpb.GetAllServiceGroupGcSafePointResponse, error) ``` 2. pb info ```proto @@ -123,7 +123,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 3. Used to update service safepoint for CDC/BR/Lightning: 1. interface: ```shell - func (s *GrpcServer) UpdateServiceSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceGroupServiceGcSafeRequest) (*pdpb.UpdateServiceSafePointByServiceGroupResponse, error) + func (s *GrpcServer) UpdateServiceSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceSafePointByServiceGroupRequest) (*pdpb.UpdateServiceSafePointByServiceGroupResponse, error) ``` 2. pb info ```proto From 552fbd64128dd9aef30def90388c639b0d75fde3 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 19 Apr 2022 11:49:37 +0800 Subject: [PATCH 09/14] feature RawKV GC,modify interface Safepoint to SafePoint. Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index 4fef07f7..dcc3bbcb 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -82,21 +82,21 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. ``` 3. design interface to standardize the interface: the interface as follows: - 1. Gc worker will call pbclient.UpdateGCSafepointByServiceGroup to update the gc worker safepoint as follows: + 1. Gc worker will call pbclient.UpdateGCSafePointByServiceGroup to update the gc worker safepoint as follows: 1. interface : ```shell - func (s *GrpcServer) UpdateGCSafepointByServiceGroup(ctx context.Context, request *pdpb.UpdateGCSafepointByServiceGroupRequest) (*pdpb.UpdateGCSafepointByServiceGroupResponse, error) + func (s *GrpcServer) UpdateGCSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateGCSafePointByServiceGroupRequest) (*pdpb.UpdateGCSafePointByServiceGroupResponse, error) ``` 2. added related pb info in pdpb.proto ```proto - message UpdateGCSafepointByServiceGroupRequest { + message UpdateGCSafePointByServiceGroupRequest { RequestHeader header = 1; bytes service_group_id = 2; uint64 safe_point = 3; } - message UpdateGCSafepointByServiceGroupResponse { + message UpdateGCSafePointByServiceGroupResponse { ResponseHeader header = 1; bytes service_group_id = 2; uint64 new_safe_point = 3; From 03e416f7d68bfb428c43a0a8eb6c91ec55826fa3 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Tue, 19 Apr 2022 13:10:40 +0800 Subject: [PATCH 10/14] feature RawKV GC,modify interface name in desc Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index dcc3bbcb..e1a9650a 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -144,7 +144,7 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. ``` 3.Changes on TiKV: -- Get GC safe point from PD by getGCSafepointByServiceGroup interface. +- Get GC safe point from PD by GetAllServiceGroupGcSafePoint interface. - For API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. - GC conditions in RawGCcompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). - If the newest version is earlier than GC safe point and it's delete marked or expired ttl,those keys and earlier versions of the same userkey will be sent to a gc scheduler thread to gc asynchronous. From dcf16a3cb4e9d7ca6c49a2f49bc5a0c05e15a20c Mon Sep 17 00:00:00 2001 From: ystaticy Date: Thu, 28 Apr 2022 10:09:14 +0800 Subject: [PATCH 11/14] RFC:TiKV RawKV MVCC GC.update pd interface Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 124 +++++++++++++++++++++++++++---------------- 1 file changed, 78 insertions(+), 46 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index e1a9650a..adcacd16 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -84,65 +84,97 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. the interface as follows: 1. Gc worker will call pbclient.UpdateGCSafePointByServiceGroup to update the gc worker safepoint as follows: 1. interface : - ```shell - func (s *GrpcServer) UpdateGCSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateGCSafePointByServiceGroupRequest) (*pdpb.UpdateGCSafePointByServiceGroupResponse, error) + ```proto + rpc UpdateGCSafePointByServiceGroup(UpdateGCSafePointByServiceGroupRequest) returns (UpdateGCSafePointByServiceGroupResponse) {} ``` - 2. added related pb info in pdpb.proto + 2. PB messages: ```proto - message UpdateGCSafePointByServiceGroupRequest { - RequestHeader header = 1; - bytes service_group_id = 2; - uint64 safe_point = 3; - } - - message UpdateGCSafePointByServiceGroupResponse { - ResponseHeader header = 1; - bytes service_group_id = 2; - uint64 new_safe_point = 3; - } + message UpdateGCSafePointByServiceGroupRequest { + RequestHeader header = 1; + bytes service_group_id = 2; + uint64 safe_point = 3; + int64 revision = 4; + } + + message UpdateGCSafePointByServiceGroupResponse { + ResponseHeader header = 1; + uint64 new_safe_point = 2; + bool valid_revision = 3; + } ``` - 2. used to get GC safepoint for TiKV: + 2. used to get all GC safepoint for TiKV: 1. interface: - ```shell - func (s *GrpcServer) GetAllServiceGroupGcSafePoint(ctx context.Context, request *pdpb.GetAllServiceGroupGcSafePointRequest) (*pdpb.GetAllServiceGroupGcSafePointResponse, error) + ```proto + rpc GetAllServiceGroupGCSafePoint(GetAllServiceGroupGCSafePointRequest) returns (GetAllServiceGroupGCSafePointResponse) {} ``` - 2. pb info + 2. PB messages: ```proto - message GetAllServiceGroupGcSafePointRequest { - RequestHeader header = 1; - bytes service_group_id = 2; - } - - message GetAllServiceGroupGcSafePointResponse { - ResponseHeader header = 1; - bytes service_group_id = 2; - uint64 safe_point = 3; - } + message GetAllServiceGroupGCSafePointRequest { + RequestHeader header = 1; + } + + message GetAllServiceGroupGCSafePointResponse { + ResponseHeader header = 1; + repeated ServiceGroupSafePoint service_group_safe_point = 2; + } ``` 3. Used to update service safepoint for CDC/BR/Lightning: 1. interface: - ```shell - func (s *GrpcServer) UpdateServiceSafePointByServiceGroup(ctx context.Context, request *pdpb.UpdateServiceSafePointByServiceGroupRequest) (*pdpb.UpdateServiceSafePointByServiceGroupResponse, error) + ```proto + rpc UpdateServiceSafePointByServiceGroup(UpdateServiceSafePointByServiceGroupRequest) returns (UpdateServiceSafePointByServiceGroupResponse) {} ``` - 2. pb info + 2. PB messages: ```proto - message UpdateServiceSafePointByServiceGroupRequest { - RequestHeader header = 1; - bytes service_group_id = 2; - bytes service_id = 3; - int64 TTL = 4; - uint64 safe_point = 5; - } - message UpdateServiceSafePointByServiceGroupResponse { - ResponseHeader header = 1; - bytes service_group_id = 2; - bytes service_id = 3; - int64 TTL = 4; - uint64 min_safe_point = 5; - } + message UpdateServiceSafePointByServiceGroupRequest { + RequestHeader header = 1; + bytes service_group_id = 2; + bytes service_id = 3; + int64 TTL = 4; + uint64 safe_point = 5; + } + + message UpdateServiceSafePointByServiceGroupResponse { + ResponseHeader header = 1; + uint64 gc_safe_point = 2; + uint64 old_service_safe_point = 3; + uint64 new_service_safe_point = 4; + } + ``` + 4. GC Worker call pdclient.GetMinServiceSafePointByServiceGroup to get min(all service safepoint): + 1. interface: + ```proto + rpc GetMinServiceSafePointByServiceGroup(GetMinServiceSafePointByServiceGroupRequest) returns (GetMinServiceSafePointByServiceGroupResponse) {} + ``` + 2. PB messages: + ```proto + message GetMinServiceSafePointByServiceGroupRequest { + RequestHeader header = 1; + bytes service_group_id = 2; + } + + message GetMinServiceSafePointByServiceGroupResponse { + ResponseHeader header = 1; + uint64 safe_point = 2; + int64 revision = 3; + } + ``` + 5. GC Worker call pdclient.GetServiceGroup to get all service group id: + 1. interface: + ```proto + rpc GetServiceGroup(GetServiceGroupRequest) returns (GetServiceGroupResponse) {} + ``` + 2. PB messages: + ```proto + message GetServiceGroupRequest { + RequestHeader header = 1; + } + + message GetServiceGroupResponse { + ResponseHeader header = 1; + repeated bytes service_group_id = 2; + } ``` - 3.Changes on TiKV: - Get GC safe point from PD by GetAllServiceGroupGcSafePoint interface. - For API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. From 89194ba74598aa75c3d998a3a4f15519c089aa50 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Thu, 28 Apr 2022 10:12:04 +0800 Subject: [PATCH 12/14] RFC:TiKV RawKV MVCC GC.update etcd path Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index adcacd16..4a3a2bc2 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -5,8 +5,8 @@ Move TiKV MVCC GC worker from TiDB into a group of independent GC worker component and implement a new GC process in TiKV for RawKV. ## Motivation -1.GC worker is an important component for TiKV that deletes outdated MVCC data so as to not explode the storage. But currently, the GC worker is implemented in TiDB, which makes TiKV not usable without TiDB.And current GC process is just for transaction of TiDB,it's not usable for RawKV. -2.Standardize the API used to set and obtain GC status in PD to improve the developer's experience. +1. GC worker is an important component for TiKV that deletes outdated MVCC data so as to not explode the storage. But currently, the GC worker is implemented in TiDB, which makes TiKV not usable without TiDB.And current GC process is just for transaction of TiDB,it's not usable for RawKV. +2. Standardize the API used to set and obtain GC status in PD to improve the developer's experience. We change RawKV encoding to MVCC, so the GC is necessary. No GC for TxnKV scenario when TiDB is not deployed. @@ -74,11 +74,11 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. - the safepoint data path in etcd of PD,will be changed. The new safe point path in etcd as follows: - gc_worker safe point ```shell - /gc_servicegroup/$service_group_id/safe_point + /gc_servicegroup/gc_safepoint/$service_group_id ``` - CDC,BR service safepoint ```shell - /gc_servicegroup/$service_group_id/service/$serviceId + /gc_servicegroup/service_safepoint/$service_group_id/$service_id ``` 3. design interface to standardize the interface: the interface as follows: From 3489396bb3f87f9d6cbf9f89f03ffc2521b8c75a Mon Sep 17 00:00:00 2001 From: ystaticy Date: Thu, 28 Apr 2022 10:22:17 +0800 Subject: [PATCH 13/14] RFC:TiKV RawKV MVCC GC.update compactionFilter name Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index 4a3a2bc2..48fb7e64 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -177,8 +177,8 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. ``` 3.Changes on TiKV: - Get GC safe point from PD by GetAllServiceGroupGcSafePoint interface. -- For API V2, we need add new CompactionFilter which is named RawGCcompactionFilter, and add a new GCTask type implementation. -- GC conditions in RawGCcompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). +- For API V2, we need add new CompactionFilter which is named RawCompactionFilter, and add a new GCTask type implementation. +- GC conditions in RawCompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). - If the newest version is earlier than GC safe point and it's delete marked or expired ttl,those keys and earlier versions of the same userkey will be sent to a gc scheduler thread to gc asynchronous. ![raw gc copaction filter ](../media/tikv-rawkv-gc-compactionfilter.png) From b2117691eecb05a8d0b31e95cea3e05ba8657fa2 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Fri, 29 Apr 2022 13:45:53 +0800 Subject: [PATCH 14/14] feature RawKV GC.modify pb info Signed-off-by: ystaticy --- text/0090-tikv-gc.md | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/text/0090-tikv-gc.md b/text/0090-tikv-gc.md index 48fb7e64..d9218bd3 100644 --- a/text/0090-tikv-gc.md +++ b/text/0090-tikv-gc.md @@ -106,15 +106,15 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. 2. used to get all GC safepoint for TiKV: 1. interface: ```proto - rpc GetAllServiceGroupGCSafePoint(GetAllServiceGroupGCSafePointRequest) returns (GetAllServiceGroupGCSafePointResponse) {} + rpc GetAllServiceGroupGCSafePoints(GetAllServiceGroupGCSafePointRequest) returns (GetAllServiceGroupGCSafePointsResponse) {} ``` 2. PB messages: ```proto - message GetAllServiceGroupGCSafePointRequest { + message GetAllServiceGroupGCSafePointsRequest { RequestHeader header = 1; } - message GetAllServiceGroupGCSafePointResponse { + message GetAllServiceGroupGCSafePointsResponse { ResponseHeader header = 1; repeated ServiceGroupSafePoint service_group_safe_point = 2; } @@ -137,8 +137,8 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. message UpdateServiceSafePointByServiceGroupResponse { ResponseHeader header = 1; uint64 gc_safe_point = 2; - uint64 old_service_safe_point = 3; - uint64 new_service_safe_point = 4; + uint64 old_safe_point = 3; + uint64 new_safe_point = 4; } ``` 4. GC Worker call pdclient.GetMinServiceSafePointByServiceGroup to get min(all service safepoint): @@ -159,24 +159,24 @@ For support RawKV GC in TiKV cluster deploy without TiDB nodes. int64 revision = 3; } ``` - 5. GC Worker call pdclient.GetServiceGroup to get all service group id: + 5. GC Worker call pdclient.GetAllServiceGroups to get all service group id: 1. interface: ```proto - rpc GetServiceGroup(GetServiceGroupRequest) returns (GetServiceGroupResponse) {} + rpc GetAllServiceGroups(GetAllServiceGroupsRequest) returns (GetAllServiceGroupsResponse) {} ``` 2. PB messages: ```proto - message GetServiceGroupRequest { + message GetAllServiceGroupsRequest { RequestHeader header = 1; } - message GetServiceGroupResponse { + message GetAllServiceGroupsResponse { ResponseHeader header = 1; repeated bytes service_group_id = 2; } ``` 3.Changes on TiKV: -- Get GC safe point from PD by GetAllServiceGroupGcSafePoint interface. +- Get GC safe point from PD by GetAllServiceGroupGCSafePoints interface. - For API V2, we need add new CompactionFilter which is named RawCompactionFilter, and add a new GCTask type implementation. - GC conditions in RawCompactionFilter is: (ts < GCSafePoint) && ( ttl-expired || deleted-mark || not the newest version ). - If the newest version is earlier than GC safe point and it's delete marked or expired ttl,those keys and earlier versions of the same userkey will be sent to a gc scheduler thread to gc asynchronous.