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 - -
- Add Configuration pycharm -
- - - Add Script Path and Environment Variable to new Python configuration - - .. raw:: html - -
- Add environment variable pycharm -
- -- Now Debug an example dag and view the entries in tables such as ``dag_run, xcom`` etc in MySQL Workbench. +- Run the file. Creating a branch ################# diff --git a/contributing-docs/quick-start-ide/contributors_quick_start_vscode.rst b/contributing-docs/quick-start-ide/contributors_quick_start_vscode.rst index 88ff1fdd84e..61fdf501063 100644 --- a/contributing-docs/quick-start-ide/contributors_quick_start_vscode.rst +++ b/contributing-docs/quick-start-ide/contributors_quick_start_vscode.rst @@ -72,8 +72,7 @@ Setting up debugging if __name__ == "__main__": - dag.clear() - dag.run() + dag.test() - Add ``"AIRFLOW__CORE__EXECUTOR": "DebugExecutor"`` to the ``"env"`` field of Debug configuration. diff --git a/contributing-docs/quick-start-ide/images/pycharm_add_configuration.png b/contributing-docs/quick-start-ide/images/pycharm_add_configuration.png deleted file mode 100644 index 525b73e6141abc16831d4228d60d504e16188696..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 97805 zcmbq)byQnT*Kcr#Qmjy{xVyW%y9Fs$+}#U>;uLpxcb6i?-7Qd@;O=s1pXYtQ@2>mj zot2!G>@#O(XZD`oY)hz;f+Pwe9wGn$K#`UbQvm?ryZ`_Q6?n)uin9#K1^|G_u?~`IP98RK#(V| zQ_?R2Zs=525NCYn`KNovjG#!+6AL6p8t$xE8SkiPEMbplREcS)(bc3d z`;2gCg{t~|O%Y$Z(=Gh1Tj*!0ivjgJ_D2uD-F?Z6OZRnU-yu`4z3<_IPb5Q(TRWG} z_mANt=3@7IIAs04y5oSHySOX;wl%Voo!wdwY8Ui_( z!I$QAV2>*PQ{9rw-Hc{-N(!hm!PlU!hl?!lo{kaKc)k>&EPa4^l~s@Z_R}B`5PR~G z)8qZqxkoQi7Z!E_;@6A(mb#zldsbo8;IhWyK&K6XjVH?-Oy@32V)1re_+ac}YK~ID z226Hy`A|Mr0G?Q;>=*`gtkfEGiWN0I+!+63n}@`VlU?F!En2TU;8Ot>%tuD4`>TRh}`ku$rx(jP<*G5Z-_*C<96=_Q! zxRLeQlCA7t4NZ2n%`#Y4W|xY_YN)c*lgb;NuT8Cv+DC#ZTvu8j8~GIw;R?B~2Q4HO z1zr0K=51T9hix?Nx?Be+E$!6&9-p9Qr$0745%1iBDJ>&l=c1`&XF|Emh!d?K9bBN( zyYcH5RV$W$IF06-ri!o#hZ$qII`;WjXU6XoDjs#lJnA;Xfc=n~p1NX?P(iOyEdb}P ztsA&vn}wEZ;+q4J2@5*s^@Ij*+vwfr~S1-pm25?7T{t>Uqx*>SM}7N_+Vi zmLC}IugGsJmRsX%Rj5%{Q+QV_xigl5p9<;S)wVKS4|%>No3id<$F_(xZ97IL-ojT9 z87q61S0YW!cX~m&*K+1j6m@#0sa?hq>K&%nGj z5%H+S$8*huweN<))}|-U@4S@)Ylh|PBuHuVP3~m00&0@ud5sF^0=NZb+++lAM+JD4 zp_w+pW^Eb53FX_t2<}a&wcy4=8+$-I*F&f+jrW||P-I&=7cGSi5lZh2^Gc(LsGbhH z0c(e9u`r>oe)+;KU-lE-#3Y06M?T?leBk!DtK%K@dE#kKDjLDfF`Au31^^)T+bw=t z+s(4DYk#Y3F6#r_ea1%P>|VwQhGpAeMNtzXh1C{w)0p$ZhoTDCiJCT4)|0kdjNp0r z?0k1w`vN$2Jxy@apb5z_u~RV=wjzD9r(gl^k8@|?MBnFBD~g^Ep3FKalV6g55X}{C zg52P-pl#4^Y>nf`sTw?sa~&6!E3ynkkh7R&O0r`%=tDV+#4YASop2vmZ7ykf*XMbq zT?ua|>Ygd1pC97$l`24L9}QgNP={wpBi&7_Z*Ct(_9Ln31@TunwgFZNBC6bhytlSA zZK%8|P6u1oGkTLIV@$ngU#pZ5d&!rkQI>#i_vM1zIwENGz=gzkWKfvKJU>SQ#h@$P!009}-wV&(yI!gx%1!YrHQXNuZHy8mrjM z%?p`NRt5&nQcDQRfl0Dk60~)sw zLwAft0aOJ}*sr$C$wK}6m~wPhjcG%A7~Gezm>e!96?)x=(>Q0uP4g)=rdROd>K4PE z#pG^3Iao1M^p>qBNGG$pyrD}z3PNZ+8DGv(r6YU_Fvz^>2>DKUFC#s!uX4UZ#lZf_ z5$z<}Rz?hG5=AE)z1D6c-7&vVqE~}1NYFF__>gU^wx~DTj`vG+N*q|H6clDOg#C`? zs{8Dw{x;ix>9fAK*lg`pdgy6s4>X&itz@>?_MnSf{)}hA3bHETNh*lId@Q(Ik%}(8 z)WO!$E~Au(irPJpM<<_vXAhNI=in#2As-Dtdt0B9RL2U=hwlQpOx27hl1#Y8ZQ|@E zC4yh;%kDVDe?q)SqVg}ni{~N6SR+QWwAXDkzT5sf1P;s{1VfTWfsYp1ZOF2Xyi-@2 z#0Y-&8~Om)rt#QvNiN;>4LCyOAzCWC*=T^cWR(A^4SKgLeF#tvuGTTm6i4#W+M}a0895@PkYG z&ISBT=1aVw>FX036pD;uN{DRj)e@R=c{wJY#k-_$59u^)NhvcFVxPd=wNu3+-@Tfl zJfh|w{BiW7efd-D`LpNk2=JrhdYo=w+&xE8ipDsozMPG~z>1w3I!;gs6f}k(k{zUc zLDwtdI;;$^lVXJW)@h53CE!&9aja!MCZaI)_3#CbQdl%D6t^vnej6Umf(lP*3N>2@ z9dYyK4sNCy zyBHk~cnZnI1A6j-5YMwts)EjRld1u*FvNkn(byNq zzHKP)?)iGPVnc(9mpf`!rw_eg-SXOJw6M24cSZAie&uwzMbhkJ>t z+V)0%Ck30RZV_UV#2!*lD15`!!q)L))>!cj#}21|hb!2e}Ld4a&XdRk^h$=b)>CPk#1=)LDU- zvy22B;e+lHOj{PgTQFc0F};nhodH1v-(ZHkH-w037f_0R3hJ$XzLFQ9l1bQ&S zXHb1ZQ_@r7GEW97U(mO@9R$}vUaK8YPuDD}V^Ih~b9AH$-;q*|O5b^lGm7l@!+(Cy zdd;mY*W=_3>K#~Uqnfx7xZT@Vb)|?zWi9vjZtY<;*SAaU|9jk9;@q;wm?_+bbt;GeQeG3$_ z066v&w~BwEU>4lk#C}@s&!6;@b-Oq7s{>%Dn8M0f&Y#_%m^8AFT%z6F8g6e+Q_TDX z6S^E5q>O6F-i&($dns)v000T&&l>`en*QO92JdiuNb^K<`qBJiJr{&Q*g5GwzF zQKUXGZAJXO`?5ay^>Sr{JHbu7;Ru5dL=P{NQY~eR3@scl9{&G`}vj zIUTTEJ~%T_#V0dfl^q`eSxEokXJ^Cn(E5A6KjD>Z|JLBaSTWFy zzAJl#h-l6Ii5bl5<@>Jd7ZMkyF5W*JBC2T1gZ%U}k{PJzu%P^-D0=(VhCs^N@&!NK zD%?QEuZ~$po1Sf?30n4DfvNl z(cNxxa~V%~Bvn)^HUp?uP?e2H-B|Befm^087#gNfVB`6)Qu>~&{GZ=%tY0{r*`dzx zuJe&Z-!*`$O$ZZLb*>9(VC^gYR!g#j#GiG1aAaD!O1nr9ZcI!otIcnNIsbtc{-yEx z`>%DE=}KL9;^AK!q80w*e$FS0JKihG5HF3Ti}-&f@Jh|7YN@zN71w;6CM1>BtcAcZs)%v~VJ7NW{R`*_;aeq_6n@P$58Sd*D zy9LCG;PT{;gK=)J&<<1KWkrmuJ}<5DDy;RlV9tC+UBWm9uV$~E6G`^@?(kjbIZE*G zJ``<}_$*I^{#6AdUktXFdQXBVPFBA89**YhBwb~V>YlA_M3URdul#LI=4P`>{t%Q) zR#ghXerZry2%~_)$T)3fF${TSfXkzQw!m7U3a4L;){ul($rV`8ij;L)w$BpQru_jL zu3^TUecDGzlVY9Emxhg}LhGm6+-w6PKJU_c^NBQ!TO&O@H8n`vC($^4YTBXaUgvh58=;%w%-2Rusn6W+(2S!qX7Aj;w$lo<+`4%-fs_K0GQa> zf%RXM>%v=K+aHe2@cihUJ8tvOs#r?^Os!ek=5C-W96@M*9SDN!_T{d?)V}xj_eg&R znKTbk*G*;I{Gh#*YBd8RjZWD};Hh*orMbL6ygkSp9p`8je=Rq5uP36d9{2`2@z>Vr z=^1Zr`GZe#L;gq>!Gq6bN^!g+xsNfS@;kT3{b0dyiiO(pql!gh8ASyg2mmlSRYOLZ zyQB&RAe2;4fHfWD`%#zPV`j{Gsjh3mhX3|=gVaioO47K8r^i5{DgwKZ{W2VyqcRQ| zkjJ$urZiO0kFQp1sq)pHi0~8XO=<*=;?{I{-+_eLF6?ju7;WNc&4q@fL85n3Dvrk2 z<#aGg^}NB3@cv0!&lCL@GY2Q9Qpr$6i-RF(cf6m29(GtAQfJ2|4$%@?#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>4irXuM@#0%pb(E#)AR&*S6!KWw>^yG;~<9 zK1C53k$(@TrBBlDsfku=QWYFG?ZZ#)?N63aw5lBN(=)YH0AhE6-p8eqc)>RG%(40t-rPC?YRRLZu!ah1 zsQrstM?8R5AqVy_G*Cm0vWCzInc+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 zMUxSdFsh{#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>!FaV^$L0R(e#>h2RFAsq3OcQG2yX~v-!$q zQL9Ajfr*W_k$Rc~ZI|O`HHj>K3}DWb@3fFPfy+quiK6Rj*TqD|OMI$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=nSz3{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#oB5LQ`zvt6AbR@RM)i}HmX>DBx9Ui7FSt*Y?ru4ilVrvq6(cHdO ztk?ZlTOQJx)?cEx9~89TTVHZm^#Asl$O_qZ@xuAI6lY#WdN!+THG#g(dfBH-c71ek{1V#BGr9}oVsgoIGf#1H5T$&A zZF)n8Y3!r~xaLv_VJ2RHZW*sE2vNn?6B-@m9;o$g*Um!uV|>5F)Ove;RaxL3{i z2RQy}oz5n(A6M7{{<#|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(6BxDtsBn*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`gezi=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<)VNYhf55=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!($V1O88EYlR+x4>uFMn= z`xLHWH8)*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{6qO0pyw62wgA}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&lP215eU&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 zGR5duUNbgP+(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@+EkO^zI<4*O#B*$@u7-U=p34AiY-ct9@Ws>F3?f(N7No zSB(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>qbGRs1z^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`ojW~RfGMZa)( z`H@%jO)*zHpIZ0&%uvt}44^g%oP{vC>-h)g*M7jpc;%62)b8BdEu-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-elf8ej4*X-6(89pBD~ z$K8(u=HE!xtS4r3Ys+rWLaLPB2#k86`$ z|I+#JEfl3q%lkNWecE#RvC(4nxqMmGJY$0-cpd7j(#pcp@;(MS+xaGXym4%4wYrWoGD5@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_u87WE{#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$;O53A*$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_si` 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<)>#Qi7ipXpCdI;T*XJ&cY|}gESM++V@_J6}6^&q*xC)w+yB5xZ^!@@$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)=}J1GUL>=$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_>ZCt07)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+|XaMC{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}f4ZclmH*Bqu7 zd=>0DE;>bcjbwkK+t~}2#RGlx%y2XJoAzakFj#b(Td|W~SDkfQ@*Hn{i#ilI(=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-Wo6UKiBzW_$@pk|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)THg&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;onFd7mgwww!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(kkpyfkLo`yfGC&%Ykz+)0yo&3GpXQ|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!upAXyOHfKqcZr8KZfKk~$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 za2#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*oH}O-0|hrLb~$e=Yv%L?cvGk?sDI%a;b+? z>PLz1IYB}YNCn^Yu)VhW4!0-m_)0!>@p)WVP2?zo9o=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&~rrxwO344J{Q&xNb7Lc+ z;PZHWIGGQdJ}N6Eg@A$*6CGVuQL%INkd~I#*x2}Vcb?o~71HYj*?om%ukt>s4DqEl z^MZ*LQ%Ig9?v*q^rmw;Jyhyn7cWI9M$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}3puAKfVQF6UGCopB9(rhH5LQIdo1O+Tv$MC%B^`y$+t7Byx zjo`zA37biAn;k^4k(oPQ4cRA7hD_%}4A%c>Nv!Gn;M=~LWpA1Up-|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>07kwaKU0`}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*4t5OsP8reTEPc6_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&{8EG`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 zbkaQ8PA7628tKny 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$PYZg6i51idC-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;cIZzc7LlyjX!` zZ4JK7dY?WvAF2K|63ERVabW5&}x_lM~Z0398UD9_`pALR)qXUr?POBI<Ba z=8K0Gwor1e{2v;9Vw-53WxH^>&xI+>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|VMt1zOgUb4P4bSYubkV`FpKMDVRFxr^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#Ojk^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+rkcMTj^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~=z0JiAdU*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%jB}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`HYmR4fQ2`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(gMoHaMo6IxDpQ&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?XB3x29d+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|*pJlBRqwfR{3NU}{h`;yuXPr}$PiE-#s=yr3X(&=~&Rcm=`7xSZ!gb9?OjagEKbinpzl;2=x(;iKcbC_Rh{9<1AV&%jM31 zAg=D~GxK(*oQr2kU2dPd#bUpCOPxheOJfX(5TJt$Vi`oGc zU_XkUe=ZE2!k9_J-rnBkfF^ku5F+#%UZgM@>ADRjy1fx_-^2{ZC^|`DTGL9!yk@^N zJ9=WgFa?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`qUD86X${ 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=*!gIjPeuXKw3Oaw$tI^I!khFCg)ZtZ&Q;w zk{jjBS{m|$wvNJXfYH-<@saB_zIl;gGIj#oJL4DHb%AsqrF?8F1(1Lw_q#{UD+svY zh1kG2E;;j(=-@r_laiAWo}tN4oU5dYvN&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%cH$}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>ePc7bLNtrx%<-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#xqZFKZF^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>^JpMQMVqj@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~NyRfDScuPKam+c&$*J~@YjL?B^X;9h{wEMSPWbbZE`382B^3G!(YxDAwxdx# zw`&zte!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 z46CDY=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)VKrB{1}pe_;r`+~Ojo zA2%7D&t0&6SudYJz)+xwL5x!NjxCRo_}jOyPNb7FP}$0*b9g%_+IYklfinS#s$$^P zJ+{+#+Z-SpgZ{(bM&LC^#0PX+ii(oTdmP 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=`@9W6U()JUhDO zU|Oeha=WBRuF6x;j2#YZY6h;&?Lp-+i!z*Kn$?Lx9;6!U2aEY03w`e8W6`R933>GJ z@OrzTTD(2ivx_;)z-}0EIwOS{uWocMk3?9@SBh96rsPnJk?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!%a34oobE*y`5!~@N(Q= zGUrUll1u&c^lV)OCF5y(`0=J3PUyxtl%~I2QzzceemOI51#^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#&5ygH2FiPmB+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>*vYrs<}>@|`&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&-?svqSX&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 zfXIwOn*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?`2xmm@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>jQ2wiZajgHlnsjgqGMH5g*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^lqGVno%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<9p&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~1ySu5mL6{--8LlPqZB<*KXe%*As z-?6ZbRl&Z#W_^8qZ=mXvW>9+Kd%VHDeF=g4#%qT!Y-wIT4kQhdfX-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(H5oa^{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<#=}bRN%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^PFeXEgOEudxR0G~57GD`XJ z5qO1*V-1NDMh3%s+!jQQjzrvd;I~mk3<1VY!i9dTxWa0X;Gl)jIZ8=tYT4|xfrlh( z>$s|d_9PM4gjxRf6*^=?LeFXM_4t?6baGJR4IwRDnlGPMYOi0fD91(}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 ztG3f^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-t1C`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>KrMkfO`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}?SZp5Y6-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*rmrRDAv1e1wDo3kHF^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;Las$O}TVXZF0MF ztUxxeB@9QJ4}5ge!?tJ(3?3jkLSE>mY`{vOl1cH4Ow8D6UrOEOY`Jix<1-jjwVkIf0;|YWz5k);- z0!GGWNGutLYq;gnFbs+@9!}o`epj~hlS#kzMKA+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{DAFvY)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)T2jb9RI8RAoKv$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 zW1hpjmYZ%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>36KLG 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{DQGp>!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@PzrK5@z`J<8&iHh?6_r*! zBUO!VR+ISd4j-WZ1dWPlR6#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)9yf1Q0{fOCOGgz z?sC)xU6XD5gdVX*6Xv+0{iD=1hJ?iP9ydCgy zDz9>!64runBTo 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)%VtqB@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+|ad2?(uG7%k{!PafqRpd1ZA`)qNg6sxFbr1QCN4UT6 z1gdW`*ghxRTOO7#6a4c4Q+IzxZD%x~HWSL~s$t=^5RBEY9IBuJlbfKFyk(|lp{8fT zSr&4VDA&bYQgiMVz}YR;+?OLw5Z+(`#Tmoi5`66(zgLhRA7A;gu&lrLy5AntxV0QE}HaqM)&q)|jZ6IKo2ciru zi`}5HnPoUZ#mZC4^LS}Fc!g*kUDk|gbkSBL4pTCZL;g;RbW$lC$t{_zw!SuK=StwG z${0iP$PtHw2Fk8F*Nh1^N=>j)mfRv6hFUJX>ow$*;kJP~-U8#V33D7Px2(Q$q<(_z zJ*X^J=RiYX9@k%{#U%Ih)U|M&dFC}L-XFLDloQU{I^rD(f1BG-&SnPX(PS74)N%_H zWKe4>9HF|$eZZ?Zr?W1fXqadV$IahsXFsmbf9ZqmPD(I?C4%5p*cHRBlaTGhcTw|R zFG!2*##XfWIu6x9V8&#T?Q#TlfJyD1zCSuL_z>{;BPAY;%a*Lx$$QtHryB^d8lfMQ z!YLZod6*zZAu=y5xSaI`FMwJUQUDi?K&+{0$CrevvSstc1U(65Azqs5d^)L1nE7mf z1fC=;@+c~H`aQ!id_#ADxP(ywRpcMA!Oefb25&}u&`oE|T2})gi2~<0w z345<_**HrsgoCD!w7!wU-9aZ@J zKk|oKh#aAwobg(@zM|3|x&sgF!N`f%R*J{QGtZ;<%z@DmW6!5#mMy&TX3fC1#OzCj z^f~#FMgBGa7&mOyCPxXCb2W^pG4Sy;`>S$<3=X2^8fW6w*YkCi_V^2PPTy^#x8*N* zKRU1ONML~fG4(r}J}G~2Td3XLpwZ2ml`fp=aL~!BkQ;&)6y?ClJclw zMJiU>aLUg^7B^BUt0c6^H-|E-Tu^0lIFBn`dUT|_YD6#sJ)3g~Ru0?0>97c)84wx79*e`Q*+(KU< zYm6q*l|m%oNJb|&oJ0Mype*w;aWI@N4QhXaV^RnI!epK9LCF)U23*V#t07xPG@gfF zSe@Zu)YP7u*{;_VuN05sa+Sv$Ic_KP+w29PAQuhU`9D8agK`ahRWl8x%)*g8h>G}Q z&HqSK8izr1&7fM)R=t@(0NL$~|7K6+XM22)W9~7gOy&uN#BVj3-1q_%Y!?heqxW&0 zV1%g^xr%{Y{!j_2u#)tbm*+$)y>@GOdv$fX1<{5(f&P(NA1ivsTFb#6C?!Uwy)n_P z>)!y&;+F23O#CK#MVD-RndApUZx4N+Y;%=|N4sum#x?#(q$mf+{XY?UU~>g`b&O)V zif+`!l0!=@p4iPx2le`-QR+t*19CaP)#TzF`5S;UCQO1#07>BW($$>qe1Qzlgm+;n z(|nOPKpGjaK6IZcI)V9$JJ@4Jvz;t*Qecn{?eG91 zB#TifUR_8MA--8{5?$FhyqLOcucb5{d5V9YH`_5Ub$q0QJ~YG7n_Z~8_ztYu=M=2SMg9gs{ zo?l!oT3fBD5+dDby%egiTnMKYq$4-EIFI}7TXq(1UcP{iK&Hv5`&yPzwNg4YB8yo; zK-GChEAgE5Bi0P;o`7|1#A;P$u7ZJi_tB(4d>+iqpSucoD*A;Hf{4j9RiVuDMZH)- ziJe!tYs=3I2bi#3tMN9a`%S93A>vhL#KXu-#`?2CV1uILQa#WyNCMC0dk2-D3ra`86EwJ zEsR@O2Xi-+TE_FG3cTw76Q;21CIv|dm%+n)ZyH)zZvIfloHi(8VJdVb;qi^wo~h z+wnODY3MuzfgXN*hfOFLZHsR`q>>)g$>h7zuBlfi&8N`_Dnn9S@Kse1_Lkusk^hIU zcYuz&>)OVr4VyHL8{4+oG`4Ltw%ypaoiw&>+qP{d|7m;Q&-=XJ`hVYVWzEdW=$tuw z?}O{Q_Ws2LESS_Ek#6yW#A1VR0~UeZfzx7SjU)&+GIRJ4?U{j9R%soj2eCFfwzFrz zcQZt2-2U@5vqQ!1jiI3nNFaTvlHZueR^SP)j>EX>Y&JKUMTm9Fs*vaVmgoCDbTjNd z=-v%SX8U{BrVqrp18aXN25Q@i36XVSpn%J6wXYqM)^76g^ zSL~xoeCSMWis#~QsA@BX0N~auEiM>u{a`cvXd)EM7Jn6b2Wv%0J*=Ay0YADr$(}h@ zB%Te=*ZEdA#w)6@H*%Eqe+<b z9!5|z{8rmG_wbVrb1k%9JKQ^)(<}LT4qV&nt3etKze(dP-1^FAY!#2XXnSc^ETfK!@dfBqW5={0k zilZXvXO}uT<@7pOs3E%7KU{wPj;N4Toz9BSA)K^YtyudDxXIN`<#p6Jt%$(gsbTYh z9Necw*j^~AIS3wnD9-(E9TY`-WX;JTFNWjoMVsuq{l@&~a5T z6!#@5~K ziNXDYQhMFd=5t}reoD&NjsF2EG24CmDaBVwP6Wh$dzAQ?(2V_5(v_*%;p#Kxm~Y|I zk$PC9)GSxjS0@IMdBBuz(A+M2vfYJiaHY(I22`H8HZuMe;Es8TFE9}gvHfjk%~si9 zWVL%YS5(6Zdr~Doi1GCsWT%l7EJ+qhVum0H&VG=#)|*^Z+hWa{IL%jDi9YvCGy`2(#rS zCikqeXAuWsh)(p?oQ0Z9yp4UIz6qXXQ@42Lm|C??hwF=M_o9avKa89_EJ6v{7VXwt z?ub~u#V2g8X}d%92(8RmbYime^jeCqdE|Q|B-Sh5w<#qV*rsOOjz#lEWGc?<>_tfeAkqVqbwJD+wV-kjGi;Er>)LoY*3E-oy6YMfiz2QkG{4;>@vn+;1%WHONI z1wfZ#qe~7kV+*LX;u-2op(@I5yv!eV29xLd7&N|#<}u|_E4LSB6FPG)@`x#n^Trk; zAyXTLTSQlFni)V|QcBH$Z4Yl0BE*F8JtLEQbhTB*76sH#^^Y-XaJy1Er2*)% z&w#cgZlcFsx4LM(7vj^pJ>6cur;WqMT4$za?XM#a(su<>2Og1bV#MZ}`=J;{BuOEn zL|H#5Bz17u8XKQAB^=8nn4PXyo{&0GrjJeC&%O1Lh)7TZXLoPxUz>|<30t+uSp0zM zXlEB2AMa6XBQ(z{HQtReFCO&?)!km~sJcw62K((>Q&0a11}m^mkT0;l!Oqn&N&(o% z4+Yb}x)#m>FpUKT1;ApB;;?{@87*>kDxCLCALi{R9@kvPFGsi^Huex&Y+mj?OX5qN zYz*_@_b`52{K}%8qoW$-*NN`Yd|+kMWO>*fiEdw)w})FsfvLF!HIS@-@N|-?06MFk z=R>Gr*l_SZ$JOVRC+`Q;yuQ@-yCjrx+ieZz7Y&@3MBV(@TIEiGFqm~W5zn}8&?F!Y(DbrY2 zTnJNo<_|A&_SKghk~%va~WI}{!_o#KlEL+s~qbQ&O*KQcB2YATg#9hg3<_I_m< z0VX?NMj@YDQZn?-ytegHEM7ud295>-=O4YvKVQ0Gp_5#OuB^vgTy_KNf0nw2@xNjG zxb2kP#dVk6+3Vr@x0_`i71t}nVWxWzJ)t8U&*z!1z17QK`-Zttip?jvnEBFh-uIjZ z>V}qGpCV{pv-&Ou&TvX)@l7W0-|U|;q+U*2k4@WeYp>)b2IiK@*B@{^YtiF-{x=V0 z!{XcCIrrn{W2baQ6OBG%+d^}Ja|ty6z@qERZL8bdeU-(q-8C>YtaQjI$EE5`VlXX! zQi6eEz<9s*D`;Q1d4JiZsY`pS?@jS=-avk8e0RIw_k1?tEen&PQEtz}PvV0tzCL%> zb{=n3@^G8jOE{zoVSc+_%XoVQF-8LZl1pz#lZimu$w?T>+oPi!Eis`yjzG-V0rqCWAwVw3$oe>%jK>WZ#Ly%s?|<_OBzQG))sXQzRcrs_*xfQI&L=D zs&gwz;(rTd4xlp(%RLIAhvTrVb9Oj;gW?emx4j<*1k2YScL=mTfrAA47Tznch0;2m z+}4lTM#WX`HmapRs(Nh^ORRBp>saWS{=-?i{(`3XD^|%<1lA9fuj{=ZNkZ--4*|$y zi|Y_pTP9^j*-zZL7Pry))jAzhK`Q41j~_mE7qu|Lw1+{BM&J++tIL(s&aUxmp!Iw_ zqX>$8i;*Q37xxRralcEB4)fqVnLeDEBDe2ts70NhjV*reFJi;-o4Q07BzS>C%k=CY zRzWH*GfE*5h8m>I{jA6Q8RgdlYvakJVnAuXGMXSH*_K>f{dvRRPR0K2)Mv%K*iv~v zIQ;arB^uWq@Ya+3zkM?NPoKhrEaTelT6rFy-(L$DN`1a$@Zzf}z`mC`-hUUB2C7-= z3BJ{}-iij24T#B))fZ^#G_+0|Qx67rL*gkZJx8+J?ozJFe7OWFxERn*Yh5y9E0-|B zxA4MugGRazeiW=955aA*VMG}+SXbAsVtyhk8eC*Zqsqq?P?llyUq~tVg{Gmah3%vj8(6>5N|c15qB! z*cD2ON|>%D>&U5@#9tfNUV|TGg-&aKU1Mm+YCR8lKeiN&sTF0vGLH$TiSvmE&aG7N zW*Vcf+~y7{!7ghGYM?&d%<>8T3u{pvli7jCb@Rl2F%tSptFjF8H`4 zI`p-}B?r$Y_+P~N)lC>J;ndB>;3d}=+WJ<5)jq#%8*2Xwx<)i3v3VzbHt*A>JYVLn z4+<>fGr==6j#HcSNq$uIV-f^(98jNG=y{&!)2X1PC1^4}WB+;4$L9M}mn$=6wNV8j zT436)xAOSF^O(C`Q<-fK(XWxlxa0Zw-iX#m$yAhe_uUHDLB#D=LEkLI=Jk@fGU$8$ zNPI%RZ)*Js1;r^%{$`rr%IH+SmMF_5p$Uo1?bIl0kGDQHn_PeYP0IQ19-!r3uP9gh z8;r&@dOp~N#fn>Bk}2!PcY(tJ4h*>9q~}|XTy0u6HA0GfgYIiGpP=sOUaGooJT?Ir0v=+>-Q{ zqU&x92Su22w$eEDPvIn&tuG9%*#ifM>+ROm4y$@Z;njx8E+rO7De^VCuSZ7pD6`*{ zH!=6WRffI^vYP~g(ZSw#jL*N8B3>7f=sP>>IzWk2K8e>V4Qrg4pOoh9B79adhG zub&vNJYN~6edB}tmQ(&dKX{=H*?IojWH~tRSSlG!>jwftvc7CPRwPMLrfwA7{G1$) zE!Pl4$yUTC6Qik#OoGc_GHRHt+&x=L@f)ahFJCekP^+->QlxJ1S35p#UkIpKUyH;O zg8Z~}Vo<#G3E|Jn^(L!cDv`qR@eC*}39KlkNS&9HmSS6mB=TZgw@)T*3wStnU+&^L zGjT8ETrK|UMo!F`J(&{SNSFrUn8tk>}f?SvUyUHBs zbQwryKx8d+ke0Zq{(NT*q&B^#mV3P&x$H6d9>)q}`Lg8h8OZ8V-Ow0LA z$NSkPn8rMc>8$~yCyF+;~!ObEnu>qs* zN&*y~XF}@D%e>F=`2ln~&cGvUf4Y^4gq%YmJp3IhE7UFhM*tM49d!F0>Gk4r|M*=?B&0ohV@#!QeN;9WA|HPgRNrGtWeDO=c&0r}&*vRru%J$ZIef#Od-8xru zH(-6utGRQR>y;|a!C+|XGnQXq0!}3~y8q;ej|OSBi_|fzHb*zk2ASKxl~;EE;k`YesCY$zsix*C5wE)%^JX(yCKR#7wJt zLw?LQh9c0(%~69lv|EJd6GlBdxaT3N#*OR*Br-zIdSv|5 z^rF$-D_`b-JK5jY87KZC+G94>atzK>Z}qslZ-5}yns#nXweIfv+nW%DMSCB*D-Q?z z+dFC;JdMjPhFkugYh1NTToPfqZ%>-6x;RvUL zhYA--{t`r9S;2XEl+!)y^=V#hS!zQ?GczV6L?Y_tiy~{|Ju)0C1&Q>f4AB@63`Vps`bH%qZ0V?`6-YMDBIm`6+fptL6e|$NY@(T7HSv?% z{{P6c%b5?5>fkW^I;uIN^5Up4m16%&$j7bByBiyOPB=}z0)K_NJFpsqUBuY|sMnpg zX3Z}n4oZz54aBCG6H`8>f<@2k;35PCu{h*LZOX! z!G^4Sh{mQP_{4t`8hUzdxP$5Fk?uqMz*kBX~_{|>Vi1cGw| zA@|sxwI;U$R(la6*LSY9ewFyl#>YpD^nD&6?HvNgO*&_CrQ@#};lEDcc(rR{EwMrs4qVx9A=9pR_Ls;pGMSeluP<%61oHx;hPaKD~6%wgGXu=6q5OKzd| z=7p1(_B}t~Lp!!xBld->bacF=LsR(_%cvF@sw*yYxfj&k!kmKAi+00s^Y&*{kNTL< zbcF=J+ws*@g8l`m^$$hpGCo_ENI-^RyKcwnLE@DREnQ5Y55piS|IvZpvHM-v;^w0M zD^v5*eWS-0e1zwnv7IWM);UsUxFY|} z%mvS~t=kuf^#^|z^oasmo`H4^DiDu`u8O@#E{R6IRZs<1qyBw0fS{dkyWO?k`_I2r zJ*|?&Sh%BTH>!_aznlTd{-;a!dLo9t(*LCd0U!AcaQxb1VBlV}mW$Ye4a*1V_mbs| z@AAJz;6{+05L%D3y}|wMDsi|(09TbUA6+zJa)n{F$HCc-?To(|>ycDNozZdG!`R%I z7d*vq_}v_qX_0x+N@X=U8T$uX2@v_tJ{!Ct?a;U181MYxx`q}w5DWB@42ot;y`M?x z`$_|i^CPs0FVN7OKW@B7d`MT4D))aN-@Y>CEFcTVggCi{+Z$I~RdA8L>0><8XA68t z0z=3Bz^UANw-Z6S;p;XW}azy`Zbh z*|Q!u*$owHev)|0BJsG!B`HjknlIPia?>8m6?N7n=+%qUfbIy?xhYZ$)5-J)tb@^p zt4Yli@8kar?1iXg2qY(0Ura$z5aUqP;2_Rzz$X4=oOoy!^|goJw*~CW_9iRr6b6j^ z1F!kg`}mqWgrKzsGzbsFa_yAE?OJ4HB&aYhT()BcOdT2BLL_Uii00PLuI*CDHskZ| zE)h-mv{}D{+2VqAC*49msyM{tH;rG8pW@PzG|Xvi+j$fXz@mTn0esv&JiTf-T+bUF z&(5&Z8gt-3u%gZh8oQD0egZ$LMw9rHvG&H9QGWZRqIpI{#D9DzCquSyrOn+{MrM*Q zW(-l^lKtwYZJ3FxPctP%m?>#G!AznOle*0=f7A%RkRiYP;j&332*FCz`My(W$fo}F zMMC}){!E+Ef*y~-f1=&qc`v!XnN$+c&xVn=~u&Yi$DGxxd)f~N)&@Njmo^}L>~ znV%s`oICQ=*M#inlOorYRFoY7>!v>4D_c}!42|7hZlwxmaUXhSlAc~9$MuF*9@(5V zD-u$_QodYYOzbxMlmrgNN!9O|?%xPY9he{*`$6+|)RwY;8ohMgV3aw>TA%3xj-+1| zk5q<3lshCEsFVCG(uSP9y#UN$%~YSbCva3~n$GnE{!25S4pE?l_-9wA%rdVT<#w(& z)6u3|Zb5m#CebH~kV@F^8=Dco7>!&|nfvCa zu0-G>hfe17l)}moMOYlL-ihVz{*|22{(>Jqb(Y2S1Dc~d9ILl>W?-C78o{xZkm5X4 zpH5_o!BY^WAV*^aXAX@nUs%g6qW?Z>xUgPLClXxvCaFYtGI+XyGCtINabXbXQ8ng& zEX!`cqXH5{t>rq;Q$Fm4mr_*qZPy^E-Xo=U9QRN>we^-H*{m9++(jA4?}=;CczKH$ zGySnQx)h&dXQ%G=JmMV`bqrq`#gT=bCKW&8O7;?osK@JWQR=adr<5D4`A14vq#F=pS_= zltSTaxb78Ncex&oyg0XjE)a4DW||f-)ypr>z~WO&zR%#ism8ygSXWEu1F4jMdwa}& zB{bMRg4j;IJ;!;w_>@v|09=;Q?T4O8#?rf(gwI*7+oYS988uU!xfMwdj&haI8(5qC z`UZNIFUAGej7u;@8zuJq36B@^3(1G|GNHA$p&>~cEEd&oMaG%<<~@UFr)8vVQ#kQ) z8O(_-L>37Ffi*>@MFFF!K@;S)M^Aa)iKt^33v0h}>b(rn!bvnkdSP&Z*d4hTI%ZWv zPh#gt(!5y5w!<}m5%9CQxhX$^9;pWmVnvwGjl=q9*ZtVuQE91B<)N%C!psaP0b8H+ z1t%;%kZ}Bk;2O8jy*y;hLQk9Q+K^`@!@f{d8f0N#r%;t5kxW9u=?6d zeGD||+`xX5N)S*GfRoGc&@z|&$wex1>THZAudfu>CIKYqkdhS!osDD_XDQZGkTxbb z_7A3eS2BT#9fUDGN?q@;mR-cYL3%~$;~IPb$1pn|KBF!GrC;?MmtDp{$H&{naN<8X zET1sZn{?!sPMl?$Vc$+0eF!{h)Z#8KPJ#ThpTNjgq0@am&eQIAz7~eTvXOHMBNvXwEG#VBj$ts`Slok(aZ;5{ zMe6thh%M88%a%}u%Czdfxp;pV_Cg6qqEuzslk0mzT9<8ctf#x#Ja>CmBfv(!?`ZOB zEmR_r8TI6JYMQ&TYG|^Lg+TlQ_%%T^tY|hR3$R52OJP1G=bEZzBtl1*@~j$1syw|E zYR4u39~h3YQR88E&ty4|qJ1feIFpRlb~mb6f9G10gu-+bm^KfyPYcd{5rzjiu^*+Z zO!@^OE{nu7^9v2q%&T)+i6>>0Y%T5u`%AK9`PeUc-TthANjM7C1OVE~2YckUiWT<9 zWq1d3=&N*Nk)&(#9&Il3n!@WsLDRf#QaDV>k4l!%DC7j|e4~f-W9yq=7Ct#Pq<$3r znx2kzzML@KO^+Fs7ooyRx&RI_yihn{G+d|wkwm7W`#if;RYV=AsZ4870dsZm0BLQz zTYwW>jWAnNdn~$$tjM7EdSfd3qu+wiTv$|AtHH{7oSK_NfTZr8NlA&B10TgART2z! z5I`@(DJA2sVP@x$i~19;>%OkmHG8jz>|>~pHVr1FUIE`r%8o7?G-*r?Vo7vbkJw5L z8dH$)RMwzKkyrwa$Bh>Sh0J%ACKC$NXMN*hXgTodUOwBYVRSb9v~0A8R#M%*liaD- z*lHG)h<`iIOJTt|8fBWH$o9Ofg$vS(zHGQBwep0Qye=r#@iazESC=%e-C5m3ywE` zt*`g73mtQ(CM5G!ycBA>0V{s^gr9Ur^=F7BaTu-&zut{9v=0#f9ZM6f^Jk9V9IRIj zmyn8Qn{4;TfF;+NOy^eKlnT5N2xcZa?k*xFL8Oy&t-uPI&_4A{4mC~}kgk<+FV+$h zBv;iivPf61OQP*gw^N#&HSI`wDrwD&BY?OKZQLinh1jPN+Uyo3PF`dB!bRb+t3cT^ zbKu=Pz~0^V!(pI&=ml=NWa5YImGImSy=kqSIRTks!lC|%Od8mbFdj0XJ_9DQs?Riz*&OnVp45v&3JgRwc47s@pnj+~=Swf0^!jN)j4^4ls)zyj zWuWD%w;?q3W?Ua0ON~F8Z>abaR4t$lN2|hO^C&Ei4>L|K4nsp%f6m@i%BcKN`YLQ~ z;A|bDgqa(XTBZN}o8(QcN=hZhr>z6$`|`Z-w8An{i|b&drGeZ=*qw1<)U|={xX~Xr zoVN~*PJql0!H*k6J+8HFLHr|9kWjS01Ed1Rv!x?@1EFDx;rq_%BN`P9$pH67!8>&y)~?GwNV z*+J_su2whoFnBqXY};%8^ibkl96wad{q$fqIygOa5X#C##92(z`V(^W7a$xLj?^QQ ze8EyOIg}v3)d~}wG@nv^?T}SQ!@ACgBc=KaIZYZVaB8Ijqo_b^>tUi`6Em}sv9Y$s z(+bs60RaI3+(tY)4kqi@ji~+>TingDR=0J@c}I_@Q(zqYY58>1Xbw>;_4+hCYyjAx zT7BhNClm3!8-%##=RF~k;=WewUO8`TWNvxTl6o(3oy3Ts2`VUO`v`E7YQHWfDcj+^ z-|=ZIlgXf;Ls=M0cz-xKJ_^I}Ft2;B4c45F%Z7zLDLrp8xefI3?s0JKsX9lY3Q!Vq zzNih`EXQ`ciO3M-c^P4tcF}y>t}eIJsNVj`>bV^=KK7~8ooV89Vqt+aDzuHEZ27cK znKCFSD5ILXzOD|4Z*5Hw+^}soy6E<|QwKuVnUBtIbC7m@`cvxE4#LFPI4B~bpLN9q z1_ow3*Tvt_+=1Ec;uz=sZQbyw=_&1H;_lbgvz8@=s%Oag3EiaKb?G-?Z9;#SvuDkX zXPUf$1{#uwpeeCkAP;Yb4RQZRPk1s1vEkok#>{7;~GZSbyB3a4&AVPkB$k%8L#H=RpnWsv@dyfw!nv z%b6@ut3Lxuwu52{8G^U6XAoC7B$GKvSoU5XJw0_Ps{Pj!n4aGHIh7nL28%~Uf(t<0 zhXA5!G5WYdqIZr+++R7Ikm=VorVqc2Yg{UOWTOIF#vth`^thZ>d<;p@H}sVp2U?+T zoWKZxt!0!6kGBnos1uXl`~-FkeU0Eliw{Zi`OoCdr-u&A`cOK(05GpTFT%lSSPtfdrOx8HCG_`Ui0Z8;^+m9 zy>VcA=?v|;6U{Us)cu0Yk0L>RcaiAjY10S3}rBfoW52{ds&@2qjNOGMj>K8sO=Orwj1aXb0=Fp@|zAp7el!E z5xRimGeu}5E(Gg2a~R9tGy3orUyIQt@KFj41}pkiYu$s<;k>ie@D9g8y0huP715`? z&DnJQ%;y=lt$kTQ|4O@+o0~>nGVDzo1#N$$D3#>>J9_*5w0rOK{i3sTARZS>=QrN) zc_;ioE&F?8k)6c>(IlF#S@RY%^IRaa>i+s=d_!_<%9&^7lHY5+554|i%**k^l>1$r z`7;vF>twtD2nMc}=RR*|0q%{Tuj3bfV8?%#G4P!*|LORph3jLpEv|JHar3U|rC-J4 zqV=cXEcevj%^@16qFCi+iUht#X>Wg>kmU=3Lb1IgPhb3$;CnTrX2Kt>-H1F+Qw39GAe#-Z)Zz=0@ zPG#<_$W>>@Jntgt`Kf=hzW-?)9G_!j7P?`;YM!Jhy>`>1s@>64KP)BIty1idyZMOm zFArp~*&A#9Z*@0Mukix@q@f&tjqgwI4$fHqd(RRi^r9JC^D;b)Q2BC~4~K>y^BEQV zzni^3tL>qZ%IvB+bLQjedbTmJKCDQ@8H#53UoQeC@dSEdtjp>;by>I5Uvv5A=_VP1 z`w6T2cutxSn7Qs`<5(!UC85=qaXcQc zZ^~OP&T!7cI=Pts>HOa@`h+pU8q)^rag|ozv%E(A_*ldW24G!+eY?LCYJZOLX0tT= zoJT7?zh%xP!DXlJ{mr|Q7cy%QPkb@p9LL<|=!(T62}4qkmsXnfE2I!2K~PT6@6Wfp z|LvgE*z8~K+h~6if1F3I-k%1W>f1aXqD{9&7x%e7yLE@$BXRFO9ac(VOX1Y!rc@Zd zUA?~F7evt1-iN^0dfZKZxs8y0mwQQwe_e;{x8vliPrElcoV>#J=ovql7y+BkIm>VV z;Jp0h{TgKZM2KYvRL-~g!a+vhB%u!3qpTgf`Iy^dGbeuLHlkD%y|N%?d0}nxe9wTs z5h-ED8pJ8b%VyT?p)39F1_E0xiuTN2EZyIo=B!Mr;XcJwu07FH3WibI@gkQfKjgf{ zrKX!>>74%)Mcbve;sj`uGvbn7+sm4105Ad1fk1@Cn8G7O$O60Qcq%Z)GODh|sw1W) zt0^ZUb3Y`-0I45OYKCDiO#M0b)i_JsJbYGJJI&$kcX!SX7l+X^wd*Z38ZPx=d*)1c zhn_-dC}nHGh(hD@c2y3r9{|=@NBwh#1Aa@iS&x5#4WTB@P&6y8w&;0NO_ygYL z#_>yPRhf~00O*$S*tRJ|t45-bKmam(nw)xjn%EvAfr zZXPyUiXB<8L!Moil~J_0SZ5x*bDj19rZViN|B)@8$izX`!_#FaT*4jcAa`|;>TE*; zsH~6EG>bE0+G@6DwA}y5wRh-Al7=#tpM!(>D|@@z&;T@q3=rwxJMht{+jkG-kLx72 zbVjI6En;8?D&NoC75sX4X`oerHgmrO*8RBRJ@D74K7aTKNa=QNESfuXC?M)v-N55F zes`XE(0~gOdZ#>Pzdes|-2x8fvDE0jd(kcj?RMyd;bU(iT;@(wqOVi0Mhe!3D|#|z zmP(s?aBX9jS`_Q~wZ~(5f}S{k^e;IL9+MhMWuopRs*}EtySvJWQMpDwU}GpW!&dy5 z<#L`f)fQghN0R9^^3w7y4nWpI!|{^TPeN)g77_)w?$uOGM=@hC#MCxQU@{x9kT?ib z^&MfY!9klCV$17zur^bX1g%G-%?UJIM4ts zrF+pKHG(`IO-Kou){^fW2Q3$Nb2Dm$BBp05(bOr@7{shNe?z)Hj60aji2G4mJ4gHD z^m&={mz;d!WbcD?jOZk{i+ zG(NrrZB^xKQqZvE?$FR6Ww_jC=4=eZj!r#=ciz{h@^}=L6!we8MyZI!2PmB_SdOjN zF=D%K05#V2Z1Xe)Ya zWpiK(h zO>DErxtQzIb$bh$63mCs^%Esuv?W!SX;3oAEL(2NHL*^PfF!{*Hy3*@}Ityncmzy1*&>{%@aC_Jg zqEL-#(>Ad36n@|}ZN=2X4<-$BtJ0~cF2VA)PnHqneHcN|#OZ^P#|>F{u%(ZzgyZgV z@|AiY%s`2;AUv;0JjDj+TAbU<;*lrCZJ)F$lFN6}vv*7)CBm|pF8wWudNIrjG7!!3 zvboWe!K9#ApUjKJ!9*b9%F%{%x>y~s-2lwU1WS}ws%R=B6D63lSYQ(?dd`#E=^Qu#ICxu?v@JFxhb^GRPZMgJ5Y3ezLt$gEWGRVTxquc3I zm|4=f9L{dibDo6eC8K}`Y|qrEF84+W_|{9Id}%WVB{UQiZxj5M?xgD*94)199nClj zi;O{ww%3gKrmnq{Ih|iM5ehLzAlQ;4TI!2u@+E*-b20dyjwEr;Zk>AlH=p!7nHxBy zTHCKvW#Chicvf*MYn8A#sB7V^W3_^+>8a~!eIaoBk{J|26$`K@9bt!)15t}YOcC?P zv{Hng$is1{*eZAkU$Ru#$tyHLX)#BI))qCU%0)*;B$T`w^Ri!d&baUMOZ}qS?uBUN znvT=Hm&)tA{P2ihnUzyh)Og7XTB{;juLlbb03MTg9mOc=PVbkEY=$X~rIxU^kdT@! zdjrqD0OOU!_#$`3Uv6wdg9Zu?yng`*$dE`oS)JGjgb17F7z*N|kU6L*v3~(i%u0PPLgG*misrbVuzs`m1%i|wIw+v&9_5pX|5*DZwSJfACqc6 zUT->FRSAX%mX)g@pXF=+0#Pqa8jAkP?PBe(xmExdQ~Y(=s!xS7E;z8YwQv}Xr-JNb zY4|8AH6!j1|LX}BoqgDJJ=XNO!$U1NxDuCGgXz$Yv!Rr?>P^(7(=OPPp#+ z)>=PC|J~I^>*c-}1!6C1oqCKfhl~-&VyZ^y&(g3>$^>|H%kF|C69okYAYAG>1Pf|g zZ+t47;n-r(wJ?vnZn8fP{&TsB7-}^w-e2>kQ)eG83|Ik-j*srHS|+;5@~EB1zVoc5 z{~|Ka;C>I_;cQXhsC^(od&?jUy1jxZKQ-05c!-IRQg96$s{w^q{{Uym`IMkl#w^V% zvVBcBvG*Qk^1dxL*E+*0!`>AnZB&25ZWvR5A=5LsskMUT$9*>PM3G>3V(-gC78gRajTs>Qa)^ znvj7Y269-==};v>XS)sD05ztT2wxqIUF|6joB0HPyl_b8(OfN0dfWIe=JwpYm#gV< zKHHs{!CcWxf7tAny^=e+d^7P;YfQ&vAb{o+>^63jD$#FAo1DkGQYg9UDa$nKhpx?u>#2&C6LJfcJf1IT2#8NQZLn5TuS zu!Dh4gaBl1()Pb3B;QoCn3IRbegy1DDzrZ@RR#x#H%49J1E#c`cbuT(s!WDk(c1W( z9SCoa&W@glEgXJ@f5d(LeEcDMM7eElexM-{*Y~s>v zQBP9!4=Fnz6fa)*u4`O`Nu-O`FsdoI*4fX1j(f)XrP{+Fbn>5@UsGqhqU@Lw;_0Gm zT$y3=uBv6XC^&fcz#>5re(@5YK3s3smBswXmTYM<*tj2Oly@%R5`>7g7$h-8l`z zxq_Qx;$aEsyj6>J7#ZhEc|I>?%6tIe zp{-94(HD6v5UWKyO}0ouw@3|`GfjK}n4~69ITDdcd6OJQafmePM}v|0FPOUczT&mW z5;Qq*Q`bW6FjYQ#)0_GEH2CmMrzI;1f#l!IY0LyVgM(&ni4fAv9{hC*s5|_hym!ou z$p;I&38j7TefN`-hXxhAs@x4AT?i2MuF-0Eps5ewp@( zVje5mkEWo8ClDV)>=#KB`h6tAgaC$D{P58vY-T5o82+qC=~!j_CJ)WY%^W50a4Ax0 zAEvRE$}AO377ZB<5t$B*l$7W12naBCfNmUC<=_Un5W!;$OI zuN<-lZb)!ZD@EwB-xC6Rm+zXQAlEC2r>1yQR4hFw8xv;7@`@W|kM-d6RK74VYqIzC zCb5L;pCbrH-p^-Ohy|t2JZU&Kr{ze%G88W9>`I4=%1$BoG}xl-Jnt{pSy@?L1t}kJ z#mliC0=}_ig(8oK$QkN`#US|a;@$OOj5}gI)}-pc>mHAIsajcYeUG4g+RO=nRlK@R zC>`)Gh(NY|Pbw+mbQo%Kx^AlC>JJ%>=%)}1PiO33smDh=j}4pXk(sKvS%K61S|S_x z0npX0Fmni9s^n^B9pw0J2<_z7pq9`;q6h)NiM5NYK65hiNu%cYN0LXj_yh#ZyKr*L zO_VU?)C8ad!!=BE&F!wVJsk_aE|RsB23UscE-8p|wcIRZVdO`7sjpLHO)m>Aj!IQ>`!H{^1HH^kboRabweNy-$ zS(yR8*BAm@+fW4}Z|)0A(1P-uh-m-|G+bb=b|&aHFXiXs>=~V`ZYkSD@h^~*CW39v z&vz`dCd6$y@!FYA=|6=D@zwPh2q_gfiU2}&)SOyZKAJ%5Za5w;$&lC~rT>XO44gevKzneeh728~D`a-Tks(vlGe5V&Fp21^GOr-|wrmbEL zX?Su^8~{%_k9^t{pW417@?p6=HbyVPa>=4DXo9x{>9p1;o@?b2$^N?aQNv|pn-e^& zQlc40l{pp@QE2lEW~s)MDy;QD*5)+Tcb3`Vl|iM$P=PJIcKwjf1cbJ6%Hokse0+3H z;Yy}qU}MT_D8)Tr>q>y@`4@YPx+f^boY$inaPoQZaNLCuz{f#!m2%JWvSD@Fk~FBv z!`yjdE%0-X0bAfwa{HzmWB1D*CImJw>s-&YjwerE=X)jLtGtk8RTJE zq_@{x`-y!Ql2$gG_uR|QO@25W?v6WC;wfsB?cdvbcF56TMac^i5fh=8jtTwP|NQP} zcoU}$G#%CdZ8||*6T>OWc}y>E<@~JPFIKX;`WnkEFXh8}+*T{n?rG|2ZM#o`Y)qjwrd~2s z|MufOsmT{)HaV_6DZl^`aupL4@V%Y^iJ$j9RI7=_3e3n85De|Vms%rCN^h)7ke_>6k>KZ8T2NoTfQq23h z;Em{QQNqRMUQry`T5E89tz@Q`C;PY|ks4IMn7nC6=k(-)s+kOTG^_+y9AzuhX)ff)_!RcGTTFa!jMmZ3DsAWpC41X zYL(bmpJGE3vxZ6aUECqT^sTU;Wkcv(ZuOPyS%z;ARYI8qlP{8pfD<8@s1rP5a?FSD zUREKju@_czR738SSmTI4XI?K8xPvOS^fz7Wp3V0~jy?TNZ3OT4 zU5Am6AqDG;1`V9!^>*s!hYdr}>&$SRvnXO%l_?&VH}i8;WU2MqDjwb^eQm?ij1n?E zyEgEd1SnJ~f^n_Sa~jA^zk; z1HK=uR`>!+^J{!W(rZ0m(reYi+r{aJa<&x1aMwTuf;+NR63c z2yKl@zme>@iiW3|o@|PXE1e#K7%r{wng)noLXIOw%1u`391_5XO5O9=moCa~b-L%^h+tj{ zcoO+}lQQCC8KIefBJ1-#DiNIm%>tnk_%FFhQ_AL*jo3co-p?)eQzoU!cQu-v3e(G$ zO!qSxTvkPcpJY!b+Qz(p$1QD4H)PLw{^;#;GH~#XKisEVm{oi+Txs8ZXSm;WyYynZ zjDgkE|FD%rf}_BhSG=oLX`|Q9f)V>A=V`k$kOP-@@v^mNd9TPOc*7Ws!S=i1goZ1z zXkK2=$5F71?bovnqM5346kMPjKtso+bQDTRNZf}aqicXhk~e2coDxo=XxpA33mTSUh3(GhHxE27`!r_K$wu~o z&?N+E*A(jfB=y$PU>tU%5NO(`wbhR*(fMeFn9ud+boz<}TpEe{k|*Vh+v6}aIP57{~p%c2n4q%>_~%LP zef|5ps41$Tr+ZGHv(KKr)?RBk679pUf}HjteC;ImE}^D>sxgv;H71d)WM7Ktv>IED zF*9ScQCD74N~lxuqhZ-(2t9lm+iY|!FRt=q37OMB%G+41XOl?_209nw#2|zQ$EY-< zE!It>AX8O2vN|jYD}*Fz3{xwaCfk~Q{q_IgFZt1ZUks3R6rlqH+gSLyO?5R_d~-ng zdsiM?ok0TT59y{n;t%O&Sy8bNvzv^eNIYbDxrgiFHcxagP3k26SVi=4^|LX8)_`mv zIYrSo?e@p1&*h~;@PkQ7~d zLmR20$s<^MT;686>#LJ5L@*J_9DLukH>hS1Evt#p#Jl*R`@6aroxTbkU}9j7sPQK| zjGgDkDUs*2uI&xTH1_PF$yHz8o<4Ru5n{l^S*yYqSA>R-N1#C<2|TXXJIT(bHTv*F zBQ#u=VuX+hYo^d)>K31vjEGHAQ8qNy zQ-H;dMl}Ijggacf+sUTVm(E{QX!{`<=a-4{4K<+Pi@`qhp1 z(b_j3Y<;lbI~N=52L*kWxDB-ZFvb8Yv@)3#4nc*D#q}A53vzuBZf@!(;k)&15o-;k z8%iZRiYvk1k_}6qVnYt9oM1Ols^IZlyw%NCG0ma(S?WX!;3hLa_yDn?kGH6!zic%j zdO1qSvhASxANDw0my=ew2xa+k5rLsIS}sZKPz+F;kE1sn?bPg-X?(4CKHRBcp^&A{ zVcMRJv%%dh3Y`bxh9EJCXL)5YHG%oG)?URiRlm=f-=A?OGTEoLoK2-==Y|Ntkoss- zQ}CJ8@J&D^@GJ5i7Dq?$HYk3%#or-k%5WyzeEOS-aq1dy4lWW znO3dmTjn4bG>PCNB2~3 z5`=;*tg4bH!REm~b!wr-kkr@TJ}4b2D=P!3e>jV32G==lq!KQiBzo@ooJO_Or=8Rs z`qx<;NCrOJ|AQlwbkZ~woIK)M>msOMANp}L6h>!8PPKALd+-tg!1w+eQ)xN?l+9#a z)z#Ma4h$IJS=9vu1lTI@kiI#{4%IRYX~2O*PCeU^%ZqZ?4A7Wf>xHbRWN>lUgv$GT|eI|U!`WN0h% zzws>u3_!#P_hRSQv%SRk=8|rBQm^Xn#{uIbYOm1lM z-sO|on=eK$@n8Q30D#}%ki$s8T3e9UG_>=T!pFe1lpE|;Pyhqt&jYQdmjBKH(Pd@^ z1GDCSh`d;1;^F0V)mTq+z1iQDmW+tV^tyP^e$RN#rJ*PwR0Z!N%J^K$D?{K@pq@4=k)&1MIcC(@c` zG-wJ^vdd6!xkRS@;oyBO$7{3A;|PD*A!6qq)PH2P#+j$!3L)FTk(O9umTu*&p}#t2 zokn@a7m3pWtIrT_|M%@7L1wt}X#F4lTx+eY7pyBZ<84!UFA|iJ*C$F7FkaeAl**_- z*Qx3Phq5=h>W)x?0XvP^+1dZNv#3pDgB2gAfPTu~pI$)f(B)#6%(}}(x;$Igr0RvOt=IQXb&=)j#D)J*db9;Z)CM4umUSG7iYSUefrEIZJ zE&o&TMF0cQBUMW7RNB$>9(d%VaLQxEe-g_tJ-j`>x>10EsfE5*x?bJp(#LCkRSw7> zU3pIBk#66JYWFUW9>I8PiXY(kXC1GpLn+q3X@_9iKqAh=mQ6w=!@S?MO`=+MT9b$E|PK z?>XSJlGc7Nn;YsO{VtWb?s-;7wc}GkBiqU0ojjtt+$`;+gMyd9?D^Kc zn)ExE8L-GXER#uXe)_Z2N+T;&dP0|;PMU3N&Yw;B-1UK29>(!E^a4Crk|tOn!U$g6GNw5B@U!!eU%a>$2qixu@a$ea0hA18VJ)PJUI^06tnG zYDM$Qz0^SqZp!Hv6G}W9*pao_A-s$h7H5@Jlo#nvHnSV;`4+0n=qTJl7BlDAnao%i zk6P={pet3lj6E(n!-k^HJQV^W;&F`z%||Qa{-=9~nGp~mDn0cH_EzdYSv|U)VKrjd z*}u72Q(^fTn7P}~_tIOvoMcul-*@JW*)hiNW4ZX)7bqgdCq^d{n%vAriX&fFJ|bF= z&pH}N!uBnE1@T)QV-LezBY)qTbcp^R(=ADF|UCq;?p704>F1~&ZpqM zf5L*?@AK!#0+kX+5F7$3C7aKeLIc(H+|D}r;F_U5NjF_>kHboTbTgdZ#ZQH#S$lY9 zHJ)8&nIVqT0IYcQO^Z${M{;Ijn4R_XFgIoXmounI3T0$G^)C8;J9wcO6?u3 zB#C^XAA97F7erml_?{wmDpz>=RbsJJX=(7v`N@DQ3}B zXO`^omAYHMB%3`W=f_S-b9~(=Q$*izw9_DZf zoh$3&V+?`fPTve>aY~Ccf=)JbuPn7%QYIE78m0Tw3mw7r=mvLS zTn3f-DXTl#a^q2G!bGsD0-JHAhqywS%(AVQwXLaQWc)`_8Ctda$qr0`aJXCPn)=*% ztHBW&n0*A-G8HZ#pDetH+`ZMM;dx%%3MlC@-Fh+#i$w3US{Xe+ z*nsI94rlUz5^A`#oN5ByrQ#UX#A69k!#nw@Dsrky^K~<-DLp~;g;-uXKr?oyT3AQ- zN2KE`j`R=!Hl%ISm0Rhu?^IEK7dSJ-9OunnhB35ri}=3=IO+L$st ztE-*UvLot4gvP(;)~a@zZTl9ehMq>(fwD3H#;>UwX`^3w)OUvK?P)lU`1UdufAp#9 zX-3!cD0!V>nzXHr6~u(v4s-!nu|!r?O^F2>0v@c}v z>}m2qWv#!PyRYw`!ID5?BI(3)8+&1WSXZ;p51BF^6~`TX67R|Ri~`WfcUa&G)F>Lf z9R<=fl@!#MHa`lYwi!x#{rVEdJpe)!mO%4tgn^tSGaGkxOX=_n)CdS?)Z+6fY6t|i z95U)qAkr)Nie6Th1_xy|H?ry-1M6IKE4Ck}Vjx}wy9rZRqQvZ-Gu*rnIC4l>nY^37 z>zh|_I%%1aZWm^u7+Y`TX_ zNxThPEU&uUFE-7%M;sGsbA5@5fI>`2WU>>=Hs`U|z~(T#2hO%rV@gQ!E|tKoaSrf9 z&JTj^3@IhlUQUHF-CpzhCu!9!1~85jBy0tJESA*?md^}%=let2}|PH zMs#`8QYJYR z?G0@+R3TxZ3F||!P=E7hR9j5rv^A@SKX#C7LNjAvoi`qm19SYSnt$r2Aj;KI1-b-S z=UgUHj>yln&nM|^mKPnK)9CfKUd>n*>QvJ{IM-oJ@$}fDS(Arhs=%n)D4lk7d7b!uv2LCgN&tcc)DuZ8$}nE{$~aIl~vU~eCR>G5&kc`}aF z2Ywle$(9O&4{Mv>#&bKa7l&B;M#D8R`W_|dYDk7kxJFVnX=~uyt(w&OyJIH-Pv$r9 zASy+K(#o<9&-~Q-h-`44_3#gdv{V=I(QbJAo?b98MPFLFGt<+1kAD*LEa2pzL4pI(Q$@7=4T@3@ZLl^WoE?m5r{s#eJ8d_njm-T6v zBVebm$}1K15`=N)4L$pq??dYP8i&)!20ra2g7}f-Ay}5bhlsjJT{!xMK=u%d0n7X| zBrud!CPcbF%|Rv|@IxsKhLw6sl_IoR%`DG8NgNu-D{alIwQ|d6PA@u2H1Tpo7@<@h z{6X1_Oz?gjloDTKKkP+`Sxx`A!s|h{wHUvx>|*;FEI^z^S@m-h-5(blNz_2x(fcxP zPuvbH8%jaWs-zDZMC#cz#72$oHAILc2eBmrB4z%Qvf)Sdm$FgF=V2|F6C9z<%Vc+n z9LH(@)Nhk2CA-mXg#z2t48Gi#Yy&qi6JXviSP9w<(b8f!>zBCsuL#%>?KZyeAuHOG z1`HhS!#z9}{{Hbrn1+?>!ja8r@-k72$b8^8DGcFs z&nhb$A&5qD!1M^LUr3ybiDM~}0K0UEQ?-$=VY1#lHsTstCRQtHIv22@n^`$P_YXV@ zWTIGZ)#YN-aR0%f2%1<%5Akx8JI6~~u2V#wIvPqFojM@9gF(L(=)N`9v zD?I$cdHggOu!!H8!Kx}t&4tLi%=O2F&?5qe0bv_CO`07euXJNZUV8+FN{q7wl+oPy^(Xryq znlh_2X0=wBsGrxG2Emg>iocpcaW{`_o^0|GYw&_uJS zo5xg39WN%a>;`ur+WLoghF}{J&;oS8*(_GiP%v|}>;8udmO8nn6`ZbjsuUzc-DZ!8 z^tVbuZnkOfYHe+8YYQ^BT5q&90Gat}Yp=Oqw5e)or6nd};@~{{N5+uZt}zjoy6-nn zJb{5_H-3I|IJ{~!Z(mmk5Pjez=zH>?EaOj0@WMoo`GNJ-1f!_u_IiHY3)Y8=dq=D3 zWa2Xp59o59mkScQ&QMthZ9uJ|(?9|$xJ-VG8P(f-+%JA7Kh(I;vuDIB#v~5wKB&AS zN3dKuBssazDP%BJ3*l4{4pP9jf1F^%lRbH@uX{LyNY96L0DO$xq^=3Om!v~LlSD{> z;l!Fp=;BNtr85#4udrAis$!K$Y;$w%4>BL2Dl)4ajAmmfO`MNdOxMkoJE3r!dRz4C ztAQJtk@wmQ|A(Uxt%GSp3gP;iQ9X^oHle%x)SZ__N=d`Yh=xHDF@tD4^vQm*eTvM8 z^d!sCOa5{$^CVDXtVU%1hK|}mUtYi-ElFmQv%nCL#;Bv)b?hSz~ zb1aVcIN13WWv4F6rpj-VB2#RJjs8WjwhdgOCHh;RlTLaZqR7O1mX(o_F)|_t`)On}KR>V2>T=a|ka6>Rd{mIx|AQEC z4u>WO26$*m_;m>Y6W(~b8~gv$2SIH;C)%s{4K!AYfEgh1m;zVPpRT#e;^JO(Cfy zuQn#ScS4iNe3SPVpoH6{a9-c`1LZHZ@U&1Em6CvWf~})7y6M(Qe1&eZ)~LO4y~jE-ho@O7V3uNqsZGb4s_G@le~pFud zWZR+fSIjWkf7B2rd-Z%E1P-kPhi1QWFgWYhT|@BM;*avQ|^9iw3)4qiy_>>@uMX2F~W z%b97LOL2a~hS_-$pZlg#ENbrZmS?)Zj_-pTw9D#w7V&X;`B5I+mF)*>3R;Jcx^tOM zLY9O*&+h;v@ZK#OgtfK0it6f_6IE$(s4L4k2A7Yj)@OOJU&fZJ9#sF$1<23hI!*?G z^wH`3TAL7K6xLa1QcR^@SN)IC}^3YwhW!(z{ORa z+W9Hsq@pp=rvMIuK{^_U!1WnTmG$#1;IIN!bP#w!(+0<5%=rEG+QljNvPprpE?d{u zitCToG3ZY^tITpLTz1_5dYDIN7jQA@1Edil>yJp}nEt3Hu?#1nR@Lwa85N(+B=G|Ab!?e`MZQNRAfCht{VGKltn-^Lh&%MxExR zDLab+Snv0j8(~q6HPU(Ul>9e7cV_}DVMjhN;Mi%p^!<2M(*=NE035&%^%0pn2yHJW z$G5IbwQQk6qhwyCq<+2xWalC}M|szq83Ybv2jFiM5Cdr=|DGd1;n zP4r1@6Pw-Gwy?Tdow4%k=Z9~(ABerWm$M0-VRw4sa+~sME8|qYtfdw1c?LDs|Hv{O z5w?H98V=h`$UW$^E6V-Gt(BbWZZ+}E$thRF zO`4Q?N-M6BV^|}eX z0?a#R)jGA0`rx=HB*MYVBo>OEY^#<`NTTH9axQ({oNV{r9d_ zYv}fZpz@DLk36;8 z*wu^sGS6j<43(Ie1;Ne@Lbvm%!9|O#%HSROZ$9YsDkWn5BM85zUy}rO` zc#MSqoKLRaj)j-;3GC-HJ9?h&?X04a3;9VbS-+aRXi9I8EK!2OhVL;!1v3qoE6DEr zy)ahANDbitEq$WM9)^ec`Oxr0!T7USze1OM!)D4(p}ppJ9~PW|(KEs6!aQn4w@5TJ z;G<<#qO1wf*Hv%8T$AUsu@MXe##+qxpXoGBQ8SxpgR{?<{^klX0`aeG$2;c?yr(v! zpGQ~-r#z3A-kMTdVTbyrlx*7JpK`~p?Jo!g5R;*nq>Mfbko0#}f_DJSjcW;pIdx{w z&`Q8j5(@IZxIB3Ox=g79y7eH6h7QXdFq;T&W3Jhh)jqCV$i(Aw5hDh{s-Qr*_Liy{ z#NzR2x}K|X#89^S9T?F_MI3n-ub>x_+m(_3s;W5#x!yk{3d>{H?A$^*t7Z<3TiGGJ zWw$-5aS4bawR(sM5D5g8V5Ir)TE*r%Ul+rP&JS(J%-prnP(_>HPDv)_kwb<|VeR}% z-uA_zE|QSQJYdT3iHvg}l6Tflc*`7z1|p+A#ktMLPrs?y=Bj|MrI^2@IPD@9=L}1ysvmB&Ks7Be@jb$sd_xl3#U1 z8vLjY4~o1m{mN-uNo1m&t-@aS3F+^2>UYIiotv9;2LEMiWp!SpSOE9QPo+d!Ts)c0 zn&7(@uq~M===k^;0QeY$5B96z_}DDkyNCmqwb}Y*Qo+y3kEhzO>Fp)8y!eDGX$?H zu1u1;Xz|b)OkqmBv_C5CNkJg9h^Q!}pq@PyWn~tLVq=>Ki0?>4e-0!@P(whIy!p`6 zGTN`VTe+oN(GJTV}O;~-D7LLBZvUaU{Q$$2-M)_X;9=NbOo!#htY z=1qf#>p3BvHc+?K(>6}#TfDj8ft6c9Z!l=9BzPRKEZV|BEFz&lN*HLTJhN?E@&NY1 zFN_Aymi*L@R>DqlxQX4EU*_=kM>;NhJ+Hux@rzeT=Jc34t4PnK7l6WTe{l>Ne8D7S zt^{TI$%N+!i61xKq$%1PFLx;aWncGaEH@8K+$vJ(&vR>jkqJWFW_@t`Fhl_k&_b>H zF+-@k`ZnG{*c346c9f{8L+eBWpv?oWYG&I_OwrJizcPBFN5?RYZZ zb~mr-{&23{;l(o{J0_X5b&(?o^>jR2029KoK_}C65RbMH*^r0)#Xe*0!C)rG76fdP zL5;XyD zR=&~^vW$D2?w%X(iy{ftF5g~wOniHU6EJAaRZTaOzl2ET?H#1 z?IIXEq}_qM#^OG`LmHD6k5jGNID>A*?QsA+t@5hgmB5Ow9`EgnTc*)*1TXmnQg1PT zB_Sjg0iUwjCG{u2galM^MUk_Po6=|RK2~G=uL8CmjU1me%T!9{&Rghd!7&feHIM~@ zCaqnb{NYX_%FN&9oYwesmJYpb-ILO;K{5uiPzIn5mVX^KeNq=;HA@!bqoYDl;dfg; zXX&%Td||^=kFOi7-@EKJxg)m8n@EG~YE6Y>#07a_H6EA*e?A=vFCF^ew|?B%e^(U!}dTLbjyqNtNS7y^UC> z9Bes&0HPYtZc*4noxm$|yHJR3`3asz?h6_In$EnXgOJNEcJ?Wn1ooNiF@Dq|GUvc=3_WB$3- zXc_v;?0HSW>a#L@2s(XxO-;6}yLp{vX)c!<(#h>` zOg1T(Ic`E(i)lTE%IgPEYplnw9(dcoA0=^)nfI=KG+fixQ_KrNxn+-&d3 zNOWik+Db4Sbl*^X@pwQ(lrC#c1*Ns2+jn%)Ncfw-*iNqRXg~5CcER z9h;QOyiqjn$$IEu9qlk!vycU6YRWX!fm&{434efQU2&qhiJckG)x1zEww@RVGGmUd z)LK?;bPg;J5O^ygb~QAdmQ&0H5FtKvaPpVpo97^lDOml%lN z?;W$zULx~jrcC20zpTIIGdj$zVasQiRAICkwlrLrhH2X`IA-BVU%}jttw+EviH^sy z%wT*Lfr2KrYCYL!r&7gUOvoYhax(hi$RMxOzt$0iE|`<9mXm%icx94vtA;bGjQ7V& z@KnZPd7w=B%X~8x?JyX0&0}IP3$3)r2vYWZt?;5iU z_Z9CSm~D@njEKFYJ7Iv}2cDX%o$&WYTk$b#wr?eMSEtzekH~W}gww?nQ z^8=yg_w8-3za92kB5C5YuMA!M#UEZBUk6eNn~Yq$&N`BCDV=d}zX~n9zIJ^fNP60$ zi}d=k{5+O9!_5=K*XHyvzW45Nc&3R6eg6P2W86GD&*QpC`OE3B2}qj4xal0C^JDp1r4o zv#q-MDGVs--aA8)hkF%#j%^|QP_Ey+fY#=`OV^Rc55VhJrH2{7&_4q&REhuvAKxM) zrrumyB~hs0E}e6uSl?FMzl%?J4*EJhTBmGTT`6qPSmDcpo{@(R4x|B)fU%z05Dl8tBmBPAE8MwFDue zO#Pqi=lcJI_f2C+jM`m6wEE|!3ROPms}U?>z{f80H^6oYIf$G ziOiqCr|dOI?f2b+1wPjy%=^%Ksv!4miwo59OwlKF0Wp+>t62(cS?$_30EIoR6&G6Ru{HdB@-EgwB?pe8tTznn2gjcJYP!R;Tvyy zBO+zpby^tOUuK84-2@S@X9x1W=`sp3JqCVG#hutoEucfgsEDinC9u^=Zxp^khma*8FNNOmxFeYYAnE0Jvf6GDdxAbS zt!piIjM!dN!g|r0oSmZ5KOC~#ys2JvM5Pz<#-Dr*BIemlw2nbY7FSvhF!Bv4Yq=jR ze?%k@wo$rpcV}a=k1l6u07Unf4h9{u+ihx<2&TqW8QI!wz(89#uO;UUz?v_Qsg+0S z)^|6U3_NI4X*T4VK)2Un@gQ&`JQUhiCVDr#u2lONPb7H0XsB;I#0|zfS7dZiy!TSX z#7^#t^k~B_pv|gcCYv z4wF?ILvzKP%si11r)P0tdre@xt7smLF6)XYw|aO2(`>67hVuq!wwrjyx&yXg&@w;u ztnIRYV&L^jQFH&hHyV7vJ{W;pX)Zipmqfw)NTqDTy)|wH*6$zZ{<(SYqdQj$=jk+< z_uZ{p=7PfQUCWC(9+p8EC!fb%qj_Ox(&KS>ziWe)X7+)z-AS$5X|D4xov^4eecLGB z4_@bjFz5MLTv^j$9*GUyE>@ZxAw70j%%^A@nm zn)3{|KKDpF)xl~1?0H)(l*U3uAw^`>ypV=wbjgye!9v2;&;y^@vR6PUMv;Kiz_y=z z)phGI8u+^5{$wTL#M6Yfgj>@A{JKyYkrB@=NvuD1hJG;d-Uv%|u7><|yAGDX*2bV= zg#jfvphJE|!Smk2paCM9){>l6Vw|t5ld%>#lw{SYloNY6C~pcU^38!sx$@-3?+(E0 zWF*mA+tAipNoL#-z1qsrR~H16PZfav*gW=ZKdHo)5NMJMd>c)o?2kBgGTf${H@o-F z@GK^JY(80V!GAv8Z0df7?3yA0MAhA2Cbq!1&B{ZE`nK8__XP{$BNWIL$Rl}}clp2A z*!TJ<3(=^Z`XQTBa;Zz1?qKwqXlJ6ne7e*kPU4D*o!%9(L`bW%Vf^$dl=+TAcJOZM z22<2AVIO1ny;AkO&Cn-ebAgS+yi2}6f=SoQE(Avn>izWH)29A{I&0Vii@2ABwESC>o8LUc(WQ*Ugu^NhZoNCZ_pk+W285v+3lD--%O=?HVA9pu8` z`oA{9^cYdL?X5Ng1Gv84FMek+faDA7hq(^>bS#pOodEabyt<*yxWME1ncX-!ympI` zg#4Kn1UXwaT|#Qn#+JvIHN*9VvWx@L=n_{|-`R;a20^qm`-BnqZ+ur zAKxbjon2vT zCZi}A3J<8kXiU{VT?~epS)jc|24%yw1xIT0SqX|?reZB9jt7+EJ6eu;rqE* zy32fVYK*6FEPK!t0$|UUno+BalYKI$ybEgSQ?~-+t@2d)Z&u<6u(Y7G_rZQ6vf%9NIEakc@fR zXAXWmKjjsIWFi(LLO3OOP6DtptH!cXihZ+{%L#>#H%;#P^s)|hbV}F_0`%1soToe( z>vM{i#XglK^oi%=Sut6i*+PBpI^mz7vJnCosL=t zqS|tzyvWg`@R^kk)#dhds{dHpq!_|1hMmxNi6hDaI+kE(h+8qmD2~Uhb-RT{HX5ZB%yrF zHht7 zwww38X$f6rBN@gdHnu$bP6ii|gpNv!v-3zVhYit#;I6H|9lcuzDgWcTJ81jjraF?j z4$a{x(LxP_VNst#4=IO=6a#Vs&!QHfMv^l7CrJ3j)iBug`Q%vJMbaBU&5FSXP$R7R zVHy7>U;kU}>P+JAe_}*ELQOa%h6B_3Ml-XjPd-=WRc-1ZVNygxGZ0OkvkOHmI`ae@ z_6|&r^_ckvYO|%jJctIbnsz{pG5?n0V03bNC=eo+{?d$~5bpgu-}@y1!7mmh+Bbw- zu*xUm0$HL>*hL;vh59w!i73(E9cx*{rAX=3+nKxDC}6~!a^peAvR(qD59SF@vb~+- zk^6B%JB)?oQ%L%)Xqq0Zv?g?(#uuzoa_JQ@M_uwy97Pc_1)U3`TuB7_ElJ;K;8gDN zoGqCR$zl73rQBxkY;wOHIL49`SgkJisGsdFZ^By@EW zy`aZ5=j*(}qUb=EHCg!4fROO{cDfdNNPtfJ%@Fi>S|D9VyL*|qWYU<5P_5cc%NuO$ zJ~O18<4+ey&xkqeeXn$z#-UX$bcM@Jd-Wwn(E5`n;1?p}!WrW=er6cwmu}?wlPCPD z@&T1vf8f|-rD&FcTLLN^QpnOmo!aU$)saLK>AjT*8g(a6jNbFkiMeh?c>QgC#wa@F(wuc8d#Ryh^dTVi;1FuJkgNITqei2Xc#s;Is_eRH z8Fj^8k3mRrc-Gh;>1j4P9-c@)fV&&r%KqLhIl(DbuIwQJR!ebHO<)TX2a6s!jrdP6 zX@E8{c}zA3C}9#*z=9OPep(MNUkU42rrSpANM0`XEiP)}XyOZa{+ih3)j`puRS8qW zKs+y~o)7z!uTUe>KZx=*er09xKBF|B`sP3{wKtxR04E zxL$xh)SShaV6~L&s?LS@5BlL!m>9y8pP3ft$7N8r&{-;F%Qb8QeNJrb2fEsLA}YtJ z7l`-Khp0hmPbuVi7S&3W>*An@q7hjnPJVySgLJvZrByRVNYfcfgoVe1hkwS5;eSkC zHf86Za=_1tGtO>vD}9a};QU*}fIUvim!^CCxo`wl*O&ks{8$>lvWIRD$7VSBP25an~ z6hvI~b!o>Ce{H|~epvA(a$zw_ji-@>u^-AAwQmVeALoeF9Q7juHg?5iI7|!HNz-)sI8PoL`Pk5eN#VXC@s~OV*C@H<1_d6z{{b#O zV1-QJT*HQiw7h2wKd?R0kwHZ2Yut{j6s9U!g;p$X6Gr({Dkfxzo_L*8($Qqkz4W=p zB|b0q!oH}si!tCT%NlmS+=_zu2Fqvww16UM)o>%ud6lSBTziI_80-~Egvy#Y_IP4B zf~L^M^h2vk8u;!5y6u%&ZRK1xK})J!?&eTki+!haimgc$tKA1Dugiv&qx^dqmmI3q zUnJm1*pCQM982lmPayx~4mg~fIht0Nbri0l;TERjhvG>2YJU2`R4;qNgt>y7MjsO1*W!qau%=!mX z+uKi^E%l8@%tFzo*L@MLQ}KL$?rmRFH(9ouQj!T^By|;f4@vrHTFjFet2#A{gjpPo zd|$p_4U2B%IDHrGIkmakr=}s%H)WvQw!Hf^)f0e&3+kSrF+%}-lf^Zk4!V?)%GA8G zw0d(U+#~fd-c0Mu8Xh(p>Qq3^brxzQxmaDY?l=!eR7B2CxX!+o5g!dVjecc9%lp4M z1%vj@UnLdcFrH0Yp?as${AQVye{R6a#tyEa1Dj$Akh7Y8L*#0cvay%2@gzQ?%GZ^FuMP93bdksti%Lk^Bpl($WK4 zAIp~1L4EOOb7r{YVR1K#ieVcN%>Io-dt)R9rAdUa%|YEv;dj!(x5r}^G{UFz0sRu| z<6^W9J*GknQF*-@!Nb_*>Q*_`h7^k|OJj_WflT63&d`NiG31B5&uq##x zIKG-K5{BY1DaLgbzFYet<1(h<)wUIwBWze;k5fy06>8O}q5(W}E*OqTkNVsTNevAO z6?ekp|AdM+6EdyX`RaEE$MY(?u&7Qxp0C6JW6DMn*S?h1!8#jh6DM8Y25&PX-QSHb zuQs?3@-lHbzI^sl1Wq_|^pth*Ev>t1ZyT=t8@Ts;wC z$vk5eNNyg`@c8)LJ!8zhPCtHT7{$U;*-W%K3!4{I9|&zu*H`B6zOJqPXx;Ub1vEcE zp=7mjo8E0>mq5?uvsv#yrIW@_45OrSs22Zvw|UFS`y^|beXkcXhAe9)0(1R@9s2nY}%Bx))v&rud-yh%Ppbe}bp z6}gtu+oz)}c)C0!6pZL624r|XCt9vIC7(E`jvNgxywrU1x{s33?-mq*xZ7F9==Y;; z(G!gH@xVr^+cLsOo zz9D;`{lDj&eQ&S%Fl*MDce++jS9jO1>gmrur-!Kz2z*|kR$YnXJ!o7=j?pk~;B=k) z!OnOAK(#%b(9~vYR-BUNi7>GEhtuJWhl2(3dmSy5EB`dLs| zGJ{q^%^)t=m{Q_mIpL3yLN5YV$Uj8T{pscClxtG^X=f)FcF7FF=es4DA-Rm7=;b_9 z&y4c9ThfNzRODDP7DwdCfAp1-gV)r$r3S0~cq!z;RITwCs)6Q29pj{3?wT){sY#B` zf)xGjTaE2()e%Jsnn(vG2k?q-^y_ooHl9VL0mm-Dz!=cp`9?bxN61o}6*qy-pSy{8 z<=A;syVeFh_JuM;|3;V}>@S~b2Jv*j0zY~Bn2`a~>6xX2PQ;A3G?42T z{aUDk29Mt-GkZqjuk31QRHM^qJIe`HFf+@ZwrTA5cCI4uJ38-B7x zN`5nUbuXPqepl&*r3HwT|5sbuv^Fp*@|d_=OqLqz;mXBO!nhLyz%o$dJ)lmyl5;qx zt3JZR#^zkemI6zTn|UGj8!Yl@4)U}7hUzxs+9^j#`}eZ}em^sMiFThM0=3rvv}T28 z+a^h+ZMz&NiTE#0`oP^X8LWue9TPRMyDRnbvgQEZNVweQ%R9 z->fYxj7(%>ZH_TC{14ds@h_OGA^D#KA4Z&P7sA)L`95Sca(0WoOvtaHJk;T?)zYf_ z{?MB8w(&5pHvHu0=m7i=woRD;UK+51s0`m`l2oj?SXs(m3jGMe_n!akNHaV*_Z&(E zik}_?f;!757k+@s)#o>o{_ioOer|M_@29Y+Mi;u(>1HWG^Z>QnZ8I=xUzw{cgn#k< zi%f>_b|$|@vV(i`XgI8iHPv>-5entIfXAKN`ki2E2 z9POL?Q9q%QR55tpgb*ixugG)fcT%5Gt?)Sl5>~!Koii6_9!&PW_}xyUEPC$rF^XXF zZicg$yL1>A-{!9!6;~b9T~t5j@%;1jcP_53K(5N_u0B@b-!848$?Yhu7jS7+rkF#1 zWL(ZEgSO-FAK&B`6kwpEzj1?sNd?}Dkq(zMmCf!^M$^#K@6r}H`TE}Ve`8zz^E)%RW>xT_otj$3-U zYcv)&5_vLQQ0$l3w}b$@vx$r^x94#-PMCuv`th@D$720eFTQ?i#&Sq&2jm)VP))$8 zDWPHknewQ_O^5q(9nSv#CN^H*4=88SzrQFlqC*N$FX-7ydNk}iD7^g94{-^y)4^cq zg{00lDT%$Yp=Yw4?1FpwMjGZro|}jTHDA)=nhI=S7wvOSQBmFLX}itaHH9Hdm#F?K z)o~4<=+?F8Km<2Czm1K4-urknCOE_H@+`&jRcXA>Po$^&oC*zK=>jZ0CH@S5M|9dS zt8}s2r*$~7zabw%-i8%^2HcK$_%E=8g|nRMB+jO_>fLgw3L&4``XO+IMHL>Yn^b{X z)ao8~^h8}tX6gR|U;(4@q_+PUWw10G^RKRN84(-CAV1~4dV)in{9Y=!REtDJvKZrH zBW8l@R|^~a=mw|3|HPW{o0MisD|t9HJS%?~Wv@&b;%}Uc<{tQQ6mzO0w1D8m3F?nt*psr+4t2 zM}S-n9xZudn%$AXS?(e8l5yTTs}s3ontV&s!a~U+#MIctRP^kvuK_d4j;XT&VgMsDX(uu7~Y-3 zZ3>?0Q+q!%*5q&$4L=1rQoyJ!4nd{r$U6Oa%WBaKD}{!>Qeq^Kg@mX%Wxk`yUe)g~HW8vWcsVb6?16*q&5 ziQL1QaWM27hPe+WY-LV$Hizi@Nd1LBpc>8QC&l8KSxGIF{6H$cpXi(cD;2KlEEwE zjCc&1qPtiq6YipmXTz;?_*WV^d?d@dWcNFoJkWh1@fD969FcW?xVgCIPSpOuCLm+g&6g}egE5(PiQGOeg}HluYasCBYsxotrafCGCi)%#mvY(5 zIYKZ4EWa4SHqNwkJR@Kq4_ha}qiG_uOvm9Xn~e=_(kPgI@I24^xCAZIUv=Fo57>MO z#eOakf+{jG)26>qyQL^WLI3i&H3gIEl`D6$NJ@6G{F@1mhbdd$yl4P?;6aX-QogU{ zW*4XyK;_pww#YW=k=Rt$^r?9(y4uX4=nWs#o0bY;%`?{RoRU3a{GGqA?r;-Cy8L<_ zAeOtPVX3g&3N{moc;o9Qs@RmoV<6lPvE$!%he@ih?A_> z_C}&$W0>nP&!QRU6m-=dXn$|MnsyRJ>~X!Y3gTJet&6E7@VlDM7b{rZ3;~wX4|xF* z*MTp_Ho0OZbIk*D!4v1l`J9Dckd)*@>TWug`>Jd#c^jZGnC>TMrB2|wkfuf54d3}4 zPeF8^ue1jJWs+~+Ahf%wOa?bpYyihSqgiVx!_HJmqpBP1Os=OE7V9M=_ft1BLz&(tzJv_)ILFbm7&=Z%V4^<>}}Q;b>5MzTH{>E_(>L>a@6Ai z-j5{I3VHEIKzU-3+5kGqq>ZFw<&KB`?fs0cPpUsCTrL&`&(p{A!R*wJ0gO^t`gcsm`Q#~&3qvf(z{Grm9y zP&UU|vl&l@Lwwwbgi3)rr~TX>&{_iB7eGEok$h6xf01f_(St}$K{$Si_BR90t0j%LWJ zm90zfG^NnqzL`+ap54R5b2}1e7GME@y70lMn4R(i({=umrLPNLe+6`R*Mb--v2Oh`NNSYz85E;gx$tRTaHy%ay|A? zZXmZiOGmBF)RvbO%g|-=*TCb8!fBwHj6(}K z*a3ThJH`-?q?IzT3Qc-abdp(Dbv*bsuK)Z%NXbmtrIWiKov@7o>AVoRz3QCH>*!dsP+tV-(QgecgdCo02N~h?4FJZ1W$e6{WRBm>sKT3zH`OlqJi(f zIOWLSoRZ@CU!1Zg`jJy&DE-ALsZjeAkn$#g%_BghUp~{3k_h?{&^dy;5I`fEG(06r z{UQ#%`8RQqK3jUHJ108IsRUW(8Az;4^o7i5Q#pDElp-R291~UlMOgaRZoRJX{Zol@P@UPAeTvt3u?DxIf=B^EfN*J@>Q|4FKL?S;{WU8~rl6$8(tfp_{9&7)*U-U|~tT0_#1JZM<|G zvZvp6ta#f=5ScvB-vCi_nfyTc)}I?rmVrH2rH{~X7OWD-`KwU4k1vVsNOuaSHs}}m z+)q-wZiZ2?#&TO7($AoDTX4Miz*Ue9FtaQC-J)h9vHh0l$$`P3gLmjVI~^b;n4|ph zS-Wx|Cp$g6rcGEe1^JUi2;vvz7iy9zq1^AW`~|!9U5ZASOdZ#LWvLdvwk4@&Pi!HV z)u)x6<${c0>O<7_1UsfbXR-K9&Q<=ZfX%D08mTlDi;(^h5u5IPt)A1N`(oy;)!g|~ zqMCrmDYGcP=76uh_Tq>1clCnK`9%Yq+wGvY3lOh0K>qc2 z>oi;HZR}df5%S$G^}hFkZW6kM+DUoK<|bC&$9_$djMNjWyz>H0G!GFeNxk=wwD^AC zi}eKmg1`QJxmd-T_-=A^nUoA4zcrQh?DX_}s;fqz?PzZZG*)+h2B%c~%LOU)=ruj& zrDh9fd5@f=fB@XcQ}DZens*EWSfSY-ojkXR1JxGid$cDF%NsnrPkDC2_xb&?igeLQ zv3fj<1$)aRh0n*vPQjVi5jJR)5tTo(0S<)o29FZcK%B_1tyssv*aAp?(c^usjQT1XhjbA-T#1DU>#4Q znc;N7n|COON62I3*TV+%4Z9a&Dpy15XLwgh8{%-R2};_mT)&t^^|{=kYxkQ`{#~0Ct8xeah;3shNoRU^-zVBv==iv6e-W0B+@&{HSuvJym zJ9^n+jU{NWbeVWIVhCwW0IrR-c~0=*7rdz)s9^>@-yv83f|91Sc)fbmMiH>d^UjZH z{b7&#X(w>s_;KKi#QVy7!t`YCO>v0KyOu*Ur<+gFQ>8l$WZYaTMm?CL;vXztv3vdI zv&IQa$bggPN0Q6^XLjS%_};~I*xg?&M}4U;^j+Lv@V<1hoGz`d?(E67146b}YtiyR zX~(7Df^lUE9V1ggUQ=1tzQ8?bu`=k^>_$R<-gTDg;HSNHzWc7I1d_|3@zJUbYRR7F zgXdj_ao8Y@t4+39UbAWds!Zlnn`~-rP?7ffSQ?K!Dn03Z84^D`DvTmBj1b{bnW37l@%~qNrA?)4BjMmGctD;#bUn* z911_KpS#r}+h_t6K<_O$HnWP+o{GUge_rP5I{TswEsST;r}0Fx@{>7wGE}R zw4``ZvoJ`p&?1E98eh6@qaS;2=>tZnE=LsPFZiMlwc1ucoI5UO?aj-?ZEW-?DvvHf)lX;R3(T?*hEGB0|M3N*nE6SWECzG zaj(a<&bFZ#SHvc9*%nx6@5jH5hS{Z&@&~M}o+=Ufqr-}S>Z>y@1Y9DT((gM0dI5Qa zxlvpwRjJ}x7R8y`;wNh$-^1@y#jlF>bed2kQ}N@^_g3hcdFMKEdSc3=I=SNFQ+0`b zj4BD^Qj-$Bop^BGx`uD-kjYF~6A9pT!!b+g+2;fJIpQ;m{=EeKhU%sCGm%?xrI^q@ zR%Z~6l(USbAq_;jmPb+MP$25R0b$7XoqNas2MGJaAGDv=s4i}7fG!jT(LG)aY7*oZ z+QeCm`!HQPsf|i2W3}=%yZB5#p`DSiuw-DOw|o0!J8zgZ_3k5(iu3!=<)|Lb7B#2; z>nHCk335jiAPWn!-~)Vtw^PExj*)@hlI#{6Mf~^iaSWtj`P=b!3TV zEL<@bw~(HNH0&P{^6e*+IIt6WY|Dc@&E@^dMcGzo^zXM1 zv13gr?RW3ws1`aYZYq3Dh)H}0EH4wWUy+N zE7fNQVlSu8R`?zRC$?m(D1PzA_BNNt5;~|Y@L3y8B)vUnXhRT_X$vTTypl-sdk8uq z%zQ{5M`^t6J;1j3eE-qcsVcMf+fK{z_p9prqexRizQwsCU%N`h*9ksQ{RKbQBx}7o zph}Z3b;>b2xAt4<%b1qnNn7vRX$iJgM#0tA(?$u_{fA2t0aFa`F8YQ&reeWzx23Tp z`@E_1!gwN|Q_uxg#gs`S+R47(O&t|h%U-B~%2U^|Ii8W;o$Tx{D|%{gj#PsBzaU*i z%Lag*WlhW3Z5D)EYq;LH79yCpmrV_;%%j&hWx^qp$INND&`85{EJU z>0d}WPavHOXCkZdZXvI4p_L>rJ*}}@a;o}wej~NQQmOD%2)}zFfz<1T7H2jr$}S|q zhc6isMbg`I*~6vcZ4p*c3q1v_1e#PKGIEm_Wts1ah9#JE;}Y4q^;6slo&%x^p1^fu z7Rw*I`@TqONXALkuGblHrB&*~>2&$EZrE}SC)&T);7w#zBzP#?4 zn)z*Z9Vf$K8=>ksto;3v*GkYLgT|em`*E>fHOX6@^3AW;`XateDD?jliT~}?r2WS}YQ?!+cvCtCVf)}iV2|T9` zTAffxIp<;>#kjukrtkIfO$S|*-HpY#WX}3v)TXkT5X1dl5DL?4l6U!Lj7^*q=kxZ; z6Mnn(8yBD{oKfFPkp5*R%iX+loge1AmB9eX?SLD<`?wdp2Ggh*%#_6YG$CbGG*2D-6xUm z{d_Hj7$W3xK{lw(Ehs=l@P~HB>;9DsF#Y6X#3{opqn2d)g=9-eTBU25a;+DzKF`;> z+5@G=fd27hvHsAB@(H@x<;@nVNF(oA-bbTwcRrbFp(8ofQ?hAIko*yjN9pAi)q^tr z&DPPgA&Nd!5%=N{t#7Bz`uc*kM`tczRCMsdVI3R@`#q`45bt8#R8Z5D=W;3d-4hY` z$l7VT$s)vk?@QU7+3?<~hwXu~h(%)l(;ajsa0FTLT@x{uvw9GcWR2&1Z6lBE;0fra zPa=$sn6KvM8xK0TN)M_j1dZHFQ?a=-M%v~bquZ}NC^A4_c5SR&ZpxcEUP+Q4!X#}k ziiz#_uvxs%7ozI;dED@^Boyx60R+@B`(DiBxy$a1er<9YVtA0oFSB;+6)4X9=~vGX zM*Zz%w~;uBC=3xtG_SSb_&Dqf^x?4eS%JwfT_9ko!}uz{sv-}c47|5d_2na?xM?d# zX@p&>lTNfzB>|#@{zOJ_9UhS0;p(n}g*B|_Sg=9@Xo380%~KH>zVmi=yK(Q&s2sly zY^3nH1JWVIIyF%bIX)FTjl?c^`^sq(Z9-T};IQg}69@0o^~0~5j;#>wjngctPlic$ z+O&4OF5kNy9YdcfDTyff$_h+6pE}Gg6C+N2OY)1|;qb?(+$aM|%8x2l}+ z#m*L6TB&ahiGTkV%xk9xV^wJD5F3J&jGNOAk%hGB8bu5=o>XtQ28zPT5Rk>Bn0``} zI!aCWVd3^Z3%xTl>l(*h(CY?q5liva)^Ezoh#1E>z*_>S@7>*i`USPSUNGu3*jZ8+ zF0-FaS@eR}tYR;7_!1{-M}xu%hU|q~^0@3Zh_;`={W&FmDBVt5?dMFr-Y{tsmL9X* zF6_VGZ6Iz?^I=zKca;&!rr?;XbXU4>Tqq*`Ii$!p%csfOJk%v1F!WEU^rcCVzZkac zJQX1O(rN}>pNTBbfV9yZqMW96N1@Y?OO>9tns0asuAb+&r?D8WIZD;=Bh+;mNl3CD z>em@h!GSPnK@9}hRrnffd*@{tM>kCm%iH~;n^0Z|DMWlMOym$pM(px_CG0?GOz`;B znX5|(!)3t!{`_FmE@>S>7x19XnWPQM)dDdu9TSdC$|v}_p*aGHWtWvG?QM~d;|*gl zcg;31oDsPgTvqP}yn3IGiVm=uE!c__WM_`=WzO%~T z_#e;b8YS;+BErVox~n;4YHnge)}o;@eh{DPvLoVD$s>|pba=QE84K>db;EUViBUh z?nQE=LJ3PSnF+bg=8b3N%`Oky(3Oz$!tEev$k{wmC;Hls9?zQOV zo46FR;x~!4<5p+K$B8`7X7a0MBw+f!v-l$ZwU+$vmsF%Nx5v5Tv7<2ty9mdNJSZyYP@#5u~BuE zIP9Zy)(=rfB%!YsNueP*^oN<)3Ul9JMOressUPD(wS=N%gcvSkm3$oxza<4>Wq_|(*-T3@w= z(09h-Ba&2YKL*4w8Hen2S$o7zIUTWf9C1Ua?E7@C=5`A}350t-+dzzZ&lc_CyE6IW z(UD?r;>XGIb!^fsnaX{pwFg0G)sev%3VB(`{a$ES<$hIC*ycFGftl>?ku^Ld4I=Bc z-aB`sm2|J*IBe)C;x}Q!MIzNA%5UQxqK?&3Bo!LsxS)7G@$vg)D$+Q&+dWhyXk=A$ zdeC0KtW=NdoJAc!`a{*S5+b=w)~>-{Lh}qR9X@h6d}5-`8kJt6YcG+ zr0SFg(bd$x*)*Zq5ZS+FV!X+}y{9Arm#8p?N?CSackKkvYX`WiVP@WLi+3)}_27|! zd&#(!3L?SeQm(G9EG#Tz`@?L6K0HSI(?Y2{mHf3|7g>JjFr*auX4tk}86yV%vJXuw zEGm+N+%I!=Xh77;rwOpI&OvTH$jHcTK`7?ik4SAx=Irc@goI>lS#cD4hVt2~);n1i zlKrGbad>!mKDhkXKvGn+-ylIG4-lg`M^0+wH$U+_IDA7K)@#CDa}(eMZ4c{F%$FOj z)Q&r-l#`*wMmcz5)eETU{BGYtThhs@jZXw24|z4D@;i2xsxw9U2MYC=U&h>RxEMj6 zJ~w$Qed+^c-ok6ees_~?JeIsY8n-JE6>TXO#E@h8n@mHQr$c`y-JGK62AA`4TH3uk zs#V7VF5Q+5#v}j6m^xGKw~jV2YHGaoV~<%eD~+977IwO^Bf*OqWOtZk?35QuD6&dZhII^QE@ z{;iO4&C}&goYJ=WT7|z!=fPEWR8|aw+hs~YNPWcYkG(L>qE7V(bvHn{4X~-j_v_KG_1rgj)v|H*!1Hr>|J7 zo2#1RYcsf259C3PwkBk6@&~@4idYKV4d2YlRdg5r;T*9B#i%QB@Yb*zmbF2;33cRP zr@rr8uiz0c))y%r&1i0P&BC9^C+i~AyLH|!yIKtxbMMl*y=Ct-)7tm5`%*dDN(%Gf ztvqK{?@%);K<;+RMCxj|tXP^M@jQ1H4_f3hrI=QJ8@6C@KK{v?oaEiCm~V)6!eCl6 z?&I^PYW6)u_cGPi4!|78T$@(*NT(+xPqvNA{7%aK@0Gd6r@#tY7mcr%QZ&Zits%K6 z(?cM8A{aPp%MRwkPRF>rQ1T=#`)Tc&Wk$-OHQG5Gj@0P-67p8x;= diff --git a/contributing-docs/quick-start-ide/images/pycharm_add_env_variable.png b/contributing-docs/quick-start-ide/images/pycharm_add_env_variable.png deleted file mode 100644 index f408372211400e908c414c3446e796abb6faabc8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 86158 zcmaI6b8u%()HV7GC$=%MZQHhOn-kl1Cbn&7V%xTD=jM6e``y33x?Sf~o$B7F4|@0R zz1He*d08=7C@d%d001i?F02RufN}x=z`GE@KQ#=dO6&jt+P#Oey0fBz8@|1xovDSj z3BL0mdlP&UcMDShzFg7%9+ ze6e5v{psk6m9y#rKcN}H>q*Zk*nT#P+6uUSKM%W6qQ0jR{kXe?OZ8d%&0BN5J+|v4 z`sF=5Mf>f@Rdem6%45dowroV2|Ei+EwsK{J0f!UBy)nm{?CNk%o?dXjQ})#}Yxk|E9Tb_t!cC5Ly_j1^($1Aa(xHjb+&2u-Ez$ok28Kn?!Q zf$mnbe)U*fq5X#g?lI<{_=}!z`u62VBAwHea#&`7#Mx$|e$AL6)>x*GBMPL$adL|@ z$j9xYWY4FfdJ-k3G}}Zn_Ju3AP}G6Xf@-dgzR4Djr#^kc_` z^=&u17!DQW*~CWZ$uCTMiq}D6Zx{kkC9lVJ)?*a6*vC<`nn``+aB+fAL0ioeVyYqG z&m{G-yllY;ilR)(a?+*+rxAvdOs7%cHeJDRl4NbkvZ7>Np=g#xZK8)lV;SEa z6PQI;J@FhuXFikbY#Lg7ZGV`aobwDVK{nmyIEm6EbXVw)S1D2!UFWuKjvc-9HXNiH zS~y##paz>XxH=aFEp{hr+oWh`Vlj7HgrbwKJ2!${P4AmHe2#V?Th>E^g8znlG14}1 zit-puDRVmiy?E=i-3`A%8mD==(=B!|n3KNO&_FJ~N0sNV5O)%x^V!N4 z`nCkMm`4@`m^P7WcEkLFa;an|V6cQay(kk5>z2;9+DTN>GbIS8j#di8;DslKEo9y? zHzQ1$sGAZikJ(int0DllTor-r0_-CeN zk;a*J*F1;ofysU2PLPF<5-i*4mv@{RW>@u$kq%MO;&QRN%@*qdZtQecLH}P<<+FMJ zFiJ(a=o}4M;trm!%^j_x?LE!{N};+%2&{~Nq&y8J(~snU7~-zf>c05c%#)n(vdOQH#8TAo${zyV;o%RT0-JH zncBtdp=*qiaJDt!(OH9*5EXkbgoBx(6;Po5+HcI~rArEn(lbT`N(g7_- z)K3W&3?|e+OJE6^KZzHlSYTbTI)QKSxR`j*^~us?!l}JNixO)V0gD4?5^jwk$R(`bKX~{lQVWMm$YJ{FF{<@P; zE0`1R#?jM0`eIhvduS8iUvza%GVcjUeE{fiwywaKsDUFzf7y-GK#}aJu^Yt^p1$0h zx)$0_6Y$+h6j3zI2FPM|{*6fMfMA}0WyiyeRUEaf1>eUY=HYduyE^P{J0pfa zJPiUhcCT$U0jo7}XLc!^GorcFgg=7W2-9$Rm6?_8Zev)zR)r(T3s=+FzdjODkf9;M zO4eGgiYurT3{Dg*I`?7m_mW;loK09*`_xQMGf)%dqt5cMNt&RxozWIz&bd$9#rcrn zwIbp_6uBpq9*|xclRBx+9-ZNIWje>c5+gx$80;IIo&9;sb zbfH|Ad#%CI%{G%ABNquvd_HdjWo2{2=~|L$5;zU`_kzz3zXdH;AlTuO0FUa(luR!b zxBlS5eZrOSIM_pN>Jq#05n|Pb>;`KnMYxjpjxZ-uwXq%pxl^#hxe{~QZku7{@XC;) zgn#4#+kfY!C8CuQfS3;z;@rAnMcYEiA;drBqiei+eI&Ico`_wtj6PfMf%7aOv__5B6fhXaAE|jVj{vPuUXmIP={roqTNn|2 z1CR2)^jOamp)B8VfxO>jp;l%C-vh0;gUH^VWa+$`^J<$c$l5UR96IbZGjFVHf4}6Oa%*{T z5Ic&sg>;AgV{~ls5O$j^6}BtePY=)iyxjTWya}VGV4w3y3L|FwMEfeMvFHt3z=4gS zX~T=<;2K-eUm=yNhyfV$s1}qpTLt4MUqR0!veTD1 z0vvvaOXu{0YPeDHpi#|nKGCpk=36raIO8f~!hi}9fmmvu*qAE7`onh(3=ZxdqL&M_ zX+jTT;`1Dw&~u-7_eB{x1F4MELM(u1$f95j7@5wMW()iAHytQk~l>$d{d}R+zQ8_`k7IOgQiW^amyGEvhrwE9mw+d_Npon6%`R$sEXa7w?G@}2@ znDF3NJ2N>j;Xrl&XemTooua4=6b53_P^}YTgk>c$7G82g1u>x+p|D5Tuf{emdbSGH zySU+uS40S5j#PZ*4Q%zWDvV#4J62ji>4jdo4oqOdTZyPd2=1VUTlCx@W8gtFp+KiA zVB{dI#60FO<{ecwN+Qs~jWf?lEDYk>+z1{IL5T>WKKP9xpzIBO9KnqyUjBz+uo#+( z2%rVE{r>cRCmfI!+jF!Z2h^>Q$$bo99kOpblC3NbF78PS#GW%s!iPI1eU94}^S{B+Bt~ZVz23U~38Aq4qK4CxLAYg@4=DpAl zr3V>>F|fPh@sm4#tu|H&>yA`oj^-cYG{l7yO9Yno!Ql8?DcG5X%J83nahF!@lHSqU zQ6vNDMW;mc+mACN=|tWMH!TZY{8FALKBeDR+3hO;0VmqO*G>)qe!FQurd(TaaY&>% zThvjYnHX30L!~VQD;6=FDTbZYxf||Yp73kw0E>6U@34Ifxmj17pR=f}5LL z;}_2z=}O)epzJRQYK3p2gBwxc_&r2!l9BSl(dPCH;I^Ta3E!vy)t^?+N_pY8ID_f| zF#DpU7Cj%%#{Tn_u5HTz;K>QHf;M|Hs0j6-bS4Y#M|3E=P& zRN^KCU+`BWng0+ThwCt%F0tDb;4($vKtwKXC$%NV4fowp!bahvmv8LO>skY4{->8s2dptJ#+wlgWK4*uH?BKAP+aiuSl%v2{ zi5td~6rlWoCF|AxjHQ`K%LC?T!{3H5KICK$S`}?gLy)J3Y<}!&$e9d3WkMEpHdRPz zgruFdibU9(?l0baY8-yYyrT_IrMgCcd-h&9P)gy{cBZBClZLs6BF-($1|B>sm@_oqO(@rN^urCcmeP{zRxqqR#hID*tAA zz2M#LW#xwR`+QYzDmfP_had-w!Yl2>IVzPdK)y(@=fgIS5hk|?E7S%0R`@JOsvZwR zQXM6eB7 z`JAbMl$8lzX)f%)=m+EY;3y0s|EG)Ks{D7?2!gmlsb;TYRZ6NR;|&E39VRb7ZYB9( zkI_>XDzW6i*=ZG|yM0wbld+*sJilE2%z#AfAdmVV%OJ{x(Cc`i3Ok{30&ryYT2s)I z67LHfM=-xKDPS!X87--LXA1=jG$=htk^)p#3Cnm}%+(ml@gpy_k^toiz{JT_Q3*4l z@?ZGG_+smBv@#lIMS~cq@vM_*EM`Y1eKiMo3VVIMR!$^)#a? zGbJOI<9@EntQT;T@Nf{S$#{S=Xj;upEb}U(EWkg(15&Gn2m(q#Z0bAHGh{4}2zplI(%pBXdLYeB90?&o?v&iA zrjaH4)yA0#=Fw`SzNw>@QUw-<-w9zccq(*H{t~Q%2+355kKA=ci2&r+R#tcO9j7y+ z6Q|}GEtFva<`2&OmbVV##%w_}p7KDyc|@_wjoRgX)Fgf-BO|<$Z-iu&Jc!-D#2B$6 z6rkI-zKfpv577fL&(wO@@<)K36q%H&!F=FIIw6Px^K*ethlZ5X4l56t~ok&Frg6i*Kse zR%*SYplhnUmm?E5!BKPzJ!JRXgXDAr{z&VF0fto){CiXx=AJb369xZDL8oWKQ4IF6 zF}cPXX?*UWHOH3g?@62N>*ee6j-rJ&a%fl66L`{f6I|;VahBJq@dynu$p1=X#^KNV z8{Z7O-;!Kf#p3QBP6ajw$>4$2t$2PrflmCMmX?xpWp5LG-dqR3ct3y7Nyt1~d`A=Q zQfQ-7-r>Yo#WLLdy`h;8J$Pmjs%)Vu)?%jIUyB#$!?u@tTirMlwjE=(#y0=K*1)_i zbH6!?N?m0#&h_ph}#LfFi$^+ zhZD=r#x4k`*L3m>s24`SJSqVPVuIWPMQQ6E;spkfHOdS04)yk1oz-cb1^Cm$jvUg> zM#|>?z43I{>(oVr6}|Q@-y9uCHCSBJb23noSd{>is=WlG;gBRiOnqd{jk%T|Aj}Ji(r>sxw0g_xX20vbLfr!&$-JtujG1)5;22SHG*>?vE zvV)1qq!)`FXCym*^_=*Aw{6Gzbq#N)Lf50*mF!k5Hcftc&uc6sD|`^)on1Zux_c!o zp=Pi#JjJ(a_H`M~cg^&@BmBWmnxduMWiRCevwHixs0U#uEX)G2fFGBT{;8Tb&((Mq zn;T+Nl)HJUw*Lhs)@)3;CUE0#M8&hh;HS3u7&yMGW>e ziuB7(74}5S;%{5*uH=ZgR}c=~{hdtBP!zcDH0#cVg8rk5#p=$(+u!qNf{%nEC)|!z z`7}>VXXzzuP&BTEk`yZ_P8%eR!n-BUq!V;1qS)F-g&Ob0pwoU8&MTm4(DNky$s~D5 z3&J|eE`cEM=UvTIUl;>dACU`~H$YCLtE-G%Jt)4xTjB7m`VebzTd>yyYvS*Hu{bHa zeQL*nh0diShDHxeh<*_>1=3Qx2cqG;4Tsg&0aG*P1BBrypN@(Tvhe{ps4!Em7|=LR zWo|N-Yph;|e*0$n&Y+%Fk&iOY1^ymFA*TjH@?NP5xnQt7RpXhy z<0(3`g}9C5Qa!hj~oipeNJAR{lm*0J|I&m-Y=k?=unb2_We@s`zOOC?(2L$t?4_Qq`gh;YU zW5~-zU%#Tpp^j;$EZfHZJ|ED*95yDKRL|px0!OL`Kmak8ljp%eD`>JPN*9{*UG3sI zeh{`9J0|yz)cHXvi^t%7!27sa9hH!77RuwCc-39TD&V(5Gg~%)dRwnYBjew|Xj{G|n8-PD&axyP|d+@Oy5N0orI7uRqC0HBfoJAnXyGckTD ze>qFYi2OQ$hJj=u1PvYJ2LSK^62bz??rRsDZtBPq*nk^1wrEfY1dnSS{yKm1h(m5F z5A|uKi_a-Xx2G*0o^`Y9!wXo?YeH8rRDpQiq=cD-xaRjyL}8RLztYm{>)Tt%(UjHI zQPZvTRgJx^E>FoK#WLk`Wy;6z1ubTod1lxg4O7$Nsw$cVU}piZf#}j@D4mwxl;7Y3 z@EPFT=t}+gMccd{Hy43w>0pi#$9x_RZlWvoGKe$A zCpiSR8#SmNvm7mKbnED5Slp)0abR!~{1ADRp4s?Zu1);aFEq-~0QQ9yu1;bCxreMqQZ=Zsp+36T9YrD49i{PW+-u_|!d)AX(qj}M{VW(GTLs~(xfs4@QgiXy6pKZM}{ zN0=HJw-SGRR7ItQB`5mDMhm{L&hj6_lPy%{0*5u2=Zmsqd4VMB#K~1>xQVQgaRqhB z$0A}U+)*)os%&H91A%EYng|>{y`ApMyJDS^_~|*;)^J1whQ{ADc2+jFRXf_(lZLmt z=?Yh`x=KEOlgmFF&d|KUE_pyseHH5!s|8MZ*njkXIM1ZumJ^dweig+4Nk)0e8D z8=?L<6;@pm=g&ysR@^jVBoL~MifPk?G>VJN+ka?wSGD<-h#3CoO)LSYOOMwEuqyyb zg@f)xY};ZTUc^~1uee>ZL8|gXu|aRDH{mi|8H^=ASpM;boUCuV;j*;hcQ)LXOr*V$ zueE=YcdIn-zo99*JxhFgnxB|w3C%@(EFZ=RJp<2ueaNHy4@_0NDiaMAJdkQLkKkZ2yrba#HJIf zds*eYk^D@&J$wy5)QHYP&-u# zgXEhi4ji#|3Xttli+DY85~-C>(^gm3G5qLhy}r+M-0mg8?+foWp7r0TU%dyke>MjQ&vmLpDnJICT^v#U>s6{*+ zJr;-EWQK1>p+d*JD|OB;D_Ls6+*WFAWJ;z!mKXbE6nK@T5x!_tc@jc;zHYi4qeak! zZgih|D_KFnF=a&r@wiUj+jpqLx}%h-s6i>=PPlY*bTypSXn0#-N^J+;X*jnP>ukS= zx+vxt>*^Zjh#KA7L8Vesiw?Bd+gBh@E?UEc&Rw_1>6#D)hvzoOe@aD67f@hnd_ln= z{BtFjJC|;y(eduCaWP5!ymn?2EWyJk@?j75r^|)nTbUImPrC5pK=h-? zp#IzXZ@zMBY9u{#9Tnl=dYI6%P>`O9{6zVx>9X9Tv1SSQPXkCi*@ugam|Y9ouxG`*73;IvG96 zpchJH+%gH`q~%U@HoMMlop2r7wp7fIi7Vl1B-@{^bm3_B-0nzxo}VW!WH^0K@Fr*< z&5#i0C4tWkOcZv4(^0<8&4yWP_Uz_{gJ5ICMMOzieQ{toow^G*)X{9grBSg_IWOR0 zvYxS&8w)a-xF#p62^?iK;cz@|a$5iqp|cT50CMYVs{eM-k#e3T=PPuz7+qkf(!9k& zQHSH4VzG1fR1O$3Oxj<*9jEd0d|(j-`H&M!VBAF3Gy?Om1`;bP-=eDCqau2$ zAqbXAqQJ9#@>;)?G9VvUbA1XB75dwvmS+cLXLOJRsva6cE~Z+T^z{!5{yo&z zGip>+v>}B6_8W@`A1xFIQF9Y=P&GA8S(um&fxSB@bUwS>lV;5T1?c~MVY#0HHgVJr zkT&Woomg41j$pQ3sL$>$-UioHvI9>>z2{COjK*$yV!;RUGudJuiToAua>9N(+>lS2 zpw_l6jYJ;Iu9h@i=f+5CQPf%+Rz2Xc$jdo5GN&CW-^JZ^ITM{$N$5ogCwTIUV5i4w z>Kxm8zA$^YPU>`+wutQk>EYZ^>&no?;QD9%SE(u=Z*go|N;Df9G#hQp?)t_#c;5Ov zOfG$NY-9)IDpOjxG~S)m#hJFfdRWA$D%;I5QQ$kn!mNPK>;W^tDo;(qGTEh+<1-D+ zIX+DO1wt2$juqgBx^RjiHY4{e{!-bwH*sS41OEZ z;x$>LY}B~rR-p}1L*!j&M$#O+#3+8YXLtOIzqcr4lwcryIR^M8UY@BZcb>!Fbcsjs-qsB=5=%}OSev&Fek?*Q|c-&~S)NYeR|NLm( z9q;4g2;FI~Sfv3G6_X@>%+?5ZQr`bMy1_&8a0@ce+UPstNYgmmwSoPr>9c~Gt670K z+Bn)!AebAS7;LgGSKV?mG;C{diY+eE+BTyGNz%L(KqI{tUNAA2Y1Du@8cRV-&N%F`4RJM(n?GP12` zEX^>B&FOaY+&1lgtVH+)?vsNcxZMPm3r^o8nc|Y1Q zcW38*UpFUz6V!#Fj9-Aly2u8`SGP`XPXbLi95T0Gtmcwr z78gghf{Gd~v00IbXroZNWF{y{YCvQUXDy{QVu~8e&30WcjB0sX(S&RaXCi1KSREZ1 zamq`ZQt%=u0CGlKT^~ZAl&%isXJjXBp}X>g*Yb(wRTO)kwV*~w^M%^@8t<~-go0XX z8K5erl7C?UKM3H&rR-}~!Lbd=Z=tzzowZTSh$jH4xQ90X*7GMRow9Hm-9ht$-PeNz z=44r3yqQ7m{h!!q`?y+aF%9Q-j2F@2edCOrYAdS^-{avCYuCfg?CTK}{9ECLv;nC} zL1ZIP{O7McKT={>ksHN)HSFNfphGtVA(6<~Pkg??*qBvxTQlKm{(9$KdD|Ikw}_3V ziE`*vDOG>@n+v2on|yn zhUb|P;kfn2Oef9466;Id#7uY24E{w)g|BdP z4NA(96ncxvTqwV_(G|Kp#>ghS8-AVr!m*M9p;57&8|T(fTUX!P(1*yw^Y7s4_}Ol= zzkFwM2VXOxX`Qd+q~qopb3~*6U#7QM#h1Yf4+*8`NW1@5cZ0Sc80F;=do|AMe5K&! zVMseVi@4Id>Y-`b2|=aH6ERV=`ulPFT8qFrY67%#IX^S$0EaZA#A@&0NVmo!vN)SG zQ<3F71mtT^^EaO-7bgdg!pH(*8b_La5BOfZ;^^+hm|2N66O4p+CH1ar%l+)qH|_#+9RUohfauf!@#8spWoLKgCDsQ5r!{fW{WSY2*{&a7=81YelI4T4|@W7$} zUp)7&GAFq*P^O)HdU%|x!tD=8qJx`SK^SFVm^+l;n^Oc4?_!FA}@i{rMZGQh-T^?NP_J8o@{~z|G|4GXK z=l=f>m;PV+CV=@_r~lWi|0m7)?{eljDqO=4;muV1|jQDN?Sy~P7CU#ZP|lZ(jDFDNKdq{x_TuCHfgW*(px)C*H2 zC1S?`HwFE9uLq$Jsx!D~tkaHCiW~|Cv#CsqaFw-2ivk}d!uT8aPP;Dhq^o2k2vDP; zCx z>z=Qe`EQ!vq;E%(`);V!3tw+;GGI*8**qWsC8c@QD$TO(@W{vx``x}UM56AJN@S?u z-Q8V(e}9$^jd02Ql$<>hu!4QM~p672l zU)}dT?|2raMq|cwabjQq2nYyVXyEQxYJoCkvLp$h^Jf5VcKZnk2ySn0JKV0TU6&;_ zHL(EdnwqMfj}H&du+Zx2>dKz`$J04{zF%H4!GA1QS65L)USD2}jg1S7iw(nnXo}O} zVOaGeAIAWAbS4mj2h72N0cP_VetZOTby0GZtsdX)?d_gDk%42e_otOf+&@QCz&(nn z_=I*7!D4lo7qgS)+SKQtrJhuTM5x%<*homg z01yH6EG&ceheW)k239AIWfOz`!~5ZbhJdiBD3r)S2?+_P;J&f3GvvVNGceS%CdvqP zUJ}(;X;5H)apH)mDC4@?>vmQA#>Vgw5}~6PJE)ZPz%2yfaSw#lhUb!v+tDCFPmI^}+|6rFCDOs=(Pf7@t;7+y#)>E>oEFf|wl7*JZ7!Nqpb zWL0ZE;FjeLbF5I37y^{25iLgSLmYX~5}Pww54YKDpqSXA;JcsTt&rm}*QQ;qO*=g~ zNfWLG0|T>S-TYQShg+Rg^_=QcQa*X}w;)W;V&OaKrI0yTu+H<@)}=BZ4ITX#K$^5c zBoZwLsiIVG!FoUu+6WaeGdue;;{T}JF9rTSW%H4{9&IQn1y)yoNXV1w=Q+_ypFcf! zpD4#?njCCYE*FX@1>&*;64t52CU{=RS$CVnCGV-vL!+>f({P*rd=G|(zfwbQ$Bj~< zf1N!&{Uw&;eZPJP?w}PlOEl%h6{S&Uugn7i!tY|Lv6;46+mFq9DUh9H%LprYo_O)x z%|54d(m?C3$`p~DQNW8onb^ktT6tvFAUDG9Xg=X`dd);2dtB6fJQ_N)U|PcMz|N6! z08^dX0_y+THiE&oV5g-)0G>4{#MC(?v@KzS-6$35ET{8%gZ}D zI$DfTkt9TW?5#Z$p+-YP%VAM{9oP_R!dUlyX$Px<{>Emy6*gVH(hD%t->!Xc{X?Zn z^Lsw3was!QmxX~6p!s(9cB)0#;xmu_`98g%yHw*Eb4ekhmCxvdgpAY;ubV3wIM<|%4q1f5!ND$x#YbNG>}p2!Q&kMp zCB4=7^kI90F^8;l>vw!}?`+NU0LeL9^#@m2Yx6@3?|6X&=a^*Vj$fP^I|!OV-B5?~-kx|3e?zBmxBDer zjh=T0#+rbs=Buf(=hnuD7FPPIPHUKAJL7=;F3#B1K*oU#f(KppQ^y0NDt~1qb#Bi0 z0fZvA;SQ6xVh*Otzi@#&aj2tUVP!78*I+cgXeCtV>Y18cbi|;E{NvjqePc0)_ZIQQ ziCSVWXUn@Ii|Tqu`bfns_cF;Grg%;I?H>qXpo?nOaGAfo#$VM8bf@A%C z;JRyu7IN6PgBObJv7-d=J9?U!hnn(Esgi~O8%ebkcO7R-RO7Ox%>7_8ncB9Eh^sYn zsB^oqRh>z!tGl{SrsE474t{@=X+lyatz?;n3O zI*K=ay+sOt6p=Ejmf+yvGxgx%{W81MT=MeTT6vvK8c3t^!Z*IoBC47kUZ$1f3MgNX z>#<*T5!KaKxdMUY5qtLR9v&Vlo0=kQnG4Ez<7Q`SW$~liclY;3Mn+O!)ag4I-ZC)f zdZ~20+g>X{6u6(PVCTLFlgt>Mwf=7=biR8c4J7E*rco{pKlpm35r^G57ZuVOJ(0L( z7Lw#&mowP?}$!(}%^f=xv6Mo!%Kcy|5 zv19OR_<7J&*W9Dm-pMVG4oHG$a158-Ek6lvZ;MV>_sep3xNPP=(G{k;J>S}_rM2wO zewvGFUWq!K0$XOI$=T2;baJ}3Q$=NEoi}kSC-5w!tI6q3Mt*rORA9B% z7`pt#{)WBmZ}Tz{%&OUa>B#I zD^tcoM>jJx4Cv`@0<}P_H`i5fKuk3InsW}8eMP5^Fywr5H=maiCWP9eGYsm^2nE}a zmQ)eYAq80MUJaGd74_uxrjJ1MeBo>ixj7d{MnnL53*&oux>)bZU^4#%#Z<*Uo3Z0i zejY`$_nFCty?0>4ggeHLv}LEleX5h^n;#`d#wQ&g0LeIyYZK~Zurg@)m<#&Cr!f>} zl>jv=!tBPWOzJiJl!$RYCTI18Pgjabl$z0(Lx|@lQwHkFKKlk>nrz? z-;OLNGeuGkObL{x7GPu^67%(#ch(LfZDTL)fmudF`=wr+s=gDLvoRG6YU5OtGGnAi zL=@ld5`O4frgl>HSWZ_LmEaq0B;Wnil<)I)CyT7DQVxtKnUtygq8z&s!NDRvK0X2h zK&j(2f7FBX@(e^pLzF2OXJ=pandMO_DYs3$?gXagEc6&u6^4;uS>BTL`{(BFm>w&V z4zx(_Yx8d{h$Vpe4Gr`8@oSr#hjO+xQEHv_xI3aBdJkTg?q7`Asgfa&j7Tc_{^8O9l1W7QC~{~lOiT()R&~Uhc@vXVA5ltc@`MJo zB3M}1DQ+%G#o1Zw#qV3iG7<`xNIB1+Qz5?ZKc~h;epP<@!?R{;`koAd;eHx2+~ivCgU z1*zZw#clpr<3F)$PPd8_X5Xj!H<%>^M6l6d#E+bpE~}J)hHUMP8^!iAl@h( z|CKiIF9BNNvsE!gwfBen$J=+jx)l9M4< z0{NcL_m(JMUaHJ=h&3mDXnfq+6O3i^RUrsjXuwk_fX)`5|5(!iEbL3y)GhXuv@*D-K~tgp7;N7hLU?#oybFm?u14>yy5N zQqYjdR1?@1i1;HK&1C}9%Pr^6SX2Rz>p_vRV|c>I%J->EwpNOtF2G!I2%!Pu=R?tz zr7mKz-56Z|x;`kO`g3JRE_f2ejsk7BW#ya7`g@TAXV8Trr|{bG^9PQHmRvBj-qwBY zW};sO_*7m_m=yKR`4u(n>M7`zea$Q+Oih6Z=#k=jIf~(x|Ic242lMSE+rxj6ZV61Z zB3vz+m%mHb`8!|wHr1L_jUF{+?%!8zTpHQAv?Na;Vel=^Z@MG(n6Y@Aq^Qt^O#dZ| ze5R)OmK9QV89Tg1YI(U*zYjiaHbpHoR47Km|J$vFY_H{)26;9*M{IHy^^8$@Pxo!! zNNF?!{vLX$`Upy;$A>r84_y%-~MX7ix`zGP*1J6JnaN{K!xrtj+E4pU3d7gVhZ zQ>C0NoZycwbgXxjY|j0<2&67Qy0%b}#Fv@3+v=nU7v0<6*X!|t1t5hNJ^-7Vnl83_ zAA0@{`#cr+ftYWXE!%y4eE|UhwR%NZycn6J2oSuSU|eq=X}0}nFx&6<5iR?f>D4ah zwa>rP82%A5_lky6_+scOFG}Ifd!8K7zJtr0&%8?HJh{-gEBk2TR2*K!18H-zSvxlX z8D!27DNt!GU2HzV#3Dk2Uqk6`Hd(CA%&5w?kFlj@Q~JXLq}}xFrlvNLziV+eHe^#N zI?&a;gsFQJe493SJxqtIS9{zZcM3N*x(>m{Ubu-d4#NNBD_ySL+|l!|5i4_exzo%htNacWK+E z+6@uqId(Mx<&8$6ga<=98Lj*mA(G7($}8Z?3U4fpFEd4Iv)YHuO)dKrHKH3-Fg7Lc zuvso`+JmO4I4sE-Sp~%IYurEd_2N8H+45-(MJ*W+|9v=`S`ls)6507BU39nDO;XjI z__4zC6=`_@H%2l!rF5KETH08Vw$k6p&kXh%s@RKI9EQ!7!p9nEG!>HWDs4z=tr_vT zzP|hi;UN-#D2ar#@F*H;>V;=8{Z_?(b!ye;t4#(xd$fH#;Ffj947UGAOzM`)TTIOa=ILjE0I?nL#pGO?ABSgz=mgbmP z-5#t?%f*1>V)~zRipZojTI`litW#wA-%`M%gjpj2nu=*&X5&v)7JnpTdZ$|U_xh#U z&-Rg75Z@9Wmoo3gs>P!?;eS6V$(N1tlfUb++2lB0jV$)L;yOQ6RywPQxl*s}@fgL) zl`TjAnmp+!dA`fmX$Wt=6$sL~%=bz!85-^y3S_LXo^~-z3r&= z&+gDvLbbY-KiwV-!!}v6Ergk=D?}D{l}mq>&IbA2?Y?1Ga(@5M<AG&QtJcy%(*E3nMK$ z7|b2DHWod( zPOs(Afr(hE+$S_&Fa}Qs{|RXEmuw^kT_@L?`AT0`y9_*4fCsC~`P)0c1=rEibEuQoX&sVPazbFoY!ID{KE1)hbNIE}Ebj_~7YdU*~kYy*&g} zR8$0no6!t~t~EZ-ekuRJqIlZ$9E5JO$tN=x)eSzjImBV_ylWMA_tut{XY}=~A5t+q zTw%MrG|%^E(>@uJjtq%4h1mxi%H!FGFG~i46I=T#JO`fg>CpD*6__iz_ zo5KH72g}1U(%0X)zK(spIkYg(eB0my&G~tPVSNuT<>OL(27}zQjpvZ%%h$YQ4*?F` z@We#r!E^#b5mwy~V-7(83b+8@kJ-Wvef?1cw!8|G-|^aIWbW+OgJumR$3+8|E6?oS zH<>e4=;RgNU-WB(fp5eCgUS7CwRS%ZT6TAD7-T%YbTZYB8H0{De5&|R;9D)aT z*Wm8%PH=a3cXxMpcMI~=dKIdH5d4GKMXV#jvx~H}7>Z-eYq`oG6`Zo+= zl3zLBYC}z_RgvT2K!-G<08Xc%>TL#`8QoWj`GzvV#x&X|oZ9ybDyD1eN&@S+@s!K- zTo?W}dTr8~y2AhT0k~xj?`$`eTC(66>95i9PA8X>Qw{l_n9b%eL`d9?Z&-IMwT)X{|Xvo(TSrlZwelGS-T?d?ZqTC5R(=ZmK%}`$6r*~T2g`Q7g;{S$X zo@Bb?+uGQxm0!A2b*fpYsim$%D*dKQgPU!T=jNxf{L?iZrD9_umE+)SesFh%hs^8G zrCXgH)(!^A9S6?s$J)u9>=hqp6kjml{afixQR70%^yO+@p~E1MU-EphdV9854t>5Q zC5i1)=jEu?I2FeL(<#AzLuJ0pFqtsZMt&%Y9c9afcYe=CiMUQ8eU+o_bs1Le*0Kos zvr*r*D|3V0X6%I3TALK8H7XW9Oih9UPGi{0^yIMF?fSz zj)2$I;ch7!5@7ILt6f3(^GvC`np$|EkXX$$UEeon9+#bJe~y;Xc4`Q~ll|i4-^}-Q zms5cRej?<4;5u8W*vvW`f&u_ga&y5aRLSm^OcgVzAgW?9-Tt|@#X;pl*&7_{-@x25 zQiYMJX<$G=34X%psJML@0y=t0d3m{Zd7YcekSTVk9rK=Zic#tNZDmhgbUF1~8A-(! z2_NB@hqd;jPkS&KAfP@l^>rfsh2;xnr|q8X518S?#L5muu$g)dw!|2b&c(3=cEk{V zyv@?N@+C{49?Q-9x%O!6HT~zuFO;qghUf0!gZK$=?bq>`cIYqF=XbHLLd1-WzvR&d z3aFZtO;#~x^n0TXsWZ~j=j9W#Lh&3=J-jae!&qfJk0qj06rx~rI8ko2NUyHG6aobZ zeg$u#+KW5Qq`m2)2%x-6i0~l5+^qZwAmNIL34A{se7Rnm|3WzcW8Pa`b2eqyJr^2- zK}@}SSL}H>Ek;Pg;nV6u3$`9{p=0yt^7Pb)LHUCUQiF|}dVw`;vy(~F?$w9lRiRCO zh|QHs20TxsQRTZU(;nDphNr&Ph(d5BwN{XPo%xkRr@EXF>OSjPu7_#c&YpnE(TI}l zg|4>tztPHr&mhf=&RJhy|0?`&5!|XIM1hTs9UrU!_^aO7WQ#w0hDL9Sg|tfcM(lAz zr3eO~L>*hfRsM9JIbhN5X5~l!S$Kr1UCjv5?k&ISns$GJQ}NSG{xS7>JaiwVyEir% zpiZ1fN#Af!8jzPn_h%oa$Z9V4xKz!Pp0NpNALf!jrwznE<8VQFC!!{Q8oiRn;rImbqIuQ-L(CU5 z&QWVJ8UcwmRx(pt7|E)tO@{^m2nf`KY$I=(C&EggC!XY}RB;e%%eBeC+x+w_@3xl<5wc;R z&`RdX9#`-PCxhwq1W`{K-CJq{xd3>cK&aw+S&WR{zY02=dXt=;Xu2g z#V76cSFq2^$X_~$9&F@3GIZ@O|zyP zd(w3I+Vp1coo|Px6Q3@7=GtG|&>b`|GsNh{*+$c^Vt=5Z{HxD8o~`ne61H7Ezqxo@ z{IWVJZRZFuz0bH+0p!W-YjaSDMTD+dJ!PMEHaakF(V>}j#23apME6Il`Kc^~FGx`y ztjrEYjm%FsnlHL9cVRq3u+mhO{j0)n{NW!TA5#*Ag@rHf?%pWaCf32aeWj5gxV;h( zf8KrS;Zy*AOGkZY@XaP~uit0{OU*1@C0cc>E24#vqI{wS1cr?1u6FW*S+3sKdKqEKr0JD|n8*(ovCK%>CzR)m~S9s>grqq|JjX_8FXLF_)0^ zenQIy&!v&P4|SF8AAIe#^50~?5l*WTCY=y#skb-MXv4?OMpue+((XI&l2|R5fQA{v zbn|g}tADlDW<0(hE<)z(PuIIucDOistdf}VXwW`P`rXQ+hEdhY&F4#Ja_YrNFIiB$ zuJNgJW1XDP(ozaBS2A^)l)~X-B>Yvvh7N!$dF_RLKb^N<$A3-8#z&R1n=KRwj^2d5 zMw%-F%2`?f-HNZTPdYuI^g7)w|IS~3?wFsVRqxnahX)l}9)DBR&W>RZiI5J}+) z_|iqeac}nm>g4J50B}H^Jf4F7`AeS>F(AsoH-b~cq`e{hT{ZKwR3bz*m>0?#kj{1d z3DFV3hLrO2cZkYyx%mlWLs1e7sk>9V*9Gzgq6KYG@T!-))%*8#W&jKW;XxXC=l}(a z-MzC-LmWfd`$G1Nr*r-}+Oq9gvyvr&Ux&taRIu3Ijn!@mV!2yQRq!8NKo!PVr=v%n ziNayK(socsQpj`x?hH42yWx=^q0ku-Djj$OjF1H2zb2Gi-R@3IX?~ewiyMJn8FgDy;t+Ug~ ztb0}LP>@o9P{B`c=g!*2%kRidNh5ruBR<U2j5kyY48oc~c^g#|I<+HS;F# zeTQA(Q~6pR=I1EcZb~m+O6H=O@Ttt+W)eRG#xO3X(ZkdmUY#!pqyPY)yY697kN|<{Xs;uuXNU)&`$lAxgm%iN(nq3T~%e^)OekrAIP`b(&$04xq z7o7Tj68)NV$*$^n3jlb1#qRzd_||l-{7n*cw3c5&v~O!5L~UBT(t8jm8;o(RVv@1^ z-7$seM5Brr7Y-*FCrlMLh{E#pH983#GW**U0}MAD|B`6))voU#Zx7MfF_ILf#5d(cxza^YnYUV;)&N;yOG!lO7sU?U04Wma-ny$O`8&7 zvggHBK<0LkNk~AQ@gRd@kN373xx=l=s>6W~2fVR>at3HdXgA507Vp@knL%WeY5WSq zI(d{=-=Vi|u7(D{?Y1d>jU57r3b?yb(Z=P3#uq?jPspD5 z2~m-rOZmB){g7s<`0EU^1TG~_4~^oZ(39n`J&#uhBJDOGX$2#2JJ^Onu6ms@-Jw8a zhk=&$<=^lNxx0OghMVQbG8@W`5e~QapyM2D4Gh-8or)>;>2>AsSt;-I^@*etgt%Nv zu1?u->lECj94o(K8(0dpBdB91j*pB{bA5S)7Lu1^$diyO00VT@!a%3jKR?B*tXDdi zv!sWAm@(Owqs$KH!`JMhJX0Qm(6_#b1fow{rpr8mbu*8H02|d9tzQE`W4{v7*1jFj z=QT);*<$;XDjqD1MYO1`1?n?kLcRLK{)DceOT7k@QPNx3TSr@5?TBIe{83eQC-LIvUw(_zJ3air3Bg%YVu>H0HMlZUU4k(EstJ;rckZYOgR z;TI|(ML+$(018GR7fA3G6vvF4x0cwI>gJ2-uLY0hIp2b-KK#@G!3v0TC2129MCtX= zea|XlgM1?-==964B$VdSdCA{4T#*9k;M8@1+>IwYFuTr{r6!Hrclr3dp?zvx%k;9& z!8h(`vaH*4dALTwSv-)i(Wq`^eHEbhMJk66= zG*QJEK039WEG9Fv(m5EkoFuwF7Gg9;c#?ppEaWcB1M~WtqxVx;YsIIZ)iFM=+E1o4 zXD)O+%$$x?S8s8>>yD)r@RUIsFY4<0e7^JN3ijR>hg1!{wz3h7N%&8}hhh~ZBqXd> z-sLJ4&z1%)?M*aaR}M6sN%DRWDzTvRr3y30Qh)LizdKM@Y;3d?(@K8=PD>P?5hK$` ztiGCE#sB>b0Ie=gQY3x?R3IXTTzAbFmep89v zk}@06pyL9SCYD=YH~x@wzOw278_W@jFR<>Q9S9D6*u=g@ET*5%W-lal5QV)Q%&bIe zdPH%SrrNGAHR~#IS{*T(NV@Bi5p(kW`a3Z9>*n|5*6j1Z!bzm$_$Vmy@O$2p>wPjT z!(b6VBst(8{jf|~da$N(r3y{6-QoJp50z^oz4RHX!3 zf5janeM&F*+<$ML?N9hMdm?B9VvOD1kjUW6N6X;eNcfZQtQ_?6K)f}q=SU17b;8m` z93k_xr>|U}!raU6^*{{y9iN+Hs-QQB$|3d4ulWmcO^g&8_ooLb4;u?Fxabp6>MuUe z)2}*P9Lfv*oGMR_#r_-$^IFy}bfL)1I?7Vbmd&Cl_&Aiz)yGF3W}1-t#_;1yT|>2qPYvWr-V0K`Y>A+sk= zLBrbEj6cN|SM4jSIN&xn6^br%N%6HftV+h)+lL{R?g#}!yC3E?J(!UgcI}+LIQq<0 zgQDcYu$u0A9RK}bZ{!QWe=$ZP2;jwVeMgsnuho*YXc@)#*vxyCz7nR1=f3Xdq6Z59 z_M%zp3;eqh=mBjO!Z4o}#8?!~VNuT}?br$nv->lNN89QFR(ty#?XO!|y(Ss9ek}xK zUh{o5t5f*hxE%-Tb!Z4!)Vs^rcPg60`M# zZcbNLIp5epdlOh2^psk{6U)5_#X4)>qh*=@PC*kx{ZBFC?*~PWr#<)RS8R)v!HpFR zczCJ*3^#-`y{oxyET)t;fBQP|cN9M>_c&j_@Xa;9=O-j4Zi;M{;?mc6pzc%_HCyNi${kY~lg<;~ZlzD7Ry>F*io4{=Dt# z_ugEhom<#HjCrHr6O)Mj; z|Fe+Sg5u0*_aoyv-d)p`38WK1NHNt@x-T?N6?ClaHv-4KImBjn@=68~hz|y2Iz8N> zl}s>4YJnxMuQWK5u-XQ4A~2dlla7K%JANN86)or7pI3I3Ywv^hu6_)rqJ$Fp^N_$m z2#{3{^aI5r|Mk|rDu3{gvvE+EseaIdftbTW?giDoEI*LhD z(xQS!Cg|ytcQXE4edp2?Nb`r?5+t1~ZLV$O})k2Vdzx5O! zRHR(eb@BG*@&$z!#A^PupAK5Ee*u8#%s}y`nAB9)r<=o#jX&n*rzVyJ6cl9yiieBU zxla-x#~bO&OUgg4xrSXDU&VW*4lcI4G7HF5S68PpnUj%_kfaOq zB?^Lg7$6Bx1OEbmV>dnT|M#}N%n1O<5^k?zsY{NUky>hacsnzx zpD2rrOvr9xUP**KAFynOqBbx~X*5})_S^0d{Qvg%EyVExj8D9w(vMUQO|9#|6R2Q( z6W8OlyZDRxC+Uo%ztwIYlZ7m^d7jNf!iO8)VN5uS#ypbIs$DW7sL;30S5fDS2kb&+ zpHr(p-URs<9LDfy-Jb)2RoXou;yBVb|L|~$c&6*U#i~6lqVu*#QIdEuTg|m)`f}Uj z%fu@q>HIZzr-Sab{bJr~^?VX>PdlfxthX75^&mtP(h&YjW~NBUw`+AyfEO9Z%VhKj ztY#impN)Ww$J5OB)HjobJYa`>NgD@(-gxNqgq%Nuzbx)}SbK}TdLbCwbdgD3)xlm} zgnL4UTUr8ftQh|xH4ZU~%}-^TDdup|Mf9zcMd#Ph%C{A%eQfMMmWVYRzTLG~j3O0f z=v=Oj6nusK{R7iEukB&rr%VOzwuT8V_1W#s#>Wb@C8|-^I1v0xK<9_{Alvs&m#a0F zm6gpDN+}r&fLOpFem5xY9OI~?ql5GfJa))zrNJDTiJ-Op=3L`av|we!T}pF5Z(U_- zDlV#X_3nIgv@wyfU?eDaYNAKUfr$_dG6ER)U1p+!$^`!xPrSp4mEX%+LlPB!gkYzW zi@m#+(hm~ZACf9AwQU_g0>lmyOl7P|Eo(G8+h{nr6SMeHe`u1-srOhk>?BJQAV6uA zTa3*|@#=49I8XX2+Cf^?$@Myp_V{ejlwwVvxhQ5QRDmI5x)QTooy&%G zpu%>+#Y#6PkvmibfnaEQE0vYejLYuw5_Jw;ybNINxEP80k8_YC*>sw_}ig@;Kby3u^9cf${rBn>LyC>ns zu`nwID1GxYDMl~)8QQnS7x)7W10!+OP4-clkceo{_g+F=+~w}X*w|R*QGzrxoi#S= z%f?Wyqa=FWckXD>&yxaDI2=lMbF$PujP_Oz*9|b{zh8UT-BNV-jt;fWXDScNLI*;x zWaX}nXgpdOD~f$3-<2d$d32MVO&qqDbQv3NGmU4a#D&T)>&*=R~j4_o+yso3|u-C|1QR7 zYeK7>9V^{SzA(2RXUl!3&diinDpkb!9IquuYSA3wGIH~%oGt&oOl?3TQ_HooJ5!##qDvXH-J%y1u>%F` z08w89J|TE996C~DdI;Sb$f?~ygd!YO`F_m;^KyDNbao#dE@NIo0t_uI;EO4Ki5CS2 zI2SIMc~F5@0PBv8b((j&>@sB+IEiAx0MhaVrRv|39o80rk&&z&I(Fo05qPi5T%QXv zV>j+0-5KkexWy68s2Y7-i{vXSD>2s_Ku*c(;&Q`U?-d<|>A_Q7*SeB(9b*cb8Y?JO zE1`^9^4`UwwI}&+00KwF+jF>$-9Or9Mn)#*GX|4ewj%k0$|(M~%)20-b%*whz{ENR zbA`45>wWj^yge3W*cZV)sy7Y;%+N&R#Vk4lx$^+Yz5f0L3klZ=j@UAb%&7HbRHOYx zi#a*Bt7Ft6dNuo_9AUw=?{H^^OZxTTqyo1U&MSYsrU+S6#nex1OJ*^wtC%^^h3L6w@XzX8YM))V~snpdqU`KYQM z8Z!k32v1plP_>?~uCSf}#+~ecHx1>PjhjtuWMgs&>^I zTkFD5H#M_`&hq6(%f5BC9c>rKekwbI-RBpikA~$qB67U*X{Hv&%8_Yx^VfO%WmpKM=w;Jy?C&wB%Pb4NG?m-68g#M?CkLkve{`lu{MX%7$; z*=;1z7+a}QacY zPAF22zB^~c)LWWyG0*u)TCm0}TuN=%slu9=%I$_vbsq$|Rxh_ey^nE4@qii64o<{F zWzBAxzynHpN3@v(Sz1H-MA5ynbmt+_d1 zNafkt>}keoRND7?P+V=dIRY|KzHe{)BVAN^g1hI3`iRvU?2R;(VR8<+?8aHFe?1qV zv^$Fz!E!YlzdT?vdJ>jnCDzn77Z;XHUv$T9AG(0*(RhrLJ~Qj_LOq|4>S%H}N+9{u z>nG#DuJUxsw}N^b5wzx{ZOeQcB;}Ahbu7fJ^C`;Zj45BNJpftcKYKm2%x@28$<-9g z(dmSNw_8q)S4(`_)@l`5*=t#`)NH1H z&j-f%usq#n{4}xTE>u@8DYT{i~cZB$F+JZ|C>&U zP1faZ_!1+B32|StN_SV_RI9^+|Kdj~iaQ_iElmRZi{)cMcKcWx&S=TNY?2Ma0iBw` zWOuD+9rU{Vq}AUCf%OwVISj?OWjsNfXJOCQ`=_JBGDLi}fD>8pS5C`x)2?{N&;duJ zBhGIaeiq`mO0eVbU<2o$KU2Lo`?tJyheuNOkW!z`?LF?WRD?k>cMU( zKVr|l92~YI$g}6BE(f>f#(7oEz*)o%VWgfIIa*+1JJxz;hzXDKn+V=4UhA>?`0VO3 z%m^EOtpp6OUIr%6@>YFA6XnIN`j%aF%OWcB_%JY+UyMr`CTU$%ZRP!Cd^`4@(T$?pyf@UJyp|>KdEMxzkf!oOFHF}hQ#HcJJ6>J?;IzM zRH0uKA~Zh+(0d;&Uah>b;)VKusY;6uTw!eIj+$$%d=uxd4F2N!@La}%lEz_u>yf!Wu&@|$lfKPm?4`PfuW+(+p%5s? zxWC#~J)PF6y(8@fmt$v}_S>`s;cYv%pQOasaP&^08~*CjdchtWuEk_v2lto~t|SNIb+yzOw@|9@l=|T_7zgP!7;pH8 zih!+!_pUl}O{WP>F>O7kG8g#y^)^{=|G;KfR(syQG-FNY1T8})!xR?h;wf4HzBAUP ztasgWmftEYP{INtgND0TD}OuLThBi)UZ*Oi4JTsKX$N5vtqv43zjHNKbn4um3J9Fk z5lhwlOmL@ofO~ddoPN!$LuGfV8qe9B;-tgD<)6atpfGCq7BR`dd?=3fUb_$cf}Si5 zzozq)@I-&2tag%ad~v&?^8Gd*o;N7xY>UfM(EX2kV$hM7vB!+O6~D`HL9x{1PP_F4 z89f=VAL2V(``_b6PRjFazD<>@r;cS7eFxdWnhSKq)$E-dtClcfmJ zJl6{oXx$U7yUukYu^bfw&l|G+lGquynQ-A1oi#vz>*8&P)DD9vGKw9Y=%tc<=0v+f ze7{KsU!9@q+v=bD4YJ}t)Y|rakIL8_8}CpAA*(e#*qjMKj0@#edT@YU2#|vUo92&K zM1-W_#|ZZ;b;=BC@6fj4`V0>LHNKy^!t4**pmNNTcca^ejw;8^j^9r=owc%lvnB`Q zYLPV=`9b5JrEuNyIl>2I`+7qA{WR7_9Y6n;FE}a+isLzA_Vh=mbc@QEe2`PFR9v2V zik?`c9%TMhIeJ+H$`UrzxFRxi^^DR;`qEZpop~+(16y*aN;f{v^&3FH;G{A3TlUMh z&E1R=W6=6eRaGiPxL5!Zz8Uts6BmZP&Bg8|Z`6{-+6RX5c+(fQOj0?T0J0>Yy5pCM zto&?%khS%1dV2l1=!79bNNwJdenC;s_X}Pgp01UZRNi778#xjrOG`^|aQ|_U^cM$P z?|7fMyriO%+obq`^G*?G=RbCl-fFQ&M0W}UFSfsx#{1SvWa_SN53G=#!3i1NzZVv> z#l{VsbvP`l!3!nD%>=#q6f0488sPFL^bWN3yZoq2{2NqxZ$TU-sHr>V+!pXf>0n6= z*F!Hdi`=U#=*SA;JG4+dcmL)T~kUq z8;?!D+e9D=(^4)=og2im9|X2N!Ok-*11}V;PN+z)%k!^-ZjT+BOTWPCd`McYHBa@D zSgpK7qD{=hR!RG=2hT zXhfbpxtCKqy&U81zN|4*7&9z`7*SH0oE%1zSwDKoKH*O&h3`DTXf(gS_`dEpKCIgC z(V6n~Xx+8aFSTM+;Qhv-`t^fZy}X#~u-FC1xud&Czx9u71K5Dhr;hjHt-7tSnQv4* zWbIQcxk8qfYEa>B5$o69SYPda+bvCgKMfW*=%_`Uz@pVB@rk^X>o5{)l75Q zrhi6^Pel9$+(SkX?#Lytsj;xsY@pieS_$((Pw``^$1JWp-TFfGi^PkuW9kB!m7NXR zhi-kg`r9umFc8E=5;V*o^{BNxRH@g<#DSq|ZZD+R>?72e$G$x78i0$7T9}-QL3~+Hr4b>n^ybwa&wK@jU_V9X>Lv+ZKap<*4ZbHx zZG_m+vAbdskq`vU&CY_@W#rfWEsv!U>+b0pS`x8wS@F1YO!lzi(!!9siVC#wIt(vE zFP9B^6@-?*GpGj1jZ_W9r4<#E!bZq{MIA-zcy*y@G6 zJbs(1j_--L1P(==FgKm4I7+VNN~qXcGK)8WU=|4(1+gioj-YUFcUAGEZIzu#fJel8 z`mP+ny+A}o1UfzthuwMjnPXEY2ao8h%X_(1K9vp{n*+rS0^scnG+`m~_1E+?C$>F> zQvzaIYne2XK~NR+0zF6L$_6JYyZH%wt^KxHc)u*IBg;JD0;`W>T_4LK&kHF|_vlRRMUWk8PA zIriYTD~0~0v6dk}wg8(6Z(q!VwEp$+SPuPwv`{C61DkD&WbepO6mVQ#ab9TRtIl0S z;gIP7O0m2%3@*1@*AcI%SYNyRqWnfB=khUAUH&i-U7wuZ+G*G6Y zvQ9NCQW>8Ex(q1a5zVKoS!H1YSxYU$n&49t+^@sOvwXka;h`X#TJ*Y~I2@-hW-&6k zt+ooO)$FZXKxQ{E2y1M18n~M;6*tu*Q&HfBA@=zTq&?y8mjm`;C;N&?$@BH~8lF-G zO~^#wU1T=Vpa9&pRap%ULw@mFFP~68hCjag`mk!c8=^B&cFAk;{Ci@${pCxns^q703WAw%(~DU5aaqaxctAbK zyINg+c%ygt;s%r=ke6YUBfox9&bMGm!9U%xSlL5!^!Xt-U}BKATzj+_&(6Wj%mL~m zrMmhV0>tq&r=!{N!MS#(r*#J=FP(X>5|G`#gSU4`Bhp1QQA zCV7X@g!4M4E-TEQ!R2Munt0Y}mnPWbyVL3QUBk>0R@TGGyl1wQU5%yMqt)*+LiZg$ zSO?hQwV>8EH;ZcN?&Ngd(1u#-fAb|*yOG7y)M$zty@lXR)&ItwNJL5%p{?wp4S(K8 z(A&rF@Be-~-MvEIk;|CQ^!2Z&g2X7@iZS(V`iaDB-a|4t&RJ$~5N`cz8^s|cE?;Io zb|I04HBKB;*Nm)xC)snY*EU(B;*4IuCFXJLv3B-f$##2&mmk__?77Tb-9{)0%R2$- zwV#7=Rh1<}{!VU?j(M1+Y=_L6#h%^B8Tz7-g*D2+=gfD!sB19q{LM=tZVE768r&k` zY7LeNyNnKA!Fu>p`yP^kgzEfTt%PIh8!tnxF}<0?_>zZ@2{xU2CDloG%0u}b>lyOV zBAb0lq6D+|@F-FjZgx*|#@vl~_|E}Js+4k;d?{vald;;kBjCI4E|NQNLqtep0TwHc z*|rIn<94~Dip@Di?{o`9~9qM+?ORnG#um43y=3G8|ufm za^>dsLyW2JdkAN&%8E5v#|+*xt<#JYb@~SihmJiuNjbhZ@fRqRKx1>CVA9f{stIg$ z_<7y(O2&IuJbrqN-|D6#%W};v-pAoZkV@EFCJi(j;w>+6$Uk_o4nd|F99#b$~y=Q$6yS&7NBzz6ZJb>Xl*hA$<*YCbX0R zuACp-zW632pryfLIK&uDrOmSZ&(DH&aEo&8rcVGP+6bi+W*6?K-IYK~chaJ4$RU%I zaP?N(O9oRgKe%rFz|#(EQQO^s*uVwBcw8Ce*lNZ2a#%MjaIC(xP_=}7r3=zj7_Cmv z`ba7mTzIa8dtvY)xab(U^CclM zBq7NCs9%k_G<=#G0`k1v66g^=og}Kkk!@*|hX?CwSsWmDOFwv+^8H2F`h(0euyA$5 z2741Wr!Mge&JvDY1rp-10qanM&v8QET!DY)D3r20yVH}mO^lShs%FnLkI)TsBsQ(c z0G1;aDHSwplXaJNgP#R3nV6~U<|qK4A1}u2VToGZ;~^X36RQ|s#5oMB;wVDY9376X z1`i%sm~A=f=Y8zew3F7i3t7*CAd|}C_U}lYy&y+roItehO1V+B26 zV?jh*TwYGj)V>8{YN?Tv zoB`V)udr~&C=KX3qDQBopx~VNrGgyvr-uhOxLY%WBgtF7$2f9Ee0HD%GIVB%nKC@| zZLQKSkCL)Iwaa+**jZhs7VJjOxndvA8((myI9m8MPoxYLPoGJ$D@X{#5)1nI zPdIx{+Q-CDb&xL%pBA+4=BVH*K&KTSoZxQN4vL}aoc_eC;Q}h3cOHSu4NsC@WTw~J zz4zP7_0_)C7N+Tvs*OPc&vOGDqZ8ioTZ9q^Lp9=FpWdb(e3u?~{j$;`;`b|3OAAc? zmTLh#J|{tc{i~HfBJuUJXVy6A1_0PNzj-G1?Xm|0{U%7bQNcGq{^=1US)o3LAjSir zPC;B8n&IQm9w9$&?S5Zi;)6JAKv*K*|0MdQ{-29~-vGS?_@@pZ7XUxS@G=OhX-lIE*bOspb+FWEv`YyU+^pZ2`f;{`P}(Yuc8!g* z;&>#2g299e%E4>$b@Ac3`jRVIm{~s`qzyI7O{^Tj0e^$D);DN%pJ&&aA(QGHlg73v zS1iOicgUb6Ky?Sa)D!gf(%IUmP$({tBCKkFxZwfI<&Eh?`FMncq2X{UnC*3)!2v#T zesOILb3s|5nWms9w*n#NCC=1@xSEVAuPDz7CjsaX{-NQf)9rNga0X<30>Zp_7zogA z8ti86l-T5ew7j~S&0p_9_X=EB6|XkBYY&`HA=7lj0iWG^GLWOWI!Yr_ev#vSh%v@f z!kF%6febWhDV6X>OSzt(mSm46njvS9_~5%b8P;kHim9zjqeZm9dll?vs{k z6a=PY(#$#dWol#^|o?nA_rH5E;Ep8c2^)4r`{_!p}x zqD<$@j1*e7zf~Lz%uZ6q`y1rc9lDFM}YnOuN z+(Rxyho`KgR{p7@{4^ci@mXeELVV|A)M{=kCzlp#x&{EGIlT}AVv6}XIVq`X;0^=b z1}db^b{p~9G8+i$Pc#qwBpxvljjm&U2EgGB5qaVv;QECIhi) z`pL@@fzST}>FGo@iYtff8+}aWL?zl%WB3?DIm3}$Gts#G{jb4Cm*L3VKa(No=vZnD`zos`?`CHgd23!dV>Kn zS!H=7;tm5(!YK6hqwZCkte`_!HJc(b9@D+rLSr-6_7dvgVkC1ufh5=upx}b9*3&b<6 zNuqC`_>C36jcLz8V|e(9s3@DnVmOTdAP6wh{!RLc&zIO*qY3=3Fi79EqbS)_bf$zJ zJ79oT;OUS|r%%Si0WtUIHa=n_+uxZxknVf?ylCd2;`|$gEMA=u6|B~d4-0+Sh0LQI zvOIMBc{{VNcT)&`o(3Y=%|p$n%~V^SAy^**j-oT2exZi40 zWk2+w$WtW_G=gt8TC8mK^qu!=b%`VQ<30`fLH-XHpyptUjL26h6A~LQWMQzhG)NS( zNTkR5WBO+nw4Z~SN;8v}9k)T05th}}#oZ$m7Z-z$-ET1Xy}g+DGp@?VIZ#HK+5!4kI`C|hoP&=aqZ1l)({MA%luDpp)>AilA!juQ$y2w0k~m0<5n}lwg1~WE7M$ zU}tArTien;FE1}ACno{|LYpE{iP-n=-;C@%{~&q5(}I!Vm2z^np{Px|DsG4)yGTf` zCcmZ*xYD1kt1Y%C`iB4Ef7$-B2WN`T8_6GYN!-|FKq+)qZAVdQdwkE;N!|YGOG*^wIh>{yk4{eY3BFN1g zm&wGBp8K%o(#3gsbwFWA2w3d0^4H5rmUKxRW(#$45DAh049fBp5%``{G}?66DP7vK z;h)3$*DxRwAz~6qDOtU?50#1t+>W{K(2%Fq?cD-lWH~7$DDC-vU1qfXw3k+XmO@{Y zc_6vAfNXB1L2! zya@A+7#3QaelQd=uFkJ_h->7=vjQrH`JAfQZazuolTUMPDLI$j$K2i6|NAjrmA-xZ2BHTdiN4}rBYo>zSyAKR$-c!8&C|8o z={u*wOw;+hISol=X;BeIH%+$$8H>i!o05Rw{)~u$TeGoVzF)KtC*-gAL%Me%@IF@f z^F4h3tu~l~{R0V`>bS4#xytIb$ydagIVEnD!&~6SHkF)Ddz&e| z(ho_dYcHD}yoa(hahd}u6;Rtlen7F))a2yF#RZGw87A*wPY)j-U#>*y6mq!hO5F^+A?8Mb~ zsJv0e%7lEl{~faz9xf68!R!YawV8baMn>co6p+f|N>52a{{RHf%;;zt6&01WcRP#W zpLg4y`sqT8)amwzM~2jgp^?I=QMe)H>o0wT1bxs5B+b zv<}>%g6}bqxa#8SNQG!)>@9s7cLU2!M$Fb>h8k9wLP5XYZ@1Y%4RC?_p(!U>F)&iK z(!qB8X(h79k`65NEX7diPmZi`Er(loR< z7de9CXM0E)mmvp#t1G6@0)PBMtArF1DT8U2wj+iJ5C2(jpQrXF&+P89T+SDWeF3xj zdmsYVUa-lhNRTsBLrO|>kNu{;F*eu-Ad?c$YpcxI6Oie+p|4#mVX^p|E$U~E|85>+ znKBo+3kWm-9~wNX^p~j4-DspSrgG5)(@%r-VO#2#hV6ESNON5mNh>%DHO{17FPOxM zK?rc1wT_c7YU(Q%N8XHL>0(Kw}|8W(#ce4UEP(ene*@!JXwPqU)J-zSVCzuij z#t#D=@Ivqw7njfz209=XJ-pxAnJBinq-2;)B1str1R|x}Wgvx~6`CRaU&OML`7cua z_dBmUFXoT+tWnMxj|eE!EJ}~t&ru&Mv43PzfaiZ8@c+Q$e-poeZ^*cU-Yb8IibSxX zAp!&GVE|ytN{>S7NcgqjWaA73vR;Y=ZlDM>2#-04yb(HdIw{$jT0e?bPY;@^ej(Sk zS)`y;K_Wv7>8Sr?Jz4*y#e@f>3;uel7|jj<)9^nG@qgsMRq*PB9x5yLH$3|%#RYYm zX>du~EH${e5iUrDxIIvC9#az2l29OSA%6^{EM)#8Tv*ERZ0#X4qtLrLC>4a?Dsp|N z(heiGl*zV6ewvj_i6MiYon+{;KF}UA0zGetd^wzP%t%FhNBklwG5ePcirfCC=VS6Y zLsFp?AZok3n;8k$7|nsgwt#Y_545qHWZ=~h6H&KY&9I<^Y7$SMHJOI6=-i;kO$;Iw z;Vrna916=k5^%#8IOyOhaE|iG@OX=+Bw*4cXAEB@?z% z+J>ix1Akex7cCyeU)*LcFX*4aOWTU_8t7qT(wl@V6x2Y6Ik3e5t43?r2%FH9HWlHJ0##Eb24a5kLFGha_0b`rgERZBEM!fJ%T z@sg0x*r;+6nA%j!}I+1K6_v1yw_QO&6SzVnn~75 zX5HV(F&$*Wdr_ha!zGk&;rG)BeA8_OP)czbQj+&?Vd#YxVaYez0TNX|Ky z*e(WMYQFsw-QR6Ds}p@bAc9|iqIW$(kF@M%6wKJgt{Gy;Y5s|e|C`0*;(NHaC1#D7 z(??bE!q;m@v<=0W(1IynfE07KD|h?`CXX?TE@wyw-RUuG8XSkts&z)Pc|2o_-ADh=vYifQeZ!1`T zXAbGt;8y^E1>Q`!A2Hr9g>OctF5d#~(VigL?K<~{)fzjk2AqUyplUd7D{}ASiohc3 zI!$FP#kU~xFnm4!!2U%VLB(>W0T*}yE--o18q!@>X<~-OJGywq zbJX)V8x14G6Tm0`RaE0C`|LuozudmD!4T?5X1kP_Na0a9Pwffg!Oi0f$tj6(Vflw4X_FATFBKtA z7un7MpP-3vM+_Sc(QXZ#bE8^(6J%_xLnoCHoE8MNLDS!?+N9(>`XNIOR;w z!e2~$$K=dgUv64Wpy1XE=F>jS!9H z-4t1IK9qreZe9OCV~{O8_21xY4VS@acFj*Ju*q=L!->Pt_uf;IoM|;rHtE&(iLfYffzRQT2My^mP0fzw*5|0ph3?9!wNjf{UCR4d+gQfLa!7Cx38OB-mV7*<2Uc9aTy$8#YV9&32(kF|!f>epYl4!gNYb?49%yWDZ3nZmB!9Q3^Fm!{lCx3A=BE_pO# zEp0a@D&JC!WeU0-FYiTmYdF@N_SwKEZ#AAhzsXHcLQR%m76F~?Qd+`@nfAW05I1L) zuGiZPXJuMjL|5<41wF*-U&JPo#d1e<006{FNri#1uvgPCY)Bm;DFED94Kgx=aOPcM zMu#G#HX7RFk>}Bkw)bu)%!jMPol?-g;)3 znQDw}K_9ycQ?9u%Oq>LQ_ z^jpm;Rlest0RcPp7-AsU`YnU>k=C%)j{_bin*IBV z2B2Y`q=AQ1Nx8JJfg0!%MG`h5K@V?#burj42$aMSn3xzVo^v8fEvrJ(;RfcuIIG=h zULwB1IQasqr0yD9KB1`XSR>3xxjIMaAhqpy{_HI*pttp{Tf=Djua;8szM9U<{_mEq zwIB-Di;nEp*`zu5kK(2Dqfa<~BF-x6sdfD&Z}LzV9Sy5tKXIda$SG<{u@G|590r+L z)*j3>{T3KceMz|uImOqJF9FebUy=ZrS_2evaIStg6T+OEn z8Ak*T@6rgPS{|K9m_qCRV`9xtZ1NS?EB2Nj%w{$^>K-I5$9I_u&$BFe7_DwFtWY!d zmHe#)%4}&rWrLc}rxm4h&vwB!s}Hd0nWPQtUy43`T`rt0LJ8=*!oHvK`Shiy-!a_T zApj#!oAqJGbVa>uFaI)#Kne2uZfxe=Q}cZA6S>V<7J6Cj@$$-&E&+Lu=iSj}Y{6Uu zGR;PVQGUx_Q{c~HSTnKfL=;rWCQCSZk8gLq`|ke&Qn?0w=Z8q(B$2h0#_wqx7i7tY zyQZD-`y_xzgV7l0S2N%D*)6uVgAD{9plO(rE4_ZJD2Y7wj!_q#uWKLYqi@tV0gcz8 zLH;#E5!Nrr^?9?=r0};Rrz%~TudzDohTE>*f*;30JWovDl20It#@f))8|(D4Dj3$> zn8_CumWD3U!atV5txzu zc}!@%mqt(|t>WymI&R2Br#Y;%_x`@6(`i=<#F7tu7&Q=HsaB!>A6T&>38L2JQo~v6 zUQhE+%KHGsmpBQB_cx=o7Vp($T{VoB!iSB(zu>P+o9-RN4JR9@S@vqK2S!fZOdM2f zHCmYsWl^DX8C>wp7-85OGi3SEf30h&>Okq_J&S*RJ4OHG*QZvVPP;Z@P<_W1n-RBK zd{D>Gv4q|T?)$&1hB#2d*wuyCp!#vSl(wANlT(L?-KhlOrf=``hXk-V6ysoQ`ejyu z=D=BAfE*~OT%XotVv{`V3YUw{YjMXkLjh9^xNuq>^DjzYEgNKuo8BP`yrj_D&7yM;?Z2TD~bIrKQDZwW(c0a;3pBBZ6k`oD=Y1b4{r`C52@U!|QE3VSwU_Q;V&EZoS z;^Shu*MDc&|wDQlckT;B-=Unt?PVJWM)L(TY+kE2YHg%aJ!*?q8&)3 zghWqiB5Ltz=YfjV%1Qm{y@dIV!CBmyco-{Osh zLMLip@wcgB`<49bfyov9=u%GOlZN5&hlt#IdgOojL4Hxe;SEL*R5kO9GMq*I<~flf z`{47giatVr+hXN*^6fEWgL>>eeehQT`eD@tjox0|3iDbkwz3~7DIe3x1X)8xT)49# zOS$lzqM~HvR7PM!eDLjfsq}%T6LCkqDkf><^mibpiYk`mjff8mmCAIfydo&_M_}gr ztH)srGHX=QN9=Llzq=pU%YLT^4~E4!%_9pZWVe+i658ZX$2ua@V%95R+!U7KF@))~J5#F(pTUx$3H~WP7^OhZj{Iiq%qDQlbP1vuvw}mTUSgqX?V-|-qsmQoc z$EOOGUtVV+(0mn`D!O^^XZ*%rj(lEto48zxZ?F>*acMTrkq~BL9SXxC`afTB4Px}6 zu)emVjsxq`!82T@lpi?&OS_^fI~asVLr2}Kl!SxFZTFrOx2GqAM>VnqH(Uxw}4RZS6|9R9d`< z<3;bv&s^zlugKaB zVpnJQ*BHx~Z`AZSaDg^bXjvp;x&?k9?l1#MK}-j$p8!QbDk-@^2>ovtMHipS@+BL% z;V90D<-=}P-Y~|ITe>&zoJOSdnFBi_dj^z&hEP!+!-^yHI|hW1S-|V5>b#|G_ZhX8 z`B^bA%G^y=NG-@ybFw~O^%|J_!y%#B%{RX>+6}Qe8 zoBXg5qp0copcy19Bm1}YeEo7Xy4!u_7nK+SJ~n=@r4%Qn9mI0856Ail4JqPKzj znVWhF{VqzMjFHPm@-~oYp7KVK+jiKyya6bgwC!El)05tSGLB>{21hxs?;eZeWfOJ0 zBR(yhyi)#XM)EW9J-2;lseMh>n+?cZ;Qax433twGpE?hI9;Yt9J_}T0suv%|5^>ijHS&3=C=k6ThBP|2I0J2Sv7W zC6Xb~(ITlUsK~K45}F(tJ+zD*J*W~AwhNzuehxhwNx%gw$ zV|sJ`WaK?#w>y_6Mau&sAoUqU-go~SJ&+b8G%?-H4( z_4oWtlM5Q3XhwQBZ10l-nX4~@kn-bos}n$^gTrvw2&6bt>#4?(e91P$|1Y${^XJ7~ zwd8nP^tp|6&1#!@r9NMMX3{2HCZ-e9ieF zR`R4>r@aLphVP2=Ln(1ki8rET?oGSGkxk|RHOSM%K}Oojs|)iH#rmqp24z?i|Bz+ySqv| zdRDv94a-LsYz$wlW1GT)-LiW{c(c?-f5Fqa-BOe*chm>t3%RZQou1x3l!=B~4cC<| zM~~;3Y|ZtYxZ!PDtDOlnI0{~R3-%Xy7TD{!3Mjc56hzY$9xHP>&#DHOXdfhHllgcS zpa3~e|7-A8UfGg9z}5#thSvI}S%05ovl*47#(ij^+Mu-kEAX;;wN#XRg2`gcjc>iNtC!g?DP4r} z;B4^J6pv%8O$HpMpCJXSrhtS{Snf7S#uPgTooY?bogxDi|N+ zYE{3Xm76b|UZAxyG1U*E3R=Kpi9?^}aDL`?s(DW6l+tVT7YIJhEan6vZP=;@l+?3| zYBd@$5S-L6>@{8oPtFi-zw$d#p|8MvCC>7o0Oih}#>s~wI?HnJLhy}Q^9+_$o3!nOn{(StI8Mg#i` zV|-2KTQDZ1_E%O}2rN~3l^;bGIgVJudo0D`hjd#K} z%Zc1OM4mcn4XrwY=CdP5nz^02>%sP1TkUzxB^h}#v-y?NWE2e9AqjC&dw&ay%|Dab zQQUDleHa)Tl8d9DWcAmdoa;=B-7MIPya6c=Ae5u z+$$>!5`NN;m(et2;dOG<3C?b*;$GK)Tb+F=72cvv9}DxMakqYeWf6Uxc=sl=ki7o} zj)c)1alk4yy1ME^EFkFY;L@e~{`~UqlpuAzI{Pf?JAW1!*6*d^Zjx}e?cOmQyqC-O zu)!%@nD22{p2ym7peu37?wb+R?`v^w#aj1U)9RjS^kHgKyIc|du9MT4`mPlvFJn2M zj0wbK2J8k2ZxBu<{+64dp>OF31q^!xh=xoVV0oMRO*E@2RHW}wfsx|8I!l6_(Ct27 z>-)KvimFM}&#ONmnI1Q7j8XRneuC#^;TuVsI1;?*2}(3rIc*SmIuE9|RfL+VLK!D< zDk`dkhR!yxn}}eS1%t zexK8x+k47Aal3pT81HP9TAW&g;Lk!g?{j)6KHlE&rCQryw4K2d&s1!~330qs>10kp zBW-yz{0>rnaleeEgzO^whA@#lF9+gwd^b+rI?c+=Jm~q`u_^QyM&uA}t~s^?dwzij zWqpOEW#@s~g?>b$1OX0`{ztt}D-)i^x9P@c^>*io$@Ov_t2)9Xa!Rrp*;2XZ@+I2t zEd1%W!obI5_`Vx;EY2^dIWg9-$R<3foGMrw`G3`YWnIcutbu+EEX<>A+>kGNLsl*d zM)6CVLj%yi${M9ZAN7VCl|>Np;P3D!S*v3OtpXw@ufc#KKn|rGOGIo@I&6p8Q$2Sp zK6d{DS%3YUzD$)M{V884lmlz^eYzLM2O4(RxlH^@OiDVBvVhX^kr5FXfWJyHDJdmI zMRWVc|1E;MKt=HM)9N4P#ce7 z8@w0!Kj+6NXb$V*Im1#C;^5Z7u6Q2@7CYsV{70YPbtj;)pV1`^L;B+f|M)7!zcwh| z4h4?^fF%a5eaioVx07=r8P}++lz)9aXvy9?9R45CeFDjZKMt-K8Tz72HBwZ>mM;Wd zwQJ>-Kd+pK6%$aUdIkERR6I-atM(6V|LGmN^%@8Gr~Nm{u922D@rs{;5dJ{`UU&X~ zU*?%fe zu(GfD2ZygAXIZ25Lym$_epUW_danzqP2A9Xkym2ztB$FWRut?&7vppUNJ%JO@x#C0 z(rNtv9q{+i$E5iOG2EA(vj+_*PNsP<3vamocoXQR;cH(Ca%cCgw$LuDw=w=*WGczr zQW2D?q3Lu#+UVx7yso#MFByd8txnwSNSGZ`VXj1*eU%)?sPp~`?>`X1MhP@uG~=pV zYCn;THBe#S;sfhQKSmNS0oQ?y@%x>Mftnr@Ct_4n*gF_1ZW~$~pA2aSWT|U-8~5@x z9`)=#ZX6)f4rW+9wbnbKx6ky){1&>mxkvIvmzupVKfLf#nlg6)k6n4Vyz)ziI<5v& z8!0!i`X8qD1gP(rC8@*X>?Bu}N{^>pypv<^5AD2lT=2hDo!8xh9ZNOXf?h}PX$!(W zDk-Py$Spl5ew8qe59dM=kKJ$sk(qvJ0AC^!{O_uQG|Bb>`i?lgiQDc+hALi6xel}B z^Tpd>5)?>_%Pf;tu7YiFB5S-gsXeAjX+7R?K+!_y`ANi-{ zZE8UyE!so5U$kjyQBsIirYr_KL&Oa@K5!aAwzX+|I;pz8PT}jcy(o#|%tYTf|cgOB( zX0`(6sPD)PtEl@`b+n5cJ9sr;@IN)uZ_S$Bp}GHsrcT~7Fw1E?Z1sW8d@gYd%@R=I zZ-ayx%S7AM|6j==+V49|e$__if_~K%6jY?0{TMv3FPeAIO_3fF)rzRO{kyutMeNpU z|3fSUfJIJ#kHLpP^3Ft|^!6`@M2!lX$;jBap0? z6GE-UcwmM2bijQ}P2G9dTVPfYu-5@BY5i{M3-VfXjE(CaK#cEeS}a9&-a@--i+U7= z{3cnfIlP1~9_u%&#&{jo6Wr?mq^B6K=e@;yvI!i@7V^{xTJqX8Y&I-zAj^5w2hFAdx5>J9CMTrb>DDqY0wYId*1lGi$K9KAX;tnooZ zt&4(-mQ#bQP$6)(KX~34nM)cL4v3C=UN05xSJ7(=m>*(Ll$YE#M0dDeeN;Fe?O5Zqd8e=*kfG0o%VNFY$;J=OM6_x3FS1Q`)X zomAhu=&NVc^(z{FPKj1hFAYAZ8MchfTOZk-sGi%$dh1OgC%pW2yd9XGu8CRe7|>Q) z)Jh4|sT-|Fx zSy*nxz`>`zKB)r1zYjx`jj(1nJAp6&gW>|6|BU_q@Ky|3e)`$%k6TD$)h{}-Nb)2R<|P`cVuiIP&hz2F!#r%nO_Yw3fclgs>Y?jmqnb(L za(l#9d2dwJk4{Df9SQiLIEadV#D*?7KQM&1Z~69G%YBdA$af9MV2yFObWP|Duz_DX z)o;aqN;9*Nl00Np%HE#vT1vZ^ML)tB$<)_BfVVY$11;FH*WeHfgaXe6zjBG-~&(R3YKf;(8pCJoSTW1^#5sxE#^Xb94vYum3F8(o91mQ|%hwNNCEeGB-b z^K5(OalM8rl_Z7Q6AN(WUJNI3se}V9-o&EDGV{^@*k9MpwS`+_II-{wQdk2MSWYqABa~xTNeVN*GU^JtOUmL`rG%% z>v)Y<<#Rgazr5y`ppcU8a@e}WCx^N)WLhuQMasc82w4tC97e`Mdq?s3G*2`Cj_?kt z-I&)&GcVQhn-3}bEdKq`lct+1OSMMYGC6u;T|)3Q6Ot;^Ae052Ip3L)Ei{fn3tk}E ze!Tj7GXANM5>MBwnaD1dm-LsmSQ{7f!&T8$zcLS=E+7;Akh{D2RfahCv1s&-m1}#F z(e{Q3Mzx2FyK^JAa2<@SV1r&B_yy>0LpW zGpXU4a=(g{#YUOEZ0!6?SsfdA&&yNIWn8x>GT$Hg$bbN#v0cHir|U@p0Dx3Qiso=W zm&scfS(disN|1Q*yGg~k7P?q zy+mjx_~BR<#I6iAbK7DB14jtteYPE6*JY)RDj&V-d+hlg;Ow}b8ea`O4;!|o3;G30 z)dh-V1syI)1%4cj&p-Gq{__44G{{ZKKE~J1p}NVF$H?G8=yPqZYVIq`{TtP%19{HR z<9NAp$0PAWKl?5YM~}WR<9)Qg_OV)rHBbTYD+veE9A6eeQa0Ng>yk2}3Ki}yWNhqq&CkywvPt%6KA zp{fATZs%YnFUNTq0RtSGi0{gwdg??;>bk)1GXnqhV2eUh9pN`0PcL_;PV)gM`K#0I z2RR4;_0P-4hO8Z>Ig622mMeBo0#T&Ggd`*@k5|VlE%30gMxb99R`H8+$ImA=*Yk>H z2>gokoz|MH=#%7c)V)wH&6HXlp3l7TrKR>(Zu9DZe{}!eeM{tE@ zpU;|Ta1tXvpgmFO{!4CfL8IT1Mzvc2xm8=Ls=CVl9&_$MeMYJ7YBQqtJ8f}3qpGS3 z4iJe(+IN|aPCEOei5nJm031!Y*X$`Za%J%=3RFp_eNEZ@0SA?YWNBY{kjcl-GkcT* zRbx{o&(~_vAipoiTnKyE)q3Y}lQd}j0va;f!FvWhn>{JG|^(Wo)d zq^NhGd6P{kfT;v#7V(23?vcK=HUkAG5ig1yA0Um%>Q0x)C}nT_GW(h+EsJ{~ZD7@eX% zq+f!ps)0ESE7`SdoO{}VRl$K+lOFO36v>-XXLLB<=Iw7jZ`U|W+R4G$F7ft6z1u29 zoYabvDpz{{b+QD*$W>0D@WA#1m<}t8Tz->hoq=K4{Tehk!P+AmsUlCW-b>P)QMh8Q zY$&{#w*H;}bG*C@H<0yE_*!DI&Fgol_uzEOF#HX&bkx}_xuo>)P?#`%SzSbNHtnv!hbEj$6pK>Ptr59l;?B4k;|AxY25Ww~SG)CCMN zl?;v0VQpHNU;Jczr=k5m(B_?ZeIN7VW{Y^}4^=>#(kU^hrF=L+B)ZLe9gxeFTTGs> zvcy{RSR^@_TKQh56rd1)obQ>&{xe3UY`hq;A=|Zia<}YY1c|)PwpFVYz(!dxa$G{n#^Di*LW|lNINDH_krhf%BnKDb3zuoFY zK{9@tO{?f@o}vK<3dg!&WCpVKBSB2JC^i0;U#N91ml3whkA1f8d7|slPJXD<)IXIk zkoSsWyst0C-vIREFdkyQLAtflyoO{muSrnInfVnF)URjd(Jg~NbuYDzhD8QPoW>5e zVm&qOl?kZMk5)db<*_WILU3HRPm^HDbsEj@Z!14nShh#z?9M_nY*Sx)a<7qa5Y+N{ zPg+(EF~2x3rt`l2)om*v4V#vivORVkcA1{H9u2D07K^7pvJ)o$+j-T*%Y{H{7+rZv z!u8|f15Fjq55JeCQPnm(VW#6CFYUDXY7#PSiV=|S7W=~GO=H172~Brtq?rnPw0`04 z5?}q%sJNcqwwh*1OSWOWoq(O(vGe!(Xt)C6i6>@(mkWc)faQ$lrY&#*Pe7a>FN^305_Y;4RSi{s+O_gCdMNSl$P&Vi*s4j914t#-Mp84hlhZ&;zbwp5x0 zB_`%p-@e+|73k=y$kOel5bk~3Bh=Nk;%zq2Ipcc%RH%KEV;8kegIDGIGqWD~xTm86 z7oS<5z6Lror090P=%nr*?58&zY}?U|4B?#hfNs`Gs{w#7a%KC_pwGULr1^05AQ6G( z`Lvo7J-}DWibpMLL}nE4EdqyHZOXmW5=xk%nwtJ=Q)4U;?LICc(n>=!u)E$aN@8hy z>w_wH0`XaVJY~)ER*{qyu7iexlJcvdRgiP5C=6eu73qi;zN8YSE_hyjpRWWKCbnEz zlOs@Yan1DgZLcu$@bEy7gqft7gIS-&4Tzzu@jL=KA2F@-NQxECyFKrd_oG9y77tVv zny&^7ilDizaa3>Ubk-+N^;rsrs7oK0-y*3WZ2xvPGcb7HskRZjE;^gmef(gl9$zAZ z%MxtLudrJ0nq%%?$tR^(T9(U*gj9xr77Z5q97^rC&bEYOOPyNN#Q;Fy_8b|3+&-gL zzWnT6#EV0oZI^fb?3fg<@z*k4C(W9^^+-!>*4_cVcTF1orS{L_wz)GCWX*5IS@uVg zz2^rU9Ndt!G#x-0TSp9qOFh@BMK4lXnw+z@FF9<`x(M~JR)ZE6=&fBMA~~Bb!LsVQ znlJzS_&J|xZMJU`TZxG2>EW+BC(WbK%}~X$4@Do#$=z3NFU|Nn06=k?3-AW;E26|q zG~fhu&Q_n3Rybsm_;XF#ygV*##NlwV+xvNYH$73@wo31_Bf=t4m#`?MKo!TAm#Ho1 z*{U|Z%_AHhl6}1!wJ4OM>o8|%_Is~Vgo8@NH+Rzs54F$lbZr{n<<9a>SobBw7aNW? ztY?Wjitn};bUcH9d`pANJ9`XfX!Tp42rT&CP?Z=S+~RL{Bx-?$146gyr~EPGd0(N> z;!hnM!xo?r!Osr-9@e?sCC4jp>=86E@|%J(Ut+fBOKQ&4_w(CA;0^Q3gRL{@A>=$8Znvjr7c9iAz8%iT$|_FTR19sdHJ~dI{BYBc0^mTd=9i-9jU1q zDRHJU(4VN89IGesXo_0PSyjU$wV9MMZ8dE)-;B6E*_d&2YGB+yG_9>NaUK9;l%?xMuzK?eK(elFMQkR^KVLF7N0pnR|&-6L`-9C7tF zya>`}XI9j#C>x_JZO!mTn%S>i4_jX55Yy1V>aH;s>zy5qbKIQ`w;^{R^bVco$5-l| z2UT428odR?Shc^T|Gn$KOs`H=_oA#>QwZx%%+1Edo6bT0NNtaX8YS~RyV==LiILlU zukf@}UgTPfo0=!p64DVpSoQ?1+OLgIs(rm|eo%g>Sxv|A95~rq%pi>KF@%Zq+4tvX zRP&NQYUJ9VA!N^-66K#%D~&1kIkcd^;&pJ)G2Y!Y5YH<8jkrb1&4BY%zd;NaKM|l+ zZ=}hh(Be~`x3YU;Tb8vPB2v!}dW*w+AE63$fE<#8TA3U%(Cb30{Jg~&(roZdwii)6 z$<;{8s5G6@hy>(Ve&AjXh7sscMd*j->3smmN_6NldNoDA7^VK)jjD0KBmg$1DaZKG& zUFM-*H9aOlN&vUoE#0Eb1t8Dws%*d8bCa%3SwQR9~w7yq|NeXt7y1$xWX8z8-C)aka(ztcsGqxm{Qo{vh;@cknP z(=TgmqO>GD8fkZNv((;k#S02QvVU9k+%A9QB{4eh)d#bx9f#9;42#z~!=|7a%Re`< z?efwVxkn0+6A`91wabHB!0pBF$5R-SH8h7~+g=*LvVURUfybnK9Xlh>#BFE>IdSa@ zvQJ9DZMa(u&hL4TCfAuHP>R)U8Q(niI57b?pzZ9Q8T73Px?ZQ?AT?3+n70QToX z5YB5Z=z~b5a|fM+%#7_|@C{n4Urho zwzeEpgmWO}`?EXmL*yq5vR&{Bq?XsOZXB*Nd_kFS?caOKj~2T?o4GnPh3gQOng6n9 z&){N*4hZmkIk_Jp2A#~CS!umr;f3989oFP9XfI1(XXmw2ChRV8OCeXpfN8pXzcI6= z82V#WBkpw=``%Zmm+L zA5}fL)g<@UY1J!8P=L$sx9E&*GUL4s&!)Z`G7`3wK9W?r(?#(eiJ4kYj$q zD5=OYv^sWa4lnE`YsE)d-*PvaWA?JKAZ!}$;w(ny>e`s85FcI#41HYt=UY6 z{%}|KGX=w?w+RoA)McJOOeIgxA4R56Do?q+-_xhxL*WBH3{3~u?Xa%a`>q<0&v#ib z&q+G@Ll=ok-rq|-)B~LUWbZ6)ua%?Q>3Zi#&;gB>>|%6bQP0=ghh$0QR}SN?cM3HJ z_F7BWL`42cyDGzWMLC{e6k>y3&LzNQ!S)pJ%PfpkVT%Qsv;7 zsgCR6A6;M0*@+&KX*RI0vy5 zErvnZ`_9_yla=^GE8^0icYwrncgjzzV&X-Iy`$EaLs8v|W&prK(wJs4MLtU`Q#pn{ z8~|g8hs4I7xY_wO<4?DocoqX|iF%C-Ta0_6>fqAk=2Ypf`&2=6J;$UFRxDUX8&f~U znb$wJv}&MaY>c6JYi5g(dL?Y`IEoidf~-Z%J#8pisM%LIen((R?~pY(gs#%Dgrm%A zh#TF$U)U`5yIG*523OtjJrntOQgOD3UxMXY8KVeKP*h~%`thwGM?)u^!Y?zsQcMz` z;JFjb<-_+~+173V{n(3p7Y6*H>6d1__~PTmeg#lAHI_<3oj9_Li!JK<8iW^HqhdkB zOhH_b5|+Kb?t3G}>u{fnIcjFFZwk!!mbEje*XgMK1Ul)kMWo(A1e?%sEgyb)4uOl8 zF4yKovS`bW*mucn%Re|YH;gHxr1Fv5G>;RZpkRpMKIFHM9_MG)cxvE#dS?6_ycauV z!&I~xjUvA4^Bg&IXY2QOAeHAcFV~wt^S@utcKjSon*_pwmcs5Uhr=S<)8E%HZO6Fm z#<-2cm@oNt*p6QO#`Sv=@(0>17-iT87F2wxzp+nN+8fL2u$`^w2oPe-9`w>;A-19N z(bQuEmTEY9j-Rh+o8^?)Zn3Z5R3i7eslx!oKKG9(OKNng$0L3$xw8*jEoZmhYd8ts zzF5?mGcfXFMn<2rN%93*7K+TW9&c0LbN?MMJaDcA@QMY4ev*Xmy?=n*X9|_@LT3&FHNgp$TyQz4k zB&{0);PahAFP<^TlhmI%E;o|Z>R)iK;l+E4A|}#bYzzm-u1)-vB9mrMX(sur=1_Y6M9*7I zu{wcEvF%Bai2XzDWVZ6S(VL1AYZdtQ+wWFIByd;Pb7sGqz*K3;Qk!~~1PrAITaZew z{kg#vtveX`&#~pUG>|k#rT|cU@^tvcynX9;f; zH7deCvgAr5+sF+ADNV|g)xdzyy(1|}Wv=@4{am+r`kRkF0|eH+6BG%=AU7p`$EVBZ z78Em|@ejJydvc6!j99`1^76X3Z`L+9P2u6;59tHi5ko^m`}+Er?Oab*g`v5LvG(+g zjPkdCBcIX@kq0kJP#MQdN5Vxg&Ch@$Tc7Zc5o_+-zfV5Zj!q@M?C+jmUMAE@cCye3 zyq#qQ$V@!HndzVSINpXQhanNO)#R#^siQ@f*j|}E&AMC}eS@-z`4Yua5&F>Cx%07q zfL8GSd&tY&Zmz-8{sx<&@5;u-#+B}~bzTk7X8CN&sEZF+G+#paoF#QJt|21yD4ntG zwvqO@)ZgC&O(boY?}dCWvCWMf4ZEKbKOn**A_5y*O_-*|rlk!Thqmwc?p&avqEaUZ zX+h^^q>Mlyxs2XQ?aBFrw?=vv=S@}G*PrkhGbuWBC4oHNKLK6+Irh#&j0^aW7%V*c|U-Z8}u8R2_pgH2XO?8`n0^!a4- z8tE+&MbCUY3+f}F?As11;P0d|caNj*MY7Q%*$!fh`!en83eRA~dgeNnr92VvgjnUz zTGW7;OL&xEGM3_}Q-~!nWy`2U!!s@5k_*wY7d`-(uQ`Iy(XUl|lkQ~GtzPaodxKNT zYkbx9?rFTIZL?*e`kB-Jhp&_9b*awol81rnlNE1b@@2)B(LT9->|OOv(f${Z%7=Wr zN%oTH$S&Fp(ng**AYc58tC%gaZ9)UIZb+T-L3^YlJAv2>4Os`&kPk7xJh1nxOj&b_ zvJ5A|OLJ|V*w2_w)4#{kyXiMu`2K4e%ut^{RdU&yq9VA5H51I2PCZxvo7SgY$c6rC zp>@w^X#{dARBGx%Eug{d+?=u2hkyw`wa=fU;^L<8t8h;AKSB57>g%sS>u$BUolJ2> zw*ABqd*O!pM90W@u)l8%jg5zFu#W8{&55Ba)u_g|Fg;VdYaeG z*HHSu-h(Ms{~7@X{P#Zc*PrEo4}&iH{@pBM{PzX&fW7~CBJM&WLiM3$hBAcE$*_P3 z-AznP{LeXN@H{*{jcH$RKpX1OhhK$>@Bg3vm+1cc+^_vk_`OMNrJPuYt|I@w^7y1u8bq#gRB&~Bty4SP5o~Ip+ z;br@H6yDAZ>1E^f5F?R0J*)e|)CLq{YUq`!O8T%|x*j$Jqk3 zY{722NdXWZK);u2dmxYNh9V}@I&^!L>(fd+dc~#gu@BPB3}O<o=;{ojx-e0eD^Q_Pp9q?3m{&td*(ZWa8cgaeN3Ot#D@iptRr+jc< zO5ZIhFS8;PbS0}@7qg0Nilk0>j5Xr21b(>KUvgYIMCF=wsSrTb$GB@PYbWcG1~UOZ ztz~_5sb5X!D8U=Tfi$W=j~YrpEbW$-TkYz$pnkzMe5aiFdjUL9cViz7!^Z@b>3vv) zaBmK$X_;m(9c`!Lpghm%|^~yPxdC8-;lJ{wyvDgueoy2 zs|fN$op~9!m_gXyq=;;DhLrPL z#T&HQBXv$t$VrA|UgY`t+y#tC_R@U$G3jcAG1%8N0hVArKV^-_{vlRNyq%Z!4S+zZ zjaG?7E8A!y5b%fXklRux1$r{y$B*SAE6KqxJVlVOs8Jp~5d-on)7vICLM#z5X|>>} zvDD=tG(Ggq*SZeNoSkTS-X6HBeHi0>7kfN_{z4juo;J`u%IY3&vQPM*8zl9!@)6j z&2xg42zvOk^tKiHI}u0PsrePVzMdkRPyrVoPwi29xUDXUSLoIhx_a_t;@zo7`Z*R1zrE1HqN2FCxS|2g zc|*_NueNp!yh;8LLoEf^yGK9HeUTtmMM8V5&97M9u1-S5LLt}ZDlQ^|+rH|QVgQZP zJflPP-zajHVi2zX^mGcWqrKDL+|2%^pqq+De1Y>5CQG#l1Usb8Kb{ox_Ovfa7?jlH zZj5CBJ>oOq#t?>wlMHET?~h9tG2>Nd5+mvS3v;~#wPRD=zb{1yxu_FfgUhR9JnyE4 z6Kn9s?^9xOvtrggo5hV$(a;g)y+`Ij0k3oQGVk{isI;owWzx!h8MJ7C*R{<)rQbGt zBBT3f&AJ;$&60sc9Rt zcl^@FpZOg>Z?9cxUI9EA7$VXI>p8issvBRxqjDCecwu9s&L|^~Y>L4z_t7yiBL*XdVaTU%4_l~D<={N|zw;J2M$JPQrQL(%m7rd*$dl*mPi;+= zS&n9%^vZjJcJ-YBxW+j&nsNx5IIn$^C{YX!Gp;O>D@&62O85lLfdU4NC2=gLTb(!+ zN_3`TRIg?{+M5o4KUX1PG=hvh-M?hxO2*^YJK-7W{Idk+0(~2+FR$;?@>kxcG`py*f97&qg}Y;a-yE?YBRhj|bZ{D+db_cro_1S2It z`<7iin{j~S=wP{?u1WQE2bbd>Zl#BlFossnUnTG9kKa+|)yo%wW}BOK0(unEGBPv{ zQK_j1D=VLxUS&Cm8XIR5n;1LP zLBR!i{FSyRKd!V3qQ^I?ZF)R!PD(;S^aM*8Mzjb1rfdvo4RavR`cN5 zU&#^6+T|qc@^=;#`2Q^;FhJAOyS{DD2`gqbsLss)`1StrLY0n8v&yok%=k72qaeB1 zda=ZyVY{fmS%d!hUB$}{iz1>w!3bBPJ{~FI@wD4?l4x-sC$*PEW~0@k+=%eVVl|(2 zNTXkcus({ghqz>Fe@WXA8ssW0OVX#RQq{-u2VcciyzHLNKI-0-xv<@keH#tR!uyeW z0yI66N1eg~`1G-`u;}osGoV!pY+YBET zx2JEDGGKRo1!Z6R+Zut;C-BVr%4t$EJ$(7AfQI2a>Yst)dV>QDanRICp~-Y5DDC!C z+5`}U^3L0RE-th>ZDZb*xNuwnm$%~#W)r9)Y>TDrHB;{}Yj<$De?EPA7yMj>nAnSU z#POhg3EE=1|FpOHkuX@o&NXHaz}Jcz@k^NcPgn{77;4RGzQt_zyw%=u=z)*I6A-A{ ziWs8zkPEuNC>hXr#+ojG&u~3U#3rOwMh!(J4QtgD-zh6jiL!vABR+5V@d**2_~Ve@ z+7+J=07&F$FKdDhZZZxBkRmJFLz!<>qd-L>vzgn%@a3drtq7asU^DcMdcEuXa%z>F zNi?IUwjm(sao^OUEPK&C;#UJ4@=so8X4;ty0h{C-mUr}+vlW<%dge5bkiigrK!*-a9 zz-4dAIXfgK=faZ(^#MZmpat+V zHI$0ABYHjAGuTeZ*VCymA}&8WjAW*h(RSulWkjGSd849gAK-QGSAC)E9Fz#=g3@i0 zx7&`&W_qsA`R_JN`5)ueL5-Q*n}fe1FA+(2y~@9B_d?c)7gH#}Q&SR1l_+pSP{Qr6 z!J-VuxBXf6K@869XS%*kM>57XA1<2{g{*L7q3Vh_&uiHxz7sqX!OHFTD6mOB|I*7n zt(#sruz@C|tquvXU|ck9>{wd65!;A%grZE;@cL zzN2FT4>_w_(|x$EbV-3sU>&K};z^v}=`WQo`PLwhwP8JaGNG>$?}sLWM*^%x!ok#V zg7m)tZT8di)}7LDqv{T(xT0)r7X(CsiVA<07WmYX$v3rqJ#|6JRhcKghptnhi7rlo z%PQNl+>u3n;led$ayxJ57&R;w*gmr~KDe~43^h)-uj9#%8#jA7ns%gfI9csW%yaWa zDSxO(KoU$g=RI+K*pWvh3k;#j#h zgwSyznF-NNo8U3k$&wpQG@!K_53IPk-J=G4k`1O$$#*q;8cAY`nnxEDB-c(1hQ`!2~Y$Wa@3B*gN z132GoHBDk<^gZvT7OopyHMw>q%#v)^$si?AW53?{qzTREW56(qQpFwc7+kgDHq9Zl$}G+D!fP#0fI&gIhRFMBpOv>$v+t=26O&S~vB!k-56R!;aSC<-YG z<lsYr5YztQM>%P+nT8Cs(`hAm&v)Xay zpli3eModFGPow~{DPzQVm+DlXBH3KcM7;emzuM%KC`bg_U15`*`U_~jPgwpA8f0aa z@}xX*7X)ZDj7%Jvx^;Z3M$V2%!YU)J#1q=IF&dMrgpV`fa5B!Hd z4~SguA49Py>!bOhekPZLHMNO##!V&QRog=UgvN5fGpC~1V1<6OgN3;&!24@kVCNBM zDB&30c6|)er?H3p&+zWd5*BeoFy?mx<$tOTgSQtt9;LZc{3xumO70dgQ8S(Z ziq_|QAZ-s^M&s5+K!29ri&%ay-rKC71A!>NY8L9N&E`Fvl$oz`|%hs|0i)E3cLNQnQ2f*_|69b zyZY<9yumr{TL1tprj_G?r|*-k;#Z?96ct*6$oEKA`H5k^T((mgt1PmBeuamki&QAS z!Tp+TUnGIoy{5Hh)k)vD*Mdr^dCx~yD^^T^k;shU~gW`ZetA|Os}9~!qY8S}4V5*DG@x4B~mAF3ZHrbzCe`tKQJ5v77~g_fxE zzwZnB>Jk*J+2<*d<(nQZB?b^hONTebN!9XDmtY$oAv4o@?M7w?OXTeBJY_?Ww{1&m zzSYqUhI2uBeQ&38dTXPqGZPg?vE z7*F2+qh8J+ev9rO>{(e=VIYTHAN#GJSCGo|2*Qnu`S3CqvWD_Xd0|S8d|ZV+0Iqx0 z1%A*%LibRKJx5{KR$sLFRsIMVxGI3=dHb-$krG33O7CuyC;sq<@cFGEbshJY;CjE= z4gb3ZL=5-%1YI{qZ-2!@w$X$%@6|h#CzENPX?4NgE+_!evC%P+W8x&+F)w^`ubsl8 zVn;N73^@Ls&q)9|knSy-EX9E72Gn}U_|rHv%Z;zjAsp`vM|@(12;YCjI34s$z+0YJWQC=u2V4 zYn@_kpSEwuSOJm9!|vQdq)XIy6Y5(u4JBBja=lINuIA}Jj>l?cZU=o$ReIcR>u+&k z=syg=x1O#}pe5ippQ)pxEm^>y`a>O+`=!+HhBkkvDg>u`<(dtc6@Ns<9hsv1$$r!V zeto%nN+Bkrp>u}*pz7p=B5qxBstsfh`a`N15f8Ltb~q_6RhQzG!^Oklw$IN{4`{8YCLk5C*Vu=QzaD@)Ze&A`%>(>oDow6wOFfIK%J04`T|W?R75Fi zMMLfMi7Dis_3?fpy6fYz4>isfK6iMM;^?Gm_Bs@m{;lm>LpXWkXB-e}{Go%Q0Ggi~ zUq1ET5%x${tKz19jSYe!`$8&1r&$S&3f^RI3)d)6hq-(?AlmsHzC?>HOg$nuCBG;9yc_5&4579r+C~Ysvz}xm0~KRc-9XA5Jmb$ z!Yv?~V|qqLYQ*xC=X)6Xd8>EfAF)^<@5jkOA$Ao@(k8ML3+G$A5ybeAE9@fP z-6T35Y~;(r@1M55&~)eex&-!qMkckau=@$E!9Y`W+_eh@Q^zNRi1K~8v}7UYLx1bt z2z78d3*ThNM1g_y-RG9uNQ~n!X1k?b6Xk`aj%WIU{i+`JkLjU-e5jIk5#hAwIlW?- zL_%*uRM#9O8R+(xth3aw5Ep%WaH# z3}Pqe%l_8)k%FO=`Td~ zyxrOT6EMfQbeACehI#GS6V}RFfnv&YgmZEUPHYYCpI2e{7p7)CJ#?rk7k6~~uf9;D zd?_z0%hR!>8t_^H z23KDHCiXKWMO7v*4l<3voZRyZ3z~)wEYYFfy5rT3r8=I=%H;+c{fp3Tsg?(dbkuyk z+u+^v(BlPM7+MJ%RN{cV%3kq(djo(tDK?bfT{mIB(g>VD@+lzW&YTbvc0bbA6jYjA zlkIZ;#%=%PDB-E2Ets&w`ry~7zkfa2VU(b?rNt|8 zqvZ>_r;%%x1Fi&*F^}zR(A#s+yTK8uV7WN+R#P{Wj178BS=gyC5*b0QyWvP|l*kd{ znVMF`Tnp7l@V_vi6aCAEyx+;TTdgj;#bmdiKXIFTasSV}WE3nONlDIUW0$FR+LRH% zlzDg?KnoA$fgoB5Fpy&vACp?Bc2tw(Q76zy@7MgdfQX$h?Le79qbYr*ls<)ma zqgt&?WMK!#*#)neq(JR^m9{}VI<6%eVf(XvA^J7ucUP0|WhkMIUml-bwnJ3pmY6tA ziO#X^cp_>R*1^WPda3a~O@#sP3Zv?(OHv zEe+tKPd zy!RGqc}6D4&v?u~jEa2f$&>QB<`!-547&NIj74u~xlDn=b4epzC|3}4T7Ch&e9?%f zzh1|ZOleMDmN8@C}Pen^?}9*R+%S^Q*}PiYgO`g z+*I>Jb#_y^0C3Ut^sEhoNzkCz;!W`$pZlS4!T4Hk;N**IsKA1N&qcdkC}u43CYS62 zx^qdUNIF2->;?+#_E3=<$jzp_UeLL)%S9z=j6$`{{5}CQj?Bh+?sXBAS zx$Tkk2s52z)g(l^kb;2Hx^GW{d$`pfyD>+&P@)Q|Re_l2=Ji`Dxnmw2V)0!hRT!kk z?Vj}pPwNeYjo3U+hnb-fvDgeB4?lVoWAQgjuA?kmnT>?aR(q~ZJ8OsMl-unscSS=3 z3rYj5k0C4><7tARg<>rbikG@j%vHJWyV!Au^Z`!y2m{K@#i5UFXDl zTFcF1$O4&ElW5{9fnChH+DhwAxVZD3+S1uIHL7QLeSLix7#Ix(FK=n#Mo&b<#PtpE z%_>*Ay1K2Wb@6G0+waciI#V<(wAfnbJ}MG6i3;GC*fzA(3MX76eoQDRC~#evUQRTc zy2ucKA;{>cqO;X+-h1*ZT=02L zT<_VsJ|TY|9l;Da4YA41DgF?j-$_14)OV#zW_rrJ#o#*UUu~iPrzGQeb|mz2{b3+~ ztG1p>!NL(Z1<8Lc9vu7Nd)CQFk>qVYwC36Kb@t)$mu{eRo744t3=)t>IMxXS0VWmIGEy>>!*bDx)w$>WmVTp%u|&P22SO^4Q z*uI~xUq*%Xgv-{l+AF39yWq8X{-}a+HElLh+}3byHmW-^gP~=I;YxIsvWM>wIb4rA zSz_Ra%wskySo~>*&0-z+um9`N(PfByUvig#=bu&a7M!d9ySKjs*&Sd->_9UAtAqbf z-E}*hluLQZBjNCXRkG!}85-p9vnN=fBFLNML)eq0tc=-lCV17!z-UIA~` zec-?RftJWFt5p{O`b6(?V3gEp%)laFsk+KH^t~cVm-G)lzW%aI!Jj1r!tHU}9Wc+W zJWN-y*@3X$GvvXexa}OvOOg%_FEZKwX|+GxQu++Ot8tQm1Gjd#B!j#8fW`+5t|l5W z)Au=>_OtHKF1)xF3@mPZI~Hlf3-!+VWIgjJNJNb*#t3oM_CB4sj!`WUxqki@j1TB>Jn4X)%4uzL_=yL|CK7wI#;=S+j*h4S5XnC zcjE#RbH9Snd5kip|L}1WegOzzrAn~B>G7HT(mo9jFI-D%C|1Sg-VIioD~pVI-pik} zgcBKOav{7fu6-vebOi^3jlBsP1_5vPC$Dm*wKI|q-$3@RH>o3(mAjxODR9}yA3_g& zx{lYsfJKCbKR%KUvXn>rh zhcCbT2`#8?`}los)q(4%VCB#svuufH!lIc1<+V?X#n7rxNs;h#s~Glq-=2B+9v-{F zYWR?p#48ex)`WuvY#Z&;>aC-GCusS&o`&9%;4oMS_vRF)G6DO3OJy4lyxVHkeG2## zsq&hvln8&-$*U*_GJ=|h2;X^B?na(bn2EBtTO}Bm4(XJu%R_Pp7O?)#}k*pqv|_X66Y_rAU&s*lZkIGw)-;)|AAL# zZkh6w2D-^ZBJBdXMNIsXa;5NFxAse7Tu<}ne36cs9C&=B8`5N@C&Ry++@)*i&G8I= zD)fHo!csm8+)M34WapYY4X!IiA*BUw?%6hh*E9c_agP2kj5SzolQXRTUnupUYfBo= zTrK8X@hFubiB7GEo13ffWYvdiPA)g4;7eL`9D~}87VV@nJ{`$_$9(qYDt@4of~y{#K3ZmPx;1H8A@{%SX5i4yFB5_ARtHQ~x&kl${DUY0Wf0PP*5 zB_pj)$-w(-Om21nAg4?kt;eF~L|yFHKaryQO|MRVRfE-l}x3&C!wY zDu9>|4#(n_o$#4;@70#6O~2K0#vkgO0*N{tyPnuXV~Djj>f>R6n&=j zorh>QWc}?Y{E}?MJnORV~cVDoy-dxZ4^)|bm$Vt{J$Vd3Vc-*D>N8xc;UW=u| zCK-VlDAVe3=LulcjF6D#Cl}im*MN{|m@GQMmxC~p7qSq)kDwheEKE*$KKncWr0mZH^8l0lCZX3V;55IK2b2Bo%$(6K|y9i2A5)TyrZa_T?7fPI`eW84DPK^Kl z2kz_#yRu#jm~vNO7tZ6DGi%u07RuUGzjgR<>~W9s-2>0iz+4p(KG~^tE7HmH&Y6X z+uSP1JjJW0^y|wxH%b0`x^rI17{9!AzGW!2e$sN}o%Jc&*iA^!=hV`bP0h(DD5ftg zh@*!?!hz}Zu$|VcSQk#`^yj^g8*_E0zbyNhQsl@>t(Iso>sSpQ+1kcy21b<+4lq)A zW=E5+-3ugAOMOB-q*Q~mRhw_UX68Smn-5U;NKV=jV)sYMiq^StW`?nkGU%kf0KVM0ChTh+`tZ>oH+rtg zn(q*_Aa;$@Xvph&bvU1$DpGucP+JjI880gx^SXelOQP5x;y=U{bwVq;!G z-G$iW;H$sjfiMieiL$PIsYl{_=S)}t`;tRPJj>SpIf@+DtL5pZKQ<1MkuMYK%)2_X zuEM{q70{J4i(H({TXE{38ZMohZhP|&PAJ!fBloY3hm%demIP&2rgCK6wl9wxH49~0 zbF#E}nMpuh5{y(?6vj42g|ldhP3sBerHlOaDgn{N*G8llU091kDj#*uOyWoG7>ibO z#)k*fJU9(Ze#T~HA4M7w1hQo8#`?kZvs~{XYj5C^n@PMCxC%#cKp$r^oHv&FV#{^4 ztG6W1`o2egXSL+|!K~kNXFAP=^@Bd+@sh;Yzoj=6p8^cOf=b552K??uJ|lfBBc0_i zGmUKe?4veq<9_^=&15TLR@ihY^V1inC*17x_#OuhEHfq@uB^O(Q>cRI(9qHs{YImC zlHv^BsQc(8(CN;p@C`W$M}@*;aK~M$>#VBPmW+DBiBvka2J*Kz7NykzXu(D_+}g@_ zIz!gG{K>wqGXGPqrM6CI{S8@GRH7Ee`4F^Nb&~PZ;9+~=)HM=B$B~yD2HXx$2kDx; zsA*O@y_hQ?%DXOz_)~#9&k#**DJ>ZpO|~noA{uSW?UIYiLKaap z{(^90!V@e(ghnvQ=ChTzt}$?XuCnH?=XKf8x3&o))%0$S98pxAayyvsk`H|Ly0dMn z@ZHc8&4rmMp~7TjJ(^i*&01HHux zsj?vi1dGbe`j=6y(y-m@(2U zowvzKy{j=D-x?6%1{=3pTUe-JHEo5m5LeYo9*mpu>UC`UZ{pBI+nJwp{ z50M@KAiy|S4tSH;o>?gOB7Nuwn}*k15=m1$eIG;1ux6y9y@Nh@H(rFbObhHHx>To* zW`p7f@VN4YiK_;#XmzA-QecM5HWH^8t9x7@%oG<^Mcem~K)~kRHYs&&^=1=^Kc^pg zYHl%)C0Ld@Y>i?Ty=r3hGMEL;#%#yF&SQKNGas_CfFp8N+r*nq@@0hO!_@*j(OwKU zySz2kG$(7PiXT!<(R#!I7cgOml2RLq`ch!q)m4f_`K&-oC2c+O{m^j_M&fb3nGY8f zyS^>#v|5S9;8?2Zk2~*alvtcw#C*6okb&RXugA-6+~IN0I0G@i*5oE|jbEgv())Iz?`+a&k% zn7R7?O)u3{rJ$|?hDs68y9~5n)v##M3Ba_sy3Fh>rQ9FnL1<@Y6e4bOj5lh#vzGJi`5-f{rZzO7u6kp@VI)vH%C?&PwV)h#W%F>09dgmYQju$ON3zPxK`X((OV_=)xRU3|n( z0IYA!H2|^ind{k^eFtcA_wf&9n4GsusY+V;4eI7}%F+-V+=|Y|4hoMSh`6()k8asJ zz8?qG5Lc<)OJ`If-1T>-%~m;=>0jg5(vJk%F9z`d75G>1ABM9vNYFe+^lB%H1F@vn zM%EWDulZ`tkE2vdlJ%GIZoEAAp@dE9m(D!yW2t^;!EyW!t?aIMqy6eZ0k#evan24` z0d-69oiGoS41qRF5HOLSGO0_&t% zvfJ}(C0NL`beIEd54Yog7@0oN^V3}x3yU-8Tq-^vNHD%|#wE7VLZ;BzfgP>nF$<+# z?q)`+3pck3-Naqd)z#5&iPDQ{>mGb3#!M*=9r28O=i2rNX6?(B53Xa0sQXxDKVSLn z!XpJF=;+X}egE7t-%m_9!|TUt;g~}kczfXTQhfSe5*ErwoRJ9+#cMA90%1QPyoGrU{w@#&$#{u26If$jqrm{??=8i%2TM;}3&r{$w zb;CuyUc$>ILRKT)k5C?5(XXFdrO5m%@GI#fQWJ$0Xmz+rxgM+^c>ksau+uCNP~g}Z zVNSa|RLfxF=TRuKVhJFa9b+tAUf>X(Bj>(~%l6jJn)=FvxJiq7--ubj5?aLR4XeBQ z@<^`_GQS)wOz~THFY_u$PQnr-hFg--8Sl=A3if!HgeBk!R*b6aaW5Or@ixfygkPm( z@`IZ{v24d&VVS4}$M<YcsF zs7{qa);P09sw|PxY<2eIxz3}s&kyxMN1Njci#Ud~CPB~FI)`Jm7KKs==}XU9=s29c zz*jR9(s4h?xXw17iXA+(fKVXQb}e!6tzUz(X}*V83N*K!yV-~xk|k9#<10^_boYu~ zUhG^Hw>~TjwW`^0VF1@dEfdJd%O9Ilp#E*x(fMYbowR4COQ85;UOU$z_O3^c94jHS zDG#qw63EZySWsnZhxw3Y-oo3{rLt4$Z}Sf4wm#|bEES7W%?JpS8s@a+OB?6?>1?k9 zT&OAzJq5?L(c$m^)U&DYNhq4gOlA=h(nmIWH9jk$eSHr`IG(bxWkLJ)O$u?YPd?@` z{#%rsm(->XsKKZb-eYeR+#hzf8g)23@Bm(?&~(xfn9?B~3in9kU0%lbk2O5=cZ{N@ z2&oU~!6L_hwB!{<7BlwA7V+>_FI+DbfN;6UN~;9~R@O=d{nCjuscG-ptvZcpW+Pw} zeiZi_pbrgJ<>06tedp~B^41cY9Vs4MCf3v^KP;}?!n$0VLVOna5|X6^}O?nf9Q<}?0d#{g>g8f zaeDrkaB<=QcrzQ(p!3k!C%|SY(~A>_v#lnZA;}*3|;+dx}mk zkq+^kD5T3 z^lchHeRQ`xm5jN0espwlbUrdPo272^jC;C@;=$f$nY(w>uGB2%=S#@oaq9NCc3rvR zBnQbAP}Q(xGSDG9kZC|vfPMbCtCxsLXtSE2`&Zl2yF2{Na;_9kJIjq5FRgfLiarGg z?d4qaoLlt)58HX?90Si~x=*pWdMWNp97W4-a+|4LBLef&tEr~B65sBw7i)Fdf|M+% z+SUOllqPHhpD1_E`+M4Gx(oK(-X(ep7HNh?#&)K`Cqz`*Umk&nygp!{s4gwl87U-V zRhd{)3GWVi7#u733EUCPsbZ;lJz~UA((!OL`kwciZ9Y^I&!i9fLK*LCFG{X6Or#T*q`?Eo0}J zHk7fknRDFVV)n0O5k5Pzy}@8pzCd-Vmaekma5gksTGm>R7vJ_RsbP+xoL;IuetTB? zB|TC}ZHLE3_rgXZya)fdr6ok*cQ#{(XEU6`F08BM-11Y$lk~LHB62 z9zDIn_u?Y=kl0VoZYn?Zc>nl9XK8(4CbT>c&tEFiUtp~A$5$gha&o${o ze$%bd!g%sDpf>!eSOdJ*eR5(zH4tq)DpOVYTq{I`b>OD_O`)QJ!4#g&h%7C!9h7_{+L`B#{&u3M7{m`f*2`MTp$mEr8$ zZd{Q9^fi^8onI;pb$50?Lqud({t5ck7egl~C>XkUczwDqA|ld#HRs4@*U^X5Vr(l8 zHxJ}P(snP`j*mhD0G{ic>oz)fb>^F=8^5^m26m)-T}uUj^OYPxwDh`>z$^KThj;i^djZw5)%KpE1LM2j9w} z0cbw>qrj^3jywmGP>`EBjd=oC5tban0(kD3?t6>T zdYr1?Lu>2eJ6sxhjT@Sd7g)0yo{bP)uJAJ)zZBLZq@1+2AO7>}QUv@sv6&KX&e}!A zPM>TZe*$+8=(_(17zvL4VHeQWT@4G_M})tk(SC%l47Y{PdpzNMPgR9IEYm;?gkRR8 z^S(MOG=i4lk)*J#)?;-PZ;J7qC8;gURn9C z5j>5Lfw}DU3=Ub-HJ%RYv*iyeRNS08(|HH2Ne*Sb_HCxxHab#ooOF6>cU!c{`Aqwt z9>=yAFelD(RW7y`E|*s+9v#BRqAo;xKC-OF1oOW+O>niJ&$h2loF;4ZG_|Y6s;bRs zV)7e#m4po)}`AfAaS-;p?_=%;zH)+JXOK@o@(n)NpyxY{;mZ|<#_*=Nv>F6Po` zRV;#d``|$4KcP7$qlT8hp}xTkQ4h5G%Li}uqVYzZ+QsonGCg2Wa_7+;X&TfMDfNs0 zQfKYJj2FxVK6zhlk^7lpSVDq^rsnK?txX^ne0-a}zJ3?q;AEb*fA@1iYgT?PYpssE zo{Va}rzKnEd+gNRbVRNN7J+=j{na^1Cd$ReUn4?!dgU=9DjuV$DPp60;sUM%BxGWk z4i1^)a-2{QS1&cu@ywWWo1xNPot>SI#_-s(yDJ3`^v9@ZXtGjLjgJv@%ma=#>$z7T z7yCKs7T{`Ppk4iUOwL!%Uc@$Y)Q+7zP=0<%OqcI&&xxgmt19EsPd(Prf2m)*l}=1b zl9slXPNp2p>sgOvLN$8xuK!QCbmhJ%oJYUwag-}b#4q>o&0Oj>!5|I3kH`>?JJt%} zOW)#sk={VCdATJOQ>WST;IvUCPQ{pZ<9OavP!$-Nwil#|gRcM?oi4mkjEtE6&?`?- zp{gqHpy@(Mb-W+%Z}B`dbpk0Oe+wj4T~im2UT?q5j&tt4O)%f9z*p^b*ta--J~Lgh zT;O6mI-0=G0{}E9UK69Q5K2DU?9a%s3Z!C1l_4y+p7HX!oSD0gOgRL{awROMayY8M zB4}Aq7X)&RS604}Y6MHvSb>T)_Vp44_0@he{wSP z1ksnEOArj|Bc|r0x+$$bk$uFK40BhpkwR9wjC%~VY`eU;ajEloY_2sa7^&0tR7k7$ z?h7lVZfR`M7&>x0V!ep|_sO%;mJiw_Ui=Np!}3)#J*RFGe8Xu?ctDLxF{$>ECE3g9 z{0r^`?S>}!`GrGdx`$C>t`+xoFqgcFxli?5uqHN*bvLwMGQ!AVZ{Z}R3db|Ri`c|O0#eens3;>- zau*jD0|NtM;*{;BcQo8pIzK;5EpiYe&B{D3*{mk^mJdN@S{mKf70IV2hy&=D@E%PH z2!AKVJB5(&aJV3A+LH6*rIe)PJNE-kNlD38X03(b8&{pA!j1cMsi7;32lk^PT@l6* z<#Qo;+8hoYcT40i4Y%rh!Fu?a=15%rqqeuVJNRf9mqJGf>QzCw>7 zVPRBzv@WuOJXrTT7C@{#3PN$rs}HQjNlM`p34Lp;S?`#GR|z zSH+UeN72d=`^@x(PYA!c-IK^bgOZ0;bUDJWzxu2%K!uuc8tpMdItcI+8s}$e)W&^* zz}PsuU|nPXgt_N~{nA?t$^F-?5;|o%??c&2vR$@q&i~@VR@=|NAJy02+1*3tudXTS zHkXGz^vjMmZ3`-0&QNx_&7gUupgJ@Ay<T*4rrRN1&b24SBH7ab?z}X-}Ic5 z7<_GMOflda;%nIbt)j=(hG=6wduo;!#+mM=sd0e8a5~UY<6eemz#~8qKWe>PepyFU z7V2=HK2fhfg}c=L6sLZ+=iemikClEeA}~~MS0iBcMfi$iS~fQOD_Uh**9V6hRnkhs zCUz5$`6frp&^=s|8VoWuQ6v`r0@mjCZm=yJpX7Z0ey>+scWLkE#y2P!YgTg{!Fbe% z0PqZHc!IQ&#AZrM*V&V@_$aEJq!U-Kh)T15+#*aWX)EAGr7|&?im1`l`1`O^($e^iH7p;wDUw5!i^Y)T9Z9a$@_TF{qFr&V2=kcDbq01nxHeYY_0O})#FF1-<=)1d-!;tG-Cd72`)Rd z8RDNB4A_|~T)g0WDKn#gz{?FX>k?GN6Gc`FRHs@dhy^YccP4=S86wR6gE8IX0zF%D z?hiA+5%Xk_YH%s)Ng}0%@(Tsx!Xp108Zn^HK9NA&^MeR#V~}%As^J-*tPHo>B@ZRv zI(atDxydNY>)%3U{)ujzD~(9k(DdKOpEqm!8<%IN$p|&RXwhFbPu0^th(Q9J;5ca@ z|KtGrR@!lybZjAUvv{>gKM+$Zdl`T%oYwo@e2=(FGAiPnVOw@$Rt&0Wh_xDAk5$sj z#>P*c6Bu6|-^*Or+3%N`n!`VP2R6Z;3}bZi7PxUu(m=yq{x|&3k($2QBvkCnjJ6x! zNxooGC$K-fzbIACbE&s(OgOcUC>IoyGrl?qANI;t`6*!d{?U7n7`arG4QpAisLkE177 zRa}!mN6YNsJ(Lm}Qi(w%I4IgbJj^166iVSx>BX0bm0=i}MpARuasPKjq}-e#N^zmZ z&#V2$;)s?YKxMa|t+S^~c)l*1=s;aFA@8hAc=3nktJC(ybT^Ae&K(}F%O=6Qp)g#t z#|=760K6bt5tfS%z`_)Uh(-tW!acg->>dpb7meJ@tv~}TqVf+Iha51U!>D5xyFHT< z`8mOo$L4qNHc1s6A$fX!W=g29jF)>V;lt5Z-O+Joy8nVZB2Iq~L1wl+8SRc`Onv4kpaRYscM> z6LpkCwxt8gjSNe|v$LCr)jC2c0#*+{SonfX$Zv5<3hDUam1ka$@mm1v!G6p?5LDaK zI5R0_SFo-4Mk)-d8ZT3T5(BHXB2cc^&(S_IzhZrD_7^KthBT0GS^%cN?dqD~so}B~ z#M#K<@+w^(C%d-AnQs^HKgp#!{5t|ea_D!ryWjKW>p?{Ko%-na;s$ZgNV@O7?V)?i z4XL`;&ES+?56%(J(Qm|s^zWkE$eZF+w2OOas4cg3GVCmwDoOt6%VXB; z(&l}%IXTy6&_AdrWv)?B9>KmHqx1^xusLrQ74Ogj9&{?#_fEcYA1Yi}>n+Xwiu76k zA{A5Au`dYDE}29U%1PMm4|F?OeJzVpv42hNk)hhUs&ZSp*O&<`Xb?%elVd&p8n&pC ze?yleHh@g=;*oxXxNtWu01MpV0z+#hRqxRvuU=2dYa`ejCRqO2Y`Qt<*#h@Ecu)HF zI_*c{6DWWE|0(UQ!=l{w_hH;_r3932L=dFAOF_CphHj)`=%G;okq!xIk&@1#Lqs}; z?(Q03=ow4E8Fbq$vHEY(oKlgoq)~cClme+?Ih9xI3>nLIZbCFfx zv6`Qw#t#ck5q~BtMa~kqzNd`E=*YMvNSi=sgoZ}%(oh)nws)eOm%pp^5Ep-kz^PJ} zkzwa!J96ras=TSV0Ty=qQnU1xk>Xur$o&1D9Ij^FtR#NF<#!)e1+3+Zl9e;#Ta?Nl zes4h3;CQAd*9kbSf4+k!s_$vDR51*REQ$qQ-OWa$He2szElXaP){|vgCLD1+Nf^F=nH1YVzU}uOb{{d6~mb{TwPzZ~N=;|L&n#C|THO0fl{mu_;*ys1Cy7?B~Hhli1e+NbQX^dU^ zG31gR8|6)I(B%JC;Qzk@-USa;UFw+D`B@!<&YtzZ!dQ*#9S1P-+g2-BnZN1v*KOw-@L@{9CD9i{TeIpn!TGfoyx64) zXYf0>AXnIQaB#4trG+%Sy;Bq&9laxh0uvp*t6CCbBxe~3gFSssLJQT_4zEyF87`LS z2;g($^717b5FoMI+??DLqz;I2u-t5TrXN(>y7g(zek_+hcQfvZyvM|&3|+&qp_y21w>{_6D23=Hlrg_IYdRs~>fy1{`raw?Oj&+O?$)`>^Hq_h zTqcPC9Tv=t=6{e;pdqybH(I7mw_Ze%jd??r)uop%Z#X)$`o@dw;^L9=FEp2~zBV=~ z5~(L59cR-8F(p^$+YySWs1a7$Z=FeQxw&|R*;urWM>{ho_Onil`O*LnX93kLzDDtI7y`}nRAsC1yUMpr$!v02v%vk z?^b9iXUfZ}UMMh0k{h~={w1ZOT6f6}Sf7oN%{-*l--G0*5`EDORi+P+5vwdUO#U~7 z>~n>H7aDeesvJp58IG?LE}DtBtDG_CW752$eJ}hytNI(qXG8yByy~b=OaE1hCKCUU zvQgqWmcNq4QuyWFS0-+>qAa<3hefF=7}$NfCde=S#7KQ=?QL<=XNVt+!O>NK&kaU( zWc{Yze{8b)+Gfw5KV68sjRcE2HmeA;S@>yS*v(mZ&}|gMtD4Z{4a6)|ShhQR`A{h2 zuwdADH-mXuQiSqz@xRk~<)8;BDvY;4f;G01j1QN<^-WyhAZl4$So7(%%j2dNGCZl8 zWb-ykox{~2q-P?0N~Fm;O_Ql2>3Cbq`>>;|qAmie7!Y&NG1%;4)z$IEd1~wx=IR_( z2W#i(o)2Mge4i)tJm0`UzcO2H%p(IWyeXpK(HR_u@Tj%ql<)w*wvCKtrh*uZjd zxIebeX7TUdvES7qvCmf4Hoz$&Fvf8RjOnOKk;r_y>+ez z8Ozon>GI!30JReLZ=E~t#>AW-@s^e~v>6~K5q4=}ujTXggIL3h0@(2#Q~oXss_yV0t$t#LY&Da^=8PVuk>mI0UZ7bic|C++EPu@iLThBbbP zAQJz|=t_EXqN1t&;Vs-2JeCqw@&%PbR=L6XGCm~^-|KomC(@!gbi8tFwEG34X0t~l zXR_a<2O~YV49k&bBzZ7Tk%m}}LRbFB{=e6#t9{F)BxGxSk!hs|g_ed!qyq^#-v5e` z2{hQi;s1+T44KD3dEVtvfccF}o~c^%=m2qabb$51ia>Pzv&Xg-(>i1*ena2>b?$-@ zpCkO>SX!*)rJL%vyhGH9uUT%<6pr@l!s zs=B_{1t4?5tIe4r9eY+4FBdq8HEz8H#6<}qwR~OgB&*hP9AE7HA2^!iZ-G2Q2-wFK z=mU-hhPJwW6EDkSGye1fl%8MijlI*=85@QMryZPp>?)tq%uNw|Y<^6#bpR_Lnd$#Z2>2`HK%8)C9A8A97Yl_IT~5 zu9hR_BT}2mkv$fRJcYLg9qzu?r1-{CtK}~8Xh(=HaNgX9V}Im0Ea2g z$;A~HI}*z+`H6^!K(#aF?NqaCCu+-d3XJpRU}=CHou?ofKUi+XZb?!BfVZVl-YNos!lI4 zR@-T6YYc6fbqq-P4kd=$D6ciO#a~Zdj9$7#V3A%Bp8CdQ*@cwt(`76qI*K3zt950v}A4%C}8ImU)Glz_lx&cQbj@Uyek{>I8i>Y z0PtsD__t8N){v#?M|!;IwO!A=w`)1~S=wUVbv|OtgmjMMc|pJMv6ou#IC|nMob45; z>~;$hcU^!ILaYCHErPlkm=vWpX6 zQSpD>T|%UnsE)f=4FqYkRTDEtdpI?h9pnO1^wBqq)5wsN9V;%5zrdRB)g$G9+Hmr> z%yG&*kUjyJ5QdHI?bD*7jIR9Q07&fZ3x!}Y*1P#x?5``X!?axrm&2yw6)k+LWq2H~ z+wVQMy%CtBR9FCc;!g+;BZC34;YTXDq^Xy1FUjqd^++w65 z>T6*4YzeW#b7G!-cflv)XAM*B+!c{_yLPy_=q_jmeozp&H{?Ohoj3|3KhL zmOk*pP%(X^O`mmbS>&|m8P&yZWW6xQho?FM6 z^Tm3U$tkAR%La|Bl>}T<+VbM~Iat4~e0m93X?`m6Uj!q!kntjEo-B+`FPr5%w4w93 zjg5_4#xuHoe0&_(v?-g?rS%wGH=3+^{gK8Hy1ofe&y@AOHBPG`!Lluw?_32_gMxw( zLqK;^VFwu8sHlhS87gw!!Bo|HLp|eFKAh{8&wuofnE}{4O%O20#)gQRufMCdK;~5( zgE{QmW~fCQQhxgh`bW+mN7G`F|lN~cNGrt;Sl8455y7SE)i}&zz554SyX#M2o5-;cN_6*T{ ziy04z-opYWvDLXjwz;&U%_=-Tb&V?MoKF2~JsyLCJGZsyq!APp0&6 zc4wicrarIQz*azqBhG8HnEkVOnF<_F(5jpZU_)o1DrZg^+ zDiie=Tu)efoHwTNB|k8x!N+7TIJrnDEb%xZSGK&RL%(ej?RK*IM#IU;8D>F19Uzs- zV0O_K<;2I^R=B%AS)!r(nr^nqWhLkstey~{c7Kyrce7JB)Qap+9l+E1#2(_<>__h>(vrfL4|KQ1cvSLa!C3d z@_sn+O$(n8-;H5h07ZqUP!y83L<>eIcD3!gnw>S@n9Nx3UYF?b+Z~j&AR9jW#?C&P zTiDrs#)s}o2>AkQV*+@_|0X!^BvCfHsU_6SDd~8fzA=}GmtJ2n=;&a>8J>dbc+Ucx zCjX~jG%O+SKU8WZwt$C;$rnOe+CZ5cOLwK)8@`Gh`9;oWj5g@Q?Ra#E<}XBY!Hr4v za?Hs64&?L*=GTovDAaYoaE#geQJ2ct-b z_`&_?MnCVk0}xQgiv6q22__~}$m-W|JbAv&zS!Q;y@5jbQeZ_4$bs*^c3Y?*r?To% z%h}&_B7$e?jQgX;UKm7F=~TZVGzCen?|yQ0m-5FjC=wM-ZJk6%VS}!WjQDrl^xn*4 z4ha+b-}x(MrwzcZcGXLsD>%-O$YI`Bn!Fr+7p#xh0}tJTGh+UYyL6q21-XL51ex=G zy>XTiSM&gVa&DKA<~8G~o3iD!oos_&!hTmPMeLcpAYT!4z{9_~Kn*5l$$ruEAHe(V z0grZbw9A6Vyw8oB4b1D^B1`f?oc(*}i03Aa#>>*LN;~U3EN90nJSl}wSNosp>xGcw zENc|_T=u3;^n!3>^1QHIFZ#Vx6|X$Jrzs6{$96=VtTl#vrk{x6fPF+w$GZ)7)!OI1 z`~;&W+;>(ucDGiz$VBxyt2!9R2PhlcUCZUg$}WaWq_JGSmP1T=>U|0y(!#s- zOYq8E;B9H9jdAMHhrSJ(K0|5YH|t==fQNT|J__GAWGi+5MRl)u-sl+mnY$Uk4*4q; z-A6@UC`KO;^;3DbGzcsHDx8ExDLSD+xO5zGQteM?KjZDDH~M@OsmKx(NxH>|$>Hu` zG0)p)u88DsnBA$9#J6m)p5*;385B{gufBBX%Ii5-2`;$^o2>PaQ*@^O@h!r@_;@bu zyz9EO_BimN`0AY5!opj?%pf>rh)zs0?9+ZFJ31;(oRpl7=?m#7{)8B9&{!4nyU@+% z(-3|8nqU3S!s}Pi^6J;M<-o|r=Jmx!p%r0fE(~)WP;SJX(tyo6|J%6*fRD)zUv%ahmsgfX1&r&$5^H zlZ`Li>vnDYT1+VJc5(<+^R4xme%zoT&bY415;csEVjwpLiU%ABodkxU_|^QgS5Jyj z&gN;StCB882cEn3*Oo{t&+LvY7qz(B0+^bbLv--7kO7#Z>ytU}oRyl;jyQ@tA6|FBUiN5&orLDXm! z?{Fvse0#`&<+LoIS*lf(RmIaXEE>chX1jHD8_~cHy+XjivPPS1u5JMtPSIk&E@*ZJ zy6}IP2qH4Vud=x2eHz7T%@Ri1yIn9mvcEWawkF|uh*3*b{b?i+4#XCDd8vt?aIWjaFI8b$?og!1r=uHuJ!QJZ$1CC9RP+=c&4SyYtz_^&ahg z@`Ob>$0h#`h`9_(RP!SR2Px8~E%xE1g8dhtDCU`wZ7PuV{riB6=DvuaP|2m+E2823 zh&wv2YkBh@oz%n4PO~%$yjydRv`3v^oGLl`3!Wz%%snEh%~d#5mI{i0^rWc9cy%r( zI^!;H;_@1LvbzM)brUVqd(5ch+ZzpX}>(DbN+^XQQ7{;0R(U_aE{v(B5I>TzSJ}ls{_!0K|o;gjFN+c zKcF~0Y?Quls&hSt@0V0hqnS`^nNdUJ&?tPvT(0$2Lt7E4XZ&z=i{H5UT=u7 z!I<|^+^+!-J#fH3oqxa|zku$q+^}Ci|2;ztIOx9^GG=nk(cUBsBbGZzfWQYhVE;Qh z28Ia$fxm@Dwiy8@{gGq#U(Wq&(%P?w`P)bO|L<|UdGQBp4l5A{_UlxK)!hC0xbrzP zLUryk6(~4fy1Mk~@rTMQ!v2gVH1+4Bl*c|_4pKZTec$AIO^lb__XQFVgkM{~2e_uq z2=4*Ug@j!2to%4Sv=-`b5m(1@89_14@=cC$X-L%L^2Zz=pDu)$Ynwv(qnUqX{T9gz zs-#D8W`?q|(lApB5SZNhD<3`x)`ZQokB}ZaviWI9E%h}=UX^+-M%i83@18l@YN(r{ zn*R&rZAJUpplMinwyWkMb>n=G#6M(!T?!^@5LW~?%8jZ#ot@huF}|nv`8!LS=HJf% zPu7>)xFWY@WKg>R7l+9Bwc+f^i{|&&tU6xR0CFXw^4TX2ZZUc|dD)@K0KWBZzy;lH z;vYU)Z{I!lH^6d$3zYgKpNX0{I!I=hWP2U?se5Q7(R?nJ%q!uGfr2TR9(+#7&bx)#=6Gx;)x1;{J#IQUjo+chT+YS%OdZa78_8{Biwc*K^_k)C4SO zy21Im1VjL#wMMie0fVsPz2{a|+(}J7m#o_5v_e9c>ENWKq!ajb^#?!?xo$5nCP)xD zldCE|bX$pf&P5$?y)%@k%7<)+>l`Zaa8Y~mGC~@aehfj0ydsX{fm>8zSq^5H+J7IA zh9&=93Q=_vuzL|>B`Ko)GpbN}qP|vFpja80^y&h&<@9Ef=A|H&AY{Fy%+xzIgqS2w zx}v2OwZ2=tVQ%o+ZQw_EGAyEi^jWH*psTh*_TY4NBa!X3azpTcGRqEreg-Ch0?f&L z|E_@>VA}-jgKBwBC8rvU8i1EDB{enF_k5r#z45$*%5UZXd9fP$#Pijamr@Yuqn|wJ zLD}Ugu$Gx_^)Uz_1i83ePoNUQaT|EK6&TNc{3D@9qo?sg^PfEIHH4p4wd?%ZegCO~ zBtPw^3>;;&U;*ZTxFiA>Pv(HlV>P~ej|O)ID!St~&YS&%0}M<|-Fmm8 zhfkQ9dwY5c_@!mdmF^r%?1@>kB|*W4+L{&fF|ev8?QyYxM|aVCjPpO)Qoy-zGb0rJ z7dPiwe{f)et?Go(v-O#?^VSbIAh>`ti)hbF(@Z6T-P3fp+x(m3-PRaMZA=CsAzd^H zZ@sI`BAzBcUtktW0i(RR@kmQc14eUYbt5AqIk|46Uu-@~*HCVGsI{}VUiqsz0Ii{u zBl@oTv2nNpZ1r1m-n^u?v9&X6ot<{PZ|vpgD=pjXOfNY!VRaz=TP^L=b-EO$Xo<^S zQA4kldimswiF|c8*p?XEuI~irpWLgwXL4?3tgAjrWbk#Q_$;eE`GxW6fJ5 z=)y)}Gvvz206bat1q8ok4$rQRu;52kX_1ZXE6IIP6cS|TvK73=A_7moysv#I?)S@B zunQOqp1={Q;jvdTdQt!CVu%D>41u43=v;laFAK0=L{SJcX{}s&?ps?mT7Wq$?6(eJjST|rg{Eed9xya{oQWr(tXzhzQzU`PABEWwe&D- z7Mg>$xTX02(~c0Xu!iO7;%0CP|H_)PNO8Avt5?05o}iIi;cuJ)@E^c#c|RXpf^`qn zB`eTid6Y&j$;$vRH5j(Y6WvTHw<9h*@y>|<%E8%=#{Dh2!SpW8`t&rJkMi`zp0hfi z^e#EK(f3?=uwn|-vwxyw7@vzts3QbQ>Z#i!j`lZR0?CG(4?6>k!*;51bIBD>lb6YA z0Z^}FQR+<$dH@cUG(ge%q^1mTafZgL^t@oA(~^KGbA|J_!fU~9pH;q zCe{4bkoDW$ySM1XVIIifNC&mM*Fhw;e?p*zdy@Y<^#Yiqf$|HMEHqO;xh+cqn%x~Q zZy}cZdlx~9pZETT+0oj%EQ@WyL412W(QS4fP%z?ejN=;S6FE%VE9_DJw7>!r7*L*R z&7FU|JY<9!OFd-+1OT8keEut;sDcYnS-`uWEn(+*ahJEw0s!5;1DXXW|ANu}>LK`b zgNWa*%awO(_HWV`7|bOx9bQzZ%J#t3U>Te zG4~gbe%9vpR_?K)^wP%oa!m9hi(~MWlp9~^2cF1K57K9Kl|Q^T@B7l+KI6jbM-TJa zlu4!tdS_RV+-5GXZ+w8dy4^?Y zC%eZV2dLooUwP7w@qU&43f<97*h9a-TAdqK(*HI;>jP9vAxnrGuwPsYkx@Ss@E3?z_hWwX}254 z^R-#8O#w#Cjwt)PWVKy^@T(8Km_i_X5f6F>iIVctB9$W5&~{(aQ9S&w^=?MP#p-2_ z!3>*%ZCN{}a6Vc?!Tp}y<#lj2aW9nM?lzyDR$G78n@61j^}EPCAl1FxG?Ut&Oh=!& zG+J&d?(Vc3CGrv!GMAMS?(9%ME>5N2nN6E&KvjzfF-^WRYcpkrU*>%0cuKy_%Au)@ zROR?!2zEN<;#+S+0DUM<%nH`8xk>@gs}>2HEiWH#SViGCBBwJoi|277xd zbyOOE_*WFc-Id*vBXcgbmz^(_X1Y?0i>0FZOVsbNJ!#MljV@G;W{pNhGylvGY z>QYiQE!X>jb)kNKawxELy|FxNm2fAgDJY)wNfA;Bd9jT}Zbm%>yE*Gi`WRvXr?Z2X zn?&r)H!-%_T1IYxVmc#`+py-3B3s;m9C>rHD_j`Ng}8erR4XBl{ha`fI969u_~E?< zl>(8hVo=-ERFD01QIUOh5Zuj6%WGnLa@SsWde&aIM5hEg$0ymgO#jS8lJFXAhiq(+xookS|d(1`gz5@HY$&) zTh!C#(7c5zT@I01*C3u;#owI8!E7~SNz!58kOV1LM3P}bkTtupJkNZOvQVlr9)at|hq zc10MSV>Lp&Ylx`vbqRlLeMZ+8GZz%|l_B!_oM(9&X`{^h`vXc2W59K9W0m(|YlGs> z&!N;oXei`~+fL0(21#H6+-QVhJuJ0E2^}pfzk;iu7^zO!oBG)dkCQe`^K`Unr{aRT z3jhBCg*p)8!pz^{PQ&x<7kda<*8!=TybDqgaB1^ zwUDSsWhTSTCszhHNjv_zSWR8Zeacu%55=W^Toj(0pO$pU-JdP6zp+V(`CYoBy{c)e zFeb!PF*}edjaz_Ys0(c8)z=li9m(@j$?pj?{msLmrp)jC)jcERZuQ^CPvfdSBDX93 z?1_CUB-;BEqn`neHgx~1vEqp|*FyJ+%4;YE)%{q_=Pwm@ph+$R0m)NsspMj~LvfU! zD<@H%jzU)GsnUsOJTVGdR)v1^xVL*SkG}zo_ch38d!LOM^0-Dg0`i*G^dzpneDNWv zpS7y3mVcGLmjxW@}_l8WS$!6Q{F;1&uOM9RADM(@vMgxn7Cw zrta7PEcw(V(sh_kaZ#R{DP;djplI4je`58lew%;}Uvj2?7Ho6l@{aysi8HC4F32M2 z1*&3{TL-I~$CXP4>Zkk867^Ga3XykyPMbe(R*_HurgNB4JSQSO>^6MBol zS_<27IL7>kK6CZ3Q4?1S;l;J00+wSsvZqJ5<-&pwgssGh+{(7~hk-uxyZD_p?oZPr zC+40HezGA8@$H0r11eNbU%Hfj# zeupZ_PeXl-(uxq5Mbmtg2p`*CXlTY1(MhfzczhN^$SQwAy5ALFpQYkK=nAB#zglxO zg)!r4>&0bRBZ)G$KX#H4e2F#Z8Svd5=3e$3cgaIdVsbdrOQYj}W^T9K;#w3GPa4od zC*m&5cIvXVyZh|xV|18t-f+cz1@h6z>DBMdVy{sy73Qw>=UOki+?ngd!4`F<+x^m+ zzBFnl5)kjxF~=npDl0!SqS5 znFYMUhh^~hOFL>h?BA5dqvK^sm_59w{DCd3MULdnC)D!{)>8>~E$cZ>!yHP!^DX5s z-675o4&gFzt=dZ%-q9 z_R))%zAZNFgmEci7t1`tCo*y7DEn6QJ=XSzKbN8xCyab5&Gaex+k|R8f}90)zDC#e zfW=c$-f=I(_PoNz+OPpl5vTSQgui_aoQ=uo)fE2h2~94}j@y2AOKWQb9e)w6Y1vp@ zYQKnGB|pD-BB#^APS@bO?eq!;a`w&ZN0J4R@XnAlIww#~D_m!`uoUm=jyk21DO}Wv z)n*Sc^AgGWTuT4$<>f$tH?B!RqNeG)Wu%JNhbt3zx4lnRF*sI*at}>8X<;o%0I8ny zV3d+ST|bK5yP;q?S6<|Ku;~z`vL-oyZ1P9PesOfO?u-mj1LBTegK?OYX~)x$(9q?j zr2%ymNI*kFLuqNLQs|-&762^uiN$L&YierhL6QzMKvpGC_Fn-Em7ALz5GyEy(SBi6 z6k!!02Uka@n>`GWnORtVp?Z`z25)n$5*6qkUWu|5=jVqQd+xu@`2~r^gp%Lq<|k(!;}JS`#o7*Xr%EOmP3=I8ka|0jZy=s)Hv4*=m!+<8Pqf zesQBH3wn~0ZFaqkzgzso%qZXT$Y2G(-@A7Y6?MK_Z~JH`$IixzBvrsaB6wlNMK&7B^ZZAo6F}mt1DM z+;(*!_%=c2+a~5b=djyl+fdzWGB(y^vS};hB2dBjHVqBUQ{3Ro%xQxNW8%y~EftmL zXo$nZ`U1h0;i`FCJ&r^L((w0fCJgu?pFcmQjOuVT~g@6Mq!iYJBn1S&JOoaKklkW1;6<*?sPfP1{r z?NSm7(6cwRew5q;O`fSM?X!lSlobw%Gse|ZW7 zSxIN8{OITy=GIqJ`_BI0HgW}$`VAn?nY*7$8Yr+v6G|-YLz_qkG?H4oOQZi5s!xKCAf|@_L&o6N?{1#eCS_aS+TaV3M5mn)?_cL#|JF1}Z{Z z6zD#7nT13~f|ElY+;1H#(oLrCd-?KZR(5umS$Wkt10y3NAK$ncr!ghwZGCc=f7t7j zWviTLV;w50p!;}aWi%m{Uc;+)lALqmiyY|siR;+I$`cIr&zxTqNIXk(-`)8d^xMP_ zwdRBMD;A!Jx$&=rC*7Gh$g_`oRpi%2GcpbSnbKMPT4&>$lpC=6du8RPtN6qH_f&^{ z_jOXvr!Bg!gsR`BrRoD!w1(EfJqY|;O1;kyQc}o){=`k{?Bo>BW@yi@$LAB$XAXSU zDMr8Q9V;VKjWiCTf9WHrn@UIx!kE|Q@{JC= zCqpE`FVD?IUeVKh#YW(!uQz%*9Vr-;%zZNonfF3&GGtsJ$s#QGhDb&fR&xl*c(Cg% zk3;lbAx;v){PiZh$;o?n$4=UcbgOx}xdB&JR&FkyIMASw#qWW@oBiox&2}7t;;K42 z*1XA%ke}nR@{(M2IJ|L$nk`obi1|pi-OO!ZWWk0X_WTiQVr*TlI01Us2)W(k!*r7G z5yTYNdfbtc8}F7Ant{DTTnsbo8Z!%XMZL`sC+%fDkym@Gi*w^oLT1g^9rn8K2C-^E zO-gvIc6&pxbekqsqN^TnDESCBpdRy*hV%6ou`x5#1^`dlW5$`NFh;JEWM~U?JmPRE z?~U6Y=gwKamqVa>6`bjf)DE!XzPenOp9gk>cs61Ot2wInccpt@n%%^OdQc*U_2RrE zpixkVnO5OAaA~zeL9ww^waA9s_VTEQ>s5F1{B<@Du!R-B9yZ_@h1MG=g`h?4*Lv@L zuCXeyW2^*Ld*rb2;q>}m>X5C_xr~vOpD{c?|se3Qixd5%31HUZQaQ`t6@D7KF=;$FBo^5e+3iptk0J zlx=sTYiOv9mSJ_#PL2yNHt@4+8IjNFTS-16A?^HvHmIL}0WE%Aq_BFPzuYw> zYfvZyt=_22evQ^0gUX!LdzdYQ938?r-GVBBZC^1>cQulDDlHX~Sm(~SsXs}-leyM( zqV*|-VfN-YH(iudnbO%smbrjK%yipWf6H+0q9SSjRT}i72gd@2pTdtYCl6wWUJ~Xi zUFrzA?^f)wV4GR9PKzV875b{gvitQUmEDae$5oF>3|aj)?=+rnvM2`}RdEr;n~D`T zK1;hW*V$L;o2iLRDAx0`AJp|vUUy~>C&IaQ(h{>DOb)&D((aBZzC5iT#z{AHby2Jf zaccbPI{Q6~VY$JNKPK8Pl_O0Wt!&jimA$7Ul4L{z#&K)}b*oQ>?O7%%`S>8;W09dz z149`f>+0dTTRjDp;B8k*djKbS@6x)18ULxo=#luR@+~Q*44u3Fjb&P}hioA<`meuF)1@5|A&YaEnz0uNn^q;YSAs*VQ=LMS&{_{mc~Wcas*y)g7@RUijBn3#w;LC+W!$>T^aVeF2ZkrDgW z`nz8{h20~5-=+-nX+K&h#NdigL=Rg$$Y2GZ zZGLL=uv&}~6BUV2_uT(>uih*4+5DxsGA_a1gxi=BnmXjkHglWM=T z@uw6Z--ym;+KL!kakSf65hUIq@y;+ke?QmLG*bt4Am;O9uo@&=Yu&r;qQyBb~-pH}g%0veHENmCNQy+xz}` zkUqn_>C<8CB6bGXuZqAt0ELP&eEgDEknAfsv+?EB?1kK;%z`n28NHVbyYkJ_-d4*O z!)`+kW`^-B_T5enknskauD~#hh%!I#)bC}eL!s*P9i9s#D zoFrqkRXhTAsUfFv6UN89MzKWHKU7uJz}C)8rtQXkQ(l_|@*va<{OGL&yvCYYcj;Ao zs=34Wk@C3H5#ULK#g*OH5ldjoWav?;`x>gv33>Mna`})Zpx6WkHD;DyqcIhz9G$8v zFzxa-W46CazkwWSkp(rm59IRcW3+ETkd^X#>~MG9@AI(R!sf6eZ2@;_i%;k5?f3is zN`3fw7rs+`edx1uq_W70pGWJIscQ#>s*s8w{ua$GlBd)E>M>y`)_wo#>gwTaDxV|O z4l#Rn;{9Jj0nV}<9~L|@2YHjLEU$is_8Qh7tdcrn!2YQD;ru2-jYE?B_@Zx}1zx>U zYPQo<7$ep9z(_IzL%!M5_;p|=geTwiYDa82&;z`8b}~!UL{)#1_w2>AqUs5zfJKrx zrH|L-JA78^A37y#>Zvm~!2~IsbSdX6>!Y~_dhrXR2PRH({O28sYB!O&IF@`L%(a+9 zObS6v5}kHfY+=6R7mZSrjZ}rc3>m9WM03oy3nG1LeULAPeTH@3FOQ$x(XX?beN=E$ z$T3JhwBYne1EPOdGEZ5I^m_T!u(1G^AM9~~!y%FY5)Nj<4bB1r63k4>ja8@VsrnF5Kw zYbn@3Ko3%KLl!+0)5%>-T(z4TlSxNsf8fpET*vKfQr&P++apsFm%m)qglpWRKc&F( z-un?UVeEJnXJg>S;oiazYb6k74u>1 zgmZj(eFFETZ#tSTPS}LTsJ^D3eE#%M+*W!%ou}DJDJG<;;ZTg}gg-`1J7r5!#=l1r z-__@CZEO0u`hggcPgxSzlE1b{B1pKFSb-K-fE+tjX*{rR!$mNt^{vG(9Y?e(w{8>K8B|KXwlr*1=!fi^`a6Bqi_oBaBzM#jl18QP7gwgh}JE!PkD_~L_t%J>|1&z%O zGLl7a|61YM`56v2<@|pC(LyvL55^{Ft&O7a|GsvBqoJ@v~9{?<_1QcN>m@cZf|0|zD)v1v3MtOQS9cDH*HUWX! zlHEyQL0ce9kH6u4A5^W+*l;p(yu0_i)aCr81}ssGwbPv=cJ<^7nSk647t1yHz%N~w zr**hX2tGBZRA0PryIZftOc>gBetr(9*!DAw#pjrJ=)}aD&ruP}$n!$#F6Df_eBP`0*jT+92Wm#f{9$KiZpHQW z^KS6`n-!6-)HcMaK8CCF+wL77&eTHeKq<844T zFg#&f)n~nkxaX52$y?CYXLV1QvZT0JiSFZZ5dZBk!{ zn_lqQ%TCu5YbN2z1D>%iGe!V?u&`)6!*{Ap0ZwoizMTqmY;0_j*T+noZW)1ZeIFjg zuOw_X`rzwP-YDH%Cm0hakE)f7OXtz<#wPTv1_lN`dh`ew9AMt3ee7X_mdwn|u|2C0 zU3&MalljabX>~Z*44C(%GAC`TCT)RNJ*z-K8!bZ{E#pAN4&35R@~g}e-MYL5wc0`Z z+{@>{wBlw~x>A(*$X9eo2~(Gu(7#1RSJwurb2%isif{+cru&FEzltZ)SA_1FrY=;% z;g#DV$>9Ji3^a^0@p~Y$4G$0Jo+VnsNbV!V2sA01=)=1Q#M1WFr)7EP_>zwNB~Ww_g8 diff --git a/contributing-docs/testing/dag_testing.rst b/contributing-docs/testing/dag_testing.rst index 7e311171ce0..0bf506c2f32 100644 --- a/contributing-docs/testing/dag_testing.rst +++ b/contributing-docs/testing/dag_testing.rst @@ -20,31 +20,22 @@ DAG Testing =========== To ease and speed up the process of developing DAGs, you can use -py:class:`~airflow.executors.debug_executor.DebugExecutor`, which is a single process executor -for debugging purposes. Using this executor, you can run and debug DAGs from your IDE. +py:meth:`~airflow.models.dag.DAG.test`, which will run a dag in a single process. To set up the IDE: 1. Add ``main`` block at the end of your DAG file to make it runnable. -It will run a backfill job: .. code-block:: python if __name__ == "__main__": - dag.clear() - dag.run() + dag.test() -2. Set up ``AIRFLOW__CORE__EXECUTOR=DebugExecutor`` in the run configuration of your IDE. - Make sure to also set up all environment variables required by your DAG. - 3. Run and debug the DAG file. -Additionally, ``DebugExecutor`` can be used in a fail-fast mode that will make -all other running or scheduled tasks fail immediately. To enable this option, set -``AIRFLOW__DEBUG__FAIL_FAST=True`` or adjust ``fail_fast`` option in your ``airflow.cfg``. -Also, with the Airflow CLI command ``airflow dags test``, you can execute one complete run of a DAG: +You can also run the dag in the same manner with the Airflow CLI command ``airflow dags test``: .. code-block:: bash diff --git a/dev/tests_common/test_utils/system_tests.py b/dev/tests_common/test_utils/system_tests.py index 578ee6cc04d..6558ae2d1e4 100644 --- a/dev/tests_common/test_utils/system_tests.py +++ b/dev/tests_common/test_utils/system_tests.py @@ -30,7 +30,7 @@ logger = logging.getLogger(__name__) -def get_test_run(dag): +def get_test_run(dag, **test_kwargs): def callback(context: Context): ti = context["dag_run"].get_task_instances() if not ti: @@ -60,7 +60,10 @@ def test_run(): dag.on_success_callback = add_callback(dag.on_success_callback, callback) # If the env variable ``_AIRFLOW__SYSTEM_TEST_USE_EXECUTOR`` is set, then use an executor to run the # DAG - dag_run = dag.test(use_executor=os.environ.get("_AIRFLOW__SYSTEM_TEST_USE_EXECUTOR") == "1") + dag_run = dag.test( + use_executor=os.environ.get("_AIRFLOW__SYSTEM_TEST_USE_EXECUTOR") == "1", + **test_kwargs, + ) assert ( dag_run.state == DagRunState.SUCCESS ), "The system test failed, please look at the logs to find out the underlying failed task(s)" diff --git a/dev/tests_common/test_utils/system_tests_class.py b/dev/tests_common/test_utils/system_tests_class.py index 836782b8584..5abdca96bee 100644 --- a/dev/tests_common/test_utils/system_tests_class.py +++ b/dev/tests_common/test_utils/system_tests_class.py @@ -28,7 +28,6 @@ from airflow.configuration import AIRFLOW_HOME, AirflowConfigParser, get_airflow_config from airflow.exceptions import AirflowException -from airflow.models.dagbag import DagBag from dev.tests_common.test_utils import AIRFLOW_MAIN_FOLDER from dev.tests_common.test_utils.logging_command_executor import get_executor @@ -131,31 +130,6 @@ def _print_all_log_files(): with open(filepath) as f: print(f.read()) - def run_dag(self, dag_id: str, dag_folder: str = DEFAULT_DAG_FOLDER) -> None: - """ - Runs example dag by its ID. - - :param dag_id: id of a DAG to be run - :param dag_folder: directory where to look for the specific DAG. Relative to AIRFLOW_HOME. - """ - self.log.info("Looking for DAG: %s in %s", dag_id, dag_folder) - dag_bag = DagBag(dag_folder=dag_folder, include_examples=False) - dag = dag_bag.get_dag(dag_id) - if dag is None: - raise AirflowException( - f"The Dag {dag_id} could not be found. It's either an import problem, wrong dag_id or DAG is " - "not in provided dag_folder.The content of " - f"the {dag_folder} folder is {os.listdir(dag_folder)}" - ) - - self.log.info("Attempting to run DAG: %s", dag_id) - dag.clear() - try: - dag.run(ignore_first_depends_on_past=True, verbose=True) - except Exception: - self._print_all_log_files() - raise - @staticmethod def create_dummy_file(filename, dir_path="/tmp"): os.makedirs(dir_path, exist_ok=True) diff --git a/docs/apache-airflow/core-concepts/debug.rst b/docs/apache-airflow/core-concepts/debug.rst index 9ab7819b8b3..d58c4908545 100644 --- a/docs/apache-airflow/core-concepts/debug.rst +++ b/docs/apache-airflow/core-concepts/debug.rst @@ -122,18 +122,9 @@ For more information on setting the configuration, see :doc:`../../howto/set-con 1. Add ``main`` block at the end of your DAG file to make it runnable. -It will run a backfill job: - .. code-block:: python if __name__ == "__main__": - from airflow.utils.state import State - - dag.clear() - dag.run() - - -2. Setup ``AIRFLOW__CORE__EXECUTOR=DebugExecutor`` in run configuration of your IDE. In - this step you should also setup all environment variables required by your DAG. + dag.test() -3. Run / debug the DAG file. +2. Run / debug the DAG file. diff --git a/providers/src/airflow/providers/google/marketing_platform/example_dags/example_display_video.py b/providers/src/airflow/providers/google/marketing_platform/example_dags/example_display_video.py index 33abc67b639..3c008ee5ca2 100644 --- a/providers/src/airflow/providers/google/marketing_platform/example_dags/example_display_video.py +++ b/providers/src/airflow/providers/google/marketing_platform/example_dags/example_display_video.py @@ -92,7 +92,7 @@ "example_display_video_misc", start_date=START_DATE, catchup=False, -) as dag2: +) as dag_example_display_video_misc: # [START howto_google_display_video_upload_multiple_entity_read_files_to_big_query] upload_erf_to_bq = GCSToBigQueryOperator( task_id="upload_erf_to_bq", @@ -125,7 +125,7 @@ "example_display_video_sdf", start_date=START_DATE, catchup=False, -) as dag3: +) as dag_example_display_video_sdf: # [START howto_google_display_video_create_sdf_download_task_operator] create_sdf_download_task = GoogleDisplayVideo360CreateSDFDownloadTaskOperator( task_id="create_sdf_download_task", body_request=CREATE_SDF_DOWNLOAD_TASK_BODY_REQUEST diff --git a/providers/tests/google/cloud/operators/test_dataprep_system.py b/providers/tests/google/cloud/operators/test_dataprep_system.py index 96f47fa3e36..dad77ac4ff8 100644 --- a/providers/tests/google/cloud/operators/test_dataprep_system.py +++ b/providers/tests/google/cloud/operators/test_dataprep_system.py @@ -26,7 +26,8 @@ from airflow.utils.session import create_session from dev.tests_common.test_utils.db import clear_db_connections -from dev.tests_common.test_utils.gcp_system_helpers import CLOUD_DAG_FOLDER, GoogleSystemTest +from dev.tests_common.test_utils.gcp_system_helpers import GoogleSystemTest +from dev.tests_common.test_utils.system_tests import get_test_run TOKEN = os.environ.get("DATAPREP_TOKEN") EXTRA = {"token": TOKEN} @@ -52,4 +53,7 @@ def teardown_method(self): clear_db_connections() def test_run_example_dag(self): - self.run_dag(dag_id="example_dataprep", dag_folder=CLOUD_DAG_FOLDER) + from providers.tests.system.google.cloud.dataprep.example_dataprep import dag + + run = get_test_run(dag) + run() diff --git a/providers/tests/google/cloud/operators/test_datastore_system.py b/providers/tests/google/cloud/operators/test_datastore_system.py index a98215a5317..8807288358b 100644 --- a/providers/tests/google/cloud/operators/test_datastore_system.py +++ b/providers/tests/google/cloud/operators/test_datastore_system.py @@ -44,8 +44,8 @@ def teardown_method(self): @provide_gcp_context(GCP_DATASTORE_KEY) def test_run_example_dag(self): - self.run_dag("example_gcp_datastore", CLOUD_DAG_FOLDER) + self.run_dag("example_gcp_datastore", CLOUD_DAG_FOLDER) # this dag does not exist? @provide_gcp_context(GCP_DATASTORE_KEY) def test_run_example_dag_operations(self): - self.run_dag("example_gcp_datastore_operations", CLOUD_DAG_FOLDER) + self.run_dag("example_gcp_datastore_operations", CLOUD_DAG_FOLDER) # this dag does not exist? diff --git a/providers/tests/google/cloud/transfers/test_facebook_ads_to_gcs_system.py b/providers/tests/google/cloud/transfers/test_facebook_ads_to_gcs_system.py index ba24a0c34da..9cb0a9be526 100644 --- a/providers/tests/google/cloud/transfers/test_facebook_ads_to_gcs_system.py +++ b/providers/tests/google/cloud/transfers/test_facebook_ads_to_gcs_system.py @@ -25,13 +25,14 @@ from airflow.exceptions import AirflowException from airflow.models import Connection +from airflow.providers.google.cloud.example_dags import example_facebook_ads_to_gcs from airflow.utils.process_utils import patch_environ from dev.tests_common.test_utils.gcp_system_helpers import ( - CLOUD_DAG_FOLDER, GoogleSystemTest, provide_gcp_context, ) +from dev.tests_common.test_utils.system_tests import get_test_run from providers.tests.google.cloud.utils.gcp_authenticator import GCP_BIGQUERY_KEY CREDENTIALS_DIR = os.environ.get("CREDENTIALS_DIR", "/files/airflow-breeze-config/keys") @@ -71,4 +72,5 @@ class TestFacebookAdsToGcsExampleDagsSystem(GoogleSystemTest): @provide_gcp_context(GCP_BIGQUERY_KEY) @provide_facebook_connection(FACEBOOK_CREDENTIALS_PATH) def test_dag_example(self): - self.run_dag("example_facebook_ads_to_gcs", CLOUD_DAG_FOLDER) + run = get_test_run(example_facebook_ads_to_gcs.dag) + run() diff --git a/providers/tests/google/cloud/transfers/test_salesforce_to_gcs_system.py b/providers/tests/google/cloud/transfers/test_salesforce_to_gcs_system.py index afd0856fad2..d556f2d86e2 100644 --- a/providers/tests/google/cloud/transfers/test_salesforce_to_gcs_system.py +++ b/providers/tests/google/cloud/transfers/test_salesforce_to_gcs_system.py @@ -20,12 +20,14 @@ import pytest +from airflow.providers.google.cloud.example_dags import example_salesforce_to_gcs + from dev.tests_common.test_utils.gcp_system_helpers import ( - CLOUD_DAG_FOLDER, GoogleSystemTest, provide_gcp_context, ) from dev.tests_common.test_utils.salesforce_system_helpers import provide_salesforce_connection +from dev.tests_common.test_utils.system_tests import get_test_run from providers.tests.google.cloud.utils.gcp_authenticator import GCP_BIGQUERY_KEY CREDENTIALS_DIR = os.environ.get("CREDENTIALS_DIR", "/files/airflow-breeze-config/keys") @@ -42,4 +44,5 @@ class TestSalesforceIntoGCSExample(GoogleSystemTest): @provide_gcp_context(GCP_BIGQUERY_KEY) @provide_salesforce_connection(SALESFORCE_CREDENTIALS_PATH) def test_run_example_dag_salesforce_to_gcs_operator(self): - self.run_dag("example_salesforce_to_gcs", CLOUD_DAG_FOLDER) + run = get_test_run(example_salesforce_to_gcs.dag) + run() diff --git a/providers/tests/google/marketing_platform/operators/test_display_video_system.py b/providers/tests/google/marketing_platform/operators/test_display_video_system.py index 78f5d4ee021..49f44948abf 100644 --- a/providers/tests/google/marketing_platform/operators/test_display_video_system.py +++ b/providers/tests/google/marketing_platform/operators/test_display_video_system.py @@ -19,13 +19,18 @@ import pytest from airflow.providers.google.cloud.hooks.bigquery import BigQueryHook -from airflow.providers.google.marketing_platform.example_dags.example_display_video import BUCKET +from airflow.providers.google.marketing_platform.example_dags.example_display_video import ( + BUCKET, + dag_example_display_video_misc, + dag_example_display_video_sdf, +) from dev.tests_common.test_utils.gcp_system_helpers import ( MARKETING_DAG_FOLDER, GoogleSystemTest, provide_gcp_context, ) +from dev.tests_common.test_utils.system_tests import get_test_run from providers.tests.google.cloud.utils.gcp_authenticator import GCP_BIGQUERY_KEY, GMP_KEY # Requires the following scope: @@ -50,12 +55,14 @@ def teardown_method(self): @provide_gcp_context(GMP_KEY, scopes=SCOPES) def test_run_example_dag(self): - self.run_dag("example_display_video", MARKETING_DAG_FOLDER) + self.run_dag("example_display_video", MARKETING_DAG_FOLDER) # this dag does not exist? @provide_gcp_context(GMP_KEY, scopes=SCOPES) def test_run_example_dag_misc(self): - self.run_dag("example_display_video_misc", MARKETING_DAG_FOLDER) + run = get_test_run(dag_example_display_video_misc) + run() @provide_gcp_context(GMP_KEY, scopes=SCOPES) def test_run_example_dag_sdf(self): - self.run_dag("example_display_video_sdf", MARKETING_DAG_FOLDER) + run = get_test_run(dag_example_display_video_sdf) + run() diff --git a/providers/tests/system/google/cloud/dataprep/example_dataprep.py b/providers/tests/system/google/cloud/dataprep/example_dataprep.py index 9f603f43fb1..cdc736a41c6 100644 --- a/providers/tests/system/google/cloud/dataprep/example_dataprep.py +++ b/providers/tests/system/google/cloud/dataprep/example_dataprep.py @@ -313,6 +313,7 @@ def delete_connection(connection_id: str) -> None: # when "tearDown" task with trigger rule is part of the DAG list(dag.tasks) >> watcher() + from dev.tests_common.test_utils.system_tests import get_test_run # noqa: E402 # Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) diff --git a/tests/cli/commands/test_dag_command.py b/tests/cli/commands/test_dag_command.py index 00364794f1d..f0c9a18c1a5 100644 --- a/tests/cli/commands/test_dag_command.py +++ b/tests/cli/commands/test_dag_command.py @@ -322,7 +322,7 @@ def test_cli_backfill_ignore_first_depends_on_past(self, mock_run): We just check we call dag.run() right. The behaviour of that kwarg is tested in test_jobs """ - dag_id = "test_dagrun_states_deadlock" + dag_id = "example_bash_operator" run_date = DEFAULT_DATE + timedelta(days=1) args = [ "dags", diff --git a/tests/core/test_example_dags_system.py b/tests/core/test_example_dags_system.py index c60b7325b12..bd34d9bb159 100644 --- a/tests/core/test_example_dags_system.py +++ b/tests/core/test_example_dags_system.py @@ -17,16 +17,128 @@ # under the License. from __future__ import annotations +from datetime import timedelta + +import pendulum import pytest +from sqlalchemy import select + +from airflow.models import DagRun +from airflow.operators.python import PythonOperator +from airflow.utils.module_loading import import_string +from airflow.utils.state import DagRunState +from airflow.utils.trigger_rule import TriggerRule +from dev.tests_common.test_utils.system_tests import get_test_run from dev.tests_common.test_utils.system_tests_class import SystemTest +def fail(): + raise ValueError + + +def get_dag_success(dag_maker): + with dag_maker( + dag_id="test_dagrun_states_success", + schedule=timedelta(days=1), + ) as dag: + dag4_task1 = PythonOperator( + task_id="test_dagrun_fail", + python_callable=fail, + ) + dag4_task2 = PythonOperator( + task_id="test_dagrun_succeed", trigger_rule=TriggerRule.ALL_FAILED, python_callable=print + ) + dag4_task2.set_upstream(dag4_task1) + return dag + + +def get_dag_fail(dag_maker): + with dag_maker( + dag_id="test_dagrun_states_fail", + schedule=timedelta(days=1), + ) as dag: + dag3_task1 = PythonOperator(task_id="to_fail", python_callable=fail) + dag3_task2 = PythonOperator(task_id="to_succeed", python_callable=print) + dag3_task2.set_upstream(dag3_task1) + return dag + + +def get_dag_fail_root(dag_maker): + with dag_maker( + dag_id="test_dagrun_states_root_fail", + schedule=timedelta(days=1), + ) as dag: + PythonOperator(task_id="test_dagrun_succeed", python_callable=print) + PythonOperator( + task_id="test_dagrun_fail", + python_callable=fail, + ) + return dag + + @pytest.mark.system("core") class TestExampleDagsSystem(SystemTest): @pytest.mark.parametrize( - "dag_id", + "module", ["example_bash_operator", "example_branch_operator", "tutorial_dag", "example_dag_decorator"], ) - def test_dag_example(self, dag_id): - self.run_dag(dag_id=dag_id) + def test_dag_example(self, module): + test_run = import_string(f"airflow.example_dags.{module}.test_run") + test_run() + + @pytest.mark.parametrize( + "factory, expected", + [ + (get_dag_fail, "failed"), + (get_dag_fail_root, "failed"), + (get_dag_success, "success"), + ], + ) + def test_dag_run_final_state(self, factory, expected, dag_maker, session): + """ + These tests are migrated tests that were added in PR #1289 + which was fixing issue #1225. + + I would be very surprised if these things were not covered elsewhere already + but, just in case, I'm migrating them to system tests. + """ + dag = factory(dag_maker) + run = get_test_run(dag) + with pytest.raises(AssertionError, match="The system test failed"): + run() + dr = session.scalar(select(DagRun)) + assert dr.state == "failed" + + def test_dag_root_task_start_date_future(self, dag_maker, session): + """ + These tests are migrated tests that were added in PR #1289 + which was fixing issue #1225. + + This one tests what happens when there's a dag with a root task with future start date. + + The dag should run, but no TI should be created for the task where start date in future. + """ + exec_date = pendulum.datetime(2021, 1, 1) + fut_start_date = pendulum.datetime(2021, 2, 1) + with dag_maker( + dag_id="dagrun_states_root_future", + schedule=timedelta(days=1), + catchup=False, + ) as dag: + PythonOperator( + task_id="current", + python_callable=lambda: print("hello"), + ) + PythonOperator( + task_id="future", + python_callable=lambda: print("hello"), + start_date=fut_start_date, + ) + run = get_test_run(dag, execution_date=exec_date) + run() + dr = session.scalar(select(DagRun)) + tis = dr.task_instances + assert dr.state == DagRunState.SUCCESS + assert len(tis) == 1 + assert tis[0].task_id == "current" diff --git a/tests/dags/test_future_start_date.py b/tests/dags/test_future_start_date.py new file mode 100644 index 00000000000..dadfbff600f --- /dev/null +++ b/tests/dags/test_future_start_date.py @@ -0,0 +1,41 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +from datetime import timedelta + +import pendulum + +from airflow.models.dag import DAG +from airflow.operators.empty import EmptyOperator +from airflow.operators.python import PythonOperator + +exec_date = pendulum.datetime(2021, 1, 1) +fut_start_date = pendulum.datetime(2021, 2, 1) +with DAG( + dag_id="test_dagrun_states_root_future", + schedule=timedelta(days=1), + catchup=True, + start_date=exec_date, +) as dag: + EmptyOperator(task_id="current") + PythonOperator( + task_id="future", + python_callable=lambda: print("hello"), + start_date=fut_start_date, + ) diff --git a/tests/dags/test_issue_1225.py b/tests/dags/test_issue_1225.py deleted file mode 100644 index 96a3ad15626..00000000000 --- a/tests/dags/test_issue_1225.py +++ /dev/null @@ -1,149 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -""" -DAG designed to test what happens when a DAG with pooled tasks is run -by a BackfillJob. -Addresses issue #1225. -""" - -from __future__ import annotations - -from datetime import datetime, timedelta - -from airflow.models.dag import DAG -from airflow.operators.empty import EmptyOperator -from airflow.operators.python import PythonOperator -from airflow.utils.trigger_rule import TriggerRule - -DEFAULT_DATE = datetime(2016, 1, 1) -default_args = dict(start_date=DEFAULT_DATE, owner="airflow") - - -def fail(): - raise ValueError("Expected failure.") - - -# DAG tests backfill with pooled tasks -# Previously backfill would queue the task but never run it -dag1 = DAG( - dag_id="test_backfill_pooled_task_dag", - schedule=timedelta(days=1), - default_args=default_args, -) -dag1_task1 = EmptyOperator( - task_id="test_backfill_pooled_task", - dag=dag1, - pool="test_backfill_pooled_task_pool", -) - -# dag2 has been moved to test_prev_dagrun_dep.py - -# DAG tests that a Dag run that doesn't complete is marked failed -dag3 = DAG( - dag_id="test_dagrun_states_fail", - schedule=timedelta(days=1), - default_args=default_args, -) -dag3_task1 = PythonOperator(task_id="test_dagrun_fail", dag=dag3, python_callable=fail) -dag3_task2 = EmptyOperator( - task_id="test_dagrun_succeed", - dag=dag3, -) -dag3_task2.set_upstream(dag3_task1) - -# DAG tests that a Dag run that completes but has a failure is marked success -dag4 = DAG( - dag_id="test_dagrun_states_success", - schedule=timedelta(days=1), - default_args=default_args, -) -dag4_task1 = PythonOperator( - task_id="test_dagrun_fail", - dag=dag4, - python_callable=fail, -) -dag4_task2 = EmptyOperator(task_id="test_dagrun_succeed", dag=dag4, trigger_rule=TriggerRule.ALL_FAILED) -dag4_task2.set_upstream(dag4_task1) - -# DAG tests that a Dag run that completes but has a root failure is marked fail -dag5 = DAG( - dag_id="test_dagrun_states_root_fail", - schedule=timedelta(days=1), - default_args=default_args, -) -dag5_task1 = EmptyOperator( - task_id="test_dagrun_succeed", - dag=dag5, -) -dag5_task2 = PythonOperator( - task_id="test_dagrun_fail", - dag=dag5, - python_callable=fail, -) - -# DAG tests that a Dag run that is deadlocked with no states is failed -dag6 = DAG( - dag_id="test_dagrun_states_deadlock", - schedule=timedelta(days=1), - default_args=default_args, -) -dag6_task1 = EmptyOperator( - task_id="test_depends_on_past", - depends_on_past=True, - dag=dag6, -) -dag6_task2 = EmptyOperator( - task_id="test_depends_on_past_2", - depends_on_past=True, - dag=dag6, -) -dag6_task2.set_upstream(dag6_task1) - - -# DAG tests that a Dag run that doesn't complete but has a root failure is marked running -dag8 = DAG( - dag_id="test_dagrun_states_root_fail_unfinished", - schedule=timedelta(days=1), - default_args=default_args, -) -dag8_task1 = EmptyOperator( - task_id="test_dagrun_unfinished", # The test will unset the task instance state after - # running this test - dag=dag8, -) -dag8_task2 = PythonOperator( - task_id="test_dagrun_fail", - dag=dag8, - python_callable=fail, -) - -# DAG tests that a Dag run that completes but has a root in the future is marked as success -dag9 = DAG( - dag_id="test_dagrun_states_root_future", - schedule=timedelta(days=1), - default_args=default_args, -) -dag9_task1 = EmptyOperator( - task_id="current", - dag=dag9, -) -dag9_task2 = EmptyOperator( - task_id="future", - dag=dag9, - start_date=DEFAULT_DATE + timedelta(days=1), -) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index f1bb6f17d05..dead9be8623 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -32,7 +32,6 @@ from airflow.cli import cli_parser from airflow.exceptions import ( AirflowException, - AirflowTaskTimeout, BackfillUnfinished, DagConcurrencyLimitReached, NoAvailablePoolSlot, @@ -54,7 +53,6 @@ from airflow.utils import timezone from airflow.utils.session import create_session from airflow.utils.state import DagRunState, State, TaskInstanceState -from airflow.utils.timeout import timeout from airflow.utils.trigger_rule import TriggerRule from airflow.utils.types import DagRunType from tests.listeners import dag_listener @@ -1070,34 +1068,6 @@ def test_backfill_ordered_concurrent_execute(self, dag_maker, mock_executor): ], ] - def test_backfill_pooled_tasks(self): - """ - Test that queued tasks are executed by BackfillJobRunner - """ - session = settings.Session() - pool = Pool(pool="test_backfill_pooled_task_pool", slots=1, include_deferred=False) - session.add(pool) - session.commit() - session.close() - - dag = self.dagbag.get_dag("test_backfill_pooled_task_dag") - dag.clear() - - job = Job() - job_runner = BackfillJobRunner(job=job, dag=dag, start_date=DEFAULT_DATE, end_date=DEFAULT_DATE) - - # run with timeout because this creates an infinite loop if not - # caught - try: - with timeout(seconds=20): - run_job(job=job, execute_callable=job_runner._execute) - except AirflowTaskTimeout: - logger.info("Timeout while waiting for task to complete") - run_id = f"backfill__{DEFAULT_DATE.isoformat()}" - ti = TI(task=dag.get_task("test_backfill_pooled_task"), run_id=run_id) - ti.refresh_from_db() - assert ti.state == State.SUCCESS - @pytest.mark.parametrize("ignore_depends_on_past", [True, False]) def test_backfill_depends_on_past_works_independently_on_ignore_depends_on_past( self, ignore_depends_on_past, mock_executor diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index a5748ebaeef..639a3528fad 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -43,7 +43,8 @@ from airflow.callbacks.database_callback_sink import DatabaseCallbackSink from airflow.callbacks.pipe_callback_sink import PipeCallbackSink from airflow.dag_processing.manager import DagFileProcessorAgent -from airflow.exceptions import AirflowException, RemovedInAirflow3Warning +from airflow.decorators import task +from airflow.exceptions import AirflowException from airflow.executors.base_executor import BaseExecutor from airflow.executors.executor_constants import MOCK_EXECUTOR from airflow.executors.executor_loader import ExecutorLoader @@ -2828,128 +2829,33 @@ def test_do_not_schedule_removed_task(self, dag_maker): assert [] == res - @provide_session - def evaluate_dagrun( - self, - dag_id, - expected_task_states, # dict of task_id: state - dagrun_state, - run_kwargs=None, - advance_execution_date=False, - session=None, - ): - """ - Helper for testing DagRun states with simple two-task DAGs. - This is hackish: a dag run is created but its tasks are - run by a backfill. - """ - - # todo: AIP-78 remove along with DAG.run() - # this only tests the backfill job runner, not the scheduler - - if run_kwargs is None: - run_kwargs = {} - - dag = self.dagbag.get_dag(dag_id) - dagrun_info = dag.next_dagrun_info(None) - assert dagrun_info is not None - data_interval = dag.infer_automated_data_interval(DEFAULT_LOGICAL_DATE) - triggered_by_kwargs = {"triggered_by": DagRunTriggeredByType.TEST} if AIRFLOW_V_3_0_PLUS else {} - dr = dag.create_dagrun( - run_type=DagRunType.SCHEDULED, - execution_date=dagrun_info.logical_date, - state=None, - session=session, - data_interval=data_interval, - **triggered_by_kwargs, - ) - - if advance_execution_date: - # run a second time to schedule a dagrun after the start_date - dr = dag.create_dagrun( - run_type=DagRunType.SCHEDULED, - execution_date=dr.data_interval_end, - state=None, - session=session, - data_interval=data_interval, - **triggered_by_kwargs, - ) - ex_date = dr.execution_date - - for tid, state in expected_task_states.items(): - if state == State.FAILED: - self.null_exec.mock_task_fail(dag_id, tid, dr.run_id) - - try: - dag = DagBag().get_dag(dag.dag_id) - # This needs a _REAL_ dag, not the serialized version - assert not isinstance(dag, SerializedDAG) - # TODO: Can this be replaced with `self.run_scheduler_until_dagrun_terminal. `dag.run` isn't - # great to use here as it uses BackfillJobRunner! - for _ in _mock_executor(self.null_exec): - dag.run(start_date=ex_date, end_date=ex_date, **run_kwargs) - except AirflowException: - pass - - # load dagrun - dr = DagRun.find(dag_id=dag_id, execution_date=ex_date, session=session) - dr = dr[0] - dr.dag = dag - - assert dr.state == dagrun_state - - # test tasks - for task_id, expected_state in expected_task_states.items(): - ti = dr.get_task_instance(task_id) - assert ti.state == expected_state - - def test_dagrun_fail(self): + @pytest.mark.parametrize( + "ti_states, run_state", + [ + (["failed", "success"], "failed"), + (["success", "success"], "success"), + ], + ) + def test_dagrun_state_correct(self, ti_states, run_state, dag_maker, session): """ DagRuns with one failed and one incomplete root task -> FAILED """ - # todo: AIP-78 remove along with DAG.run() - # this only tests the backfill job runner, not the scheduler - with pytest.warns(RemovedInAirflow3Warning): - self.evaluate_dagrun( - dag_id="test_dagrun_states_fail", - expected_task_states={ - "test_dagrun_fail": State.FAILED, - "test_dagrun_succeed": State.UPSTREAM_FAILED, - }, - dagrun_state=State.FAILED, - ) + with dag_maker(): - def test_dagrun_success(self): - """ - DagRuns with one failed and one successful root task -> SUCCESS - """ - # todo: AIP-78 remove along with DAG.run() - # this only tests the backfill job runner, not the scheduler - with pytest.warns(RemovedInAirflow3Warning): - self.evaluate_dagrun( - dag_id="test_dagrun_states_success", - expected_task_states={ - "test_dagrun_fail": State.FAILED, - "test_dagrun_succeed": State.SUCCESS, - }, - dagrun_state=State.SUCCESS, - ) + @task + def my_task(): ... - def test_dagrun_root_fail(self): - """ - DagRuns with one successful and one failed root task -> FAILED - """ - # todo: AIP-78 remove along with DAG.run() - # this only tests the backfill job runner, not the scheduler - with pytest.warns(RemovedInAirflow3Warning): - self.evaluate_dagrun( - dag_id="test_dagrun_states_root_fail", - expected_task_states={ - "test_dagrun_succeed": State.SUCCESS, - "test_dagrun_fail": State.FAILED, - }, - dagrun_state=State.FAILED, - ) + for _ in ti_states: + my_task() + dr = dag_maker.create_dagrun(state="running", triggered_by=DagRunTriggeredByType.TIMETABLE) + for idx, state in enumerate(ti_states): + dr.task_instances[idx].state = state + session.commit() + scheduler_job = Job(executor=self.null_exec) + self.job_runner = SchedulerJobRunner(job=scheduler_job) + self.job_runner.processor_agent = mock.MagicMock() + self.job_runner._do_scheduling(session) + assert session.query(DagRun).one().state == run_state def test_dagrun_root_after_dagrun_unfinished(self, mock_executor): """ @@ -2963,57 +2869,15 @@ def test_dagrun_root_after_dagrun_unfinished(self, mock_executor): dag.sync_to_db() scheduler_job = Job() - self.job_runner = SchedulerJobRunner(job=scheduler_job, num_runs=1, subdir=dag.fileloc) + self.job_runner = SchedulerJobRunner(job=scheduler_job, num_runs=2, subdir=dag.fileloc) run_job(scheduler_job, execute_callable=self.job_runner._execute) - first_run = DagRun.find(dag_id=dag_id, execution_date=DEFAULT_DATE)[0] + first_run = DagRun.find(dag_id=dag_id)[0] ti_ids = [(ti.task_id, ti.state) for ti in first_run.get_task_instances()] assert ti_ids == [("current", State.SUCCESS)] assert first_run.state in [State.SUCCESS, State.RUNNING] - def test_dagrun_deadlock_ignore_depends_on_past_advance_ex_date(self): - """ - DagRun is marked a success if ignore_first_depends_on_past=True - - Test that an otherwise-deadlocked dagrun is marked as a success - if ignore_first_depends_on_past=True and the dagrun execution_date - is after the start_date. - """ - # todo: AIP-78 remove along with DAG.run() - # this only tests the backfill job runner, not the scheduler - with pytest.warns(RemovedInAirflow3Warning): - self.evaluate_dagrun( - dag_id="test_dagrun_states_deadlock", - expected_task_states={ - "test_depends_on_past": State.SUCCESS, - "test_depends_on_past_2": State.SUCCESS, - }, - dagrun_state=State.SUCCESS, - advance_execution_date=True, - run_kwargs=dict(ignore_first_depends_on_past=True), - ) - - def test_dagrun_deadlock_ignore_depends_on_past(self): - """ - Test that ignore_first_depends_on_past doesn't affect results - (this is the same test as - test_dagrun_deadlock_ignore_depends_on_past_advance_ex_date except - that start_date == execution_date so depends_on_past is irrelevant). - """ - # todo: AIP-78 remove along with DAG.run() - # this only tests the backfill job runner, not the scheduler - with pytest.warns(RemovedInAirflow3Warning): - self.evaluate_dagrun( - dag_id="test_dagrun_states_deadlock", - expected_task_states={ - "test_depends_on_past": State.SUCCESS, - "test_depends_on_past_2": State.SUCCESS, - }, - dagrun_state=State.SUCCESS, - run_kwargs=dict(ignore_first_depends_on_past=True), - ) - @pytest.mark.parametrize( "configs", [ @@ -3136,9 +3000,14 @@ def test_scheduler_multiprocessing(self, configs): Test that the scheduler can successfully queue multiple dags in parallel """ with conf_vars(configs): - dag_ids = ["test_start_date_scheduling", "test_dagrun_states_success"] + dag_ids = [ + "test_start_date_scheduling", + "test_task_start_date_scheduling", + ] for dag_id in dag_ids: dag = self.dagbag.get_dag(dag_id) + if not dag: + raise ValueError(f"could not find dag {dag_id}") dag.clear() scheduler_job = Job( diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 997ef06329f..67dc699fc3c 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -25,7 +25,6 @@ import re import weakref from datetime import timedelta -from importlib import reload from pathlib import Path from typing import TYPE_CHECKING from unittest import mock @@ -45,12 +44,8 @@ AirflowException, DuplicateTaskIdFound, ParamValidationError, - RemovedInAirflow3Warning, UnknownExecutorException, ) -from airflow.executors import executor_loader -from airflow.executors.local_executor import LocalExecutor -from airflow.executors.sequential_executor import SequentialExecutor from airflow.models.asset import ( AssetAliasModel, AssetDagRunQueue, @@ -2740,20 +2735,6 @@ def test_asset_expression(self, session: Session) -> None: ] } - @mock.patch("airflow.models.dag.run_job") - def test_dag_executors(self, run_job_mock): - # todo: AIP-78 remove along with DAG.run() - # this only tests the backfill job runner, not the scheduler - with pytest.warns(RemovedInAirflow3Warning): - dag = DAG(dag_id="test", schedule=None) - reload(executor_loader) - with conf_vars({("core", "executor"): "SequentialExecutor"}): - dag.run() - assert isinstance(run_job_mock.call_args_list[0].kwargs["job"].executor, SequentialExecutor) - - dag.run(local=True) - assert isinstance(run_job_mock.call_args_list[1].kwargs["job"].executor, LocalExecutor) - class TestQueries: def setup_method(self) -> None: diff --git a/tests/models/test_xcom_arg.py b/tests/models/test_xcom_arg.py index b161020d1fb..fbdd500661d 100644 --- a/tests/models/test_xcom_arg.py +++ b/tests/models/test_xcom_arg.py @@ -23,7 +23,6 @@ from airflow.providers.standard.operators.bash import BashOperator from airflow.utils.types import NOTSET -from dev.tests_common.test_utils.config import conf_vars from dev.tests_common.test_utils.db import clear_db_dags, clear_db_runs pytestmark = pytest.mark.db_test @@ -146,7 +145,6 @@ def test_xcom_not_iterable(self, dag_maker): @pytest.mark.system("core") class TestXComArgRuntime: - @conf_vars({("core", "executor"): "DebugExecutor"}) def test_xcom_pass_to_op(self, dag_maker): with dag_maker(dag_id="test_xcom_pass_to_op") as dag: operator = PythonOperator( @@ -161,9 +159,8 @@ def test_xcom_pass_to_op(self, dag_maker): task_id="assert_is_value_1", ) operator >> operator2 - dag.run() + dag.test() - @conf_vars({("core", "executor"): "DebugExecutor"}) def test_xcom_push_and_pass(self, dag_maker): def push_xcom_value(key, value, **context): ti = context["task_instance"] @@ -182,7 +179,7 @@ def push_xcom_value(key, value, **context): op_args=[xarg], ) op1 >> op2 - dag.run() + dag.test() @pytest.mark.skip_if_database_isolation_mode # Does not work in db isolation mode