From 6eee9b1049d0212173f0af6306fb7e5a2cab515e Mon Sep 17 00:00:00 2001 From: Lokesh Khurana Date: Mon, 21 Dec 2020 15:33:36 +0530 Subject: [PATCH] HBASE-24620 : Add a ClusterManager which submits command to ZooKeeper and its Agent which picks and execute those Commands (#2299) Signed-off-by: Aman Poonia Signed-off-by: Viraj Jasani --- bin/chaos-daemon.sh | 140 +++++ ...hich_Submits_Command_Through_ZooKeeper.pdf | Bin 0 -> 270679 bytes .../apache/hadoop/hbase/chaos/ChaosAgent.java | 591 ++++++++++++++++++ .../hadoop/hbase/chaos/ChaosConstants.java | 77 +++ .../hadoop/hbase/chaos/ChaosService.java | 138 ++++ .../apache/hadoop/hbase/chaos/ChaosUtils.java | 49 ++ .../apache/hadoop/hbase/ChaosZKClient.java | 332 ++++++++++ .../hadoop/hbase/ZNodeClusterManager.java | 120 ++++ 8 files changed, 1447 insertions(+) create mode 100644 bin/chaos-daemon.sh create mode 100644 dev-support/design-docs/HBASE-24620_New_ClusterManager_And_Agent_Which_Submits_Command_Through_ZooKeeper.pdf create mode 100644 hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosAgent.java create mode 100644 hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosConstants.java create mode 100644 hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosService.java create mode 100644 hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosUtils.java create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/ChaosZKClient.java create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/ZNodeClusterManager.java diff --git a/bin/chaos-daemon.sh b/bin/chaos-daemon.sh new file mode 100644 index 00000000000..084e519321a --- /dev/null +++ b/bin/chaos-daemon.sh @@ -0,0 +1,140 @@ +#!/usr/bin/env bash +# +#/** +# * Licensed to the Apache Software Foundation (ASF) under one +# * or more contributor license agreements. See the NOTICE file +# * distributed with this work for additional information +# * regarding copyright ownership. The ASF licenses this file +# * to you under the Apache License, Version 2.0 (the +# * "License"); you may not use this file except in compliance +# * with the License. You may obtain a copy of the License at +# * +# * http://www.apache.org/licenses/LICENSE-2.0 +# * +# * Unless required by applicable law or agreed to in writing, software +# * distributed under the License is distributed on an "AS IS" BASIS, +# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# * See the License for the specific language governing permissions and +# * limitations under the License. +# */ +# + +usage="Usage: chaos-daemon.sh (start|stop) chaosagent" + +# if no args specified, show usage +if [ $# -le 1 ]; then + echo "$usage" + exit 1 +fi + +# get arguments +startStop=$1 +shift + +command=$1 +shift + +check_before_start(){ + #ckeck if the process is not running + mkdir -p "$HBASE_PID_DIR" + if [ -f "$CHAOS_PID" ]; then + if kill -0 "$(cat "$CHAOS_PID")" > /dev/null 2>&1; then + echo "$command" running as process "$(cat "$CHAOS_PID")". Stop it first. + exit 1 + fi + fi +} + +bin=`dirname "${BASH_SOURCE-$0}"` +bin=$(cd "$bin">/dev/null || exit; pwd) + +. "$bin"/hbase-config.sh +. "$bin"/hbase-common.sh + +CLASSPATH=$HBASE_CONF_DIR +for f in ../lib/*.jar; do + CLASSPATH=${CLASSPATH}:$f +done + +# get log directory +if [ "$HBASE_LOG_DIR" = "" ]; then + export HBASE_LOG_DIR="$HBASE_HOME/logs" +fi + +if [ "$HBASE_PID_DIR" = "" ]; then + HBASE_PID_DIR=/tmp +fi + +if [ "$HBASE_IDENT_STRING" = "" ]; then + export HBASE_IDENT_STRING="$USER" +fi + +if [ "$JAVA_HOME" != "" ]; then + #echo "run java in $JAVA_HOME" + JAVA_HOME=$JAVA_HOME +fi +if [ "$JAVA_HOME" = "" ]; then + echo "Error: JAVA_HOME is not set." + exit 1 +fi + +export HBASE_LOG_PREFIX=hbase-$HBASE_IDENT_STRING-$command-$HOSTNAME +export CHAOS_LOGFILE=$HBASE_LOG_PREFIX.log + +if [ -z "${HBASE_ROOT_LOGGER}" ]; then +export HBASE_ROOT_LOGGER=${HBASE_ROOT_LOGGER:-"INFO,RFA"} +fi + +if [ -z "${HBASE_SECURITY_LOGGER}" ]; then +export HBASE_SECURITY_LOGGER=${HBASE_SECURITY_LOGGER:-"INFO,RFAS"} +fi + +CHAOS_LOGLOG=${CHAOS_LOGLOG:-"${HBASE_LOG_DIR}/${CHAOS_LOGFILE}"} +CHAOS_PID=$HBASE_PID_DIR/hbase-$HBASE_IDENT_STRING-$command.pid + +if [ -z "$CHAOS_JAVA_OPTS" ]; then + CHAOS_JAVA_OPTS="-Xms1024m -Xmx4096m" +fi + +case $startStop in + +(start) + check_before_start + echo running $command + CMD="${JAVA_HOME}/bin/java -Dapp.home=${HBASE_CONF_DIR}/../ ${CHAOS_JAVA_OPTS} -cp ${CLASSPATH} org.apache.hadoop.hbase.chaos.ChaosService -$command start &>> ${CHAOS_LOGLOG} &" + + eval $CMD + PID=$(echo $!) + echo ${PID} >${CHAOS_PID} + + echo "Chaos ${1} process Started with ${PID} !" + now=$(date) + echo "${now} Chaos ${1} process Started with ${PID} !" >>${CHAOS_LOGLOG} + ;; + +(stop) + echo stopping $command + if [ -f $CHAOS_PID ]; then + pidToKill=`cat $CHAOS_PID` + # kill -0 == see if the PID exists + if kill -0 $pidToKill > /dev/null 2>&1; then + echo -n stopping $command + echo "`date` Terminating $command" >> $CHAOS_LOGLOG + kill $pidToKill > /dev/null 2>&1 + waitForProcessEnd $pidToKill $command + else + retval=$? + echo no $command to stop because kill -0 of pid $pidToKill failed with status $retval + fi + else + echo no $command to stop because no pid file $CHAOS_PID + fi + rm -f $CHAOS_PID + ;; + +(*) + echo $usage + exit 1 + ;; + +esac diff --git a/dev-support/design-docs/HBASE-24620_New_ClusterManager_And_Agent_Which_Submits_Command_Through_ZooKeeper.pdf b/dev-support/design-docs/HBASE-24620_New_ClusterManager_And_Agent_Which_Submits_Command_Through_ZooKeeper.pdf new file mode 100644 index 0000000000000000000000000000000000000000..fe35c04ebbc369b322bbd7a92a5f1cdde157976a GIT binary patch literal 270679 zcmd41W0WObw=P&|tJ2y$ zsee*AeFtktz+VsqZLOTGZ5#nC06IAbV6&yNoA%G^A ziGc|-6T7i711qZ$i@u2wGb;-{6FWV-@ef00Mpk1Mp8sbF?Y}B;baF7(w}y7hG&C^M z+cDKM(9`=6?_K|AE5)SNQ(_RfhhT z=Kq6Na{8vmj(?5hUzYKYaSG}?>08;F{!esa`1|}nn*47~RT04U$14AA&af(Xu*yZS0kCr4OzP=tfYYcEf;c1Gw;1TJwKaNLH?PN7=kKDtX{a3Qj+w?C zNXdE*F%bC>BQP-*F`qFvDm?nYFw|_xt|>e9$;+t?GJy{qimBz~x(5-5ioPj8Qc7M#z_AU7SewRrY~UomqveU zSpV>h?GID`i2fnuAIbjBgH_=2{08D?U zqVO-eiu@VwUlJk=^Z;h&e})ucU;r@x^?H9uD#8q4`nOg@SOLuco|Fh1fSKXn0zUxE zjQ`HE1DKiqmHq1pi>5dIp+|6~RKLgxS03jVEsCU(~UY5*77t5pY0NZ#GLv(rF?j=Fw; zo()W0UNQ^-e(VZ3@DIMsF9?CgLbTNbb7k{0JVUM_>bXM|;&OOdRLNVyxoY0=`;3_1 zx6|8OpKnXN_6pkg@1J+;yq``lvL0*ho*%2H)At3pb6>A-vMIZC-CwVd{j#sOXJ1t; zpr^D7YRjrO1YTjuz=UujOyX3qo zNV=O2eRQO#nifRwxZ_xR8Cj+r`u;lHk+DE~zm=G66v@-*K(d zygs74qHi_lOi7MD@jeu!bt3WHI#F_lO~O$f)nU8uLmL*qOyHt1fS62jmDWf;oKyg3 z*1K9uJ8NED*|ds{aOC>E@H`+{V^Hdu)wAGLXPb4erZ(}y=(oCpY4g|+OCD!sfKpTX zNZWW#*h`QO=t6bvFxByjM~+gKbeqlirNwdpt9-3my|C3*eejiWS2)jhnM3J&MM&hh z5KeyVQ&JJbbri7u!c@2n`hwmSh1TTLdLg7G8NOY@Cd1A-%GOaxDy9rBxJ{b_0?IiM=HP$$vZ$S~`0 z)W<^lz1Br~xstw4PV=e|nSzHFx&E4Q=+Pqj=tJxk9u=V)!&XvCsq4Vj0q>#ucc?D^ zHpFNVUw+|*n4Yz&R1+?sL7RN%R<&<-CUwHJ;Qp%d^%%fQQIbh&TqGo%Ffn0O>z#$ooB+XT$8VzXKoOa|YjkVC{Gefn)Gm9GM zy0jPguIx(s8Pz!TVB6f#4*0k$K+HX3e`WS6gho^itOn-`QiF!*r3POYsz5K^V_v{f zAn1m36bd?`d@xw+#Cl%%*@*`_uoiy-Z-Cpcxq4{e%~F;aHF61%RVMbSLW-Bju$w#N z_eIrp5kU|6r^&dbkuKoN&9{tTOZUiwUIRZZIhG6b`|ac1C=`&kYTAmvEn$Frz7nRx zEeyCRiF?vQ9%~ftID-ujL0>lWHTgH3y=J9XAEXiH$D$Qh=4V!=r>Qy3uIed`EF6s# z3{QDlc4i+kC^Rgi%N!pQg-U?R863<(K#iSOu+?n`PAqm57D7z#ziSOxwcSOGp>K#Y zY7{pX7T8WJNOvb339ZlkV$m&-F_JRtTKs<9vV0r$9S9N@m3%FF)xY-}e?IUFDIIh! zZPakakix(%ALS{?$SC*>Md1dta^_DEl|(8qL#hrMWsZIDD`7AzZ0FIY!du|MJJ}ug zE%lN$?}sXftOejmqd5SPdg$ba3M?{G1=OU+%;bGzxD48J zgXk0nixm;h!2eBTl($NWp($S68vGlQl2rnP*o?))6;BpKTx`VRYFSzDCO0oXGY;A_ zNag#Qnwd+F$1aF2Gqc(}3Pvv<{YWvPxnofojFFkp;8r}D)4|l}ogQzquMgdv+32!PL3&sWd8X_Hn0bVNx*tq_l=r&9xFf z9YpXR5RxFH8p?ro{vROnFnEwhDUJT zbMyvtlxVi*?rW5o5IG!}26J9IYA)dlFtJ^%$zi@rBtbQL_bJ!0Coa3DZBrk+88ioJ z63Tt$dA`NY35y0Cylh{q<6xP-k{nPi(}ZU$G#>Dz=>{-ovW9yOr5|PTDPAohzk!LS zgPV@z_+H+o6inw@jOg=av2S~&Xiq;}NW!RS;18VXMGUcP*@!b#0%J3N+ESGC@3{-1 z9!d-%!TZaVP}AElBpGy`#K$Y!k7wJe)b-IRTA?ckPr|luEze#m0|meTNb71f6o`@k z;f&}a{SEO!nie6VZ8Tk{+8RD2S^QVlqIZe~s^j+>WyShbg#*tmamq-M`C9MQ4d9m3 zE9%QYeT=wP!Z>6%9w0hI&TDlJmz{l(Lg+Zm47N?W1IJ)N z%%QxxP=D=7`UYe|De{dE04y@P0*>eU$?~R~mdiD+kp9RH!Nt#ohjc^+v#6FHJ<+gQ z3VKT29Xra@F8jy?q_D~MGN^U`_G(t}58OVPOVscl2muIkOo`A(hfz9>kD#d6{Io+} zbh{R8u=L@yW1AS%1?UY$t6{P((r@cKAjG6_G}(TFEx?QLK61PM(qPJRaL-QWP<$B&k)PO^&~(e&yc4I z2l{F;&Kf!rcnzOE+wC{ZG3jn;A1cD`G3o^Tzx8 z%5os^j0PqY8wUIb7hpcg+IpjSZI+Sqowyu8HxDD@5u68$4!E@i`e?D}p$~gUxa?69 zN_`J%m9Dx|0$k`-nh!WFK9=bF+wz{vk|2O{l7QN+4tT)QRqVTy7;VCLNA7NJI#?cJ zTjg73F`0HBA*chtRMb2Ju^RxjNUMN={x<@@y)EA=fr9BqRyT=C(m;F$S0Mw;qJwimv(z@PLw>iA6kZ!%P| zI6TE`TN?Z*Tz+8jngejaB5L|bp6zwD#F6N>G}dgtz3F{BIRWd2Gl^}SGl#O!rp2*|b*_&O++d*J z7QsbnjopSgUboBBsHnz2tg!OPYI`;#L@&t9NO}h{FkcxRxmgA4%>Z#AoY@dO7qdGs z`$1YwBN1UImk$ITHP2I!Z1vU3jzrEN@P3_wd`Ybk(1^ugL8kSrVDma=Q8Xq4Y~W+@ z=(&WmM;|@y?XyKt8*|OLocrlcklFsZhc}7mAa*Yz(_?v(u9U^F_8MV*U-}A4LPWt? z6Jl$ZmVw;KVb?d&JD)P~drgFYIaKM8%*#FJ{Fy(%V54z57xNA(8rejK=j(l=TBm^{ zQ5v$rb#O}3X#O`#%I;T}?)>&yV+6Xz%sgz@lf#wphGmM+)+@`}6K&@k#usiV@8s~c zSArSay%rR^HhDUPUD+j+!5$A@EaLlB5e<$9_Bm+B>G)R{_7{^g)k-3Q3w9#0-nv*a zx49E5O}|VWNd1&gxxG@g8;q&quf7OJsK?`4rq&)l9v5IFo4V*EA0BYFUZ>_eWMGFB z@TcPM6Y1_k&xge$AYCb$efF);2li7Zp0!kV#&ljhmC1OV+WXKlao9iGmYcVdDZ(&{ zuX-ljPP8yw9Qm^lMM9Av*6{;&zui;hfwE#q38aHWM0}Wk_n9FDHhrJA%X|Z3)!5^X z0HbDrh6Ze0G13Srxpb#G540rAZRA_ht`nARZ8n28ra~3U_anr+=KCp`5xsbQ>CngE zdLZ;7LXIALVzq>0#>hH{6{L1Yg5@jAaO!D7(!#Td#o&3u#z8{@Z899wIqILM&|^wy zN`l_&`JL^bnVEW}^-1X#Lr#ZZK#?|)1|%!Gbz>m&I6_?c8a7=7O?Os8QVjCY`<%xg z!s8+Y9oUtKBMH>fxL6z7uq}{%rckBkNc995>vRn^`|Oe+m}n%}Qq5i{Qt+ACt?q+= ziCp&lfpcULvpwKw?V9$C>n zRp~PDJEL+foe@3Y@*39djiIe&T&I5XN6@9q4czk4h43dscf;}{Pekx}j&^@!D|l*m zjN~P+iu>jA`p&#g^zM*T;l^F^5>3{vF#X253rz-_A!`~Mt4D{%afX)z zWrlJN7434>rsw2)>L9{c0{qME?y9?RI|)JY@r`;5Bxl_Rhn2VL`kan40R^(xsrFO~ zOW%SYsEO{U^rAyIspwo3dnWKv%1^CF!^rGoltIs%hW@77s-JrvOVQI%5fMGEPrhXJ z>L33t8ulj^`kyh8|7|?ypPl^QaKxX`!+*qe7}?pG|4U$JFvZz9*EHEEew2{_ zQZb9uCSi`)KsPXMLWnR~XeLe0#GtMb%3L5QKM%#Q)TT{gQQG_=q)C8HNx8JiNl;Q* zv%ySRe(knnx|7N4n)a*9^~meWJ=t^0aY}p29q>&J16qo*HJH4V4v{D9f@0r-Ou|Jq11tG#1%gDmC<)@Yy9SHC)+$r!cX_D* zmC(1@l0_W}s^Y|oq~u!epc@u)Txz$y6{6AYJY*qgwKY1lgl?m$XflybS+yl z3^%-!Cm7O4kVfLz@@3G8fqRLtPo6hLbDx9lxUkito%gu_BHyA%cu5vOd;6Yx3QUM3 z?b|oOHNAvk!fbGggFq9FInD(GlRkqVCdie=kIY=KJ*#BMl|FCIEC~D&a%3jnoey@0 z@_i;kCZ9xuNwZgkOwaf;AOkQvZ5GAqgu^CivX(mhV@QZ&>@H1+HxJIH2%o^R1MK1i zf_RUWBY>EA{;4I&9m6_KVml9ajmcIJf+NVtsW4EC-}Du&^h97gm%EK|ekKY&7o=_8 z$4LlbEmY=2T(^M9gP}c_%{#bCtb}eB&ie$$NmTb0hkGXGF`o=)LT4nG`y@*0753C_ zw&Wy``W4hWZt{G10}jIB#PEKYZ!H)WHa}}*;bcJX6_OS<4|hZW&pQlg;$lRE$J-AF z))0QcZY@MBGWUrAlE|D2ykAh09yE9&Z(?$!pWFlCr6q7{kUlccRW8^5$I#q51`vK{ za2$EQbW2t%l1*fRh@0F5wGfnb4|(ET21b;F2Z-yAj0divTrbYJ*0k)An+60vsT@dz zoG|o|`F%mUjv#}D8I1&eoQ^QX9cp4;@=`8NLYixSw8}oMR9=d_zo!{0B}0@>pPQ7a zOi$-MVVRsP-Hy#f;jw&_`aO$-IPAAC3oFV@r;T9^=s*)31ImjcAJ0>C6bQ z2G}S1oFQiF{1STHpQqvzJ>QQyC&->By~6N@rMRSG^wIE&idtm%t*DDc86aY2iCXYw zXZlmey&&(diMhjG+|n_77&QXGCLn4M$IThgcGWe42GvHr{8|wuuW@L40^Q<(-5}{^ zy|^lFkSg-s-B8NSX_F1$He-fU9j|}r6s@`eTc*kGv9uw`GMMg~GtWp-_j{BLJ9JEW zg}BqH>`15ko7Val8{sNxOWIY?SOSiE8Y-p~g0^rqrgjVrl+D zv#d!X-9_z?MZTeIl80L|;Jjn;5?hq33A_{m;mN10=$A?NdsrFt!fiL_+Ug-w3TL~) zRMrEzqh&Tg*bI+a8Q%z^`-Nl6Fn-P2RjAMrSp5t1%7FKd+k}C2|JzF~*!Ij1t3iIs zzBjDqUBs7bpe{1HrhUk$!O#hY6@UB_c(0(GR8IRu5kquG6rLdl$^*LX>4RRf=inC# znON+cUE{UOyIO=}3WT>Cvb%4zGjd-2 zI03dd=x{^)@6v&GkYDuO{Ul?9`iO7?Pr!+RL;`ZGK8PGgF#@t6*9r7;XwlF_t#(pD z3CP32;fy!+VS1m0gcKlTL20l1g<*scGT~;wLv@8=@7Rzez3nvyq>I1g@5&_0W9NM+fJulz!$h2cy{ z#cibP(s}zWdgDRKudNJMMx7QO@P6rKru?Id7v$R29qh~HI)OP#GOIXk(DofZ5wi)~ zR0}@$+<^PNZN9Dll{O^?AdAywfAR8#H)LgqC)ydaou2fbjepEQ4v=QT_w0+IcL!U@ zY2kwM=~sJH8@$@2+yoM>e(%T=1V2P~GT`jRxdRs`W z7Y=A1ur>uf|B$q5U}x+@!82h#aR#QKWgn-a5Z4)uB{nrJDXTj-48Fn&R*h3ja_)>F zbZ22Z^^k?|Oy1Zzh4kR4@__!C7=6EhGD;#jW>Z4V64B?ywv5hh98;J%_dBq$jX)eZ zU%h7i1$s0|?kEvuH%UuE#ag}R(AmKg?r21cz5WLy3m9CSNmY7laYIr$me6{aK>5j9 zn+MB@UW7`!@_jgMj_&BVYkQ_j`)0vuY-wXrkTK=1hNeO*789Bn97P5Q3^on zakp)Q=Iq;ZzTXxqtnT((UuP3rjLs6g*08?!s|OhOS#3ZD5SC>D--%Z;KPZBzW_s16MtA7AE zvwGdJFI8}4s(yIo)wW@A6xMc6j|Y;gmJn=L7LF=QbYvnq=>->V1>OQTyNI7KlJF?{ zyK0?Kl1v^d?Z;~YRWo7at|ez)ZH;8)_rqxupS@0X?B4vcC!b_`DJE_Ow($)&xYPJ} z_evJCDc81lb9?2gYhNkYB3aFE;GEc8ZNzIVce7%1)3sL-#mhE}-{2_UD%he?5mSd{ zC(43D>(WmDwKW^*G)hy^vNzNj#I>;Id{L7tJ29am?eQ=-VO73qh=BJ+L~Kb;YM5`( zR!jp!RW*cie%u{`cCP|rLDm|PwG8T}T`(l)m$&rkn91QxD}Z~4O8VepqRViK{4!~r z6`e7w^jljh7Ue+Prd9263n2l2|xYAzPrtCX-TvPafWnz|uSw?beWkx)j zRt%kXM=PsWb~WjA$NuC!xZm!}vB*dDg0xGbS7CCF<&ZW6ZYK*~Qo7Z=Q`W%+N&Qv} zq8Ny@ZoBh&+m}A=!fQ6;d7HSLPSN$lBy0QU4<1@&%e#zp{$IokXVI3G4Lb&5^jJuv zgE(j*Yfl&kEG1x|MXkY4PSSn31@5f?FTZssW-)VsTO#R$@jPC6_u%5 z`bTPGX)wz&+t{in@3c-%ds*BcXYnn~4pZD^t=(9(nT|EI36_f+%@X*Q0Sry1lD;;z z0n>{WREHHtgt|o%nYCv7rN*zTOo?o|8bwDtUp_x92w`uU)B=ruvJ$WK!MUt&8q-+K zm#kVJ8oR7VU>XcE*XBzFVJ29K2*%ThEH4xHm#ISmfT} z5dF(?CLC7SH(`@Dk+gLLH#?6R)r)pS&MD32fb!3aZBkf={3SR6nrB68l)-2tv`+02 z3pA`M>>ZY~mqon#kFia1jj|#3r6@#Ti7MAA!kGAFjhkb%mZaAuyrKjR^|`gc^z?&+ z1FVOcDXsXxh=Uz}U!Y(oy!ZdU5%~KB@&DwL9wQ^ue|huQj^(!)pobNH;t5V>`q?YK zL@*;mVKi%Qz6SE6DIN=Jm3)e@x%xbWLP;<1{R??}Zuzs6EQhb#z6OLem5Y6d#mWjv zB$KxAb5UH=mY-&4p;LiekdG^Mz=SdfU+BeF{J+EW z_v`Ba0Za@4Mn(qu|J*P!F#b84@LzDd@PhJ6TwHu=oo2EmO>k#SBTjfoqe(C#CD3P7 zhZ;?Qi%UfWO#;N#2(uCZ3E6`vHSCF;(Q4Q?uO;i}{}8JzeYL2BQES{&=#kKDBy12q z6FGX@=43FF-i~>C>)O8h@VTmOrY8i<$o zd_AqiWh(jdYQ;SS6Dnn!;x)DDZqS?}uJet4G`?G|aG8{T*#XF7tw|Smeb!IESgswH zhdU%Aos#8nyDIatk4PPOB*r*A)-`%}<8(bp6S@YG_cxpMvLE}ppC3r%|6ZQ`$`gh; zrgdY`ePUgkhVWe^R4UBMd{-sBAWZcBIi>FHGuW6)Zm#)~nOvF5+x&Ll${qVfO^MHK zdinBE+-OcCG?_Wt=yk~sGfviA(77r~oVC?$u=u+rCY!+G@K)=C3jJ~V`Tf+8eB1wl zTSQ8S@FU`1(9{G`do;>Gm*kb-ubBDuxRpW3 z6yR$3^@tV+zeKR4zp+AD6?^ExNJoApX_Nx3h(z^>7Xr!V7e6DVQ(qfd0LkR3=y5gz z$s(=rC+vlmCddHG6eaBE$e~};OBYa#Rj zWblA&N$>&o64;^jApRu%2=IaSiuVD3FX|~dIqo5d_~t9TzTaI?0>&dKzf;c)`HrxI zeUHNp?v?HG%~v`GL>KWHJ7Q3`1pO3hN3sFtxvod`1pQ{&jEx&=L$DjPOG0+gSF*c^ z;6S%Pm*A_dhwa21AOg}Q{spuFWWzZ^m}*?GIs&a2<0`c`bXt{!Ddm>H+r*)CvC{yYCglXO6Xcz$6XqTJgXfd*BkPgCH}R3^tF&kOgnNhg1fgg8^yAs~1jb)? zj(UUX;qo@|HV)$tt+s~Ex0{uS}rHskZ3i+8$s`{v)-t>Tn-goe}5FYEcg7jDUy zui5h_^mvyCwqp6^$C z&Xo^cqTBs)Tp6!$q1KDYigVv2V1qHjm!6B{v(bmbJv3ziVZ>Z$B1Mcy7U;47!PAwg zp*U10(a6GZpI#Ezu}@X3B6%$h7$M(OyyjYGWq|fMnc`m5Bp5HIBxZ$ptp{2U`N}Wl zBxu;YFKc5=CSY=vuex@y1uX3zaejwHMnA!RySNvgCz=8B@eQpltEgu?voAgs-E|5C z+li#%&2^<;-KpG%s?e#ojW&iCp7Vt=V{j_+xKD&Gw5S!;^-i2)#HT03>D7&nYlur@ zG?F?Bbu?uNKOgHu=g#oPG*1xv`6^@~rYP6udqtfH{)w6w(i8xWrv$k2X{WezJhMW< z#y>?PxYtRJ8-5s!D|-u zxHJCc2Gc#dSjK?K$T2M25U0Hfsarp5q!YW<)^x3-?EW2fJp&sjX1wp_js6qO@Bs_l(c<=e6g?FmSJxwBVCg&?PS-YUJ(=-#kHTZ0_-y z`Z(uW%hiW-b@OJg5H8Iw^^#MYCGLDrji*K)<*)ne@_vY#Gc6f*=1!2g*Bxov0Oz0j z%UdRltNM?@2Z>d2Hi<6cUj!G)k0b65 zd^Sl}@w#)Z&bDV=j8ojnZ0`%#yDcPmk*Wi1!$_L_+qa4%!9#3&$&&%DS}}{ZAtf!y zo}c8@8`v}LAA637OS??35tzBs>Zx&|?&<_oy}zgQm{#4bklXz?hqDc@Y6H(L!wZc@ z17-Qj;)})VizJI;imDHZSE<-WkH?!wx|6yWDi=eq-G#nVKf`u&YkThJmNla);buCRH(KfQlSW(gO?&1McIviHmKW{!~ zY0BxG<{3;EqfAOKOsL7HJ(Vhj$YG!JG7ofWyeH?W7VA#fiu(k3Q&2gSRg48YWOy~b zgSaQ)#Nj9Drfko=I7_#UwiUGs*KY~u+@on z8A7q?gTt6bqa@@cUUrz(zeBu~%oPc>smyXAshpFzO{CmcpUyqt&)g}ndI8sBAGXfX z6|EDKmB*8vu>RHwSXBS%zD0GB5;@0ecimzMpEXo>V%s*q%kLxI{*X~ql2GVdzMiFYBeEDsv<9r3Czw&kQ*7%&M6lyq)g8E6xCQ3ihHn?)ng*mv1A6ZX$AnFsQ zbAoGSa2eK?NTe-0J9fU5uS-{hZtzg|Oe?G&exyWi;irC%gE~h16#sxYhw8g*{}X@T zccD`jE4++g@;4kad!KiliFcqCLK+@UpNuX|{6f_g3hJ!vfCmQyblatjPz3)w`Hg;8 zGsgCG@KAJs%M0XVy*WS!HLtskHQNcA9Ei1m~4blfl z+g9YdV!+#ckh)X|f+nf+xU2%qxJbntR);E!jS9NWC$L-Eojz`LuXh_h&jBh&9||k> z?7YrKrqssW3&u7m`$gFPvELhz+VJ=azyfUtm%`8cHNL0@%baCs-{sL{WtCv!R9lEx zl$`cRxsS9)C3DMb0HI|ZBQK(D#ic{^+>g{rq5S=OJ_de=CDY}J_sKHMrdV%=TIHuV zsr;y&X@`ggHSZ|n4xD1HGSki~SslrqZSgDt`2Ke12G0UE#*D>a|d3tn+<9pP&0xBfIAca6c&AJ< zMpA=2*+U!Q=T_L%RTfukVoIl@Jj0VFY9|tU8L4*)+1_tdtEwvpi=Fc~G8vlQ(kKVU zL1o#8B9bcBYN}g@@?})R`r=8U&ovd>W-14<15P2I`Vq}I*Y?@o#UrhS-w`!Bs+vY~ z>B1Rn9-0I;?Ifjx>vi;{4M9z0>amRg^wIlAThlZ+DWk%PrnZakX%(qT#Ov#8Xrs4H zx0pf)0Zpwigxd2aO^rt_o0~->NK1^{oOA+PM`1MNYQKbGkr^64Z5(})*3sF z%;s4bf{z}+6VTnzC$hMi2&XTO3=6>-Y%GcI0s^P@g&~`pCpvD|E7@<0S&t3S>1@`V zim2+G*CuS6+t^)P1;Jk>z!E&zr2A%;^^n&~+Ue69YCTPYtOsIgyS{go4ZNaxth-5H zeODbgw|=BuYbZn)P3WW$D5-K5t*>v)a?>5NakNfnm_LhVK5}b6ZaT7YohepahBq;< zs<>)OOtw;0K*nz89%PLdZ}Q=+w`-6-0Og!+%;9LOkDMOyWJDp{OTX%qGV!P_#)5~j zG@F0Ns@Xj?IYCwTLsrLOXuh{m`0d7hg+Gfc@({X0i7rtZrYk1 zS%$UNJt`W|D+cIM9#{!r8pOdCYOQN9l9p;4U$1WAtMOh4?#x=(Rt>puZGQpM#ho|9 zgN}QcHz6IPO;154sia=Pjq`9HpJAgpExAd4*lvh)xq1*tz(=+B_82(8|ClZ|Dr}@F z`&kqDRp_pt-|5}f?7dczR@EfkN#wcCv(H&1`=9I43F0k}P3;}ZECl$e06_U%OqWq)3h%cRq*2GWZ!7AJ^sO(b@$KX_UQ| zc;3~CZX(mZg@A;(bo`_hCU8S3R&(>hWo{zHc=;#T0_p_*uSnvr0G2q+=vOC)KPSq5 z$1J*1ik@*sFDeqGm`~78bVoMd6CEM6j(t zTOO&bOok1k120Djv+0deAT3|QS44j_PII}Fr_27a-nGKlaB|Fd<7?@%;8~MZisLzd-TQzPFWct<>a{Q!Qmlt`Pg`p@_>_a()WaSF!$C+B{LUi z>FD^$I_!8C%|}gQNf&8qp29T8yM6-!7OlXEQ_S6Tuf|_iIJT09*8A|02_Zulm}F7o zWmVy|n1Pq!mYEDHbqpBiTx<1=Bg(CcP*t!#OMzCh-YR9CO%qUuM+l-J(4Hohr(^c~ zAA@7{xiWE39D#e7x>Ft=Dz}D({s6;Er|RL6Y5K*uIC=TF%c=@UWz|3bVnRJxXQTZ1 zWn-oAHhP&2&C}$k8lTc%g`xx^}z(xYNhwCJ-Ohb?f_*eyVQ;1;9^H{F;y?V0PD zvg}cBS@@jYE@Lt!$@xK9@fL~HP2U>}D++6cROd7{45;K_3Ik%o#wMbUoR3N-IMBm{ zOa`gb46Ra%&EU?ki7WG(`7PYYvx<(d7g3#2S%w_dWMa#bEJ`^Fmpj0AQD2EXO0}v_ z+wVI)vUn^FCe3I1Z?N^9q90*FIpN`Oa3Md-L{Bk)Kzo;|Mn8DD)P!PTUo`eIHc&l` z%?geY9Ing;j^!;(tY!Nrxxm*5X$e{;p-V}4OL%iXfUb_+9MTKtrMGF>#uJ`dfw}v* z`VZ=!$&SiSmKde5=eTz`J2>-Dv@Uri9{??!kRBl0GX>>+Plvk$1!;814UJvlLwv=B zMe8Acq3Tx#OVV=X-cY+?i;7z-i;x445O@+?Oxy5zfl-8ms)~qpx+UU0(fDqdl%^rC z2;f}yy$u?gSLtf-+E1QHBo1ZaB;##pXtY$BTZ0O|teBfA)7Qluo+Wi`)bw;R8fPS>QclQB2%4qwb>t}OG&WP|YFVeyf>^v^Cy_^E8c1o!A7 z8eqhdV-B<(HI5Rh&%I5MMRU7MaGG){mLmr)*+CwJjtFX7bma@$w9K9q-Sy z7#*e9<%U+Q+hAE{5AM32_iyAF@m8Q8X(Yp_K+irU&tz08ItD1}U?6(n*p2HF0y(#JhfIHB?{hpURx{az% zwl{=pCm7kp-GTEICE-rc?XnpvxoTEXBw(uPzH%OdH->FlM`kzC_M%=nO{65ssZ?-{ zAy8E^ouxhMa#pk|mJ6f5S&F0lX^tQy@vUmj61*pEENc}BTBlANnNc*yS&|IcPR_cZM%V%DYI|*{h}E8|!ZL*-4KrMpu$< zniDbC-`$%=_nyzAGA+bYD=ir_xK`ECe_qYb*&8fzsGvJkF-uq#Sr{3MgoushqNXrg zo`BY{B(pGS)~`{s$huH=FWzVI0kC6+9#-5i2ypX33a%rN&N9IdZ+NKKWLLWP3kbZvbu z%8`3&35}$#gUTrTgMHN1!fYjcPGwm2?Yz<*2Ne}zsU*o%*xt{Z6(P{Fns)OfA15q0 z&*n(VtA2d(6`D$DFE5n6Rw*pEdS9-wJn49+&>Z8uL?7<|EF-0E(Qoq+s71KcT6bGp zyu8_;CdYSmNN%dTn7h5vMAv-~=x?pfP@(R&tSNmj?=CJ&LY*&ZFe*ASz5Yc8$#x>R zP%smlY^j@*zPe&*io{ueI?5MOrn-X+hn8azRx6A5K%Y}~jA>gZs~kJa*l}V$LngmB z1RhU=C8TC2G212~11{OG&#$e+&tj`#^20!XSrbDYMTi=`?6LAOOq;fzZLPW)VHIKJ z!DWh<2j|F1^t8p!;uvv?8RH!Dxk40&x{4*>dk|6726;Umo;6^?&wo+~gWn@)t+vIlx%^xZrg zuANRvG@-!{#}=livBzd#&TL3W!CM;?P4;Rn9(G`S2n??jZyNwxbfAX z$f`Yk-CG#OLd3v^7L%1!=k2*j)oE?#bNMd8aSGF}f&Fh82 zH2X>p8ao7riV4+|{_UP3IH^AvxlREF`v79uCA*=f^D%oDJ4G4Go5zrU$*aj(x$~`0 ze|!&12}28WoY+`=h4{V(mo95OFLba1acRM}wDCtgs(lo=eQK^R-h_OAZ=5Gk8&8=^ zgn^Dv{#r3)87c>3pG54Y)hbRRwffRR4Nc5#soJFCZIS_qsz(rMaDTv23+UqYB^ussjSm7*%w8OqEd&R$3x|D2@1-8%7g!VpwC{QkT<9cj>l^)guWd9Xk|Bu~{Zn7syS6|kGB4?h3eExy_cHTh z19MdQIWwvHUER-l?TJ(^6(*}p3}%v-@k0pRuieb1p=63m4#AKT`^#3_r3FfM^}6Yj z>Q25nSL1`d3#E=){6w8$%-}+*#?|Cory}_hwV${WI<-?fBX(S>*O^~5heU@kgK8Bh zpcaW`JxR$K9c(W?2Cd83^<2LjPZAzf6zx53#giY60!?P)Nj$}hXyV5(meq$kceFPU zYglQbnpVc;kZ9jLXG%@1N9F75TUkKIO*$?IR}>~Q8cFuAT-vwrnb+<iEuY*sqr1TbJGZVz0o}G(hvmva$^a|2kfwLlTgli$cfF=rKlVX zARHBeB;Wxg$a<4v^e+9=cmB6`ch*rawa@tA3Aj+u@G=E@=#c@oWT*N+FLoA>1LboN zXgl|@(GK?uQlCefaquBtuA3NXC>m>}n1}*)MMqbECm?9_UP)g>rHbm|DpWKTYCjs3u_)spD}B zDKxUT>WJ7mLr+tv=&9ev}m#}{!~eGKBc(g(3z9QmY@Lc%Deo3}^yM1}$> zs+W%x=R3B%a(>@i(0wWr~1=n}|I| zwfM?3Qn!2&;469~5&$=n~OD%<6GOSPrQd>GUwo9?{FYl`4J zHpE6-I4m)iH5Ns#O3xIlrx8E8`ob`eJpQ3NhfxV02fK$qFgnuhwbZ}9*Z9&+zV6Uh zSD9RUbmL39?xxk$`)!p~^A>89hjx}#gbLl1P*H%D47^}VDB@N(sWNib0!z`Q%z&F< zb=dT}+W4n86Qltn)>OZx38FoHXYTNh zZQHhO+qP}%j&;YjZQHhOd*(mhc`v)0_p;f|CY?H6l}@U1I-SBf->(%s*VP{YUryd& z7DkPl!QN7W%_5_$nxknPs5dV;i*wq?=v8Ba-K%V+xOAy9s^epld<1JTbK1-3RTP5V z>>P>egw@f}_l_g!BQH4zOrxJ@^;O|MOyL8v&BPRkivP1&e#BfQ*xSL;)M1rk40X`5 z`VuMO3VW*UE5urBUmCIAMs<|-_9!dvGIK7|E^3*XK+jwxVU?GJgd+=Buw*|jwy+}`+lSZA8)XupLWPs zL8?l@`1u7zr?PTFd%^nHnOwstRbR~M!8HCV4}z_1F&CC1GU9gtwB@k*(SAMXzIa=L zsabK^;ygO?iP;!5*>Xp@^KaTuam9(jI(@h^N>Najh*e)gEi~h3a9O% zKElFNZ2@wu8gI%adJYr*eyH07@jKc!fXWSfir1*p!~5F;cPfkF=as2Z!*5K(b*z@C z&IUGc%85VUVPvw1T0xWv<1@$Mxk}`9hn^q)*~oVc0cwyhRZ+Rs!!l*D;d=c}GwhMylDZ>Cp$6WF9-&gs`owg7P**r(6OGV0 z!n)u_>zoNI1M2^x4q-+Srh5OL%nE9XWc5CrDr*MZYiq(j<;WJvuc5>>{<6vtCJ5FbJc&&DbIyhMK#+U z_=ex(-gx52+aDF5_L#?vws6_3OMR~84 z|1k6G?a!AB7!C?WS{VXwCf_WVaKE1kNiaeie8n?5mf= z&80tO^srzqY$dem;cmB4$=?mA6(uMy!-f}&mvuqP)7Wk>W6)EwmRp`MAijfs?u-n`m^=IQT z6!W573?)!Dt45dD3oI}FSZIYpR_|tW4Z|Yn;)D#>3C=~9Zp z%F42ceQ`$E+K3o@iBdedXz>*yo)G6>5_Fbc@x+TMM!~wfro@nl6Y{G5n2L1so`qH7 zLgHd^QH@%(L+nMnwBXfZ5%^bP^LzY~^s$JIYbIzoOm}J}F7q~;I9g&CtDF{Dow6;m zJ+c}pmh+TQl7j1qum|kOK;>RjpT|$8ZJU*89h3Eju4yD+Ly2|Ofqr+!Y>O(LTXza0 zDLD0Ao(%>K3hP_{CKWCCemt7y$^=RRN+QysxV=@|vu7XF*M1w|C*Ae zgQ`*2NV8`;|~F!p4){Nksx%>UI{AzQ(=rIu21BPim@uN%SjSYjoN zS_F#_@-=_T1Y#Q?%lWII5R$wnhO_5VykYpkMEqCC1gcSeRabM_M45IRL%XomjgZ_8 zf>YPOq7=NuTc|cioh>3{q~bb5U&WfOQoLCk=2!{BKPl|t+8DJ8hs}_<8|ECq-1v^4 zy;TLD$A7U;l;XS3Y7%WnPFN4E=Rr%m_#4Fk_37DPm63W$dlI&Q~a=;FQL;k}@J=G}g-oUAhOs`PtQ_q(a%y{_ZEtlQqLn|@X=@u)0uZ!IMZrckAb zCE}9OuTDOz_r0p`KB@0Mt^Zm#>Dn;4a@Dxd);IGDdT6ZY0@)_4y}zqffzf*CUnV2I zgyCYcBzmBgFf>b0k&OVuTuik_6?PQk`or&7fFpy?F&CpGP+x#UB`}|l!&WL|IyT%) z#Hdh&+`Kht;mBMIG?fg(AoANR;Itq>lVv!x0Eli>ZLXU;j;_Gx6+cJhy&pfv@gC~g z?K}}b^z#RcHTr9Yr+|Pq@g?nhhA^w%Sb(q@OD1X-tN5FW-{l`OFVypb`L9GHIFa8X z_;5coPaHVn;kG$HXdBl$8|>l8TsIc~b<{HsNE?r4W7I@PbVP8b*4=vFS-H6wY&v*8 zj<%p*V*#|WNMiw_u|?yaN!-v}Xi(?Hnl)FDYXk3}ziHFa&!Ds1X2t|O%EakrN=0$r zij6^bW5l$DW8>?hsPp5R?*1d`Ypw($kznI}%kVG&0a8-eYTk7;kg{-8gIEoh#?z72 zw(@wOh}YMTz*l-0+EFYRZeUWDbv5V&yeq5UV|zRdWsUI)SBv4%RVrp zmZ(o#y#h~Jys>W?zt|hGpF(=spD?~PSyUUPa4tvHT8s?iyFJdEc?tGTe4Qi9;ZTN% zL$|fwWfkmTK$f&V7iW)D57DrRTZO5L z(qqNMiDSm}XVQq@%i4~&eofH#H>lWPAxJ?n1+s@I;Y5CUCLbw}q~XQ##4@%i>m_Vr z( zL`QRqx`)TlXG}lrfs2?RH`2_7Z7lLb57jyaO=5Cm)Ahe($9WznGA&{nE3TxK29c^Q z{S5roQUT3BtmnHR*7MeuyVj*SpERt^I7?Lwze_Lky%k)G68}|D9rs+F5HedPj5kI|hOgzWoL2rB#%yD12J1OM9 znuQqYIRMD!wk*mMpfR?T1QjAG`Ba)Wpaovg=PemZA8t`w~shOp&%f;qJr5v{01KI=3lanrr*+YM&$d0R)KOM>fmCp5=Zx% z!NX56YV}&ZE3p{qD{q^fSZc23qf#V z_Q|e!doqg-E3%M~!zuFRf)E99TYCk!0eG;?U}6neZTb#{)-`DsiOe=Vv`8m1F_Cjj zd=Iy+p>z~jv}!PK{IQBX^WeD<{3q1&Mum*?B#J0!sAVw9rt#VRARBsXH#6_KCK$#h-Yw&7E|>3fK7(Ggm$p%n z#FqrddrI(#FDe#AT@yKY?$6O^|EFvJtIp^D6Gms`VBlc=-!wV{J3Tw=fAjTKmxsHu z$TGWk`StdDx5?SHbAs%=F~M0J2yj~fF)#u|1tcaUS1SUmf?04Y;cj37At$ARn7w_T z1UynQ!}U8<%tUSh5z{^=BRwo3<=L0)*0r$qvCsC)i&r;a_f49X&dFkzN=dR($ux(G z&jJweFH{ij-omn>q86PtE9zeiy9*dh=cLRt@arqPpWMGLuAJzyw`f;|$^5U7k9$~N z=1~Itu-iXbesPJGISRb@N8;^%?fnuTSPyPMoz~rkRh9n*Y}TNPs_Xn3=w9&w%wG$c zYu%Ucfu{Goj`Bd>+lc#1M3>p`xl_${F2$%Dd;(4@7|wbIQ}%b`^PbOhex#cA^ZmoJ z+U*y_v&EW-)bVupw)Z=66G5n$O)wXETG^?x)EwWox~9 zZ;||8Um)}yK}Rig89QIO7nayFm$H}JBK)l_r@ZjBH*QcStAd({PIkS(mWiO{DemL6 z*f{p~!z>0rOeVhn9>d-gz*vzqnox%_PM9#BvKL5fmBb!9FW6rUptu8GatJbtPqsfw z1eyoy4DpmJ_ZS2O4H2y`Y(AfhAsxP;55y!`0cN8wVh)Za2)>H66QmSGYftDL1eFKA z8Gh3Xg&WKZQyjo{M^hLCX8=}6Fd6I|>^sVj?k^oIa7#29$iI&q4Db7UEiQx}ToA&E z0Lwr<$RjR@9?ua0m7m=KkuWX=fEj9D3;zcmRRF|*0a`#EI0PchvKBc4i7Xdd5?<7R zyx%eSJJ3Krq84}@UR4m;0OcPMcFZjmIuHs(MIPdwc&Q-!pFO@juuvl9JWKun~5~lX=6jI(mSoM%w%2w#TC3}v3D`s~Z z?SNZOfXcp|J&$&{Jxza4KTIv=9;bScJ&!Z~<}WpZPK6-0J{AbOP%EUcJ=9j%J<(R& zy(-KPm`WLMh?YJW2)nQ+h#P(Y+A0VZ{5B-pe&UopQh&#;X|S8sZA&e}9*;d$ztz?htt+ zHWOV2h`~Si7eKt=bAx#8F$BNwIt0HHBm1cK!HWP^QXLvAfjMFG&4f&l7%w_T3a z=vxnW#676lpj#1`Zji3bQgMdHp=cH&6kw6Se~H`>Tcw zwqK7KK!DBfjt)!sLDd;>Yj-;`2m}xh65Evt#v9Or;0thr_bu<6*_8|b4xHJ8@2A_{ z8oao5bBEhYxit$G+w%$j_J@PV4eG(?hQ%ZNAnRY*rDBh|)w^98tgr{>6$CVa$@L=y z@AQs9$n}vy^n}PE)%3dEV)WnaIR%&OK?Q%_x&=D~(+L503j^@7zN9!mgNqmB%sD>? zXPn=9n6tiWa?T!Wer!_lLeISO8y`#7mz+$=x6;Dd_u^00uXrKWit{t|$5{Md%&DLA z+jqR|_vyE0$+ybFnctH0n}JLT-tvDMztXcuxbK36Z#d_-?r*q@$Cdx6=q;U*v1Iuw zF8o(>&&T=s?Z)|8@O}FDpO^Em+%hF>aKU^7+*OVR_{A#j^QT{y(}asbMWqj+d9oE^ zIL@r+sicd!4t?(;#s+@Oo-{g}rlS^+; zeBHr3an(aManmRHN58Zm%Fy*P+57L&i}ilRMg7Qy>a@as3EzwVY9DhmkC#QD!-!Z< z7Q1$AC46%j;*@u@C1h=+r(MiBfQU6Av?qGC*;ib&BXUUMi*rxV&RUImZOOgN?#KNI zAv4D#HGDlAt($3!Rj}HQZg!^8oI#)uX^msL(Fb6@+hd)F^2O;DkQ~%iRn&D`9va&0 zTQsJF+7tCCuorR%V4Bb&#Ir@tVVd@NVoG!dy%C_Xyae5nKT`<40qV@r=!9{a)7;z8 z=v}}qL)aN+Rc)hgk(-%eS-we1*;Wcj-9uKz6Ya&>vjHfUng~_SxX%_0-m>FkAaC$? z;!*u7ezet>DfXSzk9Q~c8^KNoEa6^AGr1&w38$mH_l@$Do11l* zkIMj?yODi6;%worzC5`@0b6E#u^PcLcZjB)4E6VpQiZSKL+}ec0n>zsx}>8+Fh?$r zC~moJe7#+A0VoY;kg5UIlv$I}l8Q5%M_235U!g&dJa0gsT%YMkt<<*YE&n=lA{}@( z1+I~{`%msgKIDB7`4Z)JC;$C1gH#F$3lR%83qCMxVwbZ<<4degu1?aspgaAR!rqVE zuXAS`uZ~J^26f!$ZKULiL-SJD=nlp9Kr`C|nGTEvkwAaf5y2EEO5>j2dA%TkF_-*2U+g{*pl~GEhuxl=YN)OWBm2mjHU^wl8kZ)zuD4bLMinwHj6M zjuUWy)UgYJm>y8pM)?lJq>XtnG<;oUBYV1c6{RTmhCWk^>ynqv4r3uaT1A5h%Zu!k zgL{~nkH2NLAky{4atr^N;rfY6qgm&Doar!NI?P$J1a1V9-Wfd5uLggRpiNm?;)y*3 z$A8ddY6*Yp!M5{hZ7JJOeFnkF<&YU;u)!~Zw1;Md=q`J@#8K8OXAWNOM88%_b3A7! z%muEV30`@;#XufUH_4M09HJjiJGlR923mBfYMWWv^LQ!*X%pl6>j;CyQ4T76pqPhu zhs=cVkSi1@B>H0NP40~N{VNqm-v2Y85}G~q=ka>FALW5Ta?H2W%jr`g<>8 zMu_Se^ezrtG2&vc)V{1%VU_k0@e99`7FNVoFEV?{%Dh1?cl%FV+bNMZ&<;Gt$#7&d zUBqYAU^m-wL3UZlY+fxLB;aFnK zXd5YY*`kLJ)YM%lX1AF0(Q?Om%e4Fx-#85MlI_47N_E9DrA=?OayJDd_SlG;J)ya} zr9@1%)poN&Ibk%=L8SFaU7uT}Wu$^QFPQgS8S47XV}w0I?X22)q?;5fka!4xg!Yk; zlAIY@@NA@kbKLTuB!l9ODW3bI4o<G^ZI%rGiB!asDC}}b>Wll7=Fz}U*hJp7<<%KM zt@)U<@}~L>2Q{nz!ejYpm8PC8MuGZhEsP*R`zS0-r!Ip{CQj!65kY{3Tnq!QTe%Pn zz-4rDWGc2RhiU>b6l;0}q3KeP!eYRICKSlsZYZUNC$b56Qy(abTwZBeGH~H~YG=q} zcyI|3(l3#XmEb8hyq15L-NzMa1aVn`+mR4pH7qggme=u_#40!9`Rl;()NnGCPRh2j zQUn@A6Cz}2Jh?fWi5@IEGKo+H5jB?{9&Tiquq>_5fCXa>afvVLHE9uvejIW< z#aqKMJY{QOqfWGHt25~{gfIiwqTXgB{jkpDw9xwWlsyFG+Tr`T&R=Qg_>k5n0H&o` z3tL#k0b_H4ReOdNA0Dmrdlj>*x`;!xdVut|HLVTh*bO3{sT?;Aymo7a;6%Lqq^kgH~;#PBt9 zni6Et-Q?g$A2&@Aidoq(MNlS5rC6Q=IecC3=C9{Mj$os5V|`ze${elmp<7D=ag>ZO zrh4|`C<`q+jOlEqx70WzNYG_J7;>Pz0=+N5*ySP;!^k~LEpcL^fWp8>fHO5>I0`a7 zLj3S9V#oR3r*TCdOiJ%;@^yZRHd2dt7% zOtKkk*u@$%}_1No#*Q6kiGUZ9$;xws(GGv*9Ms4Kr2Qa9H3{#<~nZDRk$fr*9 zFu^h&{D9EZA{%SO=F`AFKdA+xF*Eol;ut1Mm{0!Xa}#U#!;u#vU+3?FZH#57RXyGm zR@sa;->z#h513N_Vg+uXRPfuK3+Z>y}{4K&(`9RESsx= zLF&SxXp9E~dEdfw*9eI13=0H;faM>N5%s?4_W6{`tB@S_ob$}hkbI}o7EbX)$&t7Yd4yG;tm*j2^oAgfxDl)iTm< z6uH!JogG_}6Xyvl=RMDrKhP_Bk}HcMEy!Yulwp$LQK-s#<271;srBD!XX*Dv+d-Oo zy@t^ms^D2<>Ax1DM8r13($pEvSXtn<4{rlXDw{Xpxg|T*xwTy8C1>ZfnE*|g?v)06 zMP{zH<`Nk5JHyS|QR)3rgC(63r$Sn|{u_(K6V7a>&GA?Bt|l305=SH=*# znkHHs>MQEo5v3*xtWqe($ee>=*S!8_ai^pnYK~GiQ@7MNb$3yxNuwq=5Zu!v@!KMH z2G#i*CD!M$Arni5yV>?1!QniK)`5Ey4%oAH@3Y~;fb*#q!_`exlu*b_<*X} zbxFxB+6oawAr1W|WoRd}@C5UwR&A4_0m@Jfnzpva$Eyk3wbt`1xBVRgYAZ(*VX`EW z_}LZ@HE-2Un-@}NmzIQgSG%`P*YzQX&TLhy5}jb{61f#Z@QK+OR(v={aq+CDV6&DQ zwkwn`kr{!RpZPh5o76`7Ej0>bd*|NGz4<|F!DJ52pS!M~EgA0s;{fL%v4Ll8+Zyj= z{K)K<&HY-UgX}Xq>gsHR$r#I6oe>yAh>3QpmDR{-m>y%dFUnLtvM;lbG`&7e z(cz800*eEL=kd!{68adIQ3&Q0W(dquKajvB-b5wAl5yF-iBuB47J^m<;rdd) z`wn}X2VY?xj_gQkJiE4r#+qPq^L~_i(110{=;yt;Ekp^CnYyb6MY}yaLnrxN=z@XN zNs8ZJ=$xJTPVpGOW0HHQn`}-iL5+_?4O+s%o}Mgnzikbt>Bol?z1xb%8Qs_YyS6PS zOF8BvMhoHIl|AXxUI@Ahl~DgWNd|WqgaPG&kkB5C!2(gjds^PG8KxMPY^zOE?|DF= z0)aAEkoiB5tZnjuK7HorxDge+5iWLZ*~Q+1^MX!T;-ytH?!Rna61Ylb#w)hF9>o5* z^PN(yB4W-w!R4%xjUt|s`@}hIq1Ac+JY~E_*Nmynr5wtRj4y(bs+Od47IO{H?5a$m za8tO=&8y^*_i7rs%~&%quoOaxO3lV4(wZhQ>Jh2(D&Fymd%hfBt^s!dSVZ129{0Bo zyZxJa>3bLLSZ;Q3HDjRrUKoN?6*28xJ&Ky7n@3xu=CK|&o0gY!SFM+g_e$_f(GvPe zO|!QCCy>vAqgDsNrqf3QG>KbKJXTNM-fyW^oYKtAT87%uxWfZ*;Tc)KKNa#fDTl=J02<;z5 zLFas}_}dz@o=`OYMm!a_pc4-lDpyoML3w}|k7|?D^-xwu*xe#%qCg+!26>hDXG$%h zVpSFlkvD4nMVtH32HH>Pk$25B&8;gOpaM~%gsiIwVXIC8ejOP=m@)O&ns8P~JFh zIt+KRJTNdq=i9>xY%1{0m}+K6QTA8()58U?z`$SWva)P@Z5LjDYNh0v>FDkmXc4aQ&#WMjzRS zM7CO>tjHu5;-tgc$Fzj3=Kkhhp==4!za*oE2_P|8pmbDQG%jIE*5eSj+*wAI^3#G) zTu&m+Rl=syveK&3uF?W?BL%Z~Y;sD9e36*SUm%wxu9i1>SnW_@vNDWe9)FOitEA*a zQ&U@TmpZ{vt5k-nq*z$arJbEklsVEcpBNVF7N(iapinW@L4zt8i7H&rfk#obk)QP~ zNKMk`haO2Nr3wCyw>27~Yr*er!kfVq+d2wmoB~GhG`O6`DlgcYxBbolfDs3xF|Xsg zz=jzMY`(+jtHEUq3?7xP4-^BrVGp+{O@KLZr`=TNr&KiKk=|cEQu32b6*e7&H?M;f ziqPfof*C6P?*7ZclK$8JjNPD?MaUJZ3+xMRE&T>S{ypkT@|TtDn8H1@$FR3pw^dQC zVw-LpNyp)jtk$)+X73S%Vz#^xX?#a8S5ivb$rEClA&?__%z>r>c8g~|<}wCCVp{oz z(&z-~l5u1@rdiq;F%v_DtQ<0oR(84x)uO|(Q89)+bgiVMH^~|C7RL3zzZ&;aB&EI% z_w5U+CB-Hb3*4nPisdID>&-+6p&Ewfkd)Y_=^*iNpsB@>V4JjVilBKd{+MAk1YdJ^ zD?L#?cs&8y0{Mf0+s=7Fjt`v9+inG&i*9>XahBJIlOt4`1TBG^H3#2v?d3Y%{?1Sj zz!7Nyq@}=nTJ*Is*QiO9W@2E-C)taaZwDW_wPMpqho)MDHgBAV+Qw#Gr=PTIw}mmU zal>&Ebq#Bq0D*!e-deuAi%1-}4{>os6>B`VXi$(w8tiuuc~B7WsA$_idIHA|?NViu z_z-!;U{M9qpM#K=g($L(>T?UKN_G0ZbdH`2zr;hI+lz?0hVlx!bk1`~13AxYraQUs zi*PH>6IN8ftC?&{hBZ_6PPfksa7MVqD2{nR?1A`GtpW}OR44JtU)COe$zYsPQP7YB zr&*hM3J%XWlDhZ;BbQtxJE z0?FQ`;7CcpT>d`DB47H+Idt(pj`!5r%-P=8FE!Fj4wRjD8SgW;)pri^op*)q-=#vB zluTGmX5uO1DVWuXH7wdE-K;VQTZZXVHc}G}#~%0I9%=XLpKTw^CdFUCUuqvH*(lvx z#aqc}(RAux#;=Qp`}0G;kj$5ED(;Q0*d6yJ>5x&d5-qe2h`H*R0-BiuXzfwbIrwER zvqEAAnlTpfAnX^>Mk&YgHMZ?bVp3gvb;4s782VtVx9^GiOrh&D+1r&DdL@9wUU@A| z-rOw;HI*pR;}Y+4QHr>!u*k5tYU_XHBsi4}4QK8M6%IyDY>eT(+M)R$)8|bM3&*f4wgh%$cSLqbwQ03vZ47ixo@F1yI+nO>if@_6 z& z9EPt&WFF&g*{}P>%DYL!ua?~#1@IpN&nzRwqbnJ;DOZV)}LSYv9FfNUlaXMvlrz zrbOxRN3JZ8tuw;J9p&Lw@$Q$WhBb-1icey<%uLe-*JGp7PDu%nPk8pZ35iM>D_?P$ zQi0@I>~iRJR8xQ~dUZ^|QGgqJG3HbcKSx=)rq*C;MEBZk0>irG*=iMC3fJS2W|-Jp zME<=p2=K|iBuCN#R_n!Kv}4I83Etk|N$c->FGjT8h$C_~!EEC1&<#R2N3KKkmw%OV z?ad}f3gyWb)mH{f>e`q^Kw!jT9K3BxDuE&joG<~Z?F3BzZV<+l61d}SMFh;$(YYq` z!sU`Da~KKxxfb&vnsi$M7%Ew0?=J(&(i>->29%|(mj^tE~~YY|t$D{M|0)el^ z^8BodhFZv1`qN4r`Wy;+Lqolg%CLE-xHmpz<|U6X?})XHDh?vz#*0-@Ht!XWZ256@+HGG{0pvW2l;D zV65`EIWNJLXu@T~HioPe`#+ZMh<7*YpOdx3vT?3H#WvPiMttS42Wf`j)@nNM%N>S< zC?zg?$FY!Zl|Lc@T!%rAT6V48j?UWYRpx_H6L@xhu6=4X=qb}QGvyJ-$aJml-=Rx4 zdg2fZjmg#Mh`GcY&5`W;&@q;5kfklcUP^-d@%UdZ6u|bUyWHdVj znOTww(DNhcisl}@At@6{tD_IvD1vdA_O(xC`8aD?=^3HW<~BPIJub*FxzpF>Nt;mm zL{d=yTHeDOb{xZ6wMh;~$Qa{l5H=^W7=p)J;HRP`DN~DiA!>D09(k&EZth3R{?jaK zbFOQQ(xuoU<`H<2lmu%M4`Ie%noamOZjrFO0J}x=ru5iqEPEx9$71o9(8N4!JV+MN zxt)ETFALA}?YAxSoReJk+oAI*R9XhRi|e`XFG!PXxU zMC4K1ge&KMm*c=76!T_XR*G|35I-Qr^Wm^o7=(gqMU4*8wu@s+x%#*Fe{{5a) zJ%?0_O;t$Ois#9hdeY3i*kI|1hkbbtq;ZhMDX%FP=c0;UH~q`x%%si4O>?#*Zpgvc z&)%<_tvbW@L*74Gv`D#8a0Wr@Al(u~c7TlavD>U%OyxR(o9KCwjx`=C(W1R{zLp0x z3V~}eV6aHo(pF{LH}b=}Q}LvD-4wcE>ehDdz9t^w z%$)3>+@CzpWR(be6?M&XY_o1J?OHHUvx*Pcpj7{XYmrnp(WTP1(>?K7ZJt`P+PG4b zPX8zp!SL-r{+m7?bs5gaRBq6~V!0{gK1(oTt;$oXj;u@#Zi1!o=Oq{QfPrc^8Ny34 zP~O@Y(#Sp`Jk{Gmadz*CW#!ABevNa_{BUp%VuN?@n!(G1vXQ{(vYQ65Pqmn zNy69sgt`{x5-C;Z}+32%W;szkQK<^9WRT1W9C%uJkiFI?EM znV~H$v@;_sAtTx{uS&#vu4y%4z8*v5;9q{DnXB>qp7tXBCdO0D&uaJ}%VoO*c_=SD zGFu|ra$bs*Nj79U%Z~9Fk(9!e31DkbkMyq#V`?dgNZXT1lnYItbh zaLZ6jQFCxPS-+N>g?|&Zea?wfNg1$qcVp!??Tz^2J>#q+)A`xF-%e$^V}I4}VRggN z5)wQ|nDO_;)skbY`}0x?_iZcj#jfvedEb42oC;Q)4DLA)bVVYl`;QyyuuO7Th)sLf zxUNyGTr!R^HIJ0_tjxIgYOB`fm6yUx!XC~R!2rb>Rsq9S=C$$yW4`Q?addi?90wdb zM+qrR8VlxdbVx(kb|K~@sa9gyl&}mgRs(WJmxY3&>4I!3Bk)*aapebb3^HjQna)5t zMv@8H`}1tzR<=n4K}Ls1*VMpjR#ghVctn;20p)aNQX8?86lsnH?0EScwn%7+v(BNI zn5@{GgFFr{%Z$Q-!p~w?zRxq2botU(rsr6FwH8Uef0#2U#F|26(td?sK{-FYCKutC z>1w%pNyjGiP&>*N`BgKkI+6ibX)=*?NVDZshVo%TIE$s>&y8_%^MvJGjLlqEAC4f# z#uK%OjX>&O@Yz>=Mp$imCy7f5xIvj7qHKelo4??<&{lYXxSZZ4;)M zYC1Bb|Moj>{PvgOUbFtnnLXCaFili3QfS$Z8}tAjJA*qSzMC~y zghnp@=FMX0Jb|ewSAA)6HO{@R2Z^>3XkZNkO(7ja8Y@qiOFm|XLWp2~5 zZv&6g;-FnvByO7sQVk0a1vZDdGV@k)M17BKAVJY4-K9EzABL0yl0(5+A(s=%lI>{ZJ zR8Q%Xh>$(8KcDSeyl^xTfiV_yBPR}^Es%TPUP(&A**ZOYoOPXXfa5tPz_mYY&s?7$ ztk05l+?2@Fb(2ZxI!!5ogY=MDbjJ<4^IM);7-s24pO1P3OQ#J|P1v_x6Wc*AbY-$Dab*mqGD z?8ueSrso>>I$CV?MF;2XtCE^wZxqbxqGr5(TCf?b$pLHy$L4(ba_78%da?g7KB1-- zCH8(M@9%K!(W+OZfNm1|fdJtb@R1{IsmB_tAm2x7^Yt z_JlQP?(tpTeX5KO0x=d7xmNSWg(19!ZYyb&ZsxLq)i1 zET|aw-NuUIbR35|rK&^a{bcCmyhQiov02W2l7rDn0M21M0PC)%rcdWf!qMVm$x~o(?MVliwH1qhE5?&{YSd|nccJZ2vwRI^X8r{HQNH6M63o(^XWO=*Ugl0PhTM@~D&28K_oF$?wwrlcAo3xJ zUT7o*dDbAeCre@qwW<|64G#}nrH}xVmZOcTi>XIO6170xSsqOJT1s+8e9TO8VPdm$ zLd=Zo(cy7u8r&Qx{PJYE;y^7Y^gGUQzCjbJ5zFwUJ#!^29tI689)utI@uY!YE77C? zJ3K8US-mCV^T{m~`@&mv-)BrQ6ZyHfBVB2f*+EE2k*^c*_WqhQSGQ{6WRuwuppdrAwy z7Mpl7@+o#WH>0F!g9iI(-i<|09VS+`@|OBoxExbCP!*P^`T38xb(dE{jwvr^PyN|? zb!*N{m)T0H$jVoQMIFUy{8ls(Oa8ZS3FkA@ec4C!Cl=W^;iq&iYt62Td8A3GPiV!E z6WJEEZSoECBe~rIAJh$%9TqPNFA5)%X7gCh{K}x&5gK&Aq(_!})uaXs#bO6WBzEUv zX@a+n#gzr~6ml>g$L5%g)I1DMt;@LK1n{FEnwNV&G>%1+m<>DitS<2A^c2THxs1k+ z-wdAw^VfY!Pq<%0HVCVZ;~0*cw&NISx(gA)%7`BKk4kyN3Mp>)&0ZMSjR(MhbsOgH zX9wfZiTc=jqZ&zhWpuTOdQ={z@@%e&!#2%6t()>}liu%Y^E(?LakjyXu`-j!#hjvR z$B+T$YQF59IF@NuXI|a%RpQkMDOTY?NF(OkEVV$5&~R;?;u?A~)qUc^C=s^E8qKl8 z7v=dIVFI@&H)R%ZwtcKcx`*6ht4n`!esMzE2~^T-q5DC1_Y)fjbigdU^PPBh;I*}Q zZT!}f$6o=qM_FIRQ+>Eb{%c666av~W89>#{l7X7GZqq5tfb{|98A)J}%MvO9W_g-d zvFI%!_5%c$dkNz6o6 z%5e>KYoCWr-_|tOA6JvBKNwu2G}l->ZYUq!_^iT{=MtUcAJh7O%q+wz*Dk;8Tlly< zZk{Y&EHO!GnG*~&U^FtKTg-S;o>Y_3CmGQ_w!tZ8P{&haH?C+a3Cd*rU2~EWxk)DY zt$>I2j=SsRejN#$aSP0wKmA5>!KqfltrzUnD!QvrEKNHHN|uSZ$SI z63Z6Ml6Lrfk<*z9te{Cbwj-3JAx7ii+HuNPE~OWjiY= zJ+-1a;DDxkQ#ztmz{2t(NIIY|kzXRS-YCU=IFE&F1j%`L#Ibu&c+=3eW445zmV}3wETI=|MJzDg8OBg>HK=U zSA18D+HmL+80N&$XVbv-cS96saDXB~meGAxIHPatET*lERoKUNdP@jEOJ;Chg_U`E zwTe zMtgxx{}EmHM|**&5zQAnyn8(P9P$!o{U=(QjHFu(7GrtRq5cSw?roLE@wAQQ5LyWv zU1B@T>nGt-?fS)=WOa(CAjcCb(#Q&`faCePaDdx~?| zyT|5-rWf|n2FcQ|%!_98%2WQ1eIa;38lWWnKNx%G=w5#C-?!~@9hJ?s2){&{C5Gm~VJN!FSq&+DnxkgcqaX_|YZOljhcNLlU_ zICkc6jLB-_M%#mVD0RFonvfzS2MA;4x$_=Yi~vpm&H#4)gm)ocYQY5FzB$?CLFDYFIfwDwkXFP#R`%|op;+uKW8V?t zt~oLVE#N|UfV(+!$`im3*?nuyI}sc`cZn3DL}aX1sc+=?mz0D45BqaX8>egT@=4Q~ zwL@}k5{k4Ur&QImCp34+=>tzI5Kbw~iY@sUWS-Do`0@?TL{u82%n7}}+mPog)or)6+ATE>T7dN;w5+3#N0hwt2IdE(u)&`!~$LsX<_V&S*%Cz0;T zx_C1yFCwt+Td9M@0PiX|Fy6TU375pYqJSu~JGFjt&SaVFe+G4;ZQ&U6>JF-V8n#t! z8S6Awr3q)I!ZSi8Cmwn`sNJ#bMFDYckZ{A5PFOI6WBY?b`H7{_*r7rp3{sZW<*0N(aF~RTrHzU>%Pj3#`XT@YrR6%D? zJP7|LrT?WpLm4u!EXKhA@l4o<5rDMY0nOG^GEqz=3FyxEPqO~IOA}PHqGDV?22gBQ zBfMa>OVESyjHa?5YN8hC8aeiiptn5n*@FD!llSYLUB8x`As{)fAv@gBPSh7{bc(Gw zkgmO=k_8*wVgE#Q1|{xexuxs2sv5WjoG~>lv&-dUEmeX)9dJ|_(nW|oRcGaF;^HYx zls|+04G`F^YlMe%#IcMR4SyUakMKCI9K!S?`PoyqC}jl~Ajyt~C?l&|ua@}RLK>?NZn)uoG&7#y=N4~R}#-ZO$T^;`Y z0-}qlUueC9N`C$00ZDdIw`_fcf4sMfdz>06wq^au@cdb!bz%C(@SkO!7-VgdJ6l+| ziGs-!)WGiMYseg^xP}sIXG5V5ohnW;Rbbh2X+zvOq;!St)Y5U_4iJKfpa8$TntcLv zo8mRnBub0^^5i`^W-A7KBR{ZVoMlE1ZwszGUIN*-ioq9rrF;2PWTe}`&Lq`v5PfCc z^(~R~?=4u_67ZRx4OK}-3*8Q9OvAiri+k~v3(T?1m+Aw64}JXY?X-!m_j>-bMY!Yb zv!tuesb5fZ6#w``9wlE!ZeXK1`J8SGPr%A50x!6eiYw;y{|w>&kXehn!>V1yjre=9 zZpkKvx~Q(~I||y9yY=h& zy@L+HZgbgg{8-5GqHMU5RB3DDsX#JUj?5LhEf6mx4(f<~ZcgYqy;9k3oaYQHCN1j= zv`+MZX=P#thXL%+&KthRL|Mmr2{O85R5)vWr*sJt-^k#-^O1vSPIK9XgNUuUL#LVN zy#dC#TDgU-B~IeTs%z}DR+8`e&`&XoEIoSoFEn$cTb%H6hIC@-1`;CBcoKsEpOqA$ zi$@V}8V3RgPspUep~6nuNPpY$+4D4|-LSXYM1+Ra6?!n>G5!jDqvhH2Q9evquFF>S z*|J>z!Y`HBz_e)a_)jUScUU5WT;{XUrFsWVF{LekpZ>$)$wWMEW44pu){;a4%bsPA zj&;7>dbPjQP{N{8(N@GOIYLl$%2uFB4q;c&5ln&A>6YvmF@5!V6QS~^f%QCtfT`IH z6me<-JP_Vg1}TdMfw*uJ@5r*@{uC3Ppb)c}uwt&}DrYP9Qhap2#eCntk&4PJ*|lbZ zpQD4HEtd`u_{ESRKMh9f{`gvAtczQ^5qS6f<1ImzC&}(#g%0c4s0T0CX|J8t1A2!;GuYW}}>AsCHJ z0~?K`WY$oOYk==mn#P#Ms6&^a;;g67!!T_`)DC3HHCBL1d@KabR5nzQxqu5B9GQ{r zGX(o>kY-aJs=VKrsX3@dlH#+1{x=J?5>q=6Cik2?N|fCAIB_@nEnAMeEgOhKhkOP2 zVmw-f84Vk}ZZ%%zY1M}9ueLK=<$54E@g_3`*tnU(z@MMm_pTp~OmwfHw*LBb!wF-~ z*p*;49tCpkgtO^z?%aeE%i6MiT&qHuy7_W@EIk;!d}`C(aVng5^v_bgk}@lbpAG=V zfg@36ichQZ4coMC6qaM~`DvS<+_TFWC_*jWJqU?mm2ICo4g2`C1|<)ijr;Tn%IQvA zJNpk<+Ojb@48b@91SzH2f{?D{LIhvO#-^rZ!VJg+O5#4DFn?OqyWGqhqwiQ)NjP@) z_*c~Wfs?VcLNfZ>xRIdXeepmZ##>N-9)3eTHOB_f%g=Yo`d5YvTtc4Re>OfXCkKwL zmU9#P&LiJh;z}M3n(pL51tYmv!PV6jVPF+#bp-}Az^5rxdHRLXpC4ILR+#D2x2{y1 zd12txlA($NpG6Yiq-8o9MhD=A-`s{Yt)nUJPpi*zONv#nw|!b0)kQR7WhRWEKLLmAq)Z(VL2%lR7f9g zUD!uRfDDGRE?1jI;&hurnUbLbAC$C|6cnKrh(hA5?<;H`!@r+n#;+Iz{P>cW|6X5T zcP1SUkCTs01ozV?`iB#BqV>q17%&sruxjrN2SZZ^ZVZ72BUfrifZyvXDI%aE83Ys& z6S+oFP*DZN0$M40I|$HAV)0p5CP|p*75}Heii*lg>bw@LybhR(e2|qeUDI;22?ha8 z%*2k-w3HOVJc{lrcV|UH0DG~oLS$0Z+_i9p2}W{U=3Jq8TyT9Mu$cOQ1qS5BV9kUI za|%XY1CpeV@?X@z3no9))0c*PhT39>x-?0%ykPYb5CJu&Qs0Y}dSin<4gY%LRr<}J zidyGoBw!dENRY_dVRAZ>1XBkQhq0=rGV`MpDc(jfjDT$Wpy%3@f3Mn!Ws~RJhO71W zc8`_XZ_)FXt{ZaBZ;s4=IHo5Im{J0=297_1j}HBoV{Z!3*bMo?%>m9eep#(JBZgmELko!V!a7+68KJlA=Fu*=($bi9 zSh?Gg4p1YoG;x!EPo(VuzTU3LC66n9VD*C9W6_7zeZHmMbXdNTQMKvKL~?aH!LfMn zQM{054f=Td8Cr3J`{gWxtRI~!nFW&V?v z@0J;$14G;>hl@!DbIuVk4fQXAL-=M{>E=41>mwq#zH3IgAW^Ni(ocw;0gb?>ODsUl z>~AgG(T*ddbKnCcFTVHhhLXMh95925rSp^^)71S&hw!9J*bpih8EZ1-;B7?5X zKo;nFcxsjkN{wyD81pnD!0#aLaFO&+{lHxP_x8?ynU!5@{MPpMJqlH?w$6^}C#B}4 zflXjp49cN4$iHC;14vhD8bJw0w<9$J`NiEdbJff#T6G`b5C#{0V}j&D1)G%^ny;{% zuIQCtnRJb+h3~5Bk}Me~E$&iTR=69c0&TAwq!z9p8&*RZ;m1TX0z6?^VQ7hu%9m`~ zQrvajSTuTroDN%`I<&_^GcWC2Ghuj8)GZB@$u{Pu>Ux5f)v?%v9!*4MGZ2bHOyn*p zZ@;`aQN1c0*`299mwK@~XlCX=H(UV0lYO%;29&MaUAYT_jACqwmYb3F&c@XNS)qe~ zP@7w0d`O$Qyb0a;5L832M;$IINPag$Iy3S;aJt6H)9dxg%@Wji*h;Wm)Zg|R6k6LQQDL3Mkn(;mv z@!l7;yQDY5O``>I9f0NJv0r-;zLe$X2wKFZ3{&G&@0+Kal8Yf0@yS#InmRYWE%mx1^MT-9`g-WSRBA+OvLh*lsNR$e(pV)taVjv2s#D9V%3aOM# z@Sj-xzbQbJAQFfIk^W*o0;B{QPn4(lBmOABF#~}pXhn4dA)6^9MQO7%mG0nj!0sV zgrfp6XE@34gaZSs6KWPS1(d9R7|Z_+dW-k?7I@1akuSVl7_uf66T=pc#84WFCn{0= zkx&%IqHxRqgiutb@I)k1uIN9BDL|&=O88G$Bjsf6|7*B!ulx^vdD!kqmPrd8%irV6 zH&`!9gO$7Sp|!!;SYNC*|88n`tYGzlQL*&E+ObC7Sh_$ym`?B1)2(&OK*$UMAr6ou zv>0v%-cW@_#c&262*!b_m6~0enIwM_TqWcs2jEWYM5gyh9HQEUy`@lG!^1?e%47@0 zWaA2~0<>A$z;_v=EkqEOEXuL4hFuHG%qJN|ELEEEU~#AkenX8ZgPDi}w-@^_Qa#WN zSZC4!p`vihWvU8aQG63dqqgrVmSxy(eHO28r*Kbg3qO%6_FxHqkd9J6`8H0@| zMTcNGF}yXKgK6|=PD`n_!y985BN+R$0@ED<)~g!Uq+?91pYO#C_%Y{DXM{Vtb$5|O zcy)Ir!*ed6W^b3NOxS}nutjqEGQ2=#wlzDzDe*gKRPx!bV(?E%H`^C#s@Qd@p z&B^)8C*;|1SIbYmIyOc8y^cA=7>0j$JF{`-Te~Xi;Noffkp6C4XN7owKdWy0yos#i zL;bhs330{d_AdQxEd|e>{uTap?q%+t-E{x4?$`aty9RrH2+M-F>w8w+fCEP)fbLt9 zlUEBJ`y=gL)o(=~b!GGgI%V9G8tR!5PQbF^k#KA~5P(Ovzd-F7u1XspD~(<3R6I8i zLHPq%&QJ9{QtA>&|< z-slw5!b>!WtC2SD#cWIJb73!j1R>lyx3#hajys>nl8^b5++2}=g)X2C zm2+$El*62aSuY!-oMpVutYuV8lQ3ps8Ld0)t5}zoSzg~$la`5LWqEbDUf%8txD6Gq@vkiUN$tv8G&qa6P;*gw}kz^V>ltE zWfZn>o(fMeMO{K$YM{|Er8mTU$Kru>Uwbtbg9eGVU*D z&dMkoT3qerF`ry5HucK$b-?w#@eM&^1ZX0qe(yE!rPZWRtJ9eqn3`}37*`os8QWlZ zY0D?pmA7=Dd8@J9MDI75bE6jbkk5r>AYOoK_GcQGLm2w?7M$5Hlg`Q2D;L;Q))lO$ zDR!9fYu&J7IN*z`yP#UXn|4%iQ-@vZt2v{mkuzS&V_w~XNw22Wqe@KJ%OO=R3CY+@ zt!lK@7ciGL)-?upW7K<5^F7#^mNZqmQ~#zWn*5CsFpf$a|IvtLbNyBG08_myml`gj zMKsxbEwNi#h3EZ;@a~s!TB((8jcTuvfl%UXns9?X6LW*3zlCX}$fQnrg2CN4@G2T1 z z$I?%fY6hKeXLd_B#KD^|E?-Xwxo$~Ajj6J1ZeV#}r;gA)Sv&KGF45woJJ@chlttij zVudAkDYkthn8bkQ*Kz(ahE@wXqkljNwo^;^^O#cI1DCRYW;O-yQY4NB<}ri9S~D;G zNAEmV>6<1Ix2B^N)%TgMi z+jA^F^4(ae!y>BqH7cYYTAV@Tl&=`svfrx+@ zLXX9c5d~rsWt21IAmbzh{aH!wH3W*S6l_>-TUQUdtIMC z=h~eeec5hSbokh1a~-Q6FaE(I0LmkT04f|gMAtFrQ~(x$(a=(6wh*~I`PbOncW()l zdX9Z8r>XGxbV?6g^*!_^j-5t#`}|5}kJRT1B!I(d+&|SWnb3a^%=2Pt)SIDt&qHCtt<~{O8*&+`FqLKT3Qf29s_RpL*;A5>1<}F$*M9lt)%-GCErac`ivdG@ww9L_HFSVsLj-=lt#O+l^UvVkG;Oz zZ!5Wd!D&Nvd7qPyKCRv0Ymc;Er47-bJa>s+gT<%+vQ?%Rb9NADUW(wuvjI%R`$>CP zNP<-If+N5W%*zIK!^481_Teo9|R|18sl|PRCV4A!H8i3ena`;xq}QC{u;2q5W|E? z_aMrIaK`OXV8n1>g%L*k{;rrar0#NHpjijq=V3YUz=&4`AlA$8dSgUrKo=sM8?fww zh9Von?2%xEX}}*7>h;+(MaTKTz}oZep_GXNCh8E+jK2LX8Gs$QWJCbKNeJdl0IT-A`2lMC?^DZ zvo;vIXdj?SL%Kbv^su(XT~Y>o2UMEyZuqnTp`m|u6s}lHeMUo*29SG)_K16^bttz| zq3Zp^^}T~y8li1)sl+wr?yOZJ8$fA8U*On=sC#6g%YDV+%L8XC`i9I^f*S+&)H<>5 zxH^bl@GbqS;mgcjz)M715a$SXLf+`MX712BG49kh0$s3dL|brK2-gPg@~#^eL( zM8pqyN5BtyN5l{NjSx2wRL?)a=nioU>5g$L87kkOvx04p^EC1WBrnXo$6N=0tGN>A z4x)vS8}uuDc2ICA$sXbsj049DS1%Z1kCG$s1!*uXAN3a1o#Y9t3o4GNXXM7ko#u(^ z12~qb2Y3|Wd+bKm9pskQo#_ec12RYGgL$yGZz0@nkU9LiudzODS2y&!A9AR7;6B_h zC@I{}+#O3#z#CsLx*HU~uY85_1wl{fBYc;C$8;!eH|2+Y1lbk(LOIChj(f}e@@wz- z27C`4rZ_|9buLZte}--t!IJ9{dg7UfhlP zP%cKx;5I_xjoT0gpycvf+o2=Z!PDQt6KnrrhZFFHyZ>+che zkDTf=$bIjrJH44lFYp@!zA*P6*hWtOL@%skl(sU;*%<@2MmV}J(E(qO`w#LMrG5h( zpU5Mp_{?(O=?-tBKM$D!z8Lo(;(jum;`rSECI;|jIlS47oH}8Y>W^{wT-EM8h(<4< zGt24URdV_ae}k9mk8^xVjhw18%IUx3%YDO_e&g>vBt3|3wb`h~>V& zzL^2Op$>1DKh^yv1ALu-K78B1`Eh)H+X23~_8*opOM7QHK3PXjYZ>MIVjSMAdovgG z4FO+J`w!|ErM-9Gkqh#Sa{R*_pD6ne%0DIjL>~RJ7l7M_sA6#w>I(g}K*lnCrzFEY z#sa~{+=s_3RL2B}<9TPqv+RVp*MZ@G*F+7Ua3M7uueyF}b~;q)60f-z=w4iZFH|!w zyF%I1IUE5s`3cPQ88=ZrVOB@Y#C^uRWB*_@R`yR77kgyO#yjoT ze`6uW*QOSd+KI@1U|eF+M&JmPXa=}Cv-1ey?V}N3*YtB*0iT(}5A3h-fmVhi9vX}O z@kZoywzgre-?cdr1lI=@|3P!%8|NlaF~O4pg1!0B;BIsKvqW~@B5fTUXwXFf;<*9n z?Yl1)2=LOnEbV!Ydx%00Jmx-b)5Pn4e2L|eT6J;G#9HE73mllFKkkpORc$3s1nA+xUI z+yun0fJ9}7h4Ab)39eH)M@c?Pvcho;rhq9D=`lw~&h}Ir!q)h8NoLFbQV#%+?3V~1 zk$!xw0uzOq+-wFXe%i>PeY*}Gdt%mj^?r*zyuEF{j~eEJNN<~`MYaKkl0+!F(k)e>JYDx?g8lm@`2|9y)Ly4vMrh( z`X2g#B1xa0zge17a7v#=H%Cfd!I0-f&klZs6!IbPn!F%Qax0e#()LIbu9 zzwkt**M_EdQz6Y9d+dAv%B4coLh-m^xjN*|y-T+pGqfdp19fwB^g?+o&Yaqt7*qMdTa9fV{IcbV-l_5T3eMO&Klp=0L_Dln z=10&q3V&#B&vuodk1(ePfc04X-2H$|7pW0G_e+58SFq*V=;s&v0{%TlIw0eV#&^gh zN1&cZ%L7XLfgl)%w+;LS#3OgLzfC~6LQO+_Kh~mQ>c#6#e2MgQ^FCxm$ob^`K!2cj z=UA2Uj>J29{CYc=ap2k?}904wj`t6bSdZDwbsYqSXP;eIS)uWv=|Kdt> zlZkmg$P&1ho|Tky{C~lG=BCIB1#6X2HIW<;AP-c)E3rPNGhnBM%4MkEd>-~uear{;UJiU-IlcvVHDu#KP7R`vq+OQEZ<90Xgi(+sX22EEJJe1X1| zt@FLRU26t;?L?Fs?4}BLeB3q|on6DoPhKMe;#?zKH zo;IZ%kUj-F3MxTneuenCMQq`?l*J6)C~PdLM$9rUptV*xp@V-HtKe)_@*YbLAKuIu zg`wZry?=3B^tZ^*(q<%5BrrAE`J0tHI#XdRxgBH)=O10zRG{Ap;7E9n_+I>gQD^2M z)Nyq3UT^*Pg65jtHFdAOlRXFo#9fR@V3co*Jf_pvUfS_X$S%X|eamVxtn*k$xvX&Z zEvGYMGiN5T1Iv1TrYblc!!kxH^dxx4OaYeC3hrG&AeLGYF1{uQ7H|F`HFq7<2{bpgPF@>6yT< z3Rqa|5)mWo67Wwg`$5Djy{j!)Xew+jF%~52&Gbc`=5jc0#;AuG*{*FBd#P3+!MOCd z#qMwX7A?RWno*T~<`h{YC%i2;@_KmXI8gX2XB-4bo?XfUu%yJUb3FW!yIH(p^ud9+c>8MoD*FH^)IX4 zo5yFzPP}QZw8X9Snf&qbkx-_tZcEMNaugLU@0rA@0YzqC>VnSPP*AeISzXbLK{nB0 zoy+gjWqEPqc{w4;RnK#|+iyE--Az+@nzS;N9Zpk~t)wgdqdBGOb?2^!-}~*+d=9=Q zL-^%*3Dz#ND;3#otKD)X_}`_SQag>Xp9|{~6A(9C97~*x?cH68#6`#rL{jnCj`)e0 z+_M=;+U@WWlQoB_fE6ygI{zfw;ClRG{6;zxs|#ss;&$JkomSFKYEk#JS$IlGG}(h< zlUfo8zQS#~Hy=chFc5P8g@qz$BdXP_PRd7 zCPCQEWq>?^b8!G%_1~T{@myHwJH~HQaW&&(zd>aiS1c-_!<&9fSWSx?5wErI%v6w7d*PKiZVwHn z8hAZGH=Q;NMYmFg0U6D5jjJwhvaVd8CNChr6-7{t%H;LcPInMc>;B2cxU-C@>t>=N z;l|O;S&k|uPccxv$<8+e^G9WPrYYB~WXbnvovlQR@RHNab?S$iq)bsEZ(-jZDU}Y^ zaUfIi2gy$Som%b$uifWGcF^_v^h%Z-&WE?bd%LmH)_cP}*K#JDkvlE6*JUCv%iJ!j z`2vN%rC{#?dHcFIl@piFAX#E}5ro?jJ8)p$%$1a@<>u;_)bJf!_Y)z?1G{g+<+ zgs2Cf4gN}h30l4sL|sM6?R~}r`SsmS2PrDK06uR<@YV4!nm1uLJ2Jgyl#U7%($_xSb=<^H22k^U#C@BaENlE{blT#`%?@pyGRNYeY+ErsB*0n5#B`%?x zir@Z5BJwjcUQ%mq$^EY7S%kANFh@`%dp)$UU7S%6^FJ<2s9|dF;*oO?AyL?oI>d36 zEFmG66r{;}OT&b@2>@A?tBAtrK}j^Hqs;BFT~Srn?{eOMUlw_ezw&c0dQNVhL;hoO zpz84`4X)$L&>?B_PjkesQXB7d?lQLI@S5g0tg5$bJNSyG7c6g>YkhkGk~ z!zLPdO~ydg=d-rtS+13?LJ&RzYXuGtfThPl_y`5wcWM0jNi!)30ckoplseApRmuF^Lu4K zy#9u4gDb_^!1?rCPIY_Q{-7l%5iR6TpqdY|A^Hg8?=oBU9X&2BxcE+g6FD#79PGDtf= zM|V|(f-~o#Oc&Gz=}_!G30-4IK#5*PCZ} z8L{YGIlKGweevcq*Sf8ja<-OY1bYOR^|l}TDegmuQ!Tmj?z7^>iv#}*>#+H{rK6EE z5y6$S|4>i>6*)&xW>rv#l-Ni#*#{j!Qv{ifH8vka#1Dy-K(ui%SU^$sk%SZWY7Zr6 z&Jj@!%2F(c%rqoxF*7S0o-rNXsQeE~%R4gzIjdsNZ0JUZx&ROh9z5g-sDPG|7SlF- zCsCp(0`u$W48TYFpk&JGrKM=OicP(Wb?hv|4|eM10QIUF2--EmzlD^9(jP;YX1o%Y z%;QBz?Y`>0=*SHDjE#@dVp(&!J0MhC1ij!g8?+VtyO<(<5>6hfFFxuEhwa7opaLe= zv>)Mm)FcL9ihO^|sVj2=a%uEi()rYYmvSc#bp27;o$p2YU2Ls&WAl=^-kcREabVvTLs1KaV2k2t}b$ z0dlicxRIf8P%x!%I>|mYiNF>hePO6vt?5sv$J>+geeJb>)kfCuuC^2Ksyfo|5L{cx z3RL`gDjO$8A+27LjQT`rWy5O*9(7v9u+2-i$kpg~P9bEbv1=w4Of#))t4CgG+Uj1l zN3{($bz62m3vV^-mYvVK^yXr08+LcnaU-z-;hhNFXmEERQu2w4IpBrkMeNpSQR8NA z9942@>z5jKN-7VoC9UCVQT##;1ID-8AWYzdV##eprL)L zh70%aa*M2j?BOo1*%=W(Q4N;Zy4TbCUPE1*^!$&VD1uZRUaj}Is^f`L| z*$SY>$k|HKEAmgGOyCsmE_r+)eXYFWyX{e%T|sG1Td}&asLwlh+@Houyk}*jbNN9$ zkCQi7Uu3E!r|pD+>F2M-T)&fsupEsSW!!%av>b(UBbd#gTU;QV9NBvt8M4Q6SiD$e zrJW?#cUL;~3RTJA?#<~CO$`v#znn8uC`}H~kM(Mgr^S3+<#G-#Qj}6*Oi4(GB_(PN zYLe0%y~;6Q78u!FC!e&vG&mbw0#X{q+6Gd8&aRW=YT#~F95+&~ZiSk7Rur3}z9xB* zIK7V3llP`gk&Cy*(3+Cl6D{@h2XHU=k}a0_X!q7ln%GcIF3VY$miG^TCModNy{@Pc zt!F#$U8*k9CfR0mM{*G}x0h+-q01zZhHai2VY`?H8)&_y8gGpKN@Kx4)x5bHM2$h$ zxqFSF-JT(jJH0&qlhsyet8n_ap!A9cU$>DJ{Mi@ZC-Y-mi{mR3v6W3NHHA&Z{IS>c zY(N-xQcgw%_sF-u=NJ`^GDCx>$4g39RjoD)i_X_}OIziQ@!%nQ3VsWZ?gt%!|KZ5U z)!J2o5OLfJU70=38iVk!E7!A4Swb|?&QQ3CCXc=tdUBCDSj zhXBWARtGv9LQ(t;nAOn9QqVp^Vx}_Z!6OJjGI0>nBRndjr_jNR1~1f)Y}xGevltvL ztfZYNUP)3hf|s9lOdw=fREBoRA+xxn2|LVrv~Lzaz9l1VN4C@8_-Dg}#mYMpsYZfJ zOZqQ9IWw~Y>!QNS-V#o{Vj-mPNJ>^o*OyjO`BkO#SLrXhf0s%))Ppk+n$7z6okT_Y zQQUckQ)}0GjPXOoi4Y#mU?3H}DN`UZU7if5h%Q<1rU>92@$*~0TGrNf3v2v+{I%lN z&mHB`K}oD+);u|-BkP!WJHRpZ3ske2M@>^-sm`mrU8D$3r6L$nKlQR#A|}@oy{dR(l8s9b6$_&Zv7$2yI^} z0-J{^%>6d3HNAjme%a-}+leekMy+2YUB?zC7QLF9+c@7WDr5f`fe3*R9~?@q*H zW<)ps*YqSPG#apFf{}2S!gTo2H|4L8ob)qGH8RC?VHz(QsD-y;x`1s}DNu-;H=R3+ znRm>J>KOmws|A!(-`sktSFcJc9La|$-+wBZ)vxA4HEY=m#zT)N4Wd~i7Fm_wE59y^ zx0_0kWrR&~vEl1b&ks-{l26qa8I#v_OJ^SBJ;jai3_*5odx$K;<%P_Or%Tk6}LuZ`pB zZd+O)U-ekoV88#sbkpO@vz!Rlg;kF&_jT9Z{I_2Pt~^nfgL(?KaNf*)-I-_A{{lAW zr^i*X6G(Iyps6gI4?PhopZx*UygE!_Og_N@$rj>b>Ye0cHZPmlw-P$ds@*h!MYTw?unfG;QW zstz>S^iW2HvQ*{jnzD}(E}T!jQ()Xe1S=<+{ZO%~Y^gNpEo(rq!b@MjApe#7OB^;t z`*T0?`}iL8(N%g|Rm&5dW60$y`IeV0JZATE#AWDd6q^Wtm-pvebD1Q%5q++)VkrC- zHwYyMv%zsyQ(6+^UnP%mk|OsPGT|ZjF%oXsr{pWgzl4k z3u|^j(^kuATsaAbSwldP&c#1Yj;2nI<7j<6k90w8-G68*xjjfVwpIKHvd1&T=f^ah zQkhli-dxXzl~3{g>-;{32O7D@q12ozO9Fw9BO7?iJYUIYB zd-s9-QJrP0#Zm1yk$t5X6UFw)5@aP`d-6JvQpRhkAfXg1V78IzDYiEXQo)Z}#xErA zB`;^6R=&?%YF=|nY8LGIu*Xz|84E@640)1%(5AzcO+J}^_~ahQh0fF*RrJfkJqp6@ zPut)0>&D3|nzZCY<}(ug$kj_V-W)CGXk7}Lf6Z&6-ZkFKSj*CMh7D0bj^4qO9C_fM zSZ~nuAxWIyW`NV&B_?WTP-hYK-BWP-u3WP_B~;@ga=dI}A4Xw8^)p4d>U)*HrvioP(hj0~#VIIHjiyOM zx+0<~tER7vW&o^)|GMFwEHQb-uW%A`O}owBb?taxPAk;WO0GSL+fEKttz%_A{w8qw zDd^sBR6Bv|FqIbd&)4P-bM{A$<@*@ak$)p; z(Sc?-rT`m&9ZK{FL@O2Gyx6d6J?6}|`1?rZ5_r)#NRc7qi+URi9=s_NQJ1%{vP6H` zpRHc6KAg($vz;34HZvpwtJ=7415!RdMm@~5j?|TB;%AQ6*YAZmL!Y;AF&51nEhCY` z#Ylo70{cy$*<(JRviw@h(MD%FmRJ5>YA!E^e>7aa&rV0xUH^%76)Afv`pc|^2b)&( z@G?p)G1l=&knGiP)AkGVOZW?0PoZ3$o2R$QBDq7%NAu9fs;5JDvl36$xB|3*00B=F zGI(~+gp&e+o7S?ZAw@)T2FaO%RWpaY3@Zt19^zAg-FF9T*A)1db$7c&Te#=AkdQrO zeh6Bq0@5$?e|RVZ#CKFHrdFb6AqZD#{WDq>%=^V9U%)iY=7btSw-w2Fx5kdXA16{Q zs+UccQtYlR+sI`QF0wAY7=@tniZ2+a5g*Yn-A?_%U^ z4)_CQ{1FVjZLw#UElB|bkf`jNyv<@MS_?#1^X!vr8r3eA!8hb75yinh4^&$szo<_ z`+z5pCir^3`#1ei!;7wTZ+zW(z948GyzZch28>6RkW)(*o*qGx!XehZNupYW+;sXM zr!yDm;0S=yAWK_rz5=P82xuphl$>cM0;_AKcsct6{We{|(5o zeq^A9a_tF~xHz*0q7){G%g?#tQ)C1H&0_&}8^-_IZ&N{!LsoVaTyn;N#vOGXc+$NP zl5g&HkhYDslmD@sA*L&zS1F8U&$i1p)XF9t zJWw}ht5)6Y3Tu-Y( zw;8!sMa|nRZ#Wb%ivLm$>kTkOup*gpED5&F$>N=?C6uZobmoRK6|1fBlWfqv|FzAT z&iLcpU|%(skv2wM_i)YohHKsbj`8{5Z+ag0!lhn|q0h#J%;2eR=$}L8=}YdX1v${` zv-^0p;}lf69iLlIu2;kPJq0(5-EQJ85t_9yqw`?7VdABe-6nKn#ni((DN2LccIo}J z>ZzJ|bvVQ0S73*nQ4ww(@!0M&hepPzOA}~y^-?nKSZFl!5=LA5#3WcP_BaU*Fm#28 zJ}_Jkgv>Zwc4hx=5Jo8D0oh-8yK4Ef(+NwTHCoXZ+Jh~0#`}ESyTjWg#x{Q7K3h)< z`7iS{LLOxWI6P~X>{-)DNTv6hI(s(xGbsAeVU0-q;!7O4-HnawfF;VbKle4{pM{?`Vin;$TE899tj-IjX_s+{uVLh?Lpo|qg$!-Q4hz$g{ZE@aQB`+$W zs4OM4<3}Wo5(jf8mRHF+tdy+1?{Q)IEb}B%Eu=50yQ!7nZ;Tq`;~6;?vpx}Krs)Hx z{72y%$mrueJRG!+9_SPe%kIxHlNbNu4(6;jCAzb79G#>o9Ow9Q67Dj~a(geZkMgzi zy}in&0RT?+lOJ1AC2CyW+Q%{5O?Bw2Vp4i-*HP$j;<-I`Pf6N#U9`Q1hdUz?+AVSI zpQLYBPe~R@4507k0YD!Ja+MGNAkfpvIgZkYw_^2rqzr>S%7mv7?xqOEBNS}WSl`&Z zqa%8Xgvy-gi6=MJld^Vlfyeh}SdtS7O-k4%X(@q?n(e(KqZ&1k9?C6ClG|!Y+P%rT zNugybwxPGi^=7blL3W+xg;vI=ik_m(JCnb=IA}|5&{lsJPcGcT z2Db!n57uo4A|9=}<(84!kf!=96!zei#=xompy)PZNMHF^sRzvNqFh;X_!|i(KTMY@lT;bfcuUr0%d_7^ zX;hF(^9_(T39K6Z0>qI9GL8DmRiJhM(vIeSb3#uu<(7Mi8G=X>`OCI`IpSJyovXWw zYBNKyXp*rYox?VogZ9E;OQLKBqcJp|g=|3Ee7ysMj zF}q9{4LJ8Sp;^CMPR=6O#c?|3dF*0Qz%U5ak$9 zBSB?&om$~M;V}b>`)>kMA4v8=4_|LEH6jEva`K5RI_8dcyzX%VM=wz63*i^GkS9Jv zSh_a599HJ-DA)2})VmVWy{k@ND%_{k_h&^8*8|#VZot7_Ep_xk6Or2mj!}dd%s5@9 zSDv+h%|N}$hThr z$;QkD!E*}qWPp4>^9`QI88nnpOfZ@3j>8rhz4D?Rg`wDPS__RAj8fRI8Q zz}^07Q^4b6v3K=)gZ<)G4eVkZ&SH~s81KSCW3tZV6l1pA6b9RQuP00`(7P`!f;J^F zz3uF9FO6clDni>8BQ+n7qq;2D*T?lqQttUxh?d9xfK;g8Kh57?+U_5B7~_ona7{)zq``d;y#^M0~O%esB>=)h{AZJ(6zLrt7~8|PN?k3XuLG7D?;KdF`(MlrH`V=MVMXibnR)-67pO(Ns%t4 zi&@X-vMs45>iXJrPSaUO8IbHKlOEba>2X^06aba+#eA`ixlmpxNIG>rdn1B7n7@_7 zM~pY;eFE5~G2+kFwNO9r66g|m#`o>1RdJBeFfHJS)*|z$&%hP=S>TsFlTX=fM8VB- z7j-A~Jn{sA8(OE4FyCE3lVE-E3_UQp<++85okpV)mVIauPCi*oSHgtts()WeU!Fh*z_E39+IsoC_825DC0OH=F>Daov%T4lD-?0I z5NX?74!y&uwAC0i>{_~awhh_Strl`gpIA9%DxDNjfxZoP_Pt2CT%`A*AlA&I)j@h2 zQJvD8mV=5Yu_purZ?ZN?euC~E$4>?lF?ds9^rF6%N;LWkB^QnFpIaulCeT8GTOSEq za^;*=eKCzvVy0pJOKzCIoNz5j-Q=;f-`)$XFwO z#LUvMWs}FRo&cS+nE8^NPq>nC9r|^x7Z;li?i9h;Wwr>V@ zb>oMPOPZRa2s2Leno*~rw1QC$_KhH&u@*F@V-g2pHI|}k1M$w2q>^04Xi+b&yWpDd z&5sJx%~!6|BV(rly!O#EU7ye%jxyepvdRg-0 zV3M69Juop!igpJ!ASClQ#tQugPPI-vU<(-qFThz+qN5b3-2X85U0WGk`^)7YHtuKj z47Ax|pT_-4Ebtdpib@~b7n>L?ESaf(Vd&2HAxL*kTxouv_e>?KwR5P6K+a4*>@VwA zAy(?7kq8_S5QL0-y|} zs@|^1K+qBCoIgP~Kp9(rWeXX8&E$D75L#)KB`X8#oGop0UVT%{FCZ_QHIDOR?-51b zs;~&E8>Jm5&6gmv>QE4fTsl>jn7#GEdR&nP`Ic_eOl)vm_5%(I7%R@4vKt>Rjp}ra z7!UBdjU6}*jaAtAHY7>2&!~#XN!64@Jg=S*u8nuLolkzElo+Qt>$LkeZw~r+t$hkp zva_uSodWRRgOE-kwl({gr!-u zpN*ysEscw6)8D@r7xMOxzu%zQc2W>E5C1gJXt#e$fJ*v3u9!+FO=@@AAOnjiHUe6&gop>vR2lB?&X zme_LQe2cgsVsy;1f^H&uSWJ8nvfkbKVg5a7`Z;|-2Km6|wz8y=-_OBcJcL3^%fj71 zQ$b{4B!Yk(G&UWoAyQygmiNJm^O}ix*#}t62o|6_jEpLMi>RaaTKdrL z`uod|Fr}m_?_|(bKc}r?!=|&?);Lu5Gl^eyhyp`zBg@41Qn#~6bJpjj zD#ZPFUoL1Pd$>Rp#Xp>}vQWp>4Eu3*E%;hSLnkF*`#N~sQ zi&kYSP8Mr+bJ5#WuMTh^+}(mcvc>C{4Z|vQRW4K6qtkr{WPT!&b%t|V%H@MLwLRTV zkULqiu^R|+Byj>Eu9!c!;La)_j;5;eDE$eMLH%PESbpGT4s6Jk1Z8m20GZ<&cblgF zj%cihhie@VE|4pJ^zVO9jEh%Nitluo9V`qsi}{hohnneup-qN_WjX0Hc0|{P&F%;K zo}$2d?Ĝ*Df+N(A_i*{9JUUgA>w1Wh|Y7Fmn$G3i=5bn}1JCYcn;R&uG5k<5QzqC&$B&*!OA2n(c9(e3l$=BYH>&AR$_(ET#BWlrvZ zxhboUn7wB!pKb)jf3H5$O~{kkb37b}Y^TH5i5~mB_WAgZCv>^I-?DErlT4pi= z0uBxWxJv48ES8%72OI_WY1ShXQ?ABlTKMyGbXf6Z^Rby48L6-T4b;V!Xsl(poeT@c z)gUA(uk#8D3JN+pia^ISJY+fw5cExV8C>Jz!`8dIiWr;9iT;F)%dv6V>8Y>(d^idn zTIAA{|4b;IO7qeolQvDIb$NqBpJ+h>oBIHMT-DTcpC&j$Mg4$^x<)X~n+G3HH?5Vz zMo)L$d^&QSK)TCoqv`S=H~SeEw7e{7Fxl_K5CFdq!MJL7fK0aJ-lr*&lher1(zpov zIayN_@e4Rw^v?MEZx`TotEGwh{TlhqBQnTwyclj?FHhV&A<3@C6@L@g@zGDGde`fQ zd;TRobbi8R29MgmG~l4Ir=HEu3z%Xtd0>nEazV5*^hLJHk@Z5=bjS{UmFU~v&ki_d z;j^RT;rBh>)$@k6?maieIlg1*z6u$*)mT0L^DBbEm*A@Y*8Ti;1?pMgVD*LA?@=X% zZs;AZ;IKsw33Zsa*9d9M7=9^w;QYj_qW5F-_cDh~YVpSmCS?D2K7AV^Dje z8MNt$6gk-g5>~m#5Rn-Wr5bNq=pYj6Wst}VU%3*$Q}u`rpnC({U@p)T#@&L&L(K2# zs@%&Xqftlozspa&#fR$DsbL=;1X*-%tBIZ8y6x-)vxCyMQV6F5W55Gb zJ6n-o-KE{qpOrOf>!6+veBtlMEl@MS4#-#W4#xx&muuav^w_=q#;)cV97XytJg|5Q zJyZQcdt#XkKFXfM(L9hA?U^)*<0r3EN&Z{6_ZL~W_YeR47e85S$tW+6p(jIMThw;C zFx`Ocy6OjG5xb=u=f|%sZb?y6q8*?0BRp@~JXeUekd_P@T@sijk9e>31bK7y$V^(& zc8!Bkpzkr#kQ(vk+nKn>?CJWwevXJSWK732lG(I)hp&^hcuVubY7vC|jW*{HzK^ws z{qsJ8kcasBMYyf{_MS!}(#8GFuLpD$^wryi&HZMtm&!Ac9N~%Y8B-lNgQu$SC*FV`+FPg%L&-QZ{gjgTzHlKH~_x3C9t@l3GktglDz&3)l%ejx<02)HgV$=o1 zv;NA(`UzG9oZDLSgj=p%RumEtLuopPk*Xj1ox z)>VA-MO?EsrQIo=-6@eBMN;!aT=PRr^TD{5;9U~5yHvQfT19wK3UWZ660(RQNRSf0 zK{pb5UR0q@o}LVdL=JgP$B;q+wdBb(I2AT@fGL*Vq)Idve_47oi{PWO?|t*WKSCZB zQ?3RyEdJz@WeKm{-}Y8!Wt>kDll(oGci51-{SM$SKX@>xSbymQI0kd7w1&_ zr|jIDf^y9>N^X+(k&FA5>3HI$R(>XJGx_e~C85{+UbxOei0IYb8r9wp^GeaA68 ziAEM2*-Rx>RhS{cjY5svLhF~g;MT<*Kqw73%H(!6uD`>Yq3T>qTyI@v`9L(e%zw^RoYAF;%C3Ri7Yo557@l2@b38kP;ZtiKKy$Hc?igxHl@Hzg$Kup#D{3(GaB)H07i za(|`-n{iA`@&@5dFydC>MpTES#hBp(Zay4${D1Ph+5dO^ZhR(2W_(^=NGE4U69XGa zK*s+i-+w{U&5MAr#xNraj^!t+Kjg;w{HhIEP-IAQK>hHxiBA6y`#v?5%gG=EXI-g0 zF@Lo^Qy4r7R#FP;-nstw322oh(om^#A4#|!!&9)a?gmYLK<#3`@LAg2MZcMu8Cu)g z_~~yrV$XWSiaE8mH*VLg_H>4AdVg1DG8rKf=&pGwJKc8#b5xUa)wd4E@inV2%BcvT zF(M{sBGNzgr=TmkdVUM7d@z;SpB512uXNLK7mT6RTSIq!$;TYi>HN^8;>rXL1vH9b z4MJ=9nK4hXF9eFO=%pjEvf}UJi>!(oqH8t$)JN>=x8^+V6+SrI-7yA2Lo*ythbv+E zm2tZ@Fl&OxDt=51>bMThkh}&GkhpQc85mziE6?5%d#07!avjJH3DU#dML2gm>Gn!#juFKIQbVPCT(06B;lD#y=K^`z3&+i}ehVX8brNL;Y!XdJp7uL} z>T`2f&8Y>Em(s!WhF((aTarn-s-Qr_REwG#Kc>yYd=>j*6gqT3(K*)gO9!OD(nUoj5oN?z}PNh!#~XwiuWemi)8S zl&f6wRCcyk_8R2*ZRk$0KA*!zjVh%d_^169JOefh=3PF|2rhHAgy}}Eco=4QCs!!A zk06!AspZ?S6CLLYZl64Nism63%V}Y&MJM-b0a&3$pYV#zpZ4w}<@7fpf{bt91o!k3 zx+$~aDK;EU6vj9=6ja&_Ug%E|oB$GY{`Rc05qH|W1+&oaAEan5(VYi+kBm4IE}KW9 z$)wdQN~Uk}<)4l}J8d4x>Ws}MWV)6z%swQ{Id-2a%$EyeTZl(s)d6~W0#3Zg%IQx` zJpbI1=#Fj^E4iJEv&Lj61kM>~>|78aE@1YCT6!Y5ox{_{I6o5!n*-c7@8c{Cw-zFM zBB7VhY^d z9Xok3ya5C5cw+Q0%)b@{4V{-cvT!n>{{}$|o%_?p4A(mpXyS52l-K(&Hnb7!fc;vq z_P?BG1_&YxDzJVbEqaikiQI|Fk$!RyxYw3|twH*Kxo+|~4(vm7>*zpuok6kWc`_}T ztq8XN@iB3VxpED3lD5x_tm?GufWb2f{LW zIl3L&iGpK=NR0<zIi`|+Zp|o)lbzoloQPA2i5|qG&Iz)YNv}}cVQFsZXaiK-!on80eQWAMF$VDHS)vv^xtacyaW9Db z8)BZ&S3nv@52L0(=mdBT{I~@J>aK=n;Gp`b*WXrnsT*vXo&Z4XZvX_{tQU91EkZ?} zI{>-df;PzzW;1#?#p#A!w{R8k+bUIVkEIPxj=^l#f_X-oy5FO0*s){EE7+Y*bw?)6 z&#cz3$cTh>j+wjo*1bdIc?YyZM(LJtE0+TB=UNd*b(bgYUvh0~E2dTns#Q%Q=`Ko# z9MUailLE|=A=f>Nm-wQ5O~92XFmE1ZMZav?-^Z0fFPwG@?yVj&AIgds=2g zxXrN0mGO;0x)N+VhVdKLt^&o5fa(&EYeT+!9#aO^{a>%OpxZO-)`J3+eeamfyYR0! zKwV^XP5Tg$gCP?PD}HzdYLwwVB_mhkb8o@H@ywA@nMJ9Cb|0__7)@Aa+ORnnhCCl_ z^KJcaw8_!&Nb$L9Uua{UVy)7lz3kI|fSepW0 z*rlu+I2ikoaZQ;|Tz*r~a*R_^Nazm65Sy75m(^Vu1zlqXsl}=%x^zYpy0fsIddPu$ zCT;ATLU?dic|b-aL_I7ZkCI4@*%ni?g!g%|Eu(RmL>FYtodqy3`PwP&cdZ|0xIlr|Oyno#a)YALp2GNFpYP^1GxVbPG^k&3;VblWv(&Az|n z{oO)=m{8e^OLVPMvS5202Elc#jU(jJk*u4DJcMmu^OOGc35#5tvvAxVP!~ve&dEL(#}htPuF0xb%Z!z) zc~-4nrF)flc~EOt4*9FsdTik%8c>A5^dvaQC~}RpNw^gjdPWP5XqBWb#mX zKTaE{nh8B;Eh*!AYa~6dA4Z4x{B5dZ_qN1=e3I$4h`9B)t#6p&z2@hKSCY7GxsC$> z@Wx%&zLLL1vYOYxHL7m^6oixdeN}p9e(~*ur zK9I0&Reu`!fabW{I9pk0UAnTf^~*@tJ6XDNJ)ywx$wY2=Zn1<~tokVUD=|d_QNrsu z0zwg!{}+!JyxWu~&H0aI{R1NY7f>8i=uZdwZ;LzkgaXb*r7^*^L-%OToI@$ ziB+yxEnr-juF;!g7%v8DmC-z4Ay^{m?eIek=#aw5tpVFUANuI+WL({ zT_(+`KWPu<%zZfq>8M_ac1i3yRNko^!j=GVvfw4HSIswN6O^CWZ@nOf4o?f%ozLCA z@@W@YvmMXf#Nl#|svjm<+rN19&@NlvWu)`_CSEv?vZ`#@F$|^0L>L{!MhzyXo!#9H z-9@xw?Cm3=CHyVZ!W{Wem^<0jlqt>00s$omgmx(fy&);T( z0c^^~-Hn$;87-qyCGd>1D*DgjV}@S?W==2+C&D_s?08? z>e)NBlgmL4hy6URrP*wOblQzSthBn@6aiduh&&r>n$xO=Ew$^`ov4ZNb!kAF6 zP%@*|e81G>ZIvm3O;5A%Xy@C9-I5Ucwn;s}ID(aUr4Qz(d$tLU^?dQF&7q0wdN_vR zAaiY=bRb5&wWv@Wjp*`1(ekQ|1*ZZ_uWbaQlZ(GN6O?R>O}y1*6Y8H#8w(f0%0zK1 ziaARWO@H~tL7#S+znPyIcTl>zOiPWSNcY5rGCurOO4$8pn1qGy9ga~YRx@GHBEIpP zv|+0+s{x&?3*gK}k#zy%I|VGqM!MvlbR>!0b(N zn3${NQ-sac7r_+D`T-x`NZWJEU!`Q({M`;Uz@#bM978PD)(D~*Jf&?w$2Od8j_ZTM zg`l`kEW2~_$ja-#Puvui2e>kDwb5@Y9;mpC<(M*fs{;Dg! z7&w@j|6jXZbpG)|7Fpx_QhwZI;+c2COEorSOqCF4kLMSMFm9Q)A0YrD6u6Rt!1vGV zqs$2nFM+6n+ydYO{RT5eQ@!QwfkIIe7<4i*Q8QMVnD{QulIYo-e1H4Cx%YP4o>ndH zQqft|(b1V!npRkp`1|W`R1YvT##WX6WsO@q5Ij>-6y{>NF+~=zlMQ+)qWd-xvV1-AHTDzzIw74H0(>a)?K)3Hp}dofKT-u^4|uSNt2HyjP%-%p>ixe(VQ z;)NUM^g(MTerdv4b$^XOu``e;kF^=W_u#{Ek+-Cwi9{6H>Udy@Q>BaOcK#AkLtu-M z6YQXLv0jE;9}3o-{_C;-FE8s_}SVR5P>*o5-=;aG>l)(~;_ z`ke@J1KZjYdh`UHhKtL zb;SRN9%Ma$zPF8!Mi}ZI6R-kl82So^%5Qte+z`w?zzWtB=OUU6)i=3qfnv>9#p$40iz%>YQz;*_s_P1obuLxxV zfolkDC{GS{E!m!44w|#b11A>T>V3Z4f(EWJJcgD^HoCp=ZI+#MFW z^$z?=1UvE@0c^n6C|^G)7;FeoD3QMY9mu~x$pHm0U5Ee#vHqBYWqD8`gkk)wzqRzx z_xt+nhAjX1J4SkL=^0F2G@ov>1*FfrC9DTmxyM- zqLFm}#gGqpO$D3s{}WojYe?MywHkKuqTEs|lk^E)N!1DZDIG!)7J zY|kV#yN@s>tOebaNEbF4U)Nvx(^i#^BQqrJ!-kvgapyxm`_T4R0UTl{Y*e>KS^jx20562#7sY;9+5^Kj}fXhJTPUwYZ<1YZMLbPMO z$8(2tAkv;`k5oMXK(_~OkGJ6u;Hwbv`eiHP1#c_$g@Eh79)I26UjysFwBh#>>KEPT zF)*wTytjX&H9!N#3*<&%8`O?)zJohp6Dp?9Wi7k`ln#Hs+jZbsgLEa=1=SAig^0^< z{TCd;tzRhA6WR-smFOx+EbhAJdPiYk8>E&+xC6@ubbDAg z?FZ>Mq(A?tAGo(RD4`f}D4`^MX4{TTxy#PAs4@6zD56CU?E=XU0HiT~0 z8<38m7c=$@FVHSOH+VVB7c>CcZS{{3S+hOJZGzu+FJ3%dzs1Y1Te};Az2{K6UO0Q? zTQT-v02r^}$7qk}3-K!<@eaEoBCp6twDk9*$(!ESq{&;hPp$NK9cb#;A>YUYoo38w z_IH5v_XXO>!~P_r_tv!W8yU^WgB*0~Pmqs1M8l+h-EuH~8VMEoTL0`$$DZPS21xUn zjyxCzOY=S1k&2bgeTo%*|7`1{558_(pWA9h&Bn$5Iqw znqWvaY?0T3>HIx z{2~F#dFWwg;CTftH>3E3q_Y(l>%piEk`|lV2A7ME2P~@|Y-lzK3KOT=MKtOgADCxty2MZ?kIQd=>P>E9hmV}w%e-161aK?;@~{$_E>01 z-EhSRa$gwBZ~^lDqqxG++0pbkt9ZB_?#;lU2CUWGc< z)v&fb!34GT)^-Qx>w_})xQy_%7@%%W&ox5MU zDEy~9FeYQTbFmMF1_s)7i0+e-)9QhhwmqwzP|;u+`C_3O?5SSA9>d^DSMj!sadnB_ zTZQ-Qgh8>)u?tm8_TTnBKXH!_2@nZD$z~Z3zJdHOPOw;NQXI~?q{g6LD)%xnm zD_JFClly|0Zw|=*=6=xq?(9H~SA+}faD>_25)$Oj`=ii`QmDJ)x>8rf zaG2h{{uhG^9wA-<-b>~^Cr(;xu}?WdH!Y#EMUBodI#TDS25&@U4y03Ktu016>LFUrb;t9y{O z0v{0z5@)0^iNO+!bCYwlr!=i#t@juU>~dLL$D<6k7|ZGFf7oUTx5V`Fs|om`LnO3G zL}UxoM_nEy73G`)MGVn?CK`j|y`YERIND;figa2e8DpSBFdBr)SY}80!X-Gqp)+=# z_u4yyY3L9q4=CRv&4VvN%qe*9HL7^@dOD%STJnYWu7jT469FTC-sstoTX)>N*bJBh zTibX3R3Νj*y%_4uy?^A$kvj(pe$p(x&U9M?13J0e1YUC&`U=ce#Ob9frWF-GJ= zu?Z!t192G=VyDvllN|b*7ttYJd|zG?umF6B*V{}*!V!p+VNX+fbcNiV`_tCIIYw<) z=*r>ZQ{W9mZ8RI17JuFzOgl_f+Qt!s8ddb(E~d06RssGn6{jwc>YzU*k0)@h2q&rz zjYI?3`IxeHAR>ba;?dV8G%qZO+i$?6C7D8u@6Vb~B9y|49!kv$W=4kr@U>#S7= z7s4sY9{zORKZ>s~YnTzFdaDbNN7VOMhQHq8(VeZda!;1DIrsli zb#JXTmnG)HhBP$+WBlN)9=>$sykOn5{Lv;;i6v3k;gHO6xfah(lh>&zii#F_d$j7G zB0{Rs@8bYvaUq~SyT7+O6J!}O$<5Zo7^W)z*i5`#NW49w3d5xgxnr#q>Lki-Q}YxN zgfX6|nN8`gRV$!WcIz8a!+d2C)$R9U|Hl^S;aU6$xfq)!f8E9lRa|+dGY^%%ZmQ@h z>LIJT-Vkh4_NSh7V$q$@sTo0z&Cs}{*lm>+vMxBY>|V^{vFcvLqZW6;VRj3NbP)!J zwau;ipM6W2%sJh==so8G)k77wG#A3fj;9YXV2rfoXAi9*yz~Z_3%fd2m z8KG4`uFKK`kyCYxA1X8olWQGaeKX+~iE>r8fgJe{2A28gQe$Wl^55Kg>i)^4X~3MP z-?@N-y1c&r6Di0NSGq7_+wU++UxLCiCuUTGGw|;R7jm#BQ%-E`--Qdo@Ut@y?H>dv zHaoAJEVx6^POZS+x`L{>Gz2eR*p)d%8nZ~HTc+7nhvAbgV+H4y%SN$$tGhJkKR48I z?tRgfkrkhT!$w16vzj`WJeQ?CtiWR5*4Hb0A0M?jS53`2F((VmbENfRG>#H(C5?ugsR#izECu1s|_dZ(w`KQ2DZ zHkqpTYWhn+z@yOXEfe4BIgJ|^fZvOe4gly3oY-d?;6x6lqPN8at=&u!Di|&f*mYSK zzYn^V72Y1q&`b;NRnXDhnk+WKK{v}(ya{u8%Pm!9%*AOt*%9Z6%3bG+Pv^^zo~kSi zGlCDYuo%zYDysTw(?rMC(0#)H4aG;=P+;*&75jQn3sd>w_!7u2G%($)1gP##M`KO% zv{Y^Q_(0$`-!}h2>7(3Dv6tQ;^P%OTf=kR+`2cuTe+lsP!_GHf(9`ie&ybp%QL|~> zS5%(?TQ4*XJf(GR%q}|AGaXepCAM!lb{^6`Nk3yhL_L_iUQe|!%D>wIxbL(3*?C{L z`50}!`J!2axeKa&y}hJNwr`j2t`DGBYpSl^iP^FXx!P@gcZVw@l^z-0;j<_web`8X z`^jor$poT3B%>k}fj8mZ{`TM<^C!OSgLu`EV;67eG8dpYm#_IZ^YiI!sq?Dyspzh% z!=j`t`wE`S@LEHy=Rtr>j${jNd}Z(b&hUfp#*^Vuk3$!=3S<)qsd8XhmgH}is;gp) z$-}9cB{afpmcXJx11feRh}y?(-t0`{DQmlZ`1ASj$4eHJ*Izjvy%8nZnUVsLgj<`F zK+Mj!(s5$TB*Lj(6jEHka7Fiyr}pDjd_Pr4rF@C!v0a!R;jR5R{oXuF7Da_jUrilu zArjkwr>0Nzim;r-JU40h^84d83SuSP$q0blI_tV%vCpU|AG68wd}rc0tc?2ubcaX8 zJl(3bSdJo@BE`NkUaSO%iuuxBVzdLYIv@kw%Gfue!(fcfxkXzXCBr1g1PZaU{;p_H zV)^S%VK=O@5m>OMUg<=4va5KXwfN*|5AT|mWY1v9!KZgN2)*Y;iO?34AG)3?*PSos zqM;_a?&*3rJ_V~6+V!b>*}Bc~o6W3>?=CrPYjY;&Cl(;$VE~<>9qobg`dmeB`S-rv zT7EK_tGVY1d6o<&U!c3X6jzG(Pi6jd^@LY7yygfU>P&EH;$9rA?6K>6lj)cppI=;L zmnBooT)RIjudeeXc;%oObU4<3vE7Novvo_F3qAJj)KDkvzeqVNcj6=J@3-!fkv>WmS% zhv-T)5f2~k-dGju9 zC>O77^WW~g^|&1*Z#~TEH?^p1VuSCxoqIqrtsvShg+ca9!b8W0A^(|O6=QVG1hZV7 zwr%%@L#KE3r_y&Z2HEG4cM%y5vk=@#sz(y-7q2@aaf5=1bmGBi3!l`YiGml!jvLr- zAC8@2hr~}H+3`876JW03mC#B*fSaX;EKnWhiIM|VuBl7cvCM@Wwq889NU{Kg^Bjk5 z(`5xe3TKF}jC)(i5r51^$0b-mlephJ=s)6w6_I1&iQq>(TfXIpKc)en;l zM9F7(BzVK{^+|#hY6mpvL%2IJcUu^_7cjxSqTHj;IR&QXA?b+=?Pf`E5|73P<-EDd zrT*&z-K7u&-;S3)pKPB!5CGNKCmCf_GLwR%zYH<)>R8j3Kl7oaO=00r$bwUY!&V?G zQq(>6aL(WOxoDVsE#0fiCOzRV3*32%Zc#E;H*xwa(f0*$h^0gvh9ina=oaVm0V-~r zpS#4q>T=~=bHrEfjqvK&W+S2q&Rz9f_hH0qUD%#=+{?NTWiBCcc(Z zH3*DQ)zF+V%TA|TRA zGgwpmV^-D_V96XV&#oLLAIqh-eH=k0e;K&!sm{4*90LKre;j2Eyc<7>)MgotiEeap z){3ksHG1BOz@Z#o>2SJe$$($|PWO~F`R1scvW}a`G<#~*cK|1aDkA1XL(Q|E2n+^H z3Ud-6wKmG9LpT`-94WT2I>BsEa1Zg6Tppb@9ybcvbKkE^rw>MH^2Lv%-CMu<$=6h~ zj=ASrCD^6WGu=bJ!#r`d!X$A4r;h?V*I(L10B7thXSQXV3A%aeQW(_HuvIHJ|K4T7 z)-a_dicotg9J4y{A417JzTI^dNzi{9=cenz^&lWX9}$$AmHLcGj-)Y zKy(*-fsKvhDXTzm9cySEu9Q4~MN4_{ljdx)+)fmeAzpvDQk}j&;7Od*zK3UZJ;urn zO@3Jqe&Q*u?WQyN;wN|9Qed=Ru8iBTR(8_ePSVJcbZIxDi5>GEAT-t4`PVumlN*oL z5d#e~jg?C#{V@*fl;UqjlOk#XB48;{SggL4OoN)x87k8K8)mVOB8x-!;wj3nV*=IQ zcM8DEzMtpO?v=B1T88v&BBsQZGuJ}TG$qP(CCbdZvF&gRX6hHQavT+@3s`B*z=l}o zVTN3xA@um^TEi}-3=wuH>NNB3&#b&H%;8B#VJk5k6RKj^S%eJsI&)p32TPp=i7QBJ zZ6-rkh0zejzT7|QoF{GU>p?Nb-(P>}dcuxUa9sxd892s<2^Ff7Fua^rJ*y2d>vl_!N59+TpDT$rrTmMd!0`X;;JToNGg|g%}0rjA2PZ>x+X>I z^h}wk&R(|gTrA&&QXzpm zb=6A$pmr!wcnHpsA{W)B-?1a-p+=fT!MoeaX2nN8Kn;(V0`roRidRqV0A57JSY-#m zOI#u=(8ENW78)+AtO8FiS7IC|pvqzSQ)=XwG+1e_X@htc_<0h2-}fsuF?CIfIFPbJ zc5eno1A#47@HDJrb;iuxVIfPOGo#jygUH66BZA#1h1Tc^ylCq;F4Ku-!uVlHdq+t- zqrBTC81Jj?F{O|*29Owsty0~sjOM$fJ8C_XWUMLPm7~#;4rLu0%c|6&wHXDTTT&g` zHpXB@dBcu%iJ?;X!c=*%CNVM*>>s3$SnKnwqHDxb2jEW`L`=DCVOlGZZ40lx;>U;j zYDDgWlHp^bM-cMx^gkj~X#B7YFmiZNTEt_8ps4`W%+qX@Vq*1hgg#oYF{}~jZQEX` zW#CX%YdNyyp=vP8#7t(PBeXQN6g_&U@Ye3u?nJ9WvBssO*$s7{+@_H_#dsOz7^z0t zBm!srWvTG-L`}Gewk+G7C5iKc$&LOv^J3MA!cP9!8f8{1O;3TL3hS;O51_F8Qo`|Y zOGb;d!MuQq8O(P@uEy)!bO9oqCMbf33(f(R`n=%1qjH&nhL9-1^szve1|S%0P@dix zmnzvAWt>N2@e+h2$c!0&w(FYO*9)U6IhR&6z0Dc}`fEF04&6vp7{|XpQ@dhZArAdx zR50UUNDX4sY4qK|PIu6lQz5fMT-NIJz~>X{$!0a7NgqDxA|{0+I2R9W!|c5ltuo!GTz`&{&QdleD+UPy=D_UW+42Z(clcCDGRF^tY4fM!fw2ulow5i5o5 z8wV9D#~Norh2_cPFviAMl#~uJFE=a@rx+`H1)DSPXM*jQ=qUa!?1>3!EKUO@O!Ej_ z_3&C>_=UiVb#@5IRzwDVcYCBr+z%?XHYg2L3S5P4_e!CWHqo#kJ{(FVEQ}iro9&&F zyL8lNWlEeJsE9D>-d^H1o2#Fe7gibR3;bhb*naLFb>;q53<5jN_+^QmR%n)#7sQ9~ zcm>|4rSAvb!l}F8j97f-Yzkmx1n3@Tl15Vm_zu*M-(>{d13EC66EK=#9H4IpKJ$ZS z$AaF(*8#)m(|~L8?;s2|0IU3F@}R>Lb^Km<5HaCvvT2;O)xXF@I{T_trO68%AuOhI z_)BbNS8UEeDEIbp2^boUJRt>wm2iqQF8Ww)G0sluH*VGJmBwp*!|5Y5@guJ5_x;zq^y@-e*a#*_w2#~3Q~+O@QbvNvg@vyuLLw^LLZiUpwNc6 zWSb_P=ltIIkJY^^;@MK`?IHb#$EERNE*Vzw?vOriJsTVT-DFzbEvaUkZrU+AK`$mZ zIq+zhr_I{01iPDV9_2}R!r3%7IWDc0@sa6DYl~H_c*;HIz%G+kp*^Wz#Jf%UP|mKr zu#2hc@mygh2lI5IaiC0UU6`?AAUChcpc-Byqa$SyPQ%_k1s@D^U3?iz%Uz93(KOZG zPiz?SWNOY+EOoaNF(s7^pFcW}o$cQfmKtsT=MM6!$PAH*jplT(TFKb-S;MTIL zch`8?F4CAfjhMQuS*xZmR;)0YuUxB&w970{_^T;w;C9oE@B7&|F_`LBFIvXd8z!|6 zoXJmZWE65}%Qms`xFQtkr9q1nbY8b4b3jX{6@sM#+w)VGDT4)~yxi523yK7R19ND+d$}a7RSDRnE8VTkT66M!RGc}|pQA~|)4Hk_z zj#sLkB_I88QFc^QtCm$Y>=u%bwUKnnE9GvoWRwDFVJ3;L1piwym!Q%vGp}(+2Gle~Q4{FL+Kx)?EQ;*&i6 zX$`m;uxAIS``?q2bJ!KJhFn8vq;V#bZPM(Xa@vuM>+%W0kqQGD*b`&XLmX(>mPpr) zKl=_Io1G(hAz=;NEZ5@JpU|q)?oO|>_S#4dA=0@XOS-llKcT4oco!+6=hxI%Qrp;F zkN2pi*FHRVS77%sK4a17&uKdqFN^N+t=CN{0+YpVpp!i*2?ms)OT zeQ&=@ZdbF|2cOuwX}Z7FkWA2_;8E0O#s1Sfh!SldvJnIV8}>m1pE3yy{&gzbO$5v* zm>1{KBNe<0(Ww0qB}1z*lNI?fLL@yD@dRd?&7ZH-2SE6Kjd4?Y^&Hz3sDoBjm-M#e zPW8#_obDfjx|+V8x;JL6vn=aA`Kr3KH7yD;qDQn6X3-TS4eE1LDiv#IwHw+tu`3rA zVktKWz1Rk2Yc_`FUQy6bi0eKm$q5Uu&V1_A4({1BsVg=rNmQx04KA5$7O-b5rPUjk z5$ezFAQjTAz^Ga#{txFW6#*rGhcGXjHmpxCKDhCEwzUZc{(zVWp%1e`7_C+#IO=$J zgEcHF)eqpH?`OkSIO#a2p|Ha| zg()Si?^aN7JR z!oBDh^4>kQur4~v<^fgd;AU*TOnyaeSH+E-$8DRuT4iqaQw1?hscgnRCV!A;QDmaL zW5}{O>ADLa87oKUx8;^5;LobUBI1RQRJfP$(d++V?29IE2fi7`twga!tp6LIu}H<3 z@hTgi{VJAo8D;9Xs1q}xgr`@{*D+&D@F#4bx0JqF7)ZH*+u-%+D7n;P;_2XBc52$a zt!j}DC8}v#gXNX2yNo_3)G)Y&C9xT53N97NwpB^Lgk6K2OiMALk?2gRzcp7(UzPHG zYSxFhYxu+0qb_1YYlfS}yf=lKS6s0rj|MyLGXVZ}>kqtyn8_!Ju;RUU25(PtN^UGm zUB%x*svcjTCY5Yn!OR@p1)t>Rf~D;1op<~>E%-OQsf{B~iW|Qhy>0Aj`1?a08>B~& zF3_rnNXfVP<*zrND$y}zP{qSDQ0OVjDcTI*Ss$|6WE4$^nwAOjXOxia?YpYu#)2xK zD=O5KOloXu@yOb_`lb8m;086ItWtj-ELTt4-!B+QkJHM~z*%KWAl^Vi1MIL)R$4RB zf>N=;23Llfh60s>^0qD=S>F5n@nAzx@e#Nj4aSxz@n$e+ zS=BuD<(+p6vgxf{I`##3Y?}VP>+6g2f)?7TW0L)o8)S)rAxTv9F)>4JWyPB|iwK!E zsS{_a+cNmb`9=syyZ|QB|HF(>E*j$MI!F|6ZI`X{Re6bR16KFYs{7DRVBn z0B454kE1BFxV_iC>n1Hlp53k9c8vqG(WY{dm1)fvn#z|oJqLPUg8;43v$+~NEluzF zb@*l~tX67FF5{|G2|!i{!Wid~Y!j>(IT z8i%B2#U)&8!#OW0=zrAUFn_T7>a<3;t%@f7e(&gFG`}0wMC1A1Er!My^?m5b4Bh;k z{~TluA8<`{{t(%|J)a28@kAv(nLR|`ggo@%?ZmpNNE4@-C(;6`5{AP%&PdxN$GxRBr}QQj*dh)b8}kBy@A9u za46Y!;`2D1D7jo+?7vbfBpjANAAz=zwwye- z-si82Z_y{R-LkN74&Z^HKEp1+Us2%2%}k<}7c_;qP5zB7%Oh*NcKI905n7XbEOT0(B||XYkR;T3m92TT0Vv5uB@0B9^JMMepi<@B@{leKsk6>-&|YX{WRF zRJ#i&mSC)I`)%KpDpkAd5O>AQW>36bMTm203jT-O2PN&-UK>#jPA@%nSOYyY`$Vwe zv%Z%C0uT*HLIqlRa(v0yrj&jR!`j~qziFPSrXJGIQ?9F2j@1b-T%fJ1L{b&1b zXdvXvhA(8G$U))ig~vu!aMn5tycs#yXwT|QNAOC^M`?~FXC9(FiCzp@f)>uWHZJ?< zF+O6}o>m#64BpX{%7QwBlwrnnBbbD|4zZc|efGDZ6Bu<58Z;#qJkgV~yj>@2|GjT! zJXQ?Y?DF;d#~AtXM8c+rj5O*aFplEd%xE0Q5tz17?Z(olTvy-p*ATX!SC+cC+-|uR z+T7OmI|y8pYp(Nm;@CL(oR14}1Y`g7Lt-AO&%Ds)>vRzt-8si~K`B$tS#qt2ly-2J z!G1Ne8;*vnLMz7AL#?SMe72>aE5XeTy5Ig@MV~QYVYrQ<01e|OT70-+(-_zRJu&86N1c; zYC_e)X72-$(4UOpz+q!2n$EnHzQE1w3QJS8m$q4-mi5x({Uu&$e!TN|jA8F!t|ZVm zI|dy?2FU5l%+K4 z6HFGk6;;SJlpS%gNxJ1!?U~pyo$d5sa~@TKetEZ_cr*@j z2!9`b4|4`SC3D+Or|qs7pc~Wto&{s2hz%EVQ=xO=F6tNP^HhP)Fd`CyFX10$28gr4fm`OL3gGtlcciB5(-QuU3C>- z`GcK=vDxLp3h^&Vf}@mWI)bxg{ux)n+{MC>8c(>)2sNNzWba=uJU<(`9lw5ct&$iN zmjr6E13CYI1_MY8KJ%=|pN~11;uA64Zg&|plp>R20=vf()McXFJ+rpGQe48BV=Hi+-XRY48vJ{_2a+x? zF&B+28%_!o!gglmhBRt+62Xczi0?EoPgntls(tRWV#lt`PUr0}o$ZM^!c@*GwacG8 zH7;q1BPI9o1>q)x%BW;U#Fa|b%>rq8V`T+)`PelfNwB)|=SROfY5OcLFxd`{V(m>w zFsmc)R3ACV8#(Q&ncOwo?*XsNn5wQ56*QeSw%qDgBbS$^8m4RE>WkoKJDQkqFrHVz z&UKnH9!K4}TrP>qgwluw<2E(ghSWAr56=tWE8wlJu7e=^aZrz@u61vJ*Je7+^rD&O z85h}m9TBYvGBd+A#C{0pN_R9?^|Z=qhKd{j+O@c&EcW3RgqSQdd(UvcB1r-!Yfc>F zsE{KblTU-~1YrBlAbPA|I3JVO`m20ovqzQO>S`LZ?FYSW|GGg2EsSa{Dy_;}wh>$p z$QY2KWuZMS&7u|H72XdXd)>QI>=+W6%k6H5p=aC)Rhv?{1kZ*3LDA@%?=#lFk!CfW zpW_!Z-JeK29>xY7_`~*fg~_gT-+Ah`g|18B=U&49XoLduRgT3>@+;^XM2|OTGvc(4 z$tN&&G2`TkW0_+gq9)P^W|(AGOg&9ZF5$b*K6^j>f2JxArVrFd1jgbx`_X@Q%*S}8 zZ-?UN#m@0Ee~~lhXIh~>;-oduw9l!7N7A0eNjNcAFaMKN*3ss}5pEpC1AQ&M_CVNj zzMuJY9?)+|&bg@O%pghMI4YVnn`p8>fms>{A77T~xSoxYjMbq(K5lLfMWgY!+yi}R zAufUL(PXw-JwJT^KsL3(P+E2i@!VYkrtSHVL!_R6tjo6dMmk_|g$XT{oMXf?9^>2&Uh&=d{7qorAQZJ^DY z(&Cu8d+5;{^nHG#_UCHM^N#Y>%G4#Df$uDtk;2pA8R>}RXgwzhb7Te(57iCEH>6pK^(K zowz$RA+$rv@n-q9u}-F|=*7kcZyt|9H}8voCvKgo;k8t=zWEFVF42j|6vYkvkXC1z zbxCgew|`zuKkFdtcFU6MEUAV!{|?TMJU;&8)@5OL>8aSGxxYIE$Hv3*7(pX&m<-uR zmPP1-ww0GkvT+?M^1O@IaivrSrk%J~UKRN%={Jt<<{tGk^K9Klah6x-{TSa_zE{Ex zt&!`{tZo(grmh$zR4UrMKsG`Ga6XRZ^PI^;zF=achUQpzU^eS2g5{(nw{n~gLYc5= z6Ts0DS0M8VuI;zqXd;r!-6o4VqEoQ}%T_g~lg#9@0A}8yIHr(UVBFE*LM?Fu;2Po- zRf~WiEv>5j5D*V=fCmY3WZ6sz44sPU19S?tp^F! zt=>j#2&H1(md6*WB${*7_WNcC!4~Jq4#Vg8PITQ;`iZPA!@!kIs{K_rr--!g^?2Y- ze6@f21If!d)}09S2QBudtG!Nb-VUl_7kX4j-e06qr+@6iTd7gl4R`$`n59R5e8s_B?hnK}SSn|uu|PU|fu zYG~_s*H#S)sQ_Ri@S(YdZqi^vsXXgPmV!AOa;VBlM|`GaoQWc#(u#49aC;fq8u0wL74h~`%*#|~!vKOn zZK~>zYzd=rQ#?GhjvaiRlvlku~S{ASeul#xXL{0=F`0MJ;jlY zTFqtw58O*xPatWr^?BrNr3c8iN>7OFBBPyzbyYx~M_<)3zZA86D8Tx)z~(g#+P)c7 z*{O&S>QWgQg&hhvm|-alH>{lEhE72d1r9=fc6wUFksDWx4!JUx*zhzPR<}~GmKNk3 z+Zc=Y6ByANWKT_9d)@jCw|ytaSGZ$?r#|IZ=#9Z@sif&kZM5}ei8dMd`km}wldo?# zrc>%y57&Jpo@KYIMRp>z`HM8)2s}E!+rJ8fGA%>S6^VxrRZ1MaEXyv5(GB$9N&$NY z+;QpmJNude>_eSqDmVrq`wh@sw*}(#>P8)KaMR?VhcJpH8Km;$x-_*Yf}&+o*9K=p zPpAh9#ay$C3+pQ^y*$ev{@x`_;yD(&R5~=&ajub`l^vHm;igGkiPZ4-jarg9%q!c| zYnT$3sbiFfPn)>Mp<^cPYmag6q?br9YC3#9H{3_i-_RE)&x-r861~%ynwbfC|H`cx zGL^SX2Hu$MLIIxT1nYghA}pGMa_Yn##Ol|sKm&rN^Ni%Dj{lk1TVx0~t5<=RUk+}p z`Iu%z@P!+l7*|YL%_(J$JGx9+ndsIs)UQ`vdcrNaYpg(gIN6sCo5LRNwPIlvB(R?^ zQ7B`dj};LBQjysTx%tPK+6D6v>q@=6q8{djGXIOJyEh7isyKbt&VvwC8jx z@l=qtu<3l`iM7wAcDD+}+84ZZg)3M1rnkJT(tcJji;iy_j8PaMVDaKQkfa_y1dHY5 z8s?!)orw1jFMZ}x=tJkO_cR=-?r0`C&-Ex;;v9YK;kNM%*O_nqK2kkPViY}1$mYcr zx^5}pdWUxCk)iRNw;)}%(mfCC)9#+9Uv~cFa1T6^x$x2NY1Q>uD_1+XR^UC6)5_4j z5gV797?mbz%l;DRmMCY4ZrE=i@TITdYfNnUAsd6HG`&_-bwS3x0+ zi0F0vS5-;o#B9)H(GF?MvLuzIt}0B~RQ~RGnMkWIgT)6ItHCJFVZO-CY9soyy z>I~b4ut?xya8sjgJgriNEOb=f8WHfDvBsvPW${TUFw;srs4d(Ox*Pf2Ld_oc{RA)A zoM$%2yCj=U$0NiwVK6LXo#DS~djfF4T;W`7MJEGAufMt286$6dXOfZ;6+6lxIklFf z07t~RgGaLz4QioEJ_eUcamV*SnyMV4I-G(d$AjvgRz#ss38zI0Je2a$Bl zE~?&AuB8=u;1_bgj(!32x52>GQpdBJd?l;GnZb3QLxW{i%Ke#3l-D)6Yh3;)!zTAS z?OKD1W)W?J##QH9qq?fTs=3BR>qe`_hUIE0xBklBr`YHEQ^s=9*Ny~OoeXG_V;ni@ zlc#h5v8h6at1d%cPe{6a!AJqurDM$ssm*T&3Xq#`1N0fxEYDx%vXNu|xwXMRCM=WU z(w-C7cE_~z8EoD#HQ7k;1}0f~t%^XA3y-0N6`tNva7vt!`2dx3UH#73T-aO03=#^y zN7mM;hcH0N7+=ZbNpZyc5-R#b;9qdWJQYs5^kjxiz(mNWf5OAs2@%#N}LXg&^yG#=m_ShgFgIZ z3EoW&k{9&tXl0>V*(rx zlhB%;O0cF*IYZv!?NGD*>C=4U(7ill&bwd=}A1 z+NDYFPo2cR5NJ`>TV+HezCZ7Qlih75g0qPc~gHm zQh2#7_H&Gr(WSrk)axel?rAnFHLt^(&3!*I&`qUt?B)cV;F|}Yyn;RunfL}GF=9yb z0;gFnxUHm1$3CLwFderw(!?%37~<7lm$S}EX?VOwaBX!zL0%3zTaNWH^JS-f9k=r$ z&uQ8ZZudCp`nt2NKW;Qi7-`uzmS;4@zFf-CZ`5oV#!P==*k%mzj4{Ft+W%?~_|ymr z75xckz)Jih?iLL7r})PO8oHBAfkFxpVD54C>pMi2B+WGZ)!fkV=S@+arrx;KFUVNQ^MjiLgu+((RMLv?w8Mj z9w6<;2OEByI<6;F$@io?R@y(>hb7nzlQcK(rH4G*~9T zc|pG$@v>-^e{tq@S}RjOK{OHzuxD6_V^IndQRiDlW+V-e&sl5WDs?3iZ( zu0Q8adp{qNK2LeMuFw{dM=Y7c-;F{PbllWoV%a9`MV1+-gr89~&5ox5{Cx9t?R5F1 z<6IJ4WyC&gKesh}=o{`Wvkclea1*@wk!53xVk>4Dpv7g%$o!1z%oSj0u%+pR>Q7}P zOGZf26f=~aG3p4){K`y6KP=ifCtCuNXxWfjG+EKo$5e@&lXv`-)Y=KvzrS2-{cI5X z44T!Qpw+0|f-BUY(xur7g%ZVgm32%X#04nQE6?9F6R3lAXXZbS`kc3^8;#h_cAwT~ zGAyOGBqvqpGHk?F)SR&;6hoy!v>jJ*I9B6WcFk41t#maLsnGbDB4wauXx6CIrC1jJ zOOY>VyW31ha0(UpvvcJ`W!vy!Oy%gLgk)-w<)sVhYTr>@pUo4S7K`1weFkWTaOILk zod$7*G75FUMuaOR+FCjLWXgY>zPIwA=IcJnc7vu-kK;5t>5y9-8Q~$O5>MmtA76wH z1kW!XB=qKcYT z;X%32WXwrzQIcdaKbK(7Gt;Doe#z^{0?~MejGPD{v9G!U?Mf|eShY`;tY0P+Nn&2U z)Je4mFitppqxt;W)X?ke4w3B&O|1?G2Vwf*1E{@eL)+SSqugsfbNjxv>s&ZL!6S*z ziC5kgPUAwihM`^g({hHybAy&eog!-J-(bSTI53;3YVT7dInT_a3-X4RnfWa?+^i~wp+_ub-NX2^Fzz6CKPp*-luHHse8-r?{Mbjb#eim z4)9pF;y-trKiE=2hA_%cfj3Pj*bO%d2986$ZSeKz?IzUg3zQfsbmuBRS8(KVYZ)?s zc4OE&u@!Gd@!_61O%45gu<2VdaNlT-|FtcV^OV_QS2osFQJM}T8r(tom6cEGFCpy2?Z8{H za%8_@GB!08s%4gF8@wsk@yXIAR)FKj;3rnjgE9^i{3s{3P~Qkg(76#`7RsOr>cids z>5(%>3vc`zbVWn=Io!~{Bg&=`9>=Cgrtv>v+i1Rkx6Wtp3Emi<(nxCqe6_L5bOCrt z`==Hm+C;Q4#%$4IsyRjE&c?Nw2+IbY!&kDt`1V45w}jv#!g9@Lm_F)Z_4gVZC z6@F^-lSGk7TTyK_{423chK6kNX1%NufdjP}5M#CS{r`4&P{sJ-^VX2J$7~SXZL_Fp|7G!_m z!Rkq124~0qL#UTBxgfI@g4vN5yxde`_WA z)p|soiVRSuX+Zb5_`8WGoUrTi=Ew3+n>JTz*cW`w55=sgK4j#H)(=^Z>xh%=Lf5Ak zTSQ&44D(h?M=^)KQoTF|=taPk*EGkSyjtv$tNxb| zRmaFbPrU1)PquI5XY_sr{`R{{;FzN-4G*4Xc7OvYJUy}!Z#uX^Qx3W-e|utgylSEu z(whMfsXZ*OZwZx{I&{f}gGQVS%G+)nst&LRpUcFsvg9A%y2t0-+a0i?smh-#o=?)x z3lL1b(H|H5Z&aWk7OS774N|FwKT3HX=zaw!J2C7Do$A&g)301=0rZzW=l53Ttbne? z4+EX0;LOWF zi&VTlKUNb&^Rk%%rpWZLeMA_dy-Y+uFWyl3bxhzl*_OK?RJI=9T2_{X&jcV{QTLE1 zYYH95Il;4Dxq-&1EV_*N6YmW_oHkrFz)AakDUb?n0)>s7+woW}XRq4}x7%-L@>#Y~ zn*xL4()fxbj%&Kt-y^f!Kq}L~J9<(EhkwFW;SLa@4(PX|?1@C?T4RzY``nm^y)k65 znC+(Z}!3UT`O zAV^*ujXF#yFRSPOet~(n{P|+!jxWzAby2el#_oX6!yizI`k-@oH00tb!0%gn#Q4Ca z7=((#roSF=LpWf?5cYtHEp+n|;~lKv@pHkTq>CA8MLgh%+R;lo1rOu+BFd}*tM>vG zk@yMwxA1b;v@O~=Pn5H8@OS}9k8|a$o$UbanAov;oT>un?HkjHu0!V5d)VLMDV@6F`-QIv;^ z?GB|Qpp0C!u_sC74RM`Fz(a%)`y_-1O&^x*JcQFfJUfgLJ+f@ws|Z$?xoi-_f4RPt zW{A+?g?WrIXhJ=R6S!K0`CEX^i024nASj{-m+!y*U?gKxYcwi_^m`~^>K>VPNTCyE z%)D{$!o(^d@9mGSp((<`Fe2#en`NqYC<2UhIQH^3^RUVQNq#|5NZNjrCHguypC6%Z z1QKw)(2^hp^`Qy}yAVY=5{0q^rN;%i1&Xs3MI&Dk;hPdoSL1md`{Qg589UR|*cW3* zoKUd>`p+BVPfz+ACuaxMXvcuzNBQzXAW^dkmZ=9*JLn_eEM%T8a20?ymnZ(`tMh&DgiUs$`4CSIN z%uAC&ls(QIo6H4U&hw7pJCTH7l2p(m4ZIKgu=9$C>@sC;&#EgmE5%Zs0WQ2?_Gbjr zAh6PixzdQp`6?e}iF#5687G-18Eg_sQ3u_va(uJFdSrrdTf{y6IG#izgekl*;mRjM z532vuJAdq5g3<_nw}G=(MCM#A7E6DjzcQ+ZAsw3KU{T|1DW$os;<}dEn%n06%K;kF zcyQ6iJVp4v|Lf#RMH8v~R4}b&nKktbk{7=Lf0i?ym_nQoHy@6CH=k1Qg-?ORi`>i7 zqIqa>%pLm-L6AGx6kiwT)%ZsSSd~`ixmyOa50;wHfbYJCp`qkrrQ5 zF|?B+?(Fpq)FT+jvc)=sD)zeQdDyc*1rF+r{ZL=RTDf!)jb-7g4uAH76!^yGZ2dW^ z^=Ki@EeG~`1xCQ0u_xUsWBDl)A~I}uri*?b7v)a8DX^I3z`aKvLr8kOkNe@Qab68ka2fau7s>m)bsg&jbQI5+#y*Xe~N$ijnkj z-!C!cV8+6lZXzUljCT8^j@Y4pljdN?cmv>o!_hE^BsN;&$PT44pv{Jv@jNL_Tv-A% ztWEpcSn9^sUjS=XTW5!_@hQz_X+qqUhIp$FyNzAVV4{TjT%Lzfx5GBCq^&v<$;-@t zQ}1pih)vYb?7Fe8$asCZL;WO|j=xhX@WAaqw!SO>y#Q-X8Y$?Q)7n^^=``;?a{2vu|8Fx!w?)z7@wVkrKa2>ca=^PI8?IIjD z6WBlpd)@Ag_>QgYuW3+N*92k$cbYZYDuN@ zPK?kyS9z3a>Kp09{Y;ycU@rcara%Ph3uo@cpXc3pfwRo26HJ1-87>g>Np-gaRWc3Q ztH-M_=x$K`tQ&q;lgZZI5##pG*dzF-eUnk(>cd!OTSciJUR9oY1kKhH!&R;i_0HVz zLnjPCiK79Vr}_TomCUQ_S_EQko(w5DTZy208NL^g&e|GZ+L}r?Ss9LFjI#SdjhuvI zzzRiiT62jJ;bLWC={LCIG)S^Q$qIjw$Z5FB2t_3b2_0JATRS|0M(w$1N4Eff`e(!h z6zcv=AflqOqaS8PILHDn@88wHM9A98tR}YeaWfX_DfZ{6E}Vj!H4N8RXXjj_v)@Iye<&i;OBjK7 z2=qwEng{XfdMGID=W_@O^9PX*;x(5n$jeXv5IjnbN~G`8l`KNQkC+SwLlCi-jbhq2 zQbHbu&R+r`5k-+Kg*SX$4yP6;8K{KZM2_KyHA$Gf7b+t8A51M&^2twx0~xGZ^e@pJ z*>WCKZ=Ihn|A8E_9(6fTk{t7{TQ9O15Eigns0F~qoLVnfFK?lU5;Pm4xL+cfjL5-* zuYlq#NN?FcI6JIza4P}WH-08uW}s<4WWY@FX`#gXojmASlwNfd)50=8$Jxfu6*C$O zpe)fk&?x{b;VHj|xyeseGr*zXQ=ncJCg@uMCQe@*Y5_-pjXAB}Av3&Pq+Z%Ng+-uN zm?`j12x9+KA!|Q$#ZD0FaeTql0+(K8Gx}X)6korTLtfE}IfY)hMwnV; zZIIRAl#|$ki8-iVN-~{r6`*QwrDK-RnpKu~Q*t;yfJNcn1YQvo?=(yA3EGb2ET1d7=;CbB-vvqOpM8et6(sh+l+1 znva$dwE3if;IoQ;w1v9Iyh1f{pRn(!yA}W7Z<%}HXD?38ds$~0{cH=e{XEILgI$hx zL_cP2dRyii{BH7mdR^yoK0VENJ>YHuhF)eL{B8=-@4;umPgnf#6@q@34q;&TA9JFo z#=U|lydph-pCDZszUW^u`!{oze!hG)z~6{n@xI`=BDO$y{GNck!nT0l;OnWr*xn*{ zWFFYJj*m3E^bxvwUvJ*0l1>1(Y##8plo7gx@b|X)wzHvrfu}8gz6GzQP6)Rk?+~}N zrwF^cb0dDfg1Lt;1&ODweme@Ea9?=0ymR;uwgo+f1Aaewud+@cw>$jH5@x{}O((9Ntfo7I;guzS)l+3xCTL zWOyamzsQarXUIx@f15InpXL5O6Mp>choAk6m#p*${OD2kSM_fz#oxN$_5$o*5J!(h zzhF5TUMco3x}(Q|qy_k2`+kMkq=g@`-`fwb!@D<8*7w~HS}A@(mQR?&JHhx_DB1rI z@PD84TP*r(PeIl<;?ZN^ua;l^zwkdtk2?x7KXDH4p})1i{UbPfg#R^=kNpdH{0v`C z#y8jD{Ud2%=eHv~M~^RLr9GnTU)KvVUp+f)r$YQytzpe?j?K-*88dE###)g<0Ldig?D)KFI!u zVEYRfW%tHEdi4I)B`@O}>F}=ki~H>W+RW|&~e=hp0 z?zbPrQs2iPq*B`-e3_pgzXe1C5fS@IzqDbTM?#QGO1=mv3>L^NCP+6Ej@n_b(+=F( zo8~!sT*Ouft|?0(mvne8JMu?`(FxyY-H_r(xSvRA7wpT0ICK|QPxM=!fbHPC(t^*_ z0mb`L?sj1lkk^mShaO+02n?P46 zfKr|qfmLYR%`hu9u&ztP-6k5dVxm0sk^)|XUq1ufmuUegr0=?cdLPHJ|Zj* z^T$ov)?u061-^+;wU_{VcYr&z2I5%* z{Q3dNiO{?Q@JaU;O2Mk0%6v%qME?U%oX5=~sM!hg!WOP*2fOM2)SzmIcMYZfHXFMK zn+lkS>-X)3;4yddr^k1M7=g#wXHY9OH`N#C70W+*50sk&`Ouu%?gQR`1uV~4SJ=0& z6oXi+E!R`JS;mFX7ktqgqw!d2)|rvNh8mdW8AM@MeRjC{N?vF8YK3+A53wOkKRMjnvycc9Q zjxU((_Qa);b7e2sDivo=_Ed9yORa02ZT*dd_GaQHY5iFJP(A6}&;T!n4=pdVFVP~X zh=W@-#40IfEnSKqO&7@fDn;BJT->H5pW~BXcN=bwGd7(&Z|%8`UK1}sZq^{g$8I`p zNQW+%f-7=2Y`@7TiVtQ_$jiR$o$FsZABZ6-P{oNw`shihV-g5gdhoQs$<)eB?QxwE z7s1v3YZKmMgb{mpb@m|f`sn(R!;e=3AEX|kyXUjM{Dl1Fd|tRh{gc~(JdLog}+gB$h$MFenFDv_l7V+a|y zQV@=RCI=n}sD>qPB_LMzY`nO;0e8ZD18z$8c-&e|)SN6Sra6(6j_}_;1W~jo|LfjCIz7tx2q*tx-CpauMYc^$7XodUbqCzM1Px zvpz_^TE4=+brOPDJ%M$;R6CNbuh$3(Vm+%-&E zG{?j%>U{Y%*s+dwgOvY3C^N3?10FL!gFT0QM*h=i2E|XY*54K+bPb}s_C3Gw&|zNGkfV?RmV8uS&n5x`>#Jh^iV=>=aq%;^S&ZbW0`@G7pi zNpQ&@*oy4D+Yuj>JL22>1#vL4Hgd}(0$jB;VrEbL&)-!I|2t0Eze;?NYf{aQ(J>%> zp!*MRuL(U^iW`2_0F{jYWt7c;9yX1rD?Q?FBbZ(yydAq6%nQ&?E{=~sUGrs}718LL zpcT%eJX90?M=w+laN4osX(DSN=nqa^m0Uy{d}XsuCyZMU>nP zER2<47;s8Ar2ksPF99UrWleLYS?h80^5Ft`Z`HOVmGwMpfR4QZx%(&lJlo?%pTqu& zHysQwN0m)0E2|hWp550gG00=ZCJRLp?a9&YzJ%+@#%~-sty-%zjF}-ka#V>_DNwkz zMHym7?Zf&HyVTKcZjS^5Cz2#5JoFlfsd=dgc;-TgN1x~{kTx$&3_8{%?T#P)I20?j zDsVUwqYYvyDPo~x0~zdaJ^|L@|E5U;jqsMiH%52|KDLJAz04a@<9vACg4XqQ*Xvvk zTp0-J9%LkQ!(KhfZ7x_SYv)}PDx2eQS1AnSW5v8%`Nx#jS>u#|M^)v&n}d-z-pYAT z^LY#A63NNtC7dJ+uRU2ZR-6%rnlF zqH{~wA1q=jbjwbHrWlTVT@A3_e{{tMXhn>~< zPAr1!uUF(KL3{#Z0N7jMh$9X8WY(kej1_rSq*=HnctS$mGEYhNAxzY<+HqXg^;{OCt}VPD&P`ED zTd^!6&%+$FisHcdIgiy>&sg`rI9_KJz4OU8Y6C&p53wzoZ0~7xf}V4!?NLM;Bfj7l zdxC#uV}J-<(8n!tAg`kW(f%Qk8>IZIfu@&?OHH*jB~5mNt!f%5qc{DG#EFS$u;h`@ zV`ZPT7bmEHdYM2YqrC+_ih!ESy}ErLB){H}4>g(Izb%Pw7FBUMU%V#SPfoaUN zDLV%xw6rzSR8{@p7~{%Z&L^VZYzLrf(SJ7le1}L@X+?lvP$2J%xW-y2&`<�yV0A zZa?zU?ObNNOEe30iKUn}ZMAG{nu4fp)wk=To1WHooViZ5E-XUqT2IMkvrjB3bS6D_ zsM_C7lOrFQcPMb6%5Qz2)#oVxZZkh)I-N)4YX(~c!^tF@eu{_A`mS|cN zmgrP&(;g$Uo!6?5N!ngRjf5>&@2gk(w`oB?0sL*pqUbNLfR!s2h5rLJB&0yN%#+DG z9tA9$CX6m7@pt1yTOM#(krz9@5ysL4BZc^;63lamGF_NusJQ|6lI0nK(lmFZ$LQJq z%k60)bW9suwz*H*(!%n8F?LSTnFQU!o{1*5ZQFL<*tTsunb@|IiETT1W81dvWOC;F zuKu&m&A)o}>Wk`&?p?KNbyw~EJj)Zv_0}Qk6G}|5$80QitMk#lsawx8?F|&+FARcT zPXYE<0KL!TL#5M4uj>4J%I+!tb}@V8Y>3i{qZW)Y-> z*Rj~IBI?eI@!1Pgb+ z#hVQDAYlE;BMM*0pxrG4mkXYVrF_X%)6AbR~A0Klw65u$EBSBlo#Mc>xR z@<8La5Z4Z^B z7?QGRS{RyHJ-t58?+LpoXCl)g2_i$(H63|x`&eI*kyga0cK)Hjd2@7aa=N$uAMo-e zI?$+~ZIR4R{qM67CnK*0w@HsDM7c2g@hv(AbWUjtm-P_UL#Zl%x`;T*SP)t{kHxPI zVL^~9PIlk=pU-2@ug`6lpv}iMx1%)z+#&deVo52?HR7gNNPMMl|L%@h>Q$vyWL2gy zORu?Uc6R01%U37pUw3UT?X?qoc$(p%PGkIQVAQ;dSl{t>=8^mdCazKu|SPK}jg2{t97A4n1*v zkO>E-^O#T}ojIX5d=e!(jr3A=$}j~>fKlhK2f4Jtq0yr~gK((g!J%gd?0*&5tneNK zmh3?j^EN;<*@I--1V90?Kep%inho|RM*+E-2$-vik+dHyj@^#tV#72!;MxQxwXrN8F?d{c1u zk{eY+wUlB}_Un1frCWj3#nVdeJeWSKWY&$f93op-@3O zHB$c?#XwjKn)&PSCgPP_+ud^(7N;#=?Z)S`6wPDi%seoxe4jV|*cT(@B9)o{Na~m< z*&unt_db;y_Jn}rev!vkfrjtl(O67Kyvl~@Hqk$MbnWucNOv8Jd5QDi@dSe@$TlTo_x;)qaHbUx+cjsN0)6;4|C5A zayE<5Qe`)wZo>%EB2&UyD3^|~*NLl)7zfga+ZGGWeaDmP`yK5QvJp^;Y+R12oF_FT znx%FWQ>rAwP-GT@C&I^CX>NoRVTDA z!CC?{RaJ%2IXx=f&Matg!X`C$?s}0g&9COVbvL5AdHlO|c5&4r!Fp$|Pjt;Eg0O!%uKy=~VSa z*%s;*|8*NSDfDdp$i^SgtC}0SXk>Hpu4bf%y0N$mi4#dDrDtC+dDLr4ikm<^OVOw^z}ePBe(dTYE9vU1aQHn0w$^xpbG@Z8n*{zdyQqes00~sRFcX96M@&BvD@6 z96M7gCtaZ%#ABwtFs~quDY9K=-qPAE9a{H0CYT|A@~btM_8o+23q|fQPM!_3>H*~f zRV%$G!xd|87T$9x1HAI1%Ke_&NEF<=?XN>&3DUpg)<8|VRYf=Zq1+=}Mv zx=}?kCDJ1l6v@kuw$-*w4$-8lAzKyIsu9OJp8kJF?T9F#I&iON{M$v+MB*EQa^Wr! zN7wom5M$CQ>osT$5eegwCW7zXz`k1t?8?eSQ{#kR_uC^#Yftu#=Bo@?<8qTHQkvgLZ8%I$3+UlxA3QTzXp|O65mv5 zKA!_8r6)o$veXO>h3ZGpAw%k#Tjsh-XJHQgL-gVY7^L;juQb9?*>gm+tK!b3+feOs zqejTJuPX1PK1|1&pS|zC#(baUFZK6pue7T1FjaG{Wa`nLBNs(^tYyV6_fTnCCzRN6 za6}Y2ae^vrxxAMXWS3HBDD@+j3)r5UV`-H5EP}pD7R_Aj&oNHt7F+In?ptlWuRxD_ zTYYT=6!c4MhT$d3~aUnd0GU( zTfw#;G9sRYKccsxm@LO@*J9R3qY*Y9_Wk8cX>lEZ7!VbmPeGl`)YADuWZsJ7k=j^1$z`RB`m6@478)nD-R^_|0X5i0=#lC_Hf(b+E2&<_#io;;iPMzn zs%#!$ehbGR*_reZsODbw&p{lYiBSjdJE808HPyVGs6a{LR&RZbfgvVb>NhgCu)t@| z>?LOlW;W%Y`z`ehHmU;lBBAUZz1jw449HB_0vZ@21(C>h(7IF z7cPW@uVHAyU_N$;Z9!xBT^P6ynS@Fc18o!6W=0|6J!LB`lt7X`2=%}QA*yNR+$*TEC&2+GRwhO#M{^feGaPU5pE)G-jr=Lo* ziW!Z}iLS@&Dr`6xb+z)m-7l&T?Qh#!-FH3n^+Tn0BMMn5egLlm) z!SfG=!|EgZ24lRM_2ig)zuEE*lB_6S%Lw%7&Zl8uptWN;6ba?fL@yrLQ+a&$U}+w1 zTpOGH5M%7o6$VeG=B9>MsYAmG*|Azn)a?p8??4JgTo2boi$eM))4bJB1p2>ik7DDZ zLI_Q`l0^(V|Mupq*s5bXX^JarT=n|tbKa1Qcd6@cnL)7&mjaRa503X!;`P816|w|) z&kESZRAl<0+*)aHp#%jRptOKk>}C|kr7n}4n-PhMo4ZPOs1%4F42fa#D}Cr>&_ss9 ziN8~UAhd&#S23W?2k8FuXHqq8k2z-+dNY1MUT!L$7e60|iDM9G(6`HJjx2&e#MKpF zrFNzM#80pt-@xNUaf9zr(!SvGv|wk7Nga|dNjLsGW|HJ zX@s%Lcm)0&PE2aa$eyDNU7Ax7FO~f`2RjQpk0ejINsCvFTP`C9H;3Bcklt|0vzabh z<5eSGV@~xE%mNGxOd|}pev`Wwx8eF^mW?lI?Yd%>-SlxM?ZkL<3 zC`&099Sunm&mRZ@p$zoKphLi&fy9x#{#9um58U|RqCGND=hfe0gKmR{pw_46%K{g^ z<0vTk=7}&}+91q2WHF`^vcyB#Q3FrCL{SBS-}B#= zed(@>YK0_8GdX?~B$;K^ikP(Uupl;7q7(5g(|iK-Z}Dzk#7}mkH#${@l^Vp<;qard zfEz`MKkI6REf>x@QhCy)kI8)6^+fRoeI!kY($j;pOj8p5BnX)E_$7o#2@Z@h0B#t5 zBQdfMlhM`Z#etMo;4t3z({Q=JpP|g~2To~WI6_D^U8kC^x3{UQ$?J)`9MAoPs&%D} zTpsl1(CWJ_h8~Brj@Nnf1Vg9w@mE+%gOEqTNY zGdxjETosY9ck%g~2R}=%MbC+n50$g!Trs}by!u7fUZ=cU?S zI_&N9k9+1oy}b)_2+}0vk$97ZCF~~!9hUW2P6z17JXOjs_DYUpqQ?@FwNVlebk7z~ z&KYMJrUquvned7AQOw;j_5ovj0}Y|I>$1db2eo6@5qPoBBg5g$al2KbpRZqrKi3odekWbwSFOn~de{|hdIXV?&FSG+ zJYJe|kJ1C{3RiN|>76R;@tS?ECMKL!~`o7vp?ph|(N|r^YYD|G1TBA*N zk1=!ko!+*PUU6ZprEnpn#n=|av0KP|QR(I+y?)lc4_r&W8^2Bhn#x?&qzb0UlLDC) zs7SO}15&tq7MQaN4Iso?Gw{{Ce88Z7{=2}zbaxG>ndedwr_0alyR_d#c9|;6Lw4h# z#NWjQ#FJ^M8Eh2HlAlIsz*)U2!;LXLmZ)61&Sf>uRt<>x!?B|K(dN_|$u=Cl^2d6C zqcG+?M${Y9GiY@;PmONLj#8=2r7Qe1=}u+IRAGRHwq$r zGo|@{2R(Kmbv*7NnTFCIdU~uGFhb$6IceNur?Z6su$ii<(LiC;ks-XqtmHb_58R8e zl848$MSZ!x*Gc%MAM!CR3oFq8?l<_=-TS15ftcZ9Z_Ow7)g)DHL8k&qxJUbVc{Ve zNoI{EVzo##s@v#7%T!YOE6KuHXlE^2>D8$QCXl67MgEW^YUMOWooBn02Z9r|t^!UHAqUs1c6&8of*ex=`G_f| zo3IkoBLtw-qIF%GDLQVlJR7Y<49T(J$v#?YviOH>Rn{H@hm{uUyQ;8vaPkxdTCLV& z(y(w>^p(`r{FgR&Ly!FYySNo|sd6@Op5(%}Si4?A|B-aHm0jh(a@wkgk?dz=wZPbI z|Em0@>=bvyX-Hg3pVuOa_NU*W+ZpmdIx^Ygw$%>7>`Bxe+?uR8!8grQxkd5Rsb&;! zI*`S8?%pa9%ASY>BY19R?xHDD@gDDL(8F4wzeu%Bt8b`fQwzQniUG{BTk|w^Yt`W7 z%TEwj=c6@KapKP6&)#bxew$l2tUu4vD3==yjVkCW8^)WIRafRac3Q!;Y_B24hn2obP}(wT^tQeodQ8f`d>?-0+4?lRCA53ZUDMAQDqFrcYzF=!&`F&_XW6-W zoA9Q%xvr@(bN_NlREBhcMML=N;r%+6_GRdp>|2dd7*zgk?(_Pd0}YP@T2!j9r<5u5 zQ>mBziWlpEed^e3%xuj3fvyIfqgS{EupHCj%cENx@e4?D>YUMlEDvnhA z8;L~l77TJ+B;sB;YLiZ9?ZyaU-aqZhRF9i}SnW$yeE279{ z3t3WiSVs}7^11tJX%{O~78{U+ssJrnhUc2Ew-^R7i&R7k1?#o%bOI+=vKG+`U1)ZM2_`-J(@sPQk zhUWQ2+)Y3AQb)zzVJ-&rwV4{5^EVleJ&wcbwOw#%_`M=7+gd?W45qT_%_Nf{e0{3u z8qwV$UOb5RNA|)+$0=q15QpsAi$GzKoZm+=ijeWX$G*naL|Dn$%*BOA$)s9a91&ku^z*vp zVeOlmv^%Res;gGktxc=+#j}opK@4&x5}%jAsxEp(cijRJkSPS$=AZpZaF3ji!$R`} z;Oj&Qx4ykDp~l(7$j~%@h$9_;4Eaa^$r{=?xs`wV)>TKcl3-bp+Ud}uCt`TaL3X8o z_0XWN&XL2IzyZDREGdp-ICSmP&*Wa)zFXSg4K8`#!Wx z?46qm-5!M0juE%|{$6LRI-PX6DAn3Y0rtr7R-~;fI7N~$l1Z8A+eOxlI%YdY*>`&S z#*EddPEw#_CYd#R0@_u3^cF}qRUR21ihR?~M2?NWC)0}Wln8jqq-d@hnQFb$ftvV$lN~jUC|#~@wAvOu^DPt8YKp02J8(HFFEkmc zdPzr(erRaPpSAg?%D9T7X!57Dxa)#ymJN<$>GKI|Zs=!VIc3K3r**Ns5y;fa>L=wm z1wM6TnW5ZJJMM}74?>JsVDsOdkl?@$*mlyny-K?L!SVtjNFwnoUg;l8b15(Kvt+KQ z-BB)45jmRq1S(t}@we8@!arV9wO$qjX#`bX-APS6v&l6=5+X5B{Zd z(q#?W@M+g(9JWPtOOkZv4UWxU_;lav`BfTKIz`nD&8k}JRs@IU{Xbo|&G!-Gs&Q>*OsOVq<88yV(ZPa#}>xj5yJ{Kcg) ze?z+Im{pS9mCY*(HHH%%S&wy7Y^^9UE2}JVgM=M99K~&ALS@n(B$DPdp>rz?=Q7HBnvG;HWkCF-jpUg~TYs6F`mLFwUrF3deVL`8|>0;ijH$Z8XbPKuye3aPC zwBca0%XWJXZ?$w@Wtl%Zdz+WzI<8f6pxvxQTN4jgi4^Kvo({ZE&^PymS>-5rHR-V<60#;QZ-r;a9ZD6 zKJ@PEdlIBAVb$M9$%eP7JR-r$l}RNs=WT|M{i;^0N5?r<_B#ApWac?b$`au>FHhYV zV=*#b5^$;3$W=f!skv^_Bg}^n-b?fwsTd)7e{(GYOGrE82P&1bl(dy?sI#Q&wi@p) z<6AK?l1^wFxLC3x@1pl(_XGAphji{z_5&9|Ec@BZz}7&hWvpb)0_+y}V%q9unvA-e z2RjJD%kn%GnQd!^noX%%GE6fy9W=pH4aab}BAH3I;dV?Ia(j2Ve9+PjD@r+jYrpG8 zr>8G6VO?@l1G#%aJI+8@4k$I9iC|+HXJ(lOM4AT-(j1W8UXdM;W@o3rdItGRs+%tk z4REkNH`@hVI7O#nlDrlx$#}@`$|@V}&b!V-4VHCH8jUqMRrFHnCz9*$@Ak0vM9WBd zIaoJhwv2i0SlKp6|3I~=i5i9|ACdW>Ib1+o!|M#1ESu*Ij?CFm$f5gGy3l zk?t0$)}SNGQO3fd)enXjl32DZ)Lcr!v29%KtUyKp;t7D=4_ADbYVpevmqA^Z_u7Ay z&mt*ELQu_q37sMoc^vWKM8HV+?=_pnz|>zLBoJt>`Zlc5tIJ5|AVT>&10h6|RtDk& z8h+1M+2-am{^lESSG7QwDxKkTV}UBKr-b(i$6AJ?YY%sujdN)~uk)j2A~QzL3dxwu_d@tl4I}i%+Oh|7<&`;=qHNtmJwvE(K}{ zn9C22x=ZR=fnt*sl|)t5<{+eegtNgd)oxWC{UyPr0BUyfktx+HRXkG3ih(}^_)nPc zn%yS1X#lQ59i!nz_T=E|Usnpl?-cDUzxU>!H zL7ygOv>q8Xw2ep6vdac&@kWc+&QhrkiPTr=oZFac9qz?~)xcS%7LSGL3VvmfNDKBK&skK?gATpN?ud2;BEqG^h_z zbQ63!dx#`MU09zcpEX{^nT(ivv|Pk|WNd0Qma>u(%dvQD!`+~(D^yzI3+L7lw@vC6 zfZ0k)Y8dt-6xhyHrfEKf+x2?gb~^iuM{ocR(h<9CM9M5WdF7;wYnT#)(dSxayZT)9 z#R`9R$KIpQqu1T~8Y_kRWRwd7Dhf8Lt1@^gbZPq+=Fv*75W_}Wd-ZnC$)&?c7Si%xl}jtSTdxM1T2N~?hsU$Sb7ixqFI!VV0zfeSsc_3bi-Jv zxA;lQjM2&OnN8|R_DS|Jy!SuEaqj4Y zP+pUm(d0?Hx7>WOV2|zL`a_g_)bmHg0g)0;KFi9AafuSP$iexiY*Fb*-CjRkP>hGu z`*$rp$-1GOkpQ*GV4W4nMA`y>0hwkiUDaxphYB0;VY6Cd$^BEs50b=!V-^U5Xirf8;-%QX4>ntK>-B}8{c^+P7o2WW%8v??z-FqU~h|G5#{?t6^QjZOz1mBOa~{@s}bfbTZ%?i z#vGf99ecELXoP6B=4S2Aw9Q1p&M*sms;bf7JzUSc;o+gwwVD*x7ijg5Tb^5rodoW! zuLmoiH1kBktrB)#Exxv^iZwAb&V!;*k)gjg@Pf0S@#-+NvwQ|D{{stNtJzqPal>>S zZL2BB8Z8>%n9JQfsss2;9SF<67XIW$D1U~QjX%WTh0mGuF3g-ESUhbm z5acYoYp<8PSqYh5_Q`)W!W_jAO`-1j7XFF}XL_RY*najM%#l2vnU6Y|b;agjU`ef! zndu&G*l#IM;_+3ht}wcEh_Q^GDcRRaXkU{qWr{BDaBQ>DlB0VVGFLG%PvIUffsZSv zb62v8JFVv$_Gy$Mpx10UtteMMQPF4VGOMuwXR|~B2{qeXNQiGpTCCqIS8*2(Yx^-j zUi8P~sLs?cga*?kCL@TUi{rY`>W7IqXzKJo!+KG@aK&_Bap5R(xuj`U%k*u(pkVDvxy>@+PGz4jbd|abWefE)4JSC0m(HAvVjU32Hnv5a=Z}ti z6URD+siI0avx{XkGLUpVmkN`=-hbzoX&tXB8-`0G3uofcEyKC#4yH19gL@2-#AWom zcu8cLJul_eS6}v^9AS&eHN|47e1aS{j!DuIjpJXgi1i}(ywCA4bDal3%lfPX6s%N?m0@kw;zxXg`5BOwPhjdBG+E%S3<1EgH^ z%%LuZ44peacvw&B+?t@7fOnTB!l2) z2jNKr`Je(FE>nLmo@=!ouQnPR(&SdXzV-=pc@wsJOuwEh<=q7u?$QL_M-WC#)4zOC4-FLHxhi63GrEaUPN2YqJwUL(Uu9Yp78cQq0#>67M^olmmuCM8K1mr>AAwT^y zYP&OK`V`Q_C@f)|8H!#IHx}K7*-c#!U!d4Rp3cT-YgL1p4P`me#_x}a<*a_rk3%D+ zm80zUR>_zCHI0TVrYkID2B%C5b_LToPp~p3`SF+iXsTC`nUenKSz+lB7s~~^K>68|sAj%=R*vN(7 zSY>B*n$ze+Lvw*%Uz>{x1$MF#ldhO5$!(x*Por?BAkh+3&R#Tx?(*4vr#ecU8hdgM zCX7Np>c>y;YLz6~E%s&ITh!E}IOg7vbt>Z&B^4}pAXXt0QMvP-k?XXb-O?n20O2ND za=C|UJ@tC-wS;@=(YuQPdhrwC<44nnOD%*xoG&UT<|n6LG!Hr_IxoIDj`e}<9YL}l z&Q34~Q`V}TbZg_r($%&?2UP)?b%9Ei5{Y}|zWw_P1ghP#6?;duSZotyYzFqHi~Mp=WJB;FK@@hzjCxKpxx=GBXVO3 zMlYOtE}q$X6zJK1k{lCdMxxXb&p5DyepY8Lh+B|8>~bg&Ybu4t)2a;c_=EZV$N;ct zC*{T~PIcJ=S0x}q|H=TCH5!YqXAj(b!+O+b&OaPO4#PR|plU&A_O(gc_~wz$V(W}Z z(F}=a0(KN*he}IhTLd&6MO{}Bu^y*C9JHeD(O2;b0s zB8Sud<{I*+@G<99oosFn3=mz$X_kns_MmA@*{bN#->^P&eeUqu%KCbNXi2pijdWp7 z7ljxjb!Co+wTl)&^dY3TwkG?urbNf%GOZ^GBU6nH*iyxGbVn{eR5(J8M=kASE=A%U z*2Ua$C&(()(_FtS1 z1!rb@+oSo4-E&#C$9{F|)FhkEs&-D+if5i|Z6nGy^U3$Xq5}g726x*p%l=B7+A+F; zD9n-;i$=r71F{L}WxtU$ct2~T?Oh-XD@#)pYgE(DdjE6%Dv8^&RXtBOExCc4?O+wz z(35f6;jm4lVVzErD)7nb!f(QGB9E@>h`8~>wurpOQ3tF-nl$P?GMzY>X=E($*bL>G z-OvPC#<%3>!p{Yyq|L=^Kc~SmH!Mx6dAEj`=$2g439-;v^laTx?#O^k#7mQbv{>=L z_0vd0k#%IobW8A(vuMNg=v(z!0%bWa4@TWf9P@CnjtdkzvQ~9HJ*y}u1_b|E&d17S z$GLqn>4m6NW*Xn+D2fn1GA|lVQ~eo(Ii# z9GA?_{8o_Ck`z(lca_mlT;eb^8xp96iw*fLjz)=392);iso54p;!- z(D@^V-`l9g{17T$p-ISZIx7HKvMR=Zi_HE8tQp)U zj><(c5c8U_Y0Zvi#mmZ1po5VyQ{sV8X&18Q$Jd_9AYiT~|LZ0RaIupMFRoV^*pJMNLs`@j~Sc1<6r&fRuO((jB+=L)#{4;K*%$eAwpDG3{nkygf4x z9iV!sLI^!(#;S#|;TD1HPd|$icLUn{7+2QmUE-)XzoOuwEBYL)JJF62i?*y6O|tFM zYqc6qSNL-1_yN-T&h&ay$HO@Mt3&`KKFRNWAt#L1*8e<5<<@!KIi4GwlZCKbDJYhV z*RKkRlC-ek zno{2l($KLAaD>!wXbaIG#BngcLysu|js#<|Tl%RJ?YR*xUdN`CO0B@_)MuI>+x{VVMp5i>Il4rh+)hy(PG)tLmfKq3kF+L)b12jeDWf z%cE17pgvGn?$QMK5sEX*`!lD*=oO^ky{p$@oqA@ErPX-4^QO18|HaPS>D1@p(BC+z zI{<7==P=B7+&=+>^I}!rs0XsfH|VeF^i6G66v*aLS=LAN=kfk0-28X#d<|P$%sKQ< zr#geSo;158qhoeTlUiLuG+yzd3zz(ByyaE9-fA$H~ zlk_Ar&4IPUa80vA{Dyg2ZTg4${fb?d3HF3M{Dz=Q;viD-Wsz-Zb_1}^(I4$mE(vx{ zG?zF9y}zFE>Ndr0w{b_-oJO_9_(VcSeMw>RQqrD|n7ki!mI)XT5^YIm%qP$fKJ@v# zfnA|I64=YImskyGir4}xeTT-H{L!S6$|f$mdw6caCs+n!ASg@UGZ_`o2XegF*9Sv6%?L-I4Vk^GEe^% zb`c~-diE;tleD;m{lL4dtygDK+P+j3sevvS+-b;5viehZaG7q&rB+r2onQg~3zv)~ z{^U1IAFTlq*0VA*NHmD@?~pb^w0x}7Lu~dja;9mhCU+rrP%Hz~x_JC&F+CIa z+q2g-gxgdP{#Qdcg4aK_ z0_+!^ntDw*%%0gt(L<3u9hhzivmlJg@C7cAORih&7xINV& z_I2g9=Z;xUrH)_aJ3*6ILdVay=Eyc4)8=i53&7Al7T7$)S6C7nI@&M~EO*WuuhzoA1TF}a4>#9R8&V>u} zmd3~5Vj*i-*M@w5g}Bx!sP#7og@m}0g|6hxNu`qS{3K;YXuyBPMg*m$wO!<(p;+^3 zm=GdP84qV20ML@CNlQnoevPEYYqHq&p9_D}xzGwihX^Q2ALeVTKItGdH>*lpUapWV7KwP$+A z^rF?&*4y@%%|Q$E-w>7ZoebA&SLW7*0~<#`wys>IQR%^)uMJBtq+mw?t;3Hx!V#)d zzDqa4km>|sYP*vabcw)B9L}(N@H#X$j8C0gXQ7M0WwPT`XDDfmSp*Z=%Bw`v8n~Cd zBlI6Hp;gH1QisnVhqMeoFBFm#{~dd-a_?XyE+SsP>3>6Qf&cJ^JIt;gYI*@>H_Suh zcW!~Sr}0I`YV#{eIR6~^dK4syvDmYq+z%@Im)l-WW;j-c?6-K1G=9JnJx-uyc7jzml`oE(T5)`YCS{CX`(IQURJJ|(K6lIu;?r^Dli ze2G%p$&VZAOA1@`o7RBLe2v@*U+^X#Iv%{_2h7EwHOdAl-q1z3OE5Ue{rnp=l098z zCGK^Zc12$g;Q~g8kIv|qaA-(XoJviSfDXmCQ_g3%>-`g@1YDcf*kgSBb~p=lFK!A? zp_8!+zymu>6q`!Did@r^N35A-fL zpLB~w1i$`S{KLO9gilAjq_a{Ld*JcUbSFeMs@lfcE((IlIlp!;U1Q+Cv_{ ziFIwuq=i-<6mv_c&ye>}MbKz8#Z~5pi2p7FlU>#j8X!ORE{Y`1XNg5}RzjeVHUo%K}kE8Kdkf-XcVVI8zWk#XpHsC~| zm3mBPjv>9cO*wTmrEZhus(`)-hNfYEN?$JGEqeo(& zw^h7x7O2F@@kNERrA{o^N6*st_!iz6Uagb z+f`3CMbUVzeh?(s`}g+xitT-S3V-G?m19z~ChgH)&i;g#v8=ZMe)y5jYLbO7z9*7v zY6{fKCbQ^(uY2Ka)L;@b92U3~pwZyqVdAPC9Ox~d{KZ9=kD!C5F_wdT(Y z?s$*Bsgjp^W9~uPDYKvCIBIkpZ1s*4Z(`c%bOrqv-Njs8`u!qCz$WPvaUOpb$1`#nk zn0+Mch--%K8ysZV5A^?{4O##H(1wI8j9mYVWT=}k^N&CYJAC6AiQ6dxUQ}@Plc;%C z68Bl$*!M?fI{iGPq69+MSKyZGJaNeB$?J#FWs%kSaAseUyFdjx$W6==e#zZ2+AWMu zjA_R1dYJ>oXGMWCZN$D?E(mPQmAD#eHU+_*8?@hXWe`5YEijRzwjam)sM2`}U}rz= zJNSeBHaUfCWmdd2ZMxeH^cKwY0x;r@a_qYJY3ef#-zGR+hRtS`u2b6Q{tR6uaX{^P zhpRr;ckcg^sFJ4V>x$pV!NemgrbSO0E8!&XLd;EUYgP35yT?6jt9Iw$jdrj}CjKTO zn{+M}mW5d+ft@s-4cr;}`jja^NYY7jub!;E=-lZ&fDSDqiFfCq9_2-=yrURzVimtz zjpN2R%dus3r?_!=!1&Y7CCX||fG;d3?*9g#?f(hBil>7qA%nb;m9mR1A%l#mowEG4H(8W~L)Y#tS-<@)V%>Qk2a&{qPX8hlH&@D}A`wakM&x`ux z99Q_}3w;#fZvsc7CONnXOZZUuXyCXFsS=_k>ED-}0+VZ{0uyalqI^W!h>!G)oROEQ z@%bO1mO@VP@J$dIKiqLqLHk)LanT`?f0*9N2g+emf~OuJ!^zEqkpco6`r3hb@NGq! z$t2YvQ@?SBg`5$E3!#4sYod}0VMfhBiw?LJYULZ*3R-!!lQXkD1!82 zqk{n{p@o3b2|W-xsCYz*0YlZ$lo|pADWQaBp;svh9i)T~(wm51&bjx#ch2|j7~l8a zz5f{h_{YDq$JlGFtTpFcYp=cMoNLWF_lzdA5whx3cspkwsni>MfjO9yzmj86w4r^$ zVprrx7_4Vt243AM1@}}Q9Plgpj4;eE52@2=9Dz0lTmHfWszZT#dtJ}>Gly5xqqivN zDcjGJ?Lt*&eqLD&J-fNE64QD(>z$s`3}0NtvnJvi_ zj?Nk#yK29Q{Y^kS)?lEyRV;Io)JCs_2)_|iw4D?2?3;>iwe6X;%-n%!8Dp%0e#doe z=%e3I%;-0#&oHN#yh60mJ?LorCim^ZNZSqIn-<#YO_fZCwrfF2$}(Fb=}xKG`tYb( zTcu8~$0|kThnY{eYA8L+Hy7Ia9U8Em6ypb8LEds9*oXZZn`R^GKQA#bIUM)dwKTpz z3*N?A-~QOQnRAS(fB#O5t=vuj(=(4ns)mU#i`M5+XSdvNcpg(#A@-a7dM}3 zH8-rjMUtbHA4IvF%rFnQ(RHLsA#M9-G{^VqLV4S(dvmAg@L!4qZW|B{0#gAD*oNA3ZxIViTK0v#vAbOB44< zCd;);oS&gn-@?wa?XerU1D9^wy?9pap2MAFH;>Jy(~oVP4!?tIoRRWX)u<~F;rZl` z(N1h&O4p_~h?o9>1W>@uLGz7OWu|Ya!8JkB)0ozZ&mgNg^4DGF`Q7kKi3Uf=>iric zoDw`Z0?j4c*!8T*FiV(I8KW@Qclg*|X-49uvNz22aUYuWl~`z66Hfo)vef9}Shtia zl)Th(MEbB%fqDj()z#0Rd$oIScY0CgFms?FR8}dp-ZjDyO@5<(WA2=w)nE>@)9}?2 zVIqs@q4C0ib&HLVNpUspi{&H`y?)q`M$+MOCJDtjxgGK>OlUTsM2B;w-sgO8OT5U6zl5;^d7li9AAcM zKV|^78eZ?@NrdVZ1Jt>V^JfpwtkTfX^QBgev}HmM=i;qsD=%f3qX zIr47iSI+!en5kQnjQPe;!v8ay$JrW#&slq3+X&nlT8zX#3KoIuy=WU z-!p&*O3mNok&}|N_e2k!0}I|m;g6%`&yjR0RQV_zzF z`6H)wi|3a7AJuN(rg9xBia&}qslkk8ewP=hcr>M?cw~NOU%#hicy9e2s-_~3%-<;a zz2!rMr=78bFO?R_hw7Rg%6Ju71|#R^C1=3?}7<+lss+W41W%&ec#2`=b-~a&C}h> z)8qHPEAU9FdAfNbjJ+P)Q8}o(gP)6?gOL`MmOqM35RW~496b^4RJ8xWbJf*M{=spz z9bBB8sZivksnj5CegDA!M{oW?%s)CHY35>2<=*m&GXKHJWp3S+{<8%2YxvhUz-3)c z9ZdiY4FEtx{Q-WB0n`9zPSc&CJALL1-5GlNGiMntGccSx$H02=()r8mtXHnEv$C;q za0^`L;N;_CW8)Fy;S&@R5f!<5UHqoF@J#_>5#ir~(9qMq(NcTEM9+LfiiP!_;aMOXJFm3qV{cz{Vp*T8ocw(g2;y0M!uwJ_ z{#!<7$nsBW#saqC>KmI>aIAku^EY(Amoohc`QJ-N0q1F{gfY=F0qy{P%Kd}1|L3nX z6rj3M4U`py$oJJ)rN{n`kv<0Z*sVLNUzXqB@jLF>DgbavAurb2^TOgfq<<+`WbrR# zUEyjkUXwXP_6>ob2osWSqVw_7pt`Y+Z0TN#C#7t3P7+{HEebDAG}JaSlm^XYVJ zUZ|ZzMh^`RcFYIh`}u%l9#xwn-81P{H69k8O~PiFw>BrChCzHlrI7lEd$SXS*Ivt* zSRpaD7j==v+b-D5qf=eEZSFuLX$1+BxTK{DI|mKO&lU<2DRA0a++F>l*2xkV%3wLn zqkh`-*oLcB(ywDEXlB1iTf>rJIp{b26V}MJQ2KQ6(SUBOWekensxWsUiACf3=aBnL_O)i#qGMo}s`k?j zKvwCz%*q{z#&Yya<frsg1U+AOu<6Z>$asK z^88WxAuk}GMEz~xsjp#Wih@PH-?bOX_9p6D5=8%1HwP=3+zPMa3Ko-g-abz>q+o&S z({V){6Y)jIFXBv~oi#;kYj=^}Am%DXBUxK4nT>xRLzr_7=^AwhB8MZ##H=*38lkv# z;Cg(l8NmTnw#E@N^?v1xdS0Ek^$$d?YK#S|3zvJ6ArL!+AH-#r5&#&3pqhul{Q>P4cU%NKP=QW4bjNj_ikR?y> zjThq}7BEi5BxCJ#CBfH>y&_qKJ#LA+3iXp&3Hn+->N_3-u*7^9>~@sG^Hryz<=>t> zT|#%t``$mE?qaLlaxzPFFPOIYiRiGo=@|Q6<%G#kOm4D1y*pR8xhaJq3u#&r#&5oZIP`< zvvkoRTOM@azR+>wd&z571=E66>xRDdXCt7nlzTqSo2l;Pvmg*;QtW$_ReIG}%~v8p zA}yqf8ePkQ)0)gj%v!_dgQAi03IWBo@tv1qAhZ*0lh&T{?=H6RCC`z?$eCBW>>Z%E zwBe;?ySOiK-0s>`8T;)%*|PMOo1MFkkH*Vy;qranQ9_v1o1J*`!8pIwT%FLig)Y3m z=t~nTfx-y0PX;<_N-cAyHHKW+m|P9o-NP3MA*2Y0A_AmEw=QjMAFnis>SUMeTk|0D z;(wsiTwjjJX}z+L^qI`pO){TD6(ucBT?)3gpMuctB$kz3A#EvgG%e%u#U(bZu*>xh z1+hVq3Dyhv6c__z%!3BJmS=9g^tTljUWA@FYYaPVXRK-~{gvDV{KItBO6)BJCcw$G zhLe9Cy}3h9GGTmvF?3W<%%d?&!1Gp3eXk^VXVHDox_EH6)Uex(Q`FwtRI`v|l0Rni zy7ObEaK!;ZUdZRiM9s>2T@&bKvNCtrJ$R@)vL}Ius|RaLcI&L6*1Vzp48Cx29tCNbH^49lwp`Z3yJTepd?JhEmj3@;c--pW)?E6CXvExE4<`y#4v zP;TI6`6KRyOzOf`{`TIKmpJ~`xaXUn7v$8OlDM_*QZP7v57H``&?pZPGAGP%u0_W; z(DTKc%EC%+RYVv3)JL{_0vo(A8HMyF)5Rs3I{D!C)^QrD`bh=_pG1!}re649Ul4x* zTp>(=il_fA*7={6FGU%`IQt57O-$5Q$<;%~V+Al{3tlZoWR>Kbv(#_I5Zgm)tU(SI zg|0Frp9CDQ#MjxQEc3EQ@uhf*86O&`GI6=WF4*4Ax_Qhh65Ghs)**GI5<6ajT%C{d z|H|KY*n zBb@pZ6LJMc;N#i7P`83Q+g)E1otimy&Ai=(+g_Y1m11xPj<{V8T;Y%QE2nT>)w*MZ z!=YeqGE~*BkCX4ct)#S2hsSh?c=iX=92UtYz-r%K(&0o+}zmHaWIyY9(};6q#2u-SDSL}@nZUx7tJdDvwSuD z&~iEnT*)hG&@xM%=}?x?A&t?J`>ufmBWqzB9UDf$7ZVUX_-#4#amCndcHS?5PWsc) z48d-e>>zhMSVaBamym3xT8PPvTSX`L1AIb8aau3D1`?byd@3i5J<-ZrIftD%Y+r&y zt=ZWj4&0}j3_d7Zk*(zV+;OHwJA-Se<;gL%IzYaUo<~?!&`zZQE^o|;`mLOY&4^MQ zM(Vj&&w^{S$4=7rJ&HfQ?DFu?xRjwv`@FrsoYhAxa zkK9zL+{KJ~2C}$UbiI4aZzBLN?OiMwYy<$OfTg7f{J=GY7n_*PLxg;>AX>r%^Ict}`6jq67blMvm~ z$^7(kV1YbXCyfFYaeWH1?9yx;vkw2k_;yZv=%8R>bno{1m2|gba8^B`Ys(x) zFt}se0=OSg8vp50(~9bltXr4qCzjCdiA8ml07<9h&KGiu!=>$p507%;d4%3ve)%-` zR6w;MY3TEajqXW6UwefHIkuDI6#iTf1-g+ z(JJWOFMz<1IxZ_p>*vpMbXMC4V?mE|GE8(R zCYt4;LC*#OeYN9D?lMuvP+l$O`l5LVHNf4O4V7B_XtXOkkp`N3NKiF$hpq znEvZm=`%7i_B5yqR{@Xye%=JoPH#zJ7J{J-V=$V%e@gU!DKtI7YN$`UcSC$54^H~o z?m>gzW8$V^eX(Fd4+^L%()Co=t!hQtZSd?$aXnCERzst}$pDdgwRSbU`)G7(H#u~U zW8l*dJZ+a&AA}>wFUbAO+mL)-(Kz(|6zo6h&W_Nf?h+bT#D28JRZP-t z_@x~EoE`N%b)c;GGi*a(vQ2ENxcC>q!f8_VwM8QodQ#R~7x@GH&4%9PNy((@{}1u+ zWB%_NN6Q+zw=7|`e~fcMsgQ(svuNj#ylPogjKMWxRTF$)Eve=FC58_pyg7tnm$%4j z#Yg!*5Wk@n=lae&FRCl9#Bzgmz|3-51OfD&pCqMco0v_()+m!QOg%ggi||1heJ;ZZ zwM42_7bc8t#j+hD>jkx$D>GeGTsada0{irzTyU%$a(XKOIqYkNU3l=iDM3{J_)e6` zqIU@y6U$-R>}=i_hYwdYyT$e#S2dcCO@}jKn}hAfoojyq{B48L_Iq_< zW0WD0!^bA`>YirGa|Vj)6a`iMr;p2liv^sN6ETU9qkaAT(B9Oo)}YRfwvxDsg5&o5 zAEbehZ*n56#xdTI&ucuTX$ze*xI10$g~vR{H5q#xdl| z>(WxZb#EMCU zmkFu9q8H{1|1r71HUxwG26HzBJ%3+2I)_F@o$xA}*Y-zlxO; z%{y}UC8}+E(7&;9J@|6Zi>P~0EUzA~eZ$OGT|YfB-n*l0DaFC`7<3)I*jhTEKaWS_ z?Gk&ja4G z*SKBEWHW#QMvCiB?jWG@GY>}3|Zk2xd>0<@Rh1&34e@VJB^5Fe){^)eeG#pq=O2cbc#d6}7kq)QlDDKGj zo2tT%CZg-li0~=Xq%Etf97W}wwCEUo?ZRp>CC#mT6mFkeIbdpL(n3VE>MxYuIl%;b zX^NV9IrbkK@-5{1SpD>y{90J(PO?(W!_``>dt0q}hRx-Npfq45@0Fe%-C?b({AnLv zUrLlo*tQ~R7>wtrp)6pUs*l5yM{y~j1ad|?bOKNA)3N3>yWpp)V{E{1znvp?&OjO` z*wGqQTR77oX`uAv`2{_G$K!QV{Ks#OrS@#KXJp~ghZc0Oj^ZelwV_*q2{=w2Sc2D- z_`CNt);LeO(JDC^%a^>0I7+{-OL=RhX)oIRF=3~`L5-_BFmc&3z~8Uh{_<2gXIhe; z{zU78`Mfs;BzBj@Q^UBDoE(!a9QFoeZAKe)vXCQt6MQ}D{HD(JFvoh}?S!oeo1h2f zstpfGUj%#;6OwL-S?7skU z`pd|J(wM0cU`F~YWXHgR#(CZa%2*(BewnV*wNqTaBO0d+v$My(sO#y^L(AhAGT6C0 z7%YdZUE>Q({Xtx1eCVQW!LJ1ht_Cr!;E-xNbRf5n3$tQDLfAr5@LJfd>~H~cB5u3WLpjdii!29d94{BVJbHekbB%eGN{ zMc8O^%_yWlh(VsCVnlcayX4~v@3*#528t&2_$%-4AdR5Q5g)}W0w}7JXPHEE@Zmy@ zuIuRD{%4_<5`+G_CYboFtAKV-BwW$3OU6<_-d*=L=SF@FAY}ya=W;?Lcxs5 zJ@G@^0mkzi?L* z6F>%ny~RFE{t4w)rU{wS5nqm_gQ}wa zA^Fx&woE-T5~Keq`0_f3{eb;yVt7)=i~>o-6hM~u>Y#z%;iWzWK|m(;UN9(JtM&7{E1(H-vS8Bb4dglYV_X50TNUD z7Tc9mQh}evQe02VMdM(7UrH`uDPU2-*fWZsxoq3L-0e-G4~J{*(sr$Fa!Q|a>8jWc z=>%ieE=UXb;&4myLnS_TH_9GPt@XUG`J7-}(?)?G8T!3^Kx5&3#*J-4P0yvh56aA& z1JaTOT9_1w z4&E1hcM&Kk<&oP$Za*)aXwBT*zUivfTm`IpXuDdf#w@_Xw}GzRM*e6A(7teKUL%h? zf>@>C5pp1=$Oc>1p0hJLmgTq6y-xD-5oO9yaUS{?+%hJ<)JVIfS6Q7=uEt%F5G(IW z$R~(hyYLO)0=I(A1(``pWVOzS#h#%Zy(z2Z2l&*z3msiTs89v?kjB87=EG+(%1lhk*CvuI9$P(Ni zp33_u!PaG(HReLB9XyTXVt2k?$vB^G>_sc;$@!`>e^ysS2xgu&gEJQK@By) z1uM+Y0Ky%wxD9n>j3MmXT8!#TOjjCW6{vo^%z&vHu|zOIBP{C{&BG@2MW$-nT}@By!+3d z=l{Lz-=R0sGLGn$KfM#ab>~OmNuNuJZPB+&-|FH{CZAN${jtgRza#yh<;|{P@3bR3 z_hx@+n^U}(pC6}KHG15sc&hLA~fPCEQ`#R_0Uw~6%C!W_w)B^q)mh+N-BF@rv zxKM%iK$8C#pel2X2L6`@8n+&NPUXm$IE}IpEK2{7ZJBB^mHCeOgZg5*u2Nci^|I`e z=?9`mG0f$*v^ljUttq}CIbPeYAVXp5R?U@I1aD2~*)w1H4wKD1nu`0#6KQ)N_qTXU z>n8M*l9%)XD?h>1cy&&9YsrQj*=TcqjI-KWfmI1)m6BA~@P$$RO=%4jOH%(X|GB~Q zPk)C$V{w($BVLQod~g(pM_-N{QFv)r@4!5~HgMarx~3+Hv(gv5hq;FteS~MDf$Yxp5C7=vUv9+TGyk56I()qQd7aO^RCCTm-bP|`0-3(_ zZ9qwL3AUuRBf2LVd+O8q}P8oQ!;36exd_DQn~Z-dAmrX*2{y-46|e9%sa5Ktj)ENs33_(!$7-30r|xZ z>Ri6WpZD|H^>24UasM(Mw5GlOxwTLzOt(bWKac`14SOM~EVtbNGO(_N!fF~gSzs!r zB?xGX@_3~i#6_5+z`_fX{~WEX3{_IO*=iAI$GW-o3-AW3j&ay7q-w_c4kIQ3DO_)@ z4`3>>S@FWNt`I+pb$Sfn%QT%7{%Qj?EW|G|6|$@;QhdJp=pCmlM5)~X z$b~TzQujN$QWOpkQ>sia^06?xXHtxi&)bQ7>M?ixxG=YY1l=4{_7KnU%R_g}XNUZh zoq9iEE%Gq`5-dS16z$KJlJ(sZWaTOP{*ypkvDvUSVM{q)J^?N_#FM_T z$xG(?qsO^+eM4|{QQZ}*XMScY^n5oPgF)Q~zgo2=fjwhR5adV6r+9+OYpm)m$b1rrDNJ-F~SJrJYEa(v&lK@d3B64ASPy< z&j(^bi;2^i(g^Y_7;LL9wcwt~8j7~eCK~AVBhfu}(1Fl_8Bu7ev~wBpyyN!ia&Z~` z*o*?>DFS!Wx;9Gmm9Gtw5d^+kNS{A z!h~xLdnSn%%Ca_~Wv?SnV0dLQUZUm08(b`u7SlIWR{Ok0LqJB_N6Hcx-K^1H_r|fZ zP59<3ecb>iPFk{tzKv3;-03*RnfjKsB!Rp`r`Ts@r6teSe9}th*QRDveZoW`qu(W~ z>?)Uo4sAVU@EH&k=hCvb&))46$xnsvQHBH-HBM5&{M-l(GS)wl1~Zl^!C;0|dV{Z^ zK{ca=12~%Cjs2TWmsjP_x{5#zK#)gCuf|eIN=qS>;4dnE(Vtp$iBNlY`J}AIpQ?#;s~d1VM9<0HSF(s=d$f@}>Tl z^85u`Dq*vR^~!vx{Xdp$I@{j2T`rKa482KQir2^{1F;r(CYD)WfYGk zN+yCPNTKVEJNLZ0C{iISk_d~H^-BfmI8vUIyhK8exz*_clvm(t+xKnO^IQH@*1d)M z1z3%oKD=;|C1vizlsn18w~SzievQjoQsprp_CP|>osgN~LM*e~S_w8TiM2?oL2zLR z9voBbYqF)KqY}B$AYkHS*by1EnQ{h{HYObb`Q(w0i}`H3<`9>p^!}dL_ekbhx!MV0 z1_@>X!a12wurlo7ilXZwfDDI2&$1IntHKwUQ)M(<|?Wu-Vatz zUFsn#l^kuFR< zsN!Y3<`H1LMF{cza5Vrq4Ylg9gtN68vR;5BgpAqvY%#yO#alICHZWitKyn3LKPiT|ilYchAdg>SM}Z zeuKr#i5i7~(pC50DbBEL6!h=u|_&Osp#%o$3i(*SyH5o6F5^W)K(W_j=je zsIX)E758#<4CvY#)A=z;=sKN8>*A`J_R6~q_D0_xP5GoB);an75oNo{-+cdcgpXx)orU%l>b0VE)?r0ix&- zjrqLsLSl3aAD^v1A}ils=ndrh>lyMeGI!$&#UKew@Ac1k{_kaf(ppP9{!^U=r2R?K z>nlB>t2&;%Qx>uD&%j&}?E#!Hw%=J>_@RM?gwPzb$3GeC-Fr&-b7hyIu>#jAT3nV6 z8~t{9n1fs&6B84QcEsIlI;VByqtT`OZ>foTJbw79c+?`aO3-D2npyA<;H=}!mKr~QU%aMAqlvMXYHNA1kI7;XNbKvHDk&)BE5X=AeWQMaHvj8F;mnkFyr$_zV5 z_Cm+o*O1S#Q5m}h16t}d*{b;1_aa8-mfrCR*4}fd| z*2N`j<6HiVSL7s)CUB-$<=2)uAK@j;+-d=f;gixzz!e|EHPae75?J&qz-;52o?DM< zaJk9tjEpcRh;h00$Tgxm4ywl1pp zwZu7F0+|u%Myvt9FLSTx- zs7rAfHaA|tPVD#XmVUTxB*&MA9UQAI$=S?%mzx_Bm;Ve+Gzh__ccW0x77R|CteW0! zvV%Z;Gm@m+?CeNuwtOkX%wSI1Yn&&vfyo=$&B z|2s4n8psCyhPBQwfajf2@%x3pYgh^YZ`82x)g?QO<>I=|tF?jQ&S}-G{66sX|ffre&sz?GsFGxR<9A zBq~V`NeagZ<<5d>w;l~oeKWu1)31O|mk;vZ_}Sfa!RDa`31=r=k~%9_%NB9d`0Uuc zq{V|A&sz|c#WNpy0wMycx<3D~#k~E{F!At1X&bI?5v4J;EA*6R3lA6Ntzj}N6Wy0S za~D8!t$Qmd#HeHW=*?Wh1mBh6VLSw}r~krqQhExTlaH&P{c1D{zebk!4NjU|@=f)8 z!#K#XBWpSS7dQ^1JEnwZL5L3q9ZIL1RkKwWYgN~g~DFSt-T#CHfj*?hXa&V|}ej&QFX%e>}5ohT# zTaX{?Y!!=I>LM8^(d@s^<~i@|+(yg6Aq{AG_WIo;THP{=j+XIxAd6IdcY``aeVFsz z!o8q(84KdHqE{(~S)!#@sVj?0Jb#(6f6DypONapBWB0c$EV7gFV=>YcFa#=~=V)=L zq!{d6ch>3rmgOrc1nC}$FIjU$JoV!l$ zG@*e6|DVA*6YgGi%NHvw`$k~rOcFI5Pl}l~&aSCkPnYu!u!w6Id1+?l%6a1P4POAU zUW{GIwo$AYFU#PE7H{3(p+>$+(tk2v50SxI5C+F916`IZMrB<&o`yj`OIX0I%A&a} z@ZqP6Lbgm~Ll)C3pb8OBOsd5@Dp$WVSH5VcO)LDKQ4CSD`>mRQ#EitZdgq(x7UO); zYf6O>B40oYi%&s?Bbm7D-!cLklLe)0A6g99)LmrU%D6EwYTGFa=XcNDl4VYDDON;| zf8X?sl-JeO6O${TypOF1UWaf7)+hyCq@C@NIQV&u+wgG)Mj>O0tr!Btq5OU@ul58J z;uV~En;mM$ESunpc(@C42=}aedR;Lz1BwC*LIiA*?hm>hN5*hg5f<+)dA}-FX!m*_ z$B$8tHfq_21h`a5L3}X>QTwMh$4D3T{AX?;+TYxbo2^*2S<1C6!e4DEfn%qMes z{vPU!Q9J2eXn&xdV;QQb<7Jx3OQekFHm%L->V}Xh}WNJxGCyf za?>uvs9nONzZ52#SF1Q4m2Ffm&C!oTeJ*W@8HR^LWsg2RL|Yll+b4l8RgqxU_AFm1o)ZKhw1IO1qY7KO5?1umFE%=DTiI>TW@F#R!p_;Z`kj)vn zQ9ZW&ZI-(MuK@d|GhMo_-xf)tHHZi#%wxj$!AbfASwC`vllFRa@qGVf&=8Y!lwA_8Cv_GO4kHfK1SOPa>Ax zy$qcwb8|rJyPe1D?vD2&;=8h-QQ`4lfcJ8a;TDcLvAU7TSFJLZVkRznI1e?l)aK~r z2Z|9-fTwb%k!26^MH(HdJU2O7Ybt~bE8IMsU|~Ca!~EQnO$D#?1tO0N?zXfivyqw&`ubK_xJL zAn#*?e_q3auZyd*^=V>#?lT;*sqvLq5gB6Ukoy67QbAnC(z%wzKeZ zzV6FfbOFC;6XYOC7sT60`e8Vno%?K8suO|DOP2AgB{GIlJ*4^N@Ei zvwOF&Dc%zZYwvnpB1kndExkH>x;sVXOR~Isi7V;$?HmK;vJr6=?h2*h#bhUw0Z>Nl zSo(?36P=OX@%Hkt>-?au=IFa}3ia-OQvQT_)259nhAdLy6IEir8_^vpV&R@spfMH0 za(yRD1h)Cp&hwl{aaavmFgq+!AnlG@QImL-60Y%+=Myba=X00;#9#lMr1_KK)(yL) zKbe&&{)dLMnQeE>=y_Q8#NooXKezoynx2n4x6-KDB>4rrbX&2Xx?ooyoQ`1p93vZ= z9I@4znzmrzmF#?2^&J>Qqx6qP)GN()+a&S{C@>{&ckl>6^BbHOm&Gf(Ka|!+yMV46 zA1~-$8hW@VyOdAh%H(i60zAf|z_Ls%Gh7AT-8u6+i*2+NbCSO_L4K)k2XLid$X=># z;(O8`xkF7^*}N>*Dm6w{_)btubC4e*-{khk>nf=Ed5XTr*wDjbIG)60Hh23eb)EZj zWjevcXP|MiquKUkr*9pQe-p!534Cih;+Pko)oV>jzM9%sN=2O%X6(;1O~?It7lNoF zwQI@hZtLVzMYg&71HNuP)P*oe`v^qo{oZN%bD!9Ox+9rX!6VmcL9w$~sKxl0XSz7y zX83}IpSx~tr;vu`gL&R8?FRch`Z(UKtUY6+5%-)*&3VK7ZPay%Bo6BGB@D~;_Cu#i z^$)jB{et_0{ZiZ|a`F5!g8!}{m zv0iqV)bau6Dz+~qySQg6m^O7!VtDv+q0PmjiZ^qW7Vj-w_>*;z-6Y@&q5m*emHv!W zknY}KwyRCsx??Fx!ZhnR+M<^8l>EJymW$H%^#kg~h1x3V?5X*ZZO9TiBDDE*B69QK zwWoeCtchgi6{eWX*E$aY^+Xf4!2|u)=)34-?ki`|40ei(fGa<|CLU> z=}5W7HKBjjw*t}X{Gy#=yzh!B_*(70)2?+7{9y8H%kaQH?~P@2X^B}jN38X76v^Pj zpQhShl63#Mlk+0>Pm0p%`PKursqNqGC|mvyc9cw3w8DGFdWe*Sgmw^r-dgb%KFYAE zd9{Mg96sXiV>Pni+mBe8!tK&l{sr~Fl=%~>|Kr^#8k%XKY!4=UCk2V9AG!)-m;WLE zHIr#vULe2SWhFNkE-M1Jd9W5pDtyO(_g|v+&yD{bBfhG9_4(C&eDvEfU0ljwEd{aq z`str>{A=ld1BL$@0e@}%@7Q|!E5v^>_wSf{{~`&0Y5wovTYmxgFJ}HN^M|&#C9Xxg zW^6{(GbPkF=nsl?hd^joxu&d(PaZ6h^JJo^ zZY82SBjs8!uz3?wX0t8kWQg}!naufTzneMRsc9M{tCmdhVb**XYiX6ZAH0D1({gJW z^PESXc>p*g2PcnW|TE4^8 zmz3W*ytZ(heChwY#rbL5(r0bN3OiZ^Nug&UlS3n!$8kOeg_ANwVn>7Yryw99lD1>1QlcY`)lrW)kH3(FZdiUYy33yIyhFfeNx$wKBe|_+Rjvhm#Ax5 z@SPd_U`?@YX`t$>74!I_3rCBonb8+PG?4#oG3;bl!4JM~{7uSK9zH)%(_W&>o9OVI z%`Y%+T>4csvLL81`AS;Dwl0=_$&oZbyu@AJ^x+p^k8dDlY1PZVd-Z(n7kAx*9NgQs z7Jd&c!4ngsx8)C~JnHC?vdQmlAhQM>y!9+^nMTzWLv9o7&bPap_(P%MHFPS)=xc*M zb!L(+jrDgwd>G5MAGanNfsOY2%idbOf-J_Abw=#oQz>t~ppLlBy`9Y!Q|y07+|dHK z-rdAFYFDuy56Shh;e+@!P5Fw)(vFE=YtF4MzHeeBnt&sWo8CPw6aY_9as$5Mjvmy{MQ(LCb!uNtn^X2!slvZYA- z1ZW7(0J!;)TxiuOviaPeUH4UVEmKF%)~<%BoRczqPJM+Mh6rB=&hrSVbil;5Rpq3% z7&L0cQ7Rhr6N=SE7BL z992=*>5KV}ncrL4q6xT%tm=t12NgI_me+MG*FuO!wM1_@1_^%YmiRv6v%$sP%ohDv z$lBr!xir%_&Y8JN%L~%smfCfuo;lI=CkaW(8?)DAVk&k-)SGfL1naw2auFpD?`RA3 zMA@O1Ag3z92+7Edm9W_SBGN-OkX^F#@IKPjYEHu_eg=ro=uEJyJQCz*l`3*3XYGtC z%D8XZI+d`N;sU1fh{;+KDLrv_#r1mbxt2FANdP}+bRHJ+1va2EO&KW{j$RP_MncTF zM+~}0>XUKj=PBNuYi>0oT)zMt{63@*xs|Uk20k4@#T3$5>|#zd!CQF*{5$=Q>^S)j zqARkp*$t5knMCG}1rvbsWkVt!{JbS>o0-DD0BRjnE43#R9kr(vo7~`_&?@E+4Lbdb zE4;etx>!8K07><`bopYnMPgI55dCDbSGv=SwudQ>LPpbLmB;35?a4}_fpu5e{6IEC zStm1wue{189BRbp8{p9^2^`qikRBhFIoDbmq4w<%>fZ3H2CMhE!AFGa4SBnn0u__u zg*NWRu|ewk4(NLHr7i^6ue~}Y!N}-qL)QBwiD(fAxDKnBX@$w%W>rNewLO z9%s|DRF?7W`vtc{`ioFA>k8E#plxEjy@g^z zicVh;&8;s7nf6_Ug^TW6c?GkZ+=544K2yLk_7S9s%~%Y0BM|;*;CE|SIgn}lhSHUoaAdye!D%wb-7fQEcr1<1I|VB~XKc#1D`2##b_!8yQ%Myq)1=fk&Uz`_jd{aALE$GookKlsvxW4nI~#IZ74xbV4%7*i@LcrZ66JD( zyl8MpNX8Aq(BSXt@YMJipe4|$q^_YI>g)NkVRU1l)a(C>i7Nyq?WJ;lFhCT; zmChz%v5g#y>s(x8*J6ZQapI7RlvoHHcO3IUSthIcu(>{JT#J8ll;A$ptgC3cuTM)xX|W0qXPGRl zh3*&DGxN}k4mmkoyxnP8n~sXIo2*^M2~1mSc~~$~TbmdCEg}W9B%Y z)vzRndYkmX6a;KYQuM=L4I1;Q6<38@Yu>EBq9tB|db3#sWF zL(7nhlqm+9gDKxsRAV@cvDIcKhsb{9eCYi=;>uTpyO%Aypj`5XoEF?Lo}7DLd+jzX>8|_fzM3Ch(Kdg?VA=acC0VTDB~R@fBlKY_p{(jNWsGy=T#oKsgCFdF0Zx>xqnn?x>nw`h=AJ3~vWW|Z3T~!fBlWvN zA+Y$TfTDl_6jkyo9xA-=V>;?+nP@9WIs)_FaJquvLmL`9zq>0|Ir^YKV2SXR~j>+*xhir6e;-?e+yTVC+NxkkG=i5QPYrIRcgUPEO zncQM7<<#2KUDrgC|kwAl#3ht5+EYKna5~R38WkVqp zC=w{aiiRW*iUbJ-THHOjv^c?v1gAaO`}_az`_6xR?)dM%=Z#6G3Qzd@4Mz) zbI#|P%c4@~g)Fi7%)-W_HI?jJadI4{Z~(mSv$ce5J7Jl)9}znbRFD!jHW{PB*JRRO zgg$?BBfmwW?kC;td?Fbh*Hg5|-r@017~7|;%>W|){m_ECZCS>6qy(~zZ;?ohH4u$?<-hE_sxf2 zXs4r7smy$yKG3kT7Q|Zxzn$kXQX9JD`TH1F*iB0Zyi)Zn&t{^I;{*4*Ukd-7!PCG- zZ7=r%;_l-I?2${(SSQ|8^?v zcK@jt_Ucdd|L+7(vCOzUlo5YJo)j7*#0g3;-iP!7K*`*qwIu^0vR-h5Dx;K?ir^Q{ z;|pd;2FJ-Mbu{&43F&n6IWX##4DG`L8S4Z{H(&f}jz_=x81tJMThRExuIb71iAou; zLX)OMbjtY}3tnd~fAl#iR8UVdeLX|bxa!eQIxbtQCidOF+MjeSH`q2+>GS_jjX(4_ zZrkNd!%eGjbU`5cLyFkR0xJ+d5tatA-#Vm#3deLvLSzs+<0caGTeS9YIQ4)py)e*GvI!FR0YBE>I{#%muO?4p>asovVjBH7W zNrS+~y*Hq)!@LtC`wo+4a&b{CuC*_seQ<2&^aMR)z|0kU-`a_%4MOk77|w-6h`h@Q z-x)d->&92&<$a-I50+SjXH?WGH1Y^k;sY`6yra~BDK+O3a_<0iP*MnQ_x&mShX=8S z*-{XKwqwJ?zDaTh!DJ)5_<;&y@M6+(EeXVBm)U)%wRm{gBHuEmkbH|@4#P`5nDpN{ zO`S1zyuxIQoL9J#F5A|cKa8-tL2h*C&`xjDbIbG>=7)pKB5NU55;~h##)DGA63Pxj zVL+HN3dXI+(H!2SJ1x}x#SZsPdFL|DW4=X9RBb;RVs%ALELNs&|=Mz;1`K#Tf=cP=XlieSbk>A>U`2JO$j9Vu`ws#b^fACLp`se31 zuiRY{!WsmcQnEHVaA|$_e;1>Sz0Y6f&rfR|$%5B(TS?xrc$#WjF<+H1gs{EhDUnUo zj~<8ROxq~j>=52Y(8DcfoIb^veDWQMvMmI!hpRl-vJ<_^fuG5YC~nZ1QOnWsdv|f- zOg+g;Ff%{pCmlgnX-Vg$rJdi-_XXz4HDpFRJe79bQ5g=40*?iqb@|qGDN!4}reV`E zR30}n=d_{WH7Rosq{iLA+vU2AQRu-PTgXgHJR<~{Xrx%DO)jzHc|IPM-%MkK z7~6j7@qI-$x3h2~PQVvv;F)+!UZL>|1MhjehTodEE>+*o^%5apR@t(5WFc8*u%#l& zZt~*UMLGHAIZ}Sw=`&*^A&tf>j_cK450nf}r_bd11b*zZNiVCfHMQd9*!M)@aFbi; z%9HVM&t%TrTA+P-?Nt^YCv~#*AH<+_kw>r@b4I+b!{!4r51rJ0FlQr8-ljbl z^rnfnrcGOmuDPwZ%SzIrl5&1L5D)6O<{$mkpjvD8XR>PG-44r*%kiL9m8t36b^4Br zL+WOGf&iSWv3Dqfln ziDsd%9TosZ0EaT)Bs6T*@v&V6xVQom6{)CM;S2`^tXm~q3~V0g4`r->dvu312T2?c zwGn%#!jwvsP-`8kQfM^0G%NqfmKH5)0RYR$R|DXz(jF$wc?X{7mfLci^lPEC9!Ypq zFdvA0SL0zwv|WW}oCu72Q^IS0#hc~Ypj zj3;YuPDK)RSSN8RiNk6NCRvs<$)$f{@25+xeia3Y#rzn2@ z)eNEkG#6JeSL+y8t)yu1o{nF50T1~8U0k>u2$|rFZg3(t@h()=IHFed{00K}?y(?6 zLO61AZ2auz)HG|8R&Z&fPmuN!b~Y9TFd{@=h(iC+#3s0(oc{jbh4%7%<8u$;1b2X9#n$D*F>bebSMO50vD3Vd56|aW$jk zHid=GT2;5x@9rXz=&_=Vs-aTDA!`eT*TeFB!)#>XGy^>@^)-RrH0DopVIT;rDzH5b zWmLSg09>1Q=c(vDw;%$St4_Kug7WP`Zy>U4mxwxPK#mHn1--Y$m9s7dgn|Zc4c&t@ zmJYOvwJQn~fQ3IktIv}oK}^}Xv$H?SL$n-pCwixBpEB7*M-wpDi49t3=23#KQf4W` zk5q2T>I8ndyH;_}0Qy;(l4iE%!yl*IW^i3KJ60~QCr6p~>c4#pg?RoJH<{c&YS9K? z?TlJ`6(o#`Qqe`$rsLELpz?37P&N`rH&;TPyet=lu(M<0Rl|_&j4Y6m1BsIEF{7Rx zYhc;vj&Xi@;QcCY+q}LLJ+`7^A(ch!idyZjbDe%0;~q=LcB?=6*!yay*uI#BF93!;6G6FbV3DZw!a0~%{gzu@~; zcA7Ct2ln~yKo~=IRToZ#vL{YyJP#`@KaK%@)LRDx()xQwPOhmR;)1F26GF5D`W|9+ zPwwhvyI4HuoV9bXioB`V1s#u8Ay?@bFQBI)OJ5N{k;S`u*!@}a7Llpv!yT@t&2Y(7 zj@rpb>!x$h&=HF(OMpun)RDAX?+m}nqEqljI=D1bUW1^!k>qP9zXxu~W~87%HFBrp zItxwd$JRp0L;5uv$eK9qx7Iw$-BxmeDouWY3A-dL*74YHlmpKTr^Ru4U8)Mbt76$; z<+fKlszPBds*QPX=s)#avMO5%MReNuEQ57(8tya|QGKHYFAZ)D4jYzB=Yw_gu>xdh z2&0cLBk#en%=V$z;Lho^8OVBGVriT3$HVa&4p~(EgCU*@OfnRk`B`3NChowmApZlQ zf(CL8Wbkj<2FX$TH&iD14BFALQSO0`rOSl~BOPVso!cz@B|Vni_ovoxYVkT)spt-M zV%IJ5@nyWJh%X4d2QDxUKx>=I$!l{ZR1o=3=yW8S1PR4E?1`#iRJZH!y->bdRZtnjYHl%sfC z(7g}APIf2eBi9dg7Z{`9;s74NAsyz#fW^(&ba``E9-a4<5LKYEGJtg&n>8s+a4YhD z)O>RJk)Xyaqk!MjPpo(;{4f#ifT|P@OeR|~P`Tex{?so|3$!inOf1Um?na_%zL+k; zF$WA4CK4jc*1cMx>~|{lUKzL8L;0<%Qd4VA_Vfjq`^q_dvtp(s!iSE--(c|wwenF_ z8rC-dT`~98R$kYKf8R3y^ZIX{NKS<=A#EFKC@;@`U3dtMvq6!)R;qX^2aPHSCmlEQ z|M-rp(angUkvJmjj*({nbzTKwoK1E%It!v~s}02!HKv@IVC<9^%4;h`SZSx0P4=~; z$AY)3G$5C4g+!&J;($xh@HM(U+KS&A%&pcP8)l+9}tyw04Em^gt2a8OieH0D`|`P)=3c;nYf zIn)58NVp2d>o`%MazglQnr$zmHF?(CizJsBq55_cw3`+#D$XIHNvn1m2bu|Iw2pG? z&cTb;yia$IjCNOA-gVw>u>fEYjs~A5j&!k`{_3JdZ})SXXU5bQaMm@REp56B#1NV* z-7#|SfK|=8WzD6iO7|^Bd1{;trIEX^Ht!Rotk#`AraL=bXyvVPv9WJ@YO_ui z;;8X68=^w2-mQ$`GplHAg&k4GH%4NtaflC`Pq-3VMe(Mhk`U#RMzY6Azhi#d?l}`K z;b2pnqGap5s(v|T7^2x_c#h)3NDDbtKak4IzDw5}_V$HDW_CyWlg;C+97>$#-w)zb ztnRG-$=p;A0LdVc8P_iVN#I2PJ1%0mPO;9#1P;h2%H)ytI3ewcE9&j&8_6Pi zI$6Zl4=(?>`?wUkPxPWp1Znsy?n|13V5=xkpDV10n1=Y-fia3j6p|L?ENZMiQ=}mY zHf)dzT78Qcm;;Zl`u0Cc*m?JPI)Ia-@uV$#IN-&=_lJcdD2UHezbYB zY5U%fcCK2fADUaP$<;@hgWvXM6eAbhP7+Q(_?O_*DXR8+pBjB7#pfR3`wJ3JN#+|Y zw7RM!Gt5>w8%VlDjFfqQce(y2on8iI<0u-6rG&+0Gm`H6oA8GasE(cceH}lt4z<*V zsza8f>)Ko=&R!kITs*lSg0EY@IeE`&dHV$Rlg>?nN|-*pEWTyFchzj|C*6mB+G6XT zUsG~P72i4dLHJ2m8Q}RX1k*$HIcePfr!GZA#gEro$B8GOADob$*9|NlR{!|6HW2zF zLy0`S$JFz0s{X0})R})R_f3T^+hO2OI%vp*-amCU{8Rn^*GrwvemWB@5nSZVT~2Tm z^s)66bEO(WS73EJ99DA`>-T3qWQR`l85$K&B{qlGja-f^nvMFQsimTrKJ{ssVc8_S zdG9-Uf?$=Y1t5427fu>ehTiJ?~_Su6Ye|yl-3%99T#X#9M^CSO0$A*-YhW|solomcEqb^IgWq`i*W2$ zrB$ra_up>te#E`Bm0=-AcW)lBvD+b+6>my8P)DEqq=QTx=TFP-$qm&C^+TRW z&3Q5ZxcbLl1hKSi={4*6{AwoAA8WN5q_zUm8NxT@+467eQYe)Dw zP+Wi)YH*G;VvGwcf{apHHACY@I?vQas7yfQZhQ1Nv?r5(Ca$L$i97nE5P5f505TRU z0;g&@b~%^b)9Fj%bSgsFhH~J)WfeijQvB}!c07(Ge%F%vX7eN|-dv};Yvh88{dJe5 zV{fYYL)E21b(*Ke5JfWy!$41xyn;aeTJiU(3&KLlqA+bS&whDam4B?o9jbOQ?04aW z^QeT&;9PfEOMnEf`VhR=u#b0X`Mt4ZdQZQXcks7pdroJ5@c zWJ_%MLoSphC@@<5R9Rgr!R1d9r?VH5&?(VujvqZhHw^sI5hnr7X^cvlZqMH95K_We zBpz!wYiVefNVH8SGu5_lRysG68zP;%c8Y>v{LNZf#q!xOvQpJZ`+li&?bEhrL_m*E zVNBcfnSf%5^jXVUPkh_&$EuBcAl`{9DMhu)K8AfWd7O@z&&G0c#>!>DME95z-oYL+f^em;bTa^kqS;#Yp zZECvI-0(0uYf!%zT%PBcGRkPAc}^>OfBtg!q-$L_l6Ym_#t6|%e9kK3YnK+ip0M@^ zPkm%tUJWT3eHNE_ogiciXvv)x%qj78^v|xb1{?O)+#H!zt)9`Bxfkl|6?v@++Dt;< zOD+B}RPmE8EA`_DYf|W*Q`pz=m8gEf&Ny*T1iI$pnV3tlBl0L@ezR7W;OdsVJ@0J~ zzOPYLcfhI|+CzCY3@?nk8#W_-&Wk`PN`hInzHU{|A`EN;4w6W&Jvn09PzmH{gE?gU zZt`l{ri*ThKVi6QBK-UGA4AC1XI1r z{(Ke_g_boe(3#U?jr&Q_^DL1XPU*J!dYx$gDCpcMHDJbA_E{l5rx1$*2${shzKu!t ztRXH72g8T=zEU}!7FLr~j@-n(bTHYE zim`NJni7IrCOpT-uJD2^Qt`Pz|Di0t4;~~ zM3s{C%5;Ae)uRfE^V;Exrl%1d zZM$ZbSu7OhlUnzF(joVuDF2=DYH$8~Q7UzU-#iebkGU?03QvDq3B>2#yNU64;Cw0f ztbY1Nq*xlEX7EB7r19AG24PDpVY}dmXBbPol~`1pM~7^&&=$L(>0KV0y%)DkJG_;$ z2@e>827Y=pXm>Uy;Qe70SBh$O5{&u1t*3XO>mNzg+~pPRloaN;&K=zpcN)=ZKGW`I zHnSIuMG7wHiBc!S%UO)`PJ|ckr8>-bi6&0c18d=xxT%HKI5h@5q&B!F{Ovxhj{6%+ zdHLN~xSi^tNV?~^w^pf{ng9CLFL9m= z!JCQ(?*}sVJh&hl01T!+_yy@7ZpV2mc7O}>dw!=OPxm3-Gjx%vezPs5eTL91hAmsd)H}luk z--@aH#=QYwuqhTS9OY6g#jfvqkJ++l^4jb=&w~J>=pxkws!6tah3612^HO$Vk*Z?UmzVydwu4Ld5Q!i(T#rXc-!t#_lS)K1lS#=b0S!I2HGlQSk!Kq&X=K?y3{zfF$=_r$l#EFq$7fO z=k>ZhMIly8o}X_zh25J~AElvfJ?B-k1baP(Z$(b8ae=#POK|{=E$Q$|5HO6aBcJzb zTD^_Mr*p&%nbEuaLIE!H0UKeH9|p3qTnjL0+Y!P(reSL8(&NI=fm=6iEs&+mXelUW4x@XjZ?0bIGhv3eY&x4(Nw$3u(B zqE0h&O-F}bBru`3>7vp&CnNUc>XUP~#7d=Zx8R+Nz^*YI#ay*^;B}~iLb-7v*M78? z8t<|r$}rsy#`}J7$-JdrwJ}6a;?q=`*BQ&<$MhxB^enu7PeMPv)!9&YFZrZ7knnkg z@LeQ)aKK{N9T&XAN5`;t^P2$Bj$mgi0lx1oHddm$$P+xz=j~)T>Ch^?zT<|6<9VIj z#g4KAuAik-^U&5DPcom)C6%k#x**Y2jqn*|x$LCIp4qgf@zB*?gqBq&91*&|pRQ}WXAotD;D#tLsmRxCHAxn;?cLcp zcgd|iFgS1EvO>C^KG`XCOBm%djRn!Et=}WoR^NzjlVWk?jo^X1mhd?2r!ap=Vb3dW zC7aP&6~kRS|A=_N_3S~X#|_d@tsL|)S z6%JttE=3f>1CpLo(R=hBVb+{&GKesusr?T=;?uH?B0i(WqDGEtTfR8*6m`^FzEQgbo;E0+!tkyh>qoq-^Ft6)EIZby2Vm!=7dGpHqwFK(|wLNMf$}$iB z=jZVsfBkRW01_7I)_2Ffg*{$UhRwtr$NFE&I5G&X<8_+XiX4PPjl9@&o1aeBVZD}yKC zN2Z0M#UM%o9nS2bOk^{vCKgqLF0NtII(kvD44A>@+9|uxaDB^V5XAwkx?7K2%#CK4 zeR+$Op2=`&oWoN~Wo1ij*T}o?MG?%VmopN*jhp!8nHte~EH<=lbJ_nct1sZNTV|%L zfI*GWYNW843+?vw4=PZ+U?E<)8hCPjUSO@M5 z?klwv5?f6kG?Y;eJ&x|*&@^V>U`|OSPpn~GaCs4B-LXaJa}p3Gzs!2!HCDwKTXMw> zbnB=YE*zI)ct6jJ^%F zeP2kg7)4~pKTj0*UJ&Lg#+R-Cq+=|8|2?bEroO*?X!^>sUOSCs3Sn?_y&9I!76gGQ z@=M_W&usOHGDZH&j8}9jM2jtOpU}u<4e6tPi8vk{_OYeAT<25@)5nKKB5u%{O)v;g zHlP($G~gRouZ+K*9Xy~9Q{u;02uAYGx+LldkbZm@M^nb&3_g0M3eW4OaC_D2Z^X!` zk@eC7X*M~H1-(w&aGcSN+;)sqtJ{Pd>bfNA`$Gdrd+kY)J=ka^;vVqIr9|^)$sT@M zm67cVhx>ME&qr~s-2O1KZm9)43lpIvr^c&KF17Wh-2~1_4DcjD?-m}Aem2fZkl8ol zu`3R>R??)s%okB(wnvMwKOO)U_dV|sE1PsuFH3NRmpEkW0J}Y68KHZp{4B*2$K@IK z&$~amR)hma4Zo)(w}tf5AXs5CBzq1 z=iHfZMGKr=t-JYB|EM1RtSCFWu)rzBT$P!hPR zz7{#6S>v`l`X)w6Xe(v=Nnm$r3D#Vv8?6iT9B!l^a4SGgKURc2uCflr?TKez!-`s{ zR8M5qTHeWt5|!48jRn*gHM$G*d20Ahn^CkG2m*jCGT_U=EF$SAow9Y_D^A(d*xAtV z=(t!A^`N8u!}GO3{jR0e9odyGWaNbN_V-=wxX6{$;S)$`NUO4tU!uBOZP(|68 z7n;vj;(czBz+yKFv_!0U4g>H_RZ4^D&)oTf6ei`;hZW+bYu$J@pL~S0U6wYG2vo&| zysqn3-hLU9u9qJgqk7+AjJis8=fSE;9;M7oC|2qD2a-{pp0!+*YIh!S-Eh^96s=|> zFXALr1ecGs?TRzgFnK%gnmv1Zm%8gL{g!k1!(jqauVUS-OS|hAhdx+{WE7@-(N-KISvV zN3NJ-4kcG)^qS&PY-R40S5^2UQ}kUGg(3!?_eKSN4UYO97o~&C63o@>x{-8k1In4} zIkd96OJZ<&>nv}m5^LazCCt4{_EwWU>*6E-N8Dq}=Jr_Kjlek-9kJR6;x4v37^}Cl z^Ec>kf*y>K_CP$)Y{hnGZ_MIFmxqlEk^B{R@KrrJk6M7yFIialm33FP;>0AzHiipy zr~O_XD0sR{5WFOTAit!Zk78}>k`@ve=;+Q~Eq@%J`$`a-GW3sw#!(T7_YsZGxw^NC zH(eTHNt#x%Jh^=3+{gy_-aHGeeNWL@q|Za`%vktay@xGMBBlPBYnw*J97)dxwZuMA zM$(E_>4*MdEj=VbYdgDG+kY?r!7!~d&5d}$2&nc^I}a9?`}`xnGLY+ucLz#Mb=_f)84*@T$SDvf@&u@d~aix&zXsxe?rI?{b|i=Gxl ztgrJEYEE+sV#Yt4|E^%4eGoET=XIj{@!D0#&NE+Y>h_@U&yo@nB*4CfqE1^S${y*Lw}0Lv;dp{(S2f4VzSxIYZLM(`E1Py8n8{nTB> zOQq`vP)X7&3W)~meJGX;V3DD<4dk|V;qzBC8p!sixvu4}803t3Z-;&=fcw2WFV2m% z&UizG@A6df#c0+EUF|Seu%Jin?&7%$l-(<8%R`&qnJyQVxAYE6+&V*MF$m^ekY_pL zG({x+Mv5}h;@Ca2z3ZkHZN3kftEf$B3Ir@O5_IiB1}xqwJ82WuT;+4StA0BbiMPpR=azO+Inh*bVOuO#C$i=IW@Lc@zY2$MERlUUJ1N^hI{fUd zGi?+E-c=^0>{^$(miiB~W~DTX#^plHBWw&~tC**caJ6mVE(!74E{LP3%EJg(gekd& zm9V@uyWb^UY<@R|qn+#&J%+1ds~Rr18Cr4X))D2GH>H1sYn|2|cwlglHWl_Q%x083 zFC~rxs=}0M13l*R*u@e~-X(2H$(1Rc@p(sL0sb;F ziP(vLwFSsfG)u)FCZV-6sD#~ezCu`q3uOLuPIiU-`TQ>gNG#~1q~k65*jg%E`HwhQ z%iBEU+MC?O0eLulJ<8;L>u?H*hdOLMR%_xsT#d*V19zcOZS4hC4{x+onj%MBL#nX^}p{jZULFl8^nf z?H(JIA3F3m`Ub^V#}y1ASykk#>)^OLb(ZO?1kcJ0#;pLmwl+O(?E;gVKcd`u{i&8= zVLDJ&$EG7?nUg_7^VE6NQVQGy7$9yv-0@NCH{$}YnTUAx9{)8QAn05UN2AKVZ@?pl;UDye&C5XR9;O#BWlzjKG!K4e$eMn73bpLy}Ra|+oHfa@HlTv6+3X4>R-$Blt%Jq++!a_mI7Jq zZ3CRaYc}j&@$9Co9jXZ{JZod_gwJzsaZ^WO7hvCjFuu#C=!j*SeO!$gn~xX$cA15n zSSxC$;EXicBXdOzR@z3DG%u(RdT^n^;4vZAmXA33%ZGX2Ts@MxA64~4CPP3;-4P>~ zJFko#S@I%su%X3Z6$apP&X|zdW*uE&-PBqVnnQLtmlxKfJ?xI(tzL&cU@&4U=y;Z& zo0ZuFv&*utin9x_DlKm43|@4ZiPn2gz!)2W`|C&Mth(LB4F?bgOj$gEuQh6xWtUHL zH>9d+faFX;!XOyC=aX^&) z`87{bjQg+*EAQ-f)>0V@EfO42^k7~Gp;4szBKj+H$@uS)I*N!u&>()^5Cp-Q6fq2DLv-zYTu$i5 zE3}EMhmL6*IT1xxQ{j1E7SEWB@H$y7XB%*CAu^YKWn#~Yzx_!U_bFw;<;RlOy`cRV z>x0w~yDPuU2A;b8FAcxf7-+wT^xwT!%QQ>$CBG_(tLgTP8}893TkY0R@^#chE=fkX zcW}d;R_8o{r2{;kt1S-i|0Vx7uI@~jmJLmPM7DFf>#kYv~3Xo|M_^fh;J>&hCP z@%W4{#(Birl@IdXt&^9_<3Nf>JT|bcEP<~0`Y>0aEKEj4B&XF*UeEJiP5$?Mjn@Kf zB`;f>?~0Zm-mosk8=uqNt4=zhRQpin&oaFps*b{N5ninlOsAjcqdrZqdXUXeF8nh8 z`l}`VYV1sG!k9ymv17iA3&PPdp8)k?UY7d4O2{Ro{^PU+No1 zLk87Nq7VFxp7s7p{k2~ud_lje!an=rU-I=kj|JFpr*3@vq2F23XJpO^y3SGMNG$ZS zDbwg?z4Iiq#d)BA++tXTUpc-y$?~9sDF5MK(*Mt|uCO|WyuQ)0NP+;lng1gZu6xt| z$!md@MIsr*$&4dItRlS6nZ?=W=Y~By8f72GZ0Mx7MDYpfXJZO*0SULq$KMAE3*YTbHS zcmG$D|98huvov2=XCC2qO8hGZ-qtZa?InQW1ymcDij+bv#)b`>X)$mF466RyTA}5N z9S1GR{-4+gfFmbR!LViHW+M$7@p$l8Q~upz0RPN)XI<%=d74Ba%=$`cr4TJc{ASI# z;r~0d0SykxMGNcdJPmOI@nFQXt(?EXkYJZ~N+FU+j%^{&1tWmrh#xvOKCA zIm%>PgpEEwF$V{&ZBkLcY;rQ**%NI}m z?`(~7EZ~R+26RW~ew$7hTPgCAd5^m>{}F*!q*8od{4bVG$JH5~H?09rG_MCb^!){C zc%zMUlfRWB*g#qH>^ANYexHxs6#E?ic|q>{B-4>KwN5V5!pOH$0t9F4}`;2PT&!P_?!R`A=-+5wI zUXNN`|8dsfu)|N+FOxPS|3* zx4{w_2Wcd`mtsC#Y(ibDp$utprl$3AA@iRLNnLa{hCyOf_xzj{MAODy;&m<4G>hK6 z@0#&*0ZN-%P=g^C4NCE5(NW`Lgm4uxp@aZ;3(?g!oJ}Rah^v>G1SpjZ8SvUX@gcX3$D7^Lbp(5U;(;uK((amqtjg|Gn_Y%kkv zUQSJeOs~uD3p_S>HO-B*b#*tSWy<2xCaajT1V3ExzLicoNVY048jxfgr?9e5kYP29 z`PH!Foc-e!ZptNBB#6Jub3r&4JZvu?EA2RBdt=S=_v9EtX>ad+Ra#o3pGH_$MC;#* zT*%52j%5kpDe@3pjJ~^&%);=LXj}n;!)er(u~R?kFsZW!u*zfs8(_qqX)Kkoxb+7O zb9gR=Ffg!mhI(G;{*8qswA{scF^C}rSB6^${53gF4r2!Wi*xSi zb6q|lyD$ft#R7>|9&@55FEi`KPNK=~KW6L4)!RAP9kcg}E7ZjbOhvuky4HG%D&B8)Y3;lW6tphZ=h^XA*O2EDgOJz zwTR?g;iTh67KNf|FRM%6p0Av*uuYbE4Dnd-=PR6mryxjvpC`Wpzm#37>n?3uxg=-i z$LFJa;)w|uuWcWH{UmYPid!nZKobMJE{X}xyVv939RJ6joPvAQN2yrFM$zFkhPoRc z&BeU^%_hpXwe;v^Ntzsf=uaz!Vq-PlAFNDWuZglZ@z>_nSQHyqT?fK<$6VoV66<1P z;?TWuPBo}*p}m}zB#%VfvdZmLeXrUBw=W;U1VwAet8d~|SDaN373r&CktPn`;)-{Q zl}DZ!3)9Zr`DbqdW_g9+I7bmy^!Oix>&9(-2HVBJiAj>}Zgn@KEur-*TF_X;ozGbZ zAfKUeqFKM8qZM9@+e1zZQFw7F>1G=CE@U>6JI1mlK!IT1E%;`bU(%>N2tinx56)t4udd?PrD z*A>t^Kp@h%;AR8X2{VO!*GkSW8M7ZeOPeUP`9z5ve`5CDp4r(?(W%U>f3S}%f&59w zkSRT{ydS>i!?nij1=WNakaZwxN}=DB!wKMFD5BTRO^(Zzt?GV>7M?JS&J$Luch`J7 zo#MCvI_7P@>*Divh6*`ji$&(Cn|G$gnw* z0SJgF9To^_(J-z?n(ET8lo0n_!&J0LG&jJ&@PZ!;epKx0uTe{j> zSlXMlr_QVLgZYf-|H>RjY(Dx)cOUo$GO^4-K+|AOUGs@r+~3R+;50h7 z@wx{xYXhPxwTbot%RGqTd9aGi)q9LlucRYSq~W}OMmc|}|5`2k916tGJ!V#pH%=s( zd2e(N6+LormlzR!r}7se-N0Xjbamb5{=Iz_y-v#Lx~blE=RwK(xj(s7H8-TlMw+2X zFr5wubb4}HwsCR33$J-6K@}spGeydHV9X`hgeWNQOq73nlbJ}?jZ@qM8!tu8vMY9H z4WP$`mnS*-i<|}tkiFxA1D)Lh{OWbtn5B%62-!zkj77^bLh!6R3*KTb(~Mn|9JvVeHb^nP8?cx z_5>Obiw6{sPYlM5O2#iY`EJ0xo#o?R4qj`HJ3zJK%S6F6AO-D6`xphP$eGSz$dE3c zKvZI!3otLcb0Xd@gd|Jn6ovc3Zo(WrrGT7}Th(FU^koN(6AfI zD7J9duE4^mj<4Ev$LWh%Cjwnfqm3h468uee?)BIoc6>VGfQ{lqHC2##%~sxt-09ca z-X|W|&lH^Kx@&BkU;-@%*r4Nh;d1jh^eTHHHMV}^wBF?ycW)h_mNUoT5IUkLyq-qP zr$|c|c*DF-$6y;=AbtzxTHoa4IHQV-C9NbufAWhKzWPb`a=$F$y=}z>a>q9^$#%Gr zWVBNu>{ms@iw+R-Ao+NrGURl8w|W>cpVK+inU2DcuyNLm9Pg^QBhf~x%2|Suxh=6+&qnFii$9B zuBx#zy*QbpMnI#LH+oA8jgg#)+_C;*?|=LjF?)6)SF zR&ykEnV-5tMU7UOHK{P1yLd=W>Wh0Q-@^3V!HfKtl;$ks9u7Nn>i&72ZL51wsaKRKQgzLM;lWrG6)UlcEihh6s71}}Aq z(Fl1dPEO~ejT#CN{Aq$DbD{Y(r30=>Y+qBcuz>i*>=N^@Ig2`qHaZ@G7@ud}=WcR& zPqIEX@VAp!sP!G^oEwIy?%uc}W#9VYdy(`ta%pFrz>c?!%vsfjhE(Ggsk{nP?44wP z45(;21XQEfJs7W7{K-jI{~q;sW>H59* z%|}C7KqWV7_U&Ij>9j#wS%Nc5MJG?mtC2++oSKAL;nbl#Tw8tk+7^G48$6{%Dx==R z$ZXoHm%GE(C2U-SIsGJ7f^cAPZB*gzvn@2h@5r~9NGtDj`m>(@|N469Yh+tg+&jfv z1?u+PyYMU~C3q8xai{sBK?!0svOkn*+Z-y+$b0Aj`atvZKjklp+etp1Okmi4xreMC z5`Ugw6rZkYL9vGaq&p;1!b8I$mKeRp@r;uus~;+&{}m$5kz_Wqwo5@`Q=@FPJr0jG zKFWMy`r_U^mv{@7!O@^Sw}41RFZryo$!QTe;rRl#y&wo=}H>C>ur#T&?Y+{AxGcsFr(Tdf2Fy)mgFmz@0TUsreDs zUfyyA(y&iUjc=HD35pTYKEFrWd3R%f*2zw5NM5JejD4ce%g7{>1Cd4eTzTqqiZHPu zs&I{ZNmDu|BXF9Py7i91MRwel5EVBjCSlO}Lv^qwJodPH=alQ2H@x4pWP7gG<%qhs=WVVe=ucK|?T+WB8Qg={u>5<(;Z%w!v(QU`o#O$K>_p+Hkk$ z>%!w(7mEtX(m-zt#?b_&Gru@(`#|K~W2vb-qu(1Nk}j}NuLF3kt&a$`LD$W>za$}4 z_ycu}Tl5?KdG?g`WnV~??Yw;Vl!X?4pu6BV3FbQ*2Ku!>)) zj*5qIIf{au_i>5rUAa~(K42mIyXJ@!Wv;*@xxLV`_9&l;!C{m|!JuV>eGn0>totSt zk3ft&you%cE?SYFZBMgq(7xE$xmV9uk7K@Es8O2azwJ1Ogd9#@HOBFPh3cZmiF?E zA(iE_?f4>>NPEhqNQ$@w#q(H)#FFeR25oc`rC{hhP(o5}~8R-J1eT{8r_T}E|=1bKz#P4W$vYIr;hpmi?!zx}Xs zYL%GaK#xZZ^lRuaU(xtv?`V{@+jovCH#}cZ-rxLD|L}7gpQ$ahC_D}7ua|Ig#@|&j zok*Jc796W&r-^@22D$QWb!S^r-D0tW@C+wjYo*QsO$SU9WaTP)(C;^FTT=p_2$$yHgrS4d);NRiFg+`#Z1 zkKKr}{TKx{{)*o3L;e9LXo5dH>fI_Q?4a%&^OSD z+p%wcGpVZfIBGMDFzbOTE|=3Fx#Eg4&kxDFMUGBZRi1`fXL^jGu61cH*&uv|&E=EQ zxm(_E8dizLwvQ-WFq9RRnR1m}doY;(Hh>Qd%#YGQ5TelQ_nj$ld@hlp+@c|#iavX0Tm_AlX|_(>}e%p+$B>9 zeSAhnz@Ftvi*Q`7jl*ZL3SG`A9PW<#dIhr@5Lr{Uh6xEjN{NJsHr}$;((#{8zV)K{ zKsucR?-~B-aR0?0Miu}uc4A^H6^ucjC`#2ZG%iJ)T$}|p7j8bUEl*3>lx@RuEdT`V zl-Hqz&tl%)9QuPz7i}y1jWs;xreQU*MU=of1zl|GW}?l|ILsdgc%QKCo6uv!S=L`9 z`GK(tPI13vbG393a3IBcPCh6N!zRvJeB}P*dJ~JZL3wglpuY|x368##tjg$K@_nm; zznm}bo2#c@a(>Mid7MaCNVICiZbvFABHQ2uyTC1pXQtQ;PWJE`&w0H^FY8;LB}uk^ z_eZ4oVFQIAoj&#e@kjO4hvq^KAL}fHvZCdXq9U3Rxw`pjR|FtY6TgSe?BSi|dbslh zCA}qBEIxHCC{r*>679Hq7}3kG+8&xRUzdkSf*IJSfvbhR-8ysCIJn#=l_g>&&k`gup3F)My5 zDyrtdsLm+E?Gwb!N~%=FB@-J)c1a_O$7Ko9n)mkSrM~d4oVEQxC?4N{TqBZ8gl$y7 ztEMB&2v$x9>pf?AeVn3x@+C01d_{yAvfaZpF7qufV7xtp!>>ayjP3o#y)@p@v)>D} zRY9ey1+Zh}IsYSfWS;}R5-gU5iL&)+4VAEIFA+wipTv`%Ng`!h$JEJb5RmtS4xD43wvy#Aft>S@L`-?R#m^jQs< zD;{ANhn90zjDoC?p)f|~n$3$7rZ+(;Vl}pi?+;l`}im0Dvb9%_5uP z1vD4J;3MK|Jcj^Oh!Ax4!23=EJ`KpPO&-AK8Uyed0w{kk2#*Ojvd^=|oS#rRffeu5 z9<@ahuQ6 z#kT}tXLIAVAuwloPhpWlfQ#wml)|LTVB$jif?>zm4rWcVa(@vf|5h%=HNmM=!z~X* zyhZ~J;n}7guRyPCEqPq>=S(n=V+lx2=R~m>X%$Jp#IJ}B>vBJwOY$ET`igOH;sc5) z`wI*g<$OI%dyexq`_Rfts&2xVvfSC6{6;_pRD8UZ4e<=@;N^+3;*BlT#o>ZA+qN7f z$LH&93kj(2xDp1fKuXIt&k9b^-BEsv)92KBbghOo#G<6Yq8-a?K9GdiIu+tegF9yr z=&o(X9xn<;vJz!VOeV$VRNaXNY}kWxWJ9c4q}f&gaR^Jx5a+nbK!F5nfdR+boa#vS zw>}eM&ESHXK<%9|?KMk$jl>tU<@idjYv#>s zn26o!-Y4U;RMkGNx}|6uPeqvBkef6?IX z7JL{W!3pl}5L|<6aCZp=f(}lw;O>&aog~-tvkg5wNH!0l~k8N_|H4NruXam#Sx_< z&*z9_nzCo>%O?u>fQ83*o9>>om52OP2JeP;oUX2`8)Az5jUKg$Y63gyAD`8fom7Us za5eCU=(W;^$j@f(#Toa~(tmcv-fJXo-L0CO7;JN>T~%z^j0EV2B+?{ z9&dMJ44L`OoazsA-ViF=Py0LKLdEe8Bok*JNu7#o{Pp+bd1#v_OQxzh-sm!~GuL|j z&{Z?g(Ob_n-rTvK7GytW6AGjAoJ!JEe$0xj_J)<@2$|;(93B)leS;O)*)Ntfq5QMk zB+NC$Ae5ip{$OXbN{%Znt-EVaSw&Xcrx5z)ZEY)0O-ZaeMchnV&Bqq7WKz?pdQ$9_Jy^$< z{JFEy#W{55c-^E0?K6$&d<)+Onn;~{S5RJN{3CX-qI(QY z0rvfP%4f5Fs0N>w=9HXjI^f8N<|m^u{YY#0JNrIc!#(@!D8eSv0U_zN~Vs zA}_9*h@5=VroyqHpk%Uw(4+!~v8K_2Q{SfQ;xIZpqbdwG^CGQTrBh`agX^Z!^{41> zyQ;jB5_6_$Edc>zq&jq``nZ{$m;Qxyy2{QLDU|vH%{|KA8tUn%;9qeD40iZAifxdY z#<9uXjcsGgd2e`UO)kdWjSQK|=UIcL%w({#y61Z)5@3;rGauE7OO5*e>)=2b8wnFe zXI#aS@$7-_YPiJX=|0OufZ6A@q1jNuD=ZTqyISY5>{w{vOu^UCt@r0m~QowQMr z72E8Vk|a`n4!lj98{AiI)>BNo)iTE2V^#dIX@+M%%qQh2S!Ft#NzKz#j0E0{7Vm`? zimXm&L`3Y(qX(fEWRqR~SdQOmH)?Mo&Yswpv2X5b>&j1yykF?!8fcJvHCC6nP}jTcr!G(HGKQ&hbm$OKjA&3sX&; z`kwjr`4YsI?u@^Dcdx|WuzdK(NExth*Cz#8OwO|H4fyQMZpCc8-oq#t!H>2hq~;{c z)GVhcyISqUiJne(v5`_C!co^BCEAeh?jaR#uPlhX!bBXLn{)|mbarYz>vB?tR(a;_ zI!SCQvXhtB8tbA_2OWOyYbm{+Qbm`QJJR2DPZTWhfL|D_C#Pa1rw(Y5QW!n<{n?tO z`wF_rA#y2rU6Gh`m*`J~Okx)=yk_;d>MN(LEJ164sAANt;%uB9Ulo#(i@Y{mq2zCb z#o9MwzizkNwOP4!fK-}dJ-f`xB;(imRhjeBV%n=ziR?S804u)soIP#D2fr=rqLOPK z)6yTzHJ?~7c`7`Tx}Gn*Y_oF{QRj}HghCZJ(5L|NF&__}>jl7PW|yFGFd1FF5Zzu7Vdcs%piukFk}X@e7Mfzz)J z{HfD+PNTC6!!HBA0rMWljY-g?VT#|i;T09_(yEKb>Bj0rJ;=?*H4_7h9|wM1_?;Da z!t1}^n0FHrncEw3?9V9PT>|?FsNGhpEPi`aY%#8%aXb~L+`DL-7Rd_L?c96cgcb7Yf=`D2@u5u!1RN&qACxIi1@_f;Hx zM$SxDW+$SC>UR0pl?#0h{k=umV!3r@3wEFt%ibms-CQemH~khM_urNC7HU8Ax(XBo`Jbgb^o! z`zF?dPU&IuZ(f}F5;fFydez$T79;SpvwbJUT3Z@v z8~rddqdz`UVF4VGd9`Yq1+GVYz3cq;J8ovw z(>{}njR=n~^C8x+FeqOLA^6{!R{xz^?|Qc4kJbO@nfgL^!mpP9N*=%Gr|NO-p7J{0 z;5+(HN@xWe<@2KR1!emU)?LKEQ*qb-H{^L4fL$P%-+3r2AZVKP>rr?rEeH{xmJ2_x zPkYv}Wz7iCUN6-|O0>{eBu;4yi1bx7HhULK(zf?MEiIQb@9okD4r_`ntu_~>^Al6d z+zVf#SpPYn}650;Vlv;{|0zAT!PHyWzl~p9kO>3^x+<5Z5zDG)f>6bl9Z^?RV z&6DuVEU9`Fda_zNCXp~P0lPzYXqzTDKfpSWS$P{*D?AW>hkN_=S*#o<8C2K?RMTh$ zuxf4V+VXWTdvor%3Wv{`Lsq@4fQ>mUoO0kHSmuVbL$n=zD&Cx>cTuFL}p>UUHbIxm@mh zcPA4P8iG2b}{BBXTaMFpatX5@=HqN z_=}A?tCeZr=et$@s>386PcSGK`!S|K=>C=5uOIRN97HEA6d=ClnXwyAPoLva)}nRR z2#ST(sI(d7JE*T;w(&E8N(;ggrr5b-WY699ja00Nkz&SmZnSwa;slx-++;Jw83i+i zubE%9cYGExElN_b(LcQ zjZ>$iDjO61=krVy{;~vd^Gr!4zif-bJ&X$H$iM}`8XGQb1c?^S8IhDLf;s568`3+y z_JS30%x^seLgo(Ar=s3)r?3Q6;{T?7vpwfy#Ok4|uU*+-6zcgO2n5UIKy_flHO;kqN z@6R)486#RF#L=B4o=(3EPwlET9OCN9`T6<<$2eUVGwxh^`Y=ehz}k^i4(UMXUix_TtMa?wIK9Ys@( zslb&5iLgE`cm3?k&S$G;!d&eeK4B?onSHNynq&v+P;-D+!8U1QcRI34nYtTq;W0kE zuiv{7z`>#WN!`!moa9o-!7BB;oKx&REn}f9ZB~Xht?wm*QmT@w)_kiC8{tkl(Q8&2 zfA60qUn*OS!VMYR(vKSzlx&;#T_^h=wLg|_^Qe*=vr<|02gEVmeiBZ~LCx9us5l3# zf>2bhT4ZZ&TE|kpV{Kic4SV;4pfRg^9PI^bXrI{+r>m9`gm`_}23CXHDM1~;SjTa5^ac*Y5uvy}#cRVFM+6~8GB zt8;uz@Egf8ocXdoEVig7YHJN6xEs_veq>uh=gIh4sE^<^9lhqujs>nL%#TwvdEviu zN?L3VX3zW~_n>m!_bI*JnMw+M(d-RTz1v;UL~T|RM~p@$WiRdL3Hj}Qm=Q^2S!88{ zare~J>IlUTIqHk{evdm{rdC*v=dW~}Vb|%=Y)cZYkmY8Kc){M@3iE6A-NDGRqN7u* zT#}V{YYQ&G%@2K+Ni44uDH01)hzPE{=&w}pr=(mmYKgxyA4G8;o@zilDz|^%ThE_F z#rGc(@-%0(%@4w+<112PwtLG~MiWB|6t+$$vSsY~_o`B>t!C+^bjJe-$!f$lYWa7K>cQ;NO zqhEZq&*S#ti`94Zc~{dE68F4<2_z>f+pM^`7izwfgHyppCQsyR1u3`s=~$#XGsHBm zOw1fPIVry_D)^j;uexI2eh&R-gqn_(p=8<6+puwiHTQ{+*+dl8?nFRyEq(dtObKjb zIc?yejv<~UvIa-3t*ygf`BUHy<$>bCSGVojLaml5KeS8|y%009e+1yQv7b+uRp0Q8 zo%0-?RP!KlV;>VLn?G(y^Zo}h{8x%1#;b1b4UuLw>XgL`clu4$QI&1O@RP~sna>N` z`qVDmrgarGennBV!(0BRltnmZ{4s35>r?DFjL}h ztkh0EKRe|9{MOUaN#wj<_0Y&*nO)_0VoR$1VcgJpG`D)Xs7i^KSi#<3LtNn1_o})x zw(q*3>yEuQ&m0YLvb9i0?0Jn!m%l*yq-EmdBlt<3{63_>gk9@|K5(sIc~6teGn33A z<`|O_SQjG{uY5zSlLdA0AzCTZt;hMh!AVr9g)q2NIEvSpk~w)06*0oyq(2<^-x_7iBsTw9U_Fr1rvhH^nGe zsoNi67M=_U3-yZ;6PH5+aTL5bl4OQw2c{|Z z3caXzvx1a@WsJ{kwc7k`PXy=hyhD?geiM#vl8g&wnrxGeSgI@j!OwVoW#UH5OD{^x z>mFAQZHoP{Yj3_`LOVGKdo=(kGJXzz^w$vpo>`BtwnbTyUG)s>P`NWbU zJgLP`N2ebMS)UqvWD_ca;uEa+nSYB&kV(vLGYMSk@xowy?*l&h zp5Ze!Psopl6UmjtpAkr2DJvfY{mkr6W3cZYdVy!B)_UJ zxV)W=JF`y6$kCGTO3Uyc+RTEd@5mQN6AG@AO>y)68o|ediCOEd6nb1^+Be{<%mav*=tW=(aHMqxw=nm4KB0qY_ulj<*@tA%NU^BGoon1T%Pgi_1$dA9;UHL(wP4wG1RcARX$0|%DAWcq)+C5DbK(du;a zC&N@@9eVV02U+I6fe|A-I2}HfNijSxVVgfw5ha%>O-5*XIyhB14{jbgWlu;VjW{d* z9~OlBzbuH9vy+FFlZQKq5BO07#I9^*X=^6w>;p0YzP|$Ta&Ut91UNxPAa*q~Hy|U3 zoB!{O8dmPko^BRa?w}__B;(^Dr|AJ0p})S#ae=u0rY9;27$<VmS48=EfY=or z&8)2?|M`^q=TqTHT0K>BJ1dL7`RZ{9fc|C$PEY&S;qiBc+(DcGHD4FNaH;{s zo-F1+hW4q*zYMLUt%tjsl^d`D+{M}HsTue|>{8AS&Tg76W)?tEX)7;V3o8vdK$ic} zCd*sdTHAPlIQal(n#yMG_8{KBYpLmJ?(rA%zpC;tN&Y20yRNMzV9NP`Hu;xX=j0IJ zLrsB2DEI_XtHOcX2xDY{QNoQBFGMy{EPg3>7wF(!>Q zqk4_iZpsF?7Xy+;igg;&y2B?l_?Fv%l1OVxI+movTnmxCEdAoncQRL5+1j(dMX~m6 zr^%?FPsSebSE2fId&y{88ykgYBNx+6CZe?fC9E9-F;RGwNSc{B$-2wR7%J`4aV1fA zvG{2VK>EGYse#ZQyKQF+GBUCzC*}vSKes`}uX<=LK)Qj{oGU^1LF@)b-vOy25798< z-DLZ}_LP*UexBdH!sm5Zq@Wbi^BDRf!NIWc>+#YA3@)gyu>%C^?Z<>fVq-ZU;WM%5 zTQpW4i%iYTuo-qpOJg*k2iC}|T3W_TH!7pjgw7Jdh}2p7_LZtGe|N?byi_f=dX2k; z4@}5z#B5~}KUI!P4ccKt%>L#@0g@X%3)pQ1w7p;3?c~`=06gXW04YY9ebaQ;?H{=a zO<<;}eOTq|zKNaYQGKG0#*~_~dcMo`=V~;_HgFPuH`s8OoND#@*Pn&G!Iwa_91w_< z;$7GKhsPP$8~9i`5||%^p(>@5d$&(d58#NDRNW45FVrpi3mT+oMTXt4!rTJ|p2RXz zgPyeE_Y{26Uq6#(RS!x#7Y$c?RJ1tP#4( z;lF690kk$}PZbFR^+HkHtP(yJz5(*%x&!i94>s%$yAOuR3G`T0^P&KB=*08Qp;?6k zaeVP_UHkZH7qgmk0f9KeAoRG`lLpRwLA#5>&Ot@f#1a4zM6ec1u=w)g&3wlL$10wo zZRAtlU?A^o{nf%&)EltKNF;+7CLTapwoFfOFk8s`XsP{Yr|oj5x9XjFnJ|ewvmMZnm&d&N{$s=NcjK(c4aO8KFhrb1Av)nRwtBl%-Ikta%DwL!W@e55zGxjTg z_@wT8qN<~l3e`(xd1D@Pv*i7Y{>2ML#O8&EXDbqJ5ZZ@NIk_J7tke;Uc) zv3Jqx4t&r;-3Yd9`J1Vc(UK=tSz<#2_necB%}Erp^4eJPj}Is~c6N4d?(V0@o&Hxx z%bn4&mxjqf8k4^!k$=gC3DC0CBR@5=+70!e-+Y0DLj%-7!v2>I4p}1pGrJyC0&cyP zozFfAxWC{X92^v;Qq2{8r=g)yW7Hm=u(q~_gh6b|u5q-`eC$m&8~AwNsyf*2eTM;A+Y^Lf>d3pJ#PoLCSW}^{QbA%aB2jZ!Z-C8MFAy%v3!Xn4I zgZ_N~_>zx@$G4G>$<@_W-sNBxdU0`aD{Shlpn!&mNr%g}Jy&ndkC^s>3!Lb_J))2y z6@e{dR~NgVGKG7%^7R9%K+vNwJ_|d08frY1h@=KCQ)IXcUEtlN8f=Xb4wY+`y4dX$Lm=o5;*~4Xv)cOl zkVy$)P>Cpx|K-8g<@@`9gRpQ3kaV=vzQwo8hr6qHgi00`v<37pUsChCJR3wAx3jl* zMaA#s-9S;P4zCNiJ8YIm_kI7GlPYY#Z#gCg6A}ClRzE{HRG^w_S6}DzX#MM#o+av2 zw-x{0{iyo~cr;z9i?_NpoQaE*pP#Qrg3E*&Ow|HtOB|&T?)K&3!ra`P8l@uC2dh)^ z!81$P*R4N>lr2q3<+KM!4D>C9SvOA)B(~v*Nr<7+Z2yz3cc43gLJcD+x#QW*+rtOV*?Do-odBnGmTboq1 zL4YFnQeFEFz_0xBK3-mGcK*fQeUG0=L-d)I9lW6&BDEiITB{e}7bY7^R(xw>DSrsH z7|7EDYC{`bphqR_^WJEBdU}k!{_Sv*M912#D{;hS5+dxGkuL%QsIsU~ZWAsouiKac z3_%(k(1KPOcM8(cNr$aZq_UelUY}a@R-J>K8Q!_gNKM(?VtvB1JZ^Kfy0kL631 zZ2s_O0a`th`aa$QPP0@c?92PKh@QjAQkCygaI5kfS5(RM`0$LtjV& zpgPSZdU4cZ*b#v%U*H@7`#`AyM_vL!O6tk)>uyUl%X&lxp(ppi}1qg1{xX)T;}s`O4bz7rV6#l+5a z2L%}!QSz$fiaPowpQEpuL>6adQP@x+W~9~!FcT6JlWuMN9DZtGcE^!{iLtTwHeq35 z#*VnCO~-)MI|K~pki6Ej>0lQaW*O3_cq`zoV>>cb!7!=&;BVmOR>lNvR(dxHisHo{ z`RVCpz<_nvDRC^Xt(9xAohm&>4xw6mJ84iz0X_8%Gg^RfOz%_HgxU z|MxTX#DoOU&uGh9!&We)Z+nRHp;;0M2`NeBzE4}p)YTQr#1~lxR9UfQ;n2&b_DzhO z1Qf3*VWVX+Q)h(x$r-g8|QkDD{Ai}E^ED%{@_#Y+6y6cbc=?iH1l?~n)AOb`_#lBq6s z6Gj1b9FvFj00PA1`t0LuXEpz&j{Hi0Fk}doGRIqLZ~-U0M$ME;GWdzhI-o0`m)`j( zB(*WKu(}EL$w}bmo1-B;$w}X52dJDq6)DzC8>w=l{X~RvY?{^RJRAX`|$3 z0CP_vjhi_tHXFnava5yHf`kQmOM!s_Y37bGsv{*#?V>9^%iee@OKP=XEoDEVhQ^2e;(14cl zlI)w{$`Wwr-2^~j&a#ew`&mD^vt)yw(alo}Ip&$~!g(D&o^^Kvc-~()ClbccWdyL4 z6htH%ik96nvd!_M7oH*7je^(_v+C#yJ&D6rre{p?QdPUu`fZe9R7^~iOP@GEarKEJ zE1JDJOC@AR?{L*0r?J_YUB9ED_$7=05AP6i{*-IsDVLFJc)KARABDYLKVs!W0DHdW z`qs|uSFkR!Y@a~($jJ#qBynYB##8k;R#(|tv|MY8O%6WKFD#^^;7~c--D7;B#P0qa z$$(3>OGWtAwJ9`V`NtjEn>TL=UcB}>?!4mU;yNW+K%I(^M@-1caq>)!VzmuKSb@07 z%0%U|>NVa1>&q&$yNCJTZ5^!94=r+_Rgl{xlxTwuhYR}l$X^xky!X;O<{KP~+ zA9ZPILOoJKLqij3zTzqg)RoZybRHf8kZtzr(dUz$SJug!exR|S)tcGsuR*U>fPUS! z2MjYRVGMI(e{M!`(d^zs60);9?WQXLR3hczac3i56qTGPI#MvgkSz^EJEKMLu4bbG z`dz4A3|ShuK<0Zmj@bQE{k65VM{`8>z4FZwt-rRq$Y$wc`6W6rZeWqaN~dLHp3DKp z^~whvX&e}esAA>D_BOR>K#O%Kq})TmV`t1d_g%>utw1uL7?8bB_Q~2%N%|QtJKwx| zWBZ}npt&ouCU+z_3nb>Z1g=wbbMnN)2gKOuD+n(y`^qFBAYf%>b#?r!S-&V5$Mpbu?qtJ<71h&OreB?CE9TW~ZUBi;56sP8nVKs7My6NY$b0CXo}MjiJm0&2e6aTX z`ZT5}>V$1v?tRFstK+Qf12$RwtT9q-`0W2QGJ;~*&AdBZ2Rd+rzGjDuORQqf_w5B8 zUHB)~_xZW5n*hYCs;OzxS3buDz(^8L)b+)ao4{Y$Ni!N59)18&L)8qP2zm{P?(pBE zK@hE!>gM5~9Q8XwerCiAx|TgsuD2)3PtF2AeWx39aCjI?r4_s~f8nhE4>3?tX=!C~ zi0J)EkLf-pX9#`*2kZUDCqO_nTu$JRi;G5I8j3ioX7DFV&{Of<8O!9(8zXfqT3VKk zf~abiWPhrNt7^EPa<|~Nf`hDoq>tatVD7O2ffdWI(l@iD^lv?c)2Gi@d%FNly{K4=w<0=mm1VaI9#JR zPNUR0PR)*Xg4xo-yzu$In4`$F8F8GBcYZVXpR6q;uKXiv1L&+y<+IFSe2iI|77mUe zH{Uha$~N;g{G?x(y?q8wS@*j^Wq4BygDdmE& z%`*bGBPSXBmE8CJXtmNGRi4DyS?hwLN)ttoRu_I zs`%|jF-#X5bAG|g_=d_aMfv#>2TYuTRIz^+dSrcsU4a0J-EZjoApHYVeTS7b3uw=> zZ!=Hy@W_Y`#0nVr=~yqO<}F%sN5>%Ton}rTx(4`Uq&J(wIgfrp@#OX&0Z*mLD_0Je zYL8v}CmV_686gR}Al9D|uk8}WiW$D2-m-ve(DR3j&gKgWrj8EN3APFD8UFqqz=q9& z-A#y_;S-vgS6BB&y9hZ!Pxa#44Uf&NW+R_~Q!^=RY+j}$O|LyQ3B1w1m%&!D(QPu9 z`u?KjeRD)?WV~wKe57A;n#oP|2V#4vUh{~~-!-aA*H2D^RED1>NH?MUPCW_mL_s%0 zCM<0dnd-FQlhvT{7l1unP4PXFY_Mx^WrKY}}Rz}~b{8+Z* z-4QE0@ zbco$WW}tNq*}zJ&M7p;(EKF{**3ioHG(n{f+-;h`H9_7{do4<)sGtDd6Lri1=`S>5G?)AKPNYqk@l=Y)A#*_Y`>cH%};@;PN?e99#l=tQ+-5+LME zHuLiG8vRw&p?atSu$@n-&NrvIlw{!*5uq$VV)NXaD%aUA`x#OvgEsU&HK$v_H+Mc8 zsAvPTfmEO5L6`>1S;!Oc`8!ieK8uLhbtc<%oSxg=%nN0GgP-`n*JrHTf{}%#K~SiZBKb3YC5Afm z93FnM{`|8MkFAGu#uM0DMqC(OYWD_`3u9xZ+Q?;k@K}25t?t{vF4Eik8~4iJ_>pDK zVJor#m?`@ehVQ-?85^6;X*Jkr|8qXq$JbXtv5SDim3BRbKRZ{-X|d%L2yL3J!xCu~ z6ci3eiP#XM#C%S~!&biGo%NBJG;oELs^$Xm#=Fz?fkUx!!9@C8j2m4R6l{tyIUruh zj+oLFa6dUa+YTw}*L3X%EXb{Z-S$W}37g>;Sz}{Bsn_Q}F@N?t)PL~w z70v;INDY5n#DJ*iN^zR;8P|@I5#yKg(Oj`#7ax&JtWSSEMudizr2|SQhqbmW!GE+U z`tCNJ%I)TCb7DdxSDc9YY6KX2&9hWqE?9bc@|=aoIxaNjt6a1L(Mn$5GfM}Mp%Kzi z9MCXcOPOB?rKF^^>j++|NPqugV0H%tM;)xeC05fkWqS9MwI_A_Nc@zi2*gn^GY z);q7?WqbzgH)hCuE}wIbB0=dR5dZ4bzgwIiHiy`~T9C$U@9HuRV=?Y{P7{28Bq{bG zEB-Xvd=7|I8O184v333)5jwFtR1Nr%&H-i;BqLap@aO~Lw)#RF$3cID&OTrfUIQV+ z`CMj&H$j3~ol=m^$A`PfAi&36=4y$f6=tc=!%Sfkf8A3c`GQh?PuKdW1F!S5Tj6c&R_OQl_Dp4b93(1}($i-j zdkoe$zJJuHFnM>OZSOjjvi72)tjs2U-{B+B-yRo&lEpC5ypY&2<>JG2xnJ6CJjJ;p z0&X_bY%1TuYu|e@rCu%+X|NbHJ6#m&H95?|OMP2{+D$X{0;*z~$#@6HhGSd6D6RxA zhBOcwWIdcg4=H|-b<)?N41x5CF)RborOF@~Xx?VTiieFV_%j5Ej{*9${)=sS>y^VI z=)}G(Ig`)%t+jOzlglxvcYyPukn9I9USC0cj0K*Cn%dav;uYboD~l+TI$?BRc&g|A z3>GP;bkn$D&$txV7j~Hc{MngX!^<%QpQLXDe(LIY*mC!@a&b=~=kI0eg|acvp#e8% z+hjKFE(0_tA5}%i2M{d=;+DGJ`DC5c*ikqOKK&juWp=@iX!9k@Ryp&T~(( z;NJt#m`-%$thxJPm^&9{W@on-L&`HUh!6-ca?`T0btF(_OE2S7sm>M~3z7$0shz`46lfTCT3AAxIdwC^-D` z=&ouQYeJ?1u$KW~^R;p94stsRCK)3_DhngFGzWIW7Tk6hA{Pa7qR-q41Whk-C`*1G zO^2(}I=|w**A9L~RTK^b246sT(b4rgoaZ4d=@47&^ygE7XnrA5L?x5DE>BJ-q)5-o zIxs;%DFIFefbJM{10gr!2ytJasvg%TtH#QZJY$y|2GR!PStROU{?DYHn`PH6u1}#j zunfl_R%8kCThQ7QM5LlWPkpN&U&cNg#SMz{Vn{V=_fi09NCg;1>i#wnopI_A>g7)~ zlm1|v-zttuRg5QL%!g(}YWh(TD&B)hDX8W3^_wg4A1f=rRKPTRX&alddi%CXM?#C` z5r`sBVYOQfVBce0fVCNkg75U(=mMBju!E&S-+>>1S&G7CvwnFv?UE6s2ZY}}zY{fB zT2f54>bwEJ^qIRhe|3T{y}v%M`or)#3k;2F&4n;SdQmOu#k@g;*H4&;J-74Z+Zp?r ztQ^sIf;2R6!LwoKRx`;XpUTSn23OE?t!33&?yip0O0`;?H%t`K_z1n;0fP@10vn17 zit9!nlH^#@{Y7n1XTS}oN60~&%PM?;AQkVpxhcZDEChQE4sYeX)@!=J-3fZ%_7Ft{ z0JHtbGsY$5V_&4WTyv5!0x>QwRp_FIMTt9nu>y?%Ve*)pb2|u<>T6UIg&0S9>@0ag zj2gT!%n$bm^`i-(K@?RpdFmX(VZt~WBMQ-g=Mn2nIl{j1qe#^u=jubK(wR{q-;gIb zx?W17DXG&+t}K=o=$UHyoVUBKBa>O*w>vKI8Va37c~MOZzPt5krFT7iB`{mjaMI%9 zDpdwbMvftFLixk#xu=!Rtnt2FsDTQOuuHT=*S|5$=9LlRAY`?SpAH+LU)=^eIWYnH z`E`n%oZPkhOE>L8i^wL2^fJcRax;Vwl&dgZO%{19J|vG9aWaVM#kvGC zsB(v2K+xcOkZZ(cRdA1M51T>LkMsG?`Ah?fkhf2Byj~3*YAM_2_xJaJNnTSlQ{Pi` zCFC9BmnT3Wje!qk1b^fY+d$VvS&*umJ#nurK+YHTpLLiol7b*zL`%6~R2t_LE2XV% zeXB-ig&4K$yIz@*=qq(@?)Zn{qQ=Kd%A!7Y2L0)h(v+BGF2=ej7^I#tuae_d#;&G) zJ6WPE-#cEQ_Zg}e(eTAs8ER=|^)1$`uiJL>md7i&9Z<`PmxuO*40+7P7CzR&2w zuCy7#Yx!1Ev0qe7i9QQ19`{2&oaO~m>PN%XCkM*fO1*B6PcaW}ll{mLd^T4O{acSC zgK2mi*89IAW4ksqSj>i771$>E&)6)N3|Bz~F6@r&Ojl>a!EQD$mTU=l8ufOGZnrX> z&)Wm~p{QpL{8od6^p__asK+sKI0rK0tLl^sHUu4QT~v{{HZpXm5vU~HfcIBtQKa4V z56iQ)Ld1Z;ySkt@xhC&m)xNOF574Sj&|o3wa|)H~=?{9`4gw>Va&1jkG7>Y8YQ<*Z zF1x=*x_0Y*&$(hZXaVwzRAQy*zY^wl3X85B~B+Ma~dByba_Ca|zSB@EFnr68kcq z=g;g2bRcc%0IUI~(LR%)lQNo;dw1?O`ZvN6afTpy9oZy`!%7$5? zpC+eT?8dGp$JQ;vmPTAx5)cW#0WH$>!Bb=;sG%b^BUB&{^PrP@MXqUxQ2GsxGZSma z0COQ_7@B_nQbh#XOnCl_^{kNg=F?vFKleg(|GI?<;`b8Sh2c#AC&yXt0wm>^OU@%T3p!} z-6sPe<`P)nioYvM%gRuRP-3^oLIw4W7T@_%!oLLLRG)R}6Gr>Z)x@LK z(u~u+SX&8<(Cc&D{3t@*7s7wrMYU+sLUK+kpx%l-jB<01Ud{{GALX%8B;GFrt4Dr6 zEn%8&+^)M{J7rCNn4;@%DZ5`ZTCWZT#kGim>`+Dsb&GNuPZ3{IQdi@p=M+mpWT?>8 z$tt>BMhlQB^%DmAC0sh-Vg9>A|f21{7Tg8>8o z(k9yl#f-!-Q-ekJH?QN{*x_9O$mEaFWpM7QWR~Gjk`07lBsyhGO=&rZ&AgWsM_W-; z-vZ_kWrSk3t!%ZEv2V^NRb3-HqTj2iL1c{kb}ZIhLJLU+Dma230|;MyOWbO5T%dCU zMkkp4WqKX+7#~N){!lPR&U0@~E)?hm-tNAX2GD1A$0%MOIAc50xhq)hYfm-5m>Ww`_Ce|XD<@i!)9EXx}0N($@>i0^|7N46fr0HNK z*%S%#bxv2!FMjGNgF=Q+>p?Pk9VOLaCt6|MO5&fVczs7TQnec>^-5)D9D8UP5%h%i{*7@zt!rbS>MZoaCK=aD#6XMwxE>;im@BM4h! zsk{WInmC6fFID{~b~US`CiEFvZy}gTX&DNuOdlLn9H=FVwiPMp*M1izdevkAeGklS zt_IIf&BuD9=u7UtM}{$J5oZu419unWOySl5!5cuS3=tG%(R-i#NAUxM{@LMP-zk_d zTGyUSeWm4!a=r&OhGK>d(x>NY69f_~>Uvrna(&YDifdN}zxRu&IzWNS#Wc%>zCV1Z z(*9Mf^(pB*D?~=HXe{#k7^6B%$(G~%53)?nn*)^$8y{5xi&EvODf5Q_<|17dSX5E` zHX(QS$*~P-iAwhE@t?&^SnlX>SYlaY_`7C)RZQMGR%fff;uK!nLs`@hETD2X)Bg`E%|NX-uU@xZnE zjp!)fIKF$++n^VXi<<%tk!Hz(>QgSP=?iRGTucCQ%@Zh+|597C_;o2@NSnlnWmJ=ex;Eb>Q0>vO<>+NpAcwN2Z;~i-k@}vKO-bCs9E2VViADDdb zP0`WOcqqgL6p(9FiZBXE&uErYO&tKkcblV$xBIzz6q-wDB2>PohsEU0wmZymPLZHf zID5J3DCLmc{sJ3XPNyhj_sUQ?qBnYnV2!{(vL9Kei0_tx28Tco>EU-w6px zf-%U>(pBL_?NbK#hLZf=efT=OHYQtIrl6&k!}$*VU@(6P)8!|w|3hei(L;(W1X*5P#new}*=m{+=JAsJC;#00QL zl3VdvS%|^&uqa}wy4ZS{wJXqR7#%f%cu_-;C~?CVU~Ng7OR@Myb%s_1ArhHFIT8R- ziK5An!>tv39ix%eBsUHlI(k2z4KzWi)J+)d&?J}4x5w%~7i8Jqsp*a8(roRb%CIfC!hU54t8?UcI zNn$-pTb3C<2n$!64G1{XTvt|BUJ$H{d#;BQ-IS`~nLgvZqkzbC)t-JGr4WMO(-KaE zXYC$+T`Eu7x_6F<+JY!t5vd^VrMUJAe-~BfdF>L;E1cYY9;!$K@q6``XB2>Q-{*>4 z>;H-n7uYYE9RH^stt9MDJxK1?&y%QfK!x#{rKuvGyUGTC zFEgU;5X@M@zHe-pWEgKD{wIXBr?)pyWZVxbUq>8Qh+zz%;?%!E=)bU4bPDG8_VrDs zJZ~|-2xG0ywL>Wd0P2nSAJA1m(-B?Aq33{$DMt-j z?rHCrun#VQ{{eN@WS!geI%s6Df53NnE=AdHXGS8a#Zvg+(B6!R{l4Db*uf8DLL*sE z#PWJeX35j`)PMa~xSI(AxO-n1B-Qt*enEe{$v2gvVkR2l2fK8r$mVZAKPx*M6#QHA z*H>3B3zlFriT~ZICro^se$B_~-G4e%w8+%lytjZAU`^^9`5FPo1cf6fCnx#IaTj^N z{&}Ju`iz~+Lm$gRm}%ojx`btbGXZ2D9AT|lg)$B%cc|84wj zuS@+w#Ho{7sShWsx*BHKqwttSq0`t)`F*bRe=K`YzGkh^)cK~tO3B8iKyTjjSMdcQ zF3X!PsH}3E{;Y$+b9wm*tUUNA>+{!r*!lVL-Eor(D3Qk)zK8y9W*Yyh!Mrcz`A#lg zJ|(GL;1kf9N%xvi!1v}7(5EcS&D(nP8$HLj=cdWI9Z5kg!g}0i`<0BmjBkC}cF_2m zm;mZF7WX5|uuihiPcI9re*oxx&E4#c>WFeRL(^XGKH8Z=slbI^pZZj!Aq42V9}J9(QdhBP8;gXiIqqmy}$d*SvMn(aaP3`16_zi;7WhkFP0q@!W$ zNK&fYq#N>eLit^8Ysuy#^ulhLM6R|imzO{3kGSi}-db9QM0}#)8zL0wHmhVh!@-TQ z12d6lU#jVo+Eo0lprCJCXy#E|n)euMX6KJ&f|lu8p(9ZKO{z8w82>IV(6`GUZj6tw zSL&E64K9xI_t#$wjKb!LT{=8QCHn! z6jo(T(}|<)9gUq;!(QzbJop5gVbc}!eCS7)2V=xmwo~;qj``ZQlJpHXz4k}%Kt(Rz zX2#f6B?ha+Xk{H#4k;kiN8)1va3bqRfX*R!ad+71~@!Cd8^^} zHDjrDw?WGCCRH)#C;p@x;)Q@^>3L>G#zSFfbEmnvd3H{Y+`!0Fo8LS>TKqSw=P}_HrxvpCNCFtzZ-T zM<%xR441yTLd*x%aulnqvh9wtwe;4aWjy=^avr#U7nXG>iGEM_w&iYKPIYt`wJ)5R zlsZ(BkO2BHh^Yz(>iac80z*IsN~zP|-kIq9s}3&|!6k$?cmT?f&pr_NJv^3({Hm`k zm`gK8uno2P{Fa!b`FM5`IY=ONA4N$hP5-S%zw&2 zSNV!rd4~P?{;)eSH}P`wvO}bSzk7Q(+Q7&F1bxrZce@|)vhh%E<#x6$0b|vh3~6W( zDtbOft}M&Q;!#f>Ta0W4<2<4|ExMVF_$q5_TUC;lJY=NfMn;z%+(v^5Py!L%^|=vX zg-a_boT0DYWBxkU42Cd*N+2fR0xCXz2u&KN36e#tEoK}>uXJ{VL@45nzYgGFWKF_z zu&&sP!77y_l%iriGPoH9a7Mj+FvbVUupe(n-tWlnpqqdjpFf-yj27ISoE#_~{~{oh zt6K*>fcs9>85Z2+#gNYu4Ey`N zq^sZJ0gN(}0%RqKlSRD&_8Q;^qaV=9Nvyy9BJAT?;)=cGxweNUD;n0ejRw2@5dK~* zXd+~hBqW?AzLZ6_N=uIre2sDxyp4mJrj#}w+;{Wr56cJlx;WVLcGbx$iO-ir5?pr* z>sAo?OIPzn;fo*^)z6^69HfBV9Sj7JiMVT1fYbHX9Ko*@If}e#ul*VHOrSq#}nD7=vmA z>L2c>B=OQGWM=`CTzq^Q$BnZvqa{|=-mt7qNhX=pSl0V}%U3$R1tdaNag5MtY)WqvB zR%PpuvY-f~sa?6Q zUG(NRt0jE2g`SMYQ*2Yyvvbl5b28Hl_mRCAS}Pk;gjs^{FuD*_^UmXAmg1&!b4E!y zclX!g$FpfQ?$BS){jMpjXqn#Tve08Wb*_a*D#%yNn4(Y-(L}x%&JM#v@BU^-mS! z$!f}=dM4nzDC)~ADHCl>-05_&ueW-BeYTUz=hou?@cZ}g`piV|t=XFYVhZKg>AQ5` zsjm`aI5{z*Q;Es)GDARJ*Ifuy)UG4Q1kVBd7a17|xC?R+)>d{#Od!yL5gO{VvJmhx z4Vrz2ut(}0x5t=vKM;e`#W7e1UNKEVM#({3Do9)5n3FOmqOV+6t;65XIw z7()_7X#yCSap)8RE}MXy-~a*oA&u9s7jbB7M0Y$7n~CG%<6qT&&&txRcf+L0*Cn{Ll_Yy`eV9?m)SU^m0LF?>J` z8lxy}V62OBsY}?Oynwb$uDfZB_qqG>a$yyXcnG&kg0WVUH@|Le!<@g2XfLn*d`GiG z8OqOQln(Zz-%j0NC4qV>$K#hsxuV}ic@mk4##vl;r#NkN>s)&B`E~hD_hzfxqi7uN z?`{EwM$qGU6*#R^kI@UX)?adlz@8>@n#ilEAwNE;0`jGxvB^m}1V$oD+F@Ee%;T4e zM}T32Al`hn5WX{0YJx0ox6qWDn#$%CqZH$92$U$@8-Q?vLy?J21)>csX2<{@(bB$W zEp9$ec;!q83f07rUKry11jE|)wp9>?kUP$-?03=3&6SoF5K%l1xXuC`FyWaa>rU!r zNhK8(72s1q%1YY`L@L)9L*pz&lAl|Eu*Drz(6jUM;3GXGWMpJQu8HB`RWXwgRHftJ zH!vJoc_LBw-p_Y+z4Xvn8LBIaGbR!7ZIIMyaOXVM($y_+3JiAzQ%uB61S49>JRJpN zvYXs;5gzkj^;ZT{g4!00`GkI7bKayzw+O^Hp>BIKm4Gb~pO9cujA0o-39>t4AW3?; z>JzA6DswReRtRkjp#02(-EzDJ1&^iS-9g%E15A69&N~`YovHcPq^hf^1VCeyvWDG_ zfB!ycP9SIJM@TX~UhOF>BnVG$1RBl`CS@#{@{$r|CZc#H71ofv|6#OJx+0|4;OEu{ zQqiX$T-1T0bMOSk>XCbFZ0yjC=+x=K!aZQ&S5{WyaDl@gKxgOwno)0HAWl>Huku?;U7SJn=zW8q-34mhY^|`E6i>qbJ$2_OfSPajgSTx<}Pu>`BDi*s0IzmbzJUl$wq|vPg zr!};50KTh#07e9fYDft0ncJ?WG*r6(*)qW_bx;tH{&-e8Hb2uwvXHC*I?Zh^9+NJN zU1m16GZj{vmk})=Ik{hSH4qXIbcY_k13Fcdqi?Tzq0rld&b8B>$pbn}lIwc&{>K0S zxC+AfoYGw_$Dow9>lv>CXl!DCmPF%8>0FafeO=u(;OX4WrT|hCZ1-Q=M*|A~CYSDc z)5pP5WwElO#a8$fncJQ5?8Nu%bsi_#Tdy*VZu;D;sB5Zz)|psXo&ZB2q4V*<#v2y41);?1TG#`R^Uvh zit_RtEmaZ15It9bzeEXg^4E^;-kB(Tzl!4nM-~_j{w*&Xf7j%(nR;s2@*or}SlQrg z&4_T>-v;bj1n8@g8awuZ`fW_KGr%$cUr;*CHagM-W)u|2S6%GQZVX9wUmmT%sLgFFK+`Q-nw6DhYGEO_3V(R#ukBc4oDF)7elQ&_@yfT22Jv+DRM16uzQWexQcfXdD_I&cetT zz4sw}%MA9aY}m!)7l=h4TRe zGY7}dkF=k-z+%4dB$n3zGCF49V80oJ!Bs8{DE)><2JvVhNj)P`@I)s(f6V94H0TmL zgwQZTmJ~$>(>}0PgvbYhfoz=dW-8NP0?VvXpwkKXLah&nU{5H@%ez>HeXo@IuA>9y z5efv0ATQvxnOd~GvfBi_O4SmJ!L-V+Spq1a7XanKCP7I)SR`$_^(;P*mVocOWpz0N z(N8EK*2F(=zTRdzDAqVUh2Q56u7dj=0FUgjNI4!LOp z-Zhv23WP+57tQx%@5J7%)jI#NkyBEoqVQj*n}L16sN@c8RHjjWyr(;J3D9R@^cTtO z%)i#1dK?~NVq#uL4-F}wMLPS_?A@~z-wjxC?eep}hLH@^sDPyVjaBPL`880dx6f!4 zXaf^v)$hB$2Aml%-JVhJ?qR!P8Oh|BmJ(sUZTvd^r8^$G-Xhj700gUAG8 zQ+GPKnpO4-SPwB&9!>&e-4C>n+%5O?paT{jq}$A@v@|pT#>035a#(8{8_9~*)m3<8 ztiBX}e7wx6Dxx1qUO*2V+E}CZ^-c5)z#DKiH8hZW9$p0`!Z@q~R*dM&lx-lL1rk?z^^I^UrA#A#Pkj7N{LTftH7t(H`1WdjT%5Y$;{a;29Ai;S1jzO;=rD#_-FKm%x$%a4M>Z!=nGU z_`#bzO^|Mh`!f<%gaJZvz2}*gb^B}27d3}=_hS|PgBtvwzFR!~WYW{$AHNk73A;HE z+-6S7!wUAIJd|R2ph?61?S<^WA{8+i8p|wTz|!r#ug~|;Q@>GoRTqwaeUb7PBlsoy?s(D=mX9n5_!K?*=jo4o$~}v* zh&c2kqCp!7to24gV9WE9mYVA=PfT!pS;n{?5r$(FoF)0F9~hK&^Zo`dsVkC3kADh8 z;q6qweRL^=MFL`(Fqmb)QiHSrGLa#?g(PKAX~b)wn{Ecxwe{ez4>)*~71^3Yv_Al& zcF{;7sp({6@Y{!iyu6q*mpZ;)g2<$vaV8R*l*}nDM9z!|SaMH0EK-}w3aJm5Ztrk!Vk3q8tP>TIsupXsX>$JAq8QwVa zgz&1x`LC`K?oVAnA)WPf#hK_5y`qs1-N^~~IMOwz+MzqSq~kq?<#6NH&QFgx;ik!- znW7x~M}zDz;xV_Zur#m`FI+aGNkhyjP?rM;NFddx?rBoiQKGxz*Ep_DHE|eao*M&X zz(l616JvD-!s8-OzrV;UDth^ldiFTrheomvQdG~imAJ~{ZZ~1CbW`&=>f61OH|ggh zxw@j1;C(cP*IJE!fSf4qO-?i>Em~99;d|3^_!gWv)+a#03=QWlJ5ZZf+b}tK!0~_EiD~xuBmQ; z8RFwu{VZRMtY?bBdFwqsb%z{}jw0GAK7$SOH1gJAh~!+gh)2z`r7ag!I0M9XxU;~9 z*8SC)!hv#GbYA8FDGM=eC*##nHz}XPgDCedYz7dZZw?`$Jx-<<2?J0W3!fGE|8QmM zqwdkR=LhZILTWw~k)|0|rcLO@Rr_2#K&4S%JQ;m)4ia9XW)1_L-Cy+@POd<(chG#_ zl$VE!T_|G&L<2wx-pL-qTYRqHnZZPqE0{~~p)Mti>wdDXaQkBvO@);@Q!!0wYiPcG z?_9b3oCkeL`4x8O)M+P%(()xTUtIw4qKZS9kNH70@aD)W{*1bfjdnGYVEa2)#}Zx~slPmF9A`C^KOd);DKo zn}aPa+O&j8VJg24L-gnhHGS>x-BAkCxwGjlke9DU;5aGfs6B(%b2(gW z1$zb9J>}gpHVJFfIJjq3E?cTux+*FeQBi6O4IU@JgVk7d<8=#}glrATn*64C&F=qC z;w4_%Vto%180k->0q!l$FzQMf@JT4|XJ)i5A9-jzaOJd~C;&#d*Ay4Hez5#-ZO4DS z!Ln6GTg4Oty)@X1i!@O1P_AIE#T1^Q6UnV-q#9se!0Lo(j4H-jh@@Neu7wyMRW(F{ z=TTlQRVrgYaJDl^6%2&0q)9eU->~xKHJnEGEy_xn7ig6u_0j@>e1P%9MASye^Au&f zyJovW-17s-wci*RL|Mzr%f~q|QIIlf%aj^-hGk{ZZsO+(ck^_Mo1hSel3j*EK5Qqedpm(`|bOvWSy)0r1VGRbROFzS^P9@^IDB`RqDrcMG_!9 zr-A5a0HwLWa}lg@s*`GxUn^RF>|w5;5V?8*`KUI-oI~T0p0&^ zPJ}F|I*o40D=6%FCJDOj$|x#A?8=BA{*yDkX8qL-y!P?Q34>yarmE@)TQ~w(L4?G_ zJH!N=`v2rSAXxzE?B5BHhgjGefD9*!+HR;J;`=@@j>DhC+vGKj{sOf1T(;~W_J=v& zwC~uz>@EO6ATUg=*W_KBivIW-@K!AW2BfF-GbH`^D5}8W-QA7HsI|O|3?n=HzHApQ zVmkJ>f8P-x&gwR}t0?F#zwaw>0)PO|6Nn-U1a#F=Rmhqk)K39i9^pHHcX&uiA-aTb zaf{m*=u{W6kbEZarcVb9lz%0NFFOM_PGQZWXxP{azka1;XDpX_}tJ2T(3b9e2Ksz5YJiI^&Akw1$Ty$A*fWb-Wyb|?@ zQyggj1mA&z4K8L+m=3R=5=pEfu+_kuDap%|f~+;lc(^#xrjESg$Wvi(Imf%9kadCe zIS-GgP}==<@%^*Lm)|u9(q6DYRRvQOzr-U_({gYq$-bHpJWMJiEd&I_+I!dZvpqlM z+6VvII$*BB2L4EJWc33c7pR)bysw;W;t~>|7)1bx@&?f}NK6;msl-jo)OSZ1sDBn| zm1mcf3}q->GoP%9Hv)ZxJFwj${#`#Ud@N8N@fp{wgOG7Y4cgNqb_3%1?(HD(0Kv^@ zO|aep0&Iq;X|XWwZCFp&IH5xE4DR+y?V+OVtH_>17XB824tvUz3j zqN~YCK^JDkZ8|P4wRDtEQUYaUj~|Db08xB~`t^H}&Lxc^UCyEdn;8>0dlq!mH|pvF z&o`{=(w>#Q02F?H)0&Te#t6u*fXwtFJpGtcuSv^HlmH4M-5b!chxHSLFr{(kv?1C^ zMmIGVFsPY3Yn&=3ziQqT{4PKd?*y194+Opx{STyYM=1B|RVV^I9UXw97(1P=Pv1&I zZ_b@U3K(*SDA1L{OhCrP4Fr3r9;cfz5<2|DHBIGtyd0qU*YtrIk*5l4*IC?BLi6P^ zIw}Bh^HfT6$Gznz-}C4fZ)x4cMQt7TGs6S&EQqFwzGQ?sf~2B`Jw2wbEB;#}Qs=V8 zFAJ#>CdglC@-~AJ{ds0nMk_wCeI*&LctLBkvC20KAA<1h#pp&5IL2jAqTQBeYOW8a z7r(RvEegvJOK>=PzW#QK+w|K-X<1q%vp5y{_~IF$DpWsM;JcnIg~wV z?MWoEW>N?y*jg$I-5-r#Y-&t0tN=>dMK^v(FK4FK& z2jbaE>Fful5j%o@w`Y@OEv2&CFjE-21ROMgH$u)+KdQ)iUvK4reqbPRlg5e(q3AT$ z#hy9!6#StnpJ1;}q8!%g9yw3c7?9c02_X8%UKhL5G>!aDdI{`$u^Kuut*}QcU%ENz z^YRh|uV$^*u=(u143bgu_HAVr5spE4#&V2~#J^JT2cZX$`xpC;&6q~Eb_F1;^gUb@ zyOe!}Fe0Uo&KMDfH~?m07zknDg-?OxRKKwP=h4ntYD`%{cY%^E=pToNISnjq*T*~v z2vk6qst{9D;a?jjq+ruz?}$|!lwLAXjIJdka#H{+_2Wu2`jajzEL9<=0?3;(D;}To zy1YFLi}aHXli}jutS=nPEGwy}2;>qO&K05<1b9NpB;5X~LM8lFQ2|`Se05F0Fyc(R z9=C*?KcGOf_~F-RsJlSo70I#%q-_@yg*y8EJ-r{9_=vWuEvU#x&%V~9$YxirB}ZPJ ze(tHOs{`I8{mV6&L1q(A#Ps-YJ_O$u!Q!Ku0&rNbwLMlCpx+q$7EYZCuRS5|pr=QQ zs>KucUaS-U)g7+cR6AN>@oNJ>p^Ix8zTwTNJ@yGUb8{xHVs!%_V-;S#k^ub#*d@cq zUxEq{5;J7~`)q>!{_{Pp3w{(rboV`Mcp-9ZNoWHi`s1k=1c#us)tB7-zzI?5AvrcO z0G=C^OA)wQ^E4?nD78y)xh|E4OZ{$a%#6&&Lq8(PCbYnbBLePAXvC)|1$ol*f%<3` zSbrpJs*Yy>fc>)gnzapD+;BaUoB^t(OG_$XcwWoP-x)8_qyJUqwEGHOJMJjs3c;cN zyTbXD1!R>t@07D<3Gpe&k{3gjA|k5h6;2pL#-!wRjpeHDqqLs6ZadJHW=CBNAz-%RS3wrb^D3K}8vP$`g`8c9wk03C<~}ES z{X&YaxRE-bU~oK8poiN?=|8LGLXP{E!f!Fpf6#X_(06hL&oAnce(>8{pn=Q6_vuSX z5AQR3D^`o2e7lK3!B+_5xj5kls0yxY@L+ebvM&K^Np*qKjq?7BNC*5JG^g z9iBy%e8W$B*zdHN=%1Q;h~^GyUwBs9j!*t@*r_HdAT{j}DG}2CcK0-hSG1D!+~_BR z(8Hp&5&}VyCB$g9d+gntAY#!f((T9ufgJgJLNVG0v&XGWALV`x<=lr+;`!V1lX5Aq z#)%-+^Yrb#cix?LZj}=;vnTz1$uYwc_!OaT?HvOh$MGJE+(H0TaNuCs{KWD_o%caY zyzAe8cX;j*Lov^K+PFD6!C1UuY9CPWE^x)NQ~VL~!g0~6)WCjFlaETVAAXSV`0;23 zI|E-!HR(D7TbaW${%@mz?K8C!w)_xQ2ELe<?JhXlAHV3^rS?!=PI{}Y5h|x z_*xUIo0An92r<#T!E`Y#PKob@2!pIi(0HxVpoUi`Q_f;F@x2)ixl}r6P+p@!#3k*! zhd5Z=JccYqWly}AQ4$>3nhay?ygn#?!P*U_rA6xUUi(B-y>`)|fVw2mrK!*P2#b4- z&rVMff`1)R>dlxb!2XjHLd@qBy%r>F!s^Gua2fTH?z6+wf+4SESxn^A#} zv{gb^9;}?OBk^?7Zc;iPbvubugt965`g&lP@@GUfK%Ug7Ga{9x3`y%lt!w{Wb3|MC z=@%AcEyYcm!ZlZKe%L8mY#TMz>IAUk)S>+QpC5f-WjKu)JAPn~1^Z4F8t&l?`xPhc zrgXtSr+#1aTN)GtWA3)B{bJD(0~><&-5W?KDeQN>A@45xS1G{|v&wJkYM->g{$8f# zJ7W6hde5NmzVyRRmNC2XW8+GF4}NX9+?y?MRe}B!<$Um;k3uWyZi06Q=&C)iBR91M z&Zl-RBhT{ID4}~))t4JO7i2=Z&X7Q=K z79Oc-ZT8bIcO-Iq`)as4g-yghYWe>6UE3J!pxFAe&j)xehxJNT>&6L*(FhQ(bIG3h zZ;)~>1xc*3xo7KdTpv%VyOO5_>*Wfgvc1DGt>re|8>wygAuZZ_=8P0e2Ah@-3`iJ) zMUEFi{+&69;0to-Y%KvrwhzDCv?C=n^pYT*d#R>NYadF&2JUS+X4ZJbF_LKdQ%&@m zBq{UN#7dM?HJ)Hgd-Phzqp(N{jp1(+Kg`gz6oq8p^nS7??f8TX4?f86Cy3J`U#g~! z^Mg+vGZYA~u!fqE1p9otov0x`ZKRV^!DJQ)Who&`dKt0sH&5KY8g)Av$MLkr!g~jj zF`#?}8zJ%aoml(?xQMFE_urhpnTQ1{7GO=#Tk1KUL z9IrVPRRQj)%NL#Qr-&U`rhV7R@`as|8%WRv(jUrf#g7F1^+HT~2rsI9y?t9=%Wjv_!z-aayZ6Obs7URsi!0R-sSv2_$WhEamv~pbsT?px~Go_qp zkcRI~pOJp_L$07BEiG;8kZd!*m zHSW9R=!UV%bYrh1KVQ-AS$!&Ltl8<^1<5bl8?Y*jX*9b}7eWrgpths>t?z zmmZhju36s^13o|AD6>ItK#U#XjF27G&kDjITlN7IqdUE!KpeMWN0jh+#CZNS)a2k7 zt(^8}$Gg1|czGN9_l``?V~!g#^6wj%7^ZGD%>7s15*`+rb32 zrGTv9o;s0w_{34|-$5^PNDZR}gYnf~)}K}Gwd->U+gDARW6Yob-Z#=91>F@tDO^2o zE9tuM#0ldH48P{U300eM)W7=nNMw*SgO8>r_Z}{WZ;cnF-R*H2U(j;IV&)yXkL3$uDsfI7swP%fJsl<8zc*qcgF|2r#gu5UkFTXgv^d zR=hU}wYc)4cx)3l#>7j7*)Y7=K^316QTkzi?yHY(Y%ph-OuIVq@t!h);$Nm z!}q8*j`brdhh>?s<0*c#x_dnS?Py)%j0{D4J-4mvOz&k8Si4?geHqs(H*3i_`lq%!A#sSUt zw^hs{&Zq^)?N(KUWz(JQa!aq)(VLb-;ufeeZwhx3b| z^|_VD6O%UvB?)$7&E_7$EIY=#CrM?S98-HCahe-9Bro!@^BPP$N`*X&NrH8RHOd=I zK&RUxQ+5)KvB!NwtRmmF(k2#!I&%s}e9fnhxu3QlaHOpV#qk|@zm#@MUW*qrgs`v{ zquiL@ob}3okBeu#%Ra}2mdBMj&Ssd79sgY1JNjNSgvzwnAk|4jQ{R7#xy3vC%g|%E zPv=3vRsmjMubnztzY27)RN|{{ZGlipLsDb*-9<2IOc{Z_X|>iJdKH|y;I;de&&YB5 zVcxwwo2-V&y6OD=@fb-TFU|P|J&~g0OWBli%}g4Gs^5h+!CGm2!3DRxDec2dg+8&$ zZ87+Rv&uD`0?w9abX^Yj(oWqoexv$=%yeCJ1T-`ZY1Lscw>x_UiCT@WywB^|i;I&! ze5v@%@%{ly@MFIqbpzuF^rtB_PwEZHRGnrynM&suHEI(O_AID34!7lG%9iyK+gq8A zs@BGvI=UNOWmInxN%^M`cMOGsKdbfDvr}LMCpqbsuu}|B?Syf6eh8CL?e&VF%jDnp zhQda3|9Z?#QPv|WVFTkn9Q+!_g&!^K4&GL_tB!c3uP=^^PTZZWJDB&HS<=++2@zSG zr}GALsg3Z5t~8ozq=^^ zxF@{X?RFLEW6_YUZkanPY0he%Q2|r1EFuemiq08-o8*pq&lch1NljLj&vJZLurv8% z?cw;|FBXn8<6VtLaA8*wh=OS?Mh6>@r=GL^>F>R*693Yw&nJdItZG%Wed;xSe_YOu z)4Z5SQUI51&<-xPPQ!R|a~F8yk~lb3x7f-t#&+GXLl|Mp-ZZ>=F{r&=5XX1%>SjKd zoN=7)=Qsr0VAA`W&oS3+eI~?$@i5H%Dj->jJ;=kgQAwfMp-YQtR=MlD*L<(ooH4jI z`!@0=H4RyXMKVwR?{wepbYJ{90iG)s{}%iK*;f}FyzlebC~AxNeeR~r)^?Ay6B*FG zvE;r4KP%culM2u)!D`?qO~zZP6<>JtZ^O16*DUd9!fP<3om~?dW*lD){(c?+GmBr; zM#__~miC(~l$7v8`(TFFHQ^|I#|2jgDLYI=+rq3srgm6Hh* zXWOpLrTu%`-Po0p{?ztbres_4GlZWFXmL3WG$9)f3nNuj_+l~1dGgTNZ-3l22qUY=-`Ytt#cA#Ziz<0Z&oMki>FE-Z_XX=! z<#|f~aV1?Peo;F$bZ`3M+WZ-9L#PyF)+w#_=%g1FyqS7!H*+o4pd#0)%~jLfW8P1P z^$OcOsu(A&{TXyxAe_y>`pr_%PRW3i`JURxSfN)TlrKw%oX(3r#5P;&O@qITIH4a@ zm|fawBN&ygK~ht@z38czYzrz=^hzX;(IFG^HCW+n)uz^pI!=Mu`E1pOwu;tHf%s51 zV&AL}pNeoAH0^5_SJqupQWm?Az)w2bpWllfxZ~YS8Sz8$UAHHsSlBb@bfefH;R3h` zij{QoMxWPgw6-;{^}dDEqc~5Q@tYv4)~|Z_7ah^|8KPH;tQeVI4SH)+p{oZ- zR}815Z0ab&BPIqBnrTL(Q!IUR%oV98`XLCdah;qTAK5P2=Qu^q1G(+ktk!kXv7evJ z)qSD-$Y}|DpZ4>+Ag7g|Iz{o$LT>{d*un+kxvp%)g7Gw4dl&92?r$}`yyQ%TO@iw@ zMPy8J&fO4_M>Ur;8Y^)IkN>ZoB5izT`dwYf&6+v# z#UEx;alCI=gn6!~GZ(w!TNAVxZ1REdz z7k}8+ct5@?xe=~IIf$)iMs_&-(!HO!MtcP1vl@$#z0T|fD#m@2?UTmAR}m%TH#T-F z0^RBw)XfEZrmULn5b>@}3v@_MKk8V?fK?Zq;Cicx%`4sig^PVdwz0hQ{gCzq(co3% z<#h6=wBP5$8&#M-Ugw)#T6y=bG|bESeqrO zFT0rBR?6wD1aMMaSxo78i#jru^5e>e$kHDiQ*zZ`4a$YmQX9*E3R+#3Nkn|B&r5*dUWi|D;B3w3Cj}wddcs5 zL@XHBwszuA-5~~6=7Y>XU$RmZ@1F2(>FDEL_FV47f01QkQMpmrqk&j3nXObw^yIEZ zwa%`{8BwVkx4qv*KoM#`4}@zD)1IzZaTRnvE$0p!nZyVWW}7Us+6%JR`mq<-Mp|t9f{^QAkD}`52i!X z(ZedqVHW0(X%)QK$M{%YicWR<*Ks2mOa&7}b{*{RWgb-3ks2z+WARf}R^iz8n*DW_ zEViqOVzN{1--k>5N*vABwJRB4d*JoMO%Q7+PfyZgDA?e4uoh=9+ zB;c1GR!8tfWf1jL9ytzbIzf7LNZ>CWtRRcQ9g>!TQylnTE=P^rhVLNo!^1u=rb4@h z*aT>B5$lUdLq5$jPeOD}{E65g{=YP^5EwX6jF7`565YfKAJi9;qA1Q&4u0BTj1g#M zLdD#za-!tBvfUMO%!scgG<+8i-`ZoQTk-${Rq>T2>|<%G&qc{ehqR3>HJY0mKhBdB z)(e~|g*8zdhq5^dLtfYsJx~$iDNRb9(#VW}2`f@Pa!stXWLq{pTBPfbNs~u>Oaipv3=J$v>5DT%hR5)Qy@e0KdP5mZo1~eX@ct^O z;r>z_zP_9^1p5Hy`q3YwjyiQRmv}(~2#?S|U0@h(ggqP6QxZ)3v%E`k!W>wqww?T6 zya_V-YqL<3qW{@agiD%#?H81vfraeIQ)*ItM1Sqqwo=1<=)?2ZxStUJY|(!%aejD@ zZ2I881H-Tvt^Era{vY2x_V@cBxj1?L3v8>W6E#APUCv;pVn()>UwV3$65gPe7! zsrm&n;}Fq{+k4S9l0i-F-%mc9Xr*DsF4|!2R?560xC+f;oDMy`_$)8t{rcC!-P!3v z&Qj>-?bF2zKcU+->|f2Jqw}+)2Wj`Ww6;&%i=lRxzo8fVHzLjV=ciZo`@X*otfYFz zWTJ~cx7B5QHyD<1^fZopw!z@1=;thKT5Q9uzDxAjBzzDM*bTmKW^JWLZYgcf`})L= z8iwt&ajuGAy!tNkYj&Yt1Z~UUZmjY4WNTIjbJ;1*!$BpQ>hxCaLki8xn}8Q;C5^;0c~2XaHy~M?YS{6-o|#98Ls%hI z2o*jPwhVn<*(0VYU0g*NImek%ichpKend;ro~<88(|w9@({y~r*f9d&q&ZY zlcdsKU)%fqSUYw;LSt-k{Lmge@kA^vW`L1TTu)SQpmIF$ya=9af;B0;idXhyiRti6 z$x~`FXDYj9(bq9bp@&8u11H3tDXSc4sZ|eh?6y;4gZRqxJ?R23jK8oXk-VhP?5{9b zd$itFK14KN%E3HM_T2x26DN^8xn{|lB_*_r$1pl@wW=ePpU34}s8+_inwf^gE#2pr zi-l$ErYOR#L-tgCKRvupKdCdmT~Lk|wj!d`Up#-n9aJB&o};NrGD4HV+{?O{%52)h zS{_7)bnNovO=ande#dCyW?K@)L%KOa+0~iqr!(D;2x+uLF@n%3Peq-?wcz=w126+# z73YScl%!Wd7b-};=aD`a>CQExL{NAtj+pyyH`19~sO_T+?WuGdg1at_l+}#qQ^rpS zc&wrjNTVvVwqFO!JFbTeSV?3-LPy!Dc`LHdZGyCY@q=hBx#*vFtOOyE@-Gi}A!8z0 zHS_2jCK7}p3>iMd&u>$NW+zI2yc2rGziS!pF`)9p!8DG!;AP&a87HP-r}0|5B|(_L z7kemFLNKU9$PjJxr9p(;p<9Uj_Lqjw?XTH~E9Qyb_kL1k<5mSdq*0}hcpXtv6CS)( z6JnM?_#jQQ5I5YW((P*##FU(=a`nJ2{H4m=2V^8pD#UDFIHp7-mJjty%xx;k>XoGp zolvPZ_vl1RWX>=uDs`VBGk69p&d#o|{>H9UZPA!9U# zPyThQfQ{nLn@tw2KZkDggl^1zItnYJ$C%f06NHcXg$yG;1WGfOzwdvnnoG;4x=S#f_QUFvO9B6Mwu6MYiHY<=c8L=1 zTrFo#eXvi>K#G+etsByV$)1w{Z*=*e^0SOFC5=uKTW##Th={K;XlKUDoK)Y%9UP}O zdmpT0=WyvVuq!Y~9FVSaT*ipF^2Dj03(P#~U_~f7HGiFvt)!OfvT2j>B`#m_Q1lvO zY*yti4!^fkD{JQbnI+rl7}5GJQhdn#qW#OZMlUf-0o5I(yJik8nv zSjy2ME#HajQnOK8WzQBknaQ`tdx~jiRn9Y6ieh6)9Yv2ZJooqJ@{h!!#foIrt=)Z2 z`OAw~ihuUt#D5~T9HK+=Pzi-gZ1LS9&YyLJ?aje^!Rk{s-MxF8)ziUeaRJ z60H>uCfmhL{1*9^uC;`woykZhBbTR3%Tpmg`FVEP_qRoy7qVclC=ECj#Y>#yV1AOi58!t8%87ghc(TA3^KTnAUMAb z7%#yg=H_EdJJ03Zgnv7WZX0>drObPHg0XKImVDC^ET(xet`nfMq6l6m+F(M_OA?an z!#TT4_S=}H;>FY7B|iEjh6*C=Hg_k}!^T!WPJh-v9ewu>DeW2Z-MnLtY2!!s>15~H zrq2d8-?bZg+>o9gFMP&ZxL~Si7Zx1kxD99giU*(7uR(q_l1pVbt{7JIE9ZxL9wOJl zll$(rNt@+kNk;Q@reIOWM_*BVDWeHdA+~BoA|mtU?L29*t`p~R2_RK@;;N|{ZYz6` zLN)EGfpI%eB(yR4OYEhK%fj|NKh2y_aqz=y2em^;xU?9y)WBE>xv9NsdeGN6(#)^H znJc?2JUG>a$f+D}=DdV-t#p!b(3kvzttQT$zQ(u~2o)^bAbuRO-V1E(MBh@xeKKF& zw(y3A@j)bM?>VA*H6yn~9*&2!O9~v1VezZrmCK=g?SSB0C2gl-kvxsw#8hi6Y4}%k z4>xUCue8F|)(6&FR{WnF%x9#OOJqv-ePo8%u`!uXx=o&8C1ggDR-Ymll+2VpkClt` zv>jl4)MfFQkG?n{&TAhlAONX|;U;ZGKAJDAsfGAm%4%Q?fea}D+S{dHg`Oo?b&jnm z#7WN}S9fp4_IY7JC8hNij@=edKQdV1%xI5>?cG%=ocPSQZYo%W?Wk~JcSRA5V_b9!_}_^>0*32}wmj8YGnNMnXhNx+J6pBn0U;=nx4(zDNjCN(s`T zbR(&RARVHlG`!Ekz0W!O{N6jp9pk?Dj>{i=?5})TYt6Z4Jo9-z;rXFj^`{mY8%-kj z#X?hApL0njW-m=#9%`!H%TCm7=DqBmd1E~~&m~{1tg^}a&W(}iO6@0a$zBY6tZq?g z{rR(kl45CVy1*gqcB(HkX#_1(*G)U&Unr%^Vi8}m#gdAW{10ewKJ-0#JWtsh5{r+u zx+PtHo*}(!6|hB7fYR^>_v7z}WgP)9U|eHO=?8!SjDvO(TE)cjx>cvYO@x=kC9( zrondm)N}j)me;hf2+zO0riFP01^)r{`7wc~rRDyLYhO|8`qOi__&mQP4Kc|U((??s zExMI8QoOLvRbpNxzu7977S*yV7xIDp_3oL+giX@xq2KNmCfbp_<{7DoXjupoY`NUr zs;+h6;>Bi`M`5lD-}n|h>IVtk-SWb`uG=41yZK6m&yD^ZE!*)wIXS6axpSZ`2>#Ol z`iZ&hzvJ#d0zmk#uCArb&9TX29oJ4G_&sKA_A4(y_4leqeyQ;UWzCXh9R~QpHM1Kj z(1!4$!Q^Cr#Q$W+Uw1-B4t_o^(4MT*YTvB9yRlk6dHmD8UUTB!=`Rq%cb=@_8qil& z^L5^72~orK)twS0eXEvt$>J21lsrvY;Gbf+$c2;5$15J>U49l464LeqKC($knYdmi zf!`)kqFZnJZV&0g|L5I#j(j1LTo~O96ea)Tqa{$+^Df6Czm0(3=A@-vclAE~a{_eh zyPyQ0=b(W!!^Lz8zhd%mUMTNQJo05Z__9N56!pJ9MmCVHAdgRprTX}A<3+vJ#0TW# zn}!CPXF!r4vR8sn7P$B*tV-7v9boop3vnTh@WMy6BndyrSf0NE_tA(B91yndL8(uG zZMB@-GLIAGa-`5VwTVy2$1wPK6|`Ag@xaxOn60J3CI>i~`3|PYBQ}!@Uv@#7lb2`J zlUY?|@YlD7OB+Jn!43ff9Z1To930IJ<&%CZ{0pG^S-7bWT=q989`hhgneec%jaqI4 z3h~PVws9-Vy(wZNt{vc-vAy_NmU(fsYy~uZc}hS)A}41NNLzK6lxu)qh*eIHv?5o4 z!|4@pV}VFQ4ph_uV4GgaG<7)VXAOcUz_#0CH>sog1(u`(E%z4Ee7PcYuzJB=i-t2ZFHv1?#U|ppEXRR|@OGjJVsNRnnz3y!sshp1GyVD*-0}v;Su96aN z#5uUBsR@wOU#Np=LAEa}5Lh7qC2nqM0l_?kePXUtV3?GW5~<4*3@*10E)7l~q@g(u zB9odjGBQ8_1fOGIzVlcCnFC1C&_Q29h#855*ROs95e3sEKhU=@s(M%~frA1pcnik* z`Z@r5!PmJ|EXvvqTqgJc`CkjX_BwnI*X`w@wp4{8y%b=vPI2f_vlH`)cy45NZN4l2 z_G#RLeC3);qXhtjn-jqo)-(v%0?%;3zLV$Ltg>>@dg2~{xs4{bVKOi@y~)Z-Pgv^* zU58T2#j*0jPrpwGhbr0vctX!|zede~DOs6v1&fx(?S;Y4K~>tNJ8jaWrwXU|-* z;GUIex>{ogq7%S2AQ1+d(m^{Z{8!9ti&cJ_17t96tE$38UjYS#XIeZYd^dAsr{t^} z-_E|>2Sn?$X#hDpx?uDG?D_8JkZLf``E(|One54n;X;$>X8MDq^2(HDG6hb&1n?NH%X)4Wlgp>{nZW6B$wz3RB?m-0B3JgAx-M6T}2Dd^}hWe!|cK z_B`gBO0Vr@z&jIDmNnS8i)7Ihf+ERxLdo4UAjv(i2Cv#E93T`0H-v7nw#pC-7gr!I z!2>h(3K8eLd9R5aRya=9A;Pfn@$@ek_@`GrY(Vw_ELvKMs1yb}0y2escAfNrKtw#x z7U%-+cK>mK!PFs0+OJ(j%Z#}BP)@a_VXpH9JX!cx#fzq9aXMmIKYtsM2*=#znFszq z`EbE+IioV4_ovHAu-kc->94@U0rK81$S|3TKc@+r<4&V$eJ|2u=uLo9%^n1Dby`YI zLUkYkjd)0d$h|%~3bw#aXoN>D=+!)U0K?Y#B!)?dy?I_(j_oEJk4BU~nnp3|%1B3m zc3&aoH-FI zA6Qw<8wG>F@Le4e;kjexk%mg}WMo(8Jny@;NE}4`9+(HALnF7N6C)LJF;&Zd$xOxh zIQToW?*PSl!^F7p7A%4Es(2{pH{>4)8=Vb_)kzzH4ugY(qu>C4D=i$SIcqT0lnAUx z0Wx7k@@BRe(ctia4Mo9=-08A#47q(2V@`;Z!IZM9Dvg-iTftON#%>R=5Q#fWy4V@e zMoCL+0_NNqzn2EAL1W<+`T_v!d4y^%#oB)@xe*o8;hGH%c>C^M^KzSOI#9yv!Px?9 zEpN=doz3;|lD<2vX(4aPbE(m^hboXI+bRvS2fj8_57PW@t(ecq>->&HgFrlvRp|2> z8O9dID-1u{U4*4}r_hsUIB9N$ss@rS5$E>K3s*xBjT7Z39~17FhMhz=e2%Xet0)#(e6{&tNkr%SU9pK%cm4GCdw+S{N~ zoo#{Ts-0epsS!ZzNBM18&VH3V3im>MMv0~Bfvu0IkYE;x{WN%v2}_Mludwq~^f#au z#ZJcJxQ=ON3;d1&(usAGh`9HAUZW~2s&F5m zGV&Q8br#`6WCwGB$u0g6hcf(ICOLX``Ztf6^Z)+%04P2ARi5yuA%o^q{5?&fr2pPk zPxo0EhFbsHXR&|-uKgd|@(U9`eb=$m>4}D(O+3qz1Q!ZvgKah7SKWcCYyRo4&Y@d3 z9L)>Wt54)1kH-9|fkIC_NfrKg_DLWgRKpB1t0a&|P!0~{Q#*#OKgU5CnNfUZQ}`j^ z{A%E;bb6r+|GR)B*#>A0d1I#m%pe3Xz?@QXML+#MHM;cx3KT|CPxex^%V}UuR3}Ol z_bgfdLFNNo$uY1@A5ysq&vAhYrkGZ1`P~xVgWr=U$2)nKBjI^i+Yo^r*C;%+=H}+c zp!1r40x#($a4a$|x*NEB-P+j1|44vqgo%M+&Y;Y82vqh72tWb}W_u$`#qX;@JXkZ9 zXv(2W7ZVcN-tF%0!qX@ND;sO6W^X<@?@mxih>SWYe~gKi+Ks4Q4i(=6n;7|ACZE*ZygdcvNsc-2S7YpTB?q4si$2tkqKZ{pUf%0`z~Rzs{;&iA%UCh777l zN?KYM7!1KHZ}S&AFF{8TNd{`vXcc)f3oGb^;^Jb#Ru^7AUS0^N(nZzxG3J!P=V!qu z<50X28Kf36K(9>b6nOpmHF(bIYiMMM-q=5HDc?SaFE;$NH0NqDOz3(#Aa4Tc6gv0XwQJhgEoT}G?iv-z=Usz%`0Y}+(4>|T zMn)QaJt)a@FlJJl8WcM18I&%fubV zvkP`@dTCp~2ah{w@mz47xUuWjZ2UOw1ya&j6v_uSu%VBA*N29N^ja5Qd8^*BgW>e% zbN`#W^GqApy)(Q)Ht`edm**|fw1pD1#;)6>#Oi9!iph`Qb{u=!TVNgyktu7tU*28c zIYre~_wUfLKC^=MxK^qREYHh8HAh!Z@5?98Ui_kA82#luE+oV;QsS8^AG>a4#=LJ= zNCvTV$KnWUK|S|&lz2HjXWiOMK}tu(Dv9Guy6RSa8N5)Ymnh`cPv2BjRDgr+8F>M_>nUkgT0#VH z=uzSufNsg`qkrM>NZY@%WKWGx)fEP1Iy;?>Cbx?QE11G=<8jpxk&?cDxYQqTUs&hq zzBEUA!EPE8%+X5?$dk2iK&0^)V+ewwGY;?t0~w=rHrRaA*{6_;Jb&7bGYy^E(a{m$ zC^r1q;Cc4P%_dP|^?SUmNS~JEefjcbad8m{xs;`=dc6>`s+s2qovu|J-F>Hb$n*OB z!)Eb=xkL`nP~@;7NNVD^do1T>YrVMgB<(}isO)AQ{%_GMGBqQZI(SG|EHVDnj$3mt zgxMJ6p9+p2u;h__@p+3O5aczm%pHO(Kr1vnJUlfAxpV0~>YKa~AFhYzaNNebTd2v2 z;hjQkz>Yi|VTG=GPhl-Cb5vMZM_b#y;_g<({(%8??~g<6j0_CMetVW%xU;kCGp)bI zD>p$7EbkicY#LlI$G;S96ph1dbtUlYN*CRr7hF*oxng;RE)V+T&Q)HT*y5y@Sli6O zLXR!$Ip&&nVn96R{pap8?*cwNa}4>|AAZ>%%p1JJ<p@vc+AflHy?uHi(&GQ+y48p%g*{uusYtK|7H^$9NgN}1UiG7uG;Fa z`$X?CR?0L|ep%`2^{V9zP)-!Q!D_AgQ@iW-@2du}H{|8r7W+&gcFF6sBnCwodEX43 ziw;4rOd0Jbsswta6DbU^?JcsR{eFvXzBew!y8z29aY<6F_r*>^XOF z=xND?!R+oPY>~O0OIo@lp6O+A0!1U3n8{^cM%NR??Zf)IQ1G56;ha?Z8?;pIChLwM zg|{YMCm;FOvpZYQrj*(5*pc;ZtKH zFrnX7sU?8GPnf}tiL2vSwHijZW}hw-n4KGnS2WJ~NjKlGH(CQk_kl!U48kF@RDsl! z$_MMgQ+-Kr^4FbFPBO5iEM|gequZ470_?UxMK4>SysHlrR7dUYQt_1DhHPe z;UF$AvT7>UBoOGjSzF zMT==*yqo*LkGy;f95{HaAY_3yG+HF?Lgs;7Q(c|sZbJYpEpW!A5`Cyp{}$wxAvOkQ zM_@L>X=4Bm(7raU&pJ>lKUW9Qwnm)f<>m7@qb$oDCyl=gF2*k}(vl5VP za2)$^eu;;Ri;%Vz)SyCMlJfyaxhBuQEt-7=yfi^8zWL0&+MPQq=9)pO^Y?Y4Ssve- zieq5}?sh&rQMyB!mmvMnkSy}rgVc<2k#Ufb$hDKHbWTpPX$`VEJU)lJ4{%VARe%%IwdBr~O~w@X9x^(h90^VhdEw^zId=w&YddPiz|%U~)ZDyn6NYpA*SE#ww5f+&P@ZT|B`wCpU{FFR7gFoG~F>5Tpb zMjm}HFHwzq`uY^|Af8McxCxuIbLZZl(BtL2yFNB`IBjdSLvULB&)Fu znlU+jup1#b`Bv?I+w}o^KBU3&tw}YEB>pkcYVAUONbi%s=FU%^C9&~Hf z)&?ER<_6FmBf5A|01Q4Uo?)9fj`ZW6>9uCG&%5N}xz?7~{ z$zvuv)el5#9O_>VjV28g=A}W{$iLOE#Ffh#-kcsZOZ2F0^;Q!W1DC7*G4yS00*~f8V3h&4 z+@oJaLxY2KFQ@q~HG=p}JPRWn<3Mjf<24&vhCw6wK7U)7J`h`+a;zpt9|>~m>D z5`;We>|`%qy^8X37|M(9@BS#kGo>2IwYX@N@2?9_g8Bn`_at5{=j{)PkNhdDd@jjLQkJ8__Ucf zdwH4ZHj@(^K4^uVuDj~969`0jSxsd!t4&Dbo8F6*czEuII7mwoeWBP#Qb>@yt@b@| zQ{vQe#kRqre z()v##)&5f)WF0<|6blnNf_UPcdOry)I6;wJzP#d{Tg32zrT#2EY!N&yJJ)bQ7!q{A z!6;euFafDa||e#*o5*L9}(W%AqDxZcxHkN`8c!L0<~##&u9Q zc~Q+FKKM$f(Q9-IQ6aFl^PWGLH`LHzuta1dibWPpmItz<4bi!(FMTdHzj+~TkCq>X z^Xhozd%-c#`yY%DS3QKYv1Ct^y^-h8BVXyrmVT>Ft-@SAW*0BLRtO{^k!5!RfM# z!OT(*F9}f1_i5aDMQf;JH$)H+?U+3CnE}>YVNfLkpJDr`R^9_i?AJ<*3H{$pFC5`A zSbz;mTK0EM8jj_Og0W|1KZgEja{KiyGDI78Ay8YrU-_lpG-BXeBj2Qyd~&M2K>$V%O#jiP z++WXy5WQr3{HsKhUoAagqtm;@Z=$1ChQU0&8`a|H(0L)*(K|n;3a%!Y1+JwAopURhz2amjTaT zKY4G}E#qn`Fvqc&v#5ZMh2O9R($~VEwaMI4pV&BF>B1AJsjaK4E47+tCAQU8W{#nY z{S57OAUCaYP*&Q9&soKBU}FjVBO3JIkXgbDL-Mi5;YavE>+b~F*l-+ggSHz8G)}}A zL!@U>xvTs+?^7R1RaTKB`OW%Ai)9i%w{rIr&7x<~;sx&(ryqCeXGQq(xhRybT*8kC1Zo#o~hx=EY(KRWURn zkx-(rBj*Ew2jw*XQUWiw56m3I^p9+BjuJZ*U+8t6b7T8LLTm=o#e!x{=Q?*pzsCti zQ;GD$;g5f^ArO5%{jT9Q!&=SB@qq_Uci7pj0*_!Md6O_*nt%WPrSDbmKQ%3KcrbdI zySe=a3kRl@zMb2w)mM14R}^i;v3uryScQG*%yb`%q)n7r(${53Lz&<$rs|KMMc+H~ zciS4iHT+yPAGBfHkNym@L;K10ed?cuX3eRqg;%|6H3>__B>5qZjf(NW*}TcTV<(i6 zEkV}4G#EV_iI^Rl6n`dSrN`w4ZP=PpB<-#RX?MkE1M(XO=R$_bA7AY1>=b)2jgfc5 z_|u07jr8wGtaer__kPcYcL@XrgKtWuL}o&-ATn~^XV~yXahO|}ky%m`W>YkscyU=x z>@jnhsk?X+{(#TUgnPjpJP`5hf}P_;m8ClXa+ay^p38p}%_vw)5&#E?E~5DN{wIE2 zwD$C=yg3$otW5@x3rwplrZ+!^<|S~3g_Vf(;8X=+4>~D1xcfa=C6^vKs{=}#Hhocg zxQb6zv|q(Wn7wWfBBXV2IZR0SaR)S#id{jT5<5KM-D{A!-26E9dH>!3nD9h^s-iIz zJJe);G^}%$df>(Nt-6hF7-e8C>h`fYWzgJ@3O2GUTNot!vT7&rc5-nmdyt2O_TxR` zh>!lo=;acg^5us*c%QNtx+}pRXI?n`XuXxT)1_uLsv$Jckov1g=TFSBb5&%+nAP_p zUP#V#Mq#FIC$56XNT0-7dO(WEw~*QaSdNSi9w&sT-54FY{!mJ!XFk1Pobv{A;~kK1 zG2Jt^cDa?jct3~AM=gVq-q=iuzrc(5B6oy;u~3l+&0Z8iz>Q9nk$b#@JF$bi&N-32c^AD>v(BdP04+-A!Jgg%VPCiVZ+7D#4mYpR&QiUQOSJ6E+ ziqP+1bmVEpPG!P%!COghL6GMfB9d4W7Uu^=H~h{luF>?nKD*Q23|1kvLi_bP>LDwQ+7ymEfx8MnGVsdW}Ah81II7Z1-3D+wYS-uzwm1 zrmdVw#3ELG^Rezia^{cxtj*Ud_E8)+esi2dhj=le(S*DJbQj-GhrgRxzi#y-gz1(U zOMvRrTI^AGH~~68Z(bz9((Sb&IKmahRMofqFyqjI$BzHB}1wU zjy}3DY^dovym)jQH*R>_S@TNmr+Wp~{d2DkKR7%jwBVb`yQHppkPsGAGVrX<9klaD zjIzdzo{Xhj6cuL(>!P4I6Zjq{RgEEsFFSZ!=CY~OYx@r*sih5HX9+P!2%DgNWcU$WjS?n$fT^ zW1>f$Jfmxw#hk)v?sUZe0=+GVie1TXxzk=GOI?lc+{u^nFI1Pl+D1MiWEP2{&sOMT zW$gzE!r9d?o3Gt%Qq?wM(t|Uj{3~NJzX#VgVl1uR+bQza=S7SKc$|f zA2@X))2qABA93X=Ja(qQC*wFEd&shfBf^e%5SYN0KGrk9JCXKu37NhfFMJ;|z58uA zCIjH=NLBcc2iJlBSLZ8E9f#Y21FQ2WFf^(6KMys_@;w(t>J+L}6W(txsKcooKQ{(b z4u4u2WD(__pBE%fcfa-2Bj;MV(#K<>JtjFX>O+7P44C-Vf)k|(RXPt$Na49@i=|Cx zj3p>E3f?Sqn$WJAxKS~opZ?fpVa_u2*iQ!M$7&2ez9V|0hn@iK?c9R{WEO_la2I)Q zS{?O+R&cshRskXEixQpFsD$6YUfJt_&;gStm|H5xt9N7csxxT}-H*X- z-9f{jbgck~_BwgwRDK7;0NbQ#4O5ydPg@I)I^P~OL?D$jJ5O1$#Mdf&Ak+MnN>{?( z{kjWXfw}lRY^O3wGAu!7#hT@g%SC_|7bCf*@VB`8_9lFW{!9~&Q zpz2xLR00~zV4$|-KmX;M!c?gkXIoRkBMGS%bEBf^FM^gTpKpVp^<%E&{=y=g^6NMg zZZtQJKI!dWe29y>ho&6xq0hFrjH3oibohqw+hlS2lzA(*%Kg{RnJ7$J`!A2%`_;9c%!1zHVCd>XuwoD-6kwnBW$1Sc$f3w3 zB1AUubTVOksb16nhc*LbAq_S=o3bCb(TfWT2;$kZr=I@6=quT8$Ti}4OnnY5dm&nVBum&g0(Rxu z!I~OfxR!((O$YrY7vrjjw-Wx$IiuzMn!sMoy@E(fQ>Bkr(_At#un4F^f@EW6lSICx z5%pxXXFgar5mBUzt^8(|s}jngUQj)}gMB{&B-$JJA|2lr3C@%E zw8#wZ79szoL6|`hFl|bDouZHGUM`$P2?I&TMgTL=H|^X>LIzTG!HXdjW?4n-BA3I< zWNm&;)^inOuw`K?E`Hc#UAibgP)O%B7{{jB$C{)3^%*F7*^Xc;jKE8*C~gRi`4VWp z5PJR)d^|qY5twra^x{=g5nQfb=+-CrVSW;=>%FyTx=HVpv8IPt;|?;ayIhnLt<>R7 zk70a&PtF~mzUodpC42S`ktj|OoRe%^9_rfTJ&z`=!MhT7peQ~PKbH06?o{a#FN*;c z$J1W{_bxq5hIpP_xAql!_DVtjD(9!w9$OF%{Ze~ZQI5D$y=R5xLdg?1yyOWIn}Oko z(-pM*mMo1%;Mms)Wi<(B$Ba;#x#6n$=d9W=^_ z*+Ek7iqaPBjVyssH|bw>5r#~6L1c{P$*w{(C#A_))`QPd#M`S;-1l7!(IVsH7gT)L zK4NQpXHBMhbz{3DMlaWNmCc$ri|YC=6uh=9WuHQPZuD|&UbyR8cDtHu3A^u9lS1;0 zncrUFm@_P;U(IU9jh@O&ub4l`u}`8Ap?!a2_4`UXN~HNR55<0>GFNx-d7YI7=g$6D z65f<}sE@l;J#@=wAR)VP;T+q=Z_`zUEv>;Tj3aBcx2-~^Z;8yIdEYm4UX+!eHkYxw z_Yj$hGcS$lk8qb$8$(n0`YR$vYbX-&sn?WXR0KhzNZB9*%p~*?2^6xXd zmrO+F;;MpA=cNS4@Z|j&L8q$U*L^;I)V;qqOUZpVV72`nRoyT7t(U)*WIZlD!XDNt zpu7l&SLdNXEN^uYl5iEAu9vb&T=#B|&qpica5vw(quL7^{n3R?ZmsXa zH|}CwprIzQk(SbjR6PFO>F3W$R_kf>mRftLHTgfvT4pMzM(D;zqt4xSU*xFab;ki{jApUXWb z8-yQY*ST8fE29stV`xW8vXXwR{0aPhxqYmW+LCp>#uZKs8TyzxYDg_O=9ZHmCx{9p{YSQ$qiKmIgRTtZ+aaq4{bBOc(vAfF|o~3em=PH)4_wW1C($b(>2(~Gg#oDJCSBn*# z&y!GKL(b;^GM&W!OytP2w$x_ExOp7Hr}x&khb)VQ2a85IKBND4CaGWMs*3@y{V0@X zggBh)APbk*Y4#w;wCqj>cHT-*4U!Z}uhMA88={swE6bNXVBd_Z5BWai`#2~lfW8+v z7V8=GftsU%MDnx$P8>H@bS-U7$!W8r1nr?Vkc`m-c^^nUb-JXsokMb}BEQH>9xw_> z3{f``L4_2#UmT9sytwnM0_5u;+KN)R|a_>Ww6$bj07FO(?j$6{Q}6@MBGwP z*j>&q9FMY*w97`iZT=0W>ir9&jR|a1YUMemSgN*^|8aROTfcd5)uAbwhF1ltilgB> zj^}A54e?Nrv-Gf8pN3RvGaU1uOlL@A?t`Fz z_qz8wk|5qd`RD%6{Ari~pNqrSlIHo{^V0aKA}&?{urZjb0Z)fZCO&Y66!ivZ-Uj0Y z)Jz~=aJh90M=}nCBOeP@McJs@1<8z-J}5ME7*&y7>*DLpfqZz<<|Ph+y!_yQyqMZ~ ziynaTv_~7+mLiBU#%Swu5x=i$Xx5-#eAr=YXgCh}5a^&1qFc#=P#IFeLqT?X70zi7 z9u%L}NiMqDjczUpI%K}du6j%^1BpnaPQ%78;?fcA(D}FoLBpu20YnF~0(TKRNI0{z zv$MxFK%LwfkX_&tVX9c;1xnW=JSujregC=-RC!^PfRd}a5d&UFPft`tgasrUHUB-X zn%G|^h}hZL!OKoy*9RxF2`D8HQ1Hs(q5fc&cDby^Kr@xosOs)Mf~!atT!jD-6k%H} zpkU-oJ@7;xKlfk59l(i)2m339NQI^L_o!XYSWyvo{8;+UJiCzp?6oGjXJ21F6G6h% z+M~tKuSZ>kp^@a5E?t`Ah7v`DAxH#M5y!_sg{53dL_Uvprch1kPw)AkM=DiBVIV5{ zSD}iMh9;^?yW+f8R@L)1m@EGp%w1VFNt*xyg_yyaI{LP;ry5BRxH{<@8F8QkrmoIq zAxbc5g?#eq-9N8-zX5^1!AY<;iSMRsIy>fesDZ%dca_UR8Uz2?XAhyz;?H&eX*CLs z<=A&FUKc5~sihiByzJAwEnzt$Z|h~Cq>t`xq*kP#KY}TzX3^89oD9=2Rnkeu!UE|$ zh(=yl8Il!4Hhz%BR{JyaqLGJ(2LM$Z0V7eBhZH7Di47E8t#QczE)T|9BUgb@^hg@E zfn%FlXS$z&;ZQf?48N{2s_f%2F#kmXs`Zdo$@=RGG4Yc7PzwH2o=g^z1vE;9E)EtX z&E+mvewcuis`e$0MQtsu(nI+LE}X`VFX=HtdRCfjgQZ&x+m=jl_;pzyfY}%C|=)VVP}u9X6V9MHQNyDc^OgLQ_4QRg4U*$ z#MdxIR&X?jD-1MIa zheBsklD4-|LlTn4e)VbbP+8xkdu4TO)LhZBDX40`j!~| z6#*N+Q8!UA#}llgNW0iaHhes8UC5WLwuscSMUn(loC?hZLu4Og6DHIunuq4Nt_%Yj z1M%k2z9h(MX<;D;P|y(^wy&z|tLW+?5)w%5W8B~l%`%lh^#09jZ9>v^{y*Q3BuVFN zi!!pdqm+{!V4cBI^^S*(;n&7>A)69N((c2o$it(t07oD-#}9WayasS6jquucB}ohe zzf!6#jA}A+Q{ko_{BZ4fdbf~I2G3gcjS2!AP_5UMcBy!E%RL`Ld5#a5`slV@ZC3Yj z7HI&*b2x6}nQpk5Uh~tocpg2gnU`hYP4t&?Z5x^HR=*3iVWfNPg9i7Xt$LI}2h-F+Nwlvl4^Q`h}4RjCW09JJ29uJ+PfWbiXw<+=jM z?XfgKEPyddcolAyl2B*}YXE)+wA8|0K@D&;~=S{VQBb6OBc{%9cW1Jvkiii)%p&nQM#%cU*6^5kdUq`bns?l zWxDx##RtG}3|wdep739j2C&Jj8$FCM6);d8V17zxkq68EI>1-JKg#FKfI*Rxl9DWZ z%g3jHjhOgp95d2`xcaa*H`fE^2~1vaJe=|5bFc@4boc<#QUD7UJU~;s%M1*C@Bt!u ztYKKh3Zp$DqIBIoI5ec~ay@h?GjXZyP{+J*W1U$ut}JOvlIg@8l_>2axJ2C=4~$?E zo6;_dm#rcIR=r}cC;_~Ly-#-Y7Nbng4Ep(iWlDFGU>Vl!$>g&KfKR3{ zycyi%y8`$#=NCl1nE)ResOW}1m7{4p;_a2#F$BDMSt?F+S)-_#ihYd|yqA})Y{>mQ z0Wx>T6W0>3OxpejB^4Fqb}Y9U=12j~m$O%j^!O~}(XSuucRnb)isKi75PU~oyCX+# zz7+#Q$pvJD%BcVpXY==%qe-=!imNJq0HA8XCeJJqumr@!`lJKQ>ZH)V8VF&a|9t0@ zKo#C7UzA%AuY;C?0z70^5Zu;ER+TM+7Rx!?Z{^`p(fwe@3ld{crOXo*`2+TH*n;J0 zE$AV82~t!HCn0~DHi3jZo=qu8&S#~W?xDi-FiWDyh^uC2mDXHTYwi+$hY zb&Y}j(X%v=eMi{ZEL4v(A&OVQ6d)?iuU{fp)Oj=uID8uyaYYGn@x$#Aliv1rHcn1K zervO1y$-v=+Mp{P=F93>#|l?qr;d60RIDvN7B=jazI1fwyUxzeetyT$wwM0F#Pb1r zcO4R~!4`_ge=mg&zb!*s_YC|IaT!sihsHr9h)dl{=c?V(fDQ%IF*QBC3TPpa4gi{2 zRZ>E7s{~RCF(fBuL3Squ^Xnl6j& znVD@j6~{LZh)7;z1d#XtvmbQ;F@2(b<+aQ8pad&;xt`LgwqTvz9(IVerd5VmfcpceEzCU^TG?G>d zDncnx6)e*1lu3C#uYPh5vlP<+V>P47jtIyn5+{e7Du3MYGXMrL)|oSOhdp8 z!G{=EWB|Vp_a2_eXnPl6*dnDjMEnjbmT$j%_E^+lYL`_Ug1doF}2Y%ZN09|^LpfRBjk6}nIJY+VtMQx@G;ftr_L>B^#!UDv; z^QVSfGW6_Uv*k~;%EP@pCUma2cQ0}p~TfHfL0$Xeln115QQB9JH%Y1f|% zzE~z>M8GhBHH6ERkqaMXRW&0q9`f`d{|7F^+9pmTdARQVYv?4Ba@%9JE$yVy;BI6P zfv>nz+H&3%U{w9%$sWo2a>q(N;b$u4F(0&UzK+@R`E`uoimvUf!qAX1pstbD)}J=D z>M;>!V5%h_{*+1h(bC5I}1Cy-$LUEv+`rv55TDmytsjHa58{O)wWs-dml=(kV24rvf~ZG@!*a*bP9| z#zT)P1B{CK$}r(Lh#gg>WOdDLs z+eXj0yQw*f5?1%6a}--cxsQD42L4u9=Yd@}(CL z3Za;9CJC;1aT%feVH%Yu%Xf-bYJ=2xQY7bnbOkzAu7?W^=e2p zd+-wf8O?KLiiFgnPuicQ%+{Yng=mC4B4C43Bgq_J>uaf$HO$>fAZc0=11svbhG6cRw1LEg**o)myXg01w5UD(n9jr#?N zlaLpQ^v2FcxV+ij5AKsSDrA!myu)bK>f18&l zW8l5=uDrC`gUQK^j*$_)9hVKbc0!Xr=H4XVengQDr}}KP=V|A2M@HzE-iu);a2d!j z2WnWx1f=TLNi>q|&WdC7K(^q!QQ|60E1rl_n~@X^ccU;2)zxLqF0%9sD+~i*<7Rdp z4+RUkENk;JkTb}BK>_3yhAKN=p_yft7GRPf{+)!QIq^FlY`L!pvQg{K9Xofbp;8ev zYd=59JmTDojOwZpP{qAIB`3e_qayLJCr9a9i}G#j!e`0koE?rp_g7`k|S z{;zdYT#LCx4aknr(L`cpF$Pm|G_uAr;;{{tP#UA)dN%&71T17*qNuN#1Ky4+T=w_C zrR$vMn;W3yu&rcLZ1(gxu1eC0VTHkIlsfo|C3AawJL0&tf|k9a!Q{li(4h?f-9Asw zJq4?aVC#7~gDE7DB3q*8I~LoJJ5l%K8QY5UUp2{kDlwz?Rw}TkTk`5Nps`&2m^%s( z(4u=pL335}=O}o@ zKJ&{Y6aEfHuhJbd3bIq<``N40%b~`i+>hBMw|H9aT}WC7T*4J5T}d8DR_zXPzv%{{6;x~ z=5rwc(jaddg_`g1@G!3vi9q)ZF(YNcx6)-oeOrGKk@MKjnwl{p+PdONcz?Dh+qcZb zboNo51cWO9!&^dUd35DD4YTYNz=S0q4Z}(A*$UC@Y>3%|2Rv9r=YW|3RT=tl+5NES z=$kp!;1r!P1K1WG=l17U1Y0qaz&2DmLM|*eGY_)|Vio{?L6m*C)<~F00?1d7B_P^) z2*6bU&4~`dh&;tOXS4kjAVI8kSlXBv75DByk%XZVh6bzm5GIo8jVx)48aE(yM<7EUVqx z9W7|?U%T-}KC{%(mZmS<((_Bqcve)7l?%$SrT)f^8*Z()_F&m;_m0`OcmSc`;_+<0 zmzmi#q|kmuz3&ABseZC24-Jfs9o)r(HPKj@Wxu@_age#$5wE#?M7@ysY?6p_t}=zz zXDzUs*wE+&L!c(-HZp$S=8lU148$UwE|Q7um&)zit%ouA6IiNDw?vYnxY-xF{efBs zxrlbs1T1{=0ROqlG{iBmi^yfH&R47jPQ?I1_E_x?w7th*mqUnh_5f!kWRK*Ms{g*M z@|Nq@ZI(-a_VsH;+tG=^=YSY%bkqhwT>u#@w$gnzWhzM>zxn#(W#qtCZ=prFz70;S zstBksHeW~pHuI65AJs--(U7J02WS*NY+!^dLtiWe1O$L99&^JGJX1?bHa2^uZk`1{ zWghIoqTdIK9v9;uSMF)!Lrm8uw_Nzm=T~h|gNSt$Yz>PhItjmP++l&RYWh2NNgSdx zZfR}J&dg*ts^aJ2c>n=HHwsD|G8PQmx{eGkQ52gV;}xx)FYchP7QG~SMe8qYN~Lti2l#|u&@Fyy@2{v3W~(llSC2C;}!F%x5lMyYo(z##BU`7Je~YsPkkEn zwq00G^^vqtXq07haYlQ-q6qx*g)tB)*qHP@+OPdR9bWHuyeGBuW6vaNq1|7??f#C* ziRe`ZvqjotC;xU54yWTn?R@6#A4#KGohRkaS@eOB<_Owc0;XW3`9EV*IJNcOG8CKq8j}%%NjLz_wUlfoO{`XC~IeD0@b7q zlN&c)EOZ|n75c?*a-BFguXk2~b$%Dmb&JS?Q*?zn8ye6ORP=FNF6 z!(SF=e@3s8YdAs-02S!%dereZ@ADM+o?QVAo&sEs&+88jRSHv0)!c{klgF$6m<3Ey zM;!Y7PSt8VKclJZGv)vL?JCsnFVuDpCG%NhaZfCvFVIeUFdo&PxZ_i$NJ_v`y3S(NLzS2q5?CJ^w!gjEON-*YaYEIz2X1tUdew~*cl}9O zh70Gbe_5UUF1O83>5Bg6nM>pC_Xlwyrb$VGQW-f12heuT?v#Ui-FKh3pJ$?Y>ct(u z?Ts7CzBm{#@vAee6*<|d*EL$Xfbn0ayZBF#@I2XdJb6d&MCoo`aqGCGr0C`36_)%0 zS6qVsZcFb?xj_P2V_M-CT)L?^kq3>VB7Qe z-_4%v`~qiO%0E~9nVbEpgvaUSt6>ef^f$*LlR4_=-SOuZVBxrZd_0n;eZn`@E7#E# zhs|JFzxQT2EPJZq&uJETGJwbTooP8g6e%Amvyv9sr;#Uy7lXEaw(CJc@n|63)hbmfw*XnnhPc~nr8ren?DRd=Z%3>Pn=!gO@pF^L#fw8HkDkxG7@&=& zcwO<0&Ev0g7?g!xX$!F+8GgtBXv{oNRBZg-&8CSKFBpzR*n--*{fhnMhT36$OQfvX z;ml>nMf3%m)`lh;iQ`|nn(G=!bAM^)*O7~m?74hy2=t6HLCH-uwN0KZ>xR6n1{cd^ zj|BC|nWQzplhc4Ysw=mshppnz?O#Ba5vX_$kYmmpiyXgRo#yJ{5i6s1#FbsYV=Rj$ zOM@pjDlTqMIe$od^4oEg90#=_7H<+wosZdLb8^`0?=g2VNuKjZ4X^)kpVXl#WZig4 zY)Jh#WN%!IN;SE;+caKB$@hlM@?fnIR#!H*tJhKh=Kj~MVdvh+CYMVWE8t+?^(har6ltSwE505dn{rxE?5;%;Hs64Gk zGyAk?1?Ijc?_OiQ=Qo8KDWtVJ@Q)&2m$K>^Jf7ur#X;?k3E4bnrw-g)+**Q>>3=kZ z9+CPIb;xzHMo6y5?=+R{e^5SGS()|qs|O?6LdqjF<$CC%*&+L-CGr*}ze|qOP5VOt zByRmxn+?N*oCnDgeC6M_!-N-PQ10;VgE;*;S5AO5-Y$5jeTRlYo08^cD0xEC*Td4# z`b=FD0JuUNhlko#TJ3M1KZiQ;9?Em~JMqS6bnD1>hM0yy-^=9$PKo#w-$I+6Y->va!-jT~Ql^vBC%+|kp~g=%Nz$ro2521? z-BNESDjuAd%4%x3b$q(L$A++a96j{lyQbvJ%)jN>gG%+Qj+Nq;``2$3h9<>cPw>R0 z7D|#*?!QQC-&`7`I$bVFh%s~bgKHUG7=lz#Ig&(_`{no48^5cx5*tB)_Sd=vNm8wo z+0DVhAUx2&8MKy>J+bP!Um+1D^-5Kx8Cg78h}F5h#P44BosYv4_V;6{7&;CMqp`MV z`zt`@3rqmN;y3nZldHBUO1e$|dRRTFI_Za}a& z8BB}BU6O{b&6-4iw|0@vYsgwvr1P!!a24iB0=tbuGCHwtnXovMaRw)i9?$ zHha)wB80;&hhE>>Zg&jh=`jgY=2^haJTfw8~DiE9XcY7lIs+t}&om>6!KCk|V(4<_st2x9mbAJs5Ik|+^KhO35$`_3r&=-Iy84br`z?s{3K^6?XI)vGY zBmNe*6O=Dv<}kUPfXi75X_HaZKz~1x0HmJ6zr%?X!-5z@77Fr<;J_zfH{8_PY8MMy zTxN$L7IgXA5LAe{eWsg2k%D4N%f4{{5J59C7~X1K*JsB8n9Hm_D&O(#c>-O8_`n7!knC{Z(>R-fe%KQ^}N9oN!91&!6EP4 z4w@mbT^*Y;&`U(!kgPic?%6g=gg5n^37}l3#G7a|iJPXhoQ$0MC|M6xj56`2l z1r3|P_i#W5m7)!jj^SKQmM#`ZPLjgekBf(gSSjYY`I&(&uaWNSTS9 zOD)i)A>gcRFo7#IS0JCHp-tfZaU2i0XWVQ3Y-=C?BEImJu>ko1`D-v}q00I%*wh&i6=0)=dC8TfZV zOky!766U$Gadm@!{;~K{rB*>8sf%o0oGvJsanrIPBvI~p7%2wr_TpKmIxW#qO#t+b z+w$Nosd*^>gZu``7?7eZ-BJ9lYcK-zK(Fe4s87fVgQz0Vk|Djo`s)aavg;&Rpo7IV z2Wml*o*PUpbQZst23R;b35qrEBw+NwkfxRN`SM0=qtEa^H4o#XrMv zwh(R%R5x=6NK_8D;8Tk}YziUXPYx<=s5EWF!bg%&=0qnChwKlulr8&H4%Ba$2LzeAEBiNzJu>v4K8TTm^xmzo6cgWm=49L!Y>ne*KM8`_x{ zCK@e^2Uta+vAt(J$r4e3+h{7z1K?PunrX#lKxoTnqGVQ4jP!vZi18;Si`1nAou0_1f3&ZJGGe{t1rQg-T#s#wZ6*@)mw!v)yN{~{ zXX?ip{J&dv{zGibJWpP7zt8{_ zUwrZ<7B|4T4(G^`IIn@!;6P=;{pF7>0}URH_$+jn7Wel)_OUyJXb#VlbtEp z7j`^qV5@uwIoZp=ZVe;XLLUfN0RcGK9x;yvofl=dKk48Gn^|kEdmh&l(4~;M;beV7 zEY!=`tfE*v&hoWD>_!!!{;(Ii{W6-~PgAqwjrJ4@Ztx3GQ@Ve5Qt`r2<55UNC#~yG zWRW65170bxPI;DexdFzJ2p9h?>~!;YScnj6~ zTh#P;lJ*DZNjr!XkdyAei}GlV`hVqt`&xj<1JP$Yiq!ypc>#RRFY6WTDWYSa)_*6E zvG+xPit?|XEp>hZ_H*DbxUCCTkHAB$X4=;8%Kv$z^+5n4`lXw6EGG7c_R@O&(P_(GrkRig9UF7MVdVB z4$py=6|8tY);|+qSEv6Jwe{#^*=9*jN+KX3SpxL}dOHx_D$yi5RYSyW^<)E(TEbrU;L;Ah;|b~G^52{M?kGe7;L3!yP)wt^#s$q~Nij5A#>$jcKF-d(7> z05SO4fK77R88D>v_DUtK1#CckKm2cXd!0`<0i<^qyqXzs^1-qkH#`QP0c*iKKhau{ zlKc!B>|7_wggv`J^1R_NS5Q6i9 z9Z-;lc`1>367+Fcd7KPz#Sek1*P8;2fU^q0<`W273P}Lm46wkFE)HI44d4*me!e}v zJ=+K9j409|yCGs+WTXNBx(k|wIRknFOr%eB4h(N03F_X=nk04JiY`aJpdhT2pOnbqg~O3NjIJi_{m z@*#ofNZ^pwTc#{G5%zyC{hy1*1|=En6vD4WYTFLM{=ghk!TEVS7rcW$Sa>rGZs3A~ zge!x8-rmsk*T7>u`wkF@ZnMvU=}BJwIkF4@tYB1MN3DRtFyDvX0)F4aPM~V_bc0!` zKLwOVpz!K#0>;4>unS@IfVO?$_nTmMj#jzc$L1T1?cAvoC7rm_p7mrnwpx5ivQI#p;6n$ zEw?7W)$*4CbPAyBq1`gO0YAf^4+ym+z>Wp|2Mn8mRDU7RFKCDj<6~lcV)7XV|IE+B z4L(J}PP+{#Q$-)66!x=3nj@nKx{P{Smg467DvqHb4}y;*ny% z9)PTwSw8ezL!cF}ZP4HPBH=_uM{i63WwYOEI86vFfiTtNdZ4kt(qq(W47uvYV8K%? zs-%Z~KUGtMk_;k>r6eRGXpiE6?tk?^nN*1HDHf}(Gcq$p>tv6Cn;C47M7mqRZwfjG z#DpcDCzu1^a;e6)>(9GGhyF}}`ZcL{7`whY+1h9TvDj@GDVh!2X`XsAG6O((fcZAh z2nIu4hoR*4tcHS2T5V)cZ{j?|2ZevkW!7AoA%MP5kV|=O4_NyE^_uBq0gEw!GC+iM z-u{Az(m(Gm`Un`0n;@h&Y;3kbaqeWGq5)f68S6yDoaH~I&s#3WnJWb8w~CO>sJ1o0 z`>i&Nk-hyH7(3`v^r{L7e0yTzslGDUh@6d=zOY+h^4ZD%viqTdgOB!;AfbT%3(+rw zT*!8{k{ApW)?)thPiq{afSJVu3_0r%VD1AH#CT0QZGHfd-?_MPHPPA4Pz4;gF%QoP zIQuu9o`>9mZL|ou+wI&{WMpI%6e7pg$~Zv&hF_+441sE(W)9^c5E=sFIx^}~aC`r+Xr zFiQ1AQ}62KOrde;wg-UF2N05uN8)+B=Cb5=f&!8JCP3c@!zR*JSJM)HarGwn`9`Q-Y`nZ`#FFd1w%?E5?as-- zCwrLVU~r$wNsi8e6OLww!-@(NpVEA%z6sP`E`!LUbeK|)`Tm^*9o?%xa6PX>wwMet z&k~&GW<5a^>BXNpF8iJLNo!<2Kr92WcDn)^BbLs6 z?!mD_H7?PoA=WMe?*Ll;Z8&kL7bJk;C15fVQTQYG+#^-cbJ5;99!s-T976XC&NL7N zArIK9HTI!e^yA60Thss>*#@C8L?LGH1cz4<;@t)iIS677b*$r^^*(@DqliWeAgYEN z0Xs;5o0BB;7SOu^9olgA@Atnc9W<8h%q`r%Tf1XH$VWU+SHRA=QWsun?7q}F8V%NB z0|Zm;Tbq)!>w2^Va5U7El-jf)oN|;Qr4ot3YI0(t0gxc5egk;hz5+CWLrMbim!rj6 zf>+Lt^}x~lvfd{FIEr)X>*x0`W#sCKPTJF14na6g2d^6li<3=JHh_tqxGI}t$9mkS z<~P$pxTCV>gznZe61U;$c43btSP}=IbR2pFo6af~^N+uX${Kz?$3d>RZ~IbpZU_Ci7};7E7|l#K2!no-uxK&Yt> zGF9CtB$EL3a#@s~g;|Y(x!^_hx?~T|teJqNY8`g$k($1g@=t z14q9Pz)^`^87mtx8JyucY0Py8A=B#AJ~&QH`UjHa8o3pqFvNrTO<=6$^M+CmIK#Iq zYkmoUcfWSM1NM+eT%c>*fSsJIhOGB<@Z^PgU>$_b(Pu0@Z+Cva5oale@()X8Rv3RNx>q_CwbG3l74>kDJ@nr^XbTYVKFHMW(6q| zI;yP1S;5=9ieGP#sK#F){+Y2Dr*UqaSsHUbpQ$5uoLZPOJ-7j>lm`X}2M7A?1?tM{ zFqPApD`I0}=s#6RIJ9WLQn49dUj{TvR#(Rc$n#~K>;j2Za7adSa;Q;QI5?;%C}bfD z({9|zX^AZFaLPJ{J-QD}*7^3OExA>RSzh~&gg@*;^nIg@GAQl{v!dFk=HUKL) z#YYvDZvrgV4}e!jNEQwL!AT)~Ar5aZ2*}8K0QZmx<#WG21xIcQblIvVW#jjB>a}(V zBqJ=Rfg9aq$=oDfQ9W4_r!)h<*~pX0lGM%;BGg8tIWkOt;QXqn0LGq$d6a&oo!=T5 zfrFegrNaAji;IiH!(TAd_xCNhU(vH)I+EQv0~HXhZ<$wcWXM2+5?%J=ccQJ2n#C#@ zc6N58#2*v-@?O$s=I%u0Elc%9QIhpHO}4lHL8Q>1JshS>ev+YoNtcX^Ho=C(48o-m zL)-DERp`_{k|j96V`bZ7sg1=hX&Ij}Rlf zi~~0ns*Q7(jUKVzEU-dI&u zR>lsry1w%emnJ6oscYRE5fznwonr{U1!O$KHc%w8WJEZtcG$?O$mf8MLWFHH5kjW3 zF+*N%MV=sPF;~d|mZ-i3*~J|oDc3iV{`f|Ta`?CoZRG{n&gqHz>63Rxet;Md%jpuK zR+~At_EB}&_X|^6o`rJrW!__wwy8>=$%I=mdYT4&S|0}l?VeaRH=EuIM>Bn0H!@N3 z8Q}Zx?d~3?_LD^(Kodm1I2=7XvPO1mX{BxSFHB_710WuT8qq*>HZWYOC(N@h4VuW= zHZ3~Xc(-6MeOSmtmLfN>-jVr2w1&@E z_6kz^#*EZHCqU(U?Hk>mB5`m2@`Z&8F4>r$Hj}rxz;QQ)6o>C22FL5yK|(T}0&Zre zrqM~mc5Re*9)fW6HRtdx(XJ2cUhp$N3BJ<8t+uqZ05F7!pOr9tX|O~ApMwt1Z_w*V zMUChypw4rwVRJ0RFQEz$vWh=35P|Nt0@f#~;6iyNW62V$UiLen4KGuI%{p|C_m2;z z)*;}x_ug7ods7AQp2#r%?5%Rdd`?e>=E1q)GlCW->lK&+4IozlQ1?OcO*)x^54UW5 z2+EyWjnZWXZZNK!CTkwf+A7yTaJTWGrH!@cLbMA=rUG zmFXk+U}C{v%mGrW8iNl@&#?z1YJBH$7KpcG;Jb&QCWvqA3pfojUm@b{=q%0PI%_K3uV#`Vkw4AgCeyOog&<>2jQK+n_ARXO0 zk)*tiRX0>I$Dg0y6u_+i)zqYAG5uy!!qAx|a%T>EB}O`!EF7zz$ph#3Yiqdl{4DlMIw43yh5Kk$zFeSP_GRx{cHI>2`fR-)~(H zkH(MO-*8C>z+9xp90W#oK-7K=Sd^ zf_K)^((3N~;Lc0AQWtmSXbnhFt`KQzL}fP|e?)MJwU@v}&xO7-dJFlmuRn^J%?sA@r3{ zpk@287#bN-7#bVHvm#uH-p*Mzg~kJu8ic8ijd&b1ySoGCcdq=su0BX?x~(JZi?(fdQbEK=aJ!xC>`Sszp_PghAB;hX#aukLusA-> zWQ%!;fb$}WTn}U}#o7QcVo?F2wvby{L{=(@5Oab>=TuWtQd$xS1>7!y)g$ppkv8`D z*OcZQPzpuZ%*@S0D{+C}2H0?p6Q6!Wp!rl=SGUpe{D^GOTq-KL-5OGsl2@wDFE|FA!$ocZ^(zQ3=TY1ZIX^9`UxCD!2Z$em zfbvKJU0{3$#yDV~f}nOYe7d!IOMvwQ;9xrB6&R~nB&Fu}Sss824y+^N&cu&s%lG#lK10BylSra32AQO#ulX1^HEuT^dH&47^mF z&)h4jM@oFEHr8kpm#QA2@G6gN8yM1e9Q_G@of1 z13(Cx7=Ul9hr)uLSrK_X8+#uT<|w9u1q^CH6Keek2SisA5Afy4Gdu6zfHN@K($v(} zkyA`#TwGkM*hI`VfD)(0l;ve)z-+s^xR53fD_ni1Np&j*CY|)dQvQNazmh0N(HNdj3mT^mt zu9OUD0ISBob(?&I6FXCy%_7Q-#Ts~L%dLdzSDYl^RtSlRe1Bfu-M=BVT5U&2i1j-r zvjTz3US3{cZAh~$2LS*dkO5#f{SsEBTH5M155kCn{RjdpFDx_%}a}yI(j%KrC?WI>JQ7@bAEn+Wg<}1Hpu7aj1BuIT zruSt#giBWqkXlp%|6?o}M*aZ?LD(;1GKJ-2+fhygYiPurKl0of?c1GLq|x&7eY|XS zwYAsR*ZnNRf}+5_CzN_!iUto`BFXRh!Vh>jSFM1RiB0_c{2WjN#M>%;6rznh(MN>^ z7!+x?qF{m3>gW{*3e01zWGjdmcn+j7M8tA0RF9>Ax0#uhmB%Zq*Y@Y_%XQD`UC59m zfBh>dtZS`vct3pj08IM9>t&yD=PmSw?@{X>-)_dg-mWYc;|Ogs%I>Jf#-|ZI7v5NAc z#@Na<$G|Z9yXw&YZepxN@CT3{tjhUlf}Be}?Z9kSYSNw9lNKkhl3OZ)p~U^CO(2ihID$u>R(GU0llf4Rsa-NUs{8=mfG5-M1o8_9 zk($g0P1ul6ov&L%|06l%t_VD298?>eGbdiiPu<9VOcQoUAL9bh9v^D!_^K|%AT0`R zM2kYUI!fcth@&D0H^NK4s&A$(x=Yl#l&FUPA${Ou4!i%g6nWw{I?kj_ z|7}Wi2?G-&V>{BnIr8jv?VPO)NSP$fb&SBD_m3B=};{8c+wR&$*`MW6sPM3LwRzlPZikg%KdHKxg1%LZj8h@R)G082iY%dT4x|R_H z(6vaqDXw5j-!j45I@7!#%;WBs#}>0LNH58qPq!ExC~PCaP5Mb_e@)^VG*a>`%Fb1L zp`ngf&cX4ZtE%bOMb*>+>Zs(we#7MKZkLCLE8E-k%=ByXeZ2gVatiqGWqV64N{utz z7`$TMhh(k|SVL*1Q$wZYl<-YQK^L2N^}#9BLYKjjHvKTwho*b)5nAQwBeRH`SLRBh zso)<DM8`Y@;*d}n)v#48UfQsFrtvS-amy! z=n{H+r$ud=7So3$qL{d#YcPS-?TNR`n=M>;Xt4(6y-WmX667=OXAYCFUeAMEN}lkV zdpE<0H(>8V$rcUX69~cL5S}f6i-S?0O}*pthG&%6NvS}a{k_z|M4vMITzqCe;|LSA zEN6-Tkg3xW_6M}YPK*1Mi0j69J(tdD_N6-Zg5%I2(o@UUysnTXc;Tld_a~8i&W0AtY*r~->;FUs~aK)&DhV^RGdV;ZVHa9YUL%j!$ z=3_jldhoEWlA1>q#Wqbv1yA?J3v1N0fynqZza?!i`xgyG?-K8ioCx2{H8X82G{rkE%R0WekktEhx|yIYorrX+xGXPWBcO+q`4PWsRp8G#;|Yr=`L36 z;?afD(4U+YEV<7*yoL4hdzdIc>VG;1Dd%3r((AG0MwQ~lK2QouQ@+tdURfqFI{aFh?>dfL6DgL)%2zbYFZFCX2s#dzJrQOZE-5&PS>qQg%5js+(+VGqqrj`t=P6w^YR zVhwLUFj1rne)w)I_-WeXTH(eC$|1K#p;%#}FW|SqUViEGK366pPZxV+hh`r6$FIAs zrdoDY=!1={Hm?-Ww|d4g#+M8a?8gsqj@M zTaA2^Z+%8VrI9`Q? zV7putoT`MaP_RYo4}psp-_N@2Ky4j>xm>}SHaSD)&X%7(8`T$7`;;_F=N>hba5 zX_ijxFo(rlgWyiT>^$bc|<*_Mv69w4Y#N%jQY`VNk^+GMmQkLeQj2P}q z%u^J!>$9`?9|d(q+uzh zV&@-r#l3gk)3bu>D?#_W();y6p*5pNPYG@zTYS(VbM%@dBQHX>yW^R*|iCwz9^JQU#v1*W2Kv z-T}34Rm=Rw`N8VBl+wYMq@RCex`WtEZvz|lvFnfV3|jo07i+itgFMG^WPQhY7iOtf z+>k{+CWVLyT~*I2zf6OAKR`oJ*?%(3|NZ@K{$glT28Pkivo3>^`}Pcu_LS83Od0gk z>_fa4w~9x_tVi=ymRZk+|D1lLalJcyh;8|xpk};d5c70y(|IG+^Hu81CZChW>bqjz z`8hf{31n6-1Qzj^<9#$p&-+h0`2|1l*%ufTygRYGNapvnBsgERCh0L3#usV`>9^KT zrQW~FTU87FqIr`)o}g*%U@^p`whgCy9y;FF{lb6Kc_sDxtJK-KV9C_&?}V`^C+hv$ zwLa|LE_ewm(|oA&Cu$w}>+0R3;$6_x2GNB27C!9I4j3?KQIQCY{g?--C7%P|gl}PP zv)!rsmgP;sYcBWlt?VQGjKw{5THoKsGkz5#)A082$+yiw2RVr>(UxfnL8K`!E=6;_E6mG-5*`Ez~!g&`;7 z)U;LU3Om&;!2^ zi6;CJx(UXh8Z=u+tgz2JI!F#gk3TZZz`)}XN^_oB*!^%oR&BD(n=@5%GQstZiJ9_3 zqL#W(n{Ynk>ElE1Rf;$F{?vW*`NaQYGohAiN8%=u?bjYiY>i{v%h)W*d0~9p{xu5d}0o<)c3yVk|SnDv%HB}VYUBl(HpgI0#HaFPsVhVrP*LkeY&nh=0yDS)!&=Tz{8h-B{%Gw6iF+vE0bB;I&lpOgEPCu7{JiuNwc# z=E84Hp<5OD^ohZ*wAX#q`Kuv?Gz@uO>6&M5O)WL*=TDPOUdPR8Cev6OQmCXAJ|So= zOntVwu&l{5-#gtiyEE=Ogtx;_;(uP@QV0EGahIDSK8nDcNiK%FJd$pB62)}6k~x1X z(D~~E9ZTvmdcFoMW>D}!jhKohegXZ@@d@7c>7zHf)DN@U{ch$=@|kZ2dO+(&B}|n9 zCQiERTlU@WkA8Rka3HK_HACnhov6d<^?M)=`fmCCHICxWuc8yjjv?KFsdEf5#PMWx zugd!<{wJO!vnMX830&O`SNQEi6J<-kJ(t?C+f`c-zFe$@I<}KKI8<3{T0WaEnLVLr zrCv@}v$Ao3qLsIuKh18%W04ymSl2uX`Oz7Uj>V5tHZS0f6{;&@LDzjNkUczw6|Ofh zbzJ6_*Y0m%I~qgQ-K}>*s5P|R7o^paSiDhYPqicMrot(XlIHPNGihs_UrU*A7h6WM z&tPsi_V=Q~YIzyY$9wRY?FnJ+R$owG1FCk@)u-aQN3Dj7;_2tGuN@@bA=xgLsUjD* zs@Cj9YF+Vsv?q(FSEcCg0j$?8O+T_%Pd;{(ZmMY~oIa>hNq8md@n16=oR)ijLo@D! z6??V+;NDO>DC#JZH&i)BWOm|1mVvNXH^r(M8}w`$-e{C2aj17w`^cY+;$Xp6KUI+K z-YE0xlVI_MjmwYgd^6cNPk7ejd48kKtMXMlj$jHxjxX1EEa3~b#+ppTeugBqeq-qS zuiwmsAoJM`6*}Ml?`m+b73e@X09Gpb}T(Qk|cOjfm2I|(<4Kw)21>=~e9gaHg$GVFIySU~%pYb=Na7&k3;NrB zAjgm+R&kU&o{@Mg;^Unyv98_&Ct4i_Bx`myZqcQTJqnxY(h?N8-y-oC=P}JwA70%j z&J$FqJ5MOg%x1zRByp03rfI&-=5~;)c5eK6sq*%X-o_&O;kBx`!CaXANdTt;J|kCV$jf4?P$1uUXWHZ13q!0jc6a75 z(nSH;tvKgThWY%ojw*$)JY@!{7>J5%m{RJDR;ydx56ODEk^AdMY*lg2N#Q zmeqLyDa+X`b!%s!LpHMcWw`4KdwA)uMdFP;)!-CTFFDzZ*3ZOxuJ!fQTO0h-+vpyo z{;n$=;iXIsyd&RlF?e}eN%ZZR`Zr6_7gHAMxS_!p)cV#;G#Yv_wX}1@E`tc>DNgKOwOiVH$nEm!l70o3 zitlP@&bf__)4y&Zq6e`x^RqIw1u%)FF4f&q5zSFrWPW$m*CcUm*O(d=G8#ota#^4_ zxh_|ZcdCT{w9C^;6!6KtiwLzTV2vav7p3)$-b`qfedq2|o<$Ddg zs;(*P$j(g6nuI6%skFs+>$h(WdcMQXLj}qc4G}Y9+x0%@X)ynEA>`%B&r&MSTm&2G; zIH`U{@wsu9INI@uifhFv1w1T%6!-&Mo))GjU-g97S}n%oEgiowjs4Om59DIw#=_`$ zqpQkF#)@}PK)Z~U?vFd<_w^Jt6aONJ-795hNgkL;`-ZhtMY&A$Ni&C>qnvD_fw@W~ zCI>`=q}FJC7}s&6=$_d$Ym;RYNs2KGd~!(Sz|`ufxP2V+#wPrCL+iT2$SJ!Zz((i-?%wb>jf zykDF#4{f1j_ube9K0(7cD^Vi+a~mZP z&dOzSB}e!?oKM)k)n)2t`(dPK!hN(qeEEY@!WgQD=Tn4sd}Akck?M$eNz|-Ldw9La zk?4#>Od6s=L>Tgm-*nny}QS9{f|n`Kzb*h`=K;DZ0DY*bV}q0p*Ucc{s8` zy5#wZJ_*wZGhc)I7s)a;xvvg%`>ydE1UOf8#~gjk&w!f;$sq{lP?rZTa;QD-%u{O@QO?^^|X~gc}q`9k#LflC?p!> zk4eZ)T~PMB2D=#5_3xjPowarg1~BZH?76|A_wYW&QF|gD;i9A8G@U+Ki)d(LU*z~r zhTw2l0yo7OOK!mB!cr&pUV1I;9@C`!WFR`2;R`4f&1TmilIpIQgt|7!(8RUKaR~K= zEoI2IWO_ltp1G45m9d`v+0Koe=+2p2tK4*$rr76>$U7c~doXrM=wopl>gBQP(&J|>o*|PT4tD2a4>2N+8Wb4G z6`1YbZN*&ONWVE*5?zrrztT36l1F=3rll27g9}LS*Y_ubystVsE4Cd&RIcH++Q$5c zy!;QooN+BF2w(_|vbuUc71vvfFCw0^7Z_F5`f1vQf3bkDHcc3RcjeKU8fX~q8_eJZ z-ov4n-Nq?B4Vy=@W?!%3cT?S=t{?nan)W-WLyz{hJd?2l++w>k>|}pjzJ5(OZtE=x z_gdG{r3;K(t~f4Uqb;my#C?T#W4G3ZrW37mUgNjAI=5*6Iz;a!BGEAH z<*;$O_3Uj?keJ=sbWOH`V@GPkU#0w8JH^^^td6wyU~4 zzrL3;P`_=Bmpn|qzc@OIlS+~Fb9rj?Bb8`L&?bu0GQP608p_ZzRF=T=R;%2w3oOZEW%9@%2U0#-OHGvmLCyulI8P3XN?k}8(*tcPB2KW}Frk$eL}g%UN# zIcfP=p zddgpH_>R@%uf`vV(fRg0ZceW>ow)u>i6%Yf?1<2WX2h4woL7U%OPcJa1xI@G?0PSs zD)l1SyzA@0ZUK#*K3whHe!Z8u*ZQw(V=T4d8$7k~s9T)90w4h%@9IYL*f>p(qq9Qb zrK__l{a^*e4J)h`&z+$C(ef*EtJhz~m7Ymo>_3U*+N)kXiayF1JC_kDL05(8ig4M{ zrs5e+RfFIc%$g5SO*D*2Kwha}6L-$+8f7>T#b!)_)_;Ek#(d#XCm zow0P8g6OTUFL@)?vq&=j;_B(w+``^2=F>dq83PWdP_3=$N56ZwQ8sPs4xs3CE@Xch zG{yFuSkqvAt(V9QMVMOo6*6w1*61+C^gjW05%`e5*RMZeyPo5BaLYIcD|1sI6~T;iNlmRiq`w+0`IERH--~})HV8bDEO1Ima$g! zb?UhGS$|cb>w(o?!^X&O3{GuT7s*pQpN@+*c9DkGD%~Ruu2Kc@+3@7N0c3yFU>CE4 z4}0`%>I|D~ET_P^qv)9IC;8GtdQwn`U_r`4yRCznF(rinZvcB7OLF#2_ZZ3%D&7x-QV!Tc$BC^_}czJ%>u z4d+T>9#_ks5O^`yD-blf=RezTbvc~7mDIqWb#Y2hpO|1JqtDaCQDgI(f0&O=LVYQQ z9nVPs!GeF@MHe?_sJ_X{28k>C^GKHS{tt$M5Ds=jzY=jpM|{16maPu?4k>NFd;e2N z_wl|5*m9!!os+~h3fOB6U#>-KClqP+DR)k`Bd#{-q8Dj~i*e|pP zMShSTX*INc`sSpc)LP?Jx!2Z+R!3ax9gD+_&TvJH%cq@6JvD*Js<+_=pwdy^C`g^m z8beI>8KsXy4(epoa}-f%(%HVqvXsU3Ol}}-!jbx=ML-eKyQ17FJ89_G<}c3gp2X(m zUO<%~?`j(TQI&nQIJwd5AGPu(RgfKJ^Y7i+M$hEXo()d2oVW9L>cnw{LF!f9<&MJx z1Fgiill3g$sBz#@^we6Up;2eMDzobfZwDnf)mzd1S@_*p9!7khY5}C5?{-R$&CTPt zE`VA^>+VA()+k1?mO8fWwV5EaiXzi*wtS^G-2S|c)nQeHQ^j>Eey3xSwtRW8!X<%h z5ATfi>(I@(G^+BDo7#7mhPF}vuvg{cI2^<_?Zf@5`!bQvtx!rQd zDe9@0(j%?^HI_$Ihd*YTPm)%g&yhoPg4ql2MB)qOI2y4k3iG0oEC~mYb;93VKlGk1 zpH#TSc9JMUCQqE=!BnkFo2!4;Qq1!r{w!ispHa1;-GdOL6OMrI`C#HwpTYlF|HzK% zdC3%+VmovDU;W8ylX;eU!R+=KsxE6dK1c?C@?xtW?Hn1rE%(f4`Kb4mdu^jgIXQ4$ zI71Q(++6&Gslf$4B0x@bmm337!?1qKg{^MJKq$16Cz2}m)o#c{3kcj+^vu-P>zeEr z!Tfm_BvDbvaExUyfJQP-c9AYryHz2$q8q#duP8@k!-(IbB0ERc1#=D;;{Mxpd@;>` zf5tzU!4D#fx@(Mw_jyvj6QWKNFrWW8P;lL{6+jt-RyX5#tZ-1w_PyDXYSWgTrXuQe zlJi?r$>J5ml*W?nvALU00bEb%P-ReN>e0?xM+lK-JYJVJ0b?aT(DcU1rqUA-gp1dM z7bqF`R%MQA+VrTmEGIe%noQ`NEb{nd=^5BKAWsf7A($sB>8^1Sm#84& z^F?2qcQ_k+o4D(dgfwM=u>d1^{~5ix#`y8EA7@r@e=D5Mc7AWQ+zE63_Csdpap{O0 z_VYR-@&e1fuR<>Ws;-Ut)D@_uW|FL=>6w{V=v)E)C8D!F^ zvdabT#t*9059szHHK}3s2qy4WSG#cgJr28JLYg#plc0dgCjsKtj`wX?kVn+nJ8V;D z&0v`Cb``6>G2A>2`$H7r@`}u6gr+f`r_%AwP9AKY5#HZ#g?2vr>A613OqGgoiCdXo zgv!go$V@fh{@-yM|4wYZF^_S_H^~UHGx|1=9=5`T>a2&cPl&GZoj3B|HsbEYjyqDn z-lcz5c*%;u`nPSbdF_s#Fn&b+bYy^`FTvkuWp-fKD8>I}v(~{)!(jpl@_b~?3+3pn*K#PFY>qF+Z4%JD4bzd7>&N;<;Y#<&s_$0V_!QlQ?GFS zEw;I3g5JmSZ}VUj<*5FC3p5D?#UWP0IbPDU_uhY>l;DpgeeA{m-rGJn<>kvxWfk>Z zH-y-V>^MKq%YpkZZxH$K8wvg)va=UI$bK*lh&JRFQ=IYnk1P3%TDY4y1b^C&bXlbM zz+XXc@G{W1aTc>`l;W%&{QlJa7qvx}_>MDbUJf#fq%|GC;*hHZzX8qNyLqYb@e^W2 zCJDE-bhxGI#AsG3Hh-kg^jr}xLc*HpF|-Lh>hH8_z~K<%I4JF1PydSUbuU(?W<-Pv ztbvr_bDS}iBGNN&pL%i{C05(&_(mIWDy3Y<5=w|lXzMtQrITnNB`b#Sjp-+3%q~_J z+IfoLEGD0=ziGwdzJUnp?WXgFR2jTYr}F>fDl>C0Y&IF~jX(BED8l_?pcKbXQ~F|G7Ix!+v^qVd)=_9M{E3DpQ1G<9wy#S)*_1so=v?wJKMOPCyZ@H$1Wo-rrB^D+j(!P$9HSEi|ipY zupVES?yn?u{X?_F>J%WCeUAe$RQ7gHzi3g6b`plj-0>CBv2!le3_T9kP6Fowp@03`$!jp?m6M zxplfj_|fMo2V|FNibfSb7P(QdQ6IPSNF24Cp&*jo3p^vMV%|aMOik^cgH)1!y)`wV zxy08shn)(Qki+Y*3hh`-L2suMGG#m%ZqD)e3g-`-M0aPKc8E;j`F(PSdRn_mLq|;*inXdZ4 zrI=bdyOOUan*`%ZhY-KXdwXNbV&Q6yT0Bkgpsh_FaaUMfR;@X77lWQ|Jv!w z6VASOxD3(>lDfl7{H#OJg!z~^r5_DjA*%z!SKy+XkIt$>c~6hc0Ue&~v|4=gqqpAc zZ#s}mmP}w2FKDqYlrF`qgZCjt1~&(T%@*u`0aQy7&dQG(gSZdIqcR~%^RLUOf$2rB zb)+8nRuZ%3b6F!8P?WI1hv~)b_c}OUq-dAV)%Y%-*TKSiLEU3hn_FV6taFz+ca}a`->S>9LS9lm4&8=_!f}|6hmcv2n7J zGAYUE*qV~Eu!7(z8(hu~mLTK=52tbu?0`4r-t%-s zoTts%g9@4KWU%1xbFr(P1eO3px@+lqGY9gM{Q-;-0MWq^^Nb9k$!=S*h|2H4{lL{3 zr{l4z_|4-Ck~?EiDgmb%jTSB!O;X^W5pfc#&3YuG$ELhntKNuHi|7IfV^efvlX z=4j=`4vxpBW7}RwQyT4svy}TOoYy^xH%F5l=fGI(g__zq} z?@#x>fV0P9x`a&Jm(V}i$ztZQDeHX!c7gy$GSIPuih*@+y`ON9b>l|2nhfT^*2XfJjgTgm2x1y8SoV+R^6M!uE+6h zX_)_QFT=_2jPOkc*CcqJp}OP09h_TFFiP!F`@@NJ{M``?|8D`EfR^NBvWDSFR(BnF zC}eEk&FO#?QqFYq*k8-mHB$an-f)~~S;>3O$E$%%bNVWYwWqj$f9n~)_IJ%XGIIZW zi7;M-RDE1p&i&Y{F&YTDdX=9!RsGYz$5%%iec<_U^Xu1|rfpj0;`Hp}pv1I6`lno6 zh`2FO*Za2}zs8z8AL{K*pUdym(+QAx+^&%@saGEYXx3B+%ZDeSkuN3N`sX;~A430R zvV0H=b2yGuv*3XYU^>S$8V&=?JH(-H4@ZLc;)__8Aw7PZ301z#p6w2!@jtW9{;c0# zot!Y*S@b7y6~~(LY5^jwZ;r79dQw6Z19P(K2h+0nN*fi~DPWG7E_MtgJR^2WoA}J@ z*m$`bT)#VqtAvPevgokt(gn(UmBaA|LML|2XfY@cQ>Z+5H40D=^>lN%W6bOm^@Sa0 z!Jvs8Lc7yF-yd$boN+2%r4fJnQEA&nw;z&+P%p(J{&P&0Sip_@fc`xNAYXV5GEO9T zK}g#Gt_T2Wl-Z>LP|5O$5`^{x-bvrMP&P*ZV;k?s{nRCEu~=Ut)~mK_Hd~R01zHH0 zGnY0r#>~Aisb$wweEss&wdA)oJDJKXu|3vB z{vUcL3?EF>*mQoS#lozc7c?gI57&S{@ zX8QIEnV0P6g>*wFpwn!g*C2^F1O%LI5XHC&aPns3zp@cjskbB$cxkDNR^(_GM{6$t zlx4*Uum@FajIOXPP1Rp4x?*4%d`d+_cm}8o0GdH_`!I+zlI8xkjpSZl)&`|Do?joK zPV}Q6GG4kMSQ=j@|A5sDQbfpJw3%1mja$`1 zk4<1SA#%*IrzATg#drZwH>Odbmq_3i^ZOf+^a(6l;scA-r;-nel7Zg>JC3#JQ$gCZ zx_2q+s|m#wDfNJPzprP-UM%S%%PeL7ouI|yubs9_t%~< zh)H$c#8b@lH;;S)$ZJGV&%kKx4B6B!=q9kof-pdF)<2Xsj)0vlNK!lynT$6 zvDDw)SWhHf0dmV^0l+@_HpnSg>^;D88_SkiTL($95hJUKxGV^dV^`%D6*Atk=)c?| z^fy4>#+UdiL%~yo7#x&`&036L%lK`Wyb#POY;sqzUrGu9eSZkJmGIVfHFHA%sm#k) z1OEqWUl|v5*F~#fA=2H_pn!A>lG5EN-6|y^p>)>((%mH>pdg|2Al)Jc&5%+Gh;a8j z&->o{yYcaUGhpT)=j^lh+H0?MT)fE#Hs-6~uR$p3rFT_U5RZ~S!b9ZV1X_9OrtimOI>Ac!!z6P17I zqkSIZB(nh?aG}Fa4pA&Zgeik7Fz9^v;xcAGi7r9zol=uSMjZkftaBzK2KqHHX;C3# zI=?HF9FuqfS{@;M;tjsD3W@L4^9i5aiM}t5Qo?0bRCAGHg*hVy@|%s1J_d!1%YV@3 zlJ#6&(s2+|T;-c+%khCdb=PRE*(!UUnp+8uuP{k;7t_02C(PNHG)TH%v@tvQUdKSsml|k#2wY|Ta(#P!8~f#u6KbE7DjWGSHSSklqFYv62zP(Hr8=cg z>I1oyd}f#M2K4U@bCSI5q@IL!K9z48;2gYq^ZpS^Z@yDn^_{8q=xRP^*-Y>R7!zDoEfV(@Gt;(446@E=A01KQ2=# zqovxhZAns$nA1+Nj5^_C3h|atuE50}b8D~d7cosZel2dbkVdN-e?|RWRutc{5{F#B zUkLAA!%Fjr4vmYPWXvlk;B~=HiFtE)3v>->Mb=QmWtD?{<>Sw^NbDD#zcf`Bc>WVM zFHfiv_0o=~cFkcTZJjYP1aB3eym|h6XO=+)rVr3xzGlFjCL*;vQYV%n659T~-SX@W^xU7zZg zhQYh1^uH*c#|iZEP8?Z1M>*XbV`?&s>E(+IWfE+|vnhIUf6@{E5L2lge@NUZAd>QS z<&DUj-w}e*aLyEMFT*&I_kLpn*5yL8%xM}?|7DyZDMVI36^fr}V&R5S7Nx={GKutN z{CT%Yo-Qa=A(1fBHvXZGeD9V*4~kP{TUj-&SIqy^{>X*e`D}C=%Z&Yhd{it;&vG*)RmQ!PGK(_uhr81b!|LLDA@bQ$tZ*}JTu14c)$-p6YpJl_ z;Ljiu#cUghb-)k)->M{jS+6S>woYof|&} zDI$&w**$bF<8&A5JCC!;@WRd&8W0x4*K$XVzFdwxfbS* zyC*>pjn|kVO8-D^Bm0Oj!7R+m`QGZ?_8@GrFpyr{C0L0Y^%v7QNym2gek~+Hw6XC5EE0{X&)i-L7DZ#cZ zw&T`#i>%?LTN$k{!pv>f1nh!bK3;KXtFwJkmuqmV6aUguSg1P3qV7F>J&sZAGWFyt z(~bat`lR4JAkR(FJBPd1k#ljwSIN`$UxYopfg%Vda?WLfLYITfYoXsO)l`MXl!;$_(#>mTx zg|nkmks~{kMpJ_XRO50DkJUD!MNU0#VM~kBU@fhK{TMz+U8>BeTNnO=nii0zt4f#I zFYlS1y*r?s-icD9>%8@>9Bn8Q+1&V#cu&Ae0! zA=gYo_fJeinC62i<||vvu|nV0hbzhF_t209NyW*0o!Vmz-_PN4ZUzsgv*vgQ z)!_6)7|6UuL|wcSd${7>S9D$l2X z=rG2wOIdEV$KAjoTbaa9FhC(-mO&mkE}iTD%dpLgfr*d|`-8W(yRYZiTdUJ&SemLo zK&EKIZ}CZCrv&4Tuml|98{4>dLqsw_6M|Epj-!ZiEXtQRCN~J@~ z@SR@gC+`g*CeZoHPe2;CiG@Zc`RseL+1gK{$?mbNo!&Bee1h~lJ*R~qI8#!H^b1%_ zTnhBPBFiffmRO3dFxX-z0h#x7_7sD=*7CQ#yD1O8CG(%qi`WR{UH#ef3!^M6`|uY- zu|pSCoB``w4g)+x))vW_z@NlA5yZ}`%;ZtGV|aXHY3dkyg0CxYVUZGkRAI{aq5D?h z(hwm6|2jlcEE;^tl-aZ7M;l>LHoOu}+5IXQ<1XwYl{2irGpa+<{6VCWGidc>@)|jB&s()KAB+S<3mx}Q&Pr+OLpnI zU9PX@4Wr~L+IYi1f7L(GRPDzjyosrTDUVoQuRAnkEc_Of7zzCJOIBHsHquBfGE1Wm zE1bT3lPiV8sN0lNb~bwZkhDgoBKFpz-lvPi-SzuT$_h7rw*4f~IDqH1Oj+*DDZd@` z{m1<6V#^73YYKIgS|3_iqx|0Lg0VtV(#12}A8rK)x86o{_R#$v})miG-&ezP@x%!k7 zlzO01dHmecc5;OCAB?UCqs`qCiVtM z6(?Y^4Gjr}xf~ne`_`1^_*RTxTM+FQZ7!IM$}XbgXp>V2=6(P=L$|*Q^`cQoc3E@X z7)pUCzB7?$TL!~cj;q7j??H=Qgj!ix*NJyOJaRc+)5$ zAS9%f*C){Vr|gl(I9u zA2siH3O2aU0fTwcLRLom=liN*vzrYOsW94HaVcJMng_3X-gcnKsqk`!wMNfed(ohd zchTMDIGMP;HaX8vmf0qnj-6^1mG?c?HQX5zPqRhbS-Mnj*U(ySmUOS&Fl=86rr1h%N)P8 zj7YQAUT`hJ+)FNJ7~ydrxmxT22Bz2S$iQt8^cDBrwBm_&p1{33l)uG+dw1YiDQX97 zrQwO4uVZi1oW3TWV<&>v^qI)6>tPRcXW6 zO=}k~!fyh7p8KF;nd`k6>E@PFj%ZAu48bBgWMz;oA^BUSqa<5qDwvU{)AiRytx-ti zs7Wex%X-$UoHs_BuZJk^$I; zFC!5mg*X%w`1Ge)b~$SzJ6;(UsAP6+a~d}I9>eh zq>yG@YRP~sa_^+zEP2*@#!b$!Wj)7w%|sPhJRejteJSc^24F(BE1yFtA}4gvNpchn z9tc!6%lm!A#{gFH*qp{eDDQGRPO`;0XrQe+c&+k~aUWg70Se$WdkY>jq6E)?Cvu>{G#}Sxrgq5jSGLq|7=*~B5(~F&` zLt|^DNUc%3E?u&`+BMqe*oW{2QqhQl)u&;y!(GTn&^~jn-}pU(k8l!q$z$;>W41%$ z!+{F7DIuLF+WeSru>aBy&4`G|N8$tq%e01soq1KAmhl4Z)l1PIndeV`K261JEZS%R zrJ~&`_MpG|2V;u36$7xy#8b8M=ZfIYx zh6;AxpG}$n)lHebaFkDtb>;6nZEDew91sI#WqMu~)Wf*M^MxB#I{>?4f1g5<-yg(X z;MoT%r6~wF+U)4a$W2tbo%tn7|uOoUrV^(ul(bPt7IOr-icLk^W^BPv` z3#Yj&Q&)7ISRIB{GvAQ#C2z?jh&>ruJ6zZZ*K_5FB>zlRgj{ME$WOq}{X z98FtNL9xR$o23c;{ZM8n5IGyA#YwoY#;g|Fe>IV#O%CA~Y5N6D*hQwWe^&TR6e{Jb6_-{5T_^l3^Ft7?&|N5gdPoWD6B{; zvAWj-9;6djvHBUShnklhrCyWgKvjiI5_Zjw z@o~sd!#zL>3}owyt7{GYtku+??LAhcUCqPdqZ6st`j8~P*G7avf$YBeBh8> z)zkqSY$(J-PPVZCr)sgJAhMwMO%d+h!w{< zK$?}&liR=hHPWuF`tC#4QYV&xK*)gxDdS3OUcCsCV5>nKA!C?aZRL7A>j&R^)Y}fD zzSb_^q~0;2nnv2xsO!l&j7d|U?x>Bb1_B0;0iXZsK-+nT9 znfBB1C(5rYwZgeN?;b+#V6DM^GL+3cR(t<`q}x&yYFS<+qY$0RSAQ`7J7w;E)^4o7M%&JorIeQo$ny|5vzyW!uvQD{M1=T(W@LJ z<@oZ-yBG^3#(sif1Z*@oqK4S4R|*=qdsB4bFXQUslJ%$k)TUZ>ph$9@zWlC z!g8iFIc- zHgtuh_e~lNMV87(d1pd}U{b1(eJORMqPvOj`zB} z1LG&gN%qR2iucPUBbh?#rf>8--TE9bOlbY|G}>-ps%;$HN20p*J;k4I6k_tPwVdjI z0|(%Ixu0ht?80)LwF;X*J&nmYjfh*{{nU9%avjVQex&s^s^ora+}SPQ%Ao(a|2F&HVuc%Pt zF+Dqn;0YRwz5II}mW7^7dZ|o&o#MZwp?no zsaF0tsKm`W$H>_-dy`8W1D$-WJWImW`Sjwb)Ppzso?TJ-FbG#JJT*NWAdxHesv`8x z6m+?%i8}IMk2v~_d&Mx~D!G7TGRt))Rfiu^gjVItKVGd?{nqmwlQv8jUbdk(iv7To ziX%f`io;fSYTaAIE-L5#b}Kt|{O!5|dKn_989kR!1Iygze82Oc%6h3eOse~`X8Rye+|`nY%Awr z7<#yn#H=h_5)*prNj9lV{~;I8JQuE|WiE+_f^Df&sBzbOxuF9-xuPRF8xb0`4irAzod+<*624?m``RIC@ zp!D_X6pLtHr_VtbT02~&m5nze3J_TbY%)If8{>>-YG@MOz{HOB=UfN?8px6hq+ zes!FFe)}Z1ij}k<*wDEB0&h}@sToU{jiT5suzJSM2BXx$n?C@s6ZsfYgArfnCg(`d%=`Eru0@}}d zeKdhsP6HqR4F}e2G|T4DF_?IT^a(X0nv+j031lZ*#}j?J|+5ZA8A+?kPE6xX>?qD6P(lR+rnnWucE&t`$= zT%>*14~)-r1E022)mXT7l^UN`O??gRzbj|jfRAg#;3;L}7M#p4;P>Oh<(@atYn49y z_=tPy4)2NLq{`VjY30AGCF`g6=u8Z?h*lr?RXl4c(0gr0vqLbTM3dTHGUl@Qxp26P zfwY?QNnKwX#d14c?t^(+QIE2$P&NR9#-fzY@i3X=h>SPvqX&B*-&gDy)IRj-1~6RZ zJudC-lk%+lLRfn3*Jm=m#qY*NgshM?f4MNqZ@Bny$VkS*&>5`i9`Rg295Sm`%?HsO zJ4(*GIp^ZPfJvmpwyZ(HRu@V~KNZ3!sU9Ko0-KBrpVw@h_e&vrtf!k?xto}x5Ls>& z_9mBby1hxmPKk@(VCeH-TkP&i2_JkbqixV9#_@-jy=r>Fo8DD7r!RffP`luA!QUdB zaWu3&+r+v>`4L>A$lKqr;ppa*S)bigPNQ(i_}P-M$i%@%al^byL$p8(RhICRH}0I_ zTuyKeDljVNC-k~}k$;SX+#F%Ai4@(fP|-j}Bd{y86;wOxLFCr3o0`8+Av^Yu2$z-3!M1}YZW_pD)Z!0QYx!$03y z`x2p4S`X0!4f^S%{1xgYMzcnoS!2tw?%1&glj$L_j+mU^Q8$s zq%Xd2&om-!)0;4R`~*l}kwdSs6Y1!iqm+bgo!J4gV}AXKQ+d0F)QJ+X$nmCa>rFn_ z#qQeA_`iJ6hCURA@41V*$hBEg-XNfkv*HXGm=WckEDpG%bI?4|w0!|xfW{7a_XPLn zO1bvKUz_vnyOkneo>@C$7Lr=5kLJ($D+b*AbF`@+(X@}18JWBa^su*@u0kIs?E{*={jf-HMy`F=fVLv9^%;=@YgrJ z9A0^!|9nr^({>xlsXnfib1Z9t<@{|;O|fQhGHm^%Upy*)SAQFQfr0RpK)HTCi@Z~< za4t+;OmAlD$tXwr>Bik5qfdYQu{W%V-An8wlrr^BZB>!0sXSrvb%N9ql)|#*2znzTJ#xZ^m5W} z&A*nl&g|!)HRpDCmiXw-Fl#y5Tr8 z3tUV#7Ms$26!R3864KpvAenuh=A1R07!g&VPG13Um8`%bXpxDj>vEX1tlY*KMc|V! zz34|S=*((Nck2h#0S)eK&wvI+3%j5&Y$@|)Z&#IBn-SFq3pt)lSRw%%<)z%jbN%;T zG_#x?j7Yw2yV|U@CVBh=bCB{G;U>73a}7V5uvQCJ4Tkg)k1_ruc-N6roZ@y)I6b1R z<6hR5&+!&Et9|rRb{A~n`Uw@Gzf*_LZqb5ui_Gqu-@%Y2#izkw2MLl!1V9fX@BMTK z;sXu~|MN2AO~=>qte6cOL26hL!4k^;9faQ#m%zI1@mxb>Fc2?8aAX>jF)}HdY9&24 zhzB^G#WT_iJa}(QB)V8IUOL5yt2;AleJQ_E^vtI>#4;A3Xd9l|b#=}sEZ1}_BLJTQ zLf(UspzY$Bygzxu5Wwj#>Cq^16j#XIwm6H{TT}O+D%*Z668v$E>3USITiaeQ72#|j z!^VbJ(NueJ!~wX&RhkzHYJAN?$3;<3<`x!dD;z=`ZLT?~oZ+}x9V?I?mcg8nqblXK zE{kqjEbcwL+jj+bQwYsakSAXPIRArlY=@& zB)0*H2m5upfaCnZA7BoYI%~KEQSns#%AXffUGgLa{yW0u@vr`26y@H3o*LZ!kjS8# zmY|zsDJHMy9C4lFR@l|M0BX|Z=klZ(NMqdI0!PuZx3Osw*5Z4BBY6XDl-ynZcYRIz zzb_-D332QFK9}Xx%lZoY>@_v2rVBQ8HL7I4jKNtFB43C^lVbi2XQM@R_4Gzt(IMJrbm7p(jTem4f zFsEs*#;8iKOh|@RL~kxb>3fAUw)UR;?u^lYp88-Y#M{(qWF)r~>n7&{?BdZ})ON~= z;*)pWUetW4eL418EgDaR$hFBP%F=i{H{S(boP(d;$|d7)pGhQu&>PveAHCyFV|p1y z%WHijOyPun1WhHp)bxsbxBbs!+uka~UD@L8qE}``4qu|V&WpC^0*68!E%|B?0{U1| zPV_dfWb;mo5W%LLvWQ!**;R(BqxB0{pmGyf_1<3wmPlj|I5>s2O`TQS>XD298B!}^ z>rKNBctaZL$KR?_SYHOl?42CuqdY$5rpg_b>By2DkvIi{X3lzJzf~BHsFGM^XEC}=sjbQpdAL2DEM$)WpK1VNg&&Tr^WCiWA*1e=0#Wl}*TaDm$oJVy~ zvudfZj${Z#FYPV$Ir(FpHsPy^J>^)lHR}EY{=`I_7EGCSNcwX)hqCm<=`qR8IDeK! zPbYMH*0bDi5XAdi&b{|Um)DcmjbG)LQnwr0t^tw3Ir+u!j$pQE2Y4W-e8`n6yLMaU zGN(~P1V`dDpzdHYIFgPgbzTCO#+%J|9sIGrfc&{W!Bf9O_pevS6raFyG z%YeweJ>QJTm5-%NdB?>0uv8^oi;Ocs<5EqhtU{rs1JNt!kpKxpAQu1{_4`M}fnh&A zu6L`Xa!dK*lzKgdJXGgW*hS>Ya)_GR+DCqbUM&JuH-0n-Y{a{!3+TrrNs1>`W`(k5 z`Y!a{kCDA{1wSKPb~Rhr6(c&FX1XBg-TAG9k#(MItncRb(}$4lb^{mmdo5GilJwsU z>$Y0NT(_Fz%j(Y{;}j$KWD6Q`EaABGtn(bRm^uVEAn8@b6)-cK zDt&rq3+Okxr2LY0>#q341GTnu7AN~FX`6r{0Qb|xjD2-5UAxTOs}A~y)ho9(4=c`; zH6GjvDJ#+^d7<_)HAx{`%O2+qZ;ZHJ0eFvaO^QV?x$)BJDJi^k+Ik8tn|k8IpFl)t z+>w6EGEaO(@aoJ91+(4pZ~0$tp;IlrZELepwP5py%a2PibG^y8$n|!BAKtBm=G)_Q zK2q>w4f03mE_ee!yq?OaJJjdiWc+)ICMQiEe<@3O%F1E;HhljVD&wbYX{`rxw^axzM zpC>-ZY`7TLTp(%frfO2AE3`0eG`+#->+f;>88OH&EE-*415I!%TrT7~( z*b~S+;WSc+dAekAb8McskNhRtzts_%`%&3Go7Ha@U#-6vY2fOzYa^cMRlL`BNcak0 zlHeK{#lC;hZXl*fG$(^CN4{B2V%Du>GcmJfH#xGTH8A+wD_zBbB^6w)?FObXw`gJ; zT*CB=ug}&JSJG)V;Iv3br;%6#r;N=Qv2u9gZZP zp1kr^7tR{@(14 zTR&y+&2$YczxZZOWmQ4?-;tYomoAOJTCyc0>e@sN_1Tl62b)9#-!gQw#V)ox{=RLk zoc}-Ob5gdKp{3*NU&D=_u*&?Mp4D5q8&hu7%G<0|3&o-%vPTVHPe-U$D{<0|8!jZX}#)vS4#1q z#u+^Hxg&`B-nJGdq`=r>Zf|(leQi?%jKE)=vSlXt&FfXt+naTPjW9rp6Gt*)F^1Z zJB1#(st)(8FULn0=*CF{$;FgeQRsgynW|M2Bq?^``0&#W8~(luERN4Q8AcaXBXm3H znKLv;sDz}<)Xw$UW%S^j$y)=~%;|;Qi}PJ4r?(Prn?thb%h{yZw4QLBhcq1hkc=-< zd!VLhOZgLe=bS%TDskJ6Ef3=)DqrA-T33{F zDf4aW5&CK>y5J@-CI5ULNrma1ft%^X_rw2rYQSI1+*p9`qJh~b*aI4xJ}PH0Y;z!L z4fooc+zY=)59Px{?k8Y}1ScA2L2P9BY(igmme;~RUrPm_;J7++zBLl?>rF18#FlV4 zfz_|(Nqu@kh46$wndvw`f+>wK z;OF=ru7Cf#<8>zKJj*?N2m~Q>EcY%Pq2+8t^aV`{P9y(WKD|YCYX1)7v8aM4P!-~z zk+~SfQOFi?kCwB^&_7LXRS}BIvp*b-{$mW!RFG%=zkhQ0pFa<;;-4Q>q1^oEj>)aY z4Or2uu9qAEH;?``*rNaUm&X48`>3CEn~|H#PkDFe|CwndM_{5mB~aXqb!9U!C3y}a z!HMiU@R5Ah#{RvY>D8u)_d6|{!@yWw%XwaeKqtZH$qG&Xc_cIR5z8_dkf;Lzf92ZI=THP)F{V<*q%I3enzFfaW&J8Q`>L|HG zl8^CKtBklm#8ws8f3puQpeH&Z>AVRMBREX)VDO7H`GoL%DYi_q z`FgEo4jK&k1VYfiNb#FJwx=+_s)8CF20PQ-Tyufcwj@#PxASf(uJy?)o)QcD3Lj%^ zE6QH~D4?bNN#;H23q0#<-A(y_{1*kEt;)CG1OFLO4CTRrPZAJ;$tl%t!wE=7R-wJ; zvsSR;nybaq@#pLAe!G5eT39|h{Kq(u|7Nf50uT#~a}D(M^;cY*D50)mTmx@;a)Pn8 z=U(T2JI!mPoJQT3I~cr3+<^T9kjKh0@MVQwOdUb7#4Lb}Q^12pmy=f!Q}rI@2%|NtE6OX-DdidOE}wAsEqv?!Ih4R_8mr zk({hG^+WDZ%NzKQzoW;wz!%uf@*Mmkn@Jr^^Rw0>;CVQ7!Phi}xMY-LIlfhIhS{%3 zbFb~WP_wAbWx|Df=I!&#&fTv~-#uM`{kYg+7CpN6e7B8Nm1;OSveVA-ytkL)&Pz1# zJ_KW3Tgbq@78&J)#wRbZu;u_!dPwWD9DT2y94L2v zZh1m>!&&f4hFhWK!L-+jeeb7a@#-Vs_n;R=l4_J}u3YbX2->?Nm-(36W<3#1|SrA~^Mbu@|7D`A+@O3h9C7_qu`pZ1NF~M+Q1XP`AR#%o`>G8gnah+*C!8fa&h8p)QSz4$YL*IM zVY)$u)G&$$Eu?SoH6*MW=X<7uMc@aB;+_M(GBv1>o=xLQyk ztoD|NMw%bU=>v3avu+sa)L^uN&El2WMkoCi!YGt=7^YMLQK?>1o{?fk6!cJ~%M5tS zk7lo|F2hhsg_6v?+lSakY#&uA1f9w4$oq>Xq;*Oq-y_s;iy1Kik4#Y;t||u9&oxQ( zo@{5P71zTNL|M*ForXJaO30e5QlchLoTJ%@3`c@7M(9N?}?Y zPb5S|I^X4cd-hT{#@Bah^6LeHCq#8jiWCp@=%>K<9X<70)It0Qrd(S*P(e-TTCbUO z!E1e#KE$%lDOl|sX7d$xV0%XNK}-Yi7zI1*OdmCycd7Yaat;0jGQu5WYqqZNqcGBa z6sc8Pw&KB@hk(gWIb0yh8tdcT5v@VMo<@!q;+Ok}Vc8GEmM`(F?i_a=f zk{d@tyvbojdx*DQu51-!8p2tK3_lEQ5-mh&6(jphsYltL0_<=@Qspz0&Q-;&du4EP zl*RXq+NRTzu&8E)98+^YdQtSF3s15YBDv*E0Ro8rCT04i7Dro9hsHxE&xqA2y(V=j zXtftxbsPyJzZz*(kS~?hDCtX&A{EvVNBAt-tbAH|gvSCq7L4oGKIy;~?Rg#gIaI-x zbrn&r9>lC?!X`Oshd-93To z)2zCLEL!Le~s{ zBZ;rsxMYbj9H3Wm%SzLRUrWAky$yzl3NK+NUWyKEdWLqmT=@$|mM9Gk6O=$h21V-P zRA~@%O*YV^+QtFH(Cj(Au>-?74+q@n9W^NN==)Y+etF`~_e(x3erYBvZp={_boYHd z^39^rM$+gVR;CI-ox@UZYzzx`FiL5CqI3k#g&V16(VZ(9*FShGJ!edUgF%&j@{404 zCJ!8uS(pPY;OizQjlG#m8 z2QX2N$$LifQ3~AXN$16<*qwGSVpz~X+}8EV7WA&;w4YFV?(57OBii*Fm$L_=Y7iRF zd*+>953Iqq<$~?xLIjbfMc}wrsr1x1e?}Gs$1}^he24x=(@|<%#9ld{{jyDtcQ6qr zKfCKx`2?h5p6d}()1*v0jhTK;E`lxN>?vIAK zRP!U|jP_P#{6oG)7hsb!*A(M1P!&Cgf)tJdhv|r_f^S_px63~NoB(R`5>}!+uejIF zJWTf&o`O4ig_~dkkyHZZ%=;)3viwtVzvG{v&R~B^RvpLjYKEZZruL>Z-v{mn zTAKiv!35*v#TVF1ck58YY?^gOi$%O$K1?rJc*?S1WPy!DsYrl312MCY%IJa>-0fo2 z8JmC3vHhjCp|p=*^{H@qNq7nyjeC%X;bLV!^oHhtMc$y8cwmgz;o;^Wl(bjIE$mRP zt=8B;j3Zj$tYfS6oJh4)caYWYYLK+a(aClvuXMBMelze<7#m=#GC z+uOp(m>Xd*geOE4!k`&CXsst*_{-~YZ2_+cgqG+-;yqDh74u`E9{TEcTY$ix;^LQ3 z>ksOpQ@KP{nYY|YmdiaFaRgiihhP#bufbDhy>$XX3#zrmT>KC)njK{vplI=w*S)eN zqUi0bBAm-1t4vJZo)~5Fv73G2X}t!JuY5g1poD)K#&g>>Oiquu2)YjdxoV}odpEU4 z71F0bx?x=7!E0s5#D5)Nq$G^e&HM&6H67)W+};OsBFj)|QnVr#A?1`lr-;A%9OcZk zb6ZzQJZ4kZPyRi0E_@mIm`$WuS|i;831;VM_Fwz)Hn4KaugeNp_1>y4{Tv3~uQ|tx zM%P&V?V?fECpHX%kGlZ{yJ;xF8EG;9&Q%A4AnE1^tvU-Hd7rP-bl#t#!!C`%O#709 zL18IY_lGPhX%8gW&p~IvMsp%ZZDLp&+fX!p7L+`cJT(!aWm8W%o}By>Zsf9^RS@Z7 zUXMG&pH=^W44{lnUhDAdjcXe?o+14fAquZ!2_G=#-^HhbC0KNQ#_@{iW2XG??7GT>RBCAdQ+W({?dwW(S zq{?9y58~GZ&er{!{Dw45#3F*W1jawl1K807SqM0}2&sIZm@yVJ^xZC0dmS6QcpDL+w|N#^rk$9mVi8H~EB}cQ z0w(1Qa<}6`qEDe}(3j1emF&QlT}e1x_IRqN^<(LqM4#)*o6FZsrFz>UNrC&Bm@#li zK>kJTInz42`Y{j{@wemitk)Cn9wDx*glgV3Ni8+m#z2P+6`-wOOc&Phst8h)$kRkM zy~Zoq94DNKxj#KJP#g_uY5{-^^Yb!PS~z5(3x)snmWzk)A3v582Loc}8bmPm!o^B<7h6cmws}X$J zb}Wx8J?fLDGp~AYH_jy6n+{Akl)M?W6iu(lyPfO-A%woiQgHj+*XXlH+6PX8`Dy9{ zjr;LZ%pPu?rLxA0(Dh|*PwPErlAq^bE=oKC?)i&a+Ei%U2R}WxJG?*WNitwIc{gt+l6wB8! zE-mbBy&37fe}DYv#4xKSV}^yiejm+M{`IGB=VvFLb@$hOcoT^t63{D#DKhAav_HKk z|Gk0W(v$*)d&Ku<2-`L8#RdssazP@fo^R&s5L>1*0pP>9A$z2c2HRKr zzWYt0`1@Sk*p#2H!{T($O*#f zDf@k(>>355=YVd6N{)&S-G*r~B5#q&r{s1TM?=&qxz7t`M$NFOr@#h4`93%bEz=1{ zsje+eC80`r5}x5UqvhIcCgUGZ-dO?ULm)f$kcLCHOKUvo8^Y@5V!R#8_D$Y|pDY8R zZVST`(rSmQ<}ae;dffvz3r2o3Ng+9Zv$@)pPvy&mPs=&YEKOi2X!z_W8>OzQ%uGRw z)gpJ>2E>1K$(p;Z?elcJTN=y?LeMAZhZ}b%&@WMCC3TDb$C2}ax5kKgNRirbW1k*< zhG6`2(G`_B!!ik~NlZk5w*r?$Urw!qt3!9F*28A(B0&&=YY6=D5?2QWG*vH#q}{gP z!vF=4U~{;{=$!Axi|ymV+q4SDrHKk%6C}uISi8tQZ_u8p!CBuJ*IBa~T|7YU>2puv zagt#h+CEaXOddtfnzgYzxf+SDTGp%r5z(-O>aB)n&#!wQYV<6P8V607kADQ?E$=rt zyeIs3GfpEz*YB4@Blt>Y7%uym5q9F0*yjWs{L3hw1wXCm0HQC4$q^MQgtRnWpkXMd zxS=l6r|?SoRWj-ouPGi*Qd#QwH;zvpxaq_4r?`Y8Fz|r=y?V;qcD&}cj1Fv(mypIW zhOuAeq%nSJz`0=Aawhhiogpzcb^nZl?JqZzbV;@yDQ08V`2&GHS%D3DyCUQ$M*z!g z0~F6!9Xw#zt03-}PUO}52Nj+>9UWG0AFnU}lDzQwQ*C?^-|^S@PNViKE-&$)A81e? z`j_7C?vNwD4!^PcF77nVNqswMx2001L?@2;m}b7J#&mbK<*4NG*T)Z;F1EivJN_T+ zy>(C(YTG_4NJ>bH64HzAMp{A|1e8WPlx`42x?|DZjf8YaNh~@grBi8;j`J+{`+nd1 zoilT0&YUx6et-OC_RJpLv)5Wr-}iN0_jSXRpCwhZ#!em3ua|gWYU@G<8WpoC-<%yi zVA%+1{Z5CNjD26lh*`5Wl=>u!a%@om|G4;LqTG24dg3r`wKi4u-~fJG!U3MBBFCE( z0AiGs-v7w$AeNYlpJ~g2)vB6_<4GwkjXEvU*~Z}X;~6+57DM4e%=86Ad`256EF%N+8~t zj|^5DjCJz7y!dfvUabgOp__RPl@?1YPr zj*vXg9M`d&RP@2N01Vsx@q9^|_`d-7>PpP$%Sd(a!;a_aFD|!`L|pfK3~qk~1+TZ= z&PSgAoNiQ=)^UAt!qImm0#`;U{(cQu$uEgQnLc-oy17wLxlrbh_nQw~K>eQ|Ffan% zYk+chl;%o6w>`l$13<&8PXNucn0jqiDNz$0MWaGTlRcF6M?XBOTF>T_@HA6Ru2w~v~&dBNEExS{tPsVy&GlV@(=@YFP3_A-y=3tX3 zYy*3bW1E*+BEXT!Z7};KT=;_dN`;WE?S``#?!;vuE7_kPZ8?2Qt2<->TD;rJ%7hbl zw71P9pzf3mEVO+R>hz<8q(ziRI3Y}!!cOxu8?t=?b(j+YT#XWE;wg}lgj`u6@H-L2 zv~~i@bVpou&$ul{0T#V#$`Drwg^d6yTs#LBg52&W6*{o6Mlr_EFf@bfLU6NY$GZvU zWyiR&6rQ(wT`6Y1pHt-fNsuUQH?8NItp-RrQUdu+`>o*d-Vfn;@)Y9>t_G-i4w)jN zS7-C0@%tM~0T@fyL$=<*vUfYmZTM3^?#r3Rj z-(}Tw`NWYll}2xr-vTtzc?_~b@VZG}FgY`r-HjV74UliZH{w=Bfb*spbo;mSUKm~^vjw!zW0-=N0FR1bd-k(Kt zeawbU%*o$Q*BV1}KOm|M1RU&>;!>EAlmC33A$&Q;_;N{%<1{QWUnN7SVB}}3Cm-$) zsb=)JMTBgHGUxzcAGsqoy?$BCLAC0|A@bBy-PhPZ@WwdDXF7<`i&2O zzr4eO5|!vmYLukY_<+<3p{TBRx#Iq+BDDajTIc-d3){|ZEfdbTlI>zXlqV=KBTh`t zGc|EJrj9a6KfqitU#1cl&(~LBYKuMj+YzqlyB;O{$(DO#a%2j*eqp_0%rrLfu@lgE)20{q%>**+ z#KN;}CGrKzRsW)|tZDET<55f|t>u99l(ZQF*MZwyqIY&Ms7ut(liOaGsF{x=m(W0r z;#f;OB=ew`&wi0!)Ry`1 zC^3;50V*-qXPUxDYWG*$ZW8C8yJg)j-TvGwT5wywXbMBdxHFZz40tjZ;QYigU4cg1 zRlH_N&-JL+eOLPFhkc=9^?Bpbb>}CDJYny}_;>!aie&w;8U*?(5ZnW+My}Q&pdonM4-M#15{#>Q4`9h?yaZ)RDCx|P<+s=W=Zq<|l?5DF!RM># zQ?l!eJ?fgnpCr!;#na1@_khu1%v16?xKs*3+f8-mZ91twpF#>Z`n!c?Y_tt{nEC@Y zg~D5iC4|AF2;Qbto2Fs_>WI@(Mx4&IzP2C1EFjG}xY*yGtEEmk+pE9d5c+m~aKF~H z>a={}>cwTg*x82Cw)T5`!-a}Qo9f`lp3CR#($q7u=dNuZUlNL-fD3gUB7+S9IN2?* zfJC*A1@AJm?TKZ2+EazzK0Gyas3+J`ASoKhzV|DW^R$-v7$fXI0XQapKbLr=gl4yp zaU}77ZKmMNg#S8AjU|Ria$#uKe#H2}^ELho zv4Z`26txIAu=AxQU&f6r(l$HSrQ6>8pFSsiu16KdlHM!9p=coqeXmwCJfGEb(kMFr z9W3zWAuZ6TMdNBwP7IG*N(HJJ*l*jp`4;X%uNG=OnnJGh@h86^4jV!>(Hn8!aCTO{ zPwND5lHQ%~OrYwW0!29}Pio$8UYx@Xx++n^i)42giR^^K5_$;`Zi1AX@j`H{#%)$6 z&{bWgAanWwk2oG8XQQlo=d4rd@4u==VB(|KlgDaGnWkOGGEHsV5I39Ju|E2Kha6zB zo&(8Ktre%6Kpt#avoyFm((P__F7`fkH}={J{=TQv;#QUMY8)68q%z9+R}&ZwpR8qE zuK7DZ!E`U;>p^V#+U@f1L-vJN4pWIF&oYv9E$-lV_2rvn>6kjp*jK;_QwY^{!Cqz` zvG~KM<(7kQO6LoXTT3@X=~(+*pLy?;?{}i`8%gh%QGI?^Q9dSlDO>YW36Exy>gkud z-YnS1M+M6I-J@LZ=7)fr%p-vYT6n14Iq!AIDC)9116J*={`uHr%feG%c(!Z9g`wq% zgmk`nCCj3O9&{gY72gizMtRKWwtL4Bz%n5XBt~#;`YzXJuW{*oU|(A0d2VCsJ(eSd zo_TgeFk!QTk4~85+-O2vF4aw?wZ>s>kk{nFAEhw$ITYAJfLO(V>0K~n!r2G?%}Ibo zkO^waeVbHh$ z(jHe@**N%vzLJjYAV?sfpRoKQbFIPr&h5p5T>pThg!pHlC_EvGI8gDC4BT4~(W7v> z)4Uw?J>TXnZI}isPngI=Qmy>vbcgb`XxSZOM~rq`1#}O zf)T_33|H=pE$)DpVvT`1vBbOZj8q(*Bjyea8DpL)df$hNG3`rnkxWikE@Cmi}3gH!vq`@ zJj?S;cYYs8oFax119pSp+!@yH0+Cd730$#1rjp_$tmq0+ zc-5W$!aa8J5l6CXfp#pfK|39w7+bX z>e@#j;6!zfsdtHV-JQvPoJ`WvuZ|6;vm2C)rRG{hZ{Qyc#~A-Tf|DFpy(VV8CiO&$ zq^|z^2Z#-Tj{_KpUfugyVD7*V-4Wq<4dB@F_i;)I&@@mo1GmnBgEVku1o&76{P~*D z*t42s5Qv}elYKiFo`fxVohX6=CQ#ge3H*LxLW%{~-xUm=ChIfhyZym^=_hyzFMSAG z_`~NGUrY)ILG3cfdHDe}-x-GZa|n)YY!msGU4H8&Km;+ zYNzYd9q0wG3*3HrYq)#Y{~(d;C&Wz9VenM@66oKjq^=!+BhP29lz_asHAJ_r@*;qnU(RwxqXxdkA@?FF7EApwKw)HhK7O#c>J;1}7kcAB&Vd$dVOgVGyK(yIKO-zM!UXh#&CWgk6U3k%+cCqEV#vu%nNmJL#gr#ap^83_`s=&kU@7GU_0KpZo6L0{8m_(lHbeUGJA}>=v zT#u2iyW_8rJW~*_4DvRWSar(;S1Kx6SkXb2$o$3Slg2R#ZUYFG?`UZZB3JO+p2eM1X?M!_kd%6`K9 zmIC(!eu||HcDh-9-$4?V7dRa{aKwN{m*d*p5_1A(sO8nS{aA% z^S`I*r?mNoMN}Hn2J{Ppvsp^}@C+Dv88{l%VrgdULv@G$iw^ zx(%aXI^hLZ2&3&+aTUs$B5*gTnP~%`2t$x4hamzluNM}ZKs*6kiG8t^n*dkZ{)17j z1Bcm_LGU0m13g-jgSAasN;LT`scCd5733zcjDoM(wU9%1JuiNI41|B=K z+`MmaDN*62rY;X|1V9%GV-U5W!x`{StlizxRcCS!%JQT3BLq8{#WAR_w< z=sWb1xvqb#2^i!L^YVX7Y`4(7+`V=|X=usoBwAt_mb|3!qXgU$m3ZI?s8+pdECwEb$leD2+N^rmg*w3IycZe`2%)Tc;kJ%8$Tgt-`x8dz#JQen@m(r>iGPd%YmlMuH^>whr*#jdW@-?DGBs zgY&odzcXYv_&o0r(_4z)j9|6dHTUwOws&`fU5kF3yzpZZ5q@ za#ei8wsgICe#e~C5&cN|LbETPL!(m>u7?MjaO;zv3ZJUW={7M&+nqmQd^q(Vj-QX; z@;$b>@wq#2_I`JzclW35Zd#n=e#78yBlG?;^JMF>&qWu&F@f`xD|q-pE$XIE+X5g zqt8gA65kzg#V3uF^CUmx9gP}iJY$zMb3kCU=_1X{1!4mr;;|jeLDp^rtY5N4wVHgj9N|`H(B$4dKKRYvwG4a@)KP?O$T@be z$0NSvzTb^pOO;6Am~G>jMpbw5T-C#wX#f$J*anEIr@IGo1Q}UbrFA};+>|SwhD#TN zD6QKm7EEj_+H;NidmS)#ehqp)m6Ye3>wb=Ajgi$H!XmotOAhViq>nBZGMt~hWDCyA z%LD4+bdiru2;V#aUDU1Nol)@1kHc&*2_?xqUQR=1CHL2v4@SHkVbr5dGi1kG5=Tdb z=-wX{j%23B9xcx2GnK*u*9j_FAw9~V^UEaw;z%wqwyS!{t*gI00g;s9!;w0<{E~n} zRp;xr0J;27nE(ebG1Jktj3q^YKi8;C%|9{?e4>0V-M zSDT>k!SMXPgjmegSbkAx03Q|eUmqnDVDn92yDgs^((?JBFIEannnKh=u1B3pPkeWy zD=fu1bDyr+s;b6>8At{M1Ymw1%gbS6f|RPGQ*M}mAr-4=(y6$w;Vf<9b}8A+uY#4_ zZ{Pkbc`SMNKAbOZ%AY?O`!R4L)CG0j^9CwOTz=Q_?KPljw(mRZ|mR%2F{@ zp&#)vl=l5iPcpFve#Sdvhy7(k7b2C}y%1Y|J*P1#An;7*G!30!lDPi?IEb<5r~P{& zmJ|&C^;0%f*hQI=pc;6G2ozCYw0HjQDK2*-Hv7B@uBuvmL1K?i~Asq&1U#+!5&Tyc$ct_iM?skEPRAj2y}$H zUPs-aNtEODH6(PD=@d)Y$uri7^*=tR{Mo5|mP2vvn?Fa2l+&ubF1^XsKP#3*NV_l( z1c)sd)4Uelw^k0|jMQ18a)89rXcs8AU)n1UEsuk0l=w$GRLkXa{wc9F4B2q1vYord z!u(R$@8WZ`xrt)EKS3y!U|!9Z#6?`Rd-k0#IyN>|yVeTs9$Vk$#W?wEsuWbr|J+vW z2aBU2AuFJjuNKmoFtxC&qQv_Ek5~)iB8_iPykc62o!Z#Kg%__L1(DjpF~-g8sw>{$ z?aeON?Zm|W>GDIY9+(?WJ3Ajz^$!!%CNOrItJ{LSyoTGM*8!uF%Esqvn*X&2^;zgy)$h9c2;H-5tod7j2q z>T;t6ey&1Nn(m#_Y)|(HmA%J72G{jj%7(q%c-vEV89)1Du#P%uFWg#fDgN1sz%{gg zuOi(F{np8|#E9v?SB;)2(2if?*@NSmD`#gURVdWntKRYYx%WwG0J6{EiD^v!eWUHc z+Uo<)o4evZt?NmNm36LzIN{^wSM^(4Pli*T3Mqm1h@^p9DIewON7Xw!>}|I>jC&ex zYhSZOkPB;HEzswKZAR-4)|ioyu>S}lUOcMVhp%d%;@7f?*g?7P13VHUVJ>bVk`Q6G zHm9x1cd|FOu)qyRGYrVRvoXwHNQV65?}!=kGOqIMV8&9j`xC96D&oC^rt?{X$*Zb=q$*<|Cp9=)>mdUJ+ z_*~qE{A?!PL`=hUAQy1za}}!Z#g4TnO;W{iX+kpz@=`>OEjAlUX}nE;{m_0Sribyr z=PnC=hct4FR=Qj4!#?QzN{5c@&Nm-MJtkDR*gH`kq=c-!z3Icu`;dk1OH=y`ai_Rzz!R}-Mh880q@jPMRX__A<6 z3_?QCwc`|&g&juu+;qh#p{0X36qHc^(ce2?@`)-gJr{)ck!hM#x$W=<#Sib9R%|R3 z;PYObm1V0!&rTn=CCZ+grEfRc0UHrM+fG{Pg0E}BhSd6cTM%!?Ol#@;sey-Foa7DH ztPaO%*M+MUKBHOqwg=(3kmp*}^t(JjQ#g5B;+aG%_dawFk$K3Ble#qV;TDwRj^^hYvJI^~+WC$L8wmkSveffLA2bVos@#YGgFc}Znu zCGG()o^9Fvla!O_74r+4jA02oqG)W-v4^jRzmGLk}gy)r&Q1f+o)~I#*UAVC*!xvDtI$> z71*Qx_hm`i!qE0S+#?qKqW{)u;FxNki<64a72GI&TK9vmovPeAj+=>H=YyAHyTOWO z_!ud5_%D4^Jxz=V%S0xpE_^@LzjLniGTGDn_V);Hn|@og*T3F{Wyt;So;%TKp*OQa!waR|#qRG6)~L{9*Zw=OiKlgzuFT)>_NYdISm#|5 zt$^Q&hl?xNrs_j1x4@^}@mi=5_gI7&wAqUG1Y#k;E1EV+Wvq__aP9L0&{>Svqz@-fCg8K$ zBp?gzr{ML(z?v8YjOX?KMnzj-3;S$m2bqmz?44ha`D?+z^Mp`z%Anm~_fA^62iQst zu@Lpju zIzleH4WQDH-HY3wmC_u9n#5~>Gv}b(ZU^NI?-N`xNgX2>m6F&IXvZEUb5)a25s1-A z`=VyT3%M3nO)!a)i}3fxw@7_dMWqEnQwGemX-ya`#u!jutaPLwy}_Z}F7;D5#ZO!_ zefMq!6a}d~@ar~kdQgv|n2XZ#5(a>MBLYxXRkp6T(geTLH9&z{hnQQ%FA%a~1(sp? zfS0iK&vX~ZVB^E_pBcx2KLobY{MvxzEPZ}SfM*nJg8>^pxo$$p!=g|00ON}AAewoo zF1ZRS(h(OwS@kn5o8a&q?&gAE96&woqQE7cc~}H=T>~w|xbZp+ak^Dr(Gy5?jy+mT z>qGL-1g;N>6sVP`jB8pd^wF5;?Ry}akM!3!>(EO@pwM%14>10>SG z2?E>gp``Q&#K+jcenPzw`4BFOnH*cuRzk~q<}Ap63D&dkJFP4b4*FZt-=xkp*n`43 z4i5<=JCcurvT{uU;Yw zuhhmnI!sopeh6g-()^bT_01QpZ*g8a21ulWj$3wKSZ-QXk+Ks`%$k*H>tARk-;kiMGgZm4q z`ty>?<6LPVGzaFgZQ6)PU<-Aq!T`ze9r;N-4zeZNN1&jhGFBOVG0Z)~&&7rRJfe@T z&kZCxU`G;X%u&gpn$X&QWi$bd?81Arcu+u`W~svQ2%gzqi zMgsaXODFOjAUUOsIBy5Q|N5dLm@asn*WI{zXD<4HhJD;QE@H(6J6>^MtuxNS z;fDSTzdPuqC>944!crMjuILcI{X@JYf_Nz4;4JY9EI^Sxv#put%a@=Q4~70oVKluc z$&(J_2PJ@zsmscGz^tc_rGx^4(9N7dpgShR@`ci#g}rU0PQ3t1CBUh@oIeP`hfv9! z-(4Tz-(27kG#Pb2`ue~^cLonpdR07kVR;=S66#PWXx|nY3t-}~-9X8PsFiLCk>@|n z6qLTLMGCQ#wET>@$70-CFETFnhXPby$D0>&QkB!t@D@Hq+UQMeSyW-0%Rh%)3G;Tw z!jbSOQWvQj0~OQGvtKXzO7DB>YA6(eoq{x)&|-8DwGG!l*lHB;7Dss9IS!mEs7ax7 zZ{2`uVtrksRFh*1gx>e_OLs)^!FGfV`yiG8utbkm%NyS?@K$eW8M6N@1k|fMgRg4ujIB0^e;ZTkF zA{I*BEN;y%)f@h^*)^Ai!or6ioABm|P8(^Y&dgq(qi<$AUeB%VJ0Ho7-z$dxu7{5?A2BzVHsIvw36D^8k6}(=4cI9~MKR7z9xi4!R0XKd6iJ z;hQ+cf?ucYdLViD2ySe#Pu^|MFOb?Lhe58^;pyK6E}w7|H*xn zwYhn>_WWK_of}S6}gL z`DNj7!D@7NUcDz`!%;7eW!Ja+S(Nwa@oW9K>ic{z95TEQg#ZnC*iy;r z)|Z*>5^0w_orn6eSEPm<_cOKK)6LKJK7wrFU2g9$jX#|*p`IuVTeet`9z<`PeQ(YJ z#xN>to!#-P5zuT<>vzrU>+kCW^wfoINQNkH{{S^B=qCU#cLJO&valRdAV5##pEcBi z_pc%!lpq`*1Kuf1Y5i01cFrM)93I?@)vXSz+SqdjTLXU+kfW~6)l^q=u4mDFieNer ztNi+Q$gGb{mYt$|cC%@& zUcEqp%77<9R@xERj`Z$5PS}p@+3&ucIg?K*WISsw;PvU-cMCNqZI3F8>C^*2j6bbK(1ERe*OE0QiG$3 zv(=rM&P>bUerf4%Y?zFJ*3-cCPizE-2=PBN+@4GV7XW~r?ubwyEGjiSddsaE)x<4HbXf)^ubT`$Zdxy5JT56Zcr*>qVR#hbU{74SMhZT5s+S2YN1K|y{o+p>b9~}N z&ywGL==F3H<12==^9U(j;0hyh5z`iAsw*E5qt5b^g(1#TBZ)@(>{ zH`kpjs7^)$|G4c8v|Jt)pMQMawP@sIi#deR{LZ*VOEC^wTvNmSHxQqkRENLk;8*Z? zd;}G{PAl+s9oFxfBk|$LkK*h90G`BK#g6&r>r%*{bixiQbgnN`Q?CmP*LKJ17jcwC zH0nqeNOY@-cRrSOfwpCtIu4uZ)!DPNvJ{RJ<6C_&$l_x1*1(n1NdbJ4XywV1RdXZB z{+;a+^dT|=@W-YE-2OH`Ta#kj<@5MNlHfNyYXyP?kc zUuPKzi+1#y#!f(b5Y_zQ`|y)4Cf#p8t0I;oxLKDfx1X}}e(6LAm9Bc`{C;FTWF^?{ zo$~w4dHjO{V50b(mX;PIF@Mnp!Y5C1>$ktoPY^NvMaZ)K*VY054%Y;}(Q}>}@wq#( zu4v_@rG*y=-M_7kfr&$m2%p?Pqj|Y*f=2L?^q1)?ilI|>nGKtxcZ`>CM4J`{_tq*F6l&gs6RkX+>cSQ z41YnJB#%M<^eZ!C|FD5RjR)UHeAtQoF0qz7inVv$M3AfPc0BAw*x*?G-LIh>2@`Tw zeOrg)g8Y%}JAe(Cz+4UVZinj1r*`K%1imsChKbQnADn&f4FI|2#@Vz|t-p7Zutrrp zxVGYPO90VUFj*y zwZ*BUI?|&KIM2ZWS}>X_ z5pP3c3_1oda4%oLuVL8ho=6^HR*)vz&Xyz|kn&>g1VNM*>WmuWZX5fH+YQ1 zyMeNh;dbr}4rsmQgxPGhsg2KxdUj4u4$zu506@=1`7P!@$QNEDv>4Do6M~E_FI^DH z#yDp|KWP)7M}ZmJgBr}HUyPNS0K0q<2*Z8jyfYravpfm6x&o4HhXl&Cs`W-hQq+;D6g>a#21kuB_-uiC3m6x`(Z;Rk$A(-*#rTcL3F$%J^@-f zgCqaTWR|1E=zanExE;`AaUUD~sYLqrWBFs7)v_Ba3i4+PcAQ!olRDwvf;L8Fi*X^j zZ$3Vh2aEr4V!uL&W&#mY@#a9t{esiBz^A)YRK)-L=G8tHY4cQr3bQ2OS>fH4;0f(B0D;2RSYpv#T~Fv!y-wz|rXg=3b}{t9oy+eeihDUJa>?r!fwyZf!%-I9bd?_BSL2~)oMXxy z(bX*49>~<)0iOh7Hnh0#D5S&HwF%e=u(7(;Ucc_XPC=h@P|VvZa{;@S&64DatC1rG z`MkpbF7BK_tJ(7z(oAN4{l@qt{hd$CH47>yAw9;TSXFqN)GT})$3*uF3GVp6_c>xm z=i*4vavd`6|IFJ8zpk4nRI*80yC}fv>VR_-4x4O0GcMe475gtRn{GE=XA>7}0;68n z{{ph(w$qq!8&ZO+F@{f)YE1o>1-wB`ZX- z3GrE0Xkz^e{p_PZ!13vaoQVR>GXVC0gnYfQD@ z?S!p%i+43UE!u~fY7eH;w;^&X#&4PgJ*nNe{3n~~IlcII8Z-Tv91V`HuF1Z7_3Fc1 zO|q!ZxA`SeRc;_ngI_6QBv4^EA@G@ATwu25N)9_FY7ucgpNS;|f?|iEj9Pkq&|pBbcVFLQ=k9 zVq4QkXQ!E_&+iFHrH08^e#_wNplO$yELwxKobLDJ?qOQ(Eg|ZDt*UE^9Sdc4ZZ%>v z0+)bg_{-Fht@Q@%!Dj3I5fueVb@MFLM1X7=UL$?`~_BD&??KP*D)L22wl2$baoh+O7il?d?u8XdxL3AD<+h0 zd0%n~8TX?Xq&&<{4LbGRJFEPjk&FI=?T&rbI)WiqVfc_XQCC6;Psg7E+lutP)O5o6^C%-w`#mDX7r2C2+!esOq5W% zjS!=>c?N5@E%8$(%C z`dz}3FNA{&j2`tg%t-zi(Ne5h>r3XfOLj*`4XQFU(h8Zbbbux>HFa~YIN+zULd;DO z1NO!*>+FLsuaaqeBDT9*@77);d9PtQ&P>je=UB7sNIBz$~c4_*M;*Mw+&TO}8Qrmko4${RC+RMz?X&z|4kPAD=NKJ}AZzagj;w87d`Nvo%}t%9u5@hT#p zeElq-Me7StC3tddtp+3xT<$rwt+JZ*y<6b1ZQd?g05+n);;8AE>=c-G+3%5ijyrJ! zN82}ay7_8TvKK{7w@W#9v^_e44NX7E;`=`$pLmH|KmJy~PySo5x960<&O4ouS3oE( z5et6V<4Xf6eI}2!=0hKZp)`T7_O?oTA)v>?RQuO*wPGtuu5aRNqo#>^W^*s#+@>-= z)xxrhz8mt=e-QPi_6}LxAnY!Kgj7rzg#?1=gX4>WXdr%QUb`M&CaJ$=Y?(KtsAS+Q)$*+FJ;w7o|LJNhI0d9NKRSC~6It=J3>Mj} z*8LLJaW2lpm}og;b#UHgiB1*islGK}ekp6VzTPw<)`dpnKB36>2Q(xt!jw(m`BZJi zX1`VmlD8lXDbQwZHl!3wp#A}zXr>Eyg{xJ7qUc!{UE;dGD!6`?+3hn$qPhoxyW>3E zVEkZ72oCRC(UE{C{27oJWa(*wJ_f*TKENaMA0>>RRyvPv6@C||+urS(g&To{*>XkG z(aB7|fj|dJn8*$u@GwC_eChDPhw_p(o3Ty_E-hi z29(0sdB+MYD=}?S*4xAqMqIzts`{Os9cLf4A{pm{MY7QWik|8do-R|0dmsl-+-pm8;t9C>kn_=ZdIpB4pqI^LV1KIm4PQJE@TeI@m}KegOLnwjw)Ifd zS3f4J4?TIiuLDaM9AMDzlQqz7?nQb)lK3zlMB3k_G{3CY4HVs!lzyR_e1+Aw3(^s+ zF@~N3w{S>c8n`LWeosf~jE%Jn>Q{l(2QsAZd%q*|P8yvk$>b>xT~j{`JO87(FRf>h zgYN@kqKE2;hZ=HrOj}H6%cGQ^FI0c$5Ugk=3+dH9naP)bGRX|>X~QgGPW1>r$~-aK zw+z<$LmZRP&2F<_5b?F3>#=R}Cnmh@p8Q1pxpEyo%P^C$v`|Ez+~Ix zKg_)_yPb=|Nc^UAh$`Vg&&NcJom$qj?lJ1Vl@8Hd0y-`5a>6aD0P8FSN-5&shjU6v zNzpGAD3Y-LIC;x!GP$$(NgYU%$dPvuqS1H{00h^Jv+UF*N(VKm_N4 zakq8Q+{v8hd~)SpEnhcLC4tD6hS58-dht##Ps32R0e)v$J5>Cu7nOJ(!+o|m`koMR zcd;TR1Q!jn2Lf&+ecu+D>7YIJnQ?Ki2j9)5N4+jblEvQKiZ?O8C!r(Rz&%7gLF`!A zjqdqeYJ0maBN{v9vaFw{Q)A()95J#*0&s=AS5QC^1-8jjqh9idaXLAly^SPVkbar3 zR;(V=baOn=;owh?hs!Qmw@8r^JK?|dR+==4noXu)M#BD4S%92uIHEU=+OsdnZi8gA zGYGo!^i){=T0aKUncd2lnT8*ErPLX$*v(R=U`oq#&f~?d^U~`wm{mVj{H@?faoS;5 z(@n>iqOYVHx_3sQzDukZ+Iq?3Ks+S*@{q0!F{Ck|{)H1e%OOL#iY)$a0F>eM^^KK9 zW&XQ_bY(?F(~VWE;q%9qy8DT{3M{hBeEG+So4NSre$9qo>p^o#Zh=geRc~0}r){n! zS%=?x7tB-QpVlMpb|d^ygKqBBe11s#BwU;5Y*7;*207g3yWhp$l+`{&Au#`m2;Q6m zlCN;P+RcO#W|)zcFp$cWFiPPNa#7==ZA%d#L7;JHG)RoAP}uv(fo?B3Ci+LYgq5VS z3WsV64XY#Vks8{)NNCYP~vA4(EcXx9x$xf_+5d_Gff zOdw5r-()3QYo-&g%=XfJj$nSow{MG>%v08Au&-}>zP_Ty=<4(l#Iw9;G7S|$JErUp ze)yB$f(H!>{2NufFr%?nU@;6d13@B(%rGM4ndj<8MwQccA}jI>-}*87ewqw11eyoG zkc8!mhu%Kuu196H1WNDUD_;Gf-AM7jgLMpo1ziBOU>P<6~JRIz4Cv@_4iv-7wsM;gnv-4GkA|B{HY3Dysd2i>@*3ht$MOU$*Z) z(x+FN)i)75hxMLc%Zt*|9O{;k!(PD%1}+iEpmil_`VzSfj=3kQG7yts-}S96i=b74eS4jK?*F+wXml5?~Y?8U4+j=j+fx)E)$yY;Hs?dN} zPB(8F={);zE`yMc{?XHvj-bojHsaD*N1H&ZuatX_-s9_%eb5kTrFpB{N&3t)H!>0n zhQ}NAA%F;j(IJ{n*(0Jhx<%7t*lO1Q;wS|R{j`6Ry6A^-1?hUbDhf|nX$ZfY8#ARp{Q&mD-4{)$gof*ZeETI6);Bca6L;d{5Pj;R7QSrAq7T zYsW53U9kt(!_M zxWOG|5qBf}ObIgtp2dmiU(70IxK$UzJD4_)N|~$X7YOE&lo~ZHX2%kM`eR~%o@yIK zkD(>--7IOkoekwB!jvDL^{aVH=I^z z0^tUbF~58kD3rSJ+`LY0cLrPCR!FJ(JtIqlAsDg zSYW4Dz`gTeo=&{4G+mUdXcZTbHVGs>nu+22&O_DhK(RtJL$i2uMOwqCL=m$UWGphaBl97f8H01fw!d6-v zwsXP;F*e9U>}OQDn2L)YT2wXD+En7r`pBf8F2IvXPEO8G{qc`WQc8-pR6WDA5O3C` zo<(;NJVMudjnNJz1oO%HFp~Pu5?~>dVVj zsBRdMa6%B*wJuqvzrUupw|8K^w5Zy@J}JiE&ro60qoNJ8e$gFug^P}v>rCR zY5pRmF`!?lj_MHM4ltKoCx2QE(^Et~21eY?{;{P}SRqui^etmi>i_(x{y6HpB==CJ zgI`7x>z%^Inus0Al*2EGAY9x*)7VIFCg>PT8mTx6(!O(-s)kWXCm5~=%}qt){4#!( zgA7l({tm80u=whTR;4Fgg832#gOwGsTF`JzBpK3h1R6<7c=TnNBR26^HdcdK&DFCtjbR*>zhdQC-wB(%aWpoaRiE3<-c9H< zY}ux~X5ff;z1x~QCA@hz;3G zX1^c>e4$_EFjjn{Ce}$eAcT}?IKCdykz81h*`@N;wi-2JZGW^_`k$F+yrU&2FqHLu zyFxW9h3(&^VH-})+)VQoZ@kQSfVbC-_Bb)#(O3jQJE~io`N82Dk#kuD$+&<=A5m+M z*=Tk1BUjOw0>yhA8)l0iyHz(xGpIHv2ws2YQVCOrAFl3F>O^Nr?c=Q$`F`5#L(&IT zh0xa?Kb@yHwzdWm+&M|}Vi>NSl62HP_ip8ADk~SrWMs`{n#tF8x~y%zA@f@rg`XA_ zlYKB%T++=rW;0`=9#-Fmro}d|iGBRnnm5GH2w1*|OaF;x*eUu(-<_5ts&DRlY{3ljxc@KJ;CF;l&s?MKooe&HN?MJ?$HvH%LlEZP{ zUwB$1DM|bE)%(6`mfBZNo2;%<&sRXtMp`D-?#UjR>j6@!dbuH|W zq-Qf>oMX8zq|4R${qrv6hcY1)y-1K!ZNYtstwdMo>-fM-P5KwE5>cX|2X^pN%``n^ zqoG7YS;5C)=GCL`C&BD4eEaA*(a>KvN*~eMBDYE-#GgzrYs`NQJn$yfDeazS*>SQ_ zV!lxaKi1y#h?q`yu<%n+k|V6%=&EOSeci<6)eX~NpWD-?PeIm~pDk3ahwo?l4EYbo zUi_2gCMm}|vb4dFSRIBD2713?{(Wp=jep&9s|J{gVs6wU>?qnakO|&x4(|=r%iw0_ z*+J|1hyW;PI{(t}A?H{}ygKt@bvT=fYV8tJ_c(R7k$*& zFoK#Old^gtFHI6py$l{)bvMB@il%hjS3m6az{kgV!?Tl(`J4BNUOhdqdQq>*PWPZE z@Sgr#r`3BUB)<3XtVcCbBUvLGJiFrXPbLgD7nY=O!Z+cVm;~I|H-HcfvUuC_0*rkF z*ulCX+z0l^AGvPLFDA~^afaVyVm|xR!*07|GMWZ#RPNVgzf_GBW4H|ygR%V9|2b{1 zui=DBoyBis(uyuaXQ6U9$R~w{KSTHs`*PWVzxOj8r8s?$TgG{Qz$QhPs$aCba083Z zw(2XDf~Gg64JhTe+i>2uOAsAD7|&AZ_-D7o;0`L)zU{ks+Xp4Xj`xTbR}HRdv|L~T@8xjT=F?C?|=M>TzLa8qmSX>lPC zaV(nT=JNQp-qE9{0csB;&nT)K2bF{ipUjdT$J~#!K zb8&J~dH=+rXsV(;UbH2mwk@gE?~i554wM&wcc~wRXeentkbeXPkOm?%xFEaq!~P-t zAIpvZTzvfRa-~D(4~W<4KgPk3_b#r^U{iZE;K{+v291N0f}P^=K}d*#O~TsMMHTG) z&cWW%0Ro1&Qt(l*y>qa0aMo}%H3w6$NrK(2&A~Fxrd||m($;paU}qq;Gj#<^g3TS? z0|2sMYfCFv3N9W_3N{TlGgmK1pis`<)DlQE|H|m9nAw2MT`Aaftlzs@QSfsEkcwc4 zCGgC_&cn`6AtHkI4~L!^N15o3FXvN_$7O?PkQ69>*Y*q(;SvW)d8Ym1#Hr+YMw(7k zNr^(pfzug4yFt%WM7u!rWed3Mi?P*gbiL3()qvPmihWJ-U-aF~!uMl$R>ufvb@GFMq z>+43bULfF5*hbuc;n^%{A=YU@&<*-%sFSB&F_LKb+#(bm!NfP1RIN;VJ0!rI6@jkq zXIY9S(}tm?`vgJ(x9Fd@7PCV>0+WZGh_c2;VG8w}RVm~G2-fmz^iIP`yYUir-N+Q= zj1hN3JVeJ2?B2)-vZ6e+1^TH`7ROHz_WCM=!3vu87s?2+Mr7+0pu7k01q2~hxwT)a zK{9l76&GO&SwsdzOy-0r`$DMF{_Y0}VL(CbRpzFNqYiNZjcT7(gQRCPdcfMt5Bvvn z5xm6+>}irl5R7qpi{lpvUzPu8Kk>av$KZL`uJRDa_;b#i-b7D{6$>Q_<8i>ljAg;K z1i?360VilqLNTYKPfn7^@`gY@e#@&@3AQ&n#4pTAs>5wbVxjLRy;tXmHBa=T_xyWu zh^7N%_=*s{6kfG$Rj8QF3H!E^b@03OJyvl2sSRkQ{jz#ON2Fcb$#O@H8Nq=`G{}NE z0OAbGfPPF5!S@MT-q9O=EN9qBe7?!@RdS4U{4&n_`;qV#ITBLp$>EQ`i$_4(XW6_l znM|#fj%64aoLg%iO&?on<3D>Xn^{e^X5CUhpd!GQfP6$Hc8;gD%>5{Gb3Of+6Q1QC zwfyIE^h9J9^5%Jv=XIB`G0CCj7Nlguh(GCAuC^*vTSufp!}lcDyyUxcS&{?GKHca$ z)6{Jyo*9f7kM)~J^C5)ogx9>Jf7Pegc*KLRR@4YEVz@p9LXN`xR*`6P&%_5h2F0Kk zlA;a@ajG|X|ISIbryp7po;Yw$R0vOwgWkQ6-#H27qeKlXV*aaS`E|VN&Fe4z@sFq= ztF>|JLf9&?u?eyoF9CA9+egT9jQ7X`9ZiqOc^&#XSZKR9q8KR|zZ5Y%A{bo`FL?Q& zNEPPKNO-OClFK!sl{Ol(dUerN++QSl;$LQ~!g`WTWU))&r9cL2ZA(j7$J=)-Sl6CE zg1UTpkYs-c9RRHZbSeNe+7g&Qsw54emw(T6=2J5O z&P3XjC>`Yb4mT>Oac3T&aAo-dMtLm>RI$~O?{o7|-WJIG=9&N_u3}TW)ar}Axy$&D z_}ZA;I8@W2yZ4(criy-kEmU9rbR-o2L6Hl4E5)&O&2o~&?rHp{t?O3(;@L5MF)z?K zjz#Q*Pk++UfnGQ}-IkNs`vp(K1TrIct5Q$@7HQXI;nZWlq}&Sk)BaoNLGh{r?b&ub z0FJO&=r{%&*&Tk$V`oo@s(yG1Tf#f*PhJHC`s@x~^cSJJj%$xE{?kAo$Q87Xax@+; z4s`4R05oQ+k!|@SI}py3Y#gnOI6$G3|JFHBtp90PioZGT$f}1R9*2XT4kq(88a5lw zNj3%UsRQ%J#aoCa)z0XEM?zC;I&)ojCxu29dF$hlNF4BDYP8nv;zjq|>R71;hKjS1 zp4e)fJewESPo^QQl@rmS$5OSGDF8a&=p^YpQGCm<_h4UpJyM*b2+{Hi3=BD{wr%@N z@#j?=DG%yFW{OYc`Trh>J4YWWxBQ0c5zBdK`revw&%dQ`7^Dhsy_<;r=sy5O6#7m7 zILzk5x30OpMcTN4^faNITz-&he6?*}omFP6N03;&2Ca#zg)v{Y8y3zf>;P*Gnj~P9`J@u)1 z-Q{$aPey9wVG8;Z5EUWco`T!yq~>=M2kEycK8|*K&6Ku?3)(G}#7Um@zF!Sai-*GB z*pj2hq84-Pd!i=o&9fE-tt_{RGsxn_mt_OI`nHRv4)5x?68pCM>?U|#bp8OMinjT? zOSALvT2QALcDVQbXix)R+%KlSoM%~*ej)~)d*1x~1JMf4v%1Uiv4TcmA~={I`=|K7 zCA6r`70tt8r3=FZBbCO2PhtAWG`n+MQU6b#FoXW7a1Q9;3GB)LF3H>KJ3QXDJP$Kn zBl?K{4|YOY-X?DmHclQHVXw8a`mX%v{#Wn(eqA+J8BG{+7n%HvS*h4hj@9vrz;thTwI!V~V6X89=U;uZOR$Rw(#00`v8r<1{J_%YoUS6%8nBHopL{UCg; zO?Fz{IYHe{zjKpRtNo^`z-z?C&x^@5-3CxXDSFFf8#zK91L$5!{qiLzJJ3n0X*f?I^N{Q>^094k@>&1L>JI5k_yNMJ&$;4;R z{xuYj|6@$4gY6(A(vRJui0^E!hXGgvl*capDgj}R=mKoVUz)D4j zR;0=_^sHn0Cm@KA`10qN&2Lfi2K_xF$&CJSD&D&PGT2Bm#RjyeTNse;4?5yiNsZ_GK zew9?YW!5Ufwk?e9=}7098%E=j2O+6bay!!B_%gNH$51+-wbsdL(!nO zbl7=Y=zYxqf@Fr3n;4v5%%=QgEana8@X zuW7@=BH9LT^K^3`&r7p!VKARHZQNOo-!^GHzKqz`xLqbJn#bq~ zjl=w{S{BoRwtj~-m-nj7t6d%{K&PoCq1`139GA_UB>T=vV+_2`&qfEb1sr;b7Krn>UHPB$vG?5U?k!BB+ z0!qr}(X;mKj*vObyyQB9>8oCmo%g@RwP6gSgSg3-<+-0uBuXK-pGh+!B#XZ(LLtU> z{w6|E_T}LCUc>V3gaVLzL*;{`lYA^>nb)d0Q(0>9INlZ^ir2K*UV;cF|3JRV^=vA3 z;$Zn})XY!6Bw^}IItNQB5ne$hkxNCdG4~ha*?3!&T2MQ#u(OQTNLK=(ai#-zS;4Rt zy;Oz|HrDHAr4=G$7GaWZsihEwtweayRw;x$N)p+ZUXKZ@u{n6xeVwtUNwQt(b)#vA z7qU6XNMk1jpPD^X6~;;)m$~MZP~(TUNtlz-6d6TQHu{EWi0oZZGC6x!A68 z%IwjXIVF0SXV4&3@b28Hxaq^*$)eMb5Fm888Aa_47YsbF{k9#duEnn423}8_bpJs_TH7y3l(5A}Q1n3KE!wMM$(I zUpE^eS2dCpjU;s4tU=XvHp48C^kUpwTBWFaiEKLWnFUT1r0gIedyZRK{U-=gj@^=S z@7rRFK;rctt;4XXJ;0#Z#8H7n@l9=BG=yvW;y&AN zS3?F+w%l((hK5lvOs={Oq4Rv7HY0gj=oPRsmk>jCg63h&^(H>LMT#V z84BeQPS*lLQschA7=#dK`47xr2iJzl==a8ib1nN#@@sPtK3@!8NW)U8kH@^4I5p*R zB}HxBK{>+-_AdUag0e+9yRMEC`k!!NhOFUCQLi6Z1?APFM;;89Ec^F-hygc?}53xBW2JVMP4w)y=P z0xsV!C|$<8B`!;Ly+s_(aqd%Y#bdvfZ5Cb;>EZy1fQ%$rc|OF%&#{jhy!6~(pvZI+ z-EqH4VlPI4Nk0MBvB9>aOL!8f0E506HN5*;u<&6bZ@sCb584HdAkKbrsBo4^+fy6Q`>vYk$*fk10%zc%nkiHrHs-U9>s%zOQw{p5}4(X*>Y^ z-89s>qlly44c~CcVz0*2R~Pr3l2A*JbH*IGDaWM7LATAua?oE*YFrk!7+8$a?f(6z z*wL31_t6Aa+FPlh6arGu-@S;u`O{DxU-BEnR)1LcYjR1uO6MVUCdYWPa>#6`^g-Ct zEA6by((SxXjebO>sq!y&4YKVK|8TR=kSk1?K)iZ-sxlS_L5@AmT@#)kVLB?tY(l%6 z)^t((@il)V@Wpja{sCocIw3JYJFvmjFqZVMhL`}W)Ewlc5I1p1z$+`#x#vxYA9Cz< z2{(~;0}uXQ?@#cQqqj>vIRloy=)e^9Ptti_;T8x9e8sn_Px#N0@&6)uMEf`Kf6??b z9%=@rA?$CM2On1ugdd4HKRpv$u4d?-S-}8a<4oineQlLaSh-d3N{e{?vzBjbDegLd z&*jamH2t+?@E(bNs{3MhE^iZ-bj#8j)y;weEDN-3nm;!?Ha<7)ITk?_y{v7jo-#+JA-o>O*I;#8#59 zjvOt+VIH(RvXH^a3ipzeLpO=IkKl9iNU!-Ka`xS!a;5nO3W85xLhs$r=Mv$qs%Cb9 zE1zA?!D|21T zT;-RkCHaAocH*9Axy3_%M3GZpGs53<|Bi7St0+OO_OJICkO!>g73 zJKAB%gQNc!Ow+Fy-KXp?7R+D6HIIIf8-vKizx`%1`T-K&d+usW#BTuKXnD86*ME6x zmA!)Gk_ab_c{0BL$UyfMKV0$7VfFVNV!Y>COY;GIj8k77U2pL7*88k?(%%E4H&BAur8|U}pKJhwv z8y&bVvIvnSuZ3AdlZPWsoAo_vyAb=jP;c;=^-0PH%gFt(C((EZ%{ z$^EnK^7j?=I2Ienm^K^+$(N^NtZ}m{2nPI!kw>l z+mfF|kvsVpZ!0O|q6n2S!af^AFUjdjfiXP}rDCtC{=)bEZOY!yQ;teCi5bV6Z>~8X zV#X{v8@?ZS^RUPJ2MDj(G5#ow%ZiM+ZE?hQ3K#pKr?e;iq#qPoLZtad;-`nmB1?o% z8}<=oq|t-n2{MU!q^&s2H*z$g3SBT!yQ7}dC!jAD55CjmwgyuW=Y7Dp6uZJI!%bLRI-CnPLH5TCOao8v1b5Ferp%F7@HTU^=WZ9Kv)Z*#vU_ppc06+eI|%!t0%Vkfl3vNeCE&;JA?DvuHm)&JA(RGXqz_ zFUt6@b3#MZCu6+BZNH$``+H~X1jM97NVh=P@tJ#z> z!}8pb>&fRVwTm{d$63lF(=y^93RX1UiPAQRrKYsi8@NO=9-c)zoX19JA9X+ZwH=}p z!Yt%ucnR(Cmxw_U<;U@hwpO*}%lSucZta=MHKr*ipYYwpv=3(1*OTgoCh(>@;`@J% zq@d;JZPS*-@ocY?d6|p8mY;&>404&FvYKQ1^@w^EPgNlP9hADz%gkJKy%Q?&W*BUw zry_E8#`23ZLzeWlO_M{cWn=@Y9#loH8Rmb(;p~d+uF9k`RZ$lv!faW*xxFoz_q^m2 z7$rm(4Z}=dXXJMd+$JLSGbF})t0Po~soCu_zPD5;bHt*Vk-r|4?JM*nVNCp z1&TsFVd>Yh{+S@%vNM;Cn4!X9LqhXaa{T}sUDU%y5p6(i63FMxB6mo}i`AdNo8{SF zOdhe7MAoR;6URZa6Dw^(#|pqxTAOzcs%z?LaA<}5M7(Rxd&Azl_!>-Z1+EI~O^vdzgsmslAQz9*oO_3;h>j9? z-;|^iDd8>J44E9y)J6QHjThNx8|@fp%BHo{(&Tawr7LU95PoyluP(ds4PiK;`US*e ztvU!$^Q48ug4j)>@4rimozx7k(Cuo(p!nlexC^4ZQPS@AK*mCc*Y*y(vX6f z6Xm!@==h!4?DFzf?`$9?d3*MZ`T^)HNodom2>mtp_!7MNTC=7#Qs90;PC#H$0Ua3{ z*OGtqH4p1VNM}ZjK6D^L=m@5N$)~}i<{KO>JS8t^>DgP$PgT#rn|={@dANx5r6Hpr zWNqr|!nu#f(|BL^2)jDwg|=A~tI#K%#?{0Zm^Ck-aqJ$XtX~gPWPL&xDtiMfkuTZj zdYypn2v~cZEww}6MSkMQb0<#A5F0|4T_(1~>_V1GjI%FAa>W&k`#1+|*!-E#@9p1g zz+S#@+|2%-)6b*aZ*r}!4U@zfyX{6!rG)?52gTu`sA3Ky*qJh-Sg12QD8W*C>g@s@ zz=ecbLYzPHXoYyp{SO~2A!k*L@U(8|u6%o9qLJo>QAgW)N$#o715*Q|XkN$Z$moNd z^axje8?W2BOol?%cwOaG)HQ&^xhbQKE%9cdM;&Kq;em9t=_E?@r3>4&Li=o@vd^F} z@!p94sZz9Q(#KbSK*ymgwy#N4S(Otu=$588Mbf;63q3-o3+Hz<;j71C+bT{{#L`3| zXBvJt$vqvY&#`#m96l~C6l(!_;Mw|gTG(EP4JcsBTP~M z#8r%jo5a7gig)a@%U^mF4pj-oGIdwC?H0ZROi(q{qL4pN6NcEOi|Cf4#fy z=y1vA=oJuM(>J&7rdKmgY~C&nS#lTl8mfBxX=~(awc#i=h4fV6o{P@rDYYi7HsqNb zY0K!PQPz4|1|eQFJwe|+x*;}cpW6-L(+(W4Qs8 zNnNUQLMwb>fraGaw$J9qazph7IGdkoldoq25&fu@7!q8Wsx)4gf@BJgN;XSsLKb}R zT3Y7V=H`*80q%2qx^j})43E7+_xsKj*qnUl*Nq5T2kMsdXLfp@PtG0$qh|#LoisGb zForBe)LVWX%T*@3J1A<)^MR#n=*%-{v+Eq~Mcm#*`#r4Nn%yFFkYCW>YF#peW>1=T z5^DRNo)<68)uz(hvUze6lNzf;A;0x6Z&d{D;d`gt2(55nJ9V(0{^ZhMYvZFqSMOSF zK2uxsfIOSYL<2)Rs=XDPvxipj6gRYkNnPx$8&ySx2J%P`=d=!6gzeh>@EliKJC*7m z?&>7hKPNjSD$PTT@EzcVEzsUF-YOm~nu9r_ABr7gU!Dpr-?A2~)pEoby2gOSd-r=j ze$ZzbfNu(Ue#R_Bb1zTAzry0jo*b9Aih^T-TEyJOn+ep1!`)6uK4pRJknZdvbqw-^ zc8Qb={Q&ky!hcHW1vVZ}NEF#yUq83RWnnEKwu^|tO!70bv1AgYfn}w728RqaV=mZW zcl0KyckL@%_BBP92T1WQRFnw>yt8kI(fw#lJzqV(H~yY>cLVo$@DckS7LokDsqBr1 z=5dJu_xx@$b6{Zk+3c*FfeGH9X4v4(w2%Ed(>}kWsQ!p<{~Y2OUSD*qOPxwHhMz`V zw?$*!+spExBo1(tR05OXw2{e<&FqW!Fq?Z_aH?3YU7?--e8WsbV2zA38ckQFj+>yJ zvN(%eGfN_sKFdMFvUa-4hTb;K+&zyula)nh9_i-X>|skT4;+H$D8cuFb`1r=Q=x6E zeD?w@1@+zQU+S?ZA5!`1V_V`o?5PK}wF)j@ju3n+c(@s_=rYdv!NAZwzkGS))!dbB z{WCpiny9uOohpM%I3bzh@(ae-`nz*S`OM}zFl{~P)NGA+NBJ@Ybo7#h^5JtFB*I$i zUkp(4zynZ;5YkT*7Gi3wpW?&XH|HyO)TWQbmM9L~DdhT_orkpD$`(!G`dx1??Vyn4 z0k}<>E!JekcFg&K_Oi&btEPHc11&r>60FRY18lPu*UhJqMZAT5;2ml{YM3RM*6#}u zS(yC3V3kH%y^GZ3#(fJZ8HFxC!w4ANQ`sqEDL0?qqny(gWPs*N3!wWNq+g}U7;gKd zmvuCbFD@ieW(T%2pyq2#99ADqomcH=jTUSv>g9K+2r#*c_K&^Rp7)a#2>z!VsM9&& zxai+l+GK2t{n$kmeBKGXv4G1A_r0yZgvnfgdiy0Y{JOj*+HKh;69SJR%-X-@#55aj zT@)vPb>>*WG3&2^YqAil1@R^6WnN|$ZFrIHxY!)zDjKKSB$HB5YJ$2>arP)w{e_XQ(@jwG?Y=xfR^*&xRm zapcvztXR$mZ!Tb^s_70;pvRUshAx3%r0EPB3?@VA;oTpN-4l+gz48;fzE8wt2}B&X z8J1p~cSDT1=C%cADQfF=3#hirCd)hX5J43;6CD1E6Ub@aI5u!CyLNFp6NG>NOpvKa zrKS1w`ht|BAo!hq(k8NgdNU{$OL4SC6|VDg`b^{EwixhqGx6-Cs6ov;IE*7P?e zSz4nw2gmB}=_^dq)m_?K=UR3b=)BaCR1WIhF4eOGZ#I?c@XW(|k2pWLse_7nX20J3 z3vbg=dl-|P7l#`6j`Z7=(ydb-puL2e8%+7!b{zcmtY9z=>3pPCWN;YEew^ZN|3IsDBMKBU$Ib^m4?BCmJR0HCIJB!Ka_G=7TA@}3e}d9 zyVo2>i+ZhgBpt?U*;S_XCRw|1SCw}}EQ8IEyxRvwwQdIsoJWVt&a0PyQIbjy?Hp;k zKxn!vv_&wW6ozV1RiKi>vnTfRk9fy%1dUV`>q}*{>$_ihDK>Av+@rOxNZe%7u&)g7 z)izQ}E&Bj&X=x;1_N#1nD&YZem**zE=p7!OdF7rQ)=qr;)yN&E^e=H1{(*NoqE8Ww zUX|to&7T}Ub_e*aCBLBD5w^AFnO8-3P^q%3Wb(9`Q0#qVkF>R!Yebq#60@7--y`S0 zJje<>ynZj7H1*$|L%IHobLju*@cA#-O%9I#m*=Lwf-Ma%PWyP(1iy)%miitdW%a>4 z|FD?zxCP;p^w`mdP+8cKHRDwst#^4-k9nW;eq~ju<}pN_&cBku|5MLC)FHG38{CV> zh7R`^{c{p@!&S9UFdi-j z(c1MCSo4X?{ftkn1wSd%2|;AkdjH1SO;#jTaa+M3yxHHwf$m@uw?FPEdRa_N>odRW zZYGJ${nh(Lu{`=Qg>+eLuANhb>5SKs6F=$e8C9;OoLr7|pI6z<9vO2zlWQNqjU_t{ zSF%I%{P2ukuQQ*Hk~7!JdSl;$PG=1G;@!A&&?+oCsE)p2MW(aL#pE54eL00VD7vRs zp*<^fnobq%g)(WY1P%E|&kfK1`+$^il=KN{gdg22L!nOOm6pI4Al@fq$D|0YMekp% zzri(;pPHVSJ36@hO#5w8Y{z`*xI`)QC_Z$CKG^PngG8lvIk2Y~H*Ff}0F!eHiE4(( zq#ny{l1~a-MwU?3D=uKLlhmd{;H$xUQTNHghWHW^XUnfzLs!OnRB>*{n1c(P@BP+@ zoZmDH&^LDnYP-(6si1+BbPsu)Ce=0)R{7eV{Y@)q`Up=DiaeybP3>KK&P04gtJ?VS zgH_~i73(Xhwh6QqP&f8)S7;NGO@%)`p-}wYxE;79rNV6AD0Pm~C++SA+*(&6Ue2Jo z_5BIvc8mNWk1wXQgzm3^l&5VNThjs!Z%HNbVBejSCIm=HcT)CK6Y~qeOBZbLxNKrjbrO!^b4mo&u^{az19kP?k zik#2FLOk33W%yY~+epg$-gZUWnz5!hB?KRGC*nHHW`^L8+T+b1S$n#rJz&N^<8aVB zMrRZ(l-uGczS5b5x|rG)&%sr&_s{hB)3b^-`z7qMAxDK0^w% zxrE`J;r#T}GoByIsD>&7r6FIVFwYO@c6;%4Q>c=pU%Pr>#)6Wh;crrj;eJT>R|(U_SG=oIGG|k^gT9!~bx6H#b8-q?7KukHS#Iz=-4&i*wrY z!b$H%mE`;yG*B&q!$^lp<&_+yHS|$)n<;RCgh_1k`#1$Ii(d6AJ_bSnIV$&~fB%1a f`cwSh{{RnsQ)gFCXRrktCkGE2Ev=M_G}`|G;RX>D literal 0 HcmV?d00001 diff --git a/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosAgent.java b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosAgent.java new file mode 100644 index 00000000000..430b46efaad --- /dev/null +++ b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosAgent.java @@ -0,0 +1,591 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.chaos; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.RetryCounter; +import org.apache.hadoop.hbase.util.RetryCounterFactory; +import org.apache.hadoop.util.Shell; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/*** + * An agent for executing destructive actions for ChaosMonkey. + * Uses ZooKeeper Watchers and LocalShell, to do the killing + * and getting status of service on targeted host without SSH. + * uses given ZNode Structure: + * /perfChaosTest (root) + * | + * | + * /chaosAgents (Used for registration has + * hostname ephemeral nodes as children) + * | + * | + * /chaosAgentTaskStatus (Used for task + * Execution, has hostname persistent + * nodes as child with tasks as their children) + * | + * | + * /hostname + * | + * | + * /task0000001 (command as data) + * (has two types of command : + * 1: starts with "exec" + * for executing a destructive action. + * 2: starts with "bool" for getting + * only status of service. + * + */ +@InterfaceAudience.Private +public class ChaosAgent implements Watcher, Closeable, Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(ChaosAgent.class); + static AtomicBoolean stopChaosAgent = new AtomicBoolean(); + private ZooKeeper zk; + private String quorum; + private String agentName; + private Configuration conf; + private RetryCounterFactory retryCounterFactory; + private volatile boolean connected = false; + + public ChaosAgent(Configuration conf, String quorum, String agentName) { + initChaosAgent(conf, quorum, agentName); + } + + /*** + * sets global params and initiates connection with ZooKeeper then does registration. + * @param conf initial configuration to use + * @param quorum ZK Quorum + * @param agentName AgentName to use + */ + private void initChaosAgent(Configuration conf, String quorum, String agentName) { + this.conf = conf; + this.quorum = quorum; + this.agentName = agentName; + this.retryCounterFactory = new RetryCounterFactory(new RetryCounter.RetryConfig() + .setMaxAttempts(conf.getInt(ChaosConstants.RETRY_ATTEMPTS_KEY, + ChaosConstants.DEFAULT_RETRY_ATTEMPTS)).setSleepInterval( + conf.getLong(ChaosConstants.RETRY_SLEEP_INTERVAL_KEY, + ChaosConstants.DEFAULT_RETRY_SLEEP_INTERVAL))); + try { + this.createZKConnection(null); + this.register(); + } catch (IOException e) { + LOG.error("Error Creating Connection: " + e); + } + } + + /*** + * Creates Connection with ZooKeeper. + * @throws IOException if something goes wrong + */ + private void createZKConnection(Watcher watcher) throws IOException { + if(watcher == null) { + zk = new ZooKeeper(quorum, ChaosConstants.SESSION_TIMEOUT_ZK, this); + } else { + zk = new ZooKeeper(quorum, ChaosConstants.SESSION_TIMEOUT_ZK, watcher); + } + LOG.info("ZooKeeper Connection created for ChaosAgent: " + agentName); + } + + //WATCHERS: Below are the Watches used by ChaosAgent + + /*** + * Watcher for notifying if any task is assigned to agent or not, + * by seeking if any Node is being added to agent as Child. + */ + Watcher newTaskCreatedWatcher = new Watcher() { + @Override + public void process(WatchedEvent watchedEvent) { + if (watchedEvent.getType() == Event.EventType.NodeChildrenChanged) { + if (!(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE + + ChaosConstants.ZNODE_PATH_SEPARATOR + agentName).equals(watchedEvent.getPath())) { + throw new RuntimeException(KeeperException.create( + KeeperException.Code.DATAINCONSISTENCY)); + } + + LOG.info("Change in Tasks Node, checking for Tasks again."); + getTasks(); + } + + } + }; + + //CALLBACKS: Below are the Callbacks used by Chaos Agent + + /** + * Callback used while setting status of a given task, Logs given status. + */ + AsyncCallback.StatCallback setStatusOfTaskZNodeCallback = (rc, path, ctx, stat) -> { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + // Connection to the server was lost while setting status setting again. + try { + recreateZKConnection(); + } catch (Exception e) { + break; + } + setStatusOfTaskZNode(path, (String) ctx); + break; + + case OK: + LOG.info("Status of Task has been set"); + break; + + case NONODE: + LOG.error("Chaos Agent status node does not exists: " + + "check for ZNode directory structure again."); + break; + + default: + LOG.error("Error while setting status of task ZNode: " + + path, KeeperException.create(KeeperException.Code.get(rc), path)); + } + }; + + /** + * Callback used while creating a Persistent ZNode tries to create + * ZNode again if Connection was lost in previous try. + */ + AsyncCallback.StringCallback createZNodeCallback = (rc, path, ctx, name) -> { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + try { + recreateZKConnection(); + } catch (Exception e) { + break; + } + createZNode(path, (byte[]) ctx); + break; + case OK: + LOG.info("ZNode created : " + path); + break; + case NODEEXISTS: + LOG.warn("ZNode already registered: " + path); + break; + default: + LOG.error("Error occurred while creating Persistent ZNode: " + path, + KeeperException.create(KeeperException.Code.get(rc), path)); + } + }; + + /** + * Callback used while creating a Ephemeral ZNode tries to create ZNode again + * if Connection was lost in previous try. + */ + AsyncCallback.StringCallback createEphemeralZNodeCallback = (rc, path, ctx, name) -> { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + try { + recreateZKConnection(); + } catch (Exception e) { + break; + } + createEphemeralZNode(path, (byte[]) ctx); + break; + case OK: + LOG.info("ZNode created : " + path); + break; + case NODEEXISTS: + LOG.warn("ZNode already registered: " + path); + break; + default: + LOG.error("Error occurred while creating Ephemeral ZNode: ", + KeeperException.create(KeeperException.Code.get(rc), path)); + } + }; + + /** + * Callback used by getTasksForAgentCallback while getting command, + * after getting command successfully, it executes command and + * set its status with respect to the command type. + */ + AsyncCallback.DataCallback getTaskForExecutionCallback = new AsyncCallback.DataCallback() { + @Override + public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + //Connection to the server has been lost while getting task, getting data again. + try { + recreateZKConnection(); + } catch (Exception e) { + break; + } + zk.getData(path, + false, + getTaskForExecutionCallback, + new String(data)); + break; + case OK: + String cmd = new String(data); + LOG.info("Executing command : " + cmd); + String status = ChaosConstants.TASK_COMPLETION_STRING; + try { + String user = conf.get(ChaosConstants.CHAOSAGENT_SHELL_USER, + ChaosConstants.DEFAULT_SHELL_USER); + switch (cmd.substring(0, 4)) { + case "bool": + String ret = execWithRetries(user, cmd.substring(4)).getSecond(); + status = Boolean.toString(ret.length() > 0); + break; + + case "exec": + execWithRetries(user, cmd.substring(4)); + break; + + default: + LOG.error("Unknown Command Type"); + status = ChaosConstants.TASK_ERROR_STRING; + } + } catch (IOException e) { + LOG.error("Got error while executing command : " + cmd + + " On agent : " + agentName + " Error : " + e); + status = ChaosConstants.TASK_ERROR_STRING; + } + + try { + setStatusOfTaskZNode(path, status); + Thread.sleep(ChaosConstants.SET_STATUS_SLEEP_TIME); + } catch (InterruptedException e) { + LOG.error("Error occured after setting status: " + e); + } + + default: + LOG.error("Error occurred while getting data", + KeeperException.create(KeeperException.Code.get(rc), path)); + } + } + }; + + /*** + * Callback used while getting Tasks for agent if call executed without Exception, + * It creates a separate thread for each children to execute given Tasks parallely. + */ + AsyncCallback.ChildrenCallback getTasksForAgentCallback = new AsyncCallback.ChildrenCallback() { + @Override + public void processResult(int rc, String path, Object ctx, List children) { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: { + // Connection to the server has been lost, getting tasks again. + try { + recreateZKConnection(); + } catch (Exception e) { + break; + } + getTasks(); + break; + } + + case OK: { + if (children != null) { + try { + + LOG.info("Executing each task as a separate thread"); + List tasksList = new ArrayList<>(); + for (String task : children) { + String threadName = agentName + "_" + task; + Thread t = new Thread(() -> { + + LOG.info("Executing task : " + task + " of agent : " + agentName); + zk.getData(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE + + ChaosConstants.ZNODE_PATH_SEPARATOR + agentName + + ChaosConstants.ZNODE_PATH_SEPARATOR + task, + false, + getTaskForExecutionCallback, + task); + + }); + t.setName(threadName); + t.start(); + tasksList.add(t); + + for (Thread thread : tasksList) { + thread.join(); + } + } + } catch (InterruptedException e) { + LOG.error("Error scheduling next task : " + + " for agent : " + agentName + " Error : " + e); + } + } + break; + } + + default: + LOG.error("Error occurred while getting task", + KeeperException.create(KeeperException.Code.get(rc), path)); + } + } + }; + + /*** + * Function to create PERSISTENT ZNODE with given path and data given as params + * @param path Path at which ZNode to create + * @param data Data to put under ZNode + */ + public void createZNode(String path, byte[] data) { + zk.create(path, + data, + ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, + createZNodeCallback, + data); + } + + /*** + * Function to create EPHEMERAL ZNODE with given path and data as params. + * @param path Path at which Ephemeral ZNode to create + * @param data Data to put under ZNode + */ + public void createEphemeralZNode(String path, byte[] data) { + zk.create(path, + data, + ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.EPHEMERAL, + createEphemeralZNodeCallback, + data); + } + + /** + * Checks if given ZNode exists, if not creates a PERSISTENT ZNODE for same. + * + * @param path Path to check for ZNode + */ + private void createIfZNodeNotExists(String path) { + try { + if (zk.exists(path, + false) == null) { + createZNode(path, new byte[0]); + } + } catch (KeeperException | InterruptedException e) { + LOG.error("Error checking given node : " + path + " " + e); + } + } + + /** + * sets given Status for Task Znode + * + * @param taskZNode ZNode to set status + * @param status Status value + */ + public void setStatusOfTaskZNode(String taskZNode, String status) { + LOG.info("Setting status of Task ZNode: " + taskZNode + " status : " + status); + zk.setData(taskZNode, + status.getBytes(), + -1, + setStatusOfTaskZNodeCallback, + null); + } + + /** + * registration of ChaosAgent by checking and creating necessary ZNodes. + */ + private void register() { + createIfZNodeNotExists(ChaosConstants.CHAOS_TEST_ROOT_ZNODE); + createIfZNodeNotExists(ChaosConstants.CHAOS_AGENT_REGISTRATION_EPIMERAL_ZNODE); + createIfZNodeNotExists(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE); + createIfZNodeNotExists(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE + + ChaosConstants.ZNODE_PATH_SEPARATOR + agentName); + + createEphemeralZNode(ChaosConstants.CHAOS_AGENT_REGISTRATION_EPIMERAL_ZNODE + + ChaosConstants.ZNODE_PATH_SEPARATOR + agentName, new byte[0]); + } + + /*** + * Gets tasks for execution, basically sets Watch on it's respective host's Znode and + * waits for tasks to be assigned, also has a getTasksForAgentCallback + * which handles execution of task. + */ + private void getTasks() { + LOG.info("Getting Tasks for Agent: " + agentName + "and setting watch for new Tasks"); + zk.getChildren(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE + + ChaosConstants.ZNODE_PATH_SEPARATOR + agentName, + newTaskCreatedWatcher, + getTasksForAgentCallback, + null); + } + + /** + * Below function executes command with retries with given user. + * Uses LocalShell to execute a command. + * + * @param user user name, default none + * @param cmd Command to execute + * @return A pair of Exit Code and Shell output + * @throws IOException Exception while executing shell command + */ + private Pair execWithRetries(String user, String cmd) throws IOException { + RetryCounter retryCounter = retryCounterFactory.create(); + while (true) { + try { + return exec(user, cmd); + } catch (IOException e) { + retryOrThrow(retryCounter, e, user, cmd); + } + try { + retryCounter.sleepUntilNextRetry(); + } catch (InterruptedException e) { + LOG.warn("Sleep Interrupted: " + e); + } + } + } + + private Pair exec(String user, String cmd) throws IOException { + LOG.info("Executing Shell command: " + cmd + " , user: " + user); + + LocalShell shell = new LocalShell(user, cmd); + try { + shell.execute(); + } catch (Shell.ExitCodeException e) { + String output = shell.getOutput(); + throw new Shell.ExitCodeException(e.getExitCode(), "stderr: " + e.getMessage() + + ", stdout: " + output); + } + LOG.info("Executed Shell command, exit code: {}, output n{}", shell.getExitCode(), shell.getOutput()); + + return new Pair<>(shell.getExitCode(), shell.getOutput()); + } + + private void retryOrThrow(RetryCounter retryCounter, E ex, + String user, String cmd) throws E { + if (retryCounter.shouldRetry()) { + LOG.warn("Local command: {}, user: {}, failed at attempt {}. Retrying until maxAttempts: {}." + + "Exception {}", cmd, user,retryCounter.getAttemptTimes(), retryCounter.getMaxAttempts(), + ex.getMessage()); + return; + } + throw ex; + } + + private boolean isConnected() { + return connected; + } + + @Override + public void close() throws IOException { + LOG.info("Closing ZooKeeper Connection for Chaos Agent : " + agentName); + try { + zk.close(); + } catch (InterruptedException e) { + LOG.error("Error while closing ZooKeeper Connection."); + } + } + + @Override + public void run() { + try { + LOG.info("Running Chaos Agent on : " + agentName); + while (!this.isConnected()) { + Thread.sleep(100); + } + this.getTasks(); + while (!stopChaosAgent.get()) { + Thread.sleep(500); + } + } catch (InterruptedException e) { + LOG.error("Error while running Chaos Agent", e); + } + + } + + @Override + public void process(WatchedEvent watchedEvent) { + LOG.info("Processing event: " + watchedEvent.toString()); + if (watchedEvent.getType() == Event.EventType.None) { + switch (watchedEvent.getState()) { + case SyncConnected: + connected = true; + break; + case Disconnected: + connected = false; + break; + case Expired: + connected = false; + LOG.error("Session expired creating again"); + try { + createZKConnection(null); + } catch (IOException e) { + LOG.error("Error creating Zookeeper connection", e); + } + default: + LOG.error("Unknown State"); + break; + } + } + } + + private void recreateZKConnection() throws Exception{ + try { + zk.close(); + createZKConnection(newTaskCreatedWatcher); + createEphemeralZNode(ChaosConstants.CHAOS_AGENT_REGISTRATION_EPIMERAL_ZNODE + + ChaosConstants.ZNODE_PATH_SEPARATOR + agentName, new byte[0]); + } catch (IOException e) { + LOG.error("Error creating new ZK COnnection for agent: {}", agentName + e); + throw e; + } + } + + /** + * Executes Command locally. + */ + protected static class LocalShell extends Shell.ShellCommandExecutor { + + private String user; + private String execCommand; + + public LocalShell(String user, String execCommand) { + super(new String[]{execCommand}); + this.user = user; + this.execCommand = execCommand; + } + + @Override + public String[] getExecString() { + // TODO: Considering Agent is running with same user. + if(!user.equals(ChaosConstants.DEFAULT_SHELL_USER)){ + execCommand = String.format("su -u %1$s %2$s", user, execCommand); + } + return new String[]{"/usr/bin/env", "bash", "-c", execCommand}; + } + + @Override + public void execute() throws IOException { + super.execute(); + } + } +} diff --git a/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosConstants.java b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosConstants.java new file mode 100644 index 00000000000..54fbe9b10cd --- /dev/null +++ b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosConstants.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.chaos; + +import org.apache.yetus.audience.InterfaceAudience; + +/*** + * ChaosConstant holds a bunch of Choas-related Constants + */ +@InterfaceAudience.Public +public final class ChaosConstants { + + /*Base ZNode for whole Chaos Testing*/ + public static final String CHAOS_TEST_ROOT_ZNODE = "/hbase"; + + /*Just a / used for path separator*/ + public static final String ZNODE_PATH_SEPARATOR = "/"; + + /*ZNode used for ChaosAgents registration.*/ + public static final String CHAOS_AGENT_REGISTRATION_EPIMERAL_ZNODE = + CHAOS_TEST_ROOT_ZNODE + ZNODE_PATH_SEPARATOR + "chaosAgents"; + + /*ZNode used for getting status of tasks assigned*/ + public static final String CHAOS_AGENT_STATUS_PERSISTENT_ZNODE = + CHAOS_TEST_ROOT_ZNODE + ZNODE_PATH_SEPARATOR + "chaosAgentTaskStatus"; + + /*Config property for getting number of retries to execute a command*/ + public static final String RETRY_ATTEMPTS_KEY = "hbase.it.clustermanager.retry.attempts"; + + /*Default value for number of retries*/ + public static final int DEFAULT_RETRY_ATTEMPTS = 5; + + /*Config property to sleep in between retries*/ + public static final String RETRY_SLEEP_INTERVAL_KEY = + "hbase.it.clustermanager.retry.sleep.interval"; + + /*Default Sleep time between each retry*/ + public static final int DEFAULT_RETRY_SLEEP_INTERVAL = 5000; + + /*Config property for executing command as specific user*/ + public static final String CHAOSAGENT_SHELL_USER = "hbase.it.clustermanager.ssh.user"; + + /*default user for executing local commands*/ + public static final String DEFAULT_SHELL_USER = ""; + + /*timeout used while creating ZooKeeper connection*/ + public static final int SESSION_TIMEOUT_ZK = 60000 * 10; + + /*Time given to ChaosAgent to set status*/ + public static final int SET_STATUS_SLEEP_TIME = 30 * 1000; + + /*Status String when you get an ERROR while executing task*/ + public static final String TASK_ERROR_STRING = "error"; + + /*Status String when your command gets executed correctly*/ + public static final String TASK_COMPLETION_STRING = "done"; + + /*Name of ChoreService to use*/ + public static final String CHORE_SERVICE_PREFIX = "ChaosService"; + +} diff --git a/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosService.java b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosService.java new file mode 100644 index 00000000000..e2abe3d4265 --- /dev/null +++ b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosService.java @@ -0,0 +1,138 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.chaos; + +import java.net.UnknownHostException; +import java.util.Arrays; +import java.util.Collection; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.AuthUtil; +import org.apache.hadoop.hbase.ChoreService; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.util.GenericOptionsParser; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.hbase.thirdparty.org.apache.commons.cli.GnuParser; +import org.apache.hbase.thirdparty.org.apache.commons.cli.Option; +import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; + +/** + * Class used to start/stop Chaos related services (currently chaosagent) + */ +@InterfaceAudience.Private +public class ChaosService { + + private static final Logger LOG = LoggerFactory.getLogger(ChaosService.class.getName()); + + public static void execute(String[] args, Configuration conf) { + LOG.info("arguments : " + Arrays.toString(args)); + + try { + CommandLine cmdline = new GnuParser().parse(getOptions(), args); + if (cmdline.hasOption(ChaosServiceName.CHAOSAGENT.toString().toLowerCase())) { + String actionStr = cmdline.getOptionValue(ChaosServiceName.CHAOSAGENT.toString().toLowerCase()); + try { + ExecutorAction action = ExecutorAction.valueOf(actionStr.toUpperCase()); + if (action == ExecutorAction.START) { + ChaosServiceStart(conf, ChaosServiceName.CHAOSAGENT); + } else if (action == ExecutorAction.STOP) { + ChaosServiceStop(); + } + } catch (IllegalArgumentException e) { + LOG.error("action passed: {} Unexpected action. Please provide only start/stop.", + actionStr, e); + throw new RuntimeException(e); + } + } else { + LOG.error("Invalid Options"); + } + } catch (Exception e) { + LOG.error("Error while starting ChaosService : ", e); + } + } + + private static void ChaosServiceStart(Configuration conf, ChaosServiceName serviceName) { + switch (serviceName) { + case CHAOSAGENT: + ChaosAgent.stopChaosAgent.set(false); + try { + Thread t = new Thread(new ChaosAgent(conf, + ChaosUtils.getZKQuorum(conf), ChaosUtils.getHostName())); + t.start(); + t.join(); + } catch (InterruptedException | UnknownHostException e) { + LOG.error("Failed while executing next task execution of ChaosAgent on : {}", + serviceName, e); + } + break; + default: + LOG.error("Service Name not known : " + serviceName.toString()); + } + } + + private static void ChaosServiceStop() { + ChaosAgent.stopChaosAgent.set(true); + } + + private static Options getOptions() { + Options options = new Options(); + options.addOption(new Option("c", ChaosServiceName.CHAOSAGENT.toString().toLowerCase(), + true, "expecting a start/stop argument")); + options.addOption(new Option("D", ChaosServiceName.GENERIC.toString(), + true, "generic D param")); + LOG.info(Arrays.toString(new Collection[] { options.getOptions() })); + return options; + } + + public static void main(String[] args) throws Exception { + Configuration conf = HBaseConfiguration.create(); + new GenericOptionsParser(conf, args); + + ChoreService choreChaosService = null; + ScheduledChore authChore = AuthUtil.getAuthChore(conf); + + try { + if (authChore != null) { + choreChaosService = new ChoreService(ChaosConstants.CHORE_SERVICE_PREFIX); + choreChaosService.scheduleChore(authChore); + } + + execute(args, conf); + } finally { + if (authChore != null) + choreChaosService.shutdown(); + } + } + + enum ChaosServiceName { + CHAOSAGENT, + GENERIC + } + + + enum ExecutorAction { + START, + STOP + } +} diff --git a/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosUtils.java b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosUtils.java new file mode 100644 index 00000000000..da42021bcaf --- /dev/null +++ b/hbase-it/src/main/java/org/apache/hadoop/hbase/chaos/ChaosUtils.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.chaos; + +import java.net.InetAddress; +import java.net.UnknownHostException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * ChaosUtils holds a bunch of useful functions like getting hostname and getting ZooKeeper quorum. + */ +@InterfaceAudience.Private +public class ChaosUtils { + + public static String getHostName() throws UnknownHostException { + return InetAddress.getLocalHost().getHostName(); + } + + + public static String getZKQuorum(Configuration conf) { + String port = + Integer.toString(conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181)); + String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM, "localhost"); + for (int i = 0; i < serverHosts.length; i++) { + serverHosts[i] = serverHosts[i] + ":" + port; + } + return String.join(",", serverHosts); + } + +} diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/ChaosZKClient.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/ChaosZKClient.java new file mode 100644 index 00000000000..31fb9e3ca60 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/ChaosZKClient.java @@ -0,0 +1,332 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import java.io.IOException; + +import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class ChaosZKClient { + + private static final Logger LOG = LoggerFactory.getLogger(ChaosZKClient.class.getName()); + private static final String CHAOS_AGENT_PARENT_ZNODE = "/hbase/chaosAgents"; + private static final String CHAOS_AGENT_STATUS_ZNODE = "/hbase/chaosAgentTaskStatus"; + private static final String ZNODE_PATH_SEPARATOR = "/"; + private static final String TASK_PREFIX = "task_"; + private static final String TASK_ERROR_STRING = "error"; + private static final String TASK_COMPLETION_STRING = "done"; + private static final String TASK_BOOLEAN_TRUE = "true"; + private static final String TASK_BOOLEAN_FALSE = "false"; + private static final String CONNECTION_LOSS = "ConnectionLoss"; + private static final int SESSION_TIMEOUT_ZK = 10 * 60 * 1000; + private static final int TASK_EXECUTION_TIMEOUT = 5 * 60 * 1000; + private volatile String taskStatus = null; + + private final String quorum; + private ZooKeeper zk; + + public ChaosZKClient(String quorum) { + this.quorum = quorum; + try { + this.createNewZKConnection(); + } catch (IOException e) { + LOG.error("Error creating ZooKeeper Connection: ", e); + } + } + + /** + * Creates connection with ZooKeeper + * @throws IOException when not able to create connection properly + */ + public void createNewZKConnection() throws IOException { + Watcher watcher = new Watcher() { + @Override + public void process(WatchedEvent watchedEvent) { + LOG.info("Created ZooKeeper Connection For executing task"); + } + }; + + this.zk = new ZooKeeper(quorum, SESSION_TIMEOUT_ZK, watcher); + } + + /** + * Checks if ChaosAgent is running or not on target host by checking its ZNode. + * @param hostname hostname to check for chaosagent + * @return true/false whether agent is running or not + */ + private boolean isChaosAgentRunning(String hostname) { + try { + return zk.exists(CHAOS_AGENT_PARENT_ZNODE + ZNODE_PATH_SEPARATOR + hostname, + false) != null; + } catch (KeeperException e) { + if (e.toString().contains(CONNECTION_LOSS)) { + recreateZKConnection(); + try { + return zk.exists(CHAOS_AGENT_PARENT_ZNODE + ZNODE_PATH_SEPARATOR + hostname, + false) != null; + } catch (KeeperException | InterruptedException ie) { + LOG.error("ERROR ", ie); + } + } + } catch (InterruptedException e) { + LOG.error("Error checking for given hostname: {} ERROR: ", hostname, e); + } + return false; + } + + /** + * Creates tasks for target hosts by creating ZNodes. + * Waits for a limited amount of time to complete task to execute. + * @param taskObject Object data represents command + * @return returns status + */ + public String submitTask(final TaskObject taskObject) { + if (isChaosAgentRunning(taskObject.getTaskHostname())) { + LOG.info("Creating task node"); + zk.create(CHAOS_AGENT_STATUS_ZNODE + ZNODE_PATH_SEPARATOR + + taskObject.getTaskHostname() + ZNODE_PATH_SEPARATOR + TASK_PREFIX, + taskObject.getCommand().getBytes(), + ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.EPHEMERAL_SEQUENTIAL, + submitTaskCallback, + taskObject); + long start = System.currentTimeMillis(); + + while ((System.currentTimeMillis() - start) < TASK_EXECUTION_TIMEOUT) { + if(taskStatus != null) { + return taskStatus; + } + Threads.sleep(500); + } + } else { + LOG.info("EHHHHH! ChaosAgent Not running"); + } + return TASK_ERROR_STRING; + } + + /** + * To get status of task submitted + * @param path path at which to get status + * @param ctx path context + */ + private void getStatus(String path , Object ctx) { + LOG.info("Getting Status of task: " + path); + zk.getData(path, + false, + getStatusCallback, + ctx); + } + + /** + * Set a watch on task submitted + * @param name ZNode name to set a watch + * @param taskObject context for ZNode name + */ + private void setStatusWatch(String name, TaskObject taskObject) { + LOG.info("Checking for ZNode and Setting watch for task : " + name); + zk.exists(name, + setStatusWatcher, + setStatusWatchCallback, + taskObject); + } + + /** + * Delete task after getting its status + * @param path path to delete ZNode + */ + private void deleteTask(String path) { + LOG.info("Deleting task: " + path); + zk.delete(path, + -1, + taskDeleteCallback, + null); + } + + //WATCHERS: + + /** + * Watcher to get notification whenever status of task changes. + */ + Watcher setStatusWatcher = new Watcher() { + @Override + public void process(WatchedEvent watchedEvent) { + LOG.info("Setting status watch for task: " + watchedEvent.getPath()); + if(watchedEvent.getType() == Event.EventType.NodeDataChanged) { + if(!watchedEvent.getPath().contains(TASK_PREFIX)) { + throw new RuntimeException(KeeperException.create( + KeeperException.Code.DATAINCONSISTENCY)); + } + getStatus(watchedEvent.getPath(), (Object) watchedEvent.getPath()); + + } + } + }; + + //CALLBACKS + + AsyncCallback.DataCallback getStatusCallback = (rc, path, ctx, data, stat) -> { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + //Connectionloss while getting status of task, getting again + recreateZKConnection(); + getStatus(path, ctx); + break; + + case OK: + if (ctx!=null) { + + String status = new String(data); + taskStatus = status; + switch (status) { + case TASK_COMPLETION_STRING: + case TASK_BOOLEAN_TRUE: + case TASK_BOOLEAN_FALSE: + LOG.info("Task executed completely : Status --> " + status); + break; + + case TASK_ERROR_STRING: + LOG.info("There was error while executing task : Status --> " + status); + break; + + default: + LOG.warn("Status of task is undefined!! : Status --> " + status); + } + + deleteTask(path); + } + break; + + default: + LOG.error("ERROR while getting status of task: " + path + " ERROR: " + + KeeperException.create(KeeperException.Code.get(rc))); + } + }; + + AsyncCallback.StatCallback setStatusWatchCallback = (rc, path, ctx, stat) -> { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + //ConnectionLoss while setting watch on status ZNode, setting again. + recreateZKConnection(); + setStatusWatch(path, (TaskObject) ctx); + break; + + case OK: + if(stat != null) { + getStatus(path, null); + } + break; + + default: + LOG.error("ERROR while setting watch on task ZNode: " + path + " ERROR: " + + KeeperException.create(KeeperException.Code.get(rc))); + } + }; + + AsyncCallback.StringCallback submitTaskCallback = (rc, path, ctx, name) -> { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + // Connection to server was lost while submitting task, submitting again. + recreateZKConnection(); + submitTask((TaskObject) ctx); + break; + + case OK: + LOG.info("Task created : " + name); + setStatusWatch(name, (TaskObject) ctx); + break; + + default: + LOG.error("Error submitting task: " + name + " ERROR:" + + KeeperException.create(KeeperException.Code.get(rc))); + } + }; + + AsyncCallback.VoidCallback taskDeleteCallback = new AsyncCallback.VoidCallback() { + @Override + public void processResult(int rc, String path, Object ctx) { + switch (KeeperException.Code.get(rc)) { + case CONNECTIONLOSS: + //Connectionloss while deleting task, deleting again + recreateZKConnection(); + deleteTask(path); + break; + + case OK: + LOG.info("Task Deleted successfully!"); + LOG.info("Closing ZooKeeper Connection"); + try { + zk.close(); + } catch (InterruptedException e) { + LOG.error("Error while closing ZooKeeper Connection."); + } + break; + + default: + LOG.error("ERROR while deleting task: " + path + " ERROR: " + + KeeperException.create(KeeperException.Code.get(rc))); + } + } + }; + + + private void recreateZKConnection() { + try { + zk.close(); + } catch (InterruptedException e) { + LOG.error("Error closing ZK connection : ", e); + } finally { + try { + createNewZKConnection(); + } catch (IOException e) { + LOG.error("Error creating new ZK COnnection for agent: ", e); + } + } + } + + static class TaskObject { + private final String command; + private final String taskHostname; + + public TaskObject(String command, String taskHostname) { + this.command = command; + this.taskHostname = taskHostname; + } + + public String getCommand() { + return this.command; + } + + public String getTaskHostname() { + return taskHostname; + } + } + +} diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/ZNodeClusterManager.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/ZNodeClusterManager.java new file mode 100644 index 00000000000..88f14b0d0d3 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/ZNodeClusterManager.java @@ -0,0 +1,120 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase; + +import java.io.IOException; +import java.util.Arrays; +import java.util.stream.Collectors; + +import org.apache.hadoop.conf.Configured; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class ZNodeClusterManager extends Configured implements ClusterManager { + private static final Logger LOG = LoggerFactory.getLogger(ZNodeClusterManager.class.getName()); + private static final String SIGKILL = "SIGKILL"; + private static final String SIGSTOP = "SIGSTOP"; + private static final String SIGCONT = "SIGCONT"; + public ZNodeClusterManager() { + } + + private String getZKQuorumServersStringFromHbaseConfig() { + String port = + Integer.toString(getConf().getInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181)); + String[] serverHosts = getConf().getStrings(HConstants.ZOOKEEPER_QUORUM, "localhost"); + for (int i = 0; i < serverHosts.length; i++) { + serverHosts[i] = serverHosts[i] + ":" + port; + } + return Arrays.asList(serverHosts).stream().collect(Collectors.joining(",")); + } + + private String createZNode(String hostname, String cmd) throws IOException{ + LOG.info("Zookeeper Mode enabled sending command to zookeeper + " + + cmd + "hostname:" + hostname); + ChaosZKClient chaosZKClient = new ChaosZKClient(getZKQuorumServersStringFromHbaseConfig()); + return chaosZKClient.submitTask(new ChaosZKClient.TaskObject(cmd, hostname)); + } + + protected HBaseClusterManager.CommandProvider getCommandProvider(ServiceType service) + throws IOException { + switch (service) { + case HADOOP_DATANODE: + case HADOOP_NAMENODE: + return new HBaseClusterManager.HadoopShellCommandProvider(getConf()); + case ZOOKEEPER_SERVER: + return new HBaseClusterManager.ZookeeperShellCommandProvider(getConf()); + default: + return new HBaseClusterManager.HBaseShellCommandProvider(getConf()); + } + } + + public void signal(ServiceType service, String signal, String hostname) throws IOException { + createZNode(hostname, CmdType.exec.toString() + + getCommandProvider(service).signalCommand(service, signal)); + } + + private void createOpCommand(String hostname, ServiceType service, + HBaseClusterManager.CommandProvider.Operation op) throws IOException{ + createZNode(hostname, CmdType.exec.toString() + + getCommandProvider(service).getCommand(service, op)); + } + + @Override + public void start(ServiceType service, String hostname, int port) throws IOException { + createOpCommand(hostname, service, HBaseClusterManager.CommandProvider.Operation.START); + } + + @Override + public void stop(ServiceType service, String hostname, int port) throws IOException { + createOpCommand(hostname, service, HBaseClusterManager.CommandProvider.Operation.STOP); + } + + @Override + public void restart(ServiceType service, String hostname, int port) throws IOException { + createOpCommand(hostname, service, HBaseClusterManager.CommandProvider.Operation.RESTART); + } + + @Override + public void kill(ServiceType service, String hostname, int port) throws IOException { + signal(service, SIGKILL, hostname); + } + + @Override + public void suspend(ServiceType service, String hostname, int port) throws IOException { + signal(service, SIGSTOP, hostname); + } + + @Override + public void resume(ServiceType service, String hostname, int port) throws IOException { + signal(service, SIGCONT, hostname); + } + + @Override + public boolean isRunning(ServiceType service, String hostname, int port) throws IOException { + return Boolean.parseBoolean(createZNode(hostname, CmdType.bool.toString() + + getCommandProvider(service).isRunningCommand(service))); + } + + enum CmdType { + exec, + bool + } +}