From 45038b704a6194afefdb9e59fe3620ccfa026f02 Mon Sep 17 00:00:00 2001 From: chenmingyu Date: Mon, 18 Sep 2017 17:56:50 +0800 Subject: [PATCH 1/2] add metadata design doc --- docs/design/metadata_design.md | 102 +++++++++++++++++++++++++++ docs/resources/metadata_contents.png | Bin 0 -> 14356 bytes docs/resources/metadata_stream.png | Bin 0 -> 75852 bytes 3 files changed, 102 insertions(+) create mode 100644 docs/design/metadata_design.md create mode 100644 docs/resources/metadata_contents.png create mode 100644 docs/resources/metadata_stream.png diff --git a/docs/design/metadata_design.md b/docs/design/metadata_design.md new file mode 100644 index 00000000000000..61d22cf3b7147c --- /dev/null +++ b/docs/design/metadata_design.md @@ -0,0 +1,102 @@ +# Palo 元数据设计文档 + +## 名词解释 + +* FE:Frontend,即 Palo 的前端节点。主要负责接收和返回客户端请求、元数据以及集群管理、查询计划生成等工作。 +* BE:Backend,即 Palo 的后端节点。主要负责数据存储与管理、查询计划执行等工作。 + +## 整体架构 + +![image](../resources/palo_architecture.jpg) + +如上图,Palo 的整体架构分为两层。多个 FE 组成第一层,提供 FE 的横向扩展和高可用。多个 BE 组成第二层,负责数据存储于管理。本文主要介绍 FE 这一层中,元数据的设计与实现方式。 + +1. FE 节点分为 follower 和 observer 两类。各个 FE 之间,通过 bdbje([BerkeleyDB Java Edition](http://www.oracle.com/technetwork/database/database-technologies/berkeleydb/overview/index-093405.html))进行 leader 选举,数据同步等工作。 + +2. follower 节点通过选举,其中一个 follower 成为 leader 节点,负责元数据的写入操作。当 leader 节点宕机后,其他 follower 节点会重新选举出一个 leader,保证服务的高可用。 + +3. observer 节点仅从 leader 节点进行元数据同步,不参与选举。可以横向扩展以提供元数据的读服务的扩展性。 + +> 注:follower 和 observer 对应 bdbje 中的概念为 replica 和 observer。下文可能会同时使用两种名称。 + +## 元数据结构 + +Palo 的元数据是全内存的。每个 FE 内存中,都维护一个完整的元数据镜像。在百度内部,一个包含2500张表,100万个分片(300万副本)的集群,元数据在内存中仅占用约 2GB。(当然,查询所使用的中间对象、各种作业信息等内存开销,需要根据实际情况估算。但总体依然维持在一个较低的内存开销范围内。) + +同时,元数据在内存中整体采用树状的层级结构存储,并且通过添加辅助结构,能够快速访问各个层级的元数据信息。 + +下图是 Palo 元信息所存储的内容。 + +![image](../resources/metadata_contents.png) + +如上图,Palo 的元数据主要存储4类数据: + +1. 用户数据信息。包括数据库、表的Schema、分片信息等。 +2. 各类作业信息。如导入作业,Clone作业、SchemaChange作业等。 +3. 用户及权限信息。 +4. 集群及节点信息。 + +## 数据流 +![image](../resources/metadata_stream.png) + +元数据的数据流具体过程如下: + +1. 只有 leader FE 可以对元数据进行写操作。写操作在修改 leader 的内存后,会序列化为一条log,按照 key-value 的形式写入 bdbje。其中 key 为连续的整型,作为 log id,value 即为序列化后的操作日志。 + +2. 日志写入 bdbje 后,bdbje 会根据策略(写多数/全写),将日志复制到其他 non-leader 的 FE 节点。non-leader FE 节点通过对日志回放,修改自身的元数据内存镜像,完成与 leader 节点的元数据同步。 + +3. leader 节点的日志条数达到阈值后(默认 10w 条),会启动 checkpoint 线程。checkpoint 会读取已有的 image 文件,和其之后的日志,重新在内存中回放出一份新的元数据镜像副本。然后将该副本写入到磁盘,形成一个新的 image。之所以是重新生成一份镜像副本,而不是将已有镜像写成 image,主要是考虑写 image 加读锁期间,会阻塞写操作。所以每次 checkpoint 会占用双倍内存空间。 + +4. image 文件生成后,leader 节点会通知其他 non-leader 节点新的 image 已生成。non-leader 主动通过 http 拉取最新的 image 文件,来更换本地的旧文件。 + +5. bddje 中的日志,在 image 做完后,会定期删除旧的日志。 + +## 实现细节 + +### 元数据目录 + +1. 元数据目录通过 FE 的配置项 `meta_dir` 指定。 + +2. `bdb/` 目录下为 bdbje 的数据存放目录。 + +3. `image/` 目录下为 image 文件的存放目录。 + + * `image.[logid]` 是最新的 image 文件。后缀 `logid` 表明 image 所包含的最后一条日志的 id。 + * `image.ckpt` 是正在写入的 image 文件,如果写入成功,会重命名为 `image.[logid]`,并替换掉就的 image 文件。 + * `VERSION` 文件中记录着 `cluster_id`。`cluster_id` 唯一标识一个 Palo 集群。是在 leader 第一次启动时随机生成的一个 32 位整型。也可以通过 fe 配置项 `cluster_id` 来指定一个 cluster id。 + * `ROLE` 文件中记录的 FE 自身的角色。只有 `FOLLOWER` 和 `OBSERVER` 两种。其中 `FOLLOWER` 表示 FE 为一个可选举的节点。(注意:即使是 leader 节点,其角色也为 `FOLLOWER`) + +### 启动流程 + +1. FE 第一次启动,如果启动脚本不加任何参数,则会尝试以 leader 的身份启动。在 FE 启动日志中会最终看到 `transfer from UNKNOWN to MASTER`。 + +2. FE 第一次启动,如果启动脚本中指定了 `-helper` 参数,并且指向了正确的 leader FE 节点,那么该 FE 首先会通过 http 向 leader 节点询问自身的角色(即 ROLE)和 cluster_id。然后拉取最新的 image 文件。读取 image 文件,生成元数据镜像后,启动 bdbje,开始进行 bdbje 日志同步。同步完成后,开始回放 bdbje 中,image 文件之后的日志,完成最终的元数据镜像生成。 + + > 注1:使用 `-helper` 参数启动时,需要首先通过 mysql 命令,通过 leader 来添加该 FE,否则,启动时会报错。 + + > 注2:`-helper` 可以指向任何一个 follower 节点,即使它不是 leader。 + + > 注2:bdbje 在同步日志过程中,fe 日志会显示 `xxx detached`, 此时正在进行日志拉取,属于正常现象。 + +3. FE 非第一次启动,如果启动脚本不加任何参数,则会根据本地存储的 ROLE 信息,来确定自己的身份。同时根据本地 bdbje 中存储的集群信息,获取 leader 的信息。然后读取本地的 image 文件,以及 bdbje 中的日志,完成元数据镜像生成。(如果本地 ROLE 中记录的角色和 bdbje 中记录的不一致,则会报错。) + +4. FE 非第一次启动,且启动脚本中指定了 `-helper` 参数。则和第一次启动的流程一样,也会先去询问 leader 角色。但是会和自身存储的 ROLE 进行比较。如果不一致,则会报错。 + +#### 元数据读写与同步 + +1. 用户可以使用 mysql 连接任意一个 FE 节点进行元数据的读写访问。如果连接的是 non-leader 节点,则该节点会将写操作转发给 leader 节点。leader 写成功后,会返回一个 leader 当前最新的 log id。之后,non-leader 节点会等待自身回放的 log id 大于回传的 log id 后,才将命令成功的消息返回给客户端。这种方式保证了任意 FE 节点的 Read-Your-Write 语义。 + + > 注:一些非写操作,也会转发给 leader 执行。比如 `SHOW LOAD` 操作。因为这些命令通常需要读取一些作业的中间状态,而这些中间状态是不写 bdbje 的,因此 non-leader 节点的内存中,是没有这些中间状态的。(FE 直接的元数据同步完全依赖 bdbje 的日志回放,如果一个元数据修改操作不写 bdbje 日志,则在其他 non-leader 节点中是看不到该操作修改后的结果的。) + +2. leader 节点会启动一个 TimePrinter 线程。该线程会定期向 bdbje 中写入一个当前时间的 key-value 条目。其余 non-leader 节点通过回放这条日志,读取日志中记录的时间,和本地时间进行比较,如果发现和本地时间的落后大于指定的阈值(配置项:`meta_delay_toleration_second`。写入间隔为该配置项的一半),则该节点会处于**不可读**的状态。此机制解决了 non-leader 节点在长时间和 leader 失联后,仍然提供过期的元数据服务的问题。 + +3. 各个 FE 的元数据只保证最终一致性。正常情况下,不一致的窗口期仅为毫秒级。我们保证同一 session 中,元数据访问的单调一致性。但是如果同一 client 连接不同 FE,则可能出现元数据回退的现象。(但对于批量更新系统,该问题影响很小。) + +### 宕机恢复 + +1. leader 节点宕机后,其余 follower 会立即选举出一个新的 leader 节点提供服务。 +2. 当多数 follower 节点宕机时,元数据不可写入。当元数据处于不可写入状态下,如果这时发生写操作请求,目前的处理流程是 **FE 进程直接退出**。后续会优化这个逻辑,在不可写状态下,依然提供读服务。 +3. observer 节点宕机,不会影响任何其他节点的状态。也不会影响元数据在其他节点的读写。 + + + diff --git a/docs/resources/metadata_contents.png b/docs/resources/metadata_contents.png new file mode 100644 index 0000000000000000000000000000000000000000..7caf8fde161c91b968c92c89022880a2249be0d3 GIT binary patch literal 14356 zcmbWeby!zV5HBp<9g@;r(%s!5lG5GXAzjk-rMpC=OFEmKlT)yvN3)n6@Vuu zCgyWAHsw(fllZSY(BdaGcXqbtVPbM~b7OR4WwdiNV`Ap!=4SfH!o(!4@Kr z|9X}zP73mkD#aV}bgm)oQF9v+<6kd$aOpY(OwG6V`tYbyrTkD#G{q^ey@iB{zq|UA zB7_S8+~mLj!s1nmEzGx$)Uf1Jj7r8Nz_{rCG(yDRuQ6K+G7R5_h4L?$hl-$oV%;91`<#MuZ{CWtyiU!6g{BPJPo(yd*I1_^&YBfp7DHer>L5m<+wO zlKB9EL65xsxc>y<0-7*=wAz%fVpVRUNtL#4t_Y93Y98Ic6KL{tsm-yxwJ2hQRE{WK zkR74l1VfAlnw?dH$!{%s4?s?)JGWTYe>%4Znkdnq>)-UFKR^b^Jny*|urV`y0MePEx~$hWJp{@1fhOrNV1DIZHsW37gh zu#eDf5kdDf;9`XLBre8#bFVmi`F)#?+tQUN(Y?}&>D)gJh~+@}3*h$icr72nOBZ>EoZ%sH2>{=LhyVQ8x%Q2pC8&w z`=Nr5xh%{#a8wzmgX(SPzcCbl2cFa~*qUW|Y>&$_wXTKW8;%oQ=D^V8Ookou9F$!= zti1TH_}uXCm*j^Ha?Tkie0@6510$;UX+Em=dcMehId*GLzx2+J;Msn6<9fC{P3DBG7!Kss!qVTuBGn|3}2+6NWw&)O^Z)2t(05MZC} z;pm=<1)t!8eSVXDSiAJSX(r;|iSfP62P9onP~I8!Id-zDXstirPPZ%fySJT*!J)U$Uk(lYLa0vg>bs)HY9Ifx z!>v^w9O4Po`!9ZmSzgD#ZW{ae`^$LOguH1O_E5fF*JK;yp>S`;HvM_sl=qYItBGta zg&>yzO%gnLRT1AP@AkVtcJ<_3$b6?AdQUfJA)GC_|HoLszGVX;!6{thj=Dp$L}}!w z;MfGfE*>*|y&EdmP*N7S8&F&}2qo0;F&E0y;WV;A&u~$VPuiFxI`xCO+!z0Gm$Aba z`tGT|q4nrf)_EdXG*^Jl`rS^pU8now2GOW-@GL(vydpS;thi4zJ=L4Z7l%cZuXAX7 zc_|9i>GOR!k|cF_x|$(!oi)fxO$uu-&hp$x;#?Hcddhp>x;y&ynf3W5`{l9N)A|J1 zrPon*-lB!cD=*JCOs$tH$pK=c1~X^Cw~+U)&o3yBsOTZZDB9z7FU`-iVT&J6A>{kq zZGjRZJ~ERV?D$(4U(TCj#`F<=pmcF?dI9D6$JYFYiP z8?33?A)}OlLHoTM&C`C{!-O0M`>iz)nI{Yfa2THg^J4^f`gus8&+)cN%W1v0==5@_ zR4JWJ*~9d%@j%Ur}Thot^&9^NRE{6-npJ z0(|96y=fwTkBCch$#&mg=H@zIW@r!TfV z`*>T|x&@mg)OMGSgXAqWELQ){3p|h^$ySO>7~4T_P9}u{@K&Sa3&Up8fY+OQb(n|k z8{S=jJ^#O7>BE-FYfItIUg;ApVbQk51su7wUIHI$mYu?ktmc~o79#zN;7?*LB{uSK z)Ba*ZoOM9@{kV2Q;c>=jQDQSzmviX;KSzOvh7({Zc%V5Nk$R5%45H2TM81~-($5-@H{RiY}1S@ z+*;u4gz`=Wnj)=KJ@l7ZiUiEf3JJYORCFw63FC}#(JZ|dS1MWEn?c7ngNA_d+I#HF zuTjku*oddUp#rKxC25wN!?NT>XaRM-JONfi44Ue+2#YeqOC`e-Z@R_idbLMAV-giW zIHQ538q+AsvI1^pocCRkOxNmr`0wUTS|mK3&>Y&Rn%Cn2Ye+|ux~_%MAN^cmFw;Zk zoihiCq}WHmbW+q~BWp9JhXu|{$&q=8q%!)m>lBuh7D{w}N3AHi{sU)rEY3$-R6p1> z2Tq@G_-XBB%@b@;V?)^w} z7Wkf?mpJxCJxe0DwxVgDHg&A$*Rn$&p_OxxV^V+)2Mgp<{6WS0`gzMABm^!suO_Tx zn3$M806hIw;3~VpilmVhgd!k6}$Eiz9E>gr)Y(Kp@$Fo!`H3ie5PB7! zJ2K(qsE2WQ6LX{a&ikV{=BX9Nunl=2A!npI&^5cF0$Xld)HO}82Kw?%)CC3NHION$ zQ6M75m2H{Vf`2I;jYyJ?newZgd&MawWK8~f+@^s3VZ}29IXA@6@Wsq$nho`IY$45i z6@o}ebjwRL^N8*nCd!53K%(yja1b*#`?X-(99;3eEK68}nZ24-TaxUDnbB!4jZdIr zwzCy*FiDa-mespQ$J2>q*>b-A`4E>9Qf4Rc#1hrZ<&g%93F+-sOCvC%@(JD@?NN+7 z3OsOC%R28t{JqX*{NUBJHcFp8a2`8ue|e6aaM%0s^?o9|tqY!xSQy{7F*hJh*Vo%! zo*e%d=OiXnjchW9K%~XJ*ZBZdHsIQam&sTVf4=+tG2W7hjN#l&@L>)(MZ17>T+gHg zB%?HcvzCedW@@`N954N>t}{h}NrXl5D@S07)goX{8W7`ug7on6%2eir)zQecHC0UK!+?W$ zGZNaMJko_C>0hy&>wR-w{KLh16w#xH-&PnMhE+_Bs{!j1>b>SS8)Oib5A;#Rc|e>) zt2^;kKE5U3x~M$j(-kf55vnMwPoeKA`+}f^tW2;%OtEL|2n%JfedUF&1KCkCGEQ~R z$XGQc8Z0FJHGE@)f)MPDeOu<0j+OUF)32t{Ac(LWQJ% zN0Kw_(}Wg{^mBc8-Obcq8>5 z=^+CN-`y@wfg9QYSy&L&L`D6EHsq#z*+gw&q9s9ThMs{iM-lr|q#W7wcw{$;iRetJk;tKPrsJ?){I7;&)k-CUR zF;>vC4tL3st`3nWa&1JijtIUy-h<(yK2*}+9LSht8$KrPC zQY%juHXV(z17btiYGpx%&=o!;oU8Y(Haquf#0OgDvmKxE!=Z2(_AnwkDFxzuM9@lI z2`%gC94h#54!UJ}tpndd9tQO{>RjU=O1B`09O)C-GjlH5Lr6F1L*QU!z(z#o7O7)C zEuug?40EW>RtUvtsd#C*=W7PGei2BmvMJEP?2I#AaYZ*VJKnp&aJ<+(fKq$GSj6 z3M=p_$K0r0Z*%ID^x3&ZRJY-zaj#r^HExLaXJZ}cXU_aQOh4scaOqZ? zs_jyD2SZw|SRI<~q+MIt){-*qk=WNpj{n+vw<$0qhnU3fh_7l+>f7Bcd)bYFPd-@M zptqu3nLCGn^?E$(YmQ*=rR@7WWL$80DQh1tXc<{Vc9-KUPP;*DNl#hpc}F3}zabJ* z>+36A#86(Q_j8@|2LI$mp)sUsC2T3QX$Fz)16q0c(p=2F8iL~X1zDCv8wBGDmWfd! zw`-96M}+{F?>`HOogKmhKXm5AO#3GZPvk!X0W~vN7_-l;rL_>!hoXTHLt}KJ(y>$K zF%!pU*We#=qf^?owYx2o(sfZ#gxlTB)DZ6ID!W3c??Z||jq0Wp`Y$#0W4WGPv-s73 za&h%Vi@?PF>o6%)HW)~E;c=ux!6(9SzCchXsLTZ#-NSy*^SxPeGg}Yr6g!KfT|RAW zZ>;5daqVKCT-y0d_|;(QumxcnLQjs55VT7otBLg7Dx`jMRei{4fp^eb2&RiKs-tiK zUP$3elN{CLTeijB@p;tj_wkhyLM|Wa7ltJt4syrque+iM@omyP_QAUx_1McGuoL1hJeC|{MRG)EbW zJ?dg01c8?9h)KkfT)E~5YyEP1`V~pLNH^bL$_r`y$B^+fihP~&f|8;pd6lVZ!-vR$ zf_4UJn@bVx*FY#pVrU+^Z8d;VKXs4#bjILAj3wyD(eVd&{`C7wWl>Le?_DeW&rN3k zGIDc5`sStbeZgeWp$+fWucCR68)2!nYPB!lRWF2LeQwl=%T~Y6&BIvu1|lJ@^0tr^CUSw>EUn+MD+q$JvMcD&<#SRM3pHO zhmG4!O6twnfiSi9F0{;Sk?x;rUsKc$SGLD=)9tZyVF@6YQMz>C>yt2lxz8oVuPoQx z+(#TJXOv#G{eVM&nik?RqtjsLh2Wuqac#rxkL0bN=}_^JsEwbc!z1Mk!{ zAV;~Gx01!x1IGw70+PQ+{ye~NvLF2?CjM=cZtCW|nLK^8K?T#5%S{RXs6^`bVT}u) zVQ86}8_dhcT6VU5J~2z}98J&h5D@!*=bq^)EV)i@iB4HF2ToypaXOZp2=*Aj^aEV> zRQzO72;zoBCbgl&ju*Ocq9pGgw^K(Gf(L>d%9z0{lnNjG_^xi?EaFt4oI3Y~kKD{d zTE!eoqv6Y74;XG*Q-?%gpi)cO-MV9BZT3bX1=ZXt8U1W>x3Z0|gE-hMoMuG+>w4;EbL zV2dnlg68P}EoAU!jrpggyOZe&0kQZRvj`tAPB6i0h;7|QC#1pr61eOP0nwJ_j7SB( z#jtMHL$i8sbi!22&6G=!MOlHh*_w={0q^bsdQvVQRQIR5i;~~rn$(*R9s-14Ft`Z* zUfNHZzSo|o`}o~(m{$CotZu3s=9(KM^lD8J*bB6g1PHt1fVudG+*q*Jx4=NskVqrp zsC{Stk@butd)4NX;a$W9V7&=*?I1SwF?&o`&Zr9fdj?#sPrZpac*e=z>B*t55u!r* zORj7FwppDS%3NyG&O;^7DWEt=+Q^3rO2*&od5}IGe)+X0R2f^4I+kN7W6zLRce;TE z29dj_w;e|l5h(5#DP#51MYa_R5&X-jar5;U$Dz_Mrq_I83Rb#o1%#7L+=RIKOU^uj zkBvms1ac#Ejf2KMK@U#ku?`LmI4LCxc6-gJKr>&FPyGrM1-xn%u`kV*3XMBEOM=7A zx3u!1QdtJXsFQm2rncm(W;y+qVC<8DWLYqW=bvam*e`Jd+rMS>({79!q=A$O$@hVR zf*L6(k+g=?M1`yrlj+4#`qD{LWDeUT&4lT=*gCvG+C$tHEgb zdNaweAkksF>aU;GY#eVh>m60FRiu&Z&Q7S@j~nMOYqnL&5Ng=G)k!s#U(ShqVHyxv z$l1EehFqYY&yMf%d%o)qS2beKfau>Tko|=Gu+#HI z^lA=2^$hXcW+^YJuuHtI)BI!|-OU`^q~r)l#tLU+dwj3VcaWYVp4ULDItuf3AK;zJ z1i=ApoI$@Rw@;$$C6{>zBI~C2fb{VpI+I88AV-l(KatPnSixssnq?ixBW6sn3Wh`% z5VST^d?1d!?z3yxXOx5ezt{5{I--u3FB1A4-b9HF8W742GIp4Y#RFr~Eo@2DF26SE zP%?!xH^|EKMqm?iJv~})TI1Y>dYYF&QB?6XKP&l6kS8>Veex`Fq+p7&YZ=7(f?;X{ zY5aF#^SMEDdy>zhWzGeRhu)jQU%KzXm$?sXC#6=qug}EC%S7MzE7)sCY zVR9F>)^@?^hNv8f7h~n-sC{wh0ctO^O1JUu_R4PLn-_E10?bIXJv5G5=THZR+JQWh zi#9FZTwVw^o7fW5h5S0%LEaqIB5vk1Eb$CzE8St!gibg&D+Ti#67JC(p=|BfdrS~$ z4zM7+h0~Hq;u&$UobmgZr7+K8la23@FgLu)r&0HG&Y<=ygQ2-^mN%`+WTcQ)em`yb zZ+|DpX@N)dUY>rj?CJqM5xL;XBbtUf<16CmB*|k>tlGPc(1^tXJUxF^ITW2-e%0i%x<-Cn-?)KlVP2!*>G=#>L=+kv8~EW5SR2>00^a)& z<4TH5>Juo45!KvW65=OnlBK) zMr8UBMJ0;uLh?{@Hu8Fe{fQfiD}vKpE;5~_WuI|c%{apYkyxE=Q|%*=b#`Q*D zvOj)bZNLhb=`LLJz``?(HB!v4xf3C2?X*5GBv0+m-X7Svo)Z~N1D2kAdj4tD}&EF=?ly!(X~tM-ux245#I8| z!k(i?{~Y}{(9kdBo#0j|ZAk&eA8^z6qcSKC;o@;YiM?`0ha94r;@_~xB!YxoFN+Tl zZg(3}GSI?UifVD-MThFby#~jrY_Lp&vOv9G+z)y_EjxvR7AjyfYTGVWp;Rtj}p_8hki(9m#<$XPjg`CQ?Nb)_zsPBd0L+)~7TgkF+LZlTn|pbkvv$hM5OC?Vkh?Hxm7d zOrEZF%f4TmZR7T)T@mzZQ!10z9Ri7K%h>}lrZ8d*zg-@w@AR(RzEdBV$gieSg`vTFS-bSMi!p>g54%e$dIjb>dFLmBlIt2 zS-0s%HY48%1I}rjKOsxWi`&)8CzGyiR-ln&rxFVoZ)6DMsY0l7;oiMNU7Ua%+gXGK znV}C^-F_Tw6mezXqXN7x3yzaKeIuD zt{ppD*x#H`@e-34s`AKUNX_!~;A}`La2Mp8iCX*52+B!2uhK1d(Jh4|{^UVW<@!_! zGGjg$WkGvxh2eSgvqO8p`m#|AD0_YBs&5XmY0&8iU)Y!qqz^eSQN>Tk7NI_@hw4=D zFMQux*&4_nyhJLwJfdW|)rq<_e-0L7I)kAeWyEO=`ZJ0=Ro{W`B0g9(&i`6_{Fnf% zk0&yMttg4ISCU*I&oH^I+_&hkWN1e4F7AC92tQw{x$ox%p!qlw@Ln40g&=~g$ek*R&1(z zliz9qK>2x%{v}d2 zLLMIj=77-?pRy^gWz0f)8HY7&!tjpwwlM(v>1EYMG!Ww@s{jn!0Qox?A~m>%FXbYyxs4wNQ) z*;=}AVyzpNTvf~0EgU`@c2H}SXpGjzMUr~u_CkpoGhK$^($`$0b1}>-R^;nc>|~1a zOQwfWLzn0-;U3vv?z*raz=-7-tD|9byAAQo_7RXc;gf|e43NQ@*iFt9C!Y4~HkOY% z5-a^3`NGldi!N$K#%m|EILwiC15YbGl%7z!;Og^f^zJ+7qb_CIQc!qf%4TqB;HOP&G z${=S%CdN%@F@^%}SsWK>Twswitf#KPXfnm5LFeml?{G$XMXBNLv1nxsC9XLfOViXeg9zwdthP@XXKxu=dY-#S=o;3lhXIq&(pEOW~MhrqTBBz5x!X&mF z>gFhIlpZhJagb&?GkDIcFV&UMV4iLcq5C8F2zyQ-Yc1F}CIivWf)+uHDuKst^2f_` z6%l$!dkMVNV!a1p6DO^pxSd$)^2^K!@U&G}LY(7t8axocgt<4R(en|JaV;)+Fw%RB zlM(Fc>qG?D>Y7cb3BoM7CeGNLK)2{1pQ!7f^~`K(ygoE7#!ll-GZz553s_K z@?yA2AvbVLkxJN6bzm}jW#TlH{BPs&|DLIy)->=qCTt8^9Yw;_R_9GRQgTsHZIUdy z+sjE4uSlu)8f3Coe%3rSXB>(&_YglVjX8Fb!klS53Cc!~{owppc+#j!i6p;Go0%!j z&i|c!+f!gTB2B^I(uslCG_d>{;)mA=i9N7MaKjLt#6}VLxmbRefRIdNEvVs z7`EL`(bo3o35j+@BR%fzJuc@xet$1fY+Ry7z@-<`N<2gMcJ--tkm>aMil#Y9b8Z=h zsX@ia{)}f6%VvZ&@If>DxIZcfDoov|)*;(9*yi4947Y3wbQ+L6;?Pg6C<5ll!O7ZU1B8sD8D_?7POeLwg7=eaJ+|rQ%8|)Z zNX$ZscNL&TN9CGZAj9-|v~$WM)+)wA+)$CFabPuNyV5wc2W_M7S70w&SiN-YX4^MZ z^VaQN&ZJ4KhV?LxNyL67$nsjN&aTUS{41Xt-8fqV#C&GBl#l+8`c<-M<+(nm(ruO~ z2JPsqV(j*FI6JFXs;>j-)>OZJ!(+mX7d{5-{3Sr| znR3gRWwBhK14VJ?eLe*Ou22WH-(AfPNlbx)?Ha${Kv+__RJI|(exB;$DTu37VNpB}-Vf)E zy_)kLX?+2Khc5Ydbr4AkBC|(-zE`LAd8u;|9T~r!*g!{6FW2invI58WtY~E-`blxX zO`%bljgiEBJm&<)0^iaBDv_U}Gl!8usgH1osq1^4CR1E=WmSektV>=`??Q|&#<`p0 z_v@f{wUEXBLN(>NhYU^~u4oSEcS%BVKH(*64Y3;n2ZONSINZ?zK$0cY5MN;-Xlp_*SJCI0%6rhP$~dz9_kTk>!S-oNIukl zqXq6FjlKnCPg>t3#t89B0rdqwdFxpLSr&3`RPF68b@Kv4>Ki^6Z6sqRuD-+ zA%bK7ni}9AbD#NM9L#DWtoYZn5=6 z0Sjg#TKHhX1JQsgns=GU(~X~;BvIrSQ*x!vqgJ~Fu7UFJgl1b>8vo3eFfkV%y6+do7McU?K!<_dA|kCXd;GOCk#zSVJvT}xqP4=9VH z508p^zDrA3kN?O$%8JiwkRp7rTZ})Sa-WL|OFKyF8zrksr(%XS#G>B8wA%9sO&SM2 zq58vmCgq!x0&DN!RAt=3d_ZSf@OnwsrU+eru5%L3<~0Gaey-7BTKTYPa17j0h90gY z?Qews9@IAVyvFanxY@!)s4;${s5c_>{$ZjscV8fi3Y_`lbi0zO?*Aokh|o=VYa=G%hf^5Qw`l)`>!yA1H>MXY9$xbz#M&&t9=kvA@e7-?`^*jxo{db zh!4*JsDiT#DQY6^@X*JYo{_RKW<_mOV*T*O9R{0IE>+IyAuaAt^ODm)sy{PbDSs4b zuUmhZZ%ivbf~~~){>cEYN7VWVB=Q$h#Zh~?7^y-w;Eq zqlU&V)3dW->-L0-U{FxO47IV!;qc)@p8r+pwdW01{Z4*owS)H#E|$fd6!g!Ml9xHN zY=%B?VJxx;d(a8Hf0-g=5h2(;+lrtQCUcSqs~nVk2LI7$(ghLBD=0yB#Dh@|(-6Q0 z?N~kkuoFnbrV4Q2vyV=Nl_LiCWr>UqCR)=pauTH?3F(9!VJ&8ZgB^yL+79fa&J2dL z`lxLR&sP2G+#2Zu9o)7aW~)@>8-_o>`x47gc!xEaKGR9ck=Za%F7gK)#t@?PoV-Za zp&U-8gmIgFCpCu4JZdgB6?kvy6;L|!H2IfS7bAW>kbavIvHn9WM&S3i9P;GL=}(&t zxUxOkxna-`c!a+29N&H6!6g8ih zVnD*UV)#r_{0vn|Q{TSHcA-zsi8ji32op>n`QAz=9qS;p_D1$1kQ~UXYgDtK;b4FR zkciArB#Zj|DlO2r2yEl{q!>OoS9@ZA;jAn0a1}M=B;?FG*Ti@+$hh)uS;vH{n>{(0 z3nhcaBhiCp^vVfXNFetb+V(Z7Lr;e(GtSU^($#j^k**kwnwo^FmupuOKmagR5?Ggk z7DZ~9GPE2l^DanphkQoN}U^!4}H#Pi0nnN3Cnc$gpND?S${E(ssLXYwz8yGfj z#U;oFfXu_Z05ExnaQu#-l66IrzlnIrJ15xG7Uz@el{Gr|J5GjbE&hNBo=2g94=N27 zG&h7bPLvYq{$-pDDVv32`iro`F<-nEYlhEw(NKQ|SZhJz-1TnCRNy;bS+AQ%5ZM)^ z*lS6qSaJy^F*nSart=sjNclT_KS`MmX9*LLv%Ze6oE%NrdSygD^?|c%123C6oif&= zP^fWT-9CcGt+drB!VCjn{?7P7mQ*+1u9lpY4k99lPNc|%)KW}>l84G-kJr?+l?KO@ z1TBRL5;&G4+3~9p)dL8cho*r#MfDs4=zw{|_~bI565Qqp?pE%lzLzH|P!5%GZySJJ zhFcnN3-5IVun1x~8)w}$djN9Blrcb_F-xA!f0Kk>E+sRk%bHV1>$vtAq4aHaQ?ei)>3Cw*}Mv2B))f7%bIVTPgM1spgd5?SgDf@{y70bb5dpus7~R zUMpXx(+L#cI+={1QxT8SBs0`9W4wRon&$8dG|YVZ;(gY0MIo%O;g^V)h&NTASZa-f z46k==Kd`SMTa~)SJqQawEZAj3MT=%@T}RBC=aai_OjhVn!R%2hfxiHrbp~A~dYgRw zhKS5X#ZeF$cQMbx>q&|$Ni={sU=xX9sGseE=4+b$5?Ywn8&u%BNI|8`Px(QqLZ#av zeuyaPeYw(nkVr~D%w}-@WS6NS_m(2HPPG!M`I)ujcgnd!Suf$g;F@;*zb)f+nrb8W zkO1MpRRQ4u7+;d*gi@x*b}C0#r+LjwA(3dK_@adNub`m*bU!RXMt&F?sz8{w52VSH zX;tos$A`w?%JnOfyL*F`*6z}{pzgS!jgZB79PeUDLKjSU(nD#qy=YiTS4E&yDW>0r zhNC6YHLC{Q?@k*h3M(;YRGEPD)vV?OAdjw4MgGgp?%_SbCvZ0uP0QEZ>ag!>J=xF@c!YEwY< z-u7YMs^T*-O0MiC3zdo$f%|EH)apLKg|56YkCE@@G0JF0$IZl}%b7kg8^}V4hMC~O zjoR8~z5K(6%0q7ZTh??KMGeu%;G~I%`Z@9a)Pzh(jWsH3f4+zM)w#4;2D!-TI&VQW zuOQnsCP^f}Hc_IYujE^O~o{D3sHr#7=8@!cI z0i?OWC4q|#OH-P64)m4lwl;(}rJO;k~VM2z^ z&RpMXdpsX{1+@oQ!zQ9p+JaL_GiH!C3xQtoqbUZj)(Qe#I9Q`LEQ*lP@LqkWHkq=<(&1#{aH?U_=1-N#J~#xqrSPIDwa{&2;qt4H5*j zyhQZm6zJZ@CTjUI2=YFFHjOSeDn@ zMUik|2YND6y+Jm}>2DcHjn|t<*cwiM-$d*U0w%)@mfrnO3~a!A#z0LI{KMTclK>LS zb!=DuiGMuj4%Oc|MQhMVPwk8&cv&#MiD}BUO-pt#R!dKEt(Z z*NX{n;XX+;CY!+hy6&v;M*3Rm5bYN34bdlgJ?CrJs40H`T>qFohFcbTJ!Y6(A4k()_@b7`4gvLgc;d*z8zI^d` zwxsn1|0dNda?=~a0i2G#aWj15CRb{MqM(a`t<=56sByI`zXdIjlQ{f%&UNufn-2bo z{PQ(@#%tFJrLN)q-^>414E@dIOu8-H|c^}>HGx_L4)$M=uQrPwJ` z#L)>inqFcXAIXf?!v?c~XR!#eEaA$kv)8`+2(*_vVXlLzR{`B5^YQ7r)`hIL`^`xKz2~si5Z-Bh=A;DakL6Gpn5Wl zGy0unG047Qi%cg~9KE*xl?LkVTn=q^D2`4(eD%kF9=*EOA0nroulo1yK;%yvb|P-N zD{PjM3B!}ksEk){G;$1aGA@ew`3$#6cqBy`Y!P4W$w4VEM)z;+S!sibjXLYbo zWYyl{a4gx@RA3kS+euxnT~8pR`xLkN_%EMzA@EP-2i9%02QhfJN5AxrPAAA&O7>Ph zQW8smSn6<5fioqsp}gjtDdIb8n$1MNa>MsIs1{e64cBEhkF?aZ8U@TExcvkWQKAw^ z^O}X&D1v0s+uWW{lHI$$)>G;jdsb@!>epAJ&&Be}GUf!GH>#?AYAi^oB~Gef`yV{( zy^dBm5%Gx&mgx*?&F354Sw%US4H+i3)c+{j2JZ=jzV9uoKkpmTS_e`k{XPP6;nx5k z$|7p{QzSl~_6}b(uHOt=?;IXzJnVp4EL#z^78a?VWtmZM^9)sliln#(HQgcO}|5ATxWL6$yB{xUVrRD zu^z~$3ABVUsJ|e%B~krpIRu!PlrqQvaccGrx|u8Lz%l(yZ9iiM)U-R3QR~xsHB2f< z>*$^atrwolzO)3$9VeO@G?%)ZS8 z_X)yb>5zla{iL0ElY&EqY?M`^!J>(enTgsVcc=95+m=#p%i*+^U-wkccUoI3I_~pa z_CSgq&Q(Mqj(LH92c-&QYZjTxXJyJiIm}-SkN;!VF(ylTc>jur1g&&Pj-=8)6f9|M zXvwm5F{4Q~05tEb>f28eLD4kRWM{)mixfd8SEMK%wAY2|u_tsKq)wYckzDk$X;Ry{ z8~uF0+zE5*>OKiN{ZNZtb<=1mku^l=kne%yC~Agx@`<6y$0-Gd@+M@nlW8F;&M$HY zA>4sFdB8z0oz*U2(mfi_rVN>2Kr=<|Ub3YWozW^7HJ2`-@*g+_30D@FZ0<}IWT&Im z_TTrtE=TOR>Nqqah-ma2zy~w=(*_6UBw+qdnFOp6CwRiWngL|uw_;Q-GzbJ|1$HD) zmkLf(o5;>cY+VMnoZfGNqustREZr|`uN7_!L2V+oj>6MWx+X@IL*R?g0e_OcK>D@5 z5KBss_@5~K%)SJ0fGy6AAMr0-UeFv<-wNLwzq--6&@2gM=qJvp1*u<7oz7*~HD5RI z9_;nn2B+KZhu3=Kw+`LtTu(ozSp%FKpr9ch3%uw#bhy#7y|ZN|a;|OI6BBlo&8A`q zH#TXT(+aw%Xab(smNZD03qetym1}jnO|2k=kUFhU4+ECJK9={G7Q}il3G$1#ZlT%r z7ultd34)t?5#FJHKezEv&_nj?_K)7dG}kc#A3E^j%0`M-9Y%>L9vj z0iBT>5&2-a2e8z5U9UHMWnI0X>@WiXtE)@|44hZb9{%lRC9JI#5mQKz>tAZy)W8e+ zW)iOcz(?BTQ#Y4UoE!@E}q3UQf(#UD<61~a?{BtP_#6o zV4phP4xe?{Hg$INwFI!E4-yjDQ8rs0&`Z#-a~c#4oL8v5Sa{<+pN!WNFYHSW2=u9L z*hgEzpP0-iyF~TG698FTv%GhL=A0KZH3Zi-Jy{f93O_1tNH*Dn* zV%}d$UP(|_NGl{M9^W_mFu9x@dpXJ}IJM~*)Y`J-a9}nf#OW^U-okq{(}q`zb{j&5 zr^&b)?jrP{Pi=mkS2z4iTWp_^xsxA1pZ=x292%lC^!iGvac0cd1@IN`%`DG&O{?rf@VG9_G4&n^nc>}J6hdrmj>LgcuV%R;QW!FITjCFV%9+F;^OP>~mPY@b zf_Rvg0A3((S?LOaHrOU_pMyxD1r?wrf#1mp}DAi<6-f7yUFQ;El|mZ=BdQRQb4xHPcRt%OpoPu zQtVI6NQZOhdHVW-jQ$`@76t~Ky&O^}Z9(=fK3<}3;jJw0`cMv|x>*rbH@H7{knPEk zjSbOeJH7he@$v`l*Q^~KeyS>_{P{fN^D;4fPHfA1Qc?Cu3)RzQbjDAZa5M1))ZjVn zp=2riarSv;%j;}d#Ig${&|=u4KPyxJ(+$a*q^mRY*tfj3#or$Tv^AV|$y+a zt8rid(=4USlU*34&9c|Upa|q-m*4eooL0!BUhQf2A~Bx&Zw%;)AKr<$qG4`%>oD&J z3}N6gJR1#ZCh7>n@JvbiLMhShI#(1S@Z?i6t_QHWO3iZ6fkG#m<~;=U1s&J!w79qY z8p2^(_Vmg9;WY81wN>A$q&rqe?HqlT-}Z(?C66*--OB-`)6S_>%$PeqG+4Kg;WIBt z!>8qLdX(VdZ1x8R21(Dcz!rvqjPpER6bjWcF}JFUs(0Tzi;Z?PVGK?ZSW94-u`5W1C<3IY=49;gx@c&a&zuv^du$^dOvaa~oC2rjC z^#k0%AuplJcg@xI)~N8WqvG-hEa>uP{cu z7tk!exONxcDAnL{T!V_{ac^(IERfCqBZ#-@;tk zMQw_MBXRMOl^Zuoub2AQG~K$i{@KTh9N=VZ*&Tcsbs*RQy-0_9n0#(D1Rf%esOg_U zp&~5k``l@b=9tS)Z}0q0s7ocHUmUSt%zG$&dMrmF0~k0EoShx{^uthf`b}b;sb?}b za0pgRy-zdnq8!|<@bIl}+0qZuCP`6i=sgqUvN70W^HA_i$v1kwIJLyswr>ft{vql19-=l7tgfH62MIS&JQgV=~=ai}i(dML516P#gurM#AO#P3*nU znJ}9-QhtO|IdXW$`@#dAj`v>zbuAw=ju7EbALUXGX`oIcG@Ctv>;aoCOKS= z{wj=z*>FE!jTYtKAHD+a?POj`u3l&_x~~M%-6|--q#fb#&)zjSJ(p^>Uk?@5X~6wq z0;>|nw}=crW$L-T3fsPvi$uM4CAA$lFh3L2X8{np-ngN3Z?h4eMS1KNez*J4q@Km6 zmd3-bbJUxl;FI35y}DKB2wz=>`igVu3TO9)7bs7L7S98ZiA(KL4an6HAYYuF0iRKw z0Ixtd)HetrVck7%IMIIy1+MzP7xCFfL z>iu~IocDpCU!rgIKwCt&@5TV;@&n{+8>FK0BPA~r<-@HKdW_FbN%#C(SYWMu6>X%B z1k1O11M}NllTCRl7n4qW9U>#CaQA7xP~6$rhWY)GzNSzLnAMs%9pG$oWNRC#L2h;Z zdSj~Xt8oD$e^Y!H@ols4V5ZIhn30iM`XAbdPo8m~tjXx(w<)L_!bH^gT^H9L&EQ??s|4%hpq5HJ6V`ULUfV4+?hP z_#&tjVfLra{Fcz)jWwsfIAhTNMRIlhr#N!onE+u9x&aL#MK(hJQ+6CIyBdC5d#o-E z#RqfH$L)+mG=>bs&tC6)vzu~)Zj7Jx)s~I@7pBk39q%Qo+HZSW;{L~$jqcfN zpQYHzN1C8qbOO=UWh<3K^h4H*oXN@T-w#OrU;ajoDr(BqZk*a8DU7Fnp-D(n^{%wB zS6_6bz3Q<~?7AgF|BL44C`VrFDBe_Yc>$g*-y3$xa5dGS{b2v0Scn#1Bp7#ajS|)} zYsf2K5@;*Bjy{{%&Q(_*i|GtVAk`4#qWK!HAoX-r>^2iaFM1HHyUQI>E zU777pZ}*~0Y~KtncsXcUesF(9<`(hN+#>rIX)6-`Ie{n*GFw5U$&>PhiGE<@QYeb$ zw!i9Q+(qJtwB1j%@Ls-UyWR3(Tn;ScJ8*HmpMW{w-|HDi6!5>>h2A4egY{9gudI}( zAeo|=Z~I66UZ1C`Bk1OqvI0#YTfcM&{FU+#4~ok>BJrQu8ajM|L11^7c>7K*=&RW@2^^K#o=FjF{O5reWE{0 z@+btChDL<(>5q-X2Y@^drpd7j+<~xs6&N&s)pFV_KXqBY%8$5R|ug~c5jRj$_x5eY9!xO#M6*? z|8go3l=TXip|hku;B8!;%mvAn>)Iuk;?N#WDtp-+^(R$Yz?C7!MzcqOoUD9&d`lN- z^a66de_EKbeDBzp&ea1|GSP?_Ail4DP3zFt%MZlG&+ z<9OsQZYMYKWBj>4tH%FFf?1Hg_Cn7+6Cj7%+Q0J6pO;o5*SQ;?IHp7{zK5;FGVt?iUoQ!S#%BEGG;tPU&4@; zRTJ_{wamDRg7DDZr2JQr`CUZ#ED7Zm7^%^-C}oDuPSbu$GGoUXpQmPhw~k7bPp4N7 znMM@xxEifkeP@IJZ5iCkE`jTsZ07d}`;$?Pa}M4U;)lPESQQET8DySy2An_5t0-hv zbBxt_W5{4iSoP&v5?M)(YB~9@+Lt zL5Ykf^b_11cUz7!ZRh)Wd7ZDWu6o($j3RXC*R>uBbMU@P-u^n*lLUW&hmgWq+D(Y$ z_o-s@7M>-7oRcB0t~dVUSy&f|LvLJn2pg@}hr7Vr>Mzuu`+3$o!>}cZT>D^1k~0LW zcz!&p;pbNF3^F8l^2J;YAu2TqlOq^PB*y;m-3hIpI-;mKh=}{X3`Wk?Gp%OIW(aX; z1CR0mIa79n4vFbdB=8{!+pnv$)uCp~{xO%Rv^TBvk9!13f~%*}cD=laF+yVRJ=S93 z3;RG|h&NnRkUov%mv{p5mGkGLVaPBCSa>qG;TApxO zlPL<%5S_mxTDb1Pjp$c~$2=I)Gd}3zO21eJs zuDCy&um%0b{v)^XG^AL_*dTst*3#KHpG=yp-VxdKac{j2c0%t%^3#PJ4Zj7( z;I&Ki*~N$1B#;|8Y|cj+kiF2N&~?p^lezO(Q0Ax*W*jtXbatrDd5{EiXBc$$BfOjL z{i+8#@0}-ScrQJJOSo1zV^OSdHZ{BWq47uxyo>#sXzpC0%^dsrY^+bV_nt7=0*SM~ zpZn>IdmWjtRb6U$0-2&Fy|ZS)SJY9keJ~YS8mBHz5&;7q;&qNgUx-qv^Iju_NXC1W z3ohX;B}(e-VS+huhqoY)$bo0ZLv$BW%buaXi9+>{c&tfeY7(BExQh*ZTJ0)%J~7bG zi)zfq99z1N%A)GYfF9npM?*V}i;IhEW{LzVJRfwzC_WTDY~vE-(Xl?;f@c`mTZpUt zta^QNo;E5s&sh!XZA=(W6e$5`J@@q3vslEx;4sNf5YnrJFV~0tb=QafG<%MFP@uw5 zSN|;6spR73{2RZe`EQ*=;(H~Gs#gKd46tF1Bi~-gPfg_b%Cx?@fw+b3e6nK4i5(>A zP-jd{^N#%W@-aoPxrL>M13IQJXWuyPuULZ8GubcbiEiR5>xT4;rAKkt;uP{jq8 z98NI=sX^9AR(4XZJ9v?d@9`QlV4*L6R&o*w9Mp&`Q*+W$C6#FHR}L@gH=+0T8z1uD zaf@dMsN;DZu29cYLp-X}g`K%kvC#ugGyt&|Z>6T!q+3$83*WO6hsNEK6OIW3C-jl* zIqmg+%rxkgRG4dbKg`=N%+>?s+xs9{;0^{!x%}m80yl1!)z@Z1Bp!lxWafWn zyX^6UgxujbhDvG+07o&cQafNQ>N%X=>oIp@SE1=J2}d=}&te4w7JOWd)xtZ}Akh<> z_7geu0D<@Bfz93Zd1{2^vpehnY={NM^xXYYrk&)fqxNI-V=dzK+Oy`z5~mJuw%7MD zvoCY>XS8`|8vG%!q!%Ic)MRdZ)u|y^=iyI#4ReXx7MFLofNz^;0ZDdHT5Z*qQ|oD8 z6zM`IHEf68BX`kEjNGJFZ{Z{`ABy!V^K(5(bDZbhXc);$!`#Duy04m>cB#48QT<<*c^#zWUQ@&qv}rPx_|=oyPnV*);h}> z>$fPZGLY{Pqj7oFeRF(G{juGw+s4QE_t=q6^L%wp_G>H~L z^_?gk0k^&$Gv#WqFj#0g&OtInuQXq|JvC0Eob0Lb>YKv)q&q|B3yserpM!a{Bz{0N zLT!7GIj2536g^a^iHbp70BW#SC$k$RruhC>1QKQW=xv>=YXl2dD9HOZCP^C5^8n4y zxnkPxQw)7LQMlQ1dI58FHp=-dJ_#n?FM4jfRf*FGge5H%{uychrqQzIm^N70r*U4LfIF zOn?{GlO^Nkr3_%zl&84L5giSI(K+=@?a}Q|Pe3Teb_sG>AEud`gMsNDx^%M~SJ44# zNN&{@Hb;oi%gFP&!6ECor>Q?#@k5%Kb;WuvQ6TD&Fc=wTxPAsqD^agv`%w6uJx$}F z%ibFZnX54Z{jo1(XHR}EcD{M1ZA3?6^?>Zq#aab)!1h3%u9fzJ#=`juxhHLcI3(#mO4PJw5zqB=phTj*Ny~F3 z#5P*OK3@Y%2R@oI@eVZO(6B(9%xHF$b1)ne58IdP z6c3zw`4U1KujKmNu6W_T$B>ffu|@Qv$~+F_Y$b6Y0V|m699O;Knx0Z74;KLgZREd5 zDG??p`=l=f*r-&SI~0KwPm_puz=7)~_q|m1Iu-+WBYxfs#IGH&WB8g7I#ysa)^tI| zX|$~~_uFntw(uVmcm*Ckni9JZY`yWwsV7qjs2O<@J93JZ>uok-<&FW(5e;PdD&F@` zzKvMEBRM}@Bj<=(M1Mttu@@0@_O&Mqci#-S>=efjDXLt*TNu;AT=&rL=VynSN}jB; zzS&n!Q}aA?dQZ+89T*bGL>g8xV^xaLHuh$Mu$lz2I5^5Xer1ke4h0r;uY5^He2wG! z9j{7f9T){pJC#%fv!R0+!K&LbSn3U9k6vje6R(8eMKN|#PL_xm#7kF)<%PDddTQkB zzjPHt7e40s5WYUc9T#j^`K+azX}Enkqo z&z!nGCe2pW&3pHd*nK&O?5pliSqJ3U!5*s?el>sH|)*Spy&D~fGN)D zG_}p5U`N?mHG6u?H8U+^Ybz$Jrx)t@qIBwI)v>fTwNr&3UlGBHg5@s_G5LoF)8od^ z4#6>MeH0}jlNH3y7iwo?sc!GKyW>$pvRE2D)QQYhd`JA3ds+Dky}8jClL-*m_lZ`p z0lm-fqLWAwGnt$e+pDnkbf-{^J->9&J+ai+x0=XU7&H@kOG=l^0R`5;O^9IyhK* zVtnp6_pOXY)|1C%iTime*TS!{zLnn`@Q=XOuAPFukV;;6)&3r2#_3VHuR~RpvSLJW zykT@^Lbrxkk9DP@Gp=UqT23j|6f(>jYW5Kbb2rzy7!xWnIUtRnMOs194nvH!PU*1h zoLu?-n96n3$qA|NG5NC5{_IGmUjwvhO(>s3&5%u7>c`$$yynffQ{+z1n_qOwgc4~}>aELg`&L}6eN=H?6==_+ z93hRb*CjG_#2sL92eM4Jv^|OA+>)z`2(7EvEKlfm-KqR!`XuLxzf~oT<=MY<+UrY~ z&N%Ixafb^mEDGhO6f3JZN75HolX6ggx0))Ns@e~nx2^f)a6@u*%89u5(iwYyrTT7p zvH03bm--k4inEVGByF0ZH=H$Us~P3RX!Jo}!x za#AQNyx#Vkv;UF!`P8cJjN#Ry^-xhWd6k}n-8@_V0TD#WU_?UPGu*WKt~`xMT6}ei zuy{{=k#^0bK~1o6po%{NLHE(<5!H&~oOk#8LCf>)1;6UmTEBNY=_diWuHZ(E<7$^M z!nR=8*0{2!5!u(dx7VojLN49@C98s_F*=1p+BJ1uruvFW5b)#!5u%EKW#+p^?Ua~8D zc2j@Mrx&^0ZADE_0|KO2@r_Rk2Uc|8!Lsi()b4rienL<_?`(CZZFqU<*z~}7HbI5w(@IDQ` z1(;iBTul?u_G5B#HdwDd^)gxZ3&JcHWa^a{lD~zZDiDat59(9&{08!cVXB=&E-*!x z$w2z9f^whLvN97HH^W(-$8NLUD;+SBXBR->bIKw68e=Q9^xRZV&_J2XU)h8CZj8n| z@C%6@H6~~nJXpBw)K+(L*mu7PCX9%9fB&{^iMH{}o!o_9I6+WFLK*DRVz$D5Nw>tV zsMfBUULi_xG)KoTtuuJ7Y~{nB9F05CwI4Q3QQdr*9GOU_A&0AIU3<1BjiUz7gVRnR zMRkX(-DDi}@8q19a2?T1HJveDq*P8SzJ7sE=smo|G#lUDKo{Q<{Fs*~vwoUzzr7%& z+T}Q#a5{@3fWg%mjFka_WzDX^peTAxTN-{r63ZI4@qCYw zbk;THbMmM$R%$n&35jK_jFJM5a4BY*${1pEo1IX`awG~}?Qar&%aSkT-A5~`TPeMA z=p$?Pat4^N|L*yyljlbSVMS8>Wy&qd-#HT1jtWw-$1b6|_hsc1!CG$4sL-+!Y_gE8eKz^Bu`>2MnZ z@uPF^yk|Uz$ZQXfN48OsTb?k=25(o?msh{F&h8=bx1_L2Mds>ja+DlR?z~txUcb4C zy@ZfYq2m3XTdT$^2#X)a2^S}6SOxIgtKJ7pnA!G9?;RM~XRlNc&uP}!iFGqm_7L~# z(llW){1lW`{{EOXvYx8Or<8C_A%o$bk~m_23Ov7D?T$>UI~!bLo7D4*ePjBlG(=)H#&R`*;CA> zQgl#?&4CVSt!~iMCfQg$45fN>>Y}O!T147FsLb}qPI@LMRE&QYQ#mQ5va>NaB60o8 zMrFm)a}X<~MuuWR)G@g-SZ)e=8$p%}Y*F1gj>E zi40FRUfFbKLqUowyFh~@eQ~~+$FGuZ7x<{tp%JyIjTkQ=Gyj7uKGXJ?6{Z?5Tu)TN zvW0P^3t!-{8h!FTy*dqHlUdU2p!`LO4ZNN)m8anSGqc^E^ib3!JFU&{BH>+;t!3Hs zAg|$h^*k;rztxV}{#UlZlM9t*2TfJH*@24h&n9hOtRL4z@cJM5nfYwNMq zIZYlboI`vpJ;DnPq9y5HN@_mP5x!n&JR(w?uatbpH;GGc<@jyJ*sgUcevP%K8Bi8b z_jLc6uxMs$LHflTXugB)YilQ?C>8)>738a$-J%e0?6~^%A%VA2FsT#GjCp*R1Gu@s zv-8(lE<1hNXXOl`QQrEL7@u8^P&VfBFGFmC_4jlh{8T7?>7L-^wytpRe&KTi{@Kt* zmK}V=38Tq<;`J=5E{nb(i`>WS9cEZJhs54x$*$sOeThu;xC#;}HH2+4w<$6-+8Ca6 zGfuijCVuxcWprL!X|>Q%{~J>(k5X7uGgEQtU+hxRN!}w!76SSlV{>8SY@#P@71d}r zQ?X`tWIX754+WCsRxDgFi+&j9)Xv3lj0IzTG^M9+)MvotUo1T4#otzC^j3NK3;8M> z&$mz58j$bv19vQw)=(s!wh_oS>^5& zPR;J5(b89Z;alYwx@z1jb(E zMb=@xRzTQzFQG7I1gaxT@+3!r4wqM~@Xd|x8Ei)-Nv8AcgzA)LN7DHjM%WdB?C2Kr zM$NWo{J2*cnnUu1k0tG$^#&bPm~ssj$AM@ASc9I1)qLH7_E+8I9M4ymSQCBGH{@1o zc%$~-N20osKNA95Yok8xkw}t?#`oVoReMl8W#*Aw6H&x4EE7W^Vd`zbmI8{jMFFflEsd)7mM{wPpOJoXnm$9EarV&){^cxdit+0q^^NJ2PC z&Pk&zmA*6x@Yf|wpwRVog``wXkVZUFDeaG4D;aP*jzzxegE`Uwe5RWw)6YBowkiuk*V&zeiy_Xuz8R~naR|7+3Pf-fl-|oPp^LAWnYLFX^a7 zTUrq=3o=^|xPs1g+!C$9VjL`b>gLj=v>j?)W@NqdzPS&wyr(X}|0&$5u&LP7H56z! z6@5e=jI;0MeM3_BhPsnj(tHUSwLxPa$5_eRsx%6HGvC{(NRjy-IR4^1Y1mKZw2`p- z@HZXv+rCnb6lGY4bnGm!E`h8dMIFX1ff?4BUYT?CIVQo=!%b3DycTkJ=&yT?lPmWT ziSbSfk^a#=fCV?11b!h4-&sq>Qzfk|BQ^Nc-V}Zj9o&_9 zD9=T7s9%W||0xT(n@b^|GwIBCpAk^;SljkQa9l&di~5mnTM!7h z+Yp?*3=2pn*Z~?3QIWl*=%+^CufYKq+~CxczvkZFVp*hA4s52~XqkJ6XLcPklKJpw zLqFrjvhbvhG^Zi8pb2Xk1tVyIdCXW_`^KCrFRVSk`V)>D{-ZDuc__C3UKG6J@Il4- ziaSd^D5c zKRrgXsB73Gv%Q4pIj???x={UYpa_IW6zVF&DWkh+;A+; z0iMY(Z5Qpko1ka4_R71B9|Yu?$>UZQ5fxWYDeg_1xx0z()x9ced~P}SCMtSHv%i&b z#0hUOW_k9Ycx@|vku&0V+leoUXib+|{(engBH@Bu-%@Qe0{B*}YMX1E2frckGaH!N z$mP9^++0b8XmA*9v^6=7HrZCmS$ryK@CS7FyU3aa5t#wzu)CT|B_}|ZzZt~!fYDtvU z)HU+3$&uiIUyP;p^Mbb<&4aW`MYAI4T_b}@0faOz6NR>3Cop^#EH|zv^D4@H>ik=c`YkRI;18tIDXxoSYON?!?cj*~*ytPJHt za7c*W@65E;YV;e+99z2G%9OD;qC(c8`9vEvz(35g#s6{0C7w;?X?iUk<$W&QC5f{( zvW5k>xj|eb5@y@YlXPsoHJ0zN*yPFTD6)wg9s&CzY(qMs{NIeBvWCO*SL3dr9W@>v z{wZ*8Y>I#g>AFlFk>($tp2%#CeM7%x?6Dm5ebISX9BN;`o>XstxyCN(2*ORS$DKyG zH+A^JMPa>zI@W6uye*qiLMw!|8)>$spNtY~eugb6@iwo8cYes1bvk-cwEr>7)AHMs zUbmGLvnWS~vsQwNQCIO&sOw!QzXk#J0t#u^8V9lKtvnxWVtcdAU`=kNgO_aR6efx} zTaDjCw4nF)eo?9oAhv^_85;cazW^ZED}eiwKLOiw^@o_VZpns+{FZAQB?3rl$XO72 zJR7riNAmJh?P`>$wtn%3k4A!0uu>BEhpikee^YoZMkd(RwJ(9AUNOw{p$_BUT3MI@ zK(6MYEvMBiN`?h9&^38qT3c%zf|z}1A@(~)8@{&ovS;u59Rx7kra3tpmfRo0d4cI6 zC&efXwvRcC2G*TS831Gptrcc0Pp)zT?l=Lw#?v#{dU20U=X>L9vm+2&2~rw-k3O_edUvskZ^;-=z@x}8b47xr$N#tCHiG7#En%K{xuPvt&a>nfYA zeQGX^sLP7g|0Q6@BJ$mp^LpxFAH^#r&X&VSz-`kQ&z*-KGh)RQ#zhPo@6z{?->k}5 zUWwp!6P*7PFz$DAygknD0WQ;9^Qf-W&k3fEzoRY{tpz;=NZ7m)H85 zq8+{S;nyuWc{__aW|s&S!YbD5i1f#u?XuKHYf%yxR<&)nXl|bz)k3aHyf3yu3-k5P zN^5BK;3o&IH^^9A=RM(>VLnOulj*QaOF{3`-6zS;kn=%6@agrj3XWbY=L>KQg#l8> z8b>AR#Feu8U6_otyx8|cHO7}E_FmYSnAIa{=H>&5C@DG>#0rFf)d+!xeMi_+A}j&` zbbbVOm^%5a+&_4q$1o*>AED-Q&_*DNsD2kw1a%#?wug?vR5=5XxqbXocKjKZDLPtV zIV-z^o1z7#Lod*2#bK>;o8J5D51%t!Y)XQhG;9-Re?yPT=w?UwhQOa*AOtG*XZ*CBi_@>EU?fk=qglUxR~A2=Hj6i! z1YV41H~cQ8Y1@j((jHgSdt0RUEBH829JT(2;$tgb zaiLUN<+9(fQ|jzK?-+tc!?0OTWb$>-aBWK7xNn7%>)q^0Q#JD%_Fgm_ zOg@{e8UN9e>{1vUU~;vs6V@4?a0J90WyfO`5c!O(m(jq>)kikBWwqBD%H;CNIoKih zz|C?q%B!l z3ohd-s@_sP?{vVoIN?f4r>{|rmY zPeB8*`&icSWX{sM$HK{$gCDjMFRG2H9j_XCAM|zt8bgn8HL6Ym*Oc$aH)moa2~xuu zDU3p|LZd>R>`3`-jG!O)y_Y0Gyncrftl-B}8U)QoZ(MuvK7`P@$rnz4>L6XPDJn?{ z%zr%;zuIs(!R|JTmQ|F!+Bkt63RjM!i=zw&My+hNR1JxDJLY#m>?U4bV0I}Dv3`r}jUA563@VBO5BbMWFAJ+OiHnpf zUVIyBI<0o$(H7ICSDDWD8F5O`H!6Y3&Sz0EF*hSVtS9P3RgwkmjGn4n;O6>n;(xcA zjCasfvckJ2J`wgxU29*`c`Na8=e!_fF2cTmCxtL*t_HI6RJ=YTlr=I^!!2 zP>qUXZsbD?I;I_x<{8j^9R<nD*}jgmfFY94cBe!dEO0WBw0k6lN#;W zQ$K}#vo~NkP{N%Vo~%FHerp$;chWhQmS&x@NGUq@YgDv%8ys|xRG1RFwcyvgg$`*fz8R9 z^2gAiYKsC>w~Q0lp(A4%t>~Icd)$LQs?0ecGmXG3&yCbxFN2c9NlO!EVZAp!S8yN( zE9!7WrpdJBzP4XWiz>+BtCS14l%-bamRY{DbL8?gG>6uGJfs_cC+dJ~w^zl9(}$ zX(SiF3fs>VLc0)k<4MWAvN=)j*33#Txmh} zT9DNLWA44*nhN@TQAH3C5EZ32ML~X-u$WVv%h~!&#`y~C&kE^l8TKucOGyG~>6*3<6l+!bnA5GQMH5dyle3l-p6Eg{_ zTNs=&M0@L$XPc~;&ie4<(Vs*faX3h-)up1|GK-7YsUys)tw4pATwvaOg|`n$0I@2x zx<7S#el7tMf8;3glV&)dB1_71)-Y{PSukM#>H$x(DfxqnWSH|C?|QTUbWc1vo7e9j8gqP4>xwiCmSRjA z)vpgD0ivt@7BB~MWR-rhO7T_XXA-?LZkZ0517&!f^%>t!H8ND_!=R-sRKnkso#^CjGa`XOQ+*yl}Kr**u(&^KbMtIey% z?g2C4BHp1qG9b19xZS{~Xl*dg7g?*gKi5brf{2fsFe!AXluT*yJz<)X!YoUg&6oP~rx!YN4i`K} zxij?SfyEhNGOZM6GH_c)k3N-zQ&V)d*wFEP+0;WMhQL=lmVdWbOCD!^l$_rv-}dvh zC#JNdfNe@=zuoqtTbPtt13+DvDGeQAp+J8DV@}n7a`nfYV>P?8E;*@m+K07j3v?mg zuXbz}s39|*UY5`Vcq1w_hMuKOvIxE_3TFAYeanyPtcK~e1Kai8{*rF%jsu|NmGN&& zS^Ang-1n^CyxLmDY*^cf<@(@}3VJ)qY0l5+hR)-7BDMm$zIRe?nEGZuhPl*tIsE!< zvf~?iZJPQVO~3~D*({UAH0RXcoZD|^tG`)HF}TiTqWiXsa`p-}GLoY=0NmE@&ei?{ z$BH)b_y%AjVEfLykj1Ob{B<&gF8gS@SjpO(xUWsRJr-^qWc`XSjXUfqv+s6zZVag` zndBbLH7E~Sa$kFxQ_!c~BwM7hg>BYG%RJ-`+e!SJ!kSXxIz2U~$#tHM&PbG;`(>;~ zv2MK$)Hl7{EU$M^_PkAz0?mE=MwQ>3Qn2v+@Z81?0FtaEDJ>C3bUZ7)TWg{Mic89w9SkZyB-)ss@ZVMD2==PJ zcD|kh&Nfo|@c~_1E=CBboB3Ikov!{_?D2Hhn*^geZvd;Kp5h?altI^aJ*K7 zvzy8*dSA@zQ&&InxxE*!YotYuZI<;6Kzx;F)+&0;vDdsSz@k(R|1Q(CX`uQLkctG<)AI+7FOBt2(eG%=x_oSFCi2r2q&S&!PAPAU4>Lh_Fx1{$eBX zGC}E0!mQCXh73cSFxzQszUiKr_soS@ciSv)gQK;PTlx;n1}ZYeHA}$_kpD<|7`IiF zl4D9qj~{PZdw-)WRCltmKLEC^JS?E(!0%wp1Iz ztJd(_#uT_>_j`IkDaTYI6FR+C`tZJx0cGEudXDPiJwr1c@sg(?tb>1W?L$qLGu3Go zSdZs)<3|7wc8=gqwQkJ*TTwG2t?0@+7lav3;C2nbRwnb!myFoN7jh5lDy&HvC##Hc z&hn@=@HZ@YY(=yAAC*u0Fpr)3$?CRlVkJiMDwi&EwWN3*;~jwDuSkbo9Z(6s=(w0M zy0PUk2x!u5XLW?ZLMeY>vk=YT9Tf1!0D>HzpT>OQ@jG92n-+G33s{qF4C^#+#5xT8 zXSiFSjr#vtc)u-?*7C;6CrR?BWe8g-xfmFiI#+em_w!09roEmr9-cV9XJ51wEGYW$ z`+aT0e*oEAvt>p;$xUVa`XHaG2!N~iDEn9SR z@~en#PMc%Ddjkj1ojjTVBA_)CSW{@Jy1T#c>koWq>Hj$*J5lLjW00RETO(el=a{vA z{`zu;*X`F;!$fM2rB;M$44+h=e@le5Wr;3|y>m+OeVrR&1%TCTp&Glc70>mzu)=H(G3WH6)w9R^`8Y z!~Qqe>~$NcOw7HinV$R)9QoV5e;)sj($4=A&u;z<40ZwOS?YWL17ZGs187K-7g^%} zSHl5lviVDZhGNKag5CHpss|t%{tM-|BLi;Ze_5^m-yi`w_XB@;&hU?1dBvNN`#*2q0Gr#t>% z9D@VT2i6KEHnRIyLg;V(I?5&cZGY!~UR~qg)hA}(mjWz~A)S$9SyFSrB{iTbu#>&I zrf$WA0L+d**+>>Ozm16${u~q#qS-weOgjW5P?NZ8b|Rz)D8x zkfhhDj{IQ;=$IwR`bp{#7vA60Ocn=mk-}T=s@8-wi=}04M}EFZ$^B3JJ5UT|_!^1L zCbWiqdNy!Li*%B(7Sh3*Z%+TZ^2?I`JDe5n_jFT@2Y0pg9{Ohikuhn2YXCX zTw%}GN#j+~KY@_A>qXhlF zRYb@TX>M^r$K{`z`vPIh+o+DyVf$Nh;_?~PSvpB3ETZkDNR zX|_s{Y8DeLlQL_-uO;4|yb1#-dxm)F_wLjE|L#{2SZ1n0j55<=c9s}rhYUxbhtg## zC2Os9g|_Ii;&S6)r>}DWF>q6Ajn#KTBXT}3^q7R?b_lxOv=@I|biplr?(9h7sM+fy z*Ndz*pLb7B>Ts34nw4^9CBe2)LhiG-wpz^l7Fln;+jqFbr2AcpG!8Y4G8;8`hbyb> zcfm{%qQNHd-bc0+fHUD&2KCsYdmJHB6}-1 zLjL+!SyeSu;;@c&7{@yt8b;W$Pj5Kx(q=&|dUq&XS`TkG1mTWe_MDuY@E~d_$frs88b(6y^6B7*o(vkRQmMMj|j8`>+R$?{mh7GS4?9M#{yZe;&AYE7) zHgy=0j3m;$9>4%&>n+>p?%rO!Rlobj>dg#KS#y=>=_UyJjPgcV*D2>a4&Ldnn>ek< zZ*myt_c?z)Jg>-)6O@1j z_9eUzDbURKde4~eC4PD?c zFaKh&(=}o3&71*y%Afp#zQ`=Ep$IYywSR)?IU>%uwBUuV+98Ylx-b6*D8YDB=T1tP zL|Or}pH7tE(=aiqUtJB(6&2-sMp6@%XQSQawz$IQq9Dy(hd2$^R(yP#>lZG})IAyK zk-I>X%PQ#npR)<^E+Kakng6s8W0@BGY5e+ih3wYj+U#djD|c6;g|6>XY=?bxd2)|| zC;qT(!F3~BdlRzgd-z^(8->+1B5 zE|zp=-Ne;W(KVEb`k%UekmPnq7K=>m7j-DKZ-Z3#9{4M&&wb5rd;iBVw2_U_`}eC7 zj3r_OlfjMHEf4Cup`sonv%j(nV~@3FE;l~k)M~+W@xOSjfq2}V?aCam4 za;r|5q?#y5(3abN9b>fGK^sgf^2%A5aEWo^O!^`8(OYTNY6>Ya!Q3HQVML>&{t#FfL^{uB?}tb+cO!3~Z!ImdZY^{oubH7C41 zm~Rjq3{_Q5>-fx){_i_^XwDY1r`Sz_(|%Z3=m@MYRQE7*WFUr#b&;@WcYO zMC;(-;C?YjiPDv#*#5QB2mGewy=B~SZMe7xDx0GlkIT^Jw)Fy#zC(wR#VwDf+WIlf zE^BJdHyl03g?oC~FvlR#>g#CV!I|n-t6Ls0QzX*Drg*ONa8BV$Yd9=+c4<`-hL4rm zQgoPk+Cy}Px;tURLEXKUP-5mml3;j*XF%4v{mD?PVX2o!`qBM%dqkI^m&tCr4fFJz zPPx%;GX$OB%a)O;&91jnz+VWbOBg*`uf|~}^wM0s=cAP$w;12_d8*#b21Av4z1QUlXPhr)j9qTG)ndl=xx9z09Q4KBgWf^BAi3_(zeB_v z)l-)NtqOyfaD#Z%w)fo6T&2N-=EofB&s8ON@?v@i!tDq`f6es28py}u8$VV+W@2B{ z*sp{}xHdN3qFqfrzpt^dR!#Q`{2)Rrn9aoOb#?P~(PY2m_YjqpKQI0>Ccz)U+c?7s zF8h7L%{!{!)+GV|ktC3U;flK4w-9Qr%OTKH%R zc8$pU;7#}AS3_RL$@ICq$4y}G3>wf?ZID~sk3B-JLd~#!_Yv2^H3=@Y)D;;=9h|!# z8(8Xvq$nPL8TA!cp$0K(u(km)*>jj)XXBcUP3(Qmno3;WDQ`uO!-`XB6iC}~35Ud{ ziP7kBc;UCKZcwZ%;=94BTpQE>R(T`7@am9w5lEnDlko7=f}PQf8~p) zuS{yF-r$wuALB?}dFvMQOmdqtfn)tmP%6dtBbp2kMR)hb+4IT7?r#585gp&_z2Bvi zWEv$ax(no2X-i3js8~Q&w(X7GmNF7Y>nomYF^Fr_NfCo8?4}F9X=eWHN+x_cb6_Fi zS`4by;FbBtb%^BvUo;bjjkAwWZ66oFTb@lT^a&@JKX`{q%p}hoC6VwVSc6BJN(j8Paz{N zobXAq!1lt`Djpm$m~49)Ek*PZ=#n68wa$4+MjD!VvaUp7&F;!Z`eXZ2}qgLyO$^tsWLq#MbVQ)?mk6Fz3tf}d#128^|9M?g9A~VU~Vpj zy$(y7dKPPS6J>jLHe+z9SzkOvlu+vNQ+RXvvu%Z5x2f55gUA{B6=z`f-^R9p*|fI) z>d7irkEeLoewmuS!=6F+)$BGU1lG7%e`L^V{m@kF*NE11h2iMV_aN=fA!B6?$2+}z zPR7Hu?)LtJIp^Z`8f2vv+{L{s6VfXLI|T;?V{cf5%Yk=hR^$7LB-{M+QWuukbumyP z($~#iKJsxHD7r2#daCEAU!>Y#a(znm+sk)%M{4@EFVkxr_5W61KGh_|J)ofjDKI;L5TlX}?em z<|G~3x7Tt6c09Up*x&QZYUUGQ! zsPZs2G20LZ0ENGsP?wWA4Z>nz{+WGwmXUfFo1xLd@T2v0)p|%t@1+yY37%a> z-tnzx-R(J?Perq4xE@wj)W(mmwtmXc`Zl&%1gyZ8RWa}J4)H1nGc+|*yH+rPan#)4 zIQ?Y%Q6KF(Hc^Se-_L}O=lkKX9Ys+xW%CP-2{lJUV68&2lIqb@4brw=Yk6_0xalDz z73n$SE;<4iOK&sqcG{GP;-*m6)2(bsi+AK$9*NFU+f9oB5H86}U6OZq&Enz-DH*KH zGHMLUk$Tsuinc9DG6_9uyHw8k%ZS00>6{JfTn9|aEihOGSbhTe2(KxJ28xWA;)H{i4oS>g>eUUKPfm5tEI=Ut-x)8@m8bNti zt%i5TAVnicvCSc{^7yT3ayEa3Su%2tDwv=5SLcLh$>=~KZCtw?tZ-HcRghgty+`(h_VOkBD6vW_`U)?~-pHS0^=-duMdK z6h2Exv}TuOWKIXwSK?lz&ZfTk)%oIW%=FU1@o$*MLJgM2$G(rnf}L*jzT`d9ztTeR zj}NTvA6%6Y+>of+j&U8ZPMk^0ED6%)84cI!toEKXfSN>N%NC#t>7M!8IKuQa{0M6k zsAGz^=U1_+JGe^LY>0ft+}BCEuE+PD``L|V9< zA8$9U@3&vL0In^=MnD$xREOE9ux{F9(79pJr{A_9$)I{m2VYX&Jb(un6K^60MASpJ z<{2o#>N!8K(iY{4OGUCaL`s&ZSNdTbR~-v7cDeq68_Cn@KY}XvW@8tF7S(%Eae`l% zn{y3S+tfo>&dY28tu&C4`^={E;5Z*?PV8oe}z_$Ym2GeV_FH?$4O88FHv zu6|G^xNyX1Y!QCILC{$M@#dHNP`rye!i;)Ut8mMGOm_IhrSL zc=Up-%ih4C-7J&--P{EW)6yJff@B?GcdcG#*rR>GG<_;buKrBFu?rM;koOo|SPKCL zZ+&p2;O1`1&)N>K-CUhWt_Rkr^-DLZb+vX!_UagVRcjlt15Z?r$k?KnR)s7}F0tLs zI<+E6QT!#P+DzSPK;37YW`i4Qs9zs^GtU9c@?$$J{EIt}H-QZcbcS)ZY1*B&{@CZV zqWyg5C&XL=lXa+PQDp8)YW(}aRQNqW?P%4ArCeV=0jFD5vqgQbg;t|nnd z`w8Bn-YEhMmB0Vfu*x9zGE(3?3lQS*0;YO)>AQkyR^G779m8{uTo0bpq-=Br)*O;u zQ+>29(DGN|^ta;ag*Ckryng`Vs`Ez}fTa-eGp4ooo%B&zAu9Ly-|{p&rlg z1|y{jKfcu8Y(HCx+?#W(&`5sM^@V)*c&DTy`_UbN_Oo{qP{QR)sr$R3g90r>Wz_>u z_R3s?9V1pc2DYXBh8-=e!nw{(#t*{;gYNfxF8ObAt44c}75AH#-&~nb`Ox1MFi-_J z=Nc*Lcpi$FU6KNN&S<(=x!PTUs1WXB<~#uNsCGk6PQvYOcm% zGo&E9vDmS5M`Bnq`-Q*9B|!9f(D7>O)?1G3Bnzgnxtthz^`B=w5z0InTJH`TE?r1{`s$w$^3@H%~Y*s4(fjSAN3O0f_rbnSl2 zZh6bxV(_W+-ZBv~P*y*u-++Q;obk5qpX@H5`sx*xXWx>>4ACu``z{&)g&IDWb~?Md z2gOOAa3fJ9)f)0uW`O;igHP~dR6~OfK;zjq@>R4>qT$5*{Tkn<63(Zhm(Oh=Cb8Z5 z!S(Iwnb6TdAMZ#8(Ao%yXwq=LGciu`CW^>GY#f)vt;YR4 zf~6CrNH9PdE11KWaS4R<9 z(+PzpW?SGmATWi&`izXhY?pZY5K+PbHGqYm55|JGf!?k}e53~CDSEPlmxu7P6^sQ8 zIz4y*PfkMR1_#@Ys(h|FD`nNvo0s_n3;>*c`cVgr*=0@mP$NmWKnt(5YQ;K;yAxAD zav$uMdaJJW2($3zgx7S_4jY}ET_zZ4wtZH!K8FRUHy=6CtYyK3tNvV%HAGRjf;1dj z6IFO5Fq_0hHqW8>y%2KxCO}KO(1xN3Yy?EQ{5lE?kc;8|=p8^GO9rIWAo?{@rX1m( zb!yVJSe-Bl~oA^Ge0*<|#ozNhF}t*8l|3 z0!OZaMiVLJL6^DwyE!p)%X0YR2^HQV%`B7pngC1};DY;GyY@meP_eJ9sqpL%r#6VB zpt`trX}tD8<|cfUGv#i!#D2Pn$fb|(1zLip|6sQQ+850MYJVwSYY`J$+E{9R$>Xcm zzx~5QiigPso8Mm1VQ3l5s1lRow{mC!1EX!u+C$N>6GdYgS03K=lBM4uoH3J&GXZ)s zLb_C*K1Y1anV9UKO?u-qC&`T;6kqNp>RgtP@jjiPd+~|4Q}1O@FWuEwZf|vO-*S8Y z>AT1c3bJ^Wc;oj2oHavMrduH3-~*Zdky&6Zm=7G4b50oD%uLEE1->4_v$EPwmcHkH za2*9+vjr{cqL;*Z+J-reR-|Ib0ctW z#Dhjl!;gPwVhX9is}j%Y+_cMYq)riZm#47gNBZJZ%Ju^()FnB@AHL>924rQylhJz_ z-I;JblZ8N=NCk42tHYMVTEnwux@{Og6nsc){d(#IogG(nn*pOx3djl>TqWdiDZQ6AH1G; z{|MuEwDy=C0c91E?Y)lrWY^XDIaD!hm4<@GNS+I=eCoGEck>nADr4xps%%M`hMOAE zpVDCd6cl&Dbv6F$JeYOaI>Fbw%dNltKml<;Jg;b5l)qrTvy+!CIwCn{b=DvU%h?eQ z0?h=cva>dPZ6gSpjT}lMTi*J)#COQ-&25D0YrY75rUB7t4Eij}$T%9$(0Q zKj9$%N7oEtO?Sn6gQ8o^X=AG>&0Gy(!JlbI6UoajTk74(6myr@L3{I%<<=;!dzWCj z+Hv&A)-$RF4#@Ijp{#FP23Bdlx)brfVJ&D{dA|x%2u^rUFK2Ga^#}h(&Ci_S8;>m0 zp~lNJ1RTzB5kb4W53*^@go;DB8b+i8#&WzQAhH4k+~}_C{xyQ;v`te&%GJeV)6NMB4@-wbt66nD5@*`GnFU9Wd%B9q|fx%2p9Y8Gfyk( zh@*0+QExNMeP(4)o_n4-7hf+9V`+Wqcrv$r=F!lv-L27~&4>C^yy#sAO+j}1jxW5g z!6BEp%^XnW(?-tKuCQT>cCXw6K=E>>{Goi@^4m~GHBMVtMQSUY?30x{{~}XwtQ1f8 zfnd1FJDfdXQLgjASq?Tj)hYzLo%v&B;tM;Xm-kG+`!K_`T^?~j8%Ry${HoCQ=hK`U z(j1R)L#1wu4!51mIkF>&lfMp}Wnr<$j5o8HVax8yGSswpm;+MfVJHcIYG`u^Dx<|F zCAejC%)ftz3+6iC7g*xHF4LR18Ab87c-vi!ISvO8{)pNd}GG=q?alP=6|TCzFxh6Eh_Gd-_cJ|5wwEy3rx#lC zgl>?dsHP-})aIVa)8B$r>Vh#fsJ`w(%QK7qOqZ;p!KXDJk+0Z|N9uYH z=vWxro&MOCdzRaUqrjIdd%#V#&pQ=ao^=GjJp;?Sc13wXW2gtTvL}u=56ZTTAZ&2>JG`%2Ki8t~hJSJ4vZYe|<^UgU=>xupIB{doKOl zDU`tMU?s`YSBAI5&O_y387{XS52MtCgRZ)=447uj8Wq1gyCW}{blZP^m+pdW&*CN- zA{_2J2hFjM^G=B5qheY4-JEmu0FIG~-C#cHqKn0mP3xf6&6HW)zg|o?#2hcr$&Z@* zuP*|TG%`Z1SObv)`Cih3YfALtbJn-T+JEg~9!;E!;#2#o*v4nqiUs)i($KgPpSaRK zSMOKmzgcDrp-z$gRp+I{_N>Ut2)g|v!7k&WlrY~vXo*3imhX_Uo8IV#6G;i5zb%(6 zzm`$HhH$zrM%OxYen(C-EX!}j*dXWHhNDqXO?>bkLwnM%sjd<_$FK3(Srb%myLb)#eJa8=GX(g za3xk9-Q6I;m}d)agiiF2O1njmLFdKwl;v%P+}HqILldCmgyK8Bs#C&;8f4+yxjr>^ z_&H>c1Vb5+IajkO`t&1RFkoPl-cLRkt>D|2Z0rs`pOBH16^9%pAVRapQK?opX;^3h z10MJX@eAMHA=RZL9yUzh@Z?=|se#Jt@mEE=-L zKzyI$If~CP4Md-k3j+?haQLpT@}I@PeZ^3VY1*EW6Vv(EHR4|$kS2Mt{>%z}9LpOq zcjL{%682X&b|U)dCu?`EIOOG&%hu4Hvj9>SfgtG`^TzdLQ-$g3L66%T3wP9>-rjov zoO?(Ek=3BIzjXKH#EXfl<{N#>q5q7jmCiApD8aam=U*T$$q-H2$jnc3g57(J%KSoy zE0Hugzueq-SLhhog^RwRXn}FR^8p2zGj)HtO$B#SE>kha6ak{a#qDyqeD{F_b3h0R z#$Yg7TX`sSiyFT5-E^VR{Wc=ZN*i$iY6+kws{V4@rh4<7A;Bs_dh``_?532(Z*^lg z*={#C3rRv<1>0Pgp`Ttx@nXEmavoKoJ^I2Pg%>>tMfIu8tG9kZeZ9|ovFv5fANx0r z`z|#HA$uo%GcOL-VqT#} z7GW%qXF14m;IrUk9&`Qp;W)$9y?nyNM-Uj*jf5I|@)`++XcM z|0vbwAP2m&u}HLQQpH_XuiwHDOW85k@0qI-uVRmvu27wg`pU1(dlv#{uF@oIzRzM2 zeg6g+-WOJY!FU7kyT%@8FoI~kzr~J({=}O&Tc`HCJI?gfbe;JeX~57*?H0}Rw0=f6 zq%imu$4DO=bnG0FwBS5|VfuSXx`EU;>P-3%NcR@`1W*Mk<9J>mV* zUJHFsnWr3k0zUK0SvG4H2vYT2^j^1o$L~kLZQ{wM-k|U|fkT$E)EE}vz|?L97xwzO zYzF`B45-qR4z0OOi8(<>Dj?(@Z4P$L{1*3Js^evnG7KkQi^hY@`{@HJ6wL_5ObTCnmFRCNzD?pbq%&(J7GZ> zvqZc{UH87@8jEs(Tb=<)ts1LgX)rNzc0ctQx9bS_VKi4~X|$=lcl-C#>e)+(X~ytVbXbmw=5^1n$M++2_Q7BWCu%Q^6z$K_gc{OrSS zv7s=Tpp7L{GsG81BTt7E8^np};Rs0wmvt6p%3_ zavGc5Wjg8YzUdpinI+L)8taj#Ji5>E(`aZ}3i{ybm)|rgZ}!*^A!JL=apVIB7UU$c zq(Th90ZE?i5SfO2v{gLo=!?S5$)9?y5}k&fJUt!cL?PLVQt=-G14F{StSEom0{8as zUueW|{jR4Dd|6VpC^}E*z!4D+oUsVt>qVu>wG8icv80WtG1@(oh+n@@U#IE_x5?SI^6Ue>4uRs((k^|Rcc!S zQ;P+Pq0OHg!ZyEnEsHk1Uln4|YGXfPCW|fUVix}rA4KH`m z_g}%#%DTmVK#{W@EAGdO%hv)-_OTLsMy>sYKec}-jrbuW#9>Bq!(D{E>KdXqX0h2g zWez}Y_8s3zir8>WanEWWRgLw<=P9yg-{vax&*;XOtx2$Wi@N3a!IoS6G)y|ZG>3mt^%99t%c6`tK zEtVdC`>xq^>QZ@CJRO0*-0byZ$y#xvnFsuY{=-9JPC3CoUVs8z(QWtMAI8Q){EO9X zMSn*93%%7VSx?I$q60*{8_*S$eU(W z)6{%xuq|bOaJT%2Kwq3x<-F+s&H{+<*|kotokfs57RWzr#r|Q!EV>PEjNjmE<*JlB z`y&@ph*7P6Y596J_xabt)_AT>Y^p8Q!*O8KO!zerNsXHd+suKA_tEcfF*mdSet=bdi;{L<>7w_R1XbgHDItLKt~WS#aW%jpyry{nG4`6b3nBA4 zXI;D`8f4l&muH6|9HSKC_wMSuV1^}51#g(E0p6|6elQTJkQBt0x7@Qds9?$2`b2MQ z$STB(05lDmR7{Qh!w;xVRh@BewPzY?s{`SMXI)Z?jSC#qwq;sZ5bS2zk;ryvu^2xgGs&E7NF}(n$le>e?2a@>=uiv;`+c>Zt2c*?{vv7@(xQQS?%o~WbBD?RzlkcPjb$_@Xc)?g9J2i3e8GBnM-G0h<@!o2&p1;3;A=zatIh&wiD>E27f z?E;2$tNY2m!zmWkD!CrH&(BIuK)AlUU<>F|3E8R&2o~5~rbe5y=ga_pSXQG;fb6)C z$%_<|2-^Mh+d!86%pI8=eN~*B9*E#bj$|6ZyjCU{k#7SOTrD(Y`~FKASjd6TT?58| z5fXRyxy*M_x95gM*9GB0U2gNY#LOJ-R@fJ9RM||SUYdM9_fH|cO1>!TX64oC7DIta&PGmV#>(FThMLp3nu|~mIF)} zmjWLwLHc_H%8xd`x{rIZz}R%Wo>?%!Uh(DQ{C@dKUYV}T8ORhFcb5mf;l&%ccwC2@ zfQ{y^TR`zs zZVkN~RrIP@Z=I1H%TTK+&J8fJnfs|3w_d>KujRvWK=4Y95-oq~yR1a1?R-j;T-9v>ubato{x$kw?O0!DchPWG^wsjGwj4%LFQBk+T2MY-!*9bg?0rUBqdxGjH)(yfjv*`t{c6xu@#gw5i6a2Mejd z+1nY6kR^W@%An_L94hYJWaWB)1X+AJ?S-lqGV|tAcGo$aw(@&~A{I*~zdNhyWoH$7 z&FJEtFCBkpmwRMG>Z+FfvERk~yk#R*Eb|S-l#7n_BbUjYH_SNFI9~d&xL+(;EarpRBIp;d3~) zY(KoE4v2anhK)W|VuD2s%>sTn58`Yu4T6jhcOzb6MP;bw2uRqF?ij`bDIsATfxZ%6 zx9Ht`ckU4>hxinv*{K)WfYW-w0St4!%&mRGW`td^p!2)h9b*XY&86B^2CrVWQ<;V# zk-?Vzpv~PqL1+`_Pvy*U;d){ZOT|$(ouNO&B|qZg$uD_vtKT6hnm|a4TCNL%e-kS$ zx8*o`N6kWDc|7UKkf>>z#=GplLc{#+jcs;E@ye+=suJhGZswx*z=sM{StA%#J|J$c zo{c5;f(Wy}V}Zpmxq=tMZ?)1DiMQp=q0FJO2`LdrZo3}o&I*^GQ}NS3qThm(ou}(f z$!7FEjsd*AU2c{7XK&Z3Js-fdZhD%gzp#7a#-(CwS8@T z-8EdKpvyiag0b9Vq=0cmDpp<>6fsb!Q;!WFk8WR=CmfEFQN#a`u+tVn5Qlsf�F@ z2GN~7VTa1VbZfyoKdukW?Iy{3s_mPRua~wxVEc`3#_}EK;45YKA4jM+?R!tE6oI`Y zP}jTD@a?r-eKfF4ZTjtMlf}&$)3!X_d*!l^LSFS#oW}T;q6imo+QXbt;1Dt}UX;Kc zWO`0@fW0(DFdPa_w@kPuFRsXOxm4akO|(zFikwob?{6TBJ&YT;us&HP+7W*aq1$B8 z%6Oih5kk{K9fBlI<^Tu?$|c_Ek>0z;!N?+hVS?g~axwS>3GYRMk{ zAU<4>Lh(^==D6l4>9@RD{vIi?Ex`%dZ$B<5!vncJSc_Y5cK~iw`tfIeyHy77xCaE# zy!7jM=zF@~b{bRJ@hgu#a6j1SPe@8iL@>65{9GfdZE-}#6}22iShieE?Ufmk#Yhrw zepqjH3H*csJ7!FgH$_X@nlLBE?Tj8a??U>ZFHpYi=)f~y!@qwRl3cF`I#qQ`?i*a{ z+JS~mkg2%gL1ir}7!bK4B(_CG*+OHNZ>EWzhs zT<|I5v zA6hrnkRPA8@v8g|AN`M-=SXo5{U&jymns!K(Ry?xjj(%2p=dHCrT>gh*sXP_5`Wx^ zi`ny9)%9+(p%-m$wcWQiLBxB|T={AEmD4N4?S8&X-s^%6H~wgBm|X$FI%NDC@C`PS zqt|PyzK5^ma~hsIRd)0)*cX}a0FhR%7ZrqqPIG=Qv&IrwP||V`#(fFxllyJZy%*1+ zpzue}8HnadVfILJ*ktQBb!igb144Yti1imlOwQ6m2UHVSAW}Pq7lZbq*r1ob%-+!l z;0nJS@FPMWf6L!55t2UEX$K{{GrE7b={hd{F?|0$_x+oor z$8%L{XaZK}9ZE*e7}zcLfp9M{j;zhO(y3t)>c3y;-t+scs5B{srU)m_v9gRja7v&R z{FN7T{$7%Bx#@e#@L-=7ui4a1Z#tgkG4KjPXT_@(YoFZl1_zXdV{Ycjf8YD4b_Y;Zz*mA1P^OjU0jiwyV&qYA zx#hwllPHYg(2l{?q!Htd8Ai5Xk;{!M%sq>B-3(yqE&Bk;@IXOu_uB`>E@lY7sH>+- z0BiK5@sy;p@z2Ipi66CiiKR27yg9;W#*M^YqW}_&wO6o_y#T4YI`mEogNDh{__RZCQ>GAM!j z`{lp=?ZfUFxJxV?Jaei;jjQ=*76cRZ-sCkuPceS~3Qp-|<45g2o*&pLIv(WxT3R<^ zMVn+jsc$)ckp03m))Oo2BZ>K*0sq#7MXZWP`TOOd77e|elyrsP&?8JC}82JnP;&mr$0X+2=V^w2+E@aap~~h0}cQ? z2F0RcVKk}0X#>*7#!w+vfAl$M-&g2zNVI}FVNhQ40U|A(%( zjEm}f!+sS+q{SE-R7AR@Lpr2Ox*57VWM~ zH1T3vu7n|=1zbq~?%cYA{Q(uKL9(N1>NadwPZ4N;hRs{bb*=^Jpv601&)>Awn?BkN zanz{Wz?@id`1*?1jnZoSsLWp9X8TMEB5aNc|MHlVAzERe7tY}Bb~qGm)~~SMHfl6P zKKtPZAFcMPN{RTRpVE(3HD z>W&8ul$jB3Uchpi;}FXm-=mUXKKsx}K^?c) zT@mk6bR$#+O@@xXx`kLBIMeCERTh!GiO%c4%CK(L@;+i|qX5HM#)rd3GKLb?RbUF5 zE(N|6=dM1hOZ`3HTZ<7O4xP~%dS0{h2-PpjhfS`!-#Me?N)okze9~#gXomEv+_AxE z(*117t@uxSpdW~lxQzFJ#onHdAn)P$ZeQFr0W@O>*nf!Bl2K{ZL`C=y1C@_6Mqw!+A>VFeCYN4ogeD z%Xt4`Xi-R6nyE)&TIYYiwAstSZt2?CeucW9q>de21uH5<(DCLOOD?R+HG6IxJdaA0 zZl-Y;bhmQ`=EGq#%5E6KC1?k@RgdrB4 zsViyXbKDZO3?1g_328t05!wojB-JanoQe+cBX4G%)-(5~xXhJY^aFrhNFK%u@G!?@-`-@j^GXg$u9rJ%%G}(OioJ8>};A~gq{c7=GI9huL*L5?lYv) zVYoq2yW6ZIN#+Mm55}HGoZ-_+Ps$c<3aj46%L`;q>@=y(*hJ<;K$-j#%+6EpH3cl@byRHi^86L>fx0oS40^aWzBbHQqewjCXWqf5Z}vI&PCh=i zB(j86cF10sJaWum+rYQU#Z4}|cfV6xIb{qV-$i~d<+^1y`-%5Bl4C2bBlI6T&%Q+T z#A^8qH$f8Y37Pbctnotil0hHoTA_Pi|4yz`w2IVMcQPiQ(_9pCR@kpvB`fY-2D*eF zJ$T$vq0{qu%JRDkd4cU{LuG6-oKqGS!rMMT9Y~zLJOQiA`tIJEW z;TxY?5miJ}`LjFDD)K4ti^Y(ZT4*niq0{5`mD%C)%yyK?V9CCR7=5LR&Q+D?mQsjV zyM(dk3q&qm{T#?1$WQ_KTkq-VHRFl(9-%9^K#+B+A+D~vaj4-nm{x*v~rr$B9H zs%3E(&K|=FlP^8!R6lF5@*3kEw>rOxvC;)+Lj5^kL;OYG9-%QslLtUexOuTcJV;eS(+fzISl^>u) z+YPeBs_#uIFYePftt>gOW3aiL>9ZO1ePDCg`^B5Qgu1=<+l0)4fhO##V}9w1mF;@L zt^ok0bq5o2j197w{@Ni0ekYv?b^TwoZ*3+RidW)hb5@+MZ5lbh7l`V-;8&4lXf#b( zWlyJ{z*_8jp3pN0jOa~W>8-ERLH;B4tgVK6l&v zB)-|@R)eHUePUW^jwFZ!@>tS*7&!GZ>n1R;gog3q!^%W2y2XT5M~UjO*(b?&)&A)& z_%$&?kghIh9;@HSV|6W>6!mX0f!yK+Q$*)>cq`@JeE|{4U8E(^a_Yjt+G9c)4`WYh zyB@0bM6ja)CwE|HHkyIxv}FdrJ`?i*3f&zgbK8fRF&Dd4o(D-+NCa;$PzVGw3tCoa zb&Y9KZRj4}WmPj}oF z^|F9_?axIV`@SdlAiEfw)0XNiOx2<&gA+SE7J}(-gk6!hS2?^lCg)zw=~m2a zep(r`B~bQ>bC>B}P&@soHyEiK`J8)-^XU%~Qir+)lLlqSe~&f-E9S-WMboB8s}*rr zX6b7bEkr!?t*8Cu@UDUbcwgVVavv_>|oUtd{X7yz{=5l#dic>m20s?w4}F z=a?A1h|+AzY&?Ff4nhfeZi~>dm61Afc~<yDJ;_^fC!B8PdL&*CG7M$9H$;{JXb z{vG%H2Nop=4GotGpDnX@-76S*QLKM6YQ3$((;9DnUk?E?6>9g*rF-sJr4XY|gmRWEC0_e} z7ITq{qs#L4PXrC4k*)Nz__!c?1eip^2b@712bTs5($=N@)J912Po)9n!jI>6k9Qd! zQB(OI9(_4?lyquyR;(~g2i@OyBpd_xiKww{2&K4^O7Yype@1U{(c1f#Q&nFy^k2<+ zzDT(>VG_W>KfxA4mh%5X(FUFhZns3wa88VCr?LHrlNI=vtsXT-JTBRVX7-I9&lucB ziwfWZBKs-$e&re)dz#r+$v4%O*Dqi$u58=!*vRHT|B=<{i-XjmVCKLpt)0s|LB81j z{Pp-S&5-m0{nHQh3%g8znGYO3yZFA`yA%2H?(I6MR2Y`9!%kgtx>od06=D6#nx>&Q zb8@Htn2z`R`;(ayUjk5`*Nequ@B|@lpzE=Y%HmekpmIZj*RL&0PQQJeq%wC}Eg`UU z5DLl9UdBvmcN&wj%Eqp1 zm}-9%{uEqj2FV_LIFj)+KHiF-|8n-&nH=JV_oWLEsDnsi1Va}&#y-s>Tg;pjMe|p z+9l%&=|wuiUj+N$p|`8ab!MGvaF0jrv1opb6{n(}6Vsx}i5I_bScEI&H17iaO+0`* zEmMDg>@g-4e-Xq;eT=E+)0%;g)D_3d{ zRIM8m*m(^4CcIbie={+Dly)NMXsEt@ivyvt-oz9Nr6i_*1(PUVOTQHWzN0f+d^V?( zTW_kxa5fzi)oA8a8qvE{L~Qm+B$V;VK7>BTiYvDWb3Qz#?BS@T7T~Lm{}rr?(~~c6 zlhuIO^*d~p8&bb(B%fkKS*Fx4Rva zGiHjeSi*1|^w6o~OT9(de|x2nPFpLIR%1a|RdSZ9@{|>Er)3CE86#~dtk(i3l5oFM znY^XsEP^b6us1ak(Dw1&&-{bF6TkPXfpgVCM_cfzFK}$=e$RU9pc@m0eG5f4MiDf7 zc~I0W>oGz;S2TEUZd~<`B~z5-fU?w&zS6RM#@~CLFF0>ane%);s>qkT* z%~nsQ{nK0X^p{;sd($myD$6;Om4s3-wUO10BLqS?Ym=V>`ZG7+scmx7p+0Bj81^VH z&W&N%V(dh6KAUt8KWa#MqqlV3Oty$`Mzfdfe#hc;3X2q&6XhxY1Bi;8svC1DVnb)vO^q!{F;Bbp}WbVpt*n z|2~g9o$qldX_Vef`bum=s3+kbv;lX@DZBrUrl^;SA#fJPA+OzW)0DI#DjU8zdL-_2vfvkj9N*Cg_Z3ug zeW^smV<@;y8s&1uY|-DB7#DjK%P7bu{L~_@GxZ6`rQ@PIO!*F5I`#aqbXhFCs36m{ zM_0G;xy6fN(CwxpuuAjkCd&ZN+It|At*!p69ZDj$^to zLlHFc@&;FnX%2y-Bdz9NvW=Y2OypUyEDr)b{UTMPe?e(oCv=uPDk-H5mA>&(Z((T| zAwjk?)B5>zztric6sK}&KV;^{4{vlzXf=wy@J^`HWz&knxeZ?( zWrh-<)O1D?zsHddFDaK5NG>T)=kU;K(NAa`ItWCKzODSwVPPX9@W~rG=@I%Qz6Mj!(>gW-%PYAd3i4V?A8!-S3Cp4 zHpkmktXux^HmV&sp2yrnx+MCFb?X)v(=aO zFn--0#xZ)Nwvyk2W9)xbK75-ci9c*WAyZlDadMwRucpE)X=cf7UYlmY4Vay|LsD2t zoXf24oE3ks_RZ)q#c7{HdPHrH3CG!-%SL2i>$=AGSu3@($~UyYqOi7kc)CwPg&H=R zoY(5yj0_#FM)Cab9*QT2Q$F(a3Y1NS$lKbEWd177bm0ZEv^sl1;T|xnk-n&Dh^iiL z79`3cx=Wo+oz*?n@hF-UiMrg5JvHA0#rBRJq(xousZb-0y?VDHedY+gM&gLfz(;q3 zviUit8Q)Lh+%WbK@LSfvh{!HdqaKqt0f8$E>H{Z?MO@LX4_~mvWx3A^Eahs)*t>o3 zzN&wiHEC2+o;zyYJ@l%Jmgf4~ef6t<81v9$<>frY`G~`{E|XTR`;U~L{ka#$-;2?} z3B#E%lB<9OELe04XZ-K=HyTrQ3XiX8b-QR%Z$AZ8UKvI=2_rhGBTev9(y}XAYpkz& zA)BA4={V0*GD{esvp}5ysTZJWMR~zcgYkwSCy^a}Wj=U*fd#GkLO+5eG6&=1*U7)3~&W1v!3^VUm9J2^0>14d*N#dXDlD*eN38f0912)q%&%OI|ZH2cZ6vbD12 zIFSb1M=7O3EWATOGZ+1MqQYM$9;U~*p^s@iomvoGK#>O@XLi;mz;ga!eC~DH7wXw~ zk%RH3Wqc+|{Yw+rtNfI~+3oYfJ6iVQbLnMPI`-rgY{=8&txWiCUdw?OTy>wJ)iN6lT_X5wrml%vwcc7EUV{R}_b#kGvVhy7*W#|47!qTJl6i zTRJdf!bgGEhT(yhBjfYe2x?Owh_2}{>AUdbnw!_nTOtpr`A{>*@DcKLvYpd-DuxBr|nhtqpPM7!%wfW-o?)-U4$AI>#X$!Ge#n}hLpZEe1q zz>gN3tJwi9()E#LT<3W7>RnLec{ssX1TqMMkjZC=X{kbH!HK%KTWN?>>Eu6 zKB8M$m8jAhSVHXPF8E@`@6>shp0jS@B6m8`DU*1n8sTrK0%S$RH$?k zM}p(>>)tQkOIU|Xvkxn z{DF`@Lob*16CoWg&qPe}k4>z;EyV0`zK;g22w^ao(-oo0qNxMt67yDNE>W&4efb`p zhJ)995o`4CJD{4!c4(o<#>U)Sl^WJ9xW_-HrPoUd+A)9GMdZaO!ri2!C)FyG@OJL4 zkMaY4O_CWEAq=6S8Ht&MSu~CpsoUt$on#0ELMM^lA0~Fdn%hF$(rZQiRF$g}E~f!8 z3T>MJ>ToZ5YsbH^EkkxnT&l|wWhBj4St&{7gzj+VPO+O7jG>0~fAPi4 zM#9y`2eOh0a9LEt{3?jcSc>UFfwrnmKpRcVEmHB<3qzCw!}w3c6HzarZj{8lr6FA;6PG(#Yfp zxKDV5CI5Yg0all+BOoOTgWk&bpCe7v#h7J+MYW9tZPhP}+vk`t!0Z!!1VFR%A?+^y zWHHtNA>8q%jDKX=8pv2`@PAKU?}N#iTm6O;=#9edOPa3Km^{3YK-R$j@=qNc%o1-x zHkJU=k;G0#>UOONH54vlK0<}oTv@iAC?%jX8vyGBHy1(lH8sxfgPC`gBNV_v^Dw!6 zITp#*dSVL^EW>Ld6efxfmea{+oL43yD^bxgtnswwAUYm+X~D^d(!T$%sYw{VRO;E4 zkXx!8b-}$uXO+BU)(xTEdQRVR-TG&=0EB_omeZto@;PHt_5BH6ryvn_2as!|oY9uXm(^>H*q4$3Vs901@Dn&bt z0V>cf9v~5@fxxP<3`Q)qDz-6el;ZFVSQixM^m`v;cma4iHD)$s5jfABdq|#6kevW- zNYo%QUbd)71Ba}2qbQd-NKRrMGvB%G#Cz3;r4iZS{G|aidz#vSxY*bWaZpYDd<;E% zr@C=jD<4oLC71nUn(;Yjjp*+~+6L`Wlc34smxdvP0W4i>9+I>5`ZKa3l~i8P&+Z7; z_o)G-f;OiDmlFG)C9&-VCCz()sILA*MY#U-6Vd569nw`mgk%c`GT7R;UL~UfuBU(N zHhuu}yS)4~)wU9ln0mBe-e+=;*#L=1$i%GJGZV%*C&fJ-fEVOl#LNJl?iNtcKmAuS zY{(ky+rL9w2&GES4phD=0$0`P)Y4$@9)zs=06nHi)03AW=0HN<>K28&PRUd^IUC1{);Tl!Lun3!`4Mx5dYowt!0)Ru5_(NuiPmUl+YdYWpgjAOA^V2$^*9EykQp}Gq>R&)5NuikTv2*TRzd&a1%G8;?_7zUFnTvp#J}R(JcY7LvAL<*30AM+Vdf<#e3kgoji+;_Z}Esm>uu8q;J8bF*)@eT*|-zsiPQ}&9CM$gR-4kRdpvts zNZz0hc5Zkg4{gPr^j4g({k@)oBmVBsu!CCJjr(p^>;izo0-(r%rY#<%a}Kc8;n zdueulRUhf&7P!09l&TRBAgMkCoLW;3Rfi+^t!`*Om998-531l!|&_Pd}H08D4* zp2OH}QLZk+tZ;e1^)(t?Itg^Tq-X{RaEQWPz+V|D=Q88UN9Q_dLlD1WWGIEzU`T{w z4id>oh{4a#Oo(lIq5lnT8AB{)6x(9z9}V{jMYOx-Tjb)<$pUy|ttU7=>@Kvyd+>OuEi{C>4sTmC3Zo(q~^0@5#g|$hHqfwW|+S^ z2OnI`Wiwb&Zil}m)?OL)t+(!{t@*QI5Ua^5+sy7I3hZzTZ?*Wu65n>)vk-qrK0Q8a zICeBjajSN{pI<%#3Jt>zfUL&#Z5EE4poy3F`sLKer+_J&OY)K*GHo^$yRAduC`w!{ zHhd!*3#E<|{#i(vSXIwt1#DM9nLBaEIf*L;a37CtjVd?Rd1^}1y;bIdK6aqMuRA*^_S)5xZ}(%>3wlF{#vN6-_S4wt!Lt3ver32?~%3ysafvs1^UmIz(T5wd;^lQSln-@ zfHl=vibPdDsqw|pXbdgie&T@QjdJx{N!`($bb{VP$<0;N{FR>mvRQ|v$kXanvVT_p z9vn(PCd)hFtv@5Dn39_?IWE?$ACP}^b(l9b{g1c$A7^HRG5KTRV%Fmx$rNUg;?%nt zS;t@QY&-q*EeFMoXMHjGJN==M4N}XDU{>4fC0DE3-0da|0G0Oto|=0l#F395Hne2rkCE)IEA)W!|$ko=+X#!RO5`^vG z=Yw+R?cD2QXIvK3s1CGsz;A-JyG!!hID<=y{uDUIZM}%%kh5O6%VMI#Aq?H_a~=ZZ z7=hb#G&Bnz50_mA?y_}iXT9LwtD_(jbKvy&HOfn^hO6KuP4mx!7ue*1v^1nCdP%qA z^^KgL|B8!?k8{I>x%MN)5Yz8k@^ zY3SB^1WLyaEXCv0f z+H~?MzhwaYRWbR_ul@`ov*H6say9@e4xeq2y;sWVrr8m+zJH%`Bb1u=qA!Gmm*?a> zLhRXI6%xx=46rxEEhT1u-2e@DEMQ^qOyZNnxmQPb9@4gI^N@dNg zS?>L^>7no9Y|P8ZuOF_LI&sN3KcRVcptc)Y|4Z?9?xYfLKI>G^qWOcU{8LF|Q&EZ8 znrfV6BhEMh=alj7S+s~Mtg3EQj94)*B&sGKivyGyPmxDy(P*!036WoUY=xZ zDRYStmHJdg8e)+s-<*Vgiird;*;?l4%bZzGZaSaYZq5^Hz~pplRQZ1C*n?DN**fk? zvP}XU^`caO7-2K*7rmrJw_u_k4dCvx2+$@X0PKV33{62h; zN%ZKWWdAlV6qs4&z0PX*8mrqtGg$5*<%%C4=U8^cdsR?(vMrj!H~}j+%j>nliOF;l zcaaOn?YmH%i;J14Nz0vG>7BNtsO6J_7QOu3@Kzfi;&IiA zaMZIw+Q17!w^Lj4-N6N^`#{5o=*>)7V)yH7ZV>u>X;D{R?%D5f5a6>ZLGJ?^(RvD6 zh*DhIrV-9-oysA6pNN&g22Nw<*}hy;;Zo^``paR|r1w6Mw4w${n{%k|j|9f3Ae+s| zn7*M+x#%(ZrAoDsppJWsTljpBj7EQlJfB&xe8E`ef{%_-)`=i|P?>=(8gz6P7* z;nhZseV73vjW_Z?hK^<(e$`bk2G~~1mJJ|Bkf8|;`oIM7?|PUp^tK>2OO}@j(4`_U zaR~p{?pBR8(c#YshR*0m)=YDP?Y;pzcr+8)DSe+ z{Vg%JA#i>TaG;kn=0XD=6tSIU#&)iF&2g8*6Tj5-6s2|pEiM1+&1#wL4otw`HBmtM(VI*ZhDlim!|} zT3_Dx{v%d*1X+Bxss@379vvg54E|U3s7BL;T166a(cTz#ER z!t3T=I)sG{G(B=}XJ*S-=?Zx+q}iA_Y4wYeB6Qy=+-3okAb8nw0r=N$lQ^Cquq!FM z8f}eB#F@{a5?6O{cQAF4q+!1IHS_x1<9|j&`&1KC(ifTtBB0>SvK=VO7Gq07G>#qH zn^T@FQ6woS(IuuYpbD)YA5m+j;@u?Uk#(njpVb3OFDq)Ae9E=)|A+=m&S?Zdy-syx z-yS^m;0=x|U#S(*a^R*^taNy21!{AAa2kfM30Me%a=%R0z@XvDUwEV3DeKQ8LCYKS zuEbc2$5u@elFrex=U;g+gfLZ(bK0ztI+$EHxnA~vM^}$G^VVcRdcXv4lS=B>F7VA8 zzyyWW#HKU_Zv}MSn&a92o7VWEXmY|@8xQ-4UG;Fmn`9Rdm<! zn6qT0B|_=g)9Mktak$?Gm_5c`f92e2mP+TX6%r@Q6(s}D8;H+2>GSWp2BgoBi%ZlL zWsvDgtR5xq54rQ~*ui3_cf=HV&nnPE8d=zyqs9P72U$j!GOfBbcd~k^Oxvlmo9~bO z^^7C#*UxIv|EYH~G2h33qLMYwWQ|aeE$p&D*df9!dZBAO2Inl5GRSW1d>$6{{Q~Kp zsfSODc>;%^tY&7*LNWPl853UCX!c55|#qAyDqUmO-Nmp5wQmTF9Q zapxd*N6WPm^3%>xRrc{-uNI+WU%GZFyKHcLZB{NOsKvrPVjx=g& z$+3s@{7;b8bBe8a7@SQcIsbC8QDN>uTDJ{yWZa{^Eb5=^Mow!jx5ePF)XZG;hRPdW zUx`XYJ%2`YUp9n=rqsVejDjM42n}2?b+^pwja;?{5Fs*59E*hjgVPSkM11VFXnBmK zz9noG-T8BhIN6=q1qU#nt&jK1SPej~ZolOa$M(TYGhPwBMN6fuI){>meeOIxkq~h3 zq4VgA$~yvh4EAVrwbtw@5=umeR`>gTRJ6SGg8(IhXkT$wpuy_pT_Deww=yQ|6dpd7 zUiB{d%v1`5;ypGzUs<1Q(aBZMZ@kgaJKAJ)W6l&fqu67>?3a$F+Bp#Yi;oje^{fig zyXk_nPn z5-9{GP9HX+5kS)6!hfxF=L;>@jBUID({)-=m6NBq%D(OM5dA>DVCS7avZRS_ahTI>kq358pxAx zYVOBViqBcqX#V0G-vg!Ebl0m{qlFlCcTQz8!s*=%X57Ev26_({bvR3s@FGxNs*shw z8$0d2owWQ0vg$&or6gw~%pe=)0h4>&Ev`kG1zH zmvPNMDPr~5QS$woVS-0ayQt=! zdIvUf%M?FO3?VSWb$_~@MkX*Bz!G}T^vTJ#0Jg-?^5JIjIh=YEkW9HdbzCCbU z>b5=Xi*$cDUADVFsF28Tj09iC{n(#NnOps5l%*<16nNv3l}3;q5KOwskGyMo$WRq!_#4K&Ue;VaG37C6W2;XO z5_#Biw=*^Kiuhg#N|l-5RcjRo>?6@iJ1c;zT=KlQg?)+yec3p}sh(0HyAoDrPb@

7VR)?M}{4Q9$-u^pcshs@Ok>3q#vpX>bK#niS0|j|4k6oneE&{8YQ&tX*Pf|&o zlk?X_&;`eL5_dsrpAM0^b+(dSk4lLNIBYW$%qDV2MEB@N@ro*vCcp0oroArS3i?M>qAEkW~2G`OZc%?%ic&y*-Vj)Q1!H zguVAH6)G$;&kwz2K-wkj-c>dyns`M3RZzR3+9SJ3ALBPj1C3qf292%kkc3z@p5oKu z7IO+0HDg;2vP@g(-1xag?8PhjtSWhIq!B5Wm{e&WhHDlUPkEg3-vZ6uD!CekG4~zT zXPE@Q$fW+?sFxOq^UiC3R&du#!#o{rF*=%@R(%JXu47w*j+(x;s35%)}FWoklTqopep;gr+HRea9nsyk)x`&IdWb|1Mr;YH{ z-4D1{uPAsmS0a=&AY}qpukM81{)p_=ER7V`RE1r8*TgG-;5L>V164(n4|pEaI$ezV z)dCW|p+jK(JTV3=wlA=9e{Tj5O$qZ3({2wLIP-V6Oh!Gi#y3DcPcN{<`2WefVVg;BW-+ zldOTX-#}d-?4@>>$8(?3KP&X^O z-W^d`uX$BkyS?QCS3rjL%x4Sj=X)EcAnMokrL!g zz>Uc{OP`u*-GGl>Q5WTFOG<&;b+@>W_jJy^klwrfl{^#Mwt4HGPaZ9|=4147mg^9W z_0!J#JE;AgIPMQ;796oWB*RzTp*(jhthmw`;}G7V#m*C;lh{98#h&9|(@j1utJ056 zEbpv30cIy`kE-E@E`x5kz_qLFS#259TCRZY`<4Zun1-C=0;wk_)bG$qwrk52ae%R@ zrG<9TL)ySJdbP7T9=ODr>>VdLKByiy9jabu-QYm8qN?K1tnE!>kJ)~Phv3qe^TZwD z?1SK&9E9NQ9~CatPy${!C2PUpXBLi7q-ytvkLbsK;q^V=6QA8XejonW&MTcWj|R=@ zkgPrdmh#&MGuK*C;(r@TY9@*;aq|N$2d1ClL8mPXEl&<+*2hRcfrYoMi_8;^@>5UW zd&A^in`q3nTNnSV?U!B5q+}$m)A6mUmDOrITV}{I@!|tj(05@?u z3_{c~wOT%uVFO!ZaNQqhLtz(p`yQDl=TNHL&5k#0y6q*8TOE{^(TYlF#kv6FyfT4# zcd0bByXm^~{je>`T^Ox%Mv)ZBWzt=&W6f2|a5)M z-;juwnCPB5T#O>S;E~^hx3`jPm@1!5i-LmjGSd$ujcmAp2MW?(9THy zh@u&v0}XvVQ2VqN7h`3O>Vzo3^GT@(x z(Z#ZogL3ct-tS!;O069i-m{I&Nhj| z*?Kxa2h=!t_jB7Drv$l6x$)C}&Ph9~Cg>-dcPZ~-yy=8&#(dHr{z!qZ$$$Dg<|?p_^i_ z?M3EFIHhoQ@h?LyzK6^xdA+6Ixs}MEDlV9?O`uoW?h?&Svf{Nx1YH zQ-I!-13Nd7Qy=BlQbJ#MlM{fMCDe+mScQ^Ue8`}i?F3?~0|N1_GK>kncz!UjEsvw3 zYqmgV?d-kqxl5fd))ic+)cu&E7hy5-k8>dU!+KogE4ej*UF{Lq0(9CP&jl9k!Y1D^ z7T1yYC0zy5C$C`g+^E1K+RK`)w(WWEXR#iH?XIdjuYQg2ItOkA zpoN|Gi#)Jc8&;tOg3BXXvou6#;_nrSMxLM_0)3Hs*2y6&-aiSj&Y(Lb5AAH$Xgx=2 zjkf$RdG@93Lz#ncZuuVf0a4&mxYgO*Hc(Jjow0xhs^ykxtACR0Gl=!fSa<9!5!~0B zhSr}(?diE`O676&k(2TDri%xvovE6m&}8s9jET<$sqj^IE$#WOEpd^8s><8562Fs- zWAs9#%T`vs<69qc8fk+hEpt5RdWN}h-k>!HmF#kh2wTKw6~uGP}8A1m8;b9T+L)G*{#)vIo+(1-`4e_wI-_D_8L6; zo+eu^)Fx5k%>0t%IgXljT24X)mE(Q4TnOQ5nsetN_&vlieREsVa4-Z!r<4KdHRSaG zX0f~Fc;)A@Z;j4LUjoDGKq-IO{n0(k+}1)IP+?RfBU?T!M@;-BJIU#5yL zj~*T@65KALv+$;+mZ#Q)w4s8x-8Gp8&l-h8f~Jf)lXCu`rz|@3Y@I{$A$k@w5 z?8L`@E%yYllVLz~j>kRM5~<)~=H*9br3v>7y^tETKA*F30TcqyUU8rgrsp~@Mz_nP z=T!;#w^o>V7iTAYUpKxQ#mlRSDpLED)UdHL(+wePV1lCrsB3fI=yY1!Qi`&{&N8Qv zPj<09YKN?Bhj98@qt_xCw)1j`?XWwpPc*J@F99*n5hKI{NP)sZ`XAkuEr1jXTz$x0 z>KuKr(YqTg=PlX{FETWzzRf!u!>oTi85}whC$yflxHPUrbkAWet?#~u9@i{~TRz?5 zT!exp&+hEIP5`irOJLE6|0(SvdPfPs?;Q+vfy8ZD=x^7}TfvPoXsr|(27U+M(zMZ6`^NcgGtd2zhSE!O^)TdnU9wE?Ar7 zjexe-w7@bu6Hqtrg8oxudwpCgW<4b}XpKrWsf1T=n5+-Cv7yFLFEs5;RpNK z8e#tK&FBQ3)sx7hHkV2-Fs5?Qfo| zx_*cA0J5c6x9AZ&-|E4lv8)byYRl>CtoMtAiae{2AiVCnx?uROqKT(Wdra-D_FEQ6 zPO@ihEkQlcwVQ(hbiI^T*|ha{6e5QdbISh8>zecJ0bNj^Z|0UZ07f35JQ`4dwrYT} z8&28->i`<`7nqF>Hp-P&Ui){)52yRL$alyntTDo_Ty$PLd>YkNe1R&Nto(Yr^ z+KI2e-MrcLh?qA44QG9U07?E~YAsai0=U zVjxP65)f2UVgp8_gp`4Fhafq+8wS#Xf>I)(B8~J28EiBN(m6&qjLzrw`+LrFoiE0~S^Br9&__8L0)j4Xu+MB>p35R2f#} z0{KqA@Rx8VCGtlEHkCL3Z1tY$VKPW=mb`>ft8{XQ$h15v#dPuZZYD8`-FH#pxYWCe zchv}i-Q909v4A?Iu(b=kf~Wb0^xMm3bV&_7j}mt!a#Emlx-V=tbo7!6S){dx4B&IT z&HTKI*pK7o*=Ch?`)*Y~5h{2G0I-`y_Tl-s(EIyCO239LwD2($LP?LZRDtplr`h;) zSLPyFU=IDeDbp2=q<^mas&QvQ!s5PWQNwX%A*0AWtpMDUqHavxbavgdYHXBMlLdUI zV{7ddulXr#Bzxy_C;y>j=ZfV3lxzb~^U)Q!8*X*!Pby3Q4<|cs zW_iv;{Dk08-hv@-H9#T}EDq)HfktresM8HP|WErM%g8cdyI2zDS>E1|T)#gi!~1vKouh^N800TaD8?wCaHm+MWLSEM zn^pS$#{KG*~Pztp2@9#k-5yf z6->GFu$85MmoM7>s_XvH9{|gvAAK9BFxuON-VjpZ0x&g0cm01{Ixhah%KkQb=l=ZH z8@|yOE$j4AN;X`RbRvV@8!d8MX`~dRuA80U}P8wP}VVkp`V`MVmGQYXR z?sDpnxf;Nz=z7si?5kvmm5F1Xd^#st0yfJq^|mjD@`0_E;u!O#A8kWImF@FG&^fN7 z(UGJ9++96m@$yh?uT2Ib3l9506F0l;q8I0mM}Z(Ce~lsQs| znl0Pko0SbJbB99f!dwC9<6(?&-Mgx+sWLz79DO}Qm*SR(*E2iIHXl=&u>WYTfcJjX z9l>@sM~Hfmg8Nmi=2^2mHWCBAG07E2FA+^onf9SUdxY$NNP}OH9|Gp-1SC{CnlLsp z4*r&l?z^skIjBE%nep$m%@p0FbM=3GY&w24d#37$pgKoG63B%nNH_+h2LSMvpr%E^huO8?IF;Ex zAByW2Xs%1q?&aBM&5M6*4?bLeJ)d5UN~aA9=WaMQG>~rqPFud6YD<5dL+Ne)BfbAU z4ql(_u?4(P0NEaP_%-06UvA35OS$NxnJp4N_M(nEDetMGfJ%)1w@MjWW{a1%dU-!HC`?g^fFK`DGzg!o%Y{hwQ#lrK!xfyh43!D&)JS=aPKccSxG`IbpX+eYQ#2$Q_7&2K#Q0V3QMmmR(h0u5l=T z3jS|ba!zudfQTJMC_MQIJ!oGg{+(13=jEN1f&|#-R)8z4*na&xWB-xN(lh+apaK=c zj@E{wSiQqVj5=cUQ0^j*n20R`fJqsS>ti=~|3$P^wQYbf;eF;?)u+Qqt*W)T|H^we zg-FyHohdZz9}2g$@EJK^N6T9O9$HlC#Awvs7&=WsyN8~_M*S&TU7ZUM zlV0%5g%IOxzac7l*5qGtU-*3*Vlflw@_m34pZF1F)OpjG`}}=hBoBZ0HsX-uN5*blB;yYW$Aq`v{tjNrTApp}lgRlQQa#BF`RQ@5a?f=> zbeY*}P@(R)X4WR0eRqWj;v_fF)Fr+Nj9Wff8oKd*9V45x*9L)x{kadSzCLO~)TT0# znR`N~+ky*kDoh;`EdjWE_IqkZuAVn?J<*zGjfZd0?fs72ge_jS>@?5qWhf<*o#U9j z{yeFRx-8sdOE|g|I{$ge%N*?8&>M_HQm{;shNdr|6x3owk^O*P3h zD|d(a(#mdslx^#34AVAtu9@H^HvNqo**jxgVl=Qq+lek}L76M90`nh;6_y`+WzaOg zs@N!`?SdHR3~iT z%6A2jsEOo2;M2NqEbppFu2=3aay<)p&we_E;X@_H^4LxI*tz_~cKdd@kp4&<3Stus zv_160!N80ORsGLIrf0pa^i!tdV)v4{R8Fl?l>3py2jU7RG}IUDYgXuV?;DI)6^VgE zcx5nG_Cnq66>zNt4zLs{PIVS;!*zHeSJ;3{kl;#F@J~i+Asfwaud}*Tf+bAJdxTts z2u-~vM6h*tQCXPXE=|gBUC)E~ZeLlaq+14nUltPP#yjFoxilFQR@foB{0B4tu+1E1 zoFfB{c9(gNcOR7Q19CPznN)n@`uyTO=0x zb?7Ba44FDjV#idioba0qbq5vvQCS7rKQM5I>XI8cLxaSRO|4ZK{)h?0KQc>20>Ke= z*}#mex|B^OoxFS3KS;S%6fYF)m-d(RFu#Tg1Ut?-o7L-#)~z>`C{H;d#0 zt}hdhc2E=T!F$;2izgTR&XF#6b3(jn|#l3Y5qNSPJ2zd@7Wf$1{__6{-d`FnTUPZrG@n`Fu3s} zZt!|912E#MX~9OvEt!dCHVF=N!le(2el5M|e{^5(nNl|(?^AcY_zAB>(8RAL*yErJ zQ>?G;&ozO$E73N{Dxg5{a9pIC2L}FW@)}LQ{BFe;{75q~xLl$126D4kS}f=Ur4-JH zT-_x4I!S+KQkjoHliVoS=wQgfNz8b#v_MOpg*7ryuBXoFzc&``Pu{r6nUq9wnzsZ(<9cYR;AR9g8P z3L@qT99UMOn=Xoq*^Artqo3ZCH1*3SUSAIcWx6jj<1$^g4o&e>4%yW$_^l?FXE36i ziWbc&m&7-|Z>E|=>+c}S|u~MTqO4vPq>Nox*o$eT(wF zQ495+v_9!%&)?;?W)4vS+k9#{p50~_xaf&EFFv`0pEu+hVxa%b%U|>4y?JACghxZv zC0;JLZ?@%KO?@Gg_oY)wW0Yhq{{f0!zvcN8Lq>TwGFA`${nqxW6eEdJ4T(*Lkrl+% zOgb|rPD1!f+yUvp*zHan(+62uI-e@sW+pv0?Vnp|l&Er*XX=%XFRAT6lTa;JwU-G# z0>#K1Gp$@mg5qL5b)w*%KVBYxZ;blQsC;aiiVr>Wmg;;vm)4iiG5+PwWOA&FglR`1 z{RZ>v&FObr0>-f={$8-O;;>g00REc89gxc61u@NJz*Vy+E$v{4AufXUo{A`Wl{ak{0*M*C`?)mEk0P4&c)jl{T;bjVFET$RB0B#YD^Tv9QwI;Q>xr$kpU_2zJ2P``2 z=2XYHteG{HX|34a_M!6%ZF~37)gl%OyLGoHhQNcESR|fi(B(o!Ra|A(KaYsrgMM0y zdTe133ZACh_rXb_p~7r#dMn@{%QgeL_bcx@z*pN5s2qpiSZ7!QswKnqWQPnK^fmGZ z4h|0TYQy4-Lw)nX2@+ja?b(3*v@%$qbO?(RQDn~EBV{xBwMusr5e%K*J%(Ui|GL2v zR(d6==c3!05oQiJGO(~TcDG(4`v9s)ts>gh?huOS^?uX$!RA2+R{87MW0!wZLH@gM z*o%8Itlm5o^0es0<5CNNM=P)~8|$&m)jw~HDz8`Ejpd#RvXZ(MXtQOyE=4$rYIRli zi}xO;UOhdZo~{Qs97mN}tYo?eH5vK^2!6W6b~YZfnPY0kOwT72eL4;Gsbg~eg{BIW zWpP?k{OM5v5#^d_PT#V}rrIkeC}Y;^8R&I+Ao@=>FvTMRtLB8M;5*Dw-v+pNs5SBr zF28V}$2r)r=JCsp%PVwuKqw0Khu*nTJ#z7Sa}o0a^G4};ix+DpF-&;l*D(FxVVXJnZ)#rrdd7Q+{I)x9E?dd2_|w^|23s)ih@F{O+L2NH^<)lKDkvgS~N5KX4ge;x+5bUW?`)L%%+6 z7y`U?q4n4(2a^|DETp+DR~y$()UxAEZjx+e8^e8sy5AkTg-b@Q!_5sGP3?VJo$lWn zQ6^{z-6LdQrMNA%7^yC~i=n@+y2O3{3|IZNR}v3*cu_;YQjphaWaUt3)wzrtV07)3L!crh_RKSlP^ zGPlB|E($9V^~+(L`@EM_Z!Fl~TW>yuZ2h9Q;W*o8f$1nm{FmCX>BTu9by?o^=6dd0 zcE*1XMn0vy{?nJ4!fG}y+?B@L9?(L_-cA^s+EeS|KL_fz{!aVX-XEwvajWd|ddjhS z(@Dq7;@V;ww9k4z9q)?|Y5{^;2d0g}4I8EGM#rliw3 zk>PJURzgyPHz7qp^ekg3OpLegsQEh3PwfjS#+-8We5~24O1!vKfA7`%T1~eMj7JZm z#(GEuvuYabn<)OG)cCac5audalAYLcvE-8XnW~+@Dn|sT*|~#Y%(&*^+X_ zJzGV(k5Y`J3Jvn0^(Tf0{$OG0LqTlEry zY<}?lqbYWGvL7yxUP3a-M_d-4SJDy zoBBH3x>%bR+}sd~a$1|5rN=9B&?ixI$(b(afv);F<6#NnZfgZ*TF49844B@6frGNP z`^kF#uc;|RyA-Z&M{1P~dp5MO+~JQN0R+x_Es`Po+33phqPL%!|IeF~$A0ekJ$t)> z;}~cK#UbLri!d-6HI@4+C}foY?X zQFnZ$VEI6{9?;lS1)c?LDd#J}SD+fsU!u<7Y5!|Mca&Cn+2(G<1|w+9qCjr*Da{K5FjI+_eZnxgNqHv4R>e1-Vfv@Yt86bmpJErRb@ zbm})8WHVN4?CKj)10#H){nWqu!?;hijl9$fKE}w;mRi{K zmwa#jojk32BD+CRT*FByR410YRM_>w+kx__BIAu+7`SEIxT!9%B`Dd3l^S2Z*LJHu*dfeXgw3wr`>b&L!*@jiTtY&WK-@iF+3gHKn@4D?gBGs}0j{8_-lDu;`!obX_*yzC+&3icVemj}y2IQ+ z&8gH=1Gkt*iDy%z&;Y;p%33UwQb!IMfzmNoyYoJ-cZkrjvX*RBpSl?0zKhyw{BpOP zRP_O0&WGXyUecXAqpezr*x(0Z{cU3dBlvV}hfKMt)_~|hXz+BmN`P>4ITH0n8YXK$ zbf*Wqc9bOx_Rg!tuFb_)?Xd)M%D+Rg?ENrQ2Zqv@y- z+f93@a$^<_-7y{pf?sLR;IY%RSvlA=hIIpUIE-9wt<3sY zWIWvmVm3X3KraKSv6*ZdtIl|s1aGakU6aeJ(HlyIG z;^iGrt!y$k)JU+y-&{yAZBL)Weyv~Q+ioF3LMFC%#96RAc*{oGXw0-}`iUeG_KhkH z^Mx$p#5=0REVZIp>cPNcgDuR~gu<_rZl!C1hAM-FT`zK7Ub+7r%2eU|1tnGbKlFE< zJ1pUxIt@3LB=USf0>&sEwlV?0BKd04yvH-!8Jh~zM&xzStc`~fBQ^DY{@>3pcjjEC zT)F4Zb0pmp>d)3QQV_L{s?#Oem3P->V=EK#GsSN+oX%Zt!|(tTW88?aP~+2Q2=pFKBbAG_{EI*gy}ZR_SXH=~H~5e|&T z0XhEL5Ac%?{Y?y@EkfH)l`j|TdQA?DFr^Tsj}D9*ZGqIkY3~RuPTeoH@Ypf2pb~D} zlIS}M9qXI%W}o)2LM?Ovzv|vMB!{_q`a}D&>9SSaair+(Q5$>Ncht*hl-p_yE>c-u zcGw53-rG1FQejEBKujE=ct{gsx;QKSxGBN`newqb$bk1}0P-m-CO6+G-vB%hEw71A zdGAPE{nRkq={Xv=V$6XHnWKLbao`KTtfM~9-CR1wxTD}f_<{Al4fc8{!)~s{c;`_P zm0{P9zo5eotg8TCztW%l*Fr_H>An$WWC6JFlCgz9zUH4<5x=$#jNHi1luw{RK$J?R zJ=PIt3wDYokDZ{E!Z_4N0cU&^g_6p9upq=URm1x>nplrFt9irq&Pc@8(-BOGL zyUW^v6+Bk)RIVVa47eFz7fhApTh1unnd~9m9S7?D z$}ZVLlgE9PsatQ8K%e0pQ+q>Y4!`27S=|bS#yXk66g|7p)ZRR0Q$|paDWINolw<|! zk@!hB>b-=J$jM!}d(4nfp$llz3TNx`YFHU`3kyA(T??pOe@qo<6PPY!AV5;-F|v00 zqU6)7V6g@GwRDnaf(;z{s~dQw*vx~(il*F;1M4|X)!h_55qrf_7R)^_qI?bZv{q7d z8SGbiH-?LmLp@J8p6O*}Kgx<(O0 z2W}yH0cr`ASt`kCivtDawQBM~N~ae`OPje;^#^*;!O7(hme4V9=T{Yb`M-%Z8@-^U zE&73{2Q7}JrI1_JwBRWZCG7!BVg}x7ZtWE1{<~@I9+kXt3p^-@+Brx>wn^mLntm;2 zEMfmsu&tg1?d9IrTkwF@AnxPx<+=|alxB4Fz$)KW3f0i$Q0jYfXPW%|{=4L3)}O1G zdm1lXXLiHbWzb;oSfa{Zc1-35df7+lXCQ_L8FCb`y^^-v?gQIvYVM+3Z(&(2t3WQZ zmqO5-RR?sqHxyLZ*sT}Txkc~jBM3Z|40LLVC*ys*`*f9z6g8_WOmO-KcP~j1+tt7=Q3x@6ir}4Sa*JQc+K4XawpX> z?}PG0vGz}??nh*y{4!Ke#+Kq5n46+9H+}lzW5FbAM5jY)#T_2LnBcxC_u8=EI*UYt zqqN?!Ql>Z&vM*YrXSU3TnrmnC>8uD>txnu{M&j6*T5Z88aU$|yr+l=QhK5K8MO3YJ= zNR!jztMsTd=h@FKD9-p2)VE8>Zo5jZAGs<7HFk5rK6^6NP9zR;lypSmx+UB3cPJktJw( z`Hf>V6lfhQr@SfmoX%G_Mi)m77fa^A#`RL_~BGZrD5fMJHOo+{k zug#yH`u5G4@ag2Va#Mb;ztge8x-KUjgKD@r}f3<8Ifr6t^sYchxr_^^5pY%f>e95$n`QN+6aQzh-rs81!E z&2&&1Yb+{uMBf69XuvpS1*j^n#h#+$DI_G=?w~W&Qmxq>A~DejbXXm1CiD`|V9viQ z*cQKJBD9GTP-bRKN~;cWdQ@r|hWOc|hF%)R?_Jid?^QT`(yTztGcJ(@UWw~-N3_aw z$#1J_e7Q+I&)V>k=wpZ&GnDewLf&AXemF4D^(zjX_Oy&dm0{@6D&ei(>~Y@_G|F(J zCU~GLe#`!9WH8v!#fJ4C4(sT!iGo4UQ8Gd^L*i3MEJFr@&9AdLn?6O(PrWj&rZ(U8zdvmIGbN8iou{GnE{}ksD`)qWEbq@xV(D!@ z{>x%?-flXcU96sW8NpkGK$p|4dI1Q9vG&V+2rW%w%5ev73GX95ngm5M``L-V1LhUC zs_BW_T)pRpM0NawoB43AcQ?yr=^2D}lV`_G9ny>(saKumCpFW&W7DkUZ;~WX;aq*S zzFybK$lce2T+TpE`{pbZ<6?jtvOnZ7XEZt?qSgG~{Mf6;)5hv_3DpSCa<6*l*ACL%PEWi%jf-a>n5Rs@e$dn( zO67X*J~ZKQZU2*TaW$1&L=SiOWF>6MDLoOB5F zj_8>=(ene8ufET^In>MkxOj-6RR<9Mwpi}Vcw_E5UI)(;SH1uMUe##IROzE}tBu3T zd*AUuU_ayTqu1kCmEFQyKvr~ZHZ{Pd%AD{VGbza>v5NR(*%ongxsxl=`gs?dGQsqi zfw@9=9sL+TcbL()ASY^1R?O0$`$HTlujwAJc|!?6mVc&NtkTt7NZ>y8eh&vv<{5dp z*|f6kVMo@88*Aj0w>f9^D+PIa#@m?8rgwEDK9-5(@yy1NI^OQ7+iQ1iYgNs)1^g)uN|S`NFg5R#PRv({bg$4`%PoyROW1 z)V`@ZlJT~!-NBojPcvFYtR1nOt9(jQ8l*>udbH0Nugw&m;vh8vm(pAE+FH+3HOk&c zKnN*8-*SQ)K1q8!2igI7k1d~M{A=ouCLQ{mYG-T#^d!N?lTe~(X@b>J2oy+ex;IBo z&eI>=`$@D9u$Rj=z{hF4TBnkT0bCJ3mjN(GZN0wslyjp~%6oJsAnGvMNlQNt;9zTTSOI=N!Q}wNk02%VKGswbL$WJ`(Y& zUkbua+^mZytksQ(B1?D+aKkF1B8*&`05z+zmKvM+AG<&hx80996OLwjfFjzk zyPOf%aSMbv>#cqBfL143VVy73arECZ-PC93zS=KkiUPmuuw93v-VpfnmRb^|)aQMZ zaGsyJ$0>>5yy)0f~6EbPvVa&)`$m^~bZWtuHC#7K;0w>Pq~jY0@%KF?F} zcI08zfb4lXV8roXpOp-7uUeFp=adxE#DnsvyYzgh0$ZG)ok9U;78nw=SMp4oUFjb;U4TN(&V47cMq-` z=VWqV`S6$y8ne8oPAR}&J@)CNG$ovlHr1AgbF!^m)LKs?R~Y5@Q)JCH`tGp(cK4<+ zH@3T*G*IbNa9`1Q1*k|>yOIoeJklFV__^&_bEhW zBL)ImSp@4Q+*#TON4f;wT~^V<-``D$W?krt~dXGz)bd1zvZ(Ej~18oCHzu!ezk~6U1DI;g)6XURH+X++ZIV$7bUWiTi5q z%U-Xk_{WgBO+h3`$}X&a@qKvTc*T@K9_st^yiG|p;)oLcWFcmn`LoG~961c zo>;`SW(YFck<5#5Bk%7E%Wl?8IWGg0s#+5Z)J305(6D9vSlGh@L83+7y}BUG{uNOA zBLc63BBhND()WJ582e(o;(5G^HBkACE+6Io#?(3pW$mEXAeQkgSRdqR0`J6IvUJL7 zp;YaGvWg~;Ai3UPRFr0yLpz%&AVmFDT*7e{SwL7ftTpu$n>>niA3nm;GII5Ks0V(bW@qq<0gC2Fg$EfdhpI z7fz{EXJr=Z`vO)vSAh)1v}3R6r34+}&}7EX{J@S+x-22+OdNfu_W2$QyEi-zZ0sQ# z0WW-BD3om?QsUt!ZnuNfROsPfP8o+d2e!q<)4e*C`UW8t0=UK2LO9lIlXknvx~ zO)lPeENGy}#kHdANys10U*M_s+bye=J;9Tosh(^%^(8&`<0ap%DIb*X8|hhQb=5_7 z!C}k(u|T-S3ZpRJW++rmUPqg|dL$lfxqU!;Z7cl24SMFgQF7RuN0YZ%AYpWn3-UP=6xYUSwTx%^y(wPP_gU{~SzJe(qc1PLc% zqXKefa{>u?7Ra&FZFPOUrxg0Q5(a1Owo9e{tfJFt8k)qV0jrC?u-mkt;Nz3?Cz9Z7 zu-dur20MtH9ubRbd}0oq59%sBDTV|L1?P3pbg>L|Q;d6I8L@LLLeroJ(V;8`D;+df zg4z`CQaPt#Ef~D$bPz%PD?`QE`RADsOFXsPPIb|+UK;IqWd}Up zOdCm{R#qYCogs!fl%BqVI;+PC$APLZb3d{_xW~1X=w48Kb&)1y=A^(P?ez;_2!1QZ zphC$KAQTM!!Z>ui_>{{|9H@>D^F&1UAfVMnI5xY~5dh3Gv+g!%BGvKHTnVb; z=fJ^gYUM#lOCoH)a}tU9oT4Sw#czs)e!3P|26*X)XN#g)^nZ3}utfwYp6`LqF zZR)=OBE5_k^6bGEsf*cT-eZtjVc9`~-$xU+ck`$He}0L)Vk91LQSW^R-W*MV%r`Lk z2PKSw$IP76sEIj%>*xKplflo6kw0*j`$|7gq15Y0*pvru=t2hMUGxB%KxMIIcl$!9PP*+Tq3YxIIOe^r_7}S(5)ky)Q!ST9Za;fCk0ft*`Y(OZ>m3L> z60L&co;uJ}+&CPeBTs61AY!*ycfRF{oPGyf-!8V=d{IzqUInmo92A)%T2S>E=o!1L zj+kx1))@j^yc4cGMem8e5)ro&@;Ox^?KcI&CE7PE&;-0e+GD|ihRqx&rG^{RezJ~Z zYJGBMmvWZ}&teEIpH&jFWFTKSnpDEwqY$`+3`ZdST&|I+oiy#b{Lcx^b&u@_qy4mL z|7Wk|6TJth9`X#pDO6xr*} zaB`Q)Jm4Q^m8GGbIc$$GDk_qK;q#PB%M9#bP${a`z#KWh1)PAUy2}awHqD1oIQk;( ziJsmd9hKhtHT)g{XX{=pyj`}71PWeXoWjs2TIxuC8gv4ygE^z`2^O(or7Xo3avBN> zXx(hNzJk8`E=*)hoVoxX7+$peo(SKo!|+}Z5rBj-K^-}rEzhy2${dnAh&w58_=m^f>m?d5diNuZzQq2Rt<1X5C!;hwM-xG z5Jhs(FtfQ@^lE?H3tizSMdRzD75?$&SUtn%K-01!!B+v{Mgsr5ExYDw`4Eb9G8o(L zZ>0wo%B`QL0VwsB%X4E;tw(#U=a=4J_QSmZR)>wj1xi{9oQIqjPHQ`wkLaF$022Pk zwUg5&C^c>>DTXwJwX48nxldlz#ga#(IoqhGJ+iMn?OPTq} zoT)b%#wZz`Q-;=w_OlkTbd2oWL^yEOjx<<_(Hplq3z;fZ?y-Q)yo9;PPQb)Kwmwp&JCS2vd?ux1wo$TyCZZ6wwI({C!>4WuolTH8A!K#FyDle$yg1B$&efX!xsbp%3np;R zhmZ9tGE$*t6(1ZjGJO0JQ;%?`W@tVMaknsUMUaxI?~CLdw_YmMAy)lc#S}P`LghQL z#xEE4D8Hos^(66WjzKk2Z?!_k#c{+*i%&zXW2i{NjRRsLtZB)2nLy*KL)B^5HJtGo zCP{A%SWN1V>EZV&z1XK=yZ2}^UT2<*iE*$y5I9eqbe;&0uRMd;4g?bT-d$nnmkzKC z_(B+FJRO(#=VO1Dv)vP?gU3V*d#0nNJMkvqS%0aGYFh7(lUBOBDIm4m8KC-;qsa$< zrf`}^MO`VK-kRI)n*OlC-H_nPN{NOhVTa-pY+aw_u+Z_W>7eT|Cfc2p{mT6}h5SK; z2N<V@ohVX@?Aq|?jAecjOOiUacAUs# zseU@beMmkOrkzh#wpGf=PQcQYe4F=?#2vRiR%A+1Clg|YWkAx|^>l~X#NXaWUKY~u z9VNkWC}fm8`O?hyUMrq?1v*AL-~zUy8pHvw_fuMwyk9NDNZ!7})-TS?)Y=mL@?K&< zuV2zFMS1d*rfMJKH%iImNwL1OF{oJOmh1EN&wGSASnGK0SB}DUKc2{HnLGQ`({2Z+ zi=$vP^Q7Xi71tnaW;iw!O@>ZR0CmrY`*G{ne850Q3TgP52{Q-g^;9!gpmR*=Q55oi zu%x_>E@d@0Zt{6B4za@S%do<_kkm`ZkVxV*vZY5-9@UP!ESh;c5PLzTmQ9b)GkVh` z)=BzFHEeW6QXNVV zgU<@Z9C#&RLpxQy=t7e+;8)3c4}POy3q;9 zOQ+ouQPQL%bVRx;1c0r{2mFi4m{~b!+cCOh)-LghE6hCgX)E`5@SM^&`Bf&=SU)_V zCtQ!J9G^X}f|xUvG41&TuQxobYLjFLYRBsS(YV5Kc)#>&!~oqb46TpyG!lPXMn|F= z$N6ArkVd`TAj)&!@KsivkxhjBEP2EVWI;?&IKe7Nn0|RhMzDJB`d`;3;3#-MhFT+; z9`}B;HtOqs)MQNZg?E)aR4R|(+q!AYKbmD>(H`A})%dpA^yzK8t|Eu*j;qG(lT1J$ zG!BwM++EJkJyjWVRs<#*)XN6X-o+8?P-yd0r>{W^Yl*L&=oDUW8*g+G?jgC76@c%0 zmP#cjBWXcpNKQ#eSPFXm`}4f*U3*Rt0{$`W4iV}J2kf}B(Jpqa!f{MnpKZ@IM+C?V zp3>C|3MJn?hW<4OojEqqYfq-5OQgHm1*0V&kZ$*#)~I25rVoO{eWcsFdj>L4jlgQA z49n#>C44@#%E4fEk7q)F#3%c?<(~-lLPX_o^K-J$%3;5UdXz+eDQrZVj$R9iNI*a4 z=GJrO}>i^ zw+6SPaXOZ>{KL11KjvEdm#p)Zfl0n_t&)YFedT|nJhCO;&J(BS-Ol`p#+7_NfJ#mu z?-Mn!y)pXf&|y}XMZ!^-1>!80A>mXBs$R}vZ4;6s`2t!G1$m-3iZQ^B{&c4EuiLMWs=A<3~;k^Wuc>gN>0nY*}<$Wf)Tw~Pa57SewCmgaX zs;+ivz7~exE;~Fc; zbBD{{e>4wRrtZz=3hsJ|I$Gwk5lgfy%Qs(aTV!$Q8-D@{58q|P``so|{BoCQ-YUTLc@KqYQ+MNC z%hQ#}C=N#O7Z9)>hh%@Ry->+xiwL3l{+MWy2=3$g>lHo1d*(!575-iXxLB&)3N5-P zAC}*~&~B@~ZLwrW>Gy^D-M`Q?dR>>CJd`FhDYomz#mn#vUO!kXUDzy*1|vq5w86J?dWEosI)7DHhE}wsz7)Gw?*; z&UonjHaW469)jfllYCe-X+TcnFs~^CYt+B6FT?vU3?LEH0^`pq{rIs!3aC@|Yf<={3#63&W z%+-P<5%RjU1Y{gZRgzk4v8Wi;iucnRS5f?7D4PJj*8}S%WI!{x1bym1Ew$xih;OEx z$_O`Qv)l{LjFyQ`E^SI01jVE@Y}Bj0Z3T?+>q&@lt0(nM*)M+xZz}{$(|4ENA-E!+ zpVNU&qR82SC(lt;W zrN9=ges#N!&VQXm?)5=f2qmPEk+%C0IU(mDJ9)$g)81fR2-Ju0{aWg!$oYDIm|tid z;BU7)=>55^A@&$}(Y|tsai}Z}NRk;?1+rHmE*XBDDu~D0)M|qp_XdH(-=!|llBBp98N5!YQ^;bdO0@tUr?mB2{2FUY-{D9OYs4EndXh5l9yL4w_? zp-8C}`TzRH)1xf;6*-}El4?4XI-Xhsle9+oJQsUBqoW#^&ghAv67={j*4<1jPj{P$ zIDr6CGF`q6h2QoaXzt$}qC+VyB|4 zefVB-^_@qVO1R8A26Mq3z(nyKeGU&)fHnj@l?TsCPxfU1qj)vTU4cu0oY1sYaUT^t z#rLVd@&=XDd}C;CD7!hkvz>iJh%#Fu8Jid(FfK^Zcs$MSCpp(k3{ogD&pN?<0q zWKl3XDUk27*SJEchNi{O@5AXmi)cSGqvQ^OiUMsvN;2Ze+0Q%@r6*hMJ9gh&Wrvrp|q867uzdL;ofRlIb>VQYr)yNNs0JE76zGt`kXSi^Tfe2@kB7t&D2O_a* zu&~lohnB=HD>D0w=nY`tpOJ;SYP#dqm8`CNE3a)8S)aejwD_(K>;}Mc(n22LkR2c& zv^r;SeAWA%Yduc%)?wGLTy-Y)sCVhz$1+65c1ijHk7ZvOEs4p>>*a12$-D)c3}hzI zl8<>Z?)+-*zMClj{hP4qh3eltAg{rOi^ff!QOMtTy+x8JH@nRT&z!Z=c`_*Z`k|ye zc?3l~X*A*IAZ^XNXig?JKXn`CVj4zTqQw ztIe*NEkeo7D@0te*fx6h$hY_8FLlhoHs7E9uMtz^t|l*!jvUXt5B`E|uKQn!(h(_| z^N}af@gZL5*~BV^oaJuX4m_6ipwp}#VIVipwFx}4bPI&W@S7|td<-PdD&!^srWi0F zO zJirl{=Gc_XsBv18!|7St_P@vIsLTT}E-@fDUo+tQleZ=&q}D9>(i6Y_vn*k?#K-PU z(rgi=fvU8do`^V}01}Z(O!f2$ul9`2Ymap1kHFW+gtP*IR7dAJEGeEhc%OHy)u+arWnD8&oMy?!NzQF=7}=##m;gvgAoZNoY)r zv{A{@BA&8DCe&ywX{=)lEhHk5cTD=I)(6kcO@0)KlxnX9ML53DQq2KBVjk88g8ZHMrmxO`Te-&y2kyR^ z)Ko(OF&MT$IiVMUW0nd6>|D+_sxVXJbzO0|1iPZ+XM!hUPo{u$Rqq_%a@1QGJ1eLl zY{jzxgq|C>u>7YaBqaGx^!@)oe!$_c!&#ESYPJRT&Ocsg-MAVx5j-B)|Dy48`Z@sq z7Iyf5un*Br*(+Z+?6e)=K9Myd)^7usOH2&T=cW5?nZ4;{0RPHv-qAbH`FxnRETu=t zPJNZM&OqqaZYO(l@WSl8P?#XMXvq|sA z;#ik2TC*~y&(xqkM7xiI zrH(a&+|peFPX0ntaZrwn!|A`XR~CzYODK6U%UiQW7HsQGD`^H?{&rlCCTKUW3gnpv z9H@Z|<@62sxxwV^^s4mVj`x9N4F2}5&V5{%tBbyd8Jb&TYNZFpshh<2IU zouJo%C+6Z+V=`7qtS;|EE_K%7d_i%m)yAmr%VL)=fhc#uymtrTD)w)5RHV3~y=&kf zkRxn&pJt7`c4L)P+m|XIzPVqZt|+FGHO7`-Uwm9amwuuk6iZ)uD_WW6&ccH8~6-Ht4(@0 ztQup@D%E{eOgB+=Z%DrnUw%JA8m6t9n3sl;ozoJRDlIT9&%tDPIkOdFA!lubB`3aC zST8OBcdBZk;H9&7cIu2qYT6v+GFNKs+9i$9LHJ@~rC*gSFNYDzNs;}RoM%^&*LJkv z1H3jL*|h(xz4SRQ`-`@`6LC0s0xos0;gJ%^sYBNF8+{wFGYE&De`Y}u%hJ{W>X%3) zWh4D7yKi5W8l$8b_+%yVCxjKo<+6yj88IYlL#0a)V*oBy9Rzmtu}|L9D>jKTIpW3c z#fI_D28X!Jfy`V(awZWqgtx`=MktVS;i(|#IQfbS2U$V37J?ecuI31R(c7X2JoV3& z4@9ce)@3;??S3Bkr9|hMZdhqOppt5P_HQHvBp;!iS>Bq#yBMr}881@_8Dj7ReU<|P zt{Bhm(H=FOjIi^VEK6;y8oB=k3>@*>%#vTiVxFf%-uKEcYP z&wu+Y044sm(ED6w^}Y!!q@7s`s_#M3#ukuG^bM~aaw{$Vs9ox*DGXa0(O-S&*S4^d zW~Xi7PejPel|mZ8_@ldgd)tBj zJPVbsIX`sOBK6Fh+-KcMCdm_bUyr=vizvbIN<;Xq!)~DhZ6J6@um&Iuqy@tClU0#d zG^iXjlA6Q*SqD(`=+9)41>I?ofSjx4P})#AT912W$tXb-(>Wry()tdJ+?9|Z=mm$< z-Ut8i@(fDR_q07}#L~K*Hn=?De&{&hX7kl_d>U|lVl}$DEz93|-^BAT#=4s1- zA9IEHntSw_lvD9KFK%i-RhCf~b&Q;WBd7yxUB5vXm05?N#PMbs(-ZGpQ44cdrh6EgKrAshoSDHW z?++kYP2N8m7Urff)H3_p;Oa{cPY=7Sd5Tm;{_B`w97H@Novz-F{^ge0Zr&<0Y^6&4unRAxBo(Dx4$m;y}TP`KSiGSLiZ zXcc^n_%cHO=NTRt!0*{w1U5C0Y%mQRH2*WEf4@ZL64dk(tw0`}a+W_F7Dowr zCs55bDK>-yq2NHk(eT& znoGac6ALwUrSJKJdS_L5aK!O`^=8b*t>&|F-K7;_Yi#bhqcY`2q7>}oRod0wtb1JyEPT~^7nZ-KxBAF^Bx?3lWLCSpi)hCYGcfoVRmOM4 zV0KfsMZ(sq;$v=|#r~YLZy(m3?Ja)=A_JO008TeBw05|D=`57Jb7Wjf%JA+c5C*hE z7u11-?2ojPQ_oXh?HF^PB3G*vfhlA2Rj^Yq|6PiXV-?nGm@j6ni{5txnoILHM2CxI zolcC^zM3ByPR&<~i>iFU9oAb3=BY9S)~VlVXp&92Qpy!QE_}^n1l>E=UHUBebkMNK zNMR3AaM#3%!Ekg^BpcE;I8v|V_(qUwRf&KSjC?V;ObKpsH(S3JOHCwaHHfUF?pClk z({v+0BeH19592v(Ee4kQMks97;Jx|)Sq}cW{B1-RGEQpqZuQk+BzK8W$xIoDLxqP{ zZge$UW}P(Gl9XX;JSPxtdh)P~cX`ph=B_G=wGQADZrG>1-Q8qdU}SD8*B%D3j$wdx zbRlAWb^I9_m|lZdzkEhFbUIB(uCw+j@k0>{EmFbTt}>-)2D@k7i@<0MQ00qFqI?k_ z`kb4E14Mw#Rd%5@qev{G>5aG6K8It2&OR<5SRZZct$9gj-Qa%D?HFt#5A%91JF*nQ zlW(F3d&o;v;8Q>o$qhjQmrMK_HJO}+b;3N!d<9><3#)u}n^mj@+ce9xfQC%MvQy=l zc9yEaT0D$N@6NE=WtNU5xmpH6LzNRb5$ZD!4UKZl$}2wR+KyG{5alnkO9#HHn`7UVFo*x0MeQtS2!?1DUucd!z1mN6H;=FA{1ug`z!Zt~!R?{ykX zJRi?Q5qQI}Xrjr>aP6CB!>|wqn`Zi()V-fPY=GEi0H(*$n3hxrh#8x4BVN=VGgz$k znDctgOv^5w_~pm$uzEqVBi47^3vv5e1c^Xow@T-d<0U`v&ju%M0$v&Q4c6NdCzz4w zxN!exaeJx(G+D1lHOk;*28Z0aP5t}w3SKMD)3n+#rx%)HAvUe+k>wPAD`EhopEyM_ zjQ{+VeYwZ%nwk1Z%R}Lkvv=jp7PJ*;DdSP?l>pAnRj|cp3jC?|I+2M?4*lQoal-Ty z1M40}q90V?_ShVL@;?+qJ64!5DYp&?%AeZgH(ocwL%gnSeJbD;rLIOmyJQuDUtbz( zP_8x9Q4E@^j%Sotb?N6%0Cl$68r?YMoY`L3^skoqLl3Dd1Ixm8b$?;Na!WQMA*I_N zjxKJ@Mq>bUg4a=~X2HwS9|VLA#83ji3ktE(DQMPyQaZ$zOu&bO?pCP6#*m`P@(pz^ vS;aLFLYt!+`l|EwuQ&iPHl?s9r Date: Mon, 18 Sep 2017 17:59:20 +0800 Subject: [PATCH 2/2] change required cmake version to 2.8.10 --- be/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt index 0d81c21613e56a..78a7779f3b485c 100644 --- a/be/CMakeLists.txt +++ b/be/CMakeLists.txt @@ -13,7 +13,7 @@ # specific language governing permissions and limitations # under the License. -cmake_minimum_required(VERSION 2.6.0) +cmake_minimum_required(VERSION 2.8.10) project(palo) # Set dirs