From 825cba51d720474a303b2d17768a63c6cf5d87f0 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Tue, 29 Nov 2022 12:07:58 +0000 Subject: [PATCH 1/9] querier: Avoid global sort for dedup when possible. * Proposal * Removed deprecated fields from internal proxy series usage. Signed-off-by: bwplotka --- cmd/thanos/query.go | 7 +- cmd/thanos/receive.go | 8 +- cmd/thanos/rule.go | 8 +- cmd/thanos/sidecar.go | 8 +- cmd/thanos/store.go | 8 +- docs/img/bottleneck-globalsort.png | Bin 0 -> 84035 bytes docs/img/globalsort-nonoptimized.png | Bin 0 -> 14336 bytes docs/img/globalsort-optimized.png | Bin 0 -> 18482 bytes .../20221129-avoid-global-sort.md | 171 +++++++ .../querier/queryrange/queryrange.pb.go | 1 - pkg/dedup/iter.go | 31 +- pkg/dedup/iter_test.go | 6 +- pkg/info/infopb/rpc.pb.go | 92 ++-- pkg/info/infopb/rpc.proto | 8 +- pkg/query/endpointset.go | 12 +- .../test-storeset-pre-v0.8.0/storeset.go | 2 +- pkg/query/iter.go | 7 +- pkg/query/querier.go | 164 +++--- pkg/query/querier_test.go | 343 +++++++++---- pkg/query/query_bench_test.go | 2 +- pkg/query/query_test.go | 24 +- pkg/query/{test.go => test_test.go} | 42 +- pkg/receive/multitsdb.go | 4 +- pkg/store/bucket.go | 45 +- pkg/store/bucket_e2e_test.go | 22 +- pkg/store/bucket_test.go | 16 +- pkg/store/prometheus_test.go | 2 +- pkg/store/proxy.go | 11 +- pkg/store/proxy_heap.go | 118 ++++- pkg/store/proxy_heap_test.go | 106 ++++ pkg/store/proxy_test.go | 480 ++++++++---------- pkg/store/storepb/rpc.pb.go | 233 ++++++--- pkg/store/storepb/rpc.proto | 21 +- pkg/testutil/teststore/cient.go | 25 + 34 files changed, 1294 insertions(+), 733 deletions(-) create mode 100644 docs/img/bottleneck-globalsort.png create mode 100644 docs/img/globalsort-nonoptimized.png create mode 100644 docs/img/globalsort-optimized.png create mode 100644 docs/proposals-accepted/20221129-avoid-global-sort.md rename pkg/query/{test.go => test_test.go} (93%) create mode 100644 pkg/store/proxy_heap_test.go create mode 100644 pkg/testutil/teststore/cient.go diff --git a/cmd/thanos/query.go b/cmd/thanos/query.go index 8aa6d28bf6..bdff403bfc 100644 --- a/cmd/thanos/query.go +++ b/cmd/thanos/query.go @@ -743,10 +743,9 @@ func runQuery( if httpProbe.IsReady() { mint, maxt := proxy.TimeRange() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, } } return nil diff --git a/cmd/thanos/receive.go b/cmd/thanos/receive.go index 3e89a2e170..222135fd5e 100644 --- a/cmd/thanos/receive.go +++ b/cmd/thanos/receive.go @@ -326,10 +326,10 @@ func runReceive( if httpProbe.IsReady() { minTime, maxTime := mts.TimeRange() return &infopb.StoreInfo{ - MinTime: minTime, - MaxTime: maxTime, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: minTime, + MaxTime: maxTime, + SupportsSharding: true, + SupportsWithoutReplicaLabels: false, // TODO(bwplotka): Add support for efficiency. } } return nil diff --git a/cmd/thanos/rule.go b/cmd/thanos/rule.go index 0bd29b905d..1ee58cb5bf 100644 --- a/cmd/thanos/rule.go +++ b/cmd/thanos/rule.go @@ -625,10 +625,10 @@ func runRule( if httpProbe.IsReady() { mint, maxt := tsdbStore.TimeRange() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, + SupportsWithoutReplicaLabels: false, // TODO(bwplotka): Add support for efficiency. } } return nil diff --git a/cmd/thanos/sidecar.go b/cmd/thanos/sidecar.go index 7d6da14d1a..1fbdd35099 100644 --- a/cmd/thanos/sidecar.go +++ b/cmd/thanos/sidecar.go @@ -268,10 +268,10 @@ func runSidecar( if httpProbe.IsReady() { mint, maxt := promStore.Timestamps() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, + SupportsWithoutReplicaLabels: false, // TODO(bwplotka): Add support for efficiency. } } return nil diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index 30df09ba5e..cb2a953f0f 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -411,10 +411,10 @@ func runStore( if httpProbe.IsReady() { mint, maxt := bs.TimeRange() return &infopb.StoreInfo{ - MinTime: mint, - MaxTime: maxt, - SupportsSharding: true, - SendsSortedSeries: true, + MinTime: mint, + MaxTime: maxt, + SupportsSharding: true, + SupportsWithoutReplicaLabels: true, } } return nil diff --git a/docs/img/bottleneck-globalsort.png b/docs/img/bottleneck-globalsort.png new file mode 100644 index 0000000000000000000000000000000000000000..bc1c59a28f24ea30b2e59868a279625c723a1599 GIT binary patch literal 84035 zcmb?@cRbbKAHVHdx%RbP%E~4q*SdBk6r~84N@Y`A+1$&;MH{Ir8X6LneXV=VGSaZu zMO4VDH-#Qe!D4(Mx|9PyWE_3ErMA7m{uW@VD;^b&+)oR2xHR@hy z?;v^5cX4yL&)0*zVY0iryW^pNfg&Mr4D{m6E{5m#Pe{mhtGzM*{`xCZiZu8u_=822 z>4Y`?lSqi8+O;*le?EJQ>;L7Wj}qRi)pQ&P81Da4bV|GT_c!Y z8#}Q+lNat0v;Meddu7Nwntn-&lJpCT==|36&~9#Px!a^TW_zPJY<@7{<0IRJvDn=! zvAf$9HLIhM-kWZa6GbkLGVhA3rtiAc({J*IHMqy?S#FnmECY77)>#Pz%QWHpU&h|} z*!%8|?e6%jn^JZj+cl>Ojl?lB4-c04j>u&q_-94v1GHqb1%WT!IQ8ayPUZUc>X_<` z!W_GL1|eqSyNd_^a1fa%V0+`&omfor%*%?=kdf*uY5pgV4VL@Qj>hgrhp$ml6+Yck z0#V_k5k7u~^pO_KXv&1jE?yXT{WwL8G`LVXnc-5OY+OE689aGvQ`UwmfgQ@@$|#Fi zJ|8~UuPLA_=#0C1tC+ZSXqC)s(iD>E`Rm7*;zPHUOTEvBMp<;F2q<{98i!8fu{-A? zjp)}LVL`v)@D75}5ch5&;i__PGDK#zb+;=lFM8#anF&F5kW;`;_4L~yFm$yXITtod zP-K;4+;+_7yk7W^&q|xojR~x*L)_BF!-HUyzyA30!Uqfc@%h=L{E=+Wf^!3}&n?8W zO9knTTs}TW%#qlguNYIi@bKR1zAQJdzLJoyhlW;D6bLxUlk~ae^ZRidmIsanAY}xV z#<^rJFOH6kgbn$x80KI2>HNxlWao)Q80vA!Oh=CQ_RG^{$`@w)UU|eT(wYT|>ozVe ze~>T;dE{ARZ?gT_!Uk2qnNM=5-}te=;(X`bHyc_8F<;-^?j*knUmQgm*<^XdMCA9$ zfq}-oxejLZlqC8jlv{5U?OhbOzVx*>{O31EfxTO!rQX)FzMr37-?oda^eDUZWjLOh zeYpLm0vCoQ;p>Nd^YZQ+`wqKhsht@HPd%_oml6z>N25uTRs&U`gnj(3eey2JA-05^@u8&t-0Xxd0CTzZ_n=EFEdC=4Mz>L{iVbt&nc)F zhdBl_)9VZ&7a}NkeyIJO#aFsUE%sM3J zNoDG>W@NU2Ogytn7d!D>tnp(__+o;8T#0UdEDCjz(B~f6Sx!rPb^dA;(QGccbwA<> zeF16xvw*lI^gj?)9RqCz$11#AGu|lYUzr~)d+>n4m%&l@?pQ^@$Pf3}?JJwn)bYl0 zRd*SLmh+1f>Gw@Iod>oexb|Nd7u7zG_n!<+)iN@k$k^Ulo6K~{`*ZvX9sj}4y&vHx z+4*Q-KAm>weq6*S2obX&B&IY;ViYXZCXEO zW>3BgX=i)$rM_AJ%*Fna{D{b?{G=n9vELtOoqkFPp%*{gvN?TaoQ8@*WJ$w{^3WBL z)+>EyW_C7a)dduBD$iT_Cala@mV%xf$?@D>mne^q0>!8o7 zM`FE$a3bD8FK^~1PsMP^`%`IIN%>*uO!p{SbAL?Ejh{3uxHZ}?eMZI-uwmr}(|Vu<2k z8Ah%Q_|WU5+gM%nnbn1nGlfU*yMcK=9KAX+vrxAg5Z)>oUYW!rUY#3I-F_x^`&Tho zrp1@OJYRVIUM*DQ%w@;9sDU6_vq;ThwW=oPBA1>sG3#H8k3BxLkh;G(=%jnZ;}<7h z-mUrZ{KYUh`8+7(VkcjmP&M4;(HkBQAkwVF?Lo;MQLPyEnyi^GA99;)Nk8-HiJgB( z=iLLLpsbbnUik9-HhnvozdQK$;-nOMoN@JoId2u8oK>Z@R7BI2x&EHw8ZdK;efx@w zT`K~LeYy$7CttZ2gBqf%XK%dzDDua%Lbu(W?U4s&iNoWKi6h$Gh)Hl)(@DJ23xwF6 zh}`a;ht@Tk-YuYPw^WIEMD^HZYc$qwExiv|nrhcw1Q!&leZHe)Q^U7cMK658O)qd1 zQ4zfvrWd=j1>+J{(NDv2iL{`%fAw{H^%Ph(zedw0-ni6WEi;mXgG*Drb?*J0X7psb zo|lS89nnSCfAFk{fwcGJ!JL}|Z+wOjc6r7l)AS8cHxE(Mp&8mq)ZRt34hs;@EqoHN zN1#?0Xsyx}GaIF?QlXi~8y6aDW7a6i^P=OY#;eBH$4^}xB~}@kAPyr=>WxLcd2#X; z!6+P&h}cEsMisAdQ@NA74>&z_Xf#$p)F56V@(?#csS2APs;Hp-{Qjwx@OXw0*;Rb; z-7TfCk&Z>zz}m0neghN68{gzszh-$<=f!NagAaETzEa%)`1p+|o_Wsomo9YVXrDh) zyWC~C07~1H#roYkQD^Y!XP`D#q&!@#+w9r~Tl_+wN9>iz)rAA*vtaZWUe)h}SZ7LI zQ8;=ph`!ewUiooLl6Plw(PR}IL3k^AmmZB{jK0kP_Zxsw( z4X?@8hvJCY-k0Ycy!^C9IvT65%nsi@;M=hlyWFWak}9;n1KP@9DTwx#px{skeYzy{ zLZ>^J9!eyB5uX-NIC@#V_U8@J*BzT&>cnwSaL>tdG53joH&5c$G>;x{h+p_#n5XM~ zKNMAB{T*y~+!z;#)n!Q1Zk1`jCF%$S3$^e>{ruJ=$0jSs0zW0u!8a6Q+|qJIO93J} zono-ebkm>D`1f;=(+JLk#t)k>D0J_DX*yT;>U7yV%#+xi`n3tJ)n9!cL(AmQ zpY^-j-2_#FHld<`uuCGKc>U|{&ktn!T07Si4Fvx`{}{ z-}j|#R0d$eXS(j&MbSh#OeX*PKp7GxN8&4!&u_o;!aC6(?FL2xAU5L^!SK_|a zz&23jd;{AemOI2lM!lAkB+#xgeP_Z?)vT73Y$x&>UkI!+y!_2GaKs?sb3=3GISB}J zOHv_=x3IbJD~OfK1T!y_6$JpYZvG&{usQw14bcEnvbOA>;AzBAQfc4QkKwJk2gu31 zxs~L$ZLn+j)l0{$HAZjFSI9rmNaYf+&(#Yo*DUg=AH`q7U*z^fNJC)UClR{&GhP7B zoPRXQN{VbOn~+4x1ka<>iKb!)FJEZfZq#l-tZ!oAc#Wmi_gtdtHU$ZZ>n&skbXga6 zEF+q@%B3&WI$alG51UBc6#10|pF#%03c4dnL>J1&LoZg%b~j3+*AA30?5pmg8!&uf zSxQJ{CnmJMG7yJa*oPb#^R;NDO}5$v1!eNRLr4z~bsKMX5bpaAl@HMP>W9q<2Ut{t`cYi7a8TQYBT%*@B}f5(LVXT;1@_AQGn|bNi?N6EYvTB%U~eJJO}3vy<7G@g zpv~e|nR*f%(;!GenPbrxB#<19VAQfW_`X^)Nh#vJn~*w~&g241^G=O#CyU)K$|(Ld zfbx4gBS}znz8-g}*i|z0;Hbbf`E%tCiS!v7+9935rP(qpdynsf`(=TaHlvFGs@44J zb>$-UKvXB)`L!a79o|MI`v?af?^nY>fAg5@oEk{lWz0Mhx0T=~2RM$!G5Y>_j4_b1 zF9y>Vt5{CoE^RpJe^HjHKu%cWwq^!>UzHKdO3pywzxP#HA%_z=O^e|F*F8QLpK+z* z1sI*xIGOSvJQqd|gImNjUNxit<`JDRJz{zuWV;2qzFSiAijM1*=uG8VoK{oO6@hXt z_!3ty&kgXgy;%Amg*xsV_6f&w4`HF}th4sud+;0JDj?T6U2scO3c2+6(MdHvUH4MT z{`<5N)|Y2$>UTEw_o`vq)bHy0E7R|S`}bYAR0idPBP%@s1ciQ0O~0=x0T3MeMKLW@ z&#KWs=kh; zDEl4>9Jr#Tw9UnOk3VBl615&u5xcwNQUjH(2kb1FewUiRhrko?sQ`cxe%})fK&hHf zh4!uB@Up)_&9TPZz&a(GoCW7MfKmVY+L!0fbnb}%eYURqhBe0vk2e4HHTq7ovM?g< zD}tO70d(soeRPw4k8aVaH!buU_RlE+ysU}cSyup~_Mhp|TYP5!?^Av%%>3(=08K2c zj>TsFKBWQyO}dYW(E96ZqcNNQt3!eH^vhVc1fbxr!C53OXKaEE&uiY6I-bZY#*3iF z`3{x8^!@ZC{pBIv)sgVLp6V;U;h>s|(O0T3KqkImDL2-Ik6g*}?J2&TfbDQZ zGJKcps3z|;r!xG&GcR0->B(QZ_)SS>L1D{Yp-58`SSFEa6rMUpF6*(R(5_~`R?}q z8nF2d4=SH1`ljskbX=`!ia#7cveshfN9+sgt2vvU(KDh?ou1t=5=5GW5cB60rfP$81y2F>JJGG}fawgAr6Xwr%-TUqy{mWU;yMWU^z}}0vHj&n{Mw{e<=>^=opPSc!?#lWW?2e|e z>RtVLK`&&=W)*;m8VWhN21QP)^2v+t}ulCrfE*!i28Bg^SgC5&;>sQ`8L{ zC7LSAO7W$-tLR?({M1xK*RTIWgW~E0pPXdB6kL>5j*vHc&Br5ytT|9;dsANih=Drk zPOyel3oD*4{(W}s+Sh}X@+?|?tp|c@-By7D;kDcl#SluCU|en{{`%HKp1xLf);-^& zK4zg^A@A+_`2}`v*f`)qXICe4d+~pklBa^cz`dF| zU4yOw)!{@;E|Pmnu2gbV$zm{BI_4W);~2^d27@;;IQ}rxI3zD>DP{BLC;JgF($mJf zTcH(^s{y|OAwY10(Ev9A=3Mv2m%dE>BQ2!iv8JN}SZT#g9@up$;dt!!=4uTy`=0(4 zc3!E=$$KP2l6j@WtTQBn0r)G++u8;fdrEq!#Aq`_BICV*zDR4sWXqz-?$(=40H%fk zVjHj9-H8-em>hcJlezih#j827IskwweEZaD>h2qcS7(~+^w0N~l8f(0c)Ir|R{<#_ zZRqXAo1LPetrCC=bSZCYR&;328fuR1KUu7k9lOB17D=+_-L%+U8M@j(CY?6PG(**p z&)_3f7Q}AOZ)>KCJ(QO2{_yef;Sr!141*eLCqpJ-x#mTYb9I10$-f?mSs1R;%ujzn zIC$kb#~ctDax1NzZ%4-l#!)tA3d)NQZ0 zZW?YC-C}9EJkgW_>MU}KuHIQF_Y)&RpGCw_4|-O5Kd`^6^K4wTE;ccf@7vg*w}zL# zm#LS{M#ZnV(suDkZu`-}O}$kq{BeXj?D>veU2N_mr4m*~$zAhnWd0hv8n{~Pqa;AI zR_1IS-zbU7FePBJpORAtsW6DA3HoN(ns}6foi2;cQ$&qv06F+YL6>@VK9%78>N^ap3 zPJRGs)*&%7CO~PVretv|@)%SGa#!Og4`q^7$)sCBPz|#@sWcTuA>LtWNz3L0yZXO- z8zbrh#tl1f8+ZM%jR@r=I+_#%e<$k>m7_v0Lx?5)W-i|tf~`?J9#e6{iXl&<+y$93 zqZE;He9jbu!%PW>=o_ryKcE95W}tzJDCA7rvP^{0oJaxDObwPFhKjV6aa!yz z^#=PLrj4!-;GE*t;628E_DrMKmY?LZYyt7Ec>}8$NY#qldRS7sSEfLb$OmA_e{f(E zuTTqVWv`4I4&$eM?Rp{K3YX*PgF$KMInv4v(;hbEJxTNb1^b==%-=~CC|6*t5&)$C zHD_qiB>&0`e~6CFEd^o*>k;zKa{RqL8Se)T&Yg=tzT~K=Xr^T{J8>t*s!VyzeA>>B9E0gd}Ei6 z^8A`gu5&{kP<{T+u|4BTA_-jIuQM)fx3D@&LlP@lx=y^_JmSJ;k1uOKeoXea7{El; zxTQ$<>P}SAA=-a{3V=L|bzu8-0mpst!&*A}>}(k(x{4(j+a_C>PCVB9(;$OQygopf%d+6o_5UnqJME|<&?8UVi(jMAv zZXmuoI;~dpJUf@4*2%6cD^#3J91a2y_oNnZGWD+%w0PQXKYRI5=&xz4Zbe#o^gv8>T?r^aX}`YSpi46VZsuyg z8zX>#b>Fd$laS&2s7DW$ z$aG!wrpgwyTc}UKDDbwj^O12LpzKa3O^#mvWVesscA)F94G`gnfzJ|%x-s>d0e<(^ zPq2f%dEs7rwXwW8J@?}b#F&r<2*iN4{Z}(m(-XVg^Q&@**o&q)5y)*&I;hjRfrAII ze`?2rx6QmsvjEr6-PO6sKl@&o$P8N)l#z+b^v%)qPVxm>)XT49?jJoG9~=B$TMTLhh5t$>Spco9VcrjVg`oL-pXc~153|1HodH(M99GRb-jHH+jE+*`aUYstnMPrID@Ltv-UkMSi z7^8sy@_2B=n}(gAs3L)|mi0FAC6qmE8mE$R`x!jL6TQ@~Fu%d)jm7eB{K}{lPBjhk zhnO}==6j$|JS8Sm8@v`v8f4TYlX zbz%oDojstAo~6dYPl$?FIYACs3^k<)*o~L)%=-_%EuDv+fs{%SC|+ zBJmm;8EL2^n*dRDzIUt&IXA>^KJW^ga6O|CJFiI?7#07Hv8PxzWG{@=RABAlsdOcn zOZ6ZS4?D%(xiV%&j8uyln34_>WDzP5KI!KxKukQ1`#9Bpvy&?cXNH@IM?S#zxeDT? z8Ns58?JsS+p0XnF1KY~p&0nQO*@1Afh`I}h)I*=(c!;O*iL;hE+<;z8-A4#dkihuMQQv#RU7J5=$WNKb-7Dp7*;DQsc*A|Yp4v?5PIc>?|sY6(GZWN(rX z=O439u=hxoXJIPFV*U~!8iqDLaz{>fBG^tRvhjFD4>U{^z|yD2)>x0zFxL)a%eh7JV3>6ma*2LeU-TLeH zSIEQE6H4Toc(V)!^n6*LN;2zWM@zc6-Che;!3j^G@amSEC9*XaS19Vo0$_T?W_-k~ z&2sr9l>P1G$ix<9C$7Z2A;<$QtoSYNBzK_Lq*=d=BGewDY9St)tdJ6hutpD89z{}H zG9-ORU-(V4z8cQf^i~nPt|@okz{FeWxu@lm$qYgk6=5Meac8sZ4N2c1Po(`ZN4F@U z8Eh^=BZyVhALOg(Pn#6+;tsWWjg8{xK8X(+{UMNnrNZ@C;uDiOc)9wl)`U>U(a{>{ zc1cm-T3|Wg1bwywM(?}cCYy6_cU$IR8_ z$V3Gku^^~p06!F%nasW@Lcadv-Mx&0Ne2YU~Y#>rfIY6Q4%V5{|3+6mlje*1$ zOk|6Q?ZcKRlU`6Sf!#9RoEx=ue1O|qTy9vztbvTIU)?+2Ze~gmZHUX99vFvDRKl7B zxn%UP*K`mErcz~Oi3~{McqgTQiV2Q`9&=MzPK7>{?7<{ADbNINSU&9qGK1X%1}hD) z&dt%a&1M-qw!7$~P{R%muWLaEGuk6kIS=eb@GNqK;!I6Hl=%&OOchyk6E8Qf407Z4 z!^p9+LV1k?xgEuY4i0p14TX;2q!9s_=IgoUYJ&M;liku{olu)sK+XoQ&LuQz4qdL~aNpLJ~HGXHCpY zJv$|dYP#FL@#kh309&XVJ3lhksX z?#w|wFQ!cHp6#Cw#HK5;;gONVVcki#1wE+) zEFx0;Bz2k&`~i$zHtm`bF@{oSd*gkH-t-Mb(O=uZt@5=-f%q8(m1sA9Y6IPDVc~7j z3tXWcl{vblE4aVm6S;iqBlNW;R!f>Q+05RUlg)4xzT`q(aNywa&heIe4YlZLLNA%? z7r*kdA{h?_?LkPO(NOUi+R+T#5F=1r8gx|soQ+Jy%a7l&bdCp~!3k zSKP8p!k#RX%5i!N@gB>NZpmfKUlm|#XVd_jw%lkLpUl!@u^_yDI8MX8f9&&>x3C25glZkm3@EX?9_+?gJUMBgQmim0cI8p91EYj=NTXsuEQe4wa z-3(avr+1V1I!n8)@fgpJ(11nMxCHLi1zhiXLXc@TAF26D&|IjbbbNvhNe-zfg>peZ z$QHL?g$KaCcks$84@(1)>>?!z?Etmhcj~jw-jrauRH!~K2yp_<#t}&tz{$-&{m(Bjd9kZ=MQGp3X&JhCxW`3s#&Chl zPOX`jB#?RHuaiMd_O>U_+cxH2ZL)Vfmvxw#`}|Tu8#5B``4(`4ZQ-OV#cmJW;@Fvb zEKkiGAv!?<0(MF;rH6M-4gjA-sPnXRWi%b+f?!tB1J>u)_(wOTtqJI17V|d$8EVoySoy|{_j=YDya#o4EuMahXWTP zFJjK`07)|dL^?_VNB>&+QN>;MzsH;?)LCZHPyweJ)AIOnmv|2ZrK>>e>rNE(EbyZt^rm1%I!`)1ZrzB(CmuA z;$OINSS{sO))k0q)Ob9*$}knka>g$*Q!%&l+$#d^`vMEw&%+ct26$=l>9R|ub$$f=i@cFAAhOTUo!;61f~*j(oGiqG5Ka6 zWc;XXB@EwtU%HL}IQmkt$#9W4X}m=>2U(dtdcYhWQX8&+o zQ~?Ecq`)%GF9Qm}a3l@gKId{4M0XkkfawwdI9K^JKax#d+Dn8l82Hz#J=W4yow<5L z(YpYg31YEv#}FGJc4i#Qg+e^1mKAk4dsR^8^-1{&UAaoj_PNg(xfRPUo2=k6= zh+{0(v_f&y9WN~TI=lIVK@}3B*u-rh>u1d|3(S}5GT**IV(3vKFPabXY%-90p1Nik zUxD4O>UwCMIT-B*fB$OuZDvrlDx8)!t<*NAwfJB`|Ha| ze?w%%XwxO z&BTPV*uyWy)5J0}(~nlwc%nH-UcRbvmCqQ29o3=Im|{M8Z1w~K&&SI zve{q&4Y-APLUPf?%D@cWg%H}98TfE@*!W07t$qTJasfTwCA`#9i=4B_Y~Io| zCYs{%scxo>EK70JV)$jKA4H#M`8NFf$9-}jZ+8D)0ia^zj+WQ{BIpdX#75zA1r96q zbvaZ`YveKNpp5W1-QNR+ISj-@Kb^9A+@00949o)kLg~aTy$z7m(exI4#o%0{(L{SU z-pR3iXe~E7+OUAVo1HCG--?&O(vm_Y;VeJ%g*6-F4yj*&DYXCUsHop#!6($Du>+98 zxk1PENG zD+1=}HdhVXg28G|bApn7!!LnjS+)k@USYB1dG^?@Bx zfdVGY1bAl^$4l~l^B689b{?vP6XFg)Rzn%ldD*TOQlW{BM+1%qFE=4>@+O*^I-L!T zs*jal)k6e2%3O`u(`I`KsEH|h1nmVwcauRXH#{tRKjIWbgP4GZJoMcjS0z16%%=6R zLC$zgOVG$=6Q_Q9x283%xUE)UjtdgRiM{~jd2Xvr8~wFw!wv)Z(6-#L9|{IVf7;*3 z6K}FMp16`UD!vNCqivxkkUe-e?%{8mP*JBD(zqqJOookGAyGprpL!1tSoc}Hq+z+K z6GgS*aJ-q_nlPKuB%F4i#usM2z~@ccoJ@}Mkz3}| z;(|nz>?nEzX3@5e5M3{t=P(bqW`MlK( zAh0j!^)>cWKRZ@IucuKdgxBgQJVLL-OtHm0+ndF!pP+h=y_>iDcs~fr|C?5Unh3H( zhHkPX3X%={G80M~94u9!gILFuLJX3H_@y-sSqZBW*-B?j4I1w>WWwg*gz+fS&Fb*Q z0JsZAZk>1y4A4|}kQDwqh^18n>~!j*N#Le|vGWkr88ZSu2$S|l3&KL*_A^MydgzMU z^AlK+WQ_*~U=3a2WN~nekjKWOO>w9oZ9RzbVC4x5jshq|d#@fRqt!8EuastI(fy;` zODCcNxTP|VOmk?)tu^Li!}ouhG=H+1wa(q`3d6kTTfy~l1WJzTOlJkzSTIf^@j=qt z#*pI}_HKxQb>017-~un-S#C0pF&5>`zk>cml?sj`9-iX0G#`%Ir@uRl@lGd&rrB^0 zyFGZ8q<80G{WvIMpOXlgN^WcN zw1kY)%owO_QkjQzKBr#W$GtIuKPKbEtNF1gqlMxr6Z+J>12h=Ydf@3~U|0 z&=fvb0=v=l{3S(>Qb=) zqjeW_Q~`LMDwGv`PGo~u2_*l5v=_vlx7h$X3%O)*zg`cgH=XR$G2>*JnYMa;OLbZX z5h!RJc~8NwTi8Ow`}Utal^cYX7@U?Y>&M%O+xr>sCiOgnc3zqmSI7vDFHuq{WKutb z;<~=!ZMDDpK39gB8YBrp5I}ICBMW>#KxLD%&CYt3OkSJ_46$m0cd#p0pG?*C1f2_Nj^(-+(B6;y|wiyqU77TPP5I|JTen z-5OT~l>p<$d59x<{+tl^aHz8=H!i?}`B%fqM@!uiTL1i2uWpXC( z)PuOt9uO1qC#)WGw{2yh{NCuTk1Ukt+U~H%FMiaoAIK|yjiK1^8$N1t0{`_$VUOu_ zZfi=mJ6|Qfgr1b&N1F7{mti6>^Hb6yH$4VDJLLMGmRKqIHf$novbs$;SKE^wW#-94 zld7O5xT7R(dj84wA8IIq+;T|YFT$1wUEd?bpcKUiX0Czrl>A@WwtvZbiuz&6a$qmZ zAZ}bqBCYtrrBgmm6c2mz$klb)78P0UaN@q(!N2kY6UnT40J`OqoYhX0iO7zj<`soeJVS4= zOae}hXKSC8*#9^6CldpjUn*Q8Z(N(YpH^mvX$yxKj(`wWk&$aa^>_JAiF&})O8>4* z^ll*Ud0XU-is!W)wAc~gF^mB2Bx-VKwlLpb*qE|YN+KD!l(J;_O)a!99wWf}qH3N~Kp%E~?)6txp z?+?)_|I+^@FN z^Fl^TZE~eMd!KwZvMfj-BA<5XEJ=Ot0i6{mMvC{2Ojs!Lo zSpu>B^HZ3;JpplXHLG-7)H?%Y-Mojen<7@_eaUM%dVvJj*Jo3(vPrG`XQ}J&v1?Tz z3Hb7?QK;0PWsv~tzRM5`v#y^yN0Hwch@ve+p|ZdFy)#dB*{a=KFmHQBZySgNtpaW! zA?Upg$9Tg9H5_VycHu3lbr#2w;8$yfM$nscDne(D3b4>aRp2S+4b{iibEE^=CYzlt zQ!w9?vQsqy#G2qU)t5_I57OMBcf+>Ug2SgfM72qL$TSgk-B(9Iq*Gg1e!$%WBoT%| z;_?WLBFdS0vX^o&6drR|FK~e#mAEHp4?1N?{{&>4c=wZmdWq~G0>&-BZ}%MlRH{G< ze-yz3xHTs0KsL*Gzepj#>vzPpfNic9wHDO((3;m(e4n!Mnxy8Zn()B)n(%P! zr2yDQ&$6s%wjQUH8)zIU4UT{sIhn{=EYJx)vt%McLrr-Op1>(^5kHwE9-sklb>X{$ ziFAH>`pclc9=$0cu)^0Qcmi{KPpfMPNTj(!PL~%bbN0QzvRQIqT9rHBdl)|tv=q&U z;-!<#obXF?27hTrFFthn87D0vRtoy!*_np#-9CJ4rZtB6{5deE$2ZJZ}uH}7p*S;n#*6TL< z16%9M{(Da8go0HD{C0x&V!DV{N9;{`yN(y+%G2e3-$5Yr;lae-2B$b*L=TJ)BKY7kz?VHm3j%!{~PsfTl1IEBvG+n9T$E|0ar z`%aSELAb?$7!ni%~S4OA3kI3mj zZDBRwa z63L;#pm%Z=#1QVX7i!2!X?*-mvn!^4eSM$hDrM2^Sy)3` zVtRmJEnB5F9S&N41zK1(g(%L#hMa1MXTB3jUU`f9dDCMFv^P`(ANJHy z4tFtRiLjd61$Ws&niyZe$x^T8{)mTY^*y6|R~VO0QEi|ht+0nf-jCj>QX*lcqn{Gd zC z-fG7D*@7FfP!rhQdwPD|Ny4h9`*S0O4>yYotT4Ot-&oKL(@Yu=4&>aVX~HWFDDPCA3>tE7a!?7{P^m9e;P-$*~wb)##4r10^SB$t>4@P!=pPEL7;e^*r8gG6fwCYlbU@PBd{q8rLNHs zGIwbu!LJ5*ds=E$e+=G82x?i5=3@XR{{i)4dZ7HD^gqKx!oB1YBU=qq%?n^eWTNmD z(-LXd@=BzMee8{iAgm~kwLGBOy}$*h$+;b0S$n#6#&NkcL6;Bn!eC)~h0) zk`)i#&mD|7f@4o*8t-95w)IO--iwjR-)Ur**73ftDRT*OjvTZPvu83_Hzcb+sl>4O zdgz3mQiq0*2Zmp2ABNLk1)btziWi)@zf_`5@rN)Xw_th%#ojryYp4gI-PA`2V2NXk zhY~Ny&LmUy^L!HZQcCGLqSATo{L0zM9ln($jfZ=IRO-WQ-+d;0Sf5;Cb`uB zAWwkaGZFOWHwB3ssTVCH%5Egk?k}b>coGcu^hy8O!I@ZtGI+mvjH-wF65rkQ$Q1OkfiPnG#&0&Z+=uB+n*(TOz2Go5fLbyiC-e zAQcyn9OY+2j6%yN-8$6!oLml6kxV-@q>3f^n$~Db4wo7Etky(F_ZR0HD4fL`;IH1H zmV>H&8k&wHKRaGHxYBVB-E~B9VIyimq~Nx0FWZuVa~*Hs2hb$-{d`9@Y9bv86Fl@F zL}K}U@PO`d^;2q;K`Kkq{6NCB8H4|etgNzT9_c6WVA5#QQueH0v&k{04p6n0u)WpE z+m6%|Yr|5)ANE?e&3xCehA6!=nQZI_5`_QZeY#7b*(Ri7NU3a8>apIR-a(rP0+6@| z5c(J|M;HUt#V@&n-JXT(iq}24*BV`9P2r@~ehqh`90b<#Uc%=lF|&|OovTG3b zF&-X#P)cgHTr8I+XmV7~y6Rj}AoDY0QyL(CO8nHGzYstArpMspVp;(7kP!W8uX6lq z6nAto=AbwVTAJQ+xWc2(gt~;$Md=)WE;-viSV$J%}HX1 zFl359waU!2UwQ&l7Ly@mbZOc4DZEcwIK1uFL@}?u*fL3X$_KltoeZ%{wH}90NsV_- zY(UR;sCrNCOa#w@b`m4zsf*awLrleAqdZs`oN0=CUxWRS-rxK$#T)n6B&33(&=%V8 z0t$xAYPJf0iQ5~)^ILMp z9Of4IByclbSL)Q67|c>doc>6hAYt632DsCY&eu_Uk)K^T^tkim&A<_vB);%%r$t(u>SbjX!x$ODiD zb%98_+Y`(i zw7wucew+NzLUClBv&c_IT3JM$32g;kg>3vq3a=?R3{E6#bw5^$(TaDI3{h|UWiY0p z3fE%Il!K!I;gq?-v3Tpxl>s>dkWx|=X@!f!onczGnt+Genp+axh>!+v(Cmb<;s2f8 zm&wBXa6?7ja8N!d!pa-Mk-$zd-y?`roD-4dc*nE`HEktN>R`R`!|zMc)hkuD`# zBDSMdn6p=VsUY1H^~`yLKf%&dDOL(hDpYO}?RUrdKRs@h4im{`C&1O%fv&2T3{1ZD zm}ul)MpPiMpfbyRxb_jB{gnUX|I;ChpV>S5&h^2iRo+kJ92cj_+Y3D;;fQlDC3=Mm zm?YEUlOC5_w#&?^illL`f{>)~Nump66Z4<2G^1}KeGpE4FFA8dRgt%=TJM&8AZUl; zfcHvI>=^z0U@oi-TabI5*z5`D0=;>q>M}jJ{%Vf;7u0F!vywo(%hk#ZqO6+V`F9B^~2G%AwX+K0Q$P% z+yWZ1phG9mvMt|D>z1Bir~LIm`^)`ZkK$kXt+q95lfZk9*gBnacPm!=PKQttf{6F^~Gn0I*g}#V> z+y$@NsA@f=d+-$9>YBxm9)NPZ5wohFlN5#v_R6e|zzG(V4R?;Jq+E0Id5;?@Y?Iwt%Kx zKW4#1OJHY5nfHs$D$&)L-gs2B6ep~txC8j9XTV~mD;!0a#2>#f5H>)duTr7#JN?41 zkos(Z9rv37}2u?o6RZa1M|A7HA4qqrZjcUvq)pjSn2)JRoUBZG100 zjW5NxfiQ~j`6S%>M4EcXx5vXeUkmw+(Xj_FfAehl2@;vZAaOAQY=7TTBVqY-*N<@? zr5cXPuzUoYsprH~&{HbB1LUEKoQmWyjON6%kgxAbOC5gs1Jiig7QElVxyVD1H}q&% zH$AUE`<{(H(?0&wcnWl}5dfvY%0}KG;@{qgKk2i%GA|!PeZV*U6EqTLQ&F17ERF#6 z!k^XyW~-}>&_oyzh>C$K1$tPE-)Mb-2)|P~{xK2NVNUi;Qd41-;A9 zz{?9<0V0fuSs*TVxzsC4Ae%u-z3KT6Kr6JH7i zbm5{UTU(E)aa5%%aGY=NZ>8SXxp?F24~OA6qBN@-l*@o(YoVrENsEH6i%fJO=V(@%G%Sve`b z5D`weOjv)PbRlR_6y8XxYz#Gbht>-)CDV>%4}%vfjL=_(!PF{Z_Vm=7C-WYU4+*_@ zhAZHr+c}W>*h!&ga`eMN*;UQCzaI)C#Q_Mx^p_rh2ESDLW#A;ASMV*Z0C1NZt}xt^ z*q?uaar6ObeBCZG<_-vnQM_t;MV}My8Fzfx6O@%{zJhGV2y=*XlfXnFAf~+;t0^T0 zse4u2iTy+?%DUxwhZK5y(u(%n`y^obpI)Q4D*uNgB`S$L4tvAcjSUfa&8|tR))2(6 zfp(Oaij1S8qyd*KcFiWQN5Cu&9;peh_&-!#cOcY_|BuMocXsxjU1oMRXB3{0ipZ=e zviGLL*-=Rnk&2Q?871S)kV|=_vI&(qlpVkK$MgMtpT7U~cy4{xd%nimuw*uZ2L~A8AM4->Im{0;VMM^6V8=!+BOgB~3LVH^Y957el9lT3XoW9eIcc zXw1aFWUGr~Pr`_*`t1v|6^tR0mJn$tJ4LDgIi<0yTlmqR63i2cgx?is&hDcwfw4sP zdxK`GflVq+eGWONHwEbe8u`Jr3YnVLeIi3>h5ak%E+c2athAjF6(4%7`D~|sz43vRo zE$Q9=#Br4C@oiOa2l%~zBvEOVFqq$3vFi{%V51=C!ic@7zM1+du>es^*?8KVQt`7mwb!d(Cr$BmM=^;O zfAQMfSz7t*t%}@xh@6p=f(04mp)M$S4D`4I#|3m=7!j`aeq@gy6tYFY1@0XR`46@r$^&|K?viGISRk!*R;vw6ep+xD#=GxcKI-$o*hAg z=|6G^F@1p(t@Nir)eUse?0wrf=K(}iLCJ&YT|k@S&$Z8kIi*M%ab<$rD~K?HQ>7&-Ti+r4$8#QFhRZoYUTKGfOOBhg zB*qm0uqObpPlr?5Qn3oQ5}^YLwIMv zsyifKtTiMjZ1!#^pI-U}J;;dofZ&hJ+DrKmu9Udtr{+HgQ&R#T^1$|9dEMiz&~#j# zbN<$l!h;@#sy@{IVqdW>y?pfT{0olMG9}wS{%KbEM@6z#gkof4sF= z-KQKvwa0-cq;jn7vKxTlXpMq7MC zxmIn^(W)ZcVJ+khk1ONV%FIHCt@+?@#riRlm(OJ_|bxb}-7Pw*~PR z_{tmM)p0@?#r-(4IbhUd+{_|S4he01 zkvc#Fko#q4r+gBrIhlceuYMk>I;&~%F4}=J9m}^B$g!``zRzQ2ZJk0`=6nb6p}6(< z+4@a+%?wuZ@>+$~*v8%HM10F{{o3pb8Fn?<4`^o2pJYNbOOA1LH0sEtz4j0}@u}}l zWT%~yXvQV_0uk1<3Z1Cvi6n_3=pKGRuQ8Rtw0{C7)#p7HE3`SUGrXdc_&J4dHXuBZ z{M5c;NjVRC6~mE~X1YqUiOTMh%?P>Q8DN3=^7toE3yGM9$|(p|LZzGx%^qDdiT27% z?>covW>V`9Ep>D<8{l#MoqYNPY%RJNwm{06feo~}Xq}L|yAkZ4B^P0ZUn{IXnFTsA z;#Lf$4zifZA5Ya9c0Bih2dsv1I!)ov&UJHjYcTxVUP1gN%>8!K$#3M3nr>A&bsK> zRg#kY+wA!h_nXtCqsqqJ%Hi;zU#Cg#Fead~#vEko(%Q@-V;g7!VBAZ-=oa3n(~+L9 zD7`L_{XIQ<;=bpz#sT-xOOAhONT&hw?f5Ol>2!t*zy*1b50~!K+8RAb7R5vkj*@jc z2o(X2^C$!?Zt04z-2G=4yL_n38-2tXuW!(r*}lDI zC}%6&FBi8zUZpYSz?$KS*f!hW3sKE~&yp#tMY>aL6Fk*ba*U-LhW5WnkH=je_tBod z%UES~KOsG1;U$6JXi|eH}570&NEQuxI#hQB=8}Pub@qr8H^WW;W}JTKb!n zKKXJp3igcQxIrsJOJWRFp#vfA92<*~JMWssEBD%b){mEgPTi`%8gr%AooKIQEyvWI zk}zJm#~l&4gM>1N^h(Ybq)}awXR#KVVJJ?0Uf5ax*!cB*CO=~7h~oC*2otYTC$pgD zpcVl5y=LDQdaY5$k@cFth*7E!fY<%xfPre8Uw{Z=5G;V<=nAJ|H#Yp!o9k}ztNAoB zZ6Mckuw4-_S0${Q@xnEa^y3*;QumO}YR&(Ax$vAjDAJU&>4acW@f#4ImPSfAU&nn| z2Kp{JocsVH2K)c?9*EW?<0L%%oTR6nGf;=M`69gAND)!!4U5-s z7yF&)oU|CcrN({ZOwdgYSf3(_8%qvbrpVg-|KlWpt{;faB}T3@AS104a%^iY+wT&n z_rhSK_9gJiT_R`V;^|!z2tA`%$Oh9xoQ}-`{n;kOn6Lr zqOA`w!j~VomD!35dVV1agxl=Yl!sRAElN|ZJ~@Zm#$I_1dVCyP z<|92&0bOjoYS8ZArb*bm&RNogsIUf4bz*i4mgqlEV3W z4g8>AJ$P`f%?JJ{@V#5Le$5&C@sH6Jpjtra&Mp-?uu~Kls5F94{v6Yv>OqgJz?!-a z0XNjEpyHsU>Zm@L4u2X(5cgtBN9FfvDnb#K@#_mi7ykGO^3EImB2cCB1+2icQH;*H zx+{Aot^}4tGXtHz>K_F*i7#43eyn zL@UXrBNg*D=aggQ-}*`_JL65`EGMTvoYIpGKgKtZsPeQp0g z`$^pa;>~$}N`lfHG)ENyU!h$B9z*)CF9#8EO-_E@EsI{YIWo72liZQ(YQhEaJKjLg zhBHF#cT*B``>f5GcNRw@w$>K2uX3(@;oF_hX0Fi-m*2`NtT$xb>^e|Zo9>J7W7{_f zl5+JR%XX3ybM_ZwQJ2g+JTcDO>D6HyC@< zt&z$V-Trg@ioAV(zsqPKJqcAJde*(^ZFMP+i1 zg`WmNBr2Tz01@s-G)_#(g;WnqK~bSZ`zN zy>uA@yBjIvQt49rj_Sa=GqXgi%nlz$32+L;+bNsJSVS40T8U-sd?{13w&rYt_U2Ri z4$NlTHXt>jhb9iFe8jOt`+kxn{U{dF>8~XQ#XL&*&1~bs@D#Qq0}M@P9*LNlN%TX&qT?hi@Pe=P;bqJ@F$Y%b!0gg(YABP zN-x%{gGii;Z@ij8S^Njf8m}`N+B?QA4H2=2hI_dE|9iJcaW1xS^EjT4Xb!QF&X5a? zay#)2)DJ!djW^t5W~3M2PaWG_@HLEEnyJ$!d6|QmN4}}LWsDW>ETT~RP?a0m#_`Db zI=1H=i9o-yFKnpR_dGw6PtWfGJ+#iF16IOC5J|EPV)VX_iY}kL>9e2Y)J3Y2e0rJ8 zSnxzCNk~|TcJV!s+-yeUsV!TI?rtrNKvT27jBsPI?gdN~hk(3xsVB<$Em9;}ENyxZ z)mG}Rq07JPct@&h;1O**X!^*hyy+sSr*GSNy zuOs*)y=K|q2G$^9Fo;KBT%qeg)VTi1bk@zI4Es0|nBo#u3lXiAS7+M!_FRi?8!XEB z`z<#hMV4RJPkfzpaN@^klkECSY{G!e=$Xboa{j8tf|eY>DjDt+W^i%>rsa9l8!)g?5TK_>f6d5C!1BE$9_uI8Of&% zHqBg#B1O@EnOKE1VI#;|ph3F2etR=5n7?@w&}tt;LKernxBXcWZdTlaQ1z}N`p9d}1l<`WZ%HQz>BTDTpYznoV1M1R}q7L26YH%!?JbXxd& zDxQc^rrGuuL{HVz!_X|6v4N=7!|F(ZVajv4$dM8@UZMq44K369EVmjiIfyBavUQGA z$n98()nIS~0YPs`c$O8rGe29K!QKosCQnj@l)Or-2~+h!aqHiujYwW>x3>9 zzv@}sRP6A)>_ngO2KSt25eDUoPgKW`d3JXv&CVno8k<7(C3bw^HVOeYWm&<_gZkRB zdvpbtmQrSCc65J6ixv7kTP_meH*L_puIex_whpBp^X@EO^gvX|_vz73CCyCk7)eqEX;3f6M-b1p8e7wX;+nOqB{<7lzm#i_5tgsm#WWk`ux|SIaD#H6bY<%=UkABt zd1{3*9y=(Qr3FsxuW`!ZZ4>zG5NY9Q`MGY6eAapl#ENP(jvyyS2u)v9BYeKG z52PyP7msHj{kW`&3!t10_g#Vg97o7v=g_&gK_a1IZ$>t$^%F(HVgA5^ucMI9flC{Ny{_O06E1dg|JdC)o z6qG0T5JnL;8jd;jbu(!ZmigF+bk18&l9Vq-#8zat^toJ(!X!U!#(i{(SAn?hH=Qyn zv9NK}kf$-?!lXh|$&&X0#^rYuF*5&{bCXB&vFbElOnj`w-4ab4fipp?gm9$@ zz%KFDTgJ^WUQVcaX1ITdkm%%t?s^K^hWhOBp{74xg~k+KrkdC^SoDVpTJ9L-$XiXi zdza>XU-8(X%u-891U9nJ^6TVpqHcQ=I4Bycw3$d2+n&?Pp4c~PK&&33e#mDri@7u#whe5BF54}3}vTk_hera~;8za>VK z?QJEyM=?eAEy>Yd%qC9h^ zneOsnL^;!l$K7xf!lv}~C<;D4lr36ctE+UQ4T^_o!#sIzqBj9s5STwR8x#_KQ^&z? zO=V?#N1A-F?Fc1#seb7lcB?Us$Iv}gPP*utTOPV&wwkd2>L$r8mMrMc)qom{S}KZv zH!#N-As9-J$yaq8+rPdusIeS;yGvz$ht>O7X(Cb5;Ie|^P5n}fF4h`Ql1eCtia>d5Np&CQk>rzhWEdY&AUvxCN+Ut+t(FzjZ z1`U$Yq{KWwt~XOTRg&i_XSsikSIQ;?b9K>p%SQIMKFFmC#BR;>1aW1d#7DGoMAF3R zg)t+BlOc2Hkr)NltNZmixLW%U<7*aAW?Q4Q7yO58Y1TTk7cS`GV+%sxC%cR48+_vW zToRu&e|0By>(|EjrF|If;kfVmcb4XgCXx&fFW3A0n&@az6wS?YyxLmavYO?l`;B9j zi1Q$@88K|RkxB7HY)X@bq_1Pdk9Sbl{BX!R+2KDHp6=vg$uh{id8xrD$%KP^JCgES zB=ip{KQ_&M}xWJ7QLZQd44II5>I(M=6U#>DHi66SJ;!{aK$y5jKBPI)M(HrDuRx3aNNQv5lZckhquZ%d%&1wq zxosf9Fcxndppjx(y?L!6W&92IlIgEyi%z~37F=L5Ig8Xz%8pmrT<#OXGcMblU9QUL zrwB=Y{wURN-FOgXCwz{* z@EPom>P^_FQ+_`cYB0uUhvaVwRN^^kDtm(CQQQ{{2RYeezu4R2C$igbMRHwl6U#=2 z5(|pFR#MPYpOQF)Rq7hIycYHENaE<~TkljeVR+%(N9e3a%BFrL)JDiBju~R*%1M>Q z&5aUUjff<)6up%v$=9h__u?^0j)ZVwlRu{ZFx0U zRw_*7OkSKJX7HSQ`eH>N97MdtP1eGre#h9vD5U@P3eaE_t2RDv6LL32l7Tx3Pm^Pc zZ}Ns6OkOzf<7G{RoH5lIvRh+P;oL2}a z19U0btmzQ>!LKt1J{gM=Li0qV__l6;%Js>wg|dRc*64`;JLE`HaK zFfiJU*t1<<{JhS3)iCo(=6KrmXpf_Z_fEwBkiK8E=^Cr&tHUVDwb(PBCaJSLkl7}{ zD6=r-U$+*Z!WHf;t8OXOZezd5suV4ep=H+SM0Hd&<(cZXpujR_|H!q?OZ|}*GG%+0 z`m#$T7d!5yw|L*ZFV1_jUbFn>ipCYc zsPvi9FyPJ0;Kx`leGM+|u`Fl}&RJmZ(=L)B3Ui%7iMJ|@vbSO~MXNETh8655^sTFO zv@>jy2M%H?#fsV%5Yh z)93bC*5_GMI@o$_aF(S*rkb6Tn9H=sJ7b-n)WpYf_C06n&ReM$ZjJT6hBVG`O`pxG z*K_^P@B;rP5s^ovVu#R6Xm7`OpP7Sv^!4)9AVLL|fw;fZsLCcvU*_$TGI3c%D z{kO4Q{g~@`8(G?>B{Bj1X2P}to$lpU*rshW`*iT<3?@pOpkxru8j^sgx9MN1FCu6C z=y4wJCO52f8ecD(_=`j5X5IpNE=L{DEFgRG!Ia2RZRQMj_C^unUM>|BE-H~~v6T^B z^cPWD=!AZZp@?-~C)?=3cN9ra32FnHYolk{*HjB@FSSG-=Ko+AhP~js6&{Z|U?xi3 zpxYkCKTB{GqtA@omchNsnMt-VTq`c@JO9q)>e{#=-OegPTS`vGaqBeUk_{PgL|O$| zudPX=jkI6^M(0X^OQ+E)6@2+2>uqN%DZ+0;$2j94C4JJyr^1_6KlN~;yArVtm7

%JfWpfwxO?Mc0$Rnjh?hs%nhAi3mtEqUQ zs&uEX*}Jn|aYgiDLLqgBFpi^_aeP09b4=ZM0XstoF&Lz@7wFHI2*^sE9w}Y0iY&x? zWtRSxG-^p;%QSI(F!g@n(BD$)Cg|uxM)+9wOm2gwZaO@zC6`=&jzNE7G~L>^JlC+J z`+Q6+YV`SA`iy9Ao44Ep^cg>u@aP#MX6($H1o=VF3%K76}JUZbk zbLaSk_M~t!1lztn^YF?|!#DS)L+U4a8F@!?rX}{owuwv-Rm+Om&LxZc+H?8#9TpyW z>d#buJD&RkS+hL8;K=p6CIfsz_6+ZS5W9paHaWQ5+Vd;jJd9Q}E=q>1dPkS_NBWPe z#<_Ct9I4_tulb(D#;w*mczqy}IFz`6EsxUlD7xNXkD(HKir={X-ZJSxRyt=IF0`*Tj-e87{rZE}`YgvLV;dVTuQMx{bazBfDVXAtbWMN9drvfu zdY!+cohNp3X}DqNd@0@@6R7=M`i3sW&r!Z-j6bP?^9F70+74wwvyZGq6ta4Zd54t3 zMYz5RIwJoX#{pEzVyth`v2iz3ws%Pk2eNG>6TU{@pmrv?D1_-0X+^|5d(~YkLb*M# z{8@;RY`R|n<*R=7Xyar&jT}|P#_bwyyqc>PND83)%Kij1KACsC!54J&>e? z32iP-*|7+@6B~p(pR|1HLDNOvaA;$ul;p(X@9yHGeN2J+J#y!EZ!@N z7y0qS!e@!+aRH=Cv$M$uDrY;SV~NaPZ=&{WoW&^X_e`C7G)2q4JowiuHhub7f+R^T z{-^Rc&LgV7(qi_v%x}76UclMf?4LPhY{Mb^P2QGx|K0h$-^z$CnIk;uNeAir$g4`s zeQf*!rGW?yfrAsd#MUwLb^MF|b?A?xwAbAaVNQ5+{&nV74THq5G+N0_0W16wS~1Ry zT&9Z?YC0FM#uRxGmT$1TKjs0zU0=fLMrT2TtyhkT=M&WOoNziWZ-*8q?WTx-5aCS`F`caEX~ze@ zko0edadW*^k`cFR{v60f9;Clo9q+-M6;7=;$)R7vWuiTEhApxViV_1vJ#Jp)1{lZ3BqS_uV@}eCsTo_ zYI(|SQR3~n(v;PJvGWWom03dv|Rmdi4)8RLwN6i{#Vxzkmo5| zuB;O`Ob)|HKTA1C%WUFS^xC)fRmmNnd_xWI(XV&8(jBEZcB&l1GBQg zQ6z2kTYYLc)d=p|Qz-SKb9iZa{yXLLii`yli9YuLE=gR#;Fw}L=Bwl_P8oIaX+4DU zIkphBPw0;u#>U#AbPdbfq#p#<6kgygI52qax3i0Gzyry zs1^Q3D*6q*op>&WjCyrd;#h7a^=hoq^Z3%8F4jDjuXWk`7LS7fuBj!0g8$Ex-sG}% zw2d_-FqKU(ddoM(g2V0DZPVB_Y~X<*2OE>`cNY(2GT_*y^Dt~rUajON?ISn_Xr~0d ztIL+Qbtye7y%B4u%^0r3iqU(<&=-48l{pimjoP!)O^%WcdXGP^mO@>rtdZC?gGjxuWfQQg}B-%yUZ6HL!j6 z1sN}!SAo651zaDwV=B(Q7a*|GVwnM^R=qx}23tT3f5l=+bpOKN^1Vw1_U9FcmA>6V zdY+U7*iIs==j4C3fRd?eug}T*IavyL$uSkXM{X^4aMkIiI(^bDrW>#Eoit*XD&AwH zV{k1#Ko%$1uj@SghWdIPQDFL-jNJg&&Rj5zK*Sl)swfPe-~TRS0G{zPT3}o~Zs8_@ zLZZYH^NzVq+N*gJcs^uFU1w3~2oCpK@A{aAl5CQ(~Hfi zLt&rp;&(X5NZCeAis+{U8TA!nr;k=~ z5jMf(R+lv$4T|^O0{V5HF~Cz~US&l3ncuYEj0i~f&}A|tf3-sQG29Iz_Q;z*sd(n` zp*KB#I1rgr?29;sL?*+3%$m?|Ozy7fg%fMI}9PxH{PM?tH=zJgkGUOZv|Bje=IIgMZy-7H7g2Are2&4_oS8mVXFVUxo#0 ztoXfBp%GjX)fG3`G6m;pGlk0E-%|1U(HFX|mmyTdV|kZ0{pyc+EpZOi*@CE?0Al&M zO+hf*-F5C{D4A>;KX_>(vGZ7U8>={7@w@RkqaQ32REaVd1AIh1-rk87xXR0h)0bX5 za&EGU$b#c8evTC_nm?C!(EkXh$itRxHI_F{NkdMuy8ZjGZKD|Ot<>3EFf%c9M)UFT z`lN3iIh*8gG*4pl{tb3zY)^|zuC-Q?AQ;AVD|$LD_9!3@O6(bfclLHTS(+>DySs12 z?umnz>1B}v@lDu6{dN9s{?{hA)U`Lq?S0!P`HB4Tt4i$smI?)~J_R6cbpAtYWlLRr zY=iZXLAe6IDGv4Ctn>=t>6v#EiSXlz#Gg+*Q$SA>&tSr7pyYHQCWrFXMYkW=VF)qvor z%C1a89C#Fc)>nh8lymku0=%w40zC?ml6_Lyocqrcu+V4m1Rr%J=jXob_xVZDMg zGDpY7v~7WRuUV%PlP(SF#^E9f2elA+2B*fpNi04jN_wQSdPP~U_%Sw!D8IgKkUAVX zb6bVGIiacPg3Kok;w(qp1(53Figg*)J<4AignBWnp2?CZ>B`bEo-d5X_*>d}&eP5q zJd@g3-x7Xzd5=g|vm*XV=Dt~T&ZqR^=)CY+t)wgGd7k6Jg2Qu{Uqmb2@Ln>?1lwJ` zwpdZY`rZCe?3nPG)cynC&PbLS2cWOJY=TLv%)_`^C++9>k6zPX+m0doFt;Mjk&B1( z?09O0xL-=(ZDj|fl^JFd_81gnN$9g9c-&{7olwE?6;aZvbffr7eUzom?zMLUgw@H- zOqE50Pd1iG1>LbPIE&QzR6IMsueKEYBo_v=Vwi<0r5H>+Af3=MbolKjSX<)5^i53h zl8K+MqAbf`SJ{fFdj4}6sJ|Tu*}M%OcQ*bzL(L5uyo|g?)33CCx$Rb}w2j+P*Xz4a zC9S=#HV-)6^i`hO{IA#-C7!nsJJ$a^Yi%a3Q*OX6s^(#;xuG)OxD&;$(?3CQo!GJA zkd-NQ$?4a?nc;g%Zhx%xy{i$uT~A-=Q~wOikR8}{;dI5V6=q8h zfd$&fUBoDZNn#O7=h6h7gmNqE{P}U43zywYS%p1oTPokNyh`+I<`Ln`I~>@>JDGq; z`||Cg{;9Oiy_1Zp6$d{sE6=iLvGGBchr7h~<8+E?jW*fDIP<=ncr*QbCsPkK!ONqz z*WGrFp8%oI+D@SGYlghNtsVJkS9k_8_yQuR52C35{bl(f6G@OMZ~{336md=Fl+%_0 z{Mr$NyX9a<3k;8{T)w>|AtImuJoSjpg=*#%@!h)T`=USG2(8Mi9BtizK!3w^d-Sr8 z^o0lzYdoY0p47N-ChA166Y{a7=>1-ZVW;j5&;J+>hsv z6P=;8V)L!_7~qnI0C>c-EttMn%e(y5%IQ7 zSy1N{YP1 zfBFn`za<(HFoe>8D!mIv1N87~M3&MEOb5I+ettzH2IisF@nz(e4FLTa#O*n-`-WUF zfDZhiWb+5N&%(M3mw>@?@nFopo5j56;uI+UQbu$uRD*0)$5zb!yFdzw2Y!FGs~_{h zlVzQk{iRR<-b%UDoiV!|>*_bpUx1vuM#@HA`m7dZ)fvkEeHfzI+TyI%(rWuCdwN^s z7{6KvtM*#cE8t{!0TRs%z$Az|?@Ad$_^hH#e>A_P@&=Jb&i?{S%6V}R$t81v_A}=y zkf}Z|=d1Y^DF9+#HEK&o*neQpo|rvH@OsuE-xn`iFg6+-ISYgiFWB=}{&B`?C>Pju z@D6AN>111cRfthfg^v-_n;eCfSBvd@)w<^`d-QqqrrP>a>4dQr*)th`vG$yqqF_t-pU^HXeTLo#N>DcqO-eEr`wsRhY_W)8%6|YvQ z8AG_%;mN7FRtlIgyD$zZi{kps4I>8oTGq_Zl%Cl)1{=IzFBO+(vUDhlb0Ve8lUb_4d;=E_xi$@V8)6gG3fwYq+xJ{hhixjd4AUWd% zJcnQWZ8iEw@Y&Px7$krCl1VKJSiE18rVD`2{=9mE65x|9m%vL}>8X6xKI;n3!U)iP;2=>@4{(;1`MN*38CElC+^rCw z%Jx4}9i9v{BscQ|EtN{Td(CNBf1!1@D3@<)Yyqoz*MAuRtuHqrJ`Te_zVGs#0hV6t zKhtZ=+z;i7Og!;^Cg>U?G#fueHu)Arpj#Tjf#3S;&zCBV6zd&eyhy`k zt^v@b6CShbPEC6}mX0`znTfkcbiG;tcJWJiupVRircGe44g;^+jK4ys9ARPxF{Q_wjMWl#ru}76 zd&h>*=%3fd?X|Xgg4(|-PwI3R;D=2*mR9G0YPd230lRD@_4|0<)-Rt1aLG>G1j6~y z&i1O#OBNcY2@b06(B+#5h`smfq>on~UAo)Qv7(6WP~l`vV!oMt#`1WSsPa-vpwF>C zo*T%7f)(+iDLX#W%xFVr=U!5-$q}_E^{pA59bd$U+ynFo4KJG9=2LRNVi-=`gY_b! z5hEt9-_jIy*_2>F1e5cXGh=z2DjP1m7f-RSFmZEEw*fV~nxSVG-b?2y0X0k2Ye-EM zm`Zk6bRe~Hne&1OnqiWfN~s;tm7c|hz)9j+Mohc2Ryaza0 zXV}j{u2I%&SI4%a*&RB4Wb!m7_7ujU^=zjnp(h1XesdBN zIDYtu`sCp{qh6#M3EDQ+KxL6U<)VqHW6 z`6;VaE9s|ZdWUa8$OJuDkn6DT#rLdvA?j9Eizcd*^+`Ft$GqzojZBRV&N%+F^jIR5v zej^{t%&Bbq=finPp5-(?hE(}1X77Gk7x+I#PMqeSc%C1FUwo&WQKH`G-?%;|13wvM zp_a`h>2hj^-B-$;?P#<_j{x$yPo(%)|F~)Qbf4!9XQN{1ct@?$Yon0(Ii9F5Vs9mS z3#`oeOsQ+vEE~E1s2B%jM3}Np&sd-0gNdHy7vu zuf%K%z5CH=`s4x~q0jL0+`4z{-_kYaiD_c7kib9->yI?>2^`fY9v~WnmCxkQcB1QE zC7J5?CyEzBv`vCHKwLif>Nv{s>~86iQ||aVW@KT`y2MuR;yfUOj_@qUibg>p<#HNlPlCjZOWkXZ5%o(x&g<)%5>A zxVuv*@=IRk^fO}GJz9pr9S|%ST(Kv_uR^#Vz9x0tetqeS>tvDp8aaB~briEPH3S}Y zm5ZXfVGk~T{_muwVsQGBuhnP}Hv!@MAQXnmA9W~|ZlqF#Ibkpa=x|Sa4rXB6J|Er7 zyM6MZwUkOeP1w)b!aoiI+R&<+wypi?_u>&RzMbC-(eps=Pfz>8i?}cgMNnexfM4nX zXk_HWR7XPY{^~5;PFpZl%cl{HT66y6u$*Gk){ZRe-}Ow6s zwfj zHO8d*I$c<-^sO&v8>5+i(8&J4^R6uvYcw`>ppNZ$G z%{blGM376{|dOa4Z%_q)~d*IbYC$~qLc)%qdBV6+^lIaHdo zgF~OoWM3$NQ3i1%{MCzw)sTX@^2*XhAO`5c-7b zwl%0!@|Gv)D%T+Q4!-IYUCP??>P42z9F|}rxSPr?U;>pQF8cgYq+W?lWswYn?Uh8= zx&&?=Y0@~frUB3g_(MNZ=VUqZ^qi@G{NxMtfu7}PTo#bgrnNU_MeN?X#-_?Q>&6PI zMCoWQgFc3B>*ZP2Nd%Hwv;)-UoVWUOKcU+LZuu#%mIP+*pO^JR5V?;?2zt{>T(8oh z2z`NA9>Wvb51tJQLAUGntQY&Nk|z*xUYEE@e?)5GFtpAQ6Z)4{4^KY6`1O<$nz_R-4N)ei zW#qC^ng({|$qd7QFMhN%mNB3y^7dN@v5O+XI0!j0aQ@Q9BjtxW*p62JDS|tI59o}(545{^SmSYpzOdgLpB1;6zBZmua6M880h4RJ zaO40#Xa$r08A%ZZ2~><@KGHH36%xJEZwpQRG!$#DecVL~2*q_y z@um|tbqssJK#P_710#=jUWMn!r}geUtHhs&zojCIvv#_pP&j)5p;a6O0&wFap)&$V z%*!>=SGuZg{EWB=YY|&>i^izmb6e^ophLu7g$A@)wMN*_lHu<@(oz)ZBp;At{Z*-e zB}hu|-i}R0H)z@WGT~QV`IJ~{!1itg5wRsv#Wq)c1@}V3lOcD>{#SmFjhQKGjwf!A+QDWG2(c)Kq z9drtM-*|Lft#x;}aJrfj()SoIejEmY)~Ze;8tk4*pZgf`fsZ$tWC!Ii;yB@cYOD{D z+VHME_=*xDlx4fH zKQ}zV6{qo*-5rJvH@+C%FtvC>Sfpb<1LL)rKB|dCH%vOHq&Ooetm3$L!cpR8IZlz6 z+nlCp4%Uke4&yz9R=P}7Gk!sqv`U|ly1 z9Gp3_j~Pe)p>th1O~qvL9lgsglSjsZVmII_HI&a$zqY7V1acj>{y9=F?+e^IFztC3 zb`4k^Ix6B%_m-Z7FjW7mn;s7C;Pr!irsJ10~L_ta#4WP{1Ub@cOl&yZkK!@ z@Ez)#NE`+7F~pZYj%qC1&P!qOYCrB*Tod1K@SFWelILe9RYmVaffRJt;7YMB$BhNG z`Gh}ix;v_!Nbg(@GCPAdr|*QGgL-E8W0l+Eb}4>t>Y#;@+wLuU<)YwGXaD0!;GR-k zyzF`k>gn0h2@&n$#^>&RI2B`5({vb=Q*ba;&g5W$nOA?@Vq?9A{>FHHc#GG|A1c9+ zW54TEK~_M#npXjYdAj92V!QJoKdcL(TWPHfx=v>SkzWo`S^n_c|UhFdA-~YyJAb zy4+s#)=Ge3ed}0$3urVu%V&@& zo*!8^&yJ2pZ9W#bu{^e=7G0j=u??R+jO-Lw^TT^b>yD)=Brb~Hq6O3HzH-CgWl-w= zvl@PO@$W9Kxc*jx>jfudI0{J>q1#uLAlBVxC>h6wp}bf1!b>h-KcwBou5aVYPEWxR z8%5I5ragxE#n6@KKnVT31D<0;^`Xl{h)Wr0l20OoOhhjE1xU5kf#_!~T(dW?K`6DBVuH;kOvoP#_&-L?Td`lR0Hiuziaqp3|z{dn{a2A zK?%6ub78cx@r~;?(Eo3)hOQX;Tdj1fuTA>&AXVFK`|KMnhOM_s(mPu5PA{;XQE?Xmz3(fmuI= zR(sTzz(`H!3{a2?CkS!XLizOM*l-E-dL5tU*ar^>PG=_& zk!T%gN>rn9_~W||2hV#xXL%aKh5Kkx)3LAwW6nQf(4JF@R_8*#3qY98+It}e?^}B0 zf4&6Muj$mypHA;9W#i+zD&WQ4##%kxzc6mi^KW3M&T9glU|^eini02ZVRAO7f%wMX zx@@N=n=G_pJRVmA#ns@ynX?I#A}FCWQ#^Z1bsad`uO3+x#;G?s6LhuE7kme)Pdz&M zhBxG0=hb~hFl&D{oe!nvbE5+2nJVrVz=IkDeV{U3IoxUQ%i^Jyzr?KXo6~YV<$fWL ztdrJI>2ZliI;7H+>7wFEc;}nIg^}T+;~)UjTjkYW{n1Xve;b+F*eV~N{c$PxE@GC} z^y1^?_$s=CZr(F<%>(K*FVC_Dkp(>qv<7v;D`NM}J+HjX5(-j(%-(@NeiTt99%5bT z6WrjU3VL_5X<&av?n?bq*TH=Q?**l6VCWavTy3^SK2P0ea7?&CXYjaj#@EG}$>#H# z8~!2NH&Pmfu>G&jzBo1hn|^_Lm9}CwJ+5;(=u1pEzZnetotP?qt~gaOE1vI>uh`f( zM~>ZOttF4+6O~m@e%+nwfi9h0o!`BWrSDT1>;gQSDwjqe+B%S>!UUaZ|7w&96X?@O z7Bn7TsQdp?Wdfc*Glweki?k8P&A6%>O2yI+UO1h^EMMKE^sw0buV+WZQskuLY>9n0 z;tf9n*UB@t-OioM1Bh{@wy3HQ5t+n%b?NEZdo3iE*(V0Cs8jT~B7VLr++nFe6>~FH zRNQN1snDefYkNBrud{wrVdaNT%S@#I@eFT227bj)bNvR6LQal(az69_N7Z`=Qu)7e z;}JPXjxA(mC6Nf3;UuDrROCdcNcPFhhz<@yD3KLWDv4A!*~*IaLFPe5$}U^a^*;Lk ze$Vs2{kgiVjE6`yguA2dC_W}X=Er`OI?WAY^GFP3Sf$SgC|=%mTd zkEOZ01nE5XOIUB+DqdFI8RN1*ue#rNXs+BZ9vE(Os{T9A4}G|=(MzY*5Set({;X-t zY*FoU*epx^A8Wa<*zvi|a0S~|n#Cj4(D=`~bz=UsL5aZK#oo!~4;);U zJ}Y;ArfT+B9bu$oez#AWs#V(2_S9LbA^OVwepV}6bC*U`#;d;giTlS${4s2Q0zrB= zj>U#SGn8A$e*Z8FnmtzVkQz$BH{k|81lN*O;X3@i7HaO!2}7!kl>+41(*JFO!*Qv6 zS6iy~k*KOdBfFrPA?ZS_9aWp~y?OygGtOQqPILJ;x0e_lst#YP{z!d4q`>O#n|vnP z1S{`1X3LRnMrNz^ViRLrk1$MCO-sU`6a}vaeXRK+bJ!>T%oS46kIKCVHjTXd+A`x6 zXXV309#vD{SNvUGVJGw{Ibd*$jpefOkNuoD8&W5>V!l-`oF*hnFWqN+Z=?ZBhD~6{ z(=6N@l08eWaLrfn-l(b9BfJa_>AfyDi>$CMu8DM?MRvct4o%dF!yWREzOMa}ig;do zy(=g71o=nzYZuNosW>P-J9Q`vdr2keU3xMY>4RI&1Sc-aoEeygG=7HVNGH-1+v8_x zm)}4ol=El{bF}^ySq3!+RUE%skK!#?xkYS73r$C}#LwE3a{eOm(?Stb`xDiD!|%-* z>(5w&SeMkrk7}T)R51;WP_>I#4K7Cuwcc+}jf``<|spK;Z|%t$}+73Y=E z{K~yj9b$EgESXRh=!2=JpEGSq`)3XfIaN;kEw65lF7IK9^d4P~96A_~j(tl~sMKzu z4{QwiUo>~e3Rn1fzgm>(QO-aq>dld)w+fu3ju5#6u2RD025-2pxqTdY5F*KVh9Mt@ zmpHs+rjI%BhOzx_hyq0V2PLt(7#*VIb!({xme*1}k&CLQR<2mbVW1@eeuHsL!$ z2oR=HI^x78`kV@3jg>%CDSN}fGtakhK+&78yliW=#}z}G#F*}WM(Nx;c)XG&pMQ+lxCrz**RO58ih_o6S8H_6w9EO zL`fUG0+4GyZ+@{v75 z2WMUTNPXl0^!)q5+qd2o+}M3DLFZL_TdK+c#76x{XdaT6t550VTX#>`WkSy#2?`bJ z#?NxprTiM}+bb-aRf2+I8cYX2-?zzS!c70M!gguW6qRu{G#HasO1>vp zEWD3D6UM!tn&S~)z7f0)i7}vjQ#H%FRC(Z2Gt;(hh(Fc7?JRL;T5+GsL`D@bziJGY~T2 za(oSqz!MKZ1q-yy#XE;}lWYg|Wh%x%GKVu6^lmf9P`ZjS9_a#AGGsM!U#;#nE`$yp z8y__xHw%zyJ(}AK&8u)`VKr{7cR{Tx-*En`JuwgIY1l`QN-me9_?osP|HK>nj?A73 zRg$d9C<5F{yP;knC_3soru|RFC(K~D!aA4c}LhS{aqAm)q&E+!*X%iPx^zUwUJKf!UaI$Z2tJ#GYpr5SN{8kCPBWcU+?>J^TNJmx=QskQ<~sHegZq{K?H5Gjeu#(wtqn3oMqzF@z1% zy05gfqB%C4J6hkW{eQJ#x-njX$%n0jK@T!9JL|;R{`|bB)()Dd znPCK`lZQ5+BHvD7qpU*f25fYQYaZ0|5Gnywv5?#wD~!X8V{oJ#Dy@_ivFzha*wea6;&OD;>sX&VVL156SV;`u+BA`c-+^#gO(JKMSBxI>6!psAc`EJ6DvY%}iFkF?7+WU|2>vJ%d?ky0-AV=%{PQqWGa?S(vo@QL5BO4d11pQ%Xg^(9VFB$$D$ z_~^Vg;mf3w7D-ufH+FKB0QRo&kERR1FwVH+zCcU$%%eyACVe$-F)>@iyH7e;Dgmra zSaI7fWFoW_G^jpNYAB&A8RK+r+5(S3*{5-Q_ueCQsmBNnxHdph;-T#$&E3cm9iU=u zD#4a0D(~Fi6CEbVxXh6a<`u^cuV4S8jz_5(zD)#w-_2B>*y0ZUs0%=4+;yO~hc8p`}$)t)W% zJ!b}(Skhs#+C^fBPe$rdX&0r^0-3u^x_8P(qg56fWK9I)?hx>r2F#N(ICe?ZGh$d5 zfGQRennhaBf92f%uXkY`k}WI%`zih;G|YG3+WO_N3eYoDB#ayA>n|?qM(yvs(@O<_ z_nS4!`-GLC|KkDX`9e@#`*$s3gxJgCR!FzOdoXT$4;`?-tbc!+x>lnEd1KMj zhxpoeW%XNs%Ll)NQesm7KTFw>p-m?zC-~wE^ySNcp^0e{hFz|=ujLfKx4w5=&g4B4 zpVH^LcDRhb9XV~_1d+P^0*7u$h;0#!m&b7)JxMM`|L@<9YdW

DlmbiSelHJ!<@| z$i?NlD>ReZ3LlHv;J)$JjN1HGg)fsk`Ch-Ih+|i#21Y9SoGFVK+{uuCzR4JxLy(Uj z@~UN0tZUGsXcN0=*FoQL^L>=qlZp0pFNCs45+J6}8;KJOw2UJJGBvX1_h|(5U5xYvd`B~a zzY5`OCcnOX`1^PDzC4JUXJCq2fqWsJg{Sfl6xn%-LPya7MD=(gHk|6lPf`7ZVvzs2 zI4fY%;%a_Sa{rx!L8j`Ri+^SYEFRy4X^YtUFiP-Jcl=^6w1fNMkhQMyEEj>!Uolja z>&6qIi^{X%>K1IcQ?y*I=cR?>cp215tc{*;#L@BCl zIRuhuLMShTWa{B%GQlz-2%a;jlFk6phVyDkVylPVhgEr(!G1~u50u0x=1&yqb!(Au znsQwsj9jG<(v*UQ8aM~JlY%(chkHT%I$c|)um=Dx#b&Nhj8Lq^r-w2w2d-k$p`m`M zbG$T5aP3R3x!g7D6t`=()SM@zVab0S?m?% zyo{g{l@M-(8_G{$28YlX2bF2TMGFkk{{| z_}WEb#Kh%29+-Df=xS_$RbxjCC|I>dD3`w)r2BH`B)^Y{LYh8y6U_$=PX`cD!~gSl z(9V`xAS_K9zx+$zNMJJHNrRz|sAqfunqJ!v-@Yd!D=VWh1JJzwuHf8nf5xHbR2adX zUSLz`vk8eZ@y|tKa9jf2hJIAw=;0@k&CCi&57;cAJ+kB+3Dmv{x=bV0%Fa6ghoI&lR#j~)bQ$e z3+t=6Rg^tVIYklwja|H(DFhzGv`@(rbsjT)Hh4_^v)K}7cpgw)6WduRlHC)7pwWj=65VB9vy|if69HYY!m>XHEBZD8Ja~;` zU$hzw_jHD?Ljn(1^!FpAV$|w}dMD3dlP28IKij7dgQX{>11U_neO#I0s2tGD-E14~ z>fzV8yT3s9rEop~xN5d&+ur=U2d;pzDFswUF?46_?nWCe7rK(QmjAPGNfPyhE~DU2 z=c-N5m+RG_3+c)M@#CfK`w)CJx{SydiYgMCz(KMW$jeiLIyaDkwcK~Cd11lYyy+uuZOGKhEt0JZP!Ner`Z2VdI{ zX)+$dXJ{f0gI2!>;Ibc|2t;%Lcp*S{`%ZaelMi&<9F`GU74F+rc@8Q*hQiumYa|Qso-7nX!%0~GOxSv*GGsvmjp_Lff3j5|p=_C`Id_d`P z0*>5CujWHVV>bx?|1gjr()70d+{f$ z*@H|AKrvQa-A3zmI%M;W4uL!hj@)SZja|3DNFL`5#S6RVRuA8lL~EsfMky>Hq{vXa zdE77scyo6&Z)mBCVs)yw591ekhpr~zloVY0G_HK=q4cGeB}DEiD9CdAa$2zj?y1Gn zGXSMi)|_@b-Z}H-m4zau-3MKxsW^phW7Paq^qRrz2i6`t#0O)_0?|Z;GcbS`ihSq| zDKm|=#^ixbdN_F+5aDl+D|GX>P(wIu=Yi@%q$qJP@PLYQNAUC;kg-!%KKtKWVyBb> zHi~Zm_#?wQ04WXKT(Ibh=mv~X=HC#TuIKbYjm4_vcdj73 zBjN-VL1=u#7x)mdV5f?K$59MKFEs0c;%D)E6q9(2(Pjb+o*TrYv3RA=%InuM7SJ zMAg6a0~qxZ@E&(~g)>e;3v(f+9zl%Y! zAzRxw&D?i+#>rS9Q}LlKVP7`T37jE0KtQpWC{d0~4TLNkNJfr>V(O7lv=FZCi04l; zFTgQ(12Ll+j)*&itHr2T1m5Y}Q0=^k)>7EDXFv>g1Ih*o-Ib=h+n;w8rb3+WZ`bF$j}fSG5I zyTZu=zlmlcLush+X&#vmyb#yF>Rk2s8r*Tt)e=@P0zKOC{0a_TIdFWdnP!o4JX9!L zp-K`%xyIP-&C-LrhDL3@`Z)EV2Y>Q_Rb_cLsbev0P=oAkGT)I7Mdwe4EX#{O3mapE zI{$4KniFu%^DwxDgYQeCkHBaJeSiIA6`qcCsL~}ZT3q~D4h8Ob`=F3g<6*3q+ zO#B7Y8@i5W;K-@NK(gecnoJX5v$vo3c(0N}Ck@aik~%bBd_#0#&Th2*^l z_D15@_K7=&96B1(*sAinY^Jnl<$=kwGy96b?*s0;I5(^FJ5S<1pkFs+YP74bJEEg- zN`>O_-S_;=DR52$u-kdZJT=U7$jVAr7bnU9#!%l7zb81F=Q(Vu1#udjZ4_8dQs=)@ zHwrV=3zl9%aB`&{-1vHy+YhJ&@bT*J#05@X$9)nEM4658qiq{OFaS1?V{-V8t14=* zAXOI&=rmj2+5Z}JAB&Mog@g+m3eAMfx7&wp0xb0_C}qv!K?`~S)So$cWp)=0cqSt* z1%QlA&QFKCha*ht1b_qf0V$4exCSaQ60f!w`6oUW1iijfsddvfB1297frYm*e z2Fk{VAuD}q|Mm(pWXhTyeT^@9)704sb+<`01mL+f*$fJW1VDwDjnTpE8`flqm9mlO zD0DtfKpfc*NvExc#tG2?xJ6{-`Oz?PV9%}gcwZd;=30bYF5F&9Ds&1dPD3Fe0w4y( z&<8Gs6QVo~M*2@A%Lk~);^8H2M*$XQ2!nX~!LUiS&LZVn%7!7ab@0VDkX$7E@dQ%J zwSjTvPf$U>9=d~qM9!wq)VQ;Y$sH&dMNbP*N_Ags+i9^c4X;8E4UcfZ&V+s6VM9D2 z+cosxZ$u-KK<>Zb5cuuwbvDYk(Qbd+PMa{_bfic6;E;uaLfUEP%GGNqQZtty1D@R7 zsAu5-EE!b$-F4Q}<+%p~>Q%knjU=4fmfzVGqagFe+1kerxXapO8OZ#EUE(fn6 z8!iyd9NOcB;Jm&sT%+MlcWuAvZiL36W9EOmE9=DfC4jK}-J;~KsBKj6RpR(!NUy!fSPMgW^MOZgR442lr*$Y7`sFu>Lv zgSAoEJ|Z-v7>H9#fXc^SwzHQ>v40pgeMEg_cxN}1I^os{t$hKa8=*&g$GKcHS<2%4 zyF0gY+F|6|_zTnLth@&uuoeh?_jOZnSIzuet)+?gea>>^@D7^iVaQ+VvQh5pDZVwalCBxxDUmHx*q|EwDPC~w6q-&ThBqltdMq|wAByDcxRaO#JRZ_T5psk-M)LGZHzMI!<8!|jLVRYalv!h z@9sIQ3M_e#lVtBlLTif%&xS~P$ng0}Q@XcIl!GJT27`cX6ae@Z;|WN$(~jjzg49RA z&7teD48W;|hA+Fj10X;yhB(*aN8cQ|))qB?|M&B!hUqgP0`wC}xBpZN$i!{}xhm{8 zkK_Rsn4;n~4UD#R#JvLB)PfO8fiteR9b<(TqMRD$Wa8Zy&GY|$Lz2#*63v%32se!T?s zD=>^8}TSIdlon1t~FO3VWRqGQG z*jsJBj;&z+ZS8Lmbr}G7U3;bmC}^Mzf+PaB`_v)mrFjAGFv6)_3=!{mm2cV?{BC`0 zNH_8fV!widk>CKK?9+a~r@k~IlHB_Wr3~3gm}O`e+3hOTU2U@lIoaDTMSNq;{KzHLJhC5K*3*1upUwhtw4$#9v?jcVjVzJ_MrWy78-# zP?Y&5D>1&O@eLDoq^%|t?rcZv>tR8hTv}^?UC)MW&$C^BjmZaiMfw}PRtMy{>ZhWH z;^R=ji4cs)^`mZN*HeE;Oj|QGP~T~N7P-4PFccCT5Ggls$^rcEG@Fpk((iLXa5Pp|mJTq8P7S1Qb5U@5S4m3;C>1+ILG7P9Lpdu}Oeu+s)c@F$f z&#^1F3REwDxLrQ&h|(BU3s)-`6ynrqUKQYPQ0MyUF7iyybtIMx`iGY$nRlknsr0s* zsA>s*8Tkvy6awhQ--M{$1JRIAoG7lUIRZ(R3a87<+Ja!C&o}q;<>ySzdwzqEKKzo5 z>t8w;a?JSDzC%fj?Jl8`T*;{YbVBp~fs}`*y$A7(#zn4D8rVSSo5j(eAbv#YLy>{f ztcfbBtKHgzh@!8{7b1>MOXXqgtdG~R91`(O(C7RGxds91eR+s9dIN9@j^%GZMDE%8 zHL@P@c@M-H!q;^-NL@937dziF?>iZR`O4~uzLJ}bel~|H}n@@{TJqhQEdy-s9k*+oWs;O zddNkIGsMQXZr^feq+X}>%HD!`pbhs89gEo*OyCNkud7d^6F&IkcEkKcOOoxSpY5qC z6N7<&{a?05%*5Q-|3b@GVDpUbxt6irOg1bI1N^*w&u3W7nN~D__@BNqh9$WoB`$4@fyHam^e&Dx)VK z+r(L(BNX)q%=Y;*@8^N2aMul?`ExoqG&u0aHcEK`7=MsDn;OHfr+WXc^Zfbmc?+r7Ye{xmdUG`u+71)(POR zCl5Yvs~t&F7TyXdIdF!rdp2xKj9g6D@u8yjO!DMc+ZeS3w}Mgxy5(9H<=cs48O118 zMPV3%UEogcUG_P^u%@~q?fr$3$Q;YH#m_-F9^X84sase{iT=!^DcG z2`wJGka;E1nz)$hy_2yr2ExB0Cn>FxU1@Vi9P*M-np$=u%%h5OLvsx|7 zL6P0B-tYSDhxA#m7hJ?D5|RD|#j-N6YGig*Q%4phO0oN_8;S%!VhkAn6_7ZxOCBe> z8Lrrc**X3OgPFJ*tk{^P5YqSL^Zon&Z;Hhv>oZ?Mj(3gqgQrXM@hhZyIpe6w3BBfp zNDrC&(sVOBBON%WvR3j1Xp#fj&>&hyB7Jahx?BbAHS2cnjttk9JEW0|P}ICem||K& zKz$Kfw>Ya$criE^`O{bVZRo74sb@mCEM3ubt$l^iS5siVpdKi5?Brjcl%2ggUrq*y zy^C$PJd?#xjqG1f0M-vv4~ay>xZ9>G+j#87#sZe7TZ3^C908%ThitnphS-+u{%we7 z&a8OsVOLK(bjU=cv+-ab9;##ecsZkO$XqaN zamRdZ@?VIZF1D!@zPIH8qqr=Z_Rnn;guAa?jDx#xd5lC(bjKVy#d3%Jz(A`e0zLc>M6LsKJk!0o1nY9KD$ zMWVToDaPYKaTsQ=u)HcEZ5isWi^7J(UuIUKL*E&YIt|n=_&Hok3=MOwOj$SYM-x61 zIDIL(;{B}rx%Jyhqh8cxp6G{SD+Af@qm^-%@>lv^SV3b2degC|R@#$71b>T{7exKX zDtH&GS{X{c{0c!Px6a((mBBW^<)Z3;ntsYe3cso%ZAW&K3yP9ie8Lh8sq8o5;3`}F z*5a5WMrE2$PT}aL&UX}%aZF~e>u;4guizYaP{Cw9#lG%(2d^WV4*ao)ILRF$3e=5O zH|*Kke6SUmvk!7oN94>P15-azU$Jyle{{g~Jag z_p*pQZfEK{AboO7aMuc`l#&E9+axutoAGs~ruW>>kPh~j)~uFL*Q7T|JWn&Wjlc)_ z%63*9Zo^bfXEWVZ!+ybKas5jNMIh5Pp?|J#O%sy-q|dDQY!O*wl&cB1OG2XY_-%XY zNE6NnS-P^b;Mt3OluSb!TWL-oPHOM-BJQ42+c4NiHLC6unS)#D#b(k5bo^ps!D6#o zW!w79f5f|TFlyYAEjOc4*RQMndskQF1vVg5V(+r3pT+E#2D98nx!ApJ9g%G5HW(@l1V@yIfk9xsZX!K{oe1q-Y)zG7l(pzaZ<;(mircf1 z$vgiZvSG8a;H3kcGh{oNT~kgto}Tm=O5mNTw94Vj7SeRzp@G+LR%c@WNsO+`$6uF5 z^Aq=IFyd^B4at8tB$^iI#;Gv*`oH5cKBV5N4$vbnb_0>zJ|(8k%?bb_*4M*THHOPV zT(yVi#M_f{_^2OZu&1pnB!vC-nsj)Gt1HML+7?CFjfJO<>o;~f!k#!W%*OXgVTZb zQf`6Sp9b2j2%)B$;DF92Sc|Mdp$||nvYt2|!DgFfQ=ogTquNhBuwe1vb=@ekug}Yv z-|qI)Evpj`wdy=GPm=#cx;>w$t`M$>AZSS#Th;hQfB&+3Z=-As23Q8q7StoTMdIQP zQE6r|Xx3y37?W~@UqvhxfqCfT+geiSmpaVy1NWO0jn#Vs=+2zO7fEw^*m(dp30oNa&A85*-ScMG@lOn&(|>o(Xof{L^4p#$-waE-+o z?Vjeat^)jNt4}oVBun1T>D1xAtk=BN6%ZA2&~`evvN{slB+59mQOS(+JklGye^SdW z_2oU!H5je^+n=3wpv*29{VT!R>0YV4a>okc>9aSJmlL%-K2bW_`y#=t=WtZ?ad)m@jcDju@29Y^(27JR651k2 zZhRgoZfVPMUDNMUloj65)Ai7mdJP2Ps|KFk_&qb2vEFs*plo%0p?m3J<=8zCITv8H zX|AjL&}(DFvxcdHkEJ()CXQC60xfV@I_}N~lIc;)6ip%7}Q8X_3U<94Xyw zMfa4>PtPUkG1HG^Z9_S7Ft;@78EhVmNP5rAZ4Fm2IzcQbfXcKaKFkfRjn7?2`Pim3 zqnUjiP5*1|KGI!sxa4H6@%*f5MS{)FS!P;1_Bi{{@a$(D=#x+inwZBi^jAx3CrEV^}%%E*?8Mr4_vV3~3caCu zIoCf8<(by?k3c|d9)RBbwX1Q9B`qG(8wsELAG015cqAVN*w9!R zhyV5XG0I|7oM4!afYF9g- zIM4Km-v7xg^D&c*C^)D)Ax|z;h0)mWsRtpmsWbb}95^`i8a`#xd*o*j2av7ZO2W)% zVsHrZ3mYkc%b76n&}};TEat(WD*Ls>BUdAQx|_~P61AAP7-lz*0h|6@{itdPS{7!s zEbgsGyL&z@1Ehw{RyN=B*Tv+G2BU&p2nELnnxBb9R&D-CK22RaA;T^*A*cBl^eaZ8 zV3ELG)iaz_rxF_W_+A}zKTPBtANa77C@#Slvln-$CV8>e#RaObD)wSn{WaUBX)X!(5SZ;p7N7pE7}-2P$qRA`5kfw6tPheK0?gGl)21| zqi$}OYaWCEDcoiEj5cofr-MXWHE85Y$_B^^UAcRtdyF*JXQ#pjawP>JG^!V%_pNug zanE2n&gnpX(amdnBQh>(S)O{E`4y(ZvuV+pT@wG3 z#^InvRol(MCAI|I>9?v;z~1dV?_plS{HtwOogQjEs4e%~)*q;uC(s&F3&(t%dfv7Q zL9%1x?c7z~$`7NHl!RcFo`%$8$D>QVV zk`5C8mK5>deqn50eax3)=NwZTrmx_F*%t>r=Z%#A&lgLUqytzrp2+whaib4v2(=dvL8RavyKaC zs(z_`MD68xK%jwe6%;y?KO2HR-urSpp43U77|4Q^gmMwhG5n*ITbt<13CqX6{QgTF zXa77s(59{iC9Ol>CO($q@dy9R5B=ThcHCSXOEXAQ?QI){RkFEy%cYncS~uNZcc`e} zX7+T_Z&xYe=6+#AkACr;7IpFyBFl^E2-VqVCsZ!QSC(kBD`h=pJntS~otdUxo8{MtHCefFtOO49T#ppSmO$qmkT~QxJ?!&h%=lGQ zBi~T3nA4Y^vE9O#p#VR#Yd81h?op^Udx2u8YW06r%p;2Z%>GwE3gihJ4FOga1KegUzss*RuWWsaV1B&ZuE(ipbwcuz`$`r|i3`7UNG%tbN&DCJcXd9pE*Rh8qW zS^I!Y>f~Ntkd&lvPiMp&tLyzJ)=Og3I409pa>Znfwgp^7WZXc7;9ltN(XY$2sRzyF z^;asBBpjRWd>RlK+(*MTfvv73uuB}r*|eWhZ>*)+H(!G=G8)zbcEPu5tzKehPF7Vx z%d&@HDynKOAgOA7_lsG46^n^!?C3G#94f6)w|QiwsLb;EGLk_A;oWi*7c=9a_t&`yM?4D#mgUZ>BAvcCa)5L&iSeek{3(Zm+^5o zYKKTk{U4QLKK?BMigdeg`Ul+oiSi!Yee%4Zf$TCt9s1bU9`eTN#L}eg-yM0RV(g^b zR{4c-#|wWdRy19WZjU@oc0OX*3!T2J)KRk0A@^R-Dc`~t_G^>&5m_=Dh7OVj@qKtr zpCcKDJuHK{bE}cG-Cx7BQw?%CsQaJc>Dtzz7k_4LomHj(A(6&Cb$%;9A4{>^(Cxdz zBXFethi&4VVH^Fyd6DmH`Eu>Zz)Db>bs`MKV`cuT@{6YJ-w^W}8WSP1T`f29PHQTn*Jz#^=98soPH)dHp6Co4 zjV)uJ*`HOsPBYZSM)f^?)B__fL&6wd(IAej@bT7s_DkbPEA3z2QK!FB(ng~d)g5Z7 zE~uF%t9lN9lAqub>Pb{Rq7y{Fb&d7okoXP!^a*vF*(qRbf(%GTFQC&j6AZoSu3!m9Nl&8*Hrc;v+Q{o|3yEeK3i%&vHV1R6eeymEQV6n(NVqd5IM1E=&wax$zSB*6h$Eb_?yc z1yxY-v6A4(k2LMZCJNnyw@!l@ys*S)$(B}I%N~$$Hi)ji%Eheb13_UTjwaCY*vV%H zb=(xzX=MJGg)Fy|FOiScR4Nq_rAF#M{ca*hRX5!FGPjn`EcuS*6lXx*$guXz9K+c7 zimOZcVw;MB0_ZHM>~c>pzTjugQ1<~^9g%IvGUIOr-J+?USk~9&y4=eqeAv&|NCK`)3Q;ETBZ-2^P!~&HWK9Gl|{uuAlASCpuKgql4B67RTo|t)1;cK zUJ1ukfxeFIHroQg@4}q%M}oe&X_ifv#oZ%|K<0 z{zCjTyuNq?*|9x)*HVZg(3(c z>mP2#Vqam-*39}f@xB~ZH+%s=&lA@7F?-7uRU4ihS7(m#C-HZ;hHZohKE~eNl5;hj zSB~E6($UN5B%y~18sQT>V_2kgZ}wgpad=EcFX(}mdO{rW1ltQa;SCrFQxb)ODX^@@ z$Ca>|A>qRDMmB%wX-8bjcV+lX6W^}j2Y_q9Zu>Ai#JtOGuSbw zyLu(yd`#`~?&@g0{cO^+AA_M?S@$A5B>xBpA zQ(fj%j%Vvs-XT5i_fwze>Mta@)C{hi$v0w0mLLQ|65P(eMEX==y5L4)P=QU`Sga9W zxu`)y;$b-*7V7geAJ;!b&pki;T_VMSwUH)9EX_L*t4+LAaALzq_S3DMO}*D)V0TCo z3x7Q>bs?Yl!$(HT`T8}n2D=4rmh%lSGzR(q2r%3k8EMyB@sqAsdst%ajlY8s52e-d z0@|#q6LoM=FOVc*OSZ#NP;@n|gg>>aX98(t|`2 z#2G9{$1aZf>bPI{5p-BY>y1Pqo;7t+?@GIqAWx;n;0|{qZv$!Fl{L8_x)H4hY|=9& zyRE#GPuZjkVI`!lvvy#yIYkjlLv?ugr1$nxlRXDkEw81*D34G%N3z?VkXFWoz0?wM+jwi*1FAejmeo+;t1gjr2O!a$4{Dy? zqh)jEKf5CIw)FzQAe@A3NW|RSwX?edK_{J4QcrKf6~95tf66_uDOPwn3_B#{y(0u0 zLiv9?N@z{h_6vyevCSW^dfGpD!bsQuY}4`zAa$fpJ~z_(-9|HXn&Fz!BYzOldFHrF zYKvx;`TOdDrQjWzHkK%j5Ct2ErKMiw&)o`ZkA&%^Mw7SWez6B*8W%alDuBCD5iYVpKc)VXk^d^zlM&~s@oZUEnwI&41#Y)NFMfXsXp?;tLaR#7+N!&750FvtT zed?IH_^yXq^kqT z(XQyJ6)bf^*1a${#;x4x4^BI5{ZG&sVO@KY)1#}WGaOy1yN5$GFkWlQ_QAZY^j40jU z{=x7Rrypwyq-K@BC^s z8 zji-PyjcufW67CP?0toCL#w3Aa5R^u$Wc|R^`~{?#2k_xx5w{`leAl0j)Ug%{KnNf7I^A?WUEQBmVWh?Mh3QANjk#K9 zxYC}|EZXRxrX1Q{HzM@V${>gCu7D`h*wnW=f@zDyV2CaXD>mtU5!T zl43QbDa&O-*WL)sm2t%01^msvb(Oatl>zO1B!)I4aO`gvFsI>}jYqrjw$?--O(_~) zgwhvE+l{vw+*xOUK%85}oSg?nz*O!SD#?DN*Z*s%YAEgxRHst%S?YJr8p1}M)&-7Q zdxX6alQc3)a@*V!58y>8-!NJQDu&Jl@U;u6@KF=z^YBxEtx4hLs;ARb12XOOsy6u! zxEwKb$PFlmMK=|&63w4f;~(=|fIUGd)aJ83&_!rvWItQj{kD1?qD)2vxL?dNHW0pM zeTb-}NZI79JqMQzOaP(5b_sq6G!ADf+}j zU5-x|1%R=-xtRX=5^!|(&G-p~!fRg6n543(d9TIm=7|TwC!+T_1F^;#{=YMXKa`t+ z#)x2qOtKMm_>U*_u+NQ_j=e5V7fe*x$89(PO&I9z?gXxH6xF%tIZWb;)=QZwc^g=Y-%LO8kzGmS! z&XYw0&SZh@KfiPtZk||x1U>&Z5@F|TJBSPS!!@F)-~n3zU1X~5?Y%=JoF6brA@B5$yW~vN8@CTKa<=$>!pINqJ5fd`t_20VgUwbu7#x&NaBmHQtAqn3P=Y0-L z#Z7M}9leo&tPUQ|D4Zvf62+CrGMLKr0fz9LTp7e2Y@;KF_RbNLLK9(5sD!I9vLX1B zmAMb1&)KJEQXS4=pfZF@*****&`q+)Vfh|WK5QA!#R zu)Ve#LR7BSB)P`&SaABbHD@p^+r(%S5ItS+2M|~0DHpq8L=0e7Te=4#)k(=}x8A=r zE3g4)E|9JN&=V=nAm-%&0DKW`7>*Tbg>pdL293raopKrSUrv1qiZ+}3zLBb+0M9Em z#x!4hp5tR@*sXjJ<#PtSt!MP}jsKwL40l*qnsQ816k%Vy5ZoLQxt;3O__)MPAPHfs zYv1S9ADhoIVOZk+y&s)va2f+3)`9@1Z1C{ne-FP2s-DQxqa4IIY1-!aAajNpdzj#C zSe9Q1!_%188{?7sg2CWB#?fQ>|2-D^cm%{Wc;NA%M4Er>3Bn~hL)NAs`x+c6e6{ zy+OSR(LHhyBNVDaolhZPJmK2UB4GZh4fc2l6B8%RfyVFjduxg_LiRGSh*Pd^TQZ6((U=}^MxKvMIoWW+dMnVDB>+Id3&dG2G+m|-S`Hh318tsn zi>GOo&IPauQzo@)rErl_Q3qlZU~PE-T9YLBbN0i$wDI5O`Im#7;0LMsEK|4hadU!Q zjUdgO{rY=0yRiE9I&r~WeS5D6dUG#Yew%j!cf1I0YG!#lz|X`TamwDa?Eo`)^=S~Q zW9Mw}Bn;(5yIE2++d<2T08FLaPxy3E)b>nC2VsFnD|?Sz(XG+Rq8k*5*jEmoYJ9U7 zhI-JHVJV{f*pYH>70=w$!XS!%b@N7zo0|i?G$O62ztqy{oko;GP)HE@iBlfl%&3;~ z4}@%wDmrd719rEJaxIi(knce`1SQp6b;YOfr!U{xVWZD&ZNI;BXV4&GS3)>a3ec*)b;2A`4!48}CU0GMBx28#tNePk4dUF3xD#t}RK z%PRt=tCy1U)g7zFEkJbZNg-KwWQM@U?}QBEXh$LRkdX4cA^?|(QMz0&$|0_-r$~{G zHN{eZa2%4Rx^P_HY*Xk7p{^OwdMHCG441%jhr795gP3pA4yyrv`fEw~4n&7V%mNhp zrmlnSjaA#;HaRr7;FCE*c1u>I(%r{5tOG;au@6;bt@n3^^rr(Zh(cBer&xBu(x)B~ z6Cjam+EzVjOM<3#36h#cf$1kq*l4CVQuh=m2|RKORS>>}>CHMI!>?GbF}A~ahY$pN zSD<1Atf+|=B#sQzP8}C=j&x7#;itnfXIyZc#}O5T{V~TJp+}48y#TMzK}?#LU>Lv} z#qRs}+Dm%Xi`uoi4@%U%l)L~vf0#^ws3(~&hmys|s$<%bwwJ=MsXMj^ z2iu~G{gd3w&c0PQpp7K2R3|sQhggN4H&*?=9pK!jUk#x?>**E%)2pD*M8zS<0_okw z5S$$}Sr6W5`8b3kJW?utUYmvZS?B(4@4)f)^jYYnUaJTMI0j^N+ryLKGif%jPtGnv zNw5^1#fJJ`@?roqi?3O`+%G@>rH66n#NPSvZaXBx01iA4Rui2tGR3=%q)ejWX~>vb zxKv~x=|1p*CZ+$tf?>|_P;4gGX?yWgp>X6&kbuZn#WoH8V%S;ZmF=S~js`L`uYx*e z*(dxn9AryU8xA0JD;AK05*WzLflZpuSV1bDz(&XSDn^3=PPcHGv9~gyZ(Mu%VEO!5ku~b4kJKr@7;}uH(-7Y94O<&?bW)$BX2(p23Ktv6h%sb zKP%o)Us7Ag+aU*9a(zdrn4NGRgO}}~!sZOI^ZV?-hUl{mwm8@HN`vnPv9B4M7u*Oi6l1ax37J%|w*&|n0}HCy3rPj4JP zW|}m;H1DHfxp8xLP|bS(DNdX{*!|XN0^m9+s@Ly8Is=Zhgw(&c7P-T=fRdVvA#cF& z35fLU_tlBvcj^OMC+dj%cos(cF>0U=?75+{GcnkZZRNtk;7`s#Dk@k9nozEI1e`5q zp$qD>VJ}?Jh&o`#oPvD{xmAXw;Q?Inf2g|hKq&X`-J)#8XlzB+3}r7_GECMG(N0OB zq9SWGwjstAk+di>(V|K!gp%2;$ef963U-2OAXvpx^DIkhH{&1Kd^^=XyB}!^j7u@SrUU|rO z_cl$N#4%7yjTzc%sF^}!t*;6mhzTQ_yCFGKX77!>;zdye8_rpy@iRxV$; zYT;!OL=9WCBFr9~jE^cE@&TK~guw2OJ;UB`7jJkaPx1OVcq(gv2|;b-wJK-;NlghR zTWc|DIpDy!ok4-~fj+WloK(hf9fR;eaQRd)^n_t)g7N~3_Ov!;GjK;{U(A5F>af%M zwt>Bu$|;l8B(aJR3?+YQarfNz!swDI*pHI%*}8d7X&c^HOp9$Az&@{e;;1fJu@p80eagm<8#lUy9tgHCY$*C?*hnp4v0EP zPvru5iW6?m_k~OMmuC5Z>MT02WR*qztxNAfcRTm()^_NG!fR`MKeW+b#4`|M`{S-7UA?5Z{3P_-+dts$_tML^F!)53I@bAgV zWLPnQMeMpZ8#EY4pgw;NAwh4PkpPd?!VMA?e4M!ZyEin|j$BgBvr2?N!AzsBb_JhO z$a9<_2VFx00?t+3BAGW|<%HA*gf`L4nIxo@H4@A6=;N02>c zqIA0QDJR#$54|4;bW%0gxEFCn4&%0z&fGQ%IZxHS8P4sI&4XsVc$%+;gC^D6je{Kc~Xu@o1Ev_Y2+UG-ceb49CD+ z(Am&;9YO?N0U4eAx1N4c}FmkXD` zJf8N^Rh?ZB%fCco{=m_@Fan@?aTfzZcxB_*e;1>|X0tsj;>$N3d{F@K%$z43fab3PZ^9KHjVO9Ds-E=-*w++6hL!$OyA#O5Dnpi9o3GREE^RnB?303 z8hiw5_`{hV)Ns%Zx_u#Pyopp||H0YWO3GL^GmxhlNqvAx9}XHxK|x^~3y#>?Wf()A z)s>pp3XeiU4`!$LwpL%^iiZ(nR;u!tYo#;nI z_;P$}5Of4Zyd-HJZsw`!4WOVA8$JqsYn(m(kIk=))W)BsS&h z%=l?~p}J_NRMO=}1FRPAF!=_DM(!4zviB5ncXe?%q!o1k?N-Ns3A2z(ARa&*{3rb^~*H#+t!<*xMyU3M$YnNt4H`Fy<#`G0O zow@$7IHhO%z#7xHqn(g~|81h}&b=YeZTu!^)WDYp#YIQ9C=UaLqD+m}mBMRW-=ypL z6ixAW9gZ_a3A*2VSFve8AvA3`9{#nx3Grt<1#`E%*Z=U+?|0p^BWsDKq>B0-RbR*H zpQ*Bx*lI%PfwG{?3@Mx`Uj`%_x7Mpx+pe26_BU&z#+z$M^J33Zf3kEqQ{&iuX@qdn z`wG*v2>GJ4GVBM@5m}5~MloypxtzI(NQ_+59TldcK){}~y&4dHY|&>=BJNzhrBbYWvr5u!+a~PbZukb*%3J2UNgFM zG}^H*gK@cO7T^1s9kycwYMI70KiT)!Rs9NQj2^`CTj^A0{b2p*O`o+9H)Aub^9)_^ zo!F;BZbDhrKB^2#*G`@_37bo2nBxIQkL}$fsBU^qXTft_%)bwAT`XSw05hQaDxN&1 zB_qCyrDecAQRp-+4Ozr{$tTx418O#D(6p=EKQvtOD$&HL5o3r$0oS0j-1*w!nm5~M z+O9%J1#N3?Ik@&z50r6GB$u)t8zSM-eMV%wOkSaS$KU6!8QZ*3e8Vwy9|0V^wz#R$klO;LS*=0;D z-D-sm=_%$?M?jnq{y}K+v`|2WqXXTzCaSKyHnJXq7O#0ClVik~tGS_G8O)1(qoN=NTO+`^pL0nl>V})>!+l$W$!op7;C#LkuR($TMuGuBNuD&<2acL$F zrx~Y?$7dbU;1ipyKf|)oHgHqA!Oug(+N`NqS=Qp3-A8d>Qx{MwuVjm6whZ3C+NeI8 zOl5wIpw;js;&Mp}5?_0<=p+$x?F!>iPr-&j5nATxR39NqczJ&(lcR#G4k12OanTyN%rbJ^-c78YTy845 z27P(LvK1@OA|md1DQD~*x4JnjGrSp3Bb=U>ellk3$rVt>I&Hcajdc-Lof(R<*2cQv zmS@>%*sA0!9$%p;^7Lo^n&3i=QBpE3;+UJ*4Qn0OgQe4kwzizap>%%Fl4Qo~kY}1> z>C^(9?nBxlinh$-pEzTyqYXB`{T_JbS&SszaeB_LBaWvGSNe&iZ_l^f{B2_yHBPqT zV`!zpe8$ev>n8uO*5Qp3=@0r}7RE(1@Qw3*k(gSWMed#0ONvJjQVJ`iX-?&jk1fO0@r%%X4xEG`KVB}&aSHTaN&S_$52XtHX-l?jgyHFzY1Q{{ zR`TEEpV})OTEKIFd#iT~|Gh`cUwBZ*94>l}M zY}zhv^`)TR%roY^orLDgPc!S!lpN?d8Y-dBoTy;DP#z4oXizVW;gcb0=38Wt&&$-{ z7-_U>BU89e${1tvaeqRzjDFH;O?`!=R)y0glpsQ8&#$o~uchMhP2*BuNmNYT>L&FV zcR%Z$l)`4Wpm^L_G)zOk=wncnc#{$&bJ!pNzUZ(Tb*_CG+&yf z%%|R^CHi#D(-Z!^UBH{PczwLO@~Hm#&G{bZuDjEXZuS1M*25mb2@NHO3J$@nX6Hbz z$6n*MSSp$MKJ|dvo&))(@ClCiblQc2-D#=RjFJmZL&vlu+TMbpJHG6}uu?loCyq z=AQKR=FSB8$Y&VDIFE_~;od^$nGg8}ZNFLDn&yNn)PUGDGmLy2tuMHcujWE*2Vvys zWNb-)NI+r8j(@%HF{P_Df%D0?Gu9ltYEzls(vzC-^m$APJ9RdnM@RM7P%`xTV< zj%&^ZQBxPKu}@xK``mnI!yQfA=eIaHr8L24Y6fF-QA@oKiAw;a2}#-6$-Iw(>pY3B0x{iV`kqtHnq|GY6`K|Sk=$d zE^*zaceRwG4f?1*Pl!Z1m{p`_cnFx~`)=oKY!&d4rN*94Eq!XC6TmHtnTZ@vXxVlB4MDaCwEU&N%v!`OSg2K8KdcZ48wwf<%#;-71yhccDnn)mj`HQ;#Vu(nVMVexE zY%4pxVU*8>P*={p6slPxHMRP|8}>@EgWO|LH)F#%(Kr`cSj>*p*?^}3vOBqPrbs?^ zcYc2Gx)_b?8;D;1Ms^h$MX}ams#7`X&xwxj2>bO@^)HC_-`C2)$ewNps#c73tWvFi zQBzG_Ms2L&bLdR(sSinv>vhIj-b$CU*ey__JX4!`xy{1XFVEjXe@fy(^>im?h}3N` zPvUbV>n^)-W+Ym?(}*nL&UYd&B#(Eq#K;cu6Rd}dWo^972;cUO{)(I^8hI*lW?jjs zfo0jFq2)BuclTeg6a*IfwDbkjY`0h|`DN_Cuqi0zUe*2k7&B^N%z`(@k(K;&JIwg3 z@T#ugXPrx2a`ocmK`>^RnpG)>J-b@*Djj0W- zHyBbmYqI$?$!n%G-UZJFbTB=hd6KP|0e&5!M+G*H+Q+3X5(u`aytQ`Sb*kDxzCKT% zsks`nZr|VLwu*%YQ?HHc!WyTQc@qXn4_7t}C2*g9VQ1iQ@S8c8yaZ+pCJ{{ z@|aA;G7=DAK9{7ZUzXn}_;zmNYTKS2HloW%cUIffMbyjuy3sO`(z!9z(q`S>vqZ>7 zKJ&)Vc7D8=dC8zc=hHwQ37#{z(abX8G~fh7G$V7%!9lF}LqB&#Cf$yyRLta#1tAMm2h z7(-G&>BLEt{)lwVAUn3rmv0%UW>~Csdn>4=Sz^kSJkO^)pVk^&Ml@VodOkwGcMF*o zV(K^0C>ApmHJ-3U;OP_C*xdstO4t*7wH&l_{Xk(;$0vTu%Au$jDn@zm-II8_HxMGu z1KM#8IAO&fDu{n+hs#$?oFLdtRy zm?Aqt%|V+^)-qqBFd-y^_gt+G^hG*w@6IF#H4Em$xWD)1FlY=nfk=u-7OoUg>&ii5 zZ3J4L^e`l^wPvqfcXmm-3|dIfbO81gUkYDwJI%~%%&EjBBRna7T-v3z?RC0koKmJS zbrb(zHBGUtt2H~m1asiLJBKbDfhdkEmBvG9dE(18t-61$abxN~fQcWrTFbezc^;2( zSqaRLt$gaJO1-CkWo{*JkR1GA%X|Re}M`(ORkx;+@!wsOlsicV($2AHa^Mvq*8Tq2ib!CuF4a z)jUmMj!$nx&L(`0cGcGHZ3>4(D(TZ>zQzXi85iHhTL zL(jrZMFPg#`1YE6O*`vSC7keb_As(=Wc`RXb$w}0NoAY=sC6Q6h;AXP1GJ@Hav89a zt8o7Gk_Oq(3K54@k@33NFuVn^-E=0c+qfaerv zrHbaM7;F3)nYsR;x`O2{J=I2EcwMuI-H3)G$U7vZNe7_GE`oiBVYJj?q03ZIUQ z;-N6k28u*g0>=U~UK(~*ih7HyQKON)j;6{G92L*LRJ&w^H(Un8<%>Wz=2GUxXa)1F z2_|%dHQ84dR|mTice=M0(=VQsz+K@Zial-Go8}nZQ*bdtPIwhNC0+TtVB~W8MY%E8 zl(>`*-v*4P8|z9ir+NEXZ4BA|?4u2W0>)GC1T|{J!m*paS8q59Fppr+hKcv%z~xmk zm2bRb)*;`FIaa1jp}3oFr@vRIOq^A?PaRLSz-VSH&pYC%wxXop*c@DHLaDD&=~>|6 z>Mk8LJ5*}v{0w-fZCe?yYNscH5_;R%dcNwrR-AV>q#g}g|CG6IOpm26;YD>gn73tW zHIT91G5hpOeGFuhgr91)X1X6wtQ0vG_kiWhzdLTxCh+T|>d#fJu@r-EjTUpgq^o7^ zE@mK|yc$R9-6JoMhFN1{6(=9$v5)uor#oU7aIv}1u@4(;46&YNONlEvFcYu4`U^{; z`$W~f*V8L2wy3Qma@vz!W6P(EF`MdbNRA4bQxgV(y|n1-T$Zy)QtVVcvxzPS!u^Fo z&-l8=SLk8RLTwxkk9^B79dryJK1cd#mUZ|DPv`^G_LtwmmhIbJ~nBObTTd_}Dj zi@^>iub^J10?O}1ho;+!g+OUo~~`tFT# zD#$DMYWAjS?RLMO#vi6HBV{Yje=wmo%35**J(>Jcp2|1u7HRj>tl7eovK1pYGc~Gb z&+me*5OnnutywN(QnMm*xi-AmaRq&TkqtaT?+4A^^4(LMO1`qq<}1_8<%aM%rQXXO zmelr!{k_i=N<{wQJ65A9Poxh})#%YD!H!Z^L_716%zxx9 zb-jcXepl#CcMKtJEnc4dDrF|d*F;hd<96n1;u2i0uqF#CIoBj5vKQ<`;0~!kg!N?=NA2aCm?ECs+ zAE@lK?Ob1tM_xG$WrdxK*P_?AMtFarsK#zbhi$hum+uR@RK7>P*Vs*CeCMal z_=~;l_nfWLv^D#@?sYHiG=5_d3rSq9@QI33aZ*WwyB{q+NNa74u=RN0CSkPd6fAzd zs8L+`4y?SrG0$TN!=S^zJCn8Af&C)}7OKKZ=TRT=FRpk}y*bdM?YJ%NPpzq5%n;ZP`ZUH!_c=Y-$SFT!zL;VyQinr6&jqp?D`4d*bYL%*U&&!hry0A(0 zJd9Y}BP1vty&U9rMK>Y%4Sc$xvaF*Vt5>dvYn_ zE7!SSq)NmseXRH4^%nSS@^{7yTz!1q13J-`$Q}op|`^blFSuA_Y=qKpp) zP)m=A;iZ$Kfi&^r(K5N@YPOmQ3G12^N68H-Z8iry-HwZ5HB_69_mdY8Ly`FpIg+@V zV>b$3?3P#irBq%SkE$1QLy$oY7ADf9bXz9nL#nC-w0-<9aW|K`;U zuF`4B{|>(<30k2Q^`K@C72MyiiYU#4D=T3L0g^hcf&Fp(!h9+3f<8Wu0&4DGN>zYH z=Y|)5)MmAG)f$gn^BzsVaFcBqsr>G>73qrFZu~K>jMz{9HOW*Z0YTFYRCjwt(!P0N<)*Utxi>$J zR?ZIlz(v1XrBt58iUXd3f_A2o%9u&&-Z#O(LwEtl9`(-0FKC>jj+Osb_m0%rk}4bx z$c*QqptdU!eDuTTP*!^wWay?bx+;!x)2Y|4ZXHdJ| zZ8X55nIb`eQSjmabS+I$U+^j)Vf0cu{-fqeDEL8Tyb{BhfVy(~PerDZbTa?rBlH!= z7!pafJgYTMOJ;rnd0oCuI91g^M{u<$_IbimjYELVt{61GXfcAfujm+1p zy(A${)_?yGh7Ch_Z^03`G9FQ^c%B_Kb)HU*w)|{VN}SQ(HI@#?JOv(AcC2FtQP{kp z>@WsRE71$p-*f)8G$43kdlbU6Znq*53~<6_o+KjDm+T&XsM-hGe4@j?630 zEg)Wg6DFD$k~A>fYeCAR%9E#Ki4_CyQqPS$U^Z)Sfz> zt-Zf$W%~Dkyhkxa`~#z|n{+o%KVYl3t1(FqA2P1kkEOe5a&|*c{HsnRH0kY5TPpem zp)x7k(6p;fCTj@Myw}SI2CII)0z12=NBbftt83@b3DPOw#@?F_RwD^&32vKi`z7iW z127JE9Oub-6GQ~-IP|MtW2MFU1xwouKK?j14$k!#!aEeOO!y&A@$jz`LD7}^I-V`B zy*o;@f3?_My1_E)$NL^VZp4juSAvg>lttYcY(#$eV(^zkmMPUT&J>Xg;RRove-VBI zH8TAD%S==VYGnRM&SRq7I|6qTs#!@c-0o>c$WV0`_=TJKjccUu;R)fop?%IR6KU!X zx`2owfRvlfe+0rr>N%$mKr$YZA)7|-sYCqTBOuzTdt2B0N731D>t1k7mIP=n+27T6 z^4s#)+Iu?WC#59>NWK~%_H5B!=Q6wNI3n%9qhPQ#o)>(@RnA5&Pn#JiUa1@$6h@5_ zz?u|4_jFUmtI(V5In0_{KdrUucNPf039irVLE`qWI+6A8J{7#~>%Rx0K7j%)3}5F? z)n)1*wMvvFbRzcGd*x{}s4R8>xo1mPdw-zAY~rZavMV+b_=oemJ#UJIz9-)MzL)EuTC*a77)B7hflYkM`{$gss;X?P+?+c6}aG!<3WFGIG zD%PHgW2s1i=|yqcxHBFNnh+a)JtzWU-HE@ag_41%_YHW$lWB81vW^_M+pyG#^f}iF zTb@&$KmjnqWA0_b0tx&6F5syTQTl)=>9zP&EbbT30mA;ta>)dpW~b80v|4+JnuZN+ zZw*S;2-oktH3+8rOD_S;THcn{c8NR~$L0Fxn7eRWeSxt&~x|4!a_5w`@9xzyRw7ED|xZT(p zEWumT6(wx!cbcpazrKyYEGexfq1T68-6Tla-;yRCcG}9pP>VVV^>s|l~ zIICYNLwnW`(q5i1_Aro6?D@%1Zc-{wh>{=}hZ;}HItVI-4G4k7$xeJ924g}ZSLT3p(Bb}C;raxQ{69|s+JgJ^y87m;xKDtf{qOrZ zl?T75v0rLJz~NqkM)SQIno@Xwo;>MMpg0`N^0OSFkKO+yP=JrSVW`j)Fh4u47xd^M z7NoWeHqJRu&nUSKfT^F**!0`hO$@iU&N;yhi4R2+&3xXlq}Kj5G*AhbV+S-+6y^Xg z+?_xH+4e6NIKU#(_~qhJido_&%r=XDhS-F4?$iX?c|#<)_2*5w8%{yQS@idUeeRV7 z0ak7HI8@rLx!k3hNzicj`LeXZW^Hl7iVi>EEhXSzI*AkFjB04`#BVTqISrhwCW$*r z9CCB`ZAdw+w0fD4w?#TOs}^7KD5wQ7*E~6EnYuT)gX~_=-L(E=wKc z@v_l^G*&V{RjPCnQHQ*d_R*i0gc~uh^Zhn-263Yvhr-Oo^B}x@lQX>>HxoyJrfKCm zw<_gwMV?dsm$VxEIoM2%BtO8$P@~^M#P0!g+M;1tpFelAVng11juS|gkuNt8RBSJD z(D0mv2obNVGRlSC1@k-OJ<6}uJRO!SqaC|3)7Aaml#2(zbfiDsXIEl95B;)pNHcGH zn~|%6;X>>S#%|vJnSb9K;A@Bb-WeEV13HV=@v@N!0eqZIVr?X4N$Myzii1YI&z>Fb zDE)Z5BL=d%r?W8Y%o}>?piTJS7$sDhs7V5q|Eu{T)4U81u;3#gv$Tc~zC8#eBa_nj z1nS+gfWV!ynpJ&4l5Y0#pbQ#nu<+oJ-jp33Bi~-x zv7%8bN&^?^)egrA8V^;>7h*p%H=TJk3H3wA+hLY6y7uzy0EJ$z9H}kFy`Tl?P%}bP z1MjQjOt|@;fN|3t9~}i(*EtA7jqvR-=tup3Uoy1B95_knC9iM{M5x~|1Y!iWz!41! z4U6@GVb==mC^>=Q!eyO8H85*1MD2q1!WZ*JpO6X>y4E1Hkb!2IYt5^Lm6$^`vIZUn z`5*Kk!QqL5xXMD4gqXNGY$e{9dcJJ%J8~qJ<0y1akU#++<%+1!-FwKu;m;n+YCzQW zZsFKiYp};9P!BBXInHI70002}>370n`Gcr4spr_$>@V)!>NYEx8V?C2X@cci6;=x6=?)E0N3}E~eA0juLGlVfWZuBc)->amBYAdAk&J1 zvOb^*Y=#H;?jlfrJ&RTpgA~bcWv7iKS| zYy*;Ze)x?{Fz5eXBimmXg0^yS>R|PSG>2kInh*=aHXa9gz~3>0 zvMAxr^u|@8ZS4?;KF%-RIk$$efC{T&{Z~i-&FA|gxX(c1^U>GCuHAw%awInjrwS$}+i@p`zH9-vFfqku#E!r2QHw1I zW{*Qo)b>*tkO5)Y$}aM}-#7QOlmDhSrtEvhglLnqFZALmJ3C;8>o=Ihx$a!sVX#%h zuWX-vLGEUI!&(?3KlN4n_^x(Zelhi8*p&$Qu zLBj#1hYlQnU ztJ#hX!^2S=Is}WCvFDwt5@a893ub4!<}=|Fc{G{FAiOPFzb(-7j~~tA{DqpI;4->! zVS0{)E*rK#(pR}m8u3&uDFF6xI*sEf3vnaDBKTkBcGn8G-tk^&x-9^w%v%frD#E1{ zME+zBv2849U{VN!;65347^1_rCaw#FdgnEOfTzX|n5SH}L#3BwDI`+SBmhQqX6PA? z)Pyodv>~BY%c)(U74iG4F;dNhott6{PV9jW1#Z+}4cQ)olBS>A%Vf6&s+>mCsfA~R zKDJGu-q-qt-rf@{l2Y9_bfk<@!I9sAhf>CS!f9aP4lG!>T`DBL903S%QdCJQAV_aq3YR~3!UM59>;;jRDz2bSPOmn*dj5p=O=5~5Yx3HVIV!E^75P78kpF>&Ip z029vPKcW4c6Dp9ZA9`-Z$cd2C+sR%3P_!3AR97D@ZFxQwea2XBdFiD{nZ7Z ze;6oNv0!+>tgFcqxHt>jO+$m%+tZpRW&-GfZ4N)x|8rSMqk>g$3>Y~8dZF_T=M{RT zVJ7)ogzO9?MTEmJm)sxw0f%$p>YVxf7ESGzX@%lDyeF4D53M54M@`=f>Q#*0ksu%h z>%W|-{sND(N2K>#>*{FpJP!ROL@ZTCNV8Bd9I+U%Jq0jvi=x2Z0=3SNj1wl^wKS|f z-nA=QO|X0my*7EYT^E#f|Ab@UQ?WI|t+0k`K;c$t{LskP7w|vHAz6@n?~bQ{Px@Mf z8syR)<^D+t0AE^&29Kh|LVoD7H)v0w=;M(~`O71kU;uNi>Ya0~_y=fjK zefCa5?mrnGz*Fd-ZuEB=-Ey%wIRL^el+65R{rDGhP~@E9rbqI6A#e`Pl|$bd6l$QT z7qA)t0~^E$72yYOBrJh_JM|H#`ibTjt6PA%L_yHxpLlHxe$Hu+dV=@cBOe4ISVXhK z^jxVtQ^l3D>aJM93_+V0sT>G%ayabSSNeJoJFplY&{TpW@y!Z&yInk^0Ux>us-YD+ zFv0(m&0rQbY3yvPSr(+8j30|@hbF%o+I&s&Q$|`}r=ANp7kXa?O3cz7`D$rEPx$j~ zFfN%q`|SQct;4|&0pIu?s}&E5KV#-Vd;8~{OOKCgCs05ZKs%uB+mh@PUa^7c_I-?mTpz zTnEEoBdvM3=&g+mjIf3DO|X;^YjN4Jh6Z{W*%Aau<&90aNGl? zjIvg5Z8{i|;KW5kLf)+JnFg$2FPBw*a*eIo#~%0h1O9x@K@lz`Zr$K*7T_pvRz0}y}?uw z%T*5bJmFe%<8uFv|4?sc`y9wWgLn`39`vj#$5B-CN)+Hc^cv=ZXwt88IWl0_`nNdp zdTP&`1hP0#=0TM=41ufx?66!!c(S_l{@xCNMCDL-;NVZhR3Uyu{VK3tC~t#hvEg#< zxryryKg)QO29`?@z@uM+dk(S^B;L@!&VWd**kYYhz%P#zAGGF(6>9IvYXvQ0SXZ*4 z_R@!6PJps21gDGN)}wYaZ;FA&au+?AIOmBKM19{~7h#G~dubX>YW(B3Aka|&!C>h= zt|(vbK)^T98adNmH&o#MVdns1!$4T6Oj0@b7CjM?g#;K5*=7J{-wBbjQnr;cJizeo zw$B#ihr@6d($Qk?x2=~?`%DBw%Z=GTD})gMG6kX4o_c(<|4++!N+q0jh@AeMb_izS z$`)NNbvh{zI6>Y`zzF(>s|FR><2O0-%&2;&$=+bLS%t*mIU#R1q{_Wm3gXlZp!S~0 z4?8vO(CHBi%eyH-aLVk>j{PtZ! z0J9aNL2&Of47O-%jA3yvZ2du7K&t(uLD_fP=Vw}*FRi!Ya+OGkev|yndD;5+JAI^% z4qs-~x8TH0+0{RV{|kK0_CMm4A-={Fpo8BYN(eOx6zH4O4Q79=2n!Jmx;JpKhDKQk z=O^`>-uWnz=CUDXI4=}RfzG$zTO7p+VpZ{e1C$4ahJefyvYwyIz=eb^Z^vX= zT$T3JbfXwS?Kn%n$sklzzgse_?eJSh4!D_qxHA9l4Mgw7&E0Xbweb={xdU4orvVJ{ zO7FW`=aCIngtLI6ts%@~3DBYt0OINS_2Dz^U2D!eU3~Zul50jS4cho%oqlIAjM$Yq z`$F~}WKpyB{k2E8aT-b);GYqj3AJ5Ztp|GqEj%E`GB!ob=v#)Edlze_LL|G5_^;zG(P1R||@DGn(%P!pvmdf)-$s6+<8C_!28_$I%S}a?cyr z#Nm_PCs5MH%b_UZok@XwqO}>o+J*%_{y!#MhoL2-!q>Z~ekl~Nb82 zHIX#H2*Bn)WI~3t<0&)qtLEL(fBv|f#fkqicn~D34SAux3+&=8G$A%+TO0)<70^O3 z@IUSb#ev)*MG{08gPRWKWk^Da^K5CY5j5Jshd_G%wMIgi@9>)=vARy#Ko>lpaZUdw z2exfNaUf-zr~wfsB7EackERV%cXy-2N5f~#QJ5-i}9Mw z2=C83TVbGVTT*I&5jnJ0Efm;Zru;AN9F~kRM)Tg=j>jXZnxqSm_%1Bey@iDWW%IXqh6hCBsF3AwmjTeh1;C$@ z4sjj&R`GVzS*0Pk?v8<9W8_X9bNyy!d%BBWA?ri`3BW znnbOE!jtQXdB}%TIn!nfm@JO;75cuRZ0%1;<9wJ9)4uYv;)pHBpc5g?>^ikjDIL#a zjM(h08L6;#e;jxCHrn2WrSxQ=vCHTwT2-jV1NT&!1ZX&pJzifk1&YXpA#Tus z%%83XF$sY3a1BK9bZI%_my_QHl^KhUGL8Zq$5+5mY(agcU|A$^`Vf>P7!Ro*#s*;TrqT>Ef**>ymT<$ZK#$z?q80%YaExTEvnFeCx~=NH8+I2vw#B-cKyz zY4Ri$q8y4ys+N;EjlnO95#-5clqMbFItRSldS1x!?bZLjEDj0~vc&GC>E@f_Ep@ z@!@dQdfKD2#+|iB7v3p0mbiEM(GJl@lWA?SReDw*rOFk9D%7Q<##BfwH*&c*p$6KP z8P)0%^6u<-#Rm!}i>0WrqW)wY6o!jf)y@|KkJxDR2X08u*u+)gZlHTWB6!}=k5My_ zhPS5D0RA5?0^9mt+ht##^Owx!bK)vot4tMcoK=+w%22Ri6r|utVn03Tbu%-g3O~KW zWNiV;!3;eIK-%5k1ShAm=raI5whu!@c+Y+8KA>~wGvVF`$dX_RwkK_=L_x_NZFkGS zL@V))=)u~Sp@4UNj_7~m_fkViB4AtI(aN(=*H^oIIH!4)1y|F*m7W7#z@J$sp1Jj- zFhh46&K{H$jxn3g(7ZvcE8D&7kwaFb5${lIopPf8QmleG2O%9Fj*PD=0_TRZMV*d<1|xpN)S5!*s_7>8`n#0rMi) zNFqT(v#7mnl)ZuUdpVP!){~wQd{+OY2==QO$tD^A163|px%p1aD48FtwyKQ7NO-Ru zw5Ry&5yLKURIv9DXcjQtKo>}Wa8*@+ps>eRjfhmK1PPf3-@H*+S9(+PXc22K#MAIb z0nR`(kvj0yWq2ExHT{6TGg+<}n5A)WT0n5MEeH~yrn5$|6yPtk2jl}b{OW}EwFzwn zr~By8?BJjlQ}vV|L2pCFA;!~iei_&p#>t9R z7I?HQE+j;VV9QZn)xy`0fFiWlQ0OwWpWLX(C61PYt3B@srBei|{>^hMFX4C!H91aH znjYJ^oN5o2?H-)NT>rX!J*qwIc=k&P0Rq9jSw=phRF)hGHDinV-bTe_yc;tJ%rzQP3Sq->^RCi7(OlE z?$=->03>>OM|tL&UuS3R@aEP87RU%?YXN8Sk;YYvg#(FD@yna}2>{by`mx7fyER%q zflPtUa?5j?mtrVTb=YxEGi}^d@~-lgc3}CK*Y{e(Rt@2k*BinIP=-xkoHxH@3d(TdQ0Q<3{eMd6~OE_&0~s6U*Z15^88->IDkGBhbmqbYle3cf=C zQ4O6GR8VNl{`Z^~sEt5pfjSqaZfK%&f7-?QCk`1ZjmIG-oN3Tp@QSL3VBF%IrsSl> zfIN>spg5zjVXi?Jn#W11 zU77e(X5zBJ16g244wj~}Qm0nVN41PsF^N!R6c+&Rvvz%|a;|SfP~iYCa~4!u;Al}h zjn#ojavdbD2WHbIfVui)_h22Y4O&=S?AO^7pYopvMs8jpVPeAA z2Av>&6CDZVH_Q57epP9IFrkPvIaQx{DxX+})v9;X5 zBO4NayGjCxb8x0Z4um9z=3cn_>y@vEWWG?01x)9V;_FB zqtu-VlRVy`2X+QS+~FC18UbWi= z^X%8cgAmiE8bT&X5~PlnTkX@jUiT-l@U2-8O^Gi9r*xY6@{#xAi8j)#G+dgcJI%zOBaWS!sOO~HUM5{Sk(@C!C?cl?6NmvMzZ83!OQtq zlo39Fy<4DArheK~KDyUZSVb^gi@RmCDI-C4!GMw}0QzsXy|ZoWW9vrCp7S7Y?aQtNrJKgI7a@CWEt0!@$?>MI=CPd8@l zTFZ~MNzj?lKPA3$Ti^g@eX1T=HkefAGa(ogj-$sKz5ej89lJ*lg@x4uwD%ECDV<8A zNiv>6RMXyd5=L)BXsSEAfh!Giwp2t~g3vUk*4UBtdd7eB7<#L*inpbU;qs2^X&{91 zL8Tw98AR3KiO!0Xh#J5&s|CxolYk|Gf&9LS+wtlGZPsH60JY_^B)r;rdEgQe)fXm1a5%P;!Z_ou;{|QzC(C-{I9=K0sQ(t-m5eoHADyF>P zmts>zh~${c;-j>gOh}~jm^R17(j7mvv0-&Uz$t{A|K`>dJ$==DHj@dWuVJ2$`s|}g zbSG#M8_PQ-CcNz1&OrGRCbg110kR9_6V|F$ln| zTDy4zI`HluptAx#-AEekGl<(relogc5b*Ht$^MPpeCu&QmEhx?SIKkp=}5cxKNtDl z_hjS*sOq7Ut9?QTQ=%>#JDCxo-$K&bZcoBussgp?*{-7Ij2 zlYV5jEcc%nc&`Nv2njO^9|3ORIoaSU77dmy8cul(5P>cKJ*$dbtsTc;cHnM1n<8-A zPCl_ytnnvDXWy)wA#*?<*8H43UpT6~X|;i6gcr;mJ3l>kVyb$5l~z537cqKWpj*N8NbpTia78_*jkqgp9da8;Fiweh+sD7rmYa{khA)z0WkSu+2Kb6+nZ zWL*W1&N3%Jb<|ss4to}$*`)$UAs^z0p?3mk%3Y>V-PCxQ|EF}Oz9w_J(!?y8e@^Yo zh!Hmtk{6(0>LZ1E3l81q4J02Uq=3T<6uRCSn8G#ew*Ozh<)D+j7Uu_cgaya{@bXAI zg)genmZAsu@+&*r?b0gB3|gVHd97PbE6>_FJ2P^arRO#5CC@AVm8fCj@4D-CZGo(< zmtQb^R~S|~WSaZSHdam_ITnu9C7Q4K=j!X)9(Pmyt0E5Mk1Z(?w8#cCs$8Ia9u3%H&aLY=v_ zf0W}zbYv&cbiymkMn`$(>M#f%I$1vUUs@#MnWkFh8y}ebPIF@(@4!urrDxh^wfH5~ zTCaf9*Y?SJ?j9}_q@^e4TGbF^?lTL5vRIIDBSVnkH2Mz!GG6}kzy|c%wLG;ws6#*O z*YhK@4@7EunIMkP)+y^n;sD0kRyjQfVeRlBB<5cVV4C47CXFJD2E@;FS4djc&3=qH z-X%%Wtfbue=yc%r(=I?y^C2}1$>71rY#A_er)ulpgk+#SNQqe&kinJWrc>x7B)?^N zkR)4~ZatVwt#@~K6gI;1te26ZHNmS6d6Wn*yUk=(x9sfn!0%A|nUNP_dW9Ka%8}of zGZ=I)lQLfyEk3u9dIVSfcc||bN4i`A(X&s%5;A0^>n5W@l5cK=u0DU5H^gUyXJVwK zO{Cv1tJ8NctaL85hxFxJ%b8S5AKt4nfAw~qYGgNt`Bic=d#A?qFBup0YLCxmGaG|) zD9Pcxv@`FSH>M4&sPabpte=W4O`%6w`-mS^E_bSpwu{7Gb{ znoQNF!~GmBN_>!=npKszjM(zD}eg;t8pRXsh`c)GhI@><(3CiiJSlOTg1}k z;5gonyGHE|jw+4&^AiH9Gumn_&G6}jnM~QqK{uCmW|RSW+t%?I*T--D+)}OtzLN^R z^ra`Z^!kKC&hM~a;p(0uFDF3)Vqm^|VgrH1bt*We!yLAjL?WwNs}fYm&A0kUZfdge zdAlBT?m~aZ_97{wllpEV@e7K>F;5|KFSyR{#BMUiBQ+Z@zc1|c4Zldp&NnE@QP3^co3 zA(_yK;moQwLeY$L0|sb3UIZ%Qth8N-^l0B-6Q|mH?t|0W{SDmpA@YUJpnBa`UVuPD z#BxJ795f%zAl8D@0&;oJ=}FE@RG^3Czx{JVfq}H)d9}AdfmnADV3}3(NFF_!QpUS1 zSZ|;Y+QPiSF`Mk0f-aA3C=Hoh%fgS^jadMQYJOi52v~do5(?INmiLyT|mVlh%i?Ee=T1aZ>ZjOkE$i_O94nx6Ry2RR9>|=X5l2SMboPoZ~L5pFH0G z*PhyCmH_YM-t4>Ss}|7mp=crZ~$#jXW z`T7IV(XduptGJ4yLaVjXH+DVMcGVf?{mFmf{qu9rPtWK5yg#qk>-~BOhF{?>sn<3~ z232}s@Me1pUCvwdc8a$3Mv|$%JYtU{)78c!5&TI;G1$6SJ^9j^ zKefqM*;#t0;Rg`^Cy&|iP()5o4J+MWF@5l{COx95EXhyOM!T!Nm1GJI$nj$AgX9e!Wv?Dw0$g(x zpEZTVc!hZ)#Jqx%EM2GMjrW;48sh>eCjyj;+o>b`NrRg2|C>4BsZRU;>EYy81u)yJ zUT_AqWiv5mCR3;{)%P%0ft?=)IrAQHo2Upsicq1_G8l!w9?x02Hqirg1solu?C8V; zyL7O)h>T{|X{3M`7&|%2YEl0Voy30cCG?ePkO~nNDKP6+n9;3(3%zKTK(2Au7@Xsa zM70ZyGmV%Py2_vscP8k-^t(N*$k`_vr%9URiL;SV-d2T=n!^*^G%#4V+5=~>I6prV z%7Ljx){TlowezicOwq$*gsaia|0HCQb%fm)T?WU91IpM% zV`xF~^^B5+`1Z?Vn`0i8G`u9aasN-rJD--2?x?+q1h$FfwFbtQ`pqmVj`U`I>t_i>%%7s5eek~rx zr02=!^%+2{VQ?pSZo;x+VCjbcWTE|ZR=4WDky#A=xht&}&s^K=a;N(!gKxTwV^KY_ z5^Tss-Pv+|YWYwPh#|AF?8CP06Wo9OjUn({?Z`mN-_P9>YKN^V* wmAD)AGN`2hfS{_I3#bJuQ6zc^)PMNMG^SmtV$|Kz9(04^a$?z!SxFcE0@eQRXaE2J literal 0 HcmV?d00001 diff --git a/docs/img/globalsort-nonoptimized.png b/docs/img/globalsort-nonoptimized.png new file mode 100644 index 0000000000000000000000000000000000000000..d91a878e661139878ae060ca623bf5c7d49fc92c GIT binary patch literal 14336 zcmdVBWmHss`}a+E4T20EgA!6BlF~5@-Q7b82vS3L4ltw$D4@hBAtBw}QqtWh(u#!i zJ)Wo6e_i+e^10Wup7p$$#oqk(-oHBH``O2i)zVNRC88t3z`!6?hAQY_U|@sLf5#Kx zp?^KRAd46nBK^t=vUIBs+`?A9|i4l$PutcQlNhTYaC~Ie-9b~TF5`=(Enh=>iye*o$deTWU+x+3xv2Z zrL^&x6Jzn)rE&LkGKoi-P<$WI0Or&lNwS+I_~u(xw1 zyL~z|I3;MRRsMgjqykaDpkNQ9r~wX+4!)9CdTe*M?ls?dPLTj97=QMJF2P%OJ{qOB z(;zTpS1-KGb@wrtjaQ+$);RUL^1z~)-gD*wgo0IhgwHe?=K-XO-z+U{ibQ!xl+|JM ztDZ_XHsqg?Bl!1I0HRZOQ_q;oB+jP!F4Tu?iyZ*$MXX%$xjh7IY_E(F9h`sLN^?TE z`&=ED{Z5|Gv|xK>AE&uhCV+-^=tKp>IFR3HeVO=KlwWC9yy;k1gnH2ts;)EMDZ# zG?%cUXVT%739{6p-58IAkxRYZ`B6oix#?Z+ z-5QjoRaz38k`fzRb@p95pgLxx638BH0zi1yvMMU<_rx9 z2tOLA?q@fi*~6?ma#@d$orMlYth*&R$fd78UGEyUr!F`91V|X7I|tQgH-p*A9xGmz zFK1Q@&e~*^xW3XsmYP*}s{48!J=N+{%gnzsSsWj0bNOU2-7_HB9iy5w{qbOgNyFyX zw~?cHXs1y|NNfgP?H|}-Q>??rGSRgiP)5~ zr>V<0{R;`Pu6vcw&694=f^cl6wUH+Sc3?{0NGZhY}VV75C|zUrGz z-eo7Taj%8t51Y0zRrynbuE+t9Cp~C>^`E-d(K%>XkIMa8frPWObCDYC*cr)1o`Aq#CMt$(@v&FWT zExI4;T8+o977bj?h72pdDc3W#ZmsvfodWj=G5t)HI`0~)7)-X-e*DnCMdEC$+M|E@ zdAZPe6iK4Rd!Y_YMg*YM5#@>ciB`{GhQ z%U8LH)5Vj8=6mi8&yxZoJ$wDGH&5%?{d{&>KS@1e4*s1dC6CQGR#4%9K)cFf?xkno z$KE%7g9CiYch7=5re}h0X&b%DjBckoxk!R%FLQg?Es{Mm;cOL{b{mkTZ4R(5<1>ef_Y`qTW$82(@KQY~WS#9G3K1u@m`7AuS zP(NCE{A?2vD;;PQ{d@r>*Y#;@O_mx*{XTE%`xr81&zPc)XU>ZNo5J^Qt^AY>j?}p#*py|}VbS3CY^*ZiVxs*dy z)sqLM7xiT)3_s|-FNbLj-~CG5>j_?y;5Vk@AimG1gNe(r90vgLw;eZ~d|x&^+h67| zrL6zlqpZ7hd0{iMa4If5v+3aAFl2daAfH|Vb*r6r`SE_~{7V6)AiSwm&FDSZtssBP zJ~)~JB>HY_H;6UXw0=j7ltE&uw#|6x=<7U<<@nE~2FL1^ENg8LZ-5&SA(HSgLj;Tk zxR_}^8nV%pE!C|iCHg^a31^EH_;MZ%I$W>j}P8K#F}yLEy z`#gVbzn#-N{D8tiUf!IpXAPU8h78*}FD3{ZTZgP2pB$i=&OZXT8*!aqX&^HZ_)ZG@ zd!^B*A1Z>(1@4ys=aWtvqH#QDQIX^`p^`{rBmGWZ_rwj0MVI=v2u**vR5CmzI6a|0Q z-GI4QUFEhe7ZmNLP!4bCXywr{rneGMbY+E7*+WJ>si8HqnbdtB1%1 ze>2MP5VtJgo{|`~lIr8)tnmm{b;v?)Wq*?uj~;j;;J~D@)|Isf z6UUjZS_$9pZmk;k-~uh@vWz>@R-a7idkA#;0)OfkJfthj1V1u({5Wymuz6h-$#NXA zS4YHz7?NHM&iD|634&~FWHetgh-a6$T#=)`SDhLgw+FXM$rnjhNz4BeghAP`t>kXU zkd`|@#Tl`x$$b(>Be!EfY(v>EYr}vHMPnT!v;1*#=MY5rGDrE&(P*O7+-NXapBw-* zMkY}*1YNIk{WaLLxp13xq$`oAo31Yo4nIO~a6>-t+7KupuPY zKR<$COr-7x{j8ml&y|kEWqas-9Saa$LxkrFcRS5l?(a|RgeCxGn{@p;fa|_Ad|OWD zM2WS}26blr)q+S^ZKat-?^bJ_oabNaB8?`i_OqX6@VYGc=#58vhJ3ro5IWuFDw>t7$4t<7ce}xA{)yl>V6R&R4gsZUAjqQDvb=tB8w9R2%>`(~+W3lCYejuq>CG zk}?R^4MICjr{m*eGndVWY9|yRJd|{vJxp4ehz*yBGzrVQhE_!*~@O_+Heiy|J=`aO@#j|GC66o#xoJx>TwJ8&_2)J;W zos|9L!iL+-cr;gNHJ_!oqJ|tb$a3^u#g5HyqJYoy)lHgKaA31|BEC|UYy_SKbcLKsS^LaNGUn@=oxqm&mTbK!o}3gc6sX%mX6$fdh4 ztIItEmp1pg0*x85I#KQk43d4*{TLh!{U!~*u?!mOlQ#5g&;kMm-3>N#B0*19x==dO z;Tk)FLSUyK%PFBQ_U6)lphF+`mQ&hDM3dezbldN zM3sBHs~UU{lE8v81aD~j1hQs+@we`pK*l9FD~CTsz3wZmc6=Q`R|<5TDoT-(KUk>! zbxHD1jvb56v1_U?cDDkm4Wq>hnB8u*kF-OM3C;Vc+Lgwr2fo8cH@pP1?vub75N3f- zZW}aKi2)H(nWWBl#m*ib)^q$iZNeyv6iKv~J2Og3D_73)^m@pzMvZg~9k=fpu(`>; zd(l7DYM;(G`~iNZlQLQpwV5A`gXO$_S{xEltMUk9^*fif4Fvmeb%P29Ij)qnvX?Fw z0RSIG3H6iHl#ph;JZW`98AFzMq!UNTC23fs+Dh$E8F{)h3H`(N?d6_ioNbxC+5PbL*1fO81R&Hw(tb*?uT+ELJKMeWZ%^WD6I%z*V zRlgt7`ilt;3phJ0)F@+AyNa)26)iifKhE5}FGS|HKp!+|20_woK7@?El`Pbqg#}FM zNu2J-n@C&JNMdM+v@srcfRgqqTY?;L6t{{Z1Xc$++Df&v+|WMAZarK50Ktd=BWWjU z1)?>j&r@smzRVZgFXTZx`{tIDq*)8_6sKL19T^L) zK@RL%;BdY6cef{nRgRMn-~DSq`sfYv6Mv%Q3qR}Es?w)s;sN(bQpg%$W0w3(F}|5q z?@ykBC>VKdmfO$6xWcv%Kfj9i>rM5ng@zQ2l*V9$TQ!=jEinnP(odETe9@n8V#>Fh zUr}w6a(!3r+IyJ;KqWDbaisyAtG-H`Hqfl%JE663n%&G!kUTZ}R{%)loFR5?5WxF)P9+o_Y&AoTwaWY z@V#iec*v+sN%2jrSpQ=>MA4wsX#MY1fVh;n>MRRn2Mcs}>PF>$f7ywVVYc<}OzXEV3GY1|7=A4%`IbZKkDo2!{v+A-<&bIpCDp)6Q#hV3z^0Brq|Vkc zJE+c;P<{XcO#5)B%VFw`5wpFT!XP}2RmjtcY_uk?`r-yTwPL(Z5r&fRVwWoiFq zMj={Az3iJRf^@7rBPHqp>TcaF>ve$ExmHV&8nE!z3j71O2w1o5 z`$Co(_;^5Iv}H! zf~5t$q7-zAF8?FSgY=CWx5Jds&$`{N@cF!{Da|p`?(*Puf6`)eOcXv-@bES~zy?iD zE{0Z}=CaK97ehJ@=#b0kc=P^D>D}k(gslCne^wF?J+g4l81rX1PIJN(-0?ua##0<+ ziR9wChdT(QaT+Iopoj*Oql8=?Gf%sX52$BgeRJFP+wO85(^s3_&+LvgjN2yV2%AWH z1u+)BaNI?!#i}AT7;0Q(#O_zRLTy5p@X*wb;MdpNmgzCKw~@9E4vQj$Z~D|tp=g>_ z{&n7Sd+=@R6_4cgy&_idNTW`iWQCoMUaj;encJHT$8oV1Tf5A2iF%@@!uewv#KU)zLl>&s;J8;+B|); zX~{z$7pH3bJv-spp6ZzHY$lkRGx^yO*KJZz7 zlq7kY{f8wG5vklk&i3aYF$b9#9=2=;7rU>@AKP{JdGJhzc=cxhF@2VT+L^iqLOR*} zZbp2z$^)3b=zT=zfjqRd{bqagXl|vR74k?vT5(5AhQVm^gBy>P=A_C>tCG}3OWPj? z(R5V5612)&TOt9&LCZ8b3*QT&*~fjsUy`kldj!XZERR~0dWjOP?a)f_ST;xVB>Sl8 zT((&M%0_vqX{45E^Qr&7s3V$tEnTgF{97$WE+M1aTMd_6+uN!~Z8xVD%jo3Ozl9l( z%saNGyV%ixWsW~zSClMY8!gP!U_;f?HU_0F*Cy(%%)sVo^*4jXx`k$YV>v_V>8tfK za9?48;4ERor6FAO24OeiZUy?WlxTSzE9{PN9UIXQpryO$vYWlgzH){Q*>;G_F#w*6 z-|zb6lwcWlBX>l!=GCyC!<9+cQ4Kf*hSTx-1ppm<@jXbgIT$<%l20bZ z9`mv)>$ZhcgGLzSa9{pajT3 zDlb>6`$MIQ_0EbI)%G1{^B}jjhG2r!mTS3IAI*ATL02Igfr8d=wq%zGYS4FlCHV3s zX{!5vW~oKB0aL)S@s4zdSYm(DJ1zW1*9iV;vDH?Gq#0F~<=ur^=Na=q;@Q?NXL>U! zmZ@T>IW-kdD+dzdRX-=k{An^$Z@wsl=x~hIC39ff4&FU+TqYY09>p!3tBV6<(>pz7 zu{}FntN^@5dx&Dr&br(d7|!-@$7Q^7JJBl&a>NXSQd0ig zh-FbY_ALIF=wHE`Wj^|WvTo(Z%@&iN(|Oajq*&&4b>TH zyr;+6;VzQ>xsB6jnw}?Kq_AOOKkWg8?pi=;>>iE>GNk4;%Sc5$jr`*GhTL9)Gt5e> zbXkuDt^;gYyzAH$@+Qrf_d)lY(=kUwj*oLL>0z5&V&~y53SwFq zt~;%we#dFo7iZ13>ul#&4!ENpXk2^}XoaZh$h#?Bq1g?T-@-uPAz1e41E#Fk7=GF_ z)%-4hCEnR*jXy^gRO)DA01mg=kf<=@N^n~jsWWb|ea(l@{SPlG$Ns}B6V$Ah|45O5 zpBsvjN(zutmad=IsujCO*j!vqq>-VRSBt3)`sEhz+kT?iAGa~g%`*|&HupNdjkgIIX}$j}hY>?ex12Afj#1atG%+w5HP#-6qQ2ja^y zLX_bFkx;FxL~}EjKZylGhCO{{hqL={u_(iu=Q1zmaY^YP%Vji+5<=zqEjko5@h^;+}y8 z(CA&Ud}P`K1vJ)!x`+E=J8HQ(86K#?YgvVeH6+;Z-0IpH8x^Yf#Tf%pRGr0;n}}&- z=+7uLY>~#}Qsmsivtz`@BHSh^T>hLtf-gr2@?OVI36rv+c`4U~CFdGM%oktkXzo7lAvkl!h2aQh>f?kF5Lx0> z@vd0Nn~^YyFSV)2u8T5Et7fPaMnRkmZuseUvphs|q4AHa%{#*ssc?j_4Fs|JXedA< zCczbsh5}PLEQ&*96OnGi#+9CC+mTaj*#-J&T+*dt48s`}?jgmQd|k?8xE{a>WB0$t znT5015pFzmvfc3{Kw8Oo^@wXgtXR2mSX3@afv64QgQD|KkW4pHsBtw6XJFJ^x!nJC zS_J;`W$Egt@#%g9;A*P7Ap?!oym)Ppf4ED(!kv%P^h}LFD3o|%|D16%szwSOjZlxZ ziqQbGCRO8Ao81Y=pILWX(=J>~Xeb*M*`En_!FD32UP0YCmblV+YWuldBjnqdzp_5pZM-H1*FUliZmL021nNbGDiqZVw@Jlj8 zZP7)R`JzeKfE@QLfDoR73i@enY#`Npx;2cwcey3801Stp%?2(9BScPAKOjr=etv#$ zyd&vwQ7TdM6e%lGMUU6errt3Zim|VT0@o zfe>^u4V^n8EDI5BMT4J`?vZE%ocbE=z56UrjTny0fF}?&97<3!j;!Kj# zkE63muuO+58ie=%{`0Pj?jFi_o$g>3Q(ltf4`HG$(u4JILqAipg~b7e4{`e8H>Y$2 z*x5Tt{-a;I9A4U`d=k`8d0yB0OHl`upK(}dXQq50E*nJaQ=3g;fb9eU_tdXa^LmRA(7_PV z%(}!M!~ZOUhW?P`xy2S^mJTk+GQr9Oc~-MFEJ%Zul>}= z&t9ff!LF>S<0vv3%8UHoYnbR$fjBg>^^~p$9W0ZXIr<0UUVVHrO#mlLlDJJXRVWzc zc;~!~u~O4*3;_$-mtWTD)0@ArpX|z*)UFP#-*EfAP62Wmq(Ltem0kfqgAG-c&;F9W zxcAP7#2-t3#5JE5vw)b;jLSCo)3Vl3qbf8ZdTY&$ev>>C{tFGLg>Fb@ljcjc+?Yy$ zm__{p+s|ZFfPIXo^c22D%$-QnOd?vV=k5|q@iic&DVA#%h#D2aZ0G>h#6AnR7BrEz zP5=*&D%qxKqv7kL6Hx@2HIJIVAqxyAh&k|tFd@=<$-OMIl6VZM?AZO>L_mUGlx$+q zZRzJd6^;}ZblhZ9R_Gl_`2C$#<}-D+{;*sW6axpNZMpJknU_>>a!!CEh={YHPekO0 z3$Yq`e9}F-HNAp41~M=XnUo2E`#L!~Gd}2BL$+^Gkk5(q#N<>ZhH0knE(U~tsTY92< zZyn-CPdP8#mDP;}sul7Ja;0pN6`?1Hz*)jZ#KQ-bPJsf$Yj)CjL}wwHjT)0*hZ_=% zv%kmLhtcywzV++xE0L2@u)9sI&*dj#Do(n=w56?k168p!ns#FJ%7-#?#;{~s@$)e< zaftb$$;0@;w;EgQx#g`9wm+$bgTfd^L z$mp?r1|T?PnNQwR5Ta>V9=3KECq53|uFN&gMM3{|;|0CZ2BwVMFe<2YC zCEza?Lf21){r~Y~*KqAMmRQhP3&V)S1Es_q)P(?&ypM2G@t={Fa8QA)aT0~H_m@-+ zQ4Oe`wl_u{zpb)a)HtF4)!@bDy4A~$PG9S#zw;9TV@I-ZcOjr|RxJXq=qW=-^GjJU zB_!EHAv|UiT`QHs>bNRPD`=P4+PH)+4N(2aB)>nB>ceY!#w?!SZdk6FrcYl#?;n{V zlvSs?S`<6s)06br?+nJp!!vaj;kfhI>V)dTddiko*g86UWOV#&v#`pxC#l+YYRicB zvFn@tg#fKZ_m!70njK;6ezlaKmDi;=h3$_vq%7Jz^z#cxUfzN@l;5r2vsLcGn{7&1OhwffkREuZKScQ9 z2C{E9tY{*KzH?Q>nb$WU2{-En{kB^uqZ=2-O}B~4Re}KB#z6@`h(CE23-QM6oFOhw zFBF)k%Sgu{B1aHQB4UDM5Km?GOlP0VdK8J=ESz#wG-qR&i4~+N0hyv<8RJz_+N48u z?dk&WQ@v*x1OQOF2fszCaBH-(vU03=w_9N6`yoTr_LrX`Ei2=TtqbWjW2}j=rG?L` zEakIDYFr$RuIRGM<_=bt0FqdPJm1?c#s0Y@Peiw~zh-~m@T+BfGH2-}=M(Va zm}%YR<;v%=1iFBaK*Dq2rdT;am8ZvPHmd@<(hSq9L_Xr5#p}#^X@{_EHjdWJ%+n6% zs?HG2J_>YMt&udwQ_|G$XT&RHWDxU~1L5DZ=Slu8lqK0ikWcLEN!m|2nm@hMyJ%9Q zp3^*BnKW1>>DNANQKgg9Y963lqZurlr}mDS!v^*ZI@R--DZO5O6sC4RnMh& zVDJ(h+-*JWwI^PHuFiBZE_!uUH_BTt>0i7LAfYX6cUSM*!KDzsBKxo-)D6gLe*WqZ z`(}pYF!eQY#-5SPj}F|P+O=%3D{x3IhK1<RwQV(te}U#Niv%K zwBr1}c4XdW;hrdlr=>vfOn~p&5AE2u;+hqWdmUi^7Zj{jD@w>gf&R8nnhZz7@Xna4 z_{#o0aavU$6N`AHb6+ipD5jmp@;rDv4qkqR5v`#4(uPB$xd#!1EJHC% z7i-gAA6Vbqc5D31PJL7E;!EakIWkw^VzK1Jh%EtHh0~)BI;HsA4yk+CETl@8ThNt; zk7WwQpNOLEkdn4V%$P+;QWq~FL7M}chCUPAUz^v#Y}3r3>dD5eGz1E;8q+aaG(Pn5`7=thFS#-U=p z_w$d5zhw)%V?>UY@snP4+7zl3-hza3h-EU>;n^<%kC*Wh^tICuMWsa--1adR>Mdvl z-8O!B({out55W6NFE`wo!qyHf)-5tccG~*-#lMM$D1`su&z_Q;#U&^&o+K||nAO($ z0K^1)rZ=rDWZhC^7h5W{!?emS8&>~ zH`B5GQtrqIM_N~#h`nwwn9OZAfwT9Q*A)0+r_RirO*X5X+CJtv<9XL?E59ldV-ILr zS+_0TM`we*9#Prli*?QTrn28}(-#i}2@kUP++;`vdW9oCo!99OicB6@8yUoMOS{i! zxo4uW5(4ehd{@!_Q`kRyQ$#0gt!C4OEBo~k<0$h=DwgPn&NzS|CJ6(bs0_v^!!91i z=)D*1>rXnrBY0K z2teVaBw8`889#>E{``C@&WD)Et7E15Oc#)SPob1H;M%;0&tm7-N`?|rMiitI`Q(X$ zA`6wk*})}=4^eY>T6?Nk>jc5(JqpFDxs31#+%9^;5J#Aspr5W%q zyq>7oPZ*+S&cZJ;&_TfNaAM*~nakjn7fEuet-_D zI)t)6p3C8pH|cYS&m>^SnuUn!Lfv=n3285~NKjDAtcb~!#V9+6zdTY$cCbFtF#Cko z2oDB&@hPMSBKor&Egek|qa7)q#kVhbKUV5)`x5{#VrEcpWZhWxiMyQo-(s71VzFqf zecw0;K&N1%g~jri4Yy^O3SRqjN7O_=Sy|N7!zzBt>Jh_*@Q{T0q;CERf^lE@$WjOk zC;csx+3a*eeh9B304+i>6c`_*b)u3#L`|`V{_GXSMneq4UGiNXq)ohMctL^#@j0zc zQqg@jEU2Pf4<5^y}1y3G5UG2nbPF8H{nVgujJ9{ge zF9szveMbdGy}>X>n@0^}6#f`&#M+by6^W1bXS~-pY?Rn&I%V|^?I%CWfOV8}|0krn zRSDQo<>U9QnPG`$LeE1zBk9O^_28m#UwUFdc(EZlifKAK5S7@G5-ymu8=p8uC&>7SWSpr*@or{4*?2L*Gu!XD=C)60 z6X-UKku6Dex^Zds;SgQd748y}T?^H1`9jloR!9?w>D-mJnEbc$H`ft|zzaMW!$ERr8Qx8hRIe$18+{?+62)X`X75hGVZ5+pv#_4TZ#6~ZN z$GeNQb`BG+S0$eShiH%bCubDF7coQbOX{*c(p5cZzq=<^P-L6R=YElG1>@UFnTD;STtB z4<<@@Ac=NNv(N$ZhypV~8Q>=gcBdMMDxdL()@sVn8RkL%`85Us+?OgKEDka1Lv^r^~*b%^Y>g4f()9Gt}Bm^wmK*j#g5iq$eQvHy;VVG~u{U|)){BYe$MDIc zt*jS|&#t1D%&4D0kNhVq+wl_K3|;q*RxHr1l<6A6w)v7-L4DA&q`jfVTIrXEL3q$Z z#qAhBBv97Ls9m;uYnfa+L;PLHgMxexe}eZ~gtHC*NSEZSc=Tv2vh+B8Nchs=N~&9H z^OUb^s*P@a3b9_dx9cU#gkJxQk-M;+f=$qeW~fY(Xe(l8G-$qeAiK45()=KV<_c(p=H>MeB}r(8q!?nB$Ns7R8fbSEbWjwi)1Eg|g08N0(;&|1 zQTFcxzZP?7t1DYnk`^j93SOvW$)$TGLy)1HKvl{dt5S9; zn_0CzkjTS>4>}rUR$upFwf1Gv_hnP6TPb*_|mo zM3;}(M5#3NCH&c|kxOQJD&TOobHfmxDQFBly^P$UAs(QhSjr=@h78{KCIEZ{a&g~J zDD_fzJ0b^!OL?AOluHsuyNOM|6Mn`X5tHQYEI3a7u~*F}7}!sM7l2}u1nD}gmYix! z)}w4u&%hkvOJ37hcGtzc8wzXi)k^kvsR+Z#7mH3#hN1-jwagrlVs(Np<>th zrh=NoFC%5V($bb^htg}0r^4oX5w!yn?YT0$LCbn3@b4vZeN&l%vvt)mK=2iq{b1$e z)l+nYo(;lQFw%lDlXVIGEBm&-<)c53=o!|LwZ^^KhoO35K493c9E(CJ3^iBz2=$`Y z!~liW@48*lbnwu&>#LsTYhhJ~$z%NvGO4?&SW9rK$M;XFT1LgIpI*lRX&EIeYa${> zVi&zkr(%2;MgCJ#@%8;gEa2X;TD1gcL=TtZb5>(Q@8~yEO>&uThQPRUCH2c{*)9z08*ChrodpUdijaSFZpM07&S0J2ye0?ULIr}bF zd=hcAT%`!w`Qh`fjY7l&_6#Fp7Oq#XSuJ^e8b^=EY;e8($Is{6xGl#O z%8#6G+wSJ;(T@BVvJ;B$v*J&`R;S%jQ(YT0yXtaby;sn_LKkom-HqW{BKyY{bJ7Ce zE6=Qcf)gXp?#GJlA|ripN}CguX4k94(vEFI^V z192Sln|w+V_~zGws@(SL`>OTZf9(hjghJoAL>H3(YvS*QsG*_3|M_-^86Sv6@~mM`Y#xe<4t^FOPR8s z9PR6sIO%xY9`#t^Clm_nP|5P%XS(g<@S_&;IKF79{`+v^1yHk@6^S|)j-n&>i}{_Z zCNLnfkVV_)Q$)=2{r4a7n^Vt~!#~HC_zc<>L;89UQDlp^hyOf<$>ZJSc9zd^d>|rf z*b)AYLF=o57|N6octiz7aJ|i>a!1KUa!V?hNQ6(iqw+DcWsdX0Hz6aV;y6>U!RB^J z=T9o=QjwTC{g#OT#>ur>Es~IWQm6Fo=uj*E`GCA<^K&6&uGu0y#Z%U}uG?0mLw`2E zeO?Bn4Gfij`!?+1VOK0A`|;2+AuWx#Mg#QT03%eISYnj{6^l$2H!}O_qPW&_oZ@ea zSj1R`^j&F4iw>V?-met$7x@gaq6sqM@>_rOquAX?hL;4<&BQ?vWM!)il46D;CCZJOIEltS3 z8(;BNKOJU1`(ds!NUI{uw$0+lB;;{)R^k7=>6}&1Qr>?4B=f~cDuzOczF~hV5tY(c z{c^g%OQFU6C|&1ko#l9`T?(5qwUU0l`jN@9&()O;Vbl)W;`f2=p|6qCPW(0#Bj+&x zl1}7m5PI#W?M@SIkYV4-srej@dKCDZCA?6gpdJ@~u3ftRVSeTnI+S%u$YGQe7X5W7 z(cf43*#Cyp>%Kkq-CdOK7YZp7e!F$z6!B73(6!(tNy(|#T0FxPI8)Gfp^oCd`>OdZ zNSkb6b>&gj{gAZ-IODS&$m0}GR5Z8J93Fgw!{7>Wu_LUMX!ZliSY@!bY#EqkjY8o|i?8)jbI0tA@$jJPqINA4}Y3 zQylZDU^_>mJgl~2(lGmK6AZEREFbWK+$)~%`Mf5q)9!EV2_ncq#;-1S@NY7L?IH^y zC5D}!UcRssy8f+e$y*^KRL?&fYW@=~QiWbI?s;!P>#jBTIZ6KS8X6*95QJ+=DA?tn#ph% zNGt5{B$3AJ6Z2BivANR9^z=^uM83Fc*#dTItzXYuV&3-|I&CHuvc`q|{jJXcLiX>J zPY*e)MhQE*+J)YjUKfL%vO_nYYI*F^oM()^kCd`GLm*wJAGIz?=OIpsf<>Z_FNF^} z9p=DBevfRtS++A(8ng7A)^UZXO^jWsYu>kbK%5Xtzq2&qb0`a!%M4kXh-DQCWdyeY zVoqpZ9Y+UE`D{h9MzONo^geIOq-x{yX;vsb2d6dSXW0;qB{Gkc)885}9^YUH%OZe*8krdRsU4@BOiS0eH zk3yLk2ypicAE9g@Bl)ouCHB)mi#ZiW(^G7_^MZ$0O{-^8>KzXi`96i9ax_>HlswL4;N~_*6heN)qfKe19#n)~wpO;VlSOKRE5^pOX-tcxUj;lY#*lbg z9)i7Aa4@T*h^VWh$$s71>wq7ibgWf7+g@LCV^0i%`g)a_r-HFXOERFP8pU^@&{+cs zB$b9mkHf=;_Fp)`R-wolI`x4rl%GvXRXsOi{ir2)9AnlG^^pUBKugm-E zoBq@(tBUb2-LTW{+J=Lf@)JEU_&U@L9Oi^L<707kIKNNPdcES~drS?(8C&)YB&N78 z$`hZDQ$KD)R%bbkrAf5OoB;?GK9*7X_+VOmG55-0kWN7!`3xU6J${py@Zd6G9xyiq zS+rARL5N^~H}h#P3iK6UV3sfAjJT~%E=`_@yfH+i<(I=BFu zXY1Yi@Xf{YR2l0Gg-M|@1t{i=>Uf9i`(us(sDgFoCvWI9gn#>C;#IXpR}AFlvf95g z16p6tLs{&&Su{G7w$$hw{xg{4Cq|@jimErv0CFv7$?w)T==KNwse;e}1=Q3MCde^@ zGyoQz$}84MC{&RSgRp>J>!~$;*QbG`q2Tx$Q+V~B?uF`ZX+C_qidZ$3h|l%89*SvT zeTDc`g`ti#F1?am^5J3rjtvI_IzZ~3qeZ49F0aPZTPzoy;a?-}J@0L_Wyb*(Oareq z_hdtKn=424x(qwzSh8VVtpJMf>#hI(>Z-wEpP0v9&O|}Qt4iZ)8gju}Qyp-KO@0k6 ztKaRgF>|qF-)90*W$KBzHAMa}3M>G{_f@7HInO&|j0?D^z!s($GAiy<&|M3_az2QwD5=*8@{cE4aL7QDB?o<$y)ua;w`u zg#>()7d9>|2t9;;q5*_>feS`tJMLS%C)X<|F=FC&q0<-!k>dt#k?8er9z*Ag8)ZZK z?#-81aT_4V{3q^a0QK6D$B=q32H@L+awJNsZkHX~QkYVf=wflKBdfaU&%W&>A>5h( zan8v?(ahJ>MKFW0IAX+)nF)q|KW`$fA^OTCngjWPu-zX`nq|==6LW?OBq<=u?MILv znxU~$f|-Of=z<;uEi&x*I6tWjh!l@$F?3W0VM~d|EnMuPQYM${RC~|Rngym~pMyTjV7>g4M%Ot< zR^Zs5)bOyLmc*%N`^V~u^Oln2GU7tL3X7@);$y0BPZr6ZF*u`{3=GauM*JvR(`DHS z@weIiCpO@@jbp+k?4~sLg6!-Vm$C#r;>WN4gooL5JZ;Pr_yXLbKFVAesCw)2{DcdL z+zrD_e9vAy-n;+M{&=-G#H1)I``Zaa%{)*K4(0?#g#JXI0-m%F^R-$6g{hTt!Djp7 zP>IO`T?MVZ4-=I#9K4cAVs}?6nhj`y`}jP{=zsv?FI4Y49DWCXJhvmSb}LZ|nDpEG zZCbtkgeC(HD4n$Y-hM5zb?uGx61YAEhGlv%Dz^6Z58LYY9hZLtfR37#WRyh}7qx)! zQr2S}S^y}a<_c)=LkJ_zf7-rMR6kjcVHba~cWD)CX8%B0s@0`ee44#{hLjkP4&AtJ z3Pt`rgq!g0TbALoee)ICToRaOfC%~@ z8QuqLfZM&@U=2LZv`32>ySML*;Me}(gG{#H@#csbJq=oGRIbj<$4A1*?tUoB~sMn(x9IxEUrgEqyMeKMSOKPg;U3LcpgLQ3INkDy! z0sTCcvU1QD8lLj8=_n&ag-+_=HBg*U$S)hG5A?>`rR?LK59v|aAYJWBqx`WN?d21T z!3hhaBMJ1#@s5&Lv~Q!;Buu)Y(q$t3iim2(mP61A)#{GZ#$6$d{x|U!+KeNq7MTR{ z#*>br(nkvo(;`!O(tyru9N4(x%#Rlh`QEBpK<4g$HN)71h%G~Li921?Lhh+3l#Hsq$NsQC~{Habe)o^c8LpncQA22#CS?yoON`Ca9zJgK?2THakR?GttY^E^xJYMH;*z)wr znFdZVkkwT+ybgpwQ>p1N%x^YW9AqQ4?R;%?B+voh#I~Y4+sh{^e=EPhk1=23alCU< zei~We7$cvB`^jj4_$_yF)ra*E15hyexF;i)&|o`@@-6(&$k&!^6(apQt~JEn8a49W z_WFiyj!A@g7lF^EdkDOQNyfGa5C;d-AriwbwaEQ0OQPkY#;e`1l5ud+tmKIJM}u(} zhgg!&yz|ltkVbG9>)eS^=j%yw>za-M1SY)!m&kW(cM`Y!`M;MMlanB57Bftiv;>4O zygU&yeH+~V{B*zn>AXU>ZV&&Z{@(q&b!h1Riq}D9)5l{LSa~mC#VLj4+|~$5gS#p= zXc9^-hsz*cQL19MgDP7vl$4_(`yuxv(Rq8Q)T%FmPT@aiiNhxGWjQDUij!Kdyi!`5 zW9fLSs?*=n)?4CWd2`K1@Cphha!%>1M!`c|DC~bxmv_3 z9$CK}7D)?_k%O@yl$KA*T>tVsFR}1}V{;q}JKraL$5VR_QheCAw&eS}4~6s${w09^ zG96v%MoFt4qHLS|#@ykv9M@}NgzSbuDfT!ZGDFrM3g!J{Xp*G$eSH6LK5-MD@}V@lVT`#D zdLgc!#=YQw$TIa-M@6tWPO1d8dVcdiTv>tZdMX6+mietcQxZeM+4jle*5GObhP_pA zfA#Yv~f#wVfWz$~ERd#xw=E3en;rC$OKxy}UJ#fE0PiP!}7oGmSW^-V4C+7E! z#yFC&gMSop3XT8GdGL-Iw$Ih^xge@ttjMhnlkus7Zu|Sn7W6;dp*);S2=eFZ1}gE3 zZhTC4%*B1bAC^?d_@<}tC*x~PS2-9k2=OrB~O^$XAZ!j-*wn{SQ}-y;FCH`h|$0_MUd)4ZqTw25jN0!m9u?~`>$R)1Tk zV}&h3r;-*PPEja{0~3i!{MemfL!?yplS(Jc8@?moOYDpPvuJpkhlXYuiYa?pC-v-gR( zHjl$qzOH0n>1Q2Ip3ChuzNzw=^X9M{@4YKJETsH)U&dK9wFe%$XjNqY%yd2dN?Dgg ztJFPV3g=(!w&EGGw0H0Aj!w4Blw*oVEA^WSY_GHS^_G2BC2joM?=H^Kst{4@rkwJP zrJ^B}QlAz)4p~FL`1nsGX?mS!UQb_k%}tlxw`T~t4b?q8+yW*bP8tnZ*C_oCN3C-r zjDIT5;Y&i(Q&07d^xM+?dW38@WdH#5!%NG7aH83o&_rqtv6Fkjri7+)W~L4iKy;Wj>jEx3+$6+ z-M9x_q^2HQ2XFS>N*XLRCs$W3M`DIC<|pbo{gMy zD2%ThI|=(eggCpGYgXry9c8w;m);>1@GO7xnp|ig3=HX;uiKq^3a5`%U2U>k&Fzrn zAsZP|f0TUj4aQGq*-Cej(@h)1#B_nnELryw0GIP~^LS&Faoza@}H0(g+qD zMexTJryw38(Cj(++RXs_N|~r6C`0(fup(*?5o4Bj+t3%Rky~;9y~SCQ!WEryDj)oY zGPbQZH0ISCaOy0k8T!u@u!#&A5_GI_9RmPbiFgYR4EBc&UzaUD<16yutsS6oWFmvg z;K}nvz)04n?_)*R>CunUNI?oB8Y)NS&a%5`=K^Xl910sj9YVpzj8#+K?qic%8d3O~@he4#s)08r09ZQ2e1GOCsQB*uc%Q4ecLq zYX#p*u~tx>{oW^aC?CJ);+o=f+~?hMcnphner&yuB^X<;<6!oh3GS9moOY5(rE(4& zM?}TGvEh1TqdP-2Dxo9)mRd-%irI<&j zqk{(bu&qUA8gW2y(2dci)$9S&ZEcnW9%?fE+Q8hA&*JRzy$Asf9cVzH%?DTSWR%6{qh;*&FMntnI;D|N z={xFcu^%}2xl*Bqrhz;(sbX<9NT_b}Gr?IFvi?DmF$}kk8Kju$420}$nQjF~U2opD z2Ta!!p-aK)Vm6Ys5Th9i%-%J@=3$8T7Gtfxxaa_lUh&sCqmUNT>1!C#0Khp1RqzmV z85`NWRvZl})F{7G(Kpuzvo1TU9+^MaxMW?Lj2_e4++GDUOnaaKE^LvZ2;5G62tpfF zIPQ&o8Ww1we&~oJS{V3BJ-@y=7kZYN6^Ln-t0{+7Ad zTNJsZBfh`(Y0z!(jxLS{0(V@So4<_eHC}xjAmX*2Q9r^v5@XjfDl~$Z5{Ha@J0{{N z82e-P(^orC1Z&kr6@thRlRRh@B!D~So1Z`H(w^U$9h>$Hl@6sE0MsuVj}SpGUO+Wr zy6}tYBp;2QmG3>7#z25MJj~vxKm=qF;K*{zzptuS@mXi#{4whRt$Ky@0K0sKQ0JA|Bv* z;M2>{u#S)bgh#{Cf)T6V;G@{7^_axqz4(cNSMZe@0WQALBU?xdcQ*i^izPCpptM76 z5r@$%kjb$jpoWE&2Pgk)(+4&4hbivy^7I!4YU^a9=K;BXe}N_gCm~SV30Gbd#xbQ9 zqABl{E%G9gfcgo9RMe$A1rw?54&Dg(jR{4+`%!Uu=DoST@3K0jHT0?PP)%DkIFR8n z5XM@nO8$ORUQ_0&FZQ#k4x21i*E~?-Lpj7s&^1uLrgp+qF+CO{&eNNIRv3zv;f(`> zgyRdb*QIvirhG33V!s1cG;#y2ysp~1%fyWkO#|<6qTMJ{s#N|)*L$pT<#SYyEy)RJLg)JPUO3KdT0vTskYNv(FO8Rj3 zjVE-f)U)IDERYWz|E$!@3NzuF=JtiQjAX-p=;)|s8CbmgX>ykFG=SlnQ8;YUhJY*; z7k&6Nglw`6I_TFqO=QhCV#ru#`9b2dO8-w6o453!WycUs9}TJz!mY{$FYst;h_&6i zmHAt`=uBUeFU%ttyk56h$@C$*X|HXQw7@w5B$LxLB#5XV=Nnmh$t##x#57Y96)(?rt(q{@i*iX=vXkJceYA$A{tS!Mz%$+IPRW zlE;Eu1B$bKyICewbI)d($0=00)^GCSh`3+#+Ds@`&eln+N6&vh9hkHMojBUZli;JZ zK@s|TtpE&!(o2<#jVUunf{exPu;tRSlKE^1oUEM9QhjKdR>1EX7=z=H8#|~`RAzQi z!?daOnQ)AwKFGHEx4W~N@$;)YDU!|FfISJ2uIhPnSV1-Hyn#8I3gJG z<0o+jBF$S_i(qkaT;BJep+r`y!qH5wgzQFnSHUHi{Sw8hFi%`2y@m6 zxxx4LqO6PjIieHSt~R{a7EDI(iUWhABy2%VI4qvsLL>P@x&Qw1A&R9@dW(*?eNHVD9M~-0KIFqTP@)GcaOqwj;0;g(w`@WdP+xGX zrQM6ycvlayXvwEjH08SRd*&(T6omGTgx2`J$wJvcQ0C9j+Isg*5hXd4b)KX@fuL~^5REP)aRp@n7!83dQ3#Ctfr}2>yn}IU08mxqk&xG@VyKxt z=m3NwmHhe(4mOFQc_11R5^_HoF8M))LFyoS(2LE`&4EY+R7~Q1CZr}>R>z8neX6q{ zE42R7_eZT1fryR{CQFWrR>9DFS>q3~Fh&FPi56;hGH+&j3R}XAHG$a9XA zkh; z@qzJ(fdPAny$JjCLBtC=Y|_&60e{6h=+Wj-gp-mVP%6F2leT}wPDxl~ctE(IE|M0s zSqPi-zrv?NY?;tdluKO3jxg1(`kL++c0a!G62Kl7_L@mXW;Fldt1JNj2Wz4JhhmF# z>02^G@wj19&w%n^Za#xG{EN+CC#dJ2{tZ(LA%{w%t-b0o9cy=1!ifAlGTn>NkGhEz z|HkPr$*A6#13`n&>gv&@zv}ihU4oqhE4uT6P#UBN?bqtK6hs_Rq<{X0#w4T)4uP{4 zdG!%wAA<9evhie|Lvlcgn@~o{)HIjp*VL*3E5H3pLVH^H)NWxp-}hSHuTj7;=k0x7 z;xAl=cQ2Gq^*Y23A}%Pnl)W)2SU%A*47LCSYy>FW3&#wZmrg2v(2cvyP86aP6 ze*|AI_JTH9`EhmI8k1Eaug{EhS767r>p?<0_or0cG-?cD|b^)5gl z=CBe9{l&@tS?%p8 zOe5zuf0bx$y1vVCd44F1`YJfSExW1?BLP;Z9}@;xipjtD z+(`{u}%Y(r!SC)H7Tu{Yl-I} zZ?;@ z&~(DB8O=n)KXVYlDd&0_Dq7RY-PA`~#Swzgk30Z}N@CK+34))*gejyR;SbgKVMv3lp|BhElXoF&#*-;Lbfs z07*y(PDj6`{ zfIX^rSuCWh9R@#^3d^UOsXmA41<1mT1uBYp4D;%z=mbpdBeh<^lnxOlXeI^T)Ozs6 zl;*2>i*Vyl)Sq!V5$E`6Rh;` z({J~v7fB7##COAXv=|%JAG(n~{7X6=RVsm?{5728SWNbr+uL;_+A_p&Eafzm z#q3tYAGRVo0e;kGIMa~h9`>CEJ;;Y>#q@^n=OS}A_|0SnWG9NVb5QGV+sJE70nY>wwL6 zaJ}qRyhx%G}K%=brw1id2~;>+>RP%z4GXq#TO?dXDC!VWZz^ zkiZq}4H)aj<`IqH%f*sLl)+Vl13?yQdavEtyU|c5yba^R3MOrTCm8}@vu&6LwyRIU zCKUn3F9EpCNGFuQ5b2x?k|{bX27&>kV&ZFH6?Z?yPkP}}nO1BXCOB^1UlB-ocpxLG zcR*I-`~?Kzz--5RKg@7d^Vn5nj5{|ZG;W=BVMIfphc6e8NbxaFe4(LkQI%_glJ)RQ z`4L9qRE^ysNKKaU&=W%3m$0cc#HSOfRhtAOl1Ti&n$X&7-h+ga&)E&#YDkX2cy)Lg zR9@^d*~#skbtWbxHhG?nOhTdaXGy}7c$$><1FvH{0X<)e2iY@P_1$*6 zW_6I82HJ5>;!l+P0A?Q*rGPmz^#ju3qr_-l2lk^f88#cu@yEd5KCe|2dXg^Y+z^7R z0a;}X&36(cM0bu~&{Rtxo_f#U}g*j$-&WT(= z1fqKWYd{ijSy;`F$wkX7FO=-YQLQVlw{!h4KDV;$UmuAJ!uvCBlW`PgPn*s-(5un-P%LElnyNnrS3Yq>Lx;O`R_0oCtFF zra1<=69Z9J?-?-F)S_6vl+IO1oxL3w|IfaJVkS;gr?S=Xo}jr&7bLhf;@* zS5=olTpS>I3W0o&7xMmjXXAKctOmx`w|k2mztFy3^3XF`vyKPzMd4ewS>aOtV)3eQ zF-}SG4CgphOj0IFKW$xhgAq<0rn9rNDNW0C$?&@5^G@qwzT3tX>@&ms_cphMO9phx zaT1q-<(>Nt2g4&R$4zNAqNS?R>c!&;Pl)02E%MV_`Dey*1NQB7*R}U_>@W%1W0}mV z)a_Pebs=bGbOw zGzX;Kz*OdE%ZCW_uHR(oPgs+(J%$p;8CvhTLi_5ML-&2orI8b0lv4K-&8<@G$5NiY zmg0M5f$wgld>Txu?gHMhMU^K~9y>aPltPwV_yyNPdft7$BtxI8!Bad2i@{ogvz1~3 zi=(^fEIYAk1H&Qz%jgl~dQJ2%!Mfpv&JB8HjfhsJf#&HV4!LB5qRffKqng9oRl2*s z7)UZm{)Qorx*#LWCwHH}6VndwmImIcTWUKAp+$z{wXY)0QCF%y<=SN#44)GK{mC;p zIIzy`#r%A7A4=S^D$}uqK-*Nbqoq2}F!5ysmWh#N`Sz+GUY`4tOw^ShysGjT-C#@7 zK|{p~Hrp#}+0#YFVgK7`9{ZnF&4H0jlkb_}oj3bOH#!EQu5C05cRWOgY5Gc?{&4eX z)P=z>@qmoK{d`&pRqz{pN+L@>E$Ky1_CdscF$QiUr|#-um+ZMPf=x|bvc&N zu>Eo?UEB5XInh47qNynxq~CVC+)rO6z22r;>QmL=O|F+drD^W{&kw6D`j<@({za=D z@HCW+G#&`W{jdq<&!~$9TUh!4(q_e~$jM}4B^kK{0$XDc@}SJD7>0(_(q)4|2z1(t zZJx{JUn87`-cazHeoiM~Rz2mrX??q#1Erxo50D`PmuesQP?~bS^3rrdBs}2zpuU)w z1vOs%XDX)-k&^#}1|WA67fgD8iF;jp76$Qr5Ahh4`bvk)j1icYH9`^z)2aGkwDC3> z7}v3bG>qY4c{rZgne*P(X126#wbXq5&|j}cUe&b5ZlG#$V=dKkKFgQbMt9a|1)Ar# z`6cUw%0j&?T^d8}rN1xYe`GC8bTq7EhIqVgA4Z9hoj080M^=qrjbo0cT=(w`n5oLq}-n_Vx$|}*decZMoehtzUeRxnPUCb%X5b-c-xZHi%-7(VE7@&Va zM!(Bppvf_3J5>?=l4TAUx(o+Js1v7_5AuUhj^Iz%odg8@&!-mOuada+J^6S`n9}8G z(cCs;7Owz$@F^DN6~e#K9T5IiSEkw?i9myBwpE9YoZ!;49L~fFw_<*}2Y2nm3K^AQ zLY99A3Z2B_`dobf>P14e%~bOBu$KBHw9#r&uKm;E9Yv)0nAqnQHA{HvpmfYBy$>s` zhV9h|P`Eu3K!M~NRq-g-U-LRuMs6y@1SJmtWl>ICpa)*z&o^O^RFFk_OS2SH^dB?? z%P^X77*_!Mrs&^YW^|aaVOc&uSn=QOQ1UPwfb-)`%!`=6#ZGoU2=@E`pefh}rN0CO z`PTQff3hTrjgeL2Kd=E7sxTtv|IgDSeb(S65EgGETXs@$NllhUY6||5JLM-U*9E@> z{w*;fI2+h)-huzM_YUWO>j!EYQ#DP^ckDEUaS6>abT!x#vcE$8X=SNA9UNbL&kQzf z$7zHR30bGR{t^*SZ#to0<+aA+xBZ#h_F*|ww+wUFBuht*xXO6f=o6RIKI{2Qp?)Nt z)l46TGo##}$0luy&sdqlE=KO}-`!^LvkCU3ntd?u>&m4qu+J}2Z}GOfU8|FKsvBq4 zI&X1M7?R((eiLDMxE~4$>8lqwH{|zzc*5nXtT)e65w39hIwlVX!hFKXb8zuL<&uc- zA6b1;&nmHg`>qD627AmTF&hQPX~mZ3IV6^fJTN;REb&7CjK zODy-xAUo4QM}w4=Pon%Eh5ugt+Y2>hhMvkjfQs(OWA7vy^%JK*%PCj!GZwtCR&hxL zIgx%;P2w6w$s(k52|W+NodFF1X7!^C34vM2L?0aBSzMp0y}&unU=((+&+|v$h{aX68s>FhK}0iP0ea*y0UlLI1%F|ddgGGc^}(^ zEAe3dq`hISj0Y%A@x+Vd4C8hXyg9A?qd7$T#Nd935GqXuyG_;(&->?7zJHA{YHhZD z@zTMJvmYP$6dmVjro*8M?!`U-2)mH^6{Q1h$j!h*W$$DLJy4nBlpbiVXu7`ozwvSg z`YrE+_fel=yOC`FJV10biI_AOHpLg*P5Rr4I@=JO=hRMuHa?geFWV01@;k>{W&hVZ zz~(adOdr{!yVR^1k4!vZ3`CYlc)~I*@vW?IO*tVWD->yRtrFbOZ$O5-a~ooNBSXIS zywmD9YCZBq+mJLkDdG!_=bT;Sm5>(-k9pTdmV#UBHo)$ZtQY2%hKik5E@AeheVG1q z?{aa2ZarDpG=3*L@?REo4@H(jPUV8hcm(S*r$;WkM_?PjEI!Xgr*nB)7v%$;f(WdPVt3DP44Na&?Bh&37i;T)EbF*S>VMx$vXr6U%Mc zTSH=4=bMd;1hVVWYjsvKoTQY$f`O_;7;@5W1hzPG%IGy7usZl_8o}tlZ!_dDXfWfe zSM&MfsTL+Nuf5s#6dL5k)om7?g4FYXe!treC%~-ORq{0F zh#_Ro))!Lqy1V8DIp^CypiMBPF|&2J<(yFXz-fuR^-6vA_Izl1dNJVq{G3fBiPf&{ z>U3#b+bM9|+pAnzgiU|0=b>`0;rJ0{@D@O*MX4KJSLhXfENTbkE&dcbNs2B2qsQ8# zeU1__vcYdic7eT5jhX|;`zmJq~%p)O67$$X({eY z6P$q|z{Aep?q06=C{v6gja`pSepZq^3Bsp=UZo>Vx z=1%toEA;A&3?oPNo60Y9`=Lldzim@M^X(ql3u66+YIPevg>vOr6RnaugQDJwnKBJ} zt1P*M93v6rNTFJqT|ejzVD%3)<6%;(fYu7rrSH74UTUpV`Re#Lmfs?T@E*!uskxn8 z$q~IG=%``}4Gp#NE%9NSy7PP)$r^h_$z4Uu(7g4}#>7~>D`RMwHTLv<565ik4zG`t>D7my>CilR!YjQ zZrJxXm?`Jv5hnLNkC9LA&1CkNWXFHzL=kvQDaa}k84VgX3l2#H{tfNi?ar2gdLDHD zhQp}U;IW@4LQ$MCzzdLa6V!5;Y8gmc8tyR6&E4lnrZOKX@5R8a24OKqw$jI3{5b#d$NeFqM7W3Uc>5WaXSrJ9!fS&LhWX zQ84HI=iW6Q0{rIe4w>^dw{$S5{U=NuuXX7lw~V4R?I7|u^S{9|AEuR0xE z`!!5SzwHU@UCHxTU*uZYu8#c?uY}_}A?rMU$~E<43+o?w!77xwQU7+irT%QZ=?XX0 zWcS=R$NU*w7?D3x9~T=Y=vjXkbVwwp@Bb3Isc`1iZR&o+PD8m?{Tj3U?aQr8y;hk@ zUm5c#1{Ug_)c(tb%%-g;eOR9Nvn`b3XCLs}e}^eE-`(svF}qpo-Z59(J*=Pb!~YPA zL4U7fbuVO$C&fpl9yhUtUEr2W55+)wIgu+~QUgJ#3EFJ&K1!cGLs8O~+rQA256F4j zgAqc~_=Z_t`1E=a>-r>^psrdr3JRsb>23~>TPS#bAJRH2Q5nl$L$oOy5(>nO&@;&d z;(NEZR2l+wz=~;{vuo<11*e)=%TN^=Op=4fsfm}TRhqy~D z!YX`Yyl{A8ds&6q^TjWQ+$)cbEheqQ7o70Lz7Jca%%Nu6`we9=Jqwgqc73Z32o0|o zW5UqHPZ`O4BB*UOdnp6Qbb?*k@Es278T?%jcZ|U~IRqgj02a_w0MxAigObU+6|Hw*39@=L6_@Ik*X6#J)&k3~7v5#)tCYScAj& zc7om=UqcpFXO4uXeNcED-~A4uRFRkb6VIlcZvAA{7K~Z_PAEOAw?UX+8*ik15#WEC zuyvr|@GN^@>tstthoWcjqTylst>PFGaiC$5agc#9{jMWu`pQkNl(X#{k&C}x0`j79 zDsWCnmL%oD6}p6H|9Lf9T;O{8ybcvCkj0n6%?-V=fB#~M`?sIN1gKMJ(vGjU^04J7 z3Pp`aoW?uixtO#BC~r`y`LAo!aZyhYU`xudw=Lj1#`1YX;he6f6EH>Td(!3F!L#x; z=RKF+an>0=4HA>6;5LezpjGGl<+tN_G9PFA=2I`Ny4Kc*h;`fi@)uw`eFM=N#PEe_ zuZiN5D>2v+?Jj0Riy6RIR@~A@52ds+JtYMuS2|puW;GEDanodzjac09V zL*tgCQ_YPAq%-wtZHZd%PpShQ$~!cQ%ZW=MWM&ALtf?Nz+?k*GXpMWU|ozTki+nJo)4${ zD!c^42Jd$*j6EC7@4uD8tO&VNqAKGOw8-;*Mx>#XMdhB1tw0ho{)X6NP=zT$0WSWE z{GDk)V8g&^?7h-k7;-HB8u}UvTMS<_P$jd^yaR4v4MVW(-bN|6&}QKXMc?Yz*y7yX z7`h++T(vYT_@`yU|9t%|8(D(zq&?6|=Kq@2A~kqx@V2PfhGTjMm)&0>+c)zsixF^9 zQrb1ETZWta6CkY-U;uTPKl|yD^zdIIR|@uij!Yl0q-a&vNy>#b+m-reJQwx{wL~hU z6_40I%U1>Vq<~d-C$RKQ=HmVj>ta0Wz}+QVw&6oQEsT5kpIM~#r^llQ1rorsfEYYo L{an^LB{Ts57WlQ# literal 0 HcmV?d00001 diff --git a/docs/proposals-accepted/20221129-avoid-global-sort.md b/docs/proposals-accepted/20221129-avoid-global-sort.md new file mode 100644 index 0000000000..84796ff1ff --- /dev/null +++ b/docs/proposals-accepted/20221129-avoid-global-sort.md @@ -0,0 +1,171 @@ +## Avoid Global Sort on Querier Select + +* **Owners:** + * @bwplotka, @fpetkovski + +* **Related Tickets:** + * https://github.com/thanos-io/thanos/issues/5719 + * https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020 + * https://github.com/thanos-io/thanos/pull/5796 also alternatives (https://github.com/thanos-io/thanos/pull/5692) + +> TL;DR: We propose solution that allows saving query and query_range latency on common setups when deduplication on and data replication. Initial benchmarks indicate ~20% latency improvement for data replicated 2 times. +> +> To make it work we propose adding field to Store API Series call "WithoutReplicaLabels []string", guarded by "SupportsWithoutReplicaLabels" field propagated via Info API. It allows telling store implementations to remove given labels (if they are replica labels) from result, preserving sorting by labels after the removal. +> +> NOTE: This change will break unlikely setups that deduplicate on non-replica label (misconfiguration or wrong setup). + +## Glossary + +**replica**: We use term "replica labels" as a subset of (or equal to) "external labels": Labels that indicate unique replication group for our data, usually taken from the metadata about origin/source. + +## Why + +Currently, we spent a lof of storage selection CPU time on resorting resulting time series needed for deduplication (exactly in [`sortDedupLabels`](https://github.com/thanos-io/thanos/blob/main/pkg/query/querier.go#L400)). However, given distributed effort and current sorting guarantees of StoreAPI there is potential to reduce sorting effort or/and distribute it to leafs or multiple threads. + +### Pitfalls of the current solution + +Current flow can be represented as follows: + +![img.png](../img/bottleneck-globalsort.png) + +1. Querier PromQL Engine selects data. At this point we know if users asked for deduplicated data or not and [what replica labels to use](https://thanos.io/tip/components/query.md/#deduplication-replica-labels). +2. Querier selection asks internal, in-process Store API which is represented by Proxy code component. It asks relevant store API for data, using StoreAPI.Series. +3. Responses are pulled and k-way merged by the time series. StoreAPI guarantees the responses are sorted by series and the external labels (including replica) are included in the time series. + * There was a [bug in receiver](https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020#diff-b3f73a54121d88de203946e84955da7027e3cfce7f0cd82580bf215ac57c02f4) that caused series to be not sorted when returned. Fixed in v0.29.0. +4. Querier selection waits until all responses are buffered and then it deduplicates the data, given the requested replica labels. Before it's done it globally sort data with moving replica label at the end of the time series in `sortDedupLabels`. +5. Data is deduplicated using `dedup` package. + +The pittfall is in the fact that global sort can be in many cases completely avoided, even when deduplication is enabled. Many storeAPIs can drop certain replica labels without need to resort and others can k-way merge different data sets without certain replica labels without extra effort. + +## Goals + +Goals and use cases for the solution as proposed in [How](#how): + +* Avoid expensive global sort of all series before passing them to PromQL engine in Querier. +* Allow StoreAPI implementation to announce if it supports sorting feature or not. The rationale is that we want to make it possible to create simpler StoreAPI servers, if operator wants to trade-off it with latency. +* Clear the behaviour in tricky cases when there is an overlap of replica labels between what's in TSDB vs what's attached as external labels. +* Ensure this change can be rolled out in compatible way. + +## Non-Goals + +* Allow consuming series in streamed way in PromQL engine. + * While this pitfall (global sort) blocks the above idea, it's currently still more beneficial to pull all series upfront (eager approach) as soon as possible. This is due to current PromQL architecture which requires info upfront for query planners and execution. We don't plan to change it yet, thus no need to push explicitly for that. + +## How + +### Invariants + +To understand proposal, let's go through important, yet perhaps not trivial, facts: + +* For StoreAPI or generally data that belongs to one replica, if you exclude certain replica label during sort, it does not impact sorting order for returned series. This means, any feature that desired different sort for replicated series is generally noop for sidecars, rules, single tenant receiver or within single block (or one stream of blocks). +* You can't stream sorting of unsorted data. Furthermore, it's not possible to detect that data is unsorted, unless we fetch and buffer all series. +* In v0.29 and below, you can deduplicate on any labels, including non replicas. This is assumed semantically wrong, yet someone might depend on it. +* Thanos never handled overlap of chunks within one set of store API response. + +### Solution + +To avoid global sort, we propose removing required replica labels and sort on store API level. + +For the first step (which is required for compatibility purposes anyway), we propose a logic in proxy Store API implementation that when deduplication is requested with given replica labels will: + +* Fallback to eager retrieval. +* Remove given labels from series (this is can remove non-replica labels too, same as it is possible now). +* Resort all series (just on local level). + +Thanks of that the k-way merge will sort based on series without replica labels that will allow querier dedup to be done in streaming way without global sort and replica label removal. + +As the second step we propose adding `without_replica_labels` field to `SeriesResponse` proto message of Store API: + +```protobuf +message SeriesRequest { + // ... + + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + repeated string without_replica_labels = 14; +``` + +Since it's a new field, for compatibility we also propose adding `supports_without_replica_labels` in InfoAPI to indicate a server supports it explicitly. + +```protobuf +// StoreInfo holds the metadata related to Store API exposed by the component. +message StoreInfo { + reserved 4; // Deprecated send_sorted, replaced by supports_without_replica_labels now. + + int64 min_time = 1; + int64 max_time = 2; + bool supports_sharding = 3; + + // replica_aware means this store supports without_replica_labels of StoreAPI.Series. + bool supports_without_replica_labels = 5; +} +``` + +Thanks of that implementations can optionally support this feature. We can make all Thanos StoreAPI support it, which will allow faster +deduplication queries on all types of setups. + +In the initial tests we see ~2x improvements on my test data (8M series block, requests for ~200k series) with querier and store gateway. + +Without this change: + +![1](../img/globalsort-nonoptimized.png) + +After implementing this proposal: + +![2](../img/globalsort-optimized.png) + + +## Alternatives + +1. Version StoreAPI. + +As a best practice gRPC services should be versioned. This should allow easier iterations for everybody implementing or using it. However, having multiple versions (vs extra feature enablement field) might make client side more complex, so we propose to postpone it. + +2. Optimization: Add "replica group" as another message in `SeriesResponse` + +Extra slice in all Series might feel redundant, given all series are always grouped within the same replica. Let's do this once we see it being a bottleneck (will require change in StoreAPI version). + +3. Instead of removing some replica labels, just sort without them and leave at the end. + +For debugging purposes we could keep the replica labels we want to dedup on at the end of label set. + +This might however be less clean way of providing better debuggability, which is not yet required. + +Cons: +* Feels hacky. Proper way for preserving this information would be like alternative 4. +* Debuggability might be not needed here - YAGNI + +4. Replica label struct + +We could make Store API response fully replica aware. This means that series response will now include an extra slice of replica labels that this series belongs to: + +```protobuf +message Series { + repeated Label labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; + repeated Label replica_labels = 3 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; // Added. + + repeated AggrChunk chunks = 2 [(gogoproto.nullable) = false]; +} +``` + +Pros: +* Easy to tell what is replica what's not on client of Store API level + +Cons: +* Extra code and protobuf complexity +* Semantics of replica labels are hard to maintain when partial deduplication is configured (we only dedup by part of replica labels, not by all of them). This dynamic policy makes it hard to have clean response with separation of replica labels (i.e. should included replica labels be in "labels" or "replica labels")? + +This might be not needed for now. We can add more awareness of replication later on. + +## Action Plan + +The tasks to do in order to migrate to the new idea. + +* [ ] Merging the PR with the proposal (also includes implementation) +* [ ] Add support for `without_replica_label` to other store API servers. diff --git a/internal/cortex/querier/queryrange/queryrange.pb.go b/internal/cortex/querier/queryrange/queryrange.pb.go index aacce2d0c7..b918da0ece 100644 --- a/internal/cortex/querier/queryrange/queryrange.pb.go +++ b/internal/cortex/querier/queryrange/queryrange.pb.go @@ -538,7 +538,6 @@ func (m *PrometheusInstantQueryData) GetStats() *PrometheusResponseStats { type PrometheusInstantQueryResult struct { // Types that are valid to be assigned to Result: - // // *PrometheusInstantQueryResult_Scalar // *PrometheusInstantQueryResult_StringSample // *PrometheusInstantQueryResult_Vector diff --git a/pkg/dedup/iter.go b/pkg/dedup/iter.go index a84ea54767..4ffbfd5234 100644 --- a/pkg/dedup/iter.go +++ b/pkg/dedup/iter.go @@ -37,6 +37,8 @@ func isCounter(f string) bool { return f == "increase" || f == "rate" || f == "irate" || f == "resets" } +// NewSeriesSet returns seriesSet that deduplicates the same series. +// The series in series set are expected to be sorted by all labels. func NewSeriesSet(set storage.SeriesSet, replicaLabels map[string]struct{}, f string, pushdownEnabled bool) storage.SeriesSet { // TODO: remove dependency on knowing whether it is a counter. s := &dedupSeriesSet{pushdownEnabled: pushdownEnabled, set: set, replicaLabels: replicaLabels, isCounter: isCounter(f), f: f} @@ -63,9 +65,8 @@ func (s *dedupSeriesSet) Next() bool { } s.replicas = s.replicas[:0] - // Set the label set we are currently gathering to the peek element - // without the replica label if it exists. - s.lset = s.peekLset() + // Set the label set we are currently gathering to the peek element. + s.lset = s.peek.Labels() pushedDown := false if s.pushdownEnabled { @@ -79,28 +80,6 @@ func (s *dedupSeriesSet) Next() bool { return s.next() } -// peekLset returns the label set of the current peek element stripped from the -// replica label if it exists. -func (s *dedupSeriesSet) peekLset() labels.Labels { - lset := s.peek.Labels() - if len(s.replicaLabels) == 0 { - return lset - } - // Check how many replica labels are present so that these are removed. - var totalToRemove int - for i := 0; i < len(s.replicaLabels); i++ { - if len(lset)-i == 0 { - break - } - - if _, ok := s.replicaLabels[lset[len(lset)-i-1].Name]; ok { - totalToRemove++ - } - } - // Strip all present replica labels. - return lset[:len(lset)-totalToRemove] -} - func (s *dedupSeriesSet) next() bool { // Peek the next series to see whether it's a replica for the current series. s.ok = s.set.Next() @@ -109,7 +88,7 @@ func (s *dedupSeriesSet) next() bool { return len(s.replicas) > 0 || len(s.pushedDown) > 0 } s.peek = s.set.At() - nextLset := s.peekLset() + nextLset := s.peek.Labels() var pushedDown bool if s.pushdownEnabled { diff --git a/pkg/dedup/iter_test.go b/pkg/dedup/iter_test.go index 2ecd74473f..a54011b0f1 100644 --- a/pkg/dedup/iter_test.go +++ b/pkg/dedup/iter_test.go @@ -146,7 +146,7 @@ var expectedRealSeriesWithStaleMarkerDeduplicatedForRate = []sample{ } func TestDedupSeriesSet(t *testing.T) { - tests := []struct { + for _, tcase := range []struct { name string input []series exp []series @@ -458,9 +458,7 @@ func TestDedupSeriesSet(t *testing.T) { }, dedupLabels: map[string]struct{}{"replica": {}}, }, - } - - for _, tcase := range tests { + } { t.Run(tcase.name, func(t *testing.T) { // If it is a counter then pass a function which expects a counter. f := "" diff --git a/pkg/info/infopb/rpc.pb.go b/pkg/info/infopb/rpc.pb.go index e8bf57800d..6153dbb4cf 100644 --- a/pkg/info/infopb/rpc.pb.go +++ b/pkg/info/infopb/rpc.pb.go @@ -117,10 +117,11 @@ var xxx_messageInfo_InfoResponse proto.InternalMessageInfo // StoreInfo holds the metadata related to Store API exposed by the component. type StoreInfo struct { - MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` - MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` - SupportsSharding bool `protobuf:"varint,3,opt,name=supports_sharding,json=supportsSharding,proto3" json:"supports_sharding,omitempty"` - SendsSortedSeries bool `protobuf:"varint,4,opt,name=sends_sorted_series,json=sendsSortedSeries,proto3" json:"sends_sorted_series,omitempty"` + MinTime int64 `protobuf:"varint,1,opt,name=min_time,json=minTime,proto3" json:"min_time,omitempty"` + MaxTime int64 `protobuf:"varint,2,opt,name=max_time,json=maxTime,proto3" json:"max_time,omitempty"` + SupportsSharding bool `protobuf:"varint,3,opt,name=supports_sharding,json=supportsSharding,proto3" json:"supports_sharding,omitempty"` + // replica_aware means this store supports without_replica_labels of StoreAPI.Series. + SupportsWithoutReplicaLabels bool `protobuf:"varint,5,opt,name=supports_without_replica_labels,json=supportsWithoutReplicaLabels,proto3" json:"supports_without_replica_labels,omitempty"` } func (m *StoreInfo) Reset() { *m = StoreInfo{} } @@ -357,40 +358,41 @@ func init() { func init() { proto.RegisterFile("info/infopb/rpc.proto", fileDescriptor_a1214ec45d2bf952) } var fileDescriptor_a1214ec45d2bf952 = []byte{ - // 519 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0xcf, 0x6a, 0xdb, 0x4e, - 0x10, 0xc7, 0xad, 0xf8, 0xff, 0x38, 0xce, 0xef, 0xe7, 0x6d, 0x5a, 0x64, 0x1f, 0x14, 0x23, 0x72, - 0x30, 0xb4, 0x48, 0xe0, 0x42, 0x29, 0xf4, 0xd4, 0x84, 0x40, 0x03, 0x0d, 0xb4, 0xb2, 0x4f, 0xb9, - 0x08, 0x39, 0x9e, 0x38, 0x02, 0x4b, 0xbb, 0xd9, 0x5d, 0x83, 0xfd, 0x16, 0x7d, 0x82, 0xbe, 0x43, - 0xdf, 0xc2, 0xc7, 0x1c, 0x7b, 0x2a, 0xad, 0xfd, 0x22, 0x45, 0xb3, 0x72, 0x6a, 0xd1, 0x9c, 0x7a, - 0x91, 0x76, 0xe7, 0xf3, 0xfd, 0x8e, 0x76, 0x67, 0x34, 0xf0, 0x3c, 0x4e, 0x6f, 0xb9, 0x9f, 0x3d, - 0xc4, 0xc4, 0x97, 0xe2, 0xc6, 0x13, 0x92, 0x6b, 0xce, 0x5a, 0xfa, 0x2e, 0x4a, 0xb9, 0xf2, 0x32, - 0xd0, 0xeb, 0x2a, 0xcd, 0x25, 0xfa, 0xf3, 0x68, 0x82, 0x73, 0x31, 0xf1, 0xf5, 0x4a, 0xa0, 0x32, - 0xba, 0xde, 0xf1, 0x8c, 0xcf, 0x38, 0x2d, 0xfd, 0x6c, 0x65, 0xa2, 0x6e, 0x1b, 0x5a, 0x97, 0xe9, - 0x2d, 0x0f, 0xf0, 0x7e, 0x81, 0x4a, 0xbb, 0xdf, 0xca, 0x70, 0x68, 0xf6, 0x4a, 0xf0, 0x54, 0x21, - 0x7b, 0x03, 0x40, 0xc9, 0x42, 0x85, 0x5a, 0xd9, 0x56, 0xbf, 0x3c, 0x68, 0x0d, 0x3b, 0x5e, 0xfe, - 0xc9, 0xeb, 0x8f, 0x19, 0x1a, 0xa1, 0x3e, 0xab, 0xac, 0x7f, 0x9c, 0x94, 0x82, 0xe6, 0x3c, 0xdf, - 0x2b, 0x76, 0x0a, 0xed, 0x73, 0x9e, 0x08, 0x9e, 0x62, 0xaa, 0xc7, 0x2b, 0x81, 0xf6, 0x41, 0xdf, - 0x1a, 0x34, 0x83, 0x62, 0x90, 0xbd, 0x82, 0x2a, 0x1d, 0xd8, 0x2e, 0xf7, 0xad, 0x41, 0x6b, 0xf8, - 0xc2, 0xdb, 0xbb, 0x8b, 0x37, 0xca, 0x08, 0x1d, 0xc6, 0x88, 0x32, 0xb5, 0x5c, 0xcc, 0x51, 0xd9, - 0x95, 0x27, 0xd4, 0x41, 0x46, 0x8c, 0x9a, 0x44, 0xec, 0x03, 0xfc, 0x97, 0xa0, 0x96, 0xf1, 0x4d, - 0x98, 0xa0, 0x8e, 0xa6, 0x91, 0x8e, 0xec, 0x2a, 0xf9, 0x4e, 0x0a, 0xbe, 0x2b, 0xd2, 0x5c, 0xe5, - 0x12, 0x4a, 0x70, 0x94, 0x14, 0x62, 0x6c, 0x08, 0x75, 0x1d, 0xc9, 0x59, 0x56, 0x80, 0x1a, 0x65, - 0xb0, 0x0b, 0x19, 0xc6, 0x86, 0x91, 0x75, 0x27, 0x64, 0x6f, 0xa1, 0x89, 0x4b, 0x4c, 0xc4, 0x3c, - 0x92, 0xca, 0xae, 0x93, 0xab, 0x57, 0x70, 0x5d, 0xec, 0x28, 0xf9, 0xfe, 0x88, 0x99, 0x0f, 0xd5, - 0xfb, 0x05, 0xca, 0x95, 0xdd, 0x20, 0x57, 0xb7, 0xe0, 0xfa, 0x9c, 0x91, 0xf7, 0x9f, 0x2e, 0xcd, - 0x45, 0x49, 0xe7, 0x7e, 0xb5, 0xa0, 0xf9, 0x58, 0x2b, 0xd6, 0x85, 0x46, 0x12, 0xa7, 0xa1, 0x8e, - 0x13, 0xb4, 0xad, 0xbe, 0x35, 0x28, 0x07, 0xf5, 0x24, 0x4e, 0xc7, 0x71, 0x82, 0x84, 0xa2, 0xa5, - 0x41, 0x07, 0x39, 0x8a, 0x96, 0x84, 0x5e, 0x42, 0x47, 0x2d, 0x84, 0xe0, 0x52, 0xab, 0x50, 0xdd, - 0x45, 0x72, 0x1a, 0xa7, 0x33, 0x6a, 0x4a, 0x23, 0xf8, 0x7f, 0x07, 0x46, 0x79, 0x9c, 0x79, 0xf0, - 0x4c, 0x61, 0x3a, 0x55, 0xa1, 0xe2, 0x52, 0xe3, 0x34, 0x54, 0x28, 0xe3, 0xbc, 0x2b, 0x8d, 0xa0, - 0x43, 0x68, 0x44, 0x64, 0x44, 0xc0, 0x6d, 0x41, 0xf3, 0xb1, 0x3b, 0xee, 0x31, 0xb0, 0xbf, 0x4b, - 0x9e, 0xfd, 0x86, 0x7b, 0x65, 0x74, 0x2f, 0xa0, 0x5d, 0xa8, 0xcf, 0xbf, 0xdd, 0xca, 0x3d, 0x82, - 0xc3, 0xfd, 0x82, 0x0d, 0xcf, 0xa1, 0x42, 0xd9, 0xde, 0xe5, 0xef, 0x62, 0x1f, 0xf7, 0xe6, 0xa0, - 0xd7, 0x7d, 0x82, 0x98, 0x89, 0x38, 0x3b, 0x5d, 0xff, 0x72, 0x4a, 0xeb, 0x8d, 0x63, 0x3d, 0x6c, - 0x1c, 0xeb, 0xe7, 0xc6, 0xb1, 0xbe, 0x6c, 0x9d, 0xd2, 0xc3, 0xd6, 0x29, 0x7d, 0xdf, 0x3a, 0xa5, - 0xeb, 0x9a, 0x99, 0xcf, 0x49, 0x8d, 0xc6, 0xeb, 0xf5, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x2e, - 0xb5, 0x07, 0x23, 0xb5, 0x03, 0x00, 0x00, + // 533 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0xdf, 0x6a, 0xdb, 0x30, + 0x14, 0xc6, 0xe3, 0xe6, 0xbf, 0xd2, 0x74, 0xad, 0xe8, 0x86, 0x13, 0x86, 0x13, 0x4c, 0x2f, 0x02, + 0x1b, 0x31, 0x64, 0x30, 0x06, 0xbb, 0x5a, 0x4b, 0x60, 0x1d, 0x2b, 0x6c, 0x4e, 0x60, 0xd0, 0x1b, + 0xa3, 0xa4, 0x6a, 0x62, 0xb0, 0x2d, 0x55, 0x92, 0x59, 0xf2, 0x16, 0x7b, 0x95, 0x5d, 0xef, 0x05, + 0x72, 0xd9, 0xcb, 0x5d, 0x8d, 0x2d, 0x79, 0x91, 0xa1, 0x23, 0x27, 0x8b, 0x59, 0xaf, 0x7a, 0x93, + 0x48, 0xe7, 0xfb, 0x7d, 0xc7, 0xd2, 0x39, 0x3a, 0xe8, 0x69, 0x98, 0xdc, 0x32, 0x4f, 0xff, 0xf0, + 0x89, 0x27, 0xf8, 0xb4, 0xcf, 0x05, 0x53, 0x0c, 0x37, 0xd4, 0x9c, 0x24, 0x4c, 0xf6, 0xb5, 0xd0, + 0x6e, 0x49, 0xc5, 0x04, 0xf5, 0x22, 0x32, 0xa1, 0x11, 0x9f, 0x78, 0x6a, 0xc9, 0xa9, 0x34, 0x5c, + 0xfb, 0x74, 0xc6, 0x66, 0x0c, 0x96, 0x9e, 0x5e, 0x99, 0xa8, 0xdb, 0x44, 0x8d, 0xcb, 0xe4, 0x96, + 0xf9, 0xf4, 0x2e, 0xa5, 0x52, 0xb9, 0xdf, 0x8b, 0xe8, 0xd0, 0xec, 0x25, 0x67, 0x89, 0xa4, 0xf8, + 0x35, 0x42, 0x90, 0x2c, 0x90, 0x54, 0x49, 0xdb, 0xea, 0x16, 0x7b, 0x8d, 0xc1, 0x49, 0x3f, 0xfb, + 0xe4, 0xf5, 0x47, 0x2d, 0x8d, 0xa8, 0x3a, 0x2f, 0xad, 0x7e, 0x75, 0x0a, 0x7e, 0x3d, 0xca, 0xf6, + 0x12, 0x9f, 0xa1, 0xe6, 0x05, 0x8b, 0x39, 0x4b, 0x68, 0xa2, 0xc6, 0x4b, 0x4e, 0xed, 0x83, 0xae, + 0xd5, 0xab, 0xfb, 0xf9, 0x20, 0x7e, 0x89, 0xca, 0x70, 0x60, 0xbb, 0xd8, 0xb5, 0x7a, 0x8d, 0xc1, + 0xb3, 0xfe, 0xde, 0x5d, 0xfa, 0x23, 0xad, 0xc0, 0x61, 0x0c, 0xa4, 0x69, 0x91, 0x46, 0x54, 0xda, + 0xa5, 0x07, 0x68, 0x5f, 0x2b, 0x86, 0x06, 0x08, 0xbf, 0x47, 0x4f, 0x62, 0xaa, 0x44, 0x38, 0x0d, + 0x62, 0xaa, 0xc8, 0x0d, 0x51, 0xc4, 0x2e, 0x83, 0xaf, 0x93, 0xf3, 0x5d, 0x01, 0x73, 0x95, 0x21, + 0x90, 0xe0, 0x28, 0xce, 0xc5, 0xf0, 0x00, 0x55, 0x15, 0x11, 0x33, 0x5d, 0x80, 0x0a, 0x64, 0xb0, + 0x73, 0x19, 0xc6, 0x46, 0x03, 0xeb, 0x16, 0xc4, 0x6f, 0x50, 0x9d, 0x2e, 0x68, 0xcc, 0x23, 0x22, + 0xa4, 0x5d, 0x05, 0x57, 0x3b, 0xe7, 0x1a, 0x6e, 0x55, 0xf0, 0xfd, 0x83, 0xb1, 0x87, 0xca, 0x77, + 0x29, 0x15, 0x4b, 0xbb, 0x06, 0xae, 0x56, 0xce, 0xf5, 0x59, 0x2b, 0xef, 0x3e, 0x5d, 0x9a, 0x8b, + 0x02, 0xe7, 0xfe, 0xb0, 0x50, 0x7d, 0x57, 0x2b, 0xdc, 0x42, 0xb5, 0x38, 0x4c, 0x02, 0x15, 0xc6, + 0xd4, 0xb6, 0xba, 0x56, 0xaf, 0xe8, 0x57, 0xe3, 0x30, 0x19, 0x87, 0x31, 0x05, 0x89, 0x2c, 0x8c, + 0x74, 0x90, 0x49, 0x64, 0x01, 0xd2, 0x0b, 0x74, 0x22, 0x53, 0xce, 0x99, 0x50, 0x32, 0x90, 0x73, + 0x22, 0x6e, 0xc2, 0x64, 0x06, 0x4d, 0xa9, 0xf9, 0xc7, 0x5b, 0x61, 0x94, 0xc5, 0xf1, 0x10, 0x75, + 0x76, 0xf0, 0xd7, 0x50, 0xcd, 0x59, 0xaa, 0x02, 0x41, 0x79, 0x14, 0x4e, 0x49, 0x00, 0x2f, 0x40, + 0x42, 0xa5, 0x6b, 0xfe, 0xf3, 0x2d, 0xf6, 0xc5, 0x50, 0xbe, 0x81, 0xe0, 0xd5, 0xc8, 0x0f, 0xa5, + 0x5a, 0xe9, 0xb8, 0xec, 0x36, 0x50, 0x7d, 0xd7, 0x3a, 0xf7, 0x14, 0xe1, 0xff, 0xfb, 0xa1, 0xdf, + 0xe8, 0x5e, 0x8d, 0xdd, 0x21, 0x6a, 0xe6, 0x8a, 0xf7, 0xb8, 0x2b, 0xbb, 0x47, 0xe8, 0x70, 0xbf, + 0x9a, 0x83, 0x0b, 0x54, 0x82, 0x6c, 0x6f, 0xb3, 0xff, 0x7c, 0x93, 0xf7, 0x86, 0xa4, 0xdd, 0x7a, + 0x40, 0x31, 0xe3, 0x72, 0x7e, 0xb6, 0xfa, 0xe3, 0x14, 0x56, 0x6b, 0xc7, 0xba, 0x5f, 0x3b, 0xd6, + 0xef, 0xb5, 0x63, 0x7d, 0xdb, 0x38, 0x85, 0xfb, 0x8d, 0x53, 0xf8, 0xb9, 0x71, 0x0a, 0xd7, 0x15, + 0x33, 0xbc, 0x93, 0x0a, 0xcc, 0xde, 0xab, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x7f, 0x1d, 0x6e, + 0xa7, 0xd2, 0x03, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -634,15 +636,15 @@ func (m *StoreInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.SendsSortedSeries { + if m.SupportsWithoutReplicaLabels { i-- - if m.SendsSortedSeries { + if m.SupportsWithoutReplicaLabels { dAtA[i] = 1 } else { dAtA[i] = 0 } i-- - dAtA[i] = 0x20 + dAtA[i] = 0x28 } if m.SupportsSharding { i-- @@ -870,7 +872,7 @@ func (m *StoreInfo) Size() (n int) { if m.SupportsSharding { n += 2 } - if m.SendsSortedSeries { + if m.SupportsWithoutReplicaLabels { n += 2 } return n @@ -1402,9 +1404,9 @@ func (m *StoreInfo) Unmarshal(dAtA []byte) error { } } m.SupportsSharding = bool(v != 0) - case 4: + case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SendsSortedSeries", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SupportsWithoutReplicaLabels", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -1421,7 +1423,7 @@ func (m *StoreInfo) Unmarshal(dAtA []byte) error { break } } - m.SendsSortedSeries = bool(v != 0) + m.SupportsWithoutReplicaLabels = bool(v != 0) default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) diff --git a/pkg/info/infopb/rpc.proto b/pkg/info/infopb/rpc.proto index 6ce9caecc2..90cecd5890 100644 --- a/pkg/info/infopb/rpc.proto +++ b/pkg/info/infopb/rpc.proto @@ -53,10 +53,14 @@ message InfoResponse { // StoreInfo holds the metadata related to Store API exposed by the component. message StoreInfo { + reserved 4; // Deprecated send_sorted, replaced by supports_without_replica_labels now. + int64 min_time = 1; int64 max_time = 2; - bool supports_sharding = 3; - bool sends_sorted_series = 4; + bool supports_sharding = 3; + + // replica_aware means this store supports without_replica_labels of StoreAPI.Series. + bool supports_without_replica_labels = 5; } // RulesInfo holds the metadata related to Rules API exposed by the component. diff --git a/pkg/query/endpointset.go b/pkg/query/endpointset.go index 03a0c1cb29..95f01550fb 100644 --- a/pkg/query/endpointset.go +++ b/pkg/query/endpointset.go @@ -185,7 +185,7 @@ type EndpointStatus struct { // endpointSetNodeCollector is a metric collector reporting the number of available storeAPIs for Querier. // A Collector is required as we want atomic updates for all 'thanos_store_nodes_grpc_connections' series. -// TODO(hitanshu-mehta) Currently,only collecting metrics of storeAPI. Make this struct generic. +// TODO(hitanshu-mehta) Currently,only collecting metrics of storeEndpoints. Make this struct generic. type endpointSetNodeCollector struct { mtx sync.Mutex storeNodes map[component.Component]map[string]int @@ -431,7 +431,7 @@ func (e *EndpointSet) Update(ctx context.Context) { if er.HasStoreAPI() && (er.ComponentType() == component.Sidecar || er.ComponentType() == component.Rule) && stats[component.Sidecar][extLset]+stats[component.Rule][extLset] > 0 { - level.Warn(e.logger).Log("msg", "found duplicate storeAPI producer (sidecar or ruler). This is not advices as it will malform data in in the same bucket", + level.Warn(e.logger).Log("msg", "found duplicate storeEndpoints producer (sidecar or ruler). This is not advices as it will malform data in in the same bucket", "address", addr, "extLset", extLset, "duplicates", fmt.Sprintf("%v", stats[component.Sidecar][extLset]+stats[component.Rule][extLset]+1)) } stats[er.ComponentType()][extLset]++ @@ -801,7 +801,7 @@ func (er *endpointRef) SupportsSharding() bool { return er.metadata.Store.SupportsSharding } -func (er *endpointRef) SendsSortedSeries() bool { +func (er *endpointRef) SupportsWithoutReplicaLabels() bool { er.mtx.RLock() defer er.mtx.RUnlock() @@ -809,13 +809,13 @@ func (er *endpointRef) SendsSortedSeries() bool { return false } - return er.metadata.Store.SendsSortedSeries + return er.metadata.Store.SupportsWithoutReplicaLabels } func (er *endpointRef) String() string { mint, maxt := er.TimeRange() return fmt.Sprintf( - "Addr: %s LabelSets: %v Mint: %d Maxt: %d", + "Addr: %s LabelSets: %v MinTime: %d MaxTime: %d", er.addr, labelpb.PromLabelSetsToString(er.LabelSets()), mint, maxt, ) } @@ -832,7 +832,7 @@ func (er *endpointRef) apisPresent() []string { var apisPresent []string if er.HasStoreAPI() { - apisPresent = append(apisPresent, "storeAPI") + apisPresent = append(apisPresent, "storeEndpoints") } if er.HasRulesAPI() { diff --git a/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go b/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go index 112d855eae..8b02a7ca9e 100644 --- a/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go +++ b/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go @@ -215,7 +215,7 @@ func (s *storeRef) SendsSortedSeries() bool { func (s *storeRef) String() string { mint, maxt := s.TimeRange() return fmt.Sprintf( - "Addr: %s LabelSets: %v Mint: %d Maxt: %d", + "Addr: %s LabelSets: %v MinTime: %d MaxTime: %d", s.addr, labelpb.PromLabelSetsToString(s.LabelSets()), mint, maxt, ) } diff --git a/pkg/query/iter.go b/pkg/query/iter.go index 6213d2c031..272113373d 100644 --- a/pkg/query/iter.go +++ b/pkg/query/iter.go @@ -43,8 +43,7 @@ func (s *promSeriesSet) Next() bool { return false } - // storage.Series are more strict then SeriesSet: - // * It requires storage.Series to iterate over full series. + // storage.Series is stricter than storepb.SeriesSet: it requires set to iterate over full series. s.currLset, s.currChunks = s.set.At() for { s.done = s.set.Next() @@ -66,9 +65,11 @@ func (s *promSeriesSet) Next() bool { return s.currChunks[i].MinTime < s.currChunks[j].MinTime }) - // Proxy handles duplicates between different series, let's handle duplicates within single series now as well. + // Proxy handles some exact duplicates in chunk between different series, let's handle duplicates within single series now as well. // We don't need to decode those. s.currChunks = removeExactDuplicates(s.currChunks) + + // TODO(bwplotka): Create dedup iterator for those chunks. return true } diff --git a/pkg/query/querier.go b/pkg/query/querier.go index 9fe4b96eca..e1a443c4a9 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -5,7 +5,6 @@ package query import ( "context" - "sort" "strings" "sync" "time" @@ -19,12 +18,9 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" promgate "github.com/prometheus/prometheus/util/gate" - - "github.com/thanos-io/thanos/pkg/dedup" "github.com/thanos-io/thanos/pkg/extprom" "github.com/thanos-io/thanos/pkg/gate" "github.com/thanos-io/thanos/pkg/store" - "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/tracing" ) @@ -61,10 +57,11 @@ type QueryableCreator func( ) storage.Queryable // NewQueryableCreator creates QueryableCreator. +// NOTE(bwplotka): Proxy assumes to be replica_aware, see thanos.store.info.StoreInfo.replica_aware field. func NewQueryableCreator( logger log.Logger, reg prometheus.Registerer, - proxy storepb.StoreServer, + proxy *store.ProxyStore, maxConcurrentSelects int, selectTimeout time.Duration, ) QueryableCreator { @@ -108,7 +105,7 @@ type queryable struct { logger log.Logger replicaLabels []string storeDebugMatchers [][]*labels.Matcher - proxy storepb.StoreServer + proxy *store.ProxyStore deduplicate bool maxResolutionMillis int64 partialResponse bool @@ -127,22 +124,22 @@ func (q *queryable) Querier(ctx context.Context, mint, maxt int64) (storage.Quer } type querier struct { - ctx context.Context - logger log.Logger - cancel func() - mint, maxt int64 - replicaLabels map[string]struct{} - storeDebugMatchers [][]*labels.Matcher - proxy storepb.StoreServer - deduplicate bool - maxResolutionMillis int64 - partialResponse bool - enableQueryPushdown bool - skipChunks bool - selectGate gate.Gate - selectTimeout time.Duration - shardInfo *storepb.ShardInfo - seriesStatsReporter seriesStatsReporter + ctx context.Context + logger log.Logger + cancel func() + mint, maxt int64 + replicaLabels []string + storeDebugMatchers [][]*labels.Matcher + proxy storepb.StoreServer + deduplicate bool + maxResolutionMillis int64 + partialResponseStrategy storepb.PartialResponseStrategy + enableQueryPushdown bool + skipChunks bool + selectGate gate.Gate + selectTimeout time.Duration + shardInfo *storepb.ShardInfo + seriesStatsReporter seriesStatsReporter } // newQuerier creates implementation of storage.Querier that fetches data from the proxy @@ -174,6 +171,11 @@ func newQuerier( for _, replicaLabel := range replicaLabels { rl[replicaLabel] = struct{}{} } + + partialResponseStrategy := storepb.PartialResponseStrategy_ABORT + if partialResponse { + partialResponseStrategy = storepb.PartialResponseStrategy_WARN + } return &querier{ ctx: ctx, logger: logger, @@ -181,18 +183,18 @@ func newQuerier( selectGate: selectGate, selectTimeout: selectTimeout, - mint: mint, - maxt: maxt, - replicaLabels: rl, - storeDebugMatchers: storeDebugMatchers, - proxy: proxy, - deduplicate: deduplicate, - maxResolutionMillis: maxResolutionMillis, - partialResponse: partialResponse, - skipChunks: skipChunks, - enableQueryPushdown: enableQueryPushdown, - shardInfo: shardInfo, - seriesStatsReporter: seriesStatsReporter, + mint: mint, + maxt: maxt, + replicaLabels: replicaLabels, + storeDebugMatchers: storeDebugMatchers, + proxy: proxy, + deduplicate: deduplicate, + maxResolutionMillis: maxResolutionMillis, + partialResponseStrategy: partialResponseStrategy, + skipChunks: skipChunks, + enableQueryPushdown: enableQueryPushdown, + shardInfo: shardInfo, + seriesStatsReporter: seriesStatsReporter, } } @@ -280,8 +282,9 @@ func (q *querier) Select(_ bool, hints *storage.SelectHints, ms ...*labels.Match matchers[i] = m.String() } - // The querier has a context but it gets canceled, as soon as query evaluation is completed, by the engine. + // The querier has a context, but it gets canceled as soon as query evaluation is completed by the engine. // We want to prevent this from happening for the async store API calls we make while preserving tracing context. + // TODO(bwplotka): Does the above still is true? It feels weird to leave unfinished calls behind query API. ctx := tracing.CopyTraceContext(context.Background(), q.ctx) ctx, cancel := context.WithTimeout(ctx, q.selectTimeout) span, ctx := tracing.StartSpan(ctx, "querier_select", opentracing.Tags{ @@ -341,26 +344,29 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . // TODO(bwplotka): Pass it using the SeriesRequest instead of relying on context. ctx = context.WithValue(ctx, store.StoreMatcherKey, q.storeDebugMatchers) - // TODO(bwplotka): Use inprocess gRPC. + // TODO(bwplotka): Use inprocess gRPC when we want to stream responses. + // Currently streaming won't help due to nature of the both PromQL engine which + // pulls all series before computations anyway. resp := &seriesServer{ctx: ctx} - var queryHints *storepb.QueryHints - if q.enableQueryPushdown { - queryHints = storeHintsFromPromHints(hints) - } - - if err := q.proxy.Series(&storepb.SeriesRequest{ - MinTime: q.mint, - MaxTime: q.maxt, + req := storepb.SeriesRequest{ + MinTime: hints.Start, + MaxTime: hints.End, Matchers: sms, MaxResolutionWindow: q.maxResolutionMillis, Aggregates: aggrs, - QueryHints: queryHints, ShardInfo: q.shardInfo, - PartialResponseDisabled: !q.partialResponse, + PartialResponseStrategy: q.partialResponseStrategy, SkipChunks: q.skipChunks, - Step: hints.Step, - Range: hints.Range, - }, resp); err != nil { + } + if q.enableQueryPushdown { + req.QueryHints = storeHintsFromPromHints(hints) + } + if q.isDedupEnabled() { + // Soft ask to sort without replica labels and push them at the end of labelset. + req.WithoutReplicaLabels = q.replicaLabels + } + + if err := q.proxy.Series(&req, resp); err != nil { return nil, storepb.SeriesStatsCounter{}, errors.Wrap(err, "proxy Series()") } @@ -369,10 +375,10 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . warns = append(warns, errors.New(w)) } - // Delete the metric's name from the result because that's what the - // PromQL does either way and we want our iterator to work with data - // that was either pushed down or not. if q.enableQueryPushdown && (hints.Func == "max_over_time" || hints.Func == "min_over_time") { + // On query pushdown, delete the metric's name from the result because that's what the + // PromQL does either way, and we want our iterator to work with data + // that was either pushed down or not. for i := range resp.seriesSet { lbls := resp.seriesSet[i].Labels for j, lbl := range lbls { @@ -385,19 +391,7 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . } } - if !q.isDedupEnabled() { - // Return data without any deduplication. - return &promSeriesSet{ - mint: q.mint, - maxt: q.maxt, - set: newStoreSeriesSet(resp.seriesSet), - aggrs: aggrs, - warns: warns, - }, resp.seriesSetStats, nil - } - - // TODO(fabxc): this could potentially pushed further down into the store API to make true streaming possible. - sortDedupLabels(resp.seriesSet, q.replicaLabels) + // promSeriesSet is deduplicating all overlapped chunks. set := &promSeriesSet{ mint: q.mint, maxt: q.maxt, @@ -406,38 +400,10 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . warns: warns, } - // The merged series set assembles all potentially-overlapping time ranges of the same series into a single one. - // TODO(bwplotka): We could potentially dedup on chunk level, use chunk iterator for that when available. - return dedup.NewSeriesSet(set, q.replicaLabels, hints.Func, q.enableQueryPushdown), resp.seriesSetStats, nil -} - -// sortDedupLabels re-sorts the set so that the same series with different replica -// labels are coming right after each other. -func sortDedupLabels(set []storepb.Series, replicaLabels map[string]struct{}) { - for _, s := range set { - // Move the replica labels to the very end. - sort.Slice(s.Labels, func(i, j int) bool { - if _, ok := replicaLabels[s.Labels[i].Name]; ok { - return false - } - if _, ok := replicaLabels[s.Labels[j].Name]; ok { - return true - } - // Ensure that dedup marker goes just right before the replica labels. - if s.Labels[i].Name == dedup.PushdownMarker.Name { - return false - } - if s.Labels[j].Name == dedup.PushdownMarker.Name { - return true - } - return s.Labels[i].Name < s.Labels[j].Name - }) - } - // With the re-ordered label sets, re-sorting all series aligns the same series - // from different replicas sequentially. - sort.Slice(set, func(i, j int) bool { - return labels.Compare(labelpb.ZLabelsToPromLabels(set[i].Labels), labelpb.ZLabelsToPromLabels(set[j].Labels)) < 0 - }) + // // The merged series set assembles all potentially-overlapping time ranges of the same series into a single one. + //- // TODO(bwplotka): We could potentially dedup on chunk level, use chunk iterator for that when available. + //- return dedup.NewSeriesSet(set, q.replicaLabels, hints.Func, q.enableQueryPushdown), resp.seriesSetStats, nil + return set, resp.seriesSetStats, nil } // LabelValues returns all potential values for a label name. @@ -455,7 +421,7 @@ func (q *querier) LabelValues(name string, matchers ...*labels.Matcher) ([]strin resp, err := q.proxy.LabelValues(ctx, &storepb.LabelValuesRequest{ Label: name, - PartialResponseDisabled: !q.partialResponse, + PartialResponseStrategy: q.partialResponseStrategy, Start: q.mint, End: q.maxt, Matchers: pbMatchers, @@ -487,7 +453,7 @@ func (q *querier) LabelNames(matchers ...*labels.Matcher) ([]string, storage.War } resp, err := q.proxy.LabelNames(ctx, &storepb.LabelNamesRequest{ - PartialResponseDisabled: !q.partialResponse, + PartialResponseStrategy: q.partialResponseStrategy, Start: q.mint, End: q.maxt, Matchers: pbMatchers, diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 2789d72cce..6cf52efdac 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -12,10 +12,13 @@ import ( "reflect" "sort" "strconv" + "strings" "testing" "time" + "github.com/efficientgo/core/testutil" "github.com/go-kit/log" + "github.com/google/go-cmp/cmp" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" @@ -26,10 +29,10 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/gate" + "github.com/thanos-io/thanos/pkg/testutil/teststore" "github.com/efficientgo/core/testutil" "github.com/thanos-io/thanos/pkg/component" - "github.com/thanos-io/thanos/pkg/dedup" "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" @@ -42,7 +45,7 @@ type sample struct { func TestQueryableCreator_MaxResolution(t *testing.T) { testProxy := &testStoreServer{resps: []*storepb.SeriesResponse{}} - queryableCreator := NewQueryableCreator(nil, nil, testProxy, 2, 5*time.Second) + queryableCreator := NewQueryableCreator(nil, nil, newProxyStore(testProxy), 2, 5*time.Second) oneHourMillis := int64(1*time.Hour) / int64(time.Millisecond) queryable := queryableCreator( @@ -85,7 +88,7 @@ func TestQuerier_DownsampledData(t *testing.T) { q := NewQueryableCreator( nil, nil, - testProxy, + newProxyStore(testProxy), 2, timeout, )(false, @@ -424,8 +427,8 @@ func TestQuerier_Select(t *testing.T) { logger := log.NewLogfmtLogger(os.Stderr) for _, tcase := range []struct { - name string - storeAPI storepb.StoreServer + name string + storeEndpoints []storepb.StoreServer mint, maxt int64 matchers []*labels.Matcher @@ -434,29 +437,35 @@ func TestQuerier_Select(t *testing.T) { equivalentQuery string expected []series - expectedAfterDedup series + expectedAfterDedup []series expectedWarning string }{ { name: "select overlapping data with partial error", - storeAPI: &testStoreServer{ - resps: []*storepb.SeriesResponse{ - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), - storepb.NewWarnSeriesResponse(errors.New("partial error")), - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 6}, {7, 7}}), - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 66}}), // Overlap samples for some reason. - storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), - storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), + storeEndpoints: []storepb.StoreServer{ + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storepb.NewWarnSeriesResponse(errors.New("partial error")), + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 66}}), // Overlap samples for some reason. In this case the choice of value is random. + storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), + }, }, }, mint: 1, maxt: 300, replicaLabels: []string{"a"}, equivalentQuery: `{a=~"a|b|c"}`, - + matchers: []*labels.Matcher{{ + Value: "a|b|c", + Name: "a", + Type: labels.MatchRegexp, + }}, expected: []series{ { lset: labels.FromStrings("a", "a"), - samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 66}, {7, 7}}, }, { lset: labels.FromStrings("a", "b"), @@ -467,20 +476,20 @@ func TestQuerier_Select(t *testing.T) { samples: []sample{{100, 1}, {300, 3}}, }, }, - expectedAfterDedup: series{ - lset: labels.Labels{}, + expectedAfterDedup: []series{{ + lset: nil, // We don't expect correctness here, it's just random non-replica data. - samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}}, - }, + samples: []sample{{2, 1}, {3, 2}, {4, 4}, {5, 5}, {6, 66}, {7, 7}, {100, 1}, {t: 300, v: 3}}, + }}, expectedWarning: "partial error", }, { name: "realistic data with stale marker", - storeAPI: func() storepb.StoreServer { + storeEndpoints: []storepb.StoreServer{func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2401-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()}, mint: realSeriesWithStaleMarkerMint, maxt: realSeriesWithStaleMarkerMaxt, replicaLabels: []string{"replica"}, matchers: []*labels.Matcher{{ @@ -508,7 +517,7 @@ func TestQuerier_Select(t *testing.T) { samples: expectedRealSeriesWithStaleMarkerReplica1, }, }, - expectedAfterDedup: series{ + expectedAfterDedup: []series{{ lset: labels.FromStrings( // No replica label anymore. "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", @@ -516,15 +525,15 @@ func TestQuerier_Select(t *testing.T) { "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", ), samples: expectedRealSeriesWithStaleMarkerDeduplicated, - }, + }}, }, { name: "realistic data with stale marker with 100000 step", - storeAPI: func() storepb.StoreServer { + storeEndpoints: []storepb.StoreServer{func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2401-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()}, mint: realSeriesWithStaleMarkerMint, maxt: realSeriesWithStaleMarkerMaxt, replicaLabels: []string{"replica"}, matchers: []*labels.Matcher{{ @@ -557,7 +566,7 @@ func TestQuerier_Select(t *testing.T) { samples: expectedRealSeriesWithStaleMarkerReplica1, }, }, - expectedAfterDedup: series{ + expectedAfterDedup: []series{{ lset: labels.FromStrings( // No replica label anymore. "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", @@ -565,17 +574,17 @@ func TestQuerier_Select(t *testing.T) { "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", ), samples: expectedRealSeriesWithStaleMarkerDeduplicated, - }, + }}, }, { // Regression test against https://github.com/thanos-io/thanos/issues/2401. // Thanks to @Superq and GitLab for real data reproducing this. name: "realistic data with stale marker with hints rate function", - storeAPI: func() storepb.StoreServer { + storeEndpoints: []storepb.StoreServer{func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2401-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()}, mint: realSeriesWithStaleMarkerMint, maxt: realSeriesWithStaleMarkerMaxt, replicaLabels: []string{"replica"}, matchers: []*labels.Matcher{{ @@ -609,13 +618,141 @@ func TestQuerier_Select(t *testing.T) { samples: expectedRealSeriesWithStaleMarkerReplica1ForRate, }, }, - expectedAfterDedup: series{ + expectedAfterDedup: []series{{ lset: labels.FromStrings( "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", "gprd", "fqdn", "web-08-sv-gprd.c.gitlab-production.internal", "instance", "web-08-sv-gprd.c.gitlab-production.internal:8083", "job", "gitlab-rails", "monitor", "app", "provider", "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", ), samples: expectedRealSeriesWithStaleMarkerDeduplicatedForRate, + }}, + }, + // Tests with proxy (integration test with store.ProxyStore). + { + name: "select with proxied Store APIs that does not support without replica label", + storeEndpoints: []storepb.StoreServer{ + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {7, 7}}), + }, + }, + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{2, 1}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "x", "2"), []sample{{10, 10}, {30, 30}, {40, 40}}), + }, + }, + }, + mint: 1, maxt: 300, + replicaLabels: []string{"r"}, + equivalentQuery: `{a=~"1"}`, + matchers: []*labels.Matcher{{Name: "a", Value: "1", Type: labels.MatchRegexp}}, + expected: []series{ + { + lset: labels.FromStrings("a", "1", "r", "1", "w", "1"), + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "1", "x", "1"), + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "w", "1"), + samples: []sample{{2, 1}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "x", "2"), + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, + }, + expectedAfterDedup: []series{ + { + lset: labels.FromStrings("a", "1", "w", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "2"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, + }, + }, + { + name: "select with proxied Store APIs with some stores supporting without replica labels feature", + storeEndpoints: []storepb.StoreServer{ + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {7, 7}}), + }, + respsWithoutReplicaLabels: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + }, + }, + &testStoreServer{ + resps: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{2, 1}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "r", "2", "x", "2"), []sample{{10, 10}, {30, 30}, {40, 40}}), + }, + }, + }, + mint: 1, maxt: 300, + replicaLabels: []string{"r"}, + equivalentQuery: `{a=~"1"}`, + matchers: []*labels.Matcher{{Name: "a", Value: "1", Type: labels.MatchRegexp}}, + expected: []series{ + { + lset: labels.FromStrings("a", "1", "r", "1", "w", "1"), + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "1", "x", "1"), + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "w", "1"), + samples: []sample{{2, 1}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "r", "2", "x", "2"), + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, + }, + expectedAfterDedup: []series{ + { + lset: labels.FromStrings("a", "1", "w", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{2, 1}, {3, 2}, {5, 5}, {6, 6}, {7, 7}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "1"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "2"), + // We don't expect correctness here, it's just random non-replica data. + samples: []sample{{10, 10}, {30, 30}, {40, 40}}, + }, }, }, } { @@ -632,10 +769,27 @@ func TestQuerier_Select(t *testing.T) { expected []series }{ {dedup: false, expected: tcase.expected}, - {dedup: true, expected: []series{tcase.expectedAfterDedup}}, + {dedup: true, expected: tcase.expectedAfterDedup}, } { g := gate.New(2) - q := newQuerier(context.Background(), nil, tcase.mint, tcase.maxt, tcase.replicaLabels, nil, tcase.storeAPI, sc.dedup, 0, true, false, false, g, timeout, nil, func(i storepb.SeriesStatsCounter) {}) + q := newQuerier( + context.Background(), + nil, + tcase.mint, + tcase.maxt, + tcase.replicaLabels, + nil, + newProxyStore(tcase.storeEndpoints...), + sc.dedup, + 0, + true, + false, + false, + g, + timeout, + nil, + func(i storepb.SeriesStatsCounter) {}, + ) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) t.Run(fmt.Sprintf("dedup=%v", sc.dedup), func(t *testing.T) { @@ -678,6 +832,39 @@ func TestQuerier_Select(t *testing.T) { } } +func newProxyStore(storeAPIs ...storepb.StoreServer) *store.ProxyStore { + cls := make([]store.Client, len(storeAPIs)) + for i, s := range storeAPIs { + var withoutReplicaLabelsEnabled bool + if srv, ok := s.(*testStoreServer); ok { + withoutReplicaLabelsEnabled = len(srv.respsWithoutReplicaLabels) > 0 + } + cls[i] = &teststore.TestClient{ + Name: fmt.Sprintf("%v", i), + StoreClient: storepb.ServerAsClient(s, 0), + MinTime: math.MinInt64, MaxTime: math.MaxInt64, + WithoutReplicaLabelsEnabled: withoutReplicaLabelsEnabled, + } + } + + return store.NewProxyStore( + nil, + nil, + func() []store.Client { return cls }, + component.Query, + nil, + 0, + store.EagerRetrieval, + ) +} + +var emptyLabelsSameAsNotAllocatedLabels = cmp.Transformer("", func(l labels.Labels) labels.Labels { + if len(l) == 0 { + return labels.Labels(nil) + } + return l +}) + func testSelectResponse(t *testing.T, expected []series, res storage.SeriesSet) { var series []storage.Series // Use it as PromQL would do, first gather all series. @@ -685,10 +872,16 @@ func testSelectResponse(t *testing.T, expected []series, res storage.SeriesSet) series = append(series, res.At()) } testutil.Ok(t, res.Err()) - testutil.Equals(t, len(expected), len(series)) + testutil.Equals(t, len(expected), len(series), "got %v", func() string { + var ret []string + for _, s := range series { + ret = append(ret, s.Labels().String()) + } + return strings.Join(ret, ",") + }()) for i, s := range series { - testutil.Equals(t, expected[i].lset, s.Labels()) + testutil.WithGoCmp(emptyLabelsSameAsNotAllocatedLabels).Equals(t, expected[i].lset, s.Labels()) samples := expandSeries(t, s.Iterator()) expectedCpy := make([]sample, 0, len(expected[i].samples)) for _, s := range expected[i].samples { @@ -700,7 +893,7 @@ func testSelectResponse(t *testing.T, expected []series, res storage.SeriesSet) } expectedCpy = append(expectedCpy, sample{t: s.t, v: v}) } - testutil.Equals(t, expectedCpy, samples, "samples for series %v does not match", i) + testutil.Equals(t, expectedCpy, samples, "samples for series %v does not match", s.Labels()) } } @@ -1013,71 +1206,6 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { }) } -func TestSortReplicaLabel(t *testing.T) { - tests := []struct { - input []storepb.Series - exp []storepb.Series - dedupLabels map[string]struct{} - }{ - // 0 Single deduplication label. - { - input: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - }, - exp: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "b", Value: "replica-1"}}}, - }, - dedupLabels: map[string]struct{}{"b": {}}, - }, - // 1 Multi deduplication labels. - { - input: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}, {Name: "c", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "b1", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - }, - exp: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}, {Name: "b1", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "b", Value: "replica-1"}, {Name: "b1", Value: "replica-1"}}}, - }, - dedupLabels: map[string]struct{}{"b": {}, "b1": {}}, - }, - // Pushdown label at the end. - { - input: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-1"}, {Name: "c", Value: "4"}, {Name: dedup.PushdownMarker.Name, Value: dedup.PushdownMarker.Value}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "b", Value: "replica-2"}, {Name: "c", Value: "3"}}}, - }, - exp: []storepb.Series{ - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "b", Value: "replica-2"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}, {Name: "b", Value: "replica-1"}}}, - {Labels: []labelpb.ZLabel{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: dedup.PushdownMarker.Name, Value: dedup.PushdownMarker.Value}, {Name: "b", Value: "replica-1"}}}, - }, - dedupLabels: map[string]struct{}{"b": {}}, - }, - } - for _, test := range tests { - t.Run("", func(t *testing.T) { - sortDedupLabels(test.input, test.dedupLabels) - testutil.Equals(t, test.exp, test.input) - }) - } -} - const hackyStaleMarker = float64(-99999999) func expandSeries(t testing.TB, it chunkenc.Iterator) (res []sample) { @@ -1098,11 +1226,18 @@ type testStoreServer struct { // This field just exist to pseudo-implement the unused methods of the interface. storepb.StoreServer - resps []*storepb.SeriesResponse + resps []*storepb.SeriesResponse + respsWithoutReplicaLabels []*storepb.SeriesResponse } -func (s *testStoreServer) Series(_ *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { - for _, resp := range s.resps { +func (s *testStoreServer) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { + resps := s.resps + + if len(r.WithoutReplicaLabels) > 0 && len(s.respsWithoutReplicaLabels) > 0 { + // If `respsWithoutReplicaLabels` is present, we simulate server that supports without replica label feature. + resps = s.respsWithoutReplicaLabels + } + for _, resp := range resps { err := srv.Send(resp) if err != nil { return err diff --git a/pkg/query/query_bench_test.go b/pkg/query/query_bench_test.go index 1f29e12f22..178c929cf6 100644 --- a/pkg/query/query_bench_test.go +++ b/pkg/query/query_bench_test.go @@ -84,7 +84,7 @@ func benchQuerySelect(t testutil.TB, totalSamples, totalSeries int, dedup bool) ctx: context.Background(), logger: logger, proxy: &mockedStoreServer{responses: resps}, - replicaLabels: map[string]struct{}{"a_replica": {}}, + replicaLabels: []string{"a_replica"}, deduplicate: dedup, selectGate: gate.NewNoop(), seriesStatsReporter: NoopSeriesStatsReporter, diff --git a/pkg/query/query_test.go b/pkg/query/query_test.go index 6c68fc5e1a..5ff5393524 100644 --- a/pkg/query/query_test.go +++ b/pkg/query/query_test.go @@ -13,6 +13,7 @@ import ( "github.com/go-kit/log" "github.com/prometheus/prometheus/storage" + "github.com/thanos-io/thanos/pkg/testutil/teststore" "github.com/efficientgo/core/testutil" "github.com/thanos-io/thanos/pkg/component" @@ -49,10 +50,11 @@ func TestQuerier_Proxy(t *testing.T) { testutil.Ok(t, err) // TODO(bwplotka): Parse external labels. - clients = append(clients, inProcessClient{ - t: t, - StoreClient: storepb.ServerAsClient(SelectedStore(store.NewTSDBStore(logger, st.storage.DB, component.Debug, nil), m, st.mint, st.maxt), 0), - name: fmt.Sprintf("store number %v", i), + clients = append(clients, &teststore.TestClient{ + Name: fmt.Sprintf("store number %v", i), + StoreClient: storepb.ServerAsClient(selectedStore(store.NewTSDBStore(logger, st.storage.DB, component.Debug, nil), m, st.mint, st.maxt), 0), + MinTime: st.mint, + MaxTime: st.maxt, }) } return q(true, @@ -78,17 +80,17 @@ func TestQuerier_Proxy(t *testing.T) { }) } -// SelectStore allows wrapping another storeAPI with additional time and matcher selection. -type SelectStore struct { +// selectStore allows wrapping another storeEndpoints with additional time and matcher selection. +type selectStore struct { matchers []storepb.LabelMatcher storepb.StoreServer mint, maxt int64 } -// SelectedStore wraps given store with SelectStore. -func SelectedStore(wrapped storepb.StoreServer, matchers []storepb.LabelMatcher, mint, maxt int64) *SelectStore { - return &SelectStore{ +// selectedStore wraps given store with selectStore. +func selectedStore(wrapped storepb.StoreServer, matchers []storepb.LabelMatcher, mint, maxt int64) *selectStore { + return &selectStore{ StoreServer: wrapped, matchers: matchers, mint: mint, @@ -96,7 +98,7 @@ func SelectedStore(wrapped storepb.StoreServer, matchers []storepb.LabelMatcher, } } -func (s *SelectStore) Info(ctx context.Context, r *storepb.InfoRequest) (*storepb.InfoResponse, error) { +func (s *selectStore) Info(ctx context.Context, r *storepb.InfoRequest) (*storepb.InfoResponse, error) { resp, err := s.StoreServer.Info(ctx, r) if err != nil { return nil, err @@ -111,7 +113,7 @@ func (s *SelectStore) Info(ctx context.Context, r *storepb.InfoRequest) (*storep return resp, nil } -func (s *SelectStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { +func (s *selectStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesServer) error { if r.MinTime < s.mint { r.MinTime = s.mint } diff --git a/pkg/query/test.go b/pkg/query/test_test.go similarity index 93% rename from pkg/query/test.go rename to pkg/query/test_test.go index a4ccf14934..a5500b3b9c 100644 --- a/pkg/query/test.go +++ b/pkg/query/test_test.go @@ -15,6 +15,7 @@ import ( "testing" "time" + "github.com/efficientgo/core/testutil" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -22,11 +23,6 @@ import ( "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/util/teststorage" - - "github.com/efficientgo/core/testutil" - "github.com/thanos-io/thanos/pkg/store" - "github.com/thanos-io/thanos/pkg/store/storepb" ) var ( @@ -624,39 +620,3 @@ type clearCmd struct{} func (cmd clearCmd) String() string { return "clear" } - -type inProcessClient struct { - t testing.TB - - name string - - storepb.StoreClient - extLset labels.Labels -} - -func NewInProcessClient(t testing.TB, name string, client storepb.StoreClient, extLset labels.Labels) store.Client { - return inProcessClient{ - t: t, - name: name, - StoreClient: client, - extLset: extLset, - } -} - -func (i inProcessClient) LabelSets() []labels.Labels { - return []labels.Labels{i.extLset} -} - -func (i inProcessClient) TimeRange() (mint, maxt int64) { - r, err := i.Info(context.TODO(), &storepb.InfoRequest{}) - testutil.Ok(i.t, err) - return r.MinTime, r.MaxTime -} - -func (i inProcessClient) SupportsSharding() bool { - return false -} - -func (i inProcessClient) SendsSortedSeries() bool { return false } -func (i inProcessClient) String() string { return i.name } -func (i inProcessClient) Addr() (string, bool) { return i.name, true } diff --git a/pkg/receive/multitsdb.go b/pkg/receive/multitsdb.go index 878e894e5d..aa94a2ad39 100644 --- a/pkg/receive/multitsdb.go +++ b/pkg/receive/multitsdb.go @@ -116,7 +116,7 @@ func (l *localClient) TimeRange() (mint int64, maxt int64) { func (l *localClient) String() string { mint, maxt := l.timeRangeFunc() return fmt.Sprintf( - "LabelSets: %v Mint: %d Maxt: %d", + "LabelSets: %v MinTime: %d MaxTime: %d", labelpb.PromLabelSetsToString(l.LabelSets()), mint, maxt, ) } @@ -129,7 +129,7 @@ func (l *localClient) SupportsSharding() bool { return true } -func (l *localClient) SendsSortedSeries() bool { +func (l *localClient) SupportsWithoutReplicaLabels() bool { return true } diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index b1fbd898a2..2ecd9d161b 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -849,16 +849,22 @@ func newBlockSeriesClient( calculateChunkHash bool, batchSize int, chunkFetchDuration prometheus.Histogram, + extLsetToRemove map[string]struct{}, ) *blockSeriesClient { var chunkr *bucketChunkReader if !req.SkipChunks { chunkr = b.chunkReader() } + extLset := b.extLset + if extLsetToRemove != nil { + extLset = rmLabels(extLset.Copy(), extLsetToRemove) + } + return &blockSeriesClient{ ctx: ctx, logger: logger, - extLset: b.extLset, + extLset: extLset, mint: req.MinTime, maxt: req.MaxTime, indexr: b.indexReader(), @@ -1185,6 +1191,14 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie } } + var extLsetToRemove map[string]struct{} + if len(req.WithoutReplicaLabels) > 0 { + extLsetToRemove = make(map[string]struct{}) + for _, l := range req.WithoutReplicaLabels { + extLsetToRemove[l] = struct{}{} + } + } + s.mtx.RLock() for _, bs := range s.blockSets { blockMatchers, ok := bs.labelMatchers(matchers...) @@ -1219,6 +1233,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie s.enableChunkHashCalculation, s.seriesBatchSize, s.metrics.chunkFetchDuration, + extLsetToRemove, ) defer blockClient.Close() @@ -1463,7 +1478,19 @@ func (s *BucketStore) LabelNames(ctx context.Context, req *storepb.LabelNamesReq MaxTime: req.End, SkipChunks: true, } - blockClient := newBlockSeriesClient(newCtx, s.logger, b, seriesReq, nil, bytesLimiter, nil, true, SeriesBatchSize, s.metrics.chunkFetchDuration) + blockClient := newBlockSeriesClient( + newCtx, + s.logger, + b, + seriesReq, + nil, + bytesLimiter, + nil, + true, + SeriesBatchSize, + s.metrics.chunkFetchDuration, + nil, + ) if err := blockClient.ExpandPostings( reqSeriesMatchersNoExtLabels, @@ -1637,7 +1664,19 @@ func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesR MaxTime: req.End, SkipChunks: true, } - blockClient := newBlockSeriesClient(newCtx, s.logger, b, seriesReq, nil, bytesLimiter, nil, true, SeriesBatchSize, s.metrics.chunkFetchDuration) + blockClient := newBlockSeriesClient( + newCtx, + s.logger, + b, + seriesReq, + nil, + bytesLimiter, + nil, + true, + SeriesBatchSize, + s.metrics.chunkFetchDuration, + nil, + ) if err := blockClient.ExpandPostings( reqSeriesMatchersNoExtLabels, diff --git a/pkg/store/bucket_e2e_test.go b/pkg/store/bucket_e2e_test.go index 3588c5f061..89c9deccfc 100644 --- a/pkg/store/bucket_e2e_test.go +++ b/pkg/store/bucket_e2e_test.go @@ -243,7 +243,6 @@ func gatherFamily(t testing.TB, reg prometheus.Gatherer, familyName string) *dto return nil } -// TODO(bwplotka): Benchmark Series. func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite) { t.Helper() @@ -285,6 +284,27 @@ func testBucketStore_e2e(t *testing.T, ctx context.Context, s *storeSuite) { {{Name: "a", Value: "2"}, {Name: "c", Value: "2"}, {Name: "ext2", Value: "value2"}}, }, }, + { + req: &storepb.SeriesRequest{ + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: "a", Value: "1|2"}, + }, + MinTime: mint, + MaxTime: maxt, + WithoutReplicaLabels: []string{"ext1", "ext2"}, + }, + expectedChunkLen: 3, + expected: [][]labelpb.ZLabel{ + {{Name: "a", Value: "1"}, {Name: "b", Value: "1"}}, + {{Name: "a", Value: "1"}, {Name: "b", Value: "2"}}, + {{Name: "a", Value: "1"}, {Name: "c", Value: "1"}}, + {{Name: "a", Value: "1"}, {Name: "c", Value: "2"}}, + {{Name: "a", Value: "2"}, {Name: "b", Value: "1"}}, + {{Name: "a", Value: "2"}, {Name: "b", Value: "2"}}, + {{Name: "a", Value: "2"}, {Name: "c", Value: "1"}}, + {{Name: "a", Value: "2"}, {Name: "c", Value: "2"}}, + }, + }, { req: &storepb.SeriesRequest{ Matchers: []storepb.LabelMatcher{ diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index c115be4d6b..a3b42b963a 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -2141,7 +2141,7 @@ func TestLabelNamesAndValuesHints(t *testing.T) { } } -func TestSeries_ChuncksHaveHashRepresentation(t *testing.T) { +func TestSeries_ChunksHaveHashRepresentation(t *testing.T) { tb := testutil.NewTB(t) tmpDir := t.TempDir() @@ -2441,7 +2441,19 @@ func benchmarkBlockSeriesWithConcurrency(b *testing.B, concurrency int, blockMet testutil.Ok(b, err) dummyHistogram := prometheus.NewHistogram(prometheus.HistogramOpts{}) - blockClient := newBlockSeriesClient(ctx, nil, blk, req, chunksLimiter, NewBytesLimiterFactory(0)(nil), nil, false, SeriesBatchSize, dummyHistogram) + blockClient := newBlockSeriesClient( + ctx, + nil, + blk, + req, + chunksLimiter, + NewBytesLimiterFactory(0)(nil), + nil, + false, + SeriesBatchSize, + dummyHistogram, + nil, + ) testutil.Ok(b, blockClient.ExpandPostings(matchers, seriesLimiter)) defer blockClient.Close() diff --git a/pkg/store/prometheus_test.go b/pkg/store/prometheus_test.go index f77504eb1e..eb9d80ed15 100644 --- a/pkg/store/prometheus_test.go +++ b/pkg/store/prometheus_test.go @@ -67,7 +67,7 @@ func testPrometheusStoreSeriesE2e(t *testing.T, prefix string) { limitMinT := int64(0) proxy, err := NewPrometheusStore(nil, nil, promclient.NewDefaultClient(), u, component.Sidecar, func() labels.Labels { return labels.FromStrings("region", "eu-west") }, - func() (int64, int64) { return limitMinT, -1 }, nil) // Maxt does not matter. + func() (int64, int64) { return limitMinT, -1 }, nil) // MaxTime does not matter. testutil.Ok(t, err) // Query all three samples except for the first one. Since we round up queried data diff --git a/pkg/store/proxy.go b/pkg/store/proxy.go index d9245035f4..bed8ed11f3 100644 --- a/pkg/store/proxy.go +++ b/pkg/store/proxy.go @@ -49,11 +49,9 @@ type Client interface { // SupportsSharding returns true if sharding is supported by the underlying store. SupportsSharding() bool - // SendsSortedSeries returns true if the underlying store sends series sorded by - // their labels. - // The field can be used to indicate to the querier whether it needs to sort - // received series before deduplication. - SendsSortedSeries() bool + // SupportsWithoutReplicaLabels returns true if trimming replica labels + // and sorted response is supported by the underlying store. + SupportsWithoutReplicaLabels() bool // String returns the string representation of the store client. String() string @@ -264,6 +262,7 @@ func (s *ProxyStore) Series(originalRequest *storepb.SeriesRequest, srv storepb. PartialResponseDisabled: originalRequest.PartialResponseDisabled, PartialResponseStrategy: originalRequest.PartialResponseStrategy, ShardInfo: originalRequest.ShardInfo, + WithoutReplicaLabels: originalRequest.WithoutReplicaLabels, } stores := []Client{} @@ -294,7 +293,7 @@ func (s *ProxyStore) Series(originalRequest *storepb.SeriesRequest, srv storepb. storeDebugMsgs = append(storeDebugMsgs, fmt.Sprintf("store %s queried", st)) - respSet, err := newAsyncRespSet(srv.Context(), st, r, s.responseTimeout, s.retrievalStrategy, st.SupportsSharding(), &s.buffers, r.ShardInfo, reqLogger, s.metrics.emptyStreamResponses) + respSet, err := newAsyncRespSet(srv.Context(), st, r, s.responseTimeout, s.retrievalStrategy, &s.buffers, r.ShardInfo, reqLogger, s.metrics.emptyStreamResponses) if err != nil { level.Error(reqLogger).Log("err", err) diff --git a/pkg/store/proxy_heap.go b/pkg/store/proxy_heap.go index 5cdb5a0b78..f7664ea946 100644 --- a/pkg/store/proxy_heap.go +++ b/pkg/store/proxy_heap.go @@ -13,6 +13,7 @@ import ( "time" "github.com/go-kit/log" + "github.com/thanos-io/thanos/pkg/dedup" "github.com/cespare/xxhash/v2" "github.com/go-kit/log/level" @@ -225,6 +226,10 @@ type ProxyResponseHeapNode struct { rs respSet } +// NewProxyResponseHeap returns heap that k-way merge series together. +// In case of duplicates, series might return in random order without changed, chained or deduplicated chunks. +// TODO(bwplotka): Consider moving deduplication routines in single place for readability. Currently it's scattered in +// NewDedupResponseHeap, in removeDuplicates in promSeriesSet and dedup.NewSeriesSet. func NewProxyResponseHeap(seriesSets ...respSet) *ProxyResponseHeap { ret := make(ProxyResponseHeap, 0, len(seriesSets)) @@ -431,6 +436,7 @@ func newLazyRespSet( } if err != nil { + // TODO(bwplotka): Return early on error. Don't wait of dedup, merge and sort if partial response is disabled. var rerr error if t != nil && !t.Stop() && errors.Is(err, context.Canceled) { // Most likely the per-Recv timeout has been reached. @@ -490,24 +496,26 @@ func newLazyRespSet( type RetrievalStrategy string const ( + // LazyRetrieval allows readers (e.g. PromQL engine) to use (stream) data as soon as possible. LazyRetrieval RetrievalStrategy = "lazy" - - // TODO(GiedriusS): remove eager retrieval once - // https://github.com/prometheus/prometheus/blob/ce6a643ee88fba7c02fbd0459c4d0ac498f512dd/promql/engine.go#L877-L902 - // is removed. + // EagerRetrieval is optimized to read all into internal buffer before returning to readers (e.g. PromQL engine). + // This currently preferred because: + // * Both PromQL engines (old and new) want all series ASAP to make decisions. + // * Querier buffers all responses when using StoreAPI internally. EagerRetrieval RetrievalStrategy = "eager" ) -func newAsyncRespSet(ctx context.Context, +func newAsyncRespSet( + ctx context.Context, st Client, req *storepb.SeriesRequest, frameTimeout time.Duration, retrievalStrategy RetrievalStrategy, - storeSupportsSharding bool, buffers *sync.Pool, shardInfo *storepb.ShardInfo, logger log.Logger, - emptyStreamResponses prometheus.Counter) (respSet, error) { + emptyStreamResponses prometheus.Counter, +) (respSet, error) { var span opentracing.Span var closeSeries context.CancelFunc @@ -532,10 +540,9 @@ func newAsyncRespSet(ctx context.Context, shardMatcher := shardInfo.Matcher(buffers) - applySharding := shardInfo != nil && !storeSupportsSharding + applySharding := shardInfo != nil && !st.SupportsSharding() if applySharding { - msg := "Applying series sharding in the proxy since there is not support in the underlying store" - level.Debug(logger).Log("msg", msg, "store", st.String()) + level.Debug(logger).Log("msg", "Applying series sharding in the proxy since there is not support in the underlying store", "store", st.String()) } cl, err := st.Series(seriesCtx, req) @@ -548,6 +555,18 @@ func newAsyncRespSet(ctx context.Context, return nil, err } + var labelsToRemove map[string]struct{} + if !st.SupportsWithoutReplicaLabels() && len(req.WithoutReplicaLabels) > 0 { + level.Warn(logger).Log("msg", "detecting store that does not support without replica label setting. "+ + "Falling back to eager retrieval with additional sort. Make sure your storeAPI supports it to speed up your queries", "store", st.String()) + retrievalStrategy = EagerRetrieval + + labelsToRemove = make(map[string]struct{}) + for _, replicaLabel := range req.WithoutReplicaLabels { + labelsToRemove[replicaLabel] = struct{}{} + } + } + switch retrievalStrategy { case LazyRetrieval: return newLazyRespSet( @@ -573,6 +592,7 @@ func newAsyncRespSet(ctx context.Context, shardMatcher, applySharding, emptyStreamResponses, + labelsToRemove, ), nil default: panic(fmt.Sprintf("unsupported retrieval strategy %s", retrievalStrategy)) @@ -592,6 +612,7 @@ func (l *lazyRespSet) Close() { // eagerRespSet is a SeriesSet that blocks until all data is retrieved from // the StoreAPI. +// NOTE(bwplotka): It also resorts the series (and emits warning) if the client.SupportsWithoutReplicaLabels() is false. type eagerRespSet struct { // Generic parameters. span opentracing.Span @@ -603,6 +624,7 @@ type eagerRespSet struct { frameTimeout time.Duration shardMatcher *storepb.ShardMatcher + removeLabels map[string]struct{} // Internal bookkeeping. bufferedResponses []*storepb.SeriesResponse @@ -620,6 +642,7 @@ func newEagerRespSet( shardMatcher *storepb.ShardMatcher, applySharding bool, emptyStreamResponses prometheus.Counter, + removeLabels map[string]struct{}, ) respSet { ret := &eagerRespSet{ span: span, @@ -631,6 +654,7 @@ func newEagerRespSet( bufferedResponses: []*storepb.SeriesResponse{}, wg: &sync.WaitGroup{}, shardMatcher: shardMatcher, + removeLabels: removeLabels, } ret.wg.Add(1) @@ -656,6 +680,8 @@ func newEagerRespSet( } }() + // TODO(bwplotka): Consider improving readability by getting rid of anonymous functions and merging eager and + // lazyResponse into one struct. handleRecvResponse := func(t *time.Timer) bool { if t != nil { defer t.Reset(frameTimeout) @@ -673,6 +699,7 @@ func newEagerRespSet( return false } if err != nil { + // TODO(bwplotka): Return early on error. Don't wait of dedup, merge and sort if partial response is disabled. var rerr error if t != nil && !t.Stop() && errors.Is(err, context.Canceled) { // Most likely the per-Recv timeout has been reached. @@ -710,14 +737,83 @@ func newEagerRespSet( for { if !handleRecvResponse(t) { - return + break } } + + // This should be used only for stores that does not support doing this on server side. + // See docs/proposals-accepted/20221129-avoid-global-sort.md for details. + if len(l.removeLabels) > 0 { + sortWithoutLabels(l.bufferedResponses, l.removeLabels) + } + }(st, ret) return ret } +// Move all wanted labels back and remove. +// TODO(bwplotka): Consider microoptimizing this. +func rmLabels(l labels.Labels, labelsToRemove map[string]struct{}) labels.Labels { + sort.Slice(l, func(i, j int) bool { + if _, ok := labelsToRemove[l[i].Name]; ok { + return false + } + if _, ok := labelsToRemove[l[j].Name]; ok { + return true + } + // Ensure that dedup marker goes just right before the replica labels. + if l[i].Name == dedup.PushdownMarker.Name { + return false + } + if l[j].Name == dedup.PushdownMarker.Name { + return true + } + return l[i].Name < l[j].Name + }) + + var totalToRemove int + for i := 0; i < len(labelsToRemove); i++ { + if len(l)-i == 0 { + break + } + + if _, ok := labelsToRemove[l[len(l)-i-1].Name]; ok { + totalToRemove++ + } + } + + // Strip all present labels to remove. + return l[:len(l)-totalToRemove] +} + +// sortWithoutLabels removes given labels from series and re-sorts the series responses that the same +// series with different labels are coming right after each other. Other types of responses are moved to front. +func sortWithoutLabels(set []*storepb.SeriesResponse, labelsToRemove map[string]struct{}) { + for _, s := range set { + ser := s.GetSeries() + if ser == nil { + continue + } + + ser.Labels = labelpb.ZLabelsFromPromLabels(rmLabels(labelpb.ZLabelsToPromLabels(ser.Labels), labelsToRemove)) + } + + // With the re-ordered label sets, re-sorting all series aligns the same series + // from different replicas sequentially. + sort.Slice(set, func(i, j int) bool { + si := set[i].GetSeries() + if si == nil { + return true + } + sj := set[j].GetSeries() + if sj == nil { + return false + } + return labels.Compare(labelpb.ZLabelsToPromLabels(si.Labels), labelpb.ZLabelsToPromLabels(sj.Labels)) < 0 + }) +} + func (l *eagerRespSet) Close() { l.shardMatcher.Close() } diff --git a/pkg/store/proxy_heap_test.go b/pkg/store/proxy_heap_test.go new file mode 100644 index 0000000000..5ee268e59b --- /dev/null +++ b/pkg/store/proxy_heap_test.go @@ -0,0 +1,106 @@ +package store + +import ( + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/thanos/pkg/dedup" + "github.com/thanos-io/thanos/pkg/errors" + "github.com/thanos-io/thanos/pkg/store/storepb" + "github.com/thanos-io/thanos/pkg/testutil" +) + +func TestSortWithoutLabels(t *testing.T) { + for _, tcase := range []struct { + input []*storepb.SeriesResponse + exp []*storepb.SeriesResponse + dedupLabels map[string]struct{} + }{ + // Single deduplication label. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4")), + }, + dedupLabels: map[string]struct{}{"b": {}}, + }, + // Multi deduplication labels. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-1", "c", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "b1", "replica-2", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4")), + }, + dedupLabels: map[string]struct{}{"b": {}, "b1": {}}, + }, + // Pushdown label at the end. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "4", dedup.PushdownMarker.Name, dedup.PushdownMarker.Value)), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4", dedup.PushdownMarker.Name, dedup.PushdownMarker.Value)), + }, + dedupLabels: map[string]struct{}{"b": {}}, + }, + // Non series responses mixed. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "3", "d", "4")), + storepb.NewWarnSeriesResponse(errors.Newf("yolo")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-1", "c", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "b", "replica-2", "c", "3")), + }, + exp: []*storepb.SeriesResponse{ + storepb.NewWarnSeriesResponse(errors.Newf("yolo")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "3", "d", "4")), + storeSeriesResponse(t, labelsFromStrings("a", "1", "c", "4")), + }, + dedupLabels: map[string]struct{}{"b": {}}, + }, + } { + t.Run("", func(t *testing.T) { + sortWithoutLabels(tcase.input, tcase.dedupLabels) + testutil.Equals(t, tcase.exp, tcase.input) + }) + } +} + +// labelsFromStrings is like labels.FromString, but it does not sort the input. +func labelsFromStrings(ss ...string) labels.Labels { + if len(ss)%2 != 0 { + panic("invalid number of strings") + } + res := make(labels.Labels, 0, len(ss)/2) + for i := 0; i < len(ss); i += 2 { + res = append(res, labels.Label{Name: ss[i], Value: ss[i+1]}) + } + + return res +} diff --git a/pkg/store/proxy_test.go b/pkg/store/proxy_test.go index 7c1064233f..17a3602a0e 100644 --- a/pkg/store/proxy_test.go +++ b/pkg/store/proxy_test.go @@ -24,6 +24,7 @@ import ( "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/thanos-io/thanos/pkg/testutil/teststore" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -36,41 +37,6 @@ import ( "github.com/thanos-io/thanos/pkg/testutil/custom" ) -type testClient struct { - // Just to pass interface check. - storepb.StoreClient - - labelSets []labels.Labels - minTime int64 - maxTime int64 - supportsSharding bool - isLocalStore bool -} - -func (c testClient) LabelSets() []labels.Labels { - return c.labelSets -} - -func (c testClient) TimeRange() (int64, int64) { - return c.minTime, c.maxTime -} - -func (c testClient) SupportsSharding() bool { - return c.supportsSharding -} - -func (c testClient) SendsSortedSeries() bool { - return false -} - -func (c testClient) String() string { - return "test" -} - -func (c testClient) Addr() (string, bool) { - return "testaddr", c.isLocalStore -} - type mockedSeriesServer struct { storepb.Store_SeriesServer ctx context.Context @@ -138,14 +104,14 @@ func TestProxyStore_Series(t *testing.T) { { title: "no storeAPI available for 301-302 time range", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -158,15 +124,15 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; no series for ext=2 external label matcher", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -179,15 +145,15 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -205,14 +171,14 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for any external label matcher", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{4, 3}}, []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -230,14 +196,14 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for any external label matcher, but selector blocks", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, selectorLabels: labels.FromStrings("ext", "2"), @@ -250,14 +216,14 @@ func TestProxyStore_Series(t *testing.T) { { title: "no validation if storeAPI follow matching contract", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -276,7 +242,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "complex scenario with storeAPIs warnings", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}, []sample{{4, 3}}), @@ -285,45 +251,45 @@ func TestProxyStore_Series(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{2, 2}, {3, 3}, {4, 4}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "outside"), []sample{{1, 1}}), }, }, // Outside range for store itself. - minTime: 301, - maxTime: 302, + MinTime: 301, + MaxTime: 302, }, }, req: &storepb.SeriesRequest{ @@ -350,25 +316,25 @@ func TestProxyStore_Series(t *testing.T) { { title: "same external labels are validated during upload and on querier storeset, proxy does not care", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 11}, {2, 22}, {3, 33}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -386,24 +352,24 @@ func TestProxyStore_Series(t *testing.T) { { title: "partial response enabled", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -422,24 +388,24 @@ func TestProxyStore_Series(t *testing.T) { { title: "partial response disabled", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -454,15 +420,15 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher; allowed by store debug matcher", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -482,15 +448,15 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher; blocked by store debug matcher.", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -504,7 +470,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "sharded series response", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -512,9 +478,9 @@ func TestProxyStore_Series(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: 1, - maxTime: 300, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, }, req: &storepb.SeriesRequest{ @@ -597,7 +563,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st errors out after some delay; 2nd store is fast", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -608,11 +574,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { injectedError: errors.New("test"), injectedErrorIndex: 1, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -635,9 +601,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("b", "a"), []sample{{4, 1}, {5, 2}, {6, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -652,7 +618,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is slow, 2nd store is fast;", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -660,20 +626,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -688,18 +654,18 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is fast, 2nd store is slow;", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -707,9 +673,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -724,7 +690,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is slow on 2nd series, 2nd store is fast;", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -735,20 +701,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { RespDuration: 10 * time.Second, SlowSeriesIndex: 2, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -763,7 +729,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is fast to respond, 2nd store is slow on 2nd series;", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -772,11 +738,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{6, 1}, {7, 2}, {8, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -785,9 +751,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { RespDuration: 10 * time.Second, SlowSeriesIndex: 2, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -802,7 +768,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is slow to respond, 2nd store is fast;", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -810,20 +776,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("b", "c"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -842,18 +808,18 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is fast, 2nd store is slow;", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -861,9 +827,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -882,18 +848,18 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is fast, 2-3 is slow, 4th is fast;", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -901,11 +867,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -913,20 +879,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 10 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("d", "f"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -949,7 +915,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is slow on 2nd series, 2nd store is fast", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -960,20 +926,20 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { RespDuration: 10 * time.Second, SlowSeriesIndex: 2, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), storeSeriesResponse(t, labels.FromStrings("b", "c"), []sample{{1, 1}, {2, 2}, {3, 3}}), }, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -996,7 +962,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; all stores respond 3s", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1005,9 +971,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 3 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -1028,7 +994,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; all stores respond 3s", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1037,11 +1003,11 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 3 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("b", "c"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1050,9 +1016,9 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { }, RespDuration: 3 * time.Second, }, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, }, req: &storepb.SeriesRequest{ @@ -1127,11 +1093,11 @@ func TestProxyStore_Series_RequestParamsProxied(t *testing.T) { }, } cls := []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: m, - labelSets: []labels.Labels{labels.FromStrings("ext", "1")}, - minTime: 1, - maxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + MinTime: 1, + MaxTime: 300, }, } q := NewProxyStore(nil, @@ -1167,14 +1133,14 @@ func TestProxyStore_Series_RegressionFillResponseChannel(t *testing.T) { var cls []Client for i := 0; i < 10; i++ { - cls = append(cls, &testClient{ + cls = append(cls, &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("test error"), }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }) - cls = append(cls, &testClient{ + cls = append(cls, &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -1189,8 +1155,8 @@ func TestProxyStore_Series_RegressionFillResponseChannel(t *testing.T) { storepb.NewWarnSeriesResponse(errors.New("warning")), }, }, - minTime: 1, - maxTime: 300, + MinTime: 1, + MaxTime: 300, }) } @@ -1228,18 +1194,18 @@ func TestProxyStore_LabelValues(t *testing.T) { }, } cls := []Client{ - &testClient{StoreClient: m1}, - &testClient{StoreClient: &mockedStoreAPI{ + &teststore.TestClient{StoreClient: m1}, + &teststore.TestClient{StoreClient: &mockedStoreAPI{ RespLabelValues: &storepb.LabelValuesResponse{ Values: []string{"3", "4"}, }, }}, - &testClient{StoreClient: &mockedStoreAPI{ + &teststore.TestClient{StoreClient: &mockedStoreAPI{ RespLabelValues: &storepb.LabelValuesResponse{ Values: []string{"5", "6"}, }}, - minTime: timestamp.FromTime(time.Now().Add(-1 * time.Minute)), - maxTime: timestamp.FromTime(time.Now()), + MinTime: timestamp.FromTime(time.Now().Add(-1 * time.Minute)), + MaxTime: timestamp.FromTime(time.Now()), }, } q := NewProxyStore(nil, @@ -1296,14 +1262,14 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response disabled", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "c", "d"}, @@ -1322,14 +1288,14 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response disabled, but returns error", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, @@ -1345,14 +1311,14 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response enabled", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, @@ -1369,23 +1335,23 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "stores filtered by time range", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, - minTime: timestamp.FromTime(time.Now().Add(-4 * time.Hour)), - maxTime: timestamp.FromTime(time.Now().Add(-3 * time.Hour)), + MinTime: timestamp.FromTime(time.Now().Add(-4 * time.Hour)), + MaxTime: timestamp.FromTime(time.Now().Add(-3 * time.Hour)), }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"c", "d"}, }, }, - minTime: timestamp.FromTime(time.Now().Add(-2 * time.Hour)), - maxTime: timestamp.FromTime(time.Now().Add(-1 * time.Hour)), + MinTime: timestamp.FromTime(time.Now().Add(-2 * time.Hour)), + MaxTime: timestamp.FromTime(time.Now().Add(-1 * time.Hour)), }, }, req: &storepb.LabelNamesRequest{ @@ -1399,7 +1365,7 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "store matchers blocks", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, @@ -1419,7 +1385,7 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "store matchers allows", storeAPIs: []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, @@ -1508,7 +1474,7 @@ func TestStoreMatches(t *testing.T) { expectedReason string }{ { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "b", "1"), }, @@ -1517,7 +1483,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "does not have data within this time period: [0,-1]. Store time ranges: [0,0]", }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "b", "1"), }, @@ -1525,33 +1491,33 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, mint: 201, maxt: 300, expectedMatch: false, expectedReason: "does not have data within this time period: [201,300]. Store time ranges: [100,200]", }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, mint: 200, maxt: 300, expectedMatch: true, }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, mint: 50, maxt: 99, expectedMatch: false, expectedReason: "does not have data within this time period: [50,99]. Store time ranges: [100,200]", }, { - s: &testClient{minTime: 100, maxTime: 200}, + s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, mint: 50, maxt: 101, expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "a", "b"), }, @@ -1559,7 +1525,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "a", "c"), }, @@ -1568,7 +1534,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "external labels [{a=\"b\"}] does not match request label matchers: [a=\"c\"]", }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchRegexp, "a", "b|c"), }, @@ -1576,7 +1542,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchNotRegexp, "a", ""), }, @@ -1584,7 +1550,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{ + s: &teststore.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1597,7 +1563,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "external labels [{a=\"b\"} {a=\"c\"} {a=\"d\"}] does not match request label matchers: [a=\"e\"]", }, { - s: &testClient{labelSets: []labels.Labels{ + s: &teststore.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1609,7 +1575,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &testClient{labelSets: []labels.Labels{ + s: &teststore.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1826,12 +1792,12 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { resps = append(resps, storepb.NewSeriesResponse(created[i])) } - clients[j] = &testClient{ + clients[j] = &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: resps, }, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, } } @@ -1848,7 +1814,7 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { var expected []*storepb.Series lastLabels := storepb.Series{} for _, c := range clients { - m := c.(*testClient).StoreClient.(*mockedStoreAPI) + m := c.(*teststore.TestClient).StoreClient.(*mockedStoreAPI) // NOTE: Proxy will merge all series with same labels without any frame limit (https://github.com/thanos-io/thanos/issues/2332). for _, r := range m.RespSeries { @@ -1888,14 +1854,14 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { // Change client to just one. store.stores = func() []Client { - return []Client{&testClient{ + return []Client{&teststore.TestClient{ StoreClient: &mockedStoreAPI{ // All responses. RespSeries: allResps, }, - labelSets: []labels.Labels{labels.FromStrings("ext1", "1")}, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + ExtLset: []labels.Labels{labels.FromStrings("ext1", "1")}, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, }} } @@ -1923,7 +1889,7 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { defer custom.TolerantVerifyLeak(t) clients := []Client{ - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ // Ensure more than 10 (internal respCh channel). @@ -1939,10 +1905,10 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "j"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, }, - &testClient{ + &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("b", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -1957,8 +1923,8 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("b", "j"), []sample{{0, 0}, {2, 1}, {3, 2}}), }, }, - minTime: math.MinInt64, - maxTime: math.MaxInt64, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, }, } @@ -1986,15 +1952,15 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { } func TestProxyStore_storeMatchMetadata(t *testing.T) { - c := testClient{} - c.isLocalStore = true + c := teststore.TestClient{} + c.IsLocalStore = true ok, reason := storeMatchDebugMetadata(c, [][]*labels.Matcher{{}}) testutil.Assert(t, !ok) testutil.Equals(t, "the store is not remote, cannot match __address__", reason) // Change client to remote. - c.isLocalStore = false + c.IsLocalStore = false ok, reason = storeMatchDebugMetadata(c, [][]*labels.Matcher{{labels.MustNewMatcher(labels.MatchEqual, "__address__", "wrong")}}) testutil.Assert(t, !ok) diff --git a/pkg/store/storepb/rpc.pb.go b/pkg/store/storepb/rpc.pb.go index bf670d0f63..b288b5c909 100644 --- a/pkg/store/storepb/rpc.pb.go +++ b/pkg/store/storepb/rpc.pb.go @@ -32,6 +32,7 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +// Deprecated. Use `thanos.info` instead. type StoreType int32 const ( @@ -185,6 +186,7 @@ func (m *WriteRequest) XXX_DiscardUnknown() { var xxx_messageInfo_WriteRequest proto.InternalMessageInfo +// Deprecated. Use `thanos.info` instead. type InfoRequest struct { } @@ -221,6 +223,7 @@ func (m *InfoRequest) XXX_DiscardUnknown() { var xxx_messageInfo_InfoRequest proto.InternalMessageInfo +// Deprecated. Use `thanos.info` instead. type InfoResponse struct { // Deprecated. Use label_sets instead. Labels []github_com_thanos_io_thanos_pkg_store_labelpb.ZLabel `protobuf:"bytes,1,rep,name=labels,proto3,customtype=github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel" json:"labels"` @@ -288,10 +291,19 @@ type SeriesRequest struct { Range int64 `protobuf:"varint,11,opt,name=range,proto3" json:"range,omitempty"` // query_hints are the hints coming from the PromQL engine when // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. QueryHints *QueryHints `protobuf:"bytes,12,opt,name=query_hints,json=queryHints,proto3" json:"query_hints,omitempty"` // shard_info is used by the querier to request a specific // shard of blocks instead of entire blocks. ShardInfo *ShardInfo `protobuf:"bytes,13,opt,name=shard_info,json=shardInfo,proto3" json:"shard_info,omitempty"` + // without_replica_labels are replica labels which have to be excluded from series set results (including sorting). + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + WithoutReplicaLabels []string `protobuf:"bytes,14,rep,name=without_replica_labels,json=withoutReplicaLabels,proto3" json:"without_replica_labels,omitempty"` } func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } @@ -327,7 +339,10 @@ func (m *SeriesRequest) XXX_DiscardUnknown() { var xxx_messageInfo_SeriesRequest proto.InternalMessageInfo -// Analogous to storage.SelectHints. +// QueryHints represents hints from PromQL that might help to +// pre-aggregate or prepare series for faster use by clients. +// Analogous to storage.SelectHints plus additional info. +// As "hints" name suggests all of the items here are best effort. type QueryHints struct { // Query step size in milliseconds. StepMillis int64 `protobuf:"varint,1,opt,name=step_millis,json=stepMillis,proto3" json:"step_millis,omitempty"` @@ -830,89 +845,90 @@ func init() { func init() { proto.RegisterFile("store/storepb/rpc.proto", fileDescriptor_a938d55a388af629) } var fileDescriptor_a938d55a388af629 = []byte{ - // 1298 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0x5d, 0x6f, 0x13, 0x47, - 0x17, 0xf6, 0x7a, 0xbd, 0xfe, 0x38, 0x4e, 0xf2, 0x9a, 0xc1, 0xc0, 0xc6, 0x48, 0x8e, 0xdf, 0x7d, - 0xf5, 0x4a, 0x11, 0xa2, 0x36, 0x35, 0x15, 0x52, 0x2b, 0x6e, 0x92, 0x60, 0x48, 0x54, 0x62, 0xca, - 0x38, 0x21, 0x2d, 0x55, 0x65, 0xad, 0xed, 0xc9, 0x7a, 0xc5, 0x7a, 0x77, 0xd9, 0x99, 0x6d, 0xe2, - 0xdb, 0xf6, 0xbe, 0xaa, 0xfa, 0x13, 0xfa, 0x2b, 0xfa, 0x13, 0xb8, 0x2b, 0x57, 0x55, 0xd5, 0x0b, - 0xd4, 0xc2, 0x1f, 0xa9, 0xe6, 0x63, 0xd7, 0xde, 0x34, 0x40, 0x11, 0xdc, 0x44, 0x73, 0x9e, 0xe7, - 0xcc, 0x99, 0xf3, 0xed, 0x0d, 0x5c, 0xa1, 0x2c, 0x88, 0x48, 0x47, 0xfc, 0x0d, 0x47, 0x9d, 0x28, - 0x1c, 0xb7, 0xc3, 0x28, 0x60, 0x01, 0x2a, 0xb2, 0xa9, 0xed, 0x07, 0xb4, 0xb1, 0x9e, 0x55, 0x60, - 0xf3, 0x90, 0x50, 0xa9, 0xd2, 0xa8, 0x3b, 0x81, 0x13, 0x88, 0x63, 0x87, 0x9f, 0x14, 0xda, 0xca, - 0x5e, 0x08, 0xa3, 0x60, 0x76, 0xe6, 0x9e, 0x32, 0xe9, 0xd9, 0x23, 0xe2, 0x9d, 0xa5, 0x9c, 0x20, - 0x70, 0x3c, 0xd2, 0x11, 0xd2, 0x28, 0x3e, 0xee, 0xd8, 0xfe, 0x5c, 0x52, 0xd6, 0x7f, 0x60, 0xf5, - 0x28, 0x72, 0x19, 0xc1, 0x84, 0x86, 0x81, 0x4f, 0x89, 0xf5, 0xbd, 0x06, 0x2b, 0x0a, 0x79, 0x1a, - 0x13, 0xca, 0xd0, 0x16, 0x00, 0x73, 0x67, 0x84, 0x92, 0xc8, 0x25, 0xd4, 0xd4, 0x5a, 0xfa, 0x66, - 0xb5, 0x7b, 0x95, 0xdf, 0x9e, 0x11, 0x36, 0x25, 0x31, 0x1d, 0x8e, 0x83, 0x70, 0xde, 0x3e, 0x70, - 0x67, 0x64, 0x20, 0x54, 0xb6, 0x0b, 0xcf, 0x5e, 0x6c, 0xe4, 0xf0, 0xd2, 0x25, 0x74, 0x19, 0x8a, - 0x8c, 0xf8, 0xb6, 0xcf, 0xcc, 0x7c, 0x4b, 0xdb, 0xac, 0x60, 0x25, 0x21, 0x13, 0x4a, 0x11, 0x09, - 0x3d, 0x77, 0x6c, 0x9b, 0x7a, 0x4b, 0xdb, 0xd4, 0x71, 0x22, 0x5a, 0xab, 0x50, 0xdd, 0xf3, 0x8f, - 0x03, 0xe5, 0x83, 0xf5, 0x53, 0x1e, 0x56, 0xa4, 0x2c, 0xbd, 0x44, 0x63, 0x28, 0x8a, 0x40, 0x13, - 0x87, 0x56, 0xdb, 0x32, 0xb1, 0xed, 0xfb, 0x1c, 0xdd, 0xbe, 0xcd, 0x5d, 0xf8, 0xe3, 0xc5, 0xc6, - 0x27, 0x8e, 0xcb, 0xa6, 0xf1, 0xa8, 0x3d, 0x0e, 0x66, 0x1d, 0xa9, 0xf0, 0x91, 0x1b, 0xa8, 0x53, - 0x27, 0x7c, 0xe2, 0x74, 0x32, 0x39, 0x6b, 0x3f, 0x16, 0xb7, 0xb1, 0x32, 0x8d, 0xd6, 0xa1, 0x3c, - 0x73, 0xfd, 0x21, 0x0f, 0x44, 0x38, 0xae, 0xe3, 0xd2, 0xcc, 0xf5, 0x79, 0xa4, 0x82, 0xb2, 0x4f, - 0x25, 0xa5, 0x5c, 0x9f, 0xd9, 0xa7, 0x82, 0xea, 0x40, 0x45, 0x58, 0x3d, 0x98, 0x87, 0xc4, 0x2c, - 0xb4, 0xb4, 0xcd, 0xb5, 0xee, 0x85, 0xc4, 0xbb, 0x41, 0x42, 0xe0, 0x85, 0x0e, 0xba, 0x05, 0x20, - 0x1e, 0x1c, 0x52, 0xc2, 0xa8, 0x69, 0x88, 0x78, 0xd2, 0x1b, 0xd2, 0xa5, 0x01, 0x61, 0x2a, 0xad, - 0x15, 0x4f, 0xc9, 0xd4, 0xfa, 0xad, 0x00, 0xab, 0x32, 0xe5, 0x49, 0xa9, 0x96, 0x1d, 0xd6, 0x5e, - 0xef, 0x70, 0x3e, 0xeb, 0xf0, 0x2d, 0x4e, 0xb1, 0xf1, 0x94, 0x44, 0xd4, 0xd4, 0xc5, 0xeb, 0xf5, - 0x4c, 0x36, 0xf7, 0x25, 0xa9, 0x1c, 0x48, 0x75, 0x51, 0x17, 0x2e, 0x71, 0x93, 0x11, 0xa1, 0x81, - 0x17, 0x33, 0x37, 0xf0, 0x87, 0x27, 0xae, 0x3f, 0x09, 0x4e, 0x44, 0xd0, 0x3a, 0xbe, 0x38, 0xb3, - 0x4f, 0x71, 0xca, 0x1d, 0x09, 0x0a, 0x5d, 0x07, 0xb0, 0x1d, 0x27, 0x22, 0x8e, 0xcd, 0x88, 0x8c, - 0x75, 0xad, 0xbb, 0x92, 0xbc, 0xb6, 0xe5, 0x38, 0x11, 0x5e, 0xe2, 0xd1, 0x67, 0xb0, 0x1e, 0xda, - 0x11, 0x73, 0x6d, 0x8f, 0xbf, 0x22, 0x2a, 0x3f, 0x9c, 0xb8, 0xd4, 0x1e, 0x79, 0x64, 0x62, 0x16, - 0x5b, 0xda, 0x66, 0x19, 0x5f, 0x51, 0x0a, 0x49, 0x67, 0xdc, 0x51, 0x34, 0xfa, 0xfa, 0x9c, 0xbb, - 0x94, 0x45, 0x36, 0x23, 0xce, 0xdc, 0x2c, 0x89, 0xb2, 0x6c, 0x24, 0x0f, 0x7f, 0x91, 0xb5, 0x31, - 0x50, 0x6a, 0xff, 0x30, 0x9e, 0x10, 0x68, 0x03, 0xaa, 0xf4, 0x89, 0x1b, 0x0e, 0xc7, 0xd3, 0xd8, - 0x7f, 0x42, 0xcd, 0xb2, 0x70, 0x05, 0x38, 0xb4, 0x23, 0x10, 0x74, 0x0d, 0x8c, 0xa9, 0xeb, 0x33, - 0x6a, 0x56, 0x5a, 0x9a, 0x48, 0xa8, 0x9c, 0xc0, 0x76, 0x32, 0x81, 0xed, 0x2d, 0x7f, 0x8e, 0xa5, - 0x0a, 0x42, 0x50, 0xa0, 0x8c, 0x84, 0x26, 0x88, 0xb4, 0x89, 0x33, 0xaa, 0x83, 0x11, 0xd9, 0xbe, - 0x43, 0xcc, 0xaa, 0x00, 0xa5, 0x80, 0x6e, 0x42, 0xf5, 0x69, 0x4c, 0xa2, 0xf9, 0x50, 0xda, 0x5e, - 0x11, 0xb6, 0x51, 0x12, 0xc5, 0x43, 0x4e, 0xed, 0x72, 0x06, 0xc3, 0xd3, 0xf4, 0x8c, 0x6e, 0x00, - 0xd0, 0xa9, 0x1d, 0x4d, 0x86, 0xae, 0x7f, 0x1c, 0x98, 0xab, 0xe2, 0xce, 0xa2, 0x21, 0x39, 0x23, - 0x26, 0xab, 0x42, 0x93, 0xa3, 0xf5, 0xb3, 0x06, 0xb0, 0x30, 0x26, 0x82, 0x65, 0x24, 0x1c, 0xce, - 0x5c, 0xcf, 0x73, 0xa9, 0x6a, 0x2c, 0xe0, 0xd0, 0xbe, 0x40, 0x50, 0x0b, 0x0a, 0xc7, 0xb1, 0x3f, - 0x16, 0x7d, 0x55, 0x5d, 0x94, 0xf3, 0x6e, 0xec, 0x8f, 0xb1, 0x60, 0xd0, 0x75, 0x28, 0x3b, 0x51, - 0x10, 0x87, 0xae, 0xef, 0x88, 0xee, 0xa8, 0x76, 0x6b, 0x89, 0xd6, 0x3d, 0x85, 0xe3, 0x54, 0x03, - 0xfd, 0x2f, 0x09, 0xde, 0x10, 0xaa, 0xe9, 0x6c, 0x63, 0x0e, 0xaa, 0x5c, 0x58, 0x27, 0x50, 0x49, - 0x9d, 0x17, 0x2e, 0xaa, 0x18, 0x27, 0xe4, 0x34, 0x75, 0x51, 0xf2, 0x13, 0x72, 0x8a, 0xfe, 0x0b, - 0x2b, 0x2c, 0x60, 0xb6, 0x37, 0x14, 0x18, 0x55, 0x23, 0x50, 0x15, 0x98, 0x30, 0x43, 0xd1, 0x1a, - 0xe4, 0x47, 0x73, 0x31, 0xcc, 0x65, 0x9c, 0x1f, 0xcd, 0xf9, 0xd2, 0x52, 0x2b, 0xa6, 0xd0, 0xd2, - 0xf9, 0xd2, 0x92, 0x92, 0xd5, 0x80, 0x02, 0x8f, 0x8c, 0x97, 0xcd, 0xb7, 0xd5, 0xa0, 0x55, 0xb0, - 0x38, 0x5b, 0x5d, 0x28, 0x27, 0xf1, 0x28, 0x7b, 0xda, 0x39, 0xf6, 0xf4, 0x8c, 0xbd, 0x0d, 0x30, - 0x44, 0x60, 0x5c, 0x21, 0x93, 0x62, 0x25, 0x59, 0x3f, 0x68, 0xb0, 0x96, 0xcc, 0xb9, 0x5a, 0x7f, - 0x9b, 0x50, 0x4c, 0xf7, 0x31, 0x4f, 0xd1, 0x5a, 0x5a, 0x4f, 0x81, 0xee, 0xe6, 0xb0, 0xe2, 0x51, - 0x03, 0x4a, 0x27, 0x76, 0xe4, 0xf3, 0xc4, 0x8b, 0xdd, 0xbb, 0x9b, 0xc3, 0x09, 0x80, 0xae, 0x27, - 0x4d, 0xaa, 0xbf, 0xbe, 0x49, 0x77, 0x73, 0xaa, 0x4d, 0xb7, 0xcb, 0x50, 0x8c, 0x08, 0x8d, 0x3d, - 0x66, 0xfd, 0x92, 0x87, 0x0b, 0x62, 0x33, 0xf4, 0xed, 0xd9, 0x62, 0xf9, 0xbc, 0x71, 0x58, 0xb5, - 0xf7, 0x18, 0xd6, 0xfc, 0x7b, 0x0e, 0x6b, 0x1d, 0x0c, 0xca, 0xec, 0x88, 0xa9, 0x45, 0x2d, 0x05, - 0x54, 0x03, 0x9d, 0xf8, 0x13, 0xb5, 0xab, 0xf8, 0x71, 0x31, 0xb3, 0xc6, 0xdb, 0x67, 0x76, 0x79, - 0x67, 0x16, 0xff, 0xfd, 0xce, 0xb4, 0x22, 0x40, 0xcb, 0x99, 0x53, 0xe5, 0xac, 0x83, 0xc1, 0xdb, - 0x47, 0xfe, 0x98, 0x55, 0xb0, 0x14, 0x50, 0x03, 0xca, 0xaa, 0x52, 0xbc, 0x5f, 0x39, 0x91, 0xca, - 0x0b, 0x5f, 0xf5, 0xb7, 0xfa, 0x6a, 0xfd, 0x9a, 0x57, 0x8f, 0x3e, 0xb2, 0xbd, 0x78, 0x51, 0xaf, - 0x3a, 0x18, 0xa2, 0x03, 0x55, 0x03, 0x4b, 0xe1, 0xcd, 0x55, 0xcc, 0xbf, 0x47, 0x15, 0xf5, 0x0f, - 0x55, 0xc5, 0xc2, 0x39, 0x55, 0x34, 0xce, 0xa9, 0x62, 0xf1, 0xdd, 0xaa, 0x58, 0x7a, 0x87, 0x2a, - 0xc6, 0x70, 0x31, 0x93, 0x50, 0x55, 0xc6, 0xcb, 0x50, 0xfc, 0x56, 0x20, 0xaa, 0x8e, 0x4a, 0xfa, - 0x50, 0x85, 0xbc, 0xf6, 0x0d, 0x54, 0xd2, 0x0f, 0x08, 0x54, 0x85, 0xd2, 0x61, 0xff, 0xf3, 0xfe, - 0x83, 0xa3, 0x7e, 0x2d, 0x87, 0x2a, 0x60, 0x3c, 0x3c, 0xec, 0xe1, 0xaf, 0x6a, 0x1a, 0x2a, 0x43, - 0x01, 0x1f, 0xde, 0xef, 0xd5, 0xf2, 0x5c, 0x63, 0xb0, 0x77, 0xa7, 0xb7, 0xb3, 0x85, 0x6b, 0x3a, - 0xd7, 0x18, 0x1c, 0x3c, 0xc0, 0xbd, 0x5a, 0x81, 0xe3, 0xb8, 0xb7, 0xd3, 0xdb, 0x7b, 0xd4, 0xab, - 0x19, 0x1c, 0xbf, 0xd3, 0xdb, 0x3e, 0xbc, 0x57, 0x2b, 0x5e, 0xdb, 0x86, 0x02, 0xff, 0x05, 0x46, - 0x25, 0xd0, 0xf1, 0xd6, 0x91, 0xb4, 0xba, 0xf3, 0xe0, 0xb0, 0x7f, 0x50, 0xd3, 0x38, 0x36, 0x38, - 0xdc, 0xaf, 0xe5, 0xf9, 0x61, 0x7f, 0xaf, 0x5f, 0xd3, 0xc5, 0x61, 0xeb, 0x4b, 0x69, 0x4e, 0x68, - 0xf5, 0x70, 0xcd, 0xe8, 0x7e, 0x97, 0x07, 0x43, 0xf8, 0x88, 0x3e, 0x86, 0x82, 0x58, 0xcd, 0x17, - 0x93, 0x8c, 0x2e, 0x7d, 0xcf, 0x35, 0xea, 0x59, 0x50, 0xe5, 0xef, 0x53, 0x28, 0xca, 0xfd, 0x85, - 0x2e, 0x65, 0xf7, 0x59, 0x72, 0xed, 0xf2, 0x59, 0x58, 0x5e, 0xbc, 0xa1, 0xa1, 0x1d, 0x80, 0xc5, - 0x5c, 0xa1, 0xf5, 0x4c, 0x15, 0x97, 0xb7, 0x54, 0xa3, 0x71, 0x1e, 0xa5, 0xde, 0xbf, 0x0b, 0xd5, - 0xa5, 0xb2, 0xa2, 0xac, 0x6a, 0x66, 0x78, 0x1a, 0x57, 0xcf, 0xe5, 0xa4, 0x9d, 0x6e, 0x1f, 0xd6, - 0xc4, 0x17, 0x34, 0x9f, 0x0a, 0x99, 0x8c, 0xdb, 0x50, 0xc5, 0x64, 0x16, 0x30, 0x22, 0x70, 0x94, - 0x86, 0xbf, 0xfc, 0xa1, 0xdd, 0xb8, 0x74, 0x06, 0x55, 0x1f, 0xe4, 0xb9, 0xed, 0xff, 0x3f, 0xfb, - 0xab, 0x99, 0x7b, 0xf6, 0xb2, 0xa9, 0x3d, 0x7f, 0xd9, 0xd4, 0xfe, 0x7c, 0xd9, 0xd4, 0x7e, 0x7c, - 0xd5, 0xcc, 0x3d, 0x7f, 0xd5, 0xcc, 0xfd, 0xfe, 0xaa, 0x99, 0x7b, 0x5c, 0x52, 0xff, 0x13, 0x8c, - 0x8a, 0xa2, 0x67, 0x6e, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x02, 0x42, 0x0e, 0xd0, 0x7d, 0x0c, - 0x00, 0x00, + // 1323 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0xdd, 0x6e, 0x13, 0xc7, + 0x17, 0xf7, 0x7a, 0xbd, 0xfe, 0x38, 0x4e, 0xf2, 0x37, 0x83, 0x09, 0x1b, 0x23, 0x39, 0xfe, 0xbb, + 0xaa, 0x64, 0x21, 0x6a, 0x53, 0x83, 0x90, 0x5a, 0x71, 0x93, 0x04, 0x43, 0xa2, 0x12, 0x53, 0xc6, + 0x09, 0x69, 0xa9, 0x2a, 0x6b, 0x6d, 0x4f, 0xd6, 0x2b, 0xec, 0xdd, 0x65, 0x67, 0xb6, 0x89, 0x6f, + 0xdb, 0xdb, 0xaa, 0xaa, 0xfa, 0x08, 0x7d, 0x8a, 0x3e, 0x02, 0x77, 0xe5, 0xb2, 0xea, 0x05, 0x6a, + 0xe1, 0x45, 0xaa, 0x39, 0x3b, 0x6b, 0x7b, 0xd3, 0x00, 0x45, 0x70, 0x13, 0xcd, 0xf9, 0xfd, 0xce, + 0x9c, 0x39, 0xdf, 0xde, 0xc0, 0x65, 0x2e, 0xbc, 0x80, 0xb5, 0xf0, 0xaf, 0x3f, 0x68, 0x05, 0xfe, + 0xb0, 0xe9, 0x07, 0x9e, 0xf0, 0x48, 0x56, 0x8c, 0x2d, 0xd7, 0xe3, 0x95, 0x8d, 0xa4, 0x82, 0x98, + 0xf9, 0x8c, 0x47, 0x2a, 0x95, 0xb2, 0xed, 0xd9, 0x1e, 0x1e, 0x5b, 0xf2, 0xa4, 0xd0, 0x5a, 0xf2, + 0x82, 0x1f, 0x78, 0xd3, 0x33, 0xf7, 0x94, 0xc9, 0x89, 0x35, 0x60, 0x93, 0xb3, 0x94, 0xed, 0x79, + 0xf6, 0x84, 0xb5, 0x50, 0x1a, 0x84, 0xc7, 0x2d, 0xcb, 0x9d, 0x45, 0x54, 0xfd, 0x7f, 0xb0, 0x7a, + 0x14, 0x38, 0x82, 0x51, 0xc6, 0x7d, 0xcf, 0xe5, 0xac, 0xfe, 0x83, 0x06, 0x2b, 0x0a, 0x79, 0x1a, + 0x32, 0x2e, 0xc8, 0x16, 0x80, 0x70, 0xa6, 0x8c, 0xb3, 0xc0, 0x61, 0xdc, 0xd4, 0x6a, 0x7a, 0xa3, + 0xd8, 0xbe, 0x22, 0x6f, 0x4f, 0x99, 0x18, 0xb3, 0x90, 0xf7, 0x87, 0x9e, 0x3f, 0x6b, 0x1e, 0x38, + 0x53, 0xd6, 0x43, 0x95, 0xed, 0xcc, 0xb3, 0x17, 0x9b, 0x29, 0xba, 0x74, 0x89, 0xac, 0x43, 0x56, + 0x30, 0xd7, 0x72, 0x85, 0x99, 0xae, 0x69, 0x8d, 0x02, 0x55, 0x12, 0x31, 0x21, 0x17, 0x30, 0x7f, + 0xe2, 0x0c, 0x2d, 0x53, 0xaf, 0x69, 0x0d, 0x9d, 0xc6, 0x62, 0x7d, 0x15, 0x8a, 0x7b, 0xee, 0xb1, + 0xa7, 0x7c, 0xa8, 0xff, 0x92, 0x86, 0x95, 0x48, 0x8e, 0xbc, 0x24, 0x43, 0xc8, 0x62, 0xa0, 0xb1, + 0x43, 0xab, 0xcd, 0x28, 0xb1, 0xcd, 0xfb, 0x12, 0xdd, 0xbe, 0x2d, 0x5d, 0xf8, 0xf3, 0xc5, 0xe6, + 0x4d, 0xdb, 0x11, 0xe3, 0x70, 0xd0, 0x1c, 0x7a, 0xd3, 0x56, 0xa4, 0xf0, 0x89, 0xe3, 0xa9, 0x53, + 0xcb, 0x7f, 0x62, 0xb7, 0x12, 0x39, 0x6b, 0x3e, 0xc6, 0xdb, 0x54, 0x99, 0x26, 0x1b, 0x90, 0x9f, + 0x3a, 0x6e, 0x5f, 0x06, 0x82, 0x8e, 0xeb, 0x34, 0x37, 0x75, 0x5c, 0x19, 0x29, 0x52, 0xd6, 0x69, + 0x44, 0x29, 0xd7, 0xa7, 0xd6, 0x29, 0x52, 0x2d, 0x28, 0xa0, 0xd5, 0x83, 0x99, 0xcf, 0xcc, 0x4c, + 0x4d, 0x6b, 0xac, 0xb5, 0x2f, 0xc4, 0xde, 0xf5, 0x62, 0x82, 0x2e, 0x74, 0xc8, 0x2d, 0x00, 0x7c, + 0xb0, 0xcf, 0x99, 0xe0, 0xa6, 0x81, 0xf1, 0xcc, 0x6f, 0x44, 0x2e, 0xf5, 0x98, 0x50, 0x69, 0x2d, + 0x4c, 0x94, 0xcc, 0xeb, 0x3f, 0x1a, 0xb0, 0x1a, 0xa5, 0x3c, 0x2e, 0xd5, 0xb2, 0xc3, 0xda, 0xeb, + 0x1d, 0x4e, 0x27, 0x1d, 0xbe, 0x25, 0x29, 0x31, 0x1c, 0xb3, 0x80, 0x9b, 0x3a, 0xbe, 0x5e, 0x4e, + 0x64, 0x73, 0x3f, 0x22, 0x95, 0x03, 0x73, 0x5d, 0xd2, 0x86, 0x4b, 0xd2, 0x64, 0xc0, 0xb8, 0x37, + 0x09, 0x85, 0xe3, 0xb9, 0xfd, 0x13, 0xc7, 0x1d, 0x79, 0x27, 0x18, 0xb4, 0x4e, 0x2f, 0x4e, 0xad, + 0x53, 0x3a, 0xe7, 0x8e, 0x90, 0x22, 0xd7, 0x00, 0x2c, 0xdb, 0x0e, 0x98, 0x6d, 0x09, 0x16, 0xc5, + 0xba, 0xd6, 0x5e, 0x89, 0x5f, 0xdb, 0xb2, 0xed, 0x80, 0x2e, 0xf1, 0xe4, 0x73, 0xd8, 0xf0, 0xad, + 0x40, 0x38, 0xd6, 0x44, 0xbe, 0x82, 0x95, 0xef, 0x8f, 0x1c, 0x6e, 0x0d, 0x26, 0x6c, 0x64, 0x66, + 0x6b, 0x5a, 0x23, 0x4f, 0x2f, 0x2b, 0x85, 0xb8, 0x33, 0xee, 0x28, 0x9a, 0x7c, 0x73, 0xce, 0x5d, + 0x2e, 0x02, 0x4b, 0x30, 0x7b, 0x66, 0xe6, 0xb0, 0x2c, 0x9b, 0xf1, 0xc3, 0x5f, 0x26, 0x6d, 0xf4, + 0x94, 0xda, 0xbf, 0x8c, 0xc7, 0x04, 0xd9, 0x84, 0x22, 0x7f, 0xe2, 0xf8, 0xfd, 0xe1, 0x38, 0x74, + 0x9f, 0x70, 0x33, 0x8f, 0xae, 0x80, 0x84, 0x76, 0x10, 0x21, 0x57, 0xc1, 0x18, 0x3b, 0xae, 0xe0, + 0x66, 0xa1, 0xa6, 0x61, 0x42, 0xa3, 0x09, 0x6c, 0xc6, 0x13, 0xd8, 0xdc, 0x72, 0x67, 0x34, 0x52, + 0x21, 0x04, 0x32, 0x5c, 0x30, 0xdf, 0x04, 0x4c, 0x1b, 0x9e, 0x49, 0x19, 0x8c, 0xc0, 0x72, 0x6d, + 0x66, 0x16, 0x11, 0x8c, 0x04, 0x72, 0x03, 0x8a, 0x4f, 0x43, 0x16, 0xcc, 0xfa, 0x91, 0xed, 0x15, + 0xb4, 0x4d, 0xe2, 0x28, 0x1e, 0x4a, 0x6a, 0x57, 0x32, 0x14, 0x9e, 0xce, 0xcf, 0xe4, 0x3a, 0x00, + 0x1f, 0x5b, 0xc1, 0xa8, 0xef, 0xb8, 0xc7, 0x9e, 0xb9, 0x8a, 0x77, 0x16, 0x0d, 0x29, 0x19, 0x9c, + 0xac, 0x02, 0x8f, 0x8f, 0xe4, 0x26, 0xac, 0x9f, 0x38, 0x62, 0xec, 0x85, 0xa2, 0xaf, 0xe6, 0xb1, + 0xaf, 0x86, 0x6d, 0xad, 0xa6, 0x37, 0x0a, 0xb4, 0xac, 0x58, 0x1a, 0x91, 0xd8, 0x24, 0xbc, 0xfe, + 0xab, 0x06, 0xb0, 0x70, 0x01, 0x53, 0x24, 0x98, 0xdf, 0x9f, 0x3a, 0x93, 0x89, 0xc3, 0x55, 0x3b, + 0x82, 0x84, 0xf6, 0x11, 0x21, 0x35, 0xc8, 0x1c, 0x87, 0xee, 0x10, 0xbb, 0xb1, 0xb8, 0x68, 0x82, + 0xbb, 0xa1, 0x3b, 0xa4, 0xc8, 0x90, 0x6b, 0x90, 0xb7, 0x03, 0x2f, 0xf4, 0x1d, 0xd7, 0xc6, 0x9e, + 0x2a, 0xb6, 0x4b, 0xb1, 0xd6, 0x3d, 0x85, 0xd3, 0xb9, 0x06, 0xf9, 0x28, 0x4e, 0x99, 0x81, 0xaa, + 0xf3, 0x8d, 0x40, 0x25, 0xa8, 0x32, 0x58, 0x3f, 0x81, 0xc2, 0x3c, 0x64, 0x74, 0x51, 0x65, 0x66, + 0xc4, 0x4e, 0xe7, 0x2e, 0x46, 0xfc, 0x88, 0x9d, 0x92, 0xff, 0xc3, 0x8a, 0xf0, 0x84, 0x35, 0xe9, + 0x23, 0xc6, 0xd5, 0xe0, 0x14, 0x11, 0x43, 0x33, 0x9c, 0xac, 0x41, 0x7a, 0x30, 0xc3, 0x15, 0x90, + 0xa7, 0xe9, 0xc1, 0x4c, 0xae, 0x3a, 0x95, 0xab, 0x0c, 0xe6, 0x4a, 0x49, 0xf5, 0x0a, 0x64, 0x64, + 0x64, 0xb2, 0xd8, 0xae, 0xa5, 0xc6, 0xb3, 0x40, 0xf1, 0x5c, 0x6f, 0x43, 0x3e, 0x8e, 0x47, 0xd9, + 0xd3, 0xce, 0xb1, 0xa7, 0x27, 0xec, 0x6d, 0x82, 0x81, 0x81, 0x49, 0x85, 0x44, 0x8a, 0x95, 0x54, + 0xff, 0x49, 0x83, 0xb5, 0x78, 0x3b, 0xa8, 0xa5, 0xd9, 0x80, 0xec, 0x7c, 0x8b, 0xcb, 0x14, 0xad, + 0xcd, 0xbb, 0x00, 0xd1, 0xdd, 0x14, 0x55, 0x3c, 0xa9, 0x40, 0xee, 0xc4, 0x0a, 0x5c, 0x99, 0x78, + 0xdc, 0xd8, 0xbb, 0x29, 0x1a, 0x03, 0xe4, 0x5a, 0xdc, 0xda, 0xfa, 0xeb, 0x5b, 0x7b, 0x37, 0xa5, + 0x9a, 0x7b, 0x3b, 0x0f, 0xd9, 0x80, 0xf1, 0x70, 0x22, 0xea, 0xbf, 0xa5, 0xe1, 0x02, 0xb6, 0x4a, + 0xd7, 0x9a, 0x2e, 0x56, 0xd6, 0x1b, 0x47, 0x5c, 0x7b, 0x8f, 0x11, 0x4f, 0xbf, 0xe7, 0x88, 0x97, + 0xc1, 0xe0, 0xc2, 0x0a, 0x84, 0x5a, 0xef, 0x91, 0x40, 0x4a, 0xa0, 0x33, 0x77, 0xa4, 0x36, 0x9c, + 0x3c, 0x2e, 0x26, 0xdd, 0x78, 0xfb, 0xa4, 0x2f, 0x6f, 0xda, 0xec, 0x7f, 0xdf, 0xb4, 0xf5, 0x00, + 0xc8, 0x72, 0xe6, 0x54, 0x39, 0xcb, 0x60, 0xc8, 0xf6, 0x89, 0x7e, 0x02, 0x0b, 0x34, 0x12, 0x48, + 0x05, 0xf2, 0xaa, 0x52, 0xb2, 0x5f, 0x25, 0x31, 0x97, 0x17, 0xbe, 0xea, 0x6f, 0xf5, 0xb5, 0xfe, + 0x7b, 0x5a, 0x3d, 0xfa, 0xc8, 0x9a, 0x84, 0x8b, 0x7a, 0x95, 0xc1, 0xc0, 0x0e, 0x54, 0x0d, 0x1c, + 0x09, 0x6f, 0xae, 0x62, 0xfa, 0x3d, 0xaa, 0xa8, 0x7f, 0xa8, 0x2a, 0x66, 0xce, 0xa9, 0xa2, 0x71, + 0x4e, 0x15, 0xb3, 0xef, 0x56, 0xc5, 0xdc, 0x3b, 0x54, 0x31, 0x84, 0x8b, 0x89, 0x84, 0xaa, 0x32, + 0xae, 0x43, 0xf6, 0x3b, 0x44, 0x54, 0x1d, 0x95, 0xf4, 0xa1, 0x0a, 0x79, 0xf5, 0x5b, 0x28, 0xcc, + 0x3f, 0x3b, 0x48, 0x11, 0x72, 0x87, 0xdd, 0x2f, 0xba, 0x0f, 0x8e, 0xba, 0xa5, 0x14, 0x29, 0x80, + 0xf1, 0xf0, 0xb0, 0x43, 0xbf, 0x2e, 0x69, 0x24, 0x0f, 0x19, 0x7a, 0x78, 0xbf, 0x53, 0x4a, 0x4b, + 0x8d, 0xde, 0xde, 0x9d, 0xce, 0xce, 0x16, 0x2d, 0xe9, 0x52, 0xa3, 0x77, 0xf0, 0x80, 0x76, 0x4a, + 0x19, 0x89, 0xd3, 0xce, 0x4e, 0x67, 0xef, 0x51, 0xa7, 0x64, 0x48, 0xfc, 0x4e, 0x67, 0xfb, 0xf0, + 0x5e, 0x29, 0x7b, 0x75, 0x1b, 0x32, 0xf2, 0x77, 0x9b, 0xe4, 0x40, 0xa7, 0x5b, 0x47, 0x91, 0xd5, + 0x9d, 0x07, 0x87, 0xdd, 0x83, 0x92, 0x26, 0xb1, 0xde, 0xe1, 0x7e, 0x29, 0x2d, 0x0f, 0xfb, 0x7b, + 0xdd, 0x92, 0x8e, 0x87, 0xad, 0xaf, 0x22, 0x73, 0xa8, 0xd5, 0xa1, 0x25, 0xa3, 0xfd, 0x7d, 0x1a, + 0x0c, 0xf4, 0x91, 0x7c, 0x0a, 0x19, 0x5c, 0xcd, 0x17, 0xe3, 0x8c, 0x2e, 0x7d, 0x05, 0x56, 0xca, + 0x49, 0x50, 0xe5, 0xef, 0x33, 0xc8, 0x46, 0xfb, 0x8b, 0x5c, 0x4a, 0xee, 0xb3, 0xf8, 0xda, 0xfa, + 0x59, 0x38, 0xba, 0x78, 0x5d, 0x23, 0x3b, 0x00, 0x8b, 0xb9, 0x22, 0x1b, 0x89, 0x2a, 0x2e, 0x6f, + 0xa9, 0x4a, 0xe5, 0x3c, 0x4a, 0xbd, 0x7f, 0x17, 0x8a, 0x4b, 0x65, 0x25, 0x49, 0xd5, 0xc4, 0xf0, + 0x54, 0xae, 0x9c, 0xcb, 0x45, 0x76, 0xda, 0x5d, 0x58, 0xc3, 0xef, 0x6e, 0x39, 0x15, 0x51, 0x32, + 0x6e, 0x43, 0x91, 0xb2, 0xa9, 0x27, 0x18, 0xe2, 0x64, 0x1e, 0xfe, 0xf2, 0xe7, 0x79, 0xe5, 0xd2, + 0x19, 0x54, 0x7d, 0xc6, 0xa7, 0xb6, 0x3f, 0x7e, 0xf6, 0x77, 0x35, 0xf5, 0xec, 0x65, 0x55, 0x7b, + 0xfe, 0xb2, 0xaa, 0xfd, 0xf5, 0xb2, 0xaa, 0xfd, 0xfc, 0xaa, 0x9a, 0x7a, 0xfe, 0xaa, 0x9a, 0xfa, + 0xe3, 0x55, 0x35, 0xf5, 0x38, 0xa7, 0xfe, 0x93, 0x18, 0x64, 0xb1, 0x67, 0x6e, 0xfc, 0x13, 0x00, + 0x00, 0xff, 0xff, 0x73, 0x1f, 0x05, 0x4d, 0xb3, 0x0c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -929,6 +945,7 @@ const _ = grpc.SupportPackageIsVersion4 type StoreClient interface { /// Info returns meta information about a store e.g labels that makes that store unique as well as time range that is /// available. + /// Deprecated. Use `thanos.info` instead. Info(ctx context.Context, in *InfoRequest, opts ...grpc.CallOption) (*InfoResponse, error) /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. /// @@ -1017,6 +1034,7 @@ func (c *storeClient) LabelValues(ctx context.Context, in *LabelValuesRequest, o type StoreServer interface { /// Info returns meta information about a store e.g labels that makes that store unique as well as time range that is /// available. + /// Deprecated. Use `thanos.info` instead. Info(context.Context, *InfoRequest) (*InfoResponse, error) /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. /// @@ -1412,6 +1430,15 @@ func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.WithoutReplicaLabels) > 0 { + for iNdEx := len(m.WithoutReplicaLabels) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.WithoutReplicaLabels[iNdEx]) + copy(dAtA[i:], m.WithoutReplicaLabels[iNdEx]) + i = encodeVarintRpc(dAtA, i, uint64(len(m.WithoutReplicaLabels[iNdEx]))) + i-- + dAtA[i] = 0x72 + } + } if m.ShardInfo != nil { { size, err := m.ShardInfo.MarshalToSizedBuffer(dAtA[:i]) @@ -2234,6 +2261,12 @@ func (m *SeriesRequest) Size() (n int) { l = m.ShardInfo.Size() n += 1 + l + sovRpc(uint64(l)) } + if len(m.WithoutReplicaLabels) > 0 { + for _, s := range m.WithoutReplicaLabels { + l = len(s) + n += 1 + l + sovRpc(uint64(l)) + } + } return n } @@ -3300,6 +3333,38 @@ func (m *SeriesRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithoutReplicaLabels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRpc + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithoutReplicaLabels = append(m.WithoutReplicaLabels, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRpc(dAtA[iNdEx:]) diff --git a/pkg/store/storepb/rpc.proto b/pkg/store/storepb/rpc.proto index 72afaba8ed..9e5ebaa693 100644 --- a/pkg/store/storepb/rpc.proto +++ b/pkg/store/storepb/rpc.proto @@ -27,6 +27,7 @@ option (gogoproto.goproto_sizecache_all) = false; service Store { /// Info returns meta information about a store e.g labels that makes that store unique as well as time range that is /// available. + /// Deprecated. Use `thanos.info` instead. rpc Info(InfoRequest) returns (InfoResponse); /// Series streams each Series (Labels and chunk/downsampling chunk) for given label matchers and time range. @@ -62,8 +63,10 @@ message WriteRequest { int64 replica = 3; } +// Deprecated. Use `thanos.info` instead. message InfoRequest {} +// Deprecated. Use `thanos.info` instead. enum StoreType { UNKNOWN = 0; QUERY = 1; @@ -75,6 +78,7 @@ enum StoreType { DEBUG = 6; } +// Deprecated. Use `thanos.info` instead. message InfoResponse { // Deprecated. Use label_sets instead. repeated Label labels = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/thanos-io/thanos/pkg/store/labelpb.ZLabel"]; @@ -117,15 +121,28 @@ message SeriesRequest { // query_hints are the hints coming from the PromQL engine when // requesting a storage.SeriesSet for a given expression. + // As hints name suggest using those is best effort. QueryHints query_hints = 12; // shard_info is used by the querier to request a specific // shard of blocks instead of entire blocks. ShardInfo shard_info = 13; -} + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. + // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by + // this setting (label should be included in sorting and response). + // It is the server responsibility to detect and track what is replica label and what is not. + // This allows faster deduplication by clients. + // NOTE(bwplotka): thanos.info.store.supports_without_replica_labels field has to return true to let client knows + // server supports it. + repeated string without_replica_labels = 14; +} -// Analogous to storage.SelectHints. +// QueryHints represents hints from PromQL that might help to +// pre-aggregate or prepare series for faster use by clients. +// Analogous to storage.SelectHints plus additional info. +// As "hints" name suggests all of the items here are best effort. message QueryHints { // Query step size in milliseconds. int64 step_millis = 1; diff --git a/pkg/testutil/teststore/cient.go b/pkg/testutil/teststore/cient.go new file mode 100644 index 0000000000..6c72225205 --- /dev/null +++ b/pkg/testutil/teststore/cient.go @@ -0,0 +1,25 @@ +package teststore + +import ( + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/thanos/pkg/store/storepb" +) + +type TestClient struct { + storepb.StoreClient + + Name string + + ExtLset []labels.Labels + MinTime, MaxTime int64 + Shardable bool + WithoutReplicaLabelsEnabled bool + IsLocalStore bool +} + +func (c TestClient) LabelSets() []labels.Labels { return c.ExtLset } +func (c TestClient) TimeRange() (mint, maxt int64) { return c.MinTime, c.MaxTime } +func (c TestClient) SupportsSharding() bool { return c.Shardable } +func (c TestClient) SupportsWithoutReplicaLabels() bool { return c.WithoutReplicaLabelsEnabled } +func (c TestClient) String() string { return c.Name } +func (c TestClient) Addr() (string, bool) { return c.Name, c.IsLocalStore } From e83a6af7a52e5c639f51ebfa013fd7fdf4aafa59 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Thu, 15 Dec 2022 12:58:11 +0100 Subject: [PATCH 2/9] Optimized storeWithoutLabels. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ➜ store git:(pre-sort-auto) ✗ benchstat v1.txt v2.txt name old time/op new time/op delta SortWithoutLabels-12 4.02ms ± 2% 1.06ms ± 5% -73.54% (p=0.016 n=5+4) name old alloc/op new alloc/op delta SortWithoutLabels-12 1.04MB ± 0% 0.00MB ±13% -99.99% (p=0.029 n=4+4) name old allocs/op new allocs/op delta SortWithoutLabels-12 30.0k ± 0% 0.0k ± 0% -99.99% (p=0.000 n=5+4) Signed-off-by: bwplotka --- pkg/store/proxy_heap.go | 40 +++++++----------------------------- pkg/store/proxy_heap_test.go | 18 ++++++++++++++++ 2 files changed, 25 insertions(+), 33 deletions(-) diff --git a/pkg/store/proxy_heap.go b/pkg/store/proxy_heap.go index f7664ea946..b861211927 100644 --- a/pkg/store/proxy_heap.go +++ b/pkg/store/proxy_heap.go @@ -12,10 +12,8 @@ import ( "sync" "time" - "github.com/go-kit/log" - "github.com/thanos-io/thanos/pkg/dedup" - "github.com/cespare/xxhash/v2" + "github.com/go-kit/log" "github.com/go-kit/log/level" grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tracing" "github.com/opentracing/opentracing-go" @@ -752,39 +750,15 @@ func newEagerRespSet( return ret } -// Move all wanted labels back and remove. -// TODO(bwplotka): Consider microoptimizing this. func rmLabels(l labels.Labels, labelsToRemove map[string]struct{}) labels.Labels { - sort.Slice(l, func(i, j int) bool { - if _, ok := labelsToRemove[l[i].Name]; ok { - return false - } - if _, ok := labelsToRemove[l[j].Name]; ok { - return true - } - // Ensure that dedup marker goes just right before the replica labels. - if l[i].Name == dedup.PushdownMarker.Name { - return false - } - if l[j].Name == dedup.PushdownMarker.Name { - return true - } - return l[i].Name < l[j].Name - }) - - var totalToRemove int - for i := 0; i < len(labelsToRemove); i++ { - if len(l)-i == 0 { - break - } - - if _, ok := labelsToRemove[l[len(l)-i-1].Name]; ok { - totalToRemove++ + for i := 0; i < len(l); i++ { + if _, ok := labelsToRemove[l[i].Name]; !ok { + continue } + l = append(l[:i], l[i+1:]...) + i-- } - - // Strip all present labels to remove. - return l[:len(l)-totalToRemove] + return l } // sortWithoutLabels removes given labels from series and re-sorts the series responses that the same diff --git a/pkg/store/proxy_heap_test.go b/pkg/store/proxy_heap_test.go index 5ee268e59b..ad8a7210af 100644 --- a/pkg/store/proxy_heap_test.go +++ b/pkg/store/proxy_heap_test.go @@ -104,3 +104,21 @@ func labelsFromStrings(ss ...string) labels.Labels { return res } + +func BenchmarkSortWithoutLabels(b *testing.B) { + resps := make([]*storepb.SeriesResponse, 1e4) + labelsToRemove := map[string]struct{}{ + "a": {}, "b": {}, + } + + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + b.StopTimer() + for i := 0; i < 1e4; i++ { + resps[i] = storeSeriesResponse(b, labels.FromStrings("a", "1", "b", "replica-1", "c", "replica-1", "d", "1")) + } + b.StartTimer() + sortWithoutLabels(resps, labelsToRemove) + } +} From 9428625cf7a9137b3f2e6bb3b8fed0de831b7bb8 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Mon, 19 Dec 2022 15:46:47 +0100 Subject: [PATCH 3/9] Added back dedup with simple hack for tmp use. Signed-off-by: bwplotka --- pkg/dedup/iter.go | 78 ++++++++++++- pkg/dedup/iter_test.go | 231 ++++++++++++++++++++++++-------------- pkg/query/iter.go | 81 ++----------- pkg/query/iter_test.go | 68 ----------- pkg/query/querier.go | 22 +++- pkg/query/querier_test.go | 2 +- 6 files changed, 247 insertions(+), 235 deletions(-) delete mode 100644 pkg/query/iter_test.go diff --git a/pkg/dedup/iter.go b/pkg/dedup/iter.go index 4ffbfd5234..a7b195783f 100644 --- a/pkg/dedup/iter.go +++ b/pkg/dedup/iter.go @@ -10,12 +10,12 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/thanos-io/thanos/pkg/store/storepb" ) type dedupSeriesSet struct { - set storage.SeriesSet - replicaLabels map[string]struct{} - isCounter bool + set storage.SeriesSet + isCounter bool replicas []storage.Series // Pushed down series. Currently, they are being handled in a specific way. @@ -37,11 +37,77 @@ func isCounter(f string) bool { return f == "increase" || f == "rate" || f == "irate" || f == "resets" } +// NewOverlapSplit splits overlapping chunks into separate series entry, so existing algorithm can work as usual. +// We cannot do this in dedup.SeriesSet as it iterates over samples already. +// TODO(bwplotka): Remove when we move to per chunk deduplication code. +// We expect non-duplicated series with sorted chunks by min time (possibly overlapped). +func NewOverlapSplit(set storepb.SeriesSet) storepb.SeriesSet { + return &overlapSplitSet{set: set, ok: true} +} + +type overlapSplitSet struct { + ok bool + set storepb.SeriesSet + + currLabels labels.Labels + currI int + replicas [][]storepb.AggrChunk +} + +func (o *overlapSplitSet) Next() bool { + if !o.ok { + return false + } + + o.currI++ + if o.currI < len(o.replicas) { + return true + } + + o.currI = 0 + o.replicas = o.replicas[:0] + o.replicas = append(o.replicas, nil) + + o.ok = o.set.Next() + if !o.ok { + return false + } + + var chunks []storepb.AggrChunk + o.currLabels, chunks = o.set.At() + if len(chunks) == 0 { + return true + } + + o.replicas[0] = append(o.replicas[0], chunks[0]) + +chunksLoop: + for i := 1; i < len(chunks); i++ { + currMinTime := chunks[i].MinTime + for ri := range o.replicas { + if len(o.replicas[ri]) == 0 || o.replicas[ri][len(o.replicas[ri])-1].MaxTime < currMinTime { + o.replicas[ri] = append(o.replicas[ri], chunks[i]) + continue chunksLoop + } + } + o.replicas = append(o.replicas, []storepb.AggrChunk{chunks[i]}) // Not found, add to a new "fake" series. + } + return true +} + +func (o *overlapSplitSet) At() (labels.Labels, []storepb.AggrChunk) { + return o.currLabels, o.replicas[o.currI] +} + +func (o *overlapSplitSet) Err() error { + return o.set.Err() +} + // NewSeriesSet returns seriesSet that deduplicates the same series. -// The series in series set are expected to be sorted by all labels. -func NewSeriesSet(set storage.SeriesSet, replicaLabels map[string]struct{}, f string, pushdownEnabled bool) storage.SeriesSet { +// The series in series set are expected be sorted by all labels. +func NewSeriesSet(set storage.SeriesSet, f string, pushdownEnabled bool) storage.SeriesSet { // TODO: remove dependency on knowing whether it is a counter. - s := &dedupSeriesSet{pushdownEnabled: pushdownEnabled, set: set, replicaLabels: replicaLabels, isCounter: isCounter(f), f: f} + s := &dedupSeriesSet{pushdownEnabled: pushdownEnabled, set: set, isCounter: isCounter(f), f: f} s.ok = s.set.Next() if s.ok { s.peek = s.set.At() diff --git a/pkg/dedup/iter_test.go b/pkg/dedup/iter_test.go index a54011b0f1..1a28f98872 100644 --- a/pkg/dedup/iter_test.go +++ b/pkg/dedup/iter_test.go @@ -145,40 +145,158 @@ var expectedRealSeriesWithStaleMarkerDeduplicatedForRate = []sample{ {t: 1587693472139, v: 509720}, {t: 1587693487139, v: 509979}, {t: 1587693502139, v: 510189}, {t: 1587693517139, v: 510505}, {t: 1587693532139, v: 510661}, {t: 1587693547139, v: 510866}, {t: 1587693562139, v: 511131}, {t: 1587693577139, v: 511321}, {t: 1587693592139, v: 511495}, } +type chunkedSeries struct { + lset labels.Labels + chunks []storepb.AggrChunk +} + +type chunkedSeriesSet struct { + series []chunkedSeries + i int +} + +func newChunkedSeriesSet(s []chunkedSeries) *chunkedSeriesSet { + return &chunkedSeriesSet{series: s, i: -1} +} + +func (s *chunkedSeriesSet) Next() bool { + if s.i >= len(s.series)-1 { + return false + } + s.i++ + return true +} + +func (*chunkedSeriesSet) Err() error { + return nil +} + +func (s *chunkedSeriesSet) At() (labels.Labels, []storepb.AggrChunk) { + return s.series[s.i].lset, s.series[s.i].chunks +} + +func toChunkedSeriesSlice(t testing.TB, set storepb.SeriesSet) []chunkedSeries { + var ret []chunkedSeries + for set.Next() { + lset, chunks := set.At() + ret = append(ret, chunkedSeries{ + lset: lset, chunks: chunks, + }) + } + testutil.Ok(t, set.Err()) + return ret +} + +func TestOverlapSplitSet(t *testing.T) { + input := []chunkedSeries{ + { + lset: labels.Labels{{Name: "a", Value: "1_empty"}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "2_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 100}, {MinTime: 110, MaxTime: 300}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "3_tworeplicas"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 0, MaxTime: 30}, {MinTime: 21, MaxTime: 50}, {MinTime: 31, MaxTime: 60}, {MinTime: 100, MaxTime: 160}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "4_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}, {MinTime: 56, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "5_minimaloverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}, {MinTime: 55, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 0, MaxTime: 30}, {MinTime: 1, MaxTime: 15}, {MinTime: 2, MaxTime: 36}, {MinTime: 16, MaxTime: 200}, + {MinTime: 21, MaxTime: 50}, {MinTime: 31, MaxTime: 60}, {MinTime: 100, MaxTime: 160}}, + }, + } + exp := []chunkedSeries{ + { + lset: labels.Labels{{Name: "a", Value: "1_empty"}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "2_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 100}, {MinTime: 110, MaxTime: 300}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "3_tworeplicas"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 50}, {MinTime: 100, MaxTime: 160}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "3_tworeplicas"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 30}, {MinTime: 31, MaxTime: 60}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "4_nonoverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}, {MinTime: 56, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "5_minimaloverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 50, MaxTime: 55}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "5_minimaloverlap"}}, + chunks: []storepb.AggrChunk{{MinTime: 55, MaxTime: 100}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 20}, {MinTime: 21, MaxTime: 50}, {MinTime: 100, MaxTime: 160}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 0, MaxTime: 30}, {MinTime: 31, MaxTime: 60}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 1, MaxTime: 15}, {MinTime: 16, MaxTime: 200}}, + }, + { + lset: labels.Labels{{Name: "a", Value: "6_fourreplica"}}, + chunks: []storepb.AggrChunk{{MinTime: 2, MaxTime: 36}}, + }, + } + + got := toChunkedSeriesSlice(t, NewOverlapSplit(newChunkedSeriesSet(input))) + testutil.Equals(t, exp, got) +} + func TestDedupSeriesSet(t *testing.T) { for _, tcase := range []struct { - name string - input []series - exp []series - dedupLabels map[string]struct{} - isCounter bool + name string + input []series + exp []series + isCounter bool }{ { name: "Single dedup label", input: []series{ { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{200000, 5}, {210000, 6}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "replica", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, }, @@ -191,10 +309,6 @@ func TestDedupSeriesSet(t *testing.T) { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, @@ -204,52 +318,18 @@ func TestDedupSeriesSet(t *testing.T) { samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - }, - }, - { - // Regression tests against: https://github.com/thanos-io/thanos/issues/2645. - // We were panicking on requests with more replica labels than overall labels in any series. - name: "Regression tests against #2645", - input: []series{ - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}}, - samples: []sample{{60000, 3}, {70000, 4}}, - }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}}, - samples: []sample{{100000, 10}, {150000, 20}}, - }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, - }, - exp: []series{ - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, - samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}, {100000, 10}, {150000, 20}}, - }, - { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, - samples: []sample{{10000, 1}, {20000, 2}}, - }, - }, - dedupLabels: map[string]struct{}{"replica": {}, "replica2": {}, "replica3": {}, "replica4": {}, "replica5": {}, "replica6": {}, "replica7": {}}, }, { name: "Multi dedup label", input: []series{ { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}, {Name: "replicaA", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}, {Name: "replicaA", Value: "replica-2"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}, {Name: "replicaA", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{200000, 5}, {210000, 6}}, }, { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "d", Value: "4"}}, @@ -258,13 +338,13 @@ func TestDedupSeriesSet(t *testing.T) { lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}, {Name: "replica", Value: "replica-1"}, {Name: "replicaA", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "4"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}, {Name: "replicaA", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-3"}, {Name: "replicaA", Value: "replica-3"}}, + lset: labels.Labels{{Name: "a", Value: "2"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, }, @@ -290,19 +370,15 @@ func TestDedupSeriesSet(t *testing.T) { samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - "replicaA": {}, - }, }, { name: "Multi dedup label - some series don't have all dedup labels", input: []series{ { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-1"}, {Name: "replicaA", Value: "replica-1"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{10000, 1}, {20000, 2}}, }, { - lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}, {Name: "replica", Value: "replica-2"}}, + lset: labels.Labels{{Name: "a", Value: "1"}, {Name: "c", Value: "3"}}, samples: []sample{{60000, 3}, {70000, 4}}, }, }, @@ -312,10 +388,6 @@ func TestDedupSeriesSet(t *testing.T) { samples: []sample{{10000, 1}, {20000, 2}, {60000, 3}, {70000, 4}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - "replicaA": {}, - }, }, { // Regression test against https://github.com/thanos-io/thanos/issues/2401. @@ -328,7 +400,7 @@ func TestDedupSeriesSet(t *testing.T) { isCounter: true, input: []series{ { - lset: labels.Labels{{Name: "replica", Value: "01"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10000, 8.0}, // Smaller timestamp, this will be chosen. CurrValue = 8.0. {20000, 9.0}, // Same. CurrValue = 9.0. @@ -341,7 +413,7 @@ func TestDedupSeriesSet(t *testing.T) { {100000, 9 + 6.0}, }, }, { - lset: labels.Labels{{Name: "replica", Value: "02"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10001, 8.0}, // Penalty 5000 will be added. // 20001 was app reset. No sample, because stale marker but removed by downsample.CounterSeriesIterator. Penalty 2 * (20000 - 10000) will be added. @@ -355,13 +427,10 @@ func TestDedupSeriesSet(t *testing.T) { }, exp: []series{ { - lset: labels.Labels{}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{{10000, 8}, {20000, 9}, {45001, 9}, {55001, 10}, {65001, 11}, {90000, 14}, {100000, 15}}, }, }, - dedupLabels: map[string]struct{}{ - "replica": {}, - }, }, { // Same thing but not for counter should not adjust anything. @@ -369,12 +438,12 @@ func TestDedupSeriesSet(t *testing.T) { isCounter: false, input: []series{ { - lset: labels.Labels{{Name: "replica", Value: "01"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10000, 8.0}, {20000, 9.0}, {50001, 9 + 1.0}, {60000, 9 + 2.0}, {70000, 9 + 3.0}, {80000, 9 + 4.0}, {90000, 9 + 5.0}, {100000, 9 + 6.0}, }, }, { - lset: labels.Labels{{Name: "replica", Value: "02"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {10001, 8.0}, {45001, 8 + 0.5}, {55001, 8 + 1.5}, {65001, 8 + 2.5}, }, @@ -382,11 +451,10 @@ func TestDedupSeriesSet(t *testing.T) { }, exp: []series{ { - lset: labels.Labels{}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{{10000, 8}, {20000, 9}, {45001, 8.5}, {55001, 9.5}, {65001, 10.5}, {90000, 14}, {100000, 15}}, }, }, - dedupLabels: map[string]struct{}{"replica": {}}, }, { // Regression test on real data against https://github.com/thanos-io/thanos/issues/2401. @@ -395,7 +463,7 @@ func TestDedupSeriesSet(t *testing.T) { isCounter: true, input: []series{ { - lset: labels.Labels{{Name: "replica", Value: "01"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {t: 1587690007139, v: 461993}, {t: 1587690022139, v: 462164}, {t: 1587690037139, v: 462409}, {t: 1587690052139, v: 462662}, {t: 1587690067139, v: 462824}, {t: 1587690082139, v: 462987}, {t: 1587690097155, v: 463108}, {t: 1587690112139, v: 463261}, {t: 1587690127139, v: 463465}, {t: 1587690142139, v: 463642}, {t: 1587690157139, v: 463823}, {t: 1587690172139, v: 464065}, {t: 1587690187139, v: 464333}, {t: 1587690202139, v: 464566}, {t: 1587690217139, v: 464811}, {t: 1587690232140, v: 465032}, {t: 1587690247139, v: 465229}, {t: 1587690262139, v: 465445}, {t: 1587690277139, v: 465700}, {t: 1587690292139, v: 465884}, @@ -423,7 +491,7 @@ func TestDedupSeriesSet(t *testing.T) { {t: 1587693517139, v: 510498}, {t: 1587693532139, v: 510654}, {t: 1587693547139, v: 510859}, {t: 1587693562139, v: 511124}, {t: 1587693577139, v: 511314}, {t: 1587693592139, v: 511488}, }, }, { - lset: labels.Labels{{Name: "replica", Value: "02"}}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: []sample{ {t: 1587690005791, v: 461968}, {t: 1587690020791, v: 462151}, {t: 1587690035797, v: 462336}, {t: 1587690050791, v: 462650}, {t: 1587690065791, v: 462813}, {t: 1587690080791, v: 462987}, {t: 1587690095791, v: 463095}, {t: 1587690110791, v: 463247}, {t: 1587690125791, v: 463440}, {t: 1587690140791, v: 463642}, {t: 1587690155791, v: 463811}, {t: 1587690170791, v: 464027}, {t: 1587690185791, v: 464308}, {t: 1587690200791, v: 464514}, {t: 1587690215791, v: 464798}, {t: 1587690230791, v: 465018}, {t: 1587690245791, v: 465215}, {t: 1587690260813, v: 465431}, {t: 1587690275791, v: 465651}, {t: 1587690290791, v: 465870}, {t: 1587690305791, v: 466070}, {t: 1587690320792, v: 466248}, @@ -452,11 +520,10 @@ func TestDedupSeriesSet(t *testing.T) { }, exp: []series{ { - lset: labels.Labels{}, + lset: labels.Labels{{Name: "a", Value: "1"}}, samples: expectedRealSeriesWithStaleMarkerDeduplicatedForRate, }, }, - dedupLabels: map[string]struct{}{"replica": {}}, }, } { t.Run(tcase.name, func(t *testing.T) { @@ -465,7 +532,7 @@ func TestDedupSeriesSet(t *testing.T) { if tcase.isCounter { f = "rate" } - dedupSet := NewSeriesSet(&mockedSeriesSet{series: tcase.input}, tcase.dedupLabels, f, false) + dedupSet := NewSeriesSet(&mockedSeriesSet{series: tcase.input}, f, false) var ats []storage.Series for dedupSet.Next() { ats = append(ats, dedupSet.At()) diff --git a/pkg/query/iter.go b/pkg/query/iter.go index 272113373d..e30744a580 100644 --- a/pkg/query/iter.go +++ b/pkg/query/iter.go @@ -4,8 +4,6 @@ package query import ( - "sort" - "github.com/pkg/errors" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" @@ -18,88 +16,27 @@ import ( ) // promSeriesSet implements the SeriesSet interface of the Prometheus storage -// package on top of our storepb SeriesSet. +// package on top of our storepb SeriesSet. Overlapping chunks will be naively deduplicated (random selection). type promSeriesSet struct { - set storepb.SeriesSet - done bool + set storepb.SeriesSet mint, maxt int64 aggrs []storepb.Aggr - initiated bool - - currLset labels.Labels - currChunks []storepb.AggrChunk warns storage.Warnings } func (s *promSeriesSet) Next() bool { - if !s.initiated { - s.initiated = true - s.done = s.set.Next() - } - - if !s.done { - return false - } - - // storage.Series is stricter than storepb.SeriesSet: it requires set to iterate over full series. - s.currLset, s.currChunks = s.set.At() - for { - s.done = s.set.Next() - if !s.done { - break - } - nextLset, nextChunks := s.set.At() - if labels.Compare(s.currLset, nextLset) != 0 { - break - } - s.currChunks = append(s.currChunks, nextChunks...) - } - - // Samples (so chunks as well) have to be sorted by time. - // TODO(bwplotka): Benchmark if we can do better. - // For example we could iterate in above loop and write our own binary search based insert sort. - // We could also remove duplicates in same loop. - sort.Slice(s.currChunks, func(i, j int) bool { - return s.currChunks[i].MinTime < s.currChunks[j].MinTime - }) - - // Proxy handles some exact duplicates in chunk between different series, let's handle duplicates within single series now as well. - // We don't need to decode those. - s.currChunks = removeExactDuplicates(s.currChunks) - - // TODO(bwplotka): Create dedup iterator for those chunks. - return true -} - -// removeExactDuplicates returns chunks without 1:1 duplicates. -// NOTE: input chunks has to be sorted by minTime. -func removeExactDuplicates(chks []storepb.AggrChunk) []storepb.AggrChunk { - if len(chks) <= 1 { - return chks - } - head := 0 - for i, c := range chks[1:] { - if chks[head].Compare(c) == 0 { - continue - } - head++ - if i+1 == head { - // `chks[head] == chks[i+1] == c` so this is a no-op. - // This way we get no copies in case the input had no duplicates. - continue - } - chks[head] = c - } - return chks[:head+1] + return s.set.Next() } func (s *promSeriesSet) At() storage.Series { - if !s.initiated || s.set.Err() != nil { + if s.set.Err() != nil { return nil } - return newChunkSeries(s.currLset, s.currChunks, s.mint, s.maxt, s.aggrs) + + currLset, currChunks := s.set.At() + return newChunkSeries(currLset, currChunks, s.mint, s.maxt, s.aggrs) } func (s *promSeriesSet) Err() error { @@ -129,11 +66,11 @@ func (s *storeSeriesSet) Next() bool { return true } -func (storeSeriesSet) Err() error { +func (*storeSeriesSet) Err() error { return nil } -func (s storeSeriesSet) At() (labels.Labels, []storepb.AggrChunk) { +func (s *storeSeriesSet) At() (labels.Labels, []storepb.AggrChunk) { return s.series[s.i].PromLabels(), s.series[s.i].Chunks } diff --git a/pkg/query/iter_test.go b/pkg/query/iter_test.go deleted file mode 100644 index 5282f102a1..0000000000 --- a/pkg/query/iter_test.go +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package query - -import ( - "fmt" - "testing" - - "github.com/efficientgo/core/testutil" - "github.com/thanos-io/thanos/pkg/store/storepb" -) - -func TestRemoveExactDuplicates(t *testing.T) { - for _, tc := range []struct { - name string - chks []storepb.AggrChunk - }{ - {name: "empty slice", chks: []storepb.AggrChunk{}}, - {name: "single element slice", chks: aggrChunkForMinTimes(0)}, - {name: "slice with duplicates", chks: aggrChunkForMinTimes(0, 0, 2, 30, 31, 31, 40)}, - {name: "slice without duplicates", chks: aggrChunkForMinTimes(0, 1, 2, 3, 5, 7, 8)}, - } { - t.Run(tc.name, func(t *testing.T) { - originalChks := make([]storepb.AggrChunk, len(tc.chks)) - copy(originalChks, tc.chks) - chks := removeExactDuplicates(tc.chks) - missingChunk := isChunksSubset(originalChks, chks) - testutil.Assert(t, missingChunk == nil, fmt.Sprintf("chunk %q missing in output", missingChunk.String())) - unexpectedChunk := isChunksSubset(chks, originalChks) - testutil.Assert(t, unexpectedChunk == nil, fmt.Sprintf("unexpected chunk %q does not appear in the input", unexpectedChunk.String())) - - if len(chks) > 0 { - chk1 := chks[0] - for _, chk2 := range chks[1:] { - testutil.Assert(t, chk2.Compare(chk1) != 0, fmt.Sprintf("chunk %q appears twice in output", chk1.String())) - chk1 = chk2 - } - } - }) - } -} - -func aggrChunkForMinTimes(minTimes ...int64) []storepb.AggrChunk { - chks := make([]storepb.AggrChunk, len(minTimes)) - for i, minTime := range minTimes { - chks[i] = storepb.AggrChunk{MinTime: minTime} - } - return chks -} - -// isChunksSubset returns nil if all chunks in chks1 also appear in chks2, -// otherwise returns a chunk in chks1 that does not apper in chks2. -func isChunksSubset(chks1, chks2 []storepb.AggrChunk) *storepb.AggrChunk { - for _, chk1 := range chks1 { - found := false - for _, chk2 := range chks2 { - if chk2.Compare(chk1) == 0 { - found = true - break - } - } - if !found { - return &chk1 - } - } - return nil -} diff --git a/pkg/query/querier.go b/pkg/query/querier.go index e1a443c4a9..9e2cf6280e 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" promgate "github.com/prometheus/prometheus/util/gate" + "github.com/thanos-io/thanos/pkg/dedup" "github.com/thanos-io/thanos/pkg/extprom" "github.com/thanos-io/thanos/pkg/gate" "github.com/thanos-io/thanos/pkg/store" @@ -391,19 +392,28 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . } } - // promSeriesSet is deduplicating all overlapped chunks. + if !q.isDedupEnabled() { + return &promSeriesSet{ + mint: q.mint, + maxt: q.maxt, + set: newStoreSeriesSet(resp.seriesSet), + aggrs: aggrs, + warns: warns, + }, resp.seriesSetStats, nil + } + + // TODO(bwplotka): Move to deduplication on chunk level inside promSeriesSet, similar to what we have in dedup.NewDedupChunkMerger(). + // This however require big refactor, caring about correct AggrChunk to iterator conversion, pushdown logic and counter reset apply. + // For now we apply simple logic that splits potential overlapping chunks into separate replica series, so we can split the work. set := &promSeriesSet{ mint: q.mint, maxt: q.maxt, - set: newStoreSeriesSet(resp.seriesSet), + set: dedup.NewOverlapSplit(newStoreSeriesSet(resp.seriesSet)), aggrs: aggrs, warns: warns, } - // // The merged series set assembles all potentially-overlapping time ranges of the same series into a single one. - //- // TODO(bwplotka): We could potentially dedup on chunk level, use chunk iterator for that when available. - //- return dedup.NewSeriesSet(set, q.replicaLabels, hints.Func, q.enableQueryPushdown), resp.seriesSetStats, nil - return set, resp.seriesSetStats, nil + return dedup.NewSeriesSet(set, hints.Func, q.enableQueryPushdown), resp.seriesSetStats, nil } // LabelValues returns all potential values for a label name. diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 6cf52efdac..bb06bc12d7 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -872,7 +872,7 @@ func testSelectResponse(t *testing.T, expected []series, res storage.SeriesSet) series = append(series, res.At()) } testutil.Ok(t, res.Err()) - testutil.Equals(t, len(expected), len(series), "got %v", func() string { + testutil.Equals(t, len(expected), len(series), "got %v series", func() string { var ret []string for _, s := range series { ret = append(ret, s.Labels().String()) From 7a97304eab2d7c402095b649249e8d48143765e6 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Mon, 19 Dec 2022 17:40:04 +0100 Subject: [PATCH 4/9] Heap fix. Signed-off-by: bwplotka --- go.mod | 2 +- pkg/query/querier.go | 3 + pkg/query/querier_test.go | 4 +- pkg/query/test_test.go | 2 +- pkg/store/proxy.go | 2 +- pkg/store/proxy_heap.go | 145 ++++++++++++++++------------------- pkg/store/proxy_heap_test.go | 2 +- pkg/store/proxy_test.go | 141 ++++++++++++++++++++++++++-------- 8 files changed, 181 insertions(+), 120 deletions(-) diff --git a/go.mod b/go.mod index 84383ca299..908563ee3e 100644 --- a/go.mod +++ b/go.mod @@ -121,6 +121,7 @@ require ( ) require ( + github.com/google/go-cmp v0.5.9 go.opentelemetry.io/contrib/propagators/autoprop v0.34.0 golang.org/x/exp v0.0.0-20221212164502-fae10dda9338 ) @@ -179,7 +180,6 @@ require ( github.com/gogo/googleapis v1.4.0 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/golang-jwt/jwt/v4 v4.4.1 // indirect - github.com/google/go-cmp v0.5.9 // indirect github.com/google/go-querystring v1.1.0 // indirect github.com/google/pprof v0.0.0-20221212185716-aee1124e3a93 // indirect github.com/google/uuid v1.3.0 // indirect diff --git a/pkg/query/querier.go b/pkg/query/querier.go index 9e2cf6280e..5dda9f45a0 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -5,6 +5,7 @@ package query import ( "context" + "fmt" "strings" "sync" "time" @@ -392,6 +393,8 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . } } + fmt.Println(resp.seriesSet) + if !q.isDedupEnabled() { return &promSeriesSet{ mint: q.mint, diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index bb06bc12d7..6bc7df3efc 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -29,13 +29,11 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/gate" - "github.com/thanos-io/thanos/pkg/testutil/teststore" - - "github.com/efficientgo/core/testutil" "github.com/thanos-io/thanos/pkg/component" "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" + "github.com/thanos-io/thanos/pkg/testutil/teststore" ) type sample struct { diff --git a/pkg/query/test_test.go b/pkg/query/test_test.go index a5500b3b9c..b9408534ca 100644 --- a/pkg/query/test_test.go +++ b/pkg/query/test_test.go @@ -15,7 +15,6 @@ import ( "testing" "time" - "github.com/efficientgo/core/testutil" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -23,6 +22,7 @@ import ( "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/teststorage" ) var ( diff --git a/pkg/store/proxy.go b/pkg/store/proxy.go index bed8ed11f3..108d09c015 100644 --- a/pkg/store/proxy.go +++ b/pkg/store/proxy.go @@ -388,7 +388,7 @@ func labelSetsMatch(matchers []*labels.Matcher, lset ...labels.Labels) bool { for _, ls := range lset { notMatched := false for _, m := range matchers { - if lv := ls.Get(m.Name); lv != "" && !m.Matches(lv) { + if lv := ls.Get(m.Name); !m.Matches(lv) { notMatched = true break } diff --git a/pkg/store/proxy_heap.go b/pkg/store/proxy_heap.go index b861211927..9099a3b39c 100644 --- a/pkg/store/proxy_heap.go +++ b/pkg/store/proxy_heap.go @@ -26,104 +26,93 @@ import ( "github.com/thanos-io/thanos/pkg/tracing" ) -// dedupResponseHeap is a wrapper around ProxyResponseHeap -// that removes duplicated identical chunks identified by the same labelset and checksum. -// It uses a hashing function to do that. type dedupResponseHeap struct { h *ProxyResponseHeap - responses []*storepb.SeriesResponse + bufferedSameSeries []*storepb.SeriesResponse - previousResponse *storepb.SeriesResponse - previousNext bool + bufferedResp []*storepb.SeriesResponse + buffRespI int + + prev *storepb.SeriesResponse + ok bool } +// NewDedupResponseHeap returns a wrapper around ProxyResponseHeap that merged duplicated series messages into one. +// It also deduplicates identical chunks identified by the same checksum from each series message. func NewDedupResponseHeap(h *ProxyResponseHeap) *dedupResponseHeap { + ok := h.Next() + var prev *storepb.SeriesResponse + if ok { + prev = h.At() + } return &dedupResponseHeap{ - h: h, - previousNext: h.Next(), + h: h, + ok: ok, + prev: prev, } } func (d *dedupResponseHeap) Next() bool { - d.responses = d.responses[:0] - - // If there is something buffered that is *not* a series. - if d.previousResponse != nil && d.previousResponse.GetSeries() == nil { - d.responses = append(d.responses, d.previousResponse) - d.previousResponse = nil - d.previousNext = d.h.Next() - return len(d.responses) > 0 || d.previousNext + if d.buffRespI+1 < len(d.bufferedResp) { + d.buffRespI++ + return true } - var resp *storepb.SeriesResponse - var nextHeap bool - - // If buffered then use it. - if d.previousResponse != nil { - resp = d.previousResponse - d.previousResponse = nil - } else { - // If not buffered then check whether there is anything. - nextHeap = d.h.Next() - if !nextHeap { - return false - } - resp = d.h.At() + if !d.ok && d.prev == nil { + return false } - // Append buffered or retrieved response. - d.responses = append(d.responses, resp) - - // Update previousNext. - defer func(next *bool) { - d.previousNext = *next - }(&nextHeap) - - if resp.GetSeries() == nil { - return len(d.responses) > 0 || d.previousNext - } + d.buffRespI = 0 + d.bufferedResp = d.bufferedResp[:0] + d.bufferedSameSeries = d.bufferedSameSeries[:0] + var s *storepb.SeriesResponse for { - nextHeap = d.h.Next() - if !nextHeap { - break + if d.prev == nil { + d.ok = d.h.Next() + if !d.ok { + if len(d.bufferedSameSeries) > 0 { + d.bufferedResp = append(d.bufferedResp, chainSeriesAndRemIdenticalChunks(d.bufferedSameSeries)) + } + return len(d.bufferedResp) > 0 + } + s = d.h.At() + } else { + s = d.prev + d.prev = nil } - resp = d.h.At() - if resp.GetSeries() == nil { - d.previousResponse = resp - break + + if s.GetSeries() == nil { + d.bufferedResp = append(d.bufferedResp, s) + continue } - lbls := resp.GetSeries().Labels - lastLbls := d.responses[len(d.responses)-1].GetSeries().Labels + if len(d.bufferedSameSeries) == 0 { + d.bufferedSameSeries = append(d.bufferedSameSeries, s) + continue + } - if labels.Compare(labelpb.ZLabelsToPromLabels(lbls), labelpb.ZLabelsToPromLabels(lastLbls)) == 0 { - d.responses = append(d.responses, resp) - } else { - // This one is different. It will be taken care of via the next Next() call. - d.previousResponse = resp - break + lbls := d.bufferedSameSeries[0].GetSeries().Labels + atLbls := s.GetSeries().Labels + + if labels.Compare(labelpb.ZLabelsToPromLabels(lbls), labelpb.ZLabelsToPromLabels(atLbls)) == 0 { + d.bufferedSameSeries = append(d.bufferedSameSeries, s) + continue } - } - return len(d.responses) > 0 || d.previousNext -} + d.bufferedResp = append(d.bufferedResp, chainSeriesAndRemIdenticalChunks(d.bufferedSameSeries)) + d.prev = s -func (d *dedupResponseHeap) At() *storepb.SeriesResponse { - if len(d.responses) == 0 { - panic("BUG: At() called with no responses; please call At() only if Next() returns true") - } else if len(d.responses) == 1 { - return d.responses[0] + return true } +} +func chainSeriesAndRemIdenticalChunks(series []*storepb.SeriesResponse) *storepb.SeriesResponse { chunkDedupMap := map[uint64]*storepb.AggrChunk{} - for _, resp := range d.responses { - if resp.GetSeries() == nil { - continue - } - for _, chk := range resp.GetSeries().Chunks { + for _, s := range series { + for _, chk := range s.GetSeries().Chunks { for _, field := range []*storepb.Chunk{ chk.Raw, chk.Count, chk.Max, chk.Min, chk.Sum, chk.Counter, } { @@ -146,7 +135,7 @@ func (d *dedupResponseHeap) At() *storepb.SeriesResponse { // If no chunks were requested. if len(chunkDedupMap) == 0 { - return d.responses[0] + return series[0] } finalChunks := make([]storepb.AggrChunk, 0, len(chunkDedupMap)) @@ -158,14 +147,12 @@ func (d *dedupResponseHeap) At() *storepb.SeriesResponse { return finalChunks[i].Compare(finalChunks[j]) > 0 }) - // Guaranteed to be a series because Next() only buffers one - // warning at a time that gets handled in the beginning. - lbls := d.responses[0].GetSeries().Labels + series[0].GetSeries().Chunks = finalChunks + return series[0] +} - return storepb.NewSeriesResponse(&storepb.Series{ - Labels: lbls, - Chunks: finalChunks, - }) +func (d *dedupResponseHeap) At() *storepb.SeriesResponse { + return d.bufferedResp[d.buffRespI] } // ProxyResponseHeap is a heap for storepb.SeriesSets. @@ -225,9 +212,7 @@ type ProxyResponseHeapNode struct { } // NewProxyResponseHeap returns heap that k-way merge series together. -// In case of duplicates, series might return in random order without changed, chained or deduplicated chunks. -// TODO(bwplotka): Consider moving deduplication routines in single place for readability. Currently it's scattered in -// NewDedupResponseHeap, in removeDuplicates in promSeriesSet and dedup.NewSeriesSet. +// It's agnostic to duplicates and overlaps, it forwards all duplicated series in random order. func NewProxyResponseHeap(seriesSets ...respSet) *ProxyResponseHeap { ret := make(ProxyResponseHeap, 0, len(seriesSets)) diff --git a/pkg/store/proxy_heap_test.go b/pkg/store/proxy_heap_test.go index ad8a7210af..e00d9222a7 100644 --- a/pkg/store/proxy_heap_test.go +++ b/pkg/store/proxy_heap_test.go @@ -3,11 +3,11 @@ package store import ( "testing" + "github.com/efficientgo/core/testutil" "github.com/prometheus/prometheus/model/labels" "github.com/thanos-io/thanos/pkg/dedup" "github.com/thanos-io/thanos/pkg/errors" "github.com/thanos-io/thanos/pkg/store/storepb" - "github.com/thanos-io/thanos/pkg/testutil" ) func TestSortWithoutLabels(t *testing.T) { diff --git a/pkg/store/proxy_test.go b/pkg/store/proxy_test.go index 17a3602a0e..aa80cc70c3 100644 --- a/pkg/store/proxy_test.go +++ b/pkg/store/proxy_test.go @@ -189,7 +189,7 @@ func TestProxyStore_Series(t *testing.T) { expectedSeries: []rawSeries{ { lset: labels.FromStrings("a", "a"), - chunks: [][]sample{{{4, 3}}, {{0, 0}, {2, 1}, {3, 2}}}, // No sort merge. + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{4, 3}}}, }, }, }, @@ -313,6 +313,73 @@ func TestProxyStore_Series(t *testing.T) { }, expectedWarningsLen: 2, }, + { + title: "storeAPI available for time range; available two duplicated series for ext=1 external label matcher from 2 storeAPIs", + storeAPIs: []Client{ + &teststore.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + &teststore.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{1, 4}, {2, 5}, {3, 6}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + }, + req: &storepb.SeriesRequest{ + MinTime: 1, + MaxTime: 300, + Matchers: []storepb.LabelMatcher{{Name: "ext", Value: "1", Type: storepb.LabelMatcher_EQ}}, + }, + expectedSeries: []rawSeries{ + { + lset: labels.FromStrings("a", "a"), + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{1, 4}, {2, 5}, {3, 6}}}, + }, + }, + }, + { + title: "storeAPI available for time range; available a few duplicated series for ext=1 external label matcher from the same storeAPIs", + storeAPIs: []Client{ + &teststore.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), // Exact same chunk should be removed. + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{1, 4}, {2, 5}, {3, 6}}), + storepb.NewWarnSeriesResponse(errors.New("test")), // Regression: Proxy had bug that caused warning to block chaining and iden. chunk deduping logic. + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{2, 4}, {2, 5}, {3, 6}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + }, + req: &storepb.SeriesRequest{ + MinTime: 1, + MaxTime: 300, + Matchers: []storepb.LabelMatcher{{Name: "ext", Value: "1", Type: storepb.LabelMatcher_EQ}}, + }, + expectedSeries: []rawSeries{ + { + lset: labels.FromStrings("a", "a"), + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{1, 4}, {2, 5}, {3, 6}}, {{2, 4}, {2, 5}, {3, 6}}}, + }, + }, + expectedWarningsLen: 1, + }, { title: "same external labels are validated during upload and on querier storeset, proxy does not care", storeAPIs: []Client{ @@ -415,7 +482,7 @@ func TestProxyStore_Series(t *testing.T) { PartialResponseDisabled: true, PartialResponseStrategy: storepb.PartialResponseStrategy_ABORT, }, - expectedErr: errors.New("fetch series for {ext=\"1\"} test: error!"), + expectedErr: errors.New("fetch series for {ext=\"1\"} : error!"), }, { title: "storeAPI available for time range; available series for ext=1 external label matcher; allowed by store debug matcher", @@ -429,6 +496,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + Name: "testaddr", }, }, req: &storepb.SeriesRequest{ @@ -457,6 +525,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + Name: "testaddr", }, }, req: &storepb.SeriesRequest{ @@ -506,38 +575,44 @@ func TestProxyStore_Series(t *testing.T) { }, }, } { - for _, strategy := range []RetrievalStrategy{EagerRetrieval, LazyRetrieval} { - if ok := t.Run(fmt.Sprintf("%s/%s", tc.title, strategy), func(t *testing.T) { - q := NewProxyStore(nil, - nil, - func() []Client { return tc.storeAPIs }, - component.Query, - tc.selectorLabels, - 5*time.Second, strategy, - ) - - ctx := context.Background() - if len(tc.storeDebugMatchers) > 0 { - ctx = context.WithValue(ctx, StoreMatcherKey, tc.storeDebugMatchers) - } - - s := newStoreSeriesServer(ctx) - err := q.Series(tc.req, s) - if tc.expectedErr != nil { - testutil.NotOk(t, err) - testutil.Equals(t, tc.expectedErr.Error(), err.Error()) - return - } - testutil.Ok(t, err) - - seriesEquals(t, tc.expectedSeries, s.SeriesSet) - testutil.Equals(t, tc.expectedWarningsLen, len(s.Warnings), "got %v", s.Warnings) - - }); !ok { - return + t.Run(tc.title, func(t *testing.T) { + for _, replicaLabelSupport := range []bool{false, true} { + t.Run(fmt.Sprintf("replica_support=%v", replicaLabelSupport), func(t *testing.T) { + for _, s := range tc.storeAPIs { + cl := s.(*teststore.TestClient) + cl.WithoutReplicaLabelsEnabled = replicaLabelSupport + } + for _, strategy := range []RetrievalStrategy{EagerRetrieval, LazyRetrieval} { + t.Run(string(strategy), func(t *testing.T) { + q := NewProxyStore(nil, + nil, + func() []Client { return tc.storeAPIs }, + component.Query, + tc.selectorLabels, + 5*time.Second, strategy, + ) + + ctx := context.Background() + if len(tc.storeDebugMatchers) > 0 { + ctx = context.WithValue(ctx, StoreMatcherKey, tc.storeDebugMatchers) + } + + s := newStoreSeriesServer(ctx) + err := q.Series(tc.req, s) + if tc.expectedErr != nil { + testutil.NotOk(t, err) + testutil.Equals(t, tc.expectedErr.Error(), err.Error()) + return + } + testutil.Ok(t, err) + + seriesEquals(t, tc.expectedSeries, s.SeriesSet) + testutil.Equals(t, tc.expectedWarningsLen, len(s.Warnings), "got %v", s.Warnings) + }) + } + }) } - } - + }) } } From 8814cac0abdde83d1759b7f80966f82a1735551d Mon Sep 17 00:00:00 2001 From: bwplotka Date: Tue, 20 Dec 2022 10:55:47 +0100 Subject: [PATCH 5/9] Dedup is now working on all dimensions. Signed-off-by: bwplotka --- pkg/query/querier_test.go | 6 ++-- pkg/store/proxy_test.go | 64 +++++++++++++++++++++++++++------------ 2 files changed, 47 insertions(+), 23 deletions(-) diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 6bc7df3efc..07967ae6db 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -446,7 +446,7 @@ func TestQuerier_Select(t *testing.T) { storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), storepb.NewWarnSeriesResponse(errors.New("partial error")), storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 6}, {7, 7}}), - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 66}}), // Overlap samples for some reason. In this case the choice of value is random. + storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{5, 5}, {6, 66}}), storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), }, @@ -477,7 +477,7 @@ func TestQuerier_Select(t *testing.T) { expectedAfterDedup: []series{{ lset: nil, // We don't expect correctness here, it's just random non-replica data. - samples: []sample{{2, 1}, {3, 2}, {4, 4}, {5, 5}, {6, 66}, {7, 7}, {100, 1}, {t: 300, v: 3}}, + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {5, 5}, {6, 6}, {7, 7}}, }}, expectedWarning: "partial error", }, @@ -744,7 +744,7 @@ func TestQuerier_Select(t *testing.T) { { lset: labels.FromStrings("a", "1", "x", "1"), // We don't expect correctness here, it's just random non-replica data. - samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {100, 1}, {300, 3}}, }, { lset: labels.FromStrings("a", "1", "x", "2"), diff --git a/pkg/store/proxy_test.go b/pkg/store/proxy_test.go index aa80cc70c3..638db5451a 100644 --- a/pkg/store/proxy_test.go +++ b/pkg/store/proxy_test.go @@ -350,16 +350,28 @@ func TestProxyStore_Series(t *testing.T) { }, }, { - title: "storeAPI available for time range; available a few duplicated series for ext=1 external label matcher from the same storeAPIs", + title: "storeAPI available for time range; available a few duplicated series for ext=1 external label matcher, mixed storeAPIs", storeAPIs: []Client{ &teststore.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), // Exact same chunk should be removed. - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{1, 4}, {2, 5}, {3, 6}}), - storepb.NewWarnSeriesResponse(errors.New("test")), // Regression: Proxy had bug that caused warning to block chaining and iden. chunk deduping logic. - storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{2, 4}, {2, 5}, {3, 6}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{0, 0}, {2, 1}, {3, 2}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{2, 2}, {3, 3}, {4, 4}}, []sample{{1, 1}, {2, 2}, {3, 3}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "1"), []sample{{100, 1}, {300, 3}, {400, 4}}), + }, + }, + MinTime: 1, + MaxTime: 300, + ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, + }, + &teststore.TestClient{ + StoreClient: &mockedStoreAPI{ + RespSeries: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{2, 1}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {6, 6}, {7, 7}}), + storeSeriesResponse(t, labels.FromStrings("a", "1", "x", "2"), []sample{{10, 10}, {30, 30}, {40, 40}}), }, }, MinTime: 1, @@ -374,11 +386,18 @@ func TestProxyStore_Series(t *testing.T) { }, expectedSeries: []rawSeries{ { - lset: labels.FromStrings("a", "a"), - chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{1, 4}, {2, 5}, {3, 6}}, {{2, 4}, {2, 5}, {3, 6}}}, + lset: labels.FromStrings("a", "1", "w", "1"), + chunks: [][]sample{{{0, 0}, {2, 1}, {3, 2}}, {{2, 1}}, {{5, 5}, {6, 6}, {7, 7}}, {{5, 5}, {7, 7}}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "1"), + chunks: [][]sample{{{1, 1}, {2, 2}, {3, 3}}, {{2, 2}, {3, 3}, {4, 4}}, {{100, 1}, {300, 3}, {400, 4}}}, + }, + { + lset: labels.FromStrings("a", "1", "x", "2"), + chunks: [][]sample{{{10, 10}, {30, 30}, {40, 40}}}, }, }, - expectedWarningsLen: 1, }, { title: "same external labels are validated during upload and on querier storeset, proxy does not care", @@ -607,7 +626,7 @@ func TestProxyStore_Series(t *testing.T) { testutil.Ok(t, err) seriesEquals(t, tc.expectedSeries, s.SeriesSet) - testutil.Equals(t, tc.expectedWarningsLen, len(s.Warnings), "got %v", s.Warnings) + testutil.Equals(t, tc.expectedWarningsLen, len(s.Warnings), "got %v warnings", s.Warnings) }) } }) @@ -1516,26 +1535,31 @@ type rawSeries struct { func seriesEquals(t *testing.T, expected []rawSeries, got []storepb.Series) { testutil.Equals(t, len(expected), len(got), "got unexpected number of series: \n %v", got) - for i, series := range got { - testutil.Equals(t, expected[i].lset, labelpb.ZLabelsToPromLabels(series.Labels)) - testutil.Equals(t, len(expected[i].chunks), len(series.Chunks), "unexpected number of chunks for series %v", series.Labels) + ret := make([]rawSeries, len(got)) + for i, s := range got { + r := rawSeries{ + lset: labelpb.ZLabelsToPromLabels(s.Labels), + } + for _, chk := range s.Chunks { + var samples []sample - for k, chk := range series.Chunks { c, err := chunkenc.FromData(chunkenc.EncXOR, chk.Raw.Data) testutil.Ok(t, err) - j := 0 iter := c.Iterator(nil) for iter.Next() != chunkenc.ValNone { - testutil.Assert(t, j < len(expected[i].chunks[k]), "more samples than expected for %v chunk %d", series.Labels, k) - tv, v := iter.At() - testutil.Equals(t, expected[i].chunks[k][j], sample{tv, v}) - j++ + samples = append(samples, sample{tv, v}) } testutil.Ok(t, iter.Err()) - testutil.Equals(t, len(expected[i].chunks[k]), j) + + r.chunks = append(r.chunks, samples) } + ret[i] = r + } + + for i := range ret { + testutil.Equals(t, expected[i], ret[i]) } } From e02b136d8fc304273999016a61a1bd4576ed3327 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Tue, 20 Dec 2022 11:48:25 +0100 Subject: [PATCH 6/9] Fixed tests. Signed-off-by: bwplotka --- pkg/dedup/iter_test.go | 1 + pkg/query/querier.go | 3 --- pkg/query/querier_test.go | 2 +- pkg/store/local.go | 3 ++- pkg/store/proxy_heap.go | 6 ++++-- pkg/store/proxy_heap_test.go | 19 +++++++++++++++++++ 6 files changed, 27 insertions(+), 7 deletions(-) diff --git a/pkg/dedup/iter_test.go b/pkg/dedup/iter_test.go index 1a28f98872..b21e095f7a 100644 --- a/pkg/dedup/iter_test.go +++ b/pkg/dedup/iter_test.go @@ -15,6 +15,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/efficientgo/core/testutil" ) diff --git a/pkg/query/querier.go b/pkg/query/querier.go index 5dda9f45a0..9e2cf6280e 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -5,7 +5,6 @@ package query import ( "context" - "fmt" "strings" "sync" "time" @@ -393,8 +392,6 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . } } - fmt.Println(resp.seriesSet) - if !q.isDedupEnabled() { return &promSeriesSet{ mint: q.mint, diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 07967ae6db..5bfdc559b5 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -677,7 +677,7 @@ func TestQuerier_Select(t *testing.T) { { lset: labels.FromStrings("a", "1", "x", "1"), // We don't expect correctness here, it's just random non-replica data. - samples: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {100, 1}, {300, 3}}, + samples: []sample{{1, 1}, {2, 2}, {3, 3}, {100, 1}, {300, 3}}, }, { lset: labels.FromStrings("a", "1", "x", "2"), diff --git a/pkg/store/local.go b/pkg/store/local.go index 5cc20ee460..4e88c0a7e3 100644 --- a/pkg/store/local.go +++ b/pkg/store/local.go @@ -182,7 +182,8 @@ func (s *LocalStore) Series(r *storepb.SeriesRequest, srv storepb.Store_SeriesSe chosen = chosen[:0] resp := &storepb.Series{ - Labels: series.Labels, + // Copy labels as in-process clients like proxy tend to work on same memory for labels. + Labels: labelpb.DeepCopy(series.Labels), Chunks: make([]storepb.AggrChunk, 0, len(s.sortedChunks[si])), } diff --git a/pkg/store/proxy_heap.go b/pkg/store/proxy_heap.go index 9099a3b39c..028ac81a7e 100644 --- a/pkg/store/proxy_heap.go +++ b/pkg/store/proxy_heap.go @@ -147,8 +147,10 @@ func chainSeriesAndRemIdenticalChunks(series []*storepb.SeriesResponse) *storepb return finalChunks[i].Compare(finalChunks[j]) > 0 }) - series[0].GetSeries().Chunks = finalChunks - return series[0] + return storepb.NewSeriesResponse(&storepb.Series{ + Labels: series[0].GetSeries().Labels, + Chunks: finalChunks, + }) } func (d *dedupResponseHeap) At() *storepb.SeriesResponse { diff --git a/pkg/store/proxy_heap_test.go b/pkg/store/proxy_heap_test.go index e00d9222a7..448b67cfc9 100644 --- a/pkg/store/proxy_heap_test.go +++ b/pkg/store/proxy_heap_test.go @@ -84,6 +84,25 @@ func TestSortWithoutLabels(t *testing.T) { }, dedupLabels: map[string]struct{}{"b": {}}, }, + // Longer series. + { + input: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings( + "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", + "gprd", "fqdn", "web-08-sv-gprd.c.gitlab-production.internal", "instance", "web-08-sv-gprd.c.gitlab-production.internal:8083", "job", "gitlab-rails", "monitor", "app", "provider", + "gcp", "region", "us-east", "replica", "01", "shard", "default", "stage", "main", "tier", "sv", "type", "web", + )), + }, + exp: []*storepb.SeriesResponse{ + storeSeriesResponse(t, labels.FromStrings( + // No replica label anymore. + "__name__", "gitlab_transaction_cache_read_hit_count_total", "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", + "gprd", "fqdn", "web-08-sv-gprd.c.gitlab-production.internal", "instance", "web-08-sv-gprd.c.gitlab-production.internal:8083", "job", "gitlab-rails", "monitor", "app", "provider", + "gcp", "region", "us-east", "shard", "default", "stage", "main", "tier", "sv", "type", "web", + )), + }, + dedupLabels: map[string]struct{}{"replica": {}}, + }, } { t.Run("", func(t *testing.T) { sortWithoutLabels(tcase.input, tcase.dedupLabels) From 7ef3d941e2a5ee143b1e6405213cb31bcab3ed3e Mon Sep 17 00:00:00 2001 From: bwplotka Date: Tue, 20 Dec 2022 14:06:22 +0100 Subject: [PATCH 7/9] Fixed tests. Signed-off-by: bwplotka --- docs/img/globalsort-nonoptimized.png | Bin 14336 -> 90542 bytes docs/img/globalsort-optimized.png | Bin 18482 -> 93835 bytes .../20221129-avoid-global-sort.md | 58 +++--- .../querier/queryrange/queryrange.pb.go | 1 + pkg/api/query/v1_test.go | 25 ++- .../test-storeset-pre-v0.8.0/storeset.go | 2 +- pkg/query/querier.go | 4 +- pkg/query/querier_test.go | 16 +- pkg/query/query_bench_test.go | 59 +++--- pkg/query/query_test.go | 7 +- pkg/store/proxy.go | 2 +- pkg/store/proxy_heap_test.go | 3 + pkg/store/proxy_test.go | 184 +++++++++--------- pkg/store/storepb/rpc.pb.go | 3 +- .../storepb/testutil/client.go} | 5 +- 15 files changed, 212 insertions(+), 157 deletions(-) rename pkg/{testutil/teststore/cient.go => store/storepb/testutil/client.go} (89%) diff --git a/docs/img/globalsort-nonoptimized.png b/docs/img/globalsort-nonoptimized.png index d91a878e661139878ae060ca623bf5c7d49fc92c..0c81e2e45b3d3d48c7b8a598c6ae0c0bad15dfd4 100644 GIT binary patch literal 90542 zcmeFZcT`hp_ckmzj3bO=BPv}*1cHD{Q;=c-q_+Ulm0p9=0)#epfq+s4B!u3HbVz^< zDj*<4dI=#)4+)911PFoe@XR>QJn!$X_gn8;@A}r4KUn7E+~?e7mup{pZ(kT2Y9Hd} z=iaw(-yxmbw@ml#`x&)w-%lrh`5CzKa;xO+zI{>qbZ%XT1lv=IfdPVUNCsO~MMXt# z^;54L7T=4#h^@w^rG0xWF8J)8psS(a+srF*#|b)uayLK4zPRL?l9ran=hS{`C?Nt} zfi*8*^sF6B=!)ZwRG+sMUo2V7fzHRb^oEZ(+_~gUShu@>6nO2Q9~q9#^;?lgZH-TD zE}i;*ai6+9aP?$ulro(0W@)0fW7v1Xk)r5*{_&9;znuB;NB#aFp$KOZZ+2?$_VTEH z9SUb1l#mvF6z($u{nxFJkF4C(ZbobgtY3K|nY}`JA!E&vKc_D)(j@WU_q`Wae(rAc zc1j0~LoVcijQhUYyf}IDpYH``5V29zVUm?tvK^WeW1->kTkDr6|DN09mkGz$uNHTq z#dD(0{+RNA{w2d}{9jkj4H|14{r5y(8sFs5{I9p(IK;*KkM-|8y zK@1@p&EFOU4tC4BwiHY-dFgjK%+u-W|Jv3mmk&jkhN}$V zsZF{?WI5N4;lqO0F5C-0l^xZkQ^Ga4%@spT3gZW@L!amd%{Lz&&&moN{`n9eRL-rZ z)^akKK*P|_z`rIddAFBctz;&Ld^zexDlQGT!Xk;x&rU&1_F=1hu({9D76Qs9(9L5r zR?x1QTNZPz+CJUGOAk8eqtJA zQSa@Ke`Z>|X5Xd@ZSKleB1!t9acq0}t!7NzukioT{gOy6V=4 zfnjRtFf{xgPtg}P&iLkU2mdK>FL^TQjHUWOijs8(BXdeuVx50y#iOljsLq%4`(x&_ zqufPrilSEHqYk!9c{g$D=B70|=Hg7eCx^0zy@NyRb52t2#u zLqBC?GE^MOeFbCSSf+P$NBw?tBIHwM{lz-{`BEltL6m2IQ@&Kx zc5Yb;`pzsPck}f67DbIsjDTzh6ZDbFqW4I(6oqK1sfWdk5CbJ^FZNS|N|$0m2hl?L z8(lc`CeKO2J544`Kd=?h!@U!tJv1-aqz#}nM+l`e5MY%JQm&Ug2TWESs#7S*2= zXjX{-BxkQP12VRMksX_LMJ#)Ryt{O_UOzW;&6*4H{(R4LTyeO2#+d&+cBGZ_){_)& zK(t1%BdRU5rGoTTEhnylCKrMF@cWGge@N+bh*)nvL7Nr_wWx=R5;oT}o9I1X+!}7LyF{AO2u=3f8KTIW1|1!R| zi7UfXDnc$;VQbM?rhC{mjc#0O19smz+-D1uf z7!^L$LqFnrj&#c-udX$TH4i@*@r`b5%9+ycI4EOTY4Ms<*Q~F2qVE@($fkGNgkY{D zWZMkMsIWrS_G|uZx~UQVeg9fetN%(4@RFhxS-shfXe|ei$FGNrg9id4?9XWZm!F^p zK5}b%Wn?c8a=4Fudy^wMF~&qAMD$V9uL69T7`(vMWJ`{Il#V|WRo4}tCZe>2j69d5 zbY(z=711{c#)dL7&) z(alm^lo~sCD$XE-je7j&+(k(Jrz4Rn}9^m80@B&MVZR$7HgGvH)t!Yl3%|Btkmrdc(%KTeq7!%6z~jzc4$y!3mD?VB{%0y{vnDLR+`5{P zlWfdueAR?w{eztB=tAZ6>ZjK=2cg(0USqIxizl(yN6GWk6{X8MT(L~#iikzsyh8-h zwHHazIu_Ecz^N;m0onQqPspkTG(7Eq@B_S2x}sBx3L4PR5#bfp{kl(L?Y;8idkyfZ zC8q2C^`E3rDXx(RYef+DroZREx(S9shJ!^5SENw#TRQWs(M0_DaQo!cChj6goCA#= z%~;H!kACPh^XaStDcjrs5M)01A{`-HYpS@$P?eR5e-)xkdtki z;^s@XO}~sMXql}SzwHZI90ja85}sABhU3(I63=}OG0t7Va~{p9E4i=%?kx#784#9X zh=xaMbk+(j|G$^2UvA9eFYcL^dKORNg7zr#RsUtQnfT)d&}U;(<@ zZCR#!NBdu9;3ZGSqi4Sn015D3?ss`Go0u97Jhh^(W;v1iIa~A~<^3;QgPOYccqCL$ z>aNNSmk|Cy?CY-yX!SAeX#=AmR}g}kuJK9Xi#dYv|2l>^$-&p#;@j5xL9taceU;3}7&OD0`75o#lW8Aq0$ExRT zz2-8*zUBN&1diPJ*cPw!{I+V`)*Hy|$Z@+GPThe1$#2pW8oB9$DWA*&*_uoISQBbcLjCf4(7By$oEF)l z&gg@pAlTl+^w>{mx?AnQ(&2JZ#t>D%8}m!5_K$s(xe9it(Lt3&@hV5F29<8%C85gQ z9k65Jz$3bZ-NTXk&HE|qc1RC(bSiXd&gEyN!$Lm}sbQuFwdtRA`+lX+HITHSt55t1 zakpxv;w#|yJB=n7cF8@A>RR9j#kts*%7JQ_rI$l#L3)MH*=CH;B{LWNqcH+oyZX6J z%jzBIK{+Rz6n3%yhX@mOV#Em8(ZJQ&+Lf{RK%kmRU(CaXhZ-W~SOlueaV1R>>0U$u zyH4{%@@=Jn=MBtriPSA{e9zmhG%}JAhnF zy_Z@WI~wUC0{>PrvEr%C2vi+b2~`{Kz=~d`erQ}!{&7tAJ9R8R3#uNMHY}Luy)-25 zJgQ0*tz$Z;NacDO#-&_(^~wEezE9S!;`@HNPrqd;vO2cNL4kw?1~{0u#8(OS;Db8A z4V@cr6tkxERZ`iLvm}=>oPt5UqQ4xGWq$vVO`^@WUv1MocKfdoh4fqoaEdzhV({t- z$|Z`Nokw1klx$~((TG2cxv(NUHor`vve7E9UEKP6tUq-Hh>dS9kjKk2MJv9o6r7_r zoF2%`ReF*YT@&Bo+R#rmX4I;$7<`x%2tVldwRYS~o;G3i^0T+An| z`80~@Y85mdRkxl48G8u$jg;1Gy6YQ0a_W1hi%(8)m6KbFUz(1$ax#4?;Biiw;dRrG zRO+agi@qg_STyXh69ll5~M8*a>pRo59E&IdKJ%LQ_8ig zCbrn54C$`P5z=6bTiH_rK8!3Cvv9|i34SGqH} zzX1||a2^#b8u#$wNMY6OKHji#oZwX_i+hl#c75lSn~^!;p3W&nQT;^{@kp`!bwa%B zSiO|o@!qzWVY9zYNt9k3H@`uAp(K?TvC!wAwsuWCUTbTEC+##kY6We{F(mKy_|;E5 z5(v+g0r}w2mZYLn{Uoz`=dp$LG)9K&tI+@$_=UW=LkTd6c?FNP6cXj;Won(wQOlYa zND}V_R7-j$j6=cQyntPGeXVKr#y^#_RdVqb{)#C_-c)CL0#Aso>%6qv9|Ku*0-TK9ii7MTJ4UaTZuT$T={;U|9!m~w6IcgadQs9F)p=AFtd+>x6oPHTdc; zb?=;EHK!LCiKfXbWNDr5nS2FnO&nS0BHJo(>xvDJ@3*2|HB&Fmq{WGCf{gtD+n`ER ze{V2bIPd16tUG4RU@kCP8N zj4Owr_O9OPs(tcd;C#T6EKk%d`P$r~`ua~2;8rV=n+bMVSp`B(B|aZiy%v$QqXfTm z-IM$&U!%*fs#I}bK-CE$QY=E-lO|X7cVkBvF8weH`;u<|Ft_`D0sYXeeK$0J99xv} zKOE6%$p1g&yO;fcy<_H7&c%kp zEaJps)VhP8@*Tc5Nehm^nGGGnTbDu~V&lkff@&6Lb7hZzpIlVK+1c&@(>D+m&)tFh zku72nz|kYZz@4*+|4_+myvAQ=ENkN%Um17umPP?FZ|l3lyeB_wSg-`@UAONdjc0Q5 zwIj$F414&*NQdW#qG&n$>{{Rybs6Ec} z`>gkIL5i}Yww4KXJ~qE|tQ_BNnF0Ovo5T`utGuhf_}Hw>h;5|*{dKNOf#U%Jm%W4UUkYpS9BLO1tMR`7z`cpyX`n=O z#%&Fuy@yK750a$*o@TVQ2Esa(Z^Hb|J+I5hCs%Wgbfi!PpI#rgR2=dh@5e9}DLWL0 zilv8iY3~Scu0e$*g3TlHz@wPs(GRpq(xD~n9rHWH)ZGhRZQ`8A_BWpmnHvv! z^_aQ+`D$<%6^Fw_&6_817isEQwkPY$&CA5vphtGr$JH9*-_4j_Kq-2U-o;pR*qcw63fT7>P?C{`ruw=V4Vw1AC{- zYl(rS*J>H0+gS82Ujr=lbh}Z2*;=LwOs1}k6!m-1tilMZ?(!ZL)Vw{O5wsqtG*+uL z;Vs*=onCI;tX$VZHN&?TuvX{$s*JW6j&)v4sFXuxg57$D0I{Vc-2S~`mYPs~PwS&+ z6YxTVtb0Sajeeg(d7mBgiJ;BRCM{|B;@HWDF~pEa)1~TBikw@ob@R^%YJm;?KGS%l zK{~$pC>{RI#A~49QQ*?#0JM2p7`welp>C$yBg)l7*NV)Gt%=a>fu*1}g^@AQ$cX-@ z@uy}9kF16k9X`66#M1#xvPkg}T$X4z$2 z^{P{`m@eq?nVONT^!ZKj+Cl-VbX!LO_S=@tSmYNC?~L5Bkz#hyMI8a9RxO0^w5}XT zF2bH8aQ9}s)uVO;pY`cmYlUUfE(riVu zV@)IK@8EVw<4MH>w#EL;@WbaPZUR(Y)(9!I-hS?&lr3bRykS<#Jl zjZp8K)oe~{ZeVZXq1~!3kZ^oQONA)jcgALcShx zQ$OpfRGmIkyL+z3;fp<-D{1$Rb)gH!cWSpHR#aH2{7gQ*GtrYH(5Z04L{|2&us8mY;Hi3jQ%O_=o8o|7 zxw|-6d~mikr%lh6{5Gtu>kr{$p!z6XBcpl&Xd@9B{uPvQ(W6R0CrLj7AjEJ3&qdWi z%KXLhUW_-Rm(fw&hW9M*q#vUXDNV*TN3e)1j8<)t4ts#(nY1fZ!hBr)v?^=0ini{M zHsi=A?@J6pzdHbh^#9R%(|f+xu65`EKx^}V&8IL@ZNa>|^Gr)j@$LsH)Or0kyad;Q zIcew^W?f*yYZvCippMa0^4wl<;cotA4UN-5g;C=HSGu8;W1Vv9=##kSf>E=|+@sp5 zwV$M;7xg>e-Y#e~G2B@)|EYXjMEaqpn{O_Y3*V||!d(Q%Xghd?s_nwbC$bVpUGw{ya zKM-$Lv~=^WH9$28@np`$tQ~TeZOBEZ_SnZO8Uv)_fYE9<-@AujMB8Y*D0l;>WHO1X zh??fEbBI{(E_9SBIj7GsbT~nMob)pLy%Pb*ViZ(K{YDfOqHio82VxfG;ny_}D%Wc% ziuY$NzlA|I!mOA4Nyr?oq{yyj)ca@gqizbGm>k3T2xjwP%u$OF9Uvv%rA8#3HaJ?u zd~T`X5q%w>Qf@=MOi#1jp?7U>t`cDj^Xj`H?_j_DwU4^h0759fw!f>}j+pZV-+)-A zn_NE^%1Bh_8;d#oCOXA%TnqnMhwZ&l$rXa(-B$ceQ#2m9qgcfVcXDmVu+9&p3Q zC+l@ZD+o*tQ)(mGsIR%l878lJYok2nowOuGQ4nXYLNeU-k0>5sBgf^F52R~J-w|H8 zdaF1tYHum85M(AoCe3?T~|5dE;)aLz{rEm&=zUxfAai zBCqigdZkbapex1w3S?#Z1YNU5_SX{i@nW+=M+!jtSE7?Rvh09p*cAhEor?^>x6g zq=@~`+LHBa1r+`9Vf~TpZYfBGt7wOzesJVH(?L#8Mx0Rgwf!YyWmZ8}lnIY9h&wym zAmi*w<9ZmyHOt@)fFLc8U0eAmT5-)CK;1V&P;MGzD^1{_Jl9@VLeIS{=;yJ<1f)sH z1}#kr*;v$Woz&rxFsj9Sj&`G)`E0!n(re#V8L0}`QfoX1dR3()$NZD7-2)k$R`z#P ztZgk!8!crrrP|8}suLkJZ$gVIG{m0UOrsMZ!>Lu>=-$=U+ZGz2IX>g!6UG1p3nFE+ zXjs{=DnI=tH~C|25-2sD?)##z<;(4=)6(iz@1(GEA_6kIB$m}lI-5Q*&)MO`1yW=$ zUwK~tM;r7~Dz(RNcPJ@hDl%?xkBBO}peKRa5ig9|>Yz@0eHb{9D>oI#l0^)}D6)~J z{oE2RU9ndmxR_luS1SdvY-SGt7Jnr%jnOH80`!)Miu&cwP0 zxl+Yyr{vn(eaD@X2l^){M_c+mTWLh76$vCo{ZedJdDIlJm6JkO`wb3V@n;1Q?k~I~ z6mIA}#Ef;Ag&0@ZHeTy~Btr~skeq?sFvz+zfLIGzNt-{e8&8BY8aPX?=bo<%81u8i zo!Kfoebs-i%nts($D%sOj_EC~I!(q1?D7Ii$J+D$KthhDd&W1q#n2AZUbkEn+*hd; zZ&2N1Q-^VaLy!}I%VBKmp)tkZTzk`(wAc%2dfM?qF+L|} z=`gb?^cUA1)n5Vx1jC|M6{DR}hr@-WLm?Xrn+CpzV@{~<8Y>a&M0%=Zcyil5f5#GQ zw&xMXp&`akg;bp?8u}Fhz6DD8&2?;x>W)kA_3e$jVAHJL-RwY7uydfhq%mND{YxW= z0iTbH`g>V7=#ro`3XJG{>ou;4FK2J-s|1WwpZv~8YX}ui(yi^I(Y~z< z4#InJ%uq!pyD1_Zo8x0cX$dQP88z7v(6Nq6C&q31WW_17 z%6!S44-UsJSl4{qSw!!2?X0Tr%6Wuby!+*}Qu-Xs(snvJ<|!hh@vNx(x0m;zx>rD; z);^`g^G1RA3#KM!U*C5PSNe}Ma4vq%e$OyTE}^kQwfZxe6LjP=-V2ypntHW(uhDfS z(p+5r$_FFSaGC#G#~H@c-)ID&exUUqk(>>`hZL1Upm1IY!-<1}RIn=O-tC(Y+L zSX15O2l2z;qA1V7yhhb;-<+W7xD06X`w~bwjh3N3`?5k3y}>N?8gWDQ`E30!-ED&KRpVc$lPrLfcMd+mS_-=7(aI<87-U(wa&3v;5dPRUo z%71}7XWW${P0@NA%dZfl0 zuW|O%A9PO?Cm1MH`#bFf5IOw+CUX3b<&T>jAo~b2Ux#muc7R42 zukon|I4Pyy--ba1Fk468Cg=rgBF$ekYegz6 z4@h?Xy~Z#9$PoQs%w{GxaC(v_FGvlDI_?=YAS<@hr#Nmw=ZLu75tmgHqGvJw@!){Z zo$904^*UBGXfvbSduqy{ z)jWF(wIl%(lpSdv=@l+rMkw%Ie9sVKt2KnYTHvpt=aRAxR)AA%)Tz#y+Oqc}nHg0R z6`|O`yD|6eVx4$Xqt7Ml+Q6uFx6g?-DF0s9R~Kpo7K3PMJ!Lbhy34~mdRxs6aUA=J zTe;ftKU+RoOsDt;bu6!taM*WHY-z0dPG0)bWVLuhh(raBJe)U@nK~KH8r$9ZTJmM; zbuDGM!|qWk1-hNs{B6Yx2t}K|*nB@hP2(JZymaMG(JR)X8))iCqJ+yPyH8Y*mM%NB zDN>YOvx!WP!`MbgZJVmf4fQ#PSWeGX^YPUe8}4$){CJ;_Z@*TK+bxQlOrj5Yb#Gn#+S61MussY#5015{`>QozjAgVq0?!OCT)I>d1wJI7AesEi=z3`Uq|M6&h{llb=#_?1 zMW4y3IMOO*vejI&v#ZN*L;{u^avHe348l|NBC+r%OP5IDquBYKXSPSb8!8FpF>?(r zy?p0Q+@WKslEo7G9H7pRHFNs``o#!CsHNA`UN*;A4RFX^AX+MxjkS;_y%j$g2aG9v8%< z^ig*gYsE$7y2pRZ@H%WkArF%h!(+F?3&zvlOyeetue5Jc;dHe`D%0Ms z0zqVXz`Rrg;;;vh1og!!nnQPMHIal7$1vWD8+OzkOhi&3km9C#!0ALM0sJ18yc`iZ zTf4`1lwF@LNI@c5zRv553XBym-*M20t(zr#Sqoktrb(z(*)_54P=1QGCvtF-Wm=S`o4q7^-gr$Vz-aQLi8?78woX2O?w;N<&@rwX( z4(jmgs|qZ8O=xl{Rq5i7h)^!I!DYK_wXpUdI9z!^M4F;@n222w33FyKmSW1) z!#1ApQHSLNu$xTZ$!#7>iN9<}LqIZyb8+jpqbfne)be|L2Pvi|K-rpR40r6+QrKw4 zqYj_BE}sf{4^`(l8yxf7K}%>8!LLC|&%&4Fi&o&9;ZE@BT zV!<9}WHSTMWLB0<$#8Xeo^+jh3!%4pvV~sMV}l(4-A1sCU=$M^(WOw z;^0}_NT-fmu2Ni7e9yY)M3X>$MJpaJ^7A>sBsq!b#?L{4`YHTVSdj*!AK4xPI45oDmu1S$>c zr4p{~vCTwMpVNSMM_i_&Usnpx>8%F$<|F)Ok(lWf!v zw{m?~w`XAT`!x7Ncqg=GUxXK#V!eE7~r?kO^Iv zgm=nP&3h%wu4Tcpj2Ad{S;p6Anp2W(n2kP=cRCSVgRAQYMmvN$+U;X{aritJnlDJ7 zB&5byZ=ah6xGe+fI?ovEn^Eh$dV3`uKW;ud-Z1a7=^-+LBdIA~HsFGM^EWFl=(9a} z11x5G`A>Vm&5wYtm>sS0VFfW^19J|IoQoe`OyH-mbjH~3_KvQA>iM13RkQ1+=oVZ}??ACj0*Dc>GnAT?LNXhpfq0z3H-}N|`v_l(h^qNrQZ!7RKV2(+l z#se34>k(af=Xm)*m8 zw6!(&YtD~~eX}j_Ei9J+SSXtj+*9*@uDH2}X<(^+owfL{tBYH2g4j1dQ4i&ABC( zOdEUhQujrYJ{ZW0aVP|A{GxPh?tPcJ_r%7F2V(I>Z~8yJK5ZadcvQU;sBw7kuyHrL zV~YEf>VS?*G^3C_vrQ=Z>x>CPKJ* z_iG38xy@b+&fcfZJcp+!jgX^0Q?j@By0uaLYk|7<@@<*Xu^!!ZA5uKVRY6Mpe0dLN z6Ae9rW}IsV`vM+-P1ml8MmF_>)3&)K*g@swDbKa%aqlD+PJ!D3dcmIC?VlVR@;JeF zf{O3-D-zpf{LqO&owW(*+F_ihUHceO1Xlfo0_!MP^OEz1c5w|R<=aeLQlzQz+Q|Bk zQ15qJdT;hNTX5vFw~C`e-zJ?f=FiW$Q5;BM0BCM6;`f_k(Q)z812u48oXsTSxd|C1 zNskkJ5vOMAkqyv_KfhGB_ehTAE{ZRK1ez^MhQ=ioka+re?<>!JIrH9eKF?!1AZ`aB z%XC+yilR(gb8hH~js5=UfL-5ovOiEwj$u+k*xwAa3vZO(ES#Wc2aCPZy1q$Iv@G#h zOD%ft$uO6)Z18AAl4eA73cNlCJdUs@*LZJ6kFKH>y(do9_fW6;JM9%!?}k|B;8&{o z6`at4@g?CF?}f|=>g!YZZoX{*6(F{edONE_9&bU zDf39bhW@zXehX5<1*tk}JbeOszq_7>IB`L}j#C$M1fXA6zc^~+1^YjyHoZ}W4;x4p zy8Y(isn++4gzBn}i}A(D>ZFwfDxdLn-()x*$VCwwTtW*>9`F?HkxKJtH?_Y}3H9jP zfGy-R<`>GVGSrOM&c)P}BP)5EjH0#FxjfQuA|y=H>Y?EQ(fW3e3S`zYjb^~mNV z@1F+u=-?cwAgJAHF8d4aRb;RXnz2GG-o zvYOkILDl|9j~SESSNz`p9z-fswY55Hmg-Zf`fRFDmLl)@JJF%`84Ek~a;$>-+<8Wg zqO}$)fe_R^Z^e=CCaAgtl(WsOqMbw^n_hj|E57k-QT%v3fHX^BLi7U0YUv2v^9JB< z%y)`^ZoZ*E8-Gp;HPeV;A4fJ1HtYtR%*1K=t!Gu8tuve*_7MrHBkTvN?Hy*9MW#23 z7vxY6#Vm5EY?5uZ6T|IPv$lFWr|#^)&=X(>OOz9+b1dj||>w zZ19mktIM*u9!%YM0P9VEhd0VkrZsZcL%54(r7Xt#U6w_UG#@j-&!+>aN0yD*)}}{h z@Ao+H9b}3}0{xg!UGzYgvG%DB~|MjKUTez~+5CN~D%L?s&7SB)h<0`msi32Dvj zT-}#q!O~? z{EA1SZ#B8ef6X1EPK2q(>*OPm2iD!k#FvOGAYv%dM#w0uZP|oPb6RO_^i>x|!f~Tc{CIDxcAcAwC`C9=^FF&fkPHDmqE3jO^s_-iR zcF%>Gxm+N%y?WdTyMWD}b}!Kos&_;CPBD1?bVhsLh`T6j$nZk4DPl4MY*1MEaOx{Q z6sVg7hlA^cTuSR`F$CJj7X#3l&zZ^CafLBLa&dppA~Vgiuy@X|Isfcz`P@yrgE^*@ z{HG+$4M}x?f*1@o;aJtEMM6p7HxId$pK9p={WzV_lFzNskM`(!h=Fz0*b?hYr5RMf zt~y#SS&3gd57+DigaY$*KN+c7X!>cDfu2t0G7EIYlDhbk`9fv-%3&!F+MXbv%@mrLnwMC>+G6W73&<*g z;4Ql&syLn8@QQ{xW$a2iWe*xx@yWTd7zq+Xd)P4%icvR6E%YCf5JPbH*MxyvAaj5% zk_RDswMA=cL?SuF_?z<+H0N_}7>vv`YtLJ};nZE5xBxPo{6#!jv3$J7A0KF1oVQwH zSO0saTDUR;`6{=)!Bl%5W#WlR1*ct`ltanq=(Hu$8nvF2x2}=N`AY>_n z`K3~E=aM;f=Sxf+`8xrFh%us@Z=AMx{GvqwOwfo3$)K= z-_cbFm5tU0LSz}<-X<|AA4(jtwT%lPMC`X4xb)=zl8*-7q@q1Ks_ZP{;ofh&hq)kq zKs{1F!r3%Csq40cPp;3paPvuZ=-)J;jHXx#5dP`#_ax~a&G)I1o;AB7fr>WNOwf`m zhV=VquEYSOQIb*W&wq1@sX50if6$GnfZQK!p~nBQlJozKD)s-9Ll!wzU}{8$h_WXE zMz8k4wutKFuyR>>tnY-*K3yg5BA(kW(b37rNm+HD%F4cYZY=C9s1?WNJpRt>?(H2W zN`W6XY=$6oDCV#77>M}}*rm4@9G?!*A~ z51Z}fD%4V$@N&(0wc&~p4V8eI*ETKwbTzMm@F<(f3oO;{5$}L*qf9j?;(^$=bD5zw ziHzn8K8tTgFUgz;x5!e0&iCr|1(GLlD)%KCJSz2HtKrZ0*dDPJbffP)s@wf?kI8(p zA-IYq`JY6;Z6E8d2^(!Fo;$=Gt@Dp(O$0b#)`S)rM^j<~=}NvNm(0+^4pZM=`R`GvSQJoE!L5 zOP%F)R9{8Y$B!T1ruV3f?cCI!2$DN_p?30Sdjk$0rqW+Wo0#4i-jYoZxF1>SoTlvd zilw4@tnz0c3;`K_fJliEL*qS{qict&yl@RJZOOVE3ea&unA%C*DpvT~hK|>0-Lp(p zSZ{a%aqCdMIYkM(v}Og*dNHZ)xV`qK8ANKsFo7OiD*R30@2;*Ye$>Y*ZplWx#_AWN zlXAYc{H5PtyD85C@gLGw%fr#X4>#hEP*mi5dNbAODR`Tzerp)rR|v;P^_!YuLO1K| z;KZUF`~BKuANW>*7CuBo|1Y5AgWzAbHJ@p@xLfUhovvo~b)=Npy4@ToXb{;*MUQ={ zyP73j>8ow{wu1fC>KXdL*kiuc2|q+wpZn|6h@h4DhkRG>J(W^eyV(Tmt#W;T!qWMt za#9m9)Aw`VwD^RAy9maWy`dJiEnk1wjzxXa?elp-hh|^&p&!44IgwdS%b>ZO6BTVB zaL%q$vsA{|eX5@?XS#J)gcHM4_CJfq;4N~Wj5D@}i3gU|P7>93-HKR! z&AY$AdID{N0<(I?4Q!^>I%);#t+CW_k1XZM-`^w{Wua-!w3y!OnlTIEk2>_@ghXX` zSgLM)+%UANo@IGGhiE(Y)r}pzE@9Z0NMtwP9rJ{RZ^=+v1g{2NtFtTN#xQeC?qXc+ zuv!E%Nv{eY@oC1Y5`SHO5jrx-t^^vSpf*YLTHzw#XW zX}o{_LvNB0yV5S;0$p(JeM^M=$l>*^>$?qTLyY|t5rgPmuCr{t zqXHKZ$EnYzcr`~K>5FLU*XdT^H;_*t52jVq=uJP->s!2brnFdtmlUByKA3MnxN-sQ zQM4fX`5@B+&QO&sdkCCjb+ou|>?<``_=!$6)##UthA7@@NmgW*-eD*);%_dx>F>nZ~4 zcc75R5r0&SqYwgz&VWU^Z8Tiw^p6&ZRIcaNFB}L{J#ae!$v>MT(gl(pk$30n^ABD7 zB#hp-{N^gWH~C^G(jiY|Ss_|0KQ>EiW}zcORaTcP$@1k3-7DxCzY%3a1mh9~ZvmLkxV_mZ2;dBw zjXlUFR$mS76|sZyr2csn3klw3|8~N-FUQ3l0NL~eOASQkWdiXeqRXuD4lfrGo@vG& zCOR_hcV(W&W+tpv%L2`wRS?m4@@*(B^M_iKn#}Y z5$>X6QD^hr8~$<{8XG`KOHF3PeakZtR;f(oQ8BqW=}u|XVtpLqCuM(zcQ%hie8h)g z`JgG0>r;`FMkQ;xEd}#Dan8!&b7dL!`Y!3(>NB!-nJjCF?a~?j=%9(^2o;$$WQ8}%L8-cf0!J1La$=g`O?y$5W+_)ywvY& z*g3}`fjWfL$#hqeTc%}MRM*GnEL(^p(C)sa#!>-xRGV2Zb;@6BpxAktRNZv$Kdz(Z z>$G3t7n>%(a_oV0Xs;AlQ!`$xY16?AtOk-vzftuTPBQvzqheEb=mU(Yr8*CdaGlyvAbt{-wuUk$0Hh#KsPgyuAm? zjk$0jz`$WR@BIigt?N8ur_bN{79Kj@5crH`(`IPTm883m{)`3i(djb!_~D}<`_Kx6 z7V2EaL!;ccYtx&p@3kM5mf7rn1sYPd#bTALXTJ4KGhC|Upm|Bi>1lRDg~;@}6Rtu5 zK!i`;zkB1cd20kG@#K}w^;SGsl!YN~&Rr9g0IcbPf~t6TE_%e8)~!5 z*!AyQgH6^z`D>?vLOG$_744=ag+p%&z6{jd&ta9aE&z5I(}nmzLbTS}uOByCPH0`%8SsLGynbwF2mEcrTcFt1VsYX7TN^p{7LmiIfErTFX`ksGfRDN_Eake#Tky6Y91k zKTY6I<9I}k{dUJHHwc>a@+62m$Mm{c2>HMR)kGc5r~<`-A|8;@9R4 zbj@6hy!BjSX!%qM{K@kI)PP)eQ}J-qLoHE^Jg5FhVa|K3HqdzlKG4EG6}DaTQCNzco*Cliw|JZpS1Pb8_ob zO>;|z{eyx&-SlHd13QxTI%cN;^)@Nan_d_Xpn*FR!WrN=z}eHau6lq~iE;?Xm6OIh zaBAVh+oC-I(&K3R7&UdEABjm>WoV6w{l`E8zKFcBrIZ&r5-Oa&^IP1xwLKvDPkXsf zgA61pIZX0+4>1q@1s|jOmk!9H?1kRk1Jcx?E##_;%C@rVyAu*benX=c;v(A$>QD{4 zH6Ph@>Ya?~u+OObp^Vz9)X6Y>m)dGo^Y$FeqT1fc=B>NV{Q(|VI*dhqCGPOnMaqK9 z^2vGXcT~1-pG<{FQFxsW&Yry*^P3LZQz=VVP40WFImx2hjnW!*9hq+FKn}UJlPvIwffrNXz%fb)-@hL^~ATV)TwU1qgdcNErX_Jbm}L8 zh;+%J+G2@HK1q6;IvI>thV3e63rdx|{jOB|_HEotisoGGn>>nb3jDBFii3DoRmQB+ z2l^Swihg#`gZcerob@g0`#!$9V55{n*2zEx@JlE@Qh;drHj2OQ{6Ok z2)D$-p111it5rQVZzUL$^A6H(vQ0a2h>R<3ARC)HXuTy#hK^Jfv**b7=g(AZ4WdrC zJbH=<=yvVN6d@QrcpUf_eA>6~kHfh@iy8~3q;ll79jzelXz3L&Oa){-QWLXLcd=AS zVIaang=knPe!@6%QLqw=Y4asTY8J5bTtjv7Z*(>9Xhs)|c=Z+BoVbio-1BL+XJ0L> z2aB2DeYe`0=SM52gQy!X8eD+xqj$;yfWEFkxYO5dZdZ|@~QT!ZQRtr#Dbjw=sG!-gETe7KNOtF4+#paH+ zhS<&M(;t83Og{VQa$eC#iR#OT&vKtAGUs~o>kEmq(Kk$Dqkla(AeN&-Yj0RaW2g_cBV0Ro8uLV%FuI|00}@B5A;cm3D;ul0TF&T@q{l=Gb3 z_x|mD&NEZd(a&U2QAc{=<*2MarSgc-89GAe=2Vp3I8i&gu;!f*Lor;wc+93`-Ry8uJ(-TYko-<5+tVpcdWPh&K{`;u-SKLiw0}Ln$!hsRIK;tckT?zHPRF<{Io}9Qxz@>zu$m~>=6c#ocjv8DT zji>a$C-1PSW#tr0U2!&j>2{~QPH2TGdaiJ8YmrlZC*4VJvk)$+RVpQ$hVPfjSkd`9(jvbEzRCG6thP!w4mfHnyD3fspU3k_r3zz!8M`}XS1kM)eEnemAQ^cnMZ zkHQYHX-o#4Q9Z-Qzl{Kik}5W4Pp>?6TSiGG6D?QGaG@ z5ufDpC;Pie(9crvdW%hg<0R~Sx@g^}b}HL}Q{h`*n9JP04wGDT9tTe`GxXilaMy=3 z`py+@yQPRT61#*8SRQ>>PS-nDuX=r35ohkLHAbiQD(n)yRg|c`GCyF?kuJ5mKigrV zg=f*Su~?0ji4lx7XY5VBoPKUKmdwiI?Zz{Pn2P&5JH}J+*qN8GeqZrQ-5gK&Z5VyW1Pu^-(j-J_!Q^=<)&%14Y2f6!RWehLUcV2X-{fk3+k8Wb6Yv$aqdDU;*xAND9K4ZJ@)q9-w<4Q-vohcSjqn71I)A6S!NoW~lA&+;&$-e^}!n(-w- zQ5Ckn0=wN(@Vu{4agA^{kw{O#Yb-2Ybo%vn%dwo{qv2+nr`-dm?;Z?PtowWd3(IAm zUq{vij#%r?7zZ`(+YhcpQK+p`$a?BPTi0mV$xWzu@S`v1eM?h#Jj7?BPk*`9!A739 zTiyIgX)I|i@&|c=`M)P;FYNAE%gLq(1Wn0j?>|4hZo_>j>73sC`&?Z$*J_4mo))@g zQm0#6g9)oH@FK1t!d5RjDVbKcid?)v7=aB4O3RL{Z79Ma9c)0_{Cv~g8iIvU{{OlG zJF87Xr*9xX2TwgBku92bzMeauZSd;%pa1*6D?W0Ud%nyJigwtTJXwb#FZMTu?)&nC zHsxM{FQ>R|s*qV{*Uz=ScpgVT^##>T@hUxNzRvq+*Icyjug_OTxQdeReo02;25n4^ z(s?|l=FBS&F;&&bc?F7AE)NsY^^0+-T*Y2Zu^#ae+HkN2~Myx);MwZQH> zm>zd4L9+ovZioz;+>wy0jc|{9h;?C)cT*m>HJmRDerLbf1eL59v%H`byITUo44M$( z%v&;8e3FDoU(pg#T1JyU#!(y6D}t_H(0LZdh}0fn6+$Q5DVJ_X-%5MWDfG6#7>D)#{4p@eU49P5GvA6NxU7)1LA{4;!I4BF^Ihxlphh@^+F6Y)}0v?TPcP?F9+Mksp`ASo6HF?gt7+^*ut@Y-08vTmmqrw zbAQQ>uJ(trUi*u7&}^fBo<LAEIF zL6(#x6!t`gICwti!Q5H4bwAeYq9Is##Ti=G>@QY`FF9>mq9n_kgyWMlE&w`u-W!O) zc9Rt(s#0QgH}RnI@(;X%Uli-sSw3qFtU|TI1VoAGm`nHdnEpH>nR%E%t7h`Lx9s8 z*e32-M&Nifr7F}s37gPM2yjxR%4(F^O>x=LYTg{E0?mUvNCf`b0=_wE;~8`Yfu#q> zr&{QW!`^Pj>#2y>bj_mnv`Oa}tBu5H$lC!9{ttqVou;zi>tTEd?CL;Tba!*b&<+jw z$rU@CJfsvA%8?Z(IfXY4L7TQy(Xiqlf&z8eEqQopxA0sHIV+6(6n3E$Y)l7HB5dU( zYapID(_tdX3}LjmR8LpJ_Pdnz@2+L{j;J|$Y!Vu?s2QCtpQ%I;6cW&|YjUWF=vird z{}K7Ql%Bz|D_q`T#%e}1h-CJ>7TipvqY}8|{6YV6)|B&=Q)EMs6HK?RicAI{(58aX zVOKB$owPZ!fj!28rUn*N`2G=DYx`pmI0WxcG$yAW>Fv~kaT>h4#!Uj?^M%wI0*2&> z4-s+foasZlc(b;t!1I_doG4waml_Kb3j@>KF~0N*Hds&#-aeHTikd0E?$E`zrm(u1$VmR z=fG+3y1~o*nv1t68d5mI0WQwp{)TTj_0f$XZ z*3xMZ$&ex9{M^bxD%+G%ThZ{oetD{7QlfzTY(!ftm1KXS5tld?DF$_3zMEfMo}3>) zqT?UR= zT)mKVd7!~{y}XvFjGZ2}yxmqfqGt8&n80v$Lk#u@jvQhM|-t5|hraJ^xDkKTgcr;4%c$v9>M zeDD^YsZ$4mG~y7M7$a04RbzQk9^W9T@q|eO{7GP~h~W3?i4BT}3cP?B zBPQ?#8%{Nx5!cL-B(f198eJ90W>MGPNX9_9#`vNM13t(v$@b-Fn5a&62k>KeG+vng zg*{v=GCy~nNrV+<$6&E!NhUo;OTelHg;V z=-@`K4qU7|nAXoT6|0zq^U@;yxj(MPi?!6)WLq=riJRAGDrcbV~-R@`&i zcEWo5;%27zFroK}pc&NwCk~P?ueRUMggnxaa@YKeRrZo>{xYqF#A{+#txW5)-3 z4=1v8!lp^!!lUd$AL0_8{ZooNWxCX3b8@qOYoRICt#+isu4A&T>pJxZ{-E^j>du9O za)G6&3;ny7dQ=WSgtY425M7Fvq6oJQIFs${5=g=Myp{@S2=qcz?I50(i6y5t`tRkJ z7v0|jc_gC8=9WewWU9Y#7DQKSWO;c0h2@l|;Qjr^Q*`xs=;ASbk`L1!Pjl zi43QFlGN$c;;blQT#QwV!$=sNKKP;ET9GZD3IvN`h|y~_c8C;FOC8aYA$U55jH&o5 z(_U0b&P&Edi>oUq~|JqD_(sZKd(U+uA>(%Uv#%#tdGTZQ(ZDefE&=UU?o7eZsvq#qLB| zes1Orazs6z5S4jev0>5K=}e>XEn(hnKZt@Ngb!z1!oYUlmn7LHn<({KTz2z2dSY4- zLnF_Trbaum`cp0YP4^188jZ?QEgP6wVNCPiw9jc>o@UfPhlYXDasYjI5sJL#*Zuxs zF7A|AU}ZBO`?;d~a08nnNueYhj?PnW`dN#xSH!_@sV?u{FjF;D_H}%{E$pcY_Or4pz5_t@0e?WEu`#S~OyZ*`SK2TgxVQ48*pxoV@o%96Ub!2UN-e$76P z9n~`zIm9#7S?)LpCCANNJ{AJ~gOtT_t){-zqXbCC#G$Gl)W&$hFDu5s7Tw9wiOzE# z;TX~+z-;XBI1YFcC&|JanSz!}KNYEFIuRQ5a_2dq3}#@R|1HBvvnde8n_*+PwW)-zOoBPtK%|Dh+|my6oMD=hJ@j<GjDh2>vr-R(+qQ;=kILfJY&jyytJ}Lk_Al@{iNe>Mh7MiyFYJW zPn$P`J9G`e@rpR?h)O=^#PC^S=)WX-O~LBvx#L2(BN?OfgOHVidolc*U+=KoD`ILU z(-_EiP_ioO`GI)NJv#?kHAGdbnNx!Hi4SC7Q)3z9zv%5W(eC?2Pp*+n{lg?%odKg2 z%O7OuDO(Y-RJKZ6<-E;E(O~$GmSSq@3AWl$vIvv8736&+sc6*QMIF~t&LmdsQFFV; z=40ozZzprJ@(2}B+8w!|+t?>${8B}!vO-I{C3I#RR!!VK%A`cWHIiu#lGMJU!u~-o zlb4VgpRBo_>9_4JTCax>zUn!uori)l znr4PA@J^hj>SdEZOwMEwt|EzyIVVj5i#~pfnR(TWVt<|H$|}ZqgChZ~c1$>&hN%E% zYfqNZR+Dfw3-{0n`tgi=O2YF!&)<(^Ms@wzG#PWCa8T5*yYSA|BGqH&Hq*6lhN(`^ zBG5qfY^}G>%qNIHEc-dxBoW!U^C_XjubHIHaRm*r$H7`n#xtrszXj!-k*>4p6}BS| zfTBZII%f*KONBW5ju$5%So$O)no-vH+RT)$EiCHkwclR^#}XX~GB&I#^R0iLo6)4K zji?v#+a#oSpfSu*3?)2#G$c{#?$bU>^Q3EqYME0VVwfczO#s9b(ocx(r}I(!@;G4% zoT(msYJ2@Noj;*|mX>0Z5_&3DwBvE-?jGBrlL`nCzZm{rO2Q&GI6G79lhxLu#d|H> z)J^0%O~rW2WR3XNw*B2UOM>UQb=N-#M|%a{UE72rxX5ptxt3s2V(-sd0nu2YQYL5W z4X>p`^$@amilqp#UqE@1D4z2AMJVBHKlWz=lNzw46uVdgWtw=qWvk7+r927ndhpC< zuYktW;X_4>mcdD8NPmT(n%BlvEm3p>7L?ZV`B3a|*O;#$ku@kWZK9Y&RWyE-w5|St zvtF>E&xH*2V#l!JJz;goS4#RwRo+hzRq^H-SEMmxy@O~5JebNR=WC1XMHC9aO$2!^ zOxM*di4v#1>t_a8#w~I|&P^SoCU_GI`D_ZBM1YOD%1Y->dAh2NJn4XfiDqSf_UBLD za|aXpCzacFdwQLRoN4k@Bp5vY$?;#-hmy(ZbuO)pu)R9uPpnd@!tw`#tn)hLWxf-B z64umYYu7Slgg7TWIKP+1$vYJGI4Miel)oP7_0Xm$VvDZ~M%9Zjnr7se6{S*^XIaJ zMQjVu9MZn%XN%Y0jCR-BVHd}EfmnquiWRHWX4%y#axX+SKM|K~a%r`?{bHM@b(LW{2afhes1+xH{e zh%5}eZ}QOd+ox&;HFi`SYB*4+VLM%0ZWCK76|er0AU8dFgDqbYW}r#IJS_)e#E1)dr9-Ns`-DmDQ0lp4dJ z1S>`_)i2yhCvC}-fthIWI)(q@Y_8kL)RT`KnTKr^GDV8Nk$e!<^-d-y#oKRNk&^5? zZb~x3<^JtqiwR#P#xCD2$wzcJ`E0UV7Idzwn42HNNGGOMW9Js&+~hqI9NFcHW>S?p zEIUrAkl6G*19^TpO_tU&tvWXJn}*T>oPRBY?Zw(&*{4K*t)tbSYUS%u+cFtFoW}3V z#MaM+`Ro1unvqC$d>vo5Chs(e zG}Q=w9slRz>);G^MRrr@a>HFv2mY(=~R1dYvV?j=*&ppfhWPG zzM)TorxQ(v6&LJZ&kTpE3;33mG3tPD$IR3l=~ddki%S0F5%MnImy3K=tQ?(veiQ0@ zyJknSH3u^NnncEvO$3l~y;}2mk~piFr`&?IWr0+fz*IO&1e%IipRXVE={_cHA(uGjKf_5nzq6#g!%@befjhjw!-s16PGokt_11PeV6}NSba=%xeUBD^xuA&3%@cDNUuR+N-osXs7$sI0F=SmT$G zvrqeStxdvGxo?L6*ar=6mLW26I(ahemfYqvIw;nF@dS@|o z?{YJo&2AWq>$+$xx0=vaf`rz5YtgQB9p7N51M-k==7pT<0V)$`UN zDfj;5BG&Ah&xqD1e>dnQ%Bx+M_VZdM9fvO-r1noHJ-Z+eGED2H50y%8v`e|c{CEAN zQyQ7)>+THygbT0qu1jqsiX^!11!~J-4r{6De_RA#uI!^IZmC#&W-fL?gqllxI85JD z{$cS^Fun=JJHnEs$PyAp&K5gqvQNRI`xck#v@VXFPvqj1aRB37!v-(3*ig2Y0b%KsbL{OMqfG{}rV|rmt2>JZ`YM zH^0?TBw{6L0^rIiR<*>7X8<==>rKnNPxjHua^da%Y~x8*g1lc%;!B4BvXA1K%8}AW zb#1?&;$m$rX;QLd4&8g1H)tCGbUjL&gWjxRw6rko`TLpCtH|-8a(_5bC;x?HG&e{-RH(h zfKdBJe$YrhF5grtXzD2WySG-A9+OtZoQm?1Rd<;DFX&8X^2U{Apb76h%qk?ALI;IKLB&Wa%(-ZGb zY#QU@?*6N|yBQaEm(BU}bSkt=gVL4l1&c;;dlx!AO$g0lrS9fqWjp3X^rRhN^uWMq zoZ%28+{5nt&n`jDlm_u=@dvcx#pRl~p15FRoWC1Yq#+zu&8_hXtlWf!{iY?fb}j81 z3-7{Xb=p6>xVzL>uM+nX5ZA*aQ%ZNgc-3XkoN63#(=_^;lugvN;O&6;h{;{#y9JJ4 z`ZMJJ(UbZAq3yGwM_j$Kr3mL3R;Drc_6X*O?W#G0K3Eodu3e+tiM=?I012B$>S3=P z2EaseCTs!RaYr6!L5b2__bBhD=TvyRPQ=kQ@98q8vG~s3h?C!Z?^*k-{XZx zqd4l(c}D5VfeG56rT_UAKcra_9$rfiSHTB#4RthioceUM%6`4Q^-SGG>%&_?(|*u4 z9Rvdd+V~?Knq7DZ5sGg)m~HQKxMTCdjQ3vP{wNAlHF}Esz3U+gIqhns49d-|3c$Sw zH){|s?Yi?n{vq4q+_}&PB0_zk&xL70!TYtzvlv7sfw3%=8m~GWMH!Lg1Vd&}xRmS( z^sVjcMPb8t4~{qJj6ZZiORjw6+p3T)XsSXC_lHas7Y1nrO@n6@CA$9aC&|XHKWU5a zsD(XwvRQ}i%ZPDWdCcssZUAFDRDk_Wv=xO>>mh_0e(lANY+Ew{JxOQ)BLBKIGAH$Skf1^&~=`08urMHzZh$ z;IF^hHYx0*&KUB?cp_@A5S|N|Xd=Tt-rxg2o}HR3%yzsUB4`Tkow*8G{1gd+4EZ(> zE#DPRd_7%p{9q!V%%fk|KMsfFJlOo@!{dV&u(KloFP%##PRDCdYAAG^6V(w)?_p2m zt9DAq!sP5qIkY0Ho6ANN{*NH+ZPVHM-eZI3?U@kpuM(65^7Zo8^gjK*0~wkyCx>HiHkcO_P%}lKIn46-MM<=aH{RTO+w+Yir1jvH3iwBES^eVI zUu?p|m@~Lc2qDCujKEOP-rzz=yf$;X3m-fZH(5v!)CpUjEM!#o!#OMrpJe#;fTbl; zwob5)0LXMEnGmN?3{SMs_mg0JAkG2bTV}^B&r=-$`6Xi_FCj>Wv(&-laR>P?9?V#D zrn97+$6?B_1Kv8I);Vpei`?`5lztjyx-&5a;8yek%y^n7x=jId%^MG1 zme5G)wew-Uiz8nDbMwIfPz$tZ7NXfbiBb<-@ zect%tt&nN6MA)@z@DWfwkU2G-2B$};bZ2fROy2layzgQe6g)`eXZw$+x- z@orcer*vDG%mhvMDM8Y?nXERiV9DOEij%)cZa@LHS{Ct5amyknVJNt?JKGM@C}cei zhOWauzr>Fu=x8%oAM_;gTJ!HuZdCvWZlAjvR5J)+j6!C~fkZ*Yi!KatEM%F=;O#DG z+dkxue;@VdbA=!uj{2{+j-P`>27gJsNkN5Q?CM7_r1Ul~B!74-7 zPjaiiAaGs7EPESSqDv zB)*ousxkgQexGb`PHbES+b<3ZEj+m#OW=+tu|608s(j#E>bbYygE0##_X=|c?hbg> z2dt-laX7}m5Elbp7ZKQdfraDOS-B3Msz;6`T71@`cU#~c!)AI?%`bp_e_7D9^v-wd z7a7DWK|Pg!6;7mHhrk{yC{1t#=+aCo9?HPu0cZN*;)NeH#*!@1pk$}JS|u5|v>oKN ztYIw-V)EkEJqUq08`UX6(~2jcY=I{7$gmx@Y_uHOfD)~h8XM4S&{*KsLmS}FTG3{h z%@f^o@9(SMwSs8U;=2~SUbR8D96EpUt4i&g6;T1T47^KtQ)BdoO1JsbOM+6-MfuOZ z>mAoOoGxgpl=A0By|BrqMvpgNRl(m3M)T4+Hru|!2)>=e{XZwz+K5m}K8ZY)*|))R zdwz^($}VBZ%$oZ8|9v_9u2)nIFwlRTJ9XUhe}F0L#)dP=Viog5TixG~!A1XhZlgLG z0FK~1^}^xOr#H;!5@FNkA#^_B^1<(-Q0sQ>0PNaeLw0Qd_PrsS4gj_!zad*v0Fie8 z-$&YYCz&7NjQacFzwRW_Ac&;=eGplfTPkpp4>lyTr@YjqSEwdYsJ_vsQJ89LwE$jol-C5S& zQygep3)b%6U^e31rejMgP0kyIBL6+D+SB^VUr~#QUr5v-J1a-!8b0$~H=J+AAFj=m zWkhCcGP@`)Dq;lI0u9^cHVNG9eeaetmqfn{v~_VUWFNoTQC5q!wrXl_iz*Nlef-)# zs&;(IU%y5^Emx7~uY3TRqsAh3$Yw6o`B~^3sC6De#(jx_U*o?yYuNdEFg6F-1LzK1 z9r)Zz_ohiKM-`M1`vE3d6!oxB<2TAhQx?a~dvd!8&xY6>K)MUK{{K+UF6Y0K?v;WFG$;lzC%3CTQH-ndArfZl83epb zf>y8A%I8;jTC=2+k9G%m6JYmhOKlkxNHbdtkz-rHi;PR}d`B?c{?MUJ$KX%5YpHp$ z(X*2cObXdpducSFEi)6;koYPuQ1R$|jQ9b4Fgx7u9& zLm)2PJ(%^7$$sb(Kj{>{RsO3k+XD8Wr_wn?YlT(g;?M!4VDF=xuuWaSCEryp;pN{O z3@A0DEohDOwBfV)pZ}}>mZlhgFZ!^IcOtjiWT74(h}T}8pyWnM*m65!c2(CVP3A-x zc18!J9bYt2)L_(kBGusLoHsUSFrfbi3rF**caSi%>2+;t5>wLO96+7^e0_np}l z^5&LVq#^gq{YcGSfp7l!`pco5GeiCzTF{sCZ;y0zTlu#GSb&n|iFi~!{aV0eS|a%4 zjan)Lv|cHH(VRoh?f99CO|M4Wb(slWt z+`uDytex=nR4A34V4fCB4JT>f+Z3@wnE{>W@SFI=-Y03YwDd9)Ok>Hy*?xd8W9I6@~}V#K{_&Ua>)!?4qX@muxXj_&e9|Rl4*&v->Ja ze2;hQy+DNh&WtcdBMTSiSX)fECLtPd;CqCxN0iq6$72`OQ{s@v-A;T*PV3L&=1m)n z0ysNkHooDv#g+;kD_x{qV%-X166<75PYE?r3ewp~NLczwm{+h3JZ_@_M1ND`i4BS& zi6{E@M#2Tq3~64$t$(XV{}rFjw;U4wTbI3_b0u#rA~}XOaF(kRIGKJWOV{3SPQ2;f z<;vG6gL$8Y*l-x zk$pJ=f~In>mA^EPzeMK$)ub&b5jLhX_bN~PxIw->y=}pQMQu~RG)Iq=&e?VO?*ha+ z|E#taF0+0S4LBiaa6g&rW)V|bjLtdMGEtiinUV?e--vplu0JL6Baa`HJFj~?fAsFg z7%unOD$&-g|^t5aE1p1BzslWN5bi)Z|fMY{du0I&CD_J#OqkcskNT{W)`Y+mE6N zYgOKt{AzX7VQtD`J?TpT1)h!mCRCCpa^rshH1HnM@EL6#HTq`OHh%~JMnIi5ypw-t z;bN{ScC`3Fr{wcOZ1l7GSh21*n7yU{4j9nNc_Qm_K&>l5~hOW5a)$ z?0+~V5^!*U=-nZ!(qlE<)O7e|lhj(J`=ypz9qsvgVx%iC!Z|+$-01BZizjY~Q+HZ7 z0gC|Y{dR-Dek0d4d7Ju18Rj-|wJ=}spUM*;XWtO*S@^PuP*6t?CgevouOLOYm&aUq z{0=ACOB$`zsE+?PxX4{88mrM{Zs^S%GU&ir$sHe+%EG>Bvyq^LtTeELe6;4g0);Th z4H?=r0?heJ#5@{OmQZY~26uzV74Evscm|ke^Ib&_!tr=W=Xd&fx3vqJ#$J#UxF+|_ z>9h4TM@tkHk*3*{Ka_F_uv!l2oACGaEenr|Kzj_*1A8=UBU%$O^DT6ht^nGQ-*qC> z#^5#=hI?`T10D}LA#{9jnWUJ-Xq?T77zh>yJ}%~aKCZ)blaR3N&sThJHDs7if>PCS z)@yglk>@rN2dbf5SXU;BHr#Nzuct_Hgk*}-t7dcU9nL; zm;!ByGDOxv5mT3h#7iCZ^WmI2R&ODp6OOf!5HuC-*w_XfwiY3=T8DDZyUp60#2>Yj zc(P{RV!+aXD5lpux2j#iRz7)#_~irep86<5YbEA zA@pqz7kTRtB}zeJC;E(JQp&{g+CkLY%xMq3zg%Z*+^)1d|QA1Yd4QChJLy&0!CL4PkoA% zq#$a?(Y3jfyZNhTdyO0{z;I>nEzgWNvj%ujMIWxF zw91xG>j_sw|1~_kEJ8d1I;h?6RER^Fpr~e1TMD$!fpc72e(WWlINb$)2%M1P)}q_3 z@wq&$$C^_Lh4o{`awU`+DL&sa4zV)tHwGZM2GNB8od5kGwC>n~f1c7SPQ6L=cB`F4~YrdgzK` z>N$#uySp~cT~gkF4g=HaILZ8@yNOHfkmW#Yhd?VQ&Bhv_?MzoN5D$Tj-q%}jYIdJD zRvSz5?kX^XUJ!Ea>YF{cNr>x2C`EbkX%O)uu&>JpQ9K@@!ZK-KaziObbiReCDXk|g zqu~*9xJA@J1ot+YG!0Uf1)^dKA8&WlH$y`~zn9_K0QkEHU|E32X_l`BaQcB#BqI|r0DTK(_~e9 z-_C5#iisWFEA?ROpofb(QhrI-d1O=A*%wVbhNWV$$N%WtZZ>Gl)C-S_bM|{Y@k>%l zI`WU3_cN-*>>_4w{dj?pM?FKPk(TD4qO7gTo;rl#nER7+7##yBZId3@)SP9qlz{2z zChvA{DQWsCLDSv-+s$W=g};v%+1#Q8ZZZWfaCy$PMwZ`Kt2%fE#c-rTKAoLu{jvA9 z=8OBA56%ALw8Y0ZKMICK{I6wp@^-J(ojtp~HX>RoUEehlK3>63tYks#Z~Yt?zPoqy z#$z7`kHa_Uzw%$d%oDAtd!JAB@TM=TrB@7|)z{j=YuS`>D=EmMkkb@+<#&0-oexeb zAJpEJBunbJ*naN+i%VGx=T=pwtwKmAN7pBOx1`7K<6`w(M_v=0mxciM(k0e} zbCJn0zp;C1a@qc6%sq!}A?rC;n?yeN_6_bbfbW<_3CVY+9_N!i0piFWvGaPQ{9yR%yMZ>8mZs_ z|B7L`!MBATDEf3wXO5k9-1V#?R01dsM*q=%#Cy*zMME=_Xj7nr6`DI8hKpX?IQk|z zn6`t84EyUI4)3S{i*>O!t|iIJ8vI_y8YU?TCR8=@3k zZdCuDsMkWFMw^6iyV{a}ovif@RZAL6@iLNFeB_w=ZDpQPwp_Cl&wGe^#0)IJP%QPae|9~l?Hpel5f)=-KlOIgeG}SHJpbY*P4f4y=a-y^##;}z_=nkrE zc|DrNc*NJ96>4w&p*v6mG2W)WVcx|bkIu_-Zk|8J$WW`5uTB)52^2C_bgZqcXEo`0 zFJP)ts!nu_?Pv6(qYDs&z9?1bB{^9wb>snlaQmY9Hw{G$e@AxBjs9$o&^*fPn$dL_}-m z`Ev4+F@2%Yu@oyK2|t%VHn=tK$e<|C6OA*#hfEa@elWJ)E1=Ji_qv@Wt5O#cE!K!F z$8hc>Hva~UZ-@YCv#4Hn_`Nn82dHoQsk!8v_N<29X1!+5yPgbYsa4~ZSc_XtveEdh zKUP}HJo@qoO`Y22-}0H)_wWjObYJEnA776~z-@Db%36q0(X0Nn7EHu$jh=pq0J|){ zEVD{@QmRsU=#Pma>G?cQ0wj&+#`bv3zBv4MO8ylefodEQ2-i~?=(qYw%zRU_hD@`M zER+xs+Tcm8Cs1#;uQ38FT#cDR4i$uC5(s%MFC@X0DW<4_D{I zv$4UX78lFOx#83%;Z>jWx1RcI1;oX}6@Pi(i{P0`3v0hdg|ptNtRNc8-uqvm4XzLo zvXQD@S2qdSG?C5HH<$kKmyZ8m%USSu##k$OLC!isINM;jmO$um+L5b1)|--L_dLi4 zYmRl^ZWA$qHVnB~UjChu&_&C&QxHLgaP%|I6@KKvDKv6HNClOPS;Pj7xP6 zZ7SABf8$c$x}OtncDV^F8P5KY*j&^}YE&FcvGC%Cfjnga8@|5`^~IRDjJ1GD|8>bs zrWI%l%5!Nv`p#uwerY^U=R^KPFlkjs_%rYd2er&gmAU5pZSuSEXN=dZCiI?PyA%EWtk0Un% z|2o4#558H;xGeb15vr!s&$8N)BBsaZES5EM$;YnSQrw;^^VgDwReP@Rkaho3uLG;^ zTdW#Mfy5IBz07B?Le-lsb@M~$V(u`m2cF?z+2f*x5{ag|JuNK5-tpWdX|dD|iKEgp zaz2njd{*yNu5wE`9vLwk()0W`>x?2x-Art7A(8jKT|6peV$VUb> zPXeWrf8AABVt7`+4F>cE;Vgxhzy-=cdw-J<5ST?Dwn&de`2I;Ne)nVVjlWEmTD|iO z)>P5&9y;hsiTUQ6zCrSw&gV`8Tk1>t-b-ccXCKRrkRi7kdP$h?c-STwZ{T&{+4jFo zX)S_89o7sl>6Tz&PfX?q-?Vy#V+k*RcW^!7X9yrgrZiiW_v~g~%L~?5JmySWH@cc- z#Sye;q)HR!`qFSvKLHp740ql=97I!{xflI2bq%uvC57H5*?Wy@J01@Q+@sZql-ti@ zo=vhtO5dH7woq00HdFh(0@f{NyZO8J34}Odmk(}M5fAc#P??AJ>NgQp5mV|9w)l|M z_aT+}^9?`Mbbr%!T&q9iuO@hgYG;pWFh`^+SKUcAQg;4m0j8|56}rL&U*C)^|(TBxo18ufM=Pq$$!{{UBH{Z@4-T%XrM{k98^rGPLwML-CT zqy#O7?zT^lG%vVr&kZ5F(jyOvW-Sit5QKNQLnSOqiGkueJ%aF~0#ueLe3?BHE@rW` z!?nYWJ;h&@tynP6cj9E>ytA8ccluHG2Cgq33Zb#y zVk|*Uy#3Fg%1NDO=aqWS^12I}1&1w78530lA5$NeygRw4WO1Lch9B$Dt$AK)Blp7L zK1E0WvrfMzc#-B!fl~WMWt*}3>Df0LLZV2`9k7+dQX3?~)KjPLDLLK=x` zGzLjmiPItvFB8-o!yDD7e(5!l5i&+rT(s^_inolPxn!$MYu=5{pkEz7Z;=#DOQS6# zjU?t^O2(5S%X|0RX==~de30&?r!5^_TG$b|jF7rMwC>{P&SKLGAN&S`GpJ*=(h`aB z7jdkg)bq^`h<`#5^Y1*|C%_h0G^*`PaB8h5R%H>!5Qv}oD zt;xVy+?DxVW6_0g^t~T-j~{Oiee=W#y%OU=F}UPC+l;o+%NHlIT@OQF8qWRVRLVMf zak)?1>2m-9Gj5bG&aS1|+#E3cc>UCPXUF+>#^GA(W4S&$p$|Mcj|_DTCP^ja)WSo@ zAIJ9TjgOF2d+(Yxbf-wtmPY&=LoZ71x3blH%}fp2b*evcOZ{T3OR7)ZnYt9=5C>OB z#MLROrv3HSb?xtG3^WzrxuxA{IG^Sm(j0UdhZgg3N&f=w6p`S+rKER$n4m$C;~FqS zTD?L8sgY6ruoI#U&(Jx@fIjoo#~Qq56bGC#4L9l0u{FY`*Kyh;-?9 z{d9DLBF-53^Ke+;@~yImPAn-kmUo6k3H;+2F_$S6hTzJ|58x=Ypu8$ph;y&HZeAua;<0GvUX_L^X7)3kcgn{YbP7?Mho`@e4 zYYUHB++Ww>bg|%1>0TO&cS&&ER_}?^a-fV?hrOnDiq&PaLYq54Ap^(uHXVoy8`yo* zDNUAN_2*UVEz{zsX&r!34TxRPS?aIA`dQlIo3r&FI1FQ~mtQ?bG#Bbn^ob?&n52&8 zhJr;F0ZVEe%{RljUE?aLyrN`kC+r_A{Asuec|c@h7zJSwavDE2ltIOBl%Oce9&7nb zFM#86QqISFTfdCBJc7F8T24wX=eJ>t#v26355P;^J*|Xqiey2=k@2^4L&8aVkR(hj4M&ajD3hd9^xw4O;4Lg;drJ(`cVcgN>?@g5% zAFlPyrqr2$5f6F!=74THoO`vth4<2L?pa-m_lk4g&%sDra_2>EsQb7#2T>|gCeUsE zqmK-9MDMfGoKAE)(dtKXAz7mX@B>}s{Oocy*xmG#;p|F@LtexHeR!PBkvqr?=V{il zfI6Vr%Flu#UuN7vNNqnYN3~fv2Cq#sKACtxQN}YNR_k(eR;1dK*ooo?0b#<4?BlUl zrNIg8;$(Q!VIcQV@@Vmm4vuY4p;zl^8FgV$G!DBIfxnMx)}-C*y^?#0?zu zYuS&ze6U#Ol#2|{Pl1CR8x^O=jJao>Qw>)?9i27-X>wRy2?C#T?K&u-~3VPM* z1dtyvU#Y?CheC;PzIoEbeIp0#k=KyTT0DA%Na7&>hd#qnGnskx0^gm*-Dj~XOsb4_ zK5;LmVWfSDm8|&!HLdKihES&5Iyzr@rTC}2LPVs2ljh3xsQ|%~LnJ4gI|Ej_w=0pK zl|(B}n4G@swKr)z*C4zlnu*y`6cCE-B)vT+YRs8F7ki8PIzHsA==jNj9jd(#H81X& z6uDQKXUExTcx2E`zx-38jzR&4p35nl6DjB%ExZ)ll5n-l->p#UZRgzl)dYve42cgT z*qVIBrs5R?z52-=9w>{+LpAeDotmT|^Fl14>(6~KWOqkVM+>yq!6F}Two{0gq0{E$4I172I zF0*+yF0)cz&gq}B-xVUd`j;U{L`%ZY?MCjTj|%REF1Et{Y^BqhyBh$$^59 zrcTS{lFtw8P9+q3ik4W&WeE3Fjgrm#L%B|!mwlhYeRz{pxhyP9R@_57N@Q@Q|=YxKu5uyH%!Ag) z4=bZ~SDYvhaY}|2&n%m#D`(IQAjYHegc9&d#gXz(@-_z|9{wf@2DoQEp8NWuz-jK zM35pnN)r&2B3(fdklwo*l@8LS1h9e9m8KwsB3*h3H4zk05QDTJEecXY=p;aZyb~Bl zyjSKj>%H~Xdux63&n!%G%09cDy?=Y3z@drM+{x!?wwcPb?%IwqO%7+fpsw3jvG-pu^|% zX3CoJ1mr?SvB(&0W}rD?iv?OU123! zacXtP6VfzkrBS!33Kbc;cmBlez6CF(6}5vfPMHYbPQ8d2@A40@L2b_v6Qyr?d5Xu! zKRb+#oR=v&%&9IzXwN)xLu)MU)+@v(nK5rK!!nyzREx-1W}kPkazvOC_Jc}Pk~S}- z>K%IZL_u>4&Ts8=sJ*$-6uic1rrPQG;`t}A%3e(ktLg(7S0fcg#WDBxF1Lo|<`Ky5 zSqgMk&*)#C2g|GZm}q(53o7YTO{$LOHddEdwU=*k=dcF&_zOU3f^@v!G36Ju3g_@l zwn(HdTLkh%Nt_f`%UO<@N7tt>U~l9|5O z3_2s|RIP%#k1p@QqnMT$@zkkfb>@_=rhCN^=irfgkp<6U<&Bt2*Qn4Vu8?y(E;PIdPW?>&<6#6`g(H-p8qu}mj4bw;B8zQvgTFl9r~)!>NI>jicy7Due1 z6c=&~U}+~}+nAp(gNrw`GUV{amL;dEoaX@KWJZT_VYW%o0ia~eJlZ=DOU(}U?9h2! zThZ!tfNbw!wbUMCCXKNmLy2>;h)7k}8bUd3hK@|JfJCcs_UA%dX01qtu}I~Nq;$#& zB6I6geN*b&7&I0H(a)_3MTy>rs@dH=-Sy_}dZu~(?Awz@YQpfJd3 z#KcR;^sFP=mFdF{6O4VzBQ5pf4Dg~ggL5_)@~Ecz=P*Vrw*w0)e*#ePY)r8v9zu;ecG1>NNt zxO#^~9Px3j6ckdmlC`!9gw~N721)uIK2m9e3EYWp(^QSppzC8?p3ee^R}uk7lrOCj zkiN51pfm|OR|H#qP2Q%=?9(vY$qH<4GXdf5+SKPB-b%l0GVG1a47DAbV>V)6D9JmK zxQv*}C4g%sts`AgH}-k6kgUofJ{P&4T)hdekOx2FTL#M=N=Pd!1$cE3&Oyi2F3;mc zjkp&Sf&{_K2I(l@wYeUWFE$5W3Szg|i>p}V8ur830%8^C;2=Z%g0d%(z*8&zGHEVe z64V%V-{rPuul~6NJD$|#y;>{B8tC0v+0Vzw3^f}#)^y#?Gft@Jm|{sqSr|YlqxMlF za4(!eKEG?p%Fb%;>@LJ~&2JZ)D)nyY+<=6ZCp4QonH-ubwfNe#FeYjQVzfa#i9VoZ zvt5|Rax-S7xw#UvI*9{|GG7}S#;oBmqGX+jEFTP&ENzPcz$4tCIjG5No?4rnvM0%^ zr4TaJq1Mimz2!Kb8mQb-z2X?{QFtmMKyFXZ@ZUjq=g>sm; z!gnjzA3hDmxT7s5iJsa=E4}v}FkjpRPWh1z1bW2D=};=M7J2MTj2I zY&U2!PL7?uQWA34sm|U zf2F8U4Z?!e%T7ndO*TdaAePIp3*EX5S3z38EUB)!%VU&?sO-<1!m5T)(HuJ@9`fG- z|H|3@F=0>-{WbY(|as8f{F9YuS_r1Ftl<;I@+T$RcDr#5w|f7 zF?;##$vTP9x8OrwD%l+1y5B&cKM^{ch+sXpUr{D?^@$L4IB)F@$Ndw(3=f^S3oG(| z94%Ecm@{vB+lu`Li_K^uRuYvvMXbNf$g#{zEnG_PGTs{9H zz$Y{_*iUW)>qgCOb~aSlo3Zc$~~fsY_H4wLi2`DLU~hw z)&b#n4^Xld(CRs+srgEYlGuQxCi@9-`WK{DXaBz@v3w6E3Hj|Lmh8zc$}t{KctZV+ z8htJYw{1QJGMOs9xfwa-U&%`kB1?>#-?rQmb9))^(Rb{;yljI`y#P%9oXd`|!ArlR zkH{%{$FwwYxnsBo!8`)H4x$6boey)JR)N{4M6Cz?KU;(kgx4*8s~XBQUd{dR(Nrcf zFq{)_C`Sj4FQ9G;C-6&iiSBeXH{rA1vp&^rFqOjZrs_COU*Ic*Z!-f4TrT1XNMK5` zxHRV5Sesq(Mhty>lt=H{W8!{(`!}m4VmeevAepJ47l|$a#!nQ{EJG4z%R$rLT zqnam#d@OZP0E`&C~qZavApyJg6H<#F9XRwE+S_h$a6L~kOl}~Gqr!I z9{^*6?zFn{Lb$Vc%8>V>{}EwQ&hYiCj%NwZDlm`tSXCZTDV4bNOx2Qm4d0HF@o)bL z`-(k5ABLQs|G->$Bk*|OiR9?)&5SGG7+Emvg941DU?Rw8Cb)0xb%!;3edM3cvT(q+ zRsHSJ;_BM5kR4%fFaGW*L@d~EaHJ+AOvQ(-TDu`n;(r<4Ip)IkNy2HsQhPnBIZ+>& zwV&;U+Dk}q?=!@ufZ;ziMXnF;42GBc-S8!0QtvKMk2&#d$ld%c?H0Ao3^gq9ptg64 zd4w2D@B#{MHTQyYopQ{NnI>DXC;IRo%W-o9XY|Ef%+iYAdBnELh)x@b{>10~Ov7YA zhd9^1%HUGe+CcuMYU!i-Dkb`HcacorWxkC&I_g-n>CQ)%F9xLtb=;gSSD6DMmE;p5 z&Zhmzyxf~atz%VAA0NZR@0gHQoJS`W9rQbvvc`BkJ^>pD8y)PAQK}V~uT0NONmkM?GS|If^ zkMz?c(op#z&$_^1;TX8%RI5zwxw@yC$c2SUQGg8kuKxCqWtXEl8+T)w2kp!AE+YPi zS>&UDK>9OikU30Ac)nb|Asc&^9$KAA2ZY3$=jh$jyVhZZR-=J4(|*%S?==dNpP;ct$yOrt5ank{C^iW*Q<%pnR@H8F z_PmZt?;8m6w;FyQC%xn4Z1Kl~0P~mlIQGJ3Cg{|Vy98mILXXF4%LfML3a909PQ|aJ zt5-vay@-IIeQbHfy@OI&)?gOg#9J{sBbk8l{i4&)nSQ&Oq#;iH(X`buBZZ{+%_pCe zN}5(L#=dhydDngNpIauCFq1pnEatZ=@iZcd2B7@${z3lHW3f)X+YJ=u#N9z zZ{z%(d`rwr-93=nCdonUu;W~4w8>j;(`fl+Sey%o_N)1his>W5QvoJ&bn%4Tp?+5) z0*AG@ZoUt?BMkprl82t8*A%WBx-y{kr)N&axxU5s&2e07_^|TZI*&FJ24|B_e11GN zuk%quxJOt}N1b!MS%CF>FD?d+Tf}9Bp9Q6YmzF)Wt$W19u#%3Rc#R)LNGaa#D0{zT+43fpDux=g2KD_CzKPeP|iU zEXWgbr(|?$S_R4OzW&3$l=*+I?@h$^>u37k@=^C}cxZ6o>TAs_LuzOy?FyfEmay>g zO(sg~rLXWsY4_*Ob?+jLsxd7GEl20tyU?IYQ)GA{O@Z=J$Y& zVJA!e<}3Cuv*@>B6!HreLP6zBd5K%*n`i3;&guRlgr5u!TF0~EMpA)>9om#uJZf2MW z38m^yiKV?dQu&%EF|DM1DVFBw-A!oppVp3h_6&?Ww$+N*I%FMfaQPR1!xJ^fQiUL1kADf3Y!cz80k^K) ziZqku4GmmgG+*f-UJ%1f2dMw#`3$yKyVW1^+!uj^8Q`8sp}eAcDqeV1Jz&b@m%&Sc zg-&$;F!&S1G5phUO-*#a*XrYIxVFSs2yiLtmA;^}RY@hLjy0q`ir)B^#yp#2q#X0^zX)Z(XBC{b{K5&3L zf%tVUY}r)T39UU2;i1-f{cxbP!0xWiKd*b{F|^u>YaS zYg#lJWclG{gdjk9N9UA;^47$d`HWSlySLj>FHgq0G9-E<&x$*C>B#Zj;C!<&BIRUd zVrgs~Nej5JfV#D7WGclk(D{B^l&UZ=P^I5JY(low0OD!|>f#as#d!;4ho?hPzT-DC z)R}B@WR3+AL+BbIkyC5v^5)cHmEl- z#f&b6$U;d<%)fAnZIV!DuOyKSUdItT%%(qv@I&fld?)VWvi&58)2cn_yEBccI3)4Y z+n3Q&4mrO>-G6Zjung%#K@UV)eFo26Vs>pKc{uC-&{~OpjVb|)mokOlbqU{|m}>|{ zBWSoyCkvd$B9VacU;0w8 zH4PGzgR?+zk$1gnq2$52(v~AOww&$gF2Ygcgkb1e*G#fsx1H_feS5Z4Zxs5Q^w}7w zM1V{74DN?RpL_V@6uJ*PXHIR;-{1XKzV#5~|r1c3V|n(Ed^b3tI3 zWWRB*?zaVVti_|YGoIS9k`csrTQcRF&EnHd_=GRV_}bn(i(hgyO}guIRX!L>GDsqr z4r_MS4LW!Yzr^UOj>J(}pWo5~?JCUQC+8R5To zXrmFruMK{!AqSo}r`{fYtuT^3#*s&%!BAuYiXK9fQ80LkvcwoO+VnP9X%0R z97qT{k>_tIgtR%G2m)zk&=3#`Ubwlp-bCA4*=p&X6o=#%nFA0q$=$FnAgcmiWLs%y z4v?8oYPOJ>d!Y>{eSxih8D3eMiz9#@SaA*@xs^UiOD>rfqX&JX@j@JnZ+3|aR?lnA z;6Zok<3tH1+_44z6Gk?g_VYsc(YV8NmGxB0F zgD8Dp)&`|o&VjaP38b=q@juzDc#(>?FmKCr9oTq;$g1I%W*)>+*iLt>+^p35Sp zJr`IB%d^Eyk@KF}li4qB$P6wCVSLPS=Y~bPotn?aCqJU(gM-Bt&0#m<{+@z zj2BsZ+u)l@AflwnWK(;(3mIb7cj(2n)+bVDmj22WNj%+k3ikewUvouN*1H4~5%`ik zyFA(`?@4D^k2j$bBfaFlxUEdk!sjz34%6=;MHVP{zo*TdAZb=`L$rLC(|x;F z{>uLhFhF6e6Bck1iJX&Fuhy@Sj=w*ZF1tE5EIs9cDvi_v`aLb!>K;`V?B6gGNQ-c+ zjw<$?i1e8g^L3t^bVqs-Nxn-lzU9?ZwU20f26)oIJNIs0J+y*V`AZ-#*oPy5a%`D} zOm(|Y@se=~N0zW1%AKwuG!nLjoKAMAAT?{gNDJ`-vw!a2e8>FW?o7HvFN!o zX&P2Sc~IJ8BDeeB%Ft%aOE73O)ojOp#a>WWWu^T3-`UAaOa=Yx$$RnqdZa zVsyZkE~;NP$nb5HIEp+dj%35}E62)zznH{q%a}t0fpocZA%C+VAcm0FC1pz0$jr-Y7 zD;~-2h6KG_L7EJoDOtSqA;H9YGi_^LA@dxWxz>NM_9A_kZ8ok*x5JkYAJ9(n%MvR+ zVyHz~ax>se82>+TaUTLSs>bpa&c-Qr-Q@53^ z&_TX=a62zqUTuQ_?|COO8(!So;NyRuwN|S1%|Gz^CXTSy%?M^xJYMJh?x~V<7YHHx zA7+(@#k>NrGVEScz4P1WEo;M!U?GC{UVdGpPa`e&A)|8#X)+}K1^tZ<^FlsKbI7Yi zMbDit{(5A&>LFFuC(e%$+HHzi^rOZj&ZveKt<)j!ef;A7GJ;<$C>dLcX5^9^8_iEL z-feC&gPx`h!P}F$_CgJ22*;eJ@tX{CT`W{B$VQjWgLG!z;{&+J>g9<=Jx4xAEv{^L zyyCC{E{l!vqEp|66X*e)tQ&+pRbg;rwm3?u*KkZmP>Ar%(DLI>VRr z7&Kh{Ls3avhcmxgSA=wK+@PC}@u&J(n>w{~GoD#*Mo`>Q7^Ke3DUPOhUcI?pt$qot>s?l z(1rc-4I?evl*s792q*`&&Sa6|SR_q?_-w;a!rAr~w6(5W2SS!w$76|A;vd}Av>wZx zjgZXVRxP(e!BjtGMOjxy0dhkNJi=6wuH$_*Ng(rB<>cDYm0SQeaJ7y>C2f6u`;~~d zxQKwiL0#40EAPOu=ydxMi(uL1D@#PQWx-4C#PL_cPMGxcmn_>OWoGHa=dsbU&NHZl zz-aLb?zN+elI78J;;+7nr&00d%9h^DktJwtQw3V9JU(B1+TD88B8+MR7<}0!cXX9Y zpUk`P`-4YlGCBd{%D+GrY*V+FoP5V>6fG%IeC=^ccyX7K;)Obq7@Z~$R;@c4a?K+e z!_KF-cL-W`dxQd_ht%l!kC1zN5J~HG>gW;^qfdk3_a9JM%LE9+ApdfYukLe3sXMl7 zA4GG_&L%CmX8AL)g6O-w$9N}hQ0u&yUX^(jA(zkx=Nxq2I%d18Y_l~ZGl&}(5fouB znHJd?PPSj6mXBrk{WcasY~@)>eynU1DQkVD=JO5qR+?)3s!dQ9L>oY0*tU1nR+Fvi z-$8Dq*~rC`(zRLILai@`&(jGW@|3UK%}NMVN&hQ8%=C+`9KCW&E%)Z#p%Q&K+#Hnq zaluQtecQv)M`u|TDRsF*9|xhJJ!eN&a#OZWMf{>?lKni1qv>Uc2$iN&edy#BO@I=3 z@61ZA6MwjAdH*)0x6?$fPE&3Dnx}L3yo_uP?Gw&CvG3ri#Oh5={!PH{Ic3^7;;c;kMj;v>xcH zQIqpPdW^@N%R!}WCRK82ex}H)uSaHC(QrQra;fd0S(Vd=DG6)-=mpMeTSj7k)gPID zS^Nsa2Op_?M3L{ItM1q(FVHo>Q;>8x^@k4K5*p3bC61kE| zQ4YWwK4`OIfxAX6v$q15u*Q?c+?U9lTC)6Uupk4Z%0sude?1*`R{?+%bnuyAd6kO0 z221|Rj0HLEu20&mbRu;t3;gcZVMe^bLmE_k!^XEY{q6PPLI7?F8txR>gj>@-F2AZd z9n9EtYM3W?*Ty(TA9qmeREv1K{fj}0x%G|OJd|(goQCTyA7oAYP0%XaNKIjk9lY^5P9d_DN|v-A)V7-zHElz zZq6(1Dl_e?XV@qu&%Kbvb8NV{cu9$TYjOiXdT;H#!i@S>+Kku+*0VI+uMhk#=L?4W z^?ajfRpMZOMk!cj@dsH?6!`_Vt<~887{F`kftYBCQ_e|#wJg6yB!X5?n^#joEibI0Kb=ag4cjNnvx>2gmo_UAKVl-*Q441dkH{Al{+&pz4+X! zl_wS%-NrBnf~>aChLQdMU>zzeLU=X+Bc>19lRsM8>2ns+XFhy!qhG7$fMQCArO<5# zjRo$+{L8;ZqkzU-n>p4fk~s&*tAOV;<+BR(z)J3fa=_#%b;GD=Y<>&*0`Z{#Tzq%# z#hd2biQI{yjN4=`{>I==H^$>cCvcZK^#YE7v=C;12mY@YJp|m5zEAoLn*0lF4-jD} z@N$^+;z~*n5oKQ0Hf2P;UH&+H>#_~`w%PZEv>fNKXzCRet&2TzZ#nb=reS2BC()=1 z!YiSiSriwx;xZ@W`R#?DLnV4~nzB-^_zPJ_BQe zzB68N016-@Dav0TOqJEoA|v}hImpQSXMF&2DRMoE^lU5#-rrrdG#uA+a2&jvj`0R> zsP{@OkM^X0Abz9wErVW7^po(;^sAV88mG>vZZ$iQQuiE~N8{~K5Fn%lOe=~#zzgpy z!`cY&f}lNOK2lWq!lha!ME(*UjszvTP;%MG%3_18O!?}1 zNLC{aJdK7~?i_A|t<4XUpSb)0sm%rOVVP^5>n)ZBctRerGp8B_8^Wl@fu*pU@||9S zhrKaA`o9H;e`_RvUuJ_s()@_;+6WhEEkbrF!j5tQ9L8NQy3Fqo5V0NJ1tPZi83K5* zVwn7{1>V(^<<%nPCsi`4t0x|7es}Ui_9P_M!E+QTTTwnOi*V||Wj=510Sje_F3E#q z*C4P@khWv_&Bgj{E7BT9oDqWTa$f8g};9)-0_e9DeH&L~8P%!~w56LO_#nEcGkFl=oMR+>IUoUo9s zgnlGH`T@1SlKzzl$~G!IP+t8-{Hg7^7l0bdcgbY+TzQnPZf_%WyW%27H^2BUQa)Pk z(UGQa2LIE*B-b%%knj$q(k1djn!)Ru1@Q>b&(pyntZ>Y9$`k!zP{5=tpFiP-Cur^o zhu4iQ{0z=!BtN9d^rIoa&pMm}L^KmChhuqFhR~QW@ttWriKL|s=L;}mxVSBy4`Tn; z9KfBJFbLbz#E@P$H=A;aQ(^ly>Is$gmvu$D_!0pBKG^Zoa00~#!mKB@2($j;2!X&* zO#LVbBzLV<{@ZDnswifC=`Z}Nr7qcf4fnbQvx62@IHWPdMJP>1iN(Whnqw>6P%Y=< z0iOC%a>|qh8cH&4)7-RnH$9U7O8p)*zq2Q;0`y|{3G5inD(Fhtk1%7kiYmefzbkIu9$zi(*PHA1AVRxY|Y&_PNS|cqexB$ja62n@IS0uxvySl zBXndH?#4)O5LyYVc97H?LqeMNgFtO4l=&a-gQ@lq-i^rYIZ*4d>vRKX_f*Z64dTxw zO3W82J5U9JJGOTJbKC8%ZWLie?zGaM@f*4VaUDD%CdMNVy3BxPM&14HeXC-NrrE7G zUI;R8Z6ftIZC!_#JzpyG4=YXo_QM*f#LicB!sf!;Tf!5kZ~5;uuYk^tj=(`HZQ=~w zEqQrsFTo_U=lMnf1~=Woq1x-8v9cwn6U}_*LU349z%H|o;ZL8!>qf$c_BRdwT1)r~ zs^QLNB1X~%3{6^0#l7;QG0juD+WcR0Db>^d~~*9(UhTxg>-LKE~NRZv}FYGOij0E_}Xuzz%0Trl{63%}7i zD)mCLpM(^k)`e}ZkwGJXi;=UuFoWJa$-R1*%qrDTT)Ne(?Je<7nvA#3Aj~1r^w;ig zezybK?C2KSY%^0Rl;64A`h4gIS-=l77lTgJhCUskJoY z(!^6`O?1UP*xqj&5ZbFpszL9#9-$!Z+K_LcRfP6Oz!z*pr)epQEv_n?1!JtsJ&Pk7 zTV> zX%JFw#g~E}r+99a>@o+P9`~bn2K4}A@>0qvo*dLmF+03HVht(pdi@{| zv~+x2c+x)TxHWFwcz#ybRA4)S+VR`+5O6EptUjGmcXZb1kCNeuqO?FFrOI&tSqyxH z%*8vsQuGwt5-B4=^CY53=(-+3LTYE`w#6mD*jIZ$nxxjbd3=qHxn`Tc8Ux_VVFnw% zAz-5tTan=UIM+US=jHa6I>2B9&YAqf&yJp|+^mAzBNAvj*`z;eCP!A0J-$eC6Fc^R z!S5avzsEK2_(jc9w|g$pfO2B)_yH0bP^KMLGc)@y2$W0RPF6dsBr-au9df5?ds|KR zBcF;5r7P&qoUYaoFgWrrmMhJ+-~!Z^C}|0~8iOPGj!Yrl$A!erw#Stqd=-<|l42Wp z`qk_clh;Ap(>e@vWDx`nEXhhMzJjBnd5+?LT7l0ZSMrcc^`(~NdN3a*y*K1z0=%2g zzV%XFfjQU!yqP|Dm2UEOd6A}%HlyqmL+9JAX9U-4NAt*_zz-& z1y+6o>RYiBuzMew-5EOT{>?N-a!;Qu%xujsAQHF%f*xf7*2Z;?vEs0XyQ~)`lXz!k zYfky^g{^^b%1Z_fvI=nJ2!(BZX29u-+;G0o$G}M&+dl#*A~GU-s`hEq5GD5V?~2lZ z)~Nu&u#KdDF0jHz@Lrl#a%0+Da+a8xFT80-=Wh!zL9Au-fia{T22qWJU=nDbNL9}0 z`O=>r8@bKjx)6#8cC;Cbf}H`362G_=KYWc1Z2o`j@F<_lFQ5M?7~Cik_>XOg>5cyK z`7gBRc6c{b<($0(0gvxAMZYvf7xND1-xhj(o{#2;#sN*GrYq2B4TaOk)><*{r%ILI<;ii*}1tK@B&Hh8Yn|Ay%cnqdeYTp-@!&% znU-&gUUK{GsC+4n2Nt^c8TqUTEVA>H>B)_oPO0&(8ePF7M;DVce)*ryt68Y>Iyrf{xEhuhuuPSG<<1sy zXRhCO2$^H-r_BTQvGq!Th8phjvy>En{dY`y4{~+%9Y)IjcO;|D3 zTL4IXwtz(o+Q0ZdGGg3QYL|04gvGuZ@bAeS#k&2S>{1Kk9Yp}pwkumc%*#NU*FPVB z#-{Uhwpn#W_2NKKXSRM`*{y?ay>G`g9sjgG^ytx}&P;8tZ0+>WY7mI+Oi|>R@@S`^ zxNfSXa6yM2Bb{6u3SA!t$>>jabU4)NFBw93DLi5$d1Q=21A@uOW z4zJo!^30+65R`?28;uP;`x@b+z;)a`l5~-NvShTWV_R&N4?xPq}mYl-aii>DTOx zipio$zBETgZI4v0cH88ADZJHm1AcO%E1)A~g_dZ19f}5N&kgYGp=Rn0@LoQLsbZ#)@zO`DIcHo( zHLw_@?`s60p(tSLZu!HzZ4O;uD1HJl?y|UGAdUOh#pE`z6#QQ8I_|^a(!Wbo{DMaWUH`c9jc#C!Z^RpP~D zBIBoPc5#$BL|ui+NaNKhaGa8V^t=VCA_q#ucNX4KAzYBLY&oW{?)9F%MwD~eBtt7T zP#PeCJJZ!U^cCaYv#%i;E8M3h5hW8#6GbSKBzThO%Ratx1Pr-`50g(@xYOqki zU+HB3W$_omUtZLCmmZcJ>XygvF5QEiohFn2F~#^9SK~Q^>i+rqqYhnwv2VctQd4_? z7#*;sk2s|o7?O=8alH|oC<7cZu)8$i+uMLj`^jDO>@Jr$T9!2cE07P>;J)3B{wfdq zU>~}tNf8yH&eLW7Y(%nP)F2sM5G{37D8NENrq}lV_(@luD1O~FR zAv34f<%3L=BhT2pfXJvDc#(yGVf|RV9+m6?JY`=)W~O%=WWBMcbPtcZul}i~CxtS8 z-PK<3xH7wjP&P!#KxJ`O5QMCxr&v@pF$!csKhed$QtN+V_l7Xt$6d!Dpp)_jvLcg( z_2!-M79ap8x=zij>e59WAMZ9vS7Q@1le8P?I*f!;9 zQv&aO`DUI1t|kmfsmcbiqcT+28L09w=acR|8v{4?V4{F!d%Y*|Zz~gi{=cn`0tNMt z&;5!|Ha*$B>FcF8&mW*<`0+7CYc!etsDAt{*ZqIndPGO?9nzYEkqxV{B|qWyTLz_7ynm zGb#T)Br6Y@TQgGrjG)wc;2-jnG2ak)4b>DVTqAGt`Zc0Za3cHNF8{D2%-Kh!Ze~c& zE^6B||Cr~^({M7+N+=66%Jn&u9timHo<1GiF-F+M?(=bNrX>X-^7nkgTz`)$;W8TD z=c#xr?u?_>>&d2q!Pm1;dIK}F1P9e0Bl~?+B71afCtuJczFMoCGDL^TKIxlmPt6vw zZf%F}{P8`x+v`FV`b_8kNu^uWziif~wYc1Y?#DO%J_3iBJR(iB7XQgTJ2luOTnp&q zN~UxEUShc6I!PdM*isx0C$Rv5+jM$1Nw}|OSpJmaz3Z+f+m6oHrZ!Qz)inuFM69%N zYL=-hE~NDOVEtF?A2U)gNBkd&UMI%?(Pck);>y3jxcsLUZ)BDG?VV4fu;?L=_D_35 z@yF&HQOZvNrwG^Fbd`F~z3HNKlVOuV5)RyS!(zVlb7a5Z&35YYrfJ@MaeOne%VD^? zH{?;xB`eOK=8Unv=)tV1+<+iy$v(GlRw;qMTdTC?mDFwo0#+4olYXwL=+q(F*VA5i zJ$^3k7?wM+Jk2H{=Zo$UzP3(=Zq>foiFcInNoW?6R=ry78vPZ5@M`-f?+r93O+%zkth`QA#pwvvt%faLx zqrzc&V-rhRUoNzCng!uP5$w|VWq~LC$c0EM;@e#w0tymH*Mv&-=lN)Ts3k&ooHYkM zhmbjrSsPiEBJ_@wHWmrSFi8(TdTQq|qkFIK^9hXF@)^{c;A}4x$xoA~ci9Zx+?U*w z(r|aY$h*h&Ohxd|1GyQW?{}NHW_d>#e`o((_hawMLhXc{9@EjzZdok|QX6H?dki+o zo@;S=-Uwl6JZctKDqp-x6Dmd1GVgiVJ`}-T9i4?h7kV2f86zCSzM!8rERXeEx*m1a zRNGS4hmYm?PyY6Mo#HpF=rB}q<^vzi)wR}BC~A>HANO+#N;)zLn)_)JZ-as%!*f;f z{vC7Yju?L(zVXcBILdITZShS;)4XNg3Fd-VEemI6J@t83X8oz8QVq=t>@;RPS*fYb zVYSB@>1xs&V(f^`2}CK3nhiUfnfj%$%ml+^3gzl#saO@|CVgAMVl!u&K)WFQ;+LD4 zfsOhc3%j~}S7wG{*zm1ZZ#DLX`I%k97tAPz%yTKmSKmqp|9M0%Q$GUcU(>dtm3-|A zGR#oV@&VJ~E|O?t5S@E^_!aquRNlqqbLJ7A4olL#f*1iS*vNg+&nv1m6j>kMy`$8D zInYilE`P?UJ=`>N{MY@i$>&2ED0uG9az~EK@X3^GQk)pusWGm~z4(d%&$@A3mG$e+Z!={Ffh*p=dVMa!l>0+oNl8>O6C`002AiNEiCbO?Ew?`<+&t>jv zeHZm)M}k70;FX>%GYy)8VFzB9xvQ(+aV7+lXQ%z1AxoQ!B)x_!lUC08M*HuWH{;u^ zY_3y=+)slc55*5E;YTiyC~2X%spVqa-WO%sD5G~MGaQTW5JpJ{+|P+NXME@X?B>ElnCBK@ZWvaki~PM>M`Adlc>KCco#FgOrHu&V>V8qXK23oj7N#HG$a}RK zb`2U$pXpvHW#-1PgcrBCs>FNPXC7(99%{_HN*U=X!*5)kYcQ&vK=YH!z64Q>zmMdQ zm26wna5l3=mcsO|5uJnZUN5M)=_;A_dN&&XH2lZBfX`i?{B3cC=+>5yylR$4q5Mtl z%ruX~qrbI?-^3+Af8M;tym(z~v89c!d8U9K0FT?u5Y8Go2?clo?cf1zuB zg@OY*)4^AKOh^Bd#8mCqffF}A#z1l~e)npu7tdLB_vTiPMPFMu(3YX3O~jsz>4B-8 ze;dtj&TeA6%NIe1GKV;;Li~QpsLqGjxPo5`|<{9HqnWJ`#|qFJ{k7zYQNm?lfJOB z%5ei_$dVlpmS3ZS6GV$(;In+mwh;e7=D(E+oz>eg>A zP|AGv_D-g#4{+y^3JCn#x8$$MG#347s(QW?UDrwu?WF}VV>{9mX;l{6U1DQIS&PT@ zKt|3)32?a5=00;I;&IL+#wCn@M#ErJp$+-pPCN(_uC+NK?S*&24TD$F`8v}!&W2B$ zvVFwaZ5M~cC%+e!fqf$efDl&$8{$?gK@^vcJb$+~e<{Gbzfj{5{+7yCZFwOLZTjjxVcyo6J2HGM;vp$h)tJluj{4xwc$H`a0%q+~z3ggm4x4oS z?RJN+>^2^xHAigJe1*$5lc@&+rm}9tJCjvQj!mnvC6>2HcSk>8bA6%XH4_yqJN&uT zebR(z6)#a^-N3={F;=7r^rbfcqQn&+>E@@Qq4D+8b>0z^Y4p@dzwRoJAe*DwX==0r zwbwm4##9&%ajEVTXx!~9d44eX>_pjMXwleHXYC5yE;4!V*f!pzD6yU6tSqeyd^j|* z+yn@08Pn_q{Oj(Zpbm?y$ZU#nEVJ{##k84-^g4dD9my$&!#mhTxpb#Shq+HjmY(l< zgZOCsKCk_9w=!$w{VRepi3E<_0)a~_nHCcfSckoODKD30c4X`@h8tnpP{rp}KtJWqwGMSW$?YI60!?S)tods#ul ziqmBoI-<5+Up>vC=iN;1Xg-RxrqIpS<3|S#futC>wqGK<)gKyPYA|a39wTgMA8U}( zNW-tkL&V)~3j4)Y56cxS&l;^wHP=lpSg2BTWLRMh%FcQ9C*Nq&)tSmkS$NRY#&c^= z;=b=ghB<Mu5tq(365*+^Cg^_R# z%=XGRm#s{(Ky}$U#7AqVb#ch7wD_#95%I10*Q3o1Mx&?BySz8eSJAQF%far&it4ZS z!nL18sY-lQg@N|nAOl|#p)T7aP(8rwMX0XVySum~^?{08x>TT)P3rC$J@jB(zuU6m z$Wo_p9P(Ncad_>;$XeYe8Pdf{0tbEhw^vf|UPt#{4JEVkvFILX*um-yqHR=`Tm5;a zvIftR+x%kRpTgWJHJGf;y#`g`35~>%xOX`*SDTF-zOcDtj=py5S)I+!Yp~~go@q=n zB{tQ?ltVMni>CLJ4&)&wPl=t@4p1|l7--6M0d=oLj$nSYhNfW0-DZKZQ7tqCSC7Ql z?~o<-5_aDfG4nc6YF_LAwaq3^YvB^yS521a4bKy`?LA^VNvD(wd1cq()_Z5pPIROv z83LYq*k9sve7qp#`r!SO;(uVmM-t#+yrzSmSH@%(eAfg!XCvEedkRPI_^|lcw_`sOF4M!ZcFC>r#k+y0m^_-@msEO4>J~$ussLmWh(O_@* zJspLL!3^Wm ziJzEDZt|Iv?Ih+ly1hXkb!-TXawH1a?nW5)$rMDVCrp;6*L_Iu4*q015Iv%kGmP)i zZ#FWBz^wM;(qCG4n4+>0r_TDt9<*vRNBcA_oa42l7~)Z(5E-9r#1F~O)RY1u4YJLa z1n8K@jINrs-bsS@`slYNEw|2?rO@||T$!FbuN$GCQ;3yW>srJ0x#Pq_Sjy*6{?cAI zn#3;qAdwE{l1bO)iD6gD)7>46;guPDOP%jWUVjof5hD=N!QWk2*U@aigj*QRX`6|2 znDDpnn^@dQH*x>{lGF0Hq1om4PwPIJn|*OS#^OSJK(*kl<#)zsW$__p?efCW*WD^2 zCU0J$q@edM*hz64Qlz5VzB;yAcs6`8g}_TN9oecQN^Ha?K2sYyj03B2?l;JT5y8CR zz4Nv!uN2j@U2t^xk+x5TSo@XA5H>Lx`|`Wy^)147umo*_VTrZtosNeL2E2c=iTT_x zS=HB0tFSXFsTg(CNuTTEv3Ln#GWC3fonbk5*AjqyJt;g#LTLq;jC(Om^3ML1b5EkJ zNMkESHu{=Jwbaj%eD)A~UXk9QCbDcCT!;^O%Plx-cAyn4-uUe1|KwHqK?!hhBJ{+nE?rI&$sa>bGXgBlB4F zVZ$&hNn32cx8!quOY^*52YoT~0|arBw)4}4xPm4(Htu{JaW;qa(G*`PmP|lSua3Iv ztPsn++)y2RIlcor@GAznYv+~sTm z1JB9}DL0x{E;W@TbkYl`q(`3gtC(w3@4Hl<7Qs~+=(LZl`I$2&_X=ThEEjTH_yvPqhpFS$g^ zB;f&vPhmKKCR*HcsP9rjNSr7&)A@?BG+LdCJ1O~T4oJ$$S)#Vh?Sn)>wheWSG%iiVt+V1aHc%p%MeYMA^a*Wp&l#YCBNGP^4_Z=)9dwZMzMw2V0E>m~0ESIlH9a~&Hy_Aw40u{tlDOZs3GEm3tN1>vEa2C2sj(s9uEe)Nr9uj8 zLdAlB+?e)~8P;>CW70ip_36m8HbuvZx^HQx#mif?vK+Q?!_7rjPtm2fG|#zmEREX; z7-+T>b30vS9IhfZ4cd)-qs__6Bea>9-AQX`h8x8Du08p`s{8VIsQd2kp`=FALQ{k` z*+PpY%ODg{j4fHll4NV_V;O6NN+@fQ-DEd}!7w9AitNi^j3xVweVH-F@_f7Q>%Oo1 z_j>+$UeEK_^Z$I8Ip=)O=bZQZeLkNDi2m=;QnhR&QQw~R08){U1)eo~LaO0p9rSvS zEOl?oo_4214)Js4gzN^RaEZYq%>WRM$OeliR@Df5zS)i{wgV~^;bGa06Ekcad2yRHde#7^W>=8-%7)N_>otJ)q5XY)EGdG1?XgGCSX&7Yj$=IA(EN(Sf~m4qKr(d>yofNhs;Mc1 zD~jJb=Zik2x+}VN%Qi`>y6PQ$1OX^r{1C(6mU7GCkdj{5%X)QZYEmGC;T*svpbJvBfBM+HrIPabUdz^KP9!r8+sk3X%P@3my zDa+p-&27Dh^}HlYjG1XQIE78#!S?FW6Lq_ra=m1l5HmHw>IQeRKbFOL6%B0yUDM&d5i)e6gWY(Y8Bb7Apj20Sp zk_D8r9pG%R?d%AGbvDRg))HF1C;8UeoU!N{OdaliapctBaP3*n%$Z`SwF*X7#fD*% zIEUKb?qM~rxI`QkEN>?71(q~p>9FU}F>&6FojC?W)^n2uavI(n5T0h~Q&1&P*UqM9 z-)}`3(1+#DrVzw<@55Iod}!^hyQR9Fh6bR`c*qP1pGK@fcDUC33feS~KI+Wlmmc^I z;a5yDQ)HwG(MIy9;05YpRha(0Rv|L5Gn3}E$X;9k6RujT8QLaJY%stf4X?Mu#F1OkICOZ3G@63?{ zIrBH4)+~14c@RsBrEGBetg?#id-{#NBnEL9IOK5z{tTcEo*LNwo*LGF9sHX6{j>Ks zg=>DSU1S>A^ya7Bx$ygWnj-xD+O%T#L@l>!m3%!X<%rQfq`|8JlcF?4*_q2b;?H=Y zAJR`BW!hHb$L;%b?=8`?p#CE?5$QGRTD3U*`1C9r%uqds^ico`KWsF|EIyobUhHZJ z#hc7Iu$rwiNvVXwF*Rb|<}3+VX4h!89#Vox;4;`Mm@GNpw_I^M3?BIC2u)4oCxkxY z?3tn=MG;&CIrVV#FRVAfPnhJ5nxg8X4L^PB&(m)7yqD#TQ4S=}8fMq__GI~ub>PRd zbJcCGemNNicIR(M-E%nIJ!IFc#-^`@DZ9KS6s0j+{`n9LLo7tlRkpOc!588H;BpkF5Gj(oTE`q^w>UnX z=b0s7-`I4=eRJMnth~aXe-v_D!%Eqq{}X%x+ffZp++aHt+i@LC*r^AJ6}MB@)e+=< z3mdP1jYqFo^VtRcMqC=G=`{+Ag2AyiY_Q?jZN=|7)5G>BpNX5mk%`Wlh~dh=L53&N zA==*s50OWnqB*>EODg&{EqJS+xL(b~6(Bcs3ibr__RpDO9VTxR{aYJ4&TeKuPiXa9 zmWxXkDBL}76vx zIfiJLz8?C>sT`J2UkE5neGsiP=0a?~);XyMb?!9rzwT}R_4+*WkW>vB*||GVbFZRC zfuB9lT=7?LCv;izDCJ`CJ@u>MY%tuleBvTktyk7^tZ>Fs#NH{XkRCZAG$rRscerl2 zqry|0majl&@88Ld59{yv?%IHCY%>_6y^L6(uF}oP%iv4}EAMb79QuZLpYBRt9u4H- zMLXS1E~PDD_S)Buac!bo2D*GLwKm9u?n~E+_oEtomA2c52N*qVlazXpHZsfu%w z`QFf^9*9Ff%*U2w zBjo!|<%7(-$%5L9wPOu4IsB?6f^wr|pWBf!-fk_&%%K4_oAQfCO{zuRiVe}75v_En zwW7~=u6f|JB*jJNV25NSOQ!}zZkoG+H-=^|~qZdu#HZ11@Ih5M!lM93!UO$9!Q zY9ksKk^z6!m#O%U_A}IGIaSg)Ff|lh4bICC8oe=nu)C}px7e=p$+JQ&XZ)?dLI$Ot zqLSRj3eE$-2-qPnc9B#h@4pk?B(?i25FPtNNku} z-QJYqmHbf$%FkM8Rby&(=5hSmp24L%Yju}< zF60@44JK2URLT+iO43^U5ePjBdW9074^3aX1kfEBkS9ZH)1@6M!pZbS|Mpe=p0|2` zScMAFcZsKjapQK2a;tcEX_Gh<xDCOj$)f=r=q^?k_Hn*lx&AOQ#Y)l`sQme>XQ88)io_c~2TG$B$pj6xh7m z=g|%~G21@Za~v^$FeHd~O?NhXygoc=J3D-zp!2(GgC%_lY=_$xkyqvD$&|TVJDWcboB$4Z) za7tI|K<|&?ui%eu=Jvcr@lqTy7)TTfs93z*%#A{!0^F)-g>UIiI7RWXS(Jv*$oifN`Zoa*XGb8#uG-(@09GYA9g^k5O>`A-6IsNlK4IEO9mZYAWfkAKe`8j{(#7d?yMwPbL!8FS2Wk0RUoT08 zXonBib!aNk9bA59K4CE`gk}Oc*QI{-E-@xW7NPwwO#c?JQbzQpu91V7j~fLbDX%YP+qa+=RTt0(P1 zAy9U^82H4pmKdhUNg+Y#Ba>EwraQl*GiI*#$fcgzwCGy7l~N>nugmVE(aj z7a)=*hhN^8L-+HYzZ$YOXHLcWp4s>V-vn^Kl17a7v?nIaZEgtxVIvtX1;!WRw_ZaT z_Y-UEa4+*Yo{g0}I$@HbrNEM~A-MHH(fd`5<|9yI$2ph~8%&`6g2zm4@R5ft0BDp* zIS2c26gZ{A(LQ>QDn~<-ylHz!8e%pt#)Z}S)XfgS0)_4YwVO0^@}6vI&}#aXKOr(3 zBIXQA{CQyu$Y>3}T>1vXHH(0`4QNsJSm}xgx7lz`9^yT$@5(7r(GcxT_uOl1gx1=< zyL`!yec!!Ai68QJI_OvE2F5=b-HXf)e#(BKW!GgUAViH7qM|_U&)}sFYujX>#Q|svt_s& zMzy-si4oB2q?-xS>-RS)*#?{qc?L_YB8lCTSDM2gG_LhfU7LeUZt>Iy7osa!5=I7c zBGbCcvSd<~hVDw3KSg|c;OTkO&*QX|H>4+B@WwZ|QBivQq)YG&N6DvB(@Bc-(ViK+ zzb!Nu=%k=B4sZE*-2e`0i8Vvuro;Sahq7z4q~DF9A{Je8H9Tqnls3A#(H)*K-mD0J zV+blDPb()w)Bxzb$9pa`OYIE8 z{H>&grT29IK!u55>Mh65{?VPvR_%l{yLKZeAX<5y?u~U2!}lgG-NkKIbt-}PE2WG^ z3rJt!_cvXfORtE&3CJGtwzsR>a!s`8Fis6F?0^l;SE_QhUe z1Oe9my?$5suXh>qnwvwJwNr^>4>d-uax5|P={||BH68d=SyBv-IDAqKz+kbXB6vfTNq`{|Tw0WoLP)>4EKs3ptZ7Zc7w{#Kl=3#vlfW@ouLigNf+7bVqQ8MPTW=! zDS=q|^)Cglyt76QU%{A7qi&OX@O6q(^Bg??K}@W*t(>-Odp`{@Nv_o?gP_{`O-2vF6W+|iqMcxhNt6cRL3JcPZp#;c?NDVS|v zkRQXz$4Do?jd;7x4gGS74 zc@CK)DLfo4fd2DUhcK_MF^NYSIZRw?J4We7uP43kFm9BxNZ?kTKbPME>q2{C2=}ua zr0N-Y8tUzokJIfB`g80tgMPnqc8lz)oEj!$@@Ua6)cQFfm8GZz6Lx_^W-Ec?LOYV~ z4h#K@fIc73(zhlNX;BuRZ!vabxb5Ea*cn&UPJGo8&AlHo_`0NaAgD>!hp7o5tsjcG zYO?)4^r_Da!UqOvfegu-9t9OY8-m23Ti5BkuLVrLG9H*=J7-E2=aw+jyrmdY0zYlt zX?>B1k@y^R)4NsyKo4v5E0c}-m8pxx+udbF{euXsogRTq@{9JjY-dNq5N=adY*wzo zVsi*4X4U!LHaZ3C4Nc>ghHh{!!uKVF*7Ik`{*{8eN&~@PvGvj;oAJBiuJV%ps!0!D4r|B4e zZ`*1Y$uqspP|nrlA4s`9GJFoa?zpU6DiB5#haYXd4axuXlL?>d1GyrHaJsSGcrQ6 z=MG}?eV1HM)tJ)g)at9(`|g{=$fJFA4_Z_q26BH$AWMM*BAa_>8N<|pt%hJ~sK_R5 zHBuV*#DUsxo&c^tsein0`&R{-Hpf7b{i;(jtr(zu+Ds62X^`IAbS-IgZ}^Oz{pdPI zqwrV9uwI1T&HgP%opkxb6cvx^1=~pM1-yQ++$Zni2P~_S1nPUSg&hKbqw}8d<7~9q zF&bc^w(osz5xpu>c75Ns3$@F5LS+OC5C2eH{w#5GQsrwex-$d_P}bV+ICp4+d*=i0%uR=6NiSVmst23$Bjh z`T;8^Jyc=d84Uxt&RpXuXm+N~?Sz5uYr(&{jtc+-4n%*n=@Yy)NNBs*cq`L0RC#@a zjV)0BqE<#;EOfP!`No(@rxUto>pR4^NBU~-+vvC*6XACb7vN3NQ|~Eccg^m+-|bHY z*u&^{bZ(Z@E_eOhNbxjEAW)ax-vDr!f!n9Wu3E7Z;oZZo3VD>Od5#l}ShxWQ9k};n zQm;1sIqF=yN`chvEz$Gf^oIK1d9x4m^Jy5%PW(1^;Djcy*bZy#SsF5H_xG5mu!4Vh zV^!MEC!oYP5x;s-wx<`Xj_A;bxB|b<7HGyxHd^&x9k#vqT%TT`XdxxKscq1;(KE5~ zQ@jCFHF&6F*shL?doX5~vE5EjCwmcK3<*qmm2&QAUhXn)(Pj#G}^h)r>@ z4;~hIN4)zzR&Y6DtZorYdR-;@FM9&AAP=+}J4_6mwG_PJ$fxkwtK3c~@fl~P#@CX0 z(v~W&kNJCkHr)q8>Ytpf;}H)n0-36R#m@(3(!)j}Hu!k%9cPF8`y(n^@M~9rDb$E6 z*Rj#>(R`XAAtxCJM5Pb$Xc#r5^g+aES! z-V7lTP=^~)ey5XFT3(DdG$R1sNMec_!g5?9zoSb(cxt~aAs2R zBipuzCE};I6Y~C%45`upi{F^m?ELjdEd?QA{)o2rM*dPJIgw)EScf4orVEn6ne~ri z`iJE~*as;Ul=r8!mEX_&E15HcLHvZ=Yc37Rq9JhzMW4EK7;lW84cGrRhzpG6YWs=W zKD}Bk$L{(nak8kx?E*L2>(1Auv`3!l(BB*xMyFjm-2M?X!7SPQRrPOJT`3l+RA;O* z)BdB90-dWkoEh^>f6MEdR1NVT+3c6=Z$M(}=-tY>k9r){+{=uqf7Dl?maA;AgriB& zr%cZ<|Jqys*bDUkD)s)qlI<0au)%H)@R*@H{Alsjo{Q|RSES4|sTAce-IcbpE=;HG zb9~Z|i#dOsjjwdzFDYXFD^viz%2hFddI-%m3eB_b4m9A*67a3Ah7QyOmkpZ|f>P7A z9Sr#x?Q!GTZ%~Bh*(JYvkELFT#Thd}l7B`_1t*Dq&_s#{e+WvWL78Duty8}t!)A4j zhU8`F-OkT91zb-*x>P}kofQ#hy9Z88!dTe)`7s)soBw);+>>h4v$G^YU4?$LDo2RL z)W%k2wK*_8OamULK&|H>ulr7aQ^|OmY+?DcuwkdL#I#Pf26gu%JvmC<%I6{(M=w)J z^SE24@N_RU+#S;>K5Oarj?9dWeAY9C3HAB+{7HgZEjY6;RrHJ`+RSUrC?Od{!4Yx8 zmEq17XXTiR)N4;t;Je!Pb!FOosf3yZ$a+I8$Z3AKWWau*`ngQ&r`fq7+TFYp?A>fk9AbmOo&F3qjtU`+3keRp zhmO7>FR85DOjUFxxJFMIaAsCY;fYx{F5AC{uB_Nf8u6`94~~=n?Gs19B}R@Y6~8<5 z2Y38{UOpHQ!VJ>{ZatyP&DkYCMA-Y&1i_NJZFE4icce-o_6pVALKLz$q9?P*m9|}w0;H! z(fGHyT0a5peIrb?c*lJAH&U!XOE{#pO{(lWzQr7!tT;5Jhb_~VLRQjM`0Z;6Wj3R) zqiGH~K`v+T9O(#zu~~qNv+VL43BAjJuy>l8P3oGAGA+l&rC(~u{6g^&oJ8q4`bB%o z4+QUw)PN{MPcv$N%WW00xugZM=P*mP*Ka?p>bGY4P93G2b-u2jX+`=E?lf| z&vUOA{C)0Uq(j$V&3N~Vrxly--*XZ4l`I(;U2NcD>UuReXUtW(lB=Y!x>p=6^Ln(W z34qKRY%m&#Dd!Vdp6czyM(scRNs$W}^YDxsz2rIDQ4y@Pk)P&0Q2R#-G-%lQu)Nlg!y3mX+3o?qtxWP-;W&}KYiR2UbU^;4wG*ON-fvLeUV zxL)PW#*H5j@Fb);b1ARYq3b>Yrr&oaG{UvXOd=mJC(APJ_~Tw11r1VmmZ~gQ9dj&l())Euf8|OJ*y?TKub|s^L~BQVqN^9hvUvnJd;*I zb-^|Bb1l}{XJDQJ}JdOO}$)f@E-c7)M>q3nR7MYXu{6K%Y$&{rLWxdt8Ybe)xL~pDV=58!h8T2uoS6;^k5xJuY{Yd&q%pJF}%{xra}Sk z1fX{v`ToE_g9tVy@S8qy=jPXmE>_|%DgAjEbAs!BkmCAC!(gf_ep2lsq8Df{Ynclp zwQVIi2<7D_orT`fK#Q(iM8jNQK7Oy@V z#FBJ_i$8FteHM8LF#>q?Z|ny~ytTzT-M>=dCEF8US`90DUsfIvfkv=iT*m$~B{Y`g z9M=VCA9^3N&+^$#X>aq&-HbOW^FbzGDubv?HPe)KFVB)#+v??LrHJ6e%KM%RgPvbH ze6wp)rP}mj-A8YhJcjmitFpzWs>y85D4aAn@~$b@W6bM-2%9v8Je7BT4yYAJmY3yR zezWGo>t4q+1EQ`4fCY33i~J_2qQZ6=No}BFHSa(RxgRS5@Q|dNewh$JdnJsr*4IY^cUD zp87o3?yKe-FY3}Lo&?-Nl~EwvYfK!S3$m(lcvlzOBp55;N~y#k_V?x@6D`%JnF22* zlD+vAY8Q<{0IXvh|Df)eJ2fAY+9b1RFBRf6ghPxdfa6DJ_;CrG*LU-Z*4MUz7QHiUFjTsA?aX%SB>`P^%FV)G`ZcBo>9PqD#>lSadP z6NZHVs*f?er=dO~`eVyu=JTVp2(qDDPWo-|BTJW5Tave?IzJ~Q@BE$hvy3Q+S@=Bw zxf-sDh$`xrALhsbd?E4)@enpC_oTJS09f?PbQZ1tK>BYd@E;W3bH2d2I-z;OmCHu`k^lh)e_cu728CKR;_0-jxI`dwGK4AmXxz9v zaj)lX^xw{Sdv97Hq-VvyLXLa$iq>gmfHo4X?5x0HSE|M2ELk&W* zvltpcL;18|)~M{+h$Z)-gF+mc1k1|rh%oFOZPYO~c{7#41KR6?j05?zDV$=bRX6$R zh;k5xR;~QXT=VM;BMe;z(q5Ml&2)x3XfF_gzys3v(;-j%`Yws}14<9}zB0!si`_M3 zcgXZ3j!X@fMsdu*(T9pDo?1|=lk#rkA$$h9p2r*=l=Znp0K(>)$fNq&gp)^74~Wf` z3#Vf>1%ky&bv=0#W&Ti06{(P)V`28dUQ}psW~irXfzR6}1vaR_CZz|LVk=scVbsSI z)%tp_Q1e|=JsU@k`V)$b!<<@!zSSG|*pk}hCX_1Y&$FjFa6qZ61eVC_d_O^3wUQ+K zYCnsfU~ce+Vbt%eoBv&Ks$NN?f?5UD61o3akn`iM&TyVnT%C?n`XOtvhwzIO6NS1m zjqkrKo;p{5-&}^Pg5Un^;8(Eb{EjYm<3t z=cuo-R?u2hOgg{$CpT&wWOxYu95B3z-%e+_r>oboeCwB+E>o*g0i_S7&fKMUm!DA; z4Nw4<)^9VgwS4xt&UyN<+C8 zM8fC-8QTNW#Q6L8^e2C0{J~5|!NTN(2sLhe1Zk=GTy+deW4@f{_|jl)FQOu1W!=yL z9a#EBA`#>?uplpM`!s+{LrSA}$6`|WK2ZB=si;WwdjB2KfGe1wR*tMtoDLNmv(lD zrffS~Gcm6)9e^O>hlM0Vwx9IfSESD;mHH%yrbyXKRHr%x83F!X36`qd8G&|${=lW2 z(q>79gaBVsL;w#z4bvR*b4ey2l9W^w8D~^V>J2w4l3R3Z<9UeR?S{BC6lo@7sRdl@ z!-%T=V1GJ4A2BiA{ft)V!XeZ1T5mzOKObhjeEaP|H>~{VNvQ#?)2YaXD&n2bPll4F zT};dE%;Wm~D^SV#lKOc1Xr0F9sJ|4qs6oP0t)LHdn7yz4j}s)c#&84s8DHPofo02G zPpjbB`)(}*wZeiqyEQyCqsc7n#mO2Db{-K5?og*_(DnTXM>8cY>!+HM_jme(hiZ<@ zV;@gD$-VhFBe5Nup_0~BYlLYucM&*`>=QXZgp;5tDKYBN#s?1716}}?587_ z9LM_E8iclCtBQ5(wI5C_GC`Dx@zp|1``S(X`266&YJZJHyyhB49>uE1!~4%n7|9iqm+NQ}1Zd zN_^h4`8A%UTNGA9iJh*%=_8qyOR*W-;Z$EkGO|Q4aIL`q>5f}?Cw?W4IXBehE^U+< zqlo%d^J4e!9a~h5`}08*#YOE1?de?y(prxj3M{gh*L;S!n$tBq$+G@z>}T5%D*0C{ zGEH4CSfMHXWeNR$uQXNKCUJU-hYOEBKfQ>t#~s(mdWe@aw8-4J2F~Q}+lpmFM0yZI;(h>XNG&Og@Plz zxVS}1#{7r*)ia`*oe8(@UiL1*e=6}&zp#yqepDkdT6HB>0XXN$EBcP0SF!O6Y&Uiq zeO!CuP1DhKosEjT4EM(H8FjEhvQe*8YMu3WOrFiQhsW!5HeisLLSqnxgB^eLH@=C+PEe|!n6U_L9LCme3kh~SQb`Yjuf_zmFuR}I2 z>}3kd#IW0B+p8mc-Pyjs6*~1Rdu2|NfHxh0)!Y@YYvK+UH}&x-$t=6Zus@iuTo@lJqV}Kt0-fYo^PO^I z1j5BcRQh$0|Jhcj3c_L}V<5+srpJ?mDscHb96*@>qFi{FH;9Wz^KWb{WMxs_x9{%# zkBRj)tza;?*mQR9XOsk>o>cT&NMI5?zRH^89W{G+C(P>PzoL!`I71`eR2;bpF_Y#+ zwpDm~AO>HYrEPYt4!BNTt_!-^D?!{ps$x5#O;vVy!(9CG?v#t)_fho$-0X4cSyJr2 zhQI<0dmxxBH8c3q)OXZWRyhjnrW2D2Uq$nBrSB4SGKX=ZtL3z7eO08qbil4?T?)N zDqjP+25#L)JnnQK?6AT)kq^a``?U>R**9VZOmk7I;2h;|nB~Q?K0B;MH?7Jv>Y1lf@3-8}K} z&xczYpw0Iua*dAt+u2Pf!%;|AU@?<*_+>;Cm3E)GSm(1(Fek{{VyL{lH(YScCw61* zZF*^S4){pD%Ik)+L!Yb6wGYB5K?aepn+~!aY|K=LpEdhroljpmQab4{lbyK4wh#Q- z5e51s{nQ&i1<_g=c3~~L6=-LlsNnRV{ial+~x+At=at@ghzD zu6ZLZm8&4Mvec5$Y593L`~r@~N%AIDGTPu0r4|kIp(o){9i~V_2^~dmQ|&qv9}ka` zJ4BWx|HAG3LuQ}W7j8bzt~*_enoO#zrkv)zhSIJlwT(ggrF^-#1XU21gxlSTBUzn? zeO!+TD1LAgGZQudoZi9 zv+FG3tE^wgNup@3eOoUsZ9Cd)j7N;Jy>Ycj5`0>`uy{BM_rHyCh7ijWYSoV&oAW)?raIg~MzVQ|Rg~Q3~MLkGCe1=H>eWH>-%B<#mWPymMz6 z*|t{I_1T@k9nc}L=j%8L#Uq&la-j*WS$}_vs$1Ir<&wf3AFBzD^GnU2^PPXPUjv_% zev#(zK~Ug<)Ytxr%M|T<$1y1k#R!iB0!w}N->v1i4zreg6C&83$No`QttD^9o6dG2h3&ybpF?*fHeB#6s} z7zY()uO1I%&MSov9!oQ|%joVh7v612+p!Du>hvmWx12!A8BM+P2l>&ZdM!r$+*4cU z4LW=04OqW_PB8ZGH*JhxzTOi^#A_j!+rXKZA5CVCuN6NkG3E=07ba3fw|(ZEG3x`X zN`k7!8-Aix%M+^opo5}`>i#d`S!d{Pu*pb)MljPUF82l~l9Wjgq4*6R0`UyoXAzrsQdQ(zZYvX)35A z!FxSCbUJNNIp9ZsDSc(DydR$nyDK%SmySQq<;?w5#0WN*1!{E_2Tw@*#MU^&>s!Np z6|By)0r{C!cdGgwiK9dc;dk3e&`rke572 zQH!%?zvo8CSnJUV1T@>~7xb&OsOjpzgT=4*w)r19q?Mh#1(8fXDk1$bBf_FLDi~i+ zuiTw3B9iIjRj+A9h&~}6rMrGYV?{zmazlN7ZNGz6ij{f2VQ_?}ZIyA2n+k8$4E)gL z7aY*X$N|6R1Nl2UFPPcKb1t&ak@jBGV3mUx+6+qUr?GL~Lz!uUkp4>?^_D8C-MIO4J6tK+mnhgLz z<)#5B-2wqh=%3{Q+`qn_ckMmy(>|v|5D5M_0 zO3Hq7WZR0IJ05eqnZ0V@Q6vKmNqfu+!h z@xbmCpP)x_$i*#?Hu*X@_!|aet*5@?_~=&m&9}0SSZ2Cdh_*~|>db>Rn~08kn*$YS zX0i3W%P5q{$g!E}`yfQ|Sm)QJc=s?0zdHX+dEf%q*X*2P?{vNc#M^G&+zo55`kZII zG0-IiL$d#_xdCukqo?ug6iDpR2YQJBiC+ZCiUc+AR9Ovc3a(VKD=h;iNW#AtoA-+*0|%;l)OvK z<#u2$yK!U90xpump~<2eReB&U=BeaIefC7w2tbez4&oLNxk^!S@(8fx!(S!(2^&_w ziFnXU+ncPz1%PtCIH*1gP8O`MA<9tBP^R?fvXWPW0s)LHMnu9Sut$|!DYxDRv_(`SIqG1MKwM*S}0jmfi?-=e0CDX7aEnS#RXYT_y+jHJH zy>(x+PqDt%dlx4_)qF2nC<;Ck0FR(43%4!nKmZ zoanrif+*25s%*~Za_tQ|^Gfe9H#_r}G9uP`X}B>$o1`(wxO-NpAK$aO>8i6|DkGRe zSRM(!3|i z$Bq#m5xp0Ol6MRpno9q6I0)ohDc}!6jyq+QDCzGof@1S`pFWee8454{te)eF0#!X> zX#3OX7}JV}i-`F(V=UA*8QA(|*sOAKiZ$zl=jqF?1`_{iH9)GSz}*rTGPX5JCVuTq zHQQ}pVTa9jSh?}*;kP-lO-Eep|S z88kWaC7ST!C0b`-j;x#K zup0ht4`==@x_4Wh#-|^?y&;WwgI{qvQ0FIplOZg8^3|iD^G_6z+r=EOycb3g zg3^p8Wk!9td61%bTlgbh+_F#kE`Gf`Pu6E5Zp(Yk03RW1OH-n%Jb8g%LtXMOUrHS) z@ON!%?eK130_kLLb3w7o616B%L?2+vKARdmIq}FY+*t14UBHTO-)n#;Ue#wN2kDy! zK&uM9u{6-N3?{irD8E(_Ub*qQSOTln|Od^sH)GI z%pyC6_hoFSkS53CGV1Q)=KKVbBb+B!+GV(j6K`TUD{0|%q1(RQoYdM+wKy+?P()VK zdF84ALzBXzlAkqGs%Rr^n~3#+#Z?4V+P8&5V3bN&Vtd%kg>}=hhoY6eJoR32WqRqZ zLnNwh+IJ3BntGByZ>D0u(G7WQ`u|S2Adr$k)x{8;N6Ug!&zy(eu~)+_>#vE10IT)I z!J4xorU<;CxOg#%PxP$nuGgc!pbtqkd$~|HS|sA)bc$!8QWMxfy517A7j`PCfHdb~ zY1t&qD}fUbJQ*#JoL=?-DDT_q(-O;ye?lY>D=Ye|4OGHvY2eW3z*X?|Ui)^>+s2lo z_}_uC83|>&U+F5M<;98bF{s35PO4!waqivOEU@@GsoIpF7BOL$BdK(+8i$W zPYXD{n!n0oWZAbMXxkPfbHgbT!(+B_TXw_CtWpEiA7uA9vy!5rpIJF=@fd`hCDbm1 zv!wHkR&u7lV*86hg^ug81NF$37k2dJvFZbn$68cUh@$c#!&UZHtK08h;&*SKavs2i zabR{n>RSc$mui5f>642?K3^7>SGsM{OXhv|`h(pbYdtjmA5N43_J8&!+|mH|pnkIS z*>jvaAfnP4-;`$!GCod9jF@r8ULFfK`u2-+{WMDg`fyII;81TAE%BVHV|t#7)2Z@wDtKWqT8$J~(8XppwtNDqz7c-m*Ll0i znG(r^+glY+vDT|MNadc@C8iDCz{IzsRG;5h-r#|eQh39}fP}>5*49=mgi3Hc5Rp3+ zom&0O|GOl!{*F1s$&vKy7a(I`gK_c(&f!Wc1^M%I!+aK~gfN38^M6ugh{@yukq_k! zU-q(m`t~Z-zonlwmWK3wwhgC2(k{`a-e#BPsZhDgmHEKMKSKi`cSS2vNDqSD4A0^v z`H(nKe~tzAuxuLH`$Tnn)I3A-4OnOJc56YeeMx6X7H!5qdXZsbS&efgI7^U_Sah37 z^Q>2#(S`2q2j(M6FmSPSUcyXh>tI!jr62ws$J=k6ENGi8ZY9x5$~Mxf?dMPThP+|9 z|F*mZX!C&qUl=zq;{2&o+j2*%|2yvAVx5r32VwpBvWGHRIBs+`?A9|i4l$PutcQlNhTYaC~Ie-9b~TF5`=(Enh=>iye*o$deTWU+x+3xv2Z zrL^&x6Jzn)rE&LkGKoi-P<$WI0Or&lNwS+I_~u(xw1 zyL~z|I3;MRRsMgjqykaDpkNQ9r~wX+4!)9CdTe*M?ls?dPLTj97=QMJF2P%OJ{qOB z(;zTpS1-KGb@wrtjaQ+$);RUL^1z~)-gD*wgo0IhgwHe?=K-XO-z+U{ibQ!xl+|JM ztDZ_XHsqg?Bl!1I0HRZOQ_q;oB+jP!F4Tu?iyZ*$MXX%$xjh7IY_E(F9h`sLN^?TE z`&=ED{Z5|Gv|xK>AE&uhCV+-^=tKp>IFR3HeVO=KlwWC9yy;k1gnH2ts;)EMDZ# zG?%cUXVT%739{6p-58IAkxRYZ`B6oix#?Z+ z-5QjoRaz38k`fzRb@p95pgLxx638BH0zi1yvMMU<_rx9 z2tOLA?q@fi*~6?ma#@d$orMlYth*&R$fd78UGEyUr!F`91V|X7I|tQgH-p*A9xGmz zFK1Q@&e~*^xW3XsmYP*}s{48!J=N+{%gnzsSsWj0bNOU2-7_HB9iy5w{qbOgNyFyX zw~?cHXs1y|NNfgP?H|}-Q>??rGSRgiP)5~ zr>V<0{R;`Pu6vcw&694=f^cl6wUH+Sc3?{0NGZhY}VV75C|zUrGz z-eo7Taj%8t51Y0zRrynbuE+t9Cp~C>^`E-d(K%>XkIMa8frPWObCDYC*cr)1o`Aq#CMt$(@v&FWT zExI4;T8+o977bj?h72pdDc3W#ZmsvfodWj=G5t)HI`0~)7)-X-e*DnCMdEC$+M|E@ zdAZPe6iK4Rd!Y_YMg*YM5#@>ciB`{GhQ z%U8LH)5Vj8=6mi8&yxZoJ$wDGH&5%?{d{&>KS@1e4*s1dC6CQGR#4%9K)cFf?xkno z$KE%7g9CiYch7=5re}h0X&b%DjBckoxk!R%FLQg?Es{Mm;cOL{b{mkTZ4R(5<1>ef_Y`qTW$82(@KQY~WS#9G3K1u@m`7AuS zP(NCE{A?2vD;;PQ{d@r>*Y#;@O_mx*{XTE%`xr81&zPc)XU>ZNo5J^Qt^AY>j?}p#*py|}VbS3CY^*ZiVxs*dy z)sqLM7xiT)3_s|-FNbLj-~CG5>j_?y;5Vk@AimG1gNe(r90vgLw;eZ~d|x&^+h67| zrL6zlqpZ7hd0{iMa4If5v+3aAFl2daAfH|Vb*r6r`SE_~{7V6)AiSwm&FDSZtssBP zJ~)~JB>HY_H;6UXw0=j7ltE&uw#|6x=<7U<<@nE~2FL1^ENg8LZ-5&SA(HSgLj;Tk zxR_}^8nV%pE!C|iCHg^a31^EH_;MZ%I$W>j}P8K#F}yLEy z`#gVbzn#-N{D8tiUf!IpXAPU8h78*}FD3{ZTZgP2pB$i=&OZXT8*!aqX&^HZ_)ZG@ zd!^B*A1Z>(1@4ys=aWtvqH#QDQIX^`p^`{rBmGWZ_rwj0MVI=v2u**vR5CmzI6a|0Q z-GI4QUFEhe7ZmNLP!4bCXywr{rneGMbY+E7*+WJ>si8HqnbdtB1%1 ze>2MP5VtJgo{|`~lIr8)tnmm{b;v?)Wq*?uj~;j;;J~D@)|Isf z6UUjZS_$9pZmk;k-~uh@vWz>@R-a7idkA#;0)OfkJfthj1V1u({5Wymuz6h-$#NXA zS4YHz7?NHM&iD|634&~FWHetgh-a6$T#=)`SDhLgw+FXM$rnjhNz4BeghAP`t>kXU zkd`|@#Tl`x$$b(>Be!EfY(v>EYr}vHMPnT!v;1*#=MY5rGDrE&(P*O7+-NXapBw-* zMkY}*1YNIk{WaLLxp13xq$`oAo31Yo4nIO~a6>-t+7KupuPY zKR<$COr-7x{j8ml&y|kEWqas-9Saa$LxkrFcRS5l?(a|RgeCxGn{@p;fa|_Ad|OWD zM2WS}26blr)q+S^ZKat-?^bJ_oabNaB8?`i_OqX6@VYGc=#58vhJ3ro5IWuFDw>t7$4t<7ce}xA{)yl>V6R&R4gsZUAjqQDvb=tB8w9R2%>`(~+W3lCYejuq>CG zk}?R^4MICjr{m*eGndVWY9|yRJd|{vJxp4ehz*yBGzrVQhE_!*~@O_+Heiy|J=`aO@#j|GC66o#xoJx>TwJ8&_2)J;W zos|9L!iL+-cr;gNHJ_!oqJ|tb$a3^u#g5HyqJYoy)lHgKaA31|BEC|UYy_SKbcLKsS^LaNGUn@=oxqm&mTbK!o}3gc6sX%mX6$fdh4 ztIItEmp1pg0*x85I#KQk43d4*{TLh!{U!~*u?!mOlQ#5g&;kMm-3>N#B0*19x==dO z;Tk)FLSUyK%PFBQ_U6)lphF+`mQ&hDM3dezbldN zM3sBHs~UU{lE8v81aD~j1hQs+@we`pK*l9FD~CTsz3wZmc6=Q`R|<5TDoT-(KUk>! zbxHD1jvb56v1_U?cDDkm4Wq>hnB8u*kF-OM3C;Vc+Lgwr2fo8cH@pP1?vub75N3f- zZW}aKi2)H(nWWBl#m*ib)^q$iZNeyv6iKv~J2Og3D_73)^m@pzMvZg~9k=fpu(`>; zd(l7DYM;(G`~iNZlQLQpwV5A`gXO$_S{xEltMUk9^*fif4Fvmeb%P29Ij)qnvX?Fw z0RSIG3H6iHl#ph;JZW`98AFzMq!UNTC23fs+Dh$E8F{)h3H`(N?d6_ioNbxC+5PbL*1fO81R&Hw(tb*?uT+ELJKMeWZ%^WD6I%z*V zRlgt7`ilt;3phJ0)F@+AyNa)26)iifKhE5}FGS|HKp!+|20_woK7@?El`Pbqg#}FM zNu2J-n@C&JNMdM+v@srcfRgqqTY?;L6t{{Z1Xc$++Df&v+|WMAZarK50Ktd=BWWjU z1)?>j&r@smzRVZgFXTZx`{tIDq*)8_6sKL19T^L) zK@RL%;BdY6cef{nRgRMn-~DSq`sfYv6Mv%Q3qR}Es?w)s;sN(bQpg%$W0w3(F}|5q z?@ykBC>VKdmfO$6xWcv%Kfj9i>rM5ng@zQ2l*V9$TQ!=jEinnP(odETe9@n8V#>Fh zUr}w6a(!3r+IyJ;KqWDbaisyAtG-H`Hqfl%JE663n%&G!kUTZ}R{%)loFR5?5WxF)P9+o_Y&AoTwaWY z@V#iec*v+sN%2jrSpQ=>MA4wsX#MY1fVh;n>MRRn2Mcs}>PF>$f7ywVVYc<}OzXEV3GY1|7=A4%`IbZKkDo2!{v+A-<&bIpCDp)6Q#hV3z^0Brq|Vkc zJE+c;P<{XcO#5)B%VFw`5wpFT!XP}2RmjtcY_uk?`r-yTwPL(Z5r&fRVwWoiFq zMj={Az3iJRf^@7rBPHqp>TcaF>ve$ExmHV&8nE!z3j71O2w1o5 z`$Co(_;^5Iv}H! zf~5t$q7-zAF8?FSgY=CWx5Jds&$`{N@cF!{Da|p`?(*Puf6`)eOcXv-@bES~zy?iD zE{0Z}=CaK97ehJ@=#b0kc=P^D>D}k(gslCne^wF?J+g4l81rX1PIJN(-0?ua##0<+ ziR9wChdT(QaT+Iopoj*Oql8=?Gf%sX52$BgeRJFP+wO85(^s3_&+LvgjN2yV2%AWH z1u+)BaNI?!#i}AT7;0Q(#O_zRLTy5p@X*wb;MdpNmgzCKw~@9E4vQj$Z~D|tp=g>_ z{&n7Sd+=@R6_4cgy&_idNTW`iWQCoMUaj;encJHT$8oV1Tf5A2iF%@@!uewv#KU)zLl>&s;J8;+B|); zX~{z$7pH3bJv-spp6ZzHY$lkRGx^yO*KJZz7 zlq7kY{f8wG5vklk&i3aYF$b9#9=2=;7rU>@AKP{JdGJhzc=cxhF@2VT+L^iqLOR*} zZbp2z$^)3b=zT=zfjqRd{bqagXl|vR74k?vT5(5AhQVm^gBy>P=A_C>tCG}3OWPj? z(R5V5612)&TOt9&LCZ8b3*QT&*~fjsUy`kldj!XZERR~0dWjOP?a)f_ST;xVB>Sl8 zT((&M%0_vqX{45E^Qr&7s3V$tEnTgF{97$WE+M1aTMd_6+uN!~Z8xVD%jo3Ozl9l( z%saNGyV%ixWsW~zSClMY8!gP!U_;f?HU_0F*Cy(%%)sVo^*4jXx`k$YV>v_V>8tfK za9?48;4ERor6FAO24OeiZUy?WlxTSzE9{PN9UIXQpryO$vYWlgzH){Q*>;G_F#w*6 z-|zb6lwcWlBX>l!=GCyC!<9+cQ4Kf*hSTx-1ppm<@jXbgIT$<%l20bZ z9`mv)>$ZhcgGLzSa9{pajT3 zDlb>6`$MIQ_0EbI)%G1{^B}jjhG2r!mTS3IAI*ATL02Igfr8d=wq%zGYS4FlCHV3s zX{!5vW~oKB0aL)S@s4zdSYm(DJ1zW1*9iV;vDH?Gq#0F~<=ur^=Na=q;@Q?NXL>U! zmZ@T>IW-kdD+dzdRX-=k{An^$Z@wsl=x~hIC39ff4&FU+TqYY09>p!3tBV6<(>pz7 zu{}FntN^@5dx&Dr&br(d7|!-@$7Q^7JJBl&a>NXSQd0ig zh-FbY_ALIF=wHE`Wj^|WvTo(Z%@&iN(|Oajq*&&4b>TH zyr;+6;VzQ>xsB6jnw}?Kq_AOOKkWg8?pi=;>>iE>GNk4;%Sc5$jr`*GhTL9)Gt5e> zbXkuDt^;gYyzAH$@+Qrf_d)lY(=kUwj*oLL>0z5&V&~y53SwFq zt~;%we#dFo7iZ13>ul#&4!ENpXk2^}XoaZh$h#?Bq1g?T-@-uPAz1e41E#Fk7=GF_ z)%-4hCEnR*jXy^gRO)DA01mg=kf<=@N^n~jsWWb|ea(l@{SPlG$Ns}B6V$Ah|45O5 zpBsvjN(zutmad=IsujCO*j!vqq>-VRSBt3)`sEhz+kT?iAGa~g%`*|&HupNdjkgIIX}$j}hY>?ex12Afj#1atG%+w5HP#-6qQ2ja^y zLX_bFkx;FxL~}EjKZylGhCO{{hqL={u_(iu=Q1zmaY^YP%Vji+5<=zqEjko5@h^;+}y8 z(CA&Ud}P`K1vJ)!x`+E=J8HQ(86K#?YgvVeH6+;Z-0IpH8x^Yf#Tf%pRGr0;n}}&- z=+7uLY>~#}Qsmsivtz`@BHSh^T>hLtf-gr2@?OVI36rv+c`4U~CFdGM%oktkXzo7lAvkl!h2aQh>f?kF5Lx0> z@vd0Nn~^YyFSV)2u8T5Et7fPaMnRkmZuseUvphs|q4AHa%{#*ssc?j_4Fs|JXedA< zCczbsh5}PLEQ&*96OnGi#+9CC+mTaj*#-J&T+*dt48s`}?jgmQd|k?8xE{a>WB0$t znT5015pFzmvfc3{Kw8Oo^@wXgtXR2mSX3@afv64QgQD|KkW4pHsBtw6XJFJ^x!nJC zS_J;`W$Egt@#%g9;A*P7Ap?!oym)Ppf4ED(!kv%P^h}LFD3o|%|D16%szwSOjZlxZ ziqQbGCRO8Ao81Y=pILWX(=J>~Xeb*M*`En_!FD32UP0YCmblV+YWuldBjnqdzp_5pZM-H1*FUliZmL021nNbGDiqZVw@Jlj8 zZP7)R`JzeKfE@QLfDoR73i@enY#`Npx;2cwcey3801Stp%?2(9BScPAKOjr=etv#$ zyd&vwQ7TdM6e%lGMUU6errt3Zim|VT0@o zfe>^u4V^n8EDI5BMT4J`?vZE%ocbE=z56UrjTny0fF}?&97<3!j;!Kj# zkE63muuO+58ie=%{`0Pj?jFi_o$g>3Q(ltf4`HG$(u4JILqAipg~b7e4{`e8H>Y$2 z*x5Tt{-a;I9A4U`d=k`8d0yB0OHl`upK(}dXQq50E*nJaQ=3g;fb9eU_tdXa^LmRA(7_PV z%(}!M!~ZOUhW?P`xy2S^mJTk+GQr9Oc~-MFEJ%Zul>}= z&t9ff!LF>S<0vv3%8UHoYnbR$fjBg>^^~p$9W0ZXIr<0UUVVHrO#mlLlDJJXRVWzc zc;~!~u~O4*3;_$-mtWTD)0@ArpX|z*)UFP#-*EfAP62Wmq(Ltem0kfqgAG-c&;F9W zxcAP7#2-t3#5JE5vw)b;jLSCo)3Vl3qbf8ZdTY&$ev>>C{tFGLg>Fb@ljcjc+?Yy$ zm__{p+s|ZFfPIXo^c22D%$-QnOd?vV=k5|q@iic&DVA#%h#D2aZ0G>h#6AnR7BrEz zP5=*&D%qxKqv7kL6Hx@2HIJIVAqxyAh&k|tFd@=<$-OMIl6VZM?AZO>L_mUGlx$+q zZRzJd6^;}ZblhZ9R_Gl_`2C$#<}-D+{;*sW6axpNZMpJknU_>>a!!CEh={YHPekO0 z3$Yq`e9}F-HNAp41~M=XnUo2E`#L!~Gd}2BL$+^Gkk5(q#N<>ZhH0knE(U~tsTY92< zZyn-CPdP8#mDP;}sul7Ja;0pN6`?1Hz*)jZ#KQ-bPJsf$Yj)CjL}wwHjT)0*hZ_=% zv%kmLhtcywzV++xE0L2@u)9sI&*dj#Do(n=w56?k168p!ns#FJ%7-#?#;{~s@$)e< zaftb$$;0@;w;EgQx#g`9wm+$bgTfd^L z$mp?r1|T?PnNQwR5Ta>V9=3KECq53|uFN&gMM3{|;|0CZ2BwVMFe<2YC zCEza?Lf21){r~Y~*KqAMmRQhP3&V)S1Es_q)P(?&ypM2G@t={Fa8QA)aT0~H_m@-+ zQ4Oe`wl_u{zpb)a)HtF4)!@bDy4A~$PG9S#zw;9TV@I-ZcOjr|RxJXq=qW=-^GjJU zB_!EHAv|UiT`QHs>bNRPD`=P4+PH)+4N(2aB)>nB>ceY!#w?!SZdk6FrcYl#?;n{V zlvSs?S`<6s)06br?+nJp!!vaj;kfhI>V)dTddiko*g86UWOV#&v#`pxC#l+YYRicB zvFn@tg#fKZ_m!70njK;6ezlaKmDi;=h3$_vq%7Jz^z#cxUfzN@l;5r2vsLcGn{7&1OhwffkREuZKScQ9 z2C{E9tY{*KzH?Q>nb$WU2{-En{kB^uqZ=2-O}B~4Re}KB#z6@`h(CE23-QM6oFOhw zFBF)k%Sgu{B1aHQB4UDM5Km?GOlP0VdK8J=ESz#wG-qR&i4~+N0hyv<8RJz_+N48u z?dk&WQ@v*x1OQOF2fszCaBH-(vU03=w_9N6`yoTr_LrX`Ei2=TtqbWjW2}j=rG?L` zEakIDYFr$RuIRGM<_=bt0FqdPJm1?c#s0Y@Peiw~zh-~m@T+BfGH2-}=M(Va zm}%YR<;v%=1iFBaK*Dq2rdT;am8ZvPHmd@<(hSq9L_Xr5#p}#^X@{_EHjdWJ%+n6% zs?HG2J_>YMt&udwQ_|G$XT&RHWDxU~1L5DZ=Slu8lqK0ikWcLEN!m|2nm@hMyJ%9Q zp3^*BnKW1>>DNANQKgg9Y963lqZurlr}mDS!v^*ZI@R--DZO5O6sC4RnMh& zVDJ(h+-*JWwI^PHuFiBZE_!uUH_BTt>0i7LAfYX6cUSM*!KDzsBKxo-)D6gLe*WqZ z`(}pYF!eQY#-5SPj}F|P+O=%3D{x3IhK1<RwQV(te}U#Niv%K zwBr1}c4XdW;hrdlr=>vfOn~p&5AE2u;+hqWdmUi^7Zj{jD@w>gf&R8nnhZz7@Xna4 z_{#o0aavU$6N`AHb6+ipD5jmp@;rDv4qkqR5v`#4(uPB$xd#!1EJHC% z7i-gAA6Vbqc5D31PJL7E;!EakIWkw^VzK1Jh%EtHh0~)BI;HsA4yk+CETl@8ThNt; zk7WwQpNOLEkdn4V%$P+;QWq~FL7M}chCUPAUz^v#Y}3r3>dD5eGz1E;8q+aaG(Pn5`7=thFS#-U=p z_w$d5zhw)%V?>UY@snP4+7zl3-hza3h-EU>;n^<%kC*Wh^tICuMWsa--1adR>Mdvl z-8O!B({out55W6NFE`wo!qyHf)-5tccG~*-#lMM$D1`su&z_Q;#U&^&o+K||nAO($ z0K^1)rZ=rDWZhC^7h5W{!?emS8&>~ zH`B5GQtrqIM_N~#h`nwwn9OZAfwT9Q*A)0+r_RirO*X5X+CJtv<9XL?E59ldV-ILr zS+_0TM`we*9#Prli*?QTrn28}(-#i}2@kUP++;`vdW9oCo!99OicB6@8yUoMOS{i! zxo4uW5(4ehd{@!_Q`kRyQ$#0gt!C4OEBo~k<0$h=DwgPn&NzS|CJ6(bs0_v^!!91i z=)D*1>rXnrBY0K z2teVaBw8`889#>E{``C@&WD)Et7E15Oc#)SPob1H;M%;0&tm7-N`?|rMiitI`Q(X$ zA`6wk*})}=4^eY>T6?Nk>jc5(JqpFDxs31#+%9^;5J#Aspr5W%q zyq>7oPZ*+S&cZJ;&_TfNaAM*~nakjn7fEuet-_D zI)t)6p3C8pH|cYS&m>^SnuUn!Lfv=n3285~NKjDAtcb~!#V9+6zdTY$cCbFtF#Cko z2oDB&@hPMSBKor&Egek|qa7)q#kVhbKUV5)`x5{#VrEcpWZhWxiMyQo-(s71VzFqf zecw0;K&N1%g~jri4Yy^O3SRqjN7O_=Sy|N7!zzBt>Jh_*@Q{T0q;CERf^lE@$WjOk zC;csx+3a*eeh9B304+i>6c`_*b)u3#L`|`V{_GXSMneq4UGiNXq)ohMctL^#@j0zc zQqg@jEU2Pf4<5^y}1y3G5UG2nbPF8H{nVgujJ9{ge zF9szveMbdGy}>X>n@0^}6#f`&#M+by6^W1bXS~-pY?Rn&I%V|^?I%CWfOV8}|0krn zRSDQo<>U9QnPG`$LeE1zBk9O^_28m#UwUFdc(EZlifKAK5S7@G5-ymu8=p8uC&>7SWSpr*@or{4*?2L*Gu!XD=C)60 z6X-UKku6Dex^Zds;SgQd748y}T?^H1`9jloR!9?w>D-mJnEbc$H`ft|zzaMW!$ERr8Qx8hRIe$18+{?+62)X`X75hGVZ5+pv#_4TZ#6~ZN z$GeNQb`BG+S0$eShiH%bCubDF7coQbOX{*c(p5cZzq=<^P-L6R=YElG1>@UFnTD;STtB z4<<@@Ac=NNv(N$ZhypV~8Q>=gcBdMMDxdL()@sVn8RkL%`85Us+?OgKEDka1Lv^r^~*b%^Y>g4f()9Gt}Bm^wmK*j#g5iq$eQvHy;VVG~u{U|)){BYe$MDIc zt*jS|&#t1D%&4D0kNhVq+wl_K3|;q*RxHr1l<6A6w)v7-L4DA&q`jfVTIrXEL3q$Z z#qAhBBv97Ls9m;uYnfa+L;PLHgMxexe}eZ~gtHC*NSEZSc=Tv2vh+B8Nchs=N~&9H z^OUb^s*P@a3b9_dx9cU#gkJxQk-M;+f=$qeW~fY(Xe(l8G-$qeAiK45()=KV<_c(p=H>MeB}r(8q!?nB$Ns7R8fbSEbWjwi)1Eg|g08N0(;&|1 zQTFcxzZP?7t1DYnk`^j93SOvW$)$TGLy)1HKvl{dt5S9; zn_0CzkjTS>4>}rUR$upFwf1Gv_hnP6TPb*_|mo zM3;}(M5#3NCH&c|kxOQJD&TOobHfmxDQFBly^P$UAs(QhSjr=@h78{KCIEZ{a&g~J zDD_fzJ0b^!OL?AOluHsuyNOM|6Mn`X5tHQYEI3a7u~*F}7}!sM7l2}u1nD}gmYix! z)}w4u&%hkvOJ37hcGtzc8wzXi)k^kvsR+Z#7mH3#hN1-jwagrlVs(Np<>th zrh=NoFC%5V($bb^htg}0r^4oX5w!yn?YT0$LCbn3@b4vZeN&l%vvt)mK=2iq{b1$e z)l+nYo(;lQFw%lDlXVIGEBm&-<)c53=o!|LwZ^^KhoO35K493c9E(CJ3^iBz2=$`Y z!~liW@48*lbnwu&>#LsTYhhJ~$z%NvGO4?&SW9rK$M;XFT1LgIpI*lRX&EIeYa${> zVi&zkr(%2;MgCJ#@%8;gEa2XwkMo;;m4*r}J+iMD&<6WX!f$#x+~$P1b+^h*Z!d zXl#QSrb7>&Jlr1`e@7Z!f5i$8Q2O)CJz;&2&yO`TV!17fG6L(4KC5heCt<#=q^J7R4H+4S>sz3FeZmE#leLJQ_Zxvz#U)I771LsGnA3RBj<$;1Y~K6Z`7X8Nus$TP79ZPuodN&Fk@27J z5eUshk#&cVb%%l62g#Mq&kgf+iF=2RmZ;{76Rw698CEwaQg7!N$JLe)cYiCiTNg zkXY^Rr-*3S>TnRZ`oN*R zy4%ldj3M(z)o+zA&5qPokQ-MAZDTH*5p|9F$lv-|)VOE(jdzsxeU)Xjnw<~Fz7Q4l+ zxP8~IMma=rx%*-R`aOk+8ykhvtuT9n)pb6`C}U!0hOz+qdrx-D2yEtix`UKb(PWY# z+Oc2Gxz^6Pztkm0X~o;4oVdhZM0i`@;4zUU=Gv0H+=@XC$i{xXY@;N;!SuW6J$Wu?cqU!ymWjV0z$F&8Lt zj>nYv=)@%xABq=ATiFNVGrK(TEfAuPf}^W{%K_ZMTZEylVjse9G>=!UWeZ z&loZJy54$gC^1jS&+R~Yuhxa1OcnaPtCTVG(_&sQLY9Zd{`Oclzd7XeG?9djoQ72{ zrS9YAryowr-lO};cDd457-4>o!QZdvSUwRBEi%fjV@aBqayoOf%Q_V`RT=03mNSk6 z&3`3c~UQlZ=Q zJZ(zH$|Phz<)(C@2R4LQmKil#)6-#kwQ>e8Y^prp-59PjN{;6iuX(a_zOkZ9iQM%3 zigxN}R(9z6qRal)m~z!NYg$d2{pW8z;@++8?U7e6US>tCQmIz6tG%Kq;tmrFsVWui z_@vLF^UYPSpR@WhBi4!RMVO%OuA*l3=(|a>X8s?{V;JLG4fBRJk8@Z#<~{n_#exX- z8+$%bW5!~Ev&j~-XlUA|3H*t+yg{9o@P!`P!L(F}1GF7}>tb zE9qa*k*e4dE{AKF8GSEp1j^kFSeYWYZoTOmQ9?Tr&fnO_ZWIr>2m58XbbWb{ROi+A z(aUs<=S|Z^^y^O3dXQ%0M01Ryli$v&8?Q`omiGHwt>~CL3h9Sd7z^Y>e5%FF>p!#c z$=ONZw$BuDVy~R@}ktO*A4_YHL$#JtH1F!BkTgiWvVgj z(L4d`NvSx*CohYZUS8u|+ygY%mxAG)r?0=(h7Wsp*1o{X6) z`q#k|ss%tI3s6j3`<_3p5aQHO?Fx8Gq7w-1xCG|D5=9Nyh=WZqq6uNuiq&SI` zA4BS^cK_*9#DMopl}3BX%EmgGx5#q{s)1n!XTp}Y$tb^x#sdB9$4$?vfKz~Pu<}T~ zhK5|b6&rU%k6E_z?H$CAOT1yaimxic;l}*oLd@Yiw_~|g3O97~JiWZEbdk@VN8HUT ztbSj$#7FLZ(oXW_R0!z&AWMewYo}V`FLq>u<{kX@+Qh9J&PtK8yER*q-wZ&AEvyKW zQuC>Ghs7PHSSM!zZHsd66c~j`Zx7-G&nqN-4nbT^+097j1Ab9T9oJfX4_7cxD=8pN2f1-g-bq zkFcN`_-;z1w@HRNONTTBVKg`*`4hxbBiiWx1O1~4SswSXDGG&0^s0TM7HO*^ul`v7 zS!Sc(cPSAxj~Ux7Wns7Pg7b0kmu$M`vhXQcS4fvdJOrZ2)6;VyUm4R5_umdkleWzx zYb4w+2Wd8s6}H4?F!Iri%*#CV?NYf_m|m7WNRIeiyxc1qOF-4{B^d{!oPYXY*wROy zLa~twG0L0t(;o)_@VG%GS#f7P;V^7yTu>A6e!rmvDO& zZ0Sp5h1YI>4WH@94Dbq z8`&0AVIuL|Hrr$Nc^ zqdaP2fAY`D-Yr}Y<+p zF{0n_(W#OOhc2}wDr3Q22Av{rEjcz?PVYPDzSpkq!bq)G?AAO(u7d&7#=R+lD|(zN zxpJT3O?55sz^}AdZv^#0i8|~BMu!UK`tmU@v#>Eb_`B5MUmi6zgg%Cl%o*(sVv_cP zJ!A3TTb}FpP{*c;Pp9tI1i5L=)oPjNH7+P{-*-}M^eJ{DpHdE$^Ot@K%AL92YN$$m zTvfb1MzgIpbWO|!X-YW`&KA5Y1ji>Wrs!@Hl(7bh^U`j;YwtJm4FO2XvaxV{>WrQk z_ti3iN{rmqG)CmU)0omEXucG#h@4)V13c|~514Ut-hgaZ7#&(T}#J4#4DzDV1)u#w9H zYhlvy4yyx5@th)sr&-#hs*WNp_yRbF7H-nMzU0?;q}0q-y(>L(?H!OUp+O_JWF*IT zNEk@_M_ae_AMD3@INfw78jDd4JTeNiqZQS>cXZC~7MEkbh6r6|GcPqThlyaa~AZYbYI8o)1TkM>M739%Yhg*$(3A3O*v`mhTx7)>WH_ zNgh;)+gKR5V6$lVhs?&{cxE$XBJ0cIpJPQk@Qx2sg3h%+7z+e50!Fvj$V-gbg?)LL zseXI@@DOJfT%N@!MpJbOk4d!tThb_;Vvwu)}| z%*Ptcd1Cho>8!=>y=S6|$r_3Ya9+5Bpu~HY(#r4F{p}yM)mdDo$ftwMvIV| zPgrZJl%@54|JJ$;&1(VE;pqyXRGAJI#Nw<8rwUg7dY9Y6!%Fw`q-o+us=Y3IyBD#a zir_5L9<*G8S01#FMO?)L=IvGKR=R>r+Fam5JL4ik)ew~%1J~VC5$QWwbcq2UK4Muc zWgD?;O04>;rI{ADa{V-up)Qt9Aowjfj=AE$1&?GtryTtZ;|e?C&k@!>=j)dBe4atW6zLZs=j+ z)=oux-&X5>6`M_Z=fNxzhIP09{(%BF%R%Jz${UwFsOHyaz<;!Dp#XTWc7uFI1Gm$$ zG(vqBb~{Eh~AgPJNa=DdcZ2$I703tptG4TE!mS+CR;f8&s`q%xd9oL^rMXUjRpOW zHs9!j2#O)0P2di8d7gNk8d;2T0ZF-!D$v_;I}hl68GCpALlXd|fJ0Yd0;=5cLH6c! z9pm559=aNRz?^h3Bxs(;S8$D$Y4q8D`7T=8PzofWpWk8!@^uB84&{8t=xnc3T$*39 z$JD}We&g>zE`UUNIHedWq7jsSsNhjhx^{+adr|+Ukdrq2hS+8mDsh~43iT_j5cL;B znLqwR(0~9QMeHAn2gF#Q7YX`JnE*_i2>wFouT_FT=R%o*D)sA=K>s(lidrz>^C+;# zj2Y!7|Ij+0R6$(eAk;+WBdS`0Q0Q=#3l3nZu3cFZe(*15=r{dTP9*Xlmv+DVnz4W* zzjH8&57zE|?7#o+yirvA`tFsNkmWXN<8+jeKIpQbPKJ+SrsUa-CjS3PoMjc1o9`Pe zspucO_r&OcI3sT*Tn&snb@U%z%OtkeH%Ht%2j*8eF1_}wc=`xwuf$oWS5ET3C`tPl3c+&J zbHwM_Wlj6#5~+24lv6BWXi#kspQiE5pl*#HTYc+*fmqYwvZ!;dEe1O1VW13B2skXk zZFkGc_s8eg;}*T^>lkCm&)Ms-gzT?x(KKG^?WFtS{UA-q9N$alA3ufHcOoS)dJup) zY^?sx4EGx4F&1b+l>>4&^0G<}>&4*t{`}xNBMz=|NIQYXj8g2a!8K<;q$j!jq4GqP zGTN8TSg@+1m|`%u#hJ4)i~S<}fWavHN=H~c1i)8r^q_4Td#9$x?c2}f)wcRcxEN+c zZ=y3uDrmZ7yrhX5>w?=N1&P1<9mxq=Da?qlOoOa#`qz(rMuqnUL)$vqt4*2#N=9I< zJPk~Ktf#6q?Ee4oYclmndV)<|bnM=`~w8;SsfL zh6+kkA_X@Xco*OOB5i{|YUt}!Qh%N%PImb2>bju`V^HEcMRzn(6i77(+*>tkSIr6srBPYYnLzT7*eRZs!O*Hj}7~fkxdU!gSu|JDHa<5?{Y@U6|BK1RJa2+ z#>|W!y@2XphY#dcTn z(d1mZ3tmsrOLuK9stse=6@GTU99~4WEe&O=zPvz<5m5fCcu&s_>(Rx`Qn4y${*JLv z1k+j;gxlN)mwfS5tb&|`Jy@joID)fM=+nc>;24TLba)y?nz|G3dy%_#>u5~I(}}NS z$k&=OyWPd+^*huF@kg}@@&5Ne0nMk-r^4ZYEV^tW#8H{+tC07w*Jc813id1OOtKq@ z2F^b&{%E^q>4ZFNFOE+>s1xjby@{_POhDtbindilKXy?nAe^Ox({6Wjs+ckaTXk5E zC&leI^x{d=V?HaMrSg8bqT%LntD$$AuM+NyENRPx_Iff4p1x{n2iu@j z)KaI3e7Mahw6hgH`Y793@67uEALsY%dmW8TqQ*!ji~?-^PH}XcP^cVq?L9F#1{^d( z?jn@unm5~0CQFZo4~|$2F|0=t!7KQ z7#It%ysp&s;g$4Uk`3B^YoSIDY^V4Qzq4t!>dpz;-?GB_-j2)vs_tV+*Lg2Fw7xLp zV%wtaaXvm;S5lk4+wxMt>si{JuK`r~`C}~Yf=`S7NAwB#d^JX&E zgxJx2GHlVOS#AXP6O5`>UZw^i+Y&#u85R>N zD-Q`FRT*M8%m)?~ekSQ36KrKH$me1GSY&nO9yoG6!MF%D=>$A$dQr6fM{*qm% zpR~Sk@-lnl%-GjL35j7Es>-jceYH={Eug|M$03!mU_8LEAaB(2U%PCel0B*&%pB%R z8Knff^%S955S<@tq2djjKi&oHza$L3c-=G8rOTkv{8gK@SAfsfJpVOP0yoXb&18{< zxZjwJbBKl)=L8wCpDo*_ZRnu@_5^#!YLdOOwQ4>kDSpG4l*l`EV`V?(Xj5Oov5P&4$p8)>pa|7AQL|vwb^T%^DFy7=p&~Kq1g)I%`VGKqsomSaN`tR}eQ-X^# z!MjI?_bvD13*c@o(R|fXE|Ks|X}>+jYPJxd>%K?>TD+&b-z0yP(qpPAY=~eYM##fp z*~oG;AqwQYkQG&Ww2N*fSnK4_X*Kjrl+)93um(YQUPE7bK7CkFos(O|*il-cJmw2T zH8XPkBW%1@OLeBCRVWl%AH25wI5q#t5%BTYQ%rhWOibcu=WkBUciY}(?%_PVTdv8@ zjR;Vs?Eb(m#y*K-(rcHdc(sRAO{v&56wD;%E#2}T!J2@kmj1ZRT!^>-*M{*(*_V)y zQ*{zIAZ_?L{qr$G9}K$QpJSEC=U09$6x!<>M#Iqg-!ea1(-I69HqVHNr%Du`)8bSi z!S)3%F2(*}6w>d?RaV^2;eFhR1EZhi@Thz#&LC}x0?fLjof)fN{@jD)dq?3sV7)%B zvxVw?fAu$VD2okvRdBPK-%haH5tLwj3(HT=N5Ut2yY_78cQaG-icJ%_RXS4nc41SQ z>KPu--n$a3NKBD23VR7;2zp;-Z6*hp;gv$_ZFUu)`vUxdx14!?**?Jtu>JIsNL_64 zcwx=5FaRT(357p;PWmz5m@XOpBPGvfgL9<7;T>ZVd!0@dhePBYK34AhbN9M#t|oJv zUyi#P+KY8zLH+9hs#@B+$zG4^bxu57QXdl zi_fk!)p{Q)00(WWMXNUV2zSqq(B^%F!1r^&dV9XBccf^0rv=9PlZO*>LQAPNxv?AD!Vmkd1%Tg8>Bk&UhIbVa zc~{4N9Y$dQZ;P!%IY12 zzr@d;(YpXJHR zzY#b{48ME>Mq`urxC-4YFP%&tS}LOXDp&K>c8BxT2AF2}T!gA6K-n$>RkinK^q!GJ zD9O6mI_IUH2(f$ba0bHxWYA|YisCmUbq5Et(z~LMBk-6uWjuoWmGDk1U|NNtM;_lR zl2cjXWKB8>Zu#5+`TDzIjn9lcX`xgFU{Os2c8IrvN&$N{o-B5n*mg(ZbC=JaM)#Or z4>i~X$dKxL@XVk3?3b8S_dGNw)I2t&!rA|#QM%@lUcgXx^3HMPaKXJ9X(oWQoC{C~ zs7@*8VHPgys29RscXl^SVMo-T6uBh^&b@Q{ z75q@{=qy#MEl$=ATyV&pL5`RWQoO^)JZwcPRVsyd`a$ zILK6vBn7p&c1;+Hf6#wa9_;r6eu){u8Q~}-*WcuWL2mr)T3BR<)~eV)U&0o;W|O|% z^HrT_V9&q|u<8!QrD^7VlJMirOgdnDb`h5 zqpHu?vWF;qf>_Y`N7Qz89frF^{Lm?))ff-GXIat3)WKqzY1|tIeB*QLn&l~OTy2XI z-?eY{ovD)%v3*{VZa8YPzG5;Qxf};$dk4h}oinSdUxxaRpQj_B+A`M9olBRc9`ub1{jc2fh##^^+`+ac5ccSS_4KRWcs&|%B z^6WA&P?=^&NR}W5t_OY=k`Mg(iuF&)K~}MXNfZFx(iLISlOWBtz{~Ceiad+Dl+u~& z!uUQag5Q7k57;iUniPmeR^@wr^dg9xGSh*1Cl>2{V*Xj)c}}iN9qG7FZVj}VQdR}D za;dv=EzLn1@>Q#vkH%2zK%0!Fh+;F$A zq!FTun7@>-PyQum3XKynWWZbZN3tdwh+$=nd31Ccz6dz4`zQu;Zr77+>XF0%A?I2S zhfWM!6<5;R>~__1ad87?e$b_jd?{9S7rq}*+b_f&CWMDpwd!#TUCst+-i|V5eiUqt zjj^a^%trvpan!(v-dRQz@u|u<-AV8&vM(7~90*#&%T~XQHfv02;-#JL?Haw|&}Hq} z4j_tL4|(RFcn+Te?u^s!N8gZ*5;4|` z1&{8}s66WpLt{wNMv((cuZPrhCO!t=2z(bnHmObyp`n1OT2x=LetWU8foaN-GYy28I?QcB}qfp$0Kmm~@@JXeawt0)X+UrAPjH@816W;+O?tQPI-l5|9{%O=+hL zb;W6=2J%?_LeERbIIMV`Whn7U058B^=kV$mF`+UD5Jf(IOC#?e3HSHH^lt!h?R{SH zfk`Pk9kzZ{koy5(n3ew1CwhLRlW{Uai%GAT?X(@9MHBd){*A+WDY8D>CFIr|72!m< zp1E54ydqShVfa79VIW^(AxM++QlX5Y$nAurp7IyE6X!zPK7?JO*&7t(%*b691<0%8 z84djZzNPjbRuVMD`~M5N?q8(k|07l4{~!g;eCIr)`ILNneV3H&FzZ3LFAJe#hBYYv zksh|mFcw@jc%SuX>U;8x7!Gmn7O#@C06M-&=+1vMe1HE<41c)D{q*4nlmk)aU#~Wa zgL{_*p-Q>xlw99KzpqbmGZ>wAx+R+v^ZQy;wf&?C`MQUMzMgFRT_8pH8Qn zwKOzepz|j=c_k0N-D(eu&pxXVh-VW}Tnz5~8ld2}7e?c`F_3-{z43D>@&fgANY&R= zD_QpJ{O{5zOiL;D=*X54##`W48}i~+LJ_5UY1F}r9^t=nw8OL;)2-ot|?Ke3MZ(A-I&WxEL`z+YuU`57g0Bxba#U)sk024VJU{MVL7_Pi{VE zqqTKcHrnQ$F0~D12~^XDPq{6JG}uI1Ond5-vhvAvKu_>M(&?<#PShj-JZxo+UR!a@ z;L^Or%v!il4eK(wJ3GjF z?&Qc<&x_$RyYz@!3@Ofk-fGRWA%q=FQ3B18BqGzOOoIR_(o?T$%kgzmlPgA2n%`x!jNVnqBDxVpC8LRe%p%5DX6uvf+ z)Zm!f#6|j^907sVJ<{}^1Y;jYr{9{2fKW@oxGy^Cvb+K5MgB9IGfm{{@dZIUkkU_( zy>jD4YbFjd>V*!o>wIu8a`~3$8&-r}_pEU2sO;z}^Pgm{(#eJ7nGx>}(~}p>W4Ps< zptuQ+k*@hx@5QEj8(s=L1KO%%NXzP0>;QV2-K|mwVoUDC>O-<8`=tBXm_6%9-Rj`J z+*KNNZ(LyLiZDS`Ol&8n^In#hwI%5Cp7c6ZSJ`lIz?s(y=PZiOpY@tWW7jQ+RDTAe zI>LjeGjBjeC(hk(J@Gp=zK|G=^6BYVIev3m9gHGXirvskho-NI@6_&pdo5m|k2*&B zR60163=XZQPtgaZG>bo!rO)hj8%59f*+nU#wtFfgz2B3|Bc4GKWX0E!;7xfN#Oa_`C zsmoa|w4<2_RS~uvf-UQ5fs8pT-@ZAH@D>YTz~`8;SYZlG;efW|HQ>ecobg^4Sdh4y zPHN)vNb;r7;WD3m@*jC%*!*se%R|V%K2;}{Qe2%39e-Vgggfy|glcX;h1nR4&ZxYL zJ2TAsx0@cbE@DVX>SOg|x*2_z@w=xvvlu#DtE#ww{FqO?(RV535Djby476+Zr(C`KGe}{%2_$j z@4E}iCi>TMZHDm=+E@yE|4=YxZmbl5+fm|^GnSy}C{b67aF%42n2&1lZ2A*-BAa{% zg6}aF)YlhKvrb+FgeG0zibz?J^dY9VrqDiNyOnA)1?Gf$xKY+Mhtb}|Qo`_iVbho6 zeX@GIjy19p1)NNZP?Lmxbi|LoE&R#2IgwIIKX03|`hyCyJbJ)mPDe_=_AKOZV`=)V#&%jm<6g%Q{%#r-cVtZNJWBo)H) zlMeX}uRc!c!i^aC#fwnicp1BNsKyr-1Sg}v%px$gz)#-q8u)_=3@wdG zk_gv4c*1m>n_ds2XU4BD&fH}%%CaG?ja-uW-mN5T*@MpY%6~N1^yNY6OiV%J<`LI#e*;}RQR|7#%dV*+%w%c2REp()5~*x48GW;-mf+wtQkgZ` z=ac8lc0><|)xLlMDWSTEeqPPnS*6B`EDzSUjAKLr;a)Bbdr8iepRwRZ=()=`D|!F+ zyOT>83L@gu$nGZlK&x4PG`oz!?*FWnt}`p$W$Wz-z?wFOMx^z1 zjcSK5X%3q*@yeZE`ntO(!G`g;+foAY$HKiw=I8pgAoyBqyY~%%(ZdgAM8BWCBQ92) zVaPY|UE)zW$4%?9K!KB21Nv0N3`c9vq@Y!3HZ^y{AKL_29_uZ~S?cmL zC&evjUg{#WGZ9J^(p!^2si={A<&0Z&;l6zwOnT6aVX6BQaOlX~S>cDPr5gI|S6Ex~ zy(X4&^Q*eI@T+yw*l{gvqU0rb#I9k$j2zz=G*-x{*Vo&1*e~4fbtks8!xwu51J>cu z)P>}Mwe~=5V_$>ouJre5wPewtgo4hDh}Ay2PXiY03ykBvQOS2_G>HU<2N^Dk?AppG zbDV6=!@~eT{3FLM;x|47Ap$S{H(&xz%ed$As~GktAK6m$Y~J9sOJln~ccTkYv9 zV7J`~#_G~=*qg5*^{1QI45T4L&f4ZxXa~^-8+OIL4~nKM+VmQmn-*1%pLq0?$Zmc) zkJ*pShtN`e8qG3~V}8{}5a<)yc%A_tagj;drKNwHaJtsl8aGnlbhMLGfG4Q`P(k5M z`=LWIO1Awjgleao_QW@d#gOe1S(6I4i`>Vec;v5dCY7A~NA2)MmD~0yt#> zv$2EkdQs2P#{kLEONbY8GH$j^1DM~+ZmsYARM?=7NqzgMespSRg?f)V$)u-Bm?moT zc~Ma0sX@QAsX+_}b>WVVo$u>#t{iX55P?6qJS_cIfXJ=(*4EaK3fA8$T{ghnr-G{H zy%-IYNfC>5!-;J$(&)=q?NpwC425;qd%Uvt_2{MEX1Msv(^8|UD%&(=EStL!?yJIjaZP(csx zDv4~~M|FKLA(!taR2p2e@T@j`U z$lK|XHmj1v3cn0i0mV*ji!y&FJH^X?nEZyspwM1@amIpX>r}quZ^ivAk}YAdo&G6c zKrncnQ2oNJd1&h|okgIID~LvQt6eXK_q4NvyFTNBon_0wOL}dg|NENT^?-sLg@?3}3 zvlIcsQ*R&A?oo9y!<8t%$9ex(Yx72P4(?`5*YRsJ$JRsbA9{mwM|Fcaq`7IH=6*Hj zL_*uzWeCrU8W(1#xD=dHuJdvzKM>)wVG@?8H*X#Gndg`4AW!{7-Bwkv3jp8eT!^i7 zSPDLfDAa|(%(xcO9q0)|Gq zm^0V3ghQO5)l*MIL!r2@EhgDI%>_=$64q6$Y0~PNT|Uf5^oj{3n7TlnXzb23#ssT# zh#MTx^>(|=1Pdr~En*xx>R8=^(yHAeF>d<0MeFI=s^wbY|6@_kTYJUl<1&(1j_4T+ z-aD!{>ILUi+B#IQ(`4-A*f4KMwJhK&iRaETiT_v@mhenfZBR8v*+EbT&fNLNe~#9s zai~CySfzgF63%oKDsd%rezbV3pNUz_YA*$bf^TrTlkWU;(of-uQs^r5y3?OZU0I6v z`rvGb_58degYyhYDFLOXec`vdLhCAErHWFLa%FXvo1QbTJ>dJ7fNg1s&M0evBOPhH z)ny}qZgGc`S#i(uOAZ5m#)V*gJa%0@4>D5!McZXv!y9CrYfky*az@fMXLqiZxs+gB z@5Ay>idjC`iNSyhoXa~~_qQDt&mCA?+Xfgh#p5VOh=Wy#i$Ax^Qj zQPV$((HuWmSjVn!_9Ho9kC~&KVUtT!(gs_Mm$Wt|ito~SA6qaZxw3)B4{iib3tmxcWe9Q!McE#&9KnI-H|4E%atY>!jUq~Djx*7!%iTpS01nD&Znxa2$YuY=?{0*CcvylhP(S?6? zJs_1WHDKxgA1&%Kz#?+$y(VwA-fX<0%8YkwtjhWC!>w6DMGM9kF0&ziuL!hEi@z%P zFYTeqaNN}0e+&=0Z&Ci?)xS7Frw2(K|0c{Jt)3VDrEAO)Gt;^IPwa+PUT%N=A9@iu zSX^Z%`^P_F1?o+@Dp)toV08QTZI4rJnJ!QHQD3A%3D!H&Dif3r`VkzhMS5GWxxX)@bj%R)~{>b zP{u1;hyuI_5f^h{r*ZN*Gac^^#)2M>{D;URxqcNl9!^;;`)g(w^+SGyWtGiEqzv$l zfStHPH^VX#dkb85`FmMg(_!qxK-bu4CUhQMI4sR86U3#YXtBZy`>~bGLI#NbaJ@z{ z&73=f{amfpWa7h&N10;sD>Y}NTrEXawoV>=)RW1LcM5Z%G*w!FrhKT*J6SuB52z$lLD?u9eprjCPnUq&xZaQfu-KT;-hC zfio6(iufx43t}n7RkElAHg;(O53D(p5;Wv!%UH0(DJf6{jDm2P|&r_rE;4t#EuL#{x)ae&ilp zW5C>BUrB?;tI$+M8H5BKVT(*jc4l}bv1z7(&inp;QYksgyecLo*(p3KEp{k5N?J!S zv}i0R*SN^wqCNCsq7JA((_-wsla0w>`P0SF-p^8hCAXBnqRmQ`3yid*o)%}*>Ab(} z-z!Gi;o9Z>knDD_16nTK%h!QA5#i?%T*Q>%hgn}hvyrw3=h5Wn6(f=z#41};50j68 zV;Z&7cPo}Y-7Y3p`h4Hf68UAurvQo2BzWdJH1K=+SwQen=p>s=et?+a^B z%sJS|!a*YpIFdZ-=N02e@|cdBk6w zuBNufZVJBaI>!aF1=d&f`;NDWmdgO7Or+nLcu^PUDA52OI>L6whKY$OcB{|6^JDT@ zAb}tHBcOOESla^kv|@>54_VcqvwEKKJ6P^o8$375eR)km2(F(c<;k12o_!a78=)4=H6ci^ERV(6u>t zUEF4>d=fOuV({an-~}prj5RIjM<>;g3yeOe&1}3 zeG>MD2%bERL^{*ts({8y-eZ}ixDQb%<(j0BbALVMPnoB0h%E<=V30M~e+OtpY75fl zf?K0aZ3cJ~$EvT04c7z_8@!T<_CPBYPuY_vG`J8 zqtcy;xyk9W7MzZ5LsiX1K_i#!Q@ll7pG?{3Fh1;uW9cJkB7k=MCY6+x$2uey%|+dU zo!ZoR0XollwC=gj?C?$JT4_Fv8d5op0bkSFMD{3;51xgtJ^Tb=Hm1Cn}0_lsMRa$1}|`&OTHT#EO#llx4dzK+tlXIo567y z;XQrq*XuGwFP412#Zh9=WPOvU`%uDM#X`8j34}LCyY(JOAKVDoqmuGV=DUsXyzsdz{&=R(9y1a^L&sLG@Vx{b{W{f z_pc4D+`l|X7FS=tr;|I%m`+9tZetNHrswrP;w9Ou!dc}OKII(E(+x1Bj-FI;q0!*^e zSHy_J@m%1i*3A01rh}%}WbP4H9Y@oag8vLa;`FyIytY>Ry{3mM1B&U7UqA&!?($1} zy5h$&kwm}2m+1Lg{;$kqa^7=udlAK)y>pvVy89tU#5w=9eGN+Wu%`6~l|beUeDHk7 z0(<6KMV=EpX{G(=U$qUysgcBy*b34FRQbKEnN6Ha_82$I;*AJ5bE_JU#&G>QCp{j3 zzn4kw!!~kNjc*%@?LUp+QYtLhAVd>*Z_9ki>w2SPdO+5kPqeOAQL~9EmDtyw>LQf- zWxi_A;L|OqSar5OUFt9M;fLt<)q_=AKhw1e2t8B@+}s@=B)P&k0|Qo@?~m20pNw*t zebu9neYa+gt-dR>9yXEOBuv65*g?(@A7x_5%ABlJZrn9d$vvsn%q;SdYWVQvk|M#p zzRR}`C$JMnZp-oZQj|ay$mnV)RojS0%3Q?ltbU+dWTka&m;9YnD}khX^EjeD;|t3N zyzgk2x!U<}5!x0bxJIALMSvPEcJc~wr)ezy_UMc%kl9>}m7V{rhz3F<<%9h0A_1Kb z_g|UaSD)u9-oH8L{w~Sb!)tGcNyfVQ6OUqqMy&L(-NqM^-L=l2R&nfum1 zy1BX5{PbGNpOAewHomeT*0%Z`D!THYkBm76@zeYq@q@438p5vg;RuH`vH5taK0{?w zX{@bajl`|%cdY}(9YuB&=h!oMCgP`=VCqji{{latH zj0HcK2pP_a)gVp95bv80umN|+8o+h+vl02~1U-^kM|R#97Q2y?UR~IQh>gA|F{GjD zv{UWfu6`jmSie%yQbS=W@1wej+%%6m*;@{KO$8hOi@o=bYAWsGMn^}684ws1L8aM- zUKA9R60iUxz4xL*MCnC32`VEfRjDE!1VWAUnnWi8ng|i;QX(a^0HKADP;yT|XPo=0 z!~4g*>-*MvXStFYC+D1}?DE^cy`Ozv9U#dZvp}s}NpL*ML-Gq*mbiW2{2_8*>F^%% zTS1@zj+R)**~0~8sj5`2*Pk-2QsbkP1TY!bzadNvYH8mdC0jhC^yyG2A-3+;$((?T zy1iu4^C+})3F^SPA(V)IAPX zYqh08TWFUOj!I>|U)qZx-foXAx|@nTnaOW7!jZ`-I#m}klqA8~4V%zcq#@vBM;OpZ zLQ-mMzl&jTimK8>gp@3-B^DGc(SJK~9dUE09zpEmtaL>;XP;{IEuQu!l6K{$S8Uo~ ztpiuQiE&qd$X(HUKHg3jIU9^mNcVqmaBo)ZecjkM+XzoMR$NFlZdZJv;@+1N4}B}F z@%Cwov(xVG*=|L7o@k|m0QxVzhn|6e!#<`2Rkdn*%JkdzWoAt;&)$TGMNfu;8-IN^ zl;Czfa6glDKy+y6M7z7n^sQ$+`4&3(>GZP>$ThWE(<4K$+F8E!kw*yA*%Q78w1?Y1 z4(ANKvu5Rz@Q@>|6FaImMj(RA#bOXIt80&uRV6%kSz6Z-#Uod4H}bO`fDK194zG;d z>QsaRB{h$q)mG1UYzpnoY}Rqd)oa}Du$4Qdso6OHrbz1&>|?U(Dfy|vb^U$+xROshe8DkO_^a*0h3RRXbME~Jiv$1Vz1=8BCLPN zhb-^DB@y;&{g_et{1xfCEm-@5Di9!mW$$2o?u!hy`mmKq-*NZ1F@mziz!OJca4Ka`6yn6F2@4WqgT}L-pO#aX-k#CjZ2!1 zxoXg+1oMVQ>D?PuoO?4&;MOFJvP9gkqCixV|G{VKpnqv=;;cf6#h~D6>@&k_VnuQ% z$bB97{Y)?a-md4nVW)sI{-XLNgU{>cDJ0qca+iEI{_~PIl-_JvLQ=9lEsZ~BGx*ta z^{h&66&JDf!H+=h=wWDi$8-N7nXEHLj*2Xq%T~X|@UXz$(^z~PC(el+THGz%K417@ z;wNp(x?f#@raN=2%ru@-!m^y~?}#c+duo)(ItpkMQ6Rzj#}xM1(7QUNwH4h%K=%X{ za_3U|L>>yeN3USvBFug`&i+3Qnyb!g*vBPTivubLrGOyGadPsP$K-@Ff3U>yAGfIr zNUN2*!1+jcdA_42EktQ*gh`s^0cZ1S$WbLR!J!Hf#U>YB4sQwvDTilh`ZOo$8u7vc z=m2n*3$M{rAzPX8=9Pkt`DS0?6C3-7!dCCKFDop3`Jj&nCG%-~UJi?1>a+-gzf1!tr(oiuD~^`uU0Y+N~cNX7qdF`bp^$I!jN z*|p`pFNbql>Sm|hYBss1)^*ZlW%eIpx~1^)*y^(C&r)~Z<72L#D<0ke&J)m-KGI|V z2u=Czjz$_~FU(#Vv8^9(ynu*%@08_az>@}zOR(FdF5E`YqEUN|Au}7#l&j=qb5`@L zTSrz=!_O$TP}AWp4k zCijOHfx@2-Qj*_kE0tW5Vze;v-iMU3+EjHvv194BVnBB{>-`KjZPUqT6E0n)_>{dX zT$-Np*QPDr4UIRN&yiA?^o4~!BBz@_asA;pgYISsCukU#f=o4V)F2|lQTEzmBi@0l@Xv z9Kst)onDzBzB8Rn?zDU1eu$>q>lUU_@@&>p!m~L;Im6e;o0o%?A0xHkTWpTXB0%Pd zUK*+~3?Y!q-?he7r`7i{FsmGfyq+S5-*`9pk^K@tRh0NgB-WEB@gExjnSM!5dPZa8 zA9nSj96l$0!=~^c*L{JPL{)NkpnzIpySpMSuOjr=`_?mT# z+}UHFEpD?#3F_Hw;YS7)_6;)dm&}o^E2X^`Mt-AcsgRxD&)-%N#Ju0eJ_PL@a)OeE zM-mM#^|YqOt@i1vc>LGb|NqY^GCBu#W%|Emd;Gq5Q4-)IjpgCjMlE(>!T<)dQj`Z3 zw#`V6a@#<}1i!(y{*uYU!cuJBuX9>XRik2ybTt78@b)zP^PAS%bmc$_Mcm1F{Flvu z&AaSr*oTw9!f%xR!I}eq7B$!3oz5!wXPNwy2ZxVFzbM$BcS!F7Q>f%Vwm&*_&MscE zczONE1?a&jjXmeo?(hAP>CC?P>#$R2el{}CkfoXRpyMW~ z6*69=v}9s~=?x@42;Zn?^3muK-DLa-GlKmf9p@#r7qu~{K(%N8z0M5_33{aQQ(MEw z4s=T!ZR`s89j~1si*T^OJKsY@;MC6x*(vhnI8MDL0DH!u=>b$@A(unRPN~z2v&nW= z%gJJ4DRi{&qOqkVHJRWrftTy~;lm;x!#n!CJIi(h1!tFPT-D_19yfkW*5O@okw%>g+B`C)Zs z6#8m&3A&CuC}34!vz6W`e0ThafHE;g2QzDg#)n;th-eIg@x(;X{OBd!xXm75e= z(I)TYVZn!58zkw_hKO(!^c-}3)%_(C_g4xz3`*@dmWGj&Z>;^}GQhPI} zo-*}~Zs-9a9q~=u@Ep6AXf`R}^ymDatWtYTZT!LY%dQ-t)sJnIM3TsTL(*-g?G~^v z0=6%12Ob1U^&Rp@!-F@7!&CVg{H2JM4?&NuOY5f#hx;6WqlNV|s$`59TwFAhvRd*OYOhSpJ4X%D91jR?v zaO{orUi7Jd{yBHBIi4|%gL7)F7Js@e7rI6|X0<)ExgbFISPcta4Qm?G1ncIPQ#$033sFtrA^(ZArfX!( z#6<+T+fpH{M=Xrku+YrIO%DcSjVoy?&4%M8Wx5E?K9TCxmK`5H{^X==22_2=)qEcg>a#%iJcN)6ree`R`yQ^aNinb?Y1u;MU<`;hIbVl^xb{q_w4T1= z3}TgktoZ27o=+dYzWFg(>+?q+@1Po7=);rUj&W~y^iuFNaI4tv#B2FWOSUAQX0;&JoCVpaOz8(^{BPo zWjWwDlM$LX?$t5~x+WAxpO&GM-J4GIj8!}ppZp9!x1i{k@YLSxHV-2GrX33m=q2b| zI~>Xr$|}|9evZDr6lUpK2cBpoK}dmUpz&bXQk#MsBU<^>cf*Fd70!4hvr5)p^_#1k zsL#g-5KCG!3;__RMarPB1?S$mj{g#UehEuok59^eFVX?>?4J5Z5x@h{{cb3PaMqCD zXp}X6?PCbXT#y?z-IF>r#UjkY#7TVg1(yGe@3)YCN>2YI`7)28iYgqnNz1-TBg2E0 zfS^DLo#s1K=QWf;r)F@(C@#JhkX)xXOcxe~I%Gh6CWz(SU1IRY^5tbS6(BCsMgwnD zsP;;}4#W`wEW+K#{sq{V-l45eC@ZIWcblWp9LkNW^}ONmO;jxq^>Cf2pUjR3O|pgh zPY}gQYp*Y~)XV@7T50*+CA4>swOk_*J}D%L04P2$|IT&3FT;kfY}hCZEPIA(1Y(|c z+Z=b}0=L_45^U#EDcnGFPN9Q(+(2YB|6WsSHV8r`Byr%B>Fe)1u(-zYr%s{3BKUgL zqKb74s6Wa^&#lgr1*|#sTISZ{;@X^|49vzmB4_y)uZ|r%pvL>x*Q;Y3YW&jIzWic= z{RX9VWUvD#)cISIm9&*l9~f;Hqg3q$J==2fv{Il-8T-FSYZMfH1Bjkh|D2DZug0L2 z*U`%2D>ebs1^HZI?-YP2d%jJ9v^SVe2}Z3t!%%AtD4roeK-18~giFhVfHXzWQ9wW) zafw9MfrotzBzKvHd9|H)(AQ7}Vz>^&oeaW=%9~g2+mR0<7c{K`Q7Ir*gT8o#a5~0s z?6EpmzS6?0184{3wZ0}g_S!OXogl>l^gXhSH_mmQPP1W!jkK!^@PR0GxLF!6?7ZN^ zDUnRGu$=`@T5v8s0{-;B>h^}1*}I4T=Pv`NUh$>xvjQTSM|Ng-{=btbDK0&?#*G`GwUb=8|6$7ildt(C2R7oF z(D{y*#Xh03ow~;VD^_C1x@DmpxG@2^rQ3JM`2_0>+b0mIaJ zFvcoQ%5(p}Qb3fRDrUS*ya~QH`$=d1f&I4`=uaLDP?yptVHt?crQsM7 z%Gr;eAYf%b#jFeww~BSS7WJ7V|F(bvh{g2HC6T9_cCQ~2_m8zi_2jzpOO`Qdt zzC$_xZExfqcad@Q4b%tk`rVEV<|jT=Osa*{a9f`<2v8{&;T5f2su4R__k)^3Oq_wace0y~d@r?8c!#!9 zg96$;y<|cI2^r-9j~2d&4x50f+~MMiHj7k~77~=wV2!6R5@$WO!5i-Ds|!sOW#Gjc zgB@-_repXnGyenS>q!6_y~O~;|FUvGIspLv=|?t~b?LYDk8H(^brb0d>m%H#LpFZH z_lJQz*IRBp;U%*GuTJW@vQ>J76tY%0H6ULsQxAao|FWAvYRE_=4R~D)ekRBo12X_A zZ7g@7tAMsUlK$mIrS*<}D}4^WNF)$UgfWtHQ&VMkfQp-x(0QDC5!fW6F!Fh?q~ z3$aqDAiquOL{hp93(Tps?uN$`)S_2~{1pO6@42U==$_!)`uhhKCtc8G3UvDg`l1`s ziZ}Q9+x)}*=dVK-{j2U^@_E#Q%}zv~8+-Zsk|5*j!6p7kMf*R${NmK((*;q-vad1% z`83=IkEK>Q4nSydr5&Y6+VpU;LwX;Tw!VZDunC;4%FR#OJsqnld04VxrhK@pMQ134 z5sO%Oa3q4m46zu2Q=BXRCdojOt{=70Kis&~#$yw_LAK6MLSz7$IX56H3lb{iHGv07 zb^;mPDlb1om;ltTb}Kh0mBC{y*`a{ozenmawdDg|^O>qDa*)A_0B}d1Q?K&-tIny1 z->PJif?lUx9MC9NE!F20`P{%k;QMM<^uH+$_so0Br?O* z1t7Oq2PQx@M6XN+wr3bl7GaC;nPvtJq%92uj0}h}c~FnEHgYRsiQ!g)xYSd3U?dpN zKp;{B2%PFEJN)^-4CxDRp{LS*eFr|NDfS~E>Q$?Bz^h=n`THQ#?%3rijx^t?m!38ESlLor{A)pMJ4o&ksV!x;LpGBn39b;Q#_D&=k?_01t zTyO`QC&sNqC33OKtBNFQn7H~cR|j?f$>WBqJ4fFzg`Uc2+UKwX$@;tqX2vQ9?G|6{ zY~k)5{qGj;UI%LBaZ`^ja`|>`r|W;QCP!hpe{w)+X8+Aq@?fca1NL}CIsnz|(f|UT zMfm9sL@3QbI>9qDUIzecWsY^7STr+oIjadA5O#ZPateUcC$>Y-&&YO*O8*C()>kQu z1}aIOD0(a}OTS!syvZ3qb4;TF)V$ zV&-pRR(T*?&bo6w*+#Gz^eG_yCJN^R*fDZ*zni4zB2p^7yA{3T!B?`tMk&_y=VrxX zGJZ}PXuYTNR!5410a`ukQZiOeKoKX3lYXg=*(NkUgPpBvknKW{PpCZKSjBqb_1TCH z3<_)Z?y#jw)SCn9x>5(61-@CX^9f-`?+)KEP-*YL?&(+Gj?h2D`K_9l?Q)k-QA5t1 z>nKjWoo-976ceY?4wrg`IOjmJ92*<02`*0wJMBNRV_C61Erd{Hfba-x8lPImszwu8M=2P-})oVl}k+?t&|lXvQQX3h@pKLI5FgF z>*yLb0selAx%u2l3INbvGUvUvqis~Lx+9v~B1$eNtd#rOvBIuPH62w)7ivAX{#RuK zEI!h=|1p+DDjxOo9Q0eMFH(p*Tk~wLf@Jn^Xzv;*Xj7!f~InWm5lw_vT)ZgaE zA{<}>Sfo$9-?uH&c?~8b>Og*3ZtQMHh{J&U_{gRhWvx(AIJ0_ZLoL}|Xl8uKdE_24 z$zY0+kEE4XHboX=E+J+BN0W>jd0?@V2{>4?!U7twtE;maZyjX*1R@2$S`ct*c!vvY zVJzCTyU-VQA<cd+R{j&k7rweKrdSrqzJk7K(HEc}J19)jO|UUHwH@c2)5+ zaTXP0_VlZM&6ZEQ<}lK^#ww7temuWq?Ztm6gx>*$? zthQ1B9WtQ?N|+E{ZZv*dbi8Zyfg)~zGthXsoPu9xq*&DuK%~ehT$LFBsz?>`fn+zG zI{*4a3_xO7d3PD3fwzh`ERpYX+nj4$#E5C?#xQki%MG3x@acB}a?p9{&n$u%l?POD z_LqN+EKnj7<6;y!ql?6w5RTcV=eQXQ_DX&YVHo=IsBTqyiqePxk1?DbA$ep7NDL1r zTDl(QhprB%EzQ*vOPd~GzRzD6>KcfoJKlXUY-w^vLILsN6_XOT9u^aAvWp zQMCAo@?>Pv@>@1&Tl{28?}&zth-LH`tr&@Fm@g*-zGP0TTRIfKja3*X23M^L)Q{1&6(VT%VCjT{|#4rd~@ zvaW?eZJ0taJydD16+J^oyp5h@OYa?7NuVe7r${-73y_hel!ZdY+Kd{vV$2_9)_)Tm zf3qH+$dA{Tc6BX`#pz0d@~3)Go8-1zKw$vMy@k(rCTIg7KOn2a9Kp3q-X1{Cngge} z^tP>sQFlK7!3i#*gy7UmvhsZ6-Um)%i&64f9FhdqQ2f`e5MGwqxi4J@s{_Xf%;z0$ zU^W^E6q;je@(UdN5wj!t+FWs?`+E`ncgb!fKVFne4vGgy5cJag`?`qwDyt41`;w* zUol|;ka%&Oz>$YSp*ZMDMFc$wJqe*=ZTX}56sm=bH2OLf5L|jqIWZvlpgAw#C4%Sm z40Q|!e=oW7wCyjkC(HME+MD-1#>PFBYZl8$HlC}1D?+a?kiYLn^JZiVw~d5gvS+L- zbi}fO1=f&(kh1;b$LivnnBTOkAF|SVcTWSr(bE*ksTY{!Rq=gGFVYZI$T-(@J+? z>l9O1bOCKR377|2gF+%b79r;ZIl_^%(t-z^Ilks`zZ_7#e#53E$RQM+5b?*fk!0qG z!^0|u329)-2UB)o32V+;`~hBnw#)3~W@TY|qaVkPXUSujj6iOEkQbg*5KmlK5X!4E z>dl`1Ps!|m!u@T5075Gfp(!9QW9yuWq&{>L$rOxb3`i<-h|<)-MVHD@iYFW&V|X;= zCT3LV&)ms3fsdo!Ttec{e`vP9H~Xo?=bxAgT%%9kDzA!G|@>Up-=n2#-rP1cQVb|yC0gTZ#4qUhveKuWB$8BgByS|dXfxg zGLn!}p3mi&O|71;_AFngoEPLEfTZwC3|EdD`T0oPFV&w)MgAFC{k^r?1{(wE0lkRE zyD8|h8K`7fj#$0`iu4|!2pC?51(sTo&A(X28y@4~>DoMYtuNL6%s1Dz(DekMY{1$U z2#Rq3h``McGj<2F^cZOH#qXvm(uScgO~_Zdni9{@7NrPjfymq0XQ(~V(w#x02GDvtiOse`P7ut{_}E(9o;XU^+%qf{KpZ0I2wyo z0P6K*ccFK{h80Z4^_+}VEEr?+kgzElC}Lu__|Yw(A@W)l$$vJx?j z%nuL;zI&qeTO^73d2=F?FOHe1 z)93KFB+qD2t#jL`Xd6ggwBV^b??;8gO$O;H)tM7S1_nj0(g3~0d)I+gJIaAuBHNH9D@aI+?9gdoRIDs#0}M)BLTuMouT@k7Ra~rhfFcdtE?7&{Z)-m zS~}|J!rFe zb?{S2=&@%fyK}2Al)in#Pq`O7BlptXEHPP9Jl3(;(^t-0-VbGk{RwGQYQAxA_UP9r zN!6Oi+xGRE?t7qe7}0x_v7^EnRZDdsf{epCzakcxNxPoQhkHY!NsMUGx(4o@46}cp zYRlfz;#p!9*&4FjF)*@qPydCl#9}kg8=?}ojL^B6E7J);nXVbWaIMDpj1faHF}Xo& z8jsDle5;S-0kW%9@Vokq=5fFyd~?J;y@P6+Te`!$a*}{B%HCRnK~zb0)`vU)--MJfzV1qvS*a+YoP~)jNB{#Ty-5%O=s&!+f~f;kQja zh=7SGr{Z){WpQ*xqvdE)1R4m{MvUF;ujsT z%lOs$va4>{jKgzFqMN5PzXcxKen$3yGxOJQ6M9b8f^l-SaQL~lU3zEt`)P_JXzEhL(&$91`>E*byAt3| z;+hssSBzL0jO|n0@tcD&c1zEl)6WHbONRcjBMG<(5uRt*336pGFxTp=JMsWT?0aKc zbnc;EtCO*$;ZH7DX&G4# z?D)eR2WP=v_OE$|us#>@jxCE8cK!-h_J%rQ{~zSydU_ybBTr%y8y;0VVMwYrVSypR z$%j+IEcR4?cfV(36FN8I>MEU7@-WzVor<1M)Cno&Tr7xNZl*J?jYwmiVb7$Hz zF}cit5*(fb-qGOfWYwhlKeo~shM@_vw$QH(c-P5R9574kBb$GUSyvS5*2n+QZs{GCQyu$FO|0lQ%iD^F*u7$#ooA#1E90^8N>=C0 ziZFMEs6@%h;2=6U!b1-zsi}Hlq{0^oPCf)|Js$TRN&RwV%Xrq35?G5Jj1Rw42p(iY zv`rP(2{@UndG4y3rX(l@VhG5^)rUb`=tZr_R&+UOV)AgxFV!JmV{@UFCJ%hw#7`Ar~mhtykkFny<32235(%wIT0XeiawQNCEq-7%K=ADJ)n+QloOoOur zNMzqlB7I&x0&fa+qt>U|K=Ji%R@W+)7hil%Pf&~V z>f6_jvUfELIA7dlEPImw@^FXPlj&o;Nog4gIHA~@xCH+qo*=Koin1ICEssiuLAy4n z7+*fzO<(JeIG$ZP0gfjtl`e#x#iZQ8kTx?mak>(vE&8I$8huM$0vioYvtl{O3pY_Y zo2dp|2^p5Gg;wJUT4UJ-Gmz9^8@EpXf~HPrhWe}WlIV?1@fC-`v-fg7F{$IG>0${J z{sgM!VY2oW#Kh}0NXPp_DUTk~g31S^$ZWu}I0nU81-V+O`CIS9e@u`{%1{QH%YYgj zZKAWaU;mgwI8`5IyzQ_~2^P4jU;_2UD+-0L1)B`Q5#?9Yn3U)P(eCbDDWh`rMXp2<(EHF; zz$2vS94mOaS{e({(1@}w8>tMQ+!;!~?B z)7Un4%O|^(Y%o7PYrN~2T>dgENSB%vQZWskJ*YqwpQKJC@^<1vFg%8{nuy9^_Ko8`_=y{Bo`umNksjrs>q0$LxAIMiE)*Or zm9FuN0p`JZY1_?c>{ByLQW-(LZiC>H;C(eZwy<;`fkKfz6kndt6+nSbRqSsMfi&XFq{zG7Sk zNEC7o20w3M*SAC{%NbxM_H+Fxa0fUTN2Ng5d59#JBOx12mP|9yafi4DK%G0q{gkCwe|= zP?IEF80?D}DfpntkHHo2coNmnv{#~Td&DP_>Yj}JYDtXtd!ml5zcljy*trG%D`)+MadsFYlBTE>v(9Ue$^oI^fq(YI+9xay6d}uQ z_7R9F4bw}7t5%}6F?DMGkc^CpgvXjW;F^V_O@QdGiQzQe6jXv{_uJ-6^m9E)$Usbc zUlRXmqI%Y8kC4Kot6k(a*q^*o$fFk+VOL#v^=dFCM4NOW-xme1$dp>E!^(b$*O&bu zQ86=NT(xZF1I0o$aJ0+Mq0vqHdg*T?`}Asb_*agDrZBojmbxsKCv0UMN)(S3>5eak zVz%Y?WNB4m0!idBMt43S2lP?`71-w;Vb|A?JPpe=Rw};GS8JWVx4M*ch#*VwjqPg3 zyoSoO)0F&Us{LkWXqRiDwu<8`h1W}i+|En6Ho|k}`bC!;-;6Y6lvC)!NwP*}T)z3sN=#ti3Gd>Tu&hFQJ#Z=yk8?*NrSfO2ZJPYY+ zP7VS#P_V4~wQ6(W#H!)L-;5!nx^>zM+C}7gfrceroq!Sn)@QYlqo}zDqnQFYS!2ML zLCkFoS8R&72q0|T5t(VPsPpM4*U=*v@L$M2z-Ys4`F~!?R{S-dB*wwZn@UqL*mM!$ zSS744{(08Y;exGf2H_Yeje-u>W~&EDJQJC4yY50S!s;1VS!d zg08ky_b zxS5GW*F@K|ck1KG{T8UDAw%!ZC~G^}%&0QwQL~@}7pU^2CUdo+h^olOd~N@?uWKY? zz+n}wxTZwl4Q}pEolzl0ol((b9OXrbfQvo zv;JI0_mH8-E4km-^D`6|!xVUaiaDHrFT}4{WLo}_=DHa&sU>pt+^;b|{i=({IX%|q z2O}hi^Rir8MP?^fN)=0!tjH_(L~$3e;AYo`qUjWPA|G2O(x04J?Q&yL#plsrv)MLs z)r&LId?uYJ;i^OqoY2eIli+cP)!zIGh4v@;XsNn?yBXAk>YHrAm9Q`334mr#qvQ&u z9Fvp^N;pyps=k_UEUDI5WN@;DRTaN87FVo$6Z@m%fV*X>`gw#E+@(y1|B0^nX-qHf zgH-~_c4bdRBrO`&c+DAuid&Sk=q@`Q7Na2UJ&M0jJY3X?Pmtqmj*2%#Jg8Eeu^xDr zqWrveeM9TCTi6xBmHbjWy9ms=X4m*4)mEE~6xR>17zOHb*$MvX+cO*cIJ}JnNQqGv z;&b3mQW|Z+E0UI)U93y4o;{m`RZEYsy3w`hSlY58lCX9@sonz}H1I3RF;c>+qPC~S zC`OfEhFyK2t^LSZF|ziLcHJ)xuT>7so1{AtCaj9OpLgR5PN42emaf%R!rf$xdb0TM zW=6W0^|0SNdp%fVPU9!dx$;Ge@*B4Yc@ta0!8luNy#ap=dn}*}Mm1&Jf2}lEWqEC- z?0C#$ff=H()bE()S0N^vQQc;~3$4@Hwrc*1`|xbG>X7%Ij{t>QMamcw|Zg_uyH z!*TjpJ707BBDojHw<*9}?WDF*ccx$gvcnT4^=^82%_1k&_uceGD>cYzBkq=sp-}LO zfm!EBj^Zv$Q-p2{r=NFE|KbZHo5qrg%p{>ufAm(^J}gly`Q`S+c^(ERUaUp$f&uMGuPfHTDY$K;~4H*n8*iqeZz^7QL_`7oqOGaK!r1@kG5~^0F#{PGN{%!;YI0l0EO>w(vet#_cfuKVtNRanO zsDX)5<-48_grXIlT_vd&b_p~Yi{Df?sUEtoYAKe8)&=)OaL_6(W=?d-af%E3n&!nG zkRBN5Q>7;E?P*@?_-gHXA1p9C@i8C<(Xrk=KS%Q?_oB){omHh4RjtHR@x*-s%$WKH ze2b$0(SEm@vS=Vptk2XLOqHU8YZ*NP7{@IX@)1By0@~am?QClBCRJh77qw=+T6)co zGUyxj-BxnFV0fI-5L4w0`VKa3Jj93_Q~tYlmSiaN#vWHG8wcsqZ5X#;gL&Ui4Ey;IerR6uoaa~*aQ#RjG^br>0t|pZ0_n0KFPqA)V4dr zbBx7b-MhQtkiEUrfv;;VqPwW_;Y_%^emDMe_4e15$ft`HtGiC@j%V^xSrG=`0D^Ri!FwQWB!ZYZ21BFi$frf_#S4mjN>7 z#rdw55`myDL`I@ZaLPq<-%%cgl>nd~&>241d||s739+%#j~}w*K;ofkx4Jx>cV48+1HB*eDDuqEHvj5nKzc`bJj;ZA}hG{uNS(HLF5|r}#N5#JzyZu$Q{E9Z6MOmqcnm?65H0F5I!?Pg6 zMQ@#**o%kEZLa6^UJsm=Wxohi2CeR%HU*mc&4&oAUtRFmK>;o3xWJu2HPu0+-62V! z*O4^81*9-I^?IzlTHS$0Ba%>86QKg}NjtTTQCz-2^J-B(-EpB|^UVZ( z>dFLP^Esc13~{I}RW75ZtrrVfK@D!yHG#4Bp5pq6!UE$K&~AyCA)ojDh_bnj=RNp_ zX-bJa{b+OW5{m8{f%l-$$%AoiSi5jy#jPC9Ug{aUhA^!|9 zADBbBb-p!(+=Z6GDON0z8K>kLztRySZNQV{9Xl;RGs;_sgjG3UZCue8=E!Gz`1Jxe z5JiAJTjID#-CI$XVJF)}?mGj%5cd0Dz5f0#4qx?;AVjT^D+Y+d87|~4DrrKq46^dU zh4&A@@GLMH^{q|u;8&YR=s;yp+sN|(f>+k3UKvX3^JsLG+z^nC0Gh%)uhsg0m+@34 zT@ej_P2gyl;Xj{%h?gUZ3LLV8_5fvc_8=#Tz&f^mGnFpuyEr7#4CHiekm&VJc9y2~ z@&M>>%koK}SLwYCYrR*d6?A6|HXrU6;V_7efLwWY@_o~4j4-$AZuP!N{yh^`>CZa& zx_=Aw1+BaasE}1?eoGzh4s&^dMPL6h^auWmVm34JpLiVh@{fk`Nfk#?+E@g~6^q@@ z+qH@R+|OFv8m%Ow)r3!<%}`#-qV*eFccJDs$;96hKAH}8Sp$a>Z@0pS42=N;ai;ZB zU7A{5j8P-XSR{LHwOQ!>{uysbeNSFfKo9NJqIiR+=9T?3)=-1eP0TX}VA4Y>sJ!uMEKXZ1TA zpt1^lMFaQfTk>OmVOl7LdBS1d=R;pFlBZ;w4TulB^X^y(L4LYrqyKy_3#>`PV!(<% zfA@_i!FIoRHfgPG?JdOCt&dah=z&z7|48~d`U{(yg*#kE4EsV0mS$ypt+rPJIGhjx z3ONH8Hv5UGbA(I6HMH~+o{~m^e&=4}gscHj9xq(1CPt)X&IZ!t)HCjP9NV7Xe@L?g z$e&kD%KW|*eh>U4{i1p4Y@IW9H3(u`L0WS9f8Bn(gk*EZjTyJ{_$!S)r=Ny-j(05k zT~Bsh{h+>R*n5I@o5!wzk+$Ok7U3ZCkZ(nhumrV|Pe2pzB5NHc%~my8O>Ag0$5SK} z?k}!;R=kj9Od`}ZL%Y)(c*>)%JtZY<$~Y3UjLm=&$7;m_H8Wi8#H$zu3)#4%B_l7^ zT$4EL5|j-ioGZ$@ueH0>yp~tbeIbUWPz~UaaaHA(^1Xwg6c~v;9ijUb&As_pkyp|4 z=0t1|yn&i(U5DXn$k(koooK-*0BLu4Wa9A=QiBb%*jN65C;&R;kJt#4Y~yG>(=9Jv zKN}jobew2wd7~XyCVr^~Cum{Mj2rs95$WFtLMm_aYfPO%v}z_}`t*pNsgJMy$5J%} z!44jac#=MCzS-YpMtnY5%_H9yQ5@`&{CN&H+xZj&lm6=T`Ikf3)BZ>?UhHhbfjxOG zN3lk(%m#BDT@#`@#YrF_NONSQKvgp?h~|qYDyL2i`P)Em-tyUgni+Uk)g*DRgc+AV z{)05wL62XI5^llFn97mz5C^x|uk6Q>6C8xq>*saWo;tM+oaZ;s^E8RaF1e(C`7DcP zT8AA!XeDX(&4x}*J+Xrfi8V!7V))- z_4*47S!(G*pXd-k`o2M<2dD+b1^@A+8obwxOevQ+-QsXZOLmjCKfc`R{L;;R5shvx z4N{8%SfG`0^2gi%kut0-R*bjHBAk3+Y7K2}^LTA)yYOLr!*=rycB!Q9D^IB~`9u8r zW3qq~Unr+s1HyS$hH-NvA4Oud4z{|OPccRFYZ4~6=-z+Ach>4ykyh>lPqj|kbcai1 zZZ}Y4Xt8|wnJOke%V$l=N%i${%tkqem|!U1a0GZl{K>zbP#sOI)M_XTaj1Cxl>_** zH~2B9F@FBYp5SFtC4Y^5)jwpb2Ag@k2ERS@9!vXV*-IL+s!3_-d`3FvvpFlt+}258 zixzK-yupE&6^Fmp)Yjy2n5V(ZWgihRji8J2o_8@kr2C~3&iWiSnpp`&P;-)%Ytz4fUXFT)K-0f=(e2XxbsLYfOyUva& zZ@G+)f%&SKTJ_HGPbQXAPlC4hy=&41)mzEfz(j_Gl_`%N2fsg#tvqZHD_%d7&13}Z zdc7pJlCf+>A6`}sId<&)(R(WYNMY3OAI^lMK7+NuzW^F0sdd;a-q`OxrfQ3nRB zLgf1|C$}tK7r*f3T1G+q>m|7)&TtuvUF0hJeb?D;`Mn}nxU7BJ7oL)nxgiL!oiAMB*sAvK@0DZCD9j|t@7L+{zD!fOM_v9l1Fa<#zxfk41r7wYLBd9 z-x3Pz+l@B>lL=U4E3^6Q;5m*n6w7COEW&nAbGrs@T(m<>uB22A6jO_;!WtdE#LPzt>se#r~mwh3xelA_g>3w%h*j(~?vF%AIdqx%Gfq9qPY+*o5qG&3I*+J-z9x8+-|{UW1-KM(0l-OVaNsh&Q0GdXa8{ zmR2VgM=by8ioM@KXkkt@kz{B@qELh+Hyb2sP%<;TdKS8YW)@t+pnfnDDxy_97wvx@ zk}vSJ3A$j>%fO^kt2hl@l89m*%bWWBexXKRYG49UH&Kt|^Q1rgA@~S!gG=Hrm$l>( z+7Y6G3Y$Ly40Lmu{hH#94aa*bmh*}!5%Vg1qw|{!c+qr*we*}qGG_$&>EbmDzl0SZ zSnds;+CW!YHuacgc(w1p*-psLf@j?+OtFcpO3kWboB8yJqo*~qzZ&;a~2{WoLR zkn*JEZRojW7Y!^gJm>LQbB-$mO=SP}E*qErK|??fc3 z5VYV^NiVaxAfy6gPCXZgdd5;&^pGvM6u#g${+6`k{$^U|H48Zn2;v0{=PSliz;M2A z4jT|+A-VA%o@uW9L*S}8_3RL~)DUZ%{Pp2Er^#v?<6XwC4@?uqG7#|US!AgyuD-OV zy!U0?*JSJs`W&XL$5X3>Q@PzSI9DouLtCDNrOA_`Ii0-*$mN(m4k zKxiTD?!cTg&iM^9o_nA7dEWPVFaJm+-~HWX)z4act)ovI#ZN{%6s#UFFxWAS%@<1v zcCM+g4RLPxhxLTH@xO8t%5@Osf06@CJ!gD%l>XOxM7C_VnURH5@fDjo{IeG~CM&M0 zpYHqHNzaLDb}yCqcjI~sZ1(kjZMth{J=csnkQKFBx1nw`ZvkZ?m65Txx^VR}yWAlv zidgKd(dfBrV5|L@jx5PNnB8t`ROQuk2*37gzNSLeXDjuzn^5|# z4?cXu*m!XLpFdQ?+BG)#>Fmjp2VbFlUsdP{S^nj8Iv~;(Gkul+-;pq6e~NoqN*?Uz%nWk058CCKUzmk<@4+CJWZy zEb*i4@ZVuZs+d(9TaI>^Tp5=2{2HnH-9lus#w%1y@yDqXpyZ(z%l6II?>{YU^ge)xtMGKXTz;ON_h^MmH3Jr!J0ZUW60b*CXt9+{UZ^xF2)FLPP z65tUw*|z2r|MLZ0;RoYty?&T;*-K=oef1j0_4g$YEA9wZABwj9+r2SsDIS>`(6bd2 zmY{7|GV{$LWJ3XLIc~4vrhBP$F<<2q3P)1KQh7yngHKuPpzp{RNg2e#Cp}H0D?El< zx7-JcuB;1;RDRm>HpY!{OqbAkscf=tPy=WIO%<6V4jmjUtQZ) zL?dsZm{iT|eBbU=q)0A1>iEGwt=jW|P2PsDYQ^Oc#Wmzbt!wf@A6eB9d#`V{vL3=W z7NX=(lWZUa1b>udsDaI2|6;~ zyzv6C)h%*ov!U{afTsqq@ZmFZtMhyU5-gT2J!B8A{l(m|A$-%8N;<5~uEyTOqzF(eV!KJxV3 zot1AiVr6NbuHN~v^PEl2uv$N)k;)4XO~v_6E{^l~KVOF4j1`egMpz3IYh7$gw-PjU z$lW>%BYNXw50z01k7#9wzUCyB6mYEMTM1tX(6F|dToJ6M9p&uV8*{mh6#GvD&+3QC z#MrcTk0O!GN&KcU(yas%UH(llw^F5gT!dzXM(M$|jmvyOFHh9~@}i{b1c$|NIHEkP zD~q>0(Ust-;Y!8Zkmj)<6+;R8P$` zJE^R<*+e8$iyB6>xH{wwmky~Qi9HK+Ac!Mx zcxgA{x=;f^t}tVafOOZZ(W0D#pxrD;iMtY1>2>2RFNROGND`(I8t1SqftK{pgHeaq^ z84h5ka&~l%KLJ;pgHiUWf#$kfDKhmxOJ%iw6mB(Tzj@XUE0mtPXbLvy+qO54uOeLE zWl3s@9xMUfA)2yD=O~f4OJTlN6sDZqMdYPq}mmuUaF(#Mf$H=91T52?DvCHABk@%Ne@?XKGbS}FJ zD(1ap-SP&nrq0zMLBPY;GXMpe7{xb)@R4YimhkbY_UFegfd2&H&N%RAmkFAwy8lwT z5KIlUl_IHRPbytxfu7CvD;JXCryynGEPD*h%oW~k_rVDmHG1mpm2*UelUvF5!@rwE zg}@pIVXzgPt1D_LQ*xH(2bxdmWaU^^kTyDDQA%1R;%T4bzZ#;t8kjJ1yg8MIyUouDnovF=8bqbqA|-I8;hl`f9+-G;^O z(StKwv{|-WGnM%^6p46m)B18Jqu%}`X%%ltIVcPeHND5Uj`k#r?qIFcdwt zI!s<^c8X+9G~u#RNoH9{$jWlF6D??!nK?p2Ex9=FSrboZob_u z4FBc*CBZB37e5##jMYa9$8LT!P;YA2I_Tylw3y0NvpC@`m{JEcuMJFH;1EfaMJU#hewj(llv~df7Z9R z#95!gwyN4-Kk2IjKo|Tf9!5k3*oeUc|0;$BjO^cqp%YX+N^5c!k47tc&X|%xg?l*x z%iCwk=|c1zN-2a|n%GK=(pvIgYYzNL4La!x&3H`+@3(d~phBN_&KX*tF6` zjh2^EA}QT=VoOr+nqctq1qL6tM2NiBER z=07*rUJ!(5!12@(1q#z`UmYm@Uo=qZiq9wzvo(Y<$^yj%K_j#YP@GV4R!t8Wco+A{ z1Or*gii8NaqHZs}r|H>upUWO%<{?vUJ3+}nn;}ET*mV$y*wRq?H@yL&c?oDdgT|-V z0rMSc)!`LA6C8b(sHYyf?9*EE_)5{^Z1k|7I1CoOS^@G1{k7f(_IQg|?PaA7M`mH{G^< z86JwAYFF#KdaJyW6yWFw7j}+^YGGZ)2$qqD^+`*r)5?yiXz4SBoS$jnJz? zV27&$_xtGP+KLk8u=dto-!>KhCl3Lwcqo+ZHOuC3OoY8v3aj5*oq~v6{-zhSJ=gW8 z;#1r?bHRI#ap$7+KdNfH~z3b#ITQq>g0a{3q@EdE;M&*=%i3NenmkZwAI< z&hF3G+n1N{$OxOB5L0uS+NDarb3rzxWwyK| zR^6&Eo^%6D@?UJ_|04qc?tcuz+S7|NiyBVjfJ&1J?5m)C)_)`EzT(}DDT!gzo!ECJ zB1ouaX`7*Z$LFh(d#qd>t+;cFtzFGp)}xkBB_kxawDn8A-fwfwg>E~3VpnkU6u#1gl3;6GGkh=?G zhpBs4W%u7msVNN53ZfUpPm~6`}A%eriuSImxiFYOM7ws#@FlwjGTLv`8(c{k9 z&TSr+8K3&KIE|{9{YnR@?nv=(TQ3;RWpC-w)X{(ygFsePY>uYlWM$V5eRq_z$yy|( zF`&+NAaeO&%eRS0ryq(BG@kORe#ohz$RoN(xw|x_MKvZ(s_)M8`x@-!zVf!JAzJ~X zTz_JNY4dY4!BVY>5;y8^H;%d%iI6>$ho?4JYC3?@j`dwfvWRc63iwT$me# zJNsNh8T_gps?YfK$?k(T1r19)bwsMy>AaP=ZzrKy&nZK9thPPLvFj(E99BTUA-tSl z6-ExeC0;eL&*l&i4|hRG@^9S2uf+pk!eI%eEL$Cm=6RqUv?tK4b8kD4PJ6TmUZ)w| z42Fx8`O-pA+t5#{@4xBnFR^<}&t$tHvYo48S>YQCt{pkrdJ_ov#(r%4mVou2MOQpe zj$3p$;4^nO(OP-9f1||lf6wV@aiPM!Z&Qx)0(#^~Eo9WPXv_`q|GhClRiX;Z&avbW z^NdfXH)BBofcCvOpIzT3^9l{w%K_k($o+iSZT>@QH}}J$AZ-&s0Z&dvET2wcDd=Xh zp#MZMtusl!v0~)AZ=*D#k-Ep}Up=)%EnC)W=O)}wLc;3;BoF!*udWEC?~V;fjq!y> z^@+WQYU4eI#J_dh$=e;w18X<*pSYA~y-ctv>XZ$AJzCi&@@Ti@QW3J(>(J`sza`fJ zfJ5%vuo&Lgo;9!BHq$EN9qiLCXRm|JP2eqmx=gf~yQ}Jyd!D^lTWy;Qa%&j?uvqDy zjYtLjK~H&L|7q4pVz{gV!z!U3X)MyUN~TdsB&8Y`jB(l0{32I}+U9JSZRG8_PVB`N zjUpzEd-@#A&qvM5N255K_ToT-)Udpt=ay13z7KXvW5axXsc_MX=i#tHhqQV2vZ#yi zSW&YQQ1xn@J$nZKt4UyC{*Pblw2Gu;t-kbGq+~B@6zO>(m5W2GwKK_A4fXuY&8 zz?FRxu}}AqYJ4f0QVpsxe#?qN_3fU*tdEp%cl(|=Z8N+cb{0>l1P%x2r+45lqp#ck z#60`c+xFsW&)o!ZV(y%}t<~;?$AMP84PRh__v2pM#{LQZqxo(?Nl4oM@4u53FlqLQ zI_h}EzbS=8#w~$H<}9yeVI!*##>u%Gw*@rvKc>SHZq4-!R&-OyJa?*MKlMrq%BFLb z0$T@YLM{H}H*P1)x2m(L{#Xb$El|waVBc7{A_WRiDt$UXV*SO@FmM5_u8}p#R+R5RBnk=Z&Dfyl1qY5#7?mKB*wEk zUD~gM-t}?GSihChWY`J`k7u>X@xUw^;a%MU&lADMNLSO5SArmWuoLeL!KtIM536C`PTl zTNk1#8Vl+YVh^hN+%-7Ix(m@xfG{Z&Z|)h!G*joi(8_R~)WAW%?v0qMyZM4_qT9kI zdjNs#`cDEGAZMzeNqswLT@r7fJOPklCsrzYVlu1g%EbKVhIVlCV&I^n+f93hrhrwC z3TUp8&95g!pS%zsj-ZVyV?+V9*Dl^0kKdzrM{HEO4q}2*=-F$N6*%p3Kn?v*aB7OaFgv!gsguFFZ__W(D7NHo9J)0MJ(YHQcrdyP%x z04!VSIRORjZ`%0~YYms!mjV6Mz%jlf-G0mYNR^4x#I>HFbxU?^b z#x1UCN!Imj@Kt!gQhKv3MUD2EPdADDZkO(r=+jv-pQ!bG0iCtr7D!U0L09eB!4IKe6>`;omid(fQ({!=W;6>rU5>>TpXvKesKdb4i8M?Y}I#F^Dar+Eo{bdyF$ zYP)(j9Pmu?n*UUPX~8H9L?m=4qFRZdVU9;T%Ab0Y(^xe1u>-rY3h2w)4jPoY4f>}{ z@ETO1^%P$-fS)`bV!91B-nlpE8AqP-KMG|q5Az`S1EZY$ROE}**P>GTnL`;5Z#sDV?v@iJCy3>_qDH5Y( z3J0Bi&$rZ7wh$1@{K$5`*?n6Y`+rhc6?K;V-*t1`RMG*QK|6=xG?54VP+f=GGWqS^ zAEs5B{Kf7YqsB4G&xLNZM@os=7JY8w%xT<4jauseSQJ zZWPhjtw*0(hHVXZmb=)PZQ_NnXA}H<1>vnbp{o1xrw}30N4RqWR@UXW#v08li+{hb zAP??baABKZ;X`u!@&Qib8Y%rNdoRWszkSOMy9(^%%~bIoaIIwF&HBr*UKVN-DV+hj zPytTS7W4&P$V5m*%U+yc;?{^wA*oUlTh|sHsH>=fpu#6APRUq^sjmR>$6t!9_t)S z@lh-dURF4SG_%zgqVV`*oy1affE^F}YdgN+tA&4DdK$L~#X_r?&o~_A0K*;lYZlK3 zRx7v&1d>}EA7lg$s&MD@g1$H3@B-kCldn8^R#B1@P76{a%55+g4Tipw4qTRFf95tvM}9=Iu&a0@q*N z$9ch3@*l@4QzzUF!c>Vu(z*IrZ2!DJWqo{kBu(Jm-;uT(e{ox5OmfLA z=4Hfl!~bdkV}RcFWIWt|^uK#I052jQOoLKAhav9-W6+)U{CeF_XL`S6iUW0X{x1@{ zL3q@0xUpo@bss$3E_SG&x+t(-aH=g14f{rduORY3--55R#xCq}s8XpTj z-m8;Dq*mH~?y?t*Z&L>5M1xEv=rj8*gfV!Tt3rcu&b9Sim4a!k7o(2)R;IEW$e{A< zO3jXKe|4jzd7IhS;-cc?z1e5zPF#szOSCj{w?1wTn1gW%bfd>Pbs_}ze`X#7i>WuB zfGSyO9>;k=_TgZx7Yp*7h|*dpt{hhh{Ad0(P=_IDK}n=LODyRDiFs5f`7Lh-T;)41 z2|5o%U5QNAI^R!q6IB1_v#kTjljRMV*2a_Ch!LPJyOWwmBFSuj?S_3Vq?=D*D@+(b znE{CXwYRRwO3^p55vZRVg$7+3mVsDQrchY+_AGoxTaoYZx3E4=9Ff|@kr$~m&SXWw zvQz=FBT@+fyLlWw`+Td#%IAZEiJLyT)3Yxp`uf!< zJfNY%c#@uT_?Dz#20+2hV7VDfZMbYa!ise|gqLp3Annug(VGd$ZTrD-c(y_(s{(J- zjBUkGfARg|z!#5ZBfe=ZyY+CVS!6a{(qNLjvey^76_pGWxY)#QVxK_^)!HUaV|W9>^hzwt>U ziBeD;loh~&yxVh&dQLLtvyF23_`TV`zAJIF6d0468KI@yr@*f0a_1~UE5arJrnCBg zP89Vwo$c*_EwPy>YsU)&Sq|0aRCAM(fH# zkj@{Mh>r99x#a%8X|?$A*&Fn~dx3>!r9SbId_~~;j6DQ519F3^KHcYu;}Y~f3{mM? zN!TTPl2W> zlS>I01cTguZyVQ13}q9VsrbfemIPLV)--Xxb5-^P0;_$fqTkv)Cb_dMpyPi6Fv<{t z{JqDq$$xil#&-{tC*3;6lY?_b*HJ#kpvuT(&>U7dHq8tJCn)0wRH!qh%4A8<3Dtyl z*6RJ?VuqVezspw_3e>+og`4@4@{X!q=RwD*p*t7Ie@b4nQzclDd`!q~GMJe*G^kG! z?~0(}z$ClRO2=L~8PR<6&;JZ&I4JaP?BX1n$(R5we@7@3&;!On&~Lte{^jAa)|o9Q(#Yox0$y{T$qzBlNPIG?9(p*?d?DD&O$C#b4xv zf{rD!uD&>29ILcK6-boS^(_DuPkwbD4OL(zizcg(HgZDkW!hRAdH@pfamcOpG<>D) zPO@Bb`!C{r;VO$4OI`7(i9%O@CwI_(C~3u1%4x{H-QK;-_RCmvMEeD`#pe@sB!5?T zP(ww5b)KeQBF$_dPHOSjv3Pzpkd~^`xOAUK;+CNIr+cs7iIz6{WBuft3a+%pLFAlf zz)&)6>k=>J5RTZ_m^jSDOSu))v5M{jl`E8C_nNkr2YA&=6N{6yxvILsFwNaC#Yc9b z;J%=9m|3i8HrfR1lU4|75es6M~Gez#^9()eA9!w>GK$1N_Y0$<8s^)nriDBO=%DY>edc=AZ1EW4&jJn@;vkO2&K;QC^nr zfLq#!(pi*G^9m|Ki*WH|sV(Q~)_q|cE!g1K7cx?=0a6<*MQ>-Z6eLb=4v^x$iO@RI z!&&K#1?ZG+tVT|J7g&u8#q?c4!mpcDuhsFcrpf|n%Srt z5JD&H)BYtUBvUpzxVw#3fM!-|D#Rs2^H$L?A;x@&FcVsoY05Zp8KxG0jBq5*W8ZI^ zPah+E&Wy2+^ZOx&8BLUkAxg#|f=;qdJ&mT->y{gtmM};Ikp?ffI?vdC9ANp>gc7y7 zpPVU4A{Yv2tbLr%;@Z`dcJ07pgJHeu3n+`PR@?Xc;(x+9TopVD<{JJ#Cq8!FckEwG zj@_-N3!yLFw6`vbUvE@E-C1NpDJ>>(tXfOT5Z!E~|EyE?@T>e88nR}t+76HMkATeB zUUOtgm&G67v$(@Ge>|0@RpwPYdMKjXEIZ}{;C9!mCtIL2zJq6ep#;6UB3*olHS;sx z^Pk2f{cYn$^ehEeP`~pus)7Wh#`tyRvJE2YjSl$$nA!HK;D$h!{tP%nmT3BGSM)9x z!`R*!W>}jMA5~-pa%`xtYbvZzFz3ea>aPFych*Myzh@uHk3yO2eagBU$HPX<0XSXww@d1eC2=-Neb*nA{?15M_S-J`|C`ZB+? ziK-zdrunRKmI33gb2PtZTXfgZ?a}tn`!>Hmy5TSI?<)=TVUC-~p_g$&I11&>=}Mi; zq8r}`+=QoD->9BZC^Aytw2o@>HfDCSfMc3Y%a3V zn1z6ff6O!$t5zY=W1zh>tSiEqK-g9}G-ye$clwVn8a&P{>Q<==T%W%XkWYCd*B4~J zIndL5)~Z(m54hA;Rjt*<%+Y8ah(+Ca3l}=5F15Kc*KV?Ss+@~PB>5vLP6V26JF=G2 z7YC*E^2DJ7I`Ga(@Rb3Y?B?clvGBXyo1@pUJnsMdJKO(9_aUvGGx9%up$*T-K{?xO zJHOuMAwLsM+5Ba1)W3WS{r}_tdg;x$)lEBq4k(~ZlnRqI>O?kQ5*WH5o~$M%dhyGW zzzCv%f_G6KQDM-pXErN+NG9QH*G%0T<(lI2g?y?Mt9XQTe^(36i3Xhs{=|g|DB9c2 ze*FQ$zl4$u@vNmtBGpD}@1Bw@Hd`c6V<2Y+h8Q1O+tFl{O?wDx-7R2!a`)yk-(W>K z+nGMN1(h23TvPVr2rjgH-WkR9(rnr^(>;w6&YBIMgfh04Q5mj)SR>e9q_1UXC;6@_ zKwUbj7re5oF8=B{?%@R}aS5PV^)*G8`C%$?d+MaM{u5&!1!BnO_vbbuVWy=YmHgZ0%x~ zDJ#QXX%wf*ggEHP>Y-%iE?Tz9!Q|RigfD$z!Yg@=;*>-w^+3<%ODZ6UVY}L)>u(k& zAj|<@#1|$N4qpZd!F|ldVnd>#^(f6(S5}fS)?@R#V|b0&CU%Q4hSA{{4q?_lgi{GR zvWVqlP4U^xrLl~sKC1#cQBqA%HDjG90_tmyHug@Zm`U{G4ybptA9jv@R?D7r;wu$I)Q7~$5K`^1QFdhI7l;svM+ zV-$Zz9*#7Sr@wCvmxChK$HoYlgK)UP{-~1t_5zHF8U{-FkV~ngIjdTO@FWxKE*uY0 z#>%28pIRe+p3Tm_T1g?fDyfOqX2l$*q0uXgHxFgU%!_LdT9A~bMB7o!QVc@@;!{fr zSxqoE{WsNA+@AFL4VmP~${y*tq973A_oI^Clv>w4{TR*9dB^ru9%(h+3-2}NM%SjYNjgL)jUMc&eVm^1 ziEL<^k#%IP454_prJ!woM@<3T+gnv`)-cK}jqS3rzF_-_%w^G#(gM@6~^5-8?WCfyJ zTp=&~IrdlFW`V{5*nlroerD^C`W{BGt-ih(me`D7 zD+ectbj&pC(~NbQ%b+Ni%rFBI?Pdw~u8`|C(N-fW!;Jj1<~K#u_m!%(N_?++wP7ju z#U%}LE{DSewx3qJ*NdBH)Kq27%|*Jz@3bATcpzi> zO3umYY7%=-bi?P)%kEdEq%bQT4~@cq`rUW~r;%cM^beX)H~8u=hjYK*R36}4Y|o6o z`(1B=4;RJ$*kQKg{OrG*D&Q|K9NR@z2;3g7PYG^qhi!9`{Vt( zgk68^(^{9ot&ff_Cr>{rq|>e(s5#P#ox_xr?yDE%JNCzEK+M0>)5O_j+#HBqx`y2K z?=uS&lH$lobLCFGAEg(xGi7I{U^%a-vU7W5Va3hf?fXN|r+3|w{nI~XH@Z(sjyOZZ z&iv>g5Dj%^uC1t;v*UvJqX(1GrnastLqh!L3RR!+M+xzZ1uESgds&DpYp-y|!>Bv6 z_Oxax2(ZqR^&=8{pI?t#pOM=pW`1N)7h>v&rg+EJedbA(6^?|aa)=nK!3L|vFH9OH z?EhoECj{7yy2_IMU%z7*c^GFvaTrXQp~C>(I*09Y!yM-E{^~oqlIa{dy9Vewf6!S#8SCLcs5tQFM{IeMUrj~i`8R(d@CK|k z`ZQeraMCB=6IZvfmLF*ej->A~j7`pC!qVwuT45BkAWxOn!!rjR9>|CbXJd0wy7cQ= zqrSyY6`U{@%j`zfzdC$%+m|0uCO70*L#Q`vvKikkG~KROx6bRCYFL}KT;H@igWk`P zQ$Z%+mLelReR(+oM?aNw9Z-6GMi<(f=Y*)9H0Ud;p3C|)eN+nGVszk)_r=>%esQue z7e19m@7s5tJM?7a(&<>`S!9gm^>}&LokbNg7sAc)_DQAd0*vflS~JG^i$L%F_``g$ zQ&UBxt9<(pT1MT~ik5Iso-!*Xd}ngPxi3-_J6ef$uQaJ}wmn9wlYFIF@wC()_5tsD z2bVllr~Fz;w-pU@98LCV^kFXOKDXKPZl!qC`JGi}L7x!gmWE690M>K*;Lvdlq1=X` zv#QuhNbOR1#yyhcw~crU#8ly}ao*e%m{7F%E?VCgYpR_>bIo2VrBy8nk@ z&fa{JO2k?S;#HAV&4jr68NapLW6Dc+a2pToUvIrmJgY$B|^n;Ri? zXYldZ0eMplo>`xTX5O1kkR905S4dbXhj*sr%r5#^SKlteoQ)|OwiGI|ELU=@gO=_o zoEh&M*&!hj$t{G4Uh-$?Z7;BNK<5~XzwTe);?#-yfudt;?RZePhb3|MoNKb_(AT*n249ua!+n|z41>CcwMsl_x(x?R2VDAVtTY3_IXKqN56bMp#z| zG6-m^2{oYwXMAzPGC4Rj{(e69O-zz%(Yj+xk3+pNA@kR5IgY(~RbtVSsI@db_WNQV zzfZZmok`8GYmb|wlR;5#fob{%$FD71v|}rGZkjsS`oxKfm+Y${KyPs5oY%Iv@HW#n zA3JV}%o-n&A5?p-1}Xf7y=Q9aZod4TG^?J1SbDXKRSgPeL9B8%>yfWU*ydkdaZO#0 zsdRR*5|9itXD&>3d0xoY(Qmarbfzycqw-R-WgO(vxQ;=)VRM#eVc`l5b;9y?EHc*9I&Sr!#Li6ds zk0_7g*|u1j=eBk0OQo&I>W<2&foVCXVaZiw?aKL@_4W9fpctro!zH*NV!bLS^+oN< zhf!V1%Fn1A>h*y|&-Zc)ulsXywq{x*q0^*3;CVNdBLGQaEK8qGL4^jU#y?Tgj4j`^}K4 zX=;?(lHuEASkaBy&rL}sM>ImHE7w-w^qT{YVX8cS3+x6TvNK>4>POmSg-`}p6^!7Ww zT}k3FL+d9yE9AaQUh%@N23%Z%KT;#Xc?H(j3^5Z*M$iD+TT7u-<)3NPgq#6f#lW`V z>H&lwy_{a8e{7XrN?2R?uwIvnljPld4V??cGh}*_NPQU3L#+}gAbZ!9^GR_F2Ncdn z7b@R=bK0U-+;Vvm)_v#mIo}*#ubNlZsx?QJ{HrF1d~&4&{K80*$jqdjT*HhM^(bBL zAG36m1!dIx>wLqF*T&ohewlJj3V4&U>aFL(9?3}B4i%uJe%?mjV{j>F`3rro2JvoyQkGyT z47c(rOG@;W(M2WLdRxx8`$UI3p$z7`5>R#p?-gTtCv?~kXI1n??$*1UuDLsTY)dF* zm+IxP942b7oKFE&txIV+8HETFY!3kKq}uV43k=-N&S$Bf7&>1Kql${+Duhmwa*~6& zu#~)?^$GdY6>;Uns%xe|7*?e+lRi=j2nGRh1BGCTLbekaAjUls;QbYcE|>C%6+JqY zFYk;z1c`7!tVy=(FJ5(RD&Hs6t$?gh*yjnvk;H~LlJQ@E{lrb%9^)y1kHf@{N{qnI zBcUXTq^JXm9~+(fukl|9_goDrGTSw9vWxAq02gGh+K}f#Rmb#mRJ!r(puD$-_e)_b z&!M=u>A}p?=C3=OYY=NaxgP>ly6xNU3cy$9Z~~U&lV9Q_^rqg1zKvtw##Qq~X+-m& z?)7aVybm%u$3sCCY(j?3)Z0X6+7OCP9eyur>^ya__Id=3P;~vfav`fv`&I1{c!i`s zTJ13UWt70Fx2n39(_4t|BVF^~c=lODVU>*-_8Z+QNQQeni zORPYt{?>48`H?l!ls!`7nHuGaL(O=YfxPqDMU%u-hW^9sKJ{9*X{@qV727!hQM?a< z{*oFo#F1lD<5|O&iI@^7AP%oq;~cZkHO(8c_0o`tQml!kty4(&EC2kf1GG+eD%iT;|>i!S5!e>fMEtKNdspAReFx>BBg(nDjx+y}>^buaQE1pCNE+z&m*nMEB`9p?s}Lnfyd&0( zSA-p!mDG+m$McI9ASUYyge-@8iV|&;cxo4Ssng}klAGfda2=iYbIsOuGu862qv84= zGDrpTWI_$Bj`WKh$<+GdFRmjh0)imv0R4!Yua@m>!@FtM1cv8yx!(5~hjTq}E6+#a z+y~T`fAAmSFPL<#%Zpzc<)zR{9i;~4cCIB!AvhgsRt^oXKAgbDEBOZ- zBh7FU)k~KJX4XUl4L*1dPJ(aOz{JiI9cBIw|7LsPM0-CWT-|umfayTEUH;{>x>& z(|Xv#q^~|b^~clJY~!48C0BaoxlnVDA$sv>Nu@DcrjX+#*GcZf%ftCLL^RLwHERl; zgdmMvkg;n!yy-$7mkTsUk;~{MZ>L|*3fnf|=u}}R=L?Q;atFQM3^Q%Ymp#8YXGIYY z?DPpRcPT$T5HJ{|xhkhy-0I|7hBf4?QlHGLJ}@lL{rGNyChl#0j}_jfILJb`%2r9o zn!N`DF^5Us9r;{Qg<_nfE(?GDrNDUdjIah_@#<=#=bd)mt63Nquc1}n*_H*rT@LTX z0vE;NWeJA26VVQJm9X1Z0p@47<(T0rs-E0z@VovZBe^C;GJI+BdL`G}OarUx{yM8t zN$V5))#=Gdbj2lEogDL4%A1t$ytb5=Q8aaq6uvdz^JBI%YNB8zdhtab|jL#JyKj{?G*h zw90yUWk3+}iE99N3L}7+s4uWU-)%lcIgxg$dqQeky5q{Rz4HG^rOCq;7^n^Dl@zd@P{uf%tTU=uL*{x87B=B2^2!LD?fa zs56gd(9`Bf6raIbZlA^39DL{XV?&GHr%i*N#XQ2GueHYiCOOUQWbY{m7wex_o_lPb zHGOKR!|!qpW-z+N*X3#U3y|CVGV%s{_Z|xL3G( z;oV35KEL2NemdS~?uLH=*cGn`L8NWot3|Dw-=lO=Nmn?zD)9lnVR~nGe-uCU1LMO{ z?5gSYZ8>7Sc{nX3>UrJSgH^os*^XBJAtD(IzH>n)mOqYXlg`cBw-0*gx02KSn>yy; z-VpU4>1!#*@t)7A>nP)#4544VmkyY23Pk@wT!GYly9o0l>uvRIa}AO z;d@TWA&Nsz?Q}$1NPidS8Dkmv&%7zJme296+qRod1t=nxlPsRY|; zt7w_s--|i3MMX#3^)c!4UOzbu{r0Hng-Fs6hX^ks`|&Xdr&69#oRrML!wwA; zRlro=Z1vs=IJJ(i&@w3e*s9#Yh$Q>@`?yjJ`etan>=W~>zO!g-7=bQ5J`{A;Y7FD_ zFw+~T#U0ads)mHsB(?Q@Oid~iXo96ykMN0{M;@t$*W;{9y^$fp&)M=QngUz`%DyHA zx9eNa<0%h=XNb%FyXGo5aylyU<1YDca~2Tvk>M4={7w=5CpCA=kqEj_bbw2aI{wzYd|KP1N=O6@arhh!%Ec0lg3 zmHQ6fUG0}Uq%;`YV}w)lELcX*a927*Kb7l5GUOuz7WkFjj{97^0HN0zLa7>L+DE*A z@tPkQwuF!T)!};+V~CuhKuP0AAD~yZCoXU*_zX!p^?632UnH}&Fdu}k(Qv}|{J-*N*OPwp2Fhltc>m8j0LHJ+_qf^9sAUQe>}hfex7 z=;DLChRNzZCA3Ac(52Dux?WXQliIbiNOnQz>yNB1`R9?0nO=_s)N{XxL>uS#y?!#K56T-e5r1e}#6C*jqM) z%es*%>0W@*6~D!nd3{d8Wklumna9kdiA^;r1o~T*E;cbFOy$vcdQd+~RJ@kR9^UwL z9(83eJ52jeEZeb*v!-%{{>JLpk1N~}G!3#hfYa|j*Ar-!U5Y|GAhvPY@(GLqr&P}~ zC{^v_!rgmjXv@Ch^I?L!%#cs6RbLcP5#m%!HqM*I8J-DTdhw z_C(rZz}fI=z8zZ`$Hb)((S!^;dwumTf6u1M>2Lg zJglI77Z|XAQ3tPC59gsTY49-lPO7Zm(FAF|}MkI+%L@ zmbX5q;3E-dwoI{#J%xj>f6+tcGKxHBu??#aUjYe3qDrqNrjqvYo|w!=*K?oY(elrz zI;43dEBOAhH=fG}LBU)&YccOnC>F=Ly_Q_hsYz`R=?ayUiU;RXI=AgIblI%{DFq zY-A10C5%Rt(S8G64b{YX7{;ti*P#Q|ez@$sM4bJIkle-%%8!LNR)(wF4re&P_mv@b|GUN}hd)e{r; za@!UoByO5`I7L0F` zf7>k|_@eYe*z2m?kGhE^+H@{c5`>Za=H5%Oy zJt=0!54K=jD9d$fZdh}_db!K+ASc)UbEo%;T||kcxKzuzuw_2Wx;}bRzg70vLP+3Z zcet=oSDw(}@5HaVQyB|lDNC;N%Ew+O521L-8mX}+mol#}*9slGn8A_f-*k|WKEW3b z_~s`cG?97{uNr3|EmFxQ*2SFzsFM#ud*v%62OEssB#QY5-|6i)aGi@bl)uw{M_^v1 z$I`G<$pXO&(EDg=I1FdIX zUy&S77ccplWvvO?An)iG)+Pv?5L(2A0)E%<^yN_jXEd>*U0>Wqvj`I!8U@fgP|7}z9 zK#fpjsCo92ht+Q?PMkR52tP&nA%z89%#}Y`_5u2NQHION8Vu(C&=NiyTK`_3dzbnN zB`zZ%^C_@FvvlUFvPEY_q8zQ=^_cOmZWH}dUe3I|F)?5}lk5;xAF_Je%^ZMcikdvA z7sQ+LLh-xg;Bs=MZRq}*`CGV`Ql5SmYkmBJLFW$mE?20^l;9(Z$N1GxR0}n8`xl*u zti96K`la@vX7A(r%b|yP`?4;_-8Tp5IMvzFyY#L1D~rl=81@s)tCd9_&}vyX^o2=J z-~tpMkbgbiWJ6P3J$HX0m2`%a3tX;9cK?k#?e>ZPMc#WxHMMnZ!$Cw8L{LGbTMp8} zLQ#5AK#)#Ax{Y3?MM?sKfQo=p6{PpjO9%u)1Vj|1_k>Us2rUFD2@nGB#&gbn&OM&z z=QqYT#{2%^N660JYt6ORyyi8pId}i9tks5tnR;+h1$sx{bQlh&A0`3#+i0t@uHUN; zw(Bvx{>`L7Q-YHgs;2h3Ic+1p6y(wt8M7L!8OKG~A7%3}XH=o`UU4N41}gnyz6eyhM-8luX1<78FgY){ zzmq<+zd0931xKvO#vxAF8JIZDmW`65>(adsT(nhw^g+qFVgOK* zSa{|mkA2LD{~@{NoTq;sx-{2wOE*4_`ErW9O%sh~t!(sME8fCrK*f^O2G&T|LQx~ga zv|6HL^P_F#Lk_M!WHLAmm{!#jBU{sy9~V%Q5c54ZSC}uxbEz&W`nDKeeA6SL%_-@L zq1rCvk(lm}GB~9U)-G`6tAv8v?kVP={Osh-*3LZLGK}C;BW9Sky3~V|1n9r(y+tx|KulXL*EIkc4 zC~Li;sGsXLt7|p=Az$;GVS#P{o^j_xSV7bzL4LoskD#7pAx34bG!g(;ezO~XgO#iR zxjc-DEgx_h{-l-+9F=-S#>0G&G|&2;vsAK`OQ)H+O+Hk@X><~AzTs4_TE_G1p#|=$ zL#YBMrx{BRzyEa=xJGLwic8#t&tfwA>1@F6&|*`U86%DDDgbX zNx-eRcv zs;jN>b%$P6YPGMchMdU<|skwwM2{7uwPgrl|*`N2`3IwGFxvzqZS)WHOu60 zwN^c0Fw7yHWhsPjFr~_cJf~KLi=sBs`2{5SG^^w~Q`RP55mb0#08#7KI~14f%1X@81wl%~a;OT@9Rt{rA|Qc?Gk~q1Z3K$I+#WAL@BWz^ zK1|~5e6rwN`Ru}4chdZp$Qqzh@uZ9`uG$$sjmhV|S3|yWs)4E^fy5@esSs1=?TsS_ zx@`Q%W)iq0>jz|24xA8W<33+TB2&pFGkXs1^3GzA^rvy7fh?n=P!Bl3vba#)=lo>7 zei~kXFO4 z*-Q+wg;Xg5too|DxgvCT3on%;*v zQOh)cz8oOBB$&-4^x}WYU|vXuEk3fkx^RX;B2`YElk6)VAq0ch4m|mZy8mo3X%zM9 zD?%#7&+IzAFzpdOEmy-wjX@;TB-2?~d21#;e-+~6$v_zC6bl{T<>ThR%ZUJA%eKoa znNpmb{Z*A3f(cv=WHH4TjH+@PQ&#Xi7IwV!!r+5971r(C2csk#wtN?X5bcB&kd8DZ zfxqOF*f|EZvWg~bdR*)%nUC~36$o`!mhQrf&fpfFWzT|znyu+vgWL1Q1BM6~ zlkFzHbW1*VU!PH|7&x^>ZVm=4G>V>!=wFh97A`d`GM;Xs_yeF~ckhtMPkN4xo6j7I zyRgU3#(=>z%@;lvGaaJZfqS7;$Af7tz^Nim&?u>sDpKf8@~TQAlWiCC~jSq_o;}`WNI#$5C4NfFY_aFW)r!?W!e4fK(7Ca zZ$W-Wrxz0O-KwHb_u_DI?LEq~?qAYD8+To=3G%9n6md5$)my1}_&wu?XGSCs^P#7} zPn?}Tr58JoIAAYQk#TKF<~#1~vZ0+NBEF(-$HDJGOTi=gu5yrUX5p6)+uD4_K6ArP z3m0F%al=NqtXE5sX&%hClqn#kBF!2r*}8mRFh)iA{ndwIv_TbGLXs%>HHhxSFN^`u z$^D|anq1(f)1=DpRFIGb9lM01f+xMOOvv#qtb#od{+1*Sfpj*ECV*6@$FrLkFy|zh zdd2eX)*Rc5-q7oCLr+P+NmJ(B4gQ>)z43^JA*yL7{0L$oU$>F%;ri^iqsPA{Q20M6 zLs2|1(l?>Z@4qsB7jOKjL+M!t>F8Y?()Dci%>AO*StVIvW()+*Uaq?`%5Ux9&Cf9R zZb7PSI0USWjlLsV=*AQ zfOSU0z$tOdTH%hM-)b&H@6)OrHVP#AqSo4YUe$|fq5;Ykdw0t95{fQkfhtUKL3`IK z-q)m0SU(Sb_yo2VWydIRY}P6;5JlzXRZ69v#|VFXaG$Lt=}8Nz^u8WxI9_zy_SNv8 z`Ljc9kjy`VGQ&%UH~W_iO7P78CtpBZV$K?In@GL9CgKtEAC+2Yd^DgB!U9StO-93x z8Y33Pl^8aYEwohLU)q%OHs3;N!vGljZG8O+B>$H|Gu##U9>&J_0 zA4m~y|D&(^|3}Bi(Fk(+RW2?zd6z#{K}esTBL=9OYoPiwa=F(tq znF=6i@&X&%_~G?dZ`x8frX*{uNaFS!DrHX;{8vwUWU!CopXp8z_;e{5=oLWBb=sP@ zDzF<{M3_dnde)EIKHHH=lL^vayZF}-LyriX0l@P^*o<+GUYy?Trzf)bizbHWEATVe zT6?R~wy!(>B*%1OLAASE|8A?@_XFi4MKKDjmIuXRfTl;Ebhm3ITL0%IO7xHO-wIr= zFzD>$cPvx|i0IRq0=^$R5wKyCrsP<@wThcsMG@#44+)OGY>k$vS*!7N0oXoo0 z-H?e);4x0zt3gL%ejt=sw@zHyF0N@A-dTqNO1o@J!Y?Dbp&g)0$}3Z4hQ(`vZnm#Y zK;S}K+Vmq_<MX`Ja>8woA2NR@KCpHw|M z=_cLBk_%~zDl2n~o~e6$@b2sGeSkWw4lv*54b-#dwn!emcK!N|GLPPG<$*fODxPxK zV|sR69IDVUw<1^A<@wU5xQ~iwR~xIlqbzpJE|=oA+gBTGIRbm~T`6_LR+KyZ38QSm z8V=>vQ6(_K?((}n42tya>-Xw&;f^@^BvkztBr4XZu;Yn>-|BXS+<7(`=^3m$)ra?| zTM-HGdM0osi6o5-QeJ-2eXQFXI^9sF9yA&7~O>`{CEHbh&I zPd7|sO~S5f;4mrEI3I~=gsSY^TP%9G0q_!c8<+%eXja8D4HeCv*rt5E|ELlf*;VdM zmlQjLd@yq_E6HTGAoOc3Lll4JL zLlA-Fk_SQLl~<)I&(_{T*8}X4`XrXEsgDhJIE8=9zW~K+Lr)`U2{p2q9Ee>hYvH;z zB6taaiDBm$&excEAc)ksvCTOvYo`IXX>lZMv?W(^;u#5dQBvIEL-nYwYu^-uxZ}VN z))VioK!QwO${tDsc>US)@&*ytn9}Q3^$^MlDGf7w>pZGpCz)1;d&7KxBMV0NoGkwsXC^|yw~pH z&|=NvDxz`Y=PAh*QKnZfw155ad!1nQXFk9AP~(Op69Ey9Z{r-fvFvGrm&)oyLs!Hl z&*qH`<|k&Q){FBh1BQW)Js*FO{KaeEJ2EcTy$-(Q%vGjjPa4ESW_cWdKD1Y`IZR z_hF$w&$iI`!ZAj}eHS6A;P_SD>ppI2F9BW+e$05z$craJXyD1GlbokSN`I}~^8GFu zQt}}twNbKKT(|@UNla^^{zZ!1#u2Y)9uw_&)I~CsWTK&>RW5rjBvH*p0gc~fkBXS%YAOFKc&TZl|gP~TO z1Lj;Kgj9vG4sM7!&qdqa5xG!oLve*DyTFT#U;f&)BB4IrX-O6qHeQ>n_}<^7Bd-OX z{@RfV_HK#d9C*oJ=)R}12CxBqCWL_XC;ozI9?REqHRJp!XIZko#ALDs@Vf+2h&~+J zmU8|R&yYj_K~CI)AL%qmx<{EfsBA?0lv# zip~$@JTL6r{hE&l|NWoG_?PtK9fnU3X2e7%1rIOaA@H4Ovk0TcTE6qIyZ3EAjMzf z`wpc<`7NM=y-y?|C!GNWRg7E7%xZ1$YSY3elQw3ThOo$jUt|naIq|hi%Ny2O5fTgb^bI^Aso3jUimPW`CMioIl$Z< z@z`;bkawp(}M; zOzplsOfGwxJvLTZfSdOhxjz61JS4m*hq)R<4n%)gax zD6c(j;@Sx5CYyy+jk&vDSBg89xC*cnFdDn6l2^ttL%6X~)==r4l4;RGK#6X1^XVQS zqrA@2NFfIapkAMkIK{ob+L!IM^-i4`tl)g=!taM;S3Q(L%H?IG&c7R23sU1gqTj)L z)yr%4qqN({o_+yU=kh-=fK3?0nQGbNi8#y@gOcsdlqnG0B&GZvA$w+l-3j55MpI^& z`oowb5tG?us5fkva{+FYP$20w=-oXvTk*%BPoLKdnC(LClf~+(UH`O@)s z?oFUhm8ql!M5%FEARxIcGEQ)QInrRSZZ|HkpdeTj}W{b@a+mN;6mVbbzK`_9GVI7N2ZPxcxnUE)xG#j8O$`;i{5UUC|MxO1l-r^-+k=Sz{jcfeJN<--ac6S`YL27Y>k3Z3%FgycITA`_^sTz`M73!wvhVE%(^ zu^*53(s7)uYCc6gD7fyfUh%L#4)%sFv9(aF0)6dzRJ<04HdEx@Ok!)y^qMP^z-m2J z8DcdokDP!ZnFaUQGqwrtOo!VwSj)aEgcq@bkehn-LL}Z5BqQt(+5#k({&+p4=Ey5} zvNgyLI75I0g*@4YqWA!a3Cx~+fFn&W4gmtVspNUlZ>C#+;2>_Qi zR-f=^J~0;T+wj}1_i7Wkhwk?ZwJLv| z#z&2-zdHX*s?jPb7IO($jSc5FX@{u;0QuF?u6A#HOfYATYAvid6`DNT1<+o>sd~B-|svYsAqsKiCcR2i+jn{CjcHaqm@{ip&i^g zVJErdC-l?h`EG=sVTqss`uz!O9j32)fKrEJNO_m_{1d>{*8rOC#%zYIJq%Dl12nle zn+B*`Ogto(pM_Eg8x6k4v?VPtbU9ANo&>!=Qf5B2^ej+9)^q$vYO6AEa3fMO!bSU} zU*v@40X}>TAvdwI`^ls%Q139Lz}25Pi+^g$P|&;*vdS`P+kbo3iOm-hh5W^%~Pf9==^Fd(1K3`x3Ss#6Puqyt`oOv3!|fno5T z@TLgvQGMB1laNcJhL!=UGh0E_zp_Lfm$aLgJG^*pT>oPaf1s{fr&SDN1KaKF5#6m_ zl&XjvP_8!gOpoq64T#*JVqQ$0%jed5UaUiQS0y-3$TDS3p5oJb{!z4L^CS3O9#Du41P-*qaxWUZ10L@>y}6>Qk)5nA z++DbJAF=9DF2QoS=z(^{^CS6OT_zLPh8)T!Qh%Oz&L-^Y9RPKww=1UP-=qo@TPBbc zLb+u<;Q5&)_q$k?G~@IV)rBK&Fx`-r>IZ^A@0N&pjdhcxA)!oj|CTq%^vFeprr<8?>afegqDo z$kxG~B6VHrFGjf+bz8(=LvTkoUh5fUwSxvXefp0gW@EwA>7G$El=4z;(bP@kc31x17t}m{QyWKAnS+< z_0!tGf#6#XKVu@80xuadAf0FOffA${%>Q?IjnNrfy$BfJO z4~;F%%V5ZadaL4dP6F9&1p#twcjNjW|N43EWdC1_BkK=2p6vAy|(!m!M4Hk4@dX^Qa!EcBgdYC@k#n93HuIyTs2fq zLf_<|D09f}FI#}7_cTG^7wiSX*0%!p^6VY#8O+2Z;bidAB1iIEc0n&8P+6&o!tNNIe^Z-s_e4-c_Zy+`;t+ssEqLNXl98UHwHzA z-TAHE3VS~yWO(Z8aS0ARNRb-OV|Ci)KZW!yJ8U&paHr5Y$)m3Q?cBBdJ=}=JdF;@w z=!tbSmDRGOoHju&!z_C;p2|nf73ybJmD3`vaI5rXcL>U|f$QQ2tIeE&W$+PCu3Lo2 z>E*rd8I}PUrs$tpGbXxj6{~W#QI&=zPz$I{QYYX36+V2$TA?aj&S_XLOOk=*d z6WuHWhps!1=T}JB)wJNv&S!OWZ9Dt8e}1vJ>ThP2%v@2ox(BPU_!3xlPSPs>Lr6362t8$iacZGLH@YI-g#cLh$VSVb}g~{mffG!-OKD#YQ}QaUzKW zvXeifV$%aXJR>O3EwoCgj=#V%)QTM_boL{ZuDt>+F>1awzPict=Oe!ju97Sb`eKl| z41bQ@I`5>!7*lt{u@^OWxUU`30~@_Gi3?RfJ$90bjiq;~8Rf>>&-Jc`8gC_tnzCOJ zzOcr~2k-2wp49jfN$zhuCt>$ET+_gtb33Knl2B*#U3WV}0lOEc#Y+6=x1U-x5%DA*npozd>{ z&6x6ESlqQB^C_;_T7j7n3j?EOWw_bopbFi)nZRV7Cy3V*m}14XZC6gK;r z>)ohYDrD&~4nmAMYS^MoHd!lP5`&U!VclQ6?#!U{tnk;-tXrr+XobB-hn^oai}KYG zk%pSvfnq)fm9evldSxg>uF(y##$3pMupsxREiddmvbmq4X9c-G7>k~Wus0eW_{=r) zR!yz7>Z}mnU1B3rYp~r5eDf9DIE*gwyG_$2W4bw3$y#Dv`b@xd5DV8&b%wj+r<=zl z<&z<&P?Z(Y^C-g#=3T9xhzC_i>5Jac+cQ~h@!4Ciu`w`=`P za3O1zX@$0&>oS{Ib3Lq9&tunxJ@)cv!={Kd|5^rJ^9CHst3vb~%g@4xI7MBv@|vOC zj!TrY9LtCS*BS8l&&vWKnI%jPyG`w@c1oE^1a%<>S(#>M&Oma#E^?0 zmrgZFz&$tjeD7_Ahw*yeR7PhSt3%`M0=Ce*7A*!i6#=2Q3B{tEHp1qE-YntC zDGxyTpLE*R1G1CgX-#i+!FuV`lYSH}z0}1>zrjk+rMTz0t-U6kR4_j8ss>#m<$nKD zX-Hf>LEfR7`Im3A`~lZSGP`i_8MGOA8R-qpGAn!N%9#D;nDLI^Nmm zoIRtVU+Nvv=hAO%fLV)^eB^pw;g|W_Z36XM+yf9tUS(r$C2@Sjb9?(MZDLoeSB=A_ zO&wa|HRHC>P|WmQKvMeTR!xuC8CClyx6zVJ*MN*XA1hjGDCEbHFFwP*Ocs04bJ0fx zE8=z7=kKL8evOFG`0|9c1?jcpGRny9(ZeiyM=fst_4<^#ecBODmWv+_ml!;IPlNs# zgI!R>qkWJLDGi8)jb!3&T^U-qM+}C;_b(o&`A#kw#Br+55okXcx*9HqiXJ;|%oGXx zi$#x@diD@s6M)l~JBa4wc3S1PQ5{{gXR!<+B#XEiZ^g~OJKc@1g>9HYtiZ-qM}#9l zlMz%PSHUfL&u97N5vgw1T&BdmQ!RzB3(loeZ_n2busOibp9YHuaMHjL`E07}fx^CQ z;Eztqd<@2bsGNy+K9eWeU~`Z255ooy_TVz<#4Pp8p0QcaXt_%=-3fC>Sy}K)41Bhr zY^qFJEkrH+z^{J|GFj;X#{&97aEI{TY0sdo!pP~tO0OJ>YR9C;1QFOhO%L(Xp7ERx z3m@6b7tr~m7A*rnOfs>Y^>DuFfnnR_Kn{Jl|G*Bs)pccP+{4P~jzOjmX>{*v32CdL zujzs9o$-~3u8EB|6zp=1w43$zj>vbg3M0L6hyZob&#YdEX~dKz|3a1K&pHoD@qU6i zhxr+)P@wEyaME?|f6S=?=LncKm<1lyHp#lc#KHc3A*_D>i7{JNcDJct_H?mHW_F#w z$?LSoT5VyAKN{eM(wwnJLPhP6$9cQ0Z{{+IGMogu?yx@-QFH=ax(sPT=z2wrWb$%T zxbIWjIPlf1nR#Da<3h!fAj5H{H^AA$+rQ(yaNWIR&Xyxq{RlYJdEsV+LF!K}0D`lh za%vV)(#I|N=d`8nFley#UkzP4NGR0rAZDT&Pc@9{c91h0=j)6^b4PXKERZ6+{JVCX z;6>rm!L@K~CfwonhIMhO*7KZ@AcpJM6moIFMHWBRv%9;QT0ZbLj1}K;lr-+bb&TOF>3Oc|R8#7AH93+k=Q(lWrr$ zNamX|i5a{1!1wk+ol_k!rkAG9^Jq>#yUkr^VHG9Y3Z%I`=id{Z|8sdGrx!7=F-U9| z8$Bi$eIZm-{ogwsMzUKzh^fN@HK z57$0Ddj%JEN7rpHPsA)=laKQ_3%C6B?YEcI{s}kM*TWeN)i|MP9-gN%u1!z z!BUe1K0GWOxKzR{VWFmji?tzs8>ql$w-vX5^(`$^?YGqnBt&r5+O=N4jHy|Bj^zGA z@WlFMtvD2$t_DTvGbSok8N6G`||k-6T4QuaH-6 z8;Ky_@`onV403&g>{Zv{5%C;(iPXH;Jb_Y1ILNB^#r}>YOzmBhf<{#xmm%i|q$KKR zvvONk1qiY3l}W+)!+*hf5NJB;sqIn2ptv+5^BYOX<1}9}1;ro*219ei zJ&OB7vmoyR+FpsY!iVG=p{NEM6n^|#Aa%0qI#FnN@JyqV=T?~Yotp3s6FqfQcdJ?=>DDV6&CelC;3;y z;Dd#|Pe_eHq$2mp;DcP=E+$b6+Oa?J9tZ?Gf4G2L?2?rxB=BHqk=yIoXj%@S_fy%4 zxzAJXZCNEJCK}rLRoYw)?CQNqX3D7Temne&5Z@X1gh2!-sQZ$Na1lU!FY#VS+Cmvo zezw}|rP^4Odlb0BW-23Sr>v+$#%VyYI_5}GtWD51zrAe8%p+vFXmJy?HytDFlD3{{ z=GR6R)cCIz@EQv8lgF=AWZOd^I!FYyMx16X>#NnV2?Rpjcl)0Y%<9dkIjX-9O;-VW6Cf`azOQ;2;t zKrfq@a+6{yt{{X*+ZXMZvlMa^2Y`o&3eXg^+SiF4X8dEBDU(tt^FQpv8aox#S@LM$ zYH)^yRa=+s@uP;w+UShI;xC_1nh*Pme&IDl))^oPR82-RM-7OafKJbzL44_jIY1La z{;un3*^Oe6pjpqwXR2pA+QDv&f?O8Y0Z^RB_YMfL%YC|*LeBE3g7=y%XPY;d;e|lhLJQ7glyXow&Rwf-3;uJ}gPqu$N;y4cUNF!G9FZ#Qe_~L>ToA4nc zx;0aB_iE%#uDcIri}L^-^CFBZP8}zSomX|vC=4mjaEK{z#72f&)cHb^nQQkOAEJ12 zq`2a_^1i3u=#5adS1EsTx#JBoB02>-#8p!YZqz5wZd6z_#EIed=^62LRq8ppT5;nh zQcWHbSl`<4s8feW>&l?0jCE0dM?nGqQ=CX>;1-e9W;ltQ`&02?Nnxbkxul+Y?jLE} zvb0QqG)A50N%M&V)`ro^;x&f{BZKyjZo2%3+w6)WwY#XN zCJIk~ys#4Rocc!w_Q_=BNRdFo#x-pt;KcqkJ^h0;yfgNnqvZk*z^TX!gtPxMLc{xj zYG)z%r13xPyi^|odcKQciDB7OfA9I5nE&fO0@|h-8>&W6o#I#KUb+5P>adqum+!auS-kf4Fbt<(Q(IncTU^5e^Y{U3N=hCU#_n?687gggvP9R)fLnjwq0 z5mdC;YS^VqmyU;3zX}z<^!k$M1>xw>(41*?gc;7*==77*YE>>arx}cHkv0^3TNNQa zKJnjfBqY2i5m2klU;jw>KAUl3bf^2zR^UKavWHdwb>E0~?@Rxf6~un7>(xId1sUEK z7y5TUzyEo0P938mP*YOPn$3*(8RXUx1X{3NxqOu^Fx>CYRe-&md=?=S8tA)R6f?jSyvLiYD_=34Palx))UKHTap3-GU@z1GbQF!*q{!q8M`)Dx<* zxk#`>LCCyCRvs>1`F?OetEBcgDW5j2$@$2W=pNeCEl$A_A4ybBxhD?nCcr*y4MwXR zjyKent?L!`oNC#9^kfX0nt@V_HytLftmhS7J6qT&WL91!;NB-~(2Z-hgCAjYKqj$+ zhhf_E@rl)8LO)7T7I)0F`CO*a0JveVz>J5&eFOk}w5-9xMxzCwGu zQ1tBBXi2VLiJ$eAtxuKjkPZ6BGfmUkREvNADEZGH)&28F??OdntnfXO2JiujrEDO> z!YO~NyI}B2l&|v@(~{SIrCqDz7lgrnI-qtkowTOGVJxP)3|xylBCK#B9)(J`$w&mt zDjW>`egKgYx09M?6~J|U9mmv);U+*FG!EZh4NYA5-BaAj=FsX(#{L1uX5%Y+CY+&H z8_V)atFN86K#PuCqcg4VG;4=2=H33!jZV{b)wA*7X$;j4=2i!^KgZs694xnx(DJYuLl3Z(~0yC9)7zIbsyLk!});GN~_0r z3MD}gfuUR2KfYQW2rhkmq{u6&>^13>_N$CTpbmgptoc0=-6Qjyw(Gb#8_M_Mv=NuTA36G{Aw<8q!v-?Afa>Ui z1I*?`MBIpynZy5bFIn#EoAQ*-Ogkhi^0TBt|G;#`f`LMrQ&M*iX4sNEgvBb#OuHA# zS*aqaYq=U8tF7%G9X$}@9|!LXi1>r-^r5$Sv(z%T1E1}L%N~1)AN&TbivODFr!_#L zRwy8_dl%(Q4%n$4d^`*kyc~^>&xtlsCa9I3HS{t8~yR3mImS z!$tL9G^R)&PN8qj;S|bfOSuKLr~B>aul}+ZBj(8;HSDh1@XR+W#~Y?i#io;n^`wmL z1F7kh3BBAeD3yNH(o- zQ#v-Jo|-s(C&$TL?~~02y7wj+?9A=c8aNwWJ)7FmdrPjgjK}TC6uV65>J|MDGd;~~ z-k2r;_(G=ml(3{*0VLASC;ZJNe@Y19Ym4Wq{vaubMd`9g7aiS;bBH2YxMVeR;!2f&^%_}|61KWjT~XAi(*ri>FB{Y zm5o<6$X2vO8o;ZCuNvL2dNgcFY3Drf_oP9i1Po!T5oIt`Yw=X(jnD2u_Knn)4XFtL zdj`J;`YcTCEC!KRHikw>`CC`UPJ@3ZZn278VnHJ&>M6a!8&&%NGg ziHg?J5PW8mD#)3bQKgaVXClnS})X*)u0SM@4K(6Owr!r<+^0iV~^ zIGuN?=;?x6+J(@D4LpZjV+smQ z+vWLz${+W6YdOmEJG2Zr*GYQXyq!IOn+MMwkYso=O3-%xSBBod8l;gBVUMW04sf?J z7=($8Ab|~ibqZH@=ugxhxNC9&&3?4F`SOekfCJ;zS)mEiE=(Xq5jLT( zp0KF^TN%4XtqgG+2b8xn$w9x5Lbl#l^0J|p+Y83Xg({&mQKix5W}~Es#N8N};U4(} zWR#`uV1lwuV^#hUcT>Q-9pSN&T(Wg1<3J0_d`+=sz5fGN*+-bmXm z0JAf1C7?Vqk=3#FBNb)`Uy zmE6lzmHpY52TeO$n^+#t-5d9@x5k{u*+lKiUKl(gnAEP?ZN6tUn8Q;j^=V)3l?Lh8 zfa~zsj_ZZF2f0vH3`=C+gMQ8 zvoLPcoSdf@`v$O*C*+?!=)jR?9SI&hAt_dS+v~8nSMze4sI&<Y~q|#GZIF1e*0aJe)QaJKA@1xP!#TT99+=%pz&C_A35?^8Eo&DmVl;n--W{l zv|bGhAalTrrd#4+JWl zbGcaK5s5PREny#laS1c$ZhEGp1-Zv+W>Q*3SX^}6Y+fpdhk^R@e{Ke>5X@Bb+3uxqm{ukiR$kal^=Ed+ z-J!4s_}i8x9vcnWncEy!3?KLC8hI7|ZYm_?`$4kPnguQ}*Va{92_ZW2RE*A)4Ec6M zsH|~?6d0@i7C0q-9@K7d7rj1aSTf@7T|p)amAO&hdSVooG$)jcJ@{jO8yrh1q4-|0 zeYqc!SMz8q_YwFWmm%t-9qgc7bupv|^$C-iKg9=?lRrNj z9TKuA^|VmeTC{RtPBMxWYziu? zbRE9Ivqz_Iog?*~Q4dP}7JNL3HB-7fgKzj>wmxboc$}qf+50+b!dx?)gS$y6)a82* zW_+m*q~o~nbXC`J<7I3!>}U5NeFoQ7-*#icR**RR*KOVJHqRO`O_|5)qz!DY8}#xl zG;_yHdJYjGyO)P$Km~MCEUnI7Yx(8}6C|07+3xDnAbtbOYd|2yt+8!+f>!Cbz48S~y9+($C8H z2L!jOp_>1;<2lJiabdl#utGN!)V&5Q9U8d%AqJZH^WK6ATl%6ap}(keq!UsIbyGI2 zst(nm0B@(;L|{IinDu=d+LIL@49YjWOX2%|s?P9!?u|J0+NsfCzhdm0Bjw_=XV8VP zv^$Px92sR7-UzyfP`;;{Q_=5bePUH=z6BD{Hid1q6V*$lIKb-iKC&Sr>-5q5TN`J2 zRGp#1_q!Gj5>sK@DvtBn87$e4W)f-#e#6kvK>$o^0s2sn`lEJNq{qrHTgWMe%zJy~ zW`kMF?4B*BHbxivR@Z__7dvps1IiT&JjlKty~ zGB-*wPLeV2MvSx z)Yn0Rz}3(|hj7g3uxj`|_osV;?8^4TF30VGllGkA-p!P0*Z4Xu7MJCdY%E!#>>j^8 zqglJWhhS*+psAEcko_m;ZZk2!vmv?rEe!qt+EOTzozv8ZZ{#wRokfuWB5o8-`9nZ; z@%taN`NdM1e*5+68?VHP6u*FRMVk==ToNpkeRJvFWt!}(p7>q-IOk4pjRU^M6Xq%>ee}v(9Vl?J2fTr94BrS#{=}*-R?F3HVLv z%LZesHf-#6IxJ6QuhbdmIY>RL_qR(^qMoWZ`YH4mpkcg>ATA*)9Go`m<6kDJ+&}Rb?CqQ#xh$YqDiAEs=)y1A3peCgde=9?uEA zovoD->tE+M5OIWy;WpFbb+xyv9A{B-cMAHciv_DYT~wz}R*oklJnB3m>P|2il8b!t zt{Ry)uGBWI&rj_eLC%ejMuyhR=tBRQakm+R6Vj^)((f0w5<025 zv~q_bjVa*9(nSGP^uA`>zkF1G^`9<@>3Q@};)jjpc^Si`vSE_`#n2wZ>Y=zct=Oj~ zjuV6f8$eWRVIn`R4){=t;{~8<`N;Sl9fN`5jwuE(2V1a9xBLNYmpJ12u6atY7jvKa zjW{|EeKhmbi?2t7X|OS``zaJ2VL-foA>{1~NjY86aMKFE;k7dI@2KxW#@W-V2yLzG zWrtN!8}+%CsFZC?2(5cD*CeIPdXAA(+N)@p+?z@b71+G++m8%e?jA;~M6L^W;NmF4 z>)7ujf`sZxBYTQZ0A$odCA8pqaj)`0YZMq&Ywh1~3BRG%8$ydgx;S24m>P#lNmu6Q z*2_4RwcJ9d^n`J1=TZi<(qijJj3&8fjYAVy!DX7b7ss}wSy|YZUOu&vH^}U_ZI(J~ zWy|2seS4H;dSlr*iKVmx01^@(2d+D>8K&qBR4=V-Tas6__cWK)dk1O0*HNP*?NA(& z-+UpD>JCJm-tMUV@&T5gQ&Y6(DwnVOX1?{``76w(wHChNAFM@FRcaHsfheISi5PM! zWWAV2#ohtGX$E&ozkdFf_xERzt>@cpY>wR+{M&#N z0A0c@L)I6#!XcrM`5ZnD(y=GCkKG#&_zZUo-&tjkgcojwTqb&MrBzO6wxG$XJ}?iZkA9-`TN?Ki%cn-?_tPVmc^5!d*i5Gi9y0(^~u}W3Hwup zQ%@p3{h$4+`PF+eqmJuL$%aYi)?ChCG0pS#>TQ90-EOaar@P|5{#oz+$E|{2U9PWv zrZ2eq`t$GX8Fv#PgJMkBb(+3J1sDR6LX^Jo89hC^?*P18bi{6i24ZKS4;d{;% z#9bnCSNkVzU`BHp@;BvIS3V zuLX8mSOQ{C>G_}1nmxDv6?ohgI5x?qygK}-m0AN=IAtS|z#&i2=?NP^5gZP0f z@3&b0UUd5Pp#>rk;|0Qx{^J0R#8r9fsAqXIsAkW*`{tO>oH%oXNvl8>h?r*mN!gzC zwd(gpwzntuRc77$_BTirJm|V&)i>w)b^n%H7rfwTej|Fp>gukmR(iLa+QoKr>+QZ( z>cnxC6XXSf@TX8aT!D4~7n1w0iFpk4#PmNx+1veRz0==%@$Ayn4X3ZJyYzke<|!f~ zp{~qnR#~&-7OptA_Wm!>sJE5YJw0>N9ha=u7HtSuyuxeVU*_Bv_uP3wNa;D(@GD<) zcXz#QUNmdWF zcEuB#PM~Nmf|iDV_D_fOp2<1zyJd zRNlVo%Zk}%xwGou^jW_vm|qw5?m2Agg~6d77#_eeYv9J!9|n+J7Ck$eCe548)*bY4csc zbC9n8`}<;X|E}rrbwAGt|5^h)$MJsgv$M0Gm%X|1aGB4{M`iEt#a92BmA$S|-EYo{ zztQLHe&=*8(m+Kk?w4#0)=_^o12}nmX=&c=ZMj?P?SWUo@B4JG?su+v-Jgp3X;bue zo9Emx_&@9E>FK|(pZ*6tI1x442QQ#MlWbB^jz)=e%8(jSrR9yLl zfSm!ghFe=QC-YA?>bLpS!9P9e&C>5rfT2_9+|Kv4{@K!qk1zP%tVz5m%&|~E;iB+{ z7o05@#DQeQ7Z#@%+&~g&yrO2&gb2SYhQM=zW}0VR(WtN7D_{5HAg{dLo!b9@zrVY( zGWgwtgU$ae+xg|ofM;=k=&${BQvIJ$&HjJCR`2JTe{P1*UGuDSb1ZjS&YuQy|Ax(N zKnJ$$13L4-B6R_$cpwS19N~l|Cmew}3RqNcN<7RaZ?jTFrITqT@Ko39|6>ncUgX++ zr$bQr&RO&OZ_3`@0;aRFH#24fkE+@KW!}Fn)9+6f(U!a0IM)>9a)QC5)vzk4OSJ#p zfpbz=LlN%CNlNUUpyK*PC2_&!!q7VJ_^6@SQs!uwLel+ck{ZoXBfLQQ&)@v5CAiBc RrW4dC^mO%eS?83{1OQPo3{C(5 literal 18482 zcmd42Ra9JUyDf-Y1q)EP2e;sb6C}7>;TD1gcL=TtZb5>(Q@8~yEO>&uThQPRUCH2c{*)9z08*ChrodpUdijaSFZpM07&S0J2ye0?ULIr}bF zd=hcAT%`!w`Qh`fjY7l&_6#Fp7Oq#XSuJ^e8b^=EY;e8($Is{6xGl#O z%8#6G+wSJ;(T@BVvJ;B$v*J&`R;S%jQ(YT0yXtaby;sn_LKkom-HqW{BKyY{bJ7Ce zE6=Qcf)gXp?#GJlA|ripN}CguX4k94(vEFI^V z192Sln|w+V_~zGws@(SL`>OTZf9(hjghJoAL>H3(YvS*QsG*_3|M_-^86Sv6@~mM`Y#xe<4t^FOPR8s z9PR6sIO%xY9`#t^Clm_nP|5P%XS(g<@S_&;IKF79{`+v^1yHk@6^S|)j-n&>i}{_Z zCNLnfkVV_)Q$)=2{r4a7n^Vt~!#~HC_zc<>L;89UQDlp^hyOf<$>ZJSc9zd^d>|rf z*b)AYLF=o57|N6octiz7aJ|i>a!1KUa!V?hNQ6(iqw+DcWsdX0Hz6aV;y6>U!RB^J z=T9o=QjwTC{g#OT#>ur>Es~IWQm6Fo=uj*E`GCA<^K&6&uGu0y#Z%U}uG?0mLw`2E zeO?Bn4Gfij`!?+1VOK0A`|;2+AuWx#Mg#QT03%eISYnj{6^l$2H!}O_qPW&_oZ@ea zSj1R`^j&F4iw>V?-met$7x@gaq6sqM@>_rOquAX?hL;4<&BQ?vWM!)il46D;CCZJOIEltS3 z8(;BNKOJU1`(ds!NUI{uw$0+lB;;{)R^k7=>6}&1Qr>?4B=f~cDuzOczF~hV5tY(c z{c^g%OQFU6C|&1ko#l9`T?(5qwUU0l`jN@9&()O;Vbl)W;`f2=p|6qCPW(0#Bj+&x zl1}7m5PI#W?M@SIkYV4-srej@dKCDZCA?6gpdJ@~u3ftRVSeTnI+S%u$YGQe7X5W7 z(cf43*#Cyp>%Kkq-CdOK7YZp7e!F$z6!B73(6!(tNy(|#T0FxPI8)Gfp^oCd`>OdZ zNSkb6b>&gj{gAZ-IODS&$m0}GR5Z8J93Fgw!{7>Wu_LUMX!ZliSY@!bY#EqkjY8o|i?8)jbI0tA@$jJPqINA4}Y3 zQylZDU^_>mJgl~2(lGmK6AZEREFbWK+$)~%`Mf5q)9!EV2_ncq#;-1S@NY7L?IH^y zC5D}!UcRssy8f+e$y*^KRL?&fYW@=~QiWbI?s;!P>#jBTIZ6KS8X6*95QJ+=DA?tn#ph% zNGt5{B$3AJ6Z2BivANR9^z=^uM83Fc*#dTItzXYuV&3-|I&CHuvc`q|{jJXcLiX>J zPY*e)MhQE*+J)YjUKfL%vO_nYYI*F^oM()^kCd`GLm*wJAGIz?=OIpsf<>Z_FNF^} z9p=DBevfRtS++A(8ng7A)^UZXO^jWsYu>kbK%5Xtzq2&qb0`a!%M4kXh-DQCWdyeY zVoqpZ9Y+UE`D{h9MzONo^geIOq-x{yX;vsb2d6dSXW0;qB{Gkc)885}9^YUH%OZe*8krdRsU4@BOiS0eH zk3yLk2ypicAE9g@Bl)ouCHB)mi#ZiW(^G7_^MZ$0O{-^8>KzXi`96i9ax_>HlswL4;N~_*6heN)qfKe19#n)~wpO;VlSOKRE5^pOX-tcxUj;lY#*lbg z9)i7Aa4@T*h^VWh$$s71>wq7ibgWf7+g@LCV^0i%`g)a_r-HFXOERFP8pU^@&{+cs zB$b9mkHf=;_Fp)`R-wolI`x4rl%GvXRXsOi{ir2)9AnlG^^pUBKugm-E zoBq@(tBUb2-LTW{+J=Lf@)JEU_&U@L9Oi^L<707kIKNNPdcES~drS?(8C&)YB&N78 z$`hZDQ$KD)R%bbkrAf5OoB;?GK9*7X_+VOmG55-0kWN7!`3xU6J${py@Zd6G9xyiq zS+rARL5N^~H}h#P3iK6UV3sfAjJT~%E=`_@yfH+i<(I=BFu zXY1Yi@Xf{YR2l0Gg-M|@1t{i=>Uf9i`(us(sDgFoCvWI9gn#>C;#IXpR}AFlvf95g z16p6tLs{&&Su{G7w$$hw{xg{4Cq|@jimErv0CFv7$?w)T==KNwse;e}1=Q3MCde^@ zGyoQz$}84MC{&RSgRp>J>!~$;*QbG`q2Tx$Q+V~B?uF`ZX+C_qidZ$3h|l%89*SvT zeTDc`g`ti#F1?am^5J3rjtvI_IzZ~3qeZ49F0aPZTPzoy;a?-}J@0L_Wyb*(Oareq z_hdtKn=424x(qwzSh8VVtpJMf>#hI(>Z-wEpP0v9&O|}Qt4iZ)8gju}Qyp-KO@0k6 ztKaRgF>|qF-)90*W$KBzHAMa}3M>G{_f@7HInO&|j0?D^z!s($GAiy<&|M3_az2QwD5=*8@{cE4aL7QDB?o<$y)ua;w`u zg#>()7d9>|2t9;;q5*_>feS`tJMLS%C)X<|F=FC&q0<-!k>dt#k?8er9z*Ag8)ZZK z?#-81aT_4V{3q^a0QK6D$B=q32H@L+awJNsZkHX~QkYVf=wflKBdfaU&%W&>A>5h( zan8v?(ahJ>MKFW0IAX+)nF)q|KW`$fA^OTCngjWPu-zX`nq|==6LW?OBq<=u?MILv znxU~$f|-Of=z<;uEi&x*I6tWjh!l@$F?3W0VM~d|EnMuPQYM${RC~|Rngym~pMyTjV7>g4M%Ot< zR^Zs5)bOyLmc*%N`^V~u^Oln2GU7tL3X7@);$y0BPZr6ZF*u`{3=GauM*JvR(`DHS z@weIiCpO@@jbp+k?4~sLg6!-Vm$C#r;>WN4gooL5JZ;Pr_yXLbKFVAesCw)2{DcdL z+zrD_e9vAy-n;+M{&=-G#H1)I``Zaa%{)*K4(0?#g#JXI0-m%F^R-$6g{hTt!Djp7 zP>IO`T?MVZ4-=I#9K4cAVs}?6nhj`y`}jP{=zsv?FI4Y49DWCXJhvmSb}LZ|nDpEG zZCbtkgeC(HD4n$Y-hM5zb?uGx61YAEhGlv%Dz^6Z58LYY9hZLtfR37#WRyh}7qx)! zQr2S}S^y}a<_c)=LkJ_zf7-rMR6kjcVHba~cWD)CX8%B0s@0`ee44#{hLjkP4&AtJ z3Pt`rgq!g0TbALoee)ICToRaOfC%~@ z8QuqLfZM&@U=2LZv`32>ySML*;Me}(gG{#H@#csbJq=oGRIbj<$4A1*?tUoB~sMn(x9IxEUrgEqyMeKMSOKPg;U3LcpgLQ3INkDy! z0sTCcvU1QD8lLj8=_n&ag-+_=HBg*U$S)hG5A?>`rR?LK59v|aAYJWBqx`WN?d21T z!3hhaBMJ1#@s5&Lv~Q!;Buu)Y(q$t3iim2(mP61A)#{GZ#$6$d{x|U!+KeNq7MTR{ z#*>br(nkvo(;`!O(tyru9N4(x%#Rlh`QEBpK<4g$HN)71h%G~Li921?Lhh+3l#Hsq$NsQC~{Habe)o^c8LpncQA22#CS?yoON`Ca9zJgK?2THakR?GttY^E^xJYMH;*z)wr znFdZVkkwT+ybgpwQ>p1N%x^YW9AqQ4?R;%?B+voh#I~Y4+sh{^e=EPhk1=23alCU< zei~We7$cvB`^jj4_$_yF)ra*E15hyexF;i)&|o`@@-6(&$k&!^6(apQt~JEn8a49W z_WFiyj!A@g7lF^EdkDOQNyfGa5C;d-AriwbwaEQ0OQPkY#;e`1l5ud+tmKIJM}u(} zhgg!&yz|ltkVbG9>)eS^=j%yw>za-M1SY)!m&kW(cM`Y!`M;MMlanB57Bftiv;>4O zygU&yeH+~V{B*zn>AXU>ZV&&Z{@(q&b!h1Riq}D9)5l{LSa~mC#VLj4+|~$5gS#p= zXc9^-hsz*cQL19MgDP7vl$4_(`yuxv(Rq8Q)T%FmPT@aiiNhxGWjQDUij!Kdyi!`5 zW9fLSs?*=n)?4CWd2`K1@Cphha!%>1M!`c|DC~bxmv_3 z9$CK}7D)?_k%O@yl$KA*T>tVsFR}1}V{;q}JKraL$5VR_QheCAw&eS}4~6s${w09^ zG96v%MoFt4qHLS|#@ykv9M@}NgzSbuDfT!ZGDFrM3g!J{Xp*G$eSH6LK5-MD@}V@lVT`#D zdLgc!#=YQw$TIa-M@6tWPO1d8dVcdiTv>tZdMX6+mietcQxZeM+4jle*5GObhP_pA zfA#Yv~f#wVfWz$~ERd#xw=E3en;rC$OKxy}UJ#fE0PiP!}7oGmSW^-V4C+7E! z#yFC&gMSop3XT8GdGL-Iw$Ih^xge@ttjMhnlkus7Zu|Sn7W6;dp*);S2=eFZ1}gE3 zZhTC4%*B1bAC^?d_@<}tC*x~PS2-9k2=OrB~O^$XAZ!j-*wn{SQ}-y;FCH`h|$0_MUd)4ZqTw25jN0!m9u?~`>$R)1Tk zV}&h3r;-*PPEja{0~3i!{MemfL!?yplS(Jc8@?moOYDpPvuJpkhlXYuiYa?pC-v-gR( zHjl$qzOH0n>1Q2Ip3ChuzNzw=^X9M{@4YKJETsH)U&dK9wFe%$XjNqY%yd2dN?Dgg ztJFPV3g=(!w&EGGw0H0Aj!w4Blw*oVEA^WSY_GHS^_G2BC2joM?=H^Kst{4@rkwJP zrJ^B}QlAz)4p~FL`1nsGX?mS!UQb_k%}tlxw`T~t4b?q8+yW*bP8tnZ*C_oCN3C-r zjDIT5;Y&i(Q&07d^xM+?dW38@WdH#5!%NG7aH83o&_rqtv6Fkjri7+)W~L4iKy;Wj>jEx3+$6+ z-M9x_q^2HQ2XFS>N*XLRCs$W3M`DIC<|pbo{gMy zD2%ThI|=(eggCpGYgXry9c8w;m);>1@GO7xnp|ig3=HX;uiKq^3a5`%U2U>k&Fzrn zAsZP|f0TUj4aQGq*-Cej(@h)1#B_nnELryw0GIP~^LS&Faoza@}H0(g+qD zMexTJryw38(Cj(++RXs_N|~r6C`0(fup(*?5o4Bj+t3%Rky~;9y~SCQ!WEryDj)oY zGPbQZH0ISCaOy0k8T!u@u!#&A5_GI_9RmPbiFgYR4EBc&UzaUD<16yutsS6oWFmvg z;K}nvz)04n?_)*R>CunUNI?oB8Y)NS&a%5`=K^Xl910sj9YVpzj8#+K?qic%8d3O~@he4#s)08r09ZQ2e1GOCsQB*uc%Q4ecLq zYX#p*u~tx>{oW^aC?CJ);+o=f+~?hMcnphner&yuB^X<;<6!oh3GS9moOY5(rE(4& zM?}TGvEh1TqdP-2Dxo9)mRd-%irI<&j zqk{(bu&qUA8gW2y(2dci)$9S&ZEcnW9%?fE+Q8hA&*JRzy$Asf9cVzH%?DTSWR%6{qh;*&FMntnI;D|N z={xFcu^%}2xl*Bqrhz;(sbX<9NT_b}Gr?IFvi?DmF$}kk8Kju$420}$nQjF~U2opD z2Ta!!p-aK)Vm6Ys5Th9i%-%J@=3$8T7Gtfxxaa_lUh&sCqmUNT>1!C#0Khp1RqzmV z85`NWRvZl})F{7G(Kpuzvo1TU9+^MaxMW?Lj2_e4++GDUOnaaKE^LvZ2;5G62tpfF zIPQ&o8Ww1we&~oJS{V3BJ-@y=7kZYN6^Ln-t0{+7Ad zTNJsZBfh`(Y0z!(jxLS{0(V@So4<_eHC}xjAmX*2Q9r^v5@XjfDl~$Z5{Ha@J0{{N z82e-P(^orC1Z&kr6@thRlRRh@B!D~So1Z`H(w^U$9h>$Hl@6sE0MsuVj}SpGUO+Wr zy6}tYBp;2QmG3>7#z25MJj~vxKm=qF;K*{zzptuS@mXi#{4whRt$Ky@0K0sKQ0JA|Bv* z;M2>{u#S)bgh#{Cf)T6V;G@{7^_axqz4(cNSMZe@0WQALBU?xdcQ*i^izPCpptM76 z5r@$%kjb$jpoWE&2Pgk)(+4&4hbivy^7I!4YU^a9=K;BXe}N_gCm~SV30Gbd#xbQ9 zqABl{E%G9gfcgo9RMe$A1rw?54&Dg(jR{4+`%!Uu=DoST@3K0jHT0?PP)%DkIFR8n z5XM@nO8$ORUQ_0&FZQ#k4x21i*E~?-Lpj7s&^1uLrgp+qF+CO{&eNNIRv3zv;f(`> zgyRdb*QIvirhG33V!s1cG;#y2ysp~1%fyWkO#|<6qTMJ{s#N|)*L$pT<#SYyEy)RJLg)JPUO3KdT0vTskYNv(FO8Rj3 zjVE-f)U)IDERYWz|E$!@3NzuF=JtiQjAX-p=;)|s8CbmgX>ykFG=SlnQ8;YUhJY*; z7k&6Nglw`6I_TFqO=QhCV#ru#`9b2dO8-w6o453!WycUs9}TJz!mY{$FYst;h_&6i zmHAt`=uBUeFU%ttyk56h$@C$*X|HXQw7@w5B$LxLB#5XV=Nnmh$t##x#57Y96)(?rt(q{@i*iX=vXkJceYA$A{tS!Mz%$+IPRW zlE;Eu1B$bKyICewbI)d($0=00)^GCSh`3+#+Ds@`&eln+N6&vh9hkHMojBUZli;JZ zK@s|TtpE&!(o2<#jVUunf{exPu;tRSlKE^1oUEM9QhjKdR>1EX7=z=H8#|~`RAzQi z!?daOnQ)AwKFGHEx4W~N@$;)YDU!|FfISJ2uIhPnSV1-Hyn#8I3gJG z<0o+jBF$S_i(qkaT;BJep+r`y!qH5wgzQFnSHUHi{Sw8hFi%`2y@m6 zxxx4LqO6PjIieHSt~R{a7EDI(iUWhABy2%VI4qvsLL>P@x&Qw1A&R9@dW(*?eNHVD9M~-0KIFqTP@)GcaOqwj;0;g(w`@WdP+xGX zrQM6ycvlayXvwEjH08SRd*&(T6omGTgx2`J$wJvcQ0C9j+Isg*5hXd4b)KX@fuL~^5REP)aRp@n7!83dQ3#Ctfr}2>yn}IU08mxqk&xG@VyKxt z=m3NwmHhe(4mOFQc_11R5^_HoF8M))LFyoS(2LE`&4EY+R7~Q1CZr}>R>z8neX6q{ zE42R7_eZT1fryR{CQFWrR>9DFS>q3~Fh&FPi56;hGH+&j3R}XAHG$a9XA zkh; z@qzJ(fdPAny$JjCLBtC=Y|_&60e{6h=+Wj-gp-mVP%6F2leT}wPDxl~ctE(IE|M0s zSqPi-zrv?NY?;tdluKO3jxg1(`kL++c0a!G62Kl7_L@mXW;Fldt1JNj2Wz4JhhmF# z>02^G@wj19&w%n^Za#xG{EN+CC#dJ2{tZ(LA%{w%t-b0o9cy=1!ifAlGTn>NkGhEz z|HkPr$*A6#13`n&>gv&@zv}ihU4oqhE4uT6P#UBN?bqtK6hs_Rq<{X0#w4T)4uP{4 zdG!%wAA<9evhie|Lvlcgn@~o{)HIjp*VL*3E5H3pLVH^H)NWxp-}hSHuTj7;=k0x7 z;xAl=cQ2Gq^*Y23A}%Pnl)W)2SU%A*47LCSYy>FW3&#wZmrg2v(2cvyP86aP6 ze*|AI_JTH9`EhmI8k1Eaug{EhS767r>p?<0_or0cG-?cD|b^)5gl z=CBe9{l&@tS?%p8 zOe5zuf0bx$y1vVCd44F1`YJfSExW1?BLP;Z9}@;xipjtD z+(`{u}%Y(r!SC)H7Tu{Yl-I} zZ?;@ z&~(DB8O=n)KXVYlDd&0_Dq7RY-PA`~#Swzgk30Z}N@CK+34))*gejyR;SbgKVMv3lp|BhElXoF&#*-;Lbfs z07*y(PDj6`{ zfIX^rSuCWh9R@#^3d^UOsXmA41<1mT1uBYp4D;%z=mbpdBeh<^lnxOlXeI^T)Ozs6 zl;*2>i*Vyl)Sq!V5$E`6Rh;` z({J~v7fB7##COAXv=|%JAG(n~{7X6=RVsm?{5728SWNbr+uL;_+A_p&Eafzm z#q3tYAGRVo0e;kGIMa~h9`>CEJ;;Y>#q@^n=OS}A_|0SnWG9NVb5QGV+sJE70nY>wwL6 zaJ}qRyhx%G}K%=brw1id2~;>+>RP%z4GXq#TO?dXDC!VWZz^ zkiZq}4H)aj<`IqH%f*sLl)+Vl13?yQdavEtyU|c5yba^R3MOrTCm8}@vu&6LwyRIU zCKUn3F9EpCNGFuQ5b2x?k|{bX27&>kV&ZFH6?Z?yPkP}}nO1BXCOB^1UlB-ocpxLG zcR*I-`~?Kzz--5RKg@7d^Vn5nj5{|ZG;W=BVMIfphc6e8NbxaFe4(LkQI%_glJ)RQ z`4L9qRE^ysNKKaU&=W%3m$0cc#HSOfRhtAOl1Ti&n$X&7-h+ga&)E&#YDkX2cy)Lg zR9@^d*~#skbtWbxHhG?nOhTdaXGy}7c$$><1FvH{0X<)e2iY@P_1$*6 zW_6I82HJ5>;!l+P0A?Q*rGPmz^#ju3qr_-l2lk^f88#cu@yEd5KCe|2dXg^Y+z^7R z0a;}X&36(cM0bu~&{Rtxo_f#U}g*j$-&WT(= z1fqKWYd{ijSy;`F$wkX7FO=-YQLQVlw{!h4KDV;$UmuAJ!uvCBlW`PgPn*s-(5un-P%LElnyNnrS3Yq>Lx;O`R_0oCtFF zra1<=69Z9J?-?-F)S_6vl+IO1oxL3w|IfaJVkS;gr?S=Xo}jr&7bLhf;@* zS5=olTpS>I3W0o&7xMmjXXAKctOmx`w|k2mztFy3^3XF`vyKPzMd4ewS>aOtV)3eQ zF-}SG4CgphOj0IFKW$xhgAq<0rn9rNDNW0C$?&@5^G@qwzT3tX>@&ms_cphMO9phx zaT1q-<(>Nt2g4&R$4zNAqNS?R>c!&;Pl)02E%MV_`Dey*1NQB7*R}U_>@W%1W0}mV z)a_Pebs=bGbOw zGzX;Kz*OdE%ZCW_uHR(oPgs+(J%$p;8CvhTLi_5ML-&2orI8b0lv4K-&8<@G$5NiY zmg0M5f$wgld>Txu?gHMhMU^K~9y>aPltPwV_yyNPdft7$BtxI8!Bad2i@{ogvz1~3 zi=(^fEIYAk1H&Qz%jgl~dQJ2%!Mfpv&JB8HjfhsJf#&HV4!LB5qRffKqng9oRl2*s z7)UZm{)Qorx*#LWCwHH}6VndwmImIcTWUKAp+$z{wXY)0QCF%y<=SN#44)GK{mC;p zIIzy`#r%A7A4=S^D$}uqK-*Nbqoq2}F!5ysmWh#N`Sz+GUY`4tOw^ShysGjT-C#@7 zK|{p~Hrp#}+0#YFVgK7`9{ZnF&4H0jlkb_}oj3bOH#!EQu5C05cRWOgY5Gc?{&4eX z)P=z>@qmoK{d`&pRqz{pN+L@>E$Ky1_CdscF$QiUr|#-um+ZMPf=x|bvc&N zu>Eo?UEB5XInh47qNynxq~CVC+)rO6z22r;>QmL=O|F+drD^W{&kw6D`j<@({za=D z@HCW+G#&`W{jdq<&!~$9TUh!4(q_e~$jM}4B^kK{0$XDc@}SJD7>0(_(q)4|2z1(t zZJx{JUn87`-cazHeoiM~Rz2mrX??q#1Erxo50D`PmuesQP?~bS^3rrdBs}2zpuU)w z1vOs%XDX)-k&^#}1|WA67fgD8iF;jp76$Qr5Ahh4`bvk)j1icYH9`^z)2aGkwDC3> z7}v3bG>qY4c{rZgne*P(X126#wbXq5&|j}cUe&b5ZlG#$V=dKkKFgQbMt9a|1)Ar# z`6cUw%0j&?T^d8}rN1xYe`GC8bTq7EhIqVgA4Z9hoj080M^=qrjbo0cT=(w`n5oLq}-n_Vx$|}*decZMoehtzUeRxnPUCb%X5b-c-xZHi%-7(VE7@&Va zM!(Bppvf_3J5>?=l4TAUx(o+Js1v7_5AuUhj^Iz%odg8@&!-mOuada+J^6S`n9}8G z(cCs;7Owz$@F^DN6~e#K9T5IiSEkw?i9myBwpE9YoZ!;49L~fFw_<*}2Y2nm3K^AQ zLY99A3Z2B_`dobf>P14e%~bOBu$KBHw9#r&uKm;E9Yv)0nAqnQHA{HvpmfYBy$>s` zhV9h|P`Eu3K!M~NRq-g-U-LRuMs6y@1SJmtWl>ICpa)*z&o^O^RFFk_OS2SH^dB?? z%P^X77*_!Mrs&^YW^|aaVOc&uSn=QOQ1UPwfb-)`%!`=6#ZGoU2=@E`pefh}rN0CO z`PTQff3hTrjgeL2Kd=E7sxTtv|IgDSeb(S65EgGETXs@$NllhUY6||5JLM-U*9E@> z{w*;fI2+h)-huzM_YUWO>j!EYQ#DP^ckDEUaS6>abT!x#vcE$8X=SNA9UNbL&kQzf z$7zHR30bGR{t^*SZ#to0<+aA+xBZ#h_F*|ww+wUFBuht*xXO6f=o6RIKI{2Qp?)Nt z)l46TGo##}$0luy&sdqlE=KO}-`!^LvkCU3ntd?u>&m4qu+J}2Z}GOfU8|FKsvBq4 zI&X1M7?R((eiLDMxE~4$>8lqwH{|zzc*5nXtT)e65w39hIwlVX!hFKXb8zuL<&uc- zA6b1;&nmHg`>qD627AmTF&hQPX~mZ3IV6^fJTN;REb&7CjK zODy-xAUo4QM}w4=Pon%Eh5ugt+Y2>hhMvkjfQs(OWA7vy^%JK*%PCj!GZwtCR&hxL zIgx%;P2w6w$s(k52|W+NodFF1X7!^C34vM2L?0aBSzMp0y}&unU=((+&+|v$h{aX68s>FhK}0iP0ea*y0UlLI1%F|ddgGGc^}(^ zEAe3dq`hISj0Y%A@x+Vd4C8hXyg9A?qd7$T#Nd935GqXuyG_;(&->?7zJHA{YHhZD z@zTMJvmYP$6dmVjro*8M?!`U-2)mH^6{Q1h$j!h*W$$DLJy4nBlpbiVXu7`ozwvSg z`YrE+_fel=yOC`FJV10biI_AOHpLg*P5Rr4I@=JO=hRMuHa?geFWV01@;k>{W&hVZ zz~(adOdr{!yVR^1k4!vZ3`CYlc)~I*@vW?IO*tVWD->yRtrFbOZ$O5-a~ooNBSXIS zywmD9YCZBq+mJLkDdG!_=bT;Sm5>(-k9pTdmV#UBHo)$ZtQY2%hKik5E@AeheVG1q z?{aa2ZarDpG=3*L@?REo4@H(jPUV8hcm(S*r$;WkM_?PjEI!Xgr*nB)7v%$;f(WdPVt3DP44Na&?Bh&37i;T)EbF*S>VMx$vXr6U%Mc zTSH=4=bMd;1hVVWYjsvKoTQY$f`O_;7;@5W1hzPG%IGy7usZl_8o}tlZ!_dDXfWfe zSM&MfsTL+Nuf5s#6dL5k)om7?g4FYXe!treC%~-ORq{0F zh#_Ro))!Lqy1V8DIp^CypiMBPF|&2J<(yFXz-fuR^-6vA_Izl1dNJVq{G3fBiPf&{ z>U3#b+bM9|+pAnzgiU|0=b>`0;rJ0{@D@O*MX4KJSLhXfENTbkE&dcbNs2B2qsQ8# zeU1__vcYdic7eT5jhX|;`zmJq~%p)O67$$X({eY z6P$q|z{Aep?q06=C{v6gja`pSepZq^3Bsp=UZo>Vx z=1%toEA;A&3?oPNo60Y9`=Lldzim@M^X(ql3u66+YIPevg>vOr6RnaugQDJwnKBJ} zt1P*M93v6rNTFJqT|ejzVD%3)<6%;(fYu7rrSH74UTUpV`Re#Lmfs?T@E*!uskxn8 z$q~IG=%``}4Gp#NE%9NSy7PP)$r^h_$z4Uu(7g4}#>7~>D`RMwHTLv<565ik4zG`t>D7my>CilR!YjQ zZrJxXm?`Jv5hnLNkC9LA&1CkNWXFHzL=kvQDaa}k84VgX3l2#H{tfNi?ar2gdLDHD zhQp}U;IW@4LQ$MCzzdLa6V!5;Y8gmc8tyR6&E4lnrZOKX@5R8a24OKqw$jI3{5b#d$NeFqM7W3Uc>5WaXSrJ9!fS&LhWX zQ84HI=iW6Q0{rIe4w>^dw{$S5{U=NuuXX7lw~V4R?I7|u^S{9|AEuR0xE z`!!5SzwHU@UCHxTU*uZYu8#c?uY}_}A?rMU$~E<43+o?w!77xwQU7+irT%QZ=?XX0 zWcS=R$NU*w7?D3x9~T=Y=vjXkbVwwp@Bb3Isc`1iZR&o+PD8m?{Tj3U?aQr8y;hk@ zUm5c#1{Ug_)c(tb%%-g;eOR9Nvn`b3XCLs}e}^eE-`(svF}qpo-Z59(J*=Pb!~YPA zL4U7fbuVO$C&fpl9yhUtUEr2W55+)wIgu+~QUgJ#3EFJ&K1!cGLs8O~+rQA256F4j zgAqc~_=Z_t`1E=a>-r>^psrdr3JRsb>23~>TPS#bAJRH2Q5nl$L$oOy5(>nO&@;&d z;(NEZR2l+wz=~;{vuo<11*e)=%TN^=Op=4fsfm}TRhqy~D z!YX`Yyl{A8ds&6q^TjWQ+$)cbEheqQ7o70Lz7Jca%%Nu6`we9=Jqwgqc73Z32o0|o zW5UqHPZ`O4BB*UOdnp6Qbb?*k@Es278T?%jcZ|U~IRqgj02a_w0MxAigObU+6|Hw*39@=L6_@Ik*X6#J)&k3~7v5#)tCYScAj& zc7om=UqcpFXO4uXeNcED-~A4uRFRkb6VIlcZvAA{7K~Z_PAEOAw?UX+8*ik15#WEC zuyvr|@GN^@>tstthoWcjqTylst>PFGaiC$5agc#9{jMWu`pQkNl(X#{k&C}x0`j79 zDsWCnmL%oD6}p6H|9Lf9T;O{8ybcvCkj0n6%?-V=fB#~M`?sIN1gKMJ(vGjU^04J7 z3Pp`aoW?uixtO#BC~r`y`LAo!aZyhYU`xudw=Lj1#`1YX;he6f6EH>Td(!3F!L#x; z=RKF+an>0=4HA>6;5LezpjGGl<+tN_G9PFA=2I`Ny4Kc*h;`fi@)uw`eFM=N#PEe_ zuZiN5D>2v+?Jj0Riy6RIR@~A@52ds+JtYMuS2|puW;GEDanodzjac09V zL*tgCQ_YPAq%-wtZHZd%PpShQ$~!cQ%ZW=MWM&ALtf?Nz+?k*GXpMWU|ozTki+nJo)4${ zD!c^42Jd$*j6EC7@4uD8tO&VNqAKGOw8-;*Mx>#XMdhB1tw0ho{)X6NP=zT$0WSWE z{GDk)V8g&^?7h-k7;-HB8u}UvTMS<_P$jd^yaR4v4MVW(-bN|6&}QKXMc?Yz*y7yX z7`h++T(vYT_@`yU|9t%|8(D(zq&?6|=Kq@2A~kqx@V2PfhGTjMm)&0>+c)zsixF^9 zQrb1ETZWta6CkY-U;uTPKl|yD^zdIIR|@uij!Yl0q-a&vNy>#b+m-reJQwx{wL~hU z6_40I%U1>Vq<~d-C$RKQ=HmVj>ta0Wz}+QVw&6oQEsT5kpIM~#r^llQ1rorsfEYYo L{an^LB{Ts57WlQ# diff --git a/docs/proposals-accepted/20221129-avoid-global-sort.md b/docs/proposals-accepted/20221129-avoid-global-sort.md index 84796ff1ff..d2f7e8821b 100644 --- a/docs/proposals-accepted/20221129-avoid-global-sort.md +++ b/docs/proposals-accepted/20221129-avoid-global-sort.md @@ -1,18 +1,18 @@ ## Avoid Global Sort on Querier Select * **Owners:** - * @bwplotka, @fpetkovski + * @bwplotka, @fpetkovski * **Related Tickets:** - * https://github.com/thanos-io/thanos/issues/5719 - * https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020 - * https://github.com/thanos-io/thanos/pull/5796 also alternatives (https://github.com/thanos-io/thanos/pull/5692) + * https://github.com/thanos-io/thanos/issues/5719 + * https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020 + * https://github.com/thanos-io/thanos/pull/5796 also alternatives (https://github.com/thanos-io/thanos/pull/5692) > TL;DR: We propose solution that allows saving query and query_range latency on common setups when deduplication on and data replication. Initial benchmarks indicate ~20% latency improvement for data replicated 2 times. -> -> To make it work we propose adding field to Store API Series call "WithoutReplicaLabels []string", guarded by "SupportsWithoutReplicaLabels" field propagated via Info API. It allows telling store implementations to remove given labels (if they are replica labels) from result, preserving sorting by labels after the removal. -> -> NOTE: This change will break unlikely setups that deduplicate on non-replica label (misconfiguration or wrong setup). +> +> To make it work we propose adding field to Store API Series call "WithoutReplicaLabels []string", guarded by "SupportsWithoutReplicaLabels" field propagated via Info API. It allows telling store implementations to remove given labels (if they are replica labels) from result, preserving sorting by labels after the removal. +> +> NOTE: This change will break unlikely setups that deduplicate on non-replica label (misconfiguration or wrong setup). ## Glossary @@ -20,7 +20,7 @@ ## Why -Currently, we spent a lof of storage selection CPU time on resorting resulting time series needed for deduplication (exactly in [`sortDedupLabels`](https://github.com/thanos-io/thanos/blob/main/pkg/query/querier.go#L400)). However, given distributed effort and current sorting guarantees of StoreAPI there is potential to reduce sorting effort or/and distribute it to leafs or multiple threads. +Currently, we spent a lof of storage selection CPU time on resorting resulting time series needed for deduplication (exactly in [`sortDedupLabels`](https://github.com/thanos-io/thanos/blob/main/pkg/query/querier.go#L400)). However, given distributed effort and current sorting guarantees of StoreAPI there is potential to reduce sorting effort or/and distribute it to leafs or multiple threads. ### Pitfalls of the current solution @@ -31,11 +31,11 @@ Current flow can be represented as follows: 1. Querier PromQL Engine selects data. At this point we know if users asked for deduplicated data or not and [what replica labels to use](https://thanos.io/tip/components/query.md/#deduplication-replica-labels). 2. Querier selection asks internal, in-process Store API which is represented by Proxy code component. It asks relevant store API for data, using StoreAPI.Series. 3. Responses are pulled and k-way merged by the time series. StoreAPI guarantees the responses are sorted by series and the external labels (including replica) are included in the time series. - * There was a [bug in receiver](https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020#diff-b3f73a54121d88de203946e84955da7027e3cfce7f0cd82580bf215ac57c02f4) that caused series to be not sorted when returned. Fixed in v0.29.0. +* There was a [bug in receiver](https://github.com/thanos-io/thanos/commit/043c5bfcc2464d3ae7af82a1428f6e0d6510f020#diff-b3f73a54121d88de203946e84955da7027e3cfce7f0cd82580bf215ac57c02f4) that caused series to be not sorted when returned. Fixed in v0.29.0. 4. Querier selection waits until all responses are buffered and then it deduplicates the data, given the requested replica labels. Before it's done it globally sort data with moving replica label at the end of the time series in `sortDedupLabels`. 5. Data is deduplicated using `dedup` package. -The pittfall is in the fact that global sort can be in many cases completely avoided, even when deduplication is enabled. Many storeAPIs can drop certain replica labels without need to resort and others can k-way merge different data sets without certain replica labels without extra effort. +The pittfall is in the fact that global sort can be in many cases completely avoided, even when deduplication is enabled. Many storeAPIs can drop certain replica labels without need to resort and others can k-way merge different data sets without certain replica labels without extra effort. ## Goals @@ -44,12 +44,12 @@ Goals and use cases for the solution as proposed in [How](#how): * Avoid expensive global sort of all series before passing them to PromQL engine in Querier. * Allow StoreAPI implementation to announce if it supports sorting feature or not. The rationale is that we want to make it possible to create simpler StoreAPI servers, if operator wants to trade-off it with latency. * Clear the behaviour in tricky cases when there is an overlap of replica labels between what's in TSDB vs what's attached as external labels. -* Ensure this change can be rolled out in compatible way. +* Ensure this change can be rolled out in compatible way. ## Non-Goals * Allow consuming series in streamed way in PromQL engine. - * While this pitfall (global sort) blocks the above idea, it's currently still more beneficial to pull all series upfront (eager approach) as soon as possible. This is due to current PromQL architecture which requires info upfront for query planners and execution. We don't plan to change it yet, thus no need to push explicitly for that. + * While this pitfall (global sort) blocks the above idea, it's currently still more beneficial to pull all series upfront (eager approach) as soon as possible. This is due to current PromQL architecture which requires info upfront for query planners and execution. We don't plan to change it yet, thus no need to push explicitly for that. ## How @@ -59,12 +59,12 @@ To understand proposal, let's go through important, yet perhaps not trivial, fac * For StoreAPI or generally data that belongs to one replica, if you exclude certain replica label during sort, it does not impact sorting order for returned series. This means, any feature that desired different sort for replicated series is generally noop for sidecars, rules, single tenant receiver or within single block (or one stream of blocks). * You can't stream sorting of unsorted data. Furthermore, it's not possible to detect that data is unsorted, unless we fetch and buffer all series. -* In v0.29 and below, you can deduplicate on any labels, including non replicas. This is assumed semantically wrong, yet someone might depend on it. +* In v0.29 and below, you can deduplicate on any labels, including non replicas. This is assumed semantically wrong, yet someone might depend on it. * Thanos never handled overlap of chunks within one set of store API response. ### Solution -To avoid global sort, we propose removing required replica labels and sort on store API level. +To avoid global sort, we propose removing required replica labels and sort on store API level. For the first step (which is required for compatibility purposes anyway), we propose a logic in proxy Store API implementation that when deduplication is requested with given replica labels will: @@ -79,7 +79,7 @@ As the second step we propose adding `without_replica_labels` field to `SeriesRe ```protobuf message SeriesRequest { // ... - + // without_replica_labels are replica labels which have to be excluded from series set results. // The sorting requirement has to be preserved, so series should be sorted without those labels. // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by @@ -107,10 +107,9 @@ message StoreInfo { } ``` -Thanks of that implementations can optionally support this feature. We can make all Thanos StoreAPI support it, which will allow faster -deduplication queries on all types of setups. +Thanks of that implementations can optionally support this feature. We can make all Thanos StoreAPI support it, which will allow faster deduplication queries on all types of setups. -In the initial tests we see ~2x improvements on my test data (8M series block, requests for ~200k series) with querier and store gateway. +In the initial tests we see 60% improvements on my test data (8M series block, requests for ~200k series) with querier and store gateway. Without this change: @@ -120,10 +119,9 @@ After implementing this proposal: ![2](../img/globalsort-optimized.png) - ## Alternatives -1. Version StoreAPI. +1. Version StoreAPI. As a best practice gRPC services should be versioned. This should allow easier iterations for everybody implementing or using it. However, having multiple versions (vs extra feature enablement field) might make client side more complex, so we propose to postpone it. @@ -133,7 +131,7 @@ Extra slice in all Series might feel redundant, given all series are always grou 3. Instead of removing some replica labels, just sort without them and leave at the end. -For debugging purposes we could keep the replica labels we want to dedup on at the end of label set. +For debugging purposes we could keep the replica labels we want to dedup on at the end of label set. This might however be less clean way of providing better debuggability, which is not yet required. @@ -161,7 +159,7 @@ Cons: * Extra code and protobuf complexity * Semantics of replica labels are hard to maintain when partial deduplication is configured (we only dedup by part of replica labels, not by all of them). This dynamic policy makes it hard to have clean response with separation of replica labels (i.e. should included replica labels be in "labels" or "replica labels")? -This might be not needed for now. We can add more awareness of replication later on. +This might be not needed for now. We can add more awareness of replication later on. ## Action Plan @@ -169,3 +167,17 @@ The tasks to do in order to migrate to the new idea. * [ ] Merging the PR with the proposal (also includes implementation) * [ ] Add support for `without_replica_label` to other store API servers. +* [ ] Move to deduplicate over chunks from series See [TODO in querier.go:405](../../pkg/query/querier.go) + +```go +// TODO(bwplotka): Move to deduplication on chunk level inside promSeriesSet, similar to what we have in dedup.NewDedupChunkMerger(). +// This however require big refactor, caring about correct AggrChunk to iterator conversion, pushdown logic and counter reset apply. +// For now we apply simple logic that splits potential overlapping chunks into separate replica series, so we can split the work. +set := &promSeriesSet{ + mint: q.mint, + maxt: q.maxt, + set: dedup.NewOverlapSplit(newStoreSeriesSet(resp.seriesSet)), + aggrs: aggrs, + warns: warns, +} +``` diff --git a/internal/cortex/querier/queryrange/queryrange.pb.go b/internal/cortex/querier/queryrange/queryrange.pb.go index b918da0ece..aacce2d0c7 100644 --- a/internal/cortex/querier/queryrange/queryrange.pb.go +++ b/internal/cortex/querier/queryrange/queryrange.pb.go @@ -538,6 +538,7 @@ func (m *PrometheusInstantQueryData) GetStats() *PrometheusResponseStats { type PrometheusInstantQueryResult struct { // Types that are valid to be assigned to Result: + // // *PrometheusInstantQueryResult_Scalar // *PrometheusInstantQueryResult_StringSample // *PrometheusInstantQueryResult_Vector diff --git a/pkg/api/query/v1_test.go b/pkg/api/query/v1_test.go index 55706a1bc9..4c07b3ce11 100644 --- a/pkg/api/query/v1_test.go +++ b/pkg/api/query/v1_test.go @@ -56,6 +56,7 @@ import ( "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" + storetestutil "github.com/thanos-io/thanos/pkg/store/storepb/testutil" "github.com/thanos-io/thanos/pkg/testutil/custom" "github.com/thanos-io/thanos/pkg/testutil/e2eutil" "github.com/thanos-io/thanos/pkg/testutil/testpromcompatibility" @@ -192,7 +193,7 @@ func TestQueryEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, newProxyStoreWithTSDBStore(db), 2, timeout), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4), @@ -642,6 +643,24 @@ func TestQueryEndpoints(t *testing.T) { } } +func newProxyStoreWithTSDBStore(db store.TSDBReader) *store.ProxyStore { + c := &storetestutil.TestClient{ + Name: "1", + StoreClient: storepb.ServerAsClient(store.NewTSDBStore(nil, db, component.Query, nil), 0), + MinTime: math.MinInt64, MaxTime: math.MaxInt64, + } + + return store.NewProxyStore( + nil, + nil, + func() []store.Client { return []store.Client{c} }, + component.Query, + nil, + 0, + store.EagerRetrieval, + ) +} + func TestMetadataEndpoints(t *testing.T) { var old = []labels.Labels{ { @@ -733,7 +752,7 @@ func TestMetadataEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, newProxyStoreWithTSDBStore(db), 2, timeout), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4), @@ -746,7 +765,7 @@ func TestMetadataEndpoints(t *testing.T) { baseAPI: &baseAPI.BaseAPI{ Now: func() time.Time { return now }, }, - queryableCreate: query.NewQueryableCreator(nil, nil, store.NewTSDBStore(nil, db, component.Query, nil), 2, timeout), + queryableCreate: query.NewQueryableCreator(nil, nil, newProxyStoreWithTSDBStore(db), 2, timeout), queryEngine: qe, lookbackDeltaCreate: func(m int64) time.Duration { return time.Duration(0) }, gate: gate.New(nil, 4), diff --git a/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go b/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go index 8b02a7ca9e..0e78a4ccd9 100644 --- a/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go +++ b/pkg/query/internal/test-storeset-pre-v0.8.0/storeset.go @@ -208,7 +208,7 @@ func (s *storeRef) SupportsSharding() bool { return false } -func (s *storeRef) SendsSortedSeries() bool { +func (s *storeRef) SupportsWithoutReplicaLabels() bool { return false } diff --git a/pkg/query/querier.go b/pkg/query/querier.go index 9e2cf6280e..03c9e9b5a6 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -131,7 +131,7 @@ type querier struct { mint, maxt int64 replicaLabels []string storeDebugMatchers [][]*labels.Matcher - proxy storepb.StoreServer + proxy *store.ProxyStore deduplicate bool maxResolutionMillis int64 partialResponseStrategy storepb.PartialResponseStrategy @@ -152,7 +152,7 @@ func newQuerier( maxt int64, replicaLabels []string, storeDebugMatchers [][]*labels.Matcher, - proxy storepb.StoreServer, + proxy *store.ProxyStore, deduplicate bool, maxResolutionMillis int64, partialResponse, diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 5bfdc559b5..f5b61eb0be 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -33,7 +33,7 @@ import ( "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" - "github.com/thanos-io/thanos/pkg/testutil/teststore" + storetestutil "github.com/thanos-io/thanos/pkg/store/storepb/testutil" ) type sample struct { @@ -338,7 +338,7 @@ func TestQuerier_Select_AfterPromQL(t *testing.T) { for _, tcase := range []struct { name string - storeAPI storepb.StoreServer + storeAPI *store.ProxyStore replicaLabels []string // Replica label groups chunks by the label value and strips it from the final result. hints *storage.SelectHints equivalentQuery string @@ -350,11 +350,11 @@ func TestQuerier_Select_AfterPromQL(t *testing.T) { { // Regression test 1 against https://github.com/thanos-io/thanos/issues/2890. name: "when switching replicas don't miss samples when set with a big enough lookback delta", - storeAPI: func() storepb.StoreServer { + storeAPI: newProxyStore(func() storepb.StoreServer { s, err := store.NewLocalStoreFromJSONMmappableFile(logger, component.Debug, nil, "./testdata/issue2890-seriesresponses.json", store.ScanGRPCCurlProtoStreamMessages) testutil.Ok(t, err) return s - }(), + }()), equivalentQuery: `cluster_version{}`, replicaLabels: []string{"replica"}, hints: &storage.SelectHints{ @@ -786,7 +786,7 @@ func TestQuerier_Select(t *testing.T) { g, timeout, nil, - func(i storepb.SeriesStatsCounter) {}, + NoopSeriesStatsReporter, ) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) @@ -837,7 +837,7 @@ func newProxyStore(storeAPIs ...storepb.StoreServer) *store.ProxyStore { if srv, ok := s.(*testStoreServer); ok { withoutReplicaLabelsEnabled = len(srv.respsWithoutReplicaLabels) > 0 } - cls[i] = &teststore.TestClient{ + cls[i] = &storetestutil.TestClient{ Name: fmt.Sprintf("%v", i), StoreClient: storepb.ServerAsClient(s, 0), MinTime: math.MinInt64, MaxTime: math.MaxInt64, @@ -1078,7 +1078,7 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { timeout := 100 * time.Second g := gate.New(2) - q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, false, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) + q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, newProxyStore(s), false, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) @@ -1148,7 +1148,7 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { timeout := 5 * time.Second g := gate.New(2) - q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, s, true, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) + q := newQuerier(context.Background(), logger, realSeriesWithStaleMarkerMint, realSeriesWithStaleMarkerMaxt, []string{"replica"}, nil, newProxyStore(s), true, 0, true, false, false, g, timeout, nil, NoopSeriesStatsReporter) t.Cleanup(func() { testutil.Ok(t, q.Close()) }) diff --git a/pkg/query/query_bench_test.go b/pkg/query/query_bench_test.go index 178c929cf6..fde1017434 100644 --- a/pkg/query/query_bench_test.go +++ b/pkg/query/query_bench_test.go @@ -6,9 +6,11 @@ package query import ( "context" "fmt" + "math" "math/rand" "path/filepath" "testing" + "time" "github.com/go-kit/log" "github.com/prometheus/prometheus/model/labels" @@ -80,15 +82,24 @@ func benchQuerySelect(t testutil.TB, totalSamples, totalSeries int, dedup bool) } logger := log.NewNopLogger() - q := &querier{ - ctx: context.Background(), - logger: logger, - proxy: &mockedStoreServer{responses: resps}, - replicaLabels: []string{"a_replica"}, - deduplicate: dedup, - selectGate: gate.NewNoop(), - seriesStatsReporter: NoopSeriesStatsReporter, - } + q := newQuerier( + context.Background(), + logger, + math.MinInt64, + math.MaxInt64, + []string{"a_replica"}, + nil, + newProxyStore(&mockedStoreServer{responses: resps}), + dedup, + 0, + false, + false, + false, + gate.NewNoop(), + 10*time.Second, + nil, + NoopSeriesStatsReporter, + ) testSelect(t, q, expectedSeries) } @@ -118,7 +129,8 @@ func testSelect(t testutil.TB, q *querier, expectedSeries []labels.Labels) { t.ResetTimer() for i := 0; i < t.N(); i++ { - ss := q.Select(true, nil) // Select all. + ss := q.Select(true, nil, &labels.Matcher{Value: "foo", Name: "bar", Type: labels.MatchEqual}) + testutil.Ok(t, ss.Err()) testutil.Equals(t, 0, len(ss.Warnings())) if t.IsBenchmark() { @@ -137,22 +149,23 @@ func testSelect(t testutil.TB, q *querier, expectedSeries []labels.Labels) { } testutil.Equals(t, len(expectedSeries), gotSeriesCount) - } else { - // Check more carefully. - var gotSeries []labels.Labels - for ss.Next() { - s := ss.At() - gotSeries = append(gotSeries, s.Labels()) + testutil.Ok(t, ss.Err()) + return + } - // This is when resource usage should actually start growing. - iter := s.Iterator() - for iter.Next() != chunkenc.ValNone { - testT, testV = iter.At() - } - testutil.Ok(t, iter.Err()) + // Check more carefully. + var gotSeries []labels.Labels + for ss.Next() { + s := ss.At() + gotSeries = append(gotSeries, s.Labels()) + + iter := s.Iterator() + for iter.Next() != chunkenc.ValNone { + testT, testV = iter.At() } - testutil.Equals(t, expectedSeries, gotSeries) + testutil.Ok(t, iter.Err()) } + testutil.Equals(t, expectedSeries, gotSeries) testutil.Ok(t, ss.Err()) } }) diff --git a/pkg/query/query_test.go b/pkg/query/query_test.go index 5ff5393524..35949377e8 100644 --- a/pkg/query/query_test.go +++ b/pkg/query/query_test.go @@ -11,14 +11,13 @@ import ( "testing" "time" + "github.com/efficientgo/core/testutil" "github.com/go-kit/log" "github.com/prometheus/prometheus/storage" - "github.com/thanos-io/thanos/pkg/testutil/teststore" - - "github.com/efficientgo/core/testutil" "github.com/thanos-io/thanos/pkg/component" "github.com/thanos-io/thanos/pkg/store" "github.com/thanos-io/thanos/pkg/store/storepb" + storetestutil "github.com/thanos-io/thanos/pkg/store/storepb/testutil" "github.com/thanos-io/thanos/pkg/testutil/custom" ) @@ -50,7 +49,7 @@ func TestQuerier_Proxy(t *testing.T) { testutil.Ok(t, err) // TODO(bwplotka): Parse external labels. - clients = append(clients, &teststore.TestClient{ + clients = append(clients, &storetestutil.TestClient{ Name: fmt.Sprintf("store number %v", i), StoreClient: storepb.ServerAsClient(selectedStore(store.NewTSDBStore(logger, st.storage.DB, component.Debug, nil), m, st.mint, st.maxt), 0), MinTime: st.mint, diff --git a/pkg/store/proxy.go b/pkg/store/proxy.go index 108d09c015..e357ebe8fd 100644 --- a/pkg/store/proxy.go +++ b/pkg/store/proxy.go @@ -388,7 +388,7 @@ func labelSetsMatch(matchers []*labels.Matcher, lset ...labels.Labels) bool { for _, ls := range lset { notMatched := false for _, m := range matchers { - if lv := ls.Get(m.Name); !m.Matches(lv) { + if lv := ls.Get(m.Name); ls.Has(m.Name) && !m.Matches(lv) { notMatched = true break } diff --git a/pkg/store/proxy_heap_test.go b/pkg/store/proxy_heap_test.go index 448b67cfc9..6616db4016 100644 --- a/pkg/store/proxy_heap_test.go +++ b/pkg/store/proxy_heap_test.go @@ -1,3 +1,6 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + package store import ( diff --git a/pkg/store/proxy_test.go b/pkg/store/proxy_test.go index 638db5451a..120e93cc17 100644 --- a/pkg/store/proxy_test.go +++ b/pkg/store/proxy_test.go @@ -24,7 +24,6 @@ import ( "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/thanos-io/thanos/pkg/testutil/teststore" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -104,7 +103,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "no storeAPI available for 301-302 time range", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -124,7 +123,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; no series for ext=2 external label matcher", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -145,7 +144,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -171,7 +170,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for any external label matcher", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{4, 3}}, []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -196,7 +195,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for any external label matcher, but selector blocks", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -216,7 +215,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "no validation if storeAPI follow matching contract", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -242,7 +241,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "complex scenario with storeAPIs warnings", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}, []sample{{4, 3}}), @@ -254,7 +253,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -263,7 +262,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -272,7 +271,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "c"), []sample{{100, 1}, {300, 3}, {400, 4}}), @@ -281,7 +280,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "outside"), []sample{{1, 1}}), @@ -316,7 +315,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available two duplicated series for ext=1 external label matcher from 2 storeAPIs", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -326,7 +325,7 @@ func TestProxyStore_Series(t *testing.T) { MaxTime: 300, ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{1, 4}, {2, 5}, {3, 6}}), @@ -352,7 +351,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available a few duplicated series for ext=1 external label matcher, mixed storeAPIs", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{5, 5}, {7, 7}}), @@ -366,7 +365,7 @@ func TestProxyStore_Series(t *testing.T) { MaxTime: 300, ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "1", "w", "1"), []sample{{2, 1}}), @@ -402,7 +401,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "same external labels are validated during upload and on querier storeset, proxy does not care", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -412,7 +411,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 11}, {2, 22}, {3, 33}}), @@ -438,7 +437,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "partial response enabled", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -449,7 +448,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, @@ -474,7 +473,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "partial response disabled", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -485,7 +484,7 @@ func TestProxyStore_Series(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, @@ -506,7 +505,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher; allowed by store debug matcher", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -535,7 +534,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "storeAPI available for time range; available series for ext=1 external label matcher; blocked by store debug matcher.", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -558,7 +557,7 @@ func TestProxyStore_Series(t *testing.T) { { title: "sharded series response", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -598,7 +597,7 @@ func TestProxyStore_Series(t *testing.T) { for _, replicaLabelSupport := range []bool{false, true} { t.Run(fmt.Sprintf("replica_support=%v", replicaLabelSupport), func(t *testing.T) { for _, s := range tc.storeAPIs { - cl := s.(*teststore.TestClient) + cl := s.(*storetestutil.TestClient) cl.WithoutReplicaLabelsEnabled = replicaLabelSupport } for _, strategy := range []RetrievalStrategy{EagerRetrieval, LazyRetrieval} { @@ -657,7 +656,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st errors out after some delay; 2nd store is fast", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -672,7 +671,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -712,7 +711,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is slow, 2nd store is fast;", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -724,7 +723,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -748,7 +747,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is fast, 2nd store is slow;", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -759,7 +758,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -784,7 +783,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is slow on 2nd series, 2nd store is fast;", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -799,7 +798,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -823,7 +822,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; 1st store is fast to respond, 2nd store is slow on 2nd series;", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -836,7 +835,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -862,7 +861,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is slow to respond, 2nd store is fast;", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -874,7 +873,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -902,7 +901,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is fast, 2nd store is slow;", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -913,7 +912,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -942,7 +941,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is fast, 2-3 is slow, 4th is fast;", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -953,7 +952,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -965,7 +964,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -977,7 +976,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -1009,7 +1008,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; 1st store is slow on 2nd series, 2nd store is fast", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -1024,7 +1023,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -1056,7 +1055,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response disabled; all stores respond 3s", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1088,7 +1087,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { { title: "partial response enabled; all stores respond 3s", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("a", "b"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1101,7 +1100,7 @@ func TestProxyStore_SeriesSlowStores(t *testing.T) { MinTime: 1, MaxTime: 300, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("b", "c"), []sample{{1, 1}, {2, 2}, {3, 3}}), @@ -1187,7 +1186,7 @@ func TestProxyStore_Series_RequestParamsProxied(t *testing.T) { }, } cls := []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: m, ExtLset: []labels.Labels{labels.FromStrings("ext", "1")}, MinTime: 1, @@ -1227,14 +1226,14 @@ func TestProxyStore_Series_RegressionFillResponseChannel(t *testing.T) { var cls []Client for i := 0; i < 10; i++ { - cls = append(cls, &teststore.TestClient{ + cls = append(cls, &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("test error"), }, MinTime: 1, MaxTime: 300, }) - cls = append(cls, &teststore.TestClient{ + cls = append(cls, &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storepb.NewWarnSeriesResponse(errors.New("warning")), @@ -1288,13 +1287,13 @@ func TestProxyStore_LabelValues(t *testing.T) { }, } cls := []Client{ - &teststore.TestClient{StoreClient: m1}, - &teststore.TestClient{StoreClient: &mockedStoreAPI{ + &storetestutil.TestClient{StoreClient: m1}, + &storetestutil.TestClient{StoreClient: &mockedStoreAPI{ RespLabelValues: &storepb.LabelValuesResponse{ Values: []string{"3", "4"}, }, }}, - &teststore.TestClient{StoreClient: &mockedStoreAPI{ + &storetestutil.TestClient{StoreClient: &mockedStoreAPI{ RespLabelValues: &storepb.LabelValuesResponse{ Values: []string{"5", "6"}, }}, @@ -1356,14 +1355,14 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response disabled", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "c", "d"}, @@ -1382,17 +1381,18 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response disabled, but returns error", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, + Name: "test", }, }, req: &storepb.LabelNamesRequest{ @@ -1405,14 +1405,14 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "label_names partial response enabled", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespError: errors.New("error!"), }, @@ -1429,7 +1429,7 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "stores filtered by time range", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, @@ -1438,7 +1438,7 @@ func TestProxyStore_LabelNames(t *testing.T) { MinTime: timestamp.FromTime(time.Now().Add(-4 * time.Hour)), MaxTime: timestamp.FromTime(time.Now().Add(-3 * time.Hour)), }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"c", "d"}, @@ -1459,12 +1459,13 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "store matchers blocks", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, + Name: "testaddr", }, }, req: &storepb.LabelNamesRequest{ @@ -1479,12 +1480,13 @@ func TestProxyStore_LabelNames(t *testing.T) { { title: "store matchers allows", storeAPIs: []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespLabelNames: &storepb.LabelNamesResponse{ Names: []string{"a", "b"}, }, }, + Name: "testaddr", }, }, req: &storepb.LabelNamesRequest{ @@ -1573,7 +1575,7 @@ func TestStoreMatches(t *testing.T) { expectedReason string }{ { - s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "b", "1"), }, @@ -1582,7 +1584,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "does not have data within this time period: [0,-1]. Store time ranges: [0,0]", }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "b", "1"), }, @@ -1590,33 +1592,33 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 201, maxt: 300, expectedMatch: false, expectedReason: "does not have data within this time period: [201,300]. Store time ranges: [100,200]", }, { - s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 200, maxt: 300, expectedMatch: true, }, { - s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 50, maxt: 99, expectedMatch: false, expectedReason: "does not have data within this time period: [50,99]. Store time ranges: [100,200]", }, { - s: &teststore.TestClient{MinTime: 100, MaxTime: 200}, + s: &storetestutil.TestClient{MinTime: 100, MaxTime: 200}, mint: 50, maxt: 101, expectedMatch: true, }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "a", "b"), }, @@ -1624,7 +1626,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "a", "c"), }, @@ -1633,7 +1635,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "external labels [{a=\"b\"}] does not match request label matchers: [a=\"c\"]", }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchRegexp, "a", "b|c"), }, @@ -1641,7 +1643,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, + s: &storetestutil.TestClient{ExtLset: []labels.Labels{labels.FromStrings("a", "b")}}, ms: []*labels.Matcher{ labels.MustNewMatcher(labels.MatchNotRegexp, "a", ""), }, @@ -1649,7 +1651,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{ + s: &storetestutil.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1662,7 +1664,7 @@ func TestStoreMatches(t *testing.T) { expectedReason: "external labels [{a=\"b\"} {a=\"c\"} {a=\"d\"}] does not match request label matchers: [a=\"e\"]", }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{ + s: &storetestutil.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1674,7 +1676,7 @@ func TestStoreMatches(t *testing.T) { expectedMatch: true, }, { - s: &teststore.TestClient{ExtLset: []labels.Labels{ + s: &storetestutil.TestClient{ExtLset: []labels.Labels{ labels.FromStrings("a", "b"), labels.FromStrings("a", "c"), labels.FromStrings("a", "d"), @@ -1891,12 +1893,13 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { resps = append(resps, storepb.NewSeriesResponse(created[i])) } - clients[j] = &teststore.TestClient{ + clients[j] = &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: resps, }, - MinTime: math.MinInt64, - MaxTime: math.MaxInt64, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + WithoutReplicaLabelsEnabled: true, } } @@ -1913,7 +1916,7 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { var expected []*storepb.Series lastLabels := storepb.Series{} for _, c := range clients { - m := c.(*teststore.TestClient).StoreClient.(*mockedStoreAPI) + m := c.(*storetestutil.TestClient).StoreClient.(*mockedStoreAPI) // NOTE: Proxy will merge all series with same labels without any frame limit (https://github.com/thanos-io/thanos/issues/2332). for _, r := range m.RespSeries { @@ -1951,16 +1954,17 @@ func benchProxySeries(t testutil.TB, totalSamples, totalSeries int) { }, ) - // Change client to just one. + // Change client to one, containing all series. store.stores = func() []Client { - return []Client{&teststore.TestClient{ + return []Client{&storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ // All responses. RespSeries: allResps, }, - ExtLset: []labels.Labels{labels.FromStrings("ext1", "1")}, - MinTime: math.MinInt64, - MaxTime: math.MaxInt64, + ExtLset: []labels.Labels{labels.FromStrings("ext1", "1")}, + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + WithoutReplicaLabelsEnabled: true, }} } @@ -1988,7 +1992,7 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { defer custom.TolerantVerifyLeak(t) clients := []Client{ - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ // Ensure more than 10 (internal respCh channel). @@ -2007,7 +2011,7 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { MinTime: math.MinInt64, MaxTime: math.MaxInt64, }, - &teststore.TestClient{ + &storetestutil.TestClient{ StoreClient: &mockedStoreAPI{ RespSeries: []*storepb.SeriesResponse{ storeSeriesResponse(t, labels.FromStrings("b", "a"), []sample{{0, 0}, {2, 1}, {3, 2}}), @@ -2051,7 +2055,7 @@ func TestProxyStore_NotLeakingOnPrematureFinish(t *testing.T) { } func TestProxyStore_storeMatchMetadata(t *testing.T) { - c := teststore.TestClient{} + c := storetestutil.TestClient{Name: "testaddr"} c.IsLocalStore = true ok, reason := storeMatchDebugMetadata(c, [][]*labels.Matcher{{}}) diff --git a/pkg/store/storepb/rpc.pb.go b/pkg/store/storepb/rpc.pb.go index b288b5c909..db64cf60c8 100644 --- a/pkg/store/storepb/rpc.pb.go +++ b/pkg/store/storepb/rpc.pb.go @@ -296,7 +296,8 @@ type SeriesRequest struct { // shard_info is used by the querier to request a specific // shard of blocks instead of entire blocks. ShardInfo *ShardInfo `protobuf:"bytes,13,opt,name=shard_info,json=shardInfo,proto3" json:"shard_info,omitempty"` - // without_replica_labels are replica labels which have to be excluded from series set results (including sorting). + // without_replica_labels are replica labels which have to be excluded from series set results. + // The sorting requirement has to be preserved, so series should be sorted without those labels. // If the requested label is NOT a replica label (labels that identify replication group) it should be not affected by // this setting (label should be included in sorting and response). // It is the server responsibility to detect and track what is replica label and what is not. diff --git a/pkg/testutil/teststore/cient.go b/pkg/store/storepb/testutil/client.go similarity index 89% rename from pkg/testutil/teststore/cient.go rename to pkg/store/storepb/testutil/client.go index 6c72225205..b6916005a3 100644 --- a/pkg/testutil/teststore/cient.go +++ b/pkg/store/storepb/testutil/client.go @@ -1,4 +1,7 @@ -package teststore +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package storetestutil import ( "github.com/prometheus/prometheus/model/labels" From e6bdf1cb74b54e7f740aedbc54c2f6fee301278b Mon Sep 17 00:00:00 2001 From: Bartlomiej Plotka Date: Mon, 2 Jan 2023 17:23:25 +0100 Subject: [PATCH 8/9] Apply suggestions from code review Co-authored-by: Filip Petkovski Signed-off-by: Bartlomiej Plotka --- docs/proposals-accepted/20221129-avoid-global-sort.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/proposals-accepted/20221129-avoid-global-sort.md b/docs/proposals-accepted/20221129-avoid-global-sort.md index d2f7e8821b..aff9394e34 100644 --- a/docs/proposals-accepted/20221129-avoid-global-sort.md +++ b/docs/proposals-accepted/20221129-avoid-global-sort.md @@ -102,7 +102,7 @@ message StoreInfo { int64 max_time = 2; bool supports_sharding = 3; - // replica_aware means this store supports without_replica_labels of StoreAPI.Series. + // supports_without_replica_labels means this store supports without_replica_labels of StoreAPI.Series. bool supports_without_replica_labels = 5; } ``` From 9752d723ada214f3d89b70787d8b881c5c3a9ef3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Giedrius=20Statkevi=C4=8Dius?= Date: Mon, 13 Feb 2023 18:46:15 +0200 Subject: [PATCH 9/9] test/e2e: fix test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Ensure labels are ordered in each time series. Signed-off-by: Giedrius Statkevičius --- test/e2e/query_frontend_test.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/test/e2e/query_frontend_test.go b/test/e2e/query_frontend_test.go index 1d59420748..c3f0b8756c 100644 --- a/test/e2e/query_frontend_test.go +++ b/test/e2e/query_frontend_test.go @@ -6,6 +6,7 @@ package e2e_test import ( "context" "reflect" + "sort" "testing" "time" @@ -713,6 +714,13 @@ func TestInstantQueryShardingWithRandomData(t *testing.T) { {{Name: labels.MetricName, Value: "http_requests_total"}, {Name: "pod", Value: "6"}, {Name: "handler", Value: "/metrics"}}, } + // Ensure labels are ordered. + for _, ts := range timeSeries { + sort.Slice(ts, func(i, j int) bool { + return ts[i].Name < ts[j].Name + }) + } + startTime := now.Time().Add(-1 * time.Hour) endTime := now.Time().Add(1 * time.Hour) _, err = e2eutil.CreateBlock(ctx, prom.Dir(), timeSeries, 20, timestamp.FromTime(startTime), timestamp.FromTime(endTime), nil, 0, metadata.NoneFunc)