From fd1ff942910cc3e877516dc9a9f6e7aab0271aa9 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 4 Oct 2024 18:13:40 -0700 Subject: [PATCH] Remove dag.run() method --- airflow/executors/executor_loader.py | 2 + airflow/models/dag.py | 83 +------- .../contributors_quick_start_pycharm.rst | 27 +-- .../contributors_quick_start_vscode.rst | 3 +- .../images/pycharm_add_configuration.png | Bin 97805 -> 0 bytes .../images/pycharm_add_env_variable.png | Bin 86158 -> 0 bytes contributing-docs/testing/dag_testing.rst | 15 +- dev/tests_common/test_utils/system_tests.py | 7 +- .../test_utils/system_tests_class.py | 26 --- docs/apache-airflow/core-concepts/debug.rst | 13 +- .../example_dags/example_display_video.py | 4 +- .../cloud/operators/test_dataprep_system.py | 8 +- .../cloud/operators/test_datastore_system.py | 4 +- .../test_facebook_ads_to_gcs_system.py | 6 +- .../test_salesforce_to_gcs_system.py | 7 +- .../operators/test_display_video_system.py | 15 +- .../google/cloud/dataprep/example_dataprep.py | 1 + tests/cli/commands/test_dag_command.py | 2 +- tests/core/test_example_dags_system.py | 118 ++++++++++- tests/dags/test_future_start_date.py | 41 ++++ tests/dags/test_issue_1225.py | 149 ------------- tests/jobs/test_backfill_job.py | 30 --- tests/jobs/test_scheduler_job.py | 195 +++--------------- tests/models/test_dag.py | 19 -- tests/models/test_xcom_arg.py | 7 +- 25 files changed, 240 insertions(+), 542 deletions(-) delete mode 100644 contributing-docs/quick-start-ide/images/pycharm_add_configuration.png delete mode 100644 contributing-docs/quick-start-ide/images/pycharm_add_env_variable.png create mode 100644 tests/dags/test_future_start_date.py delete mode 100644 tests/dags/test_issue_1225.py diff --git a/airflow/executors/executor_loader.py b/airflow/executors/executor_loader.py index 1eeee1ff68a..ec79860918b 100644 --- a/airflow/executors/executor_loader.py +++ b/airflow/executors/executor_loader.py @@ -173,6 +173,8 @@ def set_default_executor(cls, executor: BaseExecutor) -> None: This is used in rare cases such as dag.run which allows, as a user convenience, to provide the executor by cli/argument instead of Airflow configuration + + todo: given comments above, is this needed anymore since DAG.run is removed? """ exec_class_name = executor.__class__.__qualname__ exec_name = ExecutorName(f"{executor.__module__}.{exec_class_name}") diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 2dc425daa05..5cc5cf44314 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -28,7 +28,6 @@ import sys import time import traceback -import warnings import weakref from collections import abc, defaultdict, deque from contextlib import ExitStack @@ -88,13 +87,11 @@ DuplicateTaskIdFound, FailStopDagInvalidTriggerRule, ParamValidationError, - RemovedInAirflow3Warning, TaskDeferred, TaskNotFound, UnknownExecutorException, ) from airflow.executors.executor_loader import ExecutorLoader -from airflow.jobs.job import run_job from airflow.models.abstractoperator import AbstractOperator, TaskStateChangeCallback from airflow.models.asset import ( AssetDagRunQueue, @@ -2296,84 +2293,8 @@ def _remove_task(self, task_id: str) -> None: self.task_count = len(self.task_dict) - def run( - self, - start_date=None, - end_date=None, - mark_success=False, - local=False, - donot_pickle=airflow_conf.getboolean("core", "donot_pickle"), - ignore_task_deps=False, - ignore_first_depends_on_past=True, - pool=None, - delay_on_limit_secs=1.0, - verbose=False, - conf=None, - rerun_failed_tasks=False, - run_backwards=False, - run_at_least_once=False, - continue_on_failures=False, - disable_retry=False, - ): - """ - Run the DAG. - - :param start_date: the start date of the range to run - :param end_date: the end date of the range to run - :param mark_success: True to mark jobs as succeeded without running them - :param local: True to run the tasks using the LocalExecutor - :param donot_pickle: True to avoid pickling DAG object and send to workers - :param ignore_task_deps: True to skip upstream tasks - :param ignore_first_depends_on_past: True to ignore depends_on_past - dependencies for the first set of tasks only - :param pool: Resource pool to use - :param delay_on_limit_secs: Time in seconds to wait before next attempt to run - dag run when max_active_runs limit has been reached - :param verbose: Make logging output more verbose - :param conf: user defined dictionary passed from CLI - :param rerun_failed_tasks: - :param run_backwards: - :param run_at_least_once: If true, always run the DAG at least once even - if no logical run exists within the time range. - """ - warnings.warn( - "`DAG.run()` is deprecated and will be removed in Airflow 3.0. Consider " - "using `DAG.test()` instead, or trigger your dag via API.", - RemovedInAirflow3Warning, - stacklevel=2, - ) - - from airflow.executors.executor_loader import ExecutorLoader - from airflow.jobs.backfill_job_runner import BackfillJobRunner - - if local: - from airflow.executors.local_executor import LocalExecutor - - ExecutorLoader.set_default_executor(LocalExecutor()) - - from airflow.jobs.job import Job - - job = Job() - job_runner = BackfillJobRunner( - job=job, - dag=self, - start_date=start_date, - end_date=end_date, - mark_success=mark_success, - donot_pickle=donot_pickle, - ignore_task_deps=ignore_task_deps, - ignore_first_depends_on_past=ignore_first_depends_on_past, - pool=pool, - delay_on_limit_secs=delay_on_limit_secs, - verbose=verbose, - conf=conf, - rerun_failed_tasks=rerun_failed_tasks, - run_backwards=run_backwards, - run_at_least_once=run_at_least_once, - continue_on_failures=continue_on_failures, - disable_retry=disable_retry, - ) - run_job(job=job, execute_callable=job_runner._execute) + def run(self, *args, **kwargs): + """Leaving this here to be removed in other PR for simpler review.""" def cli(self): """Exposes a CLI specific to this DAG.""" diff --git a/contributing-docs/quick-start-ide/contributors_quick_start_pycharm.rst b/contributing-docs/quick-start-ide/contributors_quick_start_pycharm.rst index d830496b272..4a3319ae97d 100644 --- a/contributing-docs/quick-start-ide/contributors_quick_start_pycharm.rst +++ b/contributing-docs/quick-start-ide/contributors_quick_start_pycharm.rst @@ -78,35 +78,14 @@ It requires "airflow-env" virtual environment configured locally. - Copy any example DAG present in the ``/airflow/example_dags`` directory to ``/files/dags/``. -- Add a ``__main__`` block at the end of your DAG file to make it runnable. It will run a ``back_fill`` job: +- Add a ``__main__`` block at the end of your DAG file to make it runnable: .. code-block:: python if __name__ == "__main__": - dag.clear() - dag.run() + dag.test() -- Add ``AIRFLOW__CORE__EXECUTOR=DebugExecutor`` to Environment variable of Run Configuration. - - - Click on Add configuration - - .. raw:: html - -
rC?5et>0eQMCHhh-*#s1Zm=_bk2vl7!-e{FaXn59Y2LeCEH#Qs-hpEQlTt@PzDd-Vfp=naaXzwcV(KZdV_EAf zF_@h^EsnO^J>)P4 z8_H6Azvy|<^C9Cnury&T5+V@syj5-~0VlBKjs0w4<#j*+q!uDT9$Yj9DL18r2i}nZ z-*>v)ckeM0@>QK(%@OCbm)HBI&lUJ$wbq}R_LV(MeTM`rfx3UY;c|AU0`pWX%<-z` z9>4i rXuM@#0%pb(E#)AR&*S6!KWw>^yG;~<9 zK1C53k$(@TrBBlDsf ku=QWYFG?ZZ#)?N63aw5lBN(=)YH0AhE6-p8eqc)>RG%(40t-rPC?YRRLZu!ah1 zsQrstM?8R5AqVy_G*Cm0 vWCzInc+T@x;Jer%uLD(WcfZF zr($UuEOhYUBMru8pX{T;YUoH+4#MVE1WR{A9jDpkX3rBzEAF!vNbAmVU}ZapO0&Ym zO3OSaO4kbs7WfV$Ll`0fhsZ#RwPJa>4Bb6Z=+DVidi!*LX|vK8Dn=4ZMk$y4`{bmu z`S86+_~mY$uMI<*aHkiJgp{9XuvI$xA4n5mbsjt9eSPxT5Y|bF@r_F`nXD&4tfZ^T z4cNAHI9MmBZB9La25M>Ri|_Mt?Ptg;MkG}ErL*Sbnpda zs7wufdDscZ>4`@Dsd6IK<<=~NSKQOan+Wo8qwOv(uk?CXvUOzX$ikp6b *b# zn^T@E2B3c*y5B08kbs~=g)OW1B`3e2LSeE1YiEv)%4e2>g?5Qd&LxUQI7vf&ZZaEL zMU xSdFsh{#aPvdzL%QPDFL2&Vx6?-CQ$CZS6Tb7W$4#ueJjIAcGbfX%ULZ65BM zi`QFRAH`>9!mMS6YAP}udz)g8DeNqsj#+YY3q=4!hfMBDUdL|h<;G{DxNl+YjgGuM z3^7am>(YqBkgB$^<6q{(=#OwnK?t~3bV6#%8sZqp>guMD{)t7g@*)eESZ_0{0!6<8 z4jLlfNy!iRvihQqMs#}ZtQNrikaTmB^To@jZkaT@$qO>)KsKm!H+7W+hKGx9n$62J z!n#nzFF$f+MEcz@9fV3~v?g23jf-Ravv?vXYl-LgPZ1Fjl!}>0t1Wd5;MG`2o3Z#c zvvkl=g931*{X-%O;4?F5O!M`)*IKUN#3>lp{4cgogmW<#`9orb*ao&Wm_j7zVtRIx z @e7H+Shc^nxu&76SJnALyQHZ*BK5RdAuSca^6yWVTiTg?5^ z7*hiUc#&=%p!`Ta-rrq}03u5PbwAp$bnLsJ(l&>OV`BjPvBNE30|D9Lc~KcrCDS)4 z97Vn1f*c`09``YD$Qo=|1W-+v7M55z9>;^S5jJ^#9$N+=#Pop-!r4Q* 7Lxy~78ROB)z0n*}?# zVL_?EcnF*A3n>jhRbf*poKscGf>AUx-7i0j)s*-d*kO>TxXjyMTMhW&5#CRj27EVN z>BjnZaZj~tO+q1Mo!r!-B1+!FTL6+94!|objKQW01(^^+De~@IB~r8ytp8f3II!!h zLHBoay@6IV$8uZ|!}Z>!Fa V^$L0R(e#>h2RFAsq3OcQG2yX~v-!$q zQL9Ajfr*W_k$Rc~ZI|O`HHj>K3}DWb@3fFPfy+quiK6Rj*TqD|O MI$hMA)yX48}KdA0FFL%KdlkASR6tXQwsu7lO-T4D~g z*w`_WdM_udY0d!gv)0vV_kB=ncyzeLrK5LH;^W ?Ta`M>$Rhv z`)@3*@;_m#&=p?E3}=tJIXAHHzAs2ADOpka<}i}6t265$NCHLI_9l_)r-=SwJK^!I&x%B?<6~tnQ&AU{mgCef3&xtQcV?SW zny+_aQ?)H07rJ=nSz 3{iYs4to7yB62;F#vJCMT)veWfBYw^4#!bYp@v$G)i0~yb>rIy9T zdvns@o*or+z*)20N7clTxUh)1F_*LFf8AUnbV`FI7jE!YL6_X2ew?Qc^9vfy8FzO0 z>B*{dkz5H2ZAG@kjIo*~QoFbCAE`bx71PAM5X?UD4nB){j|BhaYEp84`f87vxhLAz zyvAq$H5$ aVGJ8PaC?$JqI+BY!R`$WJ>$|C=BZ z7Vr=1{ptU=+y6@tj!#H1qW! Ukbd7hoEC1$z zjn$@&S#oB5L Q`zvt6AbR@RM)i}HmX>DBx9Ui7FSt*Y?ru4ilVrvq6(cHdO ztk?ZlTOQJx)?cEx9~89TTVHZm^#As l$O_qZ@xuAI6lY#W dN!+THG#g(dfBH-c71ek{1V#BGr9}oVsgoIGf#1H5T$&A zZF)n8Y3!r~xa Lv_VJ2RHZW*sE2vNn?6B-@m9;o$g*Um!uV|>5F)Ove;RaxL3{i z2RQy}oz5n(A6M 7{{<#|d}ilXj1it#v`rM_Mu$aUc*L#-+(qf48IOsh^nlT4w* z@toU+KQ9ZT?m90~gVmIMb7@(cwp~=?-hzsZ`Ib62M*htDIsaa>)JIc~m# zQlE*>uw}{8Ek1vtExtk)^HWkKL|4_z<=R<;FN*{PFd@r87*993THus=En-rKuZ-Zq z_gHBG)U?HU64n4_c5qZIn8>D2WxF*+R(6 BxDtsBn*Zax$~Ck|>tB~}E|UrKF` zXl_C%AOir(qvtNVZutWB_;Xg4z+vt94{5|2hnru<@A!(C{);g&){zw%c}Qre^K;+^ ze$gICbK!>sg8^4prujKmhZ7E$N2qVMecD)V-bU3)kvWx_B~DG$a9d(32dYkH>As*~ z{77&0XM(Q_K0@XVVbw@ErgnCD`UWj`5w)lXr+xAEn4v%B*HS?eBkoTrVXXt-F*xrI zljYi#)NhOS@2fib@}?+dcEHHb=v{O-qf6XAQ`aQYHquFY2D5W>!%~vE1C#S!`_pw* zmJp?O-c}W+Jm=n~&v`*Vkd`ge zi=JBlG!3z$F#V;AF zwL}*lpr^;y_=g^^wZhZ2oeO~fP^n^BD2wQV;=zG+X_b}9g2Q&LD#BbO3c@!uanlfh zAn9OuxL@7v*ZLmnFi7D$K1q4hklu|8N`uOTso2f*m5)U?_UsD_bDU0BY1NGSbTu9Y z-wd9uU(0g>j%^}c8eAf*pEbo1bn#QUUTzimI;s@rIIGXb(NNnty@=JsZShwICVmB1 z4b055Hy(0r?u~%MRzFKfFpe}If~?279tWWP;5r!;C;HavUY7?c>@cT;S{SXqPbcgo zuyt!FTihTzOiy;}rz!Xn-kNoW8GzdFWyFvO*PE}+nw7N~KZoNs+8iecaG?*KVB*$^ zevYGgxtdEZl$$Osmfw1~6TT(TU{s%s4*|Y>D)v@+=c00AJrK{Q8 zmrGeLXq>xuxP2suupZ^LTe}ONZYoibf-%e1`B_pK*KfN@QfqU2dL9B<)VNYh f55=4$PUSiJI6C^FDn&1VXwG1ymDlef@+m$`Bs zoUj_jpdkVJs5fp|mh>UkN>3$~@fxYV3`Xfaeu_) 2S^1jxNK1+@arDt8ymBCz?!>=UhIbf`I$)_#B2fADlO6b3z%Y zb36)y71hME1fBUu%n#a2Y`D~Z{OpgPx|lTKE9W`CDe(H1(U~R)iU+0<+)s?V7)w%v zkTdr3zh5`9cxpcYPvr+yog}N8%>^B%u!($V1O 88EYlR+ x4>uFMn= z`xLHW H8) *pv;OE&%cOR)o@f19`5JEYFjhcW)}fu7+|gw~=K%cR`rrk}0r&~srcyVA z`(BnflIIGSNod?B8lmE(8^p%A%I8(qZV&?tNN~A-5*^+M2|$(OCTX}h*za$~bP)`^ zs3wL{>>)ywqM<+V<(L}}D);#CqGioLv?8=7hR|!W9n8MKqnOfEUA0j8l6ty6baHlG zMR4%KyP76-rt9^raZ^s+8IW0i*<{-v=B<}3WV(*%Gqle%tQhXU39C$Ya&O_5I~0G3 zqxo10<-f1}QX}e68`i-3RP6DK@x@xZXl4f&m;7f<^=#ZC-R2!XEo%PCc&o3g1uqsL zndzrx^=Wh&M+ln!ApxU{6qO0pyw62 wgA}7y%o|~YG%rt{FT;Jz;AFY|h?`53 zkwl=C!tVQPd?*9;Wz}JEHl>cjxlSqJ^v(u2_S@E34*j@N@r{EkHE#PyL`|cG_3?E& zT)cAeg8n{@>Wmfc+SD~_;^!dhj)n!Qhb!?NLfljD?Ow#ok6s !(@`WYwjOCWEiQ(@qX&`L`CS zG|{{AHOlKoZ`@8Ou3ev2F{8rG^L8QX4WB4C*Jyi63|=`-8&lP 215eU&VaPK76L#a3q-W#q* zKoeN7xnB?K_oyNz0vY0KxuNBi*jwwHlS!^x;qI$rE7zs4p(elOED(P6Qew=z zT|nIhbzY~}q7fdKFl-H;Vy)0*D9o?{gr!d8Y4&a^w3ZzvCm)CuOj1#BHwK1Lks>jq zh~6XOhW*_O^mOG%%bvHXce+6`r&uBLXtU(0z&kH`>7BK=L0$3rR^`pUs!81CVn;Yz z GR5duUNbgP+(4q{qDwAFhT+u7589iYUe|3IWMD z9SArXO0sO4sWni2Vxv=SljU64eJ;kgU^5xJu$>#getNI5T+J_@!Z{y-=3K+}+v5D{ zxU)qGf(-=_ qFp7i2So9s-bBJ@IeDdt+cSi2)AnY5EoMUHAWieM3|5-O@1_4D> z)P+h`F`-$WU0qE}>__WEtKFeO@d4Ok@*9je30PF*lW5IJT(r}kLM1_;dGR9Jt5Izc z_9wrMSlw6}Ix25Lh9M^z&pS0)v*!3ofIsotdXuncS4nBfd&mGGbPPoeM$QTt__4J^ z1)D6x3+H2aM&6`s4@qNMEY4d~w5f->#LkuM7F`AbPj{1Amz$Cnr`PeAq7XHnyJlwD zOnw<`6Clr{61Aa_Cc_L%ahKEB*vvvW%mk4*lE;ZY$7rEjSOi94xp2YROj2x#*KH<> zvz_^%^#&Pvb1-bFv$Ah^A>hA8 *ubIY4wFGt1{GT6e_8A?zAX9#@5P04)t-C)- zsCl|dmCOIeepJ9CpjpuC%CdH4&HUKBDBH3m|C6Z3>^xNA635ojvkfsuW`);fG!;79 zF6kJdeusHPR5SXq<1J|q|12Q-jN{{WvRSs#YJNJ|S@u<6^*VlbQi-x;dNVdUP0h20 zh@%RUC*yc|O7&Q2@f+>Z)&TC0_mBL7O}}l(!KLl(jv~5}Y1BSaRAyAQkwBh%H#1T5 zB*9EKyqCvOdHzZ#P3!3Y)B=oKm5~{2OzXtX&Thu48ToEJCcGAV6ojnjM(gJX!3M>? zPs^qqrnOMAC@+Ek O^zI<4*O#B*$@u7-U=p34AiY-ct9@Ws>F3?f(N7No z SB(KEKYl237ZH`Zu3!z41Lz)6?6n(qXA2|=(Hxgf zI&F_A;zikfd*Zo^b&jtJnir5-YqPUce_L8@aQqX~dqpd)sHj1Y!Pj(^mR^L1_i5a2 ziJ_E=CH|75Bm`x#zhi?iG{NcuT3xrWD;rS}%Rqi=UBqCFI!0rROOdBwZ`O+)9vZe- zd*F0!+YlwN>_vTQKiPM>&Ln+2ke2fujEbLYG$5LJl5O)7c%H2HLi?B)<>%J)m9|YY z%Xb(&KcUzF6Zw;gsLeC#@bEDj@h}hLmtbTGJ0X|f(4s&_M~8P?a~jRuLh>Y-KR3kl zszz{upkoaF1G>qbGRs1 z^hGMV`wp|M@v)Z}Mn zJ?v>Wluyx;dW%2luZQG7l-L &i!%$XpV^aN`>u6&wiD?Te;g<%}<50d8Zd&{lA-vmz!$y @>4rtUWZbi0+3HtwbgEnK z>{*Tqdfap1q#6$jk%{x>&j`btK8W$w(bVLDf~zk3VWYtV(-Jp!u6w$J3(Juk=Wx{d z*7DSardu&Y?f}<*$I5^N_BYRbs=-pwCH-PCr>U##TU1yz)62YINUz0;U5HoB4o_>W z*DZri+bgPHch;4x&*!A=+m8or*8P!`atq`yj={Y#0=y>mu7{bOsovN&ceO0tuz( }4Y80@c!3>;`vfj^bD(*Ap$2guzzg4^ZBBZ+X5r?zO+1)esaEu}MK` z|6s`oj W~RfGMZa)( z`H@%jO)*zHpIZ0&%uvt}44^g%oP{vC>-h)g*M7jpc;%62)b8BdE u-MMtl6O{e7M5JuBO@!6Bf#57hR)BVCK!| zw4lxEO!dUJIjdzEhXqLGar7^lFEA>Usc~2cnv9KO#T{$?w|Fdv5nfqCBe>%1wdkpb zEXm<~<|O#VaNy^>5WElUeotVxYh9VIucreIk(=#5l~aO&ykyyCKP%GFAFP=n{ChW~ z+S;x!rn0uPBMBeYON3ZUXE$|1RTV1N(#lP3`(}toR7j&wdC0GT1lA*W5j{_2M84j^ z>JQriO?(H6Ty27v5>~@ GK=YK|RMM_jY>YT#s4fp_QEpsq`};8$zPl=DwL+*|lA zC)i#S $$XYMw66-jF*_#mVuhrM~^i04cC^Tv3`4(|72Tcc=WP(h96zN+L38 zaLQX<#7SWA>x-ag5cWJG6^vR?Wkn@A)W7Ck19pzm;Ep4fc5VdHv__kT__TeC-YVlx zooK-kfwl3Rofo7699hN_uDNng=TxP)YBV+FwCQes>6RiL-hq{Q`N#~r!_md_y`FMU zW=&^Q;Zx<|mJaETwWW1(;t%;5=dw}k*6r>aYEb&vl{=s4O6Fmf*%5)|Df5;d^Yc>B zS;^#5m`zwJ?fP=^)oSW0&0a-elf8 ej4*X-6(89pBD~ z$K8(u=HE!xtS4r3Ys+rWLa LPB2#k86`$ z|I+#JEfl3q%lkNWecE#RvC(4nxqMmGJY$0-cpd7j(#pcp@;(MS+xaGXym4%4w YrWoGD5@PV>4>FJc?44@XS?Oo1dSUc*_7#9`DF-g7wrnD5kENbEaFq zO}`uYra9Kxo-NBbj?1@a68bnQcyk15K08z}Q&ayI5&!KoV-#X<(^y9B`C8lZOs?PQ zKK;$X9BqJ*5soWlKz3;<4L5g7dwV+rNKajz-0of^r#e5MjFpx3)29vaOp#)ifc)E9 zjgHb$Q{P7<*)8-R6Mza9M)?&N^-0a?_1L_u87W E{#7z;W&nFoGA2<`ky zZb*rlS$PvGtI%e;@d`G`8C8hQT>Zc?#|T#uP_OHBKmQDBb;-%e@$m4#``g|=ynTJE zEb= !c8~LsK&9gCzTn0A6K++~-sWx0%*4V~qOZ)> zg$6QdkaC6F$nC@zs|nt5HIJ^Nx5}knB|~8^t0F>(mK%(`por9>6ZYx$-UEw8?0;1J zd0~>9vMFn@u&-}@g^}@^;HSF%c|#3v@XgbScW1!F@2XvHFsqCz;Ws@2U$^1*&crcV zrFXsuBAH7Il4x|-s-8}96BRwD`Gs_tUJmg%jpmmSNE3o)^XoldSRV~|J;M_c6BCn? z4#8mX;bEMKtW|k=xwVnekL2XLH`xKQsfMSgRkXFWD@8H|d>$OgWo2a#UrtX>)*{0A zeV?@YJC2tdtc-qtH9w>r>r&aNdph`$Qh9Jagdszmy=eDBoHD9$G90UO-o64;V%lRf zA+vA?ruxk%Gy|8Fh3xnITfaY$J5!a8O@7<+;lA$>=e?b`{Z`bPRjPhZI1XbJA^)%m z%UjLY$;O5 3A*$a>tA{%TS~NbVYJ24GT}nbC~n{YGFi zg8~px;Nes@HQgyuElQT#ZFbli9UTQ^+kQz&OLIGz9U2|Ae%lpA3K=Y%oSLetRx$x7 zGh=TO^Y!9Q8{56S@9gZfS+2K`4hOZlRWX40OouUH10evdt*sCc5T2f%lF@`HC@3y& zZb{GJ@cPU0{rk#uAn17P%43-is_e$6&@P&$R5lffh|XbV)4txv&xZjL&`SJy;VZ&3 z50CbT{V#`j+D$|)yvlM&-$?L;Fq`HT(|R}Tom*`ce$-DyQa#@6f;(2U4-OB~8c@=! zA7m_TK#Tl*qy8fs`&W0Y8Ib-LaR#^!>=m6zB`|=gEz{!$ZhZR3mL_s i` zy2QpZRG_1evZf|2DS-`vsP1Ww>_n%hQ(s`9i;JhtL^dG%Vd+I%6l!G!@6ht_!_q@l z*n1UBn6GCZQK|TExuhM%&ZmtZ;wd?YlrAnDa|`UXhg^ &I?IUmfYHG(5gnabC9D<)>#Qi7ipXpC dI;T*XJ&cY|}gESM++V@_J6}6^&q*xC)w+yB 5xZ^!@@$jU$$$VarkSWbS!hX3cdMo9+ z2e!X~!XDo5nT^*xNuj5QrzhWI`fHaf`sM44hCK%a`Z~#cjy5XW1$Z|rQ6SK`EsE5) zGkxuDK;eCnO5V9l`1w`~ow3qfhlC>+K5Et00nTUKg!|^j+y1)vQjPVyafuP<&6eRN zMlN&8vxXH$FFMG7HltD*x;PQv*>{4wYr9Ly<@ul!!H0uq+3w-RUZ+Q{kCs$rUvwX$ zKp$x>r-7%A$a{Y44-xy#?t80lZ|ZMYd!TBQ8*x}RaHnNiM&op63$;fS?iGhojBLO= z+m+FfQ;3*WFuK!*qU9~1e#c>o_;rPK3>RAXUH-1-o)zNFjfTcsI_G7fzOTiNL*b%n zM%B1Zy&0B+pf!30ixl!*m)=}J1 GUL>=$L>do2McxGM#jg= zL`_UgY-}pJy8M8xxPUjK#u+?a(E6H|l5+p}xG+C2AtAw0^D{jiKeQ|j2C} ^d5+PlHH?z*HHoXBQ(#+{7M_zqr0V)cyTjaO;1uyZ*G#OR7@{#bY2 zAMm44Cj)Y&u4)y%PrzFPs<1ryF}l>E^k%5n#39?KJFW=)nYe)m)>$c*X?Tf_o`I86 zaD7q8;Im$-4)%$tSv1>BKC8&B^)j19u36egrz1QGan1iHQ3rtlkZWVM!g1b07pj&A zX~j(aM)<|Kdri>Ul?&RpOX96jtvUj6)VPUUkBR0jyIiR~9}e|uZ2-`)CwCMH+c7l~ z`6Do{Z3duQw}*PkrmtxkQ9MMmkr*bxtPjPDcEi&s96L>JHd;*iTF)mxro4{{1uF=> zomMY?6}28CX|%N+gml3?yH@Rwkpl9?fEn(O(;c5*mt@USSVr{Ims`DV&U(GOHu83n zXG=U?oey7dF|G*g810Q9ff?^y&-H6g>)-w)pl(%1qIhJ$PeS6mcMa<9moH1Znt@92 zu$GyOACb#qt{Ye>!{TeR-*5haZ{mU-$bh1Y$z~#1`5YrX(#dQX&&S(&|NEO4_~QU~ ze0nOc=P&-p(D25)ebxJO@LTW$*4>!?i7=CbQ#;@FNV$8Y2M^M)xkCdDY22?>^6gwI zPXZI>nLmvdsa6s#@;3R!5^s;P=@xDiTBy=BEFly|>7LzHIpf0IMq3+jy2ekBO?t2h zY{l~=$RU8wW2cSdnUV^SOCJ^nkJmk!AfzfgsHyW<-YcM^e2vjjpZFPd5JGaohkbQv z4CT+nb0M5im=*nuQ?CX1>k;NKD%kWHglx@vF`SqmCGzJU+tH%p`!YL~T&CTgsrBd7 z1Q<`SK2tn#ElJbAo~JyA{QAAR2t;vmLaY3S&CDNy%#!3DDgK7W=#ze$EsuBHWh=F+ z3@3|RwX%BZ+a(UWVlLkMExg|#oRY>lG)sq&U%d{iMB!*9#p!Kz-2#3(Cs*GW>GQm0 zitAA6npd+&W7F2*ZkE!w*e=ox%y5~*h={bCZbvg_3F6XC{|{kr6%|L*wSf*0BxrDl z5CQ~ucL?t8?(VJ$?gV#t_rVFlg1fsr1a~<7+V`KUbNXT~n#FX_>Z Ct07)Q4-9a8Aj^}c~ ztS{h$Ht%{&c-(@x+$6K+v!idlh!et|nl#R~iU`{MK7{>nG3sTjtxXWn9Xok=c6LTW zLgL`y@a@|-8A^sPU*L}X*z9+Q63?kLbYB5+G|&;%)G#P1O`BFb)6vnvfWnBQaPaYq z3kzp?1z;@DvkiDHhj%*(jChgDc|7N56uLx`0<(~5aXBT2G+l#tn>#|xIN+|XaM C{aQ+tmNSN z2!(m !E-?VhxqpR?vvgWdaXS=S1qS0smv|-Rk4|$(!Jj&=N9aj^+FSh~cp#^1CXA7%p z57xhLaDM1{k%vy7uaBFyxt-)7{?jz&&MCSaZ(OiM5m|X!q9dMd=qF=Jp}f4 ZclmH*Bqu7 zd=>0DE;>bcjbwkK+t~}2#RGlx%y2XJoAzakFj#b(Td|W~SDkfQ@*Hn{i#il I(=Y$;^s6JCFY@^G@J(H8Q6%{m`oVcKqlar&S$w7=!gg? z5EK-Yj?OBUDh(M~(D;76dd0Zy)C3FCpG`Zxuq?X+e(UVRvOhL2tau)Lxwd;#s##D( z-Wo6 UKiBzW_$@p k|GCKtrdMyvT>Jw54}4yLv0KzpB; zx3yi#OhrWn5|EIPaDA`H9n+^k)$HXyJ&v$f&6D;~mBb$ldk0H$_3)cKb3<*T?e^B` ze!*3hbZ{J3$ndTsr{zh^oan$RoCG#hDYO3OWj7(g$|IdNSH?Jqiv7ei@&k^lPMap9 zV8-nq16jeD4h~e%M5Ej%BDH=oUb;d!lku}m*23ooU0L66S+e=*?GqS%4UP`-`BD)P zrC6Emj<)OJg@mw3E*{L*DW<}r_qBCBLfc=U-aulZE%+*u0bkB+jSJUv_|h0R6?Q?t z4!BWwMRg!Q)7vGWLM9UWK9McloLLFa&j1J94Np7mL9^Mo6$A(_K(MRgK;de9N5O9o znnsUxA=cV+Ca@?o0M)q2WTsM$BI@Ln!`W7~s$fpRfrx)BIn{%kMt5eXRl=ep)T Hg&8F?EL34GCFN6EplSiiM+rz7| zfoDsOs&x5$j6vc^wR)e|TRHEg0y{(G6RcPLRsI=SXn}H@#HcJ@gPk2@wGwmhXsYC= z!wdK{Tq9C5kOvj%a$8xr$13bYd67=Xv(P(%WZp<`!3LC*f}k$c@V}*#^5v8$oZf87 z(>C&IbRY(6SVTvXxESIv_j-Q)xm9k7z~P+kTVkEqq2}_7m6bN?x(R$n=VgQZrCB{y z^+=rQpiGae#|RX9YH=euIc9M=32gSH;onFd7m gwww!o!bV_2lJa|FTDB>d5~gf6xVx)XdD)-n;^&7@+tCz&AGDp8IMN7DS|( zTpHfWgAQK&KogHeP7xL5V}k3A;ks4ELRvsTceDAr@I7!8<(3?y!?Mu%iXdh%;Pmmj z@o;I%l{_>rFAtqUmSu!{SW5`FUP#EmfLlvzE2IEW*4CGsSz5w_CMPF}S9DcWRKSIi zVG^YL9ba1B4yLrO+FXe)deUs3VzM|KcCG`w;qguqs^9>!7!_LJC>?L!C##2riPab+ zF9O|%0h+GW@@EXd;)y`)a=LS%E%#yfBm5H9>UE_%L(SOjcubT-;V=aM?$_3oB2;NM zYZ#MV1upS{p~cVXt;1y>r(8(Q$Mk4r1p`!(qod {whgF zo!z>w(Xx $Mbe3 (R?B%K}G`8jNMg4llxYB5F&` zKrK|$;!vHZctUMY8v>05%*FX4@`Mh|iaR(9eSf6xTmE(kkpy fkLo`yfGC&%Ykz+)0yo&3G pXQ|NHmv05Jd@!MOXtCpi|` zcF0Drp#y-4C@Lz}J4IK12Q&-9D(-K;X06LCkPrnXY57lT122>8icrnhen>Du8*c8w zdcApHt63GKvnI?y0L@vtwuBi`jGDH#wvNurZBljDYpYJ1D*&mkwtF}!C@26pw1Wd< z%s@Q-rhM0w#%6m*2gFAjniBx@w6m)yD|_6}_5;99L|`G0j5JRE_|S%AGVl6=sfR>$ zb|w49JFmmM39f>$GSt*ue3WzPO~3)vXhboA1XL8Kji8Ja(67h_8b(lQWxfa`s9dM| z42IivH_&cTJb~fF>zRR^?r82|_!zkZ3smkiex}$t`pU&UAg&kF9qUb>*yMJU!up AXyOHfKqcZr8KZfKk~$6w8@zqtJJa_GGcp$%FJa)(!n9gi<56)*Sa zIdc05ABrgCM60TGIkq<3aU|`NLK-)?f+DPB{@Pu6BbUk@K8bbI5UVdNCrZ*ARGmQ zkD%{m_tm$dfpNoiPGxYkgf_97HWH(2B1`XcJd`9X+oe4W6V@(?J4J_7QbqCp=*)je z a2#177gP5Dip z=9(yKY0n~FR#2iY)CWgK#wMtipfZs&rEZ}-6kRK<+Uqf@;ZhQkiClluf4SMU!`{vj z=f1m$huTO*mpr_GwP@YcF6Zp*OhZpECnpCh(PoK*hi7SJ#a{cx$LFP02W|}w?j2IO zvaT)>Hny3*eok?*m^X5z=)r9|m-TGpOV=3pPMcR`qa!J4+%Q>bMg;*p%^D(P>nC2W zAcJG;l?=q{kWNRbTU*yjx0Dk$%l&v^$XW)jbFv26))v`V8E{duk(pU?X6DOAcR(vG zJ;;C56a|l^L?)f>?tFdUv1M~})39~{Fp!^}#j%(I5H>u>o3Cq;)&f8_PfrO^Q5~)i z6aWf>FI!JnH!CAUTSMbL2!LU6adGR0liR^QHYSBBFdt`K5F3=$t;w2%k~S$zzo3J* z8fqd?aa9U*^!+KJN`5hyCuO?hJnj4{%Z@P(|2A`Vyai+t$X^~T3uj}P@p~mfd-K1! zX0~~jhHhnlRhyJ9?(CrboOx+wX*o H}O-0|hrLb~$e=Yv%L?cvGk?sDI%a;b+? z>PLz1IYB}YNCn^Yu)VhW4!0-m_)0!>@p)WVP2?z o9o=IYZb=Q(&pLvw#M_)2;n_6i=IZoqr+V0=3Zi+gdn!K%E z3|B{F7$KoWU+5Mo$9TLx3FRE`8Vy@7VALIvlV!BdaX0a|N7@>i{pcrL!dGD2AE%Ib znbZm{5`GKvN`6=8NW3D4IlTDG?tRcFA67PyG!u`x!c~aGC|i7*4Wb|oNM1Vs%`(QD z=d<1k{|7pjqnQ>Y7DXG4(X|3MS*GH+2KU9w-Y<>2`0b)>9ZX11xu0G%q)lB&b)Z0H znf*xM5uVzoc~Uj_rE4V1fcO6{^_mDe&JPQXOryHxbh>`~OPlA~xE|Qis}-P+NNvt8 zpA3;tTXiJ;oDXSls_x>UJOeuMO*DOs9^9U`tk$?WWV$FnKQ3%i+$3XKygd_=GhCuL zTi=fL7lakVW_x-*I)9h{#hRE{n0BzZ`unnf7`4Oux}x9spe|AG4&~rrxwO34 4J{Q&xNb7Lc+ z;PZHWIGGQdJ}N6Eg@A$*6CGVuQL%INkd~I#*x2}Vcb?o~71HYj*?om%ukt>s4DqEl z^MZ*LQ%Ig9?v*q^rmw;Jyhyn7cWI 9M$0EaaSEk=N___gK==R(p*)s`>9Dnxv;4+)QMjF3aD)f0YWlo>YVy!Nz^vfCO#r z^hCT#gAep8hs!*CyvE-a+RmHF$M}&q*4s|rG5Ocr>3;wIkqSOEycuk6qi}3puA KfVQF6UGCopB9(rhH5LQIdo1O+Tv$MC%B^`y$+t7Byx zjo`zA37biAn;k^4k(oPQ4cRA7hD_%}4A%c>Nv!Gn;M=~LW pA1Up-|kPw{_iYu&l5iG5-pb(8b@YS|2~Dv4?~CN5}}*!RXvQ z2(`a6l{?*euv|ZuHjrTQCp*29%i&{%_0Nwo0~W*NX9U*+AI?VxyI<$hhE{5xBtJlc z81Q@#TCB(322P!9k#sp1dv<@881T6&s+!n7kJhVQCu^_Ij5{1ny(bA!Lqz=j=KYo1 z^>Q!@|5vjQ`}oB!WCFNGlcsvS>#TM=BIqRqLz;N&_h5U2msJMZ3JBzX&hA=pu~r%X zR9lQ>S3O)7**UAPJLYWfwE5iVwT74<#xUDG8_7n?MnNaq zn>ai|E@$(tNq~UkAdnRmg^`(l|NcEaIoeSn{}IgeIS$EL)1O`sFu$rD4?(NL;+W({ z=rv8oQ}lah?@dbw>0 7kwaKU0`}MK|o5&a=d3X`hMhRalS@*QJnx#!|!b{?};_i2u;`8s7 zjiDdap?vvWew$8Sc8z^CQ?9uo@QQq}>^_0mYs5vY9{vL%%FS+Kd3Ufq2q9GM#*Z=< z6tThRSmMIVDbv!PSJo=3^~v_6-H6S1buQzLb{*_Ehs-&OdUT-WdA|^ ME!)Pk(LdY{l9_xvBw%TMv5Sy;LUA^L1X(cY zhYq03GsDFAIs`i@m4O6~P#%~Q?CQ;LI%%cul1kiLRc-UR1}*gY7Mk6wN8Dz4{7@l4 z-|6(miQVnD=&9vmG~YZeW$>18O^9Die9JPdw)hPDiu&SDHRxGHUQDcZV0i<3{dU7h zlcbXE*4t 5OsP8reTEPc6_u9)w>; zru&_)rRtO!OGz)^X|!7%f>0+ sW+OLUB`eJ>~epo!Fxcp!STEdMmGG!Q%v z+PTX9fjN;o`HM%%HoDu1xC99U6XefA_|i9#O;6l}&-@Dy>fiVF8V1+fPWK=907l{m zey=dzDewD*vvnWWqm7m+LE%76h^~ yt?3GyA8E9>A`zI1PYRgBz7n~Pn4f*8E#I?Nvsirc?X=oO5!^GL zR@17YsHsf@go?a(%QsKH_j%N1L{v4GL;B-huoXHun9%45qaF&4y!a&{8E G`HZI8lF*GHA%MtLY629J{( z$yh%%2hW)r+I&kSs_$N|mWoK8bdOF=#RbV@09G-LyFAl(f;c!hHR=_6MDluidNf$& z$%pP|aaO=Q`G@>J6mxQN0-{Pj3h8s(uN8f}I`R^K+U|KEo?n|jVK6!NQ##@~b7OQ7 zbkaQ 8PA7628tKny z5C9SvCRw;;&mkx%C@U*l{Yqucw!E;=Y&Bn1O~hXA(4uW-w%@81D70Ce?OTSNDJ&$^ z`TBV5DMqiLsR?f*B(0tI<6D)u5_s8VqihYU>3>{55aTCti=T6(HPRHnLP#fv5UbXZBaf1M%_0)>q( zt8MLE%+mO7cW0|=&%5_tRoB@um12Fq9#4;M(KgPK!+_FCM_XIhA)`SZ0AXBRT{Sdt zWu_s&{rvee$E03f&-v-r1}4D3*m!0oqOYSBAQ}-65O_}8(J?VG)v9%U?*HU^p0!6u zM|X7i?2lzWc)vceB1%e0f$i4U^2K6YPnTxfpz24jQ}Zh^RdW;b%7?>;t#mVRB^_DG zzMJ|n! ueN01s>2d5IOVA8TwqxSU5v@FW0)RxoxlNxGAD3 zN`GG~S*fjAv%gvEMW+Abt@q@8n=d+4b?9gWe@$%mo9uU|l_DjSsbhJ#%yN0cT=he$ zGiz8J^8rI|IJVe*$-s>0>+kQUO`(fgv-0L!|ND6JsScI@^1Ky6j}r+a$18neecjJb z0)OkX#ld?5MB7TNI`|DdujMcDyDAJi5^1kEQGafem%X~N=7Ft#-CuUV`pkCCef;BY z8<4U3A;gbYrGj}hb5C&?yynX&o7~St{sz+zSoLew$;ikIYwrU}Ys)x}*Mee?wd&x9 zD8F^;>~H#I=&}r!RlbU7D=JQ@GHYa*M6S6-^^Q&?sm=;iCV$$0jk@2$rn8i+LCWB* z?Bkfe3OGU~_z$PYZg6i51i dC-XEqTH2_H2=uV8FfcIH9J2B{ z&1Ep22t6S5e^(Ho8Sapxl|{2i8yg*^rK3~*g0T>3gBZ~lC;l10Q5x6D^tU1>o*?}e z|4xDWOB;}ThbH*p@$X{+1Bf>8+}HyDt0MeII-rXp{ZGo#znwN*2T|`@hgbR&ftZ*W zIePN80B}aK5Doty<^EqZVu_9Z(a{mx@%y8jCBTeUd-_4BT0~JXE+PV<3Cc2lXlSSg z_3v-fg|CJSvCsH8leAHW_y2d*6b{lE)55|+1JeHscE|<4|9}1Vzph;kOa5<}^smHK z7sYhB0xPk}Z)t%aBVL#-rqNiRJ|A3GSH}b#jEuF-%{ZXXXkkFF(bmQtGhj8^mo#V$ z#8HCujf_aKB5H}(f2WpyqUVfZVVyT<`~zyp)9l`;(R(NtYHh7c8{FPnME(0xjZlXk;WJ zB4R^hqs =C7-m1)qpEzX+1K>qKLbgrqtN_ zkGGgH0slRD66I{wM0WYhUp$;`Jp%~X^coE35sa^ EaB1zVO=T@&hU)J^v^$#2 z;%pF#{wM^oqnz{MB?@mI2#jK2W4oO!FqSHF`99zOAQdZPk0#)Be7?U>EWKW+H3GW6 zQQji)&+zc@Xz)QtZPo0yMHXEA{KOz&5sHtCo5BC<1H#SMmrrK8vMk65U5SXisIz)| zTSZGrLswhz3Xkh689lAq)VWkXrmpVWY{Gig%*F%szvJM8r(8p)3hVt`r*N=GsTsB$ zUZd021ZFx|G*Xi=XL4jaR+`+E-ZKJVWrT!<0S%T|G{K;^>$Sw^IN)Z9IP5>Nu*@th zq>S&gAOAj@@L2Ek0sZkbV;2(@4Gsyx$Hm><-KE5Uzv%}k4z3cTrO^AuF+x@ R+Zh0(m<_kFK^lpkDFFR6IU3(r;Xen5>^>YJ=R;PTjIas!p{y(Fe@tATzx z-)~KP8@G07QMzUVa8#;9Dh7sFjrC+gX>w);2791y2F|Xnt&Q?Fb2+Q7yX!y$i_v_Q z4wV1Mi1ZB>P|SmVkB+K*!8nQn&N{icnAEDDw|g+!idF^|1#ficzoXEyayg)(ru6xc zykQ)1jQF>1z5`8H2(k(ggmRk#YlvlI)>otO@~+62;cIZzc7 LlyjX!` zZ4JK7dY?WvAF 2K|63ERVabW5&}x_lM~Z0398UD9_`pALR)qXUr?POBI< Ba z=8K0Gwor1e{2v;9Vw-53WxH^>&x I+>viRrcL-BVa$1$;eb?;pHA8O7IyQ<;AY z+b{8O^4uYQbeb>Kk|3ioicvsduX^vg0|EM7U6J)?wJIGYuFhBm@rT9JTK76#=JNRW z>E61I0=}{a(t6KVj| LVp zH=l?E%87h%3f-=2#=tGz-e9-As78QI>DsY^HtAoSebZ__{OX*fINa@b^5(0_FqQQ- z^~k0?FtnU*M!Voj!bDEi6%J(198VMy6O7Gj0V;$t;r#pFFChI!Jq!;kL2gmuAdich zh*q9THDQrT9@fFSw@o70DOVJgx0IsiKjzyx%SzQNdS@3~^J0d%6N6)#gk0$Fa--9a zWj+43f(X$DKztH{Ys8JJ;1T@Oi{@$|NCN$b4V?lIgfsBRR8S5}Me}u>#>Dp-u9Nsq z4KN~FoM7|VMt 1zOgUb4P4bSYubkV`FpKMDVR Fxr^f zV7KZlhWzs*z2I$*%9C6Ha+*1W#FHOytSO(t{Fnya%F+uug#V1w1q$2-o2Fu_bd%XQ zQH%ELit9H42GbFoTJ^}D;{Zi%?TqAPu;nb(&1$QI)2S%P{Ly=}b=>l|Q?6@$?B~To zsbm1@y1sS>I%$^`L=ZAEGVm(-Jv_|J#B?-OC^e08FN*NIQV0q0bFlx{%tB2=Fq#uS zSIp$Yo9~>_f^6dYb>Z>p(69N*l+`Zrt6TnUnTDsA2J(ZE61y(G56$%AA1-<#PiK2v zt8q$%L2wNf?V{rdO|cV)`*YR0ytkxbJ&AS%gF|AQ19Vt-X<*|j&hnkJSJvZVBzVS~ z7moY%A5-BaI#z6QM3v`g55lnI6NxC}#2xB8SG&)tNzX@-)K6^mc;1Yr`-!W9{W8>A z+w1lUxG0Jp{eB%&fwEkUF5CMZ+hx^R4^vqZGg{Umc`w;0PiJ}44I)~txbiKvrLC=U zY2o^ks{!PGWdpz4TAfbiNJAW|^|H^W{_I2OI9rOw2&ND7^pESF{gUOYy#O jk^um=n0s13sfyMA#o?HA`a|3VxUb5Fr0mkK5?QObt-M!eFFz0Rufz zlPwt@jb$>3c$6^tm)$JljR}
At3n4LVkMy>00FT;umn)gVp~1Z~xLs=={ikPK7qpIw3N(nT%9i|k zEJch}U)A$viFE0<9zu2{2}6Yj3k9Fe`h2aU5di?&BKq*yta18B;W`u)X(jR$YxW)c zUwAknKCI|=byeP8ahIOiso$#wPqWWQ)4Fbhytk@tvTsCH$%dSa z7x^lFbdF|_@;)B5$m@Yw63?9hnUCIsH3xUQ^UTr`{`8$Vy9e_oli>1Vq=(tIFcYha ztqQs;l=CbDQOKVSlgLoo#g)&T)>C^R-;Tbf!SHZ8R2c7$tzn4v;*cB&5Ka^69({^| zB&0O%r_q{pkBx~JJMAU*gqku#_L228L;kzVMPgoe5Q9sz%sR5(>z^w>K`S4;D!ch+ zm}7AONuvRB#gCY$mqI(%lGq_ZK=cL@5#O@|p)%tz9{q?@h>vQAh|cyjnEU(>u(3#C zH3hmdrCPPAW4kvO>ut9HP?rfzPb^hch5c7rbvQ}??yPZ`9sC3j`sh1QR({k~_&%;1 z^=SECI-6VZ@=()}WLDXaq)g$VHfm~TPonY=nG$z!;jF0tNtjsoV8+S=O2a?73FT?heF z)4fD$HNdwb9%qdNT}3uE!@xerpI*7aAOR6UqE?buH>@k2QCbmth4S+vz1uaW%Vz5M z#75)|%LNR7@=gFJ^4X)$7yLdOk&MiTJrmK>_l6)XFNKY8$zozsqz@y7=L(0!!I|xQ zXo53lGc$y^5rr744>@zlckFG$tm7RCJ|NGU7_0m`iLKT;%B$`VOpmd8>|L^1wYGN& zqP@|ansrg88}=$|$X>KlAap!rX2TIuf`l-N9P>hKM$z|RG&3sh=M~;1lT9((7-7Al zzJnN_eQ6HA+iz0?6}gXd>e`ky(NUG<*Ve=M9g+rkcMT j^Oe5kh?h%3iQ2DU z<$**54DlIC6deuqj}%|g999oPF~V8i7xhqdc@ux}Nh6L;>;Lq<0@VUd*+IeK6m=%% z2aqE$V_^kUl>l{4A)5j6|M^n@py&uxh{>s`%kkN5feL$`Znr|$B#$iw=>BBo (=25LLDa(Lwgnv{hs=E^M3Y?>I5^N$joDlDbAGXU z;5Whlv;bvM%WX$|rMXPoGje+L@SJ}1jXB|km3>rD^`2wUfdaw-^2(}IAdW^i%puQl zyugJC0fF^V>H>kKojTU{xg?2~hb6xZ%f1<+$9?n5Sjy!!n~Vuqre(WGZ{7hD^SjQK z6f;5QwYC;*V&}nMAy7_<>)jpBdqMK@$~ M<-vjgXsVjp8PJgF>goavE+D=ErX+k&9-9?# z4$!Wqrltaak #jpyKP<_&XeI+Q>+g~QH zE%ZF2O7yY6{cfNC3YB7}Ca1%?)Aex5DL`h2NK|{DJAY!*26f_+Qn-%={gR@mM+xi( zmqk}B@Bwr~^8y6~=z0JiAd U*k$|O;0YuHJJG-Ntp(OEKUcL^iAG;rRL7WTCR_NPbk{^ z&RQ(p@dz$xX8K%_4?G*|E#>`wkM=d$`9v{v;}Qn0sV`x$*eSP0MX*Fqp2G>XxTKG+ z3_Gg3MHkPVc$3|uacec(A-m3UPQPV#Zyr4xu7;V|QBY6_q=`BBvWanU+)oy2p $BtNlENX-Immoa)_ffB4l73pA6E z>J<8%lz1f}R3^#aey$jG7HWGdofr5SuC=Hr*tXqTM%l&gYVHlAAN_^! v;Il$Y(e=F|^ZP}$5&+b2sz{!2GQArIjj6Z}b} zxr-os-9`D~c%j B }m@x%mNr%m5CxS$q@%j@TbRy7d`gkO|DKtd>`d=AzOea)5H2HF)fu zpm?Do=mUHZzyuxjE&@KiFBm}hcJ5qJk!Wspw*+-*nssvo#q!mNk@dEjoE>53y+Q97 zqtmnJ8ZP_9s(KEUM2y+ZtA(i>A1!x>{h7fWg;>|!+y?sFlh09Z7PXlTfy=EEEIam? zUfcdCL)w8+)nG*@5E7R+{EM>cnPV$HLqR7}C5!@W%^~BW!&(nZ4EL*7xrD36WHT6< zU-ooGei4tqj*8QH_1v1;R^@TF1K&+dA~@K4HL6FtvTLF2K9;kqNYkZm) 8=K~rh6ogMhM>x7CRFv@PrFEPya>LF9Lpacd $kE{*XyG93h_svg&dU19cFjV;fP-(iYR&3E zP>9Le=(=VfT!+|RTzQ*y`)o?Z4~{j&frF0mGZ|&PeXwpEH@hKBw%K8}X&iSFvwt<8 z&5f~l&e^MPY-D9*42z7EmX=no7E{pB07@!A>OZ!9p=mGhL!1Rj=m*O?T3K1870t41 zIw9~&OJ+_@GVy5i=j|T3G1!mzZ$L`HYmR4 fQ2`8=r%QqhwG2I%3s(So>cXZV%h|<3L1Q$+$wi_{UMgT#Rqv4y>|rr8>4&Cb zd~}hWtm~xX?U933mG=)W4|EP4-?@^V3~`@E)t7C&O%)}xGWM*1H8gbbe0_cOyDl}| zs>V0M2>J8@JJ_ayW%bO1=S)%=ip(lsU+z+NN){U>JJ)6JLpSL}XUo13kst7xu0CCy z?XIo7=<8blTUg6WbX4-O3QJT$>C+R1mzqxiQlEP#qog`}FaM?PT-QhWYlr=o34W_{ zQ{o4k>%wIgZI!>XoPlqc2-!O{rfa3SJ?M`Ld5st?Wg`X~yr?7O9h`+RLnHTis@K+J zeh{*YTc5&TfRp=ky!<8Lihb!?3al~E>HJglzo@ipL65o#Mjgis@HzV`=4}ScNKk&( zh^Re0>D>tjes! TN58Ap=39kfuvjVB;WL`Ag8{0}_%skc zQ3FsCaJl_t6ck4PV-tV}Syk3;$Ruzh-7Jk=T$K2pw(o{^{qW!*)F$MEWnMu+fhZ9z zJv~4Bcx#l(gMoHaM o6IxDpQ&2{tjLCJEHVj8J*|{s%q?5+C-zp(Io?S`roCQiYx@U9HjL@XDDQ7i?EV z01T!_U@(n^MMeGZ(f{=?v?5IR?*@H66*`)N^Gfj~RdPF(%pjQuD()&=c=<-9r3d4u zc?gvd(BFsr-27h}%K!bR&9cg8ak}?8$eG!COaYWEbxw?rH3R;@CQZ#5sx>L0On0xd zV(pC>-i?XB 3x29d+lf$0R*@6Rk*G z`-+S?JM3iHY*L*&5W7v0tho$#wKJ&T&dX)CU?uBl%3MaZMd1$dw%uaja#&Tgf#hoc z8`~d*i1EHZ=adb$d+ykmlxn< 0YnSiC>o$~?H8e+_@>tA<|Cp;_myaVg`($Z% z)k456#q4AhpvL(z(ZzNAA}|hbp^IW`esVAF@h&eSTJD_L)hjI=22>&|SH34@uAzAS zGGi#TLe(n%0}o%1#I=^)zrpGFPItIkaZ-S3fhRZmTaeKJU6$+8=s9v(QERZLmrBmm zn2om%ncB=33@VWbc)e8L<1NBOVSNg8(JRd?YYl_4ZCPRt!XB5Cr2#o1)k&^TOt5kl z|M#N~dVXfqy{yYgutfyol6aZUbjiu=O+sg%^ZD8xNQ2pV&cS(QkFL_%jY0-y21{E3 zv7#^JJ{WP1-+1i}=@o>zC!xjW&!yBLidSc1PwQ7XAD{`a*;edCc=N?0qN+b|bK}H0 z=e1X)SsY;=l!=v;4Mi|*pJlBRqwf R{3NU}{h`;yuXPr}$PiE-#s=yr3X(&=~&Rcm=`7 xSZ!gb9?OjagEKbinpzl;2=x(;iKcbC_Rh{9<1AV&%jM31 zAg=D~GxK(*o Qr2kU2dPd#bUpCOPxheOJfX(5TJt$Vi`o Gc zU_XkUe=ZE2!k9_J-rnBkfF^ku5F+#%UZgM@>ADRjy1fx_-^2{ZC^|`DTGL9!yk@^N zJ9=W gFa?bx|{dnJ2c=f2|=uZcSYs~km$2sgZnjf0F zXMFgI+MoZn=i|v_;{+1(xWeD3qb8c{&(4%l-=D7&S5L*lr@-do#9;#Is_UdDsHa$= zTLc9ah6h@8orl%pFj|!ac)nbuE%bB|44vH{n(-n@ e z0FV9X%P}3+P#Ik3OnZkOv7Q#=dysshy{X&5YI~d{W>hERc+^3JenK-%U{9DJ_fucg4gbBY>!E&tG=~^X9JysO8Iz9k9nmS7b*j zKe8a5w0pujGwtzu@bl>%lb8Y&2j}&6o-iPQdXi0D(TR24omou3;54C13hu4h_Getp zs%TaK?*f|~wFY_A`tVrbDVG-e>UhJ0{I{7NkF&-zF^ {cGrTOeig&SXJ@lx=&9WYijhNAM0u`jo0#F{$ef4fB+!IM9$}Yo9Z*_YJgNmDrY+ z=X97thor_zW?WdX0a`}G(z7$z!Hw`g`33MwL?J}Fp40Lgr_hC$YQIjrOK8olFQeJr zTlACDm8HtR(b2Se$bJ{laPq4)dz|5s?!~G-nGY6Zd1-s1ez`q UD86X${ z69ll(fq^B$W-cogYp*cCGzxM68!}^nK;i9T*iEB&vGnav >PMgM>1>L&huMc^Qk6 zf< >txvmU4HNn5v{!dwoskeo(TbREQ_UC)Bf?EA^tF45^a0h >Zh+J=*!gIjPeuXK w3Oaw$tI^I!khFCg)ZtZ&Q;w zk{jjBS{m|$wvNJXfYH-<@saB_zIl;gGIj#oJL4DHb%AsqrF?8F1(1Lw_q#{UD+svY zh1kG2E;;j(=-@r_laiAWo}tN 4oU5dY vN&9cxEK)dr*nA4BbY$%g=}ZYt2JMa}|GM zCK^`B`+(VOtL*duU8k^FedS{mYgaRO5p;E<>!~Ivh0n*cK1)SMXLQp(txn-oVynX2 zof@xwpBIo``d1!vo^9Kj`G$L!T;EwP1+Be<6tuUm;c_P8^S;j&24Zo~687iA4e;Ap z4QnR8%Jou3>9X4%Obz#$p=8k_6BV!ie4UHGC3;xW_>nwZQ?9cWaQHkLgm9ICBf#*G z_IZc@Dl)CSeE&I79SV=o%=*D7PUZGY16@^ji0^T=&Z&nT8t388(Ns5-rpt#^lvDm{ zYL^BRbaQYN^=awx_Qo7q=1&Jy>ji!}s8U%T5i7rwMK6^IpMJgp% &Ur|Q9?1iY`?jd(8qB>WKcZcGnCx1d!{ETZ75fM2RvQt4lb|I zP%c H$}mYL)gP5aZ+T@4pmfHlEet3G6V$>EzC{ z79r;m*NjR)Czh(GDFL-FQ6i$2m`Kg3nXwnB-I#j?SP}{q5Ah??AwcS%WM3y^U+xqE z6INlkm_RBOf3g2}7&J2>ePc7b LNtrx%<-lmYvkWOTJt6)M zY)r;-Fuy20UC`7?ls;Bkk+)fRZ0A(inil(sI=36h8qTdsfC?+zW#N&{z^8dYIH*e~ z;L#i}&g|ZP{YZAIJvvS#BRf<)BI4pEH3nM>?m(_orh*XY=~a&C%*rf(S6N0rgXcfP z$cm~R#rYJ2;k{B(*j`*Od2I2J)F9D=S4wJQYEcOLR#>Fwl4O6WIEAq{=Gx*=dN`Z; z^&}&q!7?`!bIh-j+5N%r&`aySY{5yZhbe(YW?H_PduM2yQ50Rw&7}F}rMVn~`~Px6 z1x`Ai#xqZFK ZF^6N|0<*ni!K>bQwMAg^Tp4oz85m*g)OY--88xG7#LF4IR5 zg%rK75cb-TUg;H~5-mQQpE6wP({^zecpt8Vs8P*y?8r;tRQG@CdLluOIqMoPKsl__ z4```Ttif@6T~>!$rp+{*?iV9lBp%iRgP;l7#gqf8yZ(mwq;%Z4#v=`lkQPbxwDy5* zO!cJ8h-l~jKx#T}in3!YNeJ=)-UC|3+!7A&rPE)(elbU}42UC{3Y0SYlo#(LMV0V5 zd!WR^utu_G!8!*+5)-sNsDmjoNl*WPFsevd?OioO^UYtz>^JpMQM Vqj@y0=m+=NoP7sfR{qk@d`QvvpxK zAUi2Z#m&edJwJ69`41ypPCbU_VB4xH1kYos? z?FmIB+AuRs%+zWkX+(0 +cil5YxJa0c12peL^l>o-$6~tsi09KS-Bn? BP_YaRA0yL?W@`qfg{~HIVg)-X|;3@_Uxwp$f01audV%YLH3}f$tKp5 z4e`L(ZjBsYUUc5L0t#a1t`+bzR&~1DmfEK>YnnFV^*~S9-`?~?Uxm}FmXl(8+E-c& z`{7DyAN-tE<@x}=ANdkeLr14%9J-}F_%~E`>F1t3dnkoXOJP2;s8&r9ZeStah36|( zKIg7Xy^~hZNb7@yhs%}a*_8L?f| _%dV6*T!o+D)Ruco~V6(7eGV)}$wX{>e z;ugO)x_Y@NNc_YBZQV~6O|N9#&M)6{Ac5_kj>R&&Qm3XpXrT5T1->4L^5~N yRfDScuPKam+c&$*J~@YjL?B^X;9h{wEMSPWbbZE`382B^3G!(YxDAwxdx# zw`&z te!Yc8nh`{7j0j6S`J$o0Bq%GK^lZ9X>NV|#^R-U*JVt3+=cW`S z5Pq}tbc>!O2-iBUJ9?{L#x*4$m1u>- s=Z7Gfp+J@p6i$Y>J9B&DolihsJd7w7KO?3!oumB!{p`aR2Y}zloR4sd zMju|xL ^j22NYD(U}=J zR(4FNyTSS3ar?USgZwzi4^R$!P^2h7U+@BvGv*DYV=y}a*faeb?h+`&=Ipv}#smS* zm+kI1*=?h`{QQ1ZRklJY@8zmFyx!Y8JD+l|nlX>HI4*l8mm{b;*FLOOg;rgHD?5oG zv(7u#o=mF}b27}yUUR^6e$Xl|giUFNn(r;kOKgnMcC{By8O`1%Ab09KU2PT!#uvO} zoZW>yJ7aphrFgBrT&-#7EarLNvDr@MV+4PCTU%DR@z-w7Go|vM_c_$dKok947P%y5 z{JIZ12~z0a7BX_DxOfpC{2r~~{6pf{^O<9l$H{!rC9SP4EVYQ7#$ff{!mc`A;S0}T zR@;Ff%lde~l|qtoVd17vl?2!QAwtK+%tn@jEE$ie{cEMBxKxI}_IR2cnLW6|=Ir9p z 46CDY=p2Jc&qQQdbIQxxYqRTI%sMqUt-3G?$r>Il*ct(aAe(T5?>YUtS`~ zri7|}Wuf49P_nPLbuVfd`&{6UOHJ^1TxqPIL?GtBcYiqFZF{#wyVCI>LFQ$3^eoT9 zOE9l{Q7^f@d(*We(DKpkZ)P1RC;{iZ{+7EMkX)VKr B{1}pe_;r`+~Ojo zA2%7D&t0&6SudYJz)+xwL5x!NjxCRo_}jOyPNb7FP}$0*b9g%_+IYklfinS#s$$^P zJ+{+#+Z-SpgZ{(bM&LC^#0PX+ii(oT dmP zd^I6fA$T~gf3N^=uD$201Ddj;#UmWxXCB_yU3edC{k4rrkLzOa49ktQ+9j?ZlQ{Ud zUm!f}C$&MU7>KMjzp)P`U0LIdr?ccGA+));v?f0D56!E;h<`!Ag8MQAW*+i}j*IyY zB=`@9 W6U()JUhDO zU|Oeha=WBRuF6x;j2#YZY6h;&?Lp-+i!z*Kn$?Lx9;6!U2aEY03w`e8W6`R933>GJ z@OrzTTD(2ivx_;)z-}0 EIwOS{uWocMk3?9 @SBh 96rsPnJk?h%10#q4Xl!f*;_#z`+4z%O zj$>k9_62m|YRy%8zo&?vts_k0UM|Z)=ww)7aMBsf4oc<==z_J_X)2Eu6pKqp7}Olw zss{ *$>C>aF$W$;Sh^!L(;9{{+BrRp?17`AZ9d8P5~gtUU*J4-d`=k@SpX5N-R5 z!%a34o obE*y`5!~@N(Q= zGUrUll1u&c^lV)OCF5y(`0=J3PUyxtl%~I2Qzzc eemOI51#^jOU9gm5v zI$Gyqb{*d_zYLFuHVdFWKla7+@IgLZ(K8cCDdzb;GSz{dHb#H0H)>8p6SKh}BJLw9 z4+}D_Y|O$xc2k>B4u#LYGRWEMTJzrCQYvFy#7IQ6GcXOz&|_nUKXJ2y@Qa8FvaaSA zBz^)JUF$~)(nlxy?;y(Th?xGd+Ooi+$$r>H;0mmT?V*zA%UgJc<-7?KPE#qitL%V| zPl}RyO^ihIL?}<`HejrTu$H_tJL+M$TZYs6D&v!giP2#VMB;S6;WUUV!!HSqZyknC z&OXm?OA5y$Z^yTQwj&_kP~>8rcoL+0hXEG09~1LvaF7(ZKe@4I;uo>7U>)jT_XBqD zwB9I4$CQcop8Q|ut)D*cT>A_jGNQw0HW1OYDJH8jZ}yCArwj{_W9n#8X3@w9mk?2L zso`|;0Yb9>i7iiK&r7Jz>xKJ2f}q9aQY0yW#~iRzInr7`xho#7pJKr>#pKPJ#@M4G zv-2Yg{%CpRPR{cuCL+=n6HCv@iTS_!3@~dA#&5 ygH2FiPmB+cle~1N{^Bh!BNLtH54rPKA^X2l4)SHt>1CgDSL{&D%fc0ue_YV} zk2bsUPCAW0KN*f=@I$03gH}a0Yj2&9b@u>MA2bBN9W6foKDqPfW%89!FNtHVZ@$YJ z7a3bFEnzYM_cQ(l5k-_#L@~Yg|C22Fp`HrDGS5Q2l~C#i8Z#AgT!y8k@=u2AFZ|y0 zO? Rvs+&)5G2E&6jzbBho{vgBM1UQU-=>hQ*BT`KbWU1K{8_0@W>hF z1dK;<5G)`qZVDy}LQ#{31tR>LO#z<6aRDBL^Jt<%fhLU&hO;qI=YT&y;NGCDX3k~8 zTbokiOImNKlyNN{D|m?HmSeP;tdEN?7Mpb1pQlljDdm0gy8f=6|NFQxPi$oQh75MW zYYTET)mYM%#3iStC4^RV60$>B1hxE3IlO14H~Zg#+4#x$>iJ58pEf{T_f8*Dh$^M0 z2CG|Z^Z24t{=XRzlW#)mit<0LY_es_QCO6rje&qdKbzfUC298}By_UB%EA6e-Mslz zEAtQ-(@ue;cY)xrK;{b?HxCizyR+10i=f`g-rJ~@B<8dCWpi7-SWuAtk$<%9&)6!7 zm4DAvr%sk4{R%wW3kPSTeh34?a5Vz_(AnV4tRAqSr|A7=#Qg7TZu3>*vY rs<}>@|`&G1(&xyDr9vmcBp?AS410AW@tv!YDYWR1Hq_4Qr zr|DYkjJ7Fl*WBi8sL7-j`7T@a`Hxk1-W%V{_6JeMpq+n@X#hKTOW+>QK3mHbvheOh zuc3lnIP>TCPaY?+dzFP~H`P2Q8cE89)ALndQ94&-_1)trF1XzWda`1P{N##FPjboW zb=&USjdPQ|7WL+1@ION~4G~;?)btjxbokd>{ZIxzp *>)R&-?s vqSX&AY|`tfGS`0T_Wqd zIx1rYvt~s-0tE)u80}Af-xDN6al{k+;5u)zo%(k=_+n-DBe9lvaBzxOrEOqylMhJO z6O}APn-T&5fOMB@abGHb^i3jKRv$%%Gh(q%L`58p&Pb#qm_ss{h#sC1!-U6=1Nv`Y z fXIwOn*oX}S`? z9f- ?M9^?Q9OqMP; zdwMp_?l5=NKRwXtG(VA-H!FyL0hy(=8`Tp#U&W!fMN0BKs^VGvnv-)OVc{MPcU~;p z*urK@#Xcra0xyHZo*=B@guOgFAsqOt;=yF6pcSe^+-&(JrrN2~{`RuyYhDwKsOM4V zi|%qmb=zNuFXc3UQNE)uVQm8w7F1s)%Aq>>g%PMp1N|)ASzs%Q->w89`@I5asVMRV zBJ-F8ZPUw$85kLh0t$@HEV_e6dn+nJ )(~=fJ6KI*^gFIW_+>w zGLDt!tHKBnnB3SPyhEaFvU>aERNE9^Nn!Rij7eLASP{v9#{T9SrM05^M=*k=63VC&Kuz)$Er6HuVq=}UIQv@{zmASf->OvRrUVOBLf<`ZhbBRdi$fDSl5y$7 zg=o%OTL3C|cc(j(M0o-o4hOPOdW##o1w9mq-U*iD#<`z+LE?Z&QGNGX->gF2IRZY5 zbp1%<>$Qq8FX)sn*4(8ELKhYhrg=ZttX z7q-8gn*~!Uzv^zu9&7U@c4?j|rT1)*9IymNjoWmtfGw|0ae2h{SIpvI!t<~2^SfGn z$-+l2wG3oXNsX*2;cW>TJkw#T!@nRu`>9YWg8+&%DnHqC9-ApNr7dWUXCivk9YnS# z-Z~U_MugPSVuN%`<+z?`2x mm@85MW=W-6lmlYr1Q+Ki+oa#B6$u}S& zn2B$HopO&aAlR*s`>8q-E@74&q{397q--A-X_9Upwy}0;_DG^syN#57cYOh_GsH!7 zH~B^g`;x7)tlu=cH#cj^(w_9weP_P#Yd+mN6{0H+OK}^)SCHI5SYO-WqFn$?P=~eS zXIA5d0)RuZ@#Z_vDaVCHcl)mdN%1%-k>jQ2wiZa jgHlnsjgqGMH5g*3JX!2+m9#BR9HK4hX8;ssP9PVGxTTEW4xf_2=qXW5{V=$Ta zY37oD!thn^zi^HFCv;|*4gn!T5kGZ0hiCK`h83o?qCEO7o4Oa{!GN{kAJEs|AAX~$ z6qiSVt7j$Y=*WISOh*dyi{^EC4!71zanXK;i(GRtds%s%{D$icEb#Cs3=PWOi{SC) z!U$E;$f&HcWWwiKXID~K>w8s&LPI>B4f7}E)I_Z^Yf}nBkX%9*h77~=yv@iFJxcYw zND~b+bJoj)c1jwnvAjbvQ(A@9lLFb+ EkWN5uai<6R#frL^ zT4IXp ;wfH|<4+h8c?b(s-5jdxwd2KFRvqy*qsx}`Q(#q=3%rH1@x2Na1clBS* z+y~F77SBocP{u6D!Bs14XMZ)dNv@M1V0l9U?Y1tTPP-L1N4rciBJ*+pr$!f zpRCZ2yLow0kWS``_xAEqFofs5;9Uae;HCwqY%Vc34?NH1XrzoHnQM5wHc*lfR2^nN zF{Uf{w(ch!0i-6}P2w7HchKnT&XbhQX@$L;aWTVk*K*P-k`Y6fgjG+Kpsgz0y1H~* zQQu?x$@iV=N3NSfh76Bq^KorX^l qGVno%X?fX&1~G= z9%Z3oWQ;b$O~%wgcEm{0c=0V)zPR+kiJN&LL&@3Qya>sUc4D}SlsO2eMM|agoF-c? zP^hzlZCQlv{?%EM(@nzGF_@CbYqf{#^TqF9-;f03>>O^wYGrmrTO&eKnvC{3`mp;F z*G1ibx&B<29;H#PB`!Ym_H<9
p&ejzC80ph`l=gn )*JYCDL zCvH@{`Ihnv@lOvVB{|jcDr`!)c3CO0ZFyf^$9;^hzj??Ee3)KLd?{%DDSY8s?x8H% zq2y|rqVW<=%#U9KS*>nOe+Ayp8a85tM8~8)$dtC%F`k|>B6Y3P`Lvx9c{UYoz1@Em z@CE#W^#!h$pUP~Lq~1ySu5 mL 6{--8LlPqZB<*KXe%*As z-?6ZbRl&Z#W_^8qZ=mXvW>9+Kd%VHDeF=g4#%qT!Y-w IT4kQhdfX-oL)~LJ4A__bn2a71f2-;jx zQFXQGiduPcD+&WHe(;BuLt(Tz=i}v@Eyj=-SL{xQ2c~$#B;C9ykfS~QJ9jQRRnHiA zY1CTzsnE6hD(~)ePlZ-&ZHHqeTS1{=GQDi4ILbLIn;r7R&(AG6J*2isYmG{c9RpZB zm&@D&KrL_5NEg)VseP#rO1^)`lxzrITJg5UJM9q%8%QFd{IHW;FH(H5o
a^{k(Q_?r9B_A?h723R8yV*7;H ;%&I zs-c@QSV9N*nWc+DLJ8r%wa$SCFbI;fPcNUa(T>j *~>RMfglS}mO zqDDCw^r7Gd)w46ru>CGbm?7Iu?^#2+v#_@mMTP{0j2|;y(Arf=)toCAIS3-JV|%~} zGWd^zze=MMj8ngi;aPj%=kS`jz4A4tUYvrKwmaSE`nI~C!aMJG@V#twD4vzcXgn8~ zCchu1pFM|Pd@B!q709?)y$XDBiF^|=uftotIyL;Q3}P1nr4dpgfN3~45fLINvapiH z^)hw#l1bxXrsxljC^k#>k}-jer-f8Chf(+wd$^7}Z?@QHHj{OqmVpoux+A!FG-eOp zw_E%r&)+e`!g??qm#Zuu$g`lN^TYGtt^gMHZ7hMBS+M_?+pTbw$JIUAIp$bEB5~Z} z{haF^JC&41B!Ydl6$qmAuhtjIdJKX@D>U$xd+l@FOIqVONP^vqvP#3fC-_s0DfGl$ zHLRC7MnEi%cnhHN{V}C4eC-9#H
}%SBkF$kzCGb0nX$H%aR(uOM}WN%3A1HC}!}V&{p*y(>ll!f1qZ z=c~ThhaW_mLE)jmG4)|pCANx@_2i<#=}bR N%cg@Ldim!)Zy_>wc zxW0-P49^$K?WMJO`2G+``}%KuD}tySb&OnEJYmMIC`F)_(834^(ux>RM7ZnW>;+@d z2!g$Ksxd=yv85@6;=2cGCH&)<<&8?|W&k;f_{sH`m0$0jhy-*|>0d>H*r{ZV&`9h# zadxm7isUR4>zSJ#>?*K_FfrYgy0-)qdE*mMvflF(RF&lOnIxGZ^P01wIhY5KhlR-@Zh zd^e9iUAp{`Dvux_dOnD|oGZZrSP>*TymY%o^PFeXEgOEudxR0 G~57GD`XJ z5qO1*V-1NDMh3%s+!jQQjzrvd;I~mk3<1VY!i9dTxWa0X;Gl)jIZ8=tYT4|xfrlh( z>$s|d_9PM4gjxRf6*^=?LeFXM_4t?6baGJR4IwRDnlGPMYOi0f D91(}ER1A(diBer{Bl&ykp;Esww*Hg(61~Rk~#yOFj z{R_mVwS~m3VW}O#_ B14BTYkQ|;TCm!wg>U(Le?l&vt zC6zyecC9Ao@c{vEcd%X#B3I)&E2wVC$cILvUgJ8^jzqAAsaie<*tj)B*3lHt#>S`# znWMs!wzxIHpfAg8zTT|B6Ba2pka&Wb)Og(<3HIFB{A=6?efA#S#5))X>X}7P^?N(2 zt G3f^SbCgI}>7bZSj};vWAocqk8HC8H##=&ra0>v8 zp@lPLO8gAS0ig*yuL+Ssf~=D=hfYFnsRAdEVoE{#VYiV}CXR+@!n0@$AuS?(!jKd@ z=xc42ZX}ztPVfCA&vg*Ia^5>9k{H}yERoR1o@!juC7viMDIh_AF-t#2( >j>tvp=?i0*VA@SQ*+2|{kXoD5zX(h)rVrs`@{yLx0RM |=$+%~gTnkF8N*o&xs<%iw$v_bX%ZrvMK8_T*#Y3u!GmbalO14uU^w`5N#Zh97 zWlCtL%e^9f9&-4cghNu-t 1C`agzzO{6HQQYDt>b2NhsCJj!CkHB3oe{mG1>hP0`H;A8 zE--B+J;2W#)A-~NT5W;8ISfBCb3Ka)dhA+QBc3!)(o-m|ip|Q{^PmqEu0qo0DCA zBj&F337GdkdAA`f`(ZanFFJy;t&T0f$Hbuak2#ur{Y6g3D|LlWD!$3OL8@m$>6`BJ zA1A!k6wMjRIq@TH1m1*ga}B9o!%8QV(bI9tI`Z@1;);b)0s-mv0z8Up(6ZjPvLx3A zafl9hxgW8dY><63;oVrwwA&Plkj4OOi#g+nO6F@_$&Zyh-@+NW7?4zx9366`MM-d< zms}9Qz7S?mU-^;G&Bw=nnur-cj>V~kO{!;YAZjsshxZs3vFDZW6{$+qBviS$0!a}K zxYXqT__f)kI6H5xnP$|FL`!gqLnr4Uq=ebDYLTTCP6mrN)e=iOWG9`X+%vceTCpg0 z#|;||R~JCo$K6+WpO}Jg6MZM0ygXDxTR4Kds;UI9aOt|pTI(=2Iw3hUiVwB<_|$_| zG8+?%g_x4`u%o6u7sJ>Sryo7cB( %85p)uEN&w-%uP9IQzT z4;slKQo3VYBT>KrM kfO`Kh2nTVe<4#?ulhnlE;$j2uK;h{#=m8tL&HcX)bp)k zf`=>*<8R@Ge)egC0MUCVXnw|?T)caBYT1i$Jdkb0Bq&@cJMGIPEg@gDVoY=Tc0w;f zMJ~aZ7RlLg>vjs;Wo{!E$`>4i`XQ}?SZp5Y 6-rcV7 zXD3}~0W-}$J)3W8|EBNOWlx|fSLO}`XfR!qhPakR*|(|w^sQdew>ez%;zm+L&`-bo z{lAH6sC*8BBw%Ns=J8XgQ$R&x5x1hoOFvY#<4MUZ;zK|mdaj;ipSVsp2uBhm2ng`! zH@XK8re%rIOCaGvlgxPE{Ex!76&8%ydH+eMWq^D@z{OFK$+qtlkGbZ+vEL zA|#?SG%;V*)VKL%M#oA=%>HgQ!2|KH=)f(2NZV9f1XtUgJt|(X1Ot98UpDk*+C>tQ z6V#@b%k$`kPtGdFNKpvT_f5{=3T2F@8uBD|)MWVMO*wpv=t)LBHRdED;*`S8_vtE@ zz_2*rmrRDAv1 e1wDo3kHF^g-vp6(cW}6yx3QyB;Lk@>!NSAz1L#sF&$0P zX|=UZ^YiGuYh;Og^o&zxi$#O9wlnExUx#Q9w%DhXNh-Nd1k+Ey_en}|w9#gknV((I zjOxv?>*1N%m^ovjS&w33R~(X%D@xQJ3*MyU;La s$O}TVXZF0MF ztUxxeB@9QJ4}5ge!?tJ(3?3jkLSE>mY`{vOl1cH4Ow8D6UrOEOY`Jix< 1-jjwVkIf0;|YWz5k);- z0!GGWNGutLYq;gnFbs+@9!}o`epj~hlS#kzMK A+Y0aOx``dyBff3N_5yf>%RFZu5PM$-PVp!0jUq5L3K ;mf#=B1OjpcM~DyrO1HW?C^IR} zApX4C{(E_SnKgG*i!s~ozN8}6z8CX}eL>{=kGHk3`@@OrH5y|P@lbZoNj$y{y!1!C z=A(Xk!B81pUA1&z84q$&-##MVy&Aez+DtP6iAs(Rud(Z*CuB*=5d{_Ab}&31$I>k& z+fWtk_}XMXg%)xlHkUETl!&za8T^0=&eFmpM!NM@61Q8a*&;_CAAbdl)WUYpg`aAP zWwCBh0Hu}(tDm=crIl~n{EAv$q9e5=X=H6b`zM>B;jkqxDzRQqwZ!hi&;k&$+DV({ z7>jG_mEkvKad|e^oQm0mhv-LuH_)x2S7uVgOTR|ilGr@#td5gtyOIw*769n^TE8KL zS82pzXoFEi$dAQM9YGTkAN-M1@=vOEhr`1WQL*oNvJ~1tnp%r!9KRj3icO=pTQ$R* z7R3%x$&t~Qh0=65FT2L4esgIF`Bu0Hj&T|c(%#k?h{ml1GD{8YDJL+Bq=m#*y|-eZ zrd{DAF vY)6c|r` 4;XT zQOBt1dZHC^SxFY1V{L|aJzzw=SzE*?0=DpX2l#)qwXGV|_ PCJ{IZ55A9QYO-%{wjs|!? zwZj2B(9H74z7WP7yxQ6GBt5w5eLfwRfbJ>(93J9QF%|Y{2xK2k+2AO6sPpC45OrVk zc{Q5RcWidb%g|Dppb!X-{?q(cBTwqdfPDVI7QS))I}m=u#^K`a^ziY+P-*m)T2j b 9RI8RAoKv$J{XtipYBF~#>0;rp20ax&eyqrGUE zlc)B+XxsU6-geRXvhcBClLt%^kB#reysb}toE%J57R7uFMT`0BpeNdi=T+5kZS~;2 zZKr&^PONpZy*um_0M#l0`9BNp|2*C<#16Y5dfJS6wb9yhD+rzU`LN(@wQ2MABq&|? z74Ku+>gl~l=%axXjsT?s^S$pxdT}#FqSVW5wXtSTeVAM@baM4Q=HoTc2|BL?kM&{i zW1h pjmYZ%Jb78ujCG1J6ef*v~0mo#uwqW!=~7->$kFC|{x>|NH$xx^Y3AUqZJX zTJBGm4OuhTY&X2V(uxJc`N~tF@V%Q-ZJU(#?iQh<&CMo!jq7$S9a7z~Szkkw>FxZ@L=n{nP6f zAHCpAI@69?9Y^q-8po$Kg#YQ4A6LL?{@~L>36KL G zxn7Zif7tAKVyqKsqrU*U$+#%=|H~~1S1}<$Z#~{<#MzL=7K2uJ*#tu+$}7I3(w?uc zBBkmGX>>#*fxmzkB@!xzEvu*NorN*M{nGBZqx-m5!*@Pw6~t^N_@9n}xPmr`G%8jt z@?G^KH~kg_tlnf@58ab@-PjLFVk?imUyE7ler$AcUtS(Q;L$4F=REe=Y6h5H%@66_ zgqMB3?(%t>bL=I0YVH!b8Kx^%pQqp7us!|$wqu*KlJDlS+UwSGV9-F*y2dg0bY+{Z zYE<6dQS$LPgJ%H~bNc>^lEm%**&* ;BCvYC7u&FnxhT@Ue&bW}<$LCq&mj~>e|wbth#C5rOY}Nn(*^!5YH8M8 zU) ejZc8K* zv?J6*Bx^gKYdbT*b9d2ktB%f~!i(4f1{FA@<9m{Ce#4|YhL #5!RmYh zY2r{DQG p>!3f-BY#& wzFcmOh zejQcCG(PVv{)*1`mY~~k(s^n}%RXG=f@G+!kcOM3Gsv8P$yUd{eZpP)P)9nd&{|Kf z5{xJq{-{(0z+fpLhxmK6*XA)zZF1j-&+DpSB00gN4c@P zrK5@z`J<8&iHh?6_r*! zBUO!VR+ISd4j-WZ1dWPl R6#R8;*I`c0F6s&z0EJ;TgGi;DvNXq02k$u}zdOrF^}L+65}!hF;dRrFw8G%^2V zcpdKP)0oWir|z2xL$;~Ri79TiABoK4a#KC3_gYTT^Sg?J##}jCnDZY&OX{?tcaiNE zYhooQoMKu5qnVm5KGy19yVfoT1FpIpr#g)9y f1Q0{fOCOGgz z?sC)xU6XD5gdVX*6Xv+0 {iD=1hJ?iP9ydCgy zDz9>!64r unBTo z5{(LbkHZ#cfaT1dLlFv}dK2;Dlqk2lZ6Gf+XkeEY5)YEck+Y$*0Xdg|&5sMz1IW|N z#0e~qii`}bT@2Ft1%#gvXXw}M9UecJz~##uc;FVe%n6=NnT-)kON4}Nms8z@NkmLc z*SXlA-2Us =l<={Y)ScQORVps!rh)<(Kum|7ghhl`OA*jT{~$(b0ImE zfc}Mfr&FnHatFRMH5o~y7Vj**)Mq$NDV3*qksx!*pOtwv1Ll+?OcK+mHBG9Ex1f ?nOl+%i>tBn?=FFyy}wy zN+TR#vWr_yn_Tl^+c%QvyTul7Zz(6oTw%rn&r4rAQ)7GHpA#lrMu$9814ty%_Q?Rx zSjSipDDGbs=79bf2C)0w?F;i}hfN~=-R+-*<;tT2C?S%K1lB&omx6>O*F)jAOW z?F1L&iZn?DHN7Ab0jq+`e)%V tqB@Y#E4)8k?eRb4~g z++M(?Rb2nVro!@`G|BkHoSBXH=#2CBzVn6%TGHHI-RARgWe4;2_PW&asF&d*(s`ro zrbm~FTDC&1o4!xk;DXZTQGFflO>dGr3=12w=`$`dP=m`f0%XHiw$b>T8^6-@={S4H z4=(Y5mOJgc)ybfaI84gSZ--J*;@}w|Y~s=Hq3#zAxIb%Xrb8HUkT{Zjxqi^$UKXt_ zhdxR!j%#k8nZq!!=FsH&L8Y#aLMIl=zB-w6k>^M?HcJ$=(7&MLou`_nsh(UG-TB=t z_gt7MM@ 4L(GZXv!0UxAZt509O zk6(?8D2}m5IKK9oZg}l {#%4CxaXhJT(pEYtyyNP+j-$MU66@y*RXxH;xsG z`1_H{nv~Mag5)IP2j|{rX1<%Du_>fJu>Pj*0%=(p)(eDU)HJH5yTr%jw$wpp8ln?X zF#nAYILT9w7uH!-+??O3pcmH!r)0J}m#dsmt1%Q)j{i&m82##e(e6{9m*?vqAC!zw z{Qrj^FgDnl5e9^I6yJ;+YwvFtp)NY{BC4`jk82wM!+%3TKW9^RV8EY@ENi{|AeS62 zqDS2@1tF7&V^tqm?{AmBBLyM~ev5YAVJ9MrVT4CdcPr6U8qT<&-*Mwu2x8_4KAA6- zse+(b$lTBut_*wCZg8&P{_w}uydI= )b9M6Gt`YMufb zfIu0!`f)AxJq3LvKNx%xuWfgl&g=H7VasB>cJ1p$rwb$|dgOt+DVKe}ODuDnX{p6k z7?DYYW&oq7xx2yQ5#7h_pxIS^ZG3OVBZkLye_B=<$#;BSclCdO3|o(}=qFxJebMiq z;28(T!)AlI%rGL0cQKf3F7WJ#5&}%mz8n4uktMIDCYz=h{IRMST6Zm+ZQg$Txqs5( zQ-xvi>n}fcj!EAv+|a