From ba9b424110d0182a2dd8c0c6cbcfcc23e3acd34a Mon Sep 17 00:00:00 2001 From: Kent Yao <11215016@zju.edu.cn> Date: Tue, 19 Mar 2019 15:09:36 +0800 Subject: [PATCH] [KYUUBI-150][FOLLOWUP]using the classLoader in IsolatedClassLoader (#166) * reset class loader * do not add jar twice in one create function cmd * the classloader should be closeable * the classloader should be closeable revert * Revert "the classloader should be closeable revert" This reverts commit 35602a0b689da0224a4fe3dcd778afabc3f7a803. * add ut * add ut * add ut * fix ut fix #150 --- .gitignore | 3 + .../hive/client/IsolatedClientLoader.scala | 19 ++++- .../kyuubi/operation/KyuubiOperation.scala | 48 +++++++----- .../test/resources/example-1.0.0-SNAPSHOT.jar | Bin 0 -> 38025 bytes .../yaooqinn/kyuubi/SecuredFunSuite.scala | 3 +- .../operation/KyuubiOperationSuite.scala | 41 +++++++++- .../KyuubiOperationWithHDFSSuite.scala | 71 ++++++++++++++++++ .../kyuubi/server/KyuubiServerSuite.scala | 3 +- .../kyuubi/utils/KyuubiHadoopUtilSuite.scala | 15 ++-- 9 files changed, 172 insertions(+), 31 deletions(-) create mode 100644 kyuubi-server/src/test/resources/example-1.0.0-SNAPSHOT.jar create mode 100644 kyuubi-server/src/test/scala/yaooqinn/kyuubi/operation/KyuubiOperationWithHDFSSuite.scala diff --git a/.gitignore b/.gitignore index ecd2d19cd..0e8c42d82 100644 --- a/.gitignore +++ b/.gitignore @@ -22,6 +22,9 @@ .settings build/apache-maven* build/scala* +build/test +kyuubi-server/build +kyuubi-server/*example* target/ dist/ kyuubi-*-bin-* diff --git a/kyuubi-server/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/kyuubi-server/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 616a29797..f37cd6031 100644 --- a/kyuubi-server/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/kyuubi-server/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -63,13 +63,24 @@ private[hive] class IsolatedClientLoader( assert(Try(rootClassLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")).isFailure) /** - * The classloader that is used to load an isolated version of Hive. + * (Kent Yao) Different with Spark internal which use an isolated classloader to support different + * Hive versions, Kyuubi believe that the hive 1.2.1 is capable to support 1.2 or higher version + * Hive metastore servers and the elder hive client versions are not worth to support. + * + * ANOTHER reason here we close the isolation is because Spark don't expose authorization + * functions in [[HiveClient]], which is unable to invoke these methods in different classloaders + * + * Besides, [[HiveClient]] in normal Spark applications is globally one instance, so this + * classloader could/should be non-closeable. But in Kyuubi, this is a session level object + * associated with one KyuubiSession/SparkSession, thus, this classloader should be closeable to + * support class unloading. + * * This classloader is a special URLClassLoader that exposes the addURL method. * So, when we add jar, we can add this new jar directly through the addURL method * instead of stacking a new URLClassLoader on top of it. */ private[hive] val classLoader: MutableURLClassLoader = { - new NonClosableMutableURLClassLoader(baseClassLoader) + new MutableURLClassLoader(Array.empty, baseClassLoader) } private[hive] def addJar(path: URL): Unit = { @@ -88,7 +99,7 @@ private[hive] class IsolatedClientLoader( sparkConf, hadoopConf, config, - baseClassLoader, + classLoader, this).asInstanceOf[HiveClientImpl] } else { ctor.newInstance( @@ -96,7 +107,7 @@ private[hive] class IsolatedClientLoader( sparkConf, hadoopConf, config, - baseClassLoader, + classLoader, this).asInstanceOf[HiveClientImpl] } diff --git a/kyuubi-server/src/main/scala/yaooqinn/kyuubi/operation/KyuubiOperation.scala b/kyuubi-server/src/main/scala/yaooqinn/kyuubi/operation/KyuubiOperation.scala index 2cd46fd7a..98ad355c5 100644 --- a/kyuubi-server/src/main/scala/yaooqinn/kyuubi/operation/KyuubiOperation.scala +++ b/kyuubi-server/src/main/scala/yaooqinn/kyuubi/operation/KyuubiOperation.scala @@ -35,9 +35,10 @@ import org.apache.spark.KyuubiConf._ import org.apache.spark.KyuubiSparkUtil import org.apache.spark.scheduler.cluster.KyuubiSparkExecutorUtils import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSQLUtils} -import org.apache.spark.sql.catalyst.catalog.FunctionResource +import org.apache.spark.sql.catalyst.catalog.{FileResource, FunctionResource, JarResource} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.execution.command.AddJarCommand +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.command.{AddFileCommand, AddJarCommand, CreateFunctionCommand} import org.apache.spark.sql.types._ import yaooqinn.kyuubi.{KyuubiSQLException, Logging} @@ -308,7 +309,7 @@ class KyuubiOperation(session: KyuubiSession, statement: String) extends Logging } } - private def localizeAndAndResource(path: String): Unit = try { + private def localizeAndAndResource(path: String): Option[String] = try { if (isResourceDownloadable(path)) { val src = new Path(path) val destFileName = src.getName @@ -317,12 +318,37 @@ class KyuubiOperation(session: KyuubiSession, statement: String) extends Logging val fs = src.getFileSystem(sparkSession.sparkContext.hadoopConfiguration) fs.copyToLocalFile(src, new Path(destFile)) FileUtil.chmod(destFile, "ugo+rx", true) - AddJarCommand(destFile).run(session.sparkSession) + Some(destFile) + } else { + None } } catch { case e: Exception => throw new KyuubiSQLException(s"Failed to read external resource: $path", e) } + private[operation] def transform(plan: LogicalPlan): LogicalPlan = plan match { + case c: CreateFunctionCommand => + val resources = + ReflectUtils.getFieldValue(c, "resources").asInstanceOf[Seq[FunctionResource]] + resources.foreach { + case FunctionResource(JarResource, uri) => + localizeAndAndResource(uri).map(path => AddJarCommand(path).run(sparkSession)) + case FunctionResource(FileResource, uri) => + localizeAndAndResource(uri).map(path => AddFileCommand(path).run(sparkSession)) + case o => + throw new KyuubiSQLException(s"Resource Type '${o.resourceType}' is not supported.") + } + if (resources.isEmpty) { + c + } else { + ReflectUtils.setFieldValue(c, "resources", Seq.empty[FunctionResource]) + c + } + case a: AddJarCommand => localizeAndAndResource(a.path).map(AddJarCommand).getOrElse(a) + case a: AddFileCommand => localizeAndAndResource(a.path).map(AddFileCommand).getOrElse(a) + case _ => plan + } + private def execute(): Unit = { try { statementId = UUID.randomUUID().toString @@ -344,19 +370,7 @@ class KyuubiOperation(session: KyuubiSession, statement: String) extends Logging KyuubiSparkUtil.setActiveSparkContext(sparkSession.sparkContext) val parsedPlan = SparkSQLUtils.parsePlan(sparkSession, statement) - parsedPlan match { - case c if c.nodeName == "CreateFunctionCommand" => - val resources = - ReflectUtils.getFieldValue(c, "resources").asInstanceOf[Seq[FunctionResource]] - resources.foreach { case FunctionResource(_, uri) => - localizeAndAndResource(uri) - } - case a if a.nodeName == "AddJarCommand" => - val path = ReflectUtils.getFieldValue(a, "path").asInstanceOf[String] - localizeAndAndResource(path) - case _ => - } - result = SparkSQLUtils.toDataFrame(sparkSession, parsedPlan) + result = SparkSQLUtils.toDataFrame(sparkSession, transform(parsedPlan)) KyuubiServerMonitor.getListener(session.getUserName).foreach { _.onStatementParsed(statementId, result.queryExecution.toString()) } diff --git a/kyuubi-server/src/test/resources/example-1.0.0-SNAPSHOT.jar b/kyuubi-server/src/test/resources/example-1.0.0-SNAPSHOT.jar new file mode 100644 index 0000000000000000000000000000000000000000..5abe50994a0e7aef4de1f5aa3211985d190a964b GIT binary patch literal 38025 zcmbTe1yme+wmnR+puwSWcXubayE`=Q?h=A^aCZ&v?(Xgm0fGm2O|UTRTxY0h zi6`8kml+YM(2KjP|MQVMJKFri76$D1<%FLPUJ(KZ1N#602FCGkcK-X~D7d;P zn8}&i+BOg%`oT`V@b3T z*=#Yx)aw$6til)B;1Ns^PDt7wX8aZeZgP;G-#shINyHZ__#TkOJ5ROjXURJ4?^m6F z1T#bI&^U;b@M}aaKcrcVqiuP%?_e{^SmWkt4Ko zL19!-EMDJtN6%j-jvjH3&Bv`{G7Tp`CkrH1-JQ+F2+ZL@b}^SlSVOJyJ|QR$lyWZfXps@ezQ)sM$|IE z3?Q;jbtm^i%N*NuN+S0*!A-3!r^w3oXxe~O0`y{^T+x;Mx6Xe#e;ZKH45sX z^Y!^B?eC(?flfcrb1u5t^SbNJwiVOUdwTeJ+3VTPsS6)Jf6 zqmV7Q=cJO)LvzUBB~(_?NI%i&&R{$BHR4ZQ1sS)&ynHl(f)nh1fB)VzOEv)zZ=-)i zHS>x=vSN??e6^{R|Djfdn1PuJpY^Jg;ept3qY$j?-QWGyAKrAseBWZ_9T*tK8z}ky zJ5Z`xIGF-X{s)+3WI#K6J2K`!pqZ&OW;Z8@&L7|JD7Bs+tV}?^he2fOuZD@CjFMg| zDK1YMOGBqAcVfvhkQF*;Eb9J@yQt@{dGXE%A^UT4mO3OEhvC@$B)@?3WG3&%C!e2i zo74o%CuBTGiPZsEBzR*|qx4AZz`)rLQNKu*e(X9}K`I!h%B-F`6{xwdzBkqs_;K!; z;>ZsBe#baglh743=6vH?f=+1O`(>#5(tUtZte15X`Nf!2026@2og4U zRoli6H{DAjvYyaia^EP9knXIVU(QsRp%r`PC)qzPUE$-xr#dxxKlgFc{p@51E>Ocn zczn3+zO=SEtN~NqF%pb75vs55;Z?kKY}$t*v7QmUJa1hX0++0{NJU5_l0DM0KeGIX zrzR9go-UD#>biyrYj}40YF96Nis5KcF_{8|u^{ifjBE~SqLA@KwFp5dKY_^`n-NYZ z(;c*+CQ3uPB8hLwQzlX!iQ_8rJY!;&kUU-3EVbZY=0BiT?lX8BZPCu{+)eJ4tTbe+ z{eB;Pty^x?{lI#L!_c2g;B0nOg8nL0q5sz48}Xi1-u(lbT_*Q&OWJ+9_8hTX?-_Se z^kV{UKEc=HwUlcJ^^4C&>1yoQE%8iz<)5A*{|T8bCrY{W5nq<*H@I-V2LluN?;!Js zH_F#ib%-d<#*S*j9_s{g6 zZ>O$5zN0ft2aX`}G@?siNG{V?b@X90LDbszTNyV|@CZHRbE-paOzkxz?Ap;ecVU;2MB(;0gZ9I)!6xk{rbsTdlKvKcV z8S*`s_EuJ_ckiC-UVkcXxUOrwbm*ki!>t zOMbl$0_W&aV}lOQs&ojlObCo#vnpf#ASUWVcV<%~)H%#SM}~0p@5U+o)a(%FtQn3F zC6cj{`|f;gz4w@Am;RyQ&HYTC^$DDTLq_J|sLU~Ws*Q`8(0!`6i0L$#mD}2w8ZqY8 zUg$ww}90vp4e4h0ILy_IF()D7+$L^xzotZ`nhUv{wNZ6H#o2;LK zBpU&t+&-dCyRDz__WF?!;C~*KNvXq8E`95zd{DK%#U9DI<3qg_zxBJN#7FuuvjLPm zhvyN!JG>b7J8>O_wqw%8S3MCbnv2-;8|~T(;li{tQVahy*2~LfKl@Y1Sdu>%8k-Q( zliaU1A((=B@;>akArzq$fwas&0zJ8Ox*wMI4^Wku%{JrBr7(1h52fg0MkOSdj*aobV=@sXE^sUmVJV`}4 ziDMpbiC^fwv2E`fJdA`=Q_K&%m@Kk-bpQqXuB}YUfV{!{#r%D??~5ceqtG4CqK|WI z$@+#)-an988@A7znfyMFeI$Lvh3hgD#??2p^ZxV$d-Ti016cM)Ifu?1$Hx6g#tTy? z_C&=V65uh1PA`jCr^2Em!i+`SEnOi{si1b5EIfFABZY&h#d5Q(<}qPO`ePOOmF!Fg z{Um?iDa4v*+VnIrr58M-h?yhShn7RZJ~nIkywde!5&HX_@$EIjW<@^H)1{^A@nlUi zl3As4;C%eEC0pE2ghmU3+zwI;3IRHycd(E2k^2YXNsE`Mi&{vu2pfmDet!xPzxloo zdb+!L|yxfw((giJ)T1 z)Q{b(PH3!B=F~8$4CtXD#h#T(7m=-+%ObMg&_l z3>54%HYiQn!f7=wwykq7Y-_d3tKe|4UN3dDM5>gq9+)3crl|7l=H`&-MmwAdz80@YrzIV5WqT zIjjyjHq~eQ+H# z_*F>vLH*&S;-VbrejQ%p$iYXqyDGgPBMmq$lK7{sTj0a7to^6fSFhLI>*o%M4aTzY zF+E%MSXC~X>s4t|seU4z0@r7gif83ws<6)v?X1^(TV4uLzTXVv$R-DGOP+Tkw^n)R znQ(YNGF}Ix1L;pY+wx*B_=9J~!%Q*fR7=i*8F-{hm4b2QvdO1;a@^JQ)|N?91(xaz zj5dv#^evO`DVGZ#Ds#h0&}hUAf}$3^m9+J}#G4CSbPlzg!F6b{q{*X*8Oxl6Ha)%K zq7}JywiP_JI$0%F3T7*FgG4!E+}wI`cpvc|;b6uG%i?sc-g|E4j|aJgVx9QJU9k%v z3mf&8eq_GGxsje;CK;?UhgmwU&&RhW9@g(NjT^no9X61N8-Rd$>VCm196RomN3 zb44D-L$gKCODO<(Er&=*6C5r#Bu@4-^At=Bi+`x|I``5EYam$qND_HSFFF*o8O(U( z`T4Zdgp9~~?FK;J*n(XNRxso9eB%b&Y3&{kzwsub?&OR0b`AdVl!$N;w`L3x^c;yp zbqHC@exZn9$2!}?9L(T?;<>REoyyOLB095ZGR=Sl4bsT#?V8Lc1?ZKCeT3duc%*$?q4?Kgcb1K`B zCeR>r@x`5@;B-s2@@?-RtV}4-p zJK}yze%`$~61M-&iTp`t{xkJiWkT<8OUpmQ3KA}JQA|j(Ls!j3co%GiWC;%wTPgTi zPKR^ER*qR3jarJ5+47^6_i`V>9mHK;A$Gncqjyi7>pAy*hV|G|*QeJ9STM$%5JPKp zS-I?i2=Q^etC5t!c?8I%?1ATWf^_Ju=RMougXJw1g0j!Z;~O&y^EbsGnIHRjIZ(@k z41c~xRal7fhkZTaqc9t`;{!6Uzu$e6SUXz7+AzCYzT+f8jp*}ST%5=h%|ZtUwd^|d zo*I!ttYHj1jN|T@bJ^Ylx>~WSk(w^kH_4KjIBrI}NlRfX(dydzHSA&Oob53ytTJ$u zW3BA?C*&~|D~Xk1rXBLD2gntuv};sNlU{A<_D_3(Q@5krtDw$xXZOqYM``N(aVmy9IU0G-pTU5Am6@m$y#Z z-(CY~2DfyjJC|sRJE&dOon7J@YBFjOpEuoR7aKl|6z!e}4_M%$sAQN7O)hD`U@wxj z>&Mf(`~s$DCgJr-V&&UZwb}_-e`kn&F70zP+K+cVkWIr+_&8jvz&b8NF? z%)-p;cX{vORT5WcQ2x@v9u|%gg#N)INR)`0N>RB2JBT1OifFbKg72sR_H6WH$Y-?D zv;PZ0o$tpG%W;U0!ZA;eBHiE!upXL{H)!}`nO&NfVZ$tITF4nN=GzVY>CowypvCr%vv~%&5EY%Gjlo>9QjPenS{y2qV&z7sv z%#&d%MObHl;F)YnCD;}L_$;l<7ee6zIUOYr%E4&{W^&sAD4;sz*BXST@}6S&-?{8S zi;zz3Pmrc5)i2W1f7q$fzD=v9W(}JS7f3`sHV1T;et-%@z{g0~$0=+aIn^6dX396x zbYs)3vGw#+8K(ghaDU6)rLb8d6d?(Lv6DKn1$DwTZmv`UVQpuqL$?df?Pr995{yhU zW|KrJNhsl*q(hetj7yKGUme$!Q2HPPXWtZx!0_YMYPFh~|Z*L?FN1F)RPS{by2tWpQA z5%M(Nn;FjMGIW@+IbtAM#FphkS{!DhZN=5m0(e}%RIj!h4h!mB4NMo}HZEV3M^LG< zFU>McbrOdiY02^G6Vs|A5@sd2z^1=gkO0rQ>%VkDvx+VV-b3f@k(m7|l>nQ#VWOKU z3D19`E-I!aedzg!jICYv*?pG+LY6S^7{IB(bb*nxOAT@#1+1iSiFue`K^kW>aj*CU zq_Ui4ZEls_cXd9hj@=McJ?z9PFm@4;o;e3GM^h#ka{j9MTohzL>|GRnPVP;p1$~I! zO=GFu{pqWYkYr9lNqn*d`v7-bZCJMl_SE1_(X; zw0hB>B)eQdL9#cT|K!kbP54b872opBmGeUX*Y5mJIpE(3wCaQ=fy{4=T~&{{oD|td zFZ2$ZZ0Pb^3@uCY12jA>WD+x%Cr*XBjsJsOTWIdu>DlXIdg1_RzA4FpKwFgdH;| zPf|T##sI%@zx|{s)3MS;LU>M&7$YwJi+-P7r)*Y0;?WU=c|~{irVoKhDsji|udJJG zYlIRb%W$BH6k{DzEHiVE5@?I#o~+O1l5^kBtAXprq^vw@mmOsc(I*?=?O*>*YW%pf z+V0Y#sFxRdP&T4WagKk9DYj+_a}l(Ew#-tv=8Qm@(UZPMC(e}ELl(t~clvV;UQC4T za19H=c=A2R=_i&zG_CHRL+qX(Mq1NT)XX=%#AQNl(z}Vr^{}fwtpg`;8e@f6)XQx zk7Gh-nM7;xnRryu9Vv1Q#gQzb$3>74+c)G)elN+341#?5utA$oSZs$Pw)8Bu@Cf^{ zOux0);i%6|>w}wEeeAHDHbB#8crkE&;xmtAz3OdI$**&iqgB`goXYg(FkBbx?XN?^ zhiW`)Ct7sIVzJufmb@lmzO7h8@Ass>RqhlX)#)S%w(p3sT~>?D?S8fvU1{+Avb9$9 z*xk=0t`?1p-I{ezbs{}Ywbz_zE-ajnS8<{IaT9z+c$%MXg-Cj*ji&*N-VHS*s#7;hweU z#{TV?&dF^Lj+Nn8&QHPydgZRb*XO%}!K#^*bEpB^2`S%oe~}o=QhO-(&|3;E=d?DV zMK8CXKL-O-*8Hv7j!1zFIR0FE70Bv> zj;KxI2?{N9wj5Z#+)F^m4UVu(Z*@iE3&cOE`@067hefP^^j7iXe4{V>e;5A#q{RLn z{Ng6wN`1&fuj%=YJ82Yh($YCT4Gcj9)!?Yf>bh`}v3Ml0jmiU-J2rFlcQ`A!WLY7s z!X&7%9$@+)PB1ghq=TGf?W@VjlkumQ$IIiX=xng8oIYexOH`^Hs3|d~xOnCzK1phh zl%pOk12W6Ip9g$sK6s;|RvT}@PxnHBFQg?6GH@n1q&|?PZ*hTK;W}g}qvsGwAOB00 zFUZbA?m(dLN(fzs{8k3!v_AqXa@*ZXpgXUdP~EF2)vSteI{A3f)txh$pLUWSCsT{H zjz5pBud5O8OV+$7r1_)+uM;&Gaykz^fk8eEZ09BL8j$L>RmOkpTi-%|HwHt zerqL2_84x2fkyLEF<2$SeVQlrS?a{-PY(G-pCqOX6f)?0J(j z^%1j^SGGut?7QTS2Y-R68{nD`E2s)-0{<2rzviH%E&*~Crd70ZS_inyCTyWDjyQJ1 z%12Zsw&*j7SAgAKwMfjN9v);&!x*cVo#)6SaqR+HL^wTPe!9AEvb~QPyd!zvMBh7p zW*E+6Gq=`Yt#)((98yQFHL2_ns)P&4xN`uw69C5 zw#IfTPihBAuvN~IHo>%EfQHNt|D>IeTPWnrZ?C=l-n0|vTW{;X)K1bsH=v{`&_Trk zXl(j-_^4Pbpb4S#&w-Uo<_nY)&}E^BR4B^ZEir<4!FaP0|YxTT~E^pp`Ci`X;`5mD^2hHD~ z8Eq%6f1Y&d7Qa%Iej@NQVZnw}L$=+>y%Fhp^q6P{XL@fLGdj!Nx%Q8`yRKW(sQ8pw z`XX!!UT&cS>J{Y0LO&|)Dq;ICn9yd}acZv)LyTV1^I~6bsm=V3)&@$9aGWVBH)%>L zp5%E!GqQII?C~i>0cV|(@wTY>x~QK{+7Z>_hKa1>(zw%Nz9rMM@D;~nbDPWW&TBJQ zlXSQ^7`2I^@dKtsm{HjdR?)APtCS>1DRS1~kn%AtB>Zle3Md=<#ML16`sp5ln<3^b zPRR}GV$mPssl2=eMC^kMgQf2UvtwY_LtnNfzh>)$+UjwL%LsQMYdGQvbD{7l5!@hz zrKh1Vl|_&WyV6z(nY{uoIc;^NGedtWDU*L@Og3)+pDwt>aI)T5T8eP;r<` zrk|-wv2a`Y^e&4{>e)f$%{d(S^1heUefMkT86s>C+N*!zHWXEBNwv3wA?p$I?_t_q|J30wFb>=vgUHLrr z8M&V^C_gfle^&b$Uf{wc2AS_=W4x8u(3k=tvQ0SM(U}fZKS_VANt*8Y5|NA&^Ab6v z7sL@*tZ7vSqoSK{;h}q%O&6&UM(C#7gp6%@SguvUhhVji$SO7mvf4Z>N**c=gYd}W zsLV5OHK3)rz7=3ohXzqn?cnyK~Z5ESqo-t~;gX zj8>V5{?!9yszy6J?CWci#zmv4Xrvz?5_(G5buB7WD7BVuldh&q#Az{?(P5jdLx2)< zyr~t?Hel$QjAzj9sR1Om!m6&Exl5;F)Dvn*PDraN@<}DMQ~R(59C6uFrNjkK#Y?bN zOKZeokj9cxr`g`#Fw#uOw7tu)FiRZMMjWwv{;BV1Nh@0`Q6^rndCh*_O1-74ZbQxl zOo}vQcK`m+lpV!cccR4Y7be(sUdhTLiab2Am>w@qA81(Tcph|xCrT_1{P1J{voIGO zjTm3U5dwFB!_cKUM5lEmloa95MsvA`l0EHbW=@;c%M^3&#&DvW`?Aj;0N@VZF~ew~ z;PCAp9@{y9he-I5=*5%N-h3()9q zgAmOBCP@FSC9h%cWFlhkYUe_xY--|aZ2AWr7u01`(1eg*KsL28bV#1m0mCw|LDd3E zuwMZ}dGFAvs5%a+Q{lpaoDC~#Pt*kbS_B<}IciDfuK`$oU(~|*(?8b;5)`}-E=|q) zepdB(e3m&;bbc-X)*Q0J#HJe(<3b%U28bKOz|iP}*-q01^>IZ7g~c+F0Ms1y1`t8h zA(p9U01={ZP2FF~P9qG9<|x49!*5qU?$_AaPLQ>im_McwS|!F~15+bWex>)BT;rA-v3@|fmY6gS{79?Bzwfkjh(7gLueN3aqfm}b za-P*9EegFHV_e3SDKB!8s!;*kU+icEJm(B#xvGWvT!|-(zckN-qVO z)fanoBlGqS3{U%pT%rCz`6hnfvf(ks?mRHq#v#P^AU$Glwr^@=!I+M6<%Y+29wwVA z^n6BaRLtgXn4af*xW2V3*E|kC_lf7`)>Cu?ZR{hP-D?vPKaU)KZAg;mIgPk8e1%@tL8VcYupZWM+W}N)&w$-p zX=xSGQg z`oV=6E9&MS^YwDFzV&7*i9Z1D3x zW_@75ev@?Sm2j2ojid)}NifZSNzy;q8L7Ie{Kigx5fXSup=FP*ToPzaJt!0m40iC` zEfPaypER4nYSr5KZ#L~dNsK*lG}%5^d30@@Sk4;Erp|Oaph7{+5mR2y5n#-Q5VQU!VzUd~FPqPSGF~&?L9Dz%BU{ z3_n~WFakV)hDu+(<@U-dQJJuO4WJHl%T-myh#JYp>K~`CJ1&^%Rr0j_J+${jz36s+ zTD548mKcis2n*80J{>huhbE}PHRM~nJjjoYD3gh*5+FY8FvZx z^G$q8xj>C#XqKCLg{S4pNmyMw5ylZG+%9&U|TYDO7UUg4?^_VA_{HTooFYLaYC$PthOB^V*ig|=BKuE&1m5Y=SZVQVsVxt;yyzu@#7k$+Y=?l_Fmi1%@|jhv?4Y}J{**Y7p3sE z3d2!vY*XU!(o8#fR7`EkjIV*eHFAtg&cN)&Rmh=j&LdR)gOUA=2$kpJ3(t-e@+|- zyx$+zW;@O^=ape5B=;uh`19<_Z~iHCDFB<__*eZFBpLoo{*lQ69sc0pg!-xzt{5^u zX+?cfZ6^f>mwZ4b3<+%`7L0}t6&cC9Ks*Q*jvrUm`f?+ z@;kQFXytcZo~oE@jizt%5Sm!R9vW3R9aXyNpg;8wLpIh)4C%o=9SOxnOPz+0^T3)G z97?C_AG3_tXVMps@911*bq5n(SUHc(u1uFKc6Dr`sfFYp<}dyTc6F0&6+ecPkD zd7@cTR;niVJ*UMk{bm_xw!82Rnl@8RJLL2|f={cfmYb z@3kAM%tVVR&{G=Bo;i;Pwe+E-~>h$}Hu;7A>JLZDpDcDd`2 zEh#|ph=%QkRkTz^0AilT3Ma0H;w*+Y>BUIli_k~Y6)|=bFJKDe855z^1)@L&O(ppq zf&_$c16J2g1_p8wB*LB>v6FX5#W~3fT14iZzF^1BrhZ@>1(nw09arE!7u*7IzoKTB zldO}EX<=}Gd(D^KU(Ge}&H~I?7CGnE857SiA|oZ3vvpQi{POjNzK=_R`1q+;?%Kx1 z=a9}^BD-}S1p5yg#>$V3Cyy478p@z!VHrlIzLX>elCr$iJ~k&&0<(lfa}N4 zdfmn8~?0k{T?OV6pIy)`(2QDYv%mBzTzJvsQ=W-|1-4qdjyH~Z>gKA&hKi( zOCgq(b!__AO{f7ldMcUlC3LB3>GBdB+X`j@xz!vxJM*-4tpJJ_N?(+`(90D_i%A`CueP`Ozv$4X)#PKTC#s`ijJcdhGJV#YFTu8xV{nM!GN(|PYCzB`D<0)M64!1e>gA4SjrhJd5Q27m{XiTrE; zWE?`xhbHnq(K|iAcP(BxMX_dpyMP4OEiaeUr9BCR6HC0UKIHxN%4>fhg*W7gKZjDu z;qwM;hG`0q=95SRX}d;e`HTcO8v2YPFS}Zjc73q*nfB!A7~1v#+oUl|uycwLK(nTx zLIe9p?X${(glrC$BXzK&Zeh}a+^Vr=>C*;*lI2XMtpDKr1|(>Olp*f;OYzc|K(ywp zL8?LJ5W^_q+RTqBxmef}XA3nk!g!njXc=j*G#S19D8}N-I4fARvf_1Dd9(tk($qlp zVk4Q$wR5&0T#}K}^(5U#hC_v@%wyQ4Y+^OF&Q{NL%!f5rG8$j(%M z)IgI!e}P6#qEkW^>z7!-WQPVq{i-%3gtIDnS571_-PmUaJ7dCW$3fLl;k(p$_griL z$n&h_EziZV@A1^)r@Cs-(L+NK;?z5Na+2xY^|;P^Ke_SxpR(HDh#?(EdKR4N3+mXReianw zpPtF@ zwy1RFNp_+4FC4CkOYGoYy@`wJ$aLtn7^KNe(uP#$wd;++wv^W*haf##k(j_)JkTsd zKN>xN-ms!4*z{h!ax-NSVUfdKB{kKJQd5_a;55veeIja+;|}5^;8%hNk%@K$vK(vK z0n)=+bRZYi=J(zPgZC@Ik4fRNRSAwQ=1>MW(p1~-C{%*%68r@eq1$kyao;~ahPFe1 zlqp~hU}@1%yVB{KlPw)dFmYf3_a?b^Dy?gZ`vujq z7Ab(=HtkoX>t?-NW}LmvqFc|KHkMDP%-4;UmXykLhov?oX+aLQ>n&+0C-zxv5GjZ$ zV!zm}`_4m?1Rj|?KuK(un1+5dzkt&eq0Lc7thfO{rOiJc5k*h5waSmvi-NcFjv`HI z|Fdjs5ZjQu`lfJdP_}U+YJXe#26UnF?hjJpeu01;0qY%;_7mmz?Tp>fKK%Op7MYhm*nqc`F#poeBxjKDB zOaHwcce~lm2QW(amdXt5NhWz-64TuuamA4S)sZff@o3azDXUUoMO&pfnH^Pj-F-20 z%oW4y9va3tleLA}QeG-_;dGtg9qt5|EML)*CQO(kw=0E)k}mV5b{V&H@y#x-e2MoD zKdB9Q2K!uZ(^+}wB~lJG0Gu(Vcx&q3wTCtxT|j>~tD5xsVBOzV>M7yq5C2^?j_rMZ z`tW49O$qfyo>T<5HP#VpDZ%<|>T(i|4DCBRO+9WMt?8*4QmFA7`VZ`dx#96vOGe`i zM=)RIG2rR@WroQ~7@cF-ePo=Z3xZ33WpLpd7|S-SGapi@kwlb!X^kPXwWdnWs$QEN zjYCmLE7H7?IsX+B?<`lg%-09O5xAv+5?uL76q~TxQ>_;IL@F!jYgd4L0Q`mUkal^s zlX@hw)lbF?h>av?RX^=_YIQIMmj)KNGU4Fccme3pcDe&sjmQ3D)8KwYih6=`Gg{^2 z#;)(IxpkJB1){&;aPFaC(-C>OtkQk^@@=)WEx^;|4pJK&%0=a5tIVUlhBLb8YIzRL z1{SkAx@@X=!TfolCo8RtYgw6#)rVKE(6AVrkOnH@ZG^1V5r_5=1KaZ**bBbtk zbpe-B*qv(aQ2+;(vH#Ry?NJ1G>e=o-;QlNj5RLWR^C*CCb@z^8c9p8L|I0&2kn8oc zv5)Xi%9MKihXnK}qTnem_*#z0E$2xqs?|_Nt~m!-on~G+Bfs21Dn`Cd9}M~uZhM0H z{Iq%0GyPHD+qD_N2wI2KxQSw%`RNPBBDJ(%z6At`0g5;2u}43OjWEI{w~r69AH+RX z_q@>=K2dq*BhNq3PV_(5F=F&H;(0$j!*urDC!F5-8Vt(o4l(B~BIO zLXIdP`)|w_z5c0a@LRCtBH6@5y$P1DZyl8XwP5+%pnamMp33h*dmZK62Elhi`NhRb zs{ujaG_0-}v2d)8G*OtxfsA%@1(0*=@&bSx&YxcyhaWcI8YtY~ZLHt>zn7zE<1|0d z@;>WGe|oxoe*YPxoRzMn)K~@umH@8putA4!+iiKT=lx>^oVFUq*l=yLsuO3D`U<2m zRS?5{3;uN663X42831Z6veD>JJS&b?eaIDW2(FC+1fcNx?uz2Eb}w*Q$S9kSJqZ0I z^DVIF7ZyDQ0}}xMObT?^+xAW1<7VTO>1w5fKjmto5otIA1M{vhel2l8jB1Bt7IXE! z2rijgzkWX$VXWb`Og$KQFs)tC9M!l8ScIoU z@3J|A=Pb<-3o|EvYwV15=6n!);7Q`WF*1rGMjI>SSyPIN02EG!s<_lPN8w`Y_W5A$ zbez~Ro4nef*`koCxylco?GmBlY-Lil)>kHw#p0WZagS)JnV z|A;`UM&1`PR^0Qfsi;?0N{&h)R>IEJ5H3F?&oce9*i+eND6aXKIumr3Sl3#qK{qP!{2zBf~$qZlCt8ms6)J-dv0m;l28*z`?2#3-V+ z%pK0pmiwZg5%02@H(4jI_Vi=>eX{M_7@0M+xdtc1yrLW)0r#C5+mndR&QxEEQ=O<{ zUA1Xl%cgASGQJ+pa?<(YBb#{}W?$_7V0#<72xX^#Ciy8qL@RPkjqihpWSNnZ9TE#U zn+TK|h$o)9pmO|$_*bz4xmhk&{)TUuH^IXFZ<6wV6)XP`jQ0%I_Yb z0u>URs*+-VbQnF6g5Y*;zEUX}yI3AQd(XlRGd+^NdD^|ubpOWAV6feMGgGDRLwd;N zWL}Sv9Gi>d`FJYt{hIe#`^&@6J=)Kve?b>Hh?CwFO@ zm`@yI1#uICW~*ySD|=*szVtA2$1mv&P|LPJ0Sk}LF?v7Jm?V4Jmik4+VkmqGx+=B1 zhj*O#3_4~D?zSs5H$)1!-o01w16{^Lo~6AFEyi6x5GBw7W-mh7q7lIMS1VLl(2bOR zI1p0g&d){CD2BLD%rIrRzmIc05No9FVy-hQj>6b zqusgdbodQsWZF(KKJAQ85{l^8CX`e$QH~$>`#4i5cERZj5 z{r19%Q@MXd6c*}BM0{d zi0`+^A{`Oj=5F-mUBbh*+{uCRO!s6e#$=#L9K42X%Z_qvIUYfY2ORWZ}MZ? zZjGtyr`lHJAT2{|$5GV*bAwI!AI9~FZ!?^z6Rra!k|~(XEVsA!i?I6pJxw^oiE6EmaRYw{2V}5Tm_8gdY>uT5A+I zisS2BhDAUyw-6~e=kVgJ%zi$fr7abI^SqJf4sWXfOP91c@t zh+UznWU1a~4k82r?3+&cK?Ax-G4Z@&zuqIo>r*RXAsV?w5!ckTw5n;3#dXda%(VNX z=t^$$F`!0Ye$piYW$sfr1bejDu3iBsyf2T4m%G^`3LG$Y5K?z8CyqWiPEk4p0tG_{ z34gpza}anI&Daoy>rkUI3ng#b>IAWC{A$`N=oI^CvTw=MTEHZ`>xFXcgbfg<2!=Uh z5_RM4?!^N=)RNQN2Y(npEJ+y>hGSkU2&d(my=}gsS-XRhqULhV5yCqX44zw#IpXx1 zxwnwIoV8aAhT|WpT}W8F#yB;2+PJv!83!nv4r#F0*g?Io8c92CMqfq#-l$^E6s8QP zR^4(8cl4yFMW~X%fkHP563IjJkhBPLnPr-_M>l5@y%U)5fwUBLRKJh9LmIA=ekC=T zGhpUn_3jL!}{5!06Vurmy#@vxgs8ARBkpAnG9WV&D){*>@ZZn$ZrD3&l@ zLW82n{DXrlVO?g-2P`4~!s;{gIN3TsG2Txl0#HQKb+ zR}vWG;ZFxo_1UXN*czl?q~uF92YwR=^vK=$b9MH&&=GRKxLSFu&MLp%CiZ^|1b-{f zWUBp9s5O?-*C{_w-gJ-xi={gAu(|Wl zOXw1Qta;k#Z1J{as{>O2&-d*GszBmv9w&#}2w)pV<8fV;W3>^FwN@)mza~4m*T@t0a$&w;=La z^(Pu#Va;c9P-i!P0Ke~t%bw$Ae+7(i)3|D}%I+M3R4!}d{2n*~nfRFekrs{m4v@~g z<$_#YFVpR4TaGbC@TaxGw4-LKl70wlzx8WfR6mgtIZ5{mC=B>J+iYOX1N41XrOq}{ z-7u3!txm}iQa$KQtBGWOC#h5{(q6eQ!?WMK{+eoV_CP50!}a0XNUx(j4)p>*1eC_F zcgaGLyg5V^P))@1;K_hRu62Q<#4sFH+Ery6&_wHn(Uc(f=3+dm{lN80Z3m?%ve%6- z|04xbN^8quEv{G6ay99+t0i}(!cTIPleYXgJ8A0nfIeS?X(fb2j4*0^67x~LWb|;p z6w-IfHxTGv7jCCQJE0=R zz$2!o*YMFCD7j95b5$_#S!~? z7(4k;cMUgQ+K{r=_aVy$)e_EmCcrl~vE zJGi%~`Maa@fAbfAzYSSs#t!vOIAUpjl?)cHP_GaZgF@3QyA;Dg4dm$gI%s&=Qv}35 zm^THTjHexF9EMW}JiPmqaCs*EOOe24&E!0Dt;76`+uY3W`Q;L=TcjQ~8l3gM!9`i~ zFnYm;B;(>#Sd{R|og(cux`6@x8;|EjWg+fvZmRi{#)vpnPjC^#20hE~G7ihq`1^xBl zYG;c9Spmz82)h&7ju1WLQsEVPqj<|^Qd8LI_pgqC%(*~==xEbT9@BY6M&dwQ;Vep; zC2|v;&{EoF?SL=%yzQxFAvWgY@aGqtrG zs;u<7c$ckgY>w!|5Q&WfOF|ZO$9~l7v8l+Tz7?kP@WBmniRz#csaw}!ox9ul>QGB4 zZ`oyIg`Z(S+M>nW~XRY$H_$oA1T~QXS!E3=oc#~x^|1mT9{i_qoqM{-|V+&JNQ)icd zxoMgGpRfNGqAFB%ekbHFjrF2N=*0nRs`D_mO2R_zii1M%7I^b0RvS%+$^nOk@$i8! z6of9~v6nrBo6$~uTGC>hQsc6dS;tM38{F~PU7fFBOyLp;v{4)N`5}- zHkX^381u)Azpk88d3AD0dWYFsZuZRRCMetsxL=2|=DGnCB451M;Eb?atdkXdp;{*5wb~Tis z`C%RJRR50Qug^>_EoQO093r45?eE?om&wU^PodN=k+{RT1`=?|RK(w$L_p@&ufbiY z@6OT&upzK!Rz)}wm}Pet8-IRi;$y&FJpR_`kgNQX9}e8L|DOG;jYK!vG#CLf55bGh z`o(Ub$+Vxz0jJb zcKmvHrAyw>n{H!ORIj~O!_iF}h+;cKZaf@Pn%&yg`fB;j)BuE%td!_o^EEc% zxmB`rh%%roLp`LiJ`M)-on0eL!?BJcpiyGuVy7FWtP-wDa_Cw_gF`>eVyp@+wK0K& z%9{@XieCO)oP|xt3(|N@E?Zr}Q<3a!u0DR<<%7tO=d8JfjEdx9^kU!*+plRM20ULc2zOp_%ATyU~1X&XI+zLK8+xq>44Y4A9z+>1}FX!6xS`W_0 z5SMBf**;}JYc9<#mKF59Y5~Yo%m``Z@y2XqDeve}&3@$P=UwFI`B^E-F!ChDtPm8? z7pYnMzw#h+ovbbVjR!y8Dp0)tM;^R|{@*`FlBxQ~9IuFh8~v5|J19ch#;@gKF-SzA zm{NI3GVLh?iw5}?(UZ0?oNVxaCi6M9wfNClc3*a zH@jrnWWU|t`yb9dcR1&EJv}`&)jdyDS0$MFmR2X4+UY|id`&(HL=(Syd_%OHsLKn9 ze>hBFtv}euem>i^2MlF#d!*Qj(e`oKo{TtZA+JnPoee3*V&p}(5HuAWkuId{eKGFp)=VDSTX|YqL;12I zsd_BEkT7PDb)T1EGudIBZ^3Hty7_&QqJG>Z@7I?j(COEO%sLSJ{U*eX?Zsc=I=Zn3 z!}cuR(1@YTpMe+UHb?KC%A~&wB%*@$wLI_v=ZQF-p3~ACq!Dt1MxWbt+D8`%u-lY# zCF(ky(=S{8T#0UHji%piRy`vBenq|96w??n7Dj9VGFQggth-p@x@b;jY%EoYAe^Di zrKAZr)6EZXVMCkr)fZ>hUnddEDo&dUXX*rQyOA8Zt=%!-?lGgqXtV7si6)XZD5%AI zz{0zXpd?fpc{YhUo!6cqJ}m*1yd9ZF7L{Cv90%YdU~7<* zrTuvXm%4KHG&i2wgJv};bvWf5TDIB?yVTxL5tJ$jNx;U@;B9D_c!nVu+C2549)YV@ zuH<0@zA|B`Bvo*{!x*$`2fbFQK{7s+R_&{K3b-bm0i{;DbzGiueK)`%X&+@Ar)WH9 zYTq(!fIddKypUrAo@2NB#ndK+~(P>Ssx@zo%*rc8e=>5|F*e7(q=^AVYL z^kVAI72VX_Org4r^bbYHF|?~D?T>zm*4AaTVG*jrjTb$GFtMzc^JFilK#1kjsjg!~ zZP$6ts^wIrI>WhF`UGOE)7WYIkxFeA@-M#YIx0}t5p^@|zJvi9E^eR5Zj8AL^R}A!eA8+o`%Tv~a3rsuBA5x5O29Gq zQKN;MT!7im#{>YD$Q0_{8CRPvkeDG+1yDbITHQlctU=H-02xn$HLgw**7BrHvm-Wj zNrt`}6=I9O!yICFTh<6hf@kR2*iUsmFMIWCuVp5Uu^SSTR&TH;TXe{}FCD_5_w=a> zG>D1_f*c!`kB1J%$4#`#6C!k#y{{PtK`CMt`UV^ux@|}lntnT`32~31alH|~U`@|s z>pV4_e%xsF>ctUFlFQ|OE6gXo;++s5-%m97y-em|?$K?R*Yx+y`$`V<3 zRG??UesGTc-1_)<^8D2!5=CuA+p9u+Z$vgEh2Gurnk9wL%PnA}22}-Lf`c*a&-i)0 zK1wZ#wQ$%#?lO1}W6O4Gly;mF& zfrh$Whz2sB0H2(;gjlp|%wbJQ>no2eRiMM|FAssQb|*q6{Jf+jUi&ax~? zuVfZAAS(8&h@pWxJ$9$lMhxnqqq?G@1|~J@vJLbex&hNE4Aorown17!dVv_Y-j=O| zOYQJSeqrK4Lr`(6SF_Yv9LAYI_rfL6y&!%b<&U1GrJR6I%pVIo{m3t*114Ika8%u8 zqI;OJY-(Cz1vqQkw!0UW~*8 zEIXI6B~i&09{LxODIgqUfy&Iw?^3a#rykC2@rL<_%H02yHoqBzc1dUI!W4SXSNe$yga~K zO_&oI;A+rMs&89PW=yTK74oc{CF%I!F+gLJW`i?^S3~GNv7qr{7uWQPb4H|&BCiE1 zz+cS5GU6Kgh7k4&OjeThf;+JYpk-Z5+b*)s+HxGmklJKGys9jFNlHz2<7vz?KyC1e z(kH&Pt^l~;5s&Q+QG#<7v?J-zLqS*7>;@@0%sO~_v|k{vtw}j5gave|oz8}qqmbnp zPzQBga-%7!w#$JMtKv8>COxOhH@$ufOA3wa3p(zuRB{ppsGwT8-kxYIZu?}I;eeqP zcUgzFJ(mh@hclHD9OCFBhNm(IXe-m{rgHU3OpETIR~=j+K`D*f5p;^gzFbe(0N2p{ zj8U1q5KRU>t6;xRl>w`eldsPl&q72SRLDF6ku2K>A|KCBrkk>e)uuQe$=$8{dBqHm zs7t9PE6H28TvgEv5! z`08V?CzsSB!4`>HjE#S1L>(3~yFf9IwDA~JI$fbA^m2K^7$&h8JV!1fZ?`KC0~HTt zO6<|GATY*!Uc)zxLB%k(?R z7YByf->l5LJ6u|;6074#$+4C@5rc7OSvtQ^5>{J3-Fpd1)G5XsCB(8xDo?t|U?^`; zdjiv|0p>Nu5eE;eA+gMzpScX!xo{kD<&*IlkS#eBX3KJnaU)NlLDKih4ay@PJm4d~ zeqiATm8=abfxL(-kW2lamNWFfuiG=_x2>iU8=?TM$)l4}j5UIqrxRLQf6IXf%`?-bP3nXoz^SHZik( zzR&dY75v$8=0;3l!GIB{L`L$-^kR%o(5EvPyeHa1h-V^p@rzm=maHuMqOS&PGflc2 zn&z*404(||zRV+3-p9o4RBcjjHy_#Ai-@YZQVXVA&nD5#R!Ig!mFe(MQNyLfJ72b$ z$T1l=iI!|Vb?89fqqD5^D2&Bsl&}5L}o)FEDI^cWvIz(fefhXtmt_m7NTA>u> zWw=Bywi0LYC959sWsQ1jMIimJ@@FI~LSj z_$gsne24Wn^5DYdfT$*M^7Uh0(Kgn&7!Uj8Qgk&>+x66OGpTW;ddcZv({wkFPop9$ zrGDc`W(!$f9L4NDO7G zK>Z5%;Cie1kr@}VMAU~${1QyahMvu>(Z1YXZ$O$Blrif@1sP*R(4>nQmVzTEOSX}n zparMJrcwd3yRq_nI|F4Dy~#s^QaSS5Weh$07iRH(R$z^nnEk@ZU#ousOPH?xrai*rs@CIZ1v9{uHei=Psnb9b8O4oT_uZ}_ zfMjH~Z#YmpmUw63IZb3pM7d{kZl(pHhjL)bSZnKG961~Jb(gH-7Ih2;;whN|VzT#z z<&O6y)YlJZ&o)Dr1F33oUEpgr4Q?YQj!z7OdYR%@R4Lv=YxJDI{bAeUZ~m2SP~1}B z&J$m_UvCln-O>IxI_tX;c=OHqxjYhkQF)>cxQqxl6yy`>QKJ{iC>8wtkR4o#CQ!)d zhDXz|iDFB5xdB8=9t$2X%yOzJlk4QOjZ`h>tLCqsUy)w1nG^U$WeD}0uQtTn@7L|u zI}f^^_lLfHdxic^;R@Tr4ks~SnuahqKjA7#|&KZNVc(9OADW%Zp##J`=bt50Ph1FVz7{Zn35toe&aE)~nm32hd z%Fl#XUeb$=OQ#1t1Jx;PqbXH#-wgM;MLPnW;mzr=!HmuU#(N9o0J55q2KE_b(M1ax zWn@zJz;XHMus?V>VG?S`yi1C%38Y2Wa0X;NJ8^KFEw>?&kR4hF?>qMDhY-c6v`L^H zp&k{cEvZePDz_d%XUKmfk^Np-CJVFK#VJA*Dxaui%`zQ)8QObE%zgCJ*6Iq(iU6Oc znGTya%zJsM7q)Z@Chq81wrT$5y5Y)t(;;@f+d}mO?vcy*x_q+12JrogOu0_XuZvfh zs!Vu4gD6d!wN$RsF*#e#PRVbF2 z#>F9SMA`g|rtooU69M}KQ4MOY-d2QcDwvt-Cd zsxGlpLyExnDx!}|c~97!PRw@=j(hpAx{eR!)e~vhbL!4eE*Y~(?QnP#PEeapI}48z zUHGakG?^0i<+tRy*d0h#IyA{)q-Lt5=oeWfEykFfLWVjvpQ}UoOX6&_jlDI*aufI^ zI%tb%2om224-+FdHdRGj&}OaFClz)AqHOTMNR@ItdcW zUs14IVgUJe_FM@_g=B}Q5rW2J&lhNi($JK33p-7S^%%s)qn*gEU=c{wmk)`CU3yfb zIx8KyCI+u~?yT2?InevPqF&48L~8eRhyzI;uby1=$Hu$}3<>Z)0LL09b`9#Z+N9df zC@5@NUKx#bWdjAU6{Xyme6 zqMJb1V>Z&EueS(B4oUg@lB^L5$lIYTXHlD|hY=yD2Uf^qlnxUuxMm6Lo*xp&4lp8d zjD_Q+3suwC8r&Ie>qDxT!um65vFp<*{9qj3gj)}-w8$GWJ>7D8vy>nN@Q~0H$JqD5 zkqQ?-z>rlJRI`nuUNxu5x3SIq&}bqJW~oGY$x@&b{Dv%2Z7vW)CBrN=61k-O^?s9v z6)!%lG>nF;oBflPx6AGRn45TY(+&x=j4wiRZJg!OiOi<0Tsg)G6PNtv z5(iizrVKyEO7*SbkFhS`C$d+l6kwJ)`o~h&arPHG;u>I|iR7p#H%vn&6JYy2x_!cV zWXj@vItrPC#=oJq?``N2>EazgoRdb=zV*bYGDu4~JrJEjuP#o7waCsruxtW(sCe~* z^oKIHXXF(HAY@t7h^XtY_?%UAWnFc}zktWoDQ+%BRDQf5LEUy198%MeZzeD@%PKyj zL48wOJu%5TCW5N>n0Ks1mC1Dcbo|3~R<2p~o2eCs1pMK1-nrH@6 zwb|i3_yOi1s+65)ke~Sz>p}qfaJG9{m~X#b3Kka$*dIEGW45r%H@lOcx=XXIFGaJ| zUtrIfSy89GRCQ1OtiD-?yzeM2ExEt^6q#`N8EX1i0%hx0_*c|gqWmx%0`B2EI@KI4 z+0dUPTs#NA$_0Pf;ardOX|kY)JtBUWY*-3qYhoKB*-4iZijznd=kX1CPezQNu!(3X zYG*}o(u@W`35=ZqJp+;+tCOS2V>f~GQpeOPWXsw8kbSr$*BX0dT1B6qo4o8m4c=eO zYkLp%&0}c|DbC1T=W*3RV79!s2D=}NIrsd6%2wmM{lll19av!5hcp}Kyd7@NCG*ld z_Q;p)9XDJhx52x@R~Ba}PI4$;)%z%8M;Y2H>Wu zv7-3)vYQktn$XrP3)JTkatS%O_4?$dh*)vKa-akwethRCd9bPu9 z7bq$cA-a_|YbCW8AHXG$K~o{Iuth*eW=>YP-o!Xl$C?(IMF>?+ei8>`7@!T>sCSUX zOBmbzDi3;=<~P?SwerR6DkQZ@?hO6IRo)8jYouvYtQGnjr!Da$;9 zfU8u##G$gWI}y4i_lcj+K2ya6Z%F(E>Z({R9$YoX&5KZD5Quj?#wq-{&HoTqgkeoe zkLesQBHS^7RZx#|p)mN#9yPRl)R>Rx=!zwYkO{vxQm0V>9v!o|=M8AE2>`8-U#4D+ zl|1|ezOHZt5B`O8V*`b!q9Mjc7L_vzUSeYuPHwxZG50ZAp}y%i8UhTBJ(2|d5Y6~$ ziXn%=Vb)-5)s&6l=1VHEuLf(-*rHwzNa!6)SJ)J8-Lf%;wjo?%OEPYTVY1kS7@Qm~ z$)&V52egGDUcS|i37Q9FE#0IaIc9JbH|1ZYZ0)`2d&_3)wh;QHdxv(bN%L@2BZo3# zjjXrzX#fb~hGw+eRQY`DyuUPChHXHh6EWgtOe+jVYqY2qyu)%%wwy|C_n!0 zk~zmQtOPRE7I>5j%r{Eex$EOTjwyJM=$)hFP*(ivDN#&dT4qutio zyHT!L)sYdZXRSF8Y6%(8)^6cBJn`BLii|-GTkCpNLMW-pu6YG%x@quiSGGaTx68}!k7X1r;yQ!%>glKW${XY@V~ zXnw$M*c5qV8tyz-X`3%Bh91|f`$=TC4!ce7+bu}ekUy?fs%Xg|(@_B~tYN%q${nu16|!nhU_Hgiy5ZWwB%a>+=LEDOp; zwU=)EAFRC$2L!zg26vW5-o$AXTO`pqi^O-Bt_(`k6bSo%!pR#RTcVEPNFG^{XoeN9 zvaXGk_TZ44R&OSm!dvC=tEdJ=$$qre>Z;teVH`E|INmHkeGLt3c-f1|5`f$;5VueH(jGn&O-wuetNeqCZv6QTNIbJUX3)C1pFWW3vVO!A~TD=h*HLvIcQWqpu z=lp8VQUMm5Jqi1 z#g*5$^uSnkO!VH27MTgiHX{Jn_4_q`3~=;;iJvr3r8==iyib>UuJ?q3_iUej7{yH$U`z$1tV^jZI2$kDVcRNZRVESA z*a9<%*RKf?x^Oe2E)Dc)V?TunMHBd#1kFRy25OJXO<(9ljzH@erR*uYZJ@Vx*h*Eo zD+Hvh6Ivz1#QFnWR7{w+9Au{pQKil`0xjl^r7>cnT8nDgz@O2Z<)g&*MoW=E5)E(8_r zqLZf9?GKAiOMWHyC=4txntGp)1EOnZ_Qua37%ytAD$y;mtF9 zlnv(2hUaWoP64cw>jSCp(zHIM@=X{Pi~%X@40eIh%zeeH3Bta!W-p5aD-6k~*pO_I z3*93%0#e>}Vq}+>q~#K$8i6QaGSL9bj%ED812Vgd2XE@{-sh8G1GRn;_@Vkw&dA?i z)T3pjB}b*ut{dknzNn!~b-v~uvQe;jO{a@Z-4{CyL-V=W>>cdtqFvqC(uBxN{MUuh z5wKUkd1Zo75z`Na&8@5(fq#-_d-hGYUoT)DRRRzIuJFMHVc8$>XOEt|?UvEQzq`x; zU*5bK116ILpX4Jv`Ns%)MN`gt!HOFygmd(-z@uOHU2o2afSR6KH!4irL5TCK=Nt#z;N9GRTVkIi_tgSqNnw##9EwHYbdHyH!JYBEguQG|D&e z>!Pbwn%)(g8g2XD9K)5M+8wKzZBkQDVvL^No~7fmk42~|C32n48SQJK8v9}#wPVpQ zOOLo>)t(D)6v_D2P_0 zoEkMxP>#t*_w`9Nlxb9Vh;Pcd0GYNwqqfF)S$So4n$E5Oh*~Ap`h)Ns)sh9PRhk}qqO?A-U=Oy-JAi?3g zeK1^ZkuSn=pxSO&cu zu}$N@uy5sG0U_L(!ur-7uwYLUNW*Y&`^q5oo{!(!*tkiDCSeMJ{ z{mBlguzq2UNc-N*@app>%&}%~axr+ek_}lm0^GT($LnGHN}Q>sD9WrmSW5T(EokiW zz{kEJ)jrl@2vCWxboUA;?@UYwYZ+aMU7oO_mv|1!xbrDzf=R86TzpdDYkif&WcV-FM9qerGa z+rTvpML+~QSSbI_$wrxnf zUvdarE61Y07*+Ztiu$Q*>GNwS6vChZmzRabLP@KomC-#TN(v^4(VQ>xGj{of)>66! zPg%AEw5{J7TM`Nc`a1T`#Z~7m`aTWLM;%YV<;79CJMH*e{iVASN^<5xx(?qt zM_?g<83FjY=rH7_$5w&06`9;WI@Pa)uG;t`%b*YbQyV8er*SjG(H&O8;-=XEAo zam}Tl5HG=bE@)d@D(s7=5tqxcGq<~2b~(GZb#t>G%K2Cm3z*^1cD)sTcD-*|pubWg zZV@fRwR9mp$x82<&7=+o%HolYUaDIK#AK z!8_y6_DPB#lxcP4C;Ao>&!MDsXU9$|^mk7u1$Ew}9aS@s37{C+8EEL2lECEmnPB*W zG)2MfXNhmFbY*LIBSF~8juNQnj_62`cJs@LRK#bRDv1n`Brs5+M5K4sv6yvcR%6Z7 zGbx+FdE2kketx5!N)1e0fLB12dp2;W#aA;j()VG`FWWk2fu)cXgqF3|MejCBB5tv` z-oH{W4Ur@ji$O!X+6HargR{m))LHqN4jb8iVRbDQ=3cC|GX^Q3dn^0cKH!R%qSazU zNt?8Wsa|i6k|7(AZu{Gp%PDulw|R?O`etSp8b4W<$9U zWxz{jPUO(|!zdM@xCiIpDed^!nMhq+qb(ia-KibE-WJjnd0@HQa1oeTHZ7Uav2QJ} z#6H7%J5llesPpyH$jAPI;GP<1a^7S-$#0FE!N1j^fn<1$8n((o;sAee9b=5p_-3`Odby{#i75D9A!GddFzMsmhLD3@ zC^t=K2IluXOdL>KT>4MHROyP+5ii6TFr7SgaWdrs^Uryf{7g>TFN|zxFD@U^@R||r zL!*y5C(n`EjXhe8&prnai=u>T+n7e!WhPW2Vf@bH*Kng>-m&yGlXAhEHD6Q%P(7rigfbZxk7zLL=hx2 zoJl(`xcqghN$qxIPYb7_IupLM(~Y|aA#TcBr$525@nnh$N1~us-zPn^fOjV3y~U$W zot&8c0)4jPjucLZ3rsooURpmdGScajA^Ikq6?`r%;Ph242@xVjS1xiCL|Oa*>l8(* z?MN$09JbgkNq|KYvLYj*6;4|TqBe869Zos2JAFE%#HE zwF7OTRhhjEZya2?G-brKyH$;TR<@wPMf+jxbeTWY`13?>4hDdZ_TGl|YL}f*j%~2C z8Eb@Pes#%l!%2jVcsp^O-sF)Y4cwlkiU=EiFNZ*_Ite_W4E{_>$KL`Yt_>rs)t;y3%!cf=*?>=eT<8h+PT#_y0 zs*c{f{h=_`oo34A8a3zvr=}X9SL}b(%=TqeDFgxKL;lDEM=LXUvdNKM zCulz3-~h9LxJOIo@NriJ@GR6z1?~!zaWR5dwOlE>sNl{4 z)tUg)F$x;5VlW$))tj!7XQqo3(1Pt2!hSfP$xULFy;YcgT4{pr~YsuKnKO_bLjR4iWeG}I2vk` znAcEh4+Rps?Ue(qDg_LeVOFv%0xl6v5Wk`N43A*cq30*arz(T#=YZ~G3BoJi#S=zP z#W>CGi`F6K1SQHP@Q`VyAZ_S@q-RNxyx}Cvpu2V-oK8okcKA?2Xjdrg2JL(gaodJ? zUOz4Cw#XPuzUs3>Aha0i#FLAJ z|LC&${b^TJf_E}H0cX(``8e{hM#*g2D-=C6NZau*G7z>t{v#LJ(*yzmRK6M}eM@Zo z2i|z@4ZaEXG39F+I^SeDoB_EUu8^)ngoitR0$XB36cp1@S8Nbta<%=!yHhev_TpF9 zr)jsq$k}l>!d!7uKQxrQd0Jb#XG-W*=viPJJX?;1uG#(U1uwNM_VPqd;sekGhXp_< z(P>y$80X&lF1i?M;*4<^hGvP|^a^10;$^36;nMXmGr?BOl}3awj+jK^&;{4%j>8N4 z1dx>W8*&LVO^d?D*Fa;l#f$`De zm%%0XWea#FZ(1HBSC7N%k>8z_#w|`BN0@j-S9S7oi3wIK>THIt2R#Tokk>y-r-m0r z!rrP}a&ULaH#0WQh3;n5T8zD|W|lMxOE>$b%X^xFVs|Mto0!Xcg7^tGs%mHSiF;h9 zysBstgiNVe3ymV-HoVe1PwjK!Ssuz+TbAA-^|(Z5&$c?nSuplm`vZjXcQ2}i%`T;I z%afXA;I>G1{RQ35m=$g=pAg(45QNI-yFRA%;vBtHk{fkJdr`iwx%_Bvgs~0`Cc-Fk z0^&lphagpn2VZHUzdrUu(+}G;?w;a{zyK>v;QSH+oL`v!aelcgu_P7R+X5geQ8Y=`Z3 zq37;NtOIC<+!KGeg+K?c<{biXEF5~ifG%2Q+V#Exlg8_-D`YNAIP@(u2P5Hda4e|2 zRcBweMTOb5VSnT%sC-n+O`}rT;{B}s+;Yb^Vw#jzZYR)PgYSh)Zu zynfVEHdwUUL?)QnXL`sGGz-1cd!>Mj?R3y<)Uc|Jh9Z#zX2m68v=A- zn;#A}qlN71yjENtyqRVQ0k`yY+Sa>eEU{|AvJieARj8b}xGo%Va^@p#8n&ZuG1xfF zHb07%Vk*8@1T;iBn`n;8f$|9h1Ie2@c5S>Yp%APY1#Ao9MIi^7PE=8*8Ugx=qcMIL z;SrG*=Q_z%myMfViQUMOE~`i{WC9Er7c9wy?53oh=fcXx`$&IDI6np#=E|Hj4=niD(9U=+kN^(L?|-x#;=E>(8jVMaOygG)uthErfB~5 zqENJO)Vk8Q4O2XQ?qklTG6)$U-fN7Q<)bt+Ci(E{)DYICdj$8fQ114a{@m-l_SM*d zlF}0hV-ZymS!8FJN%3He7WUGtJ!dQwz&@$f#qkhQEw2xl$odbPD(iSQSoA2K69n*-^dt}#jY$+GH_)eRzPL(hbX z6y)(LsM8IXOpQYPmaUOB+iHt^TD*u6(t0*LnMxwYzIZ5F$A+{oDc&^_yB%{zHLKAE zZ;Hxkhq-U3mcFLtM#vkLj*1t)R+wh3arR1p{iG!!cscmN7dfWK*lKX+jg)-PJC+@- ze%AhJmfnvUM1+Awp#5B=w%WNKq8WW_aPwwj9L!_kWTjV%k%=UTX5%C5Q2L}SZ~(%NYrhO7;=txB70nl6+*f3vfrqgm&Jyj4!`6g%ZMa-3Z5d&2^hwV`Y-T z;3mCZxlx6Rd(91+1?Qb4g=8s_;wUYMV6CBPw@NyD>(Yyz2AhV zj(P0%r!(~L#j}9{LHEV8O#ybgW`7pr`umS@|NdiOooR~ib*3r)UT6AYsczDLCqUN$ zU}|Av20Na5=eoWMjB=aOB!ZS%qW=1mg@{mIz<*bHnCNg-1%E0aLulA)MO zj%CgK_)vAuP2L@w6~Cm8Z#GFjXQi*fyhpq)-#?WS$ba`#B)bgjBk4FUvYrsXE6$SL)0|3kWy_A zohwP8N~u1zYqoGJuEga|;E%hKCNAdy+j(babu^?j%bh@o$pMNe1O3+01e3}+-?MA~ zbygaLId?R{SeJ;a7xO1S644a1Zxw{|jP0I6G6OFC*pDe6quW0_B$9A1Cj^ZqU~#pGBZ2v(rVs|H+sG@_%cOW&rxk4)5h^(Q?mF_4gRI>T7srKW z)<^>=9gP}-^~8K5XEc(KPDd9lrXfy3JJur=+ai*oRdDQfUp}yyQ6S1CJ>NUOdHqEH zCT{?I$DJ*b6an?qK%#bIhd~15%iK6Kz|#VeoQ*F$pmITLiaBZaP~)D=<<`uam)4AI z-f97cOv#CjV!4}jvWzy-xphN19wL2iy8`3fZs}TIU`nX0uulq~WA{IaSnRlr z?$4p2c0DTvT;n$}zM!-E$l|PZIvjU~y=WYs)US3OoX^T=pnA$)vN7NjSm@Hozo{h| z_M({!@mas+)^?2mTw1qv2O8BdK`E(6 zhX}UupEo=LmeN!eVwPx=ni7)LVYhQ@8VX0Q)Bo>cIU7T6^%7iPn(4frj>U@Ljw zzUEmURpm_~K$J9`Zwz^*F>JXa)(Y$MYSv&mCt$MB?o<=L_+zhZuprY6mWXHcG!rFu z2WZ9cn&$rQs5(}|5p%)uIK6g)jSfIr5w3+5Q6>t%-Y}}+o9O5j;1syBef$LJDI~Dt ze>c3qq{ZJ4_(!|kcc*_|+V(ap#AyJodn)rNHBkJ1$XJ8|7-|6dGY5z zcTyaGA?15Tg&&9FVRtb8BB=Bif_`7={V3-yVCm1Mz!R)L|GDeXT|K6s4b1Oy?&5cU zl=ZMrul^#70JsYMwXEMbxqlLNw;uTYxa*kq|G%(1hX0=g{`E>T{!!rF+W1};_@g^@ zf*SLl;s&;Y1uhd9{^%R;ZM~u7CoY!ABm;&Ad zWFQ9HUodQeZjPUg<^zl$D`fqa%?$55gY+ZP&x)u20hZyO1Kiv3FOYsHqxaXeX}{8Fxzdu<+VFfjV`gd~g z>l6Ig=eybCVW#tY+4N*Tm;L+f@u!pj-U0JHRPZ6f!#vaX2ka@X^RDNpa^ zt^qTk{!C$h+4G+mxjTf1Nhj|SI)P-&e?s^%1?9s5dYIVn9;S!oKf-)>eLX~Ym^AGk zVUYblBK&qCzqr#6b1mJ&+;ab4V19919%kIRhtcHwCz!u&So-Ts_xJoe4~Ow#ij#X_ z1Hu0r@b^3r4}l)$nYafM6ZyXZ{g`#)A=<-@{`Y9iz{TaS)%@4v`3p6CxXJn+h(PN9 z2J}yl)x+Jb_h>I<{|B`1dtDz6BKL!RbN6VJZ~hb7&yC%~T?zN%snq|I_&;_27Z2CN z!0~&aGtK`S&_7v>hmpJYXv^CF1KKaH?P0{^J=&Dse?j~CEqWLrcQ0MY@SmjrIg|cf z?A`kAVPw?3*ev6J68qOMiJ!}U7<6$jJKOx<%f4R$JPg73@gcdpmWQtNdpvHd{|WCW zH~bFoq0{dk4-82E{Mxep!kO}umFpeKLnqBWN&}E!`p+m2Z`I#9o;xP~_XGaX?Jc|W z4`j_By!<`$@sL>m@lE{p*68nzfgSz{PycIycTD|#Z}R?~z(44;A2sA>%>5stGu%)9 zo8a#}{*Us1#{AvOM|$_SVf@)h{L%5h@_