From 1ed35823b5fb8ed300399fcb0d5619ee7235c42b Mon Sep 17 00:00:00 2001 From: qian Date: Sun, 27 Jun 2021 15:35:06 +0800 Subject: [PATCH 1/8] Introduce RESTARTING state --- airflow/jobs/base_job.py | 2 +- airflow/models/dagrun.py | 4 +++- airflow/models/taskinstance.py | 11 +++++++++-- airflow/utils/state.py | 9 +++++++++ airflow/www/static/css/graph.css | 4 ++++ airflow/www/static/css/tree.css | 4 ++++ airflow/www/static/js/graph.js | 2 +- tests/models/test_dag.py | 2 +- tests/www/views/test_views_base.py | 2 +- 9 files changed, 33 insertions(+), 7 deletions(-) diff --git a/airflow/jobs/base_job.py b/airflow/jobs/base_job.py index 18893f2110a4f..e9e696029da8b 100644 --- a/airflow/jobs/base_job.py +++ b/airflow/jobs/base_job.py @@ -202,7 +202,7 @@ def heartbeat(self, only_if_necessary: bool = False): session.merge(self) previous_heartbeat = self.latest_heartbeat - if self.state == State.SHUTDOWN: + if self.state in State.terminated_states: self.kill() # Figure out how long to sleep for diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index e2057d5a34af0..c8e1100c0835d 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -477,7 +477,9 @@ def task_instance_scheduling_decisions(self, session: Session = None) -> TISched schedulable_tis: List[TI] = [] changed_tis = False - tis = list(self.get_task_instances(session=session, state=State.task_states + (State.SHUTDOWN,))) + tis = list( + self.get_task_instances(session=session, state=State.task_states) + ) self.log.debug("number of tis tasks for %s: %s task(s)", self, len(tis)) for ti in tis: try: diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 6fb437b012b68..11fa590fc213d 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -158,7 +158,9 @@ def clear_task_instances( for ti in tis: if ti.state == State.RUNNING: if ti.job_id: - ti.state = State.SHUTDOWN + # If a task is cleared when running, set its state to RESTARTING so that + # the task is terminated and becomes eligible for retry. + ti.state = State.RESTARTING job_ids.append(ti.job_id) else: task_id = ti.task_id @@ -211,7 +213,7 @@ def clear_task_instances( from airflow.jobs.base_job import BaseJob for job in session.query(BaseJob).filter(BaseJob.id.in_(job_ids)).all(): - job.state = State.SHUTDOWN + job.state = State.RESTARTING if activate_dag_runs is not None: warnings.warn( @@ -1519,6 +1521,11 @@ def handle_failure_with_callback( def is_eligible_to_retry(self): """Is task instance is eligible for retry""" + if self.state == State.RESTARTING: + # If a task is cleared when running, it goes into RESTARTING state and is always + # eligible for retry + return True + return self.task.retries and self.try_number <= self.max_tries @provide_session diff --git a/airflow/utils/state.py b/airflow/utils/state.py index e95b4095da2b2..19a3c403cbe74 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -40,6 +40,7 @@ class TaskInstanceState(str, Enum): RUNNING = "running" # Task is executing SUCCESS = "success" # Task completed SHUTDOWN = "shutdown" # External request to shut down + RESTARTING = "restarting" # Cleared by user when running FAILED = "failed" # Task errored out UP_FOR_RETRY = "up_for_retry" # Task failed but has retries left UP_FOR_RESCHEDULE = "up_for_reschedule" # A waiting `reschedule` sensor @@ -84,6 +85,7 @@ class State: SCHEDULED = TaskInstanceState.SCHEDULED QUEUED = TaskInstanceState.QUEUED SHUTDOWN = TaskInstanceState.SHUTDOWN + RESTARTING = TaskInstanceState.RESTARTING UP_FOR_RETRY = TaskInstanceState.UP_FOR_RETRY UP_FOR_RESCHEDULE = TaskInstanceState.UP_FOR_RESCHEDULE UPSTREAM_FAILED = TaskInstanceState.UPSTREAM_FAILED @@ -105,6 +107,7 @@ class State: TaskInstanceState.RUNNING: 'lime', TaskInstanceState.SUCCESS: 'green', TaskInstanceState.SHUTDOWN: 'blue', + TaskInstanceState.RESTARTING: 'goldenrod', TaskInstanceState.FAILED: 'red', TaskInstanceState.UP_FOR_RETRY: 'gold', TaskInstanceState.UP_FOR_RESCHEDULE: 'turquoise', @@ -159,6 +162,7 @@ def color_fg(cls, state): TaskInstanceState.RUNNING, TaskInstanceState.SENSING, TaskInstanceState.SHUTDOWN, + TaskInstanceState.RESTARTING, TaskInstanceState.UP_FOR_RETRY, TaskInstanceState.UP_FOR_RESCHEDULE, ] @@ -182,6 +186,11 @@ def color_fg(cls, state): A list of states indicating that a task or dag is a success state. """ + terminated_states = frozenset([TaskInstanceState.SHUTDOWN, TaskInstanceState.RESTARTING]) + """ + A list of states indicating that a task has been terminated. + """ + class PokeState: """Static class with poke states constants used in smart operator.""" diff --git a/airflow/www/static/css/graph.css b/airflow/www/static/css/graph.css index f4c7b942c5a77..d1655b0129081 100644 --- a/airflow/www/static/css/graph.css +++ b/airflow/www/static/css/graph.css @@ -148,6 +148,10 @@ g.node.shutdown rect { stroke: blue; } +g.node.restarting rect { + stroke: goldenrod; +} + g.node.upstream_failed rect { stroke: orange; } diff --git a/airflow/www/static/css/tree.css b/airflow/www/static/css/tree.css index 05b2c81de9973..bf9a640023e4f 100644 --- a/airflow/www/static/css/tree.css +++ b/airflow/www/static/css/tree.css @@ -67,6 +67,10 @@ rect.shutdown { fill: blue; } +rect.restarting { + fill: goldenrod; +} + rect.upstream_failed { fill: orange; } diff --git a/airflow/www/static/js/graph.js b/airflow/www/static/js/graph.js index d3d913b6dd27a..5b1ffee5406e3 100644 --- a/airflow/www/static/js/graph.js +++ b/airflow/www/static/js/graph.js @@ -536,7 +536,7 @@ function getNodeState(nodeId, tis) { // In this order, if any of these states appeared in childrenStates, return it as // the group state. const priority = ['failed', 'upstream_failed', 'up_for_retry', 'up_for_reschedule', - 'queued', 'scheduled', 'sensing', 'running', 'shutdown', 'removed', + 'queued', 'scheduled', 'sensing', 'running', 'shutdown', 'restarting', 'removed', 'no_status', 'success', 'skipped']; return priority.find((state) => childrenStates.has(state)) || 'no_status'; diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 7cbda6690c8b6..726c07bfa1d9d 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -1432,7 +1432,7 @@ def test_clear_set_dagrun_state_for_parent_dag(self, dag_run_state): @parameterized.expand( [(state, State.NONE) for state in State.task_states if state != State.RUNNING] - + [(State.RUNNING, State.SHUTDOWN)] + + [(State.RUNNING, State.RESTARTING)] ) # type: ignore def test_clear_dag(self, ti_state_begin, ti_state_end: Optional[str]): dag_id = 'test_clear_dag' diff --git a/tests/www/views/test_views_base.py b/tests/www/views/test_views_base.py index 84a1ea6b13231..8f0f4f6a62ad9 100644 --- a/tests/www/views/test_views_base.py +++ b/tests/www/views/test_views_base.py @@ -58,7 +58,7 @@ def test_home(capture_templates, admin_client): val_state_color_mapping = ( 'const STATE_COLOR = {"failed": "red", ' '"null": "lightblue", "queued": "gray", ' - '"removed": "lightgrey", "running": "lime", ' + '"removed": "lightgrey", "restarting": "goldenrod", "running": "lime", ' '"scheduled": "tan", "sensing": "lightseagreen", ' '"shutdown": "blue", "skipped": "pink", ' '"success": "green", "up_for_reschedule": "turquoise", ' From d8b4f06b0ba88e612b06b89c18a38df6cd096bd7 Mon Sep 17 00:00:00 2001 From: Qian Yu Date: Wed, 7 Jul 2021 22:19:16 +0800 Subject: [PATCH 2/8] Add to UPDATING.md --- UPDATING.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/UPDATING.md b/UPDATING.md index 4777b0e10b794..09726e6f3ec35 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -133,6 +133,10 @@ If you are using DAGs Details API endpoint, use `max_active_tasks` instead of `c When marking a task success/failed in Graph View, its downstream tasks that are in failed/upstream_failed state are automatically cleared. +### Clearing a running task sets its state to RESTARTING + +Previously, clearing a running task sets its state to SHUTDOWN. The task gets killed and goes into FAILED state. After [#16681](https://github.com/apache/airflow/pull/16681), clearing a running task sets its state to RESTARTING. The task is eligible for retry without going into FAILED state. + ### Remove `TaskInstance.log_filepath` attribute This method returned incorrect values for a long time, because it did not take into account the different From bc605dbe3260ae32e45c0fa5d781fb5bfea2459c Mon Sep 17 00:00:00 2001 From: Qian Yu Date: Mon, 12 Jul 2021 17:46:49 +0800 Subject: [PATCH 3/8] Rename to terminating_states --- airflow/jobs/base_job.py | 2 +- airflow/utils/state.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/jobs/base_job.py b/airflow/jobs/base_job.py index e9e696029da8b..745f248fc4da0 100644 --- a/airflow/jobs/base_job.py +++ b/airflow/jobs/base_job.py @@ -202,7 +202,7 @@ def heartbeat(self, only_if_necessary: bool = False): session.merge(self) previous_heartbeat = self.latest_heartbeat - if self.state in State.terminated_states: + if self.state in State.terminating_states: self.kill() # Figure out how long to sleep for diff --git a/airflow/utils/state.py b/airflow/utils/state.py index 19a3c403cbe74..c28d835376cf7 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -186,7 +186,7 @@ def color_fg(cls, state): A list of states indicating that a task or dag is a success state. """ - terminated_states = frozenset([TaskInstanceState.SHUTDOWN, TaskInstanceState.RESTARTING]) + terminating_states = frozenset([TaskInstanceState.SHUTDOWN, TaskInstanceState.RESTARTING]) """ A list of states indicating that a task has been terminated. """ From 7061628739f58ae852122bf5f8548d8434602db9 Mon Sep 17 00:00:00 2001 From: Qian Yu Date: Wed, 14 Jul 2021 20:21:38 +0800 Subject: [PATCH 4/8] Document "restarting" state --- airflow/utils/state.py | 4 ++-- docs/apache-airflow/concepts/tasks.rst | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/airflow/utils/state.py b/airflow/utils/state.py index c28d835376cf7..c95c7d4f0ffac 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -39,8 +39,8 @@ class TaskInstanceState(str, Enum): QUEUED = "queued" # Executor has enqueued the task RUNNING = "running" # Task is executing SUCCESS = "success" # Task completed - SHUTDOWN = "shutdown" # External request to shut down - RESTARTING = "restarting" # Cleared by user when running + SHUTDOWN = "shutdown" # External request to shut down (e.g. marked failed when running) + RESTARTING = "restarting" # External request to restart (e.g. cleared when running) FAILED = "failed" # Task errored out UP_FOR_RETRY = "up_for_retry" # Task failed but has retries left UP_FOR_RESCHEDULE = "up_for_reschedule" # A waiting `reschedule` sensor diff --git a/docs/apache-airflow/concepts/tasks.rst b/docs/apache-airflow/concepts/tasks.rst index d4a6608c7c539..f481baba60485 100644 --- a/docs/apache-airflow/concepts/tasks.rst +++ b/docs/apache-airflow/concepts/tasks.rst @@ -70,6 +70,8 @@ The possible states for a Task Instance are: * ``queued``: The task has been assigned to an Executor and is awaiting a worker * ``running``: The task is running on a worker (or on a local/synchronous executor) * ``success``: The task finished running without errors +* ``shutdown``: The task was externally requested to shut down when it was running +* ``restarting``: The task was externally requested to restart when it was running * ``failed``: The task had an error during execution and failed to run * ``skipped``: The task was skipped due to branching, LatestOnly, or similar. * ``upstream_failed``: An upstream task failed and the :ref:`Trigger Rule ` says we needed it From c85f06957315b26a495c3296f70097645c2757da Mon Sep 17 00:00:00 2001 From: Qian Yu Date: Wed, 14 Jul 2021 20:29:08 +0800 Subject: [PATCH 5/8] Update task_lifecycle_diagram.png --- .../img/task_lifecycle_diagram.png | Bin 135225 -> 58659 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/docs/apache-airflow/img/task_lifecycle_diagram.png b/docs/apache-airflow/img/task_lifecycle_diagram.png index ad0bd9ecf49ec1d3692a1fed05f0fbb98b06af53..810942fc7400172ccc40074a3ef2a2b3fc9a9362 100644 GIT binary patch literal 58659 zcmeFYXIN9+_Ag3Pz=nV*Hb4Y?L4gpu(n1Xc2ni%KDG9xWmOv7zf?@##=^!A|yMlmp zLm%rQrq<2P>W>1rH4aOwaJ z4b5RqxGI{4h5<)IL)Xi+AKdZfYVV_=Iq;dNW<+!$+d4Sg(C~|^{JP?YLhM~ViTvWK z{7@*)%}vDG0cYodbMX`**bu=j@Z81K+QHVr#`;$qC%&d!GLyF*D42al&+Wb%zaP92iWCQ+b*jPEa?!BU7 z@8RkK?x@R(i9kdoB&4AtVqy~Dfj-U_=i%^=ZuaJ6L&Vwrnvbi2vACTo%mzcIpd_4i zY*nQ#aKGl^W#i%L;7a(_OiV&VMg-8X_cD>4w&Kglc$u%heyB3hg@|`?#A%Dm=-X*p85qORcon>%GztZSNT?a^vBp|UoPgBA%P3oU zB9!elRo$_6US8ft6pEU?n~ICEv5JAAleLDvj)96b79yb~uAw6-N*WvJszUU=Ol0sx zBQYHfKsAA^si|#5k|uzLR5Bh)mIW6GWh*NcTO%q;%LGHBXsT);+}w>IDio|69I5Py z)kKO@93{caYTDW^D7QUnQkTK$c%g`nu5Ku_wzE40tFMoga1z(Fbw-2!_MTHTNUjJ6 zYrH2~6Dot3hF}~`ybNT~(pWT7pXlN2ZcQ|l!AiQj;nXDT5&96vJ%UiG7AQ@6jjIl_vk&CR9x3e)p#@-8Mgz}IfK#5K;8H}VZAWhGi zYOkpYA?w+JFS1k(HH^5HiWbC1O43=+LCru$##YA3Q5j)GP<3|DP?G@LTA65KZHLn` z#DKl4?4>8}VGnU|aVL`u^wf;Jq-;INF4oF$2YX$6J6%1<9!E%$4%(82p5V#^d{sxt z5U@}qtnMChP)}o9HN2ZHn3y_7%AKg=WrcuvNUFFprvfpaQ2RNcB)ny6la*Nqnk02Xk{Wz(j(#xD4teW zS*(Pex3~>Vk|?9;tYwU*kQ@LL{VK*%L?bFtgWem zv<54s2kyXal}SXf|JBuyvN9Sn9^hrTjGZjf&_Umhg!CXo4ZPH3T!?55Cj!O=Y2-vv zQE}6C)$p{o_foc#b%xu+biFk(M0YPk6>DutxQeZol#U0*)d-KkIFQgTo@8-JG}YSL z32)?#b%(h+I@)`=8n~g<&?ZI_5@e!>1cE9CbCz(_a)ueXXcMiGU{63v9ynutXLUOR zgbvG*RyrQ*&$?%)v0)6qMNdqfvcXHI$jFx zq$-PZ&=;dR0v5=)+k4vC8flTS-cH6QDq8j=Ju4L{h?9z>v75d!QVox?kut&~NhFlC zwyeFep1qikrz%d*9&9{iO)~f(E9+S!-6*zpM#kdUJ;_GOq7a^vQu?+8ypti$jclZ^ z2b7pP*g(p%IJCZ=jkm6co11Bh6n`-YV?J8jifta8nI5Bs$jk^O`O$=>> zK`V>Pm|$Tt`Xmi8F-?rJvMv^?FNQ_p47A*#(j=<77)o7CpXiKI#p|lNd#e$w2#%iG zMlRxFHrn3CYE)Zy8CjrLm5I*QSg04AVrXKA(YKRvflKS#xxpOwgh*TmYD^)jJ30_B zcsoskw3e)ujtQ0MW$!7et80go!o%#7p}yfk>=rin(lQ5>bzU9Ai?vC5t*P?W9=)C)-kLaGjf*t^3h1SinW&P_{0%GOvy z!`l;UNOBX0y0}X_L1i?Qbzv@8HC-%9QW^szx!TxD=%B6kHYo*$*0r~mRW?AWO6#k+ z>$#|EVG&MVVkDxrq?o6Tl(Zccs||ywLp_IFKGz^&RN?^3XLE;YT(2isk&%S30Z3iO`xWgDG+6#om7w- z?#faWR}8qL?O{NqNEqrEli?UIH?)hjn!cflE7nC{QW_yEhINGlmq7xq@9F|Hkse%( zsAEWk+exWmHAwnK4scsr6;%f@l#78hOiEXoin7x{LG<97KwBy!RjtHa3~V*ECEXrA#1D zRnb_2orez0!&z4oZKI(nwI{*aHd>C-cG@~rl8lUjr#|pRl&vUQcm&eP$_wMI>82%y zfI3QgOR9LdT0=;(wz5zRf@Dl3=-M0DqENaBsv}<9+E~Lu+MYx)Bp_W>Y;|o+U`{GF z8d4q_C>&AQ!`WJFPX)tm?ZxpBZ>R^3XhcCm+-!)>ZYt8ioH{_gRf*~zDjL=(eK^k8 z9l0k4I?8BwDHNV)ua0-sz#HqSVQh_Ipcf2ISy{(cOAEMg$|fjX7}7>p4{Gm-MyZM; zoxFfj*3z_wm|*dmNLhU;NjDXen1MEmWTdNUjrNlC!okq46niAf-dJB2WoYdH)zE`d z)U9N7FgD^2d-f7*Lcs4yFxA10Y9y=bz2`6O{R00_{g1Ep)A0h|f4j5dYEMNA3~6Y1 zXf#z}24wTE$&6%-+E73}-&4K&G%$w)2iD5hsw|&ll$i1J7puP%<^T2h4hP2o-SNkU zj~_mAqCc2pNqfaGNK#4=bmAlJ_D>)(3m6hI|m3E!JUCRq!6CYez?%3=| zZl}z*j~x1RorZyjhL%Z*hVE#op)ajpaW``!yIh6A#WOK%3*gkwL?(!u^M& zf18%VQKiNoGzmigjSL#Ln+pGv51<<#K*S;~6!Txn;5+2|ZzKF6=Z6vv^BBE#-hU+n z&bjd~PXD9DLLd!W6Jv`3?f+N~&?o}3^51Ure59x43uTGOrTe$_``s&8DEdDn{geLx zyHJ0)@_!fVkKFu!?WhJx<~~tHMgQs6n2X=@3ksZdHm<{Xu3V{pgvrNpb8v8ku|^7} zXh>S4>kA4B()#-`n`*qQ8-Y^|WBEMi&!_W>i_fO#<>iTaOexK81~QGod~z38MMj&0 z4yMef_Gb*Lg)-J07t#oyD)!$l&1%bBv&$5`5JFny1-4~54 zCK*BVN$C%*Xo#Yv+%0KYg%4AkWFBS31+n@?Y|M4PqptRKAErWsU$?bglarRNuzY!o zRn~WNEk-x_^7?^0t$6moE?tVw)z5Bii9Fw?g+N@0i;jNqiKDnZO;kQ5G6oaTm#eS4 zJaYB5s1#q?)pqOnFs$8+-%ugcp7*lh!S7187(M&RisZXsRB+A7@ZSEv+N5{nI z2nh-<7H~rp7vz$e4L>e#1uXJ>(xz-GD>l>bh>(-othK(Ea`Ulg4-}J>8>i_Fi_L2f z2q04)j7c7CMTRTEV6egV1aWy*vCGdL-)50un_TiUjIT3aFB_id+J54!L%DH*_vT{5 z@``bO!`Pywa;a1yBZSKc5ofnAVew(X2s>*!`6r3>e ziy!lgThCh^EGV?KmU|q_HG06L#QOHi*TK@7l9FB%#TFp-!v5t#OL)KTM{0MMD9e*& z-`+h*FNEyz&pecIh5laDBy7=qg`TDHsk!w>{?J>h#9X#GQwAthd!S^X{$>s5OvzLV z0mfruyfu@}>&<0V%njj^8O;A0&KP;8=IM$PORcloZmX6E8qhpTuVy~aq#wMPcyW&-1O1M>lq zLs&THg{s~r*pTlun}G3=HQK|2OMHmC)^cu-Mc!q!L5La$byL=&*Ci;!Q!tozQ;h@LnY4+j+%uzB|%#B%St69w8oQ7ADCNz(; zHT8sBddn#_$_yQjfD{RI;zv@n%OfGj#m6mBu~Uq4KAUluog^s`u5OkBo96oKa?^}E z`%+bP;PHGP7kl3^d{rQx!(E+oioIYA=hV1z{rw{FE9|9}8lNS~+;zVO9aL{LYLD<& zGfv@NSeUc;QqVvCoUEkyIQH!Bfu)DsKTi0tNuI;q4`@b2LRJDdn2l1b&*9F@xq;5u zL115g-gy5POZT5N9a&?T9gd;=@OpfLyhqP8V<&KzWTJ#B=*l*`UukhxlP-GOOEJndsQ#$z0y@MuQPAvEEN|lNA+2aJ}g{S5s>G|Yssv1~I4}F|B#;w0Cu4O0^GW70 z$H}j+=3ZMT&B52$#;r7e@j3pADds1io%x%aJpi8-)KkeAc>K%3#a1fQJKt&L#%Du= zmyjh9kZ>;sFh8(%1VFi@`|+_g>yohGC-LWY@07?Xe*83C|He4~br9HmmcmzdIL2X$ z3!*>x9F-gSXz#@}30QtlyI97SbuCE%&B3}}i8l)p>}o&ikeF~Te|4Yc)sE8pqUUda zDXe|7#CLIlEh{gt^>JHktJa-QRixPr%_srQ7}UVs{QP{iT?)#}_xaGCG`_uZ6S=R1 zt?5E{S>B@KCC#V&KiBMb*7?lh(mk54WDaT;?yQFjkEe>?nTXj}|LW3I?u#oeSeL|>9noU~n7BAmUCa9ze?_JrUKvhB zmFmYu%`dSiZn-rY)o*;cIA{}RMjXYoLsy>XIrL>g?K;m45~}Au7tQW$&c{iAAgh>@ zKizm{VKt^+JqG*2-~J$~T5C4Te`{IR?v=^R@)G)V%c)CNT}L`HEGl9>wlDA>{%|__ zG+P8@HH=Zad<^cas~cXk`K`KSm{9%7z2bWXpUmKqku9F%xmX?Oc(s02YoB6KG(y3M zKsa4yVJb@+e(SMXz_HZw;J|X})V&eywH8+aQ~4)<5vE5|doka3t9E4JBRBSaRBu}j zdU}c_3z;gCokGpKWLkcSCP7<{fu$+v=@-V^Q7msSvVUcK?x;%F!-HcBXhNJ404z4l zyp=Qe%uJQA6%@K?-1yWfMbIMUo!W=127)L2A73rEf3I3JnqBxcWwafB&nO?i6%Ap> zG>%I|Zl|!s1n1Us^WfT(M1^p5+iRwSew%Zr{Z!tLe2NIp&E{rMY4kBH!4iHDt!S4J zHZkp@uH;u&N0{@cnh-nD7k7s$b<(*$JHJWNKFJJ^-)N#27Cti(atjUglk5w?4kREGzmIq#DPNy7XQ+f6dBmbMmF%73voVuB8R*a+FXr(vkg>JE zxOBXHm=KlhHKfT!ojy0{T-q_7a_!q7Bu8OmI(_oPojV6)=brYBf5|Ui^F0ldb19xp zVc=`O!Rz4}cB_YKhnAOprJY1}<58C?JH#jMD^9e0 z+cm?R`)`yNPCPR6wCO96)Aj zgq`9CYgqk!b;PslwT|?A>vFNP)akTI?+p#jX8N#CdHF@q@9%xTzc2Ba&^jsQ9B<6b z9s{`?d)4DLzb)G9C96ifyUOgFLX80n|5~}#H~PJWrqVOLnRlaOW2+vsWu^EXJ;c2H zycYxgG51KZMo0Zy@`Vd4&cgIrm!jqo1FccMms_MNb!>J|#KOBqsz$R-;AUgD>-=Y_ zyStvYPvn;4j@ZtQm5k=KlNaU{C+-w|a7v6hrxt_pm^4*evROP;xfE=b` z@kXl4uMBe1OL4I%+-*sw9}pD|ozHWXV|ozW^ZiLUt+2_{h_Uc1o$dQRfZY^7KL#AR z<})W0ete`C|4_5ii@5n_bH3j7!+=SM`IAY;s{6Xl32Jwbzt+5v%y(WRJbHJte%I5N zU;E)C<6+YK3rA*o7`+?`3=UE4hZ4L^4A%*-Oa@DI*az2g)!590#r=Ti(qT%gAESzg+L?`G}9I_zOnP?zHFRl>7R`v-wbK5_d(3dHk<(fpKA7mLf# z(@5aAzpk80xmYpkS6yIIrX^Cp<*~jwx+{|EvvhleJQgi4)@c2Zt$Pzow73M!onQnRbX`QEXh&wBN) zp2yu+e^y76OrEixs+ui|I_o`x+F333Pw>C|`paNx_Mo}jD-Oe!+2VQ;?-BQqtsi6e zmL}pXT0ezwi!`6{42_i7cpR6IASiCvxfiQx)%$I?pC*T#^@=F%yo~5n*mOQAh7ZoX zRVyq&%bZTjaTi{l*z$H-`qQlzPS#4;%^IT*HeK?yhpVtNRDm{NTH-a^D=xlmT~Ui! zT4A|pgX^e-2y9dsyQ;=2vlAil3;TH)GU$r<0tUvrn|$sGf{} z9a}fK{HP!5^Ia}swaroXQz#GNv*>Ld2qv7)glOx-hgq7iv%&$FZ312t2*ko&*%D3!B%hmVmBz{jiHc1IAT~`UB#}9K2i5=D$lN^TDs?7vX)?(*kvX9bn+gB^?R|%;W@cFI zW}9d|C$lFo(3&Vz&&@fVjnRpT3EM-rwc^WNhP9f*&rV!FC<0;e+t8>kC}DGvl)~z!vQgenIeizO3gT41A$|v>PNRe^!*z1zkyvw7z*W{{IzwLNlTldU9 z4jr$>3tF6xY(CNvR~%Q&5%%ehyl18wHtXw;72flKD|HU3%M%u;OHb3?Q@D#M@nNxJe01Sm-BWWHuyHz zZyuglqHR7?QF&$spSC4fpFr7NyGBYNwshWE|B&p9Ke7C&>t3eQ#1~_J=uU*sd0e8+ zod*@3;U_JK*LvHwrKIU>M=Cgavm9;?ZV)?mwX7Szb1e1cnsumtd6Ahv(fIvN=0e5l zRP4S_z|mxSLw;yVgL&v&Jy$;ITIEN>HoJXhb{(c@)Wbr28e09W3b6w=dh0Xyv5b-u z^oMKnwTFiz@wMiuCsycF<=5V+>_5b5Ik}(Pcctehi|h4n#{1+S;j1-AZ}~l3mWmNd z6i-f{VYIVLp=6zmBXQ*{zbY&rYHc~bUF&zN-Ury4f}Aej%Qy422UtdTSF#oR*FAmx zLM6Z7mGtiSGZ=VK4(z+&+!4){py;M#=LvYcHS^mVb4J+qO9Z#iN^i*HeFu-nhm_X& zZYGBCmJLy7mMkR)Hum>RM=BVU5#A5UjVHS`GEbVHg^+BtSDbGzjx{GCn^&Uoax?) z)Yi8N{TDAg+%;?**)q;e=svqVB)U{@flb0lEhC2;wcj2$ED&c?SLX@^npzn4}`3KE_kY-Di9jo_oAOX}Cl@s_M{~$ZzG?&SvMU zHK0zbym@uP;?lLbvN&VG>_)ZWhyHT&wcpK`{kX4q75FT^Ix#7^VVR}OZ_b=PcGIt< z%o8w2 zb;GKWCI3v@gUdy8Nw@FxRc3o}m~HoFVq#a`Lx+9h(WBSC-&QyqDu;1Tk0WTHZ$G*? z{B1i=o3K@2Ky)nmxFS5hI4{(Gz^GP%^Nl99C=9+U*cKX0`4ST2F%z=mN5hUh@a+TUEDTyJj)nkEO%P*^jik8L9d<%M2Zqyt&9%FZT{w_am#XQ+c zJ@q^$)?Rif{{O zM`~!ZUjE^`wb*Ry6kyUZ{ZiZCyBo%sfGhwW+f>+*x4je1RXUo(`5`WTyW8JS-$-ZE z7$u5aWiATtDchcN8Zx}b-yQ}XSF=!DC-hy?H?g*DDb9cE@0c;{TQq#7i|ORH#QG1e z!uIl19hOf9C)0-`%O5=6>1z`&cM`60el|O%nA9&)<@<0gvrDzB(@e10x!>c(;E}2g z+Z_k9*=WdyQl{w<$KqfL36}YwdW`zqaI-f?G`;}h)R+I7_ z)LX>{>ddpW&6?T!5Ft-2ILhqN;(T9{@!7V|Y3=LaBs+!U+2s$@nW^E^GEuzDeMU3f z>D*7NHE)JqTs3WdJDJZW@YXA1+LtO4%=P7LOuFHG%!y^+sxI4(*)vFk&u?QPWuv=g z{Otu+SK5~PF&^U=XIUKHAiL@na_>IaB|LfXt`uY08Ls8VuU)XHRFNOM-^TBuDMLwV zsdUXf(xd9#oy~zw1#H4;r>ot}K64@EucI!UPRPHA10qhWzVA15cKfkT!Tt5Y5T=v7Y)El(cPl@q z7F71j9I$2i@9vzQ{FeU^%3S+Rs8JUI{h7 zox*5p7V;p3{K-}Q#=yIo}4i`cv#-nH?G>cY%!CYF8?NA%0Bzt_1hU7KWzHd7p6 znZ5+cx+OV6Sj71ZP?W|2}P`QcVSMCs`yZQ&GkoynMLun%cSb-2My`i)W4IzaHF;f z79@5nrz6v5uE{1vV>XeX=|J@U@2g^*lLlV; zIAS)3BOMW(+fBjqh_|f?t~-wAePyJmRXa&*y97_uazBsAL3G9_^-*fLnqlLiVPS8T zGc0SJaW5*mk!Aj);VVD9FHYQ%_n7E@jmXSVKQ(jVlg07kn4rs2-a3tMMp=H6V%W`} z{_54MM^OUrk1TXZkH!nDH*;MpFc!&+(+U%=7(tu+txts2ofQ&E6Xpv?V&ZYV>ZBlG0No(+7sioYcnl zOm>I46^B}n%xaqM?(`KN9Pp{0E3gY(<{EL=yCNPjBQ0=tX9->E%ky^AK+VN$?6vDG zFYp5_OdI8Ve{6W#tvGYmhL7-?aw+hQ7QDH(@NrE90iypPrDWPPtLudIyLE=E+XXDJ z)!KUp4;;XU9~5$3Wq&hWkvjeIDtC@S9R|bt z5G3zmUw7uslN*r&*|qEWHKpX3^>VfM_X|>AO>1G~-&NOr|EAIG1vTYLAHIcYY&R;= zY8}8Pv0Nu!>H8crJARkFs^y(%o3&5JI;r}ptH5mDd0<%wIGNcC6;qyV*Ib}w9&0D- z#RcD9_4}}5%8}m1f?{TWU-hW+Hb3qQb*DSTTfT{BV=FD+YW2*0nxbHbg6P9Oirup5 z9s{M`D}%b81-lkg*M#7Y@WNFlVu|%#AR>5ibpiOssm6;L+&TTLp4-L+m8pKWECxm} zuY7=TTZy76=WuAv`kkueSV))La;PlNa;zyq>>6#hPLk~r)80&njqYQJ#%Tqm_XruA z)?>@vSuDs6!K4(}r^c9o;GP#81`Vq9XC21)6%`QiFRE;le1|q1qQ0e@V>rav|Jvr; zs6uLGkTEe}0k2hlPUo0>SnZ(UEn!#GE^K;QzY(}Cwzi_rxH2kyFlDkvo;qbMX;BAl zIt9&aTCb0eFWtkoScQ-vhzGBo!v0_Z;xUcWutaKBuwPlGZe14ldlVFEeV*B~^{Iu; zWP`5yk(q3lsju(UZ*B`3c2e_OE?x6-=GWfdZm?rc@3k5*^OoH7APGLMi(6$$t@9>b zU)fsjN(%BIVUl)w`cnH-4yp0C2eByFm+zaAdC05LOdC@wgdpC|Yh{a(tK$9ml^InQ1pUcWlJY1HL-M^a?k+PKTJ`+FHdu$#qZMNTLNvKQpLpexSJJWDrC zUhxs~eRwc1nEA~XPl{~3A0(Z({4(!p1=XGxPe`UE<{N3Zxkd_yk6G1vlO%_;*wiP& z!mn7QzI){}UD~ybXg9>i6&1x=nVSj>SGWj_9qE(~Su|hzwDee$)2=BshxkQ#4FF?({7?KXo>-`pf8pDnG96`?ttVQlj5r zjbvEAZbGLPXIVV;QJKQgndAFo1UGoHU1gGaW`Z!6nA1ZfXD-ZDoiGsfeb!sD(&u%e z2%MsUQ(-uZNkOTv0)jIPX87VkcY6O#w-p~o--yekjbdBwwa~!i3~-t~`Mt^|cOWBk z5(F8A!KiafO+9Bju(Y)+`rp_`f4uUEx?kirGax*?b(Lk>rC&m&Ga1F_Dq_N)$~7?P z?9`3yUo}Op?9OV%XRln$82)$@T=Ra)IeI*Nr)5>P!hHrtDOO-%@`z zlDm|uA!^2UTHax_%l+jyGC09XA3h*eyOMNgGP)qfJlV7*)#8$bh2i#&SaODE{-we7 znOEwc->xczNv2G35aibknmtZ$^4>OI7uz@8c0?Qh~hgkA&r_ zFIr$bNOLi0HBO`b<%2Zi9?WLiJ(<2%U-$h<=V5%!nDE#cRmqZE1O1pS#G_%vC1 zEyKp?t4dQ0uxd7=J)Qh;Ue`R%K5!97zU&D{PY&Kg5RAky5~o)JUCky2*N z?7lnQrFkn!C&0l2<)+`We2YVm7|D#EmMrC%+DG?IY=157BD}kvkfTyqtcv8 z$VuVOTJKxNue{ip_VU^m(CvsznbIHVB8i-E`UZYa%}(WhJIK5iWDt@}+;-N`I)(Gu zH=Uu$o$=;Y(>0p=URq|l1*w*^xNFB*s>4L5`Mzm=Ov~RHLi6Ax9+r2At93bbyC|gY z_Q?gtm&>FETIYJEn|UDRY+{m%JZWE;pVskOqEoN!pz-ACD97Ea{>65^m-37* zTpW0%U|i~ZS9>sD#dFz3x5w38i}wcBJh6cXESyR2j-MEJ|2@F*7Ua+-F#N(4y%BaI}LnevC{Tkrho5|V;V)QV|_{q zx)3#Ytb@h5R@Ax1n{?@2y_IKaE4DuI!;QA3EuM(6=UJ3ntA1;|7FATNVoN`UQdOcm z#kc_@5UU@ZAW4;LgH(soD!uQ`K9{^I-rTj!u~PFYA=^Gg1$1VS1E_*Ej@hXvlYUPo zi&i@vPA4gve!teu;(TM$K^D%5T+DL)1hRBnitGZu;&v4iAc1-6!8}`&NaV_;j&J0T z<-wX=T#xJeXLxXr)(d?bye?1q^p%5z=8?c)&1MBW#qVfN$UM)uzG6U+D`~_o)wexQ zMn1#Bu=k7O*2AlN+1#Lqd=;$j?MO!?hBm{wAf$G!l|)#r-95?pI3o(to66;TJ*TF}x=HSF(8aDFt0Ou};e} znqvKWSQ%t8rx`zFwMX^JPcx>xbg0(j3w3!EZTBbdSopsv?+Eg%juxeIua!Z(3cZd$M-w$?Ct|gv*oHvC&~LeK?V||cVi%j*;yvf?lCXk5+HtZ0F?#74oAsKu8-xo zAcIHjQ)Un5+m;koyi4*2dEl?i8!is1cSb0og?rEnOVPix%9c(G%O%f!_zUBR@*394TVm)H78rKhR)mp8AOA2JA{ zjg0>Kjm&Vix~tK2BdS$;)F$q&u};#E(0va%0lhM>!b?f=)w~|fN=^W3)G(Go(W|*I z_FQeRWx`@Z5Rrn-ILT7CktYW&=V`|BhccGKHw-)P^CL3sx>6=$Vq)HI!1j*?`ZGT@ zh&r(xbX?dZx+_V#m;{b@h1E30%-I+4AV*J5mfW1U*2D5u!=e6V10sHFYs(>BJ?x7( zdBJkCFk2PL6w0k=ev^e;p*AOH+Rc>1`mis>M0Y^M*JN^P_$RFkrl`nhVH3Gp$M&AL z8?*($;hT4T(NfB%*Yczy@RD~d2b8kjlZ3---DfylO$NQz5^7DrN?f;q)rMRAb4!rwffvPcx0iqFGS{W6&ldPtpA4{wkn8-h%f~|*xh6Zkdbm1RHUQ{p zI~N?5iXopSLs(juo#l?Eag!rNi|?#F<|cELlzEs%K<>xIeJB4$cC|M+3a@@RW}v61 zC*?WW{+gek-|k(}InQatJ!z-Ds)xS!Qda1P-Gt)|<-RwUNKSA>p9U0X?ry)&YxZEpS{IB`9&%;jFWRfZgEvi&T1 z;al~*E`Xsu`M?4Vsgp=Eo|W1+cWvzbyKe%h|`A)hwE zIX!ElWc95@9;4)QdvM3n+r?wBT`^N+aF^CcaADxRCtU*du!&U4Whuqe9JqlBlfJQlfAP0@^KN6 znGt%2D}UhKGjJN_sBq)#jx>PrVb&3HfFEF*7K(RL%ZRyEpxfif$)|mQ4H@$XYpcX_ z1HhOCgOUA5;juITc6Gf*!HoHh`95}=h&%ldcsy|~@agH_lMW>E2Lx*lwD*0}12Cju z-Y^N?eGHDp*KcD}f3`iwmnwAb4`5sA05cB`&6JvVF5(g`pLPiY#rHB>HwMDb+pDVj z_KmZK$vIvZ)r^E&#KlEtK-J|nrJ+N=;nYf(*}=R$*Bq*s0d<|(5&JU&8Y+1`vTt4G z=dBoEO3>x?xOr=Ymp5J2Is@r`4(CIsBl)i>@qm|TH5qV9Jgk6gsyU0Nm~L6$hGP!j z@?Zr{d?X#1*D52Q0jA2WJM{{x_ih7B3*-uEeTR13jRMdRi7hm0W;Uq=N<+ z*-J3{`M-E+1IQCpLB2W)PXXdVxfxAE`|kkz{};wMqCfl_IypJeMj)fEvGl%+@inS&7!+9!9S8hJj7 z20pz;UOpjOmdOkN`uA>QY5VdGi?4&@mip5;E7fvO{t#+>Y(1s7DzEZJMB;YdEPQXu%wWCvlVXMhu*xli=cJe< zlmaxx%oyli1Je1rARhq0Yw`YpZeMsj4S$f{&vFf(Lx3b)3$F{@6o~s%9r?o*K>Jv+ z+S03?!);l)6bkT#N=6@J__eF`fT(WJGMo_rE}IM`n2Qs9-$e%oFWxIFO9XH_J@g-8 z_&MD%(0BNShy{8Wn6AaCJxpUcmYuEXl-7aY^)WvyDcl}_nJqG~`SJXm89iH*;@h_X z_h+P{3H!~1pYj^Mw*o0jjVugc>HQr*(Mj`=i4L68=`TU1U+V2=T}y*IpsJZ2{v&X2 zmx~?-BpR9KGc#Kq=HCbVwapQAibsD+1MpC}otyxyvmh;l=eJt~KxVxJ5AM|Ss;VlJ6$^mVKD8*cS;33@DxRtO^A)%gosSyHBs;aKdarlt z{@aR7p!c;Bh_))%vB)J(HD2)g#y??t>NTx9g>cSIZ9&$bljuzZ%U9Tk95VxZ-d#TE zB{u_@&o$$mW0Bgdl>h`+pwhHQ!zt!ICZmx9zZS+V1kjdzl~s^oXHP|B2PiFpVPA`f z#=5258o7EN&npR?%rb!rWU@_0YH&pYx9RogS9d#(r!sPpmAHIYP$$K0Lk$cJS{$;I z1q>~G%L+^@bg8Rf`1fFXdw%NBQ-|~C)kBZ^ZkxK+@`rFybJi(dUgbWUa}^#S%l0}u z)lX>HwZ2wxwt=1{Iz9ZdmhR;lKK(3nq}*b#cJgIRiqDc}f~3<=$kWn$@6p$+8fS3k zeRaOqN>`^hW{cy%NLm1^DPUGbjO^`2V;kSirbho3JSIAz36GX29Xbk+0PKOe)f}QL z1Jd%5gXl4Ka4=7W~ zCE!%P#n>*H9|XVUtbBXGt~~@zQSF*Z<@XQQCX&5Bh}F-I*4H08Bcqq%x8bl{FMn+< z+RSTEt?1e31C;I6fr;=d-y*pEw@`$t8G&^`dxsZq?Gf4Wqs#LE{}mH{_Dbu%6IXef z&w4)y<&rmPZEdv%8Y!#Yvr|^1sQTMw<5JtOt6qaP9_?byz`*vI<#7Cx-;W$XvJI^T zPtg6TsZvJr^s@=91gKdT-m2l_ZO$@j%yAP0RTk~-+9x5F0nX)vwo3~ySdz|6e~Cq< ztG}K~^(XMPugvsKJbrN`ikV%Y<)qyMob~OZJ3Rw)rCkao1LoH>N9%m0x30FeynZcq z(V579x9FB}olhmJkdXP_X&&~)h}W=7$zGMGt%8MQT^jGu>%H>$T=V`wv5R z+Y%r%3&+#Ke(ui*bs%gG(awpd6o3+0n20{U{*!Fy`7vHr2aBktp^Lbc(D)JqM7(R= zdXqDAuM+@YM9tUl`lWMa-?srkH}%FXFbmtvHb9~i_T_Zz@mgJ(PYi=V97Byzx$N7O zJKr7#9f4lFdh^gpylOkh7x4l;>GA!&cakuoicWoZH){Yu=O+T_k3e|g7R5jDJ={JE zAWq*E3q1APr!fc47+Eu}NCMQg8pmzt`bF2lb%aNd&WbYCkz)o(+ye8@3Bo=v^t#LS@RMpZ$< zCDiliq@;^`kRn#0Dm4dClQDTU?)*JDbf@C3kLP44!L)ZyhgxV>jjh`Q^6q7-jK#q@ zV9ix>8vauC zC}irTtVt(GxgjJ1dqxFZn~yJf}J*LpkN6Y@0$)s@F_A%~wuwEpGxaOxu}{o9*IVEnoWc z!=ihHb(}Ry24I5BzvT5O4WNmJFYqW v^8J{l*VW$*)P>&d|Bo3~q3s^7y3~33F}33R_t|8@pdHd&GnwddfUt`K?)-lK_RcHw&e-l8r5+9uA@|m{wy=hi za^!`Zle4iG%yiQ8tsV_L#=RI{8}8S?EgNO~`Q>f(XVaAs-;EWqe9}s69w7zM*#24S2<&nbs(m ziuZ5N#16kh*mQzQ3+vg!ir83T)27uWfbYYAE8ETos2;Z3mDwl2TsvVqa^5jF-Uwz^ zK7Nb#CnJ{sJ{$hgF3^zaMh6ia9}Ki{)S3q)?G`>hSLlaJzGmZ8^s708liVeO~av9D(T1WbEw zC6VEOxS2m|slXYxU${R8tq*0W@}4&@w|stGUv<3Sy7g1pA3x6-_$z7R|3QEl2Zcgm z2bNgUa%ueaEl%#r3uw&m+B6Grwje7XIk0@{x4**hg%5OL^&efxatPr)JUtozafqOB z;bF>`l|%9WsMTYIJl8xN`Lmo=i406@V-&*V{>?qb9#2ekP6je}x%b(Duc@i2io}W5 zm`yAeTOK~kF)}V2`1R}8JJ>=K->U`&5;uk6DPvi5zq}zNP!B8ZR|<~+XI&f4rYVlr z0?`S2E6by)HKGQU_TFm zTPuwJ&f-ALeSv?M;RbdCFELo*l?0?6G{tp_xGeu2BmLdaFF*003(c%8#PqiuFa_@g zN-Kg{fjHplGJASCBw2vY{35fs{_f&;4E3+50L;)54deN` zg8oNn!5(jLl<5~&!B9-{VEk!)ZTc4={9&K`+AH;U1pB+588GjcO8AFgAK#F1UFq+N zZl!53zH%!56sIj%Qixq#`MMfVU?&{D{EOCK5$;C_An?WO^D7PDJA(iR1KsZf0U8En zBp83=SYHEVZ;529pLYBmkpDf|3O+wp9zA;WB1-t~z@PJ! zy;yac>E9*XM@#L`p}pt+P-KI!LAu?`zxDBr&wu@0bN@5y)7^{uk`=DAFa?gPX2@Hh z<`7c6k+=LVO2;8Up<{dMshtncq9yP*e%gB;b^u$BC{=MjY9Bxog|A}zeRP7W;pYK0 zTO1EJ!MT#w{+rIyf^8XAjAMVK&{XgYu?+}6R z|99Bunox^g$#*AnF^IDna7O`39(IOXrLlg zB6HZHQlgAIRHo8^-+Fpi@Av0?Ki}`~e6Mp|=Q`K9PJeZ6&-2{Vy4StdYrWRGf9?M@ zPzM8=*nd$&FHH1HlaG>@Y_{XP=UQLH6N_q=w2%NnV-J_6`1}yZt-)<-JHkkbS1Ya>my7QoMe6hcV~bD_01=mW>v4n-DrBTl!oFRXEn& zC0Zzjc=q{zPq^emS3SFWdlg$}Mh=~18*3>cOxHhm%D7(BXnaZ5*;lCV#Z9-P@1Io` zM;TIgL#mwQ*5;Q7({m-Mm;UL(i<@7J8!34eZ9Oq}iND(Uu7a1n#Bpm>C50t90I25N zT*Y#g6*Og6u6Oeai{*XTN=kpQ^yImq||qSYwTLONhKp_QDr zMMfaUst{B3Tg6Cg%!#C~ky!JN{w6orO zZgtRb`Gs3**M7A{_dDO_j9Ic+!Lw~7{l)6W0=}W>KZE27LY99*L1giDmpkKOX8~qV zwA9_I{B*F+Gd&$AVPD6ag_AhPsJ4I$Fi521&r$~B!ovv4Tfh+Hq-C9VXb!Ef^BinF zkvqoqxNuY{-JIGoSjT!4j;Klw&o@Ni~zf>4$=_k}xn zyY;ys@aK0udd9gqn4`iX(z5@l<6rhvua&A+uXoa|IM$nCW{I#)~yvr7WP-O>FC(lE;}{Si6|>SKjCcIRqU@{>+(3pCk*nJT^Hb>8oKjnq@zZc`nbpCcewZinatb4Y^nf*KHL~J5nP=N8?@9ZbS1e-#;k$ zJZFvYJB-+wY~$pD}EQ7vJN&F&X`9%v`h_vpu5Q) z3l#yfTZYYT2Cox+l+E*Iu@&3mwqAu4GmA*5?A!VgaJyKzkq2*AJV(db49Jb3+QN+s z_7=;4!{siT0N2(-Dc-kh@WP33;KnE0wm!RrL!iB;&2RG_YUxg)ZxVgCB^2}&OL_-p z83Pu+r!#BLIA`v3rMqy_tw9@=#wf$-?l0rvC!dA@cl4?His@}(k>(i<0d%(Fg!*l1BA)BR7~^nO+dfKjFY2Zv8AD1mZwUn zN20GcNi|4~AWd5$c|NcwW292f`!XM-#NKWOnrM~T{D5BNo#YXou_&UXk+pggr2fu?mt5BkwCQ@h zQ}cvS67RX|uL7-S=3Yv|Hg(AMm22EMy=EwI+2>B~*U!SzhSbRkJV^4(eBeBxCS_21 z-x{8C!&YSD3Gv|bJSFem+&vC41@pzREAQ`Wl1qzc;T)BZMHL2(*zXcb3Ns-=R-^a9 zieOT;-JnVM88~_pefvqLHMf4O%-B&=QyX-`Vg;Qx{sqztA@NA;UZ?GmSVZ~o)3;5i zB3?b$iMr^)fxtrqATWu9l1H#?B8|mIH-CKfC`|}cvs|@oDAxQ4Szz|n0G&vd1!Wmo z51bfX{f(&rEjF@Y(4^njBEF)#!wPi4JpF z&8K2!kC7_-P5r_nW~uwWRoGWs0TVCaXjB-#OH={>yT85mxZR-#}aE9LnmBkqKm;3ew9)8`=FKy%o-B8@W&+IP5FWnf9UZCc8;nrhq+Vz)K z`sw0C3_;67s5Mk7hKOB1rTji_@EK-`PI*m zyj+U>??_oA#0dRf_epDGk$~ZEo)1fO@Gl?^i$TE%Fhc1fL|JI?+f2cwS4e{$x84x` zm)sWX1fxSt_G~$Et-FkVl2nw!H-F@R+9P*Zdj5CVkSX2n@vywZUyJqcgSV4ze4RZ# zmwyvKZZ@|Yeu%eLIyLcd8MK!v67~F_!*js?N#T`q)$;`?ztpWGWuKu|&`q?IW21H1 z{worYWGy8vE*`=~sq6g9K|hGVvMHG;{s}&0?^accH^I_WcKINDimOm->U)ht{k5Ru zI_P87z`9D;b6}6xlXs}|v>1e-Oll%{xi_fAnG%|||AjmZU}1hQ#NDGrJ%ZpYt*>uf zFAq%pv)P23Qfn%425D&zY4@+|{qp|N>hSc0m0koU&Mbn4Bb4SJ5$!)nMjIS@EPSFL z0Cxa+MCGza`6q$Q-i&6F`2%MB#!6J7^WfwQ0i=+ju50`^kb#Ly1+m<>|21GjU{g}b zOH)mA!ie2y1rPs05q^6Ifw0j5fcZNv{^tn{%#!K3{Co7MNPyI6-5t=~av+@@%m(*< zDSqG+{05LfD?)5P4H@I|4)MPr;hz$XMdBgrd;K`5P~fw(bk<=q-Js(s!bzuoV}^fw z*8n#H{P;KR;Fo2LiO@O+?=1PflL2QI#+K&AFZM@6{ehBxLnh#SK&1Is_>V~SXFwE@ z>oC}NlWmR|v=DR@JEU_U1$?EwmvF5=UFV-F<~N)L;Y6t0_cwXr@A+MVciKAQ?_)7W zFgC5Si$%L(gZW9hMAQEQEGYiOhZ+Ck8T_R^)0qDZh=S@Jyc3_waz-1Q24jmfrPo$5 z&=NK+h^sm+<@=BRU{1kmK55QplB3##%re)1bCz}Wu*_Ui0v(B+`vgcbdeQ;S;ywEA zm*Rfd$;DLR;ETXcHA0`GZ2$G0^_@ihDO^|QdTV6` zEkZxi!hwKC<#H59X&r_Z{aoIOpq~M4MVBTrqL4~}$+fw7IG$j0}= zIA=YPgif|y<7+qF-4|$7e=e*M5-awKW~KPUN{WS*6H#v<3Tunr*xZrhH;6kHXH;9% zk^f}LD~$2Yy^RYhf+ERkRA#XhR3~65UkKn;Fh-Oh5OhUWN9RC9bh0hESr4MpJFS^i zEGLK9aM?6_4xkd52h<(Es-;vC(g$~{{@em-tnd~T4w9p?L>M;?o6mR|K_pI1rQ6=q z!U8)%zxOV-&yAiy&!rGD`t6HRqOhRgc>F_JW@H9VAv17!m%s(B3oru=0`In?6~WFJ z!V36CLv&6OFJ$li$t(tyYmZCS`$1fo3? z=5<9AzO#9%j2#^ZC*M4^5$CL>!9!^26Vg(lpg^>iDePcXMD^S-*7MhQ4_el@Kv7{2 zz)6u5z(YkldAJ+R33%x~7}uu&y=Y`zX|NOg@aNc5;@qfR$aZm+6>YzG)tPd0j}{dF#ilW?uqVbvPLRl?rU&);EOofm4-Ac8Bn`08#po1|?gc-h|X`uh8`FQ06&Pucg)b1+K`DQq)^ zIHsD41`$#WL96-a~~RJ zP45Tp%2+0ii@yWZuA}YLu#H{r_YW=KeJ!3mj$u%cbOHa=_D4Ns8~Ds$<*oGsWx!(g zXKE0X{w9|kjUes2VTbwu2v(x{6%VrJWd^}a6C!x*)xyR^i%UyqH8rUt^*7wFSKk3R zoZ0Kuo$8mx3IQ{wDvqYPFrKx`db&U$a zQf=BK5eSv4e2IPDwap&xoq+aFL6sn-TANBDVQO{AT-j%&Fm<;IDNc}0sxkRJ!u^n} zZ6X=*9Ss0vryJ(nD>|jcHaFW`Ng!lth@LP9>|*2jzHmF$zS1A)ESDwL7P97R?(d)JyL^n;25bda zqUm`%Ktv|UP){a_N6i+hi|w($cj~I=ZQz6WE+NR!+CNbxA>eYfKs72}|4)x-dMpfi zgn=uBvmqz&I=(FN*WUuH@a)X2gjwB{-{6nPeu(rRw9bDG6bOTL&mD~dkSw-XtRn`6 z8vMmfPfj(Q6B7HKnuEVTy-kOEn3>xamGqQ$|8UU|wgaBOeEWWKo|gOMGoDP%}# zrS6poXww%&ygjBORVyD7u-Ob_8F}_pMGvw`S-d;|RSWNoBehCyjfZ`2ud2Qi-39cQ zS7#CbRYDyBQ~?=MVg`J*C7tK=i>unSLl45k1vOVK-(G#^_A`y%W4aqk7`aa+`o@!f zX6H{1Y`c)Exw)R~R3_=qE_Lt-Fjs;mla;$g3(vXQQXqvm!YgzEF?{LJ4-m%Duq{aC z-^Ck*=gd<1W@dpPR+mZ-E`4lTdAjTUYopf8EPWy6Gj(x?tyd@>8FRi_LbMRKc|;a2 z`PdUlEC%>?KjU)033wkkBex5g24=+jtv_J$K}w#p0otsG1ov?-#j1l(B@CA^d2_-08us#-W7x#SIy0(owuV5`!5bk zTZMY5j5P)+TzAQ|3OqCdsIYiB)F0bUd;7#o`P;3AvSXKqOHHzqg_7RzNrq`iiiWoD zWf3W#(Rx#nG)P0ILHTumN2t(Sgl%m^4N5|y!b+4%3eMewD7>5(grhP>$mL5d%yIlK zs~w^2i3L#3!qnPwE+KvNSSvzY5B>7N=gxcem0pFJGiG^;@qon9VX_!Q5BNT>aqN@G z(Y~-fcUZiB=QSUwaRDopgw2(6i(!%trMOWeGkDK>eF0#+;$nk-*THS45u{1*#Oi@@ zAXU9NQBGnLl{m3_to}ind`A2kPrQzgkxIJF=kpj*5= zXI#>RO|fy79RrdQvYxThQ&%8>UZES3ExA_KL?}PeVE{sD^7o&P#qt4MG zlaIsO{O23PHrDJ0t2ILpo=jwy-t)TnSchj5fCo5d4ae`6D#fI-&vy#HS=PvTEkMTM zgJH-+4DxKQ-#Iiqew{d|8?`3+VE(vwUDClN!*IUxXj<>o(x13=6GAYJk~pgKP!uf7 zLM6g-6J`%X;^qEY9g46>NsrU0UqcOjL?}_J^a}C zH*@#@0Hb$1PEG1J7vPum|8ERnEVDR!`svV>b4+CR@$2^_FH$L*AKGvV2bXNB9Jl-0 z&!KOJq2t`s_YkFFXiAAmMdxD!UQcG5-2|;?LCv~G_j8!L zlM@$2w2;Mlg!nV3!{QiGYsdGzyAd4^5@Y(3DE%iFfEQeuV2UgOeoTgz(DKc=1(uEGn!3HY)%eKwIC?$z z;<&>te%7~z1l`89sEu*Rsq-2-?Wb^3kAV_F6bunRz&6on7n6O)ISWW20A~%${L)dc z%&rh{J__reiNkKdoplh3-gb7)Fs#$#(35>=z~=gS{lR;o8=FZvEPC!IZJ$tG zOO@ha%98CjHh8kGrEl<8rx80{Gqw(!4EvC@_p9OB_PHBhB|ic zfMxoB%pGua|MxR@4$uxc=#6}{;a>xJ`3Wk3Wc}GUv2byN$PRlVLnY$a1#Pe3`}-N7 zwU~aJ%DWVU(7~herT7+*w)K>Jx<9y@)+lTrqR)G;;6E&n;~s;N zL-BI&Q&bnUfN2y03=&NV85v$l0jnq`HVZ`5&;v-iLc}-( z6L&!iQaL9Z8XBm0HE6c5cD`(_~QR_RM=*fSB36vBM>TghIfD^TGKWP5*)Q_(#0lQWNKA163vdIByK=#nRQ*ZGC z@=8h;2ax=q$OCZ9MoU~#6d42iiX=T*_naQ|r)-6#2$d_wbG&7vZm`7g3$UwkNxS)R zzOVNp#4V_i|L(zFW_cIH@JEp1{zyJ0i}J0^@q6xF z%-|*-3psjJ_*I8x0Rt*Rs{&oemd?g|N6boZbzF7lyb;k;$W-82Fas3|hzu-gdYG8s zQ6wJ0qVT)df)eSo)uq=2Q2KyrF* zV_%(~-l!7R&bp;ZO5pyU11(R69rZSyR-`YzO6)w7{BUb0(;(Zq3*1ZGnoz-DT{~A5 zI{`DAWOzh;Pcm6;bSWVeXcgJbQ~c zo#P?;u!f5T81?p662JazTOgf%3%8a)DxI1G_dzm*ml(s#RjvcP82El?^|uj6MH_ zB^|}YB_i-DzH^s=e{Sd2?K7eir<1?%c7)@1;L~<9OT>sMN|#;}8@|R_b2a@kmD1Ry z>z1Jm`a9U_ilYQ(Dm^d0y3QJuek~`K&aCpPM}ko?b2tC~Ho544!twX_$Hs;N{6FtI z_Obr?{gE>o=7q*jx!-(Qaen#uW7Eg`tJjDhV<*kMIa*H$kdtm~`4AX0p6xE<V;Y$Grtw@a_OsIX$A<8M^Juk6Vpf(LRVbUDq} zmgbmK_v8vy))F7YlJShHKN_10h-$(i;k9p<58-)_z7yByyXa-|=K0&c-S`W_8wW`> zgcg2s)~%WKA6vrYHk{4e;HLoR@7j3HjMYH2t-5}9iAxjVe(hvBu4U7k@H9K{3bCX$ zQzW%!$?COw^ZQ`D30~EP#1M_k8r!jOkph)Pi)po&qL?T^63eBTziN`-bI-L83= zg?2e^&=rsA%!|;?vc$d{6yc4RwZWEdw=Db?K zc*POB2yf7uUyh#Ne-dt68NFh7787?6rtRIk!PQ_8GJ|z#ThVLlxP}g~Kx!fp3h>_p)=$CD0`6INbp+Pq6 zO?s3gbm3GYHm!hy-!ybY={_^RX`h%16UrXx!!CTI0m)gH^If`xnF}4k`EEbs{Nv!p z)aMQjdrY|`k^$YzImAu7VuO#34LaHs*3+@5>a>s9qa5ej-tj{?>ETkI%oP}l%k5^N z)PkiuM4~MI+il|g$6{9k9`~y&by=05U2dJ89pt4FUMvT^CLTd@Dl0~ zFVF4co`oDc-)N}YbY1P;_TQ19hhc;tnyI#2Sn`z{HQiMBEq3<$wegc2?2m%JH_vp} z;ew#s zb+j&S+lQ8N0#R3^0#TReXhm>62Y-C3NA1`h&p`p{o6&{u@=}vAB78wa4)p4Qhu(Mh z;&R#8+tvgcPX^(JE7Y==&OV-PZP1W^DR}(H*Qy{n(UrC48?}Y+r;1AJzP3M~U2fb{ zy5`+DR2Issi?%X%e44Cvg&OVhen&5vo?i=X6(>K`m4tSehz#4sHyZ z5hb^LpnfHkQje{huiVIWru2BjEMRP`(cbOppI&_mqrMkQ7C@n~ z16CkC1$6%ECPjsw`hZ~061VgLAjKR)T6hi?o6!7)F7 z7V63k(2C|40oBnK5Y#iI0<%3Y^&b3~mQ{k=Ad(^L5Gg=B0-)p6M_UIkVMn+SVl;5b z%t|(ZM$pChZI8>_rTVbtE;)j+xe&fN`7nl(3KM4pI;Pll(3)zL9(K4nb|3(mj9vcQ zsU+AujnFFqjvfcLmTA-_AkaD)?)KATKDY2-YZH%NxhQ*%pVkDT0|i1m{5S)E$2U)- zxgO~h(!TRcns3Wj4<2B?BIO%{;FP>$yA4gG%a(pRT6rpc<`d_OJjV|GdksUTcMwXr3 zLJx++AiwPjpr$fLG0?iG-*N|9pek^oGB8t6B;3;5`?{_3`8A>F23SPu}?$FiM5z2v_g zh3HII{cPXmawJ98Tyvui0t3ZR-q*=Qwz&?-pAAAk{Lr_LJVHrB(nb1H>TrIbp(3%s zl1+XHIo=N$zqzY{z{2gyEV8k4RHAi3?gw$7iZ9<8l0b6aT;}MzqV}aw2T2Ed51}`6 zgIds;P*d5C`Ssq3rO!aj7O{%ieRu~aPRs7F-^C{>JePkgKqjkkEAQ5c>iQAmvOPS3 z1KQspvQ#7I_C(04;500RP(%Y@o*C&1wNOpOzpC8O2a;78>_6R=iuR%tba@i%Wt0Q~ zyKzQ49g4^wXdGMW}DoOKaDziB|gZ3M1#UyXm9G?}Wo z@9Rl(YKd@F%kP*cB_787ak}iO!Cri=!R|dkRW1X@xEy&k%g(7%xpq4S>k(s%sOY zUDJo2Gc7Au+(R>TIcs?)56r-yMu6&kDJkkDBrs=)N~**mxgzrI2>o<0d~-Hj1kP*W zuK9=V>e1q3@uj$#Qw1rkTblbrLEtOFpdlLcUBpy0gCbirQ*m1YizdM?AoTU#$e3u{ zhnmMJh!X0uicvq`>D_Q{TGE+}i(dLB!{~JSg-fi^0ejzuX+lwZ;<9S=l65J=o);5C zYi{R0Y;BBhnOSSBbw(!X=0IApv^~oV+@oL=c4z6yuK6zyw$77c;BEzFmD15H9&{^ z8C15Rvgj10qW!i(`K2rB65mz|x6YL=nnkh33YbXSIBZVw#mYF;iP^t?uAOif@YYJG zl;aRGowY@NUGQ#7KkR%{xmHND*@;6msTC>J^mQDtj-3Sf_N8muhcFeE*cNkto6~7y zOh($E0IA(&BDFA{o@iaRMo}&FkcM)K4h1EZDib#P?T8lE6&5=#(QrsXPz#-rYdMgH zf>TuQpgY(ci3S-=TG$>ilk964eRPZHVI%r7QH9FDR|0gOMrf77pR$aPdTQ$A>|nc4 zhK3)Pw@|)-EN15Sj?+E#ZhQII4oK8 zb3KJnJcxom_Ad(hh>&TiCD&*wMnt#0NF0q#qk&JVTc_`11&h1vMrrw^66?B=rKaWr zF=lqk8jiZJvOA${?8~vYm@+7)#M|;r?N-^)VUTfbA~J?5Iv<@P9ry^{)fWtr?TFEL z7BTt+|1yWc&|O(-DDk+KkEW!elY(P>;@!~NI`uk^HJpL0Jqc^b?Yzz5z;5^g+S}}x zIfiz(hdJ&UzUqSxgwZ^fj?(IfCqsQZxiA&h&{50E9g7`#I4+6pX4({y>d@-c)r8fn zH=susr8RhJufn=9$f>ZQ9BEeo2!t`BkyixcYk>$3Eg`|%??fm=Yi2tRji;{ zF&7#pxfkA*vrE0h0cK9wr@IssN*HW(C8a>QR*_{S96@7pADF<_v~b_X6Nr3#9LupiWrOtP4D_WzQk+Ku5#sb~ zX3q1%@IhJor-=_;sdl$P<$_baZSgELNxM}ntC%vbygK@(DI^bazeUXQo{>l-9g2)A zqyc8=Q7n95-+LojItlo|KJ0coS#@=Fk8O~U+H`spN&A2UlGOGCkp!Vrd=*L#!1v zlPf`!tPPFJ!{leYBn{xG8o?AGkH%?W=8QowB>BUjxj!~B7F1_6?)5gb9*8!J-;!sj z2kcdOD?08dt!MBTi|o`Vg8!v@O@g{JVB}uyRzbS=1D1NKwrsUR3tpy{*yC7z8D?zE zQ$-GSkr@VN-9VN?>`x!qF1|y4SX43x*q`SZJhgwo3ro+_c(G~J@HZ*z$UR8kUc#Td z!je7tQCDE1PX+u#7huvrebWD&X$jC;m3r~#F`X6AngPA;?P6#w{@P(h_Q+o3{JNv4 zxC0T*O!#wIx3(Lm0^Hs*RoTx5MS_SOK~IILxCnofW>ukq?xi8d(vRmkyfcVD+R z{pWb;tcS;HdQ4`a;mABhu#*TZ{An%l>$jQ1a^WzFuh+#jbc&eZf5``C^N_814jJ2U z61(81N(%lR%G*#HFf4Kd|1l5~_L*s3L|QNvrqCsM(a$>A40t?Oiat33ei6A9HF%gx zUgDuqP7-^J2V&cCvA~=h)b?aV4I{2iNqY-A5LWN;l}HLyJ+kL73d|y3T$Tk9*y0{~ zyc^W$)V+Iw(n>I_XB7;G1xjFh3sxOc$rHQ-KSl|4ecMVd%+QzLd}kMa3FP1AfwcPk&r2Q zV+G*<)QIRDxJf>~i5w($R-u`{=9?)}9hP_M=CxH?WCnj7{epk20e_g6jLCN-;NZZv z!ZTeqK|aj{^PzWGJud|o;|sL(f5~Dnx?CbmDeLWl*)6&cvpbeGxfl8WLHK|BHRa~M zToBTybjD}6L3s^fPRk@{eSrD&x3g4@1`YH~AJ$r?DxWnHvin^~elrqZ5h*!yp3H+< zW8JGm!V|zgbS8BWLQPbyNtJ|@tiaP6yB;>czNe%r2BD0vaaZJw+=A(R>-!JsV6{0C zK~xt08Z@PWmoy4Bu?)wb7hMy$BK`u9r&|LItsW^2~W);?TH2 z$7ke_Zh;F@m~H$2IU^@f(sePF}@*jz8E+nwgmi5Hd@w z=c_{|`HyDAQ~*D{j9K`z_K_v%_-lSBpjA{^Nlu8|Mw61e*Y8~fR3|&$na0oHNu9l5 zbmU*pRR2kU=D!%w{aiK=cwebxN)Js$G);?&U2h>w*5+~JRC$>R3J%vw^F+wUFn-PHC!i zoMH62b}tl1aRA8Vb}aD__p==`KW7sUpD-UL%3cKd6F=NVKE;EaMiJfOOmn7328y)L zm=>sqT=tuM63H6>FjL%0(WNz7E${N>yNSduMMSN2-hw_~x~}^QGY2aZbBHRk0Mx_k z>of)Ifty;8c)-z{uM%KC5>c5`^;Yfa-676?6 z!spDXiT3XsV(RM;0YTyKrNTS4RXiY?2wjotdfc!lJurq?FEl1X#^L+Yv$AIdBDB6i zW8}*L`c%guv~Q(gyx;l=OaqFeu%=uFPOWEMXTu)tj>ZKDj~x(^Gbc3@Mlu4+iF`(( zhe#)d7iGz1IG!&4UM>8C3`JWK4*wvnE}8g}-~@%JiSt4{!d>1oNplT*kHK-9kDOyo z4+CY#k@3~YHk%m2>hm0{iHd+NB+#_eAJclyo^T6L zSczGwJebEpiT&Zb#6fc!ZTLE=(eRrd8%np`8QJ3m6!-##R~K=zMtK5WA8+(e6Vxw~ z74e!25znsnS&b2Aa9}vbz!&7+o5}{jN@NVRlB%tt%a+qKuGF9|<3DZh^g{>SA({y< zGL>g3qs`J%T%xJlH6FlK{?CuF^P4v>U}&*Mpj!sj_%a=qkHXd<|783jsq zIz= zBREOuTmS;`k$%wh+^4$#Ga!MTTZdd}HF;F-ADZGfwYy@Ljwp zJY7QqJ)98$V6d88O0@Vz*;u^4*@o)-jy;8k*41`XAbs-&S*S^VvF(qsEhRu19`Y$_ zhYwSIZkYh)y4H+t4sb1<5sVkg%H4CnHURYRieWovM;6ij&z`q@R^ovqe+S4Q&Nwm+ z^%An-AcWd+g)ulMT@Oh8AAtn(Dx^AHhN$`eGR2Uf30RRk-!>Js?se zkowJPg;u~Nor8{=T-nrn3Io)ZsrTuppH@Un}P{ z0F#5fXBV!*^`wr-onIf|inT=;cqmXP6``B~T#M({2(=T`cDVQgdhPxUIT@ZoWlnd+ zq;fIHM>jX*3CgrR?AJau`{vyKlT6jL0T+magi9yd`W`FYXT4`fvUU8>&ItaGC(M|E zUBtZLz&7|F?}VQi3Q(DtLzEDmD5iI5n&|-!E)i0E?daL11`t@xRQNq0qrgfHdPL3f zBXJH`r@A&83F_a5LpMwc?~z*k`IM!ACn;PIXgh|HG|2?D%)_ zlF8R8?F@=6p*`c$7ds<&Adb-uF-37z_+`%6;DR7~oJ@2BloTW|Ny0T~H(HjnYK5$_ zn#xR$@%JJiBVe@BTpE*$C8_&JNPr?)`OQH{Yp6dx7G#H91lt9HD>7vNCy7UkpMS(R zAd|Gjkw8m$nz;TZshM0_AA)vqtrc37jRFZ~x@O83fcU7dNTi#rh%RiKvAoCuJy4mi zx=o-ChM>Tb`*8xtyyNl8Qbo5XNsQ%!A>0Ix$&JT+n`uQipU3CV;U>Jzswa8tK zjgVMDD3?&lL@oh#-Qi-a>@S|AqAvBcXDFTfi+Mh#%tU^{fIL~8ew1xYgB z6%X%dm=nQg4I+SWgWprI%aA{f7S_cyGtxj0kTsSNO0O_$ltnadQJ+T#xqX%!?$^C^ z&^3PQCW)6}sMdLQo96D(V3%>o`w-PO@q$uu7XCFhw)Xlx(zBu($HK3J`5Gp=Jg)Q0 zvqg^1{|t%7c4NxsAQlk+ju8}Rv8f_+Bp8h`yC@_7O~?s4=aNqAQY^;JZ_0YKI2}RI zc49I2VL%IP#0`5mL%Vd?JasL=N)o72gopG6|N1E6c~erkA_rh|RGlV0Y)ktJl#f^f zyc(S)0uOlt0+AVS0RGBjjR>s;c;Kd8+UH{kfJtvt#Gns0dHf9$Jj4o-y9D_`BWU9u z4JOVJ9{5a0+LVIMxL>8KyDA2+GTDyun?m^Hl~B;i!Sv z9_dr(Kl{XtxHnY)(Y+CmKnAMsVU{D7Fw%)BGsi4{_o}`#EmUvpV11osgsgsE_&m02 zXzj%>&`|udF7^WSj7y5nECFmxkw(JcL=N@0&x`kufMMLj$R#f{HJ|OH9B$pFn@Za9JqVJ>!9(}PTfUm`?ULD)~ zZHv}MtuW&?)9MO{<=6JssNI*5Cf_I2I3<}I&N^C<{2tBm;Af3hOVk)9a&$nGu-;k- zRF#Fkg;WY{coGq5UTZl0mTEf#cVNd9I((Mb7Qt&~)?t(2oW!-=bM!3!fu!2gt;Oe+7g`%0NRch^uZ&flNJ0E!x0(%6$=R!y z3$h6ctXeQQ3jMOgui#mBo%A8whr)W0?E{3Vm?O)|$|M1HmGgi~FjR6}Euy<0LdwgX zqmUA!48@_tKkiV@D%+Usve|Tf{_@yk5>yAIkajrLhOk-Y>Mf#zu!^b5J*nsrH7d5**iWsq%MZ4g78AZCV{YwTy|0mk z3$Y~i>lpc05a055Hkgx~g|G0orjsg-hl+NiR95}srULr1745+`Up8V-o$

X?3~~ zw#G{?8h3nc{1E$!{2*XUoqOT(;^;Y3^pO@JGJ~%Hl>}m*Q^Y%BI7qP9*q~flJ7sA;sA5AX{}yb< z`Ejcn7d3O+h3Qag%u=AJEaS9=P-)*#(VOg56}&ULq!wa!U%b*+oe7&F;N{E;H@{#k zV{Kg!BtLj8mKUpgLs62-rlsr^IUQF@HY&I@{rOVLirDV0Udky3Lcyg#7o3cvzc>C2 z{@XR8;k$b_o=3NXLXtl~l^3CP22@a^WQQO6#{&Se&TK;^vv4W}f0|gL=a6|_S6?6W z@`=&~$!z-7`R5{DG=j{uv#*y^c)(Sw&*5&~!!nkx{tN13#n$L|?g{h=LdKm40L<`5tgay_=W~uG41o2|%((VMJ z%~uOfp2>9KL{$h{(^AHNxx&oe7Nn-$*JV{G!Qi5PrQ5rHU7gWPz4ggr0DASBP_5nM z)@QdmlSOF_uI`fKZwPgVgHPPitz7Q&ey%RmHPrN9oftEFi}fK=zo`*1DWMox zAbVS|ao^c16(MI;)=cHmTwEsoY@5_bw!bFqJ2Q!JxzMYGfJhxrd3 zb@6*`dG^(r>xw*;fsR|R%^j9eZKZwlV|%TaXT0AYk3chQJTs3nC;WH@qscU!Ptd?9*@yk}>0kkgMB9;pW6tuvoG zQc0hUu;%rlQsr0P7!WvIKF^hWzO1X9%IR?(aq)aKRCb{%;bbLt_qF@~LJg#m@f}04 zQR7PYolPL=b2wV>9~iXCXcXWU@e0g|n!>z$zBqD<{C+(r{6nREEWQy>Dv>)wOuPe% z?P-hMdiq#Fn61NuS@Xrs1;g`d70aXA1^Z{bj%zZ~%k-m%hEp=@+zz^lb!4)gNye19 zN}Sfio&}kId7V}|;Sc@~;^T?3p~fjOx+Zuy)@@$W$@y3}zpjq|5~J0gI`O%+yBz<` zAXw-Ox&N_nNHZUe9IyB^X6C`;Q5`3iPIKb2K;LXk`60HxyCs{vkb#+OW`^ zK~q+G^}c(s>ZR24ccMuW z#pnh@QV>CAM;eV0wr zt$wBAMpZfVYVFzd>z}oxR|(O9|f#;TIuMs+liMleNO zUT;13pup69yAO%EG% zPUtLq!?#r)7+;)W(KT!gg-AgwiPTC9Gk8k4{Ba0!>{=<2O-iVO1y5wEKi5YTe1&aM zF+5u4VEfq+YXcj-3=|}$;+M`mBu5=ADCIFmRzmQ<>knNoaH4V!_88x%7kYTCAH{aL z=1+8!Ob;^vQ42lft8=+pvD!N*P5Te1a7SiR!QNt4%+@aAg(+A&FrBZ@GLd}4ldT+z z3P)(|quLaYQ<;d^7C5NCdEH!j>+QpHfmu%2G)d^q%(eW$GEZbnUf;V>9gA#9c>m{? z^g{oz!st?DY*%@%Rn!5?y&%Ugs#W9%4=3NA?f%LH$`5hLmuQo?p)UM7)G72G9UW~v zajvibSmO7!`7gd(n;XY~uu&-J<*@H;?scd7ovxge8P}Idxx48;y;O+RFEwcgr&xS{ zHQ8$eX057F`j|nH)lc*PpZ0JiRADU2Vv2+-GQ8l@i#$ayOTa*TlHu;NaWz4+60R%s znNgf5&JWB~w`5`B7M{nk3k>{>UeiFp6eeRfCZ96$8Wp$^oN}(c&nm5Gtb?#>^no%&}WcKk0EhB0s}2zGnG% zNO`3D7@fJFOQPY!wcpTBua%<{q0U3}y>s?zd9Vb@#e~*HKo;94i!cNxp8octJVV)X`?S)KGoEA$9O zrow;dWfQo(Q`N<*q6pKXuD!u&5R7htrfvSrt(iS%#YXlMe0PL5q!N{cswKSAiK0T% zvSi|?6+UAJ1Ckfv5#E-^`d)9ndbESL&}wxuI>TK5liDglYVl^q`( zK`2am2isu|G#Mj%eE2N%)8LoG)gp4Sl@Ex{_)%k0GlPsYzv%_~ zhto_I2*rR->a;#+IKY3dqZi}N;G1_62H(6ux?FWcTPH_Xh=JBz+HZhlfettoykpgg z5XNKtRntW;DfQ%(0fc1E*ChX~u5uB*VXfk68Vz4W@-7S^pvL0zm5#Z!N|1d2s`{dC z&TS!SX|g-s$-HdibTx5o*I@^pob4PDJ%R@i9thjPg#@h;#KN>?8f8H!EGO7Xns0($ z);stp3P9=3=icqXE2DCZ&MAb|8loKG1D)m}jii*2n~zF`qrFa1-cv`}z?4`=w%4HD zkpX$LUDdb=i_&H*H@XV}$D-M!j-UgIAO_{b!ajFikJSG-52qm1hd7J2=~F@O`!iP) zw`X6%ls%%JSVmj_f9-vFIFhQ$qkE3%ws&OGpXK!n1q1wMu0^utQaiA$5nD7381h_gyPWoIz!uYwD3S`^^4ywk1 zg5rfn@i`$@HkU>#lmVL8=Wej~nC1dsZey_k|4s&jC=R@}6+mKu?*V-go{pST@`w7d z8(uke?r?lOeZb3;d+&;;s)u)idhfHCn3!|)prOG%{_zm@GXlAjh(!M3dV7{234h;w z?lE8YAybpJLrgWyB-U*_2jGG4g`76nZ3A)UTEzgB7U&u8(f4hrbB7oWQ7*v3wS6F3 z)Cv^w41XfQu(ykAW(&k{Z6W^a4Km7wA4M#@({A=&LfVk@acJ2$K1LuO8=gX_EAYct!zmo`uFN z?F&0xL6~w1h~{S~qXc5kpM(er`tBGJLLke_Sd2LhhtsW3vO_!iJcmL}fuQONIRcaY zK64!)Wq}1nQl0%bUIifaS%&ATfEGUmT=900LRL})wCIuCELJA`b2qS4-9UQ9=38}u zG%isdMJC2aYFd)QAGta-i+`d$1b9*Axey|wO$7{tAv+a%#_=Nft9U||@i*}6NJ{t{BZQvw2LP&+{c#c#Qo3aUu)@PF*bV!(KFq}(Po3Wev6jEpz&EAMHL`gCPvhQ(!{UA&VQ5A5($QxeDJb(ZiAK zLxxJ9>fUhE`Fpz^A!c}a&r|S;7Iq6+9KQl`%RQJa;vqUKr2ZChL>$xR)@2?$y#>VW_J&|1=nSw!HFh>g^m?LCxPoZNRWTLdvS-cj#fX14ocmmZ)a=DPi{qK7@GcBkL|ABY$hLA;azgVrv-}|(h~FL7>td{L z_pJCf3pKT#2p0P;4A|us8Xe)x zR0x`X8P8&Lp4Vh?lnu8QyaIvSvOcy4Uh?`7?|PcS*Abe`1fBnzEq9Iw7Et8qOMVK* zmv*S5kUoRtDG+)GNHGe9Hlfm;5p1tTA+OD$+JpIAJwue14{J{ zA?x#3&(Hf?FN4H-8YJTIo5Xcd-0!+$^3^ClYEk4tPXeNgSY^@M|NZX0rXQQ*~v2r(C{8iYUZ zh9Q`};ty(NQSUW93f9ru{_+O8GBk=7Eu)VO&Y(}GK~h0VMuyq+JP_7KMuvfVsQ^9m zz2hm@7?PBvUYB+VnVl3=Kt}N#4(WOzH~p%%;IsY<8p6BmK0N+8`{UtaA^5yDCDpmzI2lLA+BbGic4Az{@HM-k|va z-`ST*;e3cE`sWy3efJrXfq-0s5>|y%=UoxP$^Sn20Qq7ZH04s3k&V#lpNL@2l&>hgPX+#?Sp{;N^|GFQnvu1Ti${^md; zw8%#iZdM~{WEXCE&t55;R!(3c^k4m2a&x(iYApBlKX_%?^OU5lX-ccoD6K)|L(8nxu^m-&ke>P-2-g9>Jv@X0o&heKb5^%4IGg= zcFAtxp$e}>kQ|f(87auWdJ()$@c4^o0x&LqyY$S+CYchP(Tg`Fm|g|~ z+2+TD{6ebs6A8;c#h5M9QQ+0JJdzS6>}rGisqp+}+nh)d0b691d;xA7(ox+$TMjW> zVKld<{DU@$eObF869Y^S*Nn>bZe^#+f?1Lo@*t7i5XP6Kx483(s}gkSuFh(;pPxQq zLKf3fgcPbHJfX20njdoBjeu+kJP`r`3uh%>xJ$ANYOy>QMGRL2gY%6cgXwJ*Ll}&P zH(wux&%p(xS$-UhlM(PcZ2K&Yg}+iXGdT?o)|c$#;=NstAU6DO2^l}_!t*VIH<5qB z20a!mkzSlOAE|J1^u3i|+zzQ|!Jk%wfW07JaLPWED@6L!k6-72R9K`XGt`?&)4L%f z>*un|I1a1A9iQ!gu&$rhiH2Ut`-r~wv%7%h9vZ{Mr}rm^5MHLB`op?M{`nbq{{=3& zlDpIlSw8^{`3uYqO0&Px0G1If5hrO(9+N655F3xveEfwB(jg7`8+7}(A&d~v{nsM@ z$1&3S&kUaLk9G!t7g*Vg!))zqZf+jah>r<1YNJDDlMGD0Qegp3;pZ?McW0kTMivMT zl$5lzxg^MwO9B~LTe@2XdhRS`H9$Fxi>Tl8x zvJy$@RZ}$G+s#)G1!3J(wkN!OQO@`e`g3g`f_;DfLlXWBi-*3@SRi>tP-NaK?K<9? z4brHmx3#N&klN+Jc=X1XS21!fqtm<-F;aOA(hz`oMtMks@G#Aozp}SAW_a~G?EJZ+ ziiZ$J2}OuD8+qZQ-N7z39tKsT9gjMEE%~|E9k~)Bs2%5|4tY)PrLzYURa3rDPHB{# z#NWBHV+<{?ub+?>&Yvqfv3}`jN7^cIO09eV>gydzEGX2=-6iWJD!y0~Tsv|=;xd?vx=L_OvfM(1>`5sLO zNRw9mqA%~7!yg_YS80wKx;453ZHF!cl0{^E?TPmqJqGwakq$9SsXtba@|!cObxt3g z2o~vyEjFjMS};0GFgOVJI~kqu30b|{$av1lU&o4VFQMDW|J>?_SYbr!1 zq|swqWoPhbBNa3qg3^}x`}R?C&aGZRZ)lYXbKrP_WX3eTGs*-J)xq5W}EpDjFogwbR8J z=${Ll9&S2~BmFR14!_ylKW+&q&XLVQ9{*j1Arqv79TnPGOhVz2j!ro^1w~-{Sz{gqw_Q2Qy^@sL| zUz!s&<-2T+qFqzdVr%6#UrzaWmFeFNhCO4JWbH_G1lgW5om=-JI}CK?W*;b0{7kso z9)Q&>)5o)Y^e;;02Vz68S69rEZ`>tU&o3{*A`pjJJIbmp9N^-!Zk!V zEte?a@J!s5vUL_24CYGg;V?Cs&%l{)cg)jYO)eAq)@2iJGr^`-aM-lT*68ulKD4-| zH|_@Rg+T5BTch(OT^p4G6GOAo!m;YRYlh$C{2i1-j&lp(-7;Bm{<813fWvSr>r(lG zodIIVzImFwoBpD%Jj?wFS%Ui(16}mSjI#%*<4BqkvU!J5J>O+@wIwMRN>gh93Sws{ zAgnwKWo18m;5N)FN#5(*HL~Q*CyyI5)*KeHwM1Xb=$93{2VZ`N^s!~LjU;6Ai!xR=i@G~O-j~h7U zc7V(|>n*e=lg=Tvdj4YI$^k;Dp(OzyThM&07ZJuFtQ6yN5xrh0hS346<|iDuKKEXM zjoh-ytg_98#$^!ZoC5ZzAt)DsOzB(Q@Z;cDFXtx{wSm7|DF$lM*4%SGp!R8(8hQ+m z^iI54?8~G@u;_zrw`>U8dlq8{q7JBV%R;;w4@W(_9Q3IoOcy`;K6u z*sBG519YKBO7{KgE|AGM>PK)5-79(0a;=7KDV3Y3fdHHUY36`%#d$>2TTnMCCnkm! zWSITrhaMgswpHC_57c2J;;@i%8-OP-9*9?+he|nN0PV%_KfcKW)zRMZe?f`|7tWSV z37!ZGUzFVCuHalxp*bO&oB={RV9u^Ibd+lit&mNkTW8w|8qUX?En{XF;&LHycds|a zNuqm6OBu=&-Pe))NGRsbthTTIv;z8~QO zO0SVM7@WNaDz&O{Y!;p;$jtQJ;fKLn3WM6L(GRcQtd@jwR*6ts89hFDR(O{Y7cfZp z8B3v`loG>+B}Ncv5{6?A)GFemHQ*@v3MdU4y+7Tonqrom}+i>1Nw&y-H%UbW71^U93SLRZX z>Gy~Dg&ZP}@U^xwUPW{tY}g#RC|nSFG<_37k4{e)oB_3F^x7MMB_^R#css}v)MDri zl-9m&Pjtnj?BVWQ(9}mAi3F+ zAy6b$OuhUdD^qy6%nsm`<}sPe~s@ zk9dd1iDT9qo%M#!idp2Wf+TncWa%DXSSu8>j~>q}z-tnk9sN^5vP9k_qizfL z)!82vQ2$wc^F8F5ahe5v|8Ho!e8d_=z_eUBJFHQ{j_^~Rx_fUk^ z-Un?IG)Tv{@6kH^%%k5+Xdgn`#$ig$)^D13bhbwz;_@>kLtzA%;!!JpEBVPf>Due^ zwvG@Lv6ZzE$TQMIUGKot}+U_)GMiZcjaEq8$3DIZOw#0du32qtf1d} zN}xTN{q^n|sgsFzh;5N}vmzVId6pg|>yxJy$qEf#y`w(3mZ_N(I4rCO;Po|CwYMdu zAcVyl+D9HKuve_Ype_}cvS&;7_>9zjl9{R}|GZCwAl+lmOPSU)cc2W>re6^>IwJ5}1a5tftna{P7Oex41`}u7RNCg&TxhjUZDLi_16f~0C zX?AfOg}sRaiY_tRz58rhvN4t*U=>M5VA<-IWYJvLV7lv7Jf*E!JbM|$ zOVClw`Lj#xzZ?7WhVT)6fcwzLWxZ2~lrAUtce_RT#$7}#|E6y#W>%YFi6M-bNe*9(z zA(j8~L?Mv5XwX#FZuei`VAMdQO0GDG-Tm_%zwcIuNKyQv()@ny=Pm#3o`1UsM#R6f z=ik`_%j>_3+y6s*4`qN&!?tbv3ecxcoFi{W|L#Now-&m|1FfB-t4qmf0s~sr$0^h# z(m&OrGHm9poGkO74L$`Di1#|sQ-vIG`mwyRY&9c1;hz{=MoIu9GtpHyp2BfsP(Twf(^*vl)I^9FkQtf>3!#6S{ z4y5nOdb;P=!_gt~1z+&(^9N{`y#=W1o@=YJ@SZwWr zr^5At@@Mu6|1DuT*?apGy$jUeIdNMLj@sBQGC#Mb!?y@pR& zQ~mqY^$~&MI2jKHR`bdf$6`tVM%pwV&S6 zHix8fZi7ck!Yq92;aoSWRK2&h=ItjtvZuopWhKeFML!QqKL(#bSnDKsN6gQgSX?6+ z(<#%&-Fbh?<@|cokVDmu5)Ur;cw6uX)CWlrYnSfH@K>7f&+b@_W)!Tgn{aQ3WG|iA zx7%Z5m)!sAJsmZ9NaEnbtEnr?Zllw19~D7p|BjHVoKkHPm8cn)Gqou`U5(W{8sK>u#mUu&nQK_&hBwLS9<7 zKxJ{<(4ju;ejSa8i27iO&t;2yHs<>+9~T%0djwd@I7TjAL2nmPxYPHB*C{B?Qp2=0 zbo2(3#X!-{_ewky=g_#=a}#z3FE$Dt`_DusMGRx!FMs{|{*Zv5;O3?Dr79EB>T(6d z#$jw70cbL+G_sY4YLX|y0%hQ>o!1qyri(rbd7rkLYe#0p2fe!0wB*(CV+SVn*m)l+ z+Z>mjU&%@%^6SuJR_?uI$o{bq`$&L)U0A!r;gH7lm|^vs;$oZGuDI*i|4k%pu%7?meNbZ*35s+nBSDZVHR?a@~@MbRC9me`8tBt}Bc9(Qf zTQjTe-c@%L#U|32{U3e1)t0=SqdFGn7_;SfH3Y4SxB^mObiCduJ4(uwr;_n1J$hBs z^a6hIN}q)>9)p%w!*2|35yxKCIDXg+uA2(zXCqCYJxa>`&JV%w9#edul~l8HWL}i3 zU$^ST%><3D(bnF$;n!vO$-y{oO~QqeDqO?{TD%t#)##5@Fu#5Ua&CL{BbDO_sy{AU zr9EgNx}um!TuS^nv58L---j>)Gx+g6#-2neU`nNzq?7EYLI=g)Yre@)ju_zL%A8KA z=dQeNJlK~=bT^zb8wIjU+y{=VHO6E5)2yo&3R#BmX%o3K|N)J&NgZ?SI9wc0E4 zZT2pHIJNrL@&{4=YxmUitr@O8gc(e&zo8>xoCi;!6+I6*^ zSZj{-=eKS6p)|9?DVFIto;_#l$XUs{F%W9$Lv(zx>DP{(BW)zAI?YYFt(2zZPf0)T zCu^XMBk5=1v~m=byHROuHPhk~K@!s?NB4$5ls!r%se6YZev4D-!+LRGOOW_5%^ijm z+z#$-M1g8cJaJ?!`p4-WZaU7F9ETfRCY8A8ni%{#uSV)O6qG2!E;;~wUS@a&bL z;v?yeGkB0F{fy6R4Pt6ysWD#jcaL*VRo7dNYE3;@b8E7s+w4`q1Pq95HjS;^dwQ)_ z!4;#o)UsJun}E~vGH2WBTp0X8ans0i_zmurEY*Tz7Ez!v)p7Gg&lv7l!TkvhbLHn{ zzWfiBoi>WgxZVznbCSM~4Lsdwe2M#F&#or8-Z*xZ_sfD(5`IuTwkv4hV=)fvzEk?9 z=cu?u|9~ZzvW?8+fwvEEt|dCVZsegXITXCprG_ydF3nPhdwv{^sRZq;!WEoX_!;5g zgH<#fk!xDnbNQbvxPiTB$>4|a`kYwHJ}^))IBpJ3}g?(*9WtuG9Nec*{){A?yQxQ4h}{@gDzXHyuUITFbQKi$CtEZtZv7 zJKHs!W526SF;XmdCiO>EVacP&yyA>m?|1iA6&-gua%dmd5K9g_H`VnD^rovU8|My% zlNQ7eaKvF*(+|rvxiikL+J>KT+cv&hj~{=zEbk?{qVYxli$1B!q)JfRIGJ9)ENRXP ziBf@h+{w6ujcTd8LHxFov7UPmhDq&3g<+>ACScz2vslz{O06Y-`kmyB#(d0Bk@lLC zF=5-BI5Vkrn!%}`!+(wbVOQ);)KOVqmE{&qzP(j@I(g?MY#mAXZ>LQZpVzfN-V}Ie z6)MA!uoCN2+%@KaTW`FN(|D|;Ko}c9@!j0)L8A;HM!lPwi!?hBM7Uj!d$q2Wq`DXL zVUdOims8a z@O(8F4$F1d+_c4yOYPlX->}sTXLjzousKq&BtPu)jU@4WIJ2%o36S@p>blikNrZ;kuuvWnF zUD){89;J}mKG*TW){twbA)VF7+s9~n{pXf(cVV6ptGW$QLp&LgEz~Qb<}EYVkWAg@ zh$z#W8&d26BGe>o54+ZI?}5fDc2@$tsBC&Ay8N}QP( zT?e=C7#E`-qourU$IWwOm$mx=oezS%s*N(4^s&#g%F4UOCop{`q7}l^na)?@tEDI$7J4^W-fX&-uC`o9wU2%I%$DUX+vM)a+vF$TW%#LZ z;G7`^Ha6NhR3@00+B2d4t{2L{sm=Lg3~AHZ#OqB@>?p4`(W70~DEM7Ex05$-#40_zi%#%1bZ<}98;JH zEL5lQxM?hX%1}b*lh3;`H_#mSzL1CCC@4rnpZ56HyP6s*TK?dRUD@rv;Y#$ovIaF4 zd7W3imgx)~14Th@^9l_|ZPpd(LLWZKgjd1rQSaZV2H zbzUuxTiKDe-{^uEb2D`J|&x*zv6yOTAuBQgLe)U+@F_rxri(b9lA4L`5kv?0&ZFo{6K(e;Foru637K zR2Ma*;zkU1Sb59qQ+!@>P0Ox{o%qU@E}d6U{KYbMzbIRbVYf-JjZbLb$X=rn{eg07 zyKG|CTy;%UOsWp?wNqxZ>)1g1qF!2rW8e9&k>7-ao?skT2onPym12xF9HYe%x~Fed zOH&6)bWtrjQ>qI~91EJrvxs^aGo_vDlq%Hk`r5P(%@k-$^oa_b-DMow0>Ff1_2pQHM6gnP`>-K@-Kc zG@*lTtJsU*_Vy&~uv)m@-n();(~#CM`Q?Fhp`fLu?94rxJlK#IVrxF%H4M-6MbCxZ zU`iB8OE<#U=_bcWEiW3>X-D-3mc;B`9*DP*qIfC&zw4743}gzL5%DX#pqPiA6J>waBIsL{W!lzzo z-zUlW1ibI|AqVXO-`u(69`i6O!@&aA{ARao{lwL-W7K&^UKmOAdD){cbVw=^?F#p# zJ5@6JKrgH=F&8!AWH!p*s~C@)74#dFPxioJ6(53qlc(wv>XWig z8okdq|8y@K6hR%jJ^-imFxUf@DQlS`moN6df#xW_a`!6%?DY1lXJ~73K^w90u&s*Ku7;B~- z+!V0PH5?12SvA=ibcN4cwGK*?zo+u;9ZV3Reu_;(2Jfr2K4nO3r{S(xz3tIox-7og zY;2iGTwQFjG$&0M8>ZQ)^E>(bljb=E!8X444n2hRV|M3@ww{Wzing~yQ&nzlgEp`5@C0>le_qVv^n;Syk1+hU z*AcUj3Si0Vu@l-9O@OoT&0TJUs!jW1}lamt}sx>?VsoT$e<{vth}`bb<^sJ-!?ZZ=;D&H4&0J^oKGM| zrnSWAdh~Q<5TZYB`n|ptDmdeEkw;0t{e_7&>Nl-f&8EbZ^xTAjd4B1TNH{rwF|9<$F?QZ>RjsF}A l|61c;Yy5w^Zd7kk&=?4^e|NCY-3CAC(`QcQoxB+MzW{j=T%-U1 literal 135225 zcmeEuWmr^e_cx4#j(`d(Eg>91q`P4ZKnX!o8l)SfV}LPGNfD$=rBgZwP*UmcmKX+x z&Vk|Gcm$8FO?|ZL%^;-A(?N^HTrO)A0;^W}poV$1TjuH;e*)|*; za0cWw@EZ$P3mV`bu<1?tn>aXm!36sTcsMvvoO^d}s<`4Vj-HLBR*k!k?bZ_ZV7f|R z1jh-HFTM%A4AwXUCKfR2vA?WH(J;p?ywYIb6Sch|WUJ^|a6M;!{9OIbI~};VH*3x* zLXGq;=7&6AL?&$?9;hk{l!T&_D!PZ4Fk%Vqtq7fxg&v`C%|x4n1(ytA{A_6ltbRrh^j~RJG20EcFX>AjC z?H@@BSQqZF^}i+hFXR801pa0G-`Ur{8vnQUmH4l#`8yB(uaE!Ri1M$i`PbEa4@duI zHGiiF|7JCR8*~0mC;v8;`!|pO+nDpOtNGW}_+3oPP8(Zb{ae?f4~95qdD9oMA^3`N zy>B>KX=u1+&wJVqih?v#&w?R~a)}u>w=yUN3Ra@uBGd&~@A5g^msLG?X3>gkocLJx z0yyCrLANLjth=uKZ2?}BcIl;lS)XZ+ytOPou0srIa5)x91eYO8ix=eRZ4qPZMQ@wj zbbMRIGcWNwxr(G9(A?KYJm$as9r=nmAnL$o<>uS!DaFtGQ-gxGvZz9`Go0Jf)7@ve zI83q^TD0F zhBL+ET zvI04*U^BM{&-A>?VFpK%iUeCiy~r`QY9K^Jf&wq8u0HwJm#r0n*TN?taA7K1MLsbf z+z>z!iq|0@#VtvsV{E;rv&H{p#-`L1R=)lz&>U$oH>&d_%Af;0D@Pl8U-*%d@b=E> zn~qOYvL5}aVw{ihg!{zeqrGbGvYZ#0)-|C(%pyNF_e7Ll=7)bgcRnw3G$7@ z%qO)!X4$bvhAt@VKhB7LJdv0sn|#t4gOL#l_PZ45uoF|vsO4x!_2ug6x%wqB#7N|u z$j1tD$wzX&ixgu?QB_r4$`|J$N>hP1HAR}51#Gv*2&Br#@FzF*kzWf+@b&fOb(n2g zEX`11I=GMAEy_k_P^l%ycopQ>ja|E?{|!>-J=-pBBUAougu=W%y@(;zK=an0A{=2f z6x?c!*xc{w%oplyan-om+(Tv@MagSTa_RahI4kCx9$CJ^SVG1EV^Ut~0dz&1!ir!1 zx8FNRH*P~GMsRHBAdy-HCi47XOF|wYr-dF~x3zD)HY26Dx}ggYm7JPzHmxq-b5vbJ zMV9`JLxtwu`8efYRnuGIONx^>>Z#OHmPd;j`ASIXXX!~bCbW&wgbYRf9OzI(i_9)9 zg;55A#LR4p^uS*BUj+iyFTumd<<_$Gf`yysI43xCMNVV+k{8NHbJXy_kgzhShpYv* zbv(IGQotl?t^vCn4DJcvW$g9*eI0#A2qcy&acQ8hFPxM^HFvHf zSw;HE&ZaP5ic59={Eg*Tauwr2&ub!6h7a>!t1eWv$vLEYYISJ4+1$%!8SIu1E66~@#?$d2(0*$-ebocnY^mJx|XiFQLHqZ>co$2C0o*2gj6$pG< zMIY6dJ;+54cQkAW`0`|K0fjo_NPUA!1Ej!aEknEad8%;#d7FWUv>K|=x26qmYv=>J zjvKPFP(NtBr!Co2_E)X=Dw{a-=`coi?p@rEfUdwTG8fT`=D^*>m8k? zqSlDj!Z3V_fvs3A<=!2H)UAR-__2=- z0rip$qHq@+Y0|pK#^Na{DZBP}*Uu3PL5nl4cBsS21h-}eQb~RlD>QK*Pm`vc+o_N*IeItu60bToM)?ASFarwiYbc{{ zqp(LSo-oH(0WcVw0crjF-&dY(i7KGt(+I;iM^% zTqgmD1HVoj)_X3~^;c=&f{a-^_F9V4KYS4DWiL(S{5G}7jeeDKiH=hRWS{fWK`K^c z>dO0*hX{jxYRbGyONxtA_$>R9GBPq)&DeckssN%B0@=$+t!Q8QV4u~%l^Ir2vu>%? z?P9B?JQmmC4ibZSZWog&El!_o>I-d<~6`wOHn4Nn)#%?Np zEGIbp5}=Y7nw#`{e@J~G&k*zAsU+}XEBF!10}WOC%0Bhc?C&u!sLNV5PUJRGLI|Hi z9Sp|9oovV_vfmts&EM^W&bi+1YgvZ z-MM<)W+!(CYEBrht%ZdQsy|nux3||Eh)FQ*p+zu+Fd6#$uBjSxl;Qz%wUiM!Fwv1w zof-7XX8b4%|6E0laa~0HW$U`Y)@_?9wMFK7)+|c_DOLp9N2B+DvfU%RV+rd8qCZ+-Lt@~6ECvfE<(0s^K{vkgJi!!NG2%J|gSoeiK8 zU>7l&9^lhP*;zlw=5D7a%bOz?X0ht}_dv$K5nH0f) zM8aS!SPJd~^SW$xN9Q<|{caXO|GTN+&X=P@A05-Pgpp7$G{D1CYI%OguK>GW1Ky!7 z7soR*n87F)+3_N|%{Xau1OV&Ozf=idV6smtZm)-yB|4NJ?;UaD0xxdzoQ z#alDvm+*a2A=U?^b-gH&_6mgF=UDJptr~H?v0(+&e?6?aPJ5&S18Qy^@-)x7Uq~K_ zgcJXF;tiFv%7a0feP?4)3{iv4+Q53VREq2WeLbr0>#4A@u?>zKsq(|GS%4LlK)qUK z8&~W~8uvUnK3c%$+)B1(BSWEX1Os!;9vE}6V<-5lA^-4AGYtAIo#IxRTU%RJXuvQ1 zCrO%(47rUh9bBCKO>qq1^4ge}=85+GV6%??>Qp__r5dB7nfFN2Z`|;L*ha_3f_im% zm&WO=Lx}^Vg0hyDmfE;^YFv0jAeDf=({{;wX&SfR#11yP2d4-~S?`{q^|bTOyun`L zyfoyvJ?diWtq{q1HSY0O=K5vNpSr{O8qoLCO3FIF*vhfPIl2#yOZu?T?1B5vf|}J{u&sm17qs!Stug)6m8W(k!A=J|jS_93*){jf%uKwt8S+yr zUfuEn$+NSwSt--O5?*!Swo%XZR*jf8(e)Nwg4l{lU#fgz)3#V(JV$$Cm_YRS0^)89{FYsyGhA#n_{OIC zU~8&Jy>g1g97uBOt4j`7YEE59N|uc<3Edv|K6LE3dB$48zxwYvQ*rI1d3Vaxn!{Wh zo7dJaM$DiAWDa-7golgRKyx|O*uANq)C4bK#Noj*2NrSg0<&B#aWE@T{e>~XZ8nO5 z-!A9F2L?WVev_rFj0~CS#^CWPmWoXmpxm>twr*WW%gt3NDssk*f{HkCK~EX&CN(^g zhN3gFu*+lRjMhU#;%2A?`T8P0W*yhQjj1d>4054|A4fh~D49Rb<<^`cH5GqJvZYsE zaq0O#vgC(`o|dMgDQ>52lRzegc8z%~$Jvxl`WD(ve!dL1UEMxXz58ODJ-PxD1bRES zhRU5R-6x-u`8DrPoRPSWZKK`hEIyPMAF}V)aWTGtwH>PzS}N+-Wqz`BU*ce6(IM|W znM7DICekqI9@k~7LQHyUybNGFO}@`Zy`s9RG3EOV!J==3*mNreQNxRa1w1i8A}S&u zFIrZ(Fg4eksh*l}uxN^mGL65tDui}ui523Z_E^kA?o0;*!CCAA&L&FlP#Cb?+su6> z_R|f#mVH@;<6f9#hc+Q5phC+l4ML0WuLgU!uqME^1{YCgC`;Y-XvYco8D{|x{%nZ@ zf4sw$7ZOI>vG0TV`1s6$yS@w!O#9*MX}it5Wk%ylmVJ98>xYr#f49;e?7csIdAt1m z{%*pdn>$igFB*nkk~mnBwM7Zg9&2XJJQ>kVm4;zK_Lu021Q*^-@tv`ZZBFHU(t)f2gauv-MRo)pOL*5wyklBf`GjpIro z66_1X7TQ_5J9dU|3^-_Y_l397dYuMcebUnqgxn`y^7ZHo1R_GlM!bW`<%+G=7`>)F zz1WWS3p}ZPBq*Ky44_n#pk|Xg4#CIHXWqXCDD=8N1dB(rbpfSSIID(|XPOwJ6p>P8 zyyw~%$4*JI?t z%JnV^)7Z`#Oqa7XQJ+)4P6Xawy&M(4)jYSYi}!{xz_$H4X3(LGEO@@21>LCaxmu5$ zczS{TKsW|llncw=!SVK_`1A}A;=N$8t%sz~Tn3FJk^#9oX|P|!_nf90Zvu76RA1k8 zUG^BSsNGw0tL1)zUaKjLtg;y`x1y33VKB4BbR}RVsfp8eLKF6rWSxjvDR8}s=?0Wv zVdpT=_=!ZvF$ihP_Lqq2{h4IFr8Ng{On$Rp>oHI06d!=ZB&L)e5Y7w_E_)whn7KQT zW@gBlL8}a3^NpLOWE`9`sZFnl3F&ONV^R-cov|A^w-kqw&#Qqf>ndz#Ujv!xEg*5Q zqfF2xI~P2f?es9RMiElSMbVh@=LC%i0*AO-nwx*&4xJgCdmKgH%J}4syYhYU-EXWN z*}OcOM$i2xN?avP@+(K|8>#CP%bISv^gNI@)@V2{IN@UzZq^Q%TMXX&nP#g)8u-KH z+RrwJ2RYcW!8KxXGF0%zn@@9H>sh*9F~vV@c|&^BT@Sk!QIJ^bDaN9fbyx3juiyWs zL&*HZlC8G~egx1nGA@wxnd(*7i{nu8JnSo}QYDP`Cv?iIQb#PPGG8RT^U9|#*K0FV zgS33=)-_t-N-r7k)=ASrBJ2b?&DjjE(bZT~wGvo5JRG~h5$?V6=}ZN*NaHuLVdTYt;g{2*Gi^n1)mx-;-*uMj~@E$yt_Wu zoEPifleCfUZ(=He{kCMwN*GseFRSJEPHI~Eofu<(3dg~cIP%i?!*CnSL4zm%GFU1A zdL?-E_LSYekI*9`-e)Iq;`fVzn{=RB@tsc>Qbf(zOkSSZv_)+8S&g|$EERTlMRFNB zKP-0qjMCjMUu=It>s3nLv2~Oj$RfCI8vuB#LdexnM_9t$L@G zQ3gDqhxV|mcrk4Ew`Nc`c~s~4YE5m6|p@gB1vn~8r^H` zg@6Jo%|oh!LpON&(`jAc1TJ}lnE z$(^s`I{CbQqN=N=?8w&Z)fy0yT^Ps%1u&b`gmzeIf0xdy7U$%H1TI4_{+uE1t_67E^{=diU=)>cX3=xF0K;H}k@>BUYx z$}G8N)8)NnsmGFXEqJEbW;FVdj=g~=W~n5#nZ1G)NVeG}iz}sRVTH22OG``j<{{yt z9K^9(fII(f)UjKhdG?EJp#VFUo}68DY`4v2RxjE05li0#O?qqhN*Rob`=_y?Ax&YA zEur{l&BGLA8_Cx$Y&4%jtt~%_ay{7VuWrl7ZICRwm8gXGmCFaca?oImYoFJuJJ1m= zzEza0QKMaTuHV{5_7J7j+$}rXllpX!9xwV(cYE!Ws?5`B`e`kfYrd|5HD%X<=()ge$4=A5agG#~6CWp! zGT-gms;^zQr+rsgSGz_GCHrKX%pPKlmX_Fiz;8K1d$x974}7|>zu&YrH@q*@n?3t5 z@#*=mOl0E>TBD5;*ww{lQCXeUW-*v zr@j3-2|XYRWLevDk>9il;TDZ&^!E?u@SG$-8q#7O=Ca?JzQv&wHTTKX-kzMyv$MEV zKR|f;{q{m;?)?Z|ck}Q;bO29V6L;Y#kP&DUOuy&OZlzEDdO6o?&B(DX(8G%A34MN(WO&(cdcBOABlFMhcIlQ*(jxnD!h6D$3!mg65d+O_yAGm&@U!szQ?;uaK3O zQj#LtXtkPfqPx1Aqq}=0wBpbn@ADcZgGF3#2_Jq-NBWc*c0bGOMMllYBlG5pQ*z0| zJq&JZl{8EGB3u&0#HjNQcM99HxXdTlTajo98=3z7NAe}>jdcY?MtZ_q0s3Ld>o4x)FUgRk^gdcw@-KbI4>XOb`*za_DfhYY1jO*~h==Q?t zKRGHBxOum*x85t0x>Ku1DjSz_KW@mn2uQT0?cHgh>am#4OT&Q=L^wblTv4x>K}<-#%|rh(r!eevo6{WeCo{L=(yp03 zOj-2-%ejlO+{x=DHP~W<#Bh|Y-f4@2pN$Q29gxV_46V)qRWIU|<7kK<#vuWt-PcZK zTgm0|CY?7%Qvqjue^$W%laYPr77&I7JTBgt4JLGaFt^?=k$_je$xp|jlscE*GR>Ts z#>27ds4~_DarWPxLMFM^Uy@CRJJ@=y$E?@;Er+}p;~>+vyBV&-jtz3083!Cm7q!bs z2#gxn(rrQ;Oe*!N5Di=uqBgn}+EXsnNrb!+)d2gm>Dp!`Xq|)#)AJh(S$%GQ% z>bphpRx!3li8IF84b*M;OIY2F@OCq|-~dlJsj~~YNC{&~ec5lDDA=aSHq!8l9~fSR zELB&WrSTX}@9ugrHsa(u5ApTyM{>kyLv*#Gy|xF6tLojiwU zJ+p2uJ2-n&}6c^9Ay0xy)XN`_WbB3`C3{$?rAHtap!43=i=dSjlf~ zGkfz3NkS(Nv0H0BJ^ANsO|JTI*(x|)ZCZiNwcb@`bjI4Y(S(**Kc%MBoV^5YdAobM z=f_daagD**1X5k)HKCP3?sH9jg}x;hW8_%1^K?me9K9V^pXF#2j~*&MD7<^?wW*80 z$9el)2C0!(XLhG6O3z$mz|>jhBDQ3X(r;y>>}J^!fhO7CE|V{Ijbq;i;S|DF#`R0e zQui>!5%CK=amPU^F(2>r;b8;IuuF%>QQ~O|e?gBK*1)>-aMw|x-@NbuT?M(@#Gt!sh&jh#g zrkqI_e1o<6v~z2#)PAt=NKwcTDa^2$6!fX(Hn;k)!pGLKfYRm`gKcwJhLApbZsL zTeV6yC%eho*-y7xTiYm9!yFwn^~nAB`&>H4Mh{;`WgET5QfR5a#ao-X>7c+m258vM ztm7Snosrrts)X##tO$W5(*)ZaMx7q-hSjdX^!u~k4}V6L$01mm@FQ88TT?nu63Oou ziQ}MTjW1IMt8s5PQT=4q>^eDu_Dr-g5p1nEC83h;o4r{Y1zn$?6BM{Htfcc6=;(eb zcS5ZNi;SGP?!|SN_B||y-zkV_741?jg7|igV-Oi#++J0z|LMu)+)`7bZ9N*dZy~Jf z+C3B(wxl1y6}xB)Q2KK$AB5n2b>>F_RIsgS48@@g&a5_!nAS0z3hV1{!dgnxE z7x2JsfluW3!%Hf!dH9(FXD);F1Ge6n;f=M3>dMCWdPz;fUP@!^{%(4O|9I0aGPpkR z1Sxav^x5?Icu=$Y<-w7)jTtUpOYpR$q84E=r2;)-iX^wmbZH&qg;`S9W@OJ`ko2sS zu8`qKIkR@QbPC^NM^RZh^qegQGkz#8`w2@6q*2Ikp$hk3(72mzfi;rMf+K(v&Ne(L z3!KsM{b2{BKkW$9l2A4~fhZtY37D(B9Q{Pa{ern<w9$U_fyvHiv3RBwhI4$!i1zUMc#f_y8Nyf$+7 zty(?p*dg73b$b;O%4TndN5!$gq2fFF(ZQHx!d%FbB<=|S%Oa8b4gHMdH!9Hc& z<32!Xcx>?BFZP%f_p}`L`Dt)9E-~F7c~?w+tB|+G96eGd(o#@xRzc#xaTV2-m_AkK z<#qjBtjF3Hrfw`yd)<~-5M8}vuZa|cPu1PrC(@W)t7=^4Xr%G5%Izcl&~-_HP-%CA8N*;o+C$1O>K_sN*>j1B4xSZ z`I2DUtnq^PPVM5Y)&6R%)7BkT@C5barmM)K)*JEb{&B~#kg0uB35f z-23*Q$jPcCfFd|wU6gGrKLWv8QHn-gxWtfb`=N|lUB{`{46Qr3Wu1iA!KOR-X6#^4 z66uYwbQaEk&CD)iEw3t^N904A+p0C(#aEIme>l#@;rL~Ozl5mHxyAo>q|H!Qc-79M z*P2xZXiqz3cXtitA#Z0`cq_W5Y!f;8Tx&hofBA4uq#5WzaM^#!2;Z20pIu*kN2*xV z4sCoUK*10n*XaRv#mJD$3<%Pba=8_jI;4VR-U-^|UVIl~MN&C0Tj|>Dx0bU53mkMm zpHzGyY=l7gj0RS9BGgrJa!yz9fO;bnoky_d;%3C(3}| z*WW&TU{kRKv=`1YIiB73w{ir|9u&7=BMC3xDZI$P;u8{W-B9=5RA;_GgIPYrE{BafypnTD#&xkzphJJELp;46vQNihe}U;D(Verw zKuad4xd|#T?&-i~hmLh<`$C4=_VC7#ZNun&xY1AVq|}NmSMq#6uW+stzx?T`*xQHv z7l;B_J(F>t*dH>}7Nw@nRK$#pS2?OjnE!O;fQx(sn|Pb+Y0RIr030~P>nt`%Iw)N8 zE@!G5*C5g&l!%V#YTqqHj{&x~vG`_QPVS^xywgRxk1?QTJ+pgcdU@BWVG}!!XqMC@by_awK-yt9n^?}`6waQ7 zjmsLt><9Y}gDWX{GH zF@BIk&P&Fy7j`}4(=o>)=rE_V8JCNF; z7q|mYJU!6feo%N}e?|{$yqD^|m&&Th#y&A+o{yG^#}}>gPwjTUvFj3|={Z)}W`O4r zUE=62`kXWun3UTXFbp=XZEV0()V|*Ee1A@SyX-PqJ%a5)`%zbB@x2gx7dv93YZ1h+ zCm!!Zl~Yq@c3afVWoyvXG}raT__90djFsO!{Z)^y!k z(wj%KHC{}FYD{QMTC;3`75uE;GV$xX^%A_c`q%lZ3+VyW%5f?38TTyti<1YtOYjD=H6`qpkmjQm z(O&aahW-U|KrwK*semCnM5AkCZ}lN$nC-P=Jj)2$8_EVcFd5tZ@3ac0GOS*z4VY0#U^5y}OYGn0M9t;X|*Mx!1>hJQ5107E1 z$ZOo$=Nf7!j?$lQZ|t163H;HjmOD*=>W~jln;=8IrJziq?P$4^dFOjc<~Ja1?}JSd zUWUU!#>1%|WrbF+)K?ytIVx9f`PJkv$&U1w&cFBf(^#0oKwkPon^qnkjiJd|iK|#u zd5(CLu6!owh92k+0j+zjAPMZwV!Sk1tU)Xovg5C=OaBSFfvC8+eg8A#MOI#Ze)LBx zBhi2?Cm0(LN%i^-e>OBaIXxUOBdH7vXq zzem}m6z#-ufpk&+064Gl>>HnVps6AHDzd%B#GC@A4c6j>cP~jhT|J zhVK!lVfdD?J=xC<6l|+pZo7|Hv)!lN53Wy7F5+Prej;Q@0C&8UPf}->|Axqw?j~|)hHK_E|UB9s@rT2nx|>A5M4AlIG!KwD2k_w|_Z5#v2YEU~p; zRW)`qx1Gkfy`TD~4OjGiqd9*MYU1nR5v3O{0^N$ez7i9P8JXqey2r$euD}tF@$fZ0 zM(=TtWkffV`0A@g-CF@Up%DW?FRnbk>zpt-{~@@IX?e@TdykAI&_jfFr{+vPETb^) z5~JK*?^QDI5cblECp5#PgG3zF`)gLn3j8KkbUo>dP0{Vk7o8ipSK1%#Qd$3Kq=|DU)t{PN#dzpF9n}Sz%BimPYg~`y_QzgJ;EeaDG@1z_nC&I#o=i`XRQET& z&g(=&KKelm&@!S~G`ZODDpuOT$tiK!%BitCBMdLk%*vi>D7TcZ3MwuxUc8AZ}{`c=m8TF zKw((fNK}-t|Ba~t##-#cDgaFOSXDtb7TF(iqh*(}+8&kyZR>Y%#*>K$mBN~ikrsm+av5GFI4zKcX_%Bi?lkixN!APl-O+Ak zKAjX=r=AD(HD#!r3igWmzyJsPPX|B3p@MAvb0yuWFNvj1epi8HVNGOw_kLfv>k}4s zBEzv#9KW9R5E?OdLIyt97R=GCL6c)d#4Snu>Emw!l%@qNI)<2tb0jNxA8eIWw|YTe z`ge+c*8=E52c+#YD_*VjM)eqh&2anmtGNvvyT zZsruyD_&txH^ObTT=tXzI!}fA&akaUC!i1NukI|7WZ9qWjzvYyv?4zq2q(yl+UD-D zvgo!u*kViu35W&zg+wh_4_7{C4_A%Ur#7Tm-?VfacgbsFul90wGfnA^8-n8ZMEhX(hJ-R}mQGL3hAR%o=f~Wz%#wj;(vVYBB5nPXvOB?@`iF8pm)K_3YC9 zyDpcNO%pjBK@I7iG&hHLIJ=4QHeLq~t@+ZW_h+I{_84s8Ws)?x8p2Bgy8%Bo-6 zLEhYHa1P>A0feq{(IQfX8}-VPAzVx{;LLiKcx}I$veMVWKqjgLl>2VLIe&rrmh@!8 z-81o~AN{NDY{l)a&&}zXI*r@ap3{(EfO3M?>0LR1x&^aXpj;@FKPpKyVqG#Cj#$H= z`i!WMbF#b1GW9dP}eec0< zj?J)P(DG4=;eFUA);+-0Yi@bfa45odp`~sxc}*N^-%l$*!ZVIoDl|u$K9_SPAXvK2 z#&kuPE6!`<+eUR&5o=M^z}pIJDHmd|sZG3c2Ize=RqICJWzIAsRwbYAFL!MlXO{FH z=AU9r8oD)AUO5)I%-1N-9*0m*P`}KL5zBM6ByAxR_r}a>AicdSEKm2^9S#g{%k+;H zv5y|sweg@fEs{SNG2%M@hgS73NzKvlES;TAQ`!q>Nn7{sn69T6#0LzE3(Z=Af;sBw ze4Mq0jXOuOq2xL-&}uVykqnf^n;Q6QR*yKaud?iqQtJG~A|QKWV$&%vd}s=8{9ZBx zlL*#+iHN0Ut+BOm*6!g@1#yl23|^AmqeK?3Lo!j1rjB$eAi#OqwNa#tL|r??80P=H z)Gid_2pJORUYCrqDVquRd>pY2QQ9A9 z6c4Wg=7LNIx9Hitbm=G1h28gMi51EDfddSZ>#06vo>Z`<)XZ-C7jiA2ae5rI^kJp< zUgs#Pi|2WOnHGLj-)vi&Y_5cXMYKQxPlfSx-pgBK8sKC8$P_4~}Rx^{pQl4ja5CLKLaN@evpU9XNER!hBL zg5_`ikT49;gDxnJcljK~>*20bJ{kY{HABo$*C$MXyeakY>n#r8R0#R<9&#zci=WNY zOABZGenxg=F5pXAQu-eJ>Pt?B3x2_(e??2%)))SS>!~He; zVIDmJVZc?h3IY><17r1E-MAEmgt6SPNv;;lF>KR7YQPnEr{{jqm6^71ZKqzXm-8dKilyo*=R6{9DX-CWouf%fop z*lSXiEjMzlz>J?~J>46(zO9flM?f(k(@YZ40t_wq;}k$tnwtYYE9R-|3JXvdZkKv? zonLNewUnXVX|FeTGkDoodBuDF*#^gQl$N%wxY$^~wT1lVb7-mkW*pqB@i4eiNx8ZB z(7H6MqjBA1etIY{+k7FZEpc+Lzc9%`d`~5B;r2Fl1YTqPv!*teF$r;A5@VsC6$W%Tl?w?@)0^6LUMPEb};kmC3v^UOmBo<*w%4c z@hRfPid>=4nHKwcEkc`H-zoFuubC^+-Skxt0&d?`@4Tgw;E;tif4zPB?}vKPE-hS* zN!yikVhly;Lz2K37rr7zl^#$@{rLV0+>hw>=}db5^&An3+8`;DVmRxAxk|SN9ai7xf$H9}V9e&_mHu&IHtEC>@M(H|o8LBsd`%8$ z532+Z>Y2OoKRI0L`s5MJcKV4V=22vC%<|~#9PHpiq0weBw6JpN@Nui(MvSyT>UNv} z&k~QNT=-44DyxpLXh7adQjs%2Zsdv8WPM^c+EhK#vR^|1PD_fX$7 z;qqKuPOy?YbRPZ64tHOL5#IJ#CGqcK8p~7HbDE~loXXg-WDOrv@to%)_bMA5Ll4Um{m@g$bR)U4P zc)awH;&=+1HBy0~B!zaYjQ`;MYh#7MqM9v=d1u5kp+C$@ejiEgykS>IES=<1jcDt-{F3VQ65W0)lvGVq*{mK@SgA zvK5Tv3~4Ffy-S9=5deU3io~0KGOY)|MuHB7f z0-Gj%xG|>PeOnvz4q6x$J=XPyv=OgJ?j(+U24=kU!ysW=`4)`%W~XLx*${noYBzw zHGE+p|2?g&)-xOcg!z4yq%N_IgdbMQ*>gmU=E&Z!&;GPw_=BsjO<#R|swX9x>i7hp z71Ke~Wk zJFB9g``W1lz-zwkv4#PI?>x!H+p?Y* zBMM2q;2#a(1q?1X1rP3%(=?zZ2fza6i5d6~8`UicM z1<hl#6K*e2ImNU zbIQmllaj|o&cfXMo>s9Hvi8k)R0VkIVW1&@kPgM1uO-WdH-f;VKUJu^aiba}d+|C0 ze4yVjLH$G~zhA)DE~Sa=-%ig+5ierfx3|z+x7&4s`ky5t=Y|VP*Zn83nKtdxdVuQD zuAb8D?Pr~`dhjV>p9v~*_&Zs4aZHqi!cnF8S9_{Y%U4maU#|NqaQk~wG zDLR8yU6?mhXm!zxR_cz9H{pu|jG6R$EY;K%;yJ&eQ{z!nBkxWOiZJ2G9ETL5xUT^q z*5o3~eiMzXz0%S_fYe*?CSI~eiz@~8i#KMN?a-^|o?AaO^QgQG`taay5l6XD0KzQ8 z#}jmaDJCL0r2;FQycNE-6c7$z%~n9q9d7*~q`X#Ka0R8va4a0Z-zL#wKD+JQKAb1&~DR_d~ztouNM}}AI*O)2xb9`I}=(VEEqT+f$=fUi_^a5 z)LdG3ZvQF@=pi@*rF4*4^*V#8<(_pWTA{UiSLBn!L1^J+cqfx0B-;OkCKW-`pYFPr z1(@1teDQX!@FcnKQ z=eu{QTDz%-%eo|^5E%LY-5b@^961dBpN{D@5ziQmGu)q3{g!DH?VG|XG}&yOwpN4JNZ8lfWUjd5jA?no?4T0u(wHzeypFcz{zt$&%{|35|AvRSHHsea{{uI zhK2@Sul*ffZ|uHT-y<#n>8e?iC)DvCK#um^3Rt_Q&MtEVz~!b)A_1RA4B_4uZ=H-Id$QPH4C%d}eHE%-gC|TGksSUa zN6wnSTidm%TCBH4u-w5_Eo%1ki01PjE*w5eOe9!B$HY`N}=?_tAJc12#9JUvj?-H;vB;Oml35+is zmC;q3+nDdT;k`h!n?Ho}gQuKbctcuB&9p$)7wpY*1T)Qu-czZVc*6QJ$9-s$0kJtG z?E|I78|oJg!)q=QcWNy*#MZc^0JadRuTKDp9hRR2j%HY0a}WrzW>H|0 zSQIW?VQyg$Iv{R(VH(K#8c0wHsKd{4Q2MK3{KEBZG%>d6?Hb!U3c zoT?i-u)!+xPgmyB<2ZLuJp1qR51@nxHOv4o{&MkVaT4lGN&%kUHX%OMcF~j-w?6;e zDCXWETE2rmV$cV*#3HWWKL*jHyTr0ApACQ-QzR2a3gqIqABBHGNU=Q)I{y5_SDhoc zxz#5~yjG>FeYjf=P*_Q>i~!h&Gl?bu0;z)vo~>14(x@*hE9(cS7a@K$Do}g3ZM~-} zW#b(OEAWJ;fPrL*D0eFWeipsyK!PBf7d_U!-yM7gv~iC~3B}W&Qhco>WlP?lN zD{vsWE3#1K=e3lUevuXYv{v+yWuSqU#N*yKz#Ls5LvbLbl)eX?fh<5lU7NdqC2P2L zC+1gkQc3&{H`Tg&TNMCa%5bZba6c@v%7LEKVN}!wsXATC&kBhfq&H1j_(Gs0fAy(R zZ`i5kOa{$kRCk&kkf)uhJe8$g#+|KEz*ygr&87XVCtbOKPMBi;G`Xe|2^$f`EbtwI zguWadEz!Od`IvB%HdaEzM3pe8rIrjZpK82^wzgD~wKSOT%eb{2IHCCfQ}c3p(I4|I zawub?PkH<9cYoCvu%l*4_X|r;qQdco6at72HwWsnv$L5&^YkB?^7SQOwKxEC7IH_u z`J$qtL6#IrXW{RM9#cE7vL4&Z)ms8-0)UN@A1r)G-;O57b+FfFe#^45Kh)<4Ad|b~ zt2NXGm(_Bf#s`bSciQ&{8|A-%X18J&METZ=Uy~kd(63(n2bUGokFa$1%>sOU$xf)g zWMEb}1;D-C)DF!!s}f!V%LSbN z%gqr_Z*ZFKMLKp7hjx`U3MAnb#HF4l(VF?8C4a+|`4<2;yb?D}Lm=<{amroW3rr;$ zv&p+uuR;?&;N)mQZk|~UN`|}apjmLU)3ULJIy^O;KUsu!C@fxQEKDueRz9Hx{ zG_5u_nxljrqW;UbH%5VPZ#eD%-`)t)FB`Ql`unf6d^g|IT7a496J#D|=>N4IIpU#I z7s3`xcKx3pg-uif(m>U(;e86fdc41XVH5DcAn|{c;(vJP|F=&l z*3jXzY8A8A%bs9z>7c+@Jb;f>2o2wo{V%(gdRGI4nTV<6V-o}J<$MhADlpdi-A~}f zUls9#99=tqM0IdJ8hQRiKjHbnCmhuOQbqiFkRw$NHBTu!LYpKBsAGQ8rV;?Cv)^Ek zu*|Rb|GWbrEetel8g(=i?q~i-;B_qk9ktmhfs_7W8-EsSz%6)}m4n;1QC#!2%Ae%` zxh~-1YEiaJf8FEBZ$2pMV}M7=7*Hc?GtKJ~h@X$sampQ4p=ZY|eocRW@xxNNP?y2k zwK4Q!g(uDO$+AF}lmqPf)28Z4oc-mo?>mP+Kf4iT&2IYIi9KTF=a`264j`m+2?)MF zkn$gPJZ2PzY8CzPS}Niv8w1FJ9W$6Hn?X(l_m>N38NuR>VeK&qjk(8X|3I;?G+^N? zrCKb1dE$ifn(LS6Wf{w=T14>E9eiBA0k?#8=uck3`RzzLu_O5ZO*%19y@J-6 zgNQ$I*bJDyJeOTk^j|^%pi{sha>+R2jZ@CJBk7Ni+W@TYumyO0kWxVR_k6;EhtUro z=@UktSYs|8n6&fau)X;Qf40XCWc#$sH+huo#n4g9)A#%ek7hzEY<8e{a-r~ zCrtk91!v(A+xv!x{z&UzZ~OTh;1vL~*H3?R{{sX5>%-)l0Ecg*tgU}hgP$w@#ffmX z9MR-<>fwJ!^#5Tw!fe1g-)$E2|6`g=j(kZaZq^Af=`U3Li9la9z@9qXwW$7!p8WDB z!UBLMgE1Vnli`9tt@szF-3AIr8#6@q&*7y%>cH>YtvPHTi>r~=81pzhG)}NG0`vy{ z@y5$1Pqd1z%hD5SeEv*VUG<(0aZ+Z6vd428}gsl-?q5B;%99FTUr)|QtKJlj$>iuR&y41&;2Tx6JfDa86l&x=ZT~e zDti;MM~=OvP(oJbF+;Y@Y?AC_9D7T~F|#wz_jw9eT~}T2&-MHJ_fIz+&Na-l%X>#c!pn%HFfZ=`h_M;i*rW*v2wL7-&zN^Ekl>w5+i2(v zwsv&X^xw4S!)E?+2JgfJh*z{+c4la*l44V0ex5<&XP2|ETKAN*3~W9Ze_7q(s*Yjq zk&jh1Y(&4HITUoyc03~;*;-+$Zym-=P$<;kqwlHUs3@u^LMw7??;wwW^I06JoaTbB z3qZ#~X-!R-g0gaSdO&HWo(z^v!8qfXF@@usH+N&zR2eM-`L0NjJ36>xoU1)9sTg3pibiz6c;z|@9iX~r8O!9Ay^nz zS63x7nmT|2DZr$<(H-tR+0@)DV_W~?1;Mrb7cX8gocC0?V^o*4YQ{sfuu_qHTps~X z$v*kOJiGZx@(jthd7-o#xLDTPS2AT$Uuqa97=*k=(`FnXYnZ zN8;`H!a~J6($X?UQeM^|nh*W(L0lZ#vG!_4`YyqY1L9qZo@_YJ`*gkg^$hGqICivV zVRK7tL?>G7HrS!|S1!a~m)l#MwU79ttaI4_3H64*zIJqaH>(L+ONK3OOG}45G`;Z5UBMR*!8=?h#A7~j9>>(Cgx0M;MW zYMADZ%-r8*FVzd$p=T?muZ{&vvwh&0;n@ zvan{Jd3jfjO-zE{ykTHrWzFl&2NDrP(S9XOPrm+bfTEfip74VQOwo3{b)QO(&_H#IIpUz`q{c< ze*H4N2V@OzrmyQtx?Wg1v}0A{$-;A`FD9!txX%3b=SjLTEtc$~)1mkGd}q!buJl)B zz<&=28x|dv=FJ~h+jEB8$lN>((7&Gt28QqM?#?V;u8`1)Ae{OB6iW3Rt&x~Q#Dz`G zb0#-F$Mm~9`By*2DIYc83z)Nq{qZ)FPys8I@c7H+L+#_i4oF)UVK5lb!xwMgzP;f@ z4y1V-+ver0O81ovz3{2Kp23BMBZe~zLO9bgTgQ;=%m3fLMqAtRwJpK-^Qfx!J$3Oe zbJjcRJ&_cjVH_pPjlD%X?KVT~hcX^ZPMIS27lBCq&Lm!70p3mUwg&aVuOG@nQk`td zhEqCYnkQeB>`Zu*_)2qEW`w2<=QmNat@!`|| zL-To2(DU;-NWhN{rw>!9F8$@p6V5b*c=RNDm^~4NpRCq4BWTqyGm5!Y{K_wKUB>CZYrH z4#I$w+3rvF;kmx{dtOYuHSKDu{2t0hRM7lcW6u?;C(s}_Tn({uOqc1}Kl(HDcECNT zTg;z?A3Pa7CMKx+Or?XHcj~=h_>E(pIyjkEO#R1o+G-zZ1gQfN?NVB@mt_^X8JOQ(0^@W%em$3T&7yAQ-gV1Muw=5c9D31Vj-lP zoL!A!dD#m!`V=?~%~3q2Eyf50V)6+gM=r+|{Ur%oPKh2}*_Z#2j9)efass$kex5!J z>_?aSOnY@V$josbC#DTuS+OQP$Hu`y?&?n`rUf*M1n%leGpFC+0`IiYbPP1{0>AQu z2M!B>0gvd4#2PQhVp z&z?OCc-+nk1&y2{fEW(A+LdSozkbam*!?y;CnF=|IIRN&FmNmz0LXlPPv!gj@%$m) zY+Bi|afqdmZa({leAL^;&Hed5`C900lmL;Ng^{a((7~O#QBs{C;d7%FN)-bY_sC95 zyS)$kXkWSQycHZ+A;YX=b zEDgHD{qx1eV1uM4oYV~HDRSPlFg7>m8Ac#gEK8A0k-X+n_^rq$pfR_K-FNsRxpbfO zr&d&GjyTP+=eLNVMfDaT|70<5r>tz%eqT9*>Q=My)Ou%ID*v9{jM$&V1@e;M-eZoN zbeD^t9o)-uJp>T%4DeRb*+OI76=0Z$Z%i!sMFCjEuIVq~wO6>GfKp?b);n)c(#2=!14kPcNxLRYH@h z2>}@b{UMp8Utc~&GzOaFxhzY94oUX&$cMIZAn@1%UFu|T%40w1zgQArXls1AZa##u-v zaE^x?G}_sG?!A+X&f*c@u~>-2>FWPNqNS%XLLa4Et{#xgKjC% z)f{Sl;!k%2T!<4e@azfslPw2YiUbR@gM7BV>Cr{00PZljFm0ePXp3mQTu0FidIE78 z8ynY?_0C#X0l;!{5?~PtIj-n;^n%`otV-+rL>|Ur#h`yrX8mf#-o_wMUwY}y#yn*- zLUz8g_=ElZEFj&fEi=i3y5m6ga;I5){~Zl?xlklG3e zI4vhz8JU@zU!J{o*Fd;pq|$X~e(yu!m|fW2=_thl?%+fD=9Q>2bPHsmademp0%zk>Az}_DSZ|3 z*`V!Iv5Ql@;(mGFgFD1!S#6 z`Hxyp6Gg3e#X3%%72-~nj{wS{d&44{*F1Q1Z_$2gl`W_I0w6>_XJDe;$?>DoDchQg zUNC-G_+S_?mf3)(k|Jkfpjd;LwG0&7f3FE!1cYF|>mn5mJiQth14XyrnvImJ76cCh z&fOPj?d{-_eb9zpe`%!Rk!r1tQ(Fj=%+A! z0{9e8TXJO8)B-y?dU?+E+wrqW%UU1C>v%xGM<3;8>IG;90ge-UP9@`nj!dC;1EA|Y zRK9|Sgc=roX>`xplh0vl$Gl`Kq57Vt-OVSv$ep^Y9946CqQhZ2qS{uoc%<$h_Ll z0dju!c|1${1^3Z=m!_R-?e-Vmd{*e_Wi))$NK7Y0zhq&V#~1hbVJS_F^qqNv52`|U zslzXn1{NuguFK$yUKQ%lGF%8BBN|oRsVL{Ln*GWc?jIUWKB-<7WuGAz9xEVD%fS+E zkfV%a#@TQ?uPwEta!DhWv;SUQXGRC7_}iqkMM zGA1?$842#q!0bm@_7O!U>jFga#_@M+jR7no0_Y|NV#LF z(EuRpWJz8l%UmBvx9x!#$@XdkQfSXqZiPEw$7~dmb7&a(OxYxh5;j9JW>n#&GO2}y zY6U}n-EYwlCo#b zI4sGEwNYY{c19NU`{b${H#EN;(I3UT+g4{=cGIP1#za~FL} zZ!vJp=-c~3t;fg=wW`U7jWpReYvpiH7j3ssA-U&l_7)|wdQ7&O90lx*;D`~H4BnpS zO4&PO@}qkU&b3~%?{s8!`r1T2?R6LNk#^wOj(@`0+tfdMXy5O zesQ=gOY^RzFiLa=x568Dra%p*l$syq{!JFkby!`_b*5785_n^Q+PIv$on296deKgW zkpnzEn4)#_^c@QLv+8?$ErL_TM|u^Tsz432{_ z#0jxTz+;0f(m%3CT>CJeG+uN(ZehE%Cv3FLDYPYKCYs8&wFkFkFKDlGzkfbWPzFg? zscaTdp{pA!TwFR3tEO12I-OpPuJ5ejbo|yA^ghiioQ?@F-bpiQG8&xfN@GuT$GGss4&@d1;B%@)g%|ewsm+dHGR->QHiyeE9Zo zU>+-N`i{~9z_kQ@G%syimkl!sr=Bkb&Q7G((qOS1t;^TzTl%R8BOYhR&HA|(z z1cBkc&x@g{%8HR=F>Xaq97}~m7i=t(-+yZpw{Uka=Ah?J?REViBr=ygGV3T`Uvf8I zuJ#!lZx;|FMEle2L9OF6Uv^9(yvc6 zs}mP>YQ14?UEg>G+=NNc9FB`Ki+l5HEwP_$vC~*JMhDBy>6XZg3k#JjTWjaW)BBV( z7or1li{0$CbGlc!`Fs2O6${f+n;eD%sb|}2hJ3nHjmju8w83FQjBt3DOisgI7o-+7 zTfTT;T82B=v_A^Z~h}Zz)viq{q%T>#Z72RVurS$xu7$H{KoHRv8;pZ^+g|iWeV?FY6 zA_hK993@{)H@3De^hgAu$!BoExy!2gkNo?K^y*KN#;{$wl)KrTCRN*U)1S&fZg0EQ z(oNn*5K$SW7$lRLXl}A1GnneSfE3*klzUseHHhRcMz_G)DSztT$2qW=+!m03A8;v| zuRUskJoLqW$e1}q+D0Qyd`r<)8B&t!TAH&{ColD--d&c4gX9U;2+ z(=E?6yXS8{VrE)~%N5;v_hp9fslchAr)yiPHMq=B_MM48v zRB^VG5@DK(+nL8;h5q(`_Ql1|gDlmV7cp1|8ug(TeXyqhAs+{atmwBftasv;A0Dc5}^#Lg{gX;Rdj%Ak`I zT}#E9jcCt#t{CjZ@nZW+6}Be#iJDbqu%H&EBlk^{H!J8$I^5lX^?|*xY0u@IPGHSD`yA#AoBqfLUg^rpsGc!FFYFzjyL*?gdM;s=I1gyXK?)9s+ry0<>?=E|!>F)&s z*kc%gB&e4~Z-SoN$6Yp1@z;4WTtTZOjOp%m0DI2o-P{pjW|jigiI#nfVs+pPM(PD9 zZURUBZo!8*cj}gkZ*e{b*xn(Kg1wECVS6;nfx$M)Ph6ddN`}|FuWT5a>S{+0s)d?6 zM^ARrxeQFtc^a0wnQrdTWXM4tIcminJpe!>b}1k_drSM(V8Yl zcw~yA7%1>Hhbe2ZQ~|<&3olUyji`$X=y!0zd1L-m7H?_-mpRqOtfeFm6vBz(U+Ibh4^t+sxZk!F@UiKi8Vz4HGc;{Mvd0X(3p;?yP$7t(Piz*p~^Px z-Om(P-S=HUGAV3fuy_#BzG0(EM8L$v^rC(u==E!IN6?A<<$X|jblF|5$4K}P8?a

SCy$Xp;|$3X>F!-kK074W+|k~q&gL*_;t5AWh(Vj8-WDe>x}HuJF(7- z0(zHYoj4xkDck*G>G{Iv(GV#Y1;9}tEaI%wr;c4`?6JR z@@0=kW;Oq|)?qJzjUpa8-i9Xvl^!EJtI)}?b=%GX!XYv;hQ^kM+Fx5mt1W95?Ds?8ICNC<6x(S+>%0X^zMarkM3{0k z1W}A;zgtq2yQcR<8orK8SZM1sZMOI71@zMs_kWzzp+$eH1VTq65drFhYy!VR_3Z?I zPyz;2W_5kgVj1{Dp4=$<2&ET7U_QnqIwb_EFM6>JpgTDmursfwZ@w}eh0EfQ@aYi%tuFZ??^mJAz6Hz3R2MoesMYnml%Oil2295va zK}Xn=TZp)Qeby3V2k}~uQWWUmLnJR?Ia&#r&V9{PRtOVk&^9s(zXa(GYGoZ`@k+yG zt8k$O*W-CsU}`}VNV!acF44TY!WrUmN*|*6A1B5D7+9j>Mrm=)kn|>W`K+hPX{%1l zrRAZjLVj(vA7@8;V<~BJr8d zZC0Ti8Zvv`!Htz0<=ogm`KfVo`jIq9C$KClC1-4S&!sqfJqma5G72G-|5C0RX+7K~ z4Ptz4oR09`?iaXIvgek9(3__PE&70!1Dq(zd^bJI-H{n=f4dW#;uLs##N;@ml3 zikEsA&&-guzt9>0Qv*&{z|MV$4GRrz7%G9bDx@a7Zz(b#Y%%7JAcv>ua>KT{IXT-D z6y}x+ZMQo?cF9-CxVho0;wEzGUCmX>>kfvuMsR;k*H9c3VU9Ta6@YwYO!%P2J+_9(wNgiW>S_IRQzm2 zbQweLir)JhH>VkzR(4W0cC6VKPYP^K$>|TgKjZj0`+4vfQMIj&O%oByFi^8qrfq8| zkl9Zu46%`=y%0te$KBuCK5JI&;uy-nxUuV;R3%-tr(Uuf$oF(=lzV=o*)lNXzCdMe zEQ(jt!hG(PGs>ocdT5RS?!U4zxrK_ktac?kRY`LxDCX`cihM~7wRE1vv5Z%z7`58^ z&MsNVv^HnoT4BGulOmR*MLHPOHpmQ%Eh;XW_rJ5X4z0{hC~Q-lo^F8keoJ4k*wBIv zt^0WOr0A~ik8~nyXWdFv4_&BgB&OOG6c|d8v_-cC8oz5pw-?&o4}S|{&WAVSr%{-+ z*NW@?k3`#*aAqx@YRt+B!J^?;A^&`q`wf<{+Ow>;Z!ejFBzd%wVKkSnc$EZnzy1UR zcWS-yJy|U1_DhMgQ3$B%J=t1!r^S8msWtw&{o_uYPM3)=QVC}qI!1?8wiak+R@+?` zG_`5;#Gy`^-mSZ-5RuQY;B|K;n`a|#4hs<;uZ9pA=`uVeM1#~7WD|n(&3xR&&D(+0 zlW-|wTGA*4tjIalv0{q3OfsXbn{O+{y?b4TohD^vu-LtV6T~EgHcoxibbV|>#6>ew zWi)9fAxk28qcP1|{k0!BE6$h5j1buFx1!t&?ce6fWLH@R8u4c*7PY$bEfo%aId7R< zvY1ff0sHoEh6(2_Ki@6L$sgzqu)|90apP)_Il9^cUnnM!PrBeemGfAp7M;7N#qxk` zTHKMiCPrGqL9YT)$mE>S7Uq;{Ky!=TF^h@sgAmPpwpZ3?b|?Fx*9*hSsoo+A9ZWp_ znl)t;v@^r!#f0wPbkvqg2Q>C+np$u&H$;e?!P7V(zw{0~Xoz!*Ao( zrU#-)5`w7^FuABm=}}uOI|~a7mPpLr z4^!(K;ek7OGW58T$(*HOK~zlT_8|9VzV_-E+eK?jJX!5})9&?!M9zQ7HI1JJHA4>L zY_9|3aGx0@Weq@E`xVeDt*IsQ8h*RvFlnO1KedU+DvqES+bNpV8)|7=1|9)#6_}1+&x)%>D;t4}f9ZZ|$<>k(f4qdI%yCBJG zh&S`T`_p^dPpd0S22pHUQED!2ZSEq?cpuX}{=J$$R#kLz!<~ycpd(ocQOPhyB z^|z-ALZ2jyW47PHwgiyQ$a!#F=}>O(B^c*lWI}Rb)+&7eXxfw$0p$z3W&~+MD>WU+ zc${zW<1+Q0fBU?{|3W;Dx!00Z*Bt+GPU4*CtXF%S4IX?Z)a_(V7V)=GC6C2gVzE%b z!)pRle?VYU7gJ9#ta3gEBQ6FDKq)T#sJ`D2tdaCOO&M8-Ql~z1?zn)G$5Oc-9SzMP zw(6IQaLn@(Y5Zr6P!W>S!v}_w=VIFU`}@NyPdNovbCKqL%H;9(>0c1qwqp)h#tEl! zCAFdT#K61zSxD>{yWQ`rR?7`ai_|Au$FXgimW}tZr2T) zSa$WIhC$1!VMK~(FL)vvd1JUB{k3cIO-)U25)x|E>OhH-9|HDM(AyY(n%`2%TX^MR zWWJz`(4jn3#g1Q>T&X(C!xmN*6dvp{v*Um=PS6} zxRwoP47_2Z+ z9>Ug&RPp!oBc&eFNE}Ren+O38%nS zr;>s{Ik@Uq7)Z=?;1}A{X!8L?EG6Y8K;R|k=C%)f1c!7Z(j%ZFAmAivlwMTNC1-xD znE+YMl9^01)qUC4V*wA9vClD#5k2J)6pA)hqgiX1Vf}rP?M4-N${F{YYa|C5jSmc5 zebq7Ue1rfFsgMOEIb{|Aa8^cJu7WT1gh)&K%~1LW22#-Ea~l#~R*m5V8MiypKEY79 zTku*Xv*(wO{2fdDe>^sN8_eYdA+o;^S1~N{o+Tioq?lc{C@8Sna*gqfV(Mv9QW>4> z{CxS1rBP1MfdT$8p%?GtTLAW=i59$Rd&l7Wx4(qn4mIU$vE7hqcgA))^?u`n@c;Pg zUFF_(Rcoe$6Sa!)Ad83=akjhl!Ft$In@t4_*NP0swRz#^SGM%Q5ZuyN9s!M@RRRD6 zCJksjjjg$-v7zoqsK1*1sJ|fJ_>yd!z$stf>UkDpg=jxtVT#|&rl$%85cy0o-9_Sq zJ$53C5OsBGU~O5q3rbA9a9dWEcf7>wIPpdi_1q~4{Yy+ia-ObqfSkL;ZijuE^aicQ zRoPEp80+9NkC|)LAXUxVu%0E}Kksn?5DoFU#{R=A?Rm`e=~Had7Bt`t;Lbq#gcwnT zi?@v7`4xX=EmIKJ_VR2*S9?sreKj+DeM5$ZoJvgw5c|Ee)f;NfC2e2Ibqik?lKHc! zRG>vA+WO|9uA_(0Gcrmt$;KFg$sHBW#RUZg*z>hO*8+kD4ODO=^A1#y$=0m%p}bC0 z-1L1L zy8fnC@NoTqgOPws`}~(nOEM!3Nqu@5gy8`8h}H(6lAZ?eA~7-N8k?FrY=e@7QVLJ5qlnECj6v1hpZ z_vJJ02+-vHt#k zCa%)v%QCgzV1-nSf`nbTk(U#c|A)y1a@U5*J~4@ zzpU%XlOd7E+@AZ`X-Sqw_pX3!zX7-MFc4%f+Sa!weBCE_ctN=<-Zr$L7u!*&SUP+rK?|T z-;XnUst8gzDkUY9hob$Pzxjso zK~j$X#47PvjR3PK&-VYo)BdUsNYMHt9?5^`Z2$E1f4>?>-vp_C4x}^YTi49B|0JF5 z;9XBrCLsSsVlbG2R)l_*Pip&55_28M8DUfa)_>AeF_&J=e3V51H~Yt<)05-;hUxb4 z^MbV;-qRcOqM}*=Rl;!PN=NyF2M-uP%)0hxGQX^ZWleJ_e*qHAa<7*z!NNwk-sj5aYFf&IV)3meWJugV;{V!Eu^orF1t3##+^XQ{R zF|sbDj>iJ`8aY4DP0^FCdiLzu;MiCT<+e`S$B!S6&rvFiSL? zFYbLdHa0Y>4b1|IkFS2o<}*w?{QEfKP&HCs0IEg;yV>|~yFEv+xw%qxX^D1cxI;=x zO0EKMG$`gRERgg7umLBgpm#xOOl1L5rL$*f=rSk=KG4)OPZylc3*aGY2M*U?SM4`H z(eu~aAH9qh0s+wT`OzxvsHmvBWCAept72l>1?lHNEF#Vpe_g@VSS7o>d`Q7Q^X*%% ziJZ4@Z=F7UTHN_M{NI)7*dn0L>%+5 zU4TabAC0Ntl#Qxah9J4^-Ohi10&g_GbbM#N1bT4O@IpMuqM&8NXrMR1?E!C{EF}Ya z7MtYW_t?bd#Hc!78{3N>}E@e-r-`CJaC~PT+*u#Aesm(IRoe!O!0ah>B6acq5r<8 zT)XJ{WtAbE=Fs7dz(%Ke0NNY~(i@OVzpFfX6%VxS!=MQf`6#jBlj zgcctfJtmC6`ir0^+i404L*T4SGN6+oQjq#L-qSy5M$-T8UQvSiLet)D`G75RpafoG zs@0_bV8wl$w0^eSq(Ojzz0DT?gGQDPUsv|u`?vG<>>~lzo4PHi?BLiuFP;9CZgpw) ztmbtv009#u5;A8ay5Ie;HPWKjSPz}z_-lcKU^|++I@pUK>KjVn6@s#b>iOvssXDfLaym zvH!@p@Gemc(vlL~4_u~pkbd5u?|r8a*tb{F$q$TDkW|~RsD5&GYKiJKo2oin248P; z4S}1b&sOjG%f&g@TPoPI@-~fnmk_m_uN(V<8m&h%@2a}UlAkZl6~^Z{OqqSZm+v@$ zJ@1L*+VnXvzPe<&>8%F(`cKzO(S9*5bG%+rY3aaIIUpbK;G|O!of({h#29a95dO6) zAe5Xoa0&w>e72lw9 zWk0CH*MFf7Z`zSCf4+w;&JbVI%B-x^B%i@*(4pF&2SC>^;o~`u;vbuhf4@7OkN*poSTZ%HXEqX$9VsLJ z$s^vf$I6<0+br0_4UwXz#f-HJLZ=~@zjS}fErUROTSredHJYiB?K!iYps8yq?(Cto)W8!KzeH}FsJ zQ|wL_xI=KwdR|I`W@Yi3huJ`Xmn~*`tF3D885+zuS7x5RtyD6yJY(e$;rIG6`i5V3 z&d?D1YW*C6c*tXahIQWGetzhL@n5e-C-BcPuSJRi`kZoKH&tx}`W+n%@!edY%^Tnv zr)tAM$^8sD`RORX6rUT~4b$1x)ILStd>S9YB`&4 zBhTD?*2^+SjsI4ht6m1NBF(6;$j=M=;Z}I&5*l&Ur*re!&G+SM#}QuesD1|$pjk~& z!JEc#imwi2d|!HiYScBH00n1s+7CW2Tyb-Z9}`q1&g=ys{?G+Li`P~Y1Jrt=Q!AOSg9_1pj?uuo%rPnc&uu;sWzdxvDW{e^}(tv1>f2{OM$uRl>;=(>@6Iu#>U#r=qN1MNv%zt3qd9tgz=&M9 z7_V25oeh7fy0`f?6ksxJo@U@O=C0n$0Qtq63~*=u{*w3;0+*V8Fp{-5pJ_xsXjqaD z6=N?wRO{dO`V_i8c;12Ius(=~eKEeK1dXnG@AF7szd@2hlL3s-#lR??FnM=C&hNN* z$c&DTW&@eg4#X8FU_t8fW++=58-tpf?sU8pGUBR5;b!c(~JD8 zTh%&S_UQUx_GQX7BtOP4XHVjX8CvaIm)az9a5*7WI0fQ;fa|wiQ6+C@W@PjVJ(LJc zISS-YZe9b*@%wR?W3I2Sf4ZLP@%@?*zNEbJ!KRa{tYcKq$p}AlWPYcjP;@l_l1cDY zGq9Vv+4bj}54nW>mY;$^67G3$4Fky9+he?^2?z-dKo9w;;oE#r0J=m2^N@J}&Ibsb zXq0ZIcGXDc4nQ+c`tyyv!XYH*M;6T-WYlXPKvIou517|fuW^UCF=a__$;j~NDb;5O z=%$$3ej*)m%vC(8^d8os>h!#t9@NpYN4Hq?BC~B@g`NxeSs<=}kemaTjx2hSpmPl zA5`7suLO_9T#Fr?Wha8qT&lQ1e*<$H#>`J~;jZjqN%i~9xB}qI94WNSe21@kk%4c{ z-f@isQ=;vx7RPG5hWVn~28+x?6f=|>(akTClj{@)!F0d!%W4I;GFei0iMx!oqobo` zE-m$_TZRD|YDANrQ+ZKi2y-)-c!&m#amDdj3>j_oql+Acg^e>EN$AO|DS2`I2|n&- z0|mFy^vDiT&~T&)u)p+&-alS>#_3XXd512~dh$+xxnsU64*<@VfZ1Yue9@DDs5TJ$ z!hZb2v#Zdl77tfkllbGHkz^z22xc%1hgk!H1ynmV#`=YZRSgct^whzP#v*GsQ5*my z$m33zzB^(#HZ3MrV66SAd4ll8R0vXcbje)I-jEOb9A~xt$a3H->Yt+;&+|Zv_-^FJ zkRRO081}KXW14DeQG3VMIKKj8!5}I+qP4j_m(!fzCTi;5gG2NLIR}_iA<-74x4~cu z+FaNA0GiJBv5c0lV3=Ja0Jc)+R)Dyc`h`kXH09lZA~VQ^JVx`@!S#J|E9NkAC;KrY$aRk0cL=b!#*f2?E#2o!AYK_ z8cL&4I;WZQP~x6OZpybgfN)v;y=;xPPdo}%H7ZHch zG0zujqe~0NPU?c%C_C7Nj=(_p&xxXH5(1mO1tf{qQ~9HpWywHs9u&&$Ca*FN`QeUb zrXY&9eWti6H^ZxT>&yVg;dGWPy6Nq^4|Y^mJ76BFWqP~n^vr&Cf&fuA3 z{+aA%N0T*rfb$IrD<_LA>{L$8q5(7~4HQ^=q*nIbzV!X z$ROpH>FH)bMa)F3{*Xk-JxryUPXy!Gwx>^mvOzAPie3*@t?tq>igURTN1~b%jHJgN zNdPV(w7uN%SLY#c6)43q_L*BGe4En_LmxZr%ww9mx(Sgc3ZqGd#yTxk1JSq>nM(KX zpGWiN(5(yX+T?fR>0V_D5P9r-q0a&&eo`U=d59jZh9x+2}Q1hAyx^y zz3zMAsy(xIf|}i6BlLv8E@zKCuBTyAnYJsFkt`K}PDSk5ruU}CIyGH3tzLYsZgJ&0 zU;c+3;)QxJzt&K6){QCx@hZ+6WBnXeDWb@rnfzLE#@5$IkTtGd6`F?9Xe<(3Lm>aYJ%3sm}CH(eV>+Xq*EEX8Z>Xqt@Oa`%EUpF>7g)w{Jq= zBt|O!SD5_(X;RhCCZB`@gBAWvI%@8~`GnlX17H}VG|1*CYHxI9aCEma%@J-w&1X?S9_Uy#pVvIk3N2lJ z6Cw)??K$TvPPb)0fbmr}c_o`sshR~cEoz|vvK^hKZ(9dwkY{t_vJp*CfOA4fc9=L& zb>>Ime#E4gibKNoCY9|8j{}+nyz0IPjU&WsMS9QWRE>pD@1P4zNUsF-!$XI-V{>yqQheW@8?B^|cpVQJbi8?B_XEQoS zMl7BGtofX=Ws$TQ=*=DvEqc$1|jHFSm!S zbPJ-EH;C-Yl%D5DHBaV|V~u24#b$Km>~1ms@ZjJJ!x^bC3<-=I{GL)``Q1dS3(C)O(9=I4_?)|g_9`2HSIn@NPhWM7@KkwixQ971~5zw z$>CZiONkiQ<~SXQX#^S6ve-LY#rQV?kQ>zYLjsYV0W?DMwrI5zo`q8p2o*{`IvOKM#4H=rx7X$Jqh(O_eN4gDEkInAOqJB=YkfFlb?bg*%Kr!)w+@?~d9Todni3c^CA=D8icP zaN}J24x9m5nv_8MM%4D_TdQhZ%+MLm2C|=z3oXESUdPC$KX|k-9{i%(5mPpet2bmG z-5daoyN}#ZsMX9VUwMI^dG)9l4Lo|(8pQ}`Gp~Dkl(zcY+?dqPOJ=+WUF*RUxLh#p z%?<-FtBzL#=(%jM9FK==LG`JAxPIU|mNA&O9_#{G!-f(`ds_GI(F;Dy;F-*@Xm{>z5*Sv`#YOXPjI>!n_P@Mb1E_WF2GIJ0 zk%vdI$a0j!v?mkk#&S@`A_>E^tlUZw2Bd|lHs=g4e0=<={B^@cPpso#!06?0^zd7S z2l=lY&=pnO^Y?@WRXK#aDflytjl!!c6%iBt*}|MsH{aCx4I)_N%C)*DM3rNr*-Ikv z9}PxF&l+s%+6V4Dv!1+NM-|h|m+FFMFi-4l-;0ff{v2tb)ey-kPdyaVl?%gTiANX6 z29IBW%Mjm0WgbkzN)!g}{G+h_$!|csF9Ed6)T(W-rXi*YV7-K)sYHO{kSVcXpV1jx z@z9r|Kn_i9x`7JTk}y^p-Qz5?8W-}J+SI#@A^@QTj6il=UufC*-ioR;_!idZD!&CfDbehx#1(>R4 zDHh%Ca^HXwqzCh5tXN4>#mUUAMbqq?jyJ zkvSivBp@dURjyxwz2+g`7YErd_^GVEl-7x5-+hCS?!5f`NdSNgX3KsrZViQIb*Fq7 zLdOe$v%UhxK31O`=odOM{-Gi0;v|^6&RfI35a*5%-EXei_0w>i4BO8LRPBk^a4BEX zw&eN9>$<(BxA9I1+ms7Bb+-fzSx3hmpQPJTdxgiJl70cSp&*c0Rt+DcK`RFxO5io; z`<$e){zxMR9Fq;3c{Cdaz(h5f?BG)rYOb{;$I4i#;UR?B-1qmxN zf|;d^eYha9xG0tR-L$W|3AO9(k~bguBL#bGTP#v#WB|*w^fA0ZPog=5Kz_bmLjqbh zFF54X;;3>Mu>Q4gduV7ZDyEO{KxI8tfJj)&rfbEY&jTF3`S)@X8A-^{)CwLVXH6$$ z@>DXS>GSP}Gc+7{Pm7y6+0a+sA`!)_21_ zr#(Ge$bPYat_dinN!nDN-ecP!v!Ji+0UD*-PtIoK6k*NI_2%%&m(4x5X<7#Xb>DOs zpxzT?u+9M<=$Dje3xd9EIw~rv8yTP7zSbO{YI7Nu^Fo3k?^M=>!3t!Rj|!rJLpAS9 z1c3w9;$Wd^xWL_=Y(%xaC00d>?JiLM2E(Pcd7vpzhkJdeJ4O3!C5Vafay;bSOm`mi z=Vja$b$?-5ss!6!`%aB(gT1Ex>KL#IZ5%Abn0s`0hG59Ne|Ub+{6t^N0p|H9-AU;I z%1#(M?+!6_A7ed~BB&Q-GMZY%q)|?~F-WeDCeq=$Xd^hgZgV_#d z5zk4PH@2oxV0wvpicPn11CLdTXBil;5IF&IxnCv%#dw>Q3av?Ca)W^QfwtbDyUrj*^^|6-ot4GS$LS4XrkHkd&f+@H zN@7{ztr24)i}^*4h9QLwhn<-IN*80JnZ0=2Cl7c-KU+;FQs16?_2acZ3mm3(U6oPZ zJ=|+{z$)fPSE>W{eSpviFVW|{DZZ+A5>U{L=aG|b7P+C~OT?_jfLK-l^$YTO9uOj) zBZ)$Y%D@B_Gn-tuV?dsmM8MN%FKamW)fQ*7_qXXRk7G-lmEY`hvh>&)c84i6&%qc1S!c zJ-+Xn`6iVqLVmB_aII)|w#$h{g z0#x^imNJn@>+aoik-Tt5WVS>90_>q(S@GwQ3HO{=O@gCmiuR|^nZ@n4)d3{%U;Bps zsRA^Vo0IWg|J%#pzko!U$li5deUN8ZTP|#@bTnNy+yZBMH*leH6e0=~JmP_xn#oCl z!#*h@IP>UUJ)5KMRsBT|>j(r;yQ_hb!M60hc^gCNMc@9=*nU?h`3-LE87v@|M3dT=Q{x*)1Ed5X22_?XOzWJ8O#@+4m9_`TIn1G+zG0e5_rt+C}9Q zhhwSwA6b+irVjmID;Es5t^|bJto0i;03~om?%#bL8x*9wG}?8B(kPejgw;4ua(RI& z@2wJOZXm+cT3e$;wQ>w)Dop4|zKC=6cFnp_1t35KM6axAs|e8sJ#r0zv3Vqf0#onE41X| z>B(t6N462ijP~^&+=0{)c&*dlo!wFpgJzfE{U)^YJWm38&H&`;Rr^Jaj!i3GA`ewS zRcGtc$(q|=T>kciyj1kc~aD~35G2j5X-eRCSmlfi3+ z3o~mBr^CaeLE?Uy*wul~%7I9XfkAVzM>lsq`wnkHX$Fl`bIgk)r|gb54;0e`3*}%-+b*(j(VfDc{l$EO*{-F` zFhxX1N6^C?Wgw-k8)tG+coQTv4ClabovOB?cjc6(uq!mCTI8HsF$05xtKrGh+uP$3 zw8^l`U<|-x&&GXK=eL9bNJZ_EH#~z2Z}d`AkH-#CzY$uf^B9s~vwyQthTB;YXxVnt z&3S$7mW`uhU1InXJK{?1S_1=GKb2A-K)f7Bp`{54#+-Qsovx zt6q_BQyKASgA-ROsadMC?6bRQh+E0SrS<*wbrDW|NgtSfe$oY#Cn zVL(B2Tf=zr62%IO+C(n@CWv%g+poQpva-muecelQUhG+Q4fbnJZLNXbkhtA91!lU| zI81Yf3kliVo;71bla@ney|?QZi9XmC$w*5F3{X3jO?Olzd5x_06Z zh+cO^eyoYt)S`7U84ZHSiRmsi*pwaLc!Rn^d8b(d`VKW9vxkI^YJ`ir)^+rcTMFd6 z_Kgkk^ooSLeXH&%kJ626+si6iV_xbL3f7ok3-?0gkE}l&T=lxJ*4y=-Pk+}>+miFy z)Wek@`ZZ@kK^Lr>E`BJ zBFYa{QURY*t&10Wd_?P=ktIpCi$Y16_KCQ?-R1Ci!Pe_vG5@TcKQ zt#WgTpnduDb%*s`NntyQo9L+^*cFNYB@dsgq&l&T2@Dirj9h(@Y zMy0e*U{61I?8NsM&z}t_9Et4WsdDO&y9INN@xkD5o}ZhakA$DoBaYD8K+bk>w2#es z>BWlso!d%H;&dljRMQzWAc{OB;J}Y%q?<7FoX&$GgC){l5r;ye%Y`}dnkWDCwakj^ z5GE#Tn2O<<)u0s)X{^lG7bR9(7(y{I{pNizBY`ZTr+nIChxi|ha`CrMxU`ynN1atc}2&tTc4A6-cB z`CNS{%E+<8h_5#8_bBL=@V5_VLLqBwrgIkywFxWt_&3WghWks0577!N=rk9sMD#hd zD~D{%QqI&mN*j|OMU9iUjQ9)W3JIliHyx>RpF4NaW1sNK&d3NwhEZI7u>D*1aP*ZA z9YROHpAI~2o%-qk=I_Jb9l0kyJTVIWoq&ct0!3SYZz#g|j_0&?(^aiot8VtasxEBv^0sePCr@1`Wspr1O!8N$U$*lT~kjmRjKd;v=OcQ?9DvmyTb38 zQ3Vl7czzYo0mIoD+0?S`N)|1avVB%kIfvGMg_yh%W!5|g{|0lZH*Y+GvaH1h?SFrC zzwc~FqGiWu*=%H;o;SC*^a2@ki0#{~bd$T~lQ%OGJqol=3{fAg6qpLQBK=8~cL@JT zR$JPowLwfmv(L~YOjeP@#>4m8orQd5AMI?Hiq`K*<}T6|2@cXl-c`hrP{atZ{jQ{2 zBDzO2F+Dg(H`FZdwM6gCox;Qte7wSAixXu?miQKBs7^DHW=oxT@3^n@gJRTS!kdKQ z;;cDm+MPGn3@ZJ(ni+CJR@w&S+^%LpZnVCs$v4cnl2YV=#iTC&iES}OSz zp^m&(DbWdrLiB7igDo|ylWWF{0~o^-406$Qk>qP>LMv&6m>ekOFp;y2zduARG4-t| z&~7+p2zuSNJ6pQQwn1k_pSyWunZ{~?^i;Ng;j6oSiL90*V zIemgICBKiwlo3usHPkw7JaIR87Tl4qSVE+%XovYBubhhCl?15)!myH2Ej@{W+?yYs zPDiT2P~V4G!pk7n3e7nO)0TV_Yr_x@mJH6ZV`&9n+kJ0%yHk)1s2Al3CD=JBsd5(O zq-#}sv=7DTY2kh@b>a{n4wIg^MgUV;*TZ3#s?{^BIcpZQF4C{fb_dT^kW@RbToGWzTP?6pmh8;4 zpE=mHm>Y4Sj$AsxdwtrqI&!@|%;c}N{e!BCyAO}@*erJkeW}^@h%l-o&xKk<0J0Wi ziQ6O{8jG413|Q!nHOwzNK_jD#!LgJm^NN3l_2!cK9pYg7g}lxJ>5C>Cy*jz|Z?P;> zv>tb|Lh9J~d(LV$%t*CZ9*WYVEA1P3Y9~06Ka=Vx1zEj%XuaNYdwF(f2&z$HN}9H} zbD?G}KB9a2Z@Z1TqV*<1`Dh>Cs;a8lL(Y?jhQ3^=KCk6iJgq$J_M)45t%mw`U2A6M zI3)+86u)kn+iaqB0dv7j&;)-d6*CULpAF{Sx-sRL_r`RLugGgxUFX71&V;3~>JvcsS@#)mqqY6=~yYdu2c@!cn+9f%%uP^Od`h2w84I<|Q<6~ozd{(2^ zrVeStoJpM{SGC*;+}=%)-AnwoAU1-d;ELvQdq^ms5DXRJW;ziwu(%p7L)Uw!01R+i z0K(wrk9VCHtet(azjrJ*W<`=7W@RH_vR`TK4Aya!(BNfffAM6pXOW>v)83?T1Ryb1M%oQ<#?!h=$`F6#1)WN=LQ%k#|b{ zsV0ik?``-YqE?Y%AFix;n|PtK8fN7_W0N^1q$XG#b}i`cHT@iav;ck=QGKNz$2sjQ;vTQP2{4H2B(5PZ#jm$LJ1 zyWk8PTX>zczT^=P{m)bzeL*v|TpKGn@xvPf%+*mF%{4EiF5a>8uld6y*-@iU^0SZ9 zQIF7tqQVPKHw%AbN`AQSXvH~72hfD0Zs4D7j)cRc;FdgBbqZPCXa?hSyvo>5geD% zb`inA_^CT3IV-{6xJjitE|3~`kZXDrx106lI=8>uB+&e=q1SvFMC>0UNU8U zRO;>W0#BlNkxC~|Teq%ekz*5jdC{^u(s}WsqGLA!}tV^=Z0h<4SuDNn{q7f-<;oe|N>wp*K}C zvJfpQD%z5LPj`5HT#Ca{SwbxmB0>oaX*a{N;bVq(6HG{4;99J3#Ii)+0tB;C+f(XV z2i=O?XEAsV+8@8RDaTV>d2j$zgInU&B%i`R`Q5{!g~*d?x;E38`dU)XaLSc&--*eu zpDBkNLsY%Vem`_TgR|&T#QV`UbMocd%%Bg4>Z&%zVF?=E9L^kST|hWKb3V%ok&(5yNb}md#m# zY=>v@?$icT1Y%|AN9eH^FAQGbb8S37wr5NxyfzsT>1xND%ex!B14~0gGOhZ5gH zi}#7G{~#iMP7n{ZXL&6xE#j{+W^{*rMNk`e$ev_B%F_5I4A>!akwx8%l>CEdcM@k~fg( zw{@H7&|(0!q-smS*xQzTTXR)t=x3nDJeb6U)rZ9)U^M%F(6xr1j+12bs%TkMkDk02 z27`IWd7Te*6@%{3n-S!HcxCjDPgn^og2W-;)k2qXkJwZzk*Ztnk&uG>18JPt{u>4` zI~)16Oac~Q0SutV8S!%9M!fP%tMN9K1mCcBRID7$vifX+LwL^M>lz*+_;14NNZ4Z} z!h8RsTJYu{bz2+6|McR7b2{rtg#gRr0Cu7W@qLWGM`rPwW5 zcWZ0uLzhw0lj=rBM#Cc`;>=2k64in$#+mAlN@1m_SW%i~wb_h8^Ark48T~}N@erRA z(Oj6PPoIvyn!%S!&?p)?3(svEQn_N5p()CeVSl$AZk=IKnV6A@oj+i-vYxkhqdX;< zJp6u=J99@n$`olLAFG4Qq?%`Cf|!ApbOZU8)Z60z(C)y{)Kr>e3fn_} zA;o5D;uWz+)Psdp8$(`_J!>z2y_g;)KXf;jcv}!mDBA40=H@rpgS6k$zy%v6kmsum zymX<`kAtq1y1j=0aACu-J-M8(if`dSuN8Xo>#XbAl%7zkj|dJ9uGRObaRv}tYH^^@ z9qBSkIr@WGVFVweUU-DuhQ{Aym2c1Zwrfg%?(AjATVx->sdDFwIk~-QP{&#D4+e*M@G_AfWEKf-&9yWd9viC^DR&`A zCB#A<8|F=^6o-Ux0a zWHi($C!AqFj#cq{=gj_e3(7qM0Td=iuwyW@}9y% z;j-d8a;=95EGKSkFf3Lw8Fkzf<%+@<<&5AOv9D-UJ;Tn0-BUm{VY~nMI8F%j)V`MN z*sS7koD%Am)f5xCLB*}XySQ}t-JO1Aj!=AkX#OOJne4K*4Q4P-){UDdUsg6N&{0fn z`7<6jVTNssuyz}&rvQxdw#h?26o18uo+=N001Fpl+O794ww4f1yAj3XA7mJzUK>9#Zd>EKHk7Ez4Rv!BI& zXDUMNUP$J6BSs3t%o-FxMt<~?QjoQD9NyWloQoYE3)y?@2MFf_sO2$J-6MN8)8@TG zd$6D}LrhxQg)!J237Q4GV;u!~Jti`zk?&mHG{v0DhO;UWyx9HKcdr%MYwB5YYsMX> zxZGu39q_tG;HUMN!~!N)>$m-Phg%F)2!HJ&g} zGLyU4-o>ig_C;t6)%U8##a5DuLl$-_n7gIzt(Wnh>97ybVh_zbuzwQN0Sp%IdG%~| zpif2Y94N_8{I7;WYt5YlCB_N!)PQl#li5Q`w8|iU-4*8c{SG=yF)?Ig%DnG_W@d1} zJX-nokJ-@h7Epb>{XTGKNemm9>N2H12gxvEyygl9AJveANsH*SE=T{th2p&L~x*Db_tIjQMZ^HsyK2L;o1&eC`GfUUf!hW@KSuVfy{P&$FXx)psgRs->l!j5fgx zn$|SEt1i7i`{&!7n6=>N#Wf^Cxfg@%)JN(IWt4dOP(1?V6B=)G7_E&o@b@U%$@5>? z-w#;tJyo_{?hlYQJx#)*>eZO25x{-7%@0!PUYW+tub^{V5R@I{#l_7J-SIweea~%09oUHfZ z>9OW8IK+1j7%ooETb{~FN+ixIO5jF_X;%jeG+(>FwzkGd1qlmr0Re%C8j7$D7%FuO zlQXC!HimQcz|e++y^>w|r67YP$&p&?iIy`wN_<&GOo^Q9BTk>*P^xI(T{jjAiS3Cr zII~BFPY5T;6;GSGitI5PTW0Vn@$&=KvZ9pUI;eP3+q=-NI3_ELhn$jGiY40iFTalo*@#>jl$}{LvIZHl0V4R4)yxfIrsE z`8q?t*Z;8F)liuBT>8;%iNU=lik6-JJkN=OmKJ&}G3RNI#L=nm{u)E&MWff5Lt4EX zO~<82xQ_{PFGP&-9Wp#}`>4{&dwJ(T%DD{zvw2dF#DpuVYQuGH^JSb_VW%l;I$jV{ zHwRV;+TP*4i%C{DDLARlvPX6&Ss>7>`Mc_;h#PE@HU?)IXGMZNwGOs|whYZf@xR4& z?U|u_G0#QoiRqK&R#>9@1&+B#98EnibBS`#^?y>DA9}UqG$fwFQAcA3fIw|bARr5kA2y*fSayLnHqFD$`=mW34Qh)s6RdZ_BU}sgi3?0 zVfTLhu0QU>-T~)8FUzpQfBOX)eK5I~JE`2a`wya@7UtX)yK}xP7Wo@WBly} zV#yKf4gc1d6gxW2>APR9KMp?Fbq~S!Ifv6|u#<`WaQLb=>)-qYi+>rMOR8_adpr+G)?xq4(=xy%@j$hC0Ka10{ChVcUHzOB2XpP;6{ zs+*dc8k(3$!bGm?5QP!NYCt6L7(JUaXkT+Ej19NiR%@^1e*`Va>KKAaNX(olire_H~{hI0^upLwy+oR3EmZQ%cp_-aSIkZL;!N_6tUu-Epk8^fK2kc*`A+c)JroWwv# zCT*6C6>SdtOZ=-Q(y-&gkszzx?7jVtW(<%p!x;NQ2ItC{A>TnP>=PeXj60qksppj3 z7c=iaDaIXyEi5WmW)bCBW`~0`wPcaGyr%Libto$!@Zl6}8?_ra< zv~sfS@f)<~h8uV12n~y(NU2^ME2i?kr;bfaN zVqhQ?Rp-!Nw62rnxvdCYcoffH^ldstq-E3nTwx+@^wxnthvzzocAh_Llcpev;~;M`LJJ-iLJ^Cm8+ZL+$CeFKalAQho z^DCI3zdc2!up9rkZ{PeNRF?~EN{@afIgI+n{#AtJnbH#;$=6$s2xjV}&3v<7S_~uI zMS3)kQ(~3HB38A~hQrVqy})~A*^m2s9&?mV=n2leA)Hvlsbc)arBQ~maJUjr{Ic6J0w$Qc=xzQBRl<)JgFO|SZ$c)yB{~T& zGz)5w%BBlr(Q1-)`!Nteta$tVbW>#)ZX~FB)5a4eNeeE0w}1&j$3Er591#l`7!nb( zJ{x5KFdPG**A4-|=%Yv8;5*maeA^kZa)H^eSB6$PH=MXx^b1pwaB}LrX8AgOu#FCr zX&rEng54hz*A2o&Uk_#XdQ?O*R(g64=nvpIFMhDtSX+ql``xNBUWM`R+enSP&Pa2{ ze_CTf#i{2n`dZ-j1|Xi>O5E2wrw6lE*s-kIC6RJiBay??U+Z*&NOcg$K(bObp8 z-z*%5tFP~+DAo0;#knNsAg7H5XCZL=`vc^t>|C{#ZB?K;<5FJjG@mvLbBFl=Idi@> zADntQjiq%679uPOL|o-MB60d10n798vVMvQP7JQkpla=)vTs3CE?#(VTpZd+O=TiHvLn zqpqT)nlR#J2pL=GBBQL?vpS_|2TX!Gdcmo`2%Zek;(&Xfxk}hPQztE?VHcF;9N$E^ zz+1Ed`Im?pk6Sm-&)@4{UurE{Yjre6${;-i3lhIL6KG@%K!L?v=kXeLYyMW`z| z!RZ8?GD?`r)QqDIRgEtqx~?Wbg3#i2X+wrSv!zWV`@UJRv5cUFE>-gqlQJ7*HL21U$>RD8o-3GA4_23><7RN@9V8 z&8@=MQoM{vF~h#PWYGB+zV-o(7(};YBydK!Yzqdna)UZ zWD#ay7UK_rxi8I};Zw^bMT^4xxBGdylp3H05d5!z zjzZRSR~ui)lt%&mss7G7DL$5hAxTp)0yo#0zP>oEHA5FOEiWbDr43!7UY3bXN~+cr zl^IJ~A5Kc`b6#t5M$S?x1G_q$y1qWxV|p$%eeX G5Dzq=c4hJX4zp4p6XOB*Ekf ztMSW`>#4j#l>IPRhjIjIQ=EcS_o zaWy(SUJXVaI3B)}I|nT(2Fl-GOFtT|97=T>uy&$v2R2Xhv%#r9?cCCdPiNpHmmV6H zoyI#-@rku)by_ZZ3_*Bya>n+N9v%l)miA}S8xE0X$eRs51JPz6xKGTt{J}HV~5cN?Fz&UsetO>0)`RMVA9= z67qI%g6MT#-e?;T^wm+V`Nk6|f=tGqXAo*q_R)>4n@(1D@<3|k?&Lgo*m^&F^vj(E zQs74(y=sEs3Q?YlR!Z{Eq8CS?@K|-S8r+dD>t-)PZlyc(GlUp&M^YWjo0cmSfTlrR zo~HDkqHm&&$YM>N=~9Os>p%O@f5rFh_}u!6&GzBCeFEs7e9l%ZoK5ePRPjM=%6~BHAwC8NyPx3%n2m&ia_1%zp;I}VemkbDya~ppYG(xJQ zQ#p<6A$W?N4Tc=so?xgIYJROkb4)#pc{eZsw_i_B&%lW4hAWY;hLws7y5&MO|7%)& z%eA>CJe*5f&fb|=WsgGQ5bqzJPbmq0AoIFF?d8pdW`B#N zq<*jTad6QfJ@v>ndyZGi63V%Z+FA?tpMRWU>fm>MF}HKRz9Ac{14D?7bj))CFSU8l2Ju9*9s#^D zq(ARQrTN*UA+ymLf@4^j(6!Rgfy--el+nPH45ZO?ySd?XIP{GPa~4n30a3Hu1T|}D zS#uZbus(ovliYW3*#&E#R=PBQO`E}s6uq^9c9&a^b2toH=JTAIhVG9iqW!| zPshKM&W79mWtvPCUvkLOmQjdR%Q|^rT9@v;lBUkG4}u@HFFEuOPiWPQ+i^tKQ@{ti zu5hW+^bR#~@V&BAea|T8?=SIKA{CLu+~L?q!58ohN@q`jyQa~YxoBlZVf~pg%FDyu zz&eiQqSBBNoE0?XEA)crJx)x-P1hVv^IqR~?qcqp{FY&)MC|X%_P%stf}johf1S#j z;UhSRb>+xaXV>auz2=_$*<^H!qSRl6!pN4d8SSLuQ0L?!NBqZJ!}dN~lTsH?Yn^vz_es*%w;5U?gMtX7;1_C>zeYUp*21phj~fBB`e4-a z830jA^G{thXazdISOrJK=(?p+tG@X?4)4N9IMCD(2j1rMXOpb|8x#=z>_PQ)RJ*XluiJ)t*&#mvh$W2-RvA7X(N=N+l)$RVqP8}fRi1YdA{gA z^YfpD1~nV(dSV7hc$T!WoK%;3yF$*5HcO|}Www37DJy*gFzCyiz&{_gMD_NNk8q6L zaT8oXP>KcpTuIkw2CAz?POMZ}7s~UF-)6H{F}Orbn@!T4L`>twQ zc6R=f{&SKE+iLq-dLUenUMUAYv4GLWh~`2rQs+WKp3SgRA^#AS4UOZdE_FqV)o4oj zo#Rt~_EiP()pf45cSfXD@iH5UjxHe7a)mmW*On@yDK9T?(0+}CHd#3a>?KoCaLTvx zDZOCZJ1z?%C^lWgPHD$s`DUCaby3pN90e1U)@`zy?zeY*fK7W&`Us6O?-9Q$GcGYRTyYaxhUPD>k=kq z{9BvDP@NwTg$gTUU?9_-pC>pJ;XE&$jRQw`L%on!%>#}2#_Fonojz+{p9y-obCZPC zm!`)r)yGE!{Z69uIQIx^cBq45EJW*oj5f?&=&(cX9HoG-$Mt4U4Rh8gd_aWf)w zbtElY>7TA(s}^B$t6A?^n~z{~j42_1TfW#^wN7`?#|l^!du{I_qUSXYbbcqp$_D5% zsAmt5AI6XNL(hI%d}oMcG$EoOic=HbjisLvvn!7LD2&%XK-Wax5+JlieOPaGUNJQo zs*Q-r$*~P-S5s9@ebK_zr2e{Nv5aRt&18cFnpuf0cO4Q^CR1e1EI%|>G^bYNy$*8Y z50HDe>W9;<0f*&~Vv=3M!MY-m1H;f=#eRA6nNXlRtCExY`HdCljR`!QDdCSM>3#V# zRTUzy`Je@X{L)7=j)%=az41!SpZ;tCZ3o89TU=Rl8A_iU)Rg6y=sR$@4uAlnb2<&&3FZzJ4#JDodlon~E6i zR}8Ja(sC2Cdt6)^V&1(x(q>U7*~@!&1j)-NQXNbb^N|eTj_`}jNK2b}kleaHw%Cp5 z?1wa$xD_W21ln2Zjb7(3KD~X*=w)_so#)xFHS412Dbw#aV?mSBjX}70En4Glc*9ydv^|{lXTj#jep&-Vo#6R^6K zD=@vjC#jo)H$!>~=_nQ?wwk!FUhtYW87kO>TKcX81)?i?Q{kM-5u2#VK|ltl3i9by z1^#yDMf6QF-<>1HPvM|^h7?oL=bh)=)!*F@VIii|aunL-`-l z?u1t$Q$)J(CqUPK2x$Knq9gtWD6&jbx8IY%n|8oJ*S*{BUsmo;sl`9pz8@xr4U*Aw z6DiC)hulA|7Wo1#Q3qlQaOZ5Qt^MOk;+F_~zg#RZZPm-hv+bKqtgWjRe?P)0;dC;6 zcOQqo!b0{IAnQKI&L+%?eU5Y8yT#$a@k9&iG=Pg-KJd5-Y6HTkchy#qM;-hCa7m&V z`LnxX`3aV!Ertt?z9MZHcK>1P1KJYzdzd1`!C|~?=(xl6(ZB$~sp@mg7CEKPz2ir- zV?`RWd`b9-AvH%~Kt7JGaKk4@;OdZ3TN)~nVci7UtP#Y|c23@FsB83#F5Hf52 zmht=m2E?#gvXrb-)k)cF(J%yuXJ==_ar_;j(-C#ex^TMzbg7JB6J*uQSJqFA<`K)= zl{t=E+5>`tvQza#JHI;pwV}aVvLP3aCgb`XXavL#SZfJwXlco~tnWsP?|5T>MD!M0 zaDB=%Dl;?FSXaV`OzUzkbg>Z&OGi>kL(q$}GV&7T^3hUFK4|YY`=;9AoWROx8skIr zy)w%<4P2d;?CQi?IEGX_ZDTR^t_}Tp_PfZPu#uJg|pdE8BhR3g?nYw zojul}jHWB;iZ0q5XWJHNpTXjI*nq$oL{S#REUKKZK8#*@07#K{((`k35zvY%tWvR9 z?Z-Ap;fyll(*+3Ii~|%wEGo!Kbv}QBs$psvh*lET?UqW7-vFNPMTYdF*<{<=J5GP1 zr)EFVj#S}&fdyl{0vL@{d`+iGB)1P##*kZVzQDBefvip;l7~hpc}K|02F(cCAeis) zL5jQ7J_61V0!nhwYC2pR#@m?0lXrVeLf4z1n!&U}ISC=71k#wms%rW-B#@j%Qub;M z;2a1}V(`4Usio|vU-N?>g)mvA0U7zRP~G+mM|n7O5#^(FoUK9jPKA}LQ7$wto4LZj zXB*2PObJ|?xNSO6vWn5-`#G=Eq@-`HKYn z=MwTb%(kN5+=j0%ezlqDuWaTClx$V~V2a}_7=u-_pyP$v%aho>O!2KF-w);*2K+q- zjmys0loBRRObX zFp;B2&FyzzU%yPtd~t{7q1d=YF^+a4FQlLe3CXCHi6SBUHE;+i*^e1#-kc0Ud~-+W z>&5DRjil)0da2f`aOR~#8KOpx+r#>w^;=K=`7u8qyk!Z5vS?^*{N;v8TQ&zoh#_KA z@q7F&_DfJ=echY5CkPntVhn$)lRu(8=Es+SrC=@*y3aFBvK!K)!5ZI%L zomVOtR)plZE%|)X3>1A#V=7ZvoJiaZbZ2yXY}4Q8)TGm6-Q!>}_R?R->zGS)6#UJF z(eT44lFrZcbl$PAApGcjD9QI#Hf(a7P&nXyIWXJSyc2Fdls?rx1exJlaExn_N(O!q z!Jy%+dv_}RRT>dL*JO2TYisVaQW7MDb-$NVUkAd&kIMpcQ5MUBBezZFtW#4iG##%y z>#P1peg6kdl8}f|EgjB6*eh3UgSd8j0i@7r@psuq>yn&TmcgwmjXw~^awW(-d;a^u zf@r$S_EUYI*2Gc6klGDQn%R@UE-nFth-7?H+Kvf0j{m$%$|3HeAlhBK`sp$9R8cp6 z9zcw60(*3nyawR#%{(t1&Fl`mB|?c|ak!|vF_F+wl^`7Cn5ZavH4;AIjiJH8k67k4 zQTyL0ta$xku9(&p2alJ|f&@j6N=Xiem5_L`<_^Ad;ifk&@4mrXGoS*7BId8|s684Y z!++5E5>tmj&348-^mi0(@yX6>9(aDb-*W(W7TM9!VQdjkdRqK6x@!^zBXZ=_)YKrY z7xQYbuDhI0c37I$;S3j3dwFOOoclVcda0$vWkW;Iw9b@wf0M?H!T!{rr;yJ?y5J%; zc_Jh63(@d1C;^30*Kke9_+b9Ex=q@ zs18-!Wpv95sQy*i9+~@@J5on3qHT-?dD!H8hf4$_Uvhw*gY-gjY_Bamq(l3&FY zwYJrzv}F@X5*kadORX6ZRq92a#9fC>HXyfS z5>nyT{QG#(#=2vp?H(Ec`Zl)PMI{x}jGZF$kM-G6AdxdKJlT~VC%3I169gYIm0cNj zT?@qAxl8>4b+>&re8!OU7XmXS>`o5ivyMdC01QDb+7aQ8c?vG;OQOmkn%gY1Ta0Bi z-8sk8ZXw?SXP0WOg#kkD${h%{M=GLTtqby?FXdedMz-FckKxgB#MllyjT2pY#k}GPQw1LETg%7lK6nZFDG%G z?5v2dt8ByciLmNYi$sFt3UNRby0ABw3(luFG=zod3c1MNCnfX9eJGO*^C(52*R<`a+QqP=StR zG=mSmdx>*0p-x!UWA#6Nv-_vr4~B9yA}?29 zIzhV!wkFS1nc<$_{bU>PqyNOfOynq`xn&&z58riA25sz?2y_|hYr`H9N28tXLptx4r(#TL*3{zsnh@WukVpD z$bmEtU}?&I6#u?${7{(X3JRk6RS4KY*Zfvjs#-aQP25Krh0j7xaK+&yIfo z{2}@|8ur!=X#lPX56gpF(@VDrX7sF93EEX6?RapNbiK|09igv%0tKxuQ&ng1tIlx# z=?%H2KQ}W7R38{Ac5lw^0hj50KBVxkD~ZJxk)9m4x;L0CZKX`XTe5Uk-B}|9MR;VjY2?4fenMOgb77&ulNo2)1607#46MlLyKX!~mNi4vTB$O$oBpwtm0)mxmBCpr_j^3_4 z`7iY6493I50|Hmy+`POr+4yrYr(%Iqi^AflRiAAbOslcp^Q477jyBX{`^`4b$0!HeNw+|mJKw^?3>GBW|Skti1G zoaTQd-T&X)y*L`BgNkcxY(zQ?VOxk;CI=nH_mjckR0>0L0K@d0!x2{Zw{-;4Vvr6h z>8IJznSY3Jn{^L5QVSV^I)5%Bml}oy8kt@wPyufdt{A&(;|Zm~{j*uPx?39j`F#O# zPYi@TRpKv@K$Au=>X54X!(`<7+~_qRP3YJ)?x92O?oIG@-{e~dtzTs=FbaM$mb((b z)Zl#Y&VerBdhdfE)Q<$fIj8k_Tm83h7yad%eGfn?7Esa=mLrX~!@eM#7hpk>8Qdjw zvf2SiSWX6k4gBB=5B_udvRDt$^ zbwC&9<{AmSWG)A(ow0Y>J8Ow1LUNTgm9^U`-Ji`V2(9?qKgjQN)jx9+m3Us&wLC*h z;+8Pn=ZHmM!}H%VP1J^mhCVf7Uz-hqy?LIUWKC>0Nfv$l;KslW%7yF9|GsU)PW->1 z9PS$T&|FZ@*byg#e~030D_{`}zTq*|oZo~kgt`tm%*K;mXO4WMz(h9*jT6tr=|jxo z9V5f?j73IED@5m3nS)o^pge)!NgqCARr(!Wl8A-H3ZIS({K=;6{2d&aOPkz3g(~}K ze*m{ZJ&&{g&d4710iVwC$*?RCOa4`<_Wz0DF@(vd?>-XQRm=b50~(=yGo4L^b6ta{^@^G+7-nwKKxHgzYxyupZ+JMT~Ykv z!~dl83*qek>HiNYp|6})eM|6b*95Wy8hKU`NQ&BKd}5-S3Myd++S=MDRKvuuHiYc0 zsniD)J3Tk9`P@>;JkORA^i5WW@8T^w@|(vVqh7yAMu}BaC9^R`gtJ^om;dZb?MxVH zJZ6D=?CaAn$*9XXrynQy1wCdtjlOn`CMD?N@shmrwlB|D2nj|xvc5XsteO~}mtXZ| zu3$l>+E#$~L|y*MrH0dAf%63!AprBqJ5>e?@F4_do2E?<3!MQGreekGTIpZ9T%TzH z1PSleSF8@9v`XR-Eo%5W&BF)|&mP5eq_eG$j}JvCklir&8V7qfdxSIHfM`jF+TCL} z?*C8DpNteHnt`M{I59ENV({x*hoNYHE@+v7L?Uu?@!uMOELx?OM~v|Y$i2GkTaJ!} z(dO4%y!tHD&S02h`x40fwY3rAGx@PG@5zrM9%#KA_??yfV!!SB@VjcV=YuvU2C#d6 z^jJbEUp8YDXeC<*afb+BsTCGVgy8ul!iK&cB9?wR1QZ1t?2XWbv`F?Fe56*b7Y zDKHrWNUV6zU!8sSuYNU>n@c@R!Eb{ksTjxF^LUhAx}H7bdtT+$yVyXH2WVS_PM?HR zfpkSsvTlQ5B-j}S9~l5&tgqm0+@}06*Ks(ddeozSfh+;>hU#G3+1uAa3Fp2({lohR zPy)s{kod=aqmR_hpv9kI=~w-dvqiKB*#wj( zX$Ap}tisBXgcjX3h{Z9WGhRyJmczhrZE-;qS?1zd2-YM+GWdSE`0U59N~hz2(G?RG zMh86rYEQhZ>l!CUiJW-er=okjOx^9$DRKI8iiS;9EtUX2Pvz`MvmI9Qe}3N2O!)5u z-_;^v@@G2}5<0ZD4i*QA>t}Ng(XAI|uY(u9u!d?2`Og>FiacP+AOn9ESuQRpvO*3c zRio(%0BiEmUq1N-aER(A8g5zw?Q;M*xg*zE@bA$f{2xOALViYS5Qh5tAK&IE)5_67 z$6=I?l!y&$zy6m7zcjMV;K`2>vH@>iS!?}zKQiGHZIbGt^CUDlU>V8Ka65pR#wlrY zyF_NL*pM^rm%EA2kdT0kkIvnyf6px-YV!~UYJ)VVfTFDgG)ssH4IYB8f56u9C{uU8 zL~gFMMGd|H^m(3>!{fV%|9=Ezu=Z#bY@{4^oRZfIX=n(Br($y*Li4c@sg(t~=WOfr z8+;(Q9=2TAi^KZInuNw<0fBJIE(H}1d!_Hlmk=Q#>r}ry?@nC-1 zn$!^GiM9ewGIV~pyhEMeWBYcKC!tl?E?*@H{*Ey}{()YG3#&AcrQfWK_iCEH3riHAXZ#VmJ%m7mJ zS45DJ8)uwDgq)q0M~c=*DBh9Zf55i;_(b&DF^w+vXLxtJ?C;2RkH|v>QFn8PA`!m6 zQmJ;>=eBa8^{LZub09N|RATDCetl7nu!=TIPfFo&E`JW~$STwWTOly6_nGP7?k)er zhdW3sJc3}gLI+!aX)Ly5L0MIYez|sO} zDN$;7bIw8zyYGR<`OaKm?bomxK!1jW{m=GWmap)|_wAQ9QXSy$Y`>u^^pKA4`fk33cVSXI^SbT(4D)NbK(14oz$$+*+3`~6vWVxJ12{mc-Q1L=bXe*-v6Ndd(4pEV^GC_p|k2R^?6Yp zo8{?=_sI)I8*2#f6==pS3R%&t3n`wC@r3?-&@jFhKtr#9@}rnLSM9t$V*$7`-@@`| zI4wkPbD~%0!|4SW1%S~k2}-n-jnfTWY+GdtBxLd>9^=RG165fUQ`|41^pDn4BE}MFWR#SMUV9fZZWY81;R@D5)Tm&|fX} z*7f6*NIUS#imtMQ1l+5jq0q9t&ChcIZ+jz#24XQ)dCS*j?0MCH?hpoK9|n9ntG)p( z4Y(aAA5niHs@J(xgG^^00+HU=yulS)5@gw^5(6XUM|Ba3tWxZa1vfo@x&oN?z}nu6 zFuFHcEJdO(*nGH63=PGQNtpL~>Z)K76RQSf#p7B4Hi2by*RwbElAXb$Zj%F1VFql5(3V4kG-c<=j1hvePN`#yWy zhU@~&`VbP7Z}`ZrA^A@q*dwqmoC;L~G>WDnrw^UGwrYB`AjzMpHo|AKGK`uc#n}qd z6&g%ZNmB@oQXU(fzqbq=Jm-&Rp)vAkI#AFE6^GujrIZ*byjuZmfEF)nY$3#P@v{Pd zI}#MTcB!Ysa9Q8^c9_=~ai9%so;q%2=|KW3#E)*&C$M9FXbx7B>ZXpn+ zB?%tDYYz0-r^FjUVVhCH7jL;&NH&Gl$IIOcfT-WCcTMAGMmTMyB%6Nam9qbifxab% zM@28);>2_x%&qZYUYG~0Z*&I9jn+)p9?;%@^Q5M0gMA;h)8|zx@stV4oY>%#3z%)EN zGKZjV)J>kfbgRS-LoURM{qL}}(v^p~VmnIjCm<_t9-{l^Tek_=q8wGN^B2g)Q`gI# zS4A~j<`C3YIiDSp4(yIyzfpgz2ooX7{gnRxeFnf#JZxgV2|9GvRJr1sLX7xKBPn{4n2#1m>2ddQ zzDZB=cb^f2p(ZXj)?GHc8rNE+


FIEWO=$V+6}*VN@m%I-Md#{LbZqOg zMHKYOjmRA|6`aY9TA|mRGG1l5uy+AinAN2oSGKB(5+@kZP(_@=##Bcn-Yi0tg8+yl zVr}lpDNL?xb~6y#m=Vg6)5sQX1%TG(301g0mo(645Hc69*EhDy1oo2G9^GJ$3ADCk z`DK)a<0h|dqkT6pN_uee%CqAUVBEM%?b9n*^UgzSjlpCifqo9p>(UNAC+m=QPo>N!CFBY39G<`H()_8k|#`^$Qd!$2D=C*g)1ppE`NoV|A- z)$jX1UZ+zX4UvqjiZpDMttd$u8Ch8=iR`_l)6Q&I*^v<;+aa@1_9i<*2-#%)u3PET z>;3M1UVVT6ox*uOpZD{=ulu^M>oJ-S`hu9nw(=9C^`}85ZcK*=uCoIyI zmm);LI?jf)qnINyFs!@%&>Y`S@F4}2LGZU{=ZPL4Zx-y@aynoL5i*nJL@tcHO08n; z_zoViQUJ0$?80?oYG$VYrq;NNx!Op8uGe9{0aoKEF4sk}@+YP(uP=X#t{VVWI^X6` zSKb0ZVA3)=D#UfMrlTVS$$~_gIrq{H3U|$)9|HJmPULs3T9}h&eOt_nXYIXv-J3x~ zQF>Lrr)~h!yFPP`>u*6{>A)&*WxPlh<4(IL- zgeMx(($b_E5_hu<6!wi;cspq>)32OP&%%)`1AkeD9J@H+x;$VF23^r#T&Ko6oyKG{ z+byMEX161SBxmq~3k?GQ#W2w>IJyYdP*Vnm!)1y(dk6$U{dH5M!jp64L~< z3<*VAcQ!^2bs(sH67>qb$%mFY{E4;gvjkNyNZIcbt)X$-V0@_p*v`;W)_2ZV*+rww zR#1#uxt?hqzoPr=Hbkm^Lw`M4^p;&_r=tInx$>nTX6`R=c{nT#(3 z@{ft@=t9AU9WNmJjsZ-d#kbK#LM&G8s~T7gxJ7B{dMKP5g%h1|a{QEbRz)B$o0DLcmvSKKM)h zH>!oO37?o3xTfMGDLhpMJpI62;*_ud+3D$s5uN(a>I?GyS=Z9h%*e^^zH0$$|({69x(Z9u-Ja21|{T^xeN0+9KV(#9) z?Vu~z%+tumePJ@wv3P0~_)*3JC;NX59lblV7AML}ZJeVR=+`INd&WqpnKI!CJ&fe1 zhQUWL&ujkpE!%NgGj~G=5J2n6I`6KhcCfBA_6aqY=g-7|n*rllZ$UA;7PI~|wfM zDqBo973Ys4*BLlS)A^Q=63uAhP9h+gU21-QK5cfqHx{=TwpujYc5n5)$;80(S{vs4 z?lAJ-&#OmmU}#v@%zjEhFCV{k=U#4>)w1C?pZ0eWtH=0)Lr|28ENJYC z*YhBzI~N#oo1`(T1;5KoKVGa5RMev{wB-_gv4?UBfM7Gp3k{(QGe$Xk*JCaml&|=B zg;cI1MDqvZi=GqQKe8TzwakCNsbBx2^jHj-nWH9ha&nZ@>Z&9^#`WRq&+&s`))92x zWcE*M`&Tj?{uQH;5Em82!XRM27s6lC*Jitu7;G6AnH6t8RF^3S%EpG>&`p?ILrFH0 zPVhf`w|?ErkBe@hKE$emOecz*;+&9PF^{S{LHY-3Oa=*XTf65F7vR>TM!Z?*G$40` z3UEQcv4I*up~e`6>zw_D6(0j~el{COP_sea?q8aJtYp&1ZLO_efZS;sq%kSlZN<>h zND`~44T$H_=crfL)>v&2;FF^Si@)8p z@4gx^*0C9C(R^s~EyRQc=W3VtDy|2T|?;tPOWYRMe~ zz6&5Z76p=D>KW;;ia5_eNjh1?sG<<{_Q2Z7>i_L)SC>;gIBZJ!4xp54W~}Sd%mRAw z$gS^!-3h8nf$oC4tuFkySKqxgN+cmQn~6pnoR3C0t{QbcH{0=n3Oef4Ldu{BodSZ{ zfFp`NFu7yiWMWgXFGH4KBCtsVN_g+Gou)Msy zh!my~!nEy*505JqyV|1-Ry;{RJID_)%cqokF4P_*eql~!!<`rl>LE?k*Yf|{_u@l=Slg!x3?kB+Vkd>;Y^>Ld zk35A4!Mqj37@}h<)ReA!2vXt&FB2Z;8YA>Cz6|1Ge8oV7>K~`*+U6$%(myVDT%y;v za;!yQ$^$iOmzQc8h~CrMFQYGqIu{6lmksw2z0z-M@9#SDzb#K}v=a14r5v>WH#GJ? z(hyczfHu^HK3M+kvj6cbzdr)B!+j9r@^F$n5!T#4zDq0$&Bj)W+C6_ePyxuh?{J@u z#ozux>@oNQDJQ1yU_AKyX#DfefB(@1R)|kgZ#dUPUe;doKkoeZKkx-laMkN_r+-y8 z|Lq;>NTHqIr@5GDH{iGHhS#B#Zsg&YwW{*JK8wst*j0QcvIYL`nV};bh*G-a| zwBN3aKTK&5SQnr3S69F*thU~vX6H7G0tGrxLVTj>+ZIGZW&oW(sK&3xl{NpLH%7Xk zDM**&%ltQEbl~uBxSg*3N&VKC>4Rj2bpev&tgXONRz%118V@tBm)^*S&M?4lnoUXGMfoi?b2y(J|D{K5W zmpXL201EW0qy^Zzttbgd%aktm)X#27$1qGMB`DWPYLEulh zlTKy6kv9I#N;a|P``V`rT0se-!#CM7xfdPO>yUJ7H9P$E)65xm>C4KsQQT{`H(BQjVl;=f zk?}fc1I#ZO476U_gCVat%vzbCmKShZb~hqPBK$KX=&VSP3bJaM&@+%|KJqFc!;uA7>k zhkCosA)WVFR@J@zpTAEJPtto5vaJW}%0|8(o02g%rG3;vJh6X9{lL(2e9yz2?CiSd z;`c4@Gqv#i1iNVpvc>5aX8=|6+1P7FvRL3VRZ&vX_PT(yEaKR!(zNBdI3fIy_D*ih zZl0L5@8l1nOC~GUv;e~H+b@p`AHxVGpI*hu;85yI00T%7RS?fqYR->Yx*Hc9CVQOd zyw`EVGI3L1_QEZ`JMwgVkJ~_XSQvn0j!M{BU&N%dky#yzD=%hKuZS{Om>H2q{LR$^ z21Bn^MY>~>fse(|=dd{~x_te}AswJBhj_GdBV0;FB}|q@C!zkY&=rU`TBr9lp<8n| zj~21UI3yW61?<1+AU93zC+w_yB?5F|?GfTM^`@1~a7y`NScP87%Sa5Mf5;0eC*_lJ z)5$}wd6IM|**oYck29<7&esRX7kXU=cY<`QAmht2^=~$P)h_u$pZHc3kR^6U;L72Y zT&LOo@ODUwEtP%M+>`-KHiW(6C9yFwzzR@+I%qknkE~kQaNW)||8(lp=T?`BGb+rO z@f^W^Lw2n_R+-vC-I}>49D0A_(Enog!G$C~5z3s;_YEn-PLZd?zWycoUio=U@sYOT_Jl=7|NQ&6torzEB#3VQ*W|<1 zRt;<lEv3&Tma!!ZHJ zDxf+q8sm8OznI3L_lfTcrYrjNi_H3ap7LuhG_oQ@6#|t4rB)GZ z&0ejZAg7pbq`bHPDmwY`sOnZ3rtgy%DF+L=R+>i&WU&-Kz+dO@Y0*Y^Yd$O@ej|GcIlWC_+%c_t2NEo1pl|Rc=&sOf{i448Y5L%I@A?7?)pp)2d(pDJqyrUP!qe3W zJ`CoYfa^u6Si*6IUScs!ULTX(po%S2DhV__f4tb?` zlo+{)(;UaMQaljMu^Ko}IS}`iFgW@JgGv@4>@HW2R473@scyG^@MF$5-->GGUV{$Z zZiJ6fxWT`M8u??gt&@T*o91`Q$}Ue1T;&Ln(-Hcyt_I&^Iw|`@Cqd*95-b}y&jl~; z`1}VsMw@k_po5PvpPjD#&-9}K;Hu!wG2n&lDF9gXU6Hhcvf0GgcvmnR_Zx~>vGptE z$(TKEoWudWwszj4-plU)_;6Z`-xy`WIOa=Nkk3MuS} z!jzNQex=67KMc96>(2ghJ^Y-*4N<7bFp(7$6f|tPxJfo#{m0lCGDz~y3S;B|v6ESk z%qFbC-6ZD1c|n%SK;HPk`^T7F^85TWqCP6KUu zUnIDss4qE>TiX>b;bhdpEsFGo9{*Z0fBZnXmz$gWk(0uQ#wV;d&r#!)f`0%vEf`+i zCM{Nss70Wx9XGz+o*I;M6VIZxMuFIbFa`{3oC>8hjPF>aWLy_*DrFzXGth%GboQ+1MA<<#lJA zTI)mo&I^3H11fL)sBRUF%PXD*8TX?JKrN`~U3f2f;t7g7ea<1Bzo?sO;UNA zx9tk`o9>N+N5kf225CXJUE)Jh3(d)@G3No{`C-IqM!o}A{OvGLchU$6AGA4qkXXvO zHegEb1qOwLh?V54Mt26#nv9VIbjBb!E1iQBw|F$pu?tF6-vM;?biw&scOm%B?1+ke zt(u1OC?2-UU(h0T0X#Z&&KZ{#kmn70Yj%4F*xYQEx^$mQd=aFE4|h0ERz@w2L*}=A zp3-+Y6olbpKxX~w`{lx=6L3_^2G0mf9S+h~$Za0!N`5~QI++ih`)J(qc2n{icUA0R z1TTm|W*=juG4NuxpE!0Lgn8!EL2j>ZCbGj7_{M_(Kyw*X2i@x=MRq&ktPaQFgeZMG z=;M-h_8rBaL+F3E>JPs*)<1p-IDtplJR}Y|e-GqzOw}P@@aYn{wI87x^W>EHSx-@1 zxS`7rI!R?4y{LlOgvd^&&S{xfAD{iqs!tw{JO`6=hB}$QQ59z4HZX!WiD1@D_w611 z7VS3{MhHc%siD)aBE5zIksyVH#&2l>ibsJm@%1xaBb7l~g!7UHRNN`*#t|kRszSmS zxjVZ=0R$Hl8*2;RK)zLUi~AN2)_Yms0DN9&g8AdLu+FyK;G@~5K@9v&Ox0vYL*-aRXG_VqX) z2`D46dvFrnbdjwwceq-o8N2S+uU{+h50dGH>v+g_X7vXeO-$gc?Msf2Kzj%~eY5GksWvkO zUIUe-CAmo;21I|PoNjqeZqX}u`TOu>vjC@GDTB%Nc3j%=$!FCv4#{OXRTtE=hi2#xCge zg0F4~;W>S7St0dBH;@tAnMt1E9(6lX;yq7E0{J0-$O{PjwTN?Y0(U*=ZU-MPf&SzW z&`*52rAk6MCYGIgWb385Q`z0Ow$|Cd<EnR2l|Y+o;}NM}@=2-*|3|mf{sSIO1GnUldQfvz%FlX|kArY9a-p!6VoT zc9lI;rVEpRPFl>#1R2ueVT5z=cBs%TtINl^-gQmGJ(SSPDCD?hPVV8=r_20+>g1cp ztnm`OnQENru~${!?8s|+e8SuIh6gWer-XaWCGtS`G4d(*eDUO~W@)iFRqUq8dlrjF zTbFd5*gw)r=bkysxlDRSBF~`MrPoR+8m1ap7bFG;CjOFU{ZoFo4u5XTY2FYV>*)uV z8{?EuSv{SycAO^JMruzP#vIG*JYNb^gOO^RdoPnF)fiM07>~^1m?@jxX9c3?7{fMm z?bzKFS$EY!#-g&YbO-577FvN7UF`(RbswZ(Ajc`q48a6pu@oa~4@yEIrJ4{mfvUS;TCx+hj!1N0b@?25(Pa zxg=$VWjcZ=a#B(@pD`rgf*^~aZm;vMTT^M4&P>hpTkaWQSM!ygfkPPM+ZuNS-1DWM zTI?QKwn)P5>VzF=J|iFSp;CneWh#jk!zszS~ggDt&^d z^&-NH)sj{hCKVPE1VsIq8I0U}BNba8TbT5zR8lz+C+lf# z9L%3p%tL;J);af6n0v48!nbA01+OW_%W*hii57m=JB_C_(hL-?(?EIckW1*-`-O#R za}1)28gJ|^u3M3ZvzY|+>`*xV+ThY+5A}sNXN<@;lPV=ob1M=jL|SgnH9`<#O5^9%Eqn zrjsV$@=Zt0W3~EWKgx{x%Fbx9i!k`3n7seq1cF8DhBco;-gR@O~(-7#t8?ad2aq?uW7q z%EqQzN>A9ni&}AW?*MVaL10)|xRsqH-|04_QteP><<`+jB10(n=4L6G^C@iUV;IB8 zqkRjYrnR`Q%X_8|@}P!V(%mVpFZR|VyO_@bQj15L9;tY6-B1k!2#w%2fe_u@no%QX zeU~ESkabz&_7h2I@-FkqO1N*iclW}gEUo4=k6%h7Gyoy)v_~WoO(YAlu!g*1 z)a4bc_E(qY%T((M3dBC_XWfAqkti!Wd`kpH3F(ENx2T0`297vyOrUlwKEp5~jos6^ z&^a3o=w%;L&oZ&`IK96KOk<-5X?KS=WlXoFIj54$feCc2YNe4~FAZOG+bqIo%1Td6hs1htcPC4_!S*Q2)V=w6|dxy zlv^uc@Ta|{5-D4wB=4*kGi#X(_yy5h=AacM4h)*`EZEyDMRf-Y)a|LnH+_W?h&Xl= z7rBsqDe?wTS_giD+J5o6j2t@rbM2O{UNuU9g1-@SCX2P`!)*=cd~ns2YD09@Gw~i7 zkXP>7ms1G3sZ{x6OqwE{ZntkI@h(nw?`=hRTt%DAo~+mus*`;N<06QK@HNm6rTD4V zVRLPizH;|7aoWg|=|0D>u)u@k?bdU=UNgu2PPsx<582_q#ku&Xipi*f<&2yXfE*u| z2@*AA#~&s?gt7L`zGHN!nhA&Sd97u>Y?Fxsi3VH_AEYg92y@Gb3E%=QNas_}%X&PNeFt9p0)Tq0A7@htE z_tRhgq!u9*cbjVDrJReQc7OKViI212L;%574}=t7|En$l4StFYA9j7pQ!5d^v4eVF za{ioqK2369=hX%C7n6uYwD^WugeHFs;Y$1S|#LO++?hX}m3xA8PIZGDG z!yR_F&Mh1fV7Gtaw4;mqa%IGj;a7M4-O^YiZ1gGUkg_UiJta1M@#nTG8Q$b%rS)ur z0G*;}KHA6S>M;14Nk-N90ht(ia2r%|lN}Vt$lo7QR|=$~zXy~pSwx?zhx&xP`j#0W zu^5wvo)+M2MHc7o+EWqx1|;$$zqfy5wVB5Pd%*lO9=BU!Ix#?_ku2E##+fWOKRpHx zdgpiR<;JVNwDtL43KhS0nQk9}=@9>16Pz0dH*(pZbn9}8Ovhm%6bouOjildautbC? z{o}(I@0)~uy{G!7hjI`g$L6R!F=5$(#$rd#)x}Y-W|_1vmy{gb^C^JZs=M-|LhhnGG zVl#G`K#-U}zWgw1z2ugI+eo{8NoVlu@PXtH!Y{J#W59Pp8B}E|D|DfKpkJp8F##f; z^miKukT(hS`CL#`G6t*okL0D~jZ;08C%?qT0%2k#1WdOGfvrG;_p)C;ZOaI(twDw{ zd&%!mg$TF-%ZPMd_%$@$5c{W()r^8*D_Hd~p8**Q1|YC~3_2A>jl@Cx=Ggmz`a37XzY(REp^E>^nhs3^^ECJvjKNU;qA1hmHbwci5jPK5H zCtv`a7&_vJ@R2>%10i@dy{)*QP$rgvS*B7xvJ-oGip_9yq7#e4K(&6Bj2H=S|!N$pHvj#5q zsKOBn^Jk#IR#O)?s3`GLtCgM?S$1jP+=gOdLLfU|){D(+5^8^XW^T6vWmPTSGRB}% zCTt<%6Lo7%kJ_=!y2Wy^-&$q;Wp~8FP_MSeALdx+W-V#SKtY}Gn z#E@2+ntGA}G|o6rI35%W--an}-@`$sd{4utZ)fAKK)Ms%xW)5)Q*_VM?PcxU^O<$$ zHI8&Fv5r!UQo~$V(@^><3?l5Mj&rf49j?GN11snc3z0joM5&&nx7ll}bQexs$|46s zh|lamEVzr*BZ8f{m4eFv6g8~0sb15F-&rZ!9ZaX$p$~0{`5VQ#XB@dsSteWS)LtQ| z{s4c;WQyI9L0S)^P`{Z1==t+q0{P#Hw#MYJjjHNHVrDZ_u>F4v&xz^uZ$xaAXJe-~ zPU%GW&R!ABovae(;) zBs5(qG32BxW1}0~D?6X6NLq46Q73)vmoxO+9*|zo2vA z#4KUa;hSjfeKnyO;Fp+Ow_<0|PzpiZJfw`T+Dg3#)eFITi|1+(aCkkS!0Bhm&}Kj{ zq4OQvV^)7@sqvC**Y=K=rigXhCTZ6FE8qCThd6N^`@00gFVhbXh6VuG9%24mSC*$B zZxuktWzx~n{_GLEc#+)*tdMwEX3c9KlF&oWH3;Rd>)MIDr>qd61kjEFEyckSI`l9*dJ8O} z>-2}mTYEqC2jxBSgj4#6#XDc^dIMF(CZfSqng6i11XAHA#l4%wH5X6B0ri^QR_@F3 zD%V`lF^S9)sw^ zE1kl4j*pcfL75dhr;>)N3kLV-LGYBDee_KoJ5W~3^n#-C^ZDb`4f`m+B8>xN7(V6i zI`2PNC(XrR8IQQqY)Nm9XgQA;|6mgrxhSmv!ouZn?s^bpG^=3=uTE|-*Sk_$@LYYj zhsBTolG=qJ@0OWpZApC%J@Kb*0JrBVV<35}_m$z(Y!%2ZWqv35w)ln&AjviWbLNT% zfLiM7D(B=1lQoB-Hm}Hjo4-kKY#FS^UO;2b8pwh`a_@MNw=wSWa?>mYZQGYIV1j5K zpY*wK{)4ob_XxWrH0|p#3e$Z3X4nPYO$~J{+U(>NC>>rKJvk|18aF7s4K+84rLiO##w}FY2BCHn+0YRTofHp57P<( zaRnQ8GU*)2phS7sD21*64%4z>lHY+R2s!xN1Fc{jhWzZsOjE5uaJaABJP;uj)!mL0 z#qyEIn&X)SsI9UXHC(*18N@YSz(A3Rz5b6JIWPenXqBXnGro;$- z0T*s0Je7adgKcUf%YDoev@_oU$+*p8y~CE2Sjftn2E!#LRqT8)kanj}~xitbS#n zFT}&A=ul-OPG=zkT~1JcaTEIt_#;V?k~7dU75~|Xoz9jq5_uS1W>4=(e2DQ6NsjUI zM#0vzYhN&&R32)*y9g5KK)MbFB@uTH3?A5<3B}Jng`%UR{iD^EljGyrkgYg~!FoBq zVgQu3N5S&a0zhF5!+7+E2`>dytL3^{h>ax(VUTAZJ|)1*yO1RU&Y)7H0L)rBrcOzAa(LLsQp`jF(X!Lt}_v^wZa@bK$;H94gdUQ-1J4TT9b z6K}UlZ3L|Cl502C zi3Az0=jF6P?0O=Gup`N@Tq+b~&8w>a=d-M?EM-oz#<(6)jO0dNP@+k(K3NQ_20$L@)R&unxfs8@8f?y*?%7sJuem*PN)1t0H>^l6_*oU)Bb5u_P$vPQMkc+Od|`~Cl3T;gGr8v4QWum9HPD1xigj-9}o#E zUGP}Ugu;mEoZ3!H1eih4lTt5+`B`fvdaysfyqwzs2!jROnX@{9*oY!c_7Ak5KF=Me z`voqy4s?{)$|MhtPaKF^)du=w*&QV+;LEk5AW%dzzQ+T39pB+OgHL$|fQwzvy>M1I z%Zl#X0eFqlNw2O-@^JUAdcaUtCjo2l*iZOaLqmhlrR0QAw&}`(UW;-78Myf=oJ#~s zaXtieB$y`ENbl|z$}@IF`#SU1N4q~3|1{rOJ7E3CoyE!QU4CnR=Lp!qf@DIg56X2T z>K+4VL=T6g%&IGK&Zjr-zwA|J&sqP`-M(Le{8*@~5^iL4^gL8u2|_AL@9RJ?UZfMCuYOB$kinJ*Wauky@jN6;nmQ1`y|2nPZ|?W)pQShex)hF0Wc9?_Pq zpF~}gPmckWdLkZn=FMMva$`d&MGq!d0Q9ZkJw_q@DC^NPG`!k8F-+Y0^TUsG&$O`$ z@CSOV*$n>q7L=lp$2cToNp0zKWNtauJ8Qw7CFPogCs-j38aG#ZE>SYmN_CXq~#lA&*Z*ry@*%0V% zNl1&({7(EwiSWFQ5pJ5G?V+Rg=I7#?$awT{J{7iq`T%zaPqH~e+QEi>wW;r;ukZFFW!Ldqko;&8|uc? zh$kxKy99dfe%&u0g8|OB!+dRoly1dKTm?z__9gs-GUr;Q*Yra@8a5z_g z)v`&YKt$l@I|zq{{|IycJu^$#_nIHO#=2SMwb;0oW}#LCFA%>B0{>NJ;Kyy!D4^Gp za542jCG2JImiTtY9>4>r*KjB?Zzs)DpdJJS&=XVD`WcRCa>RhwuAXB%(jsTRAHQXn zyhS2HU-`-wC?@DdTJIws4t|!E(hr9BXpS7ez$|snyh?8UpQGN--~F}i-zV8cIe;A? zjZA*N33{Q1-TP>N_cr?Vd;j<#e-Q&we-O&x(tu}+d*wB_suF({f8aM;%>N$jwg2g( z`~q4!P|cKfr{+R9>%QEqc=CV)fNqK=X*Ica8hQb{FWX)CtEct_8lCFLYRRz!2jQ&? zis}fz;JP?&bIejafdx+Zs?xegs4#ay8Sd_#zd4q} zP>@o<LkA=0u&N3;q_*tUKdu)vr9&Q{p5dsXm}>3@KU-vwu;f&b9-~Oh39ekx4Th+|IShJSu^X~1@EwK-R05 zm!2m&!iE>aAz4Owg#8!@QzvcMOo|xY>Wmr5p4F3>ca=LND>LuY4bkJiG| zf;^Xn)zU56kMf-7!))hHhi^fZ;t*>UNtJR(Wc?=|Py+Q_|6w1| z0RnEw8aa*i);a}Vqn`xbsG*C=vSQ)i!!pTbLd38GZd*LZ^8ilTP+$L)=|MJ}th#1*tSL2-~%!aeF7 zlmkN85+syM=jg3CFxJy8ZZ{M6njSx~)Ve!0thd#{?~G*@qznIw`Db3$3q>Gu(Aun2 z{(Obx1B!WBI3Hd(fWjl9a*GR-`)H&aD#Mf@)U^9N%;CnbQ2qx|ZMp&LH8gmg zjtx%Q=qTVRCmTj|rx1t`BM%$~2S%XYU!NXoe3_m;{a~Qw9ICHesVZ)F7o@RDW_)Ya zM`^>0sps0ePH3rrRKNw+Ouhcdd7@JSsK{{4fU^S#HJyc$e%k5JE7hGYVi(n$W;eKA z`>16*$Sy(V$Fz4ZAN!$}h!(Odo*WUO^qyqLX*?9HPOZj6nY6n1dp zWDYzkPwl6vNDk0F;(xx1Ck_fhG#Fk1ewy4yY0ObbrGuXNM-d&OFIi%@1k>x zyv;ssra1{viwS4f%C@%OYOWz^we}OtYT9U_Fl!BCN91?68(H{Vf$dfx5hWDfVytR#=#4YZoA1|yvm3)mlqorl37$Sk@Ql?fuTzbL+EA0oJ)7|A7U#`P zX|C1N96&tmT^7WYnb+a*6j{(KEC)iKnxd(ZLV@3{MB;k+M?Zl{CEe&e@9!p)|1~efx#ZbY zOj14^5EoV1u%BB7GaZmn`i@FU-3-RJZ~2zhAp3P0(x~B#Fe}7}$RqV7#Ppzb&Ta%;^hXb4 zpdE<|oqEf}9sJ|Y%)RPjaog_ST=&JiB2vKv(XYooG5#?}U3=CI2js>1dG8$6>1fu% zFtf>fe{|gD&3*?@@#VExg1|w6=1=G9($lHFJ$6rzPGutcRa#_f$yX%mJ%NvE{apAv zhzT6qIK-pj{porlAGdp-Vj7^{C1Hx1Y>KWH0`Ee+0$>d2qCWBwE%1pSdJXNZYf5pS zwtf0cTPMHu=h(Gi`$Eo2A0n#{q;9yVMGbu(@|B(D85N?D9QWKQNV>aeK#!tcsu*KI!Ri zAq9ch#JulAmY|;L+hN<5U z5VNIf`dIs3ZWnxVa`GUT@i`0w;(eEAw_%5QrGQ(}X# zjnIkh>us!K@K?!+PKh%bdzJgWgh#V1o)bVb#TOD2AN9NXz#r*D@aYSW6St}(Wobd6 zmjor5xeRM)eR;BqvnaT^rJ>7aL#s>20q&PKy?4o!^_(PnDcqtf9_?PvfFtBcAF6uG zIP-b}Ol$ML0x3!F_IAEQP)XwgN#B7oq@CRVTF=IwM;kvS? zLWy^va&h~Q`G0E%aP-&{R@3>;Vq0*jp2_xQwS0MmKRGI*0ZK6%rfNF#kJsr%MVq@6nt{&-M#%d!`@yaqC;XkJu7K z-;93J0URWL(`f{@*8@I9r7lUZtnr|9ldRJ_X2x3QmByqSjmD{j$x_#y56YYQ+KlU0 z%qP`~j2#j)lv>K05z6a(&5JIxtWWnWr)nip z9|G#^L=1}8v;Y7e-Q@#Q{Y2%+;v|5P-|+h>jSugrV!1g56DH~7@~w?q9edBJC3(5_ zdh&CQ=K`j~u}#KxV$ueD;r)@2q~5Z#+2dLBcC|4KNEcrJFwcBh$+IAQOT#(?S29QW z(ugkXfh}3k+#{bQ8s7sb@U-E>tt^@sPMUzrys6AiZKvj(PGtm6c_L+U)C7ssaeF+kp0nN6}aipALu;3Xb(7g&25 z{cc`zm2OGQfNamK-4RjzC<4nTNtjqfUem}NzKG&OGF5leJftBWL!N#5Ii zIc!12*7zW{>ABeUs)V<)t-J75xCixi*I2E8IWzoMtUzTjT*Epof6hP?+Jit3~C#^#Od1FrTB?MxvP!}^d63sEN?L)2{+mSCdHZxwj zr>o7j$o4*ity)r`2B+!*XqLykT=bKUztKzAb!an#wS1~+N!U#tbk>J39Xe| zW%1Rwyb$+^r^3`px z*@d{FGPAuxX^Qg^c*^ROK9W{{K1$`q@Q$~Amhl7+qt>2c_n@&Q$}ivjW#>#& zW&n!xh8j%<2a(VFb_{Yj!trRzG-nk}QBMI8+VQ*}tqdg7g~9x}#H1a#k6^S&YE04_ zRb=fs_YpJ=d6ioYrFBLD$#k8w7ig22P4~yfk1N+6$e`U{-W#uB*W;L)+qI`ox-sja zNlMtS5tQzh-o+}4=XNla`+d^uH#HN;n#*j zcAjq~U)!sT(S)O)!ajTtio2U0CgzSQocq2nGF z<*pM?!`_rmpd|1^GhkdVDyAzkb3eccGS&slZMK-1a*&0SgVD|j5hB?n-iF;A0Az{1 z99~WR2R41_H=No^&B?wnl6h&;1|*FAbiD+lKTjkeYEU_tWL#ndni$|%qSL!*@4YXA zGx4XirVpo3cD3pU|`%inXom>8xKYn>2R;2fGzq0r>6eXU$Dr?=q`0Zx@ zL;`Q{Bv>${OcoL}ucT{RPWK0yg<5Mqm3@mC)SYhuFt0l{^NwzsFh4oq5BQ;(&`uyH zm!sN+(sOBnmz#POr%nAuHNW$6ghGO0I}ePECMS09G1vy?qqn~l!(1b|cz%Z0;{Ci} z@P`RRy3kKsd>T6SVmHT11a+=K-3@*ATUHeM)k`OL#^$dq((kByH8^Yy#ky!VJ)$l1 z)LjCQ?dTKkeo}WaVH=Zuqhyqzl5xU%=dQVNZ&%Ho?9%0ZXZFCBbUr&Ct)|(z?4b*+ z*^OJwNFt`kA~RFd-y}~rY4{@)-o*vbr{{#6eK;+nVNdV^z?|zfQ}`NZziTk&Ly!u( zuXvL46Jgm;IuQn~`I4)P)yp;!|MIjZzcMS5s=k!h=}%NMp4No$1!E}K$`Y>oR691M zHl@AK8rUJwebZvL**ZpB-Cnl5aR5@HddGS@j5rHqi%Rv^xhb&AN_2Kb6r;or^3d*= z#*kDT2bBr4Jj@v-{cdWR5Ej2=8<1o++7@o4ZV0NJHA*Bz zYdQkUm2o&9b+$V@Q1r0N={&nJvGu&@(v-GX(uo;B^Tu@2&$_gkWSVd$1#Z^0fRl`; z(}{iVpra5q+u|~671lZ87`gH!p1<8(eA2wgGCitCv&a@{=jg^y42oG${Ga=w`BhQg4Z{xIM7{CwaDuuJZ7v7W4<2EM?P zZyLAk5leX<@)7weaH{Ah-1$l>`1#SYa!?!O*)GzzT-a1z8H|=1C*{rzarmG33eZ+h zCzscv#;SWJS2xh6BsgXbH#av8fQO=~>C(vR|NgHvia(6)7vfgULHzuQ68^kogpHPc z#HQS|^GeVwq+Rmns~#8IlK;^VW5VDGzZ&gZ1|$KHpquo8_PT3eQuYnb9fB8m2_j)z zOkY0ok24|32(j;Z$g)12Hn>ga_V%*p_ZihKKeQ`Vl!GY_=r?2!J~26nQ{^JGI`!<} zx7=MmATl7x@|<8S+BkSXCAM5o&s${T{WamG;eOjwR)B?bA^>zhC`M7%w5LxK^sl?L z%{j{ta>t?$o>&f4K$}OkM?A%ZF`SL9Ua7f6k6hfccx-=v&y!d2!<{8`W}3@j43i-O zXId_l_>YlqlI|lQNrgT{F{e<|=*YM>(e9^Vda?7{J$@es+m_4ih9r}@L&{04FKPLe z%Y^_I?fOxvIKJG&)3Xs~zDJ);l9Sd)AY4kjaDK{Ha$1N1Qc2^cbzyJ>=W{{zxzf9x zDK|LH7VKPS?ev07GInR?FAFbGe*Vr?ukQbW&oFFf;Bfwwu~x}yhtEJVJk-1GV^A)c zfuzpY9Up4CGE>={erVW49)2V@6vY6H;YbplhChbifK%biE7x5dP1q{5&{lguG5uAg zAFC;#(bx$44dYT1#xel7W(gUaFRbWnW2mgvcWQxG6!7d=@EMvyHdpjAv!4U=Etk;# z6;2SBz?V;LPiWIDyqX+Mn?~izl<~8A!kz`z*#4opkn>&a;y1+(BT&5{5S@agA0eq2 z$x}M9Sj{+_*ywtl|1}lSCb=!NpmkG!zvGLS-~Ey@)Qc>Sl2kO_h*H#GElYUG0TNta z-%M5+sTZ}PsEup0UB zqa+PmFK(%_F*BVqn1MYzl|OAG(TYLSscw*0`L6y-kMhUwg>yhUm+@jmpA*|(S$uwk zDB0~F=|BZCdC~mPMqH`*=gY;Dg8x|arw_I17m=T3U=48y=GrL~)Q7fLZSQ*T@j&b_ zRNJ=@dj5K=r_*jm(08-hLcxdz)z5D}sUAw!x-U08W0|?emz7WhK{#440rE-EL?uLR6PPY<43MqPwh9?wMz=nm+Dby(?&zVzUp*vq5X{@85dAeE2<6 zL!3To&Xr~WFN=jUrcTp@7RFKiXo#mm{W^&E`S$-C!L{-K?SHuaF_5!ePF(zD0UYcmn9WBEQoa;Gjc}M z*=$fu?HG9;O0MPvD=|GY^#IW)Wgvea#mL{~JedNOY)dGA*}hQCK6$g?Dh73b`U7m9 zHGQ}vawgmfLy~QXy$71TrY+YR?1?QLkJthz7iypqlqvL-kYSBS?5)cbUoymgG{LMY zplNEY0khY%+PkyzJbb6^jV7Y+H=G$WIZ(|7RFlIFPglT)h6|lGBpsvSMW6oFU0eO) zZTDb>-R6;V6g&z8@XX;N(RM5Zk{7ocQN2#(m0B&+ z>vzPi_8a)aDPr5=p;@oN;eC*3!G`7-9x`=5u$uBi^;Avp8@%1*vc$segAt^%TJ6^T zXj|S|y;FiMIv)*OBQBC4#QM5$6Wy`GU;x1TLOqhO_`Nyb_G6P*m9Odcd@;`IKwa+A zD_$S&^jmts0K=TtNrZ6K@3`qJ@m8G;=W8Ht`2ia`UDaU})m{-7U#QD_JJ=GWi+K^~ zozh{7eV`A5Jo7d))Frwrd_$`=b(SkslQ$8aUf66d`L|a7mu@&vv=g9cS`MchZPW`) zfoynmhu991ZLtN&WEe8fHl=6MU#>-Z@+n?76&p~@Bx&fTC&9dhrZ>Mws-esuxAnEa zatuS*{~1otiiw2&K+Ct=`F)_^_L3dE*o?vAO!k2(XoB#Iq{*q5haN(t6SZDXmJ~K{Kx~j*ONF zl8U}80DIGDLfqSAH#+59iMb)3@bUdlq7S|eFNP6;`dqFwt4`EJJJe?s5qmCLNM#k_ zT5(H{!hri))K0B&4X+Qi_z@;Mk0-5(d601fh(f(nmxf^4w=e6VY zqn@a69tH7CZbAWQP)e~LZDREljNTbQ;TP0i9*Ny_FBAD*;#fI6J2;BB$oVbj$ zIM<_a@7L8ea;fXrPGF<^xW^d1?f+E&`|tb3KSl_^&#u^ewD8JeW172(#iw7veywJ0 z#0@vONjG%r39;h!@0V?Q3_GsSt+d@llcTkpc)K5#40mO8vJ5i9IMlORF#mNI|MNrC zrU7{}nlli@E(a^?{*9jW-q&8(l9H09T0i4^Vs`LQ*+;1zi3($_J#hF@OnOgnx2Asd zY00MN=;7X_9sa9_gW35d4;4TL&h8)@6W5k%8Ykdshd4g63?_MG}rBq4p zTX0V#Wn*t?ZPJ8}EC>Pj*>bfXzEr~s{hryI&%sgI`)@b ze-w$zRO*u^X@B*1(JZUUysUfqsSOCQuR{@?^H+ZePr)q=$+UOR@qukY} z-ca5Z!rEly|3tI?Bfu3Lhd~%U*@NGQ02UD+juF=g`)z2|QXwn_da@JeV*b24ozJj9 z0I;>^<@&8vo(}$8Zw$gu_q`!WIAB3EveabIm7rqeP-mCe#RTKDMFY|wWNZ0~p zkMwL&Vdk295Y_0)X8!Ij3(#G{>iGZKd(W_@vh5F8FqSAb7zY_dQ7KB7Djh^cI!f<2 zgih$iNE1}Bg3=)YbQF+|0fKZC6+%dY(g{t0M7j`qfV}&7?{#LL_x`=}^?hFabTmqi z>+G}kT5JEx02c&3xJ!8ZkDk;e$hz=j+Uyms!vBpw`fpS#ukd=e|jfB7S~f5@J4mLoWA>CzWv9t`|Cv&Na)>nEdG}#(4SxL0@7OLmt)%qV zt@5us$X}wjm6W!U(w~*WUn-WZO6i{@;;l;Qzb>91#qIZtt%CI*B;u{U;h!Ypt@iED zrt2@e_+Rq&R{Qo3@-`@jwvy6TQu^;1XRB}c2P{>p9`SDe71$2n2 z_EeD-BE{vW%L9Ii)j($Y|5mBGB@W@z$j=V_gPFzm#M{6Iju+7^Nf#m1iv1evWHq}r zC2=}ib$+p8G%MNGG7Qj~q!oLvMf0?rIyQHAu|mq~rx-;jxobcK|3szLa!($EXQlrHMS5bO?e&FgkEhz*@`AjI+O zbh!L8#Xh z#18ReLNwfCfbm9Xp#Wm2Z^sqQ#M-64{0a>BEzxgJVnbep0EYngoj$-(Bq zTeqkj%47_Qo7sg1lnzA$PJkm&;GC>I0O46zfj5DTZ|=A6s|DZ|$56nUrrFfWW_CF6 z|J)(VguschP(?b?cz9(b){E80V)jt%^$YS}?w)Gt=`r_`g17OYMxTrw-~4=B`@BEI zAi|^X8=zW@ZV+Arm_kG+d{nYwHLZtAX0{hM1A2EdfFu6?<>U+pufz*hIDBKBrfQe} z5u$}7WsDQ*GWDkdO&)D^O?|HGuYooHJyLP60@RlFErIwyPD^d?QxgR1_rdI zjus%?VhPM6$ce^}zyq;rfiR{^fP16WbcKNeu*yWpMr#c66f8nWv|qbYqELL~Wd2;feyo43)mf!< zROydH=ILrNc*rt1-wM$}^X;d}G@QmAtw1S1NmBE*eeQZo_}j7_D!@BP-oN*um#b>LyZf%D2}sSw2GW ziVu}|E;a&OK>WlX*IUso1;eAMpGe#uCF;O&b*4RJ%xfZKScjV@V%@GWT4muG)hiBc z8!sWRXUxZ8didVe@LPqrzPwe+$>hnd-E4oTZQ@8n&sb zS2`X1@G`bhP=Ye#Q)&~wa^caR0GYi)62!P)7@*u~Dhd9@x{G=TUJ}0fyR#ycsYcV7P;Qq90IFCk6(WAu>4}M~zr7 z(iwYq5BYmy?>%z0f&D?)nm9R4F8L01=e4c{pN<{#mNcWxHDjaquZK>CKhMhgGBd>7 z7?L|twMnC2i(Y?}n&dqN$Ta4=M~ z9f&<<%Zi=~zvf$o|53>JOU2`F2T)%OKWVJ8?(hp-dF`l$*he$wEfJRcQ2y21_S|zB z(nbK7|ID$g5iVG{J?t0cCtg0k;U&pupH!kNbR=smY)~zHke<=qPgY%x9f#b9h#TvI zpNvH_*TNgQ2issX6OzDVxk=}|pldE>0ob*doAC?-&+|zeSz+~rQ!(fM;PlqvuLgcf z(|Y{bw{;}zHrs3l!s-O?#%%~B#QmBd4$3JH&nA_K^4r#Kz$aX#>-L&2^Sj2#w7VUp z`XNHd5b`ER{z^=tYc1iA=2Vj*M>!&n-8=V9CF~ge4C;|uGHgww3_eOd_tNXqnCggM z2zC82v%p|17N+0sn>uQ_aaNTA6%E0S+TUg%n>9bx6Q#F<1x?eZMAxIa<;^fRCUS`y z@);1H)~CsniRJJr)>F+64xtju`)q-*%yOPN^&Dx8$?m4FqxM?$T(Y5!kV&?q66%)R z=ixqmV6kk=TdZPQpA%=3w=$+G-|jm|;xBEtosUbFX88~zbneYN%yGY7vqyk71t-wl zf6uR%qF-lymOJzE(2+|KYb;M5iXOjj=<0dIxi=+Bi?=vu{>{y`-%dLahDuh^BH^c~ zbL+^&`fE(8*QS&8Ii=&~`Nun&XK_&b(0D=0Q{=gH7n@Re@F5AR{MO=tznMa|In`B?j zf*E!qF~=&F`s}U2q*)pnx`m@&}J>SgdeP#QLaqmhTC-rxd9!m=O2J* zh$SEzd260`LY+F1II;xD-N7C{KwARniOeH5M^?_pqIAX2B^%6kOK921C%U=GfH^h~>}NHJ07iE}32 zV#z*J2eyr>R$V02Z`ar>P*jxW?o4JH8(hp;@WE;;OAR;=WX{5> zgdRJLW&)2!^vsx2Tevl!VZ6gB^^&sNM(*B4%yrqhI1BhU$XJ-abL!LlB{QbapwbG+ zA|M6&B=Zro;usoE8wCBpV4Fn;ERxx2w_!5-+4^Kbeb!sseI&gC-eq79RZ-zWYT{@D z{@O}8A;5Zm*5uMSw9a1Xw@HZ7IzeXII6}oL!AV)~^}UOpeIwsszbtd#`qwQnA%opsoOq7$Jje!XP#yc@aHtJmh?p)(# z=MAOCn|6Up5DO^E0=(9Y1P7(hUohN$sL{2J3boGZ19kC@AdCN|=^S_C7s&m)_6`%S6>r3La;)U_Y8v88@ zHNv}9fG&dEB=uE|EiqbD0BA22&=@tg=_^&G;j4Zm`JA&YlVAFLGm0W+F!g?7(_R)d ztD!YhDZyOJ#L9;nk)OE8?@X7UG#MTC2fA>=@Qcp2QkgOhQa-i!mYwIN6nq9!*F|33 zH~Wyh1PPL*qK!*QyIkg3V%H1CUV|%kg|r@a^Ohm*)r}qzVRo9AoaPI&No`(Q zjxm8)DMaf_#nm{il&UgHskc~HLyMpUzP`W6M9zJP{tQ<``gLdF$x@~Y#g8}@Y`+)g zB=+x`fvSxAD>ZHsc!zt2PV_Amz}AL5rkJDay0VtyC(;7ucfLi4di8V;Uy8Jg-$4_w zf<(QhlreH4f;+9&GRfS(7@0hpIPN@Pk4t=Nz3=hpqc>&i(LsaP4Rj9jCyQZvt`-VS zI)^V3EAfNxCJzuhk)v@HGUv?1yW?r|AHG34VIoO{YN{}0#tZdaD$KRKeuGR~(p{4G zu~~-uU$M_G#0jlx6V4{{YUbVG^M2gsxld?x6|)QI>&PZw(AT-n*W^LE0)_@bMm&6P z`en*NE$7>hegW@aGV`=*Uqgl2EI8zCEYu%56n|TLURglqU3>t??QyTgqX|s{dxw|A z(Uc8ZTMf%qtG99-WFxpxM;^CPdd|dJ9Y@L|wJL94tW0|x1iIDQZV%MS zV#WEyMad!zXS+a#*}S>XN{07acxA6UTxCp{_W3r0-8#dQfB~vrUalnffb@r_HgwhG zl0Oh+Ic6rkRwXWMggzeH4%b?$ClP03ML2Gm7vzqW+vA@6+Ia6xrU z^WC|+fgiJ+=aU+Nf>{2k&Y|xe=I!P+tG9owYMW1ia@Lcso(O7;>ZhLK4hAd5rmrUt zOf42m%`NGMUt)A;j+4(*%a-Iit0k#xrAgCHgGMH@$0bpqM3SFb^rR|uK4mZk@CvEP zA!FrXkyMVxyB1DG9;IH$64?A&PR263C-4Ec?F32%r{^o&RIHzX0+G}jEycc*IF}2^ zq?_VTnE|W1n%1f-QtgW(1^_U&dh>kfqqKU;B-5S$vJ!gakG&3)mdBlfi^lt|FE~rX zoEVCpxXIWk?3F)xF-O?I6y{Az&XcLJY_w+|_p<-cg~`HiP23wN?*HbQ`dT;gm-4ZF z1x00>o78>PW#bP8y^cB$)L5j;o#Gg+1hRaJBi9z|UuDRtx)Z_`sGsiT18~I(RSH0bV0iCk$ z!xb~mc@rj!tN6O&2VuInfi?5DB5;2`W2u3z<1Uk`2lSmcHvLQ@kF2KUjqRfvFZ6qq z60~!^Z;b!g-}O zaz!J)7{9l&A%wiZdPiO>KkrO>e^zI**fcpWQbsyiUh9aWIv(>tDhbwNG$urs?%ZYW z#y{4%==#Oaoi1frah8&Kryz5WEee^wCk9UFlpAd?PFaZNl#-K?=0u063BJiTRxYG( zK-D$Tr_C-spvWUXe)blB8?(YJregh}zQ?MfaIxqlN|=VhlydMDUK=ck@764Ti^zMf z&-M$ACflTrKCD5vRD3gcR1TG%((Q?Qu5i30WhQ#iNwN!5&0%Rk7^=;|_-f;`yy82- z8V_Hv%a-YUHqVlHbO^QY+}xi1@r6Uo9{*VMW`NVx&w*8kQO?1~@TjmVlUF&xGnfhM z`1@^(iwIjmm9aAh<1|;#@F|DmlN%e~%7A#}F_UEH->=6$GMHBji`C7kTJ~?U4LZxw zU8TQ~alnPIEu*__b(@K8ENwQwCR=sp^+?+L`Jm4TvtL~4L8JcRm)bv<&!tS0m;7nC zDuX3OPXW40?^*z5v3lr<$=kmeBMbn!-O0_Pk30_0k5`v>%DtG_z}GFCiRC7r&1Zg2 zAIsVEbRoK2AUOUSHne=sJiLAa&4u!-z}3K2OeXDuY#!#;y^-wP6i^wob9DdWH^Xs6 z0O&)*4LaN&x!4r@Sul-MuP&#<5^EcIdY3d}eiJ!UCd@XdJ#qoi*bj?+$57v%IQ#lOSZWXf%;~_e%H>?NOFl z{aR-WBp&(;V+q{)$xdTeiNOAvXMaZ6cq3mGkaFfwB&&>KwNUnuKc zq##W16Jim7@5YNpoRvX#jg*pA6iucQ%E}|pO3wD$IfmsS>Nup6~yqK9@BN3ZaWux#^kn3F-($;>d&N%I&4k%BZU|(|8IZ03;mP1m6uGHjvQJjDz8C66 zHn2XlQaW0avo^sl*axLoPhm8hbbkalTtJx=q0FPpW38Mk$!T>Hq}otK{^zql!2S_) zdv{~FnUV?!tDeALs7)>kiPRktW4nBdHg&a@q9NspU`)I;*RP`ds$qURHvL_@D zttk^W$y$ZTWa~uJtiE(q{wG{@Mn0p}U0WeJc^}@SU8E}3A@PVP!Cy-J@~a7NgW<80 zkAaqO8bKl-|FxrtDSt&)+*@qt;ir%D@3;46mCL6@h&NKzN)nC^ z8GCDR`)8cELy3F(9C0uA=^r7pv>osl93?z)(kwfWk1ecb3z{fyO$Aj_e2$6Z4n;5n zeu8O>|ISK(Z&R&r-3#j&aC#7rC`(CPld`FHfrVU}|UXZ`CHOG$9?! zq-lR#TsIhD#X~0xHn_?VLtFrzl_u%$8Pl1EuxXYpjw&F91}j<5qU*?YPbCSC;f4`s z;fR8f;VE?A^f!N=$<54|!ae?BK)$uyuFZNvyQ^x7j1_5*K{%lgzfM-e2AbnhO>0QT zTG4ZgIcFTIIFkA)+qz&rCaA_i(7Pzf#+I;LURP>imR(=Es$!ho0TVRs)uFiEV%*Z< ztzY(CPtX;X&(_T*z}y6_h&4AmTd!pkf@D|@!2i&}%jU9j$ABrwOyzp-gNM(0yZITq zp~cuox*yUJJ_Wil&_rLrm3AfvgKh7!IM9{a9k?dzQWP4-5u4{cQ1Cod%DjtKzMSuL zADNRHrj$LE-;SiKM(kGFSZHTB`jS)>n{)%LQ$74AVWdKOEgt6(@6i-cQ0bvnRLX-t zi?6bH#qdLvew(Klg$Wn~yjPYG`ZrVMfO^jlUrcDVvF_z( zBKpcRMk|jVpq@`a6qDTH8ECVm8^`6ncn01*Pwx)snhr=tGUVH1&1`1c+x3`U$K^%8 zg#URgIl{T(Wk6h^D>kL*nQ%WREzlhhLkX#-`DK47jIkg57hB$*Wz55F}3ZBFa`4UYw zH=rr zoB0p8`^K?0%=tdm&`1>kbw<(2ytZozL^6m5s=<(JfxCC@QhoONWCBT zNQ&zWM}OKZabfnuQ5|a12yqXFfOA)9L`W~4ba089%^&_W5~Jt|o!(5tp3{z;wcs`a zulezSCh~(wSa)i7(I@6Q9>;(jB{-WO56AY^FD$xD+R9D3;P_2UiS`z=nRRwDMpFy^ z(@ck!x;`Aift9~)f4e1%Z{?9J)=FTg= zgmkVPdmFLzCFLj@$CW$zHnyf-S-(H&TKTEvNhcB8fI-G<)9O0VEO$3*E&1WuZE4=6 zo=&RC56(I#o}D`7T=*P)y^LN-s;ZsAijFMNC+6}9wS%g&gDSom5C3AP@z6|dxc_=G z@x%}o)1x}~j-xuCa_h0W3;NkceMC>>(!`p73}i$;HHcP4 z?ph?pD<6A7moZy0lhBpy<l{P2tZF)X>; zOx5pAPd+$i)^>3)UM;Slns}6;dpusRz5+LZ3bVEH)lKcjZV+U}-oyup$k`12!4y_= z_!vBI*>w*_?!-womS=>_;}x4;qYNlqsZ2T1{BwM#Uunmjs4#&x3R_OMuwO-ULUKws zsYgyQ7F%|vhK4F|ms+#i4<^-?At|5d-ta#Tm;eWu^P|xGxVKK=fkBuVHECN3B92hf zT*`_{CQjQT*pT8eI!-l5mrb>ny4GwkBP3Wu0lT?Pd+`k>m=b0agyc6i>y9CnCqXr} zL#8sTApQ+@qXwy3ezBg_~~I2zmGHBwM~xH-|G?5Mk$8c@j< zUGKdVMKO_@2OENXp0nwco|bRXbF8*=xN(Hv9C2Q|Nc~j7gyM`sQU`wIal9&k|(DA zOJ$3S;4g7_zH~SjPJpZ5mt9od>2R43kX4@MG&T!h3(X{zA$DDE=!Lo(u@r#_aTERZ zpb|T7>3K+E9a51D5EqovvzE*ppCW42E6gR3;*lBKTD2ay7uVuS^)E%1F>Z6{e`Vdo z9{ps2wy*c^atf0aM<*?{40UAce?_qO1kz@+r=>TZ2|ph0m>34uI|^m&N1u>ZLk#-Z zw5Jo_a&;AE>m1aK62s$!mVr>ecu*vj-6Lsw02_DnGc09Hs#IAwXB^V9v@7W=P>^+P z`eW7^E0JY%5{cJpWs^fsUsUieebj6iX~J%tFm1MDJO8<_A2pAs^W_^!`a4$Bp)+Gb zMuLcv{J1S;z2geyo07ia6%pU_Dgm~BL>Zf1K1oIsE%(L|$C?-9<eTGx@fz1*Frm) z|FWRjBrU~09XlP^;VkH~2-K$fR|lPJIbeypIJm6GEjRbH{)(Yh+1(YP2HRX&J?OzXJfxsvq%$#^wts&_f!As+`g53HO*~RP zK)|`S#cOEFDlXk~pkZPuvLGaRo}sjyTdzDwoHMf$y5*GNe(Pl4a#^f<^&B{X3lJob zfUUL)9;x8q5=rw_x6tOcALyzWACY~IurjlY#l=`~iIf_Ml*)dq^V&&RqT^PtKkPND zkPn|wSlQ9%;nz-!jk{CITIoAgNu)bH$sbxp^~+J_^0-jly)mO9mNpBo^i8K)n*{hN z6Cs|Y=+U>dqdK%6Y1*Ml)?UnB*U~7@M(--3yg`X;;Ow{pJ>0Il%1|<3^!%7(eYYDk zWjXd5R76crK3R1mC`>LH5^zYeLhSUZ?vL}fe<`019XuzO;rNVS3ij#HGB}KIoi8sA zMa~1TNnmO+l#{q-G5$P_(}_ci*=&W7)}KF*iTv5QM_wV%ZTl*}9|+svE9mdURGJfG z;O7|N=peKqC;L_n@pQ7>sCF+~Z4!sd_~pS0FS{UjLH(x#CT2F}B_kD~Orcng-A1Y2 z?CbO3we_zE?87w6gIry~E}eWMUYTEm%~8cKZW=m8NIr3G?VU4aXDoawTTLIEb4rOP zYP{d=mTh|sX1!3q*m}{HKf-=>P_RdaC$_Zcr2cU49ZuWVzNTd%s(vQ!IKN6qG*(2y z-Fxrha23mg4^-q@@V%Uv^5zx+-R?3iJ3c z4+Pn9N+;tW4S0qloD^tpJ8Ug^$JqW7t4vaWNO@+uetJ)qHL%o*_a6kUJ;~g-PbBF9 z>#Mxl@r6{w{oimnHxEa zxyX=O+#^On_eW&$-h7e<_EOmBspI3Gy{gXVaz!#j>N(cax2++9^piRg&h`{3ctw9< zNFH5!3}NGjUa!G)X-T2#V&<2(Y3=w#{qo9sG+|qHfE1bkt7nB|6prg;>7&)RoOLB_ z{6mEUVxdZ1t`6LMPsY~}CDt^07s`vf<7&jY_V%0&6tV(atRG&-tPq#*V#oor_$5YH zdc1Y3DJKV_PqsRElRT%kp@UJOQonX9iB!#w^vzUIk(#tnr%9;~p@`$LD7wPz zYT&>z-(C`9{UVv7DfVgZKSJ98HaE8iG~ct>k67kzK1FLuB0odBXis5Ga}7qnfo&jd zIMII(Nod@o7FcUl>*peT-EzqIdu)!0BL2Z9>>+UveGaIs*e4%4GksaYrs%b9n@-U? zjF_vJ!U%^){IlXul^Jm{UqL-uM6fCyol*Ii} z2rF%8r^tzkU`&7#%fa5Fu@{?p{Hd{5X#mRDu>&ozjh}K0F^(|<`vw;58#KPVr_}2L z_))kkl@lslBJHF-$L{NK8$XG80Li$<=_kXSjB&ZJOhk3c{-KsN>2&-Vb@5bLL24=a za`a_wYmRSmRq@ySyKn&+@ichBskZ$eF$LyytdCxIs#+TCBf>^+>0O8_Jrpm>=tiXJ zI2$Qg%n!J3f1Ow4VI=l?I8w}Bjxmm~)NN`313XE2&~Z*8_s&*)jIlUw*jP&|4@r*a zvhR#@sPxs#z8?^Eo9y$u$W?4%>P(7N`M$-vE@hTD`CrqH&72{dCFQibE0zUxmQFJ~ z_PT$4RUimqj!2pw3S`B@G6%jGM)-|;7oBKJtMKRfxEPL3&&`+&9tz?aSgI;1*ns;N zhK7qpX|)B$km;0D6;mVZYMF7{Fncf*9f~7;bP+V-l@i6@`na=#09!Tu!uE$UxS0bvwk%F5vAe@ifsiTudW9T#&|!g*oOFf~e`*<}p>o}{1@#ff z=GY_X;6o|0h-FlD$;Wh5x+p44N-z00C?$Nfl<@h`J62^X=*eHb`nP6{BG&$BWx@C0Uh**r z*f<=dt!+aKQA`O9hDx~6e!gI&(q>70lsvl^asZOJp`y}i_0ueH?x%9VC;Mwrt|Pk)?A zF%Bu?1&)=iOr+>45!>Nu`fVOKpVb-X>P!*efRmDxrLHQWmar2M!Ifhhc>gQy-NF4F zN3@@IkvlS5Rgjv7QFi&;N@@o8%V0<^suVU#0YEye_iWrX(KnlksZS*>vSViKX(J?e)9M5FGu!GhuJz%`}87d(p_`pU>?$4xJr`E=dK8WgP#s3Va?ib=d|JyH4i zBqcHM2*dv|wOS<~(U+5PBQJOy0+e!MI(F8N-Ob@I zJm-*P@4F^AE(y~BQngp~vHZOkKOI6pIEJ&qdgVLDJ!>&oUv4Y0n9bvRStNOH^vGHD zbc#^u_jFF6f&#i2fCdIAz6F>V#?p0pt~Wy7MH~9FS<_h5{Mw7^%hz=G_fq)?-bgvW zDJfw;J+bV|-cHB^@u|L@4VAUcN77m!nE3j<#}4J@@60tzNB4Vud4;Dr^-VcX)?AK( zCpnh%FVLxaLo-13)VrhZbkkJzlh6R$VC8zAi1Aimy4%1KgBnVzp;4|`(lahjy4W%o zi^B(ON-7Jdn{6Fm6Kh0-&`$4djhC_AlE1jd+?lUdX&R8f)H3@b;b^8F<$&vJFeEB^ zvUhVK07*uQS!w2|hP5)9>yEY4duU9gI7J27EWPw3bWHb82V{G%9Wc^#VZFiZiN`l9 z`qz^Dd1pR!VnX|p)$01Y&cFq9s5t{-<11FqFU$1$lJ~O;_Q@TRp7z{5ikQ8k$=zQA_D)>TF})9r70S4~YKTio}S44X) zxIYQravUOC$=Y!LA_0b!gudy;WT24T|8k_Hb@~ssCixi>!_PWx?`dD_sz+D~m!}!h zFL;imVDZYVFua!$7cj#ZVumGiDO!x@kM_7!v0dwtbC~m|WHVkerN}#Ei1`Se*Kr&o`&`?aL5^tw9Y_L zQ$5dBBZRc0FX`WPYQ_t3saT$nkd1(8Z%mlgPek5h-r?_5eU{{S@10Ov7WyiO6PWE$jrpL;0~GrH>tS6SaNSi1)3<||RR|q;cuog%JcXPj zr`O*9_9_gZ=>yXw^QC-g@An;lw+Nv3U!y`>*A_u%Fmm%Zj%^#@_P*Vu$)SFa5Adf< z0_4MGi8Wx*)a972(-cxiNxW8+F2)i&-rz~F?ZqL#tqdQ!Y~LJjpwcUIjxvl$CdgKZ zjz1>*7$?G{%@*jS&)Uf@4f2whQdZmHlY&~;RjN6Ypq_56- zZ4Qe{4`UWE<>pdMhJ?tAMElBunTx5vbLuHlr}7{4O_zUPEX@~%&#dlC-S}1$$IDa@ zXubGL{!Wr8V~}rK9X8nW6lJbvshAN`J>7C~f4U|jO{ zwV{Xa3DnwRcQM&qk&xU|seO=R8_#`&9CDeWD89~k^9CDh9i;3|^GRw;a z)N?;xdAZ5`PqgO;;cr7yFGi5hlm1PEioUC+f6&bNf+W|NZ2stl@Ti@}yC3TDzS)Dn zo77N*xSAj5Q2|*2b%AAPGGf>me?z83B*eyUe$Le+%7QwF* zB(PkTveT`PyU)CG-L42#-kx{YAf!;-H8tTzYspUnN4^0J`ykhTSr#26#O11XZO@IH z@=qKWN9tU<*^yr0GG1d<&R1>8D#JdU(hy^5nwohe6TBM1k~JX;{KDR1>29eN&$J~S z`+;P)jRXRF630~#hm>f%+2l%b1S{umz_pL-?^Q2-ucjS~WLknwu=v4Qi#KdL%gu0$ zt1}10BG?920hOs}uaN%1Nx%kDfM_*RT58^sK~5>(5XXrUy=Qvm`1~9aL&WE6TLMK` z1mQPGfg)T;tj$n9I3^KCgr`l1_GLaDhvC5Gh8n9lwz#=YmT?otDbOLlJ3Cb>P8)eJ>MUZL(n14Ft(1el?k;OAqo%lA!N1>e_}nre{yd|Y7D?v4 z#m*8vonPBX$4U;RqHs0uP*Wln_x}T0n=wOA(*vV!2A7S zvw{T+_zkV$^N~#;71>we;*nQ=ErD^5B@N$!!HxpH$3R(ogiPmSNR8YEm%bLhwkhg` zB82t!X4_>hGpnXw-uRRO{e1K+tb`wZ-bymW!*l#di$ocG>LrALr1VuB`*u#<6Eb(A zrCpo68ia%6Pnz`}`?T6+X~qwC(HXY&M1`|nc3Znau38>HQUzq|s~i)TUW}H;J9^n_ zskQt88sp^F&{be4jbT{UK1Suoo$zEN!OR4D+%4`c%hNFJ1MF<64M|W{Z6okd)&*Ob z(^a@ssS=x?VNx%=$TeBCm#a?>l(TQp4+^mQA;5cSQoum-tB!bW$3BPt{b>38ncA}S z&ST_9X1o!3aHTrH)~edOKdw`{$*@$ri-{0JMjrwvZk&94R&Lsknz~(|K+R$Dp$%sK zVjt);3^_aG(sQMeg$=h%BB>J_52!M;7N)Xu=BCRv-Ew$)`C^jN@dqd<9VK^vz=Q@^ zP--{lf4S`RxrK^B4q!)~!hFXf6FF_KKijU=W+N72Zfd$ZIOP;YahxvfW`InzKv?2f zvYkW6F<6j^AN&K9=%TI`|>lh^m_3iMbw7BHIil$?k(kO*yX z4<>A(4215UI+YAZ{ppz0umXdT9(kp@d_2MX__IG`E$XSC9FWGtderF&;63izZLD@7 zlI>IK6OU8lzv%61D#-tI;AH~`>=N%|T~$q<0OR7M7x7As9n%5h?J=;o{NPQ{lK5D# ztpe4~Cn3faExF)qJ+$&cZl-1tluv^w%=85X9nM4;uDo6R#;j2B`_pgu9Zquv7PS09 zYQ901osHiiQ~?GDT_4>>z)&wQXfQN#kDpb=RN{2i^ulH>$7Kn#$ZE)$&cBz&u68w5 zxjoNAu2k>bSc1|1-7cg_2-#lECWo(*&YuDg2Z7MVl_QpQsp)%oFGsb(_KlvuFB?2q zfHP37@UYDpQk`UiD0Nn}_<=J}vAkf5?4FD5)bdwOb~as}#q~-%`fff)?67qVN2?X> z!7u=eY|!Ru;?2~Hj)}8+Sd+0Y_mNJDwjJ3fwuG=h3{0o;g1IRq*nR#Z3iZ=|9`VzR z8Z5FAqwy6z5d5*m>q}a|Gyrp~k3P{nmb$)>qdF}nbYGK|ie>5y98g{N)|8$++I(dB z3Ho}g6=#xiJsxXgy!^*lolDgXq6(I;|Ic^++&S20Q@v3rbLiIdT+1KM-QE&=)-$7r zI%F8(>5KKdkei;Nq^DBP=2KZKhQb)5kX7bribKq}|-owHgU8EM3wF$VKT|3Sc|JITA`L?j|mzt;X7F-%; zCM4VqFbj=B>u5CvVSPz^!$`ULY!}fd969)EMP55vpiNV|9Ro%e^^{C=!v}-Apcf(L zh6>{eqFImE7yHUopyx(tA_Kv+mSIc?Eus!YyxLnpukAI(jUZE73HNKYHz<99Me`QZ z!8jz;8k9NqLV;Y9D!b8_SWwd2r2MU6dpz{mQLesg<2eR#M5Ha(h>EWfnDkUlt*Q_L zucUuZoRcz)<~VPIA7TymHLR?(57i!J5C6?Z|Asx-Q+N}=2QkU-&&%I+I>{3cPKC%M ziAZ`{#-URDB~{GVE88p7|l~8zhe;FxER*=@LBsCPf#|YLx9ih zR9cd57GN??+RC)k{#7X2a9*ZPA-Gh5tMf8r3SnLg97?Ni&eeE>1gAR)5ZJTJ$C!U3 z12-(e*gC6}b!2{!r9}<`E0!HzbL?6Z9 zCi(80JFtVpLw<>P2uhm{`jDqOUC29sBko6WfH5+rFzqi?y`H!RykKVk4jaVvDv-;r z&4Ye4pSx%lvJLD0T$U9p|21zF93d(}=t@C2uM$AN3CtOS!dhZy1cI&bnksNF9@6(k zo5Y=6FywX$PP+BI>K<-8)NGhteEq$!0AOMcRbR7kCZeLw0)IQYAV9zO6eKJE*$+aG z*os}kcruCCMB~Tv{?A28eS;8}MozTPK>beU_n<~3XEuP1K)MppGX){svLIa6ivTEc zimUGgRn9`3+rAv-zp=z4SU~37cQa4>&!PIiE4v8hp5`NKc6t3g2X8!P9oT85 z)vSDZFyLJXb?Ab{0H{0Rxcp}|2t}aeWwxaZt;QaXL|7THZT4^!bw(Y=UtqbbL|JpXU_PhVS zVf|Pk-!HbJv=yblmwi9h$N$eV$X3?a>Lq^Kq;0(qKT^X_F1Didli>XLXe&xTLh+M} zttkB@I6pqxiqelz{N!RQN Date: Wed, 14 Jul 2021 21:00:11 +0800 Subject: [PATCH 6/8] Fix static check --- airflow/models/dagrun.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index c8e1100c0835d..1fdd26cd0a611 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -477,9 +477,7 @@ def task_instance_scheduling_decisions(self, session: Session = None) -> TISched schedulable_tis: List[TI] = [] changed_tis = False - tis = list( - self.get_task_instances(session=session, state=State.task_states) - ) + tis = list(self.get_task_instances(session=session, state=State.task_states)) self.log.debug("number of tis tasks for %s: %s task(s)", self, len(tis)) for ti in tis: try: From 7868928ee513248a85cf0cc59da248f7221c721e Mon Sep 17 00:00:00 2001 From: Qian Yu Date: Thu, 29 Jul 2021 18:31:05 +0800 Subject: [PATCH 7/8] Change color to violet --- airflow/utils/state.py | 2 +- airflow/www/static/css/graph.css | 2 +- airflow/www/static/css/tree.css | 2 +- tests/www/views/test_views_base.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/utils/state.py b/airflow/utils/state.py index c95c7d4f0ffac..f408c94b0d3df 100644 --- a/airflow/utils/state.py +++ b/airflow/utils/state.py @@ -107,7 +107,7 @@ class State: TaskInstanceState.RUNNING: 'lime', TaskInstanceState.SUCCESS: 'green', TaskInstanceState.SHUTDOWN: 'blue', - TaskInstanceState.RESTARTING: 'goldenrod', + TaskInstanceState.RESTARTING: 'violet', TaskInstanceState.FAILED: 'red', TaskInstanceState.UP_FOR_RETRY: 'gold', TaskInstanceState.UP_FOR_RESCHEDULE: 'turquoise', diff --git a/airflow/www/static/css/graph.css b/airflow/www/static/css/graph.css index d1655b0129081..ce76df7dfcc7b 100644 --- a/airflow/www/static/css/graph.css +++ b/airflow/www/static/css/graph.css @@ -149,7 +149,7 @@ g.node.shutdown rect { } g.node.restarting rect { - stroke: goldenrod; + stroke: violet; } g.node.upstream_failed rect { diff --git a/airflow/www/static/css/tree.css b/airflow/www/static/css/tree.css index bf9a640023e4f..c17cf0a6eb65b 100644 --- a/airflow/www/static/css/tree.css +++ b/airflow/www/static/css/tree.css @@ -68,7 +68,7 @@ rect.shutdown { } rect.restarting { - fill: goldenrod; + fill: violet; } rect.upstream_failed { diff --git a/tests/www/views/test_views_base.py b/tests/www/views/test_views_base.py index 8f0f4f6a62ad9..537f661e588c5 100644 --- a/tests/www/views/test_views_base.py +++ b/tests/www/views/test_views_base.py @@ -58,7 +58,7 @@ def test_home(capture_templates, admin_client): val_state_color_mapping = ( 'const STATE_COLOR = {"failed": "red", ' '"null": "lightblue", "queued": "gray", ' - '"removed": "lightgrey", "restarting": "goldenrod", "running": "lime", ' + '"removed": "lightgrey", "restarting": "violet", "running": "lime", ' '"scheduled": "tan", "sensing": "lightseagreen", ' '"shutdown": "blue", "skipped": "pink", ' '"success": "green", "up_for_reschedule": "turquoise", ' From 64b3f40d089caaba354e0e8041565c4d015f3a67 Mon Sep 17 00:00:00 2001 From: yuqian90 Date: Sat, 31 Jul 2021 11:08:48 +0800 Subject: [PATCH 8/8] Update UPDATING.md Co-authored-by: Kaxil Naik --- UPDATING.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/UPDATING.md b/UPDATING.md index 09726e6f3ec35..b18dff578e76c 100644 --- a/UPDATING.md +++ b/UPDATING.md @@ -133,9 +133,9 @@ If you are using DAGs Details API endpoint, use `max_active_tasks` instead of `c When marking a task success/failed in Graph View, its downstream tasks that are in failed/upstream_failed state are automatically cleared. -### Clearing a running task sets its state to RESTARTING +### Clearing a running task sets its state to `RESTARTING` -Previously, clearing a running task sets its state to SHUTDOWN. The task gets killed and goes into FAILED state. After [#16681](https://github.com/apache/airflow/pull/16681), clearing a running task sets its state to RESTARTING. The task is eligible for retry without going into FAILED state. +Previously, clearing a running task sets its state to `SHUTDOWN`. The task gets killed and goes into `FAILED` state. After [#16681](https://github.com/apache/airflow/pull/16681), clearing a running task sets its state to `RESTARTING`. The task is eligible for retry without going into `FAILED` state. ### Remove `TaskInstance.log_filepath` attribute