From 426671e91adc7bfafe9918bd28e88d374f77d68f Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Thu, 20 Apr 2023 10:21:19 +0800 Subject: [PATCH 1/9] planner, sessionctx: enable late materialization in disaggregated TiFlash (#43068) close pingcap/tidb#40601 --- executor/tiflashtest/tiflash_test.go | 6 +++--- planner/core/casetest/testdata/plan_suite_out.json | 12 ++++++------ planner/core/explain.go | 3 +-- planner/core/optimizer.go | 2 +- sessionctx/variable/session.go | 4 ---- 5 files changed, 11 insertions(+), 16 deletions(-) diff --git a/executor/tiflashtest/tiflash_test.go b/executor/tiflashtest/tiflash_test.go index c6384b0b99c70..57ce4e31eabf3 100644 --- a/executor/tiflashtest/tiflash_test.go +++ b/executor/tiflashtest/tiflash_test.go @@ -1424,15 +1424,15 @@ func TestDisaggregatedTiFlashQuery(t *testing.T) { "├─TableReader_15 3323.33 root MppVersion: 1, data:ExchangeSender_14", "│ └─ExchangeSender_14 3323.33 mpp[tiflash] ExchangeType: PassThrough", "│ └─Selection_13 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", - "│ └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, partition:p0 keep order:false, stats:pseudo", + "│ └─TableFullScan_12 10000.00 mpp[tiflash] table:t1, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", "├─TableReader_19 3323.33 root MppVersion: 1, data:ExchangeSender_18", "│ └─ExchangeSender_18 3323.33 mpp[tiflash] ExchangeType: PassThrough", "│ └─Selection_17 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", - "│ └─TableFullScan_16 10000.00 mpp[tiflash] table:t1, partition:p1 keep order:false, stats:pseudo", + "│ └─TableFullScan_16 10000.00 mpp[tiflash] table:t1, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo", "└─TableReader_23 3323.33 root MppVersion: 1, data:ExchangeSender_22", " └─ExchangeSender_22 3323.33 mpp[tiflash] ExchangeType: PassThrough", " └─Selection_21 3323.33 mpp[tiflash] lt(test.t1.c1, 2)", - " └─TableFullScan_20 10000.00 mpp[tiflash] table:t1, partition:p2 keep order:false, stats:pseudo")) + " └─TableFullScan_20 10000.00 mpp[tiflash] table:t1, partition:p2 pushed down filter:empty, keep order:false, stats:pseudo")) } func TestMPPMemoryTracker(t *testing.T) { diff --git a/planner/core/casetest/testdata/plan_suite_out.json b/planner/core/casetest/testdata/plan_suite_out.json index a089849913219..0aa44d94f545b 100644 --- a/planner/core/casetest/testdata/plan_suite_out.json +++ b/planner/core/casetest/testdata/plan_suite_out.json @@ -8311,7 +8311,7 @@ " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#18, funcs:max(test.tbl_15.col_92)->Column#20", " │ └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p0 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", " ├─TableReader 1775.11 root MppVersion: 1, data:ExchangeSender", " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: PassThrough", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:min(Column#30)->Column#15, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91, funcs:max(Column#32)->Column#16, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91", @@ -8319,7 +8319,7 @@ " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#30, funcs:max(test.tbl_15.col_92)->Column#32", " │ └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p1 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo", " ├─TableReader 1775.11 root MppVersion: 1, data:ExchangeSender", " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: PassThrough", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:min(Column#42)->Column#15, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91, funcs:max(Column#44)->Column#16, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91", @@ -8327,7 +8327,7 @@ " │ └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " │ └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#42, funcs:max(test.tbl_15.col_92)->Column#44", " │ └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p2 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p2 pushed down filter:empty, keep order:false, stats:pseudo", " └─TableReader 1775.11 root MppVersion: 1, data:ExchangeSender", " └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: PassThrough", " └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:min(Column#54)->Column#15, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91, funcs:max(Column#56)->Column#16, funcs:firstrow(test.tbl_15.col_96)->test.tbl_15.col_96, funcs:firstrow(test.tbl_15.col_91)->test.tbl_15.col_91", @@ -8335,7 +8335,7 @@ " └─ExchangeSender 1775.11 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.tbl_15.col_96, collate: binary], [name: test.tbl_15.col_91, collate: binary]", " └─HashAgg 1775.11 mpp[tiflash] group by:test.tbl_15.col_91, test.tbl_15.col_96, funcs:min(test.tbl_15.col_92)->Column#54, funcs:max(test.tbl_15.col_92)->Column#56", " └─Selection 2218.89 mpp[tiflash] gt(test.tbl_15.col_93, 7623.679908049186), ne(test.tbl_15.col_94, 2033-01-09 00:00:00.000000)", - " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p3 keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_15, partition:p3 pushed down filter:empty, keep order:false, stats:pseudo" ], "Warning": null }, @@ -8379,14 +8379,14 @@ " │ └─TableReader 39.97 root MppVersion: 1, data:ExchangeSender", " │ └─ExchangeSender 39.97 mpp[tiflash] ExchangeType: PassThrough", " │ └─Selection 39.97 mpp[tiflash] or(eq(test.tbl_16.col_100, 10672141), in(test.tbl_16.col_104, \"yfEG1t!*b\", \"C1*bqx_qyO\", \"vQ^yUpKHr&j#~\"))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p0 keep order:false, stats:pseudo", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p0 pushed down filter:empty, keep order:false, stats:pseudo", " └─StreamAgg 31.98 root group by:Column#26, funcs:count(Column#23)->Column#11, funcs:sum(Column#24)->Column#12, funcs:firstrow(Column#25)->test.tbl_16.col_100", " └─Projection 39.97 root test.tbl_16.col_100, cast(test.tbl_16.col_100, decimal(8,0) UNSIGNED BINARY)->Column#24, test.tbl_16.col_100, test.tbl_16.col_100", " └─Sort 39.97 root test.tbl_16.col_100", " └─TableReader 39.97 root MppVersion: 1, data:ExchangeSender", " └─ExchangeSender 39.97 mpp[tiflash] ExchangeType: PassThrough", " └─Selection 39.97 mpp[tiflash] or(eq(test.tbl_16.col_100, 10672141), in(test.tbl_16.col_104, \"yfEG1t!*b\", \"C1*bqx_qyO\", \"vQ^yUpKHr&j#~\"))", - " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p1 keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 mpp[tiflash] table:tbl_16, partition:p1 pushed down filter:empty, keep order:false, stats:pseudo" ], "Warning": null } diff --git a/planner/core/explain.go b/planner/core/explain.go index a23b98909f4a3..916947e916895 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -20,7 +20,6 @@ import ( "strconv" "strings" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" @@ -214,7 +213,7 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { } } } - if !config.GetGlobalConfig().DisaggregatedTiFlash && p.ctx.GetSessionVars().EnableLateMaterialization && len(p.filterCondition) > 0 && p.StoreType == kv.TiFlash { + if p.ctx.GetSessionVars().EnableLateMaterialization && len(p.filterCondition) > 0 && p.StoreType == kv.TiFlash { buffer.WriteString("pushed down filter:") if len(p.lateMaterializationFilterCondition) > 0 { if normalized { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d5520e99d5a39..39ea9532ef724 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -562,7 +562,7 @@ func prunePhysicalColumnsInternal(sctx sessionctx.Context, plan PhysicalPlan) er // - Filter conditions that apply to the same column are either pushed down or not pushed down at all. func tryEnableLateMaterialization(sctx sessionctx.Context, plan PhysicalPlan) { // check if EnableLateMaterialization is set - if sctx.GetSessionVars().EnableLateMaterialization && !config.GetGlobalConfig().DisaggregatedTiFlash && !sctx.GetSessionVars().TiFlashFastScan { + if sctx.GetSessionVars().EnableLateMaterialization && !sctx.GetSessionVars().TiFlashFastScan { predicatePushDownToTableScan(sctx, plan) } if sctx.GetSessionVars().EnableLateMaterialization && sctx.GetSessionVars().TiFlashFastScan { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 68cc917eb9180..05c2bce8ef43a 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1870,10 +1870,6 @@ func NewSessionVars(hctx HookContext) *SessionVars { EnableLateMaterialization: DefTiDBOptEnableLateMaterialization, TiFlashComputeDispatchPolicy: tiflashcompute.DispatchPolicyConsistentHash, } - // Always disable late materialization for disaggregated TiFlash until it is supported. - if config.GetGlobalConfig().DisaggregatedTiFlash { - vars.EnableLateMaterialization = false - } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ indexLookupConcurrency: DefIndexLookupConcurrency, From 3c828717d1dfba1bb81437e791d5f7e4c2a86b54 Mon Sep 17 00:00:00 2001 From: dhysum <130652266+dhysum@users.noreply.github.com> Date: Thu, 20 Apr 2023 10:39:19 +0800 Subject: [PATCH 2/9] ddl: design doc for `admin pause/resume ddl jobs` (#43159) ref pingcap/tidb#18015, close pingcap/tidb#40041 --- docs/design/2023-04-15-ddl-pause-resume.md | 106 +++++++++++++++++++++ docs/design/imgs/ddl_job_state_machine.png | Bin 0 -> 483247 bytes 2 files changed, 106 insertions(+) create mode 100644 docs/design/2023-04-15-ddl-pause-resume.md create mode 100644 docs/design/imgs/ddl_job_state_machine.png diff --git a/docs/design/2023-04-15-ddl-pause-resume.md b/docs/design/2023-04-15-ddl-pause-resume.md new file mode 100644 index 0000000000000..d7f43e376431d --- /dev/null +++ b/docs/design/2023-04-15-ddl-pause-resume.md @@ -0,0 +1,106 @@ +# Pause/Resume DDL jobs + +- Author: [dhysum](https://github.com/dhysum) +- Tracking Issue: + - https://github.com/pingcap/tidb/issues/40041 + - https://github.com/pingcap/tidb/issues/18015 + +## Background and Benefits + +DDL jobs are running asynchronously in the background, which may take a very +long time to execute. The Admin could cancel the running DDL job if +necessary, for example, out of resource. + +As a DDL job may take a lot of effort, it could be a big waste to +just cancel and restart the job. It has several stages and steps back in the +yard to finish a DDL job, we may just pause it at some step and resume it right +from that place. + +Also, such feature will benefit the Upgrade of the TiDB Cluster. + +## Goal + +Add two commands to pause and resume a long-running DDL job. In particularly: + +1. `admin pause ddl jobs 3,5;` + + The jobs (here are job 3 and job 5) should be in Running or Queueing(Wait) + state. Other states should be rejected. + +2. `admin resume ddl jobs 3,5;` + + Only Paused jobs could be resumed. Other states should be rejected. + +## Architecture + +There is no change on the architecture, just following the current one. + +## Detail Design + +The whole design is simple: + +1. Add pause/resume in parser +2. Add builder in planner +3. Add builder in executor +4. Valid the job's state, and turn it to be Pausing, if `admin pause ...` +by end user + + 4.1 Valid the state first, only Running job could be paused, just like +`admin cancel ...` + + 4.2 Turn the state to be Pausing + + 4.3 Background worker will check the state, and turn the state to be Paused + , then just return and stay in current stage; And this is what is different + from `admin cancel ...` + + 4.4 Background worker will continue fetching the job and check the state, + and keep doing nothing until the state changed + + 4.5 Specially, the Reorg state could take a long time (maybe minutes, even + hours), we also need to check the state of the job, and stop the reorg + accordingly + +5. Validate the job's state, and turn it to be Queueing, if `admin resume ...` +by end user + + 5.1 After the job's state changes, the background worker should check the + job's state and continue the work + + 5.2 No other actions should be taken + +### State Machine + +![state-machine](./imgs/ddl_job_state_machine.png) + +## Usage + +1. Create an index + + `create index idx_usr on t_user(name);` + +2. In another session, show the DDL jobs + + `admin show ddl jobs 15;` + + It will show 15 DDL jobs, change the number if you want to see more. And + then, find the running one (job_id) you want to pause. + +3. Pause the job by + + `admin pause ddl jobs $job_id` ---- $job_id is what you get from upper step + +4. Resume the paused job + + Also, you can find the Paused jobs by (change the number if you want to see + more): + + `admin show ddl jobs 15;` + + Then, reumse it by: + + `admin resume ddl jobs $job_id` + +## Future Work + +None. diff --git a/docs/design/imgs/ddl_job_state_machine.png b/docs/design/imgs/ddl_job_state_machine.png new file mode 100644 index 0000000000000000000000000000000000000000..4b155ad0d020efd03fd9f49307108d43ebebca29 GIT binary patch literal 483247 zcmdRWcUY5Ix38j(1r-Z|3W$m{r6xc?dP(TL29zE`3xNa@652$^GAJN|AW{Vd73oc+ zI*jz9qV%S8DUsgc?f@2azBA|C`tyO% z*;(NnewdHOJOPEnI{-U0YQ$ciN zEQienP*f*cxH%&EB}kf3ye?T+&sG6QkTx#N$X&xmOj87|BO;7*#Y(8j8>)&MI4XZ#;NuvH|g>LcBqgguRv)im9SMk;zv4hRhrR2u?O#W+KS z6@;yjLQtZzI>Hd(10iPZscDRHLR)D=#T~(WqNcN$st^jIV}}5Q8IyEj4rDtuEfN|- zCJE_*TS`PF2~mArEDmP{^N{c);gk(L5C%GS2F|2)962DAZ4oX+l#Lh`?&7Rw?Etkg zfT}ygoJhiI`lgO5U_B5hAybT!1i=Zd3f{n+mEbV42w9A*Ve0DcfDs{Bshg@A69IJ! z5LF3BVKqeqC|2Ix+MWpR7^z{LTt)R9HH_^%#T@MrnmT9*+DX_>$wS21-ND#HNkYf~ zC2R%8B_W{@Ws;|^kp$8R3YUk#v8oC{7Zgm1S|-Zc#-f^52%Mq>24ig?s-dHbfkVVR zaiY2q5)7^`Z$(5}*~yd8>iW*QPNs^Q?#5t1MYO3R)EbEqvR4qb(kAE%lQ2-MiLs5m z7(z%342+Y|Fws_ZGKSzWNO|y{q^zo8rvP*vVE`2|HYHk1Sm8{GSR_GR9ILGhvv!B5 z5v&YV2uNjlcYQL#RSe-SuZ8rKSHNqc6$#=Vh6G0q4Va_2sg*F+-oP29rzVUv_Ox*! z;H^*)sF*lz9XpyxYj6?X@y6ILEB zqHd)rVydU(>}KHMplB%MNknU@K{OSG#gJ<9?s`N`53(!C&|Ov4LEO#?Zba5IMR)*4 zuy{8p!pP7FuIA>3hC*>PW+j|$oW#lC!a%~=87m~_B%-Q{f!OJ5V&y$l6kXxkrdSDA zS5Gwp+8U>0gLP2yP{wNNy2A7z>V_&LF?XV=lCU#MUPnaLKve;c(ls?ixM(OVIx1@F z7?IGrcyYMAs)(D48^%P(K^fyFV&{Y*i6RtC9OaF0jw*^oO@bQ^K{OKa(8H;kNI=MX zXmwGdi<7&XA<`L`7@RxI*4bFY$jwv7!wFESh!w`8+>HqsV|O)oD9IjAf;c#1h$@CU zc#JqvALS@24<#sw5^UTwU98C{1A+<4-pWV}fmSh6kQde_+nLB48tIAZT1(iFp+q>r zT3-xi;-CVAt_C$iD+}w0xseUDZADy##MD*rP&YBWh$+F*5PWWovm(LNXb+-96rDUo zMXX#T4O~F9584=J0mw7JXPg!HbfN%S784YoNSbg z92_+eM3R-3f)dtU4~jrp<5V!fVOf>G4Ns|yKPi>bnm zv>inh^r?3hJpb}|f>nX2-Hztt8G!;zIcx6Rix+TBn@QOKA~K;SSWdtD(C7LGQyvbRwrYKR#c*r>ow?1V+^&^jvG z;?AlT{%E8vd zT1`_#l;+EHASQZRc5X_}x_ZV?G!d;~0K9{O4Gr9iiee zE8AIPU{>Pp;@Ya3PA;xW?y8=4PTHo%)GpNW%bbFANb95puD2Hl$e&A*{HSs4S!N$1m9I6O7M0aNIA38gZr=!%3GK9zPcJLFCxnveq{H$MeMrF6E zNP={N!(c7Gscodisd!eVdoV|e+C?2bE3{AVuxmkjf8sn5(LjQ?LhJ~(j6Fcz*( z_l=VEi@0Vwc$(K|6&h)@8$zMG7{$*&e@>QR7`Pt8l4N-A=>HaBCtU!ax5Kv5^SfLc zAYk@^mhvW{rFWQlmh7*&3>n^Fb%9O)ZMNb(Pkr*gARn6W%f>E=+Me{gh z@g&ZTC0mCTAnM7vxYCRxA&9T^Z9g{uu`cgwe#U!~bh?TzO#7=B1#YIDQTYEQfYAh4 z=b3u)bwI$`3S-}sy8`n@FSr|wn)s~cocl`Th7w!`qI4JksuGIImVU63<+Zjqt?U=h zY<+T*nQxp@K61Fm$(|og2`n4ryBWHP+Kull>Xz=9TDj?DzX=^VIAwhG+2E_?V>SMb zPKd7%|Bn+CD#T3;l$W!xCVO`@r^P=y@NL%hiy;L$NRC6?%(=MoZn8MOhL)IU$K07FORdk*jK@CefT@ca#MRn z_OY?Vo_`$}Q|DZ~l4HGvyW{7!mnbnDDRkHvm14!K^Z3PE7M2_RZx4xlBkG?oM1tjr zb8CXCnJ@md&M??A>}jDVbo*F$-|Oz%psQ5qu>JC}ib#e-$$ynw;g0{I`klh9qT5td z{xfepnR;YP>fRQug|(QsLemrDOjHnzZ`^z^>M%oh_kYIOR2w?H z#4YSRc`X3g;&l(zHPGViAnmx#dk!Xz$#J1J;%&X3JxE9M%7A=sksc zaUBd5R4xZb*~Kle-1ZCI47*;Ey%JdV`P_8o*#gpE)D#4?`*#(-c02C!^3ImE>p=lH zYte14Z`^~@RTnNkt#_VY2@G4{xs{U5k9sV37QUXaoSx?V(WFxnj{)&!o1s5$$e?r^ zeMi)z2__oSXfcs8`@hjD_j~E9J)XCEUkwd!m6zrPV6q(Y>i^pGxZ4@+8O6l*v93*3 ztf?R9;kxhtqk;QzVZKTv`$g$$*QcQ%!MyoI`k)`5F1uW+xKOsT^JeVE-6v<$OY&f1 zT^%hi{_Od0Ky8Sg!e0D(f;e}eX`c8Ki*Mcmk8Vcx+O>(S>EPc>?zAZ1xxNLL|1NME zcw`xMdWQJEuE$Gelh8*KzX9167QKD?e=*!{-ni0<)7w2ZUB9V$oNmv&r0U6T+2v)I zyFoeLeRTh-zrPB4g$FDYyW0WH4Y+4Vl``C7W7!oLkSG2VsQ-IR7XhYQ)WtjhNyon~ z?N^cfr)MZ#zygjIMaTR~?tlM|oFHvmc0>#I?->3OTw5oIn*(t0!X|P5_n6=M<*x?h zjXKcWqCXk~{_{LvAN^4~w@x`!1Okb!_ThgEDgT*-&2gvofYTQI=+5z<=lSQU71VL{4^J$FhG0^Vkx)!Dp7^#ltOz+OlWYa=hf10lOcyu~g?M4-=n> z(BmR${hGr$RUFleec2zl?x7@}#^CgH1-TgET%it^UeBBA2C5?iFX-RCa90=w5#vR0 z_Ya*5a5B)ltx+AUiWn_uyPJIvg}mS&KT$Ar!H=DvB{rVtv+Bj!%U&U)n*P$N`P^0G z=f+gu#)OW+GqW@Z$9Rd4cnI%ntd)atUct^6YMd`P_2-)U^TTzWRh4~ey5e*5TgcYJ2ltXYQQ*IAT7<^cJ%a{_jF4IN&Wc^iCucPw(OTE zYkWP-m(f>#)JJ|U*DjLm#Z)mu?_qcmh8(>3ai;G}d`wyO*&83KSxuQFM5@lHCXN*( z<{p>7=PJh4nocD@<(aZBs=Nh{m_hN6I91W390fyj0zGD;^>eI%aK%R*GihXcL z{5S=jR`mkIfmc$F7=}yV2uB^%4leWWODE&(gTto^;E)t=i3&+RaFzKGTrKHE6q=}; z^M%atW)<^HDj;j%3VaDU_lYaub>`kf_Gc)hsR6djVVH{0oKN@3NLKwvQWxQYYDH4dcpR0(c!M+` z`PURqoqy;opeJP^gQN+ga}3iIOJgIPaRPB5WP=672_XE1-9{F0nBJKW3oT>_sE@BN3aCC zGNYd5TLKrwysD4h$`qG*%5L%Azqf`g!2uWMsB%+9>^F;g$Ufh7PCJpWnJLE_gA|Na zanuflFDN0Hf;!zsQnU}O9D06d%WNB7i_*1EYVs*-0UE2N!C#S7JHdaMmBsRxWfVZ7`q$NI~7+fE5w^uJ;J++sUg_s#ym$M}C_N^L z#|gE(aE>&yzDJ>x1^m8;+VuP0hLW;1PGK%gy6O&m3@jpfHr)WUe~J{8qPxo1(|S$5 zYg^-%J-*x;&3F6RKsPN&9d8epJbpaV@aS;}T^73Gl&Do(HWFTR-!Sn$@=+8r?6rd` z$@Ff9qTRzKCJ3S9kJ~J=XKYxSUrd~Ux+s+2W}Bln4++~ezCef;D+g6Q(M^EjrtjZo z6+O?P99Une%K^3i`^oL&5)cdr;!#SjJ7t$NEZ;%+DoaUU7azr<#Vag z4&B7DwtS8?-?5gkXvtjk;4G#!qiCX`{rX8i8)?aqPp{&6*!lFI^XjGYI`x0Xu!uc3 z&vE=P{_usR)7f9$?3-tF4BmVR2-Zj>8(V}SV+8UXG(-efkaSN`sKks#;tYm^D^Uz* z-GFsvnC{GfzzT}7rpk_SH322{_=7P81f8SRA=t+RqTd`+T1x73p*4_8wCmJxk z{)biH&Zlz1`QazK13Jbpu*1j_C1!~+p);YV-U1F(J`7dF!8`Zpd5QN2$&z+`!sz&N zDsfb;H6wYdpqPmM%qBVOgVpxI_QvKV6{F2A&bp}kG&H%7cD5~0702{}L7uYgJs%>9 ziV($MkVSJ(pUbS+1b4GJve-%Qe4t&LMPHYPHqzJ<%ShA0K}Dgu&24}cM-SJl^j1g8 z5phpocbMu%E4!dIjOa(T!#VL%MY``jzi`aC{Bf8AVfQZ7AwnGG$feaq)O_$@ZwMFq zuSdKQLTY4uJ@G)^?9QKC zocRVb*>_5>Hroz#z{nUkvS?N!|6}%?r-FQunI5U$qtAKJDiX>wiNRqS2w_^ zy4VTVhFJabpatnI?kt0nW(%;ckdCw{CcsiPmr?^-3%qPDw!v(v;|B!&@=D$#FG<{t z6%FWcDT>*xSF^%#5!dZt4EH4-g8lt|=K~!eSj`s>VLu5LWJ_A0f+?GV{YVq+msY3X zBTx8Tn^55a?0Kj`36ySBNTyT)AI|6<=|Bt12^P4&K7&Nz)B{|lRRko(jqO2IfYawQ z(Z&g`Z0>3YvM8U#WaLl-!|WqeutO4;%kd|{k{A% z3%x2HWe~f)dIQ>byBHbfP5cIKfYqx?9=PuPo?($Fboxi>kb@$g$*&RD8=8c)rX-Uah}kU_49&ThK^7~iRjE0wIc z`Qnju4j?yWThl>N=>$a3FqQLoNjP0WI}ug3=+THgaun@*RCc|7(v1kK!}9 zZ8FJveW~wj*hArNgY$1OL;>?c&-QJ#PW%30SE7jMp=jGFU~KEvFQyrDO2FW2gG3fA zl2vs4$!%7mJ0cL{%^neKV^NH_Eqyj8cl(kVNzqR50qV?5V03H2)WT@%LPt$=VEJ5D zYgTW>XxD`EMFMtoO{e;~8qWAG`?>tAVs2v0>PO(7N(xWx+3&HvL16z3Tc*qwV#Mvi zc4fDw{kDy}ad6;%?B50)W1^()9H>RQnj;wplX&U{^eus+AoW;_vroeC_W(Y7W@1_E-@P7XAn0Go4fF(~MfIv~`K7)$BbF<%W0DU}427 z{qc|64_p-x+Tt<^C!6O_IIxI5PE&-KhtRu^c7=AfH`~qSa=$Ac`*XBw*0U{7x@%#! zLS`+rHR9}rkmo#ik9plahBiM8g0CB5rzvMRfj2*s!^X5VfwDr=7+HgW&24A)6%bja zq~EeWovz(;Wf#2`|HaHWjREf0%jdEdkDw>tE8^a}aTj~Dc_{=+jsL+)s+~9?o!H>S zwS;Recci1tUY4BdXw>e@=Voun(w2@v$9g}XYAc+0Tx`Db`Ozh#b2Zhud__LP?`|xT z>3X-MMK=_AD4_JS6W&o8`NqiBS31IC+Rc~v_mCo#{I2SR^u(jvF|K-pb5H){PHf^T zX^&iaH@L?9gg9C|o)YxfXC%R)dv>U%+w4?T*DUi==u-`&=DR31RM>~RkF!#5FT2F< zW4T=#NAF3ndXra5wu>_Lc7zb(bPvyz!-XM1-h1rjT z!{^kiA?nqp6?dtEW&}U>K+I2BLRFTPysy0g>p{ikea~NBM88MaS}ovQ^0cQ461b!8 zzYWocmI4Ll7)K1qS2Tl6Bah4g%}#zri*W;BQ3eT#ks;86kNneOKt2qD-?B!SoZ0X5Awis;M8k`P*i6mCWKbm?e?i1S0COz@@?Fgw0 zHe7YpLsdzr+P;Xc>Y>5|NJ@G8DI&%^WVVjMt~u;OYYxeyEpHb2wl_=FMKx0##pG4i zdq?^=CPdr?VD`jWwiuc>leDy-@2y>%Z3WYMK8b(;2-Z{_^>*uN5CZH)D9tIcC z#Jzh}FFA9k4B8utlvwSHh;V!NyjGlK2SQb1df8`fOPyVWv5w%JpyyR>%Ep6#md}+X z)4b@SwPp|#%0-PImh-qb5mgg5Qx)fvA!M_~_N+6%A4Bm<^TpCjAk~8s*&qmv(mQuw#&sO#o%sfd&3>S&d7Qjtn<~kjo}-@x2L6=Qwj^Lozy7C4VTlO z7$p~pp%&WC;SY-4?A4BlKB@(_*ijcHxuYPGH&}i9ULAZYOG!NBISjntd|C43ykfuz zMQY;9vtpCEYR1>Zk(oteNs0CFr{d#pj&BkCI+#;9(idOyoeybdf}2N9_DmJnBt-=a z>o?T4@S^QT-$Nm|QQefQoJZ(7_W&2?OmHcbm4tPMUuL1!)N9VM7a9lOLq$-Ts)*>z zz?@xW7X6wf{~n6UG0^n)kOqFksadvw8Yrun8ULGy=Y`Pq@ix__D>JI*v7zCd5gqM0 zfhVQ8xa%%n;EpEPX+0nzw4}yvgDuC)ys}~CxW_wZM_n3h7`kGWn>%8a*$V}OQj1|A z3}Dl^9U1be{9L!d=NxR>rTqEft0y!pw)j@s29Gju)N}qNYh!R>@n|h|Aa1fCCpY7S z z_Sx-df(u|4uTp_=+j;##X?=Ml8?56V8;kE9HkNTUAL^{RUq+^}}k6ceap!bwm5J#CZ5Bde^;>VqRSHOcBAlOZ(?+~+l{E^g)jstEG%lpO%N*B1Un-DOSOUF#j%-P z;MoYNdwneM2yE@k%;3R$-D3rpAxE~DU7(~2FU3@+%hGecK*=zXpgJaPN+Wcf5owVkuo)y_j#ktN1(Yck= zPVC6evcs!I3$vNE&HnN8ZIz}K%`OFFxhX|#Dze|F+LSYea@Yi~$Q6b{o6jjvO%l^4 z3v3djVkT}x#k$>y=|*;#Fk*Hb{T;byKc12~I>d+TnE1rjy84c-HD{R8cz ziY*qKx(e1W<&p2N>!;f@bm#|z^oB||HBc96q}|^B89vbzUu#B&e(ZaC!>*|{t-dy^ zz5%>y9`QDkp|@#^m{A(0$&xwHC7A_YnTs)0^i(;sm#8ALFF7Z3S&g5~!o1jf;kx;O z_@h5d_;KlpZ<`;^Cwtw5x(o#G-jtpHZ?V3H% zI`OI68RP;wbBdW0+>2`_l4trZJngSZvN*NS*~R-A{@vDW7}4Xoz@^+6Yoh<&H6ypS zkIy9C^7D9LTYJc4#ofZEu&ij2soGkZ!7H^F2d}XDO+N41{}9ykcJO`qn|%mP4@QQUv(TDBjxW*$z%II&l}NQW3pN0=GVSsTRjkXj&|Ks-`8dm0X0o72|NX6> z{jkt;(3x{Ed5nqgnzES}t1D$0JAX|f9Odc4xjUTj|&Bqr$ z&bZukW*3X%z8dMV=h8n<^?T{+E~Cq_dBNS($9GjWV#zw=FTlBAp7XE1cR1cv}dxnq_k;MWWNAjoT=OL<9CBK zBuJ5`U@zlnw4nkEzA~EnG49d`hqunN}>s9Ya89zXY{zxm$arW_woh=a%yN zyChv|rnF_>Ha=+z=UvNh3iG>^4Euxq!+IQgXK$lm z{U5{hP{QZdvS_{Ko!1|i0+`vhzvOJq9S_UKdKD`F2G1;xS;m|3uX(oZ<|t|BkPi4| z>yrWso*ANZh6dlQ#v;6D8SFavK9h{AKV4SUqE?+#ZV=~1Tcitt*XfEFSI13u=%r@n z74rvYAA=Vsh`1A^17<0V{ev~8GJX~B9tX{2XMOF9{I)!lk7vKUNy1qdaZt~P(AGu! zBInU0!OwiotWeT`>zSjV40voO0b|~X9D3o=`6(L&)(SyuZ~7=_7rayWV`bOf)YF(> z2WC19vNdN}c=!K+EgqdHS-+13w;)ul$?ITUz{Y7HN`l$Lyn@vax6VbW&@oT2Pr%Jp zn@OgjV@D^Z_`1*E@*WRFsmp{vVom(xFRpK{cJpGnKrVtYIkl$1X_(|ooc=omGI3KK z3SJQMU0KsXlf$Q7Eh)4745*PQ7#h4D`#1%y-S5o%Ts3HC5!`SLloHCDwiHHa!hfSc z3yG_RU_uOBJ?~)%L2tO4%VB6;s?#6PH-=%cNunObIA3S z?0fFxt~qY%rao?3EItzJm&mOYsnXfmMHVSTvFXBBTnA@%#-s4^HnLZ7V!)7Q19VnB!H^c+XguE7Bi~3zzX)><{iv z;Y&L6ZD}_z;z%G26r*+_CBA(^kzqE;wtst{ZEPl1wR(;-!ouUq-lN=uhEsyWg<8@p z9nWZ7{4R&0+&J~(f~*MvKbV6V8K_>l%ct(&aGT}wQo-w6%+$~S#6{h?FZ11YwW)#2 z8r9Q0CGBF4YFum!Z5mRO9w&&Eq&mjToFk{-oTHO8e+dPAXyr=Hb%MS7`w&Ax_27ov z4mI7;U$TJb{4*-|yAEu`IutGt+O;S}qMm;LWKq3md4W0BA~~EYcx!S+wcIcoupv-I zVm5DE$fZRDd-jLQ^`-<8y9oNe>B|E#8uav(VT_QPA*=+YtHyripaQMOgFV6$EQJ9XlEG}!dPL2k&E_M(vC>wD} zI>Kd&@?x}O-B5kR3pEZcXX*NVKYi&y-*QeB`XO99Ee9G~N`@}({m#bqjMAtD`mV^# zdASI_XVtEs9MF9q=*=UimvxwW3PX-Wc4rt?c@Q_OG`@2CSy>LrmCrwU6Z=i!;+ z@P()MYQk21`n68}5)WU$5AwK!MKcg~6$TK>B*olQRW6bV2_!4FAv8}}A4VSBPFJQH zXEQh8Oqw+V?Br$PdyAy^`naW3CTSH&fzqOGV{QZuLsR2O$t|MV8fw$@B!xrBZ*}wZck z%qV&(Kb{Y3pZp=sBGz;{;<+#BT$Z+OXiGEqq^CI$);RvE9Dv&KL?xu z=E{=4vk=I1NYAo)Koqb7%j%E7+F_TjKhtZthjLNjZ*C|C3>kjNc0z#dUfm0)rk^s* zG=OmkY}#dt)j7~*)5?DeF+rcRYgc`2-j`4DmB)Ce#-DwRo_I!zz)zqgsDCvZjK$Nk z1;1VHsc$b1zn9l>f5|VsaM9G?JCzUL$yO8Ip;r_V_Ip9xmk-( zhXk$$15Pq2$VW(;0oqMx7E>f_7PElJEpj#Mf9qcu)gA3YgI2ObixRL-k@i`9_qND~ z%njTJ(dBFSo+1beQ6toe^gq8W=Q9@-@O(DreErvXd2NZl^xDhQ*Xu_>0yE_uoQ0+( zsuboJuk#m}{F0~Irr$04JMo2wqQ)Lb8g(q@`?HDsvJ}&ir4O8@#u@XUX`JsTD&Lln z3JD_N2Spv1-Bxl1c7RMGNFWigevru6Rq|jCv6=o};)T1My!g7>3mhe8su~Gghrf&} zggA$ywt!ZSO53(}X=!ixjkp|IE_Q4MvoKVDaU^rXrh>Yt6EUf%s`VVy5%B_*h3Z{H zm;l3c&_)+Gyz?a1c)FDQSp%M06Ko)bd&Wf86(doc(-A?s)? z6MFXER6#F8WMoxE;p_I33``b&bUUvfg=d8bUl269@l;-sObBg)y=%n`OLBH zfFUiP{2@9+V1wFCxDG)9ynGN2r#E?cy%X?|aez(WV^p5yD>ux{r!wBDDOE03p7FOf z<#wx&vDM4Xg}lJ856ji9(4uEe!7sIK%2}gc-IbY<#lawnnEZDZKYaCNf|2iJ{bTjm z=kMfc%l5-Q0vK%U&QnEP6*1CS>ht_!8bnQ>2dw&h7h zX=?j$M#e6xeP;VGS%8WtIdyp;g!}Tbh0eq_%q^d5pq9Y8d&~ItB)}7Y$?LBjS7ch1 z6ExPauu>%OOfmP)mgq3$$5-z=`T90<@oZiG7f~#zRVHkzJ3nVSn&{=S)YXh{Cziw@ z-rC&L)*W~mx{!SDLU@PS<07%2c-cldPv?|cgG2>es37@EB*~w6BXAwAIgVg45o~!3h*VV#X14(?cdW;KI=MTq{x|aDoJI!zd$%DoYskE~O z*RPfObX$Zy?p|;lFTf)od)7eKLgVNEW~;%bG(CLxJ-dKm~CT-MRhK7n1SB;5Wn-)-qua`fIB~z`1a7D zFQb1QYT2p2rOg0rLx80!DMoBY#1D)?@@}BBz$#;se!#-rA$T$Y6_IDK*YhI^TlZH@ z*Ik=H*~e;l43L5l->w+nPapa6Cz6r4UgPtxEf^OZBR=)!2^P#GsEQhi1%;74(r*;N?$?%AvhQhOx1t0#- zJ}E_0kC>u}Ey0fE@3d{zp=^hZHK8OPrAa^6*^|$+9<(l8sF@7ETli7#u=afsPUfOj zOms8Y{T_VZ(Lfu;GM2Xg#f{{ZbdWIZnNR&|^1!Bk>RydYQHGigjlmy%2IV7f`r`FV zVPs&U+7X&{?@v%GL&59j+Z+3}g>-2~ikZ%<-J&`%$D+18$3mlGJirh!Z{)Zzg=7cm z@nig$-xNVH^6Enutu5McTnA{wj%o~KQ^WtHX?T3y$%E>h#x3(TssfTag^fWM#h6Pz zj^)oEkPVABlGMS=(w2)NuoD#J^Snna@HKhGp&kt~hOdXq<9Sv!9@-o`{}TtLUk(rP zIgeHfCW*WH65r3Ch#)U(+kU=H#t!>($wSn7&`n9M{&Ymv_w<%S^M zrfm|vkBvd&@$27d$^FtQ&8?}b9sF^nZgH>cHhVrrZh<04U8X*NasAXFzqcm;Yr$~d zZqGO!bd(76mTzALbnsI%`;(`!v;q-GEE;u9j`6ML`GI_IGiy3J_*3Les2`zXlCxXV z(&oU=ix76qvj;o*AouV>e`f7=kn7}XsDIz<2LTR4zlOZ$Hn4YBInx)0v9`TO*FrBx z8ne0Xg$H8^qjj;-t9X#XjSKs2$2pL#-+o1G>v^C3F<}4qDhmePh)aaBfwMT?w$wP8 z>^0MIjk(Ex+3{}PDV`R6yVz*;uIa!iz{d{Jne~*YxC*7fxa)3Y$;4}^!p|}1IeZ?Q z4`>&cSXW^TPdaU_34xvF_0k6;c&qAVN936jNtd={$jl$H)qiscU;=vJpqYhohA(|) zAu@Af!ml>)fUJ1W`CEM#cqh4`VCy=*Jx62+4OC$on*MRCDtrOX?%?VvlkJ-fh|m~# zu=H*+`K9HnD_M=Y%R%R;7j@F14P*}ipD9UvdGY|;*L7g0?^wIqpFcceQTf2D!~)gV zkd(5V&>k+3VsH$JlKTlt+b9pYzw{iCC@|N4ygWV+@;#sTQmg+cW8PmR_xvVU1Ac+@ zv1^Z8=jR8rR$kvTJiFqZ5+-|b4CJv=;vZ}J?eQc${9YE@)H6jjIZE%Hwu84eyL+$S z0YFl;hh8t!8>qkR?p@TfBpdSS*^bweOHuT=;Yh!;0a#j4e(3lmiXpK(CbD?7!NHq- zUf*2*g7R|ru?tMHZaulCTeWyQUESIA#m}sak=L60z%m?$rAF78ia6g#!2nr4B7_RtQAQGI9FFBHAa&{%a{ z8p@S5$?=@Xl{b0DyJd9&DrAm$_Jl;}utapLxb-Z%^acbpYIz6!RW6eW+YiWC)Qih{ zGm+6?$Z>e#A?;aHuxhJf_7 zYn+Qz4jpylcgKW=Mwlg=NME%YwOQ&uWE3eT6j==ziGV|BlJ5RZ4Yfu9;IeF zH9p%>M+UX>IXiQZ*W97H@O`7aIc^*apg(Z;w^=CLkKC6ezKSBJ%vP)>%5mNWW#}xZ(U)8ucjCYuFd%$Q#YP%Hod*|_yB;U zc@M=2l9Up*W|+9NJpNu=66~oZvBk&U(YOwM3<6oF|nCCbuWSN_(p-*@*k@Dlh^Y~xWkJVxE}~0=xHiJ z8Qs#Y3rJ&v zVhnPV?yTQk$#g<#6p7gm5T_gW9eG$D*RZ{9jTj8G2QF9f4!UkDsOs$k8v^R zpNYl#3{x4hZ`Z;7yOH_JO+oBW957EV_lh`tus>EmyUuU@XFZC@l`nAs7wfDS|we}b?^?5!R0{8ymJP3@`8 z|MIE%-pnGEGorEc+2G|#4pOgVSGZyl`v<94NM72Ltf8g1Mq=t349a)uJt&$8UxMz4r|v%A^1OBV(O zFUB9wm|HvddUpZ6;{Yszt&Ye^a}>!^XU(^pt=)yHTRvY7p8>xmf9 zcouko*#G!OTFopomMZq_Z2rTJ7Pp;WdjV{Sz9Z8N%D#EmW-WMrLXBKE?fAEucEYzH ze7_Ba9@xig=SpI2^^Eriz(k9OxqZd(0=_FkVMi{~C@a*n?C0;UhpjGP8$GjZV6ptF zzEhwYw(5WC8{r!lZ|3Nd)2aj?@t4GAjx3K?x>fYO(NC-GOYnO{WwunJ;lW;wQrEC- zX6`-IHrhJj-aI0h$;q}*hXlQCdA!8d_@Dj5D($pa_h04TGTZhfNiG9u5C7`Ri{_syc%k>-s zeuwRcz2NRMV3TysPv=}oSLdO@x{`C))u%Pkl_xd~)?Alq{8@_cdG0m))O&?Jts%O` zWn8QSvQQ=&Cb$sr(ef-}eR`nanpjh#^zFNKpilVYw zkb>kdLE;1Rh5eVq2SLvHfZ~}mv|tt-=ISV#Z1SF&Y93rjiXC#fZ^%38lqYC58E`d1 z-Qw#|8`KJQ>@>O@0xGQR%<$%g&bLRkDjz1Tcypb4J9dlh21~Rjjs3VQrA>p_`dbXE zpc_NF3pJ8i`z&-Jr%rM|mjkW2Q(lle~gHHI)E_q0+` zjK#ugHRsxu;2+H~5Y2oVz(ZY7QgAnaK|NJV+*|%&F`oMr)|cW9H{F;Sd;Owl zY%SO=6Qs?9x%C>-YF|P?Q%}6hp{>g$7uQNN$?bv2Zz3uEoj%H=3Dv6{)IGE2A8Jex zU6|h+GMPKKd`ZB2EONKE4;SqW3+3Y(S6ZGz&~Fx->C+x2utPxqQhTvatzXGp7fT)k z!!g}0Nh_Ma9AD5V4{~f+Q-6m792r}kUOdU?e1_wGx5yo~;P9 zyh;B&UjWPB=;KSA6qbEZhW0s!Nk}|6Vb8aI1o6RTkQ^>AOkX_Ox;)+dvVqD~A~NM) zNfleVKG(T;Onf%_aqt#@VtJExh(Dgi2KjBGtAax4aX353&~sc)>AO=FRNLM?_YlN* zZ+irr_R+p=X|lH?$J(uUIwE-rG?Ri_fi6?5N&cs%po15VzxqkmJ^&*%6(Q~s-E0!G zPgrF}5+XeNobIo~m9qU<%VNje9^!E zDaEv;ukqdT&FIur)6(yl(=9W8m5EV{o@Elfpuk=2a}voH5^kBVWukNw5H=ep zovxlu_ab@~PZMKnQ@U4i#HnsGF7|wzg}Y|Ip@O-#oIIsfxYHNgE=9$I*p8Nim`0Y| zkH^$*ORINZ7gP_5l4)byl(Kru*>vf(%TxjHR4~#Q;Y;*nXWbQ8=Cua@@d!7?>J0GN zCeB%>H(UzEe9sHjH}u&?L%7xDTfWIWHQgiU!$6#Ik65_IjB8q;H$8Jnn1&)b`aK<9 z74N(!8vX7uH+!r!7uUqkk4c7gF8Z0CzI5h0&}d3d5!lsqDYpB3&APD1hcseWx}jE! zuNgNq`_7>jP~LoQHnWzAu7F-nP2!nsU*GZGX~~dB{()xx?ONxfZ!n-;2;BkhWVc@X zrH#9C(7k|cje-xmousj=%OP#mx!PPLALBB9kanA_k6po@7)}ZBIqkBCrrJa56zPKY zscR)GHGC`e#sw_1?QHOh8`Mj=qi>6&+D*31L^~UFs+fy=Eah=KkT^)X4QKgDz}tsj znWrvnkneYhN3t!;`Yud*z6vAX!@{q0nb#N^rSy6M5Dt(DkM>(MpE|6+bG!DU86n5>p%DL<}W&;K!JqC z{QHV5RO9v3_zu&isVU;~OOvfkc85-GvDo`LD7#Z@yc~yVJ#f=!B77IqY(E93u^`SX z&=5tjV+05%W{y*2>$+ycU)EBaLSkiCXe!zJ+njG*%;lk<&yj+98z>jpCh&!9nA_v7 z+hKcui}H5mDzsevOZ!o+--zR^vBms@!l|q5b)dWXH#Eg=1HICWEiX~}#ja%)Q!DMo z%^nSB{YXohob1CN-#qwHHh``(oS>)8`z(#@;U_ham1zsj_9ssj9XmH0Y+oK3YAF&P zIW->wN=@bCcUEyt$G}+%nTpCR`Z9DH{KriN#R{CJc587dD&2(l4jQ~aDcjaZT|?i) zycXrH$zP(+;qhPY>n{nVX(m(nq4jtz`&lVX6<3EBWwnvxR6nP?px-z?q0q6vnsYAq z8Rf+mV-+U~@Ex92-S^hd+JVD+U)+rHJmq#T-% z3r}nOXzBi^*N*+MeuAq^z9%9lXn;L!ZA$9p%Nq2h*-&_Dp-t;uZHCu(4Ilzh;peD%YYC7|Iod+61DDE5^_D6dSLp=X-cNWr8Hbw%eELXeM! z{Nt%~Ym0NngOU~t-r*mad1Z@n3-2sgQ)NDFDb7;{4w>M6R^#~^QNF~sKdChzvt-_} zzB1WxQcrlk{PkJe7IeS@z?k**(#(bnyn)Mkp!FJ8hlu*tp<;mMjYfnwZD=^iih`8! z>)FLC^a2;p(uWGeN7ALn3w(Tmq-nL%X}Jno6?Nj;X=H`ToA05-CP zes?nqC1%=oV@p<=Dk8Q|~+Fjt=+SI$K4zf*tRVTl8n-YNj zJ@DCWk+BMPC!Tmc@AFkwOY}~lcWAQFcttId?PAfjc&2^p41kb`#r#sZ--`k5tk0?U zE>Aqyu4d>c_8x9t_?i6Ez`+zsz=25!ly&w+yY*IPDDS`4Z@SU>GX$eZOU6<`%UZE7 z5`?c1_WQJQ;u41WH<_|khs-LLy{uSJ2+!d=;?Vh@bA+Gg;hM?mi^ z_zSA2r$DA)AiYthLpFqEqnx>4momfP3lWr_%)6YLs^W{GX&hVpQu8eoiSY3IVU~>{ z{YQ1EpWg#m=y${6I)Hk+>y9Lw7Be`w0oQKYtWIQkGG1Wb+!`ISGXH88`9*RHYkNJ@E9K0SLQ8|4xdfWLuOpJ$i zI{2TP^H~)ijjZjE3eU4xIjd8{MAu|7^j&uEN;_KJfKU z)uGcp_xFEb%x=vZEZ+TsRNs5Iu_?JgW;gp|{e}X_E{=!vI}gci+rAT`+*5Uf+7y~+ zF>xY&sYrh;M%ZWZJrRmV+-BUq@yPmfhvZHl-6g}Guw#-v1oqXQO3q@m^UL#ig>o!ze44LP7PT?NO`i1S5$>EvljdvW)cWy@N~2Un z=p(wV0|2SMzX8Zu)Jf9bB)oFx_B3p|)aj$Y#U$H|A`PV8-E*0Ku-4Qr^7i*z*PZfw z$o=OTrVunL?^6EbXgUSk65WI zoy*rXb5>$tjw4M=h}QqZ)>nr$`Mz&|KvKj)X%M88MnFnRR2T?IcQ;6PJ5&&raHPOM zVYGxWy1S%CH%d81ck@2ThQF`J`;W&CHlN-173X!{SKOlA>V)TZL zuV>`u4l<+(NJDy~**3S}jNQi*&9+z7 z;;ZbUsdwDVY|r=h4pf_iNDh-;jCpFmr?(>;VGSLh2P~F~ggxqC=}ORP0gH}clB+;( z0@xGi_U{3k)2jmt1H2F~4_B8~$8_Xy-`++qj1xPA;cU~URka^ZsZ#_D_R^;wSBKH~g8T>Bg{sfh1m?R1_<#*6*eZuZ=k{|hgFUd_I$O;|sO&`GtJ^H^QA zs~A(W9<27iD#QKs+yEe4Yn$~JMSGhfRU%GBwQU-woD1)cCuX^DMgMsc-75)lz*fg; z|H^=Zsi)9=N=kRA$iZZ08f-t9{}oT)>Gcl#QiPJTasK%SVHLG3w^jIlR#R;q9&GDN z9XB~9ob+`t*L+^tH1zrVus^T1*|@zwUUl5q{)P%%Sry)o&t9G^u|nDIW=}~XM9vLQ zi#gwG{qW!<#b!?ZnDhZ}OD~sQ)PcLG#?q?!NI5SJ#*Z*Q@BsAcl?ihWsmW8!Uz~Km zy`1%yJ?9!DgkkQbEE0o)~g~qJV+5j##{*zS!%Mr*OUttHutb0ku29G5q zF^|Cb>J!5*!EJXvz`^VWGeSm}oZ~6^Cy0g6Ymf=3Ex(;)(;GCjxa<1ik&ZhxwBIaz zcuGD%RQ~Qi{`E@26cF#HkPPsJON8TLzkTZ4{b}4$U&?B#EE`YHEroqn2E7RIETLwp z<9OAO%}Bt49B(mfjejnn<=1D4JMs(`x+`fc};? zl6;4m-8=dz1{yy$mKvk*ZeJlitZ380);`C@|GVKkXZol7t1>IDo)q^aqfpfNDu0ii z_iq1}4tvDB4xj9yC$|I|U?itY42G=6MH%=}Dagb=N1EMV6lN;V%o=n*{~G}&aOWc} zW7*DRIgnK{>QBEIA)zAU8P=*UMv+oolVg7io8eszV1ItoBt*yX?7rGi6WLIt+ zRv`0Ve8l<}BB*Zw`(YsJfph z2^r7j>9#7%U=bNJscT@8Vt}zrIdBA%Xw>W{Q8ue&*igi{Vllq~XQoCEjGb@v{upz+ z&vltswKEWfUCSVgRmpo%OdAtZwxcIFwk2cygqm05&;bnWf$Mj4MyJ3rZZHT(MHl&gzejHI!DiQiDi_V2`9F_gRfZ<_x#6uLU>BPoHpFML54 zYkwgCcKF$BqCk}_6(&+;N8`Yc-WB81=@5VMuQ7?#sF8>CEwJ#ud*|@z`hJr9Y)bNYvRW|aWF;+7-l93QqGBI-}x9HH>P#Eo7+fs)~f21$|2ig@L& zJE73vE`-G4+v;uL`13cY0k6O?qU#RNg?VT$Bu(&y?Zt4#DzVc9s%r6$__Pm=54;K= zgaJ91+a8B=>D#L<&%Cd7JKBA_7w;og9tw3t8<~VGPCYKQ_(x+LVn$nz%qW;`az;&N zJ0l3c>4i91_DzQ$O==ch>zA)PY<-H?yL%{*v~XgF%8kkJl&-GioQw}eoi9)3ZSBUs zqGj9C;Zb?{chq#qX-y);3&6WffpqDAws7lg3p|1om?bn5EUoe&b|08;Q3&?bPx=OR zxZ1B%o>5`grLd(WaG`PQx#7Ad!fC|@o{|e4cla?o0rv4C2OQ?kl7Z z4nR?u=ip*@)c7gOv94|}wS9W#LczY{u9AOMQV&>mm5}H^fd{4u*x>s*B!+-Kc=jp? z=e}-M;#1!|hq=k+fw^D{17IWXwnB|BZl>Rf~2eb1wM_Z^806x1<>mcmjSzMw=@g0}h13f$!C0P+x=0tXx(XL1zCo$Sae@P9%>r05pS|6=gFneno8O zD#!vvV1_PMU`ML#O?a_8RZYvqI&DLxA53xbEVwa>Bae692~|uy7X72#RQ3 z9|zhoVO9TKBUH|R0>plNnUxfMJsS7s7uzC%_B#r74|;IVKLsN3K&7T{)z4hphp#4R zDT(cWhq)>=V0IUZ-F8)QG+}x71FsMBlsY2W-ldg+0=5(OFo2;N*~Pdbf_{y{qjnvu02y)wDC$|bvj>)}qn-@Nq~7yrC^ z(S0U=PpsdxXedCa0~Q3*kE>7qWx`Z7JQTuU;+wWx0KG>t^c zg`pwG+jFU0X{K~87@S7Zg};9_^An=K`T6B{*nQ-Gp8`$L6d8E-J$quXD3bgge=M^=jxnH6MF6`yJuHt{5Gxc>Mr1p7`LKa1h~bay*FeyNPsVUq1t42*VUdrWIxxKed5pdsR(^D?W%PkOFq_)% zg|J-6%iLmvk}?=BgcNVSOgK=8IoX;QbSN{*>maTfpCnVx71$gvBdm$=1AE$%l@5-c z@HlY!vLiV_Nx1VgQkxVy1Bp7j;c)lbCco;XM1hDVu&cigq8+E(fePdeDsasg8v+%4 z_o34KpamJk-~POb)iPpIlyr9NC57T=4dgDcczm|Rz{aAAA#%J|duISAk6yKHKvFO0 z>=?m$eU+1OC}t2XHVhd34^?U)T!g^`cQcgt0#H|TBe zG_Pl0@}~LsUMTP0BlWxnDv=RpeP14*#s0}fYFht*BM|kYCzqHudx8vVY-yn~z;Q{k zVR}bDW@~|adH?vwSoeSc7)2>pu8N=oi9F#cwqTl|Ch&izG*R*%y5m(2Q(@pU4s{C) zxrRfVnu)Z5#mu zm+2(e=6wy8uNraZ82umE<%WmP77c6CR@{&SE0v%ceq>bCNJqmh^g3Gg!Kyga%chS+ z_Peq*_iSw=D@ozy8p@~lP={*qHwc)!J{HpOR!pHj0Fkg8+g3xyi~jL(IUY5~_@3pP z{9!D}S_80C!V`t5uj=g-Kr$&^8UlujP zx;frGVsW6%#Klh`e26FDoU1tYY2MtR{Gm32@Dr<-j~%^61)0;s>Af~7>ng44N!Ns) z;|XRF+MMNGateI57>v-;2#(f}^4j~$9rO*ZdTJ;7sh4)B%h%-|-U}w1o_O0BkGPA* zcg+8YBLo6+A+C`i2DBfZK~KH%t(`?)P^bx4)&Q&b_EI*z#;CXcg-qSR-`&G{TKZUm z6#87Ja$+CO)xJy;-#iCDx)xl38dEbN?_RSYdbdoTdfEkHZlWmPK-Jf)o`x2j$`35{ zH_NjSG<#UdY;Q7#5cbLFVTZb5C%>&$xZLLlVC@ZNN6BeerN@G>8sd7}=4`CTrO;v{ zGw_JM5#DXiX-gwvkvVPzD6+R}pAtk`xiwnxZR8m<4s}?Ri#MEPH`BcsCT(olpwJ6H z3Ma6v$BtLXqXV)qyU)Z1j&{n~e(=@8 z96f3)<(N9*dnyZ2^E2}kZ}|-90{>Fl^RVL%{^sAR?0#1p10vM70T2lwtIl^{CvkGs zb|@4fTgv=kUuxj#=}EZqX&w{VX>hZ$vt3V#$xt2bijRX9eka{FOHEIoDWxDmYl!g) z-3V684n%B|kShwO-u@G_F&Lh=Nd9UAWkLorct$BP?M&pS70&u;u7{iDzh)|jm*fNj z2T&I)7ChR!Z3+iO8o3Y`X>4X27IiP^3AtkJ`YNJXW$_1x?mCfR@h4Mzzp-+bf^jmz zoCmt`NLmN@i(guVkCE(F`nPOH?tgNFWF3EG$u`|C+#O+}di{g`utMkIl=QNfD7(3|VM;Tct`-DX`HShy+_S_NDN^0GwAThFQC*m@l9iS33GU;}mbAzuOAZ+B{R z$c0shstS!qBcIg#NLpe1c8nfl|#w$sdmFP$2XBa1>T$;dsWY|3hKAE!YfgMU}=c5gI zi(mErE!8qk48@Ikr$|^#ih~v)-(PT&hbE$6uEbX+x1>fjo7kpfxsbM?!yv0hA9il_ z`&OS}&0rO&uxL{j;y|`bO!8M$=qE0P{+u^Qy_7W6nUT3V*kuWFJFqA%cuF^J2_BGX zIw6*umWIyX7`yLj(I~S@cNIrqMU_nyyDl zJirbelP!=K7BzZo?;qvjev9EHD>9yYS@#x!irOLEY-kK=ek~B$QU8vu-lw_>R%X;} zXe4>~;2nt8mM`4pEu+YBZ(hQEBv05i6JkL-PNWuyRAGyCM{d#&BKCgT4pnmh$GqAA zc8EJ(^O4+_02&etdTF9BBjwW0cMa}d3Vm1-b_pz`BS+LVO8Aw(BNehj%)3huq1`5o zA2suq^O8VR8;t^th&`#Bmt)%UYkfR6bGFTow#P}uUBxB8-EXGp*@z4HiN_2k9Mc%? zelqwTjw}M_FP4C1NtKJH*LWayA-`6yWkaI6C)woq`u9zUt)q@QDYNzRct$I~9}6mD zta=E??{H4emv>oL?YJ5_v~^16kh)2P_u+V=apG2GMYy1x_TRkFAM^l1<~N^KNZQ~P zBDBID$N?mlP%{rm#b0%vty~%Cf6H)^7EpDWq=yKTWR*y}tm>!x72p25`MWIoLl_9I&`;HbNvMkR|s0Uly#U6dd6j_nnYLY;hZ@^A>ei$x?x)g6fJR)Ve>Z@@i7( zPSls@J#loh92E6V;#UZoD?3XieqY0~9cVw1O~IIH?&WV<+PU^@OWuZ$<%&`oDu{0$ zR+M=;bK9da1dY>JIL5bH%Y)c?x{?;ra(eX&xYf2rc#`Vr(?j>kFIf*3!pN@LMD+#f zr%vMUS6$L*2`!_>C`R_n5>#WmQR-_@W+?)J*S@N!e=?~-!B3BeVTnJ1oWLoBem5WX zp&{%4dw6M{BACZL#rL1Z#uvozP^3tA6Cz( z>o7Ffyn>atApnXw^m(X^;JAUE9nU!-2{}#}K0S!~=CvbclOViM#!7_KZqw;mA)2ew zN5es)fJu$$l4-6l%N?w%9Nrd$vFEt;SB%khPiZ(qL-zmFG0=+nL!BkUCDcVtja)HA z9E<~`M7}YYcPIb4pL@&ceSOklQYypggZ$7OXCZVHG>xeBKjD|m6SWDtJ-`{t=j!E| z35|Y&`TC{nGG?{w^dPrL8MP}eZk?XA+Z-OHf`M+|vT)Pwwo2!(fE4%Ot~oFrU$Me` zQ}*6peDcvA{x5YV7FW*g73l`9(Z*HN(q$_Qb+2h)E5fCn9NwveW z!!#*>VH;w%VQK0NlJgyGP%xd)QFetx@fE0wRHTn()d4G3xeFZIqeZSMcj#4~WPOx` zT(A%|Y;oW9e7^%H-PVMc{v{niPBUIDyVLX^JijgzdL~fN)f+jO1aj!*VSyC z3IqubNVQ!<(CU-EchXRQ{Q2ugXRMfZo0optQvG3`=6h(PCCdZ67<@+J%{^zLG_ln_ z$#l#BC0%DhpuW*%>faMn{F~x90&<_p`?z z8RE>%Cd55EhCJjHcK!!`CFsD|TASfaU}nCzfAc!{U?7>s8G#xn8E;gmaOsOG8IdHw z5Mha0o#_>uphXmy`CGr=r7;?j@?~TKXryWN0IT9t16?5ZkvYK_neRRjzp*f6NGCkS4cabN8>i+&9eS=@F>9FkAqeNs50gMH^{z%r z=GkcDDGY8Nyi^3amFs_F8owX9jz?`D!S^T-Je)XeM!Vf^ljvMX%dY{k(29- zq%%^dLbaTD%7Leoy+}q^bba$at?dZUW8mI%4cSrAu5EHUq*9WjMH9gBOk6*ovEql; z#X+OHiPvW!N|?N^6UfkLL?Y*KWJbeNBgUQP?vA@N(%di?1DqIXHQo!qeND?afp-Ao z;aE5j#-73jzvJ=B#qgAv;?cBqfXSeiXN1=1K(AA^(4vwN0W*pFun`I6YHiE44ks*D zN~J0KpXR`+_%5Nf1lGiK}ZP*Qn~OVZV8<={n7xrv3yRaFs)8rS)b~w+UvD!aDOwd(A}+BeW1)({WMnuA zz8P$WkVZpuvbMiYam>gD-8w5Dl{qkQ|C8ax!ufT*buWnB@yBPmzkUy)>p>rgDIkw0 zohZDvmq(8k{Z{LkjJWxN(Tbp8TN}Yw+3ukk@(4MN$5T)XD=2tS6A|*jt?SBp1Orpw zTq6?H&d713zN&+}Lw_JEtO|KMTz^}nsaFB`N!RZ@V*KOFNpx=usd2c23UEz2<@nzfV@K`j>0y*xUFIV7S(Q;5F=QGQ~mnJ8x*M zSIQ6OcUiZGoikS3wtN1`GXwSGt3;+-Qqk1<0%@TbyM;j|jFi@iAfd#x!{5`P*;1h* z9LvoeuP{{37kARL6bb>Zolg4#H9DAP8PU}!X7e~Rdsy^vU_pK( z5dppN8@4U9+krA02Tva)zDn`^-dxP{(F`9nbt*09FZX1fKu`7B@rUw`B zZfZ_svEZYng*7SQN?7brKO>on$|{)suiyTCOn%~A6vs=xU4m89D6im z>Y*b0);u+@Tfh$^kd0yCTmU28L!gp5^^GFc{;lq;aF;7@JM+{{9h-i1Jr~S(L&mMP zH*G`$0>%Eln@SKSM|u5i5H5oKqsQ0oOiv}gpIS{^(|cSX7Ej&ve8n}M$&=&UY`NNX zB9PnTpLsl2tdkk@8v8xPu$nC6#dHxiv{5sJf_{{-8gSoLW+ZP?d$0%dVFHZg(P+tx zBm1yUAfXAATn$RQ!)qOQh4lP@n{Z3!X>G3|6{x`AuD18R?wJ1j0oBlV0`wrZS%ByZ z_+Q)p4J0pzSghFj<8$O{dq>NZ5X3wmH=lx9Jo-z#8|xF-qxB&#IqR5!G^L1MuY%SU zPSXD-_$LVmO1rpjExJ+~0~X!M$gvySaF(K)f2fH7QdHeBTQ8`Y&)ENL--77Do&qA& zaZ7LWrij@VYuLjF8zbIk!3RRhttTt(o! zn0TEvnT9aT@T}3NiR!h`zg%&eA^g)dElv3g!GgE05Br}Um*^kr3`j$S@xy*^=$w32 z&~y7A0m^_qoyGb@893J{4>fe`5^Hw7SF1{d8CaUXoZD`z9#|EZT7I}*n)LAoE9r^$ z!n$N4qsY(oeqqjD6l7F(v>}D2=9S5x7{K+(#h<2yq;swphdO5O%`<@_`*gw4%%I*| zL@L{x8G9C>c{2NTqT1BE)4$!lX1yU--?=~L&-KQv1ZOKw3w)nG1R(VD^w`CcUyJN{PlUUX5^-O0WClV+O9!|~-WPCo>RKcJ= zcn@dr_$U|%8Z)lz8Uylh_mT*J!uY^-|MHY`QYSG;XgV$hObB2xU-XNa<^$lCB+b;K zPnxB6kRb;otfJlGy3d+KFR_61czW# z?1sN}MHg>?H)O7X>yxV&WL8S2dx3tkI!ceW)-7_z6ik-y+x1w7Dft7#_Ay##bkED} z2-BNr?i_f=oyR~2bwNcnu!R7zLc_^^f{rx^YzSL2PfK~Y@0l^N-~qHtKpV2~X^^nm z45%I(gu7@@_jhzM`m$%%rk{2&9*e&JE+6RW`&y(h{PTv~Lga61 zjmslbw&U|Jibf6xBg2BOqVb&08^AR_-5U1G#{)k;;klmGY75(JDN14K59E9*m!t#VB?|oz>67#w(@_7vh zH1#abFr=F+W=kOKz>h`n0QW>c%Ek1`akI_<*R*;qvDTuS~we zkYdxl<0(G}+fO)$>rx&!Kk`0x`Y3GMssiAec}THF&@Yy#n$H^14=2ZB&}DlGx*^;w zKZ8`<)O4W3ihfG6uvoHaXLx`7-Kk%z>*M)<8#^QhiA;Y0UTQ$de9=x~8h%DW*z~!E zp#GiE(Hw6jyv{dYF@ZXb07ww%rr6iC+mxPW-s2KwRhh37L?fExnS)*n=WA=J(0H+S zUlCxeee3MCt{&5yn$fGAAPq0wvjor#B6mB(;C`kKJIzqWSRIS$K3rO}lWlqr3Uh<# zhMly?Q2yfObQT|8D3_1XWkE8Bp^~CploK|KI>bJ~gIcyT2?QTT!x0us-m+8;rt8rS zj8!Yn>*|m}Jm2WwuTl)?_@S3V-A?_NP0anpE6Em&`NFp-Je(VL+w`s@JbRk*Rq#(+knxKw3zHNFRPI(q%@{hJE}Du&+3DO$B)(65-@Pwm#uO71|xeU2KO9A##0$XUrr0xsEDZCaDdZy zN5Ma|40BX^c~d)&x&%)VCf$);TONwDn%5(q&9O%^dKMh zm51_WmfsuL>7H)W?*OeiF(Q9EyXdUJD?qxdZ75I+V!}dolm}lO zf2uLT^Kj8E7bEw;kYX6N+Lg zns=?nzH}eR6-Xp&jIf91L!lYFuNCv#xu#3?rYZAybXT26jbLh7;r$IM5U9-l)EDtE zAaxZpru%qG;Uwj!8l+rCdzQDAhS3+b`56ktOQB}%K)Ld!g=8Rf-+|0-6C1isjabFj zGzs4a{j7iQA7d&xIBo3ht|sO?(%rH_-h9I@2M8CIDqBR9V-|Ii3zi^eIo6%bgdISf zp-M4}kF*Sn22WN<<%#-@3~KHySXyOV6n37FYI3{2^3yZI9~fdC@ElkyMwJ=+55VKQ9=J7Sjt&)t6HWc2mmw*}9G>OV{y%t0G@65@Bt( zW5su!k8St%^`Vv8O8Q==PTuLRXv@w>kzfU?4tP{tZ3i2yZq!>=l6Mj~k&a>QqzD`8 zmIQnd>v~xi*Rl7<$lP!F|mliIY5ZnBc5=~8b=KEjgr?h*tcPQ;Ff2}4?f zC<;Tq7PTxS$&SiFhU6;!I*Vf9bk7YkFAFB#y}#AEE{^Ax1xq*9Y>-#$T=PsqdC3;g zWoxW&ioQTp@O(6JiS4`ECko^ik4jduFueeLqkC2FY1H;4u3S|wieWs|T4%F5UGm-0 zJeCHIXy+VeB$3DI1ty1Z{hglSM0b*<(TLDjvU0B?)b;N*D0$gho*kVKHVo52JN$#^ zbjx=NOk-IZ(A~1&r5kHrJgBOr&-zQe$lh#^jF~sp*IgPx79A&;)bD@TnKwPQKL;0_ z27Ev*9#QnkF85PoS`lIC5(Sq+MdZA?SQtw|jS9CDs2U6NVE4sL<6Fqt*~3l0=Gpq3 zrCs2}(S%|{#A}2`UTXH_1V+m<>qeQdYFd9<#oFFDBQrUM1H_M616kFb+C+$oZQ1Ys})Y>mma zNKD`YdBJaf2gWaDq;g&S{`5P_KCk`0QbNOVVPof?hLuTZi<^R+MJ~#gBOJ9-0hFkU z)#}6I{VHL5g|!Y(k!9sCZ1#oj$@eb4!y0#`dgqbv@zd6dt?Y*#*((oR7yF-P(hyf+ zj=YiBZ3Kea{{cN9laR+#;W?uD_)L1UJcDTvzn#wwZy_f)=8krvw+di1_5IfXvG%d} zUm*#FbL*6H|4g&9kLIhlxMCse^<$wJa<%a(wKw?lU93 zcck&gA!R^bt+w}|TpqzMwpq}J!}V^%ufY1};Kn16Nj-e~BkV*hDX8L_fS&PgJhTVt z+CHBqe3tlf4Bgl-E9p)oPT0|=F_LG0V#77+P0g>0YKQyoh*?yQmJXcJp?(+lqA7u` z4in>2>A#ZzjZFCtizC`4zY3>qG=w}C5p&evYjku1f%cv9j#7dP`g7hW7J-Aoi&iLfsix0aKT#5&s( z=?bwU?2CRoc%;Ct#e{@&n~bOPjf6&D}Tod}p&z<^FMiQJ73Y zPpUzg96}WGa*4l#6zr)T?i_Q^?iD~KmM7tTSjX=>_Y1hcYFBj4yW7V`kfsO`BUYkV z>OVI4(Q@EWR9>@2Sq;TiOo3qWah-2o%_u}ekQA=jVL>DKa$IN-~9u{>wlm7^q`-yFL1vyO}Dm^Cv524y;aicsRY zG8te`>I9GI_lww4>M8PsOhQ}?V;_oN!5OL!-}0t|m+DM+WY*&%eoO^u>!)&JN!zSm z2id=&a0cox-adBe7F~etkGlky$$v7X@4gE0dU~TedqfX7Bn+-1A+*q{aI~8Z7HD3Pu1cL5ex5^fV{;D3$A-ef*tzs6bxyW zUMrZ+Tn2Lzybf)?|+=1XE;q zB|(D&q<$ni=0zzH0Sq)_X29X-*q>rjFQdV?ITa&jbdKyZeRwnw=U#^@VK`|t5zwEL zU;FlHQDZJ)wW*yUZ^H7!4PZc(%&$WVc)f}pugAF2syyT85PQ1)(1x@fRt zdSL)`$`Dmkf$QrH3mKQg=?auVO*cjh4CA307}CK@`xGlcPtZ zm^j-9#=TV`qdl`;wk)D``bmN4qj}e0*brqJz~xPRWg!!r!JMBIuyRHWVAFPRlS1yC zx7^-1dg1r6q^XLJuWW?@j2~Jr#;Q9uOM2VGS>Z@+4)1P!s)TC_QRQ^mdKegb*mDfe{v}#HTf5c z8kshoO6xSoF`Aw33l_q-`1K3}a9FzH;BbZXn=AEE&${k$Rh9WkORr*#lM(2Eo?!&V z$R6;uNBw4K3kY`QAcYl>94)C@j}`VA&)2y#+=Q)+Ld@w6nA1EWJxgYN#J=MfrlSr zO5f&lvtOEF^ol+chl&#fR^h z(4s&%dn6UGQswR5?!H-bS$~_h2_LoeL3+Nwv(^-IVy(hup7f6}kF2J7(8`*gwRv`q zM2dPnJEw6H3WP9LfU(?}yoVeRbP1B7Bx!MV3&ySMirMO}CCUK2nJ&_R5D=h^+hv^D zyHFhDP&M>{g;A+AqwPEH^d~sfIYjZN;`LhtatN2C?qwWl5h(Z8@_Z|iQ!or4n?+`{ zEx4|!KAV($o3-t-ptiw2WaTQ3EkpNRA7UcZB$+e=4daNC?_r~3&f7=$vT^r;CQdg> z>rMy=Fu`4O2ra}#ftu7U_?TUf5gGo<%7NEvd+E54XR)775irJ(G3vC3-cvVOurP&# z|FDC}G`D&QjthR*&=cEds4q-(11+UE)Fl3Wz5n)fjS|8Jq@v+{MiOXYhj(BL-Emmz z&E9it8SxmcJ(bnhPqgz{yI*8k<;yum^#jAw5YG*PStFO@x9J(><>iB|x*CcT_bPQZQe7MU1+ z3%OfJ!$chA5&+Yy-?H}$19>SUV!&QxGG0A1ak-HBJ-FFdfxJErj_X!+txs-k<~tZa zGE(EM5UxkYermQ%2WeAj$Q007!HmUq)Q4VXV@GiaQu45W-47}4p1?%`^75J4W zj}WEhGI7gF$PtNBe#CaXSjnqjd*ps;OBUnx-HP~>`KT#^#UG1&p>F?WW@u<7w?M^D zPWB8O6MS69PjPLFQ)x1Zy5ABy zU3u)}X@wCqCc8$1Lm-&ez{-t>!Aogyf}c!Jibm3FnwIZmk=9WuiZtIELc_MMY729+ zm|&`5FUj^+YwiqnC~D_^0eXK*raoG!JPvdJ;dXrN6x@wU!i1hBqv|(ORA}U&@wt6p8Z1}onO(CQ2 z4e~Ikov|%vSX%Fu3=?PgskWRoqm|aQI8hPj26+v1TnGUa!4Ast3{dT6HkP~fNlY{x znemFlJpu8zA^Op#H7(3e|Q~(OAz$0pxPjVgRb>fepOmPg76jy z-{TF)^-sS`9F1;|G;Jj^8JXiuR(PS z3)xegmjq7q6FQjEr}SFq-de2u570<(0)37@QU}a2Ma%0SBoxFM@3AlPYB^c^rX1~v zxb(R0j>P=>n|9s0Rk{zsI`}CfQkiM)D96^GJ}ftWQQ`QKi%qI94qTy>YwPIigmW_I z3`v3_ENx=!njX7xFwNmU&K#j-eJ*esH`5l=fA_kn?x(IxA?7cmnBtajpycKIgCbVx z$4mKg7i=Xw5!;|<%Ah9aqHwkFe&| zK>zvOQ#y+~H*b+{)6kWL;h8|4M~D|E4{J5AWMVo}2wok=5aQ zfI=)0pXHS1x+itW*s8blqg-!ggq0(_g^9ovR=aewGs+qmI|au1+}Nq3K5ZKHU>2Z* zJ*9r7FD})rF5)B^WjMESV%Jq&5w7DdUU(7A{(O~_4$v!ZmnU8*jqV=z2PgQPEqzZ} zs9Yd+?!At5_DpVtW6cG*wc_ITUs*U9TicXsK`s`&w@UR_JS4V-it2~NEq`|0G^)ic+Z!9@ck+a& zql@r!De8AYz}Z~Kt{tKI`8*${s!10P{3;i_a@ zX+kGpXfIIdR2B75WWEQ%tA_T5Ru?eShw1_11_dH$;LOJiU}DQAtvhV}Za%?uV$8x)>9Vh~aB@<`Sg|A~f%kqt%A&8ZXgKvnv;N3bj(#3ZxgH%bl-Qt<{jVzegj&G|P@y^t@MsohBBwH3SjceI0W4?+Ylh(3}rYG^55BO~mZI za?RTWtht~1pCLGvba|gBgKlkS1V}U!0rQlfoNQKC+uKwYE*Yr3=Fm1fI*P>@2r$zY zXb`iQ4U(3TJgjhX@%=OE^#5~-Ilv_feZZe-Fz0ya>*LX4UW+c>_~J_zh8Lf)Bc#5MjKWdeCj?nA zmdeLT(Q)9O-6Bvki}V(-^e=guj%7Vyrl}qJS@L4ny#(w~URmG-rk4?X{E~&gX$CNGJ>YaKJy8=kngRa@HYMC*XRU6IvdbRv^Yg2=qdK&=?Sp9*l&7Nm zvH_4mVrQ$Le6tCn&dbWm()JZX6rD+4@!-GTm?;?GL)zdzCQ(}3If8A@PkgeF_%Az_ zcwjhbtf<0JYfe=SN|3mJ|I1KhjprtuR^;_Xu&5_gJucQYsM@qBTL~N;pw=664F}3= zI~QulaGVc7q%!X{esKH3W@v%x+OIo|wi=tGek}>fGhO7Aiq5*2xmyBQR4q}MIIKk9 zi`C<~xO!xaU6T4GGj5Ei|HGbO)VqhQ6uh4%TQIXI_HomTQ;LoO*F5H|2h6ht%X%WJ zkBZkxk^*Dk{Wmoqct=Ke=lQ-*f!(P%JIRf+{e9f|D^uWh#1?k#=V_SpDM2IaStge& zMD)536goK!_w3M&PZ7gd4ls|)7JUj|za<^!r1aDi`Pxdr5bvVwO4lQcFa#Ylx<Sl?SGsa){R9qVErWxXrp`g7T++wxGBcbRi9_&oOMXKz`a_X0`7y z@3dkNu3P489;t|ovvoET!#r1l^zuvQ%nrhh0|DCQi8@lCqxuK~G|=Vx4?nrVbY zn7;4nFTs_*I_Gk~ev;8*`HGiutj=i50aKG{R{o}%5M0{2jyvS4kjP_O0Ns=NU%PrB z`=#KCzatDNjD#MF=LPB%b!U`uYhIXGLhS)R5&vlw$>Ux3)CeLx`$xUd3&mapAqq|) zvNVu||IOa7U%s`oA_B_wmgq!z|7mM6AhVCq#gJ+OSWa!@q5a@TM2U4+x8r0E5LgUH zuf~BdeEN?_#emCMtRT7rqd>WJhr4J5MfwAS+2C&^np^7x#~a7~uXo(H?fyU&N3d+C*a!*?C_o-d%~OP@By{zLCYVu$cY{AT6~g`-hx zqpr|l*S#d&XkT9QXBZWq=_5P`SXWlbS({%m7iIdh&fWGwD_$3YS`E1~!iFHsr z=Bp)`3CGO>8k(wa7Ah9fH@VMk=$&WiD5fk z4BzCUZNfCC7Pt>)@Xev!<A#Qg^;2t=xuh$Hj=O!sOrto^)KWZi$y1t7tbC0UN~ zQibM;&v7v5OhCG4)=QAESeZc^i|$yg~{fc-@DV9*jJ}9ypgj zyw8ht(de{^FBw%Yd9?NNQKuvE1;8j_GX(AoZKQs9-EuVaegjd#Yke2p1%!UMld_>g zaIOnrNCGx+s4&f5Qo@!Y0xkD&9qdhJn#`%8V@(@G=++SD>!{Adirf~~%}EQpiGv^K z{Y02$wt8Wme2w3h`+;+@zER-jk3ZSHO~o)lt1qE~$ZrWSCq0eOhP)=Bvc2b&XaARu z1y|p9{zlcok2lk0-C6gzCOB9mV=>74$2v_T0TO;@bHJ?vMdO|h&pH`A@|XI|w>Ee@ z&TUVbd95#o=>YkG2grWt;j_g9 zI_J*_+T*(0rVEVT4;@kUJN2!a?}nzXl6LjE6U`&z^J@A9Hfws?lVYs4vkc%~1F|`K zRkEEr#e*A3(Jzmt(mHjrv_1@44<<)Z3fQJK%g5Uv6>?UrrUZz3TLKd?UCejq&(T9d z;qovt#O=L_1NziG*#)}A9CCqnfFV=RzYtQ#7dacAdk(!`_Jk=!K5f7p@7@Bec* zu4G!om*Ojs7h-Pt7{^hOHx|CSyobYFimq?VIHxw|-`A(dOnbnpxTH+NNwViiA@xT6sN z?OOY!+6ZijL<|v7<(1J=jc?seduUx!ChDEw3w(w zsAr*3BL)cmU=Z6GHYNw`xR;EqNE9vmX=vYI*Du$NNcSyWuMwC^D(2~lZ8er`ZMZhB z;p@GUGby^$zoI9cE&2NBiJPPj#<-h-Nxh?N^<%-z9ReCrgll+#tA7kI&^r}&1#{~Y zW~V#==_8qu!9@IUUu>=2aKkXst0UGQat4pttqS8hTvrV22NK^lQB(vFEp!f0afzia z(0W_WfIeoji?~lnSzM`7#=kRQKul$q*q)0#o6uRiwfD2)t=I3|8%jvH?U2%fzN3k% zYZa@!_o;l<&e4pL-qgYu2A5VI7HPE@+0Ptnjj`S?Q}P;5_=988eRCd)xV_7=OJ?&` zR?6aWC%$I4vuiNKXT5_=x6cK+_`@$k&L1D@SDTKgC>k)A-!3G9*Nhb(PQg3f^rwb2 zhGfHTI|U% z@m+1z99l<*J5Zk=^hyK7xvF}4IbSuVz80bULOiP8*4?Sx&KtGvcyA}ytIClxZ9juG`iWlyP!P$;a4lnbMZOWR>oY)OkRb00U*~iSv zV6?D%mav$N0{TM}+b>k*uZeCx38_9_%vi+J7|s7G z8()N?*e;s#-HRKw8xmeT;Ahi!9(!FrQzxBVUTiRx=(UOTe*#z!K_kQ3fQbKKpn;A5 ze$LGA1kTryA%_)OyDN_$*>v&$Uk3vaXoLfxH5EfKBk1@jadc+!zQ~1-?fV;#hlEn&uowIEN~gVi2I;L z7l9pm>h}^p={IXLxPs{g*hfwaRcir_43N&?%o;Q1c!o?m`wRI=6&)0h?Gn?~~iu5*$Dw8$o?34}y4m>ioJog1iLrxS8 zfjd!(d`@PHp6XVOxV|fJ^cO%)>P@Tvdp#5#aE}PXE)FJwPIH8-T_g0`TCPahtT>ck zQ_ZU`e6lRO9Cu^47ILS$kHeqlu1FcrL`SraeToR}L5$ZO50CO|QS0=tB%;p}h6T8t zwMbLt+Vyx(XV(_B*HV!aoe6Hd%!(%uFYQzO&mC8$nY9`u@R?OLps2juv*2jk3$u~6 zST4_{^_;)0=QA;d_|@1f6Cu7IE5$dT`_sCzQ~guc#@U!t_zU2Q7XYEWqJx$I*44PF3`%JW|FA=bcl_K-)p3;SwT3WAKr60rq&E4 zEQX3Kf3sPkO}c?PU33mZcKsx?US3qA;F*+v{b=FW4N9kx?RNVfx|HTl!y6KGXSa!% zuZ)^?wDR7cwwy09@C>a!KIqUs3UxtlZ6FQ{%;aP7F=7R@Ltt86n{XD>?Rq^lT<@J& z$g-4Jt}7Fl{XfdyGOnui2^-#YcQ*(E(%m4P(j{FY-QA^hE8X4Qy-`xSLrS_sQu0}N zj^~``dH-MDkIE1BUUSb}Gjq)~>;9?HaeMXo^OerrC|{#p86Ne$yqEKGOQpQSb18@3 z0D_Q$j8a+t6xp-5y-aFpsB|lkLxQFM4yX?Imp?Y+r6c_*LH>TYT5X`@)L>@* z^mp?azz6?sMlA44uF$8KMa_l%6Gf=AM*ShaWu_#hvxTy=MCl-1@Ta-%Bj3 zy(S)8T`oRZE_mLlxyZZ*O#bzm8goCJdbgi~Yk&V2m7WNFAYTd8FU<1N1hD^oR`Tp` znDVaAZBT`)paCNv+MAToAv=ZHJ-q0FP*tJ_CF*;h{L#muiC65;1S`8E?M9Ms{e5+$)D2I?$!#3z|N01j;lgK!tH&{E|C8b90?Iw4pVvLE zfcGE3blcCZK7DAk`Oc)C$$DMqKGD>=TmO~gDdJmlm3$LYXtb zKp@n<0j!qDXb!7DGX)tP(e)(Bt?%s_>I_v>DyF5i@C8{HGWbt(@%uBisFipXjz~0t z8jR%HO97?nI5vJ^Me}DqQ^(`V=Zl0x1=vjQ`o1MjT@=Dinl}1HDG<=B{SMQ9KWG)U zhs!cN^T#`y;EOH3eI*v@h5D{EPGFwD*_Vy=f58GMAoBUuXmLY-WZA;xFweVJ+0~k7 z5{?|Dg<2t_dY5zjefj>ZolM)lj}v|N8$hMjGbc=_Z%WaRz$~x6!_-#vV;3J2a7Ezq zrhaR!!Ql+4LO|PnTUjtV3hCHRBi%Z@5FSNqXd{l%Mhl$$*5OmVVqx1vDj4$lL;%c;Suvva%6 za6jeNrMFyrS+k9B)+V&p!jIbu7@b$$eenH%oOjZ|pXBbsfKu{LQ*uce%ZY{nl>V!+ z&eHOId5Q%2Uls6`Ue{uHR@}P>!xAee$JaC*G)HLHFS?o-(EI5Z;tcu8S{B!O49$98 z%plryiQRo3j!^(|%il@0=zAnP>)Org4%yU@ViZ!LcU^A##kljislZ>(w3#ieD$^t< zJI$)oxLPMW$tLq!HkG18Ma2bmkQC55+B-C{{#XxuoXJoqyFdR zu;j6b0p;$};~jV1@;`|l8(-jC-w0yOH2M^&@$^mi2UcxcBXd{k*_;qkS7IOe6{GUp z^SCV7ka5~f$l`J=xjkL2jJ2ZxxQ>NYJF_pr;s;{+Lc=4B@S2!B!kj_FO z9>p5VxIAAuzEj50fFnkj58+;QvV;_hPk7R6k7Hbybf}%xvR3GMtjmw{x?qIpBwZ~v zB6zg`jz;LyxDZ3n{8Mp|%Ybi3y-C{9bgL;lr`@9XuU{US4{Dm4R^3gnck{+}Ec^QF z)ULXV=rSIazw~k3UX^iP3}a!p>wvTLj>StSi{l47wf1Ukurn~nDx`;ne`+~M}#m;J7mpZ}@ljXwfQ zKsEH+wE;*eU#a;J@DBIfS(V~f`>0-c(cCvj=*m}alsyYDcjfT59}!p}sKL<5AS^@3 z=iht&hi(E0iCE8NyLt7`|0MfYAc%*OAt2(MS)vn1&KMaLy!iwnR><7M`FihOk#t3D zN*2bsBpaVb-}`g**1R5!OO&<(bKxFwsmbdr8|l0cmMxkG7HS1V8kRgXFzl8-Mdg&fw5Y;`+HY z9ITxnmnYj|JdhyBzO+=%pL@DLqEo(5i^h&y(OY2ERDQhC2ZM$DU}ExYh+TQ$k22U# zd&xTpxj;5W&vqxdHAN8z22HR?>4<-RScZ3S4VYv3kJbJ)p%ZlZ-KGBm+rNe-U&jnZ ze~pPvci2EXk0SXp?3-Fi$RZ;j|IDz@uX_Y}y6yf7tEBQ{%{o+{@!A7nJ@F{n43G2M zmyH!#A1o+6b!O%k<-ePNn|U?1)(oG|%l{PwF3-gRq9eQ#h6rMM@3E;hXgJv?IUHUS%u|B@Coj_Z^^Tz#KgnT zpTY7H3@HVCnYJ&!(3$$Ff&PBL2K_E9!{|23%4)+ZhrQfCZuDO-D}Mp}TG^??zcxe< zKoExy$O|#6x%1aatv;VNJ*9d#X3{41QhTbvJ-Oa^?5vI+s%4_6P^*y_4xHTk@^1=Edo z{}6qSpee;nXPfnx&%fY@kb)-;;|(Jog1o_Apz-e~%lU1k!t0*`OCdy&S8h`t!{STt zKH5y)g(zlHq-ol;`&^4SJ>HzH8{TJEp4#eSP2@_YM!c{%HqWSjE$OG7O7zsK@Pcr|*<@eQ&chz)@|UNN3o=XR z+%0R|_-M6FFefM4NJ$$h|25qoSDOz50`&+wHO=VX*p2odLLx?T#~mYr>E8JgA45t> zU0AgU2lfzx)faHXWGwTv)F&TQE$CfCI9JMdemTNh4+{$eYdn&>%4+&nQU29rn^|`% z)D8yadj|P4;+}s_SRX5zdaV>viX4BLFrj~rIV%bswwrO4pv%OMFO~jZcf-W8{|t_c z9fKHW#jM-Ok@ALQuy_8+`)CI7QxYL~;;~X^I(b0*tXclbEhs2RxxoYNP(b)sgVs*s zB}=&*|9vb^P9wC2_;Rxk+U0i?a7%6?Sb45Md>#2ea4i|^2c5q2j$aY+KVh350XF)0{MoURDfJ{u zD$ZCC+-33E$2BJ~H$$gk&rp zTo}Wj{W=NQ52Uzc!k8cekQ%fSntvNi;{t@a>NoLqnT?z_WQg1J?~*x9FF``k$H;Z= zE)<@yFx}>U-1?5d^=dpb)XO@xuKe0p_?5iZhcyoqIUXb>}>`#(Mzd+ToM^eGryjFGHPV-r{o-zC`piwRw)*6E%CQ{~5U)7={6P z)YgAJ>ECH%hi2`EmNyZ)E~mAkRnu-HoBHm3rqlfNkB(}`%*NHvj(7~=C2~!aB42tG zSn;jIqlselM1tQqzyB2Vd{Txa0Gw&}$K+MetbK#*58|IS+K5k~Mqg^YyCW%kFGw)@ zZ;mpj%1#0+m;BlV-9qA<7TU>y%-bfUH~e{jpEiqh>YdGsbsQnv$-SXiod~P=qmu)9&IDZ;~}8oncaa`5@uE*R*jo)sI%Zm4Ks2ww!eQW`WLy z)o~k9Jcd+^Z*~?!Bp_gSx>`UXi-&4?dV;n{|0YTt%Dsi)hVUOpUU_zfW>20hk)b8NuiarKRSqj%wD|mBjo}R1NDWy3n?OO z%X7$CM)nNo+IAfargk~9FJ2K;ygE6nKvJ$Zw3V-EMCEh7=lC{#K)yU2Ds*(F6T>y> zK>XW(f2T@)tT({@WR)k(Az)_aD8gt=tA*no*-e;AwnV>Ew+ zX4nAH%-%K_N_Xw~mF-t*#5Q-4j^Pm2`n+U$cZ~_ha`E|tHG4OlJrQ=!4lT~4IuDE;E zWimxFlK3xT$so+jt<8hLmNa)j6qE<62C;*h6ob4n2<0*fm4`7&l_-e2+A|5~B(SM)<%n6T8 zx86X_gD+VmXUTaGT067bo@v6E%%^dSx76w?(@I%4cNh|)skO)B1!t|*0Lr^{m*Y%h zG!rVHchaB>&iEZZf4TLS320rtHFv}Bgx1-&f)GXkS$w?+aMXpfhXON#G(q>TKD`BV%MQ$kQAA}uVo2RqeCm^XQbha-~4G?`}Ozc zbM$@p>jZ^eby{XryX1&sWrU{l#iB8T*cNG&B|mXHRfq32)`Y%{I&TF01%I0i@fit* z03^r~23KskdoH{k!bdJ2`SF}q{(qjJySwKD61mCz$`v6eQcCTX^9^Y9hE(#dwV>cm zxysb+f`n6y54eHTpwnA2A#e}lRQGso_*=(L>?IzbU+-_o8J7go++wrCXW!pPc>Yb| z4e8P#_JEDwYcf9Kk$+)nKFhxYI(hP|7=!^)LRceX%*cFr+_9{3g)1Aj5Cd6qf12>6JmOC!B*8Zz`uV!# zP|pc;1c8kW61IA$GLVkYT-F+!Ttnoah&zjDGe2=>XA~-(fVmCH0;W1Q1CfY+=$XBc zQ$5`BxWTjSyK5r|XAnQ^Y2n;bMWWN6A`&Xvyd2>6ZX#EeoaE*28qw=SIJ%ple6v)l zNgKczK@$O2P4iq72?qcbrH2*^#2wbn%};^jn?^2M9{i@dCQe=7?tcySryKhnemtxI zJn@}v+Ry!|OP}-BM`55ER`o*(7v;`=GPAX{O|IES&lN8?mxyiXpg)5|;Qu^g|J%u-yZhUbZz7B0D>h94O9T;efjz5byNb`p=lzyeS{G>M$T zxGy)5!g^E~IM`7Zq_{S;N+$dK9qlC80uh{|ui6*1A{GkiJUJROuN)QmVlw|mRezgj z%KTDIhRmYg;pWsD0%4H?S3z_e^DdKB;O zmLbY8f}p-vU#6!OI^8W2jF@CJjG{%K^&u_I_pWV+Sx+ppR#pytZ~uh<8vQ7Bww4~t zitz4ckDbj^t#@iiO+am_bNSru7BOKx&uA=EcI~JQ?dJ3htJC)`_${jmxzW9QC~3yA zjiD3lb6Fm&wI(Fa1=-yl(A-*P-`pm-luIt(IJ@?+JkUt)-u^#;c1;Bc?Om0r-VD=! zdX8j&Q%D#jyt4i6?MB9Ov%cYl3AaPtp>D1#cX!|}Vw}V-a;KGG%7Gmj7;KpIrZ??W3@T@QJ8!3+~mB^Q8vtQAYHjOpc~+~i97x)~`8 zir@a%7cBw5Tlkoh)$WauYqdFNAeW+SjK&p_`wr^BFF0*U6Fs}?GRY?e&D@c70N3@pi-1mtZI z*l+!65a45t0o$#fvaJ#53;0;yT;DHjJTYNpH^`6G=D$**(B|M9V={a8?Fj|^anz^$dwctaDVi9&q`dBxe@@6B^*Qkjqpd~C`jt46BjCu7q=V0>L4HWa^iGgcrKldl zL2uhsc&GhmMXc+ zR%_=Mc&ojtF3tu|V;t=>+UL9Y4Pas;RZaEbs~ci--cE|L{-o=8Y7!PT#>}r&HlL<| z>>ol>!s%vNT#d@uHG4-Z+beTf*XSc@@+jL9u`7#5ANpYRzGaDY?Q;wLgw>7OD|g^k zpc_Szd-=u}j=ndW1fx$*Bz})RFxR9)BH%J^_lIo~Nd1n!h@>hEMoTrSH|S_#m_1VM z%?`2TzPeO0pFs=0-$D;l{KRpuk)`8?FRv&~e=WD3gTXr!Q#|j3&)gqQ*k9zZ4gf9)1ctExL(kzPyvvbO6^Jy%0$NluC#HeG! z>4nR;cfk#x**pf=sVp$x4Sw!EZFiBlyJAFFG=RqMyWSmi`s&voje}&+<%{$y4Rh>N zKi99))=hzhX!L#93{@jvY^Ist+^HWL@N zt?kz@(H-s|3C-8!&02`9*!xkz@hKbmd?Lmrru=(vvJVzMbn8{yp1t`I-DWlz;1VM5 zYmXJ3=tSi(WKa}ivO3$q<6%ukx|H(;c2?&&P(r!Z7{xuI!Wdm}acVcaKTNFcbgOQQ zJwnTGG>C&>>0_%@ZllRvqtV#5I**U&cixWmt(iDymh`+co@5L-ACDii05W>kIGvo%vr=G{VL<7?eNQ5Eh*9Xao0A zbSTBBC-IV-LdorQ+5HfskCu%E#@E)29%*9GnmBxYGyW;e$1|Rb1bGmH=AMr&2oGCd z6^q9eB%DPgjc+QR!?GMM{zF z?hWI+Thk6i0`b%rt`Dni^VJ*ViRH_-8f!&~5zW(O+Nda&YQi#O_X-q&s{8Z^cM1=` zxEqfI!xPe6Hy$Z#I*6T+Ee_=^H6_bnXOzg@P%)NJ&wgF}LW;i0>~8ewuXfm>*xN75 z$*9z6U`U<$g|j7xD;c@_Gs%6A^(K_oo+^%5y$`1*nKjSqy<*q(xYOBr5;Jbvw{s_r zplr>a%no;pw>o89C*3(4i@s1L+Lb(nv*rE z&yR)aA<)Lw)BM*`jW&pyPJ(hx%Pi$f6I+cd?W79jt2XHf_;ix9EXpg9@lCxVMAF8K+^q9UE(Okole=^U5@dHWU+1;z}_>PX&TWW5kGa+zF5Ei z)^6A7bBE}c6*S-E67@qs-r(ww7C@^o#3QlX-;$LE_ zvlvELt1T4ygrN%Pbuj!5Lm1smPimo!Jly!q7vU?YY@JknP&U@qYp4N}oD(F7de%=# z(~LW!A7ZytyU@VQ;tVN}!VnP3>)4Fh=u0(}6@e#+wY7^hL9wC%blTc%o@j47^tTQh z7XO90?^STJ;s?G@(5W`wU`lV3j=}PY6c=FJSC?2-7>VObD0rQ7TW|3SDXdl6ea2~C zZRCrPoXz+_ORbq?QKHL4xh+mniFu1jSB%JT%liYNI12c?ae*cawXfvI!%0(}NoT_$*!iL-PL#rRPZd4zTliHoqKQ z|M4S#g!v5rkfJ#hP|6psCA%fvcQ>!vmhUhdyYsx7ay82cCw+K4w;EJ6+u6CZ$FhWS z&8c~7)_*RW#R}qOL`=<*uCzt{QVlk_Ykhf3jXOL$&<2<4x@)7Flcx{af9Gav?3~q@ zTE>a^LZsg}Z#-AkPdtoJokQHM%im0q#p<3sXkbhq7Z_q)5cBnt>qcWr{OY&Yx!wYr zk63uCueH35wul@n+a!0n79m2R7(wB3^RsW);Ket8;phpf*|?fFZM)xwj`|$f0Sr55 z`9bRlf3R1u1a!bg22m1gryswnJ@>9(%kAmrL?dOoe7@%&%hbY+W7ogj~qQc2+Vm0~GPbn7p z@wQa@YIW)q#vNsdsA{CvbpD0!*GqgB+f-^7K+}pvepy8hYTlz%$@4hSEc3{(5p(vF zh3IUiqGYPn^z$qeGa#1KJu>E};099O_hrZycoyU>7E~^3uki?dkeO$uW(CELCw|D^ zp7yMdvry<2H@!qn`5}MOP(zU*-SzQiBFPbReUh-KGb~vQIyz~4BRFmF(Y7yF4uc-U z#enRt<(;5ztK_1h1JOzwN$|x5E}A{tm;-x~AsFAuPIUuShjO@v!+Dh*Z`|(!`6m5# zD!%Nv>XVX-;anmxTA3;ol>)M^no)&EQ%ssS05j?IIkV`6 zMx{H&p$@xT4`+G|;3PnZhjqDj^L= z6$w_LOv61sAhmcu7yyi@ChbgDQ6q&3-*^4++r63B%|5v;%6@?XX}s+yNuR3!lSBSZ zF@}%|vOoi?z{k48_@4+cr%AKlmlDl`hCugJI!p??7O!@U*+q1~`{&eNEQ@_nT_@$4a^Y}j8!YmU`rkI&TkiMkt4#T1$J|CavF>&&&Vp=d#5yKSu$2j)t zkd45p*sx!MVht5XZXvQ|BV?)R?5P}x!>&l*CV$`q&6eGygOqwBLCn_p>H$lJg({>a zpKy!o#SCVjU}814TuM23qzze>i7biuEZ@ka%j7D)q;xPh0l_iU%XfKACf!=l`BK@c zsaB1LS(lERTdFBfB7xxo9EzaqB_#}HP1m;RH|^Ehrg3x-h-El6yCY*077Kqm@I#E> zNe|wxd3}_8Sy!*Xs6L#Z?%Xzy%WfrDuh)h|T*V-MzefH6kOgTE;@K)+EUfzcx26p@3! zPx_S4Rtb+v=H2&FOCu?%4N_K9h{6%>Yvo}s0qJ2rGjG3q8H?}VGJujM9n1TX$RKSaKb!=IKYiPfvCv6;s5Gxk<7jai`Zz#1L|wlt`voXYta^RZuc}A(eS|hfUVsMOTfciH=Bk|X<%%ue^vYa zRcO|X*A`2YVN(u3brrl_Ge=urg8d8Q=VSd>Af;~gA{A=qK;wA+Z7H2oaP)LZg&Idn z;%m|~6h%<*n1-8h+0}t$B;bOySW?7Ic$XZ@05v1lPElm_sXGi|bM>}rrWrAD16g_H zlFUR*pLv+<_`uid)%l3Ng|17HwCt4r{FpYMXcM3vWhIRzEQhA2jrkyxDJNX9Ktnk9 zGR_NM!vHv+7hRK9SKO0(G=X@p))5b#FMA@U!V$NtIXGOQis4{luCSP#Ygw@UEpY)L zyImjz0t2J2-AsChK-ZYJVeMBWf?a=@df@OzJDbem#xMTCWcISK*6eGN`)-uF%n0;0uG>*}RbdpJL^1HY**IO!~*xEMo^7TSW^}`cat+WTt z3QEm45L)iuMb6ITT>51T9~0-=36|X#Tbhg+sj@iLObU`{6mSvgvM8cj>j}V=s8Dg< z@TP&|Zk-RzZM;3q?r~m(6Cs+~KqbwuswabRc@}MDoZkNjhW~J!_Lz{33k6Q`-45LU z`ZYEB0=&CBt{p5iGYbFl5~;rVoLjw^viI!`M6Yi=*11wcoxG4UZj3xqfSg~3h5d9EIR$t>OnB!pW{&)cF71}qT?)lXK=xd4H2xl zX7mvUUo}vvruADlK8tMPD?>*E5&Xr&kM0y}b}1DYr|P_?kM`U|!QoP2;%+9{V`<5G zL?aaB`^Nj{@j|g!XzZzSh}DU(Sxa%HayuDbj8X_u1uj|kt;U0Eko;2uk0zqHi`2bM zVO(=8_MzV9#eXV2I8t>#%1#OQq)NrcsQ6eo@~$wD%m(wL`i(ny^GhX*4<*hlgGh9a z?F59bGR9~W6pJtgSs12Fy18Wq1Vg-e*6gAb%##@C^S6B7Qoa3LM(AiN+zdWm-Jz7Gp*vl4v6ja@FQgZFLPXI{G*dTMJ-H=pJh1ppR-BU`TiFE z(Aj^zM<{+M^PZ6^k$h2o9yYb3T;hNi65cZWqe86}es-830{U$0>U3ml8ptz0biSfO zk+c#5%F~brzsLp;GA7;5Xva%_Uh#nvu4abgJ%@K2?N8!zjs7ga4Sg4cLNmA}oLl*& zUgtEZT<`*yT!0%Htlf;w=$w`>e#sfr(U$W&)+sn(SGTb7y}gG)SwsWRc1T_7w~GFX zr;!hQurdqy@MDzL-rK&Bdqqgdm?M!^_`H0@e*Zy_{Zwm>ww`Sv>>uXg-(i<-%Ckbt zU;> z@uNySP$iF-7n+TWr0tgpi8P>Sl=(GS+CVTct=|%wH5(d zs#yna84b&I4NtU*yVYqGDEI71Np(<<>MSahQX9_TF+hQyi*U~n9GyJImcYcDekyA= zM-}}An^FBV|LTOC{1+FJT-0sc>5hJy z{;){IFyu8z|1>iHc6QBNz(lC^>fYgR{lV{E55s=1$Xr3&yfc|^* zXW{KFzc*hl;vC_y`i8ZFe$EungVK-#o|`+P7J+83h|vJvt{LmYC=smfy-8)Q0F#DW z#);b0BGphLF^I^lCQ?)1iaey4Y8imPk8pL7`_Y}D)4DTDPoeJr{f8eM0_Nm`o{#lO z;tc+DNu1E2^=H$SzPtg==4RUk%1>Zcw$Z9R(H6%u;#9Tas_(ZoTo_?E=eTsyQA1DY zvRlKDQe9$g)!vPJ5=h+#fNYF^CIsuH5oPzzew3(OY{p-h2~3)KMAzIAgn2<|jO1)I z_G)FZ#fkAfzAf1L`OL$!OWK@p!d@TSX+X&<;u1LY+-1(OLe7~})$gWD&nF~jj#++X zg2*XSN+wA05Re=2xANL$CBvhVsF{ny7zX8K&`S#xWmy6DyUlF^D>r&lUQ)dDmqIjL zft1cZeJX>R&t1=g7mf`M%VFNKEQmQ|vTtW2vkHV{rw%1)0_;>3|;=lUMvFVj*)`FGq_&m?A z1To(;Qq=3iy4gh~UY2#&S;`fma|gS6KD3_|t4fskH;q@TW$1T#gp`hWfp;GAy}4&6=JEs~(*oirgJ<#+%_$_4QZ&gMR;2 z$35#9p#Sp|CF-^K*OHM*4rfAWD%bW`_Gxz+G##SqaX8a3c=t`RSFgNj;Dvo;`8kp6M}c-XBF0i| zN4mJiLqym20UJ?6JFHtx_7a~*Z|{C#$C4B=-p_o9%CT{{l+;n$N&oBF*T^BSaCua= zB>7G)>3mfQJ9z^r)fKTT_>#xpe3UG;l~PA;AXLv;D#Ns1?5_dc(%%Td#LDK`M}F&g zm0;FF5IhzGnzr2-L|zM??S^VbB##hnk2kC%15FVzeF;8>W`PAI3@R^jPcO<#HOM{8 z=&a$R2xn7;;)8_BdT1Y_<(*iByA9+DMyf5Q#T#Amc=R)4?13xE1$QVF;@jiR6PCyH z+5_)#Ql+;h+iJ7aHkgie3-#fM`D{l_?)gEODJlu&WSBJK@!|-=#x+eRLTfW=P1)h5 zfb?ira=-D(^_{d_jL*Rv$+h_g7pVJA`zRNbMeR!v$b;fI z@D}pB7HN3$qEX?NFt)uqh=_=y5yxMvy}vS@k;8dxf6F8g(!nJcxo>X006Ow344&CC z4b;w}F@Vm#o!=LbJdtZ9CtpCj)!V<>I?xP?jV%SYmM#G9NwKE5bcu9LG_i2FVjupX zhjP9H_$Ei>C$T@()So&M$lm0b&m$PBOm9-Ye^Apf(O>RSdhN&FB_wEB^eHPUr3oV} zewQXOUXS}?!)5R=EdQg@qgxk+>~wMMh=k=6Fb~Z((mFEMC@p+jZEmsoYV+gftEptu zUt2?F0_}Iqqgg@$CU;G8NN>MVkEK+x;Mx_P2RwVIvXxxV&Q3e2bUX*ldat7Y+nSBF zqclzTs~d_;f%Xur5LYcmuH3ZpwEBaBhKL{Qg{UVV6VA)AP-blxNj(I@Ddca-;@E6p zspJ!Xc3sni>1Vz6`G910f4Wm*65N1(s(U9A*Z{|@&3M)3{D_>Y&SZ`whWRV8OoQwf zQj+i_c1bLcg=0??(JIR-bEXhiWXfZPKqFaa$QNAFIJ?9}?an)7pdF+kv&O_AfR2vy z@D(dc>*SeNClk;a)l6m7Hm%hH`Tw!q;y@qGWHmK-JnWgSmq1)i#mjr957B zmCUFWLz*j?6L4jAp4!5vup0!(W+mG50Oe~^CZtrW`kt9wkh+JSJWJ^X|5ax!Vp(B{ zc~p2pyyh{q&6`&VoX#J>tV@r_il(j<+6)CRihlO&Ff98=Lk(MaeBDWBE2r_Sx1m~n zOBF+YhMa=A4~jAqpE`$}H+w>a&iCQ=)g z+!&QJgO*4(NU4_L)ef0c8OR=i*JdK|JqU5+`nrH-ZiM{G8<>sSuC&BN@uOMD*kC(Z z?x?tL&fL1Q+h&%k;a0357ClY+VvryddNx=ErZ4!h!7J|>Oj4xk8DbI<7(e ziVq+2XvT~luAov0#`)Sky|Dn+yMPxCMrEdtDTw-Q0;LqL8$i0D{&F&v3V$bqd`>2) z%^`s1{*h)3>J%7iF10+tr@4{ABD7Snmg={d{Nm^dRd5q|YsO$-9|KlCU`PPy2~rPV zuJRJ-vkijT+p;;kv(rKLC9fOkHq^{&ZDAn`X}1eYX15yumNVp(8#qY?y=<3(r{0M= zs_%*faj(R*-@Qp9tTE~3Xip*)@C=X8v|XFcfYWHoNdyCopIuhwdY=d#vE4g}g^nNN z4|??A*3AVN`o5u@|74Lr#Wy;_Z+)l&A6m*kg*p0*HOJSOn_~uCM$`CN1|LcE00Qsq zE;UFLsxbj8Y^~IzRjy!6X8`G(o8(itk(r@%)BE`bqZW?vJ{zxGk0M|4xC9L^K<{9M zI|6PW{K`=<@4e19X?icCYHc1{0W?CE)(bA=i0+B?w5=!rN#`qdZTxkTH&sH!g00!) z{OlmOG-Y;i4=ZFmN1s66qPW)Z>+G^plY|tNJd>~0UD8-qwaJ5u5~nhwsMNZkS;p1Q zY(Lt`M7A~4h`QvErk#J`kr(9wv&Y9Bf9M!EH=CYD&ItRh7a0+W+}8}e)y5~hJ)sz? zLOgH8#qoF~uYRFq?DJyze*LnS$5EAF&3m5XctlOBl>@6%RCGSQf3alBbP}Hpp(ZlMsdXRBl+@Y>d>sf(9;5_r;gKaRJ>9F4pe?;t}5ej__wO0`Xpi2YZ;eCFEO?lycr$e*o0l`5(vyt%DDgz~3@_?DWtIHS5I zy)oAak)w~Hy60xCVzZ?JdgCM?s9I4-9o$d`J{#7hiX?)O1eE2Mup+U;s;7tVtz%J@=mq(M@5N+&~C;B-(iyXKP1wvmrX|7LlHW+trV>H;P zAOW5G=j~!_mcQFsRi@JQg(_7iB?)zcZVCjUq7eg|w`khdYF*Lz(-jGcN@BySnzk>@ zEa;*uMx^b(*ZOkaC+nAx*E=et9yENvGMU(~NTt7Ru+t!N>U)KQoJa(RM4X|xCY{?C zpr@0YZw4VAW%y&G6kn11@?<-ZzxH!>uZ7-ujX~Ve(gIi+}D+eHWQb zy+AQ=7f_WxY8VzZ3qPqRnazSz&#BwyB4fjy3I|01fV9(XWo?2O#P8a>?u6jO5bJ}M zuiY9Z*uouK>X7&%Q&j(x9RKj*!c>5kx?h^>hk))sY>aTS|ImZ9QnqxQK`yjZ2fpiv zL6w5uQ-ZP&a5hN&LwKXP76p(hB_*CD>NO@=M``BZJGVk7Y7+GE5^AVh_(@ZbHVdkP7FTGWD88hX75K-T%pPzpusdi}b zo`AKix<0kgaH7GOKP6qX-67-gWwPY7C^MaZ>J@hmFnO0{kh<&DWS9M3nJ?L}px2#Z*b719$JFKlWd~FEvXobW>~d zL?o`zs$#<-mJu#>7Mo)RJ=Pen^g-@cq1$N&Y!txU^fKZDal*P@8000Rop_s!JFy%g z7ZOVq=Rxs7s3U?$>zPSeGTl9UWlQy9xz@b4*D`9Ipew?Nj%(k#c%R0kpKB~cGpC)6 z;)nMoE|2TA?T6zP6cijJ(<5N`zsqL+V!LYkO8@qzs!AOY9aKGKJ!OwqjhMFCo@Q|t zl1ZBrFcRW$&A#WhTb0_P)+_iZOV#Rgbq&!INx|Iw0B8?>YMSTD=*o(YPCW3x@Jao= z2#f5$NXFwzT{t>*MYOOQGKYX*G2#*n2ns3f%k^`hVVy0)M1Sp42DOjq)F2$)PWEef z@YmSH>)6X-O+1rseG(^X(SP0kuS|t0#s7zD^=s239 zr+bpFuOnMA4};Tr7s0l7ZL`B|GUY=VdgBg<>EZw6kU#ka47`y6;@6aQ95gc8r+
`jFAW(6uRXZRCv5zA?PI1LEVNrI`tcW7o)5spwN-jay#09=CP2 z(bmEj`>AHZYPW%rKN2g$BRPEr2w)sL>LrEh z4>@b+MT)PW);9lW0kU19j3+(#(C{O0GZIE;!ipxfC+gzd(1MzkBW?!k-P9%=zpOD$ z28xx^E}E|KoOK7glYVbP5!BI0iq2zod$P&xMa;+zvs-Nrgg83bxHY9~eql?+x8l3c zW!x8!MIT~A@dD7C`gVabg;>YIS4G=zLoXDE=x=_K?Bs+&1 zmRjmdPLX6h|8W%#wN7d!({Kyb9el|7Y+nP7keUCRJG@r?HrqBt1(6p1UoY{ z6kKwvy44X(0w)_$7s`Lu)CY(;i6NytJ7>$)zNMCM^h~ZQ-tyB~nZUV90<+I|UnB4N zFUTTZC)#`-Wo>ukN8a0GBrzjAJ?xcpCgp%QXbgKgE}AU{9NkT^u|EXQ=)@Uf2DK`= zqap{gttQ0su{4WjBj1U|lO) zA?)zx^teKaK^=$RH6dKWoEs3QOveBt9CCvn>x=xxm@U40){$iMGIR!UlL@DHjz=N= zOzCG5W&3Kw;-l?e$RcekvOB*ve}1@p{JF(W_<8VS8$iU-P~@y;z9yrP#pW}ie-AO1 znJrl}{Kbli!wQ%=2t!JB9Gw@%>~820Q3)m1bmqO!)*l*Wqmb;xP+uE3zw=2|BwY>} zjoKB0;?4Uuu3zyxA{F$XQ7gEYe zbm_zU*=p$~>A}mkz`PEaXhy@@t~7G_m-(Y6M|o>cltbF(iE1$1m>*(UKFCK42Qu`2 z>{T25MBQESNe6>F`U{O=~*ZfWvS8gO)KxiLjv(7I?pz+SK3i#M);x8 zQ>(-Ep~kl@-PZ5RTIKFkP0BV?WL5+AgVa<9jANntTl2~UKx0MWgwYc5kUCcu_;PB~ z`6(niOESfiBp>$8%!?-Gj9|>n3k(T^wx`{aoYeft@gz@s+f~`khmS(hmh6CkX{`njb9!0i<<5S`8n!<&T zjs1Zsh$ayKt;rQOkI=mp%ZdO?D^W6ZG-F-U$B`?y3L_JW*{xBg`B<2u_y6jk|FBR2 z@XvF73}<7HxBoi#%x?sbdC_}E!85E*2X@RF2b*R7JH?v&X3zqDbONZD3&#)V7{>i!^u$=R zthtB)SO);5e6D_pL4MDddM?7IqcDI7+@rQdZd-XDMHJf0zAMnlX#TN2Qh*Qy*Qq-U( zQT+(0wT^JI<;)E$)7QfvRV_|6o?W?>9;93#gABUZsz(H=d}y{hw0bL2B8})(e_#nI zwGPzpR7Sfd7}az_mM3Cuh`@YaxNFUDV?S4BDUIC2)xjwBB`n(){%R!@>I88$(l_4A4+WAZ_zz00h#fTI4EG1is-d`yrO9<+no~?-)y8>CACv#uu&Q4X%vq6dy{TN? zB}nG+N-3;SiUsglky%j;T!AC$&%q0}%aF z_TS1R*`Ey}=`IeO7drP2nuyH1Y@uH8o4VcRIIljRmB9PRJ?5E^AivjA09z{EEB<~5 z?PN8_9ha_={QVW?XMtuXq?w3u4;j|)K=>%Rr;m{l<4lqGVK~k(&vWm+d+oK??)Eu?>_|Kdy{Yiz2yt~mQqbRzx6NT!HrEp@LAnUn33XFx zI&T4o1NB{{cVJp7y~1Ev*T8z04&2ZHS^_5dn(hQreyeyJipWciVJsrMLskjnT4fR3 zBkT&iZpRIP$4L#Xp3JmS`F7~LfYToQI^JYv`WU(XM*%%HT$Sa|Vpr3(cJ?fdLUe!^ z?i{U;m4x4B2-|*mPT9Z3E7cGm&%sx<&mTy@xK@R}t;QJS@F8TsEDAqabb(o~ncSwf zv&NqBH2Fyo*<_jXOQ3VeU<#e(8KH6h0AmU(G@$pz>9q+N{Uz(?o+Bf(HoYhkYDzqP zp*;_v3~sL#?39r1c*&as*>w}GK7!^CE^s5+lC+Kf&-p)w{`^QU)q%XV6|&U0#*`fW z9>j8+l-}&5*`EyE<(bfTcpBJqGLNiWR_!(WMWavXEX)?jAg?JFga37~|2SWug?IRo zqDdQK2@CrdK>v_}thrMj9x_y6f!TQWMkZT|1msz`7)M~>`Bkm#{#&E+YF#wGckfEDFu>*%V+I_p9Ds=Tg28;h>1J+L zuWnMXC4x{cOfM{`wlFcF%--gUROlwC$>*c2EVr1SM~8LF@c0xIsFTJ!a{JBgIHi@< z4!WpC6!%+XbiVjL$;#nrx#<-1|Z5Rj|MR=M9m zsaMPHt%y5gC25Vf1rHmh>C=WI463*ly+3(A`BCW8frEN+72b4yVOEyL;wGNg3dssw znvSx?0lZ?0-F4#&)#N~>XbpKwEpKvIJIUho;ZaN2-bv^lJE z_vnjHm;Lb|u1JuWsr48m4a*PGqO`!&G_h7m429{woU(UE1@Ia1ye?}T%bl@7sZZY} z!#%Q)`3JJO?miB>+-_LwGMBP6O>*TZn}TY$q+(`D$qNUR$w0Lr@S`$fDT+hE$B9mCypia^wZ3oVh(yeZ0w_Fa#)iDNk0RMR>QSvpiKNBtkoc_6zXw0aKpY-H zuMrhU%D8MEI{ZONpRbydh!QF__XJz>l)_Q9=U);G5_@8nF~aRuLuBzK)ZA(U8#dJ50#|j2y;2Ue+HCJW6o~{Q92$6F`-XauBdT|6Y1pDJbIPY@Pzh8A<59pEi5! zSw@|WG?CE#l<@y*2`euSPt=9FAJ{~EmaF+bRDRsB61+FAW8m#Hv@25glh*sL=jiy% zBO={8_pw#-tMtoQOy#8=x<7WzuuCjKG4!4Kg zrIj8-d(Y1R*igeAG5Gyo%I^Hzh2}?WyK%h5`Oui&_Z>d2);a00PMzNri9m&!88~>L zc}RHrHagQg>Vz8Qv(rkv(w>=vZml_2ygkm18>uBB0C<~R?XWI}yy?elDC(=1>@eQ4 zJ(~4<&)Vky!`Uu6KlVx75`j_wD|l^CKJwwtSVzaNS%QBC^NaNhtrLywp4-g9oS7>l z`Z?L)$i2bNK2OR{gR)G6Ik(dJ;zdk|P%E!dhTPFNb>PH{(GY$p+D=3To^@D%4isVO zM?~{K-MXQCH~M1N`l=3vAk2h#&AKS-f}nBZU4OD+v-HN$cMaprld@j^dGP7W$F{&- za45-a!2UpR#hl4qt`e1MK}B^Bg0_cV6C!0(bfrJ_FEjwH)}Is%jMzbB_EU;}ZwT-p zt@ke6RR!AtK^t;XnRWn(J(J#Gb&$YaVXa-NX)X?n{V4=B`39gH;DBt5mvHybpdBVL z(wEPOl%bt7UNTn9gfC_#y1_&I{5xtx~NplXgQpz>2S+yjsTJF0;7V*@ZyXC;%1{6yeqLi8w*UZtgf zp{C7k{7jQ098+|AL<3iDq+kb)Vc~m*7OG$lZY&C!B)LW9?s5snJ6a}ft3Gt-E3NyID$q{!XD@|~rs`81zJTW^t@tHMz0v<4s zx4p(TFn)vxs8Ac=iJ=;33ouLQhFSjQl%5IRZr}d}YJm;|Pe43n1&UVwZ;}W4XU`pY zBL2ormN-SpGOw>VWLBLiL_B1~+tDpRRmJ-ze4ZbX^GX~0+*DkD>^3*;4DS za0VZ3Y!(Z=zGvFRWMGCn3oDtp>Xvvigf91AQoum^-G#gzA^Tl%ACh(}@J8okxyJ1e zBvn>Quk^;?yrby;2t1Wxg?5SWqHh>Kj8~c)Up}8^j2v7nUtfg1jgsAGi3HoRg}!~b z#|)wOCy}rmDZ;Ho{YE)wFM`Q^d<$*Andf+9`6!Z!4w_otbMO9EToP9#1K&pdZOWQ%vD$p3v&X~uV6==Bc&*=x3{$I?=gQLJ+jpJjjWhS;ueHf{b?+DYMGO2sEK?vVW5YP0OhSdk(;wlUv? zc(c*@iNv+i=Ch=5T_18g!huK8d@FNi`$xgr5>(1omGZC@0SeI~yH=wtZ6~<{p@KNp z0z99pANWDzX*-Jntt~S#a<9kivO0aaSVvZczVNso`8iuY!yGskcy9CF{v z&(gcrko9h5v_8jc#G7)8!__7JOOotz^>xPgKh1Kq6PPlqQ+iZR?Ysd=Kr~+=3}OcQ z0|L-0<9CQWoq9)`!V3$XEQr0YJCiItTopTlFSmp>5=!Ax*$Gqyy_csi@Vz+aJe1Gw zj;}BSYr1fHMdcoa*`2-ofam1pycv5Edym0On6J7^903!`*SzqSv=CpDH!bFIicMk- z6c^XgMu?g92ls>dCb1sNfDENq^^l{f6*#=#=L2g z2Ibfps8f_0J49FI7BB1BKcskjM@Ut7DookRf33=~Qf_%s>+Gju6xAWW9fphW+Wv{U zdS+buH1b|NFpF-Bx_XBWGffmti_o(+p?H@Oj8xscqx&Awuw9X^ovxn23p0Bt9bkHu zyvov;a#zuRUY+X29uBF-OP$0p$s$D2Qs28RC<#axH$NPrAGs^hspS)GJ@Z;Q@jzMj zYtQhiZtPKGkH$s;+Lb}y_{Q!lXu}R5byJQym_LK;ujn6}DS>50oP)yf@@I&%?^0M- zo;tj4mK(4VoI+r1aK0jt^uWQjak82Njf$Vg?9=~}`OQ`?JE_pgI(?e#dMC2NKh25J zl*3+e@$x`Euk8Bx7#)}`Cpx=lTD;I8oMuUtNl=juIqLJfsd1?BHGi-+zTm907s-3r z{WaU|uiXDnel;2D=_X%4Wlfa$U;SGj+67agF1jgdK4kOTP}}GZx@VNZhfu8&e0Yqz z@1|@cwJudTKvPyVViz`@E=sxD0Vd!`H>eJQLUGh!s(X65`OqFCI2e;z0ObMGGiNwQ z-P^6eB|^p-?1#XqUiD}8T@0}>Uo!tkf2pfgn1RroV@L4D(^Ctrw`SUu-NZ*XcB%_N z$o?!+kv8shD)ZA~Sn{WiQ}da$snM;_(fP&g6`JJN?8t3#5e&d{x~E)^A!ZpJD;&+- z)c=ALH&TU!jfs92W&wLUF@Q$_0)N6=<{!pP)F|*3!!}h-8t;z|~FuI1n+1Z*b>jE=q(BJcx0L;~bKr)L(ueG6}_gXc-4OHpEp1o8bhZD4WG|bo2($pPVr2Nr~g)$Y=RJUgni#EGlTUU6CwtpoMx}&AuIN| zdw3M30%HNYp4#IE_qu$4`7U~Mr_|@{%})6Ii+!BiU3zfhqwG(km*jQS=JoEuz1teK zT}%=4rP}qrquCt^jcSFb3KB0b2|y`BD;A2PhIcFe1UxN~U=97XV-)6MV5n%%L0gx+ ze?o2?IIu(6FAY#Hx867U%6CN7p2lIbxgzojyE@#=3nw;bsNqf298fyS#Z%74bbH7F zHHfakCtW^I6Zy#x$Pw&@l+XINDCsT+t9*xz=Co*}m zTw>)y?H6?Tlpp!R7hLHM^YXr6;XNwF(V8>P;2>QMsa1C_C1`?*HZf<@S zH?3}rE=hBQ+o)_y5hZ`l;{UI2_|65@wTH9Kk1^Qwp9OnK$Uy+7NdZdNdz)Q69k$83 zsU5u{Qx*T!HHL_^}f<<~hST9PW;0;h-GMGeqTn zI(v%8rnT3gdx=%vh1#07(DiJMIy>)7Q-AYkN9+CI=HwqHDaYbhx?a46ubHYam-sF} z{(UYt{_jv7tGfohb$A+~4uv89fSl>B)Rn|`N4O;^FC5&G=CXD^G4gMy@gpfXUyYiW3tY{0je{W+B1p%2C7q6$F+b8|E|;{ zu{wzcp^Bb3f&QN0BJnQg-6?F5Z?A=e@O9BR|24FHX75<1Jh>#DtjT^*)2q_7+U;n2 z&E#INTM2}T3ACI|X`P{fF?wea*`~Os#HFpeh{vDtCbu?0M}kT;vr49;pa)a|CB$V% z3$H1gJvk5E^=N+d$(~v&Zs$%mvrT^eqOGdAT=fN9%<^q9Oi!haQ5!} zrH#kLeS3}Q|4~l`BDGM+I>! zO)7gCbeE%p)Kb9$dg|;dyTSa^sQb#QlIHSPz$nqQccJZT&loX*8L00oOW@!?Yeq19kop zo?UIh*`;0^022-qQ(0nZ!SDX#`bn43oG}icwc%bU5h)z_wk;-W7A~gj;cGYEpk@`5 z5ErGsyrNHc?4ge5DKT8&yVO;A(2d>pW2cEq=Pont5AZ(1pb)FTbG|rYOm21wWP9s2 z(AgP?j+owHN-D>20++Jj_UQ{5V%^GUtbCqP_cvM+y9X7JvGpZC0WOVo@Tsy!|;wyvw?RKTcHq$5TC~#3>S%ZIUeS?!J95YG-gNWTs;0n z51Bf2{I5!63Zxcm%HR({dE%OU&bL_RCqDT~Hk{S@K}GgZ8)=fzPrfWVokQakPx*>k zR7r-l0`Zb@nRV;+H<*k*J^TxjfW6V4XGDVT*TLNT@12loF0PkST*K)^l`Q;LOD<1h zi^$|P);4ODva>=qeXOlE2VuIt6>PfhMAGJ{HPxv2{qn{u{ljj?EdnMM?D``Zcyz29)dzW9p~12;Dwe0dCH#wlhW`f6SfDEobRV(ajh`{=Ct zq0|ZIha#~Rp8J=|OtU~O8Z7t@GB23V_4T|kPJYW6vxSs3yd86qLZs?mF^felN#s)x z{UfJCV%`07^$D`h8$>7@;X~`+rsh~&7p{or$e8*UNuetT%WtX18H#ncepzhDNwz*b^cmUIH zki}8=S}UcGtCjqUrl+C1074g1`3^R}^<;Abg)ru$)S3!9xBK-ex)1TPWCt=Q585FT zJSLUmaVgabJ?>!Y>n;EI&)$UWY+fh=2cf}FnPCNC_jW8V ziiwN9_@GAe^Wfc0-uF+pD}ygnEvLBGCmqbSq1@q|Bz;O8ywaVyvz6zTsXz22%)@^i z?Q?Pthki}*&m($0ga2d!-E~1S-(u7Q;&Uf^QKc}&Gq;Vx9{9vq3=&cvSL$96oLeE= z_Ofs>CcFRbp5UijI7~H^jydt_jFnzI1Ov0?&U7^9{fYWT;_DAwK!Siwv*wEy;x7A) zfJk)pVBsD6awq)5!=7R^7Z1>Pr1jw0re5I|GixYn7|r5N-r#ccGB}#?8BzILoaLu( zgH9=Yu3qn&W7G^e-Bd{I%~kb!{5r>}NWFtmNd#L8c~QXtUYG?&496BPbY|)JpEH;5v)m=}e#lteqSB%nzt>9W+Nfe`M*gYGw!vgkF&tlP4?LnO zOz5)AOEt|GTNnxBx!zQx;u14jttD^LnI;{6$kSDDz=nE4;a2$=n1k*8Htz>tLsJU< zKOcP~nA0oH_D<>!I|)yiKc7V{%EjGf95HNe*CeN?^f)dDO7|2Nk=)R9E2BY7zl~jxiJ=P%nF9g1_)_U|KKqhF50gnph4niyhnOY$paOVv#`8V~@yaf(vShY2~O!A^K7 z0d)Kcu!XGvx#G@{#5sDHN~g22$K^&C_t1O5JgJt8#fTTHa6FT)hNN$JO?ue5;(dir;yYTo zfUjVv*AO7%DNHGIkE+Fu;ua1&i>8Pn2QG_w|9{K-fK)~+(diB67r#sZWKojZdMzUA>ejXS=rDEZ%7S*Z{=&H2`d{&ep#52$}TN%MtaR3gS<6X zhn!y^?fcFX$7^mIlVdwA#%bWO8Kp)QI>3fPe5?$Z^FP>6Y@I*L#UJ(m$7cRD9^eO| z7!Yh|S%#VTf9A$12T|kYw$>|c)sal$WH9giWKD4 zY=aBBleJFUo<%GpwSI7LTqSI1A27|ZD)Bw4vlypex4Sj1*UA{8N2o2dANM# zsE?WFU%?IOneTvN1lERfR0~q=H@X*ZNd(>6dmB$|Pi;<^_IqQN-m(`@LP=tDKZM>c z4}9pKd5Qh@K=)-(BN&JH`Hb2QZHFM;aVLyd`r_^8m9XWhu9qNlkeuJKu)m&`+P0;3 z%9`u?(&5ydL89rodZ2o#VISOB&xp!mjvk!2qHgc(T2$TjndJoDSKsp=<|ard9MErm z*sb3hC*JRnGwIfP?WAF@RBGX1vWQg?tRtW}?FxRDlC2YqS#M5BNRa1Id?pMWw})QY zbK>hh9WZt8)=VjWCvv1|rg2fsx%ss5vzS6udaP=DU5n0x={l&akln(lNNDjLcp~{d z-B3WQN?RCK+ZltwJ@1C*4=_xUvDJ2=a3X%)IYi~Sg`(MBD;ZKHMUV($qD&m0|E}S` z_mNVc0)_@$5#ghc|JU!BpiFVA!$8a4!E7JSpxOhi?)<=S#=)n62-R9e*W85=RZr|X zxS8ha2e%zGiJ&{AhXT%4ze|iruz{qy=-Wey?yANbBjA0tooGP#9z9HY-%(=d3^VP< z2q8x2Qn>}D@IV$^cL#OhKr z%JObBrt{gkv3JwAQ4`UY-J(vxmN|TWE_ry0@SjRwlv?yyQE0*aYFl`_yQjPh4!P49 z;_}H~nK-EJW41QyZQ1d(C zcep$E3lJq}@uQE27Vxzvs-+JoMHeErvA2rX$C%KjWcD%m<5=7TNGZ0quBkta-R$he zwZCba3;*Yx{<1(F=;>zF3hShPvA@1XgMYM3r)9g#aFbEB-Pg0u>-^m&OwMD2cH|4I zq6vxxmZ~3=M<4W{H->;kvzg6CLopoAH`mbvcBNGi+rCmlV0dRs9v=f0n6JzOu8uRh zdARx)5#$h|x>4j*hX4Go)H!B01mdz$rPgo>kI{N#*p6!K`&L{Ao!ic>xxm`bB{Sr+ z{kK%(=I(aY%2#&!FX}jRY1T}3pu56I?q64n84C}sH8!UVp|(8WslE~al4RhV9T4It z>6L{J9}fEU5wd@QhU!@}ob?i9s~*Xb2p{mUyMQCx2xa+%RE6BeA-RTIAER2(U+>)g zX*iA$4>0Cg8$>!iqz47Y_B4BS@jCmq6omDR^??C|;p@-1?oJGm1&KpFQ%%f~3_~Xo zVgz-7u);juz{x@&KM>K=lBW+WcmF67`U1_BS z4?R<|7k1Af#p8(@p17@v2PXMoJjc>A-i5*ichI=yJOzK@Y^Vmd*pSqt?TFI0BQUh!ie$)>>YwroRGU!5dFtA&I^T#y_wY z3ZsPY(SkNW7=Hci<~@$LXo6mAUp=VW`f!I7==>U$kgZ&ig=dm212bS4iB^I2@5oZm4!#4cw9m68a;pY_X9__MIPaI&dMeRn?+x!)`N% zD|XCU46+mPD!j)Xzacs2dv$75Cb-1tX=QWbmje_#e9q#AWehBz0!s2nvD`EwCiBOzg+c7wib4<9WOqg4iq03E!YwuyL?kkQcTM*^-MHn|2b5>71H8iL9hS7??H1-XGX~(ROeoX}j8H_Vo|CRN) z9UMA+k^4ryo49P3_mp&0*4cU<^YVkfU2$@xU`R8XdZl@FgquCOmj-aSz}x}t(>eXK zLPJ`w^uSF?8JE6g+8m|vf$ftCwp0XZ{u50$wGO4C$tKG>B`Kh}ld9xGjJrPoSwY zbB^lG#98a`+@%LR%75qgSAIrF0f9@kw0y0O^3S~j!JAP5fSKCbR~~!BW49qhB5L(T zA(_JoeVp3fgxrJ@Hmh6inT_SRRck7=PAwSaV4;WK2A9{7hD(G zyEtr?0}kWiDqA9oJJ<0C0pBCAtA2;liLyzfqd2k9nX1v%^ym^Nvt=2HCQFBLq(>{W zjJO2J9f^(@@@M^Np~Z_1V$X_HSoon279uoncWaN0rUt(%RmHGle3l{WkHW8F0)GN_ z1zlI?;=9lR3v51-H=xYNzY%w|jhZs9Ap1KR^^$`d>P=DZ=~4?xX><;de%q$Ce@H(VClE6A`5UB5 zez<7?&vMolqE_my6$>OMiet1YqETEfSf_En{Z_i`63j0t{iDn>mdjgr=KmKprw zpSKEVAW)^~8x2>M+RX6OxSqsQd+;X#y;-04c48&{G9bfPs$7hSj|8R!{Rz? zqg2&u)}3*cGyXmL@!9Jp=zK@JK3SHs>savE?GZ1#y;z_k%X&ss+oV?&(F+UpF)>j8 zo}{U6Uy!lITD#Vhr-L|C8_0w%H(PX&_GVd5wTrH9uHJ=Oejdl*DGM@sN8gh2^>;M* z^=~^1^m2(pj`W*qvumX9w4SI0>oZS1-*UTKQCvD9)2?bmH=wP5;!^GbcpZ11S`dbs ztJ|$&Rnsk%2Kj1Y>60H#8dd?DRW>g>KEMK_)){onoG!%TUN&S`xS6$Cxw;LPtOzx@ zFgjf&t&2o(Q_G-{6v3Y8PrMe~!ouQ?Pg6qY4rDS(OZc{1vsnIo>LqV?{7mAUUl_?z z(wCrtu}r7eqKU^4r_oQDB|zzF5wF2Og@^mT=^a1MDG5d6gupCcV2$k>vt$%Xfqu=+ z55*{(NsZpNctFNsbFet=MUc8vx4s!FG31m;)OV z@#_Ybv&1)mF$1IXsjX~+eA9#zH?eR>{SKnJF7M&4W@3#V(_cgww|k@wq+`^qNhvsS zvI-k*f##pmqvJrgjJ39x_yLRIVL8BT+A-2WscUX_GleQ?h5QfbW(^X;LtktjrH2)& z6iu#Clt|1qL55Q#09_OYFhTm%gL>N-QMOKW_%AoH+R)^P z1TJzhwSE7`ef>J=5@hJxup+C1C;5Ll@p9mV>#iW4X=G2%@K~Q{*ooXZv8M$sgcmJi zI9##LTkuP%=20E!r^nE5$hKvq&SmD)FDgNQn8+0ff+gawAWR~RjvY+*OWmyami?XH zZq*jZP_M$zCio;a^j)@SVMDVr_^&J5e2tTpE~8;W)2|JC65*9>eTMt7>lJF8UzxjV zvEQok+R+}qgZx%DcaBQEK}J1i;5 zd>)A@+24lQDInM~WD8NHHR;9hjgUuKsx`14^WHr7&nmtr9q^Ojeu-tN%OtYUh z=;7HV6P#4royRbeDF+q%xmJ<1ngZB_XL9@XHap{C47}E9g+L@+=-+GcT68fk+g>;k zxrU+fzPR{uvFlP$VrC|ONsvTf@g}&aN-bvpjC*YW>S0ehYFTMf!prE;R}r0?Nxc%k z&^0uC-r}`Wm_;4CDi1KvssY;r(s)4wFa3D#)$;O7F)3!fcMk?E5H4{onv?@6*}*a~ zEN;yh=Jdav+Hbf!34%f(veHLahyE{4Z@vZyR4YVJIx?fmgAK1)IqGeD)nUDy{L;17xr5ir)JoH(n|9pAcp=~bR4ZRrc>jpdA*w8TdiOUr zO{4V+1XYPExC(;O)`mAE)^Xq<3ZO6KB|8iRWm*aHzo2&(P>Y6F!kMUy0&4gA1MB=! ztTrjLC}iJ*23Jn?a3h+vjWTKZe!-^~ipSr!E1~&)^O6pFsU_k_1>6T>@SESsgvp?o zN3goRzfBZiv@;nZGEc(d_RX}U^)jjjrdD;`#O;=mAI?4pmI73yBjZ2DcHuGgtf)NW zAcGUb$_Y-`P@oEWAcpO=a&2{^*wt8kDpmy>ak^=a$YPc_>r@A1a!u~e&z$5=5eDq? z(|OqeD_{=3_r>Dg9*RPj{0oKuetgSdfQ#a+B4zOVhpB>?Jd1jk9r=Y*S%=$)yBu>eC*-5Z>iOb(}PBynz|OX!#m{SQ;*Fv^S>6Vtk#+^Ik_a^NUsw_#Lohyuz-@ zzUpiqcQO1`LWP?>NAPcqN334FT3q9})$r4?LOuq%euyustTJPGUpAkm#7H<73xIj2 zGtI=(reVk&xNR67f(z?Pn74ru^N&c6-AQ!we_Th%GfL3mHvL&L?2jkFWZ(ngsemuv zw?RD6;6Tz&D$o_Ep_20}h?(K3zFkuS&jlK#oSy825_P}s=ce>5`VJzt61Hon7{PKC z1ZyfI?9>PvTG5xTe0=plN*Gf2BYyGjiZ#CGm(VWOi4LAiQ{(^--a9k7TO~SFLWrWOciz z%iVl5pqfVfiy_}F-`e(JVv|QVqNYcIKyVi)%Ekz%V15+I(l2%26H8SFCzc{%Tjb8=6-Gf?J8!BB#TTj^ zqRJTNY@tJI5hZR4bXlHYK6Q$&LbZycD8>{qt&GIBAU7bdoar`>1PEEF9wq3mMy-i7 zk9wDv(uNgVye`z+m7A202!W07+UTgjb|iD)p$!+ne-bm|Z`=NGR$hDCQSFj~S2x(V zHXdRH>ZNJ+H+j#B06WLj_rgw^>V?-|jZ{6ol_=(RJa*f2|W$McZhRPx=bEXdK;7}c`|v~49gUKvNr zCv$^Nh28wq{Pp^xoHC++h0fonjVAm|VG&OlCvx;(sPH36sNyCPGF7QAh=)XN<=d-L z7^XR3CEQ-&zIA{%+WckY%`0qLpH~*=v{#dnN1o?rrKWuZnUN}WYE3RzMA1nv7x#9G zbu>xbuV1Qer>Ryigmd^goAlXzffiS;0{GMGn_bbar^cDMWjY5--~(2X>3JMvpBhE~ z#dlrpSA>dbZ;&-k%q(z~VM4VRn6(uASAEoV9WBr6ecAEdV>qh)=T;5)OIj`RR4f5r z(psGl(Y3mOGbt!Z$K!!w{&@R6Er#dwU&)=fEZbiRtykO|p~NreqaR;dK{;;`Df7(w zeB3l;fvkR-5KzX<0Pivp3wIu#Y>-kqFnGN!9*b9Tu{r&$MK(P9@;aoMNn5l?=`0$7 zWICVG_Y6bpjrFd8J=PkbV+HUS> zYTq|TDLt8v_jPR>ljgYu1+-u!q`1KBiI{IcjZOXTr)GpMPIPj65=^o7l66>Y>Q1Tu z2kbvS{v{5)!r~=k5zmIqKj%*>Ogil*xu`N(=#xZ@E>x3 zwi>R^t?eqp$g6i%&TcL*oUfMgrPfN`B538PYxHZ@YE6qmyAQ($~mRS9~jM(j0OjGSInS#f`0F5%}AwTcsW+ zlkH|RO$DZ9((lw|t9+^bodRae0`-a-@U#Tc#gx%Y+%Hg!VI*f7lpL>-#nzuaV!9ko~ ziDlLl5~#G@_6(o2GN`tE1ATBlfv&;nXB^0}&3>(3YMWir$RZQb|C|t4Mx{4Biq!k@ z+V*HjEpq=!{2Eo*#4kj%W6m=6tcDdF9#RA=a6&bHHZzxX)Or+BBM<>cHBUkZ%+5z8 zp^$At7@2W<9N(v!8&%Ob?+ZwtMauM~rS%f^WxZA!1>u&~<`hx}o>tgfA_gTLY!7gW zG?-oNeumXxVgJij{h>9HsD!+XkaX1Lg8q&1me9ld($E0qb_|kf%It|}BdJ48FyTaM zgl1>>3K#oikj^KlC}}-rAjvzI!i9do;ZIPnkdN)eUUUKj%mYhetsyP~E_3#3`q-pC z3xrmO!Th3QCUj|{VQ}O;6?vh-C1809Q~KsW@Bu3TL+c{O47*h6PeT9m)BbEjAVE>H z#1uk0R_>_5O~A~ff!8?gXr9` zNaGc+jHT7=cJVy-IO;2ouD{)ArkqzoS2ejBLv6Y2FfyCsxSO#=pzK?>JC2Q)^VT+Vo>ZCvYV#0|MrNgwDDspKw z%)Rt{gWn~9T?n8iHU#EixqRWe+cdQ z^m$icgU5EYTW{nm`>2BIBO_n}oz45#)e*9%4u2x#3X%i4!=eqhOwogyK;H51={-%W zUT&S7gpUq_zMjlE3ucLDVZ2YmFU^riP4Ip77QGcS(nZm?kP*>Jz4NRnBnSyFYMKfC72*{Qn4O9N#{Z2! z4Vy|whTO*w5RV&-Z$Ij0p0Td?``DREJ_p8L1#4ZpI!-r4ZwkDNw0C<_SK-~|DK~OH zJQfZu4)C-WMS&2-%V)z4XhVI$zf*^+qSBYFs+J;Mn!~kfjzC$!c~4*Vle3Y_RCWtW ztzCuJ*r1bft;D2gJuvWdT?F&5Ncc~PU?CIoqTqu0?y!?}9JmHiJefHFf_Cpx%nO?aqRyM>$>B2OfGJ` z$pH05hDt-RgU(O#ZN5)uj7>N9aJjR?V>G{6K*z;_d|btPb;qqe8YWoo!wcp7rU^d= zYoJqSb4ORJP8R?Lj#df;DFSZn--b3&7pPu+v^zlDlH7imLvH^LS3K!E8`6J|9R64% zA`x-{)_^|qfe3@bK%Yv+0+Ql@*`=MId#;Lf3TnF5hKIG~9ZYWDro^gzAT*28Q)^hdjQ(dYQAQ;5J&!RRU*m2&8qAp$C^V!x$e*!*FK zA23@Ae^@~G+e`Q(A_Qrmuz6rZ)Z%os{CGijl`5%1li7z=#qrK9ba2BSK0Y6eUS-~* zThauJ(nxAH)!LLJ%<4P-}xW-~_o|ODL5P7q~a&w?RJ|g?${G@ZSOS_s^e~iVfk6EjF44*|8??pptnQoN0ALV}9E@+}|Tqaq7vK zZM03?(2Oe57U#K%_hDmoSHLyD;hcj@cS}JZId{QxIhYSlX3UrNO9P1E=sbumAcTqI zaQx1GfRl8Se&Wdr{3m=hr-9&lu3||aLVgR_-U&0j)WW=+4@)?LtG&hN;0Tpe1Tl{! zfu~0T=u`g0^qj}_SYiDP{X+nWs3;S>gShg9yrcX@=gdS}i~np~{XPiL*@Ux=PrcXg z@||onkG5CQ!r>MGRu(>)@kO@~IJ-Exl63iu!xDx5m?*$P*eL%p6%f37fY2}E;9kjv zcllGwn+pWF#8;@hRs;}+zyp>3Jc7T-;6f)dqs>;)0f8<#X<5N3dCBk@#k*s< z-}plx*gCJebgSAd+r1jD_Fizu+Ay_yS}3pyg)+R^&GR*PA4e{xJ?-sP94&pZUcHAfk70ZS^hr*&=Gy1XtF$@B=eN_gcLgv+pUGxvN9 zWH25p%?xfSoRwfeQnBTx@MkIRv@UZSSFtWZM;u*S3~cCI=t2GCMHXE6#M6rt+MJi- z(oJd{vXS|xaEnvpgIm`l2{b@jv_^F>uuqpld;97F^-N!gx! z1R`;{Z+>K$?yc1N#+yB2j^QG@&Vu8#okS+I5VsWSnJ$wwn(AczmsCpMN z({W%}L`@XJ2VvtVMh^hmZDTpL0}Dx_EH-(}_~IQl(|1i7{raFxhf4tca)ni~f!WN5 z9HnnY*G_<_;22ecsO}J2kXj}qs*Pv`SX!kx%9wATEf5g|q>{@(=R~!QZo2xow^U!1 z<%Z=pEZpU6TaXpIm4jC+X~c&4h%0bwcNr`#FnL zM1GB3xi4^{5U*PD#}*2<5sHD^Sde|H>z_hOhH`uNbNl8Ku+a~1q%zk&sr3h74emG9 zMq9h3sq04SD+e1|dghThoS_nrfCK00(BXeL3rMi_$)Ir?N~S?^i3eGO{4fEdZ_6b* zfz@B>27X?>rOK4$U7`zT5|I*n3{tNm#NYtOO`Qa1)B&uJLihIHxh`eAq|Kd%6$mK9 z`HG{&^M^v-emQXo#GCG2B^_EM;9}GONBgTph5EZC~Bp+G;U z_;I2!rQ(EpeqRi zd{)W%_8~i%gj+eGpgfgeS(TuwSczc(H$3k^q?E6+{MoX!_JfEvZo=Z># zbc8@KiOqv3I#(BLfBujiD)qr;b^ZyzpKMGO*F;H-7N@!Ghg3sjR*SX$;;c`=RPw<4 z<8&nrwzs=)mX^m4SV%>hRS}Fu0E$!TOf7E|iy|vma2Mz6Z!$u(A*;he%Q!fx{;6@^ zPgPsinJVQrJMQ&HaoSKuL?SN6lpr0#*o46@l;`*2pWKz>GopjrFdsun^dcMb|AdlF z2~r4RIusC?Q(jgJ9q%SN^dCPcXm-Gpzq!I-5^EW~Sm-F1uv&*vMaTB(XEOkEo$h~k z{1|pvElsh2lxNAh_nD+n<~l6d%eEYczE&B)RMM)8m&S|DQy6!r~|_d`Tf@v8$TX1&JO%a+H* zemRrUd)F=yK&nEE1H>J4m1m=>?O)#TF^imz6Remrziw6>jULqiMt^U%%J9_^m=wEk z*8!8Zov4Jw`W`{5=>G*$FT)-a7^MG9>M?=2qVg)2E;aFNS(WtR8?-i?c*}=t#Jd6z z*=9aA6S%? zR&V}JSXD)}%urS)?#0EA*Y9do-S^9#=v~;0>GobaFY~F`OlnRGOtMhJ6LlokG(ocE zNfdL1c&MP zMw>mNEy;iU?5{heD7u)28xIRby-^SU|1tK~QB`f-|F9q^ElP>fNJvU|Nq2V&0s_+A z9Ho(#kPhkYZV5^0?(Xh5zm0OZ*ZX|mG2TBegNx_vz1Ey_&H0(RNQuMgd-GZ+i(l_!iab`v zrwG9dERVx(>iZAIVXx`l5dC$SZUG9f1St>|kGC+smB-uGb`aZ^vftz&i|)+VeF(c9GDJ7{GXdL5}LnpD2} zks$2mt45V)G%#KRD93~jFbr%#dQ$ds%12uL3y@fUy-TPFR0{AUxn*gH73|L8lv7#Y z%#^_cH#h2kB4+-5p_X=ptisr1L?Av!N3q60_7{;pkXcyb#eGIfU#;1H#!4W+e?0M7 z%)vk3B;_(DQntbvJ2#r?h2_GJk%j!6;M0D2axGx;PqcV?Fz+Dy0q*u&QqgZUI6x6< zM5p?X*H2tndZLPlTR!}FPm2`Ks?3HC*rJY7I7r=p#8^Qh3tkyTR$^YxEBHzmTsHsF z2s=?Akdkm!hi|#q<~JlR6|PbckP2RJkj=C@6j^=tk$`={uh&8C(VyJd zOKMykdpy=p{(&_Vrgg}NWJw0CVYi3Tq;$(mvIfP-l1Ym+gsyIxcDmLslbO4=-bhZ5 ziUYlK_721pB|HgDv8R!D~qx?gTi<-16zC;n&MIwVQ`iO#wh z0UONJOY^y#?TvT4BV{pzRO4>`cK4$j*RHaE&DKy4FF>t^ zYYNzkE-}E5^H(`(f|Kn!?t=QDl@=1#NaKJ4_<#B7n4nbOH`d(`sAAZZ>`b0-Gvn6< zF|IREoM%3l{!?`!QUl-_{4TprE(J0zA6tJ83cnnB{T@(+bFWHX&2&<4V3GAA!ut8B z$vC#Fuu;uVo^<%%MKV1lWWh4b_6X}-uaM-wOZfnKv_1YS{D>=sz^LEBzf#+aLPbz2 zs;Qlc3MrC*WznZ+xZEx(T73Bk;V1v7+21N=mJ@_{82Ss<^UZ3_I0YU^lj>M};5{gT z%^lvci!qGpi5{6J2q-+{-x`bD<;0Q=b=R(a1ay6T<-qWFU@JCg9(T=Q9M%3^k!a5s zGYjOmP@8GQgYQ64%)o6ZKr*0FR=^^E2&o;+Nn7_wm-eR0(Ccfo4EqPT7C_vSMUIuk zn2tAxaP-_ai6VA-uX(@ICXu4=Ocmo9eA%JInAITe?`M?$=3Qxbm~WF=mar(3A!cNB z83N=;Z%b(K0jX>quhc7Iue9kodRAUwD%2l%mOYCvvX{Edt;f83B6N(&#Z*V?8O{U~ zg zAJF><%$jd$uwE^Eu+Fgr_e$QT+ryvJIMBNa_X%v1Zs(FJAoNli!jJ!Do1B8)F*+Z(cfHZU*md7v5m2es+Yz3>!FA zBfk2}xN3i5ZF)81pzt->CN@DZI6bA6GBjpY6BzJ0Ax}Di8I8w!>Wuc8iu2RSj=0@; zgWA>QA>KgnNN5Vl>$~f2+(b}m))0+6A_{k-9%);z#;XW%7n&`tv~B0CsDc>+%kl*| zN6-F2k86>bF{gN5L8*xA??7zJ3MkiwH+2g9x{KZGdGKU=!V|D((TeJr?NiTzG3r*!v>%mlR9C4@ zFyA=@M;5QV+Y(UDQ*`blkbUOdLsx2n7GH})Z#WotXc7eU)4kmtVX_1eQ_ucu3r4{V z#X0hnBVY#AW4n(EY5~8yqrZBIaff0zI37Q<6~D=*a+-GDc^WoG7C?37&I8_?$5*iI z^vDQ)nC9v^|2wny$#IuQTioG27{$gk+fJ`)ZG)n^#u%4Q_IJ<9%X|%1pGRrdl;(lu zb>7LQeaW3}p=?u?R!!t)E|+STUZdp{7Ql@@h>>7!LUibwz#F%jGNQ%izv}yLk$^`P z40J8Q5DNf|vaGJXUtvq!{G4KZcLhG*gEC{9Ah)$cEbt}dnkHngZw1dgPn#g&?g1rS zLi6oG*TRgeG3+T=)n!JRqV?JlUMQ)U1&)H8>^=cJoO zHiaiO_S>)lljB-9DI9z0J57o#I%P^T3wG_X*QE{G5ojA18!u6DOFb)p#TDuHC3eK{Uk1(*o#sUK$qvwE$zh)(RJ~jA*Q~Z% zN4&Vg-i+#Ltpvq?HHPnVXi9JC(EO!njYF7DnZQ+43cfj%4ka&o!M{HD^tscKi4SrY zXIgVKu|=)jzWESkggk|`2=V@f(YSunN^c6Gp5O8FUqw)!9n$&=nd5M&1~kf+bQ3hO z9PskOM@lDX1%Az~U9T?%SCSOr>Q3eyui^xq+oz*}(WMtR?iKJ&xf>>84o|+-J{aCG z!CLW|*?qD*E!1V4-?^~-%krb*TxF*TeXafVJGK0#H{IaO-MtR2%PY3OAQT{7(c*yL zwh;LONEW<8u??ZfDDTyxI0hOb`^Y7njxX~3)t{^PAAjr}gSsu+&Sibd7LXs9!?Mc< z=(Xu9@vH4QMpbfLp0~z&4Zk zMSAW2Lo`GW3E44-#J`13(joS4slV3C&XK%;j<8s*pYk#SWF%$ID~BZ&7>?9gwyVnZ zQhzEg;=0+o(&3o2b7+%*Yj*Cr7TV)vN#bL~h1)?;K$&WUG1l(Ol-O+k>LD?=Q0#ym zZ_P)6?YrV5!Uo0%hd@cw{wRp)Z2ofh>($#Az-1pm(VE*Y3Jdg`7BWZOu~${Q5pQ>R z2(7MMryi7WsQdS65TfF-TjsBfz7-KRfy2SSV#4e*_pTDLicaNlsL$!5pojY}Apu&d zcmuv&KOmdxzEfs&KqWz~lGGaq+Upptq#yz=6h&?8XaTPrZCyAlW|JdUifU7`4oU5UZW$`y;Xpy zTx7SYMWs?Ds+x%r^Ch##{ZjYpIrwo&a-B2y$$}w83XZT%4bDhT9!*prJe_Jz0`YD0 zuNCe*Tiiq)&bgJrAWi^uJ2(|>A*oZc#%P0aqQM#wX?GB1f1z!-7V|brm1AqJPgjVV z;J*_Hcp6H*)G!g_)5esS@S%&%SuHkbc3KU#PvRd14ga`ltga%TSL1a=A;cyBUG>5P zXa<*~C2Xt6+=BIMO&j5Ou5=z_Wtz0ws9EqKFj0Sp$&MToSLgV}2rBCg=M>Iv0@zfo z7&n`(9IDU$SWTque`A;gP>86228Wix4*SRGriAf^5(y5tAo<&}yhpaPvx_b==Fx3c z0uxQBSvy{^qM904&ySReSEt)h$RUtW4u4wH`T&+kcvU2QqWE-ha>)gIX?qq#i9m* zR-^~*4qc-_di}NG)#MD7jq0PcsY?C11IONs-3gRDw_%UCa|ELU@Xv^Sg|Kg<6oZ2Dub3(Dz~`gBtcL<+wm^~Pg7}!gXjvnf^gh8J z{K+L=d-JgRZ8x1Jb5I8*j7thiolxG0YoTR3)UpLoIInXxXJ$bitv-Hw)6%X|VT!*( z4Ss0&kcRToz31IUCimJvcKu59Z3Yf4p2!-#AeD+9UDlJ;p?S`XT#2C#g<^5-2{7|f3G#red>YAX^h>cRr_NMJK&Am$usJm57UM0 zE0QmAf3kc>`{S}cwuT8JY70d(l>M$o+2btdGQyanjy>FoA)JlQBlui6&d>ieM=+;C zr#X%!W~PLQa$o>j`13pijTtRcIda0TP&x1aS|C7i3J?ONzNx6(^8Xle zk2E--hF|ls&H!}YXr^jM99LAD+{Uw0(Yc3}qcr(79dtI9Z)+o%ulqzI>6xCrl0YJy zK-DZd!c_L|R!=jtMv7VUVzn%tEyZ!dE6}dv1v=N6{IUdcse-BqVf3ZU(w?%I)xwR+ z*hpP&9?_*{8UW$3>Rej^zI2*#gnqs^OQ`4i>{5Wg6`8$OtXd&}QZkY&7oVOfQ%C65 z8}pEdMB2|`))lPHxvzhk>T_ywk;xK-HxcMMIchQ!>;!*xVyf=rX1ZydV2zhS)P_ooZn_41J|srMSL)R*hRa&n4e=~ruXzsE;T4JkIFnulmwvzca0PkDB9m=4-Qean& z-kdk>6QsKzlNVDx@Dua4mZzKuvU|&$6<c z01~NXNWP5 z@9Ujg2^~o%d2Z2CMD0*-aiR$)z-d{n zhdo82?2)Wou$UZS;NT9XvS*I$0I>bKMsSwg!w$fqI zU&XbKm2J%AZB{y8vIBiVNvegU{@J6&3?o!=Sqh1$Gw-iRI0Ue$`->!PhJjw`#xry6 zmjG?04ZTLuCGbY%jgadNs7A#fA{1m9v_G5P z834z@{*+G#0j-H`W|1l4X{dSuQUz+&dfv(kv$70d3NW~YB!H0!LbC@Kk9MCDb{EJ~ z)tD9V^^ddIvOfkcVH_E2Iv!u-ovL!gxCIe2k#94d+2(SG41K5(a8Av%ypY8WV!J1M zQF4ND_lvHzQ@$z(p!|c6eurhs#pY_Ul*WO+G!`#CmN7$PS4cp+ZB}lqTSJt-}bD%OC8QPkn?A&P0$lH8eB)I`jr9^ND6JH z?a~ee9&`EFWbJRYQl{<+`flMROye2>5zZE(XRABI>PPhA4$s-MRf1u7ZgkMe>zuBc z2W&&>P@x-4gjtBcxjw@_ESR`DoPRiFT-vnHGu#*b_AzG4q>Ygrk>WvO2z^zZEZo)E zy0u=^=;SHh25I-fWHG82;+nN8`5&#R2i!G7q!g<~Oy|B#TB056^awQ(P0Gv`Y~@_%wl5K}|N%L^vUrIvZc|L+`j=2P+|x!hQ& zM`*UvGjCgemG_aC@yfoxX5Hg{WN)Us^#WtM{nu+m*)lD}0LjJMYA5-rep1+E-JJ+y zCO*Q)hc!z1V~1qLD`ywT$AIKC^!)hDz#w$m62^u`RsN(0nbT!LS0R6}d+H4s&`*0$ z*oP=hxAH%1rmFgI1L@9MGkrHi;L1ctSAMgvC1gQKinfO}ZX4z2H^`;qyD!ufsHsTm zO96Ag>dAL@7;WV9kYmi2XJr-FI$>?txgR_D@1646Lu0KZ$2Q#-EqP{%5t%}5ChkU0 z+Dh58zIcexG_WEVerrQt5DVAo*j<%Py1o?B9z`q_)Rgip`+NkH_CTa!V+ z@(@Si+DyfV5!PlhuRBnH07cOviC|>0a|+BaFG?1fhUo`DBM}bC zd5@$T5Rj*eVMp3`Kbx773BBmktcKdYbQCZ)r2ed9saGF2zPEGD`!DD!{eyU%+qfFgYPV>zl@DNfowVyc9OcG!;18Tm@xpU#r$B?w_ z2Yn-(x;hXXHTJsim;Sesk`?xVZqL-~>;C7DI1`nYjdfrYhiQO_jyq3K^9Ct zU=W;;V?T{qE@V~;8eitY;lTQuTe`MPJ^mfnk6@?qH6<0SnJ)e z^)5{M$fh-G{um z&w(n(>tjfIk8Fgt#x6A9V2lr41jPiwmqZB|80;qkKq5Gv(ty|Z#r+W)VEwUxLqSLEV zAY#~{rdYAt(RbZL3wQe>e`Z|(Z1kAo(!Rtmn>d_34$#}x^uulo3cq~?^C zcEMX(4-KDVPSDOC;HlrMz<pY`MQr4-+Ud zXWnE1m>dz(=0YlUg>}n`H0`iz8e6WilC? z&KAy)*~AQyH8>I`sOdpN+Hv5AK@AkvfDtDDm}72}UT^ex5z1k+18v_m3X7=$%262XDfQvT71lTtBkfM3_&vM%AlGt-|4TN2 z0yiobFxzdNvF+5QZaZ)_Hq6j-68HSAHk6Di9QXd`d^6kEJovw=SK)HUD&=Q>eu{ki zt4jCe!+Vj~P9oJm$5c=C?HsvoKyRn*^rdPUhw+F6c43<})xi=#*UfbO;CYo&r+}f& zHKRtiSCVleol3)N>&MG*oIE6@lbFJ{jKv_`{=XQDaLZWOU*G=a0_OsKgJmD2pKp2bV$Zh@bY?Qj3bfjs%_W^3 zx@-G~(uv%9Yd@;?qU(8sEe7bU-uHOIhWjMR+{8V-Q1~zeTZ3{T+Kyx@0!lTod56<_ zir2-m#oG?0mny)>S#19J2*_puInaJ{xljS%M)l^Us;+*+CB~d(B-;L8TBdZs?XW-m z&?a?lWqya=R*Jti%KpGgWT6C)d4co{7JwXBz_#JjO8!IkXx8r>3*fyvQM~?m+rfN( zxvE)>W8_8JW%746s}4_UQ5d=KgE0{6ipw4Wl@@3Hp8uTQd+3xB1H|EE>!7>ubD@CQ zC6f7-rc$r+CJq6c+pKo?jcKddgZ|6WMl@j*9iUPZD0*psd$iPYYb3Xwo(NxjiENlw zDbL=FdU$*B&Q!IX=!js6W<+&QziY=2kyhh{WG|pkT<124Y9%|D2|uYE4-{T{+KR8J z%SxakLYuF-lRnT08q=bPA!RD_RzzF0=HYN%v_&is7^oayx z%RKX|uI%(D|Laq)NzUy$c3^b@a7wdgY?e?RRy&q$*7f3w6J0%9ZF$_ST?F4k=1jP^FHfz`{1)HRi1*T<3>cTUE@)HL|xD`4AMq6q3Kh38O?E zerzBcu6-1RpQh&wG>7-QcE{))6eg(5lh}8;9ed}|0hKfXQKxJ3TvXA_FD6H5hg}PN z7rfxaY*?w+)CS&atK|?oU9kQdLssh2Qus-Fq8x5l_v^c)^cq1Db*08 z@~6YHHCGq}C}hY}S%QNW%H0hI-I?a{_0)fSJ=El9)i|5Vi0pa$uiPjEoW+tfg4ZUt z1`6N&Bh_ybd6S#X5RvRYVAIR3Qyk0G2xihcbW3BTJ=Lmr@aDO8wOrVP0;V1y0xtS! z9Ud^yCQuh;H6<-{KWeMGy+WbU-h5UEm^_rL#rn|Rypt>UDR4i$g>sdh{E{sVFuYFB zghn-Y*n2r)tlrG%I~~kvb8Bac+Giz^kT;g!PZ)nm@!Hs29ahq!0DbZ3g7Bfyk%R)S z`6Lts_I%2zsH~C~U(}jaT34H<)q%02J*>eh5sV8A&;<9L#F43XGD? z$6ot~7C@`*ShZb|Y9bUc?+BYCBr-=yTXYOBzP$HBkfF#tXZ>@!-6JYayP*t4o0!KY z_l5m`NaU*uL*wuc1O=1@o$#;-`QvbXv`V>+LTxT6I^c5!*3GmR8jTW8Z+0(*VBUr> zs}{WGmB?wIU478x3t&U5w&p<%U!`dQW*0;c_siNOnT(cVz8E1lkCc(j9R)(EMIWzG zw%FHBhqL_RfYg+1y(ePKaNyYhvGv)nQ!tEHotudU8lerk$#+}dQE}1HPvjeMqvSAB zKzG)N4CX1v^{C*->BY~s>ZjO(!A?ko38uq^?7*RUn#C&~3{0DYE*F_=+jd4RC}?c& z+uQ3$<8(NJImP4rgc^$Q&ZTqd1a)_RVRz|7G!T5!)wjMcFL%DX^h}?c^tg)j0O;7B zB#MUalWrT_O#dX$q4Unp{#K-bC{?lb^RNG7EE^*Mu=GJZr%CPJdW!HGLVIaUzj%(M zf=&N1yA22A8 zypmsHScU*l7amANO}KV>A3rx;&p}zo4 z>OjE(qh(PEe~X^5%aQr#EV<{V|Iw%QM$oHV{f)i7K-f#`)0Ykf7tPW+qH2+Nc6%tF zE2$S?6qf^}C(wy=Cv}gtG2Z&pXsi!@e>p3sJ-AA%J%2Lo1Ltxu zRLN|q+b&e=U>TTFSeg}1EZMo#pWtj0?}17f&-v2B`3XL@HqS!ebz{2_C`_fo6W&dL z{_>R&U6=~(XL_Xhu6V%9H8uMtxMLM$+?h2&M1Ck=*VRzK2!r33V679!L&6Tonx5neumSXe5ivba^Y$78$Arzp*AjZl4TTXlV-Wjjuz z{@@#)zQgm~3TM|7x3O($Wa{yK=^Ne=GVQ!yqm=X!;IjTecQeZU&BF6CJI;o98e{i_ zi3*dvjsRMj*vQqfA{Q&6MkVO9^G_e5q_D`hTiCd>%&%E0*3XiWVNaMPk5K}m)wP*QBRU-E(VQC<1aJlRgCGMoM!*<7O z>!Dmh6X|k`shF$e>MOpeYiD~(aDXis`&YJHSj z5ty%y$)7LF>`$d|*$cr#I))L(!-{2Uw##6SRJr96=+0)p)<*~DE8t7;+)Z7E8&u=Z zsqk$10?@{Y$@35}SE1t~yW|V4xzD2+W~Wn&5hev{$IEVHGnYf251`=)Jnnv+psKLQ zmjylknbt5#VUSEQ`q3nq$iIUgya~E|f;X^JcWFB}_f9d-YQ+6 z-R!GbQg@L+?HeT}(R}L^-W#y?*0}xLP|~7ydr;zqz0eJoMvih$%7$VNg$%fE{zpkV zsWW`;^q$Xz(DatlIH$LwjRq_}}u{l+3j7Fd&~ ze~}#KA*V@>AV94H)!N< zULYNkx6rjLGTAxQ3Y&jdO7^C9yofv>JjDrW_=JiPftM;h0HZNw&~vQb0r+d zlS$+;|HU@AF%ccxPk0CvL$h2tcFtv67uA8;1d4atvPfxM!ah7+^UT7FU|wyd+i-3_ z#>=DbNkOLE8PvX1!m51LHxzQRZzbnGO1De%b>*XK0EAwDBm-3i8j1DMh@(99-Af&4 z=AwOIWMyTkrDB?1B1CZ;Y(O&^k4#xVRU(4O5(?T_y<>PAa$S2`HiD2@*U->VAPQxa z)xXb8qxZqq+!3S}09sYl_+7Ns`fh^r&&h0sx-?`;B7Mjv{CN`zQ<=cq(wr1z=XCtkFYnE}jj8_|F18455RZ zZiOYD3Y#t?%B@AyvU(%ZnQKr7147w3UXb)`d;y6$DP}$u8!`3rv&WI*qW^xbjyxz= zfaIkd=*H}`-DkxDN%vSKfeDDP*n`5!tQA>ueNgb&7i3@ijHG9(4O8Id9ozKpl$DfO z_vFAvy%a+Uj2OJA339S;8a&j;g}nREB6)QHwennLRkG~vnR8Vhsd&!>1lo`o(y*jy z!Es8a$3G5BII-T5K)yNm24skbo5bS=&f`_|KC*C9>l8sK?Uxgg!`ro0DDkb ziEXc+M%9tMx=`P!YT4UuExhwP9`h1{#?7hwJ%rnwx5AElbiK+=m-n5_sa!rRSv%?X z5(SsPVBpl!^KoWITJQG`yO~JLL+;wORn?+X>6tu=g0O|b;Qt3(7V(~W3b(nCuKN$d z(=^OWO%E6ru2SR*K#T+q(cVt2^6_$51%mRSqmWu>1j_4B`uoiS1Z(7j9*4h>5Nf(H zo3`FnZ26Upk{u>@_Xz@Ro#pIA3`bX!Zy!83g*~R?}2F)|;6E)k)Evev&bF2QG zYAW`r#p493sZcIO=aD|z%AujvOg;^v^`l;!pr3S1!~>sX9=zxk^a36 z0odd58OADXXqHhisX=`{*W1{7ob2zdfIHh*)YdNh&NOh;zx~fq zC%!%EaZC|+x`=lj2In*+iQk7ZJ6>^nO-mF*I^+4-*elKFc=GJHN_WEY0Gb)|jE-24 zA@QYZT+UDWijODYAnB|PC}{pwTv zhpHBFjcoDx{fX*`D0>*nOJs=Y>l@;!E<1Y%TkggknX6nu48~&*Bpp76v~O~$I#^87 zn`$woG|I`dTk0lmXq;unjm_Fl)gUIsF__%3AHuf(%YK}P5D-xq1Q})Tt|MUy+Q?s} zqH#t2qYrefL7uC^>pn+I(2CE!SJE@mk4hYkz6bwX#iL+$NDbg%Z0wGFaLJ0Xb?R+W zlz;I@p@3fhvL92at6w7y4lq|~$S?fAs^P?9N_*{MI95d z5fg@Ku5}!ZVs=?QN6&A~ueUsOlTSRWNxmL;&IVCY|KG}u82_2g&NJ7lch9xa4We1~ ziZ+-(ELEZWV+>;vw#KV8Qy(2${hVR!N%%$?ZLa@{V0$2 zS_C795#<4t*q@U~U;z`R?3N*J=0~g8RF~hNVJ?;U9RcH$xUHsWfaKs%sj9ir*xG9n zYU;hiO=r8ouu*ODVI~r4ZcCfk5m(pv*qeMO@7As?K-JaP$PB0+TK`AYH9iAWU14vm zF+|l_L$!0if(!m~E+}_7_&UF?t^xS3(Fd0+O7~u8>kxy2v3iEv0}QZw9c%&RaL!Ee zS>YZ50Iwk6g)K%~9+=H@pWz)ge~Fbu^fmA|2R#^I4}1{#bP^&E*aA1l}&0=YrPaUhL*j=*^rZv2LL3U9?k^qPJA2CBM z%zyAOXoTw}J^vrc107&k9C$3Ih9z*svw$}P&$TU=oG7?9ZEGwzA?KQeRR4H^#d4yn!2v$3ulG$2EtK zh6~hdB*=LTuCps&1bUAQ|iZcEmyEQSMpe zw1M{UmMNcJxFv=a4@MP@|0QFnd)~m9(xD_VjL{pj!Ju%3lB`o7xY|$#=Q~BnI}US# z&-(q)c`Bc$qMCE!*rCPQyhf;h9&3F8v1O=!x++^EOAcgm*nnQO+sJ1+TeUhw05twK zeAr0|{(8LW6s(7?6^(B_!#K+Q@f*w4sn_FjmvrjcV4LxkHCd*sN!0#XE_t1NVTfvg z><|a`xdoN5%UcM}NrR27{+aKS#y}%z6<_nCn;Get<2vlHcQ^B$UPh!YR^&ZmJ9*|{ zv?X(W(RB(N#w^+MQ{&sftX4z_1oZvh7)&v*U5bG?>}8GOre^MnrWWFROyfUm0ERbF z(z`1h1wO1*Qok9+H5twX;J1lUU;VWqBj-9zR!J=55Un^?mQv4#R@XVtOH(z+hM=vR z^1XWC&SL@h(anjEzWD*2wh zoqRe|Utt2{!(>oG#qsF@H*jNM{nLQ9N;8J==~YsoQBi)5ge`I)*0p9qKndYr4+fro zgQ3&d-!LO^cW?j^Bu2#>-ZuGW04k8P3tG=|<4-J>2zvDivQr0fo!%k%PKooBIXmwn zMq;LOIrCN>$N~A|@ib~fkBeTYr6U~Kn=sdiNz%rw6`=cmzI%k*Gge1><%(9W^e5bx zZPHJ>Fb#SG^oCA@HoNRB;gwC#-+e{-cvC9lt}M>4C+I#(w(!v(^7cDv{a-qF!P)E# zY@@WinWc#P6>Wq#lxbw$wIoUeL0^sE5I)Wpi!ORkEfxnFy|6dPm$+H|w$pbE`M~ct zcr?HkKeP2A1mIcU_Dv9lqRITOjW0wuaAS7#^bCgg)uya!#jh@kq|M&cTE}i~|5&x* z`w%;zz14&P4E7zm(66XIu}|Y@SpbCOA%EoEn(w!uV+){|Jh!(?kmtgk$s}37w<#wi zUOGaJ(r^vfdH$x`M|S)L#7gWHH1ZhMeK)^3HUlN4cgJ;?E+^mgcz*bM?WZusXc5#| zyzUiJU6aF^*LpLE%~E$C(RmGlmg=M#XTadCfJHpnVG6@@s*LTN#K9Tl0EFE{v|IJc zEpMpzw~c^;xuOKrl6#$>H)IbgVg#J@D}17~SW+A7p7x40X0GDPV)T&p@T$U`Y>%gr zEt*@}uk&LYy)O8wFP;B4n9C&yiU0msAE4Bv#tE$=YYqrQ?$zVFH}8RXdnb^c4!*5B z?P`m=7S~ zSycc#mMfbh<*_NleYd7D)w$aUO8rc9Y^-_`62B8Y7xgkW$EeT_gc-ki##_&87nTsB zTG(Ln9AjvuSQ=fUibmw_Bd8CcXTiln0g*1rxBf?vrgN>9<<01e-A&tgvxel16qUU>@W6`o#!d!|!S{>qZi z$Zm?rHj7laJbL9kV-saozv|YtBB={mDz7N?F13TCb}iUoUjNc67~iPecsQGwM6<8& z-{9f_R1V_DZ&}%oKb4MC4bA3_RtLZE+`|ygNyI<@!-_dcP;D*eviq>5>IQ}DsdaA_ zEk~sO937Z4kh<3_7nn;Tjm&i7TdjgYskCR1jkjIKU>IXpKzVj7#6O|TAl(y#f_|+T zjIR1uxCvOM2N%q9i_&(s(0eR2ClB)Ni0)5k+&Lofe)>Jbus%=CcJq>vIrbi?*Q1U# z7)B-FIK)aY6f~ZAGm>&q>(|oi7L)x&H5oiWYrdc^(Q#K63YpNjD58iPPU4g!BE9>R zih#~&g6oCtHR<-WJ@4Zu`29BzeM-S|gt%-MQO1DvYD3=4dUwrktN;ds&Fk39u1~z$ z7JQ{{%H&NN>Xm?KOGGl=EyGMgF9R4MB6hE1A^`|B6ZZaIlQ*_OPuJKNxGkydB!(Hs zYY7FLn9CQqT}7I|#HY^svpLj4=zvv;!GYje8V&GGcEsmCt@4lK$sS(G7XLnqfQPZM zo{gs7EY5ew_Bj{OwyJ#T>8Tg+&$Wx}HP;UTp}JD{K=_;KFwhb} zH0l9|MJFU01>^q=E_j1kE}gAtudiLonIAbe;UNx$s5&n}xav2*#t7|eR!p9!(#wUD zji;kNyZ`cU3UHzWP)|huHn=c(jjNbF(%hm6^ts$U<7-3Xe#^@;xA8n0zvly)A*}tg z)L>$whyUK(t;wsU0rbqM=r}5bnuQs>D!>KX8)X~jVckL*%D>Xuhc^xz(N3uRbMJOV zKmfZU=`KVWb(15r-FCu%Y0uQx*~yLuEJZ?E+G) zAWNHVbcoXGeKvNn$hr+aY-tuW?aN79*)~@uII)*xA?tQd?5*C;dIVY3Z|e-_0Ckh{ za&yUZpTRW(fdX24t#c%dIxk?{w-zp6>yM3EH`qeJTIA5{40(>ltPa@AfYJ z+**Tr2t)<_QJ@c?mtO(CFk2HE#KA``%;t57ie{p>nOQ$C+kKw=Scuzv`0GzZ$d=9u z8jnPSjgL>)VFe|wV~Xh0a3vt`{Y^lRLBJC!6QUTK%^M*QTlJK-c3Qtj`zWYq#CpCc zyAsqIKMRsA5teRf9tNRQR=kFSJpT`%NpiSV%l!8=Ifb5{zFO1=d0^m;=LNs7*Qy+W z3jU&)ktzNV$*cL~pJtCZcJ?8;b1LB*xHG3=Q zJ9z9N2?KpW$mj6hruhAjI5#E9UbCzWErg|2A?7M;QMBLN13kcuev=zqep8hk;_)pk zw7gdRL5U1IA8vfr1kP^pq$NaevZ@m#{mXggUU<3VKd4wSwYe3Z!47$T zT4Tm;?`Rkcnx++tWQpNQ=d$)hF?pEKUpBLoU~luIWlT58xH~q0t@q1p) zB5Y}K2y=`|*uy`=$xT>2R!%y#e|RwsJ3HgJ6$Clye?XcJ#w|3ndbR|Kz$>A_o%z7r z`XMYx)7BSFFtD&jQ4tg>q7E{&1HjI4mU0of5ELL6(c;AYe`)nGUE*lYx*XRF^$ zKKkA+jmlm*3CVw0PJ7F8eVK9yc5jD+x#IsMB`cfvxo57-OS;s$dwGM!rc@6l=ItFh z8gHPj(}R=^{5NU$dzq{zbHyYz4Z#Rme^C^bJif7yVYvEj2oREBiq%R>$0dt2WB4cX z#Em|7a7~Uoc913R8SZFo-Nf4uQCz=74pPz;`pRw8U`g?UcqJSh) zJ5s%Z{2dc@j>3y^AgJ~`; zza-bQUgN`LBijB=I`C7#<^`r}T-F?`&1O6zd;$e|3EbD%#@@K7GUw^d{A zornZvStd2bbp9iL?m)1Ajn-%Z!rEyV2N8=2KQH07?r8$0aiA4}IRD_?8$NInWM8A_x6%g+nlh0m_BxVL-YyB^|F6vz$YfbqnDkACY+h zN_BSo)mEw!B`zU<*s>t#UXgUWzLgc-zZ`!6s!;>%EnO}OWxV_H8J~v_Oa`I-_C(pb zUu_bOD3er$xvf?-p6dk@56)vk2$Mjcp!;|n8{EK5Id8g}{P`RSDV1`^KK<>Jdo+R( zm%8QT8KeQwLQzI56OeDMa}VaPXbLR_d2pvsOT^tr;p9X*9st0B91ugd27dT^?f!>w zbLIf2uc@UV3cWDk!4r6dc!gCOy3a`b1(@88Cg!^tSKz(J5Sz3Y)z}`e0*o^E0!ODm zGR&9wHebMb?_U2o2xEc-l(L_Zs!r@T6R`>|Oc%H19yvm6!$TOm>&PU{2GUBvPi=Z< z3BWC=$hQ1#&hk}DS;Hieab)&T)vt8LQ06->jfu{! z&ES{O*?O3CrgNj*#w|{Le%2t`RsZ~n0eQ)tRh9tqe35>EkmE)00Fc1Bl@Ja=yC?7? zKZnDf+FS&ym%{fif-SD8W1%b(Wr!fMpgK3+&~CCg5%*TMr&dGpLj#4W>OB4#8%Oh< zGk3S-5eyXuc|u1%wOhyfG)h3^|-&0>SfS*6AP9{%(@ACZ?rCjQec=C$t=@O2H?;FUe zdGVa%kT+1nqHL?=^Dy{SZSQ)w0fvrRIs%d$zc)yG3AcVp$$K&g!6O^xSeB10l>GDy z(&ZHtHgy=z$V3&TXVbJkw>eoId{oR*s@saSh5!u$8IifQd44A+0r%~hs=UIUPz~aW zmh7P++oZoa4d+0*Hz`rZjr|G$60H=HL-Z?>u*$jf<3hRj=b96>gUeFprQFVdU#hz1 zMGeNmQ1Uil{Eiz#|H|{-vV~O@Fnwf+G-da%;YNoG-eTsZ4@dB+WWBID#%G@J3B8Z< zKuhYJh%X**AJ9&JHT_}moum(e@yW*D-b$|m0h#PW383CVy<|PYc*1hvo7q1htVVa( zau*&sAn>|Eq$^TH2%PJ#y8FQ0otl0Ct=kwHdrw~8q;BZ>(Eb8(p~U8wM;j~xT={i* z-IrGPfx+J}dFdR>(TQ-tKF?dl72%?LC--CBf}RoA@s^9(<`yq8u1(I463e zpLrkV`~CR61ORb6y$Az>y;jH8!}LxGU*<|`vA)FLZHzz%=D-R((6;Pqv-QgRxBu~%QON-jY!3@mfuLD90+~;|tC?o0NP}@Q4yvos2^%a&KxzP--ht&H~GR{&;9{E##6fQ1n2P9>TFF;d3PH%mZvXMBr-sm`~-&2sWix_H-N z%dg&uJ!W>_uu3-C$SQQ$=XoP0-HGv^XTY(l0jEP4pSJb>blji;Stm#70ZA|78JkmH zP0PdS8*xvFgQ(*S>YH`u-)*29)WqF2{GK-T@e=y=KVp3QTVXV>RM-rIBmf;ghdfp+ zu+^$nw0o>Rrl||`<#r_tx-Yae0NXG40r)S zK_*V4M}}g?T9xH|?(RZZ8dJkYB=V4y(i!jEs=4xqHl75FioL{?>x%qq z+q}GpSFHTVl#eI}-vHptQ2XjrHp>R)FK%>5b~6DUXCfh&r@{4yevy&!pr3|qy+z2{ zu9MJDioees6b!T&z*`S?Gyi)H;X;Y})KXNMfqeI{SvGq(GfOxzA=z89ci$BH-p?t= z94k70!UbL;z~y#mOhX~2UGqOWGOsVR1-*hS;?J5@Q~&_Mg$$^gml|1-Y63m7X~r}S zlfBKuA$O@(IKVqSo0pu3o{+9;Kdtd(PNTd%-J`QUZy{Lb{=j@B0somE;b<6&C5&$96*k%O-0+h*N0w=eWsa1uhRuV9xTLVBk;Tqr~0lZ4-Q z0GcpO>T#KG%=YBmzjqlup)GuRv!wPpYd(1&2zhM|k~sdCgf6#XZFARgNG%B}oCs*I zV@);e|FL!5@l?L=zod{PS=k~5Pd7XHk`@ZgLys!6lUC;Bq?%C?k(Yyz+>&Y11FF&ByoahX#=(%~D z{>PN~^|x(5$V|UsjURI`=yM*YL4nu1mp~*o=dO}cU?Ybo-0(YgC#8U>`E0TT+j}k7 zk(T-6F=}1eJvSbmUe_KwSl2=O>pwsd^QD-?lQvt(yk_~Jx$8!sgHw4m1>u#OM6b~% zxqMz^XU4+gqjTOjd^}T0S4~+is}rh+D)fB09}b)#va~SEYYe%s;B#C)&^LwZ)YYcx z)G?sqa3hGrsY)TI<$a~V2UTL_!}|E=Oeqzdw~muKeOt(>XDBCuMqCQdy)@;iw7TK)&yXyS zjv(WY!UM-3MjUe!xe6^say9613$9&~ED0e-j&h2c_q3B#BV4BQ7XL<$!%zCx0m)^a z^qH3U_(t`n7;*H;`FEe}ebdpw501$x2>2AIc6a00aWP?j^!(DrOekGm^C{fL;pxzC zCAmT>xj*`Pbe-y31p@cg&qv!1!-N_4j8x69%r%)<9H&o?gnZSU3SLiye(@{`iK~}^ zMe4vQZv*^d^W?bG@>qPhAbB_W`T08cp9SJz0G{E}-Agr^GTr-m#pjtFC+mQRD)d?GpTgdQRrF&eJU0 z_EO#8FOKLd>0?0+u{?XyBF;s)O?aDosgqNpQsps0nG2LP#$S)iM#oYmK~)JqV2NuU zFJu`IU{4BiMv6X~V*J$kaB%&zbM>>GpiZkKP^Ha!x>U)N8(4g7H!K)fn@iv241LXp z6DV}pdE1RS)z}W)9&JvMA8#J*$l^!aCiD(x?lD_iOjX(BaN8Wa`@lxksFzbUH2jd(v|$t~=Fu39c+Z zM=1OtIJHg#)H)Z%WR7z@7J9Hw8cIp$$k}|OYr+?;Jpe`0Br5P7i{Qsq^xx*(e6zB5 z3;8B~G3K2qTf0WZo^tk-oX9D=KVS!rGT#X0D3fjQxfj-jj`>kg_5mzO){EhHm2+@2 zQZ--N|PlAAq;H9F=-zae`U42dI3TWcu;H=Zmo@x`YAoaMYT8oWBmo; zAg6Y>r;Mv|w6y+p-h(ys@y{y&=_=Ih;H^Gm5|ock3rwLV6_0jrkPQ5B8>7 zP691;N^b?rEiy7bwcG$~u-woWw0P+63t~~rb-octZ^*`xl7Hho>1DgVT*SQ3n2kta zM?d<6$E~3ZxP%*Xf@`C%+PPD|-R)7XU3F4yJf;KD7rXUd%<(x}=+D|ZEzt1EZNgh_ zL1A~?G^(MjKNF)~;Qs50D00|$a^05n(ySVbE zzNO_VHJM#l3S#6{MJ?!Ht}O`(;na6S-+aZlzHTf!9iQ|jG4cF#U~xDC8H&Ym>g=l- z-*7{C$?h%hR^bidDcA6v`v>nPj$k_G$EC)ZKP?Zko}Qa7K;_rFJ%BZXQ4pS{>0?0- z%QAD0R_Itw)qt9M^2MxtvGTaF=HQfMRqOP|b!>1L|IrnUVPsj)e;XDdB!_FQL>|G3}40# zAVliGs2jW-XGYH-@DZNMb`t-$65Sh{HZNsn(du_S}pA+vQ;fRiW>cQ?O2d$ z_3AXN-GX*{HdU#QqwrJG*}VGRuMYkxgd818{Q-vgTQ0s%RHXl0;Hz5JH@dIn?}yfy zXU!OHo?haDA9d}kl-?v^4K?}iFOxC5Ri6dzmhw&iq3?BkzEu~2S`LKKTOY5X1f`a& z?nCY2l7sHK-AuF)ACrc9&W~NiN$y>v{7OBmYm=aU+2UGBx^T*P78%~eW$e|h0qn+ei7KsfX3Q}E z`In>m8T1Sx&`YS}lR6cZzRCY46_HNDUI1NI;qLP?9#dgxm*$;-;(s{#ejXnI4~Ec< zTj}7P_E&`)xB?$8e}XwXRp(VlN9woEnP4Ba+=~@rB-B)i(Nsr|I|ed!QNU8kR}eKm_W<3Z%tJg_;~4Ci7rk6tXKrF3lQRn+TDQ1=vUd&W zqaeyd`aM2J% z2zf&EySvT2p+8RXne|yHlVI%e@G4ubKx93heUP1|sC%>{Fi{h{m^oDBS-Ts97la=K z;IDTj{jEQBQnbxNe}le@Lidr^$Mjx5smA=xChC5r;YVbS!GSKVZdk~7oQ zKY9Nn%u?j~9cfym z^r+auv)c`o_DYjsIl4Ipc)*CyGyes0PB{Yce9g&-5HF?vu#FDljU@b8*PkIG>>`OlOw(UtBo9?UmR_jX!e z{B^_3j|rHmn)#K~r*ZMB5AGm!tYeDMbp=VBUsKe5)2bm0D;+9ja+gd^WPlWHaZfor zP}k5y87ebx!(SP;-o8Tp!_xc7&5Ej8$@dSMMD8kXkn^xDY-%3^T54=ipj5ce9{9bv z%6q3b>_^m7S)TOsK%NbhM`q~YrYg% zLPBb+!#+RKY40B4uX-sA3eRudIixMOy}o=$(yVO;7Ec9Os$4}&``DRcZ#|@Cyk8@a ziXGdZgA@)ni?ez~U?F4#7Y}0$w4Q*k5>9Hly?lQPpTBa~@LHuXiuOJ?^Dgn2SU*;2 zOcn-^Y^Mglc=A^gvRD25<{e*|#%$vcn33pe6)H0Tu2zlIM>?_?PZW)cpvu^LQH9z? zdgR%JEg@A^^WKlI)pSF`MA@6(vHam=jxLHV23?G<{Dj?a&||*@CvS-8%bD!S8k>E4 zCOqlMk#yV{pPB6P4(%bs?%IV^U~*idx?|VJY@Y})`7-t(*~!=mvL#vIheEaf)Mdo# z9YG#erJGZoxOctf-SXQy&dtXTRr61W`y@Xd9i7HQ+U*|1x>|RDWNPWXdoDG1)pEA_ z{VT!2|!Sc@Y zKISLhz-zBq-2b0dGNLMkDLpA9#~MKEjag|wH+Hx4i_VSAqNe2V`fyx)9O9q&JC z`@3h+T>H2kG>$*t1W{Qbx#^^OugrH70J4Mx%(H*b82u^n25{eUS0`rBaRaE>JL1{i zOi16jY~R@1wuMm!7n^mXdBI;y=g*&iRC?j+Q3S8|$8d3Ri4nAXUaPaON-#%8Gfge( zK9i-I9jw9OWqTS%)YiX?&Z-oZvAz$8gWMnMvfMQ$&!2R`O7>0P9@5#~shDj-mfJs> zKvDhCLsJh8W+f*cs#`l%?*#bXo-C+<=K5zS%Ni+4w>_3?JqI8+s@l1pG*>Zas1Dy5 z<$aPu&colrv7wCa%&R3r5q~uzmTx@|Sif0bgGl}NRP%1a{mrTC1-vH|;EEW=kJm9w zzy*}aF0J+b`mma%lIwEj^@H0-zh*?r=jj|)x_?Jg8i7>qy@`_q>I_RGv6Q5 zN0Aefv+q<*IY~$ycUv4w?+laf*L#7E++OvjbteiR;SZI$%NUo?c07W; zxQ$d;`r?{oLIFYk2Xe#6s^irEua2^y21Xe|UZwo~3V3>g?k)^hvhVr1{TZLqFcSai9tjcy1&a)Gb2I_d2gpM>b4ImT_v9$N3de`v>R0;9`VTi zt@0c#u^{7&*Y}dyhBtr5J;i>U|M201W-Ql>nfvq8R+0VJNoV8lg=9Z0nl{4!6mhgD zJ~92>X$8$YlaF%0ZC7NNy=A^u;h_wH@Cqbq%ldr_KcRU66p}|TaHKvVn`MEZXol>vLy94O`wIJ2%qAmu<#9YIk6`qosncmsUS^v}Qm>RrRK7t_<1jEPVM% zXUk;_ien)q7H4n#X&*d7jpHV$sY^0c67k@bnOXNwralwfie$W7mye>=2h7V>NM$Xr zn8fNZkOHT1F{46ioDD9=_-vqE%4w?90Hv#&e}poEYwxrc5(zaD1>AL%G=s3Z%C!2Z zZN61GbBE`vP>v&`HKr~2)h^+&v%ELq=*$9UL$^7x8&Xec1a@p8NO!2Y1LlBB?NP!T z5`NZfGl-_<1aTI%Id!TAI+*SniMz7g083o5&uE%y^-i|ycq{u?MGrsuABLxM$6@QR zZ2)VO421uI`^^^j@kSzu`#Jt|!lO*8+@RpkFq1vHcGFLcJQ8$?Iu*Yz zCWMym(f=t?9bd5EIUqIi!jkBAtNO?#GFQ(lWf?>?tI0%C9xY@gy8d|K41SyVUV2slruu-fn|5s5!NOXLyBT?2T02SDQyw^YrMW693_0_?-VxceGLe*N# z=cn4=K3;!TR_)rZIZq|}A61q1M1vIzdfA>Bu>3tKT*8|RT~Oq57;k%OD=zt)(9pdq zWETDDc6vsT4`oG|n!J+-U=a8&53?_YQj>CVZDN*(_dd(=?Ah=@x(|h22;44T-z4*~ znHUNS3cy7G>oF6@M0^Sp5`h`6P_1J= zJ&a7KHlgXxE>ap6mXGe@)4l7(Q=c*Vd@KKI|DhBYufGXPiIj(cAc}5wA56aX5p8AS z(4%Jd>JPhBK50IU{`+~p@jTkiRk%DtT*{|h?5#LJ9i^@raHAs)W-wq9DBb(xk_~pq z)2`l7wO4<-&~>>IfEI4^??Vk2!w+rf3O2@j-n<^I`|M^qn@W-qZlAb_5~5@7@?H_j zIL)B$WBe%URd&7*{lDT)2qM;sXYEW!osOv=A_ey9-QhT@~$~= zd$q42{HZJe1`&@l9`3fNq)W29k;-l!=>l_s)1-l=9x_%%#Y&>B}r3*i- zAL%ds4uOgRa0NsU;fKXQ={E zJjsp2wU!cm5w2AqRu|0&xfVOyOr$t!N{T1lwfXh$MTFME>V84nM?C$KhX63kgMUFZf^P(zoA6&!59bcl=&km}ZDOcjWsX{ea zmYxxCImU%NHah+zcwW|{ zhFm?9-TAKqE*4Iq$WbBFXS=A1dXAuv6Dh_&)5b@x1zQFsL}a z04SD>qGmVK?JS1^G9jJr2+2O;(X|U3^N2UM)v^{-x-$>Si2Ab_jKMTL6a4Jv+oW>- z+0#YZ^a!x2%mvB^cNzXD9(ofidQjNYtc|a+rY4$-uJvEE&u9fuE6xW zkT6EqY9`!M`f{Zp2~^~zq7P$r&X9c2C+deAQjvoN0!I!4JnA8731;Hfk1JsF#4dD- z%qRXLsxHqsb|${B5lj^G7anDFe_S;fO1Q)~w6*eQ#M%8eD3m1p)^uCz2<3>p*3EUF z%D=UEID&WX-7fV5hT3V9rO!7|Pt2%qIPAcv%2!hVZ44(>pHfQWy1~ghMS%r*a9q&- zBlQE~FN$3Zcm3h;ek+^zZCKH_@CLp#;Hl3HepX+~d6wEbLZpYutdYE0BP(*(? zdm)yHLpf5X1ZRy(ijX@{)t;-P@S#>zQrM>La?c?BbCicStQWxGlww(*<^4Rx)S2vX z?`anUgQ{^Wi(@|Zb*Bo8hYO1feSmL&&fLdj>gIW*tMz0!zo=o^sPJ>pi5~!mgQq!w zs`WCNsT~^a{3HIs`+yu4j3YVjNB0&5`&v&gZ8CJ2eI{Ab+PCl8@PU(0S%F9jHrj|c&tc0_uOX7EjOb0{hMoIcs<;y>u+|3 zMnc0F+v34Krh%)QwfjSzTx~Xy@lJ@&RyaHc6}{q4N3SQVMRp zcbt|VNTT`{kk{LFVaikjRZkSy^Gy7oUJw*O@uqjWa2yTBb@emfdySl9!4znLf0a$en#g+cxGp8PBz~=ljI)a4 zh77cJ+4(%QKb*qIf>{-qaZNkX46TYVph}5}Lgz1AX=Uk{>-M2$F`|7;Z_A}o<5B?n zV9G_K@e~#Vhb)I;(_b6X;-Fmz_^RrnJz$52-o)QxJ5YQmY(E-TXx6(n?O5P!4oRqd)~YU1LmNL)PBP&E?3D_dTKP}B`Qtr^Yvk5;Qy(WW0a1?e zldQp#jp63ZeV20{Q5;7z4}*{N2l6>f7RnkY9RYwK)%STOa zI4JY{H9Y{yLHdTppzv0u=+=v!pqTlg;a#SQx6J1vULWD|i#RD>YtbUQT+&sg)=VAK z-YKD|$=Wl2x`fct%R2%yEyUR{=P|iqPB_Gq?M3>CY8K-9N8|Z>=gr@?0q{ng9iO{1 z)0C}NWS*nOb~JSn9#+5Ko?tSNk0W67tMdcr9ec3~axwvcA->JUqN_c})EnwX{UoQxvagQ1wskC*o4U}V+af^aBlFYM;w)Ev~7K;u2KCzG9L zdT-qlXSMy&^w*2YcR3?1nIk1|j;j_&E{oP%3bq|JcG^xl=0Adhn6n>o7KAh|?a(1s%A$ z#dp8+R90@7lX}nvnPaZq{*dx&kMd-AJ9Cy9TjQsBaN6TwH~rtEhMkR*0}>l;++6oi zR-idElDK1eA;X~Wsqxid^F#1tapT#Y#8($zSUID_q(ZP{(f|pPCOd*8{;^X zqMMN-9>%pI9!{S3o8Y`p0`G$|EqFRBI>!DZp+3ytb<-v>)7`(ne0x)9(G@->Tum&< zcu(BOB=*sNUS(k3Dr?}W_w(m-aMy3~9w-@QgIdOZ>ZrM~DPoAfDebAuu92C%42TDs zOIN=BKRg$z56>vXrxA@kz_w!Px-#7mJNH@?gSnpwJ`pqHrz)pK4m*}3XYT{HymQZB{)pRlvYa>sD%Q)*h<@_tUkoj~drHxati z?NR1WvpjX*{I0`(WZ}hIu1qzBFVJu45Xt-bs4(Mw9&>YVe>$_Fd1TT}uJx(wgZSoB zlxJnQ|G$s#xFR;r1(Jgbs=ph#vJxg1$@ZPVs*PbZ%sYuHip+Ep(cxHK6NqPbD7?XV2(`Ry-m7#s+u5H55wq@uIA{!6U4FaXHGs{ zj&&0jC+asFCYo)0n~qN=S%DY^%T>vJHD5B94F*QxVM2q4Y)8Ep#_qx113=>Rfw9&A zJTsYuU(=@UyQr3my2JHL)^4ghrAmTee!uhY$TW%W!L`t}V-}tLv z;8Xa0{bs*cqef_F4LXz-ZG!l{j93&KgKZe&K9&ND@Hp&k?$H6Q1eV8&IE6HcIU zhy)*uVSrGPI=Y}Yptt)`R(8txbJxo^8lb#)-#_70Ohw(Y352~bOaD-RV*EIV&nCkQ z3eVhWO6}rVq}!-2EbD+1JyMLmdRSYnha8SL1mKg*m*vp-*EM#reeO+JFAk3s8h!6@ zdM6+VG_Smp^Wgr)=iI8}>4mt?{P6Q7(-akVuQIdII*?3>WR2Y!>S&QhjpvJrVN(tj zYP*9KBE^_>$9$E%ZAgO zhze%@haHe^{p?)_g-$c~9N!`0X1zM+-f8YO*IHk=;Yjh~%@wInmrH&5948tHABD-^ zT(vb)P*`Raa+nFxI=?eB7ES&T!pY4spk;NF?EQ^kLiWcNgjnSCH!z!@MxH5xV|UdS z_j<%@S<5ZGEllxLLhz?|{^nP;rdsqIoAK~Wn7LPP*HMUPTkWR&Ir@hbb% z{_?D*X0J3YO&SGkHzy=l7yAmWnD?2p7?`LJiISLmzE%P9FCYK;*1oJ>Rh?D1DCWX} zxWvw$$yLev)k@f2c;ckV15ZWoGtisZL=DY?2dmb}&NRq~I_`d z{d@JA^ZxpV5AV@`4&tBc%kIy)=Gh@p%QHr9+?=w7A8Oc(9Gz>?AztgyygthpN;v%k z{5jvPJCqjg%M^v?YaYq(Q?_guLk`OIjRs*gGZ#>FhkDCR9zbDnzl$4>bgO`Ym| zW3yj5tLtyCO0T6M-v!~zNP>T41h%Y3Xd+cmn87bOujc~Zq@1gA5T+L*m)&=(q)sb# zmJg9L#WA_XkgM{%5G7M)Ovl;^o>CzvtqMX4daG=e3DLL%#>ZY|%xcndBuwLjZ+i?L zoY=wxrL$*ST+QYKj?QKF1~k=|DXlK-qKce3ZuLQyd0t-17J^`P^}Ceo^7+W-j})G~ zY+BAsCJM+{)Somnt#tWRP38?Bu*G|R=yiHjtJS`m;N=PvMg*@FT7m5zA2{lR<@E?-I5WYf3Qbuw%^rMJ83wFxgE45wcYlr&LN+yHkNdZgd7 z;MDwXW?#UuZmsCyHyHw05Yd6lzA@0S3{7$_4YB&dhWg4Aw}QSVz49z}i~kM=THCnK zyUTRNtaUpiR~mfhrpb}0DbfQVHY98%}1^rZ_12l@EIcH!-1EFpsw@N zo(!in8PN=>+9~J;Vs9-Yi2KA5oyDM{eZVsq1<}>{8ewkt>of9gL-$olQJ{3?73+9} zy(_B+ydr}=6ui@v$;)uXXpehs!fSfXixBIm#P6;wV2c9k`orJKTHehZ+}yNsGw7v* z_|sg%Yi1?gw|?y9q{509b7Q;;SK;~QwWHmjWg5i?&hB*);t6_>C8#|Pt@Su0Yn5-E z8<>f;4)j})X5N1prW9Z<`diRuN!n9TIrXll9ZC()K~+H4tl;N?jwP*c<$HHh*Yz+dzCAD% zKcSj&s6*7~buI59X(4>J_lY&ZY8cLc35zQGp_x~i@$Zo+w`O>AET<>0>u&JeOQYlq zKuHsSAc> z)hbMgPKoZwIx1D_eKBGkgahKcnRfI?3YdSa(1%zXj6bY z+Naik>lCFzjI7p_&jiEYWr?P4=?1Us!SxVff^wsY1Zek)-kXUXboBy-bSsT{Jx6M> z#Z=4`(z4^)`&#mWDIza&287d0?Ecp%yJmP}_@kLJx4C8|pcg(ki=+EbEIfX~4Hp7n zY=}T2m>mE9!d|Bja@5v9Qz6zH&L$sQ!-x{_pJNgW!^EU~kGaHgq%>l~I9qQLm(p?s zvD}+J;IjnB9_YxfvVX(YcHf@R_)SqqOj_q<6Foh25Q+pEyIkz$WEWAtoih)72*n{t*;En)4ecVkcm!x60}SC%4QE zI_ByKITg#(M$Q!jF^iF6FS<|05AICBS~_^)OsbZ_Tkpds>sYt62VtxXX%BI+f!^WD zKfnI(NqLJIp2wu5SjculTbKvR!fNdXBIxzXS7PMon(Eq_M6@o9hjVtk&K6yNZ?l2r z#d%Xm>nd!b##98*o@gBaUt^t?nn!}KcvKbYHQ7v@nQ9#i^zsqEBu`jawx+*p(l%~IR;-}~R2yO%}1roY+ji218BMF0wZ zT|VeXx`)to?B?q1dk3qkNb5Zy<7=w=`nx@a8>#T-(#Tqhz9ie-D*l$p9oxMh{0aEG zRmm64yFOslR&{+KG}JiZ&>n~a_qp?E{`HX#&$0lgV&CuXsKXtpZHrvb>_OzUg{n(6 z{xCYjR=(J(QM{OJv)xC-a%70b5%n(+`*t~X3s|d-;)qBpL=jVWuA8Gm#YU*~@}TPG z&7~8v^%kqbCHmPcb)G`n!VHHmm_h8vvfavxUi11_H$$V8TQzlp$ntE1&CELDBE{O; z(Qmw-Kr{AY>NnNlewOTB?2eYT`8gd$p_Ba{kcob|gFE0PReSPO&x4axHEu3HakoS( zaFPzmy=ZsP^%$JwYyr;Fv~6Db{5fXjEKdQ1n+CFGSfB9*hyp}N5`02S{5N*c3Uw*$aM~pK1*;IQO;10D&0U! z_Jhe4?s7!C@;O-oghViV=K*r8VGHK}gk?8Pg+jT|LwVSoK7 zeGfNXptM_p9QQ0n@D|K`^(QeUm-`k&B#kbf$uE-{)FuI z2|pdE0OFg4)iOFD2LahI^gq^z z!X5V#EdaRWPOq+{ZfGVbyL-0f#)$Kv(PrboL|{gnZdHxPnn6DkP0%AoFRrw< zco@z%bvzJ`NWQ99HO>hjXHJac(k%d6miWw@ z&>M`!P z?e^XgoKZ$qeIC{DB(y{{I3W*dBeeU=Fm+dn02%r_!CIqiGhY^#a{YvCugqS7zP`j0 zd}Yja($Z<_i}XlXj#J`C@wDZER_EX5t5|)MUXn7#?DbZ~zj-w^E8Us#Je)2I;{LFDC$!k<>GW=&iBxeeDUW?`$_sdR>`4!o<86a*(3s5L%=7^T7= zb2HwvK|<4fYpqHVBJz>Gn0~|2ojpMwoBj|3KImthI1*LlEo>sLOz-wkEywA^`gV z4Q6{yk_ouGRjE^4H%{6*CU&u}x3{5NTi8SNw%Y;Dwy3ROEcLGMPL)evq|Vr1@Z0kx zC@f!m^Rz9N3a(Xcu_XZB#PF_00kyku66g8VgqF_!QEvC8k;2?{4}wBAI^)Hs!sk z@c1nnW90jSI<0`%(vwaAgeab?V1LP9@2a3(O(uq~;MG>^K&yEo=RpbFYOn9y>T1e+ z*@a_Dqg$kDAttA?NDjMdMh%TLg zz}{k77eO#o17@jD4PF#*7&9vLxHM*nBk8*d^@4~aGC)VZ^)x@nTH}Jj^y_aQtovOY zsH2wwsFHv}mz0`<2jKox`7p9`g7KLpjpz1AnT~q_Xw>LyZ5oUxDZcJoKd&xemoX4QhVY5IK(oOo;ID9fD5x`OeA3`3EeW1l(}=X1f6I)A$0er?Jm0<)ekY=zQSO zz~(XzeeDc|Q%f-Kjv;_MCBeT}iEOk|8Lb52A)moEUH^BR0uo@G;y0Da{_1oLYr(Ux zZyEZMwva?~5CR)*)=tsYUe3w4%z43bV_pui9{Z9dBUu3cQPHg|hJ`fIi`MW`yNr)H zBd2D}bIt66_sSyTw-)fs2l8?^o}NC_bksgs%=Aj0HnkI)t}J^SUbV8=u@~Cj(73A- z1k0CjzdcZ1H_SVk^D}WWkTCCcrX9QG&Yv*V)IFNK{xs zzFH<6v9+*va04_Pr=+6;#$T6|$O&+Opq;L&zviuF3u1;UO|G|1YpgFQhOm z^M7~9crhFgWWY1w>Pq&&Brjja`|EON(sj&SC(Ku0hbIpX=el?~b@vh)Hm?peu1`ib z22SQ%MZOIRI|JnaiOAa6;KuY200_c`UY!din+2kDwGj~ah3B0wL4{(Y_ym=KGDQ1w z;B|eK3V#0ud0Y_v?;j)~*_ks=c8Upsejx)pi?vJ;U-4kl8bnt4cnz*$f2>Q1W0Eic zJ~s+H76extJ9!%oMeEr8pH0?}2^*4)Izd`tdB2kCwCX&OQ3BV#iI1Mli7_=FCF;Bg zqEDkgEB&*%U(<*kX(>Ni$nD9i?0z>0#Q1JO)-)O?hc-S>8+lPHoAphq9?kRQcxN%V zaV}l1rmEt}4~VnQ3($EF51Joej#*|WpYelB1zUH<5_&lerGo(4{@m)X|3ROuJu1dH zwH4j|$3S)Tf#TXB@Kik+=?)wvM((^j8_RhxQ9k~W*|5P#y*V2G=*~N~5pi#^u=DW; zP0fQ_T}TuzZ{d7TO|{Tt9U^mZry_j^0W^2?ZolHlX(XhOI0`np#-S`Ni;gUo@d8Ql z(zQf(Zj4Rv6v{d(Q;Xdy$c!#|Ply%~XRsC#WLkR=hsBJxYZb$-OoUpHH$Oz@{Xk1R zD}G4s7-iWE(ZQW^(f$3m?BGxfnJUo?+uz-GsRD1S!#(QH@JFNXa9{+Gu*Rqd#Ve^m zcAQ&RwG7Fc7AsnspUc5$*4QiPruA$qdZ=2S&r)OHUAd)bIlV2sZuw`71}R} zBJJCfNagF}2m*cRPOUZ^$iY{i2Z{|zrL1Aq{cJqkoMNZI!c{AkoT;bA9#K6-QlzX6M(sPVbv_w%B&i$T5kK z5cIU%eakskL_4jwSIY;V_%^v0Fh&|7Ys4~I?t2DGejd!{3vCZ}PGm~c1S*8=v*>O$ zHIut-8LjE?w;&+0umlydo&6fbSe0Q8RYk~g06SWPeOA59l{-`RJto}K;p$~_`YZkH zW2L=cgOb}#%=g`U1`MBeeBQ032{;Eg9D=bb+Q)jf5pygJG8+tWz(c}aHH-9<=|C36 zkKeRh=0#owbES2xL5dME@&vDzFe|%@ei)n#f_EN-71Rkz$&{V&|A#A)08d{z|JM^^ z%cqD{+J1jaLD!8JI_V>0Go*2$r)QXAMrUj6=fVgsdq7cVoypP8>9J8DcXfRdzX=j@ z;J!spPD%M*H8_&4X^CKab9W%WMwssRpnxqYI%FS9-5vTdou?F1P`GBCyKq~cqV6j1 zFCt!vMm(nG(z^GzYd&9ZFTSN25Q~y@aE#pE*U65xUWa8MXy!CT%PfbL}eQe z!`cbhB^#HRT9$A{)EC!=V3xnC=fKm9ju6qY=j~De+C0iNZ1z{1yCyKybwKw;iSo8d z>vl5#L~G$?r`BH6)p%UCAUjfR(5cezLK=+G_q$7!*&Yg^40yZyqwsS{A)OJTa8KxD~1Vx1Ldt!GdhS zn44=_bFWJz*lr|V!ZM%ba%!mSBD@rqb!S2f`&SZlux9Ef%4-x&B@M21e11qZ#2GJb z8Q>?V+!*3Y=V`xe4EFBKHF_$;0RH2FEI<=PVN(CBNWQp!H(&E(v2v#SE!+JE5-gN( z=MEvn^Pgc$Ww85bK1iBao&IXQ6|2!i{SPADOw9 zKC_J$(Hd(@FJXaRtW(EJ0{z&~{E0rjTC<=~OHInSuVOX3kW142R{GY1U#oDCwAnqB zk9M<>kFx_mZV%;STy2e^nLhOdn454H?oA69mQ4!}u1yj~ZYvLqJvAntUzDPX1fset zKY*orJ`}!^Q6?rL?^}y_WNmb^_EP`y;uiiW&J^O=KZ@Q=2cE7pG3onZrZS`1fxpRt zz@|fEhkrf3qJ@&iIJSzm!@YBlZ!-40O|#o{OL%5a^KkAijywQx!c~*6^ep^I<9}T1 zB!w$k2eMk%JqqCv4znF0izqK*^jX#}v(nKK$Ze@#zjspgh9t!b3U!{g@-48gtmNA4 zrQEmC@h@ka`H8RrkWrEUdRj-towPRa(@{a2)>cbk|pNwOnq^5Q)2PQ zj>O{Vjx+eb^QC1E!x6(T1f3~FPSbvZ2zZDfyAIi=Q%i2vISf3ItXka4@Aw;fQHnbl(O?+hywQnz9m0d+57qKXZtLj;9XdetH~ndV+^%tmmeSjiybUgVnl1 zv_M{AWwcl_CLjf0N5;De=(=uZI@@fy-qm%RtEd!Wldg`TTUHF+U3;(&^EwzxpYJYI zJQcG@FJkjJ{!C%s&cFbUY(~;QmEiM0$-{KyXl=A{OZX35gVvn$-+6cUqVBqRssN?e z49;bGKs7|@R%YoBwNz>zTcR+8mj(eg^oJ|nW}jcVAl(kV^uZx9b-K7Rj3V58Jwqm*}P6jX5*sF__7o}O3^C}J|~J=6=CBXJ(Y>32yQ zng(>;*>pYTDx5c(jD7Rz#x-UJn*FNTBy+wA7jEvCD4OIH<%2ZuFHRCm3^*+2dZKdS zYqqV~>bZ|NM#cipK+j<`;e_=0F$gn}sMC|A3&7zdXIp9?!C%)#&q%-m?R;R2okGK= zAtdAF=lvmMh$Uy93ywmB=L8e@S89=@4>2hhwp$4(mFU*0EBpYqH8 zP;Sc-sB(ZDz2Bqwx|Yni=@*%?BrK5A(AGvll-~7bXQom4kR!An5Fn@mT%q;M8b`s} z_F&ms#kIk0?L`<;7Ma~y$f#`L=eMie=&?(`T(wmMu>O9x>y0j16q7Dn^qA!z^#IiW z6#(#j3g*L)+`}xGLcD5Uf8#b(j1>cyl&wN2-Ci=qUNYES(!ad?-LAQ*pk03~-~O+j ziLEfjph^FQM#yDt#@MTn^v;Y_X3Jq2Fp(5jxE-W!u(p7eKcE}T8?ew9Y0$EniMIwTDyO?`Bgr;K0g_Kn z7c2RA&z**a;wUa;ag@+f@6YVJrcdo~5yfdZDhRlrN0lCR?IzrnX%kP6{j#3;ZYh^7 zWk?zkz63Y*7ba>ARNxqvvGgD`Q8)|JF==< zA1Tf?%y|goZz2$I3-ae!C!r{mjWmN{P~$-^iq>Xhgor?Wu!Z)GIg;-(v4vc#9U~=x z2lZ|tM}MwN7KbJCA#J*QCy*&oTpRU$f*Xw)>++T|H3C$1Vh$!smzG_GL5y9hFR60& za@Znj29mS1_gfV^%OCIOmmuBPNjK3jzwf;ANfi{a_j*JU0?%0X9em}EKVY1O2=8kQ`(_25y zQ6-F6mnH{`HnzNqmP>koD&`4wAQ>z%)Up`GPFDd7jD4F1w;mMw0Y$xuH5I*3U;L)Eju+vnU+$!BWwsL_aw~JaHQ% z|G-EdR>{tmJzdU{T3$RjtiaFqRbd3S6fMMfQ-S}wYkK^Zurx}Vn$H6^--n$>8W4ub z#vhTRuKB@99>WID#&fTYRPkTH;-PVd?LJ)uKR}-Jn{W?$P(;o}7={e$`45Icgp?Oo zrz!E(m4&#BZH4S7Xn?0qYX_%l!i>_s?76!dC$#Tc0C3}$#lwRQjmngg7QNc2h&fz{f)z`fRQcOOj{+h@l_MGwS@fOzEzgI|5&ZzhjeS*b;Wq66%Q+-ba9fi zU(XFJ&PH?Dwg<{YsQt9qd>S{}YCeh%I^Uh8zZ8o!#V zLcnIXil8tPP&I0ks~L@kfIptyMimT$^qAts62nj-AnYlG^@Juo(||5-U&uax>~1m? zK%u6@#Z${MT$87n7^2 zTXK53GyHXYe0+w=W$;q4*RljD@^k7K5t8F~Y=g}@LbLla)O>yNlai9Ik+X~;TDr#d z!F#M}UcA^fIH*MydL4)@!phh;@fU+Ub}twtI>V|MFI?7jLo5dC=~g*4$B$MSpnR^I z`s#q-w~t&!qw#+0Ph_{hQkp^t%-SNVZDs`F7VZq-Aa7kUL>6A0 zu}%bsBzJQzjqirb2tz(}b22EWvMYD~`iYP=QYQM@I^nNUjgXM7A;@&jzvO!hLIiJ8 z54w!3{0>EqQug_22gL>;lD@dyv30mb2dBAwjYQpH%Y*iJSM5*4eAn0)aW`LB_6xTU z0>!)IyLlOxNxW*9vv!@DyT&^*NRpZxs6HQ5_31Xf4kTE8Vl(BZz~7P9 z-red~l3g}`+!qCpweiQTN`+h=&_c6{9tjbHc-jw!Cd;7a5)HPmFd~R-cdBLmp zzTwKrP9I+P7Kmk9=)&0~M`P|Qpd&0L=1<0am$Wa@YgCeXywwq%ShN%$Mn zwQ%xJM8^{;rBf>5<2Cmj0=&x!?o$L&`oL{OjpM91pRhfXY7n$0gM`ue-b4x2Uo%rp z#tkhcEuT0?!^gTgQv^{omQv}IkFbB5H_b_x%1Jk|?l)_>OD9<7-QefUVe5HawHvo0 zCLtt4C;{Rrax>MKX6vhx?)*r%s?-U8k13#)h|NPnoEB?Ep4%P!dtTLC?NS+nPAsts z({pvkW@x{GPw=+Q(_eg?%CTew#uCAtw!uA+k9EUY%ycu?iP$9?aYHA!Wmer) zCb#d`tS-9k^MOov{udc3w1aeI^i@ZKA@RSMDRBXC#Z{g-XVAbhn08zoL+-^HR1RNk z@|}qv5gWri*K4Mjp7teO!apoIMoBT#Ryo*RS)$GAYeN|#u{2dCe@fef_4ZufmsPXi zby5d6sVb?3Yy9noTzbORjOB|qU>eIEKBPZp&L{B?DC}L-na^%Cr>`-NL zZEGcZvBY4ENWJ|+(}+whMftvXLS>bZd@0I=EE{b_8*PS`LjI;g0vmaWHbLk1+{ww| z)NEFQ)sBXRrSQpIGygK>;P#5E@0kg#r!%UfnZ>PD5N#gF2CTZeDxHg*=$ibW15Xse zXvz$F9<))|3##K&=XH^`^27X0ij=K3|7A6=^0%ypPLvbbXmd{t5@BsRK^m*w^+TNg z+jIgCrE1sUIp~?f&9?r=$}|xNzDF!kuBtIi* z-aDSk_kSGEC_*UNvlNxA?5y*uS1P5HM1)ZG-diW3GG8rabwrUZD|>~qvonr8viBap z>psqrbDVmAzK`EO9>;y(=enQQbIr-O?Kf)mP@Mm2VQ}{P5<#$S?JC z@)Zq7hYTg^(+&$Bzj)Q}lIy`N{-e)~t^N%@{2%v)eW>CWX6GUAJbI*^4ATrtm=@2BjcKfv2qVvjIsba+OD^`)08{}@5GzfSWxv~^;qWc^(QfgX?A30q!_i`Ai&B*I?`rI% zEoKRuR}b&$swiaYK}C}Ei)JNn9_dqaJkd)$c|JehT&j~(9)9~aj5QJ{HTZatih>B%t4HF~l*y_Q!LZya^U zzU{CjB*MJjl2 zqg+v$(5JU73Mus89&%mO^uPxnT+>MVt@C@K%m{b}H8Q^*I&KEdj!hYV6 zIpGL++?P*raIWk>&Cp))>}a~34`UFDEn1kXxEnO(h1z2e|9YH~5pLwc4MThw{zFTnP@>s|r>Nkd}%85ab4dN@G1RJ6FL z<9jLp-57_3kdumMEE6-yVAy{5Gtian3y>d^j`Cyl2ts~5UW26KB0WPT;1=l>q1F*} zLl+iXAmV|ub>}2bd*H4&+&c#{ecX>b&w~aF9e4_W_#9VvYpKu0U&N%1=erX-udWH6#HE>8JoF)a3LfP<`W19qO*^uzje-V76y95 z&fhcq*O!>(;;B>T{438!N-1Nr1Y8Sr{-mA4+Sx8;2SW^?N){fdo&a8z#{@vzA`a{= zb9U;x^qtV(RD9X%>O&Y0DQE^c+V3!Yr7(q7uZ#g}QG__b7&~qP$8EPy`+|j{q)veZNealBj6Nhu&q2_1YD2H2u+}xDO%pO zClZ_)FUefV7|{Vk9J zZlKmR%{&4*>GOwTh5(Q09Ik(KDJ@e=# z(@9w-N{FMyn~%nz+kCfWWa|SZ$h51Gfz2B5!kykB9Sm%>z6>eW8metgfaDQ6y4*t6n0Z)2~FY;qARQCgL1*>%4Gc*SFClRKWU=3cZ$EIV^fqZg zRxe&p>fK^V_&!pWUxzQUpBCPu9)z`jG!kvRUr63YWmruUe{$n%!4H}VSKe;J;<*n&1`If2@P&g0Wq<{lu34JAv(*MMhVX^%OI^|@ z+_J2uue^pj!guTqra}tbhEic3%ml~7!vEZS-UsRuCEH%cLID~OXHuxsrinBwtvr(Sg` zjjL(R#-}q7=^$gQ?(F6ZS%Zs*3)rSnz6a zcULZIhz*dTbI>CvEM|+Sn30_ylvqZe#PeXY?G{SZR6s$`cBt$dq>|*iuZX$LT10!l z%Cfy{3hXqWj2I0^og*PIPlb7d#$ z>r%Go3BXZN)~!xqjZG<)Fl`{nSuoxs^G-n{?+(n@ZLr^hLQTR^AONR?v{}$>AW7~U+WQ5_Ng+V8&rs^iJw-5I#6Ba6$0{=x={z z{JNcIJcbqlax-Ud+ zt{X4hlZqAtEy5Br_2)@$uDO?Px-D*M*(G1IJk9d~^vB~a1+M{q=UA1aGVShrJFgfW zqBeIaUd?Sdrlqr?o~+46u{`sFxyA zAz#s0Mj5C7Y$qoJ{(uJq%Wm9T4`tv?c`Zi$GCP&8?3wQDDT&<6a`9f-Asd@#y6XHq zz(2)81oRF8J|H%h(^+@|j-ALGKdJcm)~PHj)3*qphV(XTf4^XG+ncP>%BkK`-hVD`hNj#Gju|yPgtPbK6IIrOrCQFoTyPw0>I#P;Mqd~t7uG)+HlGr*!Gi&? zIc@mS(3`o@q0jiW*yoR1oIzVfSK^9Lq&DUNy8)8b5fWZJo0rn8*yBLx z&q;BS{45ZOy>vccMoaN__a)#iO)f6G-%-H!q!DpYqR(lC7LDbr$-pE7Z_nF#8KL`# zySxw2&TSsSctL#!3B+(|uSuA|v=I=u!Xv=?g4ct&B>Q<#p_;S(g6LB%e^3k-sQVXw zrl?FuTxQn1H_G8K7VgtBfrfFeefId}&~&Qc;*zulJ#nS<9GP2s7ry!=iOI zP?bsNUhy%7cWhLUDp1->DV=AT>ps{b)GhvKY68*cP~Lj$E)}rp>cnzx`@bn1C!;et z4UgK71p&4AEm+|XCydIt8VdW_HKyuoajg5qn7HNaz;|3p2j5oH1U6|P`r968cemjG zH879TGMW5aXi0qk*sv-HD8o*4G(PABhJ!@|@w#^2lR1G4TVn|90}>{P$nr$XlOo7U z$xxqU8^2Y4_WDEYlKz%>rsaT~Mouka8uI=G2`?B}fDUQSf=kvNREGBpEjv6Kpqd~P zr{dtOOHo-_>*%YUqDq_%bvR$)QUfj0kV=KAlfGQO7j+_PiQ4Io`mOtccy>uDd<$$d zBKItWez-jtL{U2g+G{_-WX^Sh_%NH}r2>}bmY?xU7fzL*rpOy_>aOm*(N)(`P2S^Wy5J1P%8tT`%cR>+bVr;H5CCQ%XzhC6kw zS3cXzY<$NMcgErdP*uDK(e#9klChi|6A!1-bWyzPmM;c7* z82M0l`S{6hRS#vQjScB42dPZ2%k}7%2aKpu8-x}*(IQ=94OFHj_AE zt@#&b*bO74t}+<1`{LRZOdAWq{%pY!w4T=m8xQ{JmqhecyQnUbcVjCw+mX zHSj;rT#pfKRsDFy6eYrdd(75*O|MwjTqqn4O0o>?z69BK93BH#j&vIw+~KF~Tpfd- zy}%h!zjufNRW$QI#R1M7#`g_Ha{#iF&j?OOGVY@yq@<$F4?moQR=*kps`!s*oW*xT z2KR@2vTugU*fs}uwFEvW{VE7$rcC3}Ct#ROZh6pLpq10|eZO?j?v2BSpzQ8MN38Ge zd%v&1>1el_k9@8nw1LxuxB^*Ro8DL0x)X&|1k^xx;A@k}?yc;+LP<+)lEwy+2oVMr zYj<4m4g38<%#$!<0A9vmF2mm)Qr2V8MCu7=AS`0`ip%mxAYD&+5a2EF=`B2Id^jC6 z5?p>QXC4&dI?cSM}q$^qX8 zLV) zuMwZXdlUv1ps)ZB(dMa#sE7H#jN@!b4gH@TQDw1AgFo*H~`@C$U^8jAUy1@W_u> zT*3?0sdxb7WNxt0gHa(F}=?E*0Pz$`(JU7u?IRzJ-+l%1InGJwX; zalfVp^+?VSetok0){ZNnWC8r#(kalvXlYIUTKf`erF&I?8oD2?u~ReXS6rk9X;$GK zYv8^a+_E?fIgW=v6$HoHlZ&cdgI2ABNE~)Hbp$8H1l<#`1ovq4C-M1#zu%OW4COX& zoP(O4bjh!OjDRQr(*RsGS{y@zk8>ayOJn)H!Y(X3i>HepuJ}?)&?OJc( z@0c;{`scF&zV9-;cvxL3n39E_)5o)aNWQPpfE-^MqcRo5G{WZqH4etE?97CsITxJJ z1LhPY7f<1Fz<;v>$5AkSN${UlwxMv~js^vYF+kC*8(lQp1+5>%a-j*~e`PBJZMb@Y zLwo&JpHzj52B1c(_Pyn=_3e`5;q0b11OA26CnE}GFe$4G%fDgtbi z{Mxf4_>ig#kSZE3b^fnN1-0Kte(2h-fiGB1_wKwrE`5O2Qin{S?O+og6S(~}f(RrT zB`t2gLa8U-6ad<8H$s6b^0t9sGDU&ACK(UG(}1U<>vPRl8xMki^Ra~u z{AkDUSGl;ysCNoO<-06oI3DIhGyptGx_UL?DJX6O3`{082lD~MM!kcSoQjmFjl2;6 z^xPl0cn^OZEHVJlrKpcPu)&;9xHYh|M5R1-r0^V22 zI6B<}9|V!MXl>hzoj{^F5cH@%)D^4x%!G%%wxLK%2AKl#>lhV;3Ea2uZ-4l|DGM$X zKb8f56W$^(4e;K@Bi7h{&%JMN5TvpJN1DmgqNg9W-?Z}sVt(ti5L8{iIjMpawHMeB z`F6iUf-w5{r7*2LGJHFYfwD$<7v}%Q_+a`2kPIFC)*%7?$EDrn@MIGaO#oM(6L+x!+|7~%o6tllY#Ejiwafo*&} zgbnQnqdLgY*eB4tv(&F0oS0sK-yni_u$}+`5)a7p`sdoD&1?;oSk4-Zqbm3SRs&YB zVDnpYPCSaTZa=_$n1BzIB*X=G`-(sjRCr-H2bDm)j<$+#Wpja}(hN;lb!G{k!2uOo zY8>({{8RO+@L+3Obkr^(NI{-9qwX=oR?^o1vT{V0ws>q0f*i6-7WJ>NPhi1b9>E3` zA$V^5(#q0vbe9f~Lw6M1>jmWIu0g)Q!>=~)!8^HYVuQEA!t+5`5%C)61YcBjqp)Wf z9cmju-=#^BezqHV;G8ZHfEFF&`EU2V?JF>8l&#kX=`nKA=JOitsO=!qA?7re@x|?q z15^TMFM+^um4Zq3&*HELzK@(#RFpw=acen0uw1*rXA10`BWjm0W&mYI(J{uOe_QS; zHML17scr3+6Pf^+y|ryt*9IH+C?hH&$M%NI{a7YeOwn`50MFEx}q#lUzfA^!!%sLL^M zMcN_~YQ-lUH_yL%9brs-=E+NrOP3=!X`*O8Fcres_Ft&upg!k^;5hEZWW+&COMUVj z$6~>P)aK)3iyc`HCaOZMYlq?otn!40#%7jHT`~h!B~cA&PO2RHnER;7N{j2{&mT02 zKh}>%xddR=5YUc>r_zUr20L(;*DSz;zHQFCd5pRQw3?ng^&|p?Huy`BFZh@y;Du*F zjz}QHjJ%=R=qx?nbqyXvbpg$Ymd7P+3vcV5xTT|!dms3uf+4m-MtFqS5D55H;|IPt ztpwYIgg(INp|ZViiGisxW3y+;58*9F3Ooq3j_=_L0X&RwA7r})IX%x@#q-K-;Qgp$ zL|YkFPtiRq*~`ovk$3?0aGey`Yqio9Aw0?~5X#yM-zvgxy1~(FHYvdT(i0hPxeERN zau{Ck2{_sxwe%5Ra)BI>Drb-Ki*2aze5gOl(fa1KJj2$&uK^_hVlXE14i=KO1pX2j zY0*_Xg((SdYXPheOi(8y?5YCVG^7GX;)M=q+_fGPO_d&O-{)gF&9PA2Eh#o<$P1C3lF7&A%}F?$Y|%t`OeouZ%{Y| z$R6b>!iz#&;1R$@6ttC!_@oD%ASy_7;SA`{-OUtRSKux{{}1MD=xsSv^~X56-+VM; z0A1_2UxAMSbD~$^rMH6ySnLQEtcwCPQBYHD*g_y9NZ(aiOjF>?YLWpYD5)Mk|Ca#w zTPBQx@?ZG56oIto|E4IiAlsVjH}WywU0oBPfCg{8>n=zE$w6+jN+qxQH7OBU>!7js4dQ+FERvHIra*=H?{dLV>GLwV zu^NhUH9r4F$#4F~=%P9fcofm*WrduSlz+N|(Y7#KgJVO`TPMY#Hk6hAoOZD`=fbjC zCY78wuFyj}xexL(g0sib)_V6Fz!i8n9hk6XC~s?PJ`*2Z9`BB3B$zb7_k7GU9`AGy z+zQ-l##5Qt#vc6a-q-`8W$J!6uq1z(3m+&>RNaJ2-bqk zhOvnko_m;;ZOOjg(_KqbJWY|%h)0{o1Awa4?#$xR$8|#Bo^)TL3Yw}IxyPUrVt~Ws zUp-cZN~+v|C(8GHD``qVC=GSCagLouHzi^~fuO8Q1B)uCtI_<$YlykX0jX%!fKc}R;zaXFE}`fV$sc5dsNq@P(0hRa3pxSR2QGpqGsB?|){@})^6SrF&2UkB_*vF?>S!B$$f74N?MX1@ z9iMdG<`{VZX#yD@Fp2*U&TW(&kR*{00@PdIM?8fDGv>`le1f?bfO_%-&XL&I94~yE z!;87IwN+Iq0plvX2kl?@2=WjH_180h`n2V&;UO829X8Oi*vdeHQl1w{;?be)Ls%IA zQ|Pup2_ESQv;xNJCH=RlZ3ON9a0+;Kw?$_#8W!|bgOfZI8!`XykX`~ps1-3`FFy35 z1+R(>Vn&+NfEhbm-~7DEj4sm$9)(}=UpBQw_E0bh7D_}NZu+LU8x+#}7YaAE1$kbb z!JU-c0R{Xga31NlJ28d;NX|8ywYxc!I2cMD{5)kOgXbO^UM@On6O-QFU{sk2I8KZJ zUsW$FV0a`v3bJqW$IpQI4?K{g6f~T{#Fq@>&EtN+na@p6_^|GZDmOuQVSkZjM-UcQ z84pzdc_uB8093Cj4u7ZKf(+gP9EHKeel9$Y0-_S2B**u5$sgeaE#n8D^vH>c+R4>n zMmOMzP!j-?N&+6?`xXg|8e5!M%^z0;iXDpC8=#o|b?+|*G(KU7UjUYn#3(L28Dc~^ zL^@y1@;|OhKxhpOTu`AOfSUmQs~34q4o_hHH%A(+(t=eHZTEDKK@J&^%C9~}jc6vg z3N8Eo>OOqS(g10Ad*a1Ec(8(5_BwDKf5&1l8L+F_8~UkeeYcZW>R7>Qoe}XkQ%O6; z9hHU`V&6FH2744Q(Zr zAgSu)2ex;8&FhCFDrZ(;?}V*AjLv&V&br07~{(fivk^G7TJtG zz8;I}!CXoN1j+Z%~xFy>fzl8W1~q zDLd6pZfoSvpf>y_!HE%1j13nCxrdKgXUp(ZPtg|S(o-0PXj#PJXPyzSypFZ1UjDYd z@=wSQvs?GwO5|ZN7z{Mmvozc~po=F-h>-+EVD0xZE2%PU=PKp`3qO?m=IbMw^T<7m z?Wv60_0LfqxQx-bSTwp13dw?zoGc1)yNH;st9w_t0=})gi_*pbP?0PRn-d9IOIxzLF+K2_DD$Z^!>#6-EXa(-sKLc+WMX=6iPXE`*Kyz+ej+(+X++83`UBhrmn`D}jCj_5x5`)Df;VjY0NXebgBPTcKJ?14ii0ZbwZ8yqI6 zf8xaD@!h+?T)BB#aXbO42k^beL}A1?p^VJ)a}0P~KR6f2j+#6=8jQF94ki4EP|&=D zsR01iGXSWR^D)9l7&id;th%$%AG=F{sRqTd`=|=QSC)_4;#>y>59bm3- zaH5l}|2<;jLgVCbHKzOnt>#swpw5<>?L4JcgnKlC+Y6gRxddmK12GFoD4l1^;z^wN}p;9UzDsB&n z+t9<*P%h=b>fM_1g~yeJ@adZWnQ8qN0SZ!-K4ul@SHmd)pXenUfN63J!nZ$gK;ipI zUf;#Xhg>kd0DtaRT0Tf^QguC{8sjMa&Aj~2tSg)a`1XUz!yhqV84vIcZ5#mh7=*N&|JmR_ z<}m^Zb|zinI=`#iusfjl*9Twi7%m5xPE`J;1rzCz(3Fpq6D3hy1b%a2d zImQ7=43h$Au~Fw%Dli*~&a_w1ss`*BCZUVJeUh_8~C?gKBJ20nz(- z=+fD5VZudvpz=^pF!k}&=kL@8fe(q02Oc$|`c$KCa#RwO%R8(0l-=T!@Q{BBh`w9t zBEAD(D>@!BHgNn z<~$v$t7=mhHi#eN3)D2Aj+Lh3HjJ76EllY5p5fV%>}|hVRHm9HPf{qPu1dsR=SO(X zzw=1~YW{Rs6n=Ui3(vTzCv%SJ@^H+4@V*ellaJfXy7l2ERB(N-gKlctwKtHtqC`12 zX~4O0NZrfamwSqJ&-IHRd>_F?!HTIjxFeweA3n18Go3|zS|+VoeEtIRn7|d-w;yV_ z%l%L9@hxz9)(Q@t-4nk2!Gutnqx+#f6=D=sICr|rafRTbb32o?%v-Oxf187?%HM+@YGF{=c4P|fu#*&qq<@iDWXe?td(deWn>Uzu+Vs8g3e76zru@0UvE9Z2+Zbb-j_t(jCXRq z-tGt65xT+UfuhGX(z|CJYzj~%Hig5O5*v50X>0UC#(~I2YIyA4F?ig^8Qg9v0Whi` zdHj#XtA2ToUm0(YJ(TKb6*opwJ{%fhCwMW=1zw%AUrh``d>tpDA*Fa*`@*;9btmC= z0F&hC6Ari8A?70RVc0~vb}=_b#^2IF8ENhos7_-2B3`$U@?i^nkcLp1OuSp;_{$dP zISy+=!vo+wdS)jXEocA+#k6lI;EXGP7Mh2@TWIjSZukz+*fSeIF^#n~I?F(c3EipT zNOMWI@CQ=VJg&~A;q*PK(A|aAOHfH}#R%t6mK_2HBO-4)iHLK)S)*CT`7lI9|LcS)Cs}RJO($Ooe{-0kpC;o3+f>t zAicH~kS1UP(sd~rz+ONqLPa4ty?w9`vYc=y;(ADK5~8j0kATA9LgDVhU?44UJK}Fu zi2uAvyn7@rY2HJyw=m({_L5vIV=X3^AHMF!%T=0p5 z$s`kO`gzzifyYBZQ-V&}wts>{ajUJ+0p*Sfs;oA$?x|fR6WA0c7w-8<&+^N1m^zH|OQIUFee($2HRA_Axur6tG#nJ6o(U1r&{NJRr5BL%w%Q)5IWWY^xo z>`7S0O%tRDtGVw{b-0L#;QV~6A~aW%XO+JlKjG|>IQuG)4(l|^hxRqKEyPaefbfMY zws1orH1gDiL1x7%u0JPtzi+;gjltIUOH6{7D*F}C!67A0~x4kWe;PILOu#u1=(BEwyvH z^Bi%Q`bAQFNK5T#y6{5bCtg1|w(f_UL0M2yBUbFoJH%}Oh*FA5N()`7;+V?iPg~J6 zi^}b=jj#?HL|a&#B+ZcSPNOmW^Z}3f;TW742$$6XSj0RH+j~1_AW_b z&MS3&Mmh%9rs1iou6JR^`^Zlx!^G`cyQw#&{Q`)PhVjn?p%=jp563=?%u4{jY~Tjh zBNZ?E34kcr=YmrU`V593-ZWJ(qJQ52M}A1Dzr8^f0n(u&0Aq+wCRGc;lo@x6w%6Xz zPTlPUlN9`C&jVi(q$}@QRhj$DUbC!wLSXyu!skrWnZ3GB8hn9G>t=_1r7eQ^=~;1f z%&q;6>+ZL%Hl3!x*B^tT{G|j`3n6@~g)sZ(**%(JFEUB14t<+}o0H@wK*!HeUy6ir zLS!|JPZoTjz)`u{dCb_kJ3ax{hm)V22N-WQAHtn4-*(UtM8Fy+s4NG_(qQWss zlFvG2hD!e~|w`V^FmnQyxpdAb5CH`4PHZ3srI$qO;i3@r*;U0G7(KWLKE8 zA<~V)i9tL}A3~&Gf_F2iaHX5rr{@cW%{Z)K0A_;A2NRQiVzcwM4gkM407S9%GL%mH zBve!;e{R4XY<1l({}}b&;9t;In^C<~QU4%>2+E}4J^vq^aIney&d*QP6d)RjNx1t3-ur+ zClBx!ELl0s`u*yOq>?oF>VD{Wq_84LI@B-@fZuy5Kex;+~(qYWfj~$RlZs&MPH24QRme0c}$J z{}Ev8r{B1qBl~Ireh{Rkg*yPK22EuuM;e0+qAp5@A=AAf{NTBIZbkD#L38JJZ*~Xq zQ0F%OlBjZSWsR?$Bx7{@*aVZMEmS6F`*p8-8S@wLb;v9%#D+XMQtnFGCOMW?UZeeX z;C13k&QkBE@^?&$1uI@z`A`2TZD-FlU8vP`UQV|tEmiT7?U?hPDq3Z?Y27&L!*|H8 z(>X}MU<#4EbjYVg__c)A##lpbaqAD=k`F`v6b;b|NwHzAi$w_}-mg7K9+%Q-#mAaI z*IXG(eLKD()@5P+WE4?9JCNieF(z|wF}Zv*J?Q?gO3Uw=Hv|Y^GJhJNj zZf$HTdFm*+iu9ab$0U~mlI_dL@gAK1slT;(K$wBmqPiI(O`!>d*~pp+(ixjQQaQ?J zX5vYOM<_~dsFcCut$N6i@vCUy0=3i=!Yc@?T_B@XF~5i^1womU zhq~bl8A0A zDcPljY&OY#jVE_$&8KI+MU$%G<27<6Em!H>*`&;ppx0|wsta+Ik3YP=bM|!J*v%uz z@Xv$U#-7WAJnk{MH#}DQUtBNNVqYagXb;at9c zMtdThx$Ty!AvNk|KeXvZt8yo$WHH#aAVk>Fx%*jVHTgCppgwT?k1ihg1UO=4OmkAg zxw3&^N_-N_2>9i%KI|+mVtk;*v%MUtX-gy6|E2aHXYa6uR*BQ8 z;Kv-?>9wThmy}B8Rk{RP!~|m;Ry{JSH+}-EL6`Vtsbb1IWP@79Vbg=MJ-|EE{%{-+0ig zRRYgpd(&#(aXuZ=Gi&31mHs>%DFJbO`KNsIS605->u^kFW@@$c7=04$X7G6J&1QS4 z)g`WaXr+(VJa75-DtoSY;{4L*^|9o3${zZlP>qE}K>>$4GZN2>>~3lCrQJt7O6+n> zLL6xNmn{2A>WWG#koj4anmLafYGrv-#u@56%wCLF`ty28T$h|oOD^&n4O;u*zqqlk z$zWgkrOvM6$>&XJj^@dVE{z}B2WFDH6bM0l{?$Ej6;qo+yXaBSWY}>}0SatOxb47$ z5wD`M@Pu>Wln=|*S8BwkeFY|(B}e4CbPJ@550LwJZjfwb%Wc zla|{ZQtN;TQ_kg$xm)sov|*FED900H!G@z?A5e<;znKpw`Mp`ic-%{ROC$Nn*7LiP z&d)EEb6=Bjy7c8i74Or@(m-tr=9<+|={b7s#w58{Yf@hO-9?=4>mbg2@@VjvO)&X3 z{YfnRDuqgoQFQP_T?z*_yDDu~rVjA4ycWX`XC~Lio5~NBJWctrn2;Iae~@}y~z<< z;~m@N+Wru@4{EmTL(QT2MQ>N58^qy2p5Ln@*0I;rP0}Pox0m-33DwyXEJFQnp6l%!mpswo z+~Rs%mY@E~X)*un`Yue(>OIU)3-aCTU0F9Q1LO(H8K#VvwXcW}j~}UI$hhic#w05` z2W)EZq+O>~=I5TmcD7vJ_&Ud?^gXXe07}{7nFa^WNCJ^c%@3 zqa)9O`+#u}dg0?vo_*tHg`BM5E%CwYy7Fe0Z!_On5tJ*enleq~ zj@MkH;$fS~2hPC*^VGMg9&#koy6xUq`KzF9TPlXtsevM;x9{NGpM zxvWq?R{v}Y#UI1*uq>2Zq*BNN<(mA%h?)C&88dy-`nm|~g!0h()N8>^FB*UGh^V|T zlX(lx#`13Y)n)Ck*QNuZ2*%8PDkHcuce94%F{GdRb7a>C{FB$nDuYauz{z&G>uVjA za<`;kQ^_s~n6&CgeqF5W)LCjR3GiIEL&{Evz%NozTnwLTh+PffFAL%5^9!7nN$?=? z{Ktc&dcMz!pkayRM%nsu0Ij)Me8$Ffo~*2-;OUGdou}5{Az!_S?aRjp6A!R8R3H=Z z)f-~OcRAh|O?cm2IMD3F7vyi|yb%8v9!UbM!;3rFd4In`{CpG3s3oo_tBze+@ zzr8sflFE0M=W7UYrSt&hYK@cZzW!#_9CL{EOH^KO$U6XwzJJqY^j*)Va_%2Y#~Crb z0G=e1j>#p{loHODT1OvBOS(kY`-xs=zvwaWT8rFU>XXGX@f#g?jYy~ zy-ZN6RBCrD@4LYXe+peM*xjth9I>7fDt(I4I1SZws|vr<6MC%PyARpHoB3GU@2z zqU!YXgE7;Uth#Q;pv^NTd40{tO-q@J`xJUE*Ot_7N}SGaeJ?WazPYqo53@hxJmxRAh5Yyog5Nz9#Rwb zFD%C4#+p_#IMJCjbhuXEae+P9EV8W(!MRP~kb=H%YVB%9vb61aN);yNj$~qGM>Upw ze2TwWUwoa=(o>${CQ2%L~KR>8`1bNQ!?&$P`)t}m#ti}9%5;L41m%oZa+x>Q{mLu-bMBP* zSr+S4tzTf6kGpZ)I>7x&Pee33#jLybM)2sJ39izowK9kEzbw%Y8Szh~Z>;`sA6oed zxzoJ`My}7zJbXREbR{zdR;gixMOC2g zA?exdE`{P`lEh_p36$%$E$7hO2S_I0a0|oJQVa13e1Zm=*1UKN)huCWYC1z_=3-b; zsa;-O(FJ~yImYK)!Q|p7^?K@cS#oZ6>0=#d248<&)cGdhbzsP~f|b%=yI9MQ?JYyh z)sgPBzUw^9RxTrc+%zSxQrkXpbu$o!d$TGzU8xGyJ!$`N7@|qN_@~PD)>xXueDA9Tjoe2wBE-u%VOp`-(}pM zN3%(NF;`m(DXdw$6`?I6-3M3Z$Y*Q2+!`EJq%3RgmOX>KXKR+$?208;KZ$iajMWwO z6qY>SY`t`G_KVHe4?XX#`S=`5g=kb%~qIJQN~1(4ka3L1Y)_a~kQ! zW*+fw_Kr!)g?4k7ck>hyz^|Il)VD1_ke&mYiTDpq($*z&WpJ`|0Ka2CF!r9hS$F zfBHYoBz>JzeZMZ>qd3pLQ>ES|3wA%t(&cPlLo$-A>(bi%`ilwU0)mtdrIUbSZoIN4 zpS%##_ZdrGU#?Z?EBDGbUQ}4?OjMt^Ez081?vd%Kyb^y`c~dY({GNAzil$X{skTo- z=9{>%r;7>{vTfG)blO)p9SSTuPL?Y-nY;Y*`(*xDV`V(p;f3x8b(UE-C_NAx^p+k< zBq8MCcL>UJy_sbz}Tylpy$m=|C9xS_LhZ=tPjoZuGsX*a& zbDL|^WKhoZ=hy45H}$7QD;#(Ra<4;~mW0LQn?*snAk)k;4v*8U(1dF1JCYsJAbZ~$ zxrRE}c*o`KAW;K4-O4s2%uphrr;G}@$zhV#R`%3nx~$aL-Ce;ulf7oOhga9^05^-} zxy#O@=4ozUP(DPs&9K9O439U+^d$`5oG6$j==tHz&byq5Bm#*|(-m`?nmm(#n{=0_ zJg7>%Tt($w3Oabb0;JbeTg`{oX$lw}z7|^a@0U%;-dyT^Dl7eIUGAIXyXGQ~VnmV0 z;EH9|Lqm$k{;RX>Jjf1J)Ac@QcQ-p7Rg>w`LqolV_N0^!EooJGsBQ#WVx2<4$)2)H z!XQbjT{-`P$VGQDUobwUhDrMK>t6ODhv{YdlKyBeZ+&iA`j7c%M8K~r^E@!`O%(p- zw3ySbg29yuoPJI)OBsW`m)HP#yE-K>1@be zt~#wva?i`KyU$rVj}N-`CJY7SP9?9K9Y|g%6uIY!lhvOCzn-m2RW$miQTwf*@cM3- z-znwA9I1AZ#6Kx}x+3|9-goUht6p5Iy~&+yi}-!GYcQ2t>CTvev>bh&aZP%o(PF(x?AXHe)T}$D4_}6X!g|+M z#8V=lzl>Z^_nb|Va$l&`0%_ai@yzGuyxIM2SBmMn+H*g;^;K@JPnK)X+KOGzEfxH= z@B1BAQt^#!01{lyVBW^#8RiG)hQmI=Q6{dWx&-)AMSc&Ece2U zX@B3QoxQ4F)`M)@Y<>kt3183vXi*X)_;mGy6N%EKn|&3l*kjvf_2@w}Q#WyuRQH zPQgg{QlYz`IdzoYpM1*e=b z){SmGRmgSi-!`Et*RnzmbIYtMLRtOLxI7gtCy}+q$jzGh)Z;;FXTG{OtsLvz$JidT zSh)DQ-Heehcjb|q_Yb!a1*tjdNg<30U0%KxG8yWiem!s99b~_H+;4X#N4-lg$go@L zk}R*qn?CbqMg`@YGY(J=vzLY4^;eR1#Ac>qW|i#5!6)Q3*&J(n{pQ(G z1wbOiI<1w4bDBp)0zQE={%Pqo4%SGI2OW2PnbVUuSOm^eZyRkWUd+FI7w64jv0rw}f6v0U0j}aVaZ(A!d z(toj;8g|>7mj1yHW8yE7A z=I~eHV#nQc{`+;zaFd^!$RCuu#dM$ApASFpPvz^*!E&a%?yK#MiH~`5{O^I0 z;ATm#ggm zcz^`N{%NYGHjA^ZJON&N>F*{-sjqg{%ae-PEsivT-)`APDEJTq9h@Q~hErqSeg=Mv z^s72F#dG7?XOC~rU#9y4WGR{B=c@mGXAPX;jIIYcw`c|SF?Knwc1{g4Hx#FRYA9w_ ziuQCM>hE;VmAv{+G;d6GRC`vSAH$c)%81zHnjl{D8j9SuN)-!xves*_dH@4B+EKND$xMJI>SCFnD$L~|ao zD{2Vr$YWq{pZqrz*!Ojn%Zq(nLcmf%YI)@aDBH=Kw;A^n21{7$iMwo;lt8|MZEEG| z%&Wr<_D#w4r+7R?3Nmi1T8D`g8yP3E(Ww~})n#w?Xcxbe=TD8ZFKA~aC?|Kfyusi& z@$LDtd3GsR&u3>chVP_^l~d?h=*D=g&SFDa5L zAJOL{V9*JJW^*NnhK#XD@SEL13mO!*?14tS=D)$Lk%maX;%g1jd9)dKWhDy57ABZW zm`3A&aK`mFFuwDmf9EcE)y>Gt!EVO-2-`Kcfl`%`{P~Z-*kbIL2y>%VWHq}vBmF-1 zGvX-$iMh%rye_n8Oc*~H%Gi(_ZIPP1!|t;7qkKF)Wbr9UKD%R2&sRIvrE7z9R`Je^ zD}lgk$fAEE)Xg9`!(_F#O5;xDY?i9h>-1%4dK${yPt>Pd4IEeTJQ(deoE(rx*(zj` z{A1s^^Ca-6o-_D|G{NYI{3G2&PCELFbQpG-7=8`Z;O&AHF(ddGlw!OZaEbxcvUZRbjC4No>r@Ge z3ct3z9^f+|m)xu(MhWuGmbH}0%uen-Kk18W)W@E;HY^o%vi6P(b}oZj?ts8THgm~= z$g;=4+d*h+v@l!5FLqn=g5RhB*Lq|nKip5$aP5{{F-uchOqvEJh9tj|e=%CiUHqyW zrvw920aN{E%ZB}c*Lt+47qBMa6s_5PEPYins>>gG*b~+&>a0>bFiMy0F?E~yDiUOL zX@+b{zKoV+cd2Gp4lWkiso&0ewBG;n^xKBsSA zd5HNJiO{8n7l($#4cn9=6o-Qos`TGgQ)p>zj6Pjo_-Z=8kh7^m)>9U{X}O4-zVd^$ ztZA9w6p|*w>F3^73e`uzVO!N_vx@{DV@!h%!Ee)WcoB-z?^-nP?GfX2Z#cf*&jp_s`pL#W|Di{=uUI31bii;BzAR*q^uK=P?5^YjxT+6y>XGK z&9<@=O0@Dgen769*%Py{2Tv(7d|3>jiMHMBnxxXwoVg;>!hw`yJO6^+ZK>yO*Mf%i zJXDGPDZ5&e9pat2xomfNRaRnxyQeOa@>5nFkA1|=$&bf8< zOGTVGZ5c+$kj|&|!ckZ@|D0HRlkPNlCJyZRNl^6V`-10FO=kdv7$SUz6 z%^|NY-~G`nDo#p^P$7v+MpCd!OzRiBYfKtx-+_+HVqwva1*=e%t-5)|uD20#d?iF;TVe{Q~TGsnd^M*o(?{`d9|`1H#vRT zRf1`d_xk2bMvS>O;V(g;GHyDmVT#)cCk>5a&x!~T{t2JpT)5V|K1V^4c_-ih^HM|pz@30E#*HCy*x;uPXOk9oKFad1|`(0NOnQp5+_>er<(oANYm@21gTHRD~ zj+~c4ZBbSV>L+=gQSZ{dTC8>Vuw1z0=lfM5W+|7yjCs{}>ozIUYKE>@0OOc6FG{JDgp|ZcH!3Aa zOLsRz*D%cQJXj1nyVv!7e;~8_Fy}ev?mGAF`C^^LW#bc6?ZN8|!yG6cBeUL9m~?jJ zEVSiu$tVyl-MpZP=)a;Uazz`lZtto2%VjaulpEQ)Q3Y6Eu>0cdMu$bX!=?#{CPm^) zjBwF?tJr7WGCC^Ck@F+$%x;{*sAp{IVs(vgyo)v zmI5ic=WL2=6VvY1Js+6$Lg&iF?x%Zs~iPTzp)kW6Loo%$^k2tS;93Nj5xw%a=R|EhX*Q`{h1uq#E>nQr1dOJ0cQ zba0_1e|a4`nuT1vd7M5%53av(o3<~Fxc9_Y;Go$D0Q(I+rQCyL~q)#@0($3!!?esix+WKo`7G zi&84Rq#(wN=tp1*1rK+o@uOSk_g4b)Mr)4N_7T$C1 z3eP^o=H4^UEMy`EW3bcCb(@~?ye5wTM%DUIFKi;uWV5}&ai&-`q#u;+{MFZI{B~{y z*2l9xy2({f+1KTs+yRROIpg%oR>Zk%+?OxN6MhmC2}#)S{JAD@=h-4ENtqx>z33S~WR9ByMf}#n_oU4s zIi{Yf^`GA9xVA+(NX?0y6Q`8GNoowr$*J@>y3P;BC>$}A=~G##4Jla|v062Bd(!*6$_HNR;4WX&0SDYhL+_3%9Y)M`29H!bPy97+Xacivw;p)wA}9 zkdx%vx=G~=-6EHkCX4o8%4q#uEwHwJX`(Ub-i~eGoBnUvS=`gR90+-WX z$jC`+@b3<{QpuccL)z#KoZAV7_sYS#eHdzl)e&o z`8IK67z;p?Fsx@mZ=)}7YEPvteueO1ro}^(TaQA#cvUA`d^bHNYyK;HAnckL-2yxAs&k;PLzBTLB0fQ~b$UFlv&WzzACxl^PIl6a@ zr5O(bEucVpkwQQh_Mv!m(zAAS21uuaph~}>({;N=H#FE-s6yb48|CJ}0hZ@XB|R_@ z!4bme;)U0zUwxsY!yB+pFqM?Bg}HwJ-^Uz(7Ubmoy%H3*Zh7V{yzpdpXFtMoro3w_ zwP^XxcBV+Dj8_70ur@L=_H2HrLS1{jByW9k4iwcye#GF+8>|Ob&^aNSTfqqUk>nJ# zX8FpP+dw{7Dmf@jd-&C{^04qZMi6N)TNz|w^~CqYblZZU8wE(e3VjpAyPi$^7J7+I z+x^qXc4Qcf3x)X}nsRlUdD&OVAZAzsOq7>v7Kh_?#U`Ucflw=FmqX$D*4h^&bA;29 zXPM*1(hRkMu{{(?=nz=Q?90kYS!dzc9mQ7x2L1bdvHkzj!@QQTIas?-a@0DdQ~lkn z-2LuYS;5!n&8N z?5oI4?*dT8G2Diq3iWr~A6s-roDpU6L5Y8s=EBKJby8hil!k zpK9MVF(HX54#$h)TkfB!SD_N8q|xG`1M~gPO|=3qSi0zsh@M-gjS*v7i=b=SJC`5) z+8>H0MLm{G*<0DJo$-PZ|09P~)0a+1F=*k}*XjByeOSGiagZh2X+Dqc+UD%Q0cIX9 z<}Vgdh;rM9?CmgF46oTgHCTh)N;nIp*Y2$!52oC%_u;p20S;frz$k;|p?QBb6>J&o zzIExl15bgoAoL+!36nmp9uhv)%ty*N^hkPZL4DwJ2JZ#YloJBo_86VdoqBqrPw_-edyQLt9f{ZKPT1%ynwdT0Z=GZa*0Wxp zYXw!dPE(s#UPT;&YA#>&ZEbC7L4}r`1EQdKp|eX3tC+G%7)_IO0A6o~G8Y^1+BB38 zIdt#48ETP5#&AVomE4>wn#p;CTl}ghe*|G(WMt3<6We?-F#gIfDZlS7Z!tb{^-J&+ zJjkh_am0{^{H=NMO@p`2!gJ!7VWvH4&YtVrS|f=jfxZ94Tgw&kxpIq;r*fSTWo>HssivLDle^_*;v1Ab8+zx?W zd%y0VSi-iNdky+Be!JgsnE2Q-(vN3=s~2Z#kJosMd*n!ig9>h28Qft|)P_`unU)!g zM?11N1gkEGuOYF$g5R~vRmyLx(!6@)LGRm`eU?_>kj0+kK?l`7cAk-sE;f!fs63OR zQhu18#q%7Md=uXCiV4jDAJ)W8Dlh$mUe}KvKZ@Q|>G$v2S($;8-UCBKmK$>cpopqHV}?8(1v+Gh`2cTRD8Q8t#Xfu9Uzc2g!KdKfNm9 z>bb;fGL)evB@0`K` zZtF!~yoBoGeSUiGn+t1Q$#{WarVKkJ3%B_zdOm&hPHnM%;oq(*ug9DUcQ2Xhb+~+` z;u4_&6D1CEYh4+ZE<`%~LuC46E{K$A0FT243Fh&-A&p4!5+^Jxv~kXyc1R5P6u zrlW|x4*8XAsJI-RuvuR;IWj-HT7r%8e$j@!=T?Zdz?>B9)(FF=t`>8-&o}9

Jlc z57f@wQYSlg_4VPK`^x9pnUM{it2jgn3v%NCi_a8(ed7@OSf4fyz4nz3H+sbBHMO)UXy&f%I||1~!V#Cvg(j%k z2B&gd#d(mGow@j8EW5qtyuh+|pOv-t=JPfwI2`k8>@X!|C8l`Uk+KsrPvRG%mn|TH zTaA?H&z~yUsJ~N4W}OsS@4A_h6*bZf!?^bqnBl^w$5yo9{xDrE>PfGRUz@`G0ge5= z!bi*9ChcGajx&D#ZcFgN>Q7wp->*}dyOx{rddMM%nmF`I)`ob>S)83Wj0$r;C?+ar zxTYhz?(#gUAuMD-odefe1@?BFB~+uoBb}m9!?KvVE5g+Zdi^?uUxz8^m^Ud9&n(cP zYf>;=JRY@_uw$z|i$!Y-Fg;|| zvxRcjMFaHuCkZKyrNv5mu@igoaHiM`ACxRcvkoCXXs&7R>Y;!wak_~KtP6WrCdt%6MMs!dQ$egv#5;txE zxXT#}!1ZF<#^d()S8-@Y@ZO%@M=$qInBIPQ`AOj)RC*S6IIN$tm}a%FMxDq1IM~Kyq_atw zeVt-um@v*a-?b}es5g5?&iGe@xu1b>eI+1ZUK!XN-5nVJ>CXyfg2C2H719~8R3zcW zrPwb|)D_mb&r#>m!e>Q?gpGU(fV&s#-EO8m> zA$8vMtZVzF*?rOYQiD5d?8Qs-Pm0|ZJy*O0>53M8F=3{Q)XaDpJn%%b7hwg`O~up75%3#2I(7GzXmeYY$`WQx!^h^ zMUmtF5;Vx=cO{lL_+cv@cxqm!dGaG2t$O$ly~+dlxoh@%m>hRZ&fsQS7>iN3b?fOa z(~(Dpjq?U5tXA(_aPE2qGOa7tiM*jt7ng&c4vl+dI^`X?Zm`)}T5!&O`u>rPAU%AL z6Bh5y#2k|LvrSi2=#YA z)CE-jAaPIl)I|!^*ZmC2u@o&TMb`P=uJMih@pc}P@w{_eYxpgv%}x4-_XD5za#d) z^xx2YC3B>jxYw&u-cS~#>j>6xxjr?*jv<`UDszu(Q*-GylQu}Aqz^E9r`m6bTkTut zmx?kZ>+W(eH=)=yGnNsWK|Z|8di@O-<)fkI>1gAR5ji2Z-pL)A(MJKvR|IsNbvZ_d zN-uw-^QJo5Ny~7)qL}^j^H&m=Sc~0j+J+V!KloxX1xTg%6cH0uAo9J31_cXwTo8yz zF(W4@A@|jk5=4W$Kb=+K=Eto0i4y$M-ZmY43({|p+d~r3?oeXFyYNOo)< zuC;5(zs>y=q@KP;9D~dy!~XDnfEFISk{FvfFx#6YDzVa7;>mR&Eli0)Dt-QVTG1&U zoUdHGD^9R@OWqn>;9#;Uf(*|K8oBziIODh{+7|Iu6{h*LHk_I3OkXkW zTS4(-Z&^ASwOIP-x%~8~*uz}^{va~3rs4d#TEl)YNod(edFl=CwQJX`XIk1S#QWAQ zT`{wWA-}HTlTz)eYyTn{@!@KTDnr>?#ms%hL!$FVh+?`pB`-2>+KR+_SLw~~$gM;n zhjq05az+O`+;#3MERW5_&`a-@sk+FEgy0ES1Iwd^} zANLgYa3$a&&V!>~gNzbqj}>KiZ%Z43#Fx$r-sPmsO?WdBGki z%?kkaLwdn@GwFr7hz<_)AK~>)S7%D`c$IKv9h0b`UFI>lBU+4cN{1S*l3v&&$Pejo z{cN&_m-{t*YQ^Ww)!Ua3@hO%TSuB3g+12zTGa7{5xzwa0iR*p+Rx+>hd@1&A8}apT z8}Z<`z(-`gSI#Eukj${_?{&hq-13JKKb|C4n5-^*>)z)$_zkoGfbsPAw)*U5FjB_a zf6*7G=4keQl5Tz%M0R1$a%6Ix0X2G1V^1%mk@|Kw3s15)R8|*U2AG`&HH98a7W|c) zW6@SUvHQv|)OqHgZ76=7&*H+xUXq^q=R>}}7TUu{#`DQ~nQmXdcIu-1Uh>lTLu1<$ zZbGS056Kw4>k1A;1ROIBEd2mp6|U+|x2w_6j{%t@4JV1q%}@$dBA9W#MR7Rwt|$Y} zPM$>Zphmw1&(U!S@UDWiQE5;$hEk(5eprf52c$B2^ zwmOO$+S5h9Qgh%KS>_%PU^t4 zkl;b3E2NsYOG)NvFL?1l8T!&!BpJu8E{OCqw_iWg;7&4Y(UtblWmiL>H&FM@3np#E zQ_iTn)L`}*zYCq^OF+r;j~z~u$kTvRc((Iw_uy?l7Fr7w@x9g3X3n>-4gyvLR`Wf_ zJA{~S5P&=^WlpAli83YrMMG}sZ3}Uaouz_vPig;~&Up#a&N~2d4jV~3mbAw*QcQ4~@>Sld5&MAqMP`+*^-`ji|ym?>L|K59?dExcHL zLqrCMp0V#uH-7QpkCjSqfnaQ1X8H<5qrWmBe?CNUkSK#0s!a4Jv@^0fm~5#?GEV8E zRxd;qoB^{&F6>F8ZkT`;YOq780b%>Zk+OSGPqDSJ;nHSLJ!1GMaQuH3cK{n0TU*z8g2co0x`5)oV zK}dka&z?kt$Os9@7=0)MF;KMQHUkNX+S{~r90$760jCpQdLLp}pO=OZgrbg6$?*ux4eHZ4wzu;1|C{t$%*FY{gg{;S9PE&zT5<4A`N?=veN0tcGSeMx`& zz#$+?ua@&cBjoMgNha@S^j69}F#GwCNmT}r(+;h2QM>9~pdq`AUDGP)B^!!jcDhSZCH=@5Ij48Fp0;1=X!QDN2=(zxwYAF7m z%|J*ZpqHZ$$v+dhPk3zb0q+i0B}4O@KkQ~@!-TdK$DAZ*@} z>SqgaGqA5)e;yz4iq*oY`FKc}c#y-x1&!6xD~ZV#B!*xkwWNf6EExs$Lv|qx|Ce@U zS6IO;PVeW+5Fdk^ik?q5BT7So+;~WW>Yz*CWXL-~IZ~ zn|{o<+dBqNWLuP zqUS@0j;>knB`>T!NK#7;#I18{>h^v4T598*M~0_|m~)*MzXXY>Ljt3#@ffW8<|wzcHY=EhY}4u^$kvw*Hmywa2l zS~m7(pR7*Z)xRwts!qh3T9y$x4iKUCfB^3DqK+JOQYs~Zze1IM1B@kMo2?0{Q$P;J zZQvkd%uXv~{>+(?korr0&Adzf{kP#KA_-cq?DH}{$ixB8AQRjY_3$nv#D4~DN`1&P zZA;Omug964D)#C;l00xI_Ix;QmRutL-IM3{&+po{BsJjTe+v1ZR5PxIVr|!`jtd#B zN@l-Y#gWDU#H~a0&}Iba^7q%R%u;N;BEp_A%gWi1H;JEwiStx<`@3iS7a(x1T|zlf z0?7+DqVe_6QJ|V~iK(U+pl=r6H9}PWA{PA$XeM)`Q}x7xCl8oG8}PqA>QHBtC#@5? zaGfiK_B$NeQ--dR=hE8@^on}?=_A8SO`fwT_V|JGBt0BpH{tezyJlSeN7HNQy{05m zxp&eV@1Fve12y*?t&Mrfn@Q_M(x(h>jb>ya=v~B;gdelIaE3kqK!lrx_^AdLlFz;= z>eS-UjnW-tJ_FroNDXQxUV4?8P^$9SH|{);LH%bLH0{pkOGTNHozv{+9&XQ4{!&VL z`c${E%eH#SIb^3)icaXczFOL&rZ;rf>ipgT>BZeW6^)_Bc5M7vM1}+a%HyF8l}$(M zZ>vl1yu;y=HROP+7tS_)5;z7p&DSd~dHZZ8psGm5wc zb;1EM0Wo4?6SP``G>*Ww-=+4)a{Tq>Z7}D7#ly9<*od@pYk($EKQt|%C5p$+TlycN z`JUz(pC@^C{>3CP%Rl_Is|YazB3w{}Xt&<{NkKwDVl~DRSaB#%(p1;V%Ee}X=f&e` zIm{-kKxyw@6A*v@&ApT~@J-$S;jQ>5xDKoM3 z{uAHwIS?2lW$4Om$lsZ=LhXkr;i$6mKk-GH55h%gIdXVAk!J^wo(BQ)TEb30p{b7^ z4cU!^5i2$*g_>cE67S=me;deua&OD3oi8#@2Xy%4DnF6MevHsgAAi2f1KK7Q$?Y4; z@EPB*o!s~zy$tOl!HZT~x;+Su+aKQ^{~J;lheb#1)~|o5-_{n&qP@SSeEvi)<1G+Z z>n5OvpG5YbW)_N;cCG3`q=t^~r0J&$TAr7wJFBsya%$bj(Ad|4^U{_O*$=6IrQ+US{^n4uG2jcb!PwMX>9y{pF z_$9#kt0a~D_v({S{i+23^kG*FRv&9?KS+#QjjwG7 zpa!4-RM+z_-wB%=7#o633-Cmw5*nx5JY`G;p3!nVwItZoFH}T9q47&nowwy%z%`v( z1_C+_y{NCpm%4sn8{2 zM#+EK1|S~@qDXvHJFpEnqz4jIMUOX#_>%f7;F(d$eiuTr{=s6Bbcblb`ExnqYuM=> zi0gdGs1``nrLb-g@$RB<;2Duf*bhcz0G%qb?={-}9!9XuSw}$7&-gJhEdOW}`?zi= z7Jd=Hqf$~}L^m=p$Ws$wa-0v~ULiSQp2&j65T!2G>ID2=_k-q1rRhp zLsozG@OeNbBobM+c<==oP_3@c)JJ|c&i+4L6(x`m^$D-*8&v!L~gWeqV z6|5@1-|^SrB!ah!KqKrHO--*89OWYD$~{yj?MFbuKlNUJ16c0i!Jq#{ASp@TbDU}y zk{6065Ur9j5EorPN%5HA&D%f$L2kfBoF(4GxTwL8L@FIU?g;s58C|EEs7ef;oshvp;r*UKrk=WJ2%$ha`5uw4=s<#mKTd#X z?d~Mi`Om%}M(rPEOCS}angM@^rbJf|bEk3(#PvodOY^BB@^jG-Dz$cHe`t1}3BYc3|QUa;?C?CYdWw|A`*Z%p1_v)N#Ar z*H^dKW;%PZL*MYyc-Vpn+7+=$G$o`?oSBB2@0B;HdSi7tAqOw zaPUL>B9?DQrP6@HrK}gRK+{u6{)?b|1OjxAVqO!2yySn_B&ci<)ih7GnIN1jo(Wms zsP?7 z3esP9)!O}7IWr3~Ldzw&%gm>2rU;C?gM3iS^_vYm#ONGMM`IMt&3(SBOZ&ET0u3@4 z{+V<52EqE2qxx-#Wj6eI20=j&^ zE35q%1o$a_Y;-$3_eZGwAXIlSy~J36D08Zj z3~@&=!uJ5XOI0nL5KjUiCSd+0dg3zS5unvkw4lSfIr{M~59sGBhIR&;QM<=hU$lj! zyHPTsP5NWcJDJxFUiU_VK zAYwfc2VIo@$5}$RHM~-w0I2SMNkN@8u}Zk1N2Sw1($)DDBojf~f7w08ceY)y(}y7! zEc#j_q2CP3H2&2-_)jhMYk(sev|D=D(6< zgn&Q^K;HP!;DiVvr-wrF|I9l^A{ygS=~X}h@E%XSrTCb&Bw8e=sl=- zz%6fnO|m9<;VmsFdcYnZ#uJ$k@Tbsa7l>A%(Y|8$xw}JV)5fU%5mKG?c*c z&}pcpMOS9&ZixlW^yHHwMwAKafI)Pxj_u}1K$_#1LEM>#D|dj998pgImA;|hMZ4Q< z_Z@JCMq1P&5hKOU6EY>pzCt-Voxcr33BMl21yEe@aeDvGIO)&h|3|t2RYwig&z*2q za1IBPJ!V(d;x6+eJMBoqKNWFln3N!6fYDB|W zz(FMgX9t{+{|^ZXp4EaEb`?I#Cl=TcAt{IGJDzQfao0ZkLA!eyTcURSzk4(&fn!iW z;6y7#lNNkpIs`Te0Qeh7@m@C%*+oZxR7L%sN7b4H*RJKE`w%Z}r+T9@4X9c1I zO70UXs%;E`d1RLSjJ=H_ZDBTa9S=>IOL&PY(s0x@n(Mx#u zHbOxvYDlG=kqG=Bl|rhxmYA4i0{5-jf7X5Mptb++%>_tOnP2d`2}N1aKg ze^AbQAZ~BI7h+HVvZ1a}b)9&i4>Ob3-&l9-1e5h|#ItpNeact7hE&Cl57& zL;wC+7796yJ+KQnpsW#S%H_W%cC~(jP=soQCTY!Vj5Gb9haHpvN>M;R-FuCH zMyCGsb`v2Vn^qx*i2BO(0MEQ>{ubP|1!!>)Q2qNf5=h3F2iGpzyZoNn4@&*OhihjlAzUxq$}OUe(dEu~LaIX{9)LJv_()46iDfxXU{0 zUXKV#HqlKC933{bw~YPHLWL8yL1W|sT-c)9T~^>DjCT}-Wdowu+&8{s5`=Ar!=m4? za$d^zo$Zsh84mC`p%Cz#)!@7hvus>buPHx9F0PQBb>^$^%uQj8A58(Pfx1d_s&<`d zb%5jWc}&7+yKdqUxE`ZzcB&X=s7uMK`}TETfHD2QfSapaYO~w3u)2X_3@BdMSPVAr zYeQ;q`0^+uX6A;;dPi&OsN8hEAi79!6(%;drJyoHGf-lYU{zUCDVy*g@fIT6zofX4!Fb zu5CzK0NPMe*;riJwFW6YOscHjIGM(3eAtyc=8?y5Z33B}dyqlASl}&~+tHCujNr{=6 zDz$QKg9(h*D#JpTWf)I#_}!p-QQ7XGp+je0|0+kiY)E);zH0sp&Adje(*-FpdMbZ4 zITJY!^&B4+PJhPN{@t$AWXP;|@fV40qz5zKY~Q}AP)qjOB9zalR;slPGP-YF-CYQ~ zys(fsM!A@FrR4+n}K( z4D*~S7}guB;$+y$gToBu;$AWfyWGFOsOUaG9)S@k_&ar{@IeXCt+kYFZWJy=SjdD# z8{HLF4;j_gr04MS3iPj5{7_PDeVubg4;fTog;VuH1JpT*HMMUf2M(gECK{{ejb&T7 zgH$?|d>dsr0ikXs>tS#9)pu={koT}RoCjh_`lcy?wV!du3QNZ$lA5EUtLrtn!07CJ zoiRj7L_mp=A;8Zw%>OHuw^1U33DH4=7FX(|2@sU>dtqBgDN}Zhqp89)P{2=xiy;#6 zNtM$6!Xq$L?sm?$kS~XXdpfN= z?B`ikdEFb<*ONc&v+ovV*UC(=4RidJF1DukH_#j-PPuQADdUw~&L^QSN%ab8Cz%)a zC-J%zcTK8(hwW5Rd^h@ikqaY3)p40(ss_%nDjzD>W!Q)j%*W{N~0gf1wPD{ zDSJPczdk@S@vh3*kf*XjmLI-3-i7zZjQKjYv7&+@KREB@IKizgckO<}6t6Bi zUP%}&7^g~wPNFwahAMr?(!TSs_Em()3om-1}a=oH&l%+ z0;cHq{`wxj&fW;GcHJ}TaQP*1PQt!_Ad5p zxQ6ds74`m5;kp-t`ml;Hn#M#`7R@R1fpbKtn7({9`%B*9` zN@X{N2V+lkZ4?B?F+3Nip<0dez}l3#1vC}iN$P!@G=fk>uES{W-2j{~vYjb1sM^kH zGO4&;+p#iGgnqQSD}(AOoT^P60m<>hH{xSy*hc`9$o8p)Z0T<8xwgZeh{_^$UsQx^@V{*0NTiP+4pa0DRN!pYm%2ZdrkSkbJ6n7E zI=vPmXwi)$4q?kNJ5#&fOIbW)-dE3%k|IuW1%w5KuBE@(X<-9vvKW`>z2QO|q>wf& zBrYYvJ!YqQU4maK>eA7gOrF})W-v;btjmlojROfZ*y`IVGB2W)%!AyrBDH2-v964IAXuBk zIYtU>V+(3HX6bwG6ixX=mYC+YY$U@-4>D?l%A{ycMRRkygFs)s^c=J;kNvpqRD1v>Zk^#c4!!W0#g+uR))(SA>nQ zi;HZd7LQDx_9lX(dPD`Ar_Gs76W6Kcr6ik`l6^RNEq>)Vu5?))FjuGg(qQD^lyjY- z)(pbMPu3z?I!?)4SlSG03UtyX5l!Dl*|XBp$(Jr`dIki@9`;u=$ZhxcX z+LsuB%()CfCG-~Otfy&rxN^5n^?GWpC`-XqQWgx3QzfUp6_O7HLJ(%~;T`((TU4p| z6tA#_U98K)z@&Pw8%EdbjAB(t>5ayeNy!l#qcVz^&GW0Z-VwddcX~^fm+E|N>buXw z(3scMBfY|R#I`0Q6#c5JiW7WjyLuCRrc-8Qym3_#odTNEFT8QHCzyJ5$kpmv*~eTR zW0z(;HfIuoi(8tt@)y6`!mL-#mn-cL9X+83Xzbf=8skXc;!hJ!d6Q)HUfWGAumWK5 z8xUDad5mt0bk{MP>y3g)_occ>I=9798BC$GVOMuvqo|{KpYo=dQQaC+B~`rUl69J` zUjXZLt|Q#%d$zbCRvcm4H5buy=_5k%FotKPz|d(mRWsdYMFJa%QCrNh<#bC9lyi*0 zzOHvF7DbR*?~JQ!s`R%VfD3UeEq@DZ%79=U-Tu%5r%H7(OfZLcw3Z{gH?*sd!8S4< zGvzCcD{rN?-2i2|m|WyykyPH)tQu7;BR#XH+m{$LJfU6M05f-Ex<~LB!aH&mvpUV- zDUN+^Sl>IKS!zQID|eiSk5%<42qU>{ChAIZr-7U?LZ9O4=^^RK)XXa2vRK-QMUT?a zLW`(|#)J-I8i-&bjDAIN5s_>wG_<9pe-5jwvhZeuji0-1SzUPgS1} zqhp{_cX4bt65BYo)ie8W`IG&@qbI0ABC%WoauCk_tAkK`Tm$c2A_SwDaktER8Oe~i z5k@o7LDl$utUyWXQA|c|_DL$T;xC%LRRcPrhQ@8FCwlX) zJa;$mqPiyP@cnZ@?Zz~6Xh_f&Nb(6kCE=b3>{y@sY@c&+*9ZFCowZ~c>{cBIWg(dSQsCw=fJ4{=uFu{QzQ@o1{l9|MBe1-g zPEp2!mBZz!;)>HROb4pYxqAspxL>i^dZ;krM>Enj=EGC$HWyxx-b(V6vPzrL{5KOm{y1eXLGG2>yNpHUt@uDUd^D(u zQdyZZmyMDa%hf!YNXAr|J#*D^@h5$Ky zfSh$#_jumSG$ce@gk$t}P~)u&bEA}f=}}^Vk$ScBn;uTOvgI4ed_z-dUVDb3$!rbg z9}A<6x56}XqmbsXG#G6i#=kdWp)B07?88AHA2oxq*>~-EM&})-d+9_SeP?noophoO zGiFMAE+g5;)}*g*%5aoh1Z~-M1?@A^Lh^H6`79->3S^rymznmCeSGJ(YNO+Iai6@& zo1ivCMwK=5EuTUB#=fkhL)~90vV~!tt95sejYMp{&i~+GbT{g<2n`!HyL_%VZ7Mr_OF|zRn{zOT|lT(YSbdX)!I|BA7!vLb8jTWK(n04i$d!_sDy}!fZ<^DmKdOuJKKLIz6;tIDP!}mG~zr6eEcw zG1!&zUJETlOlt~yb>0B)H0`!I-@dp;X~ee~$~JpJs{4&?K7Y_c%9yTw)>y&&b32HP zn`^H1(^RA$_wl~^W^$XlEBRZdvthj&AFXDUvsoL2a%fGbF;gaY*_e_Q^^Kh}X&t(Y z&EV!;dA!*(7gh|rdfl#UtW=6B%jS%R!MRlzQ;-fFHLkF1~r*e4e%FBI~F?XZ*nnEA^tR8)M8=Zi`1)y6VNns*5N}RALHJ zi_Vd&NUv1P^;Q}hV<-)s>f7Co>nY7pRVUY`eXI236Yb<=t!t`oPXu4lTlbWC*e~+t zOoR3I?X@Q?P({tkF@uJ z@UMAxa*FVgU|WfKMc{|gYa({M^Vaz}29`q0yQ#stD!xn69_O=9-bJdC(o*V)@gC7q zF}EF>lUg+R8l88iFs|`MsWREh5#O?GCdAM@6T)E1uLx~l3GPW_BQ#Ih?3X~W4JK1r z%W*7lC#n>UTq|+*!;D?olk&9=5+vX9>46{c`_@vNfO8Fc%#T-d-ZzG(y z*h^Xxva_y5g_E9RFuza{sH{8Y>pP>OVl!-1>b=6~H*kdqd6@0nH-&7z zJ{J0z9myh9+W1293YpiVA;T_*N<%HjO7OcpM$577@&70`&3yvlvUl=L?(_96H9AJj zV4e3_^SAT@=Wa#^Rq526!)-D5ymEh{>H7zufJQ5u zPlPa4%5>hUc8bSUT9H33byOof+8W5!oQphS8=)FjOO4pon&@sb^!)=0@BB_EJ-9FA zVBvno!Lpa8KmeXPW>ug`q}pTAtn zG?c+Tsk7FLb#?P=AJjl?~bI%&vv@Bta;^*dQPy+$7VoFb$khxsU>jQP|n_t4wX7#sm(pi|X zi4Df}4cP~48~(+M+EZzh>Mei21&p12^PSy}B3INtvf0hgX6hj|BetxAVIG^GU3`Jb z(fzDaB5%3wt~UHt9eV>U2~hHtz2TIw2VLJExyPqTm)Qr&j^tnAU0Zs-pD*DMiTrV3 z&Tik9)!sZ?ea#(Yxqjc5-4SGmRFV4`s~I!BERi@@h3bhg#yhKvJyMm-bl!g0MggPw zW8I|rf9vQPI-@M&3q(G%t%b^!yn?E}eSqwt9`BMCU){L?zbI0uy{NNX1 zFrykqowtPe^X-^cr@dKc?)l&V6|44*WbA}|V?k2Sh0er5O)f3%}^BGi^W=IVZyBTchxc!~U z{k`+N{k)yQkILj}g-4RthbN7vJLYds_kf>{yVHWwS(lakYgcIId4)%oi}K(&4%jLs z!<5OlJkp0sP~v|EO8k$DJocqNv@X6wcL5o{7El_Zi<=x7v03h}f&gU$hhNY|v(obF zQF#aRrp?HSM_}H-N&iAkjvJfztvzW+TN_%G;10k%s&C-x9ofn^WaBco#w4(J>mB8H zR1sNRmrAMlGgkf;-XQ)eLzKp4#Bd1NODRZy7J zlC@_+rpqJ3gChY$@$ZwBIakXI%ef46sZ6yCqgR5ZLB#4D@9n|?>-rUzvIr&y;Ke_1 zT40Cas%Z4&sM><%;>;qgXJNERlW#9150d`DY3Q;bl;&{{dOcrFVkE7lc

bFLSWj;V0_Ww@pbW6`WFs(LmtdkNydbc=IvG}l``9D)y;|3_Z zHvTkG-gsACKOtVJ521civT>m~P7l86eY`x%t$7C_7X!+keW??QdaNrc4OX)c7}l>G zU>lpvq_3xf-JoiV4NDcq6)Nf~7fRY_QV4rWGHqLeA)vNh8Bqg$`H$TIGFDcPBsKA! zh)N{9AQe$P&!6g!$HQcD&j?0%F( z?^U#2ZC>OSkYB#suUh{kzrM5mZ+~j;Kw;oup+~1c%Cb6Wi!)LaT(%JBJ(T;vn~Z)c z(ZlwT58G{0W)_$G2=nfb;sO#MxszHD5#*U|dV4uTDLNgR<| z$H{FK@nphrh^>&Qm*YuM}gvZ)5a46ip$^!;K z{wrGydJZ|tl+h{G;r)w5_ogM~uS#rbu^;;1#fSGJKHoZ8?)?1G<)lYPznfVUfqBWt zZ(p6U5>3cT$;ml*BuZTiQDot!NHW(bznlBvj|1{Ef)cEnQ60aH<`)zkgbk1C`EWqq#h*-Ls<^3Ok~ zGD{y%rJD8TqVrlcbj(zhhf-ACa-0hsWM{gi5_R%h`S|H0DP2g98t>UVwTh2vv|c=? zYTof-pQvN&O5zKj>A%e~=h`n7SS{6jcQhma>UQwdqdg=){`oH5k40)Uh@DMr$$E*; ztkcUy?2n{3ZNJC&GB4#1t^ic|uN=hcB^7ryd?JBVg@^9@n^)2eJc82&abEg-Vt$$fe z#7KYXJhtyA#7;eu+&pya^_Fg@HNws(K)U9;C9;<u&8IJi5LO-$AKEm~CT=os>jN}k#v3*!}G?ku?Zf>U5( zZi^|4yxlb5L%4ceY{-V*B9hZo@Q|ak4j`+vlD)(JzmWa|{Z=9*X`3|(-=D_ZY-5OI z;(D#w;;7{?o{%|h9{12u+1p`8ft_(yY7c7vvGx+%cup^Yq{4(7*lc+6i1G~Ul;C30 zTa{wFTKwJXZaKkT-OvB5{)o-0>l!@%9A9d_Zj(l$!&qA9gX{zo)9G#jf{j~tsNQLw z&ds=wwAiP6#zc1+T^16txqf`Q-YmJwZJ8B7r6~O}$8W%cH@LE;TR7-0px)E-nuPhB zWJ>hmeHZX1?qS=e$(z=oG(PFSs+>rpZ$*tx`Hj4_OOH+%^)Jb%Ez$YbN%nG702V9E z#6Ha8`H|)j`(I?n=02aLPM_Hcbjdxa(4$Vgyu8jOoRu}EVPR=kuRERPGmTxc>Xr1hc*Lj8mF{>pKZQN;=AF{^=#@ zbM@7uH)Xw^o=vnS=a-vW@rNaL1awzxkRJ88gnA=gx!}u>?-H zt;ty=Lm6j}(QWRodbWpTo~xfvI2hg@^>KO1-VR@UgE}QpwK!nmMQymEo1l_-1>0{K zc&rcq z>N7dNiVCa0HyMB9bX!K3a{V7;-vQKA+k~r;j#LZ1C`AQ<59u96K|ra}tI|OT(tC)g zpfu^yQ4o-hNQV%l*GTUj3B85T%RLF8-~Zj2J9kE%%oxr&@4LItKKtyx2T=a+;OeU3 zeQ>1L6;?V`Zy`1o85#)>U_+FPfnP7mX*&D<0_cRb0DRyQm1~sIjb3sp`@#Q!PW|Sd z#VzUyVl5;h6-M`&v5w*o>Q5_9cFGZR?`SQ^+B~U0K>dim(`uKTm$%Z>D_Hc%tM(%a z0DqK&8sClLT_UqB(lv#2kl`*Un{@*9{UtYxV zj3^EpaE&)B)|5pgwayPIb`1ZpD){714kCD11Fs#c)Tpq_RT#+ashyUfzFy~{{nuI7md*0WXSC6 zTYT@W;;qrUHFkosbu1KC3E|)7Xe4^Yt25&GJ3h>6Hs190mBS5TtD9ajc(0A?vo}TCGdV{WZQZ;9L6uWKdbj@W~dftWf~>JsT5E&-V1cY?dY%y#Iud{31?g zm9$mE4ckG(tl*K>XV;$>rA9^5W+TnVCFVt1?2i}Qc`Wp00oc{&=9ItjCqS81q(Ir% z=M3f*|C!j(^9cO%nwi^7RjJs6pkk@8-@O$t>5dg{4&wJcbm> zLYR`IlK1Uc9fBVHVNe00(3YNwvR}h*)|v+cwvk-%LaI9 zE(7vxC9|&USF;_|{NjxtB1^evsDlzR$vQdS?Y*&#Udvj$UsD0Np-O97FequG$vSJ$~1N?wKG5vUHar)3J$O5f~7UL`L zWZpT6nyrCI_Lh?oft{g%ET=rCpKuQ<`;ozUoEO2~zKx9@MwEO4f|9DHQO38zd|l^N`nRzZlpuoo3ivpe-m~xNDh+S5HQZU4?7wGyi@r zgg!`|2EYSNk<-yFR+cZrhU7YWJS(c-z5>W7+=`g~wqC{&Teb}?|&X+0bl(WSimkNtE?bG-d9DF73s45_k82$IMRh zo=(ahOqOGA`UM7XrI`-;=c2$*OigIyyu~^(f1?^tzfjEfCT{%g8gu9UeAh!}^mnvVTCDwjVp_ z-bk&vj7SDjF{Qx29HBQJ&cLTRIvsNa%Q-l59|^j?IK-x2Eo#xa=fDDB60?0eP6Cfe z-k91K=Xk{**-kYF{{(67njUWsM&4gWjAip~zPUgWQbqK)#JCb;`}2GjfPgvW=c57F zXiCWKRHnz)OYc~D?>yrA?$CR$C_j$hW@YYrBLQkg7bY6V8Dc96L+I&ns&nvT_kiZ7yHcJ29fMJ3sNieUJoIN3%Q6L%swpv|j{>=i}J}e&Y32YOT?`DCW z;`{!W9Q|{Nzw?s8IY|+P5D%|K<85SKUzNy5JUMWln9jf0p1yI%G@acx-H~>`;ncgE zg{qFl)?>;*JbY7i$copa_5&j4;aB(u%Qb=E5a7k)xmh##qYakrBa+z}9-&?vBCbOJ zEIQ$i`V@yXHugl#`TTiU><&P+uWAeWEO%&bH-+o>xzY)N^8}9&n(;;ZoFZIY*VWn0 zf$yitPCj-ocY&b4xce;H%{7paVrs z8}P<}=(EBWjnV$$n=~j>jR!Ng5RBWUGI@3{R33uG6P zE*y+%8#QOuLXZ7eUXE*&h4oXX1}Y5#cTt1yWu^AaYa2eBRcKXsBsp{|%4@deIm9z+ zKaW!Y?=J^PxNDDKiV={A8=MzhN~j61L}?`sr)Pm(Y+(J(OK&cKUfr4W>MJa;3@usB zvX4rR;=0!;K9sh&8_~1M(0Du=6(@w(H8~&b%&z#?USz~M9$fhm-}tYvjd6d|5hgNv z5UyLiYJ|#o)zfZYw8)={PVieO7;Yq=;SJD!gEiY{|_ zqpU^x`yM+OlltFlpMI)hUe`4=%&^v507%5g-p95G5jopVNxhEmx}MMgdJ1Xy3TKZ6IX)FFEGps@GPcQlEy#6|!%4VmNW76p>9+irnk3>SN{aL}D&vZ0a*eCqH&%J+BU#}B7?kw z@q)^e{RvojCCDavw08oWeM^2{NQg*@&m_Cwg0uK3gvRL0Swz%F`J%PHy(sOO-+h~q zE*CS9?8W}ve2oGDS#S`qJC=vrnxr}g3T$7brE~W_hp)3cRNX8v9d$km=8a&T$%2jW zb1JTM(xj9iFMs3-+t1S)zq0x80`!DuGC!+3520m^L~=T3mJPB6A4L?q%mtM5s82ZG zY5eDWYvDGaQi)BE6Py|Z`gqjJynSI@R%C4WHg7FbfO`|ex}IsQ3~*$0y{fYl0fb@h zgAD6r$XtZ0LBsJVgSLLims(!#n5eYB4_S4$U_XkEr&!X{-xO>=^XqPIyxK?t3lhwY zxhE7tK!0~4^TSy3z~Rd3Js#Q`_Z)-(PZiRxs#LCPaDD}S%k%u%PE#Xv64cfh3hUK% z^bZ0cpvQAdrP8r83MGWeV7_oEoYwyQflh=}_TEUJ8Tt$FcHwi!>nan@xQ)!`7h*of zwS(*iD|$O@BOCjUt85PV(M#i+ofG=lJaa-m_#;){2t25PD$Q;iraJl_D)DNE9DqFd zv<*2)$VPFJDF(+*<8bmhR3i|DcVN2uqubezIw8GL(-xr`k*;I+jt>v8(Rv!ZOQvDC zatRe@E+-TdX1LuZ8OL`IQXaoyNGD^GI-r^0^`8=Al<007o!Fr6*c}}ps$DL1YE(KR z)QkFa$0T24=!7p4hjeu)H8I&cE0I&mF}Biuh+0DQE`$+n5cA^l9||%36ezug@DvGM zTPsGAll!W3_tBQA?Y-t3viBu{%Sa~@Mn0qO9awT*$|k8C?XYTZQ$wfnWn6v^Syj5r zE!#CoP1W|8@@$I(2G_o-$l5a(J8mYY9zCz3kIE5+oA))mu1# zrl;1h8kIGOF3;j5^LH-PRP=6apXb|b(5fTdhF_qilPG9T`%hhcTSSkY@knigow>`3V-%kL0KZ;ZG( z#V3|dhqG50xOs>VSUIcXm?gv`Wt$2#FaOaPUT9#}@2{&-DGG@izNvHM`6JuP&J@C! z&8ajB051us6NqnzLx&?Jjb*9no2e#-qO@{* z;+B>TS9&XLG=nUj?qW>7MVK@6EQhCmwKr+=(v%;c`Fr#MQkOUF4e-2Oux`ljzxHhQ6 zdGl{My`Bjl{wkEPQo5bdznE^1gVFJID^Y!Sxx_|Qe6UZZ(geb93w#DcU11om6j^R> zoOe&=h0F^aHt_r;4GFsl5Es!qPchOWZ#eDqzlG?jl~in?Uxk7s^!m0l)OPpntOxQI z?Q0(?tc_RqBW7eV%YqIQM%oUhtS~_`KRk^eeX?Ff+0OK?MwrKLvC=Nw97CXhg#kX? z28v)Z^eI8_`Ec0OCDknUrbl%yj2QxoYlA~;J?2^K)8GJc-S_GswH>P#@;Yo32Ol0w ziTj+I`-=}?Zq=HH|B@&sHK0)_yXEe*_a_$pi%nCPG2G3g67~DA1JFHz63HBwRO6*u zEWoNLj+ujaJr?IxCm^KM2M>1hn)%{2e`QnxeGB%2MoDQClSppX=YL5S_Mv;f!NYk? zDt0M55IHXanA3GB5!H7qOsD|_f5107bEMr>!%84q(1e;g1KGJdkf=xP>0e4v0w zmhL{{Bq_WNgKAr(0IGQmyleh9suftf81e}SojlL+VGM!qy)f}OAhvvY8NUGmWG5&e zvr5l=*^G&nk<$-ZvIXBG@6I=eeiqEVx)R}yjk<-euAEy1YZ_na1mdf+zXinXI8W+0 zyu1g1>oo-?G&b73%6s5}iNj^WNGjXj5;7@!^dmVIi602oBcBI8%Rp5NeiT6c$Gx9S zZtLK7cr~p+n&S$P`3XFNr1L${#>VCV8bApo;17^~nmV(%!d)|O$+RhSuLknsg zjWu~awL1Y2%imSr0P^ga!%UJue>Mn>vJDwMZY79%rO+0l0ET>y`}%6VfewtPbJJJ) zE-{9Ud}m-EI^U)vAy+6c41gqTHGy%dkw7yq}?3V17X9By<7lVeO8fLxl1 zqS-SIRGihGp_WSigV^0=jk49ww`KB5sn#ofgSs8HRXOE2C+7^9lk#8Y#D~5oqZ<9q zxBr&J@i@y{Q!#NM!>{6pa-jc@mW#k<1lT}3#9Gk{iz~ybRXnCivK(l=BfI{w@@0g{ zaxh@c#_K`$e_6z|x^YK1-DXa4&G&Ad&-b3#^=mFLa6dx@8Ez2X*Tif;J)p-JWPe{_ zT4=5{m?E+@qA_7ymY8lhw46>UNF?k7<^Ak{dOKmfhV)VZNn%$Q> zUN5j6P5T=v-_46lU?(urHDSU-F!wZ8Tc0C0s%+vK~)6&!i~)j zDA>;p>MQLq?#E+})3*?=*q44X zw4wCC2eX%>#LLH<#J1ii5)9m&#If1Dx_4`iItt2(J1xD1R;k|L{4tFD6(${@u#E#B zuAz^}d^J7Rr9&7)->pgkJFJKDau-xshDzQUP5*Et_tW&nb-m;hp`umV_G9tZfdss! z+eXYk|FApq0A*+~F}O8TDF5HvS#Quy5KY$RMIg6W&bSVGU?I zOt4!qNA8iZC;Y;)3qQ%P|26v|(N*r&%AVc!OZ{19nRgQ#8@EDYnvsehj4t29?2tJ3 z%aH>hEq~@V`GMG9@%EPcEj3JT+Nnin_3|ZwRnq^6ctAF>0V$#yNAG0gr~N`3 zWE2+mn8k43c)xF#lgqn-Nn&=mS)823=rLD7SE=yms3!W{~7`eEV8Xqw-1KTMz=ZOTx#tdeNWWzOB#D_72%_Pz+-&zaXCU)H5N*C zid_*}&%X;Xanzn z)ROYkZu5ui;9|1YcjL)B9d^0I2hM7N=1>2?L#7<`2tRZ&HNP~gz*2cWgWB6Nc&{&~ z60cAyzX=`SWKB!Ya?M2ikAW|MD1g9@9B6Fb(W5!=Qz;d+?bD%2H?=%!`dH(ez!89Sl#?a@DPWuNNy z@e7u;w{CD>jm0u;FGXXcjV$2dX5{>+hJ$`g@EimtgzfR!v>}0o$4<~501|3()STpB zZ^D8rAfJiawX@P3DcxACZ29(W(V-#AdTcF6`$^a*tT`9&8zTmctWYE1g~yK`We)AM zEmGAXJM*2rOBo+n0G*jrfMDjpWmo3;WNPMfYbXVouPbQtC9rcbXWXFZ%RT(pK%oZA ztGGc0;y<9LwDf+gO?Hu!2^|D4^DrjIwH;SY#mws|rvdKHb0PFkS85K^{{wrDSP^UAb3h{*$k& zZUY_c`M?~+&c_@Qoe5{y;Nb68s3Gf33+x)GK=BhEMRS26H+W2lf+kK-&mF1C^42Qc z_+`#71Sso(TW^%I4HsIM0<2ws7z{k3SuGPd8-#{3l z=Oe!b9=K{+2b!)u2*cE{zP_ALAPH^N#_0XT5)8#odjdvZ^0!uT@Kj*lAJBN9+vTX=p)4Qt`dU`&z3F|=6{!oW@JQ5i)P zJaL@1Ca@QH?EVqR!iP1%Ko8!tL|_{VXnlD!2rS#ig68QzzFIiZa|S}Kx<&w8DagM2 zj!&v4n?G*&{Dv~tvqdRy8&t|Z0_Xjj>h1;reCDKO{V=9RV8GB1lg9&C)8-L>P+35H_ z(D51ET>qm=ou&{d^%ZTmmEt?{!IbGOw@tlEF|gbgb;;ze*iURi4G@HOfUCcCtW`0H z-p#(n8LBnmxbpMu4JAEUJ##E@V{s%Q2_BlX&&swid2H4A=_8Pi5Q}Mnh(#|CvKc#U z2q1{P*88q|Z}s(mB<&x*#Tj2*hFoAuMjTgN90yd)|3#$weE2{hdk~7|X+$rZc63Mg z_ozPi6k4Hao@1jnYS9h*x@ zQ;tZ4?c}-}{ka8xO@bmA4JA9N`8wVK114V$;MKO^srnjrr+vWDvtW1j-IDkp-&TC*8bT>qJKO9mVM=J&tF;q7&z;ZP|)_?@@1Po z0qtlmy&d%>ZokNnk(4#$rr7x}@@t^JDF0#$6w%5E3SBx)*gtD&s<|6{@|2b?^~D|m z=sY8y=7zJ$&URb5TpHlv#f-28o{!S=#5|KTCgjPS3~#jG%pi?BdqJs|3-{)gq#St{E`r z(7*$V#U^T?H3jW121sL++|E(Wu4WbQoL*sxUC~k9E8w?yO*u~w=Znp>fSS$7 zNm-k{?stv<@$##k%)8q~uE`!DZx&7o(r$@#OI+XZZX=y&1sJSC%W^PSAVR!aPkMd@ zpqPK(qa|)_>bHu}5XC`{)6+oT7dsVwy7@RWe+1|QM7X93M=zjQ z=|QjSNTO#_g^_cV*L$yPUr3u{vDQzy4mxgCuyvbpUO8OuA$N`E@s@SH+>tEjU3$H7 z^z~UlSZD&;g@8IY|+J=6iKvOG@zJ4y>!^(WgEAAr02nGFIY*36k9XKrC^h6>t_HP;J; zN$DZHYhc|xfG?etfN74h)&hoVg1^-h0E1BfV^JspZgemcN--Po*;&nRi!Ob@dkcs7 ze#X*Z3l(4KBZA00y^P@ensgRV&exLg$2GKq7LouX)&ZHHRXk7+@=xN`%(KT;W<~6P zhYc}wFHJ2xITa$LXF-of+JG8jhvD+EfSd&c7(mhAMuwr19Poj>>f3zvZ(iNFQ{b~f zG8WR-d9S;`n5M+qv1HhkENm(}cw>(e3q|C0pnjvTV~@a=YpXo3&7XfY3pQonrrw)w zq*jXhbtlO`n z^1=%5IgBV{Q=S6=EM>^ZU*DS6GDc4mhU!>36kOIb-Tt;`ZSp`(;qc~1OrP0TTA%Ql z7GvIC+{|%!-+k1*?ip{{P8drb{3E{gCjofJDzl4pZMJT5b&r(IpFO!5?YjM2=1 zaS;Gei;w%vIJot%eTRv??DSf!ao1JM~v>t-m#CM5r% zX{&`03l5~XA@sSu@yT|*68qcTU;ycQC0^#dQT zzR~ht*J_mR$FnJ7)m!#R7BD)08(#`y7R0}PxVkuzYTMhOBz=0nGKyV?BktON!Qj8xco$fAJf3s?-gRH*Y*o z7j6$G2R~B+Ym6gZn_v_+N>yMvNR?#SOh)4n$p6i)*u?^hY3)v;a4TNUTFLUD zs>Hism7<)SPl%{f8&`4J7nEr)#c*}OY)3#3sACms3Z-bnfRVSnQlo3%PfMlAAYf=i z)jqq}Lzl?wILpACn4}`55+FPhEE^jtrx#jONO4-bk-O{W=Mue4CTRARcfV^!-2qtV z!;IQ4wySmjc1vKWm?ZS^cBe5-x@(z=jsac+&xO-$=%$w9h9ZixJ-PjGb*>A>VpP=@ zAef^2+Q>1uDr5 z*b-g@@0tBoZh<1wsl;#MP9!IfWcYD2{VG;UYG6J6--m&4F;1a{qQ3h+Er(LCy`1JS zILUoI2v3gc@#X*Z#J0@}-5U2s-QRev|J=BvV_Qk0jtWZJAe6?4ofkk!RKhA1N|l1# z>dwEze6KfV!grU+JjmK6X5H^z_`4U_PhS*_b;gSgaw(I;iZ3oz%HO%ODB_iN&vPdv ztBmlpkbqWD-qtCSU5e(`{5@3%&EQXmXD;H`ZBhZs0H@R3*`q<#ao8n}&9&4QAdMXB zd)c#%@~zbcl@+e-(x=k_&@!yliOXdBE6gP7B8pE9`gHTicy1QDjF73i2xy(ZA&jhl z3n6kPWAqogyu!xgM{4FNg&k^@Sxs3T<-;TvtBG7~g0BXptE!iu?f1p6xkKihIOQlx-p;m#vnNyW^ zY-0Y)kJ@eor{Czhsq+0dSCriWvzu@uP&o!1O35l#)&EDtn|umob2nKN=*P=lbAfH$ zb7r`@o*FWa!(xLC3}u3sx~eH$Yt?l8S~l&ceu&40zv^|t zsnc8i!A7@-#w}&M5J6eM5#I;XWe+slL;?ela{Q04VDAS76G!G4<`xGUsrD7Z(FQi? z_JDFAS#M6CT?m-Fd=h84ZgFS*xJp&+^*jAubqP_kgl)%9XVoSk2-Orn1ACPzHF0M9 zy*+7f{|2Nl5>DWMt{REy6WjUXE!| z0OZq+5~E>=4e)e@9f`gEijIES=Xwz0VWlYk`ziNX$e*vRixWG{ck9z)P3tFL^jI{5 zXx2ccO=NfyW3@avHYWnJg3R`1IZte;s=s)T_<3>t94^hW{pl)K4IE0j80~i^n321J zu4Ghq4N+flf{Y6mz0Ir0%mAeICWBR0U~&n#tu!m>+Ck_%nry&ItK}G8s&f!syj@we zO7n9s`wCD9$!@;m{YwLx*>Pb$W)JcZ^Wb~dOt+z$ww;-Eg zb!82KnrRPdO&`}$UaWzN5yHao6n#&vjN#1m(60kVF*e?FU|Du!@6r|Di;Ds;u4ZMlhJ>xEnad7z!eIK9N z=d$%L`)>SanldoLH6pdILT3o!vSjR^H7@6ZDA@@zk+GT|ao_M;OZqk#+kHTE?^wJhvSuPe8f8Y5X)P53-fBW(OoK*wH zCoTeIDczAxEF3+S7I_s7n-l%&x;E}hv2pv(-Vyycahtz!C(H_oM;7plgnPKWC4V^K^q;~y{NVPDQS zKZ$BU;Rk~1<-dZ8fV04QgG8BThzXbW(`c2!Q%953+1>)wKj-yo+Ke2BbsT*t{UA+R zvw2b{&E9lqAy!^JEy=LoOX0?e4x&Rv@)BuJ*mmVr@9PWcwph_8n}-y*_YOkTv|%G{jkOk<**Lea(>dP~(z57VL}&H9ZBbLGq;h+KxrV5U@%4*8 zI`bJyl~H0fvEVq?Dr0~k+G$SEqiG~`8+WHUz~QIqHU);8gSJ7x#>Ft-G8k|ua<^-d zF5jT+}+YHh^_SkDjOz6e!gi!H{_%?yMB^f6BKsBn59hrq}u zz%zmVYS!{-ZP=3*;8+*dh{EzIna1Tnht&{Z$bC%;-ESy(+B0Bn@u#C`K(B8#PW9K< zZYCgk`AmdVY$b3rUcc#l`qN5Alhp{JrHG%l_ji!;l26@C74kB+k^I6*$DR$B&+rI< zW>w|_^R8gzL9mpO8k|Ts-cF0Nc*Xa2KqUEDw0c877;qMQ&e+sa4lNhKs>V|dcUiaj z*e?CqkkeG~j@-?8)ERm3yDf%Le`1CAC5F%Y$h`T~nlJJZ(pRfP{ z2cT@4hTnb%CMoXmb^El;qJcomeibda@SlX-TL_;9rnB~gHqDfC*Y+u|^trZcn{6#? zG%25}FjZVCz;P!cBq9PBNZHQ|+CIqLljaZ#Bh8FGu!adGq8BMp6zhlOw5A?_h<1Zwe|gpic2Y&3 zj-CA7FLL2W*gpRuk^Y1&AoA{qFl1j}8D{Z6Z(%(Gqq(3BRZex3Y7Jqfaz% z@{O4|E&y5yh+ZR{_CH8;YB};ikYXMa>e7rleBTb`vJ%O+?IH3sshwEz@E0%&LQPm!8zS*FPGD|ToejHmuq ze4N!p@QL;`o8!PJ*0r>(o8oXqwNy@21pG~brwf`S05pw#Qj;gZdUQ+e zT4}_T2!m;%=T!m;vx@g&nPx|P_GU1hU$5&*BFh|i3T)(pO)bl*l`Soy@5y+`Dx}s% zxiwgL)>6~z-@GFYM4sMgQXblxWzFSdQCzZ|Lr@`=?7TBsHU{M5`IdpCv~!<>4qo5& z*m#q?pLTLN@=;*Br1iwo@SxJi`LkrA3T3dNvc`aW*TL6hG@Zx+8E+iz^D|BNymagh zgLiMFLO5ap`j?YHk7kf42}!EwKSLw(_tI0Z-8l@4Ihwh)E$WbWQ($!38Odp4Hw#I5 z`xP9NzDLV*gv}mm){?O@wqvCEUWlf3q#u)kqnn$~qIc!XzxmTKo7 zt1K?Nr2e|P2?#jp(8q>BeTLLn3c*vFQpi=XTrHqmIW$C>Z=d(e42*}{dnRKXU))( z*6I{b{&!EGoUXx5Hf3^#1S6`#_h?Z|fV$H#cANll^O!@ktDa*x^EPmT1vYh&^I%Cj zHm$S<5C!lXe8Npnxw|wy!Bw#CXJAMQx4pjkFdflMwHD~dFMnDDoOv@KG=nflmzX}1 z0Pyd&!L}&q_86y#_e>VRSyVKh*%bP*aw&u7F5Ijg{0?(U_P{poou&QZ-1|V*r(r)} zuzsn|xv#ano!29s7BzrLMUUUXc`dh;jVQJb`^vpV=Ql4Jq;eX6IVAw_RM1+^Xc|Hj zOjackv)3`Bj^EICoPZ}63Y$;+qgm9)Nl%3nlC>_zK&LzBoxa@@Ghnp2L&0W>#~6L))*{{V1>HGKTI6DwzOs7! zgu9qATy}q@Km@49jv4JkFD&*O}4~^@o+8egK8gTKW&c4 zpu!nTHgeLuW#FWV9DdR!aprX@${;46;zz(=f%)UZjc?_ziU1~Am}Gb}{Y-BL_B>Mz zs?rgxW03dE!#{9o^?Cbg>2m?(%&t;$bmfESI;Onbnwb9Fws-J(=eBEfO4~PNCRf6$Df{&XaHKp*$O;b z-W=p=z!8}i&K(*ao=8aP)H$?TUb{WAQTRa!R|LmoOFEgNXf3pV@I5h*8V!I(mfq0%HuLCkv*Naa=+D#JrpOwP_%m6~ z(BNj&P*2~_7vZs+j?&P=%) z3f+X@D5^9AF$}ORYnCYKi~n>%-sa?97Cda zvhy_kiFrQ!%p}B$zkorMt`8n}`46w}+gyAe#9KojErnP@j~N$>AUOlMZhECApb`>U z4Q?!m%DX>bt!vrtozpCZdKfZcEZi3TeKSFusy1 z?D61iZVd{?ku)lj0ejCKhAaZRzylog4_?3|%&Mhxlm%$$9iT3H;~BKoBwgDzN*$#r zt8)6EZ>|#b%~}F4l$PyG!n*;?DCwr$prx-Nvr%lkhTp0W_or&y>p{_4ZY3sd_=2ZBwojht*GzLN<49N zCf4@}u~GsY1xn5z`&=qXGCb?H)_$j)w-N~2E>ynIh4W$56{+0b$ept{q{?uLpC`YzQVNts$>EwVjznHbV5c@STxkt=ipFUo>M2d z07U=bzhPK`Bk!XOi=2u`7o-{wpa5RSA#KY|I`LK>(8RcLfeDy~@*!Gy4Bd=< zpFGxM;KU%#f#qK#wXoUSe6Qwu@AH;p7D)^Sb2)n1Pt}lp-=yl@bg{$QUij%n7Ngbt zyrZ>i86qU5EaZ-$w}If@Jkt{)l}SK#zEgdvW=L%AdX(>Jxa{770^_OQ+#Cxab8%~E zhP`Q{i!v9W0l2gkwOxe2tQ@MOyN3f((fuTVpZ9x2tSk@$etz~Uga9I7xS$Cm0@8^; z1XO8Oj%n8O7qDa|!b`};2O58aamt6hz`F_VhB<5fu70UAD`n%9IOBU;{hh_zvqcypXMX7Qy6$s_fqd%EpS9(ptcXt_=Sh`Sa>YMG>m31sr8^M&&&)F)Tc;i;x!@kXCwZFMgb`K{u>lQ zJJTwps_v(7ME@l#mwpF>^{O|q@ABx!$y;+_5PqD={FA>O*}2j>|8Km$kMZOmpuZBq?UG(pe{-csqRd z@xEv?S$^1$%xJKdy1a$_;>~ZU_ce(;4;VlGn5mDskyzs@uX|@fg)8~WVu@^cUL^lk zJ!&M8I^nKWVj4+!0wVlsqUCJP2j+K{xgE|iy>j6Z*-^dP-RAP0EvB~+eiXCXR5I#p ziJS||SJwRakW}G}?~OO3!}G%V*B=_%Dvd6+S0B_@Zk4AEuzEY+Tq8&i9r~Qs4NnRY z`q~tue58cnS~}#IsH6vqLNye@ts6s&zxz3sW+%Ic6EpgUysok@UJv#?EjKbRK_wo6 zGrf=*8SSP)ACB0c+jp)b6HlN~KqdE+xEBdn5Zbl3vR(_IEPav#sp>1->jC{T&vK|h zUAN&kvMlwly};yVeS*6TLfmrc!+RAwchF;`RQj2#T4EM`=M^mTIF4Y|c2)`GP&MVp z2eT`9WXWO!oIqbbr^9<@iNEd|p1XfS`VSr!ubKq&#u3&r#xJW>maVbj#|~eHcz?lpXjcrwu2j_PVe!sAEpeWzz-T6C#3HH0bNXgI-!J3GC z5q6-spPOc55OiezxG?QNxk|C=W*LIlX;nf*6Inffy;gkU@>7v6OSergMb_ZUyha5l zbBQN;B}rxZZ-FG|bNicL;NT{kgxiQ2%c|iWx@$v}TAO1Lvm5cQBxJMCmD5%VS6zAJxnHOco(qz7Og+oFVykn?eNiC)aO#LktWBMM*Qe=l(_0d?%HsI-;x&%VsFzRrOnDmYU+%POnqIQ@4X$5ELdF7(eO%r%~n^xTc+ zw0iVOIDr=uebvgM3zBRZt|+$gs3U@jPqX`w8Q~IlF)@ET(d+1bH-c5f@z_AE;1bOg zEqx-MMn>VQ@HPQr{+E?$OpiVKe$y+-yhzCp5Ph1iB=bBiCZcZqf{-`!LO6MZNN|=i z{%(0sK-FCRz~jWm`$ECNx0F#c%#QwT9aiR^54y|jFI_qWPU|_3<61iw!voXxj*mjy zzEgV3abzXglb z8quGK+G3H1!phV}@Z>UhyR1G{3qMr(3>eZ}6v^p71$($4h)h;=pyjdsMEBxz?W>?P z|50{E)6p6Q55dUqyva*wt}AY1o2l#5Jk&M%G|w!aPNQK2Tvdca&i| znZl_c#m3O%$}o|yu#qhFTZ!H%u6w<%i240>`%{z9D!_HKd6Ev|x9!GgQ(!j9bi5Y# zs|H+`Dg_60y0z}d#T+|*+2a~@5F@doQ!2X1G02yo{5lZ`xJ#ifv^5E`Srt|9-tK~o zZEYNbmH3754uTmvsV?~|IKZ~37WcoUo#gnLCh}*leA6V|ogfz2y7$z*R7A($(IWmkga^ z8&j`q=o}aKvbS}(oR2;h4JHqK7EnJeg~>aBv#^I>g<#zw)7}|xta6j$on7sots(aO zE%<8;NXY3rf=PVaG5Q%)m*CedW)SAR2ym3_O2Pw}`mdAzK4CV4#Lf!U_a`ycDBtj` z*8X*)r<(WK6v!z){$hR|5D;>aH5H^#N6jn6Cj8-f&?lA;g#UL@x|x-J`J)y6FehJX%*6s6c=s>K(uV@pTZyEkW?oVpPE5kxLKmn9 zXZuKTelfyZK6^7zE;bjQ5R83w|Gd0)nJ6fdVtlFU+jvOLrDD*poAqy#dB)xE>0A@t z*}3@Ur1jcJ=O*%dO?zS z<-KI0>;+v}a`3B_EgsRM%LyE`6ZA*yU{H$NFVii(BqC#%H`NKG#;@_Yvo97<-r{_@ zp9`l7hIiG4Zwv4e(kos_VOqEvbHCX?3V8k3B$?#32O2Vn3vdNLI~QKxlXCvY&d}76 zaktK^AXE^`t4U#&3kCZb@lho2>8c~=W)8QGx|Qb#u3D_dXorvNK%5#cL5^(e4FrYLV_b3d5TQQ$Ee#n*ZmpX1zVSm*jmH-o}TUN zL*_)n<;HV}PXa#e}<5QSo>tQC;{hBWRy#mR&>y5ZNYHF*t;+;178~NVGp~{3UtWTZ3j~ zU-1JExEZ-5>A1?OPusK9T1l6ECo(Ju5OS1km}c5X|VA2EljL8fN!lxVNt& znv9uok$%QOS@Dmz8UA4K84G7#SA6SIv-hl+dWEGHhwuS}RFN~fLp-KVNuK8MXC9xW zP-wzL5$nN3^i(DFTnh;jsFUXGlD8M-KiiZL11N3ec1P*+fI(28I3L{_bJFq?S!Y{ZxOvGD6A$4ps(7+Tbg-kVR9mW zuD{Esi0G+K3K|%2_PjFUQsGA-ra=4dqLMGF{%Y+N4Drx>U*fhi3TZtzEut}}$QK&_ z%G1~Q9M>rU4VJg|0CVFdeX{q?Yv6fZ*a$})Vc+<(dANvEuX#KpWtJ?`J4bc?5aYqdIFVCH>()`52&x`e5y%mjQVdDjW$XSqXGf(GY?69hxMVXXF zh!G_E2KN=)5-l-xWtSiVN^)mjX1I@4pCLK216)umkE~>%8DHR>!nvdFZ{z{-S-O&91vpPLY4$0jla5L z;`w(k1{Ovu^K=iT>fZ=x;X~RkWgmse^c8f&kD@~mu;pw%rKcqBPOT3qg_>; z5IS0|9AL;~ZDjfo8fg=u#&ESN!QDsd>0FMI#iT{lB6|_C^yXV#G{sj(tR#umR0s4P zlq5OzGd%CiKKE*IJV0K)dz~i{&ISIaERHm`Xj+J*G!U*j*F|}rTf=rBRxDnAe*Y60 zpxI;M}TgBIv6OuR7FhZ!JCgGmh z%k8X=t&qXt%k>mcbfdcCUmNBQk8MkT>=d2QbQY)m7dwh`$d;OCOzQXMr#at|&EO+# zMtLvtRt8_)5Wpb%m>`j608q<)g&dJx#mMLO5_G6;9N7%BxVS1m_U!I&&HA;|^+ku@ z{VH460GvBbF@;4{2Msl0qO(gEUan>!8JK;RKaGqs6vp;@1H*E24H0iQ_syrX{mK&E zYJoOdo}3De7S%(W3sMbc8(lcgSj6ph+{gdfQBy!JA?afriG=s0iKQM1fW`PhtiSGp z@k3pVGhB9rf&G*{bMXB&Q{T?({B)!sZ=`2Q(6}`AZR#Y`(*v$uN8&6qeexkL$rKtS_7jHn%1i-h$B(zYvrp*5pS*)Vw24Qlhr~yq$ zTLj>mOI}L2V(UPn@Mit^Ld@=|hi;=FiyhAXv86Mj+&PQ41=A2h;B9HNd-|?|zM$}P zYazF*bpcY7iquEAcs{~KxcUssTshLq^-uNqQ;<8-p}rXpP(=NTI`^dSJ+KiJtoc?l zKQeYg-?289V{_wI!v1mu2^L!bSZatI;yOI3LuoI*5OJ8K7VF+Ps}>BbMX~)~@dA78 zWhgYol)c=M$SaY-5@uA8pp-iS}YuDS&B)vE%h#1g-CG1!w;=FE#e_7rcDQJZvbBt5_%ijZ>kg~aD> zM~=WL38GMv0QL3B@=M*jYiQmp^<860y!RNI82YX707nbJnk+de;Nh}MgkI2%P zLcKBjKufBe&#Fq+GZQzDInHy#n3s7EBufPaBrSlE48v7Z|AJa*M|r!Xnt5kYHgUp0${0oIHz8uq7@9vP>LuGNuDMnkhO zbEbq}(Y&N8)4FN_ZJ_&v&!sz~yD*0<)@Bf~--MHlK^{BCAXWF0esxV0VPu1XR?gCiE>4B3KSYlqHVA$Q?rcMYYkI49z~a zK*lwF^Qo8yp4pl>I+^(icYI^NQY39)iK}iAql3L70+;OYz~$gJet#^wsA|pKQ5rqm zI+64WlvN5aID5m*K!VN9X3h{}=ZUaZ9Wp+AsM}&H2_ygZQE7nATJGug5d`gi{)2p^ z#43A3mm6RB^M5O56bu}`JYl5TNisV_zuvBZt1c}_#_4UJ$z5mDxo1CghblE~+_p*S z4`=D|@z0>a9e&Qr0P7rboLq2azPBKK2q#N2L98q9fa>1M7#r({sE4o#B6%q<1b7M0 z!j!3VemJ1W?3yz*KW_EsO6p0-PAp=xE3ZJT7IqJt-WwSJpshCOrcWpa(`BSOh|UpTU? zHd6k?OF25&`*E#2J|c#Y1CM6H#fRQsscraZ2>PU+R*i8kf1qOT078Wp@w?R{R0JNO zav#L*=ods#Ef@kgq&?(66%&pfZcz@c-57GCx%MQz}RrN z*Fp;I!!M`rGCa3OK=3!MdCBrx1z6s`ZXy0Ej;SW9cN-s~wa{ZC>w}2vah!HlU#Zq+ zqVH)vSMlTI4?Cn|Go65hsCl7sX1|JWGel5dAXz5g5T3NrA{Q-4{?n#HppCqOi15D4 z_wy*67R#H>M=!|J@+T~G|1GjU+0PEo`j`#JP#T`1=&)ec)yVEo$sHZg>MSK;oO}ZA z0Lw7jPC2Ue(-TX0k*8nuE7km}kV%UGq{!?$i^(Dwfqf^Kte+a}Xm^`AnO>!t^;hfY z0LutXG(PjoJKLHUe!1T?(A6f{1^H2c*rF@W+7}Tc*X#SRwxN;!!x}u@!(S|uI0oKS zLw(!pJRCP=Uv3T=wWwYUjI{Ldw~Qg`a8)C}qBjznWtltyM@zTbTVn9fSp^+v9+wST~h|`0x!$a|TAsdR(0IAC%d; zfha==EM@{xhRXoK=EW~s9SjSk|(>0YWknm@hruIhD&k6h72N>dJHG40Z>B z1VxxL;|K?kO?ZRUJ~N`za|A%={Q9;8^b7FPQq&NWuGSd<(L~M(bTxp3^J(g;m>LQ7 zTNm|icQo(4%GM&23P}qWE2!o_2oiv5*9Q(eDP^ouoP`kWYKXUH#2!9z{@Bu+)yo+j%XvsJhTC@{eAfJ$!HzD84aA+R8=Y}gVR(0&1QDH4kMWIA{qCQ9#c=F2<|=gYUEy@?oW{- z#&5hq`~P4xRp!Dk^@AIO-*F$=0<$0GBdO;Zz2v=T3G){sPY_A{hp?09U^Ne60_#=i z-bmhjXNybYyNMW6nE`Y-sK~gOqxpFd`FfmlHM_!~zdK~&t%i-{Sdxk->dUpYngZ-vW9Pocm(H{v8#`3tAF$i+8D zi0+Af8ftm)iSXxU)!Q6gdw+D7q!nH^4>v8<>P2ZR@Db(8elEZePg-3)EYWTtFklk; zi4@(tIWgxgZ5cAD4XkZq7KMKXZM0F-d6aNO4|kVCR%Cc^y5;!`V85rd+;{CRK;afR zJ>u8&Ip=E8tZJ^80VdsaG4NDsB;Jpxe`Dm~WC)(*EdCv)XjJSQ338>2{BNVC&&F>p z2Ut1SkV%t8=Nuh9{fh?aHMArqj|U|ATAvBD7q(#XcKHD_da+T)Pwsq_TK*rKI~ z{ogM*`UP6bFlo3V&3L<4wNxSK=2Yu?eXSr#?DdWL%X_bmU6QCeRzD04oj;BffX%y2 z{1w*6Kh}*MS{LZ>U2P{VOr}~puN}Wi>TemxeOhLHl?Xk5@(?0jY;cfMYg3J5HHG7{&frF82&Tm&if$S=)- zF72eaZEjgNl{Rh)@dFFKG3D1MQ~)zCg6yZoIVJ!+l!@|`_jYD@=$UO*@YCr#7N9RX zNw_6+?}Tbm7fo?>;uAE7y|eAm)6@L~908`LeEid&{~<%p)Q1yoi-*%qG|S?jOtbriYLC=FKpxF| zycDUwD<~GJSo0R0>pQCnHtI^vu5}KbRCKV29Zi&+4>eKc_EY~iHm-0AM|w`uIw57* zR}<+gz0c838jwk?w4lPsrfnB>;Vr4Ggj+2E3b`MgqTwR?yM>>srjJyiBapHt^{UDb z*I3Vb27WFyyPhwVSM9aSM$YcdXb}QRCPLF6$>WDvawwu60&A6_i@{hVhdy9#Y@YFm z1vV}JL_tTypE6;^-J4as@3eqZM2m1Y(G6N03S`H zrrDyd?=YWloQgQ#hjh-j+s|~c#b1Mc`=9rRivtPmjBGE2KWTR)fwSIWbJgX$al&D; zeO4I*Xu=QUTVA7hqYCKKfErOchoVoFbEqA&la_07PAB1~>KmQUkt2gPI|QN7Xdy*) zy18XzCAlT=yb=ZDWA0dyk)2_=QLgD>9?+1ERk738`%dqDz!<)|TuDTB{Ejy-Q!%b> z!8c*MnRhJZhi&fgR1vg1J(RhM!+TzjwyMen7I^r=%S66G+FmhU{V+__d2tf(sW5x& z=n4F*e?ON9oI6g^^|TSA9=2X_^ub0uVl`96g@eb@sYhJFqUcKJxG?p1#mApsc`ZWK zszDrUXBRzX$c>Dh_Jdv1q;v&8o%d@A}3{w;8Pe$=2)Zj(x-rbjCl_DHEL?SwV6s~y3+C->jMNVHl_7>?|yj>L_$XroEc|6`@;YFAiMi~ zF>Mhi-1z2yl;a3`cX%)eCSxVkN#Bhr{w;DH{?IQ!T$~+xb&D74m@NBRi$%JYH8PZ+ z9c)~$BpY%ou5_aDD_Vi4IKMgnHw*9x5drL|?h$G_PFWNmcy@bqBuk8bc<2XG|6+wo z2c;6Rfv?5lvHvH+KzT*$vzMxF&{JEQ%ypqsrduW%TT`#VDr4q5Tq2Rl0>d4hKw(D; z@01JBb1xM8zVgx2!$$95;;&{3>P?tW>$8G0u*2Ok>%>=qPRRhNq^r|l-_SOk{Dgp% z9G2a048wR<-#iE`aGquz?qw1V2CD&s>|rGGn0VmOSNNp7RtwN3nyEZqQlwbrNhL=z z@cY*CKKuwhyJCRC3UlHakSE#NPCN7a?e+)4$c$8kfCo=j0O6aYF;@+#EGVJHRXzSl zXc-8*y*>IA(VW%kK}}sIGkP(eJk~xPL5}s`_emrTMxqy4oW8QQ47@RgeABKPke7Y1 zu7HbZ!gzoe!HVg*wEi)F=)FyF?Kl{GBsTH}N^z4))LDMDZYh**juUw`?@QoAeP?Se zq$K>hJ?Yt3Kf~zL(j<3By$fR;sN`zu34?9bm4-0f@Z=7H?uzj^vlX)|b`E3@$XSuZw)C>wf3a9B6(p5?-;BdEVi|NqS{&7*XAu6t)yoerY<&M*4koNZ^FVVlYGbN-POH;@V-!lbY zosn3?)?{O3b!?=C#AY###2^(j#Dh-Yc5wI#0l>KC;Y7LjM7zr&j94L-`f-oEL@ZlD zpK3Em%hJ>%nZIRRfR!qR&7=HP`eE`L5TGfQ`eex&aHASTn0mo;4q@pO_tL@9S*bO5 zckqOdsHrD_m}9qpIc_|*PL`;2f1brso7pkT-Q^;WHDADJ`!(`H9@fdzx~RjMoSI4K z{nfe=9EsHhOr)F_k6(AguW8s(N>rt+&u2`p?q z`iD}Heq##y@P|@`!`qtSo?Px>(+BlhHJSf3iqwR&E=LG|i@rqEZj^(ZeN1Ahf(o_4 zSB!@#K5?UqIA>>y=U|2$#^hKv`Q@)L**3vLJ(La)V`&y>1J|lRrNzpGHbmJiycq57SMC#MkiShuaL6LPrJ&a4|9lXhVhT z_<@BCUwoTIQk+{46P{94d>%(QmI-^TWIAi_6x5mJ8;xApRX~vrBN4tCmn_;rv z;Ach%R1@96xnvLdPxoFCH|Wh{Zb=#(z7qn1wsC!ow>v^ zw+*QclUbRk!@EHxHsw@_a&RH2J|=KV@o>-=-ZkBhW@G3@r^Y-0$|#)_sX;*s!Co?0 z-)&$vtq4!whaR%dTv^>2Ve#B!7AkH&r|SRMr<($XD7v9em}*V%5qf@mX|re4wpFHk8%f__cF$c4(DoLP)V_Wa3hCnQDr5YF z_elL#L=EgS$j>Uysrp-V&Y`e&{WOVLCesxiu%8|2|7vJlh@0asRD)R{kc_1LuF-{N zxOhirb+o4?6>U-XzMe%iDXXcoxr_FJ@XiHSC?zZ&c?+kUCV}m&9v|8sfXKC&HlV?Y z41NE)&JyH=4!|{UA)6Go-y|aDmLB>Owfc|N;ScG%+10JUKF3Fe0GpDMzGjAi7j9CG z%&Dc}tJMfBeXO=k;N>t63Q8<_a;8434aA7Y!KN#YLnq)cn9#n>I?K!*Lhrt-z z@D~Y}_^k$6mqjQyi5=5#wKL`3-|JHUE%KN?mzY!a78nc8PA|EvK7FQvgUmu`&bi!< z3stCpAa=ZaYIl7kY#llxa*zCKZZjj<_0Mv*qoj;Bw&VFdKD(OT<3Wpng?59rGQ3-9 zE7IXwP;2o0FWaQtL$=g+*$fK=6(nQI|?5m)U};F`$@Ku!Geks%MrD$$?vhlha>m|SRRg$ zdQ?Xj0P6L`m@eprRN3~+$B(RsB)2tToK4?-+r8kSSa6S}7obS;Z|#D?Is~_My%An% zRFZP);7^y1lzk3%b$5rgS$yb6RP$>a6qJk#z_LuWE9bxm`TONuq#)dqt2kn+&AuMM zO)|p_H~e$xC;ZH0c@X0wN6>^^Gw(Ia2UsNHbuR?y?}Sg$Zf>$Iw=Zj9G8aeQ*C#x) z)WdRz%}ma+V{>=F13uG7i(BxbOeH>}(cVE@dMmqe2V?0V5Lyd1j91~4+yquJyu!`A z3lI8C{g)TZ?Q|G^A-p;(Y**{qpNXw%VZ`4NE)EUx)FW??e=X7cPro{%DKqJCmAf?e zc|YtF@Z>{k)uSK?JE*7q#1Tvq8=@2YXr_zIU`2Ja?nN+uCup)&9`wRQ5?G&;_@>>J z{UKdt)YV42>2FQ^fCIV%;>9}qaw~n9eoGUeZ%OfHZ^Gge6(%>N2Q54MGKlI~EGyKc zaT&(1Ky&#hjnJvzATcGufLsR???I%Bt>2B)+vGO+guS01aDXJwI3gs2B;!M8L?tqO>?iyeQN5z7#pBl%FxO@P znsaE!KB8@oA#j&~hpKk=xa46l;BwbaFEM(b>jn$s?(>15Z)JU!BM>C1kJ05#F#+JP}BUVeJfvz_X8 zYt1!2q+ZQV)Xe+Rh1DA&pjxF##!I2fOjh8W(>ozRbpC}<0uZ&~@HA{`kn8ldDHvKZ z!A(oq-y8!9y|Z+fiuMuS|-Ibc(_{YfO)dTupazBCK3dAUbbtlvnU%4PktQ<&Rm z=!+mvrn`$OPd?;@DqUp5lkIG!SSg;W!L06HBhtsr7UewBtF98~2QIc3S8_88NC1INKU{b(;edNqSRTLcN#AbY0KWhOGpoF5-+W6*d#di|6rlv- z^7qets3l#|BpPs`n0@AzFr)|I#7@1nJn`GM-p?nWiK}1_eZVyZ{vU^~n5~v2bxWc{ z`n8-#x}>wzJ%qO5r2PU>=?n%Mf~2szQIqlu)$~0#ws%i-=3<2rT+#j`&tH-<*w%W- z*^9dS7E{FV!TM^ObNHC5)28n~@3|cAk37`7a?5({E25`fCK0P(-1*hdi(9LiH{@Va zTcQ0_?+dKAuS9J8@utO#sYJZ9%^+^%BFpA_aWM}$i@JH`J(f}bN*WI1m}2^fL6GI` z4*nRHHe{o9{s5PG=zo`I$^^7%U7sUBWgJMYi{&Y`n zc&L2wB^`jGmEM^Y_amYH@=Ei-n9TZJd~Y+a^$0D;I(y#mG($`zOq%(7#>F&^5Y`%! z!^1GCZj_e*#DAr}!iBoRKJE2Q|y)0>bj{InvTF52IZK~3WKi+IHX85-3BZTRq1 z(;UlbwrGAg%W}{84CfV}=a^^+g6AyUsBn;cTLXU&i|@MCY%kPHf)J*kD3y=4vWDxF zoiuRn9IlDCw@u0@Xv{m~q4Qm4)(yn(`Z(^*_!rUH=USgvQJ+gB079)#4mK z@%)jSZ@W_ir*GP}>OmiO;BHQFLKY!@pwa|wgN!S)VJpMTg8RnXD8h9s^2$ zxupPki=AZ!-1;X(=#9!SNLK}rq0#|AzS(1{NdqcO8%pTVh*yc@zw4QzUi@SEs68%l zwp3`G-uxmA#PEI|pFlSV)`E3i<3F`Nn5GOoq_dywD#kEo>FZ!lR~`sQu;yGp(*vOO za_6qyL#l_#FrP(*4h zNp1U;!idjXg>fhQ&4FAZr^6GqCqv2I;V3l1qH%#Rnk#G?VEanDw*;DuY55VMBtnBM zE%ZbY<);=;yv5XGr!U!F&!AE4WO_}h@3F*tILuM~8aPYceF40=UV`R!!LY=Q-nrpk zt-C|QYp%Sr*fk~Dckcx9+LiM4Xi|h{M@l5_{w}%@2Jj3TuBT2ZN){aNr!s4hy&eZ} z9E~nl@gyxtJQT4RcZ`Nb2QxXSGvL*NI10Qlh;hP{xi0t~wMaksSZe55C$IItj~rD+ zn9mk46{>MxAv6s&E~Ub6&Icpsw_EL=){am%`dqXs=l@Rnwb*HVAy`wYZCtFLr$;tt zyYYmlUeMU)F`}>~2D)b-c?(W;Hrjf4H6T7LUGL*(o@d(c;f&gdYnT6BMKjo%jk_wv zT4tk)@HeJT1E!9K$w($_kKFk82|p>689ymkgtpdlv{=qLK~~*ijN=!?dFl{~FJX#v zD-VgXMRccQ`3ys)V9cT^ES-D%%4ql+@QkpH)|!u@Qp`JQiG5qeMF>p~7z9aj;ZDVY zIeW~8O$U=o6MfW+1&H*5R~}uGh3s$2LSq>B_w}#z_VZ`SO#ESEW28XSuku zi5MJf36}Jud`9e?!>7VkWR#fCcU?wszKK9UKkCI6~;#e=5Ke(}HzwYp&2qxo&#&ouuLy8jto| ze))ZQ-6?jglr40DJu1zQ=o_?grn9Q&19l$8NJgk|Z^;YU8YWg3ZLa5?hmv*&%#H_K zJ`>4S-bcFIno>sL;d@P7cef8;zrvS^|4)Fx9?AV`o(=pnTGr#++T__@D#)4j#~Vvy zkLY6rkLdKUQAEb)S!Yg)zp-w zv0X8%{+>VSG@22%Njpsbn_YVK(uZcTDP%9z%SoYux5z|2M~Z9_%`Iwf;yo7@7|fJF zRKnABpH)O~7z?0VW1w^Ig_N}_lSPVBy%ig}gSsF=&mJcEo&mgW@nik(*SpBSn7 zxStqts$k;z$q_~y^iJNRdG|5JvOBonAZac98FzWT^j`23;h&jd+j~wO6jD&Ytv%_9 z3m_;-TBO4?S#0$~zgtDT18tQVxPk&wiH@6xB7dR>A$P*xhH-FZv!fayzL3@CAJ;B4 z9P0KUlq0q9r5F_2{wbVxAN4yk|JzIJ;hH_Kyr!y*{2fLIv}jI={X$u zUAiR%aBDd$#!W1)dKj_SRyIKZkFQ(~viTeSOa~-)#n$m0%G2D5L%)&xHogpPp{_4} zYeIS_hQ$Os&Kx|f(9BDb$e)r6`z1|_y3P%Z%1hr)JdN+magxUO{Qn?_9Oi06ey`bB zKm1kHz#&t$K&l<#p26@B>gF6GTx)z+q|jp+eT?sYbx3C+PH6RHXC$LGD?>5#wrjnG zYNY|Q6vSb%{As&3SpeJXr8<`}eBI$y$Zgpnn5=1Xq}(BpYVmI0Va@GGo%he%scJRWBhCKv_-rCu&Q^g&s)wMvx5!fb z?n@h1oGLjuws;XLkcbt}`Myb;I8A|E4zgiZS`LM;h*-if)-1)d17WrO4s<}sle-y8 zAd--e6&m8(>$}b!RH*z)z^pI8ixp7+R;3jQqISb!KX!r}mH$c8-rV;O?<#!rHGMLD z4C`@A7Y?jQI@N1XGV}x4hWPpG)!Z4EpPF~`E;;PN!X|c5N`eTmRCc0AcAhz?*PH&sB_Q5V~Bizo^^#bQ3Y#W~?((jozxQED=!r z_@15CF2^g{g4x_u{N`)M$Qr6cL?YuN>BjH*k0;;khfY&xB&tdMZj9SX8JG;3RQ=Z8 zx=i>p2}hnj<&W(<5|~b$vugfzYp4Puwnd6NgK1U{g=Vx8J|h-JvPw>juxBkbRD!N^ z12#~Bf=GfA%ztYH<|yW%oILdWZMhPZ^>Cj0mB=}Htcdo8sFatzv3@f&yN@nmtlOZ{ zz&b9seUu3*=?TI8%ukJxruddH${w&*mY#Js+WfpqyecA(r-yXSH#+4ky4)R3OP`KB zo7DX6a9X81E|k`jN2QZDnO^SxzOHn&NZ(BQtqc2c@sUk5{Y{WCQMhItocecNe@7;T zCpv9POp$Fnv3fT*EwXExad~mP4$MiTd2W=gP^dnWr?=v7pAi?86AY)DXY0{6r zr(xOE|C@mchK|dJ8n$Vw%cNGPkW7j%@^Q*Q%t~kyvf3gIPNse&u?;$yBe-)y3{d3^ zYm(chJMOhG8F<*-YV{jCXhfK_z_7WuF=g7A^TB4)+?JCHF!wVgG5}JM)?V>qORrD& z);+1>d^Ta-Q5{SU|4yJ^r2lyC%0U6kGcu~#V(RiO-|>$gi3WL}U?VJ;zM8R$ke;vE z6Ntz*d;FJw9!xJtOMg;?X@-{>Al~7mV)j=Mkvz61-m{7Fv}xJ4;nTNHffAu|nNs@6 zsg&Wj^y6$=9H8&H`VllD{lyW11ClS5uIr=(b|#(#_Nhe2SLJO`JMy1S@OiQcHZQ#t z5R98w+WeYj)k#e|hV;c=SD>Q>Qzy@DKTxSE2*7x6VTiQS7n~a%GjXr-3BeuW0qVM$ zgk_BwH3j>reT)3D%RexE;QjN$_O!6EJ=iWUz7B89eh)EiHT<1=as=M+JysN;aPTEi zMDtedA01p;@*mm~5&+FR@$5+RUv9`?gmV*M&3`Ml@VKqUd~oKZnAVP(pCBF@?a;z{ z^@)SOA$da(7yK(~)0vNs-MRm%p>w4i7Ekl(xGbUlMte!>{!C6^^NYj@3$?DNoamwx z;)U6&1k_ym5aA&g^2Qo|PTE%Z z>9-C2*B8%?}?d8MFN2cBjAf+cMjd+f^6#0f*b( zmfL65SNPI{yJByy9*1Po=Fw$Y3X~LH%6U&=c+5jt=fUiofBD|EPZGG23y*=QkOWuQL+XFiI$$e;z>zPV`Ss*e7G`}F+A#?Ca) z7>gebLb?uMigF=c8RSIH!id~VPlS&A$7Qx=zBE3N9Apc0zWLk zn^er`UG{2i_(w2l*?WDayARA=T5HY}-a78U3xeYem(o!zA)eSnRwB|^=TG(m3ZFoF z@OyMfVTAs!s+{lutH>c`KiS=(JnGXH|5Ef;?Ndyo(3WjSBd%>SZF7l~X z#^R*m=<|Rwe*@b?aK_j1XU6vKI>L&8b&B>_ctocMpt#DKyFFLbdrmvdR6<_FTaaK(;} zFlsE43E^W#n~1O#NLz&wAo2Od0>D18LWdmYeLDbvcEK;=r~nR?Ns1F_dR(tjyj*z| zGClY6^HtwhzkBt*lp6@)duM!E%!zcsa!|l*EEj1s4klvyFtMTRlX3iFxUZO5o)Zpt zg%vpOPYd-?;MO|t5czsfet?r&pZ*&U_+m%Y$HdS;P}5|CCc$AUK9s>l=9|Cynag9p z9#Jam_&{0jGkaE0g78OO?5uP!t7ca}q5A0U%pu9`kL2j7zXs>d|HPAi(1hU`{_jh^ zLhzAX?J}J3J$&}y;-=WLQ)gGOz9^pnjqXX=WKO~DAkrmoU0mjR=5KHZG$BwDV)Hh@ z2ZNpz@6hs7pSfVAUB9=R1UZEJG#JRGM&QmKUGj^A^*#Y~Xkx-$nJ;y0m@Y+!Cm7Q) z$@hIGX{nIo9Pg_XOCz(?T`yBcF80ctq@xDqQ}+rlpG1$G{f{tg_O5o5a)HusnDO}QrSZR$CvzPrx% z;J9!dqx*&Tue-&wq!XIO<$lNcr5#(P7F23$P9;d!ZLpK&qL1`}2X3j*8;X!=1#2&C zsis~+0;fJWALCjah4z??#Lre$3P1P=Pwvg*n4}jaKh~_&;%!d#CYNnylnW(4u!5fC zCZBUI5-htZ#mGF|nT#4_1c|hsAedC;OhU!sE$Z_#Pk0{>Eux1S5lYlEZM^_xQw+HQ zNei{V+^i!8x=%)Obdt7#iN;rjC)$)UZ-om_$A;DY3}P*+x}V(NM$aNk$gXkI^THF! zTMa++qxr9D0f#=8!I!w+tU+D09~gKT{XXy8@v4SQ_7klF2=CK(V)mM}>7wYm*o1yu zodK4t0+BR9A#ME!x|iiIzR44IX5xmtv^vN1mKK9Cgr6OD>Gn|Wo`$vuK1&8YA$fJg zDxwR4Zb{WX&pC?9ZT;zEyQcQoopSVmW)EXWH1+ab57%gP*bnP2dBhJ7!4E3 zWn4z%_uQ+NgAz8xywSPt9k`1P_8I3N5y%Xed%k@nb68r)@1;Y@fcm|3Gq3`YmD~S6 zgiP5F8u5hW@b?%sW z$hif;BzQh#)A<4umu2SX$5TAXhQz=Ccjb4IhmFw`I0Zk@m%4@_NmHS(*V8)T%olae z#YR#K8ff67aiu!jyZu&vn*S^`aWwwCofJLU{UL^Ql@H^dE23ive>P@sVR#8QVl+cz zIM@HDXU&z>y(n|fkeo3Wq^$eU`}I#(X#?nt~p`gP_hwJoHAAb1a)L3!Zd(+rmP@(icRDLNLn#;t0cnse21o?#S_6f8(qZj-H zhS~$Jm8J+vesZ5vSr;oe8n9&$EuRgRZ-` z#^+I*R5Ut?i@rvioQClp>;&C(7~2Mn7r4!!H1YnD4y^NCo59>Hyi1#|L*+8I#euc( z1j_iIX4UggJYiHB_sE*XQdF1`Q*H13vu8 zmvDheA}5>3CXpnsi$j%}m9o_xW-r4^@TVA5H z;mnvXpJaI9=zV6w6bXD@`ktir!r_iCO>ARdZLXWOvUvdY&U9R^Ep=V;>tD{+R=aQ{3GhazF*7+8(f_Tx5ljohq1Kt_E*NW zpwZkBcqloEPa5b?NQj z&werZo7)KzQNm6-+Z~&#as2Lpobi!dP~m?5k_@dQ*iGzVg}YEHMVvLvZ;4soGsQAlB+s%Z80#Blz5<2X7|3Ym`y-5$1*{EoXnf9dzPj_M~G?QK_U zHky$=4#lx)WV5BdZ|Cb3@oCps2j879GB-TA+&yb=qc|MH5H_A>mOYvs{Y|IVfQv3z zjQ@d59X0S*X(&RtmS*d`AKJfSVZ#mxu$l{t|T6XH#@VUbq;X^HW6SG9I$g`78< zQS#upRS6uojtf-LM(`Ir>DAzYUPzRKhlyuNL^PKZq@trfzo-Y0ns?`H2c+I*ikWtf z_WqAaQ*t6(YTpQ|yAO%!DWIMX>|alIkLy;e(?FV$!E`qd8I%)fKC4qU>&s?gRnLhE z!MoMOi_~hhG6OlZhX;_+`Ff29OQaM@t7P@5t%+J6-pd;@lqKyxhkYjj?_I*B*qhC$ zr~zKI%8FRO9zxPuFdcibo=-U04$wDe94~bB{n^IXnxvIMB~j}}_?}1I#e}mR3*h4= z&WnYX%f0$2Q*Oc7>GpFwt;t3K)N|!gYWk1N7yyr$d6VPC-MEpQKf{wyirn_+*pGkM z7dQ=d9T8(BwFq%RIo%U*Exv6G?kF8m2?Q84CqFR;Hq+21K89buV)^()dVCaRVP*#R znD(_0F9*%2jha8#F8sUj?Whk|D>$8<)-Mca(M?RxJdU?xS3$K;;jgl!euc+v-_wK3 zqf9!UH`Y>q*yt~%9oce72o51_Xp+H=5(G!xZzGpFupZb;HkxFII^-|rOx{{Fppxo) z9etW}V>h-#(dp>LJL++FFzquJf`}iGhY$p`5KdONzb8AKI*Gi$Tl`gegY%p#I8Je~ z2@>*7_>fS$L47Jq||V9HkOL>?uJ(~;+_7fOTFCk#>@R8 z=H@mM=4KyyoLtwLzT|foKev7;NEfZ@JjFb`c>ZG+InS_X$X^!z9Yu2Z-1~>0m>#?* zvH~Yll}itwDsg7hgNm$)kGAA03t|((kURF(*17qTle1g_b2% zXswf|g&%#Kg{C}D$G7gU?F6ul`A@}HR$2t>kS%=v8)N3+0t`5b4es-oKE>X}E_!?r zLw#ZAq)>_j!w_tVOsaQx^c=L5!TC^j>I&K*81LT%ii|oyoxl0g3#6Ak0>Lylq~zHB zGMi0c-Ioyaresk|`IvBMoc+dqcZR>tna0EK!mU>TaS%Hfa)J1kT?ixR}9aV0S7S@;GqW zN_+erNpp%N;9{09)p49pv^EC(jC|d@VvC5B%o~7=K%l`d5@Cwz-X_BL8y&aDsHIO> z7vUim#(^Xui0J$W{ySx#inUs;3=e*Jxpnrmt`)tF_BgGU%k!Vsc(^}ouyZvg=&BXY zZfk$PxS;38wLTIaJxSX>(&dR8zxeurE&IuL%ZMe!O~Yn$I`WPav(awZP2m`cW^_ji z*YHXn@rHn1&Mqnz?&o)|C=CvaVGrk>Sl^1OC_}t<1C1n3TMsC@77-5vg3=I{NY!Vy z(sSlNE;{$c&gYe=HT%z>`%`aXVe(wH2hxQ2lD8t3&ylV*pKR&Kt)1^~g|ogo97=!J zvr7RP6~9GmGPUCla-&ojY>%bhBNASM4Xc#C!sgweh zA-(vk0Oj*$puu}gmHpq`|97-xM0&Mc2W&krU>`7(LK`^X-^o%Ilc^RQW&--JI;Se( zwcA)_2OVO9hQ@mK+;ov50sNM)1 zKO|ti4=oZc{+jEwOd`lE?3ly=TsHF|x6vp}99Q}UArU4G=2FQ2;YlFecvwb6Ymd=<;KZLoz8&Q5v`D zhxz7#)*Av^G)Iai`;Q|6eY&z(+hT%)1lN7r%EsRG1aa!u37`DCe-~)B4bm&a4pKKc zZh_}!s0bk;Em1@q502xckyiCMrr5LyGYx}&8T%clZSAH@ADooTLO`ZHCfKjS<)Sju^qw<^mn zGz;!iArZ(?=~;`e(?xc=HvA&^k!=<%SCOl_8EA1Y>efvokYfV)_r;RtYM>du*YZ~D zfC(!_oHPoogx}$_(@s-K)Eo0RkD&~ zx=-KkrUFinnrY;x0y8NF&3@f_CCW{EIv7({bQ$?Z_};=9qc_+wYt#xf2eQEkngln3 zE|H-OAV%s_^V2wuSKh*)_h*dY_1A)jV32#PvluH5lxy0Th!=g%Ip$UePf`f9`}Bd) z?ryOvK>L9ArMAU`ZfIro>MGGM_9MD8G>{6b`7;y&YnT+!%AH_k8b*Ess19>6=NagT z;_$RP7U{#qmIY|ue9`&+1p}&8mxV%A141>8goY^Md(o?&H}?X9y_lMVfYL4R1kQ7H zRn$Q`z&JT1pV(vM6^=Fu{7*-cSaP_IKx^vzOMcdOu9Jn5pXS=#elDxTXt45^?jf%@ z^f+!am}@1Vi_4Eqf0ovz&m(>xk@NVxt>g+s1pkAvftkn5jn7)kX{(WRB=GXf=;6CaWy=uVSt^;mWXcLNBS)=WCbCgC@ z{G$AQoRvBfsfo{lmZ&ocPHXsl-YBLF&~OGxcmZJe@>2Svp4R$KXF>#7A!!HASbBPT z3e8s$*a#!n?o;A0#%<-sA+HI~mKZ;<85ACw%qNzPRrE?EGTn{rYDfG!iIM0lXy-T) z$+y4p3pUG_QUS|p3zaWd1q^9%Rg(CTk$3F=kXL$lnLcTzM>iE&e(35$dSwW>K%%{HIBRN&3D$xDOz-K&-Cf_H?ZnwdgK z|3xa~aQ6#!WR-0vT9@XqO&&9R4i>L}yO{&YGQiC=Gr~bQ(0pN@9dhZQlO69Mr7vV6 zm-Lhamrtxldzy=~tPMx$?l`_=&m$0J6|_RRI5;r(4Lm`iq5SrqxaST=TI`k7=&m&`+MP^U>fP zas;YJre%JXBv!vXv8t+B_Bz>E`$m-2;^+p5`Qt@SKTzI9%=N`AKfEI^; zjHFSoFw`r%w&3eXzk2!EE=INE6mHJLaZ|WTVP$J7*!nw=LP)x=!QGwyqFol`JJ;-H zEzLzeuX5?_M=L$k;UMzbqrVH#Wt$pa_Renf@h*FDOqJ_JUM;h}YYOYoIqgOFGCUNn zPyt>Rq4f;2)OV(%_Vfi?0v?@{|H{T|(cGlu|sfy+$wrjOqdt$t)O{-^db_6?lX`Z{S9Pm>6+!=!Ywx_G%O9gQEp zF^BHoWG8ePE+F-sJZUr&Z)2Gahf%B4q2ji+g_%jlls*WDmBpN{?;)&2R5%o(JU0m= zX#x^r7qLTIx%n_xjZXunHl+i)?nS|ZMcxd;21^3lJ~q(pSM!seH?~UtVtX%y=8Jj5kk76yJ~{+mV@%q8yRHQO1Ziek*5ltJB zR}Z)-;It66M%=(vQ3SxS^4v^7R&nj=2}A4Ch2nqa4lRIvbpXZV35mx7$CO^6}35cf^yJmV+F{O3`>l+S2weIf)ZUep73Iw7Z zAa*{u##yKO()iT}A*&sMUPAL)GoFtT>4Ydly@526@YbT$>kuK-0dtY-GxJtn%6^E@ zVcu=<&vM^mX6LrOlEJZ?H>BT-wbl-+wI8a9F);$30E{ELpSiP3Z!xuDq{fwyT~V)PZ-FJhh)}qpFFqV(-pCjg{j|0Zx}1cc zZ+%XVPxD4sdM?E4&MAcMU#gM5;sMh3OXC64#t{eYtrnek7dO65fb6Gl3k z-UJS;;rbdwwyG*n;0e-76O2xdvd_(-oPus@;a|mB0Hz5pM%r6wBQinr1xKAR?yaCN zED3o5bXZ+os|XG&+v&gp(}E|MF!X&HArxAMKz?nAu_A=I|6Lj=`IkXDrMUjF5=3+V z{&17vLJ?zaMU53(_Bxe^vriJ%UhZ zTz3_uT$m^Vz0C|%LFgWO{J7YfV}Z&ZK5gz-9)2usr{@n*zcYs$Dh6YB>jxEDq$+9F zFI)K;>ZJzP^mAv&%baSP(7bK{vQD7>ttSo0*bnA~l3S@qfLZn!?9llub zj?r4bTs&NEdEYN6Rd(Zjd*64ww`OI6bHTm-DO~O5Xs*K7Y_eO#sR;*1uG8X`z{>S~ z)2}&}atmo}!KqI=h!+w*6scs3LxNwH#C%=mv%3f-2HLx+52r$UC_oav{CQ1P5Ad*2 zF?wQgTDT7UZr94U%R(ES61ecI?Q5+Ge?0n9WSEp*SC-uml4 zDEEr%2Rw*?MDPsENe_EvtMnI$i#>;<&kVQ@C>Z>pi{*alWlu4U;4z<(dVcc_;jeE1 z?PoZGO$q~pUYhtBBT0g0s_ZwHQj1hS1U6nSMSWPf0@@UR@p<{k8b&Pa1cA@OVK?1= ztUPCF_i+2W9_Q)@a=PBLe^Tr3B+Zfigb#Iw^qLlgzJ?jyBv)STJ&@sO-6q08BE)Jt zVXX&I|6+LhLB&pIE>0sW=>%Ycvsq8FOkZN}_~$eQ0! z2H*6h^PJ8hUWq6X@(zuc*c$@56!@{qD2nF86Vn=b08T0SJ70!mj(8lu=@mTrUK@3? z0FYb3o7~#+dHFhM9F7tI&GBE3oX8xWcQ6rt_J{YvMY7Dfuf9{O`7VJc&tu&;7x6EK zqyjok^WXB?!vTfNPYp2OT-??i^A0%JN$Y^JYH;=q28Ph z2+Df$58w&Lr8#n}Bo}~RIXbj@nKKN1$krHW|EDj&>HWrX^ov)h`jEfB8H7ZLyf@c^ zmnCr}_{Jynu2pJfgFUbBYiFr%f*srl?2%$Rz6W0r14vLY<%_L5IuOTVpMdWuu8s;N zDRmj|nOPzyNenDHYYMQrix7VA-1;4egDs!TG*}22lU~sM`i0V&3l|cc7H;CJsQKHo z{8(RQb6k&)du=cPXd;z-bkqdXp?n+E;EsFK<^r z^3=;3to9h3YFc#FUfG?hWdWc`xS|%b44)%wx8!r1ol(V*1aUqcE_VZ3nQYt;;r;em z4fT_;)QX$PmlPagXV_s2S5+>zdWg)V@ImfI-EtHEKlOoXtVm_Xzo@3zvVJb?Qp_63 zX>s6#>J)$_q0HdS8FZGUvH@)YTSnCau!~wyA46%NiW`Ubf zpwsH-Is(8~scAWZ9m`dhrFkUN-JSZyuPw01BMb5Wuy0^L0#5Kpx4_=%$ZmInKs1u? z{3RD|$fRrToDF8!RF3P0GIY+O)!Pa{HIyiS;kWG!&eLYOC_-T3y}v=zIH|nMXHs>C zAzJ&&NeV!{ozodb>G8xjj0?u?y2w)$+guW)p5~<(J+uiaM6l1jnOd$z@Qb~ zN-43Z2OCl_jMD>-3y{g+a;eiZz5e<}RjnG^VuyNulOhnJMi&|ZO-*p$BYHrobsZV)B75d2P{Ym~ytX10 z!nJYM*faIn&hFc0fT^Kl7RVNEMXf-5g^mT{InC1SBq7e_@Zi?S3o@zXzj`z zTBf}#JOQlnSOc$QQ=djVAgV zzhmpi4R2L)Kv^;TBo9Hrn~8!uA=<)r9&{a(*D5oje-zZ7-sCHEpfl%sI<}CkThBz^ z=orGE)lFj$NgaQBy2{VQWXr{}(umest?Lr6bcd9!gQ9P8BvOpTK=@5KjhUI#&iinyf~uc0d6; z3obV>zlf+4Y=D<-<6dCAbv1sk3lBNI?5}yhdZH>G(|5pQ=>{7sA56nSRV|KG5D$)S>ua{>j-Kz2f7~W66P0q!!9hmbuUG*B?LAr?j!pzABKKh~6D#$@px8F@ zeptJZ$n9hL0hUW+^tr0R+j1|{`+Rw@nV=5f%gl?FS2O!>zeSm^EZFA7`dfO=H1P+o zece~>ey7t@+AXa=1Av(|0Mww^dz!c69G~v~C_5nmI%W3z&;hCJ)a?tuAM5Gjk}E2k z1H1KYxCMh+%AM%8z5m!8wJ8A@Y zd2HTbV+}8Xa0GtA4dP^c9>hzg z*W%W}dGLjc6RWZ)v-sdXfyz8JVO3ixbIta+5J6WA>^WAH(m#6r{#<<+K`eQb>B!w;1KOQsRYv?6PbtxM)YA) z{20HJ^Y(sFSBCG;$Ps~OiTeWYs1UriMU{tD=bZ?XH^9ei@g;YgzB*=?mR@%u&_!b6 zbs^Ugn^k39JZ%Jt!g}%YoQFd=RUX3Nh{7i*m!&@=f(Hrh*28U&o3o2coLEB#)fL^t zwZnC{gj#_WUdRJGy|d;3JYWE8aUgEtG6F04wn*$$e&A`f(FgP}_rk@nAE?^}| zblvkr1^#^s1NoK1oHIe4e#YHm6g(`gqqxDs`0Ks(|{m{uNL$oR8Be*d&{Im*2;Ji}%cne7LA zVjIs^JqEW=V;s-HaLivu%Reyz`gL{zXzqP(^C9XTS_$|FNZmP1m=l!V<>4F=Q$Wu>3Jn+s11@~*DxjUO%u$K}NX{X;k zxI|eYw~FP5j)in;ov}ijtzvCHwA!HbV5ZZ)(^OXz`CGc@^^9%T!LDcJv&`FOba5OU z!OS*)?5_z731q;*B*&XB&&FHa>}LZ$ppX)b7{6u8E7DE;c@)tzp8L!#gsT@0;vatA z8{;1sJ4`vEwzpZ>jzY@hh%dyPY1O*tZ0i=ut)ILVa@}vXMtdcJe3r6%%kng7EzNuxDZmO9Fr=mTmwqgS zqfji^@#=7Aa#m+vY1!^B*j5bF@vO*~5o9rjV&Iq$2f4e3D#2^zB#qVQS$s(PH7HJb zOC`S+sTq=R1^Fw-sSC{u`Q}TiC~;YzZqB%;0h6<8@$xtVnN&`NER?>SWbnDxC#cKZ zr37c-wcqUhBxnBn51NVUoxmNvdj;k$(!n%8-M72c-%5xwr!Ku~Ik`N<`1G~$c$45+ zD%%){Y%TmpDl>~8YGaj)rKsrNz6&rCR=Jgtt3>aMor)2VzNuN4eyOmYqvrY0qjgjv z2cLqr=nL&Yt}vXQ3BxWRDbN(C$sxhUN9`Z`lEexwSJ~kJ@e1Fxe)9J}cCcSdPGBZ} z8~w+4e)~~VYq*iyT3n99h4Dzm)~YW%=xv?$)^5E&{`g3gVB_Fcw9L_Vp2DYj%-z;y zI4A@o3IEKsvm@BvE3@a0p%S>q8bLGZ&Tr%?Y~u7g9vx>xPG71*NG7xVtD6Xi{?!bb zBOU`J($P%ONbUMimOhnKJ}eL7X{@M3c*v>=5_WW$!!_}gR5u|u{r61;`M2Hd)3I}mKa1``3 zGR&sOO)iIn&>%)XJ(@7(Q&+g&ktm~RIXy5pWdIDg^pcS;mImm&zO>NSF=#KN@eSJR z|yA^&qEYaRmq6kE8R@GQ@Of;32HsYU*yc;`NNE$+W#W(KaLVgJ#UsDhB^7sUUp zTE7bmpFDGs=h41iDNrJV=DrW31+gFGSwFolT2e7@7=rX!{S~-kgOJ|xmVaE5NgFZ^ zY?QnU>sK2)!XAQq8>aE{pV*e34*eEMfYo7^A(b?q8bf3UAO7pUB4JdbsPFge3xM1E z!tjuTvt$U#C|baf=&6P0EE^-tuQ3geT8G!(DQ zd{VF`6{BSJ6Lve~ji=2y;S3uF!ddgTjb5acx2e^339QZ^?w_{#9b;>TswuDQmP)B8 z-@DZm)40JyI5IomeWHE6wu8QH8m-IgIEiGRgc@1R$^;B=s*N@`b_PM&JxRd;k#>xS zd+=JeW5|;l@$d0lXnq%@Gq7R^=bV>^=HNb)K$2jq{ZuB%_&6Md)lpNl*K?N2PwAV~ zm`y+=g<-rUQ(UREKx1{&-_Nf za;W8ON|B`HY|efc=VXfOxc9KFo_TYZ4(GJ$@eNM|8$M0r7sX`ozhiP}she^0#Kfj^ z#fqAagNFc7W%WM87r{rpe!!h*;#12+rAag78jl3&dPsv3b7~*Ea@7g~*>Sm8;s# znl+ss=d*Lub&glM{=?(&ke(aNzB#@A5CQO9`sQX=)Fo*vfMJW?WYN8usu| z_-5TL;P)#;XvSEaFW^DKo0W=q=Q5wV_4Zj`V1)nUl1#xnID)V5*&&&od54#)Lm&yg z(M?uo&A{jPXEq9PdG){mKl92}?dy0`SIBV-t7WH+MQ7^}Y+)%)%ZyvWNT}LcvCnDw z-E>@+OmBseo;1DL8{zi$PSa%DU*rrvg83{ht(62~L{M?6(6E6-N-$*n?o(b7@o<69 zjTNJI4dE~Rp~yX9=Tdf_NDTarW<9MY)7E>+-*A%t!}<0%>xGt+C(}+#J@q7BE@#(9hMg{$ zaibv3(3@2r3pLebGBNvGE1v}B2 zy>TS}F|qUvkd{+EGbHS$b{$3nF zU|HFND}GJ$*ddn~y<~Xnkm2gL!Bjp`iXbg#a{g{tZ=>BV)IA!GIw9<{3m0D`&#pKz z2kwJbcdi|N3`G4>YuRIt%b{h+t1X8w*if=aY^|Ef*Ux6^4HCZC%fD94;_<++dY(zQ zbogXG%w?%w!kCn|EXB{x%DbB;|S;_g&5dMX{ z-y^+pSTLpH9h+TN$r7MI;djjK$0Ye3i2mFRoS|W8SWAqh!@gUX-J45nKXr0nL8Hpr zH}c!NQG&$+m)q6yCjG@nw-Yex2RFU4Sp_x!{vp6BNRcxgyjN;+Is6F0DMyI3I=e{l zdt^X9doRx#-u92Dg8SM<&V%2YJ}s9O3slaqZp+Q&gNJ-65hzqi#`hiBlclD?`#=hq zl6G{J@wb)~PK!5X^(CKePEv;nCr%i;*a;|1o_M@0fgbM^*xPJPtBpC8J}>8V-%|xx zih4Zwa|x6Fixt7T3$<5hOO{WpBe?0RY|72}p2{FI@p=NmWe#tbv+cpTt4wuSEHqys zftl94#pfwd9@;d1T5ldBHX2s%p>%t;hKe3b8{oHeMbh2F(bVGU;2E-i=V`-o5LUhw0s z_HLzL`a-y1G?3dO+QM#=F`kS~?KGAM(N#ifr zMcvvz#J8Kk-Tt+%Wd^tDe!TKknpuVJRDOzU5LSZ+@p~XSb99xVXB47 zM-p6%2DzW{Ni=gf{}*lm(gcLWPl0MV|#m^p|F=>Oda0o zom?9cKRC<-^jEs#Z!Eu#Wg{Xq#HN*-k1)Df&w?xY+)35vIrh6rliH#c#mg(Wc($=B zJ&_Ne)MU7qIkF_yrm4h^x4+3L7;qz9`ml0aAIH5MlwF1M40Gt7QYn2&bzNwKT(`fV zl;rSK;Pt~j*#Pc&_k6KHJmbQGnDfz|SW`>0(p(J;P8Dx3yc?Ly<-4g~&z^VynX&k+ zeAmOj zU%9Ma{+url7jPM*Se#({q)NP~B9G0fOP}=PVN<{}KyG|QlQ?u8V_N~2?U}KJ?1kwO zcwDL51hxo=@DOrXY(#ObMeu3BL!=^g%uZv+-&~P(xDrNSGvNrt(@+Id+<7UXkoGo> zi;^|J)~%N3!9dcYGB_C}mX@>kAiTPKou^#D{pQ<`EwJ2mvoy6nJG}h+kE7W>x+9xcb z4kT|`ItN{$vdYi_xC*kg5RNQ&fmx%Vkt@4$ebb+6-j6HgfBKFf+Tf}$=pwd!kJQ_e zocyxG%j9bZ^6X0gtXuK->V#)BN6v0C>{(^~$ux*|NgUyJ%gI*bdCx`}5f97Yx-rO~ z-YxlC7G2!0yV%J8_$}2WAdzMsBZ_y7FD_5o@MYXhm#y6yCA@QG`1A2trR4PzIL*er63`skzyhwq67BqWt!2@c=sTrxyhZTQh=zB9upcPBdIB z&ShotPtP-+Rfxg^t4lf$em90{C!@l~B+q4AG%rvlZ*;Sb#j1BQx3GVl7~D6SC6VWK zBmPLxDe16lLP_V|tTV-v6xgr@17$%coc^os zK;jdex%)BM$w_wYh${JGYTb^X6yw#(c!H2STUKpWecP4B7iWqo_0m`9r9hyPDV-~a zU#rW9j?Y|}ZLAU|{LsuckEci0#!s;xon1cN%ZrTeXQ^Q@sXNE8XQ#{&XtOM{ z{(G+A+iO1JhccAjY?FKFw|ASaEi*%fTHlBXkH2B|r_E1WaUDT_`#2#u-4(`g?0G@v z;G)KJ%3obv?MLppO>rQ5gKTM0VRxXE-1eD*L8~`l__~ zwbWy1RLW(_NOd=2=dVoI!p~IyK@Yzp)=EBHxBmBw@qlr{y+i!N??mJuxkYTJcJhCg zf%`7q?R1KRzQV%B_RJfQ+3bY^7%~B=3OY{vY9RtSMR{d-O-`^`vdCLIibKWRLHi-` zQG~UuS%~`sHlwh%xI8LjBEKBnYDE&u#8H)t^n9a%g5`eD`I6!zx zlPjp4m9uSr$|`C`6b}JcHb*lp?xQ-ZEGvHe=xIXEQA{wvaFj)v#*KGd?+42%K386s z&5HAO{E-Up?BSNUZaTCwTq>}&)PVS$g6B5)+1*BEdN_CD+tTInO@P6!2|$RV9sa_H zA!7rYx8x@w8!Q7%mPSYj9u%x|eWby8n&iATD>KW-7s@^8T0*yP-_TQWYkkMH`@)m3vjaVJOJyhXtlasADwW=H^Yn46Ds7PZ3x zrkYD!T5Q~{blW6bdyKcf&phzA+5Zo_)&MlwiEIi`#+}iH^2Z~iBKE=-WzIY7H&8A| z?!L4A5b<&YDuvb0$W{SD>hz|nfXlE^t?jC&#RW6q--Z_V#hiWHH?cT-y|k>t7#Vb8 z`AFP#KT3{%YQ3@LZ5iF%{??TDayVBT@|i2lA~^3zlnL)`HB>$04B*Tjf;+zIEtz^N zHXnUVD+iFPpz1#g0skeAL6ES8vzbQ z?H{n7-*DJkmT5Al*U~LG+(h(W1(z|#BH(D*t4e=-fdPGS410GMrEmVk(XvIPZm*Y7 zt3#SB(Ej)!DV*(94 zdO#-iIm7>&{2Z^iz9g5(v6Nu&DIF?-=a=pE^pqeY9Mj-DBUMbq&3w26QsH+7zMi|5MdLQ-SBRAWK48dj zdmj1d2~6z)XYWNKniXh5OQMK-TJ2%uDI^ZK`XZS>uAbEi(0_Z>mm??6hQM-zL$JPD z&x7D010RWYU2a#!NX1Kwj>l}yhxberPOl7`C!`iraOwoyq#ZOp1z%IAGSQd`sF~qjY_Jd5n|0B>iJ@ zP>1VYo|3*uy`Hb6g}#vpM=*Yc^_i7V?(^ogxUH?m*@X4f!J0h90L}4H-Q9&+9O6bt zv}MwccyR?u&+YD)tQn1PY1$-}Z~1&@f%AFwyNX9W&*sIJ%5U8C~2O|RBo0FCMu48SwFO05L`*pdF zbI}i{z7vk_4fQ$o&k*#>RhQyMj;kfP3J?PP3MkA05N)xFUNY{j`q=F!wIPNzX-7dE2X6%W z_b#DGFR2M!2vea!3pd<;z=sFlWp(=Do<+QQ{-LMa*nXFv)ivoa#ylfGPZ zG3v6mZW2aWXHtV#+*KsvHdyl6yYF2Ne~cvQNxr-SKkg_p{&a^207;1RyV2t>igJ3Vz{u- z*7!xHR0BmQRL;thWnzlTjYPl)CV+@%p2{Mb#JxXtyI8M0%FwOhcU;qr0w4feSQ6Ph zR`9%a^6FihoG_Mc9-lwT6mqJWoS7@*|Bh_5YG5iA;!{r;_j50?8cV5Tj0|i>te3V7 zn!z;c*9E&VTn|kE?ldI>mRB8yHF+rV7LH)E4a}0LU-3y+Ua)w*cID=skum$97iTjA z*<;W1u*k@f6CYVsOx}wJ?MavPuaD@|TH`Nt%JTn0r;{{{duW-`9pikFkOLk!Q@pN6 zrmKim-@RG*?ol(&R}4X&W_^d1#pD1;h>#3^1at$&QcYQ(N4x%&Z5OUmDu9D0#FX)j z>4qDxv+ui)MjW@6B9jJdYAsTNoJw3BO-SA}1iILBXJSx4{UUBv#`%0*FR%KY77QOl zOcRXLVpFv6%idDnfj6|K(puNW`dON8^gz~3)?msJk49Is_UBLnPHFd>SWy|*JjZ>7 zm6>uauf64Z6YOU>PN-BiyNXgG_yUTnZYUu~v4v(qg|BZdCvKADgtHFG*gMB@HrJ+) z-N@J-HNeKAUbu7hWe$ouf9R`PG%$Bs&%Q$TQlP+DfLJba4~KU94dD7Wcl&~`F#b(7 z{?Unp;hs{~DTHvMXV-S7wVTsjCb1<;BBdl-J;fKqZth*{%ke=EXybFjb8(`li~(N# z)5V;HKAS&S1QBy(YDrAku^!rxS~@*H$N~Khq6f+6zc|nbxtsIGL!w~>F(^qT>lE*2 z(Q%w=)N=MHq=P!_F(&l);KMRw`f)Xb4_2&5 z*fi*5H#i-Yrm2EGdT@}>TTA=CgfkAYxdWkv%FeqZz$s|oO=%TbokpXXp8=YN@s!#% zSZ;?G$T=#LJcO6se~AiO}M>MbzJj0lN$ zR=3-h#ANYtoYj6+ucL`?F7PU*RhR)34W&B8pOJZrz486=LShp|O6a3?pC0ZyKWQ|t z_4KKF<6mK8jT*1mevV9exLdcSg=C@hmDJC(VZDeN?X-@TENVDo-N+*Q%{lS`-k|>e zN#Fq*oOnkxbod)8`U{05=`>;M`W2m#SeniMqheTK&n(s1mQ>D-BHXktdt6~15OE7y zz0igT=b(}zUv%`Ms@FK1u_SE%NRjLYSViAkfz9-4|0nNWU*j^|r&>QzrdEmnKKW+f zJWzlmXb`_p=8fDNNv+-keQ#FJQMkPbs02tK8Ezd>Qs?TGs=6C1hQdl;#JmQbO6ttY zRhw41xqR~6FqK=4>=+PL3d{a57Ht+((#apihcer*t%CDjZUnb8GXNm#=TQD@QQs^4 z!Dr8jQv9z^>CW1+N{G@QqYAD%OF)J6x(#};1(1ChD>LG;9Q`=ViZb;?@Ot`8ZtywV zvC;PGn2oZ@)ChpH6mD*>c&t=1$!JhCTeT|GiP%KyBtI&?#BI&wvPHPRHENkiiiIBU zCRY;jxU;@VMqnXN7LG+qV)hHr*h@>frvb~F4*#mp?f47_`QDW-x{4`}CvfArW8FYl zuT`W?c)1IrRZ3FECjq*1znGwq2>86!Wl<$=)6R-uE?Y2=*!`9WCzh+WEd+E<>l_7v z$hIQvy8s0Zd1!AR(|El0vx1tNsA*}9&MxB>?17j4~=~S_cZOh zeNJ%X$bChrweW{MVCGgE!zop;1Yb8b>+_uoI4$eBiw3w{aig6*ubsbd0*?R_oQ5-(EQ{ObbUMib1j(|8U%af8j1T5V+qfLozjaY(PENanEvGwnk5oKz^- z1TpIKKFKwmeW;Pu;72z4b9}N?8ijb;2V!82BlldRJ?p<#>w1LYcI#LITho=}Gc}&5 zfTU4YoJv!kbG|s2XPZXfAFGf_$l*9NAXB>XHIG5q$9gCotM&PzCT~5eHAnY{aSgO< zAp6X1$5+fWPoPCsu0A&=t{N<=+o)vtmYEk6aYGMD$=BtbD}MPbR*g6C$8F6Teb2Bx zzlHp07Vv1g5SB+8;{$N6)-~IACY@<0MMN{~rfOmoa3t#mA)hxcfB6!tT`7mNrAr>I zb`f_I?!>EL{4GL>Pj$x2R2Jt07zvvd?S1M674G#fgpeoCIl|R@T5>NN2Rd$C{rYCp zTbgw3X@GL_OF(n2WuMb}|AxKJ z4-YSNv(mWt*7tms&QZ)e=&aS#Y&jx9aS?e=pzH+^3;jdkdk)WTK=$DTmfo)cQ*Z-~ zvwxrGf<$2;y;3bu@8NM?)oJe#Or!Ey3UmguwX|d1%Cs8yF{t2C#Cc&kTL}Y{fcOIC z9g!O+Pp`~{Qj)O1itvzv!K#x<$`bhn+N(6I%6%H6yePPZAqGW5U_w6PA2{r zk5s;xkjty;3Gj7&guQzW0hjAvuwt93p^uuJXjUzBoZHPbNeAHWwFo~(kmI3gI!?}( z4_N6!KKKk6S)9!G)ISW*4QC%X=!bWryO;5w(mf~&uS&X6y$Jp4Ry6YC`X`AukYLBA zdyY#}rjo@)_6xtCg=r?w&Oc}OnG4-5bA?GqSS`co4~OsbDOK`KX?$szu0IE(Q_t+P zoyWU#Imi?Yy?^y18;X)(U^llk*{ozv!1|!BOMR`Oqw7uI*ucsH>UVBeT=o{LF~iTv zxt}=4_kZ|YKzmfKRxfJzy)+omiPJ>zp3i0(--czO#+WrTyuv&HQnrWJ*>7#5eS1+_ zBXK#v*aC%vu_K>!$Gq+V=I4F7g0Gh@>O$A@sOFPS>4;Fmib{C|sseJGe&`oC41|{^JM}C^JDQ|cfvppr@ z#w!{z=J)Ll$;j13FJqUE3 z0N4~@Ugc=0FUqLqo7lJrKM3=`b5^mYNN@Q+1)9J{kK7*!u<27dV2DdHd0`2@fsX64 zV~u86>;{ME)|t^@WQwy1I7qb`nC0}Ri?{2EH}L@edKK=uA)4HLR@lJP^@E-F7A(mI zTFPuJoc!is48@hJ;WWWU*XHM^`Zg5b4coiO&%GR|oN4iCUCU5bLPDOigb%~5KLJz( zQpHz!D8ZS*t%vrfBPM)DEN{y`ek)#G*2`c7q;|pw#5wn!?J~jp;oY9_3meULT=9Sp z7xu50YS8;X&buaGiHIG|z&+^kBBE~Gva>9r1l(VGtfD=p5Z=K`NVkC#bP_7i0wRy3A~+(>?Rzwp%6)hV&Qr%gK@9rwuPA1lL_ ztZo+PHe()2Q0o1%d|tn;nyXUMpDe6mNBcZp_sdXC1R)G|?8IvZs`9)FmwM~u-e^0V z`oKVfXZpC4l-HWK@_g(Oah1wJcK~d?yNO}cX|L4-cZoLFn6DNu*{Bu8s|1+DRt0Pi zun}^bBV@{o3Pk!|u&Z#2j8(lU(|g|hIMRQ($q`5JU^qA@`aZYu>&A;R)AGcP49b^Z zl%eNvFJMzZlyQnVHZW}w?J)OTL2ICH80pgWtVK>3IqETDNtxiFK2X!9e1vn^pO={i z!aXIPX4%Pg8J}JDP)H}lLpUMNiJx_K54JF@03yPPPNh^9GEQQCrG-d~jw;$gz6Q`A z!{l#)HzlYYYvy8JS~?I&3`I}L5|q|k<&|G}InX)hGo`|2hqn*o_4UAM(gOYw&#Z>U z1>9bAz*aK3mD#D7-VZO9n*8z=HV}jRmEIx?N&s#|_<>Z!J`9+Wg`p7);ddM+9x*XC zb?m)XXvM`Rk97ot!@;wedh0g)Pd)zvjIu~oOY;fJ1^xjc5ImN$;pZ^FYE`;iLCeNsSTj>~W|DPF_Z8L^3yce8&D0Xei!%W1cRBGsJ146n|AjMR(pu+h9uF**1f{z8|K(@pT+_V2cIQW){3!c2xt&%O?vo>?MgvcULisbno72FizDLCr=8 zvI6BR`f4;64RZIH^3bo?z-`?u%23+0#12iZTFF^_Jtn3aJGl^pA#UG1*)z6PZ=}vz zj?)Juk`L6Mvv^?o9&u}5R?pj17=UgIoTQx_e=bLCfj6ro9uCsGW7+=g!Cl?5-&^Ag zmAr||_)}sUL=J;5K)*oe!jwNN%cdqE9vn5xLQ_8N<%#)oRufZL+XlD z&SK2Bhm(jZ1J@$CqBl;8TR%e-EM`5h=WLcLsFUvo`op&)P`N1O%hk^b5=1tl|MxO> zL#Ib~U&2wHZO<`gw6k^>0tIx|snvulCrG^e4T;GPLSn-R-VQ$niDz8OQNLj~MYGiZ zksGGbE~ ze=Fgsf~VblcnuQ=_(rq_^l;%jm$iq@$0VEpA^(%!{=IB>?HhNl)p@ky`1XOTv4}~a z8>o;jS>rj2tYq2oV{dIE0|#8Ghgn>n74;(O{~{b@=wRHd51TBU)-huhN6Q}!$N&6=&AS=;li=N>YmiKQ#xZ6{j>9j{|9(YopnhXLTE)I!Tb*we@RmO*#Js@GPT}= z{td7QLXx4(<@d-iIN!Q;9WKM1;C!O^2;M0;jD(aR1-qW19NeDdVasn>shyAKhlM{n z!2JhvZsds%ei^6VJu#Zlf9$YuBt~KO$Gc&Gbg3Q84Ces#5W4v)vI+%>2V}PJ( zqj)oY$?WlEHI-|j0}8?)n~@tZ_B;GzQ%X)GN<*$U(gt7W=8M+94{M=VN}8xvii*#( zfd4DwpNqg2N%eY!R6@<6_S3)b9!Sx>IRMRb%^YR`Z=>HUR>p^!C7erMIiymWyz+I? z?F^lE?rAcgbDsY3L)3v9dr4ITx0r;4argN2N1GeRgb;6g3Un=&B90aTw|^SPzgIs9 zL6?sk%|PH#y*qTyLK?Fa$2t8iDNq^$j^qu}IAb*s5&jQ?kTK~0n%#ds9Kf!D3U4!0 z95g_oZSKbrRz^2`^X}s>1L2eHa!85P>6ZlGpXu3Z;e{M+$ykgwOWDQ$*W_c54hdHT zvw6t_JW88-nO~{a^Gl9zeThVBZU(4NKQS3(->b9}2XM2%nJ=IP5USiH0*x7;UlOSn z_-UfKo-f$4sRlS#+oEq!#*n3`K&=Jdwz#CXv##ylt>Oy2B@#G%4<@Lf;qo#p@Ou;4 zEEc&GXL%y<)%t&p{RV6pg~S_75S*!qDvPV-zrgRIDVO}hkd#CDmM~e4){O$^`rdyOM#0!TFI?jsQRBO z!Q*v-9|UOj0D=W`f@`@zo}3?fkTMV?-@Yz7$xHhGeo>aRy20{m=Y%lu81+`$sb`u? zq+Y;|ROXnRnO&|QC{mkLc{@(!@xMkyL~Q#0pFii{f!172i9KGaJNZH7t1nyN{2$;M zYC{fUg49DtlypG9k9=M@-@^KbKOcAxqq&VK`_VgddJwJQH&*qa{V^jY5Y{V+_JXp4 zF2FmeNK6TKS4dc{L`yUcLiNe=^fvG>ffo5!$Z>B%A!ou zrr!55wf!bSFzo)k!+`kxOZZ!h zJeJJ8{9z4}=roe@`mg-|jqHyBLvVv#a6ak38B~}H2fy{|0xoV zlK#Jt*K6~7x0+M@p#ua#deC2JAvs5{{>y$pqz$dNVCixK#$Ig4Xw^1U!s$e*O%DJi z#z?7N>XUA(hYITVwm(fIU#Xt8#{Z-6{GCvn-LBwmkCt=N5;29?=E|G?zEo;^IVmS3 z;dG0j=J|p~FgzY6vIs$)FtK=`q~T?F!w6NwbABo!HVAJxVl)sF=oBI!bVknp^0j7` z`jqyptdErq^{t5k8{ymM~g9N_iTGB;OJjm)Ab8=LA$g9VqZ0whj=~dmcYY=fR-W*`MMNaa z|2LrVvj7$nP~6e1V1cxBG?LZoOav#t#^TPO#npYf_Aec|BGChu_-M?YVflY%WO*6? z{wR5#hH$iZ$d=-MIA#%R%zrp4;z%hv$g0P0PM+aRhP4RnaxIVzd@t?Y`U}v?Hj;vG z)(FZ7IK5&2J_nEW0(ZmMiXuPpRwvYx==`q`zo{&>2qjqd-m38t z#<*fOOMM4gsV~nb=U(|H*>cM92^Ktb9n(QCKb8YKoPx=MASng1lFz&5fDjackspJ= zhBA~ZFP0aY_mit&J7FSqk~SUtDF#{O#Og^^VNAcD?V|W4OBA#OwFjFW^d3|%Uf4I5 z^4>`a4g5*DL2yZdqfuQlR(h?g%UG;*`wR~xa?R@pJ)sxuVj?e(wAN(4exifhaj79! zh9R_bmGAPGQVi&Im4oh_DA(_RD!zB;^h5CzkdaKyUzGZ+q3s09K80d7^U?i3=f3lH z?jpaV|99?UZ|BaIVMgKPp%XzrX^^{L*z7L%)!P(KFfS)!^{B2wli-_r#y1)y+v};d ziDqBZ{^z5DpA&JPn3-e8yy>i0^>OTXH_bvywM*zl8vh$JF!X`cwqa!?O1C;2TcxOp z(C9p%>$mM{^v&iaMNMOQrbN zaFK7qTSi@vZ8zLHV~qB!6kRTUvLd4we~(;-IT`;e>Xc{`;u6NE`3U~s5dr&`E?B}# z&7fsc{jqIsAuZ2j1!96fU6Cw~5Mk~Luft@G=4|b-Pu+~XNA)A9^oD9CA38em@jYl( zHdED~mfJP7qTC`JuQfXgi3A>j_wU0WBtsO?%5=+A$BNOTt1)H5e_ntTZg#c!e7u-j zX=GDFp-oQk>ot3hoi3Kjf{vGNYfWvEV6-ntGMjehA|jN?4|JySC0H4Z^qnJM@ZjI=Dx~Lu!@M3N zm5A6h-Tn8JL#Vnp#Y3`$)zAQ%sl4*$wdwx(kFvXocq6He8i9n3x!Qs2U}c9hYTnaa zi4ETqU&QJlr|lykNJ~F747R+RN2jWBn#NbHq~7fn?5}M38U|eq-cKrXJ0m9rE0F20 zpC?H2df?(*`k_%c_6#ay#jS9>tMxPwjByipH~$3>E-Sp0{qVugKTUGwrER725#^SM zw?Cn1+<5Niv^mq+-uWj{UM~o@CUkoz83>}P$f1J-1b+P#qq!iBo%fQNyd}?6yMn0V z9X=st^QQyVMSjRcV(SpkUXsb_kSQ#lq)vs-Ly0|TI8#MY_pd&6X3_VZ+(y$@4WZsI zw3Hg!UdTie#9CA2*;y(j%B^HBv@t*MFSjqmHv6tt6Ls+akD@2YD%b%ad>bQa*=`{O z%|jFnf^Xn9kDwO|zK!fB&M*JHdB8)}OcK=GdT?pL1t<#JVR|jaa;OZOox((a8p!Tj zsu-RT{go)MTgB?m_(0z7asyxSUGb>IToW%kw7ZLIy2|C&D5|(PcXZT%kgL;u6<%-L z05>9jVxJ|}p>#$97I#llk)`^f#4)s#sREJJFN^6XMtz~X;|_;CP`sWu^xyjPLe1UJ zu4sViCeaHMsW>hlPm`X01HA6VtHgytPv{E!M4gsIv9+3=pc1aaaDEJ504T} zO2_5N-v!kRrJjyQQH!ze3?Pu&R>E|#Fp1huq6&lZFj0pn9i57yRfNMVCg)V^8U>8l zrc>)mD(%N%8}i-)eqmv`7Jf-d*}$13NCLnpn4Igc2Xr_tK!?MdWqx%f==5qgdX`$< zTHDS1dE~Z(hvR&2!$*|YBd92bX!BQsg!{!FgnnnJXsN1YEQt!Hd*QhM6%GrD-b3V> zY!+(E#kNGP$y1Zd6UGH#GuA~Ah;b^4+1PBEb|=IMNM_2~^4PA!ZnVpL@k?T~x9EZh3y!CyGV!o!7d%mnG9||URn4{m*8APTSZ%>`M z!r7A+bSA$CdL8@n@^W17?1z*(cd|MSb(5P5p~8{Z-F4YDhE(HWj?H~E-U#=|P~cl1 zh`Y6cQtIxAMG8lloiaHQo}~)P0hbdFlS=v5-m!J5{1ZN2T#JKI_+s>bq=icHk`fUn z*V3ek{r{wE9#SAj@T9)yLHN7*-K@)s6&pt&oi!zbR7Kv2QM~?(W77$*Xmsy2ILv zEZ&5_b)^P0`B&NRKH5x63f8*}l6+aH#p*fS-5FqqTR@+kKOWCCTG0IMltp2e2_!>9iR_k(^YN1;t9eFpCiFdBkWXbUzCOFW2*yitVbeIZ`($FWp zNl3>wu8Nf{yg%ogla?7z_1cm2nqITiMRKWVPOgx1+@iCjH{G!)3avJ#DRp0Wp8i9& z08q+?4Va+(;5#em=EosZvpJ-aSBQIpmX2(W#VOw572^DtmO6EQW2Mz-Ax1S@+=DD` zagk2L#API&pHCW-k^Z+j0~y#mvy1YSrN{1KTL{U{E+M4T$uf@K2$cH9E&d``F0VyOkayxyK z+lzQW$2VuvH@sZ+(Zp%R@QS8u!~bOKzl1@Xt@YvcM|c6vO8F(y#^@in+Kaq*sH3RE z=MM?Ni^Zs;vvXhJs=2@Lupg~5{Q11G+@jPv#8Ir)$y@4cfGbt*?XKyXjj(WbFsI1N z>!KXukBP12bnU5ge5!42SNXxefK0TzW}Bcb#)L&@U{ zm}>|2Y-wnr5my5kbXOA~C#k@dlT|H0<-zeMP7-ZqRS;g&!sJY4&I7RZ3x?-z72@vL z`z~v%lu#QBvRnt2GWH0M>KQg zx>BAhc?5K9N>p(9H|p55G|BjUnr=cA7mVIvOGbY|YAGp){H@%)v21Ay~-S zupP1sWqe~$9IN71&d(M=K5m=roalEds`h%s{Gv}}GUXP*La$!?KDGWR)T#luFEo;% zWO96+SK1JT9^7p4~Nxe1$EJB%J|1QV1A_$-tp^vymLwK zD)j2%t;gna@;4ZHV}(Y|zc?Y&JJ#Q$+k^f=inh0O(UywThCs=ik7ILisJocXl`Noa zo_mXVvpDc_!|^R9BagcIp@GimIlLtNVWH09m-MmylWOxO3fbwA@Dl3p{lnZgdI8#H z&F#T@qv&vX{{9U}N|U2xt@B&2SSA7>Kk87ep+wtdYDE7^@j@szXRAb8`PJNRrh8)3 zZa3#18W6XZj2+89`z=+?`Lx`;Gw4@69%BAisSp0XajMXQMI~EC*_qDZM{ba(iW`o9 z<&ySNQ(~~pmy^EOiz2_ixK#qh&Z`((H>@-Xn%4pTB93CNt>?U-hR)496rhJ-VY?#9 z3cln9pRRtg6=~B~3BV)}jVI1&4!uBW;()_v|1_oTkn1xcb7(kQEuipevxeAVdg`Lt zPQPkJ$ND@{3|eazu(YyU(^?L~4j}MyJK<^wJ2yqJPLy97bd3G~ElWoR+k-juW?b&5 zZ_e}+kp5bJ%Sx_GKFZ}gek|jRN+fO40 zMu13-Z}kudPWW&mp6x(-kswmB=MqhleCFs|2>j@kwO`n6=2R4+`N{TBN;FCAm#yo4!x2BGWV!Ll)PZ`@t9oonW#=Gmrl9~KB)42 z6dOpN(!9zDW2qsf25d%VzlV0;thDJ}rOGTEh&#DNF3bP)vyTOUt1nj)=|1W2+&2QT z9MY>iZI3@q)^x4)rDSDWfk`>k8dB~4y=_fO@Y*+6?3FKYSmR~4h!LV5QK?OX$Q`f) zs#Nu=Y0Rdqds*q~EuI8VEhZiKDhXd0)A_HZptAm>Qntxud1T5 z?*1}@AWO)DE`c=E6(nw}Uq8@#i-)*&bVH@w<$aDHn9L+N^i8c6#>erp@=r0v0f%K& zEk>!fm?S+%<(;bmzu!R4b6d5k56WDH@@~6&vtO7!}9=3yxcBl9>(xTa)kY}DiiuEdzUF3Bp{khqY6ZLhoNr0^rd&Hr=5tJ zDS*>IoXit`eP|S`HTmAfV7oVz@&lXtG#j&G-UiR?*GS)l><6GKl0JKE%Z{W=oSHMt zr?QtLKHOWE4W~M!*dVTiFm?=-G8R+3*3UswLHDU9%cXP1L5w77(fYf~u$8jPVuTQs zbogDuH;v=!I{V}ST+dC@^s%HNm%0wPdVqv>n28z0+=JG^kT3G_A!@ZSE?d59a(}_6 z<~0tYSfjKFfV{?3OR4QbWf^^ZFPg>QcyR9rYM7l->wU z?E3Y?%BG(TF->WpbjV}zB-7drs8vbG5(i`vH4#9a`pF5`2LB3XLTOWwgs*%0HU{ZB zPBIC1#Uk6!KUz9q8o|zKK8}ma!QdYc`xA9xskz6+0#F|!-1eZ6$$Ak&WTwI)08atR zO>a%!WbJp$@dJAMr;k~h(f3UR=ijW_}`3JoPP6-og!3zl8$obcY*S6C7l+{?Q{iMjLTSz zhumOU_nmNr9)FHQZhboX8dso*;m?e5d7)X8siSqlvIcKS=E>b8lh+sce9VZjyjqxi=L;4KXX{${z1qiLbc%aiHnr>%#+Hjm zCmAYZoo()J;9JnhjhA~Lg3WBL;re#p@(QvvH@uqQXI6WpUxD1Ek0?2? zKtArjG$)Q*xe)Wy=EZ;TeV%Y4ZoR?}&6i?uLikQ2rTJj~F04?BccoOsja)k4>n02) zx^U!2Ex!Wk8ZmX`iALCbeKYfaxGU+r-r5>y)2^@V?f>Df2sFSc-;c_31WM+ZxeS$$ ze`@fBydQ=Z+!FShx|uItN;I%pqDx+D?_gQK2_BV9lzhgTxX4qSzt|d77}Cy;Hi|=3 zb9vl7tqK47IUvu-q}ltL9r7D2sdHN^hs5tNlRf`vZ2snV%K6pHuuy=xbtI^AG?~eO z*{ugR7G5!WLgpTRldpJqb+aW;aP~dJB0g{2d z+YF-+k}Uvx&4@1xtwL@BrYExXC`TTz^KUn@Q#s)rCj{r;$CFf`v<}lhM^xA4gAp%p!8_IQs5#;7_BsjZUU$7L%fQo?1U}0Jh&pv@_0^TQr^c*Oyj=!!b z)t`bp8V1!aQD%`r#rR-?c(M&c5443zlGXr9c*OlELN%fv^h%$FsPi*H7P{b5#1)pF zkYd>%k|%(@NwAlqI!B)`#zac*{f!&l`xgYpuwy5``Bn4Zi!jky{C?uY%EL%pwrLz# z7@eOgE`TTbs&l@Bh+nnV5mA{pu)c12U8*nXL$n)(kE$}MG-_7aeQxc{x>JcL-feeP zRP3T%0HzQWUK`n;F5}K7L%`24eg2iw4Uj~JQ)};xUbI@3#0F;r0*W<-n`#*W*;Y<} z-tQ`zY2h|4$Np)nXfN@aW68vCKHkgeig)cj2;rs=R}WX*%R*SL5(or!=YY7G<0?6$ z+l!!qTpz-vOEk3g{!@{H$vt(Y8fgy)FKXef38GQwnncZes#bs5v5KFp1H7oVMzT`r zG_G2>VIn2Mf04rMpfk=X8lG5DdHy+gi^Gmb6~&~M%;P&7z-JKl!rPl=L=Yob*bC}g z%G6dTqx}j|H)78eH$e%5C!eQpC5tCJ>GdSdhPjhUe@OA~BYV;^P^uk7-tNdgvCUXc z!6MI-(*VKk#m;o7YMvz9N7jZ17~YkD5w1n$^*f*!lf}9Hq)wkn=e0V@c64TJ@E)5T z=~w(@jP`YE1x)nkwE4{<-;8tiVOI$xPuQox%;yji2e7A4iM&c{<~2-olb!z07@Nbb zn-8tYKe?kyx!!y|x+{Cvjn{wXnGC8O_jzS;I7SW*Np!||?`$4~y>;3XuhC*SNY!c! zo|0c|j(k5{3zw@lU>1&O2PT#%zaAz6bE4-Mmsi);tY$EidD*sJxA!i?Fd_$CK)n6k z?Jgy&9-1s|_W1a(sH~s>@jP1;Xp4joW0*fH!T!0MPABdA6K=(1iGMt+nD#ck{4V-& zoy$HOtRGWO1peP9sk}!+dx6&F$(E&swU>dj8YgrLF~~HpWMTAc5@W!qDj`nFo{H6? zcC_uKZk7UU=7`L$SiW7OJ*1vag^6Yn>QBw!_QJ98!wxw6edYVx)kg4?9hAQ(d6werp_OuPFfl?y-P&#cDpjC`Vp45==jibcRkskR%*RP16FKvvV? zaKYs5`Kk-&M2clo(#{qbqm{rH(+}*+GyEl@dM=U=h2eZBRy-+02c|vXFoqn&1MTsc zAJLis^D3oCQgGj9nx#`#S(TBbtE>tPCY)!BwY#YID9@tPLxT{an&HR zt-Kv^)u{7#mAPg~3!Wx7n7BtMCI0b;yrN=oyA~^_@qwi0TuEz%h>?zX9sPy;7280Z zlaVlYWN6cz;SlArK+(GaTaUWEA?x7i=dIY5l=zD3T|=K&D&+7IgVdXP8_J5Pq+JUx zx0hxANl!&_<8ckzE@Bppv|^nC*8P-rKs)tC!OJ`LS-Es5(3@( zT5S;iJs?MHgzx%BN(~$}Rw0hi(e+eUaQ+rJibzn@42BY3*7qRZ#0|2sAX~kB#0X(8 zFIDdFVep_@L&WL#n+?e;ikZZdlnI%;RLJP!Knk_9X*;;cP!`LzgXq8~%e4e(N8zZx zaMtg>Xou5K{_!I4HXwS8GlksWFRrYN!MQuDy1G{RmBrM`70LoWrTS8k$~mVSxFGTe z46A{;5_|{a^zr(dS?LHIEgjc7j$MoKU0|(I(#&Sr*nJr*jY_r+!Ait?bnhy(e44kR zJ7C0NL>+R_r>*y6zJi1(f8EL2v?zmJK(A(_E0uCVA(6`gE3oKL!0#n!1FG!}_`86a zr)5T4<&0=eODP;xTQj%b;Uk}}1rrkB>F8}sz)S5lnOw6g1`Aj>M(ipWD~{hZ`?4*k z{=|)(E%5?9AU$Bv=>%MDef#&WAd;QBm#-X7?V!k&P*tds&i)4CvCq>S( zTNnXKYAE7pBJ*iw4d(_}Ttddu^2M7$RCs+Vmy|+YKP%1a6X2F6-(v(Ye#s&LgR*v3 zp^6ksF!hwJh@$$c$ZIdlvR{6q1H-(XzL%J4hpvO?Dvc|5JPofz=nTT<==6UiE##bx z${LM&-0LkaUPTfBnXTQ=l3-O-e_Eu%@@-xhtOINXO!3Eop5YgzzzkpL!Ky8S5^z3~ z_*ah6p01uKC}M#R7TUZaMTa}5AT$MQTyWbNG6vG8G)Wo7TjHLN8mZss6!7JOXZ>yo-gao%B*%+~w~COb9@?xkEx*>MJR%&6fJ3SaXddDHK^* zDoEy%{3@d%O+?ko-;@XdK%9gDT2QYzC6iW0H*N$%d(90}BEUu`XOGKVMPu6V-cfYO zxBtZb88)25x)NS)yXV7Z%*N`u&@C0%7c^rXSt7=dL%ZwPlcAiP%^j9(JAhqw7h4QQg-ShB?L0? z_2g>ovN)-vF~3*hl3+ePseH1lf{q#<362ab(UZU>|v=*(QWY2`?j~M+fp$ zZ+a|3|8#N2m0SbDbHA`jQ)_PFV)v~p$d58T%_h=V%6gcK#(CfZ5tyeq>bM(=YaMft z^zg!z_B3C>lV6N8jRpn}ZnChFlN}=?8~7b9O${HZE0YdURE<@d(P3g=61g-c_%xh> zL?Gd&zDx6LY#YT7Koh|K0o2<{N%_>nQS*q)f>&#=ryY6`rkzQq%HvO`+>)~kHzyJ03 zMrz>dy>?*~_AS+b?89%CF%0(=eBKWY&%e1ky3hBIeL~y)L_;Dmu??!{Nkz}9gOK_q33B6&sDV+P;b3Q!KFz=4tw`~rdQA|q79aJFAecQ> z=qO6j(X0;!r8)lG#z`9T_(y;&B-ZF|&JdFh*)bp2K#?8SIX~5$t(#{)bF}VfzDyK| z=;1(2kgytayr|FyfM6xJS@pO5)pR+yf5|}(Nz}kK>>27af>a@97VeZelK)ug%HZdM zg=G&Dks5qhlo5!11s;%wT6{i-M{-eg{f{pyUct|ee+cLWte%wX?a;;2qEAOhT5nPO z)4^XIuuvV|n#TSd_eJXDT#|Nh4cH27IGiv(;#`~-Ry!oUIZd#svKs=*Z|DzF`=|X| z%O8!-DZid(kKE1VK3

^&sFmS>-}8$GoI zuxBcVx1pprisK_51@9EaJrN;PL(C0BNmWI@+9D)ypZIi|0M8tTU?;Y*yiCRmZVJcN z%y{_z0#q0zJ}uN$y{2HZF#xkn${|wqNPK2Zc;)RXGgjkunr0GUh=ag=`qILsK7t(x zTa=tJnMc{}uErxRLerY!es#IiOP8Q>-r5%l>Nz2kERmoj8**Jv?>eh;PGyI*&X~uJ zg~W&H`Z~GEoO{tL7KJ*wl*=fW(fE(E;TwwwU&5=o`yVmJoZ!i7SuLaU>kcCBcDUGc zg`y+6B=ePM$2+a(rW z!o7L^Xq!&cF;?+vDmm^=0v}`yOI%;d>x)9DTl{C#6FLEr(uI94PfHBd z)KO8H&9HcJTQczF3z>cFyIpJSv(Q9drm1bZ!JqjD43g9Q4n zff=j#j0b$*HW2R^3ws@=f2JgvvZrV*o|ohKbl52i-kL#BaU|{BX2Zl`6@k_0cHbd( zW3KakhC#BZ_{qqsqu@MyBpNx6*fbLQXB_Wzx_!6*7UN?YYi5$;q{0bH2yjFBr39Cp zPPuv0X)GXXBcMVVy+L+yRO*h>>q_;}kD)^_Z{st$Y>I1o!s8o4f-a*6oe@K#Dm%7A zi9VMWeq84r6_xyeV(i{)fS_w`>FrZ$)0XKpu?HB-YwKE9bN>;l=Cc=my^{ZA06a%? zm((wMUV7<_UT68&9u>%`&#BdYbd#|tl*?Kj9pB&JO6xKzioiTJ%|=`ShW-(_&Wq%! z0Ou`EX(rVTbB$Pw-oke2;{_QwRh)dw8jEdoTzrCLH#p(SubB1Ft^+29q)DD(pX53; z>`^Y`vOJG~MBefa_-acMvp zH@1GeD@ThjjWiHjC*?vZAbdnu>8@$ocZw&{UUQD_lrAU1X5@~%7GubOZq1{Aa!%kl%UCO7y$IgTTcEiP zR{X8mY_bw#Z)F*OJsE_M&DhN>);*P-fIVxyoIbDqgOD2|5<`%TI&;r6C~PiDEki-% zBXpR2e>cm4-FG__f9S21jufqWRTVPk;OkpOnp#Z}iGW1Y$3Ip+I^cWPJuj7RDkWGf zr7%fY5Orx#_Ij7XF-sTr!K#n7+&?v+>rI3~9OdOfZOU9^0QSp~LV6i=Y& z6ui!HMRS>?hY;CoShttLx$6zao&TBcz zq%O=83`yP(mQuZaKBGy41QWt)=)_=bJ^Pe zi_UqsAOth}NUSDAL-)`DthZ|pkyaIImnURHS%!sq0dOl4f5$h@R=R zTbb)jJ|)XUJ-4;~`Hu_+paezN=DN=R+mKI6gQpdf{Bh_o6j+m_;$+1>jz?Z%0I{sc z2kjE=bPl@mhh8G(UHFh_WV>@r0`)_2`hA|XAj7$SsWAV%nHG*f?MbGKYPf?b*8$MY zV0?J1X-5(}C-g2=0wdE8f-tH1Xhv!%e{^u1jdgluOWL0wR*{6c6(*RWfl7oHHz*>T zG~qiV8S&D(nf=&Y*nx)C(gHFJ|1^ScJ)X22!(;cV4CcMtg_C`~>GF_F%V8wj{lMnk zKj)kImMi5F9tlar7IM2CsF#_~Eem+vBG9I;6;n*^AJ zV5vvpgLgew9Xcd)j2|^L+*S;;;ObueVL4hh2!5!hkV?uex&xvct%-F;Tp#O|CZ&A^ zN2Mp_={y!|{N3milNseC?|(8P;F6r0Gb)p?&%BiE7mT$1oF9ArB7@hZJx)ODSSoZ9 zE0T0gndw!Rq_bA4uY*JlbJ`1(-2njYsSD%Vmr5fyOusGRPw}!Su_@|f7E&35X3P4b zXT&E|=P)!9X4)IQ=JgQN z-bfLH^)BG~(wod-6+}u+3L+9HgR|X9G}cProqhLcJ+6h%Ka|4~Msc#H&GX#doYime zzxwf=k7XEu0I%35f6V%I?^`xV>W8$;3#gooJH%)X@h4#zyEdWvw<;5IT!|o?00SwO z`4mt6p>ihY_A$!<(Er5bpgU=(Akz$0_zGt=E|Xh3pxwy<^P-o1*d5HMB+ zX1yu*cvER#LGEOLmH}GQk39U+bOniJK1nZgygP%|FTFhNE{M`>&N+>M=Q0py?_f3+ z82i8YeYO-7cq!|Dhn7I&vUQdwdBjDmA2^}+Ceo{Yw<#O}lpXpp8*c^Wj7{@?!Ih|G z7L_7VRxAmM>o$NJJ(ToUp>sk}H*tcNI~EEaqf3>w5{o<`kwcAWaYmFkqk1{Tx8x~p z->MO5AENYHf0b^5pr_b_sXM}`wU5Q%bSy2oX4M{7BAcZ8ze_JISTx9q4x;&{5kzrWPZSi(M9i+WUQT+NXUKK=(gii+T#vxJD2Ysm?gj|u^7`fu_&D$b&V!y~@xbGq!TAxJeJ=x`j zo$_n6dLixj+M&?b<6nWq|3NoFL<$%yXCd9(Z!998Qh`2t;`k?#cmhQIC^R*dVk)O< z)5KaGmxl;tHnZQ3*)p1=?zMzjor|_C%LpK0t0mHLTTkjAx(OkQQz)X>?HKKg)Of@L z5kD@2*R&`ixa52JyP!LXB;#~bFCKjcPl3@WB550AnnvJ+fPL73;xDrT+qPY^tJx!= z3iJ3)sKS(|xK!raNo(GdhASn#)#!ENY5Y62)+1|Yz%!cwWs?|#Ipe~HdLEL#3a-Xl z*T{p*=cWJ|d|BX5js6UjNF$8qr+RRqI2L;|!_5?{vC30#*`F|lbLy{R{%S3g}dm$^OF~`cXQdmJ6c;M!f$zuJj&%@ z;HC&Rl)GpdulbyXBv6>$L>n?x=H>K4q*l8&Z{Z2FzPQK>yRP(=)tN++XgBYLvU~r_ z7{Q0k~z%9 zZxUrGUzOw!$>8>;#n?)kSOPi6_a5#js1N7flUj7Vp&b>WGqYt3DU^fyOo>8al3vRB z9Pz8YwF$tik`?&~MIkYDUqy%|!o5IlJ^`lC({?k2woWuL7jq3FU4q!9IB#a9|qs#n>kb<^8gW!7pn;`_~_kV?-$?w z_G9un>o!3aM?m62Bmln_Y(_aJgNMkh`SA7P-8b_2R!ZIzd**}vGc6F5BzHRh1!bzu zLtG4n(fN2qYx<>i+1q%PAbl*}_iP>0WMehck@LNT)UWm*1pF~$0#ywhlXsbH5XZt%=% zMaUPpb^XXQ)h4!}zQTK3HYtMYGz1KYyRYZ{VOXWzsZ1!IP5*Y&%-W5-xax1_siKa6 z-)ZH2gc_3jfReBd1Rk$H_iSiC$EUY}KK|W%^#_Nc-|sXQ%w3ZN7bf6%oZ%sZ7OITQ zwQ+j-`^V#C(>7Z*V=-z|7=m{W@*Y?q$lEel_oi>hPH6gPB)uA6ps#OHG1s9}W_rkf z`s?LMb&H~2^N|u8UFpS#r+t4Yi8ns?vN7F8lE#u3R7Bi3qA^qU^IdfI#E{;djp5HZ z|0T0GG>iJBzG_->d3N_=)4;SIt!_c^-Z*q?eyj+Q%(Ha8GqkU(LH|az!pOcA6BCnx zTs_%R=_Pt|C3Xy#vSR4kF{~AAMiy&Lm_)nednJxR)q*F!hE2t*PLnNsU=;v7^Y#4T zCwrHof_a`1d~6J)7}gtsPe~2DC1Iw=-#fE1;Bo-CtEwrwPn$Pd&CHxKOayiF5Lb&kFLq$7=oOo6 zvEp8HjQY^PfC)k+RIpUetg!*;n4a2-Bho`+ink(}L!^z3gm1^;D2zxC$-aHL6rq>T!N zBT+RR91pvv@L91Pqiuew*{$`??+J{;WyeEJb|IDU^xnmLgyg7fhC4L2IYoCz(k0Ql zS=Fi@UfWXUuF!U37_PH+dd6zI4p3<^QbX$nIB)|^!ro?M-j^v|bOqAGcjnwHD+SX8 zoSa3eF8Ul|xRg{c;)Uss1@gLy`rI>kK0dVgr?16*0FR`bCYhFIVSBhYx`4QmBWr5I42j5c5%w@*F^egOMu ztmA%ztQuyEr`ElsN)2z2oEf|S6%h0I*(UhR?KnLX*T0%JKvJ=fu4i#_{~ z!yCZJw<||Jadfk^xSwAkt(Lm)vWqn#Co8SGLK!ehF}$CEXwjR6xlbKrhfChCVI{Bw1T(GloyYB0q$BQQ zOkP@)G7dbvG|*CN+>T<=@cj6Fv$ARGiCa_gdpy3Z*I_q+Hp<_o!!txVOm0GWQth&%OIHOCka zawB-h?|Jo_D>^W1W!{phMfvKyfeS02^zDi810*v>np1i^Af(2o?%BBwWb!VSGo&%N zcc^b=M=S&&SzIK+=vp)s%~)vsn$ZF?koumjQ1>}i^m_G26t6!SSDQW6UHQ!0;42iF2&t}0 z+E4i5p_TXxg_v*6BNnGSP3|qQT8>{VQ=|0t1N^)-sj>42@0Qhv7u2bnN9&{dSZ#_J zr6rXz^NTD6BusY`Rg&iiQM6tQ?ByZzT{=HbCZzf?5s+=O zRfYnW%LyZM4N?^zWt?VwfDtJsHGteKxSp_=KC?~0{Zho2UDaWWYXx-(Ugj| zUje#&AW4Oz2V?G{1pq2;bGWYGa9T%nj<*7g%k#41(ez3*L1G2BWZda)YClK`<3F2- zeFK;LlFJx6G`>t<1|eXar~S&{ zts6^}`vafb+mgB(#ZDw#D7uJxMo8-@|@4M{I6s;hChK zGNF&TV$lO!a!EsCO>6Z0s$aAPR5*ysL@)lBL*hCC@|pDHz&A({aeoYA_Y~=o75JMUEsPF~BC4X1?}>^ zYt}un9at0Jv+YL2DdO?333uPJ#|&JoD-_BUoj~S$jgQjh^nPN!#j=1UQOPt2>I5M{ z9y>Qjb%%n$4&oC&n$J|JPY&#l+$T}?&r|;@)sbN)*WQGfYT58_cp^U0b8FSmY`0@` zwam>7*4z*_Vc~B&H2!_YW=uK%gE0>JXGz9?38Sz-!r!p5R{K+g7>8Ro>E*BIbmWi= zPy>^V`yRlA$^rsAfd49D-M%!%-Rj8uh7COX3gGOU?{2MEJtExc;=EH-jF2r5zT)Fv z(mm!Wh^id58+Hht7$2zPj?|N2H6C;-7E$>Q?+y&)>&pMU{Hw zh?N1$xU)~&)na__VC(Z)<7Hk4{?nopB**K!0&DDuU5FQNx9yjm?b*}kkqPq+zn7=va{ApD3ABzB zE+LsNMjHcRQC!k!eYXBgN@#8Mn&6~Cbe#B;k@uVdm#G_JW{|o+6uixGp*`?%;*m@e zJIhTya=;%VBz6Qm&SHhs+}*i+OU9>E%Wb{Bs%#Yk>NWx0e4iRaS&Zvl3i?m|5bx&K z)Rv`|;zQ^0r+9*;kWCQqNdUS|+t^T-)M889yfJnzP=2sfxd33{8SmIlNDJLwlG?ay$R5{uDw5qac8`4qdk3<*+^= zJURY){=N-9_k!3I)npvNw>{s^_s}YRQxwRD5U}lP%X8*8cRH@xZ76bZp}YW)Or6aS zQ_U3oB~Rs(eoSt3=jV9ku|y+MPd>j~8*taO&0Jq^aww{^a*U zTI7?)TsgF6Vwe9Fdpi zves0SM$Wvxl*cOpswGR zl)!l^3q+KTmqg^(L5EOKB~&9Ubhmd!MqDqrZJ_dTV>EZZO7aB#)?hRb3~MxpW<=;2 zrk?pQ740T97Cx16pvsPC(5U%?DWw!x{^3eEu#m^PY(s1(_nD{w$e>NKe9Jch`|9WS zQPSgjE2>&?8lMA1*x`F-D0!^3s6O|@I3DPBX zC4v#jsfXqc_V4iK> z@G>Kd#mNo%w5L8Gq7pljJ^l}r4W{a-5EZ69*1vCWIPVPE(LscOWS#>9^-MsP98zql z((@GdQ$~G zljNk`5B+LQ-ET)7HJTcn$W$lAx!5#}G*IQh>yeUfYOoAr+2dZ4pjB@}a@Y$A{s}0J zij(ADaczuSqr$$DZV)ihC7r!^AWZ0G;#uhJvS=dSZj^Foi3 z%~zoAeHj&4GLx-#mE-<|?FyL-x1Fw@f6%ajReZ{Np8an_#`x->3~k5u0aUd8cX&{A z2mC9R+p!#npu`Iq#J>VuQ%*XqJNX82KP1BxIgVwy;E16nxFMG{t2BlX*zsE+o7(;M zI+Gm@N?Pegh6>+IguWXcyE6q>d91bp6heHAE6atFBS(lBT}`7i&tXZ}$x3V};wllI zNActMOj+)*)Y7m1p;>^8np_t(r3tooqs}G`;^7fT)hJj z+!>|Y!Ok?#s1cXm&7HO-{<^M60q%LbX`p(AlYh~#S13=1*WdU)|(VrW$=|!nH2zKjU1wCJ4R{opUU4Y2RL*TNd8o#rR zV${coq!KG~gc9qxbzX8?yHyTK%Dg%Bs&-kCNyt@(?>B^bW&%K&joA89)16+1dK}`y z91C0bv*-S;X#8L;VjfT*MxGed|9;}ltvRb&*aNK>ayiF=9af@P-;x>T;wkH45|j$f z%i|)S+Au`LSC;8vj=4<7jJC-cisMcotUy=BQ9jI#1mbSN0iXrOe6=QE)xThRnJpx~ zc^-0C;XuQz>d0de4Dze70VCvqSFK^&MCYE%rgMVAIn8XSeKkREKNM(0Ta32 zojNM>?lOmGa`@;{sGT8k6tfLdY1i`dL9NXe({ruLn4QP$#ko|cw zp27cCMdTy+$Wx^llbISoko9x!yhjE_Ln{!pkzHE%{Bx)s^YNaRC#k*tL*#oy9!)Jb zJ_99M9Xe;4dTY>`qEt&EpnZ;&yYm`R+7AAR&WhY(3%6bhs70mu zyokLUqr(A|LSrFkp}SUgEO^y%Lqx-KNIJeQUR!!o7c;2}R_o@hd>vwf&Nt{`TgK5x zu-cEp1>-;Usg3`uG*Tx}fmfZNbpg(8gdu;H<~j4EtGhPuR(85-%g#B0Xp^A+pn0lT zC{o^@pUli2i>Gi~dxze5`ds(Z$GfB&JR45${&o{Nv`vqMoq^ zPCIWg+PWY)-$8OFq^}_Rg&080(5kv!j{31cNZJ)337m`$fcPkf5Qbk!`t{R_U2hl* zF6c{^Juop@yIBVemssMD)7RO$w=1IR9)imwVHaFdq5K0k*)%oW8L3s^blb9Km5F_N zk*;lvLx@H)K+C*WNoXIjXMTy;o!T;2KI~-FW!NB|vEmoCQrY91Wkf}Y8DAREkB()V z$l$}0kc(jKn*&F{2K#FyjODg6F*hfd&r;OIC5zroaM8~{mnekYESCUzDumdyC{VW2 zzrVNYNlf?%8AJq348DFg`-2WBv^^F-O%Rx&C-b6uTAM-$U;?dGVwBcB6n*CC7*xKU zH^x1nb8a!-=Fbx4tc~FE!)iooSsp?n0Yy0oqd|`9T!Shto2QwB-ZgC3@9~OI*(0^d zfrS%~dKm#2Y+$Y4?aIJpxgPT!LD=Ar7uUNI1{MOML^oxIPOa+^)MQuhZ)-;pjo!wR z&(&+bUgwBI-nzS_=B>cd{ZZT&RChGq^>sZLt53$B)el-2pc9G~j{{U$I)d}-@~$F5 zrsF-1gKA*BDEIp<4_7tS4gp=nIC^)dhJsh_{DI-WLa1L@xBya@wtpIej#xMP1U@LpbAP%g)kBN_|MsyNLD^7PUkbPCN?C|ux% zik?1MuN?DkyLGrIm!4%~JThuiqrqT44VA@bz=fz>>M_S^0+jcWW+}NAK@oN_;;hWW9g|LWNxEDaFm zU5fYB;xo3ioL5n`;?j9x?iX(XEyL+X9j;U!kzgPbyB%-X4kf-Ud%Vggb@fBeDp`FaJ2NDT6x)L@ae_0euuC#$ z4RuC?kn?c_9=V!f7|ML8@BvHN;+QUA;;sAcfaB+gG;s8(z6Z{;fE4X+^^MMP&(VH^ z=#@RXH=*;_pJ|3Y4}CUH`}B0Ja8n^bbXI*1Kz+{_#p8KBSpDGA+XZ*Yo109$9lAVF z0qlrZ09=6@#Ih0Z+NI1`*iwPNGTIIq>Ux#|##HUIu1za$;EhMdy&Hkt*$b;n?o`kd zVrli2w^^0K-m`+{Rjm?sN%_m2?Lco(_*Mh)wa*pm)G-bL6>6EGfQh1oX_VG=LgjZnp+QOO8U6dT`xwZ5t ze}t8=i>ITs1JJ7TN>I(ZoD%fP#^507BNw9_Gh?J0XGHM-(e>70Rd3t(uz;X|NQ0E3 z5(=V{8$n7M3F(rM?hZjI5s_3%K)M^GO9X>%HjRXUG}0m^@BDc6)c5`UbD!gRyyw|_ zueD~3ImVb~ACG_T)+go?=HyPp%H0{a*Fdr!kK~#wZ_CX^eW7W#&orB@7> zSywLV)!RTShV!Z)BGn`8*5M%8f~vUD%t8iMJEBty+^JFK%*D^|PLIcda@M2he3a7g zl?&Mu3*01SzQQ}i+0aj3N!M?0@_SPtf6pDKbsH=FMQ;8!!7K@y?v# z2c?*{>=t0}I5H;cjAvAtyDL=Hb;>oxvt&BQX4RouanBT(^2WM<0~Z(NWEQA5W)f#D zJ5&V`XZvo%ccLqFNW^L88EPqKuqO0(?QtE!>f7Km?Cu52w-mraq$k;aH6p@rXVmhv~n z=AOAIiolaeqRH;K{i#4(xLOnK$T_~8*Y{kHe}Ly>hCl^#E#7MBy^dYS}lE z>tx|xDg(Fng{j(QA8DSS87MC0pTo* zI!zB^!kQJ(-7R|y^y)nk$}roHUF7_Q;rvNV^MNa0coFx~^8pctn%yp7WgO`qmH7)V zBC`s@3Bu6{8&1dTzo;X_}S1GKg5G=KK4*$ zrazgl-w>uEiOqz0UqGrY*PW?L;FG4V)L6-C;#K)$gbL%o^98sR^-z=UGfF)At24fr zf%L!iQdwv^j&bFTr{EQoDsB^ndFzos9%(KapSSLfhJlA~jXc2yJ zWuS$dSIU)GnE+13Ll0yqgCMs8#~)}>Hu1`IIZBx)!%Y+2n$X+1w;h2+_Q--C%F#zZ z_vQu4Rnw3X1%YL|S3g?tf>~b#?;UdHI;df9M6wJ?q>itoXM+Wlv8d=rnd-MJ(Qg#D zliX+SNm|EcPo2DvK%PA^WH}Z-p3R7@%PP2dIjv^peg+6}cUJP< z-p(~o7`SuO%mbgO1n7%Fyy`a~F85Vj`+O8%4dYH2--8kM?Wr*>AmHsB91h;|MbHTXYY5 zs5Vf;V9De=!?niQ`Y2!{^$!_TF z*40M`=eA@W9KSg$nct8xcXwD*-Y1QvZ&?%gwgNpMOKRV8s?lY*vu#1jEMKOqATVI* zXu4`}&*sls680hJ;SzQO*st)~YpJbj-9PYm!(X||S zT^BR>5c1i+y!ACf+>w}yPU{a&KmwFaVBl|s*Qj8EOkS-U|!QhXc6%R+ud9SO+P{_ z{lTXG$VH(JbWR`0WG)LG)TYS(BjW#EAdta!Z9~N$3x9X5x&Gh zH}CKUFjJ_*Vh2?|=W3Fy8MUPnr6)aLV6kfH4!2?&ZiPyCxcuH(3vKK2sEFD8+|U6V zlHUD2dBlnep%bY(X(HN-1gAdfM2~ePdnMd`m{%puvqK=(LqzAdD+z1Az+L{jm+o{j z^Z}aC^(}a*e1zclbVbAq9PjbvUt~Xhqj*-aj{UO+CpEGGKn*(-A$mcLVsk4jZ|S?D zB%LwXhqR8&0ZNTJs*5J-?R2_e2bupXLEbkuhHNKJM(yP{Ps<0j+F>_R1T%l@770~N z>^D{Cg{N`jZu1GDYt<_;hN0{NV2WQa7LKgV%h9>s&aBp&1~ouS=B7zi+G+~`mj&kwsdyRaOu2TdP#T-6_us+3mcl?ywGSDztim6h z$;(U#0n{e(!j2&YS`pld<0xYIalh9g0I4>ZFtihvV{*zgq&|{Tt@;l6FfIaq;C_K> zF-sA--KZL&S5vGqY6Y0hGAdR($`{oh4UK;MZsF-ExxA~(61B}NYAn- z{@(?-fn0!3f2aJvFThj8Amhq8DzNbTtszrM6b7Zvg+-?tO^R$?nM#X++|8~p?=cZ} z7R#P9K!ja)7oln7{T`-vX9JMSRh9OZsGc}NOlU$EPeSwhl(UQXBf}-uCY9B6NS*TJ zDIldH<5X$g&z)`q1-!ruAhn&HAED-FYdwnLmYLCb4@m&{^N_{MA&Aj z;oLCU3qaPefaV7z^XoZ0uU6Y^;zhW2Q2WdtJDbla^@9fc=Tui34jvtA(bMG(q~F85{J zO~3q^LElOy&|D^X4^+2uZEL~pI^Zh6^c zm3CKjk1UTIxP4z=ncRQeC3(y8uyCGsKHv0cXz6oxX})G@(?gBBudao07`L69cfFjJ zgMkrhv%s&N?SuvuD-0#cZASW!{2?&qC@E%px=&+jP&KIvd(NgGr3=E;0Jpc$EvK} zu3ZSG!E0Sm^G=)S8qHI`F>@U;_;3K5Uuxv*EINvKr_t<-55?BMqj`Oq==2*sj$*X1 z<^Q(z{k`Du9PT=PEiq~AU;7;UDW`Cn`;#uOri`DEB=9r`q0tANtV;G*H}rH_RI>Gt zVvp@X&5-qVaq?hyEgGOkCAcT}BPGT`3r*+7$!63}7?uSDy88M|$fAtfBF#EAl7=G2 zn89}>awtc3rYR%>9Gt0CDHXiac#>&MN6RfKheBRU&>ekLLiH^yexVl5){7310>*F@ z0I8RW<~rlq%aw)m~1E7#iA;m=;j0{r%O>;@kPA zeN1jtF+~BJn2xK3g*ZnnfMhU5dvCdH?^vhUKs-mQ+~QKr)I%Cjl9SX$$qiXLho}u% zI42B-oMzqt<-LcRG+rH<{b)O+_F#mXxpV;jJa;47;>}&ygRL|X9thJ9aULA?TOgN$E{V3S7xVjw;YHk_uPtC|yC7wDLbtQ*tyJC1fV%kGc zPgh=Ee$jqc;leEbJ`Pj4FIVV4fAvJN(*$-#9z)|-D`=*MK@?C8rOD^jNPmmT3hTIQ zdhb4K<>`O}o)RWi(6o~Q!6Fn#5j)F6G@Tmluh_)5m^l`|*wLVDxGXp(PDolCfpRiD zBEqmgF(MW`Q)O7E6z91V-`D4yn~KSImf-hCkKo$H3wgamINLuIHEW!5BCqR=^48N( zfnk}w8?@kQYI5G-&DG37R{3sOMcM_cnJK+Qi1hh~=@?>fetj-?(DY+@g<5d;8%%t{&cNbf-^Z8gCt6*YpV$(X(4W zK5J|287$x!cmne(Gf}_gNhvS9L!j1d0;HrV)O|za4gGYtu)C!qSl!Nol9*J0L4?rP ze&1vjK`v_S4ZnQwD=|ki^~&wJ*S@rgEL6>V~`2{ftov;r6J;BvVCi$*wB9`Wl{;L`FcD1_-=suY=u~0u=9A|nc_7f z*}daKC0HqME6pxc<13tt_vT&PkJyV1y>5;=?Y&sFb6p9ntm=hM?=bu=Aei1vhf~5>8uPo(F`5HmuLkp zaN3;0;C=Dcm{XiF`D>ZIZ;C>ChAR8AkDbW>^GtoRF;DVJFkbp=WBJSTn4a~=ro=f! zqhqjqPORT1&#Xo7v}U_F(-kR$2_{n!2Vrrn8|BRcI_??b3(&#}Ry zl)F-$G+FOe4?99{yd|y0@)2%aua}ibHq7!GZ>-UExInBWNq-#!lQwN@b-Y#`c)rY6 zRdXZBtD;uMcvE+WM`%x-rF{wZBbsf|H`X4~o7xL>wzo$h&FMbka?G}iN6u0;@xrKl zc_lx?#d`mmA5NErEw{dgS5cS3N#4xww0-<*_{f>uVruBUFfQ* z>jPrmh7UBDF=@9f`+fO=8JUt!Clx(ONML6*m>;{d)ddEhGYb)sfsg{v1>;~d@r`JH ztZ?*Ao4~Q)OGL}_%K;0sT8!}M`FAj^V%R7C^{rc~yT>5+7ODdt;FHLxhn7SM4-hF$N06s~wu^f%rBI6AYHaW(#IX=o}GLu<4%j57`c+_Crx0Xc- zoI87U&dQx%MosONGg>}h+Lm$Cy#}1cjJ6kg#m_4Hs;|A>k~e&&R-h&A;!D*Plgf7bFVef8y_~z)_7BA$Y9bYun%eb!KCTI$XmBk zPI9K?SF)$=tk9Vu8}YnUQ`zWUm~pfNDy3|4q-%S@cD$xsCI8{4SXFibYvgGJ?_vbb zLnR>?ky*(Bip2JXH`OsrV-hJj%2_ftHa7hv$fb9FAa>?{5-qO`-cQp1@XUKEGUe}R9XYbTFM2|*2 zOmu_2gRBpULGBT<8!+CZ62b?<<+rDRM(nX6ChX_!*N%) z#vlt9P}yZpz^^3&mUI-lBkDfF$r5E0B;*S%Y@xj-Pr5I^tbtda`R#KL=gDw3qBA%J zqWs%9=pL04Bk5i_G!!+(d$C|9#S3Z~Yb`wbsccx79O^)=D&d_*tQ-~# zZ65=tX%|_YIopUd*0WH)r7OM3o)M9_7@=%hWiR2g#o1~2jCm%>JF7bjf|(=YS;XJ$T4|I!n4qhK^+^5NoN4~9s(0X)HlMtC7AcwW3qMT9FdFFLy2z zLS)^d8PXQi1Norl8-1S*iSucp%`c@Dx;-I>ilMH@WP(rAa~w%`fq_CDZ2+t8&E9k- zRhjQf2%%E?iqj08T`_V#r^LL|Hu{+uXTT-J=*Rk(FV;o4Cozu=M=w@l(N#D0Ix)5Q z3oqdkO~tr7-g!YJ#fWi4Hw|V-e5rR42&+F&y<1Ivr{dsvFX=c8V2Z=%4%FLr(*fAg z%A`|fL9aSa^ImV*FPuqNWMybzXHg8J`u=U46~^hrIpnL(ehy+WEu1t%99VqYL-5yYE7GmI0D0CNAPFqA87rY4|(f zSzxrc`BlQv39uI`Iv!IMP1@M)qXB3-QU~FUAQcI3F;pa`(?u<$2UOFh$vFS125;aK zO)Y#KF}C{s#`IaH0w|G8qeEQH;nYJ-mKrxLm2(;XV=Imi-1n_l*l0DC^thJULQE5! zU+vgKPbt$FNPgUreM1u0?wTDsjyw9gjs+LC%_-hzE_ZA(9s~Yl+#1d(jqj4Lk`ul_ z5S=+0!3hTeb#cH1wX(|Sn+LuN&n}|-Yn&~$_vDt#;}mR(2echh86mCUGVX3cr4|W{ zaF(~E?N~kysxFV_OppLJnn195`6V^-fSPXi;Ga}dO$M$$hedz(<(e!XJcQFRU|n}) z0wMv)$ZBAwh7NPdd#AB!aa^9NLY_~AM8V4w&DmnaoQP+;n)k5e1rIW9Z1z`tt7T|0 z!uB@3tOCu_OlM0&&pzKFD0UkLME7NVamUZvb*!oC>L%`dehbvqLB106V}r*=f!8uI ziudy)S=Fy+=E6^*@5tI<4^53)Pl+_5@J<(PlMUp1E$KBk=t|T}f)U#LH2&ai#By!d z4$ItY-^GbGkEV#8V&x5r+IVIgxibcEGkzjxG+Nf+*aaq<6BNN z_vnHkv$Di}$3nB(A>;61@>qU>iF&*W@w(_W?PAaYsElc(kMD&|?sc1)*zuxbVE)L} zxUMBSk)pk=Fli$MRU2LOftJo;KmF;B+mrWGtku=km7hi(QJC-94F4;r(zsrqCwQEA z^T88ET@>eHO_^Z#UuQ&voKcsHT;g!I{-b^}$d&(mL@igK{^7K!p@JnbzuH~0i^vnw z)YMddET|XP!(L(5dk5JM&Lrpur&%UMrCnh~To=_DkY3q{5bg)%g55v30_KrnHB!Q^ zZjJh}1U7sD)og6pK0F}6XU`S2#fyhwmZEMJ$H)lTt9&@K;w7fnFvDtvl?k)Rbd#sI z%EO{6;lyqJIMKJ%S1KRvf^jv)mPQE@W`H~}+kctPQK07P3P4z)S`&F&6wXuT;UcJ^ z&iGrU_iTONPdUPwSdSo{X&mp3q~z6U|QUG7EW>uzp?cFn?6Dy>G&N zJ+N@q$f1N(Od~$xCr!F#Ic&o0O0q0>U%0>6mtiz6KJ0#w6#V(a(6WGs81>Y`I}L|#R1T`&=TPIl z`_n3acA3a9zqbuq;*ce0PG#`zGlJ=$VjSE?mM}xA&-W*t^CRZS-3)|$_v~8#RCB+d z+sgoWS0>Z>oF>0d{50kICw!m${Cuyy$L|tTe9_4-L2MmLrEn30|Auk-n0uRpVkKL8 z&zt+Jf$z2l$G#iLSv-VUTszpQkqzO*O0AN@?aVMoXQsxP%PFy(=?0LSKCY$~%I1r5PV=Z9y^)at{i!H@+IKXq!;HwH8AfhmJqD4-LQg z{AI064eYP)IS3ttH)7ipYP{pTe*z;kpfr>5 zWS^=&#)g?mH1F+e?*aTyD<5c;BJ7TZs4 zxOQcx?@YVTR|92eM(YOpgkoKW24NDbV(Qbnvtq5#24nPqSV)|(flDrjXzJXAIziuO zB*J&IxtsZ(SDJU~E3bxKu?&pm4(hoRhm(DAmQ^Z&2*lIjyX$2)@@} zFYfsQh+ugthUEWh@rnBjO@6v{20@p>NAgiV+8e^q!$%>bEy%|i1Vxoqf=dE)3}2Ls z4Es)q5;*nKYiG$56zSE!Gdpp=%m`KG`txv4so0>=Y#?)do?B{!E2C}hC3xht0DhF_ z5{G0J>CR--1)xi5`|VF{?VayvZ4Fop^go^EaH%Iun>J#>w7vwd-hU4i?FJ3ArjD=?rv~vGa3tZNO_?B_}DiIrD9_Jg?kE zqrysa?5i9+R1Nxan{mFy{+x)b3F!vjh7aetrxXb@(dDDa_7k(!&XWyY1yA;NAnwsG zL)?Q}#$tJX3Iv$VMm0L^j_Xq*z0O_{kt~+OOdfK(w*&4^%6UgIf9GUqfTiF64tjkW z-Y1&55^vZ1!DT{mp#m3HQyUZzOtMwV4xG>!f^^=ToPEU;qT?#nSt<@Oi17w_FB1p( zt%h&jg1*HnDu|I2BNW>LabsM-UCF#khe`Xf88oo<8d-ifbZS%p3bf(tf4+yXGj0L# zAh6)$x2+4nLlIz5 z_2%Bed;K>$Ha6dkD*4D2km8|+o3%g4n;&jX&5TR|I6TpxJ0sH^N+<41G9%}Xo`J!Z z-b#}tChd%B=55Rk+&AxNlJu@+;M#GiqkS8kR)JEhGv_CkPvg6je1l-Yo*(&D-&&1u zq+tzi-c1+vHvF-bX1V2lC84W!V*8wB4UiLzfmu}%2=Bka*mQMJ{$|d3J|6vs6EkkT z!aN?R$o%x~WI;-%fD@9WjHa6mh8{z)iIVw^P`2IQP640df1Lo~X(-!#q>WDhsS7H- znRQPa85?u2?>#Q^x9HE;RCbEM`PBq*B?UqEBMnBao%o0L)Kxfcj0PWrT~>sS_Y}fV zdSXit-)5awi04&=F{*))Qkn!oELYCGj#9;H4bB3sav8)c{r+t_ELzc1yYRcTvlhL* zwEyIOl4`9vu*67KP40|qYyl;NO8p2%cP+qNHIEn4;Q(x?Zcy%Zra9OMC_&GJUJ0E) zfRS%H0Im{{q<@{z{xuIEf%=US;`kCcIyzETkLPIoD!*4_Q#?<`lS1cw>#oI}hGk6Z z*=+y(U~n}VZMbk3Fs51~kesE8gf>E%XxN)^2D1weX*oW_95TRbnSjS&N7YxLjoQMz zlx+hTn6HqjYak~f1+_n?vfvCSS;UHMo7a%Y`KO$DsaV2LoVWZ*ZJ&3v+U#CbB?e96hNF2 zIM2`bhq9dt+PD_wENUn`VP+jm5x_qQBWXsM?(yU{LsMQOm|02lI@5{JuH!=1d8y$d zJ)x0m$E?Mkg2zn$y?$zt6MUXQ+CVx30D`Ex4WZ|lg(4n0aHpoDD2+q=S5-#GnyxWKjX<;kIDdcYI>LDdU|M zs!o`WXjbxtRsl3KY|xZ4f+>Ag4BQ zV1<#%EFevMA9bJabSzVQaP&dtkRcvK(If`}T$bFmNkwZiV|8s0P*P$oP z?It=B{S5+%oql+!K`{kzh8LF{tgeiLRyO?(c8CW<#uKP|=1TQZ`Zhq`asb9j9BN9q z3+jzlXq1C3WsJNHFJWbIH59?Ws^IBFMjr&XGUFr=ieipNaq7lf-Y`X5`CiggIn>!d zsDc&R1!Zq$j9>v~c8^NJzmGVT#{C>ir?E}VfWllF=p1RrtlN9=b);ctWgTUIsnnD0 zGZ@J?(PWeCIWK+s!I2XRl&PSfXUJc9;+@0{sCB=DStX66CzHO2HVw2UBw43Kf&EZ# zS11&=zkTK?*(QcgsMt{)?x8E(nvkKoGBtrWGPY#$a4-|wgQ$qeg%iyHRRtBgGG7|0 z+mk;qLN(Y!X4?dxlye00P*5uTaL*`Iu((%f-dElXPdvtixq)4*tpG+LALRARaR|UL zK~g62%7fc4c~lLUN|tVse*j?a#Dv0+?_!`O1HyTQl-8f1KpJ{X4qd zBw6pk$EcsIml3;;N9PQWRLs{yD(50RqrKY;IZf`Qe$svP7st7j0q}}g3H@3PRl6~i zqfI-FG*i6BZDg?klG;BCz;ihOCTu!%(M?Bv!7f;|2*&W3HEEo*(%SiLOb4_s9MG+% zEy<$Tw^v4#)e8D-S52-``MgUJcm&j!ufRm(Sjt~yOYy7N%T>&Z=3xJfIm#1Q4s%~itX-r z@8~d-buL2;>5#GgU^!ZrqiI514cg{e2*W&GYhq2n(bYl5E;G6Dr|tT8?&(l+OC##| z;PV2b43?kWzzyNr{`n8`B{Wmn2ye?cjR{$v;M5YUXw1VgQf;hAxCSJ)7xt?zOLI8Q zm}(wLKcHhifuS`|zx^_KJy@;o6SttPnY6?;D;dB=kkf?QbpQ_ zxo-ai<@c}*igCY$Kg=8^%BgyKE=q3!VAoE6myVKicUZS?uJ#%k1{w8iMzk)+>Z_=3 z8}GD)2KHJSB`f5(paDZHvZ>TfH*entS;{8ZOq?6ap{d3>b51{gaxvKDi6;_dplsn@ zdNK!v#BT7WMJ9;Ts7`&NR=0S$0c^3#Ck1k(7@?U;0o{mmxwc7Hyk_=;q;3>HGxL1c zr`^m2iXf!*X;yj7xc}sDl+wEz3hFsE`B}g-W$obyOhtW;WfG{8_pKp`MJi1|AhPWv z8#sp1@$s~2 ze1IKWXm=ArS(|S&F;(?=&tVaa$9ickGEF*PkMKS2s+u<7uo32z`_&jb zcSGU*YKLcaF4yD6J82P)m@{XZv^y|PzXGFn5y(AK@PDw13Si!XA1nj*$5rIQ zW1&#L!&whx;Z#)fvkXa0AMX+J4BXWMsdQmU8++Mxh925~#Fn$CAyJ)34PiNL$Mq4x z3(bV1%>(C~_|QBbw#MOlYyyxP{Tf5g*$1 z3gGbUY$UGlAcM6BlCxS3hysjQ7{i%0wr|scl|lcHPx>oKohPw#-RT#-cLw`UmnwD2 z2YOMS_xg4+W_rv0Imom?!T+17*ahe)l1%t9LV>bePX5DmBB(@DkBQMyXCVW>-Dp2(-UFv;N1b%UrnI1i@2uT_e4$FJ0n$6=QBL9}^3aV0i8 z9c8@~66XTYq)Qn|oceK{Zmo-TiUiE#$KY;1iijV!q5SotPR^kJpU1NgNP^XP3(SBd ztD#EoU~#2~R&Y5TWbzdj4v?X-4!L@|xnTC315;pgm$ALg&CUEco=Tf>i`xA06U~6= zNyGRzWb_sY>b~Y=C33Va8!_pW4whR+^k>S^^wMjhVe01-8HW)%Dg*D=7<#cxWkYOq z&5=AqJtqkHDh{`jj=8>RdrH9jifB}!*BNp(q`v0&-{80Y9C=@{QqO$;gokWlNR`Xy z1t0&IOrefqReuGmFfb!8L7(_q;b{Wp5OoW`&1c@NwwaDwpv3O2E{mHcusfWMP)NTU zeZ8r3=^52H?~K|$bQXqZlYyO(ZD0wy*8csNBf_Sr=4IW$y0-iF)kOX;Sp~Wv$lwUIb&{!r+3!NqQT;C#$BL%3eclC*S5b z2llN}h738sxYhi?#~V8bsR>inD>y~`8TpEOy3Cwh3pn?`oUjc zB3l?TPCG=DrD!{Hfs6{393}k>Z|{WYrBR)_MmuOoul8(xaurdK{RyQ`2o2J}iTnC} zmWT7SntFMx=cZ~syM6=Gl-2P`za4{13llSA&@vm12 zgAJc`_(1XPKNZIQYn1JE5D;;<(X_vvglaBzE8=-N>UD+XkbH&hq|N8gV3Ifsx)JWK zA}aNKjf-`Ub}LkyE}|uof#jGf(Ue)-wTsx>O<3c!REh8e+Ju1Z^N6Gm z7xt|E2^+#Ee1gf4f)5RYi`ZR=KN&!bHHj5x2=ng+htE6Ek(z$4JOsfa@2C#*U~(pi z%tamx&Q}7fu2d4CCeM8Q^5H$CEV$AdHZ_Y(GvA#G^lYVNjY>$LX0PdQ>jayZD7(V! zxD2)wmo44dTgmqy%vKbIm_Vk?zd`1}-@yK=d_(N5IvNWFMg`=gbQq!R#Um`A_6ymr zb6oxxX${7I9GG22@tGYx3#b2FyqB@aIO#5N4l$p*fJX%+@mSn&gEScDKfbd)*NcEt zRo*!D<~Jka%%T72gXL*4anB)6qe6!X@M?k9F}>ud=sVERiVT5Ff;2E`DR1arLvS7Y zIO?fvpU?oQ{SWt1jxmYDR=aKk=x3DI@0(d(ajz+#h+J@?1}~eAB%#SFs_}-DZUyMF zL)y(GJbtd#EMM35iPVzE65yUf7m#%?{s1tDn{{I#uaZctYD3?Uo+z72uQO8OAIE`l zG8M=OG~+Efv=9;Qj^h68(*mZbcrQ0S3Jsyv2%;`K_ZKnm8JPH5O}6j?)Kv`|&0hAgy6BRlig2li{joV<^rv6Pc>~MM5>`>N1ya%)H*sd!2<=Q3{Be8#d zVj^Ms45TT{Sxn~q>lqzlLWOq&+xNXm3O7Q?c7Oq5quwpUtFP!Oi8=wB$#)(VZGI*Q?Q6NDc0;D!EV&E8wjJv~;xFMJK2-1xWo%IN zJq5;l7Prp|uz9Ww7b~TT;m&Xrn;>>E!5|&4`T?$XObBh${wLH+GC(CfKXdrH7~(@J zvovCJQPDQidi!_zpKvKvWEU>pzG++4+r*Y4=B+U~tvLXaDzvXzc#iXelXye+USJ!v zgB?zV6#ZkoACHeV_8vPrIyig|igb1sQ~85Z{`&fh3=R7>(X*c$}eyb zHx{&;@;-C!rto%V{X?Z|7@_UaH=@CpfC`m|;g7Hbx-M8s2r#T74T#|FVn8b@u#DkY zV-mz*Vn-sIFA!`>$Je(7#)%huKmG(c#el{Ig*;k}Q2PxkaubiC=K->RPVG-DPew?k+jmk4{g}d91gBj$3}W+v+$bYc z(=irOwk?Pzj3|^kLclZAO!NE+4#I)_!!{tkR`Xh6AaHsFqupsiZXJ}r+0-IfTj^)* zJ$Ez2wd)UDvAJytS*<*_?VGia3l1XPKuD7zL7!@TWSUcM%&jtvUrP}Qg-l+qe9c04 zyo24M|B4#`eQ9$9opxZG7=R=G6R}KGy)IOWb8guhTDlPDtzp;=Gn-%96__(F63SdZA{`r@8e~m6)&|pd%w=z`ToF?h(sG z-Ml{{&$9BQ@B6)!0W~Pl_;Mm6A0vl2KLuQ_2t?@cw&B99omW669S!KFJTBtjFWHBk zcBFN;dIHi>;|Z`~++KADs&53%wjN6HQK_+K8q{6^(ln_Tpt#((`H4Huw7Vfv={hUR8OD@G`_SeC9!gtACz&G|G8R`-|D zq|!r1+Y}WK3B}`nM-rFlZKCTsGq-_GVjZoDFAyV5SXSzoXJSv#NHUeZV}7glR5I8J zwBqqP^3ege(ZRvN2vEw3+(dT6XRB`X4aMHPiWs3uk2eD#$~6vto--XRLJ{B;!rXs# z6G$DUJ9T%GShFTbiKomSQCq~^3eL1k2CLXo&EX)YML_B;d(DGh`X*UNwx8E+Y1A9QOX*6$RVgO7cWezw=xK2)Wo!gY=;&Sf_><>HzrfL! zD}XvD49LWz)Ly{I%pVhg|5#g9JtdB>NV_2vVw(V!j(zi=(AUUSxvKqjlksQOszUO;rcF7WnJBgRzx4~B1x0R@vj*M{S#!a_2?S3bd`hL zw{*rvDs7Y8QiL3M%zA_Y5T{3?hmH%R!xHFmSQ}b$rcw{(dF{_7OkBzFH*XAg(#eK2 ziKrN)p&U$AOF=Qc17qmH(fj$9AH}&zEWKyIxzDsfbxh7ZC_|96IKSJkGQ%f?dOgRo z&}C?AheulHR7o5(d_;FuywfUpL9<--B;MKMErf3+pb!6=IK{ghd1g&MLZJVxc}hNo z!Eb3K>_Ql!p*JZsahGeq?JM#Bb>zR}vke~1W-_lk|5Y1FldK;c#fi<9^~}M$WdWBK z_i%Ty(mm)ZMjugdJ(t2FH!CX@3dDhm|Kcq&qS*AGTRgQ(MrU#1rcci=&xE`yp26Y6>e`lUBF3v2t?7e ze8Eqzx0mk-A=CxSZ$Wp#aw#YuI28LeJ^C{mJjLcmH4pA$8AHCiU|ESDh4w77oz$(c zeIYO*TaRP&c0*!$d8!PJcNC$ST>zttc7#D*RQLULzHZ%U*w1Zy%i zw2&o#^s-XQYBN-l&Yw!v5H|N+7)Of{GO%8sy4Hu4b}9@|gdQ!{7tWnZSi_;PFlnqC z=rhx{kTOqgL%;ynGP$E)JnG=azcgRC8LS`_x676fLoV){=F`w~@h*Z_S2*C!BTI?z8U{(*x!*Vd3ra0{944w( z_x-=?q0W99wViopWzsMApr*HVmTTuBCN1|B6)g4D#1=Y1S^|q&@G46Oq)Kf{r`q9b zKH<1AbiBOLT;_h0y`FugLuAnE9Ufk5{jupJyANKzLZk*k*$`Y*W07Y;1BM;3~HEgAsf z!GLbo9f#%pBAiy>Z8fRU)i9}m0E0F8-q4fnrdI#+?k!aa&LRyK@4;KlEQV0a7JU3J zt8^PCPS1wlx;aLe1a<;WL01hM0mbJ7;0oOUN}DEg!O82Qy|Bs}a6HE;-w^ACjEd;&Z=-|UShr1hC^i4&ow%&=YV zQEHru665048TY(dU{o2AJ){DLXsD*b0{k4#xo3ndgt(q_6P-pNxYRlB*5tT%AcHYA zvQ|opb91^p0872T$dKN6yM8Af9iwC3UZylIo5j3a30!>(VxZrVFy8k4$xruA{*_>&d??@@pHYUe8w&NZFPV2#*%(eZj)PZMF9>`5MhN zgjq0nLtN7~{@jtj{0uZ1C|#it<_5*)VhHkX0K$*chNn3%Ykh}oS>XH4KpN;g*)V-l z)E;z|fVR8*Z1!Eh^ix-tRx5xcVL*~D=bctt+O%I-`>y|ARX!9J-f_ryKj-fFpCUsz z9AVB~hu(lyG;WC07QKk=7`|}d_SK2z;paO%Y~yw>S*{9IZ2%e_3Bm9Qn{HbZ#ict;8oYM}9Dguzg`@+X%GBy`gZD_vY161K4^B>!PW5(C^5mJY2if(UdK z309cpJ$g94?`r^exR9=EyZE(dS+7-@0uD zkKQ$5s=KJ<+q?a&MFXI9YzLrP^$&var*$2TM!AJ*K6*jwmd<3tZMPDY0VijAT zkXO)pGLcQrYfgrAzv%n(TmR`*7?fd%WweCm`EzNeQh}&_tSQFwl*j(~Xty0e%@u<@ z6*@WiduDgUy#!UC>~VQZ`?7IYu?rE<^4@#JrcH4$CB^`!^!DxT#leCx;>wW50pd76 z%l_<(X%Zpfc;r71RwYm;Vi}-QE2pzQK&B4n9OiE4L7r0$V}2d;V{mu~gS3F~?@cesN~uS3al4^dO}(!qGL@FHcia-N?s^`Lc?vI`~X5Qf>0I{e0#AY zSBv7cFoc0n9r?W~m;=X*g;wgrSIEce1q$ROC zHz#W$X69x*i6CZ#xEP&(LUI*(Q~4j~5IO3@OwPHI{o9MXNsiqDL%E`a`<#0dj?ihf zf>OoMCi;mu{=>%}yQ8Nsdj-v7{5JvJ0Gjhc#)#3iKc^@B91bbz#-+-qsfcdG0rad> zkz$a844a40?{~x|9%A7duKBbeA{>CSqw9a}6#3<9R@x+#yIoEsy9?!1rD5~gYFIjB z1Xb_`*9GMf;QMO67v7_JUPglU=E;J@)8Yu>U=up|CN7*x{6$V}s6N%>dOJmtt%Pmy z@kKQ4Fc6|w%~i2?{`@8ChVENla}>G%DM2`&o*dWwV(5wZ`{c0jA#sp4Q>R?|SAp4Z z4@PsM*}Ns7W8t3whW0gZL*S|@xb|Hw@J7xlz6mfa17srL4po*yk_~oo<7Fv7QttsDhWA0Kul|o4Q^nz&3E!5dF^l-U#eVxMUh{ z|13<}J{GFU9%&O`Z{OHX2s`f?-7+-|beTRprb3P4srzMs3Q_Nf5QYGoQ|WW}MRMA! zQyV+u`cH3}PBQEL;qv@})DlUiD0LL%5XJ1T1M@kDOn)dB*9`InZ#9;bmOg?2;pJC> zp!C&{3nI`LwaTHC2woHEA$1$DgHC=!)u?nEB;HK$rKXaYHzZgJX)M$V@0*v-Jy|J+ zK^6h{v*iJBC4uaFj2WxczK12CU2ee$70q=Ou3n7Ly^yRA1fD5GHPRx!G&?-?VhjW) zz?H88iR;9jFdr{fa6-dnEF1EP+mi*msJu$X6C@8=72#) zEVRSR9=)9o`^a`La*Lo7l0%C+=E4d(LvjyO5J@W~5Pi@Ws{81pF8#TmzmBdE8O{Bi z5%D%WY;s>1gy<9fGWn8@S06pH)`Z2P8DMSBUgvQQWI{Y@0<3x2IG#>t$~tj*IjE(F zFFynGs3fgQtJe(7ShQPX=$%&BLmtPqW^rC;tv60EK^7_ivrG^`dcN3p(X4B9hGib| z^3`Va8=xP%(HV!L^pGS;+_vjNro-iK@KW2K6UPJH)tED)dB|0uI_do!S;T*8?uHpS zC-=ussLuaROXqMOen%8U#}Z=lXH|Qm_n~j*cycjf3v=Iq8{%%&H1Mc3FtpPzW5dYh zB1Dt7Z(*ZF=+?qe3lv@qu=2b>C10br>L-9j6L>f8*VWv!Dpqtu5>g7H0D|fC)ga0n zE?hlpFqep?VAE;=CbsU%>SL%0xGhkutnGMb9Qn-c`T#x2pQ?bKn@`81cxQ1K);Tgk zc#*g>7SbORs-xgDNJB6#Taz;bi9GV|^=bz5N*)UCckq!u<1GydCCJzG$|WHdAKV55 z?!M_Z-vVB3XZnEbgXO{D*gAE?et;(edagw z3p>sA4b!K(A2#Qpo_lVGjksw*FH0#vbmP@FlnGKm`UnnWT2^cL!|>WiP!vnHTg1|Y zKK-|}K(oG38?BL44=t{yPY^gege_bw5oavWnXQH6YVZFa3d;&YVL8r!UWTt)h44DSRG0h7MN-T~`eYiFZLSGWd@&Zr;X9*6 zVTFCZ^1~oJEsl*I<#y+fZLqY8*x5I-BIiaEa2_nJ`|LT)g`5m~fAzvZ36Z_A?(7@< zT|sXu3A|qAUAlK_p1e;k_krywx-VfKSUWg@7%pIQFG=`ser>NEl+P#kf-Ao1b-e4f z2_NJZ#d=Phaec~1?mt`rt`i!lM=-g(K3K?@9`8wq2)17VTgF4K$+6ZPCc_18gZ5*% zkL6p0%^7TbK>HRj@r33T6E>#LcYAet?0*I1-!ntW;&fay?|}20lwVw}v_1ODr3jb? zuoj#MxChfMMfUmY&*M<8-h>HKuBQJ(^Ej`Au}K4K4ufhSzqNqdvRUWi3wUb&wht&o|Cr^?$EQc=J|Ec#}Qj7!f^7Jf#dp1a-VIY7BA@|q=H6WLC6~0 zubS7rnFyrvKt{7L^yH@-K+D<~AqZd>c}4L zK2QqO>7Y12g~NK`JG~(s)Q4{1dBtpX)^O4CVJwtYU3}tt^@7_KV2$YFP;N8FyVb#2 z@B##_o%?*CuDe5t#ot^0v_FL=XItW~K(Y3`UOiJF-+R4!ob5+qz_2Q?Uv8>!qI11t zC(zSk_(H(ML+)v&{>@jSxOg4a=#J_t-fPK z_ARHwFXAOP~DSgW4>cdBZ|K1uYUBtH0354uG2`P44zckI?S7@q>B)?)g%lkF<^* z?1M>r7i!F#EW2NLdO*Z$9+k?7%i|g@hXJ3Lr`f&2=!j6g=E4hnY63ah^$H7|!fmuZ zP%O8@(o!n(26E7{ed`F*PygqGduajjH*++&C+Q)!lbrm}%r+74liulpZFx6>=L6iM zhdZN(@P?lCn~D#aJ6rQO)u|2Lls=sv=siJl0OvvD47IG_AVE39{7DXQNGLs^iho3s z{T~kq`f=>x;^=RjQpdmXhXz~>g?b_YDTBmjo9<-;KTaBYUmWI@Y^*=6f|Mo3xc4^9 zwf$av2@$66p)WWkG5-g?K>kt#DSUKy18Oihf^X#QkBwmM;H)5VnHJ~*$R1`1w^k#j zmlv076DPX$dC;@}nf9;OzysAydeG$m`++iHCVF0c85K3qIKLHw>zA!tR|Bf&kDs31 z{_~T0XJI=j_ds(bU_d^XFb+Q0nrN%#Ydl~; zw(TH7)R4{aR#|#cxIA@9@2&p7+TJ{#%C37DuM^!!<(NtmA(TYOJe!r^x+%^6&D`fXGQiI+QjVjAz9J!oDL%@_gWkN*!#;p~W z_0ySYlTvC)wl1_8BnC@V+j>KyCKyWw?iQoR9I+8Y5gNYl>k}$MxNE>r;;jv=qsF5X zIgVsoUM_|~Szo%mg;^L(x|(-NNzfoo5h2a-Rj&jQX@Lwd9aam)3#406vZBDbE0~D1 z^MJH6Kw8DWkaiLwO{pk&B?W2pO7puQN$I{OBH|_=Tt9gQ`u=L(5qzk$?4biE(!Uzj zxTs`qz~juKdY0D&(1;-iB!*h=rF0Qra%ND1k_l8wow9BY=Aza>8@5GMQ?)jP@UDIO zWRkL_HP=pfdZ&vn9~sUkY3E`Weh~bshC`^xHbH(mgv@oiObCJCdEf8u?sj0*>~GFV zI|xmNTKm3JN!PH;4Ai-#`3r);w9);8_rAl#*!Eb?^T}5!yFnid$HIxQdLb>f2cZ@( zfj{v)u=_Mj6Y?SWoJH^%$WH z!W2bQR8nQZV_v4WGU_SKH-+hrw2!}#mSiA)lN$p)1L2HpLTjU|s;eVFQ&r0O7jvEU zb%xHdiuX1Tt)Q$qWc2DI4?MGRdo5v>S!t98#wC~=P)%TARi~PwN%mmpnJh`|a}&@e zWZrxG@Lm0#U|7F6L*1HXe83#@WADkKU0th|pXYHob0(=)j(K_TpRL0T5^>1jztY3< z5DOhnA2EXl(Cj9>eoPR6X~cw|KJ`b zcTZ34`wgv#6=eRTpb1nzBxRMqT|Q3sa{`JmT-Xi|C?cLmlD$anwyQ{5Y43vrX`w~W z{=^(_hnGWSwY%9mj9{Tu0QW2G1upy?x9iv)Q5vKKY$~7L1{2)iM_ekS4}8`IE>l{& z8r3&nhbaiJ;QBu^6QK{mkTQuzy;%iQ5L^*QcZdWdT*JBa+(iMF0{f3dkOFOS2Q(;k zq7(G|$4>NC+pWyW$q|Ms5AR)<(TqV`xCp>^$6hMYZMR6kjT=z1uXD3)#;3GeUS9sU zepKq2z4;3ydlen5DRtsHmjDsy(sJO|JMd1nU36mmsq5cAWQd~vh$){Ths)kzxLUrB z*>1(d`UtbF=k*PNO?#sxw(YLygp=FNNEMId4U~IhhBCf+)Lp?+hhN!>gTy~uQrBa= zUL_p5EA~J`V1u6zGqt3JAe78~E0CGXMJ>-v!eciybw%2A|5Bnk9^qR76T!)pmNlu`Oe?d#=wW-srQ zw$43NSysF)IbZhs=Jpf!-1%5M8u~4)sUn#u#O9DiGQVCDLtMlgGdn$g6+DwxQ9=H+ z1h?2vkhpe%>c0whGY=?X`2}ojInwzO4Qpr}w8KNAs}^)NTWr~PvkE@+(~`=2WYl)( zAVndc70g6oW)XvA2=oiXh$SygkLA8k-GQh0DK-IvhoKpK4~qTY2~@a)-}wrBu9m)L zJgxrv5e(7U#2b5;6YIXsMBuK^?x{Gg}Um^^Kf@)T)+povX@M#J)z~GLcc;++gT0QlfR@ zfQ%l8p%i@hjhFzrS!OBS751uEucnLYdc4JzcBAg8V5v8_RTKb!#FxXw!%Z-dUfJ9P zI%uza`t)gj^iu#A%R(SM43W2`E4GaIRgqmCkhEz?(Oiv~z@Y6elygZsv`BA0oN4lE zaz}zf)wH8REvK(K^(YhfhAS@x;Wbe!Lao8|Kz2gey?+RStG>a7r;%`xs<*ihGZlLb z9Q8WtXak|gYZ&Ns$DU2CrOx0&U$M*s(Q-q1K6nG{ zy2=#~VcnX@P5T6eg*DxVKHZR5wbBNz%(yA-&~gyQ@Z7zS8+Uo-qtdv`Id5VX;Y@Nr zn4a44XDg`2%>~rKJ3ju)C+xzlsI5m0U~-cP`0CH8vT=_w9%|yM;3k@Gux2h+z{sGP z?r+~zV)K@fjbaLViM3QeMMxrv5Bdd#d40zsI0gzpFdyEd3nM{n0dXeK7Vq-?URq(p z@@t%&_2Zu*IDC0|cAyYg83q&-QclLdtPF;asIWyds7eNVZ>>?d@p^~tkzC}b4fK-< zjV1n^5m&*_EoC2&JVwmwvbr6Rr&`!C;_pM zJyy07Mhj2ik8Gg-@x)^l#5fX5RzT)zKS)pfgjARjJ`@bsTotG}KP>Os8$Q16F~~rd zfk%~SL{7%M3S42P)_yHAQWzNf%gI3osi(_giz}~QU&I(DERgtFqkoT;410sFnf`?h zn$ty(xnpcE!S%AP+E7;|%WohOI|6qnT!(s%8-}|no&b*TBpsSshbDu0_`5|zseIH( zSLJ-HB+nfI3VRUEXgr@93i7m6-*JU?rIA9HG&o%EG{0vjh`J zlQ`iL0)zzW$UP_NhVthC*M{v+2CRSl7=2T}2!c6LT!4Tc(Sk<1;Cr@WDHctmsDWUU zZ4lugqpt|ENujynt31`|BQ%<&SQvuGrhZVf5JV=nd zd@OQ`-HLku&|m;B2X<))TX-j#Rl*MfMg)FB%Jp4ZWChKmOX*mVK}-?Rpau+%1+2kr zl!{77_$;)RS(h>CF<7s!+UF+n4<_xKrWkQtTYo@kwI*t_=I^xvDoI%my1H-9aA`lU zU-)&t-QbPZhF`XdApzn6>Do(od!ZlyAiv}yFm4xt1nyHgKgLTIWAQ8<7jblOI!t$~ zl`_x2gB*v5d%I420Nd&fXUQ)INMTU)27>#V_tUNTSU&b@i@u{0jrh>*Fd#vn)ZsC9 z5=h!;IMjExa=*a4YL;Y~Ltc?(k2~;1qwn%QRWdf@HkPlz^Rv(T&8w{6NzK1Sb$4Hgkooz zHMA{hDaW$M8iC+B(MzclmD>a$m+ z9(BJ+I+wG&$o&muD@z%_63R{sd z5m1R@MjtOot|uJnr(d^`I5(Zn+fIrr?2PnM&!Iqk83RKDBjJ)W z<%W7iNZc{ruz5ynUE(`|F5SNLZYaGEwVQ&gl+GzgcRjas1RMvJzVG1Mu?Bi;CFBl$zhK;_rDVp)Q2i8+wC2W&` z04X@X`#dV1^BX9#FxV50FO#RiE&es?nUm^C*iVXhhwp{;Qyc8-*HHFI@jwkU!2*AG zV}%!$#J5rG+)x&}$hQ>dP>zM|_T>tc!C-xxA41036GnQ4_4-Jx9Y~CeQ&dpaRL@$d zyO!WzAgasR8Igqn{u$y4HR{JX)R&$d*$31G?5+RfHiumv+OF# zzA)Gm$?A{@NhTT`m-~WX{4ASFPySVe#mqN4?O*k0bhn@YZLJRBPl|#;jU5m%1c*pt zoc~JF6uz>jC~yW?P}BtP{-nB|B3WT!>suBp7Yx@%tQWo!ux?eQCZ$bU-T+Ao_87WC zDmp2L^a>hu*U6?fNns@XK~69qVmn8FwOp8d1kp|xh#;rAH1%^gr2r5=Yqy^~~@?$WAortZEx<@wVS~h? zf*meCu(g+KOYSCpgzqxoNC=yhO2fZMJV_zFW&f;q8%5$nauQUAEOaNyKASlN<)D3# z8i6pK!uvvPige>!Ne69?B50~E#4v{lZPPxb+|Y=}nf6qom!I+_BL2f0(6JmFtCuHH zrvai#pC2bEC@iky#Tg(rWl{{GiitK>AkOz;7^}95_dRqU3ZlFGKA3S)Vh@69W^kL~ zPbk6P@YYab?Qd&{64zgFJx6P>;hBLOWtR!v*huWcklNKz&N+CLv3=uskw9fx3mT z$Yz*9a2-0KH@rQNROfC1_I@WjFXl6?Kw88AxV;R$(?igRVhPc<^kD@>fhLW+Lt@r( z$ueX#Qy`;@6a~w|W!9Er;IRCN2WQ`{&P-w|>jxg)QEV74K#kjq?8TkumUU|Ld65RA z4TzP9RaAR}w7CLN&|uV(@_=YS(OCz#^Vm~X?P+$O4pIX)B>54}ywvZlPN0GVnL$D4 zGheB{xyL{H)U0mv97xp5{0%C_=h5h4D3aN7az>|;B7!|JXmkivPO5(|&jI^p_fId4 z8VZJq!U1Q6bji`tS{o8d$s=ci($6o^JnVUosv1ey;Nw7t<>Jxg)$oz>`FwYAGq*!$ z&-KpFurr3yv4kgoS$Sn`MD$OVgABI|J6r+Y0MS6gHe}$vhv?5-nf^Ag(2&L!i-h-BTq(ze3=`vwz&(t(j<77Wp)EgQw`Z-h1JMpIIYa#Hw7Cd;IN#yjvozf?a#7*FDF%)d9zp2f!jCu|G`6njdI|nOU8k*g4ND|i|C2x3NQ97{H!FCT)UWsVa8?R zJCF6se>u)<6<%9C%6u$E23mfpxS(Sg+l#|nWPPnA!_Tf&z7oI6D&=@q5+~N$N$w@5 zu|1Y|=3}Kf+&;$>|GA#@DsDeL6i8rAe|N6wg5pd^j?TUijjs4x72CEg{P;n2Sox5o zLjv(FGZ)Mocj2BLwQY&kF+#~Tq)n5dNuI(mtq)2Pz>PL34|_j!LS zaL23Wd^aL^KU{!2j_=-(hky)!&uby|#vXCZm0SR6@hT?DlGn4z!w35&ns#*TCxg_%sK0yZ`gPwy6r) z3Qg7Nx7*ZDz=Ugb3X-ESrqd14ojt1ao)1{A?^qX-;>8t5<}ETZ%zR+yZvT+}3s55L zT|UaJd_s2fLS!+x9ss70&8<}@LCJmym}1*WtGBn*&&1Y#%O=F+e~G%Z-er%^kMq0( z27;+%ruN$SBR9)eEE6t9G&~@l_5Rc-jA1CwUgLb@>K~qmg@iaymnuGr*g%5%HNwI5 zV|7wv!EG-{yCpweFSe9E^mC-lY7%hzmP=*n3S$zUK7A@Hs1;dK?ffRNv2=FODQvQI zvgBBB#X&Nz^65&)Gn`(o_WN<(urWisaqMnHL_}*LN028)SP&mHI|VDJefzeBh`#z^ z1V`*nfFr8?D=i!=5ght5WVGqKA~@o20vrO7g1`LKnC<1p72AKk@t7=X|J+T`2A=^z z>}t>fzv}u7d$-%|cxe-xI=q&P>&gAAquexti>^f&2;qBO1ME9^Z&IjYDAK`@fhz@2rDC+|nqaji zh^w-OP|sj8+mqq=jKrkx87U&;;~fEn!PE(j9@zQ`gN-b-TaQ9puU;2T07j z2H3gA1~n3BEnycd-C|Fj5lcM)=-nRj` zL2r}R&#;VV`d>dnn=KSvzz(!`npvzayKOryEFd6|l<@4~Hd+OML3lSaxS&QPTI$9Q zANO@0+Ym`eJWu)~RWP~Sd9VF85{J<36*_GdH2!!xCsM5FNX@;bet#@B$(G|9f2Dc% zUTGqvW)>K3@2O4vbJ=fgev9p-!KFwv=p&cz##2P0QHa!p2mcnddbf8Qt z1v*DFcKOZroAUDhs?s!v`{f5yI2f-HKR%XK4&5qiOy_<^cdaR#dg&1~rIzw`YS(2C_wkv@)ivlg=gCK2gBcz^xzIybn$ zolDdOjsI*_dhmq=Hh!FbgYVeeN1YWg%r|S3c22WN0bJm9=N)BNB=nX~1FsGKL&R9U zH^5R-hi9ODWC{KQV1AI9iuY3N-KKtF%@?vp^zj_BepAnRL*f3vT&E8L`lRpZVW4Bo z$){V?^zy*Q7-yAXC+T%PlZ~-2_R^sKGaGQ2vYiwb894ZrfczG=MMTF%fGxu*uV_w{ zydF1jCCB{_M+)|&*%rPJRE^(mQE?=q3&ud~4kcmDCZ!z4%_tatDuCQ&bO@NbY__q9 zpKPQ{TPrrNBpW7VuHVFdy43jDZ6j8Me)i*FbACA>h@qg>c$&3_$TZ%YNV)HJ?dafAQPuob4k4S~TjCyM*TuMk$ttP4r=#wN*AWF>K#imZJ-*xlr^_0gC)|fR&D_dAves%KVy+?cMlC_cYiPEt{ zN*{k51mxf~pUR#?9Uq;|+5U)ZI61g1I3g1Y%f69i>J>h4jIr7Efoqrl^N$`VnR#xf zyH|A;Hiv?dBlFslfrw9i6+CL8+g8wV&UwxC)V}%cl9DctzYHDKX=txc2O>ee>fBm{ zf@D)fDZ6hjF9$sXO|%tgSy^=jIHWxTmlH^Mv55xfcrbEnx!kyyub*1{8mwX-Q;(RF z=e}ZKN^u-;8SHE+<|P)~ky5PcuWG%D6}?Xha(CX^Y!%gg!B1=L(DEdN4`+)CkO~C}!_N;+te%I&gn3ZPnqgfbo_C3K9}Ot`R)EAJn+Pth1?(e^l2( zvi`=4yM-iE@+V`u$rH)aiQKH0a5b-TTc@9dg2MC{h`6;Gm%cZq^?=#;1J_`4_=V6? zBDdi1@LSyx*zTElQy_ggjU|TXN5-Fl%!^$l2{^VFOyT3z?jCJm{DH@})n-SnRh$DS z?^O9e=1}o5e40IwM9-J(1%0V;o&BcsUZ^5nY?*eWYhP7W)iZMqagzH%L?Z&A2mr{o z{y2?jWu^f9la|ut-sF{dbJ#%0jD!uc{EbNmeNug;kEEj+qV@wcl2&I2V?nPaslD*$ zb-D}#AYIYC_)VM-iL!z3hYKDCy~-iGDdg?%XpNMSuz`gK12|C}@rObCCqxJRgNSo# zwJSTpD6h0xsZCy*M0Is6sMIKJ0@-}&y#vfr@^1LqC*-w<3sy(-o6i_F4nhE91d8uf z^O~*Vwy9Y#5jRn`o0MX>`m| z1d;le&?>kd#EU7t^wtQ(==ViVL%OCCM#S>MiGJr%WBS2fW)0-rzYcAgJ_2Fx1*@4hQLV6k3~?EKOMK9_LJ{OYIq%kv{5lHO<@}Q>I~HylFj!8P+O0;| z!Epe+47c}-AI<6E+EPZpgp4vG$YA|4t*2PO@IH9nyUol~491Tl^u=PX>YGM$2f>b5Ikgbk@ZI<{Glxy$?jXZ&lh|s9a~JxDY)7xIPfO*HvC}lEY_T zSx)cf`^VsD5rp7;H|jLbpD7;aIbFVV<};+xJ~3JBQ?@IKVv=cFmoK9nofO(DKekZucd!%i$RCWTA(Q`+m_8@#GM`C0@yB_fA zH*UWI?c?Uf&F5_4~mk5wncP`5fd7H(*+udg>vwVm!85o&WR8tlA%RrddH$k_4_UMMq=H1mJGKk-9El^Z~Ng|C7te z5)+T4uV)|G5s5m`@tU*d)VOuGNy(uSZREES06#3)nb68JFtw)AyKQuX)0xNq#It3M zX?7Zinx`@zD54&(FfZ7o(-;;O9BlGF2d6_TWa$b;069S)Q;F8)<`v*%NhYkI1$YUb z6(Qk-ffy;`i}*nqGwwAziX9ml(P#yqq6=Mo3pH?A%53W{60z}>y#U65cfr7`V5BB2 z4*T4EYvaS;6JAJT5-h)|(_95FKuQiAHy`6h_rP9D@eCjl3?Ej$ms zQUHVRb%`(VZt`v5(|xf=YG`CWJm10(@iMXC?z!@gj*b}!722x}$h3L&)W)rW z`f{Jw35xklf***Y=NQWo7(`{_GxG8gOy&uxb<+d&u#SuCm(`kGlH?dO1r@!D2#RRQ zr67cp*%1^G8bM^iO5MA{R)FGpIobYzO^8H;%hk@$pUq#{D3h0M5h^c)rYpFJwUOdL z{nVfaTdlXQn$P{WHuhuhL)lh#(uxFJ@&GJZlX#*ZygA$>aPv7ud2dlSONQf$C@f#_ z1j|K1AP!FtX2}S4q+P(g&Ev$-!{@6;hTO{(C%{0%WU1L8H0!J3;aZVRrK1VMFLtk1 ze*EZ1_F;uNFjAbIu}yqL`|Qs)$)-tg0eW#2V5GY$<?d2b%tPogX>2S5{@7w%JR-QS}Zyl z_u*t@7*%8-d7S`qwDqWI3XNuj-@8%Vq3x5=A527}z{5uDh&u%GEe>peK4E7l0CwBF zr4fOTD`PZ)fve6iZdZqWBAYNZ3i}^a@NvkT6eT4iN}AsiwbuQO;U)J*irQx#k^to33}bWk)J8mFQ>`KQ<>PlN z)K8Z}e=>?)bz$iq`w&nqcs~@dapGbVryNZZ5%}u$kgnfsY)0ODtAk~X|oN&_FB^pm?*eT)ZPT9p{Qw_zMfM}fq=8)-#|cdnVyhZ zgyUPF#l*Y<_wu$!h#7jE>^tm$r@@K-W)!U1oBxt1qY<2A1`vJbjTEdl1utPJCc)Wu5)@@7`yWaPt?$p<|Een;bJ{O7@Xx|r$BE9^PfSa%?S*e;a z$IOqb6OFSr^*xEemghdcsTBltV*Vu-31;N&V@BiS=2^UAaygq7yIwO^3%hnoSH z$ENHa_6&k(emxd416iMrOhVuDne=*R2xG`=yeF!EugwoZ(L8!q0X0fzZb|s9!TPt_ zNYC~BMbL<#cxGp}ni{MTlL>$KF%_2X#z69vCH=%g9O07XfQ5ptu81HX_!2hoc-$wh zCH;buv?OKl!A&7sGkRI#_+(+zOC(Ux#Vu;bNV2VP>uGSxD&clGuCm|V;==LrT9IYk zYu8RWL0)y@KBHnkMPN`43cDfT=EMO33BZ%~UJ!@33tWdv1A4d$og6g~e^MS~>CFaf zZQJ=ca&%GTHdxjdwfE(ogBaXf$6Bk=I(~DTTi2}VLp9esU)WW7Ir%q|x5rqjHu#|N z?EEutMz}KHpYkvB80qwO86zQa;R0+(DMyN&0G2gd27l+NB{6s;-qd5=q)h`!Od&vW zuk{Hzf+rFNkaSm{Y|pZ^v9T#5Of^dD%y+2;HNyb(3ML+~6=e4xRwc&rWmZ6omJe9e zp^8H>r@w;Kf&9)1P)*X7*&_Q3SD+m<-$kaY>AtpzZ^Ug1u9ee~OTe|Pw*CDE##bWY5x$OH5(H8<@qGu6c z;kZQ#&KCfR9taPl(|Tl}rPl#k?r7b)2!Be#pPTmCoW0)V8TiLzCrKpzqv-VOlw2gM z(tgH*jb@7MhDdYs*v8JaW1T;#?Hno|eKV4mUm~_jp*Q(*;j}z@S=*ok#;k<#&i%NA zMhEsh`56gls{HZe$K z546D!F1dZx?={vQL>y?XNB{=r|nGSNgED1e1n04+!mK1OJn(Fr-S7SO^j#DSr)Iq|y4D~w8a zCJ7|G9|9#Qd(($#28c~;08Id?o@JgkZ9f8KUl_OjyiO)T2R$CF9x=b4SCfK$dw%;U z>okgk7X6xe;kA6GBMs)kbh!xFFcfLNzBd3@i4X(Cd5HzYIJ<0Qr8(uj;c&`VJzYns z+7Shonci~joWrJvr@uV&{b%{|xdXAO=RGe#9~L*vo$Wmmx7|V`ao-qh3U`kVCYZ4` zfn##S7`%ea+)Y}Nk{~?G&@_>KnJZiuyBrJm=nKkPhBkhSjHOEq0I*sbC7a z7oN)$#wFXsiL1H3zyHSV@@&r89z7T4&ot^R0Cy9astU^I$cd{$$iqL{SD)G!g6y+P zr*d2LV^f`K*q*|iEoH-AkNTbeyac76)WeESl>Qz(#INMwVz+6~v*7V59S(+6_QPiT zRp9FT>}aB=4Zb53Dj;Y7d_kZPF1a3{TamEg>A!Z+vIU^Xh+j?yYWg5V9-Q;1b5xEc zK4dKBgZ4!awG2aUsC(J%sRLul*Ps|)$@}f%F<}3)@8X+rXhSx=1>D>CqzkfI4{EK> zSVI2d{--s4t}tUwdN^%{D_4EyXr=!wvVy z&u*<^t6=p^<_xTb=w72`g_nX>+1j^d5zD1971veV*2cKd+x<+?C|PEl||=6)g%Y(Oxxk($}2h=cmWz{E2k_cnn^ z-%~_m(o=X$2(aVaV7ZNgQ8+j&@sQh7d8F$>LXk%QPJ{7 zaX6j?(+^2JjKNiPgdU_v6X26=%ZC$L2Bl{`%5XQ>{9`E3>~}G}EATG{5d^^YMeeEL ztFJ^L%6aahsb)aWyB-C#AN^DZ7=!$J@@o1>C^=dU;e(q5IXB6Xvg!9(p)lV1rQUEO;sBB+0eL&(KO^Uo&uBpg7+L6Ka1ab9iA>ws4K9G%y3vQFtA zM_``RtmU@%(M#|uY*R1A{k9vvgBuxtdO$!%h^V)GJAiJokUp5-$ln$%SnVxT^BA&&p&as=cEBy& zmY3?2mf$gr=+2S|<+1Ecr3b{{ey=OeGpYo*a*XQ7tV<68?H_lrhmpkn`K}08MVO0JXcpu4o{; z=3wCImEaL^7Jv`X30jWV4&>D2LF+Q8gZ8t7-syQS$i-++9|}8=@b2v5MVDg8H97|m z09ZRl9sfl){BZ=<+2ybQFkvzVvx0N$$2`La36Cc-LWOltRvSB&oE?BWSw{++N*o7T zTY1uM^{&GVuBq}1n&ff+ghifXWhpP@Jn!kcoS4X~u*&g#+HhwdZ^Cyz+R8s77u#e3 ze@}R=3&BMI5ji(=oBzv}Id~Y3+}Sfu-G8>^#sA}Nn4fEnj!g5{3xp$0Ryv!)2SP%# zdE~>)1(z>&t7jPoV_8hxCKTtKAK4o@~@Se zFGjQ?E!!3K=xxl*_>PapHWhz_JV5?E<}P5z3Ty#1j&&k=9^~Na>U!UBGR@Y@SNQ8%8UBPDaZl-uKJOd(Wc~E2kkTKK1ek*KNLjD$n|)5CUxC4_ z^NtukP!;X@o^b_-Cxt+rO=43kH)*;2IJg_mWUV>MLn3A$Fn-!~aMQ8FhwuaV=%S%Q zQRM7B8b;;Suh?y4kHewagmubMkq-9`p;YxXZDX1` zxGg9~{;WFRl>}plIvDBp%Q5uqC}XU!qX!g4ZQvTIan}!19%??aT9GvxnwsLq*_PGI8o6HQ z{$U}Cu$)E3zw<6aopX@VXif5D+W~yzk$57~tdPczz|AB7rY!z)3~7@#We_(Ql-&Wr zrxjaG3Vu`>Ww^PjS&Df#c0E=Bl-XH|iF&*)%?B&THl)WKGt2)2X6{r#E?&{FJ_atm zS(!9Y8yqbgS6dSeUd2i73MpErmuSXa241`_a^rf#0T6Y{-Lmpf!~#K$!#}?*^ee!O zL%^-ey-eB_6oeLR)bIGp5P}oM!)+tATs%C=_YEKLKzaUt?4usnJI6<^9DcE_XyB)L zE2LU1a%yGbh8|`L*B`(Gjb(<)=I4-w^W zFq{jZBux`6GE24f5rWm9+_v@|%JMp7c`+1n(~Ut6U>^;xZCOj#vh5+vlJ@Ap|8v~_ z=!q^T7-yUJ8d6Csltgl+yTDU`A*B=3K&p1;=(7hE9gR z!*hA-pAbEw?_P1>%(2aR9{e|w}75_^- zZUhj2&gr!YitgzSkRcvi*Ct2j!l(qbx{u6XUU@Wu0@=CXYKk=F$}1{|r}nLe64Ecd zy}hepxRaONn|@bndEOsWOhOC@-`IJ^nHNwxcwUe(0#ZIob^7T(1e6Z^gL?%&fOy&6 zK(c{6tqY0b(tS4q7`1izW>$7!M?MhbwrWK_=kFZ9%6+1%BnMM@7_2Apyn))VqoaA1 zZd(qYNgv2}vvxZ1x&=8bYm6tBXve9R?P(mV1haPX>4knZEkc0J3@w!Gj@67x?qezI*gEPF{H+ zm<@xBWbC2@4j6}PHeB&+EH5wL3Em4k_Ncc-ni*UP%L5_i*(2Uir1`7U-H-(?c6P|h z8F(idtf+;zPV^MYKKHVHcF27MCclY$Tn7a*%fGY?E8~OcYpu;>V0TTQ?8Hod1va2bS z{lk-#ut;1Pr8{p~q9x)8-IlPEdKu#>eu-b9oWv43Ol@|0FbUW5+u!(Zk5Dg6C(7_xV{Zg_K$oXs#w< ztw;_sB{C1w4!0nx=E(PlDYw7K^Nw+&1@ue^iYL1UI!S#wf{rO-fZA%0$J2IIt zd*(wT+@83`_;~cTiJG>mshZru327ZCa5u|5U}&JfGuU=nL+0y=wHk{~KNKMM6p6== zD74qks?zrqJ-G+L1T!W`qr_lh_tD@hxeOR`;&7=Nxhc<81B5BDa2)T9BJ+=pV^gz2 z2u+!@ZBqOna&?|wGwgUmC^&Nrifj&JM;;7#gN~exa(mDDFJ}!rC^h8yM!l{Ud--eV zYw;LFntU-eM$9|Pe4leu%ST9@HHHG;c!EALj4;FB*(Rtpq|+!ZYE&#d12-yG97Cy`*^412W`AW7 zW5ni+-tu6SAO9Xl+a`rjl-vLih$i?2k)8DqD3ZYe!5wa7XAD9d*?n)LYZ~Dro)$pW374j*y~pW`EKRkaIAI1WkP#~1FC{N4Pi4btf*aX%kGqv zlrjhDb$W;Et0HM6F8Sy~UTgHTNyi)aGu!)UL-R!ZCaWS&(o9s-B0cYFXm(U>{gbEc zqXM0;-ssvP%a>h)PadqB<0BJq-bF82!fL3cw)5bpGj$G9p!@~~!;Hg1LmeTMpLa?o z&KM%miokM5v5#IBLQLJtHr8b!@pT4~cp7_rQN%)DBAi z&u{%@iR9M+K~4NIJF?OClPR7HYi(cuE1gg^_vzoE2wGL~x8-^R06D9Y*O3#GP-R3P zO0F{c?=LTu_P8?S$WJm~A@?cTT*5uTREi3i`zbD$g>cE=PW}Rd{(G~~u1g>4MyY~= zjLl>E3-gBAihtY1LO^H?RN}+9OLR()+czG0PEJ_)3fJE+-|l_hVnRss@5lc?0){$x znAW#4pCU(glbz_VtG|fgR`WMhi*L>sWtIEi?-pznOeIEGlUNZ2k@Um#*}E#FWMcN? zEe;o`a8Wv20#Pk|B5{3^%8o)yt6PO78Wey?qdg|1W?( z-e=bAwd`b_xQLs5bh`aBE+t!4%IsBFxN_n!j?7~cjH!PSnY0w<;FOc( z{~&(aMDvSh9g`E&zJ-BUD8>AJG5>XQuFU^A+axE^Wp`FU2g$77L{KcWEb%XDS)%(1 zYU8!&9TZ`w*~x!ELvjwrHt0(Z`Yx#q=+BE$x}YWd3{LI*KmJ$%j+e>O$?U@W5i2G&_4$OBSznp*5zXt1w+ zyxi1B*y-E5WQCw*P&*MrgY_8K_o@ilhJXhcBN*TPye9X>15Iwmo2X;y>&V$tH|#|jb-!73n#l=N81mu&GgZ>vM)lz-!XJtr~~g4N9nE)*GqoSy*@`Y zr#)UywVlQ5f58aU)%wA_Ygs}5PyBC87l(Wz2e!G+hjt@>2WQbU_;}dunO%Jpt~w$b z9x8CN>MiHLBI2}^S-?0{xUPcTCC9k+7{z48I$Nr3<<{I4ks?MLl%c-ypZ;(`;+pXe zr2)oF880Q!8SJdaF5j}{KfSkTgJeJ zRV?I>wrm7jnCD6bWsTw=c@{Ur7}sBqAJC|}W@qetiI>nBvU~KlC@`;#T3uhx_xt4`qbN zvz(=Ths^4<)d(+HkdBMn$nEz*-0ijBw|pf|0n=L)R->GbMGNA;Gd}hTN_kfCeQ=p5 zjlqeR0oh?Mkb0By#sT)2-h3rTxq(9d$_F1fm9}`r^B31p{*{(G)YG3ViRqtj_;Rn! z+O({E{|Sqed7&3+^^Y)^Lv995AITrU`vCKlY;S*$JuvH3Sj}me$*bpUQRC*rzcL0M{5G68RAP0S>>*dPK-?rWH{|{-n2r9%N zfUzxQ!HZ-_0B#$c&nGd|szCO=d__B>eBTfV5BNJ@Z`L@R%dPVv&OfE#MBM*>;2k Date: Thu, 20 Apr 2023 11:39:20 +0800 Subject: [PATCH 3/9] disttask: add on subtask finished interface for scheduler (#43155) close pingcap/tidb#43153 --- ddl/scheduler.go | 5 +++++ disttask/framework/framework_test.go | 4 ++++ disttask/framework/scheduler/interface.go | 2 ++ disttask/framework/scheduler/interface_mock.go | 10 ++++++++-- disttask/framework/scheduler/manager.go | 2 +- disttask/framework/scheduler/scheduler.go | 7 ++++++- disttask/framework/scheduler/scheduler_test.go | 11 ++++++----- disttask/loaddata/scheduler.go | 6 ++++++ 8 files changed, 38 insertions(+), 9 deletions(-) diff --git a/ddl/scheduler.go b/ddl/scheduler.go index be1d4649cc6d1..a446b9ccaaab9 100644 --- a/ddl/scheduler.go +++ b/ddl/scheduler.go @@ -186,6 +186,11 @@ func (b *backfillSchedulerHandle) SplitSubtask(_ context.Context, subtask []byte return nil, consumer.getResult() } +// OnSubtaskFinished implements the Scheduler interface. +func (*backfillSchedulerHandle) OnSubtaskFinished(context.Context, []byte) error { + return nil +} + // CleanupSubtaskExecEnv implements the Scheduler interface. func (b *backfillSchedulerHandle) CleanupSubtaskExecEnv(context.Context) error { logutil.BgLogger().Info("[ddl] lightning cleanup subtask exec env") diff --git a/disttask/framework/framework_test.go b/disttask/framework/framework_test.go index 2a93020c8b8c6..a705e15a7c0f0 100644 --- a/disttask/framework/framework_test.go +++ b/disttask/framework/framework_test.go @@ -66,6 +66,10 @@ func (t *testScheduler) SplitSubtask(_ context.Context, subtask []byte) ([]proto }, nil } +func (t *testScheduler) OnSubtaskFinished(_ context.Context, _ []byte) error { + return nil +} + type testSubtaskExecutor struct { v *atomic.Int64 } diff --git a/disttask/framework/scheduler/interface.go b/disttask/framework/scheduler/interface.go index 79374c9c7cc1d..28204ce65c82d 100644 --- a/disttask/framework/scheduler/interface.go +++ b/disttask/framework/scheduler/interface.go @@ -53,6 +53,8 @@ type Scheduler interface { SplitSubtask(ctx context.Context, subtask []byte) ([]proto.MinimalTask, error) // CleanupSubtaskExecEnv is used to clean up the environment for the subtask executor. CleanupSubtaskExecEnv(context.Context) error + // OnSubtaskFinished is used to handle the subtask when it is finished. + OnSubtaskFinished(ctx context.Context, subtask []byte) error // Rollback is used to rollback all subtasks. Rollback(context.Context) error } diff --git a/disttask/framework/scheduler/interface_mock.go b/disttask/framework/scheduler/interface_mock.go index 3e35502775ca0..cd04a6cdb3e0f 100644 --- a/disttask/framework/scheduler/interface_mock.go +++ b/disttask/framework/scheduler/interface_mock.go @@ -124,14 +124,20 @@ func (m *MockScheduler) InitSubtaskExecEnv(ctx context.Context) error { } // SplitSubtask implements Scheduler.SplitSubtask. -func (m *MockScheduler) SplitSubtask(_ context.Context, subtask []byte) ([]proto.MinimalTask, error) { - args := m.Called(subtask) +func (m *MockScheduler) SplitSubtask(ctx context.Context, subtask []byte) ([]proto.MinimalTask, error) { + args := m.Called(ctx, subtask) if args.Error(1) != nil { return nil, args.Error(1) } return args.Get(0).([]proto.MinimalTask), nil } +// OnSubtaskFinished implements Scheduler.OnSubtaskFinished. +func (m *MockScheduler) OnSubtaskFinished(ctx context.Context, subtask []byte) error { + args := m.Called(ctx, subtask) + return args.Error(0) +} + // CleanupSubtaskExecEnv implements Scheduler.CleanupSubtaskExecEnv. func (m *MockScheduler) CleanupSubtaskExecEnv(ctx context.Context) error { args := m.Called(ctx) diff --git a/disttask/framework/scheduler/manager.go b/disttask/framework/scheduler/manager.go index fd5723d2684d7..9cc4791ece04b 100644 --- a/disttask/framework/scheduler/manager.go +++ b/disttask/framework/scheduler/manager.go @@ -181,13 +181,13 @@ func (m *Manager) fetchAndFastCancelTasks(ctx context.Context) { func (m *Manager) onRunnableTasks(ctx context.Context, tasks []*proto.Task) { tasks = m.filterAlreadyHandlingTasks(tasks) for _, task := range tasks { - logutil.Logger(m.logCtx).Info("onRunnableTasks", zap.Any("task", task)) if _, ok := m.subtaskExecutorPools[task.Type]; !ok { logutil.Logger(m.logCtx).Error("unknown task type", zap.String("type", task.Type)) continue } exist, err := m.taskTable.HasSubtasksInStates(m.id, task.ID, proto.TaskStatePending, proto.TaskStateRevertPending) if err != nil { + logutil.Logger(m.logCtx).Error("check subtask exist failed", zap.Error(err)) m.onError(err) continue } diff --git a/disttask/framework/scheduler/scheduler.go b/disttask/framework/scheduler/scheduler.go index b28e09cac2226..7cad7f75a2a8e 100644 --- a/disttask/framework/scheduler/scheduler.go +++ b/disttask/framework/scheduler/scheduler.go @@ -139,7 +139,7 @@ func (s *InternalSchedulerImpl) Run(ctx context.Context, task *proto.Task) error break } - minimalTasks, err := scheduler.SplitSubtask(context.Background(), subtask.Meta) + minimalTasks, err := scheduler.SplitSubtask(runCtx, subtask.Meta) if err != nil { s.onError(err) break @@ -154,6 +154,11 @@ func (s *InternalSchedulerImpl) Run(ctx context.Context, task *proto.Task) error } } minimalTaskWg.Wait() + if err := s.getError(); err == nil { + if err := scheduler.OnSubtaskFinished(runCtx, subtask.Meta); err != nil { + s.onError(err) + } + } if err := s.getError(); err != nil { if errors.Cause(err) == context.Canceled { s.updateSubtaskState(subtask.ID, proto.TaskStateCanceled) diff --git a/disttask/framework/scheduler/scheduler_test.go b/disttask/framework/scheduler/scheduler_test.go index 226d4cd4cbbf5..24d54826ee3ad 100644 --- a/disttask/framework/scheduler/scheduler_test.go +++ b/disttask/framework/scheduler/scheduler_test.go @@ -89,7 +89,7 @@ func TestSchedulerRun(t *testing.T) { mockPool.On("RunWithConcurrency", mock.Anything, mock.Anything).Return(nil).Once() mockSubtaskTable.On("GetSubtaskInStates", "id", taskID, []interface{}{proto.TaskStatePending}).Return(&proto.Subtask{ID: 1}, nil).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateRunning).Return(nil).Once() - mockScheduler.On("SplitSubtask", mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() + mockScheduler.On("SplitSubtask", mock.Anything, mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateFailed).Return(nil).Once() mockScheduler.On("CleanupSubtaskExecEnv", mock.Anything).Return(nil).Once() err = scheduler.Run(runCtx, &proto.Task{Type: tp, ID: taskID, Concurrency: concurrency}) @@ -105,7 +105,7 @@ func TestSchedulerRun(t *testing.T) { mockPool.On("RunWithConcurrency", mock.Anything, mock.Anything).Return(nil).Once() mockSubtaskTable.On("GetSubtaskInStates", "id", taskID, []interface{}{proto.TaskStatePending}).Return(&proto.Subtask{ID: 1}, nil).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateRunning).Return(nil).Once() - mockScheduler.On("SplitSubtask", mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() + mockScheduler.On("SplitSubtask", mock.Anything, mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() mockSubtaskExecutor.On("Run", mock.Anything).Return(runSubtaskErr).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateFailed).Return(nil).Once() mockScheduler.On("CleanupSubtaskExecEnv", mock.Anything).Return(nil).Once() @@ -117,8 +117,9 @@ func TestSchedulerRun(t *testing.T) { mockPool.On("RunWithConcurrency", mock.Anything, mock.Anything).Return(nil).Once() mockSubtaskTable.On("GetSubtaskInStates", "id", taskID, []interface{}{proto.TaskStatePending}).Return(&proto.Subtask{ID: 1}, nil).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateRunning).Return(nil).Once() - mockScheduler.On("SplitSubtask", mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() + mockScheduler.On("SplitSubtask", mock.Anything, mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() mockSubtaskExecutor.On("Run", mock.Anything).Return(nil).Once() + mockScheduler.On("OnSubtaskFinished", mock.Anything, mock.Anything).Return(nil).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateSucceed).Return(nil).Once() mockSubtaskTable.On("GetSubtaskInStates", "id", taskID, []interface{}{proto.TaskStatePending}).Return(nil, nil).Once() mockScheduler.On("CleanupSubtaskExecEnv", mock.Anything).Return(nil).Once() @@ -130,7 +131,7 @@ func TestSchedulerRun(t *testing.T) { mockPool.On("RunWithConcurrency", mock.Anything, mock.Anything).Return(nil).Once() mockSubtaskTable.On("GetSubtaskInStates", "id", taskID, []interface{}{proto.TaskStatePending}).Return(&proto.Subtask{ID: 1}, nil).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateRunning).Return(nil).Once() - mockScheduler.On("SplitSubtask", mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}, MockMinimalTask{}}, nil).Once() + mockScheduler.On("SplitSubtask", mock.Anything, mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}, MockMinimalTask{}}, nil).Once() mockSubtaskExecutor.On("Run", mock.Anything).Return(nil).Once() mockSubtaskExecutor.On("Run", mock.Anything).Return(context.Canceled).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateCanceled).Return(nil).Once() @@ -244,7 +245,7 @@ func TestScheduler(t *testing.T) { mockPool.On("RunWithConcurrency", mock.Anything, mock.Anything).Return(nil).Once() mockSubtaskTable.On("GetSubtaskInStates", "id", taskID, []interface{}{proto.TaskStatePending}).Return(&proto.Subtask{ID: 1}, nil).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateRunning).Return(nil).Once() - mockScheduler.On("SplitSubtask", mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() + mockScheduler.On("SplitSubtask", mock.Anything, mock.Anything).Return([]proto.MinimalTask{MockMinimalTask{}}, nil).Once() mockSubtaskExecutor.On("Run", mock.Anything).Return(runSubtaskErr).Once() mockSubtaskTable.On("UpdateSubtaskState", taskID, proto.TaskStateFailed).Return(nil).Once() mockScheduler.On("CleanupSubtaskExecEnv", mock.Anything).Return(nil).Once() diff --git a/disttask/loaddata/scheduler.go b/disttask/loaddata/scheduler.go index 2f65cad56e2ea..e08213b2240d7 100644 --- a/disttask/loaddata/scheduler.go +++ b/disttask/loaddata/scheduler.go @@ -100,6 +100,12 @@ func (s *ImportScheduler) SplitSubtask(ctx context.Context, bs []byte) ([]proto. return miniTask, nil } +// OnSubtaskFinished implements the Scheduler.OnSubtaskFinished interface. +func (s *ImportScheduler) OnSubtaskFinished(context.Context, []byte) error { + logutil.BgLogger().Info("OnSubtaskFinished", zap.Any("taskMeta", s.taskMeta)) + return nil +} + // CleanupSubtaskExecEnv implements the Scheduler.CleanupSubtaskExecEnv interface. func (s *ImportScheduler) CleanupSubtaskExecEnv(ctx context.Context) (err error) { defer func() { From 66c36ededbc122f999bdf644cf87e885fc5809d1 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 20 Apr 2023 12:27:19 +0800 Subject: [PATCH 4/9] *: support load data for table with generated columns (#43108) close pingcap/tidb#39885 --- executor/importer/import.go | 8 ++--- executor/load_data.go | 6 ++++ executor/loadremotetest/one_csv_test.go | 46 +++++++++++++++++++++++++ 3 files changed, 54 insertions(+), 6 deletions(-) diff --git a/executor/importer/import.go b/executor/importer/import.go index 5a536c3752bfb..eb0241485485a 100644 --- a/executor/importer/import.go +++ b/executor/importer/import.go @@ -593,6 +593,7 @@ func (e *LoadDataController) initFieldMappings() []string { if len(e.ColumnsAndUserVars) == 0 { for _, v := range tableCols { + // Data for generated column is generated from the other rows rather than from the parsed data. fieldMapping := &FieldMapping{ Column: v, } @@ -641,12 +642,7 @@ func (e *LoadDataController) initLoadColumns(columnNames []string) error { return dbterror.ErrBadField.GenWithStackByArgs(missingColName, "field list") } - for _, col := range cols { - if !col.IsGenerated() { - // todo: should report error here, since in reorderColumns we report error if en(cols) != len(columnNames) - e.InsertColumns = append(e.InsertColumns, col) - } - } + e.InsertColumns = append(e.InsertColumns, cols...) // e.InsertColumns is appended according to the original tables' column sequence. // We have to reorder it to follow the use-specified column order which is shown in the columnNames. diff --git a/executor/load_data.go b/executor/load_data.go index 0c14652b221fc..5e6b0c69805cf 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -816,6 +816,12 @@ func (w *encodeWorker) parserData2TableData( continue } + // Don't set the value for generated columns. + if fieldMappings[i].Column.IsGenerated() { + row = append(row, types.NewDatum(nil)) + continue + } + row = append(row, parserData[i]) } for i := 0; i < len(w.controller.ColumnAssignments); i++ { diff --git a/executor/loadremotetest/one_csv_test.go b/executor/loadremotetest/one_csv_test.go index 4fc818360e283..c9d4edc54e9e2 100644 --- a/executor/loadremotetest/one_csv_test.go +++ b/executor/loadremotetest/one_csv_test.go @@ -266,3 +266,49 @@ mynull,"mynull" LINES TERMINATED BY '\n';`, gcsEndpoint) s.tk.MustMatchErrMsg(sql, `must specify FIELDS \[OPTIONALLY\] ENCLOSED BY`) } + +func (s *mockGCSSuite) TestLoadDataForGeneratedColumns() { + // For issue https://github.com/pingcap/tidb/issues/39885 + s.tk.MustExec("DROP DATABASE IF EXISTS load_csv;") + s.tk.MustExec("CREATE DATABASE load_csv;") + s.tk.MustExec("USE load_csv;") + s.tk.MustExec("set @@sql_mode = ''") + s.tk.MustExec(`CREATE TABLE load_csv.t_gen1 (a int, b int generated ALWAYS AS (a+1));`) + + s.server.CreateObject(fakestorage.Object{ + ObjectAttrs: fakestorage.ObjectAttrs{ + BucketName: "test-bucket", + Name: "generated_columns.csv", + }, + Content: []byte("1 2\n2 3"), + }) + + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen1", gcsEndpoint)) + s.tk.MustQuery("select * from t_gen1").Check(testkit.Rows("1 2", "2 3")) + s.tk.MustExec("delete from t_gen1") + + // Specify the column, this should also work. + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen1 (a)", gcsEndpoint)) + s.tk.MustQuery("select * from t_gen1").Check(testkit.Rows("1 2", "2 3")) + + // Swap the column and test again. + s.tk.MustExec(`create table t_gen2 (a int generated ALWAYS AS (b+1), b int);`) + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen2", gcsEndpoint)) + s.tk.MustQuery("select * from t_gen2").Check(testkit.Rows("3 2", "4 3")) + s.tk.MustExec(`delete from t_gen2`) + + // Specify the column b + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen2 (b)", gcsEndpoint)) + s.tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1262 Row 1 was truncated; it contained more data than there were input columns", + "Warning 1262 Row 2 was truncated; it contained more data than there were input columns")) + s.tk.MustQuery("select * from t_gen2").Check(testkit.Rows("2 1", "3 2")) + s.tk.MustExec(`delete from t_gen2`) + + // Specify the column a + s.tk.MustExec(fmt.Sprintf("LOAD DATA INFILE 'gcs://test-bucket/generated_columns.csv?endpoint=%s' INTO TABLE load_csv.t_gen2 (a)", gcsEndpoint)) + s.tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1262 Row 1 was truncated; it contained more data than there were input columns", + "Warning 1262 Row 2 was truncated; it contained more data than there were input columns")) + s.tk.MustQuery("select * from t_gen2").Check(testkit.Rows(" ", " ")) +} From a7d636e2bbe9c30310da7c2094552828234a0537 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Thu, 20 Apr 2023 12:27:27 +0800 Subject: [PATCH 5/9] execdetails: unify WRU/RRU to RU (#43188) close pingcap/tidb#43187 --- executor/explain.go | 3 ++- executor/explainfor_test.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/executor/explain.go b/executor/explain.go index 5bbe4351a884b..ed55ebcc5c75b 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -16,6 +16,7 @@ package executor import ( "context" + "fmt" "os" "path/filepath" "runtime" @@ -324,7 +325,7 @@ type ruRuntimeStats struct { // String implements the RuntimeStats interface. func (e *ruRuntimeStats) String() string { if e.RURuntimeStats != nil { - return e.RURuntimeStats.String() + return fmt.Sprintf("RU:%f", e.RURuntimeStats.RRU()+e.RURuntimeStats.WRU()) } return "" } diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 9e18cf8413882..11271a12698d8 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -74,7 +74,7 @@ func TestExplainFor(t *testing.T) { buf.WriteString(fmt.Sprintf("%v", v)) } } - require.Regexp(t, "TableReader_5 10000.00 0 root time:.*, loops:1,( RRU:.*, WRU:.*,)? cop_task: {num:.*, max:.*, proc_keys:.* rpc_num: 1, rpc_time:.*} data:TableFullScan_4 N/A N/A\n"+ + require.Regexp(t, "TableReader_5 10000.00 0 root time:.*, loops:1,( RU:.*,)? cop_task: {num:.*, max:.*, proc_keys:.* rpc_num: 1, rpc_time:.*} data:TableFullScan_4 N/A N/A\n"+ "└─TableFullScan_4 10000.00 0 cop.* table:t1 tikv_task:{time:.*, loops:0} keep order:false, stats:pseudo N/A N/A", buf.String()) } From 3a881e47c18ec5d1192c62cbf45cf82e6dee77a4 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Thu, 20 Apr 2023 12:57:19 +0800 Subject: [PATCH 6/9] doc: add multi-valued index design doc (#42207) close pingcap/tidb#42208 --- docs/design/2023-03-14-multi-valued-index.md | 186 +++++++++++++++++++ 1 file changed, 186 insertions(+) create mode 100644 docs/design/2023-03-14-multi-valued-index.md diff --git a/docs/design/2023-03-14-multi-valued-index.md b/docs/design/2023-03-14-multi-valued-index.md new file mode 100644 index 0000000000000..0ad0243102d5e --- /dev/null +++ b/docs/design/2023-03-14-multi-valued-index.md @@ -0,0 +1,186 @@ +# Multi-valued index Design Documents + +- Author(s): [Xiong Jiwei](http://github.com/xiongjiwei), [Zhang Yuanjia](https://github.com/qw4990) + +## Table of Contents + +* [Introduction](#introduction) +* [Motivation or Background](#motivation-or-background) +* [Detailed Design](#detailed-design) +* [Impacts & Risks](#impacts--risks) + +## Introduction + +This document introduces the technical design of implementing 'Multi-valued index' in TiDB. + +## Motivation or Background + +Multi-valued index is a new feature introduced in MySQL 8.0.17, which allows defining indexes on a JSON array and use the index via JSON functions, similar to MongoDB in Multikey Indexes. e.g + +```sql +CREATE TABLE t1 (data JSON); +CREATE INDEX zips ON t1((CAST(data->'$.zip' AS UNSIGNED ARRAY))); + +INSERT INTO t1 VALUES + +('{"id":1, "zip": [0,111,333]}'),('{"id":2, "zip": [123,456,0]}'), +('{"id":3, "zip": [123,123,111]}'), +('{"id":4, "zip": [456,567,222]}'), +('{"id":5, "zip": []}'); + +mysql> SELECT * FROM t1 WHERE 123 MEMBER OF (data->'$.zip'); ++-----------------------------------+ +| data | ++-----------------------------------+ +| {"id": 2, "zip": [123, 456, 0]} | +| {"id": 3, "zip": [123, 123, 111]} | ++-----------------------------------+ +2 rows in set (0.01 sec) +``` + +That is: N index records point to one row record (N: 1) +A common scenario involves having rows with associated tags and the need to efficiently query all data containing a specific tag. + +## Detailed Design + +### Overview + +- Using `cast(... as ... array)` to define a multi-valued index, which is essentially an expression index whose virtual column type is "array with type". The index is encoded in the same way as normal secondary indexes. +- The update of the multi-valued index behaves the same as the normal secondary index but the modification of one row may produce changes to multiple index records. If the type of array element is not satisfied the index definition, an error will be reported. +- Use `MEMBER OF`, `JSON_CONTAINS(subset)`, `JSON_OVERLAPS(intersection)` functions in where condition to using the multi-valued index. + +### Encoding + +The encoding of each index record is identical to the normal secondary index(see [TiDB Index Key/Value Format](https://docs.google.com/document/d/1Co5iMiaxitv3okJmLYLJxZYCNChcjzswJMRr-_45Eqg/edit) for more details). + +For string types, the encoding result in TiDB is collation-aware, we could use `binary` collation for strings(in MySQL it is `utf8mb4_0900_as_cs` and behaves almost the same as `binary`). + +- A row record may have multiple index records corresponding to it +``` +row ('pk1', [1, 1, 2]) +produces index records +1 -> 'pk1' +2 -> 'pk1' +``` + +- Multi-valued index can be a compound index +``` +row ('pk1', c1, [1, 1, 2], c2) +produces index records +(c1,1,c2) -> 'pk1' +(c1,2,c2) -> 'pk1' +``` + +- Multi-valued index can be a unique index + +### Parser + +New syntax: use `cast(... as... array)` to create the index. Add an `Array` field in `FuncCastExpr` indicate the use of this syntax. +```golang +type FuncCastExpr struct { + //... + Array bool +} +``` + +### Expression + +Use `JSONBinary` type as the return type of expression `cast(... as... array)`. In the `FieldType` structure, add an `array` field indicates whether the type is an array, and `tp` represents the type of elements in the array. +```golang +type FieldType struct { + // tp is type of the array elements + tp byte + // array indicates whether the type is an array + array bool +} +``` + +Implement new built-in functions `castAsTypedArrayFunctionSig`, `MEMBER OF` and `JSON_OVERLAPS`. + +### DML + +Data changes cause index changes, which are handled in the same way as normal secondary indexes. +- Insert: insert a new row record and add index records for each element in the array. +- Delete: delete the row record and delete index records for each element in the array. +- Update: delete the old index records and add new index records. + + +### DDL + +- Multi-valued index can be composite index, but only one JSON array can be used in the index definition. +- Multi-valued index is an expression index, so it has the same restrictions as expression index. +- Multi-valued index can be a unique index, but the uniqueness is not guaranteed within the same JSON array. + +## Planner + +### Column substitute + +The column in the where condition will substituted with the corresponding expression in the index definition if the query meet the following 3 requirements +- Where condition contains any of the 3 functions: `MEMBER OF`/`JSON_CONTAINS`/`JSON_OVERLAPS`. +- Functions' parameter type must consistent with multi-valued index definitions. +- The expression is consistent with multi-valued index definition. + +For example, the index definition is `create index idx on t((cast(data->'$.zip' as unsigned array)))`, the where condition is `where 123 member of (data->'$.zip')`, the column `data->'$.zip'` is substituted with `cast(data->'$.zip' as unsigned array)`. + +Index selection is the same as normal secondary index. + +### Build the operator + +For any of the 3 functions, we can use `IndexMerge` operator to fetch the data: +- MEMBER OF () +``` +IndexMerge + IndexRangeScan() + TableRowIDScan +``` + +- JSON_CONTAINS(, [, , , ...]) +``` +IndexMerge(AND) + IndexRangeScan() + IndexRangeScan() + IndexRangeScan() + ... + TableRowIDScan +``` + +- JSON_OVERLAPS(, [, , , ...]) +``` +IndexMerge(OR) + IndexRangeScan() + IndexRangeScan() + IndexRangeScan() + ... + TableRowIDScan +``` + +Each `IndexRangeScan` is a `PointGet` like operator. It will fetch the row record by the index record. Since different indexes could match the same primary key, we need to use `IndexMerge` to filter the duplicated row records. For `JSON_CONTAINS` we should use `AND` type of `IndexMerge`, because only the primary key that contained in all the `IndexRangeScan` can be filtered. For `JSON_OVERLAPS` we should use `OR` to filter the row records. + +If the multi-valued index is unique, it can be further optimized to `PointGet`. + +## Impacts & Risks + +### Limitations, and Characteristics +- Multi-valued index will only be used when the where condition contains any of the functions `MEMBER OF`/`JSON_CONTAINS`/`JSON_OVERLAPS`. So even if SQL contains hint, force index, use index, etc., it is not necessarily possible to force multi-valued index. +- `cast(... as ... array)` can only appear once in the composite index definition, and the casted column must be a JSON column. +- If multi-valued index is a unique index, then we have +```sql +-- Allowed: +INSERT INTO t1 VALUES('[1,1,2]'); +INSERT INTO t1 VALUES('[3,3,3,4,4,4]'); + +-- Disallowed, report dup-key error: +INSERT INTO t1 VALUES('[1,2]'); +INSERT INTO t1 VALUES('[2,3]'); +``` +- Nullability + - If the write data is an empty array, there will be no corresponding index record. Therefore: `not xxx` cannot use the index, because empty array data cannot access through the index. + - workaround: `select * from t where pk not in (select pk from t where a member of (xxx))`. + - If a column is null, add a null index record. + - Null is not allowed as an array item, trying to write will report an error. +- The type of multi-valued index cannot be `BINARY`, `JSON`, and `YEAR`. +- Multi-valued index cannot be a primary key or a foreign key. +- Storage Space and Performance: + - average number of array items * secondary index uses space. + - Compared to normal indexes, DML will produce more changes to multi-valued index records, so the Multi-valued index will bring more performance impact than normal indexes. +- All other limitations in expression index. From d5473063350745da072c06683a622a022ca639c9 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 20 Apr 2023 13:39:18 +0800 Subject: [PATCH 7/9] lightning: fix MakeTableRegions may panic when context is canceled (#43196) close pingcap/tidb#43195 --- br/pkg/lightning/mydump/BUILD.bazel | 1 + br/pkg/lightning/mydump/region.go | 121 ++++++++++--------------- br/pkg/lightning/mydump/region_test.go | 7 ++ 3 files changed, 56 insertions(+), 73 deletions(-) diff --git a/br/pkg/lightning/mydump/BUILD.bazel b/br/pkg/lightning/mydump/BUILD.bazel index 529796e103c82..238ff5d4c7ef8 100644 --- a/br/pkg/lightning/mydump/BUILD.bazel +++ b/br/pkg/lightning/mydump/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "@com_github_xitongsys_parquet_go//reader", "@com_github_xitongsys_parquet_go//source", "@org_golang_x_exp//slices", + "@org_golang_x_sync//errgroup", "@org_golang_x_text//encoding", "@org_golang_x_text//encoding/charmap", "@org_golang_x_text//encoding/simplifiedchinese", diff --git a/br/pkg/lightning/mydump/region.go b/br/pkg/lightning/mydump/region.go index 10d0f03689489..2f957f593302c 100644 --- a/br/pkg/lightning/mydump/region.go +++ b/br/pkg/lightning/mydump/region.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/util/mathutil" "go.uber.org/zap" + "golang.org/x/sync/errgroup" ) const ( @@ -217,84 +218,53 @@ func MakeTableRegions( start := time.Now() - execCtx, cancel := context.WithCancel(ctx) - defer cancel() - concurrency := mathutil.Max(cfg.Concurrency, 2) - fileChan := make(chan FileInfo, concurrency) - resultChan := make(chan fileRegionRes, concurrency) - var wg sync.WaitGroup - for i := 0; i < concurrency; i++ { - wg.Add(1) - go func() { - defer wg.Done() - for info := range fileChan { - var ( - regions []*TableRegion - sizes []float64 - err error - ) - dataFileSize := info.FileMeta.FileSize - if info.FileMeta.Type == SourceTypeParquet { - regions, sizes, err = makeParquetFileRegion(ctx, cfg, info) - } else if info.FileMeta.Type == SourceTypeCSV && cfg.StrictFormat && - info.FileMeta.Compression == CompressionNone && - dataFileSize > cfg.MaxChunkSize+cfg.MaxChunkSize/largeCSVLowerThresholdRation { - // If a csv file is overlarge, we need to split it into multiple regions. - // Note: We can only split a csv file whose format is strict. - // We increase the check threshold by 1/10 of the `max-region-size` because the source file size dumped by tools - // like dumpling might be slight exceed the threshold when it is equal `max-region-size`, so we can - // avoid split a lot of small chunks. - // If a csv file is compressed, we can't split it now because we can't get the exact size of a row. - regions, sizes, err = SplitLargeCSV(ctx, cfg, info) - } else { - regions, sizes, err = MakeSourceFileRegion(execCtx, cfg, info) - } - select { - case resultChan <- fileRegionRes{info: info, regions: regions, sizes: sizes, err: err}: - case <-ctx.Done(): - return - } - if err != nil { - log.FromContext(ctx).Error("make source file region error", zap.Error(err), zap.String("file_path", info.FileMeta.Path)) - break - } - } - }() - } + var fileRegionsMap sync.Map - go func() { - wg.Wait() - close(resultChan) - }() - - errChan := make(chan error, 1) + eg, egCtx := errgroup.WithContext(ctx) + eg.SetLimit(concurrency) meta := cfg.TableMeta - fileRegionsMap := make(map[string]fileRegionRes, len(meta.DataFiles)) - go func() { - for res := range resultChan { - if res.err != nil { - errChan <- res.err - return + for _, info := range meta.DataFiles { + info := info + eg.Go(func() error { + select { + case <-egCtx.Done(): + return nil + default: } - fileRegionsMap[res.info.FileMeta.Path] = res - } - errChan <- nil - }() - for _, dataFile := range meta.DataFiles { - select { - case fileChan <- dataFile: - case <-ctx.Done(): - close(fileChan) - return nil, ctx.Err() - case err := <-errChan: - return nil, err - } + var ( + regions []*TableRegion + sizes []float64 + err error + ) + dataFileSize := info.FileMeta.FileSize + if info.FileMeta.Type == SourceTypeParquet { + regions, sizes, err = makeParquetFileRegion(egCtx, cfg, info) + } else if info.FileMeta.Type == SourceTypeCSV && cfg.StrictFormat && + info.FileMeta.Compression == CompressionNone && + dataFileSize > cfg.MaxChunkSize+cfg.MaxChunkSize/largeCSVLowerThresholdRation { + // If a csv file is overlarge, we need to split it into multiple regions. + // Note: We can only split a csv file whose format is strict. + // We increase the check threshold by 1/10 of the `max-region-size` because the source file size dumped by tools + // like dumpling might be slight exceed the threshold when it is equal `max-region-size`, so we can + // avoid split a lot of small chunks. + // If a csv file is compressed, we can't split it now because we can't get the exact size of a row. + regions, sizes, err = SplitLargeCSV(egCtx, cfg, info) + } else { + regions, sizes, err = MakeSourceFileRegion(egCtx, cfg, info) + } + if err != nil { + log.FromContext(egCtx).Error("make source file region error", zap.Error(err), zap.String("file_path", info.FileMeta.Path)) + return err + } + result := fileRegionRes{info: info, regions: regions, sizes: sizes, err: err} + fileRegionsMap.Store(info.FileMeta.Path, result) + return nil + }) } - close(fileChan) - err := <-errChan - if err != nil { + + if err := eg.Wait(); err != nil { return nil, err } @@ -303,7 +273,12 @@ func MakeTableRegions( // rebase row-id for all chunk rowIDBase := int64(0) for _, dataFile := range meta.DataFiles { - fileRegionsRes := fileRegionsMap[dataFile.FileMeta.Path] + v, ok := fileRegionsMap.Load(dataFile.FileMeta.Path) + if !ok { + return nil, errors.Errorf("file %s not found in MakeTableRegions", dataFile.FileMeta.Path) + } + //nolint: forcetypeassert + fileRegionsRes := v.(fileRegionRes) for _, region := range fileRegionsRes.regions { region.Chunk.PrevRowIDMax += rowIDBase region.Chunk.RowIDMax += rowIDBase diff --git a/br/pkg/lightning/mydump/region_test.go b/br/pkg/lightning/mydump/region_test.go index 2633a6541e947..d947f1783242c 100644 --- a/br/pkg/lightning/mydump/region_test.go +++ b/br/pkg/lightning/mydump/region_test.go @@ -221,6 +221,13 @@ func TestMakeTableRegionsSplitLargeFile(t *testing.T) { assert.Equal(t, int64(0), regions[0].Chunk.Offset) assert.Equal(t, TableFileSizeINF, regions[0].Chunk.EndOffset) assert.Len(t, regions[0].Chunk.Columns, 0) + + // test canceled context will not panic + ctx, cancel := context.WithCancel(context.Background()) + cancel() + for i := 0; i < 20; i++ { + _, _ = MakeTableRegions(ctx, divideConfig) + } } func TestCompressedMakeSourceFileRegion(t *testing.T) { From b2853fd31aef1238eb42eaa8df49e609cf7823f6 Mon Sep 17 00:00:00 2001 From: CbcWestwolf <1004626265@qq.com> Date: Thu, 20 Apr 2023 14:01:19 +0800 Subject: [PATCH 8/9] printer: print commit hash version for extensions (#43095) close pingcap/tidb#43096 --- Makefile | 16 +++++++++++----- Makefile.common | 5 +++++ build/print-workspace-status.sh | 5 +++++ tidb-server/BUILD.bazel | 1 + util/printer/printer.go | 17 ++++++++++++++--- util/versioninfo/versioninfo.go | 3 ++- 6 files changed, 38 insertions(+), 9 deletions(-) diff --git a/Makefile b/Makefile index d2bbdd11bb5e8..3890dce48fe98 100644 --- a/Makefile +++ b/Makefile @@ -159,23 +159,29 @@ else CGO_ENABLED=1 $(GOBUILD) -gcflags="all=-N -l" $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' ./tidb-server endif +init-submodule: + git submodule init && git submodule update --force + enterprise-prepare: - git submodule init && git submodule update && cd extension/enterprise/generate && $(GO) generate -run genfile main.go + cd extension/enterprise/generate && $(GO) generate -run genfile main.go enterprise-clear: cd extension/enterprise/generate && $(GO) generate -run clear main.go -enterprise-docker: enterprise-prepare +enterprise-docker: init-submodule enterprise-prepare docker build -t "$(DOCKERPREFIX)tidb:latest" --build-arg 'GOPROXY=$(shell go env GOPROXY),' -f Dockerfile.enterprise . enterprise-server-build: ifeq ($(TARGET), "") - CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server tidb-server/main.go + CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG) $(EXTENSION_FLAG)' -o bin/tidb-server tidb-server/main.go else - CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' tidb-server/main.go + CGO_ENABLED=1 $(GOBUILD) -tags enterprise $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG) $(EXTENSION_FLAG)' -o '$(TARGET)' tidb-server/main.go endif -enterprise-server: enterprise-prepare enterprise-server-build +enterprise-server: + @make init-submodule + @make enterprise-prepare + @make enterprise-server-build server_check: ifeq ($(TARGET), "") diff --git a/Makefile.common b/Makefile.common index f2ba132767617..66933c93fc256 100644 --- a/Makefile.common +++ b/Makefile.common @@ -66,6 +66,11 @@ LDFLAGS += -X "github.com/pingcap/tidb/util/versioninfo.TiDBGitHash=$(shell git LDFLAGS += -X "github.com/pingcap/tidb/util/versioninfo.TiDBGitBranch=$(shell git rev-parse --abbrev-ref HEAD)" LDFLAGS += -X "github.com/pingcap/tidb/util/versioninfo.TiDBEdition=$(TIDB_EDITION)" +EXTENSION_FLAG = +ifeq ($(shell if [ -a extension/enterprise/.git ]; then echo "true"; fi),true) + EXTENSION_FLAG += -X "github.com/pingcap/tidb/util/versioninfo.TiDBEnterpriseExtensionGitHash=$(shell cd extension/enterprise && git rev-parse HEAD)" +endif + TEST_LDFLAGS = -X "github.com/pingcap/tidb/config.checkBeforeDropLDFlag=1" COVERAGE_SERVER_LDFLAGS = -X "github.com/pingcap/tidb/tidb-server.isCoverageServer=1" diff --git a/build/print-workspace-status.sh b/build/print-workspace-status.sh index dd2bd490812c7..f1909b8697828 100755 --- a/build/print-workspace-status.sh +++ b/build/print-workspace-status.sh @@ -27,6 +27,10 @@ TiDB_BUILD_UTCTIME=$(date -u '+%Y-%m-%d %H:%M:%S') TIDB_GIT_HASH=$(git rev-parse HEAD) TIDB_GIT_BRANCH=$(git rev-parse --abbrev-ref HEAD) TIDB_EDITION=${TIDB_EDITION:-Community} +TIDB_ENTERPRISE_EXTENSION_GIT_HASH="" +if [ -a "extension/enterprise/.git" ]; then + TIDB_ENTERPRISE_EXTENSION_GIT_HASH=$(cd extension/enterprise && git rev-parse HEAD) +fi cat < Date: Thu, 20 Apr 2023 14:55:19 +0800 Subject: [PATCH 9/9] planner: Don't use left table as build table for semi/antiSemi when useBCJ is true (#43235) close pingcap/tidb#43226 --- .../core/casetest/testdata/plan_suite_in.json | 4 ++ .../casetest/testdata/plan_suite_out.json | 38 +++++++++++++++++-- planner/core/exhaust_physical_plans.go | 2 +- 3 files changed, 40 insertions(+), 4 deletions(-) diff --git a/planner/core/casetest/testdata/plan_suite_in.json b/planner/core/casetest/testdata/plan_suite_in.json index ccddea0fd76f0..4bcbb0c64fbee 100644 --- a/planner/core/casetest/testdata/plan_suite_in.json +++ b/planner/core/casetest/testdata/plan_suite_in.json @@ -159,6 +159,10 @@ "name": "TestMPPRightSemiJoin", "cases": [ "set @@session.tidb_allow_mpp=true", + "explain select * from t1 where exists (select * from t2 where t1.a=t2.b)", + "set @@session.tidb_prefer_broadcast_join_by_exchange_data_size=0", + "set @@session.tidb_broadcast_join_threshold_size=0", + "set @@session.tidb_broadcast_join_threshold_count=0", "explain select * from t1 where exists (select * from t2 where t1.a=t2.b)" ] }, diff --git a/planner/core/casetest/testdata/plan_suite_out.json b/planner/core/casetest/testdata/plan_suite_out.json index 0aa44d94f545b..56a5973129503 100644 --- a/planner/core/casetest/testdata/plan_suite_out.json +++ b/planner/core/casetest/testdata/plan_suite_out.json @@ -1586,12 +1586,44 @@ "TableReader_34 0.80 root MppVersion: 1, data:ExchangeSender_33", "└─ExchangeSender_33 0.80 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin_32 0.80 mpp[tiflash] semi join, equal:[eq(test.t1.a, test.t2.b)]", + " ├─ExchangeReceiver_17(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender_16 8.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection_15 8.00 mpp[tiflash] not(isnull(test.t2.b))", + " │ └─TableFullScan_14 8.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false", + " └─Selection_13(Probe) 1.00 mpp[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_12 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false" + ], + "Warn": null + }, + { + "SQL": "set @@session.tidb_prefer_broadcast_join_by_exchange_data_size=0", + "Plan": null, + "Warn": null + }, + { + "SQL": "set @@session.tidb_broadcast_join_threshold_size=0", + "Plan": null, + "Warn": null + }, + { + "SQL": "set @@session.tidb_broadcast_join_threshold_count=0", + "Plan": null, + "Warn": null + }, + { + "SQL": "explain select * from t1 where exists (select * from t2 where t1.a=t2.b)", + "Plan": [ + "TableReader_36 0.80 root MppVersion: 1, data:ExchangeSender_35", + "└─ExchangeSender_35 0.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_34 0.80 mpp[tiflash] semi join, equal:[eq(test.t1.a, test.t2.b)]", " ├─ExchangeReceiver_15(Build) 1.00 mpp[tiflash] ", - " │ └─ExchangeSender_14 1.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─ExchangeSender_14 1.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.a, collate: binary]", " │ └─Selection_13 1.00 mpp[tiflash] not(isnull(test.t1.a))", " │ └─TableFullScan_12 1.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false", - " └─Selection_17(Probe) 8.00 mpp[tiflash] not(isnull(test.t2.b))", - " └─TableFullScan_16 8.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" + " └─ExchangeReceiver_19(Probe) 8.00 mpp[tiflash] ", + " └─ExchangeSender_18 8.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t2.b, collate: binary]", + " └─Selection_17 8.00 mpp[tiflash] not(isnull(test.t2.b))", + " └─TableFullScan_16 8.00 mpp[tiflash] table:t2 pushed down filter:empty, keep order:false" ], "Warn": null } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 31317b16bfb3f..c3fb82fde886e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2398,7 +2398,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC preferredBuildIndex = 1 } } else if p.JoinType.IsSemiJoin() { - if !p.isNAAJ() && len(p.EqualConditions) > 0 && (p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin) { + if !useBCJ && !p.isNAAJ() && len(p.EqualConditions) > 0 && (p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin) { // TiFlash only supports Non-null_aware non-cross semi/anti_semi join to use both sides as build side preferredBuildIndex = 1 // MPPOuterJoinFixedBuildSide default value is false

2@H-}7Xq)Kqj3_W0P*I%(V z&V1HFWblcA|2H3-t{8!0W4Orul?a&V9iIl-10Sbb#$`Tl$*L+d@2J_sFEw4(U+l*6 zpG;hHeW*%U#s_Y8EG7x&UHA^Ahc2}ju;_|Qx%cz*)lPj;qE=r3Y} z$mt{2Ec~Kw+JjPOR^v6@sepmM!92e9x^wQyOg$m^HIZ_dW`uL*``;G-?gEI~D}DYR&5XeHBKJPFf&d6r4Op&}K>fF)VSR zY;{=&mxL$(DNeI>IxH@*z~mJ1E_;i1+a-w%SrEg%I(Utz;R7^@lt>Y*q@9MI>=LJoR0+2zLh(XdT3FV1Ig zpq+^*)buV=wz2J5AP36Ve0llN1+FKgUn!>jlB_Q5SE=o3qW7A)&tkZ8V+^nTfTdwy z5BVTsZo4JQ$tV3ZUq2C@g`wyHSSc$$LNwwAibXe)j&HHNnR&`Z0Bg$J#ut;sJ0ND$ z7BeQ8dF!=M>RU00SavJJW^ms9!Ov9$NyDgZZ|w$Te6$TZw$*K59a2Ku`Me$E5&n&Y zuun`1m~?W`X43BzUom@jIFo$3RhEX=S#7i(Mbvy8HzAiVA7fCK?ur>dJrPcwp<&3f zID*F$xykW;=NUt_A_?;R67!IJR{75Z^X2?RhkKVZq9f;0F{gaOOZ}dIQ)j8H+hKQy zi+N3{AkcoAdgyTWaM0`rvuQQ2F^sx%-GhIY-{5K8ce*`Ot?9frtU=W=Z!`*@pD$DQ zd6R%f`Ww~+;*=>XeK5~Dr!|vJQ!RUk*3^`l9tE$>1dBsBHNp^7SO>N(Ld8FRApEz`v%cf%U}&b+-;e5k|H~JS2DT`>nYrY+og`zd__# z9;34t(h5K0j4}>ax%e#oe&a(MsZW@XMp!AxsObm@ZfUV&5!DBG%6^tGKTamgFKVi- zB?hvpvc4CC;iUa@fiuS^S9)nEI$^9QbMB>gl1)R?xJ3)wtZm$)6jX+dz-H#|d-%R8 zFscBL7dp4C&6Gdm=ZTWNv?^CU6Ix+6dizho{&ny0b&zl)U3f}WL%Sij_r>+9D)}Ps`5X57irQl{} z#>2l|a>99~!Z0O?4mBD9d7r8Ta&WZkeBiW^@1+*EwI)256ut^|MW}zXO~Nk_`TB6@ z)Mqcn!yZd{c6X$uyDF@M%59?tBmo;B3A_t1=JdA$5lT<6G;&a>lPHZ)bf`XyuZwt| z%X#q^vbWwhrwQ(U1r6^%yOnCT$eR$i>P9fas4VrBUQR~D+%chMh8|H(t=*7+WG$G? zM7+navV>@#codX1kunnwR8%nn9~c8+jtPW$qPkr^_FgYv>xf*}WU#3))AS*fCX-PX z?yK=BZFZhy>KWF$y&IAnyelX4ZY;`rHOg`Zafs7ij2GwfiikbD(AUQe=np47&&0o8 zs)TQ9+fo*fGqn`vz1qD_mG$(VT~1GAU3}C8n**bAepJ||>&^%1_3z;CKerc+=X}7m zxLO@AOZo*hkFUz42u}wYxK15~&O|bj3OzV6LBxJqY96`-n8xEM@YvMVspcp-$Ifx^ zCP~IJ`^!uvA~U!Rk|Nr7Jyg1B>Tk-U$Fuf|rua=RGpDIFg9x!0$atE{WA?T!$7{{3 ziKV}n7&2p3>9zR7&hFDL@sn#1Y9!L@Kpd*m8#)fX4w+71lRUvgulNK!7g3v%OY5$0 z@|4;Ar3_pjiYXF_LLrw3Sz_EM7ZfBWMllo@jKxdIsjbgdy(n{)ZzJJ`J-*?FlM~&I zcqO;q*pd9fnow}AWa8IbZ;H)gg*2~8>daOwNvBa#t+qo11QbS#eOSj&=Ol|&F=l`&Iep`;yk1L zHt~H34o0sm=+0_H%LQ7;aqF7RhPEanv9N8oIO+ywgTIthADhPePm@X%%kE9q&r*J{ z{Co=1V#%V@98P)cVw*SQqoBqOTwpSa?Cb@ewECv!-4DA!LZ0ZHFjxpXIh^P&Iz=C zRWoLgy_BF52IiSxA8>lygEQN+=g}Tf559Q%JhJD@3w;kC$!>~wo|xcy@laaY-Z5UuJQ5t{yOn}fzbs(+{0E4Ohy zu4I&1-8-C7t~fMaIfpUP%LEqhK{7ZsAA9WTr$}&JBFbQ~7rD#wr$j>n?_vs;e#iVi z%^Vg?*CJzAzaFx5tEs(TC4}7+ZuLr(RY&Y6%wWq*q85)*cewMKhpO z%;J74BsEHkCLk-^LvjI`s*C4?&in4~1Od{Gr(MiybV;H=Th(GY+}QKw5+F-dC?wpQ zVVQ*Wx9%h6-EW>SY~cj@F4%a^S+=7}Tf{%?2cmwZdDY8!3x3~SM(7+93aguZ-Kj#U zFv<^QdCK>jL$UlWeJzsloT#fEuXk%H?l+qlHw3%|sm&5j89eQXDW6KJ2rXlvB?{UE zGx&Zd(g)F}Op{s0i%O=FGk1DFf1^?`*xbKF!RPXY5aO7D@wc9*kIBIMdq@0~dFjg- zHM*Y41n0Or5{bXaLTVar;(mFw6(lYA=)@FGU~gzxyi-0)dkM4! z|F_cXR+*f9Drw?)&y-wS&%|c(q-_d~fV3}6a1+s6pnIrdiW~(Fkg`pBG!BWW8Pu7- zu`OudzBo_0=K3{btZg{I4fHwxDM<^UtJ(>u)CEzn`F9Jdpm^b7w&6Bfdj%JF;CUJ-j$ zb|}Rb6RAXcO^~HZf#{TISRAphcl+xd>bS7**mXh#xV)prW-5<&NtMaQ8B1V7pXKtTWy%#)i zRIB_-FwHMctHfL$!UAO>;KHQ+3+|#{v6ulBL!mB_6{cE#fDm8{KVO%Yep z8G=>3euh9H<*({;)LE`oZXvf%YVWA}WxT;ou3o^sqxX>(P% zcML5H%_~6d|NGn?GTt-qAEE!E_V`c`%lVmCB@5x)jo>ZaJaC?Yd&|Y_RUO?`3gq9B z67Mi6LjO|97v1cWLfuU^APtoTwzdBJ3D>hw=Trxt_B4KS%@W@kOS4`aPXs6mm}3_Hzz_emYAff z4fUaYH^mKnQRT>nj83bJ4{R+G3QsH{lSP@l94$lP^>jyWYkJ%mV{q|c;=feSlZPr9 zWYmwzyQK1I2>fDzW`{rMYf-2Umz z7oQ6|+p}`iOM&cX6)-zONWx7WgpOmoC7&GC1yKURE-LQm-#1>aNn` z{QYg9-NuA(t9DZ)kSrW^1$^$YWSTOWKW2M;YO|YF4O}nA*9gI-k&H%a(CgI-@PA8r z5&py+OMf>$&h;zJe@;Uo7{6zi|NgP&H;R_D--&Qna|{GAg_)||sx4>Wl2u;bQ1oAmb0y1V#c_Y0W3u&C8i#+2 zn(#Oc@nTF0_}z#1nx<}^_^5oy%h!;4oa)k`LuW|Gsxj&}3JjF2PHQ9pWei>>+kT8< zRwZTmbDvj+%0!6qL5(q5-kzLg?M#SZW44+bO!+aRSWL9*ueGb;32DoYL19h1pAB5C z8KrzAx5woT)J5`{s%&84+DN8eTCP(Y>`**S$K;wksov>(&hYEBo6YslE@2Me%3M_g z$tt!6!Yn6(4zUiNFQgLMgXRZr4_q}OG2t}{AdIC{6U#+cV6}oeC26$8N{yPA-ARRP zKN&5f_ta4NFVc@eWFtN~8W^=h^BgCt9{YBTJk99E#4El3>mL{Gc?H!YtUbMy|9!-f z;0cW5EShh1S7hp4fO+nO?J!fS1mo?A5=T0wMB!dSyCo`}I1GORGNc)Na-Qpy-Xnr0 zRui5BA}nG@Hg}3KWhjuhX_XA5h~KLi>GCI$r^vZYGEBFe0M}xS7GzdQ_jS$q;zZw!Z%)ApnDJ0s_^g~z zJ$#u|aNBk+_%RRcqKP@hH$Ec-lA$8Kewjtgz-0?3zD$V^8PEJY%IJ4juG!;)>3m#? zEr2Aqynij8q8Yv}nLi*zu#{sI1`f3$t;EH}{N_U?Zhik$?-L&J9ub z!^%Ot#0PT}Xo+dS>=i|pKuOFiE6|De=Wp`%EQdB1Z7eN1Rn z2uM+gQ9mwXRH;0;=%NU}CnlMi=C6S1lHYf&AD`Vs4@#*qek+6VwtC{1j1}T=?9&Hh zgKW7{&K=L?jc#r>(-XpHl#*q`PZfE?xFIgWD|Xn-zsNJ*3eO* zN8cKLFQj*LWw~WGGifb1I{@9ye#%T+$+CpuY!Am46W-d(|iu6^%f@_R=*;a&5N;+1V|u8 zmk&vrAFaU&ojLI>+`(4IV2B3!q}*^Go4Lw0XoA+;uhu!Da$nVlk6t;1|AY$|rWilvssPmZ&8?{^fdAzj)=Hal| z?KE?KoYA?t1Ic`H@Ssb`X^u!jv)Okg`E*Sr;ay-Jw=RKIJuw&ej4OgU3?w8(fFLV0 zbjzeIGK>tCgx+q2wDbF=#e8BT%gQ}&6b>Q8ZTia2x-<&c?TxM7p#$xZ-}Nm##tw3} zVy36Q3C+LQgv>ce{q%2!{{YzCN2DarC^Od^g0ceRAa|k3{2jS`et9a!qQ?V0bOm>e zq9`M&U7SiiZ>Vb@_=c}w#vL!YBn)eC z*ckUjlI$(ZFQ&|CyQu~mZv?h5dmBp3RH@)NYz|T74du9RUYW95tQw%U*l!kRV$T#s z?SEj&nJj-sDZfqF9K|M*T!Z`i*eLd>?gz|;^ey{B6LN)A`b7^`@HoQ6-OF;b^f2FL ziQTXS9G;`moM z5v;X8sp=V}2FbN-*JpsNWcF14f|$fd8mxkBFuyX#n1p&arY0|cG6!QU<~c2=q0w6>2xFP-W=!xOGX<%vCh1&l}=a?LQgQ%w4fYxL4+Z@d@sqEIG7CJ8!Kf@mEJvNe`>8O8cnXhy=vDXjIft>0Hdw zp(XgAAiQVjet{W7+?Hr^(WK#xtu zennWBKqci5TmUnci$eIX3uAc88mRbxTu-9dGrUNQ3(I*WNQ7@6Z~d4(CGGAi^+))= z!|HmRA)IffFn(b$U)F{mL>lF_XN~2X&r{b^xEnFZ`>|lryox4))ZNxWRqRAqmOmbg z5U?AFao_EgDR)0(-7Q&vupVXdOAgC7zE(^R?a|e)pxAglb89}9uV`jAm8L51)n}X7 zu3V?a-;3CV2A|uEJ3NX0H=nv2DDDaNwh*`eN!Y}Q6r(H&?)D_xbJ06|L@{|d4$Ukl z)BK&-sN&7(>bDK3RxxLH)(`;6aHDISaE7=MT_56we9MU?1~#D1Ziv5&&--bUVy_AC zEGY1z#Kc2Uu6`(*ZJY;;eYB#3Wjq~L1K}_o$~g!Lrb2VVsvjp`Ol~{T&Mf*gcw;3^ z$Y9I-JZc44ulTeZJ$aVgb`MG2FS=P0X|zH;ECHZ|JR3s;!@ z)$!*};Ag87T1^~$bh~m{E8dg3g4uz(<;czLslZ2@IV-U>v2U&f06DQf?CI@M<>y93 zh;$q1;$0G7gA(9RMq>Cbkc>}Ntq2#1(w{}#KJhS?+FZHv;#wG_r}_n(!dxl-dyeAu z^#$x}0=K#{WLzRToFK-vhgc;~7M%YI_rNyTI8?K^o{gEJYe?-&k&YM?Cg--~_w7P- zl0>X`Ts}IZGA}{k+hNv4LT*3%{1t###XoyDE~k!Ucx z+kP%V3&qmVI_Cw_1cz)|pr=}~P%}4|$2qvPepg`=Z@NwbdQ!E7leA}Ya-8URHX{UC zshZy}N8mx9x7wY#)^wYS9XB)XI3Q%=^Z&=yTL*R3cHzQ5X^`$lx?2#C?(S|ux?rxBf?(XjH{x&}6Ju~OyPlg$tz4yA;RcqZnFufVdb+L z%Rx9obW7J_iru2$*$@nPhn|YZml|-*u##4V+RI@yUre>`CiBz1hDNER(0?!6tfCc| zM6k?#)f#V)H|zGu9Zt6c!9(&xw$2Q)gOnR(8pc!8WweGZM(5LGNvt&KTwo&Yn^VHk z$qg68nmDRg`gV^WURwL7{n${X-d?zsn!lUgw_F-v)1~c)a{-9Et)$hT$^8=t>IdtG zKG62x{P5CWVLg`ttV~`Ia`XXNc{A($TV(>a9xi;?GlP|A+g`M~xU|o=!~$Ti24a2f ztf$Lxf=?8LF0bKj19#msOQc-lmvCpUi8XQ=;BUH(h_06y>yE^!yfI(#rs+mx?D2VA z8B*(2-_)J;#}I#5O!>GIoHRd&{3NBICP|f)ak?8txISHm|KfKKXOJ(M!R~vG%jX4l z%C?V~En6@>5wCXm#X6)zR7M1#O|C(Q2U82_Ox1ksC8aoM##lNLwQ=DTOF;&kVx7{4eL79q1(+ys`uFJLZLz({1rfj zr-RI={R8gTAO<~6u;x5CktSxhARd4m<=k}SoILx!Es4xM^?C{{Vw{&AK3%X}QH zfucSk_FtzVCWvL*=>mE$z~=G4FIt`+&({G%kz@YtpM3M0c8=z}Ckl5!$eih7#xexK zgc+`Q6e1(Q5*jgwE$@v<8T68IU+7Dim9q-DjU>H&!XzN<{SXS@WNx3u^XR!;=>fLc ztCSc{v2%dCwqm+q$MHt;LzIK1>*8P}!-XhU&coSmqIO8nV#{77JGcw%!x5>zl`pB$ z6no644vDg5?5>z*%0QOxgkkX@(L-x79ry$|$DDX657ujia!K}tz~Ss@4$&3T{AzkX z$MYQ3Q)eSg@dS=jDjja0G|;7ZcoK;vB1^2g2!)dfMN0-m_Ai&{Wfn^|#s|YuGMZ{J zBxL&m7yF~%L~q_kuT+kVWOFL~GN@QGj7gTJ4HC*UyjTW>0MA_SggJK#8P_S!4~cBS zo3lB}{i*BmMdzpQU%e=EDj0=OA6686y>>6ZSEb+G&PeW9AbM+FaOiAzB{yj`%6GXn zixVa?bEd>_AXy=FARw^i;o=JGdG{cMlJUNHib$ghJ|C_MOMW_I4j)47P_xOXZ;eV% zy4ba?nzg0V*--em5@C_XD8iDlf|8@rCh+u4IJ|3>ZrpCczKxMCup0Ng_J;A3i8%}; zEHx<9c|xm@i27&|GX>N85PR{D6M-7hUMiMU~&wHUXD%;jq1=?%>&p=l)T+8^j@+4!RC~yM?~N zlhDShzIn_f484szqfA4qHxQq0J{dMtak_i)opI7vmi37M1#eSdTX@hXt7))Mo19Gw z6FJ#3ECr(^bo|wx2}F=qGDJYOxiE?F`YATA>6i2Gg88;TQA5j}C}FojWMUzyzaaBKqZlCfTq;8EDM;3o{6-qq!wz`hh#E(tW+wNl% z&G>WaEOj+hdJ}n`hU3m3 zqQbHH!0DIVuiz)9c$`d$7QgJ+(5$v=>A~YNOE@aHm$BkTK%t2t^i)46j*KlNQNlzX zt4o-ja@wJNIXmj7)%k1+{OZuwael&gd%seP5+4g_8V1;aCc7_>SI&+yMeMq zW=#}sK0Yj2(KK-0rMni1dW~z6CR%EN*cn9Mz}*TS?3dLQ4oD?v(!SGSMv1^vHY5Zd zo`QUv>(hs#YDH_FmhFKA$lr(z4OCq=gW(C-9O;ZH29G`M_cbjn_8Ffz4$t1stFD-w zSlGY~k^)$Ra+S1R;j~LxTisp|AthZmjQ--y5(AnmapuZ`pD=8K%8Wgk@77_j;dpl# zABQoVGzlLXY^vBOp>7DOh>mOE^Y51LJM2G>yPVg~04+k55jQVc*=%@XWC+;*hq(a)zPDBTN)!btJQMg=wfm z%A(ba>d2F_MUZ#`bwCj)^p2i+OCOu-C|apL&M5zuQ?i{m*S#D{mr(90R?|#OWD5!P zpb&*1&&PqwP~a+&3_HqD9Qs)JV{3N-c^GxDDF9i>xao_RVfOa#y z{p%>kK5YKu~1SJ{EVIFmF72403Q41;?}Z8js9A0fOwXs4kb*Hgw1M4lsUmy(rpBH zc)dA5wy?L#m)Kbcf4bPerUvI3(Rc2!nvFc`gYYGFypKD?6xZnSl(9~<3y25l4i#=s zQ$LDWMez>pCxnFy6#Wo!;?SY0Z-}?UY3WJNMogg-?5$b`A-4n=xN)jON$l^VN@_@} z8@|_ksemon^7f{SFudm|SZR3V^4@s4p~LN?*Xq?-r$DOh3qw)wrHSxEt|T8Xjniyb z&?Uf8jG7QyH4P`FcL=&%V87LqabVK!1VWb76KhnX8?O+OlT{Y89qPYI{wbk$Y5@KP z{N)yKfLnKME01s?Fk3LcJgdttvf1@t{IQHEWi=T=5op}TSiQ0|?Euiev|c4<)i6F$ zj_WL4R*-^U5fms5`QZBbU@bo2R!(v60pib)+ASQ30Vo(DNFk<%c@!3bda9?(0_-(c5}!4g~PO-78sIh$iEaHVU1HEct_QXvv94H&*_-An;SM}K^Njo$QR-wx!)8665y3A`C4SI$|ASA*HuYDKa=BAz_@kW@Yiz1LpKm-f{f*!kaFCYSV&TDl zP*n4GLI~T>)o7n8mg-aIfNHLNU30}EEE*HCZH36L?BO<#d)fY5>69Uyk8Hy%4Nxgt zTZri&K%Z`I3W@v>aVX~(<^|>wS=>P?l~Rou*=V%_NE=%K=(gQI*0?`e394&(#_&@I zLJ`MWt;iv0lZIt(C=mTr@H5ExOJ6K~n*Ib9uOZ`nY$Z71`fM4@`6rO-QjEUq;&NB! zZybF|Zj%fGm%|PmP5nk#{_5>o2ryWX#zOz#EJn=1HT8hUF#^pxS8BA%c!=p{b)@=B zM#O`9=x+et7x{WuDUm+NE(H|JYCpnEu8;lDJqrniVur&`7mENiDdYvsQ3y58$bY>U!u?$7XY8S{-Z@)Esv*T zTvfl7t8UK=vcYIJ4{;sANvpx^x_|@~VRoB;jZiGDFX>sGceEVUT^n`D`~*5od@Dpp&||!F%Aj(**a5mj9DA19B68HeSITKhUrLp6OQEwp~~&| zV`eThyF;G2<*8O&7rIYi0Phx_o4sjTCd2QO)zU|?&zp_OBxtKN*)wM7s0HlpLYVqD zs28+r?0&}8Db`Kk2WX!_XTE>$%;#EEnD4BcgZjNQSK^h_hvl7ij|l#Nzr4%6*iVX-RB08bgfwFT<+4~&1jIB>HD z0t*ts9g|fA>5MXk#bT1;r7~;=_{cY5fY-0v`0W9Le8`Y+s(1>fbhR^6nq$M8hpEv< z%lJvNH;j%(wddQ<2;gcJ>bt*z15q`?`F!6m;W$OexW z-M^l{a~Gz^pNvLwIhtfCg*HU|^d*Y0dGz4;92z7+=9Er@P};CAkqw>HL1(Hr_!(m^ zhvqp@FoB)QRYpe3$h?9-<wVD;%H=J6`Dznzd4kRQqt5N7n0=*!Nh|Bo>?fhr&&EQkz}c9sENMHO z$xJ$Gt!~0TSv0-Nu-1&ABNp{-^=@q_zOTG#(7d{km}8WX3xWWzXs7KjV_YbVaoNV` zjLg}1bv~7na4})983)17^?m8Rv#mWwaiyHxgNTWVu#@gaN01kc(Ia+e(90b> zeQ5YNO*Na{m_FT0!ZWUrB;k(5mFMrE11czXo?N?2H51rPo=dOdv_Qe9e%+<{`Hcs( zq@pjS5*N~FPUS?DR?~lXML302?j!p4(yE!lZ9CE>_NY*mCCjRejGx~Sn}lpk>$?@{ zhZQ|HWA7RSNZ+}}oW(|+40A-@LdP6-^U(XInyKseXB}(uR2BxtYpPbfK9kl7$u|i! zihJ!}8+?-kZC0ES%LBX&Cue9bGEH>0`WbkI7q1gXAPJ^Rc!2>Xo7?KrBCMsQF?%-aOZ4C3K9W?_Da9z(s$H zk0dzxeL$sr{Qf*TX-we{Js;Kw6$=HRXIjaOuVBr44`vIMh8gGolfJx{tlR<^sL0C^n~G@Qe^mmf-2ceZ0lE-dV}G70I*GsCVt?^{fz&|8_de}$>u58b&1_?F%yU!OXb26p)s*X# z(Fpx!xXU8PVW|%tjx!sfp;tl$RB}vqPc^p)-umeaogCtO z@?h9vl4kL17qeAre52Rs*k%WUmljgC-I2iXz2nI^%vHhFmGu5oaC?$^zN?{gsRm1; zicZ4n$$HRVrFg!L;TYIRBE7Y~t)rfa36u!q-xenkKL!|uKdp=ZE72yh+<%J9Ntw1i zI(ayL7(Q|sA^CSiQAf}dy^AR(Xx~!cNwEO-Dul&ha)hNWFXzs?c_#b>WaRUwRB$OpVScVXQCDAX|sSWsLG4fJB;Ll6t zaPHkMR(vyc#WN_Qw!p3bU{DcrERO5c$a;=C^T6#68mXpn|H=?-au7p)D5duEC@}0Z zcN1o%_v5PVr?RCVEV`g0gIKVrp2H`eT)H8+-XCFH9XTf)(k)~%Y_4srx1!zbk0>$q zc7^hMdLF_1aq z-42{P0i?c*h7**JGKMM59kFr0BWtc` zKNLSjabZ92pm7S$H}87&0=3%`fU`^~rO@#8 zI^064Ar}VRhCJ^&%QnwKXS-QVucKGh+v3X#V^Al_&s=98SDOvx;PGfxq!LUV3mji& z>HkFY9tZT9Fsy>^xFV$S*vDZ$ehm9G#^cthTRS{SX7ERZ>cG_X^%jpzw)<6_=yDTh z5M$#hd{n0%cE((x40>pYy@L&V+^=Id0w}ztw3aA6!#x!6+&ZYQ03snMmu1zA#?LUO z?G=3gNT9`l5FmjExOcP7C_`kk4;2ZQ?$sUFAz1tzTYQ~7E17iPUAKN&hS|IUsQx&6 zp1#Gy+Y_$iIkyxJ?TM~_Zrb#7a6)(mxZa&?1hWb5*9Xz}YIL}OqqlpVMA(&SV2_Ew z-#XcsSb=!s9GoLa*aZMsR=@sSreWdKz1%MAM56$kyqcVa)Yxkrvpcc7LF8LON}K@(3n_PFZdP#7he(X`^Mqobp@1-X!2ibinH()lKF0}Ut>%{!#ckkw zXZ{%7mSX#SjPcGJjhOCIkPHRqu<7#r%~(=^S<7gY`ZrARJG;&bJje|^yVR~|G?AO( zDq6UVvm%dN0f%&xg4I4yLv*8<=(R$yEEp;n6< z!wIzEi=BH8Cdgkgc^EE36nzcPt-X;qh9~XMuW?*{7*bY89<6+QS=?uvGIE+&g^e8j z@w!h~X8T7}8=@NnxOyj~rKs=w@RF6H-P?dt;nvEZUzz2;0I%#aQhZqX6V%wK*`f5Y z*V*FXfAuwq)}d>xMCHBFIDinRnoU#y4JxFH%aI0SnqT*pjU#4JEQ$7jyZY(`R!Gmx z7egEJT|%(TdneREL5rq7QDPDsnRxD;WFOCG2kP*1)bxVxhjm)Y*QyV66svVT9K~=& zS_o6BSLQknwe$E7GAqJQ;`&nVi;)f$wTJW{N@H=3kO7s+o$oFa(`C9>gzv3bp|1OH za-v|NpbIkKn48rCR1*l%?hbB+5srX^=?k2 z;NWc~3O6^}pNNuqM8b<2<)I@Yoo_dze{`>D+hfpeLBl9%uA%>=7H|(sUVn zej$1DV~&ndkI6ODW7WH6E1wjxSimzO;3Cmp4CZ)~{uKok}b zj%Q4j$i#kYma2!pXI+qO`60BCO9s!IuaG#ikC;b{q<+{<_eO`kcgK6wb-}KyH!}B$ z`rxd;msmZL?Ck(T#OKKwfi$kkez1*{nSia;j!j(G6xnk^sM3dWrxX;hg#d>2!McB{ zwBoAv2ks~>koS~POadbLS&HrV5$?}wl=2H)EA_&Sdx*q(X@7=l1N2W}iNz<#PWAH)mWyN02%U)?)Z5SVGypH#HdhF&$DJhoEUJM~FDL%+2W1HkX`K%<_1y*$BFY(AKQVl$VJ zF_8m~FMuVsIKBx$bj7IbJ2<|Dl;ao)z zN)Z(OgnR@jyl^AGC{@e)R$d0AfCTPMfzdoD4q3ni9;d)$ZJ5a0)J z>BJ_vzed_Rhkdtnt!ck5Rp6d9_F*Q6J;|h>g}RC6 zG?fy?(`I(Zn?hk|5`11RQ6)*Q-|UOkslW7u^C|O<^uRWXUQ#w^-}N`TAHxI%g|jyX{MxYX5g>Oua#?gRmAVbqHXPh-$y_ z=*1&t8}bbVfcS<_+`C7M*btL=xLNoS0&U`=**i`j;j>#vE~nHF(dGMr6Vz$2{8r>2 zLTMi@7kg{X6gNfAb*DmQvrzzg#zDPb{!<6)+B#yukg7jf#wD1mwSIw{sa&O}42MCl zZRET8|M7F;%F8GH&4!r1Pvh^}_21D4XqmO)v+h=zV}#g>^?X%bsC8q(eI<4dBDMm) zy`MH~?W6*lw+um&&DT|@#9_KcgJpKN+*%WJz^xPOyRNV6**s|Y&^Knl2~?;M?dbpe zs?wqzztJhM?GHs+wnK%TdTlz~S~cv#Z`V@#R!F$_-uSZg6(UQk5W4E4_UZ23e73u* z%?Um76)Z&jCuFqP!a}L_HPm%%Rdh99A{F8e8dj#d4-X2bFVV4$R4GG>f(?@3WZ`F^ zPF)tr*V4j2U847nf{u4?WU;n-sruAd+%zr~eQ0B+C%GZ+15GEj7~E>dDqqWQ7M zu#%EYJH#U>Il(Y0v+)AFQ_O~JJh>^DY{Kl-qf)kXsf(40v^2(JfZOub`{>y@Z>4f$ zSj=#=l6ZeyzF`CVN?s*&3zMNc`Qc+SFx^1Psw^Fh+(RQ^mQ6 z#&w)w#7_Cu=Sdpxcg1ILpJ~VgF_1E3PbDjH?^^=Z zlk1X&QF3sy(QXOn^ZNamXes!MgQTAt1wg5M-jBh#P4}BFJWo-e7l-c~JY%)93sUU4 zqE^Lf-w1L%b6&p1IO#mCSG>u5Q+ScvTl4P7k)Q1uHl$H5`bmEDoN_Z#HvhK`zFhme z#phUh6T)Mj(Zn=S_5t>i@1v>UPjV~%VPTGD)~&zPG^0LoRG${tu0F}FRT{K>TfsQ9T|KIpc9-gh0Q@ymV)540?c6`W(J`xO(i-DZ#R>fUt({204HR;*rM?AxUH{k9rrc=E{QHH(hj9ff)SnIh zt1?eYxvu3KyyPzj);@e*GDF_&DU{I)txxyN+HZwpmH>H3$uUrTemy{w9sXPzv~#(< zViQ)k)xu27F6J(nGHj1kQH$c4%+4SgA|7Ku6C1IRhfUlIm+#MzaLV-f&C;ux4980U zH@i;;OMSnh6-1wFo%!_ZLGy63HE0*_IRegIB1W7#tno8wwR`0hDFoltPs$`|d-*n4aI_O&ZhD5e=Z#h&x;$AJn$X zWH!Oyv~3gWq0`S4`?-vnKVB@e(82UT@66I9eE%zN+a_W7f7~Qo$-sJne0U*gM zYzrre1bg$5OIK{RTm|qzIeto;BHZ?2c^cLK*(}_; z?+-Lq`T6f31(;z4f)!8GL(heSXC6t*`#urd zbhcavuSJhJE~ZCKVlH+peso?aB_&%e#SV8J|4IQut{$PskPpQ&BPqb}`qwH-v7R45 zwhzi%Lc*ETau&=*3kG^ZLE|PC5nIbM)bb+8E|g?;3${n31oKxigq^aF zX)#|-Fd5QEiadY}VtmP{tmSHlCxchT@EpNxCTt|%YI;4*fy?$iBsPy7$`t0M6TvQd zJrdsh2gAULm%5uBz@-#VNXYO%`wP)uIqq$DgpLtB8+TVu*#3M1U)rH!Ao|mX0gRbA zq!<%JHh$iTrZ+`-$L`Z+wzAWa0tf@aGV6o9qCxusXx_%*8ex2qOML#9efgq&$~FI| zyK^z}{5O@5pKL)>^tT+K5hIFx2pY*>EDcl{?+p&@gZ!Pe&;=Be8gZpr_zx1>WHOon z3#>oy>5s*yd*8u&EEPN1yKGd$wEmu0G+x!zN z*Sq$)4pbB2zrX-S_53O}F0acYJa@Cg7BzblkT$kAF7R;U1^z^pQ490y?(k-saK2N* zErIPN$&#n?P_tvD3;GQE&if(7we&!{)Jo2<+}Kqsaov=6vm`0WZWc(}u!HM(;7$$` zxx11-?%lC5Uu!yYw%YQ4<(1krN-Vs*4y9iC!`#0lJ)ahVFmCq6wV#_xAF__W$WVI3 zG&u>jQKmP`hJVpR5x8FE&j|Gx{~m6Z`jEQL#laN9T^L$wE%dWr$CqQMS012f;mxPj6O0N@M-K=31yhO{#m-CvgVddot-tTtU~I1K$M4ats`9 zFTGa(P%dDt?&^$K&J8$I$O49fN5chOoHo zW_kM##s~m1KP0z18HBWc({$Yb&<)&L39)Qw#p83wO(~fUfpywiSG>giG}&3gMlqs2 zxU)9t%A3p)+J@4#iZ&NRA9MrPjK{G1ybx!q*#LHht7f?zR*fOO2YYj{2-Z?xjYZ03 zzrj0c^0((Wh!X?pk!gdc}Aw0v*UADN=}4&K%S5D-MSTMZs{i zxitC;>rk!*Rt=pe3GrFfM=}q$Z*N{wS7yYICJnE@@(ZFo4Y;e&9L3hh3>i#T299Fz*(!+Y+t~v{=*T@gO);?YdTl zMc5PlL_wyY@SC)_kdA!k1HM{gZ+!AK$Y**4u1_-Pg9ysl0gP$J$lU#i$4MYyEiWD8A{JZ(}h zXi*Y=5iC1r^bMG<)WpJP4hi*I{?$w|o>{-OKt-VA)}%?!I-3Ik&2l|>wD+Ee|G@$X z_XG?^?_;lPoYAi5Vy%^N*fJUPR*l`4=UKW-lW>^d{nAaKN#?WXP$V;wL;k)Z2Uz}s zp-Joarf<5xBvN8A%xB!NY?g&ncIlH(w9l_ZUhxYqxAM}q$T@B;SMik|Y0KWaTk&~% zDPExTwLZ-}FwyI>S?KRT6g&K3yp^{7QE+v4FogAs!A9Li+^Yks-X>^*r!X_js>l7< zg>IoW#mxTS$dd*&_h)o53c1hYa~=+NOU5Vn*3!WvW_IP^`av-=he$rfaDliJ4;UaF z&_G>hjuThG;)uK|8wyRnzj)+Gc+XI%XkXhlji@E~r@7RGVidfV)?5jtuTYU& zFn%|CgYZ(*Qg?C{tZemNEHt8AVGs;okqa3@eo%lr6gv=|YT9@&%BQ5ii1~dik6Pf_ zM94xcv>3QY*c$P2q2uO_y@(iTHA#04XJ$Y3bhNLpDw@Ag2XRy#7}EIvD^1a`e;siv zJ>RSFv}-jWeUJO^!LGy@;S(;a5n*cW;V)B73uJuCXW;#ge7RL(ICBFi;zZjR@@Ml! zOjtB>)ml+4cz_wsEWdRw?^vFR?`k5AIionj@ zb%+eKUW<28loC)D;n6x88O3P3Y76`l(1NwOQx~vRZcxhkwvpT-z-J_s*Y)LgSk>|B zDzYAf*k?{I=iSza0ZQ||7xH)Nq6B|>FUc2B_FZ3Ag|>HN<2+E3es`vHI=#0Bzae># zS8#eljk1oI_ck~@j=5>vf9$d}Ic%*J>crVMERYqT;qL*NA0*LSRN}|6i5V(YfBQ)* z%AvVq{ub+NJCiuIJETu4l5GX$qR^M9rT=A`} zIeQ|f!>49Z>+6>*IKAHH?I+15o=#-&h0H`Og=i~q%lIxym!P4e~juJv>3Po8$70%l-5fMQTx`7#_kTn}3LAgA z&2R}iteVzjVt-p}OC&yJR33i$eazJa+afFf&Dj%Y+5o<~U~HhyTG8wA5T&mi8{JC# z@Vgmvv!QaomDWg!qh-n)etH{W2A_pxM^=Zc@5B(4u$IE4;!fp-)ScNRr2X!# z7T(Bw6CP#NkE9>{>2?_=#3HfE%ZTH9$-HBVEKN@yWU~5gcTm=Qy`X=iY5LGGra%ZG z^Cqv?nHGT%ED0MKscxISN7$EiL*uhvdoY7*k*6PO5J=!1DCV=~n};v@{blty%XUJC{wl?#rcZ$a8j_8S(1&p7{B6%!L_u ztj(STp=y2;@dT879a<9adO`6-4#>^EkT5wtzRj7IBo5~*|B0%Lt-k(w*_vo3Uc2+{ zovAq_lq$g@^%N)Nob+B219$4UUwu)PL@pP*u!LL(X(RFl#a%dDL!5r zQwI;5)sP4kO9vCsvS+q=`BGL8Zua?=xSTInGJl_zwho}lGeUU4_jM*()BoE|Slym4 z6Z2t4YFczL9w(Z@m5WR&o+MYDVFUg07DrZ(FA_}+ad1;LKlv-J8u4gG`+MB08^zE; z^O~@&F$lIKDD_z}mo<*f6VX2)U}NS93&yKkEEtfD0;f5S|u)W~Yg zKqPF1Xta*s>+;b@5`10PfJpa>Y=Fh*X)F`2(ubh2hO|G3t}>c|VL$7_@??+SHB79#T zz2(o3U^fdVJmOiAd=p8oLfGhInMi^~gAaA(_demYvd}j<|0jI_|lUAhNPAHph`{QhoL#mK6~F9glT??qg=ln%hQg zkWw4Icut6eu2At^n}Z5u;!qiV@9aJd@%~QIzj4G9;ZnQh6z#-N33BnFh@nn(H>?KTfzPPUSe zus>In{6YdG7BPm?Kab!;xeIy1l^a$!j4``5X@Nt|uQBOq9x}1fS^D*F%M6yh_-%q_ zxA09&*$<{|O{UhDUGkdK`~LO;#kNEbV{5$MI@9-IwOnh^`Oo?kI{NEV)mlD@N$0}3 z-hq)OiDR|N3o^9ZKEY#JMUiCSTGxx`Zt}hwyw9s5`=N+myy7ahPEm%d5wg%hks4&? z_XG2oIZ0nFFpqT_K{%7=d>@B$7=g6s-*aSqDRkT3DisgNg=sG^LLTqPWx{FkFc~0D zxyiT(a+q|tl=hZU$-e*yW z8Qk0l=mvZ&dcxwDma<*Pp853C-Sj3e{GpdUrn4kwk!?}+d zmmu;md89wLrPkRVEMFo4w$0dQot3~5CaY7Lyyov=merv=9&1Vi9%t zHUWwI^vE7&FZaV%xnr%{_n#Z{e=xz%H)M6*DmY1u*OW?qK*U(JR;kT|7Wf)NuH^Q6 z&1&`37vgU|+L1i4xWw4ffUAqyrYf+}G3dpB`uYT?C(>%^G-q0a>7W85)kP8L21HityY~2y^E@KZ#WKhyBCo!kY@YJhg;KNaL1jF$O1DeHp*T6=Q{XspU%MY zmCHKwoN7saY+h@Q5v6e5Sv~u3WtO=#qzWcWh3hq_ z!&Q#TUeiXOGdGg`SMuT2&(w?0Y*KSVZI=p3nLzOzCFfBnC^)(lqcW0jeH|0MB%@9l z9SUMRW47-7G_Xxw_8~^N7b0K&eFg^t%njzdiyaJX`%7siJrL?bf<~$mezCkYA$IQz z2ezzYC{k=Ox)9glz611tpmp^&%Z1>`)LC@*|IucR20U?Tzd=-iDY|43jdjs zOX6GTogi80*|SJbLb^pFGm_%(?mZX2j|6bOq#q3?9D$aA`vpkpvB5?*WCqbJSz_d+ zN36RXs(;dSS5^BN{47tBrCO!{O+KW|UlnWpH(_q3;Oy%*^_i<~qno?1v!n$aQcOhB zdZY6k=JkH37t&YT9^oIy44%*yGSUSRM1hd33vj;iP9^>-X-L?PHn+9R$(%;$SE#d= z2;m(4_2^l!f0(`dASkc^D+`t)PjUh#5@Z7P(~U@-9uUT96$Rz~4Z8ggMWaCAm-%hA z_ImBFCY_K=v{>q*X5|qkJFSR*dq^eFKIMa73>Z$CkIf&j1>O{@LNtuuC|#PWoai6E zVirbbpgfXFmjfet`k1zMfVm5&A}(7Qy8Yw?XMks2j59~V;AjO_(y7Yk0SM zJV{SFX+@nFhr7_7hJF zoIHr)Q~2ym-S%b$Y_fqBlgZ_B-Xcppjg_tZ{i}%(0h-u>Yinh_2yL6L1D4_|3-Cmz z`emu!)dH}eFc8AfVng^4fQ13r8CvHr>P;OQ~aO1e2wSyCy7gx8DRlk#0GT7v{?C9)_VsFkBVo2*Kg ze)w^GbN^7Ly7C2#hw5ra>(A+7yLu>0NN2q53foYvuRbCN!rkpqQwnpNnu`XkW z)>v^?V`AA{@w_SFS7eAq1;1$2*f2-x@nTzzncQgvs8<=3@P1?-@!4NDZ6^Sn4>LMf z*l+JcIa|rpSL47Ox=7HE#l{?UCR#T;8EV5F8kIlQ2p>Xsa|9eb`V9KM=l)e!4L(9d zk8zRLga3=AdR_3sTg?|;5)J!X6K&%^_@UhHz_bW{=U`zdV&ow+I_Cf3m+wYDU_`@# zDL-y|3%u&`fqEt!x%8S|ZSAV8<~TW#=blbPYHo!ZnRrnvVKA2OX;|%L^eG2{sSe`( zJ{@D5STfmQ=O`pQNGbyWIK_4tlG6@^SLD%@Yjfb1rBIi8)h#U zy!2f+wdsKCCVHp&gec>UrlPKLRu>(hfAO3ID5RfZ2raS;q`IxV)WjTKFMFjbFp&kb zmoNQD_#BS7SDv0#d-t$PhoWy@R!f-M!*pwoK{Jg&lhHBjjqvk~u%5%Y7qz#_c>`=V&w#lty^i~QyLaq#xK9`v zUxv8n&RNEV15pm2hFJ0TFJ%V^x5XMkmkr`Zzt%u zxyayMt&{2q`K7WLbBNF5!pQaPQ02kli^Rq)kG zTpF6wCF2mim+=ZW+TvtFp>;eigqDLcZ5=HbrQwny7F4`M%KVmRDFpN0)|6MN2n2LW z*(8SvHoo6Ol$f5ZPh5AzJ^eJp1!k`~$zQx)c*#f@jH&+Pct+b0MPQb~h&KlVt9HrH zIb+)wB|)|iqnsYkmjak~={Q%}!n_T^-ixQ<9~)eM06p!s|p6u$&&n15A^I(EWqQn14F%fgPdMdck#k%Uy<#80o8m@DgwQP+~Neu&CW8yA_+%eP!Oq;{fn$}A7=gc zQJ7krUQvcxbp8t64x!ZJndITGpSzg3(Z9&vW0zm00nu@=g5HnhKYd<4@s}}M|7}!4 z^l6qhb3FRxGzCp2Jsu>N+H2z0-V5TwR>-@IrTrk!PPWoB?vB24x&z|>g5B5W1reQa zxhdT!0M9+BDQO@arCDzS$uBMH7DbB5rs7~`-OX;ai(L|EVE&Rs%G?3tBe}1X8%PUk zmiWc!Y?#2G2x9819)nnqL3t~(qIJ%21ROgJW{6TZrxBjlG-4Dzvk;CF&)+@9rly2?n!) z^XbRG?4eWQcc=i9%54->E}v%PXd9Ve{2Sj-`Q zABg^GLS5`=$kjEmFsi3>mO1ZyNjBAFs=EX()gNv=S@%m(L5@+o!=kye2%x#nSlvQG zcHb|B!ShmWPQ8|dJ?2{FYt-|S!Ftim!3#frit70k+M=KxqB(v7#*d|0pp{iUOqegE zl1^nTgeAQIY$Rm~rR9kV&sq%+*HF-oG9X@i7#=cL5;@%DI{M9#Z2cTI>xBf|lda4a zp>DSw$4N%)69}RKD`;S*P$gkl|bC_aVwPV5{V}XG~X9% z?zxOz{hVru=A_jb(v+iY9OB$0_*=_7>hW72dM{!aZLK1l~gdA78K9HJF8GMCUO+_FM3? zDNWTdUZ-WGqY_vWGfCkr`iLG$Y&4D0s_GT6oh8QB@`FBGzZZ^DJ{gB(BMUl!=g>O4 z-?NAMI1sxxdTJVZ6Y6a37UGuE<#e4jUbXku4_+@Ql$nHk6D<53RVpUL9Fv$R-^xUP zGc?|9!`?;ijHW`W)m!cons5N|usq}qVh*=|;PbJhFRR}Ncnn0ROJTJ|ikEaEb=Lop zGTOCbwo#q9VjQ%W!v8`zH!7Zi)3}9`?clPyt=}iZf%5PVmLk4hv*BQmIHWrmaChz~ zBWWKB_|+Sa_92_de_VVM6JsM(LJ;ZU2WF8>R*_2@r(Mn=H9B9ucL-@%R$GHF(fdT% z{6Y&omsB9I2p9g%`Yh7o<$pH*>R8KXhA@|oU@+m z!wV`3glVP62D~)3*-Ekx`YJ~8w2#TvZQ<@KwAVwxpU@aYy=j776x%!Jlu zs>iN#lzOY(mlr(NjM5iyG0c`}qJQx#XZpzxp+?fYs5zR&`Qb6c00odA1SE9#)`)V{ zE15N`itqmU^30n54~;?xn>}8olCYfS!q`3&Nm!gb;!0_W|FYSW-}2!JjXVR+;rmeV z`q0v`4YTpJ>A;}X{=6Z4Qt!=BU!DZ%V3si!5JAMG+6l4Kb_w!~?$cW_xfG59t{LYJXhun2c7Mf6m=ogF9`xHj2CBj6$=DMri5|5Dd ztB0AH{{MzvhMBdu9n_HYj_Prk5NgJeVUh(u6`LK@Q&)($gCiD2AYIs9 zpiF@^AJ%QJ!plAJ&UC`mV57peWbp4JN~pTiF6nxwCkqkqX4B-+3tzkJ);|*sYu6;g zrUSdd4C9@hDRijG34IOM*vpvPof&%_sBugIeq|DbNFt)RsjMysNJ8%o;kEfDIb_wG zZ*P|?|5T@WV_d@=$^_UMG=VPY;r;e6@AAB)A3fyY+(8d~>d^V)n81JanCRy)utB*| z0;x}VeVDi>bM>FJJ=AeHp8Xk>oQa6|FEM*Byeu%1@HzP14@bejYR2r2^ivj^Zr&V* z5TlP3{)Nk0`=wl`inl$mC56UO^T*|Bl(AI{i2zt}TCptf^)93lizm_rG)r)b0gQTo zWfkq|xxwCT+vV`TS))r~@60i~pMoM)Mp*~W-=*?|{iR8atAqbPw%$6btL^&&g^wU5 zARt}RNQZO^5+dCl(p?hLjdV##htl2MAqYr=bax{iZ==`yy}$R~Uk(MwIA@==*P3&# z0;OE(K^)hl)#L+zbe3kWBKS+RnLJ(!ETmPzrR4P`bY{zi^@K3 zGqm<-uLhpN0o2v{^P<~X3&`h`FR&dIib&KL-+xPJ`Vrs%JZK`d^d zo{spXgi!{Jsyqw91`!-G0}S z)qO6uA-?xg5!Qpqs~T}*93_ipVGJvIZ|P=(TH+b`Zhu~KQh-8WPeAC!%sXvkFd1d= zbcP*P@X;V2*)c7}o8b*#kC!}eLhmWc@gL=wW-$B?Uvv@R7=;r^+09hydn!CiAO8Rw zY4+E{U|tj^9qNBrE#mR50Jo^g;qK0a!vcmQl4s+)XjS>H|8h2EsgsX>2?9o$O%^c* zyIGLTL#g*VYMKCw*-pRBRAbxHMr=D}XHl6vsU=GwXbY#`Elp+a9B={&`4nIOb{oQ~;r z;I>X$xz%G8)yXM=6?t`!__cKRDX8lIzx9eQb{YQWj1xHPnf}%S#6P73(gkODA`7aN z`Wr2g1pdU)ZsaZ>HH)g({Yv=(hWHRk{IwtHydiTl9PP&KHoPva%BN`^yfrjvgj55X z0l`Y9wqY`lyCh~nIMYR+ugphmnL8&Y8J9DLo_}Tlv+JDA@o+3^ zH$RUd2G!UwF@+<4Dilfb#@C9=a}&6AI{z=67!39L(!jxjhF=8*cgfKVQ!A?ire^=H z&ZjP#+ke$UGP=Z?bo^?%%X3t6EPub7zqRdUh>A(jNxUjv)@ypY-|8@7i20W@X&cG- zipSakrEHgr6rB5Dm}9-Re3(e>^GesXC137$r$36ndw_N{+qHTvlTkzRQ5VoA5>m_7 zu~tx%xXV*MvVq+b$pn4f6#}T}UNsPCPE%wSZVT=HnHZ%sN9yu0mV4Wxg>~8yaEzKE zM%#mSJ6{Jgh6Wc4y7o$Ekm4;RhAD?~e!haj{MGW?AtUnQCqHY=10m7$uL+z{E4i92 z=sqNTZj!LAaa=DB#7UvXyvsPm;(6<*v$s)RKYOBLoBYv$Q|pyj*B2$K>y*^$tJj-%g~egpY~ELaj}uc^)Vef!pNPBX;R1m$Eeq1f47WH+xDLQO%QjEJltE2x8EoSY}Tr` zuHgV|I*@jEIf87&;T*eYGJwh{v?Hw(A^O4x#STng??|te%0{4x6Z;^}D}>z=549SF zb4m-f{o3UGoi!Ly%@4=;Fru2RK@mP!`k^n^6yf^U4a>N9)NQ`9& zy8@I#O7&T_s-~1hr4xd%_UdUJea?A#@Y|xeq2#FqjpN^*^Y=R5JV8Pdz6*UX(ocCd z(#E{mCo)zth@}sjWvZKEM-e|<rXl+mr3)+Kj$?^X;jr}3g8a;=PNpbSG^~-_9 zJi|ylfv#e1f3}p(Ln=LE}e)?Bbb45jA`B3NcB&3yqn5;*}8lq7*+-*pc=4j0O!njK)lq3d5Z*# zY*XkJXbt;^#tOEkK21OCdVN}eoL($h4u@7o1f3r4-vA{$EnW>Le-=A3AdLxC1@-hB zy*~NTF&9+6ogGFiA=IZup;bl;?ymwYeOy7L1(LW)c>c6e@FlML%onYho zYI!IqrcQTjc^rrJu>vyU+Pedop5N+GYLg@q@Wbv2&hSR{WDk=ekb&v8FAtw=^zO+8 zZdHg&@NO{LSK+~b=um!cZ&8xExloN$oT)X|jT_t@Xnm^W-FpM2!nfJ%^swjWeu zPQ7eoO?@}WMVqS&d4_4P9GbwI`%zc03u^b=<(4x6Hrv$^%Q54MFV)FP$>CRt>*E}! zJ1t^?=AJr*9-HG_I27WaA&hC$Dvk&XE~oh6N27QCK|!kCuyJ{tF4AU@o>lp_;f-O} z?4hUUiPfZ-1pP|6p?s-jDc;GY=4l9FJTv7b`zzMuHlb<52Xr?J9-XOK9AAbu%#umG zv#d0~CBZ+1CKZJF2!-|>GK{d9<&s6Gr*9zNynhA<6}vjqe)DDKc^0lK)iBdc1$K#S zwgqj*s+m$vYa&@;smO?V!NftA7n&QSxOX@Lf10r39*^cp#}C->d`Y*Ql~V5zL;`oK zS+Oqe6#qorB;Qdwu~359ARbF$Y)%HXDx{9?n|?MTE=W&xN>A%AuTAd+%`KczGz|8~ z{CvS09^9Y1Uv3DTb^+ME&Km+p)CC)nSwZ*D$&S?-78WTU{Di?)BWeVrb(CA=XY)Z`T zL0px+;2C-g4{__o6xI9gH()2&>C&TPHVZnx3ayy#DR$%P$cXB)3?za*Lb0V*1VzNuBrr*Qy>dRwr zit3Oo>FL!vtk6ii5_cbUz}alX$@t7uejU&bmgeh3di=3?BbyRTv*}0C-nT;FRzaA< z#bDPyb@x@wTP_|^&9cyjY0@{I?p{E|ru8N>p2p1!nvo`O5Nly@J79=um8+?9S;qGB z&8b}14D%(o84IKRfbp;4y?!3;H8PIxfX*&Txwf-kiXd3CIr=H5maTl1#`p(oe1WP$ zBl0Sb>93YDZO2zz(S?g|M$Du%Py^!6MKWG}D_S)K`<_ZN79#nv@AZ5#A!2fRmhoro zx{#FY`6pV7oDS_&tgjE8tUOFLa>)v+27HkbBao(0yE1z zZ=BOu&(zup2&&IN+ZPCT)atfd(*@fcQK4%NuQCG9I60DLI6-sgf#&JAkN;dAK_4_~ zsOQxjp*4@huX6gyag4Sfe)I*NT8mIfMNfaVmUmCx8lo6|J10M&L0aSQFE1<;XB(a0 z8YN|vHScaa5qNJ{1Lit$R)<@A(ieBV+l2HyZgN?<`Gv1(o&roqlHBH|t0O~A!6KEy z@u?yC8}PtIFWeCcM2b%UFY7Sf#VL06X%3x#44b7J^Hp={&0?>A(9GsQ7=EqO#}s0; zraTUfy?L?ghS_J|BEhrqq3Oc4+!nvarnP28Z{>nosI==vOlR|Y1(n|!weSck4#BN7 zg-$Wx3fNEEk(BJ)v>|*IM(C5wI)gA)E}jSbna2KWKUkb*C2I>9b=nM_)9!~R6%NLT@G?n4}9HWmAL%nz% zVl-{Pe6yz##9t|F)xcy2wJ@0gEXA_kEe&RI6ZTAI%tY$Tyd*?yt~QzjSpRFUKgUms zK&{jyhxXvMoOpi@spO4^59hovUUFgTA9A8Gc_&pOe12qO%SSVITMA8BYO-%aLM+U2 zcH$WV`C{oe@;#9ZUz_jHFE1}Bl9-jNjRb?@5=o18ZgvPi&2WM~~5C@DO} zjwp9g+gg(IgmgdNBsEJHx&xBJjOoSl(kb9fzxuSX1nFnWQdmi$VMQ zZXM*U&9v&j0FxMIH2`DyU=M{wf4W0;evRA2?pf&H^wIq7w)@az^SS-x+q-+CK2%qv zpkk6=H;CY~3$<;teZqHR3H=0?%|7~i9~NeH1LugzsQ9!-^L)D!5nK$_bXx_O$0vlj z;$Ul?{B!RWMj~+FTUbYF75uQ_mH`o7JzK1rL5`-=5cB)b-G&8>Sv(O$S<%9-#fRiJ ztL>eZr_1dhc)2_z%}N4d#)&w7F06eO4*(x#GYZOeX-y~mqVU;>JJjx%r;b-v9w@|T z6zmvp@Z4C(Hcm2-q`#|}H+d=*2%#ymY_wS)PGP)!h0fU>uA%r;6!^#RZ*LZJn_T!C zQQODgZrE(yyh!HvNYId^v>R3_nS)Alat*2#e9uS4=DpJBNvZp5q;B@Rf4tx1dbKdb_95o@~W6Mb&ep-*!#`<^y$fw|G}8( zKEWujk8q(;QEs+w^}D5YVL$+b&9gik8nMOR&ahqXRm)6Ojd<255b(J(o zkK~6P($y1{;;zbz1!kO|X$Sr|#a=9-s%%QiO0mC#;bI;b5yEJWYTsBFc0!O^7K8lv zZ5~k`;s{!Z9hS2U@5)j|#XW|ijZq}U%4vks4uK)lct8PO%?Q+TghHm^g%kMp)FU1} zxGczcewJ!h_f>av_6n6DIPI_`CR65n59MY}uA`|At(;46Y4lZ{Iaw5!1P-HM!biRc zLV9#~#TRo&>6ZnJftPZX{_I1z?{;1u_iLxE%e0xrHyU?sb&b|!!V|10M1) zcYMaksQsf{C8FkMQ^fy*9A#_j{gf-e1bY^j&G*Q(p+JUHqMY!^W>2NE_xxkk&kD@G zu6E*?(*fJzx6kxp0hptj7tx}qxpNtr0E=xdPVE8kWyy5Y$v{5*WB@Z1H^MLJ*OvSxSWsN^`0GOP0dWEEF9 zg@sDgw75(wwpeSFMSF#u)P|XcTcpj4=I;7&p`n(C>8bET!1Du_oODM6+qOZzKVQjJ zjNovwff28w^=SOo_X1~=w(YBYo@-pS<8ZcQhPNAaXujH{f z_xW6&q;wy*pr$apbHk*ryQW5%MI8#TWlI5_ zS+FQDYd6Sl%6=?V0=3*^w<3KClu2yyI9uV}4f5rs+oszAky-5NWuw7TvWap<_MI2` za4!I9?(alk^}wIY;*g%H$!9#f3+HjDjZG+RhBGdWANHuOi&^Y$-ajE|W1Pa!8hRtO zxY_7HPZ_q(GZrA#Cvc+Np7EA=`b0lBJf`PKXED8b?WjhRgTOXN^-}d@pkB_zFWa?c z+~OY(e)L(+@RI$x?R}!6Uy*zk6#{|7ip2ci@CJN*q(+lm3GnZxb?!eh5`z$Uvt$(N zzr)JqYQp)ZaV0r+EG{`nQxBngKI@#LZNR8fylqHFEiIfw@VJw^zbH15Lv*dV4-tYr zwuvyli!K69Wh6CxtOl74i`etxQSFMoj3ZN#u;GO?I0Y!(Gpq0fz^LdVT64sr=ID^D zB}7e)0ls3?5so5pBD5+7Yos&Ve@6N8w3)_{IZ1b$fdk3~OPpHeR#P<#>R|^7)Z$Iv z1}l9wAhn%00?=ur#&LN^EU%QC6ryaX!AZPMC@>8mI`B6DU&}&v2kvVnQki05Xl>Dn zWcFV`Ny0L{KxrU>io@N!ctH<-FZu;^-8ji9y+vp;N`zxkQl28sw>zsiA%EEn`$+?C z{k|tWU_tcI)kV78L9u#x|qH=le%dt#ZM^FySrEk*!hJ=vPG< z54L|!5@&E&$0{BlfSD?jJdv(qo!l}QJm0ebLbm`JTFA~!^V{9KoAKoO?c>AVS6z0e`l)8_SjFQ5S@jba&KxD3t5VmYF+T zc?bl2D7)Q<%{K=dXS)Ir&pWmV?2_TBGMc!;&lJ}9Y)iza#N9ejz8nlCgbmUXYio?} zgdk$tz&~|31m)VTz{#pLZ zK=8ZRPHh|UP*eWR4|w#1QyvAX_S3v@j@1?=97{7pX?mq*JkNS$w8MwFFo^f(xv{12 zl_NOHIG|rq>rFqFWFj5u<~eIEG~IQV`ylmNlGVtWUT zKOG7D-2xZm<#bHSF1Thg@v&=wQJpwEC!C*0T(pq*w2tpVQAD6lqQ~SL+*_P z=M|G7@?cl>&JVB~>s(7&D{-A* zHDvdA*pEIJ{$pp)A&-ThKk|xPFz9HmGeX_kCC{{0cT|I}eB3(rNNM@w5tJ7NR2iI> zZ<}z?Q)tpiMTC#af#TG9`s4?`mOnp9>CwzBNF}l1=Oe&ZM{mz=u@q<1*))0_@aH>i z)A>=IZX4z*9_tXIp<%_+Y7@je?LF<2!CtvcV%J~J?3yJJU!)ch`4bd--4SjlWI-&C z9UR1APP+f{&3>jj{oU3`-xEnD30)f|#Ftimdm<1qD@Dx!Y!y1O&1>BtOV!$ZO(Yc+ z0I*2KMS*J|hd|py6~a85v8U}AYzilXt1jv3LoV~QAcjf*WvEPQz%cdY!4fYf8!QkN z#OGW3qM7OIs|x8j@t$|3O*gHqRPLc4wN9^#k5x+rQI6nL8FcBdDvrc|BM+@WxE|Fjef)iEJJm-!RmTBrheerqV)9FSU#*`=~{Q;z0MnLUXhOc zuDTNnZ!(U2Rb&q0S|3!B4E3Q6YAVlo^%}R{|EdjdyuHMMDPppVwl^(Xq;Y9I9uM3s zRigY#tz`eH?F1DGNvt_~!Wp0XHu^<#XJX{xk^{$TUm`ufLYDwBzatS=$?WA!?I;yW ztKV>w2d&W7gf(LIZfJpx=P4dKZom9s z(ts0_ep{nA<_9~QS6-LQTZ~%M#Ech=NU*@m9j!Y#A6(VI+-u9uq0F@`>_{w;ScK<9 z{Qc0pRV`a8NxipcDiqvcHhnos z<&tH06*PM&8|UpVW;zt&aGA{9_i-LK7&N2b@V`>?vI)Y zpuhP&P@HsH=&0?d2*CXjB*izmuiN~R`F2+vl=osg{90jhiOT#Q*rMCvE>~ZTCDqFo zV#Z|QZN;0M7;`vmG)}BPfFE%!C&YHccBQOmhuR7L|0lK{jTC9*&ZOK6cCw9r%``8F zdoahoGvWPBwhxG4CmKPWoGotZXI3P%rIA}L1*JYZDNPg4c%@a{Rh6CmZYKVK31! z!Tno(aw0F{e-&v?r#>eg-Ws)LG-`#z+o&3a*X(#V^;of_y(XgvTW)rHId&tUK{J?7v#yMY@Jg$`aYLR!( zLVx#Q0*9L*j}`Cku6QseI|u5`zeo-#3W?JJaHG<9gU6pgwleEqy@%&@m(UjT(bij9 z>Q!ig7HU4kj->R?dd7l4;Cb}iJ8Q8EJFpJKXPp*z;p)#-br^f|^_a?1YuLDUaO+GK z0%lLnK2($>T;d@RTyh?%x}zerT~dTpO8FK!HVoe}pbD0gJlw9$D6cZ!G*nw|BA8tA z1D#*xMZr(DqR*gvC1m&z2=u9FXy5?}d?qFgPWima|Ge#Bxyf(j>s!>2V5OBRg`c`> znQAtl*%EnnMK?Nf)oRe z;mPEI>Ve~%orCeVw}kkg0bJo%IsWS;00QFhzR2gGtOu;Z#f^`kSTIatIHy4NQd&Gd zD5q6uS^9W~;~=ueFSd}Jz32*Qj+;Gxjdq)yCRG3EWTQXGohLHlBc3^--FIs5?9isRZA^+R0f5wxOBfYmT6W;iDvrD#1Iw8)x_26N>jfOB+Ksm z#BmoEy=l7!bSnP|h3@k%9gGVR(@gz{A??%V%WZF}U4oW^?vRwPsd(nriS+xPMhdCJV@} zYgQ~&r_6j17sHv&3_>e&2N8>*7|cgOJMyPG4)DTH*+mCKa#Hp%1?yYPr>pQ;u&EOW za-^e}A6RW*jxD4lu}o6Xu}uVSx3&a$Mmf?Q^q>~ z_{jho(Ck_Sqs{M$ey}!}KcVG{^CXr=&ybH;047T!FZpO4+w|~YM9Rc+GO=G9I8PDp zm3#_&CeQBI4BFpKdULrs!Lzl?)`(OsGn;Tm#oD;KT^H-9$OV-IKa)<~xHeHYe1l zA{O^sI*DOz?|*8~aj%{-sFl$Z@w&8`-u6oZCwa+c#`P4y)%AVzf#^4zqjPaH+HX1@ zD9lEL(X7%adtpK_QHloQ7n{q^jQJA>-FqwLA5eBJE~R&tQSd@CLU!ah-xD)F6Dn?% zQ*4Z76=T0xI2R0l_EqTeg(0tT8)ax(y&o+>KilFF8pF({=!2*a?3Zlki=-0V#e9fo z2M8V6zzY__D2`1{(2g8|fNRMvp{dUH(K!m?kLvf;Klenh8whTC|6mtJNcX85|JYV11KU{i#!2`1 z+tx-C2KS{m(Y63WT~`JNn}ZhUVsu3 zv-!3OgN9KjOpD4sEG1{4dqa)h_sIR(4=h0bKwOH(x#mY`9Y#VW_ssF>`w*mmdq!^a zX(*QWUjfj2_3iWO?XbsGfOw|z?4z(MdaZ7C(0Mwn($Oj3(v^L^p6yD?9nX$0+y`r- z;3AqreyQsnm~V1a?!9~*7?BbmOJ=`4SHxDHWC)Ex`OUOBUQ4zuVC9fcw4X5uWh~B;oCt)@?{26#_ztH-mb*Ho&syxy z5&D#4c8BiSe||M`agJCas`aT%{bgS}3LS-FLqc z|NiR5V#rLDFEV>cD#I*2P!l7!G!{7lz*R1s8{iW=F%I`zdD@I|QtNxyEh`PHk2RhM%^jo{XVM%nA>)E^B(8Srhctp-6+*3lX(b(oh|_`GsL)SBVK?2awW@U@wOhgj%CN zmVMj|K~{UMX!!l(>))!-XG;JZ!U=yqvSQGvi(3npDmEFD-2LHF+|2NC7=2JEXZj zWkZTUux3y3KTrJp68)og)37c$`q7o0`DXm|IO+|S&zNF&BbK2?kJl6fg9$V*%e!o+ z9A4OMu#C<)C+3=5C9~-+*ZNt3RTCfI=)2O6Sf-J9>d&pa&z3QyOg*8*8&$c!WR)Z< z#weQaJ&vNa`9zl6FNqyHO~ryr}j{SM&iX zzrwz4ep)&({YdLda((c{h-@+N(;Sw3INI~rb=MZ7<LmV&3w-}?VS=aI zPXp+{RmYZTU}rvas)(lxD&O0UnHhV*KA+8C^Ia~dIMQpq{me2~Hb2{%j{rpcfKYT#>|ojp>rheT z?chhQ({-^&HA)(18Lwj2kMJQ+fqW^q*ee(my#9G=>fN-H` z2o%n7JDOq?M;A1FH{$;7qe~L__cYoM=nuZ*cZmBIBWUnT)E66QQA6%B*ZE6mG(jv> zNdd(5yqk%@E?gs%Q9iP@)2y{Y(fRgT`JcN!CViz3DDMqKVdDxV38i zeZM#BHPLW7dZi^5LYs2YYc>;-_3+pR;&GJo7~Qjdn&BTg?1@MdPiYE>)Hw`U>HRFS z3qv!HqRJ*hS^LSJ@rTGFbx!o=iy^jLh0$my6YiM@Z2g5MXpU-u|41l&SIz&L63`14 zcaY`t5g>J45*4z4`A(LKE@6*)m8{44TfP7s8j=OGcit+5Ep zxx1UKa4;KvtVpjCKBGNz_`S;9WH#^s#&cbxE|hD~VmK{sb=?;(_h&{-;Sq3)!Js(d zw?Ca&d)Dy=_|pl!z2Z5WwQxqK_O)Q_p36OrsP?f>Gh*RORSiyZ74|Az#ek;KS(I^W zsaHXCe6ty4h5XHwe=F{PA?+fiZYZ$V6lo9=O<&B5iK$le?&Ag5i_4`^(6s4w2^sgw}^9j|DvM1W&Fo z4|k6qp>c$P2u)^&WAW}<*W&!Vjx$nmH<;lM7q$~b>_ zDChFDc$BZQG{d>LWwNnPwmB7YugBuqob0^h3QrSz9DylWsMOo&Z&8-Z0O^^t>d{@W zBtR{{w@Bs~2mqj0h~F+ut?FgSXF~s3op}eiBu1_x)yoOzUPQDe9xg)nEk zq@x2D$tzc`2-7F{S}@^-?aA9XDs)S^_%@^mFQvY9W1i1y6)v7$WCd@$Zm$aY_R)N1LRDDgy{|BwERzfo z39+_*!QCU<&(#*-|LW{LP*y0A=sh5~aa}lUA-&hruIt( zG*2VdHOSKbL^O8Wp?1%K&7i%ShT>AhOl7}C`0IDI+ImdjWR(_s=Q%3!6_mw!&&3ez z2{=9I+w!r46^{&JhW2}Qo9RY;d?9j8p;P?2`@EO77rTVnQt{f^w0bkYXp@<)89y8h z0Sm>c=_cXjh4ee7ge~3t+Q^aI>8V1IyOT2b@F5!e0m=LW&^~$*A%LtkFGQBk1>2v# zp43vb3=WR(YcFQKd>Ipy!eMqbeKz<54 z-1>Fjr@m%gr}Z0$F;mKx<3a|A82i}(4@!osCv$jVFj2wRoC8YI@{sRqOAabGa+O|% zV-M<f@ zt;{c1*;41GGWkCBtg{bZ@o{-Oz3UDCgK7VDx!&YbtraPIWVzYb4Y4k#LfwrYVoqy+ zvDE?`U@_LfICynovogctO1YcFzE(ZSZop{%r9e}u*Hd*vI)SH8t~ap9`FD0wU>#b7 zxA}VjsNs=+sNwO232#*Saj*w0wztzyq{!ka4guY#)8VQDxrTQlB*a83Lpm*HF!5kd z`wDZ!IQW0%!*2+LwVT?Q0*U(2TbcQn8&aGsXNGzJh_8WH=ObQln!V=8GEy>bpQX znQk_@Mt8R~b#^yl$#3gqIBUU%gqW)vPRps?Ru`K@rR3yXHuvL&{xc18i-Dk*J8_Px zATcB%X_0|+H<>)wjI(%SJ93y3#30(^qEN#Of{-mrYv`EG+I=#285>KrzIY4 z&{@|Io?v2|dfXjj;+~9Q_5~t15?$1iIx(=4wA6;S1bFyr#g`o(M&BDUh7^SWoLQn- z1NTU55FJk;<#XKOdo7>$rlateM|iAD7CosLwynaF&%5&B;}|uPYm#srJqOeu#tR`` z%3N0vf4!TQ>D_SZw&0tp9F_V}P|$jGeNIdbn1%JDF741Cf$*H%v|JWHtE@iGX{;_J z0jp$tgg|0r&{So1K<%~{e~|^=s265RqUx>6LAq_fH}^deyC{$XI5-ZkhV?8z6$3-G zj9q?f`>*}|3zBTnI4vfPO7P4;!t+hidapuTbo#gFkz=sod!n4nKVIDNb$$Um6=Znk zh^+BXx5G{*t>Fg(divH(WR)awE*ntp)T;1y#o|RFVlVj6Iq#NVLBzC*^Qjm7o^qDY zuOg<%SpM|NU+a6pswzlUZo%s2x7!yM6$DU0;)@UNb8zQ#sBe7o((Np(XOwDmy!x`m z{@C5?){pQ{Y>&f_?v%3fyX4j*W<_5HcS^+q9D4FkU04M&nrMZPy< zny^72P?UM!I}tUtmAyr+O3{kvX8*yP1}YJWRE|mi<0IwJYVCXV@+JIDhhqjm_n_B> zurcS`mKsc2Pe*|10JPCgx7@Kz=Lbu=?Nshn_Sn{$QaQuznYp#cdO}sKViVp7NWX^h zst7)=P%$F;S6j(rxYi7Sy-s<4H;`C}g&0K2sq~We)iTtx-V;iSVk5k~4*J z$YB?XDp)`D5#7#-2KEAcm8PGW&hb>WcE?7))#=pIr>(JDE_+1#qf{Gn>8_H{ai8BO zyoqv?KupD(?5*zaJbskt3NwqT)}+3s??UY!lnctu+k~(KL+nosYOkjlN4!Dm6f8Vk z?A`C*y`%e?kPde2)=?{Emgbs2apijVvzD6;m|;?nw)4HZ`mIn=cO}a}N*AM0im@-` zH^={vJx0#LJ5NnZE4`|3Nve5A@u{?2+KT!auC+Wi=UDB+VDAIhNmP#KRL0{0w;N& z7|%0c!Vd)8g;KAO{bL3b_z9~AQ^Sm^%n+~5cz8W7(di!}w<&mfrs#t}XzceH3YC}d za2qUeXJTU-aH+xHJD~AR$o&26(a7E;Crvp*JGTXPT_@?UFptcXzdimuA^0FWeGfrw z|1CR#vWE?6LSfWEquF@Zt0%Z^ufwr0V|kuRG=Oo?Os6|P5p$X%Y&NQTP9S|sx!)}r zLFo|CEM94}v?G9O=l%i45u4v!zU1b*$<@iS{bH%f|Ky0%9jPlg9_uXRy{?Z0W#Kc+ z3anz01{A*6bIX;Ux7!pyIZ8UTwAhLTf#SqLF&|I7G|@3BZ_M_Uo^V|7#7!C^_CfPo z$bW*8sK{b*gYa9nQUR9{Q`YrYdLhN4Y$$PsAfeg7pugnOs!=wZ^fEVKUAbzw)`Kk} zo(2*tQRhk7a_H)dwz5te%~;Uxuy?Aw+Fvk(MBMU-!}=!gln&D5t;UH$Q;r;3y zm|>Pi3+CuDJn6~M8LAFe)V_Uo49mJv)RPRGw7jP>vsi*~%WUYTqt~*D@}R6n$wI`= z8cGuPo;rWf`NZaQFG=Lb+|fX)l>bTSu2APdL8a7rO?sE-y?pPvQ+GGDjhmxBl8|sV zz6DLXk}3YiqhRk0!GB@V-|{|hPIv;V$@(wqVBG$JQC|`^rt2xG5eq!IgZIjh#EMd* zb0RC2M>9OSZJbB(g_%w^muHq^Takbi`IB>!0f6KJtO~{9%jRK7Uh+I(aZ0`6{IHkzqe16^ zUm+0)3%9gqe4w8_Z|Y)|LQl%8h))-eZ)TE;5}Hz(@gC^!6R9Y3+0tB6?dX>5ij<4- zzs=;ixaJcJxxzG1lCqetCj0tb)9<@lXWDqyhRvA0005Q@v~YWE;D*oJO*vd}LwvGh zS(Rj~)R$FtPTY!TlV*;O$r%5t0S}+Y8|Jyqfp}menjV_^HZ`WiI38?lA2~FOY<0-N zf~QG7X}Oua1P3K)rKKG40G^Uq(KcoznJZR8!XfLs9FgmJ*Ze#er<*au+FBwdx~_+E zD3JrH-e@i`cckhOpUW|N7j0z}W*WjaWyFr}+1}?&-MiVN?u&~pR`vr*E4%M|_q55Cc`{xQAKII(AP@4b9gt2v3EaC66a_1+Zdk~#dvZ@;c5C#SoeZc6~ zfmsE$EFaj$$&!`ohyfF4Ry=&ns!zB^HmN%uB3m;-fU(i?-S%z1`Axy6G_ z+ZF8*kibm;k-&mAMQmp%vs5Y{{SeghD;aYAO1-zv} z-Kz*6>Wg1ZrU0K6U0P~>mg!(J)7&77^5;_EHX6$iWtvKT7)R~@RMA<<)o-A3_T>5! z1IW9r$y+Oa8)<9WwWcl+VRa z7Jzv6>oY1wZmm^X;IU!IPEi_Ywdutg7oqIR$6+tjfc;=+Jb9BX9gA47kPxe=|NiHU;B`hr2GUqd#5ouR#I~O7Z zr@UdpfkVC9FDtR&ANCYR&{dBWcGl$p-yJr??g_>oCbICjt(~% zGn}`Gy^M!+OeeZP-NWJHFw*qOT}qZbke_<2YV=DF*rgp)tMWBW{KnX&%)4k`A|c;~ zS72MU-uB|w))L0jTj_sG({pZ3O~)nR^iSwv(92e3GC~G^Pll0 z8K19@SExMj(5!nmO1RID8lbuKQ`D`*x-fgRCZEj3|D63F8trWY!#gw0 z?=SI?FfQU@*=0{ozEwXt*&V0VtXm0b)xF*A0FJTV;vW^m--^l>j|#j21sG=v z!^Z6ZKu*z#4s2it4VN;5emF0e-E-@30^Q{|j*RbYiJP4BYOduBJ_D0UE>-<5aMk2D zv@SIwGv;jcn;ts!yhZy%52IUksSa{ziq1=RlT@F}giC&l4%|!!5Qv9a?iCI=H%`3z z)`vdu35R|J728k(y|hr|Ju>I zt9g!j{xWLLGn)*7pbM=T=%Z<-9kAM_!9Z|Dz@gzaF-nk6Yn?wa?x7HhQx;09A zaZ43uawXNuquIQUuLEB^6#pGR1c-9A7AvfzKswIw!{RJYt=YE|n`~BCPWlRSpIlV( zr68s5YJB5>sTc+(%;$!2X>SxmT{~ZEoxOn@K7iEzzwz`pi}bR9r0!Wb3BdveStc4L zm#*$G%^Xvp4}d$vT=S!>jREY>^CvSkc1kigT0dlTn$r_v*@T)m&lkxvyrC2L>|oZd ze}1PBA_d@v;hzefO(pCGcOfJKH7utzUhgj{EZ56@vKN?!?<4jTYUJM9V2iK{aaoMo&#f1p9(EI3YhNx6 zn0sEdxP{YV4=t{pP;vjNn_1!Q(@;hWCjoW}L;&w(FyfKDSeyL2;*7TvE1lvg>S|vs zwh@;diB|pT#UTmK!F=W{N^C*DECYO~lwsf)vpj-2)rB2PfiQ?W!mxRhzl=1#XZ-GSpNIMgL=}_EG44PxndcU}C|Bx>Ck@ztYXN$*JKEcs9gj)rRKGU- z=$#ULvTcr8D_7nSLXH$B^aJBR=2bJs0VG`)r9s2%(GgBBG%M zM69Ku7#_ienlMS7uf4G@WWQlhv$OYam)u|64NVH={Bs5+5D)a0>{7hVwBT`VClh}T zFlY@Suiw1FS46q3-vwXew}M}M!Tubp?4{kTi6Tt|B3hN_PEA*u?Qa3Nh)4d%7=P@M zORqdUn*pzM+N-1`vQM~v5V-{1qmnjX+1xj*#U>9m_NQP<-=ExCUKk=-q2{cKr;1bK zGw%nE6-kExvr^bHjmz- z>RBsVIz8g9ggIZq6SVpKbB#t3R2s-*G65+-4@r9TTJKO8#_J zO5v+;keP6(9q-RK*0YXY4fQ(FIEVLeZLc+4tM5V}m-a8X3yB3hGE*G{G;7`74%gcG zIG9>LQf@k$_XSh8wd};*lb1dLe-a)}(BfrO9pK)i$zO*XOaH6J)Z@p=OLgwhPpo0V zS4_59lXq*m%5`)<&o3|rU7MD}N-A5U9D4J69ryVVu2E7#YwkN)HVv#{DJfZ*ndRtU z0aneBqyrdBFx$F|QXo!nMbxopwQv zEdJHGh7p`AEj2_8(MhHkq!Jkj&BB-nm=nvss{%LGhk4R?$nBGCH2prc8zBU%!*tXr zWn~jD{rzwzXSoXnSce)w;IJLtiY6$8@LiQ{DA{4#R%U4jqbG_@ffAI7KY_bv#n?I9 zUqb}Ba|X#HJc_Za5PC2cdOvHfH|h{o_AB@6M&GcGbWvC&=VDE7rvriXgeqheP_|{K zcU3ez?Js?CsZ`%m5YCim*5t&vcd#8EPx?n?pzwZmBCW{G9M$X@%kCg?YN^7fcKnp^ zYul50_H!@RQKGhX&AJalLN;T(;7A4zB$Q=!NB17axx+sU}U+MBCi81_B>o1 zB);YSn2k~A4E0L$Ty9Jltr5d7R6V2uHLj%&@jUfsYXzr^ji&z(TVEX&W%q@vBPfUx zQqo9=v~-Jt(nv{zG)PP55F#zzCEeXIC`yNPcXxN)cYxvZcklhfSxNggUTxY4GZsQ>%smcr)5 zBOJ>G3Ih;aNOi+bb^(Kt^$LZ+eB?@DS3IL_*rxkPqL!AyXaRqh2INJY#|LDfDeAqH zjmg-)ySBqC@uy8SIm?<(0)goHiUDNpn;ygjPX zWQRnjI|ftg{9JRgEg~oO68Z4sD;NAoI-Eab+nm!+^K(NLT4nAN#ctRD@lu)bX1Ph| z;1`mUL`erulRcx)IE(fEQkzM0}iKfY$1W=0nC)#8cNgp|5+WJ-dGbE%Og?r>$$9=V+?SjpLSwe*a+Q z)|%**3}r*ax7U~Ipx?gGU*F#Du{o{mF9Od)jqHHq(3AI&@BL8xLFJPI9sb`Mf`~4>479GoCb(S8 zwJ(dG(is4|ibM2?8AD-z{(^4qmFwcWR54mtZ6Bo7tAJtFrfd#EJ{$Gl5@;$EsN5?> z4TXyH)?uO|1s3iX1;o=?a&PiL$YJFcrfEZg)EsCx z1;HI&y&=W|d2g5;;6)CjE@A%NK9HJiQUqIfZ`X@4!$$8Uug@lmcAS5;S!;An6HgH^*HQgsw z^%^N2zSGR?^SLm}B;V2)$ohy8k+zGYS^hQr831I7D6m=q04rVRxl!_cbAyOw_xxdn zPw@{WW=bvbnDx7|`vQc3IVTaYHrX`YTEy&@59ycLywp7ntB7Rnqf61R{_I#eMr|(a zPAB-!3i9}#d`xQ_w2|3L1xC7v!tN(JUUE2Z`0StHXFm|bUlU8{UfA$mq15*k{U<1r zBfM#Il$ zh5Xi-S*51xD0m9uZwFt>FqoN>@8G2X7mubGPqkJ$bt(=?t6ml%>PO9C5^2a~HQVl1 z0@LR&Bc)zwFj}5F-Hnm$}gTSJJ^fKG-6fn0`p4zVDsY2Ihc`=0TVEH}OG6Rs&P-ap^)eFEPm4#3_ zg!Qa|n=FRMQhq!XEY*rq@*IOOiSQu8R$gGlUG zj$F1(TAnTEN{*DT*uSy?^6GnN-~1ZT&e#Icl+SAPTrosAy$|6e+ysCq%8ZewQr-k` z=l5x3(l=sWHI;Lg=$;7Mo}xZ;@7Y__UITk@HP1d}v=?yoUl}3&Di8|L@(2`R4bTH? zv=P9|%V$OxJL3$)7Zy#ctPT+Vg_ek?+>2ep8@`z2h7%6QsPdz>^Y1ay&ko>gDr7=M zrX%*~;dmXWEZAhSgjuaE(k|5O-@#bzODWB5cl1D-yrn&olj3CEI_BS0gdm^-AozY; z0R%`)XWH|*Uc%J@O_7pjlqZV`Ob)%``#7(ZgMU2+;EWRqe*^@~;iHENKWzZIn?B`) z&;xWfbn@8>v@({O0}@yGJ&%Bq(f5UyefllmthvW!ibS8jPXua{ZVY@5`$tk29^^j< zXa+KMNudS@95^Ss#0dWf)E>T{4s?7ZhDnnDR!UMg1e9jPg{xC9N@l(Flchtc5HYtn zEV)cCz-YQ#h8H+-(T^PqRLsD^q7^cHT4fF*)rxK*V8{*=k2ziJDhC^uI( zr32khh+f{JvWkA)V2WPZu4=iD$(OA-^nNt0kXQYTSs$D9GNe8&317g5?_uEh{T!jP z!dTbv2XpZHR!r(UaJ66- z4G~Uu<%^<5!J^o7zSgc+UcPB!&7;6{hpQG883CFi;sx;Xk*xm|y{c#2e*bx&V(xD~ zGc&XbJCnVIHlNKrWi5jx$(9)EMR2OVv(w*9+bO#_Mh}QLat}g9Rx7Ew(dOqjp6`!r zN&C?-O%B&H{x4fxJ`armP61!!V5Cz#@8`4de9`oZ39v@za#;574`lxr%|Tm523@Z{ zt3d!9(JEQzEc)D-@SKD}E##S6cPmh`esn#<%2qhemO;tRZX~qevS#OWRcg}vlPiTXfNa_7Ss45MVL0*U`tKPRjyZ; zR{Sgk6EwiRvM9UdtMlzLPznZ&;kh@+4r>y7<-YVUjedF!;hQk&N%MG0qZ~Fn`yEwy zs{Z0H!YSKl-lWXG!D_TAUu^`30y}>;5h|`RY2vb(1YKbqe?7>a9tL9~g5UH#s&tn^?8>3e(bFIX2>Hq3sxiC%tI&mBD19ia^I=WT(~Z za^U3Xuktyne0u1a7GS!*>T@{br(bi8&Ged(v-hT7M=y#UB`6=!mYNKp0@F_F(!5{x zRe>K7yB=i@DFZcrTN(4c)_JqZ^2oI z{N%TLr%GXCBRcr!?qRLApIUqGGtz1suyy?CJ1yL!WDR&+_J!3Wph{%~ZfzUUPAsz_ zxK!?{Fk{Pdrk^{ZZ?mh~~NnXEGd#$1faVhoZc(>`pP5(}NsQqPV}i z%?zJNeyla&jub%|8~n(r%N&_k-V<#+^tTe_if7HTAo+mO*!Pv>I-&LcIlNs0uOF|v zh%P+9)74ZMA&vttytp+o-X4L&nx(LqBzct(8JDaSU zsf(1!BhAfJ+Z%Wtamr)}${4AECdqUSYujrlP%;^rj>! zON+B;*PGY2S%K@n;wyo1r$8&8_D_Cese-Hl&$Eml;mxq#?^{ti(v})35wYedE!TZYNoGoO`i)o%OuGVI zuBKQwTRU`A&kFF?&Z-3QB;?9dJi3=uDu2Btn3vXa;gt|@9RTZ<8nG)q2F-Wxni61y zL37Epp5e#M;WH$QX33Y0ZW+yJ>#lEm>mCnFk2Wb?#s>PZK41O6t2Qv1P(xpSr-K09 z$W*6B8SbUQI9aQI@WYTVh@K*Erg!tkhFxdj%JDf`$hrE{Wq?`NRmCiK8;RJ4;;WgF z!Vhf?i*CHbjRfv1NBp-PCsQRFqE)%n2j4L6iwk}eU?y$w3{)pPGuPZg1{$tu6+J2y zr9zRD9x`bT1a$|3U~9_lwkvo^yQ;?a#;5vHZQS2PxRDg{igd>UH9t-Gh8vSP{AoI- zVhS!<0`r<%Pr65TC;rwaTv~#EjQJMNJKw-`&5h*GlPgA>kq4}r&3!`&4ND0&GPna@ z(*h5)p<^&TZj>P(NdE~bdqiPba6RYc<)z&OSz|Q8x}p{+Nq_p& zu9>j-48vLn^PG_XKxn$=C??UZK@+a_=`Y~KA}TQEm7c9?Q}U9rJwj!&$u)FFFLpP+Fh*zvO3Veu$r+m73eTxb!dETF2r!mgDkdkT28-n=%o_jYZq>*}>*;Yg z0=lGWe_r3z2EM4z*XJh$;ol{6m*|Ol=hM+pKTC864D`DbB9G%P>JWWmqlcsIv}60D zihPZqk=-wT2}Bk8A>z==Wq&Qkq)E|PDF&L9;pX_3?^Hq1n+ZqI)j6!s8P$0l{oU9L zD&bsnK{@tHVi*jD$a|I%RK_xh>FMb{+|P09N{$#uSQ~x^+HR?OL#F&VXkRj;=>r!l ze`nVaEIgQM$;Nz1qu?b99HGEj#;j(C@pF(u-uBv%+3_x`+@3!0 z^BCXBtsd%hh24*4zBmoNV0@vRFg|j^g;vUCc@A2y_76}b9!jeNjz?Yk5w^jdP=s;j z?6$zU0?l_~VAK|eCB5&P6SK}cZ(wThy8&G?#jl3bljD^VJ+(sqzEsJ8Cpf(%LZ=$W zq!PHJA72%&{P;3U{2jQpF2AiT7i8F=T`wBsgLrlpDZsNcf=cSuPd%XI%W82F6;&lO zbwlu6*KXHoG?^Jl{qQilaeyBo6ZRZsImT`^zWY;(fn znHQ49NQevuxZ7-Mwrw=ZyE!KNgP!5+>$93tg`0MR01oK(82($J@;v|!PZykArH)4}@ANev+2L(J1TGb9(TgAu zA2)~rt_LS9mkHdM@Maj^bfFX8zp58e?{#ciSGmaJHu(%m&;gtitR|vVFde!^3O;+WpN_{Gv(d$;VRZ zTbav%gyI65?XI%FzSRO4Ab}nis^?HAz8%l@4{liCX0W(yV#(_t5Q*So^FM&HYE2yN zVJ!bV_Jz!RN%Jh32K``mS*usC!{Zm0nzg_9<|#!(wwUCxG0q~x2+fMG{O?`h6pa0z zG^V6;_x(S&2`x3)Pti2Rp^;Gaqr(?dpQp6xmLnADJtcT5sJOezA?{qSS#!nx)Wn|s ze-P|GNdrRV*a$M~eE`95_7>l!5!NA~-mOft0q)nY#^7X8*8C^Ht^8jjMS{#nLW08@ zfH`Hf-PQ1U22wXIwwejxjtDHIn|D!(YPQsB3u%2KvFGd7jX>Lom6t9hdJNMYR4&?N zg=^c^aByDlWP`pvFaPo-zbNRlBh0s`;kzSq`)#-dNRlY;`JX}pw_fTT#$8Pt$U=&AFp+@Be z8m@n{;!qQQ5La%Y{a#&n)k9I}Ei0)1@qN($00XrB{;pW?0Z)HexLRs z-e_@F^sMF+6^usafOg>mcbwE4e=uf;Q8(a#J78J^6!Co@LA8GG*^-$-`~%>EAZ4y% zmpJIqjJU&^oAC81`diXn_D(3l!TB)L? zm0&|(mkr!Kpuzriyh^4@k%;`4>=T%Lh2IL~DKy#)o5MLDH-sPkcDc}0%V3Ml3lUbh zcr^QapqPTLae^04Koc;U0D$ z5hj0lpV)1s)9DsC7|{xUKOpoUJ7q!uSbD+fv+fXxVh)q#NajNeJ|Nu-#=1V^dTY@k z)L;>BZj56z!kyQS$3R0!DCS93C^aT78_et)UzlIbScPTx2XZG5_)-2a3t}5T6n(yF z!~O8gbguF>l$QdApDjixB6=#i&9?VX7oSgLFtbu0i#x%VJ^ zzg9NjJ>d=q!cF7$2rxth>xd)?%%vkdL0-hQ1?D6_$L*j*=6w(@c-;I--g9japX=v8 z(;7{@sQi4@Jn8JMsFZegb(Kq?OqEQruO3SE76bF8X9Q|L*=u*JrhE~R`{5j29nlu} z8BX11F-7;d=5T7dzEE~QfXxCR+(!gaQ^8@%$OwEV+n?l4Mju|Y{bSX2Bvzal@P7Ec zjs4czY~o0F$>y0$-XHR{1pSSSa{5t-ZV~c^vI20S7HVh|j)n>+8uooSRDpZ2mu-sg zEcBe+?{rUV2nuS{C%m6NdV2jg@OSigG5b-wd>YC#)j`fxatCVzIok)3jx!TiJ7S$J zD$>bX`CC&`C&-TuXuen>Xp|cEyxut8^n1s$+)*-UA`@dp7*`|6?CQ zSHcgWsrE2^{K{Bi;`sKBzZKy$^>E|8UYSLm^jLMJryrhEL{As4!;zr6v9O2@AtY zH-laM#^13hT0Tdq2d4-1ao-1Aw!Z2XET31+>CRTBJ~}1qva1{QK!<(Vjd(Is1ty2X z&VaJ(wB{fdxPc&)uwee3Ds)4{ZUWOtJQuwf7YCgcG^+0oX&H=%%RFeDE*4ay_C}ti z@m#*~^0vEN;_oA!>!Wev#lEa)z?N0-ADPoVt7(dPG^^j44rwR3-d?5x_Dt=me9TFOa2ZzfH1I_?-$o9v-mRRDp3xOi)&6|wfh@(8Q|kriBYaxK&V{>& z7!h;ZYI0=ak@SO}XKntmw*Dq56B@zmktJtzBS&Ilc*3}2$GXGAxW`MEd#y*yL6CU4 znYA2;nO!r#F9#B~^2dj3`Bfh@UOJ`x8X9WyYDO{n+x25a_q9saywk^v#f#Y~P9;1} zG<2Bmt_`U&OkgroCgAg%MpBxgsk%?9*~$mITSQS>dT7^w-!5x_6qi^}^>xeH4GSu& zXT2@B60-jFi^fSxp8h4D{fDCW)LiDr=3Zt~C3e9vB5&l59q7Z5^EEP9SyM{Tl%lG<>;BzhXdmwD2+LKAuiK7-XTCZZA zpT@YBY7o&0*=Rr8jmZL;R&@DF8!Im}$0dkM26r>1!tv>7rMbRLL?Dr}21O~;V&`&C z-&Rh5(>K`X-Dpbg4YBR#T|8!*zNvuq36Uc0X8C!W{96qfz_FQ5<xfuEs9=DRDt0$-PySEiYDu~?Z}b{@WyALfR_uSsTF$4PyP|(C*EiU$J24gIRug~w*c6pk9`ERY*ukwf2+qy>SVky8Zi8psPhi@W?P>5 z4Jj+G{j^t6O4M$dEm$#omsZuAF|@SSi4}emrWKc`y)~x*X30C$Ds5dVRZOwv_8tw) zKYdi;n*U&op8>lU>o=L+>1Atr%uD}||B4Fr8K`P51dx-OY3=(o*W(*0qKbl|k+7EY zHO`BB$*(ThyO$GE$7q}rEAX&hs$~;honv#k6wmTwC~^&Y5=|s8C7fn;?;?bQ%Uuq$ zM6bdwhTT1*t(LdGwvi-B0upMFf}_xzF9?=fJjpRGSkDFi%~RrpJcKDt+8LzbT%iX1 z%X4yb&swp-98I)pC^}vI8hnEUfQ;SHsNxb&!~9S7{vbxy*A?y41v=fvqa+)60K#*? z?2}4qWQ1p}1P=RCvI8~xOEeP@<0BVcJ?C8Z$@rWBV>USlfA7s2HCNZ7T@(y?uD`mI z(x-<_y%ii0Cm+vBqL`vxt2&aa2w;l4Mxjg_a9n>_hJP7==V@kUFP;5JiEhmI_510L zDyvlvr#ERRSP{4Z^*awG>euKeIJRkUWKTp^MNwF~&gRkU5<^pJyCW51c&k;|>72I+ zyoQ#ZN`?QSb#~NVQNd@p{HX21#%0BJG12zPk{2qI23nGFg)8424@285+pkB7egqu(25OrA*%zU%!h51G%}BPZ*R@_fJf zBMYL53SUR61_+$#)sAcoFFT~|eSm+BLdsW-4C8!X`K$W6{#QSWnm8snyY_pRu-0yZ?8#uKVR()v0n`Jx>>(&(J1 zck%fK8nZwKsft7KL<9;`;D1cDQ?EIDDDyyUFN_0WZ;qUe# zS|ckXoA86veVMVAcR^>}3EUYTB7sIzGeV<9wgtsli)$JqBjLWXm$M5j>9pk{NA0+z z7-o|i(n-HKOL#?XV@qUwqB-|%g)C&GSpacL`+-6)S-5-I5RQfuDW9yQsK@hONyzwR zJ;Q|wbvvu!)Z|gv*@AECt&} zYM0_9TIJhd&KolpBr)IqEAt0g^z3pwxqwe5n65utmnELxReg=!^k|^)$~F!&u`c8O zg~OSXss>8sM$M7+>c^4OynW%%v;Jq!1b#dJjuRd!bh}^H%Y1_ArwcgCV68XyT|DA4t_r>+(}eQrAsIRi#Bi?-`zvbF34U#;191#u-9Lb&b! z{53FIwH4KT>11?T_;{= z4Q{uI{AoEe;l;##+liEP0&WGPm-II_O#B!Akbo%gqEAkWf1`CaQFp>Yo|>_;5i)s_0x22cQ_IF?GdRB5>lsU9)fq0 zcYQYOob-`}UwI)oq`lg^c7J{w*!7D4abIo-(v9c=i)shG03yX9RNkF3k*`N3zjF=32Chx>g-a@$QTi=ea{y}f)bcI zwn8ZdZ{P%mW^n?hMr|ZYD(R3huEWP~Dap0HZUoYT=ycj?832SXNdDRB<;5$OInJ`r zgB=BX`$O}-i#moHKVynla6%KmbH-lgLnLwSUjQ-j1qmay_6|u-@!8&_+7AqV;Flc` zq$zH<;u#_^*=wy_VeJ|n5+HsFph-9sdYQr$o|cio0xCbTbW}W-I`dW}B=@Z0kvu~! zyvwbsrXPW#OZXK=c_G^#;%(`FQ!Rw*cG^mCW970Yt zMLY+?5dGTsz%-BN(e1~}# z`04k2m3)Q0Z`;!}5k|LCCQgM9#7G?E-mBkvRamXP8ftN~@zls?f1vNaUO@r~eoxlV zpUIj2GTnYGr$7A+q4M_qKuC+AWB8kA7+MCPB&5Zl#rk4@(%4mcDbVDCB+-|w$hNPU zt62YSv{+u5v39eLu;RSSZMFt;pK}>vTh$N2a^;Y*%hj9#D>--mbTP@Bz01`!Tnc%! z(qFu%JKL-=3X%`L7fh$ToalO*Vv27k|CW9UmjQ%_HE%S;zCe^zYK*Sel?-XmzD@_X zk-$DUCvQXAcw7ex6uBpytZgAJL1cF}u7TjR+ewk945>MnHqJGO&T2t+IKmW zAenSnJXcSfIJL%yZo!~z$+{*%IqJ0A>|hw=9fYwa)eXtwhiu#AVeNJ>T&h7XM?A7+ zDa~CB&(EvSM+?_=QP@ejskxY}jPl%{;lrTzCMrAukN3~}6z?u?sq?Q`44C!KGYc;& z-1q@VF=_tXzK(TC=%N{kVfnaKwoIWVD%P|&r<*S&qBCYd8Tlvb@%ql(>8CcogtX@p zqAeXiAb3wg#h(JQ6F@}}Z)-_3Sg7K6C0L zg_U}Mo=rDo;s-RP@F>_&NwIK7`+PO{hpW^|=_L8EiL47DPEq<4H8%)qQV3HR%T^%^NS{d@u2Ig4Q)>nU1f@bpBvyEq}~Zel;_g z`D^n5J+^h<53kXk1?+{dMUK~ccdV)$4}#k4v@BHfgxwlym=l8RPnK8vIQ|P?{a+hG z!c|It80eq;b%d(j&{eWIl}#@=>Tc(WNEi#L7v5#3-F|(qn$if<%q()p{&ts8^4xthgZvo`R?1nBTuhHvwW<<-aLaukaMF|6s1;J&XZBtt@TGVj*|nU8Aj5wIN$hsp~Byj9Ip{B zwPsL)lrdIOW4%f3+8znZYHx!{+vAS9gTb!{tKJ*M3#T>l`2g8s=cld9Sw#qSDdHA* zWPHBdaq>jtw=%)?>LtQc7#Ip!vkCFsS+Zm0zC_G#s2jbrv}2hWWb=EY#^ixMaQsj=Q!766%bSEdN7ldvww~WkP0jhRBy66Kxm%d1!NA7^>f$F+QUBkJE-q{ zTkS_WZ+!aVRd|^1wNTw86S(qvg@PU-6ZQZ-xG%ZKzmHrc6L|7Y57x4*<&eq(jk4He z=rA!yRD<(ka@6al&j$YPElC3X|&KJ_7I;Vb2gL`C_>+d*Y6Y^&%Y19tSGhi!}2v6 z`5;3VL!13p>}|P0zVOTpU$@77UidCT{(OdvdNP%K2(7kDC^X8_`#pqyc>@DHh0aTt zrZC6L?ENW>BTAGHDccS%(Jb%pEGY6Bhy7_BK*uZ=@mDZIq9gc9+s>?zIf^&`0|$Kg z$Gft4v3=?LORO>KPJ(_m4+g5o%==+afZfoCE9{ezx75}7wzS#CZN3Gx^Pri-2Q%JU z=?y+lza{2pE!$C2neCa{Fz+Njc+UBl{ltU#HnqIs|^hw5L?HISA{CysO~ZbxX^}-cJ{R~>okU}OIIS}4W_Y% zFLw5?Rl^e@vu@gqdCzWHBfx&0UKX-Bo)?T4yHfFgJ_4g;tJ@(FiD+}c){=Az*A>r~ zE75*6gVsR4_a*W7%2mcG2@CW8Y)6_KWS>P1DZb7TD1$P!{36OiF=SwQ1egXPj?3v? z`{ki;b9_@&#vF@&-&CGEb;Y$X`r{gYFYURK(*`XSY}Owbu^Ru1O)@s6_y)^TBWcZ3 zk4q<`j{=q$g{7q8@okl$YEj=@H|K{f1NAoup5?<@bdz39>L7OHNa+El(GUyMH` zw_8DA15e-;TYsL4+11@Bq5PaNT-ssvQ-R^&Mk6p7Vfi4t_ICJL(4}c?{YC#$gv$N9 zTpA>-xlvm+TH_Ed;#x7>zY!Mk-j!XGBIFIQPcSN`-QkG4!z0o|$Mx=%nzF%`H3zTE z$GW$fiU9Frr~L&7Oq0o_B7QS@{DGAbeQk!d6Ix`BhqJ!K`gihv2>*B~Ln1;(_~g6V z>E*{6v<+iM^w)PbDv!>gRx75W2wAc(k$32b!s%8tk=wjm;oUJHClvDew0~bKvW|brP5T`KgYRu(eIJg70?0(m zB|k(bJ~1Azjpb?z%o)C?dXpXlpwj=NA<#XV%RS|5{uVey#QzF1LZO#eG$+!~pW;dZ>@hns3cTfQM=yvdeI#h&UTx_6J zGKklKdPbF+75US;A5B&-h+%J*RswVS4H=AWk}jB*tYxI5?6qFTJqQesN6Ww4!E5Ox zM#uAs5a%4Hm5sCU^lu8zucbXhZjG5(8qOCbe;~h%h%422)8~EMrA<=5U;->YU*o2L zJdi)6aU9xee=t>)`ywvyy});;;8FNPY|L!)!$*M^6Q0o}D_aU@Pf)KTNkb$kRQ-6~ zbj`+*$CdT;E8pq}b#G+yb0ipe0yXg0jYnZ6I(f}eO$*ifZRqT0I?%*0Q4+`F&P0CF zN;C2w$ZB^4U@LxvXo*?Ztb(WMV%lyYig+5Ht-wwK_(@h8S?{vLl3}K3c6cWX444Dl z>_ko)NPKrQLV(H^uiPk25alA(ozxuXPL6w^cJ+%`NdlgaeB1hn(1<>G{sk578fhPa z&{=kQ50~g-*!ttOAD(J{`FgEb=u5W9^u$TK_3Ct)+e(C4t%bjWeGr*HdWx4@3XO7Bj68%BZT93(`)Vor=JECf_nUWTBt+koXP<|NC~3w z*Sh-xX;gp=I}G&StgTegR$b-Wo|3N+?2{%=627$rpq~}O(p+kI;@YZgJK~==8Gyqw zmw%7i;}r}xZ`7WMI3FP9(wwd8I<(MR#&0k8G=_Cg3IE}p&(kYTcl4y3u9`8lLo=*^ zebe)fm!~z6NF6u%av2RH6X_K}_*5cZilb?rGFkv$zmX~2oxRgzQSn&_n{{3Y5036K zZb?c6?X5taBKJ0<1hxczjN}x_TkjaP-n7g9=1o(k^Ogz?4>I;${W(BaXgtA#Nkp}e zU?rH|9PZP{E^q5}nEvvVn~`8->u$Ie}Udw6_*xh z26(%j)jE*r>Gn)LohcJs0sbz)h#-Et_iM8}upMAQYQ(~39~~ioeU=5IvEAPgo2_u= zja5&j;95;iXt#BLJ^KPUMl-R}t1$4jK>#oPVbH3ffczrgsTR41+Nif!Um{HqALf}2 z!2MrBc%>x<<$Xupzsb6edB6OxNo78s4-~Um-%@8{lh1S5L)Fd9M_Rol>=NOS^Hnlf zOPC4HYOdYkv`;7anvD<$ho|7HSRP_-nmfs1VsFZHY>tp8jKsUsvIfx6umCz5d*>GR zN@pGYl-?&!*VSGp{@h(`dO-%jYPB$P5q6|1L#zgteE>3&Md;U%$P2k_%7egRFWWABY&n7Vd z{z<|$(%J(mI#fBRKZ|K7s_MB;@{p+7PFPw?54>lB=`9Mh)o~^);wu#?Rj+B}pjx&L zxb*X=HP2kw7QxDxR4oLFI^%3PPS5DLRdX`w*@yl!LGWL!VaNeG!RSf&BX{fQ`C-o$ ztJf;ttFX7fmSx>T~GZ zlDOaq+dqcQN9@>L8fl6@db%=I#`T&R#;p)bzW_ta!v!yH-hAs;3P>1Vt}?W)$o<=I ze)k1#0`}lvaXV)1qcv?;Tk&U?vADut?mfAcJ!+-}SoRNhnaKe^?;P%)WXJM~?U3DT zxHmAfzlMZ!IcRHWj2S4Wkt^2-SvQ7zq)I60U|66B*5z8xOMDuhnonR}l=J~aJLw?wr^W{0 zcPE^M>#km0Oo$p{^g=Kb6Um6f)u>n7xgH z0tiU>7P#NoR_k z@k9_%tJ;-qLR})pI4~QgllzH?7+q4alqpOLk9;~OX<)Me>%B+KkAn=hN;sa}#T&c3 zobRM;;5>?n#Z(X(W_&k+wLuAp*1bHfcKf!E#+?gdY?klg!S*V>0cy6{>~E`>`tnVC zi4saISB&br-Me1yB{2{iTodMjp{msu7b85J85HV}2QXb!H6%Rv(+1dP8km*aDLBrs`ArC=bKJ3oX3| z5dAg?t_sRf@6weUv2h^H2M~-ko6uuC58v2gGaQqAuvJ6Ec4t+i@DY%m>cFiLHHyIS zae6!~Jbq&!?=wog54uJ-pxOBwvUn8Sj`c3RV^1+pg{|lK`10~Q3Fj?@3VcTr?>4@sU77Ft!zSAmny+E*>Pp9Cgrv++=41hhV&}yXpH&%vY4%Zqa zm%E;LEtv>Qk?H6GMczxQq3c2ct`wN3IBzEJzBp-!LdNZd7z+#u@7^wMdJ2^2TJyFY z$(D=!^DqLi(3Cs(At@b~x*gs@E{2A2~ zT_CTaBYcx+z=DdqEHMM?$$jI6o~Z!e0D18bqUKl<^0#W1rUiLf=^49%m&RWB_jv3? z12Y50x(W!_tj;t8G@KKC=Q&!c;}W6KdPn5?K9uJ+ZzV>6v*g#vIT%Cckl&q#n%D%i z*MNY9L4K#EOK2)`7?E1jzSIv;A7uvYe-l5DW8dx%K^ojx6KOoP=p{JjVc_6QF;0JHxhXW2SWUvAR%uu+uraOAv1M+^oK8DD$^mpp%Id%Gt z0ErO9olJXFm=n(SVjtSbUQKQ|qk><+R^Ja_uT~+JNI$jSs<|;Hn~1M42bKTVQ{PmV zJNY+*;7_swIt@!cd$WyBS_^+bV8M4-wBT3qxOb*e$#O{eLb zZk6JvHOb`^nb%`l4NB8Jh9?1P`U{o(QUoJXCZwcEFs2eN`(@#hKoK+0;;Q7g9U5|r zr2IW_yr&4{CA(iNN$*#?R1JRG6&fXworjI`0>rX(bJ7F zF2~iRrMkgDU!2ernY2d>4`I@{u_rJyl)oX(D+3yN!g(aD+x2G)*(%sfu&0uGL7O4Z z=v?Ql<9X~F=G3)2GLJh(_HAkN+MtB~JKTql1R3QYtv0{tJ=pQ?Oi zFnO>d#zCeB&#GvzYd?fRU115wPX>9=T=Am>dRO7ynlq6A0CrRshA?cQqT!?*)|nw% z6RyJGhX!F?nO*A7qsP2krKi>a|KXbX>-#v&4{twYni!aTJ252fzBFAMrp1U4^Z&1K zH3a|QR2o$*Y?|Yvy>QrvR!ZSmH}U{WqeguX-m2u^njjb5@o@A8-Dkb6aNL66cO&xq zs`JhWj+Xj*=Zs9C89I^Ft6Q6a+Qd0O_}6;rNC;1YjnArOiLy;47R{Xie}TMhT+#qO z*`k_{mcl|<)a%jgYuI0JmGBr5XtZ^g10>oQooN)88h_<$?fw#S1Ln2 zy69J5ix=#fd!TU4j|YOHa`C6u5H{iXwy80uMH^Pro|Iytn`sAV8k*GL(}-7=vi%^a zEpOm(PhTv5#rEuTcOoW|Af@R4f|C>V_sEcEx{c!+VofO!M_r>B<-_%bc5{DNMIuOJ zZqVr99Ld?3%vCd?YOaRPAA4!~U6NP#26f*}Q@XbLWuq$$S-85abr!9T)_Vn!Fg>}o ze7KT^2sLlkqfk)u#mNa0`Ox`B*0HklrSF5lpS7o%ca4MX2RHuF;l7*YIe@4b>0%4; zn^ycSAsDxa(Vzhj(3?N7OKs$Um>zV_YP)+^Utd28QUmZ-H??dq1){>v zqpCPzmH10#8&x-{o9X0+;**XCWuOoZ`c_Ozw;Fa|<}>NZIW2xZtr_6F?|(f)g!&4E zuKtmyx03g0>FMf_%xLDN(I6R*fl(Ujts!jJHT2A7uxkIT;IcZbZJvfX3ACrbd}goz z=FVWimY}zL%^51%iL@x_lc^I&-uNeX06hsOio%}2S3fpqtlb&fqn)l#d^dJ-^N@5M zh%<|NrODyYmUPtiul&)tO5IN!SWN*^*8HPlmTfmA8HAh5Bo){N)Pn3u`jPY3C~l;z z`D@P)a0B30A1Z4(NQ|U63~*2dfSDu(!|tD-0KJs93SE4H#`t^6(y2H71i*~oilGA< z#=_MQr>7Drp9b=Wq6Fvz#&6?8QVg6;L1KaS84^#daqn7@4+~*tdf}71mc{MrC86Au zvE}ksdHt;D+o=doY#1a?zmVRa!QW8Xix2HsyUtFE^=?$1>E7VEgjJ{K4^)ELQbSY$ z)%CWR!t+?SA_Zo4;5BmlM(NIY+e!!P`8Ha$TuPvYRAIrJ)jM21f0snXZQ>LDHB2f? zKI5nF`WFu(T~V4_N0J@`Jt}AD`MDv++soLy)Z@c)>@8#>g}U=^^dp{!+x;LL@zIzU zI?z=qqh?yoInP6NU9AE*P>b~WVjtZ^Z8uuqbttO)nkm=axpi|KXylG;nK{W3RQ%Fw z?aZyw)G`he?N5sBg;f*fa=jW^r9>%!#L2m?nM@93Q0}AygubdQUjMPM1i5c76ZOUt z5RC?_3yAckj9^=k1&JKx4JJITlAJ;Q7)gL1`@O0t-SP!8>?3Z1UxO=<;!X|<4K55% zZSXE)m2C_W-C>=8qk?me_1+Tap{?$?2Rbu=H;bx@(nD6k7%cuQF`vG3NvD$gc& zNXwQ737BEFpapJ>Wj8!HyRyV$-0VS&Ch7&4sDo=J>K>>b-R>1B|4(wx&F;|IeCeg~ zPG@ZYU|0q+4OhOw`GfPs7=O=3ECsrQ8mJl`m`4BC0YW{#D_z4q2Mm+C6ffT$;Mvzs zGj03bra5q^$AaJzN)&@J+pAt6;lLO_(vm!HZ_zg#mNUewvEg##qk$7msPl+ zcDH9Y{{207pqzDUt-pvMp_6r`ZumEj?XSBU{>Fo+H9(wmA(O1ofhoOsuB@q`t&n8M zng+Ts+s&5cLfy&~awZ)`L@LZ|m$ycbgu9{jddxZw*0myl!6n4*1&A?l*iUgIC1&+- zMzNS*|27RkB>|45*>85|O~xMT#A$4G+kfJDgGHlz$JSob038C8U?yMM zL=9k!eBcGwAB{eWy9tWq24FCp%f-M$){2hbro&&0@5vwOTY$K+xqBc!j242}(s#h{ zl aW*%)*`YGcxSn}%zd7K*^p5?JNXrHcc;_@pOpn@)}XIrit8N``z6gu+^%)3sP z*#H%aYo#qNJs=8W6Q^G?UX{Ey8MSDc6Qf;_5)0ZpgZhus$ba8oMpL-z5tdo~bahc| zcR8Ka*g0+V5MM3l1PwO9lC7xO{S}{J;wWDB`>zlda*GSo#5#xri{_WR0D2?MY`}F= z-&8ezuD1Y!$;yzFbJMu;Kynp~-v9|{Zkz)hKcjGwbOo9(YK#^;K*Q!?!>bSysJ#P& zEa&qK_A=J0)_Xt_zNg4Ep=|uyy3RdGR20_nFar^}R;hc%9@)E?n9SJQz8{}T@I5+P zD&UxIIsGqf%CrW%j}!f=UiW%CNR#cvOgH*eZq2mOm{;T(?31RD*2ik4y=@teo)16Y zu#GL$Dnh)X*>;u_{$HvQbsO-T;i4jDY^?VQ7Zb=sPd)?%r#RsR7QvDpsk{DiSf|TK zp%!8ZkW3zgR*^h~$_M-xNP~;023yMB6$5iB2U`>B>`&LRa8ZFHY1g~{hUir7R0335 z?a4+ZQ*FGTl}`)ZhM{%hZ!G$Sf2}Mm-k~I1iHbVcDKL*qy`ZIw{S-(+VOgL&Do5c4 z3&Hq~S6@7~|N8dw?+&SK#+Nr;;Twm6np?lSfQ?aGC%rL1t;c(hC*`xFiR0gt`pUA`L;25BViYih6D7s&`J67@+0=tcJJ zjOfSL`T>2^oYmM1U9g-?)R?L$$*=k$drwhh^>#tq5RgW}NIrvtnbkuJTHRHtIgk+) zs|MNb0N70f5koHsqEc+eNsc+7o2hK`d|vS+1eQF@NlpXw)hJ}(WCptxxMGRpVxujf zWdOF}TLiu&`OZ!M$Jn*vtZ-)4j)4Eu5`)!S!kdTW-fA{SY8dqW=#6CKcJAOsyl&?| zetC2Ya@xSD7iY@~R@3ICg0D}Ov%x(CPWZtyx7L?L0Uv?A(Z57n!Likqx0XOHHHmKy z4I_i|k4doZb|K&e&?HU#G{za6YU9;wjAaI{)?Q6BB)1;C%>d90^*!};esQ79danV} zo#9NeN$0v@0mCSObW#G>TD%WZ#%aDZ4MHOw(E3{zP7tzon~yc1z}e8BiQ@5JwlJZS za=*A9Lv>-4O1=4jhO5Wo9Usp#=g-dv0u-(L-1s(r-WCPmdl(<9Y$Ewoiblj1^?vp5 zC*QAvCX4szg+JW+^lQN+V(yI$;Zr}^yu6yHvVJ!BmQTE_1p=<6_>;< zH>k)U-Cfch(*2#8A^hI;`$N~P#i0qrpjT`=eHSk0zuF%YjF2A-n*gFj@FX?AwM`GmL!$S0 zW&#aNGEp-Su>uHitooKaQ=6(tEZJSnKvnn#!SkrA(mI@PyvHC-^FuM`K24@^)EPEtHuBxi6V{ShE%Onb1 z2noBSmymp}R70&0R*)Sm=w0BFbr5s#Q)=4oL*=`?g+vHA?V#{>&pOijOKSc|tj3Q_ zU~=FMQy{XGxmfHxp>msd^YkBqK+W;>RthKMVSiZ-{Sl`J7=P0d ziW5>q76s#;pt~1;l3B^`>G1PSb~#=774_4tG!8?d_i;4)-N@KVNOa@~U*s`3UmJi= zk$%dGfaN+Y5q>T^xN5jn_(dG3)w=VqV1Lh5MP7VO93sM3(|w#3?`ocXbkAthSMR`J zbn3sLf)6-}aZh(=*6YBbqbH9PDui+W)c$}ws;2G#ngT869W_oCl9mj>@_}b!`YQI)v$iW} zlyIUlIW*)UYatPfWKrp}{dwP7|KK+1aCP^d7UfN@eA|n)2A?1}w@bV4!I-#)K__s8 z-7$1w#Z#9FP1bDY^-a)2f0JN4yN`=L@Jr)_Oj9=U~ zJ{;Avdw2%qPM-m?fCuQ-rJ|all?=#2=%aF1+;bpQnq0d-OtP$-H*D_7^&IZny{9bt zV~R)Xd%tSJiu{2HVxS;vB76Q8s>+#47eXI+vL)tT@;t!qay>tr!{7FVAvgfA?xT-9 zRXhP%JALj0zU5h_`2g8Vw}Lv%#twP|i@sVTm()y2zpuQEPEdX-;;q*HFFbFyW98_* zTW9sgsk!24wO%Z31(+=T%Ej?561<+Yw?GkmJKiz7pXq8k=`r*gql!2X*X6T8h9sYb5d0D ztP%#-7)8UJNfj{i)wM5AwArvU8+$03om-A`3UGvPpqDa-nDhg3TH;Hl>5UQ&MIgNziY)wS>KW|begh269o zoJy%NCnpy$zj-uL8*fI3v|+7?{tvs)l`Cz0CFa?6#EB{;vIMrLVMhK63iT{Ajhkdp z@v`8AgD0H2cw7O`MmHE(pImGL%75_Ixrs#3^mB;Q+XtX=$; zA5phsR2`D%rAR{qjTD8};G#jNxzZ9|@|Y+x{BAf%xVm*a^<11tv5VF_<16Q&#rHG4 zvlDFnk0esqe1A{-D3`~uLdeV*1cudJ8#?AESs!Gz&CN$v2Q+^=UAeQz`A(6mul+1T ztMzCE{0Ivzt&d5cAi=YgQdn{;Oxn0Qk}Tu?!lWHBsH1m5#6#ZT z(=?I^JK;T;_+Jg(yR`2ZjtJhInDL8bF=7@UAT6QF1`JgNeq#bELQ#02QWt zxh}K!rah`vXPHY$vsr(M;VY&lV`F>6)5RY2f}6*=gJM2AW>Yj&D&_e#`Lohvi}C>F zjp8eB6IFeXcb!m-=D(zoUKK6)H>_SY+#zL~aO0S(DE8t1qtL)Lfll#z|4_DLQz-4^ z@Yph8e8TpXoSVKc&a{d``29tP#1;)r4y~PLP^pz}g#y9kn#I`-30I`?l|RB8t5zV?1fp{LrGYs{BUBMsP`2PGbfAdCO))jk+B)HVfgUL4c>A3@?1s_lxz zN}{d^1J0Qi;C9QSdH_h~_rEXP{*`g8HPGF!@& zS=t{f9d528_nu(#b2KNMg6i1jp7tj%H)&OCUNx1N@_rIvY#CWUcw(wyhJ8llo$w#*uqWd|hiH+n>$j3Ad@n`*6ntj1LR zWXgLdD|2!~4c^N1u(pb;dC`O1EAo<%&eor_KJ)qL>GNPRj`-X1|R3_9>9x{vGoRGVtU|)Nup+bn@C=#<$6uji*lTiY7}g zn3;$@n+D>_>~VGvG|3o<-Lgha&+;;gB9P%u)HF41s99)obfhi1K{dXty~*@TMqVq; z6p{p!GyjgiShzE8!~C@c3eHyeumqQ9a8%v*zF_<14N%Yq(%T3k$9U1MLy3=%?!oF? zDtgofQIVAl(EZ4g3>+T)smG#DCG9H7ukn^?KGdP;6>B~7@w0-EK9G-do{(_%*mC5< zjh@-Cq3(Qf>=VW}%}Bd}UW*7Zm)UY9wu)_I(`WL`BN-&`qn|BtiWfcc0I$#hpwOSK zMYE}`s_qn0p(j&08{)c%YaQRqb=Z0CzWK{FR#0mqLVkWjt*i6n`%U;$9p|b3zDTQ} zP2gjzxLVX(`Nv|nSLb88=m8W;%li|VnQ_4ElaqUryVhM7#vcT3dYg02TvsZd<(~^4 zPj6Ta3=L1Y`GxkK*M`SYf{1xMP&h(S_`=;&{K!40IIfx7kIlj3ViB*@scMqq5$Xeh zc+Z@wy5qxoXt}9y{5C}4x(4$4@;_1N`&K%Q={eo}%zRg(S$UYI;OHXr&wI)9)O86G zg|E)W%-CwU>F-OzXWBBjsWe2k&Jj;=3u)pt1Bd|ehQ0kN-R$)&$Fi)I@`*6QJr8Y^ ze`*JWr=voISC&hDX@tAL8i3?@=q5qJXc7$daY_g>zr9&S?VWD!iwmVEy;hiC;X5=M zy7v9!rjxsPs|3=Z}E6o zb4Q0+i0AE58P!d0RU<A!hZbgecnp;1#P+$2`ArbEz zX2dYT2(hhwxpR2QAHh%oPp9Sx?Bc!qWEX>Nr;fKKg0f*%Q)L%XwpI*>PMpi_3RSS- z#1D*G$L;d3k>#JTJXQ>^y~N^7TUTBzU(90Rb{7`s(h3VX&ejm2ViD-O_Z^iLZLgRLk+yEAolw?Y zhSjPMy*wj~!2}m2=qgU0+%q7zCThh~)w6JWK$ECqMJ{SUfLt*Ysf68e-sOQn`knU$ zx>4pcMG-7rK+r?8i$1>uB-jI|4Ul9tDzk~$d>CZh% zxa(1*JNt~u8G2F#=JuD`n&z!*S*T9R<=M}Shck%so+n-Ica)Hul1a}xuc)Kr;k~u? z4@roDTu8!Qz|$!Ko-SpF$nQv_d9Jwzl>_D1MGcGa!`&)jS+ho^dd-}pkyfvdX^ZXL zH920Wr{0`W8}@5&5J;|bL^Mxl>2>rH*-rVMdra^FSS8ibo!P!R$L_78@^_`Tr;+qP zR4qQVcmW8mb(=U@g`$H6LypS4aITSvEL&l1onmniqh2%1vwI#g_V)Kti}D7sTaKz& zxYj`}kI?H*^NJZHxZu}d1&iE6|5xj(vzHB8jzDX)4a*GO=e)mn`c(o@P}7#EyQ~7d z-m6n}8@God(X_VvY2v9niH=LJvt+fk_Wo&!uH11lTnRu>8Y*wf&dvZ}q6HekLoMH+ zKJZ?$5{vBTUalFEN&6C~5vm&nCKuO}Xh>XW(RM56z|P#^r10Tnf4ti*FrOoTg7@{G zI3b0R*&?-UHZsFuKD}!<`rP4XPqABNRN(>|71Dl>X{`^(U6%(!QsFUtkxgu`-jI9_ zH-3I=(?I@*JVKgv8pOf-pn15ZOTUqk zxxlM3^eEA5(4IVf-|xN`xfdp4E4S1%XL%sEx96d$;sG#HxLzwSIBp0dQ_d9w{HmWnXFU zq5h9zaBawq7V0Obq;7XcZSw5KYLRNrGr$HGefDi{H+M}_DG)1)diWi=KTEYCUgC0Q z>wUyU2f}$JU!VDQh$uaPf|k-jtMa?Xmok8J-Kqt=<#&Qge8q3o_) zelreBKqy+^62W}qlx@p>_24blv2Z5bZR_}s6HU^POLxww|I=RsR3do*>Cn#GrXwfw zEP!SSwn#z&l)rX0>+*XKLZ!Hh><}6b1Aj4!OM&@X)c>L>vj-l%LW-+&}86^3WB|JEvk(-vJ8r9T!puiwBq$pz;XM=nLFDI#x#q95>zHM`Uri(7k8-B?P8joKAVD|xPd+B8IP zr;tFEji>E+*u-UeRD5NWp4F(a`?ttIyxQ|tAgDrPw>&F?!Si@P-ny7N5ryl9b=Rd2 z@@^5OJCywKG3j}@%aVozPev38Xz-b)>J}`A<6%iF2a*y?0V201U#BbO^1=UsO~T`z z@T4_Hn-uOBer#D)@7H56MI$N-(?^#7Z}hO1nCZIf1MbcZd*k}MXwH{rl9394jh3y~ zM)ezMFq|k9-1w%NF5MTNqmL@%EXn$11o%WhJit*w#C%j=_?*+`@4qd)4*M}69p|~) zgYo~RHYhg~;Hdi~B^S8|wWjvN()vi-Z&^0Q$`Z5aqWPVpAre2R?3$PSJ$T>{7RFHD z*;^eObOu-|$3=@aC$9Nj92ihRQ?{Y{kFo?vSkYew$gQl{nnz{s99vdgR&)1k?@;-8 z%!Gv?`@q30!?SIkp1C$Vjd-KDY>jpm?Dyc3QJs@0!Thtx_!;#^;SgBX#fJNO zPd?(yc44N+YSy6ZjVRiT#YY^b1C;E-X^6=@_!u_%VA_$^-TC3jq)|H#po^O$^jtf5 zA%U>&SzQ966r>3Uxa47BD0nvKf|&)u7s;Ai3l4+|rKc3o7zT-Ujc&Byu|#SB**b4p zkO!#{8!+8!KOx2`Y{}5jR}?#@-kFsl=4uCh5v7uwj8Goj4@f^rc!3h_LC|FUE!5nUwHLN>^siO>Ky7~F;RVqW3%a=qQ{JwlH;kkP9 z77E}!;}OQ zGXxp}Ed}-LGh9$}3vjq|MudG*y{?|e3ypCWH9~U@LnKE=gNTY6od+EF16qNv4lTWQ zN;ruORGZ4l0aEcjDhp6W2S?PP{(f|EZ@`@FLd!sD5K&^b$Z3+%tMeg45<2?z@{>@+ zmpmvzKIdH?Dp#jA_IHn0n*UzkdBy+afL36<|65Bj_6LF8Vf7lxsHYcc2l)w*)S%nN ztVswE7~td557z$pEaB>SY1}#`s^@D5?^|e=${YKWZZ$rCspk{VGiy_M_A?M+d{Ce5 zic&5*c#?IbK`cu%pdg#$&7}k*#%zV(V7wZ(k(W|^rLZo!YGqN#$}A&A-grC=vugsT z^{7aiGMcduz7pVB%dyxJQg?!OEH-0pmM6JX6T!hkOMiVv;#3`#{26S z$ac{)cvLm(N)!cCx;-h4qxU0!gYIM-1~swSnaF7TF1)+_yJ4&HWtE^O-bhxF<=mLc zpBoETxvnDfNq2eeJR7y<-{retT-V84yZJ8t&Xno=&;EF|pT0J=c|`JrXCtv7cJ897 zPLK4YjlQ=dRNcO3)$;);deWFgdNO6k#;<)vW<NgAzcnTZ{v?E*+_4EI*SdgTKWSeTr)en`87$GxAmr9 z?ro*prJAjn8n6<7oJ0TaY_5}CEMw#%N@jINJuZERP{u9Cq`u%9Ge8kG0<7f`-%SZji zm8h^Px0%htqOj?wi`TvVJKpUg5!^-DrGpk%Wz*qjTlj3dd$D4=E7g3kl0HOFB{chb z9Ri^|`hgO=wKZ6==R!JGbPnSh9ai~5ye~`&)5%g@H$CXI#pRf7dpIvpFi*x-b8s z-tLe0`I?r7v!aW4{^!GzqV?*zS?E$Ul{^!>lD#Thw#h29m(|o2^dstv{=pzjgm(*q5z0#_?7siS`W8=qsZgkBeJPq45r?5*l2MiPv+Uq0)IhGo`Rh z?}yl>$NJBf(!AQ2&au)fn{JubTCNGfC)t2$_;eelpuu*CAG zI98TMv7>ILw|{ScsFd1B@%`tdsb{)?Q|xsaRZ)0+AgYbIu$R!14=42|vLGEa9-;(v$W4-gB_~-6!qdWMW?BmDv zF}5cv8a8o)F1;^fIrCnWR)`=cY)v2J&dMiw?5$kpD7qO~V6x0PIVa~~-G|rQCT@Kw zX-)F@Rn&!?xZyfvxo!AR#(b*?lE|15k<$1(oQ}SP;oPCyQQSvl%(&Nr2Z@}ntAAwK z?_GkDA8$Y~gXXyBo{v91vkEe?L$2TIckW8G(!5bwt;ONy|4rMY>~CWu+tc@t5I%F2 zzP-m@`#{h6@zziY_zu0h`FBdW>!sCv{eo-fAv@3qrDUvRFFN0`_DGHGfy%-}!2@U^ z4HuqBavLK0Qqr^!_hzY$3MA-UBfe_c6Ja&t*~bse%gy;P`u=yDJME_b1jQq?V$0I4 z`#dWpzCDMK#H58DaALnm}YEGv=UODxH62Pir zMg_jAOd3Vbc9{-J|0#(sL6q_PohKdeHDG&TnWWb4PK_zyrj1rQaSMMsJ>d%jd49Y9 zWQ^J5M2h*Qb_Gkc@1lpr_awRc%4$C8krd6dwWF9v4fEL9`#cq8MR=p~WZ?(f!}$re zhYxj5<`E50VO#Xd{OLEkL$=nnt?H&3n@WVm_bMs9VqmSzTR_)%C!8#fy% z1~T&MTBXw`pXWlFRntt7TX z4sv|3Z0$QDzi^nFsXVKfNJY5IZRT79y(--iTj4BHfh)?EgT708hvO@I&26|ss5U(k zZw<7y@0@E~K>qW9WD%Ry6YzM_8=SNW%gu&M`&04G;K_WdxfO@ljS^Up?Bbt6b@i=b zSFXSeT58nGJb9e{lqY$k$2^5SrBOVW_MSiQtF$sB-?5f`Epztqeardo^uMz93~}7F zq=d)^rnm~}6}d+Fkdt$Dh)dtpV2k-)>MApSwj1xg#R?XCzx!usOSlp1^&UB=pUvUE zmg)H3UL6Z{`JAsLk(x@(Ulv>hD^fd-YPP7gwcvKmVQwznQ~mg9%gMxt&2XW^FW{lC ziO065N?1$NJUs2_AnwuH3}QiSE|`?N;qY6RJy1z5TuR}#(k**GG4HQ+$>w zDi6J?Jw|HRI}r2~7IVATscV^(f(#gQTp2}xkrAwD(rPsuMR1UAD2r#2{>UVT-k72N zdE}D5rM^GX63^1>q2mZ1J;t=Be9c#qj;`VN9ij>NZL;bnWF?8`R5{5HZIcoA|Cm!T zZ`RAIwjz3FL+NfgZ56=$PS^VbMG^`of;IY+Ysku7D@c$Yy%GW?>UkO?Fq87GjfoW*r z;c>+*n^3?54$QGH=*Rzo7&c`&-6cz*Dy60NhN;bH+%`7Q|DZH-Huq;PU(d7 zZ9j<@2HgD|)-T1N?)S!gBKxu}NF?7n-zLqx+|~0_(6-A)&HO`zubr-Jad>;MY>e| zSC4TmqMsBvfszFqRhs;x*4;P4ZcQa7r`5L?U4}*e+JRlTI-ih$$Va&y zcIWP#$zal?xUQ1g;#U&|_GdH1cG)g-6&;xVGds|5f!rVJ1^k-izRgC2S=#w#>oX>X zd;bB|mmu~y&fDKaou2uWS8K7F=0DJVo!6M!Ch;-X8{YquFo#T5IL)`y=cG-KLriQq zpY7MH1n>s|0N(L--0j@6Nm0W_E9PZkbIh+oj?^ZhJ`STx7*VX%45U9Ez{L_2S!mnj zrk&A4*ozYGi*Ia*(mRUw>lfT+*o5TrxZa||?hrpJzo2yjk5RP9vuWkO?%5e^6`6F* zgi9;z?P7)K@p5P?R3kVGg6ELd8FyZfDF=t*rGUYFj{UH~Wv7GHz?6P9s1=TLgQK=rI=-suDQr}` zD+u^*W-;LI&Kx?V7gQ5-&|sd!VOplO>7HoJ7{Pf#)CZ2%Gy(c)-Zj^m^l?)NE?=F?8;+p)AI25;o&Z?~9>s zieIRsu7|y}JUlD$8c-qMUI*syd9R1}sCdP^Cg3ZtkBWNV+pox~p+fwCAc=fyliSm2 zWeZ!Z?X9os0FAp`;v~)YE{1dB29c?R6|wjWOgiqstd5%-#dO9Pqd8@8nKo~ercygd z+rwd711jWOm7BwEk=oQqlDltnz;NJE1vK9IzG0tJ(yxY#;GKbW*x?@iZ8uqnjb*7O zioIAFL^8QVgA8omBk@a+LYz-joGaXkUJfk^lX+r7m{OiQ0ZuK(9%L%?E?9x%5#ZG` zZAhyijPwb8>GgJ4fsXk}w%*hDNMeB*`W-Fgs(HwLd(yy5N+eZZW@@&T9FuR{|S zQ4{adBeF_WPM)MN7G-EY``fH6G!{S0ow7hfj6OV5vyw+s1L-aQ_yI)^s3WKM3-F3P z=|uI$wq)&NtiOxW#S04b{QPzTw^bts$yk-vM?FuDZ@9$NS)px*M=IC?ad#IhXt~tw z&t|@G2;SAIt+*e*TIBYmGf!Hn0ceNcxYt>RP>u6bm@^_Q=+mGam`86GhxZe7dHWe< z*^Jbjd>2rABt(ws8q6;Rg4_T1QHrQqH@UGxT3CNhW7J||XbXK@TE5L}dhX$-i70aK z>VHC6hK;EyEk`ey3EZiAU=VMgwYN>!$A2i)ft9L@ahp`hs(B7^ZpS$`c~jef&hP$n zR%n{LRgGE{m_3qRa+0}JK9tSaPkPn_9uFJzAs=G-?7^9s-L`A`4487|*$2z!WNvhf zeNG0K6V7z#6oCt%KRQ7K=i~{jF8KL+LDb08DWoMXPUY({yrUgmM6y6f||m*PDv> zu3C$qYRD=g*G=3p$r>Y)e6=*sP4x523?cE&F}|HaagSRfe?pjV%OxgpI8Jw}lw?Wl zC14=ytF!zK!XyD|T$yP{D(O5KJ^ay0_*5s`YJ?41`RPAI(IRwCI03aCjH?R`#5vHa zY^DVRG1vOgm8$)qhpaE$x#-Z*&fFBjj-&3nx)IOmZx`@t^GL17@I9~F40lVEX6V%* zT=-F_f!Zj1+dYf1u|>SS-DFh;_521pJEJ{t=77IkwK0_9JInM3-<$tXWbMqkNAp^O z=F_8;k!@mZKzFP(-*2bFxs4nAoCv4oGkgIcN|@@RFWh;zZ4qU%38G2ey4f7pD zee@s``BO;$B++2-2JPhwdL0p^s3gW`xfu6fsh7hf6#ASt8XGx|4>vegDmM(($7I^k zL~fJJI}@j zL7`$1I$yMs!_*wtKQZYcYUNY$zQGFL4wVl1*AV_te*y6ngVspke5$MI2aI(h53=v& z*}N0`bUKO(%+P_q+vw!6uA$bdBi6B_H*Iw7zNEi#D*?1JCbMIeUjTkd#it3-R5>lj z9?@Xp3O6qWH;WWw(z2bS@fEo*{RChh4j%jV4J1%65n4-Szo4`;J0h6ne7XP*qM)d z83OKd{!cDx1eLQ_+w*-4ldD-_lB~(PORmfI{54bm2|>nKTFT?f=o~K z7L;I-U5+qml~H|%88Tl$0nm#U9z=4me z20ns(n7LV1vY1x9l>IqN)*vqU$5_Qlwj2QMG7OB*g2p56{*=J^=gD8w0cwRJ=ypySV?U zxogt_bhRI$?u`-(Q;-H6A)ArOFi(7-(=1}bq@CINbstkt^%1l|;9EeCNPYh*v-6Z> zuaf~v()lSQ0D9}W$<4`a4Qoz* zx(T)%@;}JkqGagASDi?Q zbPRXZH>}#R`h9kW2j%xavWr)EGO{WHGV)T`#BLo-=1z{*%(T)+#=~gB{C@#F3saze z@K2Z*yq~*OWA-32L4$6-DHt>H4|nyOXBlli+Pa;HH$PB}pjE!C4>QnxF7 zbn11TmDe%+O$BS;UC8;~)Jj5}I6YX#$X9gkn}-t^eEW6(iyp5EwTf|Cnyn+lFIu>J zCWi=X{d6qqu1N*BmSt;5vqG&t6JsyGFQ#fdHSf4iUD`p3<`|bWe7M3{(DI)>gt+^c zKDG7C9-Mk3Wm4La)M7rS?j^4V+5LDHU#YZ2M$<~yfl`@}O1{Pd>R)}}4u4Y7N0 zOk3%h@EEWVUEj-RBHk9{yZN!0+dgqOS9N!|ys;6{&6nQ?p7fJx6e^B(Gf>>~5p}LJt{R?af9d!ap zdlI`DjYdX#sUqun_~`Q*MA%8`i@&7{S8#v-5ZUB@_GV(xUdLN;GGq5}r?d75xSg*k zdSzB!7QJ*cdoGW4i;d*6I+sj)=xCc|O!U&=@sU6h61Ht=0Sein9Uu&aa`k@GV;2+J z|E{wEwgX+*^Jxj|=G@B%r@ zA?vv~RbE~aPqE&{Ht(zCYg7l510d=@@4nJddoyq@0^%jx@E=dG!Qj`iJDL`Rj+Nwt z{SYP>pS{4=I;9LNO{N8d+DVM#vAxbqRgApVZcfcQEUeh?{5)Q_l*2T}{51UDUi_HX zEs*QtYP5R~t2y#kYob`Qr3=2P+GI@X*SdhCnZ;yk`Lb+a)vn7q7CvRz^$V8?4<&VvM&6*L(4&o~)~X=;*nuHH z9)998fdVAX-@weG|6B2X7A9K73yTU_)sy@0k6l`Sz;4#<<4k{$tCg}Z?6vgieH%2u zkQ6GV$~a1r#egUgE^Z%$Raq2b%hKohlcgtpG}~JvnTLlU>2D3QDL*oj=C#2ArG2uu zFsvhqw|J`xP(ymg#+eE|8uNPEeS5R-1M;eg>4~NBp82Te8;QSr0WsO)yn zLXBk76%92WzuxAQCFp6H8K3~K$s|gJxx2H9 zKP;PNjeT3$9*kJzs()2nf4CR}B|9{2##XR?il)l!rvEzeBhwf z1Vy0g@iKpV!p;s?T6P|z<~nu_*Wgma;o{C}6zq`Sk>gn$4#oE%wTr&F2X|flGt*8{j2S02OC3$BJ(m6TJf=G``e-?>C{E z(57oMwxYE%$P*+g9z_2U!ARVK7+aS9LE*_RDL$b7l~=9B<5rw8)2uSxtU-I5cH}9X z4Ke926WJOn&Uk*X?fhdtzf*3CSEy_syt`-2PuxHM!oeuP!5u1XI^*2O%#thNRJa?* zP{J>C>;|?e-{dk0@~S)Gh$8nuKFN{Pq&?fq&qlwW>Gz)Z;s=kvKMmDbg#twU6Kj}+ z-%?s#Xe)rNuG8dNb%ooOyOw+Q1$n?U-6S)$tA(?D=!||#IXBbC-B~n11cw)Q=gb5M zBu!qK4}Z1J<(rxHWnr%9teKsjIv(oY6PowldyM(OulBSmR@vPi0)rKFR$nd~yGd@|v*LBb^+Z3Z@(Z!S5kX|Q;c`t$`23j1-;);O6O~Ns z5WbX482*(c7i$lMSOvAcTO9bjf*V8GMSAS@A$9xusIc#xN;M}tJANH2 zmQJ~>4o~f(6^ve|$)KbQ0!FwKCEh}_dHR8d5T>RRnp2cDdr~< zML^WzMS+1Ra-g5&NNy%??xtPl&6q!vTL123@|u%p)Sk=2b(d$GA*iqrf(N|+(BHzg z80z!|%LCC?U!X&vm|EXt;>%HdkH@C0?on?07dqh*0-QJSD~dqumYL9T^&qiRdI|I% z6Po!}CW*kK*^H0mkHk2$KhnmqqO4Q#Y8t39={OAegkbplUzio?JAwL_=_8G;wC$r= zwmE;-6#=-Zk%>Ml8~W^ml~h# z(`B5Y-yE?^^bKgXlZ)WF#_Y9cD$$sy9_bcr!nengqh4bXE4oNcuzue%#`(@gypm9W zQ14W6x1_ee&|nO^({~K@34SdTt)AWWAOxBp@`Y4s6E9&1{rYEN5xbAuB!byhAN`>$ zt;u4NT84$33@3vePCE6+(C38r=#J9hy9fK<-2>{h*tTwMBFC1j`Z*(dm3k!`N8K{a z#&6KvSLL>!>tP!-hG2Kfpq3B$r&XDj@J2a8OBS8A6$sP6YYHscEqSr!=yG$eBCTn#@MLF$7{XAW~O}WoQ z=}!fQK9p~SdC?2W1T!ZqM%5;>cR+8Sz*yqjV_WiV!rAX_nQj1`a&OTf#}VLrGf)@B z>9ti?ilsz}rolv9d^p~$>EN!d45{9?yin13G!b?XBEAUgh(vFbns$y-!g{&~@w(W^ z@3P*gAN)vKLv}by3I}HC;_F{A0>`=8h2s8cjsm^~{gvDPjHuYR`o>-f%(^Rh4NQuU z2TYID<_n$4Qoxo@jt~~#&;`;JZ|(1+#m;Wz?5vBKOXWO(hI?9xtt5DKk&h+(;QTvP z2TRnQ|6&w4XeY*J%vbdDozNX?p`}yZ^ChH<7Sx$em1(=fI`Nc(5)B-ln)Pj@!U0sS zTZKjS<~Sieyp&m5Yjg0V3>9}dUvG#nPU%s#%;%RfepmBOG9C$ODZxTb8+Ta&AsKc{ z7u8$%8|G4uMdvZ4%%Iv(@^B4;5(dh7zxQ zu5s!Wq@C~hAuGVTKI1U5u2zd1`2f)J-~71O{}WQEo{djOMGbSJSq1m_1%sv7GK62D zNtRE>FjYnkPSki45idN}XGh)V*_M>f>$e)J0tEG2RRdWryx(^%#De{#kpc&l>jqWc~HO{vX58! zxL%t%>Alz2MtY*0!_M?Z=E#iUA9-HsZ;b(Bd52a$wAek6FBjW#(B0$s(mG4aBCsgn z;~nRv%(f%rjd8j^>C*V>c5Q<)s*bws*JS?}hKrMeSj&upcA#To z`>zZClJ8UrL526Il|VN~bdUNA&xp;VPyAlBkI%rKZC)ZqMrqh)`_vap}hlqYle$mFuqL`=Rx`>l+z*WvEfve1GuViE*ZA3n2+T?s` zPUv+iz{FJHbjWy?Dysw(2{9|&yMM+dQE$_Y$9DEtTE$20EUR=?yDu!7kiR_IrZtFt zm^JA+k>LOq>GuCyBm#N3FKPE0yWJI)bzt|3j-Q=15)CGO=ttWD^a(L54SXMkgVCK+ z*~xeZ6TL=N`Z8Nrs=seF52jXj5fw2Vc~WxPh|v|3mH z*n^iki>o)A>+9%NLAIM&E}!w$0sa8P;7i{Ak?eq%Kf3_(QjcO#ydL)$NMA;ep&~F%hh3A;H%ZNdoL)>)2n8Lf1s7D)_vat$ootprR%372cDW9D zjFCI!B>#&4f|~$K_LD}x*s9d zz1KeC_*BQ@qbwB%SxC`JLT09Yg9NcJ2eA?JYZEUjzBC_abjV}Ox-WAkemZ6LJr?v= z2kr#?^Ee3e1Y)2VP0I8!7uw=CM!jeC(jME|Q|2|brRwOY@Tr$;K;?V!RKE^&(4WMg zibL!+F*Zq{!%r{sp@l%-5&fxG>T3;?z@KD;QZF18hv4Srg3VDGEWeH?w6-_QPC_RM zWY&U7p*|ed~Zk>CXubN>RPfl64<6VP5iS@8Pbb<$! zRWYHWN>wN8BZx`dwoX6Yy__Down((?lFUYS;q&h}GdNPf%(4sa5=S@Jqe!Ty?;Gb>a8(7=9TJ0Q^_Z4c`(i2R&c?`{bBL_OlASkI% zX81@rlYCuIirl=lcALn2aY%WnJp;6v7O-yqz4q}w)eHyWfiK6gvFex$;l{?MiwrC0 zZO~YCoWr;Po0lNnxcyt37PxP}SwZrQAfbZv>CU+dfQ+=QdI|fn9@GUkU40Yx@po02 z9eQ%;8wZgEGrq(ITEFGdqugO@YZ{hfqWRh0e!t7nMK<*u*d%1n4EaCTT4UBl&6&L9 zY%-Y+R{6|@ZS0x$A87?Z#kZXIKM+6IeZ7LBal^&E4<9|vdEzOaTtr$T2JU?)1X){2 zU&q3Jnd^??;vW$+zSrEgRs!}Cf5zMUoD2(DNE|tJH~I~LJ6U{k{^t!m5Cy#5#vLx1 z+3>*|d0yZ<&*m*cQOY#b8V2bz?cWE*ZNyGd47nSb~4vmdbr38;HV#N$sHHc7c=GBGQwxIi+RFhlXyQpj!! zh>cUCxPQ;}=O}_6JAd>iwF;nObF|;OIYPrK#MP!XF#GdQ_3X(y6szBp!(PQq;@0Kc zBuXz6=FZ>k0S~x!4|CmrGNv}!%xd&Cf9aSGuK%U>6Bgdu3jIXIH*-gMM{lpk_BG_G zvJ5FYm3HloYdu{q(gLIpbCMR8&7_t2it=5*b2}Dj8*ePV$F7P`^GUx3d5NoD@M@V~ zgRvxkBXP?Q_kTz>;T;mrVcVm5kG@vBtQkUTOLwlCe8u+$_3$zdcgJiGBbRfzpH2r{^hFYsFlk8ZH?`|9r7x4-bznjV$SK!3%Wv{VqTHIUA)t9ZRs&z$>qu4$J5D0#Zq-G{7%1VjB zga9;nM+b;a;U3sq^%$x!w% z4E;-2>6Y>uBAl<#9)AZH{hC*h5N$OJtq6eTKu=a;8wi$63H{^jrM6WJV$^&)=)ddEhO*^oxxL~hN|Z(Ilzj6P zW=|yZ;$nOfvFRiXA+vWqt9kEi>b&wTsW)dT?i4j5=mk99vrsjKua!Qw)1hiKIhEQj zhSQi+(&s={6x&Pu`p&zT*cn0Qd z?#aZO(h9#d(7Z|i_+N-Bd>^&=By)y;k_*2yn;FQZ+0yhY~j{XN5_dY3WAS1JDY7EiR{5G7gI2{(& zDD`P`b}IQkxx4>kita94N%bX5OI4>YPF9LeN=j6D9a7dL*#V)&+?|WPQGI>}dV0BV z()1%%wClfN4AA|al?!_t+OX*OXspblpXDcXTn`h1e+&m`^UyIhgofip#!i#lGAIYe zpj5ekmH6?(T)aec(5<_3W#k}dC2ri>(e*RSjDz{|qQAM`$#FhVB4MX;I{Y7LZygoo z*2WDhf)W-|N+?Q+lpu}37=VC;AYBSbN=xU6pwxgMAuWxRNVg~*Qqs*((%tdy8%1Y0 zKJUBM_pNpQ@Gx`ledX_peeHV^OF%T&7p%ZMe5>4n;d)%?DojrJ^0UMCY09!UYt+rw z0{?O9k9)H11fot2^zvVdJ`0*RG0RXEJwW(sTp^odz4(0Bm^3-;~2E)k^2RxTf*`DS(HFImO3AG`Gi-t_zB|VSy`A z#m|Yxbs2`%eF%@wR?we6?bbffowAm$o@2R|R1&*CN5he8xl+RyZ?++56_#Ts-kL!k z%k`~$yzvTiMAKf%@asgk(-**0l*7n>Q5_0Sgl&UNXjTtD=D+eNe0@)1xKUuP$5^e3PmoJYUyioyIzUadY514pCXj zREloa{*4w_DM#!bcc@Q%0TsMmKvQO>FW*M~`eY=1HUB*}i)XW{S`DdB^=>5G@L8;l z0>4f;6B0=0Oer-`>o+iVZnSQq$%p!m@Z28xCI-*0$_=)0Euyn)*U6XC=N@RrSf8_9 zf1)soYr8Hm4=lr~u4f_Xr=p>CO-LGOwRhS!T`|pVQ2k_8on-mO9kk`Npm~qVKl#%k z#Bx9b>PW`Ud|lG%$g7FYjtZnDt7_E~vhq%FjLprux+VomK`ABM%_V;-Dh1{5=S1d}j@yd*@pLp(v54-XsLXBK z`6--_V0cWe-x0US>f&H@lAOK8Vw${FO6D5Bl5YKOJs6kvtrI?u=`KR)Du3(sDL-XY za*n|jv2q1e1IAd)bzZ0Gkaa6{*pX?@JfvzQuO+ymHJgEK3Tg_biV{>A7(_H}4=S1O zD47=*A^T6TmAa7AHSgykgOM9l>ow0D*bws-{Xfgd=sp)zO^)_E?C6jgj_oGa^v{C3 zt^SI7{Q>I>9{UvjmiNs23D!-e)5+eg&b=+$9(4!*Z4s``4XYP?PlWCh zV>3w9wj$NCR(ww(5Ig&U(|WrvtaFt{#rY=~9CIi>Jg5ZYyZqYIX$`dhWXtr z3B@cTrNMSaEp}Pa!qVxc?{5X^RQAHjH?y)py%Qt|Qipor9T)@rlp2(jbm#q=uJ|F> zoozN9*qoxsO?*1Hd(B3<5u@C1%Y1_G|4dtTddL=^L)T$dVyV^{sH%{jmp-jF{7IFI zkqlSl!kK>QNmmAP+`2O-iPq8u<_8i@2R%~q6h4#!MvJU%MSb`O2KlMB*(#cQJP0mX zE2l(M>AlvrQjE&YE7+Um$^gSgC+~Wq)TIDrt}sYvjSQ!CygqS@3K0Uy8OVHXQ{vcM%u@Q4Rw0&U{Oo z4@XU+kZQt`qvoR*2@0lgKX1J`Oo)Xe+Ts#a%19amkq33>^0uDX2D!}Xiw z4&BN|86x==ofdF!TTI8$OLjH*S?Vgjzf?8JZm2X!p}rcIRO0N|spp$Bz?;!egPbvU zwCZd>Cs4j+EOQz^0u@K#4=>~|mrBUKtZKNKuA^wETNBb&%&+MGY+uf<47E4UZ*d}_Zl zNrAx%4$v+V5V)b1OMp%y8+*sb4HsnCP))>>XLJupO}sDB#hUqFVlb>$PUh~url+4S zEvm2e*^*&=?_#oC)eq5JN%@=(P~D1nH#}ANJ{(OZ;P5?X8d0Ho6$_k3#$rBr(U3Gu ztB6ujAM7bf|EYA`0-2>ch{pQ@3c`X_~dC~mR?G~3%@$tQk4KbBm z+U=w~u>?tB%c_qZImT&b4fqAT&$Z93^9+CMOy9mVT~sUJxpMeR0>5~A;P>T<;{*xP z5%K=X*HB6=U?@n6-m?9=y09pb^tfKY9cP$oflo$EiMVlmE?DK6>r>$=-{upFTt`(E`%xfGce2IfJ`Hqb1bckJb;cB_* zhN4>r@Z3W4#vP339YPjIuGDhtRFzl(2ke4m4|?d9d&^1Fk*ejZd)eu zSx#Bi)`Uely+9Z|dzSXrFz(Y7EPE8jH z5s?!1?`|db={$fZ-dPHfzfmr}stn39%u$XeY8Rl~bbgGrMB zv^_cL#L&YiSQ6Z_0<$D+-?RU|JQ<~qgcU-&r&EcmqU{c*%uc@m%cq-rYWbfb?Z0yj z9!IlY;jW+LC|NwD1UOWL`mw%)p4%D0_bT(D8ktM|=|? zWCtYbUq3%xmPyTJEm;osd=)+n+5=|!+Q~=Eh7$bB z_18vGW=XTrN=PdwSw87>_t^e=sY;65|L~X!$lV#m-W_Qa{$>p>3v!v7qJ!J#e>HbA z6=N}cz~(bm_y~9Ey}M&z{OC9s1Jn6JNpSFcLVv^!9ubAsKXR$>FEf-vkg!_dvAyWR zx`AC`4}=v0gheDAEC7TB{t_@*Q&y6EGiC+<5I+v793=n^lg+AEm;Ymd5Gj6Q&<(p* zy%q&DZvxjzPg;OY&hu2VR-O!-1DnWzR!7K>Z}%pSYUF`F(1@(v#v~Z*Ig%(q{~HU& zR*~R^b`(@>#8h9jLc;x&Lw-`cHX7BBm|^AXO`H z_eje1hgD+0YC-ku0d!XB8E(F7Ba=GWNROcB+j#-bcKuZ;JE&hAT>KP$@o?#QJbKCj zcJht^JH@_{YWo*U*l}?1ss7D3&}}f3FHncUQax_9Y=Mn_zme!-C@ghfJz$cXT^DU; z3{nzh1f<`I7Mef5fc;nZ;9u?<0LX^960pz}gT%aGW``7Imo#h0?|kio4|_t0+0mC} z6#(63)&JMKMZn!~I$&qxK0NrS<<=j{{T=}Q5hRPMp>P5z&U2OhlVUQkGI{zsNTBx) z&&23j%=8eqj}}_61mF%K`c#s|N7niEztr-8oeDddKfhUiAUeNA43FC-i`2A<+&Vjk zSdieO!6XXR8wuH?tNag{|F*a<*8yH~dv36!lNA>Ri;*(9{qq|=;^$fdbnPteCG55b zVuvpRmrCr)(ZGB0G7!b<>Ok~H$)QJq-_Qoo^DNGT;P5X8hIjEee3-jrQ9F;P(hT#Y zA1wq!zW9_aZ#)0K&H7NYCjmUYT|a^L{2#w?2XKItUa>jy24OI7Hp~mwKM{Vmsvdfy zf_FMFv%jy>WO@Fmfig8qtP&fJZLu{IM#RFQ(9}r0VA%e=q7oD%bXdrPeav zqZuZw{(!p#L53eUTycMvu7%ZX{0G|D4%iRI28BWI*t=1v)E`Ro`~wOIdUR7K=~3N~ z(k~l>3OGsa5I`?|jT5{;XqT>|06t#Szuw8>0yBU%3q=Z}we9|HMhrF!ct$;m{w^E8 z7>vxwp}?dC;3<#%e?XVBs3pNkNQIIN0UA|+`(#?+h#!3}?*6A#KLCURpt6VQi_u)z zl>VbSoX3t)9uvKP=P`J2-?wDjB`778^knhtYtb*oMC@2z%DjTTGSzS8!?=q5;E}&H zCUZZ#UBMmjuQ)Mm?AAx;Ux_Oa;#@d>N5GCK^O(%LhO+OgzVx>&bjJtiCmL>Nb(HPp zuuT+?b<>$yl$K1;z+8?U$H2tamW+Oz$R+vB%u{RpG65bfELsBdqyCE@p9W~pxfxxc z{Qb4_;2*GhJU!Fro9BBep$DPA9ISBg4|pN2o@pvWMmY}3-2KH8eUc9DJvbB&#N`FAVu=Pkx&?>4eTrVe_9 zLa8q&8cAJ*&p-> zAK!vY(8>z%xE9u^ra{@cJ=M`dDiSSst21-V)@k#otO8lUUw{Q4ijb$G)B0;Oc`SBr z9S-vj2^7xZw0P{m`}9?;J+%U^V6+iX!lK3CVqr)QUU{5pb95Ou-#>l^AyL7_flBQ! zCBTW#HxG4T+{wm5eS}>(y7gqBT}C}UVwIRv06aEDCH-?Bq4y%yx9kx*?Jy=lUWZJZ z;!(ANr+TDdu3FobeF)S?noZ#B;RD=81d?Vm*4$?x?h~*dPKN)$-@F*;>mX8qrT2o7 zZqy!?ARx-yfQQB`u3P`$K_21(_fhsK)JNE*qg!S3JelZu%vgh)d_q@?;>_axPX^0q zot-CTD}-YvT!XeI#RW`VP56LrlQ05Ic5cq&{qO$4l=1_>6dqQaJnAFO*znTufe?01 zNyEF0ax*d#uJx(gFFduVu9=(#RQ*th4&A2+S#(yID!LI1l!c=qMUb?zQZ+y-d4tI+IiZ}!!&R2NcUC`J z0|Qf8ExL7|qDCm4&zx09uXAWlo-0!jRs82E{e%c;!B=(~#3Sm&U^MoaYSo;T-nLjho1yKD+wQ!#PFMywr4z%#$lg`7c1E_?>|pkWql~tYT44 zE1f`8>L_89d>A=S6JZe%D)eX+{I>2K;Kqu0LR!E7_u$!RH!S~=cgmb+I#RlxRnd|p za_26b&^H91UB4zR5v#?t&n0maRkJ{d@HciP%g@A7RB^Z)%pSsRU+!L-!@odW%1}wq zYX(Qs2EV!3!;#O&5te%%Qoi0B7vXC6n_;iet*wDtdfvJ;|6iu2X9O@RjY6Y{3PqK$ zDNMqmIA&^M;05p?f~4#gR+56bn#zk3i%jd-UFT6L!Dg5K!Z&zQGt3ix&jX9W15xsM zd34VO)F)aQqq}GtTQWHT#o0k$^1pk;T5R?unHRNADoWO@qsu=!+r_#x62c=?5g*8D ziwZNwivGu6PwxR?&-g~# zqlqfo;CLmoTtERFdcuJ6<_8NIkx{M30!tKQ?iMjxJXZ(N(GmU=V_?&c^uH7XihO}* z?@bm2qiz0?Sb9c=<2@Pl#Q$*S^Y~!N)BhirJd4rJ&O2_g$@;Cx-8=mo+i6jBLt|qZ ziZB_!r+4BXQVxZ3mE;{-R9A;><6O7WFAM2q1p)!;Y+?Pa&c8aUc?tBU^v0j#9}fB; z0|;TIs;Ul^oo0UwPc>z>R89a^@CLlcZIdDOW1N>D>s}7aMYSLBl=O+?^A3hCdf5N4 z3nNBAu#cdpv*gn&O8cPx8s?)QieQI;cM~zzk{KEGDk<@?cWd|})z6*=h5`Wb~i$-R26>yY}i$6^NuJq^g z9*_rXg<&1bwX=#6ZZJH%R{BHI&l#L1j>lo^6xfi0+F2&_@f|DEuU@_jEWnFUTBk zaK#J%CqBa&acm~S;`Njm(2VlHXW=U#XM3peF!jxE3qRnRi3zM8|IN?>($9H(QVRZ) zXqsx41p3RB&7KWJed^a&V6#w;J@DfV`V(F*nk2rbAOf;ozr6Y`O7YhB(PoYHyWR`} zih7*QNVJ7o838VZ#%dX%t?^4Ctt`>yth=nA@1iO<`yKF?Al$@8h$+}5P$ON&NB#V~ zo+SVd75=YlN8}FaFuD|jlze*H@CDivAwsZBFOdVO_dJRY6|Qz*7Syrn^lO=&)NVqf z%a0$dokens0o4nB^W#Sg07Ku?J?B8$zvUl&287w{JAOuQp>K(StbWB5^>Y_k3eZ@u z0e;REG?D;s-~k^ZP(dh~WT6(3(X$}>W$#iQJFu#2cVlCZ*s8J+Kp=UKI7hU7ei5mM z0I)osABLt(BL-jsM)ypR5UbMnDfM48Of0Xz`&RomjW+MxLRT9$yA0Sx*dWa7H}}6Z zcHk$lX<%s2OG0|kDpwg`Xf*A^ncNVcdj5ZNf;Pkd)>V2?O5j7xb>$ys+;IU2MR_rs z3{_xp>cFk1?rFmz9zbE~^WVa9MHEmhkXhjIUy6^x_kpRk#G)_(zJvy&C$UgrbP2qz z?`R$)Z31j#n(4U%DqX*Ikj4bSGY4sCjcQE~B7i0TaI1I;NsTfdV3928#d@gx`@=Q- zJ!bLpD(oMcq-?$d%x$;Dg5}Sbp?^Gv+}VJQ3wi<;b`WBl#;Q+4rDbF7^QgZogD;a_ z2d;M_Q{o?`Lo^of?A#~PMrgv)2RV%Y_@p8kv{RfCfcvA4tET~E(95-Eg&x2TxQ-?U zzGwJB?=+fxk`#gBDjzdbAFwTib;IaI!ymgdK2qpLt3$y=ydxh~jK5q3vj-{_?JK&O zQG~;SKcZ%{x4((=5Y&EGe^vW3|sHT zb63jVYpB|gfw0ZfRZIULwy@1F-v?R@dt+jNO51N6 zWCC<&V!hPEL*=OkqMK2uUKt{VG6kcZllLqIpG`pOTzerJbEA@6eI@}s zu-iLK4z)C(%mO(s8yPPil&F9<)}O*ErgsP<9reTQ>3S|;`S2Xv5(E|^RJah1M zKV~>iL;&x#E3r#&=db6n(eWN_FZDY1#Hrr}&DK}`E*vW%vjUyE)*fn@O2Ni=hW8-OI^aqO7#FCjQUUSd(u zM++sOT+Z!iy)2MaTmZ{IY8}6dZivuk&7qK&+)B4X4GHicK*XX(dW_x&J^{Kc=@uiC zssdae$z^{fx}+5KqUdQ|+Esvjh7~Q${}2niH~{Lb(MHrx2%HltSrXuRMi2{nP*%r8 zAV!8tB1{5=!Ze)NJO5ZRI}J!W_e_Zos@X*IK$1op%O6NZ+7PUnL(Fj*6`()U12Q?0 zI_>|Ebn7($M@(-&q4g!QJ%ERB;$$D3BEVG8`om`?u60rrT)dTSXlh+x=!gzY-T0tR z;miCib3hEb_!`5vGe8{O&W#&hNXiAK$-Fy1sVQ4 z{lCf-xIDcT&t{M--xBL;b=B&N6RJnqK9!5qN?Ny8gQ2{nD%AbOu#@)q% zi_GkE-3#m*>o)vTDUjvC$DyS4w*4&vy&xCze5Ivvv41k;6E~?NDAi#p)>Ww$E!H((y-EA*mKz&Zx@HWXLt3f4ovbC0N`zx zabgT6O3G8laLOKxKa3vylxVdSla6owmLx~w|8{tk@Lki;A5Y0h6b0qlNj4<{&2;`s;S?b@~A}!&^ z94(r~*DZo#7W_Q`lhym@^-PZQnA%ql&Y{Il7Z?)&?T))G7g2>uDV+(E_fjLYS0L9U zxp18!dMKXpEgp7uDgc(CGig3KE!PXi++mU-pwPu9HD`8#7 zFxePTJ7}E4zdFg%WI)nJ;m2~ZOD z!B`U2ak4XE9iD*ayv1(@)4RZuT21brIIpJmX%jakZ5f1`jLC{W#KWj6CQgN@_)EbPYhQW}S zAB|p+^I9#?`lv*_$Ii|LLWa_NA0TATs%}Pz2R~ydb)dMCLItpg$23jI=V(C+j!;hj zj-9`lhTX&ZEHe|+RrJBvtyWnTn61~Tt~Kl*4^U$eM`+JY%pMuwD?v~5e)0kqr!fr` zA@S6BgR1F0PN3U`r1yYDop&eEiV5H@*Y1M$0y<`p-tcqNB&i%Y<4FCmgBAlibau&awy?7-*!$0cT7cKB(~mY}2Cx<>E&SxY-n* zO#0bBBM-z3xxqKX*`Y!m&5g6J21tfP&R54mQZ$ z<5WO`@S?hU3j2pF?5wJknieYL0~cBYNl2GKX09x%y~N$6Y^-GVERkJ&qj=v7@4Ait z;<4V#0D1m4Q?h^61JMN_&5yI0)UuzQ&7yqvGPshJol2aSe@dX%RCZ^?Kb zSO=}T2q>4h^=TMK8wNWY4yRN?t0u6|Kga}_LwKsX6_tW!PYiuV2fX(z! zX**)J(xZBy{N#ZLn$Q6Uvo_=qgCwQEAT0-(J$>`1xL;yva^65tq`RpXdB^xu|!CZufBx&gT59P3)I&# zxC$L7F$ec*L#LU_4Th&N^a%ivGhOSrM|rn&n*Jah%R?DsV634D<`y&Mskx#Ez9Ske zf;uvJZ+g+11)+@1Gmz&?I30T^6-@%|cn<7H^C!RSv-#C6h7P{RwiyA^WDznM8gT(5 zDiZ2*0(&$I0PxP0N}YWN%K^4yP^pTZ_F(qNLP${6pN>jUGdo;|7(xQ&E{TWHnqXrh z*#f^~a9rRW@-CzPK^Oeyb02dXB?tjEGLb6w0vH3$3;^Hb=Uiq9u2WH-T6=>AoIrX1 zJgxrmI9drZ7(-HFSNx}Btu*1ah z*bBRaGOfi0B{@4sfGaBB{*o&aEeaq!4AZ2i5DF2(hEFgoAqa3x3D!V!eVVfcPsE)C#X$&$!Yb}6i1-ASu%S* z6AEA}(-p0f(2n0BTd`P~jexBJTbXL?LqD|uhxfo%QbfA;4kZcE368{tiJ+uWX^81H zOb{UH%88~lOzd|=&6A2jHCXgSz#G_a1_?!<0}>U8E;!JI3=M#^k6)j%VzOPOJoWv4 z2qQWh*zC~J!?z>}^cesT7!?%k?K`luBY~}ScqLQdKL~@x`yl=vn?&UDD@tucpa$;E z^$37X*F16}KqeDS1i3vF2;f_IHSlSZ=LZLrhu8&JAp)4DovATpRro`Kt{6 zTV7CtXgJC9mjo!ubjSUX1zF3zd-`k`68Inv6Iuma&=Uv1ZFFhvhlz4&*w%R^O1KkV zjrF#K-Z9AXK+0rYDgoo7ffHZwJCP@S(+JJvF|k)30eM~-?h{5!E*uGx6d+e9t^B5j z`T07?hrJ$U>0opd{_Z{9xUz{b=QBMxUtL!K8I8uBb%GiM!0f|PX(Iz{9El0$qd@ST z%ZT1a#S!45xzL0IpOZW)?7$mve#Al?j$b~3N84-yJaCtm9hk8kRB2$;$6JCdt;4N7 z8dHV|gh5LiMg(NzI-o@B1V?)y%^v-dBh7viE%cnPNdW%vysGgdDg}N@aAhc``92f- zWEf<^Ap1Tj&cVRPyLX>X)io*~99T(H@} zDV`q|l&hVmF{B1r8;_}#=CgRnZvL7`v^;4ji2GbRiRJfCHz6 z(6HU~Lho%ca06f|@(km4Xlzaf!a1n2T*ISoW(RDxO-ek3*n9@{m^EsescRI8z!UYC^d7-b>YbRI{Nuy}W-c zebsRq7>o*6u9a|f2N15Z7pX5~a$5`lMxC+xrCeY%sL*a8`-5@FQJKSJS9g|U$#FxzgXkyTQQwceA#)o zUIRI}zh}L(jdZTYz+*b^#gwVYnA0G$7-(5~A*+eiUJe;lwDIknfcM1yjzjU%2*YA4 z9!sUy{<>!I%GY}SJf=@WJFNRJ@ha|mF%nK2FSILarEzE?cqXoMX^)plX>L_l?_Xpo zTQK*n6}jR>?!~fMsi^srMyobH_zg?O?$~Funu%rBXG`yFmd;64s+bgCGAUlu@fFuB zSct)^#AV1hKYdy;afZ8HQ!;`#M|ZY%d1Fn*q`KGHwrJx0!;Y-35wF#qGM0=IVaDRg zukC01SEh-I+4(VvVJ=_xNt!1oA0^$74qWONTT`j!5JjwRi(K|n(6FOXxBc0^QBP)4 z=@1pgLaeSm3{j;S+sHXpD`&8~Iizu=;+7OYK}PA0a`9fB;o?+%Mpalw8Bx^cjIQQe zArgceu^KnA((_?OyBbjuXO=+h{DmIqLSiCaQ_uC@{ zrEsaMd?U`2L|L1aTb?>SD2O08)9%(C4UkHxGU^;8=2ExN+8KurzAJIs!#|FjJkLs509eFgmW z<%jB7v+k}nuz`cu4{4Hm z9s>~;wQd;pw4N(v{+7$nCZbGjN|OSo>cK{vV@%L-A`^A9Cz#VAy3DiiLX^jJF;VQS zYX2Z7kEx}4`jPTV2_z#Sfe)0>%t!ftbf{)e{jh^2A6}<=dMT0giST);7Njm`$?2UitYdU{1rTx?8JIEWOMh zi-P~-tu?Yjjc)NPrSrqwrt7Y_>AeQ8nY3oy+^~kmP8RRaZ6k+D+=*4K7Vrl>?)ma( zkIq(~=F3fd|DvNQ!^v5J@pIQTv+{M8H&-Go)eC;=ZtWR{IWRfB=su^FUa;32zR@7t zRF(P7ch*+m<7R4Zl>zJ8*K|dL?)Z-dX%ju8_haJx^c*@jKF&)RKadpX9Xpk{65jki z(D!9<>is%ioNQmvn_Kg+q~U86D^;mgzfmyW@JP3kh(z^~=}$G^fo_XyBVDrJ>9-sU zGV{8Oy)HVgC2j1G2x!c;zMLO)*lpyt7*#T>W(wr^5g1CTpG$7rQM1t+tdOF2MwHoh zk7t5Jm2WEbZRWN>kpg>_H%Wv=-MZIaNS%;^#>EqSCxa+5-tstSRu=C~O}B6)q>NNm zDOF$0Up&b=HPez=A>$R?ZCdZ6YiM00@{zf)eTKdtwJ#WyC?`+RE7VwKUM-8FA5NnTok)nutv+jBm} zkIKcVavFtPVDV@jVN>gPts-VJ45d12LmK6e`P&KhCf)%E9v=U;yzMPf$;Q&rLN9wK z@0ivmDC#&EcJ5N?o2Rx<^m#d-4>KvU?H+D@+}A%D?kpcZO1SR2wqYelryGPZXXvWe z?GnF(v1wd)LfaNfs87J<9RR$3s)Z+t*TEB2YT(=ZTo2FoIp3o~&CWm)zz&V=93J=` zrfx=*(xRE-9iAoP9NdW87TV#~YqpEud*@e3y9Y?gS#Gb#aIdde7;b5+T5Pw!4ZkTn zCu^WFHa9)HO(4lH;7)!sZj=T{bY{a&ggUWzWx}G%?DcAR`tqLr{JL&Pm)2lEm2K6H z&El=46#hi-QV?qcRn=QKdS|jN!+G`^LX2{|ZxohxlhE7fTVXbU{D2;A7C|5~u~hJA z>=l@_BdF_GSE{!*j92-X&!;cRXxU@SVL{jE)48{lnEB-{-n$Z}+Lk^_l38NOr0HPX+L0e}AI|Oa|!H8Bmy^F@Ha$uJU zm$`k&28t!?rF=0n1WQo5k&;?!?yyw*!)A6&0r|oH%G_#Xlf&9Av;K(ADw%p_xuwFq z0UL(gUej%Vtif8Gm$mw@yqHS$9IOUxkt`;=AfzvjLC8h5czsoLymY#SPph=4QDMbD zn~QtxWEi{8-mZ!Byfwj7O}-=UW-57i4_EzuA%-XI>Qz zWmZtpYF9L_9MfEk4|?M(uyfO&AT!E&&Mkj-tJjXE`xlIA*_Y6ssqAm8+e|E{UhGTO zn)vM7xx~P9#V|ETxZze)-{Lr#$?DF!aKRP4&6a0^-V%8|*)v^fR&a;B_G<=$!2W9x z2l@*vCATSzBj>i;3ll!8no_2=Pa$$vRYfWmb>a#5fDY8Z3M#fmO@EscFiMDuv~#+O zdQMhvl7tI(C|ay4>gNP%P5nsDY|$aBEF^QVOrRTypV073=>3D@B)4j%z zZq8?W1;oq;`0^KM$0F~%rXU>TA6|&j(oHL8 zoD|0td>pC+_a6RR88cUycly$Gn($UDspNUl~B-mHdr4!)b{PQ=^PVDCL);LK;=f@<@!7BzW2;lc zk>9G%CNTzUqZUU9nL(~$jyN^vL|K~ESuRv2MQYSU>a6qprk?BPCS;&sW`hx&=pe?!QnXcr?fCK6#v#rOC4cK$WjAd12lyOrUA7<1 zj2Rry8^&QtPhu%gnC+p44 z;Pvo5uIY)Sw<48$-%91$@0l;C-$U*d&9mfB^of7>Y0T?R9aU^(o-@|@9u>}AR$h0) ztLGs7{5hdeNpA7|(IayIHuCj;{W5>SvjIDw#0svsC+o$WRtM?&lV-;EAQoOC)~mEG z&B1+rfV+wi<|@Qjy*<&X(zS7M8@v=NAA{dYnO@F2P`$pS8GMt^Y*t$#P0~~QX_+RtMXrl|v zcr8sT`)ki}?JT|JyJ7rXT-b5iCr>kggNuo@en7HN;AdCBNNByYvgx3sR38~0V>V2# zsp?fGfoG($QCliFd8roo}7LiF( zo1OS}PSEpC%DEeMp2&?TZ!e#`E}Mj_i6HdYKhM!^(KpWD-h$%n4fU_GQn!f$m?IvW z?k>ch^AI9d$scOUY96v_8ZFWq7Q*XPKRu?apmv4Im%Mx4#%Uqr+(=4&n#zlCk*IyO z2L7f#tU(`c&HJi?KiyvnYB)BwBR{IpRn1s5sGE@|swqxo$Bw>#5b49{1Wp=Q4LTl1 zJAyeF9hS9FGF^lcP)_3hPG0j)8MO z&}=jrN1GiwKlvIK4-NqIjUQgOGc+xpoWj$3QvJoV(ztlMihFTko=7!{U2I@&D2>SX z*0fu^XRQ|zxRbMD#PcL2#)8IeFj&o5pD|}f#!+3|ge#$>+-#_dtU{=hQ1ESLSLs~g zY&!Rj%LpBreB1T+-X#OLCpUk$(GBZKtv_|h#Yg3zF zDR6H8=ZBDai{(}QKApk1!!)8Dm)Zxh>97}RzCLz1E@S!=`MLhg$hH8Uo-t$GbXs}l z4`0rAc(5f`(jucppP1nO7xO-uU+WkM9o9ixjAT@F*}*PS(fmD4y@bMoV{#j1+rUs) zYIs>*8*xgbukLB#)=#1A>EE|%m(H0z-o3eI?&^>i6j}vk?{H0wwt-Cp<98<#Hwi7k zFwE{Y%_-vyn_l!gg4!Cu&eyc`PN70q%{v$k5VL{t!v%Xo)17Y&z6zEy>gGKf=2L^I zQ54pLtdiTq$&QHCh`fPIgZv9`UT$;s8?kc`h<{&w-{eH=PQk%$ z)omCY%9R~oQocAuRZ8&svb|oV+4Y<#Z5R3sLHq*Mu?!tC4(@@~$%stQ0c^}pP*Aty z$W_g-AwaTR-m@(ueVn)3W3>pnErb_U?I)zPayaBux@e6^ZrF&b+)sUd3@(QGXE3=) zK30q818Ck9WajH9pRBe#nM$D0_rf*`gVl2ChoW=*8PC1*74MPX-^^O|r%NH_&1FsY zDg4$>R_l!IOcrHAq?bAhRaD|gAzRl^AlywJjPB~XtW;_gt= zuc=4dH}hwukIf?HA_eOVK8OQzqA1OD4Bv}wV@PP4SkBziG<;Vqpnq*yhmFtEBA2rC&r!Wab z>sWA@N6VZ+Y+o)mA-`3)L2p;tu@3cF*fTk{maDlE2qTu@MOWU;&sds91qZN88RvWp z%3PIHskS@O=>^Kvk{Ly)`L#t7py`t%#`WQQVL4c?ckZ)MILBVY=gYU$r6pXolCSN5 zq)lBK(W)jV*mls3x-l;{F(y^A6=t)P^R;SFKZT-T*KHs<{?-e5(D%9zt}D!G`HdFE z%lnOihQY^pQu2)apG%6mUw9oO$Cs8_e6xh3)J*pU2KRKx?OQWT5Zigyd&GIF_TmqV zM?c2UC9hjPQo24Txz+*NUA}2+-t+;WCnlniR^Pwg#+UBF;y5Ng#%!1}KUQQ!WE=+0 zvta!|z9j@8ec6s1#uVW(?cW!DFpFtE^uxXCD~fc1eZzcRG#SsK4+XJ5jP92-P0cM| zTvgEUZS3rhdAbGtY#a=x7|oAFxRt(2A}3?3D{=%2 zay3{}*2bwMcZ@+X4sRa_DxBV5nqx2XF6rPQzoAE{rV|mr_-ruC-hLY0;cx=bC*dLz9^z5XZlgX z{3SBG>2TgN!J@YyK^Q~s}8#M0^DHD9+4 zu~>2pOdUXFqFH}jg=6LYW`nLE!8!ibCk?C<(?6LNH@8Qi-Jp zB)rx7g14E>3%+(<08$Nq@iUHlY(o~WqkHNMq83NER#ez$C){*uk#*GNs!#am1Pl5X z$@Gckph5ou#sP*|RE-0`ZIlT4USDnlb4G*$u+4OldGAVMjnt-*cuAIrP7YT$@!gdK zNqmc_zw>N{drpW;Qfb%~#B2S07K{i~P@$Js1C7E>afWJ#s4$lF{2|W=YaG*wEoTr= zMwP$Z!H3uJNZ%Fd&DC*hVo-1O4%D1}{9MBeFQaI=Co&{wTcosiPReB$Bots4x|VOy z*?DjNLv3cW(%IYnaUEw6=0mN{8Bk{z6sPNhPDkEO0#}A!&U5tMMRY2Z`qfe@8JMTa zxgrJk7n{1CjKPJ@INMcoEXHRzB&V8Cv1}}S^BwW^mGT(m-_fJ?@hEeg%p;O={!tUF zx&L(VMt&*S9cr>|w7eI*5t7(WMlMdYY3)!#Ri$R8UTJ0!j7LHGx zJqZd*H`Si>@M%Z@fz4WjhK7cZG8Zt)U4|cFF63RyM`o-(L&FIwi=Tg875O~{qOA_z zt5dBVs|3z6fY|7w9{RAb?2W~+3T*FWX@y1PN?Ps+ZgienEYjF#zFfFIoDckX&5o^p z$}s(MXP=X0xf2AfJsyeGpui$-XaiUc52pMHmc6No6 zLPjWMWj!epk&!(*?O_zMLrE%oli5H*R-w$Y64@jB`@YX|A5Wj(@AbwHw9_C(^Y$}I5?;GecZ7@_;g!4^Zk;diQJaU*?#v@3V`y(H5)>smj4_H2#8EA ziZ@G!tgp0i!BWGkA!8#GO!1kH58Dx{(`LgObk1IM2wunRk?IyuTkZRkU6{0a z5~N|O-t^Q&sPytSmrj#oh0|BF&z7Cmjk0QCOU)OPpBwECU$TKWa6!1KNuRmyn6sfZ z&O+{;NmTunN}wc4aw?mDyj5jBCAoaiuB5S1)NlwP&)P$?8bgHx!b&sV6JF9~;Fy>` z>h)!ISLE&J){VU6m{3spd&@t9Hu?GL;G!pO!5Vpy_z%L*#9VF=+0u$5}EULQt;Li%=)0{ z@j$>xUa$W#)@}E_mPS8zi_NoV>ScdH za&9(=lPs&34^B;m@wWabPt7-;^PAd5{5ifDd^wGb)?=iYIWi^Np#JS3=kny!7PGqb;(qz>Vt00fA09bS8^~~)V|K#sRrJAp zZmSjzR%y${D4#DkS}rvDS9r+Klq|2Na7DKKg*}r0ai0P?$^yT zYLCnu6dW#I9DZtqRL%66US3k$WBIx-gC?{L3NUDg=RY1Wy_;w&;A{VdL85E@P2o)# zDXO&;;mj)$)^kpM((SRE&PjGAZ`Xn-UgyEZ_bdy^Z%!Zo()#iX#C*#}Km8-_=Tdbg zx7_|VyPl|usbl*f-;$v6PRUn4BdR22`1tokLlB*II5~%Vqd~ z^GN!aXru=1+B}Er_|l)Z3#MTMwVjLo9_dSJgUpY+hbC(MFFAjE{d6|9cql?Nqe^so@$)CnUI=0JY!){BpwH3~6wx&*n4=w$D<)(!?!p@v1Wp#R}7`py&4xOGkreZ#Nnc2K_A9U0h-XalpB>&kZ&ZGqDHt}rytuzggdtbX)wn*nQ}j#zrMu$)=suuW1ll`ScqQFV#jh$feo9bpF-vj`#?JN z{5C=Wal5Z1MEY91<97bI*>AGvfL8v~`5ks@nkol=#O}WD|LA>hhk{eZp_u4|SBnyJ zXKxDQ=VT=FkWZe;ZCDyi4^8SV$lm`mIqzV&IT9`2BD{3?t4;wDSB*1 zan$`|dfXvz?pIY?QrJ8ie38Wl5>t7+{MF3!JPL0e-aMPwE%u_PvDN1wZ$cS=LV0c# z82!QzbF>gYStoWoxKV0`I9VvTA>~VFq7~rRTIk}TRrvkeh{bOwV*}5IQu3)Qb%GJK z(4!FZ{!`@ew#t{HW%7{qJ?2~Rqfq~f-)VLEBO-FkCm|?0HufhjfVqfIwx%z^#fZVN zR@~*yOs&F_Sn-CUC$0x2h5 zX{(?t6iIn7GDmxy_?n%4a(`z5AO3ho&RI_+zq6%?-PrQvS7i&ex(fy ztp|PXzblz)cb$|D2#Ffa7!>jAxqbMaR}OpHX`~8PsYKmj$UoI3n0@APZs%PA;RRJl zmyOT%J*nVeRsCe}E2y^LJTqHbYPmYF_DwCe0g%cofJ3Ur^pa2Ql4~ zSF1t&^M0qzdr#}#D+@I|6u7>`GTL1#I(pox&9%m_sKcXbQM$rGKv-`)0=$AU@D2oK z7m?rydo$tM^l&yZ<{H)aYhljjc(rU$8n(vK#;-<_Wi6n0m zNXk&GmW7hqeu#DVFn>WR55xAmsa47hzit+`Z;<7WpX(I8Dg1JCK-pHV2PYtskxgf$ z3zgQt^B|#b<#?%ZHulflwCRZK%Q@4LWdRfKu}5JGt(9Y?a$6YXetUfL>ec)xzvz)L zvvBpuOD#xcJXkk#FFw@xKwpGMN;La?*|LY2w|(R9)4h?Z#qLneH7(KN^-Cva-&{9O zcR_hYUdmX}IA8uZz2@bMP@Swe*2Ly>H{toAFV@j=gJxhT2MtDqL#^EJI zLeF`bw)R+$^=6l}O>`ctN=no4L=t#k6PDRkulE}(lq8z+UugOjuK7-e(8>f#635j0 zUO9AyLsb?SoD1!N>zY@$EMvHgkv697TN23{{Hy3r=?n#$oqO`LfCIev4j=Ds5vW@T zNSPJ4p9|0oKw{Ua-F!+tt(Wq=QkC9`Z+;M!6c+XLWM%(nN3FY$r(0h3#H+^j*qSaY zYh5??QD_~>F>Z5cPo{d8rE}f<hj~`Oaptv%b zLd2`BeYvFOu78ojVWb*&$|W}9$8d3V{`&^?_s?2W?m=L&r04fY(Y(o!=!J@V6KxKI z$Gj)wavOPq=7X-Zk43i9-E!}Z*E6r0$`O=SQi5`tMWkf+@+CW%6n{F5B|Z*5w?0%Cjkj0d_^1d5U)?xG@}Ez+x`%_-)e7$2;LV*gZ#WS2ael51 zN@J>e&*mIgw(;@qExFaA=QKlZH?F0SgeFFP9nI@x?_*@K7w;Pfjy0G}8mEG*Ka#sN z)Fk8l$G%rzU}=eHJt{SKdNg~9*^KMD0NQRKun!q6%MC!CHJ6XlnE*1aq=u0;#L=zvvX;4Pyws&Tv>wn z^mH0u!d|3A({oNdCgo@N($r#!|GVNVt}%Tbo{=?YKO4mTa5Vkn>FG8m!d%^KyUCxX zRE5)vk$-rf^@8#Ppg*7Hr={uTa1#h`D^-;cu9smX5 zgUjPAosz|`EV=d;cl?w4yF2#XmRCO|BF|ngnJPacHvZuRk^$0_oPk2VYr{wU<$0j2 z{Gvjos<5S>d}6|d!oXT2q*dWB%h$w_=3LvWlc4E0rS~m1R&~69pPggFPG{{(?w}uY zgDHh6UGqp#i)a4jq|UFeNX4v2gk0jR4H5u4o`RHe3Z~UjY~L~k3Z`tw?%Jf+S3N`D z98@_eQT&TFQTLFZxbUf;4I8gGLbrw4Ye-El$!)T#ahcCe9B`S@Z7Uisy<$FY$3G)6 zCkJJE+mGirTofpyNnq!!Eb^Ky9B)Xp|3;HA9d%Z}lD{(9rPU+%SOjgqX&0S%`*_g( z6kkMDJB2xb96hcjH+Cw@{fE`Vo{Nt>1B2>=%FB5}3=8|m`=i~2omsNX-z4jGECe0X zloXt?0t+tbH?a%DGC;QA6{oZ6*^H4l^X1ah9YRbd5=9E7Cq9)#Kwjs|h5QhtO^B5j zS0R3$e}k|PVyo-T&1h^kdKo-OWoO#a4Cf(j9Yc@8_WB6N?Szy$w ziFp-o&VWOIjvcyOd%cK>gWvBK_%q6pHY|f%3>A>biNl$}sd<;>G21qu+I4bDBX%WI z&V^HbXU1wvmNOG#J!T(;1jL7$@`ae%=j%JROL}P5j|~`eSuJ=%C(tzGgiB}rP7(R# z9~#GFRiFI%p)SN>YV_kit}#i~vHKdGNSM1WX!k7-skU*|DMR1wJjamkCzq=2E?PS3 zn&SoD4O3(N2{HHkV?368+$cu7QZ4VXJYIe7Q_t-~frloL z!X^dzrHbX=79^1`Sk%^#KAE9j(VW}5YrXSEUnHS&?sAOVczD=CLFL#aXN*78>gM(L zjK_xt%;9XFe)DVDW&s#9vDx1chmz!{Ov(eJX1A$c}AVB@%kM+$x-`3#? zAn~c~J7O&WJ{Y~9r>^sC=()sXqexi}O#%oLO7kn>it!H1LWDS0Q zn+Ntv%a!(WkJ*N{0vGqhi@zQP*zMw@Kc{y(dO@=z-_7w;ctqPVxX5F~Yi8u#mg42> zqt{1#6c+weC~P*?X%kr-7rANYJrVgmGu2C~??g%sCRTuNgjKbGJw%@;rH=Qy5)bsM zQXk?Twea!r-ODj|`Rtug`ga4O@DYVb{8j?kIr1<2Sn$X6P zSLnk&gq^4im%A*VPftVQDBKHuL(KK=hPTn^zSy-CrrW~eTH-y-l9f6`=+;tnQ!&2w zm3D;Mamps9x}P#8h2@LGj`s^H2J^3Dj47rIuJktA00On+u8MaK8f0%su`i2kWTluc zwcf+FYNkOCOllU3F}g!Gmijn#_35F8f4=BXw!hycstRqXdz`|zHpao4Y1csH*?-9X z?Xuk^psFO1#tEL+gv+xBb9GhExQ7g3)%D+;tv+7LA(QKSXI0!%5Nb;8-OR~*B}z9? zI5HJD1tcf`Y(2}3G0VHE82Vi|1xf8|Qn<|DHa_r1ugM(#m}FPoDTX&l!A(GkJDZmH$@l+SLy*~|GQ8{p?Tx7_IHlMYRHLkk}#MD zIdKl0*s$_!yJ0DZ>)=KfvUUYZY8$9Ius`TQR*e~}isij$eBguOz2ZnCM!w@_&)~!l z&4l4$m_7<~7$~Wc(A=+AQJs)=%vEv-KrNCXaVSf1?R-0J&_y*6%P)*d!$+nlTi59o z|2b|awbrBS>mE6>0kM?SjL0yj4_P#16Oejo;XZ+eyPvfujcqNIG|hIdy(v5>!-n(# zo`E*&Aw3bVlEl0Mg^Cm|vl8n~xcJ#!gmNuwIP%_a7wXHV8p1X+Nt4avg1!LcY6frF ztz@G^Uoq)tw8UQ*DEe9Ot0$5jdzXUNm=8bl`zH0qOHRJrCW;#qS|@7aAH1yDW>Ja2 zH&bx!mY4da-nl<|B3OaI_c34Y^N@QpUHVAGyD?JGY|EkWx^=6B({}Jm5|72HBAX>@ zs91Jv{C(pCcLVPiS5-bcMw28ETX){r4rx%}+JKdwpEuVSt7BHb@8=&s4Dg*vwus3U zG%IO!n65!alK0ZdOcX^Q@#q?F&fOu%Z?>f{Dw%&(5mf^umO?_r#>mD3T*gWQW}lwC zUreW!I!YtB3m&^wcMBo_#6Z8qXr8`{9J6e}oWJf7)sTzcdTDlq^Q(n{ln%U<8y%{u zStqbHZ9@vr!{@7#fEF-8RfgeN1^S$D^))v8p7HAoYJzhd&wuWt_)WdX$%?<(0tu4J zV+(3IJDxvq_G-?aRIX^4F1S}PubWzXhB2-4lGtD$v_)g*D5GXgYG2A-_8M&WeNNZ1 zPD^?ck%g#-Xh(u!_ig`0dGxylh6t^~!q-_~#3OnrxU%<2iC$)Ym`O3hu3n=+$r6k$eFtNrE68+uRQH{2s)9#i+ zOMm;h?Ot|)4%fG_@B&3KhTccM(p3b)IOcH6$ehYG%o2#D+5TfUkR?S;Ks>mLyU<@y zE#=6Iw~1UeSK2iipxB?EPd<||dT-kKbam#8rz4hppDt1A(rf?Bpvd5c^|bRb%|F9V zy0=Q#u>9bCfBCgsr_o%B&~L~2_J;j@9U+uME@wXGK)ZTa1L|3y6j4=? zQGLC4u1L2$e_G+y#o_UCWkil)e*-hv#;n^){{e6)lVEd)yic1tm>a~q@1eV76H>lV z_naDPc(QLU#lkB57#0LBtwzKH0Eu=QCehD-x`0Z=@CmaBaDXQ*zTIOzu)t30-u)Dt zs9V-aKRBneYP=d9B(#jyeA!I~)~?qW+cII}XZMPe^(h8HRT^}*;iid?iett6{P#X@ z9GplrA8?*7h_}>U-~|o{36m5<Io)5e``MO zu7rTk{b5M9Aj{~YFe z!4N0I3r<;`$!1R?6rDkoe2qVfL3lLnDHxI4(>Ez`?FNDD8_12D$%RlT9;VuPz|e0Q z_K=Ow4(8KVOjoyj^|K)B5pPEnz_KT5XF3wvDwQPZB*pk*+_^I4`9+O@qw^jKAVe0c z2@>Q=-$&{mWtVnXSSK=jZ~EG@Wjb&`M5pqXUc0; z7*eFYZHRETxQVBtfHH7)*eRM|5#MmN`KZTTLfSF~IxWx>P_Q*rYJ$!Kb#oEy z(zmy-!Cs?0>pC`Gvea*9OiOEkt60KxsZ&Tw9r!k4*hAUJkk9o#Y|bPT7BC4A2yw7BeFwrU1hT+No)(hU4)z{8o^phU~|V6mj?59Rp%B)-3=>^0bH(`>OMrJ~2@0qi_qA`Fq&yZ+z#S(185XrsF(Te(@ z50xaSeI#971rF7F(>(7fBDEgNzrjh`W7vYlcf!vuALIT#XInFiJJ;6OxT?OGodUga@BQc`p~c0?T{0V8=r#+104Fas6U?^+J&3!jYp?E%#)0Z z6I>B`KN&9fwet5MvyuqES$QE3m2CCoOpQGaS0f~Syd+BeKGqLK;cT~UT!|m7rHBJ_ z)&GYBa=XpH*5(AV&f+0^+w3%qu}*Yg!_Pp5&2TY*GZ)UIxH6126HNWbk3X*?r&%1C zO2A&zUphq!qMw$-tUZ+BwfQKCl+zxA7500`j|8POUl5t+jpWE~Yrt<8BLJ*$rZUTo zj1?GQlVp2}q;&!<8lUTR_&vP{_pQd?vwaON$+at6v;lj>h3<+udKm(1y-)$#m&=!6 zKtqU;8}1jlBSAwl7R-2wPLOnza)ms13<-NSys)%@Tle3VMrQTzbd#VXTY5JvO_Zhx z@XeF}!~$p8NViUz=>nN$I=Hi-!H7o_whvUJRHy?3dgch&Z1;?FP&ON;7!?Lk8u0V? zQj{lMD5I|q!p^Xv9G#OQ2pMf;4%dN+i@Io9`Jf9EnsZ)mCG7^o2acJz>W!6)zt#gM z?tI^alcgw;g#&O3*h5(o|KCIwYlgCd-ZX9}8K0OZU@6cYk4js?B#A-JA%m-fj#87w zC6cM(q8{GHKGLmFqM<-+`8;x@xQZDK09XBmJKtY``#-oZ_lz@+UFRYlftF0%s$&}& zaU)xijbZ*l_NyLt`EUE>gZ*aSpeF4E0Z%qe=6@;ot`8^B5U8hRl}0WuiamGz$)*x# z9`j&@hWqF*_Z__b)RF{%l@}L~3zrJ=rO5<{4P4=Lp27J#i7rp=fkF3gpG6RFFpI?i zB&hg~e?ViKfWH0UI)|Oue*@=K5I)B;A$PnyIPXZZc=dIquBEk@adQ+h4#A;qc=*JH>d#TNj>?TPI0D?rM zI|E4uv}l`Mh%?_fw(YD;d$gS}+bwHk*a&U|V8}$xG6f4h`P+82VY|0fw-PA`$KV%{ z8{ZtAYy3!-iVIOxGAEFW%0u8gWiz2DfpmZ{U}pP!t1hcE%YrHN_2+C!-{KN20B-)O zDu=ts0@kTZ0?lEeWA}rE$CVcr&@d)L7k8ToR@`GtwLx_pi8mtg8^+84rS_G_$u|6L zJLqNG;Tp1W^!O)~vV&sk*PbN5<%I?A13WMKh;D`blXk_{+i$nHPTa*SSn|xJi^oX{ zmY{_Y9@qy<>ir`{O(ZECKo7yK1nDT4_3Rae9_}(HcQ5ZN)sB2mkvIzO2l_f7*Z*z1 zBHPtQZfKZuxTw4W9dP;#L9Ub09yYeVeisSYEW1Ikt#;B0&-$Ye*RDvChua{aO!{bl`ZNY14Q*^(7sNeBZok5QS; zk>FDXH&_WuU7;Xyj38LFE+o zck(FCpEMpM=>hCGFmp(#_*Sx&@bvOQ6_{D`D?KDgR@P*thx;iofyZ&zc;P^g+)+{U z?AuWy==Bi4jD6)7IJUyAsDUt?(yF^wM1jhxF5oX}B0Sf`Oe4teZ91hos1b6q%9MBhXbX4I!exNC^xLND1{1fhiaTMrb>j>M_(Z(*>AWrB}w5hVr@Ea!N@GKo7!O%^Z{LM$onsy-5WFkOi#t8&(r z%sQDxAkELb>u*NoQ-JigOg;-Vq!~d_#&`A5tXyYWx)J!p5;I9n+^{liSX}LvHX3V$ zoipwMKcn;+4KnG2bYfmGrS{5toLym2(U{T}<%(7jTa!=HulY%2RgDDnTW9_{PP~fmy+VvVpVskiI#ia3*c#G2D@+QpA(?`KO`eSu)T{i zTA4T>D+&CTe7?;Sk6!HmTGexj9*ebh(%4({yFliaD~(u`ZLM@@v9%YeI3XJs^;8RD4v8b>^G3Z zP0{~MpbajJ1z3Um)$d$h!TC91xRP~(k_I=Y6#^9ysJEL?UxP+G0oYiW1J2GrmQQ|; za?naU$j9LhPj4xB@Rs{#)?ni(nY!---Fz@LA%R3GD^mW#aS)c)JETol=nYbph_>&h zWFd-lu>aox5z}c#*AcLH5rm~~q#*@Uup~I2S zX??AnG{hd8#F*|(@E;&uAcjpqWb%%`5FPb@l*x#Eja!kgEl4_KT9$rQJ2*ngU{FIA z4{cBbkie5D6DId0_A|3I{$w%tbOViaPM-HW5vKP*Xr9q{MTxdg2}HrA4nou5xso>e zATRVlXcpy5imw3v3>`7YA%m+rPz6a8O!AW!L2S(X8Ih|B{EDSl-~spu6$!IRaO>## z)3&HG!V4#%j4#Ma+6wP@I73Gb_?+2rL)@7V+oQqE=1; zih6$G`z2bdW@!dHG=FrxzAA2-Nany2eudo&ODC+pTv-`J+z!Hp&;+r#}0JGR8Uo2O!Fcq%@jx3Ue~=1Um?20D{%=ujp^iNHbd<#QxdtKg)pIAph@*4FPP z9$5<{e6}F*KlG~q(RRm}DX)0>-M;TUka-cvda(xp!e8P)-}uYofSxz@Waxo&!@aB4 zD`xW=EIYJV_C25yBcMSNcACt6zi+jggOAM)!s%PTvXRM%;~*le_W$Nv8Rx}yhD__9 zJGcW&q+coU!VA{E=03^5T(9;}Ify`7u!eRvD)GiJg_f4t;4mknBNr>+2C=`% z|6gL?lLxMp4qRCq;S{JQAaP1*2ypJdb5F<*u;Yl(i8&&U`t?=J36PKX)I!ZrsG#Ov zk&thQC^H0+)2jBo2A2dPr?Jxg0N7CbDe^wfJmm802Gp}+Pa!*nEEXP;rGc>1(9csg zWaz+ue+p`9lN2f}R`3nmPXfT6d$jot3Rj~WVaZyIBfQOEl1UvQ-hxXPo= zJWRIrt`l&VZ}tkzVVpnFmb8;6*{djD0AmH4R<8PuPDIEnXlVLB#Dz zQi#K=d5-ACKS%^v8;S3RyCld!ig3tzh*k>j)_O{ERRKg8Ng%XWs_|%68>InIgy51L z5QVr03+nE_6#eh@D4_wZE6+DE0c}BDE!_?}A_SfP5TaQj+%w~1dibrydIAUc_SbFv z*FM4cfNCXsbuY4ep<)lix|{gggHbr9X5A@7$iT!Rapi1_roz80mxQF`cUs1wNEGCb z2eMD}coHExvO1*mB8~)edmQSHB7OXiu22g};BZ-jIFPkeu|b39g*NE*o2rt$C?qJ8 z6g!*bfoYmq0>WAh!5V4WAu5lStN0^eGZ4Y>SfL*6-^R+ouFK*zN>R@fgxIJ3VWR*- z*jyJxLcwa5!f*JL07&s~pCd?-NQ5WG8R14?OOvB3cKX#)XOuUj^c^k9n;_9#kdrs| z7chTDG9;@b6B&>p3G}+R;t2!fk>XnEYtn>Fil{1pfJa^$eM|y;FCd-Gt1aRtxMngV zCAS{6gFRYiHgVm*nWaUlBIEB6IQW0>U0}?ft@uLrJ@iOT9!`pA01yuTtM&}{kpyzN z=fGqn+5H$zNSdfPe=CY~_nZMs?CUH8zHa`7F_`}UE>6Tft_r4P7&SFFqJ9juCy1jG zLW!oDX#P6M1J8pyrrUQC)1^=wNs1e3AP&Ikx5c%O7GF;htsbs)sYLIg}1HvXv1Ch=a*^=Y}uxcOfZ{4QD8 zZ%kYt2kQAGV&VX%gVa3fe|VAh3RXXzzX^3O4rVuyFq>+Y6;AYFHzHB+Fh*{oR!04L z&oLn3!2$kl{|D61ikmTTJIu~ZwcBR6ZDGE^iiw)3m4Df+z7gbjf4pB$)SdX9Z2<#X zA9oQAQ6=kZAz?1xv0lhUW|%lB6XLWN!G1hLek|gtL83tERi-p4n}nP^0whnvxo^i; zLQ-oW+e^s4o3h|E{WBo6k3>R)8fH^#6wrvPuldg|1$8yth-@#VwuP|e4zTpH8BzEm z9sR#r{#Q$fC|qI`3Xel(onr_sD#3tQM@C3Q>G9uUiD1!hC4}!G@<*L|8&7K%V32xX5zwqJ#k~b_I=f>Y|qa6Vn9xC@t{=yD! zkVUT+;CzY>C&A5MU@~Z>)1cJ8_KOuknO^sjmT=io*a;BrzF{0FVkA;LsPj$!tJFDH zT0C%?&=lG5VXQY;Ba6x@1jH*u`3@c$Y8}7jX($=bO+l2x* zr7{vxrs*Tx==~5lf;{3(0-F>)*u1NJA;FN< zrJ@KC{CCO4$WPS16al1S`4K~6v4$JuA8W)#R&r)5B&Lmo;7;!6CA)w36gYzMfkz|I zmOABn2#l~_?b(LI8C3I$cdoAHBakE;xPommYj2@B1!D~~W>}RsixLbNE=oUtU3{$V zg&r6=|lfu`=sYm(jp0I{kJ#JF(bIcS>nK_jlb-1Uo$fYW?#z??J5W5lNlE^D0lmI7fQ8P zK@`ro{EH4s+!3FY1GW(X2q#T{qByTc|~lI6$mLz%_(Ls^`=E#R&>K z4Mtf~6bH)u1o7qt`<=VtiMFfPQZ!SROC9L&wx0+j)Rd$lH3a9q?SFICI=gCu*Qg`YgiG)7~8c%EF`L-|IfzQ8Y~AkrCuz4GH1)azetCT5QjWiU3nV4 zF(T#;f#+8Lk@z3(*?;9Y30@)z^ChE57!j8EMl}+Mi7z0z2*ua5=z~M?H6-3%wXG*p z(m`OTM`{$PB2)lg@+Z#~FL?#JAPCf;jzBl?SR;C~n7u#|YL&qB*S`@RY?Y2BtC#jl;dy>{xZb0@cwcb=%JCq z=xB0=!hH=gl$%f)N<0szrGt9+V=zNip2C;^yhKc1mhLL(S)WmA81Z>IKz@}m6_)o|a$3aN9Y#geMzcHu}^;=r^jK3?>{52J%X^#>wztiBh9 zJTsOOHq3iY)t77scv>5WArxVDwnx!m!tMiK@Km^N5ZaK17$VUD;_D$vNvtl=Lr$R8 z#9!<(D2iQQ@voxT6v)b_lUGx6xsW`dFpuZ&ajHiaNyl^3okbHrfzz~+h9URtAY>8jp`gk1dJsV=Wba1c5lt0P~ag$`0oL1g-!j2a$@eGX=n#3S1v zw4TN=AQJLerYd4$LZsQUm@LvvTKd0poAy9AoVWyf# z(KY$lR|sc3HVGmV=o83xO)1s@hU$<~guauuZ&%hK;mrbAO-3N$Hd+;5#l(QvK^J4= z0KN%TiOJe`374U$i-x7A9c;`0vh@GZAp=XB>|!RP6pRvVRzU6fOXS_sAkYs&T$7_E>!?76Pf2hgvI{_<{lE|WaM8`AvSx#4_K8# zATaj-(D4O~_ri&WJQdEg4MH#UNfkJ9`voA;qrB39Q*i`TD$f-(|PtW~AIYbG{@cg|T2` z%=ggLYVTq#jcRv}=u5LZZO%JkGDM&B0{RzJs6ZCG`VARq4Ut2|;Byp(u3#kTbjG?g>PX69jPA@CEx7lZTv+1W# zHOl(uryNgS?+yx|80q{@cOY@*Skl1lit|hcm&W+&sdh7eSfM)&pVYyzxG}8Eg%0|XI5?gw9eo!+vU;y zNPm(XlcB<1(sYL{bZN-~gCD)eEh&yKwuyFrU+%iyRcL!7WPQbV{iVf&Z69m<@$|vl zbUL@9X-MV}bjuSoKY?$qQ$1g}z0h^oh?Kv(d^*PWlDCRTiQoA0vEZ;6Kxl-zka6ML$GWzRJ{SQMo za3mRzG?xjP|KP=RF;E;~W9Q`jJ~JD_KrYw94rA61Y`nuk^jTTBMYGb!-l=@c^jqV+ zcP+&%SAEAo4T55>NmO)0~a1!1St7hmYQMvt1yqeu8zNPj+Vz#LMMD zwoVwh<4X7sX~EMb+S=fvjA@F18h_*1BWxUB%q?Pc$(t3x9$vbBpd#rGq1W@pakC4K z=FOj5ysk9QgPDuT1~LBXc=BB`|Nbp4IOSwS48Op z>4VPmgTnzV$Pz>h;$A?c@iHmW2#Qvms)2uakd;kRQl6pd65Qp$4~htBXC(X8$E{Fu z7IyYPk$j1Rl!2zZ5)`*!1(CEjKs7^73t&{7@Nd0U@^iZ1Ee&0JsKgLU7ba z-V#-olU>XhE>-Y|b|Gu6{0-6mS`W+ON5V{vyD(-iFf)!{INV=^teQww3ML`D5Lp-6 zU!7|Jr^0q(DJMHjcftivp!(*u_%~Uf4iUG$`9CA6%j0F7iT3+^2EzYVAIML|zGOfi zqfC2M8VFs5n(xVi9boTZfEEd5gJA|;?|hi6>wo6G4z$e zvR|C~=O`JH@ei;D)EVitl7-T^==%hn->gz!)Q3zlaorQY+u(g{164{<{fw-izavcP zxL?uArfzEW104(|P;VV_Z)7Ka+&qa9MB_@?%Y?;mb)P`lHXy8H-H;$D>+uLvVX&BA z9P4_S`K@BW&&nyvq0b{ZeN1e#on6v}H{oPOXn3$;kzPunO=RcOKW(N4F%~TAs!DSS zNCBp&--r~3YlZ11C7GQuM_?F>W8cAZHpi!GL{6-sG^B62NiQs<@tK{Zs&S7gO#nO3 zkdtUEjZt`4igr}*g#Z8Kmpa}v9wjI7wJ?YPN?&}%iVNjB_k1)FkwY@2rAZVq?)STI zuk(6UZ(kig@}RuD!XEI`bjkzjGTdmKBsbV~-gctpW}l$dk|2kW88Q@P(4CFYkF>|M zYm9l5ccr$sa19{EkFp#R>x@tTA>Z{HEdAR$zF=;<846rUoEz=NMT2FQMID8S+u-=_ z1nf;1A&CPsChV3-Hcrkz4Uz1Y=~MJ%(A;whq{x%I421CsY2lbTvvbG&BjevdElRMU z=(Cpip4q#r0*#;fM6>oP!Q{wrX$S)wZe^Kcs@NhYzYxCsQoQrY^5)r=Z>?-+47~bO zLAxbHuyJnv>>!oPu0byR7F5zDiNwPqGAebWZt?B+cd6>sPv))f6(g82iv>Ftt1XOn zzJLP5Y1&{wywuIfu^j_?Pe1<*Gx%`>@@x{WQr;(2icP#HmY);kO&_!N1BH^8`~)L#ipFSu@UF-a>UK7E zFNC~HseuG{ry1c$X@xp*HPJ7ucqh)$2CTvMQ8~EYkIR*oD%rRe=FW7nD9RiX`<}A? zNutu|s$_5kWW!rDUL7Z-U|pY=W}EK&lX$n5@(LTMFW|RTN%UAm%TM)w_JxXhEwjN~ znTt2<>}tZH3;eL$H6f+M_gqs)2!X{{CHEq;-i8iF01<*CnHJt_S(0 zbr9Ag47PFVPsY&~7XwzP^T*HMe&K=?|3IQWr!<>SCHt|pjgo=PStpM6Xd6mZis@qap7jUsZv3h1%7F(38C%iBT9RBIJ%U0toP6g8tq; zarO3Zcw8mmePvpkF(wr|JN9s%P`I5b@@lS3Q^<)4v>UqydjdqOihVwTCodc;ZDnIq z%XzttD-8*^T`BTIM+jlj9O94-=-m=wA_fFY$0?3PHum3p-^zv?k3vby?D}hqXpADY z@$YLQT07vuSW3CJC2d5y}h6 zS3Y_ism;(@+-kAOKU3U;3k?VA9%!=>Te13D47)A?&zJDnj$~c2@YTAC`P(2dK#e6G zNc2hFR7>#!Bpeq++2npdE@mRAp$9CTUkt7bN2*!b2CI^HkYTu#pOSIl3*2;$JMns7O5Z5q$q$W8Wz1hF^W`GZJ5c}k~BvekRKZdG_7aC@Fb7kB;%+KF{ z?}TG+GQOhLfVu#gwYAA;$inC?kju#EEdux9e}~m2IZ0WA;y}6h-tt3QW*6Sg)+Km& zbZhTzMsJY7!WWQ<3v6yu>S&Oi1Eh*Yo&rmr46BU0!N$Q2V!*0V_2D=|_w_LivQjYu zb0Ifk$TPr9N zW_2ewh8qw-ry49ut|2ky5PKBY^sfUz>zeuF0G(a+?mc=-1T-Ex12D}TXd^lExc6&8 zi~M1ioFRwwS_&;1bWQI~9^^Ox=wD~pLEF0x52GoW06}e}CH|YZjW-$~A2a~=q~tOQ zoi8wEaOaA01#RFZS}0rIC)$fJX`a=ujpT`0+*1EjTJ*a4>a-}7Xk7=^XV2h*n9D^b zmO001^L1eRx}6R~bmA(gV66bA3A6cxdKkjyR{5Dk@-WLPZSUV=rvnu~Y(>QM<)2rj zjva`4)BOt`k+d>sVK%bILiP^2y2xrGktalK$A=;}KrOxSTIk6)3)wnGj#%t#Dnh0^!*(lD=v3;2 z6~jGepd%XeZYBi;)xO7l1?|$l+_{Q~Unm1EC0|Nx#Y58UTIucE%k8EIhl)9tUEZ3v z<94Nm%#RUX4QaUcLE?aWYYWqk{g7ejrP=5O!asrc{AoxNoZC}Zls@)b#r{h6#w9M5 zTb0|$feG%@b%OhZ;PCVAx<$p<`Lfd6rtKn>60%J}=HlF72Lj)bHy|Q=fND8ELbaU! zQL_&S_1bTBhe?vqJq}X>(cRUkCg~f zpq=J~DYxALKMb_smwzy^BM3EL$cGJ>Wx$e^H#ktT6!6Upf?AMU{VX8{6vA{&Yol3A zR@Q6i?tT!2DJ&{l%;sDgy^wNA8=hFCaSw+zDry1c9KqqNBMom`AhzMwTxMuE^rxHI!*mOx1-2o=8Ram&kn_(#Xb{@GbPmA3+3}8 zrlt&CFRz7;c-l=db1i_ivRt^l)X9Bgd1`6i?xdQ}#E-K3C7Zbyrr^9wzxWkqD@~Wa z#w;2%9~)RL_Fw2jc&DsVQj&s-rpLQ@UaZ&7r+dp=JDg`UYqBHJu_)|dtc6q4S+;@3*{AaJ_8Tz-T?^%}NNX$NC z$#!0e^CjSYj4ppIV7R>Zp5tRR zm8))#)!bp|KBX`(_x)-!JKlSS)!c}wi)pyguppnuWjg3{_PO1R7eY!y1N+q7-?2>G z%5L(Kn;jg^JdtPwx0!PH zcH5+pCiM{Kt(aItWS_bZ=2rdq2{YFlu}PhyR|)Uety#-KEwLtCm>$!$Oy|og`H@%1 zR>*2_p?{6)&VE+?GTkmZ)l}!G@-QE@xrF(s{patg-|8WUAwU!RfF?NT=LNsVm#5!< z?>-_$`4-O2N|#nVxapf4bA=^yZBn}Xo(OX9?o0Nd<7Cc{mHid^~>&|Z`p|lJ_#GYe$lPLv+b8B>%>UOEoaZy zJ~`8!J{t>aZKMi5=ASL2y%J^WJsW@EQ*TvhVz*I_P56NrQ_r!R^2-a3SK!;u{gyFa zIpaxzi5y=Pf+kNcMOi05jy%R=Vw82E{=RhE<#!qGPRAlw0i2qP-stMliZ0#Yqg_N7B+8>w$#uFmXG6s5UAPGiW0 zwcFSS2h|4&&wxoC9Mnh6qwj3P*3t3=sfHHj1!X&{m{)HOY#Z=y3Vv7Qn(^tt;0wJ} zlV4vm`#;ul$hr3{wR^pG{GHuw;^wR=W208LY3MWWN$c5H`nKr14$Vb4eqt5W)_fW#X7Tgq-Gp)n zw#@aJjXNo>Y^QFWOPSv#o14Q{Hq|cRGcD~kc&+=_0aKvVN1^3gFE*cb@c*FV0`BxLC3J^E%tzK`=kMX9bZ^JFjO zhQ`3?UrouaIzfELFCMGd#KLC0BEayk=nt&jw)2QIFXau4=G7@<+Zzs4`kg|0zT?zf zbr_H+7r3O12@(#sUG5#nJ`-yJ;0*~Ckb-@AjRFMG2-UY6D85H*d3rc@FU4*Z|F$Ca zP+<>4>zUx5Pmh0XzoUK@xMq`M|9Uo?v~f5|O1ADhsq0D2qZgW0x-x6!gVdcDnoE}A zXW$0n-h}!71N;i|`~x)R)eHiyqeSWDWrn(BAAfwN~A*=^gt*L_rO6P3z&jyR= zU7c!oDBBaThC-6D%j*{F*#<_m^ah;DP#wFxvA>XhJK=o^UD&$c)N^Ud2w_o&o)3Pn zodz-jvZY*D&8I_7O;js}rsdTuvn9jr4FkJfrhnGQ4Xmf67sdh)UHty`X8uIcOvf3w z*Ec>KV2?4d<`geK^y|}h0o_9f;QqLoUv*oK?^?y&btfr;DohR@)$L9idT?@`^nUU4 z68}uT*Rc%QS9OIFQXhbi;X7$GX~k?gJGeP0&+?1DGe3I7Tv~1Y4RQjlYoOqf$(fuK zcO`Jt2f7DcACj+Ps;Rzx`FTpZU$J@BZ^a=;Oe`N?nf#LKeWf>ltemataeIMF z>Urs`FV`rlLWCL%Mv5a3cDnUHe$>w&<9X8Bk2yJei@HPq^>B^8UPBj*$7 zyLGqnPz!)`U2^sYR`bolSyst!?GNs^&8Xzd&$R0-&Tr(Eap0;uwl=gY&+N^$w%O@c zyME2w3G+<>xk=f}qU8(bW%sr(he=)fHX&&f`Fut3<6m8%!8&Z)nVin>DOx7#%0Nh- zILo=cz{Kh__Hw`A1j=6P?{fZS=dn$&^CrXKY4n1z-zkRK3jM^6BxN7r*(6mynXE)_ z%UoU~)v6I``+DGG(Ep+AyW^>T-~S^*Ng{iXV}^{76%mfT$t=p=d+SI=_OVHXvO+>u zRvFnjgpA1E5|aJBUq-|G^T+S^PY<1Wzpnebp4anwUf2CP24q%r@)Ya%_9JKZdiCt` zL`J+<+lM;BO9Wq)oMRle3rVoZOK!#sFHw1M{sb6mU8LpQhy&9#Z$!sgQ)UG2OcC-409y5C;QYI?X>&P{~^a+8-J`OX?tI46kRYl(hX zVjj)^w*era4lEM?BqhVZt1e9_tt{eV`c#xuZ~pA->cI-9&X28x01idCMyuz|R|6VPexLB{oqfdRE?H-pz2 zDePv}`;0n$DvHiDnIM<2 zfjw0dIgHO2fj{m%qcWEKhbSIiXO;$7q}*iIB+@9l0+6Oa#^mD&!o*@Ssw*UQ}oOfXTv&XDt4mrwQ20V`bq+E-0>bWp>M z@s(PybR7D6veid&UUJW!#0Y$CfNbMJMIo&K23mdYi<@BRjM#+mR-MFg$e=xtLASP0 zLVqJ&G+l%O({M+KjILEUU*m$N^k>Z{`Q=lI?!dfV zT6#7mKNz1E65O1l9pxG|4D^_c(bjqfTskXyUeRjN25H-RmuIP4YXE=!KaLp%pvfVz zJ(m33d?m-#VO;(kD8OB&0xABElEb%|Qvs%>XO>UK`qz@4;|tEeD)Mw$99SoXQ<;U! zq0jdfq`U=mBgTwAGAn~@4&;#p^e@(0`#aOuwV$m{cXAayjA@9ItN5G@sO4l{9Qb!k zjv|fvuTMQNox?>I0WjcmGZQbgwVzCe4{U2_qsT=Vx-VFxquu}7-T+mwy;nNUa{rZ& z@p&P{TvK8sXreF)i*mMxeb93ob}0c6WIkw9+V1mn?sG2EH0uG#gTy-T+zSnTH}6%g zq2$Xu!>{j^p{Q*Od-ph!+P{j}3>4pt;lCmJFASk!jL||Q@2czjFNBZ7!#|XfNeTjn zaQ+~1mf(ZvCVo=W zHgUqkVWE_OJo}=RrjHMmJ2_3i^H^m&|Ld(azn40Qr9B~!9}^eYZ2~+`gf_O60pQzN zowuFG&3ptZoMa}jH5dS9UiIv``TrWxxj0YVzIs@3+tjll7n_N zy8y+#GeDy83Hi%=@yjdu}<0@xFtdwCt|e!prrimu(XZn z&3Ave!LZ6|SEs@8kEYZ0eQKHv#Y5KvqL8MYdo^UKOZ62nhVuQ{!KdwkEXtxO+v zen}y-TcAlhDDp4gIe_F*F8eS2R*e)^jWIm*U4_y3eZnAsTX(^e62K?Vr%93hua(Jx z0DdJED)jeOq~7K6xSwGsD|8F>XfndOxJ!D#p^3@~9tw7tE0gUWVE#<#t5fZ*Fkw(( zt$Hq3ZM0B*-bUufIU5Ki`@SA(iIzFBU$k@R4hqiBCPH3g-&a&qlCtVMw`?^`p9o&W z&IIQB_LL;;s95-IAn*xkY~ubV8fIZ-7yp#4=Ki&Sv6bo<8aHGkhsq_d(XA~GY6~wG zt8*4QbSnyQ94>GQ-L7g~3jksH^s{b%%yTVNi|3H}i>Mc^B~i@EzIp#^jX8hQIFrEl zVPTwgU7-&co79SKZMyWjv6rSZp=1Ao%sPw^AI5yQXJltO?DgXGNyCz}fOVfPoIU&J zVi19608#0W=4A*FPPP=5aLwK5iLg35YD90Q%Vl)h0jnT!@kq($p$1_pA z<#($}yv#msr6FiB>1Ra3Y1SXY08i8?Z;uf*1M*3*QfO5Q@%&1&>O&> zPwt7Qa-th{3cD{CarW>%`4{#-T}InW8@%kXbrf5OD>xI?L=~gow{FE7GZACVab7X5 z53H(iUG((iPZ9+hM@%Y}?d!TTr^8P|m1j*NeLl0P>|a$_Z~=GWj? z%UU?HfHi070^*;8V{r=um$|U`rj@hYDL3!!S=~@aumLYqu;;n&;;CQX7t?qm-oAY+N*Gk1&^y2j%cJ7E z9rvkhKEKj(I@t|R_u;{6Kcg3!b@uXhih#Q6^7yYFQm32zNmcYw%_6GOqEG%qHsBC9 z26R>i;&HrK1J~-h8ZI^(FeS~Zzq0Ff`yj+y4@zT0C_@?1s=Nh~PPC#l=D%)?$O526 zT0+KPc=R}+P8Trdp35ag;+tDp2{S~>KE0PmMZh=8@wWd)<>FhtC1+s)M+i6D3G9Wh ziS4d+m#+d|cY*Q|x2Y*uu1Hj~K2;e3=6~#(C4Oftb>zxWeh^RMr&m%<%_Ehrk3Q$j zN8gcshr3!_R82AUg^nMX7zVNdD_uUrYc(LVl#%_%CqUROWX7W%*#m4JH8zHLnyfd5 zbO>po2Yn=|MF<()Y_5cKZ>~1iB9NfVod9Hpkk_PAg*#65GB_ zy22o*DkXy=ghGN{74J_xAN+-2IU69_)x}=dbuGMnhVQG@jPr-WYKn#tF=Odd2sL2d zSb{On;B&wsf=5o_{*yuR%`wraF^b<`=9NCOFSxqX2#C)<6xY)m9F?N)GR>o+b{0R4 zh1WyvweZ})ZTPA=Q40k8iX`q+31({(QOcGbPp=_NxEuAO+D&;uZMRn&!}6a{!@%4j zz&ZCAHA6+UJNBkgNf1Keiy8SAAAyt;9n}c3O+8rbBX>vayjwui`Sb@Bj<47}9=PxM z^p^}ri^VPk&TvEIT7Xs+wA$}rwYGVpS7fb#X?R0+7I$LjWnj zNLgehfZA`z!3QWg2xGp+9fqX#mpln=R}pL~*vH>>6m|&gvkKH+i}<%fi{41eJsioB zYBCOr%X>QUamzMa82k(-Eg;psZV;)?MTa!gQ=x3f=yC*gT?9Zg)m^=LtV*iT0FUTt ziqT49v2P?ZThjqmlMIE4wEO^%EDNAn2AqYF!{iUlRnLIr!!%mGJ_xWlSy;e)rr?$p zvmc<-Q3cgob1(jiQx05)S3_#MfPSUzdBPPIJO<*x?Y z3q|2Szk1oNFAkP!IOIdMS0stf>)Qz6E8L=6GwD!q60veQk`Io$H|y-_8*w&T;$KV> zr?o_%WEASP5@=Av30t2NP;hG4vW+)=paVQ21o8+6t-x$F$RkeOtvG>(=1`^DzMkoJ z>=6(=yyw0krTVb;V_Oe^iTF%i0i{0w*QeA+fJ5sI<%p4|_^RR%||Cu9t$XT zGblkbu`VlfAMMNUZKH?%W{77ih=alqgp~J8!>VU`p1&|Yy9)Tt2NB_VKpj+xZux`Q?IRz|uj^9fv>abH?`KzZZ+>2f7*QHmm7zvjkrpXo6@FEGT`%PB7O< z)C2)#FtyLtS{E7?WrT3(H@{7J4+km$K;H>p_s-Q}(X=I>BjLnFpCq8WC;lXNsQ@Czum1e6qkirSMn#igY*)_E>Lp)#>Kxz5oj8*$yTxNd5sH>hR2<5xBq!7VIJa`&~e;owi!cei8BZiz&ryk2?Su~RpNMFv#_O_ecviTS7Cbm zjr#1fec2k8TFvh-{f?cZ5TG?iFOhS8#aP#+`jE9ba(y^(_+3EoDRd4DX%KA6Un?}puo<<|hoa z4f?jW1m1O&JXg%ZFzIS5XwdC+X27Fjqc1D`=X>ax{=m-Q$F}IMfH0{hO8}F42OK(E z%*rPjpky!#l2$%Zf{dOtu2k^Xl8&xh{4vNkL+Ac_f)5Vx##b8^4zp3Y8~{Wx^ZwU& z$OY|hN728~gUpRcMR%HN>sp)bUZ2Y?h|T%I1+S6K?5<(gj5uz@0fQfYqLgatkd%zo zVu)p<#Zfaf*CPFC1@N}?o43HH(UJqtlZIM$|G7jkX76vZGZlUDPf;GA57zdKaW)g_ z%0_HiNU0wm30Kw9Wn1s3b0`DGDy4G*ukgzPaM=M19jCc4*MYI|l@S$}7n-7; zsQ$4Qtx5v4LUEKk+>FHPIY=Es6A_Yp?%zjSgwlwiO9T)J#ptlta+U6zyYOsME~ou= zu$TL6|7$N5Q!fUEm3(ag5t=o8uO1469z&2Y29i+p|1lUO++fK3BizifeCq9W&F41t z&u|HL87H8)AiAdF0>)#)x<2t8EQc2}mP7BQ*HSri+>FlHuYO%&PEg&b<} zSQ3XPyl@3-BDw_nGaclB_bi}cNHeokx!enWhuo0uT~i46bw!W^qS-!zTKZoYOJljR z>DAZq`iS^8O&G?-u4+=WOE{_ z0?09w)tiIMR~o&I^L8DClJ|VB`ltft{*M6x?bPT0nYhxJ=nxv1{*+Os*NuXkVGWW_ zLFlay{~LNoRzx%cLelT<*=~{a0rn8L9HIW)W}MH;;@E~ZInJ;+0TrWnzn4K%%1&( z5_m5GAVeiom5B89El2DTE<5wv`p*Ap)VD*rBpK}dAB-qQ0*DZiL+#6v&j>7q@(;)w zZ4dMngH0?*JuyKSn9jE>|Ls@1(RXZh(=PFx1aY zYqbzi#EeHXva$?{;d?n2m!SQo5Wj#S2AAjHg1{RO+3+40@aUSmrefls-vEl zxFCjBb7;`8sbOY*`pd_&077#t zprRD0I1O`z{2rl#b6MxLc z23{jGpHi3|{7fkD3_e8=7caY-vQ5641$JBW5dRxdXv$!z<=nrcEC>sI2=gXphv0Y! zHJA_TwS&QS&*oSC91xm{Hz@{^T`B}rASW1LQ2VT@4tvh$ZLM{uQCxj_9Tdf@2Nk*t zj=%H)H2(fnARp2H;p{jJJ(aD7eU(OKlueo!K>48#0Mgo=LbS? zfVRHB3X0DcYS}dn>WIJdAQVRds0wiw<(~2!?W6Mv zfBvKok>7dc<)c`{#{)`7fC=>tlAAmU@1#)Ie8K$5uwE0j*Q+z<$uCv(;~dB>v|rJS zOxsZ$r9+?*OH4#D^8(P2A$R#JLMN{K3bquiyzs+ZjB<+Rh6^E(2EI+vfw2kkR7}VZ z>)-{pAK!JVB-50fusS&Q>~Dg>6oh~DcfqC$v;H z-jkSoLvOJ;}>Rhi+y?b(TRZNn|UE@S63EYJoC#)-;Iqg`N-2%f(snyky zmQ`R@mbrJ}+^)Z&-|o5#P^_$)sN6z&%CLb{4E$yWyf(#j?Kk)J+5 z+MES2zCxACJ;iCY_yuh3XLq(MC#SWfv5#an_TIP{Dpa6O&LZe}SYl>_nH!LY&jChm z^_+B`ABX|H-L0d41r(I7oo^14uRo0-t~m!{LAkV73E9&cY@F=U+YIAh0{S}# zXT$e{jeXDg9psQH0;=nq6NEJ?Boro%8EV`>{M?uBeav{;T@|1Q zLLV)S0|wWmpeDz98*k??SBsF%8VxoVrF7#o2$TO&*XHN|#}n1CAcWqUngA#ZzxwrJ zAK--MnU88uyx0;3q9l*e0_(^F(k_FYX6ZD1SQri_|E8_^liu_$jeic{mS%qDo(6^= z_s@Ezk{|j|JCQpORGh~{1zL0q_CH64aECJy1pRD|V(zrH%~<?}tcfZYROu4glf^8nQQ zp>E}12p*=~? zW7{?^hu^tV|8ksqBdE;k%rSu)WO1d>R68Vz3>vyz+=0&_OU*Lnx>e0$`MZC zdENyyY`UA7Q(mq2>TyjG;2svGpE1RDzDpVaxJwjv%cp7B?j0xs>ime_@ROAj`v8tH zt`FO{omM^I0DMN^e^Z@k7f&z(kfePeCKt}0cSP{Ti!mC#0_N9?%tN1;J3&40ZRB2G z_;`dQUYDv2)x(2o4=5@yd!Wb9e)U@B;Gf!;14X+9>!mYLA@|oefSJc$w}Y8S=y7~> z=zss6y#+SEN(3e~4;!P$`H=~dfU_IpFB)>^fhiiH$!PWBR?f-**1N`sDtYQ1bU-y; z?SX-ic`#A6vCs{gK~c>i);i+OC%D#qac0PGs1A4GJ5c=MD*roWJH`lN{HbYF;h!s* zIBZPCaT??!og+x#`+K|Vp?2PqK$(;{%pr(aVx2Pk=Kzp|+LHu2JHH{oD5AI1K(U2g zG2f*JC-f%JrJ1ih?$RIvL8qEx1&7i5BsAgcu!{RL2pp?Ncc> zD~WAR=!7O5F+7#LTF}TVkfHXdFGwhbB}Xb{nrRGPSU-{LfA>HSj64axUay+$>R-Nj zh$6K5Ag?{L5(_*;)dPGy3vCB)D7W8tt1+NM^o`a%$elst9PU=V_UPXBa$mc^$bzuA z)uJDupX_7owI5G>aw020T;BEBgQ*}eaqz!~01>^!z!^%WQ@Vb8bm2w7MYa<_x!-r+ zb8Y5%x00|T&I9c~>TQVI9gGOt^Jh<(-$IPg85gmZIU0yZZK*W!Fg=oj7y zp**-QSFnoS4rJ_!rGku52sI~qlfM3WW{wt2c&Q_ihKfMQogx?WF5?G7wnA$PQAcJ_ z9^Ks-=>x;^=>lKfKiJvNItvX`*F^Ht|GU9QFyXRw08N3pkgl72tMRxHIEsM#;_txV zY{s(o2+)!H#<_K+Pm*F5 z=zC#`*AKoo?=o-?_D(UA6G0U+NVLAeReeV8g zyxQY=4olkaqYp03fw{#st63M5Hh8wiTr;`z&6Bf=9JJsY&nfUo9&7}5DAvSuX?u>%YS+_vVrs87RcvvJkVcnt2;kzQHQ~V6I z4P0VCH?3MXt1({rRSENp?%B!@Vt8Z!Em*;%!cn^p(F;W3(yP}@PCCbfq>1%ukfTQ|j9Pk5UuxMqPTMbd2t zk2^pDr7drZ`=9tb{3|dVxa3NLy!7u1qh1Trav5d`3eaq8c7K*nG=hmThPUHpN9~#b z_-HU!^+(vXYA`(JdnY+bDU<8GdRzVP4>;V|Pa?n@lgyMm57%`GD(ITMj00tQcp0t{ z_@7{2;-S{tC-TKHDg& zjN;P!-uxGwjfy+hO{2XjNV zAF^_112@QWy3!9w@Sn&LXM`qf>-jD+9xuha(yMo?MPo@DTTKj zg_JlI<`S4617~Rro{5s0_&$?+F_hm1HJHS6vH5uFx)L~J`E*dcdFc8Lf^hp+=;1jW zP!pq7@!DzW)PIwfc>pk7s7ij$1u$drB)sKhSm6VHdA3L@1SnZFfJC!`#9 zH$aX*LmFEC9F_cf9Vd(~h4%fq7!aJ#6Y=~dXoaBsZk8W-px2#3bb$Xp+VTZQL{yY7 z9Hpz@jJ@Xu;KYZG_=6n>gQj4E6Y4-2ZHN*qun!acsdJu6h>tOn0QBk)UgC}e`e2fv zvI(Z?2yhl!+|%oX(5v1cRmXq48|}=9rkXOGLT-*I)mtWV7Kz81?#)gNt1BcjOXk*w z<`%)Ezhvq|NF4drY&LW7p@m?~R-m~1<2LT|WODQq0G$P0S_v|P>L{59zJVreu6O~4 z+=TR9B5GM_qA93*fc2PG?#6L7riKoNYHA<}qQ>|-9FSy|&u5?p1fYJuV1g9}#C@zz z*0$$sQZi1%sMu#S2-Yt|tf?gU6stLaA!&=ZWL700^JuZl`e5iPfQ(I?WdG=ZW8>=q ztdhJ|0?v1}SkSSY;1{~k`7U|If#dUCSx`~f%-h%7%il}7N>Ok{*4FyR#3#{1Qxaic z^1^A<-QHDT;w|(y&4yr6>Sk&zPS`^eSDcVik}qFY_~9}#m^nBO$kBcb=ZcV5z!Z_m zLg+GJgag#gzehN-pkdMVp1dp@;EI8va!1Z|yw9r8xQ|@X%~wZ{t8hWCD9sGGk-xpViksqKM{gnVQQej= z28Vs8=*A2H@{QmX5G6iNSyDTK?sk9K_RSgTH+9zkGB@P2P$hl=TQ7d@dK-KZtcy5d!m^%Uo3_m1rJ8~l zL>U(uEzZG{rmqi&MFhjDl1Io;KGh7fWN>-*g-o52B_tlwS*Loxdh@*%-18QS9XSQ} z z=jDW?P@_9#{2hHRHZ@5#anE8V_OQN{@07Kgi@muy>{KoaTTFCws4-f4=Ek%L~EoORo{ zE@xhPoCw`1s1LPsjTZgrar0{6jO2n?;KYV7smV+EIraWU2`s7%LM6fHh^|k+&(z6{ zzD%-3fdP)u&%-yY7GEncjT^q2hHz+%_ER8+OdPI26ILtiMLQ=U3dS1-=G2h#0cSmT z3{jJtcbigOc3R2<%4jCXvB@9EJ5n;w3c)47Nx#U^eoZxLRD#%PhvrlYcgA2MVk31m zy=tmB1)T-^pobk>pS|hg0)=j=mHNxoupocUwfK&5*IbXBm@EGv$}7Y$qWz6hgu`Ov zRMfkoj1+kxCh;#}^ygrqog?x_(W;$f6%`K)E4Gcr$rhb283?DC>-2x6LO$^!j2c;f zw_TN(V&_c-pN5mOT6CW(9HyOp@BPE<{#fhb1#=7$h-=_26K*`-*^ynHb!9Vb{2IUPs?4AHa4QowsoAnKiI|CFVgzIC_g z3mNhZv1|FpSFd@B5&LC~gUMMabo4QyY9BNxO$Ons~GYM7z zmMqU0d_IP+NQyGQO#`lXhSEr;kad%>{Ocyw)pNO(cDNN^G8N-+XIvNAMSMACYhdO4 z`z+0{dJn1>U|hpLN8WnJHNj*~e}@Ckz&K3x`g~l(Oi6H-JdX>`!%1peTpAAzPYqP{ zO>(E89E_OQTObO zW%v4df_naQf?;{E@|tu_r1NCoL;06mR<(P=Ps$A_gkAA37tL8V$iU@@EHV-?WJ*t` z$!`gj%WgveS5Mo~u^g>Et3!i~j-Du4b$Edj#>w3UYgR2+jq?+tK zXN+^7Eivpa4`pE#q~@B9lfYV1U%C!6P?e^zB~f&vIWvs&IoBd?t-{22qv`#;h~8xr z9+oHKLmBlK6%vCFvdIYqR*rl?b?{yQzCi$;Cp zk*g*4ac|=AkizT1!NGm=JSxNf&-c+z^O?4hg30|eQW7M`cC!W*e=a&eeKjA{RZy{4x%z?Osug4SY1p+= z`Z!*lmLyXQhH1MZbZMD(jM7E&#}?;hbjua{Rb{B7*ME0` z3&gSr=%GGra_2Ey2*WGmWWPEVE;Cvr$P9u>I1s|~jO7y&1s`An6;y1`Sh!5aJ z?>%QMCCnVWQ#4d_SsOV@N}Hj7u>hlq6S)+_mm$m0(X{A&O$LR9>}_5q_k0CRe!Fn| zGTe_vjdlSitf6sg>I)3L@*3yl=s*Hv^WxsE1s&d$p&0@#d8+(<0mj8!IjK!9&p5R1 zOA}$Fae`#iV;1r25bCn>03Rz^&$#+;mI2CFggbV#!EwZbE|N9^aBZP|`mb!euRtdk z)X>#F!=@JwN8Z5PmnHf+;>)0fhHhtIC`l8gl*wG0h%*kReEY$DdFmV(;GA>|Y4mWr zeb}r3+NBzL3*SiB>ZpBT{oYItZNIXN5t7+f7=y3I#FjR!N`vuC+~pbMFenGRG*LQm z+YOnTaXn=Ik1`!w|8a7pZNvnUunI5kodBqTU_-+@E{sJUL@qd?H62Ym@M>}Tl!+OF zPebppzC;!Po@#5|IYl@%q+d~gt5ni_dRzov!@=`2BZJ)ssc2AQ0NYK;*bOc&OwSw*y(MhM(OStxMMcDEPF8?O!GbbofK5%8^v%fGf^cE2)w2QTjKR=W3_xune- zobb;uJR0U%612S?`N%5Z;W7ppVPFvQ-IDSa=Q{Mt^)Y}`?dsuxQ@qb`va~BZS zZ?~vwk9GtwSfR7f%;=rGst>urKYzN~lqP#U4m-|4x|+9_hRVTs=(7(3jg$#;FP5w| zDH=hOK9cx5pi2)NGfD4jAI}09og$o=p|dy*r zg(XF>WyP@)|JiJJ*tYPhpX`seU!8pp&7pmAcIA>wnO(SNCHC9F zTP^_^9YHY<8w}no<)OLaO@1-Xu1uWA2EUEIK1`~}BzYwBL$~(mMT{R8NIzSotcZ1p1J^{p<5R~L)&Vn`4MXZx$TM-Gk6in^TV98vf&aw z1}H_b2g5ojVX;1a$6-#S*g5QVANc;fdM_-{SH+&g+&^9Z>2`2uzkZahfM}fcOli^W zY1`p^FWU{t18m5srwz)soo;DmEH}8g8)gY=Y1`lt&Pa^Du8oIC-!a0E&yh>#1s~_f z58kwYlyTZ;&YGinejh6hIXZ(pXMmddaf+#dz-@S_%&T@jo@Lfctk0Keb{a;YC7ff+ z*1Y?yOu6QPSuh$Ck{-KW0+>^J>{WyynQpyMt#-u`vmMLZP;9h5*9 zbgi@p?@he^fY%)<<5{&o5I{S7JYji*&K`8!p|LhY4q}rw$!v|5MW#lm43~nZM&g7; z!F*&;Pk8R@C#zCV8jOAOx>r2@q@q3>huWnhVofIHtuNy&iCEu|5pnC6s?wy4a^%}J z^X1#{?sLA>V1^ggUQWenfPB{X4M{Pq zgUG9lRm;3Iwlcj2uzL?&pN-h`EDFuJeYY;eRd_;=e-BjH=^|eBhkZJ66G$m|B~bNe z;OVdc!j}PdYjn^eBRnd6Xg}SBuv8Is>*1Y+!X|3aCvcBSPxm_VQtb3vs6ha3`ha3D zZtb333%t^(pa3%A{cv#xLP`T{+dj#-U(N=#WDd<vc}d(z0kt3^W} zxUzdn)D`{5wUGL+B*F++h4npSM5RzZ667l^t9?79XSqb#ER4c`sLI+qYuV~gQra?0 zmok`qTk%77to$mMSE%aBd|+Iez11*5@Yvu@_6TPfAU$UMtij?^5Hp;qzj##lDMG?^ z&@X3btmy9wQ5+Y;9h>X{{4VzK!{YH6prPMc+H0sm+D)uy{oJWjR6yZ5!MUS11}Kao zx$H5%iWL5V%`=(7HM2o`oQ$dgC=FE*txhDNSA1Z6u2U;W9-QdAr=@wP*A z*Y&}gy#0_0Ql4N5CA zh+@L6ceR3F7CC58FqGcU85YANxEG#56L&FiTX=tydNR4a_udk5e|ds&jj(NB3k=ie zk!W4l;=s((CRfXAmz%o;vIYdJ+0-_QF%#?e@SPYJHNJU!tD|DX99FFh+i+e8qN_rh zwk5ChyF(j*-vwMB{@#We`M04UyWw92J|%6_82Ed(5xYc(ik+^2i|EGuywcLXLb?b(spDa%pFXtIna%Ti@1A z8UWY5CD?2z&$#dn6cx3L{@9>)PJCuAEP1KR7LaM%p$l=`Z<}C?v#MVgMReLp2wt(z zGNcH;GsgAuAGbk**#P`Y8|P>T3V8pu~r7omQ8o5)n`rc<)mkh48Gn^0iNSk8SYCagFGHIKU|!mJ4EwNg2W}z;%HlUTbd<-r@>l|+ zrw1H6P~p;@-+I#>5?Q7w0k}~0l@5?8b+Kr??fXA49~FN1*pan98^f73`@74A7l)rJ zWy-^T2ADGj0MU?xO=TXV>4x;iA~&Z)3yf@wN-w&6Fx5+Ilz?Pd=~3l`+XJWlC0ES_ z%{q}Sn4l7pZy;TCW23o1`;&6H<~st)2X9s3Z~AYF;n-e}Udv6T@`{$1lk?blhs2QC zXh5Tgx!C#=`E`x829a^0#k@6l<^l4ZD{4m>u6;=Z8~9i_Y*!hRdTHakMYyLSPMB|V zME=$ka!UJ?pA1pW2nk@w2surR_fVHH-t2Mzq3;C70*2g`$dz_t>`a^%7hNO(_rqoU z-(7fo2dD?0LTWOEN03VihPo=1uJ5h|hKC(>+Qd`I(EWDCX~6l^Tsy4Hf9_j^qW}(2@c?gK|pxX-lb80u|VM-x^Npc2fuYCBh6CZ^^@a~_n#>H&kzYFT}@fh9`z`pPK!aBdMcixGMg ztElLoWn$8(iL(kVaC>Lrl&>>gO?pWF+|=a`%^}Y8>)*`*dD48ZMuqruPpAVMj6ow~ zB2$s7Cjt8t3os3IPp031M0z%ajQ8a(tLxPkSbA-zpK9=9xY8$_C13CC@rI@F2 zLURodo+LZ)WjxKue|#MIbB$%+03N47rW`RRK|o2e#Te>Zd>tgS^(!C0RpZX6g!>ys zP3*9V?huSZhAQ=%P=HIaQ>kQnmdXk>rYA4UZu!)8S|yK`d5Bq*z2251q+}n?E_w<8 zMtwep+AtWFr#|VuqlWVHDh72t+nww%kidbe-ffM4l(AQ!?B?s#5_oKcW6#No1MRzc zn0)1La6X3(TBQaSfTVZlIX<<2k0Ugb`aoCwKHU)tiC1A#yN{v`J9jJQx|rluu|+j| z=YHMo76UG-f`E9^F2QsCe$JKe*TR48%cBZfN#o>yZJX^m2l*_?+nU0+R8SsB-}$as z0N_Uc-!23*TEqnu6-13nYrk01Cbj=Id^B7i65HNV4KF>lDbV}H_t@F^e`Y07py&jKoL z9q3re#omGJhgC~&2W^GMaSJv^$SG)!8K}57Mt=2{r6@SYpL_lzDKAfqoMLdNEEx`H zf=NN_n?l~RscDfB?#Eg-MX*ZE_H9Q1UOG)!Wf-Ah4f-TsofQuc{sT2~0BUsMD7+(%7Y&^? zh*oKykF80H;Y&No8S60tXZ&kGDl4egJzVRZuU1sY*e6+@1WMZG#i4J8O!^*cV0PN1 zDeSYZRk_=Y;IRWD2C%)s4Qvwy9n!#E`!xDTGi~ECorQKSZ9LP60Ia1lp@%BiFF=(- zR+X#jJ42+r_nOd9M*y>&XA$nhWq9I@>Q_4~nVNu_K}ISA>6#egb$R?YZF zKjksTY%PZm4x?ZQV(O*oj(pU(8Is9e131rk?9Iku;hG||7vn$Vb(|L$Ic8Vsvu^>&w;T* zy>-T@QJt%F=TEp!fWiI85qDDPj^{}b(%7Kcr!WbeM=!4=x5|51w+ zV~97oFSk`Kg9(t?NOmhyX7d`-l_hgr>0aq~_?vef47b|Z>sC16yNn)r=Q9{#K4Xqc zkAL;dXZ=uzleqWPNT8aD!um9mCJA;#=LG2M9H`(l+x22B`}bsH@D{P*Zof2^=$d^B zC8pDRLH@?czN2LCys@F%6@XEm zg5B z3z2(M-JPaL{Wo{Yn?NZ2$ePgCzj1H+1Gp8%;{9v2nceHF%_dde`6;mGMY&lwO0HR^ z@L=1gwj1qStyI?4&qnf-L4VGfu7=TqBD~+=#~c?ssXCxBRAMQy#g#IZ4X)l$@6+@$ zm+X(N-j`RH1uLe#IluMHQzz3~I47K;Y6wTJxusNy=)J;^vBVC5)T-BQ7!HTd^T`vi zLoJbMFjCA-MRcMok_IutthP%I$5<>1I*s@$0tpJX%*|Ebr(O4SJl5v>l)>Fk3~RMm zexF9x;Iv<9AhFup*?f4u4C4_M(C~*^^j<=&LWLJ04cgrkm42I2RVpZH%swUL1OY)+ zm@N_GU^&(L*A@7yAE0CtG-_ghugoDcm6!Alm7$9xR7yU zV**pGPatOPW#?6J0&TFlaA+KAz^iAgLW)D^kHETNH(P-=t`@534CCe}B=U|yuO2?u zOVat)9k;!!qajV%FvSM2+cwZQsBua6PMVVn5{r>)Mm^pvSb~W4u{bt>>hs?+@Cgo4 z3dqn~wl@howsDCHRdJ4!+94;Rl>;m>1Z92k_`aNARvH{+zZcbE`PHkIC)=lTDv>*CJh|BPEpz0gK}kBJyBdIDMPxmm1z8KifZ z!IJuNk{CSRu%U7~>b2`?*{=(5ti$r6uk9yt>QZ_3Ub?K~*8;}f+DzNRfcp1rwsDGa z3grWFcrI-Hz*xqWZhO~%cAS=b!ejNJ6lW>`?IMOm7mX9T^ZCGdUMpAan}}HlgNFM! zxGYSUPMf}zX?V&#jct1KUf=6Sni?2{Y{tZhy-vN>+seu=ZkjGGS##6Rcx5i^KHL^9 z^?W&(yX~v}zCF}O3q$iV^JLk}2A@Td1tS68Qmfn6)OE%`?KcD7Sn4gJH@SM`XeuQc z-CHvYuer4TRi*C>wa37!Z5M+=qD1Nd_Ujrwn~QZc*y5+q@$ZW-65RIdb6aRO`yLvC zhkySR2JI=d!>7QX_*c>21%&9>22qns05qaraAF5pM~PTXkR1uyK7obJ?1J< zg!My#9xYy}tQlt6y5O0UkQ`%nIjGlBb7;4#ZSPrA98tT1-zmqS0Y-lP_BLO~gc?#r zdO(8RqHy+F|tXILj7wClM)o0KYXL(l~25t zFG*W3PCt+=qHp$2Z|zhN6Mhyy`ogl7p44}&BR@m?>E>f_dV=@Y`9~Of8icEDs+cux`@Z-K0dqD;E>eU1*YAt^`c?dL5>BTE7Zs4#{aG`iol|NotdK5 z^aT2n6gnaWRBR8E*-tjUKNTTPI^&{Fexkln9 zdZi`|$3K4D`}|3I&M(%L;mf>w18t09Lk(?n=QNathT04-e}8eya<6PNu{j*a^k-s1 z2{&&L+Q#=f{wN^H$?K7EqV+SbQ z-yM!X@K7F`=J`c{0lacQ$VAf9=#C#e#saCBQ(ScEX;_xScZ$q~-j5Ppf zN^N(Q^W0k$`B$yN3zzc%4^p*%xPhbDoqMKC5%Yq5S-! zGNB3|`Edn}cW|)Hbb1Abi%8GWRqEtzx@pSK*EFqLz=m6(iQ(-~V^tl&;T1pJu0T0t zeUW$j^WG$o*$s>C~YK9emx$JG5r!eYc{K{@oM zhtt7IFjtO7XR2(W%=}X8@nK``yR+vIJdlOjY7_8|<8lPTxPTDvGs^Mi7=IGQVQ8u{ z)yjWlk-0E(41Udl|GKiB4vu|)!~@0r>waFm-+soDSQ{$v)!gpg%A-NS zVfE}DnVZVBryvXMTx5x>RsDRS^GeaH5E!x)vqOdg#*2;5>J{ zC5A}fa!%rkVAsIm>HC$#!^}P8V#=RcoA@4I(DH2F{u2==8%xuIiC_9elrHg~R?vRY z*qFw(7tm(61btyz$-zjs)nY@WPg&ZSrca-&twQ1x7sgk#IL+Hm{m3mCtqye_eUhnm&$c{> zm*)5?C2&C6jiB7b#k|-B<=uaLD^2Ns-2c(_)d5v?%iF$)3I-xdNJt2XgosF|geZ-0 z=#WObLDHiFf`k&%Ep=!KX%GoPx*J5gF#ySLwu;{S-rx719*<|Qz1FOWXP%jPp%6s1 zFZ%nR@Q?qajZ-2oO&=mDk4jp=GV8^JouL#Qfu0Pd*Gt%sAO0#nj#tgwaSt=ZiG=zh zR@aK?dl?t5us;=vyaf_SiS5{`DBj+9{Zy1>&1OoZyP;8h3-SF7)6f1?PYl1k%#+tE z)#4AltgD|K8DJZ^lg^)1D1s#+lj}t2vad8<6(+EqZ1jF6_mgpu`z5Y|u?KulPBL=mNif_mLs9!zKUS8j_&sRMv)VgFFE@&1t=*rU& zJlC3CmV*6~;I_B!B(;qmem*85Rw;qkh?){5b5Eb#*oYvQ=!W~-9%b{{HY?^&>0-^1 z8+C5^Iyvk6mPTKMhV9Rk?731`yF?6@Ukg}?VYOuGsh+pPy^y=U9^2)lIXOE#b2ec= z$WSJgQ|h6^BS-LT43RBY+i$`9EvAp0z{n68hzp9|61(}|2LO&u`g!3AlJ7W^LWg^+ zi+DOj^<`yc7UzAR5^O~cUMpYH8@4RBAZdY4N{2`IRB{69@Ctn{nX1(>$I4pu4-9M> zq=&GP!0(`)&kZ!SaIzOT%%0kd!uBTcyepqZiE0Sp?vSDC7t<;`B&&z5|J* zzILs?^yb*K(%tfHVkS2hq71!sGhJi$;n9{plCzEThg^U*YR*vWuPFBN@s@8+ywcTl z_DlLkLp2>7J-YirIr9fC#OQ@GkG>X-;mYz0SK%*}}NldBZ{o`}U^v?Evf zw5O%&1;CXFIkHo(dG>Gz!~)>24=-={e@t^}c>hQ7{gLo|>PIzfoQVe1hyURMj4073 zs(_s}KJP15;CG5Crz9!XKKLZ{PXqDu7ZP}Jk(nq}+mh-pgjk@B4eTq!%cIACp5++M zofvN8{h+s(3K_Z3!*)tv6vJ-U=L|{Hlux&2^|~pTj);5pHVAdc_ZbzBxKV%J>|J80 zx&>`j!O*?s_V{*a$`WoMKHO<1Bdp+cdV>Vyuct3ugssEQF?at$)6wTy*r_EupO@D>Hd6}M2eQ+C@CqaWx8zc`pI8% zGZFoWQ8g#3cTs{}qj~d={m*Bh2Viwvj?{*?eysr=MGZ=%^=XtB;-DNZHDzT2npow;3nGFe~v=9_xzoddmE65H`@7ei!p4w8sl{7#7`wn z6^I>pqku(=DD_F$7+&7}JptGDlyMfLjm;yUJCFT*tln*K7b1>@PX7!Tj$oVm!+CQM zS-(2<_wm17vk18(=*!_)hgS&yQ4JQ9q<#I{I#nnxd~{eP%On- z+0)vt51PLoRegl~DYA{|l5-`^EqHcnJvi)S7w_lyM+^#wd$>jqZv9=C7_5urSV+p@ zx@hKo?%q1M;|g)={gMEr)uCScBQ+dS$TGd-ybjF(jD1B4JDZFgl{v#dz4sYD$&_{E z+J7z(3Rv>&+ISjU#})5o^bEdK!wLK;EW+eCh3o~Zw#-wq>}2LfLe8q`j5jfr(WClC zt}Y}`=zoe6zCI|#2JRB(Grz=3ULUy)`Hu=T|2*RNxjuRnNo_rKQJ-QnAw^azEYyJvdA^FdVXT&Fv2`m~W9q1Y_ByPsx{7Z7qU0JJ$ zw20F`gPC%k++_?J6lwHoG3R3^yKl5JrJI);SxIbjyX5T8(-g;tn~UQNoo0RsR{|rV z-NSU>UHzZ63OC_USu0RgRNDdbhiSs{K@{A2)7kzTgSHDyhe!2o|a-YeB8t30z@#`IWm_fsEkhu{3 z6@+~WC{?>=WYpbi9w~ho#hj7;bNQ*>U?YhSS76x{>>Z4?@B-^Who<~GxzEo#!$2_5 z+j0qKa6P(ZDd9z<0r#&Jk-&;B6I~xU+`Aff_(0(QaP@_$zF^~X-xhqz{8!;zkVweM zu~emb$$iBxv@ctOua_)b)3#~ycpBHA7fPo>Q+n=pW=wKBt&A9HzwVy@6c7+FtK7kn z`tuCXL$uZq){t~{1#uQgwProcn0#^EnB>>$@L(~?=b8Bbtd8=Z<<&SK5^`FN@ozZo zB_?zHdC;K%QbBVY?%UT4xDP|XBiOZcP<>l{y#OyV`CcWPW`qCl{eL7pjWfkPy#w8j zitbHU=o$TdYPv3#S@hsJBS%%?y1rQRlf=6%2e|{4l83lSd z{1n!&OLBN4V8i9fU=@A0W7O#WEYKQRMVQh#M-uHgTmD1VbwJR_0ERA<`B>1T!uGN32c=R2_Zr=M;t9E!`epFyM&#ld!{Eu(_ z`r|Wbn-h2r_k!gglL|>(*qXFf^Tie$D9hI8TdO(xKF)9b&m)BEaYrE0xUs#MYrL~A z%^0?zVl!*~_k#R9!bfs&8r#^n?GBefBa_gu>9kXZZ6)&hsNdH=uXOnT(9Cc~rL=#a za)uy1Gjo{v<(-NbmESH0@pma}IrNg!)&?J*(K9~uRmIc+!*H!Wy+~*X{I#K>f%tQk zN8gG6dP``*vQxh9OUH!%tto5B(4^zzGUf6#tKGvXqD+S?JbW~4$tC;C1PmiT?JoRQ z!fSAJE{-mbOD>45m#rjI@KM#iRQ_+C5)&pPBhj_l`d?-C41ZGR1 zj504P@cPAn9S{wIIb!pd9d0_l8Wu-I_469GH!3WFe~jJl*Lb7?J5w2)L~-~dHuUm} ztk_8D65&v>#g2oc@LAqDs$rAlfJt)WzCYJI;Jh>RP+5YiZu488V5o+PkmX3pMUFt9 z>HmJhIYcVPq^`~Bf>^6C+;Nh`7dZhMi6;{!MLy_ol81Z1? zL*Y1~et6-IVC7#~8_*9L1e#7%Wt$WD2fX<@{HNfUG2X(wv5u~I*`x>7*jV`KGFZN1 zk2k%GF1jdFuz?`oX1-r?=-k1~HB)=fx2az)id5bwIsI>CLnHc#IP4`Z;V-uXQHmm1 z9A8Bcn=vd#vo6Ldg|bp2l=%Dq>S7;ZxE4ZJ%^|Td%)!YicyaT40f52@FvG$+E0B8v zwxc8F2$n<+4vvx#>`tbJabdhOq^TDHU$V23xV0)h-Gh(e+6L}W~=64XO?|-~( zf-i^$@AV_ncWnuMum$};hDYIo2>A$IhSs83_$IR8s;gkb4+MZ>R@PTq!^QFh_={^d z^An87DniEe?V?3Vpn=2y5H07II*nHPb1J?x7k5cAd0js$3VQDru8Ghpc<>uN?m+Y{ zj1zEmF_pRuR)7ypIx?Q^>vnuq=9YG_+ zs*N_j)eGO0sx>H~-vsP{JSkS$i6{;;<$`y>qfG#HNz~lf@EW z+K9|>aXKIW2QeeNixcq&yJ>f+&T(kydJ$&9+{@y97)a|w?6}Z#)l{lR2S+RlLajn5 z{A%weU07UPUiCaEPb)^?!rgP}`Fo>LAB@+Wj&+b8zTM%$Pr^_E{SlgGm%}TJBa%R` zjTq*}7V{lxVGXOwB4nt+t%%MmROs}RZu)s2@Uq8Uz(YOo;NVA5x|i$!kQEU$UQ`<#t>=YtIE1I1knkL z%}vWW4q!MRf^AyU(I)@~>JG@WwgxsrA7O zk0@WdQ;3Cl0cF;WbDCtp5KV;0hGqDZKIqqoxhyO>Y5X}3$&t3hNlgUyL ztajC1v&3qomo&@D_YNTE>Ed-Y*Q0Yu8kk3Q4YGFPXDnd zztxEnbS`T2yKj8&@TD;R!b>_hJ-xjGFb}o3WvU7iMFT(w(lAVXR?TNa$jvFf0QNI6>bx&3p;L2luQLDGtC&fx#t}=&3xi|StrHSr48@*_%(VdWLy*;VHN-VS1I?EE_F`8F^c%5&QJSs5lkFBIB_m^f-Y2<61kw)` z6oCqUL{%aQj>yttRSUXIKs&v)U&kCnC(j|%a2#uKuc*@8a=9~~Lw`Gm4o=MM>QENP zNo-RC&&`YBs^%14TN9W6EkMFGO#e9M{kM08Z-&+f?5L%sxc&45=umTHl17ZadXZTs z!dILKMs_-c!xtK14w^bxd zr8uz#&5nlibl;mp!BXOVg~@HiiYO_qb|gOezR>%zPu+|KU~Lq`XLcOyB(VvYFIen* z{z|d+j~+!iM2Sc|xN5jgh}KuT&W?-i26>2nt{d$Xrs5L}R8LNvEq-!VaDbf*$=#KL zj7}{)5_Uj#sB99=b$GFVdggFhp^-Rj_;ET3p~Gu4DK=prUSG1OG1V6%O&`u#uv$PN zw7>N^8%Sa=HRQ<2bLh~?lSR;ObD$$p^fNwh9};jt+>9LA@AfmiOAfa7Iek$N2IyU#*F24fHl9&vX8AOne z-ZJuiNdsSbUF%iwzpohsuW4)Z_q9pRBQb^0%R3`crFDcVnw_$|^RG#A%gVxu#3o7x z^qz+qD8Q5_sk;rWm%i1Vmn+nA986)hW5Y%E%oI6f(xi!YABE)GgV6WCH%k!!l!W~J zbxzOAorrmZsBxNz+sMVC`Dz+>(=v>%Tc2Uhroz_+Y1}(@7M(1lUFa>$gLwby&9^|L z$1^Lzm_n0K%amhaZ=UhMhO(W#{_#025=UO-jQI=M9KPMn%a3Ya{zvd#7Xj9{qOS$7 zRdW(9mmv%_3tGoyx<2qY76^UU@(bE{GYy_^gIBTyK}jBhOZ@0OfR=du8_@865UPmD zn=6)wiV9$)+$fQn!AtoDeSAbNdch&mFOo=4@dVSJD-*-bw)4aWPWS#dN>o{eh0Pnx5npv_zXw@EG0FVTeCiw98?U7roccJ zMtoSE+AxXUFh@FOx&r5L8N?UIa12H-)dKT3M!rE0AEyVgc}1iT`?OjS9ByWAg6w~< zFA(+v$GAIxhvp!aVn|=*7$c+D8?k^%zlDZ{VzvS{)@4K3fj=TJwV$V&LYC`^bqY^h za!z2_l_VO1NqT;yPk!3rB*YFY(R@#fhZ%5j95-(o@CgcpNoJE*_D-;Y9XN&+M#Sy+ zQY;XeMt$O?*r&(u8f!qtwncqvIw@JzMH9#oo6It8dY*emW0`B*O&=y4zrj6E&+XGq zsPD3`1p|7XUwmAJpZ)-gH}nfkN86~I=ny?y1hp3L6g5aek-h`NLKZ_I@NX{>4g^_U zkrqpTm-HCgr*`%+RmaGaLL*GIUE+0(t=-W;~$4f zr5Y^_L6la|+sXOG$~VLs;lof#^2>xd$J*k6RGiw}5ZQ-8Wd<|^QtPcjY{k)W|K5wZzc=j!U*0)wVPZnH z$R0&qZ-%49@lP&RBX}wAIET&I<;AZGYW-Jg4|UW8II9xv=_ZFR!4d2v zaezR_tc=PHT}G=tJ%>#*vy+NmL_8!jMC&UuSi)4m@&!z$n}ksSN1wF}^LGJ!3Pe7X z5LJS@lt8|6cg`B5*~oKSm@#Zf;Lov!icHKEYok&HVC3O)sG@9>FVeFz-{x}ziv5}! zxYf;KAa0Su>}Rm~5f;GZbebq}wTX1YcIexPL|B-)xqf^6SiIMVDJ~l%J)=p}#QR4+ z5nBVVoS=-ve7zqEED!CCo*cZChn10(@wIQz1g-$aKM%?#t zEg&xaD>b8w<9l<+FHuP%p2Agi&uw1!ZTs?x$EV)J#)nC!Y`rIJeKE;_jfoO2#SWCg zhc^5t`6i!T){JB>oeGxiZ3KT>iGw``UP}`3K<6*AAI07U2)+Mo_I-=wYt}s&RMR?g z5ihbEZ9VP)Z@T=GH<{b_M7Z`pUm*$+^VZIHT!*jl=Ou*^?f##z1^zQ~H01KMVc1Ep zbYr-x9xfl4w##o{mGF?CKqu>c>6Q~hl4NdI+Wf)k-QHLxdz<&>67s{XaJBqbj#2+p zz29I6@chc?zySBvvdmh*(Go~;Lp3HCjt`p`1RyN-9Bz7BTR=SseF?ts zjanl{oC^q|^YGeB*i7*SP^@{-W@*RU5t>B|*nVW0EvNh`A&82X1Si}y;Izk~1Bblo zFZc8$;+saCWBL1X+gM@v++#2{#bI2c#r2u?5B@Vx@5ux&4kk3KE1~3ot*v0NNK1SMPsk-GS)X z>^Vzu#5|8ZbXr*F&uMU;dMObzo82S)pS)k97>VNN5dAl-NK8(rCF%@w6mUo70q@%s#*l-(MQ?+su)4lC; zjlclSQW1(ae{;8un<$1#?unexUXLKCx?@5Z?C%H^a^oYjK2uiS<07l^F?;KKFyKWb zw));zqW6sDtDp*(ATo8x2}9gGkF8H9G#m#FvqgRI`Z5%+7_+8AxS=cipiKFj`wsLv zl^I|5B2xcA;QJKj6Za~s84}Ktf@R8H5>HB6G=KG+4{X#hsUEmwSPGwWP=dS2FgZ*$3;Alz&p>1S$J z;qrZ9`rF`xv{?SlrhXV9B2m*t9WYK#zdbm^+Qw_*`aW zHc-51jL2e z#SlW2mhA)kqGTn0E-!9xV`p6L9vn^)M22Hv)Xl&dVSFas{{gVk*}5!}kcf4NK@gQc zU8ziar+3q;BjHxy9n=QI9z(!Y8#dpDfxQDcz`;D$e-5irFG*Af=P44nTR)s3Ws${_ z7!5&scvjWSPdr4G9Ed@giyuqSKz(hZ%L9?-Ym;-xszdaq9d@&ieT zl?qMj9Z^Av25$+6y}4llBP3%Hn6Ev#{02WH0x^*ceNMgHYy23EI$NZQ`A**FD@5@vbX8<0D8AY3$jMdmZ(- z8RTv#P-x$I2U_O2x4MF6h?wd9x&zSIa8j;b5@YNQ?7*syE)py{BwA|fC??^Cvtg0- zA*xubJr(y4vKfIL6u;>hbzX4&YguTe; zOI<0?DePL<^0l3mh=`FR5s~mG`E;r}m{CR3MqJ0e)?^o28b@H>FkD38#sCWTBt$;z zU?I(6v9A~+`(XkpW?e+{eJ$v@Cly&Gq*d;S#I+jbj?1;yUTlna?l<^fgb|RdgP!|e zDR-baGIr{Iy(b`PsqF@Sm5g!Bt%XA_+na$qTZ(A{vCvw!=0^419+a*&_G zTmn=_mlJI_fZZzqcAphw;f2bx4S&HUR{<1bakSh? zJ0AsHJB+9$G#9R_SGPR%!dlU6O`~Y8U&piPC?0nWtfor-S`c7p^}3VwRS;Va>-tcT zzck6GUmpUf&j8afTd$NTAI4Lu^ql!-+zKJx>ouC|*>x4Vr^p>fH({fX>LLn-mhkFm2B_0rI;d8sB+~LCs4T58x)~Db?e~w_&UxEj3oR)W` zge>Ofh+vlYkY^O9MHB$R5^e1WLyp&9)Ri4(W!xA|W-06tY16jo4%%2Olt^<|U+!O! zIPew^7w&dQK>lV5mTtA$X~Wck!esnjYRRH{@4k7rVD`q6{Xs_aq~?iwt)oMi`}Y#6 zeEe>82k9luW=)6dj16At587I+-oCWJM=BmI;96X8CF}cw#OBa;Y_cfN_nq5jeXGQk zWVK0BRH-}VmRG(>KeG%){DFVTWk)cQ{5bhXk>Ilfh31}VK{tDN3T-hA#^Ez!3Yf8- zq_!M@J~5H>YuW{5!(h{@ZR`ikFZ+N`V{o2c`Z^uy0a*20-VgdDng|F=EC6