From 2c874ef6143031a385c1e8441ef69ae759e1fd3c Mon Sep 17 00:00:00 2001 From: Ken Liao Date: Mon, 17 Mar 2025 00:37:40 -0700 Subject: [PATCH 1/5] Add design for shared subscription --- .../design_doc.md | 100 ++++++++++++++++++ .../img/img_1.png | Bin 0 -> 143147 bytes .../img/img_2.png | Bin 0 -> 41770 bytes .../img/img_3.png | Bin 0 -> 106171 bytes .../img/img_4.png | Bin 0 -> 94112 bytes .../img/img_5.png | Bin 0 -> 87992 bytes .../img/img_6.png | Bin 0 -> 98216 bytes 7 files changed, 100 insertions(+) create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_1.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_2.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_3.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_4.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_5.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_6.png diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md new file mode 100644 index 00000000000..15efef28c61 --- /dev/null +++ b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md @@ -0,0 +1,100 @@ +# JMS 2.0 / Jakarta Messaging 3.1: Shared Subscriptions + +## Problem statement + +JMS 2.0 / Jakarta Messaging 3.1 introduced a new set of API for creating shared topic subscriptions that is not yet supported by ActiveMQ Classic. Even though the functionality can be achieved using virtual topics in ActiveMQ Classic, to be Jakarta Messaging 3.1 compliant, we will need to support that API. This document proposes several designs and evaluates their tradeoffs. + +## Background + +A JMS durable subscriber MessageConsumer is created with a unique JMS clientID and durable subscriber name. In JMS 1.1 only one JMS connection can be active at any point in time for one JMS clientID, and only one consumer can be active for a clientID and subscriber name. i.e., only one thread can be actively consuming from a given logical topic subscriber. This limits load balancing of messages or fast failover of the subscriber if that one process running that one consumer thread dies. + +Virtual topics were introduced to mitigate these issues. The idea behind virtual topics is that producers send to a topic in the usual JMS way. Consumers can continue to use the Topic semantics in the JMS specification. However if the topic is virtual, consumers can consume from a physical queue for a logical topic subscription, allowing many consumers to be running on many machines & threads to balance the load. + +JMS 2.0 and Jakarta 3.1 support shared subscriptions which enable durable and non-durable topic subscriptions that can be shared between any number of consumers. This allows consumers to share the work of consuming messages from a Topic which improves performance and reliability. This is a simpler solution than Virtual Topics as consumers can directly subscribe to the topic and not to underlying queues + +## High level design approach + +### Option 1: Reuse as much of the virtual topics implementation as possible. + +This approach will implement shared subscription as a wrapper of virtual topics. + +For shared subscriptions, extra logic would be needed to manage the consumer subscription to the queue as the queue internals should not be exposed to the users. An underlying queue could be created for each shared subscription on invocation of *`createSharedDurableConnectionConsumer`*. A mapping would be needed to identify which shared subscription maps to an underlying queue on the virtual topic(perhaps in *`QueueRegion`*). For example when the *`addConsumer`* method of *`AbstractRegion`* is invoked, a new queue would need to be created and the mapping from the subscription key to this queue created. When new shared subscriptions for the consumer are created then the mapping would ensure the subscription is created on the underlying queue for that subscription key. +A *`VirtualTopicInterceptor`* overrides *`send`* to ensure messages get delivered to all destinations matching a wildcard. For shared subscriptions an interceptor could be created which sends messages to only one destination in a group of destinations matching a wildcard. This could be achieved in a round robin fashion. + +**Pros:** + +- Reuses logic from existing virtual topics and stays in line with existing solutions. + +**Cons:** + +- This may not be solving the problem at the right level. Shared subscriptions are applied at a subscription level, not a destination level. +- Extra logic is needed to correlate a shared subscription to a queue. +- The underlying queues would show in the ActiveMQ admin console. This may be confusing for users. It may also require protection to ensure these queues are not deleted or purged. + +### Option 2 **\[Recommended\]**: Handling shared subscription logic separately + +There already exists logic for effectively managing subscriptions in the *`DurableTopicSubscription`* and *`TopicSubscription`* classes. The main difference between what exists here and what is needed for shared subscriptions is the ability for the class to act as a composite subscription. The *`DurableTopicSubscription`* and *`TopicSubscription`* classes have already been optimized to use cursors to effectively page in messages for delivery. Messages could be dispatched from the *`SharedDurableTopicSubscription`* cursor to one of the cursors of the underlying subscriptions via round robin. A map from subscription key to shared subscription would be needed in the *`TopicRegion.`* This is for mappings and also to perform validation checks on the shared subscriptions. This could be managed in a similar way to the existing *`ConcurrentMap durableSubscriptions`* map. + +![option_2](img/img_1.png) + +**Pros:** + +- Cursors reduce the impact on RAM for in transit messages. +- Messages can be passed from producer to consumers directly if they are fast. + +**Cons:** + +- Implements a similar offering as virtual topics in a very different way. + +## Detailed Design + +Option 2 is recommended and in this section we will focus on implementation design detail of option 2, handling shared subscription logic separately. The detailed design is divided into 7 categories. + +### 1\. Shared durable subscription creation + +When *`addConsumer`* is called on *`TopicRegion`* it uses the *`ConsumerInfo`* to determine if the subscription is durable. *`ConsumerInfo`* will need to be updated to include a new boolean *`shared`* property for determining if a consumer is also shared. *`TopicRegion`* maintains a *`ConcurrentMap`* called *`DurableSubscriptions`* which it checks for existing subscriptions and throws a *`JMSException`* if the subscription key is already in use. It will also have a *`ConcurrentMap`* called *`SharedSubscriptions`* which will be used to identify if a subscription is a shared subscription. This map can be updated when *`addConsumer`* invokes *`createSubscription`* on the *`TopicRegion`*. *`createSubscription`* will need to be updated with logic to create a new *`SharedDurableTopicSubscription`* or *`SharedNonDurableTopicSubscription`* . A new interface *`SharedTopicSubscription`* will extend the *`Subscription`* interface. It will be used to define methods for managing the shared consumers on a *`SharedSubscription`*. For example + +```java +public interface SharedTopicSubscription extends Subscription { + public Set getSharedConsumers(); + public void addSharedConsumer(Subscription subscription); + public void removeSharedConsumer(Subscription subscription); +} +``` +*`SharedDurableTopicSubscription`* and *`SharedNonDurableTopicSubscription`* will both implement the *`SharedTopicSubscription`* interface. + +![creation_of_shared_subscription](img/img_2.png) + +The *`DurableSubscriptions`* map on *`TopicRegion`* can be used to track if a shared subscription is also durable. *`addConsumer`* will also invoke its superclass *`AbstractRegion`* and will need to put the new *`SharedTopicSubscription`* to the *`Map`* map called *`subscriptions`*. This method will also call *`addSubscription`* on all relevant Topics to add the new *`SharedTopicSubscription`*. +The *`addConsumer`* method of *`TopicRegion`* can have additional checks for shared subscriptions. If an attempt is made to create a consumer which matches a subscription key in the *`SharedSubscriptions`* map then the request will be validated to ensure the same topic and message selector is used. Since shared subscriptions will also be added to the *`DurableSubscriptions`* map if durable, this map can be used to validate that a shared durable subscription and an unshared durable subscription may not have the same name and client identifier. + +![flow_chart](img/img_3.png) + +Link stealing allows a new client connection with the same client ID as an existing client to steal the connection. The existing client’s session will be closed and its network connection will be terminated. Link stealing will not be supported initially for shared consumers. + +The *`addSubscriptionsForDestination`* method of *`TopicRegion`* restores all durable subscriptions on a Topic when a connection is first added to that destination. A *`SharedDurableTopicSubscription`* should be restored as a *`DurableTopicSubscription`* by default. The *`addSubscriptionsForDestination`* method has logic to determine this based on the restored *`SubscriptionInfo`*. If a shared durable consumer is added using the same subscription key, then the *`DurableTopicSubscription`* can become a *`SharedDurableTopicSubscription`* as long as no active consumers exist. + +Cursors can be overridden in the activemq.xml configuration. If a user overrides the cursor for a *`SharedTopicSubscription`* then the subscription should still behave as expected as all cursors implement the *`PendingMessageCursor`* interface. For example if *` `* is used then the *`SharedTopicSubscription`* should use a *`fileCursor`* instead of a *`vmCursor`*. + +It may be possible for a rogue client to create an excessive number of concurrent consumers on a shared subscription. Hence it’s reasonable to expose a property that users can set which limits the number of consumers on a shared subscription. The default for this property could be \-1 denoting no limit. + + +### 2\. Message delivery to shared durable subscriptions + +A new class *`SharedDurableTopicSubscription`* is created which extends *`DurableTopicSubscription`* . This class can reuse a lot of the logic from a *`DurableTopicSubscription`* for persistent dispatch of messages. It holds a set of *`SharedSubscription`* which are responsible for distributing messages to the shared consumers. +*`SharedSubscription`* is a new class which extends *`PrefetchSubscription`* and implements abstract methods like *`acknowledge`*. +The *`SharedDurableTopicSubscription`* has a *`PendingMessageCursor`* which is a +*`StoreDurableSubscriberCursor`* containing messages for dispatch. Each of the *`SharedSubscriptions`* have their own *`VMPendingMessageCursor`* which contains messages to be dispatched to each consumer. The *`add`* method of *`SharedDurableTopicSubscription`* will add a message to its *`StoreDurableSubscriberCursor`* . It will then do a round robin to decide which subscription gets the message added to their *`VMPendingMessageCursor`*. + +![](img/img_4.png) + +The *`SharedSubscription`* will then invoke *`DispatchPending`* to dispatch messages to the relevant consumer via the appropriate transport connection. When messages are dispatched they are removed from the *`pending` `MessageCursor`* and added to a list of *`MessageReference`* called *`dispatched`*. The *`SharedDurableTopicSubscription`* will also move the dispatched message from it’s *`pending` `MessageCursor`* to it’s *`dispatched`* list. The *`dispatched`* list reflects messages that are dispatched to consumers and awaiting acknowledgement. Should a consumer fail to process the dispatched message, the message can move from *`dispatched`* back onto the *`pending` `MessageCursor`*. + +![](img/img_5.png) + +When *`acknowledge`* is invoked on the *`AbstractRegion`*(which *`TopicRegion`* extends) it gets the relevant subscription from the *`ConsumerBrokerExchange`*. In this case it would be a *`SharedSubscription`* to which the message was sent. The *`acknowledge(final ConnectionContext context,final MessageAck ack)`* method from *`PrefetchSubscription`*(which *`SharedSubscription`* extends) is called to remove the appropriate messages from the *`dispatched`* list. It would then invoke the overridden *`acknowledge`* method on *`SharedSubscription`* which could be used to call back to the *`acknowledge`* method on it’s parent *`SharedDurableTopicSubscription`* . + +![](img/img_6.png) + +When a consumer invokes *`pullMessage`* on a *`SharedSubscription`* then the *`prefetchExtension`* of the subscription is set to the pull quantity. *`dispatchPending`* is then triggered on the parent *`SharedDurableTopicSubscription`* doing a round robin on subscriptions that have space in their prefetch. This means that *`SharedSubscription`s* on other consumers may also receive the messages in pending. This is to maintain balanced message distribution across consumers. + diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_1.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_1.png new file mode 100644 index 0000000000000000000000000000000000000000..e38a54f904dcd308988c20ad9d796fb87dc36c7a GIT binary patch literal 143147 zcmc$`c{JOJ|Nq<8r!_NLTBb#9?P#m^hOq>RX-!c|iV`h?Xlq()#2P|dRaHx>mQW$n zQBp!swTq>e*p=A#T9Qia1c~jw&3wM!d+zuC?(eVrx%c$wInBX&udmnh^?W{Fjk;rQ zbn2(`KOH%8VsTaW)~UvyhTlj(RLv05eZ z=3CRXth{;a>`MthD%6+x8?8&f{aW=SNHO?6A$H6%n4YhZ-lGm zsnd^rxPbC$M?QCJ48Nj$vs!hiko&uL7bDu=%p~^oq;N3juVlHkI3fGb8`n>ueg_@EVCCZ?^(Vwiosm)}UKjrH{Dz>xkH@FA z6~2O^j-Y8VH}L2}I)9IMS$E{}^E=01&0e}0Wh~q({`${LKmH<~ z^x+q<>-jf?`i)7i+I{KpR{Nb7qstt2ZFjCt07Zi->GHCq8@kgN{BAjfse2Icj!t;Q zt3BAM{gHe4H~nC@``|Xepk&G*D_(!YK>t_l&gf)D4JY5rjHyinXCLe@AM8Vkg1U>R zxT}#fpGlC#_kn9731C$JGwZGXf>1vOcs;dSk0~c7j4(vb{f1m-or%7r3i1XSZAGYt zIP0|E>+*ReA+aBGz$-X-=XNVkvd=(YO+~DbaX0u``m;S#M_%uk3)fPGgUS&d-%k+| zUKp&|ESV=T^_W5%Ycm*QJkn@%QW#F#X{Bhgz`J-Dz zdis(tJ+;d>Tjt}So=;du4*PiZ$u(BtP+bFU!bdVVP9zdb$8gn& zE+D|($@R@ne5lf&1620&d5OvG=8)U{(gD~@gL$NkgE2l_^g@L8^wL(5)4aAEO;5!( zLA0`gYx|o+hS10U@(x^|Oa0NGGzx_sNUq@(kC#I}w`wFT?^zJlu$J9p@fhD`SBhgB zfh`-!j{ljQZrhdMVzaboLF$7K&y$t0g`ltU=nNfHI%JQa*i^AJ`;jJfPR}6CGTrBc zR4y(UeLvUg-Q-8n4KTV_=5gq7bWHocFx+#G zBfjXddBd?N6XE3QBu4?MI~L7G;~;mWA840vu@cFf=kzxz~?eks*{&^)cE(a5Rn(JZdBwL#BK1V za5(-fOcr4+7aq4QBBz<D8>S&873N{SnV8<%%^PW-nin@TJdrlTxezp^ zbXEfwpEoq?@I@?>_Yp6uE6eHXSL~M?2pX}Jvl$+4WQF^Fs9pcO3KHt}dW4*4hrX%^ER()6aEHRJ7$PXa;ev zzqZKEFD4QHIoN_~;Jau)kRdMFU1Plz^HkvF3hfY|$8SGDr|*9^n`lwnzS$<-TcMnf#kZyJG|SRPRiwlmQoMM%H?D|@l@9?t)h9|wtLvQg7POYd)Ic% z=k}aNqclqiH?QwvypP=_o8}}l`H7O=5FJ5~3CM&aRm#4}Q+B4~Y~%W(X*L{dd>|^S zC(I#j$S-gX+8xxTUi&WImTA0oJ?tUxBUzZ(1=~{?RoJgo5 zDrv~3tV9_2Q|tmODU0{o!87)rT#h z*CZG(hwRh%zQ*U3%&?VXQC_k1qK`rq;t$hM-;+VP`7Ck)yDv)jz5+p+RY{tmyYg7)$@1cB4Vc7 zM*C$0HhmJUSe@O~m4t73vL6oedrdri za^@G!+W4!66^|jQ@W{^c5a6TTJf;BB>pqT4`SK}DZg9c}>R)T=Px~3$>qaIbOU*1e8)w)`O-_BOW`TjUWg$hTero+ z0XiozM|wKSfpr`V`gCrW8n`?SwAt-sj5D?QSE!h0McJ^(u7XWm7NT6=yI!@q{Ch=@ zz>IS{_hZK1#<4t9r+ik)J0p`ddWKj?CF0V6oB?}586`HI>eVZ-+bR@OwiNnw@}u<1 zh?>x~Jn@mmraZk>AyJH;?P(cZ*`O3RE8L5Oojdpfy(wXrkhve+YNV2PKes1X+C?_u z_LetGbyYU$0)>sm{Im2XhkjZX=GFUg>Av!ynmR-27_;s$r-ht#J(aJZ4wuhm+e0GQ zq#-fcyu%}6&ts~I)U&4MC($=8X{{XF-xh&$5m@^SPgR8uBm_UL4P^Rx&DDGTn5GGi zpDp&Sx#kGs*9P@pF9zT93qSSzh0O8%@L8$?MRQ=~xm=*suP607mrnS}y}BC22LjCpg->zd#b*5L@!V<@8`5SgY2Z+4?XsYE5-I2-H|U>8j1= zu~$>{xlCJSv0h<;1m)y#YjM6?`hjJT1Le8jyLP~iF`NF@s^%2hF=>}qSW6iz^5$mW zrfnTKlf9a|v)hZR`Kc;cX$leGtu)SrrynS69*Nyzoo*emV+|i!{FigS`Qw$@OAmv- z&dlgM^l9TC0NW0-5~Xa?N*b3?T9YskeLCtvDq-@&mHOGF=s z8!cFNZKF(;w7qf$g(TK5BSH*^I&`sL2g`Tnj}LVO$*Y7;sL8G0%O!38)qGsBjB}wU z!K%3dS3-5_yu%uGcBP46)eldyXJ5SAK5?qLHOUgO+jKRbUk22OQW@|wDe_LWE0LfF zniR@PU}oYl&tM~Jp^S%mf1l*o6>3ZQl#R1GGp2paZu5@0k4V*y6qTy`i`jJZ9i2fN z&dB2rrHS9if8!t%rOrw4#gv=JEu?#`MX`62-A4!bT#?La^E}mcAH+kvE0fNRWu4Lg zM5>wTJQN+0iyky(!yKGjR#Eco4{lH?o0#6f|DD`03+%=W6nD{rk?FGI{Ai18T&2S5 z{BB>tr~FU2>DI|%BaScaW)rT^A;U?W@k!ps-h`VdJK91)fZODQpB01$o_r8#R=aUF z^H;CIHot48L-ERN&`VH~17!QkImefhnSs0y#7YGtNtR;dc07c0wq1waqeA+`jwqcq zqP-!Fsh=Ybsj zmIPQC` z-PgXn7Yy_0;!Y|1`^ZAoX1=gbr&$IgKf9h{#jvlurgc4ye3oIaofB>SPBpZ-6Korj zZ)YCzwV{2Y1#hXu&5{;GC((wCwWDapR7s%kl{d77|6J84fK-c#{BR=p=?ghD-3G;* zQiZJ{X;;kC+~VY)e4$~Jt@7N8+OiN+d~(oXftfE+vC{3hPOc?{z41%M?C;MtRMb53 zig1OqA8H|YRGjA(Z9P}yaNzoeJC6jnGneDmDWYqIWlr@K-3(lZUY@~lKcA}0i>4D9 zs+XHRv}JDj`nhYS#K#%+q|I^qy#y42bQmoE|g{` z2lNT-Oiz4T>atqV`4qPDR%lgB99Bya-A0lSD{Gl9bq&AaAj|7(BLY~Fc1b0*U{J!+ z-gTkn-ic|tlqlmTQRuyOOVu&EwRy~#bJWEHKqMO31LtleVQB{!;@tB>=GjTO`K<-U z)zPr#aUBMEKIw=lx22Wy_=;dRJ@>9CfznYs?%Q_4n*l}m&g+bNZyzYEO=ck-#?lRG zDmS0(ODaU)UNrVRcMtrPY#LKJT)ssb=~FBzrl1GqtAd&s_rrVz{j8Bja1Nyeo`XGW zW6#TUxir9WzgpVJ(8H#aZik|V(&0JzEOh#MOcFXor#2VuFjJX~KI_P=eUXn6A(omG zEB+8oDAgGhZ8g44Nr&t5B&GEIQ9MRv!`)!oMZ6|3cv(L&(!nbZb|uX>L{+RkS>9;< zR&9c^?yjWI*Di7k)&|7x;fi`mX>9z^26^*|DFGeQi{pMT`jhui&qFqSwRfU(?pEsh z8JN%fB=?EcYqUewrUG4+*?d>7tK<{QK@KsQ2!=T_SO95y4y}L=61{zg z{4k)`Rd_W9DZ1Gk%dDdf9yy{bayHP7zQka=9s42%wNC(@OS>7H26j>3 zw1oIPfG{46%_fQ5>KfBo(4jRQwyK(63cDsOWekAH;fS05*dPUg)v+5C2oz^n2R}3G zlHtb7q&f~K=%Tj!W^NE9r`66u#|n!gk%_Y*+MB?X%kcAoocn69H7I=*&nwTB2Kxxj zW_n@>sBybXW0}8EY2VpT=BkFcvU@lhABrH7Ip8wZ z_}rr4(v@|;F41PI=XR{)heiqvQBmfTeLIi)2*UfqmaH9#J}-*cQ}n-!lXNhzD4W${ z*2!Eq@l8kQ{+`}#tSWdCdIDaMNFQP)$?i8sndLZS#Q$uyI8CQ?|5LVeZQWyHTssoU z3$5SmU6oJs2PO6*8CC5nrrI%r`bgL{rBv8L27?bN+4IjOP**C=FNm(T2RsM*nTwjE>H>g0&49)Am$L zY!hiBXlFnfj-R;jkn&*3I#1^^=Cw(l=G+fyppBbqv!g#Ns$_+<#d|z}7f8;1_5jSV z6#Tk^k^e^jV9DRJa-7R3X39jVNK9OonKra>d=mAsp!79)v(c|K&!~!E67pHTq-gVH z+0d#~n8ryoka+ z@nDxpa=}6T_lp>f;-gsa{9gy&HVzE;AL^5|Xtf}tAq&Vj>SBS(7}=vkejViJ6{H2xa9; z0!J-5ZM4}3S7IL?b{j=>e5o-}BEAbTn5Z{=3|89IAC%A6ZDBLWqP zyw>sglB~S)cx8^aYgd11<6}hIYcm8k*M7P{voNMl7hZ4|=c^5~V>K%1rrBa=yZm?` zjc5lX3glKWL`AKJkYl<~=?^kNQH*@#mwvEvOLF;u=db-w`y0!Mp3y3Wt)Eg(V2&@? z+x5FGQwCPYyOV0hninor#v?+Kkcp<7Pex0nk(bXQ3>g{gX6`)C`>-az)wu=tdpLh^ z()J%#Ph1vHNg2p+Dh4LzKg%~&*}Q|%E*i=k=F~L!Q;lmcyYtSp$75E9a%)(^swyB} zftyH`V8I@Q2JQld)m$y=isz+9t6}@BqhL%@Xp%<~_Gj3B7c`fotD+G4VsY(X?bq;T zMbnmTB*|eze@>f)`LIBURHMrUU3Xqqfml{hl?^5Q-j_JwfIu-h3#@vrqHVQVu_n6Q zUY|91`i>*G5x0<2U_0>l0XwLEzO1kFQm@~J(?94p-MCv<``Zj(sr@sbhA71^iQhRn znw%&N?HXhPOwgV8k&i8@YfuYJKez4hRwMO==1Qm-H}jFAZ{5G~hXFEv`gu=T1Jbq5 z*`90px*u+SN%MpnXsD8);Qw=3gD}5d1L1A=c2pCb+H0TG`rbTM6CDxO7#)Tr%ane0 zH9t!R^tDW2dP5}6?Ez4^1BO^akwFRxVb-dzNI1W%RhYMFf#UiefN8@4W%TMKq7ydv zTX50~l~iMWmo{aR-__2WsoEg)vEVR32R^V@*$``fsDk5^o9EQ3c{H98Afl3H^Kd#3 z;V_?u_R8ogK^vJF`TllXhO;{xmELQgI3(s1Z5BL|5>To_I*<25s#B^*ic%$XwiCJu zYLpP3aH}b*mZO_$%5?d5BPuOMBOO^RfAGG+&6KIWl@Rz{OJNM|k$o?IV9} zTz4!=Ew5cXhQB5M4QUQ#x&JU(*Wpl3^`W2UdU=FtUst=HfmATNpAo3~{K zI;t>V3+T}?^8}Z+cLY1pUcC)inzk=%6+OZEB`ELgty5>)@23%3{4mcZe~8W)d(%pm zc3`1}zN+`h;|tT47A2jQap1fWJD--BFS}!~6$k3*g7VmGzq9(%-*H`xtDbCbMR&`6 zp`cOyIh|ldlGx?oI_2Q^r)~+g)azOLb6z==lx?093~J2o3Iv8!+j6YZQ~LHAxuJ~R zWC?M~uebe92v(Yr)zZYaMC?|-FQB%p6KzB4)P!{v&HM64>_S#S8E@^y8EIf;&U3x` z@8R|lm*b#-jwC>?Af)y039VD4HyR@Zs>QOX7hrx}>mlF92ydL?d8AJ1_f|OKde6i= zzCcz_Z9^REnaemra>@>Q;`)u(c^Hw*KU((MypbNhNOctBSK+)66Oj{S!7!$i_UXz3 zku(EjlZB8h5PY=$c12ZZweeEv7>})<8;N+@S&sbn=ZVvu zt;USjVVvVFRC#|8tfDy|QF5VGe19p#g8knAWgsJM<5*bmZM{MV>UJz^9`j6}FXk}P zana|hozO4(%OPy50?2dhf}MpDSL)_LnbMCJda%;&8Ii)#u_fH>eD=)|IONlVoV`BSfw}JwtvStWw-&NFtG>qEVjr@_0?+natH~c zQVKpuEoNTyL4Jg1oR<+;3;ZA_ozMZef4((Sp>{=8-QO3->mR#M^e|Nx_bdu-L(m&z zKPuW#S)ulTDhHS*L{nb!V6{?K&5Q(Kh{X6a7+-NAY${Kt@qymFiK zZnxqJ8Fc5StFfg2P0JmBffqQ+6>)XnsxJZZJ+&Y@49j{O;7}#@aM}>inw2v*5OY6_ zHXUoYM*7cH*M%Ct;A!#=z04_wH{kb6Ma4zSXsYcM0l|Vl3NS|H32>qUew%zs>7STNR;r7htW(t1b;U-e0-X;14u1dFOurr?>7)N3V0^0zstvE~6Qsvmz9 zJ1kt=SHxgx`tcM*X6)0nIc-Z1?}c>2^!85%ZM1T(R?aE`miDOEIj;I(S;gZXnaa{4#VTal|&*fcDi0ODPG^@i1AXBr)iyR~CTq zTrLC*IlwOsZS?to&@0U?sBDZjjoD|9G*TMa#hcyOB{cmWU@{q0oVS{X{{1k%D-+A& z38s}=`*Rc7PKORtM0MMl(sq%P4TQtwO`hoUKw`LG(Fpudb;a*A?%6`&JiZ2*$XL8z zA^7M=-A@l8&DXL&k@+VZr`zAS{0t**Si$FB8DQs5A6j9^d#;d?xx4<?nhy;I?t?XbNu?Q05qy zPp!U;tA8h;QyB6!m8crHo;In6<+eL!c;KsNFOy6iSeEMhX_LE*yapIldrIfq3z6)C zP!5^CN|Oou?l#o7zUUPufj2n#)Uy43c2z=UNQ_~q`pO`5=&NE+S~#X?+n1Wsy0!1b zf9PTWX$a}y7chNutFw6Ef@9Kk+w45G!91QHvLAqUf&a zh&ttD$&_N(fuG>{cmhyRRXE3F*g$tG_kLu*E6(mKHdpN^GAV4%2gr^TAs+zkc<41B zp*8JhFo0ngE@X;M<&3T$2c-;RwDRTFS%uC`VG=fU1MMjVrQf{!c@I`z_Qf2rJ9677WqykP#T zZ!g_$eq=-S=NKCzAOC>C)_uT9+W!WIzw;icLuw5G{7v$_nHPrcJ0md}Sh>$B&==*_ zY*D=`X^=n}iH2T(i&#M)c@?e5@-bw)wjj8O-jxW5s3;a3`5z@k^+yiqJ8 zB13~ux8iX-q5~i}faWCjhF_1`*jHr=L}RVeeQFvsM8b+Qw`8NV|j|f@nCfl zVp^@Y^mEyY1Yh2IVhL`=rOfcddQ3E|sjbRfb*c?=&2!)hU@CRylU;K}rK49~jYo&A zFpr!xx&G!Z)j#dq`7q|CR%0bMtg)noBrO^OHtG|E9yo7gEn8SR5t@(1n&`z3(!f(=$paI)R( zXU@|-yn;vP4{xIHR@Ef7a-f8w0a<(uOPknR3VU$0NSA$2xZ+OLB~sNTSW%gjYvZ%YZBwD&=n;x(t&R-EKQ| z1@)NBY02u&rWP8arKE7PnYbW@ouu5kqUPF}4#y1|lc`Y$d_mU!R6s)VK&)=r+OdjlWkxN=( z>sll{fPSqvO@!?;!_0*+U&9$fH)=Oa3euMVsLZT(E4Wq*J1Z-bs=Ckx*reUvl}l*i z&h2H)U{yBRagRk;_C!U`ySwU+hFlN8SB*WasSEB+$oO?{d~$Ex7pTCYO9E+@=oZe$ zjQCI~cb=s3`d(kL;N~AW=ybh-99Jo#A2R(eG*Xe+&=hRhkNpt1&;S&+VXmI~Yw~1E zs0kg%5}z-w@eAqIXYSNr>AnS)nftZZbpiO`V3tsjz;1EZ#q@0Cdtu#p+cbS>dqDM!)Vd;86+3TA5(?XHpx5q=)^>`9R=3d< zLmSmpdcR4|pVchQh3!@X5J*iMF26WHr|54MLsyVBmMvW_Y^r3~+`_%0iLDQiWRdfy zb0MgI@`6vgD*xxCTSfs!Q0^rn^q-$~AB^6x`;skFB$*?M0|*h7)oJ0(gG$Y!w3Uo+ zCtXw4Z|9*u>^P~kZCJoBL(_>o>r(4Y3q7vx&c}?U@{ikx?OmzC{y*5>UvmePZ+g$Q z;pJ2SZn3+d_k9)1xN-ueD+X>+m$4+EXlIDX z&5`Zr6zC6CgNNsjiI&%GWa{%W>0|^aHwu1=`tc&eb3vz&T}RIlz<-K(mH3=#cI9|t*>>GT+*Ch=lHui>DPdMtHNKdwX#ee} zb>?R15;N3u|JVu95M0>qj9g7l@S>YzaOK#Xiq>ISO3}w{ZG|0dt>g7-Rw7hy@BV(Q zsJg`f)82WVcKjlUE-JGT)e~oiZ9=;I0nfGizqo(NtTT51A-~eD%YhUJE(6TUOb7W$ zK!sB+A%GfBW5cC5pBVa&o0Z(7DwbkUSz{W@4bO8M@%J1yGnSh(GwzuUmgzXwgA z@n;>d3ih9b?rec@K^?no7bcsjz>@)S;Pv7&JpIUaX7I}1q({E z?lxyR#*#bnn9|Y*KgsTkl3L8SU&9qzPv+~U@1~y`zp7GNUEg4w@x+m4SZx6$lHmSw zS>5d??SxtJc|%+MJMN)t7n&|u5kZYv=7(>WgfqEYa<@?**`bVeiyP6Q>aj;+;K3;5 z{mB5mhxS}MRebJgCFiQu!O1gEN{HglH?z z=Sc=9Ei)8WIhwkQduIv|ISztcx{L${W5JpJ3Rlv6S(FIm`RCI8ANGq2<-n4f7Yaw0 z7C(w=yex7vrx@)IXnc|@us88wJ8_CKMrgBn;K#r()$nX1G(6V3yh&e3Pua))2oE*U z>hUwF;j|qe1kohX->1$B#8n(ku{((t@&_iRA+g6Uu~Jf#*_VYvkr5(~Z06Zp zd$HO2DGwwg6%E8gAkR3}WyjUkS=kLm<8 z@Wz{qq>zoYAZ)wj(q13kITj|hk2+SxINEx_8AY&p=UjTu9H0QC*7#l7Np*Gv_8AZ+?2;z(M(P`dv{DX5WYc``-_Q) zH|n5DmnVONxV8ds1yqn|na^Qu_q)elVK*8fioT)Rr~#oFFd8neeAxlbLPh+SUct2KRpIqgitR|84V$KJIwXzDw=G!$Uy) zbOJCalxEEwlW-ZO9m~*j=h-@$b9N&$Fpz*pstbH;ecQHm6C-?^?cCl83Yx&w__8~Y z-)L+|J%3v_;F_0bMlZk00&)r+CoS_qEK4V=;)6r=q&wZPXPbF+@!9W1L-CAwp@se5 zKH09hw48cb&WPw7>o7{&ZSlYy)oK`$QZ3CY9|APuUM{r+fxe^C?^4|}0WcHbBt~ur zG}kKolpfE*-n}FvrXYW2sgRKd%wpADRfH#F9YJRY${9JoW_*fPOb*-zrafXJErNR? zs-4eW6rTQ8Pi(r!mDnfSbhQVR2Pb}ZGaxyn3&)KJ_?`r{HEJ_ke`21WdEF4_nShzK zkI;AtYK1A4-iM6_jrv61QkoWJ?-XNnnxX%B?0!e&o4k)c0zjZ523=#{OkYY zJSxk11z*k&>LpFbpWM_8M!n{#ooKbXs%2=x$te!>I6I@W_Z9TksJb#-njc$T{D)PX zPm)9}5RxqX19!rssB}Nf-fSL_(7np%w1B=m=}yT>&;r2u$Lzv+8X-P3sz7sa9_M)% z#v4TZVKq>mUej#Wp)F)SeE=dBZ_6+ zI`H&LkYi4XrA8kzDrbopCcoqU(F%!wOAZR3onKvoGV#o2eqa78|3CjRho&wq)yXsX z6TtCn`=K?b+Rszr9UIMMC@eWSK}84Ayn$BF6VD6M5USSpr3Be@)kWHV$J!anEHwA> z3l=ikC1u=>w2!&&H9qnTon8Uy`!-*>;^ra-)QT{93dd4+mGCMYFA$ZgF&BwJ?7=PW35;LiULnbI2*WifWEvU&T~uZ?Qu&)ufBl5SV&T$1(Z zkm`y*B%`O_kPCU!_?ol2p9=mf{CQ#y(5!J5dKGk-Lz1i@C&^&9C4=|+A=N|G5Y?|3 zyL^9m?yV+@mkyY0bM>*RNbSJirk>T;^M}i~bu-@6_$9Rl-DBK*2{GFpM81>IwQG11 zZ{=KUCZmC0o_nF(0I@B({U*9-<^2iYkdJBg;nzg}HD+&B=H{!h8Hzz=tP_}DZyleA zF{{xy=k}xx@y2S&Z1FgRe+GI-U1p@)b|4*c*L5)u&UvmH1?20}p<~Gw?7R3Pa`#8k z+0s6Nz->u9jX-Ugc`iWAYY$PE@f#N4rRp`dVOO3F&UEV3SBRFFW7YgQ??q!`9)@yH z1#Mnj&otjT5}JSrRJkDPDad3c4bd5x4`kk%S`4@R;Eqdk|I4oPinaTPociG(V?PrQ zAY_zK;`r{$Qj6-{dj(K!XKE|1a^P_rBT3s}LbUZ0QFQ>%Cu?sj*m%0g19TBUL;v^d z`QLcHr?vUnLFa^UAUl7XIbu)J2L|W)$wiN~SxHG*GzxHgYo@K1cO#d8fo>xDcWsbt zQ1*iQN}mgV$z_R4RynXMfvZp(z>Crpf)RpjB9o)z!XMg1L8c4xX#5L zY~#S1;B}Xd!{Ls>uextVJKQ&#N8y6Uc58#Us7ES~_JRE^%sy5>gFCbSZm{|h=8H(i zHS9*FO!`5sivGb`ixJc(uBkR?#tv2x*rb>(r_XJrAFgC^jWTjXWcH(J-0rY_u0*VA z$TtBJpq0c$4%WBS@D&{!QLqr?KDAD(`r9e`O%T^Dq$wYare@6WGwB(d zpM$!&r;J%2M0X3Adiz99o~s$KLx*P&AS&AX^T&4@R=bPpv7vCy;(NG?-;AmhloJs-cOXUuG$*NY}WIi`4=RJY6Yn^D3+ z$5AB$#pc+j-y1D#Hnx4Y~;MRxpMSnIt@-zj{y9&idWqd18StR;YmVL5-`Fc zn!L_8bjs5^JSbL1G)?lA{h!y2Xz%LB{7PnKPDvIhl%FsNU0Zd^luE~hOwFBmS@~w@ z+l{pg@Amte-brY?e0n5^5S$j4_2bzJVXfpuE55j%+Ax=KM}9k-u&;Ln{VYNSTQ|&) z{#7fm5N+U!7>M{F%6K+w+tYRYv}iCxN!1NG4P<(u1K~JrcJIrHAzqzP{_NbuH(6Cw zK0KZKSD6iHO|}Rn`*Ls4JY>^G1)ac8sF@F!K~Zsq&0Vn-{sV;5ef3SJ5Ujc9FvN^4 z&aJRxuDMIuucIO=Uu>qhps_^AyDj=Nf}1$Bmf{@SIdcT6k^)%9X3Ip?p<^?J#jdHO{p zEo%Lwb7Fb-vATEHY+=aVt=BVKh-Xv%7Yup6Y;!n>&6~n!&fL;nZg*eeyrb%JKnt42Cq@mk6XEPBmGmbWDl^pLJ?ia4+LJtA_1Aq!XBX$s>U(WqapUQ zfe-3vh4^t42-y0n$Cr;!Ao&iNG*HK2AQ17_dyV+jPP#%y^}3=v0lRf)Pl}p_I4!rl zzR^sb+izG_dyCfr#*04=z0Y`55P#g{atZ$sL3MFahZeLs{qZ0s1HQd`aQ{RTI`RfA zp<*sc@s!h@OQ*M`FZ0o=HpSU z;j+`gaDs(Q|IAE;BIGl1Bx%@j&u|xrhZT)5izd60T}=ul(c=B;ZPt<@XBEqe{^$CZ z2kYMmT$uvtHM_929+20q7JuW`006hVsTt%W7#Clm4RA}2t2qIoePQeY0dR)=x>BlZ z`*$%!Uq>`m-(CszFg=s!jm&}SZ8NpsLN)Z)K6398{aE%irtv_9d7kV&*Vq#CP`M;n zEOm4}-0{~nXu0n$dU|?@@#Ey8XFydHoYzyNCAbh|(0#vxV%Bj|4>+!*QQWah1K_uM zXk`lhvDGx4o8$PpM@Zj{;~+^ zd8+E{{hmg4)3aS_rtnn>5_ELynxq^?4lTfh`8oYD9s+UOe-~l6d~6nLE;bNjOJR`Z2ZM$y#|5nO->0YGXfBjzo z|0D5z3h%HD{A;?aRq>Gy89t4J=-Xv0`}v%gl+kB0tP7#W9yxphaVdwctq^stjp^(E z_j;o_RcdNR6kavb&nWYrKgK+Df+?V~t4;KN&QZq(A?lO&AG5o{4)?crb(H+Q8>jpA zy21&?EfE)MndN@nVN)Ukq8eG!x*=mR@`h<(5<<|!n{4$ZhuV-Ki95ce%=PDt%&1IS zb8Oup01bDdUBh*VJVPq#i6rJq6n_+SP2Ez@iC$mVSjP- zT#Wh@jd-7R0l4xeOgcojS7g1?c31N3%=xNyU?+vW5?9zLK0Isf`8J`NTy8{Yw*L}f zJEJn3tQ|Z0%}5)IUofp^&$`Dbl+9@jo(cV+RTDU99k;NipgZl8N=TX{@i>kL95A%JK1M8ntW(F)GtNNpMau%bfCKKA+A`OnTb&9cR7TRO>t3*W&6AsRuOD!LX`y_Jf9DUc zrnoo(aL3@pg{m37MQ=2*4e=+CCgIi{RMjYhCyZI)?zb=ivAz8H1~lq znBddsm2a|_Bun+b94g2}CXF=3u*ByS1Z!R=0gUV4)8PNfxB|)IKZc84l2w!6(5zFJ zaqaBK|FT;Gg46WC{InK(0vQ04xBGPUS+z0s*7EQwLLYbj*a^9wf8Q%F{%y79!*Q(Q zD0IC{f}pzgGNa^hHa; z*IRv-puB&=q2Hoy?f10oICrvbSIB|b(|d#oO|Z5B_JUy!{VpyaCcqf;k9pi5uTF-V z=%PJ2!&K-tYo`oK= z-Gyt`?rW^@pEGaUxx)0BCPfr-A0nvizu4~diY$k8H4lJoPEqew{f938Uphxddy1H9 z{;sHmu)Z!u44lfcW-|V=)42LpF`;3eEg2EiN43$o@1 zg~8YK)nnzlLOS~G6B&Q~T=2WlfGsQj`y3uN+@CN+$p99i6}yG~^{ItRew)dX9g1$_ z*`m!!{D_Kt{wTrFJzxo%+xX6{3BV%;`BSS|^{h01A7qNxh#IL+!XQQstD}V4IBLa? z*D7-ABR%LkB@K>e{B=8ReppRQ%Q$ z^Z(%Oy`!3JyLDg73#cHxic+LRMTzvLbb<{KDN&GKrHFI^=^=oKNSEFdl_s4?m);UO z(nBu^y(g4}x}W&2Z|$?!*{6+hzBT;MF;skVXWnzp>$-jq#NDfr)>s%M%T#}6Jn2s< zx!^hj_#dWsn0>dIi9Ru3R;Mbe61{+lT3Y@JzNXnHtx@Wu&Y_f}M_m(r(IR`#S@g?n z_O#|1YCulu3|yqB_EKQ8=DYOpUdyiV?2ieah||(rtxZW@&W|zn7ehf6A?quBH0pD1 zy56*$AkUVc+Ex<)e}=QVB0pFqg21(|ITD)b)sS3-`Yq)yza7Pv0%^%~b6m0&n)w3d zhb*LGN1Zkqerb7!3PI4C(R|G?vFuFaFJP7}qevNkC;W6N^W0|@RdSN;lb>1T6fgic?My)_Jw4_Xj5@^Lk*N84#=OX%HMf(3}(dGr0V*)V7cS348!>e0ZjAdPRBm zJ^ACPebB^R>t}22h{`l_@X|$%0cj%G+@!(keI_^bTgL>iSsw^W>DX$AKN0lI66eY0 zKKY2q0_c2Q@0;1C-cbxu^@8sgD=6KA(%17f$%_F!+_RD(;Ee4jaqfJj>F(W;dM#Z! zy?D8H@xx%YIUXrz4tSJj>mTEqI z&6e}ul)i~L(An`oCOLH)OgYbG)3Q|S;1|T#PAHSf`kBQKl(;HU%tSpS4>8pLBMMqZ z=9k>(q{OnaOaijJ4gdC)_8GA9 zCmncxZD!qAt&J*3-T7}Fb)Vljv8MhnrT)YgPhf?aG%t&HuZ{ULZLO@F0WFiVeGjsi zDTpmt&-PJlH1R0{TKdhzlE}`5-f+{kJJ@D?AuH8gIo9ald=kSr+By``?XeO&_hlhU zqngxHetGYmER!yya|%R%m3!Ah{xhw(cVL=7oCnOwEOr|3BLJfoq*lGni+F%dj)5<+ z&ZTj+Wc>E{8PwHN$e5>iR( zaI_afjaHGAk)Rs;5z0Scvz3~RZV&{!2fq+Pt%qq94K9&_yQj4p)i z9$8r_VZ&7HdcqO7;&z5j?nU|DfVSkA)aN#H6R9lrDu5IG7MSo%o(VrikZhEZA?rR- zpTs@evlsvjX#ISk{h9ZUQA^#NXw<-1FwcT_$F)KfC-tibEi0 zUG`2C4P%*wP{hKRdX6WkTW)f`+>=H!XsbFo_0F*(gr5_+BpD10FnHOPWBd1-2aIBr^;~ zlsxitIxkjx05hnTu9HfwaJDQb!o4)jb3Mz^s$@sh=23wS;Udtf=FZ%VB;<6s<;t)$biJ*RM=ec z!PR5BLTRVkuL13cp=VR}y;B8{&6w6uC%psVvyFbCJ{oEDUflzZkuKY!#$JN?SNa~` z)C9>>SMwZ%R;^zorZ$3Ys?VH_%%5hEP&{0`KJ{UfpirO749Fcii{`A;>=eky9z_W{ zl59Kv713N(Av;XqTAM8E>~})tA^^W{T{U(3Ugq&=DYN{R(C?dHh3V;qZfls~6hBP# zqZBHGsP*_NmsslHD<{UUK*>Od+XzxjQO9>at7fQXc4nb+=q zDCWH!nz`1y8}W@+>TBK>*KnS4vAzn!Tu+RBE%N38{&B!FQnLx;T;%KX11{AvDSZ z>KAj0W?NReX}8C!!L+mwMIwUF>%cARnw=jCW1lV*!RGn0Lk;n_8H@g^*$#J1HOZ0( zW>J|x)^0(HTa!f`!S-azFdb*bv2+>q0$h(ICn1wiQ#^0BrjRY)niI|kc#(XYj%cph zHb$t?7o)QU=?N6kOMd3O12CscrgF@zSIbN;s}H1uj+dTbklT{##7|Q=Kp7dxE-7l} zL(s}`^5eCO@O2pimOZ-#|AqC%;MKzsT=0*5@=4vlFLsj}PqU%sIN>kT^$c0JO3ILD zzL^g?C1HA*Z(nPsLPxQ*xNd3Ffw!usk2eM`f>VX-yE+YMYB1_*cHAfT<16mBMM-TQ z6R%3T2(EdIr!=Pa61O@}YB(Ag>g(nO9Fv6ZoKjGfzxu)*?$;92HdB2XuLd=yvWa*W zn@1esJF_ozPWe@pIC(nZ)Bc3I`{Rv`II&Bs&YN4AdjR~jj>zz` z1pSGUiHUehBMCVrBTqf+dOZ!DQxOvq8{QL+ik|FhRT0h;PXTCL26W^9lV5kmq!0Hf zfcAo*ZVguID#M+YnGZpYi(DIkpxfmF4?i{D|21;N%TAz3vMnVfCApQCj6leP)C_`{ zV4bc2=abo|W(?hqoIdsfx@2J^z;B2|901)f)c0T@`;Y&VA)in*Nji(AsLqHDw~A`D zipo}je@wrc2tq9R)SqB8P5Bp=QGmeRwZ8~Bq6a9FV5V=Lly@VefSZEY9p*wH*HI*g$~Wck-QT@vlvx zR4!ShhwpCc9QkRE@hLBAHFW2bNsvNOI9TT3PK1<~kDi!Lh;o`Zs;Uq^aheu}U87D2nj+S3|a-p!JO2lKFK$*s2v+WF0L@@@#Hm66f0c;bdzs80N#iNRlCAe`} znA--oL{%(EK(79>4*=5ba=bPFkBxE>e*Z6IQGk?2V$C|hNE*Th?l&CM6M9lAJX;+&0JPXbLy zfn>8@mvj4i)EN0iHis#YY|*=^=e}!WEw6PPdGclW-EhvnW*%od3=sg}Y~iC}Q&aC{ zWQ)Z_zLMO}JSBQ5nS81Cmmpf*PWu{w!NxDbN{HBF0y&q{&uhpHQxt+KTl11Pe&%0j zi}=NMw8m(Aqk=7`*4I_98LD{Os^S3lAhwiw!J_`l1YQe?p>o;*I+MBRX@vEY@6FkH zm{hUskb+v%%Mt!Xo?=D19;^#Lt|d5aGs=&Rw`eAuV7v9G^{0#!B=wIx5ot`c3N|ZS!2m#0ai)lDfVTm4asRp zv0Q`R76YDm5K<_fL}Hf&2wW$7k|>KmKJ?fDeo>GVQvSsR-8M4&3Xd8!=5eg zh6d&psn(`$&)d8Br#(x=i5v!T#o_acGUCIl09`PTleSAa43%#c1At?#ylBfiM=5dQ z=?rGp1h3Bm6}wA7a++9N`&(!22jG3iw(9#nhbdyH#6OY8*eu*rv(s2$uX0yKb_XF> zm1K!P9*O>94@3+K2o1%`K(flKO;c6`*!8f%ujkpP7z4mO*&DQQYVwzq?O2uCEeH}j zpwW;ean3z43gR=et?aALo$q51H7D?q9K|Be(ay|Q*)aQfnPbxfbHpkI*+=JD8QN`^ z^AzEa_Z(06jyz?I{c&u~HH3yd%bLmeKxOo;lf-=w=_R}bIW~S@EO-SuMN2q&wr-#$ z>b8%Pq^E8ye2+N43Od~eS=Cdj9Wf7HL%>k6I4;8}BXLz_8(xKZhmB1FBC`Wr{O%_c z97X-LJDEahLU`lt5S=dhXKMB%wHvOBGLnuljHejiBMi8Xs*CEDyX@TI{M9L&1p^Ge z_XTIL+-R$$tGl0hlulk1)WJ)AR#Gm zrZI~dsgce%mc{mf-|%(r8EJRLi}fQ@ch|ss(!2S%5+wU@BWGJEi@J7xZB=13Z9$J! z1~UUS=#o{TF;BMmPU>$xqLiqytC^pCbbz90l!;4WQa_Qy#Q0Z*RjZXt{(lMP{hPS4#(P?XflLbAEvm`;H&#lkudxOV_jEcD|6AS zt&W?gE8&fS!;FZnk5cp|7iOCxD+((A#?OTc83C@UghY7>OajDXJPy&=%;!p1_u|ne zX;Q>E4HqX1ImF}z>|CcOaqd@Qx`9~Hcs0y3Kxehm?=Gh_b=j7}WyJb+!Hmo_xrcZ{ zy7Z|wInA9?uT(i^Sl}&OoFXMIU7kJ9+9%i|tX<^#wT{_;yjzMfl&yNaDd4muE_K>2 zr1RXIcLc!VMzcaeo<=%S(uct$T|`~@Zx z_vpOu8U}7xmqhDn1WXEulNp8+Q*BgZO$a(%irIdGMRpZ%vK7gjyxW$NfJbf+fQ_~t zkwK)vXr5Om$#Dt5V{K9*Sd(u!kk{ES8*2E}q7U7+?F^7w!cg;P(kNcF)c?7+=w5@&5 z&{%jjVDd(5js&-Dl_G=J*4yThDqYuS9-d#Cf~3yNo;MovxE@uJTyXgah%dd%kZw@A zLeeHtjBrmdUr|y_ltD!7KL-(=gDPZOfpOBltxs4^@3RPss>5z=V9A_rNF$SrAx8W+ zi8_PzuAK^u%pTCOLsmwF0MSzt0!HWQE7@TA?|_|zKG@xprk?hiM>o(^U*%xi_>Io9 zoM5r00gLa>^pChce-}1brCvB{y%{N;A;K667{s=D?>_OJ&!0FRb)6A*_XEqzl5vvr z%AYY{%&qfDfIkb+(VYsT{_%SwqvlWFdzn0Ehb-j!{%@=pt=Gj^CdAdFJdMe|J>%uR8%-cWbfqe^^5X7hq6 zcPvP@k{Tr&(O(nI!vo$&Jcv+Q>UTaG5Es|i#m{&`h0osQJia}!pNm6^JjyO!80i3` z&{3svk#K{uzpIz@A5IOX7p9YPU=6o>VKY~l_e!n9%PvwWsjP{TKi^$wF;l5uN|cbj zA)5@j%EwUUHWc_lAzO`vzh_K`51v~Sg&T}qaca>4jfYifh3@1PN~J_f<8xJw?uF~$ zOuxdQmNxA9^4fwBdg^Crm${0Gk+s~#K2WTEF4Mt)wpnlix=9n0Sb7WeJkK`D`RH}^ zy!#6FyCcVPXo8@(-B53$zV2?D)k}dNPlnHhc^s+MDt-BuSNA@b37c>0B>utBbwD|M zbz9SsHEqycD*hBUKMj5>*c_j>ElT+xuk$uMjjZ1{&o{l9 zYhhkTJ0)T2D^|W-*wT`uV`Ei0_<^w^cz55zH%Zr9TJ8kVWyMvO&~C+=7T!`!=T<}l zR4&Tiz4!{)fVZq{8Lnf$|EbBGrp0DwLYbN z6S}Z&vHtM*sRMuiV3dyLdwnkgb30SJ_etq=^ghAAS27KIW4gI~?k|;rsa#zrSf4?Wf!3sM?T7sw-*6&?mNmdpb;Kr}eK9)c69dwb2H^rinx z|8;}2h(L3NNVF;XX=wa?`Q>5JeNB1K2Ow4f^4@&OX7k97fit&!m`O);h_dmRv9_Qh zZ_hDj8ezN{Z=-e>>Eks~ z;6`_-zdj!9fj%U6sm%cdq5aH*Y7U?}R|Gr3|~|b!~wx!dc)&D~%))2dLCp*n1KkX;U4iUj*KNWZt25PURccoJL^eu8+4YQQoe-L)`tY*x4kfzs(u8aI z-HF(2B&K|k*Y(2{0#R=Jv9kBU6CmIG1aZapWA1i+b!0hrv2`StRIy z^s^VmQ>O}>vEu?VNH%97UDP83(>1LbmQIVheAg0^H+U{z%AZK;h`C`>#I%dj0@=WJQA6Ton4hD|6sOS4i+~ z%^5C}m5DRYr^Y^(FI<@WTAHJ8%0+x+QiBnn@A#dE>KM*ZSa;8jsD)+O%z8smd<9Fa zH$Od#RpTth=+OAQzh@IZgt$f*&iKi#<%-hUS|{I!|E$_K(G?EX#&k=Im2-JL#RjTW;w8Y#dTbS#$?h@J#9r<7g~B_Q$_Mh=&udc9eMsl;hqX!8bd+27#x zQ3>OmgMLnvs%vs7J}8ZOGcWRv+4Z2>8+rCV%B|1lRpv*od4`V62bI1^lqqi?4#YxA8RrwEtW zv$1!S*sil&$4*_Jao<4zXwBXTSwfpA_UAcM^SJ~z*j{F5N6WRDBylfYdkT3|2=t86 z7PP_}c<{f!52yrD59)&KTQA0E)oP*UqHUmKF}rv|W^Lk6rlqkhrGWC{oP=Ly8R+%8 z_pDvl%E6tmoo`TUKG1uhEmvcc^Ige6)FkiOwH4Wa-fWLCF(2)P#oB6)d$ME;rH`AI^21;6=OWFiM+biX^- zaBS5yZV2Y0R_9|8rrEL$tjOKnfS6u({z_&lB*m*EIya^!%tfz?B}a1K`+bKa1CC^9 z?hRZ2TK`<2McI3=TPwE?I5&a{y>CqNUsWXgMWN?<0Z|u01 zpOCriZVptybELxlwJ5b3P?5!6BwtgtJ>q*#Okl5Xx!K8LQ{UDXi1Z-8AnlT}eNYF_ zAi5+xZT(%_`E+A67c^v&hu>Lzr}Y_1WxUJQzZ{k1gi{)Q#lbuuBVLwJZUB!OC{FY6 z81q9O66VW_sf2wWRk+(l`CATX_wVFEc@lHXX_+{HKVMe8f$C-{<9Q&sH?O>1z2W^= zKJXVf&A9VU-^dGhwTXh~%)a6X=Gh%^EMgiKM#;bdZ6*HPg%8fQgH3lp;E80@^xoie zT0xFq8&vWi*(79Kco=q#-=~=H{F7B3K<^p37XxtwACsBzgQ$7fJnhr_cE7`_U5&2=l4*ajNtI(ojE|MGduR!>$4}H#eA0?+U(1pX z0L{nPSuuNFVpT|8I~)ks9nV?`C^N+iJup-qV&rQNyj-bZMk0G^+l}< zejgSs)G#k@(D)VoRYCt(P0M#7&T6}Yv+)^kJvDwpA20a=NRfN;Mb2A|G|cr|ihqg# z;<~9HjAHTg2DDLqG&ANrMu^%DeNUW5i^u$L0n81{xm6K5A2f=IDTxHgNQRcq%Pe5^ z>bx-Dv1=(JQd!Tp0pS0Y^0;1Jxcg#RgP~-3%j7TWvO=!Tg@N2a%rNegBltY0L5}ot zBo1eBCUvhRQ8Qd1gvgBdp2J?1D6<>F!U!7)Uxpx*yc2__3MJb2MS?JxneHW4<# zWm$C)4Lb-$*Yog`906oY2_R0wWON*7Mu%BXuXbYFZkkA#y}A3*cmBt?B0=w?B^@_n z(p5lMBI4{wRQ&Xm>tAc%VN4tbJ$G#kP11|yTq{<6Cdj~yH_mDZn4~gU;q!&FklDgd8F56}nrM5_U?JJBFCdZ;__DYnbk7af>F&p5sjd5GqE4&p535T6x=D>CXyWak2CU*7T?fUV z*W=XoT+&T=xELh_InbyIxdTa)$1m_(4^p4&n%(GZ0;5=6-s-FNBLNFXOYM z(~Q4=xH5b#Ego^C@Kb?+kYBIB11+`p_JCvf=If!N91l{SS>s^JUItGBO=f0TT{onZ zS#$U?`}-@}Mmri6=skR=X;!hE-a>k$be5LQH(-jf&;BpBmPrMZTsl0l2BdMI(rDFZSxge=N-0Uz+J57DKt34PAz8YT7>(C2=i5=D^0&`$3Fb8h7pNOLN-g_O%CG{%8 zgkpL{rP+dCv#yOgFI3n~k9rkMGTu`0t0o{`9&Gsif=s?)NMmf%_^_@l=8fZRr20NSXOY{8K zHQzT|I-Txf1U&nvfDxZ{RbSU)hBceiXyGv3U##E~X6Sk(#{EOr`8lL~yBV!(+}ZMm z9X-&g#8#mHnbJOb43e$r7I2kCMn7bC!dvSw@jekUkgK6aDZ0zAAlNf`0wUP&(7Ks2 zhR9El%MfSIJ!ZZ#rWNjZi0%*1ydWbIHch&GskDdId?=#o5#<|r7oi<)pbLBJJEZ@) zx#)qPnG)+OJwDKCqSKvW{jZo$sxlE*mON5z^AF83I|t7hnM3P$)#OnGq5`idBNW#6K0JI>3}>Ll8&B@Y}|wDD7Sw{#$VtZ4qg z_N7RFOLShMad>M^0!cqhoiKm5=SFjS&h$!Wo^|-k!usUf6}Jpxg??EFz#XhrjOI7? zfs?nB9nJ(zx_`W}#aAiyL;EX!okQc!y>_MyW;dbjolW%eK92-hrI5}9>C31 zMWQuYdTeRgyL}}|@_m@HPmXwhq3As}QDc^#AB zxC(Uvn|f>+b=w;5Ja8`TOU(#!$TBS9)OFQ#JHovtJ%7mvptsRtFSHC5lM{mpye{=heI=vJd|HL!|9^IyCdyJe2<%8vu7*3D1}Nt%Dh;2l+We)ItM&v?nXFlGuI(brz~`;9i| z7_gD{C*$%Ko*2=u{ilF@CvjLXeNfQPoJU_O;g%KQ^na?$ZmnexPg@3v;~gSS3;q48 z?fM=brKJO*Jho2MD`6uFx~}1&$1Ro9F%|D^|20n-E%TQBR_sNJYyOwrQjtL@!i21G zTrFhZ8ZGLyVo$y3ZP{YPSQzQsken-7#-NQ9A_`AP_8;C~)v9)C`&(Pv(&% z?^s(m*TMG4$ngVBc*iPqf6Q+Im}-Bvl;Q~)W|Nzgk3%&rPuvUW{VV<(dVd0+SN=jD zz5LPEW6#+s;=k^D)eTYwdot3v;M1XoYX;#PA%jQLICeouAW6E=bZb*?!v0HxTfVCd0MA`I2V9gT$40PaoAV~%M|AhZ zK=Wg<2=*PGL=v*U-3^hb_cg2`n7G+&#?@x^pT*M*nyO*%@5S8y}ehzM*cm?=q` z(BQQiZ*W`nfK)#on=bax!ZP!i<%+({5f@2&l zM;%{B*`|!Q-?twap_g5&Mgpt7j$#2O?}KPFO%Bg~ZHP5n^;*S|V1SmAuLgN62s%(f zu0#&%X0*K>V@e^!_Bpoeiwwa>p>_vDh0YdeYKBAW-o*Z*y%w7i^zH0hI!fNkyLk== zN{V7rv6iasBjPL)p8ZTr@?&Rp76+O>LsY*h6^IgZpLKB;}OnppyaxVluMSkHXYpn->XH$?RnTfWe+>jQIo zE)mwHxjiTYYp0kOfFC^$VE}n4TK!W%w9cZVW1x_#J8DRg0BTExZqxx)2*-U zO!e6Cw4%+~_{twBx8QJ=GKMns0)5r_^V-!1AFQV1*xr|ht8yG&MY6$GFMPVpHpUnl zAp$PH*JCmx0Nj?})8%;}*60)wwV@A^jX%yCeQb@C&KrK5`T&i5ph+F5Z&_@6yYE#Eu9kma-Yl}7HzHGjDA zQ`Q(3bCO@%g2!#DkLoJJH%{~K17jVhq?zRIAdn7af7bwPqhun0#SCka?p&)9Ay|1@ z$}Kk_EZ$s{WgcB&CvcS_pyV&>m@Je$7ihRqGJ??VLXYH85r8Y`$9CrAjtYb{Tz;^! zLzw;bR$%b@cpjDW4QJE>@><%Ir+?U+nk3s*S*R3*RPrG|5H8`I<~VfwSmVj6s=Q@^ z)CZAA)|zDv+6w`79)c04h|2YB@ct=n}_oin-NE!OhpYNw+*6$8N$J%azH#pC+1xV_?)kyq?TT9jmscYkbWK);6K^c&C^P-1V<|GJPT=#^t&?oF^1 z*2E(CiG4~O1OHUl^8e@R>Nt60=@AXbQ_jqn6Mc%5Y0HtyBLkNaqn-@d=Op-$9R(ke zAHP-If+iix8Fz1vCiF^DeFCX21l;qn&k*|0)xB9lTxe7XaH(YOQ^^%I-rT0~j1RnMJ8y zPo{iHU|`hg-@MUlyua=P`BkLu8ZPZ1lQ|TqmKai2|nq?;(*G-^q>o(-XW# zh=DZDQj>S4{Y=?ZikSxs>-D8q4a6|V9|$07#j8({{!+3e%ycv$e>|Y)vGx`RWDZTD z?n0BML+smsZ&%I4+U5tU$(>xsQCPsCzfNL_{zF;$Su4Y@@f)gny&gun;w0o z$T&yYMqCOs-QJi=mYUr_Titgn0Az>Qer^5g>&vqke@fvk0b-LU{L^%HKQ=8q3!6-3 zosTrmO2vTdzHLg}*`8TQ-e{2i#gS5@k@LI68zf(q>rt?z3b;B2!?JPA0jAi zeLf$C>Ic+XbxzQ+8Do9KYi!0cu$@|RAJSjDsr*5hy zB89Qr$toT!3W{UM%rqkE(8%YQ!wM3&+v7!xO^dVD?xC`$nY2v?&3HjOEBZ^!paW4s zqQAbCB`c%z4bdC$w<|0rW}!_-nb%_uUWnU9!|ct$>$xRAm`scNa<@fCJl_e_F9e3Y zk=1}{SZ&=W`l)jmY>ff$IJ4fwCUXtK4_Xz&hc-lkIz9ZMrX?lsMAV8hfGocfkYz?) zkB{_mQMy&+Cm27!)N|zMsO;?nv)%O9Z1d(v!u#mw5G;G(Jt?Y zTe<0F4$97@cnbU$zgE9&ptmah-7Zc4OAu*HNYt+XbD*Jt$n`)m*Cq3yH?{gBt-x3P zkxu;!5e?Kzd(c(dg6oU~g++y4M-gdXy+v%ugK};4mb>!AB;W=Y$_;$bCyv4fUDeFf z-m=M;BC8=GPR{mT{ z-i7Dt7(7!y7&Ot(g#`m+u7)Q?&A7jQx2RQTgDQJ?dAs#nB)z#hP)8($O9ufyw&6;s z&IUX(q+otdvrt_KX`DY-tYse6*N=q62u5dLKGZ;MjJ)zZc=A8>4zcps&X_DpfjB4~ zRDu29Nw9!=Tp~#409f1PI$VVAw9|Q%8fl)THU@k28{w=+^mbeJzkkkN8#!znwk9c8 z?~Rlx(N#AsNLL9kcD_oJ&ao~GJu2}Vi*%qH6Yf%V7e-*)vMt#$oSYAe zx2fIrAUKVw@76K8$K`i_8A_sh&1nw;7|=BCqZPBDnqOmz4ze#mkq=Fb-v2JLT?*pg zUh4ZC9?uxvq&ufz;AyuanR>sW$o7k;z^ucMFhI}*B+a}_c4UJJ!-h)TT0&e=>L1hR zns&u6E!HLCxyphF>@&BY=L~n!?g%fp1=GY(?f1PlaF6~1PrNx=x>?6~b1z(F^jhm1 z^_qsC(_Z4&^)`>cKuR2C@wde1w+eI?K1O(H*yV6RlcnBtM?*o!W1pnk@;3yO$cMTnfS0MHv0m}}VY3#W1kq;Z}_(60?R0kl^^AMPTV+lyakEO%KvSKw>KHW!)HQ%DPNBSZrlo z$Adv5o#0&qjQF|*h+V)OC1k=|j|NYl#Mis3e5Lz+da`*6sRpKBU2d;G<^*jjSyYwZ z7L=Eg>Fa#57R2t{wg$|>y=%bBBQ!l-rVfTH1Q+8LqZu#S9iZ(-bqd2=h_RaLNynR~ zCyr#XL*|+psuBvAdVW!uv?nT0TdGQwRZ2L)$e(PFQuz3Uc`lEZoo+{X$6)jQ z#3(r+M~MK~@5ct)p%4+kfOyot24qv1t7T|Py?!=BFV#?NvfDCoI+ab=-Nr0j!-Gfa zk2>qS+Af+@)UMlGQIZXza>hT^0rUK7$0mkCpyzS6AU*eOQH@EEYW4+)511hH*qfDP=dRRipI^OIz6bVo%oy%S)^DDGa!L*VRB#sQzaeB2lA2tx z*5sIWK4-i+E28e}eG9uhv(~#qh*I!s`@D)b;2rv1HXBK8|5{VXwe?x}xX-d^B99$3 zp1kw~3CwIu+)PHv;$A0%%5t6$)3y1FlE^M3V1~WC5llOKT9PnoJp>r~cXi zt9M0g0{-sS;`0|B)eoq_CLQ*Sg7hzdDNTEEiG;?Kv+#)OcVICLe4Z4h&l~2dHU-n` zY5lB#X+euhGEy?=1jrwm?`44Htp5xV6T=i_q#@)U40v%wGJrdjPd=uEmy!}wf$)U@ zQz90&n`a`hh?N3#RneDCJrsufC_pqsm=P&Sk~McAR^DOp*WbR)yT_ZpzI%XgH$(NI z;5&cXF35H;wCj)S76Qmd%pu*B08t4LS3Bif6aie!aOQAbJ=6lb)m7luAS3TG!5@D{ zE}70XKaHXxc)mX?HU0U`cO|-rTk7~IZLHsiLf zi98q_tpZ;5GYVu{>YZhuee#gv4r3TZ)Opa#YZ@ZdPB%8yVrxj>uen-v?d{1) zHR9Na_c5A0q6dSQjK~`J`@m`?wAh%&ZKSNBQc}Jx&;^Qk&qXdR9#kA3L>fvlZM{Wk zhJkjnS1v@v>mHB_Pzu|sMo0Z&T-LOCHF406rF}q9!`B917SZqwz6)`FX!TNc$wHvz z5!o+;BI9Oua}gVqb*FYYfaC zFH!aP*Osq655G5OuwR%mH9We{KaZdJ2(_c$csRk!4MD}6XVJ9>xnW2re?hmTB+tb>)2CH}6E9%sD=pvvmaHT9T-4zZ%1XE7PE?fh^Grj@xqjCqiaOi3Dw zqGU-k0i71f7Mt+>>R!k}UnCP`TQq()vN1fEV9rTQ6jioVO@CQYTNZUb#?<*{FFO$S zcdv;RywRH@q8pLTvV40D@r!AzF_0KkpDdHCnVq>BBws}3k zsO#4M93Z{@t;Z}w6T_6IP}TrPEVks^Tcko;?gEPWjXa=B%P?8Pv(vHG%|PeHv}hyk z5zJQ2OSvf}BR59=5o#Rg@_)Z;Opj^3-t%IjU7;(oVDcz&f)ZclN8URHjO=;&cWM_N6!9!hQ6f%1 zZT3EbWxvfNzB&P3dGJK9(adkIWK{#(=~%yghg)ua87jOzDdZgYp{Cr4GtM?PVnn6mguq(EBtK1tNPb|bWd z6AYSE9|CW+l1MP9o#1>kwB8JqjWA^vvMf;OgesKu)M|j2De14YU0R~V-bHV5F$>cv z?gYmLbMXbJY9|$6*{1|&Z<|NE0ojkLqa~pjJ78_EE$3+xVEF<4oG-?O`%^yp3Kz!j z*kEFg$+D^ID=DrBmZ|ETG+0N8lu};xA;hRYM*;vOI^5qGJlmc;50|7L^m24G)@`%o zv#;H*!SrGi`jn5?9REsj6EB$#&A~vY0g3Jeu_nZ53(dw%$B^w>_C?A3=_OGt@r@hT zP0%wj=b3Lh^cH(H&U9yI1FMadOCFCbG+I7}8B|Q@Tkd`iW?;O4;KB{xsgOzf+X#qe z3dCfu0(Y_HRdiMB`YnV(b%$1=2KPhN#|=&!-!K+md|fL^M|!Wiv-}o& z=6ox}(QSGQW@GW-qt%?prwD_usW?^D;6=)dKsx^lX}=!x60h!+xa4%0q})AA9Fo4_ zMz+XtbZ8bmiyOYzJ0yYv%7OB4e`4HY?R|R58NZ*GFeODWuQyaNl6~n$Kr&sZ;zCk! z6kCi6clgBf-FoY7<%#<+g56{w(70$MP@uMA4hhKj{mPc}U&h3O%>yH@KNN?coKsTT zB`aXT$m>_^rk2&+dEun9jwjd8aeBUhIvie=FzITV3U`FzMk#~NRj`xTH>;tbU@0Ul z*xu1`@RLjRR9hY<&vIk`o|{clxozDctn%&h4r#98Yjc&%yiSs&b#EQ&*-*kBV1JXV zpVJh~N`MM!J7IHCZEu&MNWdI7@X04fci%Qw_W0ZXv}CdxbWj@drYugt19i9V|JpZm zIy6G{$gvN76Vuv8+r@O*u^RHC5N>gD5M`O@NV_o27=FXooWFwU_7q zTlHSzyWdu#3qNHSIrubS%O4nSs6rA|i*G34RQ|@E3v#^2sNi7ltg_QU#)n)NhPkLj zDgTIJnlpV6&L9fz)tRWg+i4MPz@2~8r%S#q1QWgT6#ewF&xyzrao&LnkhW$(ZY6583B zwRR-;bv>&AjF>TE+dOj%kbd^pfHE<=rYQ02WzuiqD2uD-_9*D=P?>&Ev@t4()qkXuSWtgJy0PwyODre!#8@~-$Bw&l%{ z!J0!fafOao>cWxXS^BBspfUFX;RZ2{(rVow71%RLS!0!ifgAG|dGC+q&gzI>o0v2; zqc+&{tcy@OQLGx&FH;1pAszobZT;mR@@8=qBXCK+EvNAlVhJB&s;USucj z4dJ1h;A&s-$WEBxnjlEwYEE z=zg$h*}^Z=(g_j?GS#O@)npzHI(<$Hm7x#*;;t`4D}mRCp>t>x`o%B%R!@xQWm$Nw zkp@~DTgSuYcGo@)dsDGZ1yvf^mWJ+^cDPTw-xso%;?J&06A%Wm*qFV5K29fNt8t;IB!6wK_FV*qz@P;&fyXxg>GU52;&95&E6aL;H_p5>CzQTrYNf7T~dF>F%B`+$H zq^`9@pn9L(54wvI`oGxcS?9&)6WyLPN1_H4hrf90;wE2Q>Lf4jpJEGfwP2w$nfCfz zc~7<)3|J{$>q7TOLa{Tl!C|F@tsogs`=ij=x`y$)L1l2wzW-LhCPqVXX~aF$6`6s> zVdi!{@S5zNzT?&x%CC(|6}2YJo7Z$u;o0O8+C+Iyq1HAzgU+esICeG zz4>)bZ*Nxn1AJch4Sf2RRxX|A;~NS#4+Em31}|;JwJ7r4v-aWNnw1M+x0>wk!ni$v zWutOV!DE$e_ukF zbgx4rFK8s}OWheX@Rx-`^+Nd1Z!wrwAb1Dd^Yc`>U4pgPoVuM+Jjo&Y^PBBaxTX?- zhhmHqcDRApgbtjL*2|*b)A%ZLu<+d(nmIxIp~kuG>$O;CqFvIK(#zK{G+s%A@m3jm<7<;s%CZ%Vk>>g$5VPGBnTo#Q7B9S# ziD_^)UWuq|xCFV@DfYGj=%AS(iYH*3JqBrPvz-SufBxNy^+~h7rIR!ESf$*7szT;7HY`LPOd7`R3DWc+o>!GC&Akx<(r~I#Vi3Q7GI@47WcIe<3}EJKN&EaRsYwJpJ*cqzQne}E;(OWy~<2m~xp{}#HBO$=e zk;KR?jn+^;lWecMe_9&oNo=Cd+mZhTLQ~Zj{KF=7ezwu-L9_Keji@zzO|bAY`$Xq{#tE*@t%aSXODq);7^m0 zUpHtC_Pa!-<0ObF(d~yIs-dh;q)Y7(cED2$J*Sv7jweT<+=u#OGVRjbnhtdZ&-|En2A&kILq3bxNP8Fxs)`?xjF8IB+N*iGkPC#RmABt zP|}uIr1zdyUHzo(w!j?ZWg_i)^cr@-sc_RQ__?2|_td)?j3cR)*L%RYH}Qh6hr#KX zM;mvk$rtl-;@%;s2+g#p4d2A7Q~4SMgDcVNa;PgKzc^(*!tk{ThhH7gSxtHV>J*k=UY`T~u!rRLS7vRhOCq z^9AoMUZ9LU{8`^)*Z+kvvFbWrVDSFx?W@Y3&0@)g2-~gq*o(kGvGhS~nCSXE zakxQGuSiq=%grH0-N#BAzhA?#33&UD<0g(6f4pe|*Z{s?i7X=H@XM zSVShMWlx^>rAgLL@JzqJ9MBN(y=D3pYRbZEsfNp?+KSfejIHYdgtMy03#^ySA^D4q z%xS*4`jN85_B((qgv49BYL8~|R(o#}_dV{7$Lp0cvHy;&%%VoyT*1#x7zvU;ZiH2{+&~#wU50I!ny%zOVWUs-C-vv|JbN^ ztf&Kt94$v+okDe&Tbfc$R~IeJH}jHgtek*RrX5!z{BZX2(AT+PL* zO=;$o9u6@h#)cFnI(-Z)9*&TFF8ABv{Qd=3%Fn`Z?dZW*gT-BwMk97pFichiTTm zPIzZs{d6>8@w`EcyO&@NVYR`p%C44cztlJn2dVnHP~^&A=|1y~)~$yU3-c$e1D6nUU5|9>g3x8?Gh3d zccuam8x(x!?m%%RE;}2rql)=EZ8ppQVifpjgxqNmfIM2rr;Yl}GyB_E;_E>k_wSvS z=EeAv2Kz~c^|fb`FY*;pB#g-4DYJQD7AlWGkSnxF z9mbA=uw99-m;{F>vGqhi#AAT`j%|fs_L`tYP-kD(8w@|kAv{iJ zB&|R??DB`+`_Y#!pi!xPrURQ@lmY=;I0S!V$|q3!k(tX3ta~G$*Q115d|RRgc$bZv zGj99BRz@J`dB`6L!=96koT*KC99I+$lQr8_$J}XyKQE>j#S=u+`LoORG>X5MaTGtpV$>vfo$6(WND{;tJl#0S~l-@H5F9Q z1AtSX@4uQ2QRFh%3hO~3AvV`@@Wz?LFkk{j{jG~C-zPQ zA*=^w1JwRzsJA{FVp8IX){7l}QJXqPxXiD9EvgnFMLRe$sdJ`r%a5|-`6Xot-74{l zvSWQ0gc&e>+Z^oV$2#W<1^R;GW}sfz4$6{-pi}SYTuj9MM#PCt9Ojg*O)b!asEe+9 z59h&T7*2lm2GZE6$su;l>9y_OW4@^WgE3!HNDZPt`ZB zpc|rqbIcsv097XH9OLiw1F+#7H&e%Ns@d5@D}59jF+O=l><(Nzg%h}Hywx?S*8wgo z^4y6m7iqdi05);OOPxQuLF@K77Fl4`B&HSMue5l>pXIp;d77c>L&oskq|M;r{#+N_ zC*I0Gc8!3u;jn`@Zh)vHm3lM;@Oz!I`(VG$^H)tJs&3oeiqZW_oOFN0_wP|)*?Ly1 z)m40uUOv*lv#JIS*_LQr7^CcDmH{FZpUAsnh$#N51L8s{7!FH%05bVgPFIzL*tTA6 z3B=`gthw1b7~#B@piL&#b43yABsR7&vK4)SO3d?O_yaKhHIsk=4aGdtF>?Ntgkj$v zkWMb%(|J=b8pygbN~+1OIUCA6rJ{n^^EEseteSNBk;)nxKMIWGlW#Hsy9oHI>89B)~KfYfM|ce0GPJ!r5i$s&Pz z#=F(5b}whxMVTyWNPlZE0b45|p=t zK$h~@LryM%y}pmhz15At3;4p8>@XVB7|14@`oMtfaGCwhxk~3qZHFw?JXZX{A6{*blzOr}8njaU>c_~$ z3zZ;ZLT!Pt~8MWjd1vT54gzjFWyZjc4Vs=aOANZ&;WJ zfSvB^2XZdcj5fbsywOu+(bBAqIfmV^{cHE6N#Bw zh<3~yp$y~ajcMVd_Vx8vtaSV+(|GaC22;PS3m52kG|$?B#n`wa$BStADDW@BH!qy%t1vCPTBqm3xw{AoH4YPW>EfFe&%uH z18kNJnUud?qL#pvrR}zjPN!l%&zYf(IpO1cBmUaHa?^2jnb* zzbiiTw7o9ZeuDa5A$;Eqei#wp%f&9UUxBc`FxjklNM97)$%C^vDnGAQI=Ljx-M}d* zAbjPv*)0vk-B)`?)p6hYe6=RNY<#KX3AVd}3@%#ct;VL7Y*9 z#$T{Y4-h>K_5M}a1dl;p8mQs-}t!NYatgumXTLqDh9;%)SJ;x&(^f0-ZlUG|sZK^ls#l{Nn zt~d?p%QL)Jgrk)S%}7pU$~VfhX5T%G zu+P0D$IDKlPz-pt9!w-`1n~V2jDc9Eq8ERQc_w}3{C9f2BfVnIw!4p6NcxK@cT!i& zDB5z*)7P5fF&FG>Oiq6oEf`xONC;_WX@^WJpUHkw4ip=K@Q3KD;pf!EHAP8{0$D85 z>#sp-IQlBTE!B6c2Ii2fa+;9>-sDz>3!QuR2c_~ruj(i%d9_I}X@2*Kdd(b=7z>1a zW+d(k6kTWTTD=8gS1iegwgJWCS3sSZxxGXe+}4nwff!Z+bA|U*ER&3I#QTy`nKj)=sXdf zJkfz@=<>{v=?t|W4b`~Cc}jZuahs=HteAACci3Gz(WUn=B4)R~DedDytZO)X_+_G* zaEndD!HEke8ALa#@`C8|2R{EuU#Auc@zBxzFc4sTahj2gDP+G3V zt<|ko2=SaM6v6MWU2ngVKZmdZ$7CS2WK9c8Vu*4qg@ z`AOMa(U8^IaNPaDC@h%hMVDU~H&c48hi2a5GcU?o*PJ>rN@8Ety84-ug@N;hN>Pct<*M|vm_BSz z_35yHi{Vi8>EgcF>Q~f3%`|kSpv!QVs;YU?9($j{KZay!sT65r5hit~J}6z9zrd>@ zt@xUBpu;^qMbEt{sPp4+)ee3X>!W8vGOB&6s(CCRsF}Lh!j57{*?LcC{cbsyITI_X zxe~LI5(*O-7&7Xq=V6?`n)On#UCSKErMyDYPMr;a*IR*%T_`w#mwoh6iu}rWZdls( zH!!li=IgW)YBuRJ@hWk3+Wi)aMG!w+ zbZYjw%FY?m7u(EoL~*1DCb>dPSWOP8wdWw$ec&gm^en_sE@@fY1#BLtb$2v zsH%bA_F1Dir^7b6tEN80-8j_ojovzoGH07Kr1@RKR*bm>`F>4lv(Vhhv+LZX?B~A5 z)9s~={KjYrc5QK5EcF>#Pal8BP;(`Tj)WieYaX8fPQsxjI>vP^)Uzp!(ekM)I=WdA z9ju9%x316cVUh(BnPt6>6h=xRa7p$TYRT|d z9dDnB$_{&9%(@V0;M9{$B+qPalOI5Xf>{L%$;VvxJaOMBz>ffiojKo?#Vzxqz^_MK zMx$!=LYQ`~+iJ-$)RW{9H{|Etpp7;L`)2E%8iA}J!&a>kgIh3zJ^n-bJdEC`_2SM7 zE?*=sSJfs9q-P1dd8;ijNi(^F1E$*E`A#@}%z8#4Hgc~~49+d%Rryfsq28{HU1g5v zw_eZL)opS#mrEdK`!$cexz1sdp;z`;a^9_E= zf)zCh+IJ0wy3DMEej5cXbvP>R&k>;a_H#TVg82QIi{@YJs3lLxbDIeDWFUe%zUcPN zOQ;p9=1zuVkV`Tq(4~jp!FytFa_B2+7emPr;YB-x_OJ)wbjWX9;kx6x2lG0-e$p3uWdfNI6dW@4x$$AL=jVDQ{UM{&DUL&5McRE5 z4)o)KWD&g)c`-y}=z8!~nPt;<>mMU6`uR7CoH{%=UE|bScKs^x9HEK!b$4Z5S$M6u zWJBq8Ty7891^W6Zt`B{SE-GKPLw9w>%6*wY zLw>z?suKaLoBcHcl6}@KdBXO<3`~g31VE%nWe-#XMqlMW*k)em)BM2COYA~RG!Fuv z6j5`j4yU?Wt%&hYzs`Nq9~xp_j~sdJ-PN9PY-ir&k(W+EUOfIL*^N(^ID@@sSR8>t zhS@47x$ckHEt;<^$>{+P19<6t*?S`6wg_}B{(?wSJs6H+eh{)9ZrxYqexGCbxD~= z!Agx^LU?DJ`)ai0(!(ih^Cq>Wxs>1zEfde#*sC0z~wuCW0`NpnmyT*8QI?aG|FMkBV9zRw$ zCm|&uE(Tz=?{iObOQzbWoi5qf2xfvye{RebMWT0%&Fr#{t;>&RNhjm)n%|q~)e~VL z5Mr58%OB^TFv&4)i;jH5zZ}xJO@n_-td?~fZi;jzEwWa9oB6JQPBC(Y$l)UU?4y5` z0WTomAo`~p=*V(qSbEL!IHu`mdoxet=tC0{uqQ(m_${;7IP4l`oSy~v5xU9*=u@{C zoUo&1<2O$(liMprQ|1um+`04n*VSU8kSKPgcSbn_;ss4;{^Ktd#x-q8 zA0qUC-E>I|+tOwpsatrUD`mP12z^UKb8cAQWc{xED}q!*hiSJ$f&=LNo+}VVL{BQt z@%27b3hVyZib$qI2e3^H&0y}tEMx?E7Pe;{%W^$HGt(w|WQ%t0j1D@6Hj^nJ3~U)QD!R5Y6D&md^{n8XfUHEU%** z9B$MOu9eaM#elocaL}`4-K9XZG~NW+0T7eeKZahwq3!M|rzlWPJBZ2@)y^8*xS+b)f z;&zeQ!An*wLM?~jtIy7+2ei`$emeC56RHX6Eav82%{KA-ES{Q!OA3Eo5S8I`iM<;B z;^$cwX@F09xJ8xHn!{*%OK@aP>rJxClwR&Z1pbdla(jG~|EEmNU1()Sdr{s!RLgs8 zT(U>3m1k?2tvaEUp?5q+@;#*xZ%tpTajv)HZSmjDuflA z($kv854l9j21bEn391i=xrEfvGjxCLHw^*ciP&P^Trx4~L0Efw$QZPb9ECy294t8bJom$YovAw#Dxq z(VmkyqRv-(Qu9{A$im60#@nzFNyzgv0?Qu^93LvQO3?04OVA*fIKI=mEDf2YQhcl? zzv4i6v$*ySebvR6VRu?A;`NTJ(SVFi`;K7tYWr2AzeZZxNsO+;yYCzKKk767UI<6X zbR#Cl!h1SH+_(SkgF`pJ-1c2Rtx_s(Nt=aIZK#1?rT#iz$fa5;eTUy{+h#X~0u!75 z9F)KGCAv(chztI&5PF|b&U0#zRkW1andvVymqXdedBTzouC*SO;TFh|kp8Ou9!)-O zI?Q;~Z?~&a5T<%k=wn;u4al*!{HZ^nQQ&to;Hjq4zt)#C4od_Gwa{E&Ghr?K+e0?^ zgi+VjLy7+aZz4r2zSWQoqUF&K{_V$lvx;$)%?bbEl_5>(`v3I(E$&1!P1|}at-sF< z8(Ix-GsISv)CFjtC^LD*ds#p@N%|}O2K4dnCVE#4LqkK7RK{+)f!pEIXwS2#dsrh2 z?#?3~Vdt7+z+5vE$&KR51r;11*8k?(J+V#3%eVljm!ug)BWa~ z@Umj2j!rqb5vs?lMXpC06rSt1r3QD~(vTHD_W)~L#WihUw|}# zyKlI6+hw~!jVzCG=C3|eRP(U?xEJC97q%tBT%|EJQIZqs^I8rz+BAf@6r}3?-1<21 z+dqDI?zU?wq`sP!B$_f}X&mNb0}l8`ece&0S+1fEf3G-3kgy$+%SG>YYn;JN zv`}S}({c)xq*>r@ja(u{+h$czNTH8@RinSmev?)9(Q`bCdbE}5r~o%KC0G9D>{B)4m3#GclhghNrYDWR&{3P;kS*IKVn~@&HWR&9kHMwAbr6%SNl@ZJ{;L^~rbbTgw5zS#q+_%HIedT|jh_89~ z1|~5ZG5lV@DMh&Z?xjHCKd6Xr)s=?FA$&8<^8Wn$B6+u!Rg?u_nW`C{+@1m-`G0IA zs@!4VTBY!w#Qzg`Pou<%GBeH2>y$*5-GPy?qQf6pgN@elGf3H4xu195fv5Oy;;a|w zLBdO;)IyV8y8o*ROqF!oa|z$E6o0lSsPr1u&8c`nW59yZOxxIo`3 z9(t7uOIVzP+f&_^h0zr`522c0lAQmSOaZ;(gP)z0Ahid%6>f+UBsXz{@&QNw@y35W zta7Ir^J7=K%UOFzV1_U)lhvuF$AUXm}JD-{fHu5+}s*mv(eEk|_ebafn z@_z!AnDc^c!U{c%-z|fqH<)0v;C)LBw<>BwKBooR*=davtXCF=oN>Zl*{hZ{RTXyz$FQK*O{RNknj~Z z7Dau-nouW3sDgS!;;`M)eb?YGX%dz5Ui@IXcW8mvw*bagD9OINsjM9AKJeDdZJ}pA zp_a=v)N$n4GC(9_zk;J5-A_QW08)jnb=7+t)=>>>tgl_+(y74Xy6s{@NuhN@4L+O$ zCY+)t@n^)57O+4zul(jZs#Oq!&;r+oy>sE&`#;x=QeNp;shKz&6HMC_qNMRpA!nN$ zvNZ)(jWg_Mr9e$(&@;iBT_73k=o)1kdPG&AKj=Bxs6(tDv=K9eDkI}*BdA><$pB9? zl&aeQ1t{@dtE_sa$iCwf+c@q9^k$2%K;~)kWQb~~%)eLw#TtY^&PD~5Ldln&H-9nq zYAI4A`h1_|{)bgK8PeQHi`HS(gtL_~+ur|bYos4A6`Xu9VElYr=B!B?y||4D?BEJL zlS`UW{X??l)p%h_B9{^T8H(ne9D0ln4$H9@c0)HW0COS}Y_MUi!`esUl67aE+vAE| z*LH0jeIKl9Kxwq8Q0l6>Ap?$UfS^0?oAA}n>m^hMX_DVvTMK_d{fZwshmaonPW!b3 zA9I;nq4I_IbDM`C@^FR?)d?vmd?0(sYdvgh93r+Q#I`bG*U(4II=Y#S3Q^}_dqNm` z$zODE*PYMHVKmBodw0V7_b~8a>=)c>S8@4Mo&Zw&>2WFt2ebUwS-+-Q!w1TU)@3wN z)K6Mbp?NStwR>Oxet5+Gfo>GeYkB@FP^%|G&%}SYcz5xp+ekj->M}|-5+mz0(5%QT zlTF-?l}0F+N?W$*4)vwNlq>270jZna5R%+y$2wXV?)9oLxS`TBp@^95HnQxK@Qk<*io-w#7%_>JnrT`VKv!BN8IVrdoFnpAyWROd@8jm6- z*yvIJzUq)EX+H+zetC$qH4;|&FAVuNgl;Cr{{JI?Y>woP1_Tda z;@h^QVA%dQFyFidiHsjWDF>BYNc;~t?(Z)4|NnMT`Xp7a&pWzLb~b&UGugQ}2=Sax zi{!NveHD}E_{6bIU7O2IO?{NqGIW2#Ushjpn?`cs^V`?yd3QphDYV{aK3*j!g8k9S z!BrA2!bt_QLyN9!C8O6maf{mVjIPY3^nYauG4cB2D@#AC@2?tAcOk}p#iJLnmUNlf z8lVxWwh9stX@;WcLKDuQU$z0vBOM(-cDttMvK}KSh9-;VxdCRv&suRmtoXBqT#JE) zd1EW#rlIpkrq{fjTgSs44gzh+8Urm`;B8#MMHW;^t?aIZr>4QMq4N?&ZCxi!guqh!?u|3s%)!?PrVyHPQ@Y!0BP~D zzJCL)z>S^thJYcM_N=H!a~sU|XykiL>l%61D|4K2;~yd5NJGk%USz{xlRs{JZw{}; zKx{Uxb(XG79;6rDPyj~is;`!cI<(==?Whq8W9uLMtZtUd2_i3(2YO;_foo~jdeH06 zMHB=+YXZ(#hbAj39xZ9yS^Fbz@=(voJ!Ym$Ym>}59oHTGbhd;76Lh}P@kifM_fs>m zPxIQVn0Rmh^1PepnSVRE< zOSn_G%~Z@o{hXBr@cD4GH|2mojQ|l?Zb{-}bmCw8U4t z+jzhxyPE6H-$ZiX4bjS!rSxx^kYDKlSdsl7Z{!m^cXTGC=I>1hpfGD6$IDv$%ya8g zG^AWOvxR`LTE~9*q$^Mc@zpv^dLr;$Uq*NX{4{mdzdtQ|{@I_|`#B=A1$G#feH%srfy5W&*#5te&#Mjw`R_-QT+jAY ze@b?6(!Zv{nw&^zkN4c4>{>KF$7420YUD4M8k?fI3kajz>i?jgzEwjFk2fmV`S&9% zpd}sJO5q?&+#5w19{#~!v!;q|>3+6Bz8s5%9PCNnw0#D(R96|XCCjVTxlB)4onr#W;9$FdGC<(3@gZLxaBXr$-<5p{2F>0jI$vukwN3KT~mNtq5Cen z6YFl$ucSl$2Z3EgL11Hui~T1q;6?vC0ef2S)}7b{Pj1=QM1UFir2V=w1$h2B-_sQ9 zD+L~_e_X*D1b1s_jz#4_UFD&>NT1yps7G5LT4Gu;`Hp2;a?-)@P4qmBc>5sN+j@ysKg@p-(GM z5C16Ap?-DGB>ms?UReX(l1%FBk}#+&aD%;Ww8-wFcXi5F=m9zPCO#@G3>2-|K2wTrl{X5nOYTAo9js2zk7d#8%8jBY;oOXeC-W3|MAIi z{`THolS0O)X0nqE4cFP&4qI(z$y)9yF%`2)<1&NJ6Xv^x5{@rj=2lkLyavf*V@s|)N(phD zS-G#ati5wi>+s%+K1j-ZZN(ZeuCN*#C_nP;=K!bk%aB?*+9bWv=XubR2gd`_-93JC z3ZlXq;;necBWy>4sHW|fdzS25RrrST z2nn5*c6&uhT2q#=JZ8fYbwbHAKC6nw(Rw3}pM5`g=r=Ra88R4|Rm)*RDnHY$#ShMn zdFPp=F%s>T?gM+1vAxc3zY+~4VzF;DKG%(7@oZ8D6tac#_65J&-C56cTa-^lolS`2 zFfuG6>HO6;?3`5polaPU%kEyXIW#E_Cf4(F!urA9;9mWD#%@Orhm)99XQp=7?*<5} zdnp^MNHxuH7=EYfsH65rqH|be-@bKFm4FU=xamQh`||)?+7t5E>{b2EE9<6ljv7m9OV8PGV+~g{#l-5S!YdGP3CTPp!{ADI!J~^RrDNJ zmrSAqiN~Hr_Z3fG84KFWNmwGcWF%Oota_6|pqNrImsFSfT3mhE=k^KZQ#4U$W479o z=V~Y0!qFbOoppHt=E|#gN{$-+Q=`mzJa_k3gYMHqT@6U7XC^G(lD6U+H6d%DRRKxB z(sq$wEC(5yOfjBPba_&I?&wL;08^robn=qpk~|d}{14Z8s8ZXFOPBwsV$x7G<#v`s9?F&%}t6+;xu7e(guJNmhxwMG(<3o_&ATvLIg7aoq=Olg(OXK%4sd|Y(%P`(a2r;RnDU2R#>|Gf}ByA1486%-zw8s++QtJpwcnr z%dfYI<&%zJOT#X__;p!t;>gK(^KpAS!GUpu$9wVC=0=F(zEwlHWFY9^!rV=n#C>hug|Bp zcMr!9cZQ*L$zL=K^M2q@dVh$0eBtr0#MCR2ZN>r7(Kni^nHM#s+*18)ajgm#Gj-~_ z=PT0I>gixkL&v=T`3@|6mzr2ID&~wny7eTiA_Qo2Otu8RIXaV^X=wRfxvYK238fg_#fq&0cM+^2;CP#w>nm6F)$X~%;-ziu zD;C`qM)i?LvWNOMFJ!+km&a;<3|U|KYG}46LRF{yb17Y_zZk1!)4){{+7RWk{bGS` z@+k|fbeabwDr1OEPQb17xD6#ZdmwBY`;87BHu3M3b|XCNY|zZ5b=KDV+C`EveSAY^g53M+Zm0fzN%| zZ^r9FDon(^fmST+T7Q)68Lz+Ynu_Ch5TVCn44o|>oo8Fm*HcB4W%&DP`>!sQsv#=jz zoFtbLGQoDkpIt?CL6wFsy=TB&-$$%JX()!sJ?|OPkn7FF5#B%{RZM{uc(47l^k;ci z(|T@D^Ig)ung3|g=)-+ZZyVgPnO^o(fBLajuq|?R;pMYM=lO+hzJmp=))|VLReYH%<5;N>nq$a7+qd`Jyn%T$D1bl$kz2rgbDm5qKDML+6q2#z#LKu~hYV7~pX-Q}~w~ zFZgP+cDwJdY?mjEx6YL;$(s+ZFPytvQh!^Z`{;zK#J$LmYe34b`m;+pic8AK?nSc4mhQHZ zugMg-8%RH0R!+QXSDz>PoCa-+_FzHfgtobi&@nrfO$|!&@>go0ENUXfz&DC`5Sg_n z1Ho&s+2qP-pfc^U5_P{)WkR_9*;w53;EUYrd#o})+$I5Q)}XF(%)@YU68+}*id6&J zZd_2Zgu|f)SpjX6$UR+p;-{^(nhl|r*{xc<*obuX1i>JoN7Lz`KG$S+&)6DW6{zKW z(_a*)cS*7;B9$3@{+~_1#N8M4`qO*8O&}#iIo}7f+*Hr~<~<+6_`Xotr7`O`XcP@5 z66_SrBiC5+inyG&p*FYau|isKq*iH{yRVv8>@}^61Myc+IyOkKIcwxrwqukKb$>elHXoDSN*GcrS zs>JSO{mn`Z1GD6op!T69!~^JVB;X!Cqnlw?EDPWCOW<;j2%#GmBn7O7d|9$+h}w&ZBio3QU=}2C_T%*oohHRV~Pp z3pb7Q8Bao7YpDGIh2yt&vMEj3{e)U(J|Eycfsb^!soEc{pAr^0sNSQndejd}1Zm@j zs`mQxiq8EO$#ZMpt2Ih$OZ&5L*#=!m&|=7K`=P$&Hn&+Xy=5zSJl*w^}%^GB3WNFYL@2qe7plNH^E67t}a;ZflSP4TPQK zLe)5e;S!wT4h`yMQ(1IK5&G?6)$&PFobVCoghBfvg_2ThtECQUd*LzdxRWusikOOF zyPl=0l<98Z^1$-~+At?te;c!h8M%SU37dEs^pY=@?Uyo@`iE}rXZ^%#(){#fIV8-$;JEB zcJljmu;tEj-=l8gXmtWQ>{j)Y-c-MQC&!L87dX-{QnHtWTpOz+(jVPprC=T-y+tGv z@8f$P-H6^t^WJ`dxo5cSm_<$4xEw0Ug@3tHt$|sZZl~O33lo{G^L;tx;2nV)Na9XL zRONES+rSR8T)MyH$tk~p< zvJ0F2KKmu5@#gO9f(}^2mjPayfbR~osQ4AjDy$Nr|NeM^DJ5i5Bu}KA z8p6$bZ$@T`XepqVT_g;4W?KSf@TN_3Jd#?d(&wgk@1c$pOI>TWjJb9i;rOLj!Fe|E zVF7vDeGPuJWr%H-E&rdC(`g+g^f(1XR;F8srLBPEmIMD@Hus9W=~Of+5XRR2{bdHT zjok^4#?#qoQlez-pTpDMlP{+n=BukVHlFRfPb_}HGVPo8%DOI)ODk%3$%3}+T(=>d+%DZka?x|?$hiG(zyxN=MzSTp#kh8dsD#$7I~VtY?x6Q# z)qsxWW)4w~I=7trClBUArhfmrn?x?r!f`c&Xu2d$3n?kqS`3u6%>K zj=~uPVMJHCEn&Jz0cSURYgvWsL8Gc5B04n-@pj7^R&wt27TH;Od4rr z$<459ol36(rlk01&(@tc2Fa9`r0$~aE8i)~N_2k;u2NO)GRGyQ+Bt2BLcYcn82~b3 zVLOMA&CosK0_{(T*V#5hHlOnMY<{0s7%2=SLChs#>xH{~i~`G$(Z znSNAUXyB296p}(|zqCee1%|@hd&2!Fu)aEANMD}H46D6~U zGp%vd{&c2!f%5B3#ep2-h8E!M)KO>wQxm@rC*nK6r%s+~Glx1_`$n?>YGEtxmcsM~ z1t{$b60JQ)YF$-ru70Y>@-_$XFFi`YQX%2;C5=3*oeJ=>Vq5v*F`%k{>ygP~vgNJY z3rA5KfMdsqmOUa(1)EC~u4{s*ox0!m#8qS+aI6nTO|pVb6RRgy^Xq7=QIFr&&r%4i zdoa%7oz}G&(e+dHhRIJBJU-qcAG6!s`fUdB7~TMr5BtBwAnug3(yEwL#jP*2aKXxz z)b^y?PAQg(9i)%J@Tv1!7D8MU_}0r3kk%GtabHMToqi}~_Lbk}a!4G!0iJa+FNdG6)93rT=TGaj z#8o({AuUQ~VUn!2-_TqNQG12cvl(zO^-ZMQlDh)Oynh$=F5N z!8VFcdjDd2ZibR(7NP9@u{qZYzHw5KqxoPbP4bK&-SbcPl!9)_J2+#cep;4*5+Z&u z=zXZgZ1?HAH_6|dk+KBgBFt+^eN^869gHa7!JOoBX=3}U*hNZeQBQaaWRUZ3zoVo{ z;%OHsNgjVnu&?g-zV~lC*8>9k#LAHIwr0A^8N47&kDC<~S5NW28EMp5H%-F}+ zNhR605MxLP*_XlCw?Q&?W8aN!WEhP7{7%3B|9@VO>&ErCZk*?>?#Y<%_p`iTuh;Vh zkT>%x;sOwL>n5tp_gL-8rEWK8+0>XK$FteBnc{_F3KyQ=(53}}(W}OeZf5e$EDfat zON$Iv>Gx68Zt>X1R_d0*_Y%U#jqYl00&QlZn5!oMEMM>O3?*^=8;xCSe9QDAuOFqa zMR|I^hZ7=wTW$~**`ITO-&m%-FY-zOLVfIh%Gn3d8^7C6Z+cA`8` z)}9ke&fQ(+aksK$F;OU>R}+(C%y)N_>|VJBbyFYFSiJ0L?J2673|+Et8cgv|Y4)4$ z>D-j6t#-M}?Rs2wK}w6NE|ywf8XrhnQXI`nrW##k+T~w(MGAb$kz$^=9NS)lVKz zt^Zwpr90B)$utj|6UnrY0z5^#6XUc@{VWjhuw#0Eu3O^u2@>23j`_9?tXh;!AeUMv zOh>cFgwW8Fun;ZwSw!Z0^H{OuxS7#^^g`sLCeQEcqU2(l;%c=^g?QPoaa0MlMxw2o z!)`4A0czWewDQvNuL~waEQD?QF&lck;!VWZ>O}k*b6^p0VhWq~j+@ZFA>71n<9~pX zSBClyq~SnI(h~mbpPVWtuZ|zFdsde31A5DWj(DQIDY)eUEGQrIT|?*^Pt-T_PA1EuUH3Z zZ-AUYSg!-Zd5(Uti6Jkg69!v;j~~ZJy5lqDVy;)>@Ge951~Am?GEE0*lfXHCGG}#I z#&iefA<+vVhh91$GqFHSp>H7WR^l}!aV7kR)IiWjDL=0+J)NwKavS^bncCkc;&*k_SZbdahqv(N=7`BHNxDYNO#~gEOhlrVhh1X4 zIdJykpd|Q?Rg`A7i}NL$EVqw8SneB~*SFEZbU6p6vHIvI@~R%rW@K&ae&yVH(9Rcr z=R!bGurfXPFJ(ykyNB&PUlaMurDgAtP706(j^Q&9C+dIUx?g5-`JMI)-r!y1D>$yAe{6z>#gy|bX+k%FOW(}Zd z1q`l!eo8QP|0z$pW5g#dlFohw3$ON;d$p-mdUo&qwq zs6y!)q>#R!A6(m&Gb))BE5?7W%(~+uY!~DuMv6fix1y(+%U-Yl7 z9`PQtV$jq+E=O)2>e$jADexUc#%}g9GfY@;YGG%QhUDEv23aTZ9l52>^A?pc#zw%maR__6S@^)#oeVx9D>a2nJ=o#veX!#&d3f1?= zPXAeb`CWsGkceEv;so!`@5+^AKm)lTrJAd!HbzRc9z6Hq#O6Blsg3nsc6U&o0Z%`d zGK^LEp!xkeuv#~;0MKCEm{%@Zodmsu&IRCIe$%DOce4pkPV=Xea_2!ivE)wV(ofAA zZrcSy+E%FUYJ@=Mgtimw4Le;WOiqY7Uzjhl!WHBm?fUl1hxaUdJsEUy!d?b7h<{C< zWf}!?8QzGdlVE2r1w5`lB)(~X7cxgYTQ8h7DwgN~`(7{UqE;-|Bo%k~SWJ;RZ*8te zGqB(0s0 zbgtVV>>U)B78dscY1T_pRkpGd4QKkc-R$;>ox|g?=A?J+K~;X+XCj9H092@e#)yDA zg}t!e>=vD0cPCe0V(jsUG5SmMw67hJ|Lp6W(S5@?JGz#$>{m)$*9`k}{SXDMqWI=& zzT|Q2lQ&1PCZU~E=P~L7xDwb8>KTsm$>-YTrZ32Qz9P|$pUR+R2%fekl2Pcq^gK+z z^9!FTW}Rd+R&m*ui#vCKO70fm_uK6(gZZ9u8T7}NT3Y99(!ddMBU0>K~CqE|BG zHdN%1qY~3MtQxt{)~oT^%`xaK2OXD_nSDRkK?ftnZ(M*!R&V}>t0RSYVJy!jQozqO zzlRM}8pAqemr6YPv{4wK33+<1Unsv17+!v7>tJFS&)s%y)&OG6I~=3*_6FKAOB(!G zOE}Ir$@CU->XT&j0v*28i`*EQG|9!q4d!LCT9tlQrLvmgy+z|Nj6FC)INzF*)LEuy zN`U?}LnydTc*Lpm%ZPY5YM&7{Bm&21;~UP084;`^8KUB37xz5zO&nc?G zVy`~Z>@d^?$=c$=)A+!o#CO#R5;4W&?P|oh@jmuzQaq1TCqFMGLaW$0lzqkojX1=a za?TCFzY#ZD!_;EUJjO>kAWBib3xd!9q>;wfs^SBTCnKte%@yH!yHb+Wx>3_dBGDdW%cMU;g}Es zf8=B9A=|wK2`;WQ*1PQ!UomkGE-{;-4f5~^al8q+?fu)5eE@r36LD$2n>u8R_*Qp( zE8FqMCswE>Mfs;*^(*HntIKu9_@W=hAAk!Uv zd~$%zJF)ki?D*7N`zP3M4mBkfvW35{i z=5GO-7Wu7Rgnz{d%jY7$YeAGYBppW1j9OXKAHvcs;MoHecxj9T-XqZQINE%+NpJq^ zu(YaE-NxB{Gx=>u*1&YPLQ#$ABsAgv#`BJZC|iAy+S&W#S&~w467fR#eEpD}N!49B zmdzdMub?y`(yAj!@ft!m(#-CnLeCWel{1+w2`^XHOMe%D9r4q&7L#>rqQdx(mP= zGxcDTUM%jhVlBh(;|m{M4&c9`=C>r-2%c<&9Sr=5D_pj^VAe|*e;jV)o=L(M-}(%B z!*-kL@T&NAW^}9Y-)Ri8QFQM(_h%-a>)dBorizvE_^-b(HZOvvO*LOCzM|HfcZo81 z1^t}M#3R2*Uqr66lmx^7Rl;_N>~gG{yzIZ6n{#0Db9SJ>|Alg>DLVCqKwgxxSlLQu z;b2PTW>GtKI8H9>k{3T=yfsh-R5`G;oIbnA>4GnE70XFmwWVx!WJ%=8pY*%rD{aLD zHx1H`>q@g4YB-gbr?lzLfqVay{>}$3nl?ZdR@iECqihD}thRADc81@)`jz{@9{=IWQEZV zK8l6O<+5Z9IIcPm+$)UPd`oAFz9GUM}j7&#}(e#s7{HOnLZy zt+qP7VQm@3oN9N(VZc>T~|W*YmNA; zm(OK}wq0gbIp+4Kyd}B|PR}oAIO!*XPebcwx>zrrN3z46jD5V?SsWKy&|6}6N||}9 zE+ldFn>Ud~a81iyO&JEp$OBL1dC)-}sW-Ugf9us`j2a931+chn;0Jz3`Cl+*K<5>k zd;u1rd!?+@giPP!&BsNJd*7Z4?hNdvXY@QL6R6U?A@STP<(hbgz(n7u@0#0**FViE zHEvjw6in4!?a;JDr8(Sid4QhMv6BgMjQ|;5;9Pgo`7QY=pmFoL?)%m@o{1$DsrvL} zsYy+Kv?*L_8%by5%m2M@CxuVFB8enyZRds|1nRbWuriGxN7bZTzcZ9r@A}a_%q|2UVB8C~L8G<)aRS zL4-pziAk*7BIr(fKHS6PI_j!kQ+I;OBq1}U6y0;7*PN#MK>uxo=A2j!6QCArY|daM z;?YSOG%ii>4r5Z%E~8LMw^#julB{GT?=*Z@ZcXf!$v4u}#O^K}HQle)UOgR1K@=+v zL#tw0oSvk#u<}h$4I}W%mc!Pf!`f;G6q^MtcpagqbeJlc!DlcXM`>? zIz6l$aV5qQw0U)1yX+1fUiAlDkK?K3=s;dj6%G%h=CKMf3=x6QM%(A zVU3s2DETQW{maF9xhbQCW=-IoR!E-9*p`+1LMG?`nY~b-?~pNhJZd)!E5%K#x1J~N zA~QM}VjAvK_7EpKt>CJ6;D%o-AJP!Fkf)m0iH0O1^68tEoKNCt@FL7J6$f@wrp#=* z@tXKukWf%+T0SI}nm!pnmn=@DO*MA~V!CqP<-7Y!7XQA{%S!Px7-)Jm`f$0dAp@+r zGJvA~&C{e@QqkaXNF`pCBaZf!r7X&ytpB?o1IJ5X7sZ**^^H!~3tZeW8l1e3DI@;! zI-bQS3Qz1~QUQxBE7sv*DmyW_j=O1E<~ruU<+W^HASYs$EGf} zS~}1?DG`D*6Xz7Q9ICgz^vc!eMf@5r(Xr)`T>t9|Oa`9?OWWQu@1Ql8js-|QzO&x= z1wsX+O21mSfW1R`#`raN&kf)<(b)F2E7%?XNo?U^E^R;dmg4 zTOEz~yTP8fe|Qs-IXzAmBjv&--zZwWN>;b~@43fUyqCP# zYA}jZ8>MRTtC=o1{oVkkv2xrXSJm{f0=(I5n-pfTbEt;A#~}1MSIX+4sju8d*RoHdFAt<--e^`l$mTalUVe{OqAi5Z3pvj2 zZ&%h2ru*iFHq))aU*i7#xkvLt9SYJXTMT8Th7}FMzNQ#ol_6{Pv_p0FIP*qOr7eW)F8Ne8Y1jv_Myz@ zMH3&ZvajCEe!b%I{MgTeetooj0OqV)eQ8AuF7x1s%HZo~X!(cJj-(Q?P2%X(EM@O2V<`%iCQN4u_)$no7acKX>)KCxULStmCI z1hVZ?W;M>6z0OPw^WBj@xE;M9yrf6!O)Jlx>lKfv^EUa3T-Pe<$_`I-Q>>X~2;nzb zvD<$K2;7vLg`hMft zzFIoB$2e=9_zmLD35?!<3qHl`5hr;7vf?Zah^Q<@vZFe^k~&8P=rm<1+p=!wSrgp+ zm3vI$wh5_czMefqPtYpMNfG^IsqwMXR$y?d2AeH7)lF|wJDO!3hvuJ14L5VE5%dK< zvL3rQQadkVW(m$N+={+6^V7FkPO@;m?tq=c8CX0K3=5k%>zVJT-`Unb=W396;lI_S zmh^EAXcZgSMK>Z2KdSBNEzGX5W@R6~3#FOsMmg7$dHQIZV`p-xZUkImeB|ec5dGw+u z*nCI!bFX^Q&zug%wUmyKmnGi$1uEyAFSc9^Oyd^>O9pjTz8QKFoDVMuk63OuyRG=T zCIPtSLSOU0V*qXji+`3fziV{FOjcxHnAzae9njoI!LjBUTR@(C%T=z?W1hb+J(_yD zbe@_9`##=`P0Hs?OS7ToIQ(FZ1X}HGOKWacFrVfT2$~UV}70V%? z+?jgoQQ%;jPo}aJU-hv_gU~U>GOpcQ_Wg&)T$ZWK17JW_HWc~*bvL8d9qrTh;=2Qq zxDqb2w%Mj7bC-Rf#%2W0c+iEt&2euDHmI=GF{lfodF5K} z7P>?rl1lPb4(^+`_z!FY-bwiQ(^iz4qgG{|BFmdH$9h)t%gXZ*ShK0oXRuwVUsy z96R}aa-rPC2_*#=`~8nq7~W__1UR-95mr1+E1C?W@Jna#dGeJi{`j|!o@0-^JP&1pB<^776m6xq&md}N7@|w$BJr>jyc^zT?$02mgK}Y-^ z<+cr|T;}8Vw$9WYWrbp1k zPX!y@t2Bh@oiu@Gm5)0FLT6uDZ7o;l(ePI-O!HB!k6#Xv+Cp;Fj#pqkGy*4qGQFL) zcM*8yQ;dPjbZreNu}w-@ex%`KXWVJe>(R6XDC{6(J`(*N#JU>xiZyx7!j&B=*DC0F z`6G-ZqdU z`FHpg%+#nipwTIUXex&G50|V{^j3V%MZosqFve=-!-T7fYyBhSoL=7P&{B0pw#%mV zAv{PIu5HI^s@2W=f~<8B%L-9K_hNM%Ky|>cUrPef5WIs^GsEXA9rJhtj>s^rQON}w zR`ey2R)RJQ6y=@%nX{ur5wWBtWmc3L%{5`HOq-9hB4&ldC5=tOX|geF{$)xLvQ1ph z+@yoRCtsuxa2#0d)$BVH!{Ek{EQp(KyR0nj3ruFFVxJM`Xk-1Iqx(k_#v`2CLcJT5 zXU;w7E*!H8aKf4m;jBfa+UA)7)=05@Px1ZdzLpP8Urw5yhNrdDW3=fr7LiRfU*bCU zLJ~4CbOF$%M>b2M{2Y%z+ruaQfrdl}%y-=mkyV;W1OSw(_KA3gMTH)2wy+8SKuXUF zrj#_)S?y%#?bvTL)dJ>HD(DGN9KTA)NK$T{QKfa?Tr)M2X=hbU9wa?VA{-6AXXwU9 z8XAKBm3kGYP%XX*>^iv;w+r7%Jl_NOu;cd~=xTq#-~;SWv2WRRna60AS+kQsJ6gIa zrx0^8DhjAtR)S&LSt7?fV>K`aQXaXFFQEZJ3%d1*IGoS0LaXZtrp(nWKWWGWQzcAYSzrbJ%8cDi>J>X z6L4-9%k}v7zpIf1PJ zAi(e0G;b2!E8NK;b-MYj=t!XwS|5AUYWnKuK|YVcXRsq~J%9OO!{RXESFS;VUSH=! zQX;gu_~55CXJXw|W|<&%JM-EXP+?s?&pP7-*bN%qZF!Gi&9Lv+ef!Wyo3?13-J-sQ4#3GZN!#oH@^SJ0Q3?zSU8N!6|n$l@3xq8y#U16 z18Myh!cSM?<}?D>eCl9f0@JD6j*8|*ek2DBSG-Wh!BADdB(X+FKb!&$R6A3+sFO0U zpB(?n%3eoQ25C)AQ)*#mQK{_>u&=tC<1Skezg z>634(NbP>c)MhlR>i!bQF7(5oGkukG|l-A zmL0xNO}WDyX#0owhv)OSEEap|>Lb>iLLb#4>UIra3QQ zg|T)z%`#;0&Px$HkMW>qiSO?Fbi0jdiTIt#>E`ElK|O|d%;+nZaMkjemO$pdp@myl zSUrq7Cc7AObEM>#s0nSiar&xvWR5OUln_;eIZpHyzZRNaTHaHw|5(DnBXn{FC*T@e z#|&(0j!E!~KElHB716NS+y zp0ECCl2Ii+kNXLGsT|z%fOx&T{xnSf!>GIsC@N0{kx42Q!xnEPy^|JyR_d^kia86J zrec&urFUw-E{s%DzH6!E35T<*&j+G}Cc3=dECO0TPwG!g_356Rum+lI111WcTb_^& zup2>bxaT0!`aL64e5gcv9hsG*Un0@c~HX1 z2do;CbM`TNbnuL-f;qWZlth%NVfwANoPW!a!U-r@qj52JWnAkU@)?BGEfdZwakb-y z2)S-1PL1s|Sj5d+>7346B4)e$maj?)qfEs~`&vw4xA+q(R#pw}XJUniE)W ztT^2&h22*Zi$7gbWF>4Tpjw0LlvXM~{p8}efRs9Vwlv!YSqqYj487!i*&US2;o>@f zub&%bpikT1>eCF3;4+P`z_TRqea8o!Y)dR1^cqHAj)|c>mUR8+dSv@M=`+g1o^bRy zpX}6f@L4vVoh_(txIpK5w2-XXU(VgORxOdmrlO&BC|GC2!(QePe$Qy3ap}0t@zI2d z-n;%mTxvk6Bkqn7=1GK=(n&>&;?j){iC5R(lrk^zR?NR1N%}E<3@7VQe-!j+`9+&P z!uqz?d83>pzm_`WMX0OsyLS9Tvec%}sT*6dIvS(vet)4M+_pw6jLRG)8>_?RzjRUC zwI{P9ACD|ymLhuu2?t$KmQnQFhe|~+2l=_2eD};OspL+Ma9odX0D=YXUz%nfx?DNK zD+Rc4LV+DLu(NAnW%rXf5??UtQd|uEel&~4TV4Pad85UU>;*Zy$4&ql7Dvp&9hxMQBILv zi975T+98woOQHvjDS=74_m*VCzB3Rcl(8ad38fF(ME_Q-`n(T&w!Y#{o#FWIZAYP3 zaS}9@?icYMV{RSautU5geO|S~;APEPAMj3Qmow)5JNluue7*7&$Vxrdw)Sz7&>h+{ zGX9viy&g%{!*=!#;JW5)c(ttwb46r{>h0$t(yMY=rV_+ht_MBd!(}WXk@z5;yw(Sn zBIGQmL_GW}amu7zVcO&)@>!6%duy_vHu|B7tDUDYXw(}xcTcgyRF#V;SqFp>2q3AY z`437STcU>LTqXY^4OjQ}?=4hxR?lbNN|v8?XRi8h7676gFJW=8s?X&`tgHzXimDBJ zP}8G=iG$IPYI{cNq|=$s-#QNdER*WGobld_e3O4V6&>@6^@X3z4pk^L&iX5te-Y2g z<+_|jU`P|8ifFucA8ZKzS##-~4vtvNvK`~T^6sW=R!Y}O+{nH4Z}l@K)p_!FYQ(VD zJn4t>?T_@+Gf}B&btOn7&<5mdI5Wc5>vYgsAd3%C$Y95sKbFYSvpUz0H$#->+mJhu zk9<(UbVRJnjpP#L8QU8y+@e0AXjAF1Qq(o=w9bhuHLcy%4&gB|*S)AFld+)AuWs>? zZQVaFaQ+jPMoeC}k1kW5(V1?3#nt$=hm|>N@_y(0cYQ3e`}b1@1)r6DGy zJ{z7For*PK&h(zBAGX2rhD3zs3TdtLhUFtm6BS1{o<6C4&eeUIyKiD{!y+&GlCPHL zr&jS=3sfCcWeRyX^pAmV)-0mkVbM%jUNJ`*2NNxDz zd5DZK=hzuafpm^QxRo-dhx>qCT)7AFWi;k0JxLCzAwk^ip6SS|Qe0+yBk717a zS}Bp-=19x10LeM(4VE5&E__W5 z61Bi4QyXO!SKZo3QV5&7934hV!}EV+RS8IdAF;_|Z?ZqxX;o8w?0oA7yMcC6lFo$= zZ7LzUadh83W0qS68-Pgf?L6ZOm-aq?c}B4^`my?^uqUTJMoJ=G4|W4HWcZPdZ%5l~ zQpV$6G@_JANImunHUhCI^lD5pXeRM)H}&&Ei4mR~pLSNd`~k1UII>&uN{x_&Mc-a6 z)PG*e-T&GH_Q&|#Era7B|AT3kYRDPW(>G3mQbR+8oWhBr7Z z9yFY%N_L({9-Q4ZWGv<|{D&xuK6Hup&)=#r+1Hm}=?p5+)dX`5>&oi=LWU8OfrKPzkb*8R%?kx}8?q~1;pwFrr>8pcM z+h%{hCO-i>qM$SUteg6oU!olL5(8n9fCS4ZwSCE0(l4R^hVGnA;%?RJYQI{Q1?c}c)2wR94dA?X?!_{X124#^043A#tLX_h4ms+RK7kA)oND$V5yRC{k!5*tq!VuATErBprR5;J^ zznZt}S<{0xCGX&_*VKhul`c#SQ!JxQ*Mx$GUb=VYtEtN)vRD7%YPi95%6s_2C!?*1 zU@b_pk)?DfgSpYF#=iuX8E7x}h&5m8*+zb3m?h7(>Fq&QjCk zpZF9im+VTIMr2Zq&J7xPRfNmyYbC@uQd>p%ZfD7BgcWQqf5-+gP*=~2ya`L1WM%qb zy9;vuJ)8s$8GUAnP7`AM|6N=eWhg`A8auu zspEdzdUCi3we8ttEv4l#I*{ZXLBh)wu2I9V@)>J8xZa^tE9{fT-~sIgmsqcMkZqkN zRL_~QY+XHb$julkR{IPPN16)#E%2}w!UKp&+vIo=yB;QcQs$dr_VJel!+|0z!9@z6 z>gbHIW|;4C3^U}2#4^@-b_`sJLDxsBlrCg1&t`!Y1h!^@T;Y9Y5rac9igCc|jr>#> zPVY#vcON3cgPPo69L`9z%QE2oco^@=VU^cl(C0XsUs>E%7`%<_-8#C zH4?_gz!y_4^Vxp=!Ewhs;fZ3^!QTRyPkJE-5|*)?+iB2tcvMZ@iq{>iAA9&2wQb*L zzBc09E{M0ExM!w7&scyzbR+-tmbJQz!Gr#&f^-;zfUV1pYpyl(KBd!aGPgWdYMUVHK-fj8F zxi2nPP=t%%b+?Pb_MTvQ8DI!?bU6}y8)e#%naI)B%6UNo$81KkZ-+KceN;y&ICwb$ zO>~z$pz*&viZPdtvy*J2j@Cc$3E^w$w6%PNQ&`QqoFj>w@wybE`z=>}kA&$5*`!Fg*_0sZ2BeIPqFPqf>YVN;Qe+Y=(g%EEiW1o)eI1@LI23Fi!dt zOCHKtHPmNo8_@+s$Mob{fDV;NGm&#nmqu4H@6mCh_l!3G6RHLmuywp8{kqRubx69~ z`JUJ&bPaK|l@efGmXu>sa`hM4fSd!!hjY2{UGoM3-p71Q+lybHZj8^T{QoU~V7mKOlaWeVuiS6w6%h*2K zINiusPf5{C0#b|1yIWKwxq7TQ4C3s*SEcl6_GUz!c3U14rl+k({MViG;miwOI=Am$2K_UPDAS_@ae1|?>%>P#yWF<}Y}meU z7rI7@+YCr|)bfDtqP09TD*`78k&#Nzr@D>*HoCg`_%M@syQJ&A!AQ)n@{kB>O3ibr zl(YvFsDoKJnN`BSn|ndjwBx%dFk@&qxM-zkT+j089OcrZp3HC?-Pb>IQcqHNxG0-n zb9ng>dOgS=^EY-h)lOO)%nbN)r3i-lY1fF$s}X)Up94l1w{Y)(14B5`=$!%s4mng1 z@sjgVO$iroh}-jGMiULXhQuq^j{iF>S^TgO^k(B6;x1uoN&kk7Da260nvI} z@w=Pr;=H09-bRvTYQ?w_XY90UDd6ZK@zLDKC)*c1zN*At7cs_hnrB-6L;j>%BlS34 zDrFd3dwm%Rs9paeps)6oEZA<#1(wX7m5~eAKQ=p51-ZHrI}omE^;7!93jtJ4^S3!$ zS(~ms2DHH-KQSm@Of&|!S&qXxC0FW^65`0Pog++dAez!5N1tV?I)!wobGvG}I zGN*aJ@*LQ{%w>p{{H5mI{&+lA7L$$Ol0+G^&!7;-+JnJWKL$m^iR`ypeao~nkt z>k}kUJj?b-pi#_C2-lPWNLi>)JHHX_`>BA{LC-TXIG;6xK}+=NG3Fwn!2gb;j%}pb zYk5Jv#n*+6pQ~+e1m5!g->LE2Z&JAKE#Y(*mLB0l_06nz@1~{TiOWOiUmY*@ji*B4 z6=kEP1n>M|C&srFts(Rl)1_$WpLg!*p<~WsuIg_XTr+!QbR38E*OGeB5~aTsG~dhl zUzz7%Vft9+=yQ0>bVtu3qJDLUL9!UIbm%*QJ?*)|))Agc&ux9bD=ZSw+hVRNk)#>@ zqb|Mk6;rMC2e+>M z=kf35#r+GXB%+f|U?}X*r`v^}Gx74y?yyzF?ZO=mj5s`*KP=D*g)V#a9zqtxRmx{J zT|pt8JGBw+YjBMrz~}y9G8bXsUjT^gG>2NG?&woH#k&~;Kbj0HOl~uKf8PB=437qTdue=nU^Tn7WJtm zpOowCE~A3(&fAUNv#4!w)>}HH2j+^Q73S-`j1{b2GgqV64&`cdUD){R3L@QhGz4rK z`kzmIpE}_U{k7a=r|NUmRYh<6sw*uhj?TcFn#MpT|F{pV`86(I!fmzJdtI(8HLYf% zyP=bfB?3CT0v|cQI+$&L`K1O}DQH`Oa#-LtE7i)YLEnoeVr~CatZ6(6RbOg!^50Go zO>uJ|Ej_@0jJaz&;?-Z9OvG7=BkjS3+ckd2ZM_}ijlkSS=5ukImYaka6eM2WxwcHN@dx*otAn%Wn*H{5PM<<(9gGm*C z#((+#!l_@&`6mHiqA;?Z?XJl0=ev0Bj{8-}rUDn%-6Qbf?+74<<-ANqe`HrV`m+uZ zyw0VDHJU5b0r4t+T4y1~&4u`xi#a7io%21<zC+K1WDrj%;VlIesr{djj0pM<7N zuG^5b3fsQa{T12gm#yxzO?Zws&r0#0vl?1V#9xl$H6?Z9qaF1<3srcA^1;zsmAjgC z+@c4K)W0ED?L_PA-@Da6WK6rp)feV_(9mwejgC-~w6CMsvwTRb$#Hk6vrI0z%BP?@ zRAuwMW)3D{M0ji(!)~pN{h?A|`?8%uUH#)H7O*kn$%@v|2szl1hGi z8`za!wOAz~c(&kLK9rBxK<2W$_fC&$aPvr7nl5#H&r0y@z>p_dOzsuvDf&2E5}mq@)ED)wQ3LpQ zWAo257##8yUYPE*_Pp@%o?L(Wfq$KieQ^NLY1T?EKu||%_N>VfmjBf2oua#chc8x7 zyz3rRY!4);ydDH!knyG#$RsTd-^FYKuCal2XPUygkI&H@#c9j}GA4~a~yPOsN z9jQGA5*0g+ifR%vyzr{+3&_U$Y{?`$brSd{j@tVG`d7;|^Z&81nl+vFqo7ekx z4c{fed`Q}G?;aYud9|S#o$uL|NWDBc17x9o!@9`;5o$ZP{jFpAu_n7LoeO7W z#p=L7T*8OnhSjS}{vJd6Ue+CqmQq*o*5^YsKWEx$@wz-5FV!fM_@9pgppqDWg*Eyb z5q**Js3TE7y`@3^w3xlV<3$4aFMyOff@e=qOt%)ODF zd7R!gFUlFy#29L@YzGjmBmJ_+t|_585()3DJaxE!+3HWXtD2CqRrC{!VzleXYVHa( zF)J|T;ecH1YJc+Dct_EvHj`-)#QGscHDjkHM%$;7Jxo8k&szAY8>{ zNYATeatU{Jze9H=`&plU^#=`61;ouLN%d_q+YY4y&p4KnGyJyKzRp)vU{Zf4c4dVAPNSTu0<1Ai$~q5@oHantLDng%B^qi1 zt|bw~tDC~n2X8r;z0dG{+b9=BSjFkC7Ss~nXYy17pUqZ>D{A8fz%t-SCg{{<7);iy zMBa0Mn`7C;?{961c0@XN>5(#0c3(h;L;@&Ic4R4q`k|WV8#oQenVcyq&a4Yr2Vc#X z5reGsR51FLl@R7LaCi7$X0ZHVHQx2@d0eypL*MM0iHD-u>m~kv$}N7}TJUH0zM-8X zqFTVFRUA+@8Zde-@mo~)pTp=aHif%djkT%D+)L|9kh#%>B=0~z{1KG6pd=uEm<0LN zkboJeXuR=aZAfkC?Lifl$b8^@!~lf8z3Kk|9kwbK83ul@r{`Ez+* zy|8|OfxjLrCDLlGNf?hixS^l_{aJyWPQEYcuE5g&6H-&NoK~u)1wpxYSc~4I%n-sM z8I1VRBJaKeg(E_8OW(?VU(r!VcZF@w$a2Zh+FX#a`RWg(pnpY-tFyb?#Qdg%pLON` zk0`vJACxpGqw>^GTM;-B`W)*(yha*^ER-jNh`)pAtKY}WGyK_p{zC}*&t>7r<{Sc8 z>N-9Md((9J@#Afvp43+&ta1PYxK!y3ZnnRN9#+={Y--NOPi}#@GOrxCi#pr$aHYj{ zDadmv?bdP`kA2$UeYAe_Wg+_7yiSo)%;?4G$ccDK@1>1RD|sYXeoG~i|KO#^3I$DI zE2mYDX<JUV}!>F1PPTQ1Dx(c7lv9{22Bbdr}vZB)W(!xb%or(|9~9r&l0upQsdiTGnh zZ^}w^$y4xX)UVC2Tp-(qRSnGCrj;#=Rt{0CVM|bUBu&vg6Zf(xjM3gNw`?wgI;7dv z)b|Uxix=(0jDmeEt-$g}#iwd;zYGq2M6H%etlzLre&r||B0ctxDN20JZ~icnt{8;Ds( z0mZz0M=P7agZHoz88b7w;@846b$bC;5S}5dlM{Y{I{HnpeY!)pLHOBt;T|^^O-l~J z)bA9)7{f_~>B7mgS+;3A-@9lB>D}Qa&xS()4QOSa68kR-Fl}6e18ryJUW(=2MN3bk z=Rd7&T=z+kzl#oO=rc|i@U^e2paDSw#!x$Uyiji<<--4ZCnG|b>Tfi`2`7UEhKWcTn zpxrXcz)B00L4{z1A8z*K~RV!D3HdVh6x z4(8}EJfJlr&(b_A&ROE7Z;l_TR0)>t83X<5b+$V~k4ugLGO%wIaC-A9Mc*Z8n{spg z3Fy=Qk}a<#-IDbvRks_y7|_g6l^U~L4O9)Dy}O!!vfQGo!B1SAk{9#vKooLQYM>)eRiu{t4>dPs#8hrN?19=OVL!-LJ+L4GIuxqus0=) z0Sd$!pBGhvE9A*PYBFG=oPKrd*(jt~zEz=h@huA9jL5kGhiLm6y7zDBu8f zu4>F$>LA~pOkm39FeMa(rVXF_Tof!r&&>=z$3@54%mDUcLjq2$(bzZ&5e)rX8SXL# zI5e&S6~`x*IU*3Y1A&9ykYTPGf0ju*Rj#tUxP6lr0MUqcxOX(SW>WV?tGH6bNtUz% z^iv$nw#o5dGhEa&haud_i!nEiw5VS%Z5So!9>tuu_3+dX;DxZvHY2MVIRZ6Y?oR%sL;I(N|2ka@!|2OuO{mbOxf)8GtYf&f5 zQJ104w3*eipoNRujpJX_N57nbWvF;E@4+FB#DiD^JS)nRo4F7_d*oIex;8D`>+=c- zV42TUW-y1)SRfLSz^Bp(VN~4=TARNWmZwn(x$H_WoIyiBDFI9swW~Dz$3-;YjT)D< zBBh$C)+RO_Ew<2HN3*nE>JS>UB;qIu2eP38XAreNvcPa`Wi`*3|fJOFHt&QU%SR@AYP`r$c$pgB9MLS<)Q%`d9q?uP{kK-qp7LSz*$ zCIhw}`znCGq;j-^K6ztRGOGvMLqv3j0BDjHfukmYRNc?1HNv0Cbbr_xUE+R%Q@rSD zJ&-%;318@g1k^x*47qG|-bo(-DVe|wYFrnyOkRI!r0TyW`sS5_mdpvGXakiNS~{W}NnijdAn3{VWg35F z*8FtG99pWiOY;>1cI8?NBwztoIHLWPH-TnSzbfDkr2<;`hn4#e?}WxoQSMvcc!UJV zR8Hj@?8rQ$+p!P6F6r}ivhLiRi~x7kW#qhZ@Y zz>+>~l!*CB>4f-~l0ndf);WSz6zrsb+cSV&E1hLmII2t4n%22_XPwab!ym}ljO}iC zHemPB9MjvT3+2I1jr+^ywjkoaVwMkL=RfCI@89RDd8+*RqC*IwpZq-V>M_vp4qK$L z3?wqVffI(TFYsUBUP$oU~lecy^xXXEf-D!zl2>LCrQwR{gn_( zqI8x(tK9M8+%Wa_%Tetg#%+^{l|gmf4ZjFq{!B}kY^?KqC4N7|jE$x&Dx#N`D>i8J z)3ZiX;YDsTZj`g5X{f=v=^wJMggnq;Rk?Ulu*F09p5$;In6fD zla1J*4XjvMeF!bamK}^tN$0{+TlL?*ew7%glNSFQiBo0?;X@EMfp@=3_@6Pm5j=#o zFR|5I@`+FD1IJWxTBumrWWzO)=j5R1w5CGuZPSSx`fy^FW*~lSRz6UegHa|SrcO`x zoJ#q(0sywzNNg%l`|U6BW*0r+>(|VY)@w$;`C$#_}`jE zHm1sCC(pnX(r(%U0tV@4bc9fQ&Tc)AunmsB(e|BDO|@N=DheV9QVbvvUPXEbr3O$D5UHXNkP{TMzrjXnITy(5 z+6KSqmCmBI^Tgf)Ss}Dc9SJ;5^UblAOARC}#_DGT3s?F8kis#vN6T?+cmmH3MG(3`qY$e!)g#B<+&)Uv%jOFAC2IW#|ATQEA1`%lIEI~fjj z_@<4vyMNh@GwMG-UlDMRJuc_9r7FHV7M|Dyb@K1WDfzaYM3OBMv0;5%!B%}TsH5%NqW?LEtLHKw8_Cl4#ix9i*A(*1z-SC~u= zjBhP%NoMYH9eAbJyjkc?7P_hNTgZ}oPxzPWh+M<~6A5(TCC&AS)=TBbU&WTv*2;U= z)iJyP3zzapLOf8K#WKC_hPBY+=$(gdpm9J(iverW_B(1x$2o5H_D_`gN<{>!o;#vi z&)m&D-R$zL($h4Gt*vruBQq?=?v-O1WCchs<+iLzGnINmZOJ4~>FQc<%wnabeEqHk zH|gbeSCDEo+b{z{N48=bCY<%BfLe|4*|w#ub!}4hSV**+(z6A-z4`P7sjwjCN2G}l z(SIev$`&NTy4m<0!+zBXaV12~@+EwJf!}`i;}BF^Coh#nA%Xr}qs(=mO1&klPxc%o zTT}DmtJ7k(a>9`vbgVRQfmiDF<=jQ+rHT2#MdJe2j_>N_xYP8ei}joS9wNsiwAria zjZdPzVeo0C9w^XeX?vGrH?sD$TG`2(X#FdgzS??0NZ zeoQ=!71di`pnxN_N=Hw62EPm4um35+Sx+NXyD{ME$*g(Mi!}SQK}>lSkWx1u#sV7?uH6^s55o&ed!t4_37N%ILs0V8K#-n@u^>0s4ho z<*&Dv!hfroQNC&RnOy92yHRb-R2KoVI+H>%4T*4`wQ1$RRc5; z$cT*oA9ySERVqP{aP&i7I|)g*Qyw3bs_-veNF!@#bfcJ>TYx)I*cK%EGF*UkrCw3z zL;5gD#l6>HP7ecj9nKt0ujMaZO8eOwcMl;-CVG_bFvnL7k2&*9>R|e?cJ7TeA{Llo z`)jik6}zjfT-9k0!>Y%gFJYfG2FD3DhR2BxalSl(q#?d#JXE^N@{r8%`})AZo4wnD zu$$Bv_N`P>Snjc0WyetSZ9Bf*!XTZ~E8c>oi3Xp%nXjloqi8_iEz~E(d&wq@wrh zb>^fW+uxho^BM+0o(w(!y+v=Or|+^!I!qKcSSe4C`sY@+O1|1!g>83El@#A7gfgDG zeWW{eWhNC6DwB~1NBchpy&IW|*^B~&K#9xjVIqV+r`Aj8oeyccpl5&HGd z-0b16*a*XSZF9YzCth0~DhAs`=k)FX74KP?UeN>pOKA5pOW%?)x+U`HvaVs4G+Xgl zZ>va^9LazztGPG49(zrp>UpUP&>h}8@ub%D2Jdj`Vu0E8rNMB$#yCl`Mq8g{v(Fyr z>tdqVXG)*9Cw0uEp=%$IVMJA263&g9*G!)#eZ{d>qM70!x6K{n($g01*FMF5X6RF! zo%c#=!ZDOviwV_!!D`emd4S5J=miTmvXl#-2{ox?L&#^3 zS=*mgFBSCz&R>fW`fpzz2EJUSCvRce8kU=Pd2pZSl?=P&E~fcbSz5@594Ekz(snAp zw{OkaCjL~~=>?|Ys7vgmoWTM(P#uOQ@WT+j1J7iY-!?)ltZ6AQVcIiHu30fssd}1O zR3?V&DHQJbKpO|QhpwrJvj}rR{nn*3|mwn%Nd!XrBsu#swhAex8WWVCQmnZ}2QgeWX)&m-K{x2-_ zV z=#lh&b;5=srQI0`qNU6+VTwGAuVc+0npdLQTSVC%21KP~u6(2H*0Z-p*})joHZA5t zBTXyViXe;p28>I@c6iipRmON9Fh{mU1cj&RBhw7o6Z82BYvGpe=eF2O0B6>t(J^DP zFS3?DjRMw^3DYE>+)Vj~?{SdRV*qs)U72-y=D3^hv7R$aH8Y6{WP1(h9sJuqFSVj?i{NJvQiFe_2o*KdC>|+0xb%Tg zX9Z?XSO9fWsI6;0CG&?r`a|lbbc(s@z@PJlMen0?MU@eGz)664z!{>D52gq`fX0`c0o{z&fqv+FDX| z1+0ha9XCUSWmhmED4rqFSg+N?KPCkx!e~@39qYD(=SJJ_90S>`9{FS#4l($^VcS%3 zfMjRjw}#npkB7I2(IM$T#0}GHQ*I!LhyqmF?go+Y*@uP2gR6ydGrM7Tk0{pPSyqQ+ z&JD%(sx6C?-p4{;uzXKga^u<+zxhS9ZgWVyt%~SgW`?Aczui{+K9#rB_rPu1EbFEl z;JTwOfyHq2mWMJ!hAPXQMo)z@Dvp+(e0TyKuv_k+*alBUyCe2}X##;c^2rA%^u4mr zb9^9U9km)?5qh;k?S&JjvWD3k0k~_G-z|&Ho{~qC`eJGBWL;!38n00mpmi}!E6)Wyj|D1P6h7jcV_G0=n!77$(?>NEQwJt_b28TBbY_I6g;R@L(^%Gjn?3^SI6+dayH9`~aFfsd8b zaS5r0)ZD|hw|C}jwBcRT#=h>@zHmO!Y{F{3xHzv1Xp9c;Scc2;;J84bX z57OnY6#3S4=NsjzbLip^=MMwZJ>uiY1YF`)UF%TFqojlwcsxTUF<*~ z#3w#KO2Y4Bfsz_-A@<<{^Cn!k2RR_JEN0CKF2}3((*p^3yW* zsHgiHiT!n_oRe6XY|ysb7ssmB$p@c%2Pm0O%Z!FX%-g5G==`p- z6ZGkmQRjQ3-N7MO^qS`E6!kcGBdv4i`_G{z!!$zF}*IPQ2Z!o%}Je zpjF-IeOQV)Gfo=JhVbj4aX;ulLNmzN6COx~HDg#0- zjBi9sFA}`o#ekfNSYuPK}smU_4ZAAhLFv?_qt-nIO(fNHv{Zv%7a+7$5yF5NQw8yD5nv{o|}@v@`n6ZM1G7o1`aAsLYDWHn?YyP1sGkVyTl-k_w%p(i(h{( z#Hr^bRE6jBso#Q|kg7@b3|C}4vVA`kJ+ek;r9YC}AK%@dZm=oTE+p@p)FIq@SAdwY zl=_&?GChJ^n67BK&$lPVAr61+q3=L0^372zkdNd3Atf|gf7;=3Sn|8wQ5(a>{+uD) zYrRww4;zWaZ|RbN3t1PR@jj2jEm)J^Oxa{?wyBiW>^N9w%fzY=FkM7Ol6*PWRFifN zgrTk@B-$x|)WsqDraS{hT@M(@GzQtQPrEd;R+IVyPSXgjelGq`Vk@%HeeuX4)U#A^ zm%XO~B^^ViS7hWNcsAb3SeQ-S2IY@|PVoKV`u)7?TC3t}nUXuV-_{Fn+${3TFMWBu z#c59GO#R@ut{RLO0us-(kbn@SE8C#s^(_bvMW?I_S*NV_(Zhq7F~?siFSdVEdYjAE z1$8|ON>+33OPbo-31{xsGowAtOz^VPvWM!=3&K4&|7$C?pVH80H2Lr6Q#ML=8FrS`-*h@|K6`J2;8 z7C!hqle`^x>)l^62}I2WvMU9@`6Ub0Guk`iAm1T<{jL42z&gidW7si{fArafT`9vf zi~aIWie$#J8mvsG=&5s=ZK1j`@S-e(y}dFR$p37N?BZj)|1ZES0OYBUHO;9oi!R38 zCGl(=H7IAzppy0dlM{cUD|_AhCUMqEkI3nV^s03W*p7Q32dfpLdxu-C6wUX5`l_7;e+@HbdD~|D@YnkTy{nedutrp6=JU9BDq>|k` zdo`@I6!vP7*8S)X&)qoNtqo_G>@4#uTe%IfTI<>i(b4nv=;W?lIxk+xC$=&)&WB0@ z3HcT5ws@=5ubEl@xY5WH{8=w?f@8WGS1-eud0tWfrp{@t<-3>iw4Ads>?phM04DK8 zxomr6c@;Hf2jd@v%{Yeq2~+o3!95$Azf*Ra6ygmso5?CJSUfH+i17max^#vLhT^A@ z9!Yp}*fxtT@}6PZN*sUhH2g>dA;S*NuqlWZU+pP|>aIc!=E7*4JH=@3u?Kx+cyYZq zlvR z5i?;%(c>gfjcV!Ms+{jij@q_7nP$w5+kQO~W$%{V_+kek;LGVaCrZrK&?)zdW9xu{S*WKZ_)sTks)IWIL=&6OXRdJ= zDui{Frwp|LMV7xR=1^_^VXF7V4gHiL+Ex3IgUYR+SAnAZoXLjPPTlr`qc$U% zyvDt8Zb1~jrwsq;2>`6~=jV}_L{19HOgNste8vVUfr@ClBpI4LhwKP`}tyx39TCctSR^ zw{-eT++42ohWT@w(#ZRRIzW7{v$P?RXvq-VK0Q zQ5KQEmnQUX8IK9XbL`&fnl}waC9?F*UlI#LYEJV2FgIlLXAVW*&M%|G>lBG6x;J<~ zv-bK08GMHfo5<(mxf~IRfVbchAxGdcR^~IUbV6~>rObY)tUc@JR=qy{?#zMGUP*uPD?R7nz9#DkuV*Mib5qxq@ZF^5jWWWXmoy*y@}Hn{zK;r0R9W-cCpY72MQd2E6Z^# zixo!jy#7Z|GTq|X5g=IeleyrG>@M_BJPB4QT~p|s2|1{Uk(N=f%Nbe{$A^5{&4LB& zMwl%U^_oN!W&O1%Nc11H8VtT<3$WAO92a2&*ACNKHqu1Z{qXT~5iMb%6#*XNa6gQ@ zpXDyl_69eYz|@FKn4m&m9Ife2{5gvL^X{|_xABqD?j43VK|m25w>whg#OZymK@zHp z$F|2wLd-RX>JykN*9!eAid4-eaS@Rokxzd_8LIeIe9baZC&UOfUGE+AMMy0^y@(c2 zuIWBY0S?Q$uM9@ZjyuLeUEXsZ^>`qV3)F-K9EMkTm{N7uTC(3}X8_r#`1I(mJ4Re1 zuVuXqFD@Md8}+v%`RII9p3Q$Ve`1>5Rif8|c9f3^3kN6#{AF&0g8Ll1 z-#ojNEVm~U1%>l|^qAozdsaG)TYPfTj&9HEMrQ+1-5G`vTrNO z;A3qoR5CwT2$2?Y42gQ03!7)RwTp0-4g7lldUftV(Jt&&%||u?-j{6$w{Z8jY)|NW z4-#{Su7}cHn=SjV7J&50^@)!zC9=)>dCUi5qMY@V#m;9`MDZ8OHDnJLJ$s&gKuz-? zr%&Sqj^hr@gLhYfJz}KigIoP1Pd{|zn3yg=1BnSszd-}7sF%OERCLK8W2)k9s+*0V zfYx1_71X#B0$JG&FWTvn0ft)E9w7$%wCvbNXRGUfSK zzHY-y0k&g0-$6LgHPMnb1#n(zvJ2<2S21j`oYEOXD)c-My-tcOi1c`G>XuWnLJ7JK zPJXym62|l?-aCM97QB(UAjYE;I9*?vMjHP@N(?4S+={r@0&&_(gF&48om;9>934A6 z)y^(k(B|M>Pxw0J0yE20;mQYYBf9Nblrx64cFa2lxVT3inlz<`G%haGtThO^Zlw;2 zH8=91#}0z=b-EVB7R3C9#1L8Cgas7%@E!EPZraa!?a$W~9}kMK5JsY-%Nsc8r>CKX z4W|sJbf!2+Wa-{?x*J6r)CC$M!HF3HiDZD~HRBZzfaMqy$uS*ovPo1az?v?F(7q&lsV4pAz*P zWt_XN_vgf8{Epy8XF%{?sS9t@da`JV&?07^+Nv3n7?E{)N>1s~N;Hceoiimz(}Glw zP?I~sK>uPA&xzvi5l5CMCBdeFPg!QhQI52DBAAfCmy87$m@?JRhW}kXF>mYbH*o!&~vm_-gmv zDzWAb)d5lHp3Zm2{m&>WU$oD!D~P*KJoF!%w)N)H0eV!;vlU&}?p$pakZqU`H>0YVaL1*cSPeDMvp@g-jU&?ei$9_%A`=m)i zK}9}3#(7J;AY4~{Yl_@rd2f51<1YY>^*+s!H@xKFYtyT;>Zp5y(u-n!+K%_JWL;cG zeZJ-kzDjednQvNB@|Y~Q?Oj_neyG(9S*4Fr6*QS79v&rH>P6C?7s7KeqpIEFcPxi{%!vjTPg|2p3 zqTVw`@Yub^uSSK1%U^{eb>2Bfga=8p14?6W^tntcO>mtN?{xzN!vgLr+#j6jwxKvwnvpgf!sBgLs~k>7Y<)cCK4a>@p?_3is($BK2{(EVO$&1J_p{{N#`O zVshLHy5W6ktHS5sGu3!IuJq-jS!^aeeKVyHTf(Ww`w0+!&= z;WVt0=9h@kIa(?wm~yle|2iIAvX3c(w*l1QoX1!7n4WbUw(><$9z{GyvjiiEZFK2g5 zke1T!3}h7rs@(K(Mo>nGmvZCzG93eXh*XXsn2rZ=q(O7->s8C>_ibYSyJEF->OOOp znnu^q@9z)^Gs^ZDWp__dmc9{+HI4}3to>TYr+Sf6obnk~*{q7V3j&^$AgEw1Xb!2e z5@|&~sT|@bWbAw+o`>CNokkq-d}lJhv)@m`%v94t>N=SM?~p5%2-LRjEwaQR69asO zgc;m%A}mU8S3m66jIh0Z94kngmwuXqcWox->`Nk#B(GBsQAB|K(MS7t5&qF^AV|-Q zB4nsXsJDH%?qFl;E&pdsi{OQ3wC}-<5LC20B{c0*6in4Q{XPaqgm1qq6s=gw?R-|r zX>4lT-R;=@EKDXI%I{Vl(7_);?<5cF0SEHXTM|XN;?5GF>V!+_>gYFnL@iHAj zr-`)P&xAHDCzFT#gFKE#g=2rLYa(<(i9f^AEU9&ka)#Qo&YZvwY%vu9Ea5wHAn?>b z$@&|ZRQZ!0y4#$DAcUuq@mV`X7r|AAQ>biaM&5FuTWR&e{q#10&1Er=X|wI^u{f&p zR~nJj1EEdrVq2F9XsxFDgZ+1}F?qixDM`qd*vEAtl>Xzs>(uyqmt<@)$9e*jaBIAn*HT3rcMTOiOsuOC1TsEq}eN1;SXC}Cg zKcalH0`Q{!&YMKar#uoIVc^Aia<(s4!_p6XW>eJtl-QaHRQc=8vRBbDL zl@7Ua%-M)KXN6Ztq`t$)p;$nseH%se(OqP*HviPOz8Y?G?J6-Id}!bEiB8utbBuvg53-MGz}L% zFN|}B2)p6Tvky8BH2uBm?~pu3pfd*Qqjp2&|AE*Hul3!E(8|oEO3aKb)6zBz z$O1B@*guXN$VnHib(hcckIe^4r8%$G!%;d%;^82lTu_yxe5KH z`U>V+#B;EBEBG`pLe57CrObf2doE#l?DVn};yCz*M#)_f5Fr>d1lk9aF~C z6Xe&Ed^Hvcc1|nn4-;TospgcO7^YOWi=mp}3(AlGB&cZPV?|y#KC5y(l{Zz^VX_Kk z?tdD}ETl=K$WO^D6++W*zYZa~9z*lK#K9r{Is2`aed#r!7%QPF5zAv6;%Uq_4ZA3@ z3j5ptwMH$d*CZatm~eRfv#yx(byQZA zj!tpKj)mqb5)5_sNQwyEPW@T`BRpLm9C2Ha*RF2A$I*Sk?N`UOpcW_jk8?>7n(jn) z5RJYi^^_aYo_n)4tWL%(@gh;fZOIeX{GGzkV>C1{C@CZ$OmaVp?Je zU^qhLH?pC41 zTnyRcV;?`r`xgpHkD(oM9n=~$g|9oxbeL8KmN7A5N<8*jzXtOQ&rUL=yn~XrFzD?O zO~llUT#d26Rbc@cuec3L_3F;?-OffNUI)>ttnZXZ@zZc7&}S#ilAOk#Nf*=RNC+GI7BCslEHTG6KqatgYrDqRLWdg=j2L~ z072!={N}gv)}T<;(i{ixyI`nY{c4XEL#d1BYO+i&0^L_q6AGw5W8ZIQ+0ixWORt7X zoV6!N9=8(HF?hJMLcf=j0kq$z&?DWwICqojYT5ODr>9vE(Ym)vXX}!PUqmo;01r(6 z^D>f|Obu_vH2o9`Cc9-u>l9;ucvX`2tXyCuckq1qC|@Hqv=np02jBY97vGiW;51N? z%eTiltH{FAv!9LKlk@4$;oIZwe3llKZk+rKJ`$?9IQo&G0EHQq8>@_01g76+T<;)? zvGrkuA9qlM=zmvdgfPm>$gH!ZLAbn*Hhb4;gtwozeYel3n^v?MlJrY1KYFg{%T=FE zp`AH?&T%k)v6*DVrM_6&y&wVJRuz=}TP3_Brf=n>c$gVa(YmvmjX2);8mb1`8@8m0 zNpHIYeiJGBb=Fod6fQU#PR`3o3XZ(-|g2+5cJun#P+)oy8#+HGH~gl<5m<* zy>fk7SA?DG-1EZ?#t_V29dj?Ac&UC6<_80G&EsQ4Z^B%pKHuIXU+o`{?ihKt+N-_H zmsq<&qi@ajGg>=Fr-)(GXGkGLP=a0#5T5GNZKMz854WW-LNRP6)rK+tX0tsgsP}q7 zG~!N$V`BWkHA-2H@CFUS77_E-x@G~oB4huDeZSG(*j6pZ`b1L}YZ}@z3v=O|O|VZ_ ztt0fbga^J=9IAQqHb#D8W{8a*P4j z75p?5o!6XDU~uI2DSqa`(7*dbhA7irQ#yJ!acNAQc}sJzM7cUv*mLs*&yri$HyISA zEh3w#Xt|)&)v7w}+^xj4uYzicH?OXtr#%$N;(no0AG3>sBp|h+GEmX45-lbXqGUYm z&uuwwD9+t@f}AWr!Z5I!{G)vfTFJ|+$<^5vvoZbtX}3riZ6<0ZJo82xRo(7A+!rk6j}fn%u^Or{EA)?Z*Fw)yfJJ+Fr0X8RM`xc0^R3%Tzh$!YkB zX4^Nmt)2Ich}Rs1J4H7n(wz2rR0(XdTHi9lS*lkHGJeMHk4q~pw8&LOt9ojBCTcTD z{5I~qFUNtPG9P^}ay(lq*$a!m4?mu=g``)uD=B=1eBam=-m$o33jn@giNPT#u)!mn z|H9{VYi2Afp~zgnaOJ8$_#K8s=TKiWNbDl;Z^OLu zTf@#wUY!*Pl#;jal1ZK-WQHk0FNT>qtgI1wetQ`_K4+s0k+q;$=ftrPn8`?6vS(kN znkI)R*)Ems+JjPdr@mBJXS->vRr-M*@402YxyZ7NcyEP9#r6`i2y5WNz2;M^4h-?B znW@chl{S_VPLkxh;xS3mHs1~}3$wp4B(XP{D@c*|Q5`VS{K^v?$-=e6O%03g)7c=N zlpH_!bl}n%BiK1UW;JMcGHf@iQL4fXGdS8~exHHMQjM$xYfFuet2Q|m3eWD1++Z8J zTiLp{5HHik9eNMm4zw{QrPrzjJPr%1=Cwc;~Fep*LxKD z6#9IJSRuc&9P3}ZO+sTI9!;Gsj&7wA&yYn-__y$so#$ zYz!R)KveNu{&8D~JHoc2;IEd?AkLroH zjJJ*>vB0^Prg8SIgjLcC^EER)QT@q|j#;*1M&(CU=4)3FADcUKLYYpH%T$ynP5SYc znqKe^a`t}#V&I-06GcWvCewC5wWGnu85SL!>8f8Jx8V z;fbPhyC_g^z-fSQrvfeo?-8^l^{sW%m+YH+v=JRcjW?p+Z^T#%hWUgrI(ZyzyNxjn zxe`2BPNl!fGt@c`RUqL8fxXsK-bk^tybTUk{Ka_hq{%oP^Vmb-26)J}|KFt#5^k+S z(!Vr@n4OvSn~fmg{5$jaLW&e6N+oNeyPes&UA#<%&m=Pou+{Yej*fn;&-CEeya{w<@8=5Th|;~0QxDSNzz;ETI@c=hyCR2+hMf0lSm zihsOAL0WM+P_Do|os}+Taa1A*Fzd*-2JWRR6Yx2IaVrdMpi2I}s-OJwm0nFQcwdl0 z^Q#SBM*|0kN5&S$H%PZQS*ep4#t3wi7Hmcytqd!MhJ81&!VrIq^I|;>Q})FnZDYRc ztfIeQ{>kR86tQ#lA_O9G9$>R&FS`m(ja2?t>_@Os^s@S>6>T%NKv3qAu!LAVp+I zkXm?UcrwD50ZaD!qfQX<(g&?R_XK8C8tb&bbH2sAU)iSTr-8t*b3XPVi{aU5p zL|%7iIr-twUO=Mh7yzM4F5BPD{@9Q8i;mo-GC>hB`;zWVR7Fc9*^frf@jpphs z!z&>zewo$Hy%VAs%f0E^@hB-e=E)+}ZO!>Z%v-Sw{dO?-7{^b#vABh?k8FM`<|=)= zeSU2Dzmt5+B)OdZzfm}a*9`Zf^iy3(i=uwSpFL*4vl`YZJ$U!zc6w`ES$dI`%B{`( z=%&t52G`H8Ysc`}J)B&*0~8HFpLp$YL)mG=zyuU_+gc`fs`4`L0d8tg`qm%f(`=TL z?19VSx%y!#&$pD7tyKt_Q2pD2`~V}6WrKQTDk^)^LB(SGc@hc;c9@iOuVzPgDDUo- z-K(OVFIMwmCwcbhapxv=-UhDRl5`k-@V@=? zC5yOry+$YVvmusk8cO;>7PItVd>%tQgiCnysFlJs+!Z~(Tla)N(=1OdSC=Y%{sBfdM5Npo?Lwu-r zHKS5T396F7(O2$G>>+=B74QXL{r$Djb3~+d8Yycecpzkf5${dmTf%8Jh-Qkgi%^_u zE-}SVmDi5zw+ck2D)KjOSE7#F&JWh8{u04K%zT;i4kK;ztU#oBKQcOhfB3f_*C3FmleY@^L-@MXm5*~^vS0*?{3AV4q`^013 zIxT54VDt3@_LXVrBIoUY>){a>FKCkgwg6f9_f%jgDs)KnV#v@dg}Z7B`&N1+TP<3! zFZG`}T$k7nc2_3(q<#S$w7PfmEqm4i)?9n>hwuMxjRoP)!9+y;ytVqo^K{QO4nu-v zCF`@({@1$hUdbd-!Iq4Mjcs7~B9(L(vn8FpO=U%&)zMfOj*CuYlhPNx@~Mo3Chhc5 zMg5;K7FVIs$MqFJ9GK9~w~{hnlL6>d5`ApUdDp0@sN-nt653i|BaR8qd)+y<)Zc1l zRV!Q#??te>HAjgx=hys}ahWcC|C=t!0_lnIDv}frcmrfN3#NvEm)O^y4ar%IrUrJ1 zv@X9i7XGrDG!SrVLVE*0KipSyly}-w&9QK!&oTbIv)wXijM(SpW^{RBJpESBVb?uH zO{|h)WvqUS3oQ*kVdUGg`WL5TRgd`0CUA(@iZxtQXq3$}k!maZRJ39`-)viVN%7n7 z7D8cNroZih*$m6-Y(n&b?JTqL#7NQ`0{AZL4QHSSWQOYeZbx4ae!cZCCoV!{d7tQb zV8MQGZ?54@rb{Q3Uw){;<+fBqj?hYhflJEu4(1s#!@`qMDp8tQ}WiaZes z*J7&S1~IS^HbgeQKx}oA!>bJn@q3|z_X#82H;W*?>_gL<{OVsS!J_-$Uuz2Bv@E3d ze?^_dpEUcXxm8j`$5rSg3napDy|sS8b$P-v=-8dh(jM4G9Si*@uM4vbxH>0LcdWY5 zknre@w0SSo9y)V#uJLW)?`)`KB;qLU?bJCZ`#$$Xjf+yI@Yc#653E6%f4KS`$!y0r z!_D>~(R~}9KR8WxV6g6?RBwwaZ-vIF|Kwq8q4j(X&=y2^?7707YN#?n0Dn;N)Sc@hQThMsyxJ|-{oraYPnkef#3wY1Lq;9Mz8)^1 zCzI)XIS|qimcQ61FVaAG5}wwaH1V9~Kb(6>p6_ zEN}L0wC9hmd;EijlbAEn9&uJEm~e&KT9_s+&@?zNXd;tVdebOg@d4*0TR1C_{GB!H z-;=-CEI??8<*Sgw8W0i(bkZRR4UX$(652e;(dB6O4+dYzcDj;uY#4$GGnV{Ujp5)HwURF1UVwU&8j{W_c( zl}KWchDkngj`IXB*`!89MnLLfvbk~v_9~`WoUdk}1DC58-bGVLz@Pa>$_&fxG*K?M1MUG;5P=Fd%J+1?#qnwLZ8pUTt)`5z_y8(~2I zKO>C4YoGZG?N0v#OaIBzb>9i7ua+|ZfAS_EO?ydvRzp#;k7VV*$03t{bIYkm(t=?P z-oi&J4zu>lSjEe@oLkn(xdhCDuy)O7OnUBw7ZE@oA zn#{MJ9=RJuC5xTpVcvkj@tsPF2Q(a_g-+PW?U0xi}b(xRP#^-S{_A+O6 zy`XF!%?9iXtD^^d>;(opt>6V>ar-+voK}{!uh&+XHs>+-v<*->8T50kp6azGN#(4% z2Z}7SiNE8?Y$MOCME22Il9l)2vv*|UIN&B$EMFpGVaJ+;xugRVrBw0#hB7FQPJ&aS z^~nY;bh<9*BgG?EoqFx!!M0pVc@C_J9QXuTyt|9KoE}5K$5Umy{}K6Hq@*X3iz|%6 zx{bn74^jMc>vx$V8xxp{y4L9V)n^%psw+h41T%xx;kvRe{6^cgvgoTU-Pg4~3Y^|I z*o@J!4I7E3Y4{rnTM;tuU;d4S6rfF}?_jrIrtVftIdVQ4sL*>wZkja-WQTmXJ*WAO z6}i*XwP#Fz16OIKkYABrgzP^DVnS=r8Eidgf2HK8uh(UbDRkf0zC6nwyKQeKOCF7E zY3-2J)Y?}k*F=)O_s1$8hJ%f0$zR@*?R&v-ybf)`zI+W$ZQBj)94D;Wb_;D?g1o@l z&IS~@(J|F(6q?;BTmr}VFAk?8@oWNHCUFncGYpx1k4Ao;gC;%NVwD0c0bIOmmbF^2KRL=aIbd1%&bcJ*7Hnrx+M}&PYiX_qHkvKHlxWaa+Dyp=n)X>Km=p42!Fq1pm4+Wg0IQU1A3c*li zy_|0~=!m*V`sfJhYYmHIpG=9QN^O{~jy4>S-*5aN|B;fQR6P>Gbyza}`ofev=$rFp zBk0I?<`0Aavq)d=d+Aim)CmS#l_k5=aSutc8hEWL4J!4R5354o!Jda~mi z*!ATmTMN>rK&I)Hzb3raeoSq$?UgWS3)qG{GRKpRwRQ||<^^9Gyj;*KhH&{vChC}v zQi=M0Cv&Yn#V(y=u@`o~DxPKj&vN-6M9mRWs}U4SyT&mJVzpVu5DOA7%f|Ph%h6D$}aysgmCKPp*=m8F;q#Tg>&u z#(-O0a)l>3$I_tTBeTH>J#Ngdc{KKXpGO_Y^JN`P;<={=KZO@M;kcld<5{nYSIU9^ z!?~fJr8I@M>S%aVjD49{AnQW05hJRgv$`ST*_{Mn015l$g^C$*USVazbA^=l<5}Z> zuya4-T^$l4yBd57&%)JP_FwzU$6%h+!7f4pZU3}B{>Ea09B~+6{`{Xy)a|)KL`$jvBM4ghCs@Cp z?F*^=Ng0!q!CiFr!&=JGBpPys`Gt;7TanM|&O|InVwO%uE-MIQlGigHOOgbe!z<5S z_7J8Ol{Wm~+dC5K)?bg!_zNElJ|m_VVPNIbKqwzYHo9)RkYl5ICC*J0=S=b%jV7pA z6U%&$Xni{r5pJ5N?D7gDHPZ+5Za}?Fs&y-RxOyf6 zEt>FcKhm5`a_H@m!r#zaM3HGge`*8^HlT>GVcgb}e6Q$Dng;L~0z=^m5kH?cpO7q7 z-7YhUI{d7^cQ#9BHaT@-niVDgUL`exbum8ISgPY$xEEfi9Z8#d)riw|8EQiu9T~(I z9A^?CiR*by=LNV4))F?8Bpv^Yk!cGrPpkNQn~Iv+M`uK54@C@lJ2?4b=_9OBn3!8MQxNkOGbe*IM%uXKEyb*IEz5QKwdipV)V$-+h92$|&c$Qk% zA9WLY9ZXbROhgT)i@8A1$&=zh@>6unCj_h}g z6G~GO_KeQ%=|lsQsSIZ$*7pDylRn;WKg1r{;v6U>b+~6TImJn*Ob}}MNv00pBUzf& zh5->Lg$_vOO%&^&s6FMM6pzc1Yg&Jp<@YvnYW~U8nx~DU7@u=CA-;dNS>H?_Jq&rR zu4^A2EUP}}U2S)kadre!!>SIJ^PSK+ zwdD&tRbi#ne?yJt(35CSm0LEnTbiDl0EVY|e7PK=&?=v$8GPtykkvdu1zYxp>CCsn zX!D!26>{|{7gw%b1qE|g0zlClbo_`g+|=HF(L!cHDivq$zxT>?mV<7V{&>u7v$x2_ zOwOEpBr-qr= zyk-IRPvO#Btrf@CGABcX>%X98!Ff11Bgf}tEX!`Le)25+KRk>c!_Hj&(&R?7>;K^G zy`!31*L`6X1=j*aMF~h#5h8+wqV%G&AR;9SLQq1DfJ*NjL{x-Oq=PgO0%GXB1yBgR z_YOirF9HdWK;X`(YwvURJ?Gpp?)U9GbgVxdL&kE>`9AOS{MwTxM}zr)0sShQ2D#(! z26CNV>(px5|JcF3%1YQy7l>s*rw3Q!WB@e5i{I1P6_=4bOal^-$g3=6{CN_Eil)GA z9l_7a#7Ce(3BtR{r^q_G_BzSs$wGT+hy*RCDAnj57k}tOs84D14iL&V3>3z$`Hf^^ zI=Dp8(@BM!^s|rv{q$Y>eV61YAB%)axh#hHJR zYSS`aTIIUG7G5oyC&=JpSBNzANz?kdQc2L$_B~@33*QjltipDju!GYeMM*eV>B}lN zEBgrpJbR>@8LzaL^;LRqF-hl$EBdV)s@v8OP-8$><+^M5Y6K(Yb@Fy$?QV_(sa@MN z+HXt}vhVfszQ9_3RfUg03e}-?Z6lB`t>9R#)c;kq zwpAnhj^)tnpzv3zw+SOP`vxO^Up{o`7CiG(0|!wT?He}oy` z`{uf$9wf{m5dq^+cvE#%N5%6G6s#0=3v`A#5_FIVOkFDF=~Epy8WYQ>qo(pbi=x*8Ijxle7}UQEqQ ziua=(Bf#y~bnp?W=rU3^vyot#AEZ$=8gP;2k-qA8@bR?TX&Q&j_cvQOmrE{B9+MzyDD!NBm`x5dR8f-t5&V`@R$K zP)zf%%B;KXxJH$l-kcvw8X#;KuXx=ZS>kj82A+IQBMM(YJFs9T?Y-rn0NJ z>(?WoD@#1u_<+H+ zv6U12UC?nQDmP?)_?rYvBHQA{ZPTt3?6F&K&b!fE)C-Yq4#(NvvBiK;xOBiBGHW$HsPzF)rbRi$@igE~N#}x#Qy*?YoI1 zTa0-_JiEq#+J-9E8TgRLI7BVE*cFk^G_Fu_uXcq&!trH0<%#dR)L6zH>$#c!TEjP; z4`@MJWs(s%6GgANS_%~@pG8-HO{o9coc-&B zs!3heDfen=3(u=PplpA`&nxUytR}R2nTS<6|HN_tB&U{mcVeLmsO5|A@(@AE&Wjgg za0r-11FUL6K1urt2^Rx1XkvXn8lsuNLO~Ni>B5BVf`nN2H5z&q+NuSaM_3 zk!-?WZ*tk>^mYBZ#PUHxxB+n+A;);)mIN~TTi({a*7xg?jGH&OKlp_ASp4aww!|w5 zTDp`O2O=}bZ=CIJG>dmqJS_0JBrF0P49)!tO^qHOdXc8uq&^!9K}BmS&z z)wHLhUo}x}WdUWdiPB7BoPL;piuDE0dv->dK%ERZq!{;Ve_Z~F$y4&yylEPjSy2b+ zbH_MOC@4>DL#syrg4PQ32|vpA7=8UBGeDUhDCJgu525;$=#*gqojotwYbcx_fZ$q9 z0~}@8A)fqgur*p^#^kV7Tp_Cf4Ou3YchfsiFY;_>^J4U<=ui^3OFz$U)!fX2Zdv7k z{rn#%16FP)`-uyvhO<gbbo!SZ)4KA9Rz z=VvuWo(R`%P!hSfuWorkK|Hp>9awDsO|bleVrc(u10*c9m1gLU zCGurv_j`K*OqKOzvHv`2`!O)z{|pnbL18nnNAWOod3Kx|} z>d`1zg-qH(u<3<>oFHWtRbPsFB|nT~5DlO@U*xE%j`w_vMZaB#rL>5bMzuv|WCv)- zFGcmMc}0(Cdb_0^r2E)TWh7<$r#%%K1kN?wYcnUs)1S9l`$;)x=cOHBDfy<XcQ@55`(MN5Ftml=G>gSF(-Vb%Z=O!X&Zz z>*BrV4eTDG+0Z&1hr> zf;%cgY>9Qb`=L7R3b>VBxq!C8h2pD>e}jX!GQ1s9L&6oUGn1?DLnZKVjv44C9xk-1 ztJb;gcGIzSQl2k$PS~|VO@Cp-c$o&OcG;R3&neP-UnA{5 zNg1T^_1hgs=C8attZ&!9IT>-s{69htX6*}$!Mo2vT>PzkwXHOmHAMMK4UI740(1us zqW}NC2Oe_BS$d4(E7Qx$sW*69Ra^4p_X)vMVnD6v3#~mbD*c>Ce>nA7m<$`MM_Da( zb&S4C=Q`;zz?N@y=K0bL#->|9n(0R_x-C|6i=*rfA&coy*Xr3hnwc@Qi)yx-pyrrZ zUV$${zHV}fHu@}Kbep}c6TT6Pv2}0Lmk9GArX@UXv!Qn?T2fA4ZIpEtDHe_rHFR_ z$kqJF_VG~-*A^K6;Ml$F7Igq2I2G}yuBR69v%f|%DrB9KyDsB{eJLkWlqg5H0&NAh zWUO~E8ts1TXj0fNh~!9OHKzxZHAkB4caVzP$OH>u--x7V&evlrZdP{L#_cT|-zb?1 zq$~+TqG9ZGU58$JO;q{td+AbkW?&Sk*rlZ&C&&VeUYd}cVyhyhy%rKA^yzVI!C9a= zJKXI^<0f5eFM-ssSGhQ@WV<-3OC+cB3LU8EnL5j5!kz>Cl4$|@^hV$+1Naent* z)mqc^B|hDe(eVsJ0NWhj@zET)!CCs}nnS_oz$JnTco?{8(vK}q%S^^tvVs3nnBj!<~Fhlj%+YZOYPf%yzTUv)bpUF)}KNe3SOT~+t$9VK_07F9UIkw%80a)TV66S_qR>W zH*FS+iFbFsjBE!hl3O}xMqJz7i0}5p(8Sp<^{bP#1n2~)+{{^PIb#uYdLr~}Z?0IV zJ$=Jund^*0yZsNjdGh%5M195-v-g5hf?!_t=I)ncVykv2bK2Ivx}Qy5{7Ed2srvTz*V%DotB4>aU?3WCP1jb(xJdc#kV2j1?6%aJdr z?R6u9_giC^rb>%JYOM`fu7JKklug?$Y|<>&U~q{k<`?%f!DNSCyO8Imtaijd9+jHj zKk)!yB+lk-CZw4vB}EmKr(RO4um-(ujrbB-U0i}e1V2h#J!`Nj`>i{LrZbT>-fnp# zwb)!!aBmdy6@o}@a+z546w4ipE9SaB-|tztB8@S zh6hgPy_{*)H_cIk!ez9e`q7_&9X7T6`jKJ#&uhUv03^wRFctK9y^8xI<`dStU?$=P zOeP`(zllzZ69X+g`2qcf=9{KBTI*N-jjW}@p64kp@Ut2ONvIt3tIE4ahXUQZL_-+9 zB~Nekr{ZMWAysI#KJ8g%>-E~DQO(3S-V|vQ0z+hE2 zG)^a9=7CuY9ZDlv!+oe~a=>QD3kUa7M)i!bE&ReYOT~UH0;*>qGFQ%GtF#PjUC)y$ z%PrFRz(UUC<*Ix@K(PJWnPI-w5MT2_`!A+_r;!(sF>;@gACTj@vnir?zG^LKZx)2e zOEk_Uw;uqa8nfX^kY~aCu^GpybGCW4b5eWHYCZNONFEi<)mOO_&i^=1{gkGdwRL4L zG(vAl-b%Y#6&O=-pT=Z`^VM1uE^2S1!3=8SSk5sXJJUW`o+9M}pi`hfFI~BWR7lJ3pMH&LYz@$bPp zD*BTbFYsyz$&Ci9@{pUM5@mom;E6#(tLJe4LzrRMx8)7!FfmKBHwrSM!Q^4-__bMg zw7}5WQ#1>s{-j`}(_;p@Uz`AMn-llDk`tzL_5nO+-Tb(1@ZEhx!szPyVfiR;gvf-Qj6z> zS3HWSwzh$>CJnHYn&kYn;Vaib8uojBoVAp8H(Y2aqct(#a*uV)%f1CsSAVEFM>Z5L zgzffAQZ@fG>Rr7jL1^v9=$QrJKT*sMhhg41zvhB?)Y=94#D&jL(SE>bu*$~fbniu1 zu0*uoEI#$8|Biv%*}eZ4v+v1n%v25_6+<$~{^>|)C|?WR`_*hg7E6v7s@}^2RWw{l z?pY=xVK3Ns2FH(^zo!tP<71sEZ$>FLSbWybvw=gprXbz{@*|GCN8Y~>1YD(F*58dO zHfW%<2FNp_@qx;dv!van=S55ZM4|^xzm#)JSU&&|g3-_a0}}=?oU(Tce^auT=;YFN zW)%;#lnFC`X93J60GW?y>--L=wB9b^t%CJpu9)a4w@3hs7&9U&8}-+`;|G-QnDa|& z+E7FE0+EQ%YZ=7z`i`!*K&gIRaY@5LJSLv@uh7qIjk3$rMX~)Udw1dQOUah z@FRd@pmW)>?8jo(UOoO63ZMqw3)-X0%$0%4v7dl>z*!4si00+k#GKmTb>9^HE$_l2 zuBq+)xHg(q*qT>a$^xG2uXT=VJB*@*c_(n#Ru>EqN`ARZrEw$De>Yp`5u$SE(Ri~7C$ z6CuW0=AI20FWE@1mFmQ{K)QJDS{Lo`rcP!V{Jw;wRuC0NeuZ?x3;!1MSbRorO8^a3Ga=9e1 z^$Upg&i<85W7&-##YYv#HVLHsQ@>$CKeImFULpo25Qp|=WaT2_?v;2h)O4_w2;34L z-Avz=CvqOeXsnuX?iwuF;$F?u>Kl#AC4OUkpZjSUpSy?0VC&Fbb`6VK;~=emrFHm0zl$eH)+vIox{ zPoWGbV;pc#>SGR9Qq^0+c17(03-784EP^qAbnY;G5mVOE1^4x6Ql{`V5m83s=NNIb zp>hw5Mj58B1)@Vcf~4B#|eBnvnlN6iAyFV6u?wKY<&k7 z*aB9%H*5v?t?vh-;GI|b&7}F;0=6lU5-)_$sqs8F%C2}>{-cGnUaog*ECK4W3WI>w z^!E$%`IUn14Xh;k_zwr(o`iqOeBlM)5rc9gM9H$mtoe*c3?ROlke9r+S z^A5#d5l?$M&7uxYo<5L901HP(Z-+9P(JqDef*V2^ix}z8?&?)bw|6>lXBi%_9-IE? z3{4mEpH6VN;o|YSBQn)g{701id*phKt+(Y~ksYf?;(}vp?Bfrlc)z0Pqx4m>z?zPP zYSUQumtAvdJOaoSW39!I%a*fK=U91wtsqUKdkUV&OL|~pZ~*1mzBY5Ve~p}gD6(v5 zD3WM#YPrF&VD8u+N|7?QRTb4#a*(~M@7V71`OWO}Vq2IPWF`Qkfb~XMEtcO_S;SST z3CCG@N-L=P0v58rGUow>QEyiXBDm8%f0wr@ZOi4g!7{UGNGRj}6^y0GZWNkEzj`Ok z;?DYce7pO+ijODXA+b9KJm5Zh)p)fIpz?c*v*h#7j{;vccC1b_A^lhUnTxpHh*_*P zEiJdCz)wg-(J8%k_N+l!u{S9KFnA0 zs|9+j)C4_iT_KP2-l~@4!)9%9J<7~JqxZ6)VJ{%-Z1B1%e1y7+i7ym4d`gs;R{vC@VZDTYr>cmVdjy7TTq= zsGYnS6(bbUaNX?}?w)XzJ|9T3i$U5SL2JkZGnj$db4U0}60iVBLIgZ06IEW6@D1~n75B1s-qo2Tcz<&XJivfqxQqQ?ZZ6b)P#r3O@3AAt}OyPz49{w%| zaLsA{OWPbUXNrf{JUT2zWA(K7HL(yo<#J}i@wLC>BD6E@)Lz@Rop;X>{b>X3M9_Mr zy_)sCQ9ZlnIIX6*$WrzP1>t$Riz9NU3btotq9m8P-lh1s^r%v^Q;sLNbXs_y?|YY0 z;|lzNeqoJTHjp%U7_-u)>b2gRV&J5Cu3nhN|EI~WA&l#p)Pn*FKDqD``GE}XNE=<`{}%4I-y5jR1t+R7r+@mT z*%@g$!Yrb={J87IEIl#;$%;oHIV|T+{%-d{O~si2BgZ4SP_fU2hQ152RE=ONFa#vKaPoHB zD`tiSD`h*yB0gHK3=cq@cXAHiKW>5yPu2_=j*9VrrVb1m!ijdx^iq=kYTV8!QWPpY zBUJ7)e640nwsvb5*aTzE{#ys6OTQb<3Ezo-x8(MIR{eCu(`#l}(M+&pk$k*Fze)8K zZt9Zums4X}t3o}DZwv}C*^SO>?I-41rJFe|tRBOSK6o!Kq7h6{uOLA(IriGJz&tu} z0h7E9y@|Yk_&85bv^Ofj=xG`!%Kr(F(HI8Fn};X66k2=#C8#1#)%k*`95f=OQzE}* zrl&-;tS{c2K0TZ)zsj$H<@H^ee)0YSKYBA-{GIfB9WLGY3u+tz(<6HF{zEhc_;9w| zQ()0G^-_*o??|!}t!1+^dnrg=2lg#V(ongvGSjBxO{W{#nl3&Ky`$=weRhcBuzlqu z3bC&PHyAJ+m}U!9=agwouX)6Qjq5g1ieU-YGkrT%e}%=8Q(orVIZz?@TZPfCFG_L< zoqGzU#Y(`jK9)_zxljiAA7KH`joR>nd5oeHs-A*(?@wuev)wRL6w8P||8$xZ?|lKq z#5JlD8p2)bnh1`m88&(b7!13RZL{AF(AIlL`PQPcrk2AhMAiV>C0cLwYtz(1T(7Pv zfhz*FBnd!s3bS^(k;pm{P zAL*zEv;i*qR~Zz~eZ3T)P<_7#2v?{TuvUz)mET}$Why@18K_j-T70zQhBf~bCBo|} zW`DHNC2VLGGoP;Itm0+A0MKJJjmyehM$IDtBT?=$HOPKcA@NIF?0ErW?X8NtN8=EJ zs+*QK2Kvkg{wY$-!9B8JSQs6M9h$99E~*q>W%y8Lm&eQkeM{= zi59(}Z78pyncXT)QlT?H#}BD0_UW;mlCdF-hz;KDFO_KmKdchNgcS*fdqJm!WxHU)HO*)3g__PRluFxbHdD-DIsOlq=Me+r{0Xd%T7C{5n#iNIGI@>E_< z?*1q{pe&LKklCqtt20#o4BWVVZ?Z4vH|P}Lg8p5|)&CELoK{kgCOI7LP1RCQTy&0* z{V8D>={N#W_n}4RovSl0-Ogdfg=9={D}3_F!aY@-Mkj zo(uE;o!)V;J&LvqvPo9ikmZ*TSUmpd0dK@i89dTh|4Zm2g+~CX+?vq4od(u!5@drL zx|?|pLTg;mOCvE~YX1r+;Wvk#S~?)ZGbYZ`NI^hIU&qh48vI}Ro3!VS889AP>jtbs zTJYfha%;7J+?zy!Rho$YlUh$>bBfHrTnDl=T@t`iMxnjk&aXcDN8YQr9X%sKR{;Es zP(QnL@exgyH^21m4b-3}Y9O=p`@?5jYNfuEOy3to8`mm*{U0NK88?4ju8(M!>qwW+ z_8et7uXY4t%azFq<#gmkrU&yZHT$==ZFDt!ffqD$b(LEu%(98uH%3rqJ_fSzDr>5T ze%(xMsAiN9T`b#%K8AosMNT_^Z2j2qqJ0zS^&Ly+Vx_J%qHRA{kCzF6CqDrALugOW zX|W$m#zP*Xw?K_=PX)=0s#|Ff`&=3`oB9B-u$=;SpH)ikoxsAP%aVR+e`Fr zmJ^K3CA=!fp~B6Be_U$_G7`lee8V)Ikh6-*Y)4Xhl52V)&igikJy zS_skwKLiL8PyNg+aq|c1!4H3oe3pGH?IPOwIYn&A*kRu?Xe{ikuIz&YY=G;2U_hq# ziq^XxQO!|~!>Bv4`NNWH4e3U+Sq}R&G5eYUta>QqJ;hpL1u3hIPjOF zmj+b1Q)G8(5+MpM%Jx{n3$#qzF`P0Sf#Dynp?u;+03%Z zkpfYLKe4KrKFpEOtOxsbT?N|NqQQ-hvuy`gOXsdob30BnURC*xRRK)4VLRm_!8A*; zG1jFILj77+Isk~8dFGTIQ*(zO3cuHWk19D-?P2EvHD7DAwE4&hQmd$A$P;NrEbROy zv88Wz^=l96^Rlr-6#^W#nQu2J4E9t2{?-t89iV##Y?BixPnNs1gyrVNL74~a@~_Zx zq%4@Vte*^c?AM!@q^lMo7le6R0RXv30+7*K0|=)1;^PcfKg|zRzZjBW337=vH<$qK zlSLpfBBup z5Hl@&8-`BD{RUos%WMQZSeH%FfBPM`uj`IcA^<}cV1+sFPP-m6JHGsrWYUDL#zqdB6PT{d^h|TSEo}e%=(feRZYZk}&buqdvhKEQ%YzL78Tq`Gq_Y~t?LRWQ$|!cLH%rXc^=gU?AK);B z?TM$5)%#RA1YGZ2lgNt0^SQt)*s`R?cq;)XU#g<~9&e3H-O8_4vgA0aRUxq)NPCW$nsV=$sN%{u&Xo&+COs5q+8xKG+ zhgs(WFyO4c|5FjeAB6WCfg3>uuQqsrrnI!D>@-)1$*{^~eTI=?T=1zzna@)DKsc|N z6>aDUVK)jU=wwU<^X`o;he-e#WmPnXQW1J8dyL0x0akevRBl^f78>%$eP5x`NMRB* zyR^X=B}3Jallzl$9?1?Y+{*eFDxMgMQUWUO{@Eg`7^J)1-)p8s*n9}_t{$RLV%xdE zBU7BEjpxHbls#2DPEbs{(Ve1iU_aL8gGz4bzQ_imK8F92bWAm}`qp(zk9lmYY;SJn zo#v=COGr^G_>vn_;=;qsY!mIK^jA1T(;lU@y5Qiu0o}fUOn=Cy#|1F7rEAW?a9t*wmApe20mI_w#Z8+g9YrKz}i5N=W z8l_aTzMH905IoQy2~)SBr01M=-XF9#B-d2j8ymitB9Jx}Jh8ZQwc?0nyPA*Z);p;g z%$@{tjc|qf#WH0zOopfvU-=+tE3Wc^vq#ejhoFL;l`D=@T5$M%-)r(hHL(ttRrDKQ z_+AH`4s4MUcvMcsJ?Vq|iLI()+aAfVew>=?MA?#sx_q_=(enlff7$V$V zUmympCnPL|HGv2V{!M_)TYYQ&%gB24V1Anb@pV@_PRHSYgHOP@j7EvJV7#72;q?Q$ z>RK=DhlhEkV@22>w_0N|g|cK?`Yc)p@uumfK%hm$#91Is#1!ha`mq)!Oq1rUV*Z^R z2u9xr+j#o`Sg?!D30l008ULHaVgqq+z7OC8zZCs;S6*_rH8Mrc*<;gt#n6XLXqu{F z$@H!br+F41^52pe(;~}r2OtitGgW>Ba;3NLGaUVv2O;;D~TG^Lm z6LUo7)~~^&?a1uWS{v*EWm&SzYfd~Y@9q0Po@&*0@WbMZ!7Q1l`y(iW;nJEDK>D=upj z_U8Rj*CMp87F;`}DkSsyhmMTiGZ}7~2QOm&h-QEL=8W01Kd%Ryo4$EGa)v9af0H?@ z``W^7`m^Y)S+|SO)!M18%4GM6+MR5L<^+i?209^F@Omd3`18&IUs@JCm-h z2EM9T%9z>ug%>5O_~&IPjnS>F3){5)(vmds_U8@Fl7RE7qY5&}4FoFGlDr~&D6Dd0uVH0r5x?jON+|9kJUi!je8O|isAnsPOxsKl3sE=!cqQSctca{#BG>pmNla?O#FKBeiiYd^I9MGV>oOyEYzCPa!cx(qkRq{w|Yi< zjWSLv@dVBDN8xYz*K|HRNV{fxt)pCw`vug5RG<4Nm-s)H=H!Z!+4h=w11peUvkw~^ zx2r$(ZqAb!BiznYMAtNSR3Ugn4MycsbK^axJc!RVcZ(+-z03mdOtU@w5$m%UJE@w>a{74f zGnybkwQ$L;aO=3L6+hr@4nO$x6j7^pCi|eSk|sK}DP;`bPap9c@ta*a;^_Z)i-v)0 zT`=0X02Y5`BL$(Q%0ocMv_lMuuk5k(gE=$Wp4c! z^U4fe54rWFK*|#p9a~pGhiAyQw;hPpdQ6>%B0D_=%4OTtw;AH|Hm0kHGo@oTvgEs) z4Ntl*&$}0lpokjkOP{(1g^grOtB9|p$g(h7BJiN5mcE~QGI!I%@Lf@|w@0}BV2EacEyU<&Uu^0g;Q_skMqur@3x}EmoBt6Q1YUyuQPL#x6%~-A@A{b-I!-XDiq*Y!+nn*=`!-4_HiO zDmqh~jigDzRx-eb;XUHr#Of_uVTV;h5(3L9+ft%e;UQ`uZ;(6l171Lph(QD+DuA^w z3$)x8S#RA0rKKKdfKH02cIl1GAP$p02Fq)!z0iRQnCDXi`R9r5uKW5{HV- zAXvq+_#x>!qQcOjg)f1&j{5OhR7Ll!PQ80-)A!_?vUXlvKSGBT;k`$no?G@xS0I>1 zgb}ND%|5=O;=zg)-vxEOaGx6Iu0Uq9cv|{fIy$fjKo_!9x<~>~ofsSNPh?3{`0p2$qR6{Ua>wCH*>8S6a~ zn=~E{`6z3*usnfPm?n2;$~gzJFGjcED%W>EOr7q-e(GrkWFxR~xyizC`(2ElH;0_Zr>h+Vb_N#j_Iw(Ya-OF+p{U50bH@tjESX_otn|Nb{J58?G^y$rgWjp1bf0~6_Q za*9ElKo2!^4%CphbUI@x-0mgxjm}4I;SO_UUyAl=JJHKUbVYQo_6$o2DXz|43`Y&g zt#BK|&4}3X#T?Fd#`#B`Q8JeAW~=W{JUfVwbh&`x?&QF59W(?q6!3Hmd}-2*=VY(A zqmv>H=knyXwBM+(gtRwtqc8{JV9^G9>HvNx!xPu^eJvw-yX2x=No=nD zj0IzzC)yF*?2dqp2diN4j$A0jY%^QgXj86qw@H`F>own73FGa>H&Qdb zg47~i1|E4xHiPDPJ8wanp)q49BJhvNSwz8_IJcN%M|njfm~ExuB3w>_GWB+#sRk?6 zl%jI#SXqUR<`L%OfuhaBw)N&S+0VbgUNuzm=i-cE{vQu_!w#jS-!HgchR~De=V`e- znl+$jpq1iyt~jppJ=q@s^g(r{Py!5iH>;@LQ#;-$X>tGfW73&%ux57=r9_C`$u*qb zlFF9t(Q!X^lW_JOntFQk78pOj(yQpQ*RDAKaVxhzLcB=I!UFe6JPeO-@EJ0EH{l{& z*O}PBW~nbfY9Vj@^?M{@QQ6h1KqDK=#-)$^df9FJSiF5mj_qn6e?cQI6WgsTB=JDL z6!l}D8Ob5OwVK~J&~&nDkP%2yA0xTF=8LH)D22~m;UV4SWHSwVgYmE89za66pT{O9 zerzaYfO2v#xQ$=f`ZnAE{QNzW@!~FLTu>kX@ba8e$DorJXyAJi5+T-eR+nyAbb1HPFee zI&vz+;?L?%0cUB;hVOPa>))gLEO6VxaU_tUuM6B*WJ}&b3A4KzZ2P zB41Sk-R-+@!C2r|fM6uXacV;;zh8WFswL!raBqkOKu3!~}LozU# zZ91F>*L9~FBT%%zn8!)IClO(p+4aKlo;+nCe@WcDwUqG49w-9Fr_ctJ&oVpD!?H2p zRX{{Q#b4x%swme5THXx-SMNDR=eD*6rcT<;Shsf6 zsk%rz`G6Q+Y3(5j^d1KC&C*{N=~5odb({17!Pz~)L>PvrM7maL<{>5tQP?`&E8NF< zO!|baam^iLST9if#nL`A`9_fpuSd+KfjHJ)CV5|$Vzei)Q&``!!KbW{Ks5B)S1JB(tJag*wKl&te%IO4&K>H9Mqipx9 zLn+mL&K`tK;mnm2rRO#-?Ve0 zF`8C$CJF}f6gJY`l~Tu}Th)Up(t2TzMzwZk!wD$Mup6G9iF*6@O$nByX}av1hH2C- z(yJjwv6G%Br9C<|ObOd}#~KRh8Ov-|f1o&PEt^;qdbYkHNJ#4+C5CQREf)Q=XJgnX zQ)Qr;cb3f4K&~Pjs22+6+<_&a;f>pwJQl|#xaWbkNw-D~l7DL_xNQ`PIc!hystXOx zImo_KJ?=5#`v=N^=Q#fIYEYHn%5H6p+|QQ2{qzrRYzieS<%!J!XJsdQReiEMdVg)M z0ZUoJz-<0nUvpbb8Lp_U8UigNZYTB1yIaYxPfU1okuxX5_tzpu;4_sqRif-ESZLxL^f`1nUNK$__SY zu)B&iPZBdEZ%oM?`YZ1Gw?)?&RUNbGrOo{I&M$o}b~{~b9Jd5(HwD4sZ%#|e#))@+ zkjX4=c?^gI4_C4snUaBd3s5uY$deWt&ekUegNFj7L>*STAEMZ#rFnTwdIATA-nr^+ z-j)28oy6E20Kae!1);w0Q@|dv#9B7#RsoZRuPF&S+5cC^QqSdp1xGk+``K6{nmY`aFp}Z#;sbe zMlzxx386T=-p6Ra*;!lCS`zE(Kyp~QI%G;%!a0$gDuU{ba7$x)4x$bO2SRijPm7Af zD)F)P=`1J@m#%lC`bcTwb(_~3JlUzmYCJmg3ajDHq;%@Sa7GHk-(L(g)JkX-R3PMb z?FWcfk9v>xC6{_RA5|9n3;c5^v}Gvo_5?tK+M6#Kh1Op24U%u?Di8$v%+sfuAbBSj zck{rONg58V?)R>UKvXO;pJ$vFb(GPabaUuVCB-_x!4(!F zcW5;<7Cg)RDlGqbY^f=JZq*QoB{?;=SjdfJ(^mMxFxFtoxR{2@)EE`ORBxqLQ_f5+ zY8w`;6mgR%QDGjb!c<;=qPDY+q4fk7So1ih$YGU4uU0zb%>nYGste|Hw6V8opiw1&1 z4N}N(;x31jYl|-t>RVi~QrIRz%dpj)+`IGjSEbIQ^J1IjgSfZM(fll8=eOo;ouJD} zs|qrrF0QBAZY!_vnVws|s8Cr2O_pfjSe2C}Sh87vu_XkUR(ZU4eAVZ>-OVYpaxJzY z4G8DsZs|~(T$I5&xW{S(=ONJTjMr<8s^Sa0lD0dW1|pWjnz1Cd`nb)9^27YB`uBJY zW+|l4Lz!OvjLwTZ?CB|AFq%dcMiu-;CIR3t|dY~mT^t^VcS=R)Z^M#78k z#gCKm?Je$6QUsTH>PZy$6q6GWi*{-eSWlOy{8m%ZxMKi;pSuF$1z8Fa?7U_yUULmZ z>WQo{^@!GAVQ;h*)UhC6^OD~A zxh~VtnsA+LGIuZPA;R*`yobL0pp~$LW2<4}EbNj@DVxF#oD4zA=lZ`o81oZ@37f){T&0XlG~fQB;!qRh|J} zmtO4j!3@>mbNi?8s6|x`KPRrSv@Wzx# zgIz?W4(Ucpk*FII!-M4uz|*kAUV@2?)|HtArghX{≻33GZ^;gSceUXKs804&)~4 zkRKtoolP}3u|AqOJ(udev~pIAFkG<~nj`%(lY12k6auf6QIsPS*uT#H&pZc~v}cK1 z;6Fb^=y4n~hF?=Ri zx_rXpBvx-37OJuH6NSV2(Jlory{`rbY4_$l`5{D1SiZO`TU)YHWjGVG+v>3p;%53u z4J)&9U3cO#_OtyK`5EB5#4Fh}M&^;eO_3Xt$vX>=f6$h)pcgo7%D<;Cpjmz+-_9=~ zuzAKvBVcK43SwdS_*(~s_eHVmUs#HctpDgSA$7Pe*o(f`8?t-rvW5$Jx`ZDNUWuYX zBOK6VA6Y9nvjc>`VF^BIQj&+@+@xhcDfw>p3;HW%A#YT?b(J=QG<+>HDZvR_3zqh+ zzj|EnQkmVvb}Q4{BQ;Doxu)+wA8OxckjnB_LZd5$hY%rSKbk~_O*jx*_Y7<*7c&TZ zS-ba7Y$)X)ji>KUvVGg40on>?I-#fUR{&^X<`^wtwqdRhRb>#oVtjXvkcQ9#YgPfn2DdN0UowzsB%HTWpxfy2t5`U{WCV7s zQx{z;c{3T^U6&`UT2&-?ALESa(Brop=&$d2)oLs~-um(C!T;V(WOyp4MJ@;E{l;f= z{%!cIBO?5+A-P~De6!z^AxAxwvuPmH=Y5{Z^OH41hSQ!4sQ@zcjT}y4`3{_N;GAnP zxTQ%=<&X`gNCvJS7&_@K^g1KGozmD<9HBnp%LzMYJPR__sWn?LIj{3@s<4bzJ1_Zp zf#ueIt^(*O*3y32{i^+7g8XNX?bcpRs{y3H%?+?eDy*Ugst52m3*s4W&Pm)A3g`Tp zl9;U{(#CXQ6b8olJ_)OiCi(KU-JHc>HO*`ixNNXL9>O`bXG~(WnVNTRypB`byhp|C~WW>{4%u#Y;?xZm$=9xSr)tzzxDL^!^o`Dx+(l z?)FylfdY}92k$L%#+FiRhdo#H<_icIuZ4md`BFP4%chv&>b&z;;LYQ5xwnKRhFzLi zCq}o9aXqbR^tW9;K^ixhCy%1q+ve@IO=FQ&YCyZ|vlvq2t!W<&OiEph&cs2(Ck#FY zu1&gfO%i!uuj^KCJA`G8yV-52u1Ft4pAA*7X&u=*UU!r;;JH%dd$b%&6Qq_Y&HOr@ zrKoXB7$cU(r~F*6J7t#|(DAD^UOhs$2e2=@Vwgl_C;ySJ!6qtqq*D?F+b`q)7jx$s z)nvN%dmTp5p{a}pq^T&VC@~-SNhk^J+z*WV-ETW*oweTC=X{wDqiYE%&wXG2^1GT79vV=G$0qsACC*}T z5$(_t-iiL*9=F0g!Z)EIjXN7TWe|!tj*IP>(7nwk-WjRhH8#{v{ee#Jq5t}#Ms0dA zmV&w4jnyKJtlmReANPCCi}_97yQvNb#!vKE7vs2FI9oI9oO*B~a_Wnx*^QP#SG=P5 zv>ZI_9Cj^(?~#2~$|DoCA~BQog{#9784BcI)Kx2Q#7xiiywFbSwg6YZrF~#-qh=M- z&ftavmC6RC?~Clpx|j`8D=L;f;i`e|M(eUX*FL}G3Ns%$L2h-&)wrKcx^LYiVD_*m#4 zyhWLK`584YnH+^sQK}%GBf60B+zq{CJH3ahfJL?3T&=cuf27icIWU}p%pcKa9Utxr z0%6TucNWIm1U`AJCR{(uuuuhvq^vN7WOPg(wJIOmg@<+2I6rjZ^?x+GI@2GtwFqx@ zxM-=0Tz&R7oIL4qG+X&)meIk(VSHm2p*gjJCA7oiD&8Q1f@`Te{Fk3Lx69|z=&%H( z*kM-0I$O6nfo^pv4uxvgDB`*Q^a>mgcXSKmQFhs!5@?>et3bj1uHUTMBOMt=LnLM4 zO$p>m>z`NVlwfzwR_Di#&$E9i!PtIT`f430N;;7ReU8^J4-$Dv)N^#dV70F@oKzo+ zeo+$_iz4bC;$;e26}=nCk<7}Gh@N)vrrn2sbFUVux}uQAh3v^Q+aJJ<-1sKkZPBC? zmw=U4a2@#+wwfpDazh>76GZKyw~8gAbHEU9-NC(nYbI_Bd+3GlCM#(RLsZ$KGki+& zt_4|bZjTm)d<*IYQ0^oi5MhVSI8SouTiu{=AU#=G$_VZSJ}`L=7dKVDNm5!Orw)^e zUHMMd?(_(L1ik18&`kyeVlkaPqPN%Ts{kmQ`-3`DN}nXF6s6qt672!*Kg`kgC(%$0c}bDbWW(e=$v#P`P`f z`cU@Fld3)OkfG!?o^I7{AM#)(s;J9mBJk!H?)!Iva9BN`Lkp2oG$hhLi%)5>p-E`) zwY}YcXM2Nlni~CJ#asBr&MgBaIQ9HV+tJRDxaE^pNX0X1J zxl~7XOJ$cm?&RD)st=8e#YZG=D^=sQ?e+n)G$b>c&-%5dQ+JPxqc%oTZ%d*`Pw}gS z-#2~mYLEQ@HXzs;*);`uSq-?eBkZdF^7A(8SEil;jf$=N@4dW@H>An2Gcm%$BwUmZ zt@43;s9|G|(#HZe`4O?cP3}h9Vp(&kX|^SAD6dj&`bcmPpYRyX9-$8I)**^R9`x#m zTP4}v^lkr{woCjISUZ5vY2-ZPo_}5?mVd?zeqdS|E%Lbe2KJ6 z)m+Nf5`J!pKJ0*?iOQEwxbd*U<3wv$~)9T<2kPIj-G9Md?U zDY4@`+jFh{`omE3BE&8ba^ulFWwRGejV`QDgx4)aYLVA)oUvg`do10GgZp4a0vo($i0&=OZ%W7jxf!IfuOr19BfmDH@DF=I9nZFZ^dzx7o5h7c#4CKw;nYZocH6fWpY6Bu2xA6J2@LQP9 z_aVwkpZOK+mf~bLQk{K5R|+wE_NCm&I+!kU6w4`S*oX3BG`uo$nGw)AYG>rU#BJ_?Vl zSUuV0McjpNA(W)VJ#t@<>kL;S(aLGd~CM)yK`K&e5*TJOGZ@ zRIul3v&ZsqY^+){7UfPdO0QX`V(Oq4EmfWPwi7#cv$6yH4s&&Si6;5669-a77-YGM|GYvtMfCeU~(;uAIvMyk+ zUbl35_)3ahjT~M9qzW*K3>Bis`rM8L#`+s2FxWT8d;E*yh}FxQ&&$=f-!Bi9vK1;w z+GgK5;&7X?TeDMTc3e$1cl+C$+vO|g^QI;d;a;x~I?npo0NcM#^~Xizt@W0z0oB!V zezQ66xC5}r;1*A2F_IL^YV3iGD+MXU>+dXpTn@L9Mjz5oK85!jychh$2Nn)Vn=|Ci zbBj^R0jU8F;2#^Gerz6#V>BtkVOUn07|fLQr;6G0_uDRsoBO+Vu51c{lFipBIl+#O zOex$xa%{ES>c33oPZfEv#it+*2!Zc;F`tvHT7H?`50jQ5ZjuwT+NEQlAu%WVIaj%M z&u1MIgcHLJ4#186vo6MAh~U%Woql#!glEvmBKaSV9$=q()oYV(#B4@fVy70K$x)7k)s^% zHXEwzY#>CUf}+!5?Ld<|!{>7--QK#csut4LLcJl2rQIeE`A*Es#b`-L?q0uBS^+@1 z=lFk<*^Yl{cXdWCCtJsRJMr?>qfn6XwHj5VUZe4WlE%L~;ax%alixz|qgBshk`fjL zq|iOg@ijwRJ$4Pu`{)LNvqBmJb?IJl$vS-ZREo9d0WYVjPSxte#YVnIJaLwd(nAzq z85>^Zm7T}>z4H#Fk(e|+>zq=yy~oeu!6<0l{)RENpKn2 zcDJ1&ce_jT-@Dr*9THReCAY(jyaLN%XxepEU^>*I3b7xr0mngX$;uJC$%VybiJI7I zwFGH;`ohN|a$S329?O@Itir>&Xn5s;m{|4~n~nfp&#v%KH__!Wj5pk;} zshUvvo+y^LYKc~Po=D@)O@Dfq^BCwz%*bZPo<5lCN30B97*$PGwFxO~TW@!7F06bD z^8t&p&YG8Avb`Iwv<#*ludkR90bG4zZ$|?1!T7%2iI`xIfRkH%X!QlG%u&v;O8f_y zMQWY3=T&jg&^7@-pC};^ot?qeysxO?wC5WQLPupX$VGJ7Hok^q!#JkXYJgvz2}3U> zP#}SyqARXiy@b4{p|x~Fg?~X+i*!^Ecm*QX-x^GF`b55QkH6%1tgC}0{@}SDn#43h z%7>Z@qiQvgj~Wv7!qX$ubIVV+R7O~HfUuuricyfSId`SmBqNtR ze8+r$(2MIh+FX|NZiL5v_usQ+F%*}p@UotUBQR0f=N^NECNqm5joJaNo~jXG%ZJe} zDSnmY)Q~t!*Gz&Lhslq$EZxxy++&^Pw6?D%OnuNKce$>6jK(b^IXzFi2GTw`CbN&8?sa`@0+E&J zmQ9rIHq}d1wVt<%T!uN5bg~k!OO?hhB}gvIEX`OszZP*T#{=Ez+lfig#b~XANgDKg z9~QT!m+SPs=}y!}(@ghIE>B(6qw3r{--GKLCrXy6a!zT7qAKZnOWnk5{#D&aur$YJVLl4qzjpCxaeS zihHojuOq&cj7bTyqz$-#I%EFBA@RO9OqO@4u7e*(NgOp z)y*Hflj%}~{jHg;n|HFr%MVnxmU0LvL?D?t>dI z8ddVci)=MrRIIjt2$OxpSw_^PFCt~^@0|rZ>|5r zr-eFDe=T;U5pkE`o9@R5G9_@<@Gmk*EWS7%|Px8;cK{a;X&A_Aei<3`wU zt)8#eU$4}+-2^8%u3~K|fvb_vY|*Cce72Ei`6#N6_+EFY>3m^h@e#{5Unwzj!q{0& zP9$}C$2Sg+QgN>(Fg3XxP_zz!%%eMg1ub0|UEz$-QCgs6|a z=q$XpyISQ5P9tkexYvZC_r&_wyYUFtOj(5cIkZQ zqylr$#5;!EQv;Jt2F91MdL}a7ygMk*Ii7`?S%`L=Qro@^o2o7EOUSn>GCS)}_4!jO z3z4ZGYPwR~3J!$Xo`W$Am*uZwNzkeAEU%2DAZThHoOPgaUr8oxwFR34n1~YM`F0J;O8Z&?lY3M-uarZDwpwi^i+uD|2%i&?)VXWSekRv z^67LZmmBq_=B=n!Q_MuPl%~p32Il-zl94d0{n7Q5Ve;5-ANXD8RJ`}T!fI`*5DDco zJC|N!?ce45mOhoP^AhJtJMbYgWmZPvf&R_?^EXG%xNTo#XLilEdDiKeeY)_(@|E1U zRaQ1|V-nRdKO*}kAa@!M0H+;je+QCnG`)d!JTkzEr>XDvb*_1qqbCc=PR;kS@&e(G{ivs7(jBogl6g~Q;(prNJ(V@q}+z%uP z`1iQxU(N4}LDywiHiylo{+E&qVC%9sQxBeNR{W}|cKc6?IB*BIz$P%%{U6*L`!{iy zN)oSkvZ$d^x&Ge*(Bw^wKU{bD7O7*}K*hB#?^+zf;LCNmmQ0sdP4~K+Al13b&1u7; zbmuzRb>{E7PMU3YpSx(b>E|KvE|6d-quo*w!`)uJ(v0vhm^&h(Ao}XvZ(O0id}<(i zZax%V-=~zZ^Efi+isgo5+k#$wjgx-q{6grnLP~;MUqg8?C27;yJMfYX-y|Bv>^ZJq z?4(3j-eSVL1W$7Xscxx0)fEP@gbo|^dUAixB*p@+6>2`2r3#z$>?h2!3o`LH;UABU zf8bT#H+)N~*{mSO@1ELM8qCYxw(3c@Aw3wa%aS3Lae5-0HpH;Da@tr0cBQ zeQw@kznoR-Qb!n@{ETL-wmf#DdRy-CX+TDYcCTYWZuCjdRETqeSXwlYq5 z${WiW9=*SZW6n8gZNOh6IaK`$=q)rKQ(`MuEL;yh0DMX~iTSMwvP4dD;R4&|)fx8Z zq}JczvFVLonc7GoF|ti7==cYv!?SoLSU9X^dAZ--{NaS^g%>(44AoC)wPdrAh5}}+ z2l+A2+kkLP@YtB4c#q^OdF=X$!zz0>s+98A^%ggx!t)}6T7Z*wr`Z`Jx5jWS%moz25`#!3D+9Wm*hV-+>EMgLwIUAKi_RkO%mml z`}($9Q7$wt4cG@nFKUwds6|{za-;}SSP@Yv+G!1?ntJ>vP#4u#I*4V(s!}eFga*d* z0=p$!mHdu|XT^}>DsxbYBDOMvV9`fddn5SYHBDj&f(cu`cJ8*Ft}Wa&-^Ol_1v}E8 zVjrgwOCnf&Ht8egrXdVA_z`c4y_BG;-oeF(Tsr8?ymcX&`b;ToRvmIkkMJaJ7_B8r zJ?iETvr~sRvr=yUa8aRAYGTh+qR+vn^I2(kAM-WbIPXz4CZNizGZqtMG>`2rQdsvq zCd2$gm1}3vc657eS*vQ%6@33E=ch!(YlFsPV#z|E>^3tkkq493$x<=%Ya&f$nv^7M zSca6qJtwZsk6rB@xx6~-o^+H2srVat@s{L#v&7_=&fM7^y6SjAm-Ejh81+zmz@(nD z*^gnv-`f}=^uhNq)Ozq39xR_SYM$*09s_d~Z2-v4$-|hO<$^WRY)(IIceffm*d)6= zTob*N0fg&##+XPTcF#VWd5MX~1}$cB10z$BPnY~egjeb!rVV@oCw%VEUx@v}@m&e? zxHkbMtJcEA;WPM`ICr)-3jH!N~n*5#xM zkgiDU@d^)$w88{itEP}q#4By)<3!8oX$Oa5$!PC*hV;Y(VD3H~$mqt7)OLqvDs2Rw ztT}L4?WwxBak|mM;G&Lr;GS`biPg`BrUO7c9~-IwEDT*V-n?wD=Z+7X!4*$P&3cm8^<<+UmCuLWaAjla7n}3@3Sz^XD;;xUz-oR z$sDHOZa$mNX(QPiR|`eJZn^^5!&25TqxIaEfluqkPxEBE{t@Ca4Aj>fz zr61gUYlLNp$C+8NZ|a5<-$W%->!WrzJUgtg@L9;Roqu_C(SA1YmOqJ@v&{SE%b;67 zJy7beXeBt8I=?~BhkLI&ddp2zV&xm9yBgmo$Rocb$kQMBx7p!>b0*6eT8~P45?2md z2f=H5)|xe(p9H=C5G5R7)L5S;5RZxF$Ne6ru0`5kG>a-jq%PU;b&V6+;V7YH%2=Ou z1K#-n;$V>ZwG3RQ7%mv=3t#BqHruK_>agF5$x785@8h-;XgG3qpfIlj?zFz61aLcd z@_Q4WfoOuYj(W0KSWac@76^A}qda%jgeeyk-c;+jM+yW4)S2k18X@118l=}x;yd!fe;C&&9~RkqhP-TI7FHWgI{ zKD+H<1vdw7!W*m9Yui&w0#)egVU5>#aNwGlBbimw>*rT|^+t^M>>lM>_Gh6bD+Et} zg!y_;Km;E9X9OQcOuO63sa-Lj@>7bKYkj-c;HkNfF!p1=XUqs?CqFIkHs4Xiw`|e= zBawna?oaddi`)W0fNEw{<1-_dLM_FeK!>q;T2=1g` z)VCxEwV#h6T>@c2p#`ZNL1Ng|3h@#L?^7xV8mo4`43D!-!QMV-OVmc#o4R7lP?ucJ zM1^K?BskY?GX#l(%q;tYQU3`_2hyLlZV9tP2edr>||4kUsVV$0^hN z=V)Mi+4Fe9Y~7^lTfUhR(x04*3&4j++E1TjlDVxi z;o$T{%j&+O*- zql$wK9`rmjGU1RXWvly~`>#0q@~pkj8U=6+hjD8F(d)Ewf&J)cygIo5&y;P=@-0{f z$B<_3#Y2OIRCeVJgw#Hu^wpoy_jHxuQ9dx6CcKmbLvL3^$aIm z!{(tbWwU@@-V}K(XBbpk$caAd6irW}@`~tR4NUR>!IqgbYGd&+N($W8sJGtB5;gTy zot~45D?WteJDB;C-_8L8uw`5521nAscZtHi7izu4^uCmP7OLb!t|!-P%SpM?$}dSI zD)9W?1*Ab>gU=_Hyt^&zL3vF_38y>#N1;|OMZjcH)GbQN=&H|}Rn^JADO3$2hk}!> z)?$pDQdG1q`0p1zJf;LJJHu zMC87b*6+QV;=LQc6`|znG~bs^TJD?hQmy+jpn*nf9IO~`( zTsY$^g6Y;q#2HY6OP0SD^iBDB?04Eryek6=Q`fDGrMcjeUJsIGkVa1VaLT@JIPlWK zvzOimp`j+PwJ^*$MEZ7onoY~1T)uh}Q6;|*{Gm$za%h4_M?(0m423Qr+&?t$0zSJ7 zDJQfNpL;$YU16K!e*bNJ+|k)I{CvXVM{C)WHx_;_4+xBz@f5RkhN5N|$7`-Ouu|+dem{#H_of*n-6`SxNqFX%5_bKzEu~VK znp9Z%Nsov;W>RrqjUwT8E~Sr~K&ccH7Nh<4%2B_aJE!vLOfmwEvehD;&u%*AiPT`$$clebF9;SalA`vI>b& z$kH~l64KAw$O6Lp%1vWc5v~hC+^JCl7uD(*yZ31v&S-ujpffhb#belNMaCf{&^h&G zzt+~Fcjol6lQF1Im?<6e!V6FGa?5+maaQ-qsyAy?$9zL%AsgAw@lX)Hw+6Ka6<{%~ zeqt^MVWP|IVVljsh{$ZIPt0r0mG@tvrL_ajd0h?0-c@y*y}~YA=_5Xlyiu)o9bWcY zj`*#njgY4yF*wv~bB$O`7i(8!q$wOg(zY?ngU z;talI!bWfrRZ8?7o3VJc@5Go#Fch9VFNG|~f~{W8J||!Jp@#XOLwb8_kXc6ohSy&P zJ-4H<1d5+w2SPE^FxNeRZ4bC-U}SO1$2XEpc{<23DSgE^JMnFz=mS@a4*@n7EiG$ z@%A68BTMJu|8{MCHY9D%s=s%XPL0gH%(JxJhy$ItRG^hi-x3^KXD!=rQtTB#TYZqS zt(zb6di7e#dAnF>qf(~p!Auq>6~CDryxO!4sl^GWF^9NDQoIp z4T0VhabRoJ!}i5yJq)`YDoR(?x|h2x9Ziv9mq|q-U3xxW3q~{7tePbLE>Cf5s!PKU zH~qzXnhtM0;Zz#pTQDmyEM<2#NMm_&?R-l9H27!jXJW=H(uIAM1bb=~8FbCq z%DofxDnJh|uRzp)TVL}o``h)kmc_awTqhIWtva8Ma@vKmK-8|2c~tEH|6zRpXJlZN zp=2=MW&e$>HJ`0{@~`gRg4Pcp2^O-BI_1nJY}CocBb0EV>SC+gA8lS)1T^<1>xSAT z`X~@9-g4;=q-U3418XuWV0SaXfR8M%`puwz@w$juBYk^#0Qtc0vx6!%!qM4It@1n` zH;Wn}RK=(%NKIej`tBdL@h?e=sXbM@D`TqG@$R3hvFhES=M=!?-eQfj1eK+XlN0RY zVn`E1@T{(E)oRQFg9C46N2!{e`*Oyydv$@_Q*a`w%#0^L1EY69pty!;H5e2fkFIW# zR|j5|AS|htam4nisA#{Jr(>uF!stg4t{c%Nf_{-@7r5|$S%@PGZ#0l@8RZ{>ab@d+ z!yAM2jo;UObYA;?ANxljHHZQYh!p@dGit}4Uc^PHL zo*Ce)?W6kYFPwJ2Z(vV(7b5ke3k5MI`f6R}EXxJNW)`l_JizYmdOhh|V;F~mYD3z< z3Ov`9CMN1s>-@4tl(=cc*u`}$;IpN9()PW1vd}sxRC^~QzPKHzNKKhue9+7Q@iP#o zsJuIm&=bwO(f(1)b9X4$T4kYr#%4}#SKFenW%PEUBBAk6BHGHO=C-(T7?P!uL{hS` zmjCL@)%&=PuyXWyME?jgP-78oVkF zioIFJg)sY z)q<97o!D=Rv3J58x4+Jc_TzCWbUw7G(?U4r*f5tgpZMDH@$o(>#$}sFXpjRy zBpdT9)E?`$K*O~h^M?K9voQ5l6~{vDs?`|tZ4wV&jPo}0+^Dts?m)PR7PNTHvl_rah5*XUxEUG=Ir_|SCKK0HFk zD+k(08)GOIHElST&M`O9m;L{+T?qz(t5NCSqycR0-R_?+d{-G&A@dsa670R#Ykbzp zo%;)Y_2xj_yrWJVR0GQ1e|8UPMppIf;F9!dOU9tvdJ|q_lcGa(8|wlS!TNsNl^0r4 ziV6g%J&&PktT})ZF>KllzZr3@wg_8@DyL;zwP!0hAm|+-%m>svK$4%Jr4K`2y=@lA z{YR;WlKO+7K?-%(aQ{6G1fFW>Q#3=moSC1fI_pNb<_Wq&o7e5J?euE&3uef>zoc|S zu2`;h{l<9EWgt`FHq})|&ObhHh|d+8G38rpE^Tfxj&NUcp*Q|tL57;&R7Q*M(jeTN zpp8y?yyMCERH?j+a45*G>Vui65rYL@xtfV#?qd~OI|Jg(E{3m<`X<iIzQRL&D5r)5R?)oH?;kS#nJ{QOWjOwiA)vDCasfa@&hi{Gr`& zf?R-xpE*3RH=Rd@a)GofKF`&WFlvdjuE^CttzTMs%{C;yq-|6?K0|(Enm_9tO4zHg zJ0+hcr*9VlL|d2e#i}y*DNs1@aqv1=lc%&&C&10w-;oRRdHyKLtYa1JNRVG?H0*`6 z+HUBs&cfOXc!nzS+l)OS{gl~63GJL`*8R8$`j_udZJ@(S0G1T=^sfP`WZ3Y3MInOR zZbqO!&MFS>_JsJAhq$ra52T|#qcXXYcClAn{YQs|Y6_cAkJF{@-KRmf*+$K63=tZM zG?>5*22aZpaawv(aFCstxU0jA zoD{=1Io)Wv`h878810IxLhT~rMUXk}v7eDNQJe=#-4(Cw=xa>m=kj zU-w7;=zJb7S}UH{IevY?!YgGlkFUB${n<&jB$G^G*hyiF7-M%QgT`Axm` zjjsA@AGf#y3ES#>3iOlBY%OZ=fcs{gfXG>O#?v?WguwE%hLYmj;(n3Ub~C_>B9~;` zED>Xdo!a{7?jm-;{VJT^^NA8KuBuH)3E2M^hNwmRIqqm!#g4phIhSl5^k^32%O`6- zUhY0s1=Y@Qy9AgKfYf*yH8Wib1{h9OEZZ!G!Q+EP>()o z-tG=EXrAj~U=fa6hqYP#NFqx3U&SJ#+k^v7pIt*D-~tpsRDv^=2EZv0n$0m3;+fL) zELyg(Tv@!HTtxduRXEFW$aB-S!7yX>1$Z)2(tH?gAI1~GN~x5v?e6dVPG~|;eD$ep zz~5i42ao?h4l+l$G(4OEt-;xocQycXoe{u8~i8g5LIKfK=1!OR)SPw_UtL~GV@HA`Q5upB6YlT z(f6vJUnj}tWU%DU;r?>sMMpIGfg@U4jOlGKb4&T+bY3dKAOjP8pHDr(uu*1aq?lM} zmGkVgb)wPo7{)rULR{!X7WB$9Xq!` z&vmii={I{B3uF5r9kFnX5Wjl;>7`XrEf9*yD{1o$2p2g4CPI6dQCy{HVwxD&!{aRDi~S`>JKGyw|d$x zKI31Y+*O-Ax>46nJCF`Xu-^<-CI{pM)KG8F53PRZ${q(_)@ zRbh)qZR2**(jrFyjauxP^bx)d?%!i5@3TE+3ujt(ov1#Vd`WVz`(sI96!TUei!~Dd zq06YJ{ph3w*lSmJWaGO8?K}on0kXgC8X79s^-U0+iD2B2tO@Ov`-ZS@?G%#K!W-Qk zQMxwZ)NrnWH`Q6!)22(E(SoZw7%6yK$`Fu!sA&wj>1!m`JvFxY(s4q9ZCCRi^6P(O zY%p^|6Sm%HHj+kAQY$#%xv3)f=2VQl2os1%~$l|7C4?B4&c;}04@;# z*SV{EI#Py{wFUl!R>^|rD>^`VdRytJ>E5j#xFx5-kkc7$(YuD*$KWs`NMT6(R;htwMMA*^Q9dpVUeFI z=h8sKomw)Ec2wb63SM`2+SvXa0oTJh0u7%h(O8s{V|0fcR5Zo;Cg&zfmSKJOZDZn{@(&4SeM4GpL?w}2!iwRU`>HHfCN7B zHk))b0@)f78!kbM&G-|_;9zrW`hKpB7w93kJY$Mujc>J-3=N_=TC~}^#Oc2x=qpeJ zy&EFyRgF6D9Bi{;e`jNv+PMJjB$?Qn{_Vu4oezEgZnHt3CbU`Zzy_8Jt@xk44$d>^ zO@3)VQ=<0-#O;~wRw|4zV<~{in7Gr1y7Wl=Z)}V+hW8>rekb*uez8O9(Qtg*v^U;6 zr{*y5F=c0>iIO$qtG?n&to_wE`LJei>jPTec@BsQ3FJ3A zN|$i}E)$;xV=q3{8>$}}IZ9roVjidxNv(&GcZZVA&O&9vpy530Adqme%vzO&vA~1r zihZ5uHlN411srbLD65we3G1EZoguW{*8e+?+^+?F-lbYnrl*b2Y0j3FDp;bSvkjSh zYc;>_14RX^^{H>jjz2rtFQr>~nsy z)R!cSN96Ed@Qeo+oVD1@80r{3 z&Go#^X5AOfN>8M}sA{IhDAfM>`TF41teGF2RBjg?Ntx>&yKS-p9v7n>53@|esvg9w zrnO5>)E~47i@4Mm)>VHdTX3m5U4-=XkqN2i1AiCms(D|U`*5!dJvb#SLN`%$96w%k zd%uT#?IFqECurLcpgDodqK0vY8qBtO0uM>lgt~f%@yceoMQl{tuUGrqCvJ_3Z;nnp z?94VTzrTI60UV(DRV-*wkQxu${3PmtKWbn`jZ#3Oqo693YiPLufXjx77N@0%B`R6P zMlO=s`=zu(MGI9A1fH}ZOPp0Eu#YVA>6lGw#pXHhW-X*$wpx5I?gZ+R7Mbp|JrCW1 z>;R;<+W6NWjn$ON`@QC9b!HYiB7bD=*x#TCEy_(GSVfrMTWDjbE_FE>CT*9Z zDw9KRq8<{4Zf4fj7ESMQ1mHJKKL6O{g6dy0K51rQ9-`L!I9##vqUGeD*umhuK*7Z;&*m-4vqguF2M_%vb59N)Ap?r&RL+>sjFtb53!e43o8 zwA?7i^~H(kxLt5tDSz1ipGiO(-i$XSRN`9b4!bQo?0=BxY35Tn@oRg!xl&ZZZmpYr z=QugyD<`c_Xi__6=Gz!8`KZ+IhuV2=Z?d-c-3PmzYgoQzEYN{4l##^HqWt`{l>$SpP@VvY_65v$t9`@B9uA0rMov!fsA7qG(^X5u&Q@_42 zZ#|=3y$6N@+_tmjc?W`m^PG@|No+07#Sha!J5`T&FJUAK&%GkHa;SVa_|A?^-#U8V zztZ+1Rm72^WKHI|Xctu;J+a>Y#SJ)`O6$GL5bV-Pvp(rnlTHHhk8G3~ReYId6d5UJ zgiKH>vJi+YL95ZUNUZ^wKR67zqzs*J>kVeL`S#|)`s{+z^%=5aC}Yrcj5L_N0@NZ3 zZ$O#m%;L22MjyAc&RMirDo7H!A@i3{jvuKhG)S@S;KrpYegO>9&TVLk4R5cFD^S6Y z+Ams><0NFCqXMPx{|^7eF23l`zZT`ysKl1LU48HRfc#nIwI?LU(KY5+Lg4*3N-5_P z>#o00D=^JDeCm}Ndc1D2>UBZ==Vv~H-Cl$A&OI4+hreHQeH&zoY*~P>`?R6h;{K(K zWA}fS70XDpY?h($kTW8J8dzTh@ml~EQ*m(UQkC!lwk()6JlA8Tc|8wC-VL~rBitHnSq0C%N&CH#l<<5f z&rystAp_K_9Dgp!X)~uExUJ^+#>>-EZ<)Ip>_W>8t8BYJZ1!AdYgt!RL`WeS+lX{d z>*?cO9yw~;6z^I!29wn$76b<+uC^^+)9ASwvkZ)fLBLE{J&_(FXY3IzunW0cFl&HB zuhctx<<{9ibj1-Kqm4Szp2}PPYU0(8AAy1I8H3|5zQF7*xhFpIugu{W6%#}o_qwDS z28hZ%cxOEldOubWV-+zKuRj1`p`}XlwE3Yj)#+EUB|nYmK8uv2CnQ?6T^dx?1|V_S zxqN3|WFX?z5-DieWJ z7~JM%EOV-3@hN!0b$Ppv@xRKm-{%=i@Cr&#zfl|u43t@-<)jE^Z&HfG3z}Y{~(deyil?jX$~6CjXfTw4L!N23(PW0hjjcQZQu&-!}>Twe;!Q1zsdYB=mJ6e7-S+rE&j1QfX36vQU`P2I1k=Y5)W>TX+IH3nw@oQvL24 zq+f#^_n3HfnNxxV;fnqXxPL#;II|0kHjv)f zAT63~*h&-4bk9GnFC?0&yyR9yt11(v;4JBY{QhN~c-FDc4ZwfUA;Lhzm;gENe?y1k zC;xRU()k+Cu3UT`fQeZ}5q4+^IMI7xu$;k4ZSZ8!ff zXt<`D#1^0pZu5>Y)L|64uR(=~MY_F26P%W;3V~-+HJWX;WOtS*NCYJI`nGMUl?^h% z9m!#Q>FU-WY)$kIDSica#T$L^w<|BTh1hT2F1GFZS>>1Fg3y3b2mm(S)PlaJ#dK>B zicVeOyAvmZ3{BVHIBH(;Z9i^`Uu3z}5LacGAQII9uiq&Id{kbjf_pnMpwLf4m02@K zck}sWpr%f0$pv$U<c(P_o(PnWh|4A|ea^`A@jU3F>We#`sh)ceyvKPzBu z;9IaXXL_b(V_w%#3$l3;ul8~&7+E7Y+}1>--tp8BjSZ41{G$-CK2uZ%M)}$W@C#2_ zHq0EA);Amhd?-M`;whxpOQ%SwOOnM21=^_CvC)VU+{I9IkzRjFVk17w%S*IU|87^1MUuYx~?2> zg%V!1g3MUgC9#RVuf{8Fq^1HgX9WLUc##)ESj4N`?Q{O5a>mkIKhPj>aQ<{3;RJR| zh`v%{o7Z2n1rU@0wc;hAa@?qnV`h8zNy5>^ju(v^kw{k~$F5{vr>nRa0qkOBlTD6% zh7tpHr+*+UMfTFmaf=fyoY!mV%d!zcG?pKje~r$H(`Xwv zX_!3L(qh`8u~89|1Na$fGK}*{H`S_5D-MPg=T|q0yrNx_isN>!u_`JMy*&WH;{;fw zBN2YkKqx`%GUXQ@g46iJAQ2}~Ts-UAxYmy7tSZ-*4Hgt^Ovd zDCI(lShvm0bj*bb#oVcL?9SVh`ySTQ>o>z~WOZMrueYW$Ef+sy9a@win}tRX$Y(e< zo%TwVlcEHVrmQ;eiazDR3>2D2dC({Z`M)Zi@CA@-kyaU}vFo+&4`~bg)PPZ!v?`Ih zpNr+a9F?OoW3yqlG%DA3EGdnmw^{ zWnI91IB$>nd}{5Vd1jBN$MEm)@@LiV^0M4hqMIpwWAW6Wx!!wVt$AneXE@4lPDs}@ zhQH9%)A6TWaFK)law+EzBPVEWjtD@=(!724+7y>)rPYfFzk7|N={zPipcEV#*Zkw= z#Q;7LWd}?55x}2F5+|!lO!`~5ASm_Z1o?z(B5&)-V{~YZhq7%oX=B!2BYr%LyKel7qH#z zMpvI{ttc>uzASJ7v~Dw}wt0?d<~ik@WUou`qYQa5Tti8jBBJ1+=eN8}($!(1+@PS0 zbS|JOJde!AV6Ay_DDQRkNQdH;9f&Cl=~Z?2STlzLKQWKTV<7MpDK)#Q=^c;D=5uz}h}7g{?>WjxlrKK5rLH`1APJ26(s zBN;549L60S{Z1i5LKLDfaw4D*$%Tjl4OAGS5J~LZ6QmF8lD<=js$W{Kp4z@A9ND1| zX|vBkTU`MW6x$S(rLY zq1FU&AXqU7{8ENZlm1Wum)ff(@Fqb|pJPw9UR4NRsun2-x91tf+)8A8-^8Od0Sw(F zexLKKkH<3hZ4k7bJgWv=4Pp`qFn}MD*s>xYQhe#|R{LkLv{^5qiqP~Ce>00+ZCLeT z_(*%tY4?nlda#Lw`$KMTir9GAKk!5IE{W>1f|?J$kWKnOYb&Hl%@*Zd*FZuX*3cae zt7y_n(j@C{{Ys0rMv&$xm$J)9t1ov7N5Y;KOtUjZU%61fRaFtE9gj=b3pTx;pp4`? zGY+th{OyTgcXE3Y#2`{#Q;k#rj-LGB>memNQ{`(Yt#>7In?Y0qnnP|7gD7$E zYWHZwr74>zz#tNIU3cY8&%_|~gBcMex!YTG!RrksUQ+;Sys7H-0dTUlxA%!?v!e{p zGQ2zyFCgp_l`a$05!n7{)#MBArGEsMU;;<P=HVbUt*^4-*SVSD$L7%3{teSPaP?a~HR^8a!XF26pXK^7e2E%5l?8ZtvB!9M zdYh0I&v~|$Of|xhwP&|&ajTH9(B~y^Swfvvd=)h=?1RF%ecoMQHfPX!V0Iag>lDW;p_vS3XpXZl28G(xuH3df2!(!&zeIDG$Wz z+OAb!*zCi9%U+&x*MXN=8jg?Em`w_p)KFCKn5^5)R_P5Fo;)oexfd2y9EncW14~at zWRR4Agx{s6q7f!VAie4N{1Y|9l^>i=D5$9$pjFc1)h|`MG@=@9 z9g;(wT|wR< z9j`SWUry7Xi#!dg?h;hRdr*X@rL=D|3Rokd8tgpl^E)5wt$xvwFbaesO7uDA_p6t{ zWN$gLLMWp2UJ9RQ^Ed9@Y}=LyIz?LkMglImL;NOS)u?+e1IT zdaP>LXsZnA>P#Q$@!Bud@?fO;21cqMWjqs*NU^22S((cY4B^akYM_CW7ZBP1QEcr; zJ(B{0kl|U`y`{-{d11+3jx(bkv$KiC{ZxtsRX&=u-}B`fuA6(j+)LrHfjh0x=11oz znmVL1!!AFPN4LVxV6_GYD^qMpFwL0uESpt1#|*^p8+(447*!59Q#m#MA8u!Ck}ZxN zsS6rW8D=%R0BpbGE8|*KBo4X#na|=7u?oX9TK73Wz3&lP8+|LZB=Q!^)`fX;B2-|j ze|Xca5M+c^l7JI{ivi%tVlyG<^eiiy>Ue7V7e>9d&E{1ZvC<;2>Ig?3#!d7!Xm5kL zYalRpFiq$`a%f5-UAMv9%K*$R0kNS!nH@FV|`o*BOrReDn#$c zbaVd`6IvB8q1W`GfI&fx+9L|?rrR9{L2yJ*mhI`>Ep~jBaRw5!rXV`^@F4q3 z*`{-=Xf^xRCv4R;u_~38nx~6A?K&EYjA#dw2M;IaOO!y1OhMnI>1q~ltSQ)Wq)y*n zHzlz>u2@6kiq=x_fAqwL>y(LZU|d-PC;#pc6V-W%QE>%vm(6I*!9)ws3w%C9pCT~g1?# zk>dxr6u&-dOXQsa0Oy&I#`G@vuWpzp&zN7NZ=;+x>i(|xzuYKiB zoh84FY1mfj;W5(-o{QTO+W$pQGQ8r`KR(iOE-zr z_r<{xan>`~maO4u6`2kS>L^_)eeua-;Nzi1d@-Gq+b+CugkMGrU#OSl7CK>-XqZ*P z9!wGQ3f^9l(s*vEo}48aIx9GZ4gFr2h+jHgLk@Xw!^Oz9pAzLaKuLG ziLWU?IH+G41EeXjUGpE2cQFM3HAV3L6yDR}t7E#`WbL*Rq`?ZI3EQn zlF?!a_R>G}k-n5t&au9Q+o3^MR52{I+CV^dK7Z#_jE+WdK)9}vw_ zpe}Yhl-Ztw{^J75w(gtSZT7fv;S_)lYGOzUtdwqTq(JvrhWO(RkQP*3Jm1iiptSk~ zq~AQRLwM{Is>T*e&Idq&lB;uiUf}1Ch3HE9pdsu|UEgH2_%v^fnO|9K ziG&*CdP#sc_5u1q+rxa;$EU!0RPrx)<~#>jv@|fT11Eu|Lx6@V#3Xq=?FoDW#G)p} z&vcSL*T;C(Hb9t4Rt@Y9dyTd>9a;=^ZcGNVFwx+kvb9t5GSvk~^lEm0t%k(L}I2n#-4r=jio=iR1PolPM2?`2lr$>3`5r7px8)VacLKdv_x zOiDpqMmGxn#!}j*6pA^j%qAkY_>HgVda9{Gvo{W68uQhth&%RRO_^UAO@zJsG>YE@=y%O(b zDj3&7@3f=ut76Z5uR8ZcH9zkz!W&FRWWgSeWVL>gezG&mbK(p0X&;2vrH|Pk&eZw{ z8dK_Inam+eq9^Y8c-uKe-?$Q$`E$rONo$+2NUz}><3?08;vB5jA`)9^u3A~zPM*06 z<}5io(kplc(jEK&u=hF{iivKqDxOrs>bbN-8T{G`o}kOIk5GKM^W5ybJxtqmd1y}` z!r2&*{wOj=)_pR=zg-DsY8_;7@324*N1hZEWojmppg3XT*gvLA9@U(K5qB@FP04p` zod2WQ@ga`ll{I__a0ayD8-6z`PCKrjd}+8^Mk3ERRxJTI3LHBk(?_X)b%eZiJ(7Yf z+I-H6y;PuwC)OO!^KlCS&$f1meB6^;$q0ioj;Qo4JTa)B86O&d3rYESt7Lpv{jP!{r&F<5JJ-0nlV^V|+y84Gx&}7O;B2fkeYIqsoL< z<|P`Q92^hs(k~m`pWn>&k)!SVWl?(euaFF44s^A^`;Oq@|65&(-K~dG3 z0z$D>w&N60i%FJbB?Mmm?GZCu*01gA_B|4UJ4~4No_mm@{-Hh_BQL>Fz3l$g$FAE{ z-l)d2Ho*l|T{m)MGhM*9?`R0r=DHx-bIwFrymun(HvjroVm-7nIB-J-VMc1{zRpsm zApbhHdPZM3x;2K-w^a*ZM(Bq-!Bk>Rl&O_qY zduED{+_EeoM8z1 z@@^Hi`%Ke;TQ_7%QlOKWU)m^@PPnx(9r?=a%eA5Y#l5dRVNoQ;S!1SN3gcu7f_+A{ z0}I0?e37sW++e2Yd!OOuRuh%bo;EGDwDY4ulsnRu;LxjLmxCxX8`dcc;rZo*4`ejo zSW9PdQbsU$KDbo@Hr3mC#>63|C_`)I=#k7jHm1T(I!`}Qv5hS*#CqMv7W535RNtEXLwGfG>(4Hbrv(_2JnN1+nut{(jliM#!&)0R zd^6dv^|0zXTpW;B(Zy}+z9s+vB)oP2!fVa&PQ<^4{{HIqb^JcLm?WA@L@`jI9b?0m zg&mS;O3;J6_o(Og1oOy%FnWdpvm$TP(rhi>UR$ zFS^iR;%0)|Ww7l>zew(@pj0(pYoMuzSHFuC{V-wS#=M+yc%YcPQ^B_rd{3#v*kQYtz;&Xt2w|AnKp zKNXCdDkop-dQRZRSZj}v-apULa6R5AFEvmt{Pm>_A0g!e1y$ukTk=IRn*rI@A5__H zmfyW-h&gu%ngDddM+L)jS*uql9E}iRJ1p-@vw>HVXco>8)k=^mYZJY(PC7koj@FC@&Zez;Z@@pwY`+vBND}Ors z&irs2Cj|Xoh-4;+5%lBh9N}aLB%>A#5xRiV41lI9&%N2^#If~&Isv-5qyVx;jZSFcIr_}kbo3^#DOQJx5zvg=!gn>n$@&yGI2XM zLxXYFVfk*juBH*btt+8u^ce$jALcl9#6}YwW_;d2e(O{!G8vqi2O&a7_qsbily=pO z0Gr6FJn5Qun`1kSmahIae`b?{9gPNzoAraJq}Un2Ek~J7oE}f~_MeHcnX{oo|AxSw zxk`2kcs>x<7A@45IuBAT7azU~U7TU-jrcKYp&>7zRe-2Gq&7M^uj?5k7S)2ns4L7R z2NZ8dz}izhC5oU8%cSSybD>m?C11p>7~vzH#Qg{1Lsj9G3^HxX4aQp+MgYQxPey7( zisv*KwQQ~ci)y46h}S1gj)%LDRNc14gRy+exJyU=3~}Pb(w*v(a{-Zs@)I}HU&;9q zb5%N=6Is$Y=jcm%M-r+pQ*boGIDXnFK%dGXV-qe|T!9m=Tg{@t>|S5da5wPZ*i4i> zos>%-d5tF~}y2&#fH-GRn_(ba%W!u#Q(+$T}24r3;S6e*)Z1!AJYoG#? zW}R03pfBK|s}deOA%L$fpnpu>@NAmX#N1*3ZIMMidI7`)4bMHM??Hou7+au)%W#@G z()7F}PkUnAVq(ggf{Lq3wx;Oegb%+xGGD4$RYqhSQwvFzkF35iio7}B5fQH8PF81M z8=@VTsPsp0o=9VlIx{8eA1>eOn_`Pci*+=06H5rbdV4-*fhj4eYBk(O&$ekT15IB} zYA7zqm!rXuZ?JzEdD|{9%zd2KzmG3^AOc5-iWEZF8j}Zs>yGuak>D8*6TTtys$Y2P zxcWAtUz&(Vi52u$c{|NIP|AL9t2xy8jzWM!pDA}%SHkS`)ZdC zA=4YU6u%&B?Yw3c+*)?wlvAzO`MRtpYTOe1~2M8rcTc~`9!!Pz-&lrMXAd5 zT{35AR&Z_dYhx{35Mv`9g-x{6G#=S2H9&yhC6CT(h{=QoMswh{DJX zAI=|pj==9tWTUrUEKO@2YHrBpYSCj(OGhUkNnV%K2CMVelkP`Z`^(rr3wvoHmW3BG z=-bl$;?`$J|0(T>PJ*RnW>nV$!}b*`x9yj86Lu7G7Ao&*ohiLUby)bjRT;dfdHzUm z_J#bao@mx4&1}Xmg>4(?)?3Ks@5s_$zrK)i8+vpIReG9#g)^(hs$bFyAD{m!CDb0W(*0ehB`} z4>|M{OqmE4K2DhkeZM^&HsNTR7u}~#$Lu1ZLgx|zdgI=Rp0vR+amh*-@GY@Fc?j!(~=iq%b>4P)VL0qMo{N8=bwF{mC`F&?(C z;HrY)qAa0mOZ>(BpH*$Kr0f5ts(lD~c}DzORUm`nwLO>`_%E8UH#khmprv2V z^D_U1%7~xvgL+)hvzk=ODc0>m}8IZwI#-$9GPa3DNQ-ryNiPLHZge_{u z4aOH-uFc=nCdKBXIy}1s`oUh$E(|)#-j$V4nvmNs{Wi6twbo$VZr|t@>w}+*a_wF= zxj36Cd=*!0$>jl8A>5kEWKMPkrg(}FrgGApi-N8ai+BQ=G|%Npwz{ylevKZnp#Hgt z6Bb#vaHoCClqE=P4t$KUwY58DuPUr8E7J#a)k->$;RE^fY;+v1m44N3nyWvFip(aM z^~-$s&-YI97e1@zk66Iu-af*)6d5y{v518Ya^OPN^3qGk^t)G7E0^znwuOW|*|D4A zy=2_A^4Y~9bokw8+XQ6!wOoDduYr1f;>Z1p@uxNSuXl{r*di%H6yAvRb)!QTt(=ZH zP3_O0@%i#z#`}v89`?A2Y&9{AzR!3ouvLApn4$CjRA1z`*srOUN*7pB62FphkZs>@ z>R0kstbw(kWeQ)(|5;=ZdINu=m$&lhBaA|E~da^bsdT0U8B$|OvOJ8){^py z294+G1Jf>RDLoe4Y;>yS0LagvFXiz8vGBK9BuL8aV3R9VKF{y*o>?$7fK@2?{M*kP zn=mB_PH(l3pt{9WaV^&TZ7-{$G2EX~3%d($>z;BfKW*w+&e~0U^ zIiWN$xQ|tN8BjN-B36a?_7DTmG7kLHjF{_BB1~?uaVU>QoOV1X?{mzWrgvOz!{8z7 zlTDix9E)AUuC|^3t>NJ0N=*Ip9++bvuTK9ql^8eGctJ5Jfd@Yai*n?~k(`!mpAuj2 zd%*mPehdh}rJ#M`iN>BPwf=;;tNqX+5@AC|-u}^PmZqj?|FkWvmMDq{i8i#Yxr7)k z@q{%i1X=U@tB<*UdB`TzUG>&u`xQVJGV}8Mb2C7xfheq+rSqT1!m4k}`@e>l;uszi;U zP*j31!8(zL>OIHfP=-FH3iML+3;lcFheQ>5O}9%`Mq-_*?#p@^M)+u!1Fq4OA(0vU zQcyA;7yB_r7t{qAwP)Ox`NNN(2JUr4o)%S$+5EMkKRn+g6T~yr0Kn7B`HL+i_}{PJ zm#P#gIQZ((wkXdj;KxOF^N-*v`7S?`d!YN-l- zS=liWyZ?*_V!bfPQ>@+%O@E!^2N-NK=Vlr#Q2eeqwGY+;@1Rrl<#^=8N|Teg|#aWdEL{bjV66kKbcNZ-U<m z;~C=J#;mP*1709}4;FW^!5LI(h(-OArqoqARmb+6`j|vTb81z+ALDH00k{4grsrw5 zeXY>W(NL27OgWF~A;|mufjh1@ACCV2KGiP%l`yNcp1%UIIZ5TA(1AJkhmy+ThV~Q5 z2~%k^L4AUeaO|u9>INR5#kTYslWJl1UXQb7Gp|`4jkqOwu*NfAX)w4`;i&gg-Ki15 zsk{&imkws{m8-ZScxT16v=K{1aJL3Tn?M{PT`~)!>Q{)|M{&}P-r})ZWbJZzE@o7j zU17?YWSeAq6*04^{Uh50zJXPq%LYcqcYSFU(6(shU^Ug|YG173dLY!@ zJvVkG-x7q$zBHgWeiT;loeCn!^B|x1^)%a6HOlnDO3zL=R$bai(-C}gt$YqHFw7G_ zTAOMl2riqI!UxA;FD`Ap*FmQTJI+>$hJQWXeJ#1`#KS!6ZIN=OI*C}xgYFA;o)yPa zEG4XYwNLxhWv^}R1VPQ3-u*}|-ls+WFvjz!kS*C)PG3k+0XzDpsF6olvox3!_B>4> z=ox!mRwjDN7j%QohbgRi%S-qN*Wr`w4Gth<+Tuo2;1tdjz_gK&{~rZtmlzVA}dZTIy1+2mL$p?XI0(zdQ^&Ad}GU2bF!$YWEQ`GNxQ!Ex%q`LkN$8WV!(Qb!# zD-?rUt8zNRJ5iCOg;Z-AXR|<{;p@H-rb>hH_rQQn2acJ&L?QmvoZuK{lo%~1*3s+!;|8vC)`V5jjm|QEQ3g5?@lj8b9Sz>7DszovH6%D=@2^ly4h5Dqg1R0^m@Cgko*kV z*h$WhX9BxQE z@FNgAPH7T9jrRUz>)HdhF4ZyG*=WfE*32s16}ae7Zj%15r&_2aQ2rEpUH^mZ@*H6R zecIZ92Z~#I)q@txCYyjUikVv}e*FNm17zp48-Mp`?HNqP0`20~?93lggtA_lRTdgN zd@0q2G&+bf?y`MjA4sB6=(=Lu*2H=oN1(S7Tm${jGnK-_wV_So fv3BGbam{$pW)~%d;=`2yU*}Gn{Y5=_?aqGz$zO%R literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_2.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_2.png new file mode 100644 index 0000000000000000000000000000000000000000..bb5439885e0694f3b6f69045e26aa9851bff553e GIT binary patch literal 41770 zcmb@udpwiz|2{4yMOmdt4yz-9Wb&+EGGy?>~$bCl-{4+jUwQQf<@ z3^+JA6@lMp4;=)4o>8*n=HS?8>7cFsP*+>~>_ZRN7Y@#L92}RTyg$g_<$QLsf!!~3 z^RU{jYj0G!0)s!_h=_Z8n$JVZSlc5BanKP=xqU0FWAtQa&*^K$Qzstjc^-SL)ceVE z++b}Jf>)REqte*4_0iqY?N?&hdebJZZ|3>;uPIZko?5P!3xAlXyPBFkXOwm!=J4ej z0zzk9EV`rou)MVY(}ajoB?K0M$|~}lTXYRD*_h@vO56xi$X4v;OgZcB(enJY)1RP= zGPf(_2J&`pxwXxNAs*a|I5u{|fAhyDN%Td&DRoqGP)q6qrW1F5W%@&1x#-A;k>>+X zGw#NueY<(%p!+R(HVjrW!e4Xnbn&z2q6b6sUcSD`r+t`fQuT6&{Cf`fLpPco+!w}F zyfl>${1xdkl#7fDI4}>fwr*dox17`9t=^RAzQ05%Fz7b2i02s`w&L~!sbX3CfM`uar-gmG|O=g_@%<1upI;>cl!kJ&XSRUC`CbXmy}2|x7U zZI|Zo!6<{1xf?_)0^=w!6mTN707sJN)AOAv{z_546pynGt1V=jbDx=n| z*qc_hY(_i#2yW*KZtz*B6tZORPZ7w%FNjl+y)+{>tq)w<)RZ1XuNB80En-$sv{u}b z))VZLS=44W8Oll~B{KJ8MHE;&ICg1ESv8kzULa^h1d88HnE6W1+DZzmF-?$k#xsL3v++D$ zUJNrt{|*XxNlR$oN${M5>C;roi*0tIDutjUkga^|QNhy2+sYd?@mrm(+Y_=IWL z4`HzBc(%G~)i`gJb#1X*G`;BsskGFQhbYQQ>;aE7cs}sTMUov8ms+pUM?S)u%oS(w zYZk&xOCHUg7Yup@%e{uzl4@QtRSI9Uq+BkL=zjsvJ&XlQw7xPqrJU$yJ z+27c~)fcrSNu4^Cn{eX;?*V~9sSuwdc;s|Shf!5^N6LNI%@BAN!|2X*#PHO@;>|lb z8k*NkE+!AXF!yPbwhE5e(GGc$xYYTe7t9)uiIEibMtKeY-7=(6%fuoNbUKTeuMJ4} zjyI)GwGZb0t_7Fc42wXhQ$?)mDT7$ioj^&vQeabRXcnU;bvXgMF2(qq)+b*d3{~=^ z$!k*LS9Rb+pEMaw>EjEwA#yt>p{80lrQ?3{fA#!EVZ2Ud$-4j-VapY#sZ`UiS<(@? z-a0>a1iVwE?|t`_AZolx`hm4a#^hEg)Pwe}P$sWODg?$nW!X|Ai7!!O^%yPSQKz61 zDzx!v6CtUrSBnM3C0cB`i4snv6m#`UUzd$SNnY**H#JUCLP7t zrlZ$Vc-Lc|@trf~suEEqi($~~j)eV}S3j1M^ z?bpYSV!YnnFID0OL_G=nGtdtr0MHMn!r~W zPVLTOiiwMGE8(LN-t0nlr0Vtf=Qwa+pZZA$CJw|CJmEnUF*RN{RRBy zWgVP5Yw;w(n_J7m!$3ei@BJoTp^A}ue9sHMf~&y z4NNhlseN^4P9(n8x%1Kk(nbgc_a9fA00CiW!}tsj2R>LHt;ibR>6CZ1ZI-V3?Nqv^ z0|$1X@;5$9L%pr_ltvv8d*;GHc=dyy7}pUm5NF&UDLgXq$V;h?HYrg+p1%!*#zYK_ zdXRYPYe#d#TGnFYJyGD3qaIC2g%J9L%-33J=~Sap5p?cyk48Or5-J>)=t--NS$PAk zXc8BvkWoC~GbL)I$7j6uE_h9v31rDVlC%S~rnk(|mj|r&y~P(;ADuhYCTWPBufgmL zOviA+nkA`Ml^;$TuOQT){T~VW{4jqLjU>73)%424KLqo>G zp|q(f-6$|}%`-0qi;e7*2I|0m7^H732p(6tXs43&$J+Pyvi9#5vx9p)S3e%^oCW_1# zBEzRV8UDdmCO%#X@p_0*ATmmo4_bL2xw0b)m^F}Z!~7ZR8jG{s$vP({S2QamE3%m76FEK*`Bo~B>PpFC*^MO)v z>*l@PBS`fFVC04{nb3k;727eJz^muE)H{3KTmCxdW2=*E&U3W92D_jm$nq{8ab$W1 z1fi#Nz@;f@Ro}YWU<3~_66#UAlE5-zyQCt)az_%- zD47L@y_TS<*e&}pES9&vgzP~L>^D#66>~L6 z&7kqI*>qF;$*Ffo%i=8$&#K&6M+#s9*_{AYR@0CS__3eNjh1k#^QkqxzY^y zWT{_G%!5Nzd}lrWtGNHCQiqbHQ8_k}YZNcx1(i=)^*Y#lf~ISs!s}S~^15{WQl(E| zQynpdB=~BwgHNw(IOxGzhkE;Q)3wV`)HOU(vrDAvw>I}D@jUk=^~*ph+>O9>gHXON z(rz#PC>H_h(~sYK_32`T6e{*O)-F(* zf_Se`r$Pd8nHu!oig^JfayqwZryau@9UXUq)Df*{nI_Fe=&KI7tWZ5)D6OFYs7HoA z|J8*${!TPd6M&Wh)Fa3zQPkQva5M-83BkQ4AGTVPQqN1FzCm9G9ZeeFD`1`_0J8B! zk?M2^Q#LGl6dyFrGig8e7!2^Ox@_UV_87N6ctk3p|jGQ%~8Ev+< zO8lSA2sQXc=yviE@n*ey7d}@^)|_~6oFv~W;yQa?oZ=ycYT7RhKq%NmG9JZ;^OMdg zOAX}(x?b$XtEG?K{hPppO@^@l>~66VaW<^#Wis?uJCHmpTo8|}D}K5)sl$P|H{8lI zEjxRlEkhjj|9|trru~0!KKlp1bPlwhFLVDBEP33{&02R4gdjKX?R8QeG%{LeRMkpQ z_l~hG)^|UELA)c^bJjv+H?h;)P!slPa9N|qJWZccr}-NdGe)II#Enbdg5((>8M6drY{)F}F=nzQBzxX$6 zAjZ(ett@xu#Enq%xzwliE1%b;3>ik~dAx!&I*Sq8z!2|r7}1Qh7X_di08CYXz&!GR zczASN+}xrfd&W%Ma%0}idh4>#e`2PwdJ`xKDVazd=r;CJ`iPd$o3jE zH4+8P>$&C=Hs-=yaL#(bIwi02qEMGaS^Ra1HPVB(WJdzCDWj}V5XnA0q{~EZkV*?r zkvcKCZ0C9+iLUsf@yk8`XC$f8z@;=+>&(~I86NeYm}R=l-qAtjb+5&9D#V#XRc&M6 zuO;rTgNxeYJ2u#F0nv_x?^) z2e@=wSsxw0*MG{HhDA+@S1wD5oY7IoARWa#=@u+cGkJbz~-n(zjc&BsDl)V9C_YkeKj$G_TJRU4c@T>rnQ(3bNj-Ez45P%09j^QcMl+vEgq8-`afM-s;@26{RC61)$nYR`va~`b0c; z!idyvVW_T-mq5wd`Xg?(zXZJG=;|Lr@sE?8kW*6F=^0cy z5?yU|pDANW%<4}>M_IL48_a*y`~Cn(jGk+0T>RHDll(YSGxldyhvFW^l`jdh1-38Oo)+Ky6oR#kZXbP%y5lIh9U& z-_)*Qv*Pq8UoA~0|59GL(DqL20JP!I_jlTAjgL*i2Ry@;rfR{(ksM1yw3p z2p8fSF5Gf-#1uUpPd>a7Ew^`T7=f!S{Szb z)WcWQ%w=GSK^d+Z-tgNOOZA=t>oIOE^CADr*s{w7Oez~YG+J!q*+sJD12e+JEu){HUQxTd=qs)gLSi-P@eS_go6O;guR=J8iXN%#pE8A9Z)m*Prq9O#GQ7 zdQUq1fKa{4(AV$)JPb*z1w$CiD=auT3l;o^mE#();1&}}a5?yLZgvrm=fXi^M(W zdg!Lu8OBtmqI8|C+s>1{sBORUp=lJUvHn-Fe7Jm!xy*DL$NKdo(_1=6w~7fzF`Fis zYvvU7ej!=E&M-bU6~yjnm^K?-`=l>9UOGH2tD-r+4>U_o{5!^Gj;#bXU&H&hRymOa zjO2j#muYIKs#L8;_%E~;s#_}1ry{Tq{SJwZY|4-Kv>#$G+!mX*|?1PqvpcSROHTYXRp5$=|y>}{p< z2&a&VNHAgN9w)a=+m0e`=dai;D3ln`ylpb z7+4k2QPvh=o6s}Lm{D2LfYg^HCWYI%e(;7%&VC|P*lmlBN^UmlKs;ywrAePiQ>k@WR>lOVHY2dBA5bwmr0ZocpyHF$;3Sn$ovAr9{W|RqY zyyq1@{VNSD({Tm`%C4u#9JsbBMIKU)r>NAfTx^@ws5;>0^&|J=bWwQ6qi0Bwz#K9z z8l?P5O}WjZYOf|NU-^*#cc1UF1i7uWfMXi6UD{49h^urinK#0#Yfc`=&bC(}XQqSY zP!m*s=qkUK8`Xgu?@JJ0r}q=l$R#H$jf!zDx!HXYXX~;$mKIe}teU?_67QYkv`OhdAk$yD>|Gwi z`OEWsM;__?htjUt+#a8&L`Dk{h3tw+>aiOtDheVWLw(qtQ6ZLc%YYf<-2NPsjCm6x zecZUSHy2Tq+rNp&FegsLmE0m4ximE9NR*9G->NLCTder?3u~TUM31|+_m=sWy2;P>>8Imuhg|ITwF!?>--=zYm^_6)EsL7q z-IlRy?>KZ)bK$wuATj*FNvkypD+T1II|Ujxaes0{NsX_M)`XIvv%{eUwVrj-mhQ-{ zUpTGdIeniqvi^fDeCb6i6&3Ve{Blhfm{9{hk`%J4!=a(WDa;Nna1#HSZ*yQ~EB0)d z-0?<{VzTBL6OYd4`PH5I)z8}3DHA7)J=~nlzQ-qjA^S*CMT*Zw>c^QX0hM7};A)j~ z)4gWSV>^585-r^*a}G4(*C=RiO&+_Q^knrI-a@EcF$r`0+dyk+i^Xymc>L7tp zjGW_cpPf`b@kFJ3u=VZU2(5)IBaJNg1$}igC!YzB)H!?M=ZWl++tXh}KvG%k{a{+P z8-U1*S|ULiF1qnsqtCYYDR*lja&ITYSyO(tQOI;4DL2~Kcto6`am8c|DnR~(-`E?~=T!ECIIUE2(gOo?=|+8wG~4AI}v z-BX&bs-_I(j>n<)T8v>kXG8XDRbIa4$h+r7)9wmI{LGx3xX@;+h$b~4dyjuvYix{s zAXviq2YM(1ji!p0hWZlqa;FpWX9MKK=tnilib`vZ?<&R{!(k8Wh1JF*rOhXJ0(55T z6=Jw2p&fVl)YvJ&ZvRXiPjoc5xIliI-PZY3FvmkKLJ=)Ib2m zoItd{82NqX@`vi^hSJBTI_PM)G{YSd^~6qwPpZ!xKn(t_f$CL`Pacrh7c;|)etH#Y zO4ORd?hPUvBnf$|A`yQG-wx|n>7(nga|V?EU>X&HpX*-rj=`w{c}tV4#!pBQikDZ| zz6mc2K|7OfTQxt!8iDgV2hN{%^p(rX@ECzlR!oA&4Mtnk1)u!G0Yol7|HUoP1B3G+ z7fn(*+nXKuz-3aJ10w9tqdU=io2`oEwH=mBzag_?oSc89T=81O>M6oU!M!6Qytd}e zl$q6zY(R=2p3?ly zr7E8O(45&Du*u<#ap1| zl`*G%Cm*WbOu6sNwp}aC)1G>~+HCE~s3Ch}E1DZ9T8c(51PlMO$^VTkL*V(NoU+|GGa*GwZCCFJIy*b1{kW?jjtLe6OrV6>HC!-IqU; zEcs#R{K8Gdi6>{-KXfsw@4pr+*~JUo1)HLND- zsNWeb!E=4~;f)b7eqwxiLDlJEo5ornK~wm_%SoZLUT39zp@R)LuYqk~hNz7n-AH2v zRs>|jj|aTD^6ghS3zCZ*S+FEHePX%2=`nw-X3d)dx&oPabrf?TQ+W44z>Ve$z6Ty_ zp1Ax+b}hrV;&Le;v<~ZYWz0ADHX2|~D!A9aH(e4)<&)mNP`v9w1n`j(V|0Z%>Xs^P zk%56dpQ{M*zRj0w8PH+8_OIareQIHsI9vV63FFp3eyQ&yjv7X!h1Kd=u0_{LiqwS_ z!qkP>h%>A;Id;MB0nge=egxdPSmUo30O>{FoSXl%DLS?)S_?Eyd~G4qV0)ig^&ju` z_^}FhWry@WM~teq(Wm*ycU)cEQ}zk&^=|F{qwDWp7XNfVCIBeOH0*=`q2xBkHdr@nV?MDKjtX1qb z-+jDhPQZEZyxaXbX4i>D((6%Jlx@dY97VhsUTzrdv3fYYITn(hxOy1OqK1^ietHgN z*u8HxbBg>J9+lxMBD-Z4f}3oR<{NNGh&mm#$ohN#z|E8dktprxkhJ|nu(s(U&3X+m z*;8yNs99+AnL&&DLvOiFV9xSLwAkLD)OOltY+ib5y?R(@HFFWKk9vJO(#ci!UKOz@ ziPoj)D%t-dW9?*i*Fm|l2MNd38rp!{-=f?xf2?uxn5^Svqg*yy;6`s)E@RJCP-*yg zwHC{^%1yfT-7s}n8uAM4*H7liG|yDMz#ESL#VSszoiRtIueCjmqQAxpj zFPl^24cC#2Dk|;x+FFU30o{b-mKry?*IX~23axPL;GFpxF(=v5{(okc+l4C+DRUXV zn)ljT*B7X6Hal(7*=yV?^%~F*67wfEXFRU#{O|i!cdob%tTTZ5vB~b2bWHul_Gu9u zH@c&AHd6LI%aJ)=<6<;+3p;fS5gX+Mb3!Jz=JNNZZ@IJ#kEvie zrN9;^p{nM#PnNXj;+nC15Rdw4YHkY!xgVx~r}rFnYGK~gEsRV5F>g=4vRKT}f%f;3 zOLfS1;ZvRGDD#@#kMWt{wL+3z$Hh|U6by^P5||9L-1w0Q*3zhxZya0uL^uM>G{Pa|1}vz* zr2d-U2=iI`*4biuw=aZN142{*)`i!i`siM9kA|xtbo&0|EWmd0bG`dL>ae9o%Iw>= z{Z?^hJ*n!WR$EOxkgX5gVgceOSFCL5&RRf2PxJ2gP-v%|ANdvJ}ccOxu$L8ofU?q`gsv z@wB4JhOiuH-g*gj^XfYf%NCo{!|Iy!P*~|qv-&q}Nyy^I8fvajrBg|pcdjCe;)AA8 zE5~*#YxfZPLn-@ZJDM zm6dvqwwTFTi*LByQk<}AOS;yQL#+$kGk#)=Knx%%AAW2A8sgTPy-`aB25ecY%KQwv zcw8Iu@{z>#S++EBO}cWFC#(v&H!5szD=_+8nSIn4=uV9AT958Q-;MX06}A&cE9OBL zzh`*A)^V-qRLtw)Ps;z>xb!;?SI1eN_lnhEqT7y|QQZA8DFsaURIRaft=b20kC;&v zz!FgYZz5E#R}l+geSs{P6Yt|<*JJ8=&QKUp13*LQ}bt88rKF9>1&@iN$wiKDIqKK?u5KsaC)$&anjz+~CEo0s2UGE2a0=wOt9 zS(%}SuigGbJ6e8&QM6IgpannnaAX-lSSTmdr;PC97V={L;-6}+S|Hr3fOl7{N3ECj zSLR~7Tg+z^DsxS$)?&4&$FB&bTdu%9m5EVbaUYqb9Yja*aZ0gP2HVjH*OeT&ch-%L zTF~0+>ob1T<8kG~({z0SVb&f6f?pA?nh-9NU8I&1cgshtEO&?cmpwsmwun2euFEdt zp5uR^Kmow0qV`gOAXf&Z1wm)gCfA#|yE?sp+n9tBAG>x|Vy=N5P?;j2aR~aE>S!r$| zMXo=xhon>eOCP+X{2)nRw`I=)e8xNgRHjPj!y=c8+va&pN`7ZP_#kYAbFUZ5Dpo9T zjPY$hm$aE}p9A;nj?c9OifQ)Hb<3i_azDVUmRHuDxLoKM_&YNv(pcu|lcF2tj%_a9 z4BDy@?P@6N2bxy%$+Vp4C}1rY3O} zQxYbv=W@T=SQe39S{U#Csm)wk^BVbTXkNteOacQimsO6De!Z`Y{?T5&$zbMeFA$%U z-aVQR|25gX$U42nXvJ)d-e**v+UM=C59a!dwYk?Tvjk3V6)QJXdT2fE=p)@wQO`5s ztuS6Hf08xx$5`gGd8wGo%~_X@Ps;nUpLP^Gnk{F|%$nF*>@=HL7JbXuR_vGWkEVVk z<&^Xug}Vj8wW%A>X^F(I0`Fi~4-VbJ%z+UA8=65NuDth(27jaGFZwYz-kZgMERFQrzD@-_wjD5*IpSwl^evkbrFM-_Cs}y=CE=0TyDhpP&Mnd^T(oL= zp(=Mvtzdqs2ddSR&F$)1mk#|@lrweOFsjaZ$%DGeq|vrnS` zVp%^gXmD*f9GrJ?4ob(X-%4sG7d!GM0o>ybQv(dKcHkXwxh#D|gPXcdWHUqJh>Ym~ z`}*9l-G${}gd6=(ytNUf_|wQ~4L%v`Iq9MqrMJpYwn^&7_cwlM7iIn2*@G5r)1+P- zUIY4kV}H$xTZqPnQO_wnvTouy7Wv2e8Vz74v+>J~C%nyWbmGii3e8@qG|Vrkpvfw4 zfud@)BrU3;R@~FO)E9l~2jzt&?YAI8o$x37(42$bXv6N}S7e%wf3|JTT8(|nQGFJ6qc>; z;11obs#D1LIbk8jku+x{^1A7(cC>$Ws|?63Cc|8TcBG=XY|acXdyry7J$&$f z^O0rc57yd)OP9?`U#f``yn1ZDd0vIQeVLaP)6M_$bkEqD%vhJC55I;e^%gf@w7HtmX_JiRNJG#~Lujw7)X3}|m3i-SyzF3C@n)FGZ@r}I;Hu(0F3aM?B4PcJ z*%8ZM5jmnm3u3Av8L02P7C)iccdd{v4jL)}qyd?8Z3&tM;}1^Q7Wg>Zk>Vc=U_BkP z+R(g__YucGmmY+#bglz1%C$^;;ph{9nOh9x+RzQSLVj2cf2}Me9FwHz8DIOl7U|xi zi&5TskPRn)UL27vxO)}ypbYgY|84z}Nx+ja!TW`x&9c1V_X#mgD#lzQ^{?xqIqC-9 zL9ZswI>{fmBoF+)E}FFNZq~T84rZ@{S2pd254#Q1<=^g{_4rs6ybRY{fAU%=Oi=lH z4E?|vxa)Vacwwi@1q_Nq+g_7beOUd3n`+~Ppko(bpuwU;;5!_clWap%w1J?{B#q2o zI&?s1VW_FeylYFQ8m>m0ACOmj&m9cY%1~6Jsr@?F7)`f+c2IQe)wq%>jsoZl0Vl5Q z#GAIm*)Oi1UAlVudUQ>ev*gs@;(~(siG7w{2V+5ILti?CRL|C6|9mMWwwxPmqOpLP zmWAce6bbG9@_vYk*#6yn;|1yWGM_?whnxD79d~3xs6W|Q4{4k*7rl?ScBbMX(lUM$HkGs&Umm=niJq6FDDlddmh<|;!rKK0qs~h@Ebgc3 z&MUPoAqJTUncq4p9@D6EJI%Ji7uVt#Q@^Ue6YbyXDo%(N`*u$pRn~8Weo)8*4;MQ> z_}J#s>NgwCpZ_%a04SPt$@m=N$&riY{KbzRRCt)xa;EHY{EKV@1FV_e!KxcBUE6pG z{cl=vT_y$y^C7L73EC>zLQ(_Dny{|Ew{hx&+r!rapx+ZN?aU*4bXB=$e$WrGECmv= z_)m@Mn{>!##>mb~sN|o0Ztvhx-?zst%Q2&gKYK5lxxEni0vI9rwm!(Ivoj}qRIjh8 zg6#-DvWFK-OzU`h?jJ}{*T^u@_@QlVuz1v7k2K$^9lh9}bZ!=O;1|w9qODIKyF{ZR z$XE6-s>8kjQ(OW0x%uFOtKUf(J%o|@tDWiW<{c%>88?mkfnzAr;A$QoDtKDsb@W#F zQ{pMR?;Nh00F&*rP@W;6AOwcD3>`ml^2wpYa7NHw48f_!o~l)!Q^s`%t~c1UQG{fF z-RQvh=cdTx^S*BDAORVK z?#QTz#$Bq8tbcww_q!eOHJTH0vYN6(_l_%%l_hoLB^jHdziWqWYi)E{f?E6@ z<~;ap5i`q6Iae|BfnJNDy1X9A<3^I0vkG5>o)!fbIZDSE@uKP4S=J{`lkV9TCPuKR z+3P*t2>-kIoNx5~wd1-=I4$NrNU`^aX)1>muj9==@Fi!n%B-y`O$Yx58H+tlOTWdf za6vtq{H$!|aXCarbzXI=v#n6FtVQdOXxYI27{rG!TT1rb;D>*U4w*AIq6Z|E&&NXy zNWo8wMt)dsjl1*yq3{3VlG+D8@%Ai!tF?N{#mUc?k19 z7-=D4IwAiiNDX>QDDfw=kRy4W@an=hzLpYBXWpQr%y0{3#~b>X*(7ii@>bRZTL;6T zb|~_YnwOiy{xb>^RK$Xl&7!gJFM9^ut6E#73H8GzmrD&*1eHdw71$waD(_$A$b(wn&875cijn5CNOdv7=qhx%`yJ$Fr_EBjj@h-FnNc6^mPrkm z9Lm|hv8LMW!l|LxEuD>v4vGmRUPS$X@0h+w)%fY*b8zbsn}i*s*jHlfTM$EGsYWg2jcn7r?d3uJPtOfC?(9fkK!+H8 z5smH`bkQ}?-<>u_f42e4)p=aV;Kn9qb~xxQ-gs+~aiJWh>lGMR!1=gMHqOIK+bLbw zLxyp*$FZ--ZA{ax|0SRXY;|$cEm1)%xP9k5^l(+=`;Dw!V^$P_ebKT#u zeDyw}WFe@K@b@?@^XO-b#NkO}A8QZqy9m<@v~Y`ZgN=62D{jsTaDG*cAdZqi)SVO- zo#!NIOQ!~W&)N#-dpUbir7cTNA}o@xokw?dFsgJ)>>0z!zFa0(u8EFu0gk<2T)jd{2i-PZ`s%x$#W#{dtm9z+aUA0?CdVdIBq=nk zV(R5gvs?b{`IC(48_#Ifu@VJ)QaVB4^L$JZIL4~oNi9HGp+2C?H}#wo&nNZs8eVE~ zSi2DJ;lQ-MQ*<6)*i`LzPwA%L=P5>(Q;(BAE_^-$bK7{&GAy~LA`4qBCcfG>GjwP# zfddkK0Pmk9RnN*PvQsfX3fVJpR>jwCZ6n`&`zdBa_2NCa8Ahwi8>?{Z_?wQ)W(}_d zZv?BwI@g1+DjLEQCUK87EVxCwK$R&|3q7Vf`&B~lRQJ-P^ku05bN z{HE^lRo`*Feb7-t zRB$Vj2=aAJku~Uo0@9$GugnK0%~)`qF?i;)y|^wq#Z^(>K zRSI{M$1`)0h8q`))n~WFYb_h)Jp=ASD z6YUix=R;FDg`7{RMN4|`#D6bV`{d&Sf9Bo#qQx@+;V2#h>YX!gphi_XjPJI zR}YhlhTrSr@>jHsP&Y^IPthJZ_dbj8^(5FPC4Fs}m@Velv~=evk6Aiq<9w~{u4W+X zG}rluYK_%X<9tWfPJ7?ir~^MOX%wkdpX5N#N;@VuMwI{E$uSBSLb116_}X_8-iI6Khf1<%Vj@4s|5B%UkuWJ5#ZR3krX zSmV$Y;yU!m6RZdiLp8V@(yY`O&HzO9<-J9f83N@w|M7d&2QD!(}Z+_p_F#AZQEKWU{*pE6>0b}Jiqa-g!uQ;)jumep{ZJ#X8C{Oba|2_(*o zPT=*Ad`^EQzkQHAhR=Ih`G3I;Kc!>QSW#2-nLYuy@X1i?AC%U4NjEv<4Cajw^ZYm~owkebt+FgbGBSwk5oOhFjZR$7OuzUU(? zn>_s2@(gFhOA-C+vj-8!Xg$fJnIUU|jB93f>QdULRii6DX}IAqT}VGwK83nxjlC{z zXEa}b-Ae6X9eWCCGj<75-fu3{R$BZ=>a~f^u^szFZ(|V8kf5suPF=KEhGgOrB5k=E z%KT(jseCZX_N>%D{;*)_yzeM?&e^{Utw&Aix8Pp2SiXvRb|0b2n+7nSEfdOw($+h| z9clU-L(?y>%V+-KIikWNp!Oep)U3WaH&G#$omM3!$w^(i!o5xaR=LYFs>^X zUBFoU!qp`?pjJF8J(24lyP{4}pBJy;^%&~U;|BfQz54@XR7I^RT&uHr*7i;aW>{JvW|qR-r3^vTyCsJHybAXlV6NO2Io2+i6lr!t^sbl zf)*I}ISqx6YJ@!%c4Z$HKaFl7pjzDS_!n3Ixz9H=tnDo}@`IPzl!h%x3%O#9OB`Suhj4z$yp?mXw<&NP zWOb)Hpt@T@D^??17rktQg88t;EluUMqRSn)ra%}emMeHq>L@!W?lF5@XkMh{4mU9+ zriD`|8SmGd^%l=)#*2tCXLUe601xbk>o*8s@2LeU^`FrnAchT>KBr(+yX2hMt*4L5 zm|H{I{s}j=He&0a(ZY+DO-k=!Ahn7C07cd9KXy@Z!wB0G%ib7X~9mW1{x)!5~S6Qsm(!5yF0n z!r87nYPbko#h!1gc5gL+{kQcKgPr|aHXEqo9jNy*Ymo*khy&gWwGPspWZ z`2BlqHa69B`gwDDv(D1Z;GITllFS>L>3@G0z;;DC-WW(;lMP+=S{xAtx(c{0RF1}#{zM5;nM1$)dLLhE1|DkSr(U0yud!yKu5}6k| z6MN6I`*_WIKh4tR-|z-PcB0v&%EQO{%<>Eckz3l?%NSD z{Y5|%L}DD-djc?(mq_KUm19lH@e#^;bLh~vHVkXs=Znah2DyxFN0aiWW1W*H_jt*z zalNZiH|&D{JR+?Y3>ko*ZlUJ%tcJYTVfzrAWVgkGPb>n~RWP2I>m=Dv#qMj-XlDtj zCrwd+tIx97ivLDZ(_9k_z2*kk>)<6ooDGxO0QMv_{0P!b!)SI9;{Jy9AQ?sr;Tqhd zm{ne9TF$L`8ax>X#A@f2Z9uUkb~O~Q%FM0PYw^q>{H6u3jR{eChBXCarL3GB+Gz|i z9s^2|H-x{S%9*>C#@u_x0dhlFdQYP+JK_=UNw^}zv4H_yitoXDCN3ON0ZL~Gcqa-x zI1tc?{O77i0K7T3@d{^zL!#odIH9%e-_HXE0{NVru0>S0jtu^ z8C%!(YFozV*Gry`-#`IR0w`1fQkp2-r<~+H(g|?NE8u@J-tHe)M!oAI**Fp5xo4N% zt`jZ+JwnfFgXammXF4O?_A>0+;UwY&Znhe8CLNI&Zf29nEZiRIuR`Es zR(>UCFKs-RR9F&Q@7-g6L$a)x^A?aHa~SLWNTMDRYY=J)m~b@9y8K0fFAG>Pty<>^ zV*n3o`(nTRJbOFwiOk~{B!v2t;=U7!Z!dDD)3jrnJ9|{gZNU zB->fxCuXq$ooxjwzbGcKXJ%G%LngEVf_VF8&2z&4!_-;FH5tAApAIPnrAtx?DQOrW z3euudjuem_HJWXdib{iol7cX$dvr*5ON>zi88JG?z~B5nukZ8x{@Op=ea`OdoO7T1 zx;~%x`Bln4qql7P%v(#S)d_I)1?58Z;k%RSwgqbQKjO{sq^O&7sNeo&@?1M#z2X{7 z3FUw`)qkSkO+x|f#xqXmz81nFJ45-+WL@I#p#%dw2Zr;i&OnuHcO(Kfb&S|c#>gIK zmQD*reMYgJMb?rJbzJ-t8}GzrR#$KcJWJS#^J0vkxf+lAZGn~h8mzzvT+RI9f8#BT zAo2D)FR;NCaI5E5zY`y;E`4Bo+uWi z6yX0)dbGeLZH0a9?c>frHX+w>9|#^HsX-EiW`dH?HSf5xDKKkYe>+K&Lo*pI~HW7jy zZ4P+C=pZN?VU z?}ft8X+r&T&$$7t`s!%h4XPYMcJ4%7_$rSB{Q3@I%dQZ{NmCGWD5_C}EMRc2(vqmV0Un&+>oOTdNZ=P2nVg2%jCqaAA2&isjmI zvfhhAhbG%9#dVC9wY(0~)4>bwzw==>wbPDpmr>Tzd5@{0 z(j-%HXP~V1?8Vn=VC@}Lkz)DuKjoSU(%3?la>jeSNilnx;cAS*FFhw7cAVd`K$2oY z!!5szgS!dJ3fH?TJ6U-`xvapxTmh3Ai@wvG9unJ4$GTZKn8O&{fP~}J&Cq?Cq-kE-a_wF&64Mdag`61G6bbd+wr_cPwen zFK7E|HyDuLXEu=!28#?LTJZmb^3y_x(`Z74#w4TO4D@Kn3p%gx!>=xD!Gvb&vVPJC zSW)R39(*PRV%&p>C(N;K1OmNUToGV7+u{KwY5iJx1 z2>jqa#D;(G?Lwaz1Gdk{F8a5PGt(FUTkKK$F5+IBpvRN`!e$_;aqoGz9S$eZT|z55 zJfaXohh?*Qf=$i*%7L5C86l-;ZY) zzh$0xDz&Kib!>PAzU-hIm{A*i@e*&xQCBfhdBZ8%UqjxVo1?!}`tgyeG@p6@$m4C_ zD^nVC2XiY6zl3+zsslT*hFP{rCgk&f#4d02e(D>EdOAgpI zuvK%IIOT@8d(r_knfsE^*Fggd2EJ=hb#VLbIyCZuz?UdtP2mFVAAL71P7gtyLeU#m46O4lcTfGR6Oz0)(rE-D4qyPgOx?3Kwd@{JO z^>FdTxN-Jl1-OK}DSPwM3nzD#ywC`uNp5AzSDOS{ovYZa%F%P}=%drlD`LEBdZ?8Z z&WY^b4lgJjiN$h(uA@g2&MS9fR);xYn}6!!vMju+1PMiO9v$BS1$iyxFZecmm|ms- z;9e8=vEk($jl`>X>b}}>aS7$oz?}z?ig`x93O5=~pZX5Uei;aeEN>6RSsNXJVMLvcN)H>*nslJf6gK zfk5IDacf-gn|Gb0@JI(q<$(Io1ygC%t%*Uo%G03Tx17K(68>=J4Y!PWRoBs`S=XLU z=DH#Bh=#r4@n6U!f4%5QFj5TBzwoJZ`xBE@=rYp< zQPh*HgFkf=To*NU_S243j}~sEosadImOswcd=5X=nO3W8K)Pejf6i+metJh3F>7nwT&Uwg}ut!?ilU1{l*|Ex3 zD9u9>qU1IySB)fx%-%N4WKRRY4No<3HJRX!)+M^Do)<}Z*&#DR?#MkMYaOeDe$f)a z-;9sk&?9@;Q(bxT`IM86Ejvi+&!{Vn6ySmX@{CxW%H(eky%XCW8Q*NR%FDYI;{fR$ zu@J5Io|Wd`N!9uR^B;O-ZMDO{NV!o`m#UBZ-TXimq0(fZIe)iD`@Q!_&W$xZ(~yj= zyU6UX&fnP*!2}`DBDS5ZP_a<@h>4)UOqMjmy)v2+>Z>TNi`f*ZAj{aKR@0yso+lCzW{B;Q2wA_^%ab9So zx5JM8*#~|f=)UsoORiodnIJraTIYE@QKF2!6N8o;Kbx;3WjGfUYxooB6RUqaaDASI za+W$ZhuLHV}mi0lm(xgvxV`Zdbpd`OvO?Owq|N$@wJ;eN=>i)q3&z>2KVnQ_OGBhzk~ zF6F4(foM+7tgO4r{mdc_>w0P(ZKrcqNtmM1BPqzOV4RMVyqe2UxW(GbCSZcI3@2BN z9a_n6MZQkFuf{^vofpHH>74P7C8J)@%n_;hXb^E|JP)V$iLt+qjeqtBVYzfZ6%1XO zj+Qvw)G6$~eR)m;we?*beN5T!w;!*ow10qORUVTKHl1$>0#p_5dnMK;6RfB1VBXMS zQ#%SRb_pIHcK;#)t|zNx(sqts>^I%*d1I65cN4}E_=WjHujm+@c)b;#mxW7n_Ydg% z>{f`Psl#&5O*13&oibB|VN)n(GFW$tH?> zPcgpob$I#8?}?{E({r6=YPt5zd?z*Jp)A|OO6Wy_7AS)0mKHlLRcY>gG!W5c4ZmooI~uTl1e=$Jp4PSV7iP9T_X$)m!9Bzm%iPC1)9Lp+Z!N1-tdW6w)ZXzN&48ti>TX}IKdN_b z@N#i;Cwd^rpyDelZ1ibC3bgOM%m-O6H(%u?u}PD#waZMvC=)MT8v}`A9f!H5kOtmhu3^(7c+L-#nohzc?UEp#PXH zJO|*Iy{>%fw-{%=2+{+%82Ml&@_BYp-@xV(3$f^ zW30>BrAlRoDZw69i0$80=w{&ln$^XYa+Y2yJuW9)D&0e)F;JIgsetqj@6YdhgSW6K z%0CnZD%Q`i1zBNq&Y3a><}@BYP@8*h6`SP_(hI4o{(N*bQk_&2p)%Lo4?`YGt~m`g z#o7*S1arTC5Mtuw@o_JuIsaa7*^r&!QJ%w)Q?^1$We(ECBZ^u9qselmq;!9D_-I}# zDk{2!(3O=tIh-=>0=i0%Bc-edsJNU-ll8t?^~P)EuKmE@ma4F-NLKB}>{bT9{}l=L zm!9o>2e5cPA5>N2RK!Jz(t^GPA7w_S-FUFsRTXx0UyAItGVNQh3CqItV`Y{ds4_ zH&N=m&-vBw#3cVHUPKn351+5F*_XYTEz9hj4mNyWrPD8dgCV71V~1qd47owUU$5D6 zX>ztutJ?xT&uTx@d^2q?v6g%n+qFpj2A}b(GhI)qQfzrXr0$oRn@Z>{%I2~=A>$jr z`x!jiw&oZbF7L|tKW%04+j&QO6rnh&W>vjX!L*d*Q)y$%j#G}6 ziTQa=R1wjqrT#+V@xaIQOiH1&hwcOdoowNuX!M_D{&E^s>d*;d-x~~b7Xg~r!(c|a z0GA1XdAp5W?4)3pIrB1FY_@EvRcFmg`X64vgyuZpr_t!sC|T`42T2V{=`-XCNS=jHKqhQ;?gmE^r-u+wSq6ts1?ujWjGe)YJVXE#XFTcH)esv~~70q{v z^X;GB$=g3eF%oF0Fp5&|isb96FM_UZYn10vCC$Cf%4{2fDt~#yChLAmk=A8-d&F$) zjre9JDgK6vZcH;nm{6PIrN#WP5cixdv|RGI=$DJTQP7wROrW{FO^c%CC??U8iT*}T z@N*}hebOM(iW^opBZDFB;aNNPSz4`WNq=3EHYW^dQ$bBkDd}V5n4HW8JIpNiZQaGs zU_?9yFXB++L82bkukWz=Py-r+<6{^mgJ#0gwiPj3@4THA)(7C0e$gt5nJkw<^6z}& zmNWJ8f=Az<_vN$L2Zg7m5wgR_;O{#H_SVEatO^z;@%v=lUvDt9aaxU}w0syGJpEiC zf(!~&kmD(Sc3?GL>&i)v zy0n#`8D@3~=+PHaM|(51r)|XGr?vx!2SK*`CX~J_2b}TVn~n)=8`kH4tjs;!lD^r* zJH8AW7$?m-D?LBAXC}5ak#__k(97Myfa5K;N1jM>qnpMT&D67cN-r4L_G@Dmjkv#N z3YRxVz8hf~h&{QXxoydB4QKHSaD+za#(Yvd**jQ1J5snr#3jz~*s1a9azW=Yl+R|CaP(w%9b}pT(Xs)dojBRv-t=(Bz1E(VmUh|w3 zf{x1w?T@t_B>5S*nOiW29(rob|cOfxtHgF1BycSg2e_^#1oL_|BCA#Bz(D!_(oO zUSgULW?ftSiLheJ`U~xOg?BXWaW)y_U4U;(hOL_;w057TES~$fW|8t--{+2{vb=by zPa&$9G3JaLKux=ra6=S0pGapfNXa)j`T%^?T#nKrKjr~*5%Ylw>Id| z_%0on{vT&ZVHjs|rh*1f@8#}ki?BxG?fNqlukYCC*(>p}?DVry2A=LCc5e_oKBPA@ zTghEMXPkt)q3uI9YLTeOaFjpLuZiprU3T`1<#cjY9GmLqieo5upMz0<@pL~z^A9L( zt%6!rU%@v?U((c?$wbojweLgjE9p1Nj6reXx{(PTIt3 z_6w7dT^v_8*|WTN&$%fjx7JhNwKVJZC*wH4UiUE7MkC()`d_W?^4?WeThK|d4ax;_ zqsy?0u+;sSR8S*ONJ60a4u0-~4<7u0$F}#zkrA#3CpS#)f%VoO16(^!HsUJ#tAtmAHFv;OOnQ}OgA|;|^1Rm%?0mevhW-&!VS#j+ z$vHGXh~ckaheUpcD&Ak>BRRtDLq>bI2SL@k&d__~9DXQVduLr{JWzKrc)KpzK|MpA zP6>(NOwiDVVSO`)|5X!rhiILdq_o?|eR8*au?ADeU88rs1zO>+y9K-|8IK+*Dl0@t zD;UzXpt#D_&%AYYvb?rdo{iC@lbLf$Q1T+cg4B;7IvI;wCyC1RQ!TLYI~xKwR}xMH zmmVMJkeyHq2zZV3e){BL`u;d!VSq$AUy(mNAHnLTtUVmJrN@Y=Xi$$intBs=aDCPm z8ft&fYIc4rk(M~S>QwjGt7$lLc?;6Xu{hS&ziv4e@G8Wl1^L|lS6yJ5ua5;U+Nizs zFO@l&aLm_6t>P^d~ z_|(U>$%T8uV6N^*GeLEf*nIAjAG;jc_%-3PP`fQ8uABi8{P#m6nIW#o`@vXl#&Y;l z0;EbkOK499c#{vK9&O0W7Z`kfl(S^#0A!s7r9hf=mTm2xm=UMXFi}yQlfecbP-mPk zCNoSX-MwjN=H(aYM61xm4vu(oeRRxh>(Z1dkJ|e_3?_1p4bRAFF0BjLlian@LoUZ=*eL2QOCoR{{Ctcv76F5liEIX zliEAZfs4IZsc=e4jl#AJ!Wu!+#NlsAMl9br9*YwuYS+K^%t;TJF- zlx`HMZ|GO1TlMXeog<7HUd`4&Jx;n4Z(tMxiwTPEEP>(|qP!ztHP>PwpK@VZ;fAL0 zhC7dWA_v??P^K?f*iUGb$e!#Jf#x3@olGVI0%w`82-R76>oo35nrt#n-wF=v#4m$CGl-nfn5N8=Xice<2U29#N1TF*s!w?fF1$)rRQ1b#E<0m-~-= zQNp;RYP}*NC25(>)2BM`7*57wfj?uf;uvik*B(h_febif42rCWULI9NW z41Er6u z=kZNvXEm-(kP2IVj{{@9RG~OtIOD0ib{*;&Ox*tSHH0KMoEJ6w{@TA9f|(3_Ql4ea zI>1dLqXWgqasj2qV#wf0%O94&&^f*KPmA7D&zWCr@|V%6TJ2$q-51rXxti;eI?8@J zDSK-jdzHrBN}?~}bP%B~^FQ|He~%x%82a5b(&P z&O2ey`OCPRI}<9VFXm2m)0*_#gAHaCob{HFF!Z4hpN!I0PLc|AJbjSWOSIthg%Q@_eaF?8QNZW!cXPc5HxA-MA?0eLWz)g- zdP+r6mM2er|D5^Ir^>N!96xK*ykbjs*zZk}((`xtVZt@QNAz-7G%&B-8|*#o=sn2?bK;_e-0VLarF@QdNSFuq#X_*FE+0TPGBj0)5CLG-ZQUpbJ@&s|K}w) zK~1m1&Be_VsXr;OH8ysC&JCwUgq6h6S{Cb`)yd?d{`%{EJgPMqb~#Ubj~@1fftOTQ z+DlYGZL75tliY|w_ls8t&P@_UNa89q|NgCXujeH9gB9G#Gl+IXsQbuZ*@6?WV$o@z&G8u z8*yhWs2ZaX6urGCwyIjd935uGA`0?<*#?X&L&!bYT&oIm+i48u9^68K#r?OACA^aB zyTbtXk7Tb^&doaubS(oug*1m=4h$o z9{oEao}muyxbmYE|Ms2WoX0npCaKs(;tl2ob#lvRC&zHLmP=AZ3R#xF-}1RmleT~C z97&sZ_O_nd%+rD9F{8LXBj2PZdYtLIC&O9G1n$e4a(RWwKPue?9AS85Yxp_hLD^@T zR3E<&(wy5CU9$KjWVMQ;M9P$MIvCVnObXVN?{f08wI57Gwoye8=vu zS-_potTi5JuNxGF(~+K}LG5y>z)z3W})OfHp)xa`BYyV0y2Q;*EZsaRpFmf>mT^fddkTDiZTbZe-JTXuHYduX7r}HPbno`l}_vIz|S2 zd(Anm(rt!hXjc9`8Rc=e+ud*FG+yUQ(QGu_{ppPND#_V;s*slRg8LI?F^(Mxm3=l2 z9RfdIw7oTbW;gES6n%AW73e>*%NXQA+Edrl;D2Zgk+quZI{986_HlGShKXlSkr!sc z(Z3uz{>>XZyB65QReBY7&IB0Zn^>A5-TuVW&Y&vxV=nG$PU$FoYb9BfkdW~LpNrW4 zBbGUFP`s+822baqQhZ+g*r;82FkfUZVsmQ{H?(S!x>o$05oyGv4uS8@VaUGm6{F zD_k%q!l+|xw1b}jqOV!DZ`p7d?N?v&0GH429&4pIZ(uf-fz$T+Nk+`le~=@mMOe!K zXIuOE7C<@HU3sSXG}E*^j~tqO6j*k(pwdQPUbg&?)0d{ElKSI*5MLIzPi;;+Q{u>q zrhHTmngLp!x_Uo(V)3_z{Jdqpbzq_wcCq6ZS1hfkm3=RaFq_7gthkso3@h8W=~3033FZe-d1F=FLxoqq zI$Qe4=T2**wFO(+^w)iK%>z6LReL5b=%iLve+}@p*=!jL58ay?-K=jP zJ)a_rCA68Wigs>98&*ai%{H6)JchjnHk{5J{h<0^h3w&wepwG-WApnVC&g0=*CI;2 zh~oPS`=9%hdTM@-F6-#FDDH7BC!hBf`-ni7zwz@+I@VA$wMCi&9Di<&6Jv#|ia0BR z5lP6Pt=JSo*L$Z|vu&RI0L> z)4a(hzkQKm{VcV$F6xf8V+AmUC%EGCxSq*-u-9nw({Eg9oykhz#V>#hQZqj)kgdZ8 zu8{lTK-?G;9Kw44L`#C$;N27We^%}7`Is_p`$cxc5c(|~HKX9wpO>tn(>=n$ zifDD4gUq)oGe?g-Ij$I7X*ml^>GNa#-+$l1-GfWq=W!SqVO)|SRB7Wea_`{6=Woul zde}@(dqUn&FA=C-;``hxi;fxq~xSX+{bt;LJ+Z&w+&$4M82o0e($9|rIZ zvm|lwvqfd+RNqx|Ui2bqEBE67Bp+Z1RVrW z8tx{}Sebl)SNh#=mliyg$!BkcRqW7v2QPeSb}+Bn z+cd;%@sSDWb-kH_$Sv@t$tbK3#qb5~R`H({ul=Io;uoyM_8Qk4U7M_o-c1SwL4`i+ zMp*6wr(0BHI=@mTumUGEuI|j*{Ka<%6MRQ?uEfN{RYglhwQ&?yJuOIz>(#yTk^f2G zn(zN1{AWGjI!d&W?1j&@a0dGtW!jp6;nRe#0 zWA;4CEe-p-A$Y87CFTTG6mx$LN>htk6_h!IW0+^hYs1_&tRxOCjply?n6Tr4h!3b8 z_ZCXTjAB+yZ$mJq3e`1=*$Yt|xu6^MctyKw=3|8cI7PqmgiOr>fG6H2zh8d2!rX;V z+7+I2{zt@~0&6DJ&i)D+_Tkl zZ9Q;caX|E^u-^bbP1&y(=QR|N|r_&^h`DJcY<9qJvrHc&iE@En zm`B?M@e-b%m8a4wfDVNKbS9;eIUBR0QnmhbhGs7YC{bKo)8V8ac#(yR z1^Zp*$trfB4N%spUqWvsMN+i>F5hA89M`+EtJ3^)RGiD6%gX&F(%cdM_jEUFCN+Pa z<($!J&b>~mPRg9hu1n)X7=wE7INj5W8&Bnhgh;}D6VJ)Ma|#H&{g8Q!JCXMmV`8ow zqkz42kAyqr<%kXPuiyK@D|2vJ?M7w@*7*6Tu@%}l{~Bc?+t}!YV6ponzBa(WK@RIV zJU0$<5I8N^(U!mdHaAhIv+`JNdbA)&McNG_Ex%q}SpyZb6*oSgCao4gn3f%K{xE)U zYUG<~Xj&R6@+GFF;5X2Xj&G5kv)TB*a9y)>q#f7I;^xDWE_@p*!@1=fd`62YH(V!~r+|UA$>jsHgBPj62YtF2%b@vKEx?p$-_cELd@S8d=+3V(j;C}J zKP7w?^Df%_J#AwdewO2y@UC2I;4=MzE3oj*J^^epGazb(uY&_})FR_KU?7Ueb3Y$;7_i4wvJZ(6aBk~3en!2_?tWm$$seJJ!W7t7}T$|6%Tp{Y`vG|eb29_ z{Mg<;Ve}nl!}r)x?$=D&6!zDra`)N33T3mu=Go!J@uc9ooYH@1DgEt*}zXm&R)qp@(LvfGsm@x_%-cIw@Y6+vp1~E4)l5W zRt}EijbnvGpNn3>j>m^Z!Df!uoqNSafa4ENCW1@&gF7xuG>Of1$gTHjrqmDnnjF_^ zmxT0|*sFf=2aV0zt_<|rZpxyRQZ;%+2ey++lgG|_L`oTy}WYU_`3_h zE;mWeW1D2lPD|SB{r;Sww=|%=-Di%FjgNHYR%y+ADEXx32$V?C2Oop{7z4s*q)Y6qR zPA-<%!UqBs>5w{bi6D?#l$reF@E?n)gG$&@?<=Q-_#fq{nTbk4N5a4E>O&7jVPzz;0Yp*J{oxv8>XWs zPYk4Ee0I?leza(omDSn$ZOt;E^_5^s-&PJ_`|iDQiMi`NlrT}rz&Xx$1ZW?}puwIo z<#wVeoyyib+mDFHu-cb+Jyn~9Msy>79D~u(XGrJKa@a}nPr&ts;a8(5%%p?hIxb6^ z^^N}V#hojIOUNVjXTQ{-mKleTPiT7+4%j}+8U`6*fXZesNi`J=(yK_?u+neN_LDl_Oi*g* z7!Vg-vX)BJrI0C>pkRnhH7uE|+zVXmn*0Kt8di6eo(`vzZp~P29=ppGvc$0>teRXO z*1>E0Od3NKN5(_)kPFr3Nk+aULj+toh-KHwYR%v?uX2ed*2{HVmgJqM9bmMieTpBq zj?5=k&tkJJfw5X<>fxolt_TOGrBg5gMCHTwsZHnmv)6#n2K;lT>(2?^9tCLR4T<8% z6fQbI$oUJeHBui6$qZ2-aq32bzJA4&mYmp}GIOhuU^hc=ySh6O^XIQGm!sI-FY-ZU zkCaj3WD{p(?P>(mNFl>brTEW!BL`^#2w)cFh(X7VY^g^%M^A#1zXR@F^vn=k3CZ_u zz3wh6YmR6@XM4q3=znQDWVPYyiT?gWI(MCkr60XqA|!!@t`-z`f2SRNj;ZXv*k0`- ze{b847_j_MfxUDQ3I%zO=$RT`@TISWw1x_W`Gn~G9m^4{JE<*swpkoO%JRnRW|vD8 z4F`I{GxQR9-`ShSOx=OjPBZi{{3$xYR0v5W;%Ca$vIJm>7BRQbe9vD(;+X| zluZ+2^7my;F-Kdz?ml;&W2#3P8X;>rDe#!#R`LRnsZT5x@LMliVk0%?ig79R&p~Mr z!24HHtCIUU#Yjn!pe%=Izdp&BAxuU|P#iA-^Zy|?{ zF2wINrgO3OZXTT-0$88R{tNz_55rvH;W=?83_Nw!!49^<8#(qA&XEAcVpR&`j$ATQ z(yzrvQv+0LbEk=YoE#?{$7BYOGh&6a(UgXPRTd~6tFDko*m##^J<9&vai#|c$3x0l6RBCR8HU&a(`i#BB}(yakiYV#(G&3 z9^Z4*o(_s=KQHtoPzi0m_GlJs;JE%SFblguZu?>~23EFyb-GqdX@(VW2VTJ0Xm|AP zSwUa&=Er*Z&=`g+$iX|cs{rgS>-X(zaSNX!lBLXV3REN?gIHAKi1kZ$6A)RzfrcWi zqS5D$8wQIPRT6FUwo$NJi=&;51^ujV$=Nwn{WJZdLF?)xu(j|9Iah*ENb`V`C?&nx z1z0(O0m-9a?&$r@z*&h}tg3`ohBQ5nZ!_j1ue9Tlc0et9Yb4h$==UeXIGwV%XIzFs zaZ!zPX|zfd6lS6hes$mb#`T`}NZcnC0pz$dZaG9}H;0S(4FIKzbddcVeIV1!g-_D^ zAN%My%gUD35!vvASM;$cSBNKTP)L?eGgqiI@W77gTcRJ~{h&(!ihM7gm^c>?POW-cFI~XlqpdopCby3tWe8&0HylxvMsr?^z)IV}ppucVe4^&n4S|)7oi zhKy)o7F*MoWDyS-7qSb`#lP(*tZfc|JE55@sP7y*=ES>Xz)Hmj+~*m1?!V0TfFQ~N zqS8so6-Xfilb;Jy38$xjQK`-*Ei8g^X$@3SVKbVddH$6JxbsWs5`22lgX{08U6KLk zC3kBi>;Lfprcn(d{?VwEu-k#Ak_$9xz9NVapH+o=5{4{IZMh&0-&uLPt+iB1 za+R_?%%*yoobSWI??NG>jAco*B4w{tLRNJL?xZ`UbWSIBy(jJi15EYFo~Ig38k0XR z6GCy@8$nZxRChRe$k}&8f$u7VUq8$H8|rT+8o?^cM#1^FM)K)_W|No%Q2K1_p<0a% zk<&+13WjhfW_L7;4aB=~iH=l({UI&Dp|r8SV0~-Kr0~w&jBtt5N7E}eX0alqNuo+H zbQ3ul6Qb4j!MZ~Ir|LY1?taxk&~iRbT%&0oO_|9_+QFVvV-pscGz?Nb@oNN9DfGl` z|Fb!67A=5zvttJ@)-RYfn!=H9ck#X>TN|eOR_ErmdeX22@5QIbnUNk$_rWW7)5m8F z>ars%x6(h#`Dacxw2y_B9OG>R27mJ~&8`Z!#cJXnCS4biSH!ZP`kCkrE0<1aXdY1IRrP@qsIRq9PdH1I5k&N7Q)p-zAjz?{G>(wh$ zY?S;V_HXZQ7{1*B{dR`k30ohxF2lr_Nn+R_&IDPH>WPv?9k@4DMuh@?~i4CF; z&DmIeOG;q(qDl*lxY&QEO0TS1+!}0FD*I^RbK;Rem92CmM!YnsnnWYVAZymaU2!Um z1bAw&A86{mmT&K4`o{KIstLuZm1^UBp6C6!Twcuh0XdseL0Gpqdn?d%I*43D$eT2_ zqdUy5pugQ2Egu9x)9{J00fYSv12N+lM>$R9$L!4u^Ed)wQi={g-}1FhiFG-sX1iG5j3*f_3G+rWFgG!P<87M`};K&pR(=b z2uLw}gkz&#>eofNM{=p{BKB^-qHykOn_R5r9en!tX-C@1y)O&^#T?akk!Rx%Sd2`0 zW8{T`TN`3ubB-B`YC^oE9PwV)G#BozDi3w9m&4!!wVluIIm9Nha@W zuGiYzz-qmpW+vk7&}5JbWg-Uu_YOQeH-ujuHT9iz?234cFz^__^msxkyR?r3Vsi%k zlSd*^I?C(l-WqTeEMnT@QE0`sIfFmfn>s(qk2@vp&+6mFru(9LEaRS>YHBO4>p3=k zyzD5ORGYQZcmg$*8){J6aF>{7Q|8aA{w9X>Th~^+96_UaakgiYt9O)7-o2OFg+t=X z`U{PAOb57kgqn0vK~tN}w|?~cB6QDe8+ax1yz9$!H(l!^7Rgz za<4lce5ikJQ54@Ub=iPB_dzM8&G(M%l3+S^N13Y3Ty$VLhk|p(y8d5|FHl?9NYstV zE?tY15Z0jU?p_r6TqKG|rO?%11GS^VJy#F}iyiekkLqF@e>dFkDTAGtesYXKC)M}` z+$VyJ{BEZlw@vEWL0^|? zsmzJRN+m=bYgqP^8@3G3VgjG#XVtwL@gLxe>hZkpIh*<@5^r<7UhH0bw|wNgNcj+q zQLKo|A9M{}4|-b}dyJ_tGDL2;ugyM*%a)^n4;U#byC`26#&eUz<)D9zDyj)=q)9_> z8t@D@tU!h?#BD~cPWeXPoQg`DSZBF+QTmL3R9lnz-DT*WKsV|Nck(T%(>=*_RMUaV zT^Dc#;tiLc%V*7+AI9?>RM$DyWHg?P3IM)yxBgJmX`0m%oZ3nb{c2$`p#`{aPr6iS zeZQ9%-SM*iQU@7n<1k+HP9L9U^gKr#P=LSv5c}j}jq`)eH@wnYo|ds1MSsmsM2@)Q zx5*9KuyoR-Qt0)6(*uL;vWUO;S!Z_H^zQyI;>0W`ux6 zPl~@zim#kG#^3C#@0UUUus0uVE3kj%n0i=SJ5%{$9c(x)O*ja=w)uCC&)?f$^}tn$ z9NU(MU-0*e-A53D7VCPgmNRvs%4G%PBT1oP*i_Y5BWR`U?_1YWF0V;j1#&KwKC4CT zZ+7?G_12nh@ij7eyxCRM^D-%}9tdds^yrBp?Xu1x5O>b*8g1}d+^n9v{ZmUnh6}IL zw6e~P8Egu8+9xd1CTcKp@eLFp%heQHCUC-J_Tg~CYEL3yWx+c}+D_Rc`|bf=tEW@D zA+~py6yCn|8T3#BbHu&%C(5@7AfdHiQRNifMTJ%wL39y=kMQg~5@0l;Kfds3Im0RO zJNeH8Wy{6xe0TW_IJn#yB7&2abhT1N_FiW+86cv^J`h8I!J&ii9^h|%HhH{%lMEss=-uA=r`axb zT63ajD9%kTp~|=#yV<8S(|)@v4D`r?0D{?~y_VUj;5 z0$~5F4Vfs3Zek$Bzn^dt9n{ZKc;v%_U>CMDd+A z7EMc=HH3*^a_OVWS78pG6(92WGFV`p9w(jSD+gKjD zsc@na`Z-jl!}4KT&-*{-F_-r}=l|Wu|GAxTP2MA!#)t9RZ(`dDLb(86rc8d~=N8^5 zp}uW><@ndh|MbHhlveoBbK9g;uC-@UmRJToWHK9atU1R(*0 za<0bN|Dc8(>83uwM)m3aCjZ<2<&SW0M z;o9qRlgHU?;cweCbq>9)ua*A(wVpJ}o!I-*@UFVZ;D1A={dl=?@5?R9q2VJ-t52l+ z9*j<7j>u2r1-x3}pTpkF&s{y1{Cio*n_~+f=#R&(8$vNE z$;X@R{XYM5p?D4IW=lKUMIj+ZF^J<&_4pG z_9jOG@ZV0z=n!F1Cizx;HstlvPVO@4YV7gLTKCtmeJ0d?K8tQBJ~Y^2+`AS&zYF=SrL*zOyJ4%74BxSSBbqJMe%q-Onp2cb4) zcxez!)Ns~yGxgfo`fG20K^k@$6s}(Q|4k82mhhob%!CT}7uoC7Yrk`c=Vk@TyWV!2 z^Zy%VLY(TY$Z&M+h-fFE+yBhOf6?dX_d1j(YGO#m_JQ`c{_pFy2b z^W&Vwi%&Z_E>0zEg4R=_1D2oxjVyo4&2E4tM1n$FPU+b~Lv`6i5>XTI4eM9e5e88J z+uf!A*VUEBL%n|gTPbNFR1%pkA$2FEtc{t9NMdBkEz6W8vZRdNER|c5#$3w~V@YF; zOb8{R$TAtb3}Pz#M9hqBR=>~O@2%$k=FfS}JkRIzJmcsb4?ns09idHv(BFB3OM=4Y^A;*as;`vv3JN&hNst^jIOoE)~A+z;BP&Ejd}X#=EHoKA$pl$F=oqd_js$u zrjO3=@R;%PNz2qJ?{Z3T?T;C}CF1b6I0y~>1dS1r7WzBkJ)Q|6Y&vyN3?`z6xpDr?xa9*bk54bgnaozcjv zAJT8X|Gh8rs?WUKDB-K7s^xEYwBP^9B51(x)$*=rDP2Q7lHlh5dXx4Iw{yeOqr#l{ zjk1*Y*|8ONUcl4qoae>##C@?cHa{Ho%-7qK?L291WO8 zM8h?L9S+dvQVsF!+&90(MEZ91?u6#kJ{OfQRB8k6xh}L_&UI`?WO#m91oQc#9*WUV zz{AyFlPx&7*OG0JS`jX+Tq#{A%dz3e2y}-M8)J$$w(|1hS@Skq-zAQ%S6xygoqGUA z)m*xl|4F{hTbcNm__A2XotXG8T9}x1s>D|y7ocvI;j#9DMHEnDQ1|)YRpSDkY$=Fp ztWKK3Je4UwWvYogbJ)F33>DI4)=Js54Ua-f&go2cFqt}&WF}J%@mKqMS@-nw{_wd} zrpm7PuI|XIHMx%Kr66C!c9-VXEoP>g2G5A=6#GPdjdp7OINSB*D}&45(Sbp&?eEuw zNu807Ir=1WF;j7Uu-)yPKkQAnrzTmo4Z|rWzTqC_%J%*rQBR&viK~8(D$o%KNK6@t zA4-9Tz6hvGcHn)%p?Gt!Pjb^@cBXS9#BHOp`lD%w6!V#4@6exUXe=`9b)0@b|1TcG z?0tv~AU$L(>Ne||ly_(yI3n9YgCjN!V6HrB&EBsGhN&EIB2EG)bOTbbaGph%4DMVm zQJeaq;nnxMNB%nd%8O7V@Jl=qALHuhkZ}YwEI!6137vkDZ%5-FR5lZ14T6WeMWII@ zuH=3H5IPd>rK2YtKR^fzGlbN?5=ejITC}J0Z^xe`X=7J~0;#>8;&O-Dc6iP* zmDgp2AALq;f9YwZmyqz9Sq4$F0Q;ex)D{pPl(n>I*2;$$n!2h})RTHVnEN=og86~c z_fX1h+9Mr+UH&wRUQs6|R@=S+-a=Oe;Md|qfh1SvUwfw$+@);B3<(YiV}#9+GEjJ2 zix<`c1R}Er&K{1+f)K@@Xkp`{DgZ+uH2}=ZI~I7+4b-adq*1-D+UMOV&e=D$aVs68 z=V|NFX=+nZJh7&$P)u_r|Fvg9|FAfmh|N9~5r;ZXfX2ytVA*XAy7L=hK{0hXFv`$e z@2>~XH{Z)P1_+p!5KH6tt|Kt|$A+e}ePn)VD8LhyZzs~P?oPo;2b}d4!!Lh8h^ETx zsAF%5d^;^m{rWR1@Ti&mKaxHr3zf0GWS(D$xIs|;oA%j=9p-T~E86F6dM&G^bH^V8 z@QYIkn3W)HMC5G6=D9SB(^8lAYp#BD3b0z0A$x|&xjS)%)z5U6d)(wJIJOht$aT7pUB{mz$qh>3G$zZjGObS(fJaDYjMRp)_};xkCT%y;8JWg&L< zeV_JQ-i$ZPBd_-!Pw_FWZ1hsIJVj~~rCzouc}CX#+LXAc`tlBFB(v8JZWZd7iDI?B ze|1Uuxn4&AGeFL3V?v(fE%5A_hcPI*pnU;S%~xTb1*RdEFiZ36x(?`YOY4=&5c)r*R%SZtJ5Rx6X7>I-A{|^z~0|)Eo!&HLerkGm^SwoJ= ztj?Aql-85jErb6Ry|zh!@x)c>b&Qgyef?Kjhu{w|-W~@4Ej*?$ZLda33?sz}lHjQ_ z9%Dsz$UqOSeww&pVIr;ZeOfsD|tG)BM!WDKCZDTXh)1n?$^^4*Rfd=q@z_?703#Aodq8DNKV3d= zi34kccX-eVDN|*dFh)LPdzSOUsRkMCae^z_JK=_yNO()A=t{It=-H_jn{TaOs`??= zFN=PXCPClOxe;(;?w+{^vi6PP;qYB`?DpdzpqJk|g#lf3<@}@&QSvGBYG-e_BmDk# z#~*BEzTkIYk0vIVf1E0&vURN=#jXRqh(^7L+2uCxHt6cA_2}tKyZ7!dTVo2uu%ta6 zcm9+Y18s@6p@i^=>p!-{ZA}OpN(3;wQC5(tct#v?E=o+?N&O^!O8vI=){v49`AQcT z|Gd%)jU=IYW2*883Ji(Zqc>~BwMJ&`+AUJlvuD7u`O)+4`bp@Juc3 zW?N*J^_Ismw~{Hz3nYc+r{J1PuKuB7Z?cY-h8sD#fvBrDl_5%3#5#vEHr2)N(S5Nb<=D{el8lovn55;E;`H!h2-y&G?gK| zgL8o3VrkZ{-^&OTL`27y-?9Fz%+$1`$iT2^B|-7@rUTOlG-$NtrFTY-L@17`B4 z@i+S%v~kS%sGG#KYbCB}lcTQ;t{jqni8F}8ec*=fKOg_j;JB2tt2MBXP+Td39W=Qx zpn9A|$j$8h|CAM3m1J1c&IW=xJF{GLM!@ctuHkOcX?U5jcjoJDdo0<-54S~pjnRb4 zwQVb2Ba^zzLKMzW#$`<9XRnEVV=d-UssIkXI(y+0B+knk;GlrEN}ZDf?b~UQZnM}B zYdsY?_a-zQd$iDI%sy(TXXW6dYx!b6#OFV@6^Q<xb6Lr$t<@A+l5f|*_XUU}@P{OZwjKYe|XuBn`sh+}m_EKeXm*af%l|6qn!y+4cm zA^tZMWdE5fQ%{@9_C?jS3A*5DLJBI(L9=FBFHtf8LbFS`v5}TCult16uo-Jb@9u3O zbAzXSo<0|U%$>Ihb=rZ$j^_^fEkPiGJ&wvvGou2{ihY!w~DJEeTNR%P<0 zw8pXq5v+vB>I8&pg z#45K$2Vzoo!ae`#(M0*ffN5Q`9W!HP#uM{=L=+#xe!&Mk;KMiTfve|D&6ekGoJl_) z3>Ri9;yUuKh~q3|TMPf;CDe)B&N2L3M{~wAd*K+Cy1|r8Ig$MnO>Ve~T+eSukJN0@ z=Ab!Un{zv^&fL2Pa-|$J=1{h1o_6SKHMrj|rlWCDCo-_(-!`>eB)5%)?~sj0sUCdZ z%2T@Vop9PLYD_(RtF(-RhC05(4BTM%o$Ts;!WL0a+%E2_#N7dgpngbwoicILT3!0S z-vlNIhd0-_xdF}FPb6W&yjQYQIInl))ohP82w-29hCXmq^37^X$M~(6(9trP)qK=S zFZA!B89#FuwVh69lmVf zKKIfxb#=YeTn)Q@Az(bIpLsa&3s@9Qj;L%@$Y$mg!a()4^ z?s*4HVS0IC{q+KRoRHZbw z1G7{Pda4GIkF&B#j#!baOQ+LZ`&McZsS{bW4VZ-5vZPD&({D?b^ z(gUL6!impB>0VE*3L`BL;i$`Wi$7-(=7`}q4q}ztAFJRR_T}k!hSJOn7%i`z2I9Q+ zh=DOcw%Cg7S)-QSOKPxL5}R_6>@4bJRlUOPuY+DW5;|ihc(Iilxnrpi#(mCfh6#^k zENhwcKqr+^$BemWyHGHM#2#MT0 z||>{!&QH484Ik zJ_k%NOMB$!@|I{b+3kP%wVvzn3n9@BnfPjhD=^nK3;Ij_l*=F<%eDaG+`oe4mY&q- zCpn}+QvI|{a!Gum#pw;X>89D~388_pkG>Ip%b34?P9uUaIo9BGDOI*`@EsmUbL}Pv z_i{-Jmu5xs6;$Et@7d_k)J?b-xAnMeAftHubrKYVvRPHm!j2^`Fu$CoOQjhNSgC}B zF~Pxym{F~A*j`wU&?joH#3to=)@TnNJGdk&q--)PBSSTO?XjwXqQ%jQj?N;3{jpNK z_^eU1i@*f91B_w1WLdr;7?tpqI1AGWxnz1TM#*)dDsI+$wVKOtZ5X*ZPU5E>8U0z^)KD6%BhpVey5qkg0ZZ`tQwXid{anM zL+-1!A!=BVXtufFLj}Qcd?wA?$)k5}g^JTEg)z+aQIqHjLsrT{-|R{p`Vdl|pXf!$ z*s^^2V8OTf)|`^FRWu%-OU@=?mL}74iXgwe=g{!h-eCOo zu&%8xU-N4QcYnj>1%AQLR zc1UHiI;ZSk&3EREb(t{8i;ZR#;d?M1Q;MC-ePkAL#RUk6Iq^^CRZ8R6P)c|XdTvo2_M@o7I7D#%ONX$tG(fSc*t4@L44g<9e1<=$|VUH z$eFfQd)U|{BQNU}`kD&9t(HG^_q&qUKFkSxnNInJ?%nmf4VoPfftg9sSy6T$G$wQx z+({o3%R0Mi0}DSY2udNm(s;D}16OZ~S#>U~nJfe|?E(l6RrcXu@*yuMIa>$^=JXcb zj~`p~z~&(NEicc8bH-D&YnLj>THMYfRAD-1P~LSI@9OPQ21;X{uL9ma_ka_dkiu^P z8Yf{Yld`*Q!^5(Xfm%h!SOxasEUnGMU-5whGQ-_u{?udJ@z#7ligF8CPr9_uetC-S z9XZyFKaf(^Nl{ufAF<*T$`j0ZTEs`16KwFP_n?bFi4!k{D;D@^ZGN^ya=bR)6Q8_N z0STm(<@)6ml9Dyy%U9Sbd^^mw`kj6E?~ln}ylFE6^jDd;ksh^R`EtMXB)VZi=<4K7 z+S#Wey`a_kQP9&Gmb<%URz^!N*KX(kRE;rlT$yBd4HbKRp+zFnbE%Jjno5TbtUPa| zcQ-pCDp#BZ_B(kM?JZCA^lU5f7{-V>Z1w_(UilovVahBeM;iZ@p(jz^X)+@j4PNb@ z=_Fydd3NpRlZPdkscU_678W zHJKblWNsmCSUnHp8@y^n^fcRrGH?N<)T_^&TnZ)kaQeH*Iu$~n&_ZELW0Uhxk4hO5 zHBQiXtILDKA7GJ7RSVNTp@VS;5@~pUo+YhEk+VcDg2&+i;I11?M))3gLZTupR4^k7 zyvyYdtI7yPwEqOL8h#ti7|9o|#0@eGlabLb5Ut?Pt(Fcc!arBjpu-OY&H^nv1t&vu z_x{J{nlRod^3pUpBgLbCg(S54My_qm�(zOI)H}L4^1BkprC;eIuWdDqTj7Tx`e` z==xPPJyb_Gy9$_FGmbBjY6TDcQ9bQR(Fr)TCo6#^Av0LAweNcxhLGrp^)u7njx9!{ zTD{4xm|b)RYOCdy04AISvgBWy8!QFk!Ebtv-*`1e!Bk1P(N7_VbM7#2KR@OM+$j3~ zJM9}|^5h9?&0JY&VMLnc4c*dp*+Mw6y?ldXln-Pqkw=ttSd z%Wj3QKU$;iBB@#9qsF0F4%qHt=63tkB|9or0J0UR7s$Hs#&F7d|fUGld;)$ zC((PnpOr=qZZ?}r5c)Iw8F6?^WNS8n?uROl@EPuuQ>&A|Z&w%jJ)bwNag+k?gM!FQ i$N!Ca-^kp^$J*M__t@pvy!{RMSe&*tEje}h&i?^bvP?Jt literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_3.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_3.png new file mode 100644 index 0000000000000000000000000000000000000000..7f917e7a97379faf861f8cce2dd099f4846f76e1 GIT binary patch literal 106171 zcmaHTbwCwS_pJyLg0yru(x7yABi$t((%necr8_018>B%HF5S{xQo6g}@RfeQ_kQm$ zxij~kIdf*8z1LoAjbR_A`V$D zu$IJIVxkda%b4R+I3F4|&=qAIQB~egCpoStonG))ax-`wUq4-+E#EC)`4L6;YV^O# z(W#dGz`mugrFYVTAG7+Cxu9%EwU8nn;ayjwi^LgG+*k_&3!SG{M%40qmuMHS(t5|8 zeOqt!^G!t6%<}-&GPcQ=`6TuZgP*=w^#o8ch_y1!R^5o&jr@*qk&})>g`j(0lGk(2?GM1U2O0RtnN8%pvG9v$YDips~2qE8h#sFXB0pTuKG za!I(TUs059kQnxJnH}Hls>_tOC8xWXnY#Wc-fCjIa#2^$^k~^V=d#T#$y$3VP??_a zeY%4Te5EZ+$@Q3NSytsa>!_^kc=Wh99b{4`7mvJ#bj>$5E_={DIkY+8DRV(eKsLr7 zduvDewTJod_M?7oK}H}>lXBB*5lni+%-u*w;K^cS95A^Y*Ji@e_jUrewJ_=$q~WS} z2i8F%`u)dK$_lD%qr8qErd5>RR`)&}eI|4!(5e?P8Rfgu`EXD-sWvIgHvGMLFnM_( zCqrIi;Oqw?lL{Xn-!yQQtu41=@QOk0ou;i>?ODa1c(A&jdx6K`r08@y$2uJceu-L>ruNuvGvPLGo63Af>#8o>3QmFGt|{e z?lqD=AcnDAK&qlXycfxw#7aJch#u<#K+~#J!3Q^N2nVdaGDyYR*)(4Lv$T%Y> znQ>}rUlhw9?8C}z_OTJXK0zXm5LSJYXmZ>rD ztkqNYTIc5RzRjCC=`A{*tksX9X4e{+y^n$AYm;e5!(@wB6P?qh~le3O*9_RT)r-6upWbIqi-hdcfYhBijH-Eki{4_-a zMl_luA}wq7bglVcLp=gNkO0>qdhQ8G=574<_<{zfBx5#n8ym4Smj4>bJg75<%9zOa z{-hPTL3MrI?Y|}@NWz?eTIHJhP(t?VpR+{rg7FxDl@e1)e>dla(6Vf8M0I0*X|g`n zYbArr{vlDPGNV^zciX8w6KQ}3AA>#Vhy#~5#(H_yS{XtnQp#;WqELEPAkR4qQz0;~ zY5Y;Wr9j<|B;al$#baTl8L9j*nS1r)sCIImHm=TONoMA4f)y@tZZ3E{CEU9F(2VP- zS&cQprLMcnyUH+&H>x7*ckh%?&zN;s~DRHnQ%BKIB3_4==K&hM@Byfhxx@2nSy>@ zZCxHA@-4LPQrYNv0Wh+y$24pm5m!2q7FO`w@Qs?r?}$m_t#m_GyX``d z=}b!ncve0*u1Fa-bEL5(ude%I_>h`wu>N___Jq74k9Vw`zbcS!<}+Qri@0?lWtmF@ zIrqy0W5t;VBRuMM<})C9&bm^O8{kvXJ(@&QHTA<$DVLS)&v?v#SnG+avZCT}K2Z0u4raHRcPBA8MZ+t?VJzx?V3DPiAYNgd8v`} zA1iEuC_$d#XvrR+nfb?nN+8xfAaQB5ux?nnE?;Es+IiQyxX=ZhdeVs$xis(`>2!V7 z<>6i(t3cjteEw95i?yi(e0#^ji;wAR<{0z7M{OWlJYFS(dxYS@th>AR^mr7~Zl`E{ zd>5TwTX2&m5BD@wfCF^3UO9U2Y-vT>^f*FDdtBE8D#DaCbmQ_b#Xca71x9&LU0GH3 z`}?J50Qpfs?Ym}2nbBN{!bQVq=L_XMJvn*YNICD^R3zj5&pa|6YHD=!n=I_NGQl>7 z)HjPLiXOuo^qfu;LFMyv=Zko!2YnROurax zH#eiEzbNQBMWVJUy_+k}Trw!WcMBFF1p-$Q$4Ue#$c)o7rk#PivaWK`YCSKpJ&`(V z^+|*tIA1+x#?9BgY5i*p$bHXw(y_=aUFZ3a#180H+{4syZGqMu?W9F>!Y?&8Ni>$E z?Xae`DnAm^f{X`k$8a-XKi6_+#P>CsLboIc1*?&6R}_i%Pn1O}N!S{bz~NG{FE)Y4 zDl#{0UE9Gct9$M53)?%JCm-lp6E5Emd;~TA(0Y_Vpg%C5E(%B5{itvBj0tgV6PQ||cB(kjn>64=M(6|iHh5}*`*qTR-N%sXFx1aUx6;8bSMeE{PJz7Tw zvdm-gs6F8;VN#Y%uQ?spAfKgkyE2aq&wM|#M%L>5#On}Xf?Eq0v>cG@^Vmw3w`JOf zb8>`-W*n?NY+n0qejjx3l^Zo$Dpa#~oCI`BnZObt)8fg8eV6-&UFF^V2s2JTqYRGj zT!T4z*aIMu%o3rF9zU%bSR+lDane%%yp9BpJ?1ds;2+%EH5aBLD+yEcsf@Uu-DfP0 z+Tn~9W!@hPCq#?c-FL_#r0!L;R5V}w(oV;tVRXfC0-M%GjwE+knwDz5tU-xM#}&#P zp;IP$@qU50*PWHQy?=JUYhI3Uz??FVJ0DxmjT9&@R((BdIrl)(&&$jePMe;@KG+wB z4G@yBVn1$96w*$s?>ezD9;4QUM22o7;!5L|8&ec@N&Zq36)kdc;?_4dpJC#Q0guxw zeZ!}hAuONQ9BGUf;)xNC*OtJ+C!0+xm;fy|)Dj}^dY}BnuFbM6BG*b(G`4ouyUDzb z(!3Fs9F)YcRd*^TQt$Gaog&Y@qwR-%oZX>6JpQGpqKU+VOE=2iXXS zfYQNn5UASX`Sm%l4e8O)MvuRqAzkg~8F}Q{uH*8%SOd?39{D+NMR>dW)>d#A`%#h| zIeEKR1tIKKR@JKFQFyB5RX<)c1ZbwAvjQt&==}4ox(Xsg-A83s=+1Wr+p;4)vzEWE z$uRP2Q=yecu6{nrv!06)W)9%MvmymP3c^}ATn_q^KqJ8pJfvh?y+npKv_^u#g_ZWT zQ=l@-#(`gVBmuARBL(uap!SM`b2yM9ADK~mdj(^@G98Ba$jw}eLxMqI5hKA_H0V`5 zq`+!4*2sf+@2}*K>JvS_5nEdt``l5Zo`+4;@#^rYYQ?mqOA&c_3 zFQ&FyC$MVG$)axm|NZ8@L{1Fjb)xw4&&zk9HlvxPf5qsb8#UaCD}Ful31apQBf1OhZ)YU$QBQqJe&66y*94^ET!$iD`U9}?B|iZ7R3A-hMuPoC|) z#CzW}OMStr#`wuB?SHS?SO~mmpe*qhT(N&aI+&N;Z=qt}g%G-EJd#ME62TGJ3D)<5U~nJUnKcm`jV>OWZ4&**uo0xg5w#i0IU&Fuo@QBBu#%971dW!=Y=rAKc(seE{rVGV(Z2c{{-)PDJ_PJcqLa}9q^%OJtqeBbKx~1>>-E%p z_nXb+QZ6N~+F^F>$`ORO5fHMIKOfq&Kj|Wv3Zwl!3kUo)n9*#*lZpJ+JxO8Y(215z3<~$ zwtC0ybY;ns%R)Kc2TPTU@o1m-GxmLit}4LweCsdgtV*d>{7orwIw@zqMrCALf9Rfp`wTzjk-%e#49}|g?nLW8yb!2>(iA{M}OZW zCJa4s4Ml$(xlCjXb2|`1!>N**E^{WeZHBm$K=YN9%-n?KPoCE6KpHr>?0J-#Ok6D& z!`!l8BeM*7WoXoqipcbx1cPN4-MJg2Sm+pvANzhl+vjF8!u&?y^!H@;qf^r7STU@A z)>%Zeu291Pk13`f?VH_V#Y1S0}tgMs&iv@o3FOc>;5gdr>uIH2Bs zn%K(4+}(>_;J%f~5l@eAT_v%roRR64A$&_ZaiI59X#XsxGEQ@NOeeK3j z-^egGi@oD7TP+a*@x2+;KQ(P#9-w%)*$It8aD``CnRh8(@Equ=F((a$udp->2)Mpb zDDv09S&+xUN;#hPQ*S*OTBEl@eDuq)be&@F)e5gVo*vw>2CP-W-0bQ0i39eVuGb!?V$udi}!)$o!)>G4z+-u>@CWuap%o-A-twdLrkj zOa^VC4e?LsWpVmzR)g_PxiXlfyf;L8zKaXvB~EFWaS=U1|9O(A+YB$OLu75yBKQ6r z&0%;M^pQ3dl{yKIR0x%+9R1W(SjkAT%0wQxRkMpR5tPku8I;L7tnnYBOd8_9ve@1u z|5XWcDpBEu3oBzH9sb;H4jl|C$#XteclV6%*Uo$fJq&H<6I{0@!U$Q*Q-m|Hf4)pI zhTu)kL;@p#ab9FW_X#WKJU)TTVL4!3^^PlCI%LPTPiaD72>^)YzdLw;L z|6|lY!3ZqDFZdOeW4cFrospBLoe2VQ{9|hTl)R^{q*ej3N2Y==-*LY$mFwWriDd6F}KQ zo%ePD)DJ4__G`@E-Wka@qVc)o;Zg{UG=t3Sf6xwuRSDY& zr_syG@O=?N|6%rpGA)~?&0r6Zief{t_)+_1B=G)+?re0VXio?*h21Q95FdMZ0dxo=~*-=G$*fe~tEM*-&xDfC#kSj&e-3 zC`|2*ch(6Ua+3(zj^4`lZEs%rf>=^o+1~5{bU)-JdDvIDNsUV`+V!0;5mo5Q>y%Jg z+d5ep|DTWta+oKGrL@*n%7s1VJj>8%99cT zLk;n;f%N4_wa@U=L=r+QakVp07mev%mz-xT0S-4z-A^9gFP7uqOR_uQUEn>%UxoknNdbG8zIqK3kI zEU1w`j&p#}O{=Uj;}GV4lqc z<%TJ%%-!%a|4u>CoRDr^g`kr%Vti{zEDg4Wn)=>!bzCQ zW0A`oCBCHHk9Sw+=aFozud@c2E(X+dRW}j`yg!=8_Pfr+#xLq6Zf02F?|qxm9yo>6 zV^VzzugTM(y;1O5!_dsqagPwFD6$&`;Q#JS$D{KHpiSLvJMuo_T(A0RV$b>SxWd7L z=RF0JPe=RSuir;9Q1mOwFsOPTI%b!$&ztAnBVExa57f>%&eihq-j(G}m5@dhM*M75aCZNsGxae?HrMN-v`G9wEgSk*p7tsCv?yyDuS-k11|2oSs z&~!RL_b`aT?RikIesNI=)CJ&Fc65JH7Pq`}8Nt;;QG%wilHdA<-O$Y!(w5LAmD7QB zuO%$H&36**WC0KWm`DZr`OrN4ZtN(Re;Cwv@MRCXEmrU@x!s@^NbURecz2ZDs5jOj zBfwzvquxZ8{nFxLzPkBnrSW9N5y<9L=%%tn@l<-zKohmkbX?G|?6GILmFYUduYgNj z5nBQ5nq7c0C{@#Sp0)(`FCj;T^h1=s?xeROTQI=Tdg^#la`%sYq}{;OTz1NZ`Rj&6xOp1&+;A zrkW+gYY67;uNnFYfYC@#gb~+j8H}9iMiJ1tNxA6)lTqE9+I=U|tOe?%ns%YL1uCIw zKl31^L$6D2JubozxsoJcO%O ze?!x{`X_7rev3z@IurX6%V)!k;v=A29eQmFl=+cOmLCa~t!fC!3VatT&H#yEvVIsM zaBo--!a*_)dDG3({tavr@tg%U)XON z#5H6JMVBHH{NQ;z0%f2rY1+j%NI7bc91S6t01b)z^2a__(LIYa86oAX?iKfEaSvyS z-dE-9a)Wh={i$d4X8Kh}v+$vK4Kb{CR(PI7F7G3Z@oKManuWfXO>zoes z!Nx_)QYvAeILzp}nrz9ptZZi7&RJHdC$U#2n`YSeMkafb;&tUGp3T4TWqW)M(kQ+xR=_HTebR~!gN!`^Up?zLZ$lTl-uy@!cWt#nVvu)I=#+f7t0%t z^&x+~{qhkR(sk{|e?#C(MI0>e25=!~-#_%m3_kbn^}n>NHbWalkhv)K-W!R|f#Y&Z zbp|2N`>dA!jABX1w>1$IABG}4k}D8d|ICFAN+Gu=i^7pOdu^k)zt!{tzbhG`;AsmY z5LTLWHzq^$Hk{DC+fhRMay+JIq+gc{xo6>ymnSpJEZfAsDW~^%VS3Mc+jrqbcH47d zH&c--kibxRD~4|=k}sG8@_Sr&*W|x&yp3KLvFwivshP_I!m;)CeoG+RtY{!$8b2xK zHO@!rF2AIZRSJ)udf2o!#j18B#L9m5?{Dtku+Nf=9;bX}O(`%*(xhUF5dQXz*xa$C zy@{@oYE6=L8*V3(+Y<&&2tK}G?>51lHJ+8r`L=H>QmjPw%-CE2$&MV~ab5_Gsk#SU z(>-akEK4J8khp^69%r`xt;glvMDu>4< z3~`_}9(wNQ2yo-^34QMtq3&wgbBo4c(-D=6=rT$f zuH{SZMvA%SQ^)-#)0Sjbrpr9-0@H?xgauQ2NZSRlxp+tItRMiNmQ-ojQDwsE1yAmR z*kV2PO>vs|@i=6^th(d3jQj{u*nYD>tXzk5tiV1Kd+O%oZp^Is9TmEC0&+_)MS@o- zNyE^%1hFn(+CJ+DqZy(_!xjIX3SH|K-!w!BJ7=)vyAPF5@9CiH?F zud=gnx(W>@Ryv7rL4FD(BV(Z5862t+Qs~018YmJ(B~1t)GYx+6#Z*+O^CF**(gM0$ zDwGIo?nzmmi;~WI|2TGMal0-d)WL}4fsUa`nBK5DmZ#;QLEDD>2KNi*@ATp*)A%$> zXJyV^uFGeVP7GK4FA8}C-mZibIcjw7ww7?Ni}Cs+nQgl``)ZV1HQLm$l19C9r-ydi;zu=57q&szi2+8AImG(0&fY9|0utXy6;O z;MzvVwAd_0u}TtiPI;34-(%fK#`Oy`|u-Sm!V@OM)v6SV%#?y@s9XU zaY^QOKPUoGitMvRQ>vWRbWIX@PcB4jtT-k1>GIKd>a#dw_6ci9jKBvT?3w+v?5Ow$ z0Kgp?$;Q18iHo9Q4S56hF@}UmqBCI9qNYs2Z%_776lF*IeaB4n(IKN2OOwogJTE%) z0fETzVls2jW7G8S*RUNaq)zl_50=9Rl3UoLbdBC1bfIwZ-5_pJw)tOw4cNvtLLWCE zn}w;aF3BQCCsiT)^qS~gVw|*BfHmQ9j+0@Wmp1l;vB)o+Qc2YO?Sg(tO%RNC5J}TP z!vd?6QCzyUlcrQJg>YX=etex_Vm_QsG)ycQo?510n~EC>ig6cZ$A}fL8c~89YA0~N zzzi&-PZG9^a3X6KOW1V~#O%e)J?g8?r-3{$BfL9N>on2W@o=yYU4mE$R2p%Z^HuH{ zr6-&H&663oafACP-X;K4m)Zi<@!No2KNI??Nji&|G{TddpR*Ndn8OeKlFYNX^6=(F?$o(Wa(B6@+Ch2ySy31{))3{Mz7#McNc38UV4rhd$fGOzk%4 zeAFV>LVxx3F>aAU{N-s-S*}Uo{(z%?9Py4(u=EX5v}@7rA^>g%3gK7t2xdkTz?5v} zDchz}KI1$ZW@uuV3V)HI+sIHZ!CQ5bYA_;F7iKMUx(!e*eziX|S?@lt6;L#UeS0(N z1PYcS6-9uPGL8i6AmeOF8i%HSYWVUE$2bg(%?7!wDW={E7cnWELE^+0^4^`Noul%V z1O04Rut_(Ib(Ol?sD1v=o9= z%WVLIE%P>fsRws$->#EW%$kKlm5;@aD<>hZe2)Wk{90> zxo*Ss?Z_K55XUNrgb>AtGU3c%=oTXP?GO_qTpFWVNCOZm=}-!&dImvh+eh z_eI9~|K_gcPtiqj9jYI$q{#=g4jz2HNK)&PkA~vG{(H)CA35_!Or@}T;k(7JiOB@u3sWklW|bk2|IzjCX;;0Ygb^UJ%k0D{$ph~T6b_^#Ne%_ukQph(1J5bIGtPR=2F z$i=a)F8i!-_&JKg{2s$Kt^wqdrq8(6=&cj!={@;hEFJ zWzRz;F);?fc+FkcNs;5M_H;i$DC>=Ts0`=fCj$!Ai{RgDz4?JJSZucWSSLh!CqtAs zBC2lz(S;s^X;)+bM?Y5|b!?!tks3SYktCfVU=1UTbxQnEg^yHmRg8m71MU4i-348( zd_zoxt%zAHK9q?9-OXPA2WR(fI^vs{QAr$5Eq_I)Y#Lle=SL3t*_lBQpkq+OJYJpe zCOE#x;;6zUCsQsH!^BngC7BV!Vzd&xC85W8tw3qgr-An769cgd^N}2>%x_}1aY2Ls##{e7&BolqJ z%a0AZ_j_)np(19`9&W&_)jq{Z@c(a+Me`di!xu_NQ-<9AFeRQ6%=EYNV)q;>J!#oZUo~Hd$67N z$lIdc+N^a_;op1~#F#VH3iK#7_nr7z6cnNGsG@;p0wIyQMr+0NXa*rw>a+01CK8$+ zpJ7uO{7I^*LEH?VKatzZqOv;|6)iKyYbMDSLwc~&!zYb#h33?Sea3$#Nv6b7qiBb& zZ<@efP`reb>N#=B%9HxttKwRSHyT!T0*iZvpqC-Zj>mIKEpQ{1^$;Az$2=?dk8a~d zOEw=0=xF&l8Illx^EPrW?;va5nN~rSOiBX26m|yhUTJ!iZYRyiKQ(A z>inzs{_O_@#A95jk$#V`Z-UbA*;df-vhK)a`TD#!jNnMPhAKQR9TNy77^~oTkGOQR z4)&qg!yU7h;4vDIS^|}Op-#Na0yvKvJrV|~PFhVP>058>RhQTDxuZMDT8vB&`nP!Q;E@L?vbm=(94u>_pHh-s zQ|s28eh{;4Jl08*xufbu-Rad05cfetiRK`v-*Mp@9mG}V=ue>Be&;V|+G}~-OYh}f z5++Vy+4=qr%P4r1thjWW+BnJ{q``a{Np955vf140yDpb%!dncvRzyyXB)`C;+f@Qt zZL_$0Z@)B}zUEktc+Bx7hz~hbj`oOB-uwogYGi?WDoPP;)xVdqSFh}ItM~vKSKlBN z;1eRglas%QmD7vmIaeK*0=>6@61>*V9K*Zck|=7^N zqsLw^#WC2_WFl7jN-=5+tNAc@lyMKb)2}eGO6W2D2y)!iY5?6zyKi!DZJI1h4H}75 ziS0HZN)&+io0WpCWZw0KKlIq(q|Phq2@I!}40cknjkJzW|-pSa#lz9h9-0_g&8@RK5C!@W|(xRXIRfiau+H>^*$uc)wI7)St?L}}G zlfSHp)O8=cVolnnzKMM^f=d^7^7-klQ5}I}>o&^fuRRWV@%Dpgu=;>Hs9L>JUC*e^ zz6?toBvR zk`U&2u})cecENkG-mdHrRXqJ6>gGNxhC5wXU!ZW=^fZC@xcaMls&nrCRP;@~e`RV} zrZPPF8WlL@i`M;rv^S3sK{lS;YEgGB+_PcpAP4W#JuO04-P+TQtKuTl)AzrNx0oPY zan^OFtWMglw55+7;MuZBk9HTx-psq`0JA1IrDC57U0i!c)NNas^I;cu@3u!)#sQWN zd}Sk|Etz(;^;XS?%{2MV&`K#^w&{ zulV`=u`HIZWBHL5H0S>Sz&|&L3XdcP^SWaq(r7ig!>E5k@Ri|69l`;CD3v|~UIN|9 z@{F+tjlQ6S>hbbzw7Y5R`e`)`ts%crrq{1YjmlkPSfC$Wg8cp-@IP9y2&%u32$%%9 zmC#&?qEa#b9zxRgl3VGo`}@Ng{o|l_nBZ1&!IjBpMYaF^Oa6jy17)4Q{E;jFDC+;5 z1r^InzDv1cYMuvf(}2{~w6xFP*>tKyUH)_T-Gor-4+};%4*7sY_+gobi_6beY3o@D zY6FO=X+V6`cmI58pa967&L04C%$rM7q}Mk$>d)^mbbxQ?ttwUdV}a+rjJlsH9E}qI z%_RrC@1WWm*68XAE-kmG+f-&6<{C}NI(^)8Q2O0#w7F<^N6d-T$U}7SANj3UV&&N$ z^msGs3-Yx?fBaNOZc>=xFd(I2r812Kb+rcQJry9%wNN1iV}M3FDrN_4EkY2G>6Jwe zAwHX_tTUK2n>Buh>#re1C6A}c9V&or>c9`#PXjO=oenyEiP?EL z=%Y-upY^#25|G%To|LB z6h^5l@1j^S%pr+kvADD*ny{(Ob1grO(;m1{H4bt$IjaZiqhC*ycA#q-vsWCMgCFDJ3dq;Y+41obP1HQy!f~6wioN|tlsR3uwo$ZliJ zHmsu1QeA+a8{k)QY-t%sbXr1U8yYLQ)gvy#p!cWnv`u>RU0jrv#<7$ND2``24}AiUihL(M7}L^phckfM32DIFc;lH6e|hNI;n~_sQ2A`V zyB+a|1G)10z`Ptr6%6LRKkojA_(*UN9jIad^R1(ABS+Fv64!;9Fif#8!0nY{>DYDc zPRKBoN6N%6KN}Q@lC^D`OvJKfX-LnU-mU-Aw&~wAmnqF}e^!GunYwz8o@qYKLR04n z#H4V*y`bbi)bOscu`gESoNI1p(FH;8kdgr3nMCQ`xiHT;>|+y&@p8LF${Zk7&`I`bZ$%{ODB;aka|*dO1R%WjW+gW zZ2{|0nHe=nV)4kcemdu`)BtaU+_kR2uQ{9zMh}d_wT~sZBt7aUN;fWV2!hn7S$Xnr01p)tji{DWI(`pY{Sns=ZZVPw&_A;Ph@EMpaID(wyVWRIt z@oHu;coPLo;)cDQp3toL2V8wF=e8s_M>)HYZQHW0`L9u#kg zqTh*@Rjgz3=z{QJfm4WR0x?vUyXShwOy=pALKL~T4#aghqPQ|j(u7W*_1CnmFFqY= zOxA7_pq`yw1Fn|Gj|YGl+C*T252rvyn2TJqeb$RO@xyI0I#G`3fTSDRf|obR_-(&Z zoWnMBGYCnNq+{n5_mCSUG(>)(U$P<>n;;JxwFU6cTK*CdBTjU|ot&pHd6g8(c)`9) z{@E^){v21Qzmj)7{yv(TP(;315rs@YT<<$zOD*YmcsL59&}!BK8q-oP$gfO0!E5$O zL?=vdS@A38bKzij!AAt)LKkhMd4O@g^M+Urf0!AnZ6sGFlwoKxcHSa%jkIEu*jnB|(n;T4 z#{qZPgqKx^6^|Z8^EGgyp%!R|0?lC8S_oz;7(pYF<7A>r$$N>Q~A&z4>6d`Y5B2VqVbW21$x0Hf;}ib+6y zPab@(`Fv?6cJi3W)bOxD$7xJr_13ZTdAuXjfP$u$nknJK$r!geBFY;^LPic}RX?p> zsU|-@?Ml(gDeFi{jZqlK3Vt*%@w}bVyQlzcC3A+&>|B)`)MR@%n6zx#4w&kiCW-r! z&A&@9>!WgL5s(RNKub?D$a`@+j~aT>`CqnV`gc6Z`)dH^c}o($M6)@HhFX{TUx{qqIoZ` zCzby?PVhJuz_Di~MBw_A+`7Q(|A-guZ zXI@wH%_5CuJfm7FlwW_x*qhloX<`XD(62eiZYWf zEXEN_i|!}Lqj@u&ez0j$$1ZWc_YoZly^gJm&v=%~D;g`JpWpWVeQ6mT=EJRZp8iDk zN~|UHki}*P4Z&Y&CP0k1>-+Yr`~g{nhzUlDgNra;%R=ndC7GJ)y1TXZcTe?!b9+0F z9+P+vdXjY+T#vF?FkfPdyFj??D>&SgI#bZv)V<4qA_M#SB@%zI;lJ7O)eu32QI+Yq z{%V}mO9!DII}c81HYzfxR09Ft5m{1WAHX(YX0|lphh9K>8$jM^SmjTSzx_abE$Bke z$py*=-nC5So%K9wOXqhS8X!s^5ZALqZ5?>qR09;C4{fvrUC)*RclVe%6M1xrm@?{Y zF9#R5Iw^{n)&=WYIhTxmUGj-wHq)f^S~pdP3S@!%V@trSJ%o$~7q#9GA`ekdBsZbw z!md6A3@IOYj50DhD9nz_Mc3%-r&#+D@Zt%Zi{czrm+5mEB-1@7+mTEs#Io%SxWtnh z7WC}^eQI!ogc^}lr2W}yH2q8w9f*sDTGbu+Z{W}lQe;!gSj!$bBZn30A#cFN<*|D@ zh!0;o#&P-)NT5MID#RaRZyi3%eb*&-e$dF@8USB8Z%aiK85EJs)#Z0ge?QbUUdN%S zdcchQzKzM=&YX+{Pee3yI~?+G=*lJCdmXb46&METjeU?1ts$k^d9{A@?5sKTyybW) zq=L8*pB)H;=5B_BUl|IDU}~nph|#ufb9S~SGnzJR-Gv-1Zx=)pxy%|i$r^2#3EXK? zsYkl5i*o(LXFEBWA}Z^%il`4}ZI1&Zta}CL%b@asjXk<9k|;<+cV6{Wy2L@QnP4c} z{BT%dBlHAahS)~EQr`>Pkh_6x%Scn7t@E;94gnt`mvAuSyNh2JXatlV3B1d*)HScr z%G-|DwaKCl6vWr-D&VH+Mm;fh$iqYQP=i!v5C?8-C)k|Z1mPwTdG@PBe)XNND)^kn z%i<9`4zZ-RQo+5H)O={OW`IWXohKVp{*f=uxStCLwsXFuAuKC|DvI~CW3>jzEodkx zBV|@ZuzA$K%*o8WXf)$kQ5fR;oUpz|x5i|Q{Fmd=rV_pGqo_T8fdoSEtqB>-m(O5Z z{h;IN(H(YS{~R$abOruB)d-p}ynj~}2Zj}FlWH^1@R-A8YujtZ(^V8-P9EnBF19{+ z3TH=aPIs~dAS>E)iBsC=+bCSvOP~w0~ikuIv5-}(J#MwtMDcOw(WehKjM%!6= zG6(q0*iLcl|-NFrlN&SPm3D!mV^-08{DM8j|zbLTeb z!jSW@w_}_240t9H;R^w)8DsR@o~p~lsc+bk9G~Bko&zr^(CobmjNB#^ee+TqhIpcA zYC)EmHoF3gQgZa|6JD26!vGEIwf;`&jMM#?!B<~cbFD%<4ua_I)w*xqn#h1l89G^U#c0C+kTB-dI-rEZ9c7=`< zBHc)ewFh2?0kLTWi`?Z!L1RN1seb>8Ix*_~b3W1I-~SiDq2rt&!S{(^<9{{M4?&&U zwoyxEljsy6gawlUt!G5Q!%1aWyHQi@4|^ zGRg1*%ffVMlVFH!n+Ynx`z_s&q2gqQ`EPZD?xR}qj-lv-{-W5i!$`GzoMFdg9nn_d z!`29Xj!XF-o`!Gm5!mM^&A)TdVJNEY(z!Js>LQ51ya}l_P&w4Bby%An6MNzAHzY{* z$S&4z_e1Q*wwY6isQMVfZBBt4v1F`6DMj3b#&kzScZVIBMASu`4mqh0mV$BjCku;F z>#!8DeLM-}f|E?fk@Z%c$$bH(IX9B{=JOqni4^8#uz^=xxBY{5ThvCrLln7(KXS?5 zE0amYXGaK82m7t~3sTWYL=*g(C=Y+$3TK_RUCf=eS>QF9%oB9J(Dyb|62`P1Q0<2S zE>6+|Nuv_DKNYUh_3X-&YmTtW!a=9HJcgfcu-XK#ZkOcTGj}7-4$$m~&xx;Us!=$% z_|FY^_WFf5M&yV=2nMiJK><7YDOF*Bu+Sgo7ZUwpuPDPoTJOvPi~e7fN9VD~i*W;A z*gN(Vh6YSOe}>%kV5cGaZ@3d{)^7vD9alKE{QYej64`!|etMFO6leQ=iRVTmW;r{Q zN-Cyz=U zc_7Bf_T6-8SlF|^^tRvsp7Z%VB?sD(plELfPU#D8vO@VjGr8LNeQm{eyQ*qZD17Gz zmV+%0O{X;2up7)G6&1B3+;p}XEpGEN-GNI57EEtIa@E*e+USvc2SLR&6sHjsBpsK{ z@}6A!^XrYCq;Jv168umi>*Y2gYQQwF!yLa0$4xub*K9@CEQU~99SjEbW*dh2wcF6+ zm(Fo1co#)(ngY#cxLyj6m=6w|7!w#0KfuB$E&L;?T3lf}{q|@m%#YcJNlGS#zK!iL zCbt}2UqzFnjZn(yRB;y>bu@O%Ssig-%!IMR5?aj@f8~! z=oizm{mLlX%Sl+5DH7Mwm$CceG`@0MYA|obyw;di19@7Oc+OLtbp#(Hb4ANp>(<1& zySW0cAE2uPr-Uk+vyYiacJMufoRoE$%F;HzV)K$OD%1=d&7oOLkgue!EbnYZiLP-9 zn@&W0u(NuPiX?4VVVc~2UceNk{2qhsgPlx?K+0fs|3$VACYs4ScubBlwYfNXan|DX zn=I&8lWJE@1VVcV^NxO73ndW_T)_?rB){la@TsYZEyT3|UX1H#qvOKK zK=iB)m2c=e_MoNDt~F#zuAS)3$`~}7c!UENKHDVRo4%Ovn?7;T&3ErqMMppg(9n4B zTOY*HYu7F##>9zb%7AyBf(hPqyNftz;&s{A8B6M378wY3yCecGmCP0IjaZ4_@Z{g` zk3$58Y9WVFw|kdwQ=|EBw+RZmt1+Dq^kNk}xl-qI^fsac7ptv%99|tX5=VQI*L0Qf z3-__oW3W-ZVIZz12*^CIZmjktXZSx{y=7RGVcWGWozf{ibW1k~3>^}JAl=<1EhXJB zfRwaIcQ?`@-Q6uQbbJ@jd*5$-zxX*D6X$gv>sV`FW)ehJMGs?CJIm7V^pPpW*yZz) zIlG_cfXnaQ_})1Ks$zEXKHQVq8kZk%b(#7)a}#CoP`j_bDxry^mqP)PewBrG{p_;w zOmwfpM|&44hstimd<`hry~I7U6`Y^AS+(!gFS=NEB%SwLMC)Td=xgoDS`aB*m)~ts zyre$yz|&TnXZv0cdWnA`ZTsLi3}O|^C5$qgn=~OQG~uPHXPInQ3Hj&vwyhFy<(upM zw!aV4|H*tZ75sBY*QUdgI#o+_=2z^#xE;Q@`W)IKT8)3cQZwtxBt|o=w3~=5``@4h zCxa_kSeag2UlW1cYr&5e)b3JZfWlyQ&f~5mXzKUyrl5IO`6f`G0j%-lw;QoJZYpjV z;hOi{PfmmV-j=*lFy?)!G@V7YtN;oVl(I+g9p1b=>hL!^h@t8G_nZvKaHEn<{4?yW z|BQ_V42I_q{zZq!|3r_n;c!0md|!!&M%(+-Ig*r1YcxsG(pePF0)R(+vZl0Gxh z8-3MJbRSVNXY0-742&ulqN0t>eOf$#Q}lnn`rnaY^JHSj5IdgUEH0R4iP;jr5HGLf zlP+c*zft~BWu)6Lpbpv7=MpNcvb|%?t~#*gbMtOzYMHhvT&Rgr@5xcO*y3uaQtl9u z5us1sYtoOYavRuCcbG#PEKH@VtE*BPQqgSFPjp=+rDpo+L`)DR(6W72*rr{%NMSc) zUUNt~qIqq2l2C83Ky6Es?d7eb=enO0x;qe90q8tijyB1BF_=E^%O9^V2s>{7IMFD? ze=(R5yRtBqTUSPkixJ-7~e=TcI_H@L@Zc}BV zOkQ1$(t>xa;q}yLFTX{V_d~4IGy7abc&5-yO&@ydu)gg4P;=0uNOoCy%+?Z_|847R zt={03qaPE!iajYy4%W{Pm-)HErJFsGvPvw;a}IsG$c#oDWX6K}7R~CfDspE+zX!zx zSB6P?t&!d+jZq5B;SZN_k)rB}AJH{&Ja`GFA7u4g`mh`1GL`kq)p^Z=4r*n>;u-<> z@xH5VM62$fk(9ShG|+95Tlchf!mWbm%wxRs*U%_gqPaH~E*__*$KKVVLTSbJyW-k8 zw*;@GqsC8_3_&g1%n_pwYs2Rk9e$)|FeiH$OC2b7K}+nJfW zGKBk}t*8%Nq4xtG?U$Ll?qoVUzsEUHAAq2TqEQ6qcJz2g2|XsPbov6%)5x&_^Jbx+ z-hV@6M1II$rIq(-hDWTbbF?pxuMT(zmu#ojH0NBx4hL%%xFA7OU(2&9&V{_ipYM5M z>YORft*bhexR&`tqI;E!DQ4T!wCNnI&U&!~rD85R*oPR*{=DRbd~1&g-iP2#+-rK} z;B(p@NOSh^+q$=E5XVeWL)%B}`5@jUgae?u>KuA7dt4Ph$EqU5PDINiSCv<##(DIx zjLuHvRw6}!(Z>PDsIoTPfNugj`P3$00ZOG{Pv>$eD`X2?9LxNP6 zlv2$u)-#Y48Oh^(F9GP|1Q*9uGeH4@I?D;-1jXrS^6wl6*J6t^H%OIdJHQ z;-*PoTKXBk>bEt{6Tf!Hz17;;i za%{rS|63sq!h;#N%-%>~5xWk59mdMA4TD}26%Lt3VVT7CUXEca?fN`LpXyAy z!7q~hpw^>;x^MbM=4}_C@Zf(G#X+ebz{qcVEEE3R55r+Tx_`-T`cll6$Oz&E?`g5o za`xbGf&YjjktH2T8)^v^D|1>>-$`jFX;a&MK<3}$iOuzywwYqu2tX#mmJGvDXu^;^ zs>6m$&#y~>I8*%spHtGsN#^oliIl6x{=%COhMeF9g71)HS?lL>7vG<($G{7T?KqKD zcYo=x@Y!MZq@c)8t;@{~#*#C;9&XmWtN2Lr@0L0*O+JNG5Gxr~qdloIgaZU)VesFeKZN9kB{y|$!#Sf9bY zGF^QOi%d3N3KZzX6_a={s;v>rq8VT@AE@{##8$VlKCP)LZ(4;>n-#~AQs@=@6BKAz zBJ#gacY0V`o^eKZAnW}Tn4=UX9j{@$A~!y?h`RQEXrT_$As3X$MkSlzyQwl<2rF8v zJY|$``a}=g373s4G@Up=WEvGbjV0wQ1=3{Us*Ybo4A-ip%Jf*E*0SB)PPTHBC^WoU zs;%v_A=uff?7`pI_>zyfr<)%V46d3;{H(H?I=cs?!`<qQ5!SRPmPlMCeb3RKLHsw_I+CmYl!QH zB;N&FU%81$TZ!>MeO6AG38S<~WCUztQcDFxEUes*$7;Txog3q6uFoy&+}}AQlJhiOZq3HXmkf7#U94cEZFm zK&H~7^fuJNx#;ubyDmL9kEAp~#m&;f2M9>67yY;MuMp|L5+;GkHUhiIm@%yvV1G)gR?+ml6a09nsa4YcTThk- zY+|F?T$&q+Pkr_Wqo}hg?}#AmO&Al@5mGc^N36IKEb+<5ybyetAnrH=p;nNx>6m&r zm|qkdgPk71-ejW$mzXLSF5JWu2lNIzP&QtkLR_CE2^mtQ%C@(wlGGEa{+pL(SFQg4pARHNUi|1z_{cDLy+N zGnsWu@xNBm?}f4cmEs84pp@@v{m&7mr92ul(Exhu2e0_c3k_SW^9K0cIAE{KT%<_K zToDX1J>)<3UKFPcjBoW^a|`-F!$R*7BzJ`RwM?T5IIgeO9SOSwOjwvCk!p3r)Y}Xr ztZ&^SdMg{kWzn|-8qT=s`VxMjQ7GSOU6?`@^|kzhkg*thh-IXc1~8!ulG3%IzWVL) z*Ve`K5pL!9x2xiXSYBB{!7MBY_Wm8!bu#5@uBBBE}Xqa9C z@%$U9_;FY}qa67Jb*>}UJ6WRMSYO-Yr9fASuT)CgQt+%9$Z?v3B7g`Cb=EtSG^BFX zMM6>t!eUyxpWTlFY+)oPP6n@W1;yX|enX5!pFlq<`yHo0#|eORhC0vP@#;vBvB11$+$Z-(&V!BVJnC4Hp#mF0hMa;YVI(GxCr4kFM1>bOD)eRq}8n zHcC7-j6s}^5{7?|)~dyft~-eJBThk=k+k`|WirR7u|Ymg>^He|Son}5$J0!U_jg~{ z7uFEOgWgmxFiVAJRG~}XU1;~I&7Oa`2BkH9$dAKxp^F7j4NDw=&var2)KBu&X1&$x52IL+OkFv z`?IM>n@eRKqN>o~rcj_%%X-jb<)ggz;pg&Tco)T`T7}FDMv8#kl}cZP`tbF?h)|G_ z;eLDa!v{nZ@$UoI$QdE=b!>(-LD9cB^4P zspG|?J!B}$`xs*E&oex?WMWHy6a9n|6s~m4F;P$F?v;3KRdhnv;zj+@!8FX~0R!en zJ-gw?xaQo5<(=3gEMQ4);LCj;;5`}7x5qxUHG@_?k>L7DvU9s=l3dRE*ZirAWIHS* zd*94n=K4*XzRl3*pMcLj^F-p~z~-=$t;nJokGKO=qRlS=_C747;GYb&k}xdPqR~m^ zV^)gKm{CnAI_K^Hb?gWnWQut*e}ds(=e6N)u?#FyNbR;`?)EX9GE?T1K=|dT%A4QL z5r8RCcT1~jRv@NnyYXzuqV$$k+x%0)mYXCDv4rOiNvbK6+sx&32XiRtq6D$L8!;mZ zSW)2AUoR(xRyu~7B}osjs9DcIHU|{hEYGBMSGKHHzZFGkXJINY74t~_G~wjCt_lGa z2Z3hj)_hUx5ya-^&P=x6Fzl41;8@f$u0ep=?7_JHvS_yM%$-MyR0Stg!XJuqc%PV2H5Z58XbEU zXa4za=~kA4J+BayX^K+4BLK1)zHIi(EZ7~IS(R=n&Z=lwJCkEKB}!cEbpO3XH##(< zdE;qX-LRUzTCe?)^k6MNMfR#-OM_jk9g`LQiV%BVU$AS{EOZg}Tbrlxm;19Fk`*lC zHgN=)Ah0P~$x1^eENR?NCl3DU)XNJGvRDev zrO1{AgyoH!L$%h1(SEe1ids4al=tnY7Th)+eDz=2I{hoz%|)mRCD^C2iG9{W5vYQq zIOd6AKqgr9A#e6whxz;6`dcz!4>B)0b`ugPu_C*_nf8^NXgw^Jd|b zp?FO3sZ_d8eshGL%~mos4;mfiI(H;HY|h-w9onl29r_-FbH_wY@506UQHi%k)gjrQ zT%Y1N5C;GIfcc9KgAJjU9{F-|QT%adYOct4;P_~l81!Q<|GOOdnn;R(@Fc*z%}C)` zY8Zi^^s+*!_=Wqh8%|TVyufLJD}s#o@Jlc5dP;{uKqbaonk^AB$0lU`$xS@sA9uW3 z!V!gh|8aD^*B3RrYO_}g(X6~FtTB9m$sLDGnELMGky@LB_H{bMDLob2ld1Zq3-3<6hf9a&0>FlK%|f4ie|)}d)sIMLU6>e#6wC0P)5Q3fu2oU?!)Dd+ISMgUfm=nQ5F z;Ik8vIq&{clUB1OXyyW0r_6QhG8MB<-xE#)cRY&-G|O_3Joij1wrzo6N4lv4qzh%O z+2d6p&S=DNCb!H>+!Ul(5~oWG2TDmP^z_lC4m8fgY96UW{_5Rd0l|{_@7~M54yJf{ zb^2wofgN&`ldsH+v|2>e-wJsO0_^hfcX^<;E9R|P&!>}9*HP+&R8bw|&#EMk?D1~* zd7`aACjav*R_y``IrnBR=1{#hPI}p(No@EVk9eKsqG3$*@NB*?a=6Z$qmsZuLX@^;lB>@Rxn|;#xFiy8OEE zr`2DvwiZ9%UcqSVVt%&!#=~+ZOM^y6l|B6nDMgV+Lyvv_GBE}Om$nSY|F>=Qs(a@^ zsfH{6VJo`Ihfh_^6N9 zsXsJ%lQpzO5lviI9F#1KQoYsXX8R8zArV8NfOd{L@$%BtnWY@~zZL!FN&TO=r<%Vy zvyI5Q+GCWot}vOfR3i}p?p5QW0s6A7$0mT9fv;edt&B0BbmsNBZbF*G z>>+$B+?&K;!TXa!c6o*J{s24^ua-!q@f9Udw&q~aO7y-A)W^zZ|*OrgVc`4zejN} z2996_Rz#CBsB)cs)EWi0#RpR8FUr+XZjS>)PEwtYpI9fO+?vUaT4e=-zR(K42r64d$ByOUWBA&=)t2ELVa zd`QaPQtVY$Z9bi%t~jJ^84{U8fur^}MpiNDO-&VniooaoA^}II zI3)ez&5v_3ZKd{y+66mNkqicBh(`(Dl+euWiQY6VJ4E9uJV(KqXa~o(5wy`>ugM`% z60RmA|ET@}BVd%K#7J`fEh&d^X!tXhj1;@2$AtD(w@m%2;KA5|%l2E){aEet5dW?u zNDXVb7WYT4zpXbsq2^EbThA07#zgead^nfLuRSRzASw&N)EU8_AH` zOdg!_#Pj428Z?mQ@DeAEVeTg}e973ho<_SYVtCq{^xWnW#%`9O6F%4kI4V3wwWh0U z2@a%I7r;h6dxtV&4ezBFEUdU%*jnQb_wX9lDUejd68wqO48$J|ohjG5zSXM3;k5g&RC6Z{_32-PCFLr z4SMg1fqEffifFO6jV}|m)m25fd-gUmgA)Fu38(t=TrIbpDP_Hm&T^(vaWt~KRGXc+*HWNPDuqjoOcG8Vd@PRR`H-aT55`71d^J~Ki<{0# z#~`<8l86KqASDL%$BPd*z9G~(7V!*v1}tLib4N|K;n0_L9#z7e)U{_5 zjh-US84x=(UwL1rlJ`XZ$W4~5;h!c#ZxzE@lz10tj^LZc{R;`N22-Zy^K4qF++I;1 z&7r4QW-@1#iuOt-ea=!Q-D)73?^VI;b}3gH?BwEX2`D)dbYNs+EJP`agJF_FCrZGo zMH4AYVWKSonnF7)JwgC%ZrI+kv?iyoY6$t-D75>k$yb(;T}_;SN=BAq8%tt%9-1Ci zuqy2frVEa%Lh`sD;pZtlz>Q}P+u2OH2Nv`L_Rud7rp>>e&1w37>AYZgk?FnjA>g&n zasA4}A_}Z_*_B*EVWj2DK!+yY=Nl2&FqcMeUO67C4UimNn3-ud%Ot_(ViT_$Ru#goQU4_!9T}*hk5eXXy`MC8Uuhd`uSrp&9$bfZkTRP7d1WG?(2l9g{VQH- zav9~>YxI&@_?6N#1ha{?kodiols`W*%~7H9$0$=e%$}=Lu)+^I0ox2-8f}r}>+Y@u z&I{*#Lm2A0xI7wo9Uz0liXZT{Uqye6P?+_h+932H+HteDn`QE!L}#0UFLb7+EF2lX z3Zyndq8=?`8pOc4HKrwv7=|(52{*%Sf>$)Y0^VmOkolO(4lJ1UUyh*Brrr&9x};-k6#n|7I> zs||+*LZ?&eg3l zYP|FR^Wj&tCgidvN*d86NjelJP1rXLcMqm*_&C)chM$k4B=W0*MB>ng>C}4Es7bzY zqGOAMh3+M|m}r07BcNAM3$-8$jZEC;x3GWSw)CMK?QO}LSYvV`<@Y0H#gJf3KRIZ0 z_O0z_aV`lCp3JJA3ppB8o#gJrp9A)>%FGVlR^wXfTm||-I;m*oe{e

R?CiWjM4>1k9+IZW*Cl3X6EyIsxg;hJ&*-oK$e|S35?Hk zNDU=uEA|uYhGweiVK&Qz;fJGNN$~YDZK7ZQn=AyqT%y1cHLZzW!B#V=y@1w5n{Rg1 zD@FxOD*98rE?xsMkB0X#FjwKzFUTtE60VFw5j_7fT)b`&uS8&hq>TayWIuutwP5G4 zyusjv><6H9cG`?al=%uJ`uN8fJj#4AeE3y%gyB}mxh3vK-?{K^r(ofczIQ<44zhV6 z+!LUO@V@`I05il^Y_AUDViCLej@uuPl{aa7T{1|u`99ndn>Cje^od-~S^KxE^J0`b zJW9`uQ~CzA?#QhUYUeqJ3+ufrVA&ySFrp{6mDV+nw7X{JOs-MGLD~ZhYr_ic>y~Hl zFmf8e+6fzv=z|ICli5+s@Ee{%M&s_=n5hGgz0nlXQ_{}R$Dlp)ozO~A^JaK0n>$EQVau? z7mQPIeq0oVmpF|QR~wK78Q=4DWT8!pT@46rYXE0h5el$0aQl%!vAbty;nixYEQr$@ zisO^nz6xXieo@+)f&O$~uu;qP-$(Gi?EIob=HhSnjs>+(=eV|C;ARJKF%C4FpU^~} z2))ZjgL51 zbD7GQ-(=`oN5F9g5zRvYp>4MX@P^Q8cr#JGX(ZaBB>DPd;`*;G0iwqLE&UPXLjD zIO)#GgsFx<0#*QW8SA~E15DuqxEo2|NV+E}%o(WRgiLt=W7|6ULVIE&g&56I`rmZ&2V*<&4yZ&|zjGkwE6px5q{Da;Nuk0EQhoF6 zBj*}nfB;hAf9>yoz2S*2N#BwP)z=KB0!~Yc7+Jo-QYQMK34K|@ImRH#U4HLM^@O!a zGS8M`l8u~WwX=`|{3Y|q1?=lDlhTP9jH^pU3Is0n7{LOBu8?Vuq70l-V6(%j*uFb24wtw!6@bZls zcP>3|?pu>OBg*cB{35Ctl%_e(;Kgw-bpo{d4Ugi6eS{8Lg~X`2zH@1IMQAni2cxF(I}dj zEwVyLDDZ#(+>qfBL9podWAc0$3h`^`HZ{HH=dVll{fyd{`uN`WkxG6wL(zMS$y@L4 zj^_Ahht4w!1GE^;_A=W2!K2+b&~SkSmilzBO& z-#sGlm#pA`3hHnd8=_zOY^H^B-1rX3z_kMq6f*k5GT_Qd8&5qt8N452ZpUiIJF64p z@!)R^H7vQHziMw_qK~-|gB5u)kedF_toy(JaLat7-sI9vpd~hI8rZh4`2|zK&sKBE zi(3z-Mot^_waE3G5QPZ6P2pY!RQslimD(DaR7H{Gf2AmCM#TmcYya>2aE}N>H44F+ z*9o{J{hy#3q=8buyyf}+P>hkO4~L1Q#w#&eM4Z1H$~SgZ->y0L(b80-#QFP!lVJA+ z-c4-z>&}M&CLK-Zd!d2Up$P0P1Xv@OA8+b=`4rQqkpb}9aoqp^Yf``{$d&F`P|Bdf z-^0QTC$x$jzPhg6Btd4joFj~t+V2y3N!;nAs&n?gAGi(TRxjJB4e-n#e(2WCKHUO% zRwDCYOiv^R*wr5vHS4MH`%iC~TIKHy*sI{XTes@w@SL=5K6g+8=>BMALuI>0jWwst zE=z*(v4ay{3R@QD|3e%J=oG>>>`lA7Ml~hj27LQok#7=W%N&AC9u(&tdV*Q@)`8vY z(e01T)!8#Inz#l#Pj6*_S1Oql^P#fI!?!A1;9fLEBxr*-;K_xXOVL%CWu zS=50?XJ1pL3)l!1RzrACYK@kz5kN}S@Nx@ zI5MTtE4}s*2!xrmp8|~M!J4nY&QJH16RNDD9WU^|zjg47)Rci;`Jpt`0UUEFa(ClE z_|VTpUSz0_)FbAS?`GMhAwz&i$3VYECndVn)SeexYkcmwrP7nD-t=&HlXLBO{K1V|BYQS_>J2VM(O4Z`51riM!uW1Hm_(33QnnleD z0LiD$f)(cirTg@$4IdAKK*v|&9Y816!SWuf!ohex+XH^_5|K?keK;TYL2Ox{74uz^ z6dyxNx;s(O^+%8lecMmnc%6TT6=3rDxHdjj<00|N>k$3@ihAj$6Knt)Eh6ev{}vW_ zVLLY~kg0GlW72CjjX4NFpwi`&QDKvIVx+=ycVkr4hq_4GU^s*&kz0i zpGDS^*S^*Q8B$*6@{chqfL+RSZ~=$_>b+KZ*$vlTEnRRcbYL4mO)KKoD7EKu&J|0icP0zxr(b z4Xg^1?=`fZ?y85`N&tPb{&ANLTiX@BBY^Gi%PWc0$4>%f=`~_tNJ6BM?@p{=qmXJeT1T zlaf+?WYhLuAK^T2?e&vcqats{SFw~O{yrDYx0;H-Jj7g~TLy$+wFlp>KxF<8>{l-K z+o-SIvUaW(zxr$21f4k+h53(Bg7lRg%cNhi^jTj)Lbs$ud)Rmx-aC#l{L;v8fQJZu zaOZ1-oKlLpy@g`UVAc(C>gUP3qeK^r%>pb=knwbX*AGCY;9@|I2j1m|W}!A^&ylB_ zy>2Zg5nVlCn4YN_ToCu`?2P&velbDE*%d#9q`o)Z zFTkYb#l`qoC*CrRpw6|HK^r`&j`9#6zMmlPt(PZ!qEXTO43s^5Ce(EluhZ)G-`f5~ z5S29A)Xtb$K|iNs4MxN$e7X27dH3>k_M*_=zqv4S$6_XcRFi7P9)@1}a*yY&LopNE z9FEMvdsP5nc{vQW$d63w@f-JZBYzi9YE{P9h-wa?KmqtISR?k_;GV@%$x15rD@NypOZ#t>ICJaENFK_wxGF&(rjfFvP4luT(od-6*5-R}LIsKtI zj5|g6$XzvixeKbex~`YpqkV*aX}aUMdZ0Sb9g=)J#Of8W%IFo~m$Mfksut=VKjr3% z9A%gTNH5#1k5}c+@l1!4qIRGcU@5A@3Zw(I9|Jjj>Z@=Phemdj7OVYujoNFnkjA^gwArBYr?E_c<{NDVAvAR~; zuq_`?Q`hMacaN(FFI_8}6VR?;MJt+&SdsFsuq3sS6<9e zJZlDl^C~1PfiJNp_zF9%+X);Mhsja3ouCjy4d{{{1y2Y3QhyfIG4>7tu-KR>l0dCS z@84HCr5x?{TbUsBUeDKfWeIRT@r_SL9`DRQ`d^>IjU?oHQTQ`TO5?%P)F3mI%3{S+ z%72O@O`@^F#{LBEdJ_nSqr^dgSIrGHMUhv7(8$G;ru?K74-AyOzImA|`gy(NeWN_T zlD3l8v?n~C+0@XoxwJClwko)X=Lv*cCu_!cShHO`lhDx=lIZ5=n|5bP`A1XmF=wfh zKAO%%H|dv4CN)CmbSkb!g~PhR0^n_y6elgFtyh%~!VB`TeqiL$?1Rp^>GIFnut7JM zX5q`Tz03#pYC{pw)FRxvHd*thfq&|AY0A}(WYfogrg8}5FWY_{&x=Q%2!4aQW0O(= z5H{0&s`Vd!Kobjt{Qty9WtF#KciM%-*QuOdDYjQL6@GQgb6FnaaJlQ(j~Qa=**Zh_ zK&DMqR%Sc=YuqFi)lR0E*xZmaM$`IotEZxtC@y1?Ph|C}=S}egkz4>ew6vu^9wSQ$ zCL@IVD>w617JiG5hH(mh+LbKYj@B~t800Qu(}w^NiE*jpI$h2T$2SRZPY`7LTE9Kf z4^ZQ(GWght<_zid$OzdQ*8{6(T14Lw3Oj6PzxdF-9u);^Zo-_(-Mavpkv%`jVzJT0 zC2{r~**;53E8DRfB_s~VHH{Ws4~4Bf+Uf1N#G#8$f0rORRn&%=Ek@ybs!cwrsDky2M0B!~W*$_U?J0p7uqv~9RCc(`x>LG7$ zXe&aMhQ<)W&ereNCyt{SJ@slsdpKEi&h7}kB7kL0$afC#{fV}OaV`(*&nr! znsfRKE7C34gsTuz8Eagu5HLk? zs>9;+%4e=li2V64Jt4I5B6AaZw5+_t-n>7cdkt3*^TIwpgGH{{F>03o`3&ei(nwO0 zM?nMGiSpno;myC8(k5-nP_@wOAdr1s4?sXF*KkbQBh#jmIJKlOR`^nKe~$(So_pJ* ze_$l53cl*>y=lI8c>7QnwYCvT25E1&SuGgV9AjM66|L3jxB*b*SFwt>q>0ITH(@qq zu~SNXN{k(*QGkT-%^SN9j_hS)^u|K=9vI;kzPzE3xB+MZ?tip_@588qp-()f$n%4XEbgY@m6<*AGBohMv#rcW=vpj z%s%!ei9_Wjx~dQ4i)9hsG6&o}wfn{eHS1$!N*$S~TmUOOfrkP(ikBI;ukRx`(k55r zpKVM~A2oqOU!EWv5s8ls#^<}wbb~*(0&OgGwb&E?^d-OUFi^J3l={3N$(5VscUdpH4#_Lj~a?gwSCNzT$WXF+R>zKyKQQ;+gVXiVn#;$|EA#F~K zNPB4kDY3G*f{4C}66p2=x5(2LX!=!fGq<DvxvWUwJ0{XBQj+Cj56Tw> zhw?UIVkc#Jz?*&y{P{kH@_XDs7^+>a*vz@*EA(Y;Sd$pjj8FE-D7wRooKW9B%HNTL zjZ!sz3$}Zt(Gj5g)9hFty<#l_#qB;HL3J#1LbxXGiY0tJ3!~9&OuM}b6`e=p<&4I2 zQ&qB{UmRM_W$#=MdpP4GkbmFTRUuG~wbd|=I>nhOks(C~+`nDgZS1^2=zAqa_Y-Wr zBO+{vmN_J!lWT_{Ps(gXICXiy>Ds(^r>|TUO6*_>#Nz1nHOcT6N#q$QUhRGAWi6kQ zBR(^mYY|#47*;4_WFwF+U41|5a^7}^+=*3V1#6^o2!CGLfWeaa_pJSDBFGXNk=+18 ztU!VDzx;c^iU%F` z8p6e0r8ATYR`$mSwE-;7LyoF)0+_=h{wkG?Zo}j>GbN7;TR$Wg)UJx@~eO1KgDafpzksAG|Q@&&n(z0t9=UBBhIVy6seWlFTycTYe9u`1X~W}uOMdHA)dEzLhG zRgX|R`zv{^TY#c8&fN-M*4q|)Dh4sgpva9g8Daj`m(cCvT-H9E{9fU^+vSDkQQsn# zabQCtC!3$u`>3T85C2ReS^K9*l=`A2dxpO4tSnc5EAaf6X!fX6z)_)lc)%ZPqyX)H z1kd!Xz>A~CpJb(RS3%I&J$&ulGGIJ+AP&;$&)aQ$gETAXJ5iPnojsQoJ8GLraoQ*@ zjU^_OB_oU~xQkI)DK)ns;DZ`Al!P1ib@J0mB_V}hX&xia_aca{Hg$cDr} zX-^b57Wa@Uk$z{2QMKJk4%JEW+NrvXTU$o#0SfI10U0xiHJ_*$)r@uN z!C=XE-EQNz^3Y7l%HhnlwLrc})q}HT$^7+iUx1d?jP<9jc~kH2M+Bz3)}}5$4QX>pR6~txa5XRoX374S=r> zhkJ~GIr~@QM6K<{S-mt_0zT_-TPx#qmYgWquJi z?s94a*K{>~;iLI#_@G#o9XZ>buqoo$>kx}>t{83d?H$`^zI*J~)24GVV+dVi(kxH~TT7a6}JR^}+gCPW{7g9 z5l`J`Z-9}HZjL8??R)By^MM~q7K9QG_V`Mu7?>TE5E1+n|Iig(g00KIJXGY0FI6Ef zOBKg5lt!p#YRL2)a&I95??iS6g%oS7SwA$Wvsx3cmB#RlI(HY^(hwZ6Z&Q|8?zq(U25n|(IA@xd>sN8=X z=Ez!JOIF+&+VXV$_$l{IjxoB^ndkNLS{C0y6xu1(O@jCDT~)$4ir@f?4=e;jh3B1I z11D#Hj|sNh`4G8LjCS579yaE(uq7h_RR=rmgbFBKInjN>Cxl{sU$<9fSi-hkfz~=g z2?@dAXs5x(Z%I6CL=L@Pxv^HbXoQ~FTsGSXF5^^3Z?^(@dHX|vUW)dsL1SyO*mcIf z$a6xfzEvU%TVmt}dk9Y_=&YL&$+6jq6`X zkc2O(t3wi<0)GZmmj9Oppb&pl4LEHG8fsqUIWt|L-?3dK=_!*-oBt!5j>_7s6HNO` zxJ?YhZp2Po57YN9n6_QOcfw?Mz~(@j{T_Zbp8b(OrVnobz&~{ee}!eL;}pQv^w}#Q z2#cAtqkND2)llS-JZaZ(Is$!J50%f~KhoJ39kC2f!oh8@$jJfcaqcI zr4bp`vnHE-5vnNtm@W6SeOgHlNECi(GgeLASGQ)M8VC|R`{h>q;1DAr4&_(_#i(Y~ z_7p9!m%P++v}A{Q59Tf{O&dWRqC|k27=6Hs9=dr^*)R zx+$WWIj0=$7=GQ&IY*C@n;)Vz$T&LaFtL9R<@SYNtB!G z4H?>&HAm_6L5^H)TRIGH2G3=`D^ox6q{#xCiruGEf z3)!_#`(!3m)Akzgs%FYRo*X8}bk8WKO^fc`FRr=IHD7-w_C0ZEaVAxlrfq0e!0ite zg)LQq-zYUugOKZ$I-5!M;CkkU8fj)iJMoci02{qav^7^XS$ z$j=TsgeO0DgLQnckcS*jQtFQ+ONc(|^i$gEPJIFLWIv%<$M@YdkQ%z(Hqq{!GNiGMyF< z-<&~t1xlqz6%}qSpghHhBEsSZeru#J^%vX`IH+}UlsFC9$g1aB^?~!jkLOUVzC{HY z#4$bI+&)m@NLyqZ!AriK@4JaT+sa!=!4xng;pq;{m(G>;q5R4=2H}<2KB2f{=fd5( z4>XO#@H9;4nz*`)5=&e$i9_^xv{mvj(AuP}j)5&jI@&$L=xD4k;A8 zY29;Gh4|BKr-vr7V(XRb0vSSQ6mx>1outi%wK>ufX)kbBQ(^sUPKXt#vILC9p(1EOVn@{Z;(cAc!@w3;XpL-Ro zWyG4ONZB*rl*j#zHE~eXhI6kotaukFn4Sr{B{n2=&7OKaR~}k%Zh@GX{WYd8?e@X4AXJJs_5>*J8 z_O(Pek_X2J%OH$tR&a4lD#|P>lNt-$upKKbOASW5Ktt*6P<3RRB6{G?Dalp04x?;7gfbd)Ieq`iN6hh zsYAf9Z1bDa8eUxpP?*p>JELVpMRR0fTaYQ=Xc-Fww3CC-g+)ZqfMAY!ocK}3he=9< zW7p1PL^;lnbRV!VN=3oODdXUBMqSG|M0c+B!r*Ca?h@b`ap=$sZybU@Iw&xP#%a}N zTM+b5rCrn!DjqgJ^KAz$Do(m7G33^AHJHT^i-fw`1`-bA(+GFKN0@&vA^ug)f)Ayi zW9Pm2&-s&posTQ-Z*#zg-_~&mrPGsmJm@~JrpZxn<8Gp=(sc_MeC)196hOe8{h-^Z z1>5v}K<;amyu>Qy4{NN3gu4^5#n{S8(fCpAzE6>v=FPYN=>`L)R%BTBdn|Y!8mxUx z_>qzzlRtR*zq?+FGWx$JH8%mZ)t zu($(JsM#A=D8bf`#F|1%;_n!O9Cr7Q)&Ah4Zfp_-hPaL8EfoAl%?;#18OEjwTXWfM zWXa98d3m}KiyJH|M!gHbns5ilyUxH@$PF0kYahrCx1kKez^OSXuR-BvW$}F{Vw*FI zoHst5ZT!H5`M$iFH@fow@%5DfQEu(pf}nsXLxa-Y(kUI1QUXIaNJ~qHba!{h&>aFI zok~h0Ap+7Mg2cDR9q-=nIp_Oj&f=N%ti0|kWQ5`Ax}2&s8uguh-G+m;;NnoVoJBUz9EYR-k%A;bZX zxDy)Ct>VqJql9k)RDzFt^}fu%RTASTQ-l71CAlVJIhq1rNQMh!9w{i<8?6KS&GPKc z$c`FTBi#>@;lq=_a!VtvOb~=U*^*~KSV%!x(TI7!?u*54^_pV@07(w_y;sWtXJ8@f>=QOc2pf|sgT-oIHJYe< zHPRQo4_e54s>dc)SQ|qiC{o?Kt(u7)Y135Dzy1q*!857{!ysZ|eW$d0$P`pBSZXxt z5n6KIp`hbYA5(fvRk3*Cr)Z=%L-@?Y?eNG^=zC%>Pt}NU*9aT}51Ey9glk}>Y!(JR?$!4G0Zp|U(H$lR@f0TK4#BPgHg zF&WbYMkqfl?1+Obz0`h0iJWF4#;B}(LV*sg;FMh6pOBl?&Dx)zwhq1;vU@J#l&nv( zR&+I0ER+aldCL*&x5`n$v-Jv<{@aDHrJwYDf)wu>n|_qeWm{f%vj>F}bPtPs5G&6(*JJYuRnd%Kx%C+<_z?Za@MpUIp{fXxix%Y!N_+cj+A3sYS@$= zGbXWG)1E$|*?FNrZaf3by2*_BPq;(${#3NWZ(4shMu|AhjM_Otd@w<+tV(4qIP+{p zW(7GwvTxDre2H3b#IBZtWP$|2R#(RZqORi;R34K0wqBwSg@`DL?ZcVu^Ui-X_SYty z!ifYd{U(`8Np>yOvU+jtVP|hR9E&}h6l$5Hn!w6bOQ~3V>S1(gA+QaP_}k=QJJO>m z8l#VWj!xKSFlS0XkV0>doHzFOWB%1LpnM4!!?SFYzU`&pS*qnH2;R17~mQ-4y zVfgDXk)scJ1JJK<5B*LYyg;yWM|8DmHD-O^TFY<#=Yw4afYfX+e|oGx#g1V61a$R-sDqx$omU{;St>NFgPRDC!T2Qq9s-j2x*A z$(CgruKbv^pWn;73JuiB_F4w#%Nkx+4fS60>a*OoAALuEeHQ-H?qMT-^=-z@kYhYp z<9(2dvLldE39YC8{Q!?7y7qaZ1%E&0_*K?)Y zf(LGjofL37bU|^JJHtCr#IpbvYA*m2*2B$^f*hEW71aYmQ~^BDI&T;5Tc4`ueX+?o z%v4ukwgBf@6u@CCi;aQU_3uKPfmhuRo+jw}zAPh79B_l@@8W5_%C(l#Pd`iU$)N#ZlxSS#$F1D{+K8Vmply0+l$O`L8Lz_P(+e+oQ06TO?h zemaM9W^}}l#`Eo&$SfYccARG0I`vZ!WV;z6F(O0vXekHw^mz~n!Z4s)ydg8F+wZDZ13(7}A-9U?(@r_0`_-pWH#Qgq@A z1-HO@SWR6nhDI8n!}3C zZ|UP8mNEh4bQhfh-wXO{;4sHzL6ofC7Grey@sXFnPD%a*oEiTWL@-R?dBs%DaoZ#` z>S>$>J8>iK)Ap=*0AXj1NHgtH`M&eH>EFvd>u2~osMd`opC@7L;uv8SPbI_n#3E4# zKm4GPiLiD4n75jA92ZLW1|tWbCv63|oo^oxKYbd29vC)#->_y)oXs^tZ4_94?f3X2 zDIb9tMm!Z?wXk(2J7@txI^6Z1NgrAXzrFN6-kj5+%^mRCpd^)n+t83>jPb6IIE}Cg zD**PImc?BM6TtSO+lD7T6hwQU=J)0WQAhqDe*t+mlm)De!<{49r-9YUIliP)TG2F` znCLY;bN+JN=SKCidW3zrKw!T)KcNB4jAA%F0K$**0FUZ&M9UPs3}!b7>oEm&4F3Gv z5YbnI`PMdh5w$-JkpoWoxZ?Gti$MqiJ3QdsB0rLw!6fnr;qdfb#aSt05N0iB(Z;@E zyp4$6+)5Hv(`+wTzXj}E_21tDr7{y*?QR2L&?IqxtFu}o_@02;2KNmww9CH!ZaxW* z9huK%vY?+EnmRYnax2oI+=}7cG1tT%a%o8vxWw3PoqQoI=;bx5+rPG|=e+V1{J3>! zS){_p5BTF% zsB^x}A}M5#QEI>Det+b&Gf12KgVMYigBZj+2kMmUmELMjzYNEe7(lpBq$C{=7-lsa zs>^GnH?J>8%!T4B2PE)ZNjK1}2m&I2E5=lDpVl3Cvio*?_LYg)9r9$lSa?b9y?r8Z z0)>a=KFIZFy#N#)HJEm@Yen0#`?h2DFCBgmv&RsC(Jkdpp!fPc@-|@gvCP_k3B-Rv zIzD?iBd->zIjYEDQBh6c()X*3)kWYuV-Q{ea*If0z<*T~JFN9D zuY9|&y(#_njYMAS?6u1nTfh+5l6CuhKRgnE#bowjZcmM(^*aU|zycy20Z~q|rAZJw z%cCdPWA>UK?6*_^>5v^*ctZ{=u8l2AMeZQ_|csM zCvjKhKE-jyHJ%kuM{;XAd3Zhv&C40Ne5puTux+?&fsB)|&4&--A3SX>p|f8!gCOi! z(c;h-bbl+a1V#~@3Aqdjjc`^+;J22u8SP1Uh^bNUQC;>mlVCb(6i=OlB&vib3{W)# z4MaYYnI$+Xx;()m-P({Jf)ko$I(FITstsx0q)}9G&PSBM+0kL)_`R5)YQU5)0(l!% z8J#?g*#SnbVC}_oxEuOj%(w6nq7`rhkY@b_Ho=Uq!z@~DW}D8_@@1TonGIDTENJv{ zsc;8S3%`tDv2TxGQ%*ioXM)*7o5-xek_k?wlA)5&7WDyU{{T&p{1Ijv@2z#4EIQ0vj>s{V@b?Sh zEQL_6MYv=_ObI}0Iu}_T<%?S=De#=_XV^OEnE4^(<9>}bbjGnK(jA>o2lgauU-z%K zLJ#cr&LUdJ&C|AZ1KZs|^2^ZYzzt6Im&xzXPB?0iu}PQ~jNJ*~HD$f7EZ z$n=)a)uKJs?=wvkq?$otuK14Ewwyi#9x+Ntwu#Xu+_Fg%xoAL10u~~t7eFXyF%A!yYBy{3YlKTCUN^%P6+GOr^yr5Xn;O zps1Okm`Yju1P|TQTqdA>;>AkjCzx$QIIp205`Mu5I3j2)`qBl$bcN!gG52D!&wL$1 zg|y~0s($QZ;L{dF?Y<2V)w5`GOfZrl@T4!k;lVXRqsQl%eeY`12|~K%bVr4mA`>K2 z6ZPkq+;r}OS!}nZv(gdLArp?`5GfvnhZrJh9|Lu)i7;0FaVEb=an(a6W<+KT@$V<` zkmvuF^A=5cJ=GeG;zwcfO*-IOhszITX~Ei>*;;8ML_-K!;-zuNxrh1p6LzG~>uM+Q zSlstP%C5T<)PuQhiZepO3Q z6r=ER&4Ur~IjNVE@-rA&MV5vv&v3ag4F}CO%>)gh{I8220A~Vcx{t(&IF2lQAYjM5 zrbmS0wfUD0e-V)oPF(kbr<*3ARK={04Hs#XYQ=EkJjQ#1!z7!UoHcEV#wQsT zh&Fj!M8;0%1mp0&Imu3LZTKK}lJ{L`@+Dc>*#?}6n%FifcZH`;auz5zD2}eYc#2Ig z23M#<;_MM`=%Ndw zB?Dq|1Mjd;dc8jz7pz(Je~@$$^lReX_h*b)607s6u{&cWwK*&M7ABE5WOfOL8KV3iFPJFI}nKB8#pwv@460i9OoC!wg2w_M}DG3WdpDk%;w+YmFhLuFA z^6PKzB#OF&s$O=55sqdWnEfUxoHL*dP}Ird=jdboZDRAP1~Z4!j@O5~WY)a~8orT%QE ziD!}47#H6~0?w}@TqJMq31&-6oWF5h)3jG?)j<9Qj#CxGCm-UfGDidsJm=89Ce?Or z&x!eRqskg9QX?lWc`i2$jyW=Y9BGI!mRLD`?rS~XY5xUCIQD9m_U7^W|GYKRkRtZE zo-iETQSVe>n`Li$GF#J_nkU}jQCf$>PR~=?eE9~DN>ksah)B-k2{R6joT#&)F_70+TG9I21bJ*rD$XI zsl*QsL8iXkd>8@;qSub{zgEk=mVGYEFj!4;0|@NVD<^8la#y<>iZ}*=(R&NGMY-<~ z3XzvahgF3)i+*JycMaZz9&hN=4Z$^j0U8!X=J>ebA(Vac4^I%z3wHOGaeL-`&be!m zWd>D7t~d0wDBDJ^uL7`T7xA_stP1wGspA*T8km0jU(^j7@*X91?$GS}m}IT&OP+D{ z*1hS$e2!xjw-CFxBX~bV)v?fJ&HIP7IvZX!r&KdpUj5O|)gr-Li0D|l$F>HC{wO@2 z^YC}@T|AepH;7|t zh^#Crj@k#LM+EYGUEil&M`DjHF$ZwDw^c9uO8E}uzUF!Q$-beekBc{r*a{Dfae(CZjs8^)$c+~1$7WZ@ zj%CxahA`1INfXlK>wQ(nRB)*%f#TM`E5mK!XCt|60;KF5K1{Yb1An{PJG9!>>~HOg zQh%mVm;+b?-4$sEs(gDsv(ixzU!>dW((hhY^>n*UZgR=~Y})bcHlW_GTS{5n#3{LR z>Bt>+Q{~)vQ`vuQC%EOhTjR2wX8JQ~Aor((-;ULd%eK~V`u)KhQ*URma-WF*pq55Yp*=v`BjmE3Lu4UPbO)^OLJVp~~i(xyS zx#IGQhgxGXCK@C(#SI>lxgb4MZ{%)hY)N|Z_Xs0FBvu>>V-u#|4xb6=flE-mxeo5V#X6){sSfy6}M;2GL|oq=84O<50mz|> z{hW=|ns(F^TP}kxUvn(v`#x2Yd<@$jB-;4*96+~0;?IgX5;JRM&=Q2#lEcP@guCePM2t8Co+Zeo!ZJ3CGG<|TEmGyA7OsN_E+r3T+9yZlXMo6F0a_+}qp zN=sqcR-q)|$q90L!s9Xvmccu~Rp8H?_JpL9~5JBxnFXg$NXyd#h3iExvzb z-Jdq?wn|3d*O^{)UBo$J{0mk52VM%M?KgC!=FZ<`V3f=K&u{S9*`4D)mv-9{OXHa! z4nLyFm43{^%VZLx`dSlQriW|xM_vk`HqCU*D-XV)&2+h~UxRf1IsgBB%)fuhf|R&B zYLrvUQhe_jxV^#(>h0cZ1I8KP7;0qg zYU2}`^2MB3UBcKf^ywdq%ipd3UTP4D1MWORs8SLhtrXB;GSLdf@6?Xw;&{|_05fcT zAY}Dq>r?*o;=SXUAjHo>5#e2KTbapY&rU|xzMcV*L?Mvqv1d9A(NiBDJrej1AmirO zz5jc$;*nU+G2fK!J37|S#%X`;*64rU+|LfEHi72A9*A>UZgLzT?;9y4u2zcvt{{~6`b~Q0y%IZNf zQ~?}m6*P@>{6Cs_4zO4s`X=RbcY~b^{0d!J90H6*?n~^=BX;x0mW`KE2!ewEcPlSO* z53t2ZPSF<~Kz|>Oukep3VDeCrO|_j{Gukq8d5zs4a*cYHpu|V8avuY+F--`xh-HPN z?W!nKUSyi2X~x8glGK4l@f8Fr(zj`%LWtnsFw?IKeg!@S#*Wm?9Y5CsKkh~7DW>Yp zBOHJ)%Mvh>U?Dp7_@7hHoWfVTG@Ay|$yGuDAF)WyKfyxMoN~KtW>~ z=mvoM?U_+nX6Q?O0riuaTd&bsZGMvxUDu^V79dny6F3ii>Y4j}W=gw*7jD%10UN=1 zOpp-Wq@a(c8)&CM zc31>YklhBqVcD1E7~?h8yw(9ls|g~Y7tU%U+*|`XaXJAqkP0IvnW^)@+V~wvp^RL| za|Q-SMTzQKr8e&br&XDaW7wb{;9!nu-zgF_;{j3-A0ljHzhf)QY31EHo{Aah8(BtJ zwHTpmGgl-Ki;V&<&~=!9PPdMm9!Sou2V&78fHn#&#V;6mvL}`KfVfHvZMccKdSACB zo~B8h|AxbyC(s*Vhxdat))pJxb2Is`08PO%-&ORvRMn!sugtR_EYFLGQV++Zh$ig1 zRfs=6k8}@gAbDeywxkF&*I%k?3H`7cL>A8d1uJc;Qm};UgrC9fpqPgX?LCA(H z>#<#-YScl)HONI$3N3uQ=%Pb-!>BI+IbLU06%pAUU{O-W3iVh10%^o31 z-}~Zp`MksR`$oEo(K%M70^*I9m*i@70^4BIyeAyU4_pT;j?ZIIT_IQdU4WFM(X$<8 zoS->Ph=ixM7COVRgh9EBi_Z?jg@Fzo42pthFWmuEZW2y7&7<@DZcHi%1YCCX`9(+B zc}Av$gvLe@5t=~BlP0s~0iM!{0I`op{mq&ovBLnlDU&#R_|w1`wjf;qoSS;4f`ukb zjRbKGK;NXhdOc~qDQ`V#CPw3ZgyVa}0_{`UXug_v0VrMajWytCY=a!0H`@fN_1Tp;gJHZ1amp+A~wlnC7r)PFMO3E{>(oGTx znV%pnI6Y}3{+4S7N;%heda;$Qs1BuPu1ufSHv1|3Yi@kVvSwX9R`@g*Bk>J0Ji>5@ zDl_g&9Wc10Mv9)+v{kG!Vc&8Ez63a-a9;}8!3(aF+}*0GM=!78j! zZsJOM)C4k;e{g<4=SZk=2J&h8f_uA9Hl902&)fGChbN8!f_*juIX#m; zi1KdFJNeIY#2ScMawW^yzs956S?=60ATe1pgRa_4O04-#4a)L70jd1`eUdi@nwu2{ zWM)0iH3G_$6p|STXx*T^4wbJ{96JI9XgjoiI@FL%kP=--C#hi`{v-@#^VP^g(dmRT z(QmMGUS0<7j8DX}F?69Bss;9$FbYGS=>8MgJ?y!>;5C&==;$|B+S`!%L#sU?2{3NfC5RSOdx|2FH^#Cdu?4=m1cfQR5E6m$8V5EZ zA2^cCB0XW>bR?wI#0fu=gr7kiNKlb@&L|GAqigUi|1}Zhy^$EUjXJ(f;<(os1Gbi? z{e#qFlx7k6k57R%+Gub>w(oRGjImVR1Cv5Mo}R1+x%d>y=FP_VRE_Swx+>fzggvyA zO}kGCJ!M`46GGr-ygA>a&sR_h(dH|{(Ht!-aWdWXuMWx$fPN|B?X{IfPWp)rc4s@v zF83&0vRPsqrK}Z*J5u`<;W1=l zI#9a_ium&7CRRiyVfZzU6E!cVF!s}#G$AW$5z$P6A?tmwD`hGW8D|8A=7Ta+gmrrA zL=E0QC*-P>w<0_e_}3A^V2DD=Eg8nsS;#y4mG=u&K`FZbb};zPT2H|fLPrpDUv&2o zuL03;%CZ*4vfs3_Jr*6$5LR1P0r8-Nan$(h=iI4wVY`t7U)<{fqMd-d2o5k@?^2eK zLo#3Vgw(V$N<~cT`{xkG;$ky4sOPnaDx<_aLo~3wyBXvFF_HIU`ki%i7vbPZjjC->VosZ=~pEwmzCo#(b?wvgkt0 zU$S00xk&OnmE>+296{s`5R!?}EE(KQvhvWHm8BPgWfaa((lF+?F`?%!jAj4sGWuZO zmedy8YfkO%&+oTxleW^#*MH%NcQG&_B8M6tM@JaCtT3oM(i5obRZMWxt}TZ5(DH&H z`eA#z)%;I2j8%U11R_Nyc*h<+RGpmE_`B}sW@tEC0D1|`pA6Ht)mDJk89 zUGArhZl}&=J59()x?$SAXQ<^xUE@-sYFzRDCEAp(!bNV?%MC8wnIXPxU2=vF;Y+^i zgELgpnEP>kkJiyNfO--mIHDgn%|^@--um8W`bQ_vfp@b39Qg>ZHV%Rrm8aA zG#kS&ho-=Je}mYs;1eF{wf}=fF+@KZ$*#4qb84|kK{-gyfbzN4-<5Ab6@G?zFz|HF zKJPNEU>^j`nu`63wDUBEwGH=ZusJ#2fC?W|yp%X*_~B&Iz9uopv?Ei~2%sIoz|^yFX^Tg%)6 zK|5dMfI8<}nZ+M2N?*!89;2WanMrAbSWJK4n)^q75#OF-@<_JRBL&!(sC|FcQ}&iR z%f8offTGVa^!r{UQCzYZbSWi5pbz%-vx&~y#0HUnp6qD(GJ~@`YgA2K>6t)fmdjAF z?}xrwt*1mSX%xAMy2Gsd49^A9_R5`~X!pi_*>_sB?@w$^voZEkMa@XN|6;yCUY%1g z^s}DJhWO$67T131*!wq84^lo`x-j9GGZ)yQRMVY|puqOLDP2xmDYZjws&6j67z<*D zPFDF|ugsVi*t@q^WtH!E&)PB(J{2cIL875Vgohj3D{{@A?W4h~K~0dH4`GSss>tr9 z#Y5h1AMQCSIT|^(6s0_I?+*-=$4;dGxzuYwXq-2&B;D0l=rHnDCbo5Fo+p(76?%)_g?OR+<8{H5@HkANcMSzT9M&DWT6 zWA^>ct2MqA0TDF;bkBvx!+&*j=wmVrsden_2__G)%~*YXC_zHH$*8C`e(!h-EU`l~ zypxIcvhJHQ+6xSw&qgnw-8NX;C@1|G91eS-=apPk6km)58$$ov?&aQNU=4Hc@fV>a zJK9G@`z{beLXJ`sdD8N|a}y&nqSz+yhPSB5Rejpu&ckSOMHs&h5Nssl8Xw_<;r~LS z>tsj;8P%|DGlu@{9V2r%B_*4F(!ZnuNMb%_OP2TMY@WQ(UD&|&NqXr*gpD61m-E?v zjWpE8by`gWcP&4T09W8(Ccai+m71>)8(-*w9K23XF4_}vX}vsXeX*G&*XU>du%BUk zvA_L(>>8oSL`VabEf&7GUuBE%?HWF95i%Y$2dS@vKG|8V(ZD6k$}n=UAObGn3;g)y zn0bJEmt=#QNBAh&npoKV^qWf3^pJ)F3CnWRRv=YDr0!5eAWHgQKq}npn)TxDJ^3iT zhc)~u%?rvKMMT!KNMMWO7dHlzDXQpS_TXxeiQ_x2l$o7QG7IjtyKiUGs0Q&Sn&#!AdJea(q zh_cH3`GQFYTKiJ8FC%*}t<$uLb(2iu#W)@(%qoO=H+q0{-*u6|b?I!kH!LZfJNdPJ z3`E%9)M^|!1PRQvjv`J!N_{5U^_ruDBc5rEtHw!?4_ibH+Ck=6KWr4!v!maojCLgI|U2_V2 zrfV}6gtTX2>BDS>{D2S<-i8zYq}+5QsvN3m=CX*788Eq%g^Iscu3Np)k-!15NKp=7 zGVYC$;bm(fLYcHMTN&U<6DnYNl)GAi-$pNvI@yT3T_YdHBYBlS^8CqZExgelq+^<(dNROTww^|XjImx zy#KL7I2s>C$GDAb!vEGI+dWxBbl^&;U!?X)zS~3m@4*g)hj_vUxq)~L>+;n0t*y)R zqpCWOMO&FS$o#(Na2rRxKYjn5gzE=!;>v}p`FWv_nHRuV^4(8?zLViM1?u) zx>2lihICT8>?dT5Dad#!!sqMz^`Hcw?AG@+-A?bT4&{l<>DRjl181KHRza+#RS%*C z;^P$$iJaEKvlO})_uv@k$GeES@=EDzn-&g>JI`0-u~&X!Ae6tL5%Mg+7%rc7Mx5~B zlFy2xQ=_@Q9f+@Z z6u^#IKhfuzR?1;tHE-0|mXkXQFd0;9i_N|Cl`AtX@^sVlf*78Y7Xz+)5)`T5?Epvj zo!J>|uWnoj`2kXd?`w5@1v1S;6xh-Vo@yoTiA{Ni{Meq9_|(&V#fM#Lqg@_*5^{*~ zbNTFi#rpC+sNLJZZQ*=&AL=zLs#BhKBB}Y(NU@!gM*$EYJ8SfXZs1F!`O$ zhaR_-wJ2ih!^Git^BPfxbhpLuc^wV^^eviG0g6vvFScJrBIi}j2S7vzG?$#;6%x~l z2B|l7!%M4@%4sU=QC2R12>S4)M*d5Z8~{3}g@iiP<~}1qPAKUdxFZfB4Mi@-A4cDvRrzGXd5ky| z-7YH?CK2rNoC-s*4a^I9Izf>TEDMOhyq4=THb2J^#}BQ8c6hu-j=u08y`H7p2}UVy8?{**-Nvy ziWg~jV>@Juz1AK!-{=fM7^EASn-*(NdmX=Mk#Wf9oOLE4YnL}O3C@hSdA&4x@76qc zmvY}pT_{#pUBO25JAX~H!_w&|uX2UUlBM|~^p)L8fuX>1?&^Jt!>LCLEUFBa9>BCg zTKo*MeX6WGU-ZcK4vN?UV|lTj@OkDbYaz5GUZucmemgZq>C3$?XAwbd>l#V=d?W5$ z(-!@Tn1%V&hDdXLfnN+p!k#Ss2`H)0P~Ny`t(?afe|@hSZa#Su@Va7sZPj8a!Dw1*c2V|VY7*EG(T0R>6?+K zMdkvErTB7f2J z?WSkt`rVf6d`D8|v{#+3#lJ+q6P;kT<=cl}0%maLimP2c-V;!-X+$AAUz~J~n&DO^ zf)lBtB)i%Mh(?S=cWH3GPJlreR*G-DJ6vasnSoC1>gO5MN_Jm6cSL-J-w z{*F?sPf}A2SMIgG_v(*PstHTC39a6f;5V%Tj6;lORu6{+0*US4M@AOL$mG>#iiZ&; z;abM420WYyzG!T$BQcVV8t_dOqQt)9J~GAZCd%j@FCZd8ijtp^3#2Ch%L_oa4Y_VO zHMEqQ+*}8HBSdKMLoKRbz%xzY6^RzxUABJUhIsIp!WG)^O?Vh$O2Ia@gwu?=0j^9K zNvMZ_8m~5Q;J>Gr$Hd%z*j-2pqtkgl3mi01(E<=)wojSNVs(XM#6k#@BrX#yHIz$nXGik(dSMo?AkV_k4#!Tdt*}+^_{WdZ_ zhhSL5P&lJ=WO)SB7t8FYF@8pflFY7EAKYW-9y-v$z0hnVn~SvRQ$o5Tx;Uw6@5+&{ z{J6Y3zuU>Cm%3cjr?;NMZcL)19C`bWd{q#dow+>GcLA?X$|H>|yi3;FT8hFKE8;K= z)l-x~l6Ts%MMd!k7rrL>5W1>G9r-bNn0x`i@>1!BSXLn=KJf6&SzHm@fz3y7@tF6 zy!zRLXhPr1H&Ce_L}qVhGBJ=-Mxv5+cO41VPK+L0GwNT+l2bt?en!!WJpR_-uDSi3 zcO;EAZ_zWwnv#(2n5ITLNw!C+BbHkF=@~2Xb4#~Fp5-^|M+TE5*bmg@Xtr={WIItg z5Z_bjq)%Chy#sEoxOp;eKoFoy&UlLX$u~qyEX9e^F6J4{sYJ^FiI>?*F z-y;E8U#lJ)8;M4AWg#B2z4A)yeMTF}iWCJw8kCyQ}npzJDkO=s?)XFHy!1V$ap@c zX{V3xWe^$gTFevGQU*HGObW5tkt%rgU=!6(yhRy54#s0h_;$oofB$KBfJ#ACJVT|R z+>`OtMzZ;+VI(T7>NVa{XNeo|E(xGWJ7t+{Q~e&FZc|Tw`A)5OP#R^Ncf4*7JCJ_V z!`VFy-GnwgN~o0acT46O=8AHE`dZX7^1uu+oK7@jU|rDky9G~;jn*E4c4*_s;g4>O zI=$<|v22?7{pHwcn=+Fy;lJ?AE)y{_|H~INIr(Y!m^mRFM-qAw189wdnR_&tBsf0U z^cFolxi1Cm@=TAX@C}vyP~LE+D^QZg;O;TGLI`nW;-BG>(%2e}kOdHx+ddCLsFaW| zYA_#r_QjKVt5BrQ_oEQD1VvRSy+aWDwY>V;v?zitmKe?3fkW-NB?V_AC?%8mYS(IEgV2tCaYy!xZPQZp- zutl+q&)$r_v55ANONaPM6Bj7i`A3MdclQc@r4%vq2{j+#3KnGlD4WY4Qv$zg zuyR6P#XWHzDZ5P=*p2Xj=|T1@@)pfJ()Mf=7A6nOgf=w~S^J*6 z)n1rVhxqmKGsq3RtmDg^wDqJcW4M)#j0F0rveMiC>W{=OnKKrZ^;+}VG?U{kC`jfZFl z35(BOlDkTs1u?kcew?p#arD?I*KCGqq1{x8av)RfF|T=F;xUULy5Y==)Aw8PZs%zB zhLC&579J1pRI*5fbx@W-#6huIwWX+<)~@}?Oy)p~-)ysH2-pv5us-if^{wL-tNvDe zss^rt^}2t(2hGANOo+Fwh)DxX>=Lh+ zcn*Vn1Z0*FGMvc0+Q4uj@+1i4Qb{^W{wv#f)%y&3JOyO2)Z^*h)yvEJFyl)E4X>d0 zA@W*S<9BKmTP8SB3*vG%qkoVaKNBCIJ$LtEB!*Q&jW7df@cGY+cOU=3&DV#pMqSF= zreZn2*cX!W`g7*JeFBFU&d&&wD)N<7!st_(=*pDXy%-2pqg9W~LTD*a7}oqu(KeH? zA|fDE30Py2&(R<4Q!p8~QW_`7@R@3MyMK6u9RQpklV3&upm1sK_^I>}Qw+yvhFoK# zD;NchDtIq|{#}BFIv0!%NmSy|AY4S5>~JkALdHOMvMAJZTC&$DR${Ybht`cY2rSQ@ z4&ch3>!gmv=`GFEyvn`PI97*an3~JHmeW;NI#4$NNtYx&O*HLepTk1xt7cIbN%-ZLJ9pCaunQWDrqBa6~Cgj=}-cym| zlTc;umU`~}df6!L#vD&O&-AnUvSE&UyJ^#kQlY^JllI6TRF;OudU11 zJlg!{lU$~W8}852Dg?{OUUl4FlWrWZu$!C5uDerW#WG8Ce6~D+D?=yn0^&MiW~{bS ztW1;Xw-fHGZpRu_;uUFkZSQQ^FdMo_M>vlc{Q!033*Y3H1f%U5D5>pr^ zy`1=~7g2HnN~XNuaMx{(vM+4qveLUC=m*Nkw-k(SxhEUugGj<&`6HB55abYiueWWy zV)NxbAxH(dM!gLpTA{^x#5yAor3nTfzbk#h3Dlk8$Aiw80@9gA%TvE31?4vi196k7 zY=Xl*A_{Rlp@ZLw-CIaoSoXCqu9N!gZZ{42zyex`6>fyY7iBHp-8Ji#M`B}M`qtN8 zUiUIG!+tNZw5mgHJ*}D4kba=64RQ2prp#ZF%FJ*4eUUq0u^tzg$BDbX~>YOG?M-}W8@q#GC{y1fyyIx-5oy{J{ z%BUHX*=|lhL)@Y$UmZCvn4vEU-`&)jHXgE?T$*uS7&(;KNqYqB@zN<2IrL9F6t6cdNg3Tf zJzJi1-8`iU36h1D$;)@{8;Qmx(J! zy=U7iZd-cE2O-p>4x2Wv57RBQhIjVQLHSgbDGR#slBe0l)HT`1D2h^A{C`YY10AHY zsMStI)g7XD2UXdSs)nbCx_Zz4yXFf zX0_RAXAnxEz%pDnQ+3wN@U%^wmhWlmJgUTkVe~kYuf+ubLX`gZ&(Qz6IKmD!B`12RmmvFIXiB9ROEnXCqT81A%1}%1GyLT!dM{~Oj5YpOT z0cAQH;%8^=w{=)`sszI+WX3^9$kCO5N9z85TJ$Gy)(3a8!?DvwR(s-pJfF*hWATiz z%_QwgI{(Hu9e^(-E21gtf~-p{J%KW}YP}>YeTv ze_vnjH?iv9Px?`U3?-H3@zFaCB-^0{g%!YeGu1XhicoV~AkqhXK<&}BtryxN|f`5ScEX19g4I9 zw%R06f5vfxYeW4ZML5oRF4zYS_G5;_;pAzxNbAO0_+&-XavGe|!s$@cE333( z*6|-po`3hI6B?p4{-W$P=dV4Yg94PD1qteo^DtjtHnr~#FHLKJG~8BJS69~pP}xd@ z*L}b@bmBz>Q4nUql#nG-CX~;yPyY~({J9~%hdUU&-L^zOYOK!qV=>O5U^N>_x#YUJ zkElxIx0UVV9o9tt_+MWU8ytGlUiF-ZO05F0N09@Fo9s0$x9;(LLVn%{zxl+WFvaK+ zRsr$gF<~5Wh!ln#65wyeLLK1t9e?k|tOSq{%|U}^v=2ECL12dooO3k^| z@!5a<|37{-Flk8Cx9D$6$mzJ@l)NL=uf@SR*>f(EgNvY?mkqdxPM|zh2rL*Y&rW@> zT(ZYNCu9x=Ac8 z8*xtd%QuhtIQTXIjY$qrPt`7gGe9;xk7H2Tbn3U0=ieRkceAGt5Oph(kP|HWX_8fz zp1unaE8<0j+jAbLW;g(;Al`1upr-j0?!VTT|9m<~WK;c+v7$Zv;}a&6}eEz%$`G}5VrARy8xCEcCU-O|l4O5T0G^tE1FeSjb&od}Q$XfA#}n3W#Z z09l@tg3iLN6*j4X63k)1IC|l5R=%gRY2xWWMLXU%mp8Ws~ zZ6gK92B=k${`Coejh-o(Z{{qw_y3gNc=$n#;OBU|X#79oCC^J9Yfy7M{sRq-Ct;ou z8LIg2xC2JRE3Cz-Sa3pk(@z4>;Q#wjp~ZnE`2l0671A&1o4~H_PRN!U2ZCFoVAJ^b z9sk!<$2QwfOqkJg07g(vpXTP_p za+s-VHu+Fw>Wuuaib(((+K+nV&`aAC$^WHWLs0y+7ecc>09$y*Jal}jFc(&uoPdlf z#Q>-(95v#3anzz}celp`p@K~&2ns|YHG*|Ot$cI|VA0>fA)g?+=Ytn3xC^!_30%*k zk5t=gwHE5Xmy)+1;t8}vhLWDACZz(_X>@tCz)7|Ugluf<9uj`@3<$BwcCB)!{Zc#}9^Zvv z?^QrEU~US0((PRE*Wb=;h9FprWFkB5M=ib4!N1?K;Fn( zkD{7`N09u_5JbR$RH=`EA_tp3_!rCSZJDgQuYXFweJ_A)NUR_M*MUU>7e4dGfv)Zg z{qPy*InVXAj}gic<_P`_pS&Y>*JD-45>U)pwm(m~;|v5al^np?AE%n0_>2z}-JtT| zI0ybT@^bgN08l#N3jt3%mG_yJgbXDy^MOTCx$u4q(K;9bwr8Vfhy7_)RaM^sEH(;~ zr#ZN*aWEY5x?`7*Yw6;2R|S)_9_Y?o#@B6%Wvqf{sRVfRRss-lKb{L5hBGCWrG}SC zEc<0L8XHiwm))*@P&}OdqfGi}`1PZY!qx+fV>>Y-r~;N~C_6p2o=xj8e{4LtDEb%#fRm zvH89oXm#F^Fl`AUXbtgS!QCXr7JOp-IV=ADBNP^6vg&uZ7=#;t1(2v&pVGXlA}Kr2 zx_*HPqK8SMLENXFzBlKxbkh$HLh+mEck7gySlF`*_U)_*k^9q!Z#0D%ViFLt_H+9i_gjC*}7!~=>iREdqn zUjalgF&aA77ckf3mXHx%qM)mjC3ez_{R8OcFn0 zv$6ym2)nC3t%gWPB+zWIT?hB2ybGLXi4;f?T^}Pv-$9mg@vr-UM##!LcqK%OaCkc$ zh;L%N^sbs}+5hI*4nS5&bWxYZ>p66OXOwy;Xk-vfni=n=cP$e?TB*L2Jaw`HC>Y%a z(hdxNfC2yB=0PdXz4KPEE%2y#rr^__jsY zFMzC5?MQ&gNA{)@U_}f;+Bl;7!77;mV$BbUm8?fAcqPm@{*c4G({fWGSWQ_VYgv+- zS}vEGlW84LXq3vClFBHubGk#HJ-o$VBm!xf2qK5oa7ej9Y%~nh)PE4&BDP{NYpcu_qK7$Q`%S@xeg?S(0cY;qKmk!jEzMPpW zA)aZ0ZILK6(cFFb7gz&z_}VOHN7C-YNPwi~g-b;;eiyg|mbh_8nD3xYP;7+VzP4>G z61&76V8cKC;tp=vxb7vF3aLNni7ijiC+J0=$<7y-f7TE#?qJ}ue8u1vDJho;z{DT4 zRvSX9@Ayi2PvcBClm?Q&w`x1^j|+8wSQ|d`e)s(d_v4jn-Y<$Ydha8}T(#-`w#!8RAN*N=a?-YWAK5hBX*K+xJGIvl|oz2ksB*reYfusn=9o?tw`r`u3$zUbBR;(7IUdEQ9e<;l9F*62%sO!t2% zAu|&%&oq_{Fu&q@S&^!k0yfWrm@>PDY-89VnIIe);C-ieub&G2rZdO++4qFH*wKEe zs53qQpBBddCINU4JRZbB0pOlwFfPjNNKh3;0T$_I*mnhdQ zdcw|8LK|Rs9~v>y8u27I8T*gE{o`rb!d2Vd<4N?~PSk2F@UQD>F)gz2L8NyH<>DY? zPoLq3x2V|-x`^S^Zd$^aHJjfGd^^%3L-7tQUI(Txiy?<&TAQ9lQ4AbMoxS{Ok={W2 z^c}*VD-y%I+s9SgMs-a)qj=+C5cS>IU9o5>-AGC=t53&e*uIS}G&VThVIZ}u3uBcd z(V#_Sg&t^U6=NE6HjxG033O)lC{+`*9M1AutZML$!==e8mlyo?4o%Zq+P64a`~m4| zX*1_}oJtal1Ln&z^_i3Kph1#MdCd#YhYZ}aQA5G@w6AYfu&$Sq+jI@`p(*?t`D{Je z+%5Y$Gsm6$f5lHn$G=wy?Dh;4^ZD-@O=|pGor=3cUz$AA(Ucd zW>mT{bdc2lZ1rM%#?+}7&h%^XtLnwQ_F2xLJFA|#)WoU1&#r&>&AP!yk%_)iU-zDA zGXDkBQ9e_PgLrCKM?8;7gx#gF$YXuY7ZX9jz$`8kUnE~l%giF8WC3WvmSnF{ zVHj5j9vy}Pe3Z&yEng*)S`Pck+(>w1*k)v1F-z{%*V?sq6XMEtSN%vG(1IJ?RsaJI zIetOQ4(U0XSRObq@DO(tKHk{qq8hW5Di?fw!}rmtv9uQ5+rFKr;ms4shYwkkPfNc) zN@FT~x-lxdJxEewKdkaXtSh48ve@lA$xl1?A=#fLuk2@nj_nS8YaLn4kG`hb&g$Bd zkYD~D5GUqimcC!Lc$i}Q!@u}*n`I;s1Luqx#UB+a3<0#;c-ROR%viy~YZ?=vV5cnm zCeClZ0Ah7Le*NtT7hwlq#D2C)xl4Z4?5^Y!Y9%ds>gV{Fa>QL#hXf!oMTh&pPAYiQZ?3 z;3eVV0d!j*xwm|b(sUo`ceuWgZar;OL)9ui0a$C_7bpBN2z-Pe+N?zgw#MvJiN9f) zKWN#IwKG|vW{kYu6_-FB!)q^O@-3r&Lkj{(Q;;^k`6jlmKu?xDCh8PY(5xQpek;Qi z$Uw5w#mu+0i4jY8|43v`>lphrx;2}{X0NWby;gthfMwd-VzN>kcCX35tkcGII+;C& zdpf$s<1vFtm=jSG`maS%-z45IG7^F*Y0gy+ih?$3I|4S+)5Z8KP&>?$7B|Sf*)v`7 zSzM_`1UL~B2>Al9N6dWH8Pjpo$_LoGu%BI4?O;5MxL1<^lrE0J#W$C~xys<%gH7`@c@*YNtaObdhjLrTv zCXoW)u#yvNG?mXBzdj)pZ|!USQe|NB%@locP5kxfml1CK73PZi%yOsVvO{mzt!|Rf zni6is<=zs7c&+1X0^5n3hS<7|9&rE5oaO*OyR>-Hq zDgRW=>yzXUrmx@c(U>uqF{lOLuRs1Cma}~Ce#nEt=^>UU>6md6TnNeAN%y!(nI)TJ zq{vb)frFOd~Aj5B^}T_}lur1*NdMmsM)45{@Aaxcc#9m>viFy0-62U0xop8KUFoRjyH zbrYpv!pC41uksOb<)iF(MIg0q6j|yh(*kqiaHu(v=}@hNIpdP5T$0L#5vX|S+oiG) zMP+NWt0I*xlXyv`9&x*rc1M=<4Vf>%aN{jm!|t7O?%l>n{Ir(pqSCpRq;uXxe&(`H z?s{OqmPh{D`&-qKA7}sC21#2Mr!tO0I;{LT%PpJTN{&TWYNL^jyAM=#r)uC7a|`$! zE^-#K?yRfA}1UE07`GO>g>MJl!AuQfO+t ze~Zx^u|T-nUhky^L?ro*C*tT|E|SaM9qII$=N>x&0duttIQdW2O||AuF_N}RCw2pQ zA6+A=<=twQGaj%Q%R@>swSa_KvwWUs@4aAck)@$OtDNxt+Y9=;ZkkDP<|&h(c+H*e zmla%|*{amJsw8?|*R>{$b9vJ796tSU@P_oI?!JMiXWcRC34(i#{bSx+6v!U((w_-! z6-#>3?EYjL$Nr4}qj4&DEz_VqZPovxtjc<7a+E!vYzYv8KJ@9)va}2MmYr`&;+`xs zifKz~*6=B}D})oATb)~kDWaIs13$gF)_;tA$d2{+|Nst z(`u4eHRGpYmzu}zkK9uml4Qm~04Ise{dl)|^4BbtNmw%T$*#w1mOzCt&OTz_p+XhU z(wI-$ye&H|4yF$orf zl413J8~Iq~JBNE({W<2h57>-24q}a9T#6$4npAwRWj^cS;>rXIe*48;YTeur=jYQN zx_3*4y?Le%R|&&|yP?e9Kz_ioR_a|jJ?{Q6+&m}L=e{gL+uB8#qtIG~e?ex~Ol)ex ztB~da>r-3@8eg*$TXO5%_PVyc4q80T5{qzw)ZrNe2OIonu-yU+SBOq^0V-{8XlhJ>;KxNrenxB zW!=|x2w3_Yq<&{J4xV44AYg#YwnAacJBnCF6(Po>=KVHV*UV%wz69YrcnSLg;i*5# zD-Ln48573~h|#;yzaXvWrKfbV5x9ZHRjyvqe32I3&e=4GK_x7c1;{VNF~GI zsT`P~r_*qsn&V;bK~v-^X#-L7QJ&$+_k>)6ZaIdknh$hq(~p_5?k~_v45mTw24f@W zW|?bV1l?$%SE&fbKHaVN=4nv@!nHevG`yAOq^ry%`}2;i<7Use7fT{poIuf_FqN~d z|EsJ%wjTwRN*Wd};Ri~$wRCj^2R&y=11~=%P!x$tCJ6KLeeyp1XA6-^en$UhcNnJ> zw!~X|a;p7cJ8AxA!&)h{lFTqi>oZMD6)H?(*3Qoums*uc#1kx>QYB?wobI;!HriPf zU%eXNL`uAJ&g$c2!je)KJGS`%)Z5c(IPFYOPB_mCGKDDu#i~EEbg~@7?}g z*I*_XnG@|z)zzL@YyYVDNpW#x7{)I)YaL05M*Fq_Slb%j)J8+^$5HEaQ&~1LBGkGa zz*lsO@{TZN5NDXCgpalyCRcX5=jAD0Zm<#x6rFZ*2+fa-_8q7ZiDk_!Y`BjC_96FV z$u!v3fo%=&Z!B2ED)CA0cF}a9TMI>Gx7^s?(Q66J3;CJO5eb!8)M;m&UY{e!GzS}A z(wwmU%CslaSw6o36|1JaUiYilR--XO)$|7}RXDu+hx9Nlucf5^ zs1PGEBHpiE<_vIQ7s^fojfhN>fE z{BdfaQ1yMdg6V=H%*=H8@x$B3c7jn#rb5rC%9aFe9#0Vw-!>WMXpC z5PveLDuJ#|w(il++*U-4cSj6NQy%=*1;h;Xs z@&8;}N2sEhrmz>4)(U&;kJOlPdYx}-gDwBDZY_2$WV=!F;iX-U?fcJMKM{(U8;V1}7e9N{!K zM!4GS+=FuvV(ls8&(bbul4b~VE#nZxH~ln?RHJZpb0IJYx*g*9KD|eExi_QmYoBN} zS<@$yqHBv+iUP1&^A-XY-wt5=NKCch$eg#)_pH5dmGmXS*nem*vadXk6o0rtAo*(k zW>ey5xoAaG2jj(_4m=if-;$Eqnp}p@GzArQL1v|<8%);Lt}AD7JDsh7<3})Q^x+G& z#F%0@8bazzwlbqs|kDoZ$=2%!eEG998PGCcuvXwl50zRjI&k5E-RjYyjB$$D@TpOCM5VusP7 zwOmi9K)=q&$N7dY$ZRPTW=gtyL#BS#e zy}Gq*PnENV{w6~ZzB0lSpQnbzK?2=mnXez+X+NA#=sf}cx!eXYS z;av3m@yqnAg=JP6=sp=B^~koUq5_=5YS@1Yq`Gf?d%p4ze^AAfx+_O5F_ES}WRJnv zvV{XJ5ec-ST<#pcvtZonDwlYhOd`dxcysr@KRz8EPP#bkF1(W$foCl^8ZY0L$KIxV zws;2^-G#V$BMe*x;@R)b>a^g(&6?%(Ij{ES8Jw%~HERQ@Y2g=o z&u4WD*=syY;kVoBCJ9a&KZ?)H=Jksf%9~Q75l|iFZBGe14ci)CzHNLBOfoz*BfJ(& zl8~a+JFb>m$|YqrSdu42%1z;BS{r%qL(K}T(~ytTT7TFQ7d6q|~+s8@{>;q^r%uwVDVjwAF*uc7Z@o6=*$rs*k_$ zUCDnTwvA{|MW3F6Oaa_H9}u|aY=2>aBF*f$EAKm->GAT~Hf#?p&EmG(h_B)mD}^i= za|jvd$4g#t`vkqZ=2wMHKCpYT97&q@y#1x94L8M%fv~mNwS-G+LlXR}A4g^)xkO2aaW~zf;@os*SQ&%P zPcW|9;z{c(@f$B!oWpjH&6ZJ9FSo~Sim3`j+dR0}SZOf{2u^A|Y_gDptBvHcQyA<^ zw3rsv$gc$`!!%cXbLUz-@ZS$JljrQKcv49;H#mC&J?B+)OEqAi6D2mXh2=kvz}ZTz!G(8O8Y$Pu`@m&_5Y5biq*#Wnk45pN|}A zuBMBaC#2V4x9&a5uuUw#dE@PPdChr+0jYoiZD#SQF*Mk+ok=I$5VqP98xs_De4R+LP7$1ogquG*NR%SWZ?52Ih1l#T067ZC;`62 zD4t=m6YD3?a{Bj;(=o$hU|$TEr6Lh(UIDTAFnM$~R&EmcN@ndXDupK+-z1>4_~LHU zvr!Ii+p-!_+LX}GD?mnRIBK@{@sL=mDMYN6?RDXb{P%5P>*yBD3^eKAI@~ANS*cbk zFSa;%ErK7M{&BdVqt%!weiHgtLq?k(u`Y=zOn{Dffi<^8%3Xb%`^o+|2dlGrcH%eL z-d7u-o7mlBO;;9SiPpq0wdGs)Nq7I+Q#7yO%^QbhW!48`f2DXb%vWDCGorog{;(qv z`FXN?o+Q&h`ce2RXVjqE09Rorkrn`H2^GbHTO1T`2s4Blaw)tXufi;zg*Oealjzws zc5Lan@!ZFRRN~c-XfKf0lOQLC8L;DMJjbFqs};!d$=d}j=_2E|gt00L1Zcki-n?E2 zw0`n_#>|xrNw7}+d9?PtL_;T?^YNSWBf6fHG~@HncCJP{uOMH!1!>O&4;Aof_9d;{ z8Qb6^ibqWn6Iyotx)=AuLGbGdPgeO?n-eVlw(z??_l+kaB4};PLEF5zx^me8xDsyW zHXKAJ)MUC<=)0|(&4A9GUXkI{Ui`Pk5a#N%v*oQ5gPanpyf}bGD8fcS`d28R7+@$zI{?a)e?DG}@axm8!SCl^3RCW5LxmQUPv+_Zovq2p18-cSz1opFqBf2I zToI6=VPMi>wc|E)7yh{;a4XM4dTKlaFqN=xpi5uHz-VCPb3M1Q4CL0sk@=*y>z8Bm zmM3@UNe`$ViRJ=dfTH)RscnXOsXN?P{JlEcnjT0fX7(`nI+SnhlSrNrvzW~}nd!{^?oH5x> zqV|V3z@@0Chb2m|a4JwNNKc;wsMj1&s&rLHkERV(U7V~2>ho2hY*jh#z%#|Lm2Ao# z+tSb^Y?~MWVQiWZSTn$6Vz!{9jm!xbv@X-C0M?`KSe>dzRTiDlV5QY+7g+q{kNv!c zCp?g{m$Lh6N4zEVa?&heXLjzq@Dw0Kk{xrrRonO6n-=fq<#PpKJzE3A%nfZWVT3&w ze-8POQbUY%I@Tuut(obPcG;W2o8G-m+o5LDHvBaenGfmf#1Fsm zmT^v)s?v{8P8ZiQn_6cTVV|QCk~9yCZ{q+CRT{Qe&n>uH zKBwWg@Rv?|FMYvlwgu*Y8<44wj&0tMn)cE___=lW9&!%GGWID}9&3KrEb)hz@}C#H#f}To!5=TuK*D_r@; z#06y0g!(b#y!+$cm|NF4!^6tem6h3OLa0vchwdSEQ#27a&EVTZ9p!QBj}1IA3mvip z+VMHKoeZZJ*axz$)A|75T!x4H#n$_DYUU@`jL73ed@mPR0)M;-f@@0Pl6a~Op~Xi$eM!9 zE>QmBtJw*TX>S~We%{!5a3VTcDsQ2q6W{)z_om*Gl+r3MaL)C(6u&W&I+bl*B!cn> zvatE7mpFIWNmC!6Q?*QL16lvv@dJx*=J71MBTBD0G1xsdpW{?KDdHqFf4iVw>YMh{+~|()8`%`Y*iu4@NA$oU&Mc_sQdJqL01hj>NPPaC+d`a2pRR9Un4; zLptogy2y-C&dmrp2YyrMCr4-Pt&XzplXo5&i_OL0%Ae*+t_6r%q%Cw<&~kCEVKNHa z!wM>Hr>{4fFkuJcpX`50sLcI1Rt5m89VJ^gHDu%hy&V9CYWk_p2;(ewQ!_y7 zon?4UoG!F$Ma7IPM=`3+c0Y_8f|Rg)Na-K;M1Dw+B~7<4q;Vh=A!f@6b(|KsuNd@v zw2R)u57YH%P*zhs>gPnrW}w#kH2o_~B+Rfyqs#}w0-=Kp5!!4?{!h4IKCZ@Nhum?Y#_o+MATN^)`d^u}7`t64H@h_T_ zi@c*yBSB|lP^tTKj{?W3P;4DnPPNufwT<3ouus=Zi;4Mz&pffnKmP!}=YqvC8ia8{ zbkByJlC_1Dk_r+V%_AslCDwff^(STm_8z{WZg9!n5sM%*!TD$d`nwzDfN5gWzqoCZ za>gui>p~U%<}5>vS-vi)cI2(TW!&HRO0^)xkJgB2?}B`>A>|_z~0yR zxC2yO9dFtiHT=3Yg157o`o&6D8CwE7;;w1T@T+dtiY;Kz8xcPLkptfqNdcP2>WZA^ zZ3UcrZ!CBA(5Yv6ZrtgA1As=A0qUH@q)|222v=h!Dm+tO-yXU^4yx>!H=E`FGFFhv z+~k1?XC{u|9!nUYTzwEh0+-4oj`F8oB#Xxj8J~z|66)6MFquxheH~ZwjJf#jtJ5* z@B2nsjJST?lgj^eyPD>n(+j| z{3LGb5Ck-+E^0T+)zrc~C$;v(Qd#S8U5j6Q{aXtlFI}^|B*h8+X4{zP>6Y3}&ISCy zSc==|L{n?Jhr`AV_a=`#*R$59-nHhfPSV^z-_RVpgS?5ZEUgO@X9+U zr95EF?eTm&p|1SgTICJg;TEy4Ds#cuwW5v;g0#M60ryhV-a%=0U*=8#U{WzxKwNzV+VLX)_nW_Xg-otctHQ}%# z?&7o#HZ;n&|HX6Xk}+Db;3uo#Px290<=Ew`LnNxhS>6VPQyK^sWtH_Y@o9Vg!F#|#nD6s77kn?R_#Mc**KzpU?aGs-9M#*z)t zK+QA3Eb-$v7Fxu#y#q&Y@(yA~$Ci2bX|aiJ*&5!k^)$PHSAUz7J^{CRptMYijFQJe zvE!=bIKBz|r39v*2MdC%ErWwD9cL;*Tb&NS?$b;5=Qq>ah#WKCCbOQ?X3d zIiuN!UhO)SnLAg(@)?wBKsMo+YY>kyh_``U?$2y8;63OKFE9w=Y<$fAqj=X~0dF8` zdGICG1Lr1<28VC#<;Cu0Du0jW?Wn9f^0p%C_<}}MDy3MI+ZUBG1agWWmGVmpMedQ1 z8Nbj!J(f&C@a43>WI_KhDq0exCAqgQb{CQsu;NFUm5=N|hU&FOnD-78wHSmYlyQB7 zy|iTbU$o*Xp(lL}Aqjt7cQR6>{GHGB#q3@H->^N@fpX##unZDu`=BhLe8}|WDqQCK zAe3eO^NLaefn|rN=(;{MZNsq85ZD>=M9SH`YhJekFHSGKgjEBu-@{(GZxcPKIcat9 z4IDEf-C{O2XDYg#(HV=~Pkc&{9k4NFB($*+Dyfz$wE26UEyL!6$wiAA&wT#Z)OhoT zgAVJ_jhNVQpY-QJf1LwRIUyTM*nHB;W~5VvE4x1!kjTK>n3G;L<|CZU?h!_7D)_DY z3w|TwlU;=Sf<^T7u5e$P*M4*Fm&Us%Q_G%ztvxW2)y54#w8O2Vbex9m17snr@9REu z`s>$FBG}MFJ%C6MpIL!m!S&aZ8f99$KmApFZRnvh;{6C9)46r#=0cxO=g>z%`w)IO zkNA5l{o@_-83$e^*!u-z>!LN6H|Fo^=-~b_u&`}(NHVJLpK0>vkbopOMiiGv9I=Ar z(YcXo@Be$T_n+$qIeLzExoxeJJO4ru{v9WLw2u%RVbJXvci}(J@b?P>1@JtBL>{bb z{q|k|eRB(XQ1vLKoCst7YX<-Rv?PpL2w~4JHof*I6x!zgGfjT~h<25EAkwZmqKV;@ zP=3al+kJUHcPVnA@kRFk`bcP~pJF|jF`NFt*i&i0l3Bf+&sqBW`-1DQ zK@F9lN?;eA-4l2Y0Qs*$%7voqK7{{Y&jD(KPGSydKnobocL9)kBX60_qT`5f^+kzv zV*L6Wn)1Is1I{6hmR$`N#s(XOajZSS@AuQ)Ig`y=4|vDu9)0uw|13E)kX<~Ng@zoH zOWmE+p@`+rnT+XUOuYvVk1>vMw?!5?0N0%DBXpWokT9P1?z37!Ffm?_Ax zlJdAp%@|XW-3PEu3*Uq6`;#+ji{#%0(F~=-#fyq&e~tj=7RX|+68!sn=*Jw_(U$0< z*AsxYS(<+L&nd%H)c+o@zsGS%<^%oiT`=E1)rOJZHhj6pQ|U4Ju3O{(`}Wa`|_#Q9vkX#?QSW5a}eNhwkaY|MTp#Q?7XW z{{VsWfSyYS-RIrXugTUG%ePi|$hv?5ENO||1B#9?<6$Bf5YrQ#2yR3iYzZ}d5?;CY z)pm90@Sh3!SQ3m;jzFv=9w5R{uI~Uzj2%b<;iWYNL2OPaWakJZ^$%}~jI-SI!H038 z|4|GAgdH-@b9q^bD?av1ZB8H_)~jeCR6v*ecSQ*=fSE455NPzn=?%;gkSLrWdunHp z)AiQ`m^<`BHpJCn$1TwH*l(?dwB=@?lr?GEBp1&IKk+ddj-Pv@ROK1ek-twWp4XGQb$vOh*XUT49jWeG8byr zeFXv#@~=z%4gXAM8A>04$=izCmmQlXZbvRyn7iMA<7{3~39#sJhH9yKkqa<9b@QX~ zigQNIOI2%S>!d2-USaTUsvxlX45jrMeI(H_^~&5=Hk7^ zxI6b`W*9D7O?)8$xa`Cgd6=P&E9z^jLHQxKQsS8_U=xo0K`VoVgYVY?+ZU=egQE-* zC&*s9>hh%MasxQ_ez5_bvFqiY6Wt7*OCS0m8)+wu>>mWJAEiY?kP!7HZ>$@zlhmh9 zfy$r0*kLxPCR$ZNXIu&s20>VU1#3i7xyqkdCkLC3(2S7pHSjXt zNTE>-=IdlO*blJ*uI=lopd{s74??>tsN$_2t9>lczYgxHa0kpcwSv_S+K7AsZ4a;g zP{Zkz*OQhrz#RVZ*#7GYfMl-1VXrCHHGRvoaXTGi_*Up7y*^pWO4(m(cLp>EePr0e zpu=J?<7g6d5a;`NQK+887)?J*0T`DfR)GVcYP>l@*X#$EtMOC@ytsGVKWM3b$yIN9QVD}r@4A=aCYSzabgxoce9A|ybbwjLx>yMoM26ok_K9g0X_kiE~3*^C4 z);RkHWL$m*Q<0zvR#iLSbDJDW<=w9p*4bHCV;07TD*0m*$biXF7Rgbfi9ae$DdMlv z^z4fu!N2EB?TJ;cYE_rJ;1reQHQO>gogCY6P71_qxrAi-JFLHQ{2Oe@5c#-t^f1T{ z5iJ5v<1ZfEbsWQ=65B{ny*Bt%dI8CoNxB;V#^M;&Ub*_&s(y1!HoMKw^wV~ljGSH4 zeis>~3ufWZaSVN>fXo6B8QudA-0FJb5@{QlHO^z;O&urb$d}X`eC0u%OW)7^nT=Dx z44pn-!1G>XP)t;nEZ7*@J$QP2>1M0|d6po<;J40E-j~kS3f@q!PDbo1Jm&eG!DB%A zTMrkz`1x*Hv4h_BAq__nX%PhGn>BU$bq}!!1sQI;oe3*-D_&|(r*{mqlu^v6La4pM zz{_a&Q_0B1H3w3Ja& zicC7s=>qna)dd?(=?N=s?08!7EW&n*T!>TW9Go<-tarim(cbn$Y(WQ^U{gh1by_#= zasG3WJU78y-Mc##JBKif=6pAVyMKnX z9u(baoNT~d1x6*=;cxi}`bkZ{cy7KwwsXpNv7e;UTtX5)3M~<=-LuEVGJ^ChP(Tj- zBtJMlT$;r*6w8Ht6y$R`)nY_F3Ssw7 zitT?5p2b(xWSweGv^%uLQgv*Y;$$oTY0y0NcgTH7||IZ&Uk3HVGH zd895jZOX^KBy^hpHMc#(oukA*NK$doa1LH4H4p#vD!jiU@28i z8_u8MC5x{ZE^<76KXK{@(oC<;N>rB;4L=)`l~wUL|2rEy%(X*9$93g?zM;A>6DuFa z_TiO-8FLbkdeJSospJao5-KOaig2B{#Wn)_`Pxa`r8;Q zbxMdxNyX~;%P%hV^J`8Z$W^>cf+1}qzmsMePr2VvXV2qzjrcxpq&I2F#h z9%w>s)Xr3vFf0<)F5iveNFA4Y0Il;plCaZS6zVpE@UxeasF6P@auN~9>&nfx!nvcDg)^TXR!O)9o z6sq?$N#V`HVtWx|w3o|ide`2bsoS2*dH>~^iW9kQdo;NxbLxXGo~8D0RJEh74!M*n z^edkhF8Z>PH^1G>c|R6{k0!Q*Y!>yZ6ssLZRXmBHIzuN7h@#p~HQz`$InO~gPdsNs zdHp;Li+b)8*bhO=PH1b9_U0F3uN~dflD0E<@DuN1RhH{^bmy;VZ!9cC64br^@h2yQg(AuOkdo;WccKiph)lB0a>_j5iVk z_VuP6L|7Q1k&M1K#Ip=BaSJLFto3R{T)gp_fLDgCI;U(9q1R4?lnqUYJFCw-CaDh` z=DV(SYZdO_=eo}|zmO5bU|mGnF@|wTC|2 zL2$L-q%{dytZw0=uyfpc8!Y#A_=$pgKgraBr0^lWgy%850Tzfqp{}W4mP#({j7a(o z&(|PFH;)RNb2m8u9xv7=nSGAAX`76q^?T#j@pWbSfG(~AqObC6_rt`r7&V2j9!sfj z)e4?d+q6a)Fv^NCq=^q=m-v4$74ZqOSPHB{Z5-6sPDZ>s<$}4iGU4W^Jz~0=9(c8F z{J_*h;Gj_wxNT zqioit$9(&^rc1B3hTnePeg9i6D{%%B6r+-<9-K#033!sxuGUiZc`b+3@?EZ+_WTfy@ zIYh>~5qL&`yrA zPO!4?JN3ejJ?L3f=m_#q=UZ0;jc9_|(pzBu`uh}v<-*L}Bv|prpjAq(Sn^A(3aLaZY_gcUeQyZO-PvyA$SPb8$ zxC>$O0NXr@#WAlRBZcWJHAicsQM|v>@_~_T??_eeJ-S+k&*$hBLE}iAGUEZ9;N|NQ z_@lNc3|E~t5+WY=G z6$7+iFEB^wdTQ#H1xiHrePz?jF+r_U>61#eC$-dVcWrqJm{Z}93y6& zi_+EGq*^Rcbf$5YW5*$OY^Mb7VT~7B0vgB7@|E~SQgb&G*KXJ=q!QOm%Z@Oog_f-4 zAzNTei}vzb-xG*!%*bhoABS{&)JN`W9Q?4T_oosNtzo>&y&&1v7258OvN%1c`F=}W zx{~m5hhV$3;oPr2XAO3kQwOtY&A|b`M%}f$jBuT{#mddVV^+TK-_{H)82wD+Wl`hB zubE$&i}w$AAMRh~Bd;kwkrq=_65k4Z-bEzafH#N}vUxGs$~uSlT+HNrFJ%T>^R=-= zYI&g!p|y=#st1>-a60nY#E`4+5Tvh>=F+`yr$b<%KNSv~fldwDCm6%qxV!!kZ3_ zQ8fH}4R)d`1^k9CHT;bFbA7+=-I)>;4NphaC}I>5ZO%xh_?@3hPC4tGWZ@Oqo(p1} zX5U>6_5#@HvBLc|*-b<5dW|CM>E`UGFS~_gsltY~B&BSUgH>Qb6TH>oft zUnaBK7ByOpjt5b{E!qxxF!}O)(0NI_gJm#&XNykUcMH|G!=SSnzi`lrmOnEK|1nM1@o#Rk9B5*?Qdw>66|A>#V2 zyYElP?em+G^xKq*IbsMZoK^TDL{HWxvNmIm)C#ZIoJCx$2A4lpiY<5HLX`Z8ex>vE9Yquv{KtqfH~ zhZEsuLOamN2a~6+i>$__UW$2xg5!KD3nSLAW4#&+=&|qS>+&Z#HmG^7%5ZPNQflxq zZnk>W&QA67zq>Iq@7BHo6+9Lwg-Kvvn#3?YB}pUAZ!EvDooww!a_2SO(4{)L+}|Z%J$I6br;VO0D6-t6 zDQhQOF$Q1OS(%yy1UMP{J!-&XPvcNE*F2u%oBUx`@XO1b7{_k z_{$V&y}epOspZ`tT}S^cdi;J3CXr{oA##sa4v2ST;?i0~1L%3H$x?WO$;n@yQr;Yv z@_;eh{G?eFFF%Q4ehmMEXEBZf0P1uV&b_cc3H|j@S?w+9@);7mO6ETrOt)v=g4O`)b+{oe2t^cYu0)GiJq2k_|A)83qRz0wq$xSCwJaCkH@)Hc38$)mRib*YzwXFEtH8mA zYgB@@taPSgW82+I*Vsln#4HXwez2?n%Z5hACA+H(A+G;aiTGkj8S&VHn0LoP?4VUd zVXP>$hRJ+$KvFNEEnr`^dTjc6Q-J#i@%p4~qUwb_Zq%D(uLPmEvf7fBXJrz@cxsDu zDZF9a;b<$|QpKSrl-cd(>O5#GI~$rw*LjxtZ0Qv#ddm90z1IJm74Q(j5FaS8ZG5G^q0au;c%Ov6V&?o+(G<6 zSO2o?Vqh7yD2}mlh?zewmOo}IT!o?)iXt;pIrxDeP$@z91b&Qw$z`)uOK0cRU!7SI z@Q--#?%ezE;bT_I1Gd}lXLo)ooeTslDVE)LU*j30xM)HTg~1;Po>0+3zi)or=OAZNpZY?(s}VL2Cq5>$>5w!D@g<_y%%OyWF#Qh%M!Iy*#;9y=wvLTb+y;_)#q=-nR~^&_d^%>1;1F`%Z%KaLP* zM;v4~WLL~rC_R{%VA2E`6^<@87+4-&f?RFMY*AG~aQAk)m)`Ny%f+deHk;hO!I{MN zjEuBtn2)=I?1YJUsZ;G^9!j=7Hu(9qq8R2jLR?3!o=%yEAu$CU!RjrKQ!kMwbO3C{ zVevNlYBl_*&;q_Ly4&Gln6yXL5yh;??kGP)l+NHcH_G3N8r<7 ztw=_O?~9+pQ7ac8Wm-?$M7tfG^g^3OBy(kZ;a>f8D~Blc0*mgFBhkrsKt=N6ICVaY z!{wj8t$#xZQJje!NBb3Ivxk?4SfSWu64R7^JS|2;y|? zG@w*^VD@ld&~OObHsMhm(Y&l31?>$nN*GR!EPQ{>EGN6~-gM1t9V#)ZgV?~P%T~X) z4Mr4bvR+r*5B>kpSH&P)=m4_EhXNx#VT_Fq^*rB$B!BntIGN5&UQcLhb$#0{HvTlL z6XGT(!0>*z?*Ur^&)VU59b8|>AES%8=1Jf~np9imn#P!CJkLV>;y$a+u=sNor}LP? z$l-*|B`%)3N($EOcVc+)7-LOT z9R_^3H}7keSidq>aqs;lN;X4$WYg9qUr+7!He!F$k#XcjU_*Lpi{{(t{f`a8KJ&ud z_VJ8mrTglhs0^A*CcO9#Oyx_ICZCR1Qp^uA6ZZ$Udj4_lC>88d8{+>n9N-UZ5SDvh z&!a?98dCa^AZ{E39w}xBFVH35zNz@y!wzN{p&a1Y-Q0M3yT$+ECmsYvInD_y8G#*) z3V|V)eZdyB9}+)R0qmAp%IFI3EP!6Dv1d~1cC>4>&qwR!R9RDp>D$uTA--)``Pf|) zJ8LF-+49$Ct$eoMq6)j++3r=)dg!*NHf3FRYSlez%k?(CWs>^vfy1tW1l9~Cf@9K( zj^mJ+0)8!a{|GeR?=Zezq?-NZX0qUU4jZT71U>kj9frsbfo78yUOvrp&}Pqd86j^| z+ag)Gffj%qJd(^uKB;GK`cWKgDE;Aa;G+05*n^{amu4#YJeu^IcTwwN3}uE5iR~h? zzl2NOpJ8?JJ`T0TQawYmTp_65V)nY1N%L5xM=bSzb^~LXXgY5AO185Gm6#_|6+yAG z2o&3!ovOFv7UF2YS=oh3Plv&O$7%rc@?}kTFutT&b1yx*U|aa+hZLk1F)0bEHu*>N_tzG;^9+51seT*NC%agCOn5QDamcl_ z3CJPLNznpmnxDVuoD1swv`LPR;E=9#Z`!%-K+FHcgOaD4V&L&!h^5O(zUn-G4Lb~r zt3~Z+z;XolhM(I`AecSACxaX{r$$IfoPI3M*>Kk{M%YK+6XKtI+WE4jjRX78B zFVuCp|7YQ%P;G1Mt%!&ZN}ouIfH{0xI3)q{yABZ$>prjXk&f$-vicD*sL z;koRr2lviR1{jvF4$$aYWxc;V3&;zOZSg109s?$`bPoZ*3frS$0HJ@neq{0|#WRdJ=OgSA_phYb@ z;D>zBKftEXdW)No5zi5=cqzuy?(p_Ej_T{}LTNpI!Bfv2rV<;yCR-i86=)*+G~|Pj zTo}1NAL_veVsybeB%AbWW=xc}J`KY|`0mc!{Erq&8zlRhD{Y11tx&$lA0>=8D|QTv z>gxyxIi086w}mBolU->qJ;kD~Y)^fX*u-LZShwcH+;&+XIgkl1Q4iURd6WqsgM#r9yKDwBBmeF*kU-`aTZ6et zIPTSzxkUj()R5aj>W$B@{E2#Qof=l~p$G8}u2I~;xse!gX9$t?;SAm+?U_S7kv=0- zGmk6acvt<#JRb(}sil7tfia9R#Cki3NS$j2U7h394u*%o=y*&4@`TBMcC!5ZW&l%e zL3{f_Or(sEwH2m?sm>rL(kPW!L-VnVc!8u_bIXCDbH(%e zw!)!=>-%OGP$2dsD;XTKw?$Ni1-*=G7y02Wrt8_tO)F8ad$h&@RmtRVBIl4U5fdw$ zO5crrKqsubrF!1gn`X&4R{HpZ+rk!e>WSaanTV#- zm-&-6#VkxYy&e*J(in|wkNWe52|W?s)7R`O%7w|Xyz6w1oreocWB17cr8(L1+nvAc zLx0`4hB(%pFMG|eA`RxBB;b4R_zjQo5qE!z|$= zFFuE+ZSoKKc$Gh=>Y5Wojh8hlRfpo=ZK_!2Nbh*~ZY9JE#_*VuKSF*sRCc~c=JwtX z+JSTfoAsr!60eC@8##mK*U!c@=?*eUCn*xCmZsj7E;wVCS{*A)u41ZPUasTSNdn?yRVy2Y27pa>t7}tD6!zT+2 zB2(IYxJf_G3ps~bo>=-gjqdd?6y3N4_95Ug7Ct{bn4Qe1pEjhCkW8V{tf%wyvT-54 z8IooY5}Vnw$R3qlKWvob)%Z!$Zkvq_HRhvpE-ki7&Uize%!}%~4{CLITWhkuFOO^A zCu{L@SmTmRsbzKA*+BlRXWC<@5rXBjNqm+C73*X_TbT8NRLf%9!DJU%eu}rQ-z2_w z-hwiPdi{YUA+<+PX^nWVZkwML{&q6!qr+pcR{vkYh$0r^L;Od1i5_NjoQE@{G8D;! zgUq~gcgfXP-p#>6cVBu}bOP^}nBLQ`De;I0S@EMAnnwV-7(ZcV-0j##qqTqeaPi%J zTo$Ds1vYnh<(2>Q+>#%_6=2zFH_E3xX3o+US_T7MvA@+&|NF;dR-U}d)bgI!w0a|u zs9A*@ZJwg}sXu#%u2BH!86z@N+Oe6yu(43(%|mv}!X#evjmSQ;{%Q;1lZug_0}=A( z*O$lnMv7t%wBZCyIN9$eaOG1UwGCn61T3n0h1>jBuEoQIkpQd$g(rG4F&C@kk{`IJ zrqb$FX963D73Z3Q@2-Uj@3m9r8M{(FKFt7c;LB-leq|I_q*}S zN+0;+9CWVlV-GJ@W@H4WZojwex#oXz>Hit}e1|Fhk#PLd=P^CPXvaGI-@Jaw&?W44 zK9c#5Sfl}m_(fG0Xnk&KJ{u8?n~!(kCSbwEu@zI@9$a#de(!jxJzk6 z9hex17~j&=HjBPbvjtGz6VssV70|Qv!xhZJlG1waGQ0)h*kisIse(=MHp>GH-gifk zQz}7AF6;9L`LwVm+ZwtpX%&d=e7~vCwZsRA9Y#D!(%hA7Cq}I0{-iip{&3MRgRko9 z4lTb{*E~(K2LH$UD~Aar$)6X6!^Sl3k#o_Il$8J`ZHykD`sQ!1mRtS#bHGeKc0t5L z4bVd6IWX{CohhC5o(9zD5nD*52nu=w85@;st}m-V3?kbjK1}|ghV2o-?TNjky36vZ z^COD+G8NivkY1bt3Q>uXORza@;c-^F34#>LiL!^^>44&jxZL*eeEU%HdS7x{3&?w3 zT~mJFy5h4vI`3aA2-z{w%(?6wJMWsjih;CLd5pujLR0;lz)v=9@4Zj;RISYZvTQBj zl>a+SxUSc81=W3bB3?)Lj9-tRn)0!Av814_iX#1{y=dj}Sysy5&0IL3vbTqnF!6Ps z$@s6XFzFZvwRWk{bq&QE>H|5K8@ax_QF$|-Xh@NUg)mqM29+ARy)z1Y0QJwh?ptci z2RVi?`Iz`25;;os7{I~yAo|?x4e7tK+n)2QVbzG~BgwA#n~?Q;+NDhei$O93<`=v_ z?$(_wY@rz4fsOairx)Wu`dZ!hdYRv48n_@IPL|hOwt8&5=<_yrudY9+R-Lt_aBP#v zRX<|7`qc&w#Ig5prrbUIxv#9h7ny(@huc~hOx`II1x8@&R8%7@{09+Cq?rJ?7PJrU$2I#+=B$G~g$&Mr1%|A$2Hq2PM z$88)hovbS}kVX8D4mAFGTnxVu!i9d^|9BNbi{9ta>s;klx1UUmS<)wQuJXDbK*qF~ zEac2Bxpl889>S-qP5)MHwh+n>Epf!|sCIF)qkoVPyFTs9SM0I1e4I%Mr6$ZI16tA3f(0X!<*MIe!=+e*ncwXnJp|>9M24gR z&@Wm{Iy2GuaH6%1i*QkoxeRjCI#5GHeejA++X0>1W`086ab6Is&Jjn&`IcuD#H2(9 zhmY)^ZaIbG$urobxOTE1Ch3>zapYc9!qN~!mR#P}HYEdcrS_qI5_wPZ%K!6NqfU!9 z$qkS(r&4i5=={MAdd@w=9?+^7VvAm9?cQ-Za|)cDy3KBy?lb#U&}W!->bg5AzGu^fU2nE)mgXX^@U3}Cbh$ZHYSBeU-u?mWM=Rt);;O_hn674 z<=fU&N?=GwA-beMGZAh{U zUvS36yQ1{xDB{oAZ{;!jXIc4Px$luJIE>9e%Q*2|yU9lac!@RpzVEhvHg9wCzR#V+ zO?UZAUNnVyPl)ERe>?EgSHAZ<8V$HVq&`J_Gjr#}>A8rcM}$au5WiRrWboYjxG>E) zL|{T|DNp0_q^JBPaKesSPl!}`bPx|?5Ht;lwMGSwNyRzZPM}=&AidQsffnk~@}Jij zk@?Y%-*v}EAnthE(X>-7CBE+}WV%l=qq3_QXmLGwPKOnjXL;>05Pvw}vRw5P`8=-= zxU9+pZV=_poS=k8%z!HS__y%M#RgJ6S3`&e(K?t^3ysRpXvRp1d}PN6Vfb%Lk|8JY zoYa)}Vu%^uAm5RrUkXnh0T*_IWn-)@AkO$@+jpz1dyn4*CDTjdo#)LQ{tT_|%se}{ zM1VcRIxc+sGg*R>?lDmS`DZ0z!o@86ixa#ki`E>wkO=@BE_+3GZ(ChNA08i~h9X&f z1aF6U7vdIhua+1x^OiSWpWs1zZ7?jtA76M9f)rceP1sDaj_jsCfnSNkR{vS3_~w{g zFD0t9hTorATDiVLk|+fJ1%r_mQ##=zE4qhdoxj~Kry=L$3aBfx(TV%eUGTg!C)t)} zF_I)hJQ)mo!`Z5KUOIW-7W)pP*015#WM^>0Cf)uR+$qv#?Kj7e)1zY8ON=jwjDl=b z>!%xI`;?AX)pEVJ(^H*$JQN>BFey&o%I91m4=;9o{;XV?P+LMjQg1(!;UN+0@cJue zQ{nP=Q@eVns)T&kni!HQ!}B$RZ0Dzhc|DME-NmyhYJVRH*mF0}O{T<9NZvfNmBM{( zNllPqaYx)hTccq@^`>i?5hJ?f5g~{UV+RDxpSa-#?f0y+$Qv>7L7PvG|iR z0Gr%ZtxPH^c0G7UAkDpN=Zh0}MLADNvUkN6^E~T&!rYKVHzNE`v9Se;m*@R~5>K>; z0{(}oMBPS9LDe~Xafco*RJR|u&e`AV^4boyHqgZV*atkcRTqB?{pXn*kUxhRZG0Qq z5sJG*Ix{3+um40nATegPzjY&&aOsp)EKR34ZU}i z0Xc2$Nys?sej}_y8^KFLNXw1*Cnn&)bxAz1q%SiM!gUL1R zJ$oM%L4-g|klSEi7fEQpwh4ESYz zQFEaPuL|`Q{yF8AO!$OM+jQX3alSQ8jiLXq!}F(F2Ul6xj}Pyt5?CiblXthYJ(J8k z*me8rBgwT8cbHcVt%QgAhS#&vb?`R(_x;sCqctkQEUC|wa19Dtljt|+U(c^UZgrTv z!@apFAZ$MC5|MNB+fV1;$FT-ZraA2Qj$cFW<*!QH{;+7Ezzn~U!h_>v z98!S!PQjEK)z>UX=k7{C1mml@1@Ah`3*BAnkce43UikA0thmrQ;i^Ma3Sk_?rm={Ecz+D0L z$V!ZM4~z{VE)wUSN8piEoadXUR2##d7rZ=>p4xk^ndEa>1aVv*vRp1RuwuD!luf50 z^Th0t87LrQULT4dtTWha)C2gM>AfY&u}b&K-AKn=>`P~_uEF%35N(stVPzW0tc72* z?~%@k_%QF+m1GX9D<3R3&m_nCuBysr1VgpP)AR?iW7j(!8{fs!3@A}Ksjudr2Ocvq zP}@1EoKpXu{O~w|MhHb^8>=`rbiU-3dPB&*5y-F<2J6t?C&4Q6tXD6eH||6)5%uDS&~# zJz2TJpBDaXmD+Vv#A0)u`d+mb$667tJ@;sU^}}g|+>HL33)foa6#IK`7#c}Ni7cV2 z=3V1-Jst%4s0@cLId7!!!Nc)hfrTn&F(TDNmwBf!ZXnw#T;#VeJg!+0e|qBJwOYP^ zQ4ndbq>K8Bn__)7Qg0BKjH5mya54@hTU|3RVC9Q7W#9K_+D#c`ODDK_w)N^I4i8%x z#ft9x%z^tX^3ip_9#^xKI4=KfXIBXV=1BH|j;jBi?X{)fG`8N}2xfk4RbhXiVn^q~ zcNyHMq+b*}5*EC9>v^YxpPG*&*7_`~}3v?-iau~Sqb18HS6HWoT- z8>6YmOi_OjhQ1iXkX^x-@;7{d!R%VH?^~eordU-MB^vVZ0eyj5VI7vs+H$C@LO~7b zb{jdG>}4<4;o&I$dyh5i6eHCf=5!*C4U6O^4&_2zZdzKQ(T@dp2K%yQUeFb0kZuxa zu^RwzQtY}pQK&&R<8XiZ2uO)9w9E~2%W@iS`*;bn6CYRalNbW8I){#pO zmdwkahl9r-3cc2Sd_dO$fzP0>N*}LhaPwT~N$sUz#!77&O}1-I!E_<>9t)!?N7t&Q z&^b8gKt~-r^hL4ftK_MZg9%rAHTb?140B0D+gm}}o_hjJhoxTPU%T+r_T(TlR!u5qwTq?U?s;ZfX2%W%+0!cgsF@*_WT`usg&Hi@%BF z?@o%L62@5Fy%?$KR(qrTHuv^1BVpWSc9*;BC_J>?`AE!!c*6<|5>(O`?~Gp?)ePY@ z-zh9%pyLL^$Y>NOe%RT~^58>6>KDnbo^D&-(ti|9;49Gch}eEzy&E2XG(n7;|E)6p zraerI7JXsfqgbIvNN*)(oTB?By`03`?nM#GaY2j}cXVkSRIC`8^t}uc&E1ycmq**a z-`-DqPW%1+VShV|&1ule>^%>4Z5av{1Ktxz!4;8U9 zhv8(1M{Ir6(*jaIY53D2ktV`lR$z7#Yvp~9goATC^n13DX4K@n&w0y<$ zQp&NAuh6iBV9v~G{vEk18_?7f7x#5}{C1VCI;jKG*cND*Z9}HKp!Lw?*!L8_qomXh zKFv?>K~8AMpu6rfzJYc3(w4JWKYbD~b!tSfFZQcJVqKZ&RmI|WICIVl=#f1=29~_q zvPP3Yg)an-?zDSrj+Pyj zPixifMc<(=bu29#G8640O|`XW6+JB7TfHX3t0a?6+?4~O27}|g<>vA>@#VqwQ zmw>*>%Pz5X?dfEU$Y^rG0lBzNkh^y%KI_7# zOLyjxcz{h;@e1&O<+&uFUhVe?6A>$Sj7Js!s927OS?OMYW)y(K#|m&Q2s0+7;%`Ax&c zkA7{`j~NHMfSf;Pw@&uVm-yt{#QfM;SZBI~cih7vD)uzg0j4~HVwqf85-mt~QR@{8 zEX5##QY*qdn5!&{pBUnnWv62CNk^vur>bTnbpFXbCvbM}y!h;1@f)0!iXhq~(mII$ zqXZ=FYW9)piwn)Tn~)mn9~H$87hVDfY#!}hNN!UlRJ*ELdIdOi*wf%G)e|>^JrMJ+ z1I9sDZm(5k(Mj!$e30;_eW~8EcU9TvWp~KQ$fNj0E$+X@D*|{t2FlTKi(GvK7;=4v zR2nXqAfw>#_q$p3O+iCciyL9YyI4$nLg_X&(ak4+et;}9b>t(>#s1{D^S(P$w;3v6 zm-D>)fC*Ov;Y_pTvBF7G?qs~alEGqu@`Be$CLv>AIjC@5K##C|?Kt_Y3^44seniRy zoHXB%6$LK1ZRY0TQL^{U`MBYfv>YHP(E>LWiqgy`#~%n_!%YjY=!P1r6F$-*RagQL zcu^j~h`KHP6)6q1qDY^(*N&-4Btsh11sp}`^gNIzxAn;nZw(4m-XcytK|sTTQCCd^ zufscXySEk^nS|6a;uZ0XB>uN4L4P-JvOLh;6?JncQ?6&X`$zgH3J}HMT)pWjCDI-=3S)q7-9(2U;7X;odvmI!4X6$0vLqr zNC9P%LjO+-!Rj#LT@?9eZiUI+6p@{V8|$wa9tBc6DU<_WnhVq`TNiXHA-Ip^>V%^B zMzyy+AJ6L7^xmH?l~p-w<-W^B%Y_fopKNl&g+HIvgPFk4efREP791eINJaI4!V1ME zVqUWPbqrs|WU%~nCAB`$IEcRRT5aRaNh2wK}>hhTm(83X?EpM76CUwdWasst!J8;mDQ=6=YWT<((8(B4hhlEWwg!LU|Xel}e5m zFNMD*$bEJPj|hdx>O6Lp{rF$0zA?<5V9cBNNOYLmC{iXvo#ii6xj8!;{vXV#OJDaO zhSCE0NM7}pPShL`3yc5*-xwpDOvJ)}EeVW|p1x#A+M#9$X_qpYgw|t$%#feu{uGa& z)C17i_q2zRE3b4KJyAm=M89InOIRJIajjA6L}khm{zjJu0$QdrY-e4+ zyBj~;;j<7a0qkD(c_u=F^ZHOW=CqbqBu=@Q@YUZ8nEw@2dcZ#?w^gXv2kC4KRS4({ zU|I{Z(hdyF1Og;Rh+G0o9TAXdn+=ot^un=n{o{wUWfsPH?ud7o*bz;xom2qC6ZS^> zVy^Z_Cp02lOEdAr^4C3wBJ)T?+-C`_bUeVx|Cubk%cN#C-D$hL)!sB2u->;+t;cA` ziTz2@dH&>XJjCu3UC$|Avr&Ug_t}~1>)XKbpfyQIbO=_UJVAR+b`MLW(i!3nqh4vz z$wKhx{mGSoS`yIi2G%1m_~~AZ)Q2|lhB(FtHT2T=7>up_>^-z?Ysi9>+o$zQ{`#(qyhX2?3 zi~xQE<7W`m?_IaX!4ZYApgZkKsmhU=6>t#up#Nw_0T!DY;4VW*X0(~)OXoFwm-QRcS z1q0Z=FN{iXSQg`Yu%^H)viKDC9TvFXl?$_SD2&o5&>j|6Zr5l}mE@&&dAY{RH~ei);_H z(kcP#CT2g0Lg;qQiCD$Pu27=*fA879|2Etp`V9%fj$l>vYTK&hU6#B0U3>yqN!wHf z{~SRa89G>D-BtwjmVB;{eX4yx>TvVZ=WAHLMoe>UeSnfHW^?#8{hu2_0Ds@`kgLvl_g7)zIVtOrNg=@3h59y=yoia1d9xQ`RFm`4UJ$FAtvZ{**{P zX<@7SZOeDjhNG*K_(BW_(YZG_QnHo@?fyPqG{Bj71T+9s6fVkuT5z~N$dq&eBoOo8 z{Z7wDZ2f*NV4pN0ltBTw5rYFXw}yi9*xbe(JaNDTOknzZcHRU4m-ekM19nVXN%Efz zJlR-~0&fHI)N4=MO#JSq{O?SToHA@%vx|Zt7=ki~xTX6LN7WWowbPGN_CBU*T4=+% zAE854I!3&3uWF_DRi z`0|twK+%fXG@pY@Jmb;PGoN5Jm}qHewPigxh;`^0dazEuB6xM=^9m(8bzX7ZA@Gwr z4s6e?x?YPp+f3iCqc(RM-a{;hry)k4?20;Ar{$<@hq!>n{f6$6$=_G|_mc15a6n|p zU*I{BZ4mCV#}&4pVPGRxky8%rG+cxv^S>!7cI&z2c?^$1C=>UZY$bSU z)eFPgs`nsP4}>Y%f|$y-E8q)z9|OB}{Ung`SkELug~D<_;HP+HRIUNsbT;J{6zMzN zR^cJevyjPbr<&{gyL8pmwH@XH{}1t>Jm?7$`2gf0(EZgQ(6l=<$w?_N^E`@tz!KzV z=hMOg%f-%TpJ8?pI^T_NN6!Lskgn{l#U$|L-4xn{QjTA} zrWpj&>J6Zh>w61k#B~rWWjeQ@=~i^xTwO~PbXpA{v5c)K2S5&dJHeh*DD}Iuu~0x_ zhL_MJgTg@Yzxx5q*OIm0CRQr6)o;#fu!OBi$^M#D^D1o)(= zJ0=MllM<<~*1k;x)U?jKAn;b@P$nI*R4MFC9=Qd7lGY&_e3 z%kU)$><6Xd=i022xP^vbqzEo|H8?nLUzfT8z6I&h<#nu+%{>i`ClB+HV+2ngYDf`~ z78*6?eR%fJWBjs}Od0XHP%gCan6eNNN`FL;C>&tSLef9;LuY(`@AP`q6AX~1dnPRC zw!|^}YG1D$PFQRgn~!w!r^c5ad)J-EYfqngrPPXyLBsyiGH~n}+sgVJX0=je5m&OZ zbNWKLH=K6yvrA(=|PXlQ7HZh$pcH7(_pQ&F?)-{;8bbxE&KV{g5LFF>lXIe z{UP@7p0!7rfxMO%V<(^ogq~FZMWTclyyd3_zurOWdewW+3XNsmswb!On7${uF9fOX5ReuPs)-6Xu7K>~7cL#M zpfb^=8~b|Y2CC4@=dvlYVW*$N8D$HYP3Kxf$dlh@wgf=Ajhlz1c`mslKIhv6GWy3= zqXOXStg0rq(<_;QquHv>f>M74kd=(Bu_4AKTSOpcv(6L37zhfXTMMb)Cr;e?`N*2R z%c5#P z?|K{32SZztZG?JxMJrcAa`fH9!BX(i`_8J?i9yA=!zQWne-F`Se+ot?HQ-qR=)0A}u<^Jph9AXEds@Ua>kQmPP%ngj<=E<+SL3}) zx#^rD;kMH_o$tBZNJOi4e?^zdpDu{t%auKHYE_n~%M*47feAQgZAHXt6=&s}vDwI9 z19%Mb7%)d)2YNn4EdtyUB&DEftCT+`gMU;mu9^=PpFdI9F(edwO2}g4oF~YIOH?TJ zp$dp)r#k`NSlf2r#HA>j=SdKgZBT?a!84@(#G=3TKD2=lb8r&!-;BBGxQ4hfzkE?% zv&1#+M|arY8$u(Uc(PQBzL5P-J~`mC#bhOt*nINF;+$uaaiio$;ZA?>JpW_g67NL1$zvfgFjV%_(F$L z8;U=sb`G6;;r(r}P(X1xkXOjHG(=5z%wwA|OOdqpM>co*lT@9^^>dX3!FF8&{1vFahc4y8Y9U z0LEtXL$EFNaWgAE-nU=ac%?5AAZ(1PjXMy9jb9eJp^}@&=Z1nm&b@R(T?@uSV9`YzkKaKETOY zU&J?E{b-d*t8NxsBgw_~yRJcirEscnT?@YRFA(l*I_a}q817}W9w^d&b6?>A4+V9< zI+qk`n>}7~KXSZ(42tg&HLga!`W`9`3I8sxEQL%h8Ta6CUYx)T0L5qzGZ@fx7dpQPesprEK?ApXaei!zaf- ziQAs0a!v5L(J+0g1@Ez~o7JGuXn;4xjkjO;1Pl%aPs2PESj>L~o@N@%I3o@!^^;_n z4^*-U$2H%EI3M(g4GY?#VqlttRvRz5<~s2%ox_fjKhNBs^^Eag=x`3J^NdqpPFW{3 zSzJ1~0p{3G57&Z4^1|~BOf2v!MoNr0ld^GQP%nhb#wcW{Kklj^d40w*>N`F29F6Kd zWH(sn?r4=;X$965J@zp9UUR=BKleYI^s_KN_<2g8cANvPk0Y+LFI(6=M^%SS2hHvLX`eHk+*K4)SjTBjY zVa^GhpBHi*bT}pVtc3%87kjjCXDVtd8wumP3ePh?Wn?alaL!Wg)e+I^jAhEIb2?ko z(_r*G>OIqX)hL5__l36JbX{2_cw%2N^F6;-{Yx$H;kE(sZ{#t|VH)E(Y6Fas&Ow^) z&zk(d{vruunu!*+vpIs7`IS4ZF@4T;@x75UmWte`Si9 zG++(S?sO6i(0kGgu0nJpZbL->ozaF^WNU$R(D(ff$si8B(|_M8+7{E$}w$X#7PU@>3+N2qD?V z>H8YRpEBT2X}cygpvf3Vfw`?Hxi&ugCHxjq&-}!2Od(U2Vkvi2<$@Fm4eKRI+E>Xl zu3G_bIK>ar@NEa={KbzF%9nY`wm2+Am7=LQ^;WaxVbwboTc>zC+%!V5<7e2-srg9%!Du%_a>pr@S>04 z{n4r043pM}TOgu1S3P~Y#Hc2SkHxBRl=@{zdxpUVD#{CCr~Dx5Kgaq?0Oc*7l4CyV zW^Q075_mAR^Df!BnV|sZ?X%|jqjxjheJyW@(1h=K*RI^_DCfq>RD2o0@{o~HxQLoa zgD8pS(Ah4j3vK0L2f1%U+MMawNi^|FSA9#`GJ(TAG2Qe=#YY*4M5n#%V(RANv#&HU zQ3N7mQm^MRJaWb4Lrmnl=N;~&1YOU+>KTmn41907vOP1Xl1;y8R@VQqu^i{K_Bhq5 zUeI~rqO)83iR+KMw$Aq)fAXCEd0sLzurOph7>px45&SN9gO&E_LibvIp~o_R>NvH= z(OIf{(?H9Uk7$9xaOU~1KGdehREffoN!&rIv?tc#&1M(EE~tPT_KJaptX9ad3ZME| zY&M-D{9py*#k;9~8`O*q$LA72jWuAg6ht}k?W;zTB6{!_@i5W0sFSAnOWExTN_l0m zyq8}&jy*C3BHHwxk})l|^%j1)D;SRtANqmYHb!3<*kGc9z$pkETdIPyu6!)m1!s^V zksnd${RI~Zb*S0t4(71?lQ4La+y3;Sc}Nh;nPE5Lf&jhSBJ5sufh+F3l~|8S*Kx=& z8YXprEg)FhaZU?#|7elHJAFO7y?#$hRunKP@a0GuQ+;?y*h|D zrpD+_eN0BsM}D8@D=|b>WdWlB$4QW32n*IrxSCFv$KtGO5QOdj%owe)#eLzE{%G(i zr&0+06Aa8vLPovRqwc+W5h+u8JIeJm@d@S%O8e1|bp`Uec{koF$ey(ydw;1} zS2{>K>mkSAPV>jr@E6*cC9kocUzzY~rkd$T*~@aL1YPXe+$FxK>9fKQzCz!`TW`~P z`J`5!NVCw%mPp~CO|JkaM0}<`H2xTlo*?q+d7JlGUt;yKbo8jrOL4_K{B5>8xC9LZEv$0JLYj5bvP&I zFB}?#dlAv3wTtelbr-Teu5&>`hsqWAX@ROzNIqn7TUp$qS%TZ!cKMIVxWoRL0XJIt zHtr^t_Fr$fuhW@@nY|TPObeUyOOjZ7W2#Ec*l)ENPtItpyl_$x;XV6G+gnz2*6Q)I zeJ$7^_EIO;p87!(0hMq>FGE86;>Omne8`JbL?>stmMwIrbU9#SC$ot6jX()ew&9OUwe1o6NpP`q0Qg|ljhrGtrFo!6FCXoP40l`+Ggg7kn-(QMwtFqSj=y!B%d3TWP@sJ+(kM*CA?(+} zZJRP7+S=P_vMup_aiI>!=*=faN22Wfv&}{c6Qh)i95;d>LX4dH%gK6M^gpG-U!W*h zui)yJsM%r(D?b20WgFza^v6a2{CYEv1NEfdk&iB9d-k}iZ^!J$q{zkcH&=vXfd_jF zT#V-)n{4^m%lMvU+(zXef{8E%R5Nu8HAnh8@pr=OCpsf;%`@6FjSUR)q9x+}ff~VA6=1`1iisdh zN8ZW2)_%R~{XQbAL@BRh1#}Q|Q)5iGrm-SnU=^{H5juM%VpI0BBd){s?WKDO-5_V|` zGGZ)ocq>eoMAGXZ^04$nV^u?qhs9Am}}4^Qkg_1eUnO9ZEvUzi}m&| znXhR#Ivh*1R9V8pB(!MGvbL0xMwO24#m8)LND=I&_{y-TRkWZLwfH@dGu^#<%Yvzz zcw(eAC6W|9G=HgBn^diIq`aHe5?|V_@Q8c92|Nu_!t89_aFQEKp`pBuAGjFP58B*C zdRMbe>*>lw^KzJ#l23D#@_}Xe6A$RSKY7F*I3*@o8qUl&0?bAAcFyY=UQfb$K+Xu9Y4pR2 z5hG>N2zL)*rEj2!GzlXT{`hb-$rO8_`Fg{2)|8P4c|t!Ge*=SO-}l!e!iO`ML)({4 zHO&2##XuWkR?wRE2bBm(di@yizV1A%1 z)o{!-E5lkJe1f;=*_Ff{+05Kbu5Vxi*l{96J4;jfQ`C%VpA0^EdC+AjT7h&xG<(ltO)9PMScSa9(= zpG~A&%e797dfBtSKbAu4EI;SRE6BqL&H?fUNg$;;+ z)^Oo_F(IKW@=31;6q9@w!=w5rPHCfI*cJ!Ie`YX>BRC0J4!T5W^K2QJ15?JB(L>#f z(k<>*Dg2E36{J)3$Yt?C4a))YiXgg?Hi9xe7Fms&fj`l8u@m>)CdB=MA2V=rZGIV< zQ~b(EMv}|-hoF>Q91;sIylsE3K#b0-8zg+jc_#qLX)Js1z3J;9_)Iy%=fsB0OB&-_ zWrJD=Z#O5+5Aq1v<%lChEy0=|c6tHN{<}9wsV`T*r|{CW|$S@0Cc zWS+(rCI7R-{fOsFq}z|&*76eu(cQO7=;6MHYiK4dLv3fGuU|*dVQ(JI6UMswGRXWx z6zS&YKx#Q%u$T43v7Kp@;9nDgOU%kq(=7L3R+Lh9^=g%{ zk9$R7?+YAnoo*+`WizWN*1Y|C#xPhkj$Tv77E6DyZr}>=)5%os%DD@N$(H$jvs*`u z-dw#DLat`rv1RaBoeZhCZR5g$z1e0nYQBGO_?CC!o9O#R_;%-JrON!4#BH|{sa-={ zkJ(|b%`LbEb@iB^+KqUiyv^}VPGs^^q>Y^1Ofy#`?yNa4Wdy?kn~?}>-t0AYx-3UAsYgk7SdUbfDoD7 zVjkUL@MOqrX2vYFgp89u;OCxN7JUk9>R(O{U_yc*SV-Z)k6Eq7nLuA^m zvevxspQ+~Mqzp5M2RSg6mfPm>${ZVzO$+ejo2hk^XFeVD)IQuocV^d>Wvz*inCI5! zHe3e;~V&(GP)_>=1Nx zQ$L=&{rr?s+HGa369=|_Wcp82liRTSQVYdmz#7EC<BJR}e{8ORsjHy zZ@iRuAH6G)pSdi+gwNiH{Odos<4;-2(WAvh)p5zUzn7>F1gOOHmDfdES2yX6nZo z-xCKM&LY=3!hYYbEF16`x_>2kA9`bBmaZ58*7nR$w}4<0zs1MyD$L_`NZ(t<8r=?Q zd~pQO$#Aq!rz8xCuVI7dAvN}eu}~%Jd7j##7cW8~RcXlWw=n&V(>ADs&->m#at9Jl z1Cp^7IlXG*esVSOklO+6xbUyXds&6T!D;Pmvf6s}g7ZPe-{6L%a~};>ZkNtc>5}R- z7LdR!v?2RbK|w7gt;p2dy(+i0V}_0+ByCuB3O-Lj|} z?KEXm;^GxVx2N{r4@PFp#uD%nE^%vih;NT(eSPsnLr69lw1P{ciTd|?H){p0KOc#u za)qi^u59B4`bUK;s83LaXB7>V%1+X;gt2YYJ!Sdv!Ucte>?o>R7e9@D_t~%Fv8jM^Eccy~IEU+9mI%5=G_Pgce?h%|1T z7tN=sAR}KMdxL51+fDuoABVW%SGQxFtXbpW7BZ{;Ujrcc++fx2R+aM}BNO0Jv;B_N zFc}8=9vvQ)iT7op|0E@LF7ZwK5s^eoD(=ZRXZ^CBAffNR3w*&Y@(eES>*tY>th5!y z%T&6mf%z_f1SJDKH?(@_bkL4W08EqzR12T3xK>wbYOi$1;J7Dj-wvaIu#FI zu->RWwV{}4Q#N0yVT;;Y70f7TUou~~r^6{0(2p{@V>tTNtg4=>aS^Fkm zZ)Nb!(&dDH{W;;#&7V6;1nYqW*7>e3vf12+1vudtjYH^X|6R^to~rCGzkp1Raeqy= z*;6*_x3$sd4%FmDesJ`;?#0KTB2!Bz>@Ik|kdEQ*(X{U6ae4}1T@b}MV&5J+R6xyI zIoEj%UGGmzUnljAj?8v9sdD|2Gk}oil>r~k0h!bEg3Dp&H+*jz=&-}ROq5>Ge{Njx zknt5DC8^W|Qk`t&d@bU8YV(ss@ZCP66bbrkGIhG|7Ikvm3%WI?{|+tLiE_W=0{`c! zF~FePP#h_vzSQYW6TG`&W{}iu&UtLdcdkU;b7!}^ztDgeLrw!S%yC0GJ&WYKo>azB zMp6MTpBds?{PNho(mnQl2wNp}_gLkJKT|>3U}g}BE_=whhGPd3RESY?I6|=;LMsLl z_Iv4gkVb9$K*;0GDL2vEK`!!JbUHe*>6qsC-F<}!!-(suwA0GHxI%J0X_PVmi#>nO zE;-T?F3=-dn|Jp9xriT{3({}cGCjavithGJ4P9|PFmcu6LqLU^%ojSiFMpzLqnU%& z*@$jUjtdqLe%1W`g&JBpKVQa)JegGbYoiOYGRH~HOr)h;AMwns`Gj?gR@#V~!>t#y z*@*O)$2?L|F#__-88mq3MXy-Gx?7W``UedB-njn+J^c#r?(&3wRJ3KtCAc!;viY@UOVi5jX48R4f$?YhVwIAn;MCUcR$S}tNec=cpzlf!aH&W(0c1~{LQN~gA z)7o(|33p90%)W;x_yo51R$O7vyD30>py74y?$kWT8eL_8LY$7{&z-jG7KOxf_XUn7 z4zTYd--rDSLNui)#vfvUT#b2^MSqz=0>()3b3g^t+*5<;kA@r#HxS|9-VP-V|FxQn zp{^}*#kU+kB6kMb5ehtnYx@cYo;lRsxxDl+b0E{GhBb(jvvuf%k)MN=)<15Los*O2 zilApZx@-4)e1R#YP#u)_>9QE;${hDfFjuCev2lqO3<}JHZ&5Z!Q6z%t zXYUU-Yev|}z=te_{xu;GT6>|M%(vAiCj3 zS0vn}07Wkrp+0>iIcG1L266Pcwr;oi@po00?RYAFOKILo@4dh@^W=f5 zJO|}%ZOp9x;~!P1&G~C1*6eQ(&F4~l^YNrm#H}CP0x5Xj!8ezh&mIm^snuGR{m56< zr*V3+H}aX0Y0f`YRyG5oHVZBGu|wv{0I9Z}w*v{1kQ1|j(BOTUI82g;WXig?V7Wnl z)6&aREi7im80sAIh1I9+(;DZUgf6Egw0|vmG?0%W@vlHmF%b4KCNyOzT}2=S^AJy} zx91X{eLhNpcl^j+JM6jQXH=mZ>hc-Obj=VEc;0O24NhwT0Q#ypOog&pC*BI1U~W&m zdr0S@Ry)9d3u1v3u7pMAym1kK!&!5iD2iV)Q1vbKW2!c8o_}1{k8{cJ*k!;pjvm+4 z#fNwB(8H<4Hw&bBou(|qowHtk$7%$q;{8Oec8okddd4>7$flpN8RT-PuJl{wE0o=o zyTJbajO87at|%p&{HzE(K7bvqW}nrs+beCT!=4yARx;$QTU9H)73|vazOgh)l?3%Q_1g;upx4&l-)>MSSp` zmdB!Z=W~W<*Pf+*Jw*|KlG-s_WDfo5(7eMNvZ0dk@f*@Y^nOOD zz=Tz)P|*orr(EN)oplgM1r9Jq9W;{+ghDk1gew50bykeA^O^@Gm(F(@kQn*)fFJpv zmKkFG^1AbmA$snlvMu|b%5A)0KFIk%(Iu0&j;PzpZP~`zjT={MxJII#j~Kr5>m+zx zCk_9~J5_o3(q#E54zIJZViJhZ?q3&kPf^OG)6s}AT#_l!Y(Dp0x>)i+&Cm(tlhZYqDIN>>`IR|a-s5G z8>RomaKjYG*0_(aVjE3=Q73@L-ulN@uKXHKV%oOX|1p<-9a z@4#d)S8y3dk(4j*77xR)AwroQq<*K65bOt9FGSEmVpvv*z9y#=uOrxAyED==anYZa z!P2qLwJv9%$m!Iq5mR4PhLl<4;#e|+%>Yh>E2Fs72IIRy_SKiKB2_s%uAdcPqC za#}28^?wzaw=IzQ&y#b@?^2VX)5f%t5}}!dsoWgATtR+8#Gp%&!))Z`D{LJOt17jH;ykgaG<287p|?X1mRA-W)f zTW(HO8|f7{v}v;1BYvmW9t^Vm0A_auh*?zTPt7)ptJmAO^8onczD* z)K`t}Uy=zWy*ANlW^lP`+HWj66LZ36B16*UPr8eaq9GK@Knh>Ra_!yFH+!j%yc}+G z*jsM8iluL?e}+LnGj^*0$Ygxx=g+zmlrPNt9KN){WkdkZwbrz{Q6+QcTiL`UC09(v?@Q%yrI1Gk*8xN7`jxWpsnjVbE?FQ zGW}!y30?Z(;Z-M`6f6bt`#+_NmjYmE$!7?w98in8geh<(i;Ez!SZM47j&i7zY37Rz zMTzl_vr}^Y=7}%q+Ft&XWvk4Vq+^AZEh889n+CJXJ-OhN6?m@t(5#<0vhU%0Z1S07 zP8a>ZfTTqE-P|+cY={5zji)Izbj=Wf?|ydkEc;U^nK(EsJ}Ep1*oc$XfStOy01Sv9 zL4G%-DvH8MJm$-;*mlT+7ILMX(1B(4s^ z$PPl2VIkmZbCk9z+v%DF=oqj^<4F4giA6a%f>UHq;eP%}!Q430_z$NArr0U{3$ z!e3EEJ6e_-qM@XJrUvBzde3vy*_7AX9k7sWiazfQOiFZ99)tURrGqs=ro0R4nVMKG z<(sN?P@)wGsddk0+XY${q=rC@2-#P3I?VaP6-Iy*VqjJ!(28I}x!XAdy<(itS7v&C z$^=MtYp}a=4Y$ChQF7NA$n`&h_#H7i8TYVoCL2S{D7qK}Wq)LYfH5KO z6eM-)M0;Qy9E8k@pzIxL{_hACMTru>i%PC-p?$)TXY-y5wZioh$dw#QUS)Yqp2cj8 zWHr8LA*Pz*A%gh=bf8)fOA$-z2U=idp5$)Nc_=qrSThCC4@&EhCkC}>uzkP4@2oK^vCfcGv1ri_WAqBN z9+pAy{TC-%r6A%!8AR|O1Wd8Dg+qeHH`YOhBY3cXXsu9v2lSoiEqAu%auEGZLtuW> zV-b{bfN_+yR~@9xa5TAG)BI;KC@!ZjPDD}p9II+34d=Z6H^>R2q@r2}yb)1~+SsAh zR-AipS=l=LId;mvsBAFHt<_(@V{nAlc?tvG zQe>jBnY;jbLl5S75;Hi8>EwGJu$~Fw?`zRm`))h+&c_M-(R}G%2Dd$at$>mEnQ`6| zGJFOT7w-s355}H_J5p)aSN-tfoud`{_M}TmuJS*1mmLvGcQa-Sp6c$(qD_5gKdNXd zKF_#McQc@c#Mb4)J=j>E;7#L~;>xrV8{Za)X> zToR3le!GBTDJpFF6d>N11BF7oGk`xcK^&tTzmuGCkCgqrOXkrw{SgQ1IE^>1K(v&h zL=%}Wnt2zry}t3uzX;t77ywD+(%6A94j#h^6J9s13?)_xuq-)23tDWd!+Y3ZYG(xy zXDyhDLKpTXQ;S3Tf2BGD-x&u%!I``O<|kSq*9GF~sjoKVAtdwUUsh0FD7u^3^!#l8 zC#*fRkd;BBAnRBJugm<6u}u=ebI9+nyT3~#d%FsTaz@l-w*qZa{8(IvbB?35z8diERgus#b6^ae&E54u1net_+u2A+pxu(Qs zCpeR=05wbhLsQ1GC&v!eN^=9>8P!D4wl?>`Ue z^TE~U&_>ZAQ>W-J6Fi1iHrW-G^w?k1z3doelfo;rBU7C>gfmbwC*)G7_+MP2Xf}q+ ztDUA7r99)-qggt(qBA&iwo6>=)_-bfnywU;!QsvPB;Dcu3683QasPp~WAfU+@>O@P zQ~G{=RlQkuMtthVF^kX7Qkfc}e2=|8mV1}dM+cm{2O+6m!B*)Pfn@?(@uOl11~cxq zRdZ9rlEgHQCmSCbJdA`xlB9FM5~UMI89ul*1q#1 z{%(@u zSYz{EO%~~&nc*%s3W+xZCcYcjQ|ay-QomNP67OI1x-zPG2d?@md_oRCKTFTjk6ZdD{+;Iy(?=7GF?$y=k zNzV}eL?>S{Bcy!c=o>9ZuF(Cp@Z55!@k65wv%emy0|opUM2=h2B$UPchpBy(s*yU7 zg=2Q*Y(tKXG}o_MeeMZLt0!)ND90w~+E7H}O=VF8;Dzzhwwt*C!>N ztCE=)a()GG4-}~8fTqa2^&Lxl>*kV2lvc=c!!IwLiqhnqP4E68%7AF$($hAz85d2Z z`KLmHg#zYtJwL|Ef0NsBY`K7E=g9JVzuOa7Vau&N{x++_s0Xhd9;V>oqq_XF;{V`2 zwJvb>9{j?DDo=Gx&ZE<0pa9JGotQTf4nMmPdVNp_JifH}3(|z3@a^(2B`vU52jy2% z-8f9jwaSVLB9k`MMx5G5Vh4XwC-!S0c|-w>wezLHygCr;|mG7VHdmrgS)mYL*yi&sIguZfBEdFD&&_Zs*?PU%5&b7i%FQ}CPt=J$#2zr6(-*r@cRQp(|UyWS2c z`WMBD0_i7>j2oXI%t1e>r>cv_d;S?j|HXBfGQ7=##Ls^+>fM*VWW?Qjdc^^X%rzQp z+rYT!DdBb6pB32u?H`lQUss`@$^)cjXCR4FXThm4Ca)UVnLC#7KO^PWuVAACOHRDq zudVIpxZL;J2@PC-9umV$j(=`;hu(Yn+2|MTSz3m)R~JL_Nd zBD{Z3z5n(H=qJXTALGLAqM|$W1_n!~FNdlL|L2^&|NRe^sKXZc1y5676%_1FG|IO1 z;znRH()jkBuJ*mX%Yj1Rq^q2`_h2}ul&Q7ypf_j8aVl z?kT8W%YzsyWNiOxwtsvdrP>}j=6PycaK}FVoC9zoa7szQY))x)iMq&Nc0R*X{^BtD z#nl9wWw{V-uB)cKxvGFOpht`BI8)3wZu4+fxiW}`d;0QWT7(wu3O_RQ4TBJZ=r&Rx zk-V$7_aDrx%wiK;+ixUZ`$vY(IfMP<|EbP*T%{g>jam?uoEHX7VyB&y5S zN^lao3Ea0Fn|3FzZu8D{d9C|GvJCwI7qVlx`ZeSJjS1*9TQnY_qe#A<*X7rjKjIAw zQY~qt59OP0&{z0iZgV+*xihQrZzkOqG>}){sgU8nUY~0)_uNq(mcN|w35sGtGtg5$ z0)a{8kh(=#UuQN_+f-fBbUee84e6U)+7_PO(fj-%KhI6r5$p|tNV^AkW9_3bMrAw{ zjUyMcffl_|Ov_F}XRsFO2zCTGGpXQd9Sjj!Xy^}MOh9M?MHc^z!vbafV1tLFA9&ld zv%JVFW1l?=hB!)_B?pbZ`rwGxa!dLrY(x}jckxMbUvN6y*8BQ0ljpVCQx2F73i;>xFub<%crxcLi&qAG{3^n5a>81Nl>%SQ;a=s$`0t zCIZ$hw$fBqkF^#JXT7^;cj^3YjQa(luRPcY~+0}!opxM zf`KG{Ij)EfMx|0I58-kYwRphO=?pgS4m#wQRC!wuXbGXRJa%=}k#sNI{Yq-DW!q81 zn4`Yft4ngD^xA9XqJhL~M!)+?)5$U!Z5()TDyM@UCC zoUEF8#>mAIk|`CF8KMaz7ssx;vwMYmMpm0`KzXLM(M8ScA2ZkcOOX>Nm^qkU$fGW% zu#?n2X-1c(cDx%fPIqUTH{PnhT;o0St%szB2WA?v3nJYm&8hHf88l9jahCOo&1MA6 zvytI6oBUl8WsGfv8GOM~rfbJFB#VD5A@7FH3Q=~;2&vnLu`uK3sR0P?QCoBLJq&-7 zVOsf0a__mSO};>*ac)s{(|1UDnExGE*N22=gTq_3Rvooc!WTu=@tK$in)YUAU1eH- zEYhM<9vmu>6SHQ4-I`=>o~VQ`V01q`7>if7YniMQXgD zYNb=Mi=Z`{wg6U`=i7O9>a1aBx%25Fta6s_5sc6D<#Ld9eQA!2e=NK<%~|F{j+!$f zOS1Zl?B<{k!tkeZE!oJKm{e&JR=~g7u=njeY12?YKo1sKm|*0Apk3*`GF)CRfRwj< zPP2l26z?9)N4uQC+AdT-`C+5=5kl(GuZH!AM4RG9m`zp0wgB(J9ie`PHn)LppcsLr=YnuTy2H zTOx(29u#t_TuCRBtX5}>QB#v!YLB%gd$R-0L!)V4uZ72T1?SmlV{m+3N-iv&if(+% zw_x>ZI@NGTcZDtC0)xKC5gwggxnzT(f+glczOKWL7tbqXNDsL)n+8rwS>5yO6-35c zT|A`LjJbS`4RV($#gyF_5TI(HCWdn8<%;wOXD`W)7O{!KparI|fv@ zc$<@>I+Z+DUnS%2yVM8vYP|`A8Hs}-k9!CGhLn$0akM6WSus|=8H0>pF3>x+dn|Nx zCJM7 zgZZeim~&>rW^$vu(S64NyB4S3D_KDzp4yU@VsVmLH1l6ooU|op2Fyt-L=}DF`XBbstqX^!4)hh+UYLiN{leS{vr{(H% z9knI>s|zrVGbt+>LFVY(jG5E<`I~72J#S(<30{5o#%t$5d$8FK8NBhFq5N$HOJOPV zt{a|FFIx++#b7vQNV|0c7;>Mjd0$kQlB4RN(Wn;kqRMy|7<7AGk+PPaV}w+#oV+%l z;)>&Isz5esU(UF9RvMlIWwCeH+er|BR;Ya%VK)F(8T z!f%uo#I2LV*$^IQL|SIFN2fVxOEPKutl_ORZ%bwzYv0szRLZw|xEn>YJi7s&1u@?$ z)yz?mV@e(nwF}~?^AYa z3=%=2D)~Dj({tj@dNs$pl0mnJC4fc!gIQpT^0mcutHAr?O@fnQX)9f(imC5es@E!5 zg3NH&v}*0-;5=-s+$x~S_dUV|ohr9Q^v9WD9Qa8iHDt%ij{(=L9A%suepU;%QgKUl z(9*&PH|nB&)f3`=TOpg3^on?5y}ZNor0} zl8vwiURd1Kh{a-l5G4E1^?=E;)kR3{Z?&Bh7oM%tL(w}a`HZQfesOc5|)9hGLK z$KA3mzs?$P^@HE>la}w7)3Xi8`V9#-V&jB$mQPNTkvyTqW4juEnr{%NPsqd8=O~=* zBnSs8B=cJ#lzO#DIP>!dkJRSU1j=Y{NG9X> zGJk%^_X_Lr<$YNa$U*MOy`S7>fhb4Lno zn=KfH6=7_vwQZgzXev7siER_SHQrvi@g~x}g0)YZ{5zE#6)fM^8ffqItvRqE8YGYb zz4wtPc*i+a;xq`=+h7I~gFD|z;4>O?F*!_awcx4FptxW!O^7CWtUIg`(T!CCh$zGcS@3vQ_Q{w`)( zjevo0Aqq90>cHQriSVN7)23?h>Uo>Z2lF8_H=t5!nzIPBlSkX7`cf@#iEmzw;IfH{ zTN&rAIh0uh^eY+9R&tXNu?O|@ry%t=S0RV*9w=2Qq#XHBG+lrj9BE<~|L z)Yr#cD9{kL;U%uoT?6ggup_4l>7Ej>0?~*ny1mx)__&HARPZw{Z0N=rvrd#pR;drc zWLSC4p%U4{&t0{d#>x_Ej^0X>D4!fvMTcdDA~&j*#gvl00wyy>BA(OaTV-N)c5SMh Wgrr$0<^KZz?#L?2WJ^7G_J0668BBrz literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_4.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_4.png new file mode 100644 index 0000000000000000000000000000000000000000..6f4c0ab685c220fa914cfa97c110a06c5ebccd70 GIT binary patch literal 94112 zcmZ_0XIN8h&@HT@qN4C13P=eG0wN`V(n+u(B1i<3PAC!Sy%UOv2uO+a9z?p-(0eB| zrI*lq3m_zr0HJ-G=Xu{B=ey4Nl`EUQ?~u9Yo>{YIO@iL4DBQTla_!u?b2k)Uy-+)M z?!wf$bLTm)Tn4Uiv)ZHqZ{X(7pTAXn{`~G+2iuS4Rwn1pJqUOHF7oPv(XIB=(L2ws zO1%*KB6-QzKkjL0>{n)52R@DG4rrIl791<`FM|7~Z}kr`iVN->R z!VV~+ujZ-ts*!g>}?pQFZ0o> z$jl$lo?f2Gur1@FU?Cn>=Ow(XMg2sm%07=jD(Y{ z_{HCB1M1+gSg(tCsG(sm0SjM~p=l&>4=Qi26srxs*H64QHfccZK_}_ceeSRQ4^ra+ zGI?i2-5Cdq+_6~tIMu>Fxh55<@6wBuHp2sA>a|35JDw; z(^F00bP4Qp=j89!i4CsE==ppwy~2Uq>5g2$%<0KqOI43^!NZTc+;c@t>j?orbK9qV zjz@iNtje_9>9Ko=#6KSmR{%+E)F1wBIX(0_-Fhdv@Ch>hl65K7??aUd?E0?U$%0%2 z3>VX&I>=WYCU>&`Mdom<7v9zf-bH!&K_flNTq%Po-usCS2Z_IgJfFe8otGZoOPo3S zh_fuL6G!<9mbGi$or|q@)X&FIk9f3$kXA(GsJ`pe?`~%jPInoXW*Y{jtlr{3{_QmY zS||}?WMoH*Xfzye+6I&)EeW(aK9jpc<8xTyGr_jr&4%PfyUp7TDDvJwZeSG3PKlVJ z4lUbIe`bEHwBvj4>3?`{LU(^Z7erzwF>e$0e!_hATR0;0DSg$Q@dVTo+Wi%>n;E(L zXUlu9W^Qr?y7!U8A!R#M%g8k=+2vOcM{rmjb8cFgv<1#2R{!gxvB80>!e+eBf12yZi5n+nkIA-er}1HHu4lNwRfG zN)DlE3xq87^+vO-LlAWOtriw}X_n!Z1;-JnOYKiV0f;auo_s2oY4;V2JdfqnNsvov z)i~x;oqL@o2+4Z=7dxMS?k@==?-dv8zaxJxj&`ygh=MLUeNIrAX+Ae*LM)O?5=oFy1SY14Wc zP~XXK)JPgnY$mN*811SzjEDsS=B)bH+Mv`D4v9&Fx=Zm?Xtt16iy=~ftU33zhTtfW_0W-8z`LWEed}bC zwS<8ZW#oZYqYHXh;hSuG^apNbo5BnwD;o7;8uYu!=%ddS*>z?mACY6syWtMVtlu82 zOFJ`C7aT7xU-?ZR$5k|RrAQsiaMSg7%r8$3aRtM$MmFEg`xkb8B4yUH$Nv0xr&S@n zVMV~`zMFGr`^to+?#G6e|DMjE#}#GVUh(+f<%AV}@Lmuo-<$C{z4*sKa4km;@!#jz z=>#@@9aw<6D~ByYjQ!dgxwc-g9CW4Q*P67bUw?D%02_nx!BBZt%g`W7XJl!E1~kVf ztqh9fZpZ__xBXjjbW^2HOxbcnTa#teLP=Z2lI*}vAG^YI2Z^^B<3A9nAAv66!E-ty zW$+;`n0`#fMZ}bzCJF|Lo6Jgp9L$px61p9e9JUvc*v~jEP4we&^EqRIY|G*>kMyoo zZso;b!MVkyxK-EBw}0!wD78H%PiWdX z^e9^iI#gTjkQ5i#y|XWFQkb6gk)KFXN>oMF%8TeupsoA1%nag7znF*m5939@>RR+t z#`V`RMaoEufY{~!ucB+kB6xpBpI4n}Pi{7Rq4q9p)n=H4CHf=3vSb^p~U|nUSpJR5)_~#lr(3j@Jz@ z1Y{$;BPBGXw;0;SBW&-G&8R)LL*MPNhY;DgI9}(ywnsU8Y?LkE zb50Ft5xL7D_bNuWK6nj7T|NruyWG#~G$-+nBVw@0c2oYe?S+1aluoq{0*r6Yl9U+Q z(9T*$cEj`hW(%eEjG)r&xS((ptatF`o}=L0*t{N>#X~e>gd{FBjWO` zy2JLXhk+*0Da5(_T$OImu&*$`@WmzO{n?_}h40}-GnOcSUn<3h$Or-odUVk1qauFw zXDwSvHiY!doTGeO!P^D}qxD_LHX_ysZI-AMLcC{N&+)aLn;CL#zRH^p6L_hsI&Fz^ z34cv4jjo{cEaeY_^`ufiq!Z|LKtYOT0b^~m`$$&o8l@>@?Wfn}V2W08!S=K2QLFsP z43ti-yFE$0M2DH3dOnhp?csjuEoiEvCC(BE@41pAX$FgmrBxNSjTP8x%)?L!-r{sI z5~i}k_YUL`VKT0nxYzomNf5EoZ`=Ayd_=-SB3gVQ+82^2+m@@)t9em@`A z#o|hAb&dm0cONa#&g`5D8Xo?ew3B>Hp*!&~6MkbLWkSd5O@5bzFi{F`=T3r7)D#*7 zfh`ycu$h0-u(%>CqLmN9moCRnW7&p zD9mhM%vg#9%zJ^LJ%W2q2QBNlioJ0B+pQqOBn(1{zof>mQHMD1(?58Y|Ib6hNHIco z>N{^ykoIT%mvKL5wzK@#;{!@7LV&gY$&G>Shy6JwPk{BeL#`gh#YmCa)&(1i0>Flr z=J5zR)=9&1Po2;@lMsmK&(nS0XZenx$tVDHKp30Ucb=gjM?C+T5RJyrhCyGN3JN-I zh=BX*M=FU)X>^w6V+h-a8a>Yi8%WM~05r1yB2Q;UR&p_a$0!`5d3=xC@3Rbt^ygW- z?`*Fy*Dni*QpGneaC8Gv|K>}!;y^A~^UFervphtxb6){e`jsIKa=D;Zk1mc6GJZkg zb}pa3`wR}epMmP1j@lY2DcFoGY@kd>_>HL{$9pFN>eN^Dh?>ptniHH;WSV@7y zAm>-}nOJED>o;R9LLR!3yKmY`82xZDIIR4}k{2TBr{Jz-;Sv&-7h_JBvik_->v>3J z^;gFERUIPL1a^p3oFYc10%vOehb_@Tc}UFvd@eM*)1w0u-+nGB4t+w^-Q-in4g6|a4=6-D1Fs%X zs-St53)WD^rC$yq;avqREybXu{d}mkPS562y_V3Nhtt|b%=qBfcXAF5QF}Os38G?% zOM1UI-AkW5aHz*_OE8K#$Ds>E971SUIcMay$wTnpe+3gQ#_f#xZdl!U<0j^|CZ-B^x%9XBnHcT*bV#sV%S88UQri83lB=59C0srV=aInYFz2T2wo(gmKlNHIA4w?Up&5C&F z4OdqCp3}YAqI2vwW-?Rx?=QH1PGNAhYbhvYLgqjGm(O;daqQ^Hq(bq*-+m1jQ*nv^ zu3O*q#mh{&EnMvU62GO~D(YS&dm+%%N$V??DxS}Hc4?{OQ9~fa`4#d=hw)`%cm!{o z3wQ!e5!V^tuV{yN7G`tB>Y|I8>{U~sWmvRx#&~f7ML@x#-e1)=EWDuuH|QO@EZ)@i zysG|{CG%3BhH{#mn0{`XnXXsV8$6znL!Ab-*6!KtWXjU}KfoM3n3GEzhJ5SJO79b@ zI^DeJF&w#*zGr6hQ!wM+>VK(9Iji z%i?k@3tRxm$WiRJ=-|AI<6pyg2QQ1Kfruj7f7_xqe%Y2)|9s;dwYxW)DV&e_K9Mxw zKPkP;fH*<`HDHMJh=+i{AR4OFzj?kl{f&v*Qs1yKQ95dLN@hxOsWoXukZT4$LkO}K zrW5NgcWXouiYEO!opm(pm-zI>^a|RGDS5mz#haZulKbRB=Jjuwkjdxps|+TpS)T4{ z2i*FUk>)%sul4J)9NM**@e;JRr9!n8Qgth83j!q>62);J4`N7yS=^IRTYxtob3;@! z_3g!2#q>~h2`a~LUFmGDrOBygSr3Au3=R$EZ^q(|hR+d<%eX?FHT^JXk5VrEP$B$o zIe1*GP_SDyrIsynA{X%VvE`&?@o=PoMWF&p+`>EO`9p7|qw=0&r#EZbnRT85hB7b; zmpppmb@5G}iqQe?a(}+DOR!&^uf3aehv22!C3RNUDX*&PZg@r(#ptb72oGIq-=o|4 z#K&dmec?)%zS<3?Wcdx%l*dal4oCAPFDMramzgA|mGUz9kQfm8wPMQm9jPbBzYZ>X zKh(fIg((I=-2!w6P4vv<;$jNm$XN^W_cv{WTK(WfUCd9fbUvgy^CHe7)Y*8!HBT3s z8gMZ)x=~P?+y@*gK9`Tf-w8}h18Q;#^Om*-!gkS>xN{$?tYUZ4lYO>Po ze}RTEUS`V?C+}W8D8@=lGqEEnZAYJlDD-eX4G6i>uGH9nQY%B5Of7YsC6{kMjDV3= z&AHp3nNPRSiug){J95|ZLN5`<6TjVdv0(myz1C$)>eI!`?WZRQeOR>|qk4WpofS!W zad<~l{WzdL2BuE_9+~MmTDZzktClwl_V_ckI*e^Nz)d_c9lnXZ^Fi>s^V(bCvdec7 z#ODPCEH0|Z^0ud$kOMWD^K#oa^zs?lI>hu>lZvzS5hhY_DM76v=E-1cZXPkfS zAIlocP4zHZVL|;ma*=*X!1cs@jNt@HQJ;IJF`_m78QHW8y{&QB=X-2|a-PVhlWOBt z%lFC+eUE}TFW0vc-{oz)j$xjenqOaK@{ZJC;6HrNKA>^fsys$TooSjkA?^h(bF>iH z#SOYT4P&2$D7HpRUvNR41Z!Xh@*n|Ca<3cnb`N10*hpz6j@E=ODR%f^9J}U+SvNt44Bx- z?isE}EX5_TtdmQiKm0zp0|1(kV}PGA%eSGcT1b|TA<*cD1SJ#zP!C1wQzcrKkpYah z9WaD%09c+T_f)Z7atOrfO=zqcoq)+9imYX$@7Pv~g%7=7ZwLgci@ z)m-iIiMA3_7mwU6Ne7aSkP+gLGptteI&Zr4VvuiFFthvozlkS^qU7%B`0PEb)rbou zrVP*ey59f3BuRm{O{FMY;Pz}kn(qgJVjLz!i|W>gu{yA-cT!Gb zans}+*Dm;wCXug;q^>6rNi8ErC*Lr{^`7G~s7j9=LVoR$H}kXVPj-N(atu<0y9TeD zUtp%r&V3B2uUPahJEAy$A%@{*kSQ&_>yELm;%?a7~$Xq=h_>i)=TH2+9 zfT#r_E?39te#NV_it+E@H*F|s$~rAtrq{CAbOwk0w1En^!}^y&m5II7jKj20HwMZS zncZG(ED)?3@Gi-L5MC}Um{eaT%v~b}kQRUJA=Ns}$3wQ8uVSsicY=IgD`)ihf{WfW+A z^o;%T?9Ac|an7{sN)h>ZYxWOb)S$q$Y|89H3>ZuTS zSWwIx!^YXH>jD6$%)1$+pAT7ABGqN`&O78Sz50{e>#%+-YW*r_Ar)jI^}a!}Ta0Br z(O`XVtdy(FnsO!OgH+d&^$V*b{=(bpB7y)AT1}I#7|Mb-AUBq*1EJamzh*|lZQ1uB zh$au9_+SimX>HSbpvfb6ke??X9=`hsA6`Lke~pKt-qN_zu6gv`VXJO$`p9n+;Kk|~ zKbF;3INjqSmBS|R%FzP{Cl{c6ZEZyZj0=T`4cQ};EHTI;iv+Tav|in z(<{Q($M6yH_w%N^D`i1Bl3L&j(>suT6>TfQzYwo9IXvujyIf2eby+?)Qq>0F5KD)f z`|4+omL*!GdhEk?;jXO-0pwD;hDIG&$`4}Wke%_5k?>gNr*uak2n67s*2)6p$e|}{ zFGo2Mbaj+-OLxil%^zAlY-uSFH4pPGR8BxV*qe><%^8c~@QN>H!Yn&NVzQk00!Ezf zY_VWHAk_}+F=-7|5K=HkSsz1u&3tTO?Gb9?ell?oT2?nz|-fzYR!4EJGxxMf~ z!xo#-;~8`~>oPhiQd_B0Y9zT#c=ev(O&G#oXrl}ZB=Oxh2VdCqEb|6!mc&xxioW|U zMZUh@6CNk9RlC!ZI7a-S1zot)`(XW@$3KBG>)O52No&fcueIDYN^1vIWID(7U|*S6 zfey;oOptK-jUUdA`Za&6EeHH-zxw)|wL|#GRz96s9b6jT&0xyy9f;Rpah6ba`A@o> zrUefB{lzwwGph2^`tJC;IY;$2&99S5-u;@*@`<7qG?Eq{qO}to2HE(vZe=PnvB~4FNXj>azZs*lWkcu_?<4wHko2xGB}v?ET?gMNJ@JZj`&nXJfL~UQ)c%$2 zX48J%f#FzjTIph&LG((CA-gca^$}Ygak*SC(Im%jbt7WJ_^x#wv;~KBVDvGM;FvrU zfED1#`68vmAwCWkuV>T5s{%Po48$17V*0e!8iRPh?zh{Xc7hF=8`irP3p1V@)*W}~ zt?H@$7}(su-S|P(cS%req&gU-uj~sAJjb zZf<67JG}LK;o3mq0!j{g@PRcX*yBeRC%9L)@FUDqiOG4bArc0aYSLDY#@=o=!39IZ z!3VUk6qqcbbr)u9fq3oZbI@%tFdya(UfUpeT|?x>h>Z}YT*kw0S{rQ#i9hO*TG19B zr2r)FF2Jvhb7~xJl=RxwOiT%t4NJX;42;)iO+q*0pIb6J*_qMk32Jl~F?qGqy6k*m zKV*`Ec6MW8`iEZ+?U~I1i2s_dVYx=$cIU}G4hHLISglcTHbTVs&*(Ui*h+XW*8rdy z5enXa`B`&^hE3ekybGQedsuq!Z^)IQJJ%gmFQ(BahnVoFyI52TYe|gCLxjkaZKJ}E zBT(3N_163mGzZvJK?vV@+{`PSkIERVACRMa5?ifHZhca>{SVzi6%6~z;y1wT&?cMD zL;1gO7|`)J+H1hoc%7{@W&x;d2Xw|?UzR?} zQ%G@2gi5WYM}8e`I3cyL|FsKn68w$$dI}lcM_c~b5g47671~d?UUn+5Et86>hLHP4 z9sg_|uSmkFNQ06(BOq^?X~OXR0_Q)sBMqq1K)XRZl+kbRd&Pt+0&Z9vcRTcI{g|$u z=10|?VFyFwC+_LtWosy^H}OvN*PYX*<(<}0lp&y8z!?AFuBv4e>T&>r*M_{_P?i^u zsdESa94Ak^b{NnW5g5((ekofomD+`pbZSfc;u`=0{qfoM<2cII6yfyl>HRN_3UX}+ z$Q>C;ZZ}d5dM?)bLsnkwbjR%t#j8&+9V|kk)`Q$e7xvuvj z_GC5q@pz#7(tzG&gZOvX-&H;Cis}xI)6-PcXfDbQ#Z7)S5y5{i_grhrHh{$Bmm^Zt{01VKFUi47DYlaFrM6NmaCjH=TVBZEBdNurKj1^+CPBqFY0)^w%nuR( zKq<|zR7K)PfHL2)p;a4JWwJC9`z_c6Gm?BN6{ec7#^!5%>pagb-e|XC8!p$6C1(WA z#$9^DX`?wvuZ+yAfZjio%k*(fQxml`AE{jyuedR7$iu3++CSJFp1~7K4qwl|+Q{G% z|9|^>uGK9_M7`faQPK6#0Eblz5%*+yC(?d81e054>XoyZzZO)d@U9S#WfJurJonIX8^ZAhzqZ)Z73AH2=?fj$8JIDdjq{Z%V^TE5?Fn5e?h*wrney>q1zgGI8WY#Daqi<)7o8_1{bNSN_qZ zvuM@oMlIayY71898KzpIw}pq8VDR6+1v{7w1P1+&Pd+y8*faF7!}o_h9)C35m2Ui# z{XjE@fPQj&GVmmSUoH4IT=`wCT7G%S=5m zj97?tuBCaoeen2tz-uwKZtrnH>ANb_{nS0hkMZ|w2IsVWFn5x9EaiFpX+*7)!ws()Kw8r6ab9t-S0gFTjXEt>w{~XbmnZi6 z5#rx=<150Vsz*?99fO)?L3CieI^BL!A?dhjAnp1Ax_?VHWFfBBRgCzL5lO*&X7@tx@w(7F?$#w#8(O6Ky0@_}>(I&k(dPd`c zHs1k-7}hXfxC-9I8uu^>%bQpg^yPW%_lyRQC#9p5h$L11Yr=J3I zmZBJ!$|V--x$lHOZ#-8V) z67qU4j$WGnwG|jF#H6H!JpBd|QD@2oAy*5>g3B$LJ&pDHRfxFOh8SBheF&NJaw40% zs7l9n#xjGXt(qgd%(Hl?ao%&3EU{y*ey?t8d$`bBjIr zyDYLMJ3DFUsXL?PKK+)*_5y47c0P=ws83-f0J@txmv8H}zTM+m@8~v5#UbfQ2sThg z(EN4>bR2Wgy{cZjd`o@)wPF;z>8b*+qHa`^>SaK&(tD$A5KS(1@&-9{+{<~>afjv% zCI__3oe)}ppgx!0yMwOY?`U}2u)w|*w|u#aE#y%TNq2@a!lx1D(-W*V(H}(D>oHRD z)!-Pf*}`vd(Di2_PI~Qda1- zxeL>1{lSmr#G>M~V#X_0KkS@OY&cp12aMJ(i{&(!#S*a4gXeUR$D~ntut)zKXs@!p zXm1*srR8uk$1B}Wr*RyGXof|2zZeUPSV;lXbG55Z+fcqB9+H5T+=LqE$?l~U9z?cy z02Tmi!5&5J;k@k^qi}`*dtVkZqpqm0Ua50voVO~`6RR_0o|dU3RZ6XX#sq>(ed3Qb zOPG#r4s7zagD?t7I$m0=G<9T7A)wcMuE1vB zx;&MWoff5Ot5}}zw%(s57<7pp32cI6Wz!9t?~GRk8rk=gLf}W0t8rE`k}9`^l8CKO zmQq`MEZ#==J!kUJA;-N{+|RHfY4@~)h!I=3c{8BP->%pZw0ZD@1IG<(hT6OIGl+Bu z4!VRTzzl=-!V&@_F~;GoY|We~d0Y#KTr|frT8Z5Tprp%wC(K*)=y>MT>l-)WzV@hS ztq#a4s*Tb*qwjojxL8q75^QZ@s^9j1+>Y5l8B95O?74y9P@?JF1AMXNkBv)716~6A ze28Obwb;sFp%(x>TrU(elU#PjURg!%>D{89Iuj82(Lp6cpL$XnsiDHPyBXj&OeiHd zF~TQ<&NF8d=hrKT7zbohYG}vy^prEfRseE(2W<|jY7?{xH+C#UPVvvB9B-JWJiHv@ zcvIqEn(DNRX#$~J=Fc%6$$N?aL^QVQw_Sl_GJn~`$;|*8$|rxR)BY%*ZHMSwJre*g zLm~>L&>mtn+Y3iOL)n9fFSvTNTfYp+Y>D^`@U}uwStbB-U5&O)5=55VI_?w!U{$-A zc^8No%jQD5S3%7YvYsjl&r7Szu{2yy$SpWxVcwielq~eg*COdrIU;d=Jmms=dQ?p} zyJKKs{w8{kx`^`zA-AC-j5^+=(n1rMZr=cI;%&lyUh6q7;ftVeC0jnHhl4!~HAP3% zx!ndQ#1@;HO^xIy=G#dl;un;i1pcWfgxX%W^%zVXDCmns@1~bhEq{0>H&N(;h(B3+ zVAS}MjTzReM<;o6Av2vHe}1%ClHB?U<&q{OJZ}-lhwP9F?|ZJ+;QpXmDsw!5)=n&C z=r-R0J2$_|J_`#HiNbgGkdNkV3h&ZEz7pc$1sR5Nq@mrpr` zGL&_h@zZamje_D=Jj=bF%Z~=tQ_f!2^bF@tqOxSUk7-CUQ)?>b70j1#CZq9fEF`U= zh{{hGyAOMDt;Rc8iF*5l9UJeRL^^;Ma{NjI-~Qb42KZicl?!34fPl|5o%Pc0F#rJO*3Xg4aM`%QEzo-TjDPN_6EaP`Y#h4Cjlt1zIZoSWXOcO6ePp^hih(}_P;E)CV+ z@*JGzBV$nP-bV`@beeVe_W?|`vzN?4MjOR(+?eeQDC`pEPbBrcW8-=1^u$1) zgIJ$CXuukoyp9H^l+gg)DZECDNIP4kabxJU#K*rCcEsPeE$j)D zOCTX{LNM>d5VBi$I)d^gSraX6mzo~BU%p;YL(6fzst^F#n{;#o9A1LWdw*_?@2dCD znLn2)QT1{kJb!NQq?~$I4n!A3&3{f3X_m_{4iUC>NLz`#jOee2^g4kz}bHK*0B{vn4WSzbX>=`yjg!nth zztAKiHdb3%!Qc27S}R0V2DrRUmd*lj*I2JNIhvqq@UYis#PlN8f3K@jrj!Wi3nipp z$qlrVbC0d#p`fk5faJ-xwlVp&LR6Zq=hTDH*pA-KP^P)tKz}8rpZleOqyEA(LleCt zD=c3HsI(n~&_iMh+&BH=zpc2Xl@QkrPOI^e)qyLOaXNRHSaMQ|%DC>2E7qniTtJ%W ztrkp@%PaKVNx24f#9)IXW5+AB(EZtH`=&BcA)*H8phKY`q=tciL50u-=pSW1mPFf& z)QSH6UKm1yiMH>L#V)JJY?g}FTa*z46=aQVPW>FiNr0pb{UO&a#>-1 zh{PR5aIs^`;UvE;aFjH$@&{e+y~yduFSB8j*sKu#g`ba>q+HvR*FP{p_T{D@r)z}- zAK#MO2%#ZI+Yf#)@%&um&~P|qyVRvw??7AuBtfTL0G1#{+mFhUs6@x*m6;P!9z$sw z*il>Yy{jUr`nEoY6AZSVR!6QUEM`Ss?m@t?_F=LdNx4z(D?O(;G) zO+&~&NGV+$RhXI=A~Tgkdm>C7CZWu4i=3EHyAH(tb)FP+CJe|n4p{T^qHfZuxCHqz z+6cr+FEPJ)v~qM&Z06Ax{c~)ZBhS!tNKZ*jhLlZhrxKz*BexwyA?V8tOoF^0ebgam zcUOt{H=%cQyWwn(6b9#&(Z;|a$sS)qX6c&dS#8YZ#cNw-Dqv{q$z8A(iD+`knJL%) zi#cOE{pJVXO=2X(7umPQgv39)82wqTOG2*dGk~!}cXpjtADj#(vBsr07 zZVHBeQdEMqnRDx(@oc~TspfH{tJQ$sbTeP~dZ|Pa%{F2)e;Yqui1a3P8oGy1-q=v9dyEI7eZeX z4*|}>a6n{QGGlLpumT(sJ&$Wobulv2aT)CHE=%V8o-TVqd-V%K(#LJNkItY(q|msM z4*|~(<23`<6F+>YO=ZR=F3cG-)o6LN1$lE8$QX0lfJ#$p6;FQdpgo2PZUf8o%80x#otrsl3|CCVW;ouSF5dfu$KCPR-UHFRTFG1*T9fO~!UH_BJ zk3u#XSNiDSM{68HS_`yl)mMrGjl$1=jWhuS9_1z%<20EA8cvV526O3Z$&F3MA@YP{ z09}-tEx$jOZP?QhFKVM1(*qzlhG>WO=;lFGfS_%r?D!^HYNP#j2nF%C_?jkztwaS z0xJw`T$v@&FlM(hUoTv-q4zG68>8LIlW7&%Xjeo)Tmlf}Q$xi#l!}`U@&l1{!T@8Thy&t0+B@qgmq21(6j*JF z+cq2z9Ja^-m?r|xIOutNAtkAI+t2YzSI>j3b2~#uU#_x=~tct z$9(kFzRKvmQJ&Qg;oWqq z1P1of$jY!n>-z?LE$u=y@20^Da?VwXv&RmMt%hz53I1YR5vW3AGoiMSy)kos3Gfg% zwl$FBhZ;wUq?;V8J~qJrvCN`CX?y#uxn_!6(LQ(B{zWt|6V6Z~QOmg9TLDaC?rqf- zddqEkaM{ah+F5~4!fZZ%C6K8Gx69;)J?H(mq!-%hgLuVGYvo~Ov!4|IAITQM)pw*K z!N3qK0q2_&;bur>V&Z}CD3Q%ID*Wkx$gMIl9Jir*_X)8w=BhRz7P@sM$0Sd4p~AIq zlK5}xDA{bLy_-DqQA*+Qv*Da#2F)+dG~I9JE-w10dJ5!L zInv&~x4p;bFC0ft=xcC@xd7Oz;nP218)vAx^wkHW&=7K;i&Kejlks=`5Ab-y)SjP1 zqfPtv95Vg27oM?dw?6u-b+@>rx0yxPGk>8tw$ADsleB@HcatCMo%;Q!EHgB)6T8oUs0esiRVOV%2-&7^kU&aW#Hg8RRllW{Ew(}emD-uGNQ zJ;K{0y?i0iiVpPwLh{jW0*%D^jmnwTmlChNdW@4UL2D?6L3DDs$An$K@@CVyKU%3m zZZ6$HOfuuU_v9Y8@ergxc#0UJ6FcCivI%4CuQ94u1pRBu<7AosBTWwkzcB4fqZdgJXW6WVG8Ccozw>Waw5NPUg>lxzVbmCv(WX7{KJp-wanOWu~yU!Pf?;05GT6~ar7n?k^W2u zV|Qa}ss7A|kBYdpeQslx8QNgvTFC{~S~UOM|0+>yL?K@GF1q`IE?Ainhx040T;XS5 zpzDxm7`+ij+3#AvS!>f2D}hs8c~1*xGLkL+PJ%erH~g^?3syU_XNt#k&!>osbe6;| zDc$QBt7@~gQPz5OD1Ck9f#}AkHb#*}=bOtGjK5{aF0?<3+yVLAG5AeYuktKU%dX_D zsVdM3={(#r%n9ge!L~gjY47gbx*%uHDsd-CQ;*4v0keSpy03t76*F5cw&vI@@Y@ET46hC&iX+#z+;*-n3M`+}$-(#E09~aud-u4!}TTUfz zA6-)tIlES$$cdbX)zIEyn&0mo{6a-1D?DW;((sQi_IhE8;S9!IM~*J&lJ!#N*FK= zpsE}zLMvwcnXBjJh9w|mD+7LwRU^kwiF(HZs?O0Lh^?g?q_9f`RC=sB_U+R{TB~IN z1?r@3oKcols7->&+YVoZ=Y?;32m7}eO+x9vOXc@oS@deX6iy?-#C7?+U+l#&@7=nW zmjh~XqV9T0+Iov#g=}54HyO@}FXncOl3&dNy7Z&P&%cdXuJetTflx{DQIb_lOVu75 zB8s07A`0KCe|n7vKzjWnVEds4tq<|4;3$vUz3zDwM-;}e6EmyQ&4(4UHRWd%jQ41? zlN?z>5`vC1^D-*l_fpSB=C3uIiJ;m*(HNQBXy(a905 zMq960J|@Jk+pgp7q`EU?TzW|`wC={_(sGdDTi?h487iXWQ%-|~?~d$_?i2E#82q7M z1`OVUY#)W&i{fGpSCtrp3@d9Qnd)NO&&{h^O`mq9Zb-VvJ}}UaY5M+}+$wzj`-%|g z^*5I`dx=mGfm<19SNo=g-X|pnu<$<+31PPH*uDfw(&fIcg)d=MIo&Yp6qkYC@gcJk zIK{F3T2GtqrJ7CNYACwvEScq z8f?gBKwo9YW1zTNFT+`dnKVKCXI~aMc>e51TK#g^b#hZvg^=tb8?gyTJAP*~nA&n< zE`HaK(i&1+K15@H#EOvb)kor_US!qkF*yuaZ1|4ycnI#bUPJD`uer>^YS&%4 z{4-0OyggAl%TTUsziMsp2VMSSKN>~4F4kY5_|ZRKk>}_vS~v>cfjuK1pF<8a=#IX6<>~)Jxmqn8Gd@aT1d$$q;*ON zuu#_>!B{2kLdHARb9Iknaw1pUtL4AdM`jgxl6bNE3I<#pXp?nIItyfAFsa`ZWbvDm zhnVIqvRWiZ9y3kZ?R@3d9}uVT739WUf0E50E4xg`u*5EShaV34h6%J^jo+(;+6_5n z7n-6&Wu`{D*F-n%Z+y^GCdpwE5=N2ZqsTv-K{9f$m&*jocQO!$iSa z?gqhWLJDtadz!QcGC;!jg$jr_0p)PKeOB-GD?rk>sTvb|CLDy2&0@*fw)#775p7=v zi0o1n3#hz)KkX4a-&o23j(n4@$}`8=SZW!O%*JXj0u~y0fI86gi5ezCoQpa!K=G*(bdJD@{#`}Dg0XQ`_7>A= z-j>rXVLJlvYA_8Owi{LC?^*3J$PHh4P2gO$lUK-nV#Cl)BS$>Tm2&@7uhsMI^b22X ziQDE6HCFGH73fw$w#QpzOaMjSC1J6dv>D?*!I>iqKt}SJVn#!5ET5kB!tnmC6$Ecz!7(ys8a?lk^DsNSIjlRfJ!7F!O{8%*>$N16; zoawJSgsvuCeF#2-B}91-_d zjY;OMTuCB+J@p_vkgfms?*GOg|J94Ru=y^_OqJ)7JJb%8M}01CJU9EjP$g?Voe(I& zAwuNjw$(n#bHL_7q1P2L+nKINaOh@eT!LP$7OA_!>Nx3} z-Mt!f7*#Y=9kYEZTGD;Qw=HgSIxYULm9evyKu7DVO^=Mtyl}tyys0ytJ%l2X~W#% z=w?2Yhoe5W=4<~)RDoWAiM3h}P&9`u1x9N&e;N#Xr}svqOB(xEni= zR4&H{`>{#HR-|0P^$Pq|hQCN9SILUJbvK^>)U^~!7k^Kck*85Hp%S?UixE`~mE z!A+MUG6DC#7&jVMIGy=Fk>P@IM4)1Uv5 zL2lw5>tO_dsI&#MRrZABn*)_(#xuTE+_)Q^`T zJD3#Zw1()0w;HlS%BiY-ZEr3s`g@9+sAPo#-_nyK${)l>OMBVb{JT6G^RBR+@i&jz zoho){4wpfm$%Cu<#7yCiG<|$O?Z$VrSU(vNVCDx7jG5_<%Dy|q-DjldqP@PU0N)An z@D2+^k#-7a4^}va`Y%(KA1fH@zWjvgkX$k--l@~?0k4EvuZ>zEtvMSd{HqAAAm?c( zm4t-7*1ewfRlz^0J?psKKdB?!H-lMgcc%aMw*bb~nRaZ~Sh7)Xg&|}BML@_Fn|>&{ z)@!=E6QT%zn+Pvphzxr{+AGb*`1T=qoFVO4p7OG;pSPXa$?x4{P)YJco!U90L7POSdj{wCE)8Qx7T2Z3biUs9X`-N(2)WI+?>3NCN&q#^IG|Dn=%3t41OVK@> z@~Q(mNQpkL6iZz{wVu2C6agjv6!4We4szQOb1x{Q)@-(WhVPB~!_xErvG$%(O>JTK z?@?5af*_&^N+1eKkx+t44WJ?*T|ntQARxWhfT+{}0qI@3NDaOB-b?7ccY)9&>x?e0rg|U?Tfd`VqyKV5e(>67`rojFl+@?d+2cVYQ&nBZkZ!G{bV+D2fy{p-E8 zVaWegt39CZRBqfZ+W#{Y=@?c3_l(Kdib%ub%UB{K7V!+;Rqf5`yz*_pW_`P&`2nT^ z*0>i;^xZrc!ub_Msa)fnz7@tr)tT%i@gA;_@t#w^otn(UK(cDm@U6FqX-K}PI@?*+ zLHS*up>=in<57;&dH8lokh4iZZbnph4UG5lVd>ttJ8T{4%R<(};p>_BAA?mio%%x$ z?}C>)Qh`$3kycI)zyz9tx9j(3aX=|6GO7xfYNC#x@iw*HUFGwK@<*ZIff zDZlkm;(z;;$2D_`tFEkHmi0lBXy&&UChaUF(S}GQyqh07=I4pfun+ED3%iYVp&o9@5lVW4i9a}(8JU)CT|1=g8He3chvNaa(m!Pp2J)=c8_1*kDPEI9Ej&4U zjhDv4E`>G?oh!Ucvp)>T$X4B^+E35?!NOz^jvo3QJ{Ae7Sbeo}#6}7wJvdqOT6&s< z+GTT~3zoyVII?a$_O=B>Gf!FI%XYovGfBs*FsOHMTji~nEm+VwGE2e|!u-5SxMnYY zTSlbMU{_YZ>YILbQ`@=m7q8LyUv?RPkQVv9!oSHQt2YgOhz)esYs3zz&mZA3USl6C z`cWvegvE9?(s|t}_fsVxJPJds>MpfkbFi?_(zvNZ7>HCn&7q$9D)CA_ELNq~yr$dS z;%OCNb9kfb?mN%ft+N+1LVg7NQlOPQdjdN6{-QCU?L z{a8rzS_`3?qn|9`n#ig)Y_8-D%EFf3H|un_oPl*}cSaTmhmV&|ZQ9vB6x{C<^})C; zGMYSmfqX+0-Upkv0cgqcvKTKJ1Y|-#`$K;0DkSVdE4Pi+qm5TnIRVEk8FG>$E;tEy z4dzpwjc109p%Nm#lnBs6ShhQ$xIMlB)oN39Iwy?OPHTFXf=oJ&v}0GTvHNqB9!{e` ztJB-#e8oc!VZmb@Ycm10XEQPrUWQ{N$xMaWj{7n0&)7S1?Q=|TwnA*XtR}5nZ#n|E zoR@QvE;Ptw-%3?nZWAU-H!*ra)D$<%R}vZ-p(kxce=+|erFWX$vAj66jBVv5ym`AH zf|KAMSE}WO06KT6lFsJ^@DU_c=X4&tHpyhsR+puO;skVAA6`IYZ$xYFC(j|FzelZr z|3GN!4pFw9cFSFlJvccybp(dlKI8j+DpwW<9A`1dA^+k6)Y;DFQ{^xFyjajVQH1b@ zp3ah9$5^aCv|Llk;{hxdJ~!oEf&3^itlZZ~i+vzAZ9B)UBJNJO6#C+^+?tn}8S;z8 znZgHO;0Zg6jY=5%*^r{p8+(cW%%?6YRhKL0-hL$x#^xGwE#u1#`JEh<1cR` zTK3E$6Yo6^+vv_zG+iRz^TJQwO<&i#vH47`*F~97M0?suAqZo)>)ZVnS!LiolxJavx~4au*E?!(ly5Tn@$~Lfg*0bMxTZ z;~hW~WnllNMRS{Ez4;1d<=b)hiHT{4G63E4R1=tu^hD$m1#i{d&B0&nf~l|?;yq_Z zR1P4n>qioZ|BV0Kb>_I>mL)QsCnvy_$nFm6epC+p}VftaUa?d5+_3AkEf)2_k=cS-^Ljt^0|Y$P^;yY2Zr zE&!{HP<&)uBC*^5ZKi7P*X2`Sbx|t#0;P1DB_Ud(DBD|zf^PT9vz47r?Zbki)$;}D zMb!8g&MJpjE<_W+@~3#mX@2%HK+9dBAQ}&-&oe~MObXgh+U11ds5VjhZx+{#8TYR0 zY<}v^z|U3#`Y)aD%$9{?rQKdr#?a`Pw@pf0Rm17au6PleM#t(6EuSpSr6-LGfB~vo zo>BE}Ej~O?rgEtZzM2lbSW7%NDQkMn0iOhZgEV@F=-|rSYWhnn6|lDLtuX~4?0qlU zf@7%ys0q}r2>v?Dmb<)9c1%QgEp`A8GxTv{!s#M2rDJ)g)@=|1sVgV=;mYboX5)$Z z4c^Dh4N6gl9i=L+o&JuNFcqZb^U;X2=^C|&>IhELhit``B;YwkeRvU&7kJ$75*@JX z=fwpUudxLDBZ>S%A6JIHak|ZsWp-WuvaC=9aD+6)e~n zpDr*L$Cx#Jci@&W3$?6-d%I39&^LAiO1Pb8Wt$ksx8p0Ho4bjgu{mG`Hmcw%WNV(k zE{v&}*-JPg|6^JSdIYur(Ev$;F&<8swDt6T1OcNZ4Z1SvmS5)Ies0(oe%r1AH{Gfq zb3)mPkN+s^$%o&XVQVUK(lwD>P1vv>7?cBcH&Qho5W?iq#VWOgp+=0y=>pNpXE~l@ z=o5PNHvA!daTA|`UYF_zpYW@9DpwwX)L}5S$>i7(dR3XisMjK77NEFIiUnG!)=abg9JQW|GuB57r{8d!*XzbH$K^;8`fz#)6!WQ!Vv&2C8Mm zekak38P&qD5Bvf5n5Si|P5^&OcvG;Z0}rr~IGb-(5=~`;4K%2I_VdBsRbBcQlxWL* zoFQD6+^N~?=I4>>U-qmKv%7)2ub{uz$&fJ_w2xXu&^Gz5Tg57b4?MP`fEjInZp+fc zwE~+FxDU-Mzco^raFE>?zwNd{)Nx050I7xJ04}(;x^j>NkhY!|)6$#k_gDUsryG{p3q7kjP=Sv!>U5D&Jlqrg&u!yOva$kiN&a>O?afd z=Xw6q7X2@lbc1~K3$}BMwLf;dsPFt z9+@GSNwrgKs1r^>BJC;#T^SL<;&oMEW0DG|yptXMOvF)TyjY$_N*Emx0g$SxPj!{^n8UnDwIH75OKtA`X&%yt_!? zk>nqj0Y1z`kSy%g=-s8%B&`IRENekZ2i=a1LOevp$)9>Oz-!f3T8LOV_PqO6MKkeXJSf# z^ijen{8-VpX5YT!>UlxdMP9Z(y9<%3d=jcVv8v>if4c)NtKj^i*mK+ZKnhUOHO7^FV1dtgVlG&RxqZ(xIltfcTxAHkg{bt#t{fiKL zRVZv9SftpyeF9kZduSMXRvRdT__xZabTyw++;fCjTnk>A6l+oaO4(}=&F{lcB80ag zM+Q2-6nqY`X-O$#FZOXiRg~|hx4p_!H^hb4*czm_{@8rOwFu*^TwSahzUf&^F93vS zhxNtchek>AILy?3-6NYd@#ti3y}ZRZM!)xV^9Xpq1oA;-iM}sc)5lvAYC1(p+if}% zXpBuZ&0jbtSOtZ)7SS^yPv;B_9sSVEU{j8U$g0K1KByKtw)<@s5Wz&vGH3HEwQ5%i zJ3I8R-ouzmA|eb|(9Hq}5EORm6cBI@n2S!<7&>Z2(y7a4tsesF)k1Q2av~W5^LZDe zWe3_Ie@2h7t@qaP$6}}M-+>y7kdbocj{`GOB$U+*IT{WtK7@6zi4KCY&y*k^=9KC6XR(|fB%9okI2;B z=B{~XNR%}5(U3WKW%hoiM_j*G(I?8lv{}-a!g*}XyuwWOTEFKqWp$HxO**}s+atNS zRvc2ZE@Rt-q-OWor>4vCbS|Lm)p-cB_og>BsZ-G!nOb7D01HGb`meJd{69u>45qF*MH~%bA4hT=i$aZ>e`!%KO%q``WUhP{7!;br^ z-aZKt9EOSq-$fH==fF{Iyp&f5kw6R_=aZ12Di&9h{11K+hy5aYi84J};olR}o$ z6m+&quJzDgnoI5wizU067a2Udws0oH@pF?8+t~Cf7R`?NF=oGbS!P~KZ3kB8tm&~B zBEclQJ%9U!`>j^WvmH+|Rt!#U-#X+=YXgWjRYHW&ZvwtvD>?9Pw8O)f8qEN@ihcyL z+MF~u(U%B`=L6kuqC~-}#;zJz75M67>l)&`1oL?6R_L4wfmf*{WMB_e*r90+@SsQ-&(ELF z^zOZ>4uRG8QFf~V$MV;~=|NK*%`$1;unC#~D&X_%+w(-%9$|4A&-Fggm+L^8>6&*NGl0)_^o|&y(4$DUg{p;JRbQJ?bjno^lC6psoea zMA*K&(RFGREA|O~mlBd>#?Z@Woe)fU7rfC)Y?4ct{gCH>w!T`wXb#;l05@6JBXM2V%eG+#?T0$J24J(8ns_cZpmfEOx0UiU+tBrdgZi*Prt3ij;h*v+Vwg0zE#yy$utx_I(`6x*G~U z3-zHGzrn+_^wzHY@AR(vSDvl2RlOY?^xI9gjSCEZjvv%y25Ot)y%MWk{vQ#m5xz1nicY zu|8#WfS!XWrNHXl6N@4TnevaJXI&4`@@{&(QL1~|yen*Ib%zCh3Fr4-u^;n)2MI2~_{!l6@fHr2 z&W}Hf39T^kUX9Cx$?^NBjpALdk}2!InNS-gW(H;Cm1vA6U)v_^HNM_VK{J1dYpl9$ z6S#|V+}}Cp-?rHMg^;Hsu?CN1x^NnGJu=3$(kjBRed-zN_3nWO;dapTP!&Mnt%Z=x+uzjOLB9-zSV2r^p> z^|q%vEd~<|mOa~iRNpUzjO`(m_~?s8>A+wblibFF0~f2#y2bM~AGgkN|MR6NU%+X^ zVhRK=jH*mslIafGJI^}oRY4N`{{V8-a^(Nz5sbOB+9(2Ajm7JIq)iopmqj)28Bf%d zoX4kDhM=qTWDhuoProIiVPL5C%Q-lr(xZy0!OK_duS%&+2kg@}*AyHQUiaz?noe*P z)}Ym8wfjM#H`xb$7tBxp8=s?hsz2BEJ5lrXn52=B9~pmMgcbIKO1~Ekc)&cGrU(K9 zB|ReLMl<6FX!d*y-KB#l%s)AX$kWZYTF~)6hD9AGv7!~Gu^CY3=C8C27T&5fSen;z z7kxgfPxVORcle}g8cDwd-~cs7C4cxi<<&1*K@lAL`QP`S`UJprLapiz18Bgb{*r=i zPE+2o2n*%D*7-nM=@y=~n+ud1s&xfz$d^6Nb+1Cbo}-`Z_ps44QKJGREkGx8k1C6* zR0ra(R{t*o0qbXOZ8A7o6u zB0^@;!oGJ|zVppKKVj|ZbPjVw5vFD~-K$Aem*f)ENhO|^(t^inDlk@dp+4~HLg1*? z;HAVG{gD_15`7YsCt#h@BzxQCO_vI6cuZ_X+|OxO03r~*@}|k?U)Lc(W<8|ho8mJ` z{P_HD^cS+5l`nt=q3PWTSYik>)UdYhSW?4bsa{t!L1-XY{Q+ib4 zt=Y>R%?PH0Hz{U0d;Ltj;Nvz6ZL`9tvO#uE0y75v96eKY5iY8hz4K$rE5;=lTehqq zNG8E?yo#i1YQWG!K#Uv>KU(#@;>{ekW(!Th)b9Cz>sdDkhPnG#jwVuCUJx*UAkY{C3e-AE|Xs#!fo`@%Jb_rG7Z1-4;b&phJ6(|MsrYn zv%kEUM^Gon3Fv2P%pbL-OtA{Bqo3s^{=Nnxm(V=^$%y%K3JT<5)CueOJ;aOvy6Z38@Sq^b|HHitm*3Q-CRcEj3uyxn|j+L&8Xhv-C1tPN? zr$w3fY6Swk4Hvr#NA;kEAHJj^W4*RFr@xuSyr zY(G?=DaFL;qn$p>zVc{Nbeyry2s|no8~TtDECv(wz)7fNQ*SYOO&nv)zqTMO+C!If z#j8$yL%^)g6!Pl&?F<6)O^F%QX|K(YiXw;?{I_qAQB%0J^0BPQJ+8S;MRZs zLmKujpvo7$a8FpZCBpgup*zWd+!VB!OA5u(ZiHE_imU8jqD`P2Kb-XEfgF`61k@|YliUA4qf4)c9d`hZ$oe~<@#J5C?E1+YwM@k3uI&Xn_ZucW}{+p~NG7K^2` zlV_B+6fuOM@C`qAUxm{1S@ij*5#~i_eTdUyLAl#F`V)#h&1+G1@t#URVjG5ULn=&rO&8!v~6cQ$A$gD(~O|LLl37BTfLY(^haKm z0eHGZu7&C&eIiVHY3cmQzjBcunULXvY(m+R8|bkv9~Su;8tRbR*4Fy&+kving}%;+ zUAxa`z@~6q6zT;r4?3+1S}Lpi1_npAu;ZodrOGDVaY^jP+K~lSzJs|%k3zxbnQZtb zR<^Y(DViUJjC18>dPvbS<%TSo27;|$3f*x!KWVW_HughStG7j`!+xtY_=@o&>uukk z%gD!4ea{mz){7%uJnC^B4!;jS_@Pt&taZt0344C0MF2phygkt1jc682;M;go2n*yp zUT17f9jB`Q4mQc~ zqLP56aPCO9T<7O874}4)j$Wg8>?W$I{-0=XAW9MjO++qtiFKjOc*2(z7K@#GSKrD5 zZ@2z&O_QopZP6pe?_)(XCy?^AL_kMpT&@Q`X{QRL7s}U;E{kf_UQ>VMO<4aHsl){- zHoeWA=h%Kvrf`E$KOl%9qLQL`oTSP3Cm=Rf-kJ532E@H97(w@w=F$pb)o}^#nHhFD1zJiYb($)$NPJPAfqeFE zE`Y{+x!Y~8Da%|I!(k=|5MYdRjlNWlQKi0eeh!ed{>IU~K>`aWiw>o8?YkEqm7{5i zyVVCN6mH@FsX-ZhS0Rnc)(wK z6t+5Xh0224SY$h5J&taLdqrwRZCPj$ANbSoR9AwV5Q+q3a}2+02feMx!US{h3|n$&T-@ZXvOk<5|B?3)&QgG+Zf?fg-|AfZ|MaBNC3MVkhXL&&plaG{fm_8kK0fSs3g+E)S3&w|weZe=qjy|weZx9!n zU&u&M`NgH{seR2%0-p9 zF;EJSBvgck7n(jedU|wg6P_tI^`7uwo_GRK*{j>TdgINP2F)DHsjO-WJknGZN>-~f z5`GLKaF66}B)Z9Tq(1+Ls>QA~uTEpguC;EW7vOl4dh3TgJ?`DN%%}P`u5D6$PWl>A z-=>V^rv@T&#kMjv5{p>>H-%v8!WKdc2YgW(#eX|-57QgQ5049$8vPD>Q59@9!ya9o z_>aNA#P7vfq}fDE;@J9saxoE0|Hm%{@f_5=vMhE4VD(q7{g3kh-;ZUN{{773jhbt; zTlI7OLnDF9SkCrPhCsZgMo{dx5j!!V@yHCh&K#({s!9d9YdOVPpZLDzJrM>4%c7LP z=zk$zrs5BJJbEM*Pao6cO_)Z^aJw!^};`fhGP?k+F_(# z^r14X658XdIUc$318pG7b;FwF=5Zp7Rd*J@k1Lh@hw zWK^9>_gs&@W2KnS@u0``lq=2!%dwTeiP*4zU1MHMp38HAuAgCb)$Jcba$0))qOM3^ zw%^hVP0cIkSol=sU7x5r2>yFooi#nG-W6~7Kq)xM-NDfCJM8D&QE5$R7a?GEVh261l>QJwC&yB(jxT? z+7o)`#IIAn|CL&f=Hr4G`30(-T2Wh$pZ^Vcn|gz#)#}GIA~I-=|NM9%D_`5Sss5r# zMd{kLS|O|yUr&6foIvJ%wsG;wApYg3?$D2tybw$vAw^zW zJbTQxqdEG4%J=NG$5zEg?pkbL961FOT*gXQ3L21cX`beVkPKohsI}2EQ@Nj5<_%#08r9~YFq2hJpzlkI=ui+U zt@kf+Etv?c>}nLSb`Z7Q5{D~MK0P)Og{ddK&aj^M09yQl>#IQ5#d4`S?LkYE!&)+krz8~zb_KM zk0}d6NN4cLBbxicJ4r?`I8dUW46NO?vtYvn$0 zcnUF+`9wL$OBuT0im9H)&fZVI>$v%lR71M@urC}_y-vvr7oQznYw;w?VZ@~W@>Rzi z@bJCFnv_Q8CK92+=$^12%%EQlq@=wCRZXM&67+Xq_WB)5~_d2;$gXx$0NwQ2$!r(oMbzlDU0#eWmpdSeQ}%j`TK7Te9q=b(m*Wp`tg z@TSt6dZJp#Cik=ex?jmlV7uM9ibaK3(o0oqyRIvW9LVCoKaZo?l0BFSqc`_VpQaEAFa0@{tPrLyA*<{k(HB6-bA}crV@{z z5_5Fo8o}=`N2Ny=Q8Yg+0T+We&>5Xy(-I;KdKS4O5`_K@ko{LJ;tm<#o;|yQ<^`F< zaDN;aL`#U(YLX8HDd6m>lV047k8*xkx<%@bHYa>OpF{m&N;EPH@A}OFVxpZ(QN5s) zt&aKcni!$~sZYdP;86AVyP35Q)-4QyoUly*u-FLL1#0Ig9LpMafTpU@miykuJJ9s3 z=FXPWoQ4j2Ij-ts_YDj~uoC)HvPe_3vF&~7B7o)RPe~(~PyF{fwH0YB8f~XWb0DUU%9x;2 zn4VdF?mRt{&<6(D6)ee&Cfx zZ)FG{WSTtexq17{pmsg_Aa%z!009U5hl4J7b`6n?`PttTk^k>->moP*gXC&s)BpKz z#Oh8uPTu&V4ADKV>wW^=qO$X;JMFEh|KUr=2Oj5oDGU#!C4gF-M!5Qe2#YUeA4dx? zGL`ydfBnKt1W^Yb;@HjCfW2|Wfn@B!B&YSTl7(71a5RvqwP_%Hm?d!}+&WPp^?t=J(YAg-eMn=R zV6vUVUYx3!_0!d5PpFRbHiVg83_%Q{t#r-MEwc2vQSIiC*rn6O0B;8V2m#|-+^)^-e~O~DCSlyvUU;7r*3 zm4T)cVH&V#jp1?erOFnw%bne@mt*|>A{C^>%fXYB#)sxNjS`&%T`XiHy`}mfsIY^* z^{v?puygFf*vo=0^@{b$M&o`Dk+Up%kWG_v~*}%%=qDz zl&Gtht>YSd*4TmBFh@+130pL-_e;?px80~%gBcW;ZsOeKH^qSSWCjl+v z=O_Gbm_~e}KZ&j-DXfv>wC*l^9w%`gC9B)Mo#gr*xO5$NU+@QU*HA47x$4qIY9Lb< zvXPH*y3Va5SU8S*KC&OY6&1snZ+heD``U6s2Y;VHWKF@8dQF4K*NCk&;hpsUZt zgOX421(G^RY$bCp2WmCSwNV+e&*KQ0&@vEHM-c?^+2qH8Xu_%au4fyaSP*B^XIc_O z%nM-z$idQWfp%}IX9Pex%mJP!G40ny0R~5y+Vy4V&G*{tqF~22S;syM`$;~PwTe)e zCE`w?=VJKHO0dd!^XG_D(6tqP57|VUq&9r>30@ktsg4x6h?n!X<_$`&68K>=+_ZXu{+RCrCa^iEaIw`o z!X@^D4D)MOZbp=EbP8n`1l7g6`mtNG&4vQ!9V6RAmBXL^eC0`H97&2uKO045bwo6Y zG-~xNbA^nCh}}og9vNu*8mX;WfjNt%+QQrY-hPQOY)uBjK#v-gHPo|8GSX1AuSN;n z&C=6;QdW4F)L;9bB~ltPJXLYlEwQF@y6|?v6vvEw!SSyiWhYUDjPw}j*?b??tn>}? z-tpHc&O9%)a|CB5H>2`c$bGO;#qd#6z*<2z-(bjnOTFY%b;tzdZ8){{s;|mT#=Ig4 z_lzkL;-#_V$*0ZFB#HGre$paj7=d#U7q;fyD;E1$v(yw#pYnVXqGrpLX1@N7GyYrg zkgcwCqC4)e`xjXjE~9&ZwYK%t*k>t?inLcZ8b0n8JW$EB?QmzEjwFaV-^%b(&#T^+ zTc$B$k19Jh@f~b%7|qyq2R(Z0rt1I)3?{d>&_!kVkr13>X7-8+zZfFV6)616rEGEQ z1q5TVR!F{o0i1}3mlGCPiYz!5`%8pKWxd~=PuSljHK5Q+sE$poqTk z?tO9XreClcaTqPOW#k2GazGTgvm=9oiVgaaOXo%{leD&^G{2$q?qj(LsVb8W=XNFU zcVmht@eY3;y;(6SBoBf~9o3sISV*u*mRWfFg^$&39v;O^*Qom8hLLd5V`)t?q_%W~ z4FHZ7^7eBHW2XH0V~O>tz7%S4=nta#d9^}fK^nVS=ekk}Po|Rk_~Ra%yrMOn zeYhZHd#%JuG9~}{LlbQG=U6WtJBpL^_PuWzW`k|YaxhcK6Y)61cE%_ND)qw5CfTvH z+?Z^BLeq;7;Mk2tqeilc3^VV%rt4eo2L)=b<0mr+EtYE%S>bJJ?18aJ&Z?={N!2>2;;j%H}HN#Nn1I8 z4;du96{7w@My@Gb6&Ob<2_}ws$xiANip;3nMgocp9o$0VLsYw+`$7k=om&LA=vRSj z^e(-wrazdc3G4swkP?s}4yhEaWyw}9!)SP%&3AZI&N{>a7!Gk$4QUQ8Ritgk}%Yk$A{Q+

zynWnRUp|jOW~Lv_-l*o}I2NYU4pZsRfx2jElgFZy7}roHH2>kp z7x2u}y*)x+msetO7wzDNLFQ!SDNvJL-d>!Rk`qBJ|0ZyE-0S>faD|Q8;j$1_8VYny zUaln?I9}smqOJH25iwFO64`q{2k6-g?jz%t0lA3RgR z-u@u?-T{FZpuIAqOgv@R2V^Kz8q;sLml*z5_v8lcahl=VeaW;M8X*^bNS6g)G&I&RmOzJzs5St}sR&hJe3fw8W3fYi)9K39 zzA&c1t!gx${d2q@;w6EpLa>m{>{GAa6>jCXYUG@4AY82@Y$3Z1rn$k8hBbvKv?5a_ z7MY|Z;DDzt&^)nbh{OXO&RY^LCZs?))lLc6^qaXrxFwX)C&*-~#0@8#KFVI60057v z!a?rEnNbghjeVD7d)_BF{^;NQkXa>RaCM&ttr;T}6W9GSpnG}g@MkatIcljl0FkrP zRi-5F9{(TOpORe@8ss3qgYddJn6e9LMU>2WYgBp}VqaLSFt-RL;AfWSRsiFkco*>IST$>4F!ahN;-*dad5tuJ1y;CGV_2L-QN+qg}Yk)$;;ym6N#I4ku|c@cakGO!KQwa_Om$j{x2h$N)$ z4)B-Y6Q-{*wAh$?s`BNzkIG_!bG?ZKE)KPJwT@gy86MgS60?o;@xUFmKtX84aI<{536@ zD;jdbcUod2cT=T+J;-JP;R2{#Qx&-98Iumi*lnqF1j5FmWb zU;ru%DuDP?J<&W0S+3J>xt4k9b@n~$@mkR0a_@r@@{L4k=tXDCsY&l*CcrjStCWCT z+PmUs4l+mdw-iLrlFCuZMrI99W@jDcnGc>Oj}?4;ab^G@L@Hp5i`y#z{#7ti!7edR zqTpqELxicg7O}lcD)zD71qh>bM4MPwRg*}bHTON8TjCE=woHPMM;xO?07i3&VVBzX zFKgTzx$DO?eIpT$WF1i8e8!AvDkF_P4-p4ZuaLR$T-o52@Vd{FIs?{(dDmDE%;~!4 zX}5vXEsc8 zBvWjd1Q$D(e3%@4lyiY#Oaq?>TYnKCO6}2FQCac_4w(r0rjhf@VE?GukWeB6N4Phb zHlHvK2ab?c7wd1wjnVR}#n4wK)#TJVa66>c%Nz~BH=gNnVOp7Aj%H=)r(+kHG6(Wf<+($PfH zRE!QYDi%kJ_Ju(;Qm@YnuO}G>#E-cWB+ToZ$l#$+iELuOwc6QB_@weI5Sp{vB6{OM z#3Ig1KC47uP*JmY^`rkhc?p}@UG`8Vy6x?0e(N=Yr;3)3O!!9&$vpmlwpE2{Y9SIF zK6U^JsRDWse)BUBddn@2ZQB0`)qAz^SN(T`%f4xB6_}0l>f0)O?lo2?qoYeU$ zBiSENJ9l>(v7vqrH^|kn_%1|x&DXwp8ckD?;~xJzL+duq{Cu()8Tx?s z`ki+fXNq@*k}~lW{7{v;VoATrKrVXOd(|e@;@O=NsV@a%iOmI+-P(7e?HiFVr?if%{7rd{ua=vx-*-VkcRL2FbLK<6 zdo5R0i@c^mR^!0Im(w7X=t20J?{<5$>%qF~dV4Gg6tmG@M|*b=J_Tseq!2#E2p)8S z8J)-Y+<2>&q3!2!-HB`@S7&qE*VfFgOeX;F<>AhpzHK|%(e8T>dOn_Qi*GApOK&O2 z{^Rq^#UiB$(iG9v%*oXEfQez@ydT$e9eE;!AX(cS$UImDc+PZg`_f_Q`Yq&007 zJf%Gm9-kIdx5R2G7%MPAl~28Gp}Ls8m_48JVD(w}Y>-7n3;+Dc9Vv1ma7Gre$%&Bh z9rnk0FSh@l()RDMQ)BOunfL^FY%P)3}#f?E;U`q3lW$ z9>y2*CwBI&v1mMt;qOn6Ba$ynCp+Rm*BAZQ2_oiD%CiqFkpKQK0K^^*Zn{;#*_Phc zwn;U37@<$HkUrTP(niE!dk`nT*5|9@QReq0pT;ILjwRv!VijM^Kuk%3%lVsNyCA|D zNsFEPwo)G{*nP|5{?KG5LO9mm#t%@e`o&Yh>B6o71QP&evgpy#r$JIY(PhN-p@Ebr z0vXBMXJ1@J^V7K@?=D!L&aQ~){!@Rq!p`+Yf*5PEo*B!G-6iI`nBJIzEs2@y#FQ=eIaCVhB_9Gf8l1(8{t$foQo}lqX1o6xsi7dxR_C;la>*#a z7yXOz=<24cmhp`1s`(YU7=9?zpiPi$J1121{Gk;q`?l&4fz2#-@Qh3?pOW!SXozY47TDx~I(A^K~c+!>K0bRQ<*Jof<(qkGW?3W122Ak4GxSQmk7jIF58R(q_c zn@~$FCjwRa;bx?$C)mElNheCOD~}p-W;=OzD(BZhr=k{;wxt+D z;?i0;kf7@fr9)9t?DdPG*=1>-#+(8o(P$NdkH#=YBB=Im z@r=1zC$ROwqs!;u?502`fZqp8eov^J+|N+J@dnDXV#eS77K-NLFu(J(CmRK3dFBd%p`SAruUgmCaq3ND@W5fd>SEY`Hz1)=SwbTqF15{dp0 zPS_Q9fdv^Md-?&lorhAeujjY2V>9IImZL!UNQtUDv@v) z^+rB4)g%2f@^b6R0$+49CpIbj``i=pxDSC0p15N+;eAPa)}p-%1e0pIY$uzBIj5rl zw$_sRSf0i?paMH_lh>STop4tXn|GG;aC=DE8027)Dc4;6$XAUCN~Bb(eVkS?1uVv< z52KMa8Woc&gESl*@LaGY-eC6!V1MV^Es}| zi!XI8KPxcs?hQkiu#~Mi=EZ_YB_-jcVzB&L<(YXAKkI1vnY#qFu42JSTp^q-g>UtMTP6j z9F`cLy;f_&SN?KgpFR_jek%0aHHZw$L{>kJ@00Gd0DHie1%h`hD1?FRi~kL?CZa|* zJ5=ToNUe!xzVIp{I+bNTldiToVL5{+`+*P?!ml?~VE>&1u617a(Ta};Zdeg>5222| zNl2)7iC!ZEUE1u%YRzurs1-B1h1sC-(3mMldKPx*Y2yq+5g8kV&@{rzfB!{P5O=zo}5rg-99}fea#SXf+K9P~PVXU@aE(ls5u_@iN_S z>5rg;M-5%LLLhXN(Zpy%Cst zfBD+(FF+Ch=%_!y^_4u@eKA>Ss(3z53R?@l#|J5nfIr)5XC05YU>}|Qc`8RQ0esth z3LG>K<~z*H;16zmtYV~a{<+C~w{ZP-3v4$~vfq6|o{8nwPS`=0dxTm?a+D!)Q59r& zh4~PhwXD9R903Dsqv)9rEo`8>qEd{^JEI(y!1R}8rd+!2+Yc3wye(8rIl8eG<7hlbmcnM;GLxC1j=OY8x7=`46v)5wdE~H<0 zqbDSDm+9U9#Z@KqfCa=g87|Nn1(DV@_iSu;bH#Id3+GP3_6j5qq5YC-NJp$P!^$gn z@9`6q29A=AiaGD;9mJiEm=)wT*Qom+zf+TTSWT4*lh z_sfmAop_E7{z6o&+z4T~cEmCbx=nk%JpBf?k;Q$t6&!g2ixQcLN9*TvFpiM7WP14t{a}mi%=r zA)f_$y82ovkcW;lpMxSya%`~qsJTe9nvNX+>!kkNuHfIGKdcTuLUn|&Lq}m^qq3v& z7}uk#pI)%rfVQ#+DsFCNn;ylbDo*sKyh)Ekcq3!YXOTy^J{q{YVNdP&Oxfo;MC5Je zQK;vu(jV<7ll|wtBd^7pr@$X5U=7VTt=dXcu+zbwC)r+iZhDe?_WyqnW?Jo$ZXqmdl znT21O2)X&%jEwL~$yCT?lgs5c|K6NR+!=B55%aMg4fT$<*rr~pqh{?^|Fi5r?RIo= zfZcVXTwpqBrI-=nMkdu^&wdATtjl)%@@0kvoh{^xebjf|v{!LoSPi?BY${etHZ$M*obIO`eZa;dFh;wdt>fjitri#;z}Ia9ogz=Jm-9b^UCeI zXGU@Kz$M$3x`xUsbxxR-o}WC7NA5X=TVXMcw@-F zRzPR{T4pBZ{qIK!7l(~^qX;kjEWZd4DS1Byl?8H6F#<#>z zf=q3zmTX*mN19NRH{TeDzi6-ET)14hTt4KP&9=iX`|U`L9TguK>xtT}5&k~%c$R?K zy^+kWJSwjQT#g)G+fb`yW5}VBWqVwK%mR5}zTZ@D7ekq?eWnq=Y)Et>VlH4F8!G#- z4NN|c_Wn>1Qn70}PZ%bivjj|gAWLOB>s6$kYV$_&r}vU{tVS@Q5Uakh}(F`<4z zjr4$ep*U4|Uh#b6DDq|^1Kp8RGs4bqIEH3nEDPv5(_^1A2_3*%7u#tuu8XeO*>bsE za=&z#UqoclbHao^V+cW4N}0U}8ku&-DS>FciM* z^quVjy(;W$AQ8pYYV&mL+*rb^lJcUO4{<(9DX`8c=n*|dJi8C*j@A@%F^;h_h>#A$ z$?|QPVN(gs-kq5UkS~Zm=Szo7fmAJNaSeb4In)-%^@x=D|8?S5$cSGL!XZD(D{I2`AyMLW;WE6}EU9*7b8y=Hp%Y)ZV+z zONZOV!7u`Fp}Lp4nJF&tWYlg&pjI8UpSQ>4_v3I|35lw-cVI|VyC`~U%6CqLnGnaL z@x_wj2+9$yJYI6;g|J(Wc=ne178>!&8>jL_M25xj?;S{cgL5rpa;47cy)rMf|#=y%FGPl8TNa61k?|4^!#be z%sql5at|tr|CoqGyBTc8K?qN3D~{n5*iuOBV>CdxsK6NZBs@#Dk}U+NVjU!X99Z%V z&%m_=m&Mcs2;aa-$VXa16dv+0fB$ZWoikDRKJJEc0Tsnsp@nubUS;ljuRTFRp>o{6ef@fF{ z3cy^Mm9t1|?_95*Vu=E#kIWla1EPp}Ex*v#0*RGz97tmGkQz5f`PeY#Oen9_j7 z9HexY1u5uz_0sVr&ztjL3A0@-NP089r+v=T*>+?24+^D;+te1B=YL*4lAMTXr{i% zl;*&^cqtefA!U*8K%?3UZi`-Iqw3p-ruSqwyB>E5D;pQLDfx!h8X5>T75KCqCFp## ztO1loHATl_gh&(&AU3J`1PnuC0UBA-tySo9Z~=e?pv(%tNaak``SG^#GYjK~p;j}J zI%dpIsaepc{mqNd!hUoa_>G9!cSZA)3k}?fA_$ee*}YNxrV13N>{;SEqGFSqy1(Np z|M`6@MUQO?yQO!>FT0%*>$6xPuE-tmHGfT)stYSTpC#Jly66mS_k9>r%a?aIVBQgZ zG+Y~QoXzv)+i6UPm~Wi{)L*?vjcsv7b;aQmfPEu#lggacidZrt-A{L%(1bn#1)Ji# z6=D$cXUy2!PQd zyO5VVYLos@zIQ+n>iZ+c&t~9m$HE9n*jRpMLxVbXXy_|=DIZ+7&=_SmD}=-_YR$`H z=JI6@vo4q-JLW5XM{e98%t`5&^Zp3GrCGw3{;uoDP7exAOne zB)uThTlQ^iP=y(BmOLU=LK<6`l~D5|$pMA3c#m_l)rB^aPC zzGzQkdeVxFbxZel9m z*;Ww$b0>KS`nz#Vq>i#_`D%%B{b4?aWj7UM9BPA!_>#0$n;6mi1PdFG)MW^8%aVIz zAIn%QDx-+{ul1rNHPqwPMUbv2P~tlGIYehOg8fI1*Lsr60&B1MtG~SM1XulR#E-x;rNeT5WO3=tiHSws41ZM4HkHJhFOImd$_`wQW&E&3{GW;f z*`%f48d^bZnBKJFJ#zEq;lHcTCvF`*eip7n{syh)T{Vs~Yh9@fk}%&ja2J{Vds&)QDsJ1Q#fs<3x9^$* zBK&Y*??ak=C^ZDACt&!?OiqBDOGmF1gE=6zP8grA>x_q}a@JDQo7YYZ zvJduSwnikA$kuy!`P-d59}bADYa_r5#Ig~w-Zxw)$;+>5m&%w26CYHvImn8d&q}>; ztVB3EhJ~WQO#;SZ!Jc(55tlh+8tgTO|KYRAR9-O5A4fdcklj!)uoFcepW8{nyvWd% z2P@(tPGGcAHX0C@$Iqt+Kabm~ei!q~c(;V#A&;`@RlV-V{=1~;s>H=J=UJH{Hw-nh zd#uh<$i~h2nOah_d@NgY|J`%#RCHE9dAamT@Ksilb8iQ&b20b#GPxPXGY){i{o?=J zSDadt8EMxKegKpJf7I~i4s#hNb-vl3ky>BIZLmM+evmoVVFZM~QGtw&f`3Rxj`aB6 zORMmXN`0UDdk@$g>PW|sj6CW!l?Gsr{26{UQ2R^uNO_EAdYc2#R}-Ff@h45>uLe3u zDMKWD)tP@$;O5Md5N_S9Cwl4 zr6d3z@IEyr{2xD85%gd{Y;9dWV~7iQ{NHe9WsbmN7k-ReWqc!Pl7u#(OU7E(Q^XbQg@IE)HLL4E7?!>0>^P)XC9~=sIbU5+WjoPJO^%Y4aSp0f`teDdN zw5*1hUca#K`W}Fl{pX4e*9iBDy% zqk49h)w}Zn++B;18DJsk)qk9MwHA<$Op`D@YJs8!nP=>Vct5;y51E=Cav*88>a)=b z^$ox@GX2|`wUK|8_g@I%pF5FAa$|#9N#u=b02Gw^?bhre@Z8h#aQkUQCv6Q3KYsP| zv*RAgz4$G{3WoUTbNplPVu9?D`lO|1#lNg1f4?(6F~<)nE+--CsZ6(DPk$-%-TwwG zJwMN~TfQyRm{Z0{-!D|2Z_&=*RmDld&IS_PV7iU-M|m3L<e6C44;xJ@%FFPn;lOmFkO3 zwq8y@0_^KK(ptWaMpCBPHPU|-;K*&a3kGj-O^dB90Yt1;i!$|6 z9Pj5F>c%;|#+v#f%-|1l*|t0X?CKr&QnY37Gnss1()>?iV}H6_Pqlx!r|1#JsoN9lKqFuWC}=(Tvb8H zdmmTQz4!um8GL$Tk~3uM0gvIeQnPj%V(wp1gZ+I@6w)Np_5VMg(&T{&UU#nl%;o&% zlXOUJ6`cTY<*2_wyqv2XGKan=tCL`JW2{0x+-a4u zC;#4WE7R$JFl3NtkjYP@ZmGlhfRmx%d;S4;oRw|FeqsI7$t*dJ#4b>4UlBV1HFj84 zmOt-08RY!vl7K2_8SQ<+>RI1o*FPAfL_iG&u-<;wyHpMgND~RCq9W)EEN4#I-n7vv zHa2VricfKPZ?{Y{mO2JXdvp`5WVXC0l=11mtnP4l-J~*#iZ@h1-x@2aih~2l&&NJL!5-L5fbHZyley?m#`a-Pg=)W~U}_v~cMIPpm4%JWT|Tc=FAW-s*&}t9 z-9hVQSN1AJI|P@RenVgOWD@-LlP$;sO2Xk~hS+1s(voLPRQ-@a)a)?kT$u>*bS5WeN_hxqD-yIqW0YBYguY70hErik6M-o;-ld}>05{})>vk{( z06{69?RQ!YZ&Y3n5*urid+;Z3ppYgX+99a*s%q-xI`$eAps2<*70o?Ndc`rAO4SfQ zaoIJyteJXWP*PB8G=ItmIALqyfrBRcf=|^(o@lwE0butbeX;kQD?Ti?9Ex$tnBR`} zakK@hiJy-AxE3Z{*zq}*!-*c3Jtd1n5abbLBFG``BN&$scytC;!jZ9d{&O{z;zI)-czVt(5(|B)i{XE^cjGLpakM^X36281_< zEq61J#Y5hNv-E_BcUzb+G8c{szyXh2yAypZK|}TbizQO*NcHdDd9C|@+&lXl69d&I zCC(#PnvYfaZ;f|fF`Hk|bUy0&06>bjyd|%Z3(*>Nt@NvyQo65Z1?SyjQV;@8Q%EP~ z685c?N4y&XYi?stgL7ptM}Rf~xZ5^a>~4$KB`$8Nd$dyy$3bO8;U|~vFCyv$;GEJy z|L>+2o6pJ^N#G>&XEk4B`oG_Jk5qL^>qEezGAks2%oLFx^5|ohTs#F(;I_lK4dSHH z6piS}~ zRNA=t>HahZJ_z+MsrtwAYWd-n4bz73#yguEM{N9Pu3P}P99zqsahv*~aWPM#avJn@ z1t!VRUfyMd-%x$}#qL$oLq8?;A>;I=XN=GL>1eg`sJM&P01_MW%tKK1z4!6^Hw;Rw zjsO)M?RS#x6tT1%UhJGJc3ZHRLweS462lH9?bC+*=L&zV^ z2DtKE=lxs-iu3K2JY$$$C$WmDLKpbRVlU*guh?#8=yJzpU|gI!L-Rz)&j5-;RE0gN zt92;f6vYd$AA-aXzwxi!ECT+7uk;0aciBf!+Mi>VodM|cC9a5=tVjd|yE;QU&;v*M zN~2fu(HSGp<*iQf;*Mrk8PC$#C*h(HOS9WdzXDueFpS`X(a7Q0bZG!ljC`UDU?qI( zBivFAK7upMUuMYp24R7w9TxP5#0@qcq4|^~UIG7jXr|XKa^YL8qcqlW>J*B6;53&7 z>X)Tqy`*tBZjtR0z~TlZH%+!}k0$}fc=jwM!VPi#%IGiv;ZJ3cW>93!`(q{nw!bq> z=BL*;D>^y51qBNu5N(06vP@%>Ocmmp%uQ`cpDQ`*p9Tmv;U7*rUdkVf^g;D<7G<$Z zb{@INyX;1z!t7^^2NNCxho=KY#4m9Dy!t@)=k=;wLcoU|(hL(Z*$&#OBs z96fpLb}qUr1dUW_lFbP)Iz=!!_F2phe0BcnMjWHq^`7Lh$C=V4+>9++yd@xf#Es`s zEORC;+uQldtIGOSCh&is`Vs+w|z_46Avbl0XbE@LHCfAsH2U$mR8_0>mo9J^(; z$KTbVQvSCmA)5Q4-`g7CFo3}vcIQ7=mxeL_iT=pu1nTk5E@m6K&Sb8QFl5hCEl7&d zst$|-&q|JhM}tC`pS(>3Hn`0=IfU#+)l`xQ@{mO*kEo&4@mN1%&Cp5&@}|nR{a;k@ ziSMf`3ASth^`t`jNt2YS;qY3^R+F@g{MWB8QHg^n1@l?vw<>3w=pJTUAe#O)`vKxl z4E}YdRRJ{$LT9>#<{wnXS^I!XsIA)pHu)%h1u?Uy_(=(kjvqK?Vl;&3PlJ(^=K{o$ zKu2!AZ~W8r5%fM<{XKqbFMigE3BOF5PQF$3by!yZi_qyqp?LiI+sD&wRBJx}(Z4D# zx&Q+DR|AL?Qq84Q8v;fjq=;Z2A!EWKf&LC$qn8=X+wYOHaQrO3^4Jll^P{m=(kugp z^uaqJJTra|C|X?kNow6}K6=QxfjvZ_(wt22rI-6R(l!;f{brh!zWAIR#IuG}=@Rj}yg zS4qpE7k64y)uTkLN-D zc`m*&t+KBysu!x-VWdnRw{>_UYN4es8cO(vxQq2c1 zMyW^afH9W$kN*(8oq7SH_hEa=N4_5E-M>U{7s0h7RT9yA#Uk{pBQbMl>SS5!6CF;UCc0M zXs|)aD>7`~WmCVE^b|2$cxuP774*waIU~V3i*&w(e|YsX+k-3OXLAL3!b=_neFw6= zVoA?$!u8H=me&M?KS^d?*oHiT;BFKUp06fQO&KjmPbUC|@_S;{qx!q+H3DXre3pl3 zzBx1cQQ3)|1`^h z7d#*~MYq%rjD3SH%b&3dM3&0T=naGr8siwiGtAL}zs*gzEI})8!Dh3nzM()^*LwNI z={-uOTy}-8N%r_-0JRIZ58!SPRf0@dblLB&%Pa?I+3Br^#lnDI*@36e>3ve^?-g;_ z*_CZ#^Oaiea&7+$Jy#oX!Z&GZ@-WpJfKC1?#*@?Bh#Wcm;_{17sTtLH<`LPS5+-cO z?Yy~BqmNB;_;o?1mmyF)ZlJs}%}E~!mono5qJvhZ+FS&^C%VqId;iBW=QnHyi$y(6 z@03PWXc2<|y8lc_NwPbC^UcFfUQZ)7i12iG( zrE&B;W$o>?hA8dbeE^4H0vr|Yp@tNNkd~lghMW|U9bQKELyT9F7^5mmM{(qj37wMt zo2K>OF2KJvX3!sRAFpe@y!%n*zV#z6FqTQiL@o%CkPj*(K48sLS2qrYn7pjN>MN+u zdGcuW`C*KceZiXPC^y#TKo`KH*ow+0=-{N_IrIzz@jRj__AQrun8XK*@E0p*&WVxF$aBrhV`A zF!fy6wv;wzv;e)2ioE!iK{kh*Q(IBBjWF&Vqu?Yo~yy8Ct zNRZ&^)9-$MmCsI_wdq{>%RE!LIL|&FVdnJN;QD>g85D^hYEqdYvG%6{`|Q?~_+Y3v z`LHOR0+Q}v+>3j4(Ph$bBN4`E~uP%dPn!gDNYOJ0d57dGsrl2dB>AUtUw`d%a%FHh6So6tK8JN&;&Jt_yiRGkKk`M*KsvOC zg6ub7z0xC@u4Q}pqiiK`st`VQz`qU1JM#n?8kb_QvM^eS^@HzzzZ<*&NO<(;@pbm6GkG zVUv%+VGdHOocdXF17P#T`vCCq0dI`{BDS(51J&n%@-}+?!>UrH&fD(pxOsG-#%|Ha z!x)Nm?e!uI5cHrmJs-^V%S`Ti>PegzaN1v16LcD)G+JCagFq2`JfpQ(p+C;s9RNUq zJg+ZBreB6sE9hd)o>lfmz((OF4vqorE~I9ibFvmwWi#XHi-5k&hsa{kmR!&im!qrG z0?tDAF-MxBz&H5UNCnAU+A+i>w-~qV#zM<58?QY*Dv3F4fC}>R^3-K;o;uLJ1avQA zjQo5H^2`WhfSuhC@FdWFlz;d!2T)FYWm%c3eTd3a%hmuBd!G!WfN~P;58Qo^TOP+J`*=D~3D-N5M&16$XSJr9ELsX?u`Npy6pU!(|;Z=wz; z^*?E)4S@M_n&ju}sR2B(z6o}!$uRl4u@14Cr`f<8V0xNl7yQ$f{Rt3LKQ(kIiw&m% zo~W)BXbO;YpxAJ0YM$0el^ivJuFvEunzmBdO>Q7Psi-eMNuFfW!ezl;-{Y;X)5L_2 zQvxd?JW*YCE%&|w%})cjbmZMK_fH#^tX-!hJIfNMWAoW*k~w`!-qwW5q zOwWDeE}Nv|cFSCW84$9xR^u=`<3o(i*qi|dU+w_z1yyPC;>r454qIlT?;IJ|Du8eP z?(((i$A<>JwDpIm4vc3e+1lO!Fc_=Lp2FL>!|acA}bU@=UQ zB3K8uo~Nz;1kn0FiEk3>7aAD`gKuZ3%Is%xY7ZwF>M3rz$EHr@GynjI)MEj=otlrI zMB%N;aR@0(OaIgprSSxD@E^$eJiDlxaz|vR3nJ#Ak-kRnz^Xee;P7X= zVn1yYA%AfWqO8CF@VZnum;Ey(|FgjXoheE!M)$dBS)rV(N@KRuJ0MtNeM6NP!hx)y(}uv~o}Q9e+7qX<4{OxLtue z80%E1xdr&iC+*<0(y3>NThS19(;l{^Y})hDRVjlcdg9QivEyf zj;*SSK#-b=H*eLDMkAHZejUQ5V7OdYIM0vxGVk1m!#;;}S95O*%RMEoBkd?NAbSO; zHB3TYd;=Wprr%PjBhHtaVObVo3-su&C|269J;3k;mfTW@ZD~F1^#28HL+P$ocLPKP zRv0o4?tT1f>Sg%^fORl;(Sl@JhHlJNn*dQR-yIJ~WQLV`Nu%2JM`aTx02HK~fqb($ zHulNv(df7iw(iq#d}-XidEn139(3;#H%0zFfKM^`Ih@#3P0BxsThgPOhh=zqInGA}?9cLFcVf`q&)hsnR(7jS#^7{O zJq>NgSIDr69a2!F66MYBRRPb^){c6mpzZ|6PcPvzo1p)$|zzOwTJCO0m| z3>pcIaDbtS>Q6#zt?C@n0jCpLH?wPsGOMu+8)VM~v^lfNG?dsL-JS&5 zuN!QzCs1(M=v5fgDLs9V>O_~>h;kvj#iU<2xjM>0Lx^lb>vlI)cV)aDF$d3n!@va@ z4)q-MBz9Nj9sxm@S9{;Pq~=cvjupD5g{A5)KkUUz=Fc@3>p3c)&1d$StCs4j5Q-uM zA&!^C{pyYZLVPL*8|ZDP;;$W)7Y$0*vDk2^v!zpzFffobSiT-L5;PK-4Va^|!^t{c z3TZ)zUB|%9%ei$h7qI(40HTb^Af1AYWN6v6*db_D`E?ys?o|o~D_Sd-XI=Q7F4O~&n3=gW(3bUX|3hWfyP1na62 zgwJ3sB?Svc4>M1~6HJp3U$DB{?znk1YFe3k|Bi&QD0db|+X_k~Mi*qtXRV=74UVOm zh^C}2>KRKtddH8txT^fS1x!x2q(;GT$7OhC?<@#m-?S$s5UY4}3GQZiXw};4gL|da z2Uv@cd4dg)Kf{m%hAmoIxaA&}hlxa2v z;2iq578}5OJe%EmGA_Bs{AD`J&(rB5Q`f<7lKE*+dfR%APwsql{V@xi&jfug_Qv^2 zGeY2gsk-*Ssa4{h8z(%(3wG02L;dua9bv@loO*n=I)IX;~0@p z_M$%h<1;VNj)rmcQ7 z?lktZ?eMR3ef_GfkBL$ra7vFYztVFhVFGS_DONCDy~~WFy$HofVmytu!L7$#4`m8^VmnvuP22D+$?DeXVwJ3&h%Pl-qI3?y}UQnw8u;9V(Aq83FYZ8ws&?SY_mF1@g4an8vx1=Ad{Y*shaknRoP{RXfF0dr-FdW%rSE6Q|+dG=_t^V zj3xGj*PgNdDb<8uOg&g1Km>b^m;r&L)YGd!!|`^#fD$Kf%of16xjE0bZp29+r;@_c zeVjKRZ_i9xL2xQ`fEjQ?k&xHoe>=OKN2-m^`ptcLx^jlxk!IXxTbv}&yJHpBLm_Hi zM@!0RM+?zF53Ii~cx^ajLs-vOt?$+i;pM`4d{LaGf|#d+;mrt1NPyg{`9?{{xjA~n z!w&1ffoow+(3`~%7rZOqKn)6{2sXaeoiITiB?P`|aVUu=)Y9@~B)$Y%OxLlNF>aPJ_g_p9> z`OR_u?JMAZPN(Eva|YbACzphWZL^=+&_`XPyGQn;lD-cjGsnc#h5$lbSQhO$;&XqU4N>;5TT0kM$@=l!LUiyO_{F{6R zjVmof;&IuLHo#rT7qwNy?ZNW8!{xe3>XJ$TA~@99t}G;j(*uC*V6*l(C?O{7d+a0J zw>Ci%-xlZXLu}vI>4{wPsl?Uq&&M%ZhQ}M=-gk5e!C7@jau-&4EEeNg2gNMN-z$h-Ez6_#oBHcI;o=WIdSvHe z9-XJ4rC_dkS6<&6o&?Q@mK7HVv>i6rS+buyPDEZpl8-0$kCNA9m48m(l5IDNY)3sS zI$M-pTxkm6j=K6;g!66fS_)ePEf{|Sg_N)9X{XLJ#$@J4$ahB59Xod^HqY~lI~mDa zeMl(bEvBvVc_X*oDpd3>IM`2nLn2DY*9z?U92bXqs__*V*dR zKjFeF=Exs0!J3XogUN1EKm#Ed9Z=L4LP+AFT46YK979GUYlbL9#J4M7N9fU@B|So! zK2?D2G^dlx%RgQ6G0c7b0V6k9O5PJhj3Q%@Naq_APl^W9zboCN!ewcc!#J)-5qf95 zx5q-d&$`}_pwAQ^A}{vdJtxaT)PWN-d7eZJhI&>hhw=-;tQQy(H6FYWCJxe#c`E&$ zNuPMj&#p1NI%{j)>bplH;BoGUGJpC@3GR+@RreZDMSSQNzr~_9?qJZ{V+-RxWm?It zS*;gIP)Yh6I}fw7>_c4TbQGsSu+_WX@n<hSgX8t6Q0b^ zIGuN3KVkPDhv~@#uSdru#@Y*}0>kE(h@7qNmKHUu59!S>g`y;n-%vt}=8V29_wYhL zK-WRgd)`ausV4;;?OQ5a!$78IESG)*G&fJTE?_la$1I!ySe}eu;TvX8f_=|?;T32rT&ybxzO(n* z9UR@k!~-%|k0fX|XhQ^Q7M3h$KAX3yDVk4Sw$}Jopt$uW0B)9(K1LbL))3IL^Sp;T zmJMWE!?crrrf9LNRbS)TxM_zM89vxIycD_y9o2ZghA3X$y?16(lK$GDVsiDIZIx(b zH#nAa^UQJ}fN%;bK@HzwADP*6J^AW5 z$V&OexRu6et4Pud@N-b*-1*2hY#w`PeI_~gEF_~xDqu%?Y+(51SZ{50dv0p82u;Kz zBZ zUUI*DHQrymF)+Mtrtq_Vkgq4Nf@Za>U0#G;;q1EXPW3rCb>R=}TUzqNVb8HSj3YEn zmN!;x67)ZGr+}>;d=xrqtpnYaWWGM~i~;9|M#>>g*WcM`7qS>kCKsmIU}^7pfLF`&h_m@WP+)x76V*&n$6cG5WC(k2Q{@974I{Nt7J>8#=r|NSUcZ@vmg4HM8x;)=CCdc>|e@5D3_z)@(0ay5U z16%wjJk_H~2Bj!HkMV6#NeAQm%PK4J5{~v1oPBm7ZVbX0i z=Ph-gSIhs{PsxV4-cxY31YN)U@r1=dtZ~ofX~Hk<@DY$k_BUfX^25gWph`n+t-lCN2wl}cAYMD>N@Fd*mtN4fW5uEmcq5;P4to6B+(2k!qXRUWvpFerf4VO^(pvn3(Z(YSN%yRL* zfL=pkKy`c86OfrkVhY_f=;I<_#oo-bw^RO|Bb9G+(~c==@?5FV+X`bwjR191pE&VA zxPty3rM>liGVPh~G*OIh=U@xk`h`5`M;gy6ECU{?-({2D4HGGJ?@b!t%r4HKd|8| zTd32TO+U3v`r=l@^_0i|4PIC=eMGf+w0^5k^lo>-QpC__~~tSG99td%IVea1K967*D@M& zMh2kK-Tpq5m_(nbe3|paFD?(QEGq&-&8H*_Z&7%^-j`5}3vd}|(Gjz=Zz!UGZdr3~ z|4EO=NQ(}dVy9(mC9h$#H)(U|ukXWvm~2*$EMhzB_iR8n)5}9HB(6Jf?6^^!>MFEh z-iUtu^?$0jCNAa!S|yoS-5M>GlBv}>L%JPoAsMtO?=^xdqkgU0P^CK?W@#kj)STK0 z3`{+qG2G`*QY6mPw>z2=EvB3u*Xfr4K5UnP!eRW`vPFiK3vl9sN7Z*WtBkUXcg7X& z`tD*S@e~~G?UgXJoRijV&b8(6%KMNq=m_FaM{6-vRkmF|SV%a5V?>r;KotbrU%;Hf zRz?zPtNOz1ZsrLJFUTfl1muB(U!ly@FonVRsh{A_2okYJ%7ZU$9Gj<^VV7#WCOy7ozh(&)I0G&55 zaelS}4~3Lq(TyHj>5m&1X}Y5%ao%9gfj&KA&_N!Mnx5FYf;o%%w6e(zY39)^#_wev zXg@adJ&X~eSVtwB5)zYGR=h>f2Vmpm{+^yDy>}7!)x9;sSLTAsKUA~9lRrqKbSq*; zhaZ@-q7NYi4DrjI2X(|;;)67e#8Jl_R=|*93^>#N3-LdXH8G4=4E~Z6jQvaeKa8_x zy=i3mxYr@YNh5Z2G4!TGC|Dt=D&i0Ef0awQZv8s8cI>SK3>UJpjIJ%Lr-?6w71Y`_ ziaey6x>w7(>%_QX{af3&`2D*NFxVB2OI_T1@z$n}z4yFth?=pvl6@Sn&r&txz(;Of zTAg+Ie|US(Xt@9P-8V>xXc5t3kVK+)(HThu(OaT~=q0+*f~b)w(c(*#C__X^h;Gy< z!C>?jT`F0YQbAJJO-X=Z)yC;pYh4D!*cWnFXU4#*P-gfLr6ur-T; zsBrs+2A0`yF?f!a%-(jmGl9Z2S~Ag?#VP^JYY7msbaEoijW`uZuO(%aqAW3FXU z$&M#Ki>6jB1A0kOKwAL~KSfC?8tQt|C_T44gDA`Gu7pC6(YL1GN24zjPxEE-x--^l zFW&ZKDriX%Ya>3W142Hq84fVMK;x-wHMp?uJa z`|UL)$uv7FYVl&XXk9@ASG+W2fFVINl&Jj5bUqsNT6-)-*1^#`uu){1q2evJ}hwNn#!RFp(; z)mA3K;>}4JBv?`y10J(ITJZ@==S(Novti9m7!T9!HAq1*wR|gg_V+B&@P9Jd=7DEJ z6pc+&Kpir3%TP#@BF`j;cXvRM@+PQ3nf?Nt2Kr>fNXho{WOjj{@-tWYEjpM$WK|T1 z_@HIN?03Gy-A2}n8ZJ@`D4t!VE%38`tPY_2hzK8`q}v}zhP&fGP?Q8UPcj2N50!lb zlv^y92e`6x3LGijEs6@U)-GFM+RRR5`ZIv%j@kJXkozax;XoK{G39V?vpe(F;b4oQ z=HV}ToFpN%z|-ddq*U6-CsW)Wx_yrwJfoL~IYJp&zZTdHa+W9Lb^p^FDbi@on8Y3H zEz_FHpA)UdV;%nQpxi8-Z`L3z1_p^2;2ZiDzdU#&=3WZ}e@uZ;ELO|<@qa8)IDXwj z>y)-)lzZ5a)dnyyl@fvr^JTLas72z`bz|&+Ub%FUPYBuWn)p{ucbd(q|Gbrf-e@OU zi4j04e~?sRfTX_sk!x7}=j8e?y!7u^j-+Jwlmb=nSr$C{?<%?SrU<@%$SKWacH4ic zLH_d<;>{&G@W=5B&h)$0RLx*#*Q_vaU&g7~V9gYUZ1J#bZKiRky*H0EN!C*A&W5~L zNE(9z>)**RBiret6|-~|qmHT~BpV2-@8Vo8n(wCXiYnu)3XSJL`ZYY(RgE3<`d8&u zM910wXwO3}(r(8>c7bB+v zb^9L}*B5Oo><^wNItQQ%^=!VR!aQ$j2vW9rMV3w0uGpMrZWGR!M;}c7#q=8JT)Qp& z_}cy3-3AYbty7%8wNl_NT4c`Xb_z5m&~~N0&`Nrpt~Z#DwWe1 z48)a_2f);nawhPSEnz!rpkMD67!JR(T)b^SelMU=FCb#~xbl2g?tIs@tm$LyW}uEX z_Z>|`>Z;z_Q!v)-^bAV$6r_Cqc3Kb6M0cXVPDOn<`fxpgak>5Q6MmzCec$S<#?Ru# zi0?TK{L7z983nzP=sknAGmql^+dC=0$0S=yS_&{guHosFLw)TN!Yy!93j3#sh^w(x zLGDcj6s)6i%EQMXzB>@kT*idiqWb!0%UKGWY7k4-uPT*NbJE@W_%$`g&lKceT%T@= z!L_d-J#~3#Qeo)>xxgZptc33BkGCj=W`$}ng~5;s_ugy0Uvm|g>L9RjgOZI6{N}#{ z&sFK9IAh;sa$-=CQYfPj&co42t=Nyw_bwTEsvE%ASg94TDGU&gb zRQ}i>{1;!%up!t`io!sqX!gk4n75^>6=@jk0ZoF^0U?4sgT++c&<_HxQYl{2zpU z)nuU|L!Lm>yQ{8D=N{?kddjo3Cr@uN{b@eFi1ZwehTeyRUSs@|leh-KSTK3-mdl0E z^*G#A{=b_ zg772w6OYyMFu%2xpqS{2_ybW&aL2zieA9X0)T!JvTgLhE0oCe%bPTmdO+w@A*J8Ss z)@#rbFrQKrKsH%q&b!uw{nxu*;L&rHw6YFc6Ct5rS9|@mVDhyQ1gu>p#ux>lAQ3Vx zbcl$AVb+&xwPSC=n|yVC8w5IWxUkd(r|W>zH@o{8^DK3L7?4c=wl;3I+IkPzRyr=b zR?G0^T;-&%$3bGb16_r3jJRnSc=c7Sf( zjO+QC@K^KmKl7~&h^AQ<|FT-D$2Zz)So34wMK1#}Gbm{+7JB|C3Jq-Ysjn?#pHS$> z=csi>!mJ;&<+d_bMFAH0IewqNx=&B@#WPys&l#Tv_}Dnp!E3hlvD^Z=?i=Q{*P%`e zlX!IUU;19QnJ@+hBj3gMIz9iv3s-i4e-s`1ubSGw%%UM&$VENTr!_K8?ok`JDZZDf z>uO<<@e%N-_ZG3n_z$#W7<}%sBim3R9|0Qm=jUceMV@I#@Sx4cRdjcvf8!gY|BYh% zaj?YZt^MkK0?|`vD5)IvP#8^il#p=*RU1ZkCwAUagjCH=?M7?d(qPG!kmrmYjTxG0I26=^^8ki?cdm}Rmb24m_|c_6u8a| z<%k;N*;4dZuSgv5HyHZ7m;!2&1iP7!g~EJ)XZH>}06cAFU~MrBt#E(iJ+jjfpESuT{xIix&ziQcy$ie_j>cSv-)CF^2{DGPjIS8Q`G{DWC2*uV zxRjKDDEYi4;$F*X4|%xIbm4ECNs8&=;y!>(BEpaP{jW>vdJ-u$^&V_8i2D!Jelay< zx9%!mb2>A;8X?r*OtAZ1@AWSyPlUEbgl=3`klI7jl*LQ!17W)2D6}85r#uI7YVKr= zUQYUD-HGc{@24Mg@Rl{+a+&sMA=Yj961nm3rKd3a5`#Wtm#dV@x%rBJ@JZBfe>aQo z&C9co8nBku8f~|fSnbk1TWnVC#V73IYS)W2uTFZTds^xQPiJC8tj>reC^LH@F-O5Z5QD3k)r@y8@i z$-weFXo=OHVw|E*6UWgAAm8VnLv$5(J`}Cc(cZs;nx7+tQ=46|U#~dgFPwx;cx(FK zj@i}GXl*?vJPYxujD$fXkbeHGxVz3Q)A=p8J%*B=B{)lk$k%GSocSe)F_cY$N5}`H z+n%!~HFZw!Ob);5dJv)Ge|y9ANbr&r6!UIubAM5#L#Qz6(u35mMz0uO=9vp>$&^+8 zh$W61-o4Nz93HPoB(Ivb4FwGP-vLpG_$6>3*dNK#(=64JZ?z-)qg3AaGfBf8V500DfR?Ak1TGkO-e5;8D=$DeQ zi4Q+J>c)??E1*>dK=L#ud(UA-K$of~CEAImSY%f2>HK_R+^rw)0G93tm)Zm|VLp`D ze>Nx2s{6krI80S!?a^uA#?HS8d=FqxC)-X?`Z0DoyJ~H1eyDpc>Ut@3yQ#TgF^l?G z^cdmwPQ3CH9Ga5zpA#(`gp9j6eu7FRH${_Mk5OkkS5~;!5j=c`s1%R{wHY|rV!jJfm#*}KV_OMaa~CqMpR{dc6@gC|r zTVMU%bb8w}Xbno2CO`8>7#>bt2Wqu zC&aJE-5ARBk2O>X70)ezZ2PDm1uMn#!I)*xumO=t;FE@JPPU9uS6xpg?jg6Vc?6DP z+kg{m>>YGMMA*nx@W~?l)xhm8Zwl-sie575L>`wb8q*!+RC&ctmZE1N zK_v#R(N0?`$H(8j52u(=P03BC3C9G#D6wY7!;7=%!v_%FWzdFWFKWsNSBiaRf}tD`hQq%2ba@XsDTV#yzx$k|?t>Jg z=^JO%*w=&yD-oK07BBY(lw8QejVg=2nRJz3h`H5{4?ozD8m-arB2QWlLVMpn2k9e< zc$mrqo!eM{qo~TN$Z~BP;7;Yoz=U9AdPT1a6&SE=p*cL`HI64c0ZTSF?o}}@(=4QC zKT~=rQ>~Ne)1Hs}Jl{UQJF8;2$eituQwdLh4vek8SJQ9D#^D_s>dFD1IX+;!uNvmYSgW9P4nl?=c4F7DlzSWB zV8&O8_8abx@9>RExH0VR`!6I(=&QAF7%n|xHG1QGq!T$v0s2CU-*NO$OXTsLqr*4y zK)xrwVs>7zANR%Ss|h%mS`G(CWAUZjQmW-C#dJ0Cdwo#MX}<);57@?~Np6zUQAN=b z;X#JYmVy5d7z4@74aJmKQl9riGghADtz-b@z6Z$K!5!?`i=OJg2nH-6mZ>-M)WL9t z6Yk&v_dn=7eQ9Xzq);P)6Yy>(6GMX0n-68QQiXhL6Ke}>lZM{TCKt~6e?m|3d-;E> zrvN9ixZ#^%DHev#W}}eKR1zgLkJ48e%%hr&NU&ir4o>4++N(5n-zbH?7dk|_I~#um zGAQj_2A@T~<$8(T-~N9B{GDa znt)Rd1)GgXFTM80bw?aLUwxWY6>=~Tc5LEQ-BgyUVyIk#Mc=r;h^)to) zvxnkY+~p_#%Jx&O2uLJYRd*h-Ngat|vye=>qa1e=s(5hi3!Bcq!(17gMN`%bN^*n< ztAB_qndgds{FEtLB(8dQf0_$^GySVl#mKOXF6@8}bzfnUbrZZy zm6Yy9%o!=-g)M$m%2N>(L(R!ZKdyP4Ei!)$kGOt#RtBk@5#R zhvg#AZw}UL&p}|y$;i!xTK^Uqt=DcjC>$Uu2mXMyMSd6_Tcs`!NIn1^q0@KyVG$en z7@uAj79p+IQohc{+^qF6hTUYqStm0i749v=(R*7B*2 zgWRGw&(got?H8v@+uc0Jr!8$1n4WUy8Pb)>>Wi6nO78W`iw|NZi4)9Ode=n~S_sa7 z@Wbmjp3+4Zy2tFDaO$4E{BS#fSKvs!Il41N+}}P$7q*)y_XhrK@yV2y;co69se=mc z04Kz2tov?@ooM_e~Bv$AKL=lmGpQ#+Ji z^a{L9iN}=jpKWDV4Qm1$H~lxeVuS@g$8#X6_T@u87pE=&xBju{)hj1MAiP(mruqaU z(6}v{7|wBy4@yi6CC4E`v4`1f+JpXbGmRVwP4`^u~GZ&oU?O=9MIJ{uS(L9Zxosopu zLo49g7BEebTz$)83X~;fY%wAM21gLiURhsOkm)&%=h0)n$>Xm9b^6~|p4J%aoeabu z&AHx6?W2Tl1yik2LJw!_Q7Bz@K5BaP6-pMR61lYBuE}W4^Q$bm;-rO8zk( zedi@3i6NyZ=m(04^g2@zb&+xoa8qgNFg&*Iej_lk^~(Nt@HAsOue8eo6+jFr!Ca*2 ziABZ>SZ@^;w{qoRL7AK;BXxSsW?DhOcVJ3vjMnIVQdvs1wx*-XsX-1hCZWy#-h=i; zcSmApLlS?r@g?Q=ivUj$dL)To8Z*!Lzv{2!13IxBPYOQ`&8XRoe>wn3LwC?6AJ%~v|s{dL4^%$zTmpR=#wZfFqSZm-YGVRSz%Y;s2N=lRf;-+KQ zEmgIJp6t}oK(0QCVYb9l=~}~FVfzHJr(sT%M}n5K^F*!Chx}H~4O$lyn;-PFW36jp zFfhA=SPL19zOIF#i9pGar3=Qp6mQ&5q5k2mEYssf6s zGWl0WL+}P^pjY@<)o8|MHP2c0S_OUtcEI#>TIgJ-$pZfE8jou?M?R}zj zLqRXyE7rS}t#)#1QEc4sM?^E%k6YgtMT=9???L!bX6G^*a+9ISPi8CTcuUp8U24(S zbxBZd5dpwGV)%3~Pp*fjR>tW*izQ=>I3+Ko=zdc(I|a$dPI8S|pr6{^pnO>^pV$c} z-+nX)P<S9JaKcFfD}P0Cpb_g=lcF5^YvX`p!Os(%>8a-4V0Gut!}0Q7Kh zX6XevK%IBeT=r@abY-i19872YLzB4G^7FbQVUF=@k9>)$8o@mya(ct7N@hR?$UUc+6W9yJ zUgvS|4Y>o^hR=>F-0bz+Lj3Jxyunzr;#NEdySJKR@_QYqrVVGv0X}*5pJIxmd!w9z z8lf`_bB!0hxtkh&HAxY)1{(#if>=ud|0~ik)5_<^Rw2w~jh1%5WQ}VqB|PQx05WEj z`Dy6`%DcSxat3V%!G@1&M6}Z{++cCgT|hSIqHZ;8)MlCf@_srI zji`iH*8a$zE}ax3S-2Js1CcEIyR_u_GNs6xEkQwvoelw*S(A=U1OZPhsS z2x4ndfoh0(ne$5GlLLJUgV1lvI2iJX^9Bj7=Dp1$#1E8`n6>m+3{P8w+Zj1x*dld* zT)A72T*!%jqD%Lxo78VLRd%QFb;wYo74AQdTd+x zW5GRuT_B2R*5?boEqz|b(^oq?7CgS%Bq$I%XPTOB#(p(QpwJZ`8`euhLhR}Mqwq|` za8@y>og@^(Y$2s5R0!BATH$0+$nO%b`QF!b$cCMik|$}4_phMem`AuY2gmZSAfS@I?_Y<^j`Vw+?TB(oxQO0DELUE)cjKNQ{%KVd_w4Wisi`zxwNNxYKFei``>%q{)GWihlp@SacHzCeaOjZ*)ruN!EJ*pWXQ?2&sL=a#_h?i8MzNv&sZ|CkhFS#7v=yTL?aN z8eVm$ggc^q?g|MJ3s0lOIx*zxPWcL%_1)AU`PLN)N-{AH0XIk167bfEz%E#mUBP4# zd3%zmUchEX4!z}dPLQy?i1Ec^l;b3RqCga+Y}rS6Kl)V_ioC&E%R_4~Zcz>xs>@>Vph zJqL?gn!$t7gMAYacG8j6CCt)otCa&7(y8_!2 zSuk_fDqyQ;u&58EcldZcpxbEP;;3leq9MyoJsw#$LZ-q*eVbndaTvRA({J(i2FaVJ z8i=ylN3)+x%EHkk#L(lKaU%@~N63f@p#;R6SSIaOnpzqi;qj4AM1t(*nu~3*Fb}&# zbBE2-i*oI)hUxJ8 z&S!4F2|g*D*?mG37Bk{+wY&HF`(*u{uA#7yArl{jYKhAY7RuZEKS*u5t|F|?@5M8w z-Cvju*Jbwwwe-z9r7 zgig&~w==Gm{OD1UE6|(?N^UB7r^cN8*vx1}VHU!o3CSnX;G?00yJ?xOHNPTtRL|lY zuj-@w%t>T8oEmY_$qXLc7-iP@DLtR4`3Si)+nM*hRCLm~KlYnwYsw29fg{$_!Aa+?j}2JJB?_*0{9xYH()>I34N=Do z248Xd7?B+>!$Q5a6PsS1e0TuCGIAuz6y^#)Xv+Gr-OklA-m-j$nd5inUeSdZ*Z20feQNS+>^PfIbO7cwu_#%p>kvYTmA&*dl0{j`9XJjl5*IQs!-*Eh>caRXLnQH837!)}So6I@CU zA!oOb!8IgN@6>H%DDj=4Z+%AJcm9e_z0A_|A-XujiGY6!+PtadTNTb^upL{oM+I-Q z+>UmJ7lyr;m791aQIcGeCgl~(7KxBDng^D+ZHSgBF7DS$7BVgjvZCUljH)JyaHNo! zWNP^W=h^7~3l=MiPI-94UWWi??3|0^XQx}1kR0q-?W0G}pcTPS^uy&;N{xiJOD#e_=OeGGs4W0TcMsFg&dXfA`P2Db$W4PVoowS@y*f$PZMo>S&fpnEJ%hLP_~lusCprUtnc7jOS31W!G z7qx9YFSUP_g&~@@iU^K(3~>vc7bmsfrx35ve1NBA7_(w-aU>R-d{zpOsp4OG9n*13 zi`{S44CnVD{Hc{*%J@Ic;!}nrmsDA{E?R37hn2nW3qD&>lk$<})jgf%Q6ewU&(P(t z8bQUJ6pnp7Gh?MB-2VaH?kCRG)nCtLc4E`p@IlQX?m1D+c$LXk+=4k>x;qxCGuh&? z!#Nr7R%Z7f4VJAfz9E^}*~&lkh7=j^nzb>%jlhd9Bf|WOfXa`n{>Y1pYm~O;&P%v! zikc+>FpoGNazvGJ@Y-Zp2NpDgJ#}t&~*x>Gjb{&Hn+Wg4|iT=@ikNxW4{&S9) zEo<|Vg$JS5x}jx~96eG>>B_nsuzrIEx29Vm!k1S%eN4x$MiaP0!@0Zmd8%^o@AoFy z@u<`A+9J!QvnjUar0tIprwL-7+b@^|o>5$CxO1O)W^9w} zI_i>;ShdtE))pVw^3vXiI_~Lyl2r0bEF;;CrXl&}N#^3H@gdJ_%7E=RyU!lK>Z67~zc+<#qi%3ZQLSB%Gu z>L`3N8s*0L%bPU0&olS571!qRqQ`kp2zhZyEAzBdZkV-n@=4$&M|F69i_~gvkJqU4 zSnyeke)ASObdAYrs5i-ypop)C7W$$|d-n^vvLsrL`O75|>Vpe|1E&aQM>8QUej9%K zK@88R1U)@-EMq+$qUiW^yIXN##I;~BTB>Y9;xiMHq+LA^z_9!i&Vbk?>bYJw@H5~` z$Pq)LHRwx36VSeL+TWy|L`G5heF@P#5po@v;VXn+dO<%Crf~a+!!&d**Fw7JndJr( zZ%H-ZmG2g&!VR4MA6k;4svEa`w2yYAF0S;PrXEw15zAby({z8BqRY8IlyfyJo?5*aJ znDi4k0Yg6l)!cRH;(Ng+S8A%xhg3J{F@9?$&b3+aQ0763Ns19qMm)$0fgsluv7q~? zo+|LQ5t#Z-S?wtJUW565;el)I)t00)Jr^LGy#Qn0y)7iLw$rKT%pLV|Z`|8GS6$M> zFPo;S@L3iS21fIU9_8Ul)*vEx0A;QH5nJ z1@n67W1)VmzYz0*?1xi}am()c5;c~;e?on950_%5W7N;PyN)^W#zf^bn%^sKE^aJG4-uS z&Oc6S_u)&1F695*aU$+R%-24Y<8-21t>TsS9t|a>;MrHJwI_JqPpx$6D_`4$*k{-V zHOwM&Rw5sMqNBGb|9G{9D{>ET_T!Xx93_&30zgk!K>S$N!EGo(&oEEd^e_S|f z&F`LaB7idPT6iGlMyh3XUv4`>b3snNTn6?Tc2&UheJnYMwqac9mb<25-2C^jmmswJr45?dxE=ma8cI7-(aTD$x67Qr*;DY4YXRnH}{b z;I`y_d1V+WMMp8zZVGA}IU8R7mPw8U0?#-^3~`EV}WbG(C+cWgkzzx%l(A z2wA`-&VRlmY7Qin!4Fk2?K}M>j9Mo7nB5PFBt^Lb%Xe8BQ3?XRBPs3=3nQ0*h8^N_ zDNb~15$|<4tPF{*X*#;Hc=PR6l7byhER7x}`78OmC$Y~Yx?ToXSYSK z`d-b%A7Ay4XcBeK42ZnEr|zV{iXCcLrn6Pr(G@&ew(|q0?38D0!ch99gSsOF1e*M}g?(cQ5hwv5R zQvDoS>l{B#Fx8R-`5_;8+E#GD)#v!c^Rh`%JHa^^|GamZz5$vc2;p(= zUHp`La@$6{&d}?9VMTAK21=MYwu~ITbgk11Ddlk0(Nq_3J{5U4&j|$^3YH zIvCZ(Asu+z)RDE<1AvnX_0~#;xvFfT?hqXaZV0ej)4&euP=*n?yhydnaxzO|O9F|? zN?;2#S=oP6uGSd1`RZ3{F`S4};$~zm-hM78P<C8uJa=scJ(-nBz8*f}-+OniBX92;1hWdoiPa#Sz7Sm|UDD!&NbKV(klJ**( z^>RT!9W%6xJK9J1#p14h`yb1{8W|XJsHtk39`zZ>mX=|b$PvM~BxtIns=&*qT~YX2 zjzuxzr_}y5?ZEJl8_?dzPe!#`wvf_v$ynH0QUiO*KwpJzMC!z^pHHh6>OXH= ziK!B8@_rvq?E9h%D1d^Bpi^PV`0{q>L8;+~isv?oX`_bE{8R7R`XI;DWUr5XOaAN4TVY{3wTL(-6#Z-H7yz3(H91a3TrSD*V z8`+CDUR!h<{Q0q4ZYeXwCQt5kxtwygGYRrGP24}_MdDeKS>hSYJCU7B1od#VLO2Cy z_d`%M=7?94t9)}Awv^BJBa0SaN z`MCTZL5EWu`?Z>w3xf-0iR{{p4#${|YXJP^Yvn;KVsm9=%y zSk_jAx?`H^MWgIcaqsT3Pyv#eO_34LS~p&q=9L@#=nUvI`Ttis&85Q7Y=o}FwCr84 z3X|!q`AAfgSPI_V?0Kyu_Nl+#vZ0fuC{(i4XMZDC{JULJ8rrXSrdBRHcbEw zu}CkEA5(kbx5PT|reJA3^+rgC+xQ5UiBn2DYnJ&vvdA#B%^vPOIOtAX?83_n26yXU z`O@E%q@2c&>NZMRD7$J)xjRq`_)%&7r@lPs0NSw6SI@-9V_huvxroT3|f+$Fq$_ z(#aw4`Ne3bh;OI^!3|I{MK;h~ZGD?Lkd+ZxCCJ%K@F*aHP0nssg&eq0^OxP4;s!G5 zklNr$N4&Rp1qblzdp)U!|9ULww7Y^!32c0;_5=;ZJy?CQODFdPHLfYBZQTna4X9o^m zmSC6QSIg+<`LCib;7T}A709d`WKZi1?5l-yk}Eez@%dv`GJPr?3F!u)5X z+Xxvic1i@J@Leri_Jt855rj<984SKxNLTeiw%HNk5*_@$I*%nD80b$Q?(xMhL+i9% zifAW;5Wc{``Qs!Nlk3&PC)~aK>k-j(wC^<=t<}sTS$z9{Lbd?=#fn{b`UM>D+*lWn z+Glv_O1$r%V?SEX_j$gq-?QS6r8O3)Kj}S|Hx_p+mgI~Uk0RQmcDzQl@}|(?`RcQp zn5n=4hhP4`d)(q4OpkxPY<(*Psu8HfGPzYEjwYVm=)VMQzfH8`NI#*0Lqx&`o1D;} z={lN%sRhNV%d!)2k6Ev959sIm@EO$u8;KP>kAk-w;1hf8`!!IR-mWTllJsZg`CjFR z4$vo=NC=Jz&ILHWXJydnPY2WEiFR3ULmN|O#u=q}(KWdjG}Qg4l3`%QzCIT8z8 zY6MLnX2uu#r$&2kZJ(h9~k z;WCZ4)Gm9c(6_Vt3@4q8wWJD`!MHh7LjTPQvMVi zjJAm{A67$Qd-SwRc^6UU=f}WsIiu^b5l-ygW21!~&$iLLTBDNx3)P)w$MO$;9mC)T-a!2HTFFjK2 zw-hgiJMd*bl-)9BUjP$4H}(_zIG-Vej)*q&L!Kb|^8D@3Zr3>*TeGf)e(JtPq{mVw zSQP7wQV9p*+9lm`rNR*L$Qv4`e|&y3ICtke$R!_>6rs~>tEF9r7^InO z&@RqpJYaNu-fV|ufoVNU0rcla5|@#8vg7O%Ve?sJk9u1Ba^Y;bTQAx#jmYG)0tfPi zo)j={rds5nn#$x#dPRlVmdN;auh4=8WTpYLUs?}#A;)jc1P}CAdJ=E%Sx;US_+?I# zabwGafZS3R@p`^cp0RPvq=kcG;KL5grs-B>tp$A=h2w(#3Z7!BpAY}$I%C+GM~=Xb zu|(NTP`+wWqQv~j+vvKI5Q1)rCnCgX>$kgjXT<07EyOv$S9&F~sH^02ml%r3S?6q| z3Ui8U@9KI3oq7UN6F~n!+6qW$S2}~Ek5?inTr>Cc#y*8I<929F8(_!uwJ?8P;eG)- zo;@`1zh6HcvdZAl+*{w%P;G#Y@wXf{?f24%WDb zAYL&-#3Yf+6bO#KSQ?U`{S*A&0M%K0%rH#=$Q|6#xSwud0;Kl-AaOa}oeF66?buul z{_4y;OD>$oH_beCPF%)J)oU_CDQU&cAeMT0&%Bxc!0G13Nvn=5>B`9tew5Xa+2Uco&!OZDL_(?JpA@0Rf*^%+5A^NuaGca z=c9|9ZcUTUlIG2FiP`S*GAm{m-=@3nA72eRw?k^J7ngbNm z`dLg;gSO-6G4A2=-B+N&4$(m-kc+DcY<1fq5XueDDC)IH?00&IO%aucg+*PU+Zm}J zUDC_vWAn&`H=VMh4F2#F;v#F09LjpOhXL%GQOck!G^$&dhOY=IhxO61qR_tPCtDGO zWj=C2tBFCzbR6N?^qR!ppE?X$yf}&^i#%sU`xSLOjV>zWLJ96=DB;FLO~XjFC`jMq z{AAgC=67mjL>M$fwveM9kK;|}2g!XBiSxSofkOM^z@m5&o2Fr075DU@BVU2Yld$0} zT=Zssv8UiK;WdEDWQ|^*KGdOZe?MJf`V2OT z_>1x;72YOKg>c{;)y@7sG3~9NU<37MNu3k#^_>wGq&uOo3 zL2DOOyT(pICeV}PyFf`?ks3;MGDxFUoWiEt(r~#PUcToxk7N*FH58q`&a(PL-+K@h zSmZLnf6FQ)m$p5PIK(e8*r}xS)(W@gig%pJ!?c`EO`B}AaER|9-K(nN4J+Lt#i{ZN z^>G`dGKuYbQ>2}KaSeWpH2tuA{4j=L3zP+>l@b0S$5bRz5jieHx!VQL7JiKc8CyFr zS>^n8v9W7B4mB85YiGqc7|pQuG!o4CT*Dcgf?Y;j!$X!J5_)GlIn3MiYS{O|fYX(v zoDQBM&E9&yV%vt=_?hF*aSU~qOxI^tyKLs6a*0bXu>Wak^boom1U=36VCyZgaEugc zVltA|5YLT%>TF&8DHNjL+C*&tgmfG|Wj<0`J40Y}VX%#cNfWS;;DA{lOM&;+M2;8^2(dS0jAZf!+#qxeWcw~w zUht2+Z~9qtfWF3gJ^tXA4<=@AeYNBF#BC}bY^X%2YcGvVhBLLrM{m1?o_*`gfX@%Q zRv$};e@NUHmphoZyO5Qu3YWn+(>cYQ*%;P-82an0(Z)L)-OVJL5OV*9xC?qGbEkW= zr>ZqXG?_Tj%*0B8JHV>8By>4rZ#lg-`1hFHH%r5=DlSF)_tLB|~Cg%nd|l!vL4HI|YykF?7J z{3iVA(tXjrJIW{Tda7WZ+iGl|pv8?HDqO$wj~}zGc9)wRCi79u2*6j`YtB$EW~0DN zhkO~*MCuD$x6>w37sst{kC+wbowRnL5^-xE!H$ zTPM_n**;5*nMB06$O#VDG_U|_|N~Tk>AU|++mQy3ZT*gVz?+dO4D`Z?wkz0g<^lPYNhQ3KfC1*=!hg_x zyu-PE)&$koO?UX1gWyDvq$5QHBp=xRRyACe~fsnr}C%(puK-8|++9NX1;*yRTGMB4cJ zQ}>h>H>b|mZB>sSzwcXj3zoQQE_K*x{C62};N0U!)TjOPKUm&0#PJ7EbNIIt@Y-IS zURm$U3)logX6(%365+urQ&{r8m&wg7st!U&tuPT zyiB0wIGUFwGtK++7zbUrH158VDPL$9ls@$yXgGy{eWFVT?cDk$CQTfQKS4aCn?Jo& z$o%z$$b;uxFH?4v`t)0ApdL@yH>RhRVQ-vAoENc63!RbIW31Ro9dRv34-mQ4qX?b` zbgJ(JL>-aNmC*Ht?1z9ZG*(1xDE}@~XH!h|M?YhO`D|?-b06eu>iGwFz))Q~9kX)` zVgGtITFj5N&hWPK@tVyX#>TNb_^@*9cCe%8&L^rJ$(Llbo=-frn(YqyG`v4JYOQQ& zXl3#e#{LN|v3c~v#WQ1=qx6V?{e-Eq3yRj-%w$=Mg2Yp>3&D+cWqS!wr83*$P)iZJT^Bye{e!CrXw3QXwys20U^YHYvK?BIlVBI^$v9y}Y?l9hR5>G$`rD0A<0 z-S$wQC1Ehz<*BbM)=%aGoe>km;_sM)7~GP+^3mBy;lTy8zM0wZd7=@qQ}2e;NchLj z6%uPu%u|flFVe-5h0U zrASw@4V{00@K|$rMAJQ*#56tbeBw(MV|dUp@)fqAKIdIwm&S=ekLM>qdtP!;nc3_} zlv8swVFa5*51m5r8t$J}u3rcS?hU#~COi=joteUqWH4ySRLTvuBE$b4u_~^4=9!Ov zf3d*Yl%$Ai3HY020lk|aGy5R(f(~+NhyLR&JoCzS0-8{gEAm{>7Bd&T=)RFHFtJ~!*+u!BIQe-hq z&Ah^}>zdHMMVHPJCmPK>2^wn{+B2vcZK!ebPn(S^Cqj3ZEM}U9Ih%=PO&gjyQ3jQm zzm3dI{hO4PiB9U_f0y9da+*@mIwIn6r9Xv#2BAw&(USTQtLgYsxcLqUn$V%0cE3^2 zXGG2DLJjr^pLC^vHb>@*(Dy$zxOdSD*Lws{^28}Xr_DbZhx3)lrY|VrVu>?G7WQ)? z_YNfQ%_qKFxvz`hOdB!|Hn1+m=>QHr(0Vf`64VP^t8lusnjmIG zl5soy;F*~1Yat9)eIX~}Bo`kGivmNRCIXqWLhUXQnwLQFbRcCG9z744~ceanl3uB!m?AU0j3 zlEGmrAA%85^?P;1pkhOw$^BU6tuptwz_x!^Krp=X8e@!F=D&cYi{TjmF*Dd>GG|E~ z;HO>jhAJ)tEEay}0Yh!q&UL`TzC)M?uWjF7!fln{?JmHtVi&WM-Onr+|M=sYwUgWs z1g6I-v8Ij3>gsag3Muo}ma!jttM)c%J~J4>8uAEN*>P$d`4_ zQv#28xd2jfWe1pZ&%3*PswZE)w)qfbnb#M;&5lR0SNTsp2dLS&muLD-jjYs+5euEB zc|J_@xz&&6H8H=;MT-3ZtuO<*CRuG;W(*ism@eG}Q=b)?H=%1BQhiuGu@e#CxfSx>TIbvo5%VrW?VQoFl`lo*FkT0D+$^2 zjU47Z0%bW(3=kCkQX7qg{-cIfA*W;l61wm@6wNEwI5k}4>A{OPv@F8<)wk&`JdDBF z&ku)?b}}$|hpYoNWV1U9z)y&ht<-5v=MyUe{s^|D@VeDvkG1R&RzDxPzJsf2WdPsu z>$znJ%!o_Iqv916jSjSW&;)R3hcG|i0ji!{;pMSs5plDk(q*NJFZ6k%Vuyc_IdDGf z>HC0uZhxzb%ACL#TC<;ZDN9~##^*_uiftQg>ZP@j?cFxp(vdfFhQl`6KvF7vYm8<~ zdSW#e70=33npv~ynij{PDuQdbHF2Lu^E=b4ew)jz#_L)U;@U=gawhT?^D@WTJ3+R= z-~)BYt~jA*3V(&RgV1e^Eo_uBm;nyzufy%aRYrhQ?uB+n!_{#I173}8 zCwwprAJskq6x`8p`lPEn#IFvZO~lUxeGjzO2h1*O!_b1%KAFk~0e|@?=1LE*fWIYH zKVIZ0Q4iDrA+Zfdd7vj^WI0%4Qy4H(deFoBlItIv*J{Iz{memP;uM|vB@s!>!bQVE zDl0%ms#wix)H7y;9*+#(>{Sc+d;&CDd6rXw*xg#@1jua6XHv#jfXcEpLA4(vDW!8a?%(s<_W>UgD8rmHf^zQ(kk+oAZFc~u9Y!uFg$kydIkkqJ zSeGy0cNZLGZ|eUk&c51`|Y42jbB8P9FaSW9Z^yWcO;O3AgGOHpS-6Zh}1d zZ`Lq-ljZy6en`9WD=KvE?*x9ha?=5G({Oa!OVY;2_~_j)g7W$&c`%@Xq-d9SiYnj= z_+0j$K=It)RfdHR%?rg}#&-$ws+iyo{j0a@iyMv%Pv=pYE*(t$3%SdsF@7JH4+!hX z?9H`2ke_Xir9nIQikJVo*UW?qy|}jokvGsxGZVejHAn!!r(4-h{&rN8MVh7*o~Wrw zL4oQsN!iWZI4fmVkJqVle#db0bmY-5umkR4DnSaOqKT5f9F=!C6T4FZS7ZYgvnz-X zpbSCk0UfBRU`lyHJz~Z4y0?mQgC%(S@BJ>83v@07o?Nr^mx4L6DIGsjgxJwC>K*#> zwM-^W^p*ALsiK9>oPSf@Etnpv7tLE_d7$4uDQX+acRWY>UQ*db(8rSETxwjacDw$M zxtqJZM23IjGg%rfDMVZ3bbj?Cr;ZFu?@>AZ_EfnKU-XY#Q+_o;$%Kgcn#jiqK_c+rypr{rjT~*Rr|?e!j*B@4cc&4meds^Cpl) zvJ{?NnsR3=_wWGNThEF-%aK3NdC^&`Mn@*gfH_E+MEqja_|z#oKde8xatgrd7P^fB z^}NqL>2m|U%APH#O0G^8cLB|a@&tx-o>z+A`jjC9+s0RQC0GNXr?Ni7k!EAOqP%RL z3c=F;!O9T%O%*LWaU9WIIgnd&Z=V_Fd!qP^{z13FbP|H`&5>Wczh4d6wZCedG(8gy zXF>0J$d`#JL&p5PN#3*5*2>l}@>3(LzV5-II<0(z{ zOyk-h#X#$nTcDb!|G4}0VGK&6 z2C`Ez_up~k(Rbq>-;B8-oj`{19Lw$5I!smHTa|l0RPHv7e(6hRVAvs8$cKJD5(slB zCk%vB$s@;|{7V3n%l+cJun10K9p9hPZ#IA3e$_@P>{Zjsnv%VP$sE6Xfufk7zrKcCNs*9D`HQ_ZzbVr@+KMLSKh;0(OLRq zQDN1lG`?$$|3U!WSW-pYmdgP5k>OhjE;Vghf+=WB1@Wym#~L$WZtBTL>`WmRSjyaP zx*Wmog^p`VvTzy6zeU3ex#pNz_g?gO1}Uh9s`O&>f)9ch{s2GqoLow*QMB3$IWg+I zg+Lkl=~RmrlQe0mnUZPBKXP>cO4a+7eMHMg1($@<__V=;yn&+XxAZ8mgtlHhgsPd1 zAK+#yB>UyR$^5%1;AO?W_(f3t?8N1n8p8YNGXo1&ar5LL)VYn=c`^Zvi>(hIG{?1@ zSD0F#m(@*@IuHq;v3(C+P<~}-nMB67#~Fyxl@6lg47HjIURP_@H*yZ@TmXkAy7SHt z#G;!y!0A+PeyOAKzS^^@$R*$9!}ph8g!@dNR}~{{b2)aiqz6dl1>95%xme>ZRMRYE zD@vG(o0AzJ4!888{v??agD~sf6oJH9rDu4u5})8uIty1%d*#>+taq}eQ>N1S~xDuM9OWq=)zZ%F7cD555zuby}2swfind zY@Q^iu(nlzF%F0_8F29GVxQ~|s|k3r*)D&~A+=Nv5BdNDHfZowjTwu87b_u&=o{=$ z-u3y;HHXDv)j8zmu;9VH{JL)0&DXFRmtMI0VmqiJX|_4_DE?Fv^8?Z!XN-veU%9<+ ze(J;WU2Q{SZHwpUzW5KI>`lw2^Z6*RonNfo3@L2kXltN0J&4@6$^Q=7jq$GyOC2!< ztri_1s7p+;A~Y*bs`*_Y)Mk;RY>GqBii?QFsZv+AxxPqy13Q3Or@ zKe0M?MQ>g`O5Y{b|6cT!eL~>b2Vr4i9#7jBPc=#HR%zKca!2oiA`H<(`C~T(e?@Ox zI}pH+1i<@>#&)(n#e$QA7w#fRtKI8E#~m)gw&dGn=&f(f|L77uiHAgA zO4TjjWr4_$v@iTAU6$p`fWDs<`Z;x&Ul>T(7MBykRfP8SI|nyBnY*ZKoZjA@b>Wc} zWCxsUvv)Y(#XHN2^0MzSy)-wBjk^13_4afRm}w^Pui`rc5~=SNq;0}MkG=R!%JPEK zfz5N2mp9ory4%Ht>KvZrgg}&PoSIhSdB7mH-oSMnZdQo|af|{xK|u{odgso*_$EF| zR2%~f-QzfQ_iSgl*jACuv#MA=7&c(1R=_qt%~dsSf&J=$9W1kAd8**+q2{|9pDsYB z7I)_x{uZn4pD=~AE>k93r)zAt89px9)I3R#4OotwF4Cw~sWsQ%s;iO) zN*efBST!zheXXat%ZCyXbVT2hLWu-1ubghA`+pC@M%wuX0dhr z>hlrH2reObU%Uv1sf)m?cqn|1-mNFylo-CR&jjfgP`5%l!@}}z39#LBkvy$2_jmI)DxC&hID)X|A9QE<@#5~x|-eF3o-G39SayNq& z?Apu-)#=Rr=k_aFI+_2UOLg-AxsrL(kpR)2X41_rCw=YR=ZmQ=D#_f&mP-CROId>K z*uT1p@03rCChPQN|Ex%l!p;(5ww9Kn@85sJytwgL0~=2)W;uV|0GH!CY+xmEne~BI zSiM$|)H_kx@@I4^BH1LlCN^YTxBl@~02$oEB3`IOSw!Q!f};NBRv;vGBUZjq{RF?a zk1y_i?fuXi^hWw~RJjK`Ywx+MzzZikFmGHS7tpY0&tE6y}IJfqopqH4|z9pLqSWHn-VucU*bzl}9} zoCbbCcTa{aQu;ZlCVaLt+=tC93huwTK1O%aCFaiScESBK-QGa`o9K2TY_2EN4|^MT zav-gfL6{bO|7SwR2)RbwC30_2|COe|X1wI0!&xEAtH=D}(dXBC9l=4HqOkW5y$JauB|*Uq z`Ujmh=tX|o)7klxs*_JtA>p6-ot(?+_OI_uBO@eu`hRXQhExa)MEgp8M!CY>j?B7B z-(Pfq8%s^=@)%_f9iA;Q=?t2@Y7q$LmC!%xj{f`zw6a)OPZM3Sa`y?mC*FpR^+u?; zKzs{&shv5F%W};uAUq5~bV@tt@)IGb21)uhoqjgkbUE7}p3<$QpCeG*)5c?c_E^W| z;77PY9JwVAj8{OqF88BYOX|?q`14ks0zY%EBZLI6LH#yHm)rRN}}I(&;+W z$J$GPsEKkIHVkHCnP+D2gbAauwNLDsZ0pw%wB$EQE>NcTx0G(4j}IuD_?2|&Sq*kW zu+`2uC3~^Ww0qA9xrq2y?bz#j z;J;4pCF$ZmGfE%I82XF`(%;t&r>i+mkC?yy4IPIGdWt_&zF__wOcN5-&eXz++1CFF zu2WH@*vg2S+n^th(u{pVW2ezH!x@#g&`Hr;v=lih@Rd3J`s7P@!Cq!$?%SMt`g(!t zp=g2K(j!zKMajXBcD1@JQ)GeI%A7#4kWKxp`p6D18#6LC^oJGhp%aGTN|SS8BYsuG zWc9<`Hg6(x`uE1+q7)~0tt81N$C1imR)ZOZrrG4_4gKEK0+Tw1T*3~)F zNs(EW8?bY9a`!U0Zw3~b+Q5(goUXrMXGTVkS>ZHM3sT#K# zep!D%7JNs@*qI3A@*(0MW5y@NA3tMsFZH{b=MB1z!7AqbU~fb2>XoKhr2TQhY&V^Q zwy@CK^XYqJxuETPP|1r8tctdDUSYhg`xjd!*b`nmI)=ELyktk*Z!3Rq*UtQ6Zz*z?s@Oqk{Ex4`x*g=B9vByZ{TES)wK-o5W0x|u zp3^U4tu38#%45wC!+Jg#qBl=#>*NB{HSE0P_VcFNFI`r`t?*#JfjsdW*Iej8F2RNc zm4ffrIUV73*l!H%SGk9i|cVfS8%byyEJ*>SJ=;82OAE7*g)(KE2#v6|Y{uKu`Q4<4r4+qL8#>?beh z`cNH7sq4RSzPP3t>3n))@o)W2&0lAWvMCb+jk`7XVS*=~qdP;tX`oafD4jJRZN%O5 zG`}ub{}mpD4ovDcn`;hFMkm4Bn6r7(vFtBXvPSdoUL`U8J%_r89r93%Jq~7QT70~0 z&_C6Qo=C1V27s(q8*h30#iRn^J45@>~HF-8w++P26>l{^$Qje-A9y z+oSXZJAEJvsvQFI$Dc0&d#yb{cM%D{mi@ClIE!Ao6%+o*j0Ncu>|m(fajXXM{~6lu zbZ3CkY|FL9LO2`PO@?yiS|CuqCY?DLDz&THAa1 z^M^qX4{!9;s6n(@$M_v0R(Oo8ztGQ%aAUqjwRP4jxeRK>CB=$GUb79MmRX_s76Yt? zoIM+wm%W*_MMcg)pgOlY8lfb22<3_sL9|)480XCa>X*D*zIU8frtbvF*Te>L%y#Vy z?*Yrkh93s-^Z|X+S24?Gq;a&L`t8l!;u1VoT;giqt1z2(j{7-C?M_|W5e}m0&~JQV zXydWqM=m8?j&bW?%h2;*-$xpt=N`hK1N-&mi)V)94We#dpEKQ#7j`n9drSr`X(Wd& zhw#u5GxQ>>|32843os?#bT&>V?Ln$GrjsPXG;W$Op5Xpa)O+6BA(zSl&GFdLQG3Ze zn=h`Cx^{nm1{G)+m2?x-n`6H0!wea?J- z#UplZNV87D{8GGj_v_f#bM<9PVT4mlFqap&9V zq%z=z2MF<&*MCG(>f4$9N0J{c^{e(Xehiz-^>}~3Tvt_O*z%YKt*x9qf3SEcp_J&X&IXxg&EAfhed>T3u5a~}d!vWycU z8w~374R+6oh@mE=964Y8klv@iKKkf{rxmTt?z$v1Hwo9K+rAV#V- zNyXdOgYb9G-qHBH5hU%Yw~!DXGkRCOrCFQ!qqoM>_G=L@*^d5k-ug?X>Dr@h4cIlU z8Q$CH4?4e?ba)UU)gZ!0OIPuG&;A;e@Sj03Mn8JnIlLXC1KDkDPIM09lfiK>7Xo{PTpt^%H)&fwp7zqezLi_`-C#9z=8XeI z)0kdBvzd0Y-J=X&#pk5An?49Q5|=#K#I}dS*tM2lZHg)!F1YY^Mf&wlxP z%V3g({V^rp+xokT;wz?2Yh-F#AB@JM7jEMr@*ZaFY`5fHI^7?2_DbWm1?ld;v%rQM z^!pn{F3}8|ai^x3EwC|JK6r1?JZ0Yd(AMa&?l13X_rku#DKhTXHjO%$-fDNuLv7pz zl%B_r_LCn=&4ID?{9Nnf=2Skgcd3nxmZNjm>(o=)L|B^S@K?%U_FF&8DaTyRNnj+&BnBo=7M;>yrNwHkg z=c6pUV}ewW##SMutr(&U^UD61mj`={Rncl72etU(9-1qPhE=Qj58tp^8e~|^Pv?8` z4*G-CYU!0+_k2rgIgVa`l{4-0Bl8{fcPIB@I^nNRY{QKYrq0MpDwaATvEf0y8TKid zo51^y$GqD~A|!`2le&55hBWv7<=?Fi=KF^uK|xJQ7L;{Zqu^@Q`L6c~2+O4kewq)= zwR!%pYdeHXu4TeQ$GFa{PfLTw!j(L1$00?64Kr5p4n27oxcXTfa;~@L4kY^wOZ>n+ zq&eey1dJUd{vMd<(1B1Qr=*~YDnJQO5&u4jRd!UVk74j?0oFeBH*xA0^BtIP5D~w6 z&g%3Pjj+_ZF{c}qCN!Qtm=5(Y|G(?DphNamP4B#&?BC-U_(}Q?<&g>Hw&*Z)s$8(_ zr7BekK+afV0)BXIbH#aUpM7S?pjGyDzm92%@g_ih8WFW|nj@j?^1QHLP%vfEE0`7UG5k{A+IP(Qp*2Q;RPh&fM^u>6Tcnb5OkA z)+|UF8t1LaW0c3@{sx{~t!lL0H~a9=`|;=>%Q2&Uwo8@uwzS6ML?YrMb4`*Dv*!cJ zvOUX6oLa7*0*Uv;GSB)>O5pLN@T$44t3s~2aIr;i6hFtc#lzZU&>YXb%6(bE0jyIzQ#d*-(y#7*m2QFGPU>Oo^ zODwskmsc9+{(!5VuII+ibC~JHc>2rzw=Wf+@G^AfPegV#QmAzy9CEPolC)@ArvWW z{I7b0swumCO^sbK?kAUWTVB*>f-D*$1@CUom-__Fb z>ErU1{!4V3Hi<$4G>zm-%->x)ekg_%4J4psO3KbH^Ep_LS7-Fy&UbvUvlBX8ep z%Hnty44NyCpO$FOcb-MllVe+w;~!Z6C7Qv~4(3CKmcmHIeQ}=TE8P^j&^ASO{%4pa z%oBeiL=9rw1xbm}v56hfvWcPkMAERN;Ju;Wdy9QJ1}?zX|9&_; zn$@N4&aRo-skOD(ZkxU8);*JB&`=EXRZ&Vo=0*W1hX1M`51X~5Afb+#`W(B~xI-2d=o)+XT8-Qu*Q%0YRN zKuPZG?Ag8~|6*W8i5JczI3E0T<#2{^3G$Gd+nW-MYj=U!{K@Owg!ftsY@ar=cC|9l z?!cXBG=1-utPQ~d_BI2@k{<0e^MeQjam!YFF#QpbFzHb~swwL*ak=bpK_{YGfrGwV z8_mSK`}nZ2|6~8^oe*61Eb#a>k7dP5X#zy3It&`j7&>pJVODg=#0j~UK!B)k z>f>wc=-mkH4MEV#KR2R8bzvY&u6kXxyX1m&`4=MTZEfy)6U~a+IaFUVqaQJUTs*nt zSiKb3sG@W_*ifDmBUdM2T_7lAfz*P2V~Bh1sA_^kwF}eGKitC}Z=y>gCK=g`R=%T? z{65_+i77Wlgk0q^xXF4erp}FXIdr5m!&26jctC=+SZ_dJDy)#8g@SGSPZGTm@t?}O1y+x&B)xCd8aSsM0X zX5U5MiPi=;*$;{QJ*%!ApO*+;TktSpW*r5N%8TvDyJKtS{1lNe_L9^}FktxEn)JBM zjNIyYn{wBw8oK>(_St-0^rsxbm8Xa<>;K;ED0`;)y4@2hb0O=KE#4-_f3t(TUW<=ncQw>f}E zWO(f2_V={%Q%p1@k8k~dSpZnCl2md3dX?rE`5P5dNS)9q&E{z+06#Sir;j?h5O;y= zpd0kDQI7Bb4hyIYMQ85$;qS$mvUy&w>u!H>I*4FS|16P`CNlJ9FgV}5#2hC>?6_Q5 zK4ACmfWmA}-GVfka2K_tTemMxk;vSFD3oHwpM{6strwbyyr2G<10y_R&(Abp>AjTu>NYJUUj4@mAqOCHJrMw!C`OG5JL?;C;iT7*T;td6~h(r4t# zuhmGv7}NBLV%scWcH&d90)B2=3S-y=4JA%FvYx zDZVa>Cs;n_H!=nsxN`*6!VtbYJqbHg%2$u!y+dm|(>p?9HAyW`z`H^ zIZ78V`<*y`vjt(GOJ%yxX6;fE@PruYD2M&Irh_P_kMMJSd?q@WuR>2ma9*FLh~(l1 z-;u+D`L3OT{klheU_((E*AlK+&sWf2zcao6H0JNT-#*D~Rkl^TW{rGy*(axj`7K;G zU*;SZ4$^DnhQ!wfud9I!sW;Z^%-@lx=7@I#Tci*r6F2Fi@ zG1F}Jo~tw;^FWRTpjqPrf>eVKWqsNyG;PQZ+nclmp(<`{88wO(Sibq@fmFon*o7SWc2XoIYsB@j+qBtSJnLzIeQ<^ z2RqOPAez+{_s44{CEvQJXPXp4v9*w?yXT2~KS?vibZw;K(2{RZIHzhs!&P!tLf&tT z0>D12;(j8x2D@I&whPulr4h4vaqPQJ3E%3%thRje25A<9F289mm!MKTgE^Eo?mHAw zrhZ8$7XEErlcK!ONA>NBQ02mU&wH=@k#X(d;@h!cX?(Fv`g{$6?zG zj?&QaI_hBL6N0QNre^Ppaws8;mF&e2LcFpEOD;;{^GKKxMZrRbrw03oS>o#ID z(UQ!M4ae97cD5keSCx9!D2>~%);0=G)urMu0e6JISe#FNI3}=>RqQhS6PdZ}ck_dx z9#Z9+$NqCOC5=j{WvyOQ@4_k{)44=a4xFgfuL-%w^uxg9Raen@hEFaL^h0|sLOa{4Fm zy&1!;(i8aAFAnLHk#r(Q_y&PCOr7m%IB0out->SdcahKL7pH4rXJfx3-*rNJgQ1sy zV~FmxwpxP4&+Mv{*3ml9_1a)W`a!#7Jr_9sn4w`qX>9{jNT`bp0)yu>^6O zV=$89pnFLq##>$1)gtWktCexuv)%xETuvc#T6YI`$VYza=<8i{OwIX8Keebm<<-Av z%53IV>Ar6{1WGaf?7mMzbd^vH#)4|~yblw3>o%A&t0=Oe+K&UpkEb`A&f24uQ6KKe zqXMz0=}y)rIck`#Rhf8bK4n)}Uo+!;86tVEC<0tI#H-)V+_+-E)`?Le@l>xM^_W^U zK2}0jekyUezM9hQboiz@#(hHtQ?&0aU~ZhPklI$iW0&>Mg-O;E>5ZKZN8XWAUO&3) zIbLC@jE%S5YQjH#@|W%>)wD~w#V@Vzq{>v4Y7X<7iAp4Xf!NC3MT(-XXowK^%h9&t zdeWhp+;mZg(P&Z@OUB#a%MUo}vFEy1WJhj2)M!#nASr%D7Knm)e!J8kcb!wliW(I) z{MmSuh2Jc1JfF4k9N+sZ8LDFOHA9s)#X>4euYWPr-=<9++84nfb3mbqvnUpGLX`ru z?9RTG%Fyz4HQ-OcfM9ZAwKR~Uw@ir&DUmB`(|vK@7X8eE@IH^cDH_0pO|#8|%BIec zBsy{sT&^?TE;O6enZ#HrWSnzL(kcj|k+g~)kY-|mfn-FyE@S0-BS zd}EqW4I%kc9*BFe1)*LJ{*egDe6{}(L4@Q`_s)*De4FS`?^fa!qR>uf|q0M|K?pA+qn|?mD>d+`Ln2(|->Im`dI{Fu7E~x*S$$H#y1g14O6;w0(PB@h$qh;YCR(t

i?ZA{xUrIvjv#Ne)<1a)Tc=&%az)2uZ$!ECa6Dd9xZnSTf{u6 zqO!?Y9<^rv-8_vvV0IYm0e8I}WogK)u!S#>csSf=RL@~)C1TPAtex`fmWm6ita3~2 zGfQ^M)*f2!GA-XMC?)1%1OD(w(hVi@X^ymHQb0-k{fuz!F43?rVMCF&dBf5cS{^d* z6*fy024f9Qqfb|R0zYiMbF_LUWe z9#6;W$t~r7T%8RT@o$byB_kCQ1@`eVpMT3e_hu&0L0!*S&F(sF6(k_NUAfq0Ib)4y zl0S{aw`ne**4;GcgU1ekY(dREV~@*zVXy6uG4C18&$V3hkB>Zg?@I9lMIbTI{!_XiID4L$Fkl13H$fZ@eLCVjrt|Aquf z+|G?Z6ymSncuXAO{l?4cE~7i;5XpVdOy_^{JvU^PZU&R^!1$?~5TopKxf;nw2g<4P zzDHEhTFO4E;4m6hIiJ;7R&9(u%h7>X0^B4o#H|;Y%U-89QsL^MguruJs5;bD{n=xd zt}SKyrcml&+L={zI#kZYpN$T32i_se6%rlSy4~)<-J^=omA8LlrDd#tT3ZPU0IN## zWHvd!Q~wziP?0<(7JjcY(bwu7k*R;B)?zVsP9i<`9pCMr!~5H9q)U6e>JfNHRhX*J z_kRQk#Yf>fNrg;T&nuFFtma~BUnYehkYf~`?D>wzUn<9znpS0bvpJR2dZ!rX^r;KL z2#$X$pVVU!{iT2@SV%)w$Gfp)TUSs-H)2AUP{vw1$3=N3DkB!!3QFOSwk^zN5}TJ}d{>Ivf9<>F~SCr)la zvnIc{#*!EQY<-(TKA|C{)L;%P(tnghI7(Wq-8S3c)+B&FRy!8Z?LK47*r4K^g%!rA zb^4Gn++bk& zCT=~snd)-r3m|0nE)wT=FlA46Qs6)A72kX%T-mq8FfisJOEoIL>`6O3JQ4FPRAwO} z4xM!8;}V8%F}RbG%?o#aviREQ(D{Va2@Ivk9REN=xhf+1M`}C+opkw^Go|?*b;zl$ zP^7^!)v!h#StQdrSxmuKT76qSU*O%KPWuK#c0@YkJ(s89#4`Z-r1;K8xio=IwrN*g z_x^|xAW%k1#j5nH;Vj6vm-S8`AZAK<)9=e4De}D&p|dtb+y{x%d418BFYM5Iv%&pn z4Iq4Y9ohNz0ItY|&PmkE?D~vG58{Whl**eUgyI~U?Rqdr?Qrkjk5BiV(Q%A)1iFs~ zMHV#K8OSWZxe{q^Lrmi|9~J^2i>P;IZ0NO)I=4hRr}ft_S1StV7K`9>p5MX8^=7K^ zCpQTXZ(-t1(cjcmHTC@kYFz=gCS{6K^1KyxYiyVf1`)m0jTjjQ#c)o$7( zRd}Lls=@s7;?>?xSgy{A;XO{)8)8sS?E}eGBvghFD(Sc`E>zq3`p3G?T;uWAQmc_nhR}SEH4Qn2ZVkV5KCtWW zRZ;GkZYOJ=Z|886_a!W2qEEQ@!}M>Si1R)dAAwZ9X4ZDTJ?6Z4D+XrM;V{uF+>a&B z|79Ec`9KK@javgwi7~xE>(;q<)c=GqL!oQET~zZ3Q672L^Z&i5@YgK2Qi11$JMcJ8 zf&P+!3zQQ~b-S6+sCTwa_5G#zk%-2|$@(qX@kGW}AaL%YO8j7m)U`EEo)P$Besblm zDOU}X;PJ8H+|%kcPN}}H6wXTv`8>#6p^ub_=h%?9oZSVl1cFZ6B)0h1=9k5zJ{=vGMV(3}vUU`tpE>%f>*gCMw(w zW(UnY0sIKud-@0G2OS&72QB7k`?@$kA0BPX3|A%$1;bVyS+hG+b=()CEdO9}@t3dD zf;+#yrAvgRBTo-%mM5!ehEu|HM@{biZxLYWv8Rgp`q=<^h)MaIlwf7k4idju(?6hv zeRbRj$xmw5g7)=I37WN>d}pd{pHuUmjr8tTt;F;^{B?Kv(Ecno5Mf{9Nm_Kzn5rs8 zl7uM@-uK58BIVa4@(F%qu2k1M#K8xT#iPWqnAt%$-20yG+nOK#-YPkPOK9%&4Siv1 ziU8m$zvgCQs90!Oo`8WAX@j9=Vq{*M@T(pDyhvTa* zxoa^tTi+Aj8_&#|CbFpH?TAtR-HG>A-O&$N67(lIENU|iPLf0SLyic02p zB;5^Pv*Y5DV5~wGTXIV$J225v^a6^Ca zJ32ELsBpznl9puym$Uz|N{ z@(O>Ni3BEp3y2Nrhe^T;CjY)Z*-IjWqsX>Xn^)OiLU4fu+ubuDk<5bE4EU91=(1#X z4}|Sg$RnK6sKxy6^7%J2yOPRFWhK;EV@~TP8;To#Kaa-USaP+lG*A6v?)*$Ey_8~O zq)zhj3yQ{NXPYc!J_A@+zT%b>z@i>DU0pQo8i#^+RsCDkF!xU%ddEdi7Iy8*0Hw=n zy5}R;L|SjuMu?>k?{jCr-Bx}%=l2_ct_@v+l>Y(jHDb~Ebho+S-T!}M5NjT?#nwXO z*zz7z;&K6%@P!@NA-p$vA6aS0`idmn`3>#2h{12<^4rFLBS3BQIkjnJm2|W}ADM5e z8l+nC!FthKyul>0y&P=_V40dM{wwJY>|zEk7R z-8J&f0db|fKaP9>=Wg~3pn)*$dLSD!@Sl-0s>&~)aq;J>@;f&T`0WQQ_dogWrtd^& ztBnF8$RB|G27vXCnG-FCL{ImvL?}R>QK(6IQfjRIp|dr*r}Homq5r()3;7=+Kebe` zp#naR7Govkk;MnO2*OL{voOM<>wXJ0-r`Yiy&Bn`3ouw1)cLSS7 zWXra_GN~W^MjPaEw1B$~6)-6emFx{$A|{F?0d;n`bO_!5*QufR|MS$ad3)#8x6!u< z00R7Krx-4aI$reEUx~4zDwxO*4-jQyAIdMn;I-ow{ z0>m1OwjBiv4viedv;hNia4NIk=~%VC_6v`@VLJ32pc`v6$!=~AZ`*pIlld%D2akOb zY^)9*M+Na(X_9FCJ^L1*1<+xC>w{7A{N7#kqO?_TdHp_+1vgWt+M5X!Y|aq?J(eL} zz~S9o)}_C4yJ2}#xGw>er2~X z3$^?q+Amu>TaO<5w?5?_vNxYT+5=%pc_om)J)^oI^J#E}f6&zMRHTwc-6(DMGEb_F z5A4Y_?$S5j!#JlnR^feJrMB#@V!KF_YJM*>-vHY)lm7AW;I|<@Tx*BvhVh5pBw#5w zhB=3&g7&a_Cp+zyDZNbo1XKL2%p^tuy2Kp-{=6_P+@e!i{yx>~-!^e@bu+bSw2eMk z$)mwz+u?NLwJo&VH_fG3=phGVk_SGn9flY~nIA5kjUt+fjRoUDJ8}5~Ni>r1%Bc!P zHJES>%(X_prP7wEp^ktC={b4~tIdDF(~MJfnbEzO~9B$zTam-mMP#2-_omXbNV5K04F ze-kcSgqicInf<*dXnDqgyBR0_S$1J4BG&> zi`-CMX6uefKup12RdTD7w8;KieqBhLNb<iyOZiWOGgxl{>aNJ5R)Oa7A3CDVC>Xn- zv*-@`sN7kXcLlJG`8=(L-s*H9U}Fqt9VPN|mZ;;{fEJ#0!?iNurv87(a+))sZ}1n- z0Vn|jNy9St<^*!*-CV0MAEt!{@qe_k2?-uXv z8destNbArSKu`5gei`nW0N_E-UPN$;J4oXJ%uF1IOkQ--6&`h?%@@+on0SRp8(R2q zLwe&WBq`ZM0~YK(5D;TxKu#eRqFfaBDO@ol95=euQ1vh`s zJX`(?GcP%TGrZbF7Hh253EnS#m&i)-h-8#l74y|_@hQs!p)^h~ql|wpL;Ih3%NCCf z=gf^d(-t0Uhf!XiRcr5X80pBbUDIuj=%kKD(D7e~r7nee%>|!1c1L)Kfd-r`xOb(D zEnh>TUL*H16wvn1-C?U*?a(6q{P%Ys=>8rs4<8z`QC2360o|bH#M>{2J^5a;ZuEBo zDGZICe5*47*E4@{g0Th8`a23U)?@aLH*j4~r9VHhNsJf10m2i?&E0*0lVh<nigrT7p8|rBbsEV6hB9KEj z9;BN{^JeYhQjzXkvG3)bWp{`LVClpZU(Xp&6$WK_&O4-E`$tY*v&)M6XbSq?6F*1^ zAB?VOSXCC_d>ep)1uPhLPO%0qdVUd(FPCzlVSb&}H=bx0N&5aLnUtP&)!V!az+zY* zTs&cy0qGn1g$D3byCLmv9gPPSHq{K@C$n9>^Yq1jt7U)F7!~zhR|i$Koyb~0R6p*0+bLue(waqX~uH=V=2XT z9_TPa9SVjq&g}_C??(Vy_=cmwbCMF6h3ZN8 z%T$RUUyMo$$A7#rF@LuOrZM(~@W|sULPS2F+zu~ehLuPvave9$?t_8HXj>(892|;c zshh!_I9~nB=Z>`U<&@hf*I)dvqz^;>~IiYyco{NX=(Ej-O$5ut(0gGCmPgs4*Feh;?cQO29|C zYYmGU$KV=Hg9R2(ztF#L0o)GLiyCu|6+!rbDiLl=4d!Mw%2fE8C zpQm7~0Rjm^YiZ8!0&q1~rJDclG{Oza#|=@F&PdOizI%`90y9&S@fmcfzYEwbkj<63 zc$65DYrtzrb_}7sA8oWZ=d%-ij(~e&rKZarWn70kAhvfjWYi>1*cbkWCNyd?@T`P^ zYzs{xPon{SnoAX#h3MTZ6u|W%{})`JNZVYcm%PBEtIavfOtj5*cFR+dU^C=eSnhby zUbaVq0F7qN< zgIT>{ZrEiS!$Xx|3^Qk<%SGs#ls=I@2*j7uFH$Lfw*-WS*eg?WM{eJ)gKn%%)Fl{KQhX2OsN$$vWe6s5O)by1UtQ6|msUt+uK%Q#OOzEV! zhXT-JY3vt|ElmthaL6O>H}W%d;auW;8)Mx_SZY4YFCyPI?Cdh(4Fw$XOK_aq)D-x| zldzT)0;H;)t}oOCA54*179yCwS>*{~_<{_QH5)aJ-H- zvABtBE8}6sDQC+6jCKNQk1~@I7fYQ!Nd~pE2~$rtFKh@n=K!8cy|P`C9dVQns#HTi zEeMx4E`jMVRR06h$+4KER1A~OB+s+EviUUR!8n@N@Ko#b1BUPQHT>Q9~W7T1@M zTcEIK3F#dA(zoOnC(cagJhnFM1{mD+t) zmJQu3VL4rvGCADjD-DWEfLBTAdXI=`@3l|bIKx5nDTnEokDoHjdQx2P=I1Hw zzlSMv_9QH|vGWkRo6OO_BYwtLe@6sU^5z*PGLu)T?PbVtnE1*O74(?WD8f1@Y+yoo z-qbV^)d9!?cYVX2YPG!-@|;(Cugf&_N04?e?R8v_)>VSosA80LTrfLvG_T>jOWbt{ z0>qTP1EGSb@$T$=Issw5-6OT|>Ulpkz*v;Uvs^dC1Bua+(0V8QYOFp}YxO%TOuTeN z3N5dNN)SA(Glg<>i!@a)Q`-=?4d7+*tQjT^OzeVwkH>uBcq* z%XpFpNhc=@s{mtQPm;5)p3){xp$3KSCzi z0&Mi)<7+36M+;F6(X=wbg3_E&;%d&~y##3}sz1)HB8r#WC|+!9cYqgb`G9inwf8EC zWy22U?*3w%oHXTYDd!vDM$>Vqz=Hg?8597-tF-Qqvu6+~FB-u@{%n(~>tD*c@BvT(M;2s!Y`ZvW)Sn7(u0>rRD~Tp57j zf9wEj2|EA=LJg*8Qn?qdZigE5vWqGQ6O1Vs5SkZMiI2HBL13YfqRgcjCybz79`W7nL|hWukdwL`FUR zy8w#~FZ{Q}(W>FYaYZ^y;1hI_My{(+N^cO}-_dv5nE7?t=Az8@e4kdQV=M=ezfKrN z6i(wG&vKY@HzaX}nV9W*^esPapm`V35RTO>BaOCpTOn;ZIUfJ-*XxUKY^Hqk zV@Q97oZ;Vt^@>h(!+P>wFEbNo7@%GaeZ2bEMATxNnb5+Uu*?8S+xA@vIX4=J!S^EN z!b~*)j9^(m{|<+Xy#acB;r>!rx6IO!Xh^d6kinoc5a}S|6(|lpbC?815T=QCTXoF6 zkb>YmRffV>CecOQ-H@!=O6-z}g|1RxTxCtOp^ZZ}^@yMT{ZPNHS#s&U-=5Ivr;X~d zuB8k5AF#zgSrkFu!*nU>oQ|zMq)YNtNt_+@oIV@am0Y6E`O>7Z6W;i{R{atLB^Z|+ zCs?>hQ;+{(Z`R-`Uzd*2c~f?wT^M8dhN7s(bVF1!FDQR~uQi_`=6))nF@HV$VC`w) z7>C-9^t1w0DV?e8T)$vI4vpk)i%c2!%$?ODRGbj#5rQ#I2zZ(Jc|Gqb=n2C0!`Qrh zlr<1BK)Lavii69ON>tdDc6p3P*_#RC*0(V7S$!U}p};VEt#i)9>+VT1DF~3~1r@Ub z7JrOf2t-~fd2-Dr1`Hu0Syci??<7LEHb({6g6>x5JB6&2O90Q`xF70RkGi4#!6ut~+eCt{^ z@~z2z&!FB25tZfD@xm#)72#j5Fq!@VHHlyf9VzC-)$AKp=2w3yn2Fgks+*0}S^~5? zr?jC@Cm>0mPyZBjWQ}F2dTknCN*HNjD{C{A{4=c3)G8n`t8@1^Gb85Z$jIXb$IQG> z{0sG?dbksL_%mwC!@28e%gK%5Xe%k=C4ca&jSBp^YwCCjzd+k}B0S4$A%@(J5@i9r z!+)Mc{bRq{rhM&M3SpW%Zeq(Xu$%824zZr*5Qr)=TOJiUCobE(ktU|rs`{P&R&>b) z3+g@9`!RAYPBiU+&^5lu$Yxg4pSyoggD&BkB4MyQ#(aPz;H-nVo#yspC;6?aetHM~fSCtBCSUP@?w&X%86Czan;oQ)`yO?gB;&*w& zhrB;!b4Hj|g)STsk`UETkrD``e}Z3u3SMjwO)IUjd0>P8wMf3Bwu3^T2} z8yg`NJ97zjNqMXix&r+;1UIOwpYj>HF;D%bEW;o`I>&4EAlvPpy$Sl0P;tRhZM**c z?IzcvQ(wd6*_WZ4A*JOQ@tapsl&`iWX5Bz)$WRy*#)^2xl0rr_yQ!6}Q0m=_LB zTaYdyu9{=LHrOMFsNjq1`UuPQ^=_0iQaMg+rvU2|R`zg9ZXcz%{4zr|mHCm(q;HSb zIr=%3netG7)l+_y^5841)`i}m7=lm-vuAxjLyxun*OCGp(trUezH-|M&U zm8=c-47_s8T0k(4$TW67{-}I!f4A{pkS$yn^#x06P|mqzaOF@mBkGn`Y7b@7QGA__ zCQe8co;q~U4n39LB*cG`E4+$Y12h0d$B$AZYUL=&sfgXf3{&o`a=c{`fC1Bhg=;BE zP~WCrBQ<8Vkfm_J$auML9AP=Z++8ApY#?3|#>#DVQVwemWFFUMU)I%FOKP9bjf=+q zK6LeN?P;vPG)*Y0$my6yV=h4fQBdcWZN+Lh9h3JCxhdj+i@7GzN!v-v1#w2tdjxZb zS@e`a{*W&o(J!G$$iW&8hPOR5v=;HPfD6ubfLkpoIZ|dywGzf=GZ)9`J^qCbI1ywx z`Ms}?3NoJumuGKY-{K1Wq6lO1c%AH~axuv*Lo2kt7dxk&6?k)%dR3ERMInLG7= zdnRfKcoDa>9)fVp3X1q*zf>9a!M{`)Ki=p$1?pFkza+Y*B3H;)bF-h%QDrmtL?@vX z91L3uM1$x|hHSj!EzqTD_|k-}2FMT>Kol=t70(&QH=8*1>ru@gyJT5^xrG67FO-P^ znt+Yj?GY8Lv7*Iw)i|xmT*R|`$9B^dYh!oOw5P z=iqba)nXOI)ma0EL3hrrQ-?DM?1)kj;0FH>S?*`t1A_j~Kd+h(J^&o`Q_p$#3#9L? z_v{u@W~*DFBm8dPz=7t>QJ0r!o$H?BZ_nIm3j@7$(8ar4}85Rl*T%LF9 zcO4mSu-y9Jpt>WwKF3IH^iw}DBFcKkGk@Ypr~sU z1PGm@sCvTs9Qd~l(#LaMQx?=lBiQiTvgx3uC7TMDk*wAJiCh2kto1)WNb!@oIRk{2 zJtY#yra{%DEDr})&LAIT-kp(H0z}_YKPQ#Z*{f`w9at7RwSnM6UZ|Lenj@`p!!e#= z=D`_B$uNk+j$ z5yi!W)}CV9BTBGtlW0pHQDzEH{z^|kMH-T4)G+S87=!D%n!m{B0qq3ea}W4dX9rtq zwEX*lp+{S?!PbvQs-!4aJ)S5#O^Or$mZ@fB&o^s8>}`K*6iM zt}7;M5c!BWz}4{~!3XVTl~#{Q?%VLSpGjMZtsNV&SgWUhv(ObXR3WVAlL0b=rrWUg zwN6#v(+__)N41-B)K(TXpxwKwts0pp7DC5#`T?`K?N2GVBeZ_^9J5%g z0ZyQ3#!sCjBU(sW{~FZ_2=rS=HbVFOnjwb(|ML-+zY%tA%f{kmz(rsS#>Q@}8}EkV zFJcB19yp)XaIE*&+8MX2;mUu!%-{N$SF2W??x^I(&tm1MZfk2qb=($;}=WzSY5)zxQmMxW&fu767Xc-s}Fr9^7$D zg9_x=E;5M&wfNv9tYKy3UxWpQf5rx8Z?Iy_de!rm4#blRX1W&dcm1s28u`#G1fT{@ zVMKR)mbVb}=qZI+Jb^TUoqL6Fqs9Si=<$IlN6E?E{A6BT5b-01A6_C& zy-Pk<+->npzoM3a9A5|Zf{(IL?Ce8WeHm(i_ol!{?c#9xCP;g6gxB#v5XvxqkY>)` z>^AD%Co-?!HP6nX`Km^DWxVlr`;Sq|Z*E6tb*Ae`)xb=l+j{~rTi*xtmNycG$jR+b zJQG28f{ZAItmo&kJ1jbX${WWxaIZtHW6*Lp^E~zECE33v-d(t+eyX8qbf&p@@Rj8& z3P@3g@RHrONds5hJH65uU~9js={8ypn0rno=nyrg-;gUALS-^V>arcGG;}xp&IMFI zJ)ljg6Izz5|NC6r`o92sk-(77{tq|z2tx0*j3~_g`11)GB<$#V5^4#g&Q9J@YNExJ%)me7blAD?k00F^#+c?)ZrX$gNV#r=I)lXd`nb3pEU3RHyitY0 zCGPx$OHDJ&s=(K#p2InFZ|)c;JCC6~%?Y!2!~GHNkNHX`PDR$M=?@@H1%;-@W}y&Z z-;WC>+WO_4zwpV=CTpP|HbTFVxKKk8Q4(IN`t!E)cb7KNxfK1QY6C1O9^MMW9c}K* z_Z`bc&O4R~W!6rdXK3u}Kncz|ivkhdXK@#RSdaG1(kJ)ypLLC+v*z`mrBmjZF$mC9 ze7H1}A3&Du6@a5MH*+KCQLJ7YOf<^Fn}0`*mN7f}aLS|k<0&fEjcbl9c|8ly@Q@8p zp$4g;wuP+iD~0^!dr`7o4_zL*4Y8;E{!eZe1e1BtB76Vf@u$v6$sk~zM2-FOe!2t5 z-dba=&YZFs*s>T1HTV#b*O~G}K0L=v@42?janFwoQz(#8 zy^;6RWEIJZm9=wk1lxx$8|i3s3&v7e#l#=YwBMZ7QQBF_2_^lYkr_=zhc5YDo!81s!P}-Ji46?Br{wKWjfIW|qsE`M zf{yK9zTNsNET~ERU_vctlr?N1Sru1t2_zLy?hU9c<~;8O0SSj@-&id$$~fQbZm}Vo zsdgh8QQU@{>S%1~N1P0X^vudn!0~B}3oH1}G?vhmql{0x#nDMLYH*TEIGy6(g9sYE zjrz>%aN(Y6cgz5jA}ufQ&vZLBjhgEeSrwJcsnJJj`fId_^nHkhj=%N|5c?V6<~w?uY44-#C2hZ3 z9o{-n;!1VD;ZD3@!df_-`gR2tQ{>4DDt5H5!1<8LQQ{Bn;SbmgO{pcre))}B?2r8w zUPg}|z5U~dw|}Cg?-Kwhv^#29*xQ>;?)OB?zFRK(&o6A}D2<_!NxNUIWv z0xv0)nZ|CsBPksp{7A@C9TS&)Kwi!UN_cy?uAVG>VD8Ope?yUCr3J040_P->we1n3 zxOeca>`u$PO)HfNvg$S`4v+E)Rnou5{!qoX*njiJqrUrX`cfr3@e7XmxW~eu!5m{- zrS0LkHbzoN!OVVn%xSoxdRYcB47XRAiS^RBv1UzGX*|bWb>bKAqvV3zmIRKgbd>G+ZQ z6I{CRuVI7g7ygehy|1ihF8dR-EnYX+y)$ad%|P}+9UL|aXG){Sh7)1-*mc+o6rrwB zut#CQ-}ETCO?VwxJ)mg6IYQW0xI-9Ypm6(K1L}H@P__^r)E)QucI_Wd?&+VHp#m5g z_sKY};Zu=;I3sriNOb$M`5yx{14tRE6NDsn2(6BVXAx#5a1l5D>T@Q)S`C{pU{_53NM_MozEOr5t2Nyk ziD2{ayupw(5jEw1yqPlfpHhcterV`bH!ro_rC~b52cgNOgS|Jv$`gpC_loJ|HQ$r> zpB+zE494+NfkPk6K9GBaMaqkLW`XyRDagH4igopve6fK@DU=J4frl~x0!1kIa6}Q+ z-2Eklu0{8u4$F}uN&%|Uvj6R`!+xz178cE%>MQxzO-Nf$vZOf17QoR1!rlE(7tWb2 zmx=~v`M+H#``>Qwy}(=G(BbvpEeUXb{%9V3e%L@KEEcD4TH);rr}U zW=YIT1$#%=U9J(o`GY&0x|X5 zfJ^Z*A=KbFq#~giV-zNXQq|8ivW?lcujlMO=zf!lkBB>;Et-08zOKytN9)+0@_=P@ zY>^@r26Ub53q&l6@Sex)rEI4iIatT+r@T+^kv6o;Gryucv&^h2DR7>5DEKx=vXnYs zsD8$oN9|d1j-oKSe#-@NP%=cY4Uw_>Vx&B`W1_BXI2F70v*%H{r=g0~zeU$te4=HVLC63lp45Y%x5% z_;S1>_=bP?u$2u@F(?7KEuV_{46y<|8Pf}ItvSE+^*!6iIHSPiqD-@ z#dw0BJj(Jc7{=FFnCi@LmhI{KOeF5R|3@o*ph*!g-sd_&wFZTH2I-G^Iq%46fy(KL z50u6o+0;K6=qFJ`ODIQb-1}aivg%C3ndXf*^r7PPEFE#A&fEnpf^JzID-EgP$nV;( z(Y=JQ+0y2(6XA+}>=a>}9P0vHH$v;7w42l#n2;mSNO7phRVFN*hBi@Nac8}^mDvsO z8CNv{){40_5uex$y>p1j@RE_ie+63#%;S}kC#BsJx%PBaK*0$=(KalaD8Xv0E&H^9 ztM+CYPWeX=7cTVwF(cZ>^xXUKA{1Bn$D8nccL{aUm#mhuyiH&JZ2hSDXLQP#6Ebyb z^7c~IvYXdIm-tV0m*w>w&+Td%UJRx|LtVb8Aw7YfcT;{mM4>MZJgf}zM{dyM=rh4k4P~uitQaUS8onqH=H_}J3F!s{)-?MuR-hkE-xH0Q1Of&Z zSEBj0K_zRN&BBsdT;j+t)=n``Mzvprn=+tE%bR_7?ML(04g9f;UMWrV-@d2=mM;#%|~oz#KKQ*%A}G!2wNU3TpM@c-;zPO|Tja(7^9d|8XZ`QR#cq)Us&^#G#5a6e%*Z5@3bfTZ7&b=O|< z`|;$g0UfNa>z!(*3n*%qwgQ5~+gdLHNl$7`>H};_+;4KikG|Za!l$JopCzQEO6qv} zhJtOngN$BCkyfNfp>8W6VCrV*@y*b$-Se~7seP4Vm-k8;5)2H+v87Fd^9!$Zyt(m0 z7;&A@{aa6#>&*C&z3YK2EdP!uxvqb&^o*n&sWN#JpU;a+){*;G#l$*G{_)r$9THx_ zq1o|2ggiTu3|_jPxS5+Np0S=|B#Wh>y|hx^qI04~6`$m!X|}>NkY9`TD{({K91W~f zGC4QCK}3qs&QHcs!y2Cp9F3D)ld!io?~%IWOW~69;6mS>;Pm$nM{)?OTiok?Yeb%n zZh7;o)duD~qcyH;( z0e+Fx@rmzC{LJjPz9-p!UyJ;MfH%iB^b+=mVHC3Btaj>ATo!K=UgrtAG`_DC0Cq=?GOy4Z?2w*{nuOiIm8 z{>ZroXSrLE^JV~%UjW0!#^O^i+s3JZ4kcBG7nQr;7Iq#_g;&3bXUmX5hspm^ckm05 zc5{j;j_EwtkeaU1_nIizSSZy#GuUnJOO$2HfEWl|r?6#HMm=+@yZaaql<{R^j32oA z>iepBPbzl(v-Nb(5+QfcPrfcT?dHH=?uls_@;ReQM>8Z}bK5l_e!aQUZwEAZ%cY}m zk?w)S1*<6zqunO|OHzpdNxR*s7{AOD^;?kx52Y4f{qF^1sz%LjB+6embk`J$ndTG4 ze+EEkD$I+`!`s}76#~L%_dpyTgWTs*Ax^PJiH`SiJ>QSYKFyZpp^Eo_CQOO#nv#B( z%wu=aJyo~M5pfk1`L=}<3&#JB_L2OZm~(hUqWS zCi}b1!@^4D$1T%k<`dXtE768g`=Sf|M(s&kqgW)Lwq&-3RcabXd_uubVu@eLXd)O} zCojjpS?bZRjve#w)UGY|m1_HSLb!yleBbrqA1qW-Y&ntk_0{)D#;)pcOLLm z%bqPVrbC#8>hw3FJq&N>(|!6jIA9bo?BHwfIz!v+0<(ZaD~ymIodw@{BkvK(Luk&| z+~@-R-*w6obce1x&99D;ikSpetfndgIV-eE`{?AGVx{qlv=F6S&|2;hl_!3()Cyg_o5Lu#MZs0PkoOCmIboIN# zan93pPCbgcmXPgHX2xr%v$TPesUti3opW81__NfH^X3A{*yozliPW3?7dGxgi67xY z6PA6gTJ*s^8+F|4c#Ir{XP>WOBl5gkr23L}63{wy0+_@LF>7n>sv+WVFs`)-?Cy|i z*(syq!-q!bDAzuN##8GU369MS1L{zG8>b$y6B zXg7im0>cJ*>X%+BT^7BndHC_t{|_8%`NlHzs~)XbJ5iN|ALE*LE#PhIqXe z&qA-g@QG>QUkD;~WrN=g=ySYJwzlbjzRMCYKw=WF!ai{v7cP3&@+vo+{_Wa;ieCK| zXqR#01xKobB=@a*44ii=G+CVC6xnb5c>B&DYFW}IuE-oHU&CjRrwm8l<^SrX$m@Ee zK{)|vR9dCT|29GvIeerlWkB_aX%B|E8O#d${8%>S@eu{hO~Sw4NV-9o4t>hW#asM2 zTuF6&GeosEq~#=b%y+0?kwcFZ+qmNDFG~i!H-hyzoJ!sCv4DPh7seXUg1+3!PPSA?m8x0IV_2Ed zyFsnTmQ=K%KZJK*HrwkgUbe!qA}zDXu7XhM;5Or>UWHpJ4!Ws|?h{Vm*oTa;lf(8~ z|FcD}WqxsapO~$isGC9NMAhN41BqE~o-!^&W;ah*VhUVxy$i-K#CH3lfep5qG8DgF zd$%t){!xws`qGUu{6!qv9f}Tqn1^Rc?v8G!$0aEbV1Zfn?+R;5fLFMgl04=<_F!5k z^+gVFMRr~?=4(})F@@H=(N32VQt@zX=-FoT@9Bjan5k6`aK%aq?Ifo}RA=g8qDtIG zU6K8Mrefe0oW$|DG}6u!e2Q?!K<72HPm6Bw&jekxolp~8$C=V%eVy9C5a(klB5MC%MY}}| z`I4%KK%&ROb*S+S!#l2#X!huX@09uG4pwK1qh$H!JTY}mCUo&;jw#J60u3aVAYMb27Og$M`vrX>!0;$;Ny@G8ht2qG% zg+jsX6aY-gnr_mPh>MgJFo6SJ^Hv}7vNEXLi34hRs8;C zqaV}*oRRkW^IO0R3>1Smtg_#AULywLpD#9&Emzx(Cf!CfiVQHiBi2+L9cF-kq;M|KIbq1-sbaN*}ad#kxV7Y)0c4usc&Qi%35+hT2Lrem5;`qA*(O zmz-H?u{eI8iJp#kpuHT;2DN`WY`w6!oUbsefmV^tOjG&Y{7yW)|3Dwz{musr?2*1c zR_o^X%16)T1E~KZ9KlY$TVD<*w|%(e7SYtyhbuVy!e-j;EOQlU5->U2k0&W@uq2Umf$+L-x75#YMSRjIR>YUrJJ~oO zCLvJ|H*{LyUf>j)&T?|J0Xvm5D4QXMXMc(mnf|lr>pUkTSox^}Hx*^7@n4EypTbER z$cBV_Rn@t=3gygINYFE$tM@38^$-4}XY}lIJ-9oHJk{WaOBpA%EuG!bf2aLxyaJ>@ zI3d1XV)m*>lN4nwKR?Xu@9@rBebp%|!8_XlPJAJ8WxW1x_69ToHeE_089r0=c3$4Q z-fzDe$nssOB&hcaupYznbj~_IAApQrUX0}{4|WreT=9ACHh%RB^p{+YB|b)YxcB7? z&fqnC0L{l`@a}m1LsyE6P4l{DJrV0`w&SlA7o!VQj*j+2gjG_|NiI<_&N#%R8aEDS zRIv`9tH5(NZbZla?hu|hYF>GzAtL@I$BGp@ThOETWrwj2g zm#nwEUy4$N#>{R9-Y~WMJ(&_pPNh{Qy8BTI^04uBT;wYsum&A`9s;(G5!0 z-h`Fc*B8r!W(TSI0xF26k-&YaB1hN*;V+DNu6X94-Q z_{Ird3`a#rJ_R@D`z+3)Fi(9=DmsQ^D3cl|9zO~f4wJnqF zGZLaLz;4rL1UdM-OSkop?(bao-5$BNN9RBhSP%x_1CQJ6)xCJ3*Q6P{J#&KNPx|Uo ze%1T2T77Kk+naTAT!u5A_vqF4+c53{rz?;a-+EJ58{5zkQ4KnpbaZ+<6}~^Z-7bTr zYURStOCqMG_D_U~itqm)%&`8pw-4s^=Cp!FwgUD99*Z9O&1X{Aa1_idIK(EqeqJEl zl>@Az^wd_UwLq#Mt-9mJq_=+^hL;tib;jeBAj&N*7`;c8P`SHF(pwH`c+ zYfBuggI$vrwW(d}YXhK2kLlXRcRtQ)lEL(Lcf>>K%}<2VccsYW9($v-H-2MVi-x^p zhDQI`%0)HK8|l2lXQM`NWQmDj3T#SER{hy?Y_41*=wqT47VS!=zBnQ}x9-g@5O zI?n=Lt(Z*N29aa$`)s}++%nhZ-I>B-YfogY^KXZm zAu$WlG7&n9HM!}8l}WqTj@p0V1-MFa30*Y**) zH`%NiJ43ltc3Z5g)wdNH)Rd9Ho(m1GAu~O(-eR8jWgdWdI(wsI{l*iac`n)H9AdQQ zF&@vg`qZ^Dj9}UV+;!l1abG1rpt+!O^#rKudy<654tXP77sYyIU)oooK`M)U=cON- zX~n3TJqJdHm(WWnNucxHoy5tx>X3EpC6-da?WOwt+w1GO>YzWFcX#Z&+l;~xR-}~+ zA@`)Q-2XYzGQXVI z9P*dJxoy{F=^H(YCwG~iHhN$pcK2b%iNy3r3Q|E@mrn)dh3>qi#yxX7{u@*Fd#8fY z$I*Qk?1F#2bgU~acyH$^9d|BLPu->cxiZv7pqlcnYL|gd@c1MW1D$6}IOopaKl;|N zW815PsGuf~d%kt|HT2GqLA~#mPY1Sg;zJ$IC;mFPFLL2!nr5_y)Tmih!TqK}hc#bu z0tNS{Ehr`F&!pe)Dy*JMEqZ?P#7@*{gAFGq;(*NOoeCeXd#LPu^6uunlTxR4iLp$y zzZt}BLH%-~(FZj#2)=D$yrWsU9Rf;9$L!!)xx2T`(qQA}5;Y5lJM5>p?=N;ZU(4S8 zg8>(hkT!{F+A@uLzEHCqnJefvIB2~U+Tt|dc>^7h*c-oP%keF?r%zl8-#SUz)u{`0 zrovf9JN7;J_2e)2ADlRI!E3A7iQ|vzC+pM=9}n&L{rMk6g&V(~@cQ*e-iaH(dfq*F zRQJU9!>I><|K)Y_)_wbGmN)oZ2A_ofVA|CM7Yf$`L`AJvLxja^(`(b1Q0H7Tcu2?~ za;16$_=EiZ2)XhRsq|E0U$LXOJC$4$vB!^Kj$_WLR`pgT(RB|SjToan)?%H9)cJx@nmUblW)eR&mfGAwZRQb z_u%Q^{Hi7luNWsaxIQ&#mBWwXnm=etRIemNIH81dWeZm8OCwbyS7Xfu8pdeu@^4Z? zD0S}HaGi{k#W7BENqQyqtJbvV!g^m1wc8_aW0h+y8;K{HURJhi-ESJ~247{#$4a;j zZSy8sm1n;JGu-3{XH^DKk6q!T@3U6hHdfna*q|76IWs$`mT%fP+iH-HqAB{f5{JY< z;b~;T=d_9G5w(q_z^)Ql#c&H+{C=2X7znvq+LTV6FB~?n7MQmMO~sYgmVykZ%Pq*H zyun_B^lPKP%bPy6<8(DCsU0vHzH$^kUzS@;uAWUqmNSTDEtNrr)!l@YeydcRHnr}B=lN@Dax;6zDt)Tqp|zEUtz5@_~#gmFKL%WSE8E?Ir$<_ zMqdMztq`CmEe<9m<}bJ746ei1=hCfj`>S-STF~eF{h5JRBnr*gpBR}<3Y7-!aw-mL zg`ajh4lTXSu0$ooU&6UPbnSn-n2y3;ANRE`TytKn_)JeT6CEqW_;=#oW>_^onJ|Bd zgD=>^SG1;r$rfI`rw!O7FJ;0T-?QeSeXrWIiO~>7_UJ(Tb&Gcf)8!3>SJ6+ZVhik& z8Zr9&Kji2ffx4H~z@T?_eXZuL`X(tj+DED&IoW@IfB1FHRt+=btLGPZ+SBU_R%|#5 zeX$Rm7LqSvLanM7rc`mJ!k_IQbeMFGN`$a4IV`4D(WVngoQdcK zXZ%}q3H0&?%r8Xr2XwZ}_^_$-V4aL%==oV*?Yb;DyzlE)O$klaQo$@Y4AT>)(zsXY zswh)BVue$2=zI(X4M`2p1ZdRGr`2_8N;yW`lvzPH-t_gzSTKF6-og$}asnRZkan zz-WHY+HGr+BGy%dDnwJ^n5;Z5I>l7V8Qp zrgte-g-lr1TRsD$=rr9hOB){2 zXKIyZd{6$Iy6z@?hF&5e2?{|Y^r_cFnH$*mWxn?{JKoN)nCr(#<>40pd2^fV*$)1hK^P~{2yD5)sTpOiJlrHtZ{mm2$ zU<6G~HOXj!6V6SyZr)3B4Zy^maGqZ21vV*<4?r&NPfU zy2pBf^Kfh=$-E5Hu+5d0B010-<&lIip*BCO7`9PvBOH)f2_YqRHT4@nypwu5DW}S` zBN=@O>_led&W~#1~))RL}1R3A|6F403ma~ zH1Gb{+e!rYz*~a|Y9J`7?eq2rhW8QZ$m1imb&B7SG$}Wcc-t6>J`i z(f4GnJqB?OcWHCeY@_;?hiQaVC94syNoN6!=Ab;0i}t1%hSKW7RHjFtOIy1Yhpp1& z3_-W*Jx07^7QJI*HiR+A!AqaL@CR1QBYvCQzj0L6LoJCo`ZNTVHUdMGboK||u=dFB z;5kT2)6r*hGYd@HnDa&83g6y|?h!i}LimWenz!ES8DxqLhka;Mev@bRcykII6+8k5 zAMwhqu9#p`w1H6(KI^@H`%q|5*5G?`I%;IFslRuq?#7|m1Ch?tc_6o@xt+Bj;Pqw- z%@|G7=oymdq_R0CpINlNB}q0h{uoK_W95B`Tz*rVZX9X>9yX?BFdjt8 z3=*O`ols80bMvE5`Lfi6R9r)4)xcDrnumqts96!p??;fnAl%%0o+IS(qh3fe7C+}8 zzjf3B_*Kb~1ewidrDlXqLY07v0bHRHYcc>**+IYv>#h4S5A(nFxZ)O5^a{iAWcD2f zTcM>`=}VoKPOS<4Y7Lgs+0Ei>tJ_2`f4A2?_DbSTXWWZmUnz5|36|n1SV)jIZYHZT zy77sUCc109z^xwIubrxvkY<+Wcx~*Y{4F&_<&ZhEe$;6l zh?5%S=!ygB{)A2$<>&(MP3KvyR%*6cp@?Vr2wKGhE#gZt*h^}Cc7NFTtv|BO=_lnDz{TIE$OVp^;jFM zxR>R0{pKTD2s9{MY(a^^ry|#RPx~6DtTZ*T@yRe{A4B2K7Z`czj_=cn%!w}FXPE4s zPx~;^P8jL@IVUm=7VB!|8Purx=u&zpBAmD25HtJ6E`T%b%;h&Cnzbto*8@;%qqaV3 z*$#8l^;aI6`;T=5Ma>mWKkv#E+WEi1uZTWjbMo!=p46zHXi|#c)qNuw#j50o%qIMX zHv_gZVrqXzOnrqNEQEHcj8)?QtU>xL#AYzHJ+%k(&fnL?I>@^^DHb!O{V{vCQ4iYI zbESlqY-epUr33M9uItp2DabEPI^lZw<6?zF%yQkWb=J6Hx7tVzEUyt%#8(g|zrx8e z4*E1pBn*6d9t@95DspI?R4c#Q0gl}hffcWW?pp51@;1Hm4TZ% zYU!qPJ`p%_!u4$#U)Yy!VP6&VQTGcSrlFYnEX3-_sPdcAiJ)WyM|FAA{NgICKK-Vp zvEnP`#>V=N?Spy-mA4?m)RG?l_~<$N+@L)Wd|jT^a-VQO+@B#H=H_u8lU!nqdA9Te zEM=w{NGLCear8;B=8Dl0^}!+Cr*#*`{8)|UA3$l*aX*}Vszr^-5x@q}aaJWIEn{6v z#(E6JAsybF#+|0VlynBoskIrk2Z~J6nnHZAWl|JM0&$X<18(Qhke(WqDg%qDsiwG} zAK%bu;_dhLDi1xZ!MDrH`}L@C$i!=cyf0xgRYAfiUkf4q*6-q>GswshR*=2!tJP2I z4Vw83{Tp&y6J++7YbuJy1lw98zPe%(*sN{lR9;h4GP#3Q>F};2?LbF*M^=@z7fhEn z)=^V&UCr`iCK#?lj}05T8P)&D3ayT}I!~M_AS~5hj#OXpNZ43JZgj6Z8E;sd90_{a zP~Yp@I@B<3GV{U*D=BGfAqZ&9jhR_k>O#t7VoeXL86@cBk^6hSqdI>bzoHnsa-P0< z25`Al)|@s3Uv}RADQmK-81e>hg$S=GS!WePHNBr%=9u+d^{vzrYOni11R;Y_3}_0N zTOHomSb`1OJ?J!jpenr*t!)$Iy_+jC-(&J!9rnoZN9QB8)Lp5XR5?@4jGvQANrg}p zt+T^i!JwVJ*yFeK zYi&-A>1?7zD=j81;?*(wX^PtHW1Btr@Oqs-k9&v>mV9;ilBlQJ%3^qz)6EE)y`^98 zx0F)}!G)~ey|JPE$ic{^%>3od;g0-(Bnrk1$o{L`zLexQQjl8d|$4WkkocOAhWuH zxH7=EVoU{uDV4>nO(3FQ(^zB5aGH2JzXcgBn4ts2uJ%Lz+Tv;h0HZnA4f|Dj^kq$yQxsYd+K=HsdzTpJjy*tou)6hR zkgsAjmy9jnxXh+UPtv6%H}sHbyOQrAWx=^cD-5nxk48XVz_r;YP`{kjh$Ts6cFEC*6i_4eo>Fz?YxF6(6r<^^m=m>NKqk+*te4Z#ucI zFTgh{^xUY675IwaX*;;6J&>&fM=xz2q_n|>vM|cWuWQZTRvpnB@q_zB)BN@6;iKuK z<~ty+M2{Yc!nNcpmW5W77*PjTTL%3|0~0Xet&NTq{%R|~d?O^+!g{&E>Ejti?pEuE zpDT`I6#CET2PGTp=b3!AOQ>GXu2y{Awo5bT$dM%Y(nGjH^g1^tda1*Dtz)<~j4GJf z{M*#kk(dzc!R1zP6c3Vr%c3(ef(&q7KkJIm89sF;RNaf2snJWZG2!8ZD@?G((r0N^Du0EFBrcR92WTf+m9v2l zK&4iC?VGfuCMfUpMWrli51C`sPwlbkH{G^=-4SJlFzKvlO&zM zp$oq`o=hjk5gOK>8pv29vcW06IV+jcqUGG;%PIHerR0A(Gk?;-cF>>B)(!O-PX$JD zUq8JqT2xpD>@;cO)vDB8VVvKwvrl;)-DF5gzcE-}{XmM?{_36U0IjbcAj9Wh%2&YG zyiN!RSd8CLu@v}+FMOr1HZ^~BbXYTHPP1RV#e-$*6mc<9J1Ewi@j~xT^mux7v1R74 zR*$V_?lB9tmXjiL<`H-gHj**&NS-_-=Ks0o-XkT8#Cq{~nUC_+nTNI5jU{a5neMhV zMw?OOx{kcu6sxBo)@Oz{MUnn{Pu7jfP>CNE zu3rebE8+IsAyC&WAf&{#LQKwSIQ)Ix%Lk#Wv-C=f#|ukz>*(+XJ6OA1kVTir_phmi z*y#C!sRl)frb_BUX)SQtJ}nM5&!lfmA!AmCN9*s6Zj}x{?mD9!IZ2qJ|F)?$RqK6S zpH=_zzT*?{2vzm0fViQAu=+_;jwQ?5_FG(~>TO)(_Ys2=$V@Vt_F2Chhovj4UoJJxIpZUotA^h{uECz! zFR?G+t8T!O&a?X65@G=&-JI|~L4WmHf9 z)4d1Ti-64ad2z3rlLL`dfFpbx0h!6&W{kQ-&6vYyK8&#%z5&GuRh{`mN(`=qjsN5|t>>39jbJG#GLaesg3`qp`1DpK_# zmd0e%@}ABfi^uA^%rM3s!)`hCu1~hDzeQcxo|j}0Eth9u`URU2*lxoNncd6SyadB< zpp;v#QFC>V#q=2#Y?ju3iP}oal)uCKr4cRVWs0Z^A0TmIahMB&+}3@({5lO0Qe1!U z$;cPTSr04T1?+*ckWybGbE~2oTbH}=(G_cKkxk6JP3?97*e{>Dl_GOwrJrHJY%1d` znhU7!ZS_p(Ka7V>+^8#=!ssA81i#$S*_uE*&2{*03o0a)T8!%y%!g5fq5}|`a%6Z0%^IniAKL_ zafZ}euixDYF|jnRb0^fN>JW*`0(A-3ydA@BU$8jiW9oeU(|uTzr?Xj^!wAN=GJAL7OrG47e7|2l-(}WslXma-#Q!MXW=~3>o&X_dUIjK za!-=9#Npe=g1o=ipG+O0Us?O*j+vlLE)nsCFe?8A6h>+?Qi*qrx;gp;Pk*6RIj=?= znhpJBEa+4|V!s_mn*U?`(Eoio_PSOFkePhoP6<=`ml`rnlZJUM289?u z7f$!>w2-_5Lvr{OO$vAqGo+5P3%?WD{PzpyAuav( zKN(M5?#$ed!>L+?un@X(`*qKTdq2j$zVdtqr5GHG zo4@aLdOpNDWEI5SB}>W=YSi+7QM>KI?9rVxJKed>6K^#$dq{0?R(+3S-j}@+uDrSx zpQuUt@9}1{YBN@RbenI;Z#yNlubU<06EpAQ69k&Z~ggD|l(G=9!{KKs&{Y6-RaMgZCu|=2hO+UB5oLVg#zOM0$7M zF+=*63Jv*dqcHAjcE0^ovsBqxVen!9Jtrdswm#J?QJhW%pd<-p^T+hY{U3p{akLk-I_)1RogmO_T6c<&S{S( zh$+K5bkgc2t1v)r&(e%RJASPXUty&dJbRm+jEx$tS+S3r*rS%$KBN8s_dB`2_TD2L zIO@UKL-Ho!{3l|P3$#3oud&#ba9o`7(jW0yHZV`N4Cm_0iAR+$l;bQ*6O%Rl$sbPd zk&_tnPe_issoOq;;TY!s$b-#J)!?kx#{Eyw?M5vIh6})x;aT*TaacJ5V(&So?z=00 z68=CC>dfH@cX6ke47?Uy>lZACljL3=g;^Ld+nvCo|5}h z_sMH9^vw6e7&&|=2NJJ|>dXoI9=^O979$Mn@6hJa>96VvR6BFU`XYg?=Twxg+3X9p zQJ<{qs`Ntf$cc2NTEw76nn;!Tp~FUZZQqCd_UMIYhZcDA7m5<|o#_{uokoH5fIUu;D?!Gjj4g8U z@9tysAal^y$l-~D)9~Ra<>yjLEB21=J1(W0#9R>M?R+Q)EdwO2AQ$F(SM!6oT(R%@ zWJ^?6p&P9w@o7ueVQ|F|KKO`+JAy9;MnwRU+p=AjlA9_B0cwg&aD}TA6y-k__oQoX zRrWOyc;sgv;I{?Jun+AIda>T$t*n$n>qUS149QKEmv~xCTc~kJr%C2L-0A?kZ87eC z{Y4T^CC4Vmsp)Fp2a|$gJL}fHNir?iv9aLQ@L6)t(v)Vk0n~H=Pulp3T|KuF{@0m) z1yIj!Zi#XoXXH;>n?h%>CB)bItRX$i&?zyzJ)M3}Gz`VCy}r|J638Po$RBx+;|O== z`z>graijI1XjB;X-1w5IbvPaSuv(zFJyhx|dY_5|{K(bZ_;1&%8Y%aeU16)n!`ks733u{X>bb$h=Jm-zWDWn^c1iKy&LqNE;rQb_`K zn}%OZbZ^=~y*-~q;E&$0R46hwy&-q8JNDsIjNkA{`q})ExZBrX=`WAf-Yr$`E71fU zFfo(AIFoE{c0fW^)azNB|M2O^Avrzt@X;_iEBoWRMOXg#-1MN~B;yeXRih%G5;t`* zUK4#x*bt{=-$lqbt!VYHb}miVWOai`H>#$Y>^?peXuU`(y-${3`-?* z4UQ)|Rmm6-6p17QcSK7BqKv>kJJ*A!c4`AYgagHw>4FTn5-6lqNQH5x2(3}|6qiL_ zVq&6T=-kbu20@=$xPGU>Rl}(oU%38}iO!C@-W_t(a(j> ztSquCYez%k>S7zeMT5eM(ve6%Agba^MHVG~t+22u?9wrJ1vS#=F z7Le#51wjjWxkeN0ToVGA1npUAe`9prZy`Dpoh(!U8yJ)FzcelPlzR&t))?TPJjXrbD5oj{z>gewIFU$;&=$QQ-()df0xg1f zvDdFyuT>I&Ei@px`w5DOzwOuesyVuETt^@O-%(``|HIBoPjqQE(B{g2E3SKKFit=X zKI&`5c(W}!v?X|I zYim-l7-#pSofrqZEa$lZ@*W+IzI{_)*?;9L!$M{s9u3`ig5w8}BB@4sD+orhW+fvm z%I4GPqZXoohIxfWs}3e#*xK4p4p4E1(dYdaAzs!~->M4pqPt2;f}E$zQ(doNA=!14 z@#FdLJw%s^op?W$?}ie6OqY-N*%Hyf(6~;znef?YONZ9%)A@mSxT&3kP62|(sAi{T zjOR*5TZ3CeJlt=xr1mp)utu?V!6|Z;f?~^pqocOF z>l&lz#M!$s%>az1hb8&COok9k|sd%d~Do%aNj6~sEqaKIc*^s;UOwRngzFf53=gJ=; z`?#)Gk1->lbSj2=;`j@#N1y7J6Xm1rG*DB3u@Ux+9G8n|B}bs~jSiG|DkZzDnP0bh zMFka|w!X~Q0-Q8ceOfb;KhL88c;VnT*DA1423$`>t?4~U`u#!K;T@XvLnMQM*6q>J z2eI^I3l7@P=CYUxGShc??NIc$XTe!Cwd?#A>o5?Z?6ljC{*x_S%$&ardBu)I^X{6|HeC zgp_(edRj?-T6L%6cJH8wn(eCJqV;klj_2K%v+1sOd#w9$Ik|O{c28|g0&C^_vu0RM z&YxTHNP1sYw;%#07F*(tgJqJPHH&1SZ=5nQlQ$>E-bk-FLrFT*(S4q9aktA$CLk)E z?N5mu$!d&z(myD8J#t+w;Fi^8q@M|48Dz609ZB%sIOo4wSZlNZ2y%qAm*A06az?+~ zUH>sJs>BKeuK7^2b3iLeqy?MT@g^H7rm0J8_pm@;8MI24}Y) zL@cK=*y>PBE8EN|B06$yIo+xhT=My1txR6V7=~m>5Emc zzcVP0pfSV`-I+bGK&P%M7N_DobF{CD|274Aff1~ zs}hMjn0d?7{EFX`FJ}^76~5ws9C169Rz3169Eo+7AmLO>2l?dffFj_NpD4IoYV znfAHS`MBRS4MwhXkM_{w>(>U(ks_3v-VqcaRtc%qk8&Q+!oAkq?tX|AGwBI#{SAJb zR2;19QEcNa11-wM5a#WgqtCB&L6bUxQIKiO`>VWe-YIkG1QtoD{Q55obXW_pOkjt3 z6?Xq*qzzqc%>J}^y77z=TZe;8u0?0VgL&JAL`Gy&79~ z8t}`ta8Y6L{KG&5U{+g)n%!HnbO6^_oE3O6;xl@mrnuSqKKBs>8%!RStYzzmk%kM@ z)^4s~k$6<1Z+U%OjAzXm>anOetlIiy#~7l`jt>YtNQ0TJ-Y@P?CnBQ|k)*VZzo07A zzRz~L2jbP2mfFGx*QUdyH)T@kABr;OqJYUZV3>Tod`FP)`+1@t7_0V{nEA#@Fk7k& zPh9*D!XmiVitaxDM#__zzRsutu-YcxTWICIpW_wed=YMDTiE`2jlrg*wz;9iGG=dO zp*{rdG~7fZZL;knP8S`&`ibs-P+(m&@RGsg^##23==PoRy@LFWRXNR&(R6a}_k@k*wwUN}Kv3?xU8k(%YM8N+sq?OW z{}6-Qcf7TK{hwHAP9ETEY8*pGe66MQ8Od{8fPt8EU*?|&ViS=t{A$W*tp?2!)QDTX zqa>W$xLocF>5{jPX05JrtfSlc^7JXs@E5W5o#;Ffo+k@VB(ZN z(wq#Wyj1h%Uujmzn*xGnzcjR}uPbZ%klOOszzNsEh;YDj0h3w#@izyxSMLKB`>=<6 zRqxElN=m8a))Cw8j=aV4;RiB~3aWw(dij}dPyOJ`Zp~O5z)YwWx0<eKn}m}LP$gT*NJrD@Hu#HppUap@T^ zGuy5>HBIIy-zYA9nChOrzO*+MduGPdv)HjJgw#>Z5z{0n=&1ep%Qbp3l{(}{+3Cb~ ztWr%%!YL+XW;A*5)B1bs#@LB@sQaI%Y_3;Xc3TD@4gtXy*1l8K5lMGDXw=*kxB2DX z$tO@UD=;;iaU&e?9eSvqpHmBz2A635TW`-bXi7_C#!kiBSg`ii?k@{`I6RAuiE{n( z8W1N`oCXN7RFo`W2rR|cT27;|i{A&X@gsVMH#O?OA5E}OB6Fb(Oa@G;TZ@2(DbWX> z%3B@Pl#ZI1gXD`kXJ%rp$td!CA;@3iEtJA@uC;^Br40Wjjs7|_ByJr}9F|0yY)0-B z^VTN4D~T3**#NEwpA^f0gloHUFP8BMq-Nh3Md~DT*zGd+2#=y)Nq>_LHsh;61b}_y z$%y-oH~P(3C`Gd`5!{66uaC2H71kPa!$C+o5Wwq;yfb@oaqcYlz}7)eVbeemeqVp2 zk!eExBOJ7Cdlb8U>mM#|C+e!y$8P==?;54A9@5*Z`Ij2zwH-rvDafF3z>FbsWIIT4 zikvuNw0no-Re-o(aQpTD)XFk#lSUEaew6H5*7FSvjGj^FHNn?|TwlMdxqhM^yEwb! zaRf7ja3lgyyL zFL0HMs{+A<70Vl9?dlG}M-B`F>hAgsz)d5TfG66UG%J%g^I%auA%6KGDC+tOBK} z-Tnuy2KK@Ki$^od^*-{RnJJIPqGS-MD_RJRXqqunonR_7RGSx6E?>2ka@^r|Q1-io z(P2!bO8ST1xI(ACP4rf?UKvmS)Hu-OstWjW8jQ?%3mW8&ldGWnQEH&e$Sd}tDiGkx z)gA_+8y4@U>)X3ry0*rHs=%1F^@nW#9W)y^s=7QSLf#s3QLg z@0QI`Xwf0>AeAxRh1yG>vZ1noKl3GhGy>V_P|o#YbAi~Gy;xB zxD#sOdsc(Y)MVzqfUBRkcIf0C+YGuYTCMzO1C++iEYJaKr!+_FwfT=*A^=s*Gz$2l z$ec^7+>`#Pv#^gIPBvV4A&t376jJz>jA5y;n+#oA_{sVkH80~g*2kv9sv{%K>cP9w z8&A$0ApGFwmc?35UZ284X=fm2MJ}!TO)hHJkSO_rZL5ONcRM@LuD+CfG+$F_#$~SF zxaA!>!fE-Kuj7OGH6{8PFG7@u4|S)Iae=1K`VRTE?aEkB!#LXhcs6etMfvaTbMoMejyz<$5=5@!{b76*t%UcQp~q zO>nCj**F|D>ew$J{*~FO<5qH`xL;u_eLYfu%kp8T)!fL4wHkX){fW)&SU5EPVefs_ z`R5;N4tX;RJd$ma=qh1uN;{1|V|Bu#{Re$Q4y+lI$UvEQAw_enI)LLpfx%4*Pdk?y+p*ZJ@FH@wY6Oc3N zjKc=)hx?b73z;q}`5fZ4UtGLk9P zQ?%S5YZLh0Eiq{5S_nWUaM<^(u7Eu~j>=Y7yEOHSR961apS!rFrl!0YC^!=J7&m`S zJ!*724suRiD zLXG&&q)4>6K3|2Q)6&x0=n2F>x~n<1+qaRerMs+&{EQ66D$q z37ANHOaH_v#flaU)aw4&VVfjLNnEQNkB>hLGp*l~Lp56b#6X}6xHk?Ki8Zb^mx|2w z7tRT~ZHi%;)V_HSAWPJz)1|Yp*E_$IHKqVYeSVASc2#2s5aXN-qZbp;_yso+$F8{j zD%`Za`r}8)>8~6P!31h1iozeI$vp;iWkVp1!&eNrU7%Dk|DNex8O4p_?A9$gp4*lq zPG-)w2waud-Y@TLMjww+H+xq%{L;U%v>JUFC$Z~Yj-Bh!<6N{S6TwitPbG9fm5$6C z(LOQe7ES7d7`!=RhIfl=e^&M)FIiM1ZBWBoekfVW#X8!%kV;@=Zhw5KiiM~#&f?pO zp4c7|VYy09!R+{GC*NacuLM~&!D7oiDegC5*X2^5zLN(8UCWMBp{w*mv(_J14LPORqdN$E=v zG>#9*4BT%As-XO=FuaD#2j_`gdM&E-i^DW!rS%d;F?+-q))JYZwBj(0n>osaQHG1W z7$uw2T4F5z$)CSS)Sg(ZyMU_$ut|5_7H$v%U{cw&zeTm%$NZCxfb2OYtZ6_{I?GTS zx!h)p#+eWJ5bF->-)%B2LzQg`XCIyKOr)FACwf;0phbCB5uS_LLE=47w@6Q3mveFA z#T*Pc^f2=mk88*e;}np-4o?KYzyo?2yi3(*_CEfDOZN6GNnmJc=Ygl8p7MTyZsx56jB z{4UC4PU(U?W$a9Us@D1w({`oQEhD$T2ec&3quNHygS2Z*vxB7$39drl1b13mnq?D% znyqJf*;XY%O*nhU<+{;Jj>F?nOnVPNU4fvWUK=b>GjDNyn4s(ua5xPa;Vo4n=)Za7Y1_?%HU|~}e~_(Z@G@6C(l893 z86g^b_5<0d;s!@eJu8Mn3ysgC76&@Dx}f<6ojCr|5XpU48uHyENzBpDdZN2X-u=tVYx)j9af67J}1tjfaj_Mo1z6y)eRRo zV^|)2gu%#B(}zvPovOBw!I7PI*$lksi4lE?n=^Av`;mDVcefu#j93&5b3Z;71mU{Z zU&OX*l-ZlFc#NlKtUX;U!7CR7)8eTAW}0I>|LE*})j_lBrvD!M-^z^eewco6iy8AO z-w5ghq~fXL`2dF!-vi%Wy2+px_-x5d1~qu(kQE#=nW*@dmyU|xd%v`Mhl<+7f2khO z6WpA!DqefcV{?0UyE|39rDq0Qz1*^xSYnpHz;U;Eqz0VV@38S4mRnlzmA-F-dX?)G za@+*v{-9CTr?9Wio>{*}o*5F>shcxKCR^Uzb1nf)hjA6q!L(^kgDrXyedoHk~k zriPNS=#L)23?u0x^&>p?W#2!N76!s=1?ZP2m_l44rqB{qE~wRtVYFtq8%vaAdT1z_&q?y*_s@S9&q7Sk)Tf z1q)}qn4Cia_H^Iqf1w4HBOZT;>TCZGp?dx$2L+jEAi&i`?iq%Kw%u>jLvVY?gQvZ% z#-xVMeRp#^_`@yy&i8%d1%;rMId`2RebjCBC$gJnCjhEOnv;egL3h%cGf7M1XQ+Pk z1J>HpokTxq(WqjMg5ZS~v z#<%b70hLP{RHM+A$lX92sqt^LEQwR@{cK%H+6L@mxNUCLZk4b>um6dnkOG#FoPGKX z3MgGvAcY+YLJBUmf}TU-&LG%XCD4%b|5_eO{%{@>8L=kwL7LO(qLNYsmzIaz8dQeP zbWyo@$Kq0kR)asVA_5b~NsL$)?D$(SKBUmhrmJJqUZJnb;Hl#uv@4?;N1r;`JF;Z~ zqe2rih5PBM%uR}g1u00|+UX)R01E~aW!%ueH20eik523;0`}WiFsTq9%}4eY!YNIw z!TPr>-OYe~#^3F(O&IgNk>O2D6O#JoxPVfL>TzB`r86 zp$Zq~_OmQDGO8mj{kau|{J9ROO`5_mm)W#v7uZA~{s18C(!H7J=A+Y>TFTik zH+5+MnlEfEgO*|7JHo6Et`z%Pad)phUAoWBd5}~jNmLe3{@~Gf3x9fWL=Ju!xk~7< zA;~k0kiz6qT6a9) zzQbXn=hi@{GivLA%YhOQ^NMgH#(S&0x%)BwBG`ydD(t4^YJy?wO&xz3`)mgyv>|&i zO`!@6r{a-E+Cl4RlEfnm`?09K-e7l1}{kJ@3I`J&;Rrf z?0dbXM`tii1cYkAo2fzX%qASC!GRO@Mo@vr_?Mr{4s>F-Sk(8-uI+Sn@c5x@u~`@k z{zGERI%KI280~3bLfXKlM?wSM;{>y$A-i{i=|qBlFn%#jF{WXL?#Ywgt9}eOZ%Jmj zYjs|*Q3)A4n-MN|LG4~2_Z8y8=QB8c)DqTwn)qR;4`}%SC?R_{-hiHiW+S0ri*r~S zNI%m%B6Xn;F1PzuP&2SYeyKQ>(*HgJG_sj3ZCKkTPdr|A&`^VZTbr*M7Bq7|$88iH z{WwxLa)I_J_=exC?RxO$I3AjG*~$qm^W_@8pBym%_MMkg6M~nrtqtWdKj(a*oPw>6 zqNqP4E2$8HIeN1S>P-r>|5-HRE6Z1Km^E1CreYxY^d#~Pa|RiN_uBi_i`wWs$I1Iq z6&9YC@V8ik8sNVo4R+7)n zzI|xIpgF2wjE}57{s;IJY;(iARW^IJdf%9?Xs#$#RcV<}v$bVTqT>d(%9Ky zgKv(8Ob>~k_~Pv;pR_Cfi@g;WzFps89N=yE+)paf3$iBpPA2GF7G%bFzfpO4Uk7^m zwY_ZFB}jW%Kgu*d_loG%fI{Bl;_Qd~=;9KxNM;7$mh6Q*A^qJK`m3b%r>riOgn@oJf8;8Fkiag^ zcCJi9E}&u(%n`{ee?O?a-GOUC0}+`2(Sx#O>b|b~-}gf6%$NJ(Ik#?GL)&RbV9zLB zWmb*z1nKUk$ld9XV#|~wDqS+om;_(g@0&C$z54SqxkqZ(808AKWfn9!Um&m_Zz*jI;(VjAnDBb4WVR*50pw+pbK9>%EgqCy$$pG7oD{X%lF z?@-du4~KnPb*MD#^0hl!0Q*^5%J~vc!3x5gUX_JHb_@n^As1y6J=z_#Nts2&_Y5{sv za)cJW5wD!HM>h;No7k6~pSOCqZin&rXyAiU^GzHj?wfgf`to*Nl;6$lpa~?bVf^Ga zU2eOaiHA3A=CJs`i3f$KI%DW1jG%mCvr%dvbO+Rgx{7p6IIbqywez{I_%)i!~HJb@@8o+jl?jjE#KD$Y) zq4t2oI&K%hRGQz*XD22SgKz#!^8EbO2!F?TQV>~<*#y%7%u&#s>If|wP{PSMF_ef} z2fte>%g^)S$KgM7JW}uF-m+(o+~M>e;9jM2@Bicbyb@wP;56-tkPA*| zTo${nO}P2?!4H#}bHJWxu<-BNdVufy@5_4IH%;oF2S3n(OFquoBzLdt{{J9%QSt@5 z|Ng-K4&rPsr*B<0De{@&sMB#446Ibk!x0~~%xpSPX$@Pzhi{HQuImWNnx-cnIto|- z{}1nFM>`ygMQLp#}pg#P2Vl2M?H_@`(Ka|yqzS|6> zms&5x6{_YsxMhYe*0+NUfM1>fg`5681z(?P-`8R7$r87e61~C$e7QKdI|4-V4G2<- z+ylj5`wv2;ze~?_o9mEwZ2dOd9f2pE-)zpi?ik0ap=xFdzwydl(_J@U%)ou|ggHq= zmY|Ey%=r8!TJayH`B^=)@Mj1@317C7xKM^R51{P{i_Uf$B#`Ur#QIrW*^h6@5+mxN zmh^;%jeCxVX#aW~Bx3~GgMNdr-w0fgQlU4NzSB5nT&1aJty-;~A!W5dV;?a;_oyRM zyxR9z2>-7Vpr>yP=!cuMc;knGMabktppp`tb}4%V=&Ab$;mei}l%#`E1g^;khptF4 zgLZhQtEKpN7bMMGu8J!mWGyLM95*pEQI{8&iaPVFiXI9e#^MFzPW#5r*V$+YrqZ7H&)KdLUVA&NsmXd{2t zs=34*c*6-tb`b-zut+@t_(6{Bfhy3r`7^Lk+{26VL8Iwsr>Qf-+fQ)zJ?bYX9vj6Tmw`-J9Sy;Phw72kV{zaT9=nE^9}k?;oaWSDtYkRQ^DYYf8Mq@K{RD zUXlfWoSR4`d`DS~4nFNyI46e))Wn&fJKBhxXYpEo9OF$`U%Yf@!jda{)P|lWlv?y@ z-Sv|bZvZtx&OUDTS@Lf`A@8i_v~rOwkjcvemRklTtEQwH$X*T){7s2t2`C^o12)B{ zRs#Lgqj*ooD7j&+GvmAmTK6x8pyp`)WzB$F%-)M{ps@wOvDHap{x9a}7Ab6na<=6(Ku;W7DCH#QN=oO38%x1m>A&FUU9 zg6fuNr5m8!0A7q401W(ye58-_ze&@9I`jvF=5&3(HRmT`t1%bJnt)jFk5c%bvNTa6 z{ntnN-=UEzg#1tEoVWYxaT{=jMkvp~@hXM5%Sgl0Y)~*s^E%5Wr|e1jYC?wcQ4Wde zasjIo4sUH=e$Wx+UvhU>TgJa@T2xe90I?Iubjhj`?UgOImh)%iIrz9eTq5YWy%(Hf z!$j;nh`S(P`Wd?ha4g32*>T%7ZLJ=8-MF5WkXa4_XFe9J#7tf_%^@rKAH$UdJFh>M5 zS~jMv_CT=!=J2;#U*G6fKr%nEjE(yh281=9O0y(`X9O$UEpzt0e{SKW?Gk>i75(TX zW=#lD*k;~_loL`r(nq$q!!%?C0mQ+Xpqa2!knm?5q$Q7jq{&G7HeU{K?SNHrbLci= zx&?F|*{0QE=YCFyxk&V;#9tTM94SiJKWgv&#+b=t_C%|dIr%@u*GOHP0J~DSLlkH- zSOR_e-nM*2#3)pfb=QnLFTs)UDdZrfBT9Al`v{ohADHgt7d!;kXaH`~Tncj`avk)y z(v=Mya9&;M?qbe_=9Koou^I+;jI~5{Ow{l1J}(ONd4sbY>s@)f>*~-=S@ESj86m~) zQ_Xd*;K1I*n`TQpp5G+6RewJ2(Hv~~nL}`T8qot7e82@m^_Xf}9fUV= zCq%_Hb4wqmLkKv?pDDB1p^IxX_gKF~Xg8=3eS-TIUy3$oA9eSJ*rci24Q@WIU|#B0 zyA{K>A}{mg9Qp;!;&Q2fVx4LDR0k|k8PwyXYVn~yFO zTSeQIN3;#0KnFYt`lgE8$r@E(08NGIqj7iYhnzIhUK*epHr_#x>ou@y?L4mH4%r3$ zRvK%XM?I-?%asSk*vvhS)h`>EI#Mxjew*(&8l&l12RrS@-+l;vx^Yk#ldQasi8<^s@yiguOZ?UP8y5DP0OgD|mpoCdUthDhqC8gdQsqHRWv zTzK2>%2I;HQzZzwO%k{1PiQ*BXO*Npsovt(RrRLo4sfq+UkkL?YyWwD6@bfW1q@vX zivnzf=;WJn&_56qoopzYQ@wkT1zy^HD|MQN`l!qo&0y1?E0Qk0^LJU{2+9ida^y0` zJT7Ripu@f0&ahnp$4RAF%T^AWl-=V8q%wShyH0;Yc^WsPpz&OrUD=U)R>hM|OX6MrZiEbt8g;r3k+HZo^;TzPhFwN7mcW`_9D`|G*H19H`vLTX-$$um6zF=aw z#G=P3d&^9{EPoM&+Lgt$gt567+fT@49Xn#DFwKX(8W777(1b?{8!sc8!FP7f!UYR1 z=9%hsvE>i2*wT3~tgGBCtc`I35CaVac68iLc_^tWJk`3y?>*tC6WtwpORL01HRnzU z`y8iH>^P?FR2yHZf#ezkJH>-w2TiLZt;u$qrXUrAcW8;s=6KnHF*x81md&ZIG3dH6 zh>2p@D(ZO;hz43hU|Q)n<8sTB^kxTevh58C#yE`D}Ws zQBl{|yDQN9^tzbn1bk$lV{BS9VpDx7BM1q)I~t)1NjEIwUebwyA!JlIE!GiU><9+V z(`9W0R|kpx=KtZuS1-3!CfD-k3<~7dG6{jt1p|5N@$5^BT|;$f_=!Jsd`AcUH%f>z z&6WaEwG>tf^f2HjGDq&dmy!qsdRT-Y4H!0*m5rUTbUXMgf`^Zo^)NfSsSEg(=F>dl z#2oI@B#a*weEJ7OkJx(nb5MTa^@weIvL)$mO33ftA6VN<30v4=Y3|j%*GqCft90{M zSb1UaG?CQOQCp}7QXOkmG(v2**i2_$b)_Ya5zQzmX*gLUPA=Fl-vd`MtbR7zXwJul zuMBH1z0#u&=2doR*|wDv1Y?_Dlz_D|yYYwq>)rYth>jpCy_XB$ThjlkY(_J(aw(rA zN%PohV+XAqf7q_azh3(w;>;!rsVbL@9C-CsSd(U| zH|FbftU%q%;QH*!2A`xxKG_H{*}z!AT2(iDiAsRL36gVYdvmGb>55A*Lk}YD^S>k& z%57TP$}G+KydTIfFM`_~+D!_s63@>Per5rbmPI3ky+0>tff2~4)J%GAV^*8?*g}P= zVM|fXzfQ`Z9%LAp?OQ5l=JM|we7Jx~czOl1K7pWmpzbdBN z@a`6(A%dxdxb}dvvqav%pPjdlE_0E=(}HEEYSW#GeY zJNAmwSoKgd1%X);{g=c~VWmw3(2-i$`1c z8CALI7$|@0KWbCpwAcgUJ~NB%nABj}AIT64h&>fIkE5025R98(;Q?bnO!38bQR*sE zyTHk(X0!M19QGu*q`}uo0}LQB#4?Nss{msA^g7AMJ z*VZ|Ac`6%z*xp4RxEME!4R1Bm<&+EN&kz`L3BZELJ!n~%3K zTQt*kzRegg6kl03yLQm;ycS~9;>A+(_ntyDbO;P&4HQL}{g=ivz9~tY%=Rfs_z+jN z(UxIp^mZ6GP*}4S8@0-&Drn7KPc_f@_^3z1(cV+()d zexv@a1K<9Cssk?<{puwQ^V$$v7K0ypdqunx3pPwZ z`IDr&Q{_|_yT89(=#kC8*+>OiU8HQW|q2;Fm^bJC*KMU=;p!Pwn~d zdLk`L^4uWT1bnuoi;x!aP${K3@X(Bq0NAjppPof z8(RhH8DJ>;mjN#`Xk`AMDx~B*v(O&VY4`Gfx6^K0N+aOi&-CP*I)Zke|5FC*m{M<7 z@KgOp5#R|)Ij2|KdJ+H1(O}VeFfHZJ_0J-n=vBQ;MQh# ziHO7-W{y~#=h6U0j@^C9LLBXS))N{UC})x5QhKH=5!mq`ZpFBN8?ftF{Fk!l5KlZ4 zX8oTy69Tr*zl)3=eLqV-ftA1-*+2PbndAbjUgEbC#=ZPA>&tZ0%oH*wYkKfDqA?+C zh11Q{$sqW?o!TF%0qDv9U!?|<3IQ`yX^!3e(Fs7WhDC=Q_?KV-U-1PnFn#5VeE%mM38n+ZXJ~WC?t_SIO*b0-`7YbV0xB*ac zAfs>Y&(foo)c5a~&+TkpRbB{Nrt zZM;^tq})n!t>tWTGUUN+%3pO|{!Djme(uy%PHCpe5Js)J1W>5SP#PIHejL|s%q%Qt+W#?sE+wmb}%xliw}kr2lf zf3a>^&!@ufVwAW`Oq6ulq?!SN@XK6wA-cBudfsMG8gMRigU*$|sfE4B0lp1+4KOZo z8PH#S({xxQ6O(KSFc`;8^|Sp55fXInW$$R;d)FA)YQqQiCy#-1Y0L}2^qsjiOlv*= zZXg+~)RAH5PIJl`{7M$R+hwDw$WqiK>f4^tGQl;lGZ9WJMZEj^mr$E6e{=J5Cf|H> zlK$g9xXQHF3XPj?DQ1E8pJ!rnv>P6yD2{Rg6Lg|_YqpV0*|z8mW`en+t5SiqT|;ZZ zQGZdTgY&An!7!c1$eDKT0PgHrlPcl9vboJguxVYGXzxHGCZ33=i6@9h-Uj{Za$rSr z+!#2B-G_rA_|X=r#`-A#*!!crfPw-X;Di5BL-)nTNLrzCLH{Xde8&BmUW0DPeTf=X zi(m!f!hRO@;tGY4k@#Fz|z*sE%K(S$(aBgZN)!4Gr9i{jRCX|RgW>lS0?R+8d{^`52(zVlg5$OSQkUXyPWpcbmbj7&z!KN~-+?7| zp`K6Yh!c5-cRr+vR7|G_`+u%Fv$=UR!Obl`XvHKgQu2=Zr{;c82Ms4-zq7X}zFtl` z`qrriLSJs>AeBHE}?-}4R zh?UF%b6iop*6r!lM}p&1&QfRDqgCR6O3+?STAwq^B?mCuwk&X?*Et>kE}G%XzQJ>q zTVEYd_MT*UUSw&Qr(q$4pjP01EqZf8mPX!fU~F-HH=nF^?zKXbgf}0=C%1w)*TwJB z6+k6cTtgZ2iusTWz*$#O^u}AoAAi-@y%i)%lMekp1c${2up^*5jv?!GAeMni(92mtl*;swgBm4M<)WbYG_F+@IXWcgAz zG~UrLKYg4MQ)&Cd?r&1@KwQHH7+T6{Gm5T6wazP~8vina8?e;pBzRD!E9;&1nk107 zJXQ6Wl^h{3Mxq=T=;H=ktWV$d8A#Lzv59>GYiO{|G|6cckR~h9QjHaNcO^&q>&hoP zoTsb>GKyq@k}GA)JMp;gFA1o@cFe#fgHyu5u+n=j4EPH~@@B=oX5=;zH@3f%CMWX; z`pWOV89)ubHCI33c5(P0;^xT7ZV?;5V@^N;Qwy>GGMV3_#$*L}XqUjO+Q>#SmyB6) zrw_k~9++{nVyloLj?StPMLpuwUKxKC(EUWL+8d?DHiK7RvA2Uqd<#(Giho)6#E#{& zctYH_!fImyLVdR`a5-A>1k<@BSMDfzDBCxJedbKPUGXn8S8?K+@Kn6f=HEz~5c*L2i${r0zy^r7BoLWLvm zeEWU<^MhNa_s-q)`s)_Mhs&H~VPU_A&Yz`=GQP+E-Im}ohJ3Z>l;7J*Wtw1T)xDNJ zlwk^JHv>C6YYa)6L((r>pT!3AXiRsnES?a*YkjIr8n3yT+nLNd%t zhcH6lH@=>N&Eyr^-R@0$x|W%tHjKC=fP4Kt<=Q27{0FF+S7NiDGtQ~VGNqoMJp zVr9wt+j^^vd9E);c^c6|$~Tq9K73UPH@Bv3b}}A%EW3j&ZCuGd_+m=ap^X$sl5+Iw ze&bXiW?!0y7Gqr{qb}2$tmZ$v3PZ7Z903LxxUII+Oyk88nqn1;s$6Lpmv?q?V|$f* z1#Kfr%r3*d-5AzH5W+PP9`1(xzWj#Pd1?pH(D05bKzU?osG6U z*9eOg4mR5MZdE!IE!EN0=L4jg!T90vliem@02Y-v+Nk9MG?0$r^st&%6I)Ru{b|O5 zsF4X$vA5pkRpSY6TTNl7d#5MkGa$BRccxXiBTgD%JrkTM=g)d=8=HKRZSOOc-H!J6 zpnW#2BuD6)S;_0b#5wfqgfWdSD2CC4u~kdY0-Q;bblJiYKr2YuWP0~ynCmaUq6r=# z{y-B=!hRjT#{L!G@le9?&*M7h6St?b#>kC=% z_s;TDwEfPr-W{5oQ!=u}to-R^TMACr740K_tlFDhSM&@wDpVG{;Py0!6>APyJe+-Z zruYY+X6#@|+OJ$QUV-2=!D%opZeoWM4{TDhEUIbJU~^*A^Xb_jBU1Po`IyG&J1p-m ze3!?#xL|a0osmPN)j2@HDG|s3B}-i%b1?Y&%n@jErl+?-$Ajp}S)g&CmpO3=F+=2$-TCBDlTCf)|X; z`2l(}w^W_5rv2K5Lz|-sw1FmYqr6hnp`cVg6{eOqO}P$qD9k7);<$$7mx6#xSHItG z@4L$USy#jdlWw))o$4JxJx%X}V#z{;67EBp*mHCH=JgX8W4HUozM6KP3A@g&C!b=9 z#ClBmcU^K&O;YkmQE~*!M-Xf;d(297l3XU}_SS+mg-~%dvUDgYt@&A1^Rx}P1>ICx zMXs1N!Lr8g>_U!J6MH2(mFWK6D=zkB)clSmkYn!*>xl@@!o>J}v3Qhfhx=|G7q$C_ z_wc*ReoYxPr!#q1XV93305x20rW_uKsr?-K*C5)5(cG`^^tvI0tEyHwkk2MRj`E|&P2h3 zqdhIa#_hgIhRVfqpiUsCJCI@CI!vM1x+)>=&ap+tBZ4JMM{*+O(B`VzlT$AC$4U*9 z#rZUrDcxPSAXU6Byp{~3nppeMvJZoFWywq~ZztJ@hbz8=3ttS9mh>AZsmWLP>nOt- zM(%v3#VzgVDipESXHp-kf=rrRT$XgeII5d$WVc7-^iL?7Q)s;9eG+NBZ$)I5P{2}fWp%nRK|Y|p8o9$W(DHJqj#nivuC%wZcipi+Lu7CMtAOp7aP(&( z>~m@8>{ID{9-3E%Z%e?4W$MWH?+-ThD2KXu$FIxeb}^h(V^}LGXhKvb?haw-qN|bi zlzlfs^LgXA%3ct(mey9)+(}#siruisqKiA$ucB|Gj&`uWymZfzvSKm42Sg%H6A_BC zZPI`8{X>Vx$@a1sKYKg06%MQ-pmR>QUHis_k3Ug?OULE|EnBisB zc)!;ZOBIKX5O>Me35M<$(KL(==INU~RYH1Ye;6UtDtC6~l_~Cgw7iMt$6T(YFfZS# znSYOUtNG{McC)Tn8-XOH$<~bdbCglcqaQS<3$%=Q<3G*ZC0q~EtH)}kIz}4vd=-5u zelx#%^wzSs#fDet54S`wG;Ez_eX+;5)=&1$p0rC}Y(5EnXqI$n2KdgLYh+he$-ftw zI#julM;Xenyy9fL80}eb(ak2e+v@jQYAvl5ou83S?sx2SkxQ+p{hH1cRX^8zB$)hU z?vK++)WMa>7wHe{8_z{23o6QW+Hecpdsq(JfbV=_OHxvrRR*^NYUU4(au#4>p=HxE zKF15O{`%Z36#KKg11*l#lMldvo09!Id*bM7=e&j$!EB0qy*k`Rgp^Ht(Ygd8y*KgB zl~WEw&m3U7 zn_|33E4PfkE_?VDvlX6p1bG{E9gPbQWs?1qkTehWT#nnJ2y=;G!Y)M)c+CznP9 ze0!jru&~&5|Ds1{zSHuDBfHmkA8+1u8Pm~1+6pZiI2dpDYLg|IIB>-amQ8N;F8YUA zA{m32KT^v;QqXjSd}&BhkC|bFsag9o*Q8STxJX$T;vjKYLsXA%-ORD0heh%`W#4BJ z^#~NxBPZ3vqQ`v&pVkjKzVcrWKAc9=^aiGC#W%kH@Qjcgrr z30JtgobcZ4l-p0ntZ__5nI)sh{jMfePeX2a*Ihzp2|HXz&X?QI#3Aicb&mbchj z`uK(3Mf8e8&J&+BfxA~X7MbJ#pdVL7h}#Yqydk+Dre)ksH=h2L<(U%W zRQD6lp6teRvX09|PQ|q>i~)s}PoA89&UWq&wjg|Q-HciP?dyb)hd*n`8{~mFtUWoX zU+cA;A(X#y%eEsK4eijTt3Ve9t}U#BHA0?jar`4Fkkh0b%^}9!mF!h-%U*BiKY+g9 z4R^#;(z@^sY^-DN%?@^;w&$er4+^o%`I?sDYBwFuTPj~oX(*l1G#Z0DOv-5z#d-PSJrbo{8WpegF&4R!{Y)4of|rx1${jm@cZ#~D9Q zBRnNVzZx?SK2cL)#ucm1!8T`&nkVDgjwZ;4GxCW^Bw>%JW%5z5IrYfWmjTl)G_E+) zuRHa^3molgBpJ%3U&A_C>L-}o+ska%`XGKJ4_V44*t{Dk%yL38GiX#%X_QX*@*@=& zi6^)f8ftlpI~Qme(nt?y9tr6sQd}0gv6I`~lCHX=`F-oYig@CZV&3D%A&qEwX^QSD z?4?-QW-gylH+@=%`_-S7TNG?hFqtjnU9uiEi`UY#EP_YhmZVATgDw0wsW02L_iR1! zBEK}&Gb|Qryii{~Xm?Mqes`#NYt@UwK}qL9lBNw{ji=@3B~^kF4dS>qB5?r7PGhnP zwO%%!^KagRtRaNT=dtWbJzh3-QGSW^$$X=f)+jlcfGsbx)IuV#mU7-WiP<@BAh9W>3++ z2R_!F8i(Eez$I;D`}?NfgQEPJ7}LD~QpaRIB2Fox9MqX{*5C4F77u()V46KT!*^o| z;DsyTJtpDbgkfZ4vKmuXT!w;18u`sQsQLpcJFg}vQX4GfaO|f*Q}1*ZUn>lI4{F#n z(SP!=;wv&<$ zl+~f%OHz&aTQ_IV{Nml7*tjjLh!xq?3t0HGfueR4Q>VhT+BB?ri%-KJ;2b9u?{b9& z#1+$Wv&lQ&NHbJ9ASQP-)dNXp*HYHcnlSK{M5X}A`=qjFURRH59GO7YRj^Zcq0D431t)9rgIBvxzhBFS_xL-^$Oe)3LpEC@~+`6DM-)rSUd1-~Sjy zZ8+r(zdJu4JYp($IMhURZLy|{do-NEzCe`^O(sz>;zLarn|ZmE7@K*+(hy`;4$om* z!mD}w6>#kyw%9l^`KlR!)v9cAXV2a4#+dG9it)$x@v*fVYE)J*JuT!4 z2i!UW%*|&!w!#ur{^ahFoi`(2dOn`>ibp>K9$B$sqC4JA@1|MJ6(bIljjoFz#)u#i z?7JcPnx=Wk{UVp86;U>dy~{IU{B+ux-+83ws~2g2m)L%o9XVptSRlIku;FIu&ct3L zmD#4iU`1>3Ekv{(q2x*>%-LrH`l{ZCYiEQEyr~>Xm@gLMs7`OzUfH_0o-utBw@WDk z(4GU-r0$eGVc!1oN#ALxRZFlz*`k#?JDul}1n5ZRQknRUCR>1PfG#3;kOPHn`uT1( zR{VFRf=6N!rL(8X# zhcUuYEmbLF3-`Io%B3l>dMeHQyf||Bi*M1x&SMwt_3^8-4?Z@N}vqSHT zRknaQIwdjjivcJ#SmZ8o<=T~q0f8V@mW6rV-XGU!gP|W_1CkRnbK9TvgAl`m)i9fZrukejtb)p8+!wSduNd>yv_$hm=1Xj14YegDE&6NSur%v_ zw}6mKN)4aBa4iUTNaw9~eW_eeOlf-);ECp@2~Klv5A%QI!Yo5Y?J9fUjjeMmV|aAU zaWyx0rY}Ap0R&wUNmqUAEwhwnr;Ay4o_g1Ewp)e8=R{2J5|N2nZ=u1*9D^v=nU-+b- z!(~66Q4*BI9zRH5Iq`n|W+~yTfKXQuV6aFu-q#MbPhsY<>?)QfFBnNCM=10FQOmSI z^PZ*ha9^+l4ELj@IZJi{|Fxv4d>vi3+a}mg6&@P^wsn=q ztB9iVL+6rRV+HNge$)A^-L~ectuIP9?F{xO^*8Q+CvmvVW+;d7p*Ii+aJ*de@}1Ar z)$=|$@d=JMD}UD zU=1=1Mk=C56;^wRp(%2Ew5fvqqP}vD{=|1${T$B6@E)GRaA4#1vQX6UJ*G58-TPz3 zd+Tx;)0nBOs|=p{4L4B=!`m*MOA&r@yt>5vp5b>9`R-t7Ex&LXPip#+##DZ@cXjdt zwmL`fz&borO*>~Q&L(x?UJa4aWQH9T*NS6T3x>MChjL@)gk@79sRGVHx18gDP^ZZK z@;Bw+3gasicjV-E>}uqMd*2v``PhyXvTFxjmwN6X)txRmD|k4{;|{4SNNZEs2pD`D zH-FDIJ)emBB7%eoHN>My{+tGG{TGaNXuJDiZIz zKb|C$<<|7I_;N|!qHdQqeWny-9T>{(>>V5&&sBBpc{qY|oBrj{s~?PELYj%YaA&Y& z_BW|Fcrm8?W?msnyF{a;VTxvrIV)fLL?6mj*9erW$6T?yFA}A1ul}y%-y`4GGSWQ9 z@HKPB+Ty6C?OiZC%^#rWz}QJ!T7~x0W*#5t6vB0Q7$-~ReH?6LE&7`&szU9ak#$3+ zxK7TNOEWKdXGR^+Zi{FRX;iy8&zOn+l!<=Nq%C=-31%pyk z26>owewL$->Mb};8T1S1J*@B!wMTUq{cO$o*W(oPb#sOpHapHNZ`HTcJFb6a8$!Ic zoec-kJL~TvEABUcl%>HqpObJEk?<%u^{ISuouzrS*r?#wv&T!SB~QXwIN|#{t95+= zh87pIoAc!0nhX0hMM^<-SN?G%N`*2eGR^lgkx7Z)bAvnjA+C(#(vR)aC_UUdv&Cz) ze9yN_>f_plyy6_+%qksr1!wXg?`>F5_7x1Y!>-i&>pq5vr9vf6y| z2-VCx2chK60U0EFua3ESO;YvW33Nj-gmgcu(;GXu(dH#$K1pHsl9znhEs`k%(GJ`@ z+!LGzZoOOrV!)meq?r1v6}j|nO04{B{Lc@UR7f$f$=qnfCoQ?G^WjE3PnGiXns3aj(im=&LMUip!-bad))?fZPzu*GC&=F$V z`|x?V{AFpw1$t7rH=9myfS)v$cH`nMGg7AJU?O>ik(qazKCK?Z%X?M}eW{p}grr>c zCxHP*YZo=La&RvZr}T^b5Fj0Uc9^9tT4kmKg>hq7saWRQVRsfk7mx+m)eEW& z?=5+c!nj?ua~qpk>9C9!4!>t!cP^N}*<4kA)H6V8uNM9^5Jf(-MMfW%XY+WdIJsmf zM?!LeCOnDPg(QW=&HC#_+1wgU*4$>}80i>ZA$x@mp;C5bNI}_B{UfLB-=339c~Wl9 zZx+?uCBEXy9dFEdd9J)UC$(^|-L_l$OjMQ3UFX4xecfxD&jy!8QPsQ>8QCX)-FoY~ zFYz$$lA9K1#jSb01)8mGryrL8U;*TqF9`3xdaC4p9(!qW&}bc9n0e}arVA#aF>{Q( z)hPc{Y<4)@V#CjZFf@4N6TzEa%jR~2a6PLL{)*`;N&n$tFaF)#Kj23B!2mc}~1&=G)`5G9nIKLCpr! z@KW}RR1p5<#h217_R7Gtp~pnyTfu5x`=NKLrXaSjtU@ib9d*ZYoxn#8f*(+*BXAshEZ=uP4C=lX69+7v~G{X*u?;2Pf|OHJ2GMTLZrS zKk<-#K@{2})Y<-0`|BP`ijBk^Zalr7^8+|k&Rr9k+cXe$^i!x9Lx+;zkLY3<8e5wn z8MpSki6pT>W-NZm1G zP3j9hA-~BwMq#B&o}6j6{WAQp&3Q!^KJ=_5{d{5N`FzB~LsrsyJZ(#$u=QKQuX`V} zwqF&}Z={5H?r%(U=tAjufIJdu!*`SS9K8G~Tey|Qm*C5>85Yi+^G*$X=UF-A_$-=nQm1iRgK#Y8f<)fHh|b8GX&&VlDk$MBExk8z0x@Se zD>l5Jvb!ysFY>NXupy5QH3m1#D~;6@H4Lo}bsNG$Pz;epXz_z>)+lh@sSfGWG47asbyBpnHvBw)cb`XN*r7wI za$QWbuyBBJ-GnErxv?_nS2%v~2yIJAJ?j6@%H5&;nvoH&kXv?Y-@4+n2PJ7MGzp@g z+rRnI{U+V~Fwto5Cm@ZNbxG~|Cil~^cS6fr^BkEA`SoyosI)$dj52Z*D;J(~Q8KkP_)shyeFuEScriifuw$UXIb>Geinf z^V}hfJQ>^P=RdJ4W8Q%XJu#ooDY07t3^J*WNz}6s->?$AB0UOsK5E&_g5a!m z&>sngxPDaGsO9akb2`wtb>(yI;~OIKiDHx)2^H$+mL7E8Qy2R|e_7Vyqkzr6nUGC& zdL@sp!);ii_-J2nXF;MY=w9lADme3*;VIu)xbKOj(3QZm$0KLCj8*QvAa^K#NV9ZZ z9%0;hwJ~gy6!sVB=7g@UrWWC9$CCM~67g<*YYGk_L#l|GlmZGS0k3>z2le4$umvGQ z`W%+@DmuB6gBkfLg(=V5-}F_yW5206D8=zvEd>TX7M8V9{^*Ao+Q39!qoW{u6RNXU zvG)5tBj@PTGY+p>JuFv-dX-$$C%QnK1Gd2FXMOmQQMthDcK<1SzYF(?#unE3v5zn^btuj0JAY~knY0pat5N1DsGaq7EhEt`EF zzCEv4oTpk7&q~SF-;Xu9vg+=CdlsUZw~%<@i+^z4XviX!f$=g5O(`S(lX~%tjJrdX zN4V!%MHwmD5wcg?_QfCZm@Sp7vX5-qtKk>T>Tj!QjM&y;<7-ifN(^|d9 zHzYJ({CuBrRU2KMem?w+-^cCgmTg(Dsqig}DZS(mD$Sz8!@Y~K&WA0}ILc70-rMsZ zOp(6B!!;A_bV~fjXaSOwu|iAqNrQ5=IZxWRL*oGPdCqp5DV%;BtAk7s#_@7#B`MTI z`+BM4?Y3vsl=d8}3igmN^qU052K4WjWeqD-31rOeX*9!=yDznu?z55XC5O7E-`mC8 ztFO`K4plUu731BKgBupmoEUSRp~0@&1yz^Ckjb!JiZui9Y~r&DiFScijcZzIm)-@O z-(vVkKRHhC+&!+wRGG31w5V*<+o38CY1;LR52N{CowN)pR|%cbCoiGSbl%+1a!0S;m%i!tw*%mU!NGfKE7pHJ6E34AY)N)JWBUzaqK)FK{Lkg4H>_hO&u>3iz$Jgx zC!LqwJYv_-CC~0gQHv&gBA2ngGRiCnH9kk5Gh9TFj#5a))vuYAZsxUp$|){Sp;~w% zeOc|v5E+|zQ2+F>dF3Tzud7Za)WKh!%X{6Y6vvl7w?2g?e)mik$YS-Y!j<*yuBf86 zQd`YgZey2tX0I~q(r^v*Jo1KSuX$(W!b@w(x>wVM5_?!;y))dVfY?%LaBfl8a4JXA zA%nk%PCoVme*D8iVD<#Pk9{ah7(|jOCrD-{wE%+hm6hH#{b)-2_C@U8`SCX`4y_I0 z4sxjzny~Cjesuxk58!smZcU!`*|;-b$@&Pq-Ns?rhZFBg7rL`{n&}UCXk6ZE7H$O% z=`YSyP;d$Gzr0Q+Tl}nJQV?x% zXKQxtkBb!5TIS6)#GQ@hIr%#2`e@l3F3GALIe6<@IrjU&UTD*TR_*m?2%#vE!S3nK zi3_>@Vs(G<6F{5QO2wd$Ejl@my#bGT7azD)GnC(iHTkEZr1J;A|A(MJof zIV1?y`>v)`X`W8V%H` zAR+6gDspL9i|=Fxa+2J4zryUI!OKz)2s zt1L`$VP%W)RT&l4rSYgyG$P_pzrMU=mncjQheV0!{whNbUQ@cY|3Ks>>c;*9v!uSQ zOO)Y-ejfx0Z|qwdrOVJx4uk3X+bgN36O+@^uqtIGRt=2t(Lggp3jnG zuwgx${p3Xh=aUMX5sa^bc%_^^lHCx(90w8w4KP`g`i5n=UP zAm6>1_EMbulZQ%lm5qtyO$KQ15DUqt1sB>-JWt#*bGq_0A2Kr1g^U_Y?ej8W=!0&T>ml%*k&W{s5Px`&d6pM?Fmm-odjkGFw4$QeNT%Tw?^xc*;dTt-U|s`5Qni7wR^$cC;-fE%MmL);HZr{vZz zvT5?)r=ffXj=UbWp*0EOquoJc&xpI;`#vj|49XS4I51pAIHJk_c_+;)^FfBnpyl{W zn0_KxUHb?$%Jy}Ob`(c|flQjm&Q@%n46migKxTg9*7-C`hNQuE)901V_wb>DZ)1cN z)Nn_Ezh8dO&ZE%ZBny&IxUL7!zx4KMU0%G`$+bG7_oRskDR0`~*l7ZQiasINU*udW zMB`3=5R%jWCdqA`3sE(q-U8n>jH90?l8@D-dXD(X1wYtXyv9vm{5x!D?Q?#(%;Ait zG}^$%f?Pk&BE^q%H=jo;wJb7-vL=i^ISrN{J>tFJ9o##Q1tz#AVlaf(0E036w5({Z zhU;JLG#}7>OqQFsQM-RFVOoh!$0iPb8-FmmqidOM+tqU|)4gB$fi8XcA~C_FaB7p7 zD?qtXBjy{*=5QmK+Km3atfl5WXu7z^{#Hj62q=oKDjYw?@GT}lTyZ_Pu!WpP)?sVX z@<4K;YetuCQpHTlz9FJgZh81EPAiW4s5^+SZGTiw>zS2JfUXJ*gwDU*k-bs{k|l72 z0O_Ei%FIyii?gv?A^axu&DR;+sMER@F(z1oa>cHsT~**3(lKq+(hvNWWOrkHGuWxM z&DCDNbMjA9lj!2>l+woz4%%F&Gm@fi$(JlEkXblr zZURo!@g{cG&5uocz$Y~4fN@#+VQI#yc8{CbFfb#qZg`UApY019xT3@?C_z|Aift#} z3d@37V*2#V=SNgJ9`A$WRl7uwzPn*^eM6gW-wRGO_Rg?rIcC_AUpJb?ZVHZq24+T} zGv%aT81*Vw&>;YnM!8Lputfpy5<#mZc`b^nAt`vX$2app)fSe=^`f1`1(D&8bv_m*{+43u}}+*=+lvF4mk+ zy~K{sUI$S^Au)Qu1NSbtmzZC(d(Et`@60|iQ>=oq5(w^65^KfNoWz+i*UlVu`@Gmu z3)^2CXB_6DP1XKnv!ii9KynEK_D1*B`>}pE2|MP~9~|wwpK=xS31k?8q-A{^&mceq zEHx3GMWQgGTQEybceE0uu?=iZ2`u0eP(1QDjRb+cebBP0Uy2IaTqDD$3P8V`+5dCB_>w~A4w zE>%ch{NW5uTMqlNuZ0`YRrNNk0nwPBEZsw`&Q?R#FQiuCt7YO6^O6_!!!Gqs(Ajwt zxGNlUhFb1uqwvUzEjZVOr(M;G)S_E>a0sHT#_U1+<#>2Tq64WGyMMZbJxNdL zpxPy-F2qZ3g@#zztT#g^sg)ndq+}EQ*xXF$kLZy9Xi7H=aoMx3)j1@_DmBTSZp(?Gh~vauQjRNcGrDlKL^IlMhZMZd6~tII`2n@MKY?Yr zT6D1lxPu2o$bv?j?6x`|t;!rzkHmWMWv{y5c!H_!8zBua7UAGBrpudLwHB$?@6t{nKb@)K*6E6X+`lcF%u6q#v4k zLwLap|4hDS)*T{G6Q_A+tmK#+q`?nx62oOULC(%H=O5!q#y5~4gR-<1AwQk>I5Ca= zm<8-eD&cV)-q{a_43{uuhrm#moAnBrJNF$abidg4xA++pUUcC+H%-}oJLKBW; zlywFj9-9FM`COBdZ@0@w2Pm?4t?!E8(&V%czId2MS-EZB82w51Gs}#4DKi)Edro~v zZJVDICKskI2sl5~RhU`$!FG`MI?;+1^u7ww6l%F?Ie8tjWmzh=!RsI(slWxSBZ%>R zq7}HPf=`QAC2DJ{C*dTYW#kQ8Fq-0ErW~no1g-Iy^6a``c%obM`uG@Bdsb|L}$PdEV!_@85OZ2%71(JzQM+ zX*7%7iIRS7qt&X z2lgDoi(e#`#X=kQ`ahmEHZ7`UF!4uLc=M*RvH7NIkW}8!AeVApCf|zuRNbCs1wm@n z|B!_n?z^p6(#O|)x>No0J08Y9%2UrV9^7}`gq{eOgI4y1EJl}fY(FY<*s4zCcZTWC z7R42M)PxFKs}E-`#c?+T(_bMin=j99&9Rny%Wt;Bysku*1%tyX2kaLW&lp03pAFSF zFbvbNebK173sW20^GHSJMcADdcUTx{Fbks2E1PwXf13;6q3JWLq~fCabc6&-S}%8n zjzcEzFHtrx_NPu9t4w6aQEFWwDwji7)mG0_;LB)oi_{uxWVN_sCfzs5> z>g{f6l3)~0Y{SnKs0m*(GS}AeXOsWQ=8Y2l6R=B>xA5og>J%>b!X~Hk*ilhkHvBlCB;=HB_qs^uo zOpZti2kGZwhD^b3QyC4{wj7oG%auHF74gqkWK%d+O?pwH zm#9}FlGxjqSU}v%WX#5>QTXc0v%i`kS*+)X2E9L{`{~|*e4uB~nWw&490GPrnO1~3 zD9!WvyWq;UcDZCjP+@48sGU8Lhlg1flt2aOe+P4(4SPFv>{4n(#R~THsEMb(KDOu* z&n!Mp>I8c-eIenL9T2}PmT_k^KDZf|c32~>EQ)Z9v=-qXwTOZcnB#56;xLISCxmFH zprbK=>MI2Yu2T)i1PEZVPC#=;vEXo7(^Mz$P+mgNk{RMY!&4w~&wDJ>xC>uDV?BUB zOK((}l0Vu27$<1%obNN9>r zNKL~#b|rS?8T|62hh2=U0_gGcs3#r_k9DwCZ-Bc}Y3~>< z4SEo2c94o)U^D2F<+n@T8QG9S*-2sX_?Lf(hpoSBP+%$_4m~GwVdF-}Z50ip#{}m%3y!iq|6FoD8Z_x)iP+rtxcHXB z7&RA+y?C2kA9!KpIe57DzMj80v6Pr?YfVA?ECUagW+m77JVERgooCm{ zWONPBc8MO%b@5n5%q5dAkV|zODNCL;W>Cu#t;;vCpUY&k z@638S>Helyv%YtcY#8b^@oXoGcECds$|VgENaim)2+zHEy-@oIR}w`sC;@guJ%SBp6%IC#6k`-s*Hj!^yI|<#WKUJ z)<t;g3Y1oc@G6nq5P66s8q3+B~|4)+;Hnra@`AlN*z8@TP5+=ZaIv1 zDQMbIn$PDGDK4Lpd5;yIqqoT87T{H(2u<^P(m(kEnUhkA42GvwZq)SY-{@p(`j->} zAjUG!JPI!-`~SURHtUMyjA<-+nJ$cq+G|4ZhmnS`imQ9-bB@e$UYs_4Ac|D3 zfPA`;?ozUGfF4JwU7T<4f4r6pusM-3#AY680(Ny$Z>fVC0(G|04|n>Ji1g+V~%0^7BM!P^=I`gV3LxEBTxPgq6SP`F?-n%^T z8p~&%9Z0c#Br`8#h&^gpa2%y*>6E_jLxf8~Wq0AKtY9LJPbhnjaFJD*I5QuEH$4qw z%KIatFG&BZAA(7(PPKl2l7D;X z$i5QKMkeen(_Fr?E_8$j)yXktVBU+T+Fv{sWY-AU&Kl3#u70#vcgZ%H)#4VU$ua1M zcdzN11>eCRy-}G!5?*4Ji8IUB#A@lQm|0mvWpAC*0vde|cM{f&p!4;5fWCUzL0BX% z)0^T@p)!)L*wd`V9eB}K27hl*?)od^g%!4zWuD$3w$ST`YkYjOiPM@Ti8yWKoo6t5 z#@Q#@SKEh-nRL~Xf0b*36QDrlMZ6)cJ|P$?6lu*XZOpJs3PLq`2Vbu*4b#!NTs@Oc znbf%2XSjNwasYSZKR02c9^be-&HJ zIJ3iU;?Z$o$-#cx*AI?2CcQixZdv9|oTk?p>3Epy8FDl&+G})>0i8xt1MJ;J3Rdrj z(*T3Y^tI@I2`)68>S}0=fCSt1MSHhJcjLrq9b(eKaYcmzJ-~{bsYuv5v-|E;K{W zKMeeW;jZjJQ4RDr?Tc+YLny8;7<)aw7_CJX))$(+W>C!E#fIaX86yr2MVMX9L@#}* z_*CngE4C5%^ zr=1)8dE*am%E-dQJKP?|(~lb5_WZKyNlOpRgHho%hDU!`b0!!t61q#bChTLkW&E#u zaRGS5o%MFx`2!FGA?fTZ56BsqTftV0tf9iEnG= z=$ZGn3lpe;e{0}zjh)??)e(Bnvv*HL&-!;eP@V9TnvZG0yXVW{03SX!sTuWJqEBg! zl`CEURY(4xxWz6Ke)gk}T%`8y*h5Pk1wB(3Hg@`zhnb)8PX_xFpoKv(?O_14uilGD zO3&{+n91Q`Z<*x>^`5Kc;hIjqZf|7~!sVQ_^ty;aUdA5v@FzreGdSa%RHau#3G68!0D*l+=rVo{H-E zv_CwSChW?87A%I%8dQ!PHAk?VJC*l@{=kY{IaYxx(HUJDDI)ET1z`W{c8j-4Wv>?s zaDm*C#=3FwNAPL@6mMAhuMVGkd5S3LLj_2bxy3PWdHTf$eRTh4*m+XJmsSx$pN^c= z5L_iao%Vd|wy>T^FkXx*?xK@vQwWjHV!zzn&)P8{rqOIzoiaCt`?FjLrOpIonlxM7 z8?t5R1SwP5NW%HMv;5TZ2JV_8VNoW5(B?gjgu6Y%;2c!FB}j0-VLVFY?x`!AY^Y;S zvrZiK^%N3SH^Hvlibr?$b`9$|RebPcbRMfY1ofT#ablr*aYSl^XMQoqOm*A~=ciqB zOA^W7?#mMmfem=v!y}AiqSGSjsMI-l0x3lu#8fO=ZSG z7AF@DkGqh~c~OgS115cX*L>mvEy?@oiFf#uw9PS#W8QQVCxL*V44^;xK$+G& zCG`Xh_dYkOXikSPIaQ$YCg@<2k@Kp)=l7UV3*9x+XlrCq2Ve@;sU>tE$P zYOvIDk4wMIW~b@x56j${Qtv552RLRNr&aL`kCBUwyfrnJqCfE3@wpY4f}1sl|4rs4 zHt*9>rKY?DExvhKAzb3LbDxn!T#1Q9Yg5Sl4S(4t35ioyuJ^|R;-igjjw*F9m;BQJ z2x>^iA+F|)M#?P=MfpBzi;Imd^)8KRuF->TJnDip-7jPILy7G)5Zy?l)rnM~Qe>)j z_WaITH!Ha;qVsJnvmNX`h=)xn^%6N{Hk3bgvUr3-Lb}vin{LVGmp^9}_ z-I$8Mj7P{=>-H$#WrTc?TovgUgn!Vz0wjM_ub7zc`9wcGP321m{ryVg{o(9z*f2`>cHJx=@_=7qt-B3Moyup;ia7hja zr|DKq2k|ft>XJ}C}uq*)b}DkRwgK5mTbkL!n>*3SJf8M2XxAlo}0&udxB zjd~*a`B6`|;*uv`9z?2@>lg)w;K*4v$D-4^ zK$$6||GsR$e6+*@auQCltwe!Puojc3bd^%=jTK7!0ueL2%|6G~YEGCU zB;PZTeTses|Cs@4rrJeiMUHi2@0cV6I6_)qF?!4Ejs_S1Dvu$8*iA=QFwYNrZzA~k zIpugx+bl>MvD&=PT>N8eq{mo6iIb#8Al5@KY?dUkOKAylng<+&nts%J>nf*-7A0u4 zXDt{EJK`fsw3Id#r!O4~9@@uqItmRg{q!^u=#L3kpt;J@;ojp;8}Tk(jHM9-2)3x| zg*O*Z@@>gEv=}pI2?Mgr`H9N5k|te(Znw}jQ0KTLyQq=c)8Zahw-cCK1*na6mttdj zg3H;RBhuZUicTJ~qJ4^|W+nrIR|C$Wa46L_) zFkGfH0gQb14o*Yh&Q&jLcv;(_QIfe2=`;{1$rv}iJEDl3MjOqE9*&cavh(VaU;u2Y zcN|=(-x%&!5m|X8qP4W`sIJ#fKjbDYn>$&DaE3nd%4N?MZt!RfAT^#W6Il1!$Wr&P zbu~M5s?@!bXYf<(c5)U=ZLGy+rHvLAKdQP~m~d${$>#!FW4KI?@yiuN+#k4pU(XLh zLpcee_rg8>6o_x^gy2(tjpo;k9 ziN#FltAhETM_46m3HsNQO_dYF$FUOZ2K#QChV)UvzkZB7aut_}N^jc>klI5UvysmI z!INpp=j%Q|Hk~`PhC;X$o?^Ty*HQg4BsRozYL-$EPGTG#T{(Y>;rUwNgaLXK#*BG= z0q+ndip6kFYoy&hNtf6F1MRbp-x{5N+*b21Cf9pD#N2WhQQV-GM8+6IqcQ-`iX}sCDi)ctoU;NO@H@| zZQSf;ozv8vhDQ!SaB*>=1dEm;E$Pv0aFaS^E#B|P%edciaF70v$bUnOM-6JenQYH0 z>J4yZ1oX`V%B6K5vPQXCOJ^+v?9M#(>Z(^MsNkF1YNrP{LK~(x^fMMd=&d;34g+)Y zVEC8=ecgEf8dw9wj5QKC0>n>r?Jk9)Cyg12X0Q0qs;|Yr#;I7_lazs zG<%P4qG1vjABRz9-2rbM)iT2S8eFLi9zB_$YH91$oI;*o>~)+}U5hnkUf%cCEyd7_ zdhtOH2{Yci2hbqtb%18+NbRFBgC0=B!X~-i81MsJcqBzy>=hEMvBJu}Pe`_FlAXO6 z5TA4Fajp=k(qh#L=3b1dDJ(K>(+G5q4>LdUNy_GS#-PkHREdd68#Q#Zm)zu*3Vy$mi z0UL|>$-hCiF>L&3z99WAaz*)=yk50-#V_ZUuim5zQ(}^^$}cJ1%CmYRN^t zkEpl!D$w2zQ#g7uD>|>Gzm$~M*k1_`%6XVFVhJ$kNfMf${&M<_*Gl91zV>>=c+~iA zsLFH*x^zz@m6%3%W6P2hmb^GuQB#ZJ((FnWPI0_xkW6_uRWCj<;tMx8YmD!eY|BMsHUhH) zH17ov6{D{_k$Wl>gen%9H@ei zbDy)Ow%%RzeF^)9QtsKzh9~q(%dV3%K{gaHNuNQsC`ngbmHfX#UE{K%O2<-$joMdM z1$Sf94vwm}KAHqyxuaC1`<`llQh}mh-Nd~^v*PBqN&$PszMMFAgjLKOpfN6d2`CLy zQPxj+l7>Bh?reGUC|C35isvzGgpYoS6G}sNb1Orcwa%)gIOqN{dd2Tx^3E=VJS|#h z`gu3hOaISHEo{`_iL4-Ft!F>)QJ-p}Lwm0`c_-BDAVm`qq{Tdqdl%P$4(-L837 zGOll$$ses!#^Sj$DG&`<@P|8-oICT-c{Y~rd#?arpI_v$9Qw95Q0a}XLyUt|?^AMOM+&%8F6 zoj&SJu+UwH24^Rv*1gBLWL(jlnnnoTeFvgvLa=Sh4D z;XLK`=!zQ6&OTtI(}{ys0b-S=gowycQjXVtam?WN1y#M8S{rNfshS*FZ&d8QhtTaRkN_>alaPnGiKZRj&uU(LaeZyL>_>m#=ld^TDjV=fu*&D*2e(M6XZ z*wh}sG{p6gI?R&FFy$&(md<)KAd&D5nMEwbPmjQ(=uLX?I{K)B`VlJe747;_Nby0| z^e!VN2Z-^G^1Q>XET3qociS4-|cCnuuV8W=YN1vl03FkpQ-I&GI<^mtK_sQR<_u+fMeP8%oWK% z<8g|^1}Ln8n7N!z>izE%wIn3T?4Eg?jR1eUzP&(68>P}l#9UUZ()LCsZBZp9!fFrw z`p|r$b-G^XuS0nWs`~rvYO~3z#Rk4u^NX~cjDy=-6N$kY+E{Mt`cMN46V~nzUmlDX zenXH57wcobbdz2aedd{}34C!lacwPU92T4WI2L|eC#h>kqVIcStm)qWa*H67Qn(=S zxkp<3=-tf7$)cBA1SSR2Lb}@MkxGjRH(JT=_(wWFiyjs8YK+W&OnX}VduOUVdm&Q% z`c&p*|1+uDwf4HOkMCK6uOO9}=D1sL**)b4wS-nCqA5y!I-~odyre-3uq*g8*T0EY zUgJ};x+h)H`y9??ALplNvcw*=68+N11CH$&lDx>lza}|moM@LO7ghh9#K=jbx5h#MBH6&Q))wo!c|X?Y zlp;%(UkFFwxq|E%6GfWeD3zA;fTza3zb%#4708Y;vlPiBwjZKWMT$k?nDK3xt9a^M z^s@HI{{2TdpNEK9z%!$YHt*Y)%>f}KmCMEnyr76@V$mJ|B(4OEU9qjwT)I}hF()>c z{D8WThCz4*-AxgZG-t^-m(V++V>wr(vNMXRTqG|w(G$#*e zfd_?W=$dCZ#H@w8Z)$v1F(6f>%DY|&l!qTIVv7!Y(JHAR59~4T3C*bfQK)P@o%uHs z;42viuN4L~N?IoCeiTwFa~@}CnnSwt5Npqu^@ofM)`$CF{DmHVFksnj;n*KPa530o z;N_}N$KAGi!S-L?pr9xs z-hORIZOjnBZ10q&<%f5(_*u}g+6GMS2O5&W4I)5A-r;lj;YHJrjQ{@*DP-&#~=^)o*;QZRj z9*C+b1(7z@sOKiMUamlI5O0hD0`SuH%vg#Hkp>UAyDkd5?pUpfY6qSJB=Vo-o1kFz z1S2U|YBeyA#(ELSj`d{kNjkKhzF8I4!)sT7H!Nh*GlP1lLFjB5I#bSh$og&PCh!E! z7g=F}Zhjqv&M14Fqi@ofA(b27u+t^-n)5YLJ#MTVFZ8`VP~gOC$Xa9-=|MMH)%u;l zDO9QzhVcH~1p zx_bDJxSu>3RVw|hKEVT|Suw0(gzCy(8YufjtAzbF~J(t!(-^Fpda#AKD{GQA^E1hRQ_J{ukn};iP)4?Cc=a$k-q5eJDBvxYe86f=1Pc{6iQ z!i>YzwJL;IpM?>2CiMb1&K2&Tz1x$fKOfPEzDc|>PXZRenP5F{y*P%W3Aj&nt~qBY zcaLQksrR}xJG%$q7H(7oNPN;_QWVkx80CRej-@upBfds1*HYW2exRfKtUQAe`O@WS z)uhjKrRbn>&$L@B^X_BywRLj@@gQ)wH`oP`$#YA_O#TDb9GuffHAL3Mk#Zflaib zot3TC>b|$?ta@HNl_>I{I(%n}ym1ek0-c_G zX3zqcj)yl+OeAE1s?)1^E~k$LEny$j0)#kg$XS(tOyu4OYzNC1qoD%R_MxB`ubNi* z;O3EJraiX`2dVHS8$>@++37p!9-$=!#dmL_d7VYg`~xn}FtxMW*ZYq>*Rz6a2It6I zIXg>61}nU^X^^&-fAiQ(7zfM6=!NFwDJZ0vGk8C&x^+ro(S3pOO%0?ZkMn?}qqe3o z15X50p??)lMG37a_H7obivLfBIJFj6}kUQ7Jw$)lxMJYO*VZYp~exJY7 ze?w21UKI$GCscCrK5#(bH#ppL%})`*tppq|X|B|nJfyae1x|r;NSK~k3_-C-yPs6< z+ncUoQX4Bh2vFwT2+@pj5Tu0dQ5!Nak)xKLlPV_gLe(LM98&96b`>6$b-?4pL?Y|Q7>6)HVo`068hg4rcSgwJX zsIBY|%I`>@9t+ud^tYrvz3o3K&jQ5OVni#g=EKvlzwP}v>OAfxI7z|&)j8M<7PHjr zD1_#8hX zPe=3Su=X4$vX$uVjTHjm;neTwz6aEI3cE~xH~tFi>{|647{LEqw>92`2ywaL+2FU= zO0|wtI9<=mim*4%j%Jc79Y(j61-`UwX|GArzmc@Us4daIzdQle;QTs&y=Rx#)s}&; zC`FX}FC!G|lje-SmuoYtCjr5lAuT1$r+D)>c=eZ*$sH@#6*(t;)et*Yu&UcJxoIe^ za~tPs>CcN#1O*DP59X$@23mM0Sk3vK%HKa^pRV{aI$s-pJqmpmM|X{Opa-k88`knBVS<0DiD*sl-?U&Y!BF=5n zjt8x9R=f1xahJuGeobnx^&kg&$WlLGOgj5MsX!Gz%H|VJ zM;phvKk`4I(`GBDk6w2Fb7i%3$ct+(HaQTmd$1~sG(2zgKUBy(UO*|U&H1q`nGuPV zjV(B2xb3tDSm7&mkRPl{g-NWNNv*5uZSa5VzzY_)-#zH-H@5Q|SdV-K9g{FPwO?m{P8lTGiu@{*7x*n99d zrc4$xQ9aR&oe#h5{7ON8k%H8gmh6K~7Uyf%H@5wv6*tv$7e4?jStm+UcAkrxztMTp zP6H<7{%;V6Rg#U;j&Au(TTd4kafizw-wwfS7P5gRP+kN|+GMOO zOJpd-h25-i>|KhS$DtFae7=BCZqkJhsPKHxJ`Z3P43uo#2Xy&AIDKp>cf(FzzKCoc zwmf&;3(Mn)Kh{|6?&dkOQ(|jecBIyasLfAkI2=Uq%k(rB@AO7a6EkOYb8UPn-r6!u zo~?AsCO|%Q`t;zxE`V)il(){By!Ez#`J2#Yf02nYa&vX>3n>xnJk*c#qB@;9BWvL% zqY5*;hw@7~il08am{iipoQ|k*XXW3K5}?f6zb7lZXl9&CqZjCQ)3-R>R)*l@Pe;%J zWn8DV4fo{P14UaU;RZ>QC)#K56;Tz}{zjCNESAj__)1pYNM(6%_v->qYW{bPtAZ{I za&47_7q$cizU_U>g8qi1f6T;Dt6PAcypPoP6w!Advgd!i3+`b!T}$(k%||4mScCulC*#3_g)Duj)+VpP{0c&q2;Y&>!wdTc)9s@jW~zIk9{~9AU-_qGDZah+ z@dOD?%Zw?26tuoA=IocL7;-4D|BK_Am>`T59-JVTpD;?|{ekK4s-swhwCEMmt|yXK z^x|BU%Clip%P-MS2&cS;c5V16yXeXCGG1#FBA+yyMNJ6pbB@g5501z-eM+(SjZWLG zj_7=+mhFYoNL=&{@=I!)i$~#MCfYLAMb-g)s5Qk-_4eX8PtmKRo-GhDY$$2zWZLnA z9z7b>6LTK*$ZZ)Z6b>yaSf%K!7#gPzE6P%U*R)TumrVU|J6$(oqdkbOp`G5#%3kOR zj$9=q5xos`3S6?Exj(0%`N>&xPv+!5a@4=RkM#A|;`~DA=m(6_XAjpa51Oc^wLOsE zHw*%=YVU4-{c+ntHIe+8a^S>MhkqhDZ8hMibjWl=IG|Jsha7}}S^$&SL+IC9lD0j& zN~vwa;Ts9;KgbKW>ju0a>)}03&scg4*l|q*OWKp4BxPJ}n-aq#koP?ZUCfdF_lQVD zPnhnn>7@ZU5%HRs%1ErxPP;4NJ{w*S-w1{NX>JIeHm^DU)9YndwomIV;W7Oi>Q@1#TA82OCh~8+e0x=+}=tBx+b_t2q8-OK(jGdNYpz zQ@su-xR01_&S%+iA=g0$*EK>~-c_Gp*ATKOku6sl@)>LNaz8=30aYd?%ePL*UZy5TPU$*NZK zGQgF=)HRB8IA?F5I;Cz3{P58h#~v{ZT}Dme@CKQ1kh-4rxS;iUp1S6SH=t{>vQ7ob z4h45Qe(Sp#Wbm891%sM6SXX<04vZaiR36j&q9|8yJVce!I^uA2*^c)!3{M#WkCZWc z>$C&a6T1KMW1>9qf2nG&eG{jxvKZ2Ex)UR(Q;$IU$5JUwy6$_cVq6EJCEW4!=bC`m zh~>6&kV;1)n0B!*iCfa~JE(W#u@lK;l4<%hZvDml-)pF>$l^n5fAlSV;J#&(U?h(@oc(@F^7r< z=-3TsQ#1UhX;U4Z4DvZ$7%aCxXi%G0!j2hdtMd+l)&?Guj-j}L3r;HCaanM6NPt?$ z95YZ{8N9vpDyh>;?Czl4_Ce?lnZV6DWxi3rGPwfYqZwvrMPoqf9X9i34eLS{6D3Hq zS8|?&Rm3JGe&`;M5IQm;OSerO!^BxKQS;iv!kLEm6xjBg@_uc>deIMSNtN{RNC{)s zu;);zy?+Egn}%RI6mXL_`Z0e_ivdrFMVA6L(W>hs0Q;*y!og&SdG|{{^uhd=CUHMgI%9|Kvd#<}Z1QPta+=@$ ze+ZQC-sAkrs=A=5XJ%KQ+-8$xvcVUzPRwy*Dgc}i6jMm3@&pZ_XQqfDwk3u6@II?^ z++A%=CYZmGh)0tL-(bj?jrdAqi5eXvf9{QK>Qq{K!DEY#;1J>_5QywmaBn(uuso`ae~@!RgVl-cFLAx#r%nAlg^7QTZe#di^h{#BD#thB z?UZoE_t$Z5z}B0u-@^A&CB3aA-urC8z{Wg!pEV}nO}FuC^-q;Gtr##sCMF}gvzEup z1C5LW2Vn1p=$dy#S9EFJfPXHwZ%%jdOHqFPz;dGfZvS-&KGVaa%uDY?A?1g&X8rGU z#wdn!pl;i}lr%}bYjK);%}oyPkK3U@VX4=DVrPm>sBu|W8y7DK=tK1jD7f~&Ng_Z| zuKhAO8n^hO@ty{O>$|QwhYIld@l?Ej%!E7*c>uemDWP~qJdE%ozFLc?G?6Xw#J|R$XlhPeLuzQ|yfbsM| z**Ys5?aCOUNRyVukAUsN!*sOAY?-wsjfb%u1x6T?BDB8yYdZyw;yH||ju=}86Yf?t zzyL=@+Su4bzNa;Xr{yBqjoo)dn}G+A2}X`AYbRHZ%@yqAutxy?=@puY9U?EIv-J35 z-dQ=CkEMLkF)~(CaBHI3YFhZyB3jt#wfY^@H^b;7-Yq%Jy>o;mK0UL`h5}VpGqopz zW^P2|E72Z@QwpM55PqTf1QM|k3{m&Gn3=bs7tFhOlu-9(<_~et*+MKn=*2uf?`^GV zZh{SkR7Pu>A;MUR;7RK17_$$HkIiSq8dCd$9M_DUVG60DHGP~ZZNNP+>`vpfWgvl) z9o)A+f!V#iAD~eDd0!{Vc7u&_2j%Bn?!x`N7`TU4TynVckZ&Ng{@m%9HWmYA_;EF@ zq~}Nr0LZ#l4>QUw-EdpFREvg=ic@8Zz;@pZAlQE(%(|ZjEahOTAfux?BP~AiSa_A* zb>i(!+hwoX4;MZFP&uiQxcDBB+sgR!R-0649O{Kw&Y+r(w~lJm+07NC)8hhNetq#m zzxJa8%zhJ+w&H8;a+>1M)v9_SA(+^HfQL%TBboO(Um2f{hi4H8xer<`hQCqEY+q2( z%Nhkt*g&EKILLmHwhB|MX8ZqouwRGSL?gIMn*Q|#^Vr3mqE;-aW+!ZJEZFq_(}E3q zRKS?|7ysDIO|;wZ^)1JbUctkgvA6Y84$g@OmyPyL2F2&MD^>cDoDKZu6Wzg)qExL% zCAMjo0Ml&)XWs*)Yv1it!OUYk*MD|gYAsXH3yMsjmoW8iwxN^|#2d6Tttdp*?arE; zd(m-GcWaIW{Jcqi!7Qc~m`jEd(;DC>i^3y;DKO>VaG8TtOQ7~EC*bDK@M|oANw&Wz zHI5DTL8E3_6S9%F>wCTC`#bETd&kCwKd%58FjV8?ardhwoy`d1z4Kn- zaRTYa#YD~h3N!L1SPh`&36-%GXdc5)(?_Ly;Qv!GETjP1Z8jA)>CH1a6^{>|&897T zLgA4nTbxIW>M5E-5PMzBcnw&F?s@So$M9{>isX%nl$wodR(E&2>c*Qrza(w3Jb6n} zQv4%7MA-Ae7*k2?kA+*h#7}B_M&{nqvz(NDp3j&Jq*Gj06FOjNJry8(4g{8fGdLz{ zyQ*GCKVQY96i3rD&t)>6(21p!7ey#d8mYAicxcS=E7#-1fv;%PG_5E$a@2IX`a2NK z$B=7WE!?|kt5MgAlR)`t+8a*Lv&Bs9sX$;WtjC{1{# z`~2UWvbRHY>@es4E(mpGs^)hP=Xt3&#!^7+Y>1G&$zd=vswPPyLm_iA?xIccTb#iD zxc#z!z6vNE12j&Ak`+Xr2?4QL?S`V+9ci46A@%(Q=`9QieY$qTzIi1?)v+71F@RHW z3~j(kASe%LYRQGCauvUGe_2jCdcq>8ea#Llq4!wN!Z@b{Q^1q@JQEiRW?D_)pdN$y z(V>#-3f+s}6jk~{CE-7g-rMh$9kBW=$H>-z^MA34ZXn#v$~UWlOkMZnuk=AkSdp`8@>a%A)ooa|OFKkHY)zng%qwWN3!(h4l_Cat}=(c1iwX+E>JtbIs=3 z(577!LG9VZ;X^VF1jp+c%w=4xyx<3it0Zh;-O%;8oi9$ZH9%d9AWdHbe^9sG@1NW6 z&}j?DP4GQ|o&C$=DJ}MDyt0hO4)Oclk!|U&#h7D}t1B&fj61)B_+U-MdRt1Tm2!Y5>U`R>J|hf5M5iHB6l?vMB8=g-DH#cMNMkLIek#_MZs zYrS1QxMFmMJ<_X>i-epv(-$CaC4!XHvcX@QM-C)u*~{=enogGu#H7WpqWYEsB_fCKBvLfemfCR<fKHRvO{< z`TJYWrIEWcroGH?M=P7(hUuL@p!VG_g3lIayk?l}(3Dyv#mwKuN|+<|&#N03H(#h^ zB%{LHd$^wHZqY6D&8jzcm7M+AHHS%ps_DD4aNPI%@0SYTxvLrveO;CrLvu17zK0sYfcH3=)+&V&?b)x^O?>=$aTW3N~4{HBUh#{!uBd z6SJaSzj@&4+r!M>ju#=-s~a>M6Iv(Lduf&vip6URBotqcuT1=V{91|_RbW1|lfJ0< zCU^ZZ$feDBaqddb+JnjC(-2J?hjx$u#k@PpGlya z7_#E^n5|UF#@1$AjGtcCJ^crdfWx?uN1grR@BF_Y&ikHgd0n#HpQB&C`-9gCrj!P8 za56e16mi;nkqeLvOug0JDX`(IE@^0|w6Ksxb(#zp)ifmOV)_|1W1=mQuXoR1oI2 z)C;YIwIivPhUPUAH;65o>v%L?cS~jxrAJZTINtm*1uSDx$!P$Z|MY6>Rusz+qs&&l z?GC79O)4D^bidwgjax|exkNzKY3BlFN2gZL{san77voKBT*ZbIR)^K-y+ZKi((51ARIMoA z38oJw(Zdr)E5*h?S-wo3dqg)Wc3j8@mn`XN0+)w)vOk#g5Pdc$sI{u7n5JJf(N)p+ zc2cP9l&4lJLQPgrIG?ziO8_Kep%jUTN)s|P5Hz6NM(SP&PuhMK9To}FsY1UV?;N^L zV-d%Y^a+bvZP8E>2Gq7pM}v2>eh$*ozuktF;D+VaKxW4ctsN>jxGWC6=Zh_K27}kW za|`?#*AQsEZBDG!GRRB;S*0_t-7KviwJEWTaQUrdQ&kDEkA(N=WH^`aKHcFWp+%?m zVn$+Sh0#m>VmqZ};q6Olw)CE&0fBG`^FDV$u=`d$BDuX;Y|T;npgFxDWs2H*tH?Q9 zfyDmRd8y{FI*nNeSyV}bqrO)>e#xIU(xoEWUb?fU4;dFS!|tg@c3b7gi5ID|E>Yaa zd$+OKK~t2VD?Da48s2{bAKBLF z$?BF*R=6J}wYTmXE<+EO!CCYod4~;OU$y+fCVhBybqZQUg*$S@H+81wJ45O%atwft zpnDzh`VlmPfB$|s3^G3BBesH{H4Ogh$s~A+YlBzdVIr2j~)p>(i$K1w@w+?B8Kv zre(#0xz(ll;=PPpr*8u^4Gy#-BQf|HmP_A-JK#3fn+q7h>pHgPpDLF>SCxM2iC%mT zu05-vUB1OtR+4bHt8JF4$%YkkHkGZ9Ld`5MISD3{dWC&y`RugpY+-1k?2w4soTV6zRqfW{T-t* zpd2jY<=*$H)WGLFHO$0{6bZ-J%eo!>V#`O-4f*1uIfs#sQMvKMx6aP!uvnw3%&I6J zuT|rrr3l9tf+2_WuTJL^Lev;86sr0L*rXlCA~#n#hYrOr^&Ee{opnTwcZ*X8Tl#p0 z4O(0D9`}IeI*Mh5n+Ii$D0K|vjSUPIUv*qP;h0Zkal1fhJy}cpDTbl8u3t4VF!+6I zN5Mxl^|DXECWm-WB>Jl1PnsdF)6 zE=NRdy}=XFtTPlx-zKe7EOHJRUKYu%kgu3*#EELK>^sO;m=+2sK&Dj3P%p8Z$2^fjwtxTf5fAAZZN9J`I| zjygK;YGt?Ye`Lc5?p)Vxbri1HL^gKM)egwG=J5J8>Jxr|27%X1y#6BV|2$<+%6?;jhiEL4@gHmmvhRPU5j}H1dv4rR3Db1_&m%y-NiEyRk>DfL! zxp(lDDdu|vbs8dxVYU7A-%e=nJCJd^J!XNQ_cT29LN(ent7@!+)|cy0LE=RE4NwS7 znSX_wq)k*)kx98-Cwf_#UN&N`BnS$Nr0KPT!5zotG#8|8{%( zUf~)!G%wR8=y=VQEiY{L!fA0+3v5PC=PdfYtIeTG!Wuj66H&D`tc zH>>#%r-LBP@XCa`wmSNAB+AK=Ah_K1z@J%T81bl$mPE&~jb}Y2Z1W}29t{8bL29oW zjk2}8JP~&%rFl$tpFsAN?miwddwM{^x7jIhr0;QUZi$<^4D;{2>yokF-WySk(K0;V z@VsCD&k;YNg0fs4*LYz>#Q}oPU!K54jhx0DGRjK#!$Q73UOx6)pY9s1<-1#^cMZUb zO|ux;>8iQgHiS%>v~fznSs9c<4gAkyv>31A)Y6!SvHexHTgTpmedbTC-3X2S>P1*& z*S#H64XS5FBri2Uz1p~~)4IeG301#_p5Cfy zTz^bYWZL;ROpNqPSs^_qi8-Gn%>)_M$8ISo&FtlV*Rm1WbP7!Iew*rHynFq*XnHzv z|fpWlw( zCxW;*lbdH0-#(2PH!0{1l8hZ-JxRQ48C`tQFM}zJj(ct=(BDA!4r)1P#|t0ZnKOGv zOMqotd!=cTlf42Fa#myMhA@kWEapvEfqc~mYnc3|7lyRy=BwNuZs=ks4_DO5_G7+mh)B}(RPf`^Xf*!kV-uh(Ke5IH+nJLMCvj3~eeRM<14%>awcB|yc3$H?1OsIJ zXD$urpq68oby}O?v!^eYfLAV!6n|Q>{7BhQbAL`WFUld?D9>8uX5QxB;q>)zDr$YN zM7}~g*sbkJ3Q5blvQsN){QgrJKP`P)=-JSr3?Fan3;Ho>EU=?`q$2)jYPq#x*Nc(< z9bNh2s;xA)w%I+Kj@uNdujKb9JFk;I9Ruc@oj>amzBL$QCSwRa9a zfOmP)a4^$GEoGS~`+Ej?-(44ry`FV~r`(w`-q&wp(L8B_gF#IF*<Ou6I4uzu! zfsRQTC_*b!JCa}JsM`kK)gwDajrd<|{XZS+u(Rty5t1#vpxir!sHxS5xPXERO37(# zTz}k^Oom-nhGWvJXust@NfORovmVgWhNplLEas?tmeNdG7!P~nmpn;AWO*hrQ>pj%biJq=ETa|9nKmILv<6p()lIur(ZGO- z)c2TOppWHhHMin@IJlk9go zJ?q1g4|*o#iCycY<+VZ`cW2B(+ZFEw{8rlSX6|18%Gh#ms6okr+lSpg&2=Y3TQ6HHZwzz2N zgE(+Eq%!Jf5(49xdu^lvjc&JcrX1*YGs^8-8pj`=3o@W&UrO|eO&xdjJ*G~T3xgY) z(3yQ{RH_rU9rSAE(LbA3MNWVNP#9oC$QYD`MJNw$PdTfuf8BY`m9il^(t8L-w^lCj z2k30tg`T9COKBxdhx`vWoSNpR1s&?qaSiCWluA1 zOLl;Izw73gOdh|6UNxqQrlE=roz!?yM>!I^hm{v{&kd(QRS^PXPZ#&TvVH6vJKa?` za5nxU?J!!DAaJYOZ?UbYtLr779{(%JCwKYVVn$EA{BLi?g@dQ%j@HW;4yKkTZDa4V zGN&mf8cgA!>~b%M z?8fi#@geY`*IdAiK3+^sg*BXDV!s(Ckd+cTGd#R#O7f^FO~`Snv&S${;=*3_0GE4; zNOQ1c-yLtNgQ!Jm2GFVb2qgw1Q8Dp$UzoU{B36z{i9YBGahr~-TTq(>;n2UMSC5Te z67-ijsi&tEzShx8Z4;P5?=%CJGYTh-@}CuP1}u?G({n6k!JhhWhhpTB0aSX7D6O;H z#_oK^M4DV!8W6zjxZM6KnqLMMufrNT&NaoTl;z&oT~jfWmY9g{Z5`j6(iCZ3erX1I zg4n@Q9PVKo|3q{$_vHspZdm#AxJpEK;;h}UuET(rW?or!MA#=ie2J);{)@-ar&lOux6ly2k(9I;!}fvY-gh#z>5@i~f=|Zz$%Q zdg5#I^+qua%l1+gZr5&P=HaLEEjxQ-2i-FqnYe^y65MKDX;AFPnViOW zHa{rVi=lys*)0CC)JM>71NFovw((KN)TO4XjFzz=D#2j8C$oKGRpl1YDRF4-+acFl zsK6f6D;ZmxW=-XD>j>m?b6_K(r)0t;TTQncU6mPzn)=I~YY0T)W*bvR=DTOa9%ZR_ zP~v@&=8G9jScGH)j>I>-^ht(jK2C1Kg0yGek6ClmA6I2M@S-czon|n?7NpoTeyFy6iC-yv(AEs9CFm~>Z{%v zmeEbzq@mIPYp8qqqD3WWYRt{?>S3foV((d5r*)0FVLdxo&;qYsP7hOcQl1SNo%jEw zy+ZR-u+0j`nrP6V-bvnrTI~rNd$VdzXua^Lqp!M14uxL6@@l?9OTg*KiaWvC5ue3W z`Bmj8y=oaQ1inS!i^n4}lUiq##0Scm_?RnD7*HZ0U4L}Heb!KnbNuOMn#&yM2cxCZ z>h^VseRl3;4+oO+vJ31}qwGe<<{ER^Txd=*=4N)6D5uXl-#UKw@i1e;*+IU|DIpk* z?!2*6dyiq^o9)(Sc$V?=?;-*_JL?R|i#?-u2!9p$1USNL=0YN)<^Z9)Y>-mzD(i`{ z9vG1IzO2Ar>8&^$0ge|Mv4THdmgGKUtE`wE6mmumr9WZADcxbBw8K1gP!`|O%Tf`)gxD8&3q9dY?QUbh8(t1bkNK+h12zqVX z&n>z~z-4SPz@bm86YnKAJU>lWFna9#4XqMI;D9)u?WJ`4+6CZrAs?BC-ERct1R~uo z4-Gj(KcY6CJ&b~$J3U03pe!#xG|I%qiqc4? z1Z8KR`KUe$D@(TQPKLS77CI=>2IIuKc_jhw#{C3Be-$FjCTKadU5s#b8$+^e*jCL1 z?>)MoCDn9ro2*R`lmwc+Ys5IHsyuNI)en_tFRfU)FRO(sF63P{2=kl64DLsuEbRtO zc4pl6jZll0mr)r)>P~$q{=baEX(eR9_W$%BiP`7?YFNI z4J7*XmX($B;>5DTxUJTs9#_#04LY-u*?(4t-CO#mWXO2i7pCClGj=3I5B9|T@w+>} zyV_H7Fsukd+sgPL=$765EB~S`JJ6g4Rl{zhkfXOyNbaW9%Nj zeZW)bjx6KE?t?y49d9(BSd`XvXta}BG+~dB7mZI^^2^*36Lu?Zog4Y&>L(#*B6D9a zIg2(1B+oCvACaRvCVcvx`f;OcKa`-2{0l@;ZT09G?~z%bT|1xK*+BOMjxpz+SH(m| z58@&)@YUWQ>ZNcV$FC~tG){iI#P6xKjTyHtQU}p34sAc0qP}et)A8`0$x9a}Ro`@& zTS1PVGiX@oH&@=rY%P8oomuOA-f3h4nEpHf$%p2=? zK=5allSqb(jBd$m%cvHF#}MbSpof~SMo%Ft(Gqfqd6nnvcB7&I9xwQt9o7+sn0Q3=Jm?&b3ywg=^ zPg3>y-F7~SI<`+nD``WH(smgid_o(V!fw_u*LNzk*kaURJNGtRx|@a%S}vpI1Mqk9 zpFnKS;!^%JzZ4F?8QWG7_GkMvD-%1{@L1#7L>X@KxRDpaAt=&p+9;G2o=|?4*}Ite z@%#c~t!`sq^0%X7UN_%?u_Jn;@Bin)*z%g}T6Vlh#q`b-`%A_;4-FRECe(FuL$v6q zfXJ4K^s*1;2Q!-^5q~2wy9eFoIMR$M5;VGD{16FtFq9Q{M;^(e5v%%YM<(*+(MQ7s zgB1qa$@mNI@jph(y;ux|{tU-iB)jN@_j_#^X3fMfczLi+julijVIsJIBRs|>f-20! zF-8yfY<|h0Su#5*?7WqF0jz%oGNUwrvvpYXBcjUBg`Y*cHG8CR;zbi5vpO7e_pkcO z1_k56u<+i#zwRS_g~_SDb3!{H@)g37kZV+)Zkei*wjQY&+$WLXx?QV5-caZmU%Mr27ZBq&gGz;9Nx}T~6lH2`yI81e6Q${ieEpYOhT%eEOj6)#$-m zXM73olo`hMG*hK>hrkz0gDIJP{L|)^S72CXn0$PZ@^G*K11B?Jy-JGHL1UKcK2`=!JPS7>J%ccUNGJZPyD?W1oFNWB@7}H z2d^F~0KLGQpLhV#?h!+F$)1aR-vY#LLQh>>Zmo9CBlH$0mvCBMJsue-7M-VIW=_Rl zO{B=98{Gwp`ww6hBd6vdjoPb~n;jt#?&CjN7++nCb3K7LPnA^EOHIP$ zj{0C8WKI$Ki2zqzA`*mW+LPniX*l?)3q6H_rDTbPOh_s8(+$!8bwFgG?%QXkAyE7MceO^VgjY^ zNB!+)vKcK_&RW*hTPBy?1NAyxyX|aPyDHOr--155#4OKIW?$vm5Sc-Z$;3B{aw@=a zl?+No1A$$24heqh#E&G%ap%Z(C5B&By-?!&FN_KcwB|93;vI#0FW;$Yzz?4#~0W9HQxx~{*9|37M4P}yeH|7*)K%k~_f zAVvQyv|Wi8q0hOiTH2V#;<}0FE#z_T1&Dq4nI%TBVtv`bC?XzqC{w-kNPgZ z72`XZUKT(p9IOq`LK@Z_;6l#tCV+0HPW+%QBD5HZl}U9!|5qiFlnqLxg+%FZ{|VQ= z$TYVpqf2ko+TGEvBlVp77>(l+Bb%!zUX0`CTsp>rp7qH{Pqt}?U?oyZ3r|W^ zYHf9Bac}7ABTWaB@X?5#ZPv17$mL>#lK(i{Yg$))Rvz~Z#YVYk*=_8Z=<(ugk!7d% z^)7?Zd5`F$DeF(-LaFHcgUKzlOdE15V9W$qPL7>M^g4+fnro8xWDNk})rThQX&N%}1q35~egbnfdJ zadoB08;Y-EB4_w6b)X;!byRMYa+(G$o?=8#L+xHZhPpCRFHn7zaq1P64_@t0b;H%c zuuY>s^oEU1I}zi;OAI7$@_X&LkDz#wAcJWdUVyK(>wV}77;ir&Cdba(E_0lC04RKt@FUpBx5kz;|U~ELnE=I+Svg=gh-s=DQkNy z6>;Xz++6+6)sHE?CJS>$>;?5`Qv5&|I*{iT|EaEnjK%%Jt_Sa?I+1lYz-SEvF~mg26OyUr zhe9`oPFBr1+M$~}zwVuQyfw3WYKRyjx$Nbz{=(h&3V1w^H4`4yRw9e^pEZ__7|))S zNCyNeDfnIE&0`$#w4TmR>0-Col>zgYJt(G<6zAz91C80BR242kN{rrwG-J$`KV7G% zg9j8v2*GU>vse3aU9;Fk9b`8CkM_)et&XQ?6_=1>wg#<{*sN=XqT99B(&}%Ja64W$ zQP5lXDo&hR+ca}p;D;}9t8C>Abc$7RqDYM7@)T#L#{X;CnNYzO|u_n zKC*;dL_iXK#>6I1S6m|7mpl2M;<{x}YIu4HyMwBkb9>2RKju3Ss|Fx>h#H)b>UU=- zZsKxYc&1yI-S_NN)8IMA*xlBxQwEMf1K;Q5bp!e1D$cxq=JB|G>V?C-aSIN}le;?d zTid}fX`QlRmfnl~0)4^Hcs8fP0aLRD@O3yicPk41Q=`cSgN#6Ukd$?U*2>M4J-_kG zXJ6wAGPh?BmhIQ&SN+#iPWFPj_w8oyVl3~2)4s!W0&D3FpFY^*Q|a;F3j?%=rKKCr z+}oa_4Ql80L$4nw93<23+%*6EtV1cCfB$>4xhJF8rrDNp*2Ntwl>zpQ1MM2q8waAv?2hovM1UYI% z6+IgZyBJwtYfk7srx`aBrLF%}<0kT#e}P6W+fbU1px3|GHP}%7 z#dhlHn^KAE>GT0hwj0{eOqapr$Z;L5=`_4@?{@p{w=eUG)Pn*=muEm>uN3 zQP6mjg;sp+Ccmax#1V-H?upGWxXFr($k}eUtimBO!0cSLug1eZ?NJ`##!^iK!887l zw7DA(Bs#B`ChD?_<IZ`cLwQCH1rkwB2JQU7 zHN=@E#1DYu&ckODRNaXpf{+-Zg@@gPFU%k#%j4%%?!$#c)@4!XF{jn)?8Mz8-zD!t z5d+nqh1Z|_boZX83>~L6M#D}gG1mj`Fxswo8T;ic7eM)J>c))u1kvYi*EZWN$9?0k z{X(;E%2PHTX2pDRd&O7shNp}Vz{0({g~O=rv%_(zMuV_^Y5uZ`Hv)T~TiXE_6_vnW z`iL~XTQIv*YtJ?({!%ick0;dlN3M39B8`u-ethv&cBapxfngt=>VXfkE`>+1LFz<< z#$^Okrx0YEv9zN$n*BZ|yI~gJm-C9ugie5uM28LD6uj|qK2#9q^Q12u9GgpE&);0k zq0S(9Mg5AydI_gIop7&De=T148qlFuKI*pKphCH~JfgV@T>6Bel9cZcDlec4msmG! zxu_r`Kx}LLLGBpU9gFol)cc0u(~zm8i}>C{8<#E0<7e^Kgc^E7?kFA{krR`JKVdV6 ztw+V0Kx%4R&G#SDl3kAP3A`Cy%6H#EBM(l@DdX>hQ8x7r6mf+{n5FV~&w;qSky^PkMTAjl zC8b7Hjre|V7rc4znc$@hNpUOamCTgw1y$o<@R`$9I7WHn>d(iKP~DFVof?TO#`NB) z((Lb64$h#Ar}E&7oe`gu?Q!o3P*xz_U{4WFJrpOW(W2~*^7M8W_c5M##ESg4GO!~@ zD_ibXME<2}^(9()cpa}c(cUswyU((@n-6j3%M0=BmYGY^i|~F!zvNNiWfX?N=j?7S z-v{FvxUB<-1mYU!&utQw5a&018&}o`$lc2NGPi6m34tG{`2`=C&i|>hl`CY8+V@2L!7vrQX8uKOUxapF^CDs!r{lWlvzEKZ4P zR-#B%>xESXTaMS+?QeE58x_U#;?0fbw;R&M?4*Rholy+r?R*WW!|M@^_%Gul@BWP1 zX%{c9CE`3rc{Kp^yDc-UY-9p}<=f7(B9>t4>s2?OVWVz{wLX%e+fPzZg zxC}&UYC7}mPU~aSP#>DYAoHw*E==&&^+kU*6tvj1+8Bl>ZlvwqZWK0}02U)s62J;3 zlnt=wl-Z1|WsbybuKc-bc~Ni(0nrS0ZKH$NHE|UrjU=3}dhuKZ97Vgv!``lhZO6`m zB{OI2ZjD;~xH}vkxd2`Si~>+KI8MPB^ef^z`xOsoGrg~M$Z;(UESzy`{4IE6Or*T9 zIt?Z8ltT$VOO`J~wDWE^z87>W{tReB2HYwQ;<~9(JU8(^?A0HJ5*6P^k9FhSQjZA9 zhkJ@{Z&o_21IW9KUMLz-WP1YPu{BYS?pj^zX}U8b6?Lu=>*J#h<0TRoo~ zdZn%3(Q!KYzo6H^i~Qev3!cMx#XuRH!1;76cuHevylSivPr;+(9;~V&YKZTB}i0tcXvJ>bp@#C#jFX zrj`6Ai!m;U@y{ZU%<# zO;T86#5XC#%Zlg7rpQ*eSILIUzwxrwFSD^A`)9+g%6;+A{D7^=0NmC}*xM)t@bD#$ zB}jyc5-He}x4N7*knX?yJ8#eClRalBAtzHnPUj`LR#;4{G`@K?zRWtD9|d~w=vLk# z&f7(!x7F+A6jk|jsoWp1qCYC2iF_y)m$Weyzz7!7{Q7f5R_qmlI+x}Ps154vIt2qz z%IndCHcbN3R&FiaY!N^l(#>xMZrHNN?ozhW*kKtqYLFc@2sotLF>88CS##uopfF%H?t^caI#*XCu{GFDH_2OyUSHN2F|&)ikzoH$`@Q~7n@;?y~7#;+QEh zP@xfgjxF31fHiYT`GFdPEdq}#iWyn}R_TdPxI?iGf!_^pfESgY2??8T1RF{V&~YzY z4Q(2)=VEwv%16!jP1~4t=wtt_@59ta$|urdL-<@%+h2~jLry|6J~g6Ka$e-wHl-P) zT$=gaEc!=Qp~zEmez!H#7(4bjaKOH;unTKh%pC%4x0JV<^mY{i4HchR{F@M(q<|vAK3Jkb{1eB{S>T3a zn5nFf-?cGd`=-kk;)DzFq9UK`7#z1T4}R#-kK36ED>W3!<}^3}Z?d+Bfd!*i>dubO zetUfD!@tU_DKz4^4lA2!$_zOO6^6wYM5o$V7VXmXl_+juT!jtA3J(KgyL}_!T8`6V z;vc+XcS(??&7`69mkQX|A3pwA%AOW}aoDA~_C|8` zV)TVId$qZ5Q1v+nODVfR$4J+oSU-8Rl>RD?S^fY);?U~aM(fU6RK0h6t@J0hhcSCP zY|;}t9%*i#M^wyi4_d}7)^}RlsHa`cfo24rOIU*burdwkRiHMYut&p-7V+aRG+0mp zmn{&_@WYlW_5?$0SQB=6c@XP`bH0XA^pV%TAFtiiuKG-^40x-Q8i9ibx+mtz_EK&~ z4WMAAJJOJuF-N9rS&g$-!5P;MompG3gPsUb4+g+!w~>!xP;;6^lFPhiQ^(Pd_(xsa zBDQU1&97y3Mvk|+z0mAv*cdzj7?M~P!);?x(Z_>FLjCAe!ER-u)~f|3ds+z70aRa zcOCkBI2nV5d5KLITWU*3WZ)GL`LNiFihQZqzdtPUA=vF2;=3!ai@R+lmfqutM%Ng1 zT&s>y+sdb8>?v)z+s4~khm7e5Uz?>bwzY6I`EAE{bS_uV7n%GN@%|gvqCLMxe(El+ z1HwcXf@08<$e%n(0~Q0BG&m*4)7PP2`qViCYSwi4uN2nzFD6r+1ANecTPX!7)*nht z1u7BahhfyxCaqBT6|@lZ2yEd6%nm+-b#eX3X*Vn&j=;3zLZ~rO z#ts!IP!gcW!JU=;+2{5<0L<&_1r}XQPt%Qz9 zzA^BJNMX(^xT{(w7Q+4X&m;T$t=>wJDnPG{L>1nQc|RgDE~gCPS~T?S%dJ!Nd^nI$ z805VZ|0QQ%I94lwJcO>Anixd^YOL=}#Gvhn*dcU%~Lxt=5n_nGxIz?Nn z|8ahZSU=wmY^7I?KX9(JtI=K-4{*(APp4D0^A+gHgH}&5MH_?UA0=<-eNl%ZCI1I3 zzV1@O<$odaGAi4jJiepjdza>w_v0*yg(4~M51|*}(p+3pcQFtLyqtX^5{=&p#(btQ zjKn*q>G-QC+bwYh_e5KqIO9`6B5gD6h~|5zz5f-De~Og8t$Me#;fEW_{gAbpGHtv+ zcn(gO2hJFz9H{y=ymc}xS*!)otjLQQ zp*?$u{AFh?t;je_=7D3zv;E}5W$1?i>NzA%g%a=8uo|WhXh2X&FQ+>I(V#|C<8jg` z_jbNY_m`uVKx%?$f@9m4F-OVkIv`ahVp(s0m8OTx!nY{yb<9%z)&-ZyhV+p-Mo;Ri zvN_MJBcg+?Mb~mYCChHF&-(7^jhR9RT5P(w2B$=52rOZ$cKD|cp{~aoqW5m2>bA#O zE?%$F&P^)*nbM+V5EkKZr^*H%_Dci1mFrdnO<0&j4?4a)xl>zX9jUYrk0FH(#hQzE zU-9o&hPFr+EbkmG4g1+3&oR*$*A7&2Vn^3V^K6z>FRmpe&?BI{`BeyXz0!W ztW(7pTOWF)XzSmZXPFYxfVUG($Xide`4FS{a}J`akH_2UmIKryVh02Jur2UqZ$|h zEEaxQmzrpI$L(1>y^PiqNzNZ#GOF zBIK}yVP84|T4pxXU^~u9c^fJmvgh9NE5T~7HxfNO1Oj^j6M&vERe*VOt2~IYm{KKnr7P^oQCseUhau$W^9!Y-bCX z0cKe-R3y^Sn8w(i53)n(gLZbW9LWXoskL0!Mh2dMH|$oDIIDf7E507&pyloAnP}P# z_~8-4Ss>PVCl|F^g#$;3r7&zv7NPNhv|f0z%ZL?A%#R22mnD|nXQdpPvhb~t*z zd2U*^-q)-0wnAkO^$Rp2B@B*{O7P0FFF*w#P&k^wtZ!bTAno8(bvGvujS3QE(i7qu z86yJY==&PcNGV;w$wl*PDpQ6tRc?-UH6!5RW-&&u5B0v2i~0-2-cG_lF=`AKT6ex+ z9lE#I=4SFtYRP}9@GrJ?7XvBCOQX=_kPLnF(w%k9GWF?s_i5vsw;kmz^WqpYy&!E} z6d)xabF5=>%DXI7fkT%X{GXC+&++5nFgL7V+*3H90JNTaS#5It zB^zx=Pl~6nM%WGpbOX~Y!xEK_u2q&Ayu-e3v_-F*Te&t~9TP+)^4KDOzy{IFX-boi zFmZ&qO^wH(%%2Gp`;%|juz6vPg4D^U+<;$dZOl)NEwwSs4kdcs)c4V-@dUr&=BPB9 zmueCme0+f(>+7c}_I`o1R1aL9B}&kK8~a}xNpeV)E^A0qrG z4+~>`rXUvC1#m0?Uh-WkRi(d`*aqFH`BNUWe;5>FZN)Vu!^%W>XX|#Qb_E5BqcyW5=%5NEI)5-{V=$n)4X7_2VRF?M^duo?@IPo?<759RStfnHl~Qbs z`8BU8!}Sm|@AZNJ9bzW!74JuMvl}vCvm6(8`K8kcjlh#dU+$%O1CPYN*Ej*qD>Y&R zjcg(ml%>d_<}`fZ3IO7QQlmPKhy3>795hiU<`ZG7)iAs8CD8Q_nM_Eg{!qMf+0jpZ zjynU%>t3=iK}j3+%-RSk_rupW*}WA3bL&*(k7od+G=l^n+aHRo^3aUBjZnaz9W+L| z9pAce^zC~bsXICU|8sRRPwQ@sj}*6{SN|GxB7WZ4%2ak$F~EAin=H=#OOivW^_k-- ziferP*SAv;{8naXK|HON{aVObxPA1k=ZBP)-iC$L@f_6^T3>%p-GQv^Fvz3e$o@S4 zIiJ;2s0JjIC;+m3>EN~m8euhZmBGsdy1zG8DUE-cFAnN>Myo}ex|4ye0VrMyR#llj zK$zj2M*r>lR}ot8v6kM2+lljEVaaH5JW6_)_QW zX4~>vkwak%Y%XDPU$57Yj<9haZ_Q1oSwA}~?{Eq0+!Z#rISj7z&E!hlH(HJ2Hv^o- z;&Dm}=RKP4*Kb2VzjiAKzE?K_1Q&h`bE{+DOcPC%F(LNBw;Wu8}%Awf)@y1~{SRbUnsDrI4BCc5VNI77}-r$1CgufP(zCLqBe?m>qj1g2?JA zIU*Y935JQzBA#)BEPCfF(=}q;*M+0wk5?{U)e7eM*H=Q#*nK^5Pxep>3Vk9+ueBzp zn5e#T>k`l2iacB76?uZLX}M#1a`Ku$>jsj}%o2=dE5AFW2vvs6+~HXHYgsc@R<&q@ zp&Im$zHdPlr~Jpp+q(Fl)!_GMMzai?K)snR9e!HPeCZ!9I6rD8SsymrR*bkmyT?xC z7jBD@+hoU9s~F(;bdx8z*QXrO9Qdwl9Zw?eXWN~Wi}lo@M>Xyzwy|UE-5iZ_8{gm< ztYVaBXJ0U^erLqWINh_)clcDi#(dSiQ^4=Z_jOgdh|C}X=pScWLrQB8jneO2o6IUm z{uOSSfYTYhRdb*hB9nP|I6TEjmKgh?2$?KLqol7D4eSic-x>e!j#To;hUVXH1!ra_ zV$P7^>;GS?9sWlJ3$pahKvayWsD5bwf4yLk^(T7WSQ-zcE9$-u)kp?$-wwQxy_2%U ziKNfG68_uN^&q1K>oDeLy;3(Q(_Ys$LCBV>JYL!p-K1EIb=z$mD@Yz;>-!wUA;!RRo^wEJE^Bj@~9627?$R~Q8`-N3*&F@@};E_g9nc4=-$cJ(fRg$V8{y(sw`B8{6R@b7Mx_~cG))=Z7LP@S57kV~Y_?WLCrVve0may2*>6xpmgn!6T?CY6Qw1+**YeOUG|qH2nf z=}{#z#98!yLlR3_{_?h^ZOa-t>;M0bh*+-cU!r}znvPr#II}Hcb9Y3};W^5%MEp>7)o6=O}L0 z^d79j0k4DUU9hUWcMjBJ-0Vl3SnO9r;z;)&ND0!vShPA01SbL~)8tbiQq-vut@WLM zSb*1L@0bVOFIGPD`vF=F8pX~B9hKXfNy?AM#meUh9NKi}Tec>pmit-Sg>_YBK<~|8 zub=(Y;C;g1?&>ZV~ZRGV1Xg>Ncx&}hs zrp}%78nSW`SzPwC*6Cr&JMH2W$IFUqhp8C>*LvrqWTxCUSG->L{M{J$+fM(g9CWs< zbfn0&t<&yv?A3#@+KQ|vldH%pVWHO6@mA>@#4$4w*a zt}pPkAzO~|a*ECXr#Lzx2y=;D=U68!;Q zVV$kOvR~Ty6VC(*kF{Haa{4mA_;NO<{ttbNN&Tc**ehRQlV}|Z?SAmxh7M(|t(1_p z*9xEuFyTCqAmRBprc9vh)Jflq;4V6jrIr%Nzx=G&3WnBuD;6zfP*IWT6&grVKvWJ4 z$XA<0jyCaAf7MMe|K^=oLL-PMvo+iWxF?AX6l~yuYVr)>>*mdyOXH@061u$8EO}dT z*~*tg$awmQpt*JgpS=#cg+el*yP&L$^kbq%%mT1gXh|&-L4@YQTG;Hxu|7hiMlBQj zqBqI)S0FYu9CY*Z^{C6L?EsTIxc{3|?`2IM0lJL41DZ4i?Bhi8AMm)KQOYjlcq8Y1 ztT{UrzC7z2a!<@M{Uff?SH1NXhF7jY{%YggzL*-%Z-e94Ke^u9zo^2kBqvCG%D~W4R&pCynXFwRM~}e!Z=&RM_R%>G(0sOw@+y%KtQ5xt zwp2|c_FBUkY8o*b2h)tf4txzB)l5%$^|cOwS74edcg?{?1UFzsNjG@LYEsfjnp|ws zl;f102ThZGY}34SwE2Xem^cmgg#E7gWJ^T8hyywih1WcITe{!rM1uVqi8?1|idOD$ zcM@|=bXT#47k2KH4DP>3;^tX1E%c`(d! zJ#rv)(Cy4v+p|@@V>FbRQj8!9iftj0ru3dW96RCT;R$Ksvx11!PTz|Z$ z9jrAKqr<;gRFGpYBT{V=MH99=G0MY>2LdRXGSTcA0LJmcCpqK3{_Xe?Jv-5Do2vVZ z2f7vH)dpVGmaGa#-kdN+RQWqfEwjH4ROlux>XN$5>#uM=UaTM|Cm3Tf&+T(YT37pe zd}TRS06hAdG!VMQs32jk3=Vge!v7unYpwbTv3E70SDFWwjzNvqpShG8 zJk*30^Iey~ZlDw!R=t2lDnr1-9oWZGXy(%-6J}hKY*r_4iqk(_8Ei>oz zXMHFeJ_Kr%3Rr=T()S~@fReh+JjP=4O!rHfYZCfmk%nE`U!_U-yIWEV+(70N>*p&q zHj8(5%fElff7pRm#D(oLpFLrUzT2}44;uLi-u)0MwN}}G4}y&>N(M%RQ8YS}dt#Zz zXgOiBN^K5IrTPWE4A~BC@(G$S(az2p&Tm_$4c_cMYAO4Tcb;<195oiGV(47h2-2?d z0_L)c*uJXGZog_ftW(KfOqrdtDNFANaq`x3Sr~EICFyG1IT^Say;9A#o@Lz^^WiBD z5aD*pwrKaEIHs-_lAc~D=g=ZDf&KLOys|Bh!8BL<-98&K3V*|bRYjCjAmiQM z*fI1Q*F)9Mg%n88a&@!D9Q`k{Hm24}cSpy3v~>9A>^*mDcL3V^w~C4FY6$0DX1NGtFR&@#@39kYP}RS|>kCv&eR>n)5}v zq98EAudR?uU7TNyEj#tzh^qg(!gH^(l#+Nnq0%)&KObMI48e^*0x-NAZ$Z9Jg|o|n zI-i&**cmJ-$NB;~H=bC0vsSsxSdTi$?IR|wMklRGFImLDl7iK3q5%u^l-XD7{NGtl zUL(MfV2}q>c!85FW;Y|Uo8&7s!?FiHY>w96$6arv+HmOg;I@*AuB;fqWT&y=Y7IP+ zA>cG*`OkxFnZK?A9`m;yYvCwkX1p!t_hE>k&W2w^An-}f4rY#6Pp$hYa_Y|kO|xDJ zlEwD6N)GGTOJc_p5I@gmlcRta$qLGS>@R<90iaz!GNQ9f_4)}Gy< zCoP~6X0v$5mJ_mVKg7J>wdeQ_Eu2>2Za_x0v~qV0+kb^&4u%b8W)q~&3AA1vCe!>& zAJH3IU)F{fKBsGQ4aG*qre0z&EdHWNNjbo6#0 zks6U9ZcE|x7OyI_CQN*9oaj1*9u!T?m(ZMh?-(Qshtg5w6e@{)hcw<8+#Om1O%W3L z(M^KQr@Bqyie1eMySS6z2VuHHSz~;t3?S0R=2LI2?~T0LBP{{0tHB^j5I17-bOxzh z2=-L*MT;dN84)=pn+t2JhItUz#L^A4~|6XYM_tMUv_p{hxiHH zihx(dMmHZ=rfPOR6QhGVXAAo}-vJ*b_u%XAQXpzMUc}_{Snn!ZYrymDe%Y)ZYQ~a* zDK@MLRL#i#Z+1GVqv`@^s(QJ3N1?== zy*kj?k)88$B}IS6fxE6jl`BHC2(5sf0=4`j`KFcH4IAm~$v}4Ly<+8Sr-b(shxY1U2(o=fwpB6O*lytYo=BU{DWlazObbn2NS14Q z9f5#y{pX|!+7iv01}Mi^i%)={sy9T-R@^Gy`h?@t z$uxui&BQpDnPP9|9=Jcc`GFb2^;oMH$bDglHEf7f4Ht{ze8vv`M`ZEud5F>LSH!0- zC_~#dg!TV(9^%-_PNMsntvgnGcANKa9>D9pGlcL!D(5-FnoWm;hSXTU-+Ug4m)ZNO zm&ft9FjhV6|J#5g^YC|;KUGm990U=OMAErY6|kii5O^%368^WyES{O=)a!Si%M(x{h)}x4Vpt8=fgm80(x#R0?UzjH4g$p# zr)KOe>geR!B;nfNx|dLfpEPrVr_|MQ5e<%71pz6c8X!PGMM1iBkP?w5LVyT_-lPOVlO7T}gkD4McSpbPJKs3>o_o%Z zd;dAc>lnT}dq3-0YtFgm^wWfFB?)=ETiB38D&`^c<;J#SSLC*|o}FuaSV7a9Wan1N z3=*8GGm}cc9o)U-qQJne9+=Xe;z>oaezX8WLRQ5#@1z*qWA6kVSQ9$uqC-E>yAAu+CvMUGw- zV(p)s12h97^I0sCdAFPS2y*kEEKKqpnq8x zn6v^U^AM2<_|vwQJ@Ye@gNLtrlQ`DCC%I?a(;(}jj^k76WZ(huF%^zkm}Jvee-&qM zMhrsrXg8lc#yfe!;h^N0IJ^A&l858wS0wD$kXL5;8|4COZlBo@SUPq4w{HqhuO?HU zpwRq@HQR6FK-p$M$1Pad6j(Ap+0=lGu|)kbzmWzEkrW&Dwuz6ta zj3n?#vu|Xfzyjt0c)9~UQb8M0E7sFNN%pz7tLz*+$z>YV|8jDitjC3lJA4}qMO@tJa zO;`>3-SSS=d!X8<=?f{I;ySVbiZj<&)+ykB`x2^O)Y2&XpWcUk8p9Q#$x$?dmj><^ z73k-=_OOO3`JBGJmxOri220@HgP?y{QmyRg8zT~2l;&%Fwp)98BF?dJ^Io~t$zTJJ zDk@Py<1|ZZntN8AOLE2fQ($>^uEut3@0FGKpA~3~f_M>6qf9v+FaKdNFPD$)OiXbF zWeO1OE>F=ie5S4fkxJ`o6BqT2urz5qb_WPot=Jg@#DQ&q(h3Je2k*qT(Z-`$hkyg% zi&>g_9Olt81*{(ovaQipJb)ljh}D|$$cAyZ!)K!_il)FZE4A0dOA>6)4mIJ+@sAP! zDfAyo?Ae^)6kz1@Us#3B6jV;BD#)4?jWyOY@m7EPOFGuKBu5lj<8~>D1Gd6h&4+di zsaWRefOOEr%2uJYigT9(Gf)R~U8bwnCi$S!MT_)8ymT$9&Hpn`pLJtm`HITvB)$ci z0y=JKi7OSPriRaX_&r=5P1N582R@EjklE38*wOaX4v2B2 zoi|m;$C<7%gxyDO&lqSMrvMSBX=X74^21p;uiNkAG)>q4VfPsj!D~abN^3(lHeXy` zjpz=_y8IMa_}w-5+$b<*1X$w%c4NAs-X5*yp?JUk7dIKg?wj0waWl^se^J)x)t5z( z=u~srddgVE2O-NGV9&EpEIFo>YxV!YfYz!`pW?}eiArRc{%5>&UIO*MI}C0+xnBse zG*ZswRSvz##cmI3rR5f9`kbq82+{jC3}nFCLpQQuz%n>j)e8@~{aOHnnVR-qn$GZX zZB@GKjaL}PjxFB1#S4V>!noLlH!IzZmC266^WW$aS&+sxAs|_lN|d?SL9*U%%u2{( zS(+PAtX2EJIXHy}Asi1~)+Wpum5ySXSK~XpEZ1o3GHU}J-mbeAe#PB@cN+|L)8}FI zkxw)E?Y`v8VO=Y86aQ+q^brF9er{O4>`_H%b4hd7Nli~GF`6I(7llS&B#rJa_N(uy z2gqR(xc1!bKlr%s1UJn`aMfVz^gsO&ykUk+z9(Fw9dNuH*X%RVU)*3NH~TeA9Jc|2hmJ(?N5AJ~ak0n+Wrgx`=2ZD8i|elf<* zwq0?d0VFl3@mGf?TZ~@NPGY#j!Rq$7B!L|TsPC&w=Z%-kkboEFZB%ZjUV(c~d~71~ zyj^cH$7*9?NluWwXQEtgoRNz)Os9Mb=O=eG7M+zUT~ppRJe2EhGLs|}1BKv%% zN`N&;>UQX$?X+YTuK~*y?3$}a>i`HSWN!ilNZHe1I;LuC7v&FH$#wmq_C{{mIsM%7 z;bB`9P@;enpmmV740Q-Dqp2NAE50Xr12$2?o&FZr1(o3A>>PR~BPi$1UjD;bg^}Qc zX#o10mHtqHYOf{(=Z05$h!NWX0;$vUMNJh14SrmWk^c-Vb8uxOh}-AVLaE@qhb#WD zyq78XHukWNQj#b=2(=(>?&%Fo8&7Ze_LYL8meHKHFPjmO2hw#a35-~q3 zy6fC_~`?H2Zg0a2g}vS)gV)daT*Od&F`zz$$u0wF!Wj~rNNYHWbsqq!|vz@N1fC6-XSA*81%{%qX&0$y+4udb~2SvjC~ zLMs}6H=Y>8e^q6kvg5T~``n!%(&xl+!tp9_^6|!ESfNW;kuge-8uj%J$CvEwrDxGvm!zb!QQ25Id?se%b`ScC z^z~`4=c#XC$65JSW?^$KClx!;y=Iv#@Kq7P=?oMH1h<}NB5MCKs`ZS=bE$#Sg&!?t z-7uv}Bc&XuA^zC<#2^(yTxtWl2*HKGP%|EsS)ooF z#iF-XJo>KN)(U=_&YjI}Z?A(F$L?5^ub-ok@0bpn6EP*gZq*5L5Qr?p;@TMEKuU}w z)Hb?_3e7yb&64qhn{$QOhYv$pB%Y&%*A2vWNb{1eWI9$p+8TVD8NB-hH^}Ii`0R0^ z62G=PLTPq&eel)BGSY;oG)+8)+uXDm&#>dQbDA%yZSZ#%-p1+0dtge6H0|hLGAytI zOU^?~vV*^PJQuJ$#dYHA3580s76bOEt>!3J2BdGHQMas8k1VjNLu6+h7%Ma?zD9jX44WzBM`12HDy+B2|H8Q* z)A0+e*>`{!hd{c9`wy9}2I@sw@xpG8y}!m~8|yrgU-@XI-v(l=Zby!Sr=XPgXOFKl zSsfGN;9}raQ~ho<99Pkoy%;T21Z8m*wRft4H;+wcK%>6qbbt}R5T@Tz;-Z18#}A)) z%Sqj_s$0deexI-A#v~flI<FO!S6|`>bU*kk zPyG@VWsQ!K7rp!F@cbM9Dx@s=u?o*qK`xE6&L`3rJ=Qh%;)y2NxivRaJOSTrGRlK8 z!t^Zup7;JLK9=Z$5Fy(sW+A*Na$g-@+0W?sO%i&{t63Bf-*gK7r(&e$QPj(HsK9+| zHbZpStP7zg>aFId;|iA)u3ssrI%A~qmkxc5Af3iTBFA=0#Vpz?I_{y9XrQfcKw!f> z{-7w$B)0zk3+;yYG_V(jOghuyQt3vC7FbDQ%(Z~DgeneA8Ops&*7(&{`TCVu;^OifTpDrJd`jkG|{x5e8B4GPx!55kiuH4bU6k``mO+Er4s%QR6XnLwA38H z2LHi9=d_Y|f%V3C<>uli~5wgHebx*vq5v`@lBH-ab1#z3yGDs-Gnnz6jeM` z(a#ufyq8PaO%_;!8s7YvAbudQ`KTO_k|q$M-ipw*gdfk<%Uvxqy|SyntGb?FjQ;d9 zv-+5CgxCDQ2wVCue%`ZXbLwNo`O#qwm=R>Y@#@%+ja8!4gZSrh3iOudR~p-%eymkJ z!Fq!iHytgsPYtW~@#qXPc{1LUgrL&-Gqx=30DpC@r~W2;SyGb ziMQ3E#ToLRQR5#6vXlbqPdieYgs&U(?)4!_Ndg)d1oL7>5dFs0SM(fs?cn(FsgqZ6 zb}l^Z0hX*f@_z8#630SSyI~D==EzLmC}RB`p*8VpmIQFGQzqZ!WO;8cGGE$0R2nZf z@s!!sw5rrGz`x=%yIJz@=QebuujrfADd`&!UYYmqvL!=PdaE-P4;qNX+K$bJhUIHS zQlArd}M86y?`6~iWYvi@b#Hu5~^=y{DO*DIgU z9gKMZd@6@GHY$X|wIDxIU^ZbMz8ZxksYb2}QuePO1~cr)CuV$QKTDS)6+Ee|c`cwH z{1%%&nlB(5F7o?9Q_P(vx?PcXZNf0{h}hSs4au#_eb(%UU0WGH`()kCG91>DoHq(J z4{}5u27W3JyNzF+Z;?g1){Q*LVASAzmdp4nwlmKuM2DNdWT^nHZ0XZIHxm9hA0Klf z1H52Ztb|?G8h{YT?@8M`!o!?q72rL#4I#W-6!OU7W}_-wnTFJiY;ocE`#`YHw4zh- zkgK6wJGTyAWfJ4n&AXI{3T6iQ`4>jEh{7RNN`gqwz^%3BS8pEBvlk>hX zkUgvgTh#tZyzFEPOw!Qe@xs-SZOTbb8)@m~sz|TX2`?`wY+~-^mb}^K3f0v=$wf{n zX>$Xp9;dJu((2QS+L#kyb{!xQ_o~p0rUmW%QJRkRi~2J87_=40AUNk*!1s@;b>Rq48a@TnCM479X;TCozQ!EHVhB0Ky{Fdh=s?xMr()*^;? zeq}V*!nyr?!bf;y&8!=T#eO#b+tV=Jp_ROSO#ZFsD5BAu_cf(;LHN2Z@Hw7|!oVcF z2}MZ^hK5c9R;e$`Kc3+MzEu*xKmHg9Chw;?IFkksS*(gvU2TJM2}~rPDi1X@T?sfB z9IQFA_T2WIOeSZZGj<@>%4orJKSd{A>ZZkWIjK=Z%f@1n=H#Z#Kx}QQ`+>Rg`8sbp zUWDC%cBqEut`yiw2h8F+ZaFx5Il*bHfAfY;-bLB6m07oCK$#tL)I4#Xm2WDSe7bK+ zVv0`9tHLd?nQm_~_Drxm#*4MunFMGqCzwmQ39>{)l~qIIkr6ggj5aYGjcZ_2>T&DOFQc-gofspiW9h_x4BhnI25>umSHK zAqS019B)Lr+ED8OzR%*t{*raLKV5xm$CGnAs+TvY2%^$=!Lw-bZ=VSQTLAUTJs0H% zs-=6>!PwA&erKl$19lzxmxe8c;i1j(OfH|+lU}#F7Ev_=jPY>K^dc{rDl2R+Omolq ztIv8Z`cI9DJf3TsYnbQ2?*`P`xs`N%P%D!&JUSbsSyl1*JO5K4uLqK~Ps)Iq%=d0C znfvT57RM^FSvvn1@!mstE>~_HAr;@TURqynHd9#XkhG9$;D_RQ+>G!?R!|^cMR@E= z2iQ6m!WOC2LQL_7i`t{T#Id#`jW_Gjx*57!e`#GP7Kh_(jm%$*ElKYDb$AV@TtYc( zymQRwh7NrOqZo|7B)9*&DaV|2%;Uz#6m~O(^)BaJDlu_89!_a)Jf5TwooHD9Wig5co1YMW*GOu!TZuoGs*p0T?Smx59SPb#4 zPkZB!wdQI7$0W(@KIG$80Vn&HO@u0xDYGaR?k~2{TrDn0vz`hBXQ$XALvsW+ZAjmi zSNcU&FGZs?PR8aUNu4mH)}@|CFXK}S$T(!FNeOLk2d$=+!WXIz!3 z`t`miXxdq5X+4Ib&*ngw5{eXi(7!%k-T6g)VXC_|^n5(_hrxnbA+}V)3 z0o%*a&dER=@i{tZ3A$+t_y8biU`Y#_M`^p3MdHP%?>O<9E7+@$lt@h}D}C&b@{)Xe zP2y~1c|p5;wkHtBKAhx5ZT^5SNf$@TK;Jim2)zu>_%~`#T*LeEXL%RvK{(|JiVarU z4>}UsTmNgCD^cFweSTZNoavKu&GFq{VDUHP44qM_5EFX@q$X$LbiB}ep|{S=SBN5d zU=|$C+=(?6YpsKCd(SV#GZ*s0Wu>#(OAsdnpVF`sW(}?K3VYF=p$ZRqa_pF5v+Cb> zpU5w*hcGVamQU5Tm#k_gtnEKy215=j0p2LV2_T&y(6Pa5As6JRf8T>|$1VZUw-+zIp=w8a+H-E44!t z!1uN!079y6eN=(m+lWvX1Zlal%_z8hVf+r;n5@lA3claew3_5LJb(iHQ9!Knle946 zEnwk1qp(-PdrnJ4HDhjSSoWs!cLU2PeYofDRS&-*l)TY>phbU-67L}%1LF#T6)f7z2{jLqtp zew6y&f{qC}qb-LxuF9$du`pA!3eS!#VQze3`PPr-*iD>Ry=<>W_(TBOQF zhlFVg+D;><=J|;R(<(=?xML~!>l1*#_+tio)?315fRCU0&f2(nm?F*-Fgvli-N5X* z_u@6iYGLY5vfMXoLTwgh{2|lzjVJ-o#^~0I%%eips%@$^On*^f3zK(;1($qZAxBmwyh)$qZU(QxPv=gwC`&TZ12S=?WQW-NC58_ojY2^ z|2+%5I`2+%>N>^&m8~IGa;nMZl8A^x_x7Ls=8Lns2ToWW)FdBJ=J5mzudoS53 zD@>wcbJCj{)J3njQ}{s@h7O4nH?$xHiWsXekcuT4S2Lrtm{h_eKVlqk*$^0bUPL`v zsOb}A17mK5YSLAFZ2&wNd#!y#rjqq|i*MftVy2YEb3zUE<(DOwkC42r1wOlIXGEld zo>h|xk5hM_hxRzmNdNYYe~^3w6YbmU_qqcWMiwroeJ-F@z9 zP1?Ddgt1;HbLH~zF+FZFlWDSC!o~7F7gG(dM^<&R*jYWaR>mv*zXk}amp+z!H5q?d1;)WO=9jP1ytH{ zu!_;EMZ-$Fg1BF8)MODYiQ(=VRkdH&)#O{sMe`j7oPXsra6#0}3YPcotTS}TOD*y3 zY4XGI`Es4VtSfcSaGTMyf3Hpps$fsO=!Oddhezs`H;&uCuo`UgAC~^GtgBU%?tPyeJ5wd;Nsd(Z&PZ6cPfeu1KE6s-8 zt->+#ZA`6RDbR&@8Y~jK+_eTdl)b6~H`}CiI~08Fn>!0V+Ua2D($k14?@;FhRc(*|a6Snwd9+^hkQX6K zvLL4szKlx64LjB`4W;sKd|L`}@G=KzMwG)uteTy0qkZMg*cD$_B0c$*s7!ysT;2l* zajYJze_1ylPoB|pqnGxr4>2HCo*+*RXiwjLJAcPTy_$b?;o-;WC36S%0MjOmqJ-#k zV8P2q%bV;g+xpPG{j-Ze84`8g_A2-9@Z*7;jUrnTejL5Q|9y%VlvpwIHApS#BBSoL z`se6-iTjV2jopTdB%!Zsj{C~9@(Y9a7lRO+80GL)b-B5mrn^T|4u=^P8KY4;S|Sci zK8v=dOsYys%PMK|hqAg=q@uAb9g(5*`kk*dV4zXjH z>r31mIeteF7XO z(lNyC+;o*^S$-GbQnXD31H>H;pmX0+%V?&9L(zIB`hp@f^aI~$_;ZIv!35k~e~L#? zT4O3MiU+mcNjXs4zklvX#BDY`Hs&bK{6ziVXBABXo7F{;@S3W9u=Fddw@IPnbMj=_{@(QrKH4DA$207Cleh zYI)RD`=n9&W~G&lfJ(C1BZ*^{70vu13)7Z1Iu1qD`*tC!26%B(gBHCB&r5~}J%u2l zT@}9ltlwpgoAbF@?HpsrIZ5nz{MEyl_o^&DF)ifs(nZ7&0_dOtR^}XlESkIrV$-j7<>d#x46|S`_xCte@o9XYoXqeSO?>-su0USe(1Wk{qprft>fE(1= zN?P{HtZRpNHF8TnY=^p4q$-TQp(A)k19o97QHBmVj!Cb(R4JHojX~&o z3JU_y=i}wJ(W~c1lL3_3=2ha#SAL#r8Toc|)$AT#Eqd9D(9BlM94?9cTK)CNOll)K z&&y)McL37SkhV%ANdk>LLv1$QTAZSmL%WD6_5&G{z5Du&%&WGBPJ5HK0mge^zpIDq z$yrR(!7T_0o}Yd~PbFYgg)#jeDCdnRc7IVm86^s#%T07ivNTS6BZX~8+|K-w&{nr@ zDdd&$e;fcI9e-YrnZ`Gze)`d+z!*g^MH?Y{pIVI9ot&sxYo8}70dQZ--;*18KB%3a zgmVJ5&l~(z2{i5Pk_Zf`w&QtG&}Q(`WQPy1&gjOjp^MW}I#{RrInv7<3Q_-CQZA-VecXfffAbl+w^FeJoYYS`Qw@zckp1@d zH9@apQjWtU^v>yyE`Lk~$R6dZE_7?sQy9I?8Bsj0HA4Jxiy6?Hnhp-+Ucx!AywbUZ z`ekA_59YU;Ad5F}TrmF*qg}KXmT#(}Y-0=!E3EgF4kwfdY43YQ1fR$bz1Ash!>j#O zd;%Hs(a<{Q0nJZqsiX~m(jOYo%Gg(wyS#fAPvP?%&LDTwQ}7$$3${qB1W+IuN`&vMe8t>7{p%u zT|rL6;ehv`Ym@0>$$Q0}nkt;kNN_C%%a39SQ5Rn*OdqDUVd-KgtUMKCNUY}Ypqv~H z+<$d7%gP(!C}07x)-WY0wPdCkD<{#^h5&kmT+Za4EF%_)&mDk5kIVQIiX?76k6KUsQlldg7Qj}r>-&{1>?tTY#Z1PI zMptEPH+}REV3TLVjvD4DyC}Cqy>p5uSE6DG7rb9`*_q=+kTPgUUaxyu3=t_)l+yv_?A$U0vx(XfrQt-$m6b2-zz4wwAaeQ2wD7 zg!DP=2r$8O!l)ZpK^sxC1@|fG2e$&_8v?X|iUj~wgc>7G$7?4L*Z9NU;^n2lXCYaL zmvd@GDzOapy@)$HAndsn0)y1H#<;r-9Sd`vI-^q>$=;@{Z#zML_>2RBGtPbuGud*k zDKZul_%CHoym~yS={b)|n|*B<+BB1{mP$F<>}(hGQ60W;tza^<#LhN@r|?K) z(|yT513f>TQJ&E}25J@K9ieMxy2ZXyq{pE4IQFpq(6VyUo^$nEb`_iVIm1$^O-evI zd?+?OdDA+QDsQmCDheBN+uaEM`L$ei!))e=M&Awu zjhTpF#MT9?2TEU_3T)fZsQM=~>L^9oiIu&Y_~lh3Vv_p~ zn4;>RAD&{{{>sp7&qrr}?z&`K?X}-*_fDe~je;o7m-de7Kt8B-y?!cTQ?R3?X0-I| zLV)cbFiwTO#H195?C1g7>gAB)#ftU+mM zRuZBj=d_wDvPrqyYukX4V3}IWzPl0B8R6w9=1#xVuCOdt{LID6mPh@wg?D3PXo?rV zd8}5i}iJ9F;{D6gIzPu18{nc(;8Dz!lQa3a%HGC*V;)6&}Wc{efp{jZVm_L7eY0e4X;9TBX|5<9!m)k{RLKEnXYp=x*|NjN^7<)KSO`zk zEUdnv{m_zDfE$_jwSR!Pg1F?kP~50JsY?^nGJk16olkTTWa0IG6Wy18^I>+4-K|?W z#O-GGa~? zSfhRkIy)3RY$I~>?)o{DMGQ98qWtvcv%ziiH+kd)r^0a@+P1mNRUPA!6VG@-=~nfA zd&F{;leO5$a~F1NVF0+EbpAcl@ll5&;9kdgrg_~jG&MpKYSX!so$ZD)a}|$UM>@OD zq!}r0&kDt-)kC(<`< zIn5lG9qpjDAa73=#ZaJJ6=~H9J@J?S%K3`(8o4J3TC(y>Z97P!FaHoQLM^`F%RPNIfM%H?72cr zhpMp3rS8^fDT?+6ak)+_DOhM{_fxC~VSV5$waYtci{(sfJa3dQhSAr6873zcxJkN4 zN$2LRrjEY=Dqp8DlPhVCi&tRc5-aCMRS8yIL((t56NWSc0s;PKSlQa6RLN+m9K?^lCE(gQb({OSdpCWG~Rq@Ud)L@3H zy&{Jl>jY>xzj`TnF2EFMv0YQcOr7kut`QN8OF;oFb9UQ;;DgKlks_pjTw$Vg>jeYo z0R~4DnV}`sq9^>Z-{g$7P($dUulGsxrobsXiuw74^%q8?fwROEhwWKYMd1Y8BRgtr z&?X9(DeUlqul~WHhS)2Gy(nD(`y!87Xw^i-vJGAmEZwMx*ra&M5O*6LcER>LjV61# zEHsjd(tG;NUpZ07b~uHVkp3#FJ7UYR&%dnSi95X-zN3Reb|LeNc!CmKw|QCNqxz_`SJY>;B_=+isG{6XLDZ%^8Otxa?M|5qj%>rw+i?sEk}H}C(k7JW6XnsY zWYxq(q5nXr+q2IvbbRDC8DgkQB?=Ypy6v5Ky@=jxWZsi|0M0%MGmDjDlF!Dkx3tdCvA{?S}H zO!#$40A;iW&3nhR`E>V2RX& z&es|i6~+}ddUVQZsRc8o!~&Z-Yifl4xhbg&hTNh))Vi2 zl`DaqAGyK1eO0a`;0$xg%56tt1S4`cp51U?cW}FgUDD}ZF)3IPSKoNBP!>!%DQB3( zit<&MD#ywALG36gr}>1Kdo;r=?S>O9bz0bfM7=|?O21%S@Ak@HwHGbgjl?(5@2bq53B3z><|YFog{NV?~o z4%)!}fX9{i;#>6gth9df*U^M78{6|aRAopT5zkk=PISLRTQhp{fyq(CUmedZo40K6I;cSD9t|;@WjNmSzQzdzmupaxP5dI=l90A9+l#CTD!*mRtxD3q#CEsfr z0jvw(pl`-9NyT(fnYew|1fL{F*N@qhV{tW$Gk1w%vvZMr+(wS!0yzrwjwZuvT2+nsf-q_fG18dj z02d6T07wMOr>X`&W(ut+%VIxvgav4%mCHLNWS6d%oo`_~t&u-AQj*Wg zY9kR{r*0?oc}fgkn_*1MV4|r|qk})4X05xsQXejKzXQXOrpTmVU&CoD8=xH`_Ses;({vNQd7Hx955)pRsPBH?XDdHZiIx zhH6lK>nxcA zs|aCAvPz)p0yjo|Z}IC(hi=x%&k~ z^9wKMj8o7+F-8#@PtI5=?dfj;;T?r`J*6o88m!;c2ZiLJ>$rLV&?}Z(L_8Fyu)k7Q zuDCVWoe|S}In*k*j|1IIN&=!>*x)3l|1#8HXT-w{ zCaNPk&@?dp|A7dny-&*X*0eW2)h?3z3>}P3Qfh43g!_$LG`D>D_P^`_uXhA>FmJk? zTeUkEq7-HJuHNQCiN^b9A!cR`jS63Bm-b43DC-2(tzf_pf;p2q9&dQ zMzcig=~&2A^Tzh5Yaol@b5{U~H;oc^+kL8wm&RyBE5F_fE_A@PG%w?7Cwb!+=X3%c z<}U4EN_y{S3iRuKKNKSpzm5Ee6?&?-kmf4boG?e@o$N}1+o>J;;oX`Qs34nb4LzW34N3i!m=gSPzY1CeXe5mX1BobxsrIt$u#9kNjunjtCg9j zl7fpMI4ZIdG-6IH%nDUN@zec7*X-2|_(4YbhIHs^{S(4YIKImU+@_tE6-FN~uo@hc zqmGwtcF2L#DpBzzA_)xSlEz)P^_t?!ZN4B+-38sj0SQv@$byRO#m#3`>^#{4Np0@w z<~vi{YtOx>6kNJ4uhsiWNF{e!{@It9_HOj`^054#Ho3HSaJ}=K4A3ew308l}6oYH$ z!IF3Tum#Dk&DmH(ryyTr^{UgZjn_gC)2KTAC6O1%|GL^}5W(bs2N6h;V=ATuz{qwZ zE5_X@y@{s0EI>WxVeY)xqi(~Kbbs7_Ck{}EeABUujRQnidNE)^8e!01t3v~G{jI*h zDMD~O>lGwnwDh7c6L^i#?O#Hzj|y#`FDU}{Iu5Sq0%fg$_}#F(Ugbe}vA2VVB>|!V zX#9V?d}ABuol+Gu6iedW6j3cm_8{F@E>KT^CS6W#)J^t6ife3WTN=l2s@lebWqJj7 zCQI9kJOV^Pa7vbcu=biBKg&^Z=lOKLH7^F7HG>qw|l>B8lwny=!&RA zAwka8bHLNOiWDUGc)&#;P$|~@nmouyYJ@`AXX?hKd3YC1gEJ}TQYnI+C)W45p^d6M zR57Uq4+q8h3Z7>&BhEh4c8b({&$YfgndM-t@z1hhzci4Gb#%;`wtHssyI2HCv;3=6 zTo04S3IWE+>vFV_KPKvU?&HAz>?hD46`{6O&RNdZ)P}bzk29zGrM7hVc<&o#L9)Ey z6m?r9re6r=4d=%d3J7%|a(fNgJTL9_NVb2z3~Ljz7kom9>?+v2iAZozZ-ZGlmMUFSPA>RR=O5HW;v^;G{4vl8l6DEq1E)|K%` z(9YigR3hWeFOF^0c`3Eca|@n|>Xf@8;!2|05sjGa_uV2LfDXF z?A!P|;^!|*AN1j}5~SuyeI8o8+8t$uI431aP(9pk{H_RfFJC8a}=by-M|`oBhNh5`Tr_srY}v0#o*>1Vl_*jt!JE%|K9kD4(z4id2-ZRqwZ^++(ihH6fu}mBFE`9;s@r`6Sd!H z%wG+X5K00Ev<3CQ0x4}hNsc1=Q5D}00r*xJ#j}Wr7%T)9{V9|0T{BxtW@6I?Un7_T|)WbLYZ^hE4lWS39i{^F<2LnFl`iLylY_hjN&MgpN5GyD-i=*jMdL z{hrbfIPp=C{N97>5Vg%S3)956!S9yDQ1MlPodo>vr3|hx=Upo>Y1tEf$y4J#5K7!{ydVV#CMU-r>{7wkqn!#+LMc;C$Gf8~>fa`!520@dZ${{2%rzwsLfq~2$K zD!sIgbUH69>K(b^7(73@s)r!)zcC_8d=N#P%Z5Wj#L9_j=gsM&j!U8g1O|T zRj1VCh(_bgz`SU=$GsvKym(IC~7zE-HlCUuMIn zhdP+pIsRCal0flKALV)jF;tlN<&=^JKV17=FQx`fg&p5+2wqCUj=h$?I@;l;oR2jZ z$gB;(Yvl00%%4@I7xS+u`DATf0c<+5 zkrdx6UXnfF-bU@1oJ^Q_*zEENWj0Ov+NQ8rsIvw5I&!#*;NnXPSTd5>D40V5S#d!nwEDWdPXcc{z>t@pfA(opz-;nUygp&fFC!2Jk4KA!Fxef#39gyEV0~sG$)_(;;CC zI##b)v2CuV@U_#qH&HG&F!341u0Lf9W%8hD#>*pN(9#v65=~@Nxk<_f^r0;TKFMxo z^KSXc4Z1_01))-b-X+gzCPsNyhZj(v z^V&ME%j?p=5yyG~fy6hCJgT$IC%kN!*b+#vb78fCwUncV@uI~m51A)b+s^F~(q%w9 z%A2H%HPI|vk(P=Zqvf-(8>WWjqOD<1H>)Is?iSz(QQbBZcbLl)cEepUE4Ep<}-NB@sP}0))`ZOF){OTlrF%?G_;OD&7NL;WI&4 zo)<;5C2c0rrF}MkTK#;a^lSTy)yCgTi*3*C%I3aj6Xc5bmdayolR!D9fWDyRy}w~T z+acSIAD8_MZIv~|)HVLOk7~KoC9`{w(|9kicMlzG(m-^@ajzGBqIsKHV-_!;pq3EKD;g0wXV8s&+ zC(y*^ta>9|`gS@h<-lL)`C#zL>8%3N0v)L?|QzC1*{Si;xG;{WdY7M%`tv z&EPlt*|j^RwJ<23>HFf7Cc3F{oIb&;P*=YZIQAQWhwnEx0msL7g3P(P-7&4QNkE2i zxxMtB7hOgwgD>jb?fO4uwA-@k{#5C6MqpDYh-G}q5P3iBsVRMHA$B;Bi}Qo43$xU* z$NdlFFUb~P%eQ*om(eK5hq1L~OfbLOycUZ($p6`Nr`4wc$$Mv>iHp)VlxIz72xmxd zip`zQJ<{Ms1OkG4Kx8ikpD};N^P-#*V@Qx|J^9Y%wAb%Z*(J z(lrbY_dAX%eq$wD)kg ztcU+FKj2Zg1Z=n5v+nRn&V*^2sNudMB9r0`M=sa_i(2BK91k_ zQxA7Z>Nd(froxbzaWjXFN^Z%TB*G*yg`o^(OeaYr$}#Q@DTm}VVk1Rj8>6Je9G6o< z4x6G`>G`g%`;X^(p5OnwuHWza{jTr6-}mA50cD&eivjLN%oJ}HOwkh+W;b@Zj5Jbc zD<1PzhseZEXUFJIy3znxk2Ac0nkuWjFv!|+=rsMK9s|`;O9qju0 zMmz5GZcp?cXC>3Vs%TiHuqgLY5R{PxHFM4lSK}~vljs!0SY=E)7z@7md7ts239}Gm zMFqh#wPQH*`uUC(G@lcvAoI%@Z&@dP7yj@N|Hw0_g)DcBQ(WbTHZe8R4cK@Z%r6*6oyRn*C2azd}(+(#+Cdxd)TKNs%eJbO+3tftD zM<_yI0ngq`qm9j{*idcbWLrRh55ENCciYv&t#RTvf|-WS(Jo-Y!JH#d6a(5Xlvoi1O?VY+oY6# zYlJ*qo6x@Xq-vc>+ryo#LAMm3y*BoK1Jk;^sxXVs)-@>N%<{`VEtWy_uxE0~r&pv5 znPDk`hE#5t?K7G3R>!r0;iq+b!^TpZ9}TMo%9Q0W$Wir3lb^%+@vwoXAZe?wsO(zn zmhV%WnBfR#V?(lDZLwC|lPeD09}mlNjC`)hyLNMs^j)0yy&rIR^Jd*%_*`YrKv`

}7x0hV)q5t$hQGc_3*#y(Tb9V|`qNWxp3_A+gz?rgx6@TnP6THK4Q+Jnda zEbd*97WW7s3;RBLt2lG}iU;S<>q$=v$M?4G-jvg1ey6E-YOfyLm!CTA!Fig=$CGf| zl9HNf8M^1(CP%r(J><)Dd?*n0jh)K8zs)wS5wpEN){vmLWdE-BI}ehX+xm5j>93!D zokSo^vA8-v&TfUiHb5qrAQ6M&)z0gr?SP-a-d&6b1d%DsBy&gUsiC6r(eokujNE;% zwjNT}!%ByY^1n&ZhtxZEx-il0_;4xFhcEoHuGF?BZYnz~vU{^@^fOT73OefJ(-#Z z<&xn2s@Gii{iTu`{krgJ`^2-JF5Uwa0z}F}TeO9(ts4ddjhBJb*J#^U2yFU^pa|S2 z`T4Acf%BX2_i6zAA0Xt7LXTzOt&-Dq@KZLvK0 zVpn4MLL>bH&G*IWo6~qNL6ry@5ue#}J%Z4uxw4e|% z{MGpmDxax>4rjA30(rw%aOW(+3*Ykw0q!p`=?TZRcN>bb6x=XtIt%K}5Z0S4y5W_vI~4TY!^!Gkq-Z+z#TyVc9g9`chijr0 zJ_Fh$qwHYt)6c!bu;`CY@FQhWp%VG;=tTBIhyquUE2Zwt^Ykh|t^pQR#SqEYhF zakg!)VC3wq^MX4KYpk2qZ3H(H0TqBz$ooCH{5x zv8sEvjczGpqv7!DL%0c3^{HLszfKwW=?Gv`Rl?M@>r44AX4__#j$%q9t9^T3kU~w=@tsJXQgk#a_>0NYwg*!vo zZQ=T8;+m{gyas$Fd3&csV7FtNW&kS(}!A`A1FaDjd!+muJ$K5dUx|{%wVDpstXdPSwLksSe+i7JK(!O-pqEAVo zCk<-KL591>Jx^9beF~5hIzRD>yq>s01$mh!R_#F57oMOI%Zl{No^`*s2uXG-Md15p z7N6eENV)sRmRu46^810|G$U;-UG08;kthC#>ygX`PuwUOgQxf$;xeczIgNDz#&tof z%Fw@=-hPc-#)}J_q@*wJf9h(=34mH6+%1b2v_Be2z^jCBvJ7)ocUam}K8{}xZGt(D z7nb)5-C1~*D4S4?@vm)52k{?{4DsJyG_J5^2pweaglaLr=2H~$F{*s^mJoJMBY3L{ zOfe^3?TlW?r}yosI%dPdfJ!3Rj5W+sv`wo*aeNqXexe|^bgPcmaJ8$`^~T+*3ikke zM$~Dqc42^a*7s1h8CWvZO-Tgl5$HJXWe^H&>NC16tS4omrg~%R`ezp$o0pcA)8r}V z8j|+sv@69?Wwv{*Fy*6NqrdELJCNM0W(hZh1esYkkJjsLG<|2-eBW^T*VqiLkPX#; zaMn*tG1?oIr?%?(LV~ilkH(rZkfF{g=ZMJUlaJ0_A-r^N(@WC6@o$Z>pC)(1U zhv_}fxCM4ij+O!X#}Q&tO-j*bslaP7k0cLNC^z`wC1IDsfC$!S1Tr}}Y?jdf$04YI zzOFWy=UlWuFeW|;F@%2yVAl9x#tb4o&Ciy=SV5SJT;*RRIF}0p+ zf#b@>T~=Zn{tWf_8>{;IAyxjD-b);ZDI4^W##$(ivRvy7`YixGawLILf}VcDe@E?( z&mqmS{U>dp(}5HGPAKc*C%FvmoPXno3wq-j5<&nGP+Z(UzHR2DqGuo_5fKyGKiatD z;0`f8PIbz>0z*L-Qne~|$|*ZKlzyKyVXmb|-vskS35irl#K-2s8wfahE~!CD!y?@S zu263#Dln;_1nBqK@qkonL!)5Y?K83Yjagu?oyKeeLn`qJEflNlw8sc$oWsd1&P8ip*Q5 zG#u5C)8WtVwTytp2$kC+>UOaF7Kkum{zRb!uQI*fkjHV%zNWLD^A2p6 z+?vb)=QlO`{<6nUivFjA>D;yCp6eFF6cHcBh|Imnokxq*b&U=sr3sxJ)q;u8h`@TF ur4lu?Lf{x)+R+O(dDZ{KL@2-vT>D-_?0moH;A!eF;Ig;H9jMsvef2+GkkJPK literal 0 HcmV?d00001 From 0507bd918c1fa157963b3e8d0d172d35271ca2fb Mon Sep 17 00:00:00 2001 From: Ken Liao Date: Mon, 17 Mar 2025 01:11:17 -0700 Subject: [PATCH 2/5] Add message delivery for shared non-durable subscription --- .../design_doc.md | 19 ++++++++++++++++++ .../img/img_10.png | Bin 0 -> 89800 bytes .../img/img_7.png | Bin 0 -> 100338 bytes .../img/img_8.png | Bin 0 -> 94378 bytes .../img/img_9.png | Bin 0 -> 78149 bytes 5 files changed, 19 insertions(+) create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_10.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_7.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_8.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_9.png diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md index 15efef28c61..8ff67be0a70 100644 --- a/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md +++ b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md @@ -98,3 +98,22 @@ When *`acknowledge`* is invoked on the *`AbstractRegion`*(which *`TopicRegion`* When a consumer invokes *`pullMessage`* on a *`SharedSubscription`* then the *`prefetchExtension`* of the subscription is set to the pull quantity. *`dispatchPending`* is then triggered on the parent *`SharedDurableTopicSubscription`* doing a round robin on subscriptions that have space in their prefetch. This means that *`SharedSubscription`s* on other consumers may also receive the messages in pending. This is to maintain balanced message distribution across consumers. +### 3\. Message delivery to shared non-durable subscriptions + +A new class *`SharedNonDurableTopicSubscription`* is created which extends *`TopicSubscription`* . This class can reuse a lot of the logic from a *`TopicSubscription`* for dispatch of messages in memory. Similar to *`SharedDurableTopicSubscription`*`,` it holds a set of *`SharedSubscription`* which are responsible for distributing messages to the shared consumers. +The *`SharedNonDurableTopicSubscription`* has a *`PendingMessageCursor`* which is a +*`VMPendingMessageCursor`* containing messages for dispatch. Each of the *`SharedSubscriptions`* have their own *`VMPendingMessageCursor`* which contains messages to be dispatched to each consumer. The *`add`* method of *`SharedDurableTopicSubscription`* will add a message to its *`VMPendingMessageCursor`* . It will then do a round robin to decide which subscription gets the message added to their *`VMPendingMessageCursor`*. + +![](img/img_7.png) + +The *`SharedSubscription`* will then invoke *`DispatchPending`* to dispatch messages to the relevant consumer via the appropriate transport connection. When messages are dispatched they are removed from the *`pending`* *`MessageCursor`* and added to a list of *`MessageReference`* called *`dispatched`*. The *`SharedNonDurableTopicSubscription`* will also move the dispatched message from its *`matched`* *`MessageCursor`* to its *`dispatched` `DispatchNode`* list. + +![](img/img_8.png) + +Similar to *`SharedDurableTopicSubscription`* the *`acknowledge(final ConnectionContext context,final MessageAck ack)`* method from *`PrefetchSubscription`*(which *`SharedSubscription`* extends) is called to remove the appropriate messages from the *`dispatched`* list. It would then invoke the overridden *`acknowledge`* method on *`SharedSubscription`* which could be used to call back to the *`acknowledge`* method on its parent *`SharedNonDurableTopicSubscription`* . + +![](img/img_9.png) + +When a consumer invokes *`pullMessage`* on a *`SharedSubscription`* then the *`prefetchExtension`* of the subscription is set to the pull quantity. *`dispatchMatched`* is then triggered on the parent *`SharedNonDurableTopicSubscription`* doing a round robin on subscriptions that have space in their prefetch. This means that *`SharedSubscription`s* on other consumers may also receive the messages in pending. This is to maintain balanced message distribution across consumers. + +![](img/img_10.png) \ No newline at end of file diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_10.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_10.png new file mode 100644 index 0000000000000000000000000000000000000000..deefbe289e88a49c69fe9ba09f3c5150982e07ad GIT binary patch literal 89800 zcmbrmXEa=I`1czTM6`%Ri53w=v>+G^BFZGuBhg!o5;23(M;AeKqQ@Yj$LI{v`{+bx zbViLfS`eJ=_kYeh&$HHfbJkh=4P&$KeeZi;*L8otpG%0QI)v)h{ae?rU88#SQt{2T zYu6pGT_X~>NeX-t&KB_m_{PLWK|%ACg2Dq$M|&$9nB}!=k7Hb-g&h&y@ zQc>(Ph{QiINj@Smh>_ZnS4Y7y&6V^c`>K*+*x>Bl-y@7-nB_ZKDo(fGJ{wJNntww$ zlr5GL@Ywx(`FDTzZ1&iXEviGWi{!gesj8Uxs;Pm=UM+LSZaf%z&P3wR2mt()wJ@S3+MSH6tJTGyZ!S zoAdpJJSjp^_)I)C%Mzi=Zci!XeiXH|4u+5Sw!n&S3Ey zy(~l8LXnByH#TI=%m(&4OxC3-nhv>!Rkv2l-VDDpPPsKXZA#`r1M=;;wvG6H*mUBV zx<4lTK#oYaPvc<30_hPk5q9m`^J}jZ<=?s!{YfRO6BMhgg6!H212HCx^*9=OeKEva(sdwJ|TwI}yeEDsdmv7k4=_%Fu)ep8!FYu-{ z?13G6P7j*|Z<7x`rK$;)St$3{L9HsvyIOa|{1Q)1kdfMWKhBMwtO zD4Z_zOYayJ8X}?RL0;F7F3#pI)36EyA^#{}%5|fr?34H|4szuRH}+O&eE%v`&A8%R zR>7;13{woa0;;6koP(5nO!Acd8V#1+_uo%;`!zY~?dl~a*q8H!A^Yg6%JH{o&-!o) zBW-PX>+}zHXPxlhFUp_)cQl<=UW?rdqGGPejFkGhWEI-=g2(Zp`|~SQLo9M_Xzhm0eW(WVOWgiM2lM` zP}{+r-&Q8Oju*61VWgIZ9Fk2=hN6=-h;2Rf2iRpE_7(-An9-V5yeeqUA+0m;J{Xt;RA+8wDy|+b~d1 zA>K!U85RYPpK7=R^|)!RU&N!cL^+mCnO|n^qt9)=;-4}uX36}$Gu&e;=u1r1opY;~ zjd@FTuq9;Z+p`a3Hf?!fU|~Xb_x)BSuQ)DKsYe>&6*L8c-f@J_1`Q$-9OCDpvA}OB z*Jm-;XlsrN=)K)v!&%s+5agJ}6)B}6>8*!P7J6r;8YWI}qbNn4arCS)&k7)#VJKF)7%DCF1@#7yrCx2^i8tcqpJVi2m3U2_47i1(%NoIiT?W0E8c_;|#ymU?)seYD26 zYMcD(q(%dTwhRAZRpA>P%l}*;O~FOj^e4-@#vS(+(LTAcD`zQV4dF(Dfs}2jI4nrfVzhs~s@eutsfr8#UWXf#Lx76#i z#{BtP%<+??cn;3)wZpMi zjFg$(T#S<;A#Tl#4oun9)=Ciit6H7|}DN%6M`w{U_r&i4-%;Qz? zl(147TrmV9j+mU7!s~909liiJEjkH5zE$AR~8Vn0yY1M3$hp=Hf1k>!DS%(mLb9y|^o#M2R zy5hdIV8QMm0?<|WmnQl<(eQ_^d2l?vEK|k0myk-|xB0tX@gAwNu{rmibedtD+q2NpT$1BX91D zzj}R65D+{;bG-JjI{i?%_-;NebHAB9Co;Rz>NY~BZL<|+c5>hU84*)mQ6EV=&&$^= zj7SrhmpL)NCG>W%Pn-7^hQ>NKY^$KFHIGq`B}q0Jb$g?5w9rRWY7Gn*J&bodP{tN| z7veDClgiV2jmHO)qjbU*lDjE3> zXZ0FsC5mlBPvg79UR>r+K3&W?Vm{rxi-ZDk+mN|vY>yPTupAj75mrjIYG9fr!#SQ7 z)5CC&L5b4ngVEg^i$RPM8f?!x1USV^KO*q4!)*$G0w2Xec z%C^KYV1mrD(o(k*uO3p2GwfZOo_e@E<=f=tt|-uA90@j6QH_nU=PQlRE$ZGn2aE!8 zgNiH9v@Ij|pIV+;BcbVN{bta2cvEDT!q{8f0~|(^JBatxKMI7qbEsqWU#7@&4L^J= zMEF4b9xbqsp(21sr1|dnEC*!D+(5Rd;!F7Lnhc^&E_c^FJhRhC_!*9Azr8w?flE3BJpXq2_oi>L|&$AQP*84Q;BLf zSGpInJH|lgJ|ZEOM_JbSW#b)dkC$mku5A8{<=CT%&!F=v8c!*6ZT`hLE+KQg(#JX) zNCTvev8s+S|77q>es=qqg93s7e=?7)of36B!lzE*cM_Ln7vlAr;RZV1vfvApf0t{| z(V-|lIaC@e*{{XtL&}D0wX;s;5PiA=XYMC4WYorim9Y2{wWsp-aTwL#_8%Md(eRw9 zXwqn|#wv|)VPb)& z+I9yUG7XKAk(%lU-S7Dl&l6}GYu(_?1{zR83w0DK%*NHkNRkA@O2>#JG7(+K3X90p*|-_xH{0`` z*#c^R$PJGUR_8%Ew5*x70|nV*Ac!q~S2!n?X*O$Zyf`e@ho73;C?F;rgwKN)>TY51Q8)B%cYSi4Ta=eb+Cv-gGnp@ zQ5%1;!*G;a4Ep76?n*GbkGs!OT>MWwwXMxHqrw{HdULN|12Z09*|+GIHuEETW)ePB zF`90BM;jw$ee825SRBCe4V!{lGU;}%%y#Je_ zzU!O6uY1vlzy}E>g}YSdFCh&lnjIQ!G$)PE7Jyne5;-IQ4C_r%TPGc~AMECe`x$*3;tYfT0=a)EHGG3#Br0r5+BucJkrgdvNLK1RVfh)WCxOalB@0E(0P+ zJ$dE2u1QwMnZvTDA|;|n`6JjIFTX@9+>vX>8RTeL!>xik42xm^U-RTo=F`&@(5`)5 zn_#G%s2ESoLhh2-DQf=ia(@L<{%ay*7TpeR2B#d-SmqvjI@C8=x zA_m)SFx=5$Zt*aRA;X=B5-ibLiJ(7O9$g_N#08qncUWiz4f5D)Z?89MBc|{NUvsg8 zF@+W@cMg^;<1>!ld_4(-@>5Rc@Z-km2ZYN&A1J8h=T0>3(!u-L3K&{qAd9YbPWPfz z=aL=Ma;HcqwQ(Y!!(6laCNxZ7*5N?4p*5@7_&aEVdOF%Svp0LfO2G@xM)Z~uc}s}5 zW8StM%1Xn+@OI7#Hu8!m!=J^-!OYh2%p$I)()oY3FCL|SOTr_pArly0sPGbRPikm0*-0Zck^z%W{O{u05N$xYJ zM)F87`em1dUl;>Z^&v~e!ie}-|M*CmwnG}vVOfJ7W;r=Xo!pYOQY0*1KUS*%@@u{a z1nz08#`x({h)If}%HnnTtW65Nx-#pU9kW$8?^fNdxjU)CTYh(DOeNVpjq#F^lbnz# zlN;F{=Y|$IyJZvgtxm5Blt<_U@>8`i5~IZ8_}Er+b^=Udg7z47u8`flW;;XZu36yg zk!ox65Rl{6VCt?wiS+5};$zrL7dv2_>c}y^IgboNle?xbx++K;i|Uf_{p{U9X2U>TPRc~mn1q}}oT4LmAe!_S~LLkQY_S1*51Y1Pu z(3gz97>#;i)>0^<=M|?(+DKiJxZpkQ%&=xfSU9IdPR~C3*-h6FOokCM@-9Yh?6O078{mkVDB&3SlKOc6Uugy6VJ&d6ohY? zjLIDelhumKGwoH#A+&`F3?KDJ4SQCyQbn&cadt5flOPc4aup!QR%?BtfZJRwRVv_WD z!21~+f;t8XF>`APN5csd+2JV(%DDHYuuDKXXuu&?Q>8m%2tm)2lEW7BS-a}s^ zfO3qhe^X`!>yECXk0SI!xJBYPBlM8w0?fjMI7=p*w$M`5EUtd3c+SFw*OFWq^te~P z2(*TuQ~6^<#mM_?#73r%CUV0{YNg#hM6^(~rzi1As}tz6c7kPAciEpeh+Lt4#Snpp zAM|E}mP3JrnOwt}4V{xEEJNKqNv+KBbh+ourb%KdYVLg7ZDzFC=Wv@$?ytNL=JUBr z&+luCVC(sE(k#5_7f+xF*f_ct3|kXT+Y8^0iSn{YaEkJr&a9n1EE+#Q8Sw4bOZr#d zMrH5bx~-jl`nE%_s%(^Ka?GABZufVTPetuNywKd9(wT}0;p@!3V)yp#+2aPczI~~b&WvHEKujw&b5t+GHiVOp0?&3nlNemVmOy~&Aw(z6 zI_=cb?97J#18__z3zYmb9hy<5suk;V{j^AZpN!Ocn{~$>a&a`wJomR_Tb{;46Z^=g z(AK!%3=r8;d4~R)mEn^QB)c^UNsKD|BGuA~CLrVyPyci9Uy=~bkgan0vO6TF_TNRW@&?Wmx zCxr0xwq*(3;n`pjyh;^3pv895Y5V>gNy$BL*KuRFKEr~veVc6NpZeeR&BP49OF9-D z{CR34oG*%4s+;+@A~Eo-=6Y}}B6ZV<=kk~@a`=5PYr}iF?SC#2*|}nErj?_N&LZ$a8v$Es-ad z{bDzGMz`^Fbo&QG{Rg*Bj^O3^>|oi0jG2xIW+#1Fz0;Cl8Q21qxK3lIXSa6RkT)Q` z?wB0@)WzR>+Tf+}ydzuCCBM$&I7gzMSNf*rMyoz(w~4q`2)K=z_#)@*x}hLJRlBwB zC->mfoaY&Tp=|p}c7P_?v(@p-p+$P}3#UjVyq63-Poz&B$wIauntq~cVgu;4JL$v! zN?JO8vOBV$O<-3rj_UIZ6$q0PmK!bvHzzOnWeY>Y?u%5=4ht+mW|20kb2T0l;KUVn z;io>a9jC3n4v4`!7OQrb=f?vcCD%J3X2&ybbKy&u(ug|!nHx?ixM}VspYu&qH!@ad zO~Ia9PbDaB^sn-no^;o>bk8a7jk`}Pnu%i8*3wZ7u-?mrva+BVeaVY?GP#Mpmra@M z=WBnwh8gtFv)KCCSHr9;K{TKwQ0pA7Jxx#8FMXX3a2QC)$RF%GNJ8JvJkVNfd zA3^6S_1yv+)gcfI=oyXAZos}j=cm?BqH9^^WwL{_rmf1Ai;qggGMHA$s? zfZN$P!r3?eiC$`QWLuY*=q!CmVT_f-IN9=gQ`RzJbY|;%GUr}*8%3UCr$j6J^Q%iw z4KDtjr;a+(0cL3>hmmq&t5B>j{| z@?8*naYx-Z=i43lyfgBT+I+wULhr>qCWGT%I`yC3EqiQgd z%!M^i&{?&zu@u@&WeL^)c3{0e&`>UG6aRQgLwc)1-|@ONkeCasZ31oObw%b*`rZ16 z9>pSxy|-+wySfb4XlxeKvtM~1eH*XiPRuRNy7%3Pouow9AP~hB3|01h8M*LB_V|}Z zf|CAJicX~C>DBAUFDYEUe7k?><2wv7;)Ubqa&?<_SP<3FhDJhannDT}RucO6RY6L5 z#hs*K6Ed57wyu>0#sd38u1*QDs0!2yPvPg@q3#iWN$y{%SMkk>ZIv9C9gmp|GdcL7 z@0euOCE0s4zFi?98CAY~uq@J*pPp^KU&b=^6}6=AIN@kAWoJA{tUTqKnl11bRfSmU z(0A;0M5R}}OWjI9q0#mRuX>Q_m;Y9+fpJ+$e+@`Jkvshz!hiKd{BpCQF`gqce8JCPkp3=qDIwx)9>VLfb5Mtwvh?0F3RsvADK`4?&A1!=FqOp zj(x%!k%8?Kr0Dm=G;GXwMCDMuGpBklUl7I&>RJopiLe_%hHx>;x)E|h>P0kodr<$K zS-VX=na=kTR+``CG96DS3+|)MN-Tp@fs)WOc1sD~`jN@dX4gF-oPlJ*puM3B+2dB1 zuZ`bQz=-i;McvWQNLisTh6Q^EB<>{%q1$wU( zqn^EA^_XzY^wF6e41vEhxjApdy~?vWf4P$LioYk@WRYdr8YCEhgMS7LMi^d*tH~H+|`>5fS7Sb-P6T@Cao+3!wYROhBTmg zzkmG9msu?vj9cG*OCG*9Q;L=C`1O5;trHXO9VDeMpiZYMsN#E2C$CT5aRd_zSYR=u zOIwF}#xqC>`s{^S*SpPmAA6nWhCDb9;dfxY9Gr7ch#j^0!*rI?w=(VE?YBj7q^9$- zl-KZdqCtlgd7u@U;U{avjG)e6SdU_X?%z6fFu^)h1lC|3)>?{)w%()e-!cO>wAye5 zPjaYuR-e%@RNROfH<(Zgcw->!ja`NJKjD9{&?}d4kepdWaQw7fN+@SewUVlu_dEy@ zA?d)+4{Te>``O}OkfeA;hNt84A^r zz;Sk zl<2=^bKS*BGYEs|Z>F7agoe?xod?#WPB0LLME@Q+Ht!jJOCNW(hl^4?G?pv&%7v8E?vWiJk_mMVl1?#i{HkEvXbaKh8Q_ECjSv$W&O^KbPEo9&;y3-wE_&=D zW1da2Q(r+uwNT>6M;*$`UE|zo>_ZcgPT#>p-JEk}nrFFS0#yuIq%0efE7D1V#QIW~ z&2bTlp+4Y_RI8A0(^BW(0(_cq(9;|4Q-i{f7MmW^NVSU}SidMV`P-e@XPpB_Gki?3 z1x)BQstG;A96)&aEDX5Iq)ZqxW@=+%)Ih0ECR!K}EcH|y5&!PnI?G;&p?nNxg}u`e_)c85@*{Gq-dcBM zqyQVc<8NHneD-2x@vmRTn$Fru6FNDq*OM#j)X35uVIgIep??k@c+S4groL|I4x7oT zZ9!R4>~`xmIL>~0#iv|eF%)ZA-j^cP&#$)3gdO|nG0TUIX{WcmZ?uj_8iosSEW+Y* znjhF-#^g}Jq|3a&p}rcW!PN;vP2wu)x?MOMSabGjD?vvX*R#w6fxhR9-%@tA@{FuI zm^K=tJIm5YviJoXh{=r@`Z)IDzW#D{d><@xxoP>C;n+dx^0Z(@?AF`sTV@pT0xH9U z2P`JDIgk2Xc^%V_Dsu6KkSY0ZTPp{jurEv;L-$>Ukq~z81Aa+uB%grV^^@t}R{aQ% zNP)x*s;-!xz2ynZR2w;b^j>-TIV?d($r=VP#|NgyW{RuundgbA3>fx<(qRgN$_^+` z%&%C^>XOk&)>u(#l>x?;i}tZ+-fJNmUgkx?Aj2L+3cX=W)b`p}mN}y!*%;C~eGgZm z)za*NlWg`fd}!)$x4mG1H={0?W-ZyoFDZMp+9*N1gd`m`8dzAv3#Gu1z1gi087WP&4QqIne*e}1q6Eq@bD%b5phPc{`#B$;915Mbuw$ao#;a9QjT2$xp1K%1 z{EXc1K-6nY8l?X3sU|0d5C{njI&CwD5s#JZ4p4g}+L-mGNv+@bIZNA2hV|z$YRzs2 zCz8afzijY)f-qb5CsHwBJ0W#nUUJFxS3j!BN~2(FvgQg3uATg9ksWbQU=jQ$T`(x{ z9UrBkPYNj!U=#=v`g)dNWgY%9BemMiy3@oWh}zNy?=I%43xR zoJ8FMm9gXf+5#i|lgYDHfpl{FkftCqG08-L(&*~2_C{4vt}nw8HLX8(g3 z!==eX=8jQ864TW>QctAS<7T!KOaXx>XJ&2@K(3=YL7 zV9L)sdY&DKnv^ZXKB=qhO?WI6@0Xj}DK_YOpvHU*-Y4G3F=MBeNo^l-3h_q5KW}_&f~?KP8k)I5eFHzYa@!{N~q?c$w(3KCBX zak5&rDk1Be5`Yhb;Ycq@@)11*Ywx_p!G7C+ee>BUiBiy&?DdKM<^lBnx4H{3;*hC7 zs0_$k1MiV|d3>pH`=i!!X2NThQ{kJX#+~#u0}l~<9G)^HQyPBerkLcvCM>vA)iKqQ z8&mx{JaiRDBu@QKX`<=O4S^j_%3$L5tj?4l^GP2h2ja@9m$OHIfcHc)M7zxGH>^dh zaMY6E+BW2(fy@$=1!#OJcq>aU}^Bi>YvJ3U(|tS;Xocq&Q`6w1D>Ja*cV)-Q)>B z>(y#*cV$)Me7cmy!3-MsBI={BGgp*jMaa<)25KqSjIGWUrUu5L*LDqB&rK?*A}tKI zdbXakc?_ET>G{PoxC3mM7_s41(XXJ8(^&t#2Zgd$(ooaWsL1>FmizCa8@Y`iW8K-l zXjh##i}U%)XyO!2=fS%(GDRou7wyjh(fNR)pWK_Td&sI=k!=Yrp)e1QgR>oQm4E5T(R?Oa|)> z3+k@SzHlgL0$Pon=#T4{9(ri_H_j^jq3Y#uP#8jok?hRw z+K*oNOpGo2jT{Md+4Eot_u%F>WwSD;7UI=*qGDC;4zJ~C44ROgaIYBrAr6Y%AMJAN z4yi5!Co#rlQxZNh!sTTxqiw;ECvx4GtsJQF-E!Ie%*RMA<@_+b03>yZmY|(ekw8Y1`jN1lVvQWu#I=5Wh-;c=ZG}Ftveex3V|LCUb-IpO~h89==2rj)^u9Z)Y3!! z3>#LPC0+?Q@3-mjfJcZtp;bw9TlNMKQehvH9#^%ekm$sctQD*NIC5fP8CEYUcs19) zQ8tUK((~-4+tDM7Kx}+jPqIH>8OAco=?fon-8(Z40kADVBJWH5CS=Utf3Jm?kjL3w zrJj#OsK)?yXn+fsQQAn^pnH-~1jyaIFENyPXn-(?ob2%6WF0PTnqUZ%;iiX zPT_^>oAus?w(0G0*3qu$fcp?-CJBLv!zO4-3Jt=^C)!Y2V}4Orw zuty&qUtcKf!aumTOttMX*%NWX7v?z5K#e-e%73ybmv2H5A`1XFQcAs;lWjR;-kVrN z$|9wBKBD$B(){6Ub&`m2LP+(OLl$L|wkMDpmTD@nrXeYtGU|@kcPGjo!vOHKF$Lvg zbQwn&SxW{b=vfVA=R%?Mb-cj`dObe+5;>L8TGbv?+Y#57qm8OP_RfIhn!Y3qt)%nN z_vqH^P3mG)VZ1Nb1+BAUDIy{KCa!OyikK`dnlhrp7YsA@i%hjUXLXvJAk-JArW=s$ zjulXJW8|c=J`~Q8UZdC798?kLZh8(Spmn7zn1u&3d=ygq`%pW)t4M&5+8M#9Hv2bQ zO+{COH~H4VFgiWCq7MvIviKcl@Kb|4)6_i=sD!J$ml4%tDi)jWu`hVtwGV|1`3^+oCsztpkeu zT|{6R|84uB*J`ol6_NGiH}F%lcK`hK;P4D_ZEDI>$=ng=8w%eedH3V0c`@>GpWq=| ze#NXGiV5GntFIPKhBiEJE6ZC-eG5Q=`T+jQHC(*nKm4E)DGXYkVy~m-*GMBu^VWR* zOO_e1Av-a?@Gt2k%^z-lf+%I1=VZ*^{zQa_;6relPfOVlnmt1=qZ+6qe1e8<6PV%u za}z@JW?mL3uBSmF;&)giGPFAujUJz0X_6;_aS_ZC=f)_)57lKgrt(}Vjm*?-<4Z z&{dx<#F&+*-{J z^R~a}-%?-2o&u^k;jF|P9)TC2(1AQpdn=unGU8>^*38^WwQh}EtAh>arh9FObcT&o zGvaM9rf<)W$rZ>yZw->92ruTfxL{OfSA z@QO;8gkt!~8(~fN4fg*BT`)dpZ1rcb)&D>_#>1uYOU$Tur&T}qWa!N-7*4nO0H9IF z1UkIdKQ`Gc3A6S^xi48t1E|%qN@c+6?FImDjk9cUHmMdnWR;W_jo{gSWBE)Oz!uO9 z6eEI1heP71TT^5aFAP z7ZNG0LB?@$Yi+Em!R6wzY^teJTmv%fVTidFEAD;rYrxyiExy2rOHN49*EC8V@=4+s z8=$JLT)Jm(?N%LrcM4lN1gx8)TEO8*)gAKkAwBybx7EIHJm5LIU&Cj-{<{9WLnOhT z`T#RKY;ZOnMlu%1x%$05U2Z?P_Nw0Bx5aI`lo<-8qAQ^j*AHx|Wn&zgh{&9$v(Rpk z{uCj*e%X^Q4Oh4NP2*abqX|wq@H9*K@LA8yj?eMG92@7wUKjAPzcuR8^n#H1pF#MO zlZ?KWF!ls$J;$EFSNq$_-^#xd3UD-K0akdyi+`(fz83?pkrCn)Xflc`gV4mi|An=+ zOhrL4RYgo;D;B&WJhNT9GfQUskCvitUCn;VJj{Sc<2Az09dXnT+?SQ%k|HEmpZ^(~ zk(b^rf@`zieE7G?mw zU$Y%xDmk%U3+tc_E^i-uE}+R1^qfAHu>f8PztQ~_D}J3QlQD56oLr9D;&}w*7W+w1 z5?T5)4)eCx$ocE!WKb=41DE7wdn z{3BU5j8^O^Z2H;95$T-4-XF&e>-x=o*nN00Zxfd8tdg^FjQzShY00$t$`O~Rz{!@DK8^o>vR8kgk z$`O@05H#Njk$}H6N^{KGCEqWJ5Xn$D50Wb7aEUyT=)AuvVRo|2FP5Quz3KJRNZaQ>1_dp$o ziqD*`ykttCah#mg-2ESkqd|Mb;Ud z@PaKTiyw0i5Gp<9M@tfTZPZAUWPS8a3_(ThUI+M~(>saPif&_l z8_YupVOPel(DRX=HuyO~<+Uu?7Rzu<*`kVMj@8))GL-4efZfYv*F@x!6x<6 z^V=lvEwRE>_{-hufTGlCy)LbG5r|7c~hQ@74+ zyHvZ{&BM+;w>%Q<}9vXP&n8em5{m z%{}tVsv+BPHP*#*t5Wy)YhqniZHHJ<>34?IxZ7lc-A@=^8&~{i4j)$*dlz~F4BwRT z+0?j6^j?AK`On%O=^-i_ehZ1=A=jrl-s}FJ1 z?rr4L>)`cQ8XH%>l`BuWm!lh?xYySXr*4nb&yxssBDNFEeF-m=gAk9!GiCq4zoRlV z?El7T2c5RqzW1Tuo!%#1J#M@_hdH^|{wY&w=vW&#h}G~~q?-YlijnE6_0>Hu4`WSg ze)(T031Tqyb*}-uTjSZxbgo0tyb=py!*k2l6z6@hpT08-_(XyS#S>9^i48AG2Jnif zA>*f8_P%FeS(#-H8bIMJ!(NBd5up*dkcLj{Iv(mdS6*EVAPKqmi zC}GvwJ`r^Ep%n<33eU`d&M}{-vZNqD;7ujNFf{&cp8+tHMz8lAm%FVrI}HF*;#Ta1 z=c~P%vKgS`s2o@wNd^OdPv3pn#Flg(#$RILyKDZXraPB%72T^q=I&Ov z#Jb-G_$IRDwPTe!-)blc55)vYpU~Wo`ofFI5nwJQ{NP8@=2F_UK~R=wc@seEPP?A2YO!LY zme6|3*b@dxvW##@c6<0jZ=!*^LHFl@Yghd{?FVm`-0>srGlG1MAlt;cWv+72c9t99 zsBRMy8@F|rg&K6;dciS%*YBX2mj!L8Y=6p^k zu(AnqpJ?41`dsd<%(H`AF|rFyZ8@>WfkIvxo&bZJhI9HIX7WrLHZt&B zo%!zpvD+I^z_ik*+&-`K)YGIq2W1+`xz~d6coHc$X^1^_gE2qWx67gxGw0uu40|4zYYSF z{Bm!$T|Y95^O7xq8r|ClV2KZIJ%+O)xWG{*Zj*-m_FJq4oamwdfS8jJjZ=Ugv2k0= z2wo}VM(OLQ>(6)r`r7dMZm?W$|JD_15xDQC9W`6`{#7GUSmX{s@03N_BYO;MfLPWr z^RC!q6m5^1cWU3V?xOz!B|^Y79at`g%XnFJTwHcYG?YRE(^Kdy+zYWsxRv|& z$ixIAZJCn8UA*lJKUJdN0ttLYW!1lxJDX)0t2uVHt{#^2(+Rf;@QD*1>PQX=8}(L2 zzDWc2)P)N_>O}0O2!o75_++*grR0Kp67R5YE|HT#r!v7C*CU#1vX}KQ>l|`-coai6 zL6Jns7EBY{4#CpPAxzltXLEX|R~B0pJ(JPoi+1^zSDtfcvx&mH4zAUkxs*h=(n3<3 zZV-S9TyXAuQEx%d=<;-YAhHQy80G-bB5LD`lUj-G@*8N1SZWRET;5ZAEQJNQxaGP) z6{OBhM2*8>Uhq9~S+1rHrbu@6M_bq30Pv2pr-QuB=BlYsO6TU}Yb(BZzy1F_`pr<$ z&&Ccv9QuK^yotvIK!<*?6hs06a;NsncisQ6GwZpN%4(#;t5;}-3+uYg6LPflWkEB8 zn1DDfIiZ&RGnneW7%~|I05ZRmF-wDG@qJ@r>Wp}YmJ)Kh+qkX~iu6%!9qXjGrIKWa zqKL((=|JaWmuq{BSV>Ly=U1S=^8x#iYygncSRK{JSd@}@Kq1@?&QMQQ??L9(+n$4D zHwWB1L~E`{#F;CW4298HRy@~8Zl@qo2y;SU#_YIaJW`O1U#a_

|EtJ>ZH0Q z=(nW?t>1m;{mE9gbVyyePk_$QR((_MYpCZaTGF{jAX}aE9|MqvmFtAVi-|k!wAaJD zP_$`F*P{abh`X8wo=3*|fl<;JAN^NmLTUHJbovIV_{%_pwC8%ft@=MGx6JM|leds_ z8J%zF4gkHhCnYWlNN#b|D;>`KfYGokDr!<)HE4PL-)$^)V(m1(jI-g9II9mu#v;k> z(a%ZbjS$z5X-+nKrv*v(nwo4O==(W^M@UxFlcNR83%gFT<_38j4t~Wfdfjoy#HBpw zHl4rD&?#}tNZNDzmvw`D)5*iA%{%=57a_zQV%H;?rR`XX08-T|IDJ(1I-etK$5Dne zwn0%Y+qbL+GUwGomlmy?qEF84(B~jf`Z3rGC8cv%9aNn>Xwy4i`0qC&bLXs=PO5T{ ze~`cANT8&zuoKx`$7dN9E3{sIr~M#%#ms8<*beWErt)%z3Q-YLR@Tfe9wsvFL??1ez1 z75A9X;2)g*95sW~{Z!&B)1~k*>NQBiYOL$x!6K^M<24(Ykd0wILp0n3UDrT%YnLgg z{fh1Ul8A-i` zruT~MC1|~pCmpwL_CHnUE8toSoJC(FP~MAtX!(K9@n(_hZC!DE()8qyTgf^1|L^|= zpbX(Q)KPO-Oa9)BNE@Nzp_TzbM#xOh6b$ubDa=g!c~@_o(XIKSZ0$U%wfr{;@n@j>7w=5k%tl~g8Ed( zO;-Kv! zlt9ODC+6>FFx@^`!YH^edXgA_1QVnPq=W0Y3XqW)i;UoK(ay<&Gmdq|8BS)GV zl(fXYx75Qe6SdQ%UY%{|B+CAb{6A!pgI5`91L`le2LIkpm58jV{t~od(e&uUE=sS3 z2A?17A1|N2MLFvZ-L-gd{6qd3vG8AMVC%<9Q=uK%oa8UDK@=(Q?y=+dPO_+B|JLT$ zcd2RJR$y}_B#Opna0!>i>8BwMiID+{-$FiZDfVc(fgcA`UO!wyiSX{KGU0;IS*44e zDwH?6KS9k_f3fQHAHD2ARjv%3g(3^CT zCcTB;d+5FQ4uTMhbnnJ<&iCE9Gk5YA1H&XcYpuQ3`#!&CUoO4nlN|O!o7m;J?tkk> zl0UH}OaR;I*!?VFy7G;Af*njeHnzLx)%ktI@)9XJD(l@g^?hLRMJcTeeKUNOsx#e4U^bBr^RS7(zZl!U4W4*>6&2@h z`%>4*d#Xn-_JU&I*H4x;vz}-J&%p*i3*M6v@mZPu5}wa8b1HUFhhQfL8}@O&!!(s0 z0C@oOy?+i|JAMJj1=~Ur>N7(eVy*#>5;sIf z`;TgGw;{g*wrM$GW_vWowHgP4oMeCM6Al;MSLHRExEXb-8`ZUz?s3$gP-vG4%+ zyHNkDnbx+1Q@gIzgp_PUix_8H;(NnMrVtbM(~<$}LF=|?RH}^N9)(=6 zwMvcM!90(;AIA0Olx8wU^_FE%+go_jLnAwGX%&|9LrYpKW&Hl02(!C>Ud`q&)&>(^ z(^|%rKIp`uub*l$>>)+;C~_)Y10^Q&6p$uYGYTx%kD)|8Q=l%*jYDITXroxLT_)~i zaLXQL5`4b&{5z$?L8JjH@0G}>J03hI6?X0jP*PLZSdt>Sz~M**rFJcWrv@UHgnC`y zqkML^aF6aY+wAltgq)_+;9E8G8aR<=yX1PhKT<8C_@q6g54R3cRX7hjK?VCV*`hQ` zoNoIJRIiAXyVIr@(COi9S>D_Cw%?9*cuJ34%o~<;u};2ErJg% znLXb_0-u5lDXoX^>4+jJk3Qe_B+}IJ+m!ki<8)hyZxdR7^pSWQCU+++WtyG*fyBP> zrjU*(n69}Ge??N5`HK`Y^uA|mA`i>vGd{==qDI)fS)@Wh!DJy`{w&8tKBX=fv_IWj z%2ZMBVHl9;o$Bz*Vnib`rjddM1`+~Nftz2zj;q2@*qwtXj^By!%&^?JV`j+`L zmPXxKf%5=D%SyNMQHNf_@O(L%tH+|Yfp0(CAOMqn-y)VdSyRg@DRb;ZU{;9x?XU1@4S|#V*%5tpGg%u{b<>X={H`KhK-CQ zNbrC*XKPukWnyF;^DsDTkUYkU()G^p#}rS_ZSw{`L(gs801)C}j)e$u{vXw#{4L`dc*}gb#a)nyCTyov-blc1ViizV!!6k%~lO(HB?d%XLUZ(zX z7^B*JxK}LV#*9G?*bf9~`x3v9LKD`a5tcha$Cxtf5l_c^**d&RYhBv^X+vEN5EIJD z9T6&XC0XO(F(aF_P*@!bw8cObmFZXeR80_H0N5+B(no z`gN+4k`A}%vs9Cfdq+I9tQXMy%8rmTZR+1q3!OYW96+Du*=QB)gP#eIvh3MBZ1~`ZmZ17DC8;)JKE>i8*=d2L@OaSjz=|M!}eY_XU~Ti*i`1S z;9E%xrhcEc27Z^=wX$axnM6GWEGM~l)t;+qZ;p&P%)~WLPxF#^S{{Op3Z7CD`itwk z8E@C6SXyaQP+#1Ry3^0cG27NW&K%u<4PCf7A&PJnXVD1bv(Y_VU%Nx6w%)AA{DlnY zY?t2MoT+=++{1NZa&e($t)99k@@VkLQvTYRVZmT_`sqr;`S^2smJs~o62%4fls^sc6Xys$m7NS$;noqw;5fn@UOc*h#WH`juN zzzum7bhNZX`Tvf%zz3ie#Pe>z@2DKO!8?k`a^$;qG7CiP^WR`0$6TXaP54D{O|zAb zP^Z_;u$Y2zn_?jpwRSPU7%P$~6tRw0)?_ zSmuvKrfS~Y*_{#8>DjQgaO|Wz!a;wcTsVy*MZfX1pr0~K+v2|i*G{o3E^2Jb1~2@= z&AzI9KcI%b`QJ_&i-f84OSdyCP&h?(qkQhd(C!{0;_qi)o8q0HBcapU({!qX#7yb*>IpA%eY5w6~h%LNpf(R z_?m2w8)JVJMJ|l-|G$pi<8fvv4@t7)9fy6pjM)_0W|~`+DrN7|rN-^jFnEW=@ohF> z^$EfKNFR$xs=m0hKt34C<9Hae3Jjo|7y2&2z9G53|HhdhZS~|{JF*=)(M3U3NomsF z?uG$@)Nog5lnp=Z_-|T{zub?H&gzv=vFnnfx9sCcyqxJ&iZ{P$k(_6hCo~39mjO|A z#rE`>QNfYl_a^OG0nHhJW{7;2y2f|U(dR(`Qjv#$@i-k>4WwE$8riJM7(7+Q^j}l> z$SHjD)_mKV@mWqvxBm}uR$DqD7>%!i`K{fr&TNxv&v%+g1n+v;Bqc%B^=AkeKQ}Lp z-(&`}E6{ml2+~``9u+YN4`Rkf7qb1W@y3CKbW0SeDe+-xk5`6Aw*Yf^Zpaep;F0 z<0QoPApsGPDL>D78@!4{h-5~C$1!Ii%}?W+jD-3CT~_xE1%xXaCFkR>tNgz@t#bmSL9PHM+r7_UACar1L%abFUQA6!!%)%YcUl6XB^S`vgQ@-Y3SA6 zSTusL0Y};oP&Hsf_>LMmZ$xm!`~1??pa>o@{x!|)@*-;|?!z^e6a$fEq|F=4^QR^W z|Hh73tAwa+iF+FB0fk&h&{iNL9j}V5&RZ$aVSDuP{}C#by|$vs3`P0zN?i>i!287}`%`Y@kqxck4@0mJ6m-zzOYKde zQ1D*}3HbJ-$P6AYQPR$9If(|`&-wZf$FT8T?<%-*nAEyib_O|Ln&Qw1qkUY}2=V)O z)0*FKmEFvAd5yHW$bz7e(P+#6jH8;H@KVCclG}=PCAY3Q$JEWDOT}p$5J#(oA<+5x zJ&Tdax~8q&kqFWNOFw8j`r32s`hrs_9ZuJlkewS`cQ6`Ow^zK?w;5BU0QsX=`T;jd=uR;oZ z87W*%ivzA7_E$36C<$mYq{Bkn4QTjjWi!ConER;H!Fc((^^k(lsqH|y0|3~(zZ?k3 zKbO4kdH4iw5U(DsI`UC?{PWfR8bio4U^ zsxzq+t6*$6&Ukf#*9R&OB78;RZIHjzH3jNCdIpFP*wi4|N7JokZ zAGu-^+7PD^Z$P6DC@rdVF5hrS-kanJa(Hs)6|eQItZ((i_r3i=*ts_yPYe6g9YV3J zVkc$w9&()?2*0*4+xaV_a(`e#|g@jTrYKO{hv z!W`Du>ES9?8Hc`dJ>#oG#x(u>J%Zb}Z}c$iv4RE}--sBke_D|p{=PezJs=2BfFDxT zY_RvUI?qEvo8dsy9!NQE=0JPI|Kh=cz4`7zQh6LLmd#A!KWfUnf!l<83N(WhkePVY z8M@75gzy@~(Z$r}vGO1D4#<&FE>~UWM;j3^h}OC}c(a42wz_QE z5m0=%n263y*=wplevT?VVBL8{%{$KW6gmYg1iaYU?NEC|Kwh+l`5h&E7oxcxR%(Hb z7Pt#J<4po)<~ulS)-jrNWtDn0irHF=oqN4VZT1tb>=mFv6zsg&(tgU{P}1#<`v>Qu zpJ!M6cXge_<6OM+Vh-=y?6*nI;zF*$>+cr2sq^DiH;b&XuRI427}Nl9vlxrJ|BZ3N z?fnfx{n~x%vu9H0c<{|%IGN+nqfeTRzfr=opbc3MJ5NYZvkIcS(UlM9)CBY?m0|6d z%EoQIxEuWLmKSzoaWRgk7KQPbOSJfB?&s@Dft>a@EpgtM@W~e9K-Vz4U9I#)u z73?`5Ez4I=I#ae=zqpJ%8-5JY$_H*ibZcU($4iD@`t{>x4X-f%LW(nOB7upqu}&ep z`ZZLx)2?!mL&ka6DPq8cd8WL(DD2Ophy=qDc#_Gj{ao(cGah`_#81oNsPqUwbWIiG z(Q~rY6op`T-f|qcshXrE* z$e}X_bYbVe9;A4@iJoF93LvH4;j;T-;qb0)O2KaRUBX8RY!e{KOu?Z$3@$YnX<-CZ zv)~;2clH4}W7#)LAj9-wczvsM$02RE$7q7+2f!iqM4 z)c2vQ7PSdm>yOR09749z_@vW7ZYZCV-)_|16s1Iq==;gRq@tt7MoU$EsY*PD9q}lG zi|MqKF51#ikePmBgp@AesuZ~sTQDRJXL1HalA``nkaKG1B@aK^QU-e~#rzC1$jK<0 zaOpd(m^UD5Y@s%H;N*YWDbVzhVEUn^!C=WiW67nmWf_B)KmzLRR}Zdu1vRR+@}~ze zK&-+QCtT5|lRUCIjp8k5QHv7%Rk$BP|HO(c&E@~L%S;3>=}29reGg(6tM0?TQ&Weq zt-K(NEs2)af`42e3bx^-uC`z{CSWZ1cJ&_-JpzGzx@C~)>D zuMXa~)TPe6;aAf!%~!E7sGr@{I)0=%Lo*ZNb;oHmpyH6lSPv~phvR9=@9Ep{9~25~ z3DFL?q+aFf5G?QX8UeIvYBmS|naHX}I4rk68K#9rX#!yYuEgfIr2xy~1fVUt{f5IF zB|la^*ME$K#Ycn86J>Xn$^`+1Bg!SlgXt7F4ZEGy9awPaR; zNyo4$cuC79Z5TXIn8Fmx$@TbkRKR5#w2}SXT2!RBglb^!dN%Qvs>ORgJ9ZEtzkqNYZZ} z2ZhuBh@3*F0t+Xh>Tda(Y_VOle-Pq7wCs;BiN`aWRI$@$l76f7h8u=x4j_?s_x`o@ zSHoe{K`P{+ZM4^EdC)`kBSTRO52JTpWZ_;6apac_@>R zNpqN4VSghMKFr*gh2K<6RLAWXvoN13fr~EiHo1on*pZ7N_Sb?UvN;bk4_hj{odaVn z|2Lw{acJ+4w)#Su9&D^9ZGcVm#?ArlMl4HYU>V!Kt(w=3C$Mqri+*d+?~K^=!hg<~ zAqf544UV`e4iF4H`7$?JfZgjG+xDYNCW<`kJ1GaPxzKl^8!7j|E3I!&Z^S6+ViK22 zB(+eKF!PEjnDW(+!mQwj~8!9(`UCuT>AJG(ECkAC1QoC zu|nsHy_^ixHb4@l!o-PZm;jff^+vKwKgsFJD?PxEvP%rmTzdsJo)#S{5}jNC9<>M9 zG-RAt!~HG+G3MIb4M(*HIHio|8$5@jQli)97x9EP0UuANqyG7DTn+H6bxMRnnCCBO zSwHZ(FKTvlf4j|D0qiPL0)V<4>QezsuqIplICt&_*8D0&HzpyJbp_ZAi|_W!AdvXy zZI1dL^Puge=fCm>Ja*sJ{5y#m`yN~3bO@`XG?H5mnu0EFsI}xUK)q7iinF9ydc!d` zdP-9k(||k+=Oe}+y^>J2Q|_5~*|pRsm-I|+1#sDHINjJ}u;vMQYANKM<}=UfgOUH# zS~vc_y{q5e-qsu(o|RSYT=tj1SFGp);QdC`fSGclbmYD8w&B?BBs6Sy9x%nb+}Ok% zDAm?9Dr$j_jKq;#4|VO zkNV$Ye(n03(II*Y@S5Z`B%Yg6nh+Yp`5dJ4@cWRpcOJ+%Y!|--U}Sv6fk$=r5p_xB z%&tgzM09(@y@FQdSc}lQ*G8D%au6_*BL8=!dBC*@v{&xVMBZrOL{kJ>DVv2EW=}hW z`Uesp{ZV0RyAPtpqiMQfLg$40?NGhJ@RxFBsutnuSwjdRG0plGc+F6#_2@w^30h=m zq+5T3NQ+=`(jEreP5$j+;xbV?FJRL%ziSOZcxiHx<&-Q-H?X1eTe~NJ0ag6k#;^GU zl#63r{V!OpuFG77`=%8Aw@S+?bI-|Ex^dmr=j2;}B zti1c#z^hMmPfVV&7m;ivRzN|i(tvPxq*UqLPqM)*2P{q}e<8eqHLseq0og3}_$0Z8 zE7lMKB>_h^jkS+xK+#!(Vo0)2D<82RR?aHzjq4|vnAZyZzpIZCP7*N zm6;ZF3VH*guyGqnw4c4-%Dnly+2m-BgpH!iX0uzo_V1Z9JA->uzYDl}cG15h-H=oKq5LvGCnFH=0hwL;8j40aU z;%_GbLUjx_jmv{KDDxJ2!Rx~MM!Ozg1bq4_W09|x5{F=V?VCBRJ=g}~N6M!Lh@31H428O#cg4S6BEg)DTmgyNTpXQlKwA7Ok3Wju<{{(~&SV!a# zDG*8EB8HFl!I|~F*hs{8r6hg<){UOSDOquCMn(7|1i1e|pwKRKZy12tD9D_XpNQ8eX@En3E zoX@4d2~gR}%=F6&ge6Jw4PfkVyD)d7PNIz`_n<)ZYbUOHgJV(19(DDfGXK^Y_z2k8k0&o0^Lw}m;hY5t8 zg%#+{7q2lPZ-@=JMNG`#jSg*2dmbA*f5s`qenmK?g#6T*XKcd@*bdqmTpp0QJ2W&c z9|3M;oH@;}KX|Zed;`Mv*UQ@mocImv?r)I?3^ewGp$!9YM(zPDQ*x!ZpJyFePtD-F zzj(dj-q0z)Tk5BeH%{@XBU!rd8P&bum3D(Kk=TsqZ4tgho424Y%t^MtDVmi9@wR;y z{%>`lLyWl#mjwLM+kyyFrj@_^kiyT-5HKE^Aiv$w1j*+Mwih&(K8t!z>MD!}q2b|u z=4W@orU;kmILxPVqH!FQoLd2+2KoFm4Io?4Ezo*a!}A6VL?s45YF)O4w!Ioo|H2Qjn@q%`L5 zHJhD;y^P5_?#Qq3727UT{JdnSFS+(Uk*P$(JIB>U280JI9Q$1dUha2D?in4($W113 z2kv=&@0fugfn#k!Nn)dD;6_nhm1Qc6mON)LYK`<=X)gu6S+f9-DAgGe${}KJYD&$t zA-xLc0V56B8a75hPmpybCSY3L9os@(Y{vXBZV63@;2M=U9f2t*WfJughqk$ z*iy16l+|uX9QNEcv*5hnhV6InhlyVVVSq*q%f2^!-^<7n*M|(e(RcOJEP|am0 z`s5&cRXf)0-kQ#rk#?O=3a9xkCBx*n3U{JXaW0nHmOKCgRlS!ZFu&!%@b2-$_(YHE zkdwzUw0!(mIUL6i$(W~&CMeqM6l_-~MPerq8P+tqw*Id2TCs=xVZtC@Dt0Qg>dV+HA@zSE})@$=%AC%#&lYg=t0t3>e8RY zu_nS}2oJxyo8ZLc#*B8JmOV!<%R|gNJ-QOVpg9ctfRax2_QMgATj~^xTz(A-G}s^e z(3H#IE_Q-TLrmaT!-@DpwTJNr?Gr~RxPlM%OZ>ZcTqL^r4Iw{sZfV;z_jfnciF=vl zed~=`&gK7f5qqWDE3*YEa-&=nHl;@qcCG;e#5yM@qjNAG$1_Xa(&vywr>2W-bv^y- zPvSx<`9GH4+7ai(1>L1h1H^)ALZ_Q)D({7f50~;X?ADY>5U}QtWMc7ytJ?SqNfc~R z+^{CC*o|EcGiZCbDyV_lL~PM^IXpL|`t$AvA=x5L$0A%u{s~ZxE2BudvI?GEYXPb{ z(1Dp?5KADcm+zHLv>o|I`#|K8o*BnypuLq~f?!E)dPX!5B?mS{YU4*gwA-)ZFuc7^ z942`Uy!9S3m$8oxrQx3;<#b0&dD|Y~XrbC@S<2NzEeYH?@rz$k_(J;_l+{Pb&T}gr zuwxG4*(SnTPQ%0A=eIC#6?>~&Q&W_={_tpXHh#S8q&;s~bAt4iAN{4l{Cw5E|Nl+xVp0Y~iqs z@p(RBFc)DmFQYs|oJp(WU7jk4og&tI=-Px!7p}fR;XZr)v%>GBY^Aw^bgkp$T?5~b z#qu>^FMO~EkvRY8Wn5iuS1z`42ZykRdKE%<5RWG%ye*^vU)j&?xqeO0q|xXB!yu?2 zc>UB9?0F*U4cHiRwcvbw+Iw9evjiiIo7U-Fv9emrgBbzztr10*C72+f>7bu)29xfK zQ)rHX$!n`=D#3xrsywh}SJQlCLu0PQGW_o;Dk)Muz{v;YC~VtGePuqpnRB#VjK)ZYU#H8?0MRLvt1WC>0zM7@0*g$ z*^J(zp|?taeKZNmB-HgH40IKT-yKvY_D1;F@9}6*Ikt)PLzJS=w9t3L;*=Ry+`6?; zMu;v~E6WR4vS2O(x(ZhyqV|BZryQ)Y%|Mh%XT(lUhcEe!G;bgO$`W6Jz-no>l&Pyi z;AoZ_R!t0G+*SJvnX+a@BDQn;%D5}K(NLXW%IfhdynCDacyJvdyWY}*M`1eCq2R-% zb}8^i{I05A{L8mAcDJ#USi^VyPerk#J$weVvVoOAYe0*0kfvHV`9A(#7D{oxZU}qh zL8#5D$(d_TTqE*xv7zCefiiBesce!Y0_QGEyClijhZPMa6n;|k)w)n)H^~-o-BaAc zq>K6wYa|9R#bap-l+j)kN}3fuVJdIMqMp)->KRGSMA83)|VH5!G|54 z{+f|GPk34Gyj7x9lDqfa=erLf#p{R{l3zB%{USd;yo>i#i7zK7kU!%E6D!^w9LksJ ztgOLB7y1j|E=2RUw)W}=>NjTH_mT?-On(2=sxU85%^TkSxoy7Tlb;{y8{*Nj{f3GH z;sWR$ekBJzMD;pmcserQU+#!tB)t3$5mx|Gt; z+~ptU24OrpSRM^Iw>!vneq&{%P5gPOZC`*RK7WDUR$L1sbvmuu*T9kO&mEF=1BjhNIIK!d= z3)0T-A9cWpuy|prh-}yz>%3+h)!}3N@y^4eVf6T9uKT1Lt$7EFZHnKVB~%-S4>KUU ziNwv{^~?S9WK$k!x8~s~)Z%Hi(r9Hb5*0L)>~6t$=v);oy@L*ynm2C$j#xQpTC5-* zj=D`iCs8bTK(xDM{iw(ZlQQ4`lF*7EA0br>QCQJ;9Af^&e=*41*z;jTua$nP%@;18*g**T5?hVgn&MlzUrv@l zrQ9QaY?yI;k`kR2Ndl4yQnlIU;bhg*T-a8zJ-aZ}1osrpwA$Jr_u4PMxn;6Sis`oD z9Ra5iG`HK-Oq*!A(5%}uVEz`;*)><+u}53n3INbWG0hm!b4zt1N1CHCxb|HN?&{S@ z)Xcp7ju~cnkDKb71w93g~e~_MEE+TF?XUT}Izxl#B82l-sZN`jjk)ZwlkN;X zp3Pi>^73S~jGM6AYPQAhM$JUMwGH)ja7|Q}H!DeXMS*ePaj8Ydbg?rkTjZ0C-ELod z{YhGK&jMkYl-99~2@oT7vdKeyXi zxe2%D8xY7~T6vokY}QPWFQ!C9f4PW{54AF|U5+GKFjjK0HE3SK56~0!DtSu6^~z$N zPWYPK4U@*%OaQJ5%I*#7B+OcCHVEHzp58YR+rj<;Yi=4|stkD&T>?E}PF-#tn z6^V-nagK~)RkdhpXmo=*45i0zDN*B_2*5`*w@;~Jq&0jwbgDWsTYDBCT?Ug=fS)QK z;up0%i7rw836n&BFvf?OHd`s9yErhE?M_LZ5=k4&HxssLA{z^sho{+*! zM?~}DNWQZcsA;?^AB$r+IAQlH8gMqK$!`4Z`I~y0<3g<0RFHyikOI$?*FY|o20n8) zNlG;1Re71O7)h$i?|p$2X9qW-S6y#U(B}f>W2x41-%RD#x4>pRgc@tg4QPA*a@D?c zF$Nb+$|OShH7gGBxo~K&89nf*C;d+oHZ!;PUQygbMmd*x#)UrW{_s(M2v#fgllsHh z;wN$|r=N$CGq>7o7CdOCgB8LmFY-i4@EEdD)74eYcjT^b%)y?;)-KjWRux4nFO_D3 zHwM*2-D^#Ly;K{TaM|=x`D6O>JEbx*5v@FKMbd> zuA`|Y&%XPb)H*fxe2qir444Y7s^|q!_(eQoQRpBC>c?XF3KfW(x}I*TYej8`hy))i z0udI%U1M4ls&CtTS4z8n_l)uvmj7~aoLJbN_HH&fsRo*Z{+)Xw70K@oOBl{)wbJrF3{I7|RPq^PVWU z)TngIQ$MhFwGgV^G}f-o#SVU+b~r_zSz8EoSIP?w`_)?1@;=?m8y8;hDPzzzV-ow8wf*A^b9R{BUcCA=j=f=7c#_HE%8Jb| zGI7Rk_tPkOuj=IT7E@3Po9U7R#2;!Gx@9Q~pBHm|{e8BPx|3)ZH^x5w9 zPr+v=yGq)X==rMgw2L#oO1P-$o;>eNLPQ)*|K4Pl6`t}RTJDn;Uy@IvCL@Ho4YbA= zq4^PW!eGSi+{$W}txeR_iKU-eQ>Mxf48DqQZ9-D(}IpXrGUxSRRjtFGCe{k!a zVbq>^F)|Oj&8C!$#f`s3~y6U!QNrJN8sK@iu+!cSh*#NR&>1Q_v6WvTVpd3m;G+b zdG&T%EMoR=ugiUr?v(Ufi2Q@*C^FY0))ab??HNTk?5EkLRtGsnH-pU=G!o_PbXrRl z^mV+}%WO8w2idNz^6*aAW#8tR%--7N^a=toPz1yEC*j2<+Io~7DPkqg^>R=yiGFsF zP~mcMT#!`5HnK8S0OWgD$o43bOvCm~m;|>&yEN$ROjSa91D|D09_(dHx`AK0UgX;R zwNh5od6(m2h-kLU1$Fqs4v)ys0I#hn+XFAimp+{dUs3X@N4LOEEw!*}3a@xZbJHB> zklk=Z&ocUXor;_{baEGE>C>)3vYekxJwF;c5&J+)BeB*o zKth~q_g1jxlU|%3cFiQM=1~gakrZ73Nrq3vW|SoIbE#{>ZIT5iqtyrwCtqq?vfUUUr!4 zHcv3)LS1_lV>h3l525Ps_VOCiK8cgCC*V-o#-a0jn$+w1vT4buIttl;MC3$>$Fi2? zP`4#^g_lA9Cw7Yy{BiW_ft+Fee zx+_v9vZ(pWp-6hgy0PO(KfgcdGnV5@A`%{`%&N|^JYqF**rudu6U=h6Nnq)S7NtdS|~-kWlns^`FM7QQ)n`nq4ZGS#^(&)V5asKe>X zMO*AQL}TQAD@Qdz4L^ts(0+_t!$aR#_0|9gHGXRTcO!bM1!YcSmV*j<*KF z=H<2*j|`bU)u;;Rg*yhuVcAGLD|fpazQCCJdT@s+l2?`El8VpA1lRt^)-%g7JFzs9 zyL@*Sn$v#t4p(8b`H)Y_sB$`};r!*T;ipqWCH4V$DUM{6rZ_H1SOvmLl|Th0l24|hed zk{;}rh2zu>FXfjGTT8$|zlfM~KZO~-@=eTXl0$hhlWUu&e6kGDqU9A%_%IvV@sM{Z zJ`iZLLH_JYlGZm*km#NR8D{3HOSIVxJjZJ{C}{jng2&Mv;G8g2RH- zO)Z@Qf-d5=4*2?%cP15iccxM;i-Ov!s8O-Y)IDl=_*z%bIsEzrv-xuI$$#>VgZtQuR6pfK6Se^0R*?-ox(u02pAe0(YaT)w%;Jwbj)e^y7gq`_T z`M`o8u9qgc{Lrf2idWC0rYpGEE6=MRc&rbKT>LrVM~>j54|uM_!ip0{6B0ARmDjn9 z+Rf9i#CI>J_1q`MLD=aAb-a$q0u`HBv_r72$@YK~AJST%$6$0z$T6$Wp0K!Uxi-Z$ zSO0y&j%T?y=o}9>E-H0T7%Mx1w{O1j$^8oSWl`STDS|I6^#}=BRisXHxU%lwHQ}6x zSuQxmNY)Qtp?O82F>r%K(aq9uMbTM7lV2DY^*N&?1$_;6q4^m{*1v^(bgxqQHW_8D zUP3cO1Sd4>_7_u1N@MkFaTc-?uga;f=H7G(x(wD##iW#sYE%@Y{K-h{jlo=c&n$X1 z$Si&bJ#D@_&&Wh<{YL(-;?iBbFcmvrr>29(Cf2rrU{wPsIQcQH;+h&cmZ;8(==-p6 zuPyVPP)v2!Co0cxrbOQfAEf#(2FaM=iGd z7`_Gd_;auHV8|FD;vF)`&93*}YD!&3u#hFgcWq~UuPC$$3CiVTksKTz{XeI;jf-N-`M7b?roK}N!fj`&iK4U}UE_MkWiO=tIxrFjUk({I@ zRi&E8G6^$1l)BlgQDIBjT$-8B|2oWU7*d!Rl~I|r4~=uv&XB=;&I&X5wY`RuuD+9> zQIf2_(}pW^yA3B$Xc~Q7Fj9}BJR&sSMmA)c;hyq`$0#Oi-nGK_znJRSAI$oC0Mm$w z!eOz5MG{vKyeemeeU26pVot@s?O1TFDF*c)Vg$pJ)d`AD)n2ca$lI(16mBMr{26@V z?w(Oj4cUby6sl-fZa+*$-p9TyFPD6u#9;`wec6HJM@o|1QET7qXho%FG&|d~>=pwjWl^ za#aIk&(_bCtO#;qg6 z8h=o4-Rv%Ec~e^i{xaFD=BOOVjGE`MekJs@qW`6QVk1ETE{0v^tq+h1OqWP4;n&ak4U>&26P&PPrLVjt_}8Q z&Bp(xNvU%%gHkQ2ED0nXojRB;3B`%ylgyP!d?kt1KSv6c*LzY;Aos7;3SIf-a2t>k z?=JZ6aXl7?o}y1_cG<38`PrJ1kLd-hK3*8Ck_&h|6|1Mj;@7vhYr|EW6~K=wM^wnuBqw30h&ahkskxWIC*1)f&vSeaGzYjx%Q;pGW+RCqr3J z<4SuUs@oep6?o~OcfE zt&{C+V6Q=DSl60f7078|D*+7=_t|r8qkBZ4CHx7M7#3Fh4DxRVP~p{gouvMCSX%)U z52SG4m;8KDCo>tmsInGGceb`N$mq^xgSnj@8$^SJq(qB4E}}*vB}xx6c1EGtcf@?@ zjC61dn#R?qbUUxTJo201wZp+5Q`YNpfVP1!6&o&1>3dXDUg16>>J@yDoJADU(3Y8* zWT3Me6F_FtPM(C^_vcvP4gJ}=Nho$k$^vl&ruu`6<`KLh>`H#yvX#KVgSBd_ivI1a zY*&tEHRZL?S+Nn#b4aR}&ShD7jzx(YknApT-jIoF4BxK4M4}_@4`4kvWp{ zw_kz>Xh{xu;*@JQqnMciv?fNcT~@=3`{+5Y2HpXtNTGBwMnS%36{QekM$<{2YFFJf z6qZ-cXt5eXxOksMKQzoN(BUaD{Xz-&sR z3D~|HZZhvmUY_nZdL;47qAnmaY^9rDA>O*K9%>Z6=S(J0@ljz7hU;}T(WZyTx&l6?bM^2a5XukN# z>{$Sgx4E+1c@QS*^85|+%u#55v@}rL{0z2mC8zx2UFx|1(>_A@6^93T0`X&f8}2ou z!51m#vUMypC5>GJMV}4FUF8)YMHc>QF;n!#w?U2PwvFgL?-Xok&hr}Zxa2t-v1!94 zmgwD6+@QU`l;0#SI`)FqQl&>W@- zksFc~ckNeyhhCMgsk-dfX|@YXsEn!@fuKcaJs7@3y)Tp@AhJD+;-;CFR)qhs+kI)} zy)rVBWcIf^%nJ)I8=83jqgnU6F1$q+ZYf-&G^w3RQ+Fj&6DaFVRkm0e=x!>U2u z*}GJMt54nK9P~~P=ic>~q}qortyE#x`L^w*KL&>FK&7*;qTxXb!-KN!M87_r)?&eg zbH9GJ>K^;h;Y(Z>!WEy+7uxUQ+f0and-+mUDke+#AN^XEyHyK9e7nj@^2MYDt1(Md znGp(*RH-AYNfV6N3j{pusXkookI~09zXwHS+A?R1I_+oLEn>$na~IfLH^<7{;*@Jf z2dOrsr=QRV$epSsFvelO>f1Wd$4c{dM3q4{n`lSw|JB;_m!ekaLy*rG;vTR!iVo;V zW`wc2V0$`UqxRSnWEjgk`RX5NaE;pCI z@cZrcsOiV&P<(aj!NRO0NBeLBJ-Rztrrji|iQg>TS~hLe>o#H0^$-<3*o#z$C=5SK zh8!IdU2Rn(eQgDcRUq>WCyrtA6n~X<={G2!SvzXoWeS!S)%>}Elr=TEe75t6dO9ny ztCB$Ph=O+5%pjfrByL*1U3XZ%UE5s9C|#<2^B%Y6#`F?5tiv7YqLBRXg|f$2RbPwv zmm-HrubV>$9dI9}4DY`UAwh>)Zx&r|4ZTL*a%rue7@*Z{__`%75O-BmH7;hkX9O7a zr=PYWhL*he^rh}4pL{&-yLq~rbx!R@Z`>1*OI9_0|7(wV_{MQm%Oq~V^6YJ_={;SsY7md>W-mw3`J z#iuqyld?TG(iY}mxaMxShu+4pcA58yi=u90r>B*p5A7Dske`Nb<7~~|rHn<^U@vX* zuV+NgWc1e@MhjUrKkI=ymT4NSp@KbF;Os>mKoQp(9SrL!t2}GwzKK4MtqJ_KMkP9N3SGi`LtO(lT zqcwDJ^8D6Y6erf>;yiVki=7WNXv4)Wg|-4|EKD5_ua|5$qLV}#jW)ZZWzem$v^!0q>m zb8w+RqnC1srH#`ccve(@hj-#_Pa8senC@on$~bW~t+Fpo9No<#_Epg}bd#x^W0c~^ zr9(9dlX$GnH#v9g^5)%UU2JsH&^x`DpC4A{%P%Fez6Aqo*>Ku^n~Wg^J@=}g<}yPz z3GfU7$vGDiEbR zH+GM(+eI@|Hkc-RGBvbq{4f5gHs9y36V=9iA$=TwRc>6`UXmhH^v8U2+HPhJ8S0gF zMnP&+%YN4%s!9=vUP636+WWY^#GHy(x?UYov^{}g8fJQ|?nz3_cxjNe_P-;lr`xgu>yDTPRqiZ`&e0ytwhdx~5< z%!JkHRN#k{|4n+eqhmGfH3}Ul?OE$_V5s_OqzGCC;>G7+qO!{gz4+z^Zl0i5hbyD= zpJK_2`XFeK`aU>oiJ3_griluIDmrFZGSA;>T9#$q;&SPNa)X8a`B?&w+ms{;N1An` z32gq+l;vPfS--ACHi@JxFYN6wWimzNPcbM^FY9_ zALcmc8WzTAU>-WKGx*l@Rmc|aJAR#}wJ@LZCT{)CKvcf^W=H)td*(xDgb3WO%4M44 zd~`a}OyZ_*L*$b)vg>lSK-#6Z(e6m<&fn=5`@VCRnWpKNUZ=iM+Ea{rR)tl< zc8us6#K`WH=tAJimwt~Y2H@%;uBk-}7GBCZA$%_V3|6kvVn!l+-V`ESI-)x_GV%;9 z?bn{q91e~%R{5vUsM-RnDo4jHTChg{r?|EMDAv<}M)7N_NBr#pL;NtU!x0`qm3cH8 z((SF&FX)^@1dqRwn&BJpSj|J!G|B1BaASyZ zwt5#1WosVj3wV2v&uYl~cNK}3-3xc0RxWtJODYWE>eu{A*;t!X1lCccc{<$lNVUkE zNb`2(x*ySG^dkn_YxPn-Md;UcWS`D{a25(7b0Yf9s2T*)nr`&?r%<-cVWPDpP(+`M z-FQ*Ds=rd%pGoW%S$jSvS&74$BIxNWh0uo$xU1dz)aAnCy#Bkp@`IbowJ%=4l+PC4|-S#~ox6uLaB@|eB=a>Ov0=W13k|<~G z$ELHcB@Xc}E&*);&C{hu|z0{*5XU-Eb%?rriA2Aw`T+X?-^8o{Ph{CJUE zwLdW&2e57fqv5E7vS&X0kCx2lx+R zTeGE1K_^1fKS`hHJ7VS;JBWjynw))#+avGK^~z`SY+Bo?^KHtH28ieSdV4AO`rOx@ zD+$yhvuvjLO;<0{LaU*2gMmLIV&R?X{Ga?d#=sP#~TL#nFF#EhsL z=n9$!Jmj++u6g$pE8~3aBor7N2gg%G>^QtTb}vKJBt9{GT&^2UDj)ay<6sdjm|Kbx z$^dj@fO5dcKq6z-VF>rZM0PKA=r3Q8N%Sw~Sqw$GM;(e;idrHubAqb*>HC@V`Frla z%bEV|Qvz+t6MXf@^d33JOvk!sl{E#7^qY{=&@T1$?fG}^p0%>`P3$=rU?i! z0-6uT38gP;lA(Z7vdVXGa3ncNWo?7O+x^4G>x-scWA}cz@Wj=+lzo)W#u?<%y>`!o zM8EWQE|0ZyQ}IX0?1}w*g78QzstLZzz1_q{RD%#no>(n_uuD6F@=?ZY8Rd1O1VQ_)vyn+ba8u)#FHE>Ij?*mtp9o@`F&c-qw zm+5k1?iEYhuY_xp;)Kgs@8(=f1o0~n6KsAw+ zlce-SbR<#Ym~1WF0bXb3#p^d95|;SNf5hRE75~ebH=stK1UZ^IJn2X{#joLlPwe?X zObD53rq`LwW=)dIZ5?E)WhWT?p~yXbf|!wdb?xxWA+m_(JGv9kOjd6m-tum}VRuub zW$$xRvJD!QNUv^4qkveXh-3S@%5xmF8Q2>fK{?Qx@4Cpsw#ox?=#xbUg8@?e4K3}! z@>al-e;`j)qvY28YvSRaw;FDM&fhjZkav=u9eLB}U=0%gIuaY=v7+W@;c~bJL(#~( zhtG@8+1W~HDKtt>=MbL2ADS%!z#jfcr6GQBt7z_z&VM6Fq| zJZN8Qz0L@utZC^CRvdAK?Y>@mCtTRO5ZHau`k4KxLTYx{XH;7#rNd@;6t$A^nITT zbnxgXdXJI~BCO3~nplLI5361*EG50fEJ2*)D|YXO>ARDo136_0B^AYxq#6i*i7N5I zkE|c(nJzTLEeF3ClQkCrcZWnn^oKe>r^J?fj|H0#-mZlKCvZI7f+pL$aa?Florw=X7!Qt6RT#SE{W9pg!=@!Ci1f&z*OLcjDvJo~ zpn4iZ6a_S4xV(aKxl7XbqYXssNhhPvm>w!|WJnHkm3ohV3rG^C5h4EBC@J2#jg4%6(wchR`i2rl6x4FK%RjH$OfJHeHD z<{qd?Xrc93E9|Cmhh-tfcoDZB=$2pgS!OEd`vN0?a0;LpEbdP)ea_0NTEy;2>^%O; zrk2GO)SCC!nU@-RR=GtN%h>NV zrgd#gJ3-+UTA!pRLo5rDwURJO2vmpX3`EO1P2-DM$rD_tV(dQ_v)|!-O@{7f6X7a) zt2iE!tVT=@BLvAeVCW8p00a9uv@SKS$c+J2+8)ud*2Ak|fZMV>aVnns#+){S&5(Ve z0}N%WOQlh!yz8WgJje}czVHnoP9JMbcQ69sBg$oDdXuZ9S{qCq#pvy^)WA^p&-s}) z7NL&P{wvwY|HOJF6r&z($Yf+5PfKCALYI8h8;ch=97g4#=kG{&2ZB3AgtX4F^w6J! z$8A(Ja%rx@Vd}eWWoC+)Q;ha|0R81$xHF#s?;V0Zo_qOqgNi#=$G~JlLsAm{X!miqTM#`HAvpQ8O$9z^+)g!s3DJqkj`aZ7FIE_Pd>8u}Hl$0FlD}vJyYUs- zLF7<5#WQu28J}s9nbG8z;lW`dwA^g|s@>llE$;I@!K3Q}yV&<`4CHr1ucILOd5k(8 z*xP^jQG54w_Ch>P^T{RP32e~QMaK@rI`;N^C90wP?(BAzto#_~E;8ouM9WBp0Zl>> z^ZOZmLhT}_!I(!uE!BLvaF}+NhUCwfr6mYm#bO~XU+H3)`e{e2{!ourOa)ep>~c5E zC}8?=XYGA<{nMXS$Um3(ZN@%R-QPJ{OJC>7tL03(mt1FL(G|oMz4)r2`_wfJm;W~i zrzs7aO*Ia;y#Z{fmv8zW$lyTxOU9O^!$D{)t-5VM=jO^uTjsemDD41f>p)76fUrhl%;d z8o{_^n^Ddt0Fcoq50pUIehu%_R^CB|d(vVWc(N$yV52c=86@jxl9L2eLturB7IRWh z1G_;nIH`5VjX=`gEl~@ieEKGVgkD z6nol}*<~7jh#QOZ9G`a%)*6W3acL7h+ zAkFAf5Vx$`D%uW$?Wu}kV=};pY(j1|!)ddKtupu{_CGaVoNn#Gau01yav0f84c$ce z`&nd>>LG0PyaT-1NU6DK!s;{)lT)5N3NtuSC0xf5175}G(9h?@dzY^SutVHJ)_}o` z?@2$!W=dwu=*@+wFVFvW=(%Q@?i)NGv9Zd+*iBAT*}8W3X9%~(%}DhXu=nX>&ZK*n zcLZa4Z2}Bj+eA_%s2C}X>@jflYCdw zGkIpE+swsWv?hIg(}2U#0FQw&a==FU1~B&NMaxv>S`T;Cdas9fbU-Y&O<|85H6Z8o zz_DU#D47)}Xi>h>B@G8DOXPGLr*-f-NAY+zUzs#gj>-Ql>-{@laZL+{JByA86vOV#sB8G6H(N}X{fWU!gTZ! z;a5uFkG+@2t8W6Z7PG?m0fhAWxmUb z72(&e2~*kC@}?|E?|BQ%-Xe3=={rVFCBMv0G4G&QspcG^P>C+SBUd8Ys|NdYL}h+# z#NnaN>ApqyIC`Vvf9uh8n}9250$3fz9X-mcK6^Xw~A)nhAbK1H{sqK7~iWd4q$OtpzlxPGHI-(tX))Bz^tH#Qu(RJg=sXVt|SDBF2&$ElU|DGH+6V|07 zDM{et|8vJ^wD1(1qjMbk5%nE(4-?M4IIylq&t(9ywSP3_&@PFtU9wV_*Ma`uU}+VUKA zN!p`}l-v+rhf9#ekWh#|6xvbd!{SpHWecSWlTny98oNj%7_b#aGG{1T{)e^rt2##O zSW`;C5zugYm=hRO2Gm6@d`9@=2`$*@V)k{T%s;dIY zdj{;WR* zA2HAMF(3T@mVy|H=!%P7fs3EVe}~Th`sv}uB^&y6f8Ae}1B?m|RXk$~{4|NJgiK{b zWa!${Khojj#E`1AcaFyz@ArE@t=1e{>Ohc&L8q6s`(At;JpX5d0gRrt56M2)x=`?I zd8V+#uph<8Hs$fOO2tOXWl#z!x33AUHZJ{JX0`RV<$SY+Ycqis;8Uim=f3SjFZQ76 z{zu{dx=2M@d5Ol2&xT|Y^sf@V`ZB*e3?8F~PEZc5>veaFM&+^u1XH;GGsw+Nvj(`f2{>hD-D-w?pYtoCX0Lsl6DSE?TnsV zO==D507U~Eg9dYSdYXUDfaY!}7ssED7Gsmq`aTY050jU%gdml#hVF?seAaER<9PNi zZ_RN7G8-y$POnG%-|}c9QT^+JzX)`UQd{j{!Ig?&A@ ze$%e~z1;w8WgZhW5ZVF5nSsk|^fWk=giFT*HC!e>ne}=Hp}VD3rk&8jbIOBiTK?Wg&ZaFc!d_XJ?NUb~k8&QWYut{P3t)4|d~cb@kT)<|ENy z{S~Swl9$9A{_8Tr&7%vkJ8{}>U|tNnVSEW5M5dzxeOWf&rsW8&P)c|qvYnjTx%~=s zobRZgN7DbfqmH92LU$uqahyZzRuX26lgX?$YHV#zPhe!fHh?4RO`7}xR;~&>rdv`< z)z9+I7OunNTbUf1UKi7sAh_^V+kh*n6TzMtd?THj_leVw)2mB5z;OUTTlv=s$@pkz zI>(z5>l4}SRCcPG&6QMT7_J=_BYZ7(f+*FDO(h;BN&I1`7^US$5gcg+@EC43;Evh+ zbx@c+1OlJm1-*^x(Ojpfc_{!{qLd)m6Whs*Hn@HaM;cpek(A>6g<$|YGO~DkUqlet z=`YS!FVm4OP_Z}Jp`0vl2s)Lqbrc(|iy4EHep{0VNzB1xrVR|YwNs6Nh@Ha@+%##r zMqonvOvZ9Q`{Fn|{nC5k?4j{m_gmv1xqPre4j?2(V#6{~_~SS!&rxAv;cRYUzgF9I zwR{wDWDENbAyp(pJn-z?>|=gL`|BK7ZFX1n05En{rFmI}^iK=@2#`B}WlVep7=a8d{0*oN zm)Ku>up{BOBq3hgFi}S8n6EouwqWs26r==jl`@j?Wg+ovv~^{$=QflT;urY^wjT=q zKZ0j@#V5g_Ju&NP=rUDY)kIv|su(GK?(fI+%d9ltZ{;&QmbO!qF09QIF54SQ8s4L; z1p>a-mMP+%DYG5HL#-yI@O>SB_niwR8P#)3vaRSq;(n6xanf1LP%GN6Pi0c8r7bZ_ zu;n~K;@lL=xKNL$Z8=JE#Fwz6ZX#TjcPSlN!d0MVumbL~b6|GtYD2M@cit+yF@!tD zxyi^%_Ia~9m|t52G(4Snv~((F-0lP}_iN7)Lnmed7?asnp`I3&Rd@)d=X&p+PO@Sc6=Jv!(CrP;O&ROS3I!Gmpqs*b$!W27r~)h0 z)R~Ou1&9aUfiwyH&6!Xmr=wndYfb~gs#7pnS_Nj-&#te)v7!FEouIO1**7TH z-IoE+YQ$PQMP7&KEyDSa0(BtR>rqB4V2IzYvZCXSp>#bRBwal&GV2sq)SRO|)@EOo zZxn(^gi4Pe(xVzSH4R#?!^w;q>FLA{OF}o6BFaL&0Wq)ikVsSvuf2EwD+ytP-TNJakChXf>J7^ zzo{sWm=2AfV<@<&p_gY(fcqMM?iAd8I1`ra%m>50OUuzAJap_ztXRyYPoo3T((6tr znXu-Zg|aO;MzYODayQd}h755N!X0ZR(O7*#KIOa!vZYg=<0z0ajy*;ATlKqp*3LKf zz~#ht{r9j(I}92mmU#xx@+a7MH=??NL_CzF=ox901k!-HT0}(WG*Pk5Vlor;(-99{ zQ%ytFLNgb3XIAgtS7uqdgMOBw!Z9N9>6H>4bghf`Z{(d=IqG))Uo0}~QuHF0pR`zx z*JZ<;3yPe7jeLhShHZ-Ddq-)?MB&2uG-^o}I*c0RM&_$01y7PZ-PmW-JQz&vAGUOS zYvCK{x#iEl@MRFgWIWz-$~}WdqcrK^v~jKII#*wt=-Q`pI)laVkyN2CB85XSwQWjz zg}6X_=Bo8SkShXa5>j7h#|=b6TlnTjAqDX0^`6?O=vCGhdj}+gEKR7ne+LZ0x03i@ld_uIf!wF{`%!1j1+nB+~-8T5V=)9-r@AocoP= zH~|D6$lho-A=$3^w7&)J3IuJ!$aX3&NEMCq>pH%EXOsSCtG3Onw%$!I{YNF;6FlO* zmrkrHi$w_dcNc^EzLTE5ldB8oTMN=HJ15OMRO{Q}+V(2DuGsFw6%bNxvr5j^6$q*O zS;Q>5?E2>fbb$v+EF5PpOfBG5&WB^|fc`$7`rEVyezC^Zp+*>|7wOqffL^Kd(nHMA zZ_U%sa_P6DW7MV1&P?N(Q8Pf`wcXLCaI16nO2}qEtu$C9Ci^8pGBLMFQ3lHs=u5L7 z&-yMyCm$ek)O;@@`zWF*BiX>E_mW1ModwFD?ZjJ6s!7nU#8_Hcia7Q<(U$z)WGDQK z{`|w5+9vxwpug@C^PXK57R7|b!Xqfrai!+RWecRT0eU|y{r2Y54`<0Dg>jpAeU~<& zXJb%%)Vtl(?Bi6Q6G|&gA%IQx8sTy38E{-Y=dnCEYiR4V zC+_sMaXcK5d(64tGJ+nuLMK6+{g?J3wf*ILoqEBehAd-VPU`lho>L#u`K{I_4;pSR}L z;-2`0ityv>fRx%XXhEl96jhd_l&qDY_!t@+Qs~x#yzQi3zYKaRNn$tq9B8Z~;~kdT zGR_XgehU-uRoWB0TOZV+F@F@Om2r6t|7(uVO&(6qWz15dayac{ryg+5iUdmokZOvP z0j`sho~^N%D22kSdY>wT|K4}k^tuiJpuc2*vKb2K4%U0=C80n5--p>n(;tb?=Ifk> z@D1RV-vj+sPMYDpnm`?XF&ts+%m+~6jvXgZMuWxv&-euYVHk{2_bcophRX4ru8VQG zY*zQFE9LlQMCv>bk!FM#$B%WxBqt2H#&doLC z>_6RJi=ee%xcL%nzy{^x*4l1myt?~{KMvJ*0+c;PXGQ|f>lr_^XxZ(!pk}&^`6^}% zA|~Lq%P_))QxdR_tpbjP&nN5Yv^|TP&bAyEPH}PPR^6{eH1LT^}~Q$r^vtr|oOqr)yfo zfhzCWw~;3KQ@GN+;W*CPInHX{|A47wvb^dK7|}HXnhv(HF*VgmAaPlu<>I7ef!huo z)f?@M?=n8^n)~ybOjIdn;85hY91sn7PpbW} zgZr6Xz2ADHi+LMw_k3yqG0^uA&}l%=l^ZG1{rZT ztM-VYt%v zo165VLwC)@KKf2(-`+P7Wkk#!U_TpScQ>1(99*ZJ+d1Ff+4dc(nY;}69RLO%J8?p5 zORuR-%UA=}4)*pp4Y2j`3OI_aIEtKSN78LwQ*+@>Y)+dPHNvo-=W_V9i=~U>kAZ^d z40|&hsJIvVp|AMX381SQG7Mvkdu)5a0Z<)t#Lkcg5FkanR1Tq908ZILQ&HG2;$af~ z4_xy=NPQ9gamzVU-RTI&z+8g`>|}nhqc|C3qF5L;v+56I+|iAeY-a;2>^b}2^XOm# z_23D$4J%cd26+kW5gmQ$c|NP?e78gm-vWFsrpZ156oA_1zBgE`js zDgHX1uR3mnZ{_vyruWX7cYpf;b7pG3v83a_uj9j(_=kL8m=xxC6sEpI>scqCjVgWV zd^sUc4iX}Ml?2g7)K1*gufqdC3c^jksP)^dbzmcWQgq|!d)d<}^i0TT$0fPZ&FquVYLL0Y z9E~HaW02CG&%?a7>Eq%@lhG?@!&6=M1t^rcQc}+9z5Tn-bZd3TJ`l4*0)M@nrtLcjISx&5aw>SKBK|cI1WAi4?TT+ z{tcgQ(NYept|1KV=b<|w1)zLE}!nFaJu z>_5Xz9(|AD#60#UXMQ5!kg4O3KdyqFRPnb+zvE`RI|u}?`pHe)z5$|$9z9bv&&0GF zqshaX@w3awXCKt0O=zrIsPCVtr%%wM!EjQwwp2k zHXp)QDR(}u$K^(O7qzCDn)eUCOz1#TKSZcW*V9oX#>gKZZ9LYlxHN?$s$1P@Xrpl$ zR%!e8Az#S6R6RvdDlz>=+?B#5QcgkQd%a&OR7W49-pPf9C*9;z9%PLl`liT_VC{2o z5qF;cAUgG4Y~+|Bw&reGp%6?B2?OkbhPEh|+^zIesAc$)qBW$)v1O6N1p+MQqh{A1 zr{9?Be_-Vtz2rzL1nkfPB=ySj<3}u$@U`!GFD^X@<(})eJ<7X6%L;`ffI<$lxY2*# zdVi>d&1}r%#zOsTzLy)Rrh7mrcFks-=`W%0y+!;hB9BEdWhE|IgS{W4-p*z47X96@ zX$doPQ&jiPVMiF_hAXaQ4l=$RL>6lb+_bPYV*c>gRS0msb6eqEIdnpUbe+J z%osL}2i#Em92lh2T1WV@*eLK8FI{Q^%eW8{pxdNG4=VbnXK#%5acyn#0ZC?CU!VI9 z^CwO}A2VFvGA2Vm*y^jiXFo0}%0_jdu}@qX=H7qKFL06}XRRJvxmw-u)bB>s-XA3>cDD^F zf9d`DgnCqU6fb$)V`>pW==vic{=681FZ+E6Lt- zJqdliy2LW->jO_9CbS7u1M6Ai`8?8zf5R_L;0Raxv-}H}q@WZ1NZ&u23(Z}RQ=h`& z083$$h25(5a5+Usr(9R*H%4J8@aQ)bhaGGOKblGC0FzHLWR&3?apHM^nQ+v6>+o7) za$u2f0MC-CwDcPZbFu%_jk3G{&)ynHvf-5z4Hq`v; zsLe>p-5Xr;8<*NNe~#Za0Sy844MYdJw4B&%x0#RH_XEGhOkh=OM;=Jz5lIjz()T9m? zT5T20$>Q!K9{mkx>SQ*COVG)$OQjj;m4lrsJIZqCRJ(O{6zeu?i7Ny>g@PmLC}{f3 zO+JWArYr-7B|e7}4Eis%$AydeO3*f=pRNRz_fpT=-0j96U+#5mHB8c!W?r( zS5)a~gH;ovC6B1pz;xmC$tW@CF(a5u93g66^U@JfxvH4{+&cH8+tkC-h6lkB^Ixs| zOkK`PDo7;*V_64Ce*c|u5YpPNJ!UGgY^`N4-rMKN(`LnL zK(+NxsRuKXAKR_OcbBkc3JZTp-XmISy=h7_p9mHnws!0JE;usgxzXFlk~dJo9WE|C zZ&xU>)#~x2BjfqDfwtJK4?jklTrj@8ta`}6@dfdDCq^Ul{jqz*D-EWz7YB|P9s(8K zb(aOcyIqt~`iCBo#fdnrTMaAY52VIvf>Yi&GAC74zwjTa{~`YCzE16Y?h5456u-_I z%PEmcZYjc9L1Di`&-!D)^t=Vb`=*I)h6!yl>`phfBlJ*GU`}tp&^C+=>J^lZBFGD@ z1K2IV3-(A92K%|2`CZl^uUs>V7O!(VI{i;10g(E#5ZF)!dr!qLFNsGD>%#GXlCo{Js~U3kCjbNmsLW2`7Ed z57(TyKroD7282tDENTJ*P8UToJ|Ap$juvt0%!#Ua3IUpg{AOx>^%wk*Oo3eN(Ac4W zlu}iVxGZ*s#T}*xmr}>=d^i4$SCLc567kBIkeJai7%c}JVh`;tW{hh7UH|l8WtJgT z4NonCFxOUOXpA1*o0j4PH}|I8HYXq8&Da%TV3abtpVPt(J?=eYxx4hMBr?MAy*7t^ z(jV!MDO2aw%IF^o{o3hC5BS~~Wqh->EMssVd@ZERu%><0KI#1%3#)iT+2Lx*xn5hk z*qo0^y^(Qy@2Te$f!z1G*4lyR>N*~Rk3O8b7qDK)Q>j7s1GY`2y<0{2tgI9jq%@B6 z`_J6>Y>w#F;8?t4yo+r!ZjVRv;8?QSbQ_Y$$dZ%jgZJ+?yCr6DTLmMP-Wz3N?cCi; z;asj^pXJBg%-*`(>lWuA>w1(=V7!JKH|L*Qb|7zm?jy-NKP3gu5X|G5d9fQFCW=l; zeq0Jk(Pg0K;@Fqpp=QwkclYy{c}NzV>-|jrMo%P}pLQh!L>rWZRgh?Slg%>0>PNew zFShqF`=C`&Q+#cExJPlqJ~4MiHKHm2pOj9GR~;OrOGeWkYGHMPdT$1ZA5`0X|hi@ipN073}wvR z;`@2ief|<@1Zu4;Ger5Gk~pUOaIe7VLXU3DWI0kE&X6)K@qR=5L?O6*HNy>EAAbK# zQE#Laur9hc$7II10Z_-La158+!OGYqL|5=#GTP2-WD)I#b`}PNaEIckFUeQbJ*T^7 zQ7>p6N~uh|JBqJ;R&EHo!d;dj7y=RFpAl=;1 zlQTn+=6~x91KnrUzHMp-;ViqhtvNSP^dNX&IHf;tSvy9I%v0r!T>*_ z#F>L)JrgU-eg5F)PD_5Z#tc_iuT=~VvZ*)3G+i=1%r&T9!+1@8#$n6U9@dvr`!X?F zWj$&2PFelWSnDERG5P#t@-xkmqKAqi*TN23g+x1A=@$&TZ*|65kP( z>HQ*TF#BG5FsHz@CUFhZTf5xPEH&NHHEd+Jfg5OTn1xa4$k6zq4GDyy6xmlCu-+N2 zq8!HTCIeTq8i>`FaD|m;BL-VdBOsin`Ri$4%Oei&qoQb6ITYvIqim0@C0B?oi>!29f?YqR1JK*YLf60Vyt7H}snTj^1f(GpiIX2OZ`YYV>j z8W+z|d(uZWDh=_><46zp=)(093VCp1;?GXEp^8@%PY=*QIr(D8b^DS z6z0}a$3*-0#~HwtL}nv#&it|#!7}{*ZcDGg)+73WAa4;%E@7Q>rDzp2SU2Cc{{GxK zYnUBy84m_1JQ;xUH(4^?y#|$&Y#@Ev@=+AFdB59L;nr57(i{o?(~IePMm49bA_-YN zjbvvLl^6JdiCJ2&H9JZTQy)=O zk@+{>l>&y2R`4j9jlfG!K?akLyHf8TZmyb3>cIvC;M?_UMhZw?+pZ?+Q9{pK(%b2j z-z?iKkYh6jTYcx5-D^AdPpKoV$FyRDzd^O`t&uGij&}cGsxTEr6NU|U*LHuktyyWo z?fISk+Iv-o+%{M4Tb^d*{JLoZ@PN$FACZ7VRXgARbWH$2+fuwPW^k+?%GZ>yC~$cS zrT`RsKwDYc0MJc_N@HWA^DIG$`O;+j(d5tDWGa^Av${1FquMzj$4YS^kM-T?;f>gQ z3SlwE!m$761u$}ZP&}`rasF5THTa`YIi;N4FY(-&)G%(PgMWV}PK5My4QG9L^i=<& zW~M9eP;tQ2#-2-n_^Sm2d)M%%WQLgZ^r`Ndr0_QA%4~t*2ss3*pI_9l{dHQ66j9k) z8Pus^amK1}Rw{a_xcHL|sHGhINzyBSQYAC!UG<73Ud*)hYs%$h>;`4OqE;&{*CEst zM;w2b()XXgb+GCncJWMI4W}kFh8ynkeWmmEqn@h za5@UKP4Cql13hGEh&L_rtQU?y#hCraJH z@h)HH#Ab?`GI-2FvTc$*mIiQAMGm<`s=TgsZ7QU9EKi^?Ghnd++TL=Sj8VvJ<)TYk z+-W7NwgH6u}A4E?Sux7D!TwGk+ zd(-?jAgtfo@Z~_8noJ0{T|qZ*R+O6WO6v;Rgp1}F+u9{yN?4}k=Fs70a)OE)%x?PB z%4L(^3pX0zpr+`a6v?QMGAbA#w8>{FYZ$qkb*ad$YwJ`gvoX?6cZWl;GutH_sNT;% z72@DqeSHt=ac+GD21XuET%+R7R;t*FABkhni1xO#liDt~y`~ooWHVVN4D0%9P>M*ut#SgicKgN9_l>m0c{buEf*lxMK; zXXcj08_Nxilhu&@nfX@03~#2X_MuR+(Wv6i$E{>E*EFB)>kNNuid)otrTI`udl z#XTsQ(IQuT&m)%L5J=)>Kxj@8C#42UTq&dXscdv_V_h0-;9ivG@+=dT&gCO9Op`=bmMm-Pqbz-#>v zo2T2>_og$BqYdz&(PAp16mkf_9nZa_j0-CJ`oE1C`$nKsm3|6uEcSfyV0gc1K%!-T zVlRA(nXLJDDJM`JcaxHB+WcNQvJF57o(Sk18{U*U_7`0b?U$P3DN)%9%jo+?lUr@R z@NT;CAP|$eq{JV)NDa(uWTnRaOb$9hbq$uRpkJAMN*2~HlUDD}y|b~ZI0xD*mB^;0 zEpNFWW@tRO=>OF{`5&t4=**;BmgWS6`^=okdK-BNJvMLM{Zesp{8u+)$m{r8vZK$2 ziriE5@n3Knzy9jz8mf4MV`ES|^-tMo3QoW(`{rKL8$R0%pEGzuE+iggP#d1USH+{STp%=Q7 z*;ibSaZdWUTSB!(E4v-Z2vf&55GdaFHRbx#65i2XYgjn-k=3ti+4MrS- zk7xdoLV{2`zW2V{Z_?UhSf`&!Bk4+WXvXDdB3Y)NIZ&`pW17RGj>zc^1cA5hY-W$z zpnnfHZ%%uO2isdYHqzQV0mj1eYjZX64e-9x(=}+zQne%b;n1!4t_ojEWJgkwy)W&+ zEtWK)2@C7srD6Up*6CUkO4M}C_HZ+Bzr7V}Zw6h5WIe4G9SS&3}UJ%*KzpAjpNw$uW$6nD|T%u=7!m%8+7e0O8vk7+nJH(yku^! zz`eJ3DH6>oOYn3-KZL5Y3*B=jS)X_~v2zpS4`Q&i65B|}BU*dPoTAMl5+mcx#>sFq z7J{Ti&lk;T+Ygzsyf!9>0pzF%GO6ePW9lu#n*9GaZV?@#go+4Ae-WnC=p3Vz5Cjzg zkuGUz*nkmIBF*R+Af+OLbc}8g7~P}028^!z;`hJryw`_JqJfe_A3ojjX= zVt4ZN`h5hMNYcM9T1);C+e{f_@6=wp^IIyBV(VKD`q9`U^kwX+PLl?XAO#0bv(u^i z06)M`2P!@_&hJD0$T$1VF}`g{Jh2S)5ZKUscg1@<=8MKn8kPS|JN~K&{cc}2k-K~A z9MGoZ6)Js@+9NFynI%xObE&C&@1f3uXZs04mIr_6j8i_{I0Wl5hg3AVMtV+V*{_@$ zVTMK2!(g}Umu9b{iYw!wb?>%+h$p>Of?V1TRU)f|%DnBXqpM59gh>%Q!Hm=}2${U*yZ=xjCf)5swH}D& zgV1f)ZBZ`h^8D*cJ-dnQA)hYg_Fk5cwfh12P%Xw3{#t~Oq>tsx#q<5X8R(`lKE7j`Bc-nE{s$nB?@v5xeT!Ccq>>-UYn~~c z{}{kHjKJBlDa>-#V;as zr`NU8Q03E1d#8VI+4Zca{cRUBVy~BacF5n6GKs_Z00n5(R+J`4^S8*8j~%TvhEwf% z=@;YaTXtH^p&D+$-JIHGA$qq9@p@qrcvGe?6)L^$xofU2c4kgWckGU+4-XxN+gTQ} z3XHfAE-U+APphi?A5%=^q-TUGP5_*oM&6pIUn}M3z4q(VB}w@`AejhUMg5CFY+THY z-0YpebHP`(R$}*#Mk?ycD9uKzdR;~A;{K@rE2mBUIY;Y6#>5zJ-16q{Ym-o_p2O0y zU0TiFJbo;#N4aR9jfLrKG6Q)R83*dE=gn>6;xN0i{Ds6=@;%TS0DKIowciprHQD4J zSud_9`uFsC`qmqu|05I?>_@o1SO>LkivX`_<5Y)L`)`yw6G>58r*kkP_;U&>`qRDm zxvoa@H;!aq%dN|;xZAU3N>lRJ@TJck|3(@J9vEwfyFH)g7*p?vLPdn|zYni0Y5 zQ}(;7FydXhFnMSBN1W?|WpLd;qdhuF(;KF0J|3ro>A16)I%_W@hV?;U9ZWBB?J_(& zcts|pC=**yPI{r3I%q&!xVo;K7XU%Omp3A86S65kneN z>7~TnAx+xrVZ;DlP0V|5A0_&KzKQ^VxX^Ws@y36$U8)TF-|w&UP+p=@oa;9^NlVgzm$-fhSiYdlDn1% zoC=Ssz2SUcumTPq@&pX93%gr&y*f{Eo_s~;-=}hG&w)&dik*1CNP%K$9u+uu=d+Rb zb!3plt}Qc85QGy1)}Ij@+{_Yt2>EztUUl!HN>v#SIg7o;xe2W|5a)r;IIm+#jjXV+ zQQBHd)LXr({#fu@XJlQ8{GV{+`{~ws5d~3E7c8kx{hhMnK?M@hztkqf9ym&vzYo@m zUNx+7ZtF&r?N9zHg6+u8v!osVk){O?MUp5UM2OO@E_wr-=(esT5+(SlAFLH`f+gLc zbU;BC0)yEgmSqc!wS}DNf7_csp2k2^Mc$po{UuNHpomvJ?!5Pz%%+|9v>A}14O7Sy z+Kc?z)IOVQE_40TC9mSL?kQY?Ej{UIShl9rP0|{I^3eM_-5J;T)yVB&1v2u_Z|oWg z%F_ltusG|Jc?{({p}PSRSI3m!71#Yk!B78)?rk}gtZoaVE9yp81e8PSo>EnTKeas! zPTwL|pVPv;qwp#Kf*~E?a3ZVOxMhil0(?WZnlY^!^~1cx{EUg%-*O(mR%o3Jii_^D z6guVDx-`w%$90)CC$V(4C-K=H^y_f&p07p7cKI&SwvfqG%ZMJs*>Mjh4OcT>=XOLB z{0{}6^UJLcWLvokIGjIRZvVjQIUB7Ejo8%kSsQsQA;qbyxA#4d{N(oavYZgj)7+4Y z$M2fn2co!rX4{!JnetQ}JG%`z1{_mbdw712ye3lskb_Nrx|OB) zJH@Bgu23_y91$d+j!xV-U(aJM8pF zoPeFL8dgPDmdfrzyykWr=wc0UgM8i@Y~zQglE@oNyl93X$)>Sto`bNpf;7La{4bWP zH3kpmN$SdPrZX^%QI8h{O;J}JjuxlCBd3LntrTNwKgEIFTgP9OntZ|@xEPSQ`aY7VpX~Y#e|_wH+LZ2Wp;i+aWljIU_b9%*$!)w2a4UgBEGu&Y$|* zQfgI}a^1oVTif~i^8kqp6-d7KF?ZI#Tc$4~{1S;iR$=10_=i?Qp&TSZfA<|6=yLzO zb#!Zc@rr2=2lR!kT6m;(J2ErMV<)|yyK9kQ+Z7Sb^kIS_iD4R%338N|(hBlq3RKK! zjzZPk#<}M17Fu}r!$=J+xFrCCH7et2yHXGg&zZ7GJhZzFBlSqN-|0hV1p;^pdK8_j{WXDkso z{5P5B#fW_RG?9J$6EF}yqB-!U)wjB0k~xsI|J${2=MJqz?>Q$`^YD7Dz{i{tq$&p} z-Mi;7sU;vd*xHi+{is)TnbBk8V{EI&m1%DV||2z<{r?b_aVd8*tK)78B~CY-4a^q% z)Kwf$o@a$4!?S!_;jlo2l|YJ8-v~0;Ec-1l`_~ zw~iNmZa?FMPiQ~2#pwlSr>{boT5R(?-Vl60c_F$q(8ccRXqx}9rN1KMgV_YceS%@Z zxsE*}yep}{HDLmtZvrAokaUWAlQ-LF4>LM68QmI)uI99TIzrn?EmD8rlX>;5Y@73s zGpVEz5XC|w-i5ta8|J=L(HFC2%I061%w_HV8~2ntDt?|2S?1?a>%F*Vt;4FQN?hRY z9x_3-g6$O}x6{pUpIq-var}A#AE9um`Y1aVa<50C^*{MQT%lzQb_kczO@=yZ(qqcnbZ@~}3 zVEJPCd}iP>?{-G_d>4sxw@3xojJ|7D+UA7T#AL$7I=ukel9@iC@G!VEk3Rz5N}mT= zqF>;si?w*pbF$EP&~t&kQ*d#-8XUZ#IMLltAqqwS>K5ebaRveeDJ zbL9Y?hr)APx%B*;6auDt-0ldmRv)uXL$W!Lu!snN_itLi=o@YY}v=B|A zLY*thH@2_ws-I7=784Cv?HW^W>ni2}4zMDyEU``e3)yPh%(@&PpseRl@14S&ZKpnx{XpH4c5{IVtMq3K92u#<9 zlNThJtUIy14gJx~X-^b)TqFfP^b|3^R9l+wcphG#{&#HbPbxux*U<7iN$@stzQ>*4 zuf=rRZK3bf;ozM|Qd`8Idj}eS3UJutm#?iWHm)BMfBPw~w1`9hX>_HyX`D1E<1SVe zU`VZ`RY6yvG$)g6fGxDt-(gWv&}n}#vY$o^7pvjPqTcqPI5+LR9y~-_Lf8l1*{8gK z&#(3cqwBP-P%0l?9*;K+vsW@d(d+7GBOwImdRtV;N&Y9Tjo6;}jAc2BL86N1GN)$+ zd9{tK1REtucEyx9pHtsIj-{lex@+KsuYAAEz}txZwHM}?qp>W4UhuL3DA2c98HpV4 zPxoJ?+Va1Mi6>G`zWgz*LbdGNNhQ&^7P3XjKmks##5lWHf3)nPhs~*!hdqcE6l(R? z-q*ae1X?1x80qGM4?lo(Ys!)DMAkanDH+J6p>6otZX3QZV)gSMVS;UK=ecGRuH5mX zbWelcg|RO&N@A>~A_l2~E%XeZl+4X<0MvEqaZ*P`zeajZuf}mMG;UNZ6>;}6=3YVR zuDBk2_$7&HVtIAAR!>Q~13~KZmF24QK(pk9#zf^zuC$Ur7y|ZLR2PE|lFa4s%*A0k zrE__hlGZEVnEygFQPws)SV>PPF8=ZY$_!Q;rbo)#w{rJ>854fJc8w5y@oT?j?zwTw zBqdX;y@pum=u)^TB6pGRYayWbO8VtT2s|!h(BNC6HfVOX0(f&KVc-v!7<*lh9L;9o z8CqCEIr$Ed!YDTAe=?I=yxtD^GJRmDyo?_C ziayjBb5u;sIym?A8kxOb0;Ar98wfvv+!Kh(=FS> ztb3R1z1-y4y~i5$Ss$KUpnflSdsLf;tM;> z+t>3rGy(fh>|etjRgbBwJ{;0VKStA)xPGVR)WYY6{#{NO4{^)`(ip2x9kSnBZVSmj z>Le2<7q^Hb^Z&7!8nj4e)fnaSbYV7(h5H4_<%DFa0%mbKxs&@t_Xsw9gg{SDmyQ1N z{M9>dv4cERt~nHU{X9RLIVbMe_<$qgJjvyuXh1NU%WH1PEfLdw{WYXdMpJZv-?H&Ysx0u76can`tgfc!4M*|54{^WPslXjw+>{s92 zvXIL3HlhQaSkVwz(F?#SW_Gs|v;PXCHWzLDqnQsU$9a&^O}9i3zf# zg3MXCt1mH|TR!(XA=FE=&BF&%n&K_~ZaF|%eVocL-CslAF4Dr!{Z~2|F60l&FH}6<#|M|U2L<@ zk~Vdc*Abo)sYPIN+g~6K`Og|Q%TaocYZB@!VS>Gu3NrD6B60Vf!3(fGrz|}3`dhvF zENUTtNW+SQJeBr|*!f@o`A;`?g5SQB#yS6)eJ>GmPi+reH~%U$&*}l1;MKz)ps~gT zIZ7e~LOCwuEx#xeZ;beMPS2R+7_#e$rkjQ^F>+VC6*Kf-(C+Z3a_Zt+=p?Ua*@uyI z(?r8|m;!^U*~1epk8St65JrRMa9T7&%gF*Cu}=Y$2L0(_aK>L&~Hxr=W`g}OB}$U55d2W!3}v1W^}25{c2QT0COoN-8@L|8k_irS}o20{LV`2Iy6E}$y%5yHUz|F)um%IeQP zr=voM=@`7e*WGQPHyP@)!FOZ~sW+qk!|`+ zLSr5uX`VjnJRj!id=y?UC*sV-ZLbQVeT4ucn8~*|^N9Qn@;|S^Q2YQADCCu})Z-`* zUjb@pUno^9lC8Fv=~gS#cVxE1;^`K)t{nY#9mmDG_wTl~>DW=GNr;xJ276~){0p+! zRP+1;uL|Dz_@Z#W%8=UL7TpxtyLt;<&47AfD4{d5f@K^emtX-kelN}-Zl`Hk1snxO zG;fo<-F1mIuTC+CS$_r$gBIy*ME7zFGpHp(C&r82s4iEl2kH*Zn-o5LQCNcY#bpv& z*q+&=&)@^CIs5Nn12-*SWyu`>!IDSZH_&u?E%{(OYi3oj?}E?&W*T7-nnyvi#w}qG zRPoGb9%V@T6^fuxLu-?BrD16lMJFhJ?B)+lTVXo%mln;1`;YH@**-g{859VtlP4h+ z8Tf96B7w?T?Byoz1>bwfMEJ&aNA?-Y`NUefE#cpu`xW!}b{vmbvk0SBen#qk+5j`4lx#wNtPuV2qqP=NwarQYCzdp-(h8c7< z8y#OWH{)sBE_qstNZjiVsq+x%^FgL>#Zkg^qt6Z3Uwkd+vU}#$QZVp&(OFYsn=6~= zx12i8JAJ|;_RFJxFdBcsG$aswPm?<7R7V4*%$*deW&X${SD(jH``QX;M?2DC z`#;r@zjCIRMJke{fMj=?6u=?b9}|M=h<1P^{`w|>l51t9wLPrOlWkJ>zo#}Z6e@JE zeF>n6sntaeIeR_ew<`_`^1`DGq4yKu(AfTLw}hIhoV19G>tb5Fu{Onz zN9}`K9?OF1;x+qeJ5$GoFV?Q=9kP1@$_ZWWZ=q1H8(uB}GU#8#j*V-1A)~m!8K97C zBA82U;wav&Q^mmPuryD~ZaM4D2|nLI+cC}p-Yi?vbszJDQkv??#{n4vSh}x|`j0~( zhq|RuHsf()I@7KXP(>>t3CGWFUQ;y_`)d0*HOCIbUN zjpR7OU-{wXpcxw>?dpNJPljg`qo1zHD2wY^B_*XLVC&$Tyn1G^{w~3KO&XAsSOCON zaPr?M*`08EuZ2zend7$2YZ6vhneG7@59n5E`uQTbr2aI&J~_?j@}#8CYgsT@GCG0| z3m?#Ft)jFH<8^#OizTW92(~_NAg<7(Cb5=1?ng}Wk>rOE=LPm*!f`;|>G@tl{S@2_ zNTR3dA61XqiJ4uc)Ssk`UvGLjJk6Bqu#YG?fd$9z8b-n-aPe)#FaOyW>cX!TMc_!F zo-bV=uqfCo90DI@?l?QIW5_r?4j(iQ;9-UFyr}s^HQKka{jjt~Qs4c*|gd}Vv zKw{574wSF7%w_1L!CX3@+DgL&*!5mO)-D_#NFhfOy@6}Y!!3(`TB)2g;tg@w>!4*d znR7qi-0Qe*YPC@$z20Bh z*-SS4cd3+#*GHysRt&!p@s5Oa+oaxmK~gt4Hx(hkG6 zRj>I+kJr2guOu44X#p6x*QP0R$^}51IRaOf&$6FjAGD-jH1qkHUvK3tfTOz_g9dN! z09;cKz_I8GdEovS0%MyV+Q=@S(HdMCofJNm@(8%!{F<8vm|MG`#p`j_vQ(L#9Hm(e zkR2DV0brzYJQINYRLBMg3n(N>* z5k9Y|xGv&9Pw`jkPsb;g(expyj6uD!BDbUP5dJ1Hf=5Tmq37F6L7j}q>OlzHv98p!+nhk$$+T+{mT?$fS1Wh z!kk9J%puu?mf%n5K=>2lj;P~VQmPem=7=gV@L*qU3{CjarM(udJ|O^I(quBHo7c6v zJZ_i%%yf}i{B$RJ^;0y{uG$IyA5w0O$noG|{b$S?U$LcUbMt$Q0%HAVo(OS3W>NkA zcRp2G^&@}~*Yy|WOZD38EGg-O)$q*e@S0-3uJC@My(K_I`^@FV%J4X}2T*R)W;G2w zG#Wc^mOhvFEDz``TGZK#?}Hn?i3;95yteFY@t{~wLJiNq(=h>*r6&dbv-ppQb@(ve zKk}wKvol~ zziDE(^FTg=B9KG1bPX57DnD#o165y1bygdsSqLEm2?=nOoW7yxp2QUDrm-se5W?f3 zl*m>2$yv8>8@;SGzs6u1@s6vB(K@%o1`=4&Fcq=#G~X$A4G_FHwD*7m3+i!8J{CXO zh=jI=r3sz}mD(CSIQZ=D1T=%~2=O?<6JcF_2E7*v!pr#E-YCSAZ_N%Jd(_{(`ZiNf znf{6-g3G+n>caCxzwO475)OtVU_1fq?2|#QHRpVfR1Egd&D}Og^T9ynD|+1=GI#+! zWzV&c;4@@zyM0#4C95#|Pn+eA`Gi@=R?cBXqhz?UzsW4$9k|ss(N&7Y(R%XSF!dz~ zB5L6}xL&OUk_k|q`CP8}h(H;?$u=dLbRhB_YiDmmEYr5G708q4y>D)YF`wZ-BR6y_ zI(-}EKU0`w&(t#N0GZX;YYBmlRie-rfH+L@0-LOmY2InDfThRqDf2F;zc8j zEoWx6ETH@jWG)<1ZG}$z2Eff(G!3r4+kjr_<#H9dk5Dc9oEy>s4vhEoRJOjw2jLxMg zKFn5r3yL3{O^lOvCB^25f>;hzjKULYBw`w#T<^Gi|T{>$Ct|o0iwZqJarcpY;LlBU$u=NiphpF{PTaRK~o$fUB0Si`*8VrQj^UOxu!*9ld&KhC(Lmx z1XgoBuaEDgeFnP6#L9g8)-3-gS_Rw?vdWS!hUA;0kyS*dYb%$*UvuMIPOVw(&m@;( zH7ML|T^9fLQIIPzBp-Tzc-LbjqhN&nAOB}IVoWEsXq;N54LhOpj5k8mlbJYd?wXcf z-`(wx;7prPk1+bZU5Q3&lJsbqPoDSeI1e*#HE#f*7u6pG&eC8wt$VaMEch(5 z`A3d?AMesi?Ncz?Y>C?R#V(FN13m&Dj*WDDLE>Vq!He(}^1o+PYJf$uD$fv}@~p{u zk$=F!==vSF#|0*0{m|S#x=t2j&q>%Q{%19^YZHMyONvdvWQN8QY0$qB!MHudH_KlP zl}oiyea7@KWW~}5;8w36-aRR6%}h%&(ivWG&XRMMrNSlZm79CFh9Xa=*U69q z+E7BNAVe+wY-(j9#+^gNj{gSaAl!{dQm6k#y9(;k6F5E8mlg2Kiacbq@!&uR?}38Sg#rpUFR7GYzgCIDddIU<_Pod6>^GN)?#(;3fw#7LERUOx0B6%DK_|p7ID7hj z>dHU(O+*fNeOGr0{B$i(w2#GKIkhuNF_r-yJ^oS)#Bfm=Z1EgWvIna}$X*7C`Lm(> zRu_#d_`2+WCZw*;vQpUfk%=#b;Q9|-1&E{idp;b=hE{Z70G}GRB>OX6&AK?Izs#qf zCcB1Q=g_c4n@(SbE>0F{xe(?C616X?{T8-}foB_V>btB8MK$U`y)r9XnzMY`s>|gH zq513YrQSm&4p~EK@?qpI*$et7U<$(ndi>kSI?~zSH!W`%*U}^u=e!1*+SW3bNyA(8 z104J9=Q-8pBS4TPvguddNI3UhE^h}cS#8Vh!YeZEX~M!8hNgdGVF^L)(Z+G610G`_ zhtvH=K0Qd}i`h1>QPzd#!rXrNbKFk1i~?oP9T)lLsjO6_cW3nRb(=NZE2-DrF8!HA z6VY!G^@U18j_M&BdCy&r7YZbh_wh5_rl((X`FNVzf49p8EACR&{1kce6#HrZNO+hH*Olof!yQxJ|zFqVL^u)e3qN zN)P?KhhH5tZC_}-Z)60xzaTbk)P>WX)0c7xZrSFXZbj+S{+^UZUy;8xt1mIoFhK=& zad_G0iYe`$hfd6Fe{*PXpf*Xm_b;o@tqh(a$V}Fee_<`&wG7%I|zRM zR(j?g>l2OTOg?zi6bZzU|J8mfAs|Hk8SZ*8 z>p94QjjyeqGkK;Y@4h}_*$tJ0vzFdb#1IKkDKb;4f0OCH^_?H`WUMs>D{HnjD2Yzb zvT;?QjE_}agxYH|^Dnm~Witqx{A?>J#qz&Rk9CMiD<(A8l^RuyU6G_}Ab&E(LxTBjtnRXR8k{2)xTJA=c^t0O<0 zPl)HiUKiG1pg+W|<%h@IJ;O2Rv#<{ey?{!nrkv&t-D(Rx-wER9CcU7QE^gM*uXz=* zpE5IrNj!c=L;SjW=~XuiR2eGE;m}4?2Tiw}*l%j^-#C|M5>~ifE5JgR4$XlN+)ESW zpwn-7mswplN@>@f>!S2&ul0>_>am0lfPSC&K^4N@Dy%g2c7uBoS~31)!6h}3Myn~k z9jHSV)wHL!O`8qs$Gqzsp&z>19b(k=2Lo%r96~uqwJJ7$_6+19vk&>LnE{Ud&0+XQ z0Y|;|YZK&1IjvfrJ6bzP%&Vvs`0lS>dna|YlR>*-1 z=zh!7^}m%mdd&xFXYdR^*ZT4vylR$pq;Dt=&SYjF zxy6c7lE<}07F%0T@1-hR+{v4-QAcOOt%PfFN4xtNGVmHH%Koib5ST}MFLJI~C!R?n ze>O;~DOvP{Cg;F0Z>4rnW?d=qtWGF&er~uF`@<3Wv9p8>eDbBDuVtliHrF|YOqCW* zWhlswYcY8z`h#MwlUOumsMe0KJNuM^$NqIwi!$X`)MJ^m7lE<*QoA>AKV)ux;l#>p8t(XL~pT$<=Q>%16nbeT37CK-0W;uo@@mV)97DNP@)+2^028MsM#fU2-3x~wn zS*BGoRxjHL5_rav6QrQ)g&LErHoi=A7r%~{bMz&H@)PZ@3Xs8H-!LF-m^|8tor1}| zxRa8O9fb3$xdAfeLJQz^O4jW`&LUL}{N-5~omO?6s%bUBakD z&!5|#^>Ks7&JIPRRDJl92Q=%EtAg~h#^fLmRqXlC%Lvi3ij_oJ+eGab?_w#=5_GguaJmXg;^Qh0C%Fu0;Vj@xA z6FOzXSVJbUpM|qLU0__R(%e$f`{m!_-ETNd+cHi^zv{Cfprf8v^Z&{3HT>Ldq$2q; z6Pcj?aW#tg18gLdJ7o8FG-~&TsK2uViiSr9g7og!bjbGa7-gzXk7cuphVX1 zTRInD$7oTy4&{n2lQ)m{exEoc{oP`IP4X@On}A;DwEDmePv( zYR9o|+nBZYh&^miLR#i*HM{?$N0q^y@Opw_CSpnnS{sJCbNc2O#g&ONNOEU0eOdya z!1GBr7FXJdJq`*UDpI!Sun_eRjwoqkU>vXP(UjGy;%OLYV2FE(=_ivi4!m?;|IIwJ zQuft=8PXTJ3KTVy=V*)Poz?dH!cf)9X?be-FKVVg|0i$ukV%7C((JoN)E0%4=5G65 zZR6M?%ak3~sN&va4{`eLCzSVXD%~_%n+bzy)Pd28u*>bCaGOE1aIcG~->wZY=*IlW z8qz*OSM$=QcpGK&ze@p?Xc#yCc5&z1DtI9CB|cablCpIhQ#J1SiEb)JFfyr7AEAh7 zw`F^i9$!aiA>Pg>(e;<#a=+ta5PP>@@BVs}xdCkJrZ;af{>#UB*Qeo;3^#Tl3;o_| z3Yk(~P>(}{iEgl+q%ZTGnPlx$4|Wp9n73oWi?5$e|Gx)@=)ArBqq*oxO?Xz-U8adf za%ST|tnxEp(GxTfj1TIX%oCQ{DFD5*FH>WCJ1{c+nX?doN8NaU>RXMaMqCZ4hygLk z*nZG)_cNx!<`JK{h_byv6hwey$_$aE4N?My2c-Pxj)QiZCJidGkzLUK3*$VK@^61O zzc$ZbcD@FG42q>x;FdkJj-%;`Co&M~J~?##hHBStB9m&PAnq@IPMNm<7@?OAgfYt~ z9lhdunp?Ai%7qZJ!@#&{Jr>Lj zvz<%Q5RItWmA%DSxcrqYf_MYf0cuiimjOrw>UYmOWI_*WC@RI1}m7)2izz26Xf zWTZ3PE)n*^m(rigP$5+wy;Un&b@oHaCo;OHL60N~)gBYwfo~D@&?mVFY5U(x$hv+B z^tFZmhG*~Dj+F0CBqSLPa!hgz5d?n_QE?Er)axa_pi@t|$sgl&+*E}9gLz6;2C1As zL;1|dfZ<(Z?NTvL+E3~_wutL^flILW0l!RYAzUBv*h2X+uYK~pA8d}k(b|m(qkh;g z_G^Q0MTko2ys2q@d08mgqDCS2>*Rva`IKen7p0o%ok0{3LukM}#~2;?izr}h@ z23h?*Mx=mYtlQG@wS;o){# z$4)@GUtsQTxPSsHWUO>C_d&bCK6?Rb11Jurr?!;D(t8IG%aCql@wb(w?nHrDbS3uw zA*&-b*pXV)F@OHK^d0Yk>f%@{h38LZK>ZSKtEkoAqLolj$ZE$20*y{-!3ih z=Rx1ZLxeK?T<~z&)VJkjWy4`4-3|>8(VfH5_Zw!D8|zb02ih z$G$Gy|Nr}JvgH#9j*X1IjilnC>LYl_<;(I-T--_Q=!O3ahI}b z{Cs$M;Fkn3JLx?p(#Qqz$U169inV7}lKr3MB#noi&6J;c@jkg5 z>Z}iE{;NsP&LR>r!nWr-f@mYm*OsE~t0pyS=EATx{m0{1H#Hq+EZOyf4!) z_;~hd-GGAhu9@S@hyZHCDG636R&OXTCSKss{0+#_B_e3A^6^}wd%-{*C$5guDE9u+ z65_2Ad{jV*H5h*fbvGzjH0ktqoc5ea2YG+;JO{aLF=#+Lb!x{xo9w&V`H}hq%nOlL z$)_Xr3SPjk9ovili1oVygDSl-gcf#&Ch9!s`8yKrlK+3%GsWE|~JP-hMneFW)Qs(*dt&U`qX6blFJbU* zvjBSF-4D6f^Rk*0{YL6Iu~l~X(xaz4rrx$2t%8GZG4QyIlCH}khEQxS=Kzz;Shww_ zmFYi{RE2kQgL_O@>y`AMMxZY$%I({A5!|u1ggPY7mrZZbApO6O26DcVn=^E-NtKkgxMTSZXi{9UYp#PCMInI( zHb}-#MMs zlkIRCXxQRyX!;?~X*cgm=k6VWB2>_)Hnz3v-P$CH;=F*L&?4h;XQCxG4`SH0yfG5y z+1QAy^^tRoK)ZB*YRsRH*4LqZe86nX8R6;ddzeigLByc&iEQ>|kLM(;5&pX08XpWKh5!x?*AP2@+6q?%v%~TAeX^@6rL$k zEu`>2U15Jxm>LQPMxA-#jIiVAUdh#a_Qm4e{bTdp4$zLD1y2NNF_)gFGDiYG^%k){ zebD`XBNa$!HuB|+Vpc^mO(B733w$*12m9n@oG{B%H?#9Fk1@vaUtP4e)SWr7y&=ci zM8YY%HY(3E(Vc9$srPQGJL5jyc|J!Ix{C+Szuyut_-bX-`-RI}oHn>L7>vXB|8oHx z2z?tRo#}+8yITnN)UbgjknJ9m5Ecm9$2&ov;do zMNv;peVp~NckIgVl3r_Q^;N#zE|8+2S)Mko-rr`XPG(qj0lh?}a8GNekrzrvx#P$$ z_j}p+cP<5NadfG6daIX?#yGmi&Q2~twp=z9WI{WaX5KiuZ~LE366?>GAnESy+y81H zs`LfDGPY1jc~>0y{C@A96n90Be>J4pOKV}h2#Z)@I*A>{Zuz`b*Q-X!oy!F`+=b7M z9$jpw6r%ddB&~U;F3fI0w|nfu#b6L|Gz@2Q7k;0RMpXMy-JGB4NrE$?cbJaL`5+MO zvfzXd=$u5nT$#Q#JOJeOMiH_rE9pL$@1UjF-_W4dHSn_E>;YmWX$rrW>hLw!ZJcq; zEAbOE7}4J>jo~r~*VPnVuKkBRCD2)<_Mf_ll2%;ir@I)2^wrpr?QgTJA}kV1C3-Pc z?*b!PGKph<3WhDG8IPcc0|jij2xGMjWdiPP<{KpVFv)30XhAQq6{_f|Nyrt+b!mG~ zFIWw!pfA`yMvvrG%U7(NW?x2lyC)uEJdJI|yd@TXADAAsJtKr6+d_K3rr6ur_0*WJ za9CIv$=P`D8>~f6cfB0ccNco2vQhWS{=#r(m8?_ivH3?5Z0J1~i0}TkUb`DJy~2~} zT=bA5A*m93=TDKRajN?jY|t`tu0P%T<32^dMxlehC~^~f@B_G?Z3clb%0w#x9~wComN8!MjVDF1JXbE{l>Egu%%@VdRY~CX=L%iya>y z$@__;i`RzQnw5|@psU7LkzE}rvOa{rx>)jo*X~{NE9hVu=tZ7dkVDJ!P>FX;xZ8b% zK@vg>WsauaNsaQF_Mz+uVXE1=;J+D(UA(qo{_3Y;fx>Tj3@j2){XKX4Yp=#!cggk@ z3aIdoC!-G}_#&ceN2GS{?<6{t9|&4-?rp9ov@sxK@^qQd{GR76Yg}#5F5Y`vKyI!G z7ywLW{W8&#{`v@L}~TDZSD8$*J35^35+G?ghRHZpDu<{0#cA z%N70o(-3XyWrpsAUdTrI0eHAtro z)eI+hiszKwjJ1I_D*f$zwud}me5~W|RCwknqIe?vRm95-13fba;?q&kP&;k*zixbo zy*T>8+!IvvXSxp?Sn}~WVytL&Iq~Wa8ntXcN3hW4pnbV<>tJC3eL%FukX`?313J)? zxEIspurKg)=fisGi>@~l>jt0$i|(^^tat>$dTLi77x}T;?J{%egK5An4`j{`CT$!0Zk|$hO>gcA51f!RXoU&hMx&GH|){)jxCLM>=7QSyE zL6p{SZl6@4k#KcWIE1f370yc)={IvDS-Ary3i^7rf&m%?N9w#@av8~}z=)X^T3NVo z+1<~~xW(_bvV4uin%|JlI-8Hp!`8FX{?jM>_n9thYkWVK;dId)Pisd03q4KjvfF*5 z_lIfGsry48dGo8O$(hSTk;d<=_1*N9A#4Wqu_@1K6AtE);| zc(8K(@5Qj~*-vsbc{tUt93|5r^En<(%9J}$7+K8g+jc57IGZ9xwVHwW)lk-X^9a!i zm>}-R)!$KG+9Z2zANFyNp{OtJL7PplFPq^&I3*_FRP)pq=J)AT%Kx|Suk-hF!hn4sm{X6C?D;+3oAW5! zPYQcu=iAj7pseg3>aAqj~;!cL`(@z1kO5|Y!0&l8CW%<5hrcYU# zVscFH+wuWsK+KN%Of9RVNDE7+C7wS()P;%HDZ20JEcv)owz5(x`M*8PUR3SofxYEZ;nBGc@VAMgAtUnOtYd8-~l>Lj>+;DAaOL5RG|`?qotY zJ`m7~i3iRTVuYcJQedao*0`Yg6NtmC)9#qVEirA?5*^{gf5A z#}2``t}E22T&y{lbKwaSe;f|SEh;YmY%NfV+7$>l2%iU_`TdC6$L3Ge(#2~ht&Q%1 z3X^@v2NDicJO8kj zuS!v{k6`RcqMk}kgmVLZzqG~-9clD+YV0eHQ1PTfoxmT%)t9CP$>Axt;>pP!;vwW7 z4L8nOz_7m;X|WX_6*NJdOzcM>?z5rABlcX`9+rRIt;h4iCR0TA6F34T6Lc)zn0yHL zkGz_U9$kAK1nU`-Q%p5~hQ3K$?^ky=o6%`3b6}o`bTZ?w+~a8AhwtwJ`HC*M7;|5? zUmm+@H<(Q9Y>L*1gA~UD;vgoIA{J$Q7FppSByh|{NtY#!zKOb}9@qI!hi7_#@*|nj zSr?w&AH%_5*!8h(j+VAeUEeE;1vo8@n3+kA~@eecf0YkiP`=uRRd&Fj?bN4-s~>kw9g)%*PGoOHJAZr~Xixll~D=WR5j^qe@zx4cTrErWo;{^kGADJ70;} ztH&_U1Ky2`|^n9_r_9i)m+)O#8M=UsK<3;e*pr&PBN+`EUoQ+KHjpXBJ&->); zbsqhGz)?3}m1p!=DyQZY?(i^gPj8yWEO;TCWOi-wiJyX%T#>j&mU0fr{Vovk{{k8) zuU*@3ay3j9DUA2dof#8d=pfq~y?o>!4?XpRLak+(O`=xmPh&;B+AaBlHo12Dn?^>o znqPLO_`t`yO)|_ET@8JOQ?vOmvHXeqPKoaa@(=~tp%pES4!53D(S;E$F836rniSnW z&34-5mB?i|hfj6C9dFiBE9eggF%nEz(YdOZUBHv4=ke3wHa*O01y|C|=x&sOEn zrf1F_9~`Vv+dp@?ef#@BzEHnll{MuYn^ct8f$>d?4)ApZ`X{kz1%e&;MlSlNR0n8) z*B052#-M;DZmY!PxaNPKa#CKbcG?lJKKZOUy;?7m@}GRf2G*N^gaY*(Q4K1nU77Lh zVJAS-bsmC|qd~8~y3$a9ydgX7iVW_p6Jr)ztr>`wzZ#+J$8Ww=N*@ph2o2*#W;s#Pla&jXjR!ir*(^->b$Odlcw`iL9P`{g7RMsd5>6t+zs1%OeNu*x}Xv+-kKk5 zu1*Xg2$g!c#6^-&Zt-e+cU{QFGAL=B##){^#FYh44gM4?YTVTpZ=B&bmqwi2B-)k2 zb7ykjz*2CJh}hvlAT;)V08T{FfY60MWVm-$*!t5;Rb=bjO*I-4XxDG@&{&szoJ~ad zi266NRMI^$Q^75wu2d)k?-6~Eav&LI#OJem?5^9 zlogyO^vkjFQ|T+oTtp=WHn|X={|5$p;Ij9O$A#9^0H0a=B@v)(78hy{z4uEoC7(N+ zu9rVP#taWkPfTEbEZ9CMJ_H`X8SUu5GmSO|_nHunJe;)_9Q(29hT}ArYFy24^+G)5 zs$Jxfkx&6s$%%f@Ut=G1TUON{+?18T)zZ)gjIjASYlZER#rE(rDcAV5yKxTHnzm>{ zo(Z_?e^WSVGu9tz=XAMPU7jzBGk{8Qn0DFWU$5wS@ahDBg`_TAnny&27Nt9iV!IWd zD3S-@9hvEf>$`ok_o)Ov+d_{AYu#*Amzcf;3Bv zCE$rWP5;z=7B}2*Un7(T8Riy=o|K(gqK3^8j{b@bxLaj;6vN9GZj*&lV*$Dj;#>5! z8$O`w(WW(i;q8i2b-l-g)TLS}E{~1qZo%W^Np6PH7-MJpi@*>YLw~45L>`yZRPn|2 z0Mad|JuTB@q~-U|V9+!35KVPpsphG|pE|54H zf#PFGILK%SPX6xits8Qti{MfkDr;(^wJpz{$X4=r72>O?_%`isGu8Q+*srRUxuZ#( zqQi-bH2@E^LV+rG&sGTA+Ym<6$q2WuFolu+jOuN$hbIWsR#T=4_>Cn1ksO?$re#EN zEUiI#@z2cq%KFc6*nYaVd*RpW6(6|{r`JBwUh{!Q&x;ms7Isk#|6IrMGSHqLFI1GG zKZICX6g}Fw&8X5~I=vPMjftF6m`WA41)IYzWUFFx?|gpa!QmtT2eT9`>v6F_k0Q!o zJHtV#Mc2zxK;na&ZU#7a)aWH#L|@I%L=!{GXabQRSe<6OkWn>#Q!L_%DaUJU=Is8K z24zRB47s+@ozWtgisom>Ps_=aDGRNp`8NTY2ja8X^&icqPj-fHQ;@m<0UourHh}tJ zidwLgWz$M7vdjuNG$1~OpO$)PaG~V0f6RN-WTHC@=lN;EojRwL-qH5>`#X|_2Ym4B zRQGe<+^1tc$J~Chg$dVb%jZ$XJ#ql<7r-A{E&7#;;q(22qnwfad4b#EI7am>EEg@j zj&|n}=7snIxG5@2-Z^v;A_gUa^Nd-CGz-LS%Pp@c)M(soz4y0h#I0Eiwmiu{|5!6w zL^$R@{h}e@X07QBYT^b&k2eVg#8u+>$=Qv`hQkr@$!L-3fh{uJ&BXeYHE&1u~!*}M8RA*9cxf`9Fjp?;LO7~vN3(|t@l#GT=lQ>UcD3`n8o;KU?LU}UE9IMOaVCx>k^nFK zzevnJ3qE5cGnxb>X* z1oCYNzi1kD8NR0Oa=j78_w>hAGOisclVrgceQp%9t97xui1599&NB1^*)Y!8Amjb2 z@OA5v7T|vIk{Ca~MCv2BU>q9?6(Ghk69D zyezBj?r;3^=I`4H?3+?>4&#poHI8SAKX?QlWqu2v-O$w*he^4bi60>xGUj4+n5DLi z6-2d~F9d59L_abxE@)sZ{mRZa{8yyi1xi06)3tz96${GEc~+nejIe_ksk7wj_7}-n z_ zTDaP3wwKXaI0#;s|0Hy$TLg*jgE7t?Vu8>Z^_=J00=3T$VBJggCN)|vf#F&UXeH_r zjDd%)1TRSAUILXPf>}R;d|L>*qC9KczGDcs4`l$It+*{|PBN1F$8n{A zODjt|Ip2cp&av#WV0(by)#O!dT}trN-X}Lg(Vw?BgA#uk6L@1cb}#61y`h~2A04Va zYuTA7*8jmC?_=MjjAe^$w`^X+Iu+#nwO^~2)aie*6-Hki{#fzm+rcAD5sqPvVik;cKx~40yBo76qqCMt)ST=jbX6 z(%)mLh~$OI)z+QPF_KSD7pZ^UO?X-o8=fN(xR9&xS`DXbN>dobA4Wg(l1dhe<)r`a zCv|ddE%iJJ*$^!StJ8M7dS7UMm+PK;jvmjlcfyp{&E2SY|JC@6ZToi<1!>$U!UkYO z7neuvvp5r}>eq%DtEBtf%T|mp z2m~BW85e6xav!d!feMCpWB9|VJ&dZtY97nQn!TOf<{lz7TwE&hf$C@%eoX|q9P#_6 zWY`{>BbVR~NZDSgo4Rvot4J$k_akRCG#E2HIsv^YSNo3{N6~wW zG=IRINY4BHVh*CC|+^tGR`?6Q-U>KlHEe7{Rh2 z3~H}atQg*{d93abPMO;J6QA^F#aMOCAQR*-+TuAC?fJE-8!+fiXt(|LWHUs(5wL%P zQ}bO;`E$-(SU79|EVfl_0wSM^=Cg)oJ7Nct4Vn8+X3e=Iiv`0cDHNZ15)8=ojqVx& zv>R#;@DZYhfFXgHiWOR)JduzU@=6^xbuiJdavDUqwLldJ6In*0>F_XNwb1K!!0D^9 zYyK%i256_rngxJ8#}Ly$Rl#E-QVYAx`9^I#C(H5Qnt`Sj&5VW3O>OLhgO{;37uT}f ztq_)T6OrWv9oq&?zhus%<2LV!cILlL|NL9YjnGjH#Am|FdZBe?-h0O(a>pV6Jnpd= zzRdz+9uA*%M!==S0#eu4AlM_vdvo08gCQE9@^9+uFP_2xuWzUVEP3b-T=ts=(5z=S z*aR%a>(Yepj%qBdvnkmvhryi(APPa4%SGB|n^QE3#_L_n$C{s`R`=LaoABHu`hC#+ z_MWAiI{;o`VK*8P8Rh-AtjmbtBWIN3n}0bcVz}al*{=*!WLo#HLnE0@`URgT!t~=) z8hIo3cE*k=?g!Fr(+$hdNr*Ua>zoO{W4J{Dj|}1^F*PZ7Ht4;sCapH`e_Hd_AE>DF z@+_2|wLJu&%^JunmXjW(scDgV4!J8U3<7kHh05Akxc;)j~wK~pj=jw&R) z(X$&!3xL`5&xr4Nupf@)2=6yKEazjCYC)A1+MrX9B_WBbr~5;6Yj)1bcA$~f{bv=obo~XF zbtW|m0ymcW0h%Sz6EgK<8oKM*&?>(DcwsqS)q#;XCwOATq4oLOr=MQ-zlU2N`J;v0 zUuJW8KaM%p=8Va3+LCp2Iu|{)WP(9TdW?Z9V0K(HPH(mGl-0}Rds9NSR?gEu(^gYA zWj9N&geG+z$hA3e8NB%5B7>F3gXEq-Yhx2%gAdQ@pet;5RqpC!()e=4cnDUZPIApc zSr;M#PPc&Wzp1V3z`XR#5zyo3L^5|h^03jXZoIaY$lexgb@T|Eeh(n1)nmOj=AlBY zEC=6}1_4&jcR+SO9VKcW&j@vqojdN;3>X3R?FL?*l~6TZXSkW4g_Hd!6?Bb9aMJLF z&0T{{lh+otYGeuTVSA~tt?sO%Qrpb9`%wh#2VZYW_^QtzXsgOhJ=ToiZTf%rRpS*^ zsf>6L;9Yz1qjVUpemsw90R+(6-xBcNpo^=`5j=&wEOL%^;1TgoGSEzYU44sT*{#D$Mqr+5&#n>}>5tx}!(Y`@#Qc zLE^w`dJM%6G+#cEoCk(cPj-#3EKAD5x1poqmsr3A_{;l@-}86mwm!pjOcp1N<3^}> zFZrj`$NtSkj4o3rM!Y2-L}cTH6m3M7-#ub;qJ=K$FEc-mbS1d30>#SWJ^X!cQAKz7 zdJ8))ASBiwplsq@lAQ>~$WZk%u93^g&!<-B8eub5W9@IaMHLf^RC*q{0ixG@ih;%8 zW}Hy)q!TuK*RRlf+tE3MfpO=pGwcTPB=fiw*2yz;P0!X8FoLY%;zkf#Rs%;O8EPbC+SZhnY2vX9dm4obPD|zwsu5N7G>!2Ln>R|>*=c4ZiW~>${K3!36w4tL42;q?+k_{ zOs6ECs)_CgK*eXCgT^>jYd9eL{mXLQQzdGC4I3IoT0h9OCGrEVnc@w=1Wan_2V2d> z2T)&Z;Q5j|DWIhg2*9RSo0Mfo*cyKIw{hcuTDEX{H1jiqv$t)7F8@S`_4Tkw(R_O| z0DH9*w&O)ag_xxShxJvwWHj8+>NnAz?Fjlqr75|RWMy$=j7-H&4dQKMXBFwa;xU`@#4K!M%MNt|xQT0=e3A_i0Jbv;WJ`-; zUD%noTV6pY6U0}(rNWArbxH0(kGM7aNKv0i=74o>t&HL*3Xgq&6t0BOBB3@)vXyP) z{sepy$+4nX6J1 zsFBO6bo5VT0OHn#hBgbW^6pv_`)OFunRelmu3Yi4ih>B927t0fX7E2Sk3fzE$+ehU zVBPkaBWzo2Yw0DEW_}644dgvRxi{}$`+km2-7kst{YGdMGsev2jC>YD4_?lN-V-mm**{wOL{ZP80igx+5f5}*#5#IYRDtjdZv zEKoR2G=261B-sXoXDe)RXIQd~j`#|=I%w|)eDJ;Q4|p$K4az*+qryamYB!)U|3jPl^21W)cuU_e+p$Ws&8A+ z=RiOUjqI5cBt|t!ZmUSYWtk@QlZPGcKh7&)Rb*|XT43ilT|2O4msyb78*@NVp58k> zPk{f5jkJ=}>~ujOTT6h{qdN_Md(ryB>vipbDPF4W8<}~8O9~rOrpf#s`kO*7{>Wub zpEi$$3uC5wqg&&vFOD|5+Pf{vKO2u|IiomfYXz{`xfId+2vV2zeMcwBb!H( zOUIncXYiU=c+CoON`LBZX#z|}n%@r_nzKB))4a(LX8c#NIuZIOVyF}$!<}KPtcg)` zx9W&)SKhg`zWu|RX8-?aV@L|1(oXDcF9)M;WnCMl9Z)0J%Hl)0c=3kG!KredI<+{C z*=sF#HkKLJCMl=ZT`pSPyq|=LAKC+MnhnYA%P+;S9M-C}YkOmY@HKSkI`UgmL`_@_ z9sKZB1ZM+Tswp`g>Ed=_x%=7m_#E4^3Q#weJaR@U!@RK>p#|5?9uEB$I4)v^SI7b= z{EcE(?>(n}3{(ke1`<^bmP`m5?sYTIrCp5@4HHXG3McIibV!&+d|S#ZM*1#7j@?ZV zX|U4-#6q5$JXGNY%Uk2NY-AFH{{ee`>Wi=0^#qgW0z4#NXvgAL}66^ zXoRnW;)QfZ#~UuqLMpOKw)!!#YkRf00$R5W^n81a6tvO9EOznIcdWWfnpxGI_T|C^5HhmK|C!B?lIy}zFF4&0jK8ididd>-o{usUiWMrPvsyRCP~lTa3*xi2(PVh zvNjX>S@u!%5YLd&yd9~@JxQHT1vhPnRHN) zr6#z0(i$1I0#1kfcqh38#dJ8l8=+J>s|r15v;Q z6v4IE^)yHb2%-o{=UtcT?J!SP*Re@o6* zTm(;LsN8jO*i(1%8QtPBK6xo}+gmnykcKNSdBKF{pL*}IgUZ0j>?q-5_wfuM(ie`9&c`c$$^uKP2< z0KxE@U-XY8BeUK#DbikoEjFgg;o#@RwdmGS0%>+Sx(`GN6=R3Hl}DX0J-c}~yF`fx zj5rQXtFTn45-2x}Y^aCti(ou>PSu{QNB5G5N$zS~R0 zPNbC@QV^zNJ)?uGB}K)1*viCmDKQH?#RsC#eW@E|cYeam0b#>>G;>ekWooZYfD2!U z&@6tR|1tRqdLcT-05ZSslFonT_igpS=D`K4N8I7p621*B^KFvG3GuzIt;{5O_={~0 z39^uy*al<8k9M!pAU9+t(DjX(qL<$KP9W3;j%Wrs%4OF_|ccA z-E6(T2=N`hhd}E~UN6rO_ZdzEhK)V%>k5K4?w1lcq?WGlL=6R*-`Nim?14=f0sSlD zZ0d-Lu`rENMGQJt_NLOUHAFa>|MAYS^&>9QSw9Rhi;c&Eyn)mM20z89^QPL+{#PdN zKJY-@qI}~5CYZ&x3O9P*tlvQgyW#@sflE(xPKxMBhh1b0_b~Hg<miNzqxs>;^;uLDLXUyfl)#BNtIc~Ct=P)3(Q{inid z-TnJu3jE$j zw$C4ad~m}h&d$%0gL$pv;=C2C_-^5jk#i!YHHa$ULFK+5GQ2NSxCq)6Chlv*#VnAy zV&~s_-J~)l8zfk_e^)RIHu5a|b=w5Szem3RJE6@_pe9%{1-!^+_Fka$u=M(}y(k`lik-B_O@zw1l$ zJ0f$iIT%%NOfP|WaNAOyfJpaXTf&Jcknpa}5vKJD8w?Hvz<_SNJ zJ!LB@@6Q{=^!xt2%*RsZsO#?SO+Wu^VDZQa&Suy35akYBGPapZ-Rlzof2!~zlvrZ; zDGtLVi|ee-uv|<1UIO)@FvmKClI4?F`>G7VT(%A&p<)4S8(0!i&2NkZg{H*Pksm5S z(jdYr;*x2_d`iLuuo-*Txv8lSii#YN=r64uK&Lm|jQVr_u7uWvg{9gXBq{9Dz72FY zY~4wkXY%YS{G#qA{)h;{w!9+TBb+1KB&ffxV+rbU7&$l+q;>Z_n<1UQVUa#gi$pRU zf{0c)$#+R6y>UH`+-j1%dP0|syDewk45V~Vkq==@>{NM!t_%V*knE#K(pS(WDk`ZK zQl^!+vS)W7E}%zUAN_4{wP-nj*)AgO4@!9sNx};VnD>ulHol7z0@Huh>Jk$8;I8bI zzSr|Y+lI1N9^KT;3^Sy^F}FATqob%TBUgSx{DFY0snVE%!HNmF(+%5BQGY zRkS%iL@C|bw1EF-SZR=0V0}1Ja9d{m88-eQ9(}v9ClRW`u*jnhyis_Q^j7dPwlk@| zSM(Ae`7`=cxb#c-zsKcyHIH{5<8@kptaJu`92^M>C}D2RJx!{f-nLRsSufIl;u0m# zahI8nnZwtHmw1xcGGP@WtS1z}tNEqk+-h=gxqwjcr`aP3S0W>N)+dLj3Kt`|7IhzBf#obnQ`;>k9$1P0%jLZhE695^-6U zxL~6mTC2S-=yo?x<`P~!J*vOk#oZ&cBih}OjA6C z22FD-y)R;$o=&~>od`jIl7!-Zn$}Ueg!R=li%bwJoS%AcxHT%}T`|_1FXTca7iouo z9!E7n$=!_iKZvA^RQU>qrq+hG+Clm>|63#N63htpVPTGV|H%vgW30LT?;F6w?0R;ea3;Db#l^Y0$I|ltAa-_tPH4hTBsVE`?2relU_c zBZ6e4L=Ky@_*A2!{gB#F1F@@q1u@0@AKCC--h%R^|)RE6iSl*k$+*$ZLuZ#4?$HJHs7m2FM*D zqapTB#GXly2sT59+Gw>3EI+deb+Hdjb#40{8%g*BKI%vxYLc} z^&T-yfVdV?~|~>YWO!qpnAW#Y)6i zg`4QF9v?#LRWg)JzE_ZH+Mu8GcHjCQ^3nalP`(wb2YcW>85a^)qmOD9$;^JRba(q= zah3EAZ*H`#(Ze~e%%6On+%*!GJDWDyQp%0?so$ieG9o5$EX2Ju->AT_d{`-CgDKe8 zM6D^~{zb2@C0H}?qB>8e7hzfZk_QPsrRx~a1?jekbX9mht%33w<6oV8DYMN7qdc9j!Ml)>rMwl(68FD5|J;CNrjS#cvMtOd4Hv zEUlD>N$WgSo7VJu%!~OF*c=oH^Ka5ELo1BvVda}?bWo`kLR}KTyFo=^dQ3l!{qNrU z@3sInwACD@9cT`7FGeD#_T(MNdsPpAd%tYLO!4%MZnF!zj67Y*VcL-Ho9!m&*Sl`d zBLH}QEy{CbC1ltBK$&SAhz}`YwfgJUF1dPGboGh?2IO=!afI*xefd)Hpa7XC4CdBRjT;d#TrA) zO7YIfJs1}9>e$CqfYR_pCLH0nrlO$fw>P}cYpB4P*a%Cx#g}~wc317dDZZAE z=Y}@QB9Ij$q=cy1_k(`Xw>o}WrAbW8WrZtN-2v}zS1Iwqg}e=Zgzr5pj8q#Lj!=Vf z(Z-Qr{>e&3QRcG?NJ+^VVAmcY$`@Hq7owSG2=ocP6Xj0wZx$j@ZpHv)3}2OB?A?wu z4>7PIEesnm7}ivO@2<)GD;t`8UuN`2-is_g>$WFPN2h_DuE(v$`|Wwc`_Qz7jfwm?alk9JmehpPXlBf9w}#z^N;j&PeVMFhI<*Sn9rKm8R( zj%lvx7~#I?aw@n#-ieOFe5YlmW z=9WV1L^XNFATT5`#czQLnNRhpH;)%unh8R-ZA|KRH&CRYsoYt5VS4Xwj%jT)E*6O# z0OSPU5N&^J32gMjM>H;gHQ%dVhqNlTIDAwhxhGgo@&ke+mPcYK>D!q4-xmDuc8+lk z&1w*l6x^-`qYPtmU{-z3AJ_WN##YClFSqKYU#HNp{%uM;>E;)PEc%#&*aB_(9ohx{ zFPfM6#GTB#uK(k!Q%+3XqbL4araGQL}7@Z_f}r~ zOU-~v0%-qUli%U%D1W8HIiSs97#&aB1I~_Y*A2J>=>U?WIm2;Y7|!!C1{pdqR#ZoK z4x+VS3oaFsAD?$?Wnr)$rpqg&_-Z*2#+i>A+h&D?{mXbbqm2b(NWzZ`B|rJCE13(A z^gb5+mYHFEVKaW{{(j`}Npm18p{x;W-bvKXOc^){te)!YTkLS=OcX5hxyj&YtHpkr zbVM!}F(ubip!wbHfx5KXOfHn7?o6fL`UoI<>EWZLqemTeW!S|LaaXwxHWwv^1kE$T zDgD#;h@E#0Dw8_DogU9>pULkX{^)r7`SytN6QA3vess%v-(N|+EJ1%+WsU|tb54|l zwRJ2+=)ekxZ-~RZWnZ_s6Zz_&Zw3X+=P92@^S-Uiu*Y6Bly#YQIRG}HiKi{S>>m9GhuZWAl!*{KR36^@P1c>- zaQI2mmuiB+igk~DCwPd<6%T5%5VeXM*8L54;3bXmaT$US9zX_?o>I%Tew zl$4~*x2OcTO6#Is_)ofT*shI)S1!N4ue6Q+B;f1tmV>3fkMdll(N`5CUoZ3pey+rm zp^_*df3^0rFgtARrrc50AlikcM^hIsL9$1;x|c*c`|y?K$o+!E2*%Q$jR5LEDbIKp zP_DTH47B%XiH$LSryAnUzT$ax7PAB|H3P9&+&a=R_CeKm>jgLNJdwRO)5 zLX`2v;si~gvDF>XIINZA^9i^#GZ!`roVtIev6I5Hn2N5LE-EO}MJl)mqP8L+$ zS}j+ZqZZ(x_Qu<-uAD^jgxp}hY#RLfB}LBZD3%6C9Hx)CE-PgC1**SX9Xo+Fja51I zSGX%>(w##-ELYy`yHoXg-Z zh9O)YZMZ2z2P7LLzk+MMLB)7F#9Vo!U!w8rG&t;jw#1k2ipvmYs$3jyE}iRK^-IMa zrgZZ~ayl-NxY1chLZhzdEChAN9ny*$!gYV63onNI)1FqTePdI%?kOXh;)SHP)_Z8J zuaCw!E~bwLR5^7EN(Ov>T3BXlzInquwwjnx$s+SM+t9nvRG02~z7M;nA%SGBl?kck zQ*z#0$O)leu&HRaOK{ri`@rEs#??o$<_YI~x&6jZbII_k;T7Xe{d6LV6XgplR4l>_ zU?5RQT62~~_|j-8^Kt{JVa|{@+`OGe>n#P5FB)?L+jO$veo4!3fsZ00DH19tgq?{Z zHkTebv1lU618^itX%e0UjW?W5KU96NCEnF$+`Cz5_t9L7n8bl3x{QiRbU@i9C7Vie zOE-d4WgU5PN_x91ruo?QWc(I!dEmIo1eVvk-Q!WDj0A}(@ps?V;og3XnZ3{y()KJ; zOmX(QYT9$6B6zkrK(KmfEk&Io;nH;gF1$*V5!B;VkvUro3AGS`NjVAh>sy#U5-JAu zTsjaNb|tdw3x9U&f|K=!6OYhUjb#0?mV@^WVjS1sLaKbvyn9}HMQJFZ3F-yC1v%8$ zpdKI{a6#+bd{6oV#<_u4B7#zPCo>y~D)ci6U&8l3}e;RY{|Mk*)57YX0l__PYWpp)*@$2?9 zj~!Gn26lpPE0hzd#QT~@x=ArT*tMjPGk+#;JSbvd9@MnCLfup$=S{)%eTZv+q%U(v zf{C=u?#$?Aw5@v=F<0o)eB_uqBlALC%&W#63gDS+q_7`fNRRL%Q($v{?6~+L?d%0? z#7&(=`YoRWrtwRkyx`m43|zQJBM&L^L)@A(zk+;7w47>WL4wYZI8YXe67x7Lhj$k; z7Np)wd6=Bq+TG&HC8SBuO!Cq;1v>6u*7P{Vt>)KG;do`MxQ)yAhLg-KhMQ)#k-yhO- zJIR3-D9%Dp9=SY)zyfKw#oRX!2G)hx!tzbKVO<{glLd->nojZ(Lpr%A^`Yc3-Ghe$ zlFybRu))E_n!aSX#h#_^cOiLEYtO?f#d(t_Rqqn3O&_v$x36~gADe2ok!p1j#*n+_ z^oJb(14>nYUDCCja<|0BcoD6EKnH$k5vaTlF*PVvEVXNu^K{^eglPX|dq zc|H4cREh|ak66_`>fRibgzD6p?(wn^hF)~rDU$P*Y*#^WhV|!gHD;3pccHiKa=vyw zcDr9-;pM^uJ%dGSCG!ahv0NA|krj*jVXM&0;(BjroS7&4rP9?A7_toD5Nf^C@ z$vDeJoFNgoM>3Y7b|YqSxH~&y-!Z$KIOQoB`bYw(RkT|5$Ti}b)aWM_dmd2V_m=c|ydf#6qpmG_U z+dg3ydTI?Za`xA|>Py|>E)>p53}By|_!hX*DHO)Xc!c~6l7)*;D=rHe2^}sc{`>Tr z%V9XOL+YSzz0$KQY8L-S6Q@o;K}Knrz9f8a#)A zm>*_iXIjOb+eOem)nx+@tbd7lH*9eV5$N{`1~CXyd`ywx4HrZX?F{WL9VFiWH6Qhq zS`+SyVHc?9aKj17^O18#(?)?n%D(%&pZ1vPw;;AW8VMt}rBl3~e^S zIRy?^yVV?FYUV7i(pCr--Ac@*Rh6>v#6EskZrY+_s1%DM?XV;g8m^n~RfV8eW!U5{ ze8h%7IXQ=d2J_j07b-Vw5LZ8)<8hnEe4*|VoywF-v>SrQd-#jEluPRmZwN(SLanSL zyq8J8{{YNfU>)$(&HS~FCQZ>@f0@x_c1Adf-^3T-$z1vlwyfnE9AeR{ z-qRcM_Z|=sFs`dSfBJSib*0Yy`#R^I!QoH(M6(;YiK(chX*XxQC2GjwxmE!4v-h&- zM5X2+mh@-V*;3FH8+GP7^B>h5da<^EN-rq(o_%%A)c{R!MBB)NrT+Kz>?(;~B%FyK zgU2hx?L#ufO;o?7h{5da&&5sCpqNPpYqECP`-)RHv%`nr(Ei%P@~nWLEkc77)=-?8|8MI&Tg<2{W}5M zzb2YOaPcSYGjpB6j;6(NRV-63-Ak2~7f%6c2Svyz>ROj~>;H9m^cMi1a*2o9nv9CZ z5l(aXoV}rYLZVJwMD`c6>W~y{EeL8^RUh4<$RbfD{!PUMtD*}KTWU825hdOtj>gh3 zgR<9tCShbi7%WyA zM&2Qk)VuNSUSg6=P?N@!_%vj7b@eY>DC7q;72?#fq$l7aO(xx`%J4i>gJVLE!lKmd zeC+DW-?=KjiwEXy^_y(lq%8hZz#}-)Ez`bs%#}T}9Qs)Rzi?Ot| z2_1ST&9anw-VggzEu3P_TY@sD-pM@ph_ETI>57h&UEFG9> zeR*WUjS3>*uzE_C_23QgmYxF}x|B0HCRy{?*}i&kdX z9VMr7yItlw^}MqPU!BKrf_@v|P+zvW_Nctom^KbUX>h!le91+Ls`Q2?mJSsm7e}4Y z#i+Hf>7FzhIb{PoD}*`f@cci3Ft0t9@i?s7cU$2S9mEqb8}yBjNr$eI4M%KAGy?2* zn-;^rx=9o^;>Y;@WQlqg7Nqe>BtO}m++#(TR;sYazfs>~DFyD0XcnQQZb5Gv-FOU^ zgMk4`>aS>AQ!WA$3T+?-ba@;yCYXMc^~jnSFQ7;N;i^4PI zbHe5z(r3?}EeZ$`QWD0UeB00xO&ua z>lGJbLQ-*@rFKucZGnODBu}$!z^D+D*-K&mjcZ-^zWtDj?R1`c*4?h1jKXNR8>OvL z)oiSrvKmfVnQ$dEu=U3n9riOdXo>fbT{5tQ^)L<>Ct2!Kzu2jn%&~<|85QYY5vx> zIx=&lhq1PR>*i$pZ-$d|(rz0q`^h6_q!SQJN=qQ|L|1! zRn%d>Re9zabP2in^PVw|dS>%fIBk?uezTtzIM5q6+MI5)WZ8@u^obsqEsV?Z4RA`6 z`}DT!BMSJ(z#fzV(epf6T3KFs9_^By^!r`M|GFt}tdoVuixKx?9M~jS z2FV86{kOr!A{F6nf2JcBR!9`&(v3ZiKZRG<)5jb)uHl`Hi1FmQT{iQ>9$x~?R`N?I zr9X+?WWn_C)p5I+j0NX8MnQ}h=|pa7oH>4JCzmR-9+T9OECUJF3{bnZ`}vG68!$CE zF6Rc?&Hifob?}$Mb8!Lc+<3VwCryBRP%>6AaX)dpg`PujPdW1~_2PH+v0VD!tjT_5G~E z>|wT5Ci{D0#zbFY>oOxROl3~n7$Q*v`en5*QFE*Vy&7z(8%Xxe#@?~qurHjA|Gb~~ zHlVSW_JDiiWij;S_Djhso+_T$&XZ8*Fe;YbGR=Z&femlhaG6)T8`?s_C*>6Wenwg^6hv)WH}B$408t={Xkn~j$8 zx~wS~)8jcb=6io{03VbLLQOvnSm1$Y>F?6o%RzaErmX(J*M%huC>gLVQ<>##wJ@7P z_muDH_(qNK{IKIfrLAs?pYPJd75BQp=G54_v5YXeXBqb-r_fkcgu0>GK_TP9NNMac zPxs~=fzJCy-3vSflU0}&xXQOW!&SN6=s1!pi;oy9eeN~L@{@-=kiqiXUcqE~8_`O9 zK*_JyE+Hheaz9>tETJHf3H+0dLyC{-Z>iP@3nmh3N?Zn{%9$e8YXpxn3FjqnUOh9h zNhluS^EyWEjJI>x9IdZt0r`hcB(98qXacUoK8D49s1qpuAa4WZ%gRByJYo z*K?XhdmUbq>6TIR6SIL=L4I z2h?3)$L1meHL%TVzXhHA?w-tg%{$N<;1-xqpl(Y>5>vFDRc7r&T_|QYihoU%lAjma06bfAUV z6T-=C)Bjsp~0ECyh|_U%d39LoP!W<4^bYMSJFEqlEaW21hg5g z6N#6acb!jS3M%CkO2(6e_j`lv?w%Zs z8ym}9kj{zk)(&y@-BS@D9+>4s__-b|gtIvQ^6Q@4tV!iNk~#c<=&5sJwLUUC)D5WW z3ygpr&3e^Taj{_g$XJ#vG<8y!UUwguk%Sw=%~q|L8#mvY)maI~ zOufV8VL~?VzU#_E48mZq#WNWY_oY%8Ns~?$WdqKSH;T%drv3@}f5{W5H4OxqNF4p*sR?`b_JS_7dJwvWG9a48ZH-d&~QWXVmUvN`>S2Yaa}$2VKp_5zuYw9 z8C+-&%t;e~aBa(ePk!-fUrU`u(38I2!|j@RxT-Qw%zLY#Iy|d~>X#pm$2kusD=V@+ zgkIWD4ZpT$Yb0=fN?p|0@pp^Cw6%^ihCI>5!r)2s;+rq3Oj#v#sw^gTO-ZNAs-@IJ z0ep;2wI|-V<=#KQ0nb<{JB^26cG*s=ir5+`7gp-IRx2twbKY4nN94dIrZ;Gc`0KvU z`&IU-3_a!EQ~@e+MoTLZ<;%}eZGTJEvcz3BCgPM?U6Yf8-G&I9p6S%hfNJGrVyS-# zEyX@DXKG$^HRj!{PPQy|##*#+5@B8z143U_{_Q>)=bc?={qj;O50R$EZx$?d|5wzN z1|)%X?di4jD;3Kc!9bi`Qd2Q91#_FSuv}B;RVPzYN;4HiB3CSPOijStGq));$J`># zHPJ@gx2e!7&O7z@=4wpS{URf!fOQ_bP55u92fL+WXFS4(nyni?Y=tycWLiDyzN zDa+zl8AkXOdfheWzl8QNK zx*8n8cNa9g)u(&wOe)Vi97@<q%@^rG(2@7)_dZa5d=4;X17oE zWVL(#Z_f;gN(Z{0!h8*wJv}4((W$iA^4i74?PD>_cOHPg)lGIOsP8s}NjTKHGANcs zxQWO|V6ccCpsWbN7T&Kn>=Yz`rG20xz0}LO-!sha&B+&7UPBy4ChFVnX0_F}utp!c zm7PHO;&?cC0qX++ao;zTeX>qrwSL=+jy|^7%wr~>mkbtfEd79$491$>RY^ocy;|3L z^~b|K$>FISq(5QK9I^2hvgR!-C>D7Z#O8tp&Dl1mCCHPA_3{DlHMoi&&)I2Cruk9< z<{rtnx>62b$>~lMR)WPHYe!^DHqzo7S_sD15X|D`);&qk_MlzJ9#fbF7V8$e8819C zlFv|Qcwj8RJyVy|2p3c<^IW7;ANwnb@kVqZLfWTqYa|cHh@`d0=4t4t58;U)W7|&t^_#{>+|+ojb*^ClfCj9{?>VjZwbU>RBQ)F8 zy3#OF&L`7!#iym(R_7fU>+j96u^+md3EO>%zsRZQoksjTe+M}ca&$&IU-nbScj;4s z`<;`!O)&pOr=*D^4*lueVGjv81ld}zVaB(5amxCs>3iC!0?=ab z$5{jFR)!Zf-3^9kXNf)m0;CLJ`iY;L?YKs&GO4dx;@W!)O&XjOY9^B2R048|a1*ME zQ^ob!q&&(8xYoqEkJF&Q91>Hv<6NTXLsh12L^|gY*wwx!Ro;$v&0{oS`+^VtF=E#?P8Q)`!gKV~UL~S@C=t@s<>E zQu3KURcS~GsFlc`hLoQ>eQa#{Nto+UddR#0zR_dWjP)nTgyY7Gar1(7@fp-u_e7H( zzBZ`weD(4~qmFD~Fb%Sll^13})x^AT!dnc#=R+yf|2$$bsUknggMhs`86y%a+OX%# zn!Gy%-(v$E_-~KgBdfP~ts;U3z$6sZ@1y^!18%CSrWGc*gc#uqHvDKAH3x?pMk}xh_2e zN0Os!u}KH}g}VbGVT1u@rVR1TFphR#;x`tz9=eYIeezRk%;m=XvvkpqDbNBjlL$ZK;p z6(s(JmsnZ3M`w=L+$*667Zfq(>Ur&jtm9SLdA~w-Bp1UDYuZYFEOmmos(~J#)8ez> zIWeJHIx%Hq#}CLBU|uhWgs#J(5mA_Z@|`x{<_tR@7O#s}qP-xTPR9ZbpmM7Tmu1&l zm*EVH+{(Mf>Bsv%ZwpDwsAIT!Oq2USpQ<697rk~2s8-1)Dq07uwvHwm(yx2nf%jt2 zY#N9Uo#H{KYV|1xtTQ2UA^72=UhPr^3jM%W;;J9DpF|yWx44&*Ut0#obe1}fE5LSN zD(#*ujVU`Cz~z-wwm=Y04Q5}E6ZsQ^r7Nj}F%tUdS6c+5HZb8hQTJm_A~SBE_x0+F`UIN~}wcC$>kB9zJPwVtnE7Y;Sh9d;qd8+LzKm^CHJUOZxniNUctZf+>7mf6N4imjG#2{~zveXiop0dFH6T zQ?6LKi*g|_ra7PG^Rr)4Hjh85+3E@-2`PC!6f0xd1=XT+tJL$koZyRZ@Pc~L(F-V@@LyB9>?2sPQ!+X_>#nxX#Xsp4!`e z8FZ#fN^1Yz@x4C)m8|yGw((ZE5m)1{$N#DtvCsH5mgm!U1#RP_>Zuw*KQrkFcINk% z${4wy9KG!!3-ytO&0CvWyj!n6QYkI8D-`mrWH|3uXQ*AN>WJ3bScz_SVpd#MoZHPC zyDZ;b*3|ul-tgs^++1b&WPoqBo2BB|eLza`OYsX+)|2*t-`{>cPWj6jTyqfWL z?b*g@e~iHlS05n`2T%^=UgFvXOStmki$L^=b*gS%7ziB%Uwj->_5g?vNT7`Y<~Rnk3=8s$&*TB)$1a5`0O;BQ zYC9;}OebDFNH z<$v{NuSWLoDn?K-1cKw?yIQK34{6nAAJfcHwB3Vw^LqK=6_7zP|UStCmIvAJj zI$NZrWO&DUx37+5$d1}ec1M{22;Jzsy(0nfQ4q%8UTg1D08l^UoXguUahMC?!GOOr z@})Vgc2v>ybmu`TjqCy?3A6GroemBVUU~#M0e*E@wcL!E%~Jl{>zEur6p`V~W%~Qv zF*_E3UJ?JLbT8$cRtKH6=?RHwAS*F)3H+)?ckI+XY3r1vnE#RCx2V7AP53n3 zjvhe3!(IdEW3526e1LlF+@w0&--1`LMK4bM&~zd^s5O_XeuqG&Murv|f8*!hpfv!> zbV^Ab;~V)8)nR_^It#q(o8Yyd^Q!^~??JkCo4phZ!F~VM6LoG&h;KiT>QfD;09K_x PNY|B1P8X{#+<5pu<|jwZ literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_7.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_7.png new file mode 100644 index 0000000000000000000000000000000000000000..a7487476746c6a8b2f6d023b3e9a2fa517aa7471 GIT binary patch literal 100338 zcmZU5dpy(c|92(HM-j>^FtPT}Wn6a@33rzErh5wop3k)ONT(e%KbxQ+7M_aLezoY^NS_ z@DNG-sNRHVMD6|kqP9=%ufKhQhli~mz7G3q4s!p#f1Z-%Y6nHP}XZ>H?_WR7Wp>0)ySi_U)92guql z;TmDs8pcz8y`l6k>Ud!4<4=7bw1>z=WnQ}vMCImx4=+s3Q&tp9_G5zb}%>G`m_$ z{=viMcgkjz1hm(alV-$Rl%I>1xU19^6b5W@QG#!nQd!ZUwsV=J<)=h+0be+qAV86|eQnB{TK%9Scm)JNC%gMJGedKaQPR$C$r>)J$3Z+nzU zZ2Q>&jGJr7({~7WgCsbcEj}@{2#t}DYYt_|#Oa;~(4D@GMR~NEgfe&EK_gG zyph}@-ln4XEMS#}?4n3&n**F=p`VK33+JPhdIaX_z@5^;ef=cGUf_If3@~O>5T!y4 zTGlqu@v6{2XA%AEQ3=E946Kwd3FyAw6d98oo2BRSPNV@5*G*5R%5JiqRUv* zKRfaYnHw%?kb=T`+y(7`hON_pl7)RjNctBGw3&ArW@11**ub}#gY{8m!FB4o(+42W z&achewJeJcOI$PwU7E}%+BE?8w+A1XlssZe-tdTS0szpHuExCT#$l-Su@Jr05|R+( zTDe_VFFtfNefRc)h5F`?bB28U>wN_842xku`_L-2Y6G#d*jE{K*eD8&_{XsXV0(B(#DNX7q z$4AAMX?Q)|forKdtpVlRxTL-(0*}FX3BJ_t(f=MOOfb@>P(Gha^NcTZcwEs&jXB7H z+GCR!3Q%R-+pbOLR_0VJyeHb!f%=ot1_CxVp$JRMzFSoom!wJgV1mCE`@_xtbmzup;V3HBx=#pa$a-z9rx z$^UY4*HfnC2zF-*PF+j-eag;(3NV8jx1r$k(1W$q!zA$uNO|}%77Cu$IHCE-Oqc30lB3G`q6Fg+-4RZbfgm{ji+6LP_O6jo9yLd$F zjOlfX^nRrRC29QAOjuim;v_8PZm8;58I)786J@k9b#mL`Zxjrcs=ZitJaCWCG#Gwm zKv^8e7R;XmbNHOsv{~4p#SM~TOd;G?FD5G9LGxly?TdrodaB<#vI@>CqPyaVBIjna5YBbO8y$TL`caWYFn;Cl`Ik*=!+>+=l2 zBWc)X?{M$SxyWK39RPQzas>75m5)WHL;dE>T9?YbMmCxPa6ga?k6z%`hOAXVBvf1u z3H{=e0Httj2cBid`TEbSzEH!$=7K+L?kH2=21r1Z(#!M5vT z^CWHR>l$#hr(!&`tNjtkf@*aH7wUrfbb{4id!+H5M$dvm3D=27`2{ht39I z7JoYLo@s@HsGA>lMSSNuPmGlRPKuWlRf?~aKR1u)3A?&8DWaEDd>r3`Wzq=++dRG|vi~8ss-YMPO{kwYdLo}4asC22w{p+gLny-y#7@9Dxa^a}v5FdIo7KpZr z77ZH+BEU28QHI~kj^A*GF3oe95q_zu6ZqNM@Wgv5s{C3cYnjQ+_qM-(!yW0 zdy>{Nw7ymDuHQwV@f%huv@pwEm!7m-c@M(a8C@m)0;d%#{crTHpoB)X z!AE+29yzPHBWU=yh7TD!Kz%S4^1Z>79FclfuKCf@ zS4zWpSXYjzJH2wylYYL%%6#~D!vj)b25;KG`4Ew5l^$z91JvDnIrWu9td6xyr0X|7 z;6+wEKK1%K3K-XQ5@FjxC8o;n(?Y8P zeN{u1b%D{@N%sO@g#VT-BN=TO)OP8SV<>rNZ=@TVkngB`#n{&xOLR%{o0VGXd%K~{ zDv>5_p1*0qvqexw!!y9E%MJd9;fs-(GTE?M)+!LgnU1l8Ct5O;+7(&J2&w|UK`WQk zGw!vc2|qLx+;ogSk zBSGiwN#39TJDy{`e@cHz`t#~LIP2dL&%@Khi$P2pnLJW``0ps79Tkuze)j%#i$hlJ ze4o2UfpYHue0tvNGpW0aa@kJ~%9Coi=tX=>BRwWEF^^5<*oY<&in2Ovei!)OqtWo= zW8=gy>~5MPBcr=Op5oOiKO`u{GB2bG&E0ZBBNw=Xs*$HW@{K;X(;QjG3v+xLs7%*R zTVkz5S!)E|06bjZV8=)#r}Kp6TohLr+Ij2}5NJNoY2zisU~<*X<)jr~r=9d(7t-fQV9R?*TVy{DY* zl^Dz|L4EgExDDC3*hvvYbCkSnq#=?x_2w-)^i`;#cRZ<9f`I#RLBLHZkJQbVQiU~~ zv;y}NmeWoUWTqRM0w{Y{C!hMt#y#_6CuH^SUo_2~FY=4bJcVw4_xCOe`5SnoKl1FU z7~E`g4c%ZTCBy-#h%>`Begi?ID|*nBOw#jQWNHDVksoOhISoU~jC(I>qRn=Kuq?-x zr^31$_rsCvaT7h+!d(Ng|DG_&Jw~*R{;y?llT28dXDBL~f2}XZP{cqd7eDWAk?T#H zVXBu0^{Pe2wLE^Bis>6YnN%p+wper;UzktvqL(*gs&g6!NCjUbd#ES=a-*ucVLc@9Y%T(F8FF7g zZ;$NYCu7J_4z2FFRpwag6wRsE3gkvNzu#<13ssOp7nj`0-%ZDC&LU9*=Lw~ZjaiLw<-Zk_#H1ury7p22aYGSYY3u~+mZ7kVzo?Uv zSfkej2x|A*wb2UmkTY7oiPxXe<}+uSuQuqt1e7$4%BSkVp?&8r&nWO zSDKG^+gOnOZRTHTh9hfOv*BU>?>C9!gyH&RkKdAUsh4H;8`U!t;ix5bt>VCccZd(< zUR?3lpKU*e3ft-*-aFeJgY3r^MLt|mPa&Ebj#%W6Nn(k9>G5lW z{ec64kbTNQ1kb+uH#u~qrpy3^uX3<5YSS{&n`#9YlkX>(TL|g7w|j>YaMJ9cpG5*l zoW*PeW>*Dg(|?eTY+Y5DR$DO2A0x_UP*;ns;IE0)Dd310GIU{MS6MAwf*1M1HMetY z3a5tH?-cfn&dr097UC%Jt1joMD6P=2<-9Xf!1^wP(Rrjxs6P^U+V`c9c1C#@P{`(w z7-h~_H9XD2GXwZ$@4k&JWnE?ZC?1&8&o||`XJ6HHE7|XjHQ(rsU)ZVCerO5+Tsz%< zPBSp=6cO%zo34}<`WuG2@_9vXsBbZf<72})M^N3M68WNZX+tHqz^)Fda#T~^u@*sV z1aQOzrq;TbXq-9=nFQ#@Gh(O+PBU`2(S;jS|H!fgV0TIs5Ge$6|J)(w{L~6Gg$)}5 zAWz}W5+Tu-MG4sHu*p@n#n)K79k zykVIbUYI-{rM7@OZ_wpB39;+Rwc@-+*hL{GXkWoZl=*!SAdZEJ9 zb-{26mrb7C5UsIv>YSZ06}-HW99!!4>(F6#g$dPze4o|M-w^G zrMEEp&|kfOG*aC4UuKigEj?)tJG>HlI=TEPhUnPdGZ#YMK`_yDI{yE0LAzQBQ}-8h zpkwHy&ch6Z>1o};$Z102)6hk(jd@GQJ4TTd*%QHVYcbvTVp|!(gSD(8ocl7*`7%VP zJL}HTEG3RU4fL_T7gI7=o~1SN})iY|7KTD+NZHLa%L(~Dt|yq13691QE6z>hx>pQcLjBXfYRPB8rSV%fNF3u5xS zSfSS3FTX}MdB2j%9~@6!_N2j43)uNCuJAj9XShSJ*3LrE(TOzwi(xwtQ==y$R6m-8 zVH)+-3X!gWRgyhExnZjoL9F~ z|2LdTvHobRn{6LrO&gU7yc^2I|IALK80qD1SFXbZ43~?GCD?UOIaNmQ#}9aB09vfT za3+J{6Ad7!!>l!kJ96lif^P4wpeL!(g>7UFOk#FfdUX*9iD|enhdc(&Lm#}(R55Fx z8&RtlY>&vhJG`m;uw}%tY;-roV}iL&<8BR4)(3ecAFO9uAP$u8ToNej9i;G)q*o!{ z=-e!^MG|xAL2ylNOuE*TxbT$L)`zfADE3@wo`N%UnpbN`A^JYjFhp84Txt608<0v< z%W%Dst%@+9mP}+aK10VZ8R$TCed87+^Ur32Bo_%O3UQp{vVEhcOndrmQdzi5G|wg`&7Ahd1Y}KU8xm=^38l|et~NmqRsuhM zIC|C2APzn^rM@@w@!* zo^HLxGdK_{{%ttFy{YwWF8&zU&gh(D?xrD+PBR-wUS`OCNE!S*vQ3*pUINr?Yg`@C znC`RFC@E%Tbe%SJk*BYnQLW3s)rm{~Aea)jr6y^#&e5S_Dg)Ja#< zn}-}sdmxT@`Q(onC4rWBeEtmzJ1qiMrB;-dc!vTbsu%kRvx_-rbklyv8K5B zdQigRvUu6BtE8i3tww7+oX+!GrbWvA0U?x_NdFat^5cWK_W@u`iT2Vv_s7mU4m&Vk zDp9tNF+oP?=_G2T*D39)4a!Zf3YpZ@JG8!aqzq+K`RJdCsr(MLv@& zA<=gZ#9SUt!;{J!>%I4LE#Uc5dWfEA>aEv5)IL8`f$*+In!?I6!vIQEd;uILhIMv2NnS^BGqYcw zyUkE4duF@Op=kYB?5zy83wq{x(o6Y<&e0kvaZb!A1lf5THv00%P=3-yAK4(hf`tKW8}UJ3~xJ2`!kLheUj zd~Iv~qh`YZlKdi+z**fLvRb-baa#o}UkeSMp8G-ka|%6D*We}oh_#fEwE9PV?k#|d z$psfAjN6F&WX(Z$Cg2oSdMGPA&tpJ~>>qsfhjziwxobPkuOZdN?92#J3(#Y*?}`3i z0q8@YAPZm*T^fhstGs_`?0FsfP4yC#qUchA84&+BT@hNuviKEt^+MtQGAopq3pv^U zR%Fi&S8n!K^g+GSD05m#IMrpnUE(5pj#e_x@Y;>~WwV(pyOyDxZh25s#LUpb+NumS z9t}PKVxev@A63N}Rz>oL*zfslsQnfPYVA%01zFLvd$iDW zyby|>AU?Aew$+)#>wg<*`}u!^#=>%;JjKDBk#rmx!rT;;#A;5>1z+n+d~7~A z3kWVmfjjC61PiJ{)Y`WSaEY zVMJn?^f<}&b1~J`E<9jaVBYD3+}wOo2djcM%fVcyj3TQsH_s!Bo>~ayBuz8EokKF~ zDoJ39)<+fDUwQt9DH3UXAnwbx{(|*(!$4i*$%5LSk!8Y-1}|xN3HRUsLyjdkM@zJ0 zvMQzU5EQNRl5X`0WvYm%Q>?a>s75Fp8mld;XeHAzn@)?Flda|DSp*g7nWkqmx^Nq5 z=IFoSoe0dL4^Vs0ZHc~2VSxR1MK+N>&(XA&;gf2TCL!DmhfXkh=*%i(H*3K3gxBi{ zu9nuVrt_1Us*^PLMe5bHhiiSaUy<=fV*qWYveDiVxjtko9zft#AH&$iRc8<-k@q}` zslI%xf3Su3H{vFG&{{*D@5SBDAKMmcOB4LKT2|6jR z>Ad{b&sf--F^@wEO-gepFzvKet`LDCa@QP?NAcQ+cGm2|cR%@
8cY|}iSIh)yX-4ig3dy2W0lLhnYDKW>-?aWosw!c`1S4 zkn_Z#8O*Bj$90Dg&g8imj~Ko4f!rHfH`JZAR2HN=^WytG(3h?@kd@JIMgckCoh8pf z0CX+Fvz5@cP5YCEb2`|js(n*ySD8`js2FKwk)+TtdBk0-WJ#pp$jgXc!@dtKt9!f> z)GqTki@iy&yR@>KQ{F6ndhyOWl*V&fiCfU1@-_XcTlGl$48D(?up8>Rhq~0{p9r^B zFZ{WFX>K(T&$_9|uM2-s<3i7I%vQVN+su9VcWm;I{{x3@3184VS3YaAU{@M1b;OyF zx1V|_S*K^CI&a+VcP`sd#nlXczc%#&gNj9mxXWOb>9?v18mCcls+$AI%uNj?Q-=$d z()&R@Gvjiq*|(XwapJnko}vV=xD@bqlsJ$0M%UBipd@**BirR!{|3v z!~e@gVHjj1xhJvih5<#o=@szd&j`D4>0T40Dbm5ONPYo^4O7i&yH-9lYm|HC&$Gy; zx(?sXqSvldsrE`c#@~Rt1uI^-qs>}6DZnha?prYdj(-gVEOC;NG{RyRC7Junel>e~ zLzqoV6(Ky^T!hF|;q4=%atXAYVGOF6O^MNP>-TTH1Fb}UfQ0>=`z$-wc?II`ieA8H zwFH}j4wRUK9Vz0MbT8OSpr;Zp_odGH3o?#i@QBma3?Hh>7-46@MlCV?5hdN;bn1MY zz4=;#!qMunjlBMmNr5uk?V<;}id=5tY}T-kx3qLYsxN<{L+|GLcs-BVPnv;e?i_5@ zj1!kJ%l1p&{Z)$NES*IH59W`0+YSrS6iaP5v8l?4#5p*jTTH%9Jxz>g=(A>v5j26l zwAiZbzpm#q$sb2OKIaBf+n?2QmU}bUBL3zlGEHlE&=BWX@FTWUU>&}Zt>@^=%4rmx z-wY2&0bN+>!V)X9xxrj0(p!56AYK?`=Rv1%u+lZZ{rMX;2OIeY!mpzWVhRmwc6-}O z^4x#WZp=M`3=W-mik>5`XULDj*T3*MF0p6>dHIhKfw~Ur6g|J)?rD#*DzjbW6AF}K zbkH$1*x~C@H*kWM#+CXG-5_n=Nn8zccGy1T89uYm7K@qx!UblyB7i>k1PfEw(~_U&jNjlPb+nT z*H-NRg8{~S@~q1}14#x&#S?o^{x_Lh-}d$)GBuNX4FyzI03{7_Qbz? z({|d)B|w3I*{T{J1^Ml*w(zSmEnf&UG!oiV~1h2s%28?_-b1HZ$0IGfWCSx4)) z7OdwZGD}_>-(T{al{-;s1d303?lE}eu8ia4+`hd9bLcf7xJoCqs=s>4JyG80H`i(3 zhXNaoGt3n&GvCj$D;1*T?dQYk(NL25VEp@eh!oKVJ|#FS!s&UBTVViwTSA*YluB|r#64hqnY*M{nE&g>gBzsPFd+d`u?^#or{$uIg2wiyZ|HI9 z2~3Kuzu|3Fez-v{f2Nx(N233iz-`Up@E*6beE7oU1#)8g^0+qZJg0xM@#sj*N}K-v zXV$|D)H8pxveK!LR|4`Pp}cu3Q*A$cr_~057r$wm{L<>@@c@;p=DeY-V9Im;MoWf$ z?IXia&5?N;DTTQ?GFikR78Z;wd|rJq$y#YKic{PP6Q5;RVKqucgez^yrb^r&j$gP{ zvqzV|5m(uhk)Y14Bz8SAhiUrc@HdZN{3d#Dt!jMV=^5E)-emlhlY5RyMuQUmeqOWM zZYR!(RmH_mPXcPZ=C%xpljjou+H;$P+D_B!PCBcu@mL$%B9$pI8b!NDPYawhtn7+O zpsF3LW`ithXqV*e#tt^iZg2>{e&{xWqROf*-Ipl{`#^57IAFZq7-M|bK%g!zT3asgqK_bU-xgu_x9 z2~;EQv^o3Y#rr{rJ@uM8)5fn+64+9gEjnw*-551U6|9f|&SIz8uxtA;OH=iG#jg)? z(R=UxWeUoHx2pmi3Yk#mt?nrakf*e%UJ%}aQ1H*R{8VY zc+e%!D-&i*u5sF2Vggd*EJI1mR9&=d@hmN`|LAP2W5wZl<(GS&Te7q)aYbtXgYRtz zBbbZlln9{WULg<2a|*&7po%9BW^yYB*JI7szfN!D`(c{hY!=P8Dh<~naXw;}1pe_t z+uz~*v!6PAUY<8Z`-FWs=Xn&e8lXvs_SVv7W;Ee95P?;$>i%Yc%M2<@mEu~+DEWMOq-)h~05UPG~C3ZJ9I!7q==maOVuh*!nG z%oU8chx$pSBafShrPsI?X>YzH-^)C`z@fHX`?w54z0hkf;mc)MHPkDxIqY0R8~0dE zc~J^@E-=KobekDJf4k~}(RQ5QPTU8Nx|#wEbMu3TRs4D}Kkb8u2;rBm0ANUZC)@lD zRWzmqzy84k8nIrwoWxoIlRZ4nqpRjE_hgscV}YO5ESog9mH5Lo(4!mGjVLu|M8%nU zu}I!jm&H5EO^y<)T#Lv3zF-vO2JLIAyOWoSe~mr5H|5G~Z@m3RLj3n%^rENK<2LaZ zA}`kA1(DQZwp93QY{S@g+}PN5Syy%<(1Q^5;#bEIyIGz7M`uByA@;jb!8OBG;!@*s zcLdtqQr2XOaXzxexT#AP!kG2WY|J{<<6_)gDK_ARJoFX7jH|4A(Gf7j>DT4ii8WV< z`OP*oDtAO+h+V(A3>R=obx|Yw!{%z$ht1pC41UF8T;?U?MNv8;L{eDB^I{ZIQ@W=S%p6UPAeki3o}x{IQITsNu) z+XRNH)VUUgvp&oqwkoo9tA3f<0)nN+#~zu)_9FP=!*5N!)87?cv|m1uDaNKoJIim& z7Gr;$P4j?8Es(?ix|$~4-6NRQ)6QP?#};2f&Y9Z+I=NC=+Bbbn^B>X0+tA-|MUZq< zgwhVm#~$F4JwA{N11F3J5h^hW>087!bK52(vv&sK42LTCFn0_USijASwzHYlG-cJE zrcFCQou-|5#l1Phn)4;j2{TjZV8YsZ6zK{Nbkl3R&x7F+YNqvrt?RxZqHMF`>sCbv zCU2*f$2TJHXGyZG*)09C-UE|*w#6n{LGQAvQCYK$s<|im31H;JKNk!otMW#=d5ClT z%sYl7vVlXXEO8EKk8dOb-dymZG#KBXX$Ydz*3Pz+AB>#(e9iORP}9pw@AfJE5vlTxAI`}v{>y3UnM*^0r&+mv z#}o|iThH_iMJ~RV-@q(gbvY+wOw{!6u;?dg*J-MkfAzyBj-=a3jJG9@t$%wx^HHTO z-J>wY;ewt+XGgEs+|{S0Sp0hVd3kF$%|I%32=C=B@qwB-)NrRTvPv#ApUBfK_bp0y zi7&!Ig|W)MD!w#wwDw^0c=Qta*9ksgUY!BYr=F>$GmbM*`LX(1!{}8~Tg}0IjW{}# zKB9E^lLHY!bMgF^T%0L=_`E`86`#5%HmbGD+Wmy3W2?4c_aF{Wj(Y=vryc&>MaXYW zsqo>7O8D2ruqyXP=o3z7f)yuB<|aJ%y5%T*knxA@!g*W78CA~1a8IJ>sO&|L=xe?x ziNLBxGItdr@OT6Fi(-78SZbBn3SsuLVa5GQ ztQSXQl@Z+gO$~F@c}JadG5Gq<%NFA z&^6#`+{-I=Q}368&FyRJ)41_Min@*J(9CN^G3j}}{ZI0FYz_@1U}!T}+D226sWN90 z3lmy=@|{)%v(1*)-&2i_XdpYJ@cSZHUV^M5e_Fi_W3QVT`oMubHD>X=P7^|p1T%#( z)z!yheA(eUzLP`NeDI=3)!>w7@wXBRhVd1InkQ9K2O0V5Oa#hi?riT5zu+ z?Ja!i1o-eX{4IrQ3FZ-o5Rd{Aw#og4%m^#QysO(Qw$t?439$xZ5No4Oi%XX;F0Z z`R4)U7Z42PVDORwxX`hPh!_gIhL}5kV_NO?dQ;1W-my#`8&%3Wz#*xZMaQ91o`#m8 zu0nZx;=OzmY`g>~P%rv}uTR?hZo+?lo_>3K4|ZlqlfSF?X3FyPKjOd>3$P;;cq zr=|szry*%<*D8)m}t8$QLFfojqNF@@T-&i z=Qx&(jn1fgeT`S$+|VSKqakp+AxGBCo8UZ#M_l9{5u7kPHpXsQ*Z%dbmYnD5FWcwT zIytg5k5NYj#vV+V9TDIss-e<^Mr{S1TUV~{Cj?(yzHkM&W~C(G=f<9@C9*70&?o_2 z72gzl@eoiw@}+$CfpdcRuc7*PM}-!tJ->xd3mjR2J^ehOTPMJe`REbxtM+&$X5k0A zQJp0y9G@_V^R2J4-<0=S=ckq6K$pf*Ht(v$h#jkMIpEKCocNcmE(DT~r)(LpUj!?L`Er?T22Y%RDBq?+j4Q_yss+ zo_iezvU8_uzC3y&g)DqgAi!7Tyh*2eRp+hS;)DWSX5+JG4sXANvWFN+Di?5#js#!q zq0l%G4|FyKXUZ2a_$1jbNo@w0 zU0;6xHGLnyX&g+ng>0jBwo^F4hOwX03(KcbBH;c>gL+e)-zZV>GMz%DqfuGZb8k-E z0Aoc^Hv+WeJ#8eXoY?8KFen$d_#k|!Ve*KKvX@lC2d>J<+V`m)eF-o@uiHP(5j>$f zc~PRNnmq+LXf)M6vgJ-f!)g8}y2|21H--7A5?9OLbd#`==D-a4`Nfz=wT z9xShDchAG`-4^O>8x^r??VnkZ$12KkySCHSPs)5P;7IM#1XbAu0^EK{r|uEOah-DK zsdagb#(FiBU-gbYI;XZXWn<|2;x67vJ#006ZOx7f*zXP@U@SB91EFX18y!8tsVnz| zDBefD>Ff{cgD?0}1e%*Z@%u0fZFa}mbb~1IAJMnpQ)HdjpSd)bIYmpc)b?e9Fme8j z&0wqoiv{Ui`cir))RpYnREH2q>BGjOo4R=hZKd&d#glZhD&p(N@tz{_XRhh48lC3X z;paEWdtdZ?q8X0cU9kmPpxzZ0OjIA)RYTn2;5Qei`^1DaN#V7}!hXL3S+Gy}-P-mB z7}kB#aV<$AZ+o?G<{b~7;!ufL%e7IJp0$dst%CRnV>y*s-%1Op7jW2!M}B$xKumm_ zxYk8FnX*t17AMJ!Z65=8T&uvW1VEC+t41~^mZV_fKNpyiAiO2XiBcu_ho;IoHvhaf zV0ncdZ$GAH$=mq#)J}0&)sbxw*wd!mKtsq&^CR_}5wH``sjXuRaPMpoKCxiUw|pnX z>rv`qv1MnSND$m2BYmuv1Cr$M{%e4SQ1F)Cim0~{c|}c*9JBok62ysfxCp5^qbZ;u z^Yho~R6@8Wdxia7<|@x(oy4N{k{vV%XDqp5&Y2oybE|eTQsbicpMdR8D*EK+fldxn zF{CJS;HOBi$_E*eErd&VtmvzJ!JlvYrcW{)M{Z3yfw?a1N@5=icy8BKDXyzn&69TV z_~~_yn#gph+B~vEQGHY&8S5((r}^ejj*?obv6BA${U7!fJ+5}-TIt$gLSA7mPsZs> zJSU(Q>2Tb~MDZICm_1*|RJ7E6ZwY$!G6IlsDye9pBvWzo_Q zUBVUk!Gky6YZ3WSCAuQbVZ}$b>8y2`d`ITNuQ7Ci7sn{Hz2$9&j)Q$q1E8vhb-FV> zqKLRzU*=WJ?xZktT@cUsym-uTlUuP{lN>f{EO5m}5MxjSE=4|H1gv*>TulClu8~gM zzhc!BNyz8c^ZAdJ6SwIcOe1)t=oc_f?g->vd^~MZhuZ zCf#mHV^fQ}X#a3`2s@9HRAY*TkvqI!kNg;v{o`L+^-cQy!@PkKVRy72mttlDJP`bR zFL*zCHBkCsMs*o&U_Z?Nc&^AaCjAuAFnN?C=+xMo3y(P9hGV~r2znxVN>nzaxr`<_ zSOt`BdIO#x{t8z($k8!+k}lGrVm(ou>?30!&v@{>07j^`n=f80$FCzG%lcFOAlD_CU)j z>h9RV+>6fr0TCvlC-_w^TZ${aD0RMos26_qGt|F~o8eZmWT2U}Q4+ZyUk3W#*7rn{ zOuo9ku(fkbwJ~_Z%3LQpa+r7ZPdYbnaf(-sZ@H)7`LS_xB@+@*oI4aPDb+0AF`b@D z+@x>NBrY!Em=ms_R%ii5NpK(iRueh3qeBod`xN136qdK3`DyINV1M4lLvjB_mWEgQ z$G_6hff#r}_t%X4Rqjd*J*MV5e4@$K-pJ&~$P?9Hy?Rw%6@DTgeMUM6W=zbsoGXu^cDn}lrtL-x_xA( zBdbfuJN*(Y&m(riwe^|1i9AaV-OIgzHS`hv#u6O7>}1bbCXZnDHZy!sb)mrw)ZP+sLBgKQD&iJdH&gy3*eHXQ&Yy7WVxxvGI5Y{8$ z*aT(7%=k*_CcX;Yn3%&MnSG_Otf-kefI`aSr#(F*aP(WY@Em{WQfY^8t;GF?Mt-&! z!;rPP%Nu`hR1*K&i=fg@Lv5Pfas;vW&yzA&q>2a+=wP<+k;&(Y0V^}>kZ!+??Vf8f z?Ufu0RRea1;*gyM=ial!P~2kTUDKqqb0$)y@hr?C0p+fbyzUVjI$O+{gsfT&aE{7^ zZ_^?yL^lUv+`eZY1?kZ=yzSlqJ0lu2(d7 zn&!{#%*!fc9n0T7rqV(V;4l!IoC-VW6kij9K3(Hwr@C+a(cJoCRY^JMornlK)lNv_ zSCTNXsD6kJW&ZitJYVO3YiR&odX|djz?jVx*g~MceQvbOOr4(+2Hf6EGsY39OhwXp zWSy5X)B^p*txsVbd|`~706=PyWVFY^Ti?As8s#g3bH&52oGRDd6ug4uqtcS!%dr-A z0;Mcv@%nrn>T}G1X3Yr`5gwuaO1{#DQd_RVpTE_b>02cqO9YUdq3HFz3@+ z!N1P}2lj5iscg6GL|D~TylExbrQqyCnKLgecK6<+A*oc-ie`8C}7x|#*}bHG(2 zCC-W}HUCK(1#jwXQWNqEO~?u>FFE=d-4GLIg2L{VDl7TWI29Fl>%Kz_QQJjY2ArxQSw8^gKb{Ik2!ek!2t2 zv6ER-eKR$74Fgk{K)|Y<%*V>X?fWzRY?No`O)l%*7u98X@-3*3C!kXQ3nc2^0IZTUjSgsXHp09==OOjS_4n6@ToOAf0=#<1^rPjEHlIeLD zX*uKixhrCjXKfl|zBjmAXMy1=v^J|GBIl zc04^AEpoyKL)$;*j)M8$u-MSCzocu91>f-ZTe%+cZ{FtE63-G@Iywawi(oka)q3#S zb%X!*5`5EBjXuLHao!ivVz z8DxknOveD}+`kKOfcfo^)=^fE)%4#M6~;_-+DJk^c;GU);4jCN>XfpPYjdec>4Z^Z zPqF=*x4!>BC-J3YW^dpn{oW7hl03S8O9n4T{&5LPWpg1SS}}S;B?asY6LmmfXo+Q?k z&CIjBavcUKkLx8S{Rem$M5JXqE<~yb-k}LXw`4Fxbd6&xAjiU(@46Sj$yT*vN|(Td~}9@)=l$uogZ0k(u}?^ zAggzj)ylA{i;Q!=kI9m6&@`s4F`y&GRt^>e6u$>#@^RY*H3bphw6#@QqF$0riv*By zW2h7z(ed8@j@oku^6S*3vp8i>kUtTx1XarL9ZPi$Wg=lb=?D;=ESNlSQa44p(l(U{ zG6@k%u$@)k`v_m4|D2ysov`E5vY5euNnB@jT~IB6mUqh>DHf-ux@Q^-KHX2WJSK%PY)X)7)!6;LuB8ipMjL$JIqMN2$LD zX56G3XnQ_2uX6=%C_Mc?be#z_)a~E@Te;mta>odRFp{mItYeTZ*^;tkiyLfdBSox3-nxTyyCNlzydT|sx(u}LVus~SmUy>8du z)H@VCeb+oh;CG2_^d_oG@OUaoe)aF&Vvixk>5Sh*$KDM4$&k@8DRyE?H2>T$(Ef1p z4B8|&6qgeFH{=SdtFHA$N>(NQtlq}oMM**!W@F@sW37yFeIHFMv)J)1SKLG+CY>u; zhoW4K>q;=JPbcH$T7qpORdVk%M>09iNnX1)h}^rkJA~)Z^zM@RrS3BZ*_)o4a;Cj0 zHo~U`Xz0{fUnN#c_>+Oxy-yAmDh`cc$kYqXXA@?|^q|MV-pT>U0yb^Jv&{wg5ny+1 zf14acVk^|m4R%25H#K#I+?wC~_7jsojQKUL1HFg$BG){;s&^EVQIX{@Q#*Nt(JgEv zG$|TBOX92AVqWm|@E~@d&4h`J{7mDW!dTmC(AUVENg~Xosc+({NhRzCjL;KwaqPQ$oa5aX;fH zK7s9ArA6)<%Rvd9luOu$L_DsZ`#o;L`H}bGOUp45t4#JLd?z*Mrygs6lP1Qmywv;XcZo6O3m5o49vG5@0)5P%N2&8{LVa^B;|JYwU53fuGJS_EdYN3)rF2V<4t%ymz0K^6FXZSQ@$qt<5Qm z>Q;Blo4b~ymoiOKWYg4%(bKg}l^8Vb-4P5iLPZR8*jE$SPj(5`k28&r`qEj10Fs`gF|IhTyErP$ml ztNB&|p8ES%00kV}B*Q5T>=Y6=nwys!^a8xSKu2}LX!+6Lvky`5f^;l37a8S=Jeh^Z zC=Tx@idDKf4q}aP;n?-6@o^B)Rvh z^4{_R2uuvGOmpqjZ+shseY_3cc;!i{AS-K~-_2eBDB+S|W!~_FBOAqBiQ|l!T~(dr zJ9-@K_%Re?$oDbdo6l56y)}+_Uke(gddm{D#p&LY6$RNeAzrY7t@FID6($zMDzB8v zNIYkaB2?lMW)U$jsrBfN5#W~W)&To-3*=9_VwoJ&UFxoU{qDxL)WePodQC+lyK`W0 zR&k@hYRkl{rY`iEho8wK`j8@1)FD{@>Mg~3Q*K5Hqp6eq!I~dl8^NaP)K~2m0Ubi8 zReB#gvyEraQgoc|o3W!ub^**H7LdgM^&*< zOsn6-;ab4nrU^RimMI(oU(L!Yzklhy8vL5q9josmZ|@YDa8G8NH;m^}p%LP}3hk;N z*2HW47RCLP9f<*LcJ^s5i+glX)%R{oTHCc!2szWGQ3Ud!m5ZHW`q*mJ((bGNZ`P8hw3c%;BA)ziTx zqAq~QW?45_=R-RB;RRJ;>~`xd4em}nS~L%!?n+T0BtZF*EW@mF7GFzk#sZM8~%J~j653^fJnH8|I&hwh(7q|D`Oo%-?CR1dB0syi3 znn86PYh|=>o9P*VV+MI>qzoWeQ2;g35HmmPR51fQ1TT0h5cRGlImUt-xqLI)0&DM_ zXdI2Ykrrfv=s7^xSTefAb>_kE;pzSfvqhoDWak9}F*?sBbB+w%dD;Yvgni2g|M{ZF zjK{1HKjE6Y`wC05?BRMtWII2&?yMu+fFvVw$vU-Ozg~g_FeA~%cN??Rrn|R2@pn(I z`hSHCwR_xVjUzV9YtkVI6pYW;C)VGHXUv!&MO2M;CCGf!3kVY=T|Y3~gFAF*^Aps4 zxIDPA{-UgN1VzoHG&D4Gd;wiYZ4wT(eF1o__fP^#-m$bONe~c3OtzD2gMj5)%Q)-v zV}GDbGq|5j*L8H*xfQ*UU@&d!Aro?KC=T4!CktjYcoUZyO2&|(#w00Xk$&S&xy0SS zm&w4)1lFtJBj`Ar`A=x!Hw3r@7x;|vu;;#7n9X^8&|%5heq$3 z>E`>vn_vr|)|-As-Gq$gxR}=fOz``SipTFlLyN-Wu8E#LAPW-m*7w)Wl~#P1Y8CeW zWv6T>E#VQ>mmSV2T&kVSzydoY)xzt{>tFlC{o~BD4^|oPA`r|G4)*PmeUcu~a?>@u zK{4U=b@vOJ;8M=YnbxHE?rYS#$WMwnU&f4X&Jb)4PDZiZnt6+sKKo1 zvKCMCuYs>3YHuGIK<|i zk@G?hzWpja8Yw%pXM+FU7t!14@;Tnjn(=#m?qEngEvcDI-|sH2t@RPOZQxM_)(B1PbD1eBtviHu9RRwx^$s#VjvE^oz zret-HBhmXP2X>Mv25ZEm-)W?0U&4>5Fk8ku0tNa7kIzyee)_D(8#BHMIX}iTZO4W> zRSh#SYOnoe?={1cKsHfivyI7k0%j=EOC=dT+azA?*t@rZ^+AME?{59(&|Y|CgnqRg z+Dd0X`m_i5ssUE#cDS{X8NJ+BeL17udM3N4qFA(JS@6l!6g+=)e9i3ww#jMm0_L&7 zy~z`QTB&P|5fYPX)2;P^p~$ZDY+h>h0U?G0llm{cCeN}z?F1{BIJo`J++Ts5HAo~4 zv7(?TE)+8+47bVl(?jLT1jIDE3)krvc%B;hfi8S{;~iu>>))6=h((=&4&4uCgjJ~{ zC9<7d<5W*Y@+He#ky=~cG2Dv0%6NTj{PDZCAm%a7)uhY40m!XXmxpmw6v8ypk~5aN zu-j46XbWw8Muh6Q$QLjQb-9YAHg10^#vm#DBAPq=l3#Z>-mbH;sG`K%F@vH`LIzXQAOP^h{pCCmW)0t1|@d z$RMUj81FbxnppKFm6DTZzv1UM-wfH`{uPIL%p00+e5uk`K5Rv3zRNt!LKb8EIqm;^Y z6Hc>c^jMt5ow}T*mP@Vb(@A0MHvx@O1J)iWteI7GENSX?y<``S-Z#tqmJ>)cXSN5@4!)s~_2b;pD=I=UCb_*PAeM0}}*M=`J4b@6ZlkYe>z=|>% zM&WMDXH{i3)qtakLA?rwRjJg!N@m){MJqAcqH-P0+LaiZ=tr>m8}n1E#_WuJyZ-P1 zw{(9cnnLEja>GWCCVj<&r_`u7?mqdL)#q8L9q=~yx}KHtIL9%(SDCa$e*?B-O>Nn1 zxzXtN&W&Rp$lV;D#B zdCJ__Jn6f{(%cf>azzhb@w^UoDe*AA-@$2~*k12P#lUNt^7YrUN>xE?%PwFrE#B2p zB_iBaJM*VAk=;8H&nvky+!GIBi@2b znN)n?Ivovz>hwA6KQlbE2{?LmRGZf27;mCuS=TYA77*MVw!gY0Yc72>ALo`9sF)~-5a`QrS9-{CH%(_mQHcLUu&m8(8s+%wHMM9+06rzI;KLMLq?(q1 zBn2^4&(&XGVTvd?f7JXiWe-n|Xg=RMg^A@3Mei3K;1KXpb~NSzC}~WN@86JglaW~N zg@#$KM&}D1ES7dqr8=WNJTQwr$ zYYH9nOQ6mnG zfBTg)gWNz|Jo*dpHpcfy$SGd#aXhmzH`G-m$da9HAVAC9sFrcYluizzyY^f$OXV-iUFae}%`VD8ZDC8h(p?lV&}t$d&>ss)9|<0wW2#D1d^eUmiw`FRD?8;n$BR zW}Be})z?ShGzgpW9f~DY5j;Dwp8wyX;8WJVeTUXUqxT>-W>j4A+eY^%6{AU=$WXd6tcQK=#J05Ql%-@5ku8Y3AgJyau zeFpKU4)3RWp%VAn-8k+qQuzf2$}NEf*LM-_cC_(gnfeYSqPQ~v&Hi+GEdToGnwf?nA?K*QL4{z>nn2HT?lLrPfY2Ng? z9p}Qw^wxMcDU@m@9P_v!LN$L!d&0ulrTYm{TZo*fWUyS(A-5EEZ6Jc;Ox&)WQw4!RjKm1Xf9WWng$%MY9OZPSr?kw^ z0%QwVX4dIexu+d?UGDLkBTYLVD#OmGGiF7D9o3DZYi78*Bu35d#6Yx35`n3CQ-Yq? zoAEDoMEO@o^{;YMe~Sp?L{c9+HMJy2b0vmUOa~)K2;r`bGsEibyPs(pZZQfh%w=?P zzZ4F}-I&r&>M9XFHzVpo8rIIc2kq7L?rxZ2Hj17-*QJ=KDE*XfwD?jTZ?d{v`B^z> z{lxIV5|*UcpwvhYm|u))gi`Xk`X$OX*TA7St8W1 zU({h0`)#-15cWK1-n1~Sn%xfhhKNxx4U;OkkfH)bi`!DDgy2DkX1&fp=rXSoltDW0 zuK%D=+ZRg920C>_#iOpvXjs-5l_JGah^Ii|qL#PqThAOB<64z+k`Nos3b81tRfcQtFRr>yCww3a?j zt=V`e+61+kDnCcSbzb&fds0v!&+e|UN4gyR7p!mBuR6qd5+dBKI$D0v@(L=syP3Fp zAiQ8!CugYtp+LgyOxHBHb)^dZ%yKC6Z@geS?5AP~!u6EtNdn5da;LH1QW=n#1UQ8N zi~BPOFC=SWer$L$6TOz^huVIZ0p<-3(E_ck0 zyqeG76({*d0ZD>bLGteyOGRr*I*tEKnT(F)sf^ycc)Z;==dP!XLXh*Mc;RU26uSaFv3tgk1ZXiJ-8)TmW+aKn>B z2=6XE6RwJ>_G+;I*nU*xSiq&THxdgl^QV0{nI=YPN17~>zFtI7SR>yVX6X1TUc zTFercL~<3N2CXr4KX!su#9HK|ne^gWMJp`Mbj(~wRHq4QmzHvfmhw_1PSce>W}x~y ztRB-6&DIzPov8(ksfp@o0xJm*ZX>B)EDYMP%8sa19I=l&(>T^c zOXISjA{j($EGd0!5(T05~w^d9KO1_JHmBm!N*#o}}juJo2vxx?^uk=Hz$7o2ip2yu8MH|@b>WG>gk zxo3Ra8}Z&cOiqZo${WD@mYqezw_0zv!fP*`iZ=cg(SY-(aexM|Bv5jh7>7|K_Gh_N z#)4)vsrCNG`AUt1w*^*zP-kgZ=s8dOzRWr#pLO+o7s>!p*vhm83Lvj_g<>&Ykkzol z@GRkC`rIR){SYVIh?(>-)-x(%IUuF&9wSMO!D-`k@y8|Kh0_O-``o<7R+Lt}Xg`#e+YM)BC1$eQ1R! z20qp61DNbXp%r~*5eJS)uDr1Kla3@F`bi_bYWr{2_ig>&4?@xRcO{jRA#IthZzVP( zriFQ5%%O~7+R&ToQO&`T?7WuRKE)cQVS9^?Q{U8!} zM5d;&`YMOHJtZUMBcAkK(h#pWE*J)aM(te_H6Lu~S|)|jCh(2KqZ;o7v<9^13a>Wo zYfpX(t8Xt9 zXcU1gFZU~R`KSCYE6tTA0(%a%eSzuc29Ur1Tsx~Y~VC^EB!}ZL)^Hs zx-x}P!bfALJ@<8OtIm%J&PvQToC?cg$uD{K$F=wgI`jAqg%rofu?f>GXxKega<;4e+ z#ml1s!A_{7KrG}x4Sz*|``?aNVc3(sSY6N%JdZSISOyyFTHTG8M=AL$5pJ2%8iqW| zeFc5qoQ)KlMM&M}y#0dv-A?G@0PmDdR6;aYNnU?tjGADqGHVqOEiYeh+rKw*&o4L* zj&5bIH;B<$dT-8*7H>ey0f1kPg2@U3o&#FBi2Yb9c&wFCkknZSp>&p-8;j6wm zPiQ?~GQfCw^F2Jzu;o6W{sYW>rgWpCvzcMnMVX($%W}JqUkWaNKd}#ed|R$cb?YpW z$L8uW;tl@b62cp=yw}a`H|nnarm*tqkDs92x9>0%AgDglL#1hwlFNxkCn3K{hP_rW z?G4z~NAEsu4%mHhtw*;yP|4g$B_!Nn;>#7%Bf-&UP>7Ur=_A9>-=6<^IWyNiB=kgG zZBhRMEoPZ5r^L!eorxe>DvTC7-LPjd@JDx3Tml5n<>)~8(d$hw)sQ+|bjE%5ohbyx zcgZgh`xzfgCz9 zENL3>zoX2kOM56zeA2s+MBk2s7g4G|l)53bN5z(ZWzpfV`CPtA-YKCp_xB~s+AuS- z-v64GHkVEl^auw-ouOB9dL)BzR}9sfTpSan51QP&cpk?gJd+osW22I>b&kG|R{%jr z`z=S3oi1tlBLeaz*0u!MnI+OHkCzlz*Uvx!Vjc{<_tJ1o6DW#oop5~>78FD`?kn}^g6(%>Kc?+ zQS5B6<$4r{UY3{Ry|xc_BmAA7_uj3Br_79D#aeR6o+MBxPDa#v+;I}%VJ9>hKy$zk zKQg1QX(;ovIX%(hfX^Kso}yN}$kMh?{K6>i_G18PBUg5SmhP7pCbNr4gH$Y+QTvQX z1>Q0=S}SQmryKogpF!Q@Vud@5OUNQq7Ry$hl$++c!k1CwfzeEGa9*bQs0|i#_B8tZ zWX^H=bfXR9!s&YcnBOeoZyJ$M&V zX`!VLKZgZI@G8IfaH}bpN%X6i&XU3rKPZ@`#A2t&)q>kcp8C)_W19(n&U>4ca>@UA zhehxL1z_RTPA>quS=zzEB9-%m?bYmJXM4Yw@(i?D&KFj$1jhkHi@e&!G0_oG>8pMd zw-}WXLtcmyqt8`=9m_<>FJ50A;i#Rbu@8)=`>iWnM9C7~yg{+2u{Q~I?kth9vGXF% zk}+y}InuxX%`x_RrGsrPBA-HXVX;CbvxIFRS=wK$Muwt7TiV*3<5$vSi+2+WrBBMw zq;t=?E9-XZZpno%>V`Zjsravw{_j_7aIIA zYEYX->Gr5tBKvjWJ>SZQSsma`<29f4u!QH2V(PyiHy3=9Oz>F|>QCKAZOdlupMLeo zb~h~bW(vd+bv(REJC*qQEs!xSV3$~S1;Agx$EEEal?k<`vJ(`J=kpP)+?Fry@QUK4 zJ@6<6TH@@zKkk1t>E%pfRB3$5Htve_reIT(+t9~UobgIHV)hP~*is21Ybj74tsPS( zH1^0`#2Sq=HWs5TQ8IK2@tUnLecSgDL>|ibXNIav8iIG#pB~ZvQWA2Cm;=&`N0p=X z?jL3>+?8xQY%t$1pz52aaeDB@8rf^uTzk4XtZmwBl8v^HN=*WJS8h zzE=pMcW5szu6_SEQp@f_|9ng*92DArE2F;vCs!H9Vbn?5>>cDUA?NtEo8*q1GoD`M zCfV>8OLw1D{w+fVC^L&LpGHwwq8v$M^T@J__1D5<VjPKb0%0hVG$z z;fbXoQ{BGpa8}6vG$icoyR%+2^HlPzZZ3ODqv5I*tCaTdRM07QU^WPzlf__rx3BH@ zb8JtiWF53NWIbuO$Loh`Ha#btl!P3OP%3FoNCNkf)Kh8+vE{6K;8S=k8Q?Zd`2ZNq zvr@kFWe!#}#34qsul}2L$2<{$;Y+w-MbMeq8BG4Sowt~qvj%+5w_+Pxlh$0a$atr- z^FUFuYryN^NUcQeJ{wPtra%P0GxY4iZ?a>zf!z)XG(F;3(&LN01A}MQXdNuma#HY8 zj^Ek$(8=fFErwI*iUXH>Vq4hEO9$o$1frw+;>|3{4oI%S1=^EY?Bwj+;-3jaUpxJ zk-vV`oM8IN$o4!>)7II!(_2BQz-jqoQOS#mlhuYyCnXq;FX?uq20%x=04Zw+Tu)dR zCJM@_YXz>;H~Wo0nksu$tr9Z!fN>S=Ji{_kjs^w$F{7Sgmhn zNT=bOg~3-0g)XDw_1iL8LenR542>)7+PJ>kErT8bc{E71$FIin2}@J_zcW?9oEeU= zXms^+fe?t-q{o%)!!Z@$fX~+d#Z{_i!q-_jX^Ta;-c9ekHsQ3WJQXhcaZoH|DlJqy z&o(@^Eiz>$t!jHIY&&{1OYiuikhu*O@l?Gv;=VFc3yec~9qhEfH3F3DJ<6-^lq)2K zV#)gS@@S?vfAsOnM8c`|6g~ z{*k135u28{&|>Tm+lv)k2e}LX2WsfoR>_q3v1;e8hV^n((?24UU;q09q*juGVU(2Ru+N1i%Ax1vh z;Is6!Zo2r=Rr~3fD~oT$tIRg%#A_;seNK*I*{zp~Ix7ENy8cX!`SaNBM@F>Ir^gAX zv=U=-b`G-$!KR1P51M$L(XxKAAIKik*&xhyqkYHT4r+zwb?poZxE>wC1adWW%BwWg z{bql}b)lo?nGG>yeEEd;!G&qBI?YI)sbpWY9aHQ-6g|Q#)cXLv zNZ3*OM#2MP*aLpA4TmkSWQYHDz5?s37StgzvRCXsXOyNUa7H0)enn*K&w?31dFNHe zRa??8wDvOE>R^k{0o%JonIV=g)c-iFFy!@v5Z9}^KgbmELRH_T3%9xy^-%HaP?zqL zqF)0&rL0OP$6kX!^O)RYYcJ&08DARvF)5_#aDmZ?;ZDIRVQ!C?=iI>;@_t_#ST7m* zpWAcyKHB=$RpBa)noq>+{P;DtR`NtoYdb6FbB^beN_VUgr+u}x`dDF)j+V?Vtgep+PFqgD|M}l(K!5fHe;y4yKm3ySg28Mu-ib_Fa(IncXWjCW zz+OJTvlq(&LmwVsL*T4ARA6(P18II@smNyRX~0Zga)uK(6X#j+xc%Cu=gAR(Owp#~ zM+`MT_fsWpY7POPHnK0iGio-fT>lq&&e*ZlnRuQ~%(Z;_jBn~vM3rwuTzs_f;XF5k zmkl*ewDkVc6%+~9mbq@_8|?Ii(>`WrG3O5mZ?K@N8*AeMb1>d9F!{5)s(nI|L)#VG zV&>}uDch>6MDhiGpUh0wy1>cI$xh`7aW-jHb}6!xg=yQ`%9XTnuPjBj9OzO2G`1{vqm>{Rrf^Egips{gfod!Xykj=;2+te4obj|nxDtG#n(%Z z`K&2LgC-QxUgXNGh1t`*TWO*E;~!R$0|(mnRGUenk_=qy#XeDqHPSa*!fPnDfI#(^ z1SUJb)pOx{m!-iNB_Sgy^rl5a6hw3{?VF7PT$w47?}pBO(;EKeov9GsRYjnw$P6i; z;1xD+)@p>uuc$RaWWf-%?M8?s`M5Sv1nQ_2Ulht8>HZn7kn^H;bFRC>xLw~cD;boh z?Z3@GTV<98v&63Gc?h0v*tQtX|Lr6#e(`MO+{p0& zi=ZeTx>@R^aIyBSg1U`b6s2xTi(c6QZN_#mIO3ppnT^1>3b~C78xCnYBLV6j_&mBl zBWwtzir=(<m5Hn} zAjH9&ucE4WX#H-yYB|q>4&^^#W>i7rx2B?)aZ-$*5mF-7G%6}E4pe#es+?|Azy+M( z2qim9kJ{VAaxG(r#K*;)t-<;h5-kgnloP7ddqS)t-+eJfq)?~+Y}&1et0`5x$gx5^ z7%4Mi5n@fpXO4O?&o=W^rQM1Ppl#&se=ThD)F8vV`EijN`uS%rZpMV-(^76SbBBoY zDlZVRzZOGnDH?WF40Ea9{If^=I-)%t4u6y2Ij*}CS4DnSw8T|A_!C5+kGa5xKOf0G z3(|Ltm^0t85uqcGbOCm(9QD~d=~dYhmIitT1F@Z#R8;QUWyHsO7iVJM2D_$X2FOW_ z=|)8Z(#rZkBs&hwTWMFGiK+?NX@p>ht)#E=*cVnKV;<3lLcnygA;S4OPiEIzEt5aEd-8*RWrtQ=P651~L& z=07t{{^#e%RET@Lf@&9EJ-hv1yv~0H)q&47luH?z6XGQq|Br75-&)W}8CTHg<+Xdq zr1;^FXJl(YN>z6GC+XJ0YL^%Z_H%_7?r(pcP-(PI_y>$T`AC-13oe-nk3K#qQ5jW} zdXsS7KBgQp13zfp!SBRaV#Gdk-N$Js&kpIa&MC zTGCdW_VFln>8~rn)j9v3H#??%J-+d68iTdDj&h}qe=`#+5c^2>f7Md$-Yl2e79{l+ zep??;qq}Pq>WH~GacpKf)VqCN`$=q7z?i!;AxtsZ>D>CPCeo-u#m>XsLyz(@-D>ao ziC0Fwo{r6RjMr@Ck`b;-QvBaqAN=T!b*P(EI1IrM<5hR|E)F-dT-TG`fHb7~JPU~s z=V=Vh%u8U`?Rd1g4k6Qd6oT$ zhZy``WXGxqMhg}Vot!7kk=eL}w`vo(VW-O7OAlmeT`KGGwUE}mOT79P4!8NVt`B7T zW(XOZf6#F}N5!&F)P5s)kJiP3l!n}tfW~7;y1@!ew#@G+DcF}(yn-%ZkRjd!$wE?$ z(#s)J#rC)0O>QHiQBT-V=4Jf&xau3hU^W$&Gi9*w&rKeTP&85anm3`7KN?&O?R$gU z-05G259*Diw<9ZztgO&7(64qZ?D+)LU?uqn9aV>yc45hSlp!l!R9-gxqiL%954g!9 z^)<2KjTyd+C0J7?{22*a zxSpD%c`VjnYKk7B1SWOlKW7`29Uc?Os%o)d1(Tq&T;NHp6$!UC2oDI@2?@uAHo6Zg?j4+x zn#C@jR!Ix94J3p(UE@NJMPzj<^K9SN8_gSn?Ze$--13#y?irT23-0GiKTjyo8*XBh z&Hd=3Ze3KkD6Va#XS}+&siT+FI_j2m4a0&7lKmF*GMMmDV!HVKGJM0Ls0%_lEB~$2 z)5~=+$*=FpY@>6yXX?`2BlQ=cVkL3tXS)v@(VM=;(l9&-9Pw;L_C15BmE}ThS*;*8 z?*zYwiN{uKGubGKE=d$F;Rzf%L%<`14h4m^7hoo+8n_Noh$^aTiFL#+De7Agb-EJ9yMRG{$u{*gvn- zctAOjhWPx@I%+4c#9>FCz?03KwkNNMl2BD%6qpo6#MziaJKtJK^fhci~>AEUB2gl|P&a*nfk% zo$mqm@ggACV=4~PM)nCH?Bgl@Jxj2dZtQZtM{HydLG)}-)A@~PA}0X~t=3`O&vSUqf0N9ox%URBJAi_ePXaj;@F%iF9tL=qCYLUZ zSSl}GihB?6ZiiDkt{t>hVEG3D8w z&&hL5?RQ?U)u^KA{&TCCJWU~Klkq!)1)%IQvZ&Su3dmz|a)j*tZ<8|_DMws1zk!#F z502tZa(bEZRM=uAlF~}XRjT$<+p8IJ!Dkcfiv@!DEd77ozM>~L37<+mB*(d&K|Vrs zzVW`YCnyMgM}VSF7uwt0uvpbF>f|>B4T>glwgK}Z$inoaW>m^PX$p7JgY_=1`1S2) z((-*q%Wmc6Y=&W*$!jkb6xc_|Sb1REYOyg*A$gdyxK>l`#?{MP7lxH9g1p8C*|A|EO^>*%Ao<){b!R>;sH%9M}KI6+KgXFkE3!-)FJSi=gY;*0%LT+aposflX&Y(s) zC$$A%ir$~XSr_dY@f|&-?k|0blbIImgW3&#pgGD1t~aOT(oZTzQTd%93{S2R#MGV5 z^7-3`Irp;|d5w4TV#>*(RS^ImiKj{Aswcw1b34a$92l2{pRR#)+%3IoQuL;5 z$6*5P&TXd3?$(#>@-ZihqA#!Fx+Zsgj^}+ynA+yq3F}K|ueUP4#3db!&3F1&8$rBl z;ekZtj}xHktLP+qJ8eBoML3 z&uXU-IX1wRuVpej1j-JgbS!ZKH||(&`zx{?K84=>swur%JO>0w%WrLVC&T5L(LbOn zx8GDZtbubW;)1Q8&sL90ey&Y{r>|{Ql9G?F8Ort5&f&7@N$YlWj%|O@_@wIoL*~ar z`ip7MQX}G3rWXAh1U7dgay{ni&vZKJbxMq{qVb5YE&lJlc?p7KA!ez)b}JpbesW&e zqb?V_zs1%H@z4nn>#@5eHFdJ6<HW~c3OwoB9rL*L7bDmmF zdYB`pR=wtmd5T+;B6Ib}iPXr--6gV3cZYD6eYZ9@QVP^^*^8hP>JV3@_`N8zxsZ_+8Bew0!K1ia)P~QD*7S1u*{@g(p1kX{3 zm&YBw_(W@i?PIdw_C1KlUZ?v3u@QM{ z)%X_+-vK!+=gJnT%da2Ai8HT#)*EW)lKuo#cw5mFA>5Rtp{CDtm0;SG)e?+!R)1-G#zqxat;^~yh~s252+dkvOD6E z$TfE7liCwuaK90O8s6DoC}^NGpCm$j$v`nV~$`~0pJ?$8}Ag3Clsh@2{p8NKb3L2KE+H%TcQH`9*A@^>zn+nankfjkjh>EcCibEP9`Ql;v2B5a+lpQ{;Ij zBzqfBQoLYxeR51ro`#G&cgZ-LRs##;$PZFXnGsIn!ocXw^7aF3r8JEbWSRB%Gz_>h z!Y#e#L{qH$IK=1sR-XzUET|lVWuU=G%hX}4L5m@KCF|zP4pQ&rmm}~t==s>9EkuCL zW(CP7z#goE=-#H-kxnv}&PJUfOMu1X5nHpZhJT|GJIftbHT;bu)cD6by9A0-$rE6Y z&DS`2S4M!ousHx7rEg$A@jNDgZc$+*c%nf3h#>`6d_s=2$0Wh{Y#$w2*ML?|@uFCc zn8wAzBUZKDYZ@0vZkj#)3C41jSeK;FRwcr)t2)V~C>w<*ZoIbg6|K6LJv1u zw7f-Tu*m&|9bhzf+_AhOXO+}bdefm3o+;2W%N~gtXFnn6c zC*PEY=WfiH1G1ea}&X1u?&otz4Y})fzco!fDsu zR|Oh5E`mq=G85lFw%F}3VPrLw3p=;ewG-@4w;#bZeiYeEf6&~@P_+(Y-oCR1&p(T& zn?TMmcReMCfy>Bk!%ZXxK7Jg%TE}bc*ZVO#ZwL5%Q9GA6xH}UEH>e%&s1(@9Hx%`V#Y&vXNciy`NMkfAZ)o5XP%w?gA@+H9;`vZ{9iX9k8yp#qa`3Wh zc8eKF3}Bew&((9_3}06yoc{-t($U2y{Cr1AC;f#tRDWBQFM(HRDIx0z->UH(1mzQp zftPM}I;&kc(cjFslgcI<#_7*KzV&j{beA{I=iT1?d;GVp+$R}zM%52_jcG(>FNBd+ z5kEvCHQUjb=fDf|TuAHef9h+vA$lg|YWH9XdqxF>)$bJdKjSLP^i_Du_Z=mbqmpg1r zhRLH}ze0vnEbqlL9lxI6;aYM!cz8wM3bRWhTWJPPV#=y`Ce|q4oC~`!(f#9QMa$~E zud6OU=sJbNG?qlLI~>#}^n{zt&8x&6NpsENfiH%eZ_kx6;VSEk1i6pD5qw7KS4|M} z&YAqi*;CAPy&DAf2`@0BtgY-7E(M0zGdol>Ub>IY415Uj3Z%It7O9^)xg&b*2pfIg zPNOJss1K_z-ymzfo^W*Ac%OYklcBll?*{5~>=3ZftK*gZ$_U|*T`v^?X^v1o2m zDGnH=2@-c&WMbtn4vrQlPIlc4GD61$?vT8gX1C7k2I-%kbt_32Bf=_-$Bva)0tMYB z`S-Lmzk(Ey%L8iFU^h#Cp>A}dqry*-t%2P`=?TtnqHJFKdV#0 z`MWIL{K8A3vY>(!S;#sO+Z{x=A$rWkLGyOsE0EPe=TbSRWV75!KZFZwE6_AV_vBhf z>tJ(g=)%}#b9K}1AE`sY;#%rCu@{~)|FFsSp|$1JiQA2*B9I-gUXH$m1Uep2l~E7# zR*0uqiB-KmFTEH5#xQ6;l7NggEY$rXEIbb#xFS9kGKwleL)BJ%I`On_i6N`hMAXbWZAP&`U)}5l8 zq+4CvZV@CeB<65DeZi;-{TBby4ZLJ#^@;}@l%$A`VEQ`?1YU&cZOP-81df2( zwAg?hoUa&zn!`#TY^3SSL-kS!x4+r#)>r)76HNsV|H2j2{|GQG_R|44-2GnSOGlfjTw z_APt1gjC4BFWHwFOCe-m#!eVZcK0=XKHvZU_q+f9bD#4&=RTb?oyyF~+xz{xuIu%B zJ|9n!^B_Ijp|__WgzZT;F*nMxzE*BCDi!6n%{25`^=KlV zZ>c`iwYVlJxZ(!Zw6F8?qYon@F0Fr8~2i zBk`xWx^3U$_DP$pFco^7gVx|}5B^wa^GQuEg^pasWOB>K1C84)QHv$6KWz7JS3lt@ z=vDXNPYXo>6?$z%I4{6cj6UR8d*9`}BSN6ElIi`CQQBN4->ls{zu`mat18S~Mpt#S zNVX*`p;$FD)BfgVi8^obb4I-hZEN7`1|mn(7JF0s->heoTFA4d}iyc zUw8oUOU=Y_E87vX7i}klm`}ow4+LOQ&YcRAYFFF}qiQUFHhPoC5!;(4#v6kY+Y-sg zCj;hhd{etzR_*7#&URg#mGr2Y{$g>wyXo1x&u#OhAIDRB(C z7)iaV#^)xPcB9A0p3?LQ-Q61^8t;!1KKUN4(pCIgr51TTz?)O%KzG&4^z|!d zg;05d8{($>=vB?c_{0n=w}QLi4VtEmt=Cvu&P8g$-CL*2UI{BqvR*O^^ODfC`Gv}T z$;7lcF0p$-63^nmKrE+96ywpT)*}5dSuOS?g%78^qzXH`D^31xhbO3!Yrn(n=R&u9 znEg!nR-P&cC^S=UMOZ_sQ=l=HHBuKd%!Sz5gJGnjt+tcb)GsLc)_Y zjfDpedBP(3^s>iT#GbIH*i-1fPnA3LO=kUdxPNtT_fuEmI%52!29>VcAYG^*E_pu>cr zd1Q2;%K4oip~Nd^`GQ~&$Lfh=5N}@p(3s#O!*nJ&_tFogK|*ctBbrg+;u~iU%UV)q zFS&WJQd!y4pJN+e>*GCqZDrbVn!W(^ly1(%urpF=!Ws9Og5{F26DH4-OHFTfuvdHL z2)Iu0+F;(+uXRZU4q>_#CUMighwaBM46&XQGc^}gmhze|VH1rqW2(^qag&DX4i zHRtA>^VAN6##c=(E;}{hmf)6HEh$I6Ie01xRjPvEKi)zL_7P{I9Xw^rim@eZ95FVZ*m=#uUW3<>njOvsn1q?KOa%F@)7#_GPA9a1y@! z3&t0zMl%md4`3v(iIhG=w0=ya<;1>FM!yZF6H(#~@s}FiAAH3Hg$MSjM94P~8N0(C}2kUAZ)*A(=o610t~VLTvm?_B6OlPHbB_}%=Dz9#E|R+W$| z@++_K8{L*9?##xfieaZt`LtF+hu;bTw9A|6I)4=rR7bxec!U>>)Pw6BCw{_6aikx{ zHdAO3$}&Px6@V%}pcbe%lt0Mo^sV9`OBLqWf>x7V;whnb--aaLLLBt_SR&m5yjGbG zKB(>HlM3Y@_VMPbhN4IjAI32HEGcAenQW8p0*!JJIdWU%?oyscx`NAxC2sG{=%(HK znLB#0^_V|Dgz?&ozC80hfHA3U@#034^x8K6rvFO8h!`J}v(69CLu%vRIxjW61%6+z zzhwO_wN7+gqeF0h#oJN77+zca)>fOy8oumFixLUT&QuP1zj6YlNF|SU-%WS=F!Q5J zBB3Sf)i81W%P~e6y6JxoyhXSLmjrn#)+13|Kg>Li1irwx!3u*B-Va_9{`TIWy73j_ zj6eywNNAFP&KL;Dpyv)-`XXt`EBQ=9f;eITxqJa}@u>*3?STA1$>$2H#BToIPD zxq{OY&Q3HhW4*;DSqq_iy&`8}%2ycbt!7>~uZ*MbyoZb84m$tY(>Xs?*&ua^2oV4r zRveO%bs=t2&iz+ZCVd`BNCK(>+wt$w$Onx6?N6$~*m~stb&|-!A`zs;zcFNVjA)aa z9%!fuz?;VpSniqLrf_X}+sYbsBO34MF7(O|0QYQA7eE)hO0P?}{_k{^|mW8{La ztlezOQ_rv&5}y&l(+Zau427%S*0k~7pqp0M!0&6akIW25j zuPx3W`QMT|6tkA!L-EvOpR5)|S6o@XoNFk$nrcHRu(gpdbL7$=ebdVRvrPNNP)C&z zL}m{igu?nh6&F*L8UOk4A3c*G>{DcnMZlfnGSOM2R zXiZeag!8L(t(8-XTSV*_vjl-~9*0uE1bVtD>?v3f1U|duMXH?Zs)$?iHo_jm7+I6} z)52JY&;AjHm@>cyCfm+T&!jpCoH3x&*y#%qn^t^U163|JX#Avn&GG`d>=K{m&izv9 z#>NBxAQv8;D!o6d!TXGv!{qv_osQN~6w!oLNRZ>~=jZoi(7~$m&q^QSe&=QgEokYa z-%l<}9Z>S@u*-&UY}`u9YXq4fHrZDnxo_N%3-b!Y!Fl1Kynkdr^7(3Ve%=}HLab3s!-8}6WPk>4hD4iKPF_3kVXiW0fngFnstFJi(XMy#`Mm`l8Xb- z9C<~{Mn2^)SPc2HS2ekhF_L?)?{e2xc81U(Bi>4mU?4v`f`Pp6LUiM|3QkNfD zzl9A+0n6Pvs83~jrM*Ns$)%;YEQ=aV+|L4`#dd+l%X#>v1d^f&5x5!rM#0 zDaJ0!9&hqm1uQC{KmClzjZFjY-<6YYd|QM0d(AGtLb(wP%G^{h*lkX3HXHOf(IlGh z@FpIre1M^lZsg)PRu43MrSSZCwlY_~h?b@oKeT4{?%wC!%I&B+-e_q-U%T5WEDc1xnA!Jv~sy1@6gLQwf^*AuNjjT`T*VH)xvWO7}hVJ-t^;VI)ktKrrm zHZNeZbFX=A!OHlXnk-)6kupmNm|iH>ROxp=@CqghZVEn#!MB#&%Wp#=ibaS)NMeJ$ zP4itZmJ9THqCooiNi0nkT;Wjn*1{D76Py-q3%@n(0@NF)H9#&s`Z97Mc6TI9ETQP8>ic|5G%BS)VIY-6Z!Y<+_eq0O$?7I1% zYjKm$)nI&PB#cVumt3$<$#sdQj)r6iVYVE!?9JBCTx|76UJIkUgf-)7>yj}ush(5P z2ncf7Oo2Xjo#%{HmI|n7Z4hz(!d;yeuy&s7_frrQ9=hfaUOUce$nTyHGy(grJ>6c} zN6C`A*X-=q?tC?h?gZI5VNT4m{KibgXRmPT2)nI`pZRKreBul;jGK~~1rc-4I5H7U z2XSGWOLX7)JxR|L8i|?oM)y;six4sT(7~%=u8&P@xX#M)6T)0_!rJG0D057%7QnZ< zlKXtdGc@s2Z2#?bX&pv#T=pus_zj--cDINC#I(Qv1+COSowqOr?$ z0c*_WZNJWgthx!KLFvtkj<}#hDdfj^ciH4X1>WTaz&*|DXlz7;P>yY2ZNX&+WFbb8 z_j|py@Csa{Dk-G#s33)rGjWNJiSxXb+bw?LNT$)de{r+ypXQj^qa-tCVX z_%XY-9+z#adhPiI*tp(w+_bMhh4;DsM`nA5uIt9?rHwY;ltdE+)!^;X(_`M;4iU#& zl-YJ_Xva)%U05R})~&TZSK78b1$Frb1{K7yp>f~Xs?ivc>MIe9tuG7I}VbZv1DOC-n77RQUE*D@1P9XFn&NLK@o{5}i{9xd&5 zAUO8xm&Z!=Iz((o=_Hn#DB#LJ^engf-5zK_>Puj`Z(;lMTVM$eo?(z`Pi?<`;^l;u z)^QfkKeZdFiiCUL?7b$YLOiwJLOOTe&@wT5)=F4!=Uv8_f4g4JzgJ`jT1rN&$Qy{5R0)cpLWD5o1_ z%aAE)AhDt5G_qdUbh~yo33QZ1(9KIzD8=FCuNZBN1hZf6)8Iv=a6;Z-pu?t$6Q?bDt4mn7?eU^EOtvqKLc3}zp9lfZw{pYmO*2K5nLhNaxgZC0L z)1ABHPCUz;>{jlDn`IIb0u9uId;AWRhSX60M7=>H3@XE0J%m$EW6avRN4KP~G80CJ z$^SgvgBq8*cBW!z<9Gw|%$p%F5j@21j+-UOor^EMzG~O=Cv|W`#7nL;-CM~?4nL5q zPW2m!FU(Xgy<+jYRs797aSpp{shwae@vP_C&Cbj)wGzsl!{EuToI(PewcL1@;E@2A zz`i?HJa3@6Oqtm2%zJQSFV>b)28BZ#*M^aJe2qFw9P-TW&_?xpB>t7?(Crat4}%1q zpuy`5);$pEp$$Z5w)n_oj0I@zIRTQZaMd80!Tor)!~@L5#!QKM!$*%??!`!)DIL#}GdnhtcOO;lWra8_nGgcq zBV;aMR-2LdEi-{>TXXED%5-(M)E^u`jj382Y z>E}y|f>i}B;7)|wIuV%o;Se~@f<|8VJ-;Pnj$6^Z441y~;|aZqw8%DA{9Ba=yY=qE zQ>bvcs(#>0xFzInEwIe{<9R$EqK#WhZw~+#6PKlD;xJ+S~ix`rT2ch zU=Dm*g~9#qZrn3^Ch@_WXmXP|WbcWA`{v5&E#c?os>zM^a_H|w^S7_?YVL8_;oZ9P zLmS+!OCDGYw1R>7E(LLt-b-A;{2znM zk*dMVbTYF3U6WANNry;C2tT?tGJUa5QK~RJGwJ^CIJ}V_$=>5uRKFh+&vHG*Q5h>f z^kl%0@n)a@Szd_{$|^D^e|(8in#QH0V^kQc=gDz<|u z6W;56H1CNDS8~|tqd`XqpV}!2aOQ3+eYtvLx#(%QoyT}zwsU@4X@o<pZw zx-VT?$qdk_aiNXBvdvQ7sz_eeUoD&F&lx2dq`Gy64<)z>Pl0dR)$6zDnR+Kq8*V&M z<6yhfKm4UOF0QrHblF~t6#2;IHtwVT7~59*(Lwcm>9dSFd`**(x=w+hF<-cj5IKnY zNk3@!uMCI&MM(o`+h+DGg_A1B0s~H?4spPG^vvA@8kNEC8n()5u&qq*_CihyaoDl4 z+=Y(K`gVW%qa@F-_n1NixeUr~2}a{CNeVK8c2=|&$dXI447vZYf0!x6quB*xl?1EG6w9w_q}-80 z*cmx&$_V}N{g2hBjwM=}Pxd?dL7U>Yq2FP8dljgp zDpX3@XH@8*z3VYTHg~&7=8&U;lPg(oiRwds_pVHe0HPU@*P<>NuJ5Tvgq!ONYZ36=AsT zY_+`0_JD4C9Xei~At2lH!xNB%?Et}D`TB5j^V74X{%gAyaQflCCWG?sb^e_Wqv3+4 z9K-Bzd*T2XGmEQ3_h#FUXI=Gdc(q%F3;G$dvFaIV3=1DqYCcJ$KVKd-1CpHW5Bq4d zZm4Ow6|ykFbfj0IC7h0}-0JR6w?x(&!@M#;T&+OSpRVoI%r=u;oyMjI!q%S=G78clp zgo0E~1kB!b!y_Wk+bW_9YO(tBK4g^S zM`4FMF=V$4L>8YK-(8Ts%`P_WXOQ>3>3yPCt&qtuO$^MnOW-Vh=Y)8i6(e&B?=4s) zo3V%*r_SB)zJlA?d8>k_a%n8l)Ng(!Y-lYLNH1wmF(quc7TQRRrA@Fg!X<})zt&(x zimpl=Q8E{&^y*cGWHZ>jP$15C<) z<>x)mMlEvv6;vBbbr%?x1FM_RH2|pDz2sH#!hyj_{N_?iV+~c1-i7t?U$MF~X;uE5 z_51>p9naf*{gV13m9lr!qdJ&aHaoOeqPB#cg7d!-YgmWgrG!D2N)ubz+6yD9G6GSN zsx-q_?0YAW*O++Cruf4iu_GgyE^U4Ob|WV+yeeNL9wE(+n}@Y+Yr?F!9`4SYw< za^P<3efXHyL`l|$Qqg|4s{4@HVH{jCxnlw;hKyr%=QiYbR+M`M_@oq6KTXj7x zy;hSg;zCyz_Rh&#w5sGfbXx3-e!iUlve@d~r@Z)8Ay1u1w_i8ii=ApRg6(QW%j;;s|XYm(SwJRr*n;NAVc;zQMa;aGsyJ6F+s`0BM=n>rbhZ?y( zibt?`5V$#=zV(M~9k&h!5i#iC*F|(-%mMRC`i`Ra~U^b+a zT}^a0^Yn|3X56W&#+Eim5v2^0gNTnT2R4(U0ws!j@2}wD^5Q*VdklUYJCpBMSGl() zl*trBB&}gAidY=)<~$M3V4)F$Ab)I(+s^+?g@@yaj2Qf*!Aud|9+o$r*x-o%*F7Q& zoPmfu~1Vs?+URs z)^nP`vRIomtLG-eT3WCiPjiIgX<#AM6*8h_&3S^-N2?;e!hiV{$_^U;=AkO@>zCUC z&9vzIeMIK&uq_U+?TMYVB-FD;tnEB=>n~ThAUJkcK`Oo$H?!Ou^mn@9_){U7RO8;# z`BU4zwKzA#sC2%|^kZnbq|&5=uW%%GO|si>ut0}nHmx|ge0O6(VY%U}e=#S6a=GWw z@z;LPV$IR;D#b?Dqyb6f%m>+V*HTtHZhtsc2-B%67UV|OCxn68fRH*Ch+QDce(v&_ zd1iV#AX8!iqUWr2yRC<#Z7WpM@K)GjI zR>(wZL?|0!m!H~2K|-1h*?i#;wkG;bATi7DE5nw=E*sXT+jxjYmM%;T z5dk(e0B5G~4Y~2l4@G=4m^#W&c5Pvusub(H(JXVgJvjM{%OQ|6lqw`Bl|@)uUu}}l zs!#sS`incUE|nxi!K|bm0qF1Er=24 zM`K%_VuDR7JkwBfdcpZrd-oE=(heU&+gqtN#P_Zq45Ok*<*z+a$N`frb9Nh^y~I+< zuN-`W9_+5Xdy-afmwXZ5mS&&Yz+^IeV7njs^}=$P;BX^ z#S%H8!DZ=J2TMhphD|2Gd0) z2n&fkUUD+yX)1^-RfAohahhJml=to)FL*FkfWVDc9W#Jjj0ClppCYZYOLosJdLEAn zJ}}y9iu(yjT7sx^VZ=CHSC*~UPkwlJ|8J9C>%YBzU-pr!|vU@i!N z00$5MxivEZuDQUqh*{s8zg3$mfa&oT7!Mn8HA^c=Dfi{uR@kKjZ|#K7!s80m_%$)=$XV=DokLr-WZr$$!#|e(nQ%a52-u1&l>`m7-DU zrnPmL=VvS{$oJx>B>1vyMuxRj;ESN_n}5v4yH5WT-wRkaQ`K_*T(DC?(2ypSt>qZR z2d&>81P+QjoD+jFg@#WL{0nO=eydjI$?;hv`A1}GP*6E&!Pu{m%rg;Rm6KgNZ;y(# z#JRfuV!`XK|HJS}Xrfwk*Pf~De9hns9RN$JD%W2JB|l{>fu54X{mGb&q8g#^#}1jw zWLS7iR4Q^sm0g_Ly;O71P?Ogr(%IZuzx4Lcn{{K-*E>>2TcP-*pB0_q70v-Z8ckAr z%b-UKIT?lAzVY)b)v0O>xi-8uVs~SvesxMh<;1hAL4uzCDQi5q!7(cy<~nti@i(MP zR0rCaU=-5x7ca|Dgw#xeehVwE@^PK^iv-Q4`_J#DOq&vNRCBj$3R-oINiNeZ_A#+VsxhcMg*HcPF4w2Q(c0a@ z<$v3JXE$Bv3pv!v&rIwQq9h&~|C>sQ41!GV+?c_RiGf=qpJ+?Gym^rUhp@vLcCbdu z>V$euD;Zh9+T(K5IM%<{T!(kLYdegC!wC#4p8NG}XdBbFi^J!1JV0>YQfE|6-ZZ_H zWW(wH&N^Mg_@7~TmuSZto6{o&7Q}h!U~=45CVJ;KgZuBq$ostJ^eyr%?Wci!=Sd4| zXKrC&L&-pXQP;1lK)qANwDp57Cb#}FWf;6$zDQTl=c{p0@Ee%Y#yFs)$0mCpMt?RA z1rvO9mmy12mmRa;JD{izS!>AP?s6pZXcx@gxkB?xvo81_2M2t@!G~pc`8LhJ>)x5)r$s zQyo+R6A^0;ooM!~)<^#}1CM#t{@59o6h*((oUFZ6F4EC{wr2VPF!X;f+ceAODWT(q z%1D`gR;Uq6C%lsoUI;5`ixsY~0XF4tPpr~O>V09$;r*|DSn|;%h5RmBYr)+AxPoVn zcf~#!wEB-zc`g5c=v4Nq7;1=XYCgm4V3|em&o}%}Q$YV4uq0bx&ue(xt5!Qn5W}9! z!cPju1_n)ABr#4Mp3}f-Ej>(m$Oin!<<2lb{Bnl5W7oq+EkY4n&;Pf_x8mXd{PCS; z_ZIoPu}+F{i~1KY^Iu%z6bL{!?!Ct27~v~F-~ThZ{qL`HpnwL9e){)1{r7*%SicDb z-mkC7*-TRzAs&!D&$Z$Kq-O7v6JA!9_xhQ(|_#Tfl6sP zf97%30I4D$$3KZv&sHY={E;a^pke|PSw5ANwyyS#DX&IuL z;`^?`T99FxC6*7u#Jy~30(s0B@qn~CoGzrN6pM>>?gP2aA zp{5|0H)JE1*}9&O!bUN-DhZk7iiKF0kyxb>3m82oXi%sKu@9{@)QK!^twkY}z(C`M zZM!hhDyKARw&r2=)r>@JA3}Ju_u!u!uw%*RtJCuNWLE2xFg`1TmGX*S@seEGs~U&r z2Om9$p|IL4MH^hK#HOxynV#0YOzTmF(a6V|Pm^j6l`cty?Ar?PyA<2`@uWW|2m*d$hD1t6A_ z88J4CPnB&z{VdCcHnUDWKX@lT-$I$54g}*f)CYC`UxEzai#VZs%#p@(7O8nUmuf(TO&1o z(7EzB5On=OeH`q=O`?NaHlF@}3#0)pX2-H)3M%|0V+{Gs<5}eekl*)M%Kvfpqsm;@ zFc1D@QObsS`Bg>J-N(Xq_xrK<$Adg3Pubtt7nz+FMmLib4@j5-Tdw5>r2{_f>dw_NCC=5YoVJ_)18#zg~Gmv07IAnf7+K6jQdL4jmYnP=2kfL(;yEUQLR zQ0=c6(1rPcT`vJZhhUM4r5?M4B=_#YrRD5pgJ1}%HwHb4?me(A+hwzh5fHM6X*bTO#IP3EBfY4+oPni}zlZatmJ7STQ- zhb8TqZ#@+KM~X2RAT%qMlQ7*gpR`9&V1KKj zGVeq@PUhjVo9L;;IQ@z5Y3-l1<$8VE&nL!l?>!MX%P^G5l86o6^4Ep1mI6t~edYK* zk?8=ohWpU=-U0Qd)G_s;^g$bs%<)e$s)+?fibK6rMykPnq+iBQdLy5g9O-eLK6?!e z&(O{G2+Qe!hw%x~yFYSLF)oEA)g{liVB^~pWqZJ07GW|X?{VhQd z+|4sdEjb1l6h~x@oPXD*S!3&BU|T`U0N7uzSbjexa}_0h{t_-ff?S(lbT+!*uR-7R zjXnLnyBNvQmE}Lz-jIIKj4h*HQ7On2F|ecUbLh^p(+aD2jnV?dGHYWVl4{1_d~v1nB{-+~8+<<;@QM0rv$GET`V#~~P;nV^p|T-rD`*k6$? z!1oE5(zB%_a3|kFSA#+80x4aqbsL5}1ZL}tFQ?C*v{i%M2Jb#nd#j@M#13a4`zD*F z^c`!@R_Yy(d~9~tQYQko2-QaW9l1RjVA|1ALu%_+*g|!E1~bGnhMLaTXpCH` zI29F_iruunXBEjVH%ewHLg{f81hEY)A$c_T3C8kA$1 zmA&YD=w}4H6VM}eWQO=7S)C@?Wp5>=G>AWC zEwb;dg~f#k)mCp93O4+*mR{NAxt&nf8Pvr3{QBEF-fqIFQ@Ae+~icJe`PlAFyOX%aBuKoDA<)M_nNLYC3;(=+7W zGjAkJ@W7+q)wQqnb)Wm!T&i%10G%f((k~M=6n|^p&m9P~e$2?>qI{KSDWQ@aTId&C6oSW&|vqo*59!IxYH>go% zUHh*4KKKaXKXpJe((APO)L#4RP-c?rV_Wf-Hm!H=vc??)^xk1R)a?{wa|X-_8^hZ3 zN6Op5mi->k55QJ3JuHZE#jQh*@? z|900xB}IX@c34@2TqQ{udo#Et0M@>d43=z3$jPf`Kd*xIH4a+mlVTB^Ur3a*30k)B zZm$Umb=WW|5@}{lP+@o`bAbV~#2Sc2x-uPo-&zNgy$*_TdTwZPRx>;;99z8sr|zooDT*bmdofs*81la>kF zAa#i6ix`(R{>Zt(>=@Jrkw$71)TBEGD`F|$YgHAU2c3j1w89xRfIZ4C|TOfJXb@Ix&92c)2eJ~@}Tw=uoOVE z#{m81nJtcJU}=yS8PH52W>8Qy?q8Mn00n}5+UXpVGGV)PAAE-CAtWJyIm?`4RbPLq ztCQ%p+61UP{WA$;`<70?gI~Tg-PdwG=RYg}5$|8%iFn@Dq$a3u#)JGz_yEY+A_|zL z@W2!>FLb8^xRKvslR;QDxiucD0*LI~tzSy$buerd)ijKsdvJ75x8lJTyIg~p=UuPo zn%m*$yHwV%2_eb~r=1}+-HM<(t4~f7#O6JT zdr~5H)KOyOLjV{mnSY54RQ)4aGLJ?CQi)T?#xy-Pe_M2Q1
nUygkof{N*aP>+&s zOU5$BMR2ykUaZRY4;N{`Z@dIS$!D*L2xWY2jp7AUDsnjcqEe#XUjPG`5J;Z6t~AX6 zon&+&0nX;&KLmyZOmMz_WmDz1LDrVTjOpenx`ddJv_uUx4N(dGkGM=$;ckH`!CDtz2Fd$U>+w{7Ot1-BF zi55k;qk}(Pa(9^$mzV}?mFQQQ_(y8`KDH@%08aI1^~iu>+fJ9Fh=B}R{gRY*pMyVu z2CUvGPZ3`9oV-q)m<8OT4T9Vu9KJn0mloD(Vf~=@0p$_IzI$RhG{a$Z^_ZA1eEiHLyoGfM<))lHMBlY=W~wTf7S%iA!*k;Qq&b)3F_ zgl>fhMM0Aszb3!bhLde7QA1{X@8>n+kU45IAGn7BPp%9=IYsea0YJsvnG_KVcK=Vr zG1QyQWE^Q(eEoZ|4rek`|6}j!&0y^)9=oT(LW#K?sbc-@>hzlvd}c_;&AW%6`nab?;C@?#yHY0LgqQ2k?}b z{Md@#jCa*rgIPPEmc|;?V($)*RZ*x$-cxOb6?I>ZWt~~YHNUn}1jbv`vb84B{kzYZ z7A|kkL#u^J>!vr%$A8bxxQgnSbT`6}db+-xI6A);CZBbz=yl`52?3h)>n}P>i`G0Z zWnEhBx$|BnvY5heZn!tpLn{4c_w9hZ{AdoX7+__TyqRrvgM!E@;L=kWM~dm(1?k1> z=9S^pp^w;-Q&p)Bvtyyx3IcC&CRe&s7*sn&H`E`K6^gU7^U$}^O)r@R2Ax{D{YZ~u z0VVw47i2glMp#t+Vle4v)^!!4>jS8FfwWSB{_CD7-`l35Eka$2k#ly*?2j7c4nmz~ zwcZ7$$rnE8C?7MJ+$DXyMgC;G@N%lG_AvE2wI4oF-r8w5QZR*O&-~>Y>vBUaMERn$ zYea(->p`}jQ*!30R+6Cearp*Wv*R?6qW>A9WI!gVedx|dMpH0aIIWJ_hau2(Esh$M)-8&=wiYeB(FeIbYWVLsBpo*)Z< zhLUjv+FKD#4mXuLiX+uUTm=U>^T|7E&2DVRndrL#jcf=GRFgx!Pvn>DM2ezWxJ>1x z%aTF^xb8;)rT|o#Kix<3>nL z9QQIlUysFa(Ar$lv5vNyPW2)fgI(K9K@mB659%Hsw5;WMr_!IjYD8+no5oeGRugan zP}UCLi)HpQO7I7;_jV4jnfvNcnnC$Mr~tk})4A=lWqQ3SyZAs+N2N_@gAt zBv+>*(6tveEb4Pz;IMA>Yn|wN|Ca>>$kLmhd%}*fj@>Ww;p^Ya^o5P+LMsRlL$3p! zw04Y3v;}@%(81XJC;y|;J&;ygqEc76X?E`~zOZ|(m@++Bpsb4jr|E(0M6Baw*TNZ0 z0tLq-u^;I@m+Cj;y<`k_wU%QJFK6x$zgerXwiceQdv$rFFDk`*eBhD;qt#CzW?wNM zvvb6guY$g?T!nMdyQc~DZpJBa37o$|p0m%>?K4X`lYqf*v1vuR8Jxp9ya*A5pD8VC z61tdt%Piw{8`O^JLvRJS_Uqd%@^Xu_=*COQ(dX<@iw@|=Z4+mn&G&uYU!t>HL8DtP zMKhl=CwtOut<(Rgv!?b9=v9^@M^x=}@jmn>x6|$ZYPkRpp#YxN(_bU=E;hXirtRRB z-)Y+q4qu*`KCPzY7Js57GKyGut5tlhoeoR-(CMVqs{j*uTsFy^8@JqS(x@pfW_=k( zuXpz=DO`dkoA86{>EOii(Fd`%^F(^I%*q{`yWMe_D`i|naq)xJ#k9#NZi-ZhKYy`K zzaMkUV>(Oq3l;!$%kO7;&Y!OaZ9@&Uibvw|N4g6+dqHcph^Dq~HFl&a|aeJq!^mHzkO$T@{xCLFH($#oMucku(ge`j|(w0*i7;JQP zc;lSi3Qo3sQ^~d$_0Nzye64JxV+2kaCL~=!UyJUBoIAovJdlL5hi94t>7ZR0E@*`~TW#04 ze)UbREWY8lG0DYkZg6e2vN*$!d(rDT&e^2A$+~6RRx_8AdvRKbI*uh)%WsfO`NjW1Z1;IL{%m?`v#N}*CK0TFDDEU z^>7M)Eo}QEhzSflT@AVR^4na_4ogvc`0XiUUVQ%Zs>HvO4dFOw} zuM`pG-1hnFl>%GPmc1$ExZ0_tXLPgjdwOj0Te?h`>F?nWVyZs309Tihs+KcMEA}Y=2(2G z2=_IkX<p%JSTap^v1=8y;0OTKL5N6voy&t0PP^p(g6q_sXDLk8RCWP;_9?l(RgW zKJP?*}`v;!knM zs+@^LdbWdAVEIY*8+#FHyPKF3^$RpzpI6oBTey)A4*iu;m*o{2%#ZDCzd9t!qTu(dlEW0!X>>f1SLH7$7k;_s7F%TJ(ZW>JM%lK3Nr^^zzu)6)UzEOKskOi? zG`FEU6~W@t?1^E|1QUJ01#X>U<+|AQVj_&1=^!|rekZ!Kp9;3p=x0t}T)%y8ET2^y zf*R5rJE8C{aA;mesC;INi&`-m7sCH}+UTK;Zam1fnEvxaE^+@Z(E1gq4ZilVDldZa zdY@9(@3$NgmcAk~Y2~w6fDIf*fPvkU=ejjeUik;>3XX3?dkX=*TY&t=KFS4~?-kGW z7UG&+6;;air`#=)_5H`v1Coz~*V^q?!b_%ewhWl7KDyMAdWdnUvy5E|s~o4YSmC5g zToqRClF7QOMgd?2M85Whpgx_EaB6^`cM zKQ!Q%2RbF5e^T}s7dP`&PK?t^O(}ECpO%r3V~V8ynUb-2EdPp1S}Hj`KU+ZlRa*A5 z#kV|mcavG?1nqwyL%IK?C4b#NKca2Uul#u7E!XEA#>Mw#r2d0x@{0qttYEA3nWj5W z<3ahYn(hW-%^15in8|4~F@u@gZPZagAFB$)B!#2))1aL74t0>OCd*T^YR~9r>CX-M zktUplA7UgAi{Ens%HGzy-a2X^go&r1LV%cC0eoZO%fJGgA69v3bESpe*veT<-n8=MS3h z_!*Sx)tNTEvN|1SFZxM2*KXi8q|`K`Rl8Zp>30Jr<7G(th?b7(fSRewoP*+P{Ephg zfxC$U)sKRrFC|}{_;nf1rRJuG=XOZMKqFZqU9Yi75l%2+uONpzExNxw*l9&SXuiNy zu*zSW6hHohYeROg*-$NmIPLTGnGoWS;oRnc9P<=^sATSpJ1WcR#{XjLt>cKf0?@uY6a!Q{uVbTQ% zPBTT)&x98;B!0t0+7+pLZ#VhaP7VTiMIi-CRW%`QQBq6NNZX3R7xYgahSlj?2BwtH zrcK`Fv!X`y+m+f!@e03P%%^SdM4a5!%91?>+_CN*Z9+2hwdR0OS~huq2{ByTFTEt3 z?k(D8b}$x3RYp^z{t2I)0Cx)0Nn%7BY3fd)@wdbaQ-|gquuf-hu`=VBXtxKIh;P}d z4zU{0SZ-{;l0`=;EK2LLHk4X3L#eR@7JDwSqlszljqrFtl7;?*SXQ?#OPdj9P0k)ZXt|uJRsR;lhX=>e> zfZEGf5C`;*oj7v4C`xx#1x9xKE;Ha3fmkpsA%a8&OB%{-EuvV7W>v?x&ki1cU{&U7 z!>Dbv)2CSgE)PN+)p)~;CSGIS1u7!_y6_si1J`M11V@0%jx~MBlNgycatCCvvgb-9@>leZXRq5S<~QOm6$?in6}XtT+z#+YSzxd+b~67zqJ*+`XX_xpZI<=k zORW3*V^&nd$;n$xp8Q)I3=Fuum%HyReO`p*^JijsRi}p*-WZ%}1owacPm1wF;pe|@ z1_eeyocudZsMo|MJWBjVBjlAg(&MD-tH03q>y?qkABQB0bt6=??EYgi0euOU+CG5i zjn<-?|2er$Zg*aY_mu+;!2yuumbdt{^Bbf5+m?Bj(qoG#IaK#N*IE6-nU9g`CLA?g zQ6wO#E%(1>kP50gLY>h9+nW{|usInUWOi@+0(|=CNZPnEh6FAccZ3Om0sz*5V7I2We2^Je zF>u32J53OBi~Nc?d8mc7vv&#T)w>(PqTIW|NTnCk=BJ}fC_^@nYy3*E8z*m250;*8 zSax}FTsPMCp|)Ygf327(3!vRQ3hUAbk~=dUO51PxvC}@3-)%k`@?{Dw%FB>^*>GZ$ z+7+QRktRB(!%9v_LR zG1slo&QY_S9d*kzLUH$P_zi1!ySwL{fBUn4GYa!wSKzk;=`PT4`~AP;7jVteo$b9G zyV7yq=lh`+qQV-02$VOx)0x*|zO@xNRp+f=mLx?%5xQNmy_W#FVXtDL9rr-(FVVI$ zs#-=;Kbe_hGtl{Bp7n3yhf6dK#m)SJ-j^YTikaYDPb*kpr~7M77G7huJZ2{n&oo8Pu^6ZeZzzkR`@i40_0n+3BAK6yBQKA^C&fPR z9b!4VM_xp;y8&6_KEIY(VI9F-QDZ(MIaZ}3gFY{JE26zmepvh~`qIM*2Y5k3fRfUZN;xAX|l6fgOOoEBxf8?BgFDH8Fseo;SQ^79iH_IPiBs8$hlLRnp7?Fw(w} zh~!0Bl)wC9N8hha_r$oMp@6JrWqBDn+ruM{FGtr0_2=375#TWJzTN4=yVwGv`h zyuO3d9md=2n**wLQQ`){E9mq_@tCSmuouK>cABv4}I+Bt~FpsWTMXUHBMTT^Tp=bJ| zmcv#<07Yn*FvM>k%zlC8YqkI>?J6n8?~bvo@W@SO)T`u+pPWw?@0eZDILXjlx+C|t zTxRVuPl|jepy4%qmHdYHz->%gEBP3Ru!)u5S8w&z2@}g#Wmn?L;dBuSua?=WS|Dv=(2HMuTd6#l ze1z#EA})n;&nGI}H_XrO#7CZG-`C>p96tDx@2C5q7+>c9!oV?CQz%qii1vKjqh0Q6 zzJ8@7HNf{wgQ?IxO8{bP>K53Nu?#SJC?nCe61j02?i1;lT5?CIn2`6Xs?^D5r7=vH zkX2&HKvV3ueY8>j+vrF!p-yZ96riNL0!ihN$;5Q+K~Y~S{BUz)M*dd>Smq{6Tol$ZyH-=uKSP!RJB9+Eb(Hv9olh zrLegWuUO-^h7bV;*pc`w5GRN%-EKIMq{hjAZ3iOhd#Gg7B;p*P^z+K3MyBk?YyPpb zD(Vrmh9-NwJ+r?7UI##b;{@m)TIaJB54CwfirQX>a2K#fphBLd*L3Zep#-S05G7sc zSWfheTiz}FzioL3LHp-@(}qgA&YY|G!gH@Z)4s3qn~|uVU$?t|#l#DWihQ2L`W1uRrN?J>1)SovbNUdN(EHQ;7eIoEnM3NM-OGIe6{1 z;Yctpixgr2bANQ^G@^c2=d{w5D>LNBH4{_&Mg|;_qfYis>m702DNMWARt^_4{_@ z>g5bK0IPm?PmxpY-8!$?k#1(lY6U~3VOQ07^>}8CQ#n5a>q&lPNc47cjA+toJLA4X z%c@n$Psz#;2fIHI-Z_kxB!_(}zgr7FT~VH*QhC~q2-h@bH#oToH2$=Wp5k8JZ8+Zb zy)5jj0>H#=P{G5B$Nl;CZn;}jZnpLu$od{hnq#n9Ay4~=v5P*J0j%6L9Me02cC9+K z!&07&f4H_il-*nTjfzhOQ_hA?mwnExN)wxYQMt4&+27R~wQzui`wa|aGw+QH6g|cV zlSj@Jm#I{k7mWfSO4wuq63@eWj~l2u0=u+@OiV2oNokwfs5(VWVcxotE&Ys|Nl=*C zAGk>%7jhU21IP4A>TWhg1;GwXKM4CAbx%<}$_FXZ5Hvuga7=vr6}z{Q2aq;Wt6Kn( zf&^CD$=A~zfG&g$BpsAx+g04v#~1^woy5^z%eLO{8tp>>k1S_=Ft$zYe5kGDONRdw zZM?d_F4ULOv*kV-fzuIU?na7}a2H9>hv=T?&JSH;etl%PsJ&F#FTF2?8+vw((#R8g z8K{}nO;WXA8@O9Nzi}{!0`{xIa{bK>4fh(&L!iN+(W^s|d%(KXHTYL(p5~82k1uY; z!G?KW)S8E^l7sZk(fZeYK7hHmoXhC6lgd9;q@!vElL}6Nu=P>caT<36Q2e+m?)iG%eeW{R1~MjxySGJU{@qQ^t(uO%02z?QQ}+S2#fZl0 z!3SFTZzn)=#c2I*()PxGdxK#pXtXuupd{Y#4o<=QQ_@}Un2NPNbUn`lqw1D0oA^om zBXU1<^Qvk^X02?g+`^ivj1GofC*A3$T#LH!JXnW~0hgg!{6_mr20qRmHF>f; zh-jBw)GXHYDEWyjeYBQ~jl`BcuDh7l*>YLP(RT~D1{KXQIia;fdO&7?H2-yBg&J~S znLD#zMQ#1ylmj5XAojHftjC)`VEy=Uw0h zM6MIU@~QP0);JuMyR-YU@Zc7gu-8~Ol1gz+TsVtRWLj0zLqGq(j6gJ~1Y;|;%#Uxy z#feahK(?u5b4AdY7Eof4h3))oc&nYj@TI_WI2Q@5diU`UvsC&yL7M#rKn8=}UmBpk zX}g_!!XDmp^u{aHQjBtW4p(-^_`Fe-$L9;+^HIp1JrRP4=v!mlS89~Moz*&3x-#e+ zW-Ej+muto+FUDP+w@G_0X?4NuVi?1vU^PCyMF_>bNIkis*GVC)M?6A>EkId#pUb{<4UGpjk#+SiFS z{Kty7cP&MKKEY0zdN%>wdUd%}D z0vUau2+D7+9szdkiM2>Fu;=XArQH+mG7~o-lb9h;(*rewcUs{DK1i7;C&pmsCbKNO zYV{eVq3OMJ^;?m{;cA_^>RUSaAo@^ayr9#O3YwB398ta(q~8V#OYU}ztd?@@RNW6) z$g$VYpFs+lRj>^TBNHf{e@zVkW!+9AK|`R42>XOu;Sd=kdH{snAcURf?=VG~fuYaT)8GE3 zdhbh~D(5DK>7rQgDZgNnUHO&;Fta(l^{d_1!APOW?%0cIS_B|As!JVO%Y}RtCQs^+ z6x~SO;caKBri0T6 zk*tdxJ}$*^V7bTWtRgN6|GLa0-xA~!z{JdY3)n`IYw^N$WV-Oqo9#iGOcYU0HS%|> z$&${V9Ex`9ePJ_XDq?S8@Kycdd}*$41%>)Vx7p6M##yT});)&1c9`pM+{4uyxGt=7 zs6E{2bnPIixw{iy)<9*l^|-&PT$&fscBRm8Tyz*A^Wp`Bo9w)`&sc4R zr6vU?1!0poYSmd@3FN%#_^H`w<=&Rj`bt(Kq2fb&XcxAXb^8*Zlm^PSj0d{_s!pDu?x14?Ss)NeCDs`4^8)I7ccA=+?zW~@u% z0;k;tQ-b+RBDEE&-skPlcicQ{B|Fc2)$?@9a>gPU7Y0o&el33R7 z>c>?pn)o6Tqi}MrUs8stoW|0@x}1lZ z4VfjCG-w?6TDt8={yRD3t7!Fc^RCOuwi=ZCLQX+hSm&m2fj4sLdqf{?+gvbVxdVHd zJ@#11efys|Iw|eLuSHyE;_7-xkBf-01OX9U6X2;*{DCg`anTe@?LBS%qzTFAj%8h! z#!c{-CKTg$BPwuj)9!{2zi8LM0BV8C1umNDBB9ZYsbMg$_^70a_yf@gB0q}KXIDN= zT@n&wByeeoKXA&x7X);j^)ilKpVdZn}2?_%u5?_~p# z*K`I4eK#3b4D^>q4U$KFB01Jzpjy7(3jOi^NcHLldwPBHZvxMZG&jNDN);cRfhd-FdW4y$qWfSV+ipZZXR)>8m&V=3D zWCaVYj`v{pT+oPeu!U#Wnz?W1U(|1rZPI1ZxAIHHxnO}eNqbKkwA8ICxB^JcKQ8*w zg<#zW0LHFrB6^Ox+ab5_d%d{?%1vbFRIz&b*5pp_D>3B^wa*)_<^xYFf<6ZQZFT$I ztjYByL)PssX`d~Rn9^19t)T{>MeV(E1Lu$C6nmP7Gzvy%``0dRe~u$a|CxIiV`fM5 z-iTCr$!y-V$ggqX0g`3o_L#NU2eGEP4s&`BzZ*D4IoLqt!l#5FwnPs(H5wz}(;qN@ z`y70*iwO}<(=C=@7iIgxl+05(&OJ4IrR~Gm2OIIiJPl}L?S=ee^{sMB`^a6_nsSEwoQ!Gj8%3@`treyob~p zQ2q2cK|b!g9MjT2-;o!v*~if}zxifqCX$s?T&;Dw@CvNc z<2mVXA$sFd9J=pzfG{2o$kQL|0~tM6wjJy}uQdug-TY>Re{nl(kgaO6-J#{0=llX~ zJ1Eq+5IDEdAsJY2DXcTV_I5sjjVP(~`EQ@(juJ-va!LG!gaw^s{eWDH#5%39)SlgC z)8!?L5JZt#-HO$H=qjJ(6$Hy@0n2mmr3p7jmdWC4xCUal0^KDT)EI7+G0Lkksp-`{v=Mvz)8w1!RnrLPgo(*4Z_?1c zu6rHkLG}%|T;KViyM0>OcJy`kd%JCpV_jS2G`GCjw|k^N&ZqyS{j?P&I(u zc-7q54BJ7x>UZ7qp>W{y>D?-TeGNgMlc z7`zZm#xu#~dH>Kj?UVbL=FqisE5oxQl4-k4S@~|czTL=^OB2?m?$)wM`u%_=-{Szua7iBgvxTByt~Of&c*~6B7de! z0&~uOT@MnR>}#;{UV1-A7vCI4jCKsrLej4$#^UO2FuD{|QQLNR@2WFG$Y6`QzVcI63yBPZRg=3!d}UUC zmC4+SOki%=ew5Ssv*=7p6xR8N*m#c|3=B;^rC%<^IQNsE+Sji792>@ECG0tQ zxE$A{hr?o$Y=z6NeUG=|VXZ+Q<~FEX2Cw)uho7M|?yJu`ryyq}`h6Q;7}sABlGpLJ=bsrm%`g=I7yOUi@UrM6pP5h`3V8?a936!0 z89x5VwD2Nf#a+hgbg{_DT*~b9cc=H8+nj-CbNppghk=Rc7b{rcmYjuB*MnG_G z4V`|#)NrAvQc4Eni2Y?{cy9IpVbJMIRmYt2`5pLSXRYhjiLc4TY18jw5hjrAr#>dF zVCgL=5gAg|QQ!%tHC%_MtFCx;BjB9+uNBhFL?j9V{qzyWNzP{HD0^MwB-h+|7KO^C z$EeHc_JSNPyvtg)eurY5Mw9ZK+_smOUnHM&_UdtEi9=!SS^r83zP*GQKx9m1Fv@zi zMk+Uu)JMaDgXtTlqEdDgzcgey71}mSZGz9sipc{DT88x?;o6c!FQSBkr}_eh$FS=u zJ0t{Pm2IC26Z;+18lWK84R-i*sz3bp3S4xv82Zy7-}r6r!)C50~KD?Kr>s z?>@bt{Qv$<=Or+l)q8m?!j>4ENISd;p$CG%2rTj+j@D$2y*id)LYwy<9;!e&VND?K zxH9NmMmTtEK#P3~TXVVefnbP-^-+Ie79EY?oG6$6;;izm6kQ3?)WU|JPk=CZC|JUJ zo|$#58p{Y-{n&XrO&SCZx+QBg2@E=26RMcSSN|KfOU|17(~51lR<(kS<(KU(r9&62 zi2lKD-<^0~3CK-^wBV*?8KLOC9#<{PaTC|8thcy6hMq|X5h)oUb?z?^`FU)zYbq4* znvZ(F8wf;vsvin}G4M5`@N1CL*-r2-hEnx2es1)D)yhr@=i$sDJX2k0fqHB?=4aXckN z0l%Wz-4bVcjY-nJbZs(Xdk02ZD3ron!w3`%ypN;IoyzOE8s2Ys`B!r3=Ae{jF;WhWS>VvP>(rcrnF>8)D5e2PmTA zC;eF(*I0~FFXKVL`C>xTfb4LiF~@9J(0RV_W3EM=TH{VW6(Z89P7O6B@JYrIj`q6! z%i6YwM7&mh1|lbZ98IYl<+!2PrVEbL40Ay}_4zhyuq1||qt%&8TylbG_M!reoesE6 zSx=IS7RP;9S^fTGm^${@pBs*PD!aRTp-UVReGl&XVCy{tL- z+F<2oieszREd`tqrZsIiH*4yOL3HKR-bAm(FKx5qVX=u$)7n(rgu6h*MI^UumvGr; z96f)_ME@{s;NvrcOl-v<99%fcTPFcr!wz$p`C)F9(?DA+rh~ssYr@HKqBJ%v00Jx%bKleP zzgAbsZn>U~l3lPbM^Ui|^dD)}cM~%7KU^ks+f^$6=LEkI&JqU^JTd!$KqU(lg}=G2 z451vw`bZ&_(9S1;m!p#4ihIjJ%7^D{r@#PS8Evfnt3p$Vg8g_dMr^zSyp&3#O6>o< zlp4!QjrD-o@JH&875SGrp<*2hfO2mFymqu8-?-7X5LS0gw89xkP z!X*e4MY4zliRlIy<|0UD2k-N@Sk-}C; z05FWzSNn7^V1U)}`P6rM)nWGa;E?0MPQMG|YBj6zkq$aj2P0wEJPZPPS{B86I2lAI zoU}Z%Um0fo^YJG4eCsgzvoaWucMMWJzEemDe+EA1FrbLS+S9{@69|FiO`|s63Dm*! z24vbc^Jkm&k)={7n_EmwiPd~#Z^ROOm{OJ9C`r{gp$`_W=gKb z^>O5O?y7F#sOWKfp_`z`=p8OWYyWCbc#Dovds=vVFbAGCwrU4Vx}rcP^WtW8=F)qd z2L1+PaE%fyY&P4x_*5ETNvMl2;>47en}w~P^m$We$20D$k+1!9PpRs9;CESA>)1FV zgDxq8;O|Xr31Dq**5SL{6+|+IHPZ&=0n2$Hj|hbYb1R|^Mfs5 z0tp1uXw7`qVx|z+Vq@9$7O2o6#~kiQ0ij!B$V)*i=S(7r0!X!3IpvBJ^fT1$%C}bO zhnTlsXUh)j%~T7II*Pmf5^t%6l{ZeR#Ya~jH$9Vtmz9BK--A{P2zcoJ6>kM^3G^C` zQ6$GLFvM9A-1Q79waM#FG1N)j55`|bpl<6YaRdHf%!b@OTxZAtonrKCKrG4ZrGJmDqbJ{uckHjwnm2S$*_oqKA zp8$l=n8;o`76t?WH)=dOyv|F@&xnOE(w+k*f%*W1^B0_}ESYc)*h$=3-X-I7lTZjYqG3j$}DnCcTyxT^(z;FGwy^QV>F^ID6e4p{_wk`pS z@N=bD(S#DgN_LBoO0k#R39s5VdX#M6biYwh)&#rpodla%9sSAn2JI$?u^OIoxZL#9 zk?p-BSrWK6RUNKIIv5Jj8AiOpEH9xD7WPk%a|$uR1;yUOa#duGJ6_j77GInJR5N)n zY0EX+rmu*=k~!7sbnMwzMh&L~8gvS~%yScK=1_oEADyY-&XCoX+@4&$x47nVYg+lH zbl<9cI6thsFr@i7$pa|TqbVCicH_4_g0G;uznEo zP+Fd_qn1&Hbh8+>MC8kEGPN65kKN{5cURmNubIUU+fgp{6Ehyr zP?Tt$XIY~RQpDu95|#aQ2bf0h8md#P2ey%kM{k&cC#HZN^%cPW2X>CX(4eH3Wv$XRw*w*bB_~bpisqKl|5L%6K zs%>-A!Wl(1XHN|b@(?2fptA8EwQ0L-3goi-)MOPuZY8A;t`E6IjSZ$tZnu)@*1dH~^?`Yf@T&IZi6_o6gl@5~%V5JBWXXHOk5+Su zFVA#2bG&O|mZLhQI2gN+vyE{m#QwQd453mrVcGro8$ls_mG`_&cTQ2*jJYC$CV5b5 zWZ8Z1u@sAV;9x<>wMF4RYsplw(aIu<+p2jvUWiK>QrpZSoQc?xw}gikm=D4|T#Z3D z+EE2wkd(u^UR6jj6C%%4qZS}?*33cuyiL_+O;V~oj++SGB_+;zSEtgb)+M2 zL~1H#@EX+7JIm6$HF^JY(uXZ212+Y~G3p&s0=G5o$G_V#nv=jdR`UtZ8TBI> zR~6C5k2#nW#NOX3%UFdzuz8i5e=UrjfJnyLGt9V4+tgvnd^$$+z!MowIr(@ZH5S<& z&ZE=7%qk1^VXe%W1b`7^U8SWjtkt%0j;?vfc;^J&w&2Ar4R$EJ{XJ$wSZ^`!y^RG> zDjlF)-d`dXs6|^vZI3Obvj2keBii~cOk@=de8(5`>zI|8z;&V(KPKpa>W-1yk25?w z^f}k*lV67llmfBdZGx0( za5{&*eeLPIX4cI?P@LuI2b%*b;d}s?>+Fz#-#l+M;o&gC@cXI_g-X+BIlv>n?(=`M zaeEF&&-&61%OScYN$vfm`-*sXYrVx!CLncirQ_$+g(EDR9_ZWpi%`PP7PdXtk|#=?LSx zg=qI)RK$mYO-lu19fb}JGU*T74Dp9IOHJ1hJN0)Ntyr+x>^dHJdVGyD`0ELw^D3mW zjb4YNlbacGM4#niC=^5mf&Jx66m2v3YpS3!YOOH&;;r%3hV-}wxEacouuY+xC zinKv?kIV&dLk(896#cI0Sler*ABjZDi_&Saw(Rlp zu2vlb^C0E|Cc(W?NqIi|?7*dRAIF0Hu0y}wit+dNRVMX;OyLUW--zdWrX$@dIOq`ONCWc9r*3U6Itk}EPPPhTew8M$8z~RDBXw%Q-g79sYCuxHY|^ zD?v7kYyNQ~zv1eOsQ|!^tDQZRL{; zx)0s3DLqt*D?B7v9_oHrh*@`rU9)Ug1mukk!)77gSDldsGiqRE~J7EXD8A zrENTlV42`ck!_iF-J`P<+h9;#cn}h;u%vq4$z^lV4c{fqLFW(<%qdnK^~3SF(>t}W z*X4=#7pP7Tj^dx}sk6z6en-}-+Z(fL6s2-K2Ji+D*q5p-<#UgK@ z&3`WBD+FpE1t{6Wn{?MFOw#-u9X(dfHVuM6TmZPsnc64R!UiA3e3(p@B1HfqfZW-P zJLxI6E0V6Co^M|o5UeAo7MF@}Smqfyc~p&P zw(WQ*(O}*gtZ9=Osf+bn?d%ZtbonJ(K;%CHb;E__d3)xrTp7G!pA_3zCzl5Q%Q9ju zuAQn&>5UX7_P#I%y!-26tKph5B$rSdgLE#0$Qw6G_kno!pnqZSm|pKCM)0}tTS5We z=a(7xAWf6UBp{S~2a@2b^F0Pmz(ZRAnCXzxE3qodyTfki=K#8+KTq9%YXLL&@Dx=# zxdq&IV~%?X{vL5u1;Gm7cT4Dn%4yHxZBB#Jpj?d^`=5 z(%ZG)8jIw_-M}&7=$M~Z96s9r5XFl73FN~3HANK%25KTFyMbbRNOyg@@97Ksg(gWL z9Y1zF-lJm6ojU=od}g9vr}kVvWNBFueEgE1*%Ni4YZ*u}_+v?5BVA(>Ayw@}Z~ju0 zTkaK!JC@ptQl)-6#c4Hr z@V4;rQ*anHA+=0e{nIWh?sJqYn$M?dMR!#ZwZ+<*PmDAq&M7Oc5Se3L%?^WtjYa1) z7v8f~{1GJY%<53yR1JJo4&H1A=%)6?&xLy#`Isfmqx>iBP)fS-(yOJ7p(0AVHiWTP zG2z1KoQO0lwGgJlV&;9wZKM;B9ikSF0WXw$1+0((JA@2Y6L{;OeT5yEcF!FcT90~xf_fm_3@pu?l~^$V^n)9ef}2XD4^HS zsW^PLvh<6b4{S@4ynyV{Xs~yGqF+R}-qYJPF%DdF!| z)1#YIrO{!VR73IRHxKuYdjSe+A2za{7xuZs|K2&8r)@=7@YAH(B)9s=8b3cLt$!A%Ybw0 z5oCQxT~7$Gf4b8iwC76`xg{Z7)Ltn4WgF+&cR0EyvP5&`y)I^CIo^bgR#$QA!l`hp!G>zQNkf)H>5CKl3LXTGd@kjC?0hA~iIVk`>iD<(Q z{e3~0nYp)R2?&hSR^5KQe$o1g>FoA{*rXdE8Ri_du$>}VUg3FpwA$ZluhnGHEkD6- ztrlnk9)*ZPgh{_(FoD})Mg@MyzWH>%DiNLExDwA4 zqP1BdG|GE&W~BS)2HZ-`+y)yq(UEowCbPRc_hP3wFE3E@P28vfbT!oQ3H2Cf$$y+P zeiel54V3hirkrk+vp(OfC^FDh585*sh`i|wc;MWihxSrU3dL>&|rJLO3Ie(uwC}@ z3C)_sYk|u~qN!T8?}iZ`2q*GlPMtpw50Q9yBfFUKNGDlq>+Yn8(D8k5k3S8^O*ERQ z03`%yErY)PEgp(#wUrd2RHkFfLtaS-BG?%9??m5n?G!VbBRC|$SmmueU-owxkT3@N6v>`-pwZrsYx zrnsx9(a`{r$1b+b8NL23CMgb59R**~bs*l<*=<6O$7B456Qy_AerXPfR5F@~B;Mwk zh!m*lZ~dDl5VQCNJTWHWoT1i$9{%MeVeVDf>Q89VxOF_HP%J)wdWV@7E#1JCDck?r z(!6HHp|XQ0S*r>uX~?j2+`Pki^0kBDmn^jOwQ7h%d7RYx1A;!bZa?0DIP>E2AF7V4 zp-Y&xG1VYoggBwIONN8!0W_UPA#MjGc{mJ`jLRrIB&CNz3`6M!0sLBRDV|IYc2uF2 zb4?9EP*BpSe<0TUk`EgWY*IV#b}v?7FLO#%@ECRTehsKd7K?%CkXs5V1nt&PZqT62 zQnPJd=?gR^v{?(L&s+H?eeL$Ipx>$YCjV;hXVx&-E09;@zza1Gy9PT(_GDWPv>F#q z?ganZ32KB=nrn}}nuTTU|8CqgXd}sF8JACKFlBGC|CDX8xV;tkrRhf!YZ7}!h`C0VCkB_CG(gcb_0Ts*Zc4LT~^ZVTpvCN_t|ZX#87N$$7Q+YM)1WK$|IPOCH6g&?%ou4QZ-QzU@Wx!M%e`}g}-0EB>%Z18gA_Pqb_%LClLz}w;0kjJ##Iv<+Vl= zoner0K><^sGMY`Yg;@xhXocU=zX6^u=lREO1qYsBSeR6Xd*-Ljh+!g&a~S7#47A%E zJ83EMCjtwVMyDO~6*VEzSWo7oelH)=(v-#Q*11BRTqNr_;|sSs2#ZJ<5_NG>^zXxY z=fLp~zxmNEbhrKogPDl32#XC+0>ar92=8-%}>DmxwR|!$^NRipLnF4=b*{>Jt+&wtG zWEUe8$Nz5fneeE@_luFFk$a@N!(LKu3gy^D~44x%Jm z^kFQm%4caM^ns24u$}@l!N9$3Ee%E}#;Bei$*v)E+{3S?U!`bUJ zcAH**3PtVA^WEZ7Q&M~1QLaCibZtAQD7?@`=D_l!sSbWo_11Rv=tW|4y>MizS0!Fl z7P*&JbBI~76Pp)nBCeV%cdKPBPJu^`#A53_O=rxTboy4*!WK-Bk2xRj8U=$c#+k3U ztgLvmg8%uJLh)#9I}sQ!nG!wj1j#Ip%gjgl(($p|+Qqi2lBXp~R29QWiwRfb{hPYO z#Y~{JLP)>nbkQvzG2N!$DNBBpE`up-i`6AV)pJ#8S z++l*0oi#FycmzU{yA=T9l)rw86HCt$mTXq_LLb3?s0@!GX|{D`cD<8oKYiM``EQ$6 zWsZxWnXK~mtV`pxSiwGjU5)`dHd++jHIkxwu;YNK230&CNvqOwT{{O`j)0P_PE{96F|lSN;N`EIkD7F<>xzZ~Jr4aS*Z_b9~rS_H6AZ6j^OzSb)N!aW3@`2d*9f@Sf!Ssi&3E{2n= zyAJlzGHBH?N^Lieo>7_y{1@F0|Kq>VIeo%`|E6UtlDfNUbng@n`zC)ld!xo zon$FpXdWvL8KT9)d}nEv}E{IJy{qt}Pegi~hr zH3GRVwoh2fD89wnna&CuuEfX@tc(^tYJGzS5eskMGOaM>N z;WhgT^`@BG^EG}p4oom=*#^gOyZ+}67{;0c^Q^~VDkDDHxK*e*K&KTd-v;Wz%dF1p z*eM~YGHlgm9{`r*!f)3|g+Qr0J7GZx2Cu$WId0AxbQPbl(;QSjL~-Fy zf4l1kIM0_(Au!91{OxDxvp2x|ci)c2n}xp;?2#9>n# z1u&mdsOJP;W=wPB8YWsyW1KtyLIqU3mt(R&PKk+XTSQRVbvr>5f;G13!&tedAt}qR z*Z-MO{jbI(gT0JFh5|H_KhSVq^Ww7*yVKal^?>Rb8R(wz*%oGTE0?&2n-1WcN(OXkltk69BZvV+v6 zHM1soSz!qq>y{7!i1sc-2(R*+4YrCM-T&iWUzhH(h>P7xUMn%0f>O?`eK)jog7L>y z(QYCo%hAn4LVt8EPX@DDi6>$&E(OB{Djt3qzis*i1UuZN%)Zwf_3XjlQ!g^BBCJ@O zTvd1i3MbE%4_EO4dKOPgeX6nGf(0@td6+d!_4sQzp-tr7O%Amc^zj&qjr3zQ=|=B` z!?&!pZw{%P5Y)o1u`}4``(bwU$&;hZQ{F@dTfz{2vX2AZ)Dt{0*sPHPYlAu;8SJT5 zT|4YgS>$c7y;nDNvbNIMrxkpMn3@@a-TS({6O>X$#7Po~%;3g(f_BqoT-jXoUNvhW zQ&+g-@lNrcSzWW0`{@qn;2k%JvZAR}ZqHW1nU|d5Mmr)-gD%?uOYE1cYH3HHDYO4V za_6&vOK-~@`1AQO_U4%PEo0!jVLgSFWWc&pfL}Up0d2F+@kxS4UvQ8ZgwM1Wa_Tx} zgpTc&#Vm70PjfbrdQ~IguFi6T;c^mi827B{d=Yl8NoER~lz?)NQ1RP1~xNVQ)4KZs026PC?*6ehq0N zQ8Bk_=LJ<#s=poJ0)|)Z%ylE?7eG!c)$hmie`=w2_f)ydB8PPCjHH(&hRUZMo=VqP zmnz2hTMO9pm~a2hU2L3h)yb;YRW*Md47XlLrv*?@-t=3>c*V){WmW!a`8WPr(gA#& zdLm{IN1sVk49u+g?X);ixjWr{C0$p0Zzf?WPr&n&E@rM}84E7^Fzb0}!MIS_9ZPmN zQAKr2#Cz+gsTe#KEg7ijm+Kie2bcu`1LvQ|(#UIJjJbw>ps>E-6O;evj~X-R!?+G< zW6^ChU{W29^ac!T7P2@30#xh0d&i35K9Z}T#|m}F4kzwdZqIiozexIPbOxz z>;b!ECA=wv!?RTd3meiOpT%>ivw@t>z5!fMon(5iARsMFx>QQ(ZkRA& zlyr{Hk%NsIgTZ;`ug*Dto%4s+rI*09=NbDvpU?Zg@7I0&IN01+y*}P=W%Fea)O-yv zggpvljwg}k*ceb)*yk=qzbmM{S^rir`BU6cu9;gV?q*7F^3V~Ka@EdgfLAT+rAd%hPc$3*~a?vaRltLzap z_|Hmhd(gc;xPr3TwJN}V=U5k4qOd^^`0sWCe<%W~v(c%hjAi31%GmuAtv@)e?szdS z(-YJZAjCg80OF>l-oWg7^w5qt7Yv%F>R7g$38w&#|3_W7rSM>yr~yX-rRw2x_xV!8 zU}4SeYkBD(7SNgaq5nYNqhFj{D0PjsuBm@2H8q5>%x(?M_~ zsPOdMG#-c#dZKj_cW2wxO(!!{w{pF$)zFX*HLU>}W68ocZ--SDJitwMqn7X1;uLVQ zQ@~7m056dv%czeO{|YjCob#QjqCsVp6tT)jc?>SCRzObarQYOKOz3O#p>A5pYePuU zMAAZ4r>jSX6X|YVyYaq4P|lIqM1H7x8je5@>V>lJy=YKe`4&*C%p{xX4an=0RN2PX zWBv4Ny{5(6iA)}Ws=a~j%yXzy*J(U82yE3%qz>bKm$s?XJJ?~DS>yWYbTyzY*1U%Q3^pM|SCHdUg9N+I~Hb zvLJ0wK_YIrz&z>P8t#6^>(kyG+f+wuNptHLh8Kq60-x6W#nC_p63Z_942)L6`yFHa z1Q#*IG@-=mrw(%eFpJ6*F5j^kybdkvL&i@c1$s$yEM88>eG=@|?Yv0(JdH&?OD*iS zYp>cmz2ZiWKeMyQ_kx`8ENV(SoA)sd-1)nGyjY<}Hgq@7GQ3U4&S_l;>i$l0a9nX{v(6JCS*V_jq_r5n4r1h5aB4*-I9hBt6SrhNh5$QIry zSl*3FVIRf*gv?0vi;$`TwPE7Wg789>XPBC?Uh8oR5_h9O?5otS7*JEJhAAHh0Rzm^ z2x2#aI+0G|rIDNr(Q^B-4A2*gws--qyGNAQPzIJl9a7@~rB>}=j@hqT+y)N(IIPk| zbC++-o`98O3O@4;2uuCV4WHDtF%3|K4m^M}fG;fgrb#yvBBt&E1;ThhvQVwcOTX;Q zCr{PMY5^fadWwpcue^Fptu1D(){rCn1#LMXc9LPceO2=diG(h*lhT5>>d-eNs(?G* z$+o|^d0(&Q*}@;10bd{qy}7~O-_I1@ag)n$Hv_aO44`+VCjdabpQ9g>7obhz&IiO| zPbIUewyib{l9k3>C-krP!rtqc1iaEpgLOXjS&`(jP9dC~92>j2N=hpWOT$E$F{tw8 zgU9aa=`DT^3qy3qEY!V6^dofhU5-mu3IPXW;`#Fm>FpM-r_y^#C$>@8 zJ;2`4h!2vkH$MP~Za?Z;?g-!Lz%IT`2xhHGhPd5M<^gSgpmih1Q1vKVmW!Jz6e&1- zRox8R86C+dN)5KxDIb}bdQeL`hA061!0ynu%V>JQ!+7si#{^Y+!UHkkN3dV&Cb>(m z3|8njmW}sI#{DPo4yXuT3p)>|ka6v$AO6<8lFyaP_H|ZNRHDiysM`tM=0GiA{H&EA>bz zNz~|d*=u3NbM@t04K)AtFuYA&-0rlIcA{!9^^jPG(1DFGdsM0#Hb%#?HH9^h+T$67`m{XFb?QUp7R+4^RU0b#Zeo zT;P^Hc<=djuKd%w33>^F|d|c zQ_dNpzVy4T@ZT;l>u0Jysb1Nz;FmYBRi+YPttbx=BijCklT_+N++P%O{!FWleKa0x z1mUn@hmh+Lb3dlm9d|XcF;cKX*Hl{%!1m`l0#1^c*ah9UsXlpXVmmWhH@#;F`RVt9 zAGKdmRB@tmw1N5|dj#Oi8>P~$=?=E}D@C(c_}@ktwFMKGpP76d_R}#GiIw8k)6;03 z)5EAnv-scxl9k{oHj96cJbx|5+4|r7E=e^~0)FzKY5{$Cdis{LT}$Dosq=MQfWA1t z#}Zn3XAvR?D(NCnooEGiMj6AA=HyP@;1HpCz zr5@JpB`p;g6T>5*Jk8ClWS}gEFTeBnV)Wr0DWo4&e1c~~WRG#~IfC}L$pLB3? zfI<~)8|~c*DVT|*(R4tf}J3pD`N&>?IV>|={7<;RnW zYo+p`v^CRBub8VhLd%mWi&kVW{h1f5-(0dLgHz*!le-4M_; zxLHR~(OjTliu-^?iBJXjb5wdafBeM)a1It>9jGbCNY<6DCK)YUCcFFNauk;CZC-#OxdWn&N%IfE^*j`p^G|p+iy36N z_eB0#g*}pE`A4-wBj_01O~J!ZQnKRJt9p0psHwtLtw`8Qrp>5arxKoY4RUARv3%(h zGnPxf=ag35+`eYbNZQ1!snaY*4sm!cO`VnN3SO`V97&3BK7XS54=b_6hx)JVSP(11 zj;g$Hi_s!-rO_#HRUyz>HJ8Kwxz;#7-4A_Fk;DiaTv_zvn? z^)%6IQd8#N!+gVjgS9i$L!uuOX?0 zEow#t(Ox#6of@a_NaX*E3INuLSAw&1-Qxx#Zw^YUb=piQ=d&Tip&kTR=Dz9;>~#rd zp;ZPc1`Eay8~>2beqmcu+}L>L1pnDo$MB4S0S_>cs?Ov!ef#*a<2+A$2U_gh^DQ^- zArxqyhXvRD7qcJ#S@TJX8b*&ZfR0F_sj1-~jEwjQ2?i@91|NX9woOB>4WE$oyHIN3 zO;@cR*bneUaWHgM?e?=oG8z2QO#6YXX7Djz*me$CVEFV2*)#hL;ZN_U4=-kkOJyrB za9RsIpz(>e7f6-+LY#P3lh^sOE%8G^(z`rozA8SQl%g>kmJ~T^__$&7tyg?^WScE{ z>Jt2!x8rZc;}=nyhUMU}3pC^kEnta$h^-?3>Hl-y#QXIL_Pd0q4L|m?H#_oQ1*O9d z?aB@m&n>^SuIvpvY5r_dTYlyv@y?g$gw%$43yjoM5eO%fKagSQh zvOcq9bFM3&M$ZU|^!ACRO;Om9-ihHqX7pL*GT0DhK<6^A%nR^IDAoPGql6&oQ zL^beDXazS0Yz~}kw`dzZ2XEF|`zei?=R7qPGzEG+Hm-rns!vzGVOy9+? z82HXrXBHh^v}Xv=Eou7Hlaqce;YNCBYBP{7N(Q(ClCs;4fl$q!&;xK`RZ~Q89oRJl zH{`4(=bQL2_~Db&v$jX?6X=4_81on&E zSWrXhqxTOib>b9Cr-5arvLo=!CpSJJzL(RG{$-z-@lWas4fW^VetY+it^2sMSp0!< zJ=X%|N-f7H(%Gbt^9sBJ&2oG{OuCRfcLRPd1~RsXbiKG+r%9tugQQxUlN_eE2q5w4 zu4sO_{&A)_9)657Lv6~=fsK_^tGp%Jidn1nnN_-Rzh&9+eSUb3`mm~g3R$dMc{kT) z_GMeFQpDKSa63Pg+2a2Z_NN74H*dA9s%tQ8*j=1hmFOT|c*IgZ#8$r; zjGyN&+Px-LCpm3WHa4sOtFh_Gu6ZqZzaYPQOrO1ItY1+Ui>lnrdpBKImhgQ`ws^*} zBySL|^1j1-&W*Ld9QrU`I)0{>4g&*|v7(f1M8p$U7U&vlD~XfE}1>*Uzn z;V-{0BEcics~ZBS606-cmNKc%m*7_*?BZ{&XxQdnKgmg}?`0r3f{PSS0qCOG;%nSdve086Z)C38dPH9mECv6A z)o|81;M~z$rY;0MoV_mvH*)0QB4r{X2xyoAhq@}nG7s&hHI>6WJi{QB)ciMN{^sM5 zy8P8hV|ef5nS!W3ZCPPUvG0ESH7_W+n`YOZoUQ# z1kn=LaRY4|&sLy}Y3=$SQpg%&)!(b==uFdM z=L+qmH`DSVe8!yq;D%7~L&+IO!Qn|~beP;k4?iqg&oLR;dTWTBXIG|0IRt@%f zJMvyK9#It}ol*3T|1e7NWmN8@&e58V6Dg)-&DE&kIa6mo_B=?q{6A!?ImLJJ_w^Es zzgK1pb5y&o@$nattfeELdChdS_=n*)-%0y52Y_TxoQ5%Xc5kpM?cVxj)ndKAmS~lm zk>WN?TO3g$ByEQr@H}X>z|Edy+#oL9?6Sh{31DlTv&fmK-(i%m6^+fRN@GoUev`q| z+3$Z6ua8G8t-QrOOBZpp8~spsh-}1a6#*s=H*afmj)~ILMp2OLm#sB_WMgC=Zcf)8 z)5LTAa!xUO^mB7(;25jgwIRaZSU!A52w`089g1e_wLAvYu(Zh=kXl_kU>#$F>^|A> zbLqwX%dYfYdfpjgJsbp;<1-;OW!eRi&AG4&lsC=dK5dIMY?MsrrYAHERlJCSLPNIg zscx~f_^2^qk7!J&CgN~1*OGxP4M_iH+O;{6G6&yZ(tsc@?4O}u{Mb6`!O#=DrS>gv zf_Hfs{@fKCct&;F)oH6KF(mZk%M_W?dTAED3E?z~RU}J=)Wd-jZu1>~dv~=k$HoDj zg}@Bh>91whSP35%CIOVQ_3mJs=FR!O&FboThRRI-iLhF}Tc*4o&e3QOzL(DMq6hWI zFM}j$Cape!BSre|@wTPaCqgUk~;C zh(?0^xU%&uTYRa;giXsyPNmxFdro!xXPld)eT$zxAA=)rmPBA!C%5S%{?FOqBWjKLvIKGXGY&)Bj*KN>A*vUpO_{$-#W zDY=Nsktux%=U@U{38ib4w9bP1Pve$pVT|KZJOv~D?$L~Fgglqmxo*6%N(@DYZHo@0 zVsXKgU*WNNI#V5*tYd6K-)43m%-dEuTYt|f(lSdx^fx#~Fh}hks0BF2el!)0)ponA z9o!Na8CPx{qPj4z3MK$kRPCf$TTbUm1+Lo#R)x@eG_uxYe{PN&$ze>A3SYIg<^!S% z_ZmO$6SdPi`fW8*-?Yu&x9{I!@!S8iBeQpX>Gk35`p;$}$2!hoX%qe6ga{wrg@#;4 z+}ukgj#uo1eX*Ldb24)phL0wmLfkoezuuzAI9u1KW{C3_W(RWE3KXl|;tnsj$!j+D z{J|eyWIOHa`4lt_wYhoD;TDfUN@=#bv70VprBmkc(!1_-J*%KGApAeXVfNK`I-EZ*uN%4Xh|)^&b}>+H7Flsq;E0;?E=IUh2Fv zkDIv+dx32n>OwH_aJws4RQsnsO^nYkR&3<=P{7y?kGbHu(3+|)fJV8d2k<$JTlql$ zJ>pyy%+u2BpwcLnfc4S{K_j7Kzx(DN4wUvJ=p}R%$ zOhEb#-pYB~w1%hL(o)-btIL7NG)T+PQRsxxT#1!EdG^m*-rgV-8rHr9_U0{HxL^IX zDYti1F3wPD*>DMNe0ayNB42rDl9jiN-p8r4CBb&XfQ{R3^4)M%XI0RxDl&RMzB);( zdb%iv&xcgO$PAB|8ZD>OI|K|e5yXtVN-G~uy{<9QRuxMP6fhP0>ADo}?7sG-mC1Q9 zjN}_4@~q+IQv&qqIUV5xLU46oFv3Zh*QsXNJm%dMVVl)o^D2myxhs(RO5l{JH}xnA zB@Ik|>$`>f%gD{nYGylEuVkSxC<=7Ut^ z!h&l|J%$bGWH+Z&zVM1SN`0~~J(v+S8_?yu7r+=~w&04o7++3Ax-beMmN-oRGdG_c ztX^wb-C0I_=GCYmh}MynR9KXq+Tx+l)OZ=$e{)x-Q$Gp7FDMZNP@#ahcY~_!>c9J+ zBMg8It^BQ(o@oai)Xx2zr)Cs%`d!T8W(JmH68&yZBV3;c=GL=(QbcOnJ~$u0tBe?z z%KQHfy(a@3G{CGYNI#_HV(MwNL?TUgcWu=-7SB670~vRp`ma!B>dE^Wn`uaw@9~@U z1}8w=V^ca#-JaCqjp-#xq9p4D8{g(^j{s7Bv$jmkRME>eH7+x1+`6qzLOK6vR*O_q zWG(gIB)7j~od1GS0qQXyRXv`_yv`^2-zqgV8U9~K4ECzu5;h!FU5``$BV+zQ|MNF= zo~-`{7#f&eKCFFsY(73z#P(War3t5$;04k+=9F#$qi=6s#+gn|lP`7D`#diC4(Z0| z`*?=t3EKdI?AuSHVH2rf=xbS>ll!GOKu1PRg6~hffyIDlddi4PzAi_R7ko~J;_(C7 zlXWgq@R^Q|lN=kNx`xD~U2no_jMG}|QD1K1_dAW#dd5ozB~7%+M>qI(r&?7`el#kI z1(n_sKf%x5gxH`3c1b!Ot+&%q&k|oNHLn>CX!$COx5k|g?JEhU-wo&~c-iK(L;Ach z(gk+W-oA-2MG(Gqr0WKH6q}uIBG5CXUs*o=kBB*^z&a{`f$2)nXDV)d&XB(d}0 zWGcAw)lU5@Vyp|%jKLe7<(Rh#@%oev?%t6pe?$7-(Ut4#PJUy9R8ny5WiIR-H%W0L z4L2&*&#QV6fzJ={C0PXp{`7n{ID(OT_+!j=etj;Yxx;nM@Yp=qt7+I;iwlHuYpJRIWH4Wj8mzvJUUZ`>0G=)uSA z8|kOVR?=~p+;a%wCkBh#$oPonXdsMD$)mg&7E9}VjCgfpd9{WY*cIfbQ0Rtl#?!Tx z%9Xu0N$B*P#BRWVY(aHJjpd$S3c686GY8k3-w3EORX$+bb4R*Nu)<36P4_*3{P^qu z5Ce0)0UrGrIoi^t1PPT!)iVBQaKf4mIsf#dD>VG#H^SfX!8D zDz!|sJK&E6ycg;66f!kTm5tWtsQS!!d?P@UG|{iaHMIS1Jy$7x@izY|p!+zMi>^|s z;&|$8*?Qu~XsKBCR(3>QFCaw$_Ms;O*N<7Ud#;yDJ_lWln5UTnng~}%tf4aJOs`pv zqq5cot1V>$tRt+)QDtISs%WCco3WFCxWv$VjON$iVZS4k=+?dVYGp3b+r?dYw=1B8 z7{Cl%BgNiZ|A|NI1+CJQ+>6W>W?2R#P{;#|9^N#@2*-_Q>VA_5;{b7HU#vY&8 zO(Y2Gac|*0D%T9~ow;61_SuLK%ciA_>2nO4=g8Z?LQp5i?$k_f9l*u)DG<)$_V7A6 z*gMYJT!EwkaL>w(OcVR~`hFbHjEJWaDl3(Y>O!X{R4@(SgK;PXg{qgcpzQvs@0asR zv+#L2;oY3vuXhUctgdP~hrSAr6uKFY+JNVuBUW6P^MsYBL73QkP@_=&}J3I(&^*PUm0N8;u7p%5vKwhx|8gVpg_)@j5cV7wj zE2o7mjk{p_y!W6UktRmpPD3^Ub*Hl>7I=oN0Kx4ZyL`J>Ma@<@WMbFq5X1cfYz{XR zKly{y$PF@BL zDQK)Hr+7R6`lM_;zYx|NsB1F7r-W#|xWy#A8dFJR5F!FD{{aj$)z)7gN@vfY*X4Po zkQl}rYnG+6@x=-soY03BKd7*?MKd|@9}(Bj{6J335F3$Q?v6MiX3fU1FjSTM#02pm zj%T28aI4iS>Qj}snP%^tPM7%M)3o<9rp_TzW^jk|g~X{>h`hLtQngqj?3W)}3F015 z*hGh}d6ZWj+A6MBJYO@pgJzc6g}zuv2Raf(+rSC!5xzk#!PW4?pVjSQQC9BJH>O*f zd<&ABf-2v0jCv(e?~uI?qA7J&#V0)GBk4OKPu<7Ixmg)-k#sppB5sii?pMv{GId0e zN*;mM*p&ce9vfE7m;^@aAo|565bm*hOJ<_Y$$|7+YxEfVxXJB1{D+a3 zZjM$Z$v&A z`}6&EB6?fjOkY&O1ycVA^uGR*@G&5@&g52S25KW7P4G}+JQ5Lg0qRM>>l{`XI_6hG zM1-YoyB zKm2~D2(T2D<^3DqD0)C3XectC0{gQ5wQGj0pQIMqyjth4P2v?51*+~T{0I3)v+{$M zK)?jH=1IQi#G*6wT(4+1dnC88*kRpzn}x&qBJ%SqaZx4p8-@81T6%kuYTN(hE~%KV zG+*07l%){W4iNx<*-`Z}&oG^R*&<%RCw7pEl)wYzpM$vAj>fmiYQ0ls56A)-F(80h zR0%av*0MPzpWp27=aVR`{BuJq{if1xC!414ehdKjSgammhv>5e!;hOG)q+0)d28^p zV%(^t;DC==>1?w?zu{HSPVx{<-}{-B=-=)*ZD67mhl+@|SeP-zfRwqD_6_2|P}$Kd zVKdHmLG+g>^&)-fSL)q+dX5@bH67O+Jkea0@iK$`@>NIVQB-3c(1SQel}FYafo5v= zs`@3Ny1|rX30s~D@!ySamK1iV4ZL`2`SSsAVcClRmZ>{;sGFA7QRAEccAN{|Xz%|; zU-y0teT$jV89pWhxZbN8D^}M9_o7vK$EV06RSv7jiS6cFbN65APf{Vgc;F#oMHMXb z>VR7^T|BR$yNDhgrWzlr;@|YwY;!FLHLmd_Dti%GN!h@Gr{^?HE{!G0>jlxA*;^k5sK6E8 zK=0$E-JfY1NABbt`~KDj57u}wHIDDe7K(rxy+t*TRH?PD1AXRF544Z+nOC=^th1Se z`NP+NvK0gjC<>^PU*PB+qgHj;fJ1&5IYU{HBFx-o23>0w(1FuWRu-S|Xl2Ln#Brr9jzod;BnzJH@d z;b^=U&kJlJVboj|Hkv)a4tYVIkF9R3l@pX&pdEX(m@&mN3RuJl*b~3d z6lwrRvvJq$0~)=^#Jt+tw1Cs6nd$*{z0wM`YRN^1r1yWQjRO_Aj|)%Nw?n)STav?v zRFHQ(b|L)zIsoN7m9k;wco6w?a#^a%#;SAmI-}KFs}FXx-TmiPRf#o59ujnAwrzNy zea%BUql&$yB9l6g)Cba*>hWF1%VL_5YLPqwKphrisrNTiJ4|eAHi;Fey55M(;BU#J zHQLPWHka9Rd|D(Is(vhD?A%P1ApdXtn;8O_Vt!WszcXN;_L%1fPadTo7gRmK%A28b zja_u?`*iOuWJW{T`G5PM>(noz_KyKPd1I z_$5<2QvcaCefW_}P2YrO4Y1Lqj@P-5^-W_UdjlO;BF0W{E6Je?ZtB&F){veAk&FT6 zachWCAc&z#1D|`m=&n;Sl{&M#x=st|rS=1@Iq|VZb@oIu@MI~kYZY?nkNh6Jl_GWI-La=z zRjZRM@e309uPIJ$)W(uA2XPN3HJNUrYdU_@-qCR^z(<&3#wY-0;-i=kOWNGm@>vlg zI77Y@2=u+82oYlv-==f^2&u>)$tSjLlnv5wDtJB;Bepp5H`C3*J|?RLYFDfw+xW0*c_*8g6Eibg4gew zH+kR8{QSts0zKx%+uO$$Ya(*2QoT}@ zCiZUq_3!&ug7rC-TJR^!9$+Y$#{SyE=Hz}FbCd)czksupNG-H?+Bvb_Gs zhCh;rqXlhDO%Vq8oYM_(!|l}6--3SW&OY&-IN~2j+EoCYHV#|g*eBDD7x}lS{WVgv z{96goNSjT}g9Hca@6W^opm8>Xckd^j8V%>-=9=YlnpiDXzs^&y_SG*$y1!&$o-_t@ z(o9Pw0-i_wV9myU!aohWSzvqLgn9wLLm#T?lTVqrc)_JkX8MX~2?$v|=`q%9lm3ZT56og!I#?P@< zfq&ox<3{%m;DqY>@}fHq()Nd=8?N-)ANG3QINrSw*2&-7wj$b#@z&w$L1+2gL^9&W zqo(CYvO zCB=WlTzSzik{-??Js9G(EIE46lFaog$8RLu;KrW@>F$1>UD4&z9%%mE_U=1BxQ1WP zl=y&2Q3heX!GOC&g;0-wF;UQ0E|M{q6Ogz3xOJJ91?$zRkI=mbt!)AbQ2*iNT~f!a z=ykCJhveB2qqenfpma%br3xYk*OvV`W7=}?vUb>870vz8rRnaibI^blYV&-mYpn;L zhCVm_K#;6`ONr1qTX(jpo;xy)asm{hctmij16zG zD=gk%wsB**i45JiYYlR{-KF)RB3ux?>3{RgLzSg#FPefv8A~o< ziyyq88!Mh={K?q(A4kjhd zH%mqmP_k_a&~Dk$lsqi{wHhxSG#wV(Qm1+dY%LBHwjGdl(3pd-RBc}Pk$=MEzRdz9gqV)0bUUF6m1j_{N7B42J2bAnTmV-c1VQTbW(sgq!X!eEc996jd!rU6kc_?yI4q`1wI4o_8T)1UuSJ7S~<8pwW;-nY0 zoO~E9_*U@EO6f(X;Lkj}zC?aMqRYpiQw`mfJjUJTDu zMBL;cj_hR|^fx#*vbr3(rp~ayFr7)Y zW*oB_Xa^At)1UV9dfffCsZM|wOwAF_|3p#HZcJl3{kS7t(EXJvhLuYM z&kF|y3W9EZSOCMk#&&~q%kGJR_b%=|CX)!xQ2yf@Zt=K#3fnAEt_^||*0pMI!C^GO`Y z<8MpvEF-EtS0>{juMS_p=g@x^)T<5RQe(sTz{aJWQA&))QW|85U0s~I*OqJeILdNn zcgofrC9pW-=~XhBgHU8$j0_2>ZT;AhA5z@ZuRe;8Ovsm);=ns!j|v)35BS+poJa7S zj@oXfP;(QOE}K1V%*Ve0C<~siPtWMb;1TNv93wwnBA9!;^||hv69D`;$>hpHS@JU9;aUNC%CsCm)5rTY=Ox#ebraoBZw zHCU#PVAB+so;J!3E>Uoyrx&0BfYTK`FsdfLl&&A`JXgW<-uc~7d)C`)I3AwU^jrN4 z0`JKFQU7iy)(Pyw{Zx1wc5W{>K4}B{yb~RSdRTZ)W{H9g1)0436hxq9(=(j({7;0q zF-BWOwg!lDB1?QeQ7$ql6|*lN8KllcR!FJJQ{D^jA@@*h4_Ul7AUw?fvXoOK0QFlq}GdO)llfe-;iB>H~21IGe=&)Zl}X~ulQ&2@9P+-_oCNn%hBUC z#}`XHx1*Ne$KAnNGbR5&YpqN^Ayi_xC8xj~v=%_3!Krs1>eJq>q(KbHl^BQBWtFRECjQQrFY z^n?V<2BT*NN|xPy3|}{T8ge&ydY!&|@e#7rOR+)k)XyZic$uxV4)UP0a*aeA>_I~` zS|GS5p?u8uHlnT8S09>Pel5dko^I56H+nn1Q(j4k*xMs?KN!!EcQ6~HX6hbQWtm>- z<<;xZQTu7aTdA>Q(yli}okxr9rZ83yC2#T^xZ3`O7m`(LJ9l?PJ=)cmy-r126h~Fo zwwC|RQ7wwEZOfZKbkPoveQ`2buX^&eDhD(=VpTmIucD0Sl2Z*}>ofL9WU@r#?S zcLa>GyFFQ0w$z;X`N6HoYntycIBOZVw;NTd=-10v3rjcL1cZ19$Say7fK!JuSV`)) zGM@=2NTMuonfvOo>3g560-KV@L++!7Xad09aA^Vs)&yi^5qrrLwA*F zO!r=A%PSLiRHTZddp2}WBuEfq1o3e7s+mdqC(S5TydA#MGq?r%4$rPLv-ULjYnnxg zz|pqd2J)*=;0$ph-^#M%YtL#?noqUXq1AFUPUcXYxk$k!C!OD+z^d1|G;cw6d@kG9 zM__4(zrr%A0rv5i)(bf-E) zPXD<98|-+x;LugF{&2eFj7k#R6;%&BZOkr3r4ut^L0?pty6X6tBU4A6R()8j*TjBN z2Ff>WCSd8%ZZ@O+r7B!1=OTkKqv^5vP5#yZn*mP8w_l^dMHJbeQeT$U6JE0X7UHo@ z zQJXMs#(3`6zMWBp&LsP?jltiVzPOD%CgM6XE2B_#PU@@aq|;2y?xm#-{Y3Q zd8@a>nwja{e0!3+nrdTcn>Gyd^Q-oJureW=xyt8N&Ej{Qlf}Co$%aF6O1;ogbN0C> z%`*bv6c?jCPi}Y=7m$a3Gg{tz($Yf?$}?r*XRc&N*6H9X6fp5lnx=E6>T154Eu_Ky zdj2h!v1hJVIk%Lavm8%fc^jqVWM;3318;4A9ozD~xiyi(YoI@U>Wt|cUE}0h!(lC8 zdeh$IDBjc59RSo!>B3xZxBWq5W$QdNAH#jzMuSK$ovUyM-P?w5rJt;(Bfe@QpbuTh z4?e;}8!=F#K z2y-@OZcDaf6B>&ID>mc-5czjygM~?D?DOwdUxuJmPqk7HR1o>=#4I`CGDD zZHv(XkRwH(W1&>f2-%HK1(dtqTp=32Y_+wRdy#Z@(56NSM;!;q2$dlS0pg&jTdC#dbWb!_bw zDwL&{W^l>T*v-9+BU5ckF*1Z;m%5nz)2;Ecc)Gg7WL~(Gkha}sS5Zpi)L1*Xv(D?1 zZOW^j5%#qgl9tUegEgT_J_<5N*lV&%wcBARQdk!)%`qlQWdTI#2MK|gwr`^7*cI;_ znY8ZZ{7_=vWa5yGsjVvcqZ_JClS~ZoV%^zRTAWqBXtyyqy7;xoVU74_=J-!ns_dx| znm4^Cj`R}YS-+bJ!~U)`&?Z^M%4r*y-s_g+0;fVcaOWERU!7tnWozOW|A=`z&)ZHs z8`n!#9rwDYg{|B)yy;!U+&dS_>?=YpjHl~bZUDxtS=pi%Bw>yJhjh;ikgh2j^Mi^0Dj zPZrSQLph}pvb)BiejbxwbwMRboBZ&+$hBUz;)#fO3j_tgYb zcXN;|0sWmo;&8|lHh-?P(ZT|ON8YJz(GqMoS$H!)b-Qh?jK58e(^zgrptTMxnntjM z?*Cz5P-Ef3XN!)!OOt5Zcr#;rZ#Y=v~|X}Rc5ivjFpD+J>&Ev0YJN$ zw*`wgUDl(&V5Z;=+3Wfz`7cUKp{^wYSAqgz@Fp*#Lc%XzlMUD@=<`zqq6 zni-7IIXoQN<8$iJPFbmFVyh&JApMsZZE^XNX7F({E!7Lp>P_}fY;Bn@EO*@(4SMLsjvoomwMamuJ4 zWwc9gX7MPU*US)_ejyb6^H6-~AoXOc?QGD@s$L}M_)Jcn$nH^0it7i}BL)8zzt28* zA{mcGj@UU;Z{hS4{2Y)(+9}Prq;DlRrZ&-^rz&=&;NHC$H|sJZT<`9W(H_j=MiYT2 zU*CFRSzt905s@R*lrg$&Y|-@ELp{xFL)LLC)it^mM$YZcIrYmaVj=%XFS>Gq%IS?c zx?Nwy^GxKBt|20`aK2!AhB7tNQ0^U7>`{f`$L^JY4x>Q^yy&b~6EM;4_^W%8V!Jms z)%{A^35DMk)OdWa^`E{xyg5&WHD`s({~$-gfW~;Q4N^o7VQ$wvWT`9iMEx0@GLkdx z3n#cL;api|H{z@WcYpanzk{LA$fZgr9O(p(uF-p_U3HXOw9rtNWzR+u75pLQwArnTKdj4joAB;eT!UMv5L)Z3q3FXzcbLSWR+>c>l7>SnQ+ij1hfNgDa2zK#k5JDM@M^=QyLMqGK38wWH9mFR$~4m%)*_H^ROJfWu*k$d{%_4 zIq(;>UfJjWskCEv1EX~4DCa9o&6NcPYulFf{^&m2&Hs%;_wz!+1!O6!IID=?HmzT= zk0CZQELDlFpgW2(7A2Jap|bt`8OtlTXq0|!LfkdJW$}IB1G}CrvzTtumJ=Kccjs~Z z6*D^TDBwqIan>e69v&lvmeMDhO>%KV-x(7%k|j)^c`r>}(3-eCx{rh$BGtLXuN)+j z@D_%B+wLp_>h3jayl`@{%D2XYf*Eo_m%mCKrs{I>SRbsyo-&>RWj%qMEI=B4&bMZK z=f#w-cKO&hecP%*u0^Y#+AcK*Htt%Q?OW21crNU0nrv@J#{engc%>z+s4*L9`~wc; zuC&z3XqzewW}@Gx7vbK<4NVN=$pLc-pBiN`AEgUJKb+(pk8rQ?DH%bL(l~YVq=4@F zZSy|rHid6`B;}pZATTalH|OuQS#dRz*G@PLZJ7pjY%L?fD_7;3(D`YqxfPo8 zPtGlb*pCh`Z1pccEGvX8xRET{xE#NcGKBa=EaN3qig>bc@hNpG1*{nMv?ysGF`CjG zHLwKYew`F8*IH-H<66KTTBR`pMbt&1`q&1y>e|ya-{RLnV&-8H3;S13N>ffy&iGIH z8%mHPDQKzHFYLOX!s?5tBA`&jknAt;i&c*A+Vn$0tE6AJjk$7>YPQBVL6H@h`=G@% zFg5r}PCaY^i7Lx9MhL@hG|$E#Hy z4!vBHKe2+HCq(e6OAYzG^IWg&_~~iNiT;o(yI{8lQ6sVu=iO+PSgs$woppgB6?L+X zYSbv7_Nwiq*Pf4D|({Oebb~2k zyf|WZaqf1sypYL^41ZNaM&4cY!7~4^ikuou6-Zc0t_T5`b7_elxM+ zX7nn-v##THqn{bOU{Sk@nf%+pWNA4~2k}b`2WhBM*)&d3IJfQ|H<<}T8G*Zvc8g!_ zdvm~dMX^H{mqJI-rG0lBW|Sg6w;!$LnW8t)6vwf~DIog2iQ8Mg7EMBwJm~mby>;+} zPA7hBso#Ic@&7|GG?3qabwA+b+5K7kDZRcL1ZwChIRdTdd1zg*VO6jl{L=#7PYMzL zOv|uz)yMj0D#R!%?lM^e(TmzfX>8|!edNk_6T9bYsW=78T>fm+2ri;2Nnxo)Ch(rQ zj4jKfJtxui(GOqn{{aUC4zNiTaYV-UQuK*O8%(PG3LjPkl z`{Xht^-|apEqsX+om$i!TOm>{HieS^wVp6BwR|BDy5x6?re%7`jBNIJr2ZDU-$F#$ z+RrRY-`x2uv#0}b8|*nZF2~jyKY>;($^`@(&X&%uo==U$9*d*I=6~AgAC{F-vC;3D z>AEtVW>@`bp(qimRdGfAN$GuF8iAK4K-e83)GPlzyEZF1R-N@I>spm=>fi?p-C3=Z z91kW%-|d;rb^vmJElsk6=>S9^jD9=sN3s7 zdmja}{qI_yn!l-`;1TyRx_{k593q9!m`9P?%ma1T6#_5tU=gz?yVSO`)b4>c&yHm{ z#tCCG6q#)5@YUC&JvAy{mPoMoy+jG%O=q85%!P0y063o zN%%nhkdBm*dE|cO?Tg)9?2=7}3q$g-ZTKYzpDfN=_iN2fC8_0!GBIPw?0@K-Bmqg{ zckW73aI~VB{`k4xV?rIImM7Q>Z1J(WrReb4=#-eUn8$Tz%%{vbW*1G>P0y+rj?}Mi z#TY-2!L(q)D8{2RgXg~dF_IxIqAGIE*}&leXLPFTy}87;e(gp=p57G@g8ePJQ${in zrcat+pY%@^xvIv~O&K&PjydMAvGFZ6GOZ3OdJVei3*8*YMeuMq-L$u5xPY1Te=*#TbyNl_k9`MF#Tf2>wdVp+cW*(N{=|5jyCv<6!#4~}nhe7Bba zuMOSI?M2|oKcNfW7oWG?9}ti!D-M5$srLQku-v!T2ixs?M%aoW*nl(&!JqQbn7CoJ zenN#t>Be=041JuJpo;F#b_zNu+-gSLDuZs3Q_1qjQuuT(oXfD@_Gw!?Sj5Ik^g8xzz%#F;C_m|h=P-XJLOm%wiJO;tWjd3MX@pp{+R1J-L|P;w1#dD z$ev+XdRt%jvCeC4dcE->Iv*#0L0*di*LbqId|ZcPA`kXUJ(CIvBBcuj-qOcVJ>mpJ z1Tomum%)qemQI3T48D8rZZIX=bACAy+jS0mqbf2r=C!e?y}ZKuN2z$H$tDNhV)?H= z{R5;(arYqX@0g_3jP2^|b2Zu5tMslqX z3%1-WIC5IJ3)d54vW`s_*g`H9v~rZAS}r+MC*C4OP1J1xL{gO5aEaFYWn{2AyIg|} z*$lb~*^o0KP9|Bz3Rv&Ni5*QaQh<8!oF|)`Wuu2nxGd+OK0WE}NF#mhM0Xe6JZ$Lf z6=+E7+~kWoQP7ju`aRRPEX0gLqTBVHmjkx1o!OsPSe~VZUoA46XIB<)#TMHvmsjf* zU>X3=cG&7(@2KtWcbRL`%C!k6RPX<_ts=p3zuh|dUS{Xa(TKX2OnWIxMkXjbA9Y?$ z^pr$#g&(okC$>83VwJz_ONIWM0NuQ@C4C|qU}q|U^W3h{G9^PYKzC9>_Dw-v0)9IW z#m??(BuOxl;S{Rz67B+Y;iRYFGwqTGli7>N)i=D>y@@9~i%-mp@^ZhFXs&9I1GehqICJ;yEKa7p$9(9Ukj=hL z!`9SYTW;qdXb0fTvjub|biDCfb@cJr2IJ?gm3H;(J#}G$M;Lw~VcAp5(XF+1l=W{f zfple!`d4)~?U>CC)fHeQr|)#1>z`(`_6D9=lg}5Z0`|+K>K{8qv4^$7d$rNMoEw^$ z@YM4*LDjMdw(Qu{SX|k1rH%gG+0^Es6{avO@a5oiY+2@%m*A$_TdrB-g&Diw>~-)@ME3Up@x^UT!XN7p8PC>@;9EONI1TgNplFOWPY zLcs!2;f8kxa2V4Mt6{TOO22zAp3Gbh8rSytlf8Vwe>uw})?=x-1ZmnjB$I%l-U+{x zxtpddq=^9wngJ{}weRt0SVO&c5)*IEdY`3sa}wbx$hnsy$o?5;Z#o z9kcvIeYi8{W2dL0CV8|4Z%d4cv9`}mui3v^jm-wBj+*Nn8UCSY!oIFz_Lt<}-m?Uh z^b%re$x?^nt0FbMOlV$KpMD%ck^PVK9%6d`NGOY|%CUatUGY)rkh+D7^Mm^f&)L`^ zjr#dULmz-dWY$E^&cmy!tJWJizc>8s<&0(7+_qY5F2ruW7O)t?H;Z#eSjVufhLv+f z_P?BkAqyvg&@vIRHSmx`SieBRZ8_iSMG@133Z$bq4!=2;q!knRzUlBL<#2_G8ZGov z8^S(_98;KI;G0FX#B4LH7;o|-0xl{G!@y@UB=9o<^IksSH-E=@>S^H z&a_WepR#Is~Urbfi9z38-eBVavo*>5(SGlDtggMqF3>32+$x|$yjAcSSym(f$#Fuk~VE_wewNj1|iSDPWX3;PtP66zU0*AE4#BgX*cn*+BXiEtIr z!;xNBJj5s=nzJCAL42?w)5?`gT}_;xld_*i=|{qZhIKCJw55D|mKHphq~HwKVkX@k z3s8nR3X(2b~`p)uyj+$PvkMyS*r_D>5OAy_VJ9B5GU}>wp)HaqvrD1zt zQ0IWJR%DF}i&3I04bSK`rZLvo4kI~hmD_RnwN-`=E_H={Vkp#WK1|3uY_^>JyWUVo zw69@S!sxV4JZbT&E1kBC-0|9g>K{uecHhl2oD+x3+^6lvs1m1=oZ-8!P(fMfPESAk zo>qoMRB2#RAM6xT*JR6YvgB6$P*7*0FG{|qyS`ei7Wwx6pNbtil2*i@VCnrj%oDkv zkxZq09}c%*hSCt#>#ege{HftB+;-0G=8r(*&ob5i!$-9UEzvmLT-ucimboCb3Dqwb zd>+h+6Z&=k-9U86QrX?b=BFc>|t<(lA45t zu*0;HiFrWr+WpC|#>q`p&|b-v!RiaVdr8fiL6mE2SBd@^J=W-%sat7@c!)|{=0w*( zO{MN=zYSW7#pFlOSxb(I4tU>~%XWKj9>fP?Ytk;-vebRB3i8{B(tY)z9Y49&Exg|E z-e%M;mB~7A?-X5R%ed;9?iO%3J(fdGNw*qGF=I9p)w_y^BiaIpDNh2$V$iQh+WszM z@!mu?BjN^8x8_09(o*Y0A^__SO?FgD*9LrM4vDKxT+nkS=S$xUzr>L2Wy_5y+ad@= zr)CpTK+KsOJ)$~BA%qj6&5B+>fdZ|pob(hPzkNR&>EyeYq!dX zuCNPdW>41>Ev1tI=n zX{MIPJfX4$V<@=e5?{=d(^IH`f8rMy5mk;enxkScrCrHs73nbsz{N^~g2MH0!UKEY zE1qUQWfI;raP;ribH1X%`DG%ge4&Rtc4mLDp6vj)8DkY~0`aeo!cVip!y&U}8&@yzNMa=`i24hO>< z&6M>BcRV>g!FbLq13-7{3&w|FO<+E~5(ghdCvf!ZN&`o~kgz@+l#1xD*=6OE)d1cU z-IWk20*hq^E$red^hcDS_joZTR>3kxO;p4V@LNs*4WiQf--Z0?ie~Lq@PR8NBnreH z8xDUXvl8R&oZVLJ+(LiTTTqhNG&&mC`)T^>!5x6;dj*aEmqz~|i18B$BB}x#CuaW# z>;3!Z|M|MWl_!X9pG`xlULZ7JNm1a@vA`+McxJ;_SWgs%C(`vyk{yi@KCQrtaZQq>OL}IzyN%fl@*s@z%M3H}BjzVvC0m|TKz!p% zrIrZ0be{~&Qt`AIC}ShXqHeIt(*)|KXKA;>rG1rooRH0h2t$SsTF@LAH>Hi?TU>qkk#Y ziLqcV#~;zAk^l~7cTBiXL&lbyq%a27c-K>2QabOpAe=d=P=1GN8$(b@~i#2+7RX3#rkhuX_W!LzobJf(>hpDvT4s;BAy=?nVl>`Ey&YRxB^vH{E z*P09$n^K@^qmn<1KEn`z^5n~5-$@>He+??UHv!(8IP@qU(zDBb>^;*2_EpLAK;~ut9+hqD8@HZ?Bk$qI`&jZURKzc4RcjAJzMXHW zTUui?)+$r&@V;(F@)R&Ii#7$?6b^Wdt2$V#1hP&-V%|_nT&Keu;mq8G;l2Yl!*J_> zVUUw|%vPHf$Qd5U6xX)C<%0AunR#vG0tvSDE!y4@1hY?q50$-tZW9kFvBN)pBQ0Y7 zr!zK^ta&x%;v>oD5f;41@VQ4f{LK{xNSoUj^pkrE2Nl zR}Q?jbkNqeQ6=-RPhQ>nWvClJ`}1y{cz(u$@<8Q`=-dspgk9~gAG^$`^svd1Ae6Z; zoyDwYp)@A&Cfu|-H(ToI6*HnLf6FlYeL;_xmnIs^mDrY_eHV=l3d2_iOp z=^>pGDv*P2w`yy*0y1sO?cy}(w6%GE^t{49rI=|dErzYTbdMrwq}Yn0q}qrm3!PcJ z-FB7c74OLjFwP`K>{Lb6JtBn6zLrnMy{eV$yH{@4_-PF^0$2zm4DXeXMCof-V30CQ zc?}m@rg*+)4Xy{NL#7SWGPQcI$WggK-9JtWxe}T}nld3xeLc{=8P=M@b=07@?CFDn zmQNop8Xj)23}6Cfl28YVfax?aoVsr4RHN?q@meVP(e&mdH?yiUG##PFldJcG>w%jV zGw|#{%*l+k6Y-JDmv=g93|H0+SH6Px*EPU3CRFSk->wWADkMH4$P6e}p31g4G9fKQDwmmDO&bmdJSK`q`BV zhfjW^`x@q&)U;*xO3j?|kiq(HJl0eU2eSN-e$?{XT15-)X{dK*$mcUQ^hzs|H3P*| z{?n9OXY;u-AA!0WR-989l%XV&vRDz(R%aYfxm8qZ;YqB}PyHC05ZV&A_2jpQanb*x zOaBYdkQ3T#p4S-;^&;A+4~Cq}FAb{IyH$b@_WzU&T(~@RQB)OV8bA&;?vs6$J_|UL z(C#*Qi?QbX<`i)K2|JQ(*`>|iBlv5|6Lgo@$ zNwn#BRNRqC>s4{Lz}Kx1U&^?{i&MGUNBR^KX>6g>SFD1?uI zX6IwSP80Td1e_R4KYD(-eJ%a?z4k5Jx=$UgEi{PH8YzkN9nGoUqy?2;vVxsXTCudx;Kt7^gShi50yN;YR=7=gpqQkJd@O*)x(CHEk5qWX5 ztxixnZMZAvC)uTst*FUwwQ=HlJh<)mm5mMi!NATgeksimLeDoKhfU7cZVGX_9Kfrz zq=PyqDBNK6>Pq>CN6+AX_GmtSQhx6hvA){9Jw;KI(cBz*DjjRZHG9+Hm!3W0GGwCV zK{?hSr{thOYik|L-z@T`G++R?0KBX?1hBQ=3HI}t&KBwQ9#ly`7DI1@?vz36MugM` z9$;Bc!mNxO#uK+6)k}g5Q4e2vEwww*&?_>66G2y2JQEy)-gzm6Wtu_s7r`8$ab_RpRg=O&ASo>DCDA@$crgr<%`Fv?0sshVB2gUbzGh1r zT4>@+g}0S9P#dB`tu%^Mqz`zr>xUg}5Bck35r<h!_4GFl$lIS=f(XY* zHgm48Dbd<^iv!)q)pfjoBiZ0^ySmlJ@pa0TmxT;7#rE&lZ_&!iDR>C>iqELq$3@XZ z)T{hen%DdSwMkAIKls9-v8LD2*LCVtqO?D|M?Ssp@xzAFiEmgiIQMihP8nMxC!9j) z-?Ur2fFt@swjTfikX$rSyzTW0tsvto4ljLw>YP71g~O8vGs+9O$wXHPLtp>U+L zRWQKU{33$BEp6|9^#o(8#iUgfeY+l+CVz}|Ds=1R*m(N=pQc|4^Fm>#MCcXe+T_VP z1u}2@jROxhzc6X+8p5>i*?%8GE8QfgI+WBGio;{=wR&(7lV{V*M{-E7h0BVMOzMCKSiGIvwJLVp{s>P^veOE?A|$hE z)IWX$R#NlYLz0O^Y@k|H@x>G+^2P*9j*KG@ggsl9=~biP4UY$+3Up|>c_{AYWb@L9 z4HmtvVY)tn|IZvr5Tf>)6d*EoS!$89R~L>-n*TcWwT1pl;jf;{>E`(*HIBi? zAo}sG`JiJ8KUaFWr}}?X4)$Y}ZLQvJoftGS=SP2tZeVYjp{wRk9i6^zUEQw`&XG3p ziy76CHg)zqg&;9R5DIJbkgPJTB)CTpW+gN}fi>J*0kA;Zy%jn%6sEtc7Tloj`8!Pm(jl+LXp-l&MV*as zT6Pn^cW_?km34b1tBf)^Rgtnv^wJ~6r;9`EdIi-Y%b*MC`urUbLhF-}f7-@N zYx^nKetMIcsWU~frS6#d5R&`#W$=kWk$T5XAomz}x4dB5N;rgo6KCQ1FQ|7inhMdU zxZYlE4p>+B_Da9^A_-MA!yv_au4@K1dSBTw9gQQeyOo99!QB;3oqy$da}WTL9Pap_ z_ldyft>l%Yfupe^`%CP1HwI54RF{x~XZ2P(wE!$DJ{9mWo9v+|cr9~!kI3jVz_{W9_xXpG>67}8(ACBC z)yU-J6;NR2#JVrezPddUP+fl*U_PKX8^APX2)?M_$8LD`Mz>rBdK{HgBZjNpjXnK1 zOE5i_cQwW2&vk$Y!g(bdh}A9DMwwV&%2?n`8*NU?Gxkj9QcHp3rqAC|feg$sEFldamfo`I9Slwl5FP?`g1@bq)O`aLE(bYEui)A9&xwUe^L%NTf z5;NHS_cNWu8n}bL-?Ca{D~#9OCj2d%Gi{Fr;#0_KKSMz~|Fphvw_$GiQeBq@RgAw* zY__rgWPQ5$xwbZ6q2=IWFsvMv(=7Un5~XAiKIotm!U)BD#$6AIp1zT+7@06x?7bGG z<^fxr9e!rw8t_N5ni+5-)Mw4SlY#e+A-%yOH7}55I zKh<0(Xs4AYo59Yu*ld0ct~VQ5JB`@Wp^0+r*R{|#c39(9(*=HBg61Btt#fRbu)j7A zw4@h-KGaX=mV~0idTR>+>*E&17EFj6br>1u`i2~Q_!S21?9la33!sr^yB;8K>|NQl zy=wiDpV(D#g!dq?ASgM)&TiV{I^!f0aQY2*Gb(Ejf zzFIQygjiJ_JRuE}I!sYD(~<;pWe{QOJ$EdV1mZ?vcB-fAMHRYZs>awM5pDE*y}N*A z=e4$XsAJwD$j<+-aKFMGRWHByYPAElGXHwtEtVyX7`hQEzAAbA*?TTqKz%XeJy>yK z0Z5(2bV9MN4X@MBlXeeKk!`giE1fZaGu&@D)xKLXFhS2Ttht(d%NDX+p03YE%e55`zFg}yZAi5B9enWxuWV-raBkL$YI>Yt>5U~ z(QmOCrB)mX?hoFUJ3ut}@z*B^>9Yr>tOHsF*jZR!TP#QWA>Z}%j*t7&NB1R*N?DFL zdmz^ko7rm{?x*H4xzzPK!NLkqiEGUln+J;*o15{2K`ciptq~nbp$zezfA*{scDmGx zNWUAsF+~=DoTpvFmPhSNn}^d6R-9y7lg~9w70Z%?CzDaRTIF2pu=2gV#+nGpCCu<+ zTFm`1CO7XgDT2fz=q2mqAyxK`z5VXCWRuQbcOX_21@qsBHgDaI@Jz3v*x1_-gxi3zyYES)IB~3-{>It8 z){N!S4|OG~_6s2ph7_U{sQ;z%JbepD#TuC!BlM%2&DCofJR12;J;3Nx8|-cY6q+m< z99H#-TcI5Qq(dD|Qv!nEI~w$K+01_4R^;BO{Y%L{3B)z@5sV z6yM^})$n%gU*EOaY0hE%-|S@*K)e}BMQf6SdYYr^+~O(srH9vdgD2nRpKjiE95(f8Po<7A3xW1fjqx` zm#}fygf6;21ahH=3?uvlM&5$~a2y{pN)ulQERG6yPal#*KGK}cEC~@V3JCtF(owST zg1xlP+>u7?TGtL>HmMUHp#|YqyVYx4K?4OXy{N0djaeDl6SsN*W5UsQNC@ZE-656W z6EjtEm_&n+Ip1qA&*@SPe19@?{QVX(9ieayvQ2~V*!QVC_2U=p8!258EH8CRhgyG4 zsrqdQ55)$Z1?`b74Aa=uiPhq{Q6)?P@IH7AE03`S;K z*FMKWcLO_VLjfnWor{*z6z5F`9X`p?J=1WerAZI@&QBHEn-Q;XXw(>vthLmGiIk@8|-ZRt0B& z5t7GuQJxvg#~;}PY=1^B{vQke{a)7R;?SY%uY9rMdAJk*F4d6FN2MraBM#OA;GZ(% z%ent5xOvMt!mMEs>d5z@6G*`h+a;Brd*}nlOwpkS9zPml^(j{_{C68YS*@sv?r!@N3uH<8v&|DrUn4)dmtMg9dv0KHe3NR9XGx1>Tdag{gIJ(b9!DhJHz&iDz> zpe7$n;<3ZLo_p}63){bLtZKWLa{vRz!MELL24tsuOR-iO$nI!N8xelW17JKdxc?o0 zRx)}_Yr4pLs&I{Vk>&#&F3wSa4(9=iwx@qKBt6Fu@->6G2kv${JdcxI6^Zwu&J_Z5 rR$ju;Zy7!`016w$Bf%As`{2iwlDUYW+uc6^KgKuA|0ue4|LOk#QCtUU literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_8.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_8.png new file mode 100644 index 0000000000000000000000000000000000000000..cb78c8d450e4cd9bac24e93ffa62ed20b2ebcba9 GIT binary patch literal 94378 zcmb5WcRZWlAOEc=RYgl{j~bk<(W9}p4U zpua^1ytCwG3kF`8*eNUPs4FW!)NygLwsWu|B6=L-{#{g!*!+I`)fm$=3Yq7UU!-pa z1Scs)CWbIlyYRnNc1iaj`@pfR@;rQC`u^Yuqa+$n`%1(0?rVvE39hrcgab&CtdQ5v z@zwF(^x5>$XZC1}LFdg(W0a;OXxa3g$!-nH_qn#Gxdp3le(@kFo-|c>sP3`H4%ASH z3V6RujhvS7i1zULQ|`LD;pD5ozexEeZ9hn?Kzx`u=b^Jpw?&|RYY?xHO0CFv@yYX# zeM{jUS}!B-&d_=v{7m5UnsZvna6+oK~_U~HQV)Pf#@(Wppg4I_K> z1lh;QJeY&g+69Aq6e?0?y=++9s01FBQZHHvU$LyWEad5gCVwY)#W30A>?Xi0>g+I{ za7P`#GAefdY0CK|7QZvx%&)W8Cd5vs+`zN@?F4bS=~oZMS5v-Ud6shW+^`BhC_M5l zo2CzPw(%2grUq-5KCVB5roqM(&Ub5OE(&q(WA7O@w8*qBHSKZZXHErh|C4T0Sx5mT zS^)d2!LDUMni=?qva|$0t zN4|$K+V8hT$!$s7;s(GF57x-?%~aPub>SKFP`3Zjk!-3kILzm1l*T#-MjG8GyIyd= z-=Sj4gR<7g$AJp>VcoPW62WjL;t~^n2Ma30#9@wQS>yblHc}|6b|5o`X`{%Znga6Y z1d$(9NBE$cS_#rCwG+R3v9>0+XGlf-W|u1s9Ft$mjD{N^WH7bkwA$&62kK{*_#`{a zgU)gXW&}2NIJThP+4AByZvAn${my92J8OB-1Pxc#6HZ*gA5C|BiD2~cSJbot>#^a) zuhd+u^~E@SQ5NsV6`n=+R)ZH+9Ef||&C*Yf?$1x3PLx0O>}XfY zQAEtDJpy(Uq0;(D5<#mI-~Kt*m}Mm9i{@og1_rctXCmYrg%04h)9CtAyD6|5ad3*w zba+gWc6yf?X|Ny2EdotF0Y1QBKV?ksGh~3S^c$c9|t&J;4dE$7=P1C@cXPHu62FpjQqyby3&jlOuzO=ZvzGOpHlt(N+PqV{SOdc3cP|&Dl6Q33=#hsOAM86(JvX=C7l80J$6qJ6**{g(6Rc4F% zI5C2xVO)TCWK%%V+~Z;Dr#@Uz`Qf!3lZ?K{`V&J2*2S6=1rB=jUN2>{{kn5ecW`Q` zv{SM90v6{4jW$JlY0;56Y^_As5VfT<)eDnW!-`*e?}wPRW#)S{{xNkYKj!b>q+Pds z6U#~VEw$mSrom{(NHBa%yC-J3+(XfBff2`=()D=I5F4UUtc6{!K%wIuqQK|o3eW4u zP9r^{%C$HIvY(%wzC35d?U0n%24z4RBFiI#6ZDTil!V}QqTq*%g79GlN!d~Kc;2|a zy+CE|v_Doh5`OB?0kS$8L@h*$MsXaQ*=yqGg66PjykGc1D<>bJw_Ev_x(UBX;g><{ z%^=}WI93b;D)zC4r+`+(e^*zs16EDwM*r@&|4nhWsfQ~`MVq%X0{7N3?8w#D6cwSK zmYv>gqS#=r3`2;72M#rcj2+d-ziix{X5U7OqYgT=Sy^6gs~G~WtJbOQ=hJ;{`Rzk3 zn2{FyrX^=fma-p936)`)ego6z#%T47k55A0`))Tyms6#|rt>-MS$w}Kyo3+kk~Xa9U!RM%$)`>fxK!n(LD16octVP+$$mI^ zgRA~?7o83JS}L}}qu6{d{nMtRwjDc1&7~c(jK`iCHy~#TxLmRn+igbN2m6RiHWpRD z7UMx9HY~0YGsqPq*LE!EFwJWrc@h1W-hZC&>_*CYM|P^ENdrQpd*-VwgFf@ZYL1Ed z>wTCNzmKyn;0ays>@D5rE-yUINf1!}P<8ODZSepY6+HEgQ8ufU$f(n3v{<&_nfkky zczj1V0k0xMXiYA9<&5R&-JN%VAP^RDoPQ&^*ZBX9-usV@}2ZcB8 zt8(ltwqY3<;KNCxg)<=-Y2P-R4v?Emh6T%J82sxcu%0fH;^CyN{uk?}!!*!1hv88} zEJpengDLA_`2Iey1{4?CU2I#E7>t^;MCQ3()W4Ql`)Y9Q(o;{=+QMHk+{X!4Xj9lE zkAI!R5+DeR53iF^srCv98r;;=x*s1IcD*+3`J=%sD$`dh%k?(hB}X!%gzJIai?;=N z!uc*aFuaPUu#t?6sqZMIVD`P5lc_L_aqxgx%+Tdo;r#$NN=6or@VG`o5P)BX5y<|r8{ zk?;c{y(p`p*NEYauPEau!b@#`lp_#hLABaZrQkdnYJ~IoH-`Tc!xj_N+Z5Y8+4#qP ziB_^^L+peF+Kq(Ejc+R?K7vLp4jI2~^%laMe=jgx7|Ao<%rEC`FDiYO-AmF-BCv3S z&ry5e7*Xuxe}-F)5g5fw5xNX22dRVp#+PLMV&IU(eVESQv)QaL>`KWks3xb^A9BF0 zNToaMyk6>c@^k&PKd9QjDPLJ>;#mHJ{B0ijJDmpEv%iLQ7_efd;=J#B_V$Vq@BnZ< z?Ps6;cbA&S)2fs1gRRE@ZoeErKT?#sowBqPAOvm^6RiGVZ`WN=BU99lWDqR(RF- zu3QK$fR;K^r(h6G=>F6BAFzRxE*^__&C4dp@95=Zrw%e$s8X3+~Hh6 zM7Jof&5&k|oQV)`V-hjqV%OTl5<~}^iWL;?x=*)fk#kSnJ{A3+z*6>fM?Y%a$C%z9 zk#%3K!=jqzoQ|xDGd~YG;F+xaY;)S1Lu*7 z!V)f;hTb7Gr{OU6h1}0NbQ9=RpBaw`RI`BGu>c;m2tk;AkX~28EyA^M7H2O{e{Ok9 z{lD`LW&gV8xvEhz-o@Hp+kKwU@4S@NQ|Ejcxp_adK-7NAfc4bcjwL`E{lO`p3Ki)q zjcufhi3P7K!AYv|f5yNi7N$VLBsGeL7{^K4%Y&vbtSk?09e<^e=bO(&P4IlR1+`cVkoHV+-hYh zISTk&MecihyN?6OW+c1~bgn_HiqkS)AJ)wXY}r~7vFz51LOk|+n59uK{Tu9~)R6Ge z0v!S`v-F{Q-_rMcQSOp)vuJk zG8zdLLcvY`kSO5#J0iWx*gnWVjYu5FK)grvvR?^%Jf!4kigxK$1;cyV)eH275hk+> zd|M2sPonrQZgI?@ggb^sma=Bme0>9bK3X)k`{_&rak&OqrCumgysi|jumLg=QTxl0=MC)*M?H6jOs+xTNRVcnoA7ke+|(pJO|)@MM0D^`+w2^i~K z6a&IY^C#NW118voN)&u)8NVT9GtYOPV6;4b_2+=7d1 z(YH_fp^Co|--Di=XqPN(le-EtZ`f-fFZKnWXT93{n~ zE)H={=1|KBRHPp>-2xD?8XSRwa4=4&(8exB$+z}Xg^slO#|(>^U=g1=-CbfQAhFj^ z<)*(MOu587ty4J=EPTJx$RrB&7nsPV=M;ThYwh>kAmF{<3lEY-{C2~qp#G=B61Ht{ zyRH}11aNS^`RrbYJq}*6`QG?tNTQ{MhzE`PF_6TIM{Oh9sXjN@+M8mT(V}K3Dz9F9 zlhU(1|I<86;=@^Sw`ZtcNF=C>M70FEo27my4oxs0=wcv`xA6z|{R#;%Vqj^!X9J>@ z^tw4qzNLbX^Pycn70?SP_(g~4P$Fa8u@{=)eGKshO804B0y0Arc0~QDQn@`;Y>8p3 z3OBXoVFzjEn^liH%Zk33Elt`GKf$y0k0kn!8rryLoQ~vGlhA;R&jzFYMts+eX4bac zi@b##=9x&gy=TsbFUw)!hi3tPRC}5 zIdBZ{_~?4u8rfUy0|7l6CWP3|7x1%_e*g4Kx!p9NA!AA7!4;GPojDz?@W{*X^TH+( zycfVH%DiG*pBm0`5y@Wr%F{0{0>C?_r$dRlM%yllHG&barz`V8jw|{Nr~kI7oII6p z|J-RO;W%z|)498t)eUx^CL~nk_*HILiPYT9QBhEF;4n?S*uOk#P`I!eXPKNOOPLB9 zR5iBmSLaI`@m&(nSl#l!I?F>p6gj=midg&;4Kn!U+woqY6Fum-EQb3fGV8ki{0h(5 z)wtu_^W6WRC1>831@;`8mwV#wad~})6H6CSVBKC&*UQZ>Meq)n)J`BxW`D`jIKd#9 zHi-TjM)+lTRgfoyS6@QeC7{g}JZMgYqB!9bwiBb}k%uMsy4_!7L@+s>Qpunkr$dGq zXEqeNbeog{bm)HqlO#8_mwr^cPvd;ue~Nm~gp|&=^n8*uAAS1kcZtIQ0xcvPpcj(Z zaK5;@#Hohpfw3b4>5lLC(Zc7Dd?&i7dp7c-q8!enY|VzkkfF!W)Uq$xntmL@?X6zw zjlUEgFX3VD=+?4Lq z3+z}uT5c%`Hk8%038YfNd=E!j6h_B5@Vki%M)gRu)NH&86KP$v_vC_X;J{J+QQcH& z`a!`KV6Znwsmox#Ui;KoR_E4xQ^OgT`};Fu6do7yj&7o&*~%9v=wQ3$*(|w&bei!N z8t1qgM{B5GPl&HMs$G^}&4@5$KpKJ!wxQN|c%b^x4(}4KR z|Ior7HC$cbp=++Rk3tY%gQI3sKPR=2YrAEf^w@Yf2tqDap?N_RRxqQoBVbp!9ev>g zXW{IN>gX-8c_XiA4o;6>@fvJY%|9p_pV|qL(zIs-TbYZ)%@&S+jMefUO!WI5mTo-~ zL8yNkP$IuE)$?MX(8P|*67=CS)ub&Bxg4*Te&LXArF(4LZfT;~vn=iYH4AK#!VxP; zpSQt@hy#+cQ|?7}qQ%|i?b?;qE!67bD$*Ka)@&$RqqCkC)1E6{Jp1VH0UFBSM+o8z>#9Bg7YN98P_j*;3{>)%jG_{oc_W75SoU zJ^rY1!X_h)o{$|S`=58a-1K1Gs+QABz929m$Mn6y+vy?eCPXhu4?n3iUxtqhbK<$& ziSOYAU)5Zjh@bP(S5@#&?^O-cAKD>^HMc&QdbEI1XW0G#-=k#j6)Q``w-;LLF6JNS6I{EFT+|ML@32-5PYr@ZywXF{I(!FgN zvkTm#qNY>JaY%kYi)J$lF#Q&=ORR>v%_$aBNw;hFfR3T51mDiZ^iloT)XEE~mxNm* z6UMU)+M~PcH$^;~EOFPn%I(y*G-XQA+>Y*$I}xoLeOgQZsr=7-A#-3@pB{aqDRPPb z$S$bSC74=$6>#CfV4k*`ChYzV0^vfDJg0n}@plI8&z#Xqfsh zu*|3_?Zp?m<+l1`#M5K6mrM^!5+AYgrF&X@3KlTtg#WjN==bLJH@+k!LNA(pF!uMn zH9^eB>4yY~U^;P4M`j|Z|JQtkc%Pc@v&KF~?>S1-q~n55d1)T+ zK80x*bCriXHzA?E7|&d%qEB_}xmvd_zWFn`9)ArgWSKd}{NA|S&r`_l2kO0{S2W4! z41z7kNx~2E>&tpIAUHFga5k|8<2W<;^zV;}yAmJWLq=Xn+A$%a2J(G{ zABK-TIlh1XLX#p&UtdwT37`7whA-$-?E+hkct2SDrbFrn(`DH)b!Y}-KWQClZECRs&~1mN+*o?-IkK>y zh=LA;)XN`DmU4S8roDEn+e_Qt7{7a$)q?T{a}klr#hN~}NrbO6Rod16RR+yiv=5$J z?>(`H$<&-3BOwY*=(t}?YuutbeeFN)A9KyLFOz<)b$s?cI4zt0DO zyalK?c6jL5=P}(|zjr&w>JQ7K@)*GCv+jz-?fSLz52k>!a1TG;LZbkN6aBAK{`n%C zkWDo(^I_c?ncqGqb&@lDVJz~$(U@vfQZKr_J$u@pFPCv$wz8!#k1}He{+2PFdCD&d zn4Y}ER_??F$wWCTWw-=td#}0@d#6+DTmoqf=OQ9_Jx)(s-3Pztb(fqiwXgIfGxgA% z#8-V5$`N~O2Ql9{96;kUOTzEDY}avvob`Kp{m$(t9;G)dS_1q`sOCG#Q=X1$gUI%0 z^7GPv;P+mCy)+Xnmyxcjy08Pv)A8Hv(h5jv4qj$ncU}a$sF)q$lKX}{WdNlZk`@Ub zu_tVn)x$N74deO5nG<=xU4q6A%BIlPq6i;L)ZXK&gY-Nu%Tpp~-?`KCtWYaRa zQ)VL+>6l0hYkxi4=@j4`SsuxFxbv^b&s|nMq@6c#*twl-lW!2T@6xQH`0`_?B!yrK zCEj_cSC4{N3nXX?Bsh9cHJ(QJfgYD}u#L<~q+}j&bDXGoQHA6Yff0(l)igh9g`3y4 zM{Kn7E=cFgWTK2VxaAK~(OP*+4!k=;N+1?SpI`-qe}-8~j|8ZkCpJKyhuDh1a~cpyT6DZax|A|=ZhiEhe%+Is z7yIki%VnRGToC>7x6;gl6t)o)I6Ijq6d?n8>kvDjRMQBf^GqKf7#r~($!3pFr{8V; zartv*ho40Z#1Ij=zzbEpMX=*wq1>nf{QeIN35z5aTC*?ZQCM!-AhWbk`D2N>;m+Kk zD47(G4VEA(n~1%f{DmT5#uK_t3Vll9un76T4GilQd*VvFN}NckXrv*lp=NSsQiP#p zOrSHJb54!#G&~X_?VdSK1&*Fg2guY=<4Of(y#yei+U9lBz=V&pqqD=;?JONB@TEK%cgUmEK@>rB`}NPUF-0fQ>XV!auN3 znOcVDW-{q}ztwm;KVlPN=;IW#Ehb`6;6H9YzU|c13<<=85VPnp*@5;%tCJ!Aqrt3D zcI@uD-U^p9p_eRJ&xhE-0VuzJ%9G+(+N@DKw_IfA7TuB~Wwsagx~NlPLghEiS6kDE zKRCGdSqCW@otJlS~P>Y^ieSHHtMz9z*PD*H2r zM?i^P1I3Azh*eDT=J3g%)5k!!?oezsVwPpoEC}t$)rGpmIL?7mwo{TQajQ{vMJIJ9b#GJzLPd1k36%5Ike;Fb~&pa zE6232KPTNF4Igjuwc{EmQ?A*)iRMI($==G8ky)y91SzeEp76wj-M=d}lWEt^nR5pS zq&yGrX_*ysyhANwikA{`Dv3YNwJdeB3_g%s&!_AZ4{kwCSM=lSy% zQok*j!d0muzuxa0&u;&wn?WUW?^p*ZD)z2xk?xOFK!3ArD$$hS1yVc=$ibima`(Ve z>J;Jgdl#So?lkJ@Hw^o&HC6bD*+yQV;ArHz1*FpIx5+f0^08rjbUFO(>38wb446Yo z?)%}64`_Iuch(5adQ#?NGcNo{czY!{eIJvh{O%nadnS##uyCzhPjkD7=0^)mAal^y z@h9}fjE`GOJbUNgXhd+kmD-9W7>u(SmCFkHDccpmBNpEcQ9CH})yL}p5yyb&;SlBg zeh^l*!L0wCc3I5WYw3S@eJ8cl55s=-cQs}PMEo8vA21MuOwMNGSp&6Zq!>bH7(bDA zk@nQ4KO0b+e?YTE0wM-Kc=#P(mhfqvB9;PQ5Dj*~OL92G^(Q_=SbY`gB-r!U(zg8J ztqpbhoF}A1H&zx#>N~Y=l4!Z-Cu2qG-+}*eG%KTiSm)@b7wTay#X{v*AKO|)>ikSg z9`hJJr|;e5S( z{dO`h4{vKA-NYwDkLRD59@e!9ImV=K(Aw^Opn~2U|GINQY7#v6hC}-bMR!HGpY(+$ z87VzsP1o`(B=2r&;og&bzCrRm+_gik6mm0s(vAdta0_3$Y^ns}XP^(QZwrI|i6l!s zNuvl;R#4uJTHUtstjIE!Dj&6d5@}Expaqw&geshye_$y3CoPN$vQJMhRcg z>kE$7^?FI(`45+084K;DFGAosi)g9~m;-mndq><#HUw8GRY}fx1#|e*`7(;epUaUA zc{fZwR0YWr)!vC$_mwgwE>mz_yg>%;r1-4QWI(E(8d&oCN8?a~CfOYjgy3?=hb)T{ z9z`M&toelRE|A%Q;TF<=F~Xpm$1y}1RHR}%G@G+_H@CB0Ewp{Hfm&-sel6RCA*^2u@Gf?VWJ2v9Y>*ipsS$^V+CWcUiRC*pu1pmey|6|yf-EK^r z76ZcX!7T)D;i~wCKrh|=Te*LEJ!c4=e#Vxu<3O2sm)$ZPB;c?&e*Sxle2B1TPAvH8 zah+3S6R1nQ+m7W@CvWp*gqHuTIm?mA0vU^)Mt+)E`*9?yPqjhC@b|wcJBRFP7oh!) zs&RskIvqP!!SyQYTNQZE?&D5?Hd;N`BB3V{b9}-}Vx?Q}OtkBmLl+;SxYrkLwdncv4QxYF0V*?BYFEn*zA(@4ng|OqQsvsbS=G|2Lq&l79hA3 z@y2~TWD+!;COlUV3g>Z@MTDVN0r#=6Bq%g$kvV(n0E>A>R+Rm*L_Jece&}80k!|>8 zrhJ6Tw5xRtm(HVsU#ZSP+Z3m2sl-Un_scn*VS?KM;)R~j4w_kZ_WV6!d$qt z=M!Ju(m^3bl4(hY8q97v45|)s_#`7gO~O_hnM}j5Lvt+c)r$yglIF^lun$G zz;%aEDwHN8|F3>PzDK%1%1$x!qZLIawza~5u*f8=aN~E&bO5Gl0ENy{%2hr0#-ft`bO+{SLXG|twb(j~HQ|=^%rA8tdkV1TRnjSl9 zj1xm^1R~$|-tNq$K?%Mef|_KfXfjrk;LZ=4I{j2_W|o=;`i43PfqNq`7NpeXVAKzI z>7BJ*(fV%pAEDF})~37C8JUo2|J@f+n}(6VPJ%-68oB>rUn2er9tAbQ@GaSw2254; z;OLFm^xvf(y~J8)s9c_k*G})=R~fs}rnbkyEDY5-z4YKx>=v$4aCHzxDxMXY9J);mR_(gYri43A>5 z;J@%PWPSP@9R+(_(kJT&z@x&zQUBs7U=whc`l_mRVjQrlN>OIFv z!}h3ZM|1Fn<@d7a^+~@jg6yckZuzUU@muLg%-Q5JWOam{oddMai~D(@Q{2`& zZnoDxn=nRPu7wsyhm{;>Y1UQ(a(qh`_y339jLR77bWT{7Xy=R;ebdsZ-+;&`bhJl% zcIR4HAcVWcPGtUeGB>|6X?dPj+PEn8!K+QZyan`9C{g&PKJtbFud}XudQC31t)lkZ zfGJ`9EOJ6X@$HlWO_N3Tj&?J@4-R`Sso_$KyFYxiUQkd%Cvv?KR9I_SQfPvUL28K(svDl$<24x zpZ}XgTYF#Y&WK@>^`ypm9{QsG6k$W}zi6nvuaGIT{N43t2JVKCzjFl)-HSph31}58 zU(uK0D7Cu6PQsv9l(=d3D|RtyKvst&RID!rAa)g3N&QV6jJ~mz>mB8GfJOefPrE?M zHVJ9CoCIZVj1LDj+r31wIcQuSE8+$^d78P;E9RR}XEUh#uU@kuFUX=2T?PO*W~^KEwmc$7*0q{u)K#K z`AEoiEXo!nd9;JlZY={IBA)OWru!co!sPNp7FOin7FeWVu$p||FqK9{k6vM?O;xiN zZ4B7Phv~#bIU8c7meu19Hz zVWG)~Hd>yIkj23&Ixecc*TF7@fQHx7tE?WupdG&$9@G^y`q5&bGcQNH>&GYvn->Ma zy_8m%|*v*cVpFkUG&a|0d2haw)|{rCzt`b_tTXhQK+%5lPaKL zuNfCiZRPSFDPVi)}#uX`eO5-=3w|<<={e*eMD1Weg;f9m0-(`pW zyY})L1#%0#O2k3!D;MKdu>n?n0Bs?Zsmpy=rfokkzzu|@8YG_bAON{GEFcy${&r7Q z_)kB`@|f=Z$R0An)*k>alF45ljXtq|HaK~nE*#jWtht=8cESW9xo51z6mVxRgQIJ~ zC*#b0BX#XOA{~MNxEtK9h=F$UGXO$zXaIOCCwg{Cp&2 z`d$R~TcUoy5)WwKV5GvB@)e^cKvu2h#6`mnJ{VE(k>S+Hu#$%380&Gp!>@Yu_l+~) z!hy%(G%i02EHweU6n_I3)3+DFV+((7jX(J%wmnDrdyoFA=MSD^xpFA=M}K?)g|9t% z$vKtNTI^5l#d~+}J*WCB+VS%eh?63Fz8P>p-*os{mkS!#ww+<-j(Ql|d#%;aDLcd9 zdk-u7LFfHW-D9H)?os9Njcp(hp!!^Jm`at3&E_Ir#6Eup3jJyZpc{u6olN8YY1k); zJyg>v>eUbmUrbnhEqI<(U=I+UcaH6>7TtgL)e3e+Ri>JjC?vM!ED}rb2#iJ=SXPS8 z{-Dq{B7{ke+K=^;>@D2@^+p(Cw%@VTp!w5_|CWQa#nW%yU3I#+@rcMuSbSAHP?Q+D zQ~{P5Q|T&4>@LN>YX1Ddrs}oKkZ9%CS03u=(hjA@>8Ax5!w};P@cO^sUND(4KS6+C z1$EJJE9A-a`9wYuNpRh42LP0Oqjbg-U8_auJ<0!S4f@JCx^qigRSePGL!z?b4SG&( zFH(ROc0xai=Ipr!;*w3C(i*^V{&bQEssWNAd3CqRVF|xyc+=E}530Ryx74+-OpdC@ z+|SA1Tn3*ZP9V7r6l+&Ip$cy2%ft32dq3JT+5E-mfsTBG8;$pPOLZt1&uPbfH-mZ1 zXZrY>^BG2DUYz#2UA~LAg(5ReaCZ^9$B#6lMCS1P8Ztu=CK+pHLthu6lo#n}d>K!7 zdtX<-=u*rZWau5*JVS=M6&1aKS#I<7$9aQ*QzZ}uq`~%uKUHab0tA(UiOgxeGE<EEmtlwfS!j$t zfBYw8FzV}pY!SD=s6>Swsw5scmKY63OjvA8{ctai_v2%_+h5Z={%s1|8CEbB_xIm( z7CV89Him!a&1U&^&UfCi`&uIKVK}TCeKqWAY~!EmP`s^uFB;>-ZeY)zR3p-bVVPbZ z!o!%@Xp6?)k@xcI(EWK%4EpgVeKIFbB7++znc5;LyVetWSfvb z`(h{U5QAHb1#*82bLX?Kb_Dz)ikyq{gW7HQA%_2@DIa9ROW7%vtM#i>y}SA)(1IQ< zkR0yA;05wCE`K%5=;JZfC@VKJI*uY&=59J$f@w=I> z!w!V5BW>!K-#GoBkjnxyYW1Gt(@3_kFiFX|hg|4cmR;0{k6-8+dwigNO(?Qx^k2wn z7{|rqJ{Ds9a6@tj&SFtxuR2`b;~sCA+4OsRK}OwmKVPmw{a-3j;`?>tH|RLJXJbZ? ziHqkDu)DqNci7Z#*rUR7lC^S=4rH^4xr+}4AJnib<^_}J=X)H!PHZ1ZYwB z;I85nvHD82Z_-O)aDWUkAC4meXqxD<*Kgtx8jQLXB^Ij?#2}e)`qkw`cl|pb zXX4qvbymKNvyd|=ZdR2EuO-RU^Qer1DeBF7+;2jbNl-W7e&bb4ykUm5Z`r{LggEZ& z6#SJnbF;Zo>j<7-JZ}wUvZcQ}Z;@`lH}ti1QY@d*y{mtQJ!9%SE+y$ove+%My_;>^gNv(duV!fyL(#Pr!%ZoOnWY>OgIew}!`ti{4 zSmCIu-~Cz(uaTRaqn-*kJ)f5l9ZEVcL!t*jPIm|=l8Yn3{pE~>>O1Q5`_b+rO4DNN z-$NUQI?>}}Rr2GVol%vw;H8*X_H-_`awT_eWz*bx%pbb^GswZ0Azzm2GM*}9_f^RQ z+y`~JvcEaq)m?{2UW?23lgGvODH#*|fzu@$_4~OZ?$Q3|3v$yUD$qM>ont;J>wSCU zO>{(C@zK5;uqlRy=|~H3R-PgfjNARzRKTQ zR=BFF>4iHEv-$21PF2(&{M`az2-BlZsGpd5V?YIDjOUmz60mtxx9wZA?RQi&buChF zebBGLHVfP<@@x~RUIs7AOaaL27gj&#woyW;HGw?K56c?Q8|7L0u(rY5)HP+cZp!1?Kz-xS5q?> z={O=<^!{3cJ}A(0edw;<^S47Nw_cL0_8?@*2KT;YLKtY0To7&Aw=m+q8x7leqat6o z*s*{DR7Bi5K#Pfw#Q?-={aprpg*#wXV|YA-Cgl8u0+(wQe0@*EYSMI{{|E~(=$j*n zxuSl4Y5caic4f;w;{6-f2dX6IWRkL+ym7c{q_!--P{=rCbnt+BCj|{e`^!7BY5VTB zWuLX!jXT|QMtohL`^mvRkiA$&{N$H`HoxEf51Id?;T=<9Z@_zu&&0`fO!S~0$1si% zGIk?u+&E4;ROycgql`^~LkzGD)lmb)E04_|AQy{I$dpN*3;fKRBCmgCzv+6Mt9zbp z^d>V`d{u~0#jwcUik$#|pLH&}`@58AZ_jn*nvwe9v(9({#n=)S>ALAcD)U7nJJyf*!wmQAP=I7mPl2srTs2qvjtC$P?^peaG<9 z!si;K+R_H^wr&A*mCaAq*XQdEm-a^;st}hZ@)16ze9AYxUKS>Iq#bDaIBI=1vozTO z<$jvvIOZ&QAJDCT*Gw)7kJX+2E7JD$0yOJNZ{XduE&p@P&;GLk z6)|Mj)%wzvDFz5>sR&yNhHm?@axWQ$e$}*fAN=3g`1FCnBB~8rk4fkDvVnUaKQcG2 z5-D}J345UKyA4Pa^J*(J>;#Lrt&bILr=R2bK9OpJz5KDm=?TQlet{8<_=JjDt9L6+ z{>ySR>U3Mzypr4P&g|`WK#iTcR*tvC600<2r)q}Nw&FH+_hZ90WXGzs*Ccqgi8%s$ zuf5eON=5J;{rdmyW z3NZ}rd;w3~j{p{@cc+U<85P;yCJ^Jv{Gt)_>=HN!lIh)TG(F_}K|9l@Q^Uk%PT@k! zKIQ0+uhu~0_Lqh^DyJ1k=WwGOXRi%Zy9Y!4PVn$>p?=-k8#~b@scadhWzq~U(u4D# zN`t;UvMyh~lUDuF8vig(5jmQxIQH9UO~K_!LgmW!o$rE05bm z(%oEVuC@I2LcnqUI!n%@?GaaHM_3-Mn_gAd0PtZ7w!Vd0E=T5T@84B6Ydc94qaI8e=q{}#N-O8fzwf!;1NYd}KB#yj7Jb6}w zXzqmmnundHWkw^YuFP6g6;207oW>Q6-n~mD2ZRn0imvK5U-kJ_vAw7EWE`N00=*IQ z^O9ulem%i0$#g~M#wKZtNb__el1IDuzqy_g6X+h#8n{l};U>AmzlKf^Rhs7tk)8Kp?yL>)y~jre(B=F8 z0o6WV@%+y|TNcou2|S{55@h*Gm_vqVZ~sx~8)eycthMRk1P0qTDJ1|9*es#g&+uy{ z05+N6hVg4Gv~vh|ui~A^15o%K6+;rC#=Y2E;6i|h%2BxjBBRsoN{Z)MY@fSMtKoRXzZO93J|2F}s(DdJdWt3n6!D3-0-?PK3Kh^K0)m)p)POs_nXpz98 z9+?y+xe~RMds%xxlh&OmF$BO+uRVYFL%K11ad1mpZnNxZ)6)SapM544UXv=bz?_*| zW&_=UFB;XKs_b-l4~%4QuPB4;YJ;>CwePlUo4dmPd_tiI>5+%x^=ik@(;0t*-CSG` zg%r)2oU?;qyN`YYUpYL)FS!t{x-Ix)<)%1%hGr@K#A z+_D_c5~zofKBBr%Zl(^T)czjR@B2B8wC;EVa#RJq%}y~sarYz5JwexI=uQ_VeE1Ww zF0nmGoKzx0CGaJPZPJyLVG;-SUGTFYX>RS6ydjaoPOPI1rjefgykAYMbdSZ27`|nN zt?GxI<&5dyGM?zEy(almSTJ0d%gULp_E1t0Qq8iovp`X`ot|UwHNA`N*O+X z(od;tsE1d;IdoUqCT_Lai+A)w^sXM&wF;U#O+VE?Hxs{X3<9SUe^0tE6R=RH3Zf$w z(M;&z{c7`!hcmH}?PX@m>O&C0kqgP=N&j}AmbI->l)?a`a2)+PQZWdWNb85eaLTpM z-x8)6$p(|LHL>!AkjjuIT*Gd%ZNwU82TAQ<-it9M;=A-z+O!nEc0_!&(t|_kgfGLw zZrp`8qeX9^0-QQ&2({)vFA`K_M>((|#`H!Q9*^S2>>$O+;P_#wt~@R5ssZ<`b*bX_ zPV29;(YsDHjESkju3oC|iPsL0nX!6x3sG{y`ODJ}S9pn<{6Hk6Ohx`o#NFcm_qPBp z)`u@zxer^GNN~YE^8%6}(9Ur0d{K(bkN!BbLRZR=;>!H3t7C+SYrSNJizd;D_r;uqgG-HtW-GC&)uu z>7j(N+9gT5JD(R@Gl=f8?D%Vx9uQqTR~Y)_Z-uFN6%=^wf zbIu6v%<95<{^I-l9>?c^!p(TPOp^{ehpF}~)`Bo~!>_-!TMS7Ciib7Xi860Qnhik> zp7!&HFR1yhLEn8?GjaVGzK4l3Zx9dA9Vbu%qTbH5bW$gYDV@IRfL%fLe zI})l9>2~nn=Z}QiC+>{?#DqBm)rba`Yh}NeRTM@_jv4rozJV6up?6Q3g>E8YC^rKEg#lbbgY0@K4>8#M1j%RknS& zb}=DbG6=zRx3r-RM0gQ#FdAR%;~eWwxxFdiQXXPJI-eHs+Mt7(va+OuozakkTQdh9 zK$eJIJ;NcwXDG40mYqt0#%`EUDDK=WC=#Lz_|ogPQzCQ7$F13hk)s$t`L9#z`Du=-9* zC1GiF^pXZebknAG^>SKJ)=V+o2L9p;t*_Qy@-?hIeSzq+h;hC9I@Q^N?-#y#8nt@w z_JBf#cOc&A{;3XBte~%Y$cBZY4*a`}xS@E%A|H(}qopyHgznl4Up{{6jZCYFAOg;J z#sf0gju(`IxC(c9^|)5VmW`iKd!W;?Pq}f5Ls}pBzAdZ)M=DX2zR{`Tr6OKXMNxZf zf0KIGdwnmJkZyS)41vXn;!p~%NlmzAGUMs3&J_DY?14)7E=DD|C#Xk{)lF{3dboJA zNV|AroUaO|{)3%lgrX?jaC@{I%+<^oQ!$|KPo8AS=<@RmTPOrIbfExCaAo5Ijs$xG zaI?I|8bh6VC`xQgo|fd;PxWNB|8ketU=_iBC~t~*MO&BT06P|cy`@Q9|P(-X< z%n~M%9MDce^%4Ij-$TRllS!2L7ucP&h-cv<7n&XSE%!;Xi;x=4WoqnGD^NNHI^li8Q>m$HKB8-~)fr$!;qPXO z6{QdGZ!Hja)kKt}s!Cg$B0_qxSF!uC_j^FtEj_q1w+j>QkJ=ns`cy^qUWqKY?!;=@@_;oRP|4;pQQ=Yq{x! zWhmeYh57XWve6o|gy9@N>NVyIEMXdl3oyJ(o@VHVwG-JBaL}- z;W|Z0qn-oo;LAuthLa7Oi^!WLgD>zXz9{umaxk!#ms&ah`*NsXT@ti=le}F_fA-S-QQsQ$Uj|y zqa(j2bbb|a_m#M7(O`?p;}H>BFAsnDbr_pgc#j9p=KnPugik8pP~HZ!Kg0gR8Z95{ z{zg6=S$j0cECg*|4teD+ix|J_{yYAqk1PTqND8NI20h#dnaHnH(&6bgiw}}wiCHd7 zK7CptuZJg@xT<2;6{SUFBJu1%k55SgYPZh(K;k+6{UV`3GS(NY_I0MMuh1=DZ#8)} zdN#Zitz{>uoNQ^12HCBm$83Qj_tH4*fJ%Y_AL8>QiITPmL~ytgr0{&zS}l(9T}c`O z8nwn7XW#=-F8BH3g?G^b8sVOU|Bib)KdfzgObEYf3UK~J_2;1kyTm=8=YMdORPR5k zHjA=(mi9CO(y@j|dFP&$8K^nvDK`aZ#^U*1kGlq-(qamc`Ckh(9(z2fuM&@xGVVKe zICLp-_P9$>F)y3bC^1eTk6!!2eO2N78Y7FxGw0r7&mKP8ctuu}xBotT_#Hn_$J2u@ z{CPsdhGtL2G5C7oxX%5ok|?g;2@{^T`-+kNjr+eK`}>qufelx98m6SgwludCI(Dk$={wBv4u`*TdI5R=? z%8CiyFBP0LtN-4gs7*mdov`nLx2CQ*vuUIEYw^kZly6*diK?^YssVPPdU_EndVjkp zQb=t%N<6uxHIL4i<@Ew!M?LkM)lpffa`C@y-Ag`=Nvw}#?0HJ7A1xd*uq(DHu&=De z`xy~SJj54;yp?~8{afBzK3x8dynh`|>-YB8?RbZ0y<5ub@QZPQ0qXRXfA z3Uf4T_!9EL?(i_FWN+_b1x1KPz92l4HbmH!iTQooU>pUbgDJ0E!>+pjFi*4%l3J?aR+cbxar-qY4A6YT1 zlEXLJ%ql!9Y4IT;EZ0ruWX)t&Y1H7CgHqnTYf%vCvj+{_(R5f|ZOc@oPCGDI+frVm zMm1goJUiI{0TSjXMRtI6-bHZxitY*e>aV`ql~Zi55peOqO~mfNX0^ngcsi`~-1yP` z{Tq&=uMeMs${)H9q3iluHlMmq7CVT4M_khCVy_r1*p(_YbxnNNop@8M)YO~hV-PU> zcFQihk1>UN>*&34YNu#;{X`RWOTX)@$+Z1G(fQuR2F7A(uu%2-`3=GKRdjsa^ZLCO z7J`Xp9B+f5ze0v>6vq=kGiCJlKsta{!Kk?6-eCWA^I7)wvG^T|%0n%X{(V!`W{+2? z1EhLCmNuU%y-sL!@d$xLg|hI2J@LMU;U7^7ZK$vDHJG(=e0v&y9M2t{)} zBrt~SKifjSir?|(RV6?fd-Mc^@+==7xaUkXl07|1E6NSTDzNaB2=wVhAd6Mjy43Qjp zo~6iOyGVC@7yL&+rCm{^nTT?qySylWdL3!oRhNg{rKY;ZWhA{g#kIy+FcV_)NS>91GX1}FkytCln$;fR8p{gWAfMy60QlDClOC~u573AFu z&0ANnUmP%oz?Rm%ObwH`50Y;X>fg_-n#ko>pRku=JuZ1n8C!@zSO_OV+#5ur&Sc zxr(Tx%r*WK0bO|>obO02;qO<`q9yJ-P9wtS>@u}Vxm4>jC8cXL>(B&|AmFocDXmHC zE5W?>Za2^x{F5_Q3DkO4bV5s2Oghgp7=NL_D%lZ?RQrU8>{&e8Y4YueWES)Sx!+c} z(BXFHVw3+=?V}Qyqc6COJyaLQnAB3PC@uVU9A??NURG{ZS~A;Y#?(8+e=&Sntn_<@ z^JA-V>kO`}Nm->;PaRqV)$Y)P&2BN_al~i&f}(!(+4>*qXA@kc$7P^U~W6 z6?E5;d09%_f@insceq#AU<55n%*0^hGg|!5UncL0M;Lq<=p+RVm;$1Kl3q%1KdQC5 z2>?-)?la%qk*tA*BfRvImW?0kbL*Y8fXTOW^ff0aF5g@)SPHnVjIUu$2a4=>RMGBc zu}#$VugLzze~3+@Str-Otv1nto7^>||E??2M|b0|Y+@!Ea1eJ9Gum3LBQt-D=0cU)#olH^N?nB%JrT^k(0%?R7fZZ%!`MVLi}Cy zpF87+)EO0a{$U-atWhJa6;8|Ks*n?EFP%SO;4RMEQlbNFCEX29c%wzt)fGw|Y zU&OfGuc;{KvxHqaaQtDg%xENcsNfx|93t0;fau)SgxY0&I)jLVl0@d1<_HLwx^m+6 zw+(QU`*4UOh{=yO9^r#MR^SBv!e8t7??#|co_FO~Tr472|2wnRg_Wa}lw_ z1*It}`OrS`5YiX@CHLA%FMUNk1AG|_VaGq5RuSW-E?a(Pw3G;E43%oDRS^ylanf0V ziRT%_s1yA~%&Eof(Q_L&dgmz%<1}GL7xouAGS?-cLV!!fMPWdmMrF=FFvR|&xYOC| z>87R(9WWPv_1Y*^1Gp&U)@GUN)EA`G)aoX z`Sa@70gIKv#2b(4dzEV(UEh0atYXm5N%fA|;8x|k`)~MWmB*x4&KkoVK5gV3RH$WzOzn;AvfhK+9fYl};%U03`*417`8HlGp70 zo`a~5Ue-yq6%6qv@MDJ1T}NvYw$&jM@T0?7`%RnJ`c~pqb5R`U&<>ky@4hui823Z z=}gt}zlu$p=)(#8%!%j!wz6cxk2zK+<$4x61a4R->~cs=X4I_i4t@krAS>^)4$B>P4~novL*;)MR2_F$P!^Mj9LN)W|A_I_+Q{rqFT6>Se;SU-RzyADlC zuSkTG&6E5&{-HVTulvI;dAbRKzf=O?F))ufE}{?hFylr{YW)W{mYjPlU&FJgco|P7 za-1?1yy+d7bs)5Pkg;o03c-ZPu3t+JB^Ij`wW-p;)xyK|e~kWi@;AXnIgY#SWmJHk zWwd@YZkB%;t>-hTWCqO?G0qxIbtHTPa6?JpfbK?2o{v7*inIr_+}6F_6n^Bv6A^T? zCizXB%4|~rEoikG5?`6@NvY*xW{$S+D7D&H@-p9-GA5lXm{s(sqqagy<2rz@<-J=B ziHoF!%%lZ-Ok$RKJcEKx+ZKjSC7KVF6_K}XT>8Fr{Q>V=(y=c8#Vb%;jWt$gwC>>F zabVjm3oDIgJ)81po#I6C+fG>Pu0-}MM|{+_Gz+H}NAYjx`iXRk@Gn+rPWoR@h#FG~ zwSVlE?yf8Z^u-FQg*+E+cG!&Q#BrMy*lZoJWHju9P3&U8EGgsW)Wbw!;n7O*g`Wz^}Y5oS{#L`lP!5}+5xn)-F;OmY;g z?m9=^kL2}$`ahqPn6m*M+)`8)NshK)6TCg5Vevc&)h{#h5i4T6OLW)O^DRO@bUEH( ziAC??Yl%c;DSAcN+@M2vU5a455Ap73(`YzVLtUOw;CZ*JB=jV-d6IoQ2ZlHgVuHj; z)oHkN@C<~U1xD#d11!8exfd93{gAnxa_Tq(JWf`Q9HVR>k~#x%@#%z9glGpVZcEP| z`kp7vVomXp;x^od-Q?uz3IOL6mLG_1B@aO%SFm!xJ*`%TOW>{UKxIE|1rqP<$o>&V zy8y{J2S30_2HbR)ESRx#4}$D$eyB|V5>3un04OCM`-`#~Zmd$?TG538ut zPL{=TdX*xJQjGZWzGyYAE*)DYdtZfe=l9D$P$--Bx7j zcm0oCj|A6-xjLHFrQ##fP)92C`w>9nnsX5`ng7D4#e^QHlrWA9;RLzYrK@X;}nZos9E1SJz! zC^tAHjbJ!)zgsI17h!WO=L)v~sViu>Xslo{rc48Q^67p3KU_HgoYhrooyuww9CXeE ziRZIG9$LJ=Gy;^BO!{ag`vdElf#2{K`GceDcX0n|1nXaZcyd-zI$`A5Ulljd+Ijc) zheLfV_{sy0c|+921aAy#3Z+rVu-nA%XoRTKO6ICOXE#^yzY?4NsDFjWyUjZVJkfyc zm{O!k=dDp+kA9;ejlrxz;YxRD&u0&dSM!u04=z*Bu`G>#3(J(x(^n~X#SC}1jGU>G?k@w(mgh z-Ym*ojo!IP=7tynGI*EQ1TnIHshr(lye+{J`IswE$|FSrtZx=*{Sk^2MHm&(IwaZ+ zJ}f2_kFgzV<$q)q8&8e_^YXi+!d?J26D9x_)!X}i56c9QHU5Kjv)wxc%bVvNJ@p~D zL)^I7i)5xnlCO770ovR1Xzxe5z`70JcwS7U1sjmjQk+H=a-9Y9<&~3e5(6kLSlc3x zd-nYI{*R3S!!$!L)e2{?c7}Y*i>Z3kqYr^S3xW8w1ioF=jGNZ`omx}2j znm-2R;+Cz)QK0;3eOSaA5YWUyo{-IwY#%S?v!Gm6Otjd=RJab?UPxl_d0OfiHXEM$ z3%_`X-Ao$*_e9m5MJpP2L^#C^e*{$jfbL(!tqIQ37cjLw1Nq7!W`waDXppXVcl@FJ z$(~ym2$b}?DkWSTW3@&-(>MPi@tsqqzSpR((*P@mN};TvwBS%!j)R>b zFcMZ(&T3fUMy&B`r2r?EYc0UImoi&y^^;Nop`d5T=Pi<^r=QGET)l5tP7M3V==CYJ zX}SK-v(Ut@@WB|Tvbro_H3&=3e^0ww3CQ#@7yQWkDDEizwDYqyLW?Mj{Ky09_*$py zi@0)%{mQ)wQ&(VClX_A49RGg|w5SWsjkx#vW)6D3_w<4F_|c}1`+8PvRO*qD=G;2h ze1K%ustSCIr>F-RXDeBnKO7L^dTINE+Xen;y)*QXo_!3Y-$EzpnGZ6l`1j9n;(ers zB(4|*dcOMZZEY3+l$EtFKI%8$dtH^k`M3(P%R0vsP`&(BC~YTnDU z_M2!jif8j_N>7TSh5j59%LOOSK=*Pk*?gjL7#h-nT704HlQi%OJIE??Q zWPM;UezfaTEwL;r1Zh66tkoR<`w_=H=Ks2iHHb6Q=H7U%Cx2Efx^smgxEwAK*_Axd zs$V_vzEiKI1a#x~jL3dzoG5>*{#$1KFJ8#M{!}Z%1;nA)C+^GVfC|d-Vl*1){T2S& z0jNy2MoLFb<>f*mFZBu!9KMC_299ihW7ShVA~7cfImNEfWz?&J>&FW*5^psGE^-|BW$KFYda?#chr&x3 zkVQE0oMZ99$}+NgZ7a~Ds9=u&LA%MY@ClCF74`tlZx{65Ns*Vw=MK7n%rx3G3H<7~ zwQ;QDCmwAq0#`jt7_a^w0OHs@o%gzJ6BvkK`wJ`{vwBT^1dx~D0FmHx;1WtsaM zGd*iP(J4Cd?4<>CsS6upSJweRZ`3KYNU<6wo+3Z0Y~|h%@~j$3`ZE8Q2O!IwsmBP8 z!Ufk7_7?nz6(MJJ7Q|2@_Im=ghqP(>Z;nak1NJWlB#9N0c=htW+lUe>dUt}AKIHWC zIjydepFKz6+uKu{9H#Lh$sG@wH$wsTV^~Q((MrOKImRiGoXJ(WX|o-7+kD=*k@6I~gT1^7-H&U+C!K&wu8bJ(ixifVA1wx1qKEc3ll#`)%Y}ac>3UHqJ zrjN%Xt>WZhROR5eKMS}>$gDz2b2xz2ZwuJ%au2{T3@cPbofzGC0+SJjNX*y@9v9`-ZCg#(HcR zPAv^O<LJyt(Br6nn z4pbr5?yORfd{O#E?p8U@=hbpL)8Z54y6A8CEWmpi5Z%`7-y#y;DSpiKB4_aY=r0eM zm$!hzs^J#^bep>dL|=Pimy`Wjzu!DuQSE~;;$YX6+*UlHzNHAiV?G-#qIbP0o*rvE zwk%AoLwsG;Xl|7uy0uYO@TF{S;x;G1%KCeLp5oEFYqcS~6xpONjZNkM!lM>{6KR@} zS`rUJw~DVz_^}XkEQvrE`(gpB%MpV^`7OnhdImW5>oy)q3xd)dahCt06$8{xhQ>jn z^RutUDPm72Mk_bU0s#}L1sDJqT+ROhk7fft7q6Ihem#i3^)z|aMx}121hA%ODE-T{ ztoa#{^A$j7XLEFE;k}g^J`VtUx>T*FxH-#E+r9n++~ps_`%PHCJpRm9P1~Vq1b&%+ z7~1G^@$SolDPe?_6R?O~fEuLa=(by-{wKD`{l_#Q@fkM1=AX7iFjmdH+KNs)0NIfs zVDx?5fIE#Zlsl78r;~sCB`kQtMYpv{-KMQ0p4+_UFDacu`hFQ!ZjJjE7tbpHt{#A$ zT@o&G9+<_H{%dEKPQRqWcohWbz&V=uhqYTZIaEp*G$%}JoZdpa+oTD`@a0t9)?8)B}_P<@E z)OYGvF&m49Oc2E(jAr$peIfwNwVewF)^njhvWp=R2`*MOym9U4+=uo4KjmI<7_9^; z$G^Ai38^RqW+w3>#4>BYdAKBPvlbyl{~8b`(1XR7*y4kOR%=+hSJulJE$+*}fhD5v zZhUvn<`c*$(KJ(qPVHkI1)QsdHrde{O-IFaW$_4hmz`FaFIw!hWYYn|J_K-CQ!U%r zdj#)|X$7%kMEpB&P1;nP*AmT~0hFWeP-|gKEh8?E^Zvwr?Uf*(uH_OENQ$MTNy z$i;I*VsbGZ7M3TKfA^CxJ*9tN7JXPr_fE?Do<Qv+ZSc)I^at2M#nxC4wS zJ`yitVi%5U2OnC;0oWGcL(Wzfhy9iE_;O%db85mE(>wp~S5G`{<^3%_j+8yu!(MZz z`FN)LZ)8VSyM2(S+5*JvgnH*+Nfi_1viFTpp3Q#QSV+bAhMb31l8_73MCESfwA$)m z))3i~W4aN3g2}ALTa597<6U+7*3tA=cE|J;n4=3f@m%(UuFzAAq4IydD%O2^8<^y5 zH?l|G`@Z(;K}y#_nN60m+nFV6A8U>=7^Hj^F?kaUo@P=ac(2#iiRQoScr-+fKd;1^ zar@>JmPrDU)@qf43p(c>ei9m`FIaqAscF_x`Ka(YAkBghci{xzL_+{~Sg_x-Rt4ml z#@ydKgKxNNlURM)z-IRam{J_$MFk=o05d8X*Wr&YWtq0F5OVlZkIW0b(Au>BxnX6h z9}^~Fq|X`juKR~_x;-ffxpo-C5LPQEgkH!Iu&Hc!jxhHl-QjAi?{fXg6KmhxL_vXVu0feDqpg%Dy9AwH;o_-vGa+{OJ>GF_t-#ra93cw0n8 zSc1=pS(3-SEDj)*K`iH79cUqpcUw^jU0N>Br76`zK6(6{|5SQx(rH5#abN16+3Me0 zTGhJTv&vW~xg&}#3a+lzb){YFy7ml7#qmt?J-KvUKv|L#U{?cgwOI6}; z=Pz)ugr+svd&XtPjsAwj)szh4NZ`=4`1gt%proCgI`|H)}&m{Xk-d`_*HK8hi_+U!umI&6CfPm34R#3I~~Rev+Gj15t+&gna^cgcu-Q2keIz{Ii>>OQ)@qiHW6KRa^oTU??j`%<&GnO~VL`-4L*^WN^H}M;@ zaM+8DsGSd-l~7Z1t+mJ5ZUj~bMsa)y2gvz)Z5<#pF_imgdmz+6B9V9pwHWU}H$_#qyc z8aqH4?70&OWEQ%By~|4vg4RqKrVbGCt)NtFxWHIF#XQ@z*`I*A9W>)Q`^^=`9)=%M zGHxO7vBxV9vnc+gsZ{<{Y7s z{)eIiz84!||J;Xvzl!+^c+G87uD-MW&N=@|+o<#KQjwO|GppTOKbUn}F>l;;&>3D9 z&1V50WG{@JANPO6I2Ok3VEO;&*LM}`g`58auZN#_J0KkrorO|0*(<2ET06)rXlc5l zAX!R!ZK3VV;#)^TT_6CnK4ma#zN)F@X4LSUWvOk<7G|mHZ*Ng*RO8h4jSSd^0Q)1< zLYqd_bGcCN_O?25^R^z!?1HV}^BjRghaI6t#}=iXcjWrD5rE)lJ~i*fV%s}nYHr!6 zRC1pH&ti|mtdic;?k*?P6>nv_TL-`A&%7@|Rx<0rnf_mB2=M4Dcv<3lCjIz_ZI82? z?Y`qeWswBaq~U8EwH*to>}Kh_8(@`5ZASZ`Fbh$u?0>TG@6dqX0KC51A(3A=wd9lw zH##2RtnjF#&eaJ=;Ii>cyH=22T3{PMXjGSJb7+_+Eurx4mhhinB(M2zN*Yd$P;3Lr!kuj zJ@{EDl}QI!2^k_KaQFnJTaK=Osz)fyYFyxAo~csdJfAhS!m()|5}fh-B4E;A)yPx`TqFdH}D) z{JFT`r$5jJjy`fxzSF)MG^gG(n@@GAZnw)Re>Ij|KC$9buljJIe0%*H_&0onBEAa# zX*9WFGFCX}-H=XeC}_G4y_|ZlU7#g!@$CBHWI-`W?O?#CYw@l9)Y~hF*RNE5i?21c zqZem*j?dB#>K>cw$#V>AiM*v2S<4JEKll(YSl!WT?{FMgXR-QBzodJKb^JS@r}Af@ zXGIjYz9t9L6A=JwWyK$mR(|EFN4b55ca~-T>Ml|%jLQ-^T!M!t=?z7vo^#cwe#UnM zt{p;BgR7rmhFR_I!}-9w*z8d0QAL|kN+JUKlt)W7=L6jJp&@+rte|07%IaEXr{pKi zWi}vdMT5)X_#qX%&Ebf?hEx}iMI>>K$<`6HbdeJ&wmwhU?nyld@8fr%j& zu>!g&sPqHml~uN2uQ_)(fhp0AU+h=8t$fQ*{4L^H3J1OdJT8vp%L3{G$Ejs^BYHq7 z`(I2(!^zBWFpetcWAoOT4}M`oR+dhNh1TUycs9B8Do3e*9X!7CE9~9{33`OU6M)h6 zR~=6Wiix4E9a4PD9rre8?_w^umb){l6u%S!l;wIx7SFmm&ObhZ04i2y$9mvxbDk{sBSpz5@3gR@aeDM9t^+ z+H)!_T)T@eTz7p7$2oMow6PX`g)uy?lia@bFaE zr7dW2N3~|?!@Yl}Yo`I1flY}wj2#vaORN@Fa7H*VNm{HK`uJM=54qh$#)tk6o4sq|S=Q`OWTa@l?KDGc4Dko(4{6cpFG?fuVgd{+g{TGuMMFuqE^#r z<37;7rzHU$UBdr0DAg*xfgeY6WK@J1jzk;i$cM&5O#apx_{bo?nLSrHGLjDI zS7)9!Zk*-Hwb_cvR58o1EWX}CC`@yR*46zZ8M2&&W@a}AR^%Rpo-J&PtQZip1aOL|5Gu!=+ACVpPBD&D_ zBoOz?n(!dC!%|dhk=m`szse!(wHd(;I0=gLX44#TJhX4Sedi@CkjwhfaIlZ++y&=*fX6kb``IwpqI-MB|>d5z^`57bbo36HOw3+Cq zia&cAR8rX!xS`8IJWq$)D8a4X$$lZ^-5u)ip};4wDQ@!BH|zT^O1cm~WmZi7{wuxs zd4hnnwuV!5Fb{3+y3t!Ww3eujO$hgV>zFM7V`uSlj&&@KiN~$gBP@vs=`rvJ_6UJwsz^DyM=)Tfbk7(@buosCQdq=z zL~Q|}Kf5p&CO9GhutvGN3@6R;>Kp{CQ#SyPOYvviJdR` zLhCn+?&e!f6ui$Ng)iW%*y}WEp~ZK)OON8oiG()NW*R%nM7Z0vaRg)dlAY7o0hDuC z_>umD7&a&WqSdUGNTu_S_$RLdN*h*UU99N8LL}-}0u;AC7&Xgiw&Cq_>P}AmSTRA` z8?a^PYteMqGD#*c+J#x@zyNn~3a8HQC-v_6d+XZ0hd5;2P5rzl%|RXjyp?PuUO+&jUu!T&kO5A#+CkSWnClyfb@{eY1p!ljLYd7^@ySatW=}_rll`m7z zYMsPn^Db2$WJB*a-Pf^%2n_~};i%)kRDL5B$n!Kw8n8|NMMOtn^!_`r{|LjUrNL}K z!W4Vcv@R7lFdGm_0m{HWNt%4n2~Pt5530H*-y>5qO}Krn~J7q{B1Wc+-9R1>*zo61+H>ANui*0FC>0=OCGJ!F4`^%E)1Z$no zUsTqS)w1Fp;fh3gi}}HiH0aQDzm{o!L$A)z;(d7;MDn3j7%+oB;8wF~1#(zg>wIsr z#jlxfX+*r<9%4MlhqDs{p{dSk(RPI&KRXzulmn zvWnd9KzIbDtu?#HrW`TGm{>{dHPPJE`5NojB68Sf5PXw4?3vjrnFy7~V3-3!ba#we za}}L_@@Wr?eZk!I^97}0G2p9Nz$Oqgvc3;IE&|=nlafj+Pj7)Poy?j-@}?5@)Hx8& zAa{eH8NwKb*By6!^p5ZB1nXhngyM=^EHBV$djPYe4UzbHr&6iM4G>pjy529&0tZTp z)ZN};kfkL}S~;D`qLc@~X6{r31Yq3SOBF#0oBquRrpcOcoKj?|y-pdlGH{Vk=p!oHXcq@cPa}jlZ1JLXF zkel*mT_IGc)y!JPs%~)B7Emv9QW&OzVhRhNn#7tb|AhzdWG%w2fUyU`DQ9PmW-s z#4WV0daop?;emThe+|^w$Ze{L_mD8lP8YFExKkyNc^vXl-7spGovSHb`$P z&N{2iQBL5ivgiF*wDx$1{ZYw4xwB~sca-zfXSbT96nAW8kIOS$+mhB+ICvFbQ+o|? zz%0j_g$4uGCafrCSJ(8h;%=UhQ4HXMH5t5zOBJI4>*SWw(Z%xl! zIzlhV8B$*V6g`-1JEAPJc+Czd!sgkA9$Y^YWEB)IDC8JAMYfx8xJ^pgzbEaq({u+& z9J-kwdfI`zIVYvHtN(o6_y7QVa39I$U5BhZW*d63{7f(*1ZbwWgop#N8X*`5b_nlN z_UPf{WuUp$_U!uT_cOuxkd>G0wAULl9MKt-P@w-;`$n;M37|C`E&m>~6VeJy& z-@v1gw^H^Sq@CtENPw}a=MH?nUbs1*z|pJto zK}69>(ZfCT$+?nba~+BnwD%ashICg00jJU1anv)w&tFz=C5&z;f0WXlM(jQa5nQQB zjvlUlLu3?pRE-v8pT4K9qCfS-Et8Ed1e2Xnb88pInC%v&7Dv_6xl+7-7|w!WxRGu$#&!RYZ81WJ(}eKWXbpG`sr{6)M;*B|O`r|+ zGpV6lzBHToXljR^rIH8_%R;gzvfcN0{G$*a5purakt>Xsz{?KsTgUVuM@IBTL^l%VaI63db7%W5 zXK!s!dHVEY8TI5B@O0eGm$3Ukh{6qlfQrULCqfi&A08?l2GoTewmQ#xk6|^tO9Fpx z(ACk5+UUU_D;1&EGi1Q{>a9m-cDgh)ti)6NzABm9B3hE%G32pqER7f^T-i;$(O z#O!0!MRK{b9m_> zr6--M*rCuDE-PQH3&hwtn5xOuQy$ZPDevbO7DXnZ&rPvc;F4pxklveH01Wj^c-_3a zb!n`s!O1z(R>SsqSZpERXllOB1>j;;y(1g-oNv1HT1a5yW5ae7byBb`Xis5(`wy)Fus-Y13um!E7f*e$LfP=3o$@31H-lcM z4AcXlu3o!K>OLJAoy!&mOAJ$D|fXMSItkdtbo1&4S@Fh@cwLz)Td}cjf z$GrfWVNlLa8lumsGgDQVv)N!t){X02j%BT>l&E|w+8giREPyEY-vB;q#c{ADmT)knpbe^~Mb;e$HRjCfYW&(-=&;A7vwW<3&dh;QFPNTj zBT(cS*Bg<#&abL%uTIbi7o*%ISOU=@igHyXkt(D94)fO1*al9 zgi|LXH}dhPp>HHL)tw`gSAL^2-X~)X9u5C|cD(D}&PIyUpfqpy{v)oLEL(NiC_qQ)4;8 zi_+wP0w(hcv27&db9l%ouv(oXQ&YNDtoTMk$X;5xRTAZpoyy?q-B%E}pesO{O`6+B zR$UJG8+>%=Dkcmrc!4Py9WIiAptw4#fQuH~`pM|Fv0}`w3#&3qu`?I^k)`s?K<4a6 z=X=k=BsR`Eoo+X(iR9WJRGU+35>Cc>4_71EOnyC#=-Ko;Rw$kTT@KApu*V{iccu++ zFL12doT)Efn8!#!F0f9R{3?y`@japoWG(O20%dP<72aUghUBqCq2!s}$Fk^o)Amuf zK9~EIPH8qQ{m1roydD{wI{wpqeqwTs|!k&G=?a1*WFi;YIzK zqx(CdKRyKU5tsz}1&FWO8Q6XUkZNL?pr*x0pLPkOBCgUX+;%$STd|9!zqanLwX_qP zED(YjNe`t-S7T+xh#Ku}Q|xFb@xzW#<8rprHcolkgyyTFjq%=AVn8`hBtzpgQVh&^ ziwFrk!9~j(aN-|k3y_EWRB~7mbwO5YYC#1+ynoI2J+iGg^S43aTBDL5^l>Nkr zqj8fT)CsF~JiPetSid{j*UiM9wu-oZ%Mm>CFNvojgOsN1WB;I4y%_oHrC3Ds2%bU!AIg>YMyo; z_2FD6a!eE}rHXD&d`%NA2bnM_O>&Bcz2H&%N=*Vkp z>2eVDK}OAVxYC=>s;q-$4=U?E77^=6?V^^c>7%Iq{O}V~UT3}|7@@`tk9xQs*#Bwzc5HvzmJT&+(uvOX z>l4YkWgGEQ#x)OBNe@E2c=O3&tnj@$pNo z93zDOgx1EGjV9s|vQ#2r!qmH9U`N@CL6GfCr2leSM7FxxDyGp=Lv!+bvPBmxII5PQ zwGW6>AlU9f_h`#GR$VA1rd4f-^b~)jPMnVLSUFG((OzG@_u#TBK}y0VLS>Z z1z|Zg<@R>{tf%n}NA6IaRkq?TOU^k}7ZFxF; z+(!XnqbAhk)SQ*?_jfIiTPX}r5_XC53rjto)l&;*difLMqAZUGYq1hRU#Zt6V6zwL1*&o zE;%*ZmN%$Z^$v9(eS`@2dWlxZ@awcWY!_vv<&Hr6gtm8PF2zFqr-XN{0+xH@C>Uv= z4Nw>J+uLG#!9No?5Jd!54`g)Nht1ja^XiBrhPJ(bcBJtYN4}L*-rv?nj zU1+|}@;e^8()k?WlZB@m9G0&PxFu)dQ!j-V+#3LjTt6sMMr4TuK6F={QQ2MWF>Z3K z%T^GqN8PNLxw7dLTDy-GQl{wDl(=TWwlC3eO{Hfp4o~rJ4(aT~@Z4s^H^`3f6mQ%& zd6N9(bLE(~43W}9Thm-{%<2Ef+J6T%*|qJzxQf87G!f}dML?v4P(xP{kgg)VDG-X( z&;+E2C`Fp|UX(7qL!`F=p-2xPHFQuR)F5TAct6kY{myUyvG=^QhZ#pHPRO;cb*=L} zKF4ta&*p76kY>2j(@Lm{f4IXsVf)@6m{aq~8d~iie8(N1RN3y!@mHAK zABeeq))@3V+ug19=db3zVVo#KBnt96i$a1pNNNx9Y*X>7_8_RFCx(D>qGjG3t z*F#?ue7XVV@V%^&@!2S=iVNab7&Vr%%1t?u_qnomKZRLsg6-KZuzY)y|F;F%gQ~h9 z74ZY;Ls1`#cOHuG@@usxaf_M3#QnlD66xZs`c{9md)stG)vy_bCrHQ9V|zg|eu!{M zSgoiFemQ&Nm~^1VN5`b>Fi>vY?o?= z4cznRvKH`s`^GLW-)|N%&zM&*XuAKfvHYcheW13lax6W(oZdg-`p8Fj-_{72Pi#UM z3*_z~y}p^WL`Z1W^>5>~u|1(hYnS2^^7+iuhKOdnMm`=*(O><9PEcnd%d}~WF?}dX zak@6xy+W+N;qaCoVv2jnQzN?lfx*N=l!7%<_IT~2LSWf5=_4e2+u6yWXQ1;t)%4B> zdK1|@`IWK9IU!F5!#f z%aC5`C63N68(JTAaT9~(4+g^>yct6AS(-1nkBK48FI}2*Z&UiXXG1oQII(D z6P~Wq*j$f;>rt8L+|W@Mr!I)&F#0V=*N47Sz6~epE%+?p()-YxE`)k>wvTU6l0$mS&KKAAp7gs zDxb+SGd{APrp*%L_ol*|z;z3&H<62%i61GcSwymY6E}?T9l$6Es+&M5W*SydP6%%S zMAw+QL~a6NDQ@EYT>ho>`EzS~ZGnM*dUr={T%rs0;7tgfUOl(qO84DN<@(gYEDfC+ z@2X#46*>g642^yUOE2P$W;|H>ud=u)%q2VY97#C#vSI8?9Jasqe4(78n8BEahEEhu z$I4W()=40DtdqtRrkZl5zaBpw?H^Mfxvfog%v|V$nWzfJdIJuCs@$wwuDW8ePxa%P zqV;}2YJG^@+T%BhKCce2203i&XfAxp!LJxDpCY+f?FZ?F@k{zm9gL3a@A?WVaEbD^ z4UgLehQzjSr@f_wj&6@9(3OMi>xQ6?)$vjj})?9jTeP%(BeH!7I~D| zKInwSCn2#Ej zA>rE(LSa0pD9Le#hn8+w<(tl)(RKPr^%?U~eu@41ro9E8ncOYtgoBHl`6j)8UIqV5 zKIw^=p?G@H&~K07`XuB~Db|OV*?Kk7zL!oVy<_LHYa{Y0TT^+YW97x3n+$UWsGV*j z7GmnhN?(bOoIVW_cfRQH{ArfKtChZJ>#T05DBj!RBJ8F$82u%Pp4+{0U~`Cq(flA- z@~thgn>M&~Sn6t@z)pJqn*svMs_ z>2gf$1-R48-{RAB-L=ga`j$?S#ckmXmtN|i3E%9szGrMuLHH+Xhd!R+)c^S$-(;7<5?iQ2aoynPz zFOKiH5*4{~SG6%ST}$WNsD)J*vEy<8cQ@Hp$fa1>?gP3SiF&<9>C2r@2PC^gr3Y;# zqph0ULN z%@bNyuwx<;q@)NawP9+55fPKT7#k$0OM09y?msF4n={RK4jGjm+{(uIB5CPfHpkp! z*ItAD)ZuXFweD9WHQSco7g2rmN#6~O6NV^~s@fS!L06yNUb`>=nTt}?|IPD_Ka07c z${g*mq0_D~v&LI3tBi!X+U_HuPn^t=(3{uGbPHY&VBTlkP1t3CbfTmOjFUc9G$&=z zb|gXM@>iEScdy&}0y)0xTW3~RXpLaYYn2FT6@0dm34eO_o73{&l0M?b*13Be_)&{T zWw7B_NF}=>UA5XGo5f5~{L@^xie!#zhz3IgonY^-qxd=|@Lba+ zg{J@oO;BB}$|~F{M4WPl=7Epr;O+4_#HT-ZutxT3v{=1K?pvg0I2vmxTlI8$#hU4=&%)dKzrYL{Tvsh*2Qr^NOn<2 zAvRC(o!$d7eZahLv)MZ4(WB)FE#b52h=pN$gY`z1IP7080q|d{b1$ZJM${X>$GBjh z?MLFx!B^r#z13)lMEMuh#0TrDI--BQOk(0%DGo|#Kv~>V1?s=Kw2Io2=y;@=#6Y-e zT)_w(bmgb|sMiDRtF-kuTE=3A6V@Dm+ATNBAl)MojZ_W}wt&)N`+CuH=cY06z`5+^ zKnFKDRytkEILYgU;c%W{?9+%$XxT991_x}{{JYF@BC#GdYR34Qw|E_TInel$^OX=5tR+R zC`Bj(qDpbu)Vy6)aKYZF0l?72acf6*qyPR>(H94G467;IxI=3t{cNx^7UvBu0#76Y zL-MCBw4z+&w1GeaotMAGreZ&f{y#5_FDdx=f`vXBW@u`PT$iR=Xs_^!%deMBB8!g= zsY<$3(|bb^aAl6%4n|$DqPW;?$T&(LcMdGjqIp#Bu>-P&S_>DinN%W;KA?xkeCVaW zCYTIif2qh@u#nvTxII$sD{n2aci%LH7Ton5q`K>PSevNV$~y> zDolKY`1SfJKQH92jjLX>ge?=CBA)P|Q(GrF2ODV~(k|9viu}y`&8SBespm%P>p!Q8 z1vX1GVT0Ao9!DXOLd(RQeM9|X{G@k!(VF_zH6QE1j{(wZv^X#n`MDivyr+*%o_LN~ zLDZKfVI+VvbY!6b(oDMxs7 zxXN?G&0^vmrqEv0;-3N?Fri)^rKLN=B;9wKm#n-aWbU$04_9mub5hJ~GZMsR$yamM zRIVXNMAIblmW!Hr3B}Z?Z&0y0>s?J2mACY*dn{LW3{ukX2R^&n>5|l^Oe9Rot-1cZ zzFRmQg-cge{Six!j=O=Kc-X_<_bXRl<7-!sYN8N#JtnZx(>2%sCED*orT!c6eb>*v7MBdOv<~q+jPSF2 zZiVZ-=g?OYZZTa#X7S3QaD&ySI!iL#jU64hryxo9OvuwLhchyfZBR88VNF55Yq(nM z4EJJ;6sq0%t$QI+MvyuGl!%@==rgCbi+wgMeka^n)~k+-yb`AxDRSTwzc?CedUfPRq;=ESF&PM+317#ohGqw5> zdm20DVcjyZamHP$XJb+kE@q2*67IjZZ3?l%6|YVJwAj#Gsv^Fmu`afv^pxHHoyMpx ztL=W6JI7qb_AQ2tO(oDycE1z0-lZbcM3tz$CU(dU-?suMxMO>DF82ynYpn<-jWG8t z_Rb24>mFmr9)3^VSIsHIb1vpEOq%P1XA{o9>xJ|);70<^5p;+s+pandVdtz7<@ntA{3=7y7(A`B7I&rMOXJnM@b zTkj~7#bTg<3=s`NkK{@=fWAKOZR^j0YK z(|dW>->iwY`R>6Nda;?!B;Hu3Wckal+~SWDU-1f!u!M6x)>nyoVmQ8XQKODTM}tt^ zLUw)1ja!4=p}T-h=FCMA`j$xInyq=>l~U; zsDL69w6Lq7u)<8bH+-GbJq41s9l|Pv&HdBep?*C753mB`PP0GhS(BSZQ4KKyv@4Kbad zdSt(SA>6GJJGN22;rpKO6|O;%Ez1n?;%ukZ~^Wd0$obQc8-qQ?1^>`~o8}C7Y z;VV_lYud$e-%ibsOATtzPgdn7o8{ek?cO`MoDBX2>m@(SRks<h~7=(Iqdf}2DoF^k#zRb4k)thnb<#A(XzU)ck+ z*~Q2Nb0tfJ*qB&I`!l3;A{ORZQO4MuzIyD@S2y{R<5FGm-mHw1{MrZ8D%H(9*H?XV zyU2=`GvD*D;0$1%3x{EU+dl-(;5IdUwsq? zQk6Q%lLSSl4K0`V-V#wVEEAY_t4?Y6lOnsF_OGp1z<7+?`$tbLzYth?Mf}U>ESfqK zs5Zs*l@Ho6GP<=Ayw|ofY{e~mF!p?P_hjI;Kqi-+8+IJ?@h+LE%k7$~%yHEG_biIh zqMJYK^EtHOmeQAfG1Otwo59$w)y+@Yb9}u+9MnBjkiYo3K6X$ z_EKGBe8BE+iS8wP#UupiKigy&b_pyY(DF+dD%K`-<**G9uLtm|8>%$!4wMY=}5U%g$?zy$= zXt~Sh`aCv{7_&2Xlh^4-g^E28r!3P#MzN#g-SYQ$PI2t;^&28}^wG})9w>(f*`Uq% z#DjFf;}V`M7BqJQ9B4VmVIqNdV_e=nPfbkJHS43XZ4$xVEAQ%j%mKkaR#jViYSzc$ zQ$*NDU+OPESf++ToBy-D+ri0T?h^3|#{DtTC{OaP&JK?dtw07Gwtk1l#07y%UsO61 zLh{Hx8=%02Mr0r#k*~VArnEhhO;_rze_+xvrfgu=kqoh4xUC5VHjCuYP;cqlEybSL zW?W{jdQU3IXfL5Kh_e0P6fau|ezG5UFxbv4p+i1}-9(abR0YE}8dnkKoTav#Z5Dg$ z|8D{@g8-fCoJ*lzm; zKKcR-6U0?`PDnJ>(~Kj~<=+a{v+I9Z!Jay6$|xd!Gq3)#U;J_HP$mgtOhV)?P#R6B zNd}hb5u3}}y(SPjd;Mq2-F{yw<0Cx=IUpk|+c8$k0J;v_=Rhk>rYo5c4hSKMzO(-x z>9qOh>u6E(Zf-@f(E=bEhk%CDqtJ=JB%bR(8@b&j0_}|k2a9q*w*NRzfUBh9Ir$OW z?fSM0YPwB`f|FpBd;|U&SQbn0^mR`Ao#eKtxwc11g)MQ?2F{(zW9^bE{Rt(c6N?YL z$QQGFjsAe-rY7O`Xn$7pS)-ybVmCh(MV|kgh82kqcGx;Z=5lV^GH=H{uL;KT9Mr4otvh$-D?DK34Ima{9Yq zFe!X3dXK?<2UmI{8Xj|-eq$tMMu?ga>JBHnT!-GuGYsl}vOCsj zD|Y6tDNdLTpI8&___eF07P|Ug0w!DA{*>(QTaNadnUpy@XlFn_N1ET`%Q7rd-jmv?3zbsH@cpcA z$36$l%;1Yop+8d+r?Hto537&&kGCYqFnO8Gv@BLXQOulHjtilzy#YI^gA`2NtFkRe zD7~K7U5vlGrflqrVj->rjppRF;%kBZbyhEF)^sYEk<{SOJykq*P{V^Jb1b*LvZM+;3fL$5DnKaUt@wF&OdqIg4$Krx`4O@JM&6%%-8hM zjQuaN`*C4=|A0))|65;CjnbHpTlbCE-!xUK|$}2L(60yEw@pY6@ z|Ie!yfy?Vn27@N0;nf!=`qJwh)@uK}nOkn6r*D8+^(~p0-3))jcu4n4=}82}5)%b; z-8$or*tzbfU`nhn>g)2`Z%u7pwpkw;j(@slW;_)1M(3x=9mQ|ezo3gc3)na$HTy{T zL0(DWaQN8+!wHLc0Ls1S39@sw8@%>SI=D<_b@Vh{hzL<>vPcu1v*0+szR*T9OH@EMh<_t0paS0FMNVNj$Z! z{z%S>s%@PCn+^OLO2Mv)&O%d}~kb{L1;_uBF#>o}{Xp6X`bz$-5{T6qzqxOuCl6fRHI+TPy?qN44=r&Id=^T&a3|(|JM|~ z`a)M61KegFgbRNUS5da1K;*ga;TI7X3$H z;@25dT{>eefqkMWcv*7dw3c6xJ9tJ$q1t|3T}6`zOlT!RQau#EPg| zXe4}Dl-{cTv8{8zlK8H=ZO`){=nn=tbalS7x$G(%cM>80dfGjEBHebipbr+7lOz1$&fUT(Fxdc(0$AE8k?gNSyW#!cda>6Cn zb0D|UhsMX;lUhmPrmZB0pt~n3a#j05zo}Y!xc8>U)FQkdnhNi{uU^@(BQ)qivDzC3 zR}Difk2^_nQ=8ZOgb2MTpZj#o<~I7YT>;`Xv;cyLSOFyN$GRGI+Vp(YBq6-~f}x-W z;-eOMq-URbpd*;sH(4+-In5CA%^`aZKk(W)!)J+uT>=aiI>(fP_f2F#!tlx&K!S9B z`&EsBv>ZX{-K0SlS#W@-ss&>uaSs()bkSg$Ovie#K+KV=Pn>oHLHU@{7Ug&iEZ*=p za(B_Ouw7My#rdFcL)b{1qtQLN#hq)8P0!W(Af8E9Udo0IxlT*9f=^fOIdMkD8zG+U zmqmd4VY9W`;}cr|tU&0sy>IB+f1gWcWP(Q?@|GWdlZe-~I%P4>Sa&{?{C3)NNy>do zuh7AB#1O4F6mpUzZS>a}1V+OZ1@E4-1&&!t+1P=jcDb(^&d>XPy$O zDYRZDxVF zCkTsg2c7<$awO_}uFpY_AFcg?Vz+OpM=*G@{o!2S7Iu8S`N$KE>{oBf4SlK;%O6-( zUdBhYjky~UlU*Pm-sSB^t>nisDwwO4nro*4>Gc#coOS!Owj%d({J`otplIi0pcpW{w4uZw^{FCRKzH_2pU z{BV?;Y>Y&Bc+vlssuAd-UY!*rrALS4d8?VmXLv8&iivaY;o~-|6K-M z$RA&@A1=m!E^nX_=ta2>=HMXM+=+a>fEt})pG(9Cn6kKZgoYx362LLwO*m!Rcqu1zK9$7%IpjIw@ce1;`E-0lKi4?>MlmT>1zMmJ$YFoD*Sx!0GUAVQUjOp zVoOVDAV*{-8+y&shmRR<0BFo$KqfH8J`l$~wOq=ZF=JZ2Z1L1Vb0bhDcj|_*u8p`y zf4LzU{ZBF7S{=D)`sXJ6GkV%sgA2T}^ix58`P><8Z#p>GprGbGXG{mB@9LRK#%CYk zhDccoI0`k{`?8>{={4*G<~REMG@O__SrHEcY`koHb%|T?NyQxqKd>!l^wDL_2|s-? zct!*!j91J0tKsH*s7tA@j^i5W@JLeAEl+|a3lwddJ&+Sw;4&FFZb9Fdx~>~9{duX} z_&q&w0Txv7$96;(WzkpW2zxXheXrpWdDoMTE%sEk94$8E+v#2|*S@Q=hQyeRm`ZyR z=Ai`<@Wpllo@SImu=dsaSd)3&qGExEX}xy484kmb&diT6Ekcb`dPBWE>Yr(N^Ab<9 zHU0(`e+tv%;RI)6Wt0wK1evfAu;rlLy}ISuqBoC$lZJab+6zO1`E2Lx-iAK1+dv;dpce?2W90D??e)QikokEdy$&>>J(GbawJwVN5;2Yl&yfEz# zNLgb&^Idf6a9JfLy149tZLUP6RM!B6r<0Ff?gtK2qmGss>8XD_x3imAsPoj8`-b>? zA8CY}>IVf+8qef&dyg{8UdDEZAwg3el$w7~EZ>S%fDQdCk-ZA5bz%QFMZMN3MOyq2 zf6YsW?;&C{mhCU@UTW)VGh&g>&t%PBvcXz=Gk@8#j!hv*+p0;>rZ9(UG6j>~WOMQT zO1sc0u4-=&L8mEb%cmN)Z>>bW#WlC*1eL_6N16L{hA3o4?P={ls&^eni3LPVhBvim zxL^{@>V5WEzCE={jUd<$+6i`ee@A_DqSQ>nB<1!jw*CACFpzb18^nddJtom6@y$-} zSn=2h)^*|@i1z|nJD8SzZ1;sQ?h&tg2X?&m`fP#94<^0$nW}%{FM9q%BMizyQp$;~ zmbRc?gRHf_xj>*^D~S}DVuFNP5;%He#5GP=!T7)}hF#82gh}<2J&R=bAs(${fpfsg zhNtl-n2)|rOODpNrW>qpm@F#iQGAIQCo&o3Ux`wjC*%I>xq`+d6NaA>Y0Q9uoj(_{ z0NCdQjC%57Ns9GT_pe@15+Z?54BIv|L^M)s^K(OQC zBmKV=^FSi49XA4sm4(y1Ko=&1$?4aX*p#ag0g|4g4T%OOYAG&*#B*)>x=a31&jxic zb~9Tp8?|gRb{X+9MV0QK!(aJ(?+LMgG&Vr?mnR{6D!CG>c#KBFcY$NRrhwS3{me+- z#m#n7bc&;9ouI`B+#tFV#{<$@w22&kEpvF#=CZ^o*_=Np=?qPg=z3UkA}WBm1ZVsn zKUCaFWm4_8_l)bgOfC@G_0Do`7(Sswj~_v=nTuN!40pBl=Tig_Dz@h9*n=6J)5wao zK$$8I5t8MvhQO`#tNHX+jvQ6TBQBt!}tn3?u zR}Yc~Tyqzk6TbZbT-|SPUU8vRbnoq#kVm_Y}e<13f0{3DsZ@jR~= zZn~R1vBXWWN1ZP8T{AyM{4q$v-aRDPE99!{!1(N#%`O!QO#xr9JFRNIn*}>r|1qwh z(@Gl4=t+B^Q;XCtA#Et1Ab5AfP`82x83&vKi?9T<%*l+K9pUPG!9c#El+pfI-G1|5 zAdInLp%U2OoIdax-SRU{^$U^3b#nxPp?SLPP!vlHx5Zu8>Oy1t7GJRQk_Lhove;f| zvj6o*2^crqLz?mcN@#k@p_dA-Vi_mc_g5 zzH9Q9`5^XyF5-?@i;mFvDYt05EaQEM3|rM>8xTZx2~+2{O2A`FX7knERyd6bIO1HrK24}Q)c&a>*c8P zem8*G#4G|}C!iu^Gm@rX_Fj``X98Z_U5m-E!R@8y+jWX%5))TU_aB`929wC^$=6>4 zpD*rFMAIHH{YvGHoJmWV-!AFT8Lw1KyMHsIJm!LF@56c$gF`t)m4w$iRSQQi9Wee_ zefvUnJmvwo5a&EsdA~~)W8&OfCs2yCd`Ajy#Tml4yPV#2&2pU7d0Of*P_1_VmtG57 z%{~L=X|9E&n3qrW(NJwv{Uow8#6?y77f02q5OOjQCt|XR1>%j!r`U@D)R{ZMbrKAo ze_ifRb3Flw2n43bhOptz$DJ!-+cf1sW3iN8N#P=SMSx;^g`#wmbmM6Zb9tu$+qm!p z+LEZ#{s|z1EH~IRd~Na6-%3F_X5rl5|C^8`>(1xI#(+Uk;zv1aj=(c7ot1`Vk1&Y6 zedd>YZNpC@r8s98BjAX8?gwE*&ELg@@YU?jU&elK0;7fpC=ZonaZ!Jrr+=+CrHR5( zFBE9=m{-}XXnmYa*beJZ%Py%V=_K=*g+!dlTjzOwf0V@ev9% zx|y!jDdLY3(jpw3pWSIr_O3lez?bLzn5v&*UXh~1Pz-o5_&MmHlQ*Z;P)vlEx z2PxwCSqOQP?jMtk%1)OF?mH(?t_exK_d9B5C#iWY7&Xs@+TbtTSj!cDohLP8)56%X zfL;H`Q0O~FqaR3PT`GPE@x!BCQnp3jA3r5cFPne+lJr=B5_U0&6&0&YEV=keXs9J> zSu=%Th@~R<>MFugz=35_IS5Cw3$}J!q8KTigmR+O1+d~Y$3tPfXf+KDm*f%hHTC%^ z?E00{0QU^%tInTiP7s&;8|wXK)iSdA$fSOBBep$p9q|HF6uy{P_ndwkxtg`_hXcSl zZYeb%HZps%PAk?Kq2>W%aivydAI}lYx*7Hgh9g{iZ*)f1U&DT3$lv(hD=rzCtzxI) z;G$A%s8jR>_QvHaB0eF$_nBvb>wCZ%|DJnYtz_uK6-MOdjluE|EVD?umKdlby!&5s zRt4JZlovIuEb`IYsF&b@0%^mav>_PCjJ2vKoVf{@YyoVMukQQ#3JVg7x3V*f%I;vC`)B4;KSOp{ zvO}1*kudC#bD$21IE5m(K3Q$EFj3+DgjV$YQ!e~UWEeE^LPGH?%FE{U!J@?-4D^QW zNnDDa!_h%7bP`0=6l5085}1hc)T&q>kkg&pzW`?}j~6w51u`-Um!ot@5JnvzR zC|S)TkqzB#Wtu~&2;RN9tZy)L$x+(a57bObYF0#X3Ya}?WCk-tLwo8<5hcWZOTOyl%vdrAw+^QYWi7(GV|_X(zvf~RTGZb8KpF{?6;g` za7&j{{6G=V@AI>Th>9ighUu#0M9yiIY1f~NdqoP6!Av~R8csN{&tWKL1<wcPVv~*Wu@dJlU{|@1H3R zcifjzYFd5qPgm!dhXRXGaS&WM3mB6Ta~E{6+=@y3zMLgpwpp$L5Vsc%*ksGp%wKCS zo;dQXr$)*tRk{90_U5g*ph4yzr_Um#^AW)WocJ#Apa^NfAI~unZv(v5ytaOiUr1#j zF}Xy^I&I?gOwYgL;u7(F77LvtM)D|mHp;sLl!X$p=AoCo8AOy(>s^SB$<qT5-3t}XVv)#qA0m^QudBS&r$uXWnKN8+UgX`*O5{(y zQ%-gnC$R!jcLhH##_rso`Qly(VJXnZNlnw+Yl*-_NT*V9@xZ=H%Pg!1#)KM0(VjnH zpE}$Y!MDY`5bfviNsgGVz21}Of?g>+gfN`DWQ47f(}dRl62~E@j>j=Z6!pI4;uCklry|l$28Xhx_Falb zp@aquZ|z%Na#)@n;wA4T-uRMXj%%yet%z4?Sq7LX$#~yO4IdCWie;AY$)ui^wxWLUCM2TYOrnE}UySNcdo~r10^T+q0i$Sraxb3QU%JiDI@|h`1 zF6uYjK`(_#wq(>uaH+SIrMTyqTuxO=_1y3e=nWmNVx^(x(*Dt1MH-1O5Ehr%qJV5^ z-lwC1axt@{P_Hq=R$MN54WE~N@QG{uwMWh3nfk`AlZJa^@Nzh}M%CO)8+h8;MUrnr z8ki58{!IfT^ej$O%t{%j+f$4)*6gl1gP9PVo|}>^@0r@QR$HyRLXW?J({TnjIcCBa zRlWZHeT6hdmgBRMAfNV25$byvOEgRTBG|;{E~zEfvoXqBQ=uAo)ZGCOb?c4qziC%j z`CINQysgk>TfMB5%zq1pz0AUt${+}nft40Lao$BxHNtdaogO;fMm$<>-4NtWw0qDZ zff`yy-^^avb`pHye3OctcN!K0uB{;j?^=<`Ld(4CJ9+MbsP&dm4Oo^Pe4Zu_HU#@w zbRzsXpdH#?@65@52I}Hb+Sx5r%TVHJ4X3x0fo)CJ7SAGlHn*0omT6aiCN4==LYSo? zri*(B`Aka8rPT;j-PUKz4{Sl?+^NbJ|6o4D97iL6vcfWpE@Ag7_b*v7&rE*xABh|Z z8X1(jVt4W3Ym+aXt+}io&=_tWJC3%UEI#)xlhiC4NUs@3DhZcf75#Le6NBdhW6Hwl zGd4z8947Dr6%QUm{H@JAi;|nWo0)9QY>o*1O3Hn{vRJLfqjX`rYD%+gaG2ff&%)ag zRMIuUqy@Nj&8wZ~@6HsyjjX^-s-P!?{1;1*t5e0U>6*A!R{XJt?zhzU!B3hQpY3f( z?UWg(NE`7>_w;J=%Ey9#-fs|Ih!Nl#-vh~q2ahx_Nqb@b@Za~F#}sY8$|{*?j4M49 z_zm5=i~1(fSlE2mkTIAg6&zSe(*9%Idg>5slm0wxxsxdppBqAmhB8dN&Wu=5Ufs5R zxpd2Z;W52dmW0cyW=Gd9ouaRqA&i7lSNy*O6(^;>s3-L|NSf4>r+zGbKW(J0MtV<@ zybA~%>EGbcAorKS!05fxNTkZ`NtE8Vkd?z?OD=C(x1{hB+0DEx44SK$i_*a_L=yFV ztiC8)^ra&F{b?`k-1}CbYpS&l9&sPx8`Vm4S+@w`J{kR1?I3&c)57=4@F3UTFXC^q z5EeX|X?mxY*WZqGi(O$6N;OQiSfa5~y{aKrv#_kEV~9^CcQ;ERc{7~EAhY$73Oyhl z0gbq0_{nMM$4c9VfFT@n+~!{yEFYORiF+6D_;lfC?b;9?Gy?2>d;Qa#1oV}=T2Zs4 zxjz{&abvC<5tOjl?gdWOO$tAq*oC0V5T7JI!Gd{`u16QF;wu`OThL!#gkT$<#oJ5n zF%MI8NR;LMbeO%z(-0@xrEA8HIT z`nSh-zoac(2l1QTjw_6Rz3Wmhxb4v6tR`>aX5DvUxlJ3`TmRhN6HawHzC&ZarI}%A zNb8?Rya#8Im`dYTmG-&yT|7#md7f9ZdqKh0bvC_3Uw323<{rl3yN5 zB)XQ_nt-3gq8@C!YFX4Hc(W*ySq0>SmH(uUtaDY&-%PBbiN@x(@={#WHsVWRV~@ZQ zN$J+Qn(Jj>thh(hILcov&^OFq{45M7e+~CZ^^;!&sL1S9#L-!X2K&7^{D#;d2TZ54 zXrIxHbs>+*nh*$NxLdDJy7`Db`QHG9DoYwyVbgm9M|$j$2W*>I+&i;c5PvYbc}D^g zTJ<$PrqxjK$E-#{q?urgT^(rQnS#7!p57H7*EvY4vbT*&H?V-_ z^0n#SAV_TZt&g$&B&)aWvDM>auh;LW$J5)K8VxFfxiPZdG$pmdG#R9OeTu zfk=;V{hB3(W22cmfq!Ccc2WRFbGCkEjH%hXVpr+k#EVn$4LWG5)~uL@;U3+(4Lq=; z<{o-Pttdp96t2PDuNqgwT=xF4{wAXlO_|Z~8j<1-NY=ET+(&N|;DL2|x2(#4oH;odS|4H@ ztFD-TSe(cDO|KK&PGr~}DUwcd?TZ62tnOxt8V0mejt0DtxVv9eO?xuN zK|l2l7_iFJU)_5q_t1`e!~PGn>7SN04Zf>h>)?X?sg28}BesJEVr$)F?gSEEkM2zZ<-S z_3yfK*rE#OtGw4HC#hwRYwElo>accHf^|r2R8J$reyaB_~*} z*2A%)fG_vWMj`XjdXmn-$c4C$iFT>pYr6O+Z}On+&1Kl1H7bWblkHdQIxEzX2jQzZ zO)KF&B<=nY*-<`0gw2?+U3Dt_?Oh4`9IKz&(OOQXFBa6jGNf) z8-G*GY(}6n^YX#;_8OQVzcRT(pDI$6<@=eaO1q?Bt3e?pawBu}`))|Qp{5rfI^7|H zVHCP(*0gGlH}{MWs&OiIMJErsRdZj`05dDA%(Nx-Ao!i^_gS8m34xKdhFJWpfvNl#)EYzGC>+eK|Sw%-`qpI0|$Zq)0zto->49Adp)B+iseK zzHo8it$xr9xhq{~-BY-%v^DGNU|io?_ObtdC&fU@>8wd}zMR|g{0al>zkPJs^SE+aY(eJXycf10i0^h=As zQ=nqG&(|>+dh8!!GvBNUAkE*m$yoC%$mVO8K^0=6iVogdx>p6KmL zJLT6w3Gn&qtG=96*xrzRmQAH{mesBt+-@K%@F=OqK0qOH`YKIV+nT?6#mpvbAR(h4 z4%+mwhPa{%94%d{-f@9A&4&JgA+A@piTa=g|Kj{NAuq9rO32*wo?9@&t%_|=b)Z;N z-4wk$#(DNRQ8P=Z-kKO*g+uSbcgm&VHicOu&1B|8HCmH8)U_KfTv+dK>4V|8bv2eF zgBJIqdpJ?90c_nBlW8vpqkEd@kD=@7DWv1=Rl$Ue$*N%C8ENdZ@{VF%&{yDR<5WyN z9rm_~WdxC?@t)ZLogh+3{~Yf)`|DXLv=_DeOowsK*z?ar3A@iS*`sLJ0-IA)j)M<6 zzung8Fl2n!NHgdg=850$EXRAxaVHg}_G4`ad*PLDBSc~#C2<~& zcB!N05^qi))QkDt&NSlc2EM4$5j@?R#;LcVn*|}QTaqtx_cEF8bLCydO3LxZH4?qI z%yQ!TQy*O%kaj6%yu_w^3xg3Duhy&lhA&$do(!j-@%|*qukN~T{5`<$ilXN6b>6WP zefl}m)s(aM6@Hzt2rf$QxmIzECPzdAMi&@IRTg^OIcJ%90=&P-D;OUM-)*wME`8<` zr&D6He;_qPSw*{=ZU6kfm8*i-!F{d@x4uuRlKacN5KOx-^=8x4W49($?OUH|-v@6% z7!7HaT{G4N(|F_0#n=GZMTRzs^OyyOFA?+PCc<4{97#_k+#s<#Puejq#M_*4gr9y_ zY}b3DGQK}{3J&~ZWLvN4obKl>l6zD7roTqoerwSwBmTM6KkLHcCF|cq7b{OCBXSO< zHoSKvf6sMgFy;tHWgi|xS}ib}y{x}N%}pf@|NQ=XP^HSzmX+MMgVF|A;zfY5j zAqXVWR~~4d^@Yf}eN0Ry|M`!=6bau0DQ8Mcj>vi;RpTO}bs4*K;7qs zhNz8^OSz10F|82n(@w*x3`7cNR7hy?Oz;{`b?+L;%lah1mbZR)P^24@khi+_c3#7# zhE_JF7=oyHL$U+tN#k8~_8wbh0BJQ-4RIax{v*i=3d-)QQ|8(2z?0ItlW)Z* zV?@1YE_dhrR@X0ihj7r(uDwWhHkVw5rd4D^*B4S>D*{QEd&y;ZEJ=Fc3T$8PT^SGW z@yWN6-=z7bv8@sf-^s3)=5rt-WN23+>LXr%$F-C%-5oLarsP`Gb(2;475fXlA(Sv@ z#A5$wey$dg??@mDdZ4EvO_3v1rP`#&oDEU!+y;1*_1ObFA^Bt*U`Vj@0P*vCzI{O1 zDm5KJQ5^XFk@Otxadlei-1AU>TUi9}(CYGrcOmYx2z^n*$E-*sY7-v`xMWMR7%DTx zK48dE!~SR&abgi=VL)uZS}@b!=1*bgtkkzZ9dEPDQ4P=Wx^J(U}qpk+zpsd?j zX6WWNz|@^h+?t2uN@6p}J;E!H9@TsHr?$qM2J*+^KSWngKm^zir|%C)DG2;PWk6Ue z0dV-MmwU@EY1SM~UH7M#B1jFC1nER5{-@+x%2~Zz9Z>irP6&#r*DW&NRqz6FuVg*; z=9hmFr15q_6zKwil3UpY3b%FLrtxn{>7`p#JrUOj2x?5+Rw_=QjUe2g$Xy`l=7EH4 zo1{U49NsW^LQpcdL8sIEXlkM<{`_*-F(Cw%T>vODR+DRi1DB7mCv$7SN?4ep#O7&H z!tO=wDi-`^aMZmp7iU$ZWJSph>FPI+Zi6WyZ8!t>Bw!YAV{}8=6N9zVM*$cskd-2q z87A~fXmx`Q=$7{!-9mSmC_UBQual~^5wsgBKxlk(HtniE@3WGP`AN(!SV$hmzWsp> z(Lvo_4B)l}>PU}Avj<%neT@XIv&1EBlN@y?k8LL7pek=U;p=34yW2b2hOo4=DA?-G zIYmN!AKY02m-wUddrS5a81OfsdF!ziZu_flWzaJ5 zMJ~Dy9(hk!&)t+cX+cV35G&p1k+^gHm;eBr2wc&`Gy&nq>O@g3SmpeFW9E`4>la>| zX~7#8&4pvW5QP)R<@$GBB9d;qCPG*DAhVsJxgPmhE{+NNK>)oLfA-~dEc?NWn>i)s zFBxa-I-tHFPq%9e)Xnt=7sQps**LS9?T4;1f1Z9mr$7&e2}G_d&VD_nc4r<5KkI)> z7D1Q@(V|oN7E3;DaCCKmkfgZ)-ZK-Es1?gFbItkWbD$;0vO2OnNwulj0P6c;cLXd# zSlo$4i;EU%+Ef@4#xwqCsnk=I-?C{*uT z_n80!v4aR$GI(R%LM26Ti+chg`*BkFgt&#EPtQM-4VIEa5K!DJxb@5J?t zV(N%|WiEbOyi;MymITv*NmV0aTqB`L+et`N-#%&CMZF_1oPp?Yb-ekT0fO0vMN5yD0X5A>EB|1s%40mx+< zcZu4)?;nVmw>8LoQnmq`Ab03kLch_`b|H2%(q6s+(yk@Ax9qh%)!}LNtZ%==BEvSi zN8-=;6;<`X&@%cP^p7(vf?wiJWD&FQH^DOWxQvtSz62qMlxt>-2i%sKW#6>NrQ=r)LB8_2($H>i)+YGk&4 zBPQh=##&D@k{u#Il>!9g{tNtr=Yl904P?2?b|P&VO#OMZ7_n?c3HS8s-jw+$P)~n;XA7_veq`Gl#rr zRy%w9k1T-X-x45q>!QesX@g4!qpH9CUt$^F*;9WjDt3VNS?S6OkEKLkQf8}oU&L8% zW@PV|-#4(kRbInPT&YDOwmGI{e!~|@m|#wL;qr~b3gTO>fP67WZr3XMebA~KoFDp9vrQHkw-u&$=|zw`$kTR=kxUXFN@ZX zG}D#%;x^L|g064VzF!JptZ%xW{3q)JowR*gH{1nP=$Exaxa_f?7jC&P-_0F#{}fBJ$yy-R4EgMG`mTDlhe%KpEmPZO>lP1kZJfFFq`|B+Ve{qMPHb= z76&Gu&5<0<6C9+KljI3z#z4_=;0EIl_PHM?=~%{h`rgGnWk&XiQVOZ=p;U4VZ6EJ&`vY^KEVvA(w1 z6V}jQ&0oa|oB=a=V7U&JJ!Nlz3fInq@IKoie^z&KOiEjVmT+rek?fYcWdWHcfuzH= zyw#f0`4N~&Qk;d+0Hq=YgPSDDHVcYggJjs zzCoSeae`Fhh!tmZfu!O&{{55g%WFk3PS*))DLmN6k#H0s2m8y2Bbl&Q7vpSRb{WQ>KjZ(t#?8|fynAG%Q%?>bbHjfrg6EDfvN+lhMf}F8AEvQ_ZI>C%E$@9O- zW4m-jin-Dr<8DVZwu=miGQoKVDvb(w%WrL-nEo+7|3S+$l*)0YJ-^+0YO&q z;=FY8;>Wv9K!Zf|N_1!M!+{%O`np^6IpQZeJSp~cl8uv2kJlTYu-`X(ZTR%y(*na{(tqWq>9%|3(WIhgtY>6VCzV_5^nxkhYAQr5a34~UATv*Y6VuQZZM+a&#A-~&gLF7q<;si!6y!`iMf z&3~;;+cxYV>L)BpWWDzY-rfN?o$K1v=y4(}?sy*vP|UD!vIg(Y(uqnls&*ULR%Dbp$CDl-U=bYs{f2=N6f(FWb3W==Y!5Vw zkk^y+J(uObzlbfNN;_LjF`-Ez)4fNr-a%G#n@N8oOh>_cLb?4mahD$>4j+un)V0v8 zD=s<`WhXxopAa$$_ET(+y{lVKIzdJ+ODaI&xk&O>g-gG-`G@as`FZ49y$yf$l<@1p zS)-82pZDDfS#p`6$P&GCpfPf4bi7^dy9LCvX9J(&l^Jy5Bhap*vvd;Tac=(rl1Vwb*i%j$M#rZ?+zN0f%=ZJQh5m((u1tG%zy3YADFC7 z=t?$r5_=D+9e3W+Qj<;%~oqO6rL3Fl$161 z-r^?PdP-(4DA7hgfaFx+cbtNW-gUxiv~ucEmVB3Le!0doF|>wt25W>2^e25|j&-C> z9%fJP*#kBZU+aQTlArQ2)PE%QDnl)MZ=H_PNo+z^ zvob|$4TQyIm$nc}!R6PSj$hc00kfp9r<)(mVocL9$+F{2`|$Tc?;YoUHo3)b98>H8 zJYNV|`j!YM1{`m6eNDMu$r7hh=igaFCba?Y1IxIGBD zWT=_k<3X(a6*?{ju51NSAlng3D)*!JVr4(+SfgtMH!NvQr>fjc4bSe(XY|5T zs@{mXajd;F2mBd%Fa^)^k6-@!X}_pJw2~L2Y7)jXdTQ=WXDiaAs7QzZ7P9&Pw)(qa z;V|we^b+E(Bo>iWuY{N!9dZacS%5oOn>iP^)#&z0wL~ zD5oj5hss_ZLty3rN|H92^cl$^ko9&7u^J+f`pHD8XTqbEPJY^nd#DwmXYTOXsS$lr zp&xx@nlP$Eqmm%_OUVLKrDH!Ul1?<9?a!KQ_A`onrP42epwdTC%;l;)qb({kq24o- zv34&r1j0eZlVm1d)@xH*{kj?^!Fl?b>N2bG_%F*+_x{a8cm4p$TWzJk?CqzuA#Ybp zk9e6!&hEocJ93)SmNB^PVHb`Q*Od{d`P2>Ua_b%k5GS8}K4N{CN%@l`=rCX5=PX66 zScS-rZ0`rJ1l>;!KWDkPA@LT^jr2mda}6Ns5;X7?iel7lIz)Sy)vrne*1>MI2^E?d zL1E8KgMP-u2Xx8r4cGtJP&l(DC238)m3q?%E5WsKko`lBABwZ8sKw3tmy)IT#SjWhd5JVf22$$4EYHdDGZYU$dznut1i>+bm|>HX&w0dbBXw9YS|ll z{3m=y8d{%Pv8s)47!4Etdx(T|Nx-YRau>0z=FytPwUt>=-?jczOt>eZgj?u9gj~Zx z2l_WjujsJQx{KnVH0Y;4QdE(zuH0HSEx%ouH)6wM&yte5}F4iBj)G z392Hrq^P_X74$ftd*9G_WWB)uDppkr@?DKe%ldncy{a>-7Y-HyMg_Qm_Il>B;dO~X z(5`D)>UHs#d_)EzE|!pe+B=?^n26*MeY@!;Ne)e}sqUA9ZpUFykR z8mWa%*=?CGMc&zwY?Td534%2VT&Eq*tj00)UMmk|4`STjfC^7nCV85rE@+eovk3VO zug@X^5M{lj+V>;af-<7`tZ^ZFJ6$IWBJKRAZCc`;Hs2^&9`m~|FdK6GzJ8?)wF?Fh z1ykwFD2WRfTu(O`o<-XjCQu4YT@5mOSY$`AY$ zUx@(en@D}iI7(h04~?buYLz`Jt>`t|YD}h;Anh_I&*lx=ja?Z!r6%7~k^3duZVy@z z8ldmXpQR@j3jJaEFT5C*C!Iw&mmSL*lyfedh^tf3vtA~>&I+wU1uK)#_S-DPzfBYH zffWejUV%|Gx(MVeF=13-7x800NN_?)K*c|$9-z{0L-xz-hm9h}yV-6_I>4nGzIVsF z^?CU$e;Y^r5oh7y1DTMG_h&C9rXRot)HVH|L0GJpOO+G2vc&}%v>~rEw@{q2N}N6z zXwNtukTV+25O}mV{RUkqF7xiQ*cT1LV+H1lJWE@$E?oAdI5goJ59&oYk zsjosqW?p+{*)s~23;mF*cW)M8Gs{Lo#dNiFx<;NY2{DJP9}%M`T2nf}tpars{Q_Im z56eO!xD84OuCUeH^5$WZtib#0N=+)K#yc=|Ndsf-y2Zew_>ke_02RrqdSmF`>rB~u zc^0H{%xRR0_!7z%@p#ARuk=UC)oeZ|o3=#TXMYqx5_FglI_9C$V5fFv?~94}f6_@C zzg+0iBM*QEDBzb;J-+9`$)Cl#F8jowe+^N3yuS@#xRKW>Hw2Cs@xxrFaP0Z&Sqdnpeb5G`76@EL>8`NnwRETGJ@)-Vzn=Y6t6hN#k@&8hA2{ zN5g#LUDG7DZI%jz4)@Q(U|&D9j;ei1DE`aEyO4%`lZM@nr8l%<=QmAIS^vq8A&pHd zi7LjiWU9>~AA?$pfz|efaEr!5yfjQ>cAkb)#+20v#9&CLVA-na(TW^{uAiG*py0v_ zrLnDSo?BUhJ6UD#ExX$Pq~59K9yzKsa+%5V=s0qbR@WCQeiB6G{Xh5Z22_8qmQGo> zb^f`A1MGWY5Ugkq$!TwOPwTYK9RX3@4Kb3uSZr2O1{N|U#1n6I3)^WfZYSM^u}_Z< z$BJTshvI#Bn4E@INF^cJ9F4B?{fi7zX?<3YsHp2GBGy^fgnds5l2WRUvqYQJ*&{?T znFYeCRK++l&Gly*M6+y3*nHFMjy5p4&w|mgTd((4c46b*rjC(!)%3F!^jsgzXo?MI zU55vzwm~M_N2A|;LBF)6{$%d<;&~j~y6S?|PH!C!`f5QI<*@nS!#9W$)MJ(@SxLd? zM%kpYrYEnIv;XG<&gS76^3|_su&P#?EFvqo3RRN#%&%HOBs3*LptiVe}8Wn_k84+3)W*<~%tr+V? zngAoc2A;MTei4t?gS?r@&XYz6s}@`YkfnpAD~!C>|9eaMDT>KpOY^|WX=Nh~tnN#X zKO|u%RADc$^b1co3ZYAw*W6Ru{9h{8#q5 z9kw&@fh~w^L{d!#46<{Jv?x7K1HUUwIZGX;)qSF`Z_-rO4B%STM@qyhl87$OeCM~j zHaCUV9od~g!v)UTd!8Pk!`$OOTv)O9**3-8m%|Chvp6>Cv)-Q-{UpN{^-q;uX~T*0 zGjp%sGmf11m&m21701Mu%&Kr2TIpz=t1)WPQ&zm7^xEKF@V5x{{xCb?sg-UF9~c>A zwR2&MkmUXw6c4^barO)n&*^?<&dF>YcCZQ^FzCJn={P@G3c{u zxW4(LFmD=5s$;*vM`L!wiY)gm72+nf6KPPgCtP$+P>MkQk6viWHDqlLMa$n z&v1E%gRDYYv4xl&=O;33((ah?4KWvnqz^Txw6B@h^)TFMNgCo7t{{WdymUtY1q86O z!5ATu(kg=D=k6o^+nEHCY2coO&)~lY(+|(qCXm&)3gT^9X80kYfE$`*1#Y zMGkVlQ<~ksPi9ptZ>R&y)o9ImJ52~*pjEc&lqMyG@uPGbHT?qbjRt@2y&!M*fG}t6 z{U>%`f7iz(Kyk$BCvSgy!L+^^GvwAZLOG^XK(rkx7H&WiFhVNN{lez$jkA zNNd1o*ndU5s_u_2+LGm7BOS);7>B_p7TuqISx$}Pd-HTz@yu>gdBfz<+g(7z0$Jdh zT}!%#b){sQkDGEHLxEsCU}cF%k*5_JWm5~>XSmW{98qf}e*M&-!fgznZ{;%023COh zzq4F2rW!zUmqx*{m5m4LUWnd0HjE;06ZLqut2$X!IZ*U0{UD1g)q7PaJ<;f5k2=PJ}XYX4_-gj2O z=U$V5@p8a3L^kDFFZJb65Xga|^qfn4K@iaq*?MPz4t=)%JfdD_1Cq-eobaOl%>0dx zY%IqVvRYm0L`4SRTe%Wt0O|{@iNe$1E;-0zpzqABVG@9E_C;UI0OOZbpa4a?+{SQ; zq2IAw>I5Ix;R?W%Hm=(?AkQP9uMB(yv?G}x4Mq7>43*~ux4Hm7oor({D)BAjt)2S9 z$-g&I&#!TrUkx;R=Jd0<<5>q!pe7T!v4t-eoTl&K&R`kd*(1|}34hVyV^bMfRwW`{ z0~#+cw(y0DTTl4id$!o=fb0(6h3QcV@6UIQeO3w%dY5i}P;oqwy;S7i2C6bYln1MV zBnF_JLZZnt^F(4+7u||nR9B0=Y3CCG_3JB*K6BD$j9#`$7eMR}{aTU?AnQpDUIE2Bp;Y?hJK>+SW} z9`N-Z>$QLS$K zN)0sRo+^L+m2{o!C(umJ^G8uJqksx)3c)~E8VwRfcG(l5tNEvAJ3!VMJTqwRz2_@9 zlFJuV^SW9cQA(WV?HX2yJnXqcxqTl2TwR)(Yr2>*yS``1FGPQU8MSjxA}19+Ow`3( zf%(aZ9&T&-t-i!8Udpz_PwXs!P=Q#B9Py54NfdYlw;yP*pz4p5#&u8P13H9rLE z4avARb^wHYWlIfo_Rjg~Km*c zzj?L-I<0%FXWA6M!3%oH><7pjeDP&|0IROmk5`6;XF0p^`ZfiS5p&(R0+&jR=T9w| zRo!e;GlTPapNXnK-}#+A=tH5AU1ONT`gxZN*LRHfm5b z_-P=|Y~#=A8-rVl5L9Jmw~%sX_7ZiF>r?Cv%S; zgI><$=nCSE%QPT5Bc`iMya?HgFVn!IysbTqj>m85sW@xo}wDPaED*7+IWr$8Mz-e1Hwn3WUGa^BUt ziC2#P;TiP-uv^`>c7sPNX*cIUG8ZOJ%@Oo*YR#aJn8O>yN z)gx`X-kW^RiG9M%@hwXpE=TFtr|Ctr(%LO;*Y`?dRL27P`7+?FGJqzXuJ6Yz2zm;g z`onZaR>$JGRoedx%cOAHExS{*G({{q3-*#4$^si0G7_l#4PAX&<(?l1b3U?c2 zF%>l!zRH*&@k`-f#cn4Eo9eNwwuj&D;0CIIb8eo*|*RJ5$e*!MT zD}em2;@rqwSJVE4Z>Z4`NP19O#WJq@0Ohv>_!@7KS#vZ%?4~*X+;s?*61JSuPQ^F; zmg6B#4N=RhLQ9#=Er8r#+ZxG{b(wYt+%@o-jgC@QEeds4e0Jq@DQlQwBnI!TSuLE~ zvJgvVmU2^Iez_3T1pN1QVS)JiLm@2|@j*z}a61I(ovYlwFc42!70yX)z{6Vj8_7oe z%!A$U2=K*uat8)S?Uv#}Z2N%{>D~hX=B$!9rsZ+Hl<9At{@i0`Jm+Z1C1np3(@p&2 zdW7F|S1+jn^`7bmuiZxm_Cl&l-WeY`EO#p`ioo6b)l)xGdjp0CD-Wmj^4x6xP!_=2 z!D?kWgVz_ua%Ne>W!5AHtu-(4c=hV<=>^yNAN?a7r&|S$6Fruiz5vYIkR*UguKHnR}&vw=l-ZB9-bTO=x@x?3t=Z-H%mzu z?Cb0?KC^!gpZEr*p9?x4pMZRxfMILS9bw;3&C;{W$7LoP~F2^%QF*;uH171yA zT17>{y`L5_)bugW$OnFBz^A^E@Ik)Yhk&OfYh&&veh^zKQ%a(U zy=HaGIK5jL#`6zNYOc)W;#;R3iKGGpJBnXe%YhHmtNuv~s=5We+TITqfUA$+0>9n1#5>id zhAHYcT*ozj9#no@6vw-=><|*G1iOSfk&US?4)_*Q4N)em0Z-$&^0y}ZnhjnjzaepQ zVM7(xFvRO*(bH9d@_Jg$6t}~#ElUl=Va#<$TMK0zyLo|JOjE%g2P19Eof7g(dACgH zb|1YKBUO(2RnsDOum&kPhJ~_7b~7_)kfn|&c9K?o>qn|8(y3$!22;Uhy~K=|x)qgo z>L9$PSFuVi8+RIUBt}?o#C!`U^tIMiQuQpC(eHCCm61k)Ff6Q5q-Zvj&+F%}7fyds zqh1S%^+v{dYybOTU2HRs3@z?NbETD_p2z?z6o;^~qj-AwkCe0LJ1}$1{@<-Q{H`Z{`}4{rF|~~c;mVH>8mq$lynH5&38KwuB|I-Z*_C=Cdu2&l*T9R(3 zoct0{n384J&~Uz6{StFIux&Kt1wpLu6!{!Uk!2Z8eiX2aOp@wh2kGR%;2&;_6U@cy$_sLwky8wP!!Q9!r;w_*@jUontDp z+U%9>w*+PvEy^wAGEab!4gpS-2npI(4iClM zt+vdi%Iz$Nwica;RvaB&w>78Zm(88NN<I&$jAq^uYqwms5d3JoSu>4{%#Sxe9Gte>Jv&O_KHh0?DC3VyMq~a-Y z+`fvN7IXd6uz}bcbyp}aqi_3LCsD^Yx~0q;&-o%^-&A>+f~;wcOR4H=KNsM$bc*<- zWhN?Z^p^bT8sf(n+dtp5N0iY|zi%eH7Q7v`-%kp9Lz#9X*et`?-zt6nn*BVj0HEFD9i)2xqB}gaptGwst zZO44<>im>-mEwkY5Zai3)#)o!L(fjO*FdB*)XOt$^IH27 z=^*JcQWj{Q|2mL_)>pBQ>jDdMMREKWbK}~6F(Y>=jd9SN`^*8NEt}X=?Itr?SLml- z&UN34Oy^%I$>U<2EH$HAJKZxUu4{3Su(5~DTlM3^m8#^`fOhlqTh!;dI!7yopKLz{ zJsU2rb%uj^{)qqS9oT!%V!ajTn%U?FZHVjwU+kv!VzU&b?&mXhHIKw{=*mv^mb%^V zjJ>}yHim=>W*a%rRH|u#c(`)Indo1Z5t^CsN z^orSA*2!}In;T$#vi8-)ws)Xb`5l{24$mEZnyN(83&&Pd>(RvH>~+2Y$yDk#eAF}G zpR`nz@YA4SHNE`-;uviotDFr{R_NF&&1<;2hf?(^c<3>DmH$-)j^mrO)60`;yl_a1 z`E>zFhL3#OS>9e!5pz{-lugP*WQN%?{ zZE6O|tlJ;E)`%JdE~31aImYqli=xamve3%Y;b31>fepMnrsuczCgosK;{eo0XK%-* z{ujqeteej(%T%T8(0@>oISu1vekTT_(JwMfM6<6}FM9f>W-u&qs^b;=r+*ynt}0AZ zb(2axecG+<8mT|k0nWG*(YY2-vfNz(Gx!3p_myVXl$VOP-PQJW<>}|Ic8xczq*n(n zro2#~ePS@&hWU1YhkRcY(o4R^l&uydL*X{@ROkI$g z_RVIi8H#@p+gCW@@>TrjdoNwGYyBJ(*fYKI0J!!_)N(F$fv;~HD-4##FRZ_)<-0tX z`cUIe`&tflO(}aaUwKN?%UL*PvvWRm^3vJd`D(3p>IZmsqU_;1#o9jxpS#RH`^pFW zbMUaYMCK_q%L6&~+@d88|Ly4PlN zSAB?r)g{9x6ab)^c{8V@uZB1frS=YJ>ArQ*vB>9h2jxHdWKx-^RHewSurZYAvsPP_jSb7v=@q(W+N%u%|h8>h)XwV4uN(4sdSiZR{^R1BPrq1&aH zh^Q6zrt7>nu}XlK>GQ)c$8!P#2yqrA z{xcgr45QA5qUQ|=%#3wjV~09xXYU~aq?k?8dWzk>x7nw(gL>{6BBFw5A{(B=JnXx90XuPh$<%XXG_FZk}E=B-y=&Eu{gqj0#UMJweoOq=`}S! z^o3G2GGdZKPLDO>V#utJ8E^x9j-6+8I@ao-h-VXCsE$W`Frtd{+!mrUsFOc)tW>P# z<%+aeZ(OA^B2q33!R1|Nb(j+H?B)FLEC9@KL>l%3Xn>3UbS7)J+S#WV0Aj?tj2M!h zXX0{?A)IpC?~|6DLw4N-ih!5EMVg>EDL7OgS=!NodL-Suj1P zYgtPz-`$E?@EDY#Z+iRZ0NKd-li?Dc#|#TWtWGx)tr;P~T1Ks0EFSI(IXmc_6x7)| zY8CKNv}%3g!|ZSynS(39@gQ^Lvm10qe-bJL&{?&Ji+j6MMYvYRR1f`@qcT5_e7iW; zyx~SspJ+-R9U{Fw^IRn|zJeB6N#PrEt>pI-o6i`iQy}#rC%l=-$^}@(`aj8}migfI zkdq-^Zj3Ag?sK&gVj+0!k=9?Q$NaMX^Ai-Q+%dJa%t9PFzkO1A7?s3(VQ5D}nbcR7=I| z#o9z-E-VDiOrkD#CgD^3K{;gQwrv4gyXxDd9Fdo8VOMmeDAVT@?UXC9%S+ zcb{B9uI?2i^!HRvZ>(69HEeB=hWB~iKX5vlC0jTk=*&3X`}sF#FyDiIOp1OMj^+EI zl;7d!OLRo~iQm=pr-?1u(?Ryv4c2CPQm29}QT+n2Oz!h;)7G>KD+qDiXfd6iXp%%z$STud`xbq_EaujRJbLW2iPnaS_QR441Guf zrvXUWSU=qqpKKW*o}=h*mqS$ndHzr;NFz-iK~i{yf@}~YE%DLjX*U=XAzi80lSb^4%2Mat00)_ERqv&u!V3+|34jUNjtK7 zD4N#TgsSBK+o{0XG{okt-2e|f%8nWujqiA9#5(tz-e1R;fY&flzZMee(GFVBCsf9aE$}{$z3#jfCL0xzbi7gvVrqO7TakL7 z0rGB4ky0XS`?-^=xE1$=5A9pfES1B**3xjmI4VN?UMj*&X>$rJR0^3sv51S@Z=qkw z{mIziG&c(CROEnIC$z>{+8;dOD)aKFpqhwM?=rbD$plHbqVDP)qOa$<{XFBi@{ift zb)4ZsNgJh2{np>QTeHdQx7K9m)f=^v1M<~Sfyd8uKm(Y-gYCwO`OgVoIjmdOWcPwZ ze3wTmJV5TvT)d`GjqHNc&J)#4gY56&vWX%^y(D%gc~Mzhg~Erj7}7dxtg6~W22xhok|gbW z*Ot@>w@7q)K(Mkzd=$@dtFWhTqR)gN^ zfNjg0F1Jy}DA_87;R-64Yh196`FMw|RczIr90QY;5v}UcA5M6S#!>4nmsYV{xQVC1 zqey1rX2)la#{>H333}JJ`(XPK=Uzr#$0&{N#-D=10#pliV-Nq3&6Yk`YfYt!$~*(4 zZa6dZ9<$f=@*SjJuA{GQr~lE1MENP}Y_(4c)}<=zv11f<7{?lJ=DJ(bFPU7rzHW7w zyQ7xj(lfC!vhmdE9{h5M<$mu76)JZQDx(gyuz!uhmZobZm_|=@cgQ=e4Ys^byVGWSTrs7 z=ZvK(LAcVC#)!$%JB1^H()1;5Ytx%oqhn1l56l7Z4Al+H zCrXbMtZ~ubEgSDESgwur6NVkEbPu=bkV=JLrH&5Ys=LG*%XS!?@<5TNhzR~62iqm` z?j?}9#AXV~JT&40s_HhvDV3nQ77dHxT7i`d-dK9gWa-sCZTlANx`!$gJm}Bcg7I8Y zOv&BqQ`DoQwiILgq}NgX8b%gzfxB@^us}x-crC4Rkip6iYenq!IFq{LiQm^Qgc9+f ziu&LH-_WM!VJ>`gKfRrUY}S|AVI2-k^eV2lhO0I-r>Ds?5;_`vGUPlalXQH6wu_>k z1F8uUWZV}RXon)y4p3N1isNU*cD&vV!C8qaj6s0%i zplf&Py2)f=2Z8UR3;#YF&H3nH27y_JyX|h&z@d)ROsCKv#BS0Zq{3!`+;DPNvwCC! zwpR#uPdO?NVf`9#`jWw)Az&JTh3=Vp9d;GquniM^bX?{}vS2&Q@)znme8M8EbsD3* zcJAS zIT?E+-E5gW>15~@C`4i-H#3O_6ICj{Hjmd-w*VU3`J5^1#Q8 zCX7#Gx7hcU(D-GcQkRIOy`4vObo8nzK`C3vZmIV_BL77k9Yn3 z#pVpw(_MrnYV|D3pW{#$pnz$G6MQ_#xnUkPPni-_lOo>VXX-D%kmI5qd9%TVM~ngM zC0@2}JURfj6Fx^kphGS1?E^Nc^T%YBUvhQmEPLDeLTr1F~FtY}_)PH2a?EFt)TQ#f7 zj?G-%!57fn+*hyJ0THpfLZC0lsyz4i*Mw(~DJGe(20aZG#nAL z4~a~)3Er(g6>2A&9iU|HJkc2KfAiLiX-R2S5JCU3>!g(UlGf#@A7xj#uQ3`S133~f z_WWEeEbU%bLXY76S_gUUxw)aU#E9;kbW<1G?((xqXn$1;$QD(}go2{tbKCnr{+pwj zHIs^9&f-4L_kpf(XMFRf`|#3}tpy;47akHj_jn`yt;P@I*25x~F1KRG2peJIG{s%0 z-zK1|8YN$Nr*_mDl;I<6MY+bopw7FR0l`F;NlTh?H01)_6E?)Glpo{1osYi<4ep_# zQt`4kt#3M>EQD5TiAfG9`r_6xJ?(f4(~;_HYg&fJIk4lxer(DZ{#)gtfc?boIK^xa za5fcO#>8sbJ3UAXCRKT(fj$&%mIr7n9!QSsf5HjhpiZ~D@r zqC7@2r!0+_kIP@Yh=hK%dN}~Q1}?%$hpR9)vfEm_-SZ>EFX`=s*Ne(3o9x&@I}bXR8E-?osoAm!=_?T>61 ze+*cS0Arh7ZbWDTvV+B25R>a&pyJYqzuqzlQR($JY!aq2onte42J{@kkW9uRUyjRn z35b!c(kO3R?hqrlI;`tIaZKa#XT9+unn*>z;jhOOzty+TlIpi}9h1C;(BS$^5hfdX zE-4qV(!oRZ&dqmf-SUf)#0z~S{L!e7CuBs(p!vgp_Hx7LS z12rCp7p+&vAcE;YT1_wJFl1A8t~)rb_Ov70$4^5$nDx)neO`0Urz9oc%mMNmn%r%M z7w68#s;Aa;C1yG)G@BZin^-QQ0y}C2tpn5EqXB&wOy=yZcFIF?>x}1gDi61r!{lTZ zVrJXP>piJ3D!ziwz{6p1-=V!bdNm4dFdZT?XfoNeF6F_6UZuH_DDl_H>-&3@y=vld z3K?;gJ)AG!oMI{ODAo%Z=)hMbU?PJzcd^hRM2A-|-c&XX^M*@b2= zK8OW+u=Gdlg@Mf?tn$GdKQ6nVLtpa2lEifpFYUt3AjvR;sf*~D2=G$a)-TGh)SpH; zLm|cs02ZT8W{z6?$XG6c=Q-+LJY}amg?(EQMc`KhIXpZpXaB~Li^pCG-^^z>Wy%J*02oY z9n3Z`xTncMdD0lguC4+oQ>WYyxa?_n1 zNepqz)*NDbHFW@U5+|BJKt4r{WD`gBDE zM4G-Ry{IUt^r8tJMGQ!Ziim&&5fG3LA#@Pw0zstLh;&gp3B5xoQbLg$I+8#lEp&u= z@_ygUH|NZm>&!WSa$SOuWbbG1wSM=#K!g3`NIwt{dIK4z8IJkMhKfy_z!H6A3>8D4_ zARjZuA2~nRYq2sRAsNRuY<7Q3r^!?M5E}#~p=i>%R zZ$+XP$;puw=))wTd;O?3#e3~IWLvw}=EBwdc>U(U6)r(()!l`H*d#^rs|9lq?&!=| z{qMU|$m%Uuy{;#rXLxUc=3dq#{B^L%wC&n-xYYvJAl|WX;J^jEG#^zDL~-nv%PbBk z`Kk@#u3#?Y*?E@n$n0k^6V11IAz>9g^W$Q3Pv)R1UJ~W<#x!8`{NQjJJLjt;dRRt* z2iYMU87J$WFTi$OTi?an|Hok&GOK12tPi(fPpbXQK-ts>squ#Y%WL!1)vSxE+XTaf z;=a$dNxFDvRg{+7jBx%<3i4kz{in9{2b060^S=GVf(D;*>+e^mn8+M!$SNXahP&*` z7z)UoK|rS9BV>z>$)Tn_1R_~ur_UDvIlH3^ZK2%7zN>HlgTK5OQzB_w)Dy-4qgj03 z2`27;9R)RH3ssgnC(aI^ap2R1n=dZ0{lD|tE|fML7i&pLgY$Kz)q9Ln^Sw2jmuIfO zuP|qraZHCS@ZE~5G!3x+)o`_U^X_~aUj{oUP+{fWSIU)iU+7eEwJdPFT{WCKXZNOa zz4YKIuc95_6**yVGyMGbIXmf#`I*^Ht-@`p%L~@eFp4&dPfWgn464hWk$&=99MT-(95k5^ zG<(m!Uu3_UCWh2ZzffDmY3Q6a+a=Y1T|TyZqEN=pn8GMgdm|IBG|e!Z%P!IuLZ9PT z3st#C16)5hve;{>d3wt2+K!eu&gV4=s7jTTjjTACnr{4Ir6Pv54-&9 zdEJ1&F9k=Td^EB%;XdWLs=R_xvYo=#JIV>pIgnJ{b8{K-<3Ep-4-*!_nRh(jSl5?_ zZ#;Y-H1MJ`Vmp0BNjzY+P-AyjHIj3&sskhxs_c;i_O&$rMK>INMNZ(|a# zv+BsG-#_;OsI&i22dL*1ATO*iK%MQn4W7DVWXep;hT<|u5Ijr&GpH@!lKeV6#hEB?=XfKy?W~Z22Dlmo1Ls?h) zB(3fq(~t`I84UJwMXo4_eF}_mq=MR<4gYZRh=%=c8swHtNCK=F@BW55t{z+84->SO z@Y;I>a$SV55bwDo%WMo_kqa8NEPc}5oIVAD?&cug0xWpTSIuwcNQTdGy~X~a;z!n{ zovQVZC6yar*gSlDHr4@F+jZ@Ku-e9hK1LHR%Kyb`cd4S@G;<+cYA>dj!u|_Ox{m45 z`CsuP2?wAs*LwRn)TUKv{vvRtIBZRQHkpj^7!eF9SBx&QLp3PD;UT0Oyj(HCU?G(4tnke|Ez7%HA_c%@o z5KstZ4;fJV|4_^4lmF6&+V!DXY`T_=#6-#xFz`k;u%*0 zFf|8try@;qvw^W2DKN>PTD-C|+1-)1K$f~UAzu(22snVV1 zW6K@%&X}UU;;+d`Orz;Z5k0oZpJRWldUaq#wED5$%^SgN&7(|>+J?h%i9k`UWC-}c zYAIFv!2!Z+>mP(yu*3n7Dxq>rch;qAmskMGiPXdF$D+fQ|(G5WuBVzD->;T?XB7gSuNpj!W-E zaTd6S0o$jNADX10-=)#fF`ZTU3Ca|w>$^wjHjZ>lW5=xif=c9F8MZ8iQg_*`I%VRkv8=4O2k z+h&^(JOOZALV*wwC(pHT640PMiK~Y1&EEu|IO?yEAegFBJ9wUk$M=I%*8vm&>^&Jp zHM0XavKP{5ZrinX6>NG-Ht?c8=f}K_G9<9SNZIk{7MQOBng1vuWI0YVnPvql-m+cKMO;vcNwMfAK2b+NTxl*LtAM8!gmYTa$o z>6uA~p8^Ikw`*|`$_oXK368c9gV|~9BV4pzOGqBCUg2;d;QeEXsg)fZwYUP1>{fw( zPJvOJtyWhL61ywJ#JSbS%Ioun_PS-?zO}wADEFHN8!Y3&wchzVcu;)EqGvw}dl7%~ zh1VpdVjK#7iz$=On{@3Guo-Xsqc-MSbmV98UzqmR%Zph$JWr^Y_Ai>bA?Hw-y@5TP zifOAeKxf{4o`ZWTFmIYX0?{pQbl(4Fb=3*#&bts6FdKQnZ~0XYFAV+=%h7X*cW2YW z5Mi+hWU2Mt^$h`%1Tc%cuingSlz9jucQweI8#C;;f@_-yEJ83El9!{aTogi2fox6m zlMEy=Z#Hle#`|Wcf+~ZV>OAW+sk$dOTTTc1b#B$qIqlLTy&fBTvck^LLVT)rpKMk> zyTC$g*?}ayQ~Ttx?-ixPecg=HpP1L8?r93x(gbWg4cSHH#XU9LG=$rAV$GoX?4-!3f*(Bsz|0Q3yg8z@mS2;+9%mynCzWrQb z&olY*=33lXt{Gse0G8~U`0M<*;Z?U-0Iu0KMvZ!4Xt_X9*BX-symlFs5S5F~4xz|c zBGl72^$;k~Jm>8VwVX7d7(oLhIuj>w1IdN}#|qhNx7b$Usmubr85>&Qkm8|W55cBE zoH(&X1GeSSCS5D--wnF_$Whk=B}!zR?;b{~ZQb;{@`slxr}3TNQez$+X0i}9rayZ% zeiZs(bb|WiQ~~~0AZnN!#KY#m2{5_Cp~46MNT{=)=zF>!2s>>sik1-KOSSdA;I^M% z9MEy9;5~uN;ULf@{ZHmTDI?_?tef*trIv(zp1NcbBFY4qij>7ZX~nfMA8RG9hhydO ztj3cWAS9UQ*7SwBbD!6yR{UPO%v+i7ZVl9j3Ht9hflm#_6j)y<%`CXrN7bv;ZfF^| zE-^L%7bP>BI83kT_I(gENrH$k)eb`;Ko2>6DOyVA`w?TJ6d-&y#hM!iG}<%ho;l$a z04!oC=G1Zg7ZLNhlDb3~3g{Lta|gQyyia}8bPKJ(Q;XK7A>=DNTxl0L8QlU()`u@5 z@RwYvHRA88qtiVKQ8!B;gpXS}XgeZTZV?{7JW367d#ZWb{5qPMi; zACvq;#-WyO>rEnFv94_Or~E4Uxra%iHmQXJ2LEZO<%;$#c%ako&HxX0%wOQIGZ%ug z>A8N6voG4M(06DAEiXJj>+s_AaosF$W>NjH_@_mv&*3>|V9j~Z77IhO`#;{EFP9@E zEhi}X$pBAhxmX%Dpyk!V4|O?q0@^7F!u9HAUI2q{^I5erVvO?dJ1I)OsB2>fe#q;^ zsGj6WHP!T(84Htu%|_jPIms=`M%|367U+5VRjMV@DOVq@PpURpcK(wh`+L8g3J6b7 z;Zp=h?UBsV3*vA>+^#xSIpa*75_X{sfGF9)MvlflXGmxw`yxnH+&v$T3$1 zd$t%BHIZ9H1RB_jB4&WtKs(M8D4=}!|Fbj6xUgo%%bVNv2IlZ zFqfiaF$o4sx~bu_tE*J9i9~H>k_<|`4bmtWb>6#yL=c}tfL8!e8>tpoxi|H(k>Wb- zg6(4kx;6Ko!gr_Alu#;VcFSd+HxdzuH z%``xj%ss=gp;Wd)@KrGtt)_0RT1;YJf&M-B0%s;(%Jnl_)4TsWT@iIM6Sz5TBVlvq4t_)n^_|f2g7l4rU{eJ#&iShXffIUsey&_jY&&*`R zuVw;-#{j_61VoiTD$1B$mhiCckM$v+!hvC#Bm~g0BvBQ;kw-jH(o0PzYfY1$7C+&E z!Q3&|2gei2s`6^~+JTgRj4}cNfT(VWP}%gX4%y;Crapj}#O7fyLHMNWmTpQ;z~_`G zo6!R;O{@ugWT=^AlF55P4PAByP8cV#DO zn?@y=e{{5vA+kMliiBsg8M}r=!WA=pPS>yd+~7><{9R;9e%FB3S8?djUT8l1`lkfR zm@_3-zm0$nXO+D7#L4d>Udf`L%j1cT%Po0N+(&0q>(49><|E43?y60 ziOtfd)Opso2KK)|(}tDi!N90Ul{!Evfal;x9ggIuwo#}y?cH#Q(i;RY9~28#O{f_c&!V|yvlVkf*!T93K;VJ0mL1vOy2K)BzV+Axr!|f(8Umm-K;D4= zx|r;!?xwR3jg3$>;@cU{+^sydj2xFZ7W*2g!u87LUAWn#PRE$ffvG-WsR%*XihbLX znei;G$N2zfE2&lniu$ z`ip5&fLcNMg{5YRp}qsj=nvByB23Tm=?2DBoe#X?!JEN`Q5H#sa$b{L5DfEehRN~1 zk2Ts!`#{93-6?7;lkm$hl#s_?V@53+$)vTgw_`o9y=K(}y$>Km8<{Enz!AYcV<-(4{y z6iiaP#1m9$Ir@yVQ>^pIQ%E0G41o>_g6vo;45y^>lhEFen{NjICtcY(VBCVDPaE%y zR9%lj@ld)~nh8YEk3s^>-D4ywDQwPrR_A1C?VCB>z|Yi}x7UrPaW%#KlJ zTPz{5CjUx|S_dPhh`X*zuP{Ft3}a@`8G3bFzv{XU@eS9TeK57sypZC{!M9HlZ@!a} zZQdyoIlF|HsFmljOkef+c+u1XbVxl5IawugbTyR1h72Wp`lAOODw$~wOXh+Dsa1V- zq4Woa+aW4mq9t<=0P&H!3AKzO*GrXH{a@WwNzUPhThX|iPC%aXX21fzvi*nv=Ag0@ zJRX34Xs|>62Y|YEXSLi~A#Us4w{M>RP!yFUlaDhhNo9F5i}$xw{rUAG5)|(bw>2r!73W+ z_k#qFUn2m-&3YFYlz>1~q~X64DZqo=7XZ8c9CoQ>HkqV20PUx0F3@RCoU>m30cSN5 z^Jfto@;iK#E122NOp>6hfMnZQX z60UuitmDvHy$fj6^OWx@00|v@XYSH9(Zif35BKkK#C&JZ8zo>(CAgoQ&jU=;S|{u3 z2eT26=G%3 zQM0s`3NB(-qt_#Pc44nA{+tGwbrip101Zx>j<(qJg`$oP?u^&MCkSi0kLU7czDBxv zz1WM5g$+PRx|i0low1&N5GP=k(VWH}9xy8E8^rk=cA%CdAR+gs2wA3O^8m7*(?a8- z*unP*A~gu^+3gf`8M` z_(pE*4cKU57@!Jh`oTNM9p`4j<2eRQV|=#gq?pe~#YXkt9f0HP*KSwY`9lDcojUxO zWSbC)59Z(;qIB2ifARag156JoZ}9BR%8$%eUWGMMK}bTKxSg3|!={GyNF)_g=0q*8J4%axfhvK(eODzgp~Z82)UKP6@&8Y$Y%G z86?XolC(O6ACG5QbtwWP? zVJr5{uQyM4H)GHoRUGFIFHE@&P9Pn|d(SS(<;&~(d|VFyyiK8wYo99PHUn*r&{)pj zqEODqQ1eL+VJN?b=_0&|oQs^8<_Q4cH}HG)e0cp$`33wajIL79{re9l%vUqAVB&;= z_-J6qZaI4WRATTKvEe_j;9uy#tbG4K82D^~958J%=P6hmafQs~@{^yYRI$5k{*0S1 zSZpFNX&rDl9FeGr3ZKA%OQFDM7H+%Q0G<7i6NJcYI4Ry?m&rb^hS?VZ* z5z5Na!H17BakNy0(_N8XZkxJV(qivz#5>Yw6<`>QEyQgwV4lg$2DwA^q3lwc`|s5A z`)|3jrft(T0`u1IsZgysQ(lgZ~v}*&N)hXV3IcRkIvy50G}gR z1^BJG&6woTuQ~Nl4wP;!-R8e`n9G z4p$lceOzZ`Dl-1X&O`i;pZ(*t!+# zF2DE?OU&cp>kQ2?rjPqY^S6Lwyd7^fm=U$#E;9M20X?TvW@=uCOn-r<+DK-c#EpNL*#Nz`DFdbi8UPv-(L zVX#*JEgv!6-@y2!VLppUc2+Ly;WIx2JCgWV6WxCDnKRKMhpFzZ-|wwl%*T#Csk7QD zTG;S8xKwtv(hJJ01GmVk^8Fshe4v@_W8`E0S-pSLO(}esko%}^1?s!Np5`D1CZS!n z!2Rj&3a$k4d*huFKRTj?I7n}#TWWr{i-Js?&rJNn-LmqCKmQ}L9-_SZlw6HfoY_R~ zcs~^+F^&}R`$h6FvE8fCbmO~$O&gK-6T3b>sCSLm)qi3sTnK4%>heWLO@6myG??vp zFwE}^#PqMpCmY$xl-;5qG>Quxkp4l%UlyEZ%H@R?Ry_@NW5vH;7m5PQg@z-Qqm#pT z1D0&%1a|0*tmgAu=N<3H6g(MN;AlS#?YLR`exqDsqb~qj)1_}VnTGxl=C472!qKw& zi2v*|?A$6RbM&@_ZFk!}eZqsLr}LLn;8VA}_H~v)8kpPO9q7L1ZImAl^Fk&@a9Ac( zrx5eF0&swMwCM`zjXS*Dz~8sKpu1%-37n966f1}BG!BId6aPOMA%{2rN0pdZpCGJ+ za4o2(7pkl0vu2IL+al}n=2tV+y>mrP3*{fBGa@TFDT5Pt8TdpL2VwL};fo!_jSyB8 zc~NfCed(tXFznb)5l?k9qzw~}6&Nl!Yi6kbemQmj?;L~b_y(V79TxEvoSQDxthM;c zrFnDy`EW#Bzx+Z0jijPg*w}-1PQCcMX8iNM6FBt6GVmN>fgC|=gn-K9)9w4Kw#p)p zd+d^?uXp2^YmL93qfS7n+{SU->DLlPb*J(3ufJw{4N469jq>*k!z2H+YztlIQ|u>} zZF`&{EZKf2x4{&zo*%c$Gn}mS?smE2ZhODEI|+N8SyDzkmC!E?oQ*)K>W|SKAt9Gd z#Yn!jxVVp$A6;5T7iE<$2pkj&rLSe`&n+ zYMlCtBD=On9;WUtTp-XUu(B{HH`k0Ms=>=HR=o@#*0bK39|92!%paTP%mmk9vweWm zXxF8f^wbyYZEO;T&hv5qcK>qdIB>WhL(H2@1CCts-C?Lh0Gv>vM2Ik}ZhD6RB#FgT z7G>~9>yHYx)gLAzEh0LH_Yo#n@A4&iIy`);T+KY^2e)Y*Z$Jvpv|a#8O+k-5H69WE z;jDcyhGjxJ)_lfnxuWWU3WT*_Q&+o(lY3hv^Xt!gN{8s+11KdN&pguNZrizf!C{MW zlXo$MUM$Vd-DMx#x(DWWCDaaC!+yn`@#5Rb;+#P0JPiF5^lon|_PpYkbTh?V zN^Jl9a`2IgC!eTdt&O;zSg;r~xCUX+L8b@Jcrlp`fIY_XaFU4736gbeX}*?&l9}vt zf7rs0UnK8o|m}z0bm! zdgy)6u#|(fFIj$rC}?koiSH{U(rw)2bga7%f1{0foqd5sJUbK!u5*qzy_0w{Q2wFt*vQE475h2N2i>Ku!*pHh@1pa`bFllx+G0vr$_P9OKFnFK8MEq|*0c zR?zybwV`<$hC@QDD$p$y@Ja|#vckbE1c_fn-99>QJrou&a0J5A(Rw!BS9UjB*+1P@ zmf@lH5eKi}??_e-jl}3iwomn~TBD5B{A^LH!tRNH{ujbPtW>-HAljFC@Vy3nKP%!N z)6CeA&M#^)j@i1IR&XJwZ1*g_9XhKkNo4sNZKhlLzzgH=U^l^&$zaZ4vYKd#$Q9C=uG06- zSZ>=nzq453^lY?Gkyy`Cr2I)rd?S5;A_-A1PX7kN#7{wr9+0jnod`QoKsDXEsm%Sg zZX}M|_A9qoMKYsFlaK3qrfK%7xFJ&QfeIDs-yOvimZ~%jSQWb|mJs)dq`|llYE7D4 zuauro*Vy+bQWAOOyg;0{)oE;R%?zaEk1Or}xR3x*{KigxKGC!G?Gt#4X1R5i{Z6DD zIh>!SzKF@sS5(*XIp1H=5zp0Vi7T_q`onihrnRx^jU!2iYrJMS2_AimqsHUdl#Aqx zrb9V(gmawyZ9?;pRv)n@e>KSkCJOj{mHu;Jei~mR;uSKP#}SUo|3MiI!055b99-35 zxOZ{yE@|#N*L^{Lp7*HxVKVCcH;-?9L<+Rr6$U7j{MEP5+5n5E!U|Qrr>_@O+CK!z z=DXbnP~#7@LFgyQrD)|lst8j{3?L#9J?>n6&5eJ?;Af?oQYo|-NkV^jEXyu^u=EH6 z^r@q&HjNo99F*#V3x`!w_UCDHJg6utCCuUd;L|JC?c^SXwj!u)9N(k5cpESZIqQ6< zL%xL1;dN5-qL83Ze!v&E8=b(Xqp6YWP1vlX0?hOMcm{j-JUnw;)!*2j&zq3tKogs! z+WojL*UTLa&Q5oOa_}YK7?f|%#ZRR$Y`MM4;5Xb$w`El^KF%2)j=c)yB;~n%BOfcC zU1F{>b+ld-zej?Zc=Fu~2#l@h^TJ~L8?%*2^DkpH0^N2p&w{@#9XE*xEa75@f3m93 z^<=$uQY6A=aa1zA-l)_zZ$^PUTN;5CfK-m3qs^WnuO zNrl5v2^MZ)DvOT=+tp9hs9j>h=grV*6_}^q=#N8M^;+Ewi*aqE{QkZ&3p4J#{il=g z|4z6r6#YWW|LQb6y)`8(FU587SJ8`=%Ycgx9AcP+rlxlOr+m);rpF3`E|2Q2{bq9D z6r2DY`b6frl=q({zRUS-K2Bq_(uC#Wwf0ivbP^jk-sC53lQko>2F|rzTDe8LvP>3R z`CjTqO5u~QL!d`P?q77`oND!v#~mP8zXcE~j=FC{v6C3{V}3x>j6FsUywwnq^Sl|gjs&~Q+~Y)ybG)J0kY~} zhnw}Td<<1TkGlifY;jk+jEJh`j*g5L=sjDuEVa{qLr(tne9w<+UAFU=sv_QeC!o zqo+k|wmrug#B9SE(OGG?mB~LUKHl?VPq~?xc6TGRC6^~N@=+l{ZpSl58y{|3!;B_= zB}GkFl#~>KB`i0CFnbqsfPb%f$+Sbc<*8vK{?E(N#K66YyX2*wE9N&pa74Zar^0d6Eq_Pui=DD>nE;7q_xv0+1 zJaEMDN6hqC<=lw_EW%D9(;&Uy8l!#vRmhx7#H1~w1bj_r#Y^|KLLL23uDW1)M0>Mx zyAwB_gB%(ciK_UvBtFzY$jMrn33u`Y9PJ# z@eBUIoD~P#PFxdJCuRL-IUGpRVXxKU`eqCEa0d7h zm|0pd8=jh(xQI`7F!fN;Bf z7}WH+cfFb!&+$v|(QIvu$g-4~t?Z0?X+i0Esk~CJFLq7eJSdlHWnOff5L>mmQaB8N z8Ynz2cYXIhCdyCFbUJg1k_wR@zsRW7tR!0v%4Bgbft?@lvJJ^! z5Wao&$BgyYPoXGnSp5ULg$rz1;ARmQirUF^Tr{Vqzmia!rNdW*eHg71h&Oy{G%1PM z>oclx;eqXKGs~|<%b_A{Dtik+V9lyWIn(ou!lxBFE6A&3Mve&_PC0dOp4X>alD}m9 z`>L?GHze_sk}o0MH;WxApA#pVsT9k?nbAe_!0K{ss|k;>t;9v+y_QkRT+SYi-+QP_ zEm{bCeK|Z%PYzTmDyczV7&;9;d`m8Z z4>kockzWweaZ6N9`}4G-av;?pXw-bQ9~W7{oquQ3_%i%Vv~RK6Q2UzSkX>?!`m*^4 z7rJ*!aj#L1gt?+dmrQ;Z1wffQnSZlkZVjwsnX2gqeY=bk7d!y6eM+Vcb80<;Dp-pc zY#YrMk%Xr+8{9n5hL-Rh6nbZoD>#)jYlO6Cu7$%!(KdDm?V>T)COos4ph|?-VPlur z)Hvw>_@NHEp{(zCgc|of+90&4?df{oYL-C$)vSo07EK1lw^5djpVZ`leNZp zLJ=AhkEbdM;4!Gf33iwPw#-KcY~GW4_?pxETzkT>%he?Ok{T&}|IV~8U>GubZtGK+ zJ&i4!(RA3*Gr=Y?lkqF=cps-**rTu6N`a}M>R><0(7Q|^Xh~oS+j`!3*K7lr=t-BJM*`0rrgq9P?c?f zarEiK%aHl6SG%O>KO{4IO3U86*sdl-OLw1{S%jlq^*)P0i>lfs9ePL{cZXWLCb!ha z-g?UT%F(LNOp?azpP7dBi4~vzhGDt}L&MShU&d9d9`5ezTdk5viDLN_grK7qh2H;j z3e3`2^ZY!qXSUdY9x2`9cD8ONH1NjD+l#-O$DqXZQyS}fvPrcm!5AWv+m7|pAeTbmQS`-Pq(c+HVbG^a(*`)uM&9eXR7L_hn{+IJF~bVEOj4=pVQIIkWPfn z@Vw+NDYclH6P|Ib?hZCJM&az6tJIM(6}Ta;C!=;;&{0?pJzqoH#k&V#5wjeU#5l#_PRSFg&AQyQN)s88)4NSV_>F&1 z5_b5{H)lBW>LcF_x5SE@{eA|ix+^0{Cgk?5svg^6N+J(NiUVE<4>#R9T1e0+-HVy4 z@PMZE81A5Em5q;vM2MUhP`jV=i5`M_hn}VS^WC4&&U?gvn(R@6zn>z%7kh5Gg+M|Y z#Dt>yp1#vAYehPnZ{Te8zLyePKeJ)YUREE+LCwtvTf3ff9fyRZoj}&|wj8?a4M9ph zZ8|0bZ^w=5GDgWSxTlNKT$2wI!+HYivqLWymoG@#_i2M)xqC&Yc{h{hAA3Ok(#9#N zTm1wMHt*cFr5)RvB#iw!oss03oj0@jH)BwFzFfDMKb=I6=rfpgZGNpOnLXZ*Io;pt z;J2E5%Fga>_3o>3M#WKP&;G{&vy){rf39UKZ+7E27u2xaCg{>hcfQm7(Hm%g4t`0U z_jFkG?=a_yuh>dl24$;>7Fp|gKfYp${HliTgYW)ve!%VBDxulK(_FK|+-2EZj$5#Q zb{U^B>71I**(s`S(5jaD`h^QLzOpQLxO6zu1)PO-x~t42#p(nvu_b2w zU2ma|(A98a3OZ!*`XMOIeMiYh`SZPrZ|gTPrB!@8ngM1!8_6|d)@B$H66VV zqLdvc-aATR_A^GseZKEx`pXP}!4~^M@QNpLzvgmblev3VlIss_!kV^3o674A)k{o( z7g|MHH+;Fnu*QF$)8E`{CEUz!cU0m~USUsNm7=aawNV_Z(z4Jd(TE_(wjnR9#_7!c z#e{?eB9>=*8zy5UJ-Shx6#*1T06bPh(C@tzoQ7cx#7!Zm;W%|yM6t;qvdCh_DF#XBI;z~&FR<6(_@!ZcL!aj%cs%? zqwz?dS>@Uc;*Iq(UOckrM3;aDefcT#B5(A7TV-9)6gxrI1nrpA)4)s*8$;Xe5lo=)!xk;w8e}Yg1~41$)F0b!|1~o_4kJ#G+XZf&ChTyXKp^ZJLYjh zw0&kzqh{t>CF-|M?K&@mS^0J8eRuU|9iKVfpC=##r_Z&xZ~XI_LIoF|ed?uyY z1k-bIwzgem7x$?%C6<{UI16u_T$^yb0pm~*rsv=mKBlzKO7kD=%C#>v2hX;KXXyC* zb`CEE{w0uRgZ*qnS(Y;zbROxv0xt$08!>-!A6XY;ge^i;ZHI`rKWapt>I{aBc9uCl z6Nkxr4n2GMbvs+C)8ycmZ}hO)XK^Q~X+FEh)+)4i_7Y++8*@^~EU@hgxaUmqSqxcd zgCaG&N%x3WpU((^Xl8RdCd&e(?GHY8td1!-}Sa&BD!(uvv_YzOaja8h8p@!o=@HPy50(5GGAHNUr zZWqn8hRY_GP9^#+x;>A{!~enQDyJfT3dEV!I?Sse9ev!DU^pTC$T~^E{iG5PP3x^_ zS#@16ri)@YJQpY6XCA4qIC;`H_GQ&%yYY*~SG#}g^2u(fqv>*9VQWO=yiPSs`($8u zkdQp0aK1qwg3A3-+ZmQh!AIouHqdZXx$QG%MTGJTLaqXQ9q7N5(3dvRF5iD zO@z-2*9*@h=E>#niqcg&Zro3kZ7BuE3ie}8$=7=r>eq2c?cY_`TiXO3%9u=-J0wsM z=Z@yAhs+IU_0P(D+vZ>rKIQY7v$*s=46ho=SGUxe1kp9qfJ1V%NWrx3m!MsmL?nDYM`@@~ zlEY)^*S&A3PFg>fmD>sTra1G&W&&1h1J{X(h_63ZohY)x3mi3*vAei8(+^&nNVB;t zzuW6`-(frHZCfzr-96Xe+ExcXxi~-{(#Hs0A`l9{NUfiHX@esg2J8UJ+Fpuy44kAd z5D(pb2P?aa^(kbZyK&}qx)4L#=>0$DKgO@X9+T!tw+83KXN~aB^{!<1l!?T_w!E}K z>}naLOR99_*6ps0QQcfrf^&=rew9oNg3QZz= zIQQd3{JSp-StjnT&=u%|YTw@@RVF*)B#l`lU3`*%mO^2Stog4-AL&MYWh266*l$7?n85TOy$0b)L`_ipz+<#x%C;=%8p3JMy6kzJawMJG^@o3 z^6kLhOZ2v+-w7UDueNO7oEbg;wa@)&adF4aup=?@YvD)aI3PyH6(kG5Y!tt#rnS{KJyLvMdoxztw zg9Z}%lh+y*WTZ>y$C;ZT#7D)o&N(Jlb^@OTh^p@TzZdKTW!ZAo2gY^13fFsd8gMZ2 zuFV|Te2F7{1;uGIA8($$TM&VK{!jUSUpX?Wb!dM4Y4&r zl#5#`_YFTxs%+;~;kFo$w(kkHg7{~q5- zRol(E`>L4>L_-W;#w$u+Lp)LD_UC6NAFt52bw4qeUR>D-(vLtpOXc1aL@;sEqbh2( zvdiNW#gM2vLs0qNz7@!;IY2zFzIEs{oKq~_jtO)StA+XTF1IQr8!(@on8E572~$Og}plKk@=mk?{>0)!Ir;aR_tFY7k%8P8d_F5SA6 z0$+oko0n(jVd6S;o{?{E3l0x*(~4XGPB}Hn>B>pl#lc$N^JRRHClmH;LiObet_np*GH2w4$FQ7TgO2fq5jBYQTPW4D z-}x7wDvpLuw~Ycv??VQMZXBpTz`J1mkQHoECr79x8`Y(^bogwo!+Z?&jzb#aV|O$@ zl~yr<7Gs>b-z|s?>}z10J2Ws-AtCpfOdLC*$sW3xLN|q(p0-QH)r7UJJAbqH=h+9e zl9~-wyFWV43!{Wy*|zM-sF8>J1K9TLaPN8$Sxr>x#ai@ z=XC}%Dlm$_nGck9&A&Qby<5$5oUU6{^joY1ZxYj>TR_tMb}Af3{Y$IUMX*y6>DbNw zrpX9dt-XbGQ!jR3Tb8$tdGl5;T{u_-GNu|l&OwgKv){JPb=b29#OcKO(&lM3xa}jb z%Elf!SY!iZ{rZez+L1`WmIxDD%wJ`14~emwzq2Rg*^`YB1+T7@W`)nK6IOB7?B@VC z6)K;XTx@iC=R;;kFz0@{2m(3ta6-FW^%Pz_ZrS_hf)lc#D|weg*ood-F|qsQKT7*y zd{#ZkrJBhvn$Ou;6zUu^Ov0pz&t@Oj{niL|1ziiCUs5zwILK%Br@oj%?5cI8AtAjn znWNFBBNn#R;`pw`cw3po_tb~&ryliVv}bPG;myX%&0e#NF%EIqsJr`&&4rmFu|mS5LHt?FP!1a`pKQ$oStC;Z=12zTEyCC5A0~YX0ej_QzUY=$@@#js);?= zW|8AJM@!@T<#80SarFJ!;wst7MtzHkJDp5G6LR1D705)MfOoB>v&8e5l(hv1AWzXn zVqZNu1;k0GD$4H;zN$U0v25&GetX>9EhVC#URuT8IVV$iwMIET;p>BmMG?Fa{#!V= z2Q=@&&Gjfpo$UY1IBIgsV>Z83+yT5y4rz539lXhyvnl z$9p5$6}5$R-5-V7uRH?9Oni(9U2KA2vK0iJ5Q;;(e`6s*4bZI#m#z+>Qw>Qbr6f9@ z0v%0V*V%y6zquv%QaqSDkO_fg5&s2|#K6h^L~l~AZXBIn!=A_!iz6E0i0k5-B<7wZ&bJL7ugl)tzb<&lYdcjX z!r&BO+_9UsC4w49s1GTI1R_fJGlj8f^9v~gxbT`d+}#T} z-Rl*UBW-4z@VEZJzc;C!*MK0}v{in2eM)L2{IZy8eG6O1MeDE_F;rl+_l@u*UwMYQ zgs}Z0$RDQ(vH*SrPYTyddR6PiXv4($y8yY!{qL!i`GQA|NyhI}A8&bVENzTR?&7b5 zmuE{*y=Mnr(^z~^Y0;}{=^GQFSWJ=cG;wz=;!cRVHLGDIi7yLYo*7NLDLa*% z&h7Dc{;9cL)JC>+9^A64gfT1d7RVtRT|m0X`Y zHN&M}OERBXY>fEoDtF@Mpf(YW?zT+g+jU>N#1CGsG-&F$lG)hm+Oe#vj@eJw0!f>= z3buIkyqpyozW%kNL4GTK%Fjw+<>_pEfJ#R=H`3)-u@oG5o9Sr-=t0uCT7Tk=UdibY z+kT@qLqZ}T@Hcx>cBJ+zoXTxZZ0s=v0X@%T@!E%thbiM;O<@sc}E z+d_#ft3Y=_+_34U4yQN2fQn4$?1BbnLAxSE zRD#rAEqM}dmaNP(uyC*`OEdE8o6{;A$i?J@jv^f}VsUI*#p_=0e?M82pYY-+f7{;T zPg`p@Q-1Wvpq0mxN+}`3`IY##__v)7RT|!{I;I<5>=JcWyfU>Ymk>Yw%tGtSbT6+x z3|R^8)86L+Ek;Im=EK=d#(CmYNgTzcQr(foUcsWKo4AE+lj`oH^M;dBxskZx$v1=F36;*nOkCs;zHq;;u4yf z3L)Z#JGdbtp4XZ4o%7fE=kO2L1D+L27+wiy*IeRK7QFDHX zu{TJ<2@)A!%5p8EsBsTJo);OnZl)uY#`6e=f*9^+IL=4I+PAghw8{fLg@R@sk*xYW zY06K!z~Xvp69&MqF<#4p{n?@!xrDw3NhD%g7X%fdjvOhZudI?7y7t%vQyA_L(c{E- zr>znix)v2>iY9EaVW|dO4e4#H5lXCSO((o}wo5;Vqhvp8-{P^HGU2Q^fSjx^^jd$K z#HEIX$ZpBM2PjUMd&=W~h@yT-Be%k)8^eBYjZjQB$lq=uvcR316^p!+;a`PKMyQGl z%nr8YMbEU8P%nJ7DKH&7H`G7m)bD}ZYt6v8U#GoK$`Y;na+Bk{5=n)gLSg=})t4+6 z6?}T4GAWQ-nSIzFe}Mwc5Ohzsf}C#99huX-T*1_P6gGV8yE=pZ{NlD`455=2lR3{~ zn}xWKr1-}SGHWJ*0v&pN4(X53$WP2_$f-*;p}_p>%^c}2sGe3*>$3nl#M@j6noj{r zt97fsXh0dmyd&YCF+mH01a5+uPU&mm_UTL1dvDsUUH{?QGDxgUJ2{es z`r%O={ot>iP!a(^kF5wo_TRef|JAn7ZX+*L_{QX6i3AnBKN`FUvO_*uUzIwzc*69%_3q;%ro%uhQI`=uCMpXv2fhc zbNp)t8V@-JlESKqDEk^hIbg~&b~{vRKilm1x@mGQT>r^Toy}(BmFRNP$0L~VPMctR z+PQt#y#f$6dUnehiYa)*iFlm{pn~jBH?jqc?~3Y4I)i;GEDH_0&ulnj%+HKG6j@sw zf!29IQaj;xzg1akS&U##5DecmPYx_SCAg0;-ucFCKh<2#L%U{sj~VYCnuR>roap~& z)KJyobG8-4GmC4oQ%+$;ouhf^r<&)s5-iLR_IX-$vF_d`K*+62B0eMhv=+i?3gQ8d zv8TCv1By^`=0&p(Z%f|9#tppCiig9~s>hsWae#5yeqaQUz>QpqmAKj1R*drE+5rFu zZ*6u$h#&J&A7^TW7gZa`N>3rQ2>VA_76L5j4 zDlfQgXWD=nb>80tJuncQ+ffYHtNH-gHU6J=jdT=M5Vw%t$>AKcU`=S2V_V?yQ~(n! zY-55{b5^=8srNL6Cc2sRx&Ul{I~uFTJZus)$LTVH1oD?-g--|bXX4zsb;`dsh~NhP z*~@yP30vkBzh47(W~os=m5{wTt`6j@yS{lHchU^O$U_&wf&!YxEVgE>0zIUGtX7df zqdL~qetkV(DyF;1NAm~q77qN2^q!q*!z=_~UbfyxH08Rm9ee+M6ww^B*KduM_i;Hz zw*~GF^yvgtCh?qFr|q~Ztj9ZM_N!Jgu>h5#AcFBCi$4!;(aG-{>e$?OT^Vm1+YyV- z{>-od|23NH>M!%O>c4Brb8T-hzXlY*E_UFQaROLUUH7Np(!ZB{DtngK_wG*}N9%eP zIJ4zqQ!2_PgWRgK`pxJ=As`%RX)P?KxP|gg7ieSyKWiIMQPcKtoTYvB$3 zWIFz58_xbK9Yb{{(YzlWfQc(MxI2TJcJ^Of-cAmV-&&_{xjSe~IjUz6nPwwgdaim&k<$$ib-p^Ac1j`6Z6q zW7!&_1E&7DlguaC72F6O($!H)Or|CD<(63q21YY!LH~Kcov0_jHZ;{Qxbg53`TDlww^0>^k7Pvou*215#HkGPFoocAbG;eIbwRGx#?@m$B zS_q}cMRiXppFG?szo#HiMvx}sw+jv}fHZK6S=GSL0pXO)X=q6tjqSteZHVV^^Y2EVVl(5tDtWadmKJ zLzLBe`y>c6ZQNC6N%5x!S#wXJ?$xh$+=vq9bs0PUHZ#e1hphf}3p+v*Ta&~s6PSvr4Q)xJ~V5r2AK?9k}-y>LB$VRSjuz;I+>i`PE(0`@TmI&Hev`b_cZWQhtF z@3kqr26JXALV%10-pS0M*9F728d{LKbZ1^17c}@QLP`-gjhpm6U^$_*;GcUo`8^Ht zV!EPucim}S7~v&ydlGQvBT`#LbX)AwZ{KVzx}NW}m7<$#t}r1s?Wx1*j!g0flZ^Bs z4Kp7MTRK!ModTQH`ulKl-;0RAf13kjPvdE!3@i-0ZoRcJ0^9%D_mW}c&?_-ls=q_W z&6msmwoy^L*JPg(#BPT*z77=Y#0o(bKF|5Oe0RZO*!y6~YD{yuB^BEQgQFOJH3U-H z9vt*4lW^R~^Ro)Vy0Xh{NXGcqe)EOhR`X#)iWuNu*3tCAFR%@}2-JOB9 zr7Mm-@hEM=LA95$#&J%E=ocoAi&wxu(|>`<6h~nlF$v{&*Zt{fN@GK1V8R;_zelbM z0w^vIG*T5pJR&uhPJM!9N@-H0a{oi_;d7e+?1!D1r__xU&Ys2Gt=@KoQ;zfgUE-ew}B zcrCv*{Ums_`GI!;M!qqTYq~YFp}W-!`~Av2oZs<=l^)+Smv^wSGe2Jb8=uF%?@Uss z4uIH1Y~4yl+1Ti$p)a)eZ$w%wSa4^cM(O&qdK4GK#OCYd&bDeh@hL7D9(DJY8F$Ui zJXBSzkhDm9(U(s5{|wjPvLu`L9H z(u!#NtJYz`68x&6ujCjT>3wZI2S2Rm%@IJ_QVIiy(#uTPN{Tdmy(_VkiMSIixx`i- zC%6<=-tbrzG>Otc62`anXu+q=vZz$^<+-@S4H$`^HQBVHVEC1IAfDm!*wb!L$$wZz zqtQz}at^22$hMOB>@0rtNqnw|84I8s4HJ8W)$I3YNXF+7==ZYiMSoBXM8D72$5yaQ@9(>s@t z=J2F;cJ*3z&OYEPCJk}!{vP1WHwCe6JEpk|?+1JSzi!&u15N!SJUOD-rH(azr+Wk= zJ*nccTGlkJ1l1jZ={KbmPm2x*cXG+o8lS^TKwmLRT^IZKGod+Y~W&N1J;s~hDj6)#b?D6o4JT17^;_u`VovW zyj%~4G<vPBqrBb;t2*mSCZ+T>YMnxYmY@|Y>7wd%!cJT|qNL-cR%o(||{R)*cV3o6|YrW8fKm{tB z?e&=HE9Nul2ECyQEq^L=ha>Z+FMMfyUlywz1`+$L22sNvJ zgIA(wU!>u?xX$_{P~_F{M;qJ9ua0o-fpASzJ+}bhOw?C-tjN@LK%=KxGy;7R$nJ!r zgNZaID6tSWeXX*MhE#hN;u979HS+^!oXTC$T-^4z7I=szzi)0+ra!Th%Fq`*Vx?l>M1d{A!FBu)aq+>xWY_OHZ?v-pyv2rE%K zhOqvLzeZYEBQ;~LGvU?jD%SegiCWGDX(+|HTBooft2W-&DMz&ik*wr^u%Q=$J$ULTrw8FsCdN2M+ZFEYPzzxRMkzhI zKx-)6m^KXOw%J%PY%~MVH=mFL;nRf&-FLiX&j-JKFdkaF9Z-MG*y`|G}+wr{^;CYy+@*~pE1%V(;WHP$Mu}W-vq-4$Xcvq zNtCBxOiQEwOHfbapmzDbJL?%RbM}?InFLJl$=F_5_&Se>OL?3hZcYmR(OQj$`aL@= zA^r;2<^ky3}RjUeCz$GA(G@ShjI+{pX8v>OU#gEuSv8*%DUMWxWrWK zu|!LU84GVje?{ln%y%n!vn4UXS-1Co=s$mHp{3bKk1lVs*m{9(`U&9O1rKAuo{T)H zySFSD2%jGRqlgO=tJUeav7k}SE*=sfbbqsP8bY(#bzAElw9~e$8OJ_eR71Tm**tz? z)19wLC5pQ{?Bna)RGO;N&Z-(C_qA49TBLSFh0@4A6VL+5E2GET3|Wp#F( z`%+Smt$k!c11j|a9;#$w zP4C;MR8HJfHoG7s-J>Ih1;}WW8?(2k9!s4V`7j1B8i!08Oa6ow%6~%(ayD!58)xd3 zI`R$o7Aknu{YX|_8)opW6_4;7NW8)(irEFo0I{hPdcJlvj{;5>!=q)Rng`Bp&M(1D z%3nijjMQ^m;A)OvhsbIsbY#G1<>fQ>KP$V7g2lcxNll0s*P?nkW!Op`3JJO`MeqV%y2_1Nout@xhAOadaYC zQX1quE|u|;1J8!+nN*JDHnpuIcYR|ZFAE!ZlA$Wa5NyN|TtU7*X)IV`oMY!rtztzc zG)#Wz`&gx~02;o83c~zx-c($yRix0HS;gvN+56V^Tq5v*&L`EB!}vl3b(%Lue4&5T zv-iGBMaUEoGXNX7TmxrT;gr`{$!(a~8B$3igJPf4KL>EBX~U)uXI&eh`N%JX1s;5= z(31hnluLw3vv^l);SK!JT`tvM!-B$ntXY)qc74Q~pNbBBaT-`z`>1*s-$ z=F^2+1bw`np7)opXU_V*Qqe1iQ36l(t>vnlRMe|aM}2q5EMoLwPS1ixM{e$I07fEx zzwq>$@(y!xZORy$3!fKHT@prIv1a)&2Y?v>p!?S2#Yb1MbwwaRwtMMZxR>@SkWx0& z0cDApH!X}^R-wHI(#<8+MLK*VoY7u3RM0C13A)q!!9ac&I5JIwa{u6e3(pS+tbn9@ zdJi)5X+zO){t%$APkqw8x5hWZZX_G7CsTAbg8GXgcrI#6S8!()>y;X*INztjB%Z+n z+&*(QS+zFRs!e@8oZaix@Qzr~3F4{d-|7=@R`(PPmyvQnlH)M{4s1!6uiCOJbpOU> z0nk`)zP}6~;={CljtuLI_(hhQ9GE`miX<2aP^&*A+|;>*w5Yv?I4W}0;cFj!{P6mj zBx9O1<$O(9MiB>4^RFIeBUuugdER|0J-YhoSI+B;ku?SSsMlC;we=HV2Owo60g1OO z$!2gCUaA8w$P$PS6@M}8Ky+wi>lxN}|G(Mw`Ikwt|1j+XXG*@yz!ZQn9XF_pgN$r! zjvNqs$h3OJ@t5C zrkV{F6m|`;;)YYsFJUV%kh$B&sOer;0ZlTpI(u&ETjl`OUt{^@CdJOKbp&_mFDDnU zi`n`uz8fUFrW)|YRR`PHwSFvN{NQGo<{0bl3TK^+a9jC~4kiIFgY|kp2=sioR7_;i z47|S}P)r3ATvvHP6u6OF1_u^c%wJ|!txT0-;9^714&sC1@R>r@4GVvo#|*xN=Ep>O ece)yEZH=aV{?RLr`*8>GxpL9@r}7`~{Pth$_f3ib literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_9.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_9.png new file mode 100644 index 0000000000000000000000000000000000000000..3ade55d2732fba2546655dfabb24b6439b714021 GIT binary patch literal 78149 zcmb@u2T)UMxbLkZpn@Ppij*h_$fia%5L#jd1q21Rpag=6ib@Sghae&XQX(CsL`6Wh zQXwVh)_ax%BnbFA; zXHFbCbm-*G8`mrj9Xc{{=+I%%F)m<_S5sX#uwnP?>ebsfuUYnMv^&G^wel7CXh;V0*m}cpO z;2jf>Q+NOTo#ZiQvAPZ|)K&3Y-rC<{j<82IUrJ+1*3FzB;CNF_Ee7I&-Aa|jyYWW# z>~AxF<(!W{{zqMzml0Ds?pM`uu#*3S^yra4y|7*ZUrRh@=3N7=)+TxW%2*4z_)Vkx z$mg?Y_g3U9M`DP)%JphBY7y(2TgOzS*Dd4dQ=@|Z>z|SoF!Fp8x&c`st=YGB9l7x} zxwmhs#l_r?kqH*txe=f9;mTz$^fmClxq11pP#u?8=|iLx*W03}udeW2J$2a=L&p!#?{5-Dn|JME=xrVPo5MCEX{i*4Q4Nm{R}7=Af4JXwW7q+=q_^;+yL68c zwCAwBw7<7B_~Y*09AQr>HE4_*V}D$0mmIW1{^8hHFvBM7vqjOPTO{qA20lF} zU3Ndku5p*Jzqy~YzeCxF6Mv;rOG`_0iZj>0{iVw57C5>YINEcjAul)cy?Pi3Ri(Jjmv?KX!6%BUGhqOWOc@dDqxKN97L_+;3Do7h8f}|{hJ9-)OU)2wBZD5By_taf^;MXWGu} z%rvXPbJN|8-(O9dsxM&_jCg$A9Qft$5=-bU`M5c>J*7=Z?NBMOE$Q;#*_bNZ976Jq zSL^mwbo|AmGNWVopiiZPJ+gD9@`dEX{SNQi_+D{?zWm;yk`2$ufHk;;Uq^l*eu=Q2 z@4C?Ec-0l7gVC`{r6(CLt(ahJ*tOaOVz-C$bacM=;;c@Y*X|%;*X98czbxO+`vn2Bz}gyND#w8z>>#ipF?NVpzgtCYWVT0Ke@m+ z(92)?3Z2PCPcY+8MrUL&fVZ2t0r(0ypET8ls97D zKk%&LRB~YBy%rmyfX%{C+*fT^+PhAp=WRg~vW4@T_ne3by|T674Dipgj2)7gKPn4L zM(dM0^?Ef|C0J3;&%Tk4_Cm&rzQ_{DQr_?uiOAOwF7Vp@7Bw%twClFo%0gttS1*@F zsu+j1OZaaeyrmNZ|4=D97;W)@WjnCB)~5zJQQITexI@+wBoNDAw~|lM!z`iYMTr}K z9k3^b`111l*mJT(?HZ40BT5dW zptWg`l3h`KS^{qGg%&k&n~OOV^5Mtgqj-JjF6r>+n10ZFz(p8Yrz7_Mo_BowCr|Rb zj7tIe=ELVxw{DJD91fxMW`tx?dpp#;HN{-#3f#?|CO&ZYr6Crd7hQN7cw}*c0s*b--d-BncsaQ(6Ldr6EXz4@BMilF;c(0SpgO(Z{Y%2IV6jk^Fz`D6mpc zF^Z(m-^mJ8=^=Uc-dDu#bB~xg1B^d%@DXo1BA->Zxn--NMf&C8e5gUETNmRGTu1eS zJ455R#>}q{(YD!R@2k8Dn~*k^&sxIb(7{9Vh`AMxJ;hyaLmeMj0ryYw_1QO%b49l z^<&OmxdCdf10FA^BUyqf$-pO&xkRJ_+RmA=@}F^QEF ztiBehn;wtJ$8W&P9*~!ESdY`)l2!%CCpvcX_~(TOZ!isOdvL+q(tJ_nl0q!M%|MFg zZz|O)ChLcJbiSrHV8WJFY|F$7$wOWYRgZpu7Xb-&CT7DE(YSjk#zGgf5Ox%pf|;v| z3*1Z|L(uVRHddE>TNl&#D_fy-1h|5cYH;bmiXnW_|IssPH17@4Pw1hq(tJFXwqo=* zv0iq>Yiz-S*Vyy{@o)OZ=pQ9UHs2)pb+tWLFO(>ig~ZFY2qJ!1sy*zP^cu|TUW84r za-1jk8&=L_5l@3P)5RVg&ngIw+-)4t7FgQLQfZNfAJ5`b#VU9#F{qQQ zkDO1YKY(vIj<0oxw1T&rP_5(-Sjl9j%J{@LRup{oLKcGUXlcl+yI!F@ZtR1`?+dDu zN}8<~atRkw+X%~{YM(^nrY-{S zI&n8FiZYg_B**pf-?K^~}mpvQk8Ks;$-p?*2AnVK)_l zivIp*?^7)PZ*R%c$=lDZ4B>4VUZ2J0#UxK&qe!YTxs?U#+o`k~5|$0uHPh6F{*nx%br__lhM`6>n#8@kD< z>K43~?%I*M*}4#GseI8T4f)eZb#V510AOdIHI}{xi#(1tsdBG^knQ5PK3-#c1x^Kk z)tdVkVVw>mxvwMjH&n)d=){CK?}p>C)`NH6o))SIS1Dak=q;5rUNA8nwc2IS;W<$eZzE$^Wx@NMTS1EA)*9EpDf-Zr5pOSaZh--@1?G?Ii;`fzICop*W!VQABi=MdGzJMr*=eAWt z9ru&t+$2|124wJtTdnlX8jLflPeE&4hcBokfQZ?ColD{z0TXL`6gM~jC=$aZ%+4YZ_&y)QHe1qggtrSRt(zaNNon-A+nR6bI`o$xb z`N_II3&m^jxC(VvS{{LU5e`1fPoh!_tFiHt{=tfqVO#z!UQCk_2x;tGX=XM4=eaTy zvRt(Z-NVTQ;!K>>^Ra=BJF^n|#Gbn!&yCh!K2ufxQgJ0XcOFY!!gujK)po6<1FrhAs0334?EdY?>i1#a-+?*bY)9K9 z`i{nwWdh!H;5PG_Y0H@ok2=Sz6i*&g^L`ghbTr&eWubYnt5S(nJ-j0*K^8^h!M)i` z;ZbT`sj;h5$<7%G!Jv76Pm@|l@{RdoQmsOgt+Mj*m`J7dY}%Cp^BGO$I^We~_4z}9 z8{SrRsSuk8Ee|g|Eg|L;Sm@A!O5^0iq@)-(Qi~cI4SBPvKjY$oNT#U}UvcQc+h{x< z^Z=iqh(Kw`BhNczSBs~mbYU))-2+YxMMM>pns*j2fwq2fo4E$D2NhHc)|5`mOPhVV zH`6H25X=?rmg8Mn+n#Tfn#cc+(b#O4_syc5VKq7|Kr~b*rfAY}cHNQU^cmTQP8CzW(xJu#H5*OWHq3-F?A=L`-_Y0Yfkt?bSBb$*)9ldc!l z;CO8zU9PmZ7Gn(}P= z+Dul#YY&UR$EgqPi;DYWFtP+5`4K!T0OMdAi!6a;j$P>}gptFl)Q- zG~PPXip4nMWDEmivT6)5>me_`PPS8rHcP0}1Vc=8zGro+#4>J+sF0I3JpZKIU=fXd zk^Aw5#T0rFmRtkbE-6x}s_i2I6R5NJ{yZqbry!xoaVkGK65>~657AO=Q!K%+x&Pi) z;2t+u1KB=EuYs@rSXR*k2Y+lzJrfH1ceCo;+L)huTcUo<3tEJN8DeiWd9$s-PHp0n zMY`zkWZHP8@xCO-JzGcH49Lc2-AtBnT3H7 zksM`Ho~>euq*&}}12Njy#!_uoiTRqNa>vzD**9BJ90r3HPBG+Nd&GL_`2Chb{DsNE zLFzJyjyMxi(NzZ4<2qluaxXq5w!($>4jow(ei>KRXc!p%wFd!&CM*cG@k=Xs)+3cZ z_Ngh?R-7GqM}aF(71@4%|ySnTM ztZHde(p57RW-!@ku5CQe%FQ0iu~p24JdQBQ2=q`)hKRh8JgS-Qw^Ew8+WE!=-ec3p z)q4x@;zA*JMbWb*{rBEx=c^V*Mo5~ZgnQh{S9QOG^~z+?uF(;9vh_xMM?0nei5wue z#m7|7V$nX_13I0)3FWJm&e`~GFV+9oenkm3?X(2=l@6l#*m)EkVdC^3@r~p+@nx5E zp6C$G4uQU=pQ#2M`rF8;i2%74*CKm=XCK@S%ytRn`liQoD@M-t<-(<-_7 zaZg{ld%82bD~G7HzOd-8L)I7xB!+xmtuK4RiTo!69Mux)te4dxenKlOz z)%vpt)mZ0E&ml_nVVNnEqL*_v#Ltr?lkA+WzbJ8@qQ*2k6ZzB0&Lc*q6NhBYQt5tVV;FwBsbUR3VU_lsm~F;u zVf*)Mn1E$NUUEaJh3A2vJdb+YeJHj~cG+cLs z77_J%ZS1G&xY)}SK{Finlld$dioU_>$&RVo|1wu6ZbR%1xMChfUH^y}j5W#-`>cCD zJAr?V8gWa{IS`ebP@aU$f?YS8Ey2&4gX~qNwc2f>%`xX2!GB|+QZ~^%-*#mTJDRQ= z)Iv=LKgyt_#e^ak?iqJYtSzw5lv$gMWL->f)C{@fKHecS%gZDtawwSCbgfQ38BdM9nZ zL3!L<5Cc*>XIMM-q>i+;h36)nzo#cCwGvF16hy&|q-i z8yq#77x?$a_{|P#EU%`9N~7b&#B(_fuEViVvY>{I;uBUZra3jxcqmND%euw8gIlYw zvAn4H>)!2H^!!9bG%cl_#}5w{r4?CP8Jb2@YMaM_1h2PN3nKq`u}>>5Uz2MANw@C2 z(-WK>V<+V`Nb`5IUze(o!Fk~|CB48+;lLpxnhDsbs%hc=j< z(?b5SZiS$KBJ9fGbmxVgSJ$^bBTgzX3bvFk}=ZRv*$$$cZMCBjqvI3z$S3J>I7iDLcxjuXyR zEIC)xor%Phq}8dk!+skKvj&K73f2g^8r#i-*sA|Q5Zg>F449nZ-(Tj><*}W<(%tTg zVa|A!RSmC=al;^!QS=x|3&;jiYQ#bUhRmC@;_FcA*oCp;VX2t?&7h#|Gy*Ga2B?rS za?7eGRk|I43S$Jq;WNbn#_pfV#mwgBh}KMBS2RRsgP;c59{WUME!SpAVW4wmhi5_j zh91O8xmtYCcFC_#SQh-TY^={KDx(s0#8FxKTXTOFXm@%>0Yg);fR9MZW*-x6=7hz^44ldC>Qmi5G|-g)`Sf$ z15c-qUx!DgM&K9+=tkgARxl^j=l=BQf{V8aOz=9QRBvdUuiJHh!!=j8dMZJ8cdUGL zcTxMGSafs8J)*%ppAV+BVMY9TIVN*=-XWJhHoS!KUgac}w}BZ;pq)C9X<2!rN)9SU z=Sk(j?*c6YRn7C-@+;kQHPUD1*EGQd@WHlM{6)pmOT{}vQy{Kr^ z$sl5{$M6vcOmx}#GCeNJWo9;9_lXXU`rvUiHJk$>iC9Lvs5EjFcqx;7{83-WnD)$hqQPTMP> z{goNS-Xx-{e#OkYvU@GYyIOYl5I1WT4pwk0aw{h4vUN-As2h+E1j&JS$}x&VntN-x zFoWG8$}arD%aF;Ty{_cigvB%6Q`GNG!O+d6+KC-@A8_kuz*yh57j%hHpY82F0(2Mn zD)a-Ni%^nNz_JBpa9hJ`kaWdPoWmK?e_>*IaCFb>Ev{CWF5{)8P9#aeijmC55mghU=P=T1*EPbb65--59kxeb1IpXJ zmDEPnB&g)_x!3%*SihdE&!&Q~n0TB!W9u?^vTs+i)cXDI5DwYJ9POH!7oCxIe|l=e z+{HE6E22Rwq6#C0lwlSbx`9{J-m;$;(diVzInVcWIu=tt%D12EyEPXhUZLFwERc%0 zLT}rj;G;&)dyP+=@*S^My9i+M6ZrK4o(WmQ)zjn?+MH6367%UCnz-~3#{!<^_UhX$ z5?XZY2i_5v{Z39u-s9(QEM+~-uYxG(9`(Dc^r#|7$4;)QH_(ZgW ziGgojcLgO%EiiSr{WcA@vORcI)g6MDFS*P|$?a1WO;R|X>7BxtxIcmVA@y#v79yh2 zYcxp@lTvq4W^6j>N)Tkczp2xIlsz&torTSG0kWE0y%g963YN5pLC^$~Z^%M7D<~O@ zZ@yocGhDZ#Z9Zo&kD&-^+qg~N_ zt4j*;Vq%N3H&4kqur1x@`)lUrg}&{MISu?Ueb!?c!t7s#x;0CIOD5ib)oFBA`g~CH zepl!}JeR{s1(xXMj1q!nJnP?eQ{k4mPrXprmG^qYA5v=ZYAf!^YYIWD3f&ojP`90d z!rccg+}?zWROq}CSb$O8`AvbSgw;Crbi-yiWg$a@Wm~!@$JLz?w5l$j{DND9zi2Ez9-+> z^HB~<-}9PW#SiC}_76bPK&4 zqU2rCYAe6TQ7>y0h1>TE0FE}11g2yE&kV@^85yXr@Ps99`|kGbpGUStG2^0L$k%8A z1)YkaNmcDwjgb&+rrvH}8jN@Of)L?ov_;t1ITO-vu+jS z={H1|)*jv{ExSb?(ZI3Vhz;CFNFCrt*^2?z6CsCCds(pUteG=?<%TVt96&PrCl@W7 z-@DP(x7EhXszFgH@@&hT4Dm8<22j46FP?=@sv(D#L<)y)*o$w`YZgu_j z?V{oNELC%XTnbBQZvXAAKSvRaZWpkscmU%`+R;3Hqn>9zdO3W+q-i-Xb$i!TfNmjL+?eqXR7Ue#->8Y zdaDlsn}66a=|TbXqwhs^ZJ3{wels{XdTmCxd-HkhmS5R0X*!P|+r5w*3@G!Oql5N= z&lnrj-P@v>HFCN#y-NCb}t`=1`)#$EkK2?=(V;(>gvILY1rqpkhEyi^eEfx0eGaRT>P8rFT z?#&kNJ&y#pdowHv2hoEK6D4LtVCoU#Njxr?7f44txa-5LomI8{)!v;HV(&rKC*Lu2~;qK)oI9 zUgwrW0SB4`VQ{*yYQvIZx2E`cY?!u_31KX7{0k?c(cgub^@Oj!mot}lSX44JseH-^ z5$Ce^p;$BMlnH~h<7522-8-!t!rzAFv|-BVjW@G*d|G>}Q%2t7G|cxPBHlm{`(etZ)dVt}C(ba<2$X@K02C#>bIjJsllmxEZp9M-CSEWRD;XvY@@(EOM@m_2g`E2k1q z70K;|{!RPjjT-tHAlF_xHX*ma79wSh0-V;X>Q{-x=_YNy|R{uZt0iR>Q`Sf&bTZbaLQ!=i-V!9I>bzn3Hu0*WU6n-@&f2lu0y;sRiIdza-0-c_Z08w! zw&w<}0Sxeapme~*1aS_QiaCEDaWp0N19T{NMpPR*x|r_0uQaf+5!Z+n07lVvzotJL>!+Ttv9 z?moqsGP@EAeZiFRqN!fs9By~J8bG|hwck9(r&W;(gMSWel?tf6-E8Zd$<62hqs9x?4`hHBeHm42!sl6+ z+~ICv8Ag9y0PT!%D@a?>SZ;6i^-1*vTt9Z}oazZ|_`3$ytBfs(e7R-2tqbYuIE025 zU3_l|F{|61MC?7@EaZVPCJof#Ge&#Gv?6YOOD_R`!EQl1o+4;(%X@2+360*QP9?vl*x=aU zt6Ch&P}`Rs-qqcrJw8sa9R9xlVYQrt$FbE9yirnFE6!(b_O zB%D@^R`&6cd_}$vWGL9rp{FCCWR9-$p!IuyW}OH?3+huER-(YzXSY*|rp;5ACUQcR zBCSO3Q^#uRPZzw)Waq#Fs6Ki;UMP7N`J3gE8oH!QFel|R_iwN2{|O1SR8UoCX1pir zjE0(3oHdiSvflx2d`&Kk&(m*D97Odod|$g&(X{jNbt5Q1`2lR7Maf0u%O?c{SL-#h znL$z(QO)ISe*A6grqy#1Dg9IX>Yr1ri6IYTqhSHd!9@PcM6(QVMNo)sQ z)YYd!qDxlp$2bGbhs;>|BfEClg3P@)mIbqZG0y|NfPXjHAOxcM%!e8!_&f?Vx}tiK zaah{DIqOg5w|Kld1M)Jy9&Pv#+=fzIE?%LYqU=$;v*R)z=pgTy z*)>bCU;7Mio;sz6dWeg@_^*&g&`=i{Ve8QwMw@;fZaKtrQsj>CqV@OX?^`?SJTkQt z6+ZG049D zoIGA42<4pnUs@VIMbsBl1DfAe)hH*8IgCfRs;}jC)MKkkwz$>n`sVF>W6|>Z-M7ve1-u(;2kZAmyF!+^)$regE<{E0Kc|jOxU5~D3e^M+{Tz#w zENWuF6av1SwOal_zZyx2qbGCX*~f04douEf113RtAQbZ}1{{+a6^8h>o`ALsusQ9v z)%i$S{M-GayBB0R4hCC~gPyEZ9OZ$}I!hOMC?Kn4j2Ez?zuk4O);C^&n?4UfKL1X9 z%)23wWp|&MXM4Y)8I*0k{os?7?TG65gn*L#63D}<-ku?2ON-dbu=f5~NA-FAI#fi0 z2QGbb9DGdLcdS6g`&~`L4cPpU?W1!<+Z%i9lp`@!oyr#~=8lc%;fEThVvjC<`2Nlz zc%zEIuIk{GfO8VKwzSoP{~uW2(hCfuB&9x{jE=Vq1Ibx>mXtr z2*ZgET`aQb-Bjnv0sW-9Bded|&5SWVo@aSztoyS+%&7<_bA2Wx6eO~>#5FW8AuPcx>HRc^Z zVj{RKX&G{ip8L*f`;M5}zSIYGEosEVPQEv4uU8wbM3#iz>trDl$^171z)AF^-BvvN zvx0$Qpgjt)a`>&aa?~jO!RP>=+zj>If492nEM3s4J>DaD&9<(!TS3`mDkZ4-N?j!=~FM z*sr?k8~2b40A>>)h@-@{2vv-qmlqLICT(fMcBEUW-p?PV+o}8bALp^MQy`-hhJHU< zsi#HKQ$rVbIJ3sSdXP|(=L+$ z@liess^(7zugw3+^Gtg9ENBbzu(qmw>wZu1PgJa_zOr9~wZy1D#2#10t zAF|$J>aU2P~K)p;6 zlK^pTQNlbj+2LC3XnLGcl+UP+tNl6hjsuq-`6&5VBqk+QeZ`g;J zr^4K`RB`n5psY+kr+B{YO}OPuUuBN_@T=Fi#`cI|EihH$0<)J<0&aYfDK=*1fI@ z;N&J7beJ#J#vwfd5|#IyCQh0vxlUuhU6r&*KgS$-6jAgT0hWF4ZzP9f9J3r%<;|N9h_jL0 zkcyYR$OZy8I!dl^lhg~yVdxLV7wqxB-bM#b2B(m(`yU)9baaI!>DH_bE`Iv3jx>Bz zMApDpxiP3{rilY(_}v%d&TFl`ls&W-!|h@m!GV}I(V|CU+-;@5QYCY1<&tU5?8Rv8 zgX-8S5stkJ*>H3Wnme6eGav53gQ<;8O38x~U zo99ICuLjpna<_fFo1VLM)_u-qRCm|v6{pS2`wL#rfRZZ;RW)~HgM%yzGOQq>hg<+o zQbGc=jA%KG($=c|(MAeddVmKfK@d|quRWqYg8}v5gI>S%N5T{b(jHfe6zZ$oC|xyi zB3>@odfCQ3pIsfxzuckW+u@|~=AP5R)yGK~PTb1Cd~(XKBD%zlW_!G`_8Lz((Rk=Z zLaUwRF7^0)wMB5)NnWV3hjYSyeODiLS$(f8vCYirkji70WY-W{b>TmInb2Ncq96OT8d3K zy|VT(UoD`UI|KbP5`z~|^3s}L`xuj%ul>Xa^-&EkMj&(RZmqQYC~p1K4*aDS4XRG> z8Rt;r)}5O=sy`&W6>)lW8YQ8pg?f0nA?zkIq;MLm*Sw}}bYnhNdn1?}XgYi)UD&Co zKdK)}MGf_=%ZZ|dm&zr1d|nXSm~*Qivk=mvd+TAgK|NBgJ!CL>=#`Feq>h4G$(78c z6BW5mqTKuv;5tAmRoo;YB3;|5kj&_t{cqbnK8cJz@NM8{tCPwNod(<0SPFcrr%~S; zr@&(vn-&`-EjF#{Zq&(VBR#*DryT?E{P?>5KTP06_e(()aW|8$^VwvI4e}-g-C4P= z@FJnyqY;P(l&Iz6(g1==TwuTC~&lU;~2+t>onOjK81nHZJ z^YO@%7O1vEzn)wcHql2=9TLKGZ{eE4)YPZHkONh?x2bB|jfQCSg)%fM65v#*L&?4w zrJQCbR;Jb5KgBD=gg^z1Z1&x|`k>&T?DTLYmIiG5R73F+rt_3-o9iOB7HqftLvg#8 ze<6i#O*V3BJ#G$^l{~K3_?D=F@Ax#zV8wzpwVb83$0TJvQMuBA<>tQ{1Oi(k7HyD;D(FV*lJ(gJ-t<;Uxa0q#bUPCkNZ}=+sSL$ zP=z*1g3S-5P8%!LhaNWbxIUAGqyiC+-3y%^%>AS<6tr(uXd?4=@(A|u6OfBNeip?GG0jD=H#l3f{68EK+FOC(}UQ0?qs&xh7DL7WNxj`{ywMCG4u zi@$4^QcinQ^Jm^yp{mz5^;EUhysM-mqIJvli{yJzb^0y&nO;ADjFVyfzh|8AXPGg(vfO?L z``2SmS6j1XR4UYV6+PfFeFZ%Q*Ki(rLRq%r6-e8vivs{D3>}KH3Ib+LV?@MkBpeIP z3XD*q0T}C^$f)?fqp-ome{zAB(}H~j@McAL%r7p&DeI`w$UWeMLeKGc9~_bTCx^q( zqJBIdhSwh;s{XeT^w?gxdebT=40n*m{@0=5$>l{C4vTsd+xuM#sGp4$UA4%G=Epp_qmmAK z1Nq4TBr1{@=@>AONRz=;N?w()sWuGn%sHxnwfN=?Ha;LRG*yAHGdtjA3FxBB>IHu{ z^<5VUkh)xkvuygbHDQT-^~6hX~}dI$&E9BVjTFvR;tdX z$63&ZA5@L*23_2Kt+cH=zHuF+b1ABDLdXUnpjqLl7dx5+QX<_+Hu{!Rgo){zZM}0G zqC6>}zA!_QdFDpP$^03)x8@d|#ZsY11L=S^% zTdga+lyR{A5JQS7$yD!i8hLDr?(o=i zZvWaAy2kA=Dwb9`;(2&Qxt*~6Ff#|}i}-mvN>1~Nf4pa%h10U&s64)8wW`^Z$7$xh zF+4^}kL~o#fYU1;T+S~Ei~D3wn~jbx52w^Rs0U&rHx3St#Enq(H-MBhnJrzVbJ+n8 ziUao`IryK5!lKPCQX!Y+-rYWXjrWzs0A7H6*wiiikZcU@>Exq{zCL$CX#@|!zVnVT z2UkdqCKoa*$dQ*wjQrgnC8vYiryACWnL~avfH`U98CbVv=G7>a0$l$2HfXk!cIhpE zeBTiiRp2BCC~6_)MFBBYM`$rrEns}Djvu2gf{Z1{7HC^F?Cyl*FRJt_!B=_3-3#GBL3+d*GENlRgf*n~S-m~#; z=G106=imRi`K{{b`joSx7OlS45Hl`QU@KZZ#@YQT?J%>|z=V@G+mrj95U;5teiabR zAL||nTA2G^rjpN&)01P9PH-PjBT)Zp_^v)de5P^?@@~1=mXb|V157^z($}{1EH@*G z?-hVQ1emyN?)*I^_;g`sH8^wuXdt-nK6qNL#p4ShE;>mNkiNnL zvU}pPDe^2LU9AwP`giXgbjM`GboTms$={@iO*Q?*v>x)H-Z7HWL4A>Ezw>YP@T6e`MZ!PMhx9z14Yb<>1 zh^V!DYVS>hxq?h8P>YTlKPcZ5b=~vU3ab2&4d!Z#K;KY+mD4;W(^F-xRdXY7+g(n{ zS2g{7nu@|b?l)cAYsWf*soI`HyGBxaCr5w$jwsF`zcC<*qL&lPr)4|PNl?6@BpkUA znSr$7jZ685>{kU{je@PNPl?tJSVtc|KAM9UFpn~?mBXQVO&}Ff(Obss%R{wIOI^^% zl0?e!rfWL~(p)8=G>7^nbnq6zSeI@^kncEb9+m*0FX|I6$WjV9#pu3p)dx2_&%NQb zfp<8{>bhN7wR9(?mhZQgpyXgzN$^VWVRL*?N7)O5x4TJh+x9zKM{OQ(h&R-hyVM;g zKqVa{Iq(9qY&`(xe}$TBR|~v=-ft~SDAU*4R%RU?1#BQ@rR?U&b9O>GpqWC&IWRM* z%(0>mg8~G2^C6lU){`sCHm@W2kN-oPK#2DLy-mP9>~8WOu0QJuRz7*Eo12K>t>!h$$t1!k91TNxUq6~qBEzB^#H+ug}goZNx;MJw_JjMv0(6=SH8QyxU}a|xf1P~ zUgTXhIGjTUP|&=XVF%jJew<%z#dM5crc@R`!Y**Z5uq2jJBKi@hFWsh$5!g69&e&}vMv~p(*n>Nv}%HJd6NM9OLDgsoYoU@r;xZP0vOb*UBa)nGZ5$j4D@3Uj%JR)Pl0>yzZBx&%<$VT?l*hM!upZM=`sghgEae( z=7%-@F{O35!<4@$jxsTo7K3*pB0aKo!P$GARwotY82}ne6||~$X|yIL`Se6Y>?V=k zZ+g^H)=M2N2>*-Gb03l;(2$tl;>o*-tMe+VOICCtt9X*=f~G2pk{#%wiXI%^T%8YB zvl9FknO^N8QI)3qNO>TU`h{`5>ig?f=hX|7!nw-mykb|?NkMXbILGHVmTDuV7SI9J z@1In9{|x~B(D+e7giA5bfe#2}4?JchzROrw+xBBDV z<6$7x7nEAR;W;~PgWVZ(bMG1eA*wP1>~dhvK}DwfKxtiDT(+vFo$dxIP>nCb()*m; z8LZ;~1+aGDZz{JM)u6+y4)07C_xLnZY|sw9`)i@}-g20^THuUHJkAJ!*So6#3TURq zs@8qBK?c{yt{Fe+nv%KkBR8A>Fg9R9HrEkOKl2RdVU=m^!>ts#r8lw?Gjp|ByLKhI zXTKY_xAx+nGo;(;8gB5q+%@Df$u(pwU6D8p7`L zkuCUBp>Z!3)tp;Cf`29Ii1;kG`Pj5YGMQeYRHD?$fX@=2S{W8t_Z36|TuvmvHvKnZ z#AQ6HEHqx5mPPsh#h9|lzMP7d+}gWZbu6uLL6ClRWoVWAPz`-i0(ai@b{^$DempXW zl58AxvPx|i@irQ)oOo-M%iQ_f@;fEur&>cN(t$(Qd?O~oPK*wvTa1Jga-D!KOw#cX z80|qznk(b1DjEJ(APt9sF;gj~+LfKrIjsQL(fwDU-|v|+%v>3OTQq7%j(mkr{?RR` zijMdfEeki&N7Bqz1DE_q*E+0`zCBsenzp;MR=9G#y*3A^#_m&!&B$uf3{l8D-H!Rb z?=$D>ff7EG1Tbc6!TbUC|4=w0L^3ESKo{!*h%9E+!IRRP<>a*}2Yu9blMys-B)8of z=(KgX#qakkq>HwvRaViyHmMe*V{pPF(q;%**`2YNFp}p_U=OR+99cfc@7r`#bg8&; z^8kaMnTR(Z@lYTngnc~lE0FSwKhs+)%pcl0NUJ9HRvUw+DoP4Yw$%_z{@g#Kkdpb` z&oj*Cw#q^Ymc#KZjk_cMb{hIExge zQv-i3qm`AprTU8Cb z_{d}X)Ndscp&tnRG39xdDG@Y}0_g=#Uj@A%0mL>rX_395Yz)9fC15Uuqv9du#T`-$ z|Fnk$w`VN2!ly1ii#3x504Z!XG`VK}u>$nkvgzh^HNVauzZfB}8G)`YTrLM~Fh8HX z^o^-cdsA+9yIOj^<14G=T*qWAh4s;Mc(t-kIN)&D1 zAhEE{6Yu9&D~_S9Eb|=ul%S(99~tZ4=}KbRPxq+ zUCj80ml}k@{!O>Kiq^i*ogZR|HlX(NB zjme-@B=vCAcdWy5qI+UW@GVO$%-G4&l{Me4h6cxEQnpZn0|)D#M|$f}=wX?JkfUK+eCs=Y#)^Imwqr*n8;KBQC-K@nT- z^8hwZF;m%ac4eb=v(=0rh}@xnGUrE!`n}Cvc~&;RoE`tWuVgQ|XVdns~976*PDU84?-)R zeS6^jsV<8ziW0U_^vtSr>9~w zseE^V#$%LZ4TL-tXHa2c#(-^?No8x9S!X6r37nMPAJYsP>Yqw=JP~PS)=R-`6nB>t zc;-i_-TuBcX|@Xdh(M}HxiSeL>HKpU6}}_yt9q1Q?M}8G__7(EcSAEz>N@Jv8d(5i z*0W}+g&gytN);|1U6?myA1&J>=fTXdQ(E~fDSAyL}N@RQrY>M?f-N>U?cPlzQTjvxME(n zPwomSp_?aN+(Kshfh$}CP*i0D?{-m9mW*s9N@P%>4ePYrctfNwAU4k1-y=C)?aJyj zHzICARYOapb0~PKGfn}UGM z6K1RVeE%?ounibz0(sm`4n2krMaOo3yU%>3D_J}JC1VldO^e{P6IrV6JgSFHUv`3k zgfB$-Cj8f;QVr$j7HD;Z;j*SLO%f65gdS-+DlG&=x&%;>CN!tM<)k{ml_>>73%&Wx0olwV>-IwLK+D%!ljc7 zz%dP+fEZ13awK$GsJhlo;4_+PFmoEVcX*qrb)`wz)EC zy+8VEy^Q-nQ_u^|;f&H;ZC<&k1PWYPZbOgq^|5lQkz5o76l+rEa#3&%-&h^fm86Un zdE+vC{bRqH4<}wEsX||=-|@0aKEQNVy1>=R^duSXFuLr64l)crP(H{a2=(iFk5mzOH zDiT`c&-Kh6t?y^I>EK%=h+XcF5w{zs^v3SfrVP7M^r>>e4!AIjPzRi-=}Eq_4}S+z z@#79@9-8c@kQd(TWGucF83TF-tKcJ@2)mO@j5yp?=>BJP-WkJn0J4SaD*;W7Tno&Nl z-zA5ai&zFa(B9mL)w4D6Hfvm1&x3)WA~FpI=h9^W%CkUW(5K{?NU+J(&uW6dd89p~ zi#cwTTWS`$8SXLX*dJ|2xyZi1dpCJ6bf;Bf)36&~By5MrR=r|P*4)4Ip^^qu4#dg` zlw>*q zj;C!hX!OD}cS1Slq`7gp{IN1qs=GKzO>4mr{^&b&(I6ivYiT-*sFl;5phQl5CF3=G zt02!JQ{A4_6N!FtuI_LO{eI+dv$PX;8yjkObo>vJkMXgNKhUo@<>YDJs{YM&i28P! z-(42e zJO90$oHH<6XPxf_vQ}aaZh5sBZ7nxs+@*Q!(>O)L((;2CeQMat{J6&khPB98MzLW| z>^SfisbOVzrQG^Op^~V|hvaq^`LvRAszNmp7frevkQf4(0+G8|VSv-{Wgk2R&?oSy zk%4i}A0ajEz%AlP8L;sEOf2sC*o&7+E84iv9)5ky#JkOq%Fl>qlN(8E*y)d+j(c26 z3n^Vv)i{!O1B&%mfu+5td6u+nXrw^b;(9iBwx#gw^u6g3?L4NP9TAIV_vSd;HRwL6 zyIh87XUlEE^@PI?6xiG*i{;0mtD-p$E#&?lQLi$wXsVj&?qaZm6IGK>WAOX#C2JxH z_fhP*=Xbr*qNOt`?4&RKTc3Ac#r8g!e*j0=r|#GOHaoO63Lrt}=Y91T&+5y}UCAVW z(D7x?jSVM8ZLbF>Mrtu4IZwDFXP)|QQdTlY-stYTjy|t(6qyk!!ENk*v}m-R;1`s} zDn-C!XlH8b!?gg#5@4@#D7}_d^}v36jZ)p{-%~Lx{aHDI0sT~*hI6W0H;f%IhW2FX zS_)f`XFqQE9IB3#%j7p4vmSw+ad)24G&@P&Z`O`KlYH>{?6mC$wU4w^{1~NdNzlVYW=K-Qj}fKUkPUcMBjUUE0eDI!>E0alK5wdzmN~p@l4GN+1?j9e(U}{1 zP=6%0l$axe5fmW~o7Kf;LeUfai={L{xmnSmD)z?jS0#$tXJtC%hoxod*GH&!X`($K+dl!|^Jm zZ_`Q$KUXlWlNMWS;u4#78Y51hQH}y~$Vu41W~z3u!yJGdT*6A<(xzGaZYjhh=tY+V zk5nurNS$skQj{oM zEL7hGi*fEP5}rgavbCY)m-!7t9fCDYmyK%B#n7*B_9T_8!kSlUE(|lSV(#!FVSdAG zJ3}>4@z)0?tj^I?e58TeM#yT1brBS)YHByqJVJpezeA)2aC82`IZ7P6gfg^ZK7o5hG``TZJi{Zg;7V%NS@w7$i5J0q7I__1tR-PXR`cq}38#V(6c;V4a3% zYwVKU!JoT5O#8=8Gj+5u>96XRy6d9Bbh0Oh1C6Dur*`F@ExmSMV)IzD|L?9qIa%ai zWZ>DEYs$)AHrkWEJ80eF6o#uRAf*E0+vL6|cH5K^cxFQy{djr} z=}NHz+;Lnwz8FI#=TwyZW;pjuvC!n19)$6Aa}vimj_pEcp8 z^A(ctjL5TQBz??Aj;W9A-cZNo@Z_dSJX;RViI(+w!Rbk8Bp!tm1qr&A=$81iAKj<# zeJn{uMVIAUn`3--Z=Le^uGjO{AM?ubq0pu_=1X@{xG1?JnaQ;6`2C@;Mm2YW!hh;E z`u0BDv}m>LC0%=Wh<1hKZm3`Eb|}+r3IDlBmqUa}g%VD^iUK~rps9TgI$e&Hv%b?$ z2l~ez@K>$q+k&}0b7c-0$7M2cb!_yfLbR-`5!}q|_U5Qhjc=yEi(=Jia;fS{*due8 z;~7max9dvY$V=^sv3qEoCBIT+BEo^yU$<|GBM75<0mW}|0%1?Q>0{4Lw>sM5dP%Hj zdU0_omauv|x;My1$O%lI){p8y(1T`jK1@n+Zp}2CFHy*wD$6t*0U9aRwRs?^vtBNK zKV`sc#l)V+v}olP{PVn2}lPrAC3iK$8-V+>(r7z zRx@(b%JG;9k`Rl>qttO&#q1+GG2Nee83M$LlxycV2~KyQl$9a9*n)MY^DM zH7e9D3_Rk3J$NvuiiZ@6U_iV(nNs6sy*2ISREG%2=|U+o0GS8P=+*|`w{PHH26 zsf&T>f?&FmhH3vo`dQ&XeS$}-2MPS2u^=k^3E#$#cIbs1o!FJ24ph6>IF{W(&q?`z zl`r6x6&&9$8eep!0lz4-j0+aJjk|7|dbmpy+XGzSzah3)N?Hwz=RA57iBZSIcH3Sc z{V?*R$Woq^U6|n~R)aa_v{$T=vO4*?!av8{{rT+_kmXCH`6x#ViN1%h5FBB5If4*9(jl2?qfvv(ne_4%w z(~_Q4>&7l_OPW&WjjqCs!l7eFpbr&_toL&fn$&AbH_P-5ZmxtoG_wrAW413|CfBd3 zVU;MQF$el;#J`47HP~ZQIWSN8Qq{~vWQM0!$x%jJNM{0sqoYS$v2n&QRFrhFgp_V3 zBu33_mMFs2+r8v)&~T<+sqbe?=>l2Na9lrm&$Q`;(cXpe%i+LaNo|-b89$io$C|v` z1WcL+fg6)zg%VAEKzFkCtLE5o7sb+u6?+icnk9{VNi1!i?!e+ z9H1EjUz#0>87gsH*5IX)e##T;zdLHxT+aH}ff zriHIJL2?d`wY%js`~W{Z&$cuAp#6bpI0oM(3TTYyWoe_<)Ps5sBjI9ro#27yycnX6 z{1(*ie7tkPOe3j?OJTm4-W9s(-0^Vb!)FIBlZ}GOST;G}pShj%BCz_(5~Bx);DTi! zPmEiy#)%ut(7VX5g=;2NraM3|CVlXxgGHSiC&=bw5ZZeCrG?tX%Ny43Dg#aSVuj=H zHa3Gf`n`?YBG@0{nD6e2HxL3eNwipkXRNo-wl0c=DEg*I)VQ zHs`}(dD;YiUc;<#ls*-Rm1&McuI7nUt-Wd;_A_kgXYn z;x4?Y`{+4M?WulL8RPxT#b*E z9@fiYGP`|TrAEikSXOM&dqs1a&GaEDWK#`P6xGJi)`C)yXEfkKEQu4S3~EFo2`uT%3#rU>EDC@XM!Qbf z(HmVNmUWhi0@ID(K9A#grZs3YPlsFc(%85kR3<#!UV;rL4KM^`gsxfGdyJ?#Kk=J&cD0qrlf_~^eOBOKTDR!oAjdG{>uCpAd@gkeFk(+4jbtOdAVETbw+>@TX@`lt? zp(s6eWQQ)=iSX^xW5=$46ZG_sf}ViKQ8V&)*BII>B4g+)&lXa~zwwkwVWf>^B|Z2u z+Sf+wR{Ep&&anG(nMCb|!N&CkFK>2UmVsY%oJBK7@Neot(;e_cK)5px9ufuZO8taS zNy=Hs&Ihx$EA?vFBmQA8d`W|2!Ou?aKM=$8N7Xd;B0eRz%fp^Rp*0AZ{_FSZf*W^e z36FS=hTPx;XyfjxRNT}-H8tCb2Hfmo{sL}d$HjKev9D#d;TQ6vteudkDi93cUaNeP zhvf7YTTE~b9$2c3wO02jd-KrO`Fh^Krlki`V<9G}Blql!g__`CulZYOn84FHlu(}? zO!w)ch8*Gti%~VW#Hc5rO`AuOZBfI%3lY4?ren|ScoM<=Fs73GSeR<*_FYc@ANj@O z4!w~g=i>8^{LTM!2%R1mqWMNdw>q=6fEqr4(N%)91nadd@`>K3U7FQ)%eh+`D1w5 z!?@rD|HB*8OX0|lCG|_uBM?;utGVCK#QFvK3pr65$Y*i|Wp>Bg5n)Ow`-!AD-#o=d zy+7ZdK7Cvd{!pKwcE?)Doi}yptzk`}s9E?ZLM&RB?>26aS^j9AX{EKZ4?zq}^gl}& z%v%IdOz}LMQfLoyh7M9M^X6-llau|5v)R>2Cglz%hoN(zkGWJ8MqX7<-d;jriC^ zE{?GtOMPXsdt6|*nG6+K)x05~B;f~VQThHg9iq$>f{vSUon?=@*w(92?QFpgxyyj}zREhyN&( zo!D`x?*_ID-Uxs~Nf2?7rQMY$s@xnU)WpY|qyHfU2Vop=0FwSwK*sGQtt26BOp&DD z3)0KW-s#HC9-9c)7x5MC2orOPb*2WLi`)t?6Sj=|av*+sG3AGoWm*UO_;bec9$#9XH;X&ZGlhji=9D} zz+Yf@qwlP4{J0CPzYy5PY9?|lHZ`1^?~m>%@$g$6drY7mPV8!lGgZxJopVvsl%=PC zi(0x(L&j{6RIC3unTdGr8}}TyG91{*)Aje7^{-ct0ZX`*`0tBa@GOm8eDO+3QF;G@ zp_eB?{|IVL^_gQP#$H*+xJl*0+JeweaUEjefI!4o5?67CzGB2EeDi212AW(UPG?>> z^Giz^!o@rZ_#gcQpYJ#4A9Ny7Wc~d8P*rpb9`MVI=k9b3bE$-44w^_6da=~0J&O&!NT3A4A!|p&@i9~q(i?bec(=>xp>1Y< zB;`{iM)7IMaJ{&m_(086>AbIjWv)4PbgodOXloS-$R3RNf$#RjhpqDCx#>mJ0atBl znN3S!rxj4%!dt-auB1J13{EU(tQwkAv3a@VfggtHwmmoh+y}n--YeJ1U%B>p{nvIq zdy-z>fgaBY`|ZvHn1>R>%~M=^X~4qkU0oY}Y%l&+VX}f|@|`GWk&m{Wirt1+N_;94KiO3!?NiW9ZN4m!RpNFi(0xr|@EJY$r)%Bl z`PRI$tVz*uS*C}7(%pJ3IY;|7S5H`GD8%Mn&)YOF4QjOexhL+c`(r0K8j+%d%)Z_u zN?Cb(514ujDwam&qdXp?)HBK_~q|Xf#5CfG5%L*|=~ty{PI-g+&C*3+<|H zBFU)BGM!k0_#0${C9u+(akRyx?c7dGn8mJMkTu3PZ^R56ce=pqedXEn92yMd8ywCI z)3=RkIZp_lO5oThCkmhHlcL8zg}Vl#zQp=a!f|H?zCJJT@RK{nN!GWYND%~-C_Akt z{ozmy^0oEWg}oTK_u~Fj{T_H-DcLI&@|R*);R0PtGC<&`jB<6s)Zq~TjHchtVP~b~ zlNB~JMMxw99}*t=7l+~67yIn%h-yALlX98U(4Nyc5_XShq3Z|eCAr2LxO&`pVM&QZ z_vgl=?&VeG6l;PXkIF0u3*GR3E>7%O=@EqigU^oyR&x;0@dxuW_(J18H^a0d<|GNP!7z?-U6m{C@zQq4G#jkU=j`UoEbu(e)bl$b-@1 zz44W68G2~nH8VbQR24mvtQ1|Tc-1%JsbeXv0kX&&M;;{F4p%ta0&hactcF1o}k?wW(K=10H}zlc0H!o zag4Y*8Y50R7O<)|%mLZZS8{22@}qp^Zs81a#kx~k=S!l)f=g%-gk$G_DS{Bl16oWp zChb38`OHVMdGWFrPycZ$e)PJUX42S7bexsC{%^)8*V|KqTdjhdIfx*lHNS_b ziSsn)&Hr|u2B=-~^(J)t^p*clj8|-~{|hyGs>Wb1#OK2J<32g(TR*F(TjQxmwa0y% zF3ISbqSeWxV01KKXD@ueEshP##H71$uz{1(AwoVE10x>`*z}; z0cqn}Ri(uy+Kss7em%LZeo9|y;{+u#r!vBkbdJo4)Pu_OPl7SkW*r+%P0!~^Ks;7E zqBaOxJwsQny<5Au3S?G`9QFSNnbrRgnY&$Oni<&*-Ln5<+$;ciGmm=lU0k-GV2{d> zIV|G~mfbMLG@&C<9L z!YZIhEQ*qaD^r5_pVo54{fU`M(zPJY<_Z+guXs#<^BDzUId8}GMGs!4%Ln)|_XKF? znuONuPx-;UD|RqFqSRuyp|8-r@{+3h^WJ%%i@^|ikJ$nQWfMDms(a5c@qTk2?!v~O zrgAv7L6D0qD%t>5D19R01`YY%agDdWK5xv9y!5jET7oZPWzNe|&J?Ug={scB2|e)! zpx?Si;F~-aln>#sQ!t&PugzciL8Xp}M*o;4KlQp&vL->?=Fg=uNsVIRb+U+|YciDJ zRk?uS{WGjI-hCAjSm_cfOlv_!zey+YjY{gph5u=45h*pQNOF%v#ZXL__A6!q?XM#O z$egXTig(!k5Ox4jk(U0!{mU~M?X~R@g#|m=IHa@}=b5ayB#KOmh7}8z&>}Ur|ECsz zZecn8lEv(bi9Zq~iZ*?n{Kf=idcumwW8KdHN3`)?*+!?uL3pV15}YN(KxR(jH@~-F(QTYTf#{wxd2@WyI)7=HeJxVV6b+fEO zk75pD`VlSt2g)mgdw;?IUh!h?gOR{ zC8wANy?Io*j@yjoS8O;>`jm_Ve&)r0vW6{>GYC9u7zzImApCzc2iCbBLo*Y;k3HiX zrbk0*g4aIgjdh_*bE3il9sk2Hn6*0s?H;f_Ee|rtYc0_iad1bOX-yc_-oz%~{H7{4 zm+CVS_m_ikoYbAS3c-e-0aKfN^N6Xfg+ZJQ1oMtcki?D}5QGWkA4Z0uxBn;Y2lLF4 zG5!A=>Hfc=9a?81!}@3Q`j2?fLfsMOv#E5xs>)f(IeE`u`LKWoc{D1u`TTNz=i)C1 zLm!xTCV*$p!GwNON|bJe{@xj|(563zzocGdRaGktJ&u9-c2z2};QnkSoyD!swG&|O|-yKWOHx+wkZv;Or$OFDe@L+`zio?H5+^qzLez?%0) z5j8`fU;X}`UbU&pliKK=-0b+4e`~8k*;~6?-cwLvWMN@prm;~@c1qZ>fww8*D!E7+ z#bQ8#d;JP9H_Hx1g#4PJqjt>px9-m_)aGa(i%6~U=PPk#ft@@3bdJBCibudsecBpp ziS$ykxpvv^YM7)mrcp(f`D+$hMj~6pxnEU_F=e1&G0VqY!lLfZdcubR$wh^F$GHH% z7?lHrm*@goi`t_zW!DSK;L+DgY5j?IB@F!(Y7*w!mN&yqhEdX6m_)8|vo^a3GaEO> zvg4>Ra4f}L`>Kj&H5Q@4vosW0PGWP zM+nMiqOGJu9h-}wd@kxT@^97S1$~<2(-QDjeX+_0qmTrjC_m$JOtj@EUz^z8QwO%? zmMV`Hab#X&e}(W8ha%&5;CfhS@(ei-i2E%)Bi|3V6vN*aB9ah?z4)tbWf$5i^oG0K zk0GGr!rxD4i1mNXM~Qtf=}L`K_iMPBgbE6tjbn|gz-w?@a!*>HivKjg;W$Tl7Y&mj~sik0Aw40+PvKS>C$ z3##oA5$_=;q52O8Il`LW6^4%yyPWUS(<3E*W;Wys5Jx_=E4Y=S2;7UgN%xdHFP^aG z`21QuY3@#^gY_q5g?2IYX?$LJT%hHZ4e>N^kigl|lcgF?7ce%LKub728Bhgvn;!ap zGWhf8nAH3fnjUJ26j5^`_rVV3+rCYSlBaNT?hz<7!H#NM9`B#*IDyVKD&fA;6oP$_ zB*C+aaHV_tY`nH)&UC750cAJ$*6PVnYVMt$;|7bGT&6j^qq?}|JDgoi#{HhRN;5ZA zm+lT_4`eXx5)14~-~M#+%=67%p;z@ zEY`*X4~-=4?@k0qs2Mj!ga}!2LKo}e#YpplIXZ){uUCf{1l*PTDg={gx_5s9B6tqL z@kuI?BXjj@&olD(z)&%$Kg~- z{UCyRScE17p?5+X?M7C^V(bndKmXTDy5NS1S6+l=LhrwR=kfC=&qa|W$Sb>PrwPq^ zqN^NnYy$k@#Gu3~YJf9~L6iC2`=Xyw>c%XxZilvdL$6cqJ4rs1nUUAvTF`R{+6UmA zuo|bzU3={?MTu1GlM1c?FN>t0)z%S_FW}0*Co&4-Aonz+a9i~jG=?Km6(x$-ACjEm z8rZR0_1)CawfkMFM?WkvbjP^a?V+5@9V*k7-Wq0g`*Ucl7%RJ#@zR4z$_L-c%-Gq~ zCS##otsX&LQ@(pu93z67Lz=ez`Z0J~>M9{}Iz?^lYt70o#B(9_Def!?!d*tUUv;#I z^MP~3+hgNUGto+B>vUQ3Pc0lICnGLoLv1~NM)p)vtl|?ll4funpBW=KyE-*YmR)o$ zUpPgbrM+(+Bvk!~)>0Qx*TZd&i`GHHskTaq=#Kf-q(F^&aEnSUa1@Ya1J&BiIs z$z0ML>?#0avX7RZ#-Ep{rl72=8KINqDP)SVclt&wkSihmJ4V zxe2o!XC2D-<{T2?p0_W78rW|WNId7iQ1olWwAmMqQ;NyqHSzb2slE3m3WZIjQE9?! zmp%vFWwUhtnAsQt6Mq=!JQHv@LqGXxGFXJWAjojKMN2v0cDVl)-R5_SomUxt)veGA zu$n2++7q*tT~K%=+k`fW@pW+W_27q6_8mpXrrsW>T_taha~N=vokLOb(@~k@b6s-9 zSd|9la+s6c=I{KuT@Ln+L8v#ev#mWVU6NYykS_LYmq}Hg21sc(ia~B7eG#5yRoPuJ z*&gzduWqg+$XA;`Ay@|~pAq3X?9`j~U{#c z&F1Kjmj?OXet~xVK+WeY&v`tQDneaJ{w)?Kc0;VZ%p-I+GIpmRfIs)Kg*fow955Ps z!8d2p(DZcG!2G6~x}uo=fiM1w6kJ2@nm&#t-G`#hpBG>da1>7p^-*c_DN{V-tKOVY z0K2BF>LWWIQ#C^O-^AcIikvbTETJ2JbUQ~tWhhbnx>{Q5rbT8`Vz&wMg}s664JK1)IHW_;RC^16VEesWqy8$=PSDdy8HoB|`l{5^33VvEuruL9RXr zrJPi^CLl?g^&ycVvgQxaP^?)ndQ1Y_ zD}=ug`t;d*f}Ud+lYc9(K-#N>qk||@c$~tK&gZLHWtnMpalyz2dpDFi5nB{M*BKZ& zB4~&jwHC=_kcK*o`yf)A{k!b_HK@(q8nwS5c}6GAqY>vM5%#^Zoa)C~KGyUS`DQaL+!%As_3|o zQ=IPn=1MNk0qh+ZHxS}hXI?H?&9F7UtGeGUWOYY&E*kBW()}o&d6&6Qn|qU^R=WeU zK4G1}ubl}gQJ<#u1AUb*6YSWYfU8%7?V;&f`wcArm*mwhWYkJ8{?{bCCrZEbA~LBh z{8AnV)d2h?+I>n^Iq!P#L#6^kW_NO_;O1?vEAwFI{-cTfHJh?%%R%p^;QNo6LD0tyGoFP& z@DOl%0`Y8FT5vz%5I%5KIiLeUR8p4v=Q zHExjA-LXpQs1zWre^oLjY3@0v?n_{|-@IWKqwYP5hSjABgrY6?!AFtSDs`IK&vM`? zs)}PXp|WjWnMO$cRg$LNYN|+Q|12g}&A0<32hs%w_VEuRFfViJyADpMz$t$>3B>?b zh2yd>^_az@WScPMV|jfk#3JV(Ma<)p8{FgTDNeJpBE?!Sd4w^MA$m#sZ@Qfy?mAQV zCayA@I$*y+Go-w3gV!5ENESla3lDur9&K;065rYMb zn!M(7c79e~jOmpK*rb|Iixi2uF$9k*H$03Z9KLALc=7|Tpg|UYzaD;Ld&01MFCqUV z?ePEe#Zd|}L-;_(g@4Q>KVIK=cn%V#Bo*o?%@ZZmM)(L) zHB)KJ1VEJxm>Feb!^iv_Ia zl`KW~ndfRnB4!mipaQ)9ZdTY;eH`WEie{+dj=;Sbn1U~#PNi3!{v^vVkNJmp+~zTT zk{TGkr*c2pxAHcDTl?_S@h48OD|97enX{0ks7Uj|k##nf0f+~>MED>eb?WzCTHfHe z0)Ac2n5`EwZ8;KxEJn(wg-Zrd&&n19?c!UuPr_^Zf;Pdz;PZuvt%X`OvLgPfI~@Bq z1(=~5(p8d*0R|%ySQmb!DbDf0N2-HHL&E-A!!ng}GoN_Bzc%@qyv4KQ=+b&2&`h-w z*C=~-)m8BQbNE|T>ZPG5$=B@3n5?o4$g0l(tG%{tr$NAl(rn3$1o_*t365R!XNRh_ zkXgl0kCeLCBXN#J1J}0(U55qJW_rY1e@Z?6#C2WMhI6ELoGoblLE$+5^Ua5{BsuOi z%MnFFKl0mt24w9GyU=ROYG{=o)Z1Q%CjmmDUi&V=z1qUm3%7T?bGf{}M5yXxR+;n*@k2tjQO|!p zzx>ufmTI89QmjFTCP)mW#aOlgRM31zrP8-tPiD>%0DOIhLk;bATC&Z0Stq=j3-Zb( z^25H_WLv(qyn z;qfttNxs^sOaOXydyA_QC@y$a1)T?g?;vNr${f-mt z8*QPRog(?qjz8(d1JfA3A=+AtC15Rr0(*`U#lrP+95x_danJm`z;=}?1+a&szqVq; z^c#nL14kgD1(924V=~iKX4!=Om0ngR{R6sOuL{T4zSX(zrU0CLqX{!})) z`>n{Tz&XgyUm(eUGhlfNq#{N`*jIHv!Z9E>{Rkbj)U;O634vk?K8$$Tgu4EYMs_$Q ztq>-J)NKhx0PBOp5CAKEyt9Q5Jke>5rgt&Z_ok)L~qRO5Rwhe9@i;tRv#a>V1f3Wb#v6SYbMtDAdK zPj4o^{S)-bdhqUg)xam|o^?lBjhiDfTHB-k_Df2b{HHzOa**84bT}cX9AAav18M1y zBmKFC68eFch@0`*anJX5PuV8%<6#Q9d$YR!Pp_7ZI#b?LHIyV_^J2;r$-)B~0D6r9 zt0wY=Z8$w5#IfF<_BRM@4=>EYdt*O-*}Sd$RDWy1TDf9xZ{+(h&FN~{54golnAwIu zdQCcBF#BF4W2jZ_l;6%oJ-gp8{&NP&b2J+b>_sOe;nL5WK+j-Dv-=X;i-1hV2P%9E zxE;tBNT@rdkX{LuaRZ(szSGmtKGk8nywBy| z!CD_|sUbTlOoTy*Zz{)opW;<6Xmbvbn$VP+LC8C+oDzwh9tg}&kPa)yY*g?* zwUre@QDVEdTmIb)s*V}`feWs)d{hR$bk3j0wX*T78GKf;Hrd0XM8d~=z=zpC3lQ?+NpAz zUpb)uJbHiEdF5Lk@g}Niwv#D)Q((CuwCT+o_Smt7lgh+N;F)HR{3ZKjwCuTQ!BEnK zg!qe`F?bS}n)#JFQwz7wD=M(|2i?>z`a5pppT?r{4s|Y4zqPO2tB=qpJLb;^?uV(#nmT zrJvaGH@4E14>BugJ8IZlfEP(td2Ho+>W~HamrKBr$a5>p;^33QIA?qB_5CsCWgw>6 zbEg@5t$Egc-=9|Yq`BslLxyJ5INPlmnKc#KfEf%W;Tih2jVXJWER3HOwA@ zNnUWBWaUSkh|VWuwFN2=?;+ftf0SzXP15Dxiy| zT8ww`WDDzFE-D&1;KQFNFMA&CmWC;BOpteyNZM+ZZu&!BCo%9FZR8Yn?sFHRpb>6BSgVXTWQ}?GG*unf zLkYCyd&+`wV@5Kp>6yQlY|}fWmPscj@d9H*n0BxfqiPb<-AXDWbf8kJtzpU{swJg` z_3$jMbGvL$Rb8@mPoSl^RI}aXyV67X7oK+Rq#vq8dpLpGGL`L}=+`94rS@qf(NvvXx7$YN9lv1e7apsPDM2?EG0Y^ z^`Sy>F3Lmhk?cy;bB~Wa2;=DEiC@*KJ5C&Kla<>x&NC-cU%$#K1`z1_4cUbn);gEg zPa-p~9UFcs#PF~(GWIHaT{!^@We5*NJ`y|nOr*Tf`-ST{fh#PaE%CHru$1YKPsDn< z-r<)2_pQuF5o+8FD$D05S@I&&SM$R`H1g|xsc|D+$<#O?Ls?Q=_FLs$ZWQxufYw4q zhT{zDh5?x4r9OY-7vfmNKUb^R0I>#{(tUKfvz373d$6fb zqz&x-d{`c@-n4i>L}_!0DV%t8InR6Q#Ah0Npj<+mZ)Vj9Ao z?_jH2oM)mjth1{3&bvg5l|R^9pgPp$*CK6=Jv{R?v%sC81`sTU2-{X;VHK02PVwy9 z>U_R#<_~Eej_pyZN(bL*zdTHxooyFS+x0#~+q=3D9%=JIa zzek!*8E&li)P`d5LY9#5jySW+SR5J&!0qT3CZu@~HLcC$<9psclEp;UKraCKyD}L3 zuvUrkPIALr;O4AD9t^}yOHPYE()XwB9HMHAlL;~=7Wu>Z9gkg`yD(2j>4R2xXwl8* zf=+yn_spG5A+8Ah0b3tyNTW6bZ&a&2O7zJEddLeD}Eb^JAe8;=I2qQ#9h&(rKJ+ieoIrGn}*-Ji*t5jAjuN6bKBH3QNT!h(a-;)`?aG^SP}C+ zfDN;Yk3X?4()O^PQ5MPOB+RN$gKdzyxaRt$ue61ZVdId6kTv*(71X5#!udgC^l{S;yQcwh>1hjQ5U z&ED(*P*NOXm92Vl2z5`eYwQxVG7sn&Tih*cs=bPW5Q^3~|5w7M7pA3Tl0`0hi{g4u z?up+9PLZW;GMB41jNBrb-O2cNpJ7>M-9*R}&E^N}cq3je{|^68j?Jd*cVxQ;Kp*(q zg2eq;@GaooUXRSDEri^$IqP2V#G*F0HD~arBD3xn_O|&l9}~ZvvYl@bT9mCa4Zg$0 z6K$`mer{6YH088NUVsXlBpnl&z)^WQdv;7bbabPn_J8nN9qQluxMUBpD@ywA^s{JXZM6=?fuS?4~Fhk%;H9^+=ud79kj8Yer;0<99al z{Cz9GP%=)U3QZW;u9q$FVb59Xwf7-ChLPdkgX86B^YsrqK@1fObG1x=%kAEBxjA`a zVRLbyFk53i&3dtHI;rpeq*qaQNF?9l&wFyrk?9EDF5J0npAAVx_ZC4YcJYHq98=}WzsOC-8QJ@=`Jg4M#Bj?>2%7rbqA`Q6aYVqFqyUNWUdD8m#! zOP9*9>p$jj)_O=K=lg>p^w62_rnisb-8z`ty)qS7^~Y1QYJ~W|z9n#kR|P~Ek|1rt z0L&FW@$tVf78g4|@mq=%uf1#QPxF??`ng`f&=zeElmSbl=^%z(*w|?84L`%9T9Mqo zvf&7i7~=`*9|G2=j2}iO3s`zwyW!`AaBK zQ`peg_s~Lrj+Oxx=o{O^cOHi5We?Y`m-P~83N6^8a7-WzdYcdX(sabhRU);&$4X`T zSntsmG%GQ*eTa``F@Ej8p2D}7>rtF#bwKm~d=!tJeJr>Oc2m2W`+(&*;Ew&4)MY%n z_S3ZGEW;HoNs&P)sG26n-Bwu?!%b+0iIb|6G|SK>nmOX(uM7op=%(hErXqtoyC)f} z+hPhICKrr`(OZqx)c?*_eh>Rpfz#>S=Y*t$o}Y>Y2EN^&V@Lo7&2#MzW7mf^ znfiW={@obx9Ipy)O``ctQ@v^4mevRDpt2tf4=lo7pEypYS)8xICM1lbB#)AYMCG-U zo4~Kyu`=|fCv7#g1(mkG(;A8@I6u{(={D6k%~@_?#j(8bQZr>ee8^>aYlnVDx~ z`+ctO@B5p7@{e=(zW3+-em)=1=gXU@3i&dsmsZ{%TreBM5uTtEm|mO|WZgK1TTsxQ zQ@JZnBzd9%8iuwq1DwhJaeF~8qh5iOx*P36#HM&zsaO6_j=@N3KKs5tQ!ez2XsfbK zF(I)$tK+(5?q+j4#9rqsqN(jVMYe$01E}*p`-^+_+m~bkMl4SCZEh6vHe!bUjK*po63~NUrD^H9%G|R2^5z*@P zjLq4qhaHh|q6DRR{TjIom615Iq7j+an{*3gwNb-&^-X zFXhfe{tn{mnPlDx8f)vK|4L&WhW2p*j>8-&)sKDxW;`2D_|aq8ZFy8ErT-s(ey36v zpC2-lF_$gc65~th(RT8H%e9x3!Gh9T)T>(+%r}|2CJP+p=a~n(Cu=x<9_?BIGTs1r zDc2Fq=yt=b5&x2V-N@53&$jUWS?&zrKWi}U5sBXZAzr}wLe+>4)7!_fqyxMptiqDO z&PL}*ZI(COWq!3;VY1+g( z|F+ofBp2kdpUSr;evnU>$+mh-^R;CsP>`q@D7RO|hx+7p#V*ti*3d_N+E5S=y#59| z7diGj_KK5zm&emH#n--R3u{lrNU@f%P<+|{G`xBv_2lB%jY;Ml-OmA^? zGQ{Zme1v$$*};%!pt~=p+x=HI z@-}{%bYG;G0@AlfP9)ygCH$9|sMb3+l^ABTmHDdcKJPJn`t%sQR~^22eQfk#S*FWr zE6!3rgQE@I8jr`%>*~BqE6`xWws7?7)pcadMhST~H7lUzmb++#)m5*_Gdp_Cg(xth zI!O<9j}F~vp1_lnVBkU`Wd0xOA<#u0B0DFW1z>V24eY!e`Zckv1Jm*2@xWVfu{I5tRcc5-VNT_cpVaNNME9_pv+9ojP+Uy9}6$Pig z>|+g#d2(XMOzwa-6Gk-8aE}~^40e?yNfL`HXtYP)cpSSRBv$Y&4%Ii+jx9V56ZVq8 z1v{(qFO+W0fpu12sl7!^Na|T{eOLVzO^rb-D@-v49`N?MbVMrBSaV_&?@>zRu!zDa z4il*lXKB#MpLkYq4up_3<{m`){mD{vDjgJ4Slcza#|7w~Ei>2;XvV=I!Y@GA0vK_? zn$bjQcJP-6a!gHzF-`-!G+L)Y`Dyi&G3K~ByPDDmgY~I-HrC>yK|bbHom&~z`L};s ze?DrLI6{sVT{GN`_n4C?kvAHD0{z2Df7)4S{)t2b0`x{>*lbG-oCI$yFfHZhq5os@ zSN}8l2V|bs&C0x~6PcH z(nC+j8r}@_+2v$mLz6i{G!$c3<2-h{(nyLpIjDt<>0D30^cQf4qGvs4YwwX7A$7i< zEJ@~dVF?J}x4ja}Q0gtlglggTX8+>CpkBBgBQuJqMnAlpf~=ohw~Idy3p#5Rl$}0b zjBq)yIV4@04%zRiA${72@CG43D*S$anI|O@MdSm)2R>6koB|Jwxzg%Omy8g|vp8zD z5g|Zo*~rVpboiVVMf*;-q$|akPxXni?@rKQq#E|ZUF98fm{|B*1(MZ&*qoaS- zY~mmYZqDhR0i1Hi#|Bv){Eo{wBbc*#YQRx~9U(*=@Ogn3p8^MW=j5*}$~F`wYz~I+ z#{3d{0&A+s6^kZalXk+Xo%%u4ao?&)cK?D-gUHiry)SLsa$p}EANP}t!d)G%lAuNF$N+8JHXityJ*{Mt>3FlM!BLvHJ#w%W=D$(1S%;d) zge~&m9ErenLEjx{iOhYsz;E<>{pa7Q9f(e>5F2XjimV{cgq^k0qumI^yu)`&pw@U( zI|~KycId)Yo`OIz^|viELi08xt=60wQ{9Sq`p@egCbaX5d<-c${X0AJE>TfegeH(u z{15xsIH?-A_F!S!robV9pKv<`oom}a61NhGKcP48#;FlWUDqd@m0=)KKe8CmPk(GU zOv?&|z$mP3|C&P?=fWh^T0;EhTIfs)6Awd!AcIOWqutA3Lyy-E)Q~If(=1|J=i3p! zUFHNK^elRvAFkMM<(&{dXYV#y*m+}ImzoFb<%CziE^(ct1lGvYNoI`$>aY_&_q+ZO zU+KS3r`v|?_yOn-Ng1BJwo)(&OfTTK2uOUmG9A{6P&2#k4fsmar<>E~dm!yzNm3&diOq8#oL826rU&Hhf!PRz1JcQzRR}FAPQv{U z3l?R8g7yX9Su+mOy+m>eKi&IJiX%1r*M9!X5Dr^B{BuMq(ogK2J0uDAafiU0Qd`q1 zVAoGl{CjIqq%FdkiBEQ=Qn`_vB**U9Jy@``gi05Sd`#;yAHAXrscKWGRzl2OLx~Up zTcSRbj#uo^XumkV`~2!EEv+?^hVmw(v_u=T$0)BT`%a9K8)ki#K(17Gmx(*y`2#{k ziwy<1CwuK;tT#|v76KYB&a+liNchfjvvtXJNV9RNH9?7W$T!rxs{JvYM*h@Rpo9x) z6n9O8&u-?w`QjU-)@1i=egnR5DzE4C;^e`uk6wENmy^`vQBp5kw z!W=_fFCB7VWiCbxQjx||mtfC=y*=Ip%&$xW67}kY5tIA|(7JCnE{s_#>6MuzeW_36 zq&3d_`paV&?BlRzC%SjrU;%5O)YGJyar8Tkz@BUPOf4BNz{d>0~PDRZf2sLyAi%|B3 zqv_|;3YQAmU@kU&RZAbuL+x>U&u79a=-??tRJ>b#vDyz*w@r`X(LlUKeXW{Kwp#XJ zFs0ReC8_4SWcP#lelE+3t|tspLmIlF61@)-U>3u&k2#H%C1oo#!31I~f#D4g63Q$| z8@%&zJs`(QtAHkTlUyv1H|x(~wsBnpOUD+Z+pu$z%Ug3d(ioKedG>RUIB;=EDT{>k zRs)5lb&<46v~|xCV7_d_SW%tDgGRr42jq}(b{8!Gt!c!`FigzaQCfw8s4sLRurh#L zm)!jN>gSs@Ss&mlwNyLUA4B?!kje&13v14vBhmWL47<1BaEcLwzKCk#e;%P)x%W z`JJsPoyA;}eK6i+_o;;D3tcT4+r>7ni0^Ze`Ci1^n+fGz43oK|@zsrW9V(y=-Rei( zwxM-NU_!|*f%oqhQki{8^3-llm8~SWi)HKzB8(osxK}zZBYiOA@0?H!pEi+jQI#03 zGRh9H--s`DK4Qh3$xT1({bsQK28DR2sCRLxXbzb1wY8=QOa*~GaKxHGj`+Ma@T2n6 zjan8*#-5=PDiS3q1LbMcHD!rQ!2HR-H7M9;?_42;ZpP-eW8g<4b$aaf<RR>IkW&374cwq87c|kwX`xmZEt(;-1LDS-ag!Ju?^U z{^@K5w<*8#f%v+ZX&V&aVqiNv)!*q&xU490ZaOtItwp_HdwkMXi+QN0~vhB*rIGW%^s7u+h-+Xx{a zr1#ercXmgL0;P;|dx#5i1+(-Es_1dAi5-$3GCZ>XA%9z1X^BAJ7PwN?oXx5YWd8cfHEf3-xNopWn##r6e_;JT{FWY zse-URZQ4vfT#(7wdkbxMPON0>Vl@&yWG}L<2N^iLZB)M}DB!Sj@Ms*j{WwIX`5jAE zdclA!{H~&J#&ofY8dT{eqyK>1g9zH)2gfqS9@i}rYT38mQ@fDr@rgQ*Waq4zHJX;= z0mY483df77KX^3M}3{2ReKzZ_=Um8(H0qftuy&h_V2)j!M?QWNgiNayF< z<~MKpSyh_%Pz$Qq5Cv{-r3o@CtK6%&5FXeQrc(2uO3WE7Hfg@t&N11zH$1Jw@<^QG z($Y}7KL(V4UrL3}jh8JE-dmUaBMY!<5CcO*MSCiW9l@P90E5|WN`{ehSo2rlL-&+& z#jpc4IyonT)X{(IR^E!C?Z)ndT1ijIs>zZ(8Q@E}gEZro!Py`U2h8aA6qYXMuXb~hEUxW-yP36o)modSx`L};QsN^6GEbIi(>f7OW=A44sisB_ z%R%~M%+t$DqYUO5wsYn7(`n-(>sOBjI-__Xwh4RC;{E{QtX0KC!eA~bJ&8%SS^Llf zHP7tL;$f_kU3-Fm{6uBixTpu6L~`wRK*K}#(T?Du?_$fo@HPL;l&sb$9*LhHSC%H; zKD=Dg-hd;Qh+Td!lhqoLnVQKiabEgr?k!7W}<4#q)4YT&D9`rw7*!tIDiP z%~w9*DMqmP}KG=dYtLn2Gj_YB_VMCcb0fc*1{TF=hgIq zHHc>It2a`Q(6qD|Y*4@F1gop6nz2xx9OmCl_sY{4OOyU0Ke~s>^g(g9j5H08_p2xH z&`Ylsdd;hxjSqg1W^G(%J7}O@tV_it5z4E9C{4t^x)EXWQreiqaB^K>roFRU(%Iv2 zbITM*GW-uQj7(@|xMZ(96AvSOr1H7x&o*tiTmF}K9X7JFUv45V(y^^nMTQKDYvM$O zca@K15IB&y@9&GBZL8`y@}*{tatKb-wC1BiBa|L)~7nNCdHvjP5nb$7B*P9w)T&vsF1={Lmg`AYvG3Hh@g; zlnV-JuZzI8e+cn+slC{$cqN)2WCb1!ql7(#c1ng4fzvjB@3l3b2kX`vo%nkvNmuYn z9vo2`nocuZuRi!S%qI3L<3Z0}Pex0eQOpO4MAzw;7W) zAQyNLvzHTlqZBc zzZd}A@q(i>-T{ULTj*)wYv4e$0j8gA1C8fpqWF7-Z%h`NtEItRC|>0VH51ykQvivA ztDa-BD<07BR>rg3CMi|eYB`iUzsim6nzp^ zBA^N;s!p`9y-x`d$=ZwtlARHUiu%~^33Orp;nuO1l|Wii0$^TrVGpsb z+n3@E?A@Xd-cU!=GtdK;nz=c-tHfml1E12}%5Af#4no_ARD)>0Q;I*2ahv%yRU_lZ zcDY!Q4@GGKWO&7dCz6(GCzdihi6=S<2F~Cu*i*^z5H{n41l=v-auUmHO#0RWgTvZ~ zZI6WZB$>7iDYX3@FUYc0w{YhscSn1b?Ho!r;G_@B#S2ZO=p9i_h}4}Zv1&9C>d7jp zq>_g&Vfkc_9Mscl4@1d8v^_(_g&UX!O^d>GD!NF6q?Fj3m=Sa9>zM}T|1J2a zy<1|ugrVe9KOIrN%Re;4VWQcLi7_MqxUK ztCuBqCzi#yh07hpR3#awm&f595-&A63_o%X?vsW45OxEQY)sKU#Mj^3Z+YyzPWsK@ zGjJiL{vjD!qAFTs$tdgAkE(o2vOhPHxO+}+UN|v~3lWACoe;PF5VAw8apNa& z)Ez3H>UZ7tXLPgAay#G)PgGFW?>p=hM*wXbbrcYg1oK$2xQ*8E*E;?h5@_erEb#EW zc{_l%{>>-1_(jic1%|qZa@a=YrS~&M>w}PijebO+kjQdFus?p}{eaxia@9$x%6*k& z5oa0eE|yf1-IjXS(4qQrU$S*qgplzlBsSiy!NNM;OQx=k$5ARs>YQ1gfaUbR@kDxG ztS;jNdO^hgE%W+a_C)J#5q8T!qNY8rvGi)Y5sFjReAWnrhX<=DU9u!eaV17=nky>GeFO}3+i$f4g zeHLO-1L8TO5C%J!I|*%87^N@AUg6&KE1*yZdWy7^A*NOBw&@9TFG!B})qICePIZjk^EVRbkL# zL;vpo)n3AmjxfPF*S-8e5dm}i!5Ii|GM-d!iV(A3BzLRg#GXX9_dtZ6%tbaUNEI57 z{%T?h%!T4^rczRLww-E)JMN1%LV!cN1CGOQ4g)hORb4xX+3fCYe8EL*UBSe9cN;5*=ej% z9|E{&sDf(srYpLDWX8j&kRP>pW8ZrP>AP8zs&>StWOC$HDq&jWBMEK4B*8%eW2MM} z_pejcD~#mDL2t7tm%pip?Y}-MuH~3)kYxIR=gZt>pa(DMXf3iH&I~%76`Bg{AtV|-B7^TcS=;)TL z;7PLg1bQNs@2#imlirO2X= z5XN!T7ws1De~G@~;tQvV+dXH!`WEa$_{RJf#nt6;w9KcUkDH!~8opLwGCd+Bl+)(| ziA(siqWikbF9#p}Co6vXFDq`N@t|ezzftifo`TVr;WjSwFwJ51q{oH=&@xKtLoB}y z+V0~z#{cEL0qaJ4R=LdS;?DRc2oJz)&u7n>>U_6uWdxJrHX%et;mnBy9-r~51kt*G zG0gHb%C%mZXSo-N<%ZUQ_bp5--fEtjCnEJ*f}OwLQ)hNAXlHEtwbg+JT#{iVAfCoO zlKj7rWs@oSN)DQpHw$5h_s@^?jG4Sa;T7A@Seq)h{z}r7?r^F(*A!5G_gx)pY#zXr zU7OP_B{m66&wLyv@X}wg2vxDCr*Fzx#$6CskQkQ~{cqIn4cK_if(9p&BbM`P;tl?V zdG+zy35QWN*x7&H*(goKzR(*!2bfhIW^8;*O7ols`QH7sfm8n7y=ob7q59j-GX2(}xf$De2DJ&+!3$d>%;bn4i*EKH&2z7g;9k4m4sYov z7M$HBh7_O3d&;UXE*MLiKRV2U=m1@&`8EkBqxrKXW^Hjsb2+6(1INlm#bK-9FAEX{ z|C91A|3C={@b4VMQp4Qgcn#k9&C$w^(auU)@x_`Ni9qd$y6q7WpN@(5b_A8QqPLP) zJEUBvOz9deysJ4Nrw0^_!f7>_osatli?s`wH(hv9@#-^hus7Q6A$ybVU!=Fn8nE`Q zmtmih#=Qj|W^Nb9d~_sc<5Y5IICEmYKo5a3gqU1IAcN+_*w!*ET)vZ(5=@FGbAbAM z^qJhjv@mB%Y&gS(PDwx*ng|O3}Fzd z=tY}LW8KmsN)!7Fp$ZzNX5zV- z`7fCSATdD1_XS(!7@T!biD@f5`tJYHgV1^MpB{v=l#h)A<>jW>Ld5G>-1bOyn5*Tf zP8}o96nwb}Px)kV6t*>9q1+qIdXO1X0_2$;#A*5bC$wVj32 zo^exBlE6@7tm||e<$-5h%|e+txbU>^$LOgWwU@1v%`+SD))NMNR381GtiaSJ8_*`- z%Qp%C$#GJ`8XCGjYy*jfbrQ~9@wwpZ|q82r2vRTdRi8=5$=FTGR*B+Za`Ps4gq$(hJ*v#YIzxC%oj{z)u>H#n! zxC`C(3swg`Neo7j*@~&vb&3}6o6H^ocxhb?Ful9qU^fFYSzL2WI*`_grLL&{M${m^ z1-?xsr&Lo`t`J&#lPA}Ja6T}gd9}OTI}oBgc`ZKod9$ViLOCO%UC0sfVm>5EsOviW zc)?18qcdu_ByI0`@e?i#qP>@OJ_)up@g1{*>%Jt;^iDFNRs*r!vH^hhNv3;x{Ls-iYKYnnpJp>w%i$%(P7 zjNvj${8t*|`e^{uIwG8(s`TyN`BQ;BUd(6fDN?5J{OmYBQtjFqpy2hVZiiV;+^CLuTDV=bl=d!e?M09^ z#SpD)5c=M5?nwei?TM4Q{>lgywFj))ae&wA=P8)#r|<2whQ(_U8q7DyM*x=5A!M%t1h<}^n9%}GMaua;!$nA#NvVot|Rc5|9+0Ne~4T`%xG zwUHUbVC!mbvYPYZlgnV#t_$#`1a+U)3)zCKe+b{AO$MH^e_RL&Awv0cXO0Mb?4h;1 zC2>5}{Sv0tG84s%0)$KmYQ9UFT2ALQcXUL?pd*y{MaAv4jT(YqVy3^pr)&m*>rrP_6CTR;1Y#?Qg=LdDRf- zpFf(2kF_@GJaiSM`--hxhKw((gN#{L4@k+G7c7O1>egIs>5cNU1V#cLK)iEZAojIc+ zsWMai4z*nLJ3;D9zP97?f5?eWDOuskO0a0eFx?PxNFgOb!G&v@DBY1=`^A|8Ed)*i z01ziWe&i!Ff4!^DhnZ?WOT{kA5jJRt*1Zqc;=i%O5Qm*d- zcx2#xqCLk|MyHneMB3RAQFQ~6e=ah0;`(?+@1|%Qiva*rPcQB~NGTCh`x1W>a=7JC zb`L!lH5gBIxJYRyR+A)fw3`~r%76Y`a#@5DozI$g>=N%g3o@#d2HeiH1wCK%@Cz9G zdep9bF^qzdN(REAg*m!kT?8&<`@$s%r4XbjaYVnddN5A+r1v1Owj^Z%HYtLK4yMjS zml8CtYWWzYZ8>Ujn{2PL?O=2q)`p!^|JE+uv3ARe?+60eA4%B7sz?qaeX3VJI*b#? zQ-3%1$7~ATj*ECHv;>yuSOIY#Y@2x>&L%&`O%LZ~Yak%Mq+lk{`4Ua23KQi-Mj`Np zT@+oqBpsB2D#Zh9?A*N{9O)Xcw8+=n!T zDaU?~%j3bR?`tzZYGms=-mmqv@)Frni_mW-1UYL3VBy5kC%JT_HZ!klTC4J?QWO)U zd9Apx|ZlA;Y+*g z>%a~v2V*03!@y>U{lf-iV0wEci!W464~pl)k8j2JSe(jkI7W0TJx>ERG8bvLwPl$t zKYBv^_cm-1O|IauXJ?I?OAAmCs3>=wDz~eD*O( z;4;S-(aNs$LEfA#$|r1ZtJ!;M|2_L#FYB8~1{Ey-+r{higfs-cpr+-HpoowP+3%P~ zwhijzT;lwLSB`7>QrRRM8(9^tQ%t4IWB2|F&@cOq8$5n9N_A zcrO41EMR$+BKjJW%hW)jKxTxDRDISlkcSl>39-S1CXS>kNWQUsJ9+bqStV@JN*;09 zw}p#M#5Zxs$IwDL+a3r?t~tl06?=^2yOW;A>8riq^F`k*<8h;tpsS}1?5zSnpPM!oD{VvcZTU!n=`I3|X+7Yq% z8PmF0`e1GCKzZt8i2pOA(GlV2JakMmmpU0JVwKq7V;}yZVD!4)#FmGxzX&1d>Zoua zMipik6J1Ek=@X9}vYJh}T;wfm*`CWdHH80$NP6#UO3WF!+iL9K$yU(J-L@%|9&=Wy zTE3Xf;Rh~x8~G#WpGDNFmM_kU{}?^*VKQ(~y^5$alUM{!N8V|>K0r3iO`n7gFIf&$ zA#2xd?Z*v($YWRdUlpUHXM(-#=Lh%=I`>NXifNzY1%=^pz|QPC2V!AT?7Ibk^X_JJ zG6!+?X`2YL@{|U_zO4+F#1mU%ml7^h2+JyT#`2i#Q3O5e|z=<42^f zMfY(l_8~wX;V;@MWknoQd!3kWlZ>?u8Q|<{$p$2XAap4@dvn&^q_@6{=YdXV#@oY| zC^qT>0~>paw>=-*v^neQwgJ51?xyEXd%>%pt1osa=^3G}BTL^kRwG$F-FmBLkz9V8 zZY4JD+F5{|?0+~RXuC69QRjL#KsHaCOxI={g@-#2CQseNZ{Le73aX^{J!qbc{o#_m z0Sy0s%SO#41T07Fl7SVjR7v)zzbgDE;f_!{k_Ef z(L$)TglEI%$nXBY`rtYFiamN+2IY3>w$~SH#)|Q^YZMcGwgEnc(kpF&6brLdwq>qQ zk78Lhb@zFTuArc{#-B_%jZIA($rt+BYP3{!Zp4IyB1K8YBr$XF_2lZyNnkEjY<T!N30(=d6Lm=VhB?i2 z+Je_sCfi=#iPbVNL+X+G4uInzWPg}*ntwtz)wQeHed7PrI;<$Q)_l98GZw-0*{`v= z5x$Z+tGg6F_sCGLaq%h1(?;F3+XHn_@LtCRy4ik zsUt9o%fk+`ma~12BIpNH_e1f_BMG9Cpt;Fk7B&g1(U2p=NL5cPgah`}1N%N%YWEnx z1o{kS$RTYJp1!0Yu-n74HZC^VnlM<$f&4|N7oiZI`v7UVW&C^l1lUa@A?hDd?TP!{ z7bBo`3|-x9m;lXEEsJvO{i5B#3y}YI0bH;R{{wI!j_yTh1K9MMb93)p{NxIc%y!73 zV)D^v!DFbXm1P)382iMDvwW{<+C0<5%pet9=3f-%-9U8OkT<}XblZMF-h$#k3y5JH ze=goHXt(_XusOhtLt+E9jBtdnhWDy=4M0@!`esBpVvwSwuU*OanyI}XD6-zlfQ?e2 zx^y+fddkj1sl?n9mETjqS-_1{6F`)7!DRIDz&G%We*ugI9?=@+yEe=PVG&qeey0y# z$q*f8v1iZqt*EP<8lLvv$rSj_UjEc*)ZmT(HcPV5H(C3`00{;#=q^}TRW+s2#IajY zyhaL!!yt%l8^A)>`U^Z_#f1(=i!5G+?tFsbbYV{F(X5U#&wEY7@RwjYC_?YY){Xm$ zOW*IHQD((Yki~>nF1Gt6<9oGB4}>c2yS;&rw1tx`1(0A}LI4W`ljUY#Kq?>4rumEI zr_53e7sV@qe>0v1A{DDX6`9sz!obYuCZfiNE<|_e=@Zl%t>wYl+5SofcGGnaMkqkf z!B>aK(2RCKO179Q6>C1LGFeB8BMB8p#^rL?%0B!#XsGktnE@ZpAg#9sC(Xaxtc*fN zwQw#-jZ~?-w{Zj!0;os*>1E(TaYghJ(9#!cYQINK%p@WMXxIJrpCp@aHz5VaZdDik zxqP1IQu5?A^6#gkru3Z!LHDZfT!*y3f0k;13VO(gI3&y_-p=I;i99T^vheFrJJ7^( zxUZsJCd`~3W3~eWgfT3FV^<_h(-;)lkaUh-OGzMBs}Y#j9j6-%WrWtFe(S`6jFX=l0*z)xNdu04>nM7 zD@ugJKQwWMDw74BGmlg%8q)kEWrY~H(Ohi5nk@eYvf6SB^KgLgt=$$${JEPg)D2Pr z>%xL{REq}N=5@I>xWQlx)i~&jTe5Dmzf)U!KZF{nIV3mcs3xstI9{V|H>R|2K~)@dbBq%7#x8>t=y5%HA%VUDo*i0O5Oj*F zoo}m0de5^=w~0pi>D%=bjssKr_9pfZo?clG51=`moiTCWg1H>1jIjgfL|tmIBCM&; zKY(Tg2%nwS8fWA?Wj2tfDzHmjA~f6al{ft8-yi$lzo7emKR6RZ|5UmimbUGzRRBwq zjBeL1;s}DSq77`v?k*jz$ZvT#k@xQRa;EGzX^GoG+tISa#%o9z8IV8pt!@nG#Dr6& zjTI`BVPq7}GuSz?K8nd%-cf0AoF(}|e=a<;ZWn%S@o8Fiq_TGA;0O>bu_K1)u_^hZ5(f|K6yGhyq`rnOx zH)K%#Mrug?OcuZ_;v9ZWNY`b;j9rgEe;DM~2YP60oRw~?h?k2Bpr()k-2ivM@^r3> ze9?>}<m|ExugJbkUO}0EwJX@A<$L8 z&#oW|6kIq1UCAXHJ<$O&49pwByJZ*LI1?WyLQyrTVxQoIV$Fe`0caJOtcxA@GdHbQ ze-eUqy$V|E74lz7bfA8HGrZ5<`B|N@FK_x^55s<0K`ON*xTcf>6J1!DtuC3|>qrhR zg^9!OJ{~s}Y2<{SkRWQWK|FatE_asYHPr`$R63)DmrU?cM>zuN6d(?}OEWuGScHd@ z-0{MifaApTCd*lRtXLg1Uyi#bGzKI&np)bOUqD&uh)h@Z7Nv4L`7qB7%{Lb5d?IE6p>Y9~ zq4YW<)xiam%f0-uCro$Z;-I?s( zgoP~U2qGM_!t}fI^C=69oUc5Wt%~z$es-TrHC6uq>#<5%FH+^=?zq6Et~Zwx7wcl$ zof%`!CV$Axli&W~6~ARy!QsdVdFS$LtJ*v6weuHxqUgZ4wsfG|Rje4ws}@U90Rwf% z8_AGZsEEmS=QcU_y&jtq&|_w{_~Jej^0|Wx3@kNIYCpFKN`ZI@h5YSyJZLNlW<3-U zUV@QwcZ-xz&|Pkc9$DgxD`2NuuNFD%zh;s&8<#hX_SuQ#+z(%b>r9x>#j%i8ajkIJSyD?hZ;C+p0Go+te z%iztl3){E1!tK3~GDbb*Nn*35vN+kEh?4gfDP<3|xG{;s+{CmiQYA$V@b)OM-L1>c zyB~d;J=)ZGr8wnZ^A_|9hS!UNj9=lHp!bdze(@qwJXYIxiEf;3XC3J0zU)9Hjtq?# zsm!ySqxqRRb0Dr0Q2f%1Lq%wHa66yuDGLg(>er&#K<lOPGsU+)XApOydWy8m6jNu5TsC8b(Uv3m44r$s}0 zVWPZ3XbCO-zNAhc!|BQn*Y(zD;B z*3zPdK-_i*Iw`o1ITYOvZ?bLVmg(gIfV_DQQbv!n1aIh7G9a&#IkGP z&f4W}&SW0Wi5ce!3Z>+Q#jx%RW0#l8P}gF1X2SxYC36pMq+MZ)OS_ORE5zdL72?)O zu7_t}wG-@%N9u%q3$tEU^PRiQ1p&E>-&tk}E19?uY0(B=>|b7mHRycI++eh{!5k&b z1rAe+c(eDMHNnFS!@hYthY^o(PV?E`C}yC!?KfLH{bZk`)s!3?eEr)A^FKFCx{67= zGjn@*ntb@dp33Ragyy7n!R?f6MF|!3d#1BG5RIU~4GZg6w~rSV8o1Atqo2H}Q8*mU z>CI-%)YU8O9U$kIAK)6()l)V1U2V&K8u-2LM1#_hD{w&>54Bi%y9}dM*X7w27=QHU zNlB^0gPTTBYhH~prsV9&LPNq^~?oF5$Fwpa<)+Rx2YW=aO%FDdP7Lv*BpB}z&! z^q6aaigZ2Sd|IcxHJQ~t&?Jc*q??C%XNVeEM@6GfJ~Nz@&cjt|;z~;W6Xpl#JndAh zO=MJ*vMsEOq7SWSAE-!B=~@z-O(MeWUg<_n9w;5l4Mu-LDGz2nD6k)c3JTa+t|QDS@U8-|_ZVV&2IVspK6z)7@{&i=ODvfIw~#^2lejp$%_7wqRqWM*k@ zWw&aXuIJsPKWW=nC0_3TiVN8fLVBsNJpA2T87{MM-&|i|>IQU&>Q!Qp34Xd%`M!#V zZ87T%lhFX5bMx~_pwnSRc^&q1Mn+lU81<#Xq(i*bPJ2;cL}$f%3^uv7C>0w|p>mLJ zFea{Ye!XQA%(?b@WuB&_bUUfDbf8Y*cyzIcu}1J9x&atZ4&&ja#6&feRAaOm!s2b$V5iUS_FvUau!|2cBTy zih<_wS4U12y78zUb}1ct_E3OO z8Z;@bwh*bzGH(Uke^U%k2m_6S;3L52Mu8h}a9)jWCSp4`zZ@F4EF^)QMnBV766KX#937fcrFTBBY)cSD2 zI++i?bJ3=n7nBH}dqeEfhBBs{KrhWE7y%8wLwGNh$%YOJ>8xbi$=XgYT9ydd@f9Cd zcq)p;km`j@@wS-VVntxR4_Bk;sJ}{PA0TgchgEk`%=LutD zI|oinb58XYUBgF5W>Eb>0=S++gfS&>$%@PWxS>aB*sIFwcd3%QJcn-&u$b1ix5{=;&j0Hm4&D_mMtR`lLh_4UY_ zpdxJ^|Eg=AohtI(Lm3~aqa)`>VC05)8jIX?akbC_aQv@FDy}(TD zJtER`N0?4BP69N!M-kytsdpkcdOHJZmBK^dZz;S{cQhB_)s#G$UQq6@C|mtEKp=j5 zD0XEbgk%o?iMFgJ)#sP8D*WHxaNOAoQhokl{Pvcuy|tz5)XT;wh+~cxPBd?QjMyTR z;YGa3vstS8YP_P5ReX|YbU$(6(p&g5prp1VKF+K8nWLsOm8Ae}GnxUZ3OSW*lPi-~L-b^cbu);@>k*mmesNvgrN(tFa+~^lUEEz)wrLH9?QIYhB3$EFJc33pL zH6{LR$FjRkyYpBk=ctyV_`C#<*~D{<=9iPVF0;^M;~${3Vt77H%nZ0U@Pe39qIhU6 ze@Q0ygbQx2dJIChXNE2xieR=BrFWYj?XmP+Qvg_vK^#t5O-S8G*#<7`4L|gDilwHT zY*DE5neWp0J#Zc9(9B}6VFSAFgf~T?Xj_?shMGqwxfgrHEhWXALu7jx1gow8cs+lg z`*q9XLvIa>PvQIGK6{5<*3L^{S>hH$P|wGzWeEKE4r9tfs7YbR^~|sY9>**(*#QSW z^u3S2D9%MwiJy<6jDW!bnRL<>o?N+ykm3umVa=7|*H83c)^0P6`V3E!4}n}8fqpqA zkJ}nNWjmRXsc}8M3)r0TGdwVG$z=DxiqKbO zIF&3ueVagOdYZM9b~ycP{&snu>5zb^cSd{HhU*XGOb)L*@P$M zUa2{>c*apA{X>wF=Qsq_ZyAYJfp|Ku5Te=R)xVqY3c+?b5!I3GJ@*LmbpS@s0te$pLi=t8*=1d$k3g8} z)Jyp_GR18WesmY0hVp=g%ZOpV7F43nSsutBOUZ+~*C7U--$VO_qN^3^bR0V(nT2jb z7XIZDDO9}wcQ8?=fg6MsvWO1IP4%#p9w1s6bnuLb_T( z2Ho>A`PiNoAYimN+fHZnbXLPYL8q}~#%$K>WE<}eshy#8e&@{bG=b-l;LH9X4Tfim zG9qoSw4d~v^~jM!L9S+g@v(X^*6L zjL?PcJDe_+odz4C_>E+YS&_p-dYQ!FjW#9vjt+?Rfsd?oxYuj|j|w8Op%-FU$m2~r zc~CB8#6H(~en&T|Va%t3#&C zdx1jdS&)OBL2z9kE>{`#ie{<%hv-X{c9z6l|J~X5Cn&SApNHG#@FQ9ybvcQ;j#i#e zC}M}~2=P#VchN(@JE^XnCoTjT9b7#>{bnT|5t3;{++dMt2&4Csq{E+66>>D5(-w~R zT$^mBmqkPNK&Pnr<>5g3A~gEhW-%1K{uvRjzCQ+BnE`$Kd* z#d@x;@drvPZTQ%YQ*Jf!Z`B8S*Vhr=XZCJ=kEqRk^$xlR>201+puDT3bgSE_lS4+$ORP>RO6ERXg<%QiP3I6NqXy@S zDE*w0t?W?_un)Zh6*ZRDrd{6g1;<&J?c%uWe*;BBmMb{pxHHu^ac&(tv)uzr^gE*; z)JHkIi-niEN74o??=S7cZ+Wu@7ZhcMc*cQ8OG-Vk*_-~o1L1PRe!E_NypvFZ(~0l9 zzf7*}&wfRmpPc=vlLz#-pm&$aLoY@Z(l}hs;Qe&*+SSVUq=p`f zNS9s`I!JHQdqO94A`nQV1UNhX&v$dK&J7pgA`hFj=UQvdF@EFXTjAKs?+s1o`vu86 zWsjxh_EVERSJ6oAj9KKCzV{B=u%NK;WZ|UZ78e;EX}>b1k&c*c0_WIIpM^mxY20L5 z!Lr13nUIHAAoJPw7W_&mdek#KzW z`X_%^z^*y-;%bY7|H1J(;heMU_4FI>euYmVRla2gHC76S+C}k<|5g1)I zqwobxoXA}D1$rOgx4JtoY;|(<QZ$&RE3%^`J&Ee-liyKE^pBBS{wBg^q^s78G^QvM z_V^9E^xxWU62|R16Xo&X)T-7k{f}x_50mYyU8 z<7lEZR?0bfoe;awU8M{Ji+^ti%b8ja5~{=%r&*i(*(sY&Q<8OxTMp0LPgqitQ1p2s z${k?At_$6WQRy_eoiLT#WUaaxR_EJb8}7fM!vHtY7mQ2Gplp)%tv^SHH={R7F3n5Z zvWp~Z`w`zGNXe!sB7sc)S{m-Lho zmF-^vHa8sjC$DpI@9b?s%6x!g@RsIewk~RcNO3n~gCR7d3|3 z4FfMuU!Mndz2fm1u-;qJW*EIe@14@Uti7k3+UvvJ`JUZ9nagfx08zBzw6VeWcb+}9 zMQBfH%h=bFGxRT3AF$|>wrdXJHyelg_rN~0mxu{%HSyx-J%S&YtEKIP7s~L3hAI+o zg$k$!SW&#hwf4VYixSqEBbhBDD;HzR6Q)Ou>xyauH=kd<34B z%xl0jUkO*$1h2e$rNFD<-y5g(!SgOH)!aqIxCq=uzEeUsYW{$e;HAnpNb9US%eVi5 zT0#meb7ubc&gIFlX7yDDJ%{3=xw;a*haRzJrVncZoR&9M{2=8;LYTnDzVC*$wJw^L z9cay;W4@EU3c{m>zIkb{bk2iidsNw%l?e0nRVtP2cIUqe`%`MdyP7xpW^C7uDP?ir zIAA@V=v%KPX51mmc+r$=&a5Vl{5M@+`M*Y!g$$#i{|JuINy z&iyKo$HP{(fjN*vKmD~+B#Xp{HNQy25{Eb1Acn?b~SYF3J zi#;7S%q}G+NAQ`9AZ!~tPmFLv7_;s4F5};gDuM9Ps8*X_mCS9Ft$h$_e~M@~{-}#8 zGM#_@DYS!jT=Ha)UA5X@a&F(m_WV~pD>7^JYUa}c>z*oBd%wg$9nHm`{`kDFCfaa-oDD>S52L)sve5T3`oSdap~b^+ zuKlBD`kxM;9~H0HItHu>ehA$^j2PIJ&0TZ$Q;ZsKNnd|L;~1GI%dq#RsdUdIHZpdx z^Cu8f=d+X4HoU>ws-`gYlxvpY*Dih_9-AQ>peT+Z-*qq`Uc3dh63{cqjVvvArpdCdz5*tNHkJdul|l z9sd`E35@FHE5pfwUG=Hz^Y)ozIzbSE74y((vmg)kn7;2DPk&hn%Ezwpl+eK{lV6C>P+l# zHIvcT5@Wg)G6FIS%)zmu;@1zE)NIFXBu_*Zi!|78(w7jW_0z;MbCmQCPjPdfqEhLT z@V7nGugDG{t49{BS9Hsq%pJg!cf8?qY4(G-oKu4q-79pWe)ugf($@k$HCVD#^uV`#U%VU`!gAy+Sus|?ZD5vJ4!g=n&rpo2>V7-*a^QpR z^i9rTDRXw)-Zy>YLAL&_;dBRN#ox)@NWiICXoP%*q&|CBYN}+mofzBovZbD8E_`>* zcC+qyi|Ux{>bK!<^7^)Qoe zwhAhK@OQ$L4OPaDR`~QL-=Axo%&6{@4cvsVFDoL8OJ3fssqd+P;U&GnHufJLGnBB@|R*$-G_?E@HsJ(!v?dAXa&9qT_}4cuMhBxAB*W{mo6U3F&C(Nj4)W zO=*Ittot$BmjM2Tg&WGVEt4%}ZjV}qExr#f-0~B&S2^W1rZ&k)g5=Tn+yzW~^14x# z4F)g$q(j$@HZ}v6Hv$fjL~-j%6q6-8I(qOhdA)TCWAo#upMLpx=^Ek}xEY`1Va=#* zPv0b8Z+^Kqd*pS3Geu6c+eS>e8;qdqtv0_nYDm;!y9R}VuWBdccsJNWkmDOwGKI@V zt->k;S+-8owoB)Ldax3=1tjOX=vtvHX8yD3;IAkenbwCHTY_h`JJfCn=S~m&m@&F9 z<~Chox+CciYi@FQehYCJPVZwuh-9SOCC+=rnlPuT9}5*0?8TpTeAJp3b6Pg(7@6}v zF^2Ax|CHFsE4o@&j(^@JrZUfbdLOfPthK@?BClrnU8 zp|JRS%+E-pbbP&bFB@men9pV!bX~fK?|!rRuMD)hprGS1QQA(|9JQh9GN1^rQEj5iCD z_1vX&raW!6!XVsXv85(eEU+je+?UcE+GASj?gbXN%~IKz{aD_Usk`>5fFA1l@MzSE zFM|Zh=z@iW@1;|hOidX76rm8@58Gm7%$F?4&x}_{0PXO~iF(54y4D;<;CbnP@pmog zG|kta=Q@l8mU{gw0K*hiR!qu;NE)i=uCgZ&+E_{hV|#2q2Mi{v2dQ01wRpPcWPa%g zO=CSSZLBXObQS+fM8)-9nOI>u)%%m^lW0WHl(y@%#Ag1pI$66bK?@?cJC|wYZ_s_E z@{_pO6ms`o)PBS771?LU1$pUc23bd8^-IFZzaR!u5SM&kgMsO>3_cKBeaEN{__1hG zH*IsCl4Q~ogF~nj52IavM#)hj9Vt8brM81=-&HPK{12MOl66{~V~dURW_)yom9o^% zzVg8EJNyW$=Z(eV1kZN~(L}bHH(cL2d83ov@*bg6#}6o-IP9uyFp%bF{)*^#OUeH! zhcrDrWi}+;{3csZ)Y;Cj*vR)f-fveb{I)Tf!taEYWih|!Y0hQBSmUO+eo@?23`Y31 z8PEoLr5wrTKqqU}=Alw5t7|h;>mz4mpl5cEtLUk5$8&e^75b{v02Q`$Ni{QNqmeh< z8~H-nw1Y7R>omr5-70>p$Hm&ZPZI6Kh9jN|VR?<2{0^19N_jpU#?97VXbu^tP?e3A z&hjgeCNBdkX(JxFe39rZ{jgSket=704UK-x`q=3(ZxzF@;3@KQg0ywC z8Nkh$J@B32bH)3S$FPc<9M?#^cw7}o(8v=!@B7NjQoZz0-dDhAMRI4)_c$zaXJh4c zWWj(=jVs>whv4GyMk#!o?q)$z7(8`*t7M=2DmTx4M zMDiJ}6s;9fCwM(QRXMK_9lROYP!4@KKh6EvPwm`A)V!DQ>ZBCrqu>=`_k zoCoaUd8V-nRi>MsBT8K(u^{yFF8mf7U5X3mrx7K70ry!Y zE9UC-QqYIZi-CDig%&?meV!_v(EVK^&)@HB{fR6@8w<2ETEcS?r$cG_LJ5D<(1RT%3I9K1wF- zNBPjSAnK4JUvdB>-}4lWR!p<&J0UGTle`hl(uLesBNz`PoqHB6;u-G$dMyRzWztJg zpr8MOw#!cu_2SU!!JvnfoLyrgE~JTcT34PBU_S-O3iRzjFg)GO?HccQ-725|sn` z7F=Lc>X#ZD4tY9o@;dP=Ho_s}9lrLS|Lsjg-os5l>i8$iI%j|6qHS((rZjJx3)*mH zxzB1P$@YBbWZ=?BD&$eo5lo$|yQ!;HeXxu8jDsw-#2_hIX0`Z|v68Wo4AWFH<4M`2 z9|^y1Fy;8XH_J16xc%|=on8Qt?%pTw6iL%3 z<`InE|20z*Uiw}iYd*+Ys$bVBl$Fj3Bn6FmZ@B!3L__$DJ6;kwBGa*@(kofYPrd2n z(*q4Nl~666k=cLoH{w4 zu&n?7oL`;uszN8{`m0S7i%Mq;j+u! zQ#CIHF#Q20GrOIuMi!mPQs)A zN9~^!n#@}sC{xB2Jb&`ahn{+rf~^AaC@mvu>cm2l-)TuYLmsR5gCNOf5D&a14crF1 zTJ{|gJwK03;uNO1bw@@ECJqQbgkMIo(JJM~SZ@?LRpf81Z-<>6z$G+T^hb_Y{SEw+ zdLYQxJExX=^ZPfXiil zWzBu{c;p~8;&K+ZZp&ZMsb2SVo{h{y7jxOG0-`nXAaBO^88R%L9x8=no}YcfjZ4Oo zM!#!?G#??Fr&F6wAC30gxmBWY?d|HXE=R`3gE$!1A94H{A!Ws}=he@w2;kUKT=GPE zvbhu1(4k9zlXgF=cv;444Z=av2?+;7pwNt}A(NqOekf3HG`x1>n}KR0smE9|j2v%2 zvrYT+rcp+3+M?QLTtSmcTEB9B-sF_x^(DS(;t0t}5DX0Sl%KZGCMT0!=Q%LFmMNGaK3mT|F8E zIUhl+-h_?LkyNQ<0~_&x(?0Fh!?aU(@(o*pES#3m0mP@6`O&#W2KPhbbF3C~@fvj- z#o4Vh$oT<8f*hhC&gj{#gm&XT$0qn!shUf%PLyaDfh4^>evLj=PWzw{|EMN23?qQH z#9CzDE3~%#N-{B}@bl0_O;#-=_Fj<}Zh*w`6)bYP^d^3OEPQ0M7ihO{J>NojijKt5 zZYVR9AT1z$Be`1>v8Qn^5T+gUULB(P^FE;4b=&`DAq(BF%9DfbcPq;~IVblS?h1Cz6HH z&VlSRCiD9_a_SAt;LF3nYrhPfqMc>Px@WsNAZNi&{-_6na-tq}Pr|?2*KT_JP>KJU z(6D)!LekXGOw;Cw2VB_hs6=9#{3|^Vy#MiNh8@PF1mLlW z%clm)G4RWhk!j+9^;8v;por=jLuHS0bGa~? zLC*o_?0HZt$NM>Q)`>j`;E>r40H!Airo_0v-xJJ{TkRfCy+=w)U5jmywO@jTp`JLw z)I$Bvee{{NFD;c^id6UrW6vnDKYjqftNIR-l+)L9H2sLde1`ds=1B<1kzdoh!c&$_1S08QHE5pfh%9)A+7I%i@nZ{_;#Nl=d~ zSFGk9`QT;f`P8uQd+yoFr(_>v!C^+==Q7)*Bz`K_ETYBn1xNU1 z^Z7~he8V#5jwMBdPmk&?Z`lXlfV6)0P)yzHK-+KJYEkhJp#Rr+PS($j59H83a1N&< zl?_L9kuyO8#^6z6I&+2z_b2bfZ6C@tu$9)emHL`{DAiYNm6tlV5zpX-z)G0lB=9yRq-fD8MQjFMxVby)XUh6Fy=d^8)6f>ge(ug3qN3 z%MvQ*A%Xbh@IN2R*$EO7)s8z|!1OC^Vn8LD0KGEQoF8*H{_532YgAT?9xs_VmOn`y zcVrqWa+3oXt+Ne4@RfA~ITZyF4tRu=jhy}>e~^7NxDKnR7s2GkjH?VGt0*y9BV<<* zj(xdlafc=+-X$Z{7g+F>o(EZ6v)V1ie@G*Or|-79Hpq6GdGO|B!>X7BnG37W)jLPF zF5yP~x9b6i^C*h#;VAhV`4n zC7si^GPM1Aqh4wg`l2YgXnvQ0B9Yd*a3!$va7LI6+&9_Mmt=ebzVZK{nU(p zV(`$MLo9?RTqGe*Yybq$xm4yqAim=AJNPhurr-i~)?Kw=UQ|c3stX=xAT@y}n<}nw1Xrl_Y-LIxVmQy)1%gGqpm%Mxj(Z?}PA7UXDo8iNzQXxFEc}c8N8^px zI@bE{?STCal~`!}w{B8Q)|=40*${>&YwG)vrM@`g`XQNht?yALh9e_ZM>kHdW!yPk zm1v$mYe|9)dQDWKwcDy2RvyFf=K+j9CeY9R0Oz%OI#>OOROBx=r@Fv`5u0*zyVC0{ zdmQ&o|5(~L-D{khRO_eZZ(WqUr6Xb&Osu;B$xs+SZ>5qD6W{il5nj}nwtKKPmTEq^ ztiAe3Y4SKGbRV-qnfvs$zW-L zcHv5>D%RW%cpIj5>r&olhWBsQ3x4IZN(GNTxe_B2WS)3X`!Y@Vx7L3-D660Y4Ob0s zBuwYb^`oJ~q}{{p&uSludLdHhb-4!u<@9bpv!)7R&S&v|>NKWTQ*20@u1>ufHt5ZO zMz5#(6Pa0LlNshU0eCov;8kE!;G=(eWR30l8;oL+&pg0Aw&foLBR5L7FC#BKHdvFM zoQ{TG@>#=qOOo-*2s-#u^GSU(#sDCh&djX8l(0TG;Gcbd@`I((5>eE*Q^-NqB&TRV z&VZOIovPD`vxw1q>;%(OD3A}=f7>E;*cKqv3Y2*chZ|LSVlZ>4+OZsgaqX2J)A>;O z1Ma-QLdfIHleZJ~08bru+0fWg7F?W0a}rti=`8zE;pxe(nx_{O_NpBfjU~n`1UcD- zThiVknw;#6qNSxJKAK`mxm;5Lo^?of8c=Sb|YNCTayv(@zFrX}ZG5?V>C$E%rcJ1jiHe2G=$Z3n%gxAs` zEjk)rd29@+{;8EC!jm5b%3+gdVhayS$G;bpwTmAmE+iaJS_8d@K;rukKb#9-s;8=X zI}Z>Jzb{Yv;TBdKs&aQ)DK?YkH~NgvoNQ#kzoqR@*T|j`9{4p$_yhfCK6PaO(oSdl z43U8e%5dz5ZPxrwmv1T-#+EcB0penCKd9?GS2nb4T zDiFeKD_^`oculo|CK$|xLe^!9z1UMt!<_)cGnW;9wKJFseO5M_XAP0TRec&X$Vl_y zOQ}E<8%giGTv`z!<(SVyZX4)e;0A&O(^qRFXOjfXc*C!3?bEmRd9Cj-2@464`se_Z zBTz*Bd#heRc`vHkYI2b)*#2#ka|3SxIbPp0F)`slk~OijVz7=`1gw@mTafYMOnIvw zq`ja{-J!G|d?j;$TUIilV%K`OjYl;Jo+nfW6i(o1#3{hKf#jiygmEk~+-vAoY~x|n zu(&aIEpp(e`zs4Jv4@R`iHGI8f@xS~<=$w?-ZLpoWU&7l=@w=oWb<8bna%l#ggHCc zYn{-g>O&0!8cLy1CPhr~z+8d~ z?*e>~rbfzRat6@;Fko2t^4K;f{9Dd5{f5cLKdvK91{-=nYS^Hhu+fUN;N$b*rS+bg zUaIA?Xg}r^%z@*gvhk9cmV8$m z#b%1eO#AdCZBcAWw;vELgqaG*fwCei_&xs{hK)F93I8VU4wb6?PgFaDEtdez@9?QFt%%r$JYfzQa+@8`Q=XdLhqrNam%lR;QKMEz{$zT$}G70xkZ$EL*h6_JOnYC z?9G7j~pUM64ICk#_12=I@>(0hC0jJOORP>C+P3*0&1~;MV z#)CSH9Q=$)*q;bXG!#4wZcomqVzSoK-CdvF{Am*Y#f52GUNF6Co4>zZuW8KF6JOzt z?_2)DS^idb&5nxF^R8;jIolQEBP_S-HRHDOy{A5@Hp;3i^c_$YhZ19aD9i6RXD0l) zU6)7uR2xQs*LJ+7ULfp^yn%}JanSSU0ibKS&|BrPjQmu!RZ9f+6C-n1MrYmDyYe=v+cpN?e6C~ zdKoMbBlrO}qG&yV&&t=2mD}weV>*pQlWYrp&+8Off`LJPgO}fZ*Hgp){a1T-0^mz4 zukAD}<}>&N;tw$?ZAA2>C^(ZDzsnRv~ z&fm-y9fgcEMkh!20Gxq-5x2QzpqISQ?W@dV(_8Z3)CTP?w-CbX_uP5TI4VW^NlQ+t zn%=s*80SyT#lJ3Bs6|3(6^@x0BV_k7{k*P(Ch|D(@QLCR7O-K)sePJCf^2?X_?%@O#oWB<1aRA4?leP+=r=F5@xj@?~qW$@x8CDU=s=I(*NQ7AhM*q zy1x)M&^Q)1p85n6_fcia{`M5~zx}jV=1&=az)A|M{f|8IMZ!v_n(u8-Hj{EAW7<#; zn#-qkSh2coz}3>@so}f76(h3a;MD{3I|DymI`SB2qeu^To~Q0JpKuP0GQ`7do&4`^ z%XUj2X(;fqI(Hhi10a6%iHWU`Grge<^s{Hn`*ZeW)vgerq01y40@deaPxz)Uq}u9t z+9$qzA`N=HN8D=}L#-S~_VeVGG;Vo)4a@m0Q0B~OApQ3yuWZE2q3AZ?1%Pyn4G9m? z^}zq!;TOcdH!v#?_brb|=jOoNw0xzFR1aS*y^cV4j*;}PCVNE^cLo>4P72xlqKtdf=E0yHO&L=N}`1qZwCbeKmdp<(ztRlHD#6T{-%MIhGGMn!QN| zJ@no1*I3yZU7v^?Mn?q!FV~$0;_nL=1f5i#KYO2lC#vF#t%?^kl~Q3lqWS0iUL3ul zwD_0}`fpfFSG8gZhE=1^Mm8wC$QrS~yG&#otl=%j+oYaCrTGP*O4u>s1)9Z_q*LW3Z|v*L(|^p~x)6$ePXa0dF!H#4-%8EbUXyZiYzTqG2p4h<_Z zGFk$TNT}zZ9EO=w4u%$Td)d_Rzknk5zme2IYF^Vy$Q5li!!9S;*M^zl=~K@XW=9~X z<3S>nc}Wm|^k~vPQrp{PGLgTUxsb8IHgn{pcQOwc^$G{la>g}R*EUMV&Gx6drKLiCn0TfT-)W>5p8q>m3ZJPT(Y_8mnlc>QKGmJ?dLm{#Z& z!uE^EZX2`zVN{zbFf0B+;X2B+VK-QC=CE4j*pF!8@3!C9OzL}l3M9l%t^$W<(keLy zB<5Ka*9Varq7Sh`j#L!Hn~o2PI);P%vCcp|Z1(gjVu2j;g9eTNYkn9@sXw|gGJ!Qe zA@^sr8T=XI9#evU@9}sq>BxqDr#nTx&& z5`{ByaAMcfa>0rpO0q|H`NdR$jc(AHmU=US-U{4uDiAdXAn$T?lqqd^`L@mDyfoM+5aP)D{Rt*u?T zhlS2h=<#Tz-}=p=RKB?zbyZ(gjx*7SsnhlL1A*BAo51yDO_22R7N|ywjf4 zxeDC8Cef59Hs>$8SYP@0c9#+Rt@5%!O>3c|OrDNBmB8)Ei~sLoEv`+Goo% zzA~@=>l7GI{iDASlTPa}%@yb2&4=OrI?c!9{bag91hW^DV^k@Rf;>0HV8*4G#TPZ* z{ZC;RB77~WbT$^@p!4=kN*DwtiS;zif5CaUR?=U}t&5?eY@)!^~BjFctov zozM=z#DX1glLNoR$w@t!_WGp}qc$bY^S2uw+xIY@_;x=@WyJ^?2xtunQx?!WaOWGYX8uz*XZxDZDc?4thaCPFcIp1unM&>s7>N}S?}7D{&4ruzNPN30lq!E(e2)wMRD zv4=d0`9q$u(6d}+t+pcj6wrea*K06WQCMdcA$YYuN zHz2uzOa#DHs|fbl^_Tx{6A^zS-5avx;_HAm`zTtvULJtN*#S{js(uDo<@)RNE4)dvL0VWS0QZAe{JQSR*h=doCkRW?q%m?#?}Mo^Ru1vx$V|0 zIHYQaj0Gw7Z{^eX&pNH3S&dC>FdHpbZ59iVIn@2@;jZCw@p2W2891V8v)TR|5qkMx zgTRI>>~NOftU&o>kL63+RqAp9%em089K}=d+bgPfZ7l=6Q5x|id4Iy}H88#RD~*m) z;B2c=MLnja|AsD!oGpe4PbGt9(BD4^2_GKH9Bu*eA+T1fkt6c5fQq5N{sEe|JK|Cj z*HR?CX~q`5+M8P6J!L%KWrQu{v|ZR3{iOYw{wq1j8BIC=-O544N@DuX;7??OL`@65 zxNS@q_QN4={tOr548N1)LTZPQ+E)SSn)%6^dZE49l$R3%Gm~4K`cGR4zYbqGKQ2AD zhJOYKm3uI}ZLsn8d2NJ++?Fnayry6yR+k4~p8%O=JND_jL3~wO1ak6M@eHWx&JA(d$!B04S`l#?a`yZUFm z2?h{-a?n;4LKV}|?!PbLUx~bk)9{|xdL!YiL`Yqe%`0|8xagm1>)5+hB49=UdCOj7 zcJ7te-E5fOx)PVQ=Zui$Ai2h2M#bEYx(r|oAN5FDfE}E}Xt3v8S=YWkY#}?!3#N)U^r>*>+al z{np&|&TF#eaUG?}?B5Bg-^xAW87u$n&AEN^9nu9T=N|Mb5MYA|q!$o9%noJ9`CAcN zZA`Q7C?5@`+!*<+PyUlWm<|grrBVuvd-fYUfgR1G&mdOiAzNP0kxK=P z{|oay{sAh1sFdPk@=G*N0ybw=eJS}asexfLS%Bv^BMS@y?cl@IvI`g0GkS`&h-zvZ zA|Ld}QHn4pTh(tSZ~B7Q%nkVLgb zZid55fbX=Q1rtZ;rofCkHLc9zhu{m~TfG}p*j$N5tI9oK%Gb9#ieawml3jhG9vc4= z4ujLcuZrKNRjBinJUtl2Fi}Z>?QGRR1Cq<%`UOd?g0@B4y@E9wcHh+6KZ@M1ZU-du zO)BumVkLuQr-JyNif>E_*YnD&sgfzzQY&PHpB1uBEp*>lXDEFF^nYkJc|1#e|L2=9 z<7;;2cC1K+#9stO`hr_~fS6X=Ui-T}8Ijxuq}pt{^2ZY9n|b2Dls_XBc3xU$xfqfc z70Hkkoks_(1RXs@4L~8AI;iNNzlbUFpk?%3SJLNNv|?f}`h31wdS;FIqHjK7k#rOD zF!%f!uutcq#&JL+-v-7z)88!K*@gb}8MTwE=&;ken~H? zQiVAA7%g;Gr7NO|vF@V4xdD_Oal<~?KM z9c=Q(d%DjE$m3<_pl<~;Q@w;ghreOE2RCqJGK966P5ecwtNA|MV?0}6)SIIQD0GNC zh0_h#$`s`(Fwuaq--pc$l}psh?f(`*M1WarAqX+a6xQUM`qRga^Cyks7t7bie*HAi zZo%nDJgBZSzx11d9}OM`s~swk+7xEM)Y`4q8W_cl(NQz zAx?mUakFjV68aA!7<^QQ}&mv{=_$h8l#X$FdGOXTAGgOpgwErT|VkH}yIds#W zz42@AyV`51alJxu7GWAk2@9e{PYjR|4KBAdQ5(M#gPx@M&OPD4ynmNQ>CgEbGli|I z7$#)k*|%fMjL-21s#gq%Tv8`7CyFU?s>u|%8lov~rxxSU;a09aKMUJTEEn+cOcZfF zX2%nB+vv927-qDsQu$vDk0Fd=qFr5K*#I%DkhMhh9`0E7Uj1)t%@K&uNS{|K?T_ zv9z3N_jzN8OJ)Cgf5d<8D!UDX0Xxwdpd+{2BRcIN+|wh#Oj2X<%4f-$qv*^<^B>uZ zK{~;qd6%||x~_s!T`F9R7GD@GZJQd~!i_j@r-uJoqU(v}lVo`N)}>}q1#(3YMISGg z^!}Rk^i4lOo5?eCnORL#`3v1OTMW{Fx?Nlxzc_^a3M(t=NSb%U8RHMle%qI@BgoFF zJKj&dufHk8fNEDS-zVt%L+=QGWUgwdQVJ~cBLVXoS45RQx3EfReAja(ZT;a}vS_hq z616P0a^+fW6EWn5CxWcffr+90ETW427pVQdH)73rXXM9=Vs$;450t^}K;GtwKIXig zPrm%9+W(|FWL!XfeOB!QPTT8+M{ z9*j*rjl|ccE8>;ymwFjx#?~f%O3I~w8u8l4ugnlctDm6u_OCH~+Ng;yPxN0Ye?f;t zdz0tx$gsmWUs8GDFOB1#gZAIvEa{h~AKvx&oRhZT>s9}v9hI~0zQ5IY*_${WzH>)Y zC|Tn6DxdxWC8W#tV$7hXs0VHebrTX9)P04Xkr(qZ=Bb=h;9Wc6Qb@0l5_ z=-qT*QKEKR^tzA<`km>ovju8}DeZv;4U8S*QHL?9gZKqQWIRFP#Ny|}--9u`9w1eI z3o7Fkp%47cD)|);;KF?t`Vl6mH?K@_|svx~)M46WPp`jhTk( z*rtN){MTK^;OH5dR~<%$v5$rGCZRoq1g&O?1&+q(H$js~;@!YqMGql>8iH_oKh^E!RS@+1aGdFTQ(zK$=w{ z>|}_mHgi!>;;1+48xdv`MZ#WNxEr2fgBelqNLhOjDVUek4c}YEMKs5IHC1>#{p6Ew z!;W!#R;cF1K||KxwPbFAb~Yn$Qe9Uq<=FOjAsv+ZR%Of*;KJ3dy%?qx7iK>-BO$2E zD{b3X=_aj=oTJGHLr)B_I&4J^G-8~kQGmdiZ_4DDH14j%y0P)z=3oE%JK(;ZSuk@D z?Lpz8Lwm2#?Qc0ze}#=nCiPm6nBGwY$So4B*cVo+#oDuM;g*H^w-0g2;?q^V=9h58 z<$~W?AK@_f+BVA-HlDokvb7ztUV<^!4qi?9!2YG@Q~kxc&8kFk`gCoCx_r_1McpTr zXxIyB|1)@^Vh-kyX)7d38L(#&V-UL*ko&WF>0Z|LW@hZ))j3FS!-SJ1-jynpcJtYjD7_o^umuvkq_h?ZQBOCOENXM(eCN!OY%MN8uDLh^WCVkQ+(z#tynBqx*UP(dXO{LfK)6PCFg)eB z6ng!5>vgG+a^d5$bNdtLTNZVrWv0VPv%w65hN0hiXm`9R?e+y`t(Vv%Kr&vW=|0t& z;JoT4gJ{|p?haI(%+D-1=AvO#)M=&s=iJn4RZlS+D1|Pp52*Vpv6i%P6toS^9ob(2lnHZIa8zVBwN`n=S zvi}O(oOvE4sJ?-VGx^`LHde)NO zkWZXuGww-y!yTqoy}@@>1gp~h&r`()R_s`*b^KVz$#s-Vj^L~GtekQQuOAZlcTu5e z<&n?0jW;0K%*oNUw69&OUvH0Sjk-IgyT#=h-jdvO{bIQO9%D-T`@p8ZzRV-O{C`8b zaSECJr&Qi!AFma0Qy?y?@YyfivX)&tjG^YY=j_fVOnptfb-PFKkE|7a*=d-elKAoq zmWR#JvW*Q3!+K{6d)Z-qh=9=vmS`vudW}H&}c3uI&|SenShfrkD?xQXIJ z@b5tHXQ!q=OERTFN6b2& zVvK4Lll$dB;sSF9D()a<0val$M5J=RwqDm&Uw^e$Q*rCW+^ytj9Q8be0H+c^|P1#t4`m?$; zLPd`3z!GJTaZeRUipM~xgItqoY-wVQN5@jmFrS=g4V!zqhh@jgKm{-mv}X4kQRK0C zP_l}M1zPFGmIqWUY%;N+6ql8VZ-Ba*+kfnzeta_kjscAC?36YntiUw)mQ8Y8aYCsA z&}m*_ffy$PLhLvH1k~)U)QKdh6aW7_Oy3@)wLE^z?^D(j4nC+e)wGP-+5-la4=qNt zy-0Fp;i!&H&F!?FS)&2xa?PSM274Nf^arW+k{d>VDDp*~C^zfcc+E zD%DJh>rmAtOub}CE-#-UL!rj59$9w@bnnq}TM82?U9~F~kQ(@7GQ5KkE)su>DYV7|%Bf30Vlf#VXi{?O!g4 zbO_V%N?2IuYU>b>2$9={uEha=VQbQW{0?g*jEx&(&=6?Jj+kPRUXhUOGMFm$KT=A#ji0|<9V!2)28bad|DVFnG%U$| zUE^yVS6Vr>vm!HXnITKbp~->LR+&>y?F`9O)RanvFmX<$GBGpffy`-{Ln#VMfwP7a zjyUC#q#_i>F%^d#@w~M6Ip@Rqu=xsHFW39>pPu`7KR1uiCRr2RMrgm~%@IqJxP*WU z&js?5Hk{MgfH*?aM;cLZAa_dQI2w^FD>OW)FcT(Cr?eax3*kkvRO=aUgOBtqin4y^ z+*r%&RcO;w9zkAIO08SD9Yth42GZ<((W68S4nKFJ)P_Gl(pdbW^F#E?i1}S~=BDRI z^0t-EX4_WVuwp^Umz5*`D~-3S^!1LTu8-X@jvD#V*yghByryA4+9tU@-nS;B^m*U` zG*_rRgZGGi;Bz1ScFciRs3WntGGzkuF+Pm@tKxxldN^|$_hb<@n}C@KzG37y+UxS> zz1G}uL3M;!SsV@sucp?f9|Z{vdV!&=fG%V>IE#m75Ai*q@{gWvb4!M>0XNjYMKK4% z{~(G{5>`v@{=F1A@%Vj4U*sL`PQXJy7sAHtFLjJz!6d!qLIUx`jXh+`>VMiqOvd~WH9hTn&(WCt5I#b)&ZH9TGc zxSRLHe24Q+FPfplce-N+6`~PCB>5sw5kvlC9fCK{}hQ>j#{l! z$z8rytFlqaDs8Jcvy~t9DC)k?%Px|8s08HGygC3L#|Zn^xAX_ z0NPaE2f%s=W@}d^v(LxiriyQw^~*5#_wUic_zooiHhF(R^K1QGWXRbKe1FZt2 z8&rk{B#zV@Yiti*S{f&#-;IWgtLaN(l?NwgOKqq>>?Z<7!~48Dy?g(4MHnErMESUG zj%13XsWQbiZ%jGIPQWd`na}(nAy)A8xj(|W!zbj+!jvV_NbT>Eq^!$r8Yq5gB(#_( z!yth~?aBdJ+9o>M!u8aIBnRG}U6R005ybS|aLXk6l(pGKDYz>=6uutQMvtXHU8b&P z0&xY?W%JofOQ|d|^RbK750Z4X2%W5Gk)3)W3!mSoT?cH0t$8;B#XG^FWk6rGr{1=@ z#|O`6eN`a_r==_zI##&Lfs{R}ZhcjK;V-Kh2*34{s~?;CH32r%Dkhxf4037^E}K98 z+$ikZw%S7Op;4xL^YUwp;1MSz5n=(ics==0^})fhCzd&G3IGBxU({Jbhewoq2mva_ zISqtN_y^H}`OGc(&|Uyj1IT!5z=OXSXR4l{U&oWx2iglOr}+Oh`BRUo29ys8l>Li! zppz8M5V$khldQL}MB|LUJfBeRyTUzxi`?_erpo^R0qg{Dg&!C0rst=M{6K0 zZ#osZVus|)0PDBGRMS2HFlDK|6Q;CDP7Sp0Ki15XHi3~YOpyUH?DT8m_q>_|N_`)J zhCr`>q>AkYWxxM6A&XY@es+zp$0X?L8l%up?p)S$T+kS~Gyge1P5T5(J2XHu_%cnW z$^b)_L8su!GT--hT+Ywe;EGPfCrjz``QK9AiK2jLvp`Ylo%Ul7D=|?I0=<_;#!=G6&W$z?rfOe{+wm#$k1BkkUdQWWPU5ws#7=m6AhE3|0by8 zoDFw{Cyx)tiO<+&t09cRwF9nZte7K}>?+Sus;6~N6};>3y*1lBJCq9y;BG`{NC@eq zfX~P{FVllbmj6V)($_@m+laVCh7Q+kM%M0WFi^}#4X=@du?m3rR)DZlzXyu#)3Q)^?&%>XZ>KvLKX@rFu8v+zX8!sP74v;$DYNzP zcz3?n_;mAiPT-U)-SMWbF&D@}!st9%2?RG3xWXlcf#-Dke!Mct0#dDF6(jwNF|}Es zs-ltRUE02nFJ(4)D&ehZciCW+?;Nqa)~(wdh#iP+d2y-6|0}IUZ#W_bp9bjNYfTdX zeLwk^jk|VskXA)pcVSg;#Ok}uSeMz`r#up=JuA+qDMJ%M59T;sX|SVme6+caL7l%F zta-uPvPmtryYzZON2Hp413#{=K1$k6GPl8lU4l)Q3Bh?G^xb*_&hkj6m4oRmu z6(eM=BNQjl9#);@Kr|d>wb0tCYhShytdnth^p7k*8JFm9$ECkmDVA)*vHr=&lzYmX z>4ro^e#4B*{78JkG5M1GOeKDNw6Y1z`swVC^UvIAbCcxGOJuBMbG2z|K6FKdo2s0+ zue(Y??d%`GnqA+xa5j8Kdjl@oH_=u$c@)^dS#3-eSLDT;`@cc6qFSKZbtcbD8hJf& z#eK6LN*N}oxYASwn~c7v-vZs0GLk?X!h{RsFJGJo7OP~msU`8nemFHAOlR<43bCEf zVI|xCQB2T48(E{&A0{A&`!~sbI#fT?l~#Dk_xT6+mE9Duil}(X(DjK@(^GR>78KLn zd{H-i+@=BO0D=P;J2QvY7rZBzqIG#aJa3U(O64Bot(5myi9nzyFDXq(h+h4bb=7YE zZq-?bvq4`Lc;*hHM`)H$(@Zb?YOHGTQml4mik~A_OJ@HiEFj9S-I&(g7IRPGQMDpg)z*W#hnSZF479uc#Q!T5XrX zNTu~_#D+W{*YS#;L9>7}^cG0Wd?BXT4kya4S*^!xF&fwd96ba+1zU`NGvYv-Q{t|f zkw9c0GW-u`R{znodFRCR2I_=i>c$tYsv-jq3a~(y1D-bRMR3(uz0=8!otY_V09K83 z@LgjNK;bb6fO0%CyT3cJZs9n@KcbAIM|ESO)riUi%J!l% zdWVA3ZKpp(`#J3j1ax!4mdjdYXDr4ZRrs(Qn-oQ@-`J%yd~Q`rf5LT&LG>bQGo*15 zoLD@OeW;eO+$Q=s=f+mV8_36FSdpZc&ogUQ51QM?PbO-gKvD*;ZOeYYMh@ArrInQo zp9^Tw4|mMR0(J!lc>v?L#x@y02rPT`KU~va!5=*5#b%ueAhX3Y)9nt`>Gv6+lgz(Y zPG5Nd*OrKQ?64$;gKruo9=s0Cm0JWXc0Zx0l@A>P0PoF(RuBDo<}CXuu=5aIpWQF6 z=3dRO9lebR1`2$Z()8P!+6lJNzHqZ1g$Pge9QZp`$8eEbvOYC^G?_Bcel%sMOQqS= z)-!=uGEdO$mm)H0CRLMDUgbPQIB%DNOWpwPhBctsi;pE<Rc zr3?ELcun%gi+$}15NK6rZ^(cf?kVDwxNQ>R0U$Rl_PMn-&-6wXpDxlp_@)kiKaF;P zxOFghNmE{XW8;F6ZD_ozBXD~|@{$l^40Wt-jFi%mh=iH0Yl{ps#AIS=XKZ)p3<|8Z zaXC`j<@FhW&{l*Gr`GW)Ygq_K9zzF!d0Fk$f$$NpH#DLu)Hsxcv4^SYoKDtx-AlP_u4a)5`^enpo5ypzNyufbX4cJ8*V&jnx zqy0vN*ev#eS6dyR2)k%sU5}~-Cf`g&TsYp!f@p0VKmVSor9}#3n3?N9VXq(V)sM1l za4Y#-g457mhZ+muDxb;Q=MjW@_93)qe46UkbIN$>ls#4R{`V1cQC6LOofbxYzCA=H zZl70hwe+iIhEsNjTEA)t@|Q(=*w??~!L|o<%b7 zH(MGK;b^5LCG>LnMFAY?St#?6rUkpEDr6V*$b52mu;77#Qg16cT6J9uFaRCz4Jg2| z8odzrgeuAY7U@fCevP6sKcIrt>#WGGcikCC5i<3{uLci9wGiJwjEEc*t+PVH98u>O zoZM48l2*284|BAos3k>DP+5rj9e4;{Z)RU;;pmpfgdIqOg_tCevJHcm%buta)Uehuh;Wb*j=pH*sGd#^V&To562%>J}fhBk%PQgO3+m& zjk*hbb*<295GiHhvMRj-uQ}V)KsMxoo|lxv`5dQhonHO`DsyIoexJbJq#r)t*mgMc8x_)FbzYeVgG?72$pm31kp8nk6Ld&iI)5W$ey z7YF{stX2V_)r1wbk3P_(`t-yQvP17vr+vlOGeK>=IUM#&0wxqeM ze;?#T1K@fC)%QQJ7z+v*CH;b<1A@kK;pif+Uao>RqWAHSCt|@n9R^Y!!m>VfK{66$ zcz$KXjmp2fCf|m3L|SIg1T+zt^i$5d_s~wGu9&>(dr7ts!!3}wMVCFs%J`DwC!kAe zwGa0x2H%e2I6wSq%1xu@+lx?6<%>YJFOrshF@4!ZxbR6FQn+d~-GyAGnya%1N@g{7 z98P8yjYx0;zXaLP%DkOsj{q|O)X{1r-AAX_SvkYo39zz~MMW}J+vkVqB^dG)azkp^ zfS#y!;_D`ePWYBi*BZW&Q|zM|UV<@&aE{`_hm+}r&WcKFwwYz9Li-Py-cEoKa~Nqj zm~e&J7x0MFMT5K}5`$NCke$cs8O1;uHM68K$#gZ@r}1MWpNb)WdDlZzZW^L4#lX6m oLw5op@kKJ81o@OlKXxR#xyQ~sT>pCL7vN=SYI}uz+3m@H1KcVM%>V!Z literal 0 HcmV?d00001 From a131b5ff0c8edd6005d9b4a4bc799ab4de57a111 Mon Sep 17 00:00:00 2001 From: Ken Liao Date: Mon, 17 Mar 2025 01:41:34 -0700 Subject: [PATCH 3/5] Add remaining sections --- .../design_doc.md | 58 +++++++++++++++++- .../img/img_11.png | Bin 0 -> 50108 bytes .../img/img_12.png | Bin 0 -> 84181 bytes .../img/img_13.png | Bin 0 -> 98439 bytes 4 files changed, 57 insertions(+), 1 deletion(-) create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_11.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_12.png create mode 100644 docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_13.png diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md index 8ff67be0a70..74ac1695865 100644 --- a/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md +++ b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md @@ -116,4 +116,60 @@ Similar to *`SharedDurableTopicSubscription`* the *`acknowledge(final Connection When a consumer invokes *`pullMessage`* on a *`SharedSubscription`* then the *`prefetchExtension`* of the subscription is set to the pull quantity. *`dispatchMatched`* is then triggered on the parent *`SharedNonDurableTopicSubscription`* doing a round robin on subscriptions that have space in their prefetch. This means that *`SharedSubscription`s* on other consumers may also receive the messages in pending. This is to maintain balanced message distribution across consumers. -![](img/img_10.png) \ No newline at end of file +![](img/img_10.png) + +### 4\. Shared durable subscription deletion + +When a *`TransportConnection`* stops, it invokes the *`removeConsumer`* method of *`TopicRegion`*. If the consumer matches belong to a shared subscription then it should be removed from the relevant *SharedDurableTopicSubscription*. + +![](img/img_11.png) + +A durable subscription will continue to accumulate messages until it is deleted using a *`RemoveSubscriptionInfo`* command. This will invoke *`removeSubscription`* on the *`TopicRegion`* which removes durable subscriptions from the *`durableSubscriptions`* map. If a durable subscription is still active then this will throw a *`JMSException`*. For a *`SharedDurableTopicSubscription`* it’s *`isActive`* method will check all contained *`SharedSubscription`s* to see if any of them are active. The *`SharedSubscriptions`* map should also be updated to remove the *`SharedTopicSubscription`*. The *`TopicRegion`* also calls *`deleteSubscription`* for every *`Topic`* in it’s *`destinations`* which deletes subscriptions from the *`TopicStore`*. Finally, *`removeConsumer`* is invoked on the superclass *`Abs`t`ractRegion`* which removes the consumer from the *`subscriptions`* map and calls *`removeSubscription`* on all destinations. + +![](img/img_12.png) + +### 5\. Shared non-durable subscription deletion + +A shared non-durable subscription will be deleted when the last consumer on the subscription is closed. When a *`TransportConnection`* stops, it invokes the *`removeConsumer`* method of *`TopicRegion`*. The *`SharedSubscriptions`* map should also be updated to remove the *`SharedTopicSubscription`* if it contains no more consumers. Otherwise it should just remove its consumer matching the *`consumerId`* from *`ConsumerInfo`*. *`removeConsumer`* of the superclass *`AbstractRegion`* is called which will remove the *`SharedNonDurableTopicSubscription`* from the *`subscriptions`* map and calls *`removeSubscription`* on all relevant destinations. + +![](img/img_13.png) + +### 6\. Client side methods changes + +New *`createSharedConsumer`* and *`createSharedDurableConsumer`* methods have been added to *`Session`*, *`TopicSession`* and *`JMSContext`*. + +*`Session`* has 6 implementations: + +* *`ActiveMQSession`* ← Will need to be updated with logic to create shared consumers. +* *`ActiveMQQueueSession`* ← Should continue to throw exceptions as queues don’t support shared consumers. +* *`ActiveMQTopicSession`* ← Currently doesn’t support creation of durable consumers. If durable consumers are not supported then it is unlikely that shared consumers would be too. +* *`PooledSession`* ← Currently doesn’t support creation of durable consumers. If durable consumers are not supported then it is unlikely that shared consumers would be too. +* *`InboundSessionProxy`* ← Currently doesn’t support creation of durable consumers. If durable consumers are not supported then it is unlikely that shared consumers would be too. +* *`ManagedSessionProxy`* ← Currently doesn’t support creation of durable consumers. If durable consumers are not supported then it is unlikely that shared consumers would be too. + +*`TopicSession`* extends *`Session`* and has 2 implementations not covered above: + +* *`ActiveMQTopicSession`* ← Currently doesn’t support creation of durable consumers. If durable consumers are not supported then it is unlikely that shared consumers would be too. Note this API exists for compatibility with the earliest JMS spec JMS 1.0. +* *`ActiveMQXASession`* ← Extends ActiveMQSession so will take advantage of the logic added there for shared consumers. + +*`JMSContext`* has 1 implementation not covered above: + +* *`ActiveMQContext`* ← Will need to be updated with logic to create shared consumers. + +There is little to no information in the spec about support for shared consumers in *`ConnectionConsumer`* but it is still a Jakarta Messaging 3.1 interface. New *`createSharedConnectionConsumer`* and *`createSharedDurableConnectionConsumer`* methods have been added to the jakarta *`Connection`* interface. + +*`Connection`* has 4 implementations: + +* *`ActiveMQConnection`* ← Will need to be updated with logic to create shared consumers. +* *`ActiveMQXAConnection`* ← Extends *`ActiveMQConnection`* so will take advantage of the logic added there for shared consumers. +* *`PooledConnection`* ← Will need to be updated with logic to create shared consumers. +* *`InboundConnectionProxy`* ← Currently doesn’t support creation of durable consumers. If durable consumers are not supported then it is unlikely that shared consumers would be too. +* *`ManagedConnectionProxy`* ← Currently doesn’t support creation of durable consumers. If durable consumers are not supported then it is unlikely that shared consumers would be too. + +### 7\. Openwire changes + +*`ConsumerInfo`* will need to be updated to include a new boolean *`shared`* property for determining if a consumer is shared. The current *`ConsumerInfo`* also determines durability based on *`return subscriptionName != null;`*. This will no longer hold true as the *`subscriptionName`* will now be set for non durable shared subscriptions. This means a new boolean *`durable`* property will also need to be added to *`ConsumerInfo`*. +It may be best to introduce a new v13 of openwire for Jakarta 3.1. This means users will need to use the latest version of the openwire to have the version of *`ConsumerInfoMarshaller`* which creates shared subscriptions. Older openwire marshellers will not be updated with any of the changes to populate the new fields on *`ConsumerInfo`* for shared subscriptions. +The *`isDurable`* method of *`ConsumerInfo`* will need to be updated to work with all versions of openwire. It will need the following evaluation: +*`return (!shared && subscriptionName != null) || (shared && durable);`*. +*`shared`* will default to false for openwire versions older than v13 and so *`isDurable`* will evaluate in the same way as previously, based on *`subscriptionName`*. For openwire v13 the *`shared`* and *`durable`* fields will be populated and so can be used to determine durability for shared subscriptions. diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_11.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_11.png new file mode 100644 index 0000000000000000000000000000000000000000..c3e81d27093babb50ece9d49485084a82b3dd576 GIT binary patch literal 50108 zcmY(LbzGBu`~FSD08~n(L`4`W-Ju9b3rM$gcZ^g~P*PxY*N{d!1_(%v-e5>GV5HQj zjr?8nzMtp${{HawQgqj@I6vog9Pjh^`1OsdJjtzxw=P|}M56HemByt@msfyadc;J) zPjUuU)PX-3+seqiQIL^&^v2EE%GSa1(k0dy&u=`hub5GEUyRedyeaZZ;FIw651*4> zL?(tjAaw(2%eZBD5!o@W$i506o~0NWeIQV~bYES`{q9@-u>|*d4eWtrxhSXa4(oU$v%m49x!{cgcWl&~sK*~3oW72OX1#ntCkSM@tWA}p9joH<;|5!4WyP)>aB@slN-Kd!a;qPwp_Cge&YSo_fR~Kq%P|W zsO^thfPEu>hhMrReM#Zfi?=?P|IFMxt?h(K?Q94S;itZ8JcfLFjWQS0sC_#1mFg*H z-u+7J@gJdY3OJY))K%G@D)wyY#xdVxK6%(K(jC-rv6oRWG~?lYk+1>|96C+~H<$U< zZu_r@Wp1=Ai_IlBl;J}!jxNsUzSpCJ&R2p$n3WPnl};JY(Tqe~=;71yi-XpXVxvvF z#r*{6&cpNbqgJx2k<;DiAiCkw*7M`B%)*Qbi)1y4io*4~dn!ldXa{`A9`^~9xkj=T z*D5dE2s(&mImjhn9&Q@nLR+L_3i^iv53A76gc^F|w>h^W&B`}RSJhs3}s+l z<5+2tGNQhxlV!+Z=;kb!qhdq8VcGoNiqu7eq_a@tNF)><>ucqg@RBc)gHK}Iu_^lyAK=0rGW3U1=pb= z@q(pM!?hj=Hw9b-o037lht zU|J)~nL-MF#}@oFLaiM=(#5iooTnCuU;#gs@QA~It;4H-uWR0PEOuCExZvAO1qDDa zUan3ZPq|?6)5`vkul9XI2^lb)BYrdvhO_x(l&zp|Ai1Ay#E=5H8J)0VVYHX6&`sOI zN#>|;7CkVrHAzzWiKSu|v}KH?-eh`hwWn+XF8eD-wn|_saPo2d zoQ|pE&;p0k1l5P*2o}1=dLI=8SNEM6WQw=gzjgYh*H1M>NHNyXHyrLEF0)y{aCV%H<#LqsPG=(2F2p6C_q2 zULbA|bTB(R9&bG_NvhpP67Nd857-;K>wBrBhTq8foYSz6Pzt4;o+sji^JYc}C?w!} z#28cNF3yiUk+c3bI)afS>n0b+~_j+u6onocEzPuOG23! z{SlwJhbq9K@qwlGM)q?`92mTyu$&|72Io4aNm)8XBXa}1;C=+7D z#w%9689MMk=W`dZkOY3Selt?VJ&c;akebfN|SJkCJXxhQ|JY2Dlzg2(Q^k24mc&$IQI}Gt%3TxwSbte|JJoHw_ z#u}bUM#gXG!&%P38QsH266B%a+|dxtnQ2?8jFlVuY>ms4h})0D!&MQVOt6sNAQvX! zoO)A^4H~o)I2?`QepesdW(o3y^Qz2hGYolgeAHh4WNnAEIIs3ymO;XKH!r4^eTZ5!eV@vijDmPPM}CVINXbCQuN6A8GAyUMe|w~Rz8l$8}miq!nMnOV&FJ|>1IGH|Eu#&}mx;;fN;)|26T ztH6SxV!uKJg{+qcoo%3ZI>^eaqO5WX8eob7oZPniJ3cZGmPb};@*h(qls(oD>f@}V zW7tdT{v>}yC>fxT2!~-M_6a>b+YSmcI(|nlpV(C(vo_h!%PcX^9OZPH!KX5CD+)tr z!h|+7ekW8KserZ1y7=Ehz4ur;8Ijp~w&`5V(-#&+>Ug#~g3D42G*z07ecC_HBHhA~ z-~V(Gk6rWl?-%M)-~Hm|6psz7wHGgqRA>95#i3FkskBiBoJ{?d%R`|r)*Xfv53N2O zh9Q(>6v(0@GN$$?)|2uwm*%<*GH!+nwKfqJLFe5b_x=_A)&V77JEb%e&K2<_CL6|K z!L*Or8-c*3xpe-CvCa=LhMMtd61rdY8ct=ofb{Cd5?ED}3tyTE8)o5N`|8~ae6Oz8 zHRxLJ;Y12ZTu;-;eXt`3mb>GEVXNJL&n|K`XMim#n^D4uZznZj_m@%;3XSqS; zAwF+n_3QY6bKu_b%!gVod>Su^n=8%7CeM2uc%mG#kA_}nL?42aLDTONu}c1LKJ49H zS8lIhMwwZpgY^TIraI_9bcVcN>9GIO%@}m*vOglP^BIi(OpQ9C4|udx=H0yEvhbB9 z$}GELs-{p}6OsO>`edt#Y&&4vFVtY}7!}WT9TT*sWb%;Vup*Si_yhl?tP+V1LIv43 z_|Io?=9aO#t*O@=hx?x{GD^V~>9-%$&iZb!H&cnkMd0%GUeo_=)n(E@_b?6B22<{k z6)}pDny+;&;zPwN^>?}w0~^U-w8u#{w;K}KiHp`H0GYp;s?F8?R5>06I)L)$B~y^_l24!2157- zg>%`~%l$Fz9b8R+cwtM-+=cWXUbJVP?-#(4p?4s?SBENFaSP$aoqyuVP=cK>{Br8J zwaLSYpkriaR9~UVjlJ}3e`{Q?TA&;AbxHcjj7i|0`)nD;HNH)&%t*+nh8QfE9q#yE zvON~>A34O{Uk?BKSPVZ%XOLy)d8%hk*OUirislC#PFRx-`E>EESDC*GSj`S4J!;+# z2$h$_#vB;x%%Kyu&Uyv}ryP7{_cDZF8%Em?TMkN{N&GfzbTBKYI$``PKXdl}xV1>Q ztQQubc0^Xv`2I9)x72VY9MI0;y39^C>lkN`QFWx{zGKDf;64l3)XbJ3om&snLGVj? z2Jq5P#GK!D^TwB%?`IQ$lydob`No}SpI=Yx<{k~)pgKRBy9n?Z&1KL+?o-Zq0`FBF zIq9JJiHO+9*`-u)SXq9A!C9En!olr#j~gnNAwC;**ULE+qdl)t30=t`AKY8aNy}$G z@3)q8uOAB7(kz`gJDLkJHPh@t`2_@}*7y$j&AR)*H8V8&^xOrtR740OpP^+p(s{Jd z^%knGxLHZzMM`YT#o4wACCG9e9ki~QDJp(E5?uyAWFiCi>`bpnde<+-Duw947AUo7 zG#7q+@+4iaJoKOca6_B9ylh2$MWln>1P@08HDHNxN2bnt2=Ddtg*w*FcnL3jdD7)r z`r#Z2JU+vlAGDpzUe{+RVzRx!y5t%AV_c<#1~CsOUTT=Pwu8&`zO-~qu?n0-W@wLp z2{3X#I~XrV&ZIh2DriATjVG)#144j8!Lx^~m+l-eTup(F7# zHo1`@V$*2Su@u421IIO1K*I}a(r;VI@N+6XsmeTaC_Pb7^VO*72YI@=m7Fl1?}Q%o z$*sS0-2TzcaA0~(b6GGPWPR&2^8%l_A;*u=Fs}c5inp#k^IXApCE}kv)%h|k1N6wB z%EYh0on-0cA6PYRC0WvUW)!+0_0rrp^lzo)N^f41S8EV^|7`H4UO~FoWW{2X7^<+E z-s^WZo3kbcmvXDS`~3^B{Y=S<$HVo<;rxR_`|%yrk^4*50TQ17bUG1Y!%~4JiT92$ zl44`n-%cue1uTi9pp%86Luj=vk3UkwOe>f+fJ%3)QQVw|I+&be6UhjAW22gn9Ty@5 zGn=_q+eu|GdVm&!p-INzv$h54A>OSgGh>yVQ_CET!v!9j>r|~6PG^y%k6ylMGvkbB ztMZ|7YLejJR;iBbRrV>T9gOELH9t2W7o6@^1`gb+qf+fkZ2n`Z)msHoY1JI0sBhFg zN^8ma_XJC~3}<(E)$9wdrf_MfZ4z@BU(-#dla*ttbP=)dG&Yz7ucU7{t&yt=XG6s_ z$E%OxGEbKjPZH{2`B2U2@rFUqkR#ZNglFRf1UIuCc!UNXpx=y*TeRPFuBv?17rn~a zY{9es$5P8l=0)+2GkAHaV|633+N;yWhJr~GHv?0rrlG10Q`I?KVUtAs@Wco&t6X!D zISiL~;nh1I!I}DL$gY1Io`tsE=hGaAhQ`axV*S66BgC|38`s2f1(|`zE^tsmMCt0V z3iXcyFT!Hmtg?8LSYxmSD9`ey8b>0&PL+|tC1oD&4Uft~d;5vB@wz|#wdSXXb*%vb zYAGMRmjC40TUyY2GmC$H6YYJiEKE8hKD@6u%-Hc1Sgx8kn!)FbxqUG)8Krrl<_lJ1 z3A%|nrpymTSm^TECEz00JHDR~s*I9&3E#tKe)iiP0x}It&N&GC9I2D))My*<89rLK zUm~>fqB}Gd;xwZoVsutwe37Ic zapeZFh5uIds@72MNV3Sw{dL_k(FpaCIz7^~)puFXR$ z33uGGk*)ZzJs3+==5rRh#Ohy7bA)#A;r+^-`?|>Pv`<;RxH;Bv)16Jq)iNQE+>GAu zCuP#+hhP5`z0)=Q>(8dUB&?IYoJ2lbA>`w#6C}}sYQHghk=xmIhkxLFX|NsD23zqs z)L+r<1guqCkD`@t0-PrLhz=gtvkqKOkBvt%*oX42YTEYYErNp4mAr<%gJ}qRseFue zO*bymYRRytzgow;k;bJ!s9UaUSpU5Sl=EIJgU9`qam_*tAJg0#a<$9gINRR8y5)J$ zwMlD`S^m_{X~TfZ1lF;Gy=t%_f1LtQ`LvgZ^fT&EYQFxgPo=&*-42t@Tk;?X7=joi zyl%(Be){OAQs_lQ*>e6@RsH8L*N9&^%X!xZrY7#YD5Obcps14V^~L@ziGS1};Rark zNG@DgR_{}(KZpyIUCU?UH}BK4;Oe7M(HV|=e(3XG^_lSat^`dwn}Bbe6u73ow-5S< zISQ&|R<7CbES);1nXEADR@27+Y~6ej@o=2v%LM_;V-F2$_)%NAt!l=2$-4Y&RwJsa zFW$w5Ke8yPGAtw`ZPotI$4tL>aWE@+FAH?x5&@N>xYicY63JlvIO{G+=Br&+3$(B; zzn}Mi^oX3#XY^~HcW&E+&f?{9_vd+Z7d;JAs9)q%X4z|_|2eafcB@}w$N)&Hxe zbxVo+{q+4(Zno$%Z^skL`|pQMNn{>1@=ESbuC3>FJ9j#-tyc^3O}(s^T}_$>J<9Ol z|ND~u_j_EluCb1PPSJHK$T`{m(T2 zd?)+fEuRl?X|3ACIZBID9xBnIg0ZoxzB`I(lEzp91iAl;M~zB4;$G(CZ9=8~uPTP) z6OFv|0!~14H@_k<$QsEv_hlLEX@u||&yAWaaw7$c1cpa-T93yHP*hzTrFFBTry0Nh z`}xi&X~>3(lUp3k!J0#3ZKpGgDZ-O(Lj?XHyTaBng+YaIvwtqgJ_@)xH)bSns&~&6 zq+0bbF@7lJb*9%y@+M39-gHIYn)>(>N6q0U)pOXKJ872v<3~nNg3KR#l*Ksj8&qMp zW1MK;8)R!_Z=MlZU4AeH6Kw8$#IH2@i(%_;ja4yyD<_^Tb|Yxv0Zv5Y266xAOG+^U zV4!td)gRWBOVr+9VXM>K>qbsq>IoiK)wx->+j;A)vo}!C6&{l$q;f|$=sn&9^mQ1= zmt~!_nhRcIT_$*&4iYDqV013ynGN#*#g+pGg@UaPtPT-iGak09vY5f9SzdYqeoNYX0TE z#bj6J9hOp+Oou4g4-5xyA|?XLzN+xR5B8)2celrMmYwmKgjUzT&YdcICDVE-vG zok!AYh~fyN!2M@ZO48~-O9MNZc8jKE-3Drw(?05a86oJN|FoZLA6Ah##OFm8iNJyS z(}H~3Z4`9O)>o3cm88x#{`>#DXV#PFa_=+xnR$HqY)j{sLtkJ|$sTplr_~z7{O4|F=9FBc(I;Y|q*% ztHlF|EkhDj3T{p<-e+b>Cs&)S9OaADo6I9MC`3HASyN%}H)kkj22a*ZX!MI~vG zC5!_Gp{`wnkjurX6q+TvedUhM6x=~ZO)P!kSWnT`7JcZXhp}I)k=S%bk}ubxds@T~ zw9QV2>|U5+C6~{cB?1u!OmTJn2~{8ec8?9ND?jX7ah=?xfBm>y?$bPXb&Xx?moFtP z%Z6|OMGFBM^yVUyM6;Cei@$io^eC-fssvKxH}f6 z#I{|(7)47eu|KRo2ec48`ZAf1bo`>^QNz&A>`_Y{G*>>gQFEnR>Uqog?TGxEZWB(o zX}>+OD;wfhGW@7!%bK=C|5PJYJIonRexv1t8r3&hmewFh=l*L%qWmp|uqme|lGOMi zf5}Z-HClpOxpVTTN-a|JxTR)VAA$q1m)Fq6J(EHeJnw7r%x>xuC{vFk+)FYkfIfw` zo*$HJP0U~u>gX~KN~}pT@%`3CbQ|TZ=L8uZ1-45)kIH2o0=yhLzTDq3@uY}O2DMv)6KhidNd^(pC zwxa!)o_OwHTIgJV-Rq#=_URVYG&S8LfHmPi6&)c`XKQL5PZ)N;m>tb{mv6>xG5BxV zo6u$)j~1XlsGQ$+KV!5y&MdKCBN8RDjJrxF46cn8nsCBz#e`SV$A40%nTb+lVbWg| zZ~@mVn~bhD0rXUzMs~b{x9qR&@TC4gsM1^#!QVf>z>HA_ydwhfeI`+1FKGb! zGglnRO@E;!{S6vTus}kMy(&(;Y;g1aBAsTtw;UMPIrWlpb@i59-Z2Tb(8m#Fz`cte zlsx$pv(HQX0I+7+$*NHeOv#P;HJ=t7)L(R2T+ zntu*T2ZH>rdls)l^C%+hF+F9c!G`A6b#9-wRf72VuG^K%EBzq;a?dJYLD8#Xb9hyZ zLCN08Ud9m&9^5Z=GBG44n;aeXj*&87tPxn=hG59rdDsi})7%`Gd>uewGxAp;tj%e=kngAp%28Np(|AM=f19&2Y*9LlnOIS+ zpZy9-QSAv@1m-^L-|tj(@gr@zg)BI4=S5N*d8juRSSCPgA2~)uN0pHTAdu6)Buo~A zxfPdxD^f*~tESD#%e{aCrVS&XCH~3tZNP+O;#i!oE}z^4Vw_?{@2KAzq9(a9%%z=j zD%zg;Vjd#o=)jhGT}SjNn{^|bH5U!hR4sJmy44WU_b5xh0UC2O<=l!+GLNEh3Obq6 zx!vfYJvJEZ@aH zs1^b#9IF%zcLh)<8NtMFl>8(-Og1st{H@fSh;ol|)74cuOur=SUe*l>8y{f4W>mV) zoArI)d>7VVLHGzmlZq6V3 zb@>L%Gn1UTe-&i{*$44Y6rlqeP#(_((Ln*TOUk71q$|9h3bV?s)b>ul3CICrGs#18 z9qf}dC?KHEQR%DB9}Tp=(8QMge9v{AQ0MhY560`gE9aYY=cQ~a@Gm(rQ7ev4?x|sM zh4#AYAjJM8AIvT`F=;Xk*3UL`BJdMTJTIJ8;`Q^9}Ws4@||Q0nAaJn4np z=qxG<6TG;7FtZO8Dh+;GUHD`pg-Fz=Bj!rMZxT6W&A6er7wKCb1-tWCgQ{J$NSD{C zpRwP#5uETYo6Nwu*)u;88VIK$OB6?LG>q4{Tohhpi+7iDPaFT`{DBw53U}86 zI%<6W1<&e|o~NMc#MO>i*!;)aMb}v*=&3W?7j#_#BC%#ll6W5)p|@sHMcRMAP9~KO zz}e1P{*W^&0|y=mXt&XfgkDLXhW2+L0IYQ;Sp8GmS>N3v;ZE(=t+RaS!>SYzzrt?3 z8`*twk#A(Imj~DCJUE3YS*Z!I{Dx%BU6zs28R`ie~rK*W5>6Bitf28Ra ziF%{*2Q&F*#1c!?xga9!{UKb6v7f=S;FYn+Hym%PZn!Q$mIzJUE;&(PHn6{UU5-*;@uc|-C*xE!qR z04Y2lw!CXrMI#G!QcS+$zZ1+TE`Gin%}5Y+V@8`^@2kG&Yvu`>Cm#x$_LKx;E6ht9 zSJVK6zHuqLq-tF6`c(nEAez8oJx%k{1zVDR1p`W)X!qxp+Pd4yO4M}h4NrC~JlX4P z)psg1 zP|zcI6r!M>RY5HG zfVspztZ%eIrvx5Sg*GGe)782QjV}G5;qOK14%Gv(yJ@^6xNB+Xl&(*C#nveu<|C6i+f`b)pt9QE4Yca^zY zj%er-_$j*_B)wK{m%xKP6YGpNh@L=OM8sNDl;RAUa1`Z-uXr?mBRIbP<}7nU;vswv z@a5u0suKEjCN)TvUpAd6E=k=qCe`(Q_=)KobIwVpeS!e+WHX>Siq>*Gg3(a+5ktpj zf-umv;8USd^BYZVcSJ9uZTBfZ)%;;`^l0V4C*AzuIQf7v{i5W9&mA;X@Nn`EmyUI= zP2Qa1Cj9*s7DFd15Z}XtnMukJ0IchpT@yDjT8O=kF#<9ml0O#JMNujxFDK z=GUJZu0q)P7;>4bg>{C@sN8yC@TAVy&ML`nO<^t)*#zR|a*a2In_d>4}0%h&}S-iUQj~b12Rd4i-9^&JOEJ$|dW2LrNo1}?STTrQ1sUbao?JA=A!$iHsC({ui4!Ho3}k;mV;9_cmgX}4M^Lr@E= z@*xwhvj&z!hsn=P%nU5ZR}qHki(=x+9h9E@u|vuXtw21P)37mxDkcfOEK{M;mRB}# zX} z%yJSYI13%3P)&Di$QyY3Y;BiiZE9aIf-9Xz%8dVJu5DjGMMfU^%1g!|+jYvduS3fP z4*r#{3o&B8kJsX}F5X_7f7O6RQ%vL1CT#I-&)2Y6sqb(R(>=a1Cc4DaT2pKz0n`$B zWnZZD6o{KKbWGFEG2Txrn=+loqbYYje zz29@Fb~#N1OYw(5DJp#Ky$&*USCT9;3+{uv=!F{4z{G~)qow!5F24Hp3VE{ko5ZDG zR1fI&J!YRgL`d%99E}n7%h!yI+==d9hV}K-yuYbLroy}`MezAKz@RN_5nEHfPaFR5 z+bmAx-j1wbIdua{aD4sOghheF$a4AtBwTH1xt(PVoh+QfEJ+@kotPtG5jWL@p`ky) zfluhipZ_Jn6+U-+UT1EL^+!>9*u_b{W8z#H4K8qY-tZe8nVAfI@3$s-e62_=f?#D! z6cv0SBPl30w;wwJ?`qR95*b<A39Q)2B?se`9c zRHG2tiut_wNq#xC@4F0v`Y1 zInhS4s&dW)&vRf1ZF1o0l}){jxQv}5726dw5e}11(tK9kdoo%eDIk73Lj$JX;@ne0*pr1DlUJvMz$;KM-zOt;jq-0FYy)zFnMTN$PY?;@WM{ye z;zJ|;x&Ng2iV36tPFrkI2lwx6HAAZfM+wfMd^%Lb+-|kkL(^Z6=vdNxPM7}jPoOqP zuT{QS(E^6)E`XvU0c6xk2w;>=aXs>qyj$fSjU~%oY=(@_(ytVJ)BU=(m)A$V@<&d` zSe{X-I<3=yb0k5VW(_*E+l;Cy6CjN9i(r+iPMh~lHopD11hTU~a$SEBr{Y}lClBrS zmlCXq&4Ke5{d&=CD<5=ZyA+o*4yy}DgzDQ!LY_p6yWs#VU?Q}Yzgg&v&`CT@8H_Xp z<&yejsfT)S>C!w^AX7ziN_9V(ta@r<(9cE4V0^dmDM-IRc?`1Nr`Q29hmq(twIJX8 z&pgIHRV{TuWQ~p7*dm;clN*MK0R=zsur>GhU|-m=<%a!QtOF~QCzE^VAd?I~srn#v zbSIxyJ>UF?(CTUt=P`<2obH7ZI9-R=l%o!(Y-%^_<@EBX6N10h-nh!J^S&sr$D3fk zyn%E1JA$tVV2f6P`+?(hb|{1BA3);&rEW!c$i#PkP9|r%4~mRk(b{@J<~!`t%bfW3 z7zn(i3ecD5Mu`@+JkIS9>huM29|fvasdm)RZ>-aCs-m{_fc<{Mxz;Vd%;Q-nW<`Mm ztm96jSW>0PR{f&gj76@6_oQvX-<%ViEr2Iq`Ce*N3n3RF`DBv+;&95@WD0z7G8gpQ zdoIBK;tb%N&OQK!k_hBH1QH)um($&Dve`+dIzH&g+#tCqrN4Q6i-on;z0B(t8Y}`jxjbian#MHd0C*m&~jSSKW9lz1Owse@}bFcA*4n^ zjJ#)R&P~z^1G;FeNB4DC1u8#EE$M#Qu=~V&>-tBRg*Y>8j=JqXQz}!>$EKV*L9g(Y z-&tkg1C87(0tp0YJC-5|46E1mHvvg$aKgP^)&k7iX#o30Mw05>f_E8yaYL*4jR+FG z4Ic6%f~N_n>H*}ewe;4MNEbzMi|cmtQ5o!T+I3|9&-!32<_%DYq6$fXhv-~?sv-ed{>qY}%jv0@ zfFNr=`r11bR$$to#NAwtQlRp%qKEP3nFu8@u}fM}^gK#WLlEL3q}J)4+MPt)VZ+sU z-;Gisf)SxJ29&bShF-?ySle^HA@M3zc}4m| z0%_Ze?`~*o!36?*rhnj2J7Lvf7D}%jFE9~5QG6#4KvO4Ujd`x9ERb2bbhA`l@kYs$K?~)Y3iJ~OKMT04hteK2FSk7;Dv(VDrUjdY&Y|B$2 zcqT#x9*H0(%_Zhi`8$IL-aMlBhN<(Td9U}u$MiIPpuwQrl2*m>a`iAedqlxgmBdOq zm4U9c8+u515-&q5?4VcxN^VK1amz;ownn|{d;;Hb2=E8RlzgtV@$5-!5#WVJ z93xb*E;f0xKVo7!Sio2i6ZL01QCv-Dd^D#!K@bo890xQzSqO z5Ve0qQtG9NZ6J$L0OJc~{K51LM9piZ~ssIS`1K zX@_SBM1=CL_QLh+E~_-@HgqSf`C{M5nVc?A+e<*RL6*qh(6e|M=Wk}4sQrdDqpNYO ztcu&8$W^~rLl2$7eFoas!$Wv~FgVL36ngsTu4s4NzkH6m(3`RidamOPfz-aK7VH3- zcfFiq9!_{1pzl6qz@XH$b-Iz#`4M@w^O%NY(0mHE6-K_o&%_t)@Rv5 zjSRsc5P7J{v0B(qNXdA%^7W}wheu|bE#z+V`)DsNt2pBD;Se}?p+SDdZ0`7ev?4BeVt3Xn-~c>O?GbWst=c_+4X&x zJZX(%{8G3}n@1U00t{PT$;sl>8`g5MEB8uwAJ&T-$}tP*3sObEBTLOJ4gbpgz9{Ln z61!{y@XAGqM1~2dFP&Zqy*KSM8|-m%JD|*z5CDcPqIva%%1*V_RPeZC*gImfxPU!CMk^#vQmU zxf)k<)F&JRDNhlB9v+UXaSXFx6EAn$IEW1CzCN2Da}D2aR2s1HFKh57^agq zEc`l)vFJ}b^-8ve1TL!V{n0k=blZGBPI?x4?t`X!-PY{nr9@uA;)Z&}uFh?~>RC0f zLgYSsM#)LNZs5XtcieUHH78$WDk&E!y<}VWv*P+u+K6{}=$%IGSwSP+2CD{7)h7-YUudX@S+p_81WDz;u|Z5 zxa&tiKE=R9Noae>`(4&$&@}HK^glA6?Vim^HgKgkrk2$X4dQltW}tOucy$LHJUlf| zNwjq!6%@e*AMxO@%#LD>tEIPLF|tVh%7ulVAEPGwT76^s!waj@aqK%0d+Fvxl_ zDbioig_$J%xsaY0ukqKE%%xDB1t#0YIP`GIm*|3U`!gW!vd6xQ*~cbMxT(sTjsEL6 zc<&9jX=row8Y0(~)AC!VltKEMz3S@7_n8}{o?-$XpQv`Te?X$F&1VkgJ{5aLlJY0i@8vX?j z0z{WJdkrTpy*1&)+I>5i@bI#a$TIroNqYGf!rplE_`H%If&GLDgT18cRMyjkVMk;3 zcT`~s1Nvc7;tPUhmlnjI+Wh)AKv8__GJT4g^>IIRJgHKysc3!s0EmOF3O5cH>(82* zFbEYY<$CP$l1n6KioTQ|KNMKW`fJckBVRs3PI^sOn!?DYfB=ma)BJ^e3|Dlng*%9w zPHJt}z6&G8Im5e+a-unK`5U~@)g!j(io!UE8d%$wa+SA!*O|>@j*P;@B)53@uqzGj zM8ek9gu%NQ&hpzs_?@Jii4*I_ROeN@$!fEeD2%Ziatl+5O;M8k{lh`W;wJNjl&9<= zA@;=B5ZmgZ&Qkbq^2q+dmBqoLk{26a+!YFv0_fpu@{(t(eVW*O!UU}}kOQ)Vc3)fV zG{`_=trg=K&M`7i4?G{GpAbfF$739!GPuCHa?fI^= zG+yk2da_w^+zHT-sM`aWyxL!1#f zTZ7cvyJr6G`58Ec=s6S%rw4GcH#wy_-#7|?W)J-vp7yw=ef0`O=9;6m@YKBx8(G&m z6IAj>0jo@T^_!Fy@MfPTLN7zq7e08DPcP4EB(~>;q<*PY1=lM&syH+-q0i{V^jeMu9N?ZP%&af=3`j?@=;h5CNY@nFIQn-yUTRHU0VWY~4jPkspr{Th;l4Z$wN-y= zI!$jGQ>+2oEi^1&ZU<)bNW0j4b}t64Q#Y^59z0MyU=JO?A%$xTOTNmt`ur+U)w8Xb z%U3z*KNc`!rbgezo#X= zC5yrl63@x;wwlYs-&yO_)!B02F-5~|-wF)1QFcz!XM+kPBH`Z9?i-@ti;k%)tgfCQVA25a3K_4+#JCbUOmzvoPNL#!Ij_q@AWeav*W%coJUnW> zIIH2be9yyX?W1FidHO^B2V9;sft5u+(z55`rh3KzMgo`PztViZGF@fsThkBk|876R zz9Qu-@I)+^SbxTb*)@C44OTzZQ7>S;^f=f|Ke@Ncz*&Yz%IrY7ky?9rJr|P?k zj+k-Gfram+NROd<|JbC3>txa|N=x2Dz3?bz{xdy@rNo||QHfr++@wR7Mpvjgc%3~Q z!C+8*tiij^XVlJxY%S4~XI%G=kMByhvc!uv;Q*-g=Io(9cE>th-(29JpORT)eEbO5 z#-ZOuU8;f_M31|2KXlA49K5Xj0n$N>zk?=3l-{C>77jGRPuqnS*Mwn#+G&pJl;tG$Y( z!0SxzK6;hXskIcHh6SjxC+V2&bFeo<86AGtfLOlv?_NxOU@vBxQpO=ga>BVSal#IN z=CIA%>3=rw>82B}hVE<9VeA{RnH zNT){GKR;+$s;cc%QSB2|UdZPTuuUftdY=79nK=&W%jJ9EzBUC?h>Ea?cWRz>a$hku z*b}&-yZO(tG#xm;9iW1EEnnDATZE4J)dM% zp+E+=YiI5D;tu+F1N6tKKzG7|V}}dY{umuUr{`bUi$F2w7ewdM#lN7IbpxPYYe?U| zqmc=xHOu&bLCH&0hNoRvPfCTGW99^#!5MpBQdNtCBg(fbeTJ^nc4jonNt;{N(kRt` zJ?{}{K`mfiQzp5qJhdGD*12?CUyNpWQ_Jd_E{-BMm!E`f-f||{Ieqd`M`OK;_6}2> z?=0XM$5bqrszA4g+~E&lByUmSeWL+Jtz&Yu9d|5BL=32Fcl+Ld9wU>%6?2Qq(j#_w zZ=0$8By3k}9CWCdmu$>*`4}#+F>m|cbl9@4QSWYQAJ9W9kJr;KKNFu|{LlXd5FBYN zh}M%A@|$lptIG1rOVcBPL0r{(u9@r#JHD*S;o_~kxN61)?y-1LFI8_8&!$Ckl+azbO{|Q34Y}Dl z^cbV-2a&jL8%&Y!d>qd!Bg2zwRUiz?BINC~Kt)5(_!K`>F0;xiCQr9dD!Zx!*+*DQ zbZM#BX0>K|WmkvvCDKXrOERQu6zlio>C|uV?m+(x_XanCJQTm3hC+-M5E$AA-{UmP z`4u506jim8(%PuR(8?CW#S+6*1PA-4M67Y#$FC}2B9iCTz5Q-P*X z;Zv(2t>eFX@*VU-yDLdK+(>aRSuyZ^EaPc=)j=EsBN(e0n5J{B?Dm@FDF5tx_4g3N zbQc=4QJ0D4aiLE}QA$UmSJ*#FuYI@QZDDc{4LeRpOpN#2(zY+vok1e`T^)T9fZ1^r z#v@Os*}=!X^|k&M@#D}?!8K0q)UJBS46sp5-g3o7<#u#Jixt0=%}W-MGmlhlijDeL zeZ~1NhPyhfc?0e*)++yy>prMnTl^(*#WIVRjcWJ1@=>&qg$in}GO0AGHrfq2A}M>V#`ls<9W zZsXpw(0v>_vC@ilhpVY`<4I?hPN=ByR`HPxG+3ZwExmK5pd~gq0|vO3~&FD zwp`?DbcL{U+(b>y; z!T0g3Muy@xS2GllpAO#(Az`w^j1frA{J^?}XEkGLl^(~JzrJnQ`QX*jEyTg);Q_R| z+6e@4N(wJyQ^U9?@ogWv!!BA)Xi_kDeB}keOiVoAYqjx2l-IG_sdyoH@@{vebpJ*> zn&6@$YrpjDiu0Bf#%Mh?W<{el6m{uSaAHhlhF7Nx%#KEeTdXw>jnnnWU1{j>wFjwiSdF+^BsE46zP95?p^Lj-;RM+F&eY@w&OT=5fsXSw8 z?`OME(j<76`GNI_lD~&rY=O=P9x{VEH+`%(5|6@*9`@em=7G5^KyGEg9i*M~+))aH zRHzf=zW37|UlMJC_SdfKT5GAU-AWf4Q+#!CU_fkfW{kDauVrKfx5@3WriJo-Zz0O} zi%sEknQHjwP>;VoGa$1oMDAMY@@YPEJcU3|v8VT^#!IT`)Zjcd<;<)p5yI1y5$pnU zk5^x(e$`?Qd0@=m*Q)OJ$9{mM4mF`4P$Nm%Nz=I@9&ZR|e|jW0XH@PJBc8VWDd}o~ zHfgl9Mu*!}4qp5dE!e65^fqliSc%4k#?tY!XE=Ih{=C9dd}{Uf9%B!EwsXT(SC;He z1W@2ut!L-DZKnppnpPM0`FPGKD4IAh92aag3N5t@^Bt%l3r=^5Y z=r&8oeKiSZ^ib04p>_pA2nv2{Iq8T9n@zd8;|Lxt!Q@xB^mnrn*?Mu>o7f?LzMuIb z=GU@)z?cH#=OzoB>#|@7^p2rCIj+Ha#?=nrq69IWa^-I#Ma`KEH~>RV%!tOdkK1NDn5I z#|u^YtZ+U1aVxSqDSMskfxC3EP{^^flLy&>>V)BENI(`7d-|h0Rh#LU?&-#jTEiEZ zuD|P9GOzBsm?SobI=b&D3OXs9g=&LnN9Ik})85Fw+pxs5ZY7g58XZj_o68;ELksb& zX2K@x#9=uzC1H#XY(RD_K6C4o_5$i+IeIu^Kvh<_E+fmUp{`6ZHqShqM8O;{xJiTn zu(k?#fmRUpZH$$;eSLSYVT1L@uuRtm(THGC$=qdIqq{%qw3#TAH2BDiy=XYday+hn zzm&yqaJeKcG)w9iy)qeg+WOLvpdBoimtrUYSZ=p?Qg=2_$c#;1b$(Iy^9f>P*|7OE zve?x;nn@iZL$t97RXi`?d8ruuoKzl&80IK@SjQh`%NDLi8d5Q)M25vwWHg*1_t!Gc zdHwhF?1v9;pWF8+G+66g%4XrkdYp{ht{sgiG=~2ajo~H|Xfg*paJlmZ(c=}iPf7vq zj22^Q$wwK@fNX({xf?|jZJ89i&C|TlA^yRJ&c2WcR#tykDIA)sGPw(F!gM`4^oWy| z%Jy7HCV(w>1xNumF)N$k%RcCNj}!K53aXJPI!a=@JBad)6teyBImX z>b|SJ`BpOV>UX;Su&7pCzK?JoeJNI67jxx|oxY$gw&Ujz`K0#k7ceDxKBFq2e5~GB zi%EdaRK(!924w3S>;;qA9w#}^xsEazWh-7yTUP2^OO(000M$rLq^Hl6Z;Zp+JKAX za(_G#+~ws@FQ0}I?M@Yss%ohS$)mK2kpO~ymD^DU^~=S_x((>^s==~}&1k**aGriL z@((o4o-dS)Io{(JFmCR5`Kr0l9J5TZLjN{?x(3q0cO!E^FeLD879RuI>8il~`WOQ7AZSV*(Pt-*tO1Jwgk?DP3cZFJ#k18kH+fsvtaHsCDzdYFP;po}) zAmUEQ)UKVCQxt1eqz|(MV&#`&9Q&^UF~#6?j&IplaBt#gSsZjkJ*x82IgT}IH>X8R zcMC2kY@qwz-`&yd<=y9iDnke`7*Q!)w?rEu(5q!e&p4IrI(VwuNo*zLN*Kv(f69ox zI)n7mN)SK@B7TIqlMBKLvdn2m#3F8`h`t2|4sNy$p5kp|QWy%b&EyGlbH|cQrRzjz6H;)4D zb3?B7Tg^N3?SE3jhn&+5Lj;{1fWW+U1|B1R>#w>cc808d<@SZ5JFbwvl3*T&ERLWb zM(_Ev>8gsG%%w%pNhY)C*XWp&2s1O%N4;C)?NgVu3K3D$$`NsmFf|-4BvixExeH%Q339=Z>3(+2}c&!0XB*VYD*fPcF!>l z6O|{o^ge-uiv9lNZl=tJp-Oo>Z%?Pj8!E8xy@7XYJ;S@3%>;%_E)%gJ1`Uoa>PlNXbX4jhT(NBV)ALVsi@4cdIt$7FHOcdgK zJRS9B7z@-psdrW|H8GQ`Ikq^Eqzn<4FFmOuIZ1zhqdm@Um7*DW60*d?Ntm9iu?Oc@ z!#~7Zt6Yqo7e7&-2c&8yw|j6O?W`f$XVK^ubU(`F$Kzn!avBnst={P5yxW!YUau@3 zmO0$CHeBv%W&KXv*rF2{DG+xZ@8b;?V8XXaVsZRR-T>Rr(}+fIAeS?(DN%pyxH%T7 zzhUFA;IV|~$S^eIx4EBZ{KHzORuOi?8w*H9(^eNH4oi;k_tSFO`1e(8AN!@-ahvmL z;#$V-{2J-dxaAtvc&eKOi9&N;yTA2Ft`^cUBFuAjq?Zus$2$#J(9 znw}$MvaE=MyD0AM>W`9aNv$QlNQs@8YaU*+mF7aa@F^NZ{RdUF`^z39BQw`MijgpX zxs8NRtel+XLbvE6EBo|svoFFFlhcV0nDH$H$BO9!490bw^DRe=M;6@{Y_LhLdiUh+xh1nt?F*#YN%S%hbMdfGD~Hdy2=&Jmk@+#o%L_Gp;PrWUSn-BPZJMnNPHf>oUe>JR#1 zgP}wEw=~Z9xa&}qZ@o>OE^oq6MpAc-d#i&QeJ;jqZ|b{ipCoBeu=HAi*s$5A02H)2 z$!E&ij`w$4=_RdnkXUO+)yqqE#+k4iz+rjPV`>KQNaOsF$YitQJacSph$f+X)WZVs z(Pu8%+rr7lVLFuE`j)W2gJ3ozUV8`2+=6y@E1-EZDeSPVr&g5_uzY0R8c+#4 zC)U!?D@0i2l~BoOyOJ~xUWT}d_dw^&)pd_Klnl7(Zb#Is^E7{idl1#h-DyiD0J!J(+WiiI$8%`|#3p}_ zPwyk^2=8G`&(M%(k#8_=KIaswJ_QpnmY%~(HxxMZ@-+K_Z?4jK@8K$=;8dB_{NcZJ zsQv8Oc(MybV~VL3h=!_*;j#}lU1Y@*oVfcm#~L=xUpdvf{oW=XeGp$ULgl*a;N-Lu z&wZR1)u*qL4Fw#8M8J+V1m0-zJDdNb9CT~NL1c4GBS+F?=M`R375yQ=f5}E7XIA-i zr+2ZKjwDTzU5$63+vKo?Bc((qGm1p@kR>CshFuPBxgTRk1+Vd58pMw<%I!uthhuqc z!;G<7A{^TxoLaBDGq7E>KBdZTe{{Otf0ZQ1r#N;YQRomi#j+KzVucKC)&FcaL_S98 zYd&6d0m`ar7TmdxuGjFgGH#L=VBYaK(8T9QiDD50o`-tvlx0yW%Pj>waS312unC1A zhTgM>Ir^ykLX*B*~)sRc~j5z&yr zN#^7vI;*KnE*9{IO)&!oFv}R1$q+?X#jdG!*MnQO)X9Pwoc?ROY6Xv)en!*XFo=Or zgbdML%)5_n87FLA%Drw{SLA`ccwK+SGnn@E;RlI32W(IKe|2K?GGa)$54Zk81FI8Z z&)G&w7rguL5}Lv?yUkOo2MfLlN60d^*sDapghJlvItv$;uF|5)29A*N8w8-47KrLV#!_`MSuk6kTER2kQ>Zt+?E5e}A zu-FcZ29vT%bx!)6W|_Is%$2z?+DVi5mZFE;SQ1mV_a0$J-^%+A{q$|8!a|8pk{`c^ z^ryB%Cp=}`&SxPf2nt=kBU=uUuF@X#33SE-^`28vjJ-;M^V-5Nx1=&n}i+=9% z%^ul4RC{&XMX=V0@9JaQ^C$`VQAx;BT5$M4inMU>P)S!qY1>#rmd~6(cPfR$^gt#- zlseyW^DVhXdz7@+piDsB6i^Wzsc1yh<{4bs(UaT&QGsY@nQ>v!;pZa&8O`WT>W+ZG zk!w{%0bbYL7%npLMC>-JS-GCFWO-jPo^DAumvRRHE_j?l{RD>sF>aj244Krn)cQi} z!ONBCfY2U6G7KSvIfuHVJNuLsT-1IdBQCN=Np$J)^oaD1mIgBl_TSb$(Y}eO+m}jq zI2GKWC%L$y4KGZr8}!BDdv1+f@+X+e@YO?xC!f72{N2i9_)hmVsiD7`mN2ndgyYIB zt1!#4Zv`T%Y%rAF05KZZEVF_&Z**Wlny3X-@(*Oa&SK}uFP&c%fDdtcmkyJ#76D0o=f7fG4GYuN7;!>S5u zyDzH>Dl08h-0Ny|KB|Y^5J`CQlv7_8nfd$a?j>Pk9^UIN z@rLuV`&dbn3?^nSD{~U#Z>@DU7g^aMkBbtDb(S&kTowC=-&Yw;05cTagrpu!$p(Rh(9WbvfEu@%Y<-OuULD!p!~y8!NQ;eVS26XTU;oLCz(pvNl=FgQSqlPd zAOj#>>m3|cbxkpivMhTfElF|#vUHBRQ<&V_D~A3->Es#To5Gesug;6^`Gnu7))-7R zctfG4GSV?J#lS_eOV#yDCEkW4X6)|?{f)AW;C`D6+_kvoX5Cl}=Y`f;3x21HUmfr_ z)~rq*`!6fybCb54U4!ZwM`Obc5dTp#D2vys zQ1IF7HHBZi&s!K1g%uq(7g{t#Q%KJLI1LHC7xo8o^5?+%D34S1ryt22oGFf_WfR5R zmN@_}zWfub@6-DprkWT3VY7hq{y|iqMUd#e009l~3yio)15!OV68!%cMH4ra06Kf# z(h4Vr^Hd~84E@!M|FT3>Sy6$%mEDys;PwByj-1#Xzp=Y0cA@ zOzh9-D0_c4>^c_mYrAb^F8-g>AihVZc7eTN=Js3d`=r43vE^0T{{^5>;v+LyGWj=@3GV-aQc>EvRas*^ zV)&0e2ORc?lp^Qv2X}mvT&{q%`O5waNcDsbO<}Yzf3<&^|KGe6u%?K-*Wv4f0M};f zC|b<);$#ay>VGDX_;XWTR(UtX>`NgE(i!(ZXd^&X?x19|oPyNYbFgmU3fbNOKYC_t zm1PtApY+9FGF2wVM@=`osTw&2_3wlC8&{ZZe{uS5A#b9XXIkhfEu7=1^QMmeFE~a9 zH~P_{|})?^mB3@VAj_-Hp84ZW^HX9A{5Q4+4(l_3Wu@)0xIVN#xbl)rP8-=#TaGIpK#Nywvou2J zYw%wvRM7-*8R?v~o)cwe570ReA}RL9qWPvJrQTQl@8I_a-nma=SOHe7{8#+7kQe{b zZ2n63ew1vc?rDYu2;M4?+<#2c-tdemVT&7O+Wh!S+z=&lh&9-O0BZB&c8Ogk?w_0b7mh<-@wOi5;Cds#_&;2nfC(szaIJ)(N#Q)_ISl6m2f7(tc&yl?Cm@NU$T%8_|~Xo zP&8HbajAzRL_SA>-halgQO*c_iKor^`H&FlMh0_^$S1S_QMkt$^UkE3p0=hnR7- zVXjW#+X)A_L`9#AthKS3f=(tAq!JVv?Zw5Py%jspu&Uns2L}titc2#`bGFMGTw*h5 zvOGF_`Z@c)twE9(#LFy^qva``u)zY2LcHC%arF_Mmh1ONmB&(Q{Y{G*5%R<=RSRGK zFk_s6M3n)+2qW%Ec!@4B~0Ma8;KShND~GT zSRih*l>*SS7xiusdVWBn$}Nh*b!}_ERrs_=`2_UkG!q}@>K^CB7qD=pC9&?wgS!)l zdt+Sp3SiVSk~yVv9>aPg_fllYky+_uX-Z`phBqL6y;<4G>Gz`V*fWF=InKj3>VO4m zV^}oCX7f!9msjM^A^{eG7S7{0i#I@ut^nSvzeEcp$oD~}WfdU13SO;C3g;kZS&g(^ zzWdV(Z=5NNQ6--ssYC3fL#~!e50r3XAkTe}Y#>aO`wCb)T=`ajSfV%l3!-dqfq zVIgYR_htR<@OBXZ=i6l(ZrBa~0UQAw8h$@8d!?dyyYDYd+5ix61C1_$@EF7cNT}o- z{K0(3lMT>~g733YyBolqt%Z&c{cp*(LE{A^g~_(wFe*P^ zF<|pqi&9O#=q2?4c)~elAZy1T90|j%Q@|AVI`bC9%wV)G0ZgXc^Qqo1IKpS~x-k@a z8T5xB0{?7@pi!OYQ!{o0xaj|qdq8{ygQPJ@WLzIYmssD7wf!p8Um!!b(6!Gzr#;Z0 zv8?o9%N1DNfRDuJ`D0HGB{KVCSNh6c^L~33(vI971@yl0G^HlmL>tlBsJ;)$%DtaI z*dpo2XayD@I9^0OsROV4VAbAKE+bnk0H{GVtpPxAViAzSiNGAdVzz?kr=*|J9S<`^uxQ9H2K~oM zo+ir4kJpTq=UBkz>G>aqPkWxu`h(bFm0-07WK^p^0P0x;qc>ng>rYloq?`ObcfBVV zNV@wFKDX=fa5DZ9$f4aX7KeCR^o{G5{R!TI<(&!W?X^D`O=xW=(9_OWvLg~PE%pBK zCo89kF76IYY7N7u=;9N_3^ooIHkn1QCTWP#O$M!?nZ?Gykx5^DV7u%_X{-*cq~72T z=lP08g0?YVSALTARe3g<&KNMT062Gjz^+^RHoEM{+|sRw%Hf#$1;^9WUYMiZu)4j$ z6o+O+OIV14@ee-p-pI|255e2()v|H#Y4ZBB1Q7J2EfsSzRCBA`mD$k+!BLl2Yr?{H zJC@cYf2diYDc0{1Yn{ymfZfe`E^ss`mit_2;q%>`pXCW>02D@6#fC`@9xE=F_CEd( z_8AmdM~UYh$8~ZuU42MqQW-Rm?eKYr5jpf2JJ7DhC@5~xi3D9|27nhc7e61RjcnJ( z-vFp`{DMzh&U`Gp>3L$3rewA2I%U23`KKYS9?&^rG`tby-h=wN%FvXD_yYijGhgs< z0>bfVdi%??GT&9##Rl_1r+t9NI{M9F7uw5shnyI0bkZ(fEvAQ27QB|R*y z$?m%v2NLF~#TERQ=$LH7b29LDYy1mf>r^tl`JtiVp&)S3mUEQ}rgzI-BdPBH3C+OUOJZ(;lAbHLoY*>;pg*tJ zG2cYKO3+`>Z)&kKX=lvx8d)k*yafY$?^dSHen|@EM33T+RW!fZ?E`>YJrD#-GUF+g z*Y3->8T6f3u?Fq6=ud^SuBNuT_Ncg9!eYs_j8e?{z9R-i zg)FVVdWZ6B&^8}t`#2YY>v?{=#b)%E9_L~l=NX=<)|;!(b}!yr1jWSPY;{XDYT!>m;h@I;%I(%0(7c+w@b0x zfz3y@L>B(ID8=F-bS(Ic?YlzRtfxJm<*MePqQ&t~D$p=Vc2d=cZh(i>uUiy>bSBKV zSCaeGj5T${Mi-a))}H1n!?422N#lgYywd(0fx++q)Jhrt{wFAm31Y2_L@|)xGFCoR zKKOMZe55baufCon@*BDlajfE6ah%Q$fe9X_VOv-NkjzXM0eGq_?^h#x92*}Z{ZAxT; z_+`iZ$fZ|10e&Ac_unT!`B-jM)Qr~p2ebqN_+~MTm}8b^3e;!Ll#Itr_*4snRB&9H z?rsJfM^|R+!zYzX9BuZH#H17NoX!^g$5B3WFYVb! z0OTwaM(M5axz104iX}toE33j!VG4XZhmyk$>SN7o0CrA0N|8VBzS8j6mr*PVBRT$g z5RM;hw;K^;L=<_}czy_&;Gw$s_S{gScX>T5mU=PM6Vqe3>X;ZV(bF;s?aNyvw1)s@ zVYbRdS)YZ6BEH&%>0<*LENz5oRk~e3Y1S-eDEL_S;eBg()X(DOc;9ot_g}9W2lHD7 zxl=iV`>CNC)vz+~O@;$TeEmSr@~9njc1u^y^}YI7tR;CZ0Qb`y!RFsCg1}3@%Qn2e zBi^sc)sz|-0@!dhPdBnsgpx}t<}58IFN%+C6Y^xw-xt|FO*`2JsosuETg!Pi2w7Yc zC+~9WLn3o^b$(vUv~QNi%V3uUOcLul+oV-)9V^O=9czze67$JR;1>4D-oiY>6p(FC zXTQ{cIIjOZRt<)rJ>|gX^tIMs;#d#03T4+|L5w>!l07ZVwbY*8Q;cb(W}~>b>gILq zcxZe34x<`!zo(|x!IB{~CR=Od2NRo8{&4kRUsj24hGJt4y*Hh4q4Y+VM%a1Nw+-(> z7MW|#a%)$Y%sJ`gh=aicMD+)hO4q4gxz)6kEw49b48jy{4kI&VC6>fnF&=XR+{`;r zL%z?M*I^+ZA7{9cutbg<=+mlK#;W#0Y#e-oTyn7g2pX#Bs2gg`tQ-~=H|4^#o?0Fb z#*9Q?h(aqpoz@9yW!8k$B+XS*`?)?Bit+qPjkUZfVdTfRe|o5`ve6-{_93c#w-e3p3)3CBi7Qk9%U0-6NJzkwx0Rtsu=_8e99JfbBE|ywh-> zv+en?It&+#?sSF9C)uUO&I75sKxShlm;}vm`kO9+nC3$2N zouA~elD|s=jX<}S078ky(KnhYO`4aB-luXu<7yEXf~X#Rm?h`(F^KnS#r|wz7AYM+ z4x>Akt(2_j5{)B~Dvs6YnLF%P=Jq-(n_$W>zIFP81b&5s;;G^u_U>QbnXu^3L&Lrd z>ZgG53=%)O!S#a75msVpSCTL#|B*kpdYi^-x)_4c&2y|gBVlTny9v2!%xhMH!Ks#4d472(k9l>?B+^v&8@p^_m6Zx!bGoB|yD>Y-gltL>ab&h5f*+0CRVI<8 zIOQ(P7Q-KOf7L+12?Wh+j|U4kMvgi+BGO~ph$u!Jm?wdl6w3hwD~9ajD~^K*R3X*k znxvs`Jz4agt(xlo{d%(faD*7!HglM+lTC9}sF}~s zbmfDS6?xC(c-judwOLL`DbmQU}UfcGypLU2q8%%zZu|XWEDOGW`Fbet&=FS5&V5 zM85gIa?922$?n~$l8hXZGe^}zakTE&&E$76Q>8)^i6wsi_?-F?4gE2x-Q&mJR8q=c z#Ed`Vz9wY<`~ufk@jFB6=RnG?Z!Z*IJgE9@ee4V!y@r;q+3amy)?C(m&DNXe+voNd z=+ru0TaQkvIBqJxLJew~#3+g=j=Li6DBr%Zc;N0bCei<7+k7Q33(LW`s5MB0dJu!_ z&(GTAJ77P)&*s^hrqPyP_Ui5xy_IlKsUkq~M_e0L1zC`M^H zds7s-*;0m~zw?0hWJAB}=w_j5#jqHA1_9quUYA!mZwO!^n;gT4yR1wO4 z5-ehn+f$Omf5edY?~nsyf{rrJUK(ey_S?85`}R=y_E5)xzWw~eCy%LZn9hHXY(b7n zQe5uhoCs_`$j2c3@k|%}anII{?r?`UuRbe%$@9;}P2+jn12U=M%C#_(1|^Hm{g-eT4GB4GaAkpm9qe!q+0QFc zbY^WZwh&%b{NRDHnziT?m#Ke$#T|S{3o`rl1sY!kcS7jbWbK*D#5iVM5rC=4DteE6@^X^B-@=_dqTl;NcdTNH{~7xUy?YbA6MdWhf$;akRo^n@+p`uHO@J$_fr4IbUF*LEmg&XL}fGPP0wHOGvF% z{3lz|Oujom8(aMEg-;lMHB>k+&ODf$)ZTw6IDtscpFIHg)Pb>wLtJ8sxMlZRfUbhq zjdAuVo>rp;|D2-fy#^ALj?wi~uuBe5Tmla8o+PxWGSFc?EbDS{az?}b@38TsvKN3z&zPx^V++fD61yuxqSAedf$B9{Qb@AXKcgT*y+4cbarGR0 z1dVN*Y}2J+^{H&M9Fq@B0j1yqV9mRz)&6%7)B{B^B|nd7P<>t{<=0c=X@Ca0(xQ*%c>TZKx;(a7wN2b#*+q%44&aOB$WxJFUL` zMi23v$H)u*>=fN`sdsyEYLZL^O1&VSMe?6+h{jdiL$?I(DPC1CM+PhtOwFwrC(YR%KU$a_He+@PQqgkYQk4rV^WZ-6dz?e44VR+lU z@zmY_o4^7$L*Y4zcm5j@!E%ZY5L`cC1rW4*R=AL^}>KdsvdyLj`^FTjHN z!5KgdZDN}eqi(y{)!^-?7{m!n1=e(t?Y&_yi@j57VI*}a*n%isF75QL)v}Azk9_ea z%3VOSxZMU4QybeLP*ZX>dqf9%dPQKPA8RN%JHCJB=W6Y-eokONfL~Z|k7kqS-V`mu zdwO+q7uE1GsHp>2>T(D}CkE(H#lw;T+@=16-k@q^ULlesUoX`5$ueNukaJlnXKHY8 zSuK+hd@n3?d@?N1rzLHCxCK7#wkc4Fe`2%K&g_`q+H5Op`F{>89p|`m3cwuKYB{ck`k@_q6{?j+d9mH__R(%|3?>2IV7j0bcZi~JduGUAp=tCuZk-0r`g>6jU zZ4SN4_6jS;2oSFeID5WqbI7?I;pSeo0p;DOT{PQi&9=!mr?#o04)w3Mb5Uz|rBG>= zm_HlDfxXTx*<@r*hY|DiRd!(co*v+NxniunokY;$piz}=j&#u(wRC{HOzwW$aU z<)q_i;varbSBeV3uLrn z7N2GzEnPJEzvngqOEIWvl`<5)$f;ArHaMLh5ZC!zM(mcWILo1!dn?0h`)Y5n1y5(C^;>{K^7jPfQZ`4H$Krep zh0dqsVP&_EzlSW2a_bh&p@+Epx3ahCgE@`sx6)Va400@3On|C~p~*3Gwd9Bmtj18M z_2$b>wMI9m(Og|er)83PmlHhU1H94^1!2!XniRmM)aP|Q4)r9&o>%zPnOGM1U6EYa zVNu&<@Zc+}E3v1q6Y!00a?)^GZ|wvSmis}nHV&B;Q0C~j0%$w~Hvq=SWgVCAe0-+I zU&iYyTNk-N4v6C+DorlijiF@#M9aq* zfxX_Tb17(eT_&042XN!1IG*cbVpi}k!sidx;y2l!Zl>H7rMx}%KI8Xb>RW)>Wjtxe zNKfBje&p{Z67ReOJPOqxc$Za!8g{9qz!GjL2q7y?xvWk}0GaF*I8ccTKwOFU1gPs2 z)a8L=;r*Vb9B&FwO{W~l92XZi*SGFcoHcD{d2MEO_}l`v>Ah_p=pm1PBKT}b0i-== z;`!QSz^}d*s$?X8nCJGO=g|80#M-~u7MgsRXtLfETL*lVCxZnS5`huUwT&**#miRV ztJXzGI!&Xdh!sub?vp-XPsls*XS59Z5u=`f7&=Ha(rt0tJt;iL_bcTEidL9GiBQF8xMABFy0P}O$usDld82e8n5yBp@J6-8C zfX_F4aRNNxpeM9YBrT_Zqwu~rPNL2-@p!D@mc~H5&i)qy(vD?7*xoeV1839(P@8#7 zK=QVUb9Il71bO52bfe(q?ELjCyXMDtvYGDO+cVYngkK_wlZqRFS`I{F-x+|p84KSm z3k!jYG~_Z1=omZ+V8ItAIg<|p6@x(|ajA*WVO;6_?e#nbH8f=fWb((80f_!|(zEXK zZ4+UBcO+mSoh~;YpZNZR{io057bOKjN5X8q>ruTVo$F1t6e10KC+*AJ#*5sz5*uze zMb&`{p#Q{8sot5dUhMTbP)H|K^a50Mve=tp*6ZQ*m8Mt^g1QVPmJ$%Z66|2}ig?un zRgxh6z0t^Pwp=QOTkkr;_f@gB`d0_dvrJ83?c-p9dwEfqeR%Br0%%&tBe4#@2fh6+ z-weK-f&~P9qUND{{L3JTjIb07^h}z794$W~iPOz{x-kNv8GXR1|F#nBgRp#g0g6!R zUWArg6XH0xte8_|gs9s&5S97Mh+ouQ`#%iTa`T_1xZ8FHrktlND#-Fim0dhp3abJ) z-WEcruJ4d?MhTlncVutfni_$MHPJ{V@g+!{C%=lR?ltmbZ@UEBx1EqQ9*`JviTi~y ziy0BeQ?Zo{w>F;lSqot{h8+ONsr5-J1w&NxK@*k#>kp(3q$_=~ zJt2!mMTF|Xiq8W^r1ke3_AVQ&_W?3r2oV_ugsp$YrTNM7^7>9Zw26_`UwsEBB(B+^vH5SMV3D!I9QC$Ppf{Rr z81|dh_Pb1=ne#>K8NGrloNW_+Ga`IFVwlC3gaSDP=MbmWGNX_hbbn%6^wyo-dhMNh zSvvtWqJaV%Uq(!yG^>^KU12L{eGI0OOw$6-Nlhn$c|SFCycuw-%1wb{$($YwLdRPd z{0rIn@XcFBv{3fr;3KfbieH~Hmsqb$nEB?&uYPPJ2|Z_k zBPjvdE7u_(417r4biI#ryDwGPIgq(5A3daRJVU66gTFy^D68Q9%J{{$4)Zc|$Rq6q zRKvD`rf*p!)KLP?{gQ4H#b;fd=KDUvXFi5*NmO8~3|`E%jhXefn8qPQRDnY#`|~%P zudH6=-2L!0x3r1Klf#LicA{MZ-A8wr2w^8t9#~&TbO8o^OTnEpweFhu+!_E1M<;jS zYVr5xpszKv=8)|n+2BM57CR3?L<29un|cx~aZQ4^e}4n;Acme83ZEDnRy2K^giVHQo}fZ+S`HXOo!dN0Q^EcBv4-XN|ATT zCwy~reWOTVhx?pV_~?xK!=3_yzau47#z)3wQzl>x9QO(|6s%D$LX!-Oj~9t+0ww}& zL|RHyFj}J5BV!V8Bs2meDlE86-b5-sZGBB{*m`GyM>8V%fSuzgIa<*~BuJhJr=MTJ z>`=R_^ouo@lwVokH;?@Xl2EinZ<9AGW=E#VCd%-4Z2gHF>H?%<247il4o>WiMP&Qp ziA~lcsd2c>p7;f6+%tL^a)8*nlWM<9#E}%eV)mlJzOXm~-wWup(fygolQR#(i67<( zB%#P9IF0OEizuc^$(MO)H}4;p1KD8ZaYpilD7Z$b+rye?Wp&b|ITL0wFD(E`L}sif)kP( z`3dm3L0(17o35|i&1(x&G5pwOPu@TdCodx(l=uW=ufH^UD&RphNyvY-Vk6-)C&SS- zqf%fO6^DH%-TX~wDhGFPoi6j8Z%~U=pKZbw=y+isqf)#qs zK5G7!WR{Dqo=G5`Abm$L&&=ZhGe$8}1LaX;AO;_r0AelTweU#!T=V9y&@3jTvR6Py zUYW!x3k6xq5!{lCS^j2j9_$-Kws!#H$wCAUelB-r1s)+HiHuzc-=*;NuC4?0ks-q|@|!m8_Yp`Z z|Hbw&QCxZ{FqJR^Y*a$}s7@eOu&le%WedFE?N4g8Q!aYl6Ac@3*2J#rWW80KA2Cq{ z35h$#X=RchNmSq)5uwzWs52#g5DA(gLJeMg%&M>#F2|^a*db%6pWzQXe(CrR3m`4v z0Es=yYDYOf{lL|04U{=1#`9Jul$c_ytO zd*3ldSaKpI3+XPLdGL#HUHj|=Ze<Q4FjY;h z{`8z8oL}$w4RbC^0ZOiZigq98#aZ2k*mK%T?S;s4u?gjfu!t?gt%)O0y1IJlR3I@Y zJ>N`60>k_(+tm}ZE&as(o9Ozveyx4J{{i*|Vf13Fo?8+fZTJI+fl@!*+<^tF8o3&{ z+8+-^HICxE+tL;|2h=?d`H-TWnRyTYH=(Hr;Z{O0MksN5Q+?RaR7Abe7aI>O-?a{8aFM)%mri)Zm zb6_HLCPPVjP_r0CFo_>bPpu1y6HO7i^>0DY9*=Vgd*;{BvJyTUOJ~5iYpI;X-3ox7 zpx3NL-$N9x2fN!T+?wF^*ej>Kj3-xK3hBf9FlaDuM;x1=1(sO5=-S5_t1Vn9Bh%%& z;|U6>VArbr4~gbNQ`0{yqTSxPs$sF3w|=xEC(o_wYOa;)WH8j=V^F|SyAe0aKaloc zjo%DejY#8MWt^_3`IgJ^sj7oD!rw&CD46tOISI)Mx7Qqlx1MPtc1EhJh$)DnVWdad z*rovIZl1e;?F}O@XTNc}!yQM;!<4n##ZKN!9MKT2(@o~(y1Ry?)Z6h8D8}-iH)#bv zN>J2D=&Pd{e;8G|eaq5es))?Y(d&h|_Lo6yno-m&9tYH%SL@#s;4OG$t(D)<^%7kG zl<-PtNMf7+s*uNs!=gCPKHHay%_&oU-n1;)5rLmRN5jf7ysj?<1$n)Y!Cv266L`7o zU>rZuHChs{d_6xIT};>ar2a_k3KHN_8`aWw>Bh7kWj8q3Il)138kObRUeqD;IP7m6 z_HEA&vDlsX`l1&G#Scon_D+=Dub9@wI2)KEi^KTsm*m9DY?irPc%0^!wP7bL=M$`Y zh|LNWj0^ZEU78%cG;aUO5{p*uD6vCs?_ip=y?xY9d5)Mo@G!*)@uZQ7#Z6q|O8iKi zBZvIeoC9>GgyXjWgTk1Tkb0{Bbb3n=?uWHeR4-MK&lkCHGZu)adwhX8mD_QSf1PY%cBZL;uqF5KH zJbY6aN>}wZaPcNR)Yu*?#5e^tHNRP~DYQOXD+@B4-aN5mGlzTkAoVsLpX(t76MZ${34nYr^8oUGZ?4xvQ@SWBboN@rN>&!=ZhQ1`Xr1#-GAXmNMjc zg*0m=7DOIP>BxIhGnU;At5q?}EilspVEq*jbH$QEZsDbj1fHPI{a1c8mz4N%n3eUU zO|sD7bb2&X|0oFMjS@@4ITVfQbP-2k_#XP{aFw%db)lmrsFvdr!`gEAv)e$`6e zPlU&l=Y@of7jFnSx_Uev5S%uuT5}*V+HT)X&3k^0R<-0}yXBMYqs}~+zAWZ$>Y5to z^OcrO#UpxErK|Oow^<^wm~M0l*+eOwQUozqZMLcyTFiN*!|3z-X|`bjEWP*VZAXOv z$c7ep_XHA<9ywm?2iXZ{zmluSs7N1pF@GfAg(D1hg~|xy;2|Ml-xFZpZJVe8q&kRN!03P3~%4(|Vuu za9;PqqTMNST?OrC1Nu~_z1;jyjYUcvP}rw^1aX8Gc^BP$8_2HLcf7sEJf!)>R%)dZ zxUZ&x&qKC*4fP+30%?L+f~Gu{*QQ^m%gjIAohmoac(%r`Nd3-zW{I}*^b0avUbn&Q z$TQ8w&O?v*uHQg%3%7j{A~pXuk?te4hk(TIlk4&acObMu3W9@HklX&@bu~Uhz75Z} zi~LF)9weXcJ}@KXp3l6XiDGH+F0(}y+rr~Ds-X8#Sn}E#_p&?z-C5hz7Wa1f70%fP z8>GwA!&UO(9p@5b&dEgrPJ1lxo%BWhV+1_;v$q@Y%7?yMGezM9FX?9rG~AY%`Su4` zqh>zKs)KMG`Bl0go|zPyRxuWq*scGSl_%seT!cmgtXz#%+*(vl;XJT%!4-is&U{S@ zx0W;o$yX}tP8f`_=#X@a2$Za)v$J_@J}2}NgD+-Uwoc?FBag-U?yx-${F3zD0%IVR zBihEG$YNOk&?Q|kfr-t|&hMS+1T#HqiQHm@ zrT+`}hY8rU)7b(I(+;NI)O+ie_n;t9>0Ibzry-uF(aD1ccXvblBXBwS9%OwWQsG2a z5PvL+(n0qN=Dr-)z#G#r*xtBt`F z_TKx&27VEC%8E%8)UJ+9uB^HUCVeG@A}Hr0E5bO0CSt`ur+Xj>gpBN5zIfFCf5n~m zBb)Er{;RyvQmR_SY@3!AwYQ+9Ld~kZsTq634%%96gGlX}qN1q1_o}`3-qhZEKX>~6 zJfH9LAAJ2xTuJWxysz`Tj@NO-kHm6EZdVtNk(1>F^J|}CWM2P!{FxVWx3!MNmA{9* zN3aNTE{R$zZ*dFGP!0UJn=dDLeBmF0DA*H}EJ@8Z;!v8SAv|th(ogF;1gR0W-iCoi zck40}@7FC|=ruQrzRN=ORmtX;o{kLDf#^WNF4<~A``v-fA?gpPQb zg7sAAj~+MYHaZMnFjPCmh#XC8mb*x9WZA4uG5%ik46-r*pk#h`)<%^8Y_kN856$oo z&I|RssHT0(NVD4ey?iHSE}T=p3V+100dw7U-S7sC`uE_-hJ^qU4QYg6#%N0TT_bv^ zUKAEb_UwDuzS8wLY&G?mPt|)=uP5cjPo1DD%EDc&B;RGWy!JK0`K^rcDc7T zK3c{Of;32%;H$vp8(i-=aO?$kG>$fnNwU*O? z`kOXq%PL+gV_zT*Ipiywy$~p5J*F4EOE&#Cbb*bSFjk==9CW&cQj0!HCwq$c5#>@D zO>(uCe>QyGUC2Z8Y5`EB^r-QKQ3?80!lF@8dR7Hhc) zc>eSEo{e(<6g-o+z{ea|?g@&{K7&*xz!1n`g+q&fO3|gytrBl_+uRTqA>8v9!~bPN zDy)4FDG2!F$tUG$KI(~4Zd_;A`q=I9;`>6~vt@~pD+i&;go@OGqCP)PvQnp#8X4xB z8MWR!O~ThR5Z^md?C-^B(%&)U&j`l{qVN+||49|>7HgcFhp_A4;tg}}MR@+^eilqk z?HAd7L!$XL;_8`kclFRzw|uNEze>>_MM zMEQ0d`^@6p_Vl8ZS0Z!!Tiuv+woc!GZCnJYry-M*n$dj8mCI|FeDq1I%d*7hOfG7_ zblm8x46_Tl|p?Gi9iZusNysRx~Ib>+ph;t z_Un+G#tm<@0&LSA4t|MfURqu7crH=o9P@YaL6+jD7Ytt`6KGlVwi{J1A$8R)he{Kb zIs=Wdct5Kekc-*T%1g>ZK9SX*HTHD{#`Pn}Y$?P}hstP`F3|JKwTD)9Iz%%urJ{Vq zlej~Pb@oM{CnM!jep~2RHqqFM227w-*1_|8^MDY>4sxHT|8`;U`-t5q2`5M7cW$1V zfK0Z(m2dJC$T4Ibn$e2${bXP)|Kq#p)w7jY9pDbN^+-#3V(Dz+R;fpW5=Q;|vVjb{3e(F_^op zsimuD@hGWe^%8=S$HzpWs&*6>oG{ny7dKjEc z8;DqgBuU7xeSPc|2MSrEY6f#BTQoXp!gq2|Md93&J170=uZANY`I5GzWYzg4G8_@*226>)T!Ew? zFFpjO!>qK}N`Vg^N~&3%3x}5N&2~s9302mO$p-B|PYeherX2m*EO%km3Kp$-VQ(uT zPv$WfDPxH^ScyTTSImXiX80JB=eNpwVuYEeeU1RQ;HLVpIqfHvy=ZZslQ!okYZ$1J z#9bp%mxua8q2|m%hN}(yAxA#Jm?w9;FZmQC9teh5NE~cF!JTctBA~h5eYJM<;5hpN zwBOTZbIUqE##S*zwpg>;1+&2|J`N)VP8yEOcVW6KCq-tda=aG(w^_1jmW6D=JP=Lz{J0}dQ;ziXT1aqGqtumDNabgDFPTsj2Jmfb_>LTHoc+%U-nWOt*OPC?b@Kl*Q z?lb=Dj=XW=QoLOoF-n_*9M+cN;)2HWnLp(SRB`W>-nuJh zV`TDLsV4SDGnyi*`}3rQkHBpHej;MylLaN+=?dv+JsFv}%E4uI>_3V4mB(l^U&O?k zNCebJ0PJ)FdC|I`=C^%1HE}ZX z!|6@tbsd{KyWylX3ro6=gdw{nE#*=wG?LIkz9Rb`TLqIA;nX)^Dc!tr&!I=0&?Icy z-!bNfrh;aFkC$2DVabUzJ^<=jtbua0IOa zS<`I`D{8v)A`iNbuSJNbb?LqgVERhf62nI#+6?y1Nf2J!g91VIVVhHhJzX18e%^nvvWE$K~kMJ`Q}VC)NiRi&^Pg{(Q}?QCSK4e7jm|f70VUUy62-PN!&YW!s+AEc&y&iMmr=p zY%Q?l;6zrVdIZ66VXypnD}l2^MNx|hwPpCkWvH?7eZ@D{jigEF}s0rFvYBnkz|%+k7XOV$+;JM{I;8V4iHi{Z(jGi8sc6#qCrzB zLgI;?54jrP;mRTpp{vs!jjviZt3-dS8kjayh0m$*x@*K{i5oQKrZrj?uJ-53-x zw0WOsYzt7mhU{Fta<`0F)zgF;*}Wy)zHslH+BXpx=^rYIxOTkCaX&`n56{M;sDdOg zp;KbMELC21i^Kgn)V+brQ185-y!|ah0+5+!rpE|wwUl8*(TX39*m9y@ny1=S1Lx+X zZ0K^ZgQ@3Mr%ZLFA5zFpd~b>}<^nqn?h$3GTxRe?M{Q*6Z+w*(aq1CS{}mQ}Hitn? zy7g%KlGkK}%2P7cQ`8%Y^A?zs7^z<`gUe3jkx^$U)t!cdn@N}_WxDZ+J z{aqGl*X`EF6a&K3l(wB4?7dp5?Q|Sz-@-Ia%vDex@r0ogQ2fg4oDPQutKBkL;vSKUb-4JCq(hk4 z5(V`ePaIL+qyjtSVdT}5@~%H-9W=%96;q7*eW>_$_U^KGVL=IzCqFsHbc-4Lx9U+5 zk0Qt2I)SavdGq-o3_MLG$5=17LNm5-P6#9rJ~M;z%Q&DbEhmUS-ch!;2T$onLT5ii z>r(DvE=F_5OUnk_GG*X%OF{a>{2K9By#1(;Zk&&x+!N%^}d0nO~ zY0TDHCYRgrSKvy-gtyuESy4_X&*Sm%I(F2$&R9cY!ZwS*)JGlf#h4;aAJ-LqqLdw>H zhbK}--2KNJ#>jfjJgthtw&Ye!wlZ>14&g&ps*40jUn*UB z0|&=d*($JkWFQz-pApVd9q*X_cboFcb752I zCbQM0g=jIj4SE$oRy4;n26xf>i6PMOMX7BXV@S6el}zo*tLQi)u@pnNt1^{1^(Fne z<@M$w1!!y6_wTXn>h63a$}a4cUHcOW*iJe2AlVWa;?TW*nOLCoN*^<>!ngay#H*3g zFrlLCs9QKhQj){xPI@wF;1vwGEfi&5lE?ABS^^NO0`A#w0FV!8EMcxduOGs(8afY{to<#CjyB_@B2T1dF0^ z?~;@XoN}FK@7^J&3epuK8KVfBMx~14A?J8gO6?DE(x=@#$Eq}#XEUUS4$`er@;EM( zUTPa+5o9BDS1PD(RjqvyXcv|(BqzE$#iz*XW9-F5x{X~;u@uFr>|IX*%Kg zii^gv-~_~AEwcTq3c>NJKXLL_$FKLriOj12LlU~`Q{AaG4HH&jXE``?(baGmnZ2MdCNOBc8jcNHXJXr7DaNr@|E%;Pd8yObm@55{n+x4th% ziwtES&G~GZU+V0wO7>*wEwRR*2bCOvsqkERTzN|h&-ghGKA^p>#?k1Gi2>l{|ETN?~8B<~w_@i3xL{i#cx*)lS{O+4V0xVNM-C#iK1O ztvSqYq=~(4aY3KlDkS{%+be#C&WGa5!acEKfL#ld6GPLp>2q#y=K@N?$2He$$~s|r zgP{nWO_5%m2T~*q9q6-IIT3k4`cuh&yJYo^o7tE!Q+Y?@pdkLbDJxiCMww?8dH}x! zU~sX@fy6C7uRf#!AGCeCmQ5zK2wpLG(fiR@3s6kSpa2RFPyQ%Y{ew)nLWhp__G8r zFovvzd}ccMP81UQA@sA-7nl`EAE2s2g@7M39Ixmn2_;f6;6X&#CT;DU;~fcpsMr(I zB49weyjuv&jbzkFj>$rWcdLcLZF0jL0J6iZX$@Ei$S;ANY|!nG0^LEkPrFMgwD3Ou zS}~2Gxwrj3QMmI1=mn3+^pMWWhVNWC+M}!9Ss7cWyFTFTzDM|-MzPE&yhHIZP>1R4 zarcl~o_aSuzI9y8E{h){wiwieQ|4~PI4yFcP16P4}m7&1A$!pMaXpqT$ z5ZyrI^qf@(4j2wn;XkxK8!O^UU36yOPEtJOsQ`qZ3(XOGxCY$#Ltlu)@^S~6iCsWh z9}lf}(%E1-nV;RD72LMs3`k&|<~rJYe`D!*FS-^{Mec($Hwk_sj2#Dgft@{huG-sr zy{L1XcY)5|EuteO7tS0JsSsZj^GRS5(ReIzOBvWV7tLqfi@P1k7{QzIlZ(vY>YxCX4C(j5Q7Yxf z71##hz^L8gvoqkKx1)7nb~~>A0CvC8_Q_v^0rTA$ilbZmO;%KDfWU2p&cn;Kx8+&< zR`63P&wQl54c5>2x1CPH)ii~do9*psFFTS!_Itb)W^iY!VzMt8)=NNrumO!)Ko&qI zeRPdoG_`=RCQg4@WCXNYy*~_d2lUtcq!M}&0~`3Hk$#0=CyGRE#$S@lo>|}%|CnA; z%aX{D+U+Y|g;z)^QQtUjeYDTw>qzCai$c%&cBpyu*?B>9wGI2@A`wQ7jk-P?l6Qu1_90NNH=MaTDpNrH z@^!}tHyy(%ufX`r@#OTcMzjh|lq!_OURo>KYxO0#?Pj*j2fQINet6(m<8m$l7yqgL%4(JZx@L3tN_ORu(dx_ABh6xNrF3_f1FIBKg$iO!S-c*DZ}r#J5=#FQhN{IW{F= zjhK?!B*uP!PM`jk(Wkj9tcvY^(emRbE5!;)stJIh?k9f~Q+aY5=fR>PqfVD{S-)y` z?-I2SFo>ypOfX1ssK5|n>%vswxpXgG3gl_JWZ-Q$;ZLBF^5kddyi>sRUjl~XCTwxs z6k8-H*wQF3q9-uHJk|e@St~jPa?zqLzG;5)xyf_il$L^P)+!?T+K|=>V*3MSTA<6f zo)AnB)jDIr9HA5l1~Hz`&DPY?SoBnWBG>?D{qIi9j{j~Gg-(dIHeqJzt`>5Y>BX8Y zqt0taG^(oO!F(eK4=hs9l1lkCw3?Kb_@|G-3K zH(|(eh8DNHfT>&?Qm1Tded}HY--Y5`;@TXBHzGca_ZswUd)iX>zN1@&#vZYsTcp+k zPCALF*nyfs05{+dVz1IOb+?nXmT+LW!(u5tZIp1$-mWurwJ+&pVlF64-u6bTt zMQ^-1khKxPEJkf$%rmK}y`t?v<*5k9U3$cLhXMNP*Xz;587A{%G%A9DYDBS^nY?RsiaX)yMs!~A8aFr4!WCub89QAzD)q2*>rz^RwTspA|NiMPTH zFIt@;KPVG-ZcnIpK<}4K@ExujM<5=x_${11r7~();{Z`+U~`F z&|7(+B)N$d7qIVlDMbG+<3xRRuGs>hg6k-t!&Zfu>qArKd2rxKO z@m_ASLuEWCBy`KCM0>^zrjNE!5Qzsq!Afgz80d0I?L!KW$Q@U<8zp3p>$0l)|3Bon1FE4OfsPmiqaB}gP@ z`x*jx0IIujYRk2Bj2Cm-=~x#aR?T1!d!3E{-fCKQGDC~-HtDa;yOE7aW11EHk%5dy zC&RZK;v$J2HCi!k(&xRx<|%!_=iFG|)KaK$JaExtPW1sE;?8hL#Fsv|KL_23qC|$c z?nnBetw7B|eyZirbfCe$Y(=qZ1{Vzem`b1DkM(C`5Eb5aq(M|rd=N`*?j{mXYW~`o zub*pyGxN==I`3O(yOLFbZJOUhntO!bcFn3bhq^|2oJd2YIOscCg+Q;qj(pjN-Cei< z%&L4=W`D5B@X{Ke!&d-}L~QlqQ%+F2K{YTsPY|2MBd37{8{nLZy6ph3i5`GwsU#Ik zK>kt)jzt$B_u08=OTUNMweR9`2k9;A-^O|Rbeb^olKl?jEDuG&g_>}ZZSH$+T+Oib zM1jT6&QG4wN_C#_40lIBc~c#l5%+U1a4<-aIJPikcr_V6MS+W}-68^~&I~S8X zRxJr`n7cPgeaWKucTVCJ`Z00*-D*@*fFQ&=tC&Kq74mF8btLcHHHh)!MLkrJ@g?-J z<>g!9gX=C&S{m+k1rX@A^C>5Fn*0Rs(ArNZbm+7NYZr2VW;fK0)WjEXg>y*YGSi96 zS7WJ)7xVBD`YpEQh>C^1hkJmkd2{-O%Hdi^-2pA>OdkvZfc!tuycv%7Uti zDyBWcbWb*8Lx!z&j&?-aCoR0zw*xh(KV&m0f;IK z1B&w&2J3e}>-{`}4#E~G_5tQJBMhl31Kq(CQmr3&WJ&V}+JiL|FGb9`17je8NDBTn zft#hh!TTJ;gGW*T#8Iq-pSYBrBRt`&9JFUIcGII*h~*dvNSX2cN2CQ` zvTm#miz+`HSHH5@)~WX8FW!WhgMarR{_y(YFu0s*P3DesgTQupwsRydW;_&TtJ+Pa z#@*^plZ@-;j&K@&nj&kx1=;#>y8>CZIcq{SIl9vv;B7KJ3(oYl^U1Drh)P+IXVpt_$UKb(b& zr~zMTIOb$AvW$Th!Cp3Qmo6Uj|xRkr8zYt~1^j9d}Q+ z%TAD{ruT6sJ@)L__gvM5X0H_#c~gS3R<(dHy;6I-s2VA@`;kL;Omt0cKVf_J0q(^{0v21K8*XNL% zHd+RoSWTi~r#ePuDt1y&&PceA^Jb!{rwuUt!xJ6r4NP+qpXaLZ;fH9ZKOERQD7~xX zqv6NKg6{=H?fG`zr5S6iTh_qE*#&WzemkEd_dTIWW=oIt4-(rjHtD0+_bXKD&EG#h zIoJ{?{s|SZeL%2h#d7#eyn+iuBF5t^ap8f2m&&!?*w;6feP69SU<2WqC#)>t zDVyGe1C<@O>(J?7M*lY^hV0oRJNHbPcC97ibkc_BU=Q&$9(*;1#ts_g#61gvdbJZ= z*ePg%ajZ|dqCK9g++DFRce!D^x>9{hK1qaPZ^VB$g$qkKs1{oa>%@@jR z9oqZFj5SW~6BE+{1}#6A^U)dc>vI7z6M`MZ;A)iz8sGuZ(7Ru6@pkKx9^)o=z&!39 z!Ta9e4WVPw$mZ@_*0~;Ue=zWNddSl7E1fna^ERO0Bpq0)Gm9{ihb8Ve=>v*Wuh3Sh zirxSh4k5>W8RbB21bxNwJu(vvJTUn7VtsaV0_T6w_3Fc1TO|E^ zh?8&Lg|riO-?y2+<8MQCr6h@yF5*$4CQE7tTXo^}RRZfA+Wo^5(Hx4H$6*yQXIhuE z)SmszNwsrB;}re>*}{{O;hWwT+78BT;#EZuWZ0}d+GD>j8+~B;2}PNLv*eZ_*?B*u z`R*hlIRzXZ?;uOTit?9nce2EJqeo%a&#&sgd6l&R@FCy! z61m`IspO1;yhTOD8GlZHmciT@p3O^6ueYBYO#+@I5&!yk5>6-eQg<(`Hl}z^9!L8-o)6G{9*)T#c|8H^Uu~!ml6FZ>otT#jQ#y|M}WiU zyO{<4{>n+D9Gk^pl>ZZ*!;vox(DG6r`ix2F>+Wf;!!2bhdvY4u`5Af}NW9UM%e2j) zUORS?U8SZa@tXkyx{}T_IN|0I`NyKHB2iuCwHLq@M@CIBihmx7qd=Khvj#Fog5*I0 zs>)iXl=qGXy=^i}x`)TV|Nbm|u(H5-Ne`G_(1b0>N9?G*B`Ep3rbl7Y(u_7_-mQVO zS!s4T(Ai#aSjLFuJ#hGW>J%z5nCZL?V~{6@aFDMk9vY|lApQ?+sx>-{*cxYmq5+r^ z^sRt--({uQbF*1tepFyMSN zgVc{z_`vMOJYg?KiTbDk^H>W2+8%kFit9)Jvw~;DT$$$IC63E`y1IR@_Q@x{P{p%` zis_qLf4Mb-bDJIu#ZlaTMx`mcbNinZWohh~6b)A-sadVydH?$&mISOfgLl(7TrwPo z-Gglrfn!_78_)3>fV*${p@$5#{X%+}zEB8MfotXebqxura|7R95GhVeme21T{z}56 zJ3-R$u>W6$FR(nJmjpEGge3;AD^NgB&4i|`DOQvv?ji=6Fgzt${u=gw9eNUoe`_=c zYScEXdq&<(y6P@Z>Hn`Sh!NexyuBg;{dB1v+wZpibne<;y3DLZaL4Q9iA>RD3>3|yXQu7%wlk!9 zgnIzG@YEa_D3xQ`|K2cv-y;1Gu^tWpFX_J#m1rsonhqDSk)2g0(SFJ2hgz5|v@|A+WTwJ+fPn&1tvmNbNM zAN5UDO}?M11~pI1l`i09JzPZskF*br`OM*0LpiR+V>p1l7`6#P{JupgyK_LtdD|bi z$Zle~kD5)CJ(@ocO@Dn;c0Z6@p*qC_9|OexNQ)B39!d6ECE?&}CHIjIiw8+@)cPC_ zA4%9J(-c7|$gcOTf3VkhjE=FZKX>Pp?d^2Tcl*Zd99#nmKS>nDE!Pi9*Ys(Yt+Cs< z$|y&OcdU_m)(CvAF@5^0Q*STB<9p>ue3#(o;$QRNAjXpmWmWwh$Af^GCJxv9N8}+F zt+ToBTGFykDwiM2#2-qGezDAy-XOSNK)LzwIc?uOsyl3!~NsOi< z4kWvrBL4gFI-B8X`{(1lJK2AhUa?CACb!?RhigYDy9=a{Oh!eDR+BdHI zzubPYbgTASwgML0$U??3zkaQ?AAvgeOGZ`@&9K(4rKHYQ02Vg8`eyCE^S5Km!N<0^gwEGA7>z8Q&6?XqP+CPd&n$K*?qAzlVctD>|Sd+Eww%6bnMsx zn%dO`8EB*wqy7oHE0-tpS8o=uqOs@%Bc36*%#j0$xV3p7H0eR6l)L0I3#Vhbq&xya z9)aso3ISY(jI^%}IOWYzMT9fvTK8RRd8Ei3L}Hsq5E^s!Obl7{PSM5d?WLUFzn5z# zEyd?@D~N-{Z;M~IiAkLKYq%)tj%y~OTX;5FMMpO)*0dVoT53!YA;_eLK0&w-muxi& z3eL|cR*<>zyoa@7LtbgEfk%HaclwJV%0eYJbL$PAp*B^dm+xLiSu3yE*Nc<30q<(b z<(8*DUY$YGSwLNFq`@O+zte%=1ha=~Y~j>JI6WQ(fyTJM|4Z_i)yUwzo!I7}q=Vf< zfyPP&LPjyP8P7NIE46Mu6f2A8Z&Yx5_zquAfBLF(&vHX0g}%VMEwS>AQ$+*ynchr} zTbecMOA1SysNh8m1k76?nTz;>WJ-dKd)9tt;c1LMZ;uCvw56=wNETypKbx~G+CM*# zzW^615q}F6eObMWAHZT0%A}pJkhwczdkZ5c?cwe|*~`GMU)2w${dv(jRORoy`9+`J zaA3%xzZuj1b18cji|EhQ*~*}YNg)}N1H@_mNF)bw9<)c7oy^5z6PxW{H$R?UEPQ4c zH^QSYY52tC>vn{ohuYa=;W=c@9JzN~k0;MJSM{XlGvh@iM7K;@m*+>gbaLdBg&cRK zmYtJxzx8-R(3BDdGR$84Z_dmhZ-6yfg1N3U9B*kR;SAU)r$2MdDe8!8?%xfK4|J9| zWUDRHTba|3WPvS=!1?LRRejqFzb3R(dMpA8T~CQFi+0ROZ#s`8xnYQ56p9e$xsj<^ zy~fqMb(!wSufO=B!$Mo1*tcIZ7P+U*oQ_5QYRz%FNA24le!i%>q_pJE0>kdMnTUbb zcVBOLLijo6e&}L*QcfbwVuOEBFirOkcUXXw)55i-RCtuZn4Gvs0500ua zB}u3MLgJk%-<94jo7A5+*^snms*H1H%Xssx+td^}b$xs{aChnDyclQZG z`umIxD_Tir)ViVYWb4INjP4I*I0e%wze2ZX#pfAnj=q0f9aZ%w*>bB^@)e>N`6DT> zjF@wKu}iOTE7k4GS*7+4@KZdXCZdksQ(j0nGk9Xsu4?9wv9a@Y0exAPKucXmywYnx zKn6Smy89Gkx;x<7#romAT{4U-Z9yD++JiZ}PAAm6PdLtkVVfV~Wxp)k3$0~T?)PW8 zy{@zv4#~uLjI{i+90c6)Sj52jk5k})5K;-4 zxkFOIX9lTbtFXYO;@PjXikxp$&5!>>>9HGb?#Xnz+ms$08FjEb5)QVjK2bOIw?K15 zmayl-t_JwfOup?XgTxb7smX{xKInwrdxlGcWv!$yPB|-N;@ce!^%XT6`7s&6zncsr z?BZIWE?+iQ*t~OEm1_RJ_!VvKPX*PQRd0BPQ$sjpfph+rEi21`bU`BdYeE0NIQSt~xAO(qtI?PK&RxLIv9x85gLHNZluE5tlMV<4h5)9arVaQVIMjJ~3 zu3#spR%KqWwtNm|KjX;ab-x$QqXvq7^tA@*kb%)-ACP9u;BDhP-D!&)!I{{P6$5@c z#ZJjGoL8*2w`Phso^bB%U1tX&5x*_`ABDT9Km#J;k@=v_im2p3On7$EOq>wn;?6ehp_!M%US-ZCLIEmHzLW1DWw`b4? zU=|LalX=wV%UN_!jVb+4jseKG+1Q#JH}DSMyndzjF4TLq%Al&OHN)qH{7R%7xWeYg zWMA>02m13t7MYs!Y~F@WtoDkAtLNWT7OaK#dgmVtI_{1jsiT4FG|mH>Q$(Efl~0bG z^jcjQO?Mc2uqgvaL<0eC|No1!At!PC@z%jke`7HD&tAaB@q#wGXty!7QS}t}WD~Di zQ5Bwx?N8^i8gHjN18RVW)mfY?ZTICMW{pi8(&s3aBwd9@mz|)a&ogQb#NSjhDlY~E zfJCz{ilJ_m)ylMmKEio*YG9x80k(%Sks2QSk!E^TwkJ`K3sgl@wW)O9r5)4 z2AWL1o96*Q@@;8nsb5TB!ujRB(Jw$_Ir~$3_8Q;YYjV;#MuPbU@bgAm>2;2z(Wn0b D44xa7 literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_12.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_12.png new file mode 100644 index 0000000000000000000000000000000000000000..e3a99acb4ed7ff1d0ea62b1703ffbdf842cf2bab GIT binary patch literal 84181 zcmYhDc{r5s-~Nm6k!32`8I>~Gn(W&kDUwhL*=7{7@7oN9B<53=Ns^tUQrY)y7(@2# z*|!<{Zfs*4{O;-de4pR*PshPAG50<9^}f#Yb)J_u5B0U#&hni-b?OwG&b>R2Po1L6 zJ$34I@IQ>ecdT!dF92V_9W*r`>S$_Sdg$r)!ok`0)T!$+KJTRN(LKA^d@{&;n_20O z+-pUKkgx=esQ7Rmc27}bP0uu6Mn|FfyLTeGCocB<;gKtwx$sEW>)aFB{*PY7$D2E9 z`O1?1l+YSMPq*xycIwJD)#7f<)M7lmjzE+rR1^IWZO1WIfb@zK?zweIyVDbkk2DhJD^Z2w=w;zzY zH6V3K{j^Dk!S){;lz-Gv#HmxaPU+myc!E5=GR`#QqHUx=mV~SG-`D;*_UV)C`8)n^ zGR{6ebM-9q>j~yw!u1T@7{@PHZ@s?s{nEYI3-`{k-92;p(xr1hU-n)M=5AsO( z<7NEF&dFi$$+`gfU6$3ViZtd=kwLKLLJ@47EX>+Tb3-VnGxj zqojY<_LR0I*dc5eZ6%7J``b~u1MdlCOEZD}SF1!sG|_kRJazK3umyt&=m6fByZNZq*Wexn z<*)2JF%z}fl$Z&T_LjZoU~02*?fS{_`XmHlI?&*hhThSjI`Nvsokk=;LTe)q?g@sH zrRAx+!6&q^e0-`Tkrvtf(Ez1s)n!H4B94!2U~Ve{{# z3=rNu+~|>a-Uv)9H@0+={Bu=778$h@D#g1(@D1KE36`^%x)9HE14&?!uMx`^3$;M0 z=GQf^un19;C#iF4#{>hxpIa>tCSbhQ6t*bJA$oEby;|N_OV(*1 z*?fWJn2h~dFiz{{ zp<(xZZe*Aa3~ zn8fL4R3jFI6w!z8LG)`C3x2h>`%Do<*ey~nsphjA=BozH#(M+b$a60+B4u#1#AzcL zjHinW@WlpC_IQHgVyLC|M5C%wviMp;WD*{at5;VJ_nQK%g(Ey-5j$pIu}{YC7IDuS z4fkYC<$rbSKtv3}&;ekhpxz$IiZ@QjSN1c~F_T;EX99X6;NFu_zF%{xbRg_6Ra~z` zZ)q<^durxd9@%zqcafL^GAgROE$ycTg0AgunQRAJx%l(;ndt#_-ZdhD}H-T`YqL%LA?iiw0`hXc`0;048aVIV!OXleDsOSDfqt1(sc*6W(rE89jwli?K2z0XHyda- zQAGY=3sy5_Hd^BmfP+9TQS%$j=njyj6=BhOmb{Qr|5dd!iOH>{82?j-+Hw}=0glRG zdEg(z8tc4A;QS0q4NEft#pCa3A)lzl^t6Zb!_nZqgjyoO@~&3d&nHPcn?GkCF`hnY zTbEccoi)eST1%6KVtb~)MY{K8(PhSagNtXC4_;x1f3|(cb;lTPHLzH`L^zPsP0Fng zreca-r$0RfI%&Cw`}f-*cgGbfoN?F+jP}oQ&DOk)+>xHaF0#NqdLS=NlIlnW)|2Y9 zCe$|ZS^8{u*{vnCB-~y+|E~0mQILJu*=*e0@EP``q+qVBVx&wM%p)iehP# zEd}3xlmDo_{9~li!98c`{K(C%Ygkgl& zY1~h{SezgDLN$f8tKqycdM1ZqSGe&XF3+0kHYLJ2>0)<{cT@$hZ<{I&jqI474Bqjt~L|q1^Z>>Co$gk+Doma^pBM{qi~JUUK5dIU_&#IuYoO?$`ca>H>9PH!-$JWS)u103 zHp>(HY%=FfLt0-q^t}|8n-85)9_MuUVl{ZWF2{uNxE{tkl6B<0h;i?Ef+kpG1@`uf za0cfH?8Q!-v_G=F2O%nabDL?%_POqYPYXJx&FuMi8WRb%NC|P_Sq@;ZNx?HN~LqfCUJQ-=)voBTy0+6%2(_Bp_-}uEA;sjwKLO?cG23}p6}Oo zEYBdWY6jcocgLQu-jROZJRuUYYwMS*w$d*T~gnu8HjQ(k1uo+v_SLE7y`IKO32tmJ?K`K$!2fd$SMVv-R~!F9Rk;{6oZXU8N>GjZw@~gm9mC)mL)p$}ME8Hfrn&jH zLv$doxsz;>VOewp?w_nT6`w2C|CGMnU}fLjre?r6Qs{YSa-r&D$8Fu(B@;SaG z1}u(|?$H?=WHLjwZ@$V>jqK0x9C)b^Nd20fOT3Fv)*1BtQiu_$$U|(rHDO&3J~e=@snpiCJN0z37%hO^4srbiV#YAZLlvF}@q5FHJP_KRTk9gd__o7=2>R4d}0 z9g;+U`Pi^`=oh@}*RYj-LM1)tu72t@B3-_yn|LC9Jfn=7^MN(l6Qc?JMycivUNNkS z_T5@#YCOEXAq^X`+!L)4U5ZUCdihZ03SxWPQ0U7`cOKCRhhMp)L26;K0H3!1;spOd z7t5W5!V$mTBweG>L(IQRf5}|;x4i?kg=9iAsdXP?9_>Ws8EI|n!O50~M!daSE2E>1 zlDwtPHIYUrf{+u)`}^hD%LZ8yf}!-_>MZJJT>x(gp;+28OJ&@lqEXVd@EHdL)m9Pn zOvc9f%Q44#=qO!sPmo_kWN7tQ0o+#wj7?C@I(mK7cq}zM!*X!HyR=fWqVk23Lr@)z z&)XW;eTE$Zn{REg5J+law?v*<^G^%DFsuqK*$QN$8plYbg57(iv2j)&H{(Nhu~!U%yph%-_$*wO!nQtv%KKx=Wf&ejt$4CS35=aEqG|nicIC z7%5$uR{wt{w=368HxKG>)2ZLqZ!+ET8gEY}vFmFp@8Rooax(?!>mbsnw`?U1|!Wg#5V&&x+~@Rv{%Q1LxNdShw2P zgKLf&`eI&JJ-;yY^nuXkfVfxW{F!8l0z=7%hfV^;sb(KnzwLVa*ExU6)$J2&spEet z_-3wtn~@a zm2oOLk~K~mkTKF^-eM?nkK*4Edw$%Xwmom~i3b>dr^dv3?iOXL>`6=4L(05V!&>Q> zLK(I}XdUWsSCH>sH%3i%j8azaa`?+YY4!uDTNC7hsL(;&pX}DI(FtK!=OljBbTQ1%|6nN^tSiX0s zb_iO4KCAzxm9F{^-AGj>%qyc}UDIl7F5u`II_k~)ChPP0%65!?X^{!FOPmvLvRCdN zh~+;-^>6NyflCaDTC8qvmu$6){Kw|{+e*s4pY;vrVD#s%eHC1#Qz!|UubStyU}T3% z@diZ`&ixE*Htz1fWlqX&`4_&`reBsRZd|Cd9i{Ko`pK>BLedAzi3BuZ@uGR|HO>7$ zI#{7YozT|ty$%)Hue zWrGR+pr7*fJ0NUP4YSbTebR02s)s5|_bO9emSma_evhvQ?UFpO0b}l!?Us2_F?+lU zfv@_r9x)&hha-c0*mC^QYD4wJP9Etr2w6w|1yS>;0Djo;gm(}*R0sw;S~L)HMK}D7tw(E`W?|1JuGAthi$1`-sZY3 zfDU?bl|0e7NahTZN+cnQm$+g13V)ArjHnIu+P%VSeat+(h1{tBFprBPP|^!u9_`Jc*lcpNCku;AYUR+M zY`*;7$>|icyKnG!iQKUcbk`D@-~3lNnnz!G@e3&fgtiZ@G0j!-I|WX?K^K}4h+?I z3|)M7lm{z2-GR#vRNQP4#xSXoLpr2paqzu*ZrA9QjbqWzb@O}(6uqb$YGm2Das=$(bFbBoH?VHvTG~QrZ+@oK^aprg-L&fo>B~Y6 zO}qS!lXLO|#+S=n*tUr!FIM>YReS?#^k-G}BtAix^W{Er^>Ry_%qgQX+Ns=o(zx3k zX%RLF)}h?I1c(zSS`$#s2B!YV9dR+2nO%Q9@F{V zV78gIi_3C3y;sMic5`LC+OMLXGp{@4k@y8r@LmHZ0>1sZp&fC&N$N{K_BP+ymXnQM z0sdn<$t(TP!rY_ur{_il0;d$W2@8pFV`!O1OG8nuy6liN

>iu-13)P5Ry&f|xi{ zlJ$eN{yg8(vR=ph<@(5pN{rH{=|J`OgI){S@C4~qa@*QSKllJ@gQdT7bn;Z#jL}GhnrCA#GNID zo=NgLZPZUD475j&f#Lf*sDLq)^$Y0jjyo5bPV2KVrUF{#g&Un(@5jLLR174LoW^|s zcng~+fIl*|9~2iZTGW|rX#^q2QoQ7!@C?nqhS{%yJ%&nOTDO;NzpN0^b-F=9O*Z!D zL^TmYPOX zAx`Yp`gzIcej5=#2!jSbL(YmLl*X3fyL_&G)uU{f-;@OZ){n&wLQ8^c?wA+SM=a{; zA7I54hf}Ax%j5G_a!Vlt?az%p(kUfba)JjQw&jDfPikA~61%puzhoQL zHO~d0;lZsb(9TjAj*bOSUOF@-5_DPBx|ptH?izGUPTdvqmV23@Gn^@d+q=G*-MM8` z5;t%YD`lZOuk1lY3oz6%>}^e@agV&739H@wF8y=*!WbXMV4>OOs@nr!WIqnI7eU7TkmB=P~XOS&s+n?#^A`zlMCs~qG=KZ z=92G%2#MQ$+>rs4F9*z&^s|h+LPtG9E>Ppy5#a;A?q-i%ckBkketBhwi=(k_fT%d> zG6;1~w|@5fh_F*HDaCA7ub@|~&dXn;zKi<1t7PU6);dO1?Wx4W(ZGYUqCxLqNvJm= zyMP!G2F`*nM=*zAzG1Y(6)!J#QwFz?4B-7ZxH6u2EnMB82`v7UQPpE@O|zvzOCtYl zm{8L7^GW1vw|95_zwo~y(j4S*)T3Ve6p#8O3)GPEuYfV1=skaXH9tB zIf%Uhx##w$Y{`RD3bR5_S-v-yRGQ(LjN3Zr2Fgi85=S4$6(fm@+($mhUrn1FV!wGV z20iyS0`s|xUx~5%x*P3>v*tHnG5>0UG{&8cUuzy-kZA-ad@;lCkXOfs^9|byXm;43 zdn^~WCvu;CZ^6#Tl`S15{c3WzV|!3)cyn$s!l0{t$l787RoML5GqTk_i|P72Z0Y3IFnH1 zB%u_uk7))oBJ6s567-<1Ul~7OblHk>!RKKm&v#0UyLC9c{}pvMVEN-elE&wnYFrpq zQAQd$cH)Bvk2@juujcpW#HBN1Bw`5$Bm|5hl7kd3m&{RBPUNo4|^5Jg1hb3mrrQx^fH!NApBrxzKt zj%-#LVYw>uK+xu_xpj!5_QYd#@3Qz_^JU(#GObLVC4b3J!HWio_x`x_#%flWO4gtvB~u%D{+@gT)TijdND)1mKP3^@YwWBMS?AYZo6|p|BhwAxnsfVfldN~ z9MD)34`FLoV^q}F>5_x9tsH6O*OYHEY9ZL32L(~3w*;OvLr^l~EVof9HHGgXYfQa# zi}A=#U7>p@j(k8>5a<@gr@q9AFV=ab;()sgm3&>6l}KYSrGc+StU@A7kt8?ve{T@5 zW8kzvvYH!6OJr-s@2B7I-XH?Aa7BpqQwK+Qm@ei87>4^52dazf zq6eMW+CD*NIw9|dFVDna{DJ^?vg1$Clws6X5D~UDz1zeC-Ht(^M7s$k09gS>i{wa=a{5x*BoAyD(M3cg~#FZHoyg~rMc z^ltC{v)RqE85@nO>uIM4rT^M|C6v(onCW&?0`-IVL9dhQ>_6#Z*1u~*cBF4lj+QTR z2CQINJ?C2MO!hN%Z9)f?J;s@2dkmEK2-&!gF&f(GUb;1yF0(?)Lf}39$J^%(o^45f zTHCgqN9p?%>{HUO>T@MX6!>u6WIDUyvxE~9M9zs;^b+Y(P_o2~_94E>&vPBIry(dK zMjo|9d;l*ohFgi;QJs+yut3}M|97Z%Q@HR!NWDV zpVyxPI2ye0Fm!X*r{b+1Q40}RqN`D9!X}DW^v`XxuAdBM`L3z&wB&iBq%T0DrNAY~ zNSsq|;<=u>e{<{cI=yHdPNE{AA%Jg-!~@+b-s_6(N`#`@pNL{`1WJoqk#dpq81+WR}{#c4I-V#e-~H`Yz$& z?k8hkG-H>4sg9N6UMI0X5IFeNKEt5UX*wJ0JzH$L_(94awS9Rex@ajz`ej(&-0tFA z8K_>Ka!UNYBez=SfPlwPX(0AjT$bCAo2xl&&#L|D5sw(d?s>( zuKhVkAS1pyC$eYt^dtGh>z?S3-;3Gb$a@%Gt&fdIV^rCYB1hM`M^<)!mnueUxO23# z93Qtmjo?mgw)wP7GMfD9Wb-HR=IIkDUiYK9VeS;kqYIv?JTPRhcdxzU_X?+t+uYP& zaVB~g6Y>RUy$;x-%9zhVV=Vr%H@~2Ee(=#?u*;;=u4qTL#jKc~5XxCk;m|j5>xeq9 z9&D&{^y-kn49|;`OCCI-xSf=tKg+mW+k394mb&PbTBy0%`rwj55VHP_s`vzt+Ebg% zE;%srEb_mLI+o#M`7J^9uU=7_dGxv|I3OFyTHbh%AdzAeQHfap3McDY>X`!S?Q|S* zKuHsP(ETq*y@GehZbjS+Y*g(j_=gMUlE;qs(@!cb+q2KMFzH_QV&jD`bfVY1*2V3O z>?65@-0Q`qctsxVGcTm#MUR1)hTzj)hfR5~7Gp{Rlu`Mp)>DxT3}XH#UXJR=sc_ zNv_Q;B0Ob`IB8b{ta!L@L>03J=zZqqWJ}dT z`@?+0zkacS*Dgg0H!JDJJJyXOGC+AIw$gT40S6V28J(b%HMGMiMWhSy6~D+dom??G zz9w%$q~qpE+x`M!3qHsS9ueqFKlqW(2c{HY*dQtH^<=<*1XZ+KuJv}tpc}>EQWh)E zx%u;ujpFRa=7`Pr@v=%LRlRZHG~eIQmqsZmtM{i-meLR)PE`6!RVb^w2x=q3Szl5R z*RfBdD}Vi7ilSE%)H5c66^r_9ofPY|vkSZ@5*YwsWgF7xVs=LHN26>M-~%ilKclus z0D;vuW3a5j$MeBzfdBz>vL?`s+<@<&ySZ4$Oi|VUj-6?)n}f{M&&oo41|9KRbI)bn ze`3%5owl{NQq4mFnF*V`eB321{TagBHgtH|8bB|P%L+GlX8(;;J#*>m)dz|>U4e5l zz$fT&>s$_}n5$m?-SYcEv*vx-f`H2iQe6BJ8xA-M z|Bj962Wf$utsSB@9WJhY0zE1d8_^94`27wN;1RpYQ`~qYvdu6}8n@?&jZ@Y-ZVpX%C^jVl+jU{hQkN-zL=~{Ymr(z^K7*AeYT2mECV( zRVhOxOz<8idCS=+o+B4s=VhJQ7rdy8;G>yN!2MTDNAss`MB1_bwcx^7SP?pqu#{_b zNPBh~nAGz-$2%0+Y2)%LBd?-3De=SY#Q(+25^gkQ=&I=g>j-2{Tb`+;nBOs+ zUnBpNCGkGZ%Vwnp@V2g#VoujeKxJ#7FXe;g#iTpKOiyGq?p}_JKQc;6f`1I+#kG}x zi_aL(Qv~jI+;SeoPYPYrQ9bZ|ZBQT>%pW$MsdW;KR8O$J+)c-d?gGm6ZaKb-Q4S-6+Qec1cU6bd=e1 zhUBdPjLO)vii7_A3Gb?nC^r}tC>x+hwONFs`^`o{3xPmf@M96vWjs{FYkiPR+pvo2 z*voIty<1hD1m6>S7CFchDLF^r9(MRc{}hyY_qM!rSCNeNWp(lQ5i_FvNp}@TrJz3V zk(galT)zp#oU?>m0Vbdy{iSSft3bdAYH`$vpx!mtQ(Z0TOPY36dD;I^{71m5nn447~*@G&k1LrYgVHamK8b8ZhtV;cc>a=w~Dw${-Tt_?psFJ&b(_g zE0ezeKs;TN#W-yEh{j#}ux#J{UMWp>wwqt_GMSnch5VLcDG|=k`D{}PdMuKq2DUDU zyBYOznrK4LbnD?a*^7mne>jB;>SnK5qFmy=5$fAUYag@)QvR4bp91AzFpLvV;Rd0g zQ3`oNi^O?sy|=a5Wb@?6u)OAX6s}D5<(Oik0dOJ7Tf2MB>b1G=f)+G|96L!hz+#|W zpMdw%2r)fVUs^AX5xbZZNEo3B@Uqth8vm6KGB#66e$~7IG(yz>7;CG};c4K+SW9*4 z1R!X=o2#j_GvHwAMuR4J^BqAfyC%!|D!Sz|rerz3cKd5Xhg;AVDL!~V-n_aN7`ck? zKwXa6M`eQ*O>#-0_w$Y_;&C(AFf}reWX<2LrIK}N7UKEl&vv77`rhpRC!3LPpqs=0 z?sy}k{XRVqRq6U>coK(__Hvven!x4q>zi%zeR|_e7-+u6Riu2RnBv_Z4$Q4M{OZvSSItQcFW#t``T2n2W^`fYqeF! zWifSMCM{%Q^@|$YY}4)8ngzTb)Clw(g;iaYP#O@x)MOr4tcs~%Rzo=hd4a>Js{85)SxNI@6%=Q{yo%FvGB3RIgA-=rwsl-d>IWC8-?I8MzBI z#RrRxKgg_TvZj#opm)(AFPivf79t44Z5XAQ$@`zBwU#Ezlon3!zN0ABEMj zhvq+095hzleFpA;A^B38YMg<98yxt6F~PHn^loD1oj!Cx?aX3H>{&x~r7N#H-5+ab z{1_yAb>x+??1f;0H=FYaSA^1>N{pJx_k|2tne;Yg>@5Md=C&OIajjc)`>&b!SoTtC zA9&CZDvON*b3dL5&2?br+_n?x#ZoKXo#FFm@~(ug+zsjylU_(heP429Z~d$T!ri5TlhYOdsu%7ZnNQAt{N(Gj8WUp=rHw(eCKstRc*&Jq8z7Ggbmux z96hfI_a}I9+Bs)C$4Y0N|5Essefsq;wUsNNAU6efTm4kbi}O&JKW~jB@s0s*foC|o ztqu~U+UW#!lgV2=KOTT;&EB^z=#H9geDNGV#N*nSFYO;h&>foxm>&6pHj^*o7K?=J z|94C6=5;(tN{Uhm@1(c_oo~qR5?XKH`zobBEz`t;SH|4I00tyX*&z1K$rav2$j`uo z_oG{Hg``1h`2yC`oo4cmV(TgDjGPz$4WXV1eFwyXJ5c4}0MaKPyGI8Xzd5-f`i7q^ryp-;jlQpxkrm{eIfq0* zJa2r>XIHujzLDjp5MrD?@}`Hzitj%0{*0w7*waAGmN@bFm5 z@^4a}6MU`)46c$=gulY=w0*axz<5cPJ5gg&dj>~p@pW%Y*E3C+NfYldS}fLi zzqh4Nq);MP*G)e*zxK0|cvqH`B}n=i#IRF!e>^$nFs{Ywd*OxnwmD)lzT;?vyIXf%M+DLcdggQU z(v@F-c3W~EGiEi1R-D2m8m>T5NPZ<{*jXQyu#qI$I5hfzN)R(GoE6S@K0tG$-q z>LtF7(E6s_y&&MeepfoRZt?UF``Ma5UvJ1!>!H+oHHbmtpM01(WQrOhUY<_1=o5C3 zC6ms84@wg2_;2rPa;U|eSIh9IiDV62aJ`G$ygpSc%nT9?DD3}>&?(gtM}9X`P93_O zc4-#eMjZ7cO0)QQUj2Yrw{zE$!^$Sdd`5ek z__quw?u@Ye3IyRz>py&o+*W`YU*^x#>?2IwG#bmjC&LbrjUtS0wrQwC=&!?`Zhx(A zvOzeJe~l{L@yl+GzWcZm)oxLVrE#H^13fxE;ya$#31`BnUm?bSPz+VS7p{MlHpIP|HkD|T9&2v&(kI@ z#Cfl+gyg)E-Kd@yQ_BBwE**t0SKrQAZ9TW_A44qq7qh2F-1k zGyF`awx&C)(e2MgEv1+A(VwCduufqX>N5vAm8;^ntdShf%M!$y*V?K}k8;6ImiN83 z50pE7+&RQGI+jUt!9W#S25T^TK84&y>ocmmNDW$hq!<^vTs9m^<9ZmsWO~znd`3ip zcMS;4oegXux&ONtX-ZmY{HDu9WpsiPA^!-es(!ligpGhM9($Xt^TLen$G~usEau7V z!5mI`c;RNgi*tzuH9n3F>Swo7{CjET#e#aPVmsZiJ!07q&6`v5%s`JwtC0Lh1Nq_^ zA`iLlcAS2-Z#99d+5BDis4*q8WISbHDd+mO^krGqH5d|G-aqJJR&H+T{MGXD?sMNn z=i<0hDTMEmC=aipVop;6t?J>*wcu8=U^4&q3kS4UUGWX+#pk^22c_vi-x8OFYo$|Z zrQy)%m@arK7_Cv%OApaUFuv)2mC2UERv%nF5*lRGY$4vj z1KW?-d^|v;_E4*SQbF^1%cd7NDh$kd+r;7P(e1%=cEMKmUYQD@{3Y-RErl7Ecn3`67$^Vx>l?Qt zwbo)%PW9~Qvo0>Qkq63aOVK+^K;uN%xd-oTriAz|)bdlYmVA(*x8aSKJ-2mFotTZ!<$;7Hv+476a7~9@J z0lKcgX{G_B9+)1=JbXT~%nx36k?$4cB_pg52toDzwMp=Ka)nbUo1$ph~|GjI-agHpx{upKmE6{F!csf=CS+3 z0PSebBQ+Ta)IbE(c77O?TJI_r$5;NgP5L=!K95nMTh#Zf>Lb${)T@2do#)l(60F0s zfTlu&4kT3;xBV3=C{Z;AB>67vt3V!EMG{Bs-&aA5c|_@NNPVoG;>@@AY$>>PD^D2Q z5i(dcP&!hf9+hg|b6+HJx$sut1s4^Ez$ufJo&;M8e&?T2M?TwgigwYV?(<7S!gfKG z!t$T??hgRa3s0g|HUK5p%(e>@yhoRS2KL1EHw8Ii)!oW27euQ+%>`3QLdT?9ZFzIc z+R~XUc9~oKPK#&lGG`+CE{U-QrF7ISb~gmn)Qd}tsg02@4C3m5A)nISxZz7HDA8_| z^EZY_?N=Txro-ITLyaS-Nj1(OpqhcI=nlK_r}N73A)wmlx|2+91d>=lAXJNHI3T%& z+)D6=)%Z~^R(+t1RCdlCC@J7OWVjxsTnjomoW!FWHw4#_+R4@36DuzioB*K$3j`|9 zf4H6n&c>wB%>AmS@|<*bjaq@%OpNwllJ}n;3j3FQDIZ|*l~ptJHkSx+uh@nKWcQmG7u;fX)v z6;TqWclZ#$*9a{$2*a<~LivHTzl)U%rkIJ};K%nE-x1{87gpaC4$Wl7Q)i)|{Ggp| z{&d={;fDi&4g{3v6-Hsx!u-aMu5h39PySXtT0O<`1>5MB;c3--xO0U|YY~^=6916J z9{Vo6dYmS2S3M#$hyX^KARq7IgZGw9@XCQx9O?EGtnHPc@Nsg66AhCI}MpgP$(P9^Z91M z6!0+K@JL`K*J?ga-8#nOdIYc9W9w(XZoh$&hn<#~(}_;^=7g|H#d__f(1sdX-)dJ> zKLP^4$!kfWi)hvWo7|~`KNa)?lAneqUgT$2F5Wk(dhC^;RAECM!X6C)u-TnGHy1Ip zU~Q;AidtQ<>VImkCloRH-bp@2&4kZKD8JdB#}$)aKN6|WeHi4-QL$UIKCWDPqyf$t zYVdF31sGSrP{YWvv0oPB+rLhru#_+k{<3~p3(%rkK6jN5Qp$Joe-Y6wg1U%il|{3} zh3u5<@FB;y<+Z_iW_}#Uz(lhTi7cM`2oO@wj{TC=-;UpFvVfA3v*D=msQ?7Q1ynK8 zpI7Flo3wgrqWsng4MDArU(o*{wN{8lJ9{}}Rn{uxp~DKxthy|sOtBWv!!6v+F1P*YKKe1X@avItjf?-h@yhuF zqjrRk@)2cEX(X-~lrpva%wDAf|JW08SJOQx$x_WF9RufJ;MEa3`#m3+^H+Hm8a2u* zC+p@t;#7h(YATHPt%jvzJYQxmYZ8A9&JdD!SS6zzgj?FJAbiuz(J>@Hp2KW;@gZkq1qZRbo;p!y_j>P6>Z(Y^@B)VXzy}s=_SYZniO0; zX8F4bHa6}<$hXGlY?ws6?>HT2>r-e zNP|yGp#178<3M|NQ$0ksy{%3?D#1&Cz4Z1>0CWAF=cUhht4jCu_m|6GD);P`JQ6pU z_yDJf*c?5Ew6)UlTZcqSJ;(p%7SJAS;)g!8q!h)woc&XIr*^+ATBk`#9d#S&`J@Nz zJM^r>v~;1KH;m6QL@r&(qSCva=EynS5(_$PQOa$?+AeL?ImvjWNVpRvta7Bv z?Av+$2(VU;x=2kq1HMk&xwOq5Gk026PzF+kofbn3l9G9xX;VExWZ{IxF{Ow@q@M{L~i-A}+F41Vwl9$3kn z25WeV7Idz~JU5|!!0gue50LepCd@DM?BT2EqPiun@752#cJf>cRuxYr@u%&DL4r)L zw;|c{vzT4N2E8}4fF`hS$t{<49mu!!H2SY4aTg<3lV`5F4Bl#9xUZd>*jMx2aFAeE z{V{t`JTAwgw{mU6w{6FNr|XuGip0nN zN4%&G|MSN4YA)-Nd7^cMS-9EUS#25P*SD8)Fbc%%aHwx^9Hs;`jjypk^u9Pcx!5SfMi4fG+JywP z>L`6-SS(?x;H+JWcsdf3Zd67#HDW}!l&;=M(Hi?OKBI2?z3k@{J@;UNnBL08pg0q& zn#i{@U;m2)0Di4s^0Klc;E@vBk976f^axEtAu*+~b?8l)rSANLZMB;D#;bNk%F4eW zTY;6o&Oe1;pT4NnGEC^wzw5r*ZKs%0tG*YiZtr$$y7`lb^uT+3+G~8`Hw&>0R)w*}a1NT%%L8a?c? zOJUb~zDeWyV%GBi3Atk>-t9lJIC}YI7bzZN_-Hp{fU_+1$Z~!f;Vbl(kk_d2V zekYx6j&R;^sI!Aj^L9MisjQoE3tK}cN3c5-^nbH*1>>Gvt1-Gvxo^>a&d#X$bG8uA zJZx~$gb+2IMwJS*Wq=qCZ!|OMIbSXdyZwE$=TqD|Mr&b%(A~fmWK+TQc+R(w{&>OC zrL1pz^0deVlru_4rPO8Bgy46|;~;v7M`0&*)yRQ&*`aiFNh^8l>X1d3!fg3=2CxF( z#GR)0+6T;}2Bu1jW=s29bl&{SX1PuyZYcRuIAj0HscCuvTvtcg1x0}a?ci-~jQW18 zx`Q2wu*Iy_^a~OVtSPqn*IT5D>};S=9=dM9q5w?lQK^TGy$G;d!00Gr zx+QGpo#u$iKuYsMfqRkX7DNxf@O_0AVT}+b)t-}B6v|+~KB}t80jK;F%3MUr08Oqg z&nJGVeOyj{u1SdqbU*zdXy#?VcO%ApYw?W8gu`XqXD36>S#i(Gmy?x8R%gc?W_4@+ z&8jW#b9A#?&hAJQ?@$UnvnB9!?$RQ?uYaAdLR`H_Ld)}u9Ex^#fZtNvqt?tcIK4(g z1i5`p8>ct|mxn2Ir_6VN!a-6b%VMS8Vf zo+1`zcTF`#?^QngK4?X!I1vX;_>@QfV3L~q(e(JXQ(asW)Cb;etpDw|@sHf1>cS1t zCX70j)F7AV5%ukmb{i?CO@BjD1R?u;h+?p8 zw2yF31#{$6S@j;Dntt1o_c4A8`InPj*gknzQr#`zgsEb&9px(nmAE7JjU zOP)Dt#_|&)9Cy|Y&`X95wC6%Zb=5Q0DDhXS%V~5^mz@?`-W+{Kp4Z{<=_T|$P@sGq z@OeSmdXmz!N$kC&ex7z-w5;n?)^^CefBCP7wo-%;_s|>0w^Fb*?mWMmWj0z~KYIUT z+&%Udzz#I~zLPOH0qW-qWT07>nft}#AE3IaQbPD>STqhHDDoTMiX(L$uXKgeO0^xP zo7ZkeuV^tHn$rZhJX17%1dCPfR4y5c2v^T&`yNBl0#mxyaKV1XA^@y!B0O7g(t)5Xe$0C>2y~^~%O3GKk z$xHYy^je?VzI}ECbm|?n44cs4upF^D3GjCwS{htCe#RCfq7u{q(YxmL7iL%wwJnTSO_FA#9GMFx{d*qB-(PRxQto|Fpq9K6VS3qDM;J zA#fh!a3P@9NGqNF2GgL}C;v@7S08RvOuC}7@@0ZB^0YHJXur5!s|-4m&ju4aBmF)9EW@9@>|-)saL zrz2y6qEaKE2Vq+Vp)#b}MEz2exG?KY~9Y|*RoHchlt?*6uXfRnt#IhH@Q z)~rbf`Vz%Eg^lOEmR(m>+H74felRo(_`vmLS1GsuoJheq-{awwOX{|>uNPBZY3z_2 zKyCrI^wvaH#`r+LK02mwVFz0SCAoiqe-@yE7$cOE>1;JPv_1PY zr~+&|xnx}K*lAeGQT=`#qj*manPc4~Gz`3jqQq+i=E`5r?}Ug;~>MtNhj{ z=I)H72ht927$3NBe`0w`vClS7zi)eD9j)}OUrNA}eBSHb@P8)V7VegV8g~b`vEP-{@TM3FRGS_5mbx@zPJQEA#=t3W++$Rs6%JUx4TooXS@XO!gNWP=R;Y~@0xfglMW)Ips zp2{*XFi`OBuRLW4vfAdVhG(Y^6!jN9Q8m`l9@KmRweq+qL?tpQBoDzoX>FEV8)ehm z`vl;~ib)CQBk?`jiJ?)k8W%oFfxoPf3Sv$=-XWW6{J3TTb26`fvOjmKy4W{Yk5*rL zVa~9#Fe%R&o#$!VLAg`6?+YJ!KlkTbp7SFsg?UzULh0wXS^PJZJ)gVKyPJi_IYU2= z&&iiNC_3XmrkWIzUXYAazm{^Z-iL*C0Is7ucu4)|`o^4-jE=Vt{sK<7 z-GU(Z8r1`}`D(3oEsexgj~&khD<_Rt+HZ@g>5m*#_8B zZADJ*%)5iF{L`fdZ$za`ts*1~jH}@WDr=~4?YC76%BU&P6$_T{>RxH@D&hZ+rn8J| z@_qlev`G1(1Su5->4s4QML-%vMmGouqq`f4QKM5@7~S14I!2G~mJkp|_#5B<>;Hh~ zJh<;&=k-31<9&;mmR)>s^7QZKq$v#TC@l~q;d1FtjaL>^!2;ZL#g!BJ(J#Mor~3+> zt;!{lP3vLhs^)agArMGvDs;2#JP`Oq{1!N4Ty{-6>j?q2_JO|O00wEGWN8EHp^sT) z&|$R?jC{+t*C}W(5a-xc8Ye?|w91xF zL>31AH1zHW@wsp*(WLCw$EY^~r&jbexb(O1botaQXk9pz259o3Rk4G5Q^R=pB7N;s zSb~OFNLfSY2RF}7F=){TCSn6Wx@+agL`HMD=6)8H5FA5;H95_@d~4b}G&HH9XJw3r zn!y?(sJ=;_2{{W@6_L49bjrL4A9^EEn^WwbLsFU~ac$%yQn?mJ-vQ9pY#P$g5uCSm z*7Blx-(VQ%$o%ydU0pCwG_r*&0nXV7QQ3MMUAh$}*oqm(yle=K6B8@=1I3fb846aH za=*BfLcbXyu7n+MB4zgw*9f)!6d-3mfSjSM-pwtn%pf87g*$ubofQ! z;$IpM4TIBep_{cti)}U)s$o0onvT=Mn=rw$}qOj-A&7=G}?_+kz>M*O-e*v1d*GPCD75Kydvg)zT20hAV$JoZi!exgXs1K!JG$bwXbzQm#S`1ftkQEP*#XuvdXZFp@Y`j3GJh4e+p zfCD@qGjn8?_}}ZpqN}AmvTn_6r)k4$JcFBALFyY6gzPIqbDf}kaMEte>%=GucxM{< zAmaF)FwEC7kiiE==-EhYH;1SGxyh7m?$V7j1_Ptl_oea=VkW8MtLhTX*Oz*S`tVYF zNPz)!nbi0&filBfy~dxNGitPbx~;U{soVL4h3fNta3`{l6E#IY743b!jvnQ{iyp1y z-zF4^6+ry1CGkg+TRM!Vqe@-xd0r5Ll*eT2tLMs8RCJ>a5sqO8RC@L^$! z?#U1K$8`NK{F;BPqI39qCx`9fp6v2yMY0y$zdZO>&M0uZV^Pet@OP!<@DtaD7J;_O zsh&>|22bCQ5Ft8uVX99@0^nP6F|nG z?U0L4ZSX8)caz<+&h%KmV{#WAz|YnB7SL@nTEdYH*8D);_*&mjO!Vh>dqs+4h<-a>>`G^ak+jz-ib9YZrrQq^OE|P4}*LA z7F4zw&*Fu?*vUarbc#XQxcy~0<1rKvzZ`0b%BLueEZ_=zbW$7CQ^XngX9_p9#Uy>}wnVXi|#jFWq7 zNBU4kF?d{;K-1Kq+BYV!avoQ{aNeB9uu$8+l5TsVo`hG)SwRJf>~&)x7me`d{Fxms z@uK8_hK%9&sR~21kXWk%N;g{wfgUH|8-o8h=CqA5ojh)!M%Au9K%mMZ#ru&D^9#(C zk1%L08Arm4e*o&MH_FC;wIxM&z5?*2y(YN43se?o#$$k}y(@>rN207uNmyVzZLam( zEh0PbIUV2r_Luu!Id&ACj+l?Y=qO{#!`c@4x!8Ya0v{MK&U^l#M@BbMaNph91*OnW zdo(NZAWVc)2FyEg=eexxA-29xt~vho0e;I8_V9qdHT+7H>Tt-&VdkNoepH`Fb7mMa ze9D#&gOnrIu@AXu-KwSxA0x_H2D+wQeJr7B<*>6!YB*| z-gAwZo2*l%ISWT+0HCW-G+&!EPW=_vkbrOB-nlFv`Q0dYdPNbvj57YRGc8h-f@Ndv zA0uf0O1KK7Gr+T+Ig+xk!?6_lo4-|8^IAZ6GV! z%>LpX%e3pk<^zoiByfFL&Q&Btwm4e8Mx?Ln%7xXHJKPcQKO-D3&A=D*BVq89f`#k< zm_>WxeY04wMpT*NG(9ZshtkDjal0m+DAV0Ztv|3-+>!>_RKFL*s?0r^yfoh^?ATKP zX~IB+K6<1LzTK%)ztI8>GvWVs8)M6}(HCUIKLNxg?fu10&xS{BU29xv8Dx22`6fO; z_)3;_DSXgU-h%CHt(uT?xo}(kisfR4su0-OS-t4Ec`y(5a9E6?jY^tP7ji;X`)(Eo z^3_7fP>=^Z-Izr4b3pgL(Dj}sDge0EKLCRkG-T;k%v-)I!nD}589zRnNtjGP zcBZ66<5hsvx9*xR3F=gUNG7_u_NZ^tvu3U6sribG^#cEp<9KF0Hn;p&Gz@oba)EooGn z95Zfq)?#HhxoaX0Vu6Si7t@Pk^Zjv2;P1-r6e_Xbk$<*4Yl6B|bh&vPD22a+OufxV z7#+w$RTfZsH5AX?Y5ASocg#q!8{$n|W+UQe$jQ>>eyH2RI=#@KyWmZr;pWRZON-Ss z?y^{&!80wW4Vzeo8ClwlVsq2gho`qUrC(A%E6tDH)cuWKVfsYI+NK!H)J|r>rDtA; z=Fkt@s0tOYa2{FDV~6(e$*u1iFWS7f*WXT1C@%lFFTG{pbA#e<^0E+$K|y#e zhiywP$d0Fnii+N#YKoYVVh{DS6S@+j&h-#D6x+ix$^v94KG_NVeU43B&MzWyUA_08X+;v<@*(C$SJ!lLv!l>{^G1=l*~YVMXAypi4t!R> zHbSV`X^*uwwX;IHqlKMHLV)z`)Ky_+4M(hJt^E2~wa@qBlY18JYot_vT#X9hDnJ#P zD4Up_Ue>LPBdEQls1K9cD|@1?&$G}c=RK^uf&fXMZX|d#b*~63e0hYv9~&IHO#2-pp&_-&RGLvrI#er24U1{?xMxHmFg^ zkg3v@)VOdRN-lba#YcsD>Aiiv_(tvO*EO5Kpg87whGf1j;|U9;Rtl2#WRrpIe8l)c zO10zK;5P;0P=DF}|IGUp=$tC~nEX|YAHyv;6bH7yLwuR;(Kefono};LJtw*-@_tAq zYjoRr+srdXzkG^MNxGqH|9q6$3aZRdIaG3ApyXL=C!k@-yQ3?;jqO?ebCpU6&{03P zSA&XWep%0{b(385FK3rK?RGDh4r*aN^0+4yf=|tccw4Fdn-vnDk*rvqq{wtESbfiD zA$~p|=9}>a<&Hmcz~{Ahf=ci$Tus3Phfmm*t{n)Wu;@3^cFdfHmRk>|x)y*O7WZlS zqYH+mdiDBE`$iS>Pu%v!qY6o>KTW*XQ!#eamb@>_(rAr6Y|jtCkK~*QM3W(eLo(V>OKk6_?*Se6Ug&s+ZECh1OR_V^Kbv)96(@S$PJ5jOgppON^WN0qBpc*?D><ZMq#fuZ|VW8VHPMC=+*_eZ8WL$SJR>`LAj;b*i2pKrj4Ip5zE59mcYWe$~bdkm83 zQRfSwe=CQy<+Zy=<3mpxKtD>6`>Z7UYd@%4c4E#*+FSFnm0u}wFbfZ|=U3dM}SF)-y%G_-l4{b*Vm z)%&+k#t@lSLaNnJJD`I~HS_fA-v$IxmvKDvGTKH5LYqkf(Y@oOaOH3{Fl{mYdYbW9`1yi)q*s|R{BD}~rVEu#JT zMCe&{^yo0+w@j;!W^vE*moc5Jd@?%_H4Oa}$cE`f-d~f*;aTu)j_xtC{LyOK@zpc> zZ;~qSKBVtCt*G6lr^4T9broaw0&#O^=OmbjWjB#qxiHv^=B zm~M2PP*4Q0Igzt=?(^pSu~5RxYc3LPJxc9D6s?-Qh9}b#Y4!M{XDeA28869{0Xbdy z+hxqb%B{M-B1dRoztlVcsmwbO1fJ%C4v=S0vC!3oHLH{K+jl@ppc*#_muI`RPtr@j z!#%D|!N!4Q%Z4S8Y2~993ZLDKvdvCI7He+`-!BM(6=+~&5P3IN1;RU;HPB>Cs)lv& zKj%-wkF_nXBS8&XKCHjPRlH5S7RL{YA^)Ps)~+3%3#YG***NJ1Rtx}`2CH6VsEw6W zAOOG}G!Z5%iYu$#MQ8px3EouuVKHZI<$Cj{PR{mck&pd6uQK$aD@=j3<;PE7G<)&I z){HD{O>|QX{}DI$;-~Ej$gXx*Uud#>M1j&bkIQ7Q&%IaT$U0U~Pcs_@oi8)2VO>oi zQ8{ap9GHD28VBwZSfc|XC>RXRlj8o<^B(x$+iI=->iAPSkL#YS2cnAd`0?OGa#MA=QI4z>@x2cuNFk=ZO9F2yFS5Hqt3_8)H0 zu`&nYNe#c=zIG54&dm7425a};6zvDu2**T8g@Q3bs^Wzf>|#gB+re2Pi@8Q5>}`N5 zV$yoIBtE(JXsYE&5qB+kZ_woC^T_obd((3q;x~&u4PR(edV<#Tt9AM#{3auT2f|^Z zra@c_4l8Wd&C^%0VniCkTO4G>2~p0^-O{RddC!?Mg61&$xw}U5($^r3>vApLaM6{? z{`|)Rq+8qTi?)l7%ju6OpKXl7`@?YxgPXtirWDM^ir=<5BmjS|Wjf?Fr!E9%zm>u~ zd_>N71_Mco8v1lZz6u|x>Q;e=W#2IsUH&s5zapP(q2H;y3>~qnKTB25luh#uvN+J9 zLzR7T8{SK}da0cg-l|tYQBk027@cK!vA6g=reRhfTXNO7NBZ6*O2t_(k<-O;ax_U! zna#_l7eYrT4sqyt`?h@Bst8ZDlZztU-VdT1!VdlRM8q9qaAi90n=UrMzrEi$5%C>A z?1a;;Nx^!r-Kc9_%;8E4$d-FipKG#>cwurxlL$(BWN+!(!RFJZe*J#EjP#SVDv6!1 zKbnrSz9h~{L@zDyuti6x#?Vbk7a%nt2OOUqkPPBphB*EZo+PGhcysv2ww>5y7z2(h z+9I4Ef*aQ-cKH{-cI-cHDu6i#Ov(ifVz)-ou_+&@jnr>+e%UQ{vk$o2vRa}j&kIJ& z-KmIk*6a!Fik-4|ShgbXfiTy`fsBoyVN4|CCxe5I5byl>Z!7JhTcKrRE$acfeIH{Yz$*>BvQ(FkE?nqh^@l;9P{AD{WR(jm0$#W8O!@Rh2;Sb=t z!Oo>nGPE>)%f;-mKbZUu5Ps#C76mB1N;=RInI|y3?ygUsp~O%wZYxAEaVX~<;~+p* zF7(-KTNxWH^IwcGhhDIIL+1~y+e6pAkRa;gyi0Rht5v4LmMOvv)lTiYcaoL4!)F)j z#B3cVx)gvs{NF~-mR@#c$ob(zC)-Hco(awfzU#LJ!7a;w&_`Q+3&aiT+vIRymqKht zEnH&z{y5u>?7M&6_%hYTx4C&}fM!QBWq!>J)0`y^PnK=_a#TyIw1u}*^N*@7q80~I zF;i?Ek@;pbn_f88CV`GiU{?#Gz_5#l#UppSSi^n``k~!X2Xc`xsN`J$5kuYlZV}ri zKYhkY8s2N=U3^Bn>Ncl1Y(IT5skC2$iDcRh=6o@O)@o9lPH}W`mQH>DCt~Vb zvWYNRgaGl$GlKTkPH6eH+#KLKJb4h-i193m-LqQKE}-gaYV+vCg$dh7+-GGf z;b_^3%qIJ-m=C}QxG9sks`5mc_w1tI!EaBImPwB*;U%25O1AKfxyx5b=z>W#W?u2( z5rbHq%3Av>j@gs$tx`Y0t4!@?BIRB$;_+-JbQMH!f@2mn=DuS_sgkD{AuwGXH*P?I z+mOUqOCO}udYTU#R}uEkTk?8#EltAJhPo>|Ybfez83J7yWb6FibL8$r7mDm(CS`-~ zPvu^6?LUUKJ*Et|^H+l!E|;Yi;;=fy)$EE((4n=#?Y_gue`poG5q#gcBT%uhVkeB2 ze0064=krN4L_9;3)7U)^$xumd`kk!AVkh;jd(vFrg}iL9#T@!Hy$FB93@%G%N&aZ> z)EZvkQ%G<-RODF;)!EWs!Uulw8?d@TAC+jQ)g-waROq-Y)bJ*HpUk(o0ZG{@_-2C9 zaUga~zyslRWN5~osmZO?v=Sj{91#IKcg`UTfs%mEwwfsOgSa@pE}cI0l>*$7u=7_1 z9ZpC(>01@}?I-&dJH*aX^-|y|ZEOA+qg>9|JNVoa&e^4HSL%DFzPc`#J?*tk<9-^C zM^7KX5wV{8x`t%!GFsqf?PCFtFb!aX3fm1$VcWdGXF!k;Sgsm8XeOOh zBxsQKJPVz06BdnK?W{hM0xHmGG?9Jf$hM25WD&Yn77PdiP5e8(BDJ}~5tZ({D&B%_ z7soN1)C*4XkQIa{U@2%Szq9C~l2d^h8ho#d7}~FOi?11#Yas2Nqy%7|;xsSVn!{2~ zdZB&7YfBPn(AUz{ghsc+re$M|AcK;tMTe?0nUOFo$QJR}S#h9oinFkTs7`XH?$Ki| z^RpxfZ2J6f!fuIbK?Q~;77ulo4?xGeGW~;+AW8H_hs)NBGzaV+)_1cHr*qTmX+pbP zGZ{i2SJMZYfiEy0I{-j`oh!9=zA3giM~pL~QKFiB5_648*qxv zoX5w~6ZZ%k(R^SLxM#93v2V0S!>DhX&Sf}Z9b{Q~7x`}cu9 zn@}n#sBNhDt)1!8HmmUQ|(fG50WnH)FKkuPEbcKOjs)U zu8<2h%{$>Uj07`|7XOiQ8{vW1;=?6Rc<6W4kfxu;Fm|W6EGTC4!I&3_IG7WW29kPwzZPe8k#H~)pPYfDzJu*h$P|)fMmi5ewtQqdH zNbk8U1BYheIa)H}vYG~AstjA=SZ+woC}H|}(S41O?D+6h29pbHMOZIOtJ!0-*0W^o zTdxVRto+)K!71Oi&D>1w%sNQ$ANYTp1CFY699FuLsAyo9oO2o2y+kgHMq1?=APyOt z&ez;@GbsefuoQDR^(D+)COJ1Eg#UtIqp$evSooYP*PT{`?}^?NQ1b)QHDIRRlSx6iT{|56a%21Zh$#i{D`C$ z*{hsE8xYjy_D!l@QZ|Xgl5Lh2B%*TF_E_A~2m2fDMju`=o=kNis@5>t2n)rlG4N+fw#o!WBp#AEz zEMM+g>_svV8H$$YV|F*Nh% z*BtCrJ6IG17sW5MxmZ3@t{o^;Uoi{Nl3XNL{0L%KG@w6A{akk8e(_DZ_j~RGm^a>) ztiP%nVP?V`=c)D<_8P>k^ohPJuO`Bzt~Zj72_UAZcrKh}U)t~ZsX(cnUQS12y|TGK z8%R5-SOPLqO1r51Vw1b`6NUQuBE9rL4DICDL^lr3AFzh0r7AXqtn}0OCT~)`g_G;0 zCMN5#MLh8POPJfY0;*`3rJIyc=(lam`6@R$oT|O1Ui_uGZsU;2cH$T<7MrTU+J@Ft zT#!*k{!CdLpT%tOfbi>}<+JBs{Ekgfi>d^bO(Jtl;y6G46g(__x^4;x+UGpOjf+%LZ43%R<>H&(DdAgn&EbX_m zrH$_r^owZ05v$zJ^X1PrtAAz_aStHFvYjcV#aY*N$Jk}}{X>vTo#yWNE=z#Z&U*9y z1Da^4vRdh8EBLqCs=-jBA`Di>Nx1gx{fWVXo(Ydso%yOA!5Bn2#<-jW9ovx zsjBQpnkoG4ILQJB1Cu_G69#0rSYrK>fBwAqDHat;HXcL&+V34AdR&pf)3olZN`KUJ z6LIEpQ|ixP{;281L+pJdJ*yZ|7ObHVvr^{+qi1@<1_0gYvE?Pon3l>dG#>o7rBUHI zBY5>KYCFR-gIGBWe}U11nLz7xHGIlc5em^DV`B}J8_X~5h|M|KI`xL8$|aQ7Q)qE) zKTY4e0JG_U;uVuJBw^+gShKXJj}pJh zsF%$K4w$S+N~tuzXVl;CG~Ui|WZE0T-=&H?wy2YTt0lf;RLN4boM!@a$sj`sZ-?al zwDI^L<1rH>Qi)Iya+NU^_+C`o@y#Fn@IqHdtxo<|G+&2S-8ALG%8Cg%z_ie|&eWxX z&eO?I^!z*leuGQ_LNz?As-zg4f-D7~j4%lY-ZI+eVzSdj3xleU<7b;sob-f$(W)OS z8D2GJLyLU&vuVE`V`a8Y+ZL8)9Rv-XH7!7rfE{i-0MCz|EY7C*a8{=Fp#!#T6mwCg zih04TI#Dz`oH~Opp@+7p&GF|Y!Tk~>*Y~(^c=Q!~dj6-PfsHB6m?(3?^2j}?bot4F zH7b$f+vNw@%^&I7yq5&9DinXqM!3+zR%6#sh5mM|nY-?BqI&0x97Z})9nVnv9h+AG zr`g!C3<2(B3woDnHH)!lvk0Ra(U7DpAJWT_1%sF*Tmo-b`_t4K@wY@kHuuPeschbS zn<46DEEPoyD}Y=>Z(zqMk$xL7N#&NN*0`X1SD6cDz5G)y{u(%k#Tf3L3_Z{t_E%5y zt`-lQqTJTPt?BkUxE^E3a4qgNn|FLeDPgUZdX3!4$U-lEZN+h2fpvc*^XAEXBwG0_ zqC0{11^AU=1*=RQ2RtORfBYLL6(6X^Y&6jA;hk{6^d@8#b#0slaeCQTf$1oR zob-)L-B8luISI_VVGO z|2?vJz2E6>R~X;Ic~%4ro9Bwg8G5>eHswxXr5++ZnafdGroLgi$AX0_+YnhU8g^cF z!y0B*i!o5SIb@?D%H=^Cz93l4X94Sn3?KD-8ww{dn6#Ospp*G_U<^ z*u^9r%N z?uVi4FK3?gQZ1x`Aq!=l3&zK)# zCa3Pi5ZV9QfR`fYYo1HnJZaAX_XXnZr&K&Kvfq8L!-G!#TeA_UJ{I7SETWxsno02u zL`B<(b@hnyh-34l_%of>Qdd?z2by%O;WeD=$l%f`Ra=exPeZOpU?R&g^pALQWlxdc zac|0|!ErWZ3GE3Ftxw%rwy48GqD=ARE2eY6#I%~PO;-Zq9k~C5bIhd6cGN>`=71&! zh_Mk(QI@@No0qx_mdg-|qwbC>ABE2y!V}e}!(tU;EXmCd%LhOBxOKbji0S+p;8r!5 z9v>4INKjqc(nVRgeq1;s^}yOTLqW?$`{O;r4^kR8g2Ic8-%OpH4=(pBr0n?WeZFhd z`xsxP&chvtY_iecLTK-(596{eH9Z2)-D%af{T$5*2hETT2A7J>+&J%awnYT(BP#g;O+>8~gBO(XOl_ z4fs55jrprb&#ik4reC+Q99Cg6Pel^c+^d0|>fuBn1C^9E>AjR^cGZYPGHcUXQ{&do zHAUg{60n#XFn4e~(mp!6>L!CWhGAy7I2nl75UE-oiPVClco%64o&(?fV;Uz#4IpL4jN~1n; zF3rPDsnzJ`^bk_5i#ZXN&CI{VY7`LpVnrB+KmLi4!QFy1me~DY|DCm^5DqtPaJjwd z`v=5kOyL=sj|2U+JzWRdD3a7oJKcPN+c5M7fcUvir-X}3*_@b|c+bBUOx(P{QhH4qngH~AT39Y4XEgN4e$+9$uRsb&r8{~Sxbqu)EZhd0_~zuBun?bK(w4V|+PUX&w3*74Z^ddFddO{o( zn8o%dE`mukeb-o|UTrF^gta)}e@a>!NXu?L7HkG&pY?Z0;+uN&Ho*8UbKAf86>sW< zZ6H8XBoiua=@rLA?MA*7FPFPo$NUTIG>~_f;$SH%6e{oTG)!=1gSz6v0~ML6Z!9++ za*};mt6)W{d2=@B-6MB%!x(fS0LQy*NDx1h+hpa?oR+iBkjyrCtv~ddlQ*69MJuW^ zPPWsJ`_*0J*~wsJ;AN`3p5EDMJ?Td-81>?9{UD1qx$3iQnGb8^l5WLKVIv3KXBa`0 z1)vmMxh^N>44IM~YMZjgd+W%65lS2aSYphvC#916-k0|*UC}J60G0+>dJi+v$e!lI zl)z)}e}q`8Z5G!<$Y3$os(0ev~w3h7CcuT*>1Jx|U;G@nNEYIt_ZX)R^R677<7N;!4LcDh0(k(EH3 zu+YH$NH;&ceM~#DLOvf zHl-KyySEvBeY%<2$q({KD9?g>q(syF>zef5U!AXBmHm^kNNGg(x-LYUHKoT>d~P2g zigsA#VR^Mgb>M-=ZVKh3aYHn47m$%jOK`4XC#2nKS>A<&w#OJexhE-|$&Ja*- z%9HhcxS3>%I*jh-VG^b<;^@^WY0vHap7+3^>^0F(U9PoO*Jd|XR($~JhWS`?{RYkS zQ~52$uiF|PceTbOUw!kP*fFn5~B9lopH>oID6LQpCnlXCSUIQ zlV~XIG1-Y}?hsgPrd378E3ybQrI{%uJ;WpC!6Wf*wccEk#WG>tbk^~M^J*L|3JLN? zdIgGK>RHbU1n5@nM-PWI}USTw*QX&ZUJF7G76QOwV03!6hEfl-7go#=?C+ki?T1Uy%9qaC~BKXonfw zOr2cf-At{_WwA@AbtKf=_6!I9mCOC*&Pv zI61gBA*j7`Ue-2RbfbW_pL%u9o!Jc)X6MeHM!m%T=4j)6-ow+aL!-d+_l$d&{(IP7 zPt_TDCNMcMswiLm^p}JSO=w-(z59fpvyXLmcNoGfxq$7pJ7g>w@`OPTCx7P2la6C| zEcH&KlWb-&iVVZD&JhMlSv-9AiZc+SzyF|&Y3}s*yC?05%AGATS|9T8;yB*T<{L}@ z3i?HMF$0(Ra4%QU-n8d&&S@yAhs0_Z&|^O5R3bY0^R|5Vzq`wGN#@x4_`mDK_%=fL zmK98&VtpmUr(BiN)BsBClxEWI$#8bkhM zTV0T>5r5|G=eUdPv-)o=80^LU;vOdYg7w>X+1$`l;V`j*&p`G)^voGtrKdK=NTOTa zPGlsGWq&oc`}B^tJoSPDvoYG^CM8KvJAh@Cee+ywZB6dgp9nv;niLZb3!nTowGpbk zBUSDr?i7a^G=901pLHDk#TDZ)Pdc(mi(%65y(KUO^cR^USa{-J%O(E?Nd763M;^^L zi|^a9XP|<;3+Z$|Yv&)Qb75nllKjFS>EO92Q}#4ekf6<(v~jIyuK3iuT0>{5;tMZa z5wNz%te+3?C+)eJ?AxCZbs5vs8hg{yJZ|envmept@nH{T+10;vTK82b%l2O4Mg2#K zAI*)>kHVRBNC;>pW?IAF;=9+ne`iC29ru#7<1;KmtIbR+1O7_khc5p0Izs{b$8+MZ ziZ7{OlrtM!{TTE^cvKM0cwPTYcf==v47}lQ^*ZOw{9B4WL5Qu=^ZZ1c$!92#61XBj zk5!7UyPcw4dlpUCNK4;H7dT-b{L>wsgX zfLc4~(c_-zTR{H&{&1-*6Ap-wOd6H^@H+=k<*!Pvt)u0u_z7Lb>a_^Uz91LdfA3bB z&Ll~ed@K>E4<%zGriCU9W@c~4_f3t(v}7yEJj^Cw4)Nw?r`O&n<&&#zMmwJ?6c-Q< z$8dPcGvZ9=D+@Oqb9I~#%%6?kB$eIesDw*qONS*+rxZppImqJjg1+!VlI@$!1`EDc zmZ+f}^V0tAfk?3^oV)rEFn&6l&G-<(I|_Z%usdWEZ&Ei;SPsKGLtXiMaD!_d%^#`R zgOF;Og`DJ+(3ZAJxrd((zkM!{~QNra8?^#$yA~srm~NRA46dv2KUa_B1_94l3-f3(k<$ znkRK8ovop6nB#t1-^W?mmnQwTPebBueX5*2cKcPYyD6?iS$CMcLyVJPPkZ4wvtq3DVd9A%-D{$^>l>)6SA?Bm z%u0O{z$>AS@Xb)!?!Y>F`W)l#sV2o=dvTlonH&Mrd2-$UR%-?)RF1#i#?nowD?}uK6^*)i<8t#d)d|6KHQXYR5!rIliHIw} zA;DoW3FrGhCYyaapWI4e&rX3RxgM0si!cOhHj`?^)(|?E3$c4soB6%eEh@YKAAHkb z-ehO1Z&Js*UVT(~E&{c9;~e$GFQQ6JZ6y;{BWj_WB}PjCFljG1=-6gqgpW2TDP`@D z@kWV7agl?aPX*VXK^ZPGTSm(hn!@Xz(cM&)S#=9gL;3Wc@r4B8jHi6SPtpX8^;fH= zHflGsbchz*)0Mb((6`4G)MOsWi}GB-R4!qV%1U~Hh67#qV#9I~Avd)>c9B}US`>mO;lWsURQqjw(L?a=~a4 z@vwL58BV1MsvhQb(|+%Nm9{6SMas$IB6eANkyLa;CQ4cP4oA`tbTQ5s(jq=VZ){4r znOK9`b4d38y2h|nqPSD3?6ww6w1v@UPRBz?k(em4 zpDcYqHH=H1JCcLeSXcO8D7}X@Uq9<*fpUD(Sx`%o~u(OTZ7k3t0pEQ>CM2n%N%`$1`>$=u6 zcxJa;3XZ>PT%KM(*G-8B!Eeu{)52?{h(P~Q!c{|-BR%WV0Ix-sHe<*nGpboZzcgu% zzNlT(PbDwraCKe%$lH1Q$LgCD`MJ$kk!~0}N|+4S3JfWF^6$$-jKuX5tK=OD|rIW@zez_|K{T zPODdG(9w%Y|C_&lb(t=*8@P7PtgJ<(;_?yH z0kdmkyeZNxg5_MDuyXtBaLuGvJfUm*ZajK}m5`v`%l+Oi{B~e(x3uQM_t;r+~SON!_5gf;ovahIWI&WLA=D7Trz)`3VnufuVxlmlJk<-MmLSm$wqL% zb>BKmhT;Iz^QBf2w@uO?>`hx`#Egg2LGRr>x3*!7>{ciM{L0v_!n}eu}~$Yj{%reNGnZtZV6v z7oEwL61rpK;73?MvlAE{+zZK39+BQ#f&u*44JMn zZ)URnv1zd=|BA>TjuTz<|GNNSS>pIpdxWn%b}=toKwkd3XI>b=-`szY0RMTq=hJ<` zuI=DAamFRYrc{B5HuSi(fwz`{ z+Od|ir(`o&-1wh76;<7&&Fsw^UCRSD`uoKMb-z@r-KTWwCh7f}i!t{4AAfg*(A}as zW@^IM+8ClzXEyedztZbgB!^a|k(M0KvNv8UQhTTLOBj7N5|xuo7c+SXh$AV?=s`$a zv|V06+r=VLA5ei06F~qK9C!6b3g(BSe2(rH*=K(L=>yVP+|Ii*<3B)1l(SdshSaP& zU1YKgygLJ+Pwu51{j zO0=tbiJ&MHdXK!v2*{w-R=JpW^B7w=Ou)L(7a~N35_z2U;onWSAEn z(E$qQrrS+6y}?B?`DqkPCg&C_yNffDjN#+f0X&0%S+>mxpI zfbjPnPjSDMH-!Hh_uNTT*25L3bk4&VyrKE9l2du#?_%CU%GXb&0m8Z4hjw!~O=jY? zz3YZkY%TV!j(wzQ#Uu|H=}<|t!FQ%EM#+$onULZ8W#k%?8P2EYQaD;uU2y+PT?H{1 z=b2~#K>93yV;=w5JEQEHaOPkRJrurFVKBgOt_@jH%D{oRo-hcX1+x9?Y;iqm<^WG- zd#jFldH&Y>=I-#o??Irv_de3GOG-#}v69=pe9Dn1tDZsoMlO%=NIDNt; zL0oJ-Jq&SE{VrQny^c+Elw@?1LbD#Nztc^^Vz{As)p;DS!lb-9&V)A`cl{FAb7vMN zdx7YRqNf!=G7{VSt#}3?urURzJVPwqE(_mrV94EjcI?x`oT$Y`@=hl6ZhZ5Uew6+F zYFM*4Q=IqacSg7G_NPm;f8YBA5K2cuzPC;6dWHDIJdMNfMB$vu^9-WcKp!IN9*o$R z4_O^9ULGBR?!h$=FVhy#QA$O!C>FMhns??|U31!`#%Kr!dT2=#IzG)ojiq8o~Bz!{y`leTe|4;&I6Lp&UsU$A^(&Jc- zCyTvAY(YuJiQ-WY3FE6-L2!JNWgeYa**V!AAmdg%y$kM8#$I@6Q22>0_6n{CB9uYEuqoY@MA|IeKEjMEZCf2vUGG06 zuI9{#)V@hfJ6Q{Eq^0)bxXf$3hqn{%8%-Es079>DIiLCRr4X~M5h1Tx9^7pxB5&op zOrIUA5~{ICqn;%@RqLu_Q3d^yzp)uU7l$vJ&&%{0uP<}Z)syh*E(X40a+G7rJ^CGX zI-Ah2VdtO$t20S?k{JCXiPVC6LUIpjs}=dAH?GTEJ!CET9$C_h)Q>ixS97_&ycLl0 z8j4$V0_jb7Q=3(n&@p16vT@{){wKzWw5zj`I8K9f~T?-dACN!3^e(x*|pM zPaJB5=B_lJLaQY?JB^KW4cN(HY1lN0x#X3@tkw{`m{dkWay@9N^A$Ee0ZY-$`U8Xg z!<;ak@J~dF4ntr0V^bkV{Orwz+4}?7wAvSH3lXz?zKfT4Z zdB%F9Ltp)MY83P8i6SbP7E$GOUEH8*GTZ1!t{~zr-|qV^a}a!+6x^l6z#>K{cGH1e zG5(3FsBFV86bI5ncfA!Fh1G_)?~NGe=)AA6cr<=|Gh4xHM)U>uUDxa*+)obp-;A@K z0mv=ZOhSk6;?<;@RDvT(kOEL$vjQ<~fCtstMasp%A24(VYJqgxo!UhXKD_e_|gIfMcf6U6clJuf5Ybg2?EMl-= zdiSco$gf3NdL66ABpJC-gg%vQnIZ`QI+AiyS(t>+psX8q%LYyb?`@7K-?`&(U9&ll z{@i_hODj)ZYgH@vH6dpPf&!ms57T9=qZq2Tr7;n(gxX z-DQ$Hp*W3U=kT&#oYbhx&cipzhfhd#6tS$k8*tIc4j~-tqor0m^`b(x`OKHK;+3@e zNJ61X=d7pVBi1UD<@79UXF3i_0$j3Nz@b__(L`p#;)t}3gc=TU!sG~+I`$FNiIlEc zviVcD+Zavg`sn<}Ac4-SfhfU<)el__f&&R}1}RlVE>!SyQEE@v!?;=;00d3?b6YG6 z%=!%*(n~l2XO*v7`Gh5{sY&rb%|o?EN=-?a1k7F-l)M`ypjwQ)*ZwRzDCKgqHxZWd z=f=CMIz_ri`?%`31H`-pn7v`EU3y`$_!1~}i0$<0=jE1r0|pC)b!sV76``LSifOm0 zci;Nt8p!Twg!e%d7&51_vT(Ir{l*FmdxHiE(){x(KBvY=7ZT}msRyz5&c|MSS6+y3 z0(f4Flwt0PS<&a(?Unlp+FG~CSN0ft%!*c?8f?ijw|4D&w)6B$=s1jw1Vxg1ButAo zRu}-cTLAgSmJ8A`p*nYAW6m574?<43l#CmjKOTRJ?k2JKod~OU;xS*U3<2pq+x-j8 z^DX~*`ynLA&ftSnJ%+G_f1l2B{=GH)=W8Rg*Usd1QHb?o<38Dq%Pp1@KZ1%Ea*w`6 zXjWf-Qgp(nV}RpQM9bmi;y^ks;sUOJ$XcDi;sg2IO4JJN6ZM^<=7Sw~Y;L-8f}q`R z)EGdN$6?fTj)`OF^sDbq9cQV(YI$9x5 zC08=4Cw^op4r0s`Mm-6|riJpS!{TyI>sdUJ8$ z5?@7e)$>AWHso~r?9a=WZK>xA8p;ey6(h4;#v3}hyFNvgf$a=4lya)cK6HW^CjHN_ z!31Z7{+0vWqlnbE8Cch8cJ4EB1jQE>71hEvW1C9N1ax)^bal3bqBlVgdp?&a&uNac z(x2tx)wk#)=cdQEc4VQ22YLoJ;i#C0E3A-)({nLsAx;M9DOsJepMsAdYeI4X9%a1Q zH&Bwr@yWsOAk-860eRy$ldG{GmFb`+`*AbRC!c=w`^wLgB|7fWV@cP);HB;5x))Sa z{4j%0=!(o+Bzs*ayf;ELF&yLWXe9nFhax%%8qynOJ^NEpA)po(+hXPq*DMqIGN;hsTPpafRtdcN7@(Uy5BUqntp&oDaKNWML#FC;`t1Kj&Ien~^ zx5a3O{E?sbJNiW$m+d@!`RhGcs_u;rNEeq(YwzLtlxjtPNT?4ztAUQ)8U^bPN7j_r z*z!}$8C5ZYXIBr14#^Lw#(-_Klh=_`%jL#5VN~u|Egyejx+pbh$ONZmp8BX?ntgDvtioxwreCqftW#E-tR>JIoUN*Lq`P#(KIB3N>4jJ>BG zZoqe*y*Sgfa%pYj3F}o}d@foxk*=Vz6WZ?>ey<_%-baRVukU2mL{(&}=h~S|lA%Jf z?W>xQZTU4TsSgBT1J!2w<-eZRDV=Gn+TGvjM*=Et2{gjuvmw1(Rds8zLs*$-PJhqV z{MJaHO`egw{lYG1`2LEO3t85y3}-5bmEQKa(E1(Udj;z&G<+lJTH;s?n*#c4uE;+% zkG^*4vj88A)o_=S$0&kuxx)43@+9wCY}6Q?zH&gZe!RfL&)J{Aor<2YZ@;b9yKq0L z0_%M)&tFlcm}$4w6l-F~l3W;x@j~q7`H9@p1T7y^wINfStk|qOgKZgicj=&g^SZ8t zlJGT5Wmom7ZG-4oA7!)hZfq#UC&AdaXM8siT!S-e82ZcZbJvWf3$H=*UoA^^US6LD z4%c2~bnWh!+4;o%IFg#u-}I>~GuBs^^vQR%dGmDmS(F)8*nQS$-eiu+r!W#tFUv|)x#s>^>b?8s)epgZR(0sm1NfAl+HpSNFkQM0oYFkFrP1Zli}ZiK)~vn zxa3^=(6wzsJJwZ#zA6w~P*RQ4?yb{$^fobLR z$c*Y0?#K)R)%m8p7yogGEJ`Ff+KmPLVF+Z?Yc3E7Zy=A~$j=CWXm^Nv84dJEe@&c~ z6%Bz@v*(^HJ!s}59>E%say>^e6Ld^h?2nySHQ9^=}v@G=)sXZ^|#WupDm|LfY8?LO@%?okCHz zRuYM82@1|$*-xT930VCDVWMPh`u$Wa21;08|H_ zMD@-29NHJ3a)&CUS=tct+C9y1^jt+q(dYDprqm$+*6BSIrX*G4vrX5b?sgHRL;eUW zI5?UekGJ1_jd$kMAAZXM5*7sA?(*a(o(W3qkWywgb*;r_>UAKW0`9l}UcV_wKgZSWF+esTu3ipZw2t~)a`IqeNPBvFjssM%KyOD z6EcPI(Ds8Mc6Uk058t;hvht_(?Ye!Q(%lOnULyaHqOnudn`%sVpC=M#v+?ap*@%<) z>{#Bw3a-J)Y7T@NzRfAtE1F%+^F3*pWZ?h3v+?AL{(pY54p-px=3jSEfHOHx1dKe_ zOzm69G-Ac&b#&Cnl&0}~D@`0exPdi@Vm#?M`(b7_1Xht6?ut|hw@7;35*n_`z5JZ6 zY(ogdLEADxK!1ekX~tX`Ubxf!`1sM`>)I35xuy>sm8!v`?|rsUN%i0plY#AQ_K|>H zPu@i=0b zv{ET*ol`lfNzWar0ZNh;DJ~yd1^Km6PAJR)(&943I;X&d>aUe@X0n}b?8-B)!^J6C zDnc*x{>coENOlT$odKDwWtPyC)JhD- zJ_Cg))_%ql9fIk-zvyd1u}P-a)9l6Ls}GVj&M2C^Z23Q+<-ig&Jj^xrP&&p7Rx#RERd=w{Tw7+~QVG=*^W-4zUf_|hp8D5Wi%vV~16 zua}6JfcYDUwDBq3;Nv9>dav>nE#ClCkKy~6b#?_GMMr5p5anPaloKQ!MV-d<@Q!Ux zlZATYh1=8zK|Rk4x34vQp_8*FC+W^yL)ck*yq?7W3?pD58ItlXgIW2HRClCgu(x+J z$8kSN;(zg;^LOKS7GUE0&s9B5s$xzZcD1ba`p>>&Rxazc$f$O&O-XrbAvn6?sATnJwC5tO5>aYttQT3$?8A zkZC(5mPjZqQ2qA@p?|M#W{NY^D~enqHI#aCmu(YSuSruRh2{TdMJe>mJg5a6SPDnUWAh#@wrI$K_!+q1gYS9|tZ^E*4rp_!A0NK;l;gGY zbdeD3Uxcz-xVWOhj)VJBhU^xoR_C>UbQW?{thPr0IP95)`UQmzRuKndWDC zpb|R%x?r_io~E(5>PTLr+s2=x3jV)S(C;hK&noZEQ5R?(=J=$P>vTMOXYUK= zHi7h)gy4riHV>k&^gKHunYO7b97#RN>y^d6D~0qB97q^Porv`^n!k6yGQ)|ch@~f1 zMXd6ys-04MF^$-q^fvSM1HVFcWy)ni>kQ_l=kcgj+Mm3M#^!scoQ62UezcWfmSj9p z=!5S?zfP!c|5?r%0<*X=T_9&Xr#DIdFni(vogDJ!uwy)%rfRCkp`5}R)*Su_QmsEd z=O9u#j{=aic6M1;j6IZO-l_W~P%gkpeC>0?xCvCRB}VcQi>LTpxPV0nb1F}bVaRscHo0C@3XO3*$XPq7C)<$h$u5bQBd}B13t5& zRDq~2TVF@@6n0Vks0@L;TBRm{NL&n)(?L-7Du=S#Esmor*?+FXc&W{+ZEl4WzFTpL zAkvPCNb2ZU5kZWvyRJgDAEj1L!GMqr2UMFqtBd4=59Cm;p=yL#b`(5@Hf z=Gu;HA3sk#*Bb;LSkYT5I0oILt-YGks>9iC#=4|G+BR3ujfKreeA+&CzeGlP#>t;# zw6KgkzseWWjG11K0-sPxY;&zL5SNxk8lQ6dMe74o=-lzQ0Lt`a@OHO8ce^-{c%PdM zNJ0Nx`zrYFTg!Gs6%-#ovDDj=&Mnm1_$+Bc@_v!+os+F+5d#eq*Wq#AP&jWlYu(xf z$VRvLq>heG-q@?h_|H1?A-@xakXJfESzQ?9pH&O^oN~{A7dfScllD${QnZnv=054W z@<0O$iEL>1Rn0<&mE2LLBeO#BtAe}&7! zfTiN!A2GY1-$@=@XydTEybZ0Wv*EvypFTv2UylV;#7mLaR^&A_e={4__`bVrSVU8# z%M>Vj!PPZ)0C)5KyrljY>yD28&%1lR zp6{37fepg+OiqbM(I-;8#wTL;xR$%R@NUe#rciXok!wS08=N`Y@I>Cfwj$~m1N^Q{ zu&#sGI%6_LB{YkA^f1;Dy*kqNbl+f&KhucgloGmP*nr2bIoqSwp1!|Y@=^g82=HK{YP*-321M-v#+tO0unV+fUrvIL^;X<1i zIvm%C5GTkj)LdWh8T=X$WNsybvhw!`>oKromk z5%9QZP(^B34dO{@d<};@r4wu)*U>J+}e9jZg4MgMVka^8=YR% z?mH3Al~^7+RQ*i{8EgAGhp`~~ZLK%r$N?tq-++%H$F^~P;|?`;H+<-l^LN<)YRI@j zUkUcGF=5NTxX3tXoryKg4@h|anzf^3nJ#{oN1S!DlXGj$>1@`TG1R_P6QAnt(f;>O z)!>(X{#TW$Gabt!qBRiLP6)>D+79%+MI+gq%xkQCphqy)qCbfKa|+UmAuMY{h{YyvRh5UEeWRp);M@O zSL}easH+Az^bZ2Cqv~PJ*EeWU%%C|mwE*-IYR4~+EPkr|NJ`93pAN}6kWoThB^-r{ zI;o(~Yh-IH35VcZ84#TvC1&lL)uQ80YR4eAr?q0sSEI^-(V5zcK23e4WTqP)ZKqjt zKpOPr90{0aqOUo6>Mxw5J4?&{CN}6A;23&eOjr5Wo3|g?id86O3|7_bVM{gVz*PGV zFkKkTF;q?A1VcU*_e{>^4T$z8xg+&iXl@jEf$D7*xl+ib;Yvy=WG!u(=4j2qq+AsL zpWVLR;gb%=PPl#Yy}GjR)wkEP?b&KNXnv#6P`}o($l`VjKo-oRbs2t9yCre?v-V{s z{DC6pKzK%``qCFW0I>uS5SuPl5fs~CS55l{KXk^=TqQ60db0(&wdCB;BkcOaMPOc1 z6|l{Xn!rRteB^o8`NBCeyA8+}z9S2a8Z_PZBZsXtvW-mYWj&&84b~dTXC4$17~Ql! z(^0{q;JuxXIbiaFAiL=$n$c_&v_)L#FS1`UdtG_(^aMbw`dzHEN_s(>Tq!ulsA4V( z=x?@{uX&_e&T;_Ss>R(U@TOwm=tpLd~qZM$A+Fq&L z9uFqIyg@^pEC2$fsfDT+RKs7H6ydPbN0Xu7~o;verqPa99NK{3JdsB`O06Zg`%0hjoFL7b zG)4|(g8)DK)20q_`*e1Aj&(!2B^e#6WmX_UKP9CO_Fm5~+gne5GMpFcfZ&Wp9;-ZE z(?YFt1M7=YvY2)8>N+jAlb=XX&eHzm06SCiFYvw2B@NTxA^U65Odw_aH(LOurf1)+%Ez#~c3KfW>7I4aLriqsfK#*v z0rteA%>Wkm!mYD+T5@3MK>t_cP?@>$M6Gcc{28s>|FPrqxoeuu*c6;9FZh>h1G zZNO6 z09|z%`uvEi7(h)CTLPfgD~mDDzL$EE^X$dGLG+m$J27sIaK@Y!Jw%s ztCuvV0P-~qK}dMetLLkRF9F8F8(-h0Se7N4C`XN6D9%|nO#8YH!0@<9{h zsuI2j4`h+3-#(kXj(8ut`z3VA7{Lm8c{@ zo2#?J-lNtVp?+y7h1ppXGL@^&lB#Zm5ihZ85whY)H3pzNK!LjEgj6lwr00ZaOY_9)o- zq{_mO=FOJ?ZEI^EO$Ocvj*{H$qXU%o(X1#7dSnNl1zP3E$Vb9b+g#lN)>jMrP|{i~ zz&C#BTG%{dXFo~4{RN7uwK3NC#4W?fF z;0`AQqFm8<;|pM3%zLDbjFyBdys$7V`hD9Apvi8L9H%2w zf8d!Z55(se&;j&w3(L0`ET-qfR1Tw}XsBMF;=zPYg`G=1ruENfuKYRoF1rE6XCH{G zveo~h%LjMsEThp zw%;Hy$oW|L83Y0lvLwXTo{s&7QvP{uRIg!&un=n2-Gd+%_86$61{DofNh|jAN_?Sl z{mo6+w`gcsz1x*)c8~2Tr8Sy?r{DfK=j?=l6*Zxx=Y(C%EcJFi(C`_d3qR%Clz_W8 zRE^-^D9OY3zn?D_s#$C7gy2~|^@w~6%M+A!;NY^>aqw$mAurHchn-;z; z^{gx2R)&Wyy#$fXxK|Oo6xKI-E(r1?R{sQ`b9|RL^IZRHv_e z!IZojRGkOwu)I#(W9Ic5e8qm|P}^~IEpR+^uKi}AeWVSbo{j!XZF#?|o#v?QPOfT=<}nOc zIwzM2D6-rpO`-o?mYRX?d-gdrnK@lBx!x0SE7dyqpJs61ETcx2>smTZty|F?fh4oK z*ff2$%fmklCwYs|>r}{9s`AfbjEp4sPfQGBLODt+|Es7jXczROl8N}uj_Ps}>kTEH zlut(=^P}GWpgye6U4@5qhyc7{HZ4jnvkE^P8phlz5DwAWUwa6ds`GQSO=;jgXbS6nFJ#dCmHLA!YMEWh7Lam=k z{Ow6j&k(kd*Cv+9v!erZl#)U~IcQz7)bvTfn{OziTwE0*v8-n_?+vt`;+Iu=0&dED z9M)i?Ua$;yWKn%z9`hL26N;H3nfd#nafPgGpo&Km#<5D28y?(O7d3jyZG)!V9?%z> z;U@x`^kst{6{w4YyfMY4#z-)f%Xoebm}8>lq_AJvENs^XG9DL{ji%E2ZVS73>HQnrIGrRIVG0?*rU&BP$3 z&hFE>F@JY4iH;Mfbeg~If)PZ9@GDKLMvZb0y5+g=b&LbAI{c%#_PfJ`_V1y^Yk>z}oFX_1Ki&+PjtM z3vz-sVA3J0-&+A-=9Z0R8v6W5qoZA5@6x&5JaIeD|E4Gu@i70g)KP#rJHNN?B z+2}SYU2(LI4kV!8@%sSq;3-OlE*2NPV1 zjCqaDWdICq3r)Xn>ZNh_N8`-c(8}6fE8$-!oy7jtv=Jsdnh#U-z-DI-Gkak1T{Ui6 z-6z;=CrR2jLx*9e;#~di%HzK1WX;_U`+L~{E2h-~;H=(FU-Rh2%xv?T@qf_6GP4D# z^}DzF{E_=k_8Ph}^%q~qfSwLjDr^n%OFtjqoxC`LFNXttt`VOTYHb6nULyY`pCt`o zZ7LPsnKEL^ob%nvx;Gt##lq&^s(WQ_iTey zjW7xLh1h-U8Xj&&`xi;6M z_^W7#IO*njKsRwP`#@QSqHVD|JGF?)hJzbqm1OWtCq-@T9K0}faN)p!@G zRNU2Lly>Li0v+SK6kZ2|MKD-8A0%<*0nr^07j&LEx-LnRphslXDdyFW45=cI zt<66G=z6mRl}@glU*52SX4_;i89D)Z`J-X#57`GgLgY#EKsDb*!#?D@g{g@qGTBlu zkz;NMm}q>|38At3fSG7Sr<{*qtAm=@&V_&EvhMh)f0FO{t_cneF8c~9Z>EgnD(A9J z3UCUmk2nmtu~IZUM27V~9HQHhU&yBsC2Iw|nUqiGgBXDje;=Hf&%(35K(W{k|FNXu zYX?zOe5z4Zeym$=MICeh-Dh17?$14gQSN3=eY=Ni(kA7ltM#Fr01k7cXbKZbqsWVM zS#`TxMWyUF5*+J!#@$1c;}oW!&6a!}C6RZdFh<1{(>p1xPVLs4-kYI#&E3J(P~+K^ zQ?r{FA|`Z;c9otJxt_~w>06o~0gO+9pVxe(Z7n-%*XbT$i>W}ztLZxg#ZgvKO2QDo z9b6%zxR6-K2&-)5(&x<#s~Op5E2*5Pn=MtA!djNXOIc2nyxDebpt?Wbfd>_;`Yw@e z8D^8^AVN080X4J&o*d>FSKZVf&O_h!AZI*g;q;q%8g$azlgV#`1mXq+%nbEtF0Pjn zPc{H;6E057M56JS{OWWmwhR_;t>-gLWURgA(n;D-`p?q+*HGs|ZrpI=uT6}bWRffW zLN~mTOue(g#z|&m)-XqQd<;Wja=XOm_aS5-m)LX2wVWKqPeusLr7uRkLs^SLoH55! zS)f*CAEY-+9$;)yv9oj4B{7+QR4e$VvCU3}V8_?oY>jbk>HHBW^{aG?SeR z{n=d{Kbh#S2y;loT_$lPL)EAYl@v<;)TSE&CN1km@mzQc!HlfJ*8`sp*Z_gGevYPA zMqhaS%uh`o&Ptp#4`_{HK=a}Tj~#>C+>lX&t{S^5rTKgjPJ(Yi(VyR#2+XgbngI2_ zEat{1&Y6J}DbN1`6rARIO9h6qE%fHn0&X7Kt4&|`(&Y8_1L5AO_&_rE zYL>maUueOOTEBG-pjB12C)3s*9*XIfe#-S zI@F~epNf$_qqoB{TO4i%TAx&3BH_}J?_>!sc`CzBS!s+`1-zT2wh$P?$Ep(qcjJ*6 zjshISbzPG1y^VNZcph2rwD8*F7>W5#<6@dY4Ok&ftM3HZL4k$c*lc1_k6fac(Kq3; zP`T}7>2%wg*)PX`N_*?FPa<|x31(l<#{LKYl;Q79J zo98$a&#c-Wn&VIR6A#jqkt#(gN2yotlf)NAO~0(G^zO-YiEJoNN@xhTS2NabZ|Z44 zjbdm1@O+!a4}m_>hq)5m_H5DBg%_C3gynf?MEE77!@P5gg>XkF z*szfW0$o8>)j-30DR7dzcAGGx*m|e2q`taM|8wGV`Wkij)KTEVKbOIE zoVek>T%yW7@K=vA0)Ks6HHDr6W?!3Ja2auUM#Nk1M|7gBMnBw{miIOSEx4Oe`K21o zhrM$=0uxoB!i)s@9AnSi6GSD=Ha;sa?Q>sJ-b$#paKP*-r9>}wwt~>-C+jwTN`sGS z$0tQ4T+c-xsJID1v7`0gUghZ^t2;0@_*$#mM&2)Vx4#6;?x(DLlItn1@4H=3e%${e9$-FrB~D-UbwDFL-lY#J z>-rX1ckelvURDJ9IxUmxsHCfyT_yZ`i5_@pT%V|;NN6a*p(jF_;%s|; z{e<`q|F67L2ONHXw%o;^$Lw40JcVLWP40hwwYrdMDtN_F;5(>@MDTu>5#GK81fEF! zl$qyOukw=VK@$&R=lxUSBhKW5AGS$79j5`*I6C}7zrWR8;^pK5WN-RP&d2BNnywQg zI-iO!)HZxN7vk5yw{GElT;$1;Axg=uAk-zq8pvVlJf>qMA(#1!q@Uub^Z;}#TKs)) zoXmI<>a_U~#AVYndjW4PR%N%nU45L23XKu2d)YSYNv{%d$;K*l^K%vd^5OW-P?7)w zan7WCmwnbLn=1PkO+476rf7%K`olWpZ#f=9k`0Ft$T?ojlZd4vy5h4$@wjVN@)Z1V zLf%}M&99Kz>PX{mIE%fSr{gu!ygRrbXS*;@bBVqGSf~A}!Z(;+RPa!UJ!b0#g?2aP z1i``h|L^DjM|w(b=~Xu9Ro3Y(JuVAn<(~b(2M8j%_+~*X*S~vSu6Qu76!2P-`hBbA zdZMkwwUZNnUbBO@bj>8yW;PPN8Y0c>;UuntNRWtp4GYV@{8GD1>6xRz$`yC!MGQHD zlR%a@tHwzw86QFf^`>2|rujebttGO;x3^k13sg0%yB^bv^u#5cImr~15VF6NCi60_%0bB0hQdeb?6zx8_WBtLAA$x!zfh--1jY64AJeMQao3w9_nN#}; zmD=3YjY1^58UcO5ZSh^j(Q!Y~Lm?ADSX(Y@FV>xDryNm`yxlEUo^Iomfcr@lBhQ7C z{SlAhOl>DhM$#plMXUYGla=u9b`kt93)q`kdB!BxU-8SLRXwTf7JFDL!>TB?k8RT> zeCwOu`{kfzi2qqoA8)Yl*!selxrCh(+nYM+0j*1qbbH|e=V-uk3PxH_SC2!Bj-oYD zs8Jg}`Y+!y$GxHua=pF`p_dZ^t%YgdWh3DBMp*H7fGSfk+|J7KI}@*zlfYlqG2y;_ z7fGqLbct&1XDuRPZu^Zu2T!*9?Z^>?obI?ZE2vhGWFBdTp?aIZ;-xSz(Q@+Hz$P8> znHr^cflxB=R0`!?4ZKA^?k+Kut;l*sP*XIta4RsTB^XiO@^4+}l$5Wsc+4StKKI`; zb@dB;v+UR<*q-PClv573+~`N?5yN}@-+H9k)en4O5o~MM-sw!R5LU)!_!h3%zN-EtqTW_`EWvDS;(56t=M6`(`ydmwyIyr;AIO5n zds)5k9pLpUEtl_3hk@;$rrr=DuHW|$4b{K&&)|?8L|0?iPSB%SJ&7BxPpPfFF|vYDeiW3dhE>BpDaYeRtc*M zm&P043^ek@Z`-Br#`)dC*~QSGfeCM>4@y{+qkx^o1mc4M?;+Uja!P$fO@nF3fLk3uz1K}gsB%o;y((hkX*-C zJUWSXzN{J>kHK~#@guJ>+dH1K06b8x?uY)vzKid6ji4U>wpM3%go{l4kKZtNJQ@ko z-_@{R@tPxVfm2c(Kv$vL(va`--#rls@7B?AN7kWh$%U#qZbY!SXH_H3Uh?GIV`B5R zhs;OOP}@Rkw6yf3>MYpC>-T-XD90!#i2M$((aoR2JMiV0%dM%AZ_LK0l-ai@9@Ud? ziWi)|ytwHv2!Zb!VYeN0Bt#PKiCpNH$*P^9Oev1}Msdf5~<^wGH-$ubc}8#zFZROTX8&n|4Wc-8mOfGJVmLDFr#&awQrj+ zh~W5Z06XW>PN57E4m42hYueYs>|x%jINUfJHP9OP+1-1mD?d7%9ONg+4wbJYk)xra?hN_g!{B206kr9|+YC=DO z)(kMmy~Z%^wN#a~>mM}q<{jDwP;ps}G+iAaUcbM=L2X5@c41*ZiOXF>=OBDSe-|~@ z-j4_3&}V!U7e?Cd0R(RdbOnPQzap#gbMf#xA|R1ko*`O`Aq6D2p1E1ClQ~tW{4v7K z-|H+*Vz$nP8Fp7#q511$XxjA11<+Chnl(<+4{;TTNJ+lAemB;gQ1EZsE+j2X6*9>- z(t^HZULL?{MamG~zNaM^N)(lw9}qgSolv-LdstK!cfAsZCls)pkGcS@B*c5G+w%hp zwPTa~Ge3npIr;%)vzl(MdFHR>n?oU=16$*TKJvTs?Axqk+k&%37txctdKAzvtId?_PLsiro*#s^ICG`qAp;JKSeQ6H{`~AP-?dnjayI>T5w|F4K-g zudHo^0R)0{R8%J|+nnnR0kbFuzs1#_3}cQ+SRy!QByRG&ao zYqkwgo;kqS$?$V&J8)cBSZW|56%A(1lCgAt_yuck0y7ID_#qRu&guXr@EyP?2~W}% zHUhr(L5*a2#D=a!VH61Yc$$F*aK1fXHn-;f-d)O{AK~xGRK?z(k6Oiq#f;;>hjfLB zF;!#$}C{KD)(^^BdpkHq%^skEuED7!$p#E*-QS?nt0Tm=dxl$X}>sCV^AgG= z3&R6a4FI)F`PsI*KYW8ETnH!CIf_mwk_V6sIkMDIr#W{m$CnRhO}sW<@aaXYZZ&lR zy6+3Ig}KoNh~Sr;8t>NHkN%)&`&sPE>>Mz2v4Q7`XdQ@Y6qGOCd4$&44<=w{cxqVcu*LYoJ6 zZ0)4PiD|hoT2`2!N5dZG(7N@^#i3KI&oBThW)+k`D-R0?6lm$h(y_1|5V-%qhH`53 z4YGjtt3!=}qW^A6xptz*>Pk?{W!+!Tqg|X3RmbbPH9tH2IG;ZHwFp+{kymnO zsCsJP_z!H(c3#NUQ0*9SiwBSziU4USzju!TL(70H53JHbWHN1Z3b(jkwoWuG(6^xD zm+i(L4UE_y5!~<{GTXc~E&9uOuy(J$Nh?c}UFdcEYf%*gFZh_xB9+k^8DPag05yId z31I8$1^nmK68~EV2-(leAi{wmOU;~ngKcV{siQoD1{7H)MO5H|3R)`PQfK>eV{0@) zU)B@WKWH6UVd5>1NJ5l=HrVYlH7QkBg}P1C>cn|ZzUq;m)qU}M@KWN5MY4cHCUDd% z2WTb7+RVuJku?X~x6Fx2F7+k&{v0x_WRD@2rT?v_V|@NydjwdOgoXh&NyZ3U6uE~> zVx7S$>&971V~hEMY#|^2dUczEjJ*5MDEhW8Fq1;bna=^G*<0fSuO0PXWnKrX3C{k-oPxo&N0Zr36HmF8>aZAWaS>OKv=UsejIyR@IZFX zC+zlSUzK47M|1vYY16-l9_?baRA;}J8uQUIoN#j&MddhGis zZFSDI^YF)fRHF-N)l2FvKonKgsKnp7vC$>qg>R(zaqttZRoZC|OFwYU#Ozmm zc0zXo*ijEi<@AM8-k99{?w6P;S5(tU)pXTLOqTyza|S5o?ODt1P|eC3dU0Rf_Dnlp zKuSCoQwVr5clBnCUpDNVKT%v78t)`{#`dgBXYrFQwPbK@&K0H4x=h;>T(;MJE3AO< z#{IxSb!3mS_KwyWZXT62t=%>d_=R6`+Kc(#`ecjM;X6yN7;<@f*qjQQO!5L<2hEm$ zi|Nx1sE*;^e^)sQ2&0YEPHY9_6=?nA<&1aZhVXNS1GMh&iZ@R-;@lsQdTWmnfpoVg z<+p{c-L!{%Xdi}z&*B>0EFxB9^Q%Sqi%U3UJs#W;PUfhJ~Y4IZh@i$z6w zQ>!I7itDM)1k!oHBmLGH9 z+0$l##^01}vimk2N5w+Y5Rn;m_(Q;`<@r>P=+P9qH9x@-Qbi`nbQR;C)}hP3eOLUc zADoFSiYc{`nkIsW+hl;VOT5x`N4t-1MzBAf&4rx830WKWG)if=OKF?ShjNKrQz6Qg z@(($2N8|-s?AaJfL7*wXuT=a2Yn7F0W`#{;0uU`{kYl8}l-ik0?Ah@|$tcOD|7TJ> zDcE`pIh^CQF4asfLEL)+Q0X`KLwS8o60v1T+3rUtZ+McDHgnDNInF3JaP+ZQ!MON&_Gr~ zu8rvlkmo#&93)~{L$;&FM31|Odu|+uB}1FC-U7u53nhXH%%0{Lo>TZ-r<5X zYX%zK8CzRy+mqZHMT86XzU7OSU0FY+k=#p=nsytQED+baI~r?LxY(%{4Pm8CGnREC z`3~P%ah@!h_!d(7JKZ9Z@|P%yL1fOTtyJBJ$%dt-TM?&WsT+S5E|E1TkQ_If{2fCH zoi%Cp8MPZ{7P170EL%b*-VJC|(L0D3n``bUD)S4BXYlpVzL30@Qi0NoTsX7ZnrOgr zU(i<1fKewelP!S4Suybpkx($WlZcC&R{e*cu0HX38tnYnpzDd`aMS~ zCtg!t4H7Xf89h(yruc%-;@SV1Yo5Kl0j!&MhU51%j&?CRpzfMVWTCj1nD{0^b02*S zRV>V$t8IR|a)zAQq}+++;&D5&mz&%CgSkCJ!OFXYOgDkSm!Pjmc9w=xT{BGcRUk(~ zOGBp}@Y>W9J0!$LCz*06uXjzS40AugP684AAzmpg!Z6nvw zWzHP%!oV+HSMhtUh}GKAxyB?V^jzDKz-8FA2FFmn0@ExqZvf z0vM|nDCuGr{~upp85U*NwGDzGQVK}J5JTsHbc>V-2q+;XEz%&RG*Z%yq!NN6odY<4 zbT>nHNeM&E4B>ap?el)O&wG5wclggA%ynIR@3qdg)_I=W&K?a?dMt|>{nos(_V~WA z?*#XL9=(@Mmmp2?(qmxk<>FKrXGOBI+vlE)ZfZMU@z3;5n-CDn{`LN2i7buUk4#kW zze?+DY5~*Yn&WE3Ds@QR_APA5O+b$$hzqqhFl*J9*%hX6yN%(-o!TU@)A{m!H3$N( z(>D4gcNSP*r=^)}xaGM3wi;CuU5wG9eC<+WazCr(9$qpzu}3X0XOz!CaG;RWwH$qS zOOFEG{Vp1N-l_>MpLqM%Au`+#EY-V1v2KyESm5>2@>h!gT%K(B7C2J8qFXaEwOl%T zzCQ=2EpO|5BzPzECP?ce$<$;na(|OO&RC0EO}4jJ>O8qf+In|c66Java)v<%M&Yhc za(IPW&)v+PhL06revy2rZflY{^^{>sGIAhAQ5?;f!;D{DT!NsPdkb*15FMC26bAGa5_T@-@7SRzSzjadh%%BTbs$? zMGUCRzK?LD3wJ2^WzPxq~PljBMQdju`N`V2}8VQdwv^EZuP z2P=4egy&D>`Kl!gzv;jX#JJ3y@**c|H_JaC$5}_Usu(a?Xa9&oVoBgw$EZAI-Bw6{UrqZZkmQXFACFRY zzX)BOP++V>#i$zbd+c9LH_=taMAqz0Pe&HX^=c$jQveT1>1IZtN@tfZ8DgX?G^BtP zle}#_^}AcDv;ycG65{Ia%{*mnmjhRfvwbw9BOepsgvjc+^7eI$Y8$Ks`Cl41qecn% z)~eLzQ+d_n2IrEBtm(bqKf`*UPeZG> zgw?#2AQ)pnt3_+OIu?PxU-^+Os|0&?cBJBL=-zz>?eI~W&TyOKz17_NC&`8~SXE$4 z9%J&r$;#t0N#2>8Ivlzl>5Vb-g#_1ml8-ZBGAAC?&wEvjo4Moqr0h~Rf=FO?Urpa3tu!md3`8vvD-~KS^6-YKGdYV)3<4eThhs<5jWm~U*6ci{Pa@kdc>UA zgcO{8A5vARqawRAtBnJSmViC@Vg;?`bQu`&0=JTc-3+$movjL`452Z3X!fyTmLO2k zweW?}yxqE+dU{t%*$r&B_Df&&^CgAXHe4gXdUCN3{_0vHas+r<5pzp8gW0H_z5r za2`+IZ=&pbcMWq39;Kf@Vp^Fj zwbNMv^f!FOZB#L|#D}uL;NlK+dUcEd(2eQ6e@-RBzv7qiPG2m`{-3_w#o_W(F{Eo+ zWJ$xyXgCcuFarnRSb>@Sk~YB5Y5%dK8W=}>jBU+6hx2Ungv-ZmXrxIx=< zsdCa2YB#O#)^WmCe`v#FYddzhW%o(5SvRKw0-Zw9n&vTkBX`5Y5Vuvj+ka(=-5F*n z$%dM=qAYv}{5M1{cL8trz^Ah&>mcsM^E&mIXl?mvCL z-DYBFXm?(ULQK5ZRe6e9UFL8+rpJ@xa)19Q)k=wds9I8vk_JCKQ!p z7f;pMhs*~X!4`*5N99Kl96!uyTYE)XS9rW1UeV6rdy>}35rynESb8u09WK5i6MOPw zB~lmPIPP$SeE1hK8vm(p1CA)5i#L2F6g$vM6|rOq_*ZWQJz+)3w5jFM#;T4LuYaS4 zZnJ3=4r<*!e)jcZb3S)7I%yxw#P{6p{z}_IjJ$uX!+QRnWO%pdC!Dow?9S9gcdZ0I zpNF!#9;&H7F3<)~!%IfLdZbI1qcj%^q*GK5;eF11Y%QNG1;u`HVQ9AtOhG5a)|D^4 zF1tmGTFPo-%9eg97jKAmIo_nE;`FPtHyp~k67!EY&sMAphU<;?i1P$Y$|=cN7F8y! z+-OFmlsoKqX_J6{`U6~Aho)4>^~1VN!>U{HYF*l{yqnmBx6Z#xuG!&FRNh>+513op z?%|mwdK}BuK%&uTu!sWmxZl^(pg3A)hzt4QRi*E3VqF&okFFxmCtIou)#8+QbQygw z!&C30dJLLL3yfu~yO}(4Yn)oJoNDf{FS0ss8&bsQLp=cxoeS4PLgJrmUgx~VO)Sa! zua`-tvv6x}#(O^e47h{8<;0==*p)vN7YV;_FS3$*EMu3g98Q*$zqu-8C^R)r z;gd$}?+TtWo3a+$>Dcb*cMo5%JSWKgs%ZE23FXQT|83hos^@_4Pb#GNGE8%)v2HkL z%NvlU*a7?S^vZeYx7LD)Qr8Q!X|2;Q*?M&$*=Cl+JOu(BF?3zG-R5K6mAa}Mte#zc z-yqri21eKD&9_nZi*qtPzmxuLF?GzEE@%7W<4{+wt7GV5oAcO zN%t9J$|U_VD~DGJ0!PJW;BMco3D@;C6B;3B@mvj`djbQ*N{E$%<$@^R`MbZ^@j*|a z1nu+b(re)@RV(UHkEg>29Dbb)wFmB#0r(D@H)i!_`B6ztjXD8=0WHb8N3P6F%h-Mm z=kGd0ql)oniU>7XJ%7YKw!5S)#kpd%ZM=UU|BE4|M+R2L%aELw1};Xmx10;>F_|1F z=8z2M?Cl2B)wk=qp$`Hed1~sO$Q!D#mw_2&Fb-+oXWqG=1GF-f8%D}jqAu0Q#k^fZ z^#cyk(zpV|_wBxxTmt+O&qmx=nP-zoj0~$w8W9s<&k1_N-KPl2lhz*MEQY--azG_6 zm<7B0$^7cV1s&*+alOMd!RDG!;~rK{pIsP}t@SS4@ww2H8k?4Vd2!nYzZcjQ)Q&yA z$ED9R@McRyKH$l2xv(z0?_7x|mA0?Q{+qIsY{wIq>iU8^_ik7ok5<S>7Vm#Pmt!Xe6Y@z(ObR7-LxyS@2*2FXDZCxEzr170rPQ?70I81{Le4%gp@ z^S*9qhuo7;viT0a-9G0+;S`cs{2nvJ6pu+?sXKJ@A^vG-ahocK7l=M0kKb{fZkE{m zn%v(t^xVfd8P6$(Fs5NuVR4NSgl=kCRRHEo1B@K5x+@~Q-oOlXx|d8fQ!gyqObRMJ z$Na0SJGy)wxE{?Cd}5lpS~^e`%yQk7A3LoJ?Jf@$dLn&1Cu6`er)N4s|7Yzc9jg85}Rs?Q8jVHUVPG8HceCnuad+FYTgv zT;@)`rNi#}0D_r?vU~aQfC(V;iw!G{!U1h*LtU;no}z|#xje5Q2J5Ima1XdeR3J>+ zSlO(F^FFo`r(0AMYGIUNR9lW*ueY@`xbm)PQ^@v<7C)=!_q^<52f z3{!zj{ao7m!?i*sn{5@d`jtV>cz}J2tP7uomu@Uke*L=17nFW&%`4o~$YQkMFKpuc zI4b@2V33%dHQ7f6^YMoBA$?uOkDq3PmxU&qoF3qKP|GkxJSbWD`H@zK&xKZxwXQ=; zLn;&&k<-@tetLP?mx%3|siWg`{pzQB(JI5%&aqxivaT zO}7E1LTlb(N4G}C_q0(G`CRZ*K9N7v2qRbyU2Jj(7I~L)t6tLlWe7*qV1Nw zai8?!{QlfemBAJ@*ZYBF5gZ$1X;-oJC|eP|;lQITA@u|+rNa2x16xOHQ~^bGBYONc z<^FQ%+K!rzmmKsB+Ri-vKtc*Eijr#xB~ix1L@x?3x_7rCr$?E{t{UoP2V#{4g}B5Pd3z|#u6x9-84l2F@hI6hgtrOSmMlAQ?@zxk zxWV*h8La&y=`&l-?+AGOgAh7(^RaiS4a@8@S<9~33qijWx|@b}2yVfd9b~4fT;#8S z+tv+?3Re_<*1lSWH{{Rn2qL4#nLF2dPoD%b=)tLv0ttH*RN#UrN7iMYnAp@w=8dP_ zrv`?r_O)uCOh1&*zloRi)j3=CKF3W|K?ZdYl55$>dmpy5uC#r5{Z5Vi!|#BzA(5%7 zZ`^Z6Za4fs{d`569HmMgud4d^NyrO9|J%4vgCo;9BlROjKSExOnF@MeJ%o(5l7`+Q zdrC$|zD9aK%*^p!u_|sPE&0Qj2O2njEms$?H+rYK%_8aU``@*mdFby8+f)r@0kXs{ zsZA-kgnbqi{=5nOU}y!!bZ~=GZj998^w|tmJN40AACnS1jTu zNGfGM4j3i-zgCN#(eOR}i68Vc=_Nue3m)Y<$Y#zO$x7)N1dWMXFnlHZIbegw47%6l z#=!P-EycuG>U*r46b`9#0+XAJtqR};Vz-CF}vT*gXW#g%m?#z ztxjBU_ws;_!K=D;tUdO%xUHAd+Yj(aBKboXEIo5pvOCT(lyiis3EA_q{rRat(gqF% zWoJ(fWBRkWZ0f@|ac*@R=OvnA*JiT4gIG5aGe`MaKhoSZv95KQD-{L0k)(01nm>|@ z1iB>5eZCWNq?U6YS%$C$j>}bO{XUVX5&Q5|&i>kH)ruu(o$6iO&FQQ4v|yB=?du3~ zi$kR`wacv5j$j53+nkdg@h@uf8gFPgP@Q3B#{TdnAxo_&5O_@|pJ3aErYo zd-J6C?=mkmj@q+qI_>WmJMfsckMxWB!6RD?U3zO%-qA|fS40I|J~`>+yEo6GFy=@0 z`??qBT)F)xX2?}-Pm&AuB)!CqOS6^;9(ayyI4f>&L`74hWw$Uy|~x$+Cn6=*DM2? z>l1Qop8`a22IYlEi0YlJ1=Fq(TYuBxnL?#-sCIOm=#d3Z%RtiBb2$7HC*mJ?YpTn$lExDqt>)0I41O^?cdu2Q^=Ymz zS(8MECTAfRlgd6*lU6Sn3r|wAwt&2?~;QOl0G8h z@B9+nGG}89%ZuN0CA=4k+rRE`_uD1Yhx_L1ZQ#XUec!({6SNwQdW+nvXIQ2(NmO#u zUp5^!iK1?{cpv`3`>Pyb`OK=R0c`TU0L}8xYeR^_`s}7hCir|9P&uXHk9(UGxNWLr zzO`IQys)~uWy-Kv)5=E(-kT3T+8ICp_0^K-g?1}D_NIrq^b^11pZLp215nnR(#g8r z1)4+OH1!Mbk5{)Smme@@O*A1{u}*}TtV}kETH$4vo3!-Z?E3m)vB zbX(;o#H_eLfapF@(vbBB^)efJ!P;qZdYyye$Z(}*A?`tO));{vl zVg!Qtw@Ja7%_N9nWXI~c(U*)Iv zIf`@vXRKfWi-N;4ea{T2QXqpJEQW}FuZcbv8=2H@xQMvr*2}sK9`8xA}(6uZ_UK_zw8e1^=XZ$ zzgy9+&eAP%%P3KvOnZ@5er!M&EiJ8#P#f#!=OftrEpgTO$TOtU2at@l_j<7afLGb(jM!LiF*7mHr!IFQ0w$Qu~cS9c87 z$0Q&T^7OtgZ?Tq?mFYXiH;_cF%s$xGsk!t=cW*>u|9)1n&3i-161`ad$=J4MRGrS_ zXiL|BlIo12J5BTVbRU-V4@0-8yNDTDA5u6Z zocR{t2bFPDf;PZ<^_f97A8Eu1hz{%HL(W61`uh$*O}V})+`nG2L9MppaFZ|iW=hu` zCBRwIep+Lla}ugrut`iv#c#$ML8CwEI8| zo<}G^23^(E-F>ttUp20h*mVFt|AcrW-{l0|NM$p(#90m-m-+%W^;f(nmDN0v%b>h> z*mjk>CkO-hScaaqsJZ?jR;l;REy))~J_{60b@Tr6c|BCrE>bttMAWz=h0t;r)pAYq zjq-W#on=4U;sn3TFwtHGn_F+Cn&^|sTY$MZeArw!`)^N=S}Wd^*?h>X1{S?7;s%?* z=T}W_vn~4YFk-=wB++8=*KJDo70ZCVbb;ylH?35^<7*N>9+T)La(FuV^z<9u87UOh zvikF3&TN?A?y0-RW97}WZ@ciFT7wbcQ5mdJLiXL;aSjvtpAF{9*6iK7sHXX1YG)N|-#>t1zR5#rG!?tO?JvNk zdrq*Q8qUu>g=Z?Q>JjJuzs^$^#=Wv&q$%Mk70lY~va)$wvQK9_0%hG-vAI0WQ7nvt zYSB=>K!Hb0Hoi&HZgG5;6ut%xLzNf3B>a((rNGW^G}3nK#IC62UepiY8FO|uHd(T# zPBF{=&#t*%sHD$vBBumk?}D;l9dvHM^gZL$ELM`d29gRAv5vWcyKrb5MqKtlhrNPT zmF0k%_Uz3?>#fX!RNSOFss_(*8+VN3I6rL1`m(6!RF%p$m9+Svcb#SmH$87Jf?MYg z2Sx1RETH))b+QFB{=?Kzm1{K?=apPs-aIJC7iH@+b@Gnj@U>#TIAU$`dmd`?3_aKY2vH{$Hdry}|?!d*`6p!VbQNysa7?6?<^;bdt*NNo`%J_A|VrBR#! zq1$Ofq=JU;JpQBLE<6HHWd4-O4u|Y94i$;-&9wAsYI6UR6>;k13D?IatD=m86>2Th zEUX_`es1dBfF-C)qeh(J7m#@AJ{iVDIIHI&^5%;m1N!KvtLu5rrCCY0t{RczXQoLa zX4CG7Ail*LvvQw_TKSsW1oqcst{sRCIh}C6$8CiqK8By8l5=1TaSEv2spLC!ZP|+PZ0;$k- zNI9P|2=PYmV*2R^w5d_?QSW83L~8-Nvv%%VpVv5x7H>P3VC z$?wb{UwjSv_`>;uXW!}D!-9vo;iPAQNzt!ovayjN)$5e@4Oykk=t(@ksn*lNTziq@ zLf9^+XEyd)WLv{^?aVzKxsGAb9gBlMz(viXwIv&@JFncXsOdy)G9>MtEj~owKpEd~ zeJuC;c(dDoubYyp=W!3z$*LFTXn#_beU&`UZnHL2XZa-;xv-dxjUbvDYgTeHNn<_0 zhj0};L9EQB@496bwEyku}WTkn9IUFb=A>=PASQQ?_T-nbaUe%cx~Nye%agi7?_a4;{#MrzDEgMP8@&2izSimT#oCNJz~f6Pp_6Lec|;V zAR@TD)LbNc`h_l*-x9Au?=`fU<^}Pfe#1eW+ko)zA2LM8d*plOOjkl-d&e6>rp+YZ zfh`_8DDQM*UG8_cr}v96@ToCq{b%cEFP5Te&+lf7jDf-7MAmW)oWqii;z0wkx@DyM z)ctjhCc`v=0s#uT>Wrlf)n<;#w!kH2C2r_w!>ZEg=WlmSx^(ngo^?Sd|Ab#f%fJ># z*~4~zaw$#v{JCSmVR%b8w;z?!Dyi#6!hr%MU3RxOcN$~myaa9N@wU(>+9D!ZB%j5) z-_nxVbH7`WUmQX#MDoi|rkAiLzPg(fWy!a$=Wi0%7>39eg)({&(`2H4sB|%TX5V;& z#BSQL&(EUu+o-G5kn?*hZwgq#2xyS6Ukicn0pGhe&oo*!>8%v2yoKtPd^Hs_;Z9`lE+uu%*Cv zy7Ui3Ba^`6p7<%*klanhq4CvOYKk>>Q#_9>+Lo>$=c~0Sxf=(JV^msn9wj7e?3&0C zP0;LkK=EzJ4<7muXXd&Y+Pn}A6VlDWjOZzsO&<7@%k@t0b>(QnqZk9E4TG^jWuwzS zSmF+_m+3&YVA8GMym~JaLzq>c%eEr31UT$kuh7xYnh8bVXXBWv z3D`W|MBWU?{@3l9=EE;zun~A7BIYHX8G+Yv@;~i6}MO#gR>)@6s#Fkwm!!o|RiFkpzm$3)R;(q@jfN|x2EhbuC! z9JszQLAG)AS8x#jQMK|n$26b<`UmU-h0tTX0m5F^p1W1?r1S|ydlxkgYQ%NoZ4#lP zLg=Z!<6ifq556OIt}X+h<@btXT`D}jUi&iV^VJheCmKsNdmpd&m#+6WG4@598vwCA z@tdjFf`_^V38dp3!#AX1qdR!yZ$O&ociogH!hA;6*60wgTlwIUsu75nR6M?WW zUtsK>nzv%Ta*rT#Ql;QR%vVp1JHr^Ac=#CX&f1rocpl~)zTxhTOU>fDYL%#O z#BLIcWuek@qOUN;#3pczw8Su8Ma?LTC)J?V)r1ybFWB-Q`zjb=TtQ0QCB|;=HI!kw zspX=HN-ELBiH>OnsHqSb;TUuy5S(&h0f?B+1*D*WAx%=++|35Zz)gRb)yUr7_pS4zT$OfQhJ|BOYT2@*s`zO@@*--7~t-H-W zgu=SY7fhx;oew!}){8W=XEEX&L-t23D9lOi7boA^J!+6y#?XXW)~@vk&&zd2);zHi zpAq6BC-+R-kHGDAB|WiasO`Fmw1|-IQU*~u_AeTS=HFj<#jHw zNY6un5;^oj_?S^`&wDu>oW#1H{1e|~4p|WMSfXzB4q<5JF(0zk`BX`c)HV4T-G-q) zUH8{4S<6%hp8ToBH))j+>tSvMQy?pA~_7wjLk4OG~pP!`b^O!0JCN(uIIZLI&=C8~%1yM8&}Kxi%Bci}?0Dw-n*n0o(^NXnHW; zRP+~T6YHoSH?Nt)(%E%BIB=0mI@Y*s(g<>xvv2Z?$Vn8} z;~QVA*U92gPFRf>XcMA+3Vf|}J;u#q_gSd=pJ#pmo!D5i`k;?N%>RKRfA(j8F1A=x zdL$6$fh=z;>s5GqCyi*8In1e{KR!s-jK5uJH~#;AWf>ZSrbU7CnkwRch2A(zR0Or9 z6_@^!Nx7tGm71A_cF}&$v848n+fFwsMGS6L6NbnV?LlhU)P+UI`O>JMzY?ybQ=g=K z>7Op69)dcP#(yvSzkj>kBJeVJR~mH#ex>d2_WFFmOF--8m%zI>SsC*_4IA{Sc{ZH8 zoZK_=9pq|+)%)QYs{znUkcr>vWd9uYG;v92wjLRN z&UIPn<7uy)O=@|tC~`0ubXizlQr7r2DFJL(=vakJ@0PZi@%Z-jQ1ZE=X9t&uOnha& zWYq^0Z7H+=R9|STpbs1WtO?Mu`cE!fuApWbUeSH#){oJO(YbXzPp!vKk8bD>CF?D! z$6t<6lt44Ge~BG*Kt|}xO$QeqtTqIE6Ji?f&Bu!#oO%MCx6B8s-r6r)iWXa_7xYun zY|0iKvb!zBIC9I06|4AH{QpGfKOoM4$mP43Kmj|ILMp1OJr(Nf;1Y(fp14t@GKgW1-{mwG#==`m# zZ&~^O{dJe$^{UA$vSg{9*h?UUaN-k48p(CbeBiv@0k}I|oMHeSm>WVqSonFgPlQdLlY^f`znK zq$#GyCxagv%6ejBA3=<^%Qu?>j-;>|G>pd5grkc)3_Z76N{w9kh_t?`UKH@pDq?`& zU~8M8d`_#>ESK>w6;v+k=w6pFa&E)LL8Xy{I!I10*_9%%+e$1;RHs5!AIVRQIjI9+`gm>Yz7`|K z|4UD)?0^|D>i+6~HW+4~2_4=_%q8@xAg_y(b5`=6s?MqP0rWl%KHXA_m8eiNWa!|a z*itk26?(rkZ0vqX;uR;~?iK#W%8ol}GPvES0UUnb^f)NhoK=JFQhM&7&5DsD`B?ik&RV_Yc+ zE~eO#`qk|2KCU6=xOjEoB`|+^Nc#J56XO#3ql6&y($F%~zeF02Ytr{Pz|mpQ#udFx zdIUeQ#f1Bi^1qKLP-lGyZSW18MO>(A=dm-71cINidI&zzoO}DkPM2D@&L)Yv23nqy z>5S1PI{I&i`@dA1p7!zUH(IO$R{*ef3FZSx!I~sIb|z>(;s;o$v#4jW3wN|axO>Ro z7F3t;UQL(P^+-g-Sp>iGWiQDETt-EI1^m7c!-h(-A2F0)02WC)A$*#uI-Km((ZZ>2zjWD9s!#%wbEm{W#P@1BWQbe?Js>XT zNPQT4VS;%uH}?e(xTBN!cfyO9C1gw{28^p`b_}lzaD#qvb~#BgHON^v;_*VlZW`twG zH511t7hvhqZ5R1m2p&(azbkkb@@}6Ca9Z93AofmT>*CvA-!n0Mz)V2U%bE#b+?Kg{ z&pAhRMAyM?5+L1Gieq?0vZs3Tn6h-(Hlss~TX@NLV3>saJiqEK`Y0~I*KWyuSVnzb z3$T$*n`2ScX<3tktIwrW7p1^_19qmvF2X??sy< zmtZwCikf#bv&S*Tvzez3G~gV)TW;2ca{YaEF5q~Xm`+9>DTp*&j!C@aQum4F85U7c z*46?|c{os^Rt5cEDrTz`XR+!BgzbyhZYlp)$zgmA92PNWbLGC5WxWe4*T$+Iy z1@Z3=F^-jPPo+GqXBMo)Qbl|BZRIXz1_43)Pd2snHE%L)U{Wy}dWXa}%5rJnZk6f7 zLrn{{v3@6D^q_VG_gjGT5kPt&-Q++y#fDv0If|cT5!@fkEn6Kp{->k&KUrXz2uEe_ zV1pUbw>@45Gzq&^4^%IeMYo?pg{~(3Cplc#<((`D7#qMr6l9MM?ru!i$3ICNSWQ|- z0t+ks$NP0nVzEbQ*$ArUq@O+2M>fmD#u729me9;e&W-( zuEf!5j)kqK;}C5H@k^fYw8Y-!Aslx6A1{D3N5K;_c)Y=3P;(hRH9ad)MTr2UwZuMx zm(V{(i|ADp%(G3ND%SUjmjCIb$G_DguyiipIRrk&Zvox2ai68p#1X62%?Sw0j7#_v z2{mH3Ut&tJM4%q(d<{r1(-Af5LHIjvKjwB2GLjEZ=u$Ud7xJ~7Cd_T`C_Lb{pG6@a z4oiwK>7y>CC2!klq?qPw$Ebwk7tIJlyqhvm8CHp7auS{}_q-|PdB4?rwvNP;z0afj z?n#Q6?%P}n9Y_|>{q=10BsiO#n7lf1x0fjLhDaY+oq|YJk~6w63zEb;%kEt>+DT$| z!Zpz|QUZ&IlmYjKj}(FN8hdKZjjv1~HLeF}UCmOWr9Ru^J0LoX3MQ%$Z+Aw-)2$$l z4Y8w1DqCWlg;D8fFDkQ0nN{Fk*8GqFi06t`*eBAi$a_PLevMSW;#Yj4;q>lo$Oy)Rrgz_(qaEjk7lJv~i9@5FTJq>4U+ ziPjODnGIQgZXpeGe`HoyO*w}tRYlmXr!LR`(-HsY{E$RpCq64b-Q?T6f!VyI9mqq% zM>Y*aEO50xHTsHiyI8w+q-yL^@^}{>)N>*&npN>F zz`*89L_<=9$ zvs6Dw#Z5Xh5BxYE)|P|DT`~)`k2YBYt0rq{0kqs*v@HV(@5)kX!XVj_9y|0&BL23p zDWps{DkPgQbR|n-X&I*e*C(RHSts({msz_1sB*}VAL#ksT`q2N1rv%NwZ?bo`R!!f z9ComA^WD`li)<6`Y>X}R1rhn~5J~}#?CJqr_~VLDcU2z-x+AF{Bk*W5i_)W4x{&x^ zQIO*l|3t6N51|lfCmfblTn=){rtqX#AZieP6F zW!XX=-)f6C<^BtQ84w|Va#ssEAED!-&k{gORrl1}B7S@XjeiwREa|3Z`q_=E$Z)y8 zTC%e?*UQ)`pG{90*H`||K3uu31U25)^0Am`m|*DT!7~nGCOIvm zo=o#z)!ntX>mcnK=~kJ9;{hMUoJrW}izcJ#C^MQqF>tj2C@@5vAWiGxXI(!XY z@AU99)f^Hv&XK|=nGVzuvfpY8<6wUF8_02GB1UCN;Df(cbbGcHOQfkxJLqCY+ z6W!MBzz;F-smgthf>9$o!k|f5YXW`{ughQW6OMK`AcwlBo1ne-kToj5)gNJX;zRis zfhq}0#&MXCxIHlCf*lF!2=%w1=6O}^4GbE^IS1w>c!RG;|80EZw_*9*fKrF<%A!EE zD4hMtx6d_XAJe{mC>M8g<%IM^Em@VfMuTo{XSW{R2a}xL)@Rt5(W0A+ofD%(cNlN< zeVvDSpcPUPwGfZeUFqZ^xmk(dA7b2&K16x;jRkb}K>WDIWjE@o^_DP^t5D;M(B{}* ziJv4YsCjYNtzsiZAAH{jOw8(&GFpW7^2Z}H=>9AH4Is$YYY~UJ*{}ZxZB6z=EJQ1H z52UAjAvW5>&n0RB*2bPu-1@9ks^1%8RE>!$zdd;@*q5bW~f%@+fYX!a+8j>N24#)oGO zYL~(iU<2FZ%7&}u=HjX7$Q5r3PW8;>nML^L9jP>K_-|VO+D~Sulf8eRhkCu1zuH&g z^mxrfm|kL7js6nji_i9@K#-0{u?z>d(+f)}mpPq=#==JOWsX^8Wm~_%&e}$(ypd!^ z&}*+w>Omtji;<#I`Xxp#E#UsL5V`fcnY6!X!`x@?Sq@TtdaBs~Pr0x`QPyD6cuaN! zXRL&rmBmw_@ckFRE+0}03J8kz(Ik>)`MquBtc188V$(-!h`&63a)%&XNXQb~FG8Kp zR~rb+ie(^s-;(m^a94iZFxMbve~p}YkS!xxE&6UtA9vVUikY~e9!eZUrKD6Si;ubi9{cMs5DK@n7#`+n)tN;304d6K6t9L8~b!P ztUHYTxJukErheiZ_QAU*YL(M0XnK`$o=5QJ-XM*{p@3XOlMUaN8Z>wA$0@?zC%_qcZaXn72KQUKcJo%w_fW zXGPdDm0tkoiTgO5l`;5%e0C%qK?KP}TSMQzaF|RZ#Vc~6BpO#Dc_ZLdr>tqw33@Z# zTAHtTYYar&HQh@KCKbF`gPo}Qu1NnHSH={xocB~JEnNjMS3CCX<~wtAbiA}Zo6N336#)k@L0 zGQBs#w$op4W6`ajVKaY{SN0W;) zp~rZ>>UZGR+OaMV$J&?(g49T!G6|a(yOAk%)rbU*YQ;JqhdQ1rwV+jrTx+CAlxGFw zBvnw;_gL3NraDG)EzyN_rT%COUPmAS+ESO11L=>|?1c`u{>iAAD5Us<`i@js8QQ~t z`(=KtC-Q$4G*B$K?`;Vz{HuYx{M$06SP4(DmV=@#meA2B#=u7ZqCS^5{{{qwjoTy} zWDiRV=^aIjbnA#F+2&nZxkUU1ECW4SpZtq!J7?g zQ7k>+*q@x>e}*N`)GKpz>%auUU={6Jry@z^t~yr{3IiTQzgda4R5ZT0#;ngiC0f6* z81P-O&4mtMI;<5~SY1Z{8(=bmfbRNyLUWLG*siSOqfL1gUt4L?t3xcJ+cpiHM;}T2 z7(c5?MB-SoQ%QJYO6Nwb&%OZg>uSYHk`;kP_JY~`?h1?Te*IMdL}8mVCd#A3XZbJK z&WKpop2b5uCL zkuUtDsSZc~E5~A&ozt&V95iOg0b^bOrM^;&_pX)aW9I-9)h@?-*-kz1j9!@3H}e}B z$q*o`xqJOsQ+nIXweuboZhLnBYdp0(cyfmP>(Ymv#@u+`!v2-aWzTJl6V`=V?sg5> zu;&gkTwNSAIlJ%I!lx=eZ%xTk4}DnB``_*PUo2kDh(4aQNW(2&%s_2A{+6qivU;68 zBGruy-=TOnHdl1JC-f#-5#|k}$}=G`2BzQr4l!U@(>}BgWGAo}Q=Y z)ARhk$MOC7!ygXE9QWM!b)VOHzTdC&y!I#tU#h~-hFIxMaaSYTR)wr4L+=_aYr!pV zDR~k6BdbCSCibdNAA4z-6ML3kAR`f6tC}GH^VJAw=HEdfUM7;$@r%g##S{*~-b9fZ zd&Pvm*sH(vU@Q6j-?-8(>%VZN{@wB_2IcBG{DtN?8$`S^|}V7CzNyfrdp~A5-uL}essm6;M2zh|U6rNnRGnSAf_ew<8S6dJv)5jH;Gg1yce?T2 zCQ3BbdBgctUPu^mwP?Z0FJ~AB`7TN*1}}$+NU^2|Q`<$ysoiUmZXHKBHxu+Jrh`E3 z9_B#5d=82d?9rCiUhEojtL+br4!Ja3R{QVF*hi@bNC-xQ?Z48R4UXQ)3FhFJF?9b- zKgS{2C}~)V3bi0CT+yJd(l`WL5@qahMiz+f)omg?*O2=`Bs84;O04b(ET@HvDMGHs z#MiS6S+9dA^86yA-clI7A*VAMn~eeEqtEv;Wi2L3v-1?Z7B19V^16+`Lhct(y=f}y z_;0n~V(LZVOWz)Av_OD``XWB?{40FPsqrRf{HvFQ>_UBFYB6giOR^=m^Qs#VE1~4% z-6|-$JZQ89nG9qPrNYgTkcYZoB2^oU+kqs3mEWta!KVpCHOP&m8s+f+8G*JVi*`Qx zXBLmJ*%QE&++5ja`loxw28MLy9ijJ6)wjRzMF`Y4s?*jdZx8ps!Ifk`oOLM7Re6ji z^-<0|oK4HaQ0*#5ySLOUL>yo6n79SgKSrQ}4>$}X|z7)45ayz>p5Ao;5Dc*~I;UO=D+jF61 zRL_jkMH@lG`7^cCpt^@Zs#gBe&mh~a1S`3P-A86PZ;FXi--YHnEySkl1!8 z)&x=Ga7aiME!0o?+1dflKzPBvA4AyAfxIdkyx%z%%X!Pv7>x2-)h;Ebn~u~-RJ27x zS;SC?-`RQR)~Z?)Sn8BKm$=+SO<(zey(NF{0f0pb8+~l3ZY8W~P-i8)7zC)<#=o!_ z-RC!0z=%AOn4H*EvGdE3(Qr>-lEcu8a8wIhYW-!+6bAQ7s4oQnf@OgJy4HVi8wogq zPX_alv^LSO?>l=YY*HsMy{;A9ysKo=M|07CmaRJDhrQNkL(c^tESdcBn2~!u$uuaq z|I<-V7FC&hf|UTwnzBE+XO0h;p;_pp=xx8?v@{i8tOdgDaT<}Jho5uj=N^;ywg)B$ z8m1Q8Yzd0l9Nzc}i0z@$x@(QXA(I&MfS-IiY5MF1G(;LZ3@J~LF-alU0Efj z4~}g=c^y&2VG^(%6McDoHEU52z@qP0Oux(SSkS5p_6>t1sZuX+W_s_~Ny6MJ~^6FbOD*9mV(Ek(#2|r%|;MZD)M&xZZw%P)3iq;Rs=8^NFeQu+LB8F=-)^v2rF56-Nij5E)j7kVeGa_;+!&|eQ_ zj-H|55wueQ(|ZSYaV7*7ps!aG(}9$GvQObrd#Tk-OE9EGW?^;gIC{IqpOICHtL!Uw zHk0jKNwgZA-E$giIumVw%VU}j-u(>ZL zfNY+~=+6|k8e8QL#bUsv{k_^spt6YXCq%4Xz%aHXTVUt4ua^~vrLGg|Inre&B^vX$Zzx33n zm2~63CAoKUYa&J_&(bf=tPTV9Mvq~`+YLGOx*ER5AVlv|_wfe;TooNMeWJ84J$$Cb zSS;CZOAFE`2 zG6cjN{Y&BT^66K+)R(T>L#91lW3zSiyTWI+z>9(~3A)@Tn8W$!*sfBh6SLuSCzPi# zOTVgN?vGMd+MBEY)k#EpeCbCh_lqwVcGX>uyPoQ+dS7PEQl;j%nFQE%N7-WLc&qFfM%K&*7E;YW6oZMR(DT4$z>nydp z%Zo$6nPcEGQ6B3KCu`&K%iIGb@S3GDrFa5;-VK!tKM;23*PwXD8GYTq?nEalmcm z%mZ0+ErKDsmC;b&L~>MKMBI0i<+1(nrFHHvBj_0wdiNL`t+`HwA5WP6p+)geJXvg2LI4mJmfEa)w0lexHhSy@fh1<0>d>t1<;>}!)3go2cBIC3~!o> zj(8=A11&rAXe_IhtXt58P%=M0YF# zmAkk%xL_i*yuE{)lDux8;tEuE-|LeE&He`LIpByq;s zVB*iZqn{_oe4d?5j+{91Zxjekv4G-UD#_%k)W zqLD;Z?keHyRpJ+@1~pq_@in1Xx%H-5kV17XsrpE~~Hh35No-ePAe-YQcDJwEw>Q~RNa2%n~>Y<%T{e z!~&(CbfCIvIc=#XRa8kAA@9-PbJjXILu5lB&r>C!ij|oAjR>xEHd@;mXUbaHc02Q* z`{EeS_9m;-BMyZDlQB|nAHI6D$zHxZUmqi2zTT3fueh99MZHx{tL(OPv_AMuXlJ!Y za-^5iwM2dYcVp=(y$`8er-EeGoaa{cc*g&xjq^UNgFMf%>q2OkoXCI6KWyeV z6lbej=>SsPjKmHLJI<}Bm@Rm$o{fAzL>z_@s&V$@_t4Zos~$)2cV&%AwUMB}7dIBO z6%~$W^jsCXchf+?<0vI2cR-8_L)vMu#h}3%r;ojN+?kK^efyqMTNGrr5YYZN{ zWO-oGoSu%GW`u$x4?UrT{=|S+^A&aS@hOl&AZRz%g4edk)tPqq12ZHlUZ8SJG#XuN zYd{NFO(^Crlkucfsw(yZHa!(7&29=APU&oF945*mY^{%H@4> zEa9fpD?pt}U!?Ui-yo`5Dv9uT>4|@~?4ZPF1n(syY5O7}LS!t}JK!i0nX3;d_rq ze7WyQgHEj#T~HPm5Jey-Tkk4uv0br+c1~T^7e3B3>UW<5Ad6sM!JkdFR6b3OV8|$+ z9wEecGYH{v0y>BWODTKO(#ppk`_$g-yJ4Wc(4tBFD_}u_xCv!SNvpO05l#D^ID_`| ztpjAP^1RC3gU6yc4qlGtVgVLy^VohRWw8^JgWONmD@x)hQzEQ7+p*A*PjTvzjLtIh z#X!iT)9c84nsR~XyZ}vTb(enupw1A(`MEst6A)UTk`dOrN|a|pthDGHRoT|ttSTAg z!i!Zq!?hLcMA^+cBES;@#rx}iga0dR0N$Y|`-%vTPb04QiUFr!zHN>sIBH+^}0*?2rKNCK`1z23!=zr$(c4YYI&43m7|{dAnq#(O9DAltJ0%m>lD!BS51cqOt>(hv(ZaXbHJ zI9aFI`0w-%T6dIQ#u}Zn69}Mn=lsX8b&hVSdd8o~`SDlKf-mxn={{-Z9_09Ecu0Fq z0T=+Yf3Tz1M%$baKk3;opZfp30vX-o|6PC)IgbJCDQm6yzz+jw*_sbK#CL*^H|}{FSHl@+U|C@^)intyja|7$xe!=?%?op$VTe8r_ zi$<4g&N7wX>#a^pRs?FkDlXIa#TORuGMcgHVh)sB??+**%wKS(33&zXtWX{t-| z6;f2{WTkbpu-pQIqtu;%M(r)YwE$?;z@t?vXz79SOf)9qC%6q3N$uul z6IcW4=D5*Q3CT`!p?rA$>ZY^5@QEvh&-u9^U)WfmG0-uBdQX{e11@%1sf+kwg&OMj zEjh1jG^FsSQwnEy%KviwvK;>7GHmV;3Nv@xsZ8 z6Wv5}|%qT{ry%N*LG;TOu>&w4I^}HYO_42uBjc$0RyifhzCI1v&iHCodg!XLw zdvnF-bMyX6H{F968{ln8G!AqX_E3K9$(QubTTozuISzj}w1AWSUJypsUM%d87e5?LP#Fhx<&@i{@oj7OlGqOv)CES7WjFVeXzMQ6*!F!d-9rRcf zL49Yu4nz_crkfz1%%O?=uKQ%z18hu50#n}EAI8Q`9TEt^G2}mC$44_nVI9vNi}2t~ z?8k$x^h9}>eWT{}HKoti&C4>~9wG9S%@OLrHRLdIOTSj^=uo2meGf*gfmtP9OFJJd z?YZ;9o2+@B$0%I-=xNC9>#?8d+h0k8wiSrJF{G%ylUXQjZm78cRV6<4w^QI6wdN3Lcp<0!lv0OI^1H3ZSv~Z%8EU2@Fg#-8vcUE@F|OliDTK}GKX3Z zaG}dU62af@`Y(9if zgexHMce)F28|1bDvzdBD%U4|N#YNVT(r6YMwOTkiSkdCf%t-Yru~O_Nx z>xy@^3F(X#3Te1z^Sd>6VnbLZgAdKVCS~JI@B2gQY;{sswN(=5dSQqo^-gYGjE(P_ z5DRL5J`q3OQHmP6%pp?MYPyx5&|s$2)p?BaOTex~0_yWmM>l$kn>2QNpZopa3A3US z&+3mfv_A%ITJ}OE7k;>WWc%M)Ux|BF`~S)Z zvlNz&*QT$CQBs$}o)X1#J1VD9p((-Q7@O6L#5S-V@tq8%pa(YnS$ORo7&&G4G+pM= z>HhtPP89%JHCOT&*f~UTDijKA9=w`Adu&Azs}i^Og*s8^|5jc`tbu_5-g(0-32L*2 z4j@Zdh>?u-a(VtMLGy9Ngs|7iB~W=lS~`?L(Ne|tm8}3l)_wJTF8pUD(z>d{5CM!4 zQ{)mRsTrk`9>6E9wzTQYy;0XJSf{eOwny{OzkZ!i^vuYc;8Z4&=l<2 z-J}>ZwKwSiwBj%>D_j~1G_)E?_pVtgN-QiG@^lkUbzCK305FATBRO0KDQ2ZVygL7P zx#hSBlo*WS2F6jnkLsfts?y+$y>JVB)!SE`8V_!1+e#fr_3yPdQypKmX$wC*#RxB9 zwUpJE@4_GO>WGY2B*=p{#VA1tn4}EuY<5?ql0l0pfi!_(J_f zhw~ctu?+v=1CEl5T7zp{fZSI`r5P*D@^e8~)2!|#k*1Zq@=7mAjrx#(H(MZK7>6!xf5@`nM0PMlG zXz;f^h}gzr1u}8EKDTOe@GgF7oLnBhnhQoA=$ldXHKsVXH-H!R&P@&_=30$g$0;mV zy#vf&QatA~h2IXJ<9q&*ipy3@Htw5aCrv?;gIWY4pAi-Q^wU26Q#M&?&P<>m+fe`J zL!5R+5`4Yr#DtzbSS8J4KkO*2h+T@27+{%UGD#J`!a-_Mo(?Fclk~! zN$q05-%|>#L2uW!Tz<@+7=WsgP_fp_w3* zD(=K&cgD*NPG{4~aLma=*x0jw@;V*vv3KaEEWPrI?mLt6d?BbTw3)zquSu$xeWvv6 z8CVHV@>V*@B^Hf9Ge4)v^_|5@3QVC(LE6-NnGTGY>}u#6_O(k63{TMU4GC$~F#F|f zQiZ$+0iE%X-N(+R7MVl0zW!al)ik4g&^v5gH!X06XDMFfxWPijTMv_tMd+DgvNQBq z@7{OCuOO|xU*#DtpI+%HcEu|OtBp6!m}NVpEXL$EWAP<8^u`Ta6fsyi)Oj+~ykmDI z95mZdY~aC4Y!2i85|aKwL0zipw2Qr%&T~c4q=93su0pRv8k>shQ8vj+#G$()N9q?8 zfFR~dC5*MaJy$+=KMgZmlx{$X&9=++9l|^0Hk9qD11^*eZ0y2 zp3QmCqh3YGZCqEQ;dq<`ttJc1(td~sB1fd>Q{OgGK%#*xp3=c9>GYwVp0Jpvs{+C2 z9?Fy8FA9s*dBiVIB7>b2>bG6zar(@+aC>I@EHfB=zA@xPuby{6detH$aU?vjpqsN1 zu(BzkLknkgGLhfc=|lGzFp>USJnK_?D?Mv?Du3%Q65$*~kuj~qL_Q1DV&Wt6PJa$* zRY9dm2G`MbdgDp{N zyXF+Z_v=-s1;Sn>abcXxMFKnf= z=D`)yeQW6%yvq1&jnKU~4EZCmh55P)c^vjeh9y6Phi&3|Z%94ZNBhM0{s7l|{_2iP z-jma5E4wtmz(J3~{UM97$w9u}dsx54cLQloi$W_ObW9&4j}IO#bX_0f{ed-u(zVZKG`kI`$4WM+UH{a>$16mmew8Q*}w3b@`dZ?2-7rN1qk=!R{&l6^Y> z$A7j;efeyjS6nKuss+=(5l;}>|K1$vKTb=G zLd_-K`o&8DeU(##$5I5Z19UK}Zl6X^pB{;#%M|5;^g%YlWs^k|UQ-mJ(0}9M&(d%c z>Pv-T(IMY9vKGo2g4nODcqq6unFz}b$s-e)SKc0%);nT0J3#z zA3GAg`kRmebpETiKL|VKjwW7eMt?eh@NNF4d6o@UedOA`L#JhG$ueY{Qh*GFp(!el z=pWrHAj{NE<5phAgmsXBvQ%t4g1qNaFoy`<>6Irm9C*a!cmQQ*i)Y`1O^78{BX$8C z)y?ZrL}59j+SvtekWcf`q3iN)ae`Fylbf7p)?|UMGNlD$5rYU10zo!LDDDv5z#OLp z8t)}5s2B#-yNBuWOiXMDqR!cp@@XHrHd=y#oNuIlwRoVBZ>xMj*x&}O^f%HUDvNNuIXW80r^v!P)n@I}J#%_hAGqU>YABhcw^LZ3OS%Yn z#D}I|WWHbhP=Z^EN6bugLJI-hpF&dl4?el?~ROZ8O!lvVau+#J>2&AU{2g!~#0r(N4P(1s?0Z0pS@W_v}1yYHY?Fo*(3 zd`DNYyB$__9~Vd1Xo{7iMQt-%4d{T|E*$)fkLKrEPz8>Ryi3>tJd|p?k*G4~eZLt# z#RQ&s73VLHXMpk3NBU$;?|c=m(F)kRpIiH|w3v0JxPv6UE|6E@u~I-XsS%2pw5r#- zjp#-?etRtzfTH_b!7YtzfF+QMz3D9=F{LB0!G^^Ys}EF<_@98$LnEu^NB0t@J?aF5 ziiv?s{c+J`$5amk{P$orr;mZz{PXzer+y|7?W(+7E6Ulx4jfqq6?|gr7?)RADRQg| z%Y&~FFI?jDaE|f{LI>3L??S;4Eb-odqJJC*QNY5uOnlts;BBx5HjdFe{?_g*V3334 zIX6!H&Wk?s)YMQFtUR7<+aqlBeDl!@RYsAOUF(eEI4j?_&6OF8b9pD@@=kTr)@*Y* z6Ca-qzY{xHdeom32+0l0Lg$8U=GdVsTT)u}M5L~d62>M0FcPzr@qs0ZnC^buZhKzG zMwACP>mLcFP7NxL1GamXLoP4w8Zc@?3AJF8RHR0bnayRUk*4EuXv~b+S-Y_1%zp+h zL^t}RdCDyfFxQ*H_YXw9w!HFfbhOcH`n216@7Q^avlMWZy)a~uZi?5(ToO2sG-H-$ zH3lY;Zxeq`hpNABk+(qg@Zl(u1m8eVcuZ#AF}}zB_%V5&G4#S5?fUPO|Jk8O_`d$P z4OVb(g1&jm+`_43~aAy(9kPkqp*5Fd+>T&`%y!4N9jdgu>R} zr7hi%fFy^al;X@izonU-E^7P-yNzz#)(3jH_k}xVA<+*Hv_su-tOc8oL}1@$+-&)L zA^}oycwXmYz_!zstDiFa`@7!w_&Trv%v{DPD3`KRM6j_|wijsk$aY|KyUkJJQDl{< zaM}32d%~*-pkg_en~o;&qF@GR0+x0JLdrj-(eOPAw*n!9lby1H_2FL8Kg^Z7jOV%#f)2B#w=Sd3@)+WFkoCG%t;Wj=nFn#+O$ zxiufaEgZ9foaYxX?_A2Q)XbTD3n9;D<^i8V)a%=m^B~XmzilgD;btwMzw6|1PoMo< zn_JNYGCu(D@gVrHL=ReaY9>I^oA(5ojj8vaue(taHVkADz{`-gw7)*FOq;$Nm>QM~ zd}OUqidmuiS&2%dPyIKRTpZ4`_Rt^czcY|W)VUBY8&Hbd8KQDj_B4bx|4Zz$-o7u< zN&52r3-7xK1@9RBdKQ>&*hBMi7B)H2FF9Vm;1UT6!nMm9NmJCJ)IMe0z?(O(yk}$R z@T21|^S<88VHuUS9P&;|TTcIy7-l+sH?!TI*Bpim!sMzl>?OFI!*OFslcX5N1x7b4 z4yYmCsJiqF*k)`JS%b39p1eset={0C*?%fr$VH>hVxf`Yx-+hL48Gv5gpijk+2y za-QnJY{9APn`i7i10bWY^_x{0OTG$ryx?p`tt$C#^2gzw_3kxRU)x?2d#J~dnV|BM z;?}Dt_cg|bjV3N+pM!SXOu%x-oqW#lwRo<;vy6{(ZTNUW0V^)-c1Q6fKl6TbON>Q_p3c=OSA3{7c1R*haNy8+%X zC&QSfM`f0B1Ro|fPP$oB<^lHhK@OP{mRzYoRKRzsMzqUDU}sn=ax_O?|6%NmYlA63 zp{#5AUV^WuR->y5i9ll0%usf}129ZzzvPTmv*6l(_H3`U=CqAwtvIamPw|ZFc%9S7 zWHDT)TkgR;q>nyU&IlvYiX_pbBCQKJ6OrkuR-iKC=Tld7o(nDooUH6L^6{@B-vkAX z5eR~v)U;7(Mk>l=9}!iFe2c(YRnMy$Av{qX5Sw6MDf+w!{TQFuJDJ&caKZ5-oaOIv zfX0=ka5$UtT4wAp%|u!|meH8#1VC0s7s}P7xwam_MG=%0_Uy7H_7v`5`Gxs18iDO( zl#9tqO4M@uYdnwyeod?$lcl%qlVK@Hd;T&m7{tnl)1fsrwS0W1w&Gztl-lE6@4z6{ z?=^tUm1S(qL;rR+w;|L({1MrO-FEPE#^T9rG2OClv>|W-DWg8yl1J;F34nWVGVQ!| zBBsu+^>87`4@AtVXfXRkJ}5B%z!(}19HQK@XyDl|Mrn&%99Z+&0OU#)1_IaTxcn~V zSRPc=ONh2Ht`H%J-gpP%@}?;odGpeM0fq?m+_!X^)K`89^oE}kVA*&#D3uyk5jE&T zLBj*{P50(jFL&JZ2KDwj4i6tt8W`Vj;axM^-pHmYdlh01xesysm#td241)?%R7e{F zMZEsZcnSuPdD!sR z|70FpKGU_eHvh8s;7oQewT`nj-pt5NtkVgDmF@`#Ix@RD8-TV}D<1x6Ib3`1N0(d{ zRkBlgaPnrvxYIAG=t1=2GDd~jW7GCBwRo{R7x0UpP%s#^=;!>SXIa3#9KaI@`eyI~$dan6m+Qb(Z+y6ud>zQfBMn>p87|FYHH zgTQaSlNUiTsNR5A9ao8ZDMkEYCkE*vmEHRIWy}*kdza6Au9BUQL4reld_xkrTL4YE?ajYK5S-*99m-Ha&5n6#k^|WI69XQaI z#VWC=znb(;x2&x%F1J?tn?9~3!+g>MdQ-gPTSVtrp!9US0Pd1#ke6%k;5*kYw9LF<844>evN<(FEA-)<5wI471fC>R%0i|K{zUO&KE@xI#!!;Zgz4XzL zieEiainN>`8b67HK%-zh<(Ek^+RRfh@^h@)UvMZxlARSQTrx28rkskCOe=1%OxEOi z0Esy@FK_S>V!>QcF{=+bag>Dy1I{h*_E(73r!DPlbPZ#>%R`(tIYq^WcyrwS3=R6I zYMG7C3eXm9SnCx4VVO(pvmXh$EP{}{&>I5?=-4q;ZBk95(@Vh=xMq1>wRh#Rd|eHw zVU52UN(nj_NbAm60=PCe4A7=|Qv#xxR@Oin$EeZ^h>#4LpXi0E*vUxC0IC$R1cq;G(%~HwTL1Kul)= zHNax3(}`gqw^Pxv|L$wfdU?*PrHk`PSawLkOp$a>>Q?QGXC*Br1OiQo;x~|)Qu|sT zX>DeP#&oAs@)YU*H*qk%bx}N~1aG#iSzR(4E(>0|ifAbkS~D4jJkkLx5!<2FquJqz zeb*Y^GV;QJc5&xrd$Z7j(6E(}`Ij=T69HQB@-~xIyrzuPK-#QVDL6<_1`l(YTBK=hRUkR{ZeJY)$aiw$f3Tg6>&1e+&Nx7pm!;qGYEpsZk@gSlIy z4w(9?tQ-;@FPb%*_e(dY`b6Rey9?dzoKZEmHJS+izCy2A zEnB-dr3iS)pk0;MnwyOXo-9xt)(&CtQ-}rk9L{yLsgDE7NrrQAz{1Z#rNW^AUvR+J zpRdZb*FUNI92b+(Uxaba^&i%{b}aL{&%|0JY=cX6Q%Agd6R9nw|1HV?KE?CLXE&Ym zb}2OtL4ibf zPua%Gi|1LO)fFcbG(!ltf}8e#ytGyT+Gu7x9;~ODcIjkeyHC!|d|T(tTXPID>l-`q zoN4gY++neFjT$IS{9c;y0{vmCpq+27XnyQ2U-n(NaifYVp@?esb9U`@U{<(_@N&a9 z?5EE9369~lsJzVieaksP4#G2Q-8zpW<}bA;J+E5PNfWG^jO0wowHW>0<(oXH{OumGJu{%G)F1Uk^P=-uUKR zcaz>$pFH%V9hJ1(v*SK#<@>b{bKjtdqNzeV1t#IBBK_9qNNzO?rlYvKY4wxKJUN>Q zB3o(8`GG53jVeDrRs`$gnlddESL{GZjjwFXbBkwBbevR#@4+K@3!|)<-_*q7?-q+bqJ+G zf-~YDD~5bfj$T&zOr6Gr5#2)TqZx@qgdafi=mDHJgi{sn6tVU^ypJhh+{)x%-KxVm zAB;OI;Xx56RpEnQ_r|Uns7D`FUEUzQ{t?-Dw2;zlrf1MA*FaF6>mH=)Y}tJf>3J27 z%Q`i8W(#(XSs1R5uFuutg%z6%%vX)M*jEI0UcysZ39W)ewSyNlP|>};>j5Gjo*Z}Y WaQG9A&gX3desr}At`IL<`u!ioU(^-= literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_13.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_13.png new file mode 100644 index 0000000000000000000000000000000000000000..300af3c7300353b7710c0483f059acacad529871 GIT binary patch literal 98439 zcmZU*1yodR+x`sDLO<(zLIg>@96bx1Fkirv|dGS#Q$Ny8J zWO%}7azZCoO=+ieSKN0rYcFL&hh`uB86kgBwM?R>^x@$f{=e}b=G705#fwEaJonCS z&JJfUXHWegksaFIIGOsTuU-hO8R;7C*HeC7Xy?u?+<2SE5QF!;wc1tg5E4C9kH^F2 zrJEW)%kLWL>iwhg!^XC~kJi!3gSTl%fxLx$!&o`Aj!wO%0q@%anK)iHaE+Io%Qy_I zhPo;%hCiGm@jCt%&x~LqTo&=p4(!cUzIqQXZOnhIz!e?wI)cuh94#M{mnkKQ>nOu> zqpDgnL)wZ7`7QX^f(R@S#p6!%Z>zUbdK8sVWRf!KeP66ISuY z!vWEAPoiRpLLn8;VHLQOUA0Jac7KggT|FXRyjB%=*z#!1Y;TF?=B(xe3^|9mL}8<1 zGSJKJM=DbkE+-v%j_>Lab%PUQs?dFz6Bl6Zo(|5XkLS!~xG#p3>mOCeDZ2-QI%&&# z)`_N8DitA07nL^oX;pLZDv=%G0o{os@@*T$o2n1owoe6nb|PAvuaAp`i(-@oCfxt` z>s%exbTCJh9|&zKh~Au3H?L@q4>V1Viw$E_PCcB} zSA)Vu;19*K?m%(cgO|D?Tplx;9qJ7hHsvFfD_=2_-8?& z`}6Div61}8ZB;ULycQHIiYY&>)q+miD$5JRmCQvdL)dwOVN0fX;S@9;?Rv5keIv(X zGNe`%D{VTirWzDV7BhBhr2OZxAUT5uMj_ut_BakO z;<{h*AL;&S)2oU>h4#-v>U~Nnrc5u{_5SRUKFw{gpz!&|DYluzIm?S|S95*G*CU%Y zk_|g3fFawDE7CU0!d$&f`3_!drHl9(dM8^3^zw~7@Eb=h<*bpK2(Sx5^ zYZ|_j3@77G6DjZ=<&8IRdvh-~4fu(G(@bx8{5T0-UbhaCFOgfjW)pBa)!;kp3~v$N4~-v#R=SlSzG6 zbq`3ddofa<S99Q=mGUQa)up4|PT|RouUxA=ImRy_ytO=J@=VJU>%9$ydn!e2!dW zl$?$SN9E(=o*x798UrM=*CoI+WBY@Q(vN>@#q~%l*+N4!SI70825kJ#`jF8k#zmWS z3}gj&IL}oq>A3Sm0~xO=9*25kRPWlOF)2d^3iH}+6f|)D^YWY_kj#Mbdvn_o7a8BE z8e)+&beanK1rFL_MbH;ZTx9o+0F9`wqIH7!_m*vNmXkv z%O%mRN*%j*xTKvQ#_zF`{U|5SlYVB)ZsdUhASJAr3A23{NitL< z(9#VG7uty7-|Ok7R3c@ZqOe(iU6n6T$>tjd;@=y0@YU#fkPZrE&=EGrxO(@_Dn~=> z2}Nd_^7lZ$LIMG{wuMvv?*)%0kl^s<7qJ?ZS9f0$68Z_>SX6ja5}%_kjx6Yvq$ zs7TU?#BDwBt(-_$uS}rkC}q5#o%^#g4HfC}{lw~oct$B~h5XC7sRzq;#FG?y)+VZA z^m_du^suj@i!L;$b}e0$CUuAF0B@_5fthwtGJXk$d_82K^5obu(7<|B9c z_^N7{5-wELN~l_Y&W?LUYS1`Y&hAq2JAuEN6$kab`f*dvZ!l+vQfe-5Eb(+a=TGso znHN~}wdU9zM8Uc4j<^FTCeJn|ny=Kwim zyYyShs{TTo0ra-RtdvQq7Y*&7vg_~24?Dco8I*S|~tx@%qo;LT8@;9;uJ#`!g z-Ugh1|8@_l4cUx4wy8G=f!16}#TE$UIlKXZcF4odqy`UPONJ<@5_$J5Tk%H73R2hY2?ycZtm;QI7A zu>g0mG=zw&8SUh`k(+>Y>g7oAfG1p+62LGBsSnJ}uYLf}F8SAc?&*#LNe^D12B~ev zFVYu1=6K+8_0n)9w=()|-s47U!(})D$V6Uz}wjy{ZVwcfzQDA>Ud*F zny8-zOk^t~Zl3k>0uxO2byV9d9KD*v<$K;6LNoWteY>=i1e69eoLpNN%noADkD*sb z!yBV|_WgljLiFF7yJYKa$KKj|e2$@D-4kWm>ovTFI~7Yme{h>vNcbj6+%mtA~OwG9oVF|P0HZoxFmOLgjr zFRqp1af>z|eaNU#Mza;-?(WgagVC6?z~DUVi*K(GNwXA)+a-A0xGS{m*LoY3LvvOC zZ^!;Bt=h-16|eXF(d(f6R6S#?bRgL2*3V(1~|oa?wr(9!gV!QAW! zk>{sk`0Shwd%uo)jjnlr)~+Ud2rNX&LAtxID~454B1Cs;_2qIe=H$_`+7y74CC&JT^?C#2#fYG;~hmKJS07 zM+}sHw^6qVi-g?w#f<_SHS+=I$B7Mr89EP(^zzeObsM5)zeN&o|^^v zBnZd;s~xxI%d95Xc{=?UWB0;#?m2`ntUkLE8GBOn0_CJ?#dj_c=JC-&^`Iqj_^C(H zgXQ3HSAi!~nRc5R$}iroVuD+SwNloDg@rup%kfl`XtLyly-iDGy^bDJxLaGr^hX9z zslZZcY<*=zm6s+*W|>maS87$ma}lFtQ>Y_>KP~N~K{5GK;S%8zH?l1s!X$CmPJx1! z;yUMZTJcv?Z$%PkR33My>2jxHIa(=?K0=7jz@Zgq4JJC&nao)bX7F!YOT6`nJZ z2YvyTjBy5#HRtqRm#AtZU6;RahaYe?d<7G;+bm}5v)j}i^spTo8yD5CDh{i{d{iW6 z-Iv0l?!p^6G02lC`waL&sko)ue2RIr0#Zd36}wZnRrJx$AwShgQRMn;X*fGlJQ;C) zn6Vb=4)Hnqv!P^kp`MO&zM<-y++6=*ND`cTzLo(Srg(%oyWdNBTr8U@Yb-%$Y})rC zub_EX$&M$h_r$7UGjPrzNHC2WoCAw+7nO31-?x28rnJEr0dH}lwUxb^9Fz?-*qfCc zA#+bghm>uW$I2F@M?cwD7Kr1pQBz4r`Yt6Fy@VoBfZB)ZUocV>TzlBO-$BV7&%onn+|qi$lAbhf8p?4H#J2`?-Tar^|T}Y6ucU`Zrt)Ktv(3i$uei-y%@rb>|rZT zwW4(&4G?wPZN5Qx0&hne80<1;AjtEBXU2Kh!e}F`_OKhjGhWXwDZ$;}%nLY8RVd*V zYU#=*PjF{2j_$v}dVvL^#4O?~wO?!aK-oLetL~|4t#g*)85??$yuKOk-J|QuoNv1v z>6xo1o9tr2QCaY+FS1SNc{ZoTOgv{()2~taqcW;5InEX1+bf(KapSMsDzqXrv!41B zLoh|2k@&H8f2-2ggZ8H4~9RXL8}th4Ei6>SaA1T5=3Vx2{itf6G!- zW|in%48@>Kj$K))EGumGk`*0(?sZjvW7R^Y*vh7>6CxxaBCkAdMQ59iEXAgU=wIgM zT#;<=808JMNFkT+e3S8D{oyq4&)Hm2gcB8f~*Hl(SFqzh7IFwQO>P#)`_5n1e zZ>hx2G@lsGOR-BnnQ@y7TS^KhRS|jy`+W-ogc>h4e{29dOesIz^WCC}LwZIca6B9a zRav5RIopi`Y08Ni4pMv3iS6^P4n4cxYod4tQ0CUdN0+I-4o-ducK$$4;F+2dImG7&;2o+q&(79)Cqh-ahN`)MnsA;(GsU$7!=YZQ9vH8S+MjK&cld2__!r$K;U$GVW27<@;zfOSkxuj>$?ZX zWj~btwzXyiCCS8*EFLnkTMB|nDr@E;*^EP3}1*$&Z(|jr?DvWGF??X!!3R@}WvPu*R z>!$KAsFx;HRIW8qw$UquEDFB?As4f24RPvvc3$+|4WcyVQh)oXI?xYo>TVt@*)P6O z*4Gi(cHj)+g2~*!@rb%sJuD`qZ!_jTqlz8)=i6lZpefr{WtlQ^dO6 z;aMgO8fxPr{#MEbitQTFwB^I5WzW-c2!cIxOAo(GOBcHKIWFBYg&3LE-lXm-E9PC#rS)-0SCtiX(VQza-6XF*+FIwaTY zDjp~BH{Vu6bj79&j{x?31keQdw%9^)m_-`;%o~B4?$P#i{QMdE@ErHIWknE? ztJE2bO!|7vwYn>E;CnI9qQ38A31{WsnomLFfFqr{F#Z&)t9aVmJ`cVAE;yMzP~4r5 z%)7&&$h^9}Y>lCGJlXt%INvi_zolP?SX{z(&yU2gcp_GW*J=qP$LKkvG)9pOQo`;9 zy-Ij3T=umFJ7(u2&tTUjZ{7tG2w=P@GU}o<@SSv_Ot~na8EWg+x)TNdA%1JrYy4{l z^{KB6&%SF+HkZ#BK*%j@2}DM~20lYS1J|B1P$%>Hlok-8j@t67vSI6L;A8nf8jRgR z*AfE!Em)=E6< zACbM2m0ZC|flGK#N4ReRs504HDJ6=z=bM)GVn_uGna6@jeKY)@>Tb=AgerPSYFfU# zi|%Az^eknHn$;2WCNf`>y@lltUUfwS42yawhSN`^^kO8NJSr8B0%e^ zY(zSfF3H;;fDyD+?H6}aH?|_!zN}y;`pL=&t4h6{lKrm)PkPmyf(O8wA8q-AA&KHw z8>)ih{G$Z}JuqkDd2Qo&SE%Z_fjOxfpcUv)HuL&JC9?a^%B{4#z0zfAPf!d4JVsxi zC~Xi2{R~dXBZ^`w=C4+Q?8pCHS3XnkfGc7OOPp(-MHl_2Hwo9QYadGrbLgC6po<^5 zqTuggf@~agmc=^ekRNwx^t^0jjI0Q*hA3^Vj|vD$09yAtEVsfwzpN+LA}8SvPyDag zeMfq&B1thj-Y#KQRm4SjyVhri88@k}vmR^Se3`Sgkn_v(s?6Kt^XO$uYKwl0x0E-# zXck2`lZQ>Ln$>b#e#opMXHP5?spy?A5GQD8rzh*%ibrfTN%cETTaJljB5xpOS)8Jn zSBXSD9TB98P;$oV`VyL3w)yR&%I~3#;#Zz=l=vwZp~{;xl`k`_LtTKoQ*jGf0Zx`8 zag+c@mDy)T3HgI!%|v@Ph;rc}RXJ|&%12tSPlUqwkE#F;4L41x=67gtyMHFh9>vIG z5kheB&UDg-Aqd=j88PPeLElMvk-YG~@P)gK&NxHG@ONeEb@G-ccG2$8{K^FpbyNB( zUDxl(a28NH+03IjotH)eL6LKlN8z!ti7Ji$dA|yEq?@j z&r1Xs|LMawNK)2r764jmFumT&u15RjSn!T9JlWJ;>OKi(p&G?I>rxS|Cjd> zO2vlrJI3qmW%w?d>LCB(L>i}&Osz;FmsMlKPevYZjI7xF5}bIGms4Qm6TkU(Kpj!d zq0~5!c}B17;czFQZ|l@S9?gxJeP0*97iY8Ab*7h^vv znzLzyVtr&ypD5}{fP_+rbrgEQ- zxhj68>i#DLVNpTs7=>%_a85(C=a$3$=Q5TGZh3v=D{WFPc=-<@uP13Gtr;EpYmgx| zhX5G*mA=JK0b1n5jO%ct^xmb^=F_0!7nldfv(CEh=>_9DekoF;oH5VauH)`%`3S7| za;xne@Bk@ab~OGPb=hEP^&kN|pkL9oX#_BDINrYx*(lrv&l6iIL}R7PF!sowPGDI4 zp`SCJ7=bWeuk(rDKZq6IDCQM|=U;ff*Qi%64f?!WC@cfJ3{!s?G)PP|16TCzQHX>t zF018Lu+kjKht|bD*4lgnoWpF#vgy%z2sF8hpHVv5*)afg@*;iFOE$o{&~H~(yiWl+ zO&d&Uc{rxhvGk}BGFGJX65VbLMnUmH;w-LrN2k``CwgvkI(42k@Vt=%OOYPLtSjD3S=}M zDcHE5HI4aiF|6;kSN{s177?RWgTHn2+b9C9;EXULxvq;nWUQl`=Z^cwQjTxVVuX^zCLF-Hjin0eO4EC0LjH~`e`^5a7ZE|+ePwGfw#OG9rr;JX1P+1MgMf!I^WYjU)CCk>hbUcEPYKTVQPj>S< zsIDB3sA*B-M=E;MDuYL1lE{Y=%JPSPmI*+$d&D{)x>>+xB*iN5zib`PCqi(prFm zh+pnBg*a6(F6R1oYFS_45d=^51c1ISE#vGGrDD)rtIO)R=cpwBrIggB%Wroi zukI_2ibS8nJO%&5e1KfhSKGJZki8;;?n%rhq-J>AptFJ?iW;wYUN&)~H?^7-eE|wP z?78(b;TGE+jWu7tFmUQ-wE0Y7DgUFm$uokZ4&t?43S}{IUA|RSbLYmcro9~1uZo*4 z-dzB|EEC(=X`zZ*3!L@T1^LW+hDjEta0xbKzNOA$=cC?=&pGR{y)}JN?eVx z)`Q-M{Rsf3sNEi`-)%|(i&ntn-fDL9m^2eqUkRp0I$C#tKGl zk%;){REcCpAs2Ev8`ME|9u@f9wf2iAc!~+};`}`{XM+ZfF@+2x7NRBMl{Z%rTKI^5 z#$(BKj~pMpegr0U-VP)ZC+9CGEmJ;W?4JBtJ(~=eL$%J!QF5y?u|5O-pdrK=d97>% z|Cx}V_OC_ESezTy9}tW1d@vI;2fykpOQ^w-qO_*)uGDuyz}U!oIiUili=_FD7yN>I zDU2@gF zwY*TSCLy2WA4baMgW_Jd;Om3bVCJIe+__3en(L&^h8*ld4T8_!bhc;{!WiEL3i|X! zm&La=RT2*rO>?~$be;zigroz|(wmB-qynYNgHUikxGb`q2 zls@gMQxoQVuo!<6l27(NTYBda&!iqR72b1TPBbSr_Ps2Rqv~5^p_gemjvjDT&>&*G z=mlvc{r{WLX$P#uPpT=lWeQE&sfddWVOA@F&E)q*-OK_YY0mq5>Q)ln>iI99=rC|p z0fwJLH-=v;2=r;9E?yS}EYQZ(+_kI90M{)~u5P~KP!`$9@eqjlIG6O$Tsz*SkEe6l zzvY%&f_WIO?mz^_X9`kFNs3or&x-3d|2em8$05`IYS*yevHf5yel5dS997y0@pw8q zA7a$Ec7af{rGv&O1v8fELCUA4EZ(ycu#Ia zdOriGDPsi_ClD0l63m`tx2f7Rri(W{|{@ zz}Wcy*Ks}xi}OcNDmf_mKp2QG1EaUFMz1Z?3_$e*(6dPKc%#-qi2zKZs#V}FvUe8V z4`kYYCO0EV1R^s}>^JU?CrTa^UQ5++jSyb#8sk8RbV@RfHvd;!s8M5H;(WlYx*{gu zSY)ryrWgTQ_RZHo%g&6SgmbluV`eZk!K?no4E5XHS&hhrdl#Y4ocsqfehBoN3}k^pjQ<=0~DXT^&V z4@KfyQfu$HbUh2o;X>?(o(n%fgWU~3n#GtM^$X7R1(HnumI$^m>2u!G2M_u{fg=Ym z1)3Nk79Yto^^2$9*t3Sit(TzC_ZArcdu8)uEpbQ*EEQa=#7`R7C+;^nBtTE(6rJ6*RiOn&o6kM&7k2A>5nqo)@qt zM*c7}GN?@ufZfnNp?N>h!6C`Oo3GpUfRxlBD}M~-`}GQUP>nn0SM)1Oi&T^Boh_$7 z-LyE9Pu~|tO@$Z*QM+m*MnM_CAzaBn{ECk;-lHEVX(6tqgJFM-A#8s&|J!!$g|#?r z_6Avf#O?gip#LSU)t-TQboQ8$wHAHQ`vEC=u!syL5EwU}e($CrZ_UL=h9wlA{ncUr z3B*!5IH(*0QbPYi_6&uH$VD+Dl-<{f?G1!C;)igyVs~g9qw;gQr73)SUIX`u?q9Q> za{lx1wM)i=V7E86H&cie#T)GAtY;Ad`}&~fdr*05|DZz5^y>M}%m6{b{cRjSF+;81 zgWP-gv}Z5pCm#4oepM=Ruc8kS!~$&6@kh7*f&TH^y2a+x#@&@_J>ef9$lT{)#PHbv z^i255*xRrTnx1P&WJ>~&gqrzN<}j9l&TV3$gTg`SVQfRyvJyePR`{R z-}745L9ecYr@=Bhmv~x?s++o!z`UT-=X55)5O6UYaspngPgAcD@MATw|8c;JayKmR zp3&Qh?btPHfkm51U+!h(GP5UT<@j{qKLtOAfsH$`pL*lhW!6$}aH?uNfwa^(f*YXL zS-(3S$-qvV`L=5sHn+WIw{eUWrFJ}jYD*QluhryUf;MS}(Cg1TRh z%8SKlAQqUKU{dN^vXqG-j#`(Fu{%3f72ozB=5(5rX0DpYDUmjSah%J7J4+DfqYq87 zGt2LsFyN-QtVdw@VAg`KL>PU-t`xV+x2<%(DWx&~USy8~t;e+BZL%KAda}N!$+hz? zFS2u2to=^fujK!_)54vZ}6yUO1h-q~apk;~0 zUTYjK_`c7hoUr)M@^8~f=z+W;;jc7(2m>Re?{i7&N6;5W$U1`nt{@wgiSOB9N>B}D ze;Ea2CmhV*4FM~(4a*QQsX!x$Nn}=X%0ELK<`-LWGI*9!aJRG7<4IFKzDYh|QD#W- zeh%MMvbROrc^hZ)Bo?a-NWMQLGc!3L5N$%LoZvwH8jx*j9XW=G-hLa(<~nf|O3A zX=`{Js$G=pO53NV-T#nTY-_Mjts+ch=5-8jpM8BwJVc}9^I?jl!KF8P4ArGK8iI4sh828? z@9kEn_eXD`JnA#-oI~#%#~>Z|#pAdC38`@3rAmxTlfkX}9XC3sA+9371UnWJ%>4s?KTd>u+_wvF>X#T?jVNc~~?!RKm z#ls6f?p_O|6i0udb`P5mFyJ_dx}r%$vU^r+QjlNq7NrLGzPs#wI8zhT;} z;QSdI?nzJN9hFB_(;-G8fTsr7Knh(5S>cz|ota8rrH@ zk*Gp&+s0lKedP}LcL-9_aWDex+tyuiyoEAbebM{6egK%W5>Bt4J9-!__s$ciRTED+ou z@zEoKe`r2rqFJ&?$s*=Mh`&)a@Y^ZPF1(mu@{@^Jv0CBDvGa~)8X%ztKm%zqZ`x#= zc<(ZltYSDOy2^_uGom>5?6t}A%a*GRo;3l+79QEOr6eD!!B0_RlbeqJipGzmm_?oX z8Qu0ZpFi*=p#yVwmp=}59rrzE@AmPIW4{uv_2Vx-m%*EG?@s{hkGz-3F1*4K!l2kZ z3${mQy{rWQqJa~ScPrQ@jL{oISY!EFmF<+9I=IeM^rZ2Kg~MNB_zW=aVQGL2KG(s0 z9M|w0o{~Aw1`cxpy-e|V%UWF9xc)7)Y^wrR6)s|2a@cVXolTwUa4WN~#Xko=pL_{r z5ym?g?@~n^#Grd#?*)K~6&`m6OhxU?gX(xb+fhSdi-w!4e%QMRbr@qO&GqHeD%U%; zX-fiX`S8r%`7B6GJTa&^HRntnSPj3W7_+D?P_a0Bq|!S2%z>6G%DB06tE20^<8jR$ zjWhlg?01ecW|7`(aolq}W(3>8E%+H$i^Xd@Y!6y_VPt4sI=fL-7VGEQozW$h13>(y z9GB{!wyL)Ab>9k`)RXyCc-kLrtif*1Ypx@yh$&3DU?QDfOdu)j>I~oO^NMMw-rU-)Ddg5UQX z*!Kd056%{;7HSs5oq<$^YWmgDz#zA+tV_YaUwGuDA0kmO*a^-QB51_@aFyHwud%I` z+9o$W-S$TiiS-I8zelfU{?z71NWZ#U?2BqFxs0yf=#z0XA{i%Nc|0Q6<*O;dSw0Fn zVvK2~sKp{kvHMT%GWr$-yZ>@xSn^dImFwvHMJ!a| z&q2QV^TNMuiJR|}YGMlsOm~j5CY2@wdj3!@mZ|W`*yw+S+sM6&2#i`1!G*rG`3e;q zYP35a1+6ZolGa1>H(k#1w}LFjA+Pp($}1hD)K=`ZX)itJK@WY}pPxsr{2sC8Z$9%b zE|b~g0jWs9BuXV6|HAw>U*!4CUbw_?)8kH$--uJ_)25%_({DqO|6WHNSSKz=K8`v< zKBj1sPjKUgq_8lXeX_?a(*Z8yS!Q(W#;hXUnIQt*%>8Y|y2GAg=d$l8PFdk4jqz2X zr~~Gov7u$7tYSj#taCCkLA=+o8}=!azdKc0KeU!*(+Us|sHWIHj|0}4+`JqU19uw5>-E8tM{{ss2Qx$`B4%I`Te zok`QB>!l~ejxJExGtraKs<6y|`dIdt{gtg(AzN?zD-32;odhd}_=@(E&E_fObcC~` z1zw~RlX*|&x0lpCUoO$Wk6^z2fdh!qy!XB{+ zgCyxBBQ`0Do-;+nLh7IC7tZt4pg1+@8PhBNGSXi3`J1L@5~qx7u0y1I zJRMq@sh{q%Sxl_%JQ7*1W@ti>Q2IQFcU%n>C_9(J_Ot(=k80f9{Q+^C{O|-)Xq^V9 zS^H(!O(rkXw!AfrdaDa-R_%e$vUH(a)ZZtuT|I|5&wX&jc=X#F3h!BHGc7#( zcx*o<&yWCYYjpj|~;!}$wa9^xj zIY@O!O5XDk2i-@kGxB`fY554Z*y}`m_Pl%3efs%xNxU}rmYGm1hKf}53LY^&hW+aU zdM_)NMki-L6UqarRgFm?$A`QHf<_*$J*@f5^<3UpDdP^zs||+(O1?k_?eJ6ZgJq@^ z+h{B4RUpab41lw6uby=B2=f+Sksx=fhoV)rXHsmDq#G4_I!mbeP$8*oc9##1Yl~dU zKAS^!tZP*Qr8E|zl_5~XoBy)1bpz}MyF*Ov3x)ZkVO3Q?w;2FnqxRrcTe#qCBJ80T z;1{eCMNtD8$MebF_dJ7Qr_-ruJ`7_4Z0e)du5=(cbWV^UimK2^2B2Q~&Miqi?T-{Y z(Z0ACYfir`;PTl}@@W5cn{XW1++52CKs-Ni1h0)%z|NOrCPrJqG^sUzY{Dw zC20#*ddK?IOZq#{YR2_}wkD*WGl%7MQY)(bP8igd_oQ(k0rECd<%lX#rck>}fjD7%MhCVYs)3)xlbWJ!DH+{r z&ehfiUXaCCj_R^;eOfHSb9=2_f8Nf8;OPh0$2no&EomL|J#b6__Bd?Lz~tf-pz5kh zoVDgE>2eqBxR+^(_#=^L708*0QN=vwGgnWgrzo)2?Vi26+=%Dv@LwYbk)7&m6!$Jv zaLL#|OW?RgCDz?K51iX(zX&?V<2~d8kYii@0P8#t z!0vo8rXRzUZ2irVjBcl?Bt5p^O`dPmLT%Ws&0$nR9_ujnNy1a@X*bXIMpuQPcUyj( zzNju+Qu_SV28YeI;Qtz&7f&#Y1~C|~n8q~i`epzaL802Ln4#!<0UUM9YqTw;ihdUT zeSX~cYJl|g2c!Cpyp#y96H_2btPJdUSHsml<7L2e>g8p(zqmea&aedf*b2a~Sxbsc zFhU65<+B0rX5d7-eGgDRB0Sufc6A$hPz=+()&q95H{cSL!G=&)-?eU^)iW(@s9+tN z*VTQ!M=|;u5wXkoFoJs^Fz54WH4_77`|s`m?JZW@txsz%A@U4Z~%>wLNxzM_Cy2uEc8ODKMw; z8#Plbf2F}Vv)A(+J?ME$&n-eXU%PPrr$U*bBT*um{g@xB#>2cLk8!n3w@CdM-E_X1 za)=T5;E4hb(LX%vLOV4}2NGBi!+K`!nVyOwGP-^GpZt(pRKeqQ8U;Q=^ zxzuKcE3$8zvXu3zqLp5$dxkwT_GJb&F8{HbRE%LyDK-3?9^lxRW(36@fegfeb+SG9 z!EGYzchy&_eYd67fU<8=)v`rDC_4?*I1ju><>b)R{_?M#5HH*GaM}qYAP!xQdBK27 zRorl(h}ffKC@0bXfugCA?3&74A0`7ryhlKN$h$QaP6tzq8xlWRi5Jx@esSzA(ynVS zPb`!&uEK`ZobX{_X+Q5L;cVIRn*DwUT9@cr&R-gK*4`V3IdLZS`qP}Ig(bF0t`olQ z!jmt%0?Y`3W|1P%qy-s4;!0}$oZGZC71e0KZH+t+SOqQWj|UUOHVa>@_*X@F6MsWd z(L#?-&XywAf>s*b%Zsfkq0?fE-+GT+lY+e@T6^0eU%@(y?fzYna6|{bc-3EGi(E-9 zd(CUrtMzb!xKGU*DeLy{Qcfo0#_22zF%iHa>16eV=rNEd(3WbI&fQNXKmYj;2QZs) zcF#RL-@F9F!NA<>)}flx7xl%a&>LA|I33(TAJJGkOE5N95uQq=CJZ;h_R4n_UiL_|y#-iDQrtoPIyJ59sEb%#6zH zXPW7Y8UHRl^Ip{#R|F@5u>%oifr!2>w8S`S*Y%Ch3#{K_SgES%8fTlr28*33 z-RtmdAsdB%WKDx?zcvWHXs1Ok;ylNFT&wA=ED}Un^y&+>!%*J%s&;ihFx`aw!#q<} z-p_?xjcK8-OY%-;=rj(9nKS-PG(dx9jBKFKYCCmL9TKm?*}a%F+^E2A&StVv)M2(I@R zv$)JOVrAztXji0JDY#1y{nau`Gg5w#J~sao=xEMgxVHr^Ptrc6aquoZN;He;F7{)r zVVZX=d3Gbu9)d$7cb6)N*#p{7=$&M$Y;On&)xyDA(Zc~Utv&yM2pq6VrAGZfqvN;9 zOxLG;zNHo`3^VZ`u}+03ZPqXpO_e=%-ku3UzV^Kw*dl;CQN^9Qf@ANyXTc@x} zjk=v3LSNyF)5#Rd+X3TR$7q8PAeO+mO4x~MKpEz3_bV`z|EOfQg-CAC{BP&@j?l1K zgKlLj`46!8=4PZ5BjH4P*GuZVfa(84@JwWBSLv2jS6l}wnbMq*9rom}5}%zr{CJfo z9}CQjboAY|AA^>Cz}VPEW*}h?&C1f_T)CX!s-X9Bc>t zN{}TU?&41~)U0I(o7;Y`fXt11HQ%+o+1#+~2~YWp-vqDrY=$*d#o~Y+`w_zAvz%~7F1&F&|r`rjY+gUQXX|16pf5{or;0KI>1wbX|DKC-v zB69^)&_Tz*9m)Jxu~rl$cu=13{m((<0SQl3rv)2hML zxyQ54{pZtMMx9>Vc;~4-ibbeaXNrR`tve|_9H zMurSyDNEL@E%qtvFeF=+k_uU~3_^Ax`&7ujB(fIKB72r-48}f|5JGm@Gq$pO&UD|u z`@Wy&danNN;`04|&pDs>@;b!TXHfZrR*Cwos9=%o0FR?KLF{vI{1@Z~QJol^%|(lI z;-Fo2sa-H>wxQWTll!^8{9Ll`3r*8uMosQ>f&51y0Vz^X7oZX!wb63kc2E9o)L5m0 z^@?TaldlV>@UPpld*>;C%?I6-wI1kwuxI7LVxw|ot9)vIKA_VupQ*nw$ZMdI^OVF% zL3>njBXm635?UBD-G1m(5v>Ua4sFHD#S(>~f;Ea(S~rj&Kg2i5mdG}pAg zv`s3i?1D*hRv}n$cZ^oV=RmH6BJs*hTctP5g`k?*3t`Z;5w92W-<7|S(>?jr?tay{ zzY~w`WYl`|i<>Ess}=9+7ZO>QNnvcVgPnuNN^?#hUp+dh4256VSyHgHg2l>BM^pg) zV*We*poVlOAyGz|zttVQe1Cp^ud+)q5e{6BveXsH^0iJu&&I1Uw~Amja2rR%HgR(e zG#-Qv@XmGlrVzxwZ1pXQpm()nTgii4#nIlfeb(O=CU_YfL6P;^=!069-%~(n=;;x0 z=UM>=thBWO!|QtF`uOkmG?s;x`XP%)Vx#xoO_BRcMM>E@Qj4N<>4A4&^9KIaYrYuS~q#8cF#1vAFl?W`I+Sw$li<}*AuJlTTx`m?1oRogBW z*Cnp~=5@nE9D!h>Hv!s({p6Myxpc&|pKv8fRy^fzQ4%O6o0*=>hSs0Jg#5>02pLZ* z@`|JSe9t7pHpZSA$@^N%F>Dec);Rnw9F)|sJ-;QH`CjiBsw$kI$7O_dlR!E`f@S2qwp2#{A)r=Sc_TQkn}m#XJ7_@$h>6 z!JGt=o3kK0$UjT!UT*`ls_k$%sMG86ywwF(zMoYo=*!K2XQCb_imvOjRDyGL$ED^| zhI!z5@^_K~SY+6fDOQjv+-1&Ds&F*51X)Djxhm}j9s-!wQgGWJUG3zlFM>*n`5uVD zy#!KKaa}N3y5Net(~4L574@Fnb^-rxQ?+qc8?<+wx`P%4wM=;<(o^G$HP3jnnD{1M zt=VNKJ|fCa1o)i@mDLU}7=kXJt_eenniP1Yd(=q;rQ6!a8vNF@(!E1&>ebmkgD#PS zS^9li{C@kV+E49}1%#ekZnA0i@~I7c2Gjb>WOZd%t2*L*nE(^Y*a52jM;5mplaZVS zoCA3#p-`bxyIuNH&z7?t1|qaVi;qQkYcCO1<>mh@P!R$umQ-#(W?$~F zx&}XrP>WztdR4!|AgjbRdO)=-6a?p%SFvA>eAVl4!LhdeB<&fM z5@Tr3bv~wakB;-6i2|{SpOz>dS_$OXFX z)$tj+3`QFsyWH5D5i2}})K^1vqEoePI5;|^Q{87CptaGR>0qMI@>+Nm{_~J^)Mb=RK&rER zC1vAI)|4y{myG;`A9K+CmMwiu?H0=@;0)nnX$ko*Q2AX`)W#+s8&2>OyVSE*LX7f+ zS>-ex-HSXWao3914J9?|0s5OiIbK2$D9QeStnfEh6suKY|5lprQ$2O0ss_V_;Lx=H zU9tsq^i=uTQ56z-WW`@;P%FT!3>O*LBVl%1X`fq9!kB1<&?91QRq{M$gEgCQdcT=yxPtt>1|4Vt z7aL|PN#&xZHszOb|7FL@(yOesnG}dUBny7?WF}>2Ohl(jh>M7f=y~h3Gu&+2`{cBs z@}HWXIrjDYb}6JUU=gpeZ66?i7%Y#biTDsUX2}nss7^}ljYt+Kts5$wkaGgnV{8>C zc>MlDQAdv_BQN#T#KBKl7Gw~rxS{MK_37c;Y}^+RFOhCm!dM8?5go}o`>$ z>YRo z$^8r@$7Dz>?uI7s@}aiY zx{<;`?=+tJ3{?Afd1U3onFepzh`>9IKPn;qes*I0zXWIM6k_dkPy$pW)=SJm`u-`~ zr4^C(7kr~mA>7Iox#eDU{8+h?VUMyV-W6;{8iB6pyU(R$ScSX4UC0|ocD0`NTb$wah$$eDr zI+pi<4;|l)4{PZ0>`{6#G2PO%v@rXvH|ybdhrfl#f8<#9WSZn%lYW`l=s&vDx~E$- z_*%jgXz5k7trsV;tBDT_pOzjcu&xVydcd`n?c!d_pfs)5(yYGwckFq7HeYx=xh1ua z*EhH?p+MZiWXm&L`{l!Q(NCkd6jU5vZ?owin~dBlS{3r?1^UDXyAvaXa{2>`N|Vn7 zq9&Ca1w#0wCXZj+a$dW1lK6sYzLwvT6$G{cP7Rt%pH@i8WtkXw>0@Z zmIk%e6-@)>p)e0AZU0wI{NL)Sh812wh1VG!Ed>@C;DJH@+^3XM@ld$?N_T0P)hLCa z^_ey>+LT3XDPCOPWPJ&QDoI}VJ*$5i>E9huYZr~es|?8`E+8i%_@*_POZHw|r_?h^ zol(l!{n9+sT*l>M!2>b+zIX#8@ujcg8^1JHq;f9KIYb@QDMi`KIipvlgp>c>)zG*} zRb5Ft@Z+a{%jKlyvnon`$%~GWg+tf8{FUz1%M6G}yg}_z6p1h%s)KOU?Qxfm@euN6x>l;e|_@eAG7?+#g(Td&5TGIef<*OyH&kKvsM5J{gqRV?Wp5)aTr& zbun4G%|ounjk!_9A+M!`t4qodWHuUX?3o?KA9`9|<+1HOix^cQJU=jam^WpVY#3LH zk>r=Np{hp*T5)n%aL~N1Dzvx$_glf9N8>Bjl;K@u24Z1;Ewf}R)O&QRJBdNdinAAA4pcvE&3|s@5!5GcRCQ_kAl#e#T<-px@IjPQjgC@q)0HxS;U3Q47!t-rX(l z0q%|?8BtJo;d!{D{j&dec)&uleukrl%s7{gsREW}k;s*cHOkyP6O{GDhIWZO`Mh{n z1OmP$h=8#z_x!XMktAuxtSPl|&MU*cA}%3K=mUaCP_OSt^hD(}hqhZsgVPu}TYu1^ zI&OdpC)X*EHiM|)U zaZQU-bkq0cl17XQ(3cl|9Y(OUT;A7jibqV-oblE-j+JuAAobbM{)l4T%kxux<}X0c znE>hsT>ECIFV(LK8c7i$LxQmasu}+ps@S=iNPE}i%_uB)R7UNUH`6lkGi-s^9^yu* zEu0(ZE-jd9xA>KU;lwPVK6neXMq(cg>IVw%4MQ~R&z$XBRIBmvKb$o+syj~tL%%lg z(cZ+#vhX|{bKiA|K}EqV2ijq`c4;Igq@MNK3UrsvBn7QY*(euwM5z(pMac?na2HG* z{jS>Go`Xxj&RU2no3u{vJ<%O>cH}8$bS@I7W>owpp!WiB#amf-U%0zIhs8>jNLSu5 zIe(e*SmCU^1&ZT5MB)aI&ZN@RmvdJ+cH|__=N6(Cw)oGBrySa`WQGLI$LtyX6WekY zu)2v%X7KQguALamAr~6@D@PxsTXr{uYeDv_gn7TzY~!JbJg}w+278@Swn%}*DjwHK z2Y@hhM^7&I*tUWP=31=M@8JV~cZ;l2kUh>PQ$ISVp~YbWRnw~;}*#W zJ<-xefpzs?zu=3JEom>L_SwZTl_Q8nq*9~tOVywv@iYDFj!L@kCSRT6N{Je|oX#>4 zIo&K!R|mDQDDH;c_j6H@Sg{Im@47schdx@5Gm*u8RmOE*ef^9#ASzOs{Rc35AN?iC zEYFgU%LmuIaRx1%G=9w}S56c-iH`!D&)pJqsqvpqS{|CM<;_m$gB1zD6-m*W1@+<*5WYxq2I%bkD2d&C`|s!1wjb@ zq9n~U-p{lN&mJwcjW_56<-Zx9WU*}xoKJ%71@v)f)Jqh(1nZHIt1F zx0NJf*=$5y>_$HNS9d*dVdULqb}!gG<7!_4C(NWB(jvKLe!w*F{LbC;!Ql}`&y$Rk z0+5{eHE>q#q*ZE{;%&C0P-WKJ=&5?L6T~#ZF@1Xyh%|OQ(UPQw7xvkKGX1`XyMG6t z@Ala!t;y{f^04vOmpCT;2G3fDXpQfr2opR45m92fD2&_d#*x)US0>w>M(0i+bJxQe zy(e{GqGEL4`5G6L4C6?SLm&B8wuXd$KdJn)vnq8*?&4@&EO+{jD(8#4_ummFAwh1$ zN+KDA;yph+3dUiNM;vbtcUsZiQ24YCYFBwc4WX0_ldnjsCSBM{fAINy@TIgh^t%5x zS%miEYy2?2%X5l1OkJXs^6qMtZeg&W3j5~e*mUV3tnxT^A`5>x=rW}p6>3UUz?|!i z_rlnrLT5KUygUF<|9oB`~f zdHvr&G`K99;^*0ARni~|MV<(7NJb=N0l(O9w;xy)|^5~|ov_0Sjz#{W& zaPGlvZhq>s9}3H^@xN=Oe)#8|r)*Xl;@8L~#AFhDE<`AI3h%D_tsQbJ=s@=bl8$16 z&pHL0mrj*cdj!9cN`EhhK!hUsE5hexc6?6tM39y|Ej+t#isRl9Jj8KN$la2kYkX2S zrs+e7H~2h(*J^8-=AC1{6&Iix=_)5BcaKB}w_!7}O+Zy!mt5ss$2Rn6aJnSddwYW{ zL;+b;M5G$u)KDZZhry%md zYv%jaB+Zwg7Fpo!_{OoBtNvW%8zb368*5<+eLX`ftEh(}AfI6tp;0Y|(GwM$~E*31;3*iccA(;e;V+|tO z=;;Un4{IEmY>`HlXzt~vnI)L{IIVVD@GI(i6hn%D)Mn@xatuObgeXojxRbR@!5;6X zq?Q%;Y9mKprzk=|C#$+P4)rkG5=7jclfDNYYNBzH!&a3~RZmX1Soem&@l8u`Wp;K_ zc}&{yc6bned65hwG-8afx9sByfpte-lxZ-q4sSS(J#md11}s7EEE83c4dQ`{P5$Fh zsK@Uf+{sMHH_#1A-*4TdqAH3@{_vuh@hR7a2P(@iJhU#p9pXf0p z{S=|*hRdWfoxLYgipjxmhnrAhIDIi;-?=cdT1VP%OJl^d0{LV_U8nt^DV(o3mD)-Q zX=_DmrfaA$2ww+~Cot;i!IZ$GAO!o`6jvsDKn0OU@I0`WXDw$JTtFcktl~^yDLgfG zj==5g9J(H0=Z3d}JW3|sB)7Eu7@Ob(T9J(S!`zQV70HErcb=w7IjJ6`mRggZ9uvDI zt*lC#6E9%ST7aL#WigHA_A&^MQ}BS7M4UxuA!`8GWm8%r>*86TiZoTG5;Dj|xz1wE zQ||5&lT%RGHZyu$_o?nlP_@O=xHVqR^kS0u!ECZ+S#t;aJbx-IZ8ZX3DE-Y_qx z4{A#Rns3RmER+DgbQT-oyHI)=_2r|*H=1Z1O}wN>!I<4OH6{qN$sT07TtpXwk>2NQ_7L39j2))gW6J@BkU7lx9I*o;}9zi~}Y5JHsJz~6MH5qHRfe)Wo zEYS{z#o7z87nWR#Za7&kiCR!<AY}Y`yF}0 zeX52z#UzcidD1+@&UV2{_=(R3Ne;_QVG72iS`K4I#RFKX`zw8}gqhO~gb-C4_bCq6 zqV)w2)>?~A&js+OsjavZN<~ZV5#A$fc%t-GZ8GO=j zZ)T5w@tMtJHl(fC!uh={()&=`MzN1zN{%O1pIZ#~@xOV7j>l@o=w4$CMXUAS)X@|Z zq-b;O{2$q23wI?eNBTo}yR6eOWKV04Iq`OYv-gjjwMkfJUIyfuiltVgm?ZLS$!q~_ zuUcYPj+zPrN}pAGE>!lsddQ0^+4U%V^a(Tt4@OsO(mk+6cpwwRVP;k zR5G}1o7!d2p3*jQEre&8Ccyo!r6FKXS|HAerF8~B&;H}zn930&WOZ)kk?HG9vd z&}(ZL;OhK(!nN5J%6)LJCpOIg3GGG771KX!0I4zZlI$YkuHu8DXT2(mWCat{zQR%q zuT_bV6GF_xHCsOPu3ny zHo3as!!ys#S8nva-Rv;Qy7;R8cDKxH%b@*7e#!Rf3PQ>^V9p}mq=n@0)dd8qqX8=U z$mY?&6Rwc!r(X)FL`S4c@&6k55!~VzEkf0BxuomDMW0KWcsdH#j0lSH@E_Gtc&S-c zd+q?Ui6T@0DyLa$VI8I~&v2)*M&MMOp4%HeFOBc|&eohN`G$C56~`}z+yI%PiMMgu zd30B)kc=aYrXeV|gJ^Q4$nP)td{JiKgG*fa3oYElzkbsgU|F2bec+xT(f6*_Wq%?2 zqW~;f`CP&EC9Bb+=RyV{rLSX^gQwHIoY=mo3?Rhz3Y6?-3kFi8NeVW3fjT=E!NGSw zq^?_TGmK)%jzmQNBaV zklO4&S-Uf#zxWw9-`v?AUs4fYiXIhG?z{UfH`-gr_Hl&M?Qq$?5tsg|Mjgi=E*)=M zKii2YKbP*ftD)c>=s5SnwZBJ70FRjvPGt1U^VN@B z%G>IdA};Sy|8&HFORPjG{O2d4MXhcWYkohtdiV0M`$E*0te*xmU(DZ2P=8oihzJoW z_!uMd?vjpgUh85P819k*yx{*C?*4yXXieI#VZThDU(G3j#H`qBchd`#X)jaq@TS-) zD*0UVC$!FK8}I;b8<|96QuRi53WVE5`UW8X+dT+Y#~(G9)oern}5i-S3Q;? z*xq^_PYC7{xW&^mBED|pOKN6{(1k7#5OIDfwY4ZMlxelKq6!v>ZY;>mpNcfo$22TAG7#I)7C*f5gS z|B5JTFovx{Dj@oC9H@R6oWIq12EvbrG`9xs?7b}(K}vJonhsdyk;5mTEsth%X85AZ zPmtDA&XM}?=Ub&BUA8OBX~h7SGbxn#_au>j<~qab|Bx4Q3Ya7pb6uxCDklF>;qBVZgY-oa^xq;AeU6?7$dWito6`u_fLZT`$wdtGyxr(UMERk+Q|;x~^D z`~uR8BYB&aRlg*SrloZ#z2wJ-HhpMkhhLqv+O*neoDcow@M4C}tDX<9c!RLl-6}l`FY%0 z!pd#-IvP-x<@}pdoj7DiedtP&W8IizFZp7v(ih03_)XnVY(iU;d#mdAt25IeEvi9# z=*Yskqq;v15@UV0K;ZWuX=}lqs_xz=Aa2eT-|}p&L5Bpz1wAXq_5dGjcX@d1_E44# zac-d&vLd@(y7A}Jt$Lqt1%XwM?!GTp3|jqE4!y4E|4`eD+b?2O~VXadRfj z2>N%co}fP$ClGgjAtRm?#sbPKe&-rwzuy)g$>Xt8ERE#WXFu$%|4*s!FO_7EKP26A z4(6=*?Hhrd@tiNVzkd1KExp-ioa#(u(p;<1Z?181~Sy0|s?dE?X30^!9*OD|h@1j>GVQKJR7M^O+ynB8{Uu#>( zM6vT^6oKd)a#E1xBJD4b&8+}KG3L#9_i6sBb0?{Wr!e+M2EGsK#|b;R^{|dDt-p&pm3Q_=UO_yi!s&eo}Tou`2WU zaM3hcGkEFa=YL=-_o;;QXyUaflLZNfyK43auS`wJy^gy8UEBYwb}b74<}$tJo*VsC z#RIfJ+KcAeOFVwe7Sy5#xLrPO%?x> za{x)2M1DNCTjE#n7?kxu^m>tO%$J|62$dDv;A$nq2DU&1wtyU2o$#}-oH453yWNc> z8%d&?znZK|ve*;u}gf?a4&;@8UHri8|tGp6Nsscr zbBc{hIi8Ykmg5;vi^Rzc2)`29wYtrpu!Ury@&jMkS^Ea?HCTS0wT>=i?8lt$&m|>%SP>ptkrM1!n9X9*O9w9yMzG3S4DOiXN z%##oK(ZSptG-v~IE*2~Mb*?U?`^V3-`2h`aBobw3{$q0YRg$COT!X=yL@$Hk#LYxl*!;J7`Jt`Se4;oGs{*85I>FlKr9nxfhmMyV}#*TVw)&yK*UEi41-x zS0{p;nB565>(i1}!6Q>t0wH)ukX!CqCGcXn$t46`dkhq?ePMlM%W9b@NO~$k z4dHYyQQZT<%ajM8VqVcHUOPLdN!n;+^I*bQlfoMqXwmCdk}XzYsv~3$2bTni07FM2 zLb%E`;W+Tr3Y@WZkwUgBP!iFfq9Q4juk;jIM)U$d27@D$ZkC(vNA%%o2_Vv8Sa!V$ zn1Aci*#Z)_iqm=YQKnVGP00ttXOrc0GkLeQV6ZkP<}m^yyX=j=Lo}-qs2B$Di-Bal z-+;+pgNz6P)Y>n1i@0c5xWD1ub0aWm0)!q;ZXeq1lY{*#BSHnch1zkpF@$SHC(KPZ z&}3i-pd3L59QR9NJ|5tYkDiVXbJ3vpYRzx|LmvWV&1edz?Zq|iL~Xb00H#vs{LCzj zkiDjoz{>?FC-MdwK0vi0LMbuIT4QNqIS9dmVPMHKLr-roVlL*azZ@WAZ+?J`@gn%s zx7U%0rz2o3^87IdLlc$oQq}hy`y)lNQpv&cx>L{6AJ=h5<)wvEYNLJZK@b}yhqrvN zM=6r&b(36+@JR0liW?!C0-FWIs>}VU9C(p&FG82~R_`%#l)@_NcaA+7RD--kh%$3Z zF~PLE01(dkO*kc%edN)Yd#fGls6U0=a*%IrJR%dFVb{qmg;*GrdPnQPX_uPNFtw9a-+kx~hu{q5I-5;3Xwo}y^(QCtS~0+H$HEOKOIOm9x~ zlJ3#SoIdZH1Pa9`D)c8DURagu$G{?e5tnf8BqU8vcekwx=z5F` zDSA&{YCS#a#F>7VXnf6R2z>Qka<^)0XaF7p%ga96D%cINEy0Sfc*=%;mG%F6B9=*f zPcFfM4F}d%%kJ8^Bo32<{1t=>0?pv>+`{WZsER1>=B8hG@IuHVgCvx->f=B9RpApY znvwVYQe5h7?*AU}p#Y&_zh{`}nu2lxYb&t(5C8>%JTuEa%OFS`NHe#h4x7{2TAtND zVE2kIAdH0*US>Q_i=Y#)=9_oKU5>KT*&3n<8ZKC7klW;4YW+z;NGBHpF4cncG%gcHv zt5HuSu7CCc2toQ?qqE3wP4$`KQ%4{ur3@RaLVX{Ax7P1g$3lv?{|*rnYe9|RvaLH2 zhIl8bG96+w+c9ZV+RUm`7%V8xH3k<<)w{NXkv;Fq#wOslG+^S=L$ z*R7WC=ukKM1hN9Ja5cXJW~?y6?%;f`n9lBxO402w*ks(LxrYDrYQP|?&BVtc^w8=& z^Msz&&k%b~-%Y0W3z6cxG=5axJMWU5N@gS9pti6iH*xz*0^*ZC9A+x& zs*hEZ(=O&vHlF*{7P9j8Z8f}u?<9Xz82>Z6bjl0S)6^G6&pkWGufR`v@)_N)km<1H zfaBf6rI??q^0IC3N57f>sQSDXIp6-Q&2w6G{T;E5g(D$tf6dbVR7qzvU5})sp{gjs zL+}7Y%Z8;$IWJ{w?bbx3kiA9zE{u46TcAIPVT`6sY#zuR^EH2?z@r%e%JSZhR6ViI zF2!5g2Xwm|fM8X8iYL*2_f)y`f;wzfk%Wgj!L>Mjsl$Hn9Ew66ttBw0NlCw1jHLI& zB&?c!z=t=W7GVP4;sqzqxwM`D%%>)1u@oEk^3-GJ#lAP&X{?x+JC>wPkR>dZ9Zqwf zs<%^V+mK0I*ZE=X0s~$NlGf=w0GO3rdgnNX~AfY(aKv_<~MdFGt47SnZjXbY<#5c~;W%UI6 zY}=zS%0>g5^z^G-q<2(dxJ`1~D3fnL#0kOSAo&)r4C8u8<;w!;_suXSnz>O*_Muiv znl5nH7Vg{J(zV_?8CMJ0>1u$GhzgQ#?`6x1PqHXWk6^Ifp@6Ftd0vlNEW>P|Kzx~2 zzEqy6G|D0XD$6iwAgs@t=rrt3>PrsNxQl1p;QR9bX-5hE zW4c1AGOF|570y!kqMo03mU2&$6`oB?vqGwl0t@rUNBKQ=xze<^n*2w$Z?tr&_MHW( z`38P~W5*Kfa_QDHI}A}H{;mDKz@1)>HgBP6b(U@Z6u9)T*23J+-DkQxt(da0zSil} zm2&DKy8LeTl2cAC#)*VW+)zlEm+yeRJd4xzIhI-E75k;gPZ4lxc6#LPwuq$j5|}I1 zo!2QCGDG2Ou|FTrh|zNK=^jgc##lA9c%BV#eqsX2qC2}|U&CC^Xt~qKwv;HO(Ul=t z_r=Q5b&E)(#U^$>$~ClQ7nM345w~+~U#^Yg2{Q5Lq@90!f`U#H=>>#dL2k0f6R+)Y z$2H}t-90WdYrIO>yUWzEMpcMqzG7 z4e+My{60Dla4z4F*ht2voM+OaVh+8^$HcBlRh^&IMk{UNs9I>1*!?NmH$X7@sbY({NxiX`mJH+eO2sd^+z2P&DkuAeG0w(!n>CmpbqrQ1?NP;@ zD$UN*?R)LA8i`cMQ8yoG{j;lOo%f|E@E{eh-4~!>ME{gGlM02QkfWf%?e?cy%uv|G zt4QkM^#iHTiZZ>1$53RPSInQ*g7c*sSiOw?Dzv7W9|ITcZ6=<8o}}i9j_@1AhT}|% z+o0;iTO~oUvNV#4jrhllR}a)=ofy*|%-Y`LVCl>xVNK|80g?37LUm#xiWrVwE!mC4El9=l!}vWtF7% z@f&=7edErM$nZMtovuwb!9l##xqDQLoOH(5nw_DMpG6I(y+6T)X@0*d&5sLzL(}U= z2Sw5j&%0cQpE-bjZ9sCt{dch2>7Rnyq^-_MeQZt6hN%jTZ+n6fYKU2<8r0aNFRV<# zoP^dbzudBwB!slEcub7n6lZxzsWnEAw;bGE&!FvpvLK*t{#>$?cgwo-s(pc&AH$Wm z_8eIU=bTJy?FoZ^V2lg9WN`nJk zy!MM})e5Ox{V3*%+AD5?NsIdCH{lf~8xNNB)6#;0X%bu3h53O25nHs7v*9)TiTHTa z7T#Z?$h#s_mx2WYe1GlFzeL_Rd$u!DGmz7ndzQoMhRBjfd`_YK#@8Y!PavMBY~XCd z=6q7QdVN}g6(gaLR$6&kHSo}Rj@|02DQTONH?791<%vBq^|bpczjuWtcX~6zh3^wo zWMM~z($)FykhHu@isi*WL2p~B&3X1azG4#VKKJjx(syF9Nak+I_b*NQ?-yxLRW0M8 zH|M|N?V53@H=z+1;?5^E`!>5FGq#HBAs2R5@J`JQ8H*$w5j(+LU}Kf+U7^#BiH|8y z)~1RHTMDw<3QCUKuI=d1%v=h?@D&%ze04R+@|ZP{sccXfv76xgAz|3)*-M!nnCO2_ z&2vK9E1*w;noX+NKS8;_EGSJ=sjp3}sY;ija&M%gvhJMb_m9u}pKkw;JW1pb=>BV(uKezYLf5Aw1C0 zrKrA_!>Y~IPJq0V7y>wtXz}wHe;e*4^(>EGX`z3e83ZY3K#VB>v#}cs)v2n%61nY| z1AaNDgD`nl*AA)Tz+iTm$7_{qP+ZpI`v|Nvx7d6&FUV+GSbTeby%wB81RhWr9nl%x z@oG0IZf-Dnn=<|;H|nXLo3*^su~ekbnI5n9=)|YovVbg$+m%Lg6rTG#=kx{)*JS^d&P9F(Sn-88Po~?*{)78aAX_oX~q-N0aBJ z>b}?)SwHKzp^A+`y&arJw$N0-i)g;oW#_GLc=n8hvkf8+$_0Nir7T)zLf&;MXNyfL z-mCLYUehC4^NgsG^aHoBJuvi6!`=oj?qMi`L1Dmgw3J@VI?S>yM~5rN;Ke+j?=GUJ zZ$nhVqb>d5y6@;IiS1Xht9jjEoH3q|?C@-Ddu6D*73tr1E;q@4=drj9>>_EZ%cv11 zlQfrtBkDdS7MZE1^)NsXl%5sM8zm#pFB{>0yio4v3N%*`FvrPP&U*G;^+#uUCnMv6 z0EY6LJwu>z`asYBo&srZ)&#Ae3{)PjtoRegWuegoi>v}#fI^#Fq|7RROKwozPt}>v zj(pCs51@CYLN(lUruVJkCw)7Gis$Tm8#L!DfB1~vPgi~F{anP=k$R*~ec&;gYVG48 z85_ENLN)*$J#@5bcA)69Mx^#B1a|_wPeAzgXY-;+0Ia%nX5F;3rUmEu`rceuFu zpUohHNzzRy=co5RSv~qtYbLTA#Ezs4@Qjvl-hA2#9z=D(iVg3Fi7EaY%3uWP>>I~Z zDS^4vC(!1!YuBtzU|m^uu}e14IJO1a>P{yO<&$qqD#o&P5S|5G2yqQur`8`oYL~_7 z2Jpp8m>?$`5V+-L{NoJOla?zOzL4PIsi>I+LU7T1_6|Yh{e$r{9WPpT`(&|aM%xnw za850-sNLky32DWJ9Bjv3r;oQ?!2FGiY~hREBTG4l*7DhElX|a-I54MrQV>y+bOfYC zT|<=2l!#qc@MN=Uz-aV${#h;JPtpl*1y-Hs#RluVe#t^fb~o z(ma)8D&{@tgBTvhj?!6>!ZyCF1-jm!d>V++-_fdAN0eQ>z*NhWKBY*TeU7tOyB-ER z9T_+3E=kE4Tp?dU&J%V<55HGADomzvt8R1fv`+z zu2M(Z-*XXpJAM*E5h^z%5$2Ghvwr3grOny`OTYVk#+;03&slfFw2(iM*8{8%&&cv3 z@|9IDHSB{)=cdd;pg{C>ij~nv-pWX3stNFSHetqS`05CErXQ$C4q1fV3-c{*0>bTs z!;><+Dj2s(Bi!y&_K6DuT4=zmydUx`4lcQq43Rn?dZBoo+*f*ETsQf&feNw%q-mqk zc8UlHtyhXhDAfHT_9Qbjjg060X0&yq1woArcm&Vf(TVSoZLgkcn{1?bX+#L)=q%gu zIa3VBlr$9}N*Z)e%Ivpw^3!z<*!^n{ zM6zemjkKfwb1;2KWd@44GNS{vM^x2g@*?pfl0TvKR7GQs^9+hZ*TQ}x0w$r})S>g# z4*(Fn@nQ3hGllA9C@V9P*DQM~seE0O%G=~Mp`4L;)h*^}xWqyMJjOMCok#_R;;uhM zkh7KaxLtNBEGMPgv>ryCrjLV)k#kZ2vJZrQqq&)56>f}x#rAunZ>lsNRL2B3fuB$$ zXw5o7P!H&ghVq(X@XdUou1Q~ilCO*NE+IIF(jG@5e6z5SXS51PZm{tv?~zsBXPlkQ zMIeT$phl?b^}TcASc`D^fENye{(dFo5pFrSscHF;Ec)u1Lq1GNmbSN%B>7?{ntnk2`wMwszscC)O70tI z2+_>_Edf$&|{H>hU5ALI6cq189JSU*``Du0VyP zm-gl=oWZ#9>WwJW(3Pb%GS178H)KZ4$V`!5rky_QJWIea z5C-16dC2ncgf3)095->?d5rF`LU07|O-Yo#rRAk{@M{T^kzQ6++*M{*h;&T*=H~Mx zC#!*;_KdR9l0n=B)KEi?4QObI8JT%C}PG7Z+f3gN= z1+0hI`>;p7q~{!IMQg_Rf{){4gl@4fb9V(jFupmVvMyG9G~&bD;a5k8byH0D21egF zM0XoTN{3uHcl}9PvDx1>8C7WZv1d^DKL>|M3Wzx5T#tAO{}YOgP-r=Qy|s-aqDN3=W|z`NeU^iQ{gyxv zyaiZj=Da~TfcEPE>a}s8-Qwxa65pUO4vMfFP~uR({VxO+j8vr&{0f$!I!7ILFz!#qvhy{9n)?icet zEH)N;j6$9!R+ey`Jfl;U3c2_0jO9ZZ=ZIPwRnblSgZl$-C+b*rg1qTFnW5ZvWlsy*1NEj#BEEO7_^NT) zLB!2|kp&wn^d;>CwRA-itie$v5;kG-l5kVX_N%s{pEx!Y20g_y^7}gjhQ@&nOTV-s zsgNq^Syu6f4;7gKt7#wt@2qqZ@@V8)HnCDN6?)vN{JYfq{Haw+iIS;M-w|Rt`@-3I zKXz(vqL*Zj4gM)F3elAlIp@%O*vo^xTyk~__SwkxHTS;{1pc;^5Av_(@KSQ|x;riV z9T;l{W5 zv8UdL1K1u+nmxaDST}Z3u%jDO;+(bSoI1%wRds0reros}i`#V549}nH&u4oIVXD$P ziR@u*6y;;xP*PQN|!!^%*TX*M*vo;ptnb<>5lc z+T$*VArGCk!=WUnIL;Th++CR^A!`;xq4fPEM%R9e4cPu!7ua4J8k)mZpchz~?HYQoLZRJ%L+-6k`V<9wl+Zr{4!gc2qqxJL| zoJ-d6xBol6nK4bq=%ZK~HlbAMX8WvnHx%l74}Q%{te)+f4-OTZ_8ND)JW*NWNbE8h z9c{7g1K@Is;)_hi(iD!^x}wk8rgb!KC6f3_--T;DXs(OqwDLTsq70n|)nq-)Q43x< zT){49-sr8-OcrpfJ4yWc&mZD+WJo9{;__y%Bm&`?L|i_LbEij!R4G6FTw7F{dp~Ng zwkUf~GFoh#K5_BBz0u*~!ct;7gG=RSdx1F7V?|!e*@lB%U8|n;t7i7r6TbRqcBvq& zx`=BLDN_ddQe!UN6&?U4RB?wtNS_ ztaj6K$Nv6~nEE#?FW4FT2bjKZ zLo*wtN?zp~p{@!f(Y3_!Wz6>J(S;_X*%%RcVrdlaVo~+$pq|Ftmv$O8UW?24YHWez zL(%H2clCgJ!O=}#OL7ooaAWteriVsFGrYNo9wv&7Z;gfxCT8w?kcTlc>)#`v>H+n5?Ke zt*XuMc9DA$-Z$>46ruP0b=IPWE0flANZbtG{5F9dJ3b&vcH4BAfIC`Fa=8MN$;lPi zdtX2BN;zm_U5WH9-@7d`?QQ2!>#j1)|EqSG{@%zCPUi9gp49{Eyev`^r1jxsE6M%} zem$n_-5Y3XuC|AUi=b!6KzoH8Q$(wp^S!)B2dReFgN7n|n+-wCe#7~GCs)lqmz(JS zEMX<_jFUw&kfnW)-`V>}jr)%8ygqEK-$WO*bB^1xbE`sM zIAGR_7$z-p?B+YT0A;!L_s)esWGALxqT$a48aG!EecMQg3$R_9vQ%~p7e(Z4LTco0 zgme!zbFjP3XB7u-UUsS{c1=E}$f(^9;OapfalAzuHnLUHuI#>+!I2VFs0~nV#1(qi z@L6ndKx$mvqB-aRY}c~XZPO~0^9=vouBnVOVNm|vx+yjclkl2>S^fkF+H5l^oA}1` zmU76ybK>1ol9O}KLO(-2AUVLHv^S@WX#^mYWF&~fD&(R%>f$)Sso*U6n#DAt+&uj(i`=f!Thqp z+XX(k&Q2^mv}wA3EhHco;;&o@F^*nFh5~Tg@!>{g)t8Ussz888QPYljiwq0h5W)!1 zSbBC{UM}OsjdWR2DLni^n^)t*?7Vs6ewb%bP4@oSe8;lIpU-CoHr?fm(nL9?Jz?W{ zO>yij8z&xMZbxuT?6kn}3*2J-DF6glhJ09haNq+h!XarC zw|dn&1$cMc6nyc)BOqOZD-kcA4*d!64rP^C)g}hTl^f)?;j?e#aMWkr$j~lR;Epdz zJY}c)GAktUPR3CLD>Q94O+TVPI`)?8$ko3;{eEB&xC0~A14NdFe8R-aZvt{}&vMh@ zEIep@fBes#Y=_Qg`Yo6iH#(V(W zVpD=IrzE3jc6-B>Z(;A?D!Z)0_?|27k?&$~`37#ouq0o9az>Ixj%Cc#h{rsD{t+1V zK5Uck?oqIsx;Rd?LG=UFofRqLh8A%9Kh}zdEU2ofYfHf~oIJ*1gQ%52>dXBQ1oLUe ze$>CI+HwRx@4c$BWd$arNffp^dHP+&6Amb)Kl5imdYSng3~(kX003D&Qjkft31ty8{f_J2{V`jT$(*&{k9i|p8?pm7oO?iSi7Sc6{0m$Z%@^$K2761m^ER+-RVx0 zzzH9HD-QZpIgzSvdFNIvZ;Sa(Z;MB-@D4Inm{Rvbyr|-y^@T_#UaR-<$!G z`OPsD$BLY-S9N(1y?fl6JT|jjroy)J4ZgnGS$l2r9NJ4U9!rtn08psswXc%;zgrzQ zq$toi4@8b~A5f9V2)o^_0vX&)O z1*JvP`FDP}w{p`#At`&lUj+H8q7iK+#pp5b`Ja~}tC`RxJ(pk3ewuJnxKy-Ur$ooy zR<7HTXK`TvXsCAl_Gr5(olkr-^+eqj5gcPdVuZ2Y5DqY`kX?aC5QA&tByW^?kEk6-<60p(XsXHTYM5Fj}AG9xfaM zM)e<3Ed~Of6AicW*>|8QnO+kGsO?;Du&#?Mf@VEOq}@3O{Q;-WGv8?a;vC;`qcV^B=}Vf@YQNKx z3kKibn`rivD{Zd_n3yDAt1#6IHpP+*=QR7njxM;+&#f_?v}B8D7y}ZT5+1$Mf~@nC zmOA^z<(YmWuR{Fw7urq{fJ#9~%WD2N(NA~A@QF?~Y9s2Wc9)a}NH5A>>U=rlNABg` z*knH+8+EWiJ-HPcP3;yu8qp|qg7?2q$=AZ32B_KRC$aHc8K&V$fnq=S4kmUwIazYv9`8z?FHUv6BR`pLQ%LC-0@Da-%*5?RUL1(vl?Tk9t zZ9b>bA@YWio_sW4Y^jWL8^esa`vxD!=hgNnhgLhWlqPy7s{e9i!6bZ9gxxOc|Eci$ z#%+Klq<-uCYH}Z3hD+o;aqKWnm?7*ZtW%eAu1e#>w1Y2s?CuI_yNbfbpa?V5hcwKs zpUY#VFAtWAiuz}JGbM~`EIiXP;clfxrPDrN4TE-!k!ebBL8|H72&BsU=aL)usqBBZ zrs}j4;wW1_ty#1JN3Pz~wLt-t<9E}j;iI=Q=Neh`FZD%WQU-6oF3HZZWPWNGLo@Xf ztLIFyxmPu{-uXlxfeLfg8LQ6^l&~{tVs9mMwSJ{B&tF#P4F3;@CIX|_p}09v?x;9L zkndOk?ax=%27adbFqte0I?K|Cj5PHsmN7J`MZ!>}As?viKPLv^MmqC{2-fTXK zKdsXVe_lu$bi_qqv1HHYwg1HBkPzCV;WV>DE;S-xqIJ{?hlR)9`^*#iruxafom@3= zque>WcC=kRqq%-NWY(81h+Xv&F6@Tn23Xe6>z*8;tibRUu&}3rW^lM&V^X{a*VBaAqLw7%U zDn|k(FWimx_GGsVJQ;Mt_EDcr-FWrXM;En*sZ)q8?YOO^QBBjw!lDhxq`TCD3u94I ze{WyLABp;;&N*%GT6tRs2sSavMvCj<2Ukuvx80}opYt%*73y1|&0ee8z+Xe$RrsRH zngb@a!_*tkYEwsBTJJLNK9YIOBb3O#^J;z#$>BYnUbCQb$fFIDAanI7IQ@PU!D=|N zeS7kq=Zi!Zbw4Ikd;(Wv4N$^yv_E%V`r#C;ta82!otLG-^Uq|bd7I+3B>VccSJ>K( z1l#g&e6bP9sY%k+&SoGKC3B(QG$yEy=tdw70TBvnX}%e=6m);p(HLlQbgd5OgNSLJo}bg`3TA zhm3IKKhx1Mon763S(<2eWU-t}*wLf?l-)<;U&IZrt3R@jKY*1%V> z$Hf1kzjMy9=S0PsYWf~EgouoI%{5(Ue>kD=%up$b{z+MDN91`%cG!-p;^?if!wze| zoQS$+(6XtrVKk^Z01B(L@T%TFh>sP!qNklMf;B@fw5#ZZ{qun6MPTJmql02R=G#BX z86bivElN6HVF(5YnejOe39=|m5@vS7hNQriLKjA-VYSvO(|t=$95r=9bW>B~UB5LF z+sU%mmB@sPFqOnVwGIoAFiAMxNx-V5SB1?bA^B=VE>5N#6q{r*hRh(45Dr0qnk3eE zT_S(D3qIKAV-$BkHn`lO5#GSgjDU+fIOlgjv!m@|W-(SHN^pGb`BT-Hp}L3__k!Mt zsmiU^$VMF&eMw6e9A{{TP)YHUyHuS0#&`KI$Z3ZGP4Hi)uO8BxWm?cNUpl&RG^>Az zkJSFqFqE+G(Yu&CPHs1?9`AL++IRBygr@h4E_k32va2O6dz72FE7h>=859Yd>%XRV~i2_jd{q4+ZY?k z!z6?J`y|NWx6P0vsTVrTqs7T!`cETNRD$K(-h4sIuW~R^H2mbDs*uv*vO6%0 zcaN_-w2Rw_<+_yJdG{%jV6%|uCuo1Jw7l)E_%pljNvDtbummZRSZ*8Fj)v!PN)Ev? z!S%kJor3m75!Cw8%EAN5y1-MhRX2O}L*R{b)CxkW-k$AU*B9=K%w~x?RaPi@u`Heq zP7DpKCCnrhq)FPk)|iS3T101b6OyZHL94P*VcG~E)3|RpX#Rc%Ib;S023ah+=JN`d z??w=l_#TaT-MzNyep@&g9`a@>$C^RxDB9Ke>hpmsL2kHdr)C*;xo2VTS+=m~3RZ`E zai#w*yC_!J(qi*KIo1HRNb6?16v-R2aanM|5_E22bp3uFvVf9J9Jp{hRGj^UcwRua ztiXOuKX=P|YJTeXjeTxm@_@swq5XM+XXt0z$m*LMzn3Yl1POWxc)!nj=MCFV2ZPs0 zKh*<5b8EP=n1EPDKbKr1+3wA5$F_zU{A+_RND!tLB_?1&d34R&-+g&{%CE;pJ&@^3 ziOKI)vruyVOxHe|EuA>ArYS5sZHTZEQ|uC+>ORdy^UocrdY{dJ$?H}Y4XDO@whI7! zO(?|aSK$KiuW*5p<2M?#kMH9>$t4n8_lCwmls=AsQ7`~R{8d02d>p_@%y&WSjymK9 z!J}4PQ@|UWBm~0}^wim@59L z#ih`8)#o6>lo>gLzKe>$73S3rCcFOwSMJJqnp5_%TUkNTmpBMH6Qa@3wUHYz^kGJW zq3&rBu;q!-L&bp@rDyohdEf>Z0W1zbsi{G%f8q6Z=}C}P!w2rO$jps2Rkt4k!P2YH zl@iK6#;5SRrHz)CgK$6$RNHI|X8%-lC3u+GBElux$kWmB*wpa51`}5dEM`Qz3so0i zNQ8RjcyZ$%*!fBKv2rP64x5)SLXi#bjIK&{hI=2E{00X zuM{JP+GK<$aGonG2U#x-uM>;9>i^~v;)6*cM587TAwjvmuB35mc z4-Qs6^fw?(L>Cx$96EB**V7+J>NJhNH_>Ryag7d`RX{%T7BLbOpWJe=Nr=h>kSnFv zNtI_Llk|1kGnr~vK?{eyZCp$O?HVjV6n>Ly0hPs*t&n1zASC@Ed1V*$x7!ZBWu-c` z2`9kf9k0_Oz4tAL(p)2z#WZpPxBp8?rTLxWb%k9?55&h&gQ~bZ5X}>jD!FWi>yV}4 zg&+o$!#L6(7^u<^b~RfOrZgcYWZouECCqCw>UU@3^^M8naO1ceRCisqsQO;pj{)Kh z_luIBho^AT%O)$mj;W^=QmZ7%8NBuJJe?qFrWk7i zcs!HzTD5&LMxG40bcbEby3*oRP%L$@7wmAMoh^R)H1w=@J^mwQ@;gbK^`?OpD%=Gz zxt6c0+t0zfK%8V}DboU%D)N5CA+RJxhqu{X)D1#rO`Z*MzN-bq(&5)wkc&EF+U~#0 z#==42Zz?sDbCEaHkoSavj1$Dnm{nv!OD?ORyy59bfW^>@lF-81hZ*MX!}izquC1Ku zP%KcqgtxR6=v2)n4Rf~+`C=VoP|KDSFIA9@NgXK2FZ2aUAA8sQ?ek z11HuBHlFWI*Cyrw8xQEZdjn1CWdBOwc4gVfwsZZ-ml2rA636uiC$hgvg1*$n&o5qZ z4VGF`UXh9zayfoj7^ zJNmt;n_VP|jhlC7t(S2ghAbZpH$9i5tkTt>JCMxW-~Da7X{hM2&s)_UR-2ax%YAKX zLp@fD{FZDJ@Et-t{LP8CutimZ*SF?$E#*4Z>d%3i)VUU z6g1n+=(K+Sk2dmHB%8oUJo%k|?A)6W|H}cAaEhjWs>f=e>h#A_Eawj@QlI98?;=<@Kkyhsm)P;CA0p<9d87svNab3)7AW$`$_t&5&i` zDv6VP_0QIyzKr!<{u9AJys1_u?I`InP@5Vm=4@yRU;e zHQu{Oc@HWw?}mk=0+mg2DK8R|5O|y+tK|>thvJ#tCbTT>&;S0O$4`sx>|7AhjY~J; z*I904-<%RihWFd3Gw&tiI&qBzbhOiY7?=xe9#`h!+4Y`jj0R!l`;|IsZW>vgl>cHfo!Ii-@`u!5 z@I93-I!o0@@c9mkyzroM&Y}89?2KAg;Xn|b!$ZgJcurHTXWRhI{CC~{`#n8#wHTf1 z-IDD%lE_JF3X=ro4Q^N?f3eI*lsR`KBDQCLy)c)L;bk6z^@NP!Xl$ydzmK40j5ZXEDpoa+szOPfv&~ArJ<_tOAE~gwx{xMJE2NU@4M@7t_gEZgaG- z2nmWsl-|dPW+Z+8xStzs(gkOU9L7cUJ{PrYhXk&4K9i=-=5HAeP2n--F`caY4OWXm z4%+c+w9u6*em6(BaOl0H#5$Z@-)y@7op8KxA-6lhc&~N6n0349Zk*tZrTE^!U2+-$ zt$3IJXbg^TB`blRxs2-R5cVCu zs#^4*rA4Z#jIp;9^p^eeGL_fQaPRy-KZ8w#jXGnf;%Y^qUE*rGd`a1x>^CFnWKB`h zzBQLvhkwpWu4=8ml^1hG4BmS?h8Lh>jVm>3GglhWxG0IA#el75ojkrq<_!7nRqKAo zHtaXYvK*h;qj8S~ZsTzF4h{p>ZzyzI-M2hf*R%1Prlq$FgnLE4qp)U{T~_BBb^p0v z{=KLjQV6J@Z6tS(YVSvB28PkN|CTn8mVUiJCJ@$|qLl7mws7~^w0xy#T*dVqPxj>9-Yh=FL~V_Q}2K~$|DyTodq{1*r9lJAkWk*%EO zqB9YlnU*iFn!yGLLe0BU@$#1HWrJ9&7Ot{+)v$b^?Y0~;bNffz4Q@j1I`{|N7l{cB zM?n`B6}68iSp!Fkj`)(V&4ML+)1slBq+a7++Qv>Blyic+gXvqc1bmY+uX5b3Zhu`7 zp^)_;B*>EF)#9C1JRH4ko+{P~Ui4`_;czpgT3}wxA*ZR8+wWGLiVnq|APF1&Zv=3Q`XU8;V0CP7Nb@lFyh0#T#${wGDv+LhT~>iZbvUQ5&dZA)Myi6e0YoCV5jNLQvB3NJ&=*=i{zWZW>Q%4aq*2Ux?_8P%>v20YXN`Ydu$dzDlfL|0Ni6y<-M%%_sVC;hAu4N;--~IQaX<>788laR94>-_@ z?J{sv$3A8ifs?759Uod7@4%kv-wPzOY8=Y1i6?IrQ79Bpp~llwYT*yx2bFKNp1y~? zl~xeNo<6Dozx}T5eP`d7$*bFGA{xTE?=3CCHMTm`?}Yn=ceB{eyV$+Vs)NCkj&sc6 z%FB+1G&EzD?7|1_;iPs&hJ$HK4=k=@hzC}0UwY?euI}bNXokXa{0v?Lx}TijvpGSD zQlPI>w+d8np&_2x<#0%!e_EIiHa0&5g66jHV9+g>*q@-`xAxc@!T`s&sDq>~g{1=P zv94s!hp3N!N@w8=rR>y~T8EEz3L=*;3E^+NYkheolm6!ozltgrBvuh>=Oih1P0P8An9 zYkYY6eHSS^6ZNvW#&3=~Z}2)y0KHKkOK{YdN1;ChF^kIKRT9ec#DUi2^diHrWd}h0 z24+$Jh>&ML-{rzyGv!ekBY*TOfa){Ce@HadV@E+sG_r_ zzY+SP=N?2Kzr_tFiT7UHo+|+-_}uP;do(aUchAZHmpuvDXOM`AS8P_)+KR@QJD8`Y zI1Lo2VAOqzT4vGTwvQkB!?WYJzlAz^t^+M~Hnl*Y*ur;wL8_$XS7CFR-DQv0AOIl} zB3N)$hl^Ec$jeEtL8U#-|NUM{{_);*lcFOsH=3t-)3xmJYGTXCO?gA2djqs&T2N~eL`QkGqjy$Rc>s3LL^ANJ&1D|^-76I~|`x`>MWInXa&A+cX# z+MpJ&0&<5ukO6s-oSttn*)n#PdMRXmMY2&!r^+<6Jb&!TSjvZ{M_97$4&Nn6bBX}k z$Oyo96-unw1aX59hc^`pa&IzEPk)_A-AarfEL`mpnKnvfp>GZnOEC6c27#a%?O7S1 zasvm+xHDBp=C9_M?p8x{AC$si3j*4)#2=1MI{q2Ua_G+_s5&M|Ml_GWtH_zhZd>Z| z%nZec*MIxzSXk`$;y`WMdy{6ewPBmBLQevc^SyuPIZMu5R>LDi4`db^%mxw%^QzhA zCR%XwSyh2ngedV%liy}#dzMxPe9TDuylgAXph3JKsXvI@|EddeXmKDKT~!pgGSL_S zT@Lo82bC9IRu*5di;vB;RO(TT?#a%iq6#)O8xS8|56o-0WFA&rj^o8ap(MSh8wPzv z#RlM_)jJUULrxDF1CTAoEJ9oHJ@+eE7t7-RfYFp^iL)g!m__d~$5`Q>z~I=xfu61j zIjvpv=2T6PLq|3bO0wozSa`PXG#A~LqL2n3Rt=7fR5qE=($EI5T)&10t|<}`FTBn@ z=y$03K{ZC(<7>TNn&s8hEHG-1xA%O@%akIBQS1SsUnTvx#<#l$!)M1`uh~@IuKo!) z<%M%;poX!OFXWY}z-?>9R+?%!AlWM`PfUMg5ByH^{s^=-T+eo{=RW2A8Kmux;3BK` zO?CmPsr?KXDI-AS-S_R&9Y~#Xq_F9UgQ2Pe9VAFH*MYe9yoDd>7dM%~1dL-G+qrkl z(F+ee548n-mOX+9C}gL17Fa|wnLvFMz>(%UOMB@H$j@EYbwpG{Crk`kg$VhCG?8|#aJzh*zl zvFViBfFKHA5PMh%=@U-G3{82aY>NR9p>f^$!@P zL(2_^M0H{Ymw&Ou-QIX{EoaL^o!UbyP~d@hILcSqWS!h~>wmjJt$>cd(OYxB089sd z)g1mwXPoZ1)dBQ7*+RDW1dx|f9G|G}voW~gkCT8Pcnnu+?4xY}%3Swj@}_$09CvM8 zl!T0e-?@fzf$gmUiagq5Gl{iV{E+|600s?yw}xMNAZiviQ!eL=2Tcu)OI^bV$F+& z2X;Ue7j?}UcxQAF>%OXCK!wwb1B&|Z-v06ulV=?+dU>5v3WrK%`()}N1$P{vrh5fR zqajG8EzxyOmi33FNrNE>H_0HS9(*6fXj!0E0&+NCRrHJ4==L{0BzqI7P_b-j%?(QC zcH`?x|GJv0@5oo1-)CYy6E8a}M!Q$x3!$}c{V#SdDSdEl&{-q(*n$(@W#{sSh&&KCHCHd&k^R)4&)>F#_E>mN%TiR5 zlXtos8mHB@_!@S8NS*bOK@|RT*)<~HE06?7?ZY$Ux;LUsJZ(VC%*B;tYap=;AKh+e zn!4I1GAdjMPN6<6f8OHZJYB$V;J~xiFih+Hd`;;GR7;ZDCLtiS4k8f>srjrx%#j#? z%82tX9BoJ)sheJel-gmdptjF}1BAi)4p6trzmV$)aC7BoTm|0tBG+YeeQugJ{k01l z3@x7$A~jY%Z?uqYMISh~*~O`=fKDN?oP<$XRzk-QTEw!JWnu5aEi=IAb5fmmc31*9 z_w1+UMv!>Dz)im%TMuCBxwrP`}fNJ$k|y?Wm8#p+o{5_d}l2Sz|+@P(gZJ-vA2+CBl66BPR^P<3figIJK zA&<8UIrKG=8{SQ2p`(4VcY75OUQ)fKpnpcmZN(sabc40sbw-Fgo*@XTgWrldLq(=J z^DYp&;yDCY+g>MQk|>ils08^_y){Iq(!BM^sID&AA|`2O=iZEN#EG5Rfl4rKr$gki zMDaKsDr#UvC`w=yC0uY0TaT_Gn%#j;cPzuDwcS(?LaNwHBF0T)41+xx=WlQ=NM~JJ zif>sS(!$n@2td+@Pt4mL%J3MHTv)crv$6zF`QIE>3s*~2YjB9K$oVT<_C^rJ#KyjP+=z@xmD7?Q`&iQEyX@w^fpYj%-8En(9x{G)J@x z!Fl{?0@JDxohS%fq6|n|f(N8rk<>RFDtrCfi})u0DFC0e8rT1iVkY$mTEDO~6m;8lc9C(lWOnIDD-qt0MiqSk0A}R&I{KEKqjqx~l!u!@yFO6Uo_f0h)g^-t7I9a!gpa`;k#_N@ez>Q|y5}ulQ#OTI_Hy6m zX=H(n^~GvffZJR9&xb-pMrRC-K6tu+j6U;1P@%`9p*H0{q*|q9v>$b?zxRub#(N{d z`GLeo740s@{U~SS^Cy`*5qnf(O#|DYzU&tj87T#sG*zGlt81>4rwzzHddg!*CYyZB z`-Yz&-cugJAkrYnYW3GLvo(L+jnpNMsA)ZiX&tGV%wynaahncv(;T<*&Gu~_@kY6Kx@%GJUL9`+17mW9JZ#T20MIRHS8eD=*lF){%|A2Z)sdx3cvEIwU6okpb zJ*}mCjB5j7euqDqPF1M+h)J#@Wr+PDUcYp%s>MpvqP(Uw=MFAz&_2=nGEz#%qf`D@;7>!1OQH@I-nNCm^dz2d z!Hd+jc4`M7Kdo3wYTOR34W(&+j(SNOx9PrI5GXbv&u=-y#|^{`2C-+x>ESl9w*Gi3 zvaz)B3z=<#Ytq~H=Vnhdm-A}8sAUBy@hlFG%fFLOCkB1$BYYTjKgolveH_yM!@a=F(t7%8?$=miQ*Kq1+Q40nO(lS63}WHG$=4GRXp3AFIx*CeJ_h~v zbQD**{+zduq_E}3>^=0?p!2vbN=pvm-T*xZ1tHdHb#o!~rnzYbt!3AXu~=CY z_{D#+Rf~@2ws9LHGr2M0koNohi<=)#CN!ofbE9dpjaw9gqfTD-vZ;avnQDOoz4%_! zhXhIQPF2MG@wCWu4>rf>du>^xLWM#0oN>qkUDdpO+ISNwNT1L>S($il_2R}h_4Lms zE_16{<}Eb4##6oN!MkL$O=v^? z?nDF8QX+iaob_B^>K$Go5Aut}cPK0b*_(ylA{%-vA1Sq#oQ@&bRQY|ALjT@K zQ+gWxk{#_7o+77ZwqU65jtR9I!DeepMHnCOz)7Y&JsaUD(d>ZMn{QyFUIWRG zLw4I(jXmmD(;@Rk}dBQ~Yr#jZ-5EvMK6qlag2oLAbAscaNmmtwdYT7C?fJavTOFBDaj z?G2+Qe@)c0&vd0AtACEw2i?Rp$wjLTJ@zQL91xq#KNy#+()HD*Sml2{EdR<7G-E;e zwtzuG_CMla0U?-w>yk#TtaMMcG8+;?)&r2=tF{AI- zvT$|rssfBwOJv5*P-tNlFOMiX`Q7+U>Hb&5|5xC+dh#9{IfLKcQ7aZ`&Z>vmrH7v{ zy}O0st*HEkCC_9IdFY#G{*+y>FU>o)<~oR0&;*w*RV@cOEsMOb-Vz+RnZXyE6*HcJGrnRU48}(C@?(hc#6?rV^tVVg4uQ#TB=+A=?i9N>I2pJ@7&~090w0`|aZHBkQgkRk`t3T<6Ss`zLww0#S7q9C z>UaEM{Y|(qJ?QKk=Sj+}wWdrzWfDtjYFzN;c?}J5_V~QKBEw+7f?;gFNe!y|IBq== z{awf~-{e@>Ev6?tp*&0caN+pGc<&{>q&g=wr#hG3JhEioLU#|-h555em`FC0>h!l` zHdDg_nv-X3(?&gDw{<;-cZR5pQlcBH9&5MYif?W!9{*^4?zb*2-Dz}WIM@{lsRhb6`Q!CRlyXE4ulpWNL%|JXXu-nBY{p-SMpd8ulhvD$c1e$3^ zi8V-)g5LOb`^gQ;q7S9ygjLO~qS#JiQn*IoLbO_hm~ygr-aCqX^@W z_hykq8{0pRfBeKMm|Hm^Gt{wcB@gIV`fBPSv;w*o5ODT{=`vn$ zHI3_5x<2aFIE1O}ArL*0?a!5KKEFTy`N^&rj14yE*|47YLn91!YA^*5KTO7WtGq&n zup+SmfJE>o6au%Y^>jZ+LIhbAhIQ_$L%l0FFwT(rM>bZE_N zFuf~dv@vISY0e#v_xhTLPYKz&7PKVzRIu=Iduqv*2OctqAYF)L?m zkm$V-;Aps`Zav*yBGU+8;Ry*%9M3%Bh3w%xK7IeZ@3nTj2AUuFXLAiVyhP8|zN zrQJkAKtGwz5baPRlVEa_`4E=}hj4g3c}^eea&+w~KCO^pYwP$D@Q8!x^072-!;fh8 z_N=(Cdo`O=*t3LC`<)4&NJ!yLxKTcryT%&T63j!T!(PV3gvkZu<-m@h>i2< z)76BZ<*VwifbC%KL-?}uV-wEmyRz8;^vwH(J>vZvO2-E`_@P5U$8^DOcl%Ob0DM{wOoOy{Av;ibN~IQIDad&@@KnB>X$~OBFycW~C?Q z0ATN+`&(~&WTljhUONGJ3(VpeUI)k7)7H+i6(4Gf+um9jh@k-GZwIE|w>A=FAKSm4 z)xYE;(+>Exgx3?ks5I1%n|4G(TJi!TKGp|ON+PU<%3BNaG4ho~UqilAwhnkHE&$rO zbTBQ$VNe3E^M}GKRP-OpiiH$@;t_bL7;Pn(ih0tai0~3 zRAg;r^QN2t0Jx|_QvJnRQz4rvnL$qLI|2!bdPx>B2%6BLBH-f82NmHA$6`PvfLijY z$ZF>>kTxvG7RU=LU4+X#dBjr`?|tz1-V*sD>1*+ z)iiQ5F{Mt^Oe5q>0WcX)MSrxBer?YtG3>Uy7zSTl0iyzt&pN0l8)w*!yoS1 z?Z>@0Qk{sRiX1e!O8`!sFGymcXt=A)LOT_g#y%MOfvVYU@xi0WlwucSRFgV+0W=N^ zI8nULvRS`u>U&;@9*PXryW#*kyZfv)f7;*YFV`%Q3T2`mZ9)B$1@Pu7L9#NkMx*j% z^1y?h0qd%R1?ed$!t@wqcLxTyFy5m6#BoNzFbgD#463u*W7cw8D{t;}+`g?{npc$SC#4=Uw}O|iA~Te_A2WHGZ0Fc7a?j86 zfU<uP!0Q_`~lFLaYE$N1ZINj3>IXUi*49~R)LGp-oJUiD*Bns^DFApG!$hSuiyH^IZ45REI`l@`)X77C;#-xjThRJK}s zfR5s|C;SkYG7=g%P{Kn3%dr8@2Vw(Wuu_wYlM`Jy}mA7TqSEW3Rhr1o@H@rwR53yi%c7~WzY&@f<* zYBne`4pkQB3UaPvU&v^LSgSPRE?`o8LLa+!IuMXk?(d_6K|?|M{%?;Xr)7Nz*w3%L z9$_jNrTDeNF1ju>_Ar3RlYCJh1S5rm7K3~y+osD~QeVgPD0~l5Q?np>fDd@Rl^u{- zH*_g}CI|UeyRug!<;Ucg?%4*{h`wSXb9?J#{Cm-0ZfC;j+IibNr)zJ8KQ|1Cc|R|S zc{_3otC@5mW}#`dVeae8L#geSU>A0UjPj;I4sXDYD6$)Xf6w>ng&&n_VO@WkwX zX7Q*tt4k$B??L?PrM3L?pQ5SBPD;#y!~?0aoX4-ipFBHNFmxajL5#gd zC;I1yF0wrymhUfi`=GR@Md&$Z(}c0B9_`h#9_B10kNMG`ws%qBaOquTYx5}|+b$~Z zjZxO6A?O|X+!V3yU5DBU_<@MquCV!Z;_*Jo5{I#+C59AsO+3&Vi+gUHBATym#zDsaByB0fe<>AUe?{NK?#u0vbUZ{jkX%} z`y-e7i-X@a*ru>QQ6e!^?Tu|Qus8mZm<&vX{_R1$=dYiw82CnbR}8sx42$^BQY5bGXJ@KkE6Nq%(_p_3Co;{jhAA|*GRY} zNjB50JdfiAUEf>Mz}}jRT^I5EYQd+cZ8H1>j=v(+)l0iklEbiN&U#64tZC|{-RGeI zFA0ZwT^3}IWyxRFhkOD2G{fgTY;-c8(x1Q~zCRp%q zj>Cp)2+U)a36;7y2F8e+(4p06af*~`<#t{yrj-3=)t$%nM~?4r2j8Rh3O_?kpn-BC z(90 z%-hGU&E;G{l`e%QK3kMl$Cd)z8P2$srt~>`Sa+x=PSnv->7O^W&;;D>A$gSQesc|g znuWKVq8zV_1Hx-iWjiUTyM{P=E*sKqrqN^<`cm^XO&@P$GZe24C-#k|_z0lEUc0KX z8f9N;r98<;_<1$!nx!_2awU>`)t#;ilA_G?Pl(e?kk%4;i|BS%$;?*WAfiuc?Fk1n z3pWkPCVQP`-Q%l!ZpMQ14#73d@Xs<4s55(d`XG{tk)CoB1kYMd78#-|jZ-_K`&7 z?#0@(@=R(#{ndAzE8+^Sml^!d^eR(3*qJM6#(8HxW8HIZ^sJ22GWIODO+7>O@hp)Z z@B_f>g739;!_m@Ma4Kz-l8<9SPZ!4YKyzk=uSP@3r#(!>=|{QH17=+8+YkJ$O!DJ) zBHZYv-@RG=4I-bnW*`;mN$K^0TNx}@Ad|ruu8_83E(f&`llJGQa@|#}?ibfa+ep8z zruxgVkM@v|5xOU<-6J8*`sw3sw4cLE8B&x+iIT*6=+Zjh{t`5@Zb6!Dj>Ghg4ty!u zUjLjN;3{tnER8JOjdSgD>RR6#JiRMN4jnR>lue-{zvhax2TLP)-LE?H8!M?cLhfsf zhJ2h>E5)Mw5yCIy$8}NYqAgsp{qOJ?t5Fjn^efd%jtU`G0Ksq0SJ8lEsmn5l217Mp z1@#V_)HPagK;)ee(gfTRL-<79$Y;1xXSJfgE_-M+NzE6rX{ri1<@Y?l)!bEKdQqtdRTCi!K0qzx=g2l*|$8Q@Y-UKNBwFjAyC|s z`i|9S5q19;*L`{in)s^ZwAjyTBF}!Djlc-bv$MN>O_J4lNz1xEwZfeLqpRVo<+D)D zlWf~vAPQ^qGDOWINV=E1O(mK%cg_TSJJEf@+^XAwEUZi zPkvYC~)BCKN1kQG{go)X*q17vFjNI>!d+RH&`_pH77U+#R(1v#{k! zCG&qw5nzb@qd;ZGa{QO67hD8`DK_P+`!fQ{CJ|ji?i}7_qBgS8PW{H9n_jK$Ac}Sl zt?|@78MBDq>nxp{TghWseB_u^si@rogy-M?Y|xA9w+8oow{w-Hc>~1s4(~$3Ja!G$ zW92y%vkY{?gAV%cbQQgfi+BIa#Q3key*i;E6Lo6-l2-4Kbl~B2z5WdO^4)P zyb!HmJZ5luAfgfI_&b!NtLacXbwiO&ItSdpI1YKUc?pVon0&KY0U#q8Md$HX?xz}2 z1@x#=S4s>`y2m8Jjq2Q`V$>pBcIrHsm$k^XK~Ov!BXX&|j$>O3FCF)Pk+0Xiy1Jbb zq7*;gN}ItOa2L~YvND4Q6)Szg-PP{BD;dRA?DTyhc+}M>Z&Nk~yE!W1rcUM0kd8$@ zFq`U&u9FV)rSZ^1>Y=uQ!5%9J%ZWdtvH)7qTK#{uqHJHsY;IEQVZRZ`0G^wv9I5zU zQR#mo)eb4n7-X+A>}o(=V+ zGVMb4M4`c?mT$`Ym+QsOk|U>U3xpP8e3h54D0S)ot`9ok^O*21vwfnt>O(*m+4gv4 zpEeoM2t1Ido`p`Xj`%}zQ+}S7ES{8v6S&u`(7JvN5{-CVw6w_?R5uV~M zF)3fZ)@fXfk@fl<{%;Q3F#_5^=C!ZVpl5!lr`jMj^}y@q#!j|FZwP>`ie0ph_^WV= z@Sx@cnSD+8LLrs@&_YCm?PVN0KiY9Ur4R|ii?^rbn8QXk;b^UhJ~Xv?D5NSl{)_+q zgTQWJfrSv(dBQ23zK6}v;{~mMulw3uSAxUY-pdAVs^*)<@E%(+F$n88)Zpm zS#14tu`S>om(HV>aCMNvZ)(JeF5e|X6{)j`+SU&C@X^F1$$D^wcF-;3AKtv#^(lv8 z`7N>g6isCm&9ATEVhb&rxd7qLdd9Q=I)eXwnWP!V^2RQ_QK`u9HYlht40e4 zi4oRI=U_EmT#K7*I#}R}e3`A3U4V5gW8J#TVi{^GK9cRfMvE(5DA;EcdvB6#rt<|q zZPRf%epsw)&~0AN;pHu1>05nXuv)mK zA~<&X<7!Q~N*fW8=-Uznc~PfMMRz;mFAl*;{uap?9~JyoQdp)4^y;iU@V2aB{S1b5 z;2Hcoqv!a*E64Ba41&_d86oeRKp1AMuJwB1H_Bh8zZke~figK2!%@DkLdN;O8H0eQ zyY8^guP)yq!cP0F^r70nAZq0bJ*W{LPFPuS7x{24KVeV=Q9ao-0+6FLp3^IroV~ zeBDaw&dn>_8u-_ryT;BN5sosCQIlH!xLg-6gJ|y7_&ro^2|%tNRg`<*V_ep&d~Ius zgwXd;{Xj@=n;_382N+b;dFmeI8;!IeoXOz=Li^l`Pb5zKrRPo z@mTGU3Oj8%kjZ#I&ux}%g1u`B&XJ0Tc6t8(BHrGw#0cfSu_9DHWbt2`@U1#B^W&#` z2)6iRW}Zky_*hQ*cbOI{mO)+x65;;8Dew>!S~=lKO3k_I+$fHQCM>M3U8`-)3A!+k z-AE5E5Q99`Q^<{eAqtLvFdx1na=V;s!^B!}Bl-aMj>=oeGv~qD`zzc^a3?k*{;per zaeSlr=Y?jkS+`muTjxY}>b56nn#D3zy`9m)#moNX(fvJqhniUHZC+!#gY_)Qq7d&? zR&WwD9%MN_lGnlv8jjh5?Rhc#@TF?Sf$O6-0(5c=1!n${8Wm9VXLmT++ARa=GUuDs z3a5&uQIH^a9rTqySYDlzox}W9llPxxbG$;v-YN1(m?IRC3vj2NvL-!)*Nsj9EnG3` z?d!q+_>uLTs6RVj&RR-qg+r)27YMwEmABYRO7fcJNJlEQiydACgAeWvJxfe2vBhq? zB#{LTof>~aml5JSuTY_{`=%KC#bq^?ceR|BMn2JDPc-xj_jlVptVWs+z#dIc3!~a@ zo)zyjwN*~4a_;nd_84k|C%86rr6^(4Mhu5TuDz2WN*GvBK&VY|iIaha8|97#J-Rv9s z=nSX$ir)i!VQw?7aa(OA4N*xgxmfW&dh-Z}(Cyec`L|BRfn_?+Rv#`|?aGekROeK$ z6nh?O=xc(w@M}^(e(!mpjD4jWxw1`iVK^j4UAqEi% z$t_5y(p>{GbO^YmOS+^)x}}B^>F#cZA%_%(y5FaJpL5TB?m5rB&-~#($TO^OeQUk% zuU@6p%;H_T{!}|)^Z?2;tZ2XRWWctnr^{O@Ef!A9UAtD_?~bfv=RerZ zJGax^Ih=~Yg?NeaNe0`EvYxLl1j9E>0u6dO3@{6B33-Hbga&n2yCM%sAtaEk9$YM8 z`;J0nZrVxj{CO`L%wF8&0TGKTq|{ZJc-F*QeCo5-Q3e}2$`dunN+V;MT-v~ynqgRJ z!1mBI+aOhDTXO+o!MAM&n%1hJ*|pBzE8R_U#m>xIb4QQKou-zA(5dwUa*2JL4pssy zhBx#`eeaTl${gmw*$Bsni=8{FyHS`tJnZY!?)Pu$A`sCk4V)^EiKGW}?I#GI3?Jdp$SHIBa| zk#N2>S-w+H(sY&aRM9*VqR&X$+>fTvbsZHIg1YX2_`vdgw4Mvf!S8ar)lH zeA48Wy7+q0H$y95;h$x5=Zv@}X{()C=>Y&Z8o5ER!;IlpAaWY5JzlG>(^|dWNQ*^E zEvU>_A0DX-w=;XRSZSOTKD1I^o`8v1c%CmarllTNN!j0RC`0C(;6A0Nl)ZXMd!=~wnz4n#aAFos z`u=UwPw8NAXj6+5xgKVtNXFFo5n1C(Y;oDh@&=+S z_)v;bt5Y&s)EOhY;c%pz;uglE&CBNDhOl>nFQ|m)qnu7gy9z4IzabZ2J1f-;u?5+Z zl2*&kv#W~bb(#C@@pJNWmdJ2MTz$DJ4*Gm;8N^~&Xzzk?gl8;m0AYSr-K<)z@x~A| zfO*ueOYziZHnxQuof}*o85G(H^#hD1cUiqT?@hTO!7N4zRK%BI@2g+l%4q>unOFUb@Ti!na&{+xZBb|vA<%m z=eAX@jXaL@lrm+?pLej4bD=Q}`NUJnu+l(tBcF6)@O5svECGEMX--h<=E6uxjKBoDCN$ECdsgLFZbAyghkx8QUmPIL*{Tqm{5Oo_>G^#6)r~hC9kMZPr`7+?O5Y<~{*O=V% zP@09p@C3leTgjCk9?Dpp?eWr6@#w6%y=C_i^Ll}EJd|1*3LKEnen6esimIDgwbD;M9CJL7Ep&caV`04-SMmQm>aO~#A(raslsAKLjh)nPW2qzCW!v(A$48z zraz0}MA_k8dYzgQk;G(@QGKnoJ4m#{d`e^x+Snn*%{L1Wc^1t-$h=Ql(SY<~7~aOA z?@&9Ovhx%%c#9C`>3a@DGaszbN`p0A?)+)Wyan?EP+F5Kue)@67*M75qwfMfBsdUJ zqzDd%h42(s89t$z@!B4l-GB}0!0~A4>+#UE>{gMvk9JoOkHl> z%qn*@oBWhFXpKgP7dN4Rk<}OQ%6vT=>`y`DlM7G^9o?or)sa)$DW@TDv{G71P6DY9 zH9W-mDOZ2p7CgAEbe@YZ%F5LQ@r2*V9qSmk_8o=KS((;S_+3b?DfRFUfMHH*uV`D{U2sl}=9*r}e-8eG7u z%BBe=Gpb@w;Tb*Sa+ZBy*%Cwe*ufiNtDJHm*Fk&L!LhKpgXrhB?fW`<9O>lg70=9J z@h*C4t`L~+0xng2>V8bF!k7|uVT3~LZR4-+0iSMRC|@!_IodJf+JoGR&I0BKE3B03 zDb?ed4fSeEW+uP;!tv_H->zL&y$1(V=Z@E}J?n)EUK|4xO2^>|-MTp`w2IBc+N90? zgOoTUYQ=7&IS{WE({Ck_9|1>nW6k-ns$yh=Cmc(QA^kDSNtWqF&&Yef%DB?j{`4@aR{L z`TNbX#*Pp@)RYLJ{6%Po#!};LTMgslAJy$>qJLh<2FisQ7Ke!ju3uF!E8Y#Fz{~qe z4(3>0`pzq@Sv=sfSNyf;1L0G^5eD83!4%*@+u;1LyBM>ptj&>{grcjx&#I>%2ORnO z;T~<7wlz~-qsDo%Ef;bBYkF}_r}PW`(=LrFw30ORAx<~ooAVZ+1_ri@UuKRzd(?F& z3(Ss-<_S#Nf0+K96?B6Q!mDixEOc_*^%jXW%yC|iMcjD9se3gPhaAb#v6)9UbUy9< zlv=7a}$#M+XYl zzdeHmMG`XKmA+>S&!l~Z(#O306tgR%nb~|Q*emPvPT~ZRk?mGT-rzqY1^a)EW%xyJW~gok6m=0B9Ql!Po@$$@U!KVMN~I;snD(tg{W1w_M_%tt z(6pg$R`q2X^)|cHt|cCC7l_83x%dTC#i3ybEc>cy z*f;L3Ny`OZ3?m!iPdRTv?7Kh#L1&WD!LQk1L*~MMTY3qCrsH+Qs}ehyh~ z7o@g&pA@}suv#|CY9aEwc1@QtQferNUTebU;zQ*fN+VjFCzXIp(@o9k1Ujz)xY?%N zAi?RO6`JujT-UY0cUI?#oq-OkstMiua8OlNNx``JEK4f0yvA%5E7gy0 z#j6l0jIf}}^Sb%da(nJWSlV0Px4v8!zK68Z#>!$xBLZzP{cAo>Zy1$M$3L${U5e8y zeAHqw5iwow8EVGo-mQMB#(Ki}wYxQANyrW}iDiG<>3l9M*80;>W&M3zPe(NCK!)9+ z)~nowv8g`2*ndiohMRQ&i#fyKVnH_P&@?{Orx!`1O=du*F9TOVMs1(D1R9v5rV@l3 z6l~rXYrCR*)1gBSZUZ4tIh$~IxCq;OY1fTDS4lMdd{#7wJ0#UNQ!-OeF!5TBag!$I z8TGTKj&e3<+ClyOXk^Z}%=z8jfEqWak}k7ntEn>H$d57`ofJPBfQS!^;6VcF9KY-< zRT;;WKi7F9SH$J^R(ZcIkAKMZT)Txd4vogNO44fz*(2>R(1B z2*ahnowFAO{c?67%H|Xu7*eh@fh_H7R$Z5;5Jp+^~O63T>9S1=gGP%^_-Z$_OH)(_^_!+^VSnx z8)tpgw%B{x=XiARS-;5p8vkvd;%e(x7v2V9yohX7#^Mm!$N6zVJacq#_^S;|vutwF zJPpM1>*;(L{a(xvWl2S5SFxxh1wB@D+iV%EHHN_mzp9S9G^POV3(J0^-f!`V0^lSPKKm!_xzGda0bL`o#y0Dyp3;KD6{DLT<{3864K>SYs6U<*_*j?3JkS-jepkxxF&U8eo)^peSG^g7Mm z-^Vo#5MEp#T1~JO1bDiu?oTmU)Imq94{1m5@8eUOaeKagao|k6A6AMh8Ubx9vcpq<~SApm;OF92+gFs zkk@!GVUaNrR(#S?s3QUi)U9wX1iYVk{$ffi2sO@%ezgjRp?%z3!}}4vY}%*l zbY4()-HRs4U^2}OHbRX>%n+0A$?;PszB6IhAUle(!)W2YRuN%*DpxVG;)Ry#;%K); z8Yq;^v(MdQ6_T4u$mJR~u(W#JfkF7G%As6MDj_8-Lrq5em8to%;Tvsu{wN!Kv6>;l z^#k(m&wlW2$kDogWmgk(HE()jBrYP-i)}U8?(@uSv!D&LeBIcPKt-5tP@!S??AwC} zMqo8KDX8Gx%HV{gI|0&|y}mvr!?BZH7&^u0Zod^4L0p`ozM3Sw{vxUNnGAE=19egu zJc)~ce??`*Qc`83sZwkCk~`C$)Cr=%(gE zEJ{~|Lm$D#Cn?XZ^D@mcD1`F^&pkHh7!GdxQ(X~B5mt&YhTkxNbrHe2c=k#Xht1?E zi`CAA^<+}7cafrRp?Vee_YVNCNgFQq=A{Of4ED}&PjHV`XM&r?*Ma&GVe5}U(*N#I zsk<(<(SugCXyrlJ?{|r;GQe2*o!xX@U#dGEMPr4X{_BYXuH(O-caTFbt@E!8FiI~T zDI+>ta!sbPL?G)@A@u<25s)2HG-O@TgDK|2jg}vLR2G-fZO^OZ0vyXjeT#UT2Xf=x zx*zgIW1L>5$;CO^i*0M0d=vkDYW@5B!GGUDeh6+}6uE`DxvB-*2=RhN%K!P2MWn{my|QXLHMwLzK^MP`my**);xOAsCFctdo<_a z4Pl{4nvJgB1Sdq;!pdFs%^W3+RZ8uh+Lgs>S<{8i!9pq14OF3RmL}nCeNpbn@LeKZxm(15DOQo&m&U+ zaMaP=b1&A>1~fZAJsXQ8aTblCt1%5VN3*&eRTZBcb-WC(rq9~Kzw60lXC{2~Fa9um z;o=L?HGD_w_!}7kj`Ad~2ru?iz~79{DZ1>>5RW}Xv*p@ZfBX&Z00poTWc~e#Apk~& zyPPOAD1SzxeS-B5zlc=f+~iG(Jby=3H$lpBx9`>QlT0XTfG|AE!}`Yjz{cT%_p+c@ z=H5-w)8EzoDi1sZav>K{T4${83C)7%@X$xAfZl0DxMVUjfk?x#*|`(@K|hlfZeoZG{y5oGSOT_sD}3J3rdr*rukT zUDfkcDTJ0Y5dWeC3jT6~byt`Ql`nyAaVoV@qozx(?`UQ!Y5FN#Cyijgr(3KP#_ewj z-gtjqlrn6t+J}-IN&!F2*b8pDJHEG0uT@G%+0njW|!kax9XsF(pi z`34-&4=cnqY?qtdTU!|{ z&koie#_j;wG6bYl9>6x5J;7?)H8>y*bnbgcTuc#V?)}C9^i%Ci8Z7u#xVJaMG!L(U zOc7Cf*9G5UP-lt;$c=cg-ljCK2M{wmRhlr-{SGwQ6^3m3F_(zz9^cSd+4EE_hy5z` zO=+v+7=qif4}p-q)cg{u)zZ3=C=)o6B*DKlCK^~5NzSjT4jGWNM7zMAuEwsNMa0C& zaSz&GNv;9YbINzDvFC(=CM?b_w={M^ti#^7WpnPHs!x4_+?@E0Vd`|_BK(2M+`y}S zq)%x2bl2USy?Ax8hPCjHm?D1zkVrA}=E>2du$-!R;S>ICb&(@0i}};ZrP)<@?lgI`$)bzKZqJ_~K; zam;H(Ju{w6Z)4$}uC`0#cFvPmbV(|Ed;kF1WAJ8(ehC5Hl+e&v`STrjiJw$ROaK~; ziKVUS=6Bz$bgeMw=WMzw=M7gj9H zFX_5oA^sYn>qDgR`1LkZbrv|sIz4@hI`?^EX}_PvK$+}OZXBhznl;lp8B2PXvCd!3 z)tsu5|70arz~nT!XW;Pr0Ksm33$C)h16#K{azkE>`zhF?0jGeS_H)MAm_f+Zb_@rp zF5|i^ZoVk)JneyXi~|6J*wpsg*bcw(m>NzM&|Bk+OV|M@zba=M5bekXzj$Gzk-ulQ zjeBOpFVhr9J%XO1Du=*843&_12 zMR7FWvB6VXn{N`jTwX@}`*$~ai+gr(7@T$qSiW6ZA^@Up;6Ox?I~GW08lD@N1<CgPa3eJ#kL}RVnABdeC`<6I$pN5UjBJq<5^wVXNH6L_N?!jyWuu2_od&; zYc)Cbm(L!k+}fbg@?Yl~ofwS2S$&N$#131Yc46HvYkytVFYJQ{D^iRb8^mFgI()$N;y+B+Ud3jW<3}T*#U_V|KyUCw++Jgp@iw*(pcKuNR zIHW2E3^3Ww__hO#Ag;|+o|)!~K#Cw3jwk(Q4rGSN*0dfmT3G4a?|1Fz>#q#0U90JL zq=W(SFAj3X0T}92928wk*?SgI`N9g!}{)Ir#pk1x^drX zMqIht1*0_K_n1z!bNq5$08S8whBL`K@H?*mz{4t9Vz8_x?8#Q zN3{W@3MrKp{1uy#S{C`RQPQq3Xo_3HVzr8hDxpp6-;MbH9t1d7o1#68z--jY8Z?-K zG<8TmABhpKrX8-bnbw{^2jkue?o)m+*pG^{4pnL_tJd*J#R~URlO2fzl6^a78nCaV z+r-w>f!4ro;dfD{4U*2UfMK z)nZ6E9i4yclA=!5|(GfscZ6^)7UTlM$RI|bOnPZ6|koQjj ziU%r18Ai{;!`kr%+p?Tyo%Rint=`XHJ<}pX>R*M>e{oOtNe=hWG%Td-xRSG_Y|aNp zIine`_86ysX#p^?j3O9>ozo8h^TQxmOSytAhT=EBIyG1oU#o{XEk`$HiUIV<)YYL8 zMyqK(g2*TPT^3wVdAIv zakbZGbq-jR41+NBaT(rCot`~Y`HBl~QGvM>L?vELUv%!)d0hV7J-2hPCH?c+a;NN& zOo(Qh#g0|mlKQokfWIPl{;Xj>2^{r?2KQP&-YO_M$){mff%Wtgx^WQJEbLyW{3$ zLe{vM_6)(oJ@1R+t5Fw`+fAZCqL#I954iuTyohh=pQJ81lGg8YvQ`LRz1iPvP3-$>6ciu=|d?g@C)54Imzg2F>a ze2iFt3lh8N+dWsJ&8Uby0NM7R2cTL($9*;1Z}3)gkRoH2mBHE6HggK_APEyQK^n=* z4KVA9(0gE-Z63Z9f1zAgDRws5Z<~1^w70R;@h0TBrflk(xs8>lK1%^|d2B4#K!Q+U z$O-VgDUUboIOlEx1Vk^3_a2q^ptJ}69qboXWie;Isr_;9i}7AqBap>6>vvOvPG{=H zSTOm;i1&HBw*tt-Pk+siq{^hn4HHXPGCT6p5X+q}u0xDDeoq)v@y@Pws#k~T{Jksp zy?0HV)m^zGd6>CGETccUC-(4$t;T+3$D!F@e-Jv)aa=5f3_2fH(vVO7>7gC5=iW5l zl(!yt)AMCuH1i1KDcD_=5u6M1y>K0QEmIkOISXXWD)I?J4}bbqpY{cNi5{pA~x{Q*AdtCoOX>c%kfhAZ2f; zeg9sP01*t#KC4(*1(sQxR`1D+J#f3;#69IKE?Ut0 zKyZ4;3HPo@-z!{Z4}A48ryHr5%w@nogy|G6Y9$k|Z@%0pzO)oxjZtOs?U|lCuA+RJ z?6Kcjcwv0wF{lBloT=at!1{}y_*4F{o~|k+;K^~m1P-VRI1u)wBv?1uJGK6a9y4$O z@Q5a~Qsl-rqJ}g|ds7uwiq|xUDXZ>%$fH%7>&Lmt-IqwqRFW7ZP=axm*n_M7y|NM+ z>!I9{HQzUN`(i{9p)plMnGhLQpX;ndbv*j|$4|o{rdnOb8l%YV*4!x~1E4^UWQ|*2 z(atw;=%bIdEA+2gN^UJyV>u0cG5q%u|DW>K%vXW6Bk0#5;pMorVEySiK$=c59Ao*e z5U#J%A|0{Wp2bI!%K+?J?lVZ>HJJ${N*!}iGT-+5NISD|J^Fd8j{C58h@mU3H;`N> z5Ohkv>nh{ExqJ1<2g7n1ez_i=XtXPCKI`v%1NS;3g-5r}+>qZjW}=NVWmI9qYZ7Ks zzrTUSnYZJiP?%hbpYD(tSTOM$#L1+-I`h8T6<}g|{G?$M06}Gzvw51@*IStx_C8iA zZQhRi{xV-)nTK)xN8_!miNAckzz{=PGG_KE4)etm9pL6H0 z&=UgEsQc1M_%zZ8Zou&=cnR9@a@zrxy)cYy30+*|5Nf3WCSvy z^qK&&vkipF6c|o$?tQ~6YKobCQ}sh0DAeNXB!-~*6!oOKa;*9eB+;bE7Rg{1GJ^f= zk!d|ZSFocjg|famaYtJscfD@xpQ=MlcdI+cnv)PhpX7810Q>erIQrc_+u>hU>uD$;>uj4P6_LGMWvd zx_g@@Dv}22H%Mf|-(qMPV>jCL%B7s5CmBc?w5qRAZ5fVX?PNk+L|3(iJM?);8y@Lw zlryKqHixd5N5i-;bdA0TdA&2JiSfzAUMv_X{*3a~eHr+v)RMu~Fob2*gVf+H+pnjz zT>R(x`o5jn>$fMn2TX+GX~hojYFMB!zDJ1c&3`)bFUr~S(C?`(|9L}H`yiVn>vt(O zPLcCngMM#)XMqcV_R?2yJK4eNu>uuegZbIe2uoP?pdUWL*k2R&zOCrO#-rQQpXa6^ z?6=R8_>4c1o6Wv?1ZgNd;HAHztQvrS?RIXWWZz^JZSrdf=@obh2n2LK_8){W6PR&s z%LR;sY7TUcV1vc<9=9t(}f(_n_9bEAPwxD~sbVnN12MMxlR`Q)hDIC=B+-@VE>MsOBrQ z@>w#K#*yE%Qk#hNZi!T$8uq~)M=>{a7HYOIcQYgT+RH=>C%hQ-P3iTzwW<{hBTqeeW`UMCzq^$ClaM=}dMHHYHwjgit>BeLyd{_E!>%iDisS zLaNDjz~%#U-hl2J=LewIo0i^F(c<=BT9WTFPECMHzH05o^9Ro}+rbaSjB)o2_#T*X90 zk5}g}ad6EAN4aIFw!fiMdK~|<&osLuRz&C^>>)ELo&DbwTj}8hbEIsH_13KN;=>j8 zI{#(?V8-l(9-^kqKV&FNzvLYIJW`Hqj&#}=I)2}@KE!{B_KYNR(RlttmnWltdx7J3@Y=(0b6}&r>?87Zvy#$EQR?pV zS(m9@G7+W0pWjn#?Nl^&%*?vb@Bg6;j;SCk&=#%5#|}Lq_9A>{Fr|Xp8@MB}e)-Oe z7>IIxi_XOY#-G1%B@NI`CHY(GH{?CC-;fTRN!oT(qPh{zAd`xhEJ|8}xCFy}lUgZL z23mutCX=<;_V?1sX0wtB8Iuz1`cDh$M;*8w_y!Yg-ymMXJRdzZH14t@CgBli1cWaW z11FKoe-Ren8+pU9$t9CJ&~Dl{EM1l<30G_w$W?SC)Ke9#h-En%$vb)h#C>SfG6hib zIKMDhRO3z@$vgCwvfYr9segouynE4Nz>u8YN=V`^mYmP}sJOm)m~@q$*#y%74Shy~ znjzx0=XUJHBjcpv|6R(`=w&+`aJ3MmV)G+uE>Xr9sKm*bZ_eF8Ep7@?w0sJMMzu(JKS%I92?zS0+FxeN!JTv5+#+PKAB zDE1@y=7}PfWZOFJnzGX{PWle6GO2}9d&s4`#S=N3CyxAQpsq}0uE@$Ci`-st$ETC9 zP%5fb`|MU4ia4RfNxH4-3=GnSjpMTVhNbu^)SwYB4bC6pe0Uj@_$At>bJ}(|vee_S5cnRmAA{P+Mv5NcI>BUA5#Oe(Ut`8{yXsI{4zae+j@IB?M|tN! zF~)af^$20lfgU(0tao{zzzywCf(o;UKMz{zkEYdJjhU^`dZNFGF+YF%MDAk6{`ORp zB`G8F?#>HDQhMfn9d4_XnM*D>`$C}Jr|)W$|g6jiNKLC7>v)fwhQPrC#r6Z z??|q-o_#7qv!0mti6$fGUTE?u5G+ni7mc`l>4UCOE=)}gC!c|tN^TdM2J>r3ojca# zJ79+=AZ*Bh4F7g|iAyK50~C|?!;yWIY-gIur`s-)WvNXs41#?^tp|2PvIAPM-jL9= zgJe{N8Pet@+8oy4vTQS0*3@}^4Ct}MKniLk_X)Pn4~?7y7VTk@t42wJBHuWsP- z>&BPWcewyi!XA7eFYcAu&txk|GO3dis|Q{1R1FOuS~6#3lU9#OgOu{jGV>s{=!p(i zYeebfsOzOXH~rd5QP!|@7fJDipZ-4#*j6tkFc=Nl{bdl;^%WNHq=`6fKhF=Zd_Gzv zgrqfG{8qiP7C)Wy=GFt(Qs!`}BBZ1t6+I4)SzHg;-`fo4x0U7WPcXy`cfUmuL2!Yn z*yo^q!UpoDtQr3rQnl$7m8Rwd`{O6`m*2C=YCCQGEm4fkK`DrJ!;z+sKPXZ$AF4)k zuU#V%?`vr5@c#AQdwheKw?h6ehv=l4ECRY@FNkvr#~Bi5ZiDEH!ZIK65-lk%vqZ#0 zdq-RRpG1G#(F{z#Ft(}usL4&lC)~P6btiJw-qSGel@|o|lFOv;?RA zovKO)#mQLsa1-BcjPFNKLfxDW`DZg1N1FoRF{}Z5*KXKOV=Gf}XRsnPoLX}g7%UN4 zPAd|t`tw>*P;2lMKZ^IrN=bH=Hd(26uwRS>CLwLSED1Z`iK0R5*Uu{HtTSgT6|34? z*_fRaYM8E+Mc?tblRmwCdH4_&wkJQz7jBi>HIpp2SA`nS!hm4A^GF&?4Zqf zvv&=FQ;%=d(w*jfu=>hXlM`zi6GR};io5d+X0QYOWo@|E+Cl%HWNlXTe@oWtnB9#R z{)fa&7WHt`4FEVy8-T-Hq|sbi6Fq^t_E(SPeaGPb%XS%*_z=1=U|fbu71!OOdAQ_p zaU}?HlPS9QdfQutekBuMJVcF&y4qdwyFIWkS&oQ=WK!omnj9OE?Epa8sktSycE)~2 zyWLPZB3xUbbFQ`!uM&II3Sl9HCL>6Z57!4x-GEt9&s&?>WQ=}X>-+f8r>P?(-Iv*p zcmjxl=L6vf#rKdZ=#|R_)Cs+82j~%j4a>Mf?1iuFce`HEov*F?@T~-c7A}d+i||}7 z6$01>b!UWo-&Z4!@ojnE=v|ihqDNnw@#yQ>3`OLtoJ`!0`Sj$@bP1=c%fP-S$;LO< zqxj?voEp*?C(P-+HT5=knUoXJVD9AXr+ksvs+I_-urJl54ISCAFOT`)G^A^b73^&2Cfjx zJH7RT#LR?qUGig$_FoH4?kitwk(dK;TX?wpOeXZZA6;Lgk1t+tB)_b6;q$%_x%{*~jT~3tuZskxZ2H7S%|E74dDYWH=#g%sIjW z``JdebI*p*O1$94lNZeC9d4AW%^nt$A3qigbr&Y_vNQYebs7EW=(t`Rl0MLW$Sx6S zVd5CvJlQjzK_GZS-#rC4`JGVW;hfAB! zjJ{4UJ6i`rZ$4>xl~-M7ES)-&meO|^3gbZEnbDrE$>(E!-*;?ZS`Mue=1(!0k`zeV z{4w@NdDglO{~MNHDLLQw`&u3D?s@uzk#wJBW5OsrWqV~Xyy8l`sbO^hMGp*b9eyIO zMBd)!O>dpmFpP_RrbH!k1j4mRw==IW9#d&RyxwF+YEsoQjkOHRuWBzSn`ecSiQQ7e zkp@-xNbgmPlTM0GRg$*xmA9{ao0|SQ0mJAZ z8B~$~5DG!cTF9KVAxn+&-)uxG8wkWLQQhDR`5ymMIt^nJ3#w0)0RO|ME9BWaDCrGz z7yb2aJ`ti-4$G@R6II3o08PV!(wNC0(R%&ptXdh zw%BXT7`g7#<7cL9hicO1Ow3_*!YApSGIKG_^-r_Kr`GQqI?CqBxcJ2itx79d(g#n| zyFxqOztJ0J@C|AkAC$TO+zV+&j&x~qd>KB#s!{woMc#0*xyG=o8@e3;5&!=1s1-7I zD!W;IP3NcbN4@cWoB8Y>&$jEOl3ho-koLroN6ahz?e?b>W1^TQN2@okT~0zJ11$3- z2@S~|8;i!Ma9q}2ba{x@nAo;FL$avje|z3t>t=lDECT~bNZ_@-%M}58lNMLzGIpRH zc9Y%uwU(Z!zcLhsVJrTis{9ShCHdLAwKc&}j1Xdn=-R*^Cq|9rmuTk>vB|3`b-6_G zrLa4T8f7PQDU@hC(8r0)PF!U6EXcZ{@}F zhBxsWiKxEU+nKc$%9ySkSDeU|Hd<-ZZw>fD9ee+|T)}dbzhdsEYj8|D>;4J{dZe4A zF%}eT_K=7reG~jgqjG8JD@t2Y`B`K5STApW_!t%0NW^gd#e;m7>4XX9Tl~4Nyq&-M zz#1^+RaceD9yt!7?aFx`?az_?Rk)N`iG+Z?liAVlUntaL#w+@ zSsl){Ujmvjjcz+GHJfo?(vHFUQu$qX$qp;Xktax?d1@Iu8`uxdkFn1Ydbdf{?gzfH1riJ;* zk{4M`marf^)!enD2j%r#i-~Ov8>VJ34xg#r;+5!kQFM0Ds z{*mv1Fq#x)2S0O-TV?%6+xJcA#hTA6t^Qhe6cZg9W;pZyo!z|+tJor?gV5sai@l^4 z&q!qfKt4qi+}dki;{c*oPt5yJn7g0(YURoY`C1#hh!nV`;yUyLxm%mplDE2e4m0va zRf`Z^>!8%oX&+aTSxt-`(6x>;QH1_01Ou1Q`YvRp=Bt?W#(`g9Rlh=@!ofF*SB2$c zj;XoH{on9l4gUA$GA8fRt1sbbOcyIu3v~=;u9?1xN8fKMdEF%2O=(z7cb~1>iTmnE zcdOyi`@#3s+;vRRo?r_UIG zsn*+P4;SZ$&u8Z^^v-OFu?px<)+Hmwe9hTS`p0E)m4!hmGI0Z1lIa&!`lI@gw!#9|E_9l#E-3J&7uXAACn|gvQXam#mEb$cv`sM{Q(z_2MZbfh84mJ4Cd}2*hV` z$M@ZbcgLF~@9T;*kCG{-X?XBBkr(X~?KTa_`sU8~pqlVh&He)p1SNr*hWT1e{4=zO z`RjXw&4BT@cKBH_qW9S0c|UAkW&zK79(kd=d6Tatvfi6PLnMP$lFw|uI=8^+mahE? z88D*wIOX2FG!&f`tu5xZs2BvG{GK)yZE;hZi9sFG&!xre6uQE9ql}Gnie<BZKSG;7vO1k!+*Bu+>v}L6 zCVWz0CT3N(y>kRKQ^$KR48*(v>WF_92s^V$O0VAwwX-%*tTsFc$6CVyGl{+1hHBar zOyC3@zO6{VFLeS=+u2_BL#AGWg!Y~*;7XZ-VK;-hEZz%VN#U861(+5|8!!!v7s3xkz)9;ZZfW}yyrW6x^ z1?4a{hyF^GO#YKI0k$r~pkrzfNn+QU)OecIn4k~9`PEe`kZo|llFk{K$W{s3LfKs# z@?yXP7^G}7Z@=fD-`s^E>$?bFzbHF-Ep~p5oN-y=A=9f#do}eCBSzQwkJpzyj%%>y z9Eux00d{S55esm>50B#5XH7ePsOm8D??=fPAXM7aQrtYLDxES-q>HRxRa?3X)j~Kc zBCV?w-7R%q1WAAUj!bDT2Bd1}rsF}@{ixP{;4m*Zr>ra=%j1h}=>4N7`RC;sQ9XwDPOKAYfr`v?UBb*7v9p1*eeMVE9i6u%)QXr{m<3OD}aeq zCUKZ&0;tj<+kQmM93Y043)+i>7e7!;eGQ^71t%s6qOL%W^APB99EI27jdC;^Hq^E^ z*TDH*Ibh^tG*2qzei#h9{!>PenF>CJ{H=gjD{_oQ1D3eu-|D1VAi_~31xOI>y}J9A zC<0$4H_gi}X-c6B&Bz{yP)5TDLb zsX7}P_1hDX={_ZET7+1im?{1266gZB#Ww?W;f#gGXa7N(lm)*G|3}CSVv|*02t=0^ zD3;1x2TjRd51Q%lt3C(m!!UA!ekB;-Rl|HhUl;r?;j|MfL?je) zpZW&5`3&G6v0S5Lwb3baq&nu&&%VI=UyrsHy~Pi0cRIpA9lI|g|^zG6BZwn>vv2aqY&`foG*`O@8Y-G&XzKosM zBz8dsidpgVk)w6p=wIBwF6<#0P1mm-6m_C@PB$MKqqWK=!W0ctG00t7w4}A1t&sGz zxw)$?1GNX`!D^;(6&Ye~RODv8e)T}}FccozIn;038KQgM88V}9K6XVXpkd<1AzqBn0xtB7+TdNjL}BxMHv4g6hPngEaM&vc5Fq-0H*@@1G4*mEGi2pmX?72u);ss-dwAs* zW?~-px744n80r+ig}2D!=gx*Hq()0WtuI}R0l}+bP+YgR1yKh_)m6X72@}mdh>#>r z0Q-4CMB^{_MnTXDZwL3I9L`y1CQ1))`gtcG)W8gb+?~M21Mit-hVdLNJy6u(`zd_| zn5pFsxrOQA*QI@AbL&U^+M~Z}Kp`gXjZ8XFU2tzMkp!}!xC`2-Cj4kbE6E!qiFa3S z3V9{!eNzNr=pF^5_jyiC#V_B6d`Sk1|9kHwQaDJ7?r(46Yb^dp zDu4nC5Cu!y8q3)B0(wQM{?ojY5x}Ls;3-12*uqRd0(~b9I+6c5J0Ec_K->4OLg!CH zLD%zNI%IF}Jiqo};?`d)G$#O7T}?oqa~Cw>+6@PX>1cb{JZ-LE2j0(DkZeV*vaptq zfv?Fo={ZY)wq7ScIQNLck4Ckmpr6Z?{JSQ_ zJFYGQZhPO)5QT@sP53Yszc3d*FP9V3y|fi*JSzsMWsFJsDrtEM@z_`aKV_g$E}$~@ z2nbT{{gdxtr~(HrdNNAt^JLh9itDHcP5C)3tQjy=IPs0D=KdC8{a;kI|6KNixO67{ zBBbeT3a_NOwvaI~d8!+8fPjMdMA!M67{49hK-_N1=lNJF{e{IolbW=%#+W@ilZEs$ z8<418Z%HenG!}Wt+Z+;k>+c<0#v40;g2Fh7(j=F?85kLx8CsdLFkUxzH07JBzqm@r zrg>U!oLZS7%W;7{M^wX1?5V3&*3a{9?rM!iGK4p=a`G_fV|Kk?Gd7o=hb*iz}Tc)rR zBD>I=0m!i%oX(qw0In~u=n=OWVwJgal`16yq?8PqL2J#Q&qN$o1S-)PLfVfNr8=m8 z+Vudc?71qm?s+C@kb)WbLIy)@EH0mNx;d_1}vTXX}4 zA{2ES74BMWQ$OkGyjc&JO~N*H6<^!b7}wz?Mi!D#t(uyAKX4m1c^#T0Nna!ED9Zax zSy|<0x0FO~LD+|!xe8+;G6Sk7-nc-j!X6G}e%+TvkcRp+(nym`(Qu#SC~sLe>KBkP zfy4~nq$A@cgbflDZP1ENt>=*5+iw}K5k4u4b@F&e<=iTGte(5^C5(AHl-u|N3$OPJ z%tYGFb=E}=srim(S`xBx+5ItGJJg}7K0NeGqkx<_#!(L)`y%pyjJU{>xjfWC7ldC5i7iTW9I0*H(QUlcfDSCl*~1L zgC(5xp5m+_X<&T)y)vo=1t?b=okZpKboDuZT4XKAufb~HUDEp}jsJXg(GDnvOkG=5 zlQn`}x7Zf~1$g@Ade)k?dhIUh2zrZ`;lhHb-uAbga!a~j{L6Qr=xv#CiaS}%(dm%_ zhX#G^$pmpj)4lSJK{pG=nd^#zmcT zu5@(d{JQ=~$jlr@78@Xz5}bKu>~LMZO>$cb|M`z`nUR)IBA%gnEI z^Xo0#ecOrGiW&4Onsr~gEGSHW3!O!0km~ITaCUJ|b&y*Bf%waP`b>Jsu{+qoUmlN+7YYbP&kf}9d@XV zPX6hZO&x68Db$g%%vWFy_G7$Wc^t?39{?R}FAEKE`W^e9^}ElPW0H^P<~G901ngeY zo&r<8A5i`(AE;Xx}1aexXB6u=a7C7N&JtLL7>8O_xxn z!rm}nZ@RH+uRhy@^M)!M6VOf=LU+E-C8JE`ytudeJX%8VWc@aDtJ!uXD-`Y!Caxd( z>N@gvoxNL1F1L_Y-y?~;qHYx@?9r@j<-yo6U1k>tqx3drw|sx;q<47O!GMF*0Ocv_ zhD?ZiK82R+ODe~KAu`e0g-wS0!Aghff;RYOyKZSUWStp!u#E)@ZMRhzTya8*cCW{| z!?Qzrs)#f(q@-l{x8CKSGp-XiH9IQKf{@3PwS}LDw_m9r)^FVDtRS zbdVCNih?L4B1*5)dx>-iO+loIbO=F3K%{ppvm6rTGv{0%{dpMir5+|Ud>_#4#ITEL`>a!6!DGDj}G$#?vrOL=RxEDNqNTj zmA`FR@RUy&qd(uhOK{tq$M?bn`&FKj7902Sz=rAXpzC@E#|-TVk7rhKLNd{ z?gZTD@WwMC9_8v~+8%;8w?bd+eHhNF)4Bc;dr3FqDP}uu53^7K(F#PIFy)Jmk+qYT#Grx&H>-r1PihZJ?1dckR8=>*IZ##VkQ={x&-)=zSW z1@I2B>K1%auENc3|7>h>KA~$CVUW5^9gZx!)j{p?VyjcN-`D+MHWi);Q}-8GjTS8D!FM+n7M}G3VwZzcj8`k2b+YQ zx%cKh6oVC=3@Un@#h6C2Z-4eeWwcnN{8ACM^nhEnCil;-^LhUM62rA%DruYQyqUsOnyTchv8e4vE zmmWH6VviDn;FtvQ)$34pwPj z+@$j`QAeZ#FgdX*cvJ0ZA~4n=+jt0RXq)Q2>9B6>kOK|lW;Ari@6(PFLsbVpj>)i# z-x|HU*_c(7v{^H^y~Q;8NjWwiECL0px^7y2Rq$4KHbm?QpO**TH&8U$0m_TP(vABRykb{ulOp<<-I@%w zk!Yt?J2cgyn74pKATjG8R(Fn@IOUP4TOwvhsr-j|q5vfgt;R=qr#!eY+RrD5!b4gx znxY_1vsZdcWND3c0EL7C7-{>E_J#WxwwPCe=n5<@9IAZI(|7JYlebdul_R>G;N4R& z=bt2iW}fe%IHYk2>tylHLtE>JJ+K(1*s)1*^u_4CAK@@5baU<#siW`RykY#Eyxi5H zA7J%Gm`Tl7TSw1CL(J^*US0|oRfU9|+9(XMKLaXZt>|7;Ot}4Nt=&rwJ=Jlns(pbj zcIs6{Cxh*sQn^%Q_S^Km2)4q6FZEry46$E~ScGY!3oBQ?4;{kC`4U06cNhp-O}~g>iVA$0>H7 z1Uo1Q7O))y0TQADz5D$s0!6<>H1s%!DD5Bja!G%iHBRQsj-#`wcV!Qi`IQ;IW#f#`qCZitvkTOc&ms44RRb z9-LwscmAb}>YS?t&>Rbi>^I4rg;#&#WvNcBXCc+s~rW>GYD+LQi;{r~KZgFssJE`W!5l`LQcMVrk>6+YykEbE?ATG#kb~+oo z5SC*eb6}^WC(GV9Q{Tt;AVu7d!JzD^yaij|MgS2LCjjU^K@A7b&_-QKD4?z=K3-G1 zkda`A@4_sH17O?yBap;h-I4+tWH5aAVJg`==C?$LS)(@h%?CiGV_6ph|6BYTZW-`f zaQ$0#L5cAO7X2LKn)gRiN1J|9L|*v1c)I%8&7o=9m^l(|5r?Tu<4Qx^Apl+;kZ0c;YxxFovx1 zmNT6IeV3+0D!Y)>j7MHydS>(1&3^Q=v^Ul>tIR{!-H{8BQc>?f+p)RUl|HUm>)Rqz z*JI(!GgVJimm}vV@qUSu6(=hNW0yHQFwLj23DW@WCS8EB|Kw;SX*kj`>jvD!k!g!k zcmDnvwmOQf-5E-s?wR?C&5iFqYi;W*d_n(o*pYEyO{@hABxfz$fdOe}v#prtR#ZY`=s#_VB0L*;c3XhBdnx4TMV|*X3iAc84nu^h$ z1cu=8P*M(?l;-eOgNXHl8({y{LB*7Ktpu=58{A@&%;l-Smqp=di&Ne9hU&y+PlaYQ z>}zI1*OOkyPZeg;9KoNwe%a{+eI<^b-evg?jR*}*(wpnp$M?QOZaUaY`O|98-O#;} zzk1S2&-5mN|IHki0pL-{RUoOjMRGBfVXTxSvr3iU<#}T-yaCQ|<|9C^^OgfHaNX$= zksZx|CMu`19HuFgpxvF90hx#+nNDSY}Pi<#Skl4r?$EeQDmphO5s79Hw9}>ID zCTjENj;dgtROqVKi}az5%TJ|JO=hlprW11CNnnZ9DqW5a6;_xDbLBA64%s`Ri=)7| zcg)HCJlN#5V67x#Oqi5`fL` zT@>+0QCZ}9Uts(Dy49~w=PxI5RMs0k4S&~%Q&FRRU01uwxWAovfL3fQ1=M}gryeO( zxB|JnS5_F%_z@K;i;)LCEUf(QyofHJx*4D4bB#lQ>fF&2QYkOOJb{;yo7t5Et$R&s z0fsnMR~TI7KOVbf^e8DIDAAZmI@=m{kW~J&kwELTVw~16^jIbR)~K#w*1%Pk=XS8Z zH^;i!xhP1iqc63n>5ElJ*yYp=SXkq=sc*+sW|bK+EuiL0A#>d?M`qI>s@g?n+uJs~ zm9PC!{xyD9*UUn_?5jaS$^~}L4xI3J@T(CBJ@4MQK_Yvi>zJlcy|L^CfR|lXLO1_u z*D`s7z&`az8c;b6DP0E9eho(eBrcKqjTFyD`8)0Y#|{AC+>$1O4!H^_Y^4W}D(8$$ zP{ausY8lTYmf`Pgm+jgg&=eN3>TR7mar~CNUgflY-2;2;H@A!eTd83)6dhwHG!T#D}a$0xI@$VPNX{@Kt5(z$i} z<2)k0n&jnxGxk9Hn^+)Hy1eBJ&?FL`#cD!nPD$UR6D{aCFHJ~6G>u25k3@pR_Qtwo z1%B|@I>)5Eo(p(O=V^7Ce)(xM91@#G)cUdTJcQjgmyH?j_+H^&E@x@x$AajJ3>jy@ zgkVValxQnjHn*Z&N7c%Nwz=?a=)x7-SsLA<$Udl~u!@Q*I%TZ-=QC{z#)0C&u*t9M zv^abS>V~6=9?R;z@)JX6H>(z@RF_^X4<30`yr()7CR|OBSoKjJy%zITb-wT9#$=*v zTi@Lc0=;xPU%ks7^ksd!(5nW-@$2)f~NJs(fT;n8E zM5VBM^=ku*BlD#2G;TA0F@S{HIM5wM@y1j1d;BZXU+k3)VLZM3@{0kB(vNyC7YFGU zFL%9DxMN>6jZ%m|c1)x#typ>Pl^gHfRO>bi?d5&<^OiS7wG<_>kr-dWfsWL%yXF2p zYZ_UBNKW#;+Qz88?n2h^W)3c&w*i&=3(d6 z>_0p|!}9?*M8)oK`-ALh1n@#6T+T!lP5C`d&^gq)_>H<6{q}wEl*xRN6)zB*yZtAh zpqMUjF)7k=^iM{F!e$btmj!(|xhaaGCRw5E?8Q4oNxmb{Ea6z4*E*2WLt-fQf+%Gd zoY|nJMz=XT+9YMadk^lBW+l~Tc0iFWF%+XbHu}5V012aT=uq(y(qqzAVAWROhE_pO z3UzkTd_j9pQ5-&#)^xs=LzH4^I5Hy|Gc#-8`abky7{~h(z(t|hfPI(wS)c*n)^ze@u~IlrE?7&z;K|2BByWgg2pR7%AVMbA0PbxvXibU+b=tsX7lin**7gSBCK zS@U*#=Z(Wjb4!*u8~Pc89tSoq;i4`qMXo6fJ7`U|dbqwhe|u~-;h4=l^+Wu0{2)EJ zq@?a9Plu%fBe{b2NqExm8>dW#*z?a{f$a)=U2U=&Wl_R=OI=jj%EQ%{y-VFTR~Su!{9;D;M@&LeL~m&W;6 z?o4RYM-@EwW$DtD(DATiU4oiF^f zFs_}wbAnExlYiD3HvlalMg~NYC4%5dJ~uNB{evUqLk_&$FuuPzqnQTYByM2U*YqHZ z9>@8eP_&8as5u#c!1pIX2PIaKW`;hsj@_MEqEys1G^yALH5+jcnd*mXZyYnO>4F;V zim6jY+9@jHH}koT!X`{MHeb1o3&)}atsX2;$kZFx3Vi9MN+AA73tdV4ldfU8&Q^Fa z+jl)bF6a%pC)LY)1nDXdk!tjPnfB{y)iB*+pw$uHxyrYK>n(S3aDREWVr+&o$fjj) zP4nzY*XhAgwTHuVJ$5&WL{;035#=_g`f{I)T^(DIJf3vJhd0pPBmRLd=gc|F+|b80 z1I<)bdFbg=_k?@d)y^5SLTb4O@fkGo~bjHPdbEF;J>x#H7SZ%5oY4s)o~3s`o>PS0B@_d zhs9ab+P;WdfOXCdm@~EID&s%yrW`TRsJ{C7ITnY~nfbCF#;|=n9XJ#reog}tambq6lhF41CDI29TiTR>X7%k z3h-h(RbxDEG7sTC%PLJ3-CEz!s;)s7(uO7=+_UfN&MjY88fF<;_EThZ2#%vHzO&-& z)*M-lvI@(zxIT8_#t>leYOR?XXK(CuZFTm0uw^~QM=0qAP&oO=vF?}`r|D7!9e0oY zT8JW66nB*z42523>?P}y{Mxj7ht`%MjCyf9x0+bZ78vx(%2^3=^VmCxI5*CsClG*R zWiYo3jwS^< ztw|)6EGz$(&t@8Vmpi{VSU9MR`&McAwwLy#jZd&~jPMjUN{LoepWSulOSsYnBKz_Q zC!S*N)E(I;5Co3*^Ee-T{$@>pw(W5NRs&seyjN`!Sls3mWRmJEbQ5q&A1#o=M0_p3 znmhg7>XAoI(4NB4%a=r9pww(tob1A<^7ETf;z{TfPy#`MV78E&q<&&1NFwHg;kOB( zbk}Ba;G^R3YBJ3UW3@pv7a;C`Jr?V@)S*%nI;%o;(f;_23oPc@hMKQJ*!p9iwuPYC zsle;Oh&sBNnOWNft@O-~_6K?I!Z>;dDprB^hE3Xy(OTu8G;3Ub~~@=veky7KV;_dRxbqFBKG;(`dw! z;S^PqMQn97er$RFhDQR5XRCVGJ!JDKU`oar+I~V76#*Dq(Wg6fSo^g?5~{!5m1a}Y zTZZp=Jxc>)bA#0P7glOsZ6OubyJzhpp%A$(KOUDkf`ki}QR_w3#6$Yu?hk-k+r zTFr_U!i3ba=~pD2zhUdO1mHZyv(hQ&>=w(X>jpKAlpV{B@M9yCq4ZI2hhuYbFV-oB zwwc~JArBH|r1#vlT20@Dehniml*8R2BBXPlV;FAQF+s+W7IbVo%lLT^pxUu24O?eU z1k^bL-vcXo_98NgIm6$TE^!MBT6V%~m05A*bQjC<&S@4RyM-qsg>bNw4{$?ZAsI<5-~zbF7Q zqwzB<-&z`J!fNJ6(GjF}SFJ**-~w+~)0l#!BQH9&pe$vN#-1Fku#Gxi5_Yrvc(dj7 zm9j<)+Y?Y(%YaaORBTN+QQBAK$jksUuy>9?!HZ}ahFOH=Tti`RQT#x>(mHLg6U zWYkX0v)ZMyz<-)>5WZym2(~_~IvBNhd2~fj+`|2J??7cA-_DnMaYqGq^sd1oyX}qS zq1whL4V}T0w~%GtRL|HAei0RMlW?82r^hIqFwhd<7GNJae3yrO98Z54V24jr{QN@b zTEjN{dar9upC71x$l~n$(phgV<5kL^Ji9t7)CcYqg=mgxZu+;n*1=@z)HoW3*X0@n+1BFOo(>8I~F_=@m*Ad z^{zY=rF1-rZRdU$z<@Elay}J0@r^|ukTfo@8|m80a){bne_8jYVegZDA)4ev{U+rO zTlu}=h8f?&^8hNL>Nx+#@Y37bxvMR2IK|Y$uc@7DvfWd7H!6AhAB1O?xrn)vRxls@MSd%4GKT8+HmGfW9NFipZ7Ny1G$dY?X9W^Wy?T$$75(>wuT3S&WHU_Kr1DD zzfm}LHQ8Q|$|p{==u}a6hp`3phD4We-to&gy7p@ze;Hr@YK~LdQgICBVlRB|OlwI` zdq|q)ZqGANitDCT%uqMWepArTxAP9;@9i$V-j#E_2C#{;Sy%8>?lq>C&Ca+f z;A6B>@_27imE(d`g9+)uM^0z{9iwjO)j5P5XZxj$D-HsY*zW;e8s}b1hK~JgiqT{9 zJUIVgz4@8)oR)5q0q)+ZprDL#Y7@hjrg*k4OL)1}EVb7d?)HaF59%S=_Mla^@=L5h z69cl@D~#R8J?Wph&-1=W-REj#<-j7lJ77`OzIewqi1duMBoQDfBH^dFHh-)YEtk=x z_*~0qrG5pmMIRe!(-}W57Tt~+ul`v)@i=JA`2*MV3+EqY%mU3nMJVr-6bW80L2mA3 zbz_1fn9=B&o(@CECWiULo&2s)*D5}e^TcpKTrLEOXu7lO(nvqqEoSF zT3vp543kus=f*P=f866-c~Tk9g~l{G1}Ttj3U)$rzy94S-m>+E<%xdvEZszZ9u zo}I1d6y~*)9B;afT%=?pZ~3(59WzeN%t8;-j52*qOy%PZAG&S+XUaqclRS3+V>afW zmH3W+zTdgg>~qrJDl%B1Mw9-helbgo(&-R!xnZg|U8VD%PT+|X!k}|kPx2}97C^j} zAk9}mgIu)E)LbcK^~4)T&VQ>ub$9-&Q6oQy3>TAD+Y^$^Kmw zjqUv7+O+$oznD>NiT7+>imVkIpW8S;yrE>7tR&ZX1e=zAFOAV86gu@Rtyo=PHjNFS z&8><~k-i$dm@}mfwfl zt&SCR&5M?@oHK@EYzNf^kL`mUZ+F#3rOW;b!koH6*?ZlPxJY$JnYM>fi>XcA*^E^C z@}U{GqxEsky+#MP^faP1?rQI+OsWHicNMd?981<7j*pFl&?*U=Z(CT#g1DpVkG!w7i=Qf_oA?FaD%+6mpF6~u z>eq1!w$*E2l_WW?ARRc4;yRIlOTN`zO;oMV*>%wffNse)c>84I=p|w0h~u=fcNJO$ zeqOcBWvDZyxQ2%>^W}Zb5>iO9K+_16>RZwICt6dTwRn?3IM*k7gM&p@T%m1s^&GG3 zskiRB)R|tjP!lO0W*0+R^K!KU%#-V?Y!Wlq<MiaLTuV? zZd$EpG1c&Yc||!){Pv1k`0DuMqLf7>^MFnwN*62grde5<(Ck(@_G#xbqifC|+{+CS z+L_VhDG&5Dk!^5Z_|hphVRV37L`B%O(O3K$*P3$HR{`PGz-@1bb4XVV+LhH7E%Xq~ ztEen6{ocU(#&wSUWmy&lZCAh+35QCPWx=TJEGLDXk@5>ZXL($2>ZgyreP-slKDBs# zvhO-0($m8KIBjJHH&d9&B$b8qaX#4_8HHYgt5-ks3$?ttTUzE+uAR!^K`oD)d5gOp zn5IO!-h%6pLYZv}91XXrGELg(lJ;&YVf0+bEgTYw-RF4<-zb;H4k2fU4#7MJNm#-M zT0os9Un8txyfv9%-=>_Er&*tzSS{NBCA&o9(6V}_n$^t4nHw;Q{BoA7;CE$@`5*qM zDg|C96Ff4arx7b;kN^9X9}zWPtOb0692 zy9tKIn>o`OE$aZ1k**lho+cM^^YuI?{wrAC;;Hzk^9~P17ZggQp=sw9Jst=H69i4<_;|?|lM` z$$7}CGM_r$euwtfyyZI^+^cUM9oGc#Bzh>F%{>m%p%v zZDuVWR%8oF7g^)9%Bw#Se8lN9@L$fJu`v=khkA3J1#~)9XXY+fNn3=xwB!4XxJ|LN z-GgE$9{EpNj~N#qd|`0CwhN9q5yNjpqtHhcqi^YVbXUKLNx1sBph`89JQKbAy(|0M zk2h4s7D|EJd{xOjQa4{Aba`^IaVg;Y2bS1K^$oDG1k)g)HM|cFnXdh*4)2fdHJBSr z(8FJZ(?holNOb@QfEymWUf_VTHwzDk`xncvteaA6c&mKAY)W0pd)oG_k?q1HXu>X& zXje%!H(e%9FhNvtsQXTnYb+Hvs(qsUX~hlg;@{n70-La1x+EyYy`nC{UsPj1_N=m1 zXc2|npe-iPJ>$erL^>DOFGH5w@MhYt+j^o^6j0{P**js!vWF(>J}sU4T7@gWWrFjI zCZFf6aA}3_<9$uock^B1d^3?#eATW6RGIfV+fNrzjnYt;TMIBsr7lz4Q|l?i{OQLP zl_LpGZ`FC1p(hgcR}zg%jYYG6ZVG-_^Iwl9gvE6W_1lNnBxs<5tS_F4dfaz{(4+en z>x0*4R#y!=hJTQ+qG5i4_&P>xVsj#bz9oH-MAVw`*AId#(d1^X6^!UWUE~IQ)2bRT zc?7?^XyqcJ-y*>~ey`WST>2Hwgt6o+F1Xy9*5vWYAz}yc5f4A9JLu9_+{~C}>{j-` zKIjs2BeM7LEy^?#)S#xKva_*OIIG=O0UzNE&DoN-|CoLBV)keht_-{MnF zVsR6jJ;1|c!PTodUshz#Er+oWe{l74;NxQTH`CXl(or-qgHyMlD;e=IXdxPD$kud9 zh#g+vWu{jru*+4AUEQf4BEei+IY@fB*~7c*FbKiLNFrB1waAAJ_|`PbKt1pQ@&1Y9 zL-z4%G+XSL3^0wuFm8fn{Us!y9g^f?tbhZnSoY7P{`LI0pL-G6dfkD5&8!x>G=^L) z|A`wHEDNelGOBjrHV(Zs#E0ag#gKYhwm3iuR?syoeShm`TpF|326GjS)`1 zNO~k!0R&Rix^@8ycd>{@%@efP<|HzUg<%GF!E46ta{qkKUKY2G5jO@q-8Q#}RrMU9 zxf@PwSgyTBcl2Uo6>zF<Tv*({%%GnRWcbXHMW6&?%o|5Q`q^zOpZzFgkznVb-jSi%ii+yQRu=fm%h5zn%=+8 z_u(~e@eS{8nU^!TXl^UNvZN=;c-^6ze-b7o#q1puLTdI=!VwYVYys zWjnd<&!cHL$nyl6B2HtJ11FTHZGthpR%^3MCtlflZuQH|K^r~>tWuW2QN=yV8PxSy?=g6>Q?o70&S=tLL4Nw2(dy%egkRI~o=pUnsP{%OhcQZe z2x{#?OkzEVq*pxj| z5)#~-wf+(aXy6vQmtz03*7Mm-f7IW3@%QW2Jc^Ugut9s;w$(`uI`E3Il}ywEB~RG7Zzno*lt5mKEMVQitgpi{)$FdV2Xl_4UI z;O^~ge+t_ub3jGg{}S(HSMY$Ou&$NIdc|LSiz+@mL)!y_(5Jf)Rev!UF|(~J=GSc7 zRY}rc)y3g86~kFQa6n_Pc$Mg-k=d-;wmb}gkN!k zG$+&BsPtoVQ)-1EkV;m$g6++!g9L7p<6$lFXXI2{P^*pRIT~=!>Yc3*eyqJIol|{1 z=Ejo5knwnP(oRjifXLcaW8FlY1aYpnlb&f;f@wM5YB3Y2szP?uiSn&F*aRSInp=u? zlzKgiX8j?}Gz}Xlm$+w&4GfspHpD#fB`mVbm2`j66xHzZ3<7g!_i7e^75yW6s^d?W zKn=L-g=-f|>DD)al@nhw$W1m@0bYG{@k=i>;Q(NL@ z_sjoXID0}%=X26=*=&5!&gCuvnEm%ziTWHj(P=g_`V`FS>NPfZ{BXRW%WOd9f>Iey zIU#kum94FdY3>%da4YA+zfZoJT&|zP!Y`@XLIFg>q?6;ET6s9w08he)i@F8r> zhg|8&d!xv(g!MO8m<$tLE6v~a@!``orN*BQ!dc5oFMklup4d6f=QZfAcw3jl{3uz? z`pDi~2kBxZYKd}q0#J|n>c;lB$o@vD*d~(=Il}Zq3(0{ho0>IGOH+bxgt1me9gkHfAO!P z^Dql7TT!be<_jBSHjEvF<}SD@9vll407cn_x%}cqt#mh$hZqF0% z$7mC5=)Iu9bXV~uX^G^~h4X$`tp#pT&QPr$dJ*uGi6lgm!8uis+|-sOn?YTt^u&#j zm=8uMq4i*MFkZL_*EaNDE~l&`#9Nf>N&B$6O<<(8UMwwNdz!AB_^C|#XBR=(B^GW4 zC4&8>xN3E#q+CN0?p%3oFrbt!sWcbNw#VdT)D+S0HcJ%On=h(B|4Zaqsq|{2VUAVd zcIUk73!QXbMEvg0GS*_ht4#zj$8Rto)pZf>XWVLkw)tp)w!VY*&R8T%UT-uZz7I_H ztdnQuzuY2WF~`yaZMCw@j7JB=`ey%+GpThHr6mB*Ckqi3w|936?8As&s(ZxjDxFG91pUT^_(>gPf2d?TucX*vh>U~xhg>wU zT631;3L~w_>@psbDTwwf^LIq2lX*kdXC$7UP2WhG{B$J*51Ax!T9Wx7c9MTY=^uYo zW_nrZ_Qp6UrW|K1^**bTPEW4tGW%oXE!Rawju)LtNNa}@tr`siMAQ(+rT!yub*)4R-P_E zp>eF!Do}gosek8w#p4#Y4nK@1x|y$Coc?}ME?1{4f}umN&QKzEgw;^npj!Q;hH4dq zG_BKS?AMAR;QDb()+KzMYUoDI_YT2r7fIkSrUo= z1moQR6liCeUbkSz&FW4?p_5#$$Y|M5n2i11svVvhK2f5N5Ku(doLw9PhIVuAj_U=U?C6pSa z?!5BPYal=R5Vw zf_I~wUpJ&2_LU$|?MRMo{%0JG)Etd;t%g}>(@OvEM(cu{LcIVsZb!E8BAi8&Xl(r& z|KFZixCQv@P}V*IoCIuD8{r;H@qJw6^aZ7{XyEZ)zbiVo;k}KAYK=tZx-oL6aLQ`> zM&-XwSg)~KgDL317c?-vSgTjOeU&DWDI2l$vRW;MIeY!GzO9E@f06ah_82$9dA4O_ z42&t)g#YqSrg6!oGg#jCW?ZQ=UVEuYFvI9Tcdi4y5mUt%UU$+Xt&J8bdOkAFFi|cc zb&DMe&$N(EtJQ!-3KHA!vHY!%q4&0H=n*-sp)vpC#2Qemm7yzd`mA3fiu8W86pWIN zw$DutH)M3XbpeSeaxt`?sKfHDO+5iRObJl;`KrsQbus=(YKR}Kb z)ebBcT>erS`?Vs^CAqREQo5{khCQ}drwwutTnD!o6P!kvE5pa7g^poapQyKcYZgx| zRqig@gO7_;@N3;3`4BEO{t>9vvUj?8D=6use<{rcD0M02}bPs}1t5t%Y>>+cn zmA~!M+TXq|*&kYp>a4_cxmr-lNCBG@9{)bFx|h%);17j-#3HTD$t8LO=+T&;SA81& zTf3uKFj{WY9pe{?kZKF#puq86{_H}@&>6) z^5?RV(=@Cjc8rOwyY0=YqZIrgCsLnyq@En z&LnbTj}-p8K_nKf<2%V!kIHnO-koN72y+|U7c*XgR+vaq#x-)8jL&XRZ0wcjaPKAN;^u1}S7aXv9~X|Q2M2ISGd*I4aUmw3AR|3?S$gj&mBI<6V$OXip66{a06IW{U9;p640SwKG`N$A!3XYv|S z_V#!C1=OHAXiST$jyMGdvI5=g5DTHJuvhL)ZaSPj>k=!xy;QFl|<-zNes zi+y;w0mi>|WxGu?pI7FyY$ze^KI)?T10TZJ+RG%9;optmvexCX9l{R7*>d($Kf|+) zY9r0dm$edjHu`AN5w->bs-P(T+Um@B$d=Y#+%vYV06`UnDo0-Mb~m!8P&Y#cDt&KW zs$>w4&W;QuTi5~bT|L^v^?@e3zGlYvVjTjTkcw{540jC@g2KlWW^I3f&6~_BB#~3? z$3{OaQ>07vfH_R*(EN)sInK{iD8e2q8So?WSlQoQqAj%gGbm8OR9S;sOTjq%_bb)Z zUD=EG`W`c8FZ#Ta7NxGynmTWQVw}w2ltrP1{5|ZMc;5F5|Gu(~nuz7g%*Xto8#nw$ zB@DVg$w(4nnS@#;qm^|gmEHLEY@N$<8zurJCEP=`6Bds=bigapIg`g|d_5?Td|Ss{Hg2!L z0&?%Jb%;* zh%EvneW0lL0>PFyYJZ-po1L;Q5PA->D|hDVDNB4LM~?S%FfAds7Ul6PcO`}@A13yx zU|QGNF6df)j`A%T2(K6k7u?MhGnB3)170v%Qi4g)2~C2_De!=;jvdOD>u*#6t9Udk zNh+pnP;E1p3~hLDoWYrk-TmsBK@IW_!wwfzvBL*A-Po8T!ZpMd@&8hSGf7{n zGf)@x*(O*9oUiq9bPTV#&aB-Q8G1#pn<3qyKT%&(G@oaWq@YcRSusebUBm)X00@w#4ny|ns4BSt>wp3#>Jz;z& z7eX?TZg_9T8NDm0C5=~u=7@3#%0;PoHhfUWG7&#aipVD7SOGu^V@nLU zxYTZP20;uo6KoXahm%+ai~gDfS)cBrqxePLSgz~I69$CLm zvq=ETgpKzJvt#H$DVN28Y$4;F9+xYgbgA%qyhNH=l^*Wr6@O{BUnL)uT3mdB@o4{W z*4;0~9sCaLwyw2dYv-{07eQ@C^k4~uj{nX-GDpgh-iKL@T5`@!h8CsKZtDB%>m8Ggx~D3Om>fns^h|Lz1(NmLo+ zwSBL`^SzII7j-x&GsOH_ReSaO%X;5>vTaT|vxTbxYgyL&s;Et-)5z~@7uv2#HGEIx zu4=O0~U4~qf(u}k~5hQC-w*ErE=p$YCApeiiD=+ z(vB0s<~^vO7KC+M?0R^w4s6^F@A3nGrdZnTZ}eS4gnF(pM@O(HiPBE5y{>7~j&{t)e8R#!yF~lx1lc`ODIxnt2kkTi{N#8s#R4vPpi9Opn(4d zd)c7hDOfTr=2AoM|4=y55(7@C)zMNcE{s~@63v% zXxLdzSgJxNfOIuV#fln|dv0){E42{C51gb*k5}=|_Qb|Ust$?WH=g%k@ZdDzm0ytB zJYbSF4_+H$NJsaMwZ^R)!2&|}V#CQntDPn&$CsMA_~|0|2BjA7Py=&?oql03v2Bmp zKyoe9f|6O_q*#MsW`qqC-~RUp%sSmoTrmY%RVptT=BO=4KEGl-So5wzs=9nEwav<) z)j?N$vh@mnjw8B8$|GcXPLR=!6YAFaYqpX>d-j^0Lt``oZoGD>*Enmi?|do*!1szg z&MkfC=kUyo=f9l3{t51voH&5*hN}XJX@fdZvi}%PKXF})56lh0_1GX@yNOz869>V& z8@G+=_&28t82>ZK${ECMvYUGjo>vWo9rsbYzpMyDyp|v{jI$u#Q@9UPDWPgaH4(c? z{gcvxC(OH8WawVo)ZB?yk>W1L7@2)1)roZTy*EG#!V2CkX&_g>vh8ieSL8fTQibH} z0wW8`>~Z`NZHUmIc~$~2(urL+UTckkLl$J7NxPJ<*9PF(Rk6dGaC@%sE=0i`L5j@R z=VX_)vD2&`PY{AEJhQ0koA)0*+j=p0LDFt5BmV{n6-4cXYA|h+^bcpMR64fsj})tP zG&|mf&8RF(5~Rku()01gxoX>ZT5(;uSL89Sds{n${i>laz(kEM+6sqtgIQ|nK+!5( zgcx)}1!>EF*!q$1z4qMRKsh#Z>TLH>*7V zWIqwIR?c9uA-GR;*;)@DL=%3%*#Q;9$`7 zEnnJ{KtZJSBtbt1!Oci+?XlTAnIuIEBE}^p8&`Z(=3P-5wy?hi(jswy$$|p69M08R z$C}^sYm?(2L3iK|sh(^|FD2ulw!`Oj~&X??F zXIw>^MW-M1b(DbJhJjm7ypbCj8|aAdlv$AT$PGBWQ>IpD{sQYIRH;koN=F_X%MR8_ ztbig?rNSR1aS5p1->Po3mCoEZ4`AMwMNG&dnKdEN6aK0H2|DbTevDuD1*Yqr>|vAn z5{K-Fe}Q)O0-;u*jqg3#z)xsf%|38#_+b(AOD|y!v8j~5>+v71jTJH<&Pa%XY~l-O zoc~dYDS5kclj5|a5tC$SZ-nSrNFV17kQ-pta~-FU~l%YO^r z3692zZWivyp;wt@LSz#8XH7KSUpk%5NAcs@;u>bU;y6+f3ulT^qAPSB4P(WXF~abm zi-K}hxqpvi2r2g~ghMtuzKu4EzWNv(^h@=W8d^qfSq!J90g85e25J;->6Dq^|b%4wJ1=LwiY^+$Ad`_7l*dpj83|m%TyaToG zNZT-A-;dn@@)IbdF><79Nvr)-6%(QXm>7o+|1#sbvT{rpTVo*CctEdQNK+;P93HWe z36$?VF=BQNP`*9dhNQo~IW1T+pL+yVTB*7QP+^#QPJO@DL6 zCkxb4oM-EuV!Cq;Kd8f`v$w)r`;Hk2`<$wKII!T^aeu)E`aRe)I{FXIFfJ3=s&eN99sAsZC{u0MRf+2#_KKLQqi=nT|ynB!;$BQe=n-Ax1+GLxM;K zX&^wD5fY{_B@6)~_l3^uuDjN`@DG2mR^ZFDQs%i9|*I`9o`kz__ zuliJ;q;qm?>DEiz#o2ZCRF=8Pi?&V15pE8K{Md~OSl&q}c_GKBA`JDXx`M{)e7)FS z=7}ReUzG7lP@LNPbrEtoXoqL)n|ctKM`5N$<1K_lK($gFgy&`|l>GaNj!@4DQS;t# zSNiV&Nc8#p0T))l?v_0IKg1U+8tCl?kGe3%3ud|6vL~LD7XfUdH>=5Cf3J1G&G)q| z;!TEU^V=Vu<5?~CkyV_%m^R%c;kQpOP!geIO|}zxyHmD&ourq`)wkeU;p`FW$JITC z7SK?qajnU_I*mPr__JEFmws+8{zpW0WMd?6G9;pcN*!>HWwqw^yPxkrEo0u>?Y{&a zyh{3uoc-xaM}wd5XjzBc0k))gklNs7{gZip=cr!j;jOt^YM=3{2#_R2FAxs%L(Ens z<55&~7@FbtdWt3lkO5mY}(?R)~nTiE@oN|M^vh%zxLKpQBCd z&v=C8jGwR=D^g15;bx^-e^eLxU|TNtU+|0hp||k4LpCSIeM%ZMKNay$G1>flpGwgl ziZh)=+*$c2nJP;t!^cHbV#zp)VfVsRGocK6sW95@56Fioej&AUaFT+6{c~v@a>;37mBuzj!cL@T2h~p|^7X$D1U_tPknfi;_C^<&?dSM7g#Vx4Vm^cx)~X#IXDLtU$^_~B3T-?N?!+}e?Em(l5CU`8i=<)j5; zLvN-!+B3WAw^-N_p3cDG2`BiielNV1G7^6Dlz&0qY=&oTu`q56X(K6423W!5uZ_EC zgws_V@g>Rt=$_ojGd;z^y0~ni;vag(ikp~VccSaD>;<8C83KO_kKP3_!nzQwGh_ry zc*0NA1nW2HabQo4u8ULl*>PqKSRaShISy z+fKf>d7^g3#}-H)0pfY?zlw)sUJy;Nxsx(;waqjdW7Q426*qd?>E+DMob^<{pQ8T@ z2RX-4?DhVwF&9)_+uiOubhIQ`^53we;ln)+ve6_lY65eH8(vrL#Q6n>kfnj=p@-i} zniP?GXvan!5M`(`vohW1sSs@2GjAtb{FER}cV^*_cRBF6^FE_?%;mV`nL;OhvkxYF z`TUTKryljWTwL%nJ+0~iiBV?f<~#e^djez1%s9sf1vyzLc7s_bjg=)egGpzr9RdXZ z7&Na@Q2Lt)nj7|WSJZ-IsQvjbK97O&vW#e40b)hIcNz@Fi{%>&mtrFx6L8cGVj zRQR`!@s8>LE}9LDOh>Kj#h@gh4+RaPH$|E~<|TG)sbQs=FMb$;`7mDdlW0#>Ir>aR znJYT83P?u1Cfxc3leE(w3`%xe^!*^(vnnwVW3r-yWur}Vh6iA}FH=Nme^L!LZCiW= z=LicXtL^wwv03{xZ1q)dDxASp*_=nY?tDva&g}TE>D9AfFVkV!i>4b!$pNgKZfD{p zD$oV#*suxyb<4e)$^2K)wajmrDqES-=dx;F>mAjiR>>g$zdr znGuhQ%0Y7fOb3s3aa1=H+gN2AaR?tx8P_buw=u~r2T>z;Et4e!7dm|vV+S3PTD4_5 zOKlGFap#5gz8hhhr4;_y-(;4|Qk}&8c_dF)Zwi=TEksxFA(ceuTi<1=9l54U|RdopdW9~rx~8s;UdO5SD>XGx6(Yth!& zOJL8$($M<4xftZIP5%wmd`{g%kpj6VI;J4Szs2dsw4k>z?oknP$!I!Y(t|8-+mD&B z^)CeOYQ2citre z<`~=*g@sR$A{X1Dy*V9+69#FUU~8*NgEg=g-IT^G^$kd!HDmYA1ueH16J7My;YZtYR=9xrTgIbD~I9l8l{^fnVn)pU z;{6WU43ZUD^%}~r`l=h*G$JL>{bFf8zq{DNG%({_y<7;fbL4u{QLl6nZSQd~F3T}R zMTR2(q|VdOu`e>8Sc3<9d@PS&(NrT;>XXl>PRhTjU(LEOK+o1wGde*6J@uqPOP$@?A zNC})zrVg!wt@+yXXL-MuXB|rGJXGuUWELdbl{frM60ONr`@H_qkN11oeC_HDk>rkc zNd+^naXn$O0rzYuKKlQxRBgYea{_X+L-tHSN}gGX zl8^Wo0-V{ka5XUN+!=7j{G}cA+kl2+5dZ(E&e=Iumk_&Ued&oA`iMnROjsuV z|9{XdU3_itYH8K@vs1!Idb*x2%3mO*_PMCX%V_#ptN&Hb@*zbA3N? zO|xi*I{H=0X((a7&vT~3aX0$T7YOw>)DYbwF_yA8z$cvoFZZm2O7~w4-PGVW)K;ek zLe@KQZG9uZO$IBnoGH3dMI3jBkkTpbFbb^kM zB$0gkvpm*x8XJS@Man-(4}99*hY$VDJ5GzF$aj4^oI)t80qddoYo}u+fumYla_f!{ zlWhtaAQpe=u2`;oc@REF_#sfSXyLkh;dL!}M*SvL5qCsqZ&dK7Y~AZYzGbNXO=p64 zK}5gCP&v<+2W7*r)EUd4W(Se&dR>3)=efPxNu^t2G{fRQ${&C&0Q%v!PG~Dgmvk0D zRl$J>h(g!yZo0)={{Mb`4P5U=mG~?NZaf|<%P@$yzXW%vBOggv;EIxDqavaj<0Yew zv=bNUb`BF#s;w*($b6LL6x*Yo5XOkXL{?Z9>Gv!z?HPlh>OEXNifVys8Mp_rWmG|5 zq3_6!pS4nVUViHG((+Umo6kUCZyD11d+xY2h4=^KxM3nm9LjfvPkBMygNhJ{GJBgvyF(Da-{VD~Ax7-rjQz@iy&rxom$(a6!GypoKltn}VDgM*Ai& zG!pK*KwTqb1vCVj8_-Hzf#|%y<>QI~9tRo~6D!4kQ9AT$;y(#!cDo%pnc8S~sz8o^TA}n*KH6}~n zH6^KXa}jZ_dFBB68?ljB6uO)$a3t!irZN3zz5R#6GYe>v@@T!H_6ouR`EqWd?cBvW z?*QIYe}xqzM#hFp&k0l`wNP&aarpG45wi{ds)_E1Dj z;*EAt=0tOX-3(KYBHOgIknQX)k&nzr)N|YWIlx_?-KDXgd=UK8#qOp}Y|_ire+PTH z&8LB!^h3t{h=|lUDBE{Lf9>ZY{sV#dzO;PI9*md3f2D(HiX?uKPPZm)oz<(p>F4v7 zzP$cj1?{Xip!UpkWos@XjK$KNXc>5#=F#&(z+q?lIAAR>LS(embKxfuWiDiEkvTrb zG*T2b=ax?hO}3^&!>ViR0#d3eXe`;BZ~Dk&zHAyG_dWrtS`@CCOLd>X7sjrQA*omXUrSIQzUI*6wFgoBN0oU z$p&fpP%tcMnJ00MsK8Rl#rQlwlFicdt5TiA%$k@mM6^%{O5-jSpqq%bkscEnTF5^K z2<_21id$OC!?RZSE+NUeiIonK=#Ajgze*1ADQ(KHYa@q z4|k-3@B_QBl0#9-6AE4prY-Yvjp{2;)n6F6f4~!IxrL|fhz`D*L-X=EfzE*K2&;@0 zH0(V>Ixv`8^7H0;0%PF1b|fGXd+mqR?;C-dT8a)d&yH^_cb}yHI%mM1nVrcmOqKoK zQ=9wOVW$$T)Dyljv~q}S3W;#(TZWd5x}i*w;4`V`b`Cy=0?4bC=KCNQ*ASPn#gIeF z@)l=WbD00+XnZ=Sg>W3<7xHht4s%dOs(>+kjBr%V-%?F+5$ zH>BFm$CYWWq_lQ~)?K~%E;6t?0mv?w%k(KD3$F()OqX`$F=b_QL98;r#0(}AF`F+_ z+;|{`fr8wm!08!0o|IK%@Z5pQDn@TBVUIqlUE+5`5FmbnjDMXI!8-^KhF6?lGE~Ly f5Ctal)N88x>(QT^MZb9ue7T>5pQt<@c Date: Mon, 17 Mar 2025 01:48:38 -0700 Subject: [PATCH 4/5] Add Appendix --- .../design_doc.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md index 74ac1695865..d128ffa33a1 100644 --- a/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md +++ b/docs/proposal/jakarta_messaging_3_1_shared_subscription/design_doc.md @@ -173,3 +173,12 @@ It may be best to introduce a new v13 of openwire for Jakarta 3.1. This means us The *`isDurable`* method of *`ConsumerInfo`* will need to be updated to work with all versions of openwire. It will need the following evaluation: *`return (!shared && subscriptionName != null) || (shared && durable);`*. *`shared`* will default to false for openwire versions older than v13 and so *`isDurable`* will evaluate in the same way as previously, based on *`subscriptionName`*. For openwire v13 the *`shared`* and *`durable`* fields will be populated and so can be used to determine durability for shared subscriptions. + +## Appendix + +Links to the official spec: + +* [https://jakarta.ee/specifications/messaging/3.1/jakarta-messaging-spec-3.1\#shared-non-durable-subscriptions](https://jakarta.ee/specifications/messaging/3.1/jakarta-messaging-spec-3.1#shared-non-durable-subscriptions) +* [https://jakarta.ee/specifications/messaging/3.1/jakarta-messaging-spec-3.1\#shared-durable-subscriptions](https://jakarta.ee/specifications/messaging/3.1/jakarta-messaging-spec-3.1#shared-durable-subscriptions) +* [https://jakarta.ee/specifications/messaging/3.1/jakarta-messaging-spec-3.1\#jakarta-messaging-application-server-facilities](https://jakarta.ee/specifications/messaging/3.1/jakarta-messaging-spec-3.1#jakarta-messaging-application-server-facilities) +* [https://www.oracle.com/technical-resources/articles/java/jms2messaging.html](https://www.oracle.com/technical-resources/articles/java/jms2messaging.html) \ No newline at end of file From 473de416ab4a4d3a52a00f5ec7be4d2569a9174c Mon Sep 17 00:00:00 2001 From: Ken Liao Date: Thu, 3 Apr 2025 00:42:10 -0700 Subject: [PATCH 5/5] First version of MVP --- .../apache/activemq/broker/BrokerService.java | 2 +- .../activemq/broker/TransportConnection.java | 7 + .../activemq/broker/TransportConnector.java | 2 +- .../broker/region/AbstractRegion.java | 19 ++- .../broker/region/BaseDestination.java | 1 + .../region/DurableTopicSubscription.java | 12 +- .../broker/region/PrefetchSubscription.java | 1 + .../activemq/broker/region/RegionBroker.java | 7 +- ...haredDurableTopicSubscriptionMetadata.java | 46 +++++++ .../region/SharedTopicSubscription.java | 93 +++++++++++++ .../apache/activemq/broker/region/Topic.java | 124 ++++++++++++++---- .../activemq/broker/region/TopicRegion.java | 115 +++++++++++----- .../SharedSubscriptionDispatchPolicy.java | 35 +++++ .../region/policy/SimpleDispatchPolicy.java | 4 +- .../apache/activemq/ActiveMQConnection.java | 2 +- .../apache/activemq/command/ConsumerInfo.java | 6 + .../apache/activemq/util/ConsumerThread.java | 9 +- .../console/command/ConsumerCommand.java | 12 ++ 18 files changed, 431 insertions(+), 66 deletions(-) create mode 100644 activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedDurableTopicSubscriptionMetadata.java create mode 100644 activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedTopicSubscription.java create mode 100644 activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SharedSubscriptionDispatchPolicy.java diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/BrokerService.java b/activemq-broker/src/main/java/org/apache/activemq/broker/BrokerService.java index b05c3ec66d8..db046561a4a 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/BrokerService.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/BrokerService.java @@ -184,7 +184,7 @@ public class BrokerService implements Service { private JmsConnector[] jmsBridgeConnectors; // these are Jms to Jms bridges // to other jms messaging systems private boolean deleteAllMessagesOnStartup; - private boolean advisorySupport = true; + private boolean advisorySupport = false; private boolean anonymousProducerAdvisorySupport = false; private URI vmConnectorURI; private String defaultSocketURIString; diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnection.java b/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnection.java index 9079e24d676..6a67ea7f691 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnection.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnection.java @@ -192,6 +192,7 @@ public TransportConnection(TransportConnector connector, final Transport transpo @Override public void onCommand(Object o) { serviceLock.readLock().lock(); + System.out.println("[PUB_PATH] onCommand " + o.getClass().getName()); try { if (!(o instanceof Command)) { throw new RuntimeException("Protocol violation - Command corrupted: " + o.toString()); @@ -332,6 +333,7 @@ public Response service(Command command) { Response response = null; boolean responseRequired = command.isResponseRequired(); int commandId = command.getCommandId(); + System.out.println("[PUB_PATH] got commandId: " + commandId); try { if (status.get() != PENDING_STOP) { response = command.visit(this); @@ -576,6 +578,9 @@ public Response processRecoverTransactions(TransactionInfo info) throws Exceptio @Override public Response processMessage(Message messageSend) throws Exception { + // [PUB_PATH] + System.out.println("[PUB_PATH] in processMessage"); + System.out.println(messageSend.toString()); ProducerId producerId = messageSend.getProducerId(); ProducerBrokerExchange producerExchange = getProducerBrokerExchange(producerId); if (producerExchange.canDispatch(messageSend)) { @@ -628,6 +633,7 @@ public Response processRemoveDestination(DestinationInfo info) throws Exception @Override public Response processAddProducer(ProducerInfo info) throws Exception { + System.out.println("[PUB_PATH] in processAddProducer]"); SessionId sessionId = info.getProducerId().getParentId(); ConnectionId connectionId = sessionId.getParentId(); TransportConnectionState cs = lookupConnectionState(connectionId); @@ -683,6 +689,7 @@ public Response processRemoveProducer(ProducerId id) throws Exception { @Override public Response processAddConsumer(ConsumerInfo info) throws Exception { + System.out.println("[SUB_PATH] in processAddConsumer]"); SessionId sessionId = info.getConsumerId().getParentId(); ConnectionId connectionId = sessionId.getParentId(); TransportConnectionState cs = lookupConnectionState(connectionId); diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnector.java b/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnector.java index 78ab04cc703..8257a50f408 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnector.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/TransportConnector.java @@ -231,7 +231,7 @@ public void onAccept(final Transport transport) { public void run() { try { if (!brokerService.isStopping()) { - Connection connection = createConnection(transport); + Connection connection = createConnection(transport); // [PUB_PATH] connection.start(); } else { throw new BrokerStoppedException("Broker " + brokerService + " is being stopped"); diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java index af77b1d4498..b56e198b9a3 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/AbstractRegion.java @@ -143,7 +143,7 @@ public void stop() throws Exception { @Override public Destination addDestination(ConnectionContext context, ActiveMQDestination destination, boolean createIfTemporary) throws Exception { - + System.out.println("[SUB_PATH] AbstractRegion addDesitnation: " + destination.toString()); destinationsLock.writeLock().lock(); try { Destination dest = destinations.get(destination); @@ -154,6 +154,7 @@ public Destination addDestination(ConnectionContext context, ActiveMQDestination validateMaxDestinations(destination); LOG.debug("{} adding destination: {}", broker.getBrokerName(), destination); + System.out.println("[SUB_PATH] AbstractRegion addDestination: creating destination: " + destination.toString()); dest = createDestination(context, destination); // intercept if there is a valid interceptor defined DestinationInterceptor destinationInterceptor = broker.getDestinationInterceptor(); @@ -238,6 +239,7 @@ protected void validateMaxDestinations(ActiveMQDestination destination) protected List addSubscriptionsForDestination(ConnectionContext context, Destination dest) throws Exception { List rc = new ArrayList(); // Add all consumers that are interested in the destination. + System.out.println("[SUB_PATH] AbstractRegion addSubscriptionsForDestination: " + dest.toString()); for (Iterator iter = subscriptions.values().iterator(); iter.hasNext();) { Subscription sub = iter.next(); if (sub.matches(dest.getActiveMQDestination())) { @@ -342,10 +344,11 @@ public Map getDestinationMap() { public Subscription addConsumer(ConnectionContext context, ConsumerInfo info) throws Exception { LOG.debug("{} adding consumer: {} for destination: {}", broker.getBrokerName(), info.getConsumerId(), info.getDestination()); + System.out.println("[SUB_PATH] in AbstractRegion addConsumer " + info.getSubscriptionName() + " " + info.getDestination()); ActiveMQDestination destination = info.getDestination(); if (destination != null && !destination.isPattern() && !destination.isComposite()) { // lets auto-create the destination - lookup(context, destination,true); + lookup(context, destination, true); } Object addGuard; @@ -379,7 +382,9 @@ public Subscription addConsumer(ConnectionContext context, ConsumerInfo info) th // DestinationFilter.parseFilter(info.getDestination()); + System.out.println("[SUB_PATH] AbstractRegion addConsumer creating subscription"); Subscription sub = createSubscription(context, info); + System.out.println("[SUB_PATH] created subscription: " + sub.toString()); // At this point we're done directly manipulating subscriptions, // but we need to retain the synchronized block here. Consider @@ -404,8 +409,9 @@ public Subscription addConsumer(ConnectionContext context, ConsumerInfo info) th List removeList = new ArrayList(); for (Destination dest : addList) { + System.out.println("[SUB_PATH] adding subscription to destination: " + dest.toString()); try { - dest.addSubscription(context, sub); + dest.addSubscription(context, sub); // This is logic that added the subscription removeList.add(dest); } catch (SecurityException e){ if (sub.isWildcard()) { @@ -503,13 +509,14 @@ public void removeSubscription(ConnectionContext context, RemoveSubscriptionInfo @Override public void send(final ProducerBrokerExchange producerExchange, Message messageSend) throws Exception { + System.out.println("[PUB_PATH] AbstractRegion send"); final ConnectionContext context = producerExchange.getConnectionContext(); if (producerExchange.isMutable() || producerExchange.getRegionDestination() == null) { final Destination regionDestination = lookup(context, messageSend.getDestination(),false); producerExchange.setRegionDestination(regionDestination); } - + System.out.println("[PUB_PATH] AbstractRegion send destination: " + producerExchange.getRegionDestination().toString()); producerExchange.getRegionDestination().send(producerExchange, messageSend); if (producerExchange.getProducerState() != null && producerExchange.getProducerState().getInfo() != null){ @@ -551,7 +558,7 @@ protected Destination lookup(ConnectionContext context, ActiveMQDestination dest protected Destination lookup(ConnectionContext context, ActiveMQDestination destination, boolean createTemporary, boolean autoCreate) throws Exception { Destination dest = null; - + System.out.println("[SUB_PATH] AbstractRegion in lookup for desitination: " + destination.toString()); destinationsLock.readLock().lock(); try { dest = destinations.get(destination); @@ -564,7 +571,9 @@ protected Destination lookup(ConnectionContext context, ActiveMQDestination dest // Try to auto create the destination... re-invoke broker // from the // top so that the proper security checks are performed. + System.out.println("[SUB_PATH] AbstractRegion lookup: creating destination"); dest = context.getBroker().addDestination(context, destination, createTemporary); + System.out.println("[SUB_PATH] AbstractRegion lookup: done create destination"); } if (dest == null) { diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/BaseDestination.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/BaseDestination.java index e34f23a4dda..7b150ec19ce 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/BaseDestination.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/BaseDestination.java @@ -644,6 +644,7 @@ public boolean isDisposed() { * some way - such as to send to a dead letter queue or something.. */ protected void onMessageWithNoConsumers(ConnectionContext context, Message msg) throws Exception { + System.out.println("[PUB_PATH] onMessageWithNoConsumers"); if (!msg.isPersistent()) { if (isSendAdvisoryIfNoConsumers()) { // allow messages with no consumers to be dispatched to a dead diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/DurableTopicSubscription.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/DurableTopicSubscription.java index 6946a33fa57..de039f97357 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/DurableTopicSubscription.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/DurableTopicSubscription.java @@ -62,6 +62,7 @@ public class DurableTopicSubscription extends PrefetchSubscription implements Us private final AtomicBoolean active = new AtomicBoolean(); private final AtomicLong offlineTimestamp = new AtomicLong(-1); private final HashSet ackedAndPrepared = new HashSet(); + private boolean shared = false; public DurableTopicSubscription(Broker broker, SystemUsage usageManager, ConnectionContext context, ConsumerInfo info, boolean keepDurableSubsActive) throws JMSException { @@ -115,6 +116,7 @@ protected void setPendingBatchSize(PendingMessageCursor pending, int numberToDis @Override public void add(ConnectionContext context, Destination destination) throws Exception { + System.out.println("[SUB_PATH] DurableTopicSubscription add: " + destination.toString()); if (!destinations.contains(destination)) { super.add(context, destination); } @@ -381,7 +383,7 @@ public void afterRollback() throws Exception { public synchronized String toString() { return "DurableTopicSubscription-" + getSubscriptionKey() + ", id=" + info.getConsumerId() + ", active=" + isActive() + ", destinations=" + durableDestinations.size() + ", total=" + getSubscriptionStatistics().getEnqueues().getCount() + ", pending=" + getPendingQueueSize() + ", dispatched=" + getSubscriptionStatistics().getDispatched().getCount() - + ", inflight=" + dispatched.size() + ", prefetchExtension=" + getPrefetchExtension(); + + ", inflight=" + dispatched.size() + ", prefetchExtension=" + getPrefetchExtension() + ", shared: " + isShared(); } public SubscriptionKey getSubscriptionKey() { @@ -438,4 +440,12 @@ public boolean isKeepDurableSubsActive() { public boolean isEnableMessageExpirationOnActiveDurableSubs() { return enableMessageExpirationOnActiveDurableSubs; } + + public boolean isShared() { + return shared; + } + + public void setShared(boolean shared) { + this.shared = shared; + } } diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/PrefetchSubscription.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/PrefetchSubscription.java index d59717444c7..3d995087a26 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/PrefetchSubscription.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/PrefetchSubscription.java @@ -622,6 +622,7 @@ private void addReferencesAndUpdateRedispatch(LinkedList redis // made public so it can be used in MQTTProtocolConverter public void dispatchPending() throws IOException { + System.out.println("[PUB_PATH] PrefetchSubscription dispatch pending"); List slowConsumerTargets = null; synchronized(pendingLock) { diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/RegionBroker.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/RegionBroker.java index 844fa120299..be7ed418f0a 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/RegionBroker.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/RegionBroker.java @@ -204,6 +204,7 @@ protected Region createTempQueueRegion(SystemUsage memoryManager, TaskRunnerFact } protected Region createTopicRegion(SystemUsage memoryManager, TaskRunnerFactory taskRunnerFactory, DestinationFactory destinationFactory) { + System.out.println("[PUB_SUB] create topic region"); return new TopicRegion(this, destinationStatistics, memoryManager, taskRunnerFactory, destinationFactory); } @@ -427,9 +428,12 @@ public void removeProducer(ConnectionContext context, ProducerInfo info) throws @Override public Subscription addConsumer(ConnectionContext context, ConsumerInfo info) throws Exception { + System.out.println("[SUB_PATH] Add a consumer in RegionBroker"); ActiveMQDestination destination = info.getDestination(); if (destinationInterceptor != null) { + System.out.println("[SUB_PATH] RegionBroker creating destinaiton incerceptor for destination: " + destination.toString()); destinationInterceptor.create(this, context, destination); + System.out.println("[SUB_PATH] RegionBroker done creating destinaiton incerceptor for destination: " + destination.toString()); } inactiveDestinationsPurgeLock.readLock().lock(); try { @@ -462,8 +466,9 @@ public void removeSubscription(ConnectionContext context, RemoveSubscriptionInfo } @Override - public void send(ProducerBrokerExchange producerExchange, Message message) throws Exception { + public void send(ProducerBrokerExchange producerExchange, Message message) throws Exception { // [PUB_PATH] ActiveMQDestination destination = message.getDestination(); + System.out.println("[PUB_PATH] RegionBroker.send: " + destination.toString()); message.setBrokerInTime(System.currentTimeMillis()); if (producerExchange.isMutable() || producerExchange.getRegion() == null || (producerExchange.getRegionDestination() != null && producerExchange.getRegionDestination().isDisposed())) { diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedDurableTopicSubscriptionMetadata.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedDurableTopicSubscriptionMetadata.java new file mode 100644 index 00000000000..39a26000269 --- /dev/null +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedDurableTopicSubscriptionMetadata.java @@ -0,0 +1,46 @@ +package org.apache.activemq.broker.region; + +import org.apache.activemq.util.SubscriptionKey; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class SharedDurableTopicSubscriptionMetadata { + private static final Logger LOG = LoggerFactory.getLogger(SharedDurableTopicSubscriptionMetadata.class); + + private String subscriptionName; + private ArrayList subKeys = new ArrayList<>(); + private ConcurrentMap subMap = new ConcurrentHashMap(); + private int counter; + + public SharedDurableTopicSubscriptionMetadata(String subscriptionName) { + this.subscriptionName = subscriptionName; + } + + public void addDurableTopicSubscription(SubscriptionKey key, DurableTopicSubscription subscription) { + if (!subMap.containsKey(key)) { + subKeys.add(key); + subMap.put(key, subscription); + } + } + + public void removeDurableTopicSubscription(SubscriptionKey key, DurableTopicSubscription subscription) { + if (subMap.containsKey(key)) { + subMap.remove(key); + subKeys.remove(key); + } + } + + public DurableTopicSubscription getNextDurableTopicSubscription() { + int index = counter % subKeys.size(); + counter++; + return subMap.get(subKeys.get(index)); + } +} diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedTopicSubscription.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedTopicSubscription.java new file mode 100644 index 00000000000..579de66c717 --- /dev/null +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/SharedTopicSubscription.java @@ -0,0 +1,93 @@ +package org.apache.activemq.broker.region; + +import jakarta.jms.InvalidSelectorException; +import org.apache.activemq.broker.Broker; +import org.apache.activemq.broker.ConnectionContext; +import org.apache.activemq.command.ConsumerInfo; +import org.apache.activemq.command.MessageDispatchNotification; +import org.apache.activemq.command.MessagePull; +import org.apache.activemq.command.Response; +import org.apache.activemq.usage.UsageListener; + +public class SharedTopicSubscription extends AbstractSubscription { + + public SharedTopicSubscription(Broker broker, ConnectionContext context, + ConsumerInfo info) throws InvalidSelectorException { + super(broker, context, info); + } + + @Override + public void add(MessageReference node) throws Exception { + + } + + @Override + public Response pullMessage(ConnectionContext context, MessagePull pull) throws Exception { + return null; + } + + @Override + public void processMessageDispatchNotification(MessageDispatchNotification mdn) throws Exception { + + } + + @Override + public int getPendingQueueSize() { + return 0; + } + + @Override + public long getPendingMessageSize() { + return 0; + } + + @Override + public int getDispatchedQueueSize() { + return 0; + } + + @Override + public long getDispatchedCounter() { + return 0; + } + + @Override + public long getEnqueueCounter() { + return 0; + } + + @Override + public long getDequeueCounter() { + return 0; + } + + @Override + public boolean isLowWaterMark() { + return false; + } + + @Override + public boolean isHighWaterMark() { + return false; + } + + @Override + public boolean isFull() { + return false; + } + + @Override + public void updateConsumerPrefetch(int newPrefetch) { + + } + + @Override + public void destroy() { + + } + + @Override + public int getInFlightSize() { + return 0; + } +} diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java index a9e07874e05..fe41f091127 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java @@ -36,6 +36,7 @@ import org.apache.activemq.broker.region.policy.DispatchPolicy; import org.apache.activemq.broker.region.policy.LastImageSubscriptionRecoveryPolicy; import org.apache.activemq.broker.region.policy.RetainedMessageSubscriptionRecoveryPolicy; +import org.apache.activemq.broker.region.policy.SharedSubscriptionDispatchPolicy; import org.apache.activemq.broker.region.policy.SimpleDispatchPolicy; import org.apache.activemq.broker.region.policy.SubscriptionRecoveryPolicy; import org.apache.activemq.broker.util.InsertionCountList; @@ -79,8 +80,11 @@ public class Topic extends BaseDestination implements Task { protected final CopyOnWriteArrayList consumers = new CopyOnWriteArrayList(); private final ReentrantReadWriteLock dispatchLock = new ReentrantReadWriteLock(); private DispatchPolicy dispatchPolicy = new SimpleDispatchPolicy(); + private SharedSubscriptionDispatchPolicy sharedSubscriptionDispatchPolicy = new SharedSubscriptionDispatchPolicy(); private SubscriptionRecoveryPolicy subscriptionRecoveryPolicy; private final ConcurrentMap durableSubscribers = new ConcurrentHashMap(); + private final ConcurrentMap sharedDurableSubscribers = new ConcurrentHashMap<>(); + protected final ConcurrentMap sharedDurableSubscriptionMetadataMap = new ConcurrentHashMap(); private final TaskRunner taskRunner; private final TaskRunnerFactory taskRunnerFactor; private final LinkedList messagesWaitingForSpace = new LinkedList(); @@ -133,8 +137,9 @@ public boolean lock(MessageReference node, LockOwner sub) { @Override public void addSubscription(ConnectionContext context, final Subscription sub) throws Exception { + System.out.println("[SUB_PATH] Topic addSubscription"); if (!sub.getConsumerInfo().isDurable()) { - + // Non durable // Do a retroactive recovery if needed. if (sub.getConsumerInfo().isRetroactive() || isAlwaysRetroactive()) { @@ -146,6 +151,7 @@ public void addSubscription(ConnectionContext context, final Subscription sub) t synchronized (consumers) { if (!consumers.contains(sub)){ sub.add(context, this); + System.out.println("[SUB_PATH] Topc add to consumers: line 152"); consumers.add(sub); applyRecovery=true; super.addSubscription(context, sub); @@ -162,39 +168,58 @@ public void addSubscription(ConnectionContext context, final Subscription sub) t synchronized (consumers) { if (!consumers.contains(sub)){ sub.add(context, this); + System.out.println("[SUB_PATH] Topc add to consumers: line 169"); consumers.add(sub); super.addSubscription(context, sub); } } } } else { + System.out.println("[SUB_PATH] Topic addSubscription: durable"); DurableTopicSubscription dsub = (DurableTopicSubscription) sub; super.addSubscription(context, sub); sub.add(context, this); if(dsub.isActive()) { - synchronized (consumers) { - boolean hasSubscription = false; - - if (consumers.size() == 0) { - hasSubscription = false; + if (dsub.isShared()) { + System.out.println("[SUB_PATH] Topic addSubscription: adding shared durable subscription"); + sharedDurableSubscribers.put(dsub.getSubscriptionKey(), dsub); + String subscriptionName = dsub.getConsumerInfo().getSubscriptionName(); + SharedDurableTopicSubscriptionMetadata sharedDurableTopicSubscriptionMetadata = null; + if (sharedDurableSubscriptionMetadataMap.containsKey(subscriptionName)) { + sharedDurableTopicSubscriptionMetadata = sharedDurableSubscriptionMetadataMap.get(subscriptionName); } else { - for (Subscription currentSub : consumers) { - if (currentSub.getConsumerInfo().isDurable()) { - DurableTopicSubscription dcurrentSub = (DurableTopicSubscription) currentSub; - if (dcurrentSub.getSubscriptionKey().equals(dsub.getSubscriptionKey())) { - hasSubscription = true; - break; + sharedDurableTopicSubscriptionMetadata = new SharedDurableTopicSubscriptionMetadata(subscriptionName); + sharedDurableSubscriptionMetadataMap.put(subscriptionName, sharedDurableTopicSubscriptionMetadata); + } + sharedDurableTopicSubscriptionMetadata.addDurableTopicSubscription(dsub.getSubscriptionKey(), dsub); + sharedDurableSubscribers.put(dsub.getSubscriptionKey(), dsub); + } else { + System.out.println("[SUB_PATH] Topic addSubscription: adding non-shared durable subscription"); + synchronized (consumers) { + boolean hasSubscription = false; + + if (consumers.size() == 0) { + hasSubscription = false; + } else { + for (Subscription currentSub : consumers) { + if (currentSub.getConsumerInfo().isDurable()) { + DurableTopicSubscription dcurrentSub = (DurableTopicSubscription) currentSub; + if (dcurrentSub.getSubscriptionKey().equals(dsub.getSubscriptionKey())) { + hasSubscription = true; + break; + } } } } - } - if (!hasSubscription) { - consumers.add(sub); + if (!hasSubscription) { + System.out.println("[SUB_PATH] Topc add to consumers: line 214"); + consumers.add(sub); + } } + durableSubscribers.put(dsub.getSubscriptionKey(), dsub); } } - durableSubscribers.put(dsub.getSubscriptionKey(), dsub); } } @@ -258,12 +283,14 @@ private boolean hasSelectorChanged(SubscriptionInfo info1, ConsumerInfo info2) { } public void activate(ConnectionContext context, final DurableTopicSubscription subscription) throws Exception { + System.out.println("[PUB_PATH] Topic activate"); // synchronize with dispatch method so that no new messages are sent // while we are recovering a subscription to avoid out of order messages. dispatchLock.writeLock().lock(); try { if (topicStore == null) { + System.out.println("[PUB_PATH] Topic topicStore == null"); return; } @@ -274,6 +301,7 @@ public void activate(ConnectionContext context, final DurableTopicSubscription s if (info != null) { // Check to see if selector changed. if (hasDurableSubChanged(info, subscription.getConsumerInfo())) { + System.out.println("[SUB_PATH] Topic activate: hasDurableSubchanged true: adding stuff"); // Need to delete the subscription topicStore.deleteSubscription(clientId, subscriptionName); info = null; @@ -285,9 +313,24 @@ public void activate(ConnectionContext context, final DurableTopicSubscription s consumers.remove(subscription); } } else { - synchronized (consumers) { - if (!consumers.contains(subscription)) { - consumers.add(subscription); + if (subscription.isShared()) { + System.out.println("[SUB_PATH] Topic activate: adding stuff: shared"); + SharedDurableTopicSubscriptionMetadata sharedDurableTopicSubscriptionMetadata = null; + if (sharedDurableSubscriptionMetadataMap.containsKey(subscriptionName)) { + sharedDurableTopicSubscriptionMetadata = sharedDurableSubscriptionMetadataMap.get(subscriptionName); + } else { + sharedDurableTopicSubscriptionMetadata = new SharedDurableTopicSubscriptionMetadata(subscriptionName); + sharedDurableSubscriptionMetadataMap.put(subscriptionName, sharedDurableTopicSubscriptionMetadata); + } + sharedDurableTopicSubscriptionMetadata.addDurableTopicSubscription(subscription.getSubscriptionKey(), subscription); + sharedDurableSubscribers.put(subscription.getSubscriptionKey(), subscription); + } else { + System.out.println("[SUB_PATH] Topic activate: adding stuff: non-shared"); + synchronized (consumers) { + if (!consumers.contains(subscription)) { + System.out.println("[SUB_PATH] Topc add to consumers: line 326"); + consumers.add(subscription); + } } } } @@ -295,6 +338,8 @@ public void activate(ConnectionContext context, final DurableTopicSubscription s // Do we need to create the subscription? if (info == null) { + System.out.println("[SUB_PATH] Topic activate: we need to create the subscription"); + System.out.println("[SUB_PATH] Topic activate: subscription: " + subscription.toString()); info = new SubscriptionInfo(); info.setClientId(clientId); info.setSelector(subscription.getConsumerInfo().getSelector()); @@ -304,9 +349,23 @@ public void activate(ConnectionContext context, final DurableTopicSubscription s // This destination is an actual destination id. info.setSubscribedDestination(subscription.getConsumerInfo().getDestination()); // This destination might be a pattern - synchronized (consumers) { - consumers.add(subscription); - topicStore.addSubscription(info, subscription.getConsumerInfo().isRetroactive()); + if (subscription.isShared()) { + System.out.println("[SUB_PATH] Topc add to shared subscription metadata: line 351"); + SharedDurableTopicSubscriptionMetadata sharedDurableTopicSubscriptionMetadata = null; + if (sharedDurableSubscriptionMetadataMap.containsKey(subscriptionName)) { + sharedDurableTopicSubscriptionMetadata = sharedDurableSubscriptionMetadataMap.get(subscriptionName); + } else { + sharedDurableTopicSubscriptionMetadata = new SharedDurableTopicSubscriptionMetadata(subscriptionName); + sharedDurableSubscriptionMetadataMap.put(subscriptionName, sharedDurableTopicSubscriptionMetadata); + } + sharedDurableTopicSubscriptionMetadata.addDurableTopicSubscription(subscription.getSubscriptionKey(), subscription); + sharedDurableSubscribers.put(subscription.getSubscriptionKey(), subscription); + } else { + synchronized (consumers) { + System.out.println("[SUB_PATH] Topc add to consumers: line 347"); + consumers.add(subscription); + topicStore.addSubscription(info, subscription.getConsumerInfo().isRetroactive()); + } } } @@ -364,6 +423,7 @@ public void recoverRetroactiveMessages(ConnectionContext context, Subscription s @Override public void send(final ProducerBrokerExchange producerExchange, final Message message) throws Exception { + System.out.println("[PUB_PATH] in Topic"); final ConnectionContext context = producerExchange.getConnectionContext(); final ProducerInfo producerInfo = producerExchange.getProducerState().getInfo(); @@ -371,6 +431,7 @@ public void send(final ProducerBrokerExchange producerExchange, final Message me final boolean sendProducerAck = !message.isResponseRequired() && producerInfo.getWindowSize() > 0 && !context.isInRecoveryMode(); + System.out.println(destination.getPhysicalName()); message.setRegionDestination(this); if(getMessageInterceptorStrategy() != null) { @@ -527,6 +588,7 @@ public void run() { */ synchronized void doMessageSend(final ProducerBrokerExchange producerExchange, final Message message) throws IOException, Exception { + System.out.println("[PUB_PATH] in Topic doMessageSend"); final ConnectionContext context = producerExchange.getConnectionContext(); message.getMessageId().setBrokerSequenceId(getDestinationSequenceId()); Future result = null; @@ -580,7 +642,7 @@ public void afterRollback() throws Exception { } else { try { - dispatch(context, message); + dispatch(context, message); // dispatching to the subscription } finally { message.decrementReferenceCount(); } @@ -777,6 +839,7 @@ protected void dispatch(final ConnectionContext context, Message message) throws // AMQ-2586: Better to leave this stat at zero than to give the user // misleading metrics. // destinationStatistics.getMessages().increment(); + System.out.println("[PUB_PATH] In topic dispatch"); destinationStatistics.getEnqueues().increment(); if(isAdvancedNetworkStatisticsEnabled() && context != null && context.isNetworkConnection()) { @@ -792,7 +855,7 @@ protected void dispatch(final ConnectionContext context, Message message) throws return; } synchronized (consumers) { - if (consumers.isEmpty()) { + if (consumers.isEmpty() && sharedDurableSubscriptionMetadataMap.isEmpty()) { onMessageWithNoConsumers(context, message); return; } @@ -807,7 +870,20 @@ protected void dispatch(final ConnectionContext context, Message message) throws msgContext = context.getMessageEvaluationContext(); msgContext.setDestination(destination); msgContext.setMessageReference(message); + System.out.println("[PUB_PATH] Topic dispatching: dispatching"); + for (Subscription c : consumers) { + System.out.println("[PUB_PATH] Topic dispatching: consumer: " + c.toString()); + } + boolean no_consumers = false; + boolean no_shared_consumers = false; if (!dispatchPolicy.dispatch(message, msgContext, consumers)) { + no_consumers = true; + } + // Dispatch to shared + if (!sharedSubscriptionDispatchPolicy.dispatch(message, msgContext, sharedDurableSubscriptionMetadataMap)) { + no_shared_consumers = true; + } + if (no_consumers && no_shared_consumers) { onMessageWithNoConsumers(context, message); } diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/TopicRegion.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/TopicRegion.java index 68b01f4e07a..02b6cde3fb9 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/TopicRegion.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/TopicRegion.java @@ -57,6 +57,8 @@ public class TopicRegion extends AbstractRegion { private static final Logger LOG = LoggerFactory.getLogger(TopicRegion.class); protected final ConcurrentMap durableSubscriptions = new ConcurrentHashMap(); + protected final ConcurrentMap sharedDurableSubscriptions = new ConcurrentHashMap<>(); + protected final ConcurrentMap sharedDurableSubscriptionMetadataMap = new ConcurrentHashMap(); private final LongSequenceGenerator recoveredDurableSubIdGenerator = new LongSequenceGenerator(); private final SessionId recoveredDurableSubSessionId = new SessionId(new ConnectionId("OFFLINE"), recoveredDurableSubIdGenerator.getNextSequenceId()); private boolean keepDurableSubsActive; @@ -120,7 +122,32 @@ public Subscription addConsumer(ConnectionContext context, ConsumerInfo info) th ActiveMQDestination destination = info.getDestination(); if (!destination.isPattern()) { // Make sure the destination is created. - lookup(context, destination,true); + System.out.println("[SUB_PATH] TopicRegion addConsumer: looking up destination"); + lookup(context, destination, true); + System.out.println("[SUB_PATH] TopicRegion addConsumer: done looking up destination"); + } + if (info.isShared()) { + System.out.println("[SUB_PATH] TopicRegion addConsumer shared durable"); + String clientId = context.getClientId(); + String subscriptionName = info.getSubscriptionName(); + DurableTopicSubscription dsub = sharedDurableSubscriptions.get(new SubscriptionKey(clientId, subscriptionName)); + if (dsub != null) { + return dsub; + } + SharedDurableTopicSubscriptionMetadata sharedDurableTopicSubscriptionMetadata = sharedDurableSubscriptionMetadataMap.get(subscriptionName); + if (sharedDurableTopicSubscriptionMetadata == null) { + sharedDurableTopicSubscriptionMetadata = new SharedDurableTopicSubscriptionMetadata(subscriptionName); + sharedDurableSubscriptionMetadataMap.put(subscriptionName, sharedDurableTopicSubscriptionMetadata); + } + super.addConsumer(context, info); + SubscriptionKey key = new SubscriptionKey(clientId, subscriptionName); + dsub = sharedDurableSubscriptions.get(key); + if (durableSubscriptions.get(key) != null) { + throw new JMSException("Shared durable subscription is accidentally added to durableSubscriptions"); + } + dsub.activate(usageManager, context, info, broker); + sharedDurableTopicSubscriptionMetadata.addDurableTopicSubscription(key, dsub); + return dsub; } String clientId = context.getClientId(); String subscriptionName = info.getSubscriptionName(); @@ -185,6 +212,7 @@ public Subscription addConsumer(ConnectionContext context, ConsumerInfo info) th sub.activate(usageManager, context, info, broker); return sub; } else { + System.out.println("[SUB_PATH] in TopicRegion adding non-durable consumer " + info.getSubscriptionName()); return super.addConsumer(context, info); } } @@ -256,6 +284,7 @@ protected List addSubscriptionsForDestination(ConnectionContext co Set dupChecker = new HashSet(rc); TopicMessageStore store = (TopicMessageStore)dest.getMessageStore(); + System.out.println("[SUB_PATH] TopicRegion addSubscriptionsForDestination: " + dest.toString()); // Eagerly recover the durable subscriptions if (store != null) { SubscriptionInfo[] infos = store.getAllSubscriptions(); @@ -336,43 +365,69 @@ protected Subscription createSubscription(ConnectionContext context, ConsumerInf if (AdvisorySupport.isAdvisoryTopic(info.getDestination())) { throw new JMSException("Cannot create a durable subscription for an advisory Topic"); } - SubscriptionKey key = new SubscriptionKey(context.getClientId(), info.getSubscriptionName()); - DurableTopicSubscription sub = durableSubscriptions.get(key); - - if (sub == null) { + if (info.isShared()) { + System.out.println("[SUB_PATH] TopicRegion createSubscription: creating shared durable"); + SubscriptionKey key = new SubscriptionKey(context.getClientId(), info.getSubscriptionName()); + DurableTopicSubscription sub = sharedDurableSubscriptions.get(key); - sub = new DurableTopicSubscription(broker, usageManager, context, info, keepDurableSubsActive); - - if (destination != null && broker.getDestinationPolicy() != null) { - PolicyEntry entry = broker.getDestinationPolicy().getEntryFor(destination); - if (entry != null) { - entry.configure(broker, usageManager, sub); + if (sub == null) { + sub = new DurableTopicSubscription(broker, usageManager, context, info, keepDurableSubsActive); + sub.setShared(true); + if (destination != null && broker.getDestinationPolicy() != null) { + PolicyEntry entry = broker.getDestinationPolicy().getEntryFor(destination); + if (entry != null) { + entry.configure(broker, usageManager, sub); + } } + sharedDurableSubscriptions.put(key, sub); + } else { + throw new JMSException("Durable subscription is already active for clientID: " + + context.getClientId() + " and subscriptionName: " + + info.getSubscriptionName()); } - durableSubscriptions.put(key, sub); + return sub; } else { - throw new JMSException("Durable subscription is already active for clientID: " + - context.getClientId() + " and subscriptionName: " + - info.getSubscriptionName()); + SubscriptionKey key = new SubscriptionKey(context.getClientId(), info.getSubscriptionName()); + DurableTopicSubscription sub = durableSubscriptions.get(key); + + if (sub == null) { + sub = new DurableTopicSubscription(broker, usageManager, context, info, keepDurableSubsActive); + if (destination != null && broker.getDestinationPolicy() != null) { + PolicyEntry entry = broker.getDestinationPolicy().getEntryFor(destination); + if (entry != null) { + entry.configure(broker, usageManager, sub); + } + } + durableSubscriptions.put(key, sub); + } else { + throw new JMSException("Durable subscription is already active for clientID: " + + context.getClientId() + " and subscriptionName: " + + info.getSubscriptionName()); + } + return sub; } - return sub; } - try { - TopicSubscription answer = new TopicSubscription(broker, context, info, usageManager); - // lets configure the subscription depending on the destination - if (destination != null && broker.getDestinationPolicy() != null) { - PolicyEntry entry = broker.getDestinationPolicy().getEntryFor(destination); - if (entry != null) { - entry.configure(broker, usageManager, answer); + // Non Durable path + if (info.isShared()){ + throw new JMSException("Creating shared non-durable subscription not implemented"); + } else { + try { + TopicSubscription answer = new TopicSubscription(broker, context, info, usageManager); + // lets configure the subscription depending on the destination + if (destination != null && broker.getDestinationPolicy() != null) { + PolicyEntry entry = broker.getDestinationPolicy().getEntryFor(destination); + if (entry != null) { + entry.configure(broker, usageManager, answer); + } } + answer.init(); + return answer; + } catch (Exception e) { + LOG.debug("Failed to create TopicSubscription ", e); + JMSException jmsEx = new JMSException("Couldn't create TopicSubscription"); + jmsEx.setLinkedException(e); + throw jmsEx; } - answer.init(); - return answer; - } catch (Exception e) { - LOG.debug("Failed to create TopicSubscription ", e); - JMSException jmsEx = new JMSException("Couldn't create TopicSubscription"); - jmsEx.setLinkedException(e); - throw jmsEx; } } diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SharedSubscriptionDispatchPolicy.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SharedSubscriptionDispatchPolicy.java new file mode 100644 index 00000000000..fc4bba6bcc4 --- /dev/null +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SharedSubscriptionDispatchPolicy.java @@ -0,0 +1,35 @@ +package org.apache.activemq.broker.region.policy; + +import org.apache.activemq.broker.region.MessageReference; +import org.apache.activemq.broker.region.SharedDurableTopicSubscriptionMetadata; +import org.apache.activemq.broker.region.Subscription; +import org.apache.activemq.filter.MessageEvaluationContext; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentMap; + +public class SharedSubscriptionDispatchPolicy { + public boolean dispatch(MessageReference node, MessageEvaluationContext msgContext, ConcurrentMap metadataMap) + throws Exception { + + // Round-robin dispatching + System.out.println("[PUB_PATH] In SharedSubscriptionDispatchPolicy"); + int count = 0; + + for (String key : metadataMap.keySet()) { + System.out.println("[PUB_PATH] SharedSubscriptionDispatchPolicy: subscription name " + key); + SharedDurableTopicSubscriptionMetadata metadata = metadataMap.get(key); + Subscription sub = metadata.getNextDurableTopicSubscription(); + if (!sub.matches(node, msgContext)) { + System.out.println("[PUB_PATH] In SharedSubscriptionDispatchPolicy no match"); + sub.unmatched(node); + continue; + } + sub.add(node); + count++; + } + + return count > 0; + } +} diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SimpleDispatchPolicy.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SimpleDispatchPolicy.java index 7d8e2f073bb..9d2ff35be2c 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SimpleDispatchPolicy.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/policy/SimpleDispatchPolicy.java @@ -33,6 +33,8 @@ public class SimpleDispatchPolicy implements DispatchPolicy { public boolean dispatch(MessageReference node, MessageEvaluationContext msgContext, List consumers) throws Exception { + // Round-robin dispatching + System.out.println("[PUB_PATH] In SimpleDispatchPolicy"); int count = 0; for (Subscription sub : consumers) { // Don't deliver to browsers @@ -44,7 +46,7 @@ public boolean dispatch(MessageReference node, MessageEvaluationContext msgConte sub.unmatched(node); continue; } - + System.out.println("[PUB_PATH] SimpleDispatchPolicy distribute message " + node.toString() + " to " + sub.toString()); sub.add(node); count++; } diff --git a/activemq-client/src/main/java/org/apache/activemq/ActiveMQConnection.java b/activemq-client/src/main/java/org/apache/activemq/ActiveMQConnection.java index a91349b28b2..681ef9de5eb 100644 --- a/activemq-client/src/main/java/org/apache/activemq/ActiveMQConnection.java +++ b/activemq-client/src/main/java/org/apache/activemq/ActiveMQConnection.java @@ -154,7 +154,7 @@ public class ActiveMQConnection implements Connection, TopicConnection, QueueCon private boolean exclusiveConsumer; private boolean alwaysSyncSend; private int closeTimeout = 15000; - private boolean watchTopicAdvisories = true; + private boolean watchTopicAdvisories = false; private long warnAboutUnstartedConnectionTimeout = 500L; private int sendTimeout =0; private boolean sendAcksAsync=true; diff --git a/activemq-client/src/main/java/org/apache/activemq/command/ConsumerInfo.java b/activemq-client/src/main/java/org/apache/activemq/command/ConsumerInfo.java index eec3c5cbd01..21948e512de 100644 --- a/activemq-client/src/main/java/org/apache/activemq/command/ConsumerInfo.java +++ b/activemq-client/src/main/java/org/apache/activemq/command/ConsumerInfo.java @@ -117,6 +117,12 @@ public boolean isDurable() { return subscriptionName != null; } + public boolean isShared() { + // HACK for now since the OpenWire generator is broken + boolean isShared = subscriptionName != null && subscriptionName.equals("test-shared-subscription"); + return subscriptionName != null && subscriptionName.equals("test-shared-subscription"); + } + @Override public byte getDataStructureType() { return DATA_STRUCTURE_TYPE; diff --git a/activemq-client/src/main/java/org/apache/activemq/util/ConsumerThread.java b/activemq-client/src/main/java/org/apache/activemq/util/ConsumerThread.java index a49ff6f5a4d..025f05a3c69 100644 --- a/activemq-client/src/main/java/org/apache/activemq/util/ConsumerThread.java +++ b/activemq-client/src/main/java/org/apache/activemq/util/ConsumerThread.java @@ -40,6 +40,7 @@ public class ConsumerThread extends Thread { boolean running = false; CountDownLatch finished; boolean bytesAsText; + boolean shared; public ConsumerThread(Session session, Destination destination) { this.destination = destination; @@ -54,7 +55,9 @@ public void run() { LOG.info(threadName + " wait until " + messageCount + " messages are consumed"); try { if (durable && destination instanceof Topic) { - consumer = session.createDurableSubscriber((Topic) destination, getName()); + String subscriptionName = shared ? "test-shared-subscription" : getName(); + System.out.println("In consumer thread, the new name is: " + subscriptionName); + consumer = session.createDurableSubscriber((Topic) destination, subscriptionName); } else { consumer = session.createConsumer(destination); } @@ -185,4 +188,8 @@ public boolean isBytesAsText() { public void setBytesAsText(boolean bytesAsText) { this.bytesAsText = bytesAsText; } + + public void setShared(boolean shared) { + this.shared = shared; + } } diff --git a/activemq-console/src/main/java/org/apache/activemq/console/command/ConsumerCommand.java b/activemq-console/src/main/java/org/apache/activemq/console/command/ConsumerCommand.java index 2c8e042400a..cf50c0f5a55 100644 --- a/activemq-console/src/main/java/org/apache/activemq/console/command/ConsumerCommand.java +++ b/activemq-console/src/main/java/org/apache/activemq/console/command/ConsumerCommand.java @@ -43,6 +43,7 @@ public class ConsumerCommand extends AbstractCommand { int ackMode = Session.AUTO_ACKNOWLEDGE; int parallelThreads = 1; boolean bytesAsText; + boolean shared; @Override protected void runTask(List tokens) throws Exception { @@ -54,11 +55,13 @@ protected void runTask(List tokens) throws Exception { ActiveMQConnectionFactory factory = new ActiveMQConnectionFactory(brokerUrl); Connection conn = null; try { + System.out.println("[SUB_PATH] Connecting to " + brokerUrl); conn = factory.createConnection(user, password); if (durable && clientId != null && clientId.length() > 0 && !"null".equals(clientId)) { conn.setClientID(clientId); } conn.start(); + System.out.println("[SUB_PATH] Connected"); CountDownLatch active = new CountDownLatch(parallelThreads); @@ -79,6 +82,7 @@ protected void runTask(List tokens) throws Exception { consumer.setBatchSize(batchSize); consumer.setFinished(active); consumer.setBytesAsText(bytesAsText); + consumer.setShared(shared); consumer.start(); } @@ -216,4 +220,12 @@ public String getName() { public String getOneLineDescription() { return "Receives messages from the broker"; } + + public void setShared(boolean shared) { + this.shared = shared; + } + + public boolean isShared() { + return shared; + } }

UsOM&&UQmO#Muu=uJ2qif+kG+u;&Is0}kP;M9J3xaz zAri60WnTWxpO%fYrpY80-_4tY=#-X@J8s+F{pYUb9KJB^;E0c3IT%_`ySZvUBoC0~+g~vqnCbY@|M^(o->hYklS)0 zQ7=Vy4Tn^UYdGZM|DD*IA1S#kid*`z*7B7q@E(RQ(pv1U%1NB#xpK~m`&=s~V4ikr zr2(MYahIhxjK&!_uhiz??;e)8iB01mG#d!zTHkwv%;F+J?E@;~;`~lX(`ds36L3+H zNvy|f>X-LLk@zPFE@df3WX#B{wNIB;q+z&GjLqh+-CriokG=izKN05seDhZpyZXU8 zbKs>g0j%H14Sf&4r-8y&Igx3l`>dq?*_k4({Su0(psfy@PetY*X$~IxBpSgq2!ADp z>g>J5`BCy;V>%c~lOGmT8S@YsAEsMOp+pAhlNXG7HS zHEkFzL^V6fJ+K_(A1QCOcKAqPAY+8Rz17dzp}rK?327swJ#S4J;eSgj6+64YwmH2T zm$hj_0b99Y-Q&To(@m^lB|>+!(pUDcnwfWdE>3{uK5kT>KQLUEv17@bEOc-m?e7ZZ z_>yE6Rv7@>Qb;I}k$9p^UKH`C9Hl`9l?V(R9ZDiPZw%YA<`~j}DUT~FkijTy1vr`6 zz6gg}xA_D0NWMD#zs2JtWz%j4T+*MtNd^9kwJ>#<7w_j9oyV@Y$IjfdPm?pZ+!phN z)m$mkZ7nQ2CxJwRu%A+5>7b*UCTt!dcOtYbAqe9P)vxj1~-J$RZh{^Os%W6U&=Iv9w^HauX}h{dt8Z{0TH z-;)Ib?3razAk(q)1dD2%$P-*dW5>!oOmcupmsFmCM*cwTA;*vKWI1>d4# zosfgGbSca{SY+m0uWx(`-{Dd?H3_KVrrX`{OQBVJOaE>DCGDV+K=RZmc|szDaYlH7 zWC~d#A=vbqU&+4UR1?)v2r0_0br_`|*6ZNx2zoZ`MVcG(p70Zw0;m%kB~*e%nPF@4 z&G_aVA9avAK~V!(T<9Amx}Geg(T$Q~zUO4&N~zRFsW}adQ>+QEHzn4oz@430dSPIb zo8YBrXlK*ultSX6InJT>%X9wDW4$gFb`+L;_s_1Rlu9RQ#x!Nih)!y}*Dm8yuxD?* z{Lhr`(1sb7^ssVZl=g#6a8wJ+(w&IoWy$6j-H|Pby(KXj)~pDXMm~6J6^%*;zALoE zEc%N=jg5?w^2Fj0*DQ{a-MSc0Rd{dLnCxKo$rqB3?K(U;7A%(6x!QQOQm7gk25y~T zIYKy<)jP9xnWwRxy~Vle0X3>p*G#-gR=S!GYwtwCU)Fm!i|4NhfRr`nJyBg5TJ{ZUSTYL^8zsdWaEK52*SVl@!;eX3HmZj_l zy*7&WZs~Meb%7g38ya+DB~kh#S-p704M8U6d8zERH-T)ub{n}Gbc!8LtgThwp6JA_ z!S8p{Ix2jOj)CE;x=P*H==OLRb7jf(|AYy8QNL_S@wN% z(jEU0@qFf4L_%%j+sW8-b;gFs%J-Ic$A>nuTUev1tI`MD3{~aMQu|NNk7_;mh<0Gj zO?_~u-~sGi`5!3c9}}d;G(N(V5F*y+U?|-_s64;<`E#+w4*59aGT^>+9`+MnwAN!Q z`SiSvIS|u-1cb2cM}ho;tq@D2*mSG#A$IOg24AyP8k-jD0*f@u*2|i621pjh+nfRjGfxNBG^6_6%QqRvC%xY zc^UVey=10e0xCo_nccGh;FTXV+E#wHC%FYeH)P{4fPTqMVs^rt7D$OTuE{x8Y?5*j zlrC5|0B`*J!(F0#`}xT@a1;*a=3@Ak+PJsb5+g~xW<)rt_a`@}w{QjTGV1 zh%(PI2MPBkHRX?Ex0R&P?@PE-mm>|kOXK(kkwxCIjHVRgw9P-IYjAGHZQciDR4hHLrT!eLqK* zY=GfNVZ~MmO1E{NHN&f4<=F4|r`z+^bY-1Z-o=G1pf2(L>S-oeI6$WMk<(iQB5~>u z1X2?K<@RnbtW=j0g91K4=%uESi2c)~++6%E{T&u*VjgJn``Qfq#8?pd8>j*Wg4E&Un7T5NnA-rd00-Vg+NxUm%oH(f(MTl$flsb0m#Pn@ne9`X zLDz|=wo*#aJ1_14*&!G6UFr$3Glgq5O>V%zq)#~C&&XrPXC{{DQ=1sJWgwjMoSNCZ z3jz0kBR>H2G^0LXm^~sQHXH8(-=HV370W&Wt?2YIK?e7)*D*G9&cL>aKDG zJMYWli9w!aJdxpRE4UY=)A=@mHWr}x+HJM`jY@N0PuA{i806@<)%ClHSr`r&k2%}P zoZ)ScaEo0b7Xa4G)oF=MhyPC`2snD;sezi?gGBsS;QZ(hPC!~k$ch9H&{S>sj%UC4 z2u{vXoG@L|G(nqm+#iW}KZb9>@-2*Z`sGp8G*6hOt{ez z(-1gBNxSNC?C+2w5?CBB&|(lx8UN+p2HEIWcAQ7}7B3(iPZenC=ndv-Z<`~zSiaEt zUrQ~}ouxYEZh022%aBZ>svHgPmiwEI5nmS_L|~ZVyK`o{ug6QGw&k_kAqwL0G#T>} zw$dE;C;z+_Aq;;jz)0w0_wY$wFf>8GOO!xS+$xd1nNpKkQqNSwGAk9EEC(EDrJ&N4 zn1y-yx$mdWCRy9#*VW^G-J0xb8$&)EOSC@89&$5{@|sAVhh!ex{x9hE8Bq~a`XEv7 zFBAgCPUC|K5#SPLAmc{raP8vAbL$ZRn^umX#56OV>-m7AXOZ7whb%rWw=$}C?-I)Hm92_#R+PLs19*5MZh+0O zOSp-G(EQK`@@-2L910QsAQ7WwaVn}?e@1Lg@g0sLeTd+>MiV*PnCGr6M?^Pn4qfm| zLCr-5a62COlhEaOolQ|_4S^W1nFKO*?c8N+SPL9=C2<|6Ahvc2wQ!*&-{qhe$E|yT zV~30Kow`t-J|7^eb7bJ(+b3~Rmq28S))iz_*TPf9rt`;_%*vdXFWfb-F8 z1P9Kqv@kgo^t+R`RcXv-+l`~>``@9isk$rsCvt&x- ztfG#AZX%OW)#0oc$Sp1*_Ujs%0nG4BA%%quQFSt)Wg)%RAtnSs4<$U#Hgs%h*&<^L zh|RXftDH{ta{ww*@Y5ia{s+7 zwC0u%rfM;~MtvLN3%P9m+INi~<;i_qFyxMPUmyKZqpagwfc2{xDKH%zacz%gzG3oL zS5;BIMyel6>s-4J)u3)&M{h_xG(z&OTI!`LKgq!SHzMHNyQQ#zgLP@0WU>3lS%fgs zW~Tg*Na{7wGaRS=I8%C>JpbEu@ZkFA4*4%<46-;`6mYa>I+FzIIQ+XCHhp(6ic-J) zS$pCeapIJzsk_^`{F>004!l(?4fOOg$QqZI8A_|!?&lP2_w$&{!9RvNq_?9&Ky+m5 zFM<=PWA{BNe%q)!u@79{y%M7hjC_uP-u-gaK(bv3NE%(VQ8DT`g94jSg<>(zT}Y6L zqg#%qKv2OObXzkKs2wl^1Y|)bSnL~gQ!_D#8z4FYOpoQ}p+byOdZELz+knD%MiBTC8EJ!Mi(8cr8WaU z3WIHK?~B!n61ADaGaPiHjw=5s+epZC7?VRJ#&V0WAKIXHY%A`mrP2)}l=U6ejX4Ko z+Zd+eU<1kQBTw0Rz{mbpfWBjh1v1$NI_Y$~V(PPo6KlP=IeTC^xh(vU-3@F~U-O1+ zBu-qos_1(sVOMap$f@_F6c~vhoYyDWMTH9@i5?9QYqQ3V)}S}QDmD0UunCy|H`ml( z2GxWeV7f{CB0Ub^d2tPO!YU8cpsDYMV{UzN`xv`amd=cteaiKipfQ6PeQ?cJ`HMFa z8lW}_o~_p&Nm{dd)Z9J5j%@`WCb%30JQ#TF@9XwyP=!OF1|xr!ntba%KLz0B!G?M& zgfzwxGUv8hAP!FAx(hr9j^A{`75!h_+$?lHrN>qWqJj5cbhSnAkQy~O3pI>)p)`5&BMy&i~&5rtjc z`&``57&%_9@~zoUHn{D+fi#%i$m4UV<2(rOm6d?Nl3& zJvX9&9kA+@6W>qw4-T}ex6dSOjBC)D{fmqJRkv&D&D40(tJ(aj)ZN|_J{ItX@|L=1 z73j$im9mXH=H@9I#uC0)GVGN0Zdw$Gwo|MwaX6MgWH<96DrL^Rv%q@1Q^zqwjQCV3 z9fx4t9&NW3CT)qp}BdU9*eF@iDkN&13tBZFn1$WBr)P!o=ePd7jTZ9h<~W?qY2*Tyt)aS zH{DF9kK0SP*yJi~?C#+-`qjR`*ZqZl|GOEb;3r&2Nxe~PI^A^9+aBRGNM$CG`RG$L z+Fe<^$+TxGuD-eCAcICvemJIcHD(MNx0;UOL9X179ca*VT9ej^adaIsyhyw_Nz4Wd z6R_w*8Qza-BIYgq4_~RjGw`Q46@}JYu%J8(Ae=4$0U#ui{kx^m#Yu0k8-dh*9t?I^ z!G;hNnBBu)@YtL*d#+IO6?h7n%!SxnHLViYjXF+pPT;7fJ#0|X-}J!Z9=YW5khHe{ z14JwtlOLO45qE8zQgAJCpLOvTVD=Ucv!4~juaRsfG#*8MFRJS;v3`wqd5BJt1{74? zUq*3;D>Pley&iE3@U=Z$6Ms}nR-(u}?xAwUPF^dE{wRGv=Lys!S85o-pEXeW#G@B` zughgWhjd~mp-zS2uz;o>+i0XH~!N*fI-SJ$Tg9JpW{CacQrBTo=N5sS)6?4)$LG z@ujK(7K7I!o;7m1N$8uWklt5NawAK%)%+I=f0Lm(?WO?OWd|9{=F;g?wCv7eQfxDt zveSC;vatQD$S-UHl7+sf1Cce)QsZaLpopNKRsk!L=edX%84@TITli0GU8Y) zJcCsPJHHbssF!U6g00(eAK>j&w?Lm(b$8nuHi5Srr9)OqpMv0wn&W$201My<7TU~< z763&|X7@1>568+EItgzUAaf4NP1WkGT9p8qdj?GaGVB<1(s))BRM4htEk9v($6G+} zDf$hlax~BknVa!AUC);9>8MEAnBJ}wkff5eQ?LZsf1*J_ zy3FR1A}Jil7>8vhjF^c1!X+h8*IFdnjfo!GKdcE`#riP9ipaQE!K*D61X?G2*~5m& zql%I00#5$%P=^*ld2*eDFOkc1VZn<_a+^a zP<&V1&-0t-oZtJ-IrF}A{yM{q!!V9cSZm$uzQ5P?xvn{{^BPX{R&&)+pG)e~4X256 zR^%6wOUQ&0Fu7@Z+(!kSmUcQF$HBde4ss_vY!efk9)yO1f@Y+d9zmm6vskXs3SBme zFWs}$ZMDuQw=_H|ok!?u!7)6{ZbtVBm%6_F)wFn-y{>pYi{0(RtxRfm$!p<$vlBsa zVo-AdZTXeBDt6Bgsdbq7&y4oTMy-teN==fzxr{LjQGzRF@6t7TX&A>{R#j*$tUxTA znNd>9)OQ?qno4LkKY;rK$O!rvAiUh*Jb6BJu@NCAJ48rHf8lyB>M@w1e7>@Ho$GtH zeuIvXEo14M)SH_RWIYymME24MP26CUi4_%y)SqcjS53uTv|;EC?#z2G%J-S2?sNqG z%w}%j&(yWYodWljg}EgU2!bidAjC`Npa&F7IXf1`=6#r2=lkvbjA`JB=c!eB0X8KE zu`b9(Q9}ja{nwY}q>|+L)#qu1IQsBt3bxWv3jyh&USaAp%I2$VvZORb0YhFf^h3Ze zL1dqETteGxXWw<}2B1xd%U@&Wg^C$nF3Yl3R85h(U)r={^~hU_n4Ctiss|R!isZ{oDb}6hEC? zhUuY9yN7cikLN8*CW90>>+-`lvP;Utv`{{iKRd3(6 zKhLk64^vI;y*q!+;W`<4vXn>>wKh!}t?p%UB}v1nG9;HeuGU>_i+F{!?%&?N&sVog zxH~%Ie;eGW8?e1(@a6>oQcV!s8C-LlN|mC(gr^jctPq+0#KA)bV&b2q%X6S}^}3T^ z6KMbIE#j$ z(oNA;_30;APblB)i`;f~`W^L*M510-vakf?rfB}k8*vw6l8`9gFn?FT!br3T+mj1z zt+{gM+)Mc;U5d%eG);VkKxJyP zz`THR`#bm3J8_WhBWndvd3ZIPng()Ch+v1*K0(%BO%iM-@awC(YOW=wuBf7y<9$i z1-){78&Q_1d?yHibx#Ur(^ga@FIJ2Aw{`U0n9@f(t3Q->tfJ&)m$&Z>2{*aN1?|ko zgkC)YMTKRBr7EG?h`why<@wb>nFaB)PAyT{mFd#b`PJ;qGL}_lj%82rHw+|ZoX`5) z5f6#dvy5(r2Fp0gF{ke{88?H>@XMlC%{EN(vT4>FCa1llavnCWw7-Lg-5pM+mTBZ2 z2!QpraHqjT+1;KGH$08UG72BPmAqCk%a-o-Ze;NKOF&c2xogeSi#p7;*3e#fmM6UV zw4l5vE(PM!I=5<4{2KFbIdf)^PvumyPWZE4rlfl8P9C@$_)9w?+Fv;%5Lx_?s}7e8 z{ir(l`4PFzemff%qNZOl7P&K+vh8l|tF7;td%wo)*mOn>K|fO}}*dPyblAUxxmY>V;UkC6O`@a@vr zRlo2OAj++vsliF*`*bu-RyYtCR)$mxWDM#ye5Og55K!b@DDN{e2j@#GnX4AQPl-&*PI7Et#v5mlq=~jYUnF$ z+e;|}Vpp7YeZDsRy-^E)0Xp>fjk4RXXHt*yx@B<7AHb>8?@Q0sAWbF2TZ`*phxX3>lKoL5scVq0B;{%1ZpCzo!>$^Sp9EQE)a$|{T?orm;TD?1MBw>+RX7U~R z(tV(BG9NlLQFC;CNYoYnME*kWeZ%7W%P+2RLpT#<7xUX{WS!IbOHlqh;Xs;7*0La9 zRX}Cv?oF4Dvz8x>=N*fF>uHx~Ep(xUX4`sE^#X=_LW2jDYP0`+?1nPo%#fkA&qzq^+WoI~=VFqyR ze56yGi3#H*=4pCwCY1E6f+}1by$iueeHSI7@p-AHtV`nu!POE~$cl0LB-Zl0Nwr~e znK}0a!DthZlB4|@F`%9ncmYGFS2Y)xohUJ#cRr3f`p)#BBBTZ@5?MsxCbjt%aY`7? z$D1VRCXiZ_X&}b{PE)8MXo7V(WjYU+7NcENU%-^nBPVT4Q(Ifb@V{d+jIm;^AE{9E zn@NjpsHKh!0G`ru;f~t5yxpJh+=&&$w&gUD!tO%FxRguVVK>d~ZX1#ca#l(}9k)n5 zR~Yuy#QF-bP=dP;Ok04K)hM|t&&#iQbUQpB7!;EPK+(6fHHI~3pMkdL`GG24NoA9@ zD8G2CZT;8BB0Dv32~fFV&Q)By4yqClOMJysAqjGn9fs+c8z~SvjO}`#R$&2^`7~4- z6)N1+CAi~ho+=Ld1ZLfQSPV?ZDA4-Ya{oyt7mE^LI@aWhu=zDdUaCh3TITw!+0?*T z*t?Z(>yy?$r&T)v4>#f4;5GWdqWd=P3BApguazO5+kq0@rf9ECoOOio?;!-kCRdrC zPo}yD!q{n26TqcZ0p&^~zj+-EFkhLzQu!^-`$UuK`>WCAo)IbB2~2W8a>M>}C$`s; zk#8zxr*O3mZi-Rm{Q5YtzE*Ms|Hb!l+Iz&Hse(Uh2N`|LnWhiSG+mh4N z6aOBDuJ(F8Ee)J6)JAR<=lhXw^?I~JDS z1}q$VNAQbJ@}&+poRVt2zOy%q`<$weB$cf826-LzG-s8R{CJHzd|GU;3dzkqY^y8q zycFTU-c6bPGVu{y8t>88>yjcI%OVqvTGGMyjD*#vz*m%wzH1F{>sf3G#gZa7DmZm) zgOs9C+gwRP=jIg-Q3@krz0$1P{oeIJ_0}lr+Risu7X$JP5j;hEH&qQ1SVE+j`eN}j z=yp;@PV_4Qz61#dWd)XH&EuEL?=9H)wQp(aJWI|LKgHqYEHj(?gj%!vT%vDk^Tcoo zwfB?^e5mdgMnqVRb*X9>X?)a9dpU@`S^S!#+A;MiIPnwydS`nCuow5kPj0#YS9$z zE0yAOk0AN`c`OIZWHF%_TUMoO4y0Ark&Mw+$=K?ta~MNZvey7|bza6UL%~N%VW)Se z&T)$6k5}XDV38PEmP1Fyi`>hcZbkDqf|lP=@z?vdb>0`W@n;2ThktG6SFI+6iER3gSQuTqjD(fSECg;DM1j~&ZQmB`bZl1 zimy7|&+U#WM?$r#uU3a;JEgP`<(#_EgJt{nzv1zM2F6+HcF&FV7>h>j>@2K)}C2(oVmlZ9?sU{ox ze`c`UlL!j4L*gV$mU8M*2ZLE?#oc&>VX1fxutCou9511 zPjuskJ5upVv~mu{ZT3mCY$dY#hs4ln84@fWK<6LAHhrscE>$6!qOVsd0Q;@(<3#6* z{QUZSwElOXXKb~Rqp{2E``e$WD{x8f=tjl0^O4Jh`YATn`mU52tL1Mv{|kTiiL?D{ zvOm^Wl4Hn8c(vVVK1 zuBk;4Uh-Jc54a6WKk%fUVvlEvMH z{gPPdk$@6MOf>z4&g&x}OCxQ8%yThF&j&^<@CH+h2SkMIOs~&-IWP4&>cgMoe}KD_-JZ#R?-{85jh?|IG_6+bS)SX|iwvh9 z|9!eQwVM0Tj((5z();U-OY=Rf1uzO1 z8y=@8J4E4rBmRmlyq3_ep;o|$w?0WmM`cSIMi)l=F-i=bvem{^nE^-?Abw?9aNrhG z16FKtsR6;u7GR>_J~;xjYwg%G&1w|_CwK1F@2T*)@h@6jN4h~Bgfa+g$6jZQJiEbm zW!?L9&e{Q`adWGI&bIqwyFlt$P$)D2^pmU$?tZnA`kC8twG~J(oLV4@eWgDyPS@H% zj4T*SR|dlujN=P-1!-n(v#YJOj+Atc%VJ!*ZbFODcBBQY%1!&C)n{6|bbL+X6(Ks{ zxOHy4>s$KaB($D(I9gfuCr-hQ+Fw*nABo!@-=MN6h!e4Ne|xw zU{7Cqva>WH`CBsBrStc(Z%Jcs^U|190-~+5IafMEZaO0I5lMOwqy)QM*{06B73zanac>ny%aO$(*a3UZlQY z>VR?*F+S@6;*#OhKXGGqosMpftY;%t(^D@o71bw1P{RU>dbMP{)BV;UKW>F^cwV5W z(ZdvS_W9BBC*#gis;Z^TM1xxV)AFM6&?33{N36Zo6+;?7bLHs#QOlVf^NfoG+*iR6 zi_0~>d2G@br1VtEWuC4vx8JpY-b_U|YX-XUPh>D{c4V6m;}n_{!N-@zT1Jt9RarX@ z>LeAIT`aiqMw7-ujCM@I!TeBIzj9ik9e8DRUH>8pei)l4gw!Jts2tCi-v|guhH%h% zIUCSaGL=$TC^i^lAG9Yy4?kOIX4W&=$x8-3Eh%@Z%+o%LS5U9pqpB~@d#6Bc*kEC2&TOo;fpPm;EAPxu}PJ$6heF-8 zn-#QXn;1_&)S+C{yxPwd^mmOXm5g$Wdc7(uVCz5+<&b7N^jOdg$1WFehlAhqB?}s?)9D;00Ept0&QL0#_sIlbubAmyT<-= z-t0UV#`_?P-n%@oP346rFajtRKaaAx3;k5vd4G;v#oDz*#ndt0=_%R``nHozpp z9iaBC@;xH1yx$~sX+Y`ckfd!VM#dV|%PyU-4bwj!;(W1hW;SY|PpYJo8D*}dHTf`C zF8R?5Fd6GtIVDQQuHIxbNN{1>30@3gsY>6r<-=^SkFi|6Ock9KE85f4o3**jLsMTq zzaMJn^N5ppdFX|&w{+!WEDOKq>Uu?)6DgkA<=e#vVe5WeYv8#J@IZFmj}6saPly%@ zc3}HxSI2^V{^H>j4U$Gw=UCJTFYG}`J4*pK88bk#gg5^tdL5T}saRI)P9(>|HL6LR zh92YHBL^ew=)!G3!W*Y<-tDn4Jzx3Hvs8pc?1T9$RH_f8v5G&f?Njf=Re1~v``5U@ zI-bxF(C@I-rv2+8_=P$gLlNUWkJ8K7IzX>3Inbqqv>7i;oqCY2-hG`* z6dD)5r74SouOD3Fr88nGNt?67rKRtZJIq_iykot(CI?y4x(fIe=w1t&ZKx5Yuh3a> z{+%0@RzP;gr9nz_p>@Zg{7|VAeK)M|4YovQ_v*9kuBD@Vv=O zsrg&x*ISscWq!TM?%q4F7Oixc#k0^w3a{ls52 zCkJMmR$lUfa}Gn&b_)TN$RqINR|#BSG8KpYK#1v!`&aXmp0eCdYV(ojS^mQ@$0%4uvk{_FBx1&A}cM*$gX*=&4 zu!k)ww;r>9PXGK`|Md&{occ#j`-mfD-t(s^7Zbajjo>&Qx{$#!o zaZuD~RlSJHA*BA%J3gA5hnhfm+SidEP{F5(fPLYhE!mC86p3t4-0+(WhCWE1ZQjW& zJh&9nh>@Iq>p?Ej*8UUv;2dY*-8kXF8@&aTYtEA!5pU1^;SX3`dND?mY0995U?6eO zy=>S;ObK*qA5SKUF@pR!nJ_z&&1kH|t0*vr$L1at#?n${HENg*W(Z8mR^EzAoOi=1TEn@ad?BQ2g9{{uXA|U*21jHdW}4t z7Zx6QpA2+Octs7CO3y#SOB8!V)=S&(_og)$!=)tU_BvhVU!wa%IOC_@p!hDFf9-qv zi}xSUI#BIDjqt-m9fp2v;=_rfZfbLBK$jEX zjGwsP>dHh(1DH7VN^z(%3iL3=PHQ_iGO1p$`WqaOrp9TwY33{60mvwpF5yMCRCh1wp6kp9+^~iH}hM-@&yWA z4jftBH)9iOjEUhJ`z!K+yXoOcduB2CoV#zt7u;>G&Ze7v!EhDThp+L{(Z^NEq}BI? zXk*LywdDYoli+F{KEmm~&2$*$;hmlAj-0tA`+Zb!^Nabi5$JjAbCXI)gk-E6v182N z-EuG5vpLz~P>^`Jx1yrr90K+|}keA7IRGQR27_iqeXjP2qw>myZm)V?*p zQ`YD5W@R+~=8|edYxDJZt4X=pf*z;e7E;5CTJQ2agC9ghTUnzGAF;(a8M!>gl5$+} zwT5ACtc1lkJaOW4Iv4Se%!kvH&vX7kHo-scD-5O6yrS_d8PC|evidl&nO|6M#U&pb zImx;6xs(4;bF8$?|GMh*t=Al4hz>@ocg&(>ifY)Gj&WsdalD-}`!+^S_~pBD$Bpov z{Sn_L>Wn8JZ8k))E2}z)M|W+EjSn)%_3{7eKK>q* z0@~Id4F9-qees=X%-o@}q#Z+)$I5``orH5$=DIArr}d)_M!3F6F6J)wNV#BF0NUF4 zGomd?a!yRoBfrh4_e@MaR9pdn7qPJ`8X>dRaR-s~jBi6P^Pnv$$`C0*FMa%|EpjDt zq0%LX1T8Q>exzR4YSfDwS~W4hBb47}?9HZ~9YyWKY0QETUUC@u@F;nCLF540^t!;C z=wMB+UJJYe8}1rS=4qbE(UFNTx&C`h{8NL>T?-+`{k#WAF~1ww2VhUf<3XCEL07)#+Bk&;~F2Gd)0XD)9EzSxJV6z#-_-gXjFF?C`X<{f+1HGVHI8ujhT`O!Pc z9sKdEG#Ey&^8W((*t=nXI^%r!x4r&rO!pA#LU;4Z3B2VXF9l)&Jb!y=fY zg4Sq!mTb=e=3n0zgFM;Ic7IJ>VQ)QNIMX9gX7Av(qM!#RUiDs#lC0G`8cJndNIm~I zPCc~=0&DlTF%;~FSH9w_7z;kbyevPGXWCH!NS#&nWfDy99mMs#(1V|}2H<+sD+`6| zB_FgSfU87$pq}YjWKlai^?FY%D~Ulni>qlJMHIaXj*u3OH%LI{JF)map>YNP(cfM> zsX@H;9gbDd_>eGp{c>)t(^y3jK)9q)J7?Ef-i>)t4<(qdM^%};{u()=P5&f71(}C^ zc5clP`+f({f4H{pCG~z(ax!YV3~R6%@v@NoOTat$#k-OJ?N5#ULU&W+ZwTyj{tK!P z1+k2ovP<`hzP_%Z9Z5F9+5#HQz{>Yr(lN%X74ZG9tt)}XN7Ei&utyw*``WV28)XU@ z#lP{gj91DiAGVQMT>b@;<-QALAKJ?&1^Yi~^o+TVB;7fPR1%O1JMKGC+P;?$b7eh!P{qu^zVKbOxzVIeZ3_Ew_zfNlM}L7X z)W#G4e-PCI_U3zZ28-shp3y)5u*V+xw(gg!ZU6Ba)it~*^ofUeu~Jzr!|HRyzsXrL z^b1g2(Ka{!*#mt%jzle*ud3FOIWD}Ijr7WW8xf8}z#72u(O0zJ7%S(IU;T=~d1Huo ztMosx3fKR@Do#KGKC_QKe3W8VTYK-xl^{g&x)HVx(I;v-Q7lpx#;eUz+IMIivG-P! zg@vuvjK0+0Y_DreVqV&Ol7Spdxa%c?H;8h;JEqp(q)B^~d%O4<-1=4hKM$#(+`R9K z;cew=WLK0Q{*73q(p4Zz;tkF+;|0AA461fnCaHcX}(7|C1sPw|Nd@vIf@!o6|h!q0xzZ z_smzFF#p<|rhe3(@_s(KU(1R9haIklPyLG>{$loD`%(XVzqJXIzWGNUzc(LB>mgxY zYXJKq8iB}nUsb}{_B4N1%an|ldFO_8@VGtq!bo_itlqu`cyXv2?e)Z|7#*lhe&e@@ zaIk$&z8KRx^vj~1<;e(JGz+%&mXzGW%k$%yQ9tN8|-lC z1OVi;mY-EWf>S@6bQvI~;yvS|;mb+I3&!RACFjGsQwwa})&79zwrZttkasU{r|dkR zJV^nhQp@xRrDLwI&bx!WgIFbWH?QsfE1}?5UfI9QRX|<>C0K+(^h8t$rXVZX!|{Wi z(PZhXSTlb$yV6>3P~l*h|E5M0ym*k;CJo~4?nxz+&JpSjXH{9;`YG3qQHeSp{VuI#K~ z8MKimEONh=mfR1mJWJ9{=xe$d_=m8?$TuY|k2B$Xangq(`Wd{qN*j~qcL*R~`HtQ9 zn&R(>BY^*p3q?;})>2T07;&nx6lNk>$9f`@N?j$bafMUV=ihTUCT|W|I(awm zZZ$Ca19e&0P}kGGpJ#pRL+pGZuWk05)HowTo%@qk-PDK%T;?*aMxpKZd^TD7z4Ow%jwYbHK~=R0>F7qOvm;;*l#(pyp(#% zMA8G)8Rb~ykADjBJmY?-5ZX$*O)a*vIp>b5 zQKTo+#{X?|yc+l^EnG>p-jM~FCx7s7-1yGFaO120#*M}Q&v0YRnrpAD%>R#YV~nf( z!2gs{p1%y%^V~~4n<}AX6-N zGxR22+Tq26u|jPlot0ZkSnr23x2qEPRABun3dlRKgRva2^U;^D@z+-z``FP5s@^@B zB;(}8_Y&#X`IyX)#5Y9IvLQ8jld)<7CbuW)J+h3IEIL$Rnx^U^|m9cRygE z&g$pJxgFl7u=9uC9pyVVi5FpRlahKB2sJgh+|=MbIh^w1YcU}PwM>ziVzcFzIgTS> z_fiys<=+BX08XF2n#$JBu-~nT$D31J3j}Rj0j1i_TO4au;ZG^$ii=Xb1~3gLbjRwQ zQqeUi0lDCf+G9p=X~OMg>%5g2SFtmQNkyVq1ZbZgULXiy8wT!n@Q}L z3Wu8)8VrTWA@o(9fizfc_L+J|mAZM>_>c%uU|Y0aA#@OXV`+UMn6E0o&oiM&_h=#t z7>XsY&wtvW%DLLVPPL*eV~{DyZ(;&er^*OAAg!Tv%5!Na6+ljgg+^JwSogMw>r7=FQSYE=#Pcd@$aMxp-`@5W;@ooQJ_7Cg`6Q*R* zUc#>Q*9f?+^^bV?LBWw#*kO6q=z1EA5^WkkCb{*ody@S*r{U6g!zWljHp|)N?-%cm zO;*yjBDFTOofE@KYCc#PsSbH<0*UHqyrQu5WbCbgsS*IrLNPb_-SN2eo6G%KbcZaf zXwnZy?!YEl2r`WKn=g)g z2XOjKd(>L!G!Ck6=E%vIkU9$H&Vg1#gZ2bc4D~ADP?TL1(?XN3z_EM}(I+P8hv(Vw zy>+E1$q;I#)9c$_%K(PO2=y<8D5DACyMMrc)ux?CMl=Y_^GyLGBx0|z!ffj1_*OZL zbKcM6;xIEnnjG%e2uN%`HeU5rX-`nyH7||^RKetNq}TL*LjBX8+79c_&^2E|)Vob8 zTpLTL?>FBq!uK=kMoFCbY_U(TG86Oh>vot()NX58Eb@0*Jk0%uS23wH-V0pGr@DkeS`Up1Y)Y?cRKGj;Yd$Az^Y zUzbDOlfVtFwMK1J_;=V{rTQ}GD!atmmn3^e2AI0#&Ns9b0I7Mc3A-TGNbK1I3g85A zWM=Q`vThl<*{%h^Yxs9)m% z0EM9ymHQBc780+|K-Rm<87}~mWsO?ufZVeV1u2pFVtY6TC)DCTliGCF{IYo30=*My z{(ZDclig;R=6VYsfRPJ$o_E-L`~+x8?$dA?8>t0ypt8r7X z2#qL{=%^@eh^d#aFvBZ($W%WKfLo1Isg~0G_AwT6CCW*SEI3vl*2MHB17N=Z^twM_ z#-pP>(qB~s%Jn#C#T|>9i(+L#V&C1`(z?XWIduSjQQIL^8%tt@aU!+UV) z_q@I`SIh5wB8~cP-%fU>yi(_>M)f|282WHgsioF@Up!rw0h@nU?G()shhtz7d%d@v zriqH8d^$|IPon;gpkCfXwCvkZBzJtXLWGmtuY{D*G3|sS>T?c-gjd_7Ei7eTI=QPt zBHOb5*a`Ps-L!eXZsrSxEWP;sY@V|EnZnI9WsM@2?9U1Ctp`){>YMz+_!!j!kN2#H zHtc9Z{U~#MNXZp~sd>C&n&Rc@C+94!u`SuI(Cz_Mi-L3~v@kx;tY6iFKjdYzH;p4N zE%LfxbkMO3kfyLlmLgm%i@v;s*73C#L8{CN-mdZwE@s#W^O`qB7r}JS z0iMUMq2TK3W8hMkb3Wi)Qr66r8}<8KXt)(Yy{T&P!75%n4?JNR`?OeKlruLXPQE=Z z|2qBNmAy9Z_)FAm%+0H@ko~>d#cU09Swu|=V9`>7b0wq?G9z~R8*=2%60kM>#?jcw zVPSmqaY)H-xL+rg$tIOj!}1nn<(n`S;>vaPue?ebh^Fhys;rjAq>gH#)`pn(X&qDV z9dHGtM?uu8Nrm9gh;SenBfpa;o;0tUfn{CkNK}E%t3hiTxO9X!9npDyN;9uGo^4T_ zs_Y6E9&fP&?zjm1C{+)76&hq$cWu-=osh5Asd0Q3b=B}V0qa~6! zeIfiat>t&5rYu$Li8UK?uj^9grKge}!bt0R)!IFSkHVzKez>FesdCQTfl?W)8!h)L zKVw){`bJb?%R$u<<-_*IH(m50hC)Nv1+``JqR)Vnkr7i0B=^btzdNq|aoFx_gIeCW zvV5&cCHY`}D*1Sgwq-ASt7GL*5eQaE^L;%H$qDA!hGQqOHE$gzwuDPTOTCjC_b1e0}tGglX z&9q;E&F^`(G9_?&9F5KV_Ty{>*|aP0`_5riSiLBm=TQYc&7|QB5)qMCl_MqJIa`0=+WN+f*=Gdmfz_2svN_f z>A~^gr2c_H`;6mK?0_T!2x8g^7aPujy0~ssL>La~AoWR59&uZAOY7_zG;gOB0Me|f@h)X+z9&xA$IXuU~J9Wdn zyCiQxzzYhwn9jPIH(R^XJc8HbBXxggI+BPNyhu}%FJ*-XRT6qWfDRfYmHZPYPVR5` z4bm({`R%Q+R@bA3@Zskja?h7BMV&EH1{U#O?w~glR8l3CERL#WMyQTYYd(=ij^4d| zk~seKOXjhON~Rchq=!qGD*w>dae)}OgsXgRY~yLqWy7&wV4C>?1r3L{&aN*`cC}*W z$=`<&^zM-Jb=5E+1jUZHkhlfNk1dQDdw?Yu5l|xBv`f zDu|ay0?{VwW}_SAgkN~U&pctYKw_Boys2ze01$`K>9C^swh^X*%zM!8?a zzdvY^ibtIq2!P~}c{9^b=BZqta{?g`V8V&<*w@j(qSo4dBoLY59sG54v#4iSVl^kO zu(H6J4U&{!kH{T8&Q&?4KIX+HD_W=zC{N#?mc*QeS9~5Wzn!1JROpO&)Fa%L;s*1` z$34F)r8>$UA^G>yMAS!OKa0i^Af$-p+DIdm9Zl2ofi>uZ+fy_}Ek5BB&t!E3mR}pK z_tKduKVF@5IUtIvWqzJE(t^MOa)MJ8AqV-@(~yI|=X7^SxpE>BN0<=~sr4z?rl@64fw7S*sDgLc5UkL$7-Ui*T5Iqq{fc}d$K6GG#>lEnTImKZ>m%ICyLbJP5FDAOoB~PQq z?DXutkBG+8F4xU3{F2ed%H(#X_1%fn% zeTY^zgkUX5`KrF3`56CSdR)f;OFga&SQ^QP>+2zKgbH>)slX+cPYXTBl;1ss<;M`p zNy9msfb>iHCCMcSev~FkoJsOxo>}gthFIZpnfRky2+8j#nDN?g20oNe11Xs+S@OI9 zQ|#ev9o)q0D^WKevUHP9pJWX?0wNk0cz8<$EP^0~7V*((&+ao|cmfvhmMt$aZ$@KN zqOHk-oc6@*CkV0%vY$}Wk(uMhKx0&djdFV~i;@8H8mHe6S2@p{6PkT8m1h7hwC7B% z5b$X_|4REo0f8*N{?@wfLeh0J>@Jr3Coy4ie>Q^8$q^Rj_sVtn=1pc#@lJJnqPySL z&avCx757(bXzMDWS>&VB+C^L0joH)g2S3S2p(gIwAEiJJ$8HZsD=KIjh$-SU2hy0< z*eK2Uj>$<6jRlFy5_Hq*ri;J-DxY3J2>ThCfA)Ox*FEntcj`XDUPIf#0;5ZZj(5B_ zK0+{$XSJheP<@7b=D^kFIMK%00#OQGSAqWleJFcQUaY{?^(%M7$ZlyOg^H`*2Oc_q z+tR&N-b)sqA#ujogRQD7Xed-?RWXhA+ zNbaa+KAvXTlTqczIG(9xwvMq^aP@vaQ@0=uZksU8OhX%h<*g6%9mt@#H}_=T$2<-h z?KjOMyo|ILp(ByAcWO)KzI#`*m4a#s+hpM1y{aPJ#h2`679d`JIpju&3RDmP;lg@O zs9vp85gH^F;(&al3HYRpL~%%F3e`X;hm8%t*t7Rw*kxOfkyT#;VGgy)=cc;B7|@$c zW#U!*`+)eW_CXTW{*AdC%T@t}C41+bba*E#evH;64H}_CLu`Eygc0p$orsaL{a-|M z4hp$Ys@B^vAM?0rFv;A-?Cw)-lT~vv;!{I)=(x;_R_nqMEXS`lfp(*R>b-(Gy^!<8ZLGHUi7+l;js8#E2VIU9^6q4`% zczm~immn@q0&_(vr*6<;6)F{6dRGjUQN1-|DPBcX@o*&zJ<#xdII>c9|G9Rnj@?sI z3}9t^;^x5Uu+7zY7G=qdw9elwfb+ln;%wCD$#5A$wZh-`BVm_C>UdnG){LXHN20s9 z>h@-|3nOtWa>)nFcoy7I5(vfa%Qv6=rGh@F-d7e3NtTYJ0SuOwEx3EZy3!*->!p@i zc*T5De<;-DtuT#REsVs^<`NC$3iS!czKoDQ&HB=sd2%1EG_8$-u9mq&hFo1Y78ecdcH+*1B-S%tEaT)cAf{RpX0xI+0+b}d3 zS{3mnvz+@jR*Gga@DhUWSP4SXnPo*RY--){&r_&)WK)KxLIXFHC@z+sL>a_EO~IGXzsHRAZo-JH^|I3!Is5vi!EM?028r+g(tRW}Spk z=@p`r-_M12%0iNHVdj$@Q{y8Iz5(wofNn@TLS7+s=zPKD3 z3Po_pN*6yci&+SUoF;daos-|RdmmKg@0@0ebu45TUq*E*6%B{vD7ZbMJEa=h|DEaV z%u~_C3S75GN(lH3F1eF&SV&zrllHzquDpi0Xtz}C)8N!D%WRAABZ4m4Kxcw$+#~|*-OUn|38?LnpIfI@&3vYfJn|GG5hTy@r!ho=!Y-1v1@t7de821 zl=&&8DMl&M;)&KwAE1924bLnxRd+mn5e)d~ZP?fmmJ7IK`=}u4Bnms`q_)bSPNTAt^3yD7( zrvE7I@McbJJMqAAB%Z6K*TgO?8eAScaywi8L2>E;KI`q3m4t5@_2H#dr{@c*<<%50aHA9lJ*6YI}y{EThOYgX#- zQP9hRG*qCxOW!H$8rjVjG(^{`Vc4dSAZT!`I&9mrtb_XqSi243Q{7aF&|wxP))r{c6b5~HeO=qyg~`h#MEJZnBImOk+9 zh_m>kCOJ!p@ES<^n4fTguV;lEBs)5<@#(#&n73K91BFR{twmZkM@3I4+FSKrWq>yu zY-el<_ZA!mE+c!rGNx71!q&J9yh$)fG&ow#?PIg+Wat5j`RD$Obv2ux*?NnDz~xPi z#%8|C^Jg@ZeRK_gWigTCM|=+ITh9)x5v~LscBf6{%hip` zojpdrXj)SVRo*?RGz?IZ!8bEVW0sT{V1JH-y>~dvO%&^L(9gVWK43$K0Ur zY9HAJxQ0S-1xCqHDL@)w<#~aTY9Z-U=xizfP>b3BmBd%>40}W;Ks&6&1lrm%2n;+F5 zg2&s1GPNq+i?{2Ub(Tdc>KA(R2z2MS4c~%A&%FfTdp(ey922H6m&ch_az&Rvio{Z^ zAf{uc`8))ET{^|e?)%!(_t_~4Gj}S4yGvFI6vGd4qBS4@4x2sK8qyJ1&fcJeJfhqA z@h)LjZ@5y{^A9YVIoY6a^@q{;4TkXi-Sw=JS^(69M&fNfY5xQk&3OMT~Vlgqx&XLICCc1 z)}&fUHwR({@48n~2d<{)be*6&d-iFBO-tu<%71|4BU&xRRucxdnuJh~rLIHa`8sW> z)ongUe52is29?<6DZ0;_>l(1^;Tx5eu~D2Q`oT<=-S@1Zh(`>(O_Gu>ij&z*!1cwL zRL%jr_r!$Jm$z_xtIl~OVAcg*`lqnh`X6)(^)21B+UQ9BTYneWtIP{&-3l7z!Dw;8 z?nZvo4IS9oWYg+ha_Qx*;6s%g?mclb!V$ctJft!--aV_`*`N&t#Go3T4{6#B<`m_y z9y)BGFmBk0@5#Vu%b3^md6iPu%G-_?x){Y&vz$`u1iGwGCJjy)+XtFc`Kd7|E1&M^ ze3{Fl@F8Nc>8v#7d3w+Uf2HOme}xNlxZe4#V`kKpY4-B%{~@BUPLx0OnbO^O*KnTO z7VkZbE6FZ}#5%>90Kvt!a?^N^-1r@3}1%)`BB(pP>HY||8$%PzQ7uL~han`DMwJM(9Gts{eVBdLc zey(2CPINX$8u9UfaP{9LhrMNZd%&cO)SDv@2VTY-0BgMG~W-}!h6jN~->R%6{{>OCL$hrAu zV>p-FC_1w41O62N9SFgT=}r_vgwU%$?D4rrY$30QXIUjwN z#@(GVX8g{3wk~Jh`i;nh(B4xzD-8ad=qu>xpgui2S6`4^=M&iT2RRx|%ukL?v7O== zm6=a{gc=4mdJ{Sm775F6J-Dl-#7?!i_zsv;%3QPmBi)mm;Z7=yCMJx-6?ue7@YEOP zfXW2iLYA{S{*O_O86Kn2Wi=`=6*`O4N|BzW2d%qJO@F)=D%skc-m;|jPYC*yHTC6z z zOB%-jTplj1Qi(4l{4I4|J>R;trPOeOkynSDHzPx6!^H5Wt=`+n*&;*DN@D@i85eE0 zB8{EylLhHsqC&_>cVO<<_aO$Kr0TH-e+PZi<}D5EMqo0dCLv`DV?Ui#$8DEpPj zzR^O}?q2P9o!09(_U|SpMMvL@j@Empt*tfn8mcCpR}>?$MVg11Eznwd@WLiTp6X6~ zsqhe9?&kKtlMk)BgN=sPQ+E$Ne=P)yO1WG=@7uZ1=Rrume;PS=rf|CcEKZqAw0N%& z;-Xy0B83!cB0ul#QlVor#Gtv(*h(gz7>_U?oZX%u!`&RTz zM$E4OvS!$%k6*$7p|L)>xi19LWgU1P2aVdzBS+unQbmemZeA|#B35N?J$SbKy0h@W8iLcFT_N_% z{*?=vmX~47Kgf~eQT1k!TJRC|FCL4LvKX!9*IiEh{lztnsZ7JhLcOX%%jZ+bAa8UC ztSi}?T-1J`Xy4t8Je)!f@hEamv351s&46fpqFM$J>sZ!e6cR&GWLR9mC+|cW%uFRhAA$| zy7eET#L>gl_(weF$}B+jl-3K+@5HTDLc2k0g1?OxIpOJWVQkP-iOTLN{qrlM(>`6?@$KAQY;)FY|NM5#v{tjketoV<3b{r zcsV%k05^y8*Y=^qyCUj7*yY>jfDMi>817-&sbrgH`vS0Y-8e z8#^-k*mFZ!*M_`q9NNRd;Z;Gt|`B?AXA7St(Qd71* zc#`;%pl{jr>j)6$EDEl7Vb^u>DHpU}2Z&DF02aZmyhJIpLvG#T6ueDpzhC%Sorrd* zsVlHUaVBpQkk|F9K&REL4o7W@Fv*hsw2R+(LSmjJ>IpJi1nI9*=g|qv)#tFH+j9YC_fAM~{z3gOdKZh_`#IMT zWxp9al6Z~o;_a`uQ58$naw|cG^_E;R4H9eck7jVYK@;7;xwc~}tW@NBiu;mDgJXy~#gll8g!(n>B{7FbN6qTSjJ7mE*lng!}&ij6Jh z*cp33cJSg=^7F^_%7!_Gmye*a7qsbdYQPtf9F=YT~zxqPU8;Mx17sH(r_mj}s- zcg$V_h#Ky0^_vDI@l_uQ>De7a;0!6THk7voF3&^2KAZvz$xmtg(J&G)WIdZD0ERi2 zQCH*FQ`B;#>x2n!rF-+26aX|oOHgii?k85up9TqdXVi1e5{Gsk>EZ}p!C8XtEY9!4 z*xB|);S{J`uyh`p2g*FfW+f@9jigsoY2hsdd^XHG*f7=08;IGSRucOkvi_NUrl_+e z>6E>FdH!U`+VC*>rTrT}06B~{0E7>F2%)M&e0Jm<7}}XQKayN3y?x;sPv{pp&&~S_ z>-++iMgF-!-=HC~VYV<+UA=FRFu2g?oAMlK7m#z{lp)$-&n-|nC6DCuz?ec#jOwM2h;${8^O1xKmA*ysG$uYQ%4xct zq2pLV!1E2V&_sXwi<{2Y5)I1T4LLnqjI~R32;K!a;uX>g1W^$g0>QBLk&Mm9Nu<-x z6W*QappyO5lPXkw1fN^OL?c4-FxtS)WW+nWmtdRIjxXz%DqO_60F9ve5em4n_}U+& zot3aaTkxzdDo9Md{hjBt;aQ28J4)=2O+};XzBX0#9$v*HI(+9e6Gw9xNw!&&^Z}bo z$p5#wFpu$IKg>OTd!B7$8AGAw{+P_*K+dQIoD8mA7grSFlsK@e$!Lf-k0_y*v?Mh( zt_CEb*;xRzjjfwPC-QDT(d(2{qHc7?8CbCVv*hB5yqh;&v=j1d%eXJ;ID@sn^0w9) z;9{yo+;r`2rg%vFMboBZhfpCht>8DDGB+{{&}DY*+*=sL=PX%>6INhb#Tg#TZT|wQIoKw+ijcFP?*Dk*wOV}b;%sr6mesEu zHGs_($l|I7uBuk(Xj)SuxaM~HA&0%Isdd1tY6}pP zkQG%e4!H*J7d!ys3d-Q)lgLP%f z@3%=s{?b?ni!ZrW)d{U%zhEyX{cir-E~CurX`1ssF<+#*spayzU@j6dy)^~LmY4gg zU>qqY1`Gll!B_FJ)|;?w>%Pw85A_S_S0!EG{33>(3_1hv=pQu1He^z>XM zvW019(C`8~Nl8;hXf>(Tso&7rZk`Fs7g#Yo@7A%CKH<0Q1GTdlwm*9o!ZyE9 zTI{j3sqO~-Lmqee3%~gM;Qmes=cN^`FERJ7l52@4e4loe>)A`aO&x42QHD@;3w!n8 z-M~W8rXYSSJ*SMK=6PhB2zTFpOrnt~xkp3}&o;dgpTi*apPxk1uXK=8qf@s&hjdcJ z+Ji!=*okrsyv)VVUayzUQb~ z@Nue1e|=lxa?fJ9$t7SeDd3wTcw3N($I=@|rGKXI{dSo`xx8!PR6E~@YgG(3-p(#F zpH^2scV;ej$>FJ(wWip4w(Bvlsw92i=>rO>=C(&*3GFY)dzJN=FdzO1t{QI7=$HqF zh%F1~NM9G{TkGuCgn=h1T6(Fuv}zMpWNlE0c7831Pt`YD_wYnb`=w0JGT+O&kp)zYjyQ-2x)0BFs$XR=Y1Vrk*ew&B{z81IV zm0{_zd8&-##^=d!xyn$TovV?*Eob*V3WihbCjO0p?F*Lq-s94jA&zhPH@!h*WKC%p zLu+xC%d)f*pu|+~G&0M-OyXYL{!h*%5twv&R+dbSkb(6@>2f4#4|zFj>Q!1 zwx&NN()_FUu>q4a3&!dxMDXaECnt=ZfR;8Up4O{i^TM8{tR4C+9PCVylE)u$PJPu$ z)JzYuy-$@$zRkLnA$8kMQ{hz!y?Kcqf zoIhf8dZH!nphF`job)Q2EvPJGxo1rh2URDZ2q$#PNplEzuUdZMf!&`^C`wot9Uq^3 zDWj)`l{bp*`nT{lyyCWS!t7PWuuT=!1bVr8%2=w%O_k3usp{CC)yEt)-756gYel?t zy5TvC+v@!yCMj$G+e3^S?}HQ)$sm&NQ~xz-j#Q_%A(iBs1VNEHjPBBgLyj_%5lJb5 zEs3h?f!<+Pz}ExQ&tGofiFDe)5D%`Armgyq&_v##OC|D8TjqrxdN3-~-b8lH-3X*^ zwD5JvXcTAsvgpm-2ZNDB-uVMs>rfapz6mql)z@dDuAi{lq3xI5z-Er1;YY5^%Rupf zgIhbDMLANlX;+8X=L6Jrv5QSS_s~%UKX%B%=+jZQamHq(i=UMg&8^7tg|p;MOS+V9 zRwrJq?({DMo679Is@w9wk8a8CAez)L>&reScY0@EQ|UW$X(kThcfz4GC>$lk6BXWq zpHehmKr+thqv~nzMF8GnPXR^yTem`G}wBPyRw5sQP^|K^B~%2koq$ z=uikwLo67VahMXIcJP*>qyVSDB^5R>`HJX5ZU5K79(pp@zpvsRL%8f#rcYPOa*l5b zhVJo4yBD(~r=NJFmPITIIS%%w?dJ<&p9;AVnHN8bbY`r+WPQHrEaVQHGiR7H%W2J- zCGs43WpOs`4Q*pZMj$EAGxR4*uBXn@o{r10a&|i7w#Fu-Nqs;|L*RBoiNSBeSbIDF zc1S>AKeti78t}tw*dML3djKw3*Qx$8 zb|IJ2!Rb(#YU>VN$TYm%gtMP2MdYtt0qR_dX#ne%jsjc>6=f^1RwjloTwKd6^|j8X z;Qmsskd~w)QlMBH2`8bG`)IN#X2u%yRk+QSSz1KIjD#)S3{Tn+b zbN1sjfMF;q$%A^HTWb584s(JEsztyIxU5x9f9r8ekA$gjm`9{%ei>ieOU`18B%gqN zw+n?FkAu1n1^F##S|?%QxBfiT3qQ_z`KJSQ1YI6e$dVS~*N*T2y#;V_12)SYaxDaB zPp$L7v#Mv0dqZIjGdmJK&utnj88sDu>q8{dcOX@yR?!XzfMm^(Rg(CVIL%A$Z8BcKgdEl zxFap+jq)7;yZBBD-3Nt%rxvTfvfoR6=YlOvy-Dh0x&2fZ!<)+N?{e{D_-44Qg*sE& zAF3Vi{GgOFYc#BIVp6>d@hrcyYwficD^JNn8Rc5hQ@FQ7)OPbU$8wx*FP|qPou@-E zcq!FH`mF!!@}^DY^awMB^H!U###%kNgV94hPh=|fUspAsQ58glHEAKP+CYtqLWVVv zeGvkoN80c|NtMu&sOkY%*|{RcnDeq9?p$ZHNA^7z_YjU0!%$zB7f7`=EoQgA$xu0> z%uBBW=cQO#@3$xRgT3ThgnEUGV!OxAak}Um* z>CzuJy$|c{xm|{EbE;?SLQ{bp#~gE1yT98LHAmv{H+&Yes@#Y8=5#O@3EcGAltE&i z#lF!`{&^}Fo7VqJF4u-#DpH+}90RaHe{$2m>}#c?Q+yh^zt4btT2yCXfF*NA7~#dq zJHZ|WC4xFK8xJoBwW*n0U~h6Q{~pFxh9azHAK4FUN$5)T{vdw}wWuIVep$63l92agOih35<4Jtq1#Lv8! zu!Hbo>&hF5d13vkn@Gk_KErhs_FSQdr2SYouVbUI9Op*D5g<9xZjlOv&_hyh^wPlo zNRG|@(`+2MY)lg&*DbUb<%H{-rhd0eETOJ{V?uerJEWiCrDR#^z0ahTRg4Zlp%c5= z41OCwq@)8kGW`R?6Zh|Qt~nG6_-uPKwB((x6@4M41jW_wxRzMkzjl;ubnHkuj0AB_ z_oIU5r9xzAC^4Ci_r9QxF+g929X!0Iho3WLT5Cj{|CRoYXi^F*!*2!?y|=^5c1ugd z!+cs3fA6b|HTITGNeB-?tqjV%dJJ1|T=nwIQviB_zJG}`PjCKD89mbnqImJ|b@<;k ziw5J*X`*n%-^%*zeA|`hT3bB&s;yct&Qz`kPxwFe(WOQFA!26ga!UA+hHzVc2gcEN zv_G28MU@G7D)#CBq(8(`?YI|2*6w`?hq|26s+g|R@Zac+AhSjZnWoY~nU7q^dVj#8 z7hbT;uh)TM%QIK+96LDLSIuz-HryqCq$oLciVJ56n7d0pE`d{rCRIr4?@@|H+_Bx` zbDA^iK5BNuQAn!{GdSh>z5iI{-u+@r%W&6n`v_t7&hU%``?YfEL*{ju_wivW8t=Y5ZddPozwRZTp!#GcsTgWIMDaY8LpR*htXtn_{2QQw^iNLNh(! z(dy1VB?+5M0SI5#$|Y`TOojZM_(fVJmMKUXk!i8}XX}Ux3zza=(EEhf<8!z{-Kqa` z0PB@xA&eqd=s)f@7i!H@J&0Am3)SKj6-&)z8CI_0t~tP!+# z@g4mBLPxa7GR(ZtM?WBeu*JI_n`z(yEwf~#q z{Q0mTU*XThx}a`|QYtq{9vLmZw>}g1y-AOl;dWY@kj7iOH#*)GglXjzIWTD9}VSf>n)+Tc*oXQ!2? zeoBGj)fg_abT+jiZkpO`&4%0%(Z$mA2AZoyt>I|5x{wXi0O@<1ut(()FuDt-+}fOL z#^4}fJlOgE8mrHR;CfXAjNybBqS-+hB5Q#cdC26=b4cw_qTTjk<8NhDuy3n=ncx_h z5u!XE}BrD@HXnx$W4=*P3r2ZL#Ik0s4oM6p*t zJ-kFnJetTqyZpIY{5D%y`--bU$D?~e_ngcuW}F4a;^VB?UBDt$MHk3$@CL>0!Oh+p z_DoD^PtGt}!h&awtU_aoljvc|UIfJDj`^OP$42gDZdGDFyPlS=;v*2t^Ac_ZL3){l zKAxO%*PNez45%*})A`HqF3>^b6^DPW4&ZJ|h_6Ww@KY(o6I|Xi$i4u7c%Z|?hGeP* zMT#%icbP4l-=3r|4FndSuRe92%JtAr>Sh-Y=-@isiV>sS9%1}W<0IY`i!$x2xI+(e zwuTrwA9`*n&H0YaOjuRlSu(oG)G77$aO|u}@444{hj-LjNOms%tBKFsGUa<{&l0HWFn$YX|6Tkgxa*_Q<8!y z@aJk>`xpudyPK54Y0ES&twmnV%{supH^chjsl76Oi!;TgF3aWL^&(k`ioW|EoHt14 zQpwki9qhC4<)p066ba>J0$)F;c&B-U&%gChv8j;fvR%giOtHzqPcY&`|JgA4edZx? zXn%cTUlqx<4s|wJm;c*jToMMW2IfgK{A3`xYmt|I7lb72l=PqaVb6LcX{mapiJHUT!Tl|Gl&j$$H$;q|hgM@Y4F zu!*hUs-~)pQs@_o++~Kiq#rHSx#;ZS=8%`Ju?a>V5olA&Ah+(|$k#PF2YVt>Pgx+( z9Tvr4<)4&9+%}c*xtO5HD2U6(y~|TE90(B%V<{WG@AZ;7$#JLMJz6nOh~b@HM(7T@ z2DnF4bo&Fyi_2Kzb{9oWY`bMbr)J3>TUhVjuLJag#8}DOmomOXO^YPKb;0NJc~TA6 zg|ptQZT&f*g5Q5We2l#2(C^oST#~x=(gn$CH_ry^SQzDBfTrb)%uzm zp~K&TPC;LKLojGXscJU0)k&~N_fmzjq3I|Nw6Yj901W9BS96caOG?FPK;juxGA0M> z+E_rOGswE8w;R7zDprgI;XaAG5A!#dHb~%DIn3z|^6rhvzFE#FEN>zqT8yUj{jQOd zloe*@EE+%l+ORu_^PFTQFgd;M>wMmObrDRW%h_WCvpa5&0cp)wc{g6CJ80m~RhI^B zZFbO?(mFOMw%oq2(D5>RQ=q!@mDEbG@%q!qLBgaofn_#;A&X={C>hwYR~$22M+dSo zq&_gb+KUz>!@;BoX^`;4g`4y%ay0a3wz4j4#AL3zkGP~LHo-wg&4-pl?;qc4k^0Av zGDD%KINCs3{lT<-ZI~OSC!HGgF0H;0H-!=B8qiDP|CO`2y} zLmSYfb72w#1_1ShU{Y7)}_6R1NhI1alg6}rNO zrfW!DV9K0hK^qUy_PR$AN_0=2pWO3M0d@C z>g5IM+Updn;a7_Ue7Q3|OH}FW1Z`+%&F^Y$+OE=RJx-*FFzVmE!lqvp0F9*$m0msl zA|S$RUlLWd#tYpvPgI(Ttm}MuN|ceeOc>m^?-FXWdQdYf{HH{<+4_88A)+$WQf3ze zR&LjL)b;5}s;sGVHsRDQdP%62&DC@XD+bzSZ$Fjsd`WyB%|M4%p_Os}C!oZs?WWWK zZ5=>u-D&{B301IKpjLAjpLl&>1Un|V{*Y$4*W5`z48_T7k19aI63e0q@p&Zb`T}K- zoQs}}gOFSuv8_EnzDKE-ZUd7fN%pkX5{1nu6H+vj_)b7R-Fg4(mT>2L4*^O1FXO4afuwrF{C@;2F`>&lg&`BM&`YroJsq57wh z#=b)-7_HiF0ja*@dO%vzC43pi1^2##lohkhr|cOx&^IUVuFX;?i8fMye=lSoQ}JcjoPhVs*x#JcQ=HlRuvjI8}? z_xrbaOJpw;KMb`+t;w9$%*Lq5wjzcrEHI3}valO{k`4alb27OL6>lpoISc zH3cr)7b_y??V|3B`A5KZJh-nPyq3D7T~(`9RzSQlaROAZzVf0xc?)E!LA<<2;l*ek z2sTc##`K>64a8Fx{LRbs>qEb(z$y!bt9IF&DG`YJk>H82`K?mdA>w zhF(S|iPA5qzjBbHKO~kixkui4R{UAOqp`z1Z7FDgP_`=$)j&5Q8&!=;T$Z9y@@dEZ zSF2Oqc|zFhfD1rb9R>#fJitv)XYB4-=L0niU=X@8wPrv1oTavyAwAr8!!@R0AJc>- z>r$azRvtI%=71XKkVJkZmKDKp`#?Iw(nbvnRGL3T4(8|s6416F8I$*v#D8Xkl{K0f zTD87HNFa#cc+aFRbDFU)RlPz#UdIga#E~jvwr6^F2R2J+SQqb-vic-kR=C0U5a`qn z`rtKfj7Op_kUM~x)4cGCFxh`;aX`c=Cq2D)=JdetSjVrz3t%ElK3gQwu%yaCn8)G? z$lEai)eT0l{gdYDooXgGsw{;T4`*-5T~5Dp!gO$Y$z*X4mYRN^l>QmHkcu8wm$t3q zRh-qc(pQa2sv`fa>3VphpmymTu$)(6Te@!V$rMME3Z2I zE;~*{Wf*l)htjCwv(}f?VkP9RM}Y?Y&rWBw?IC_UYre(tt{sh;*-Y$u3Jd$A=8fmJ z_Erafwh2n)PNFY1F`0q}=$EH2SqvEe7h&o*>PxKa>U6-K*G=w=Iy5}{>Sy{`75^ep z@Tf~e>YO`W&_#@CDvn%js7oX1D{vd{!ku_&>`7&Cz%ving}lCV3a_NhFV$(wgfh6X z(lj)IxxbR^)y)9N=p&RTa&T(r_^U`l$! zOUk4g$|mpHleRF%PreepsmzsQhnls?4lT0rG0VhrLRNIVmtL$Sbhhxgpk>c*=ogQ4 zhWVO!E(ZF-&D9Mtq+fy(Qx`g1V%8K_f2beiZqPKm@K@XrZ0GSXIo51A2;15Fk_@Vn zH(Y-U>*RrC#$IOZe1<|U2Pr;hkAl9A(?%R*+|#a)p)ouo`)noUl$rAwf;3GuDb zoUxp+Mr@y~5h4UTgfe312LOY_9!6j{Pg`$F)~6OF)^C~hahpgZrvQ@h%Cx&NLnRK) zf=7$9B|iWMak|2}zy$peNw|FaGFCo%8nESE5Cll+jV9gl;eDW0w*2sFkI+m#O9T{B zi<88z@d_~k5_De0?d7!7I*Z|zqhSsE{+h#>|E97c1CTN=pc;4T+2tb0{-8pF3zx-$ zw;Ue5N7wPO0Wgw?S)7$zZIny}g-BI92<|&|&L0=MRUqhGGfDH}i;aLdwcfRJc}qVs z>^5?hTC4xrd8F@A7IGYzD^@DDBCGhZMN7n+yMC6Ky>Wm(VpJqb1r@!1^!ol@bgXa#g?>r}8HDu{_^ z4emnfL6?NIp|zY8Awy$#VoZ#0;ei^wvs~s)b^q<)!rQpBdLm^JO0>=DH(VDY_M?@sw+rAnn5dre_|An8M{jEx;+#x#`+``{o6?&L({C{m z^`f(R+od`v_g;6|bZ~{QL9-L*fQ@T9+iUyC<(7k{XU=lSwcyO-lA*FuT6r$q&!n;DjDc}pcyqE3`d=Mp~hjYK+CR| z#M(X~kIQ8ja_SiArGE&y=$P>;zFv>K%0C_eovbZx2h0O~!dCcgKmpb0&~|s) z8<1QM-y!VN(v@$FXlQVJ7P{9iijBLeSL*)Kx7ukP78|390hrzO4D)4H6_d|NW!APj zQ4Opg2!evD4$r_+t&VwF?2qB(j&<0Zov`SpFK*H0Bf6GB8c|UEyh6TXjK?R*5Jq{e zJ|9MSDLAaIz(b%`emuw7vU`*=vUd|>Yi-zJP0@+?u)I?0=c0*P$aUwB4(UiO!US?B zuglrf42YA9cOix(&PGJAr4I0y)BC6E#{4dcvA&w{9mbMQ!xvOj1yP2p_nqUwzFX@` z-N3_NBKTpYVHFW{wxU`c#wn@7FtHelw)a0Wl!;l%M#_~ZK57-jsojpz?xzS25~*e* zZ<#0!RgVXpI%V?A_07qimH&qYu-|j*etn2A_G|E!7ISTO*RYZ$Fg-|$O%#4n*O{dO z%nnN5H2?DI{t+%}K4l&bm^RUDZ(&a&`&J?+Z`gL1-t^GY2F8~Id>n#0sG0^Hv!O|Q z`1pN~=`F7#EVo)g!*;X1k4apmw`znJiTWE|n*+ z-Jy*O0$KcAn|O9ypkb@NLuhFq3jy1=XU^(Ul$kt$8#s)pRTA{DWRrfOk=0ZKxM0UQ zu$tKFJt1m5==XNO>OD1))rGc#g41_ zp?d@glgES@QA}VuO!Z#@u9aDB3FI(b-@$Nye-KVeMIoc-@CRIjI5pU5UwUc`k$S^0~<*-5m#F;pkcYQpAX}`StD({{i~6*0C1&dup_D% zR#7O1tUhBK2ztd#@c^)V%+I^HnsyLiTal)2^dZ?V^O7qF{sY zcH#@gG|J9l>M3u#2&2Du%+n11-igsVR>o(|-}kV@HYlfwMb(89xl+id30i-2FcHO| z_2vDfe443XsMl_6g29=tuhP42a^7VJ>+8BU9n>*Ug_n{hS88 zmJYB*aNyp@r}$i=j^v$G zW*T;WNyZ)k3}iP5s{mK*+?V8ERthD428=VVD<=o2j@+tSn36XL`E9-W5pvWG4|EDy z-Ht{iu$2E0uX22v2nq?w0? zm8YNFXRqr=%$iCP$fHy4h3`-he53lt?WrY8fAM^%a1K#L#M6eM(F14E7U5{2Rvq7q z47!6^Pzi^Wf}v&PySX7TNQUbCJt7Jw$|;|RnQM~m z8Li{R;vs$gzf8yhyaUa2DM>k|^~<8=wAh^iuernIWxR#oiN!1!FZ3_PqXyF-kmV;9 zZ*xFgllS6)N{QRsvM>6J!|qb#JEhu5tHXXg!+g$5Cg1X0SZ;2nI3#csatn1xroXz{-$)Sc0Er%20Ma{Djax2J3D2aP2=Km~vo_?D)S|k{~X) z^Cm5+enoWI3|!dVxr7 z!6$(Y&BzjYveA;Eon~U^mZs)J!|o%mL@m9<1JU)xCD;C?3L!PUX&-uQU(xvW_EBta zeEUe3usK}enFus#Ez&f#*j@l^FVr89D(ItEd$kZ5XWr=ytxH%h=kq{2KM}c%O?O!j zKFc;i8Nct*^X5q>+@fYAEsOQ4NYyp7LwIENF&PndO%Z6-eKK1YCECkS+IB}u&k6MY zzsRbgjRE7sZfI;$){7ig@|ZH%BzET$%%La+(?@}tZzV=(}^iCBx;A|>#S(?Qrw zCt1VB9UZp+J=Lk<8ZGkD* z2I2j$*w(q)gE#GKc8;V~_$X)~@@iSK4Bla>r-{P&Pi4b?te+BcjZ9&f7>+4J*W;+bR-~QlVB+Cd zKg(G2JZ^>&C%oMZXg~j4Wd%#>TVbPe3H9gcs`+lu<%``J`3lvKJv z^XPJ{ssYf$-b-)X2(S-bJ}Vi^n%|zqx;g3D4uM4Oyr7xpGV&_T@Wu<{y(i!Yhha51 zCO~B4dxWf1h5l=g?bOMIN+X9^(kij-_&hx8w%$ujPU%?s7Am(ap z4q~P+>=HzTr)vV9j2C%G!}#n1fkF7L={Y6(?N;v`v5;Fy zdioIjDOiH4cj;b8>Va2iR^Qd%O=!w;3ve4oKcw>c3$6V-oIYR&DfC%6w%7iAHuC02 zr4=1M{Qw^{3hG@sz=ee{;qC~BM9XRnrk~DX1sFd-suLthu7(sR^HGA6MUJ-V1{mr` z(QW{maQBGj6>^#Pz-oFE*?vdK_np!QFD%Ge)O1WJ9#CFzPJS7ZJ=UR+%fwCpsey|7!uu=$n9ONm4loRFadFyhqj zS-+0ik>0XP`G+PsPVEU4#hv^L8Om9nQFzdXUA3xOvQAo$5%hkl<{-!1l&)l4d|(1X zwmqB>mK22{h&&_}YL{9>To!Hfq<BB$VHOH7d$(RP8g{VOKdL{ z73##=gg_wc9Q&Wduu*KT{!!FRiw-brM7zLlX#z3mS6W6}2hT%URA-d|OnTtx?}bmt zm0f^D@Wjp2*M{!kll%TEM(qO(W|ue84%WZ&yL)VWklP;_zCaP7AA&lZZb&tzfoWLu zd){BeYj01WuAaP}majXB|MhX*>U`lv;?XiYvL$VxDq z;=P|5N!`eUsPrH;$jZ~cKfIn4c{;O>bv|>mI%4%X?4G3T$b;YBiy^hm3M`J0Od4<3 zfR$yMr1kr5C)4QrC3lakdg6Xh)L`%jx%(I3mXgWX{4Ll|1_Q;(g(}`aqw>L$2zj`z*n8j7vIfE5j6G8s7Q# zed*l{OCi|3iLV+gp>Xc1V8xDGQ(Vyd)?T+!^Bw};%~0X{xw#oCv^WeG;xZ1Hs3%j{ zb_a7}F?p)ZgB(n?aVl4W^`ZRd@w-tvThA1zk`}dkO7^o%zD$XS<{4zl+Oq6x;>r}d zrpe1Lq7m+70Z%P-;AWG6b%cEXCmN5IGI7k*?m(4W=SI+SWl!|OAZ{lk42 z$X0A+kI7z3M*wxy((5dCn-%_DMIzR8?be zjkf=E-FjNKy7W*6W6ne(Q>(ODXo^k2g1Ya#eR(&bm^~WWrcy6aw69B))e@(Eq3hh@ zRqTrih8MkH@F9qz&(m+>KCTXxt4x_rjB6BFaE4epH47$Zy?L3H(%3=G{OYzqE9rcx z!VSIKvu7lu&XaQ1mtt1*gq z?-*C^(BlPcF<#pZ5zFo+)1g06R|OzXF~kX4r|dan4=#zO9??eZW5trG$(Y+3nUlu# zYpcd%q)w)yy@UBb6HLn3^S|ZiGj$E}%(H+~VqK={j=AALom$?)+lFA^H^3f?6)xnh zn?FBFQgn5_e~`9C$*@jka@G5OUb^k8^jBJX!@DB6 z{~UNsITOWs@V1J(wv)DyB@Inuz}eDyKX~`82d9c4r$8M>9Z1Ch@{P2fWp70NL8a{*v%;)dlsdE2{Kn_ zrZYLPO8Wh!fT!lmdX(Uq?biZgL(|Q9%;WI6gTkV04Nj%m$reyH{!>f2#@^<%3Qo3* z_I#9rDvLv4(bsb5i5MBMlelHq>OAXkCp(N~&MG=CbXoc`NISE5Pv7!ulU6dY)Vh#U zrDqjfb}S{iuex{j2>cDMYafa~agVTgbl(v3hz4v}==frDTaF&?e7PiW^@$XuXBJUf z>l}BR&!l8S??R7{$~$LP1+v1fH@Oh(NNW8tMe6ICY)De!{TME0%E)b=hM^pr<~XLI z@EkdYxy5L~^OnC|8a0C)&mO|je2W!dIi-Td{|0FM@HP(0o|KM~`4Pc}7j#*qn!*R4 z<)n9DBQL*VNiX7E>k?V41HWscjB!w*oe=?M+&ka4-Jr?@hIVdnYeY^Jhkf{|>{7y* zy`B_f$pKv9g1=(4(j^SC82XQEj1ag4)nY;vR(ay!I}ZQ-`a=r-JpJ0^!k=7m#$xL~ ztEnAUC3f!Hl6Ym=ujcXsg-O15(6g=Yp|Q6?Ys7%5X3xKas=*KBe(FyQ`MEido`H>e z&vb55l{f_PxDbXr(;Y71^?NE%m8qbwkxb{cTy&VAm$ZyHu=i1B>>10|B1>h7?NciJ z{=Y1uYiV9~7k|0y#uagUps0R3iFND4|1t%lI5zt#*XlWht7KbJrx(iNIw~j{PAtw!+vVne_xbPt z-XzN@IceSB+4a*YW$P+GEz&&9PXTseO&j83^?3uqO1EZR?>Ts`U>4spaIOzM+Z&)$ zbeDumVz>UugQV>-OKw0VfhxN*D=gEFLrA}Ln-BRR;Pf+eZoB|pY#pdTNw2Tu9?bUO zG)9)?ULKWqW|C9XAJH-cvwh{6wY0-1 zGBpsAaZpvL&x#<{Idd~pz`^`@`HIMD7Qk1o?6846luiMZiG|zmmFBnClhl6-E5{TG ztV_45`fb~39ot>+HR&pg=lXXEAS)Y_Z>l~&;OkBAugAvs<8`n(fm8wEP)7(}SUrY| zDJ`kxDLJEWg*Qf#;;Q4;*y;Z-kN0u$7Hr77c*t?ulC_*Mj+-1z0$J;9Csz@Dm#SwBXUIM;ef{uO87rn!!G^XE-GzO#>5XG^SO;L z`T@^11FJ!&0)MP^hd~+;KdXwwy3$jLdFl()c?t?^2PBomq`&c81A(y;?6)eXGafm9#Y+Ieqq){1i%+CRf_&EHw%11s zjqn>M=nSC$a%a0)&Hzlz+ArH2JhNn!nw|PZgVuv2G_IrI=0l_50|T4R^h`Dpiw|NDd zdIOCMgJCBdz+AUbJlTttfzno&L)6BSFIGz%l>LxSR2Z=h+BK;r*g0n!I8#cyV=}Ik z=@w9`e4>_^kD4CsF}jcNJD>>cHh_#_^9YoU01V z@-OoXn<>Zty)R#$=F`|J%TJuY4XW8vJuE|Z8q8;2q#wJ0Z>U~4{GwZ#BBEo*2kzS@ z*n3rotW;Ya0s_c%+2M>tH!A~9*%_R@No?VVyOeAXSF1obwwjA~z&0xH$ zN8xe_X04>0;y$%y1Z08sqq|A|_KUL9IQV=Z6&aAzWVr+TD+ibnfost~Q1YH0D=%0} z6Cg9mJ+!D^GtaeY(M14~8y-nlY)7r5gKRw~+zkkR_6hguwE*pVIc4rnh|xG>BMG#8 z+<4uY%DwwT(1TVCK&_*(qf;|28S&-eBJYXSbDVRc>}oF{rfM)is*Ul?X*? zsE_VOxO#7LFU0VhX0Oa?q`wos$#IbcaKES4h-#Ce=|RW5^ctY&BJ}~Tp6J*h&`fKH zbPS8F?m92HZ>3wI?-E@xk#=0qQr)9K3=3KBl3U9{`L|var5X1xKi$|o&B0q`9ep~- zPM$K{at9&@f+lmuxT6$;F~cBH+?il)QgPf0RqRs$Wd0T`ldFe-AL|!a*{oszJVk1k zSuNhFDm{&UJ<(J$FHYhdbF10=(%HaK^Myk#m!}0U_%9Wh3S!F6?|OOmzr&vtEu`ve zvjH@biGF7$B3ewMenBwHwT&SMIFbN6@_J)bgb2@{^d$q?GNSxm^5wHFRsGQw<_T81 z@>W%u?;OG=mO}?t-af@jN9mJR;@qdTlj1^f*P$4F~H&CaaGIbEP~9zf6eX(Kc@6gOD?H zM*-rDGVPf4VEbxE8QL1e^>^p}cE^9D@**UQq8elu`szv^8Jsle##;;1U!hTzBl>=z zUypMI9lp42Nna!b6#>!*4;(qKJdM+J4xxEkyx5yD+KvR`3@TJY_se7_De|)#dR~;;qVvFPU9;-=J$`+0A-O)4qLF4$Y{$ zY7_ZHFzX6^$2!y)ZKkBTof7>7tJw!+#dWC-h$~WUnTsQDucp9kzB)V;B*AtwPqt%y zG|F2^^IHlWI;l zi&;j8SxlNrlf^%Z$ueV~l5zY^)+Vcq&LyJ8EGstkmqXnazhMYJ{EdWZGaR=IAjbt2 zYjw`R&S2){N_HPq>;oxLWHPO!|M=P!k?`ipL7=XKavv4_X0+F$u;DlfM7k@e_z{9t zV40I0++4A_o53p?2yF4_k*X3}Ld@q$2ThQxDD`^Gs59za)5nmhIXcd4$AODe2k-bH z=1o<>{skH)n!o&vCr3Z>hn7)SxfOUHa`RM7p$^xt&)L!#D#%DVbJuc+YWh~ra`yc@ zxy)lEDk`}4?6@@-rl>_z>tk0SPTOO*G&PTwE-Zn>__41x%iw) zF0N+gE-Z?6%&=Zv@N^%P~ zrEk+uEHCA4X_5b0VwzkR-LP2+UMAmkavbZmdWz?K>7-^yk^-6|U%I#tW6b;d(v zJ91A|;7Lt(GUYxI<9Y})o86(ct^dlXbBSBjHH@eH8B@82L{SzI*8HoIN$a(%O?dSE z`d{#^*+_Y`oF67e%ea$+s46ERY2-|Y3G!;P{o>y(wcw{4$yTlHa=5jt+i}Y)OTFo_ z@jGAW)+~S4;zypj87k$(tvZceKrGoXcmdjQ0bj$7b;32|>{15n8x)rq7<>o_=2B#* zMkfm#@D-@)$sy;pQEZ-(Maf5k;XFRNy36#p`!NuJ{r2mEm($tylAf~)4s@~kH_zF+ zjEEkvnd5qK@D@3oWf6Q2AVdqfv;yc!`7_~WrprOENwQRw5M>BQ4HZ~V9N%90)88M) zQnEx~zCc^frEH3uSxIV&gDsuheV&9kO7J8`&i*T7!lBQ{;b>8y{8Z6sAS8#=e+0(* zT>5Yo(5_%sXD_H2HGk05ZV_kSBaW-I=-9~8taJL?ezA)u4uHt0XWW&ibEWm1_kIFR zRdjEl2AAhc`qGXCS(@dPU4G5|Vx{rl+5G%iZrRg?1AoN}hF@3LI5Bd)S~cSI_~?0^ zITG_K9QWqLFepD2{8V&gl8e8)l$XC{tS|a{cyN-$wK3C{>5COR)9Z^*W5sPmd7~fj zOv0kNXfD2+;n9<%^ZL>Tzr^?Qys=!I*ynu7gp8*C*U5F#m5my=f}T4@e%e+q@BP^a zME*JQDei+m+rKl#$yN;j71T^QZAAp}<&)1WuB6M{JfRXseJ^~D6`Qx@_+JT*o8|Nhx2M(VOV_ezN&TKVo7%B>7`!|Ks509EOwDZyLV8xH{HPRg zXmHxn#wdTF;pM__rDOY#j0g;S^!W+*N4XZLH3|N#+^<*i+QVeD)^~ zDVTJX=o=>Z52;=5ZM6pGm8zUIFU>vR$-#!wDiW@SA(8#ki(0++!ZOycyW0G=3~uQt z2WP+*M)3&-h-_7uwJ5hu>goDMS2)*OmO?B7zc2di;kxWPIPW;jH;mBw$#G$BVAL9# zAyWwSjZZ?l9&mjS#d$%K|M1`};A075&R0S7(($a;*yuYM45jl8s>9}nGOW5-{jjtw zM@B`PFvt2Q%`rXVG9Fy6{n)N*@TP50#Wxys-i>&Brmc(3Nayug{FCJab<5%1hwW2h zL18Ag*JJ=i0WhqoVgEG!upRtzGBXh_|LIUQq<82@TW(v)V)eo`%4p&8nmr&ecqxih z)$+Tmi+_NIr7!sz8)MqErVPuMvbP5>(Aaw4do;!Twj`E6Yg+$~@laA8>efeC=Vf|_ z9+7XTM%kV&bAxXRuU1aJ`~RzOXJt>^m{WhboM-zpRDx#8@wz>Px540PW&F6FMukh4 zvD0dA_^f(K)4<22l|UMHZ^ZJuQQPGcCv};-%!Yy4@_};ZP4#vqU0rbru1nweTGE15f$L(%^XOuA~*SCu>bs`9Q=juaIb=r!P$zhp_%Elf{vs+iQ&LM==< zy^-1Dbdexk5Ea51G_|}<@RPS?@*s`AfBIzM&vvjTLPBk4WX_`gx~qQ?hSYa@6hw11 zm~kehQEuT8pTzhtD#e5F*#rLs&w2m;O>{I^Q#8(d+wv-!b<9zCUSF1_unac;+25Q+ zW*CcPqiRkTCp?YFvpx~F4w7Zb(fTE=hz&m1k|&c_J27Lc)?6xESBiM2&IauZ(gUwb z(e<5<=WD@cDj~!VzngtHXcP(eFYZg17fimKr|Dv#W2-;aRJX1<TGBI~!KNIM9(`pnk(p_3m27i+4B;%RO<$SMW!x26k4TEjAAtMX=l*Pm)>KewNmY)f);!%bJN>%@s0 zc0zQvmG^ngaz3)E$cqJy-)Pl_S@yIu1M1{6c{lhyQ(HMdnqePPI1?S@8w;wO2bNwI36!M&i}-)l)nF+TzDme>=K znMXC{8=OOap1DJ6lh=hn+}Ea?D*=)%;deV@E^<8O@^tGjyiX`gB2>se*ZNIlL30T9 zVqf!R)f-wepLobO%>=HO{3B6`bO<&$)9tsTTPRj;O4VQwPC_~;8%L%`aWzM`riMIt z1SbmB+^&r30VH>xe(2c3OnTt|*iW;1I%Xr8Ps^vPx0S!!Uenm389nIKn<*{~M9Uyz zy|h__sO4__s{38xAkFr)^-$gHGfuj4;JBIJ*xRFlN&AT;N*D1Q^}GwM&;AQF%;qaK zyK?Q+afxb#sK;mV$DX|`@hufi*iRpt*s5>oo}dAe=y2ovx$)-GZ4E&2{q}ByFkK<9JY2F-V$3DRt${3}+Lgh>QfkMK&R%&Ic)Q>=y{l=8`2aAfQ0;7hxN&3iImo%+ z&N#$Z$_M$hB>lh2t{!Hwvl#zB1a1e8_=u=3_oZUnKLtaIZd+{WN~Z#@?R}p8a%{_F ziyZIxp$QG^ZzqKb2UnoO7mlu&bb{Ls!{B60)@H&S*B9ld@YT;TaxGC$f_`>#)F%jC0tNhA1*Oa*M4`db*ubLE_fjEtE4F`o}92fl{45#b&5A{^4h}Xb9b|U$V&Je?Hv5Vu_v5 z{X4_EjUHF->)Ps56oj$n(%IgMAK%io4nJIzk9EZa$QP;&8Crka^|T9a3Dkt`$Z_%8 zL#*vqr46Yv=?ZD(-ppJ2rP`n0TXshaLXsp|P?nHcAU%4V%=i5{XORqirEkk3eCge{ zSVTEp@!b|tV$X_2?_IRho)CiqIn|wH_bjP}0jn7#`Gn=LOg}a?HiAX5e&owK(0(3! zm-%({J8U6MB$MlAQfKstkVGVnLVd=Ny=Eec8d09%VbP;o7FPBhIM0v2pH8*5xuGzt zq_O6LrQXA9a9H@0;3Q$KmE+oKxW@8Bw+0Kb_WyzqA8Zr*$TQId#Bt`acS!emL`ViPI#jCIsHMzsTac)ocF) z)^|5v%*+oiaf4?}ynJR)4NzS!4FRs}@S?8e$8iw-EPKbyiV zS8e@FcGK2VwGVvX3bHi))(%`-;b61{lNLt9UqJr)r{X0+kZD>#6=k9pc6rnM9>b>2 zt1-!HF4_gVQ21@jh{A`ru5L>zocX?!$(sF7F92J6ft*oLar&U&)~TJomhrbQyMq(4wEAp!+uIVqx$_@&uxrKzt*E zGg&XtVzl07>bcMyz%LMjZsaegY$+hPios7_`zU5^atn-n(p)E8-8ToUS(701jkLtf ze$N=U#$O#)-b}R@9IvZ8Q22?8Oe2ET^HpT|}ASZUI# z42r8y4k*(y%8JxsV1hSng;ha$Fg)8ZT{wOK*?BO@b5h5%{;L=pYXr**W28mDs0C&* zA*j4&KIANWtKs^{Wd|4GQa+&fW7-wkp*7vX$wxS^Gan9zGzH4D>_PV65GmDEn=bhX zZC8RecU2@?rpY3K&W7{-k*;gxdl;Pf0@}N-*Z`>hDF?((RajIa@34rF?J*CXk8EU>4<)VQz2rKYh zQm_8qcZ4T{#)-RF4dg^fV6?kf zPO0P42rkIX;qs6@Lh;e7A(e8v0rv1I+HTXBBkfAykLS7p>Fd9pzyJJebZ|-dmeGz6 ztr4I?_57iyZ}f-x{|EYW$g_?m3Lry16C0NN6HaZ*o$)xg!T%GP55jTomaD2@A`CxzBdoqzX@ zIF>2zCLFQ<2fAs>mHpT_SoKUOw_O(k@3#Mv-X54$$Yk7r?yHAYLLKPUYt=07h*P)2 zp7F;#^$X&<-P{h0I}gHW)6s3q=Xz8dmiHQCj4cW-`Thr1i1?2N3(=+H9_HW8Ig^IS zt7C07XYmPFXt8180GDKRKEBO~aE5EAo zyx$x-{6^p&4{aFbb+_@1f=F&^bI-*uJ(r+#Emp)O{pT3R*P+~s=Rk-6NxMtLJ}Esa zAC5kOs&{la14M&A?&nck0d;sJilcb)xt4YRHGr4(`$P{XhpLzRi%P6C2)nLF%IYg# z(umaE4xrsOz6p}&FIoK;6Wh|$ss%JJEQyURXul_ydOmzNc)KONzD6E{cxHhU-eRe} zK5DnxU-}x*>x=%m)pMskdGP6R7|Wp?4+d75!jhoAwKCI8dI>?ocMXI4m0b{E__{;H;shns{ohC2a8y|xDX8cz`F zX6BgH52kWiqpy$`N|U3zI7F8v_$unfg`;)~J6af`A*15}Y?C>oQT&vg2C5wxAWPt*V(2{Blce)MP2C#G`h3vZH z+}|(f)8`-dY%~i^o}|C&kITfB|LH6s77Tsq?p#w_2$L;YYNtWQ{W4hxsVHa6RGS>p zoJHq&+sGF!&6OhQNW3IU^e7c=rHqmts#PQ*Yj6Ui`?NS)hWWJ_z#oCBJ+1^tkU-*;sDwHTjn@ z`rFT=fs-`i3SINU!Qv-)tq@wxBM&o z6-mjH@l&Mqy)+$Lzitq@59ItGm{kvcll;I_Zf~#}eZloe?dakOMcIq`>+J<_vdvXW zk}OqD!BB1g3}C|YFK@H7sGU$y(tQdA1k}e9hsnSo7&Y$qau`?+-AngZzF5&4Y&U?S zBb;VyG)(8VhXxqyztuIr)h`B|wg7x%?)rfqhv86SWGq-(!aV*oW7(3!6 zC}Hzp%P$BQ7TBm_&1W->RcQxJ?Tn*&jM3k1R-=$(RjUua1IC5Ua7k%TCry`1(zu31 zn7`C6+bYn-io5GR$1>I2^_O3y&~Pr1nTBbD(d%$J%V~HvG_r;+p)hd%E@f4AMGbkp zX3|KFL>r1?Ob4mtICzHUOJAK|(B-%!`a~VD=&6-2(MUf5im9BbnJ3{U9Rnw~iH~Y@ zjp6r&{~rw5S$xzN>HtSZzJE$76fxv}&!}I~!{ehaOCCJ)*ChjrTb44_B#Y$B?e6|n z!5^tw?iPKE^N{J#Ev>&%oIyS1h*UGlq7axor^uWMB9#lU5 zGV!2GCN)xl?QDY1b*Lb5@K+ZKO4SFs9i?_Ix?O=|Mu(|2=kC420jh60((b z&=}z_Ren+W$%Cc`g}r>HTY~yOFu2O;GxD^3F2f1H$qkWLb5E`IfgVoUafa7X#>P)A zPb-9?tReRg$HE!P#!KsCN(E-J3540^5xXp>dAStW&_0CQqWE~THDpLY;?ofavxwfi zk&Ax3Jo1H$w}n{-k1H2cz^hJDF!2pQ1Np8$aA&_CfXX!(a!cKIGfgH@U6XTXozo|s z=~Vi)Y~@qCIm4)hO#rBrEW>_xXpSs+?xvgLQhK@rIYYQxcZ>sYFe>@sJ}zDx3Z8pc z(RyvU5;Ak1VWnq&9_>#X0rJGFO#)xA64d$npX7PS={zcL8+b3j@ZGb{y$%$+YveL% z{O&2>2_7+`M9aHx&nUdK#u)`N(bxc;&Vf)~+bmwa*!5DM+9^&uEXk6*%_;+y74_;8(u2;r1(IVrm!2c{h=^OoG|sj~cm2({ry}5} z9^Yov6cHSx7pDntux?>hpcyCVyE(|odZT{HE&wK)`MCL0FhFq=6tX$hBQ`kz2l~?M z)FOBL=zZ;6H|=>oHhSqFKC+C|q;@K2L%?+5dfenDn3H z;-2MsdK9Kh22pzNI_{NuyoMoB7R3oIS+54Hn3Cpy@~$se?vSLH|A_x%Hof&pT0);8 z&*2^+)ctAy1}I}jg?9A|HtWIhJjZG>J;?0aiY#_&Q@9|XRs;PS z+crhxMXta-BdbhXJhN*(6WiQVN3^UHr(#7K%61olm99*1(DFgNg=~2<6eg@#u|OZl zJK#;B;6|Jl6k6!d^?v5gr%4AaiDC|1@^yOBd*Jw`ZQcQDXSe2`gH@2=U6jS`dPr9i z+s{>qaGGfDQSNJ~@6GgwY*jG)2({edL0>=PwH7r_8pnH0T}PBGyZCs66)XK(9w6Q9 z0ov~Ra_8TATHs~y7%HZ!*N-4WT(2#2Nq(4nPJs|*P3tt;hOLAPceNGx2PTZD8~}Hl zQiUIJdoXFGvw@3UO<{Y8b!delz>C{LVFSR9(ddFjQZh{~6N3w@;b)cMkvzt1FKFk5 zqHe9GqRCfN##=%~EJx%%?p)%Y)n@9Dp3cJx<{s&~S3F!D$65k4Ka<*=gzP_0(-MB$ zXu$H*Gv>y_rJK&aZKFI$)ohDICU0t?Q$Aw;h%=2jF0Q!J2Ee(d)Gu_0 zbuKvjgC~-?lx2?B3&y)#IA85OjBg8$Xn8UJ&MJhmDz-Bj_2dwEJeVQQ5uCvFZ?<1b z5z@1E{-DqVf&LK08ajY$dX+WZ=w(!@g-bk8GyJF02*n`c#C<*^nu;JV*D}J$PA^`@ z>8@EMW5iW9!`olSW(9p)dD!@Z3A~B&rBpkmWfP-Zrn*?!^WqBi@%J$je~C2zm9^3^S9^x>(1L0gM`76wkdIx`awP+@(K zjzYE{ag_JGO0ZUx`_PS(_jtrIMb~?FTcI%HL`WK$*AI;C^@KvMdMLFEh^ohn;MelB z7h$fBpRLMoYxY2-o*Hc;Gr&a&R_0!C4!(G&kAyinOzqwIJ5GBG-Xm4Fk06A)STk#I zEvGrSdMaw}q@bXev;M1=EcB^NA?}Q#x9@eG>D#6!9lhc*xeHCq$GVT0Di%z4qU0YX zsXk?C5g2HTU0EM#p~fMw_>Lo*tEri~{BjSvWET(QKAF?x&`AS_%q54!&OldNk3HCnMT=L9*EerE*!1652BX0w^UC&xV%KDY6~$kjZV zU{_(WU3-@s6}5e>rcJ}HkJG8EsDf(h`&+?SjY*^DJQcmj-|ibSstc>#8T$+|Zg+?j z@4$kD;nQ?7Vny?m@7-`YUGGR- zKX*=~PF3aME1brbGs_!4xcT`V$EChm+~Z6@X28BCh*4W|u54C1tO6u%3y+(>RxZp_ zH6x>2xa(VKMHl^NuN!Ez(Sq(TSuC`aJd2f-FK2$r5u4l6eAO27V{_LJbIC2`(;6Fr z1lIW9Jg3NT!fBv=frO|3G~Oi~6_|0a(RgtetOF)DLVy#tFY=LrBPAKHKH3eezxVf) z{mVoH8YOmsnVrD%u*OG1lg6ar9yddb9v=8hS)$_dABqf58=vxVQ?^NY*5yXo2Auvi zKk4T{I4b+WqS@$nde{Ftd5==BM94kvKD7=6SHX%>>3I*LWUyye3P+RU2~&*LMQxno zx<&dZtX+(>jwl)PR5%zT3Usm@s$doUo79^Vz;6lQRlJ|Ta}6Z`BTvz`DOOEcT`zRJt%4d{LPKOs!S@ zHAXA`LrhNm?rrQf@91tJ+So@-{@oyTYZk^m@qY!z1EU4eFmt+z!sdB1x_^32f4~-+ zieX5EwO#wlzTSKTi1>@1EM7i@AzyUv=!(fHbPw)NKD|%vccVtPTJVDBk*b%veTLUY zy7pa}fWjn)tj%nm7TL|!q7vhl673&5OuLTzu+s_^ z`VQ^-dP<=D;=@Hjo+_u6MY_dTRST&uh7H)fTHLNWo5MCHo>lg^ z8T|FCRKd6b5&53)qQolH)=hFw0T+Tn)eR9i&pkU1!nLv|4d@=OfUb7yGdQ2GOMWO! zph<}+^?*Fz?&YazeZ)^-Id!n0($-m7c#IlMor7U=O%KxPl@rfusw!Nn%zhFf7@zpq z&-ZW2)g?e`+nN>xA}nrRYIq|dCc~@#?%PdPixI)6gWQ*3KXmiM~df zYe}_j<~2vSk|;Bm&C|rHnq}M?^5N#7_U_Ljoou)DvBAMqdOWp3IosLu!)~S|Lh8GU z?y}sh<}F6uF$+ucKXDM7%6Uc+jHZDHzJ75!#k!Qg68ol4+$R4@2y+Nl6yolZYdo6% zd^`XuG10oLx4Hd`UUvyPHe1l{{ve}cV+9S;GC19tm(ijL64ERDQVTJzPrC77r_yhj zF=692NWnaI8>Hw}Dl6}`7mtsrt`k z;!_P~QxgqFh;|$S3OhN5K7wAQdFtgni~lk+I3g5jIU43clbnfO3v8N(`=fP3yauL~ zU#)V{+b>G{+ETM0?S+Fck<+xt z{Vn7`heZX26y#k^Jr-__VX@x68%Z@tsKb8ktuy1wob9p_c`519A!>OU#QZ|Mq9Jm7 z;(#ePuY2p!o|$fKU$G5n-ipe>u=r;+?OJtd{WD?tpweE*ls_Xg!+xq)V~}OTu7Bv+BUSu--)zw{{*z8jr{Br_Mf0KHn#=t^ zmRXlDq=w+rjnoH8MO(2gaS+A5r1{yGTB2UVq9&q6Nz+qk^Sp7`^cOFDyKqhPE06&u zs*12}_2>hZ*s}E-Ey>2Z$>n@_CJKssrllApVd%lN2DG719WL)&&r~>eAToNL9?ak1 z*=`cK+LhFQK}+Ph=g#8P8PM&1t;s@sp567sU=XsqY)lVw8tm<0Zq=m#2_n-}!1PZs zuTpESDym~B@iC6(!)nO~QtX-wL8Q8`HVBQO=h|#R&6&tK!s8i3wA@Nrq=<~tJH{7` zYrB)CpSYc`*9yzHs7ZT5(ECQu;2e9;wf6;$k@-7Y4*2z*Fe#QR0jU%4_@2X6AD z%)Qyb_c4+wx~_#$U8IRSp=dG4E8`v+$bF85O=@r+-|4L(tEK%hzxeTlhcP=ZQ97bYrEsGHm-Eva*Dvuj{#TtuXdjHVo+!od_Cy9Lp5*}zx- z8~1UIJ=nyXnWgF?KcRJ>*%;4MgI?sZ91&a5_b_!Y6}MZ8IE$-flqRZS5{}uE`kpj= zmBJiC$H?3Q)eq9Qs52Fe=gQ|`M+#@G`|qoGz!C;x$WJy>pKA1W_TCRylv`gz2$Xu_ zsZ|VK_>P|0is|a?WrOvWD6+`tklllzsP65CNT6jBnxZA*Lkr6K&2Zn|zS(zmJzk;c(nP(= zz=O@(J`~4E^qk|Z)Q8NuS{PzP2V-y-b4JAF{L4iJJ=rs=)(n-ZH6nb~J$Q>|%HgL& zDCt38g690K*bZ9%jGI0hAb9V9Yq3dZ z`1gJ|6dNG89ONn7IhLtyZ<19MlT4QhUG0$PCM>#u=z+bYHIkZKPH|r^m-riOem+qA zu$RSkeDgQrP{YBKX?86-pww++tzrF5MfA}IQt-%{7Hc&~zu0*&!#4tXHTHu>&+`f{ zmvl6jb2P_sWz51va!+Z4uTlgv^>e#dLe4T1={6rM9c52fIN50l8{?yKVTkVR-R9mD zIV;OJe(Q1ZIiHGf=v>Nw9|F2Z-_?*`#iv+!66SqbSm=D1dAR1kfNUE7R1VDziOGdA zf;(Dy1h{(@jIeL!c#=-~(hW?lSxXmG%qR4Z7`6We@W1aP;{m$IsNyeHI{RI}i}n_w!>P_7A(gFnZbd zL*wpn@Mia^B4l5f#r7O~RJ9eML5zai<@ADqN{aMqnUGTU`iZLP*NK~vllzsT`YH-TmVa=)mwnE-B zk5St+t&81Oytc6NM2dLs0{kOkwzh0l09cHcZ~u}=8F9(3zQk-tHJ1UN-o?7SaLV$w zC~cKJM}YHVl-&#CZ@I;-!*L%yoW7wKu{9X&IzoK2ejXU(-)o@?3m8e{7Ldph;YNpB z-as%=nSZ3ov`ewx_$-fK!Dw(_H7MP~&C8H|@|UqAtDh@vnl^9&=>eqF2ax<(s7XSv^mIWnLy} zV2jMApa0h1nAKJE*iz$`aorJXKYSWP<90~uStrhd7F83aR!t{*e~z&&t5sLqtp9jr z>Ogw6nG?BLKXV>ZL{sSAwtRfDQr74BvsTD)*uvh_(6QG*Y@5UKgc~+hu&TIP)(ppX z*L>pIk)E4auD6)jzH{%Y{>bOM_Y6`3x^N>Q3_6~8<4vCI@6^ub}M2z7Y@{Iw%BoPN0eoA z>NW}db{x>m;8j+#yNsA=$sBpi%&bdg$XBU4%gUfhO0^g|{?BXf?Be(s#5<0I>{fTn z9TQVRd@r`fX>!vx2n|#E%G@SZCe2=*<)2p-cp0g>@7VGi~b1CF~pIOtyT`$)y~| zpJME=8y=0p`z5U$ho#eVk3k7k305QZ#aF&xf$Sua4Tw(+wUF?bW8_=@;>zgkt+J)B zZBZMexBo;1hdZipuSZoLtefWF0GHK7XJc0ib%9k_@&>kem1zMwxbd0JMkdcnm5#}& zj?GTWPO>_wrM;=@d{@=OEy|Lp{4sSX6K&=n>ibM~J{IKS>ffp#KxX>A`EJg#Yp$bf zK@dSh_RQj~=$)15Q)UR21vzIgMFH8{;yb)rs+u3T!%`7o#WX;C)Ay8KQ2qG6-fZjc zxi>u}a<+4}URgES3Dmv6Zp(Y7MYY~2H)+x9=ZOpgduS6^bl8fRZ82AU)CPs}v2 z$7MAO86CQSL!*s*ax`L%=c*YwUqL%sestbcpp~&5Z|RU%aXVNQ>TXIu@Gs+QNYPe;ve zv>87msFDZm#tdkO-()3l*pP#5(%MuSpM?`R#hKyqdX}3F5qY-@%2Buq+9 zuvou8;MAry(Rk{JlD!VpvC#qw^$YujAKY@Xt);0Ik7IVdaCKd3A9MzGxlB^ zSQh(-@3<5=c$S^+xJO4{AOuS#H|3_1Y-)aIg!`26jH}!*@j)^%;IoUz7=GnCv-<>O zx?8q3n3Nygu^ltECnsGl4akbFF1*WRFN{~X_SM`SQuV>|t<44FQiiRV6^?^l{;?dX zdkW^=_c&rdye`{gDZ7D=Nb)L`e`-;3WaCvf--+d>&g$v#sp&Jt^?ax~jY+8H_!Ku}{_%*>J#y1w(b+j%ACf3w$hy|4w$(&M^>s&~1QD(>i(rWt;ch?v>Rs>03(Cz8^@YUjoEFx?4Qgv^*`9J9 zn#H%QG{!5HJ83*=ibcDNhs{feB_%jSt2%cnS%>>nWAxJE5*(P6s8&{dt+lANTP`id zP1K`+t@tol&2(>rE#DzEO=i{Q={9uMj0{gS(lr*!kacR@m10po z8Se6a@y|<}+g=?kKORj9RNpVFwRZ*TMfA!9PDq`u=eB$P zsHRR1p$^-&x9|!fxV^^6uBy|vCOlX@UG^vjy`NGd=3%XS!kMs1n3IiuINR14lE~9i z=5iNzqDhNmmJ!(!$W!w1zVU~8_Z1J#J~h0@>Xx2}pHUl#M{kqg2R(kea zNzX>dzI3SR_C0P5mi<{;XxR&l>-CMq9i@6jfkA2~k9PAIlIoN`_Z1(JO*}41em?^Y zY^fUb%c~A{UKG!g7*-`(fd^+7x*P}3kT_RWU3Ub}6#0%%O<`MOzjoZaEwv7la<&IV zeI?M=G7edzG@q2B^`F<6|GHMGXz{9k8Kd-6V9AB1;Dg&UM3~{}ip~LhNZ&VW{9a$y z?C`-K*HwGCmwQDio3h;rl~>>2#6Ou8P8{;!%uFtl6uQ;dePr z!T8CLQ%U)1>sfetST`C(&WY9hx3eF;v41@MF$68MLmF{1O)EpbOQo^Lo2Mvp2QYW zgPbKsV@TaG=!uv6b3D1dNtF?al&epCt;8YuW>_$mbHL6d%KJpex~&^MtIJ(c1Fb0$ z<9e(lWh2c?zNF5>t65wRoRt> zC7HJEX_=<7va-}%XNDxb&4&FF;Aq=*V7Q{%`qTa08Uy@(prF0S3$IC4+t5R+(!L3 zYGNHZi^k~DZYGLU3}Q~l`j5fiZ3E6I?X9osb=EKgFsbcBor%Z@R(il{WVIP&MW7Y5 zcBVg#GMS701Gi9Ha#~waYd+ju*Mtnt$ha+LIvL$`(tJI9kUp{TkAQ;kMf5gYZ|ZOG zFRxk??U11&2PA1(E#sE?H4!sw3+;lGy9=X3@7S~o&GE8*WNU^=qb6aJX%X`4 zK0iI58+ZUoD9=t;7&n?Q&MD^~*3>K9;yjIVL6Uao=TVfRSaye}?7LttnTF-mL*bdt zeeIO{b(fQ@6{`jD`b4QV z`uNC4%XjY^b7zz&HfQ}u-K9A4XW~6}?3`aVw!;{;#dj7;TK}{^@Hwh+mU%knLfi0l zSHYNl7<$6JV7z4Mz%70X{Rtf;nWsLXZ~ceuuWtPOqZjJK2Hn?3v39peD+OM`<;+r# zCBFV=!{NE#07Od=VXJ?45A^vWf&qY5wOXyK>u%O5Pp;X*sEc9CgYedWfu*`RL6=%| z$2TXY8^M>3SWUD)ck4zd=SOEaS?brXdN^850ANuN=r9&bokXsy4*1d{COh!Y4P#@S z46gVG3gY-31xM}_=2hucFtgUVntZ2#B)(^?uv!49d; z20{kKbyoC=|7;{;=}2Q#)vaYx{o&txb|pnzerHGOPsrT80rt?O!DXQ7*Onr~5)~b8 zCeU)0oNMe!<^>MQUUcbaClT-4NIu;)HAAEY^lLN)g?N0cv5?g4udq3YiI#ZF%#hV9 z&&V15)|g9n6!`cFgZ#coi@~i2hPyFo&L#vA*EWADosqW2WPZpRmdKm2&K8m zEo9{0(yruI0^I^adCJjS+Zd}X&_pMwy~chNRv9=o)K|EDt%+XT^<#Cne6#bigH`&5 zaqS!fo?+MA_r>(Zs_<(wVI4@x)UjVUIwgsK82@lN>4=(FHc%F~f( zayms^qTv)Fae)_O4_^TWwT-*Yh1{P0DQdVh)qLJR@pa`bZv%Wq!^xC=4^Q3A`1$Zi zlJ0waAq9OLj}-5-#MYvQ8q;W@m}0fl3BC~f7A>utcMfMfl?wtMKYc8C^A`xQG z$xdRKWbgJPz*_Vz2kL6Wb1!X8vb`NgC=iLwzKYgo&9-m+-f%4?BSus+dI<~3hka2+ z&VOs#db6(^y(W>^ZWXgCINd%u-LG~q&Fz(yMX==T4bw27GDSD7ANHNG#q6x8jhnEK)-4Nq4A3e~=V zYt#&nt~?cD$WtlKF5g~h5OnzHkUMNdC1Dl_xH7h5XYX5@Ur1CF)Ym}w8)-N#A@r$g zTu%^2B_ylMK`8m224M(!mK2z{aP`wYb3^Px@ldztz}EN%Hn?Hc3t#9U9^aq9m+$1Z z85wjYNuG)o?DmMBh*WkxzZv&`n@ZDBi{kBB&S%G7jy+I!?t-!m4h=6^L|n_VC;eFX z{_F^A7Y>N#+M1u2yj-a@Wm%(*<}YAqk0g5b$t?4zQnrag70KAjmfC&bLjmV~Yganc zl@YZ(ca8KQWi&(Wxn=WSL;V=B@gc{Hm2m))Ey@EFi~2b63%iS4LRP;+)Ixw!%R9Mp z^L4HiHW3#e6MrO-Ctz@#EYmizvF8F$h(X#a&9#^4>te6?M# zL@gv4Ls(M;SCkrPJ1e&u>q`YmrnJfxCyA!p_9J0{6>N4?I{OK&>qVZIm(x}vrq~_z z*}u!_c)^!g_YnkuryOm7T`glWiE=%U*r$^zg{w+$X}#CGMPY0TczF`y2~(V`r-hVXWuK zlM&DyS)yyHZvH$X9x3#5T==b8XVHDmy9JFSBZ}|ZINZGL)(Fh_nh5}IR5+Wh_WBd| zd&U$KjT5<@PAD%n!FAOU!^~Ux`fkcx@@lfJ_m}h=3A=y%k&&P+=2L>Y-`Kde9EB{Q zwAfdJo2s(KEq^?Uf`@#29U!$fhoNU3n%2YtY76REP_2zsH8rq0>#jF3Db6t15c$ELP=m74>@t5a0tE7sa|ku>oi7i6C& zX8#?*3S8r4AD5Df4-0#tvFPn_V2t=rp_k^W8aP$n8+`ccUrsI5y_@#dknCBzFcw@L zFz|8yA@LmIdClcm`A}9)87d&GmH{-U&JH2ak;+~~U@}rtp1ploaJ51-=10_D^PY>m z%RwkBCv7oUIVfkxLUEQNlTys-R18;%gf)wWaDgkwy8!Zji%|wZrS;U~J5MA0m4JVm zx*51H<1)0pE|x{!#`hf1+qWTm)ZegulT`0|DHt0_jpcA2{i`VmTe(EN5(UCoV z{b~DFGR#?K>V!+Dver%pKFlzewpUR9E#SW9920!+#b?^*BNh_o5;7$|PxnM~e3oH2 z6B+Mi&4sr=fzn4q+Oht_u22q|C6;sEg(zL}R?cg0w=%%v|5Vep#N#`hyF^k8(Aq~V z)%I|6SW~7Y!rCS2UQ$IkaBDM2zsPWL=S zR*}=`e_+%032+t}(PB5X=YGRfX;7DQFYfXYC42=+n?GCR{%sF&M={2qmHH@0DTK~= zEBuxHShXL2Bbdpf&%6NIm&k1`SCc-@`O+C_(SGnyJlI3w65ntVBfNmJI&_GNE?7Ikbhm%Pn-7Vr*hwA;6S`Ec?eW9@}{+FX2<~n>r~@N-h42c*>Zb z2%$}+X^^Zp1=a4i}44rVC(x|4r$2D)8_qC z$!D&PDP^tB4ZQftBcqg*_d16N(ueltGL|DVEO>+#MmifO)*CX`U8_8^YBIbVBWKc$ zz-dgIIy&IQc=nN&(+{7->1_K6fYgpHv|(Bs*MAfl>sG(SD#GgVRlTYaQ~^rB`ce#A z`^R_fR9wU`$Q}_XD7*p#OQv3IbGM&K1fqNs;0`{Q(@QUsOj!>auaRTbiKxF5$*1~o zE@RUjQ{HG_h}MB~{k!E0VHv5eHj%Yc1{D#xPbbN=0upW3bK=tYAc%D1t$wn?8u>ks_z74 zCAlgd?VV$|;vg^KxkZvTk0e`!5an4QgL+s&Zq*XsH@Ajm5U)N&6l(y5D^yKu!XHRe zLYq4=i4SX-Cknk0U1Vme;mE+B{X!MwL!Vd6%M!JIoMWzj07g;#do(Ot_xt*5&Q#t@ z_5@RMU6u(6TU`lLsEt)6L(gdY>@RZJ;wn*d64XbFk#R+B zsEAkFRaj(R(Bvcs2#dOE3r4FAyjFH^dk(IL^ll%B+diwhE$8e>4Q3E}ZU!WTMv9BE z2g^|)B7>K`7Ow2FED#$*93}9^G_i1J)Iwd7t=OP3%es4Cm8QBiZ(;=`;aQgCUna}( zF~z#_Jxd$K(JYp|({P@Tb zoBjLzf~JR+YrII9=vA6NZgR|2ZR^Et?4X1gZ!uRp=={PK9$0=Kd^K0m+zN`{Zr?gd z2vVrA4cDID*BCMJ%I&((1UHnvvt?=9OU!8R!p9<|>2D7quZqOk#X2U@qNC1qLlfb-yl$+Tk<+X}Xqql&>tvV1fAf!2 z3z9HIV&fgvD}o6j3<7MvJS-T&k#SEWmJVG1yy^d)hzhyq+nq*G^XUII{oU*1KHTyH z+@hxgIxJH+St`5p0d@)aaqZd9h(*l>+qjM0@MUXoeko~>*b6e>f_4r1+^?ur#S{xK(o#9isbLPgB{pPjQBFGp<70o{T-pKj`SS_ zNiGoj|53{#JCZ+PUPxaDz$NXvRMm-TU~nl&P~)fNf~o1N@AT)3P3Oqh_P9>i#x?oF zV{h+R{&^WPM?Dk-r;>6cD=GtkV#3bq%cW$h#Sn1MBYBANu%hCkQM}%p05}bJNl$&` z%g^ukA~@4M1e{`A@How@@NAWk9E<$0H@~g6L~FHz0$QT^bHue_f%K1}_t|!@$yIyL zQOXC#{V^amnHnI^jQyYv^|1pEfb*-T5q5tHk0XLZmc@yshtjkqQE#wCrzr3~njWEj)HVR-l;dX|W*H!R^WU2FEoW5C?-|2Sj z_H)Z356I)s2O|Iqf&FhI^bg#85qIHLRRn>v3nMK9=8+8o=$JFjAjcAhEB#_`l0ENv zYfUw9p3I$qel_e>wve(U2rkwpDIYfo)v_a(RU&2r)hWKI!W$u>wFz_+>eu1N< z0j|r1*goj8aU(Q;X&HKh_jn+a+7KX}XNpihT1yL5KSbr%k5nn$_D9f&!Ps9xKLL;1 M39sYje_goqADhs3jQ{`u literal 0 HcmV?d00001 diff --git a/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_5.png b/docs/proposal/jakarta_messaging_3_1_shared_subscription/img/img_5.png new file mode 100644 index 0000000000000000000000000000000000000000..b89c0d089d805bcf87c738cbc136200c597e3bad GIT binary patch literal 87992 zcmb@uXH-*N`~N8*AR?fmv`|E(DZM3h6#;1ph`03KL?jS;s3HndLa)-LOYa?|hTf%z z-a84s{twUnd*-)h)~tCoYrn|iKyps@zS_RNpN+pNRG#cE-Q8QaZjmXxee>?tE&Q`v zxA5rh5CK;VOL1Mm5BioeGO7wPGEY?PY|Je`o87v_5#|`m{}$iq5&C+J_BDy<8{uyd zLhtXfuTU|*G~{-iA7t#3oQOU#t;)U$9GH1DI6@;_vGh<)$^QO(p}*1gbMLT+QiWpN zu6rlfC%BpOnd2{vA?=!-gekhk%EBV62HN`jwV=@XHolC!^?x#-MUXsis&taaF@_J+ zlJLK9(@sFm2swo~x#yPKuW#9SXdEn){*!p%BakOJjGz7lVb^W^)vDEpja#;kcf9EI zjcwmbpp%Ls>i+CQx5Ho2?4E4oOJeS6KHV8AmsZ5Z^;xP4yy3yB!B4$uF5X6Dro4Vd zgm}Y$4TDu-D4U3ADvXR7hysc}etk_YLqfO!c|O4Je+zNvRhK1Va~A9%A#%Hien_1+ zIL71lhLoXU|9-pvx;W{dL)KyC?bVWZ!~f{T-<_B?AarSRs}dl$T|M8X7a`6$2mgqwB}xkn58W45QoDU?(N~?D}f=x(2v#y#{qEiO3MZ+E063AA31p z6$Qv+lGJ|9tm4TRKwMGys#i z)*fpr$y7Q8eDYG5e5Qvx?Y1xcU~C=cb(ZK+^+QV5*W_soxEw~E6EB5lZ}gys!Er1}?Spg#;a+7Oak?7Sa@Ncb&u#u~4|9tD1+rqU z()~bB3fyhVBH07_C5_(OFTEwgSn^j5>V@0G>f06CHPxAUBk;(0dGOflsm2yM^aO#FH=>LK zv)4VyUywCoq*tMbsR*kcO5y8{k6v*f9-tSb;O)w3vp>zui0&T7t*JfFoo4#*C8ci+7WD@A&`sx64Xx!5OQAEHP#bCL?z5_D$B! z#`4n*W-=ePABCuyDYS)TtTru|aEZ%VE!%k}eSr7%56>w|bW;p!S0@-Kjhz(t(VOU_ zq@$B#ax<*lmu2jOkiu7*S0cejO5TKYsZ$l!N`*U-`e_eZ^au;5zN7}ZFaJE+G2S{v ziI^vZ%`56i~(7fgSC3d#+dDP%kAM z{lMfqzm08v++}Bfr)W7$RIlu^T5_crBv58k#w`#Pg1KW-QyiU)jm%OGNwd!@ z0X(4uy43F4<{kGd{xF4Q(^i4XPFJRVOGFk78~ZGi7jZQ1Yc__ech}l(hiH|-i*0^T zZsq291VE>IcwU1&INQ}mSXn=$fEKNl!KS#M#N(+qc3<2;Hes!g`AA^jFO@FQbTA_I zPa$WGcN})3?-7P((ZuRbO>qHpzti@|U-X+e2eilO`SC# zQmIuezg88uZqyvcjrX2$<>uAR(Dm7^>^5FsS+@9JaFAk@0~LZyt~=C|gp9d(ADd_u zXRpj!_|+-|mX}KuEsMLh_J8!11IzPhhK;l4eEGV%$CQ?JZ2sqp--a;k(*%bS<008!PV(n)^j*@d@8;hB z9faCnP~8S=%t^@*?xNppzqL~S9fsa3Yt%tQn!Wh{j#@Zg!uhV}aV=(0o+%Mp8=uy= zDPsNbC*jRlDSl~xuv+|~MTAu7KcmMCPk(}WZfWtPa)9gKQRGCWIgYO4T1XmwYwc^v z{|=WC*cxT37ILANqb^#_bvjf`!N##`Mw|WW)Hr=biG_UT(X_zk2P>J+7N65D!X;am zX*N676%oDiWJKlxBd+u(fDnjyD4{$GnAHDEPtr0676pGvm!)U$)5hg zC62dTx3vS*qTIx6a+e3)YPZy;RR6U7godY|oxx}QM+BIcCmIya&SU(!zs|AU4iTj3 zhc*5Q(VY1n0ysp2#?mUb^=)rVGrQ`)cDlIn;gV*oEi9RA;VCz)DYI=r^XmmY8g`lT zR^bDepM)0`cTCjcs7$4Hq8j+%Sms9^S*At{y!$uEqtN5CmofdhJflF;R=0q$S|SR9GJ+6OkN_wjs0WYmPEslYO(V zsQCn44zWk0N??Z}=KS?6_{EAPcb+dM+lpngz0sMCkI46HqrLdIIEy-Fiyru9DhT)8 ziO(YC6#oy+SF{UDu-=j-XjpZHKlriP(kf)5^~$7i)r!fe@}|Dweh2>i67(14`b#Cx zUU045rIhzG^)_p)FLZqvxkdD0ReVxpR6GP8q<3`R5lVK)TC)s(;aS!gqnB$x=#-XR z)xK5HQ?u_yc#@2FfPT6q$CR*T%3fL<_$Xd&Y<#l&+T^9a6r+7=9$5!n^PtoQ15RdC z2K>2cvOLP+rnWg)|FzvNG57Nq$n!Ut=G5gG5FUbhiyP#5S6U=$taC+&-)Cc$>1@N~ z36uqy@62(qNb{ddXIk2-P-!AM;=iUTlTn-b>qNpkUQ7@)9R8wC{YF_z=al=GWY!g2 z126M=DIT25eo7+8Db(gAb+G6=0L~*O$96tC({NSCDf(*LK7M3mv5@T`EV4 z`Ovl*QeVT<$UEzY11iaeYBffIoJW7+7+dmv;p6Ukg)4(7k$b1N)z2T%syh;hj}UE( z$X04Ir35ZUo_&H$5fjzsOjSr8PYViXANhTqHcHrdpY{9I!({P$Y=^3bX-ft4t7aTu z<_qXq{hD3ZeZ$Z@H3Oha=UI>ARJL^SQXm#vx;MIbDJKr+LIuCwcq9Wbk_gPIuu+Vqu~mb|b~&;hJ+NsT14Zj=q#eFQU#e z4_YMb&6Tfn9HDL8ieQFt>5F*^j1S{MN?COB#S*(bit=_*f+6miDMx6(dOE+CSy#hL zNp;)!S73MUq->rv%)oYbTClbv#---_b;YPt3pFh+WDr$u#E2o1ZkMHTW)X^&ePYl7 ztm&UuoGi2cRzKywuyDify-$iQMyG6MfUN1=^t{`wByY@X5UswoqG>A;$`?{$Ss9&RFcW45Z;1P6f7?0;-JaMlSB(-B+a zl)Nb2o%$_5B18wi%+Dv%=zKr;**mKo&~}n>B`O6Z7JQmgj`+qvPpaM??_`n?=}v5B zS;<)=%EPV1K!ikGbOq8ZjNj6_30fL_gmp`=$W~+(8P{oy8<0bPo0lE6lSf>EI7!dK zYrOo* zu^nYIrGR!6#-e1Rn4@lz`^#ln-KD_%`JT$2_qwe*&XHyfLdJPAb(Z{@y&(d|b-vyF z>WF9`S1qbfe7x_lZ_)R0y%8%_?m;6!fAYBga){}KO{gde`kw@utG4-X<<^`P0%4dYul=WxM>~uM~E7)Gd=(TXN7qajgv@9>T+b-Du_6#$Kc6=r6y zmD$wpDBH07sq9<)Z=DgCsOn~T7QFT%SPK8aiu~<#T$eIhu7kiP6iflnYsyUpo&8R$ zQOYWN)$?g{g@{vD9$b*>T-W-h4w3yT;gQt7L?l)tAaGgHE`Lgb^FmVu-F6YRU|gN* z+8fbc^OtlZZTCmusSfLodlPw|xU~;ito-DiPd^aIz?812rx7m7EhZ`sNvY0B6y{v5 z0y6ul5*tsXjTH}}z5QlwcI`@+kyA4Dib7b$8t8HkY(y(bU<<^-RI_iS*UGC%y*c$s zXi{KOzoE?B0++!`kM!kFu#(qr$98E1L)Po6G@)^$YT;3s>$_h zKe4HINZUCaMR_g~K%u@QYbFMY44O+VEM%&`{Hl+7H#Mu8X^`-J0@S$!-<6=yVQ5`1bMrqKP{}1EdE@g z3%=SXurYn8C>Aa+YFd2NE*d{Zaum~eVdi|(oVe*R)CeZ+2gz0!Gaj@~1WT{#4ikHF zSzWS9HA*$Xo*So0a4>CDHD$Uh2jtH$A8N9#vP_+|Egf2ymsv@@k2eHwztIgko%|tz zra=@xAckp0QnE;Q;mhmYwj?8NWDCSDachH3ACA7Od~bA}HTvOXu%WvkzUDPqFtpfkg)@=-E|@q|8GUqOfS zty0)z-uT-1o9e-#{Yn_aBi1SJxQCw>Ie2wiL}vyf1{^krU?#G}T$Rfo6a+=8XtIJZ zwU})<21sm4ttN22P3^G_{mTuFEPngLT!VVIi$$hcy+yjQanW5}>R9S_1e=zOa)h7z zP-{+qmAlIla^rl!PfAEMWW)orsSg${zkuCi_^UI$K*d1Qbzc#8Pd~3;$YsMqN|kqU zPU^TC=DeYvT;sMPm>f4rl#J}7bFc2!D{$-9nHkdC`%0}WVmEEbyU_1CT3Vw~xA2%V zSa~Cr3%8F5anp7FgGanoMvWXnx$_L^cAaf#G)`jHq0WlecPxPfjuC}6J&mVCjMABU z-aPfh4rpf27ZYf3v5+V&E6+e=CVkoo+wnX|V{)|d*wj&{swhtWp-!Z?=6o20uvSPq z8}M9`T;_zWHyqdRvPzkSl0p-!5pKirGb5?&*MmS&RR8h#w?_T%+UCavMWx$F42MGE zUiGiw$vzSIKV38}Nj;>W-`_Pk$3MrrY=zK<`)rlZlnz*GIkr%(RR%ffprkN=8!Y-t z+I!MK9{s{pR5_qcYOL=0C&%2Wipj=or!eC%Q<*=nOdpsap1Gg2Fzx7{c4;`xp7uGv z6H5|cl{+sjph~%}daz)+kBw<`KmJ23RV0p!>rWse^5`T) zh#r#iE^?s1M6EvI{Y#4o5`ImR#l-`yaqTQ7 z@%mzQx8Wdz*Kw@6&v6Li)kLa4un%*kpA`!aqW)?{z5h|Cy%K52P2K&5))--qC?SzP zpXjf7p}!x1xde>W;Va7LyvR?IE3xdR)KiQ-k@sUSgvqOY9*OWuXdk|NkX9!U>y?m@ zMuCkLCt4lsweiI5yo;%M|5FgM4m1!ojt{b=CG#J|RJF2(_l)o8@-8uxUiLXvlsbkW zdrXE@h~2kTUNS92dH{wr=>Zthy40RGg~O`YSal7|6B~h1v2h)S@75phB%QRfv(v-6Vl9gSkk3KxN$-lF#v!D7Yp*dTOy}e2;>eAdF;w19 z{iX7OjCyZ`cVan-2x~AwSFAC247D{|e{oDV-V36xhFj5BQH2u2J=90s9!#gA@ZWGT z-N1of4`;QSL88$_$+=OOzE2iYq8WWhf z3vVVo7vG>@TL9(>P0OQUY7Se?aPL-hlzZA&m;Z*g5I!BWdU-U( zEBbeQZgQ-_=`5tl3-GDo1+;g|jaMzTmVA($!=qTUwIeBX!)(^9N?)IH_PM)s$S{WM zvwPdU3sygyBbJ8rM!C(N&sd_wS4I0L=Cfr;U~ZGcHO_U54C_k?5f;!gk~|cojJolp zP5tbXMy>XS!Bspz`AYok#fpRA-Zyp$=H*4x(v;av{bNP3ch@%`wgXI$M-8H3*JF2T zpTe&yuKFCao8(|jc@CK06lRGHtPsggoA9o*vuI5aS~LurJ!yBa>uX1(fiN8TgR2m< zs2ESPCGa2)!@JUk=YS_o?;p&MHC{P7dz0!uDSrS;18@K8yD88+gF{?eS%zM48(g1_ z)xIz|?aG1L+5jPV*td6>ssLz54=XrtzO+c9A21)yterV50Ai_gt;3yHv&=DBD3w~eQfwT2~pvdI}n1liN-N6z~Th}o8+HBBUF*CO&67tS7UGaCdX8aFrz8fZ7S*_l*T6|{>5 zy)Y2%itY;xT#mG4VAVqEbJ-->^+ZH3`ImnV=9ly>CAU;bvTIeyM+0xrgyxwkmGqP* zyw4LzU+9eq@28<{4zswgB!*S#21QtSw(9srN^F3B54)y{BQEtO^BfKX5evuhy?F*M z=Z*-G^Wo$xp-ph9*0qZ}JFi)pw33Z?i2D*t>6PA!*6z!mGx28HORk3Hkq^wL)-EKocu3qhNB4T8V@o~x z?0xg}ib$n`S-*V}uE*5^>EtIqZ8R#rTRIaKe+b4e4e{7ADcuCDp!7;X^LL|{nfYjP zXpgI(!%?=pW=cBlj^jeCKIpASni~5$N6-zI%|1v-_arB}i~({-+&t7waV zq;_=Q4>VBr7-E8CG-NcM%VcSJ2}YeoU3OmW*uc#%UnrGY0BNEW<2)-lF<-YTX|u6= zC1*G)ewov7#W|SQs1c*s(MjmVjVExi7U&N?aTG-ak8N=Ao)osuAsvwC4{peRj_Gv(~&upjI znQVTI+7`)xbd_|lpY3(*99DPSOb$j(l1->@RnXP&_h-*a^-PHt!n?v&u$hx6;ZMKe zL0PFHAHj~{(GK>yE3q6*7*C+?`mkBXRWf3I@TM`jaQ&pQFncjFfRY>G#YdalU!^A(MV{;jkt6(apK_7oZhahhI0chv2gKrbtCiJSt=vNB|B{I`FNdO zi^RO1bpxV$RdTmlog8-8)j}fikNOTS`P$vJ_ZtLHa$V42dBNx1(B;?ohKT(`xfO=2 zKM8PkU!h&B{r=l%=XIoF*9)%mL-x%EMt}MS!nvr2%^D5Zs9=#)%ESFwTDJkYq>868 zV@Iy+b(@LXks(r}3JL6Fs?Z{mgfrYH@~pW!w$}5FglI}vVv&hE$hwCe#D5|eZ(jq| zhkdhcSFhEBOBIDN@zHh5r!-054EtxQVPb<$j4vo%K7EmNdG6hFc|6;=_&(_iSrI7= z+_CtV%T|Ja4Loq5haZxSObldaT$`EQ+H(Kg>r8qxM-gP%J{7~Wbq%dxirc2eS^JFA zFglmEAMAG|^yU;uYxc@~$HMNcb$3d_7=Ijt%@pLZF6CQOvsX^83omQFFBTe%&%Tt< z-!r+AxY>KLBA57QaCMq95`YGv@;&5|k{?hC+R9uu@BhCy*fX$7Rm@t^lxY$C!Dq<`#9idYx?I!OJ0XACo6TPr z;c*ULaiIv;KJvi3#;e9p&eWTCzFf(O!Y9|Hu{s~AG(V8G3a|JuT^OQsDK9%c-%)4L zXQ7VgPd`3frDI14-}NKU8Yb|+iWIN!4#xQocaY(ibne2Clt)0$G1Bg~<>3rraV`$j zD7wj#8KfdiD zUPK)%c$5ZB(A;R{Df9cYsWf*gOt(QGgIVLr^co5)a&M9K+WYjl<|ykz@A@_?CUWm* zphtkd?plGz2_BxyS}Pd^;^>L~)UU!q*T1=J#0l{~4UT5n_N=6)g;EGS+S;t7$_UW2 zmjPncf0~J5iFJ%^y^p|;mvAi)M-ZWzlu%Am0;=zF95&1~gEVC}#WQ`y()W5jv$)UN@~;I|ioq6(2w3Eal)$1;fV18< zm=EabaOD!}+S3%)Nux?F3?+^WQ`4QZ5J{pFY(-Mu=!pfQnBy&AT`3ad2@=%tX~qP) z79tmK9TjdrBc`xO4Dq-OjZ$}~H>a1Rf7$)i&((Gh5si36A+h)8Jw?`ZkL1j?IpZ(N zU3<4*u==^wy3M|b@5hbTm+NB|5TkayG0Ua0*qTA_5IuxgPRLo9pM9XpK)x4irrVBv zay-39mntz{Vwj(vsF`%S^US<=YWyV=N;=*sji1v`VJ3UoCzQk3DLMq zJiL3eJi5ECV@fb}U7OGlNl#!Sm;qw@>_>m#4NEXMneV*Cep(h&^TWOr?RI!TeNGuH z;?mGE=i3|-jmaJG7J0o+x^W8@UrjPm=w3J=YoSN34mXfF|8lV@T4j+AvOJ+%uD z+VJ7%f2RE3Nxv54k*4TuA8sU`*f)Ai`ka)Ols?d$wA&i*tOFI*`vbMh<6N!eTpEwb zTl^M248vLa@6(l+^Cwru0J@7$<%_CtE;DTdWt|uyW4i>)%^#Y`Fgj|B8aX9M5uoVJ zZ>Ep%8H2^m?v?J$z1gQDDfHiSCEOfju5W42WxhN=?A%|HQDM;DLKCk;b`tK`2lVGi zF%4oBDT3(=_vgwG5icd{k~(rcuHl|`&u&`aOIuhq-b3s>CzS7S?f}RbF7{5_)my<78>Y=RlG%a{LM0X(*@3f5U+)l75SN4FQ&~E=ZrNoxP^$=|2l{nc! zabxJ4sdWwb`;X|GsCr<;d(82Fe<*v|;};N?1!tEkqTo_JCA`n*Z2&ebHfJ$xL5<)3=oLMQkMx?KE~Ukt z{6t@q6P@dHt@N!}h{ecE;t>bm2kySuK3qTt^&|CTkFlVudF+su$n~TyA{a39{_(*G zd2FSkEwdSeanHgDwwo7X11Y%_h^Ciq?~f{Q$Z#MCw02=2Ku2l3wJkzyxJV~Ex1Uq| z9C|?uaZk7}$OP$Z>y#I9+;gQUe1R1sY5b@Eq-~;2tF?1Eirlj{$yJnjCG)E&UbJ(q zsGh|>p}f|{Dv?jtCkQICZP3CxgGWE)l?d!nXxnaA8l^|y3(jf26fn5}R1gNX6+-Y> zf~Cn6#H&bd`BE9v{ubU2j=P$HQH#I-B>={_TuQxK?ked1|lh7!cTW zu(2#A!U?`+yHre4NQW)E1+v@dJKsAX1zYS@y?ROIq2%=>(B^TG_$DD(%00KW_rW&W zf7SBVc|3%?Vt_QREX*@~x9W^Z=&h3>*FK8UJD&02h#K{A^>Lv~+X%VAU#RzM=aP|a z7bWE~bKsXcy5L@nI#}umxANri*0m=2jm2En1vhoFv$tMcq+MpIR6NX>CE=Ma-VR1} z#Dk^Sn2@;9-su$O9NmIh#mUf^)CPC!f^t$R4YK~)O~|A~8{ZNZch>ZG1Jqp_{E+)E z0nZq*KA94u_b&NK$IouQ_nGgBmwmf9oL{`#Zd0KLPtKBhqDNx|ge>7X1)E!!EjL1-8XpxVAaKkTxcF*C*WXFpbf*q$a3FOT0Jy*d? zBwf&dFtyNagLl>|^<7|Xf!Ma3iVx2ZzvNCi!gPY4{#5s#!}4;3f!0uO0f_RkyjLzU zI@n~~)8c{su+zMgy6rscb|wiGZ6tfL&@xSJdYJqVflmA4Hx$c0^Ck4|hh>erU9QgW6I{ddY6C-7)_gZt*B{^)Q1j+)B(mnT9_QPKI;TK!OU+;0* zcufAIH$NqZ9ie9U5y&14?_lYZeMwGp$PU#q)n-SS$x28}epT*LjSf88)B7s$XSpIq zo?%5T`epUU;c+=Et|5IVpRtJg$YF*3s?A6l&LnATwS z*E((+*g>EUFC(QKg8?hE)ip61tTB={U9o?@kqS?{rlmII;eg3EaW~Xa70sp`yAg4X9v z35|&TeI;5!*|5!AaL+8BqWp?u_$aL38NcVcqkEIZ+A}W@yYWg=&3QPIH8gAb2Mcf4 zdsvrz!!k=!l@n`$?m}Ha+xg)J8Se&ubc#w&f6Y9)2Im>#_&h>DN3;lRNc}lOrHB&& z4;l}ovB3VyUn*w$(!KoUF@c%EEhVNdjU>wimLnDlwWZanoEm1*xWECFNXagbB6yN7 z6STb&>GGfMSVK!FX63JSj{!hWjo+I!4}cr$GSSbjZC4G!)g97i_zmvf+s;0HJAz^a zP$iUSP+$qN2v4*z0~}09bLWU#sNGVQ+nUs*VR_jKaJnrBxwx zT!Gqh+keQO`O=qFUo`tJ^4~5#rCXf*)ye{EDEr3L75`kOTJ)cakqhmb_T;ST*n6h3 z%0W65v-cvujwb{}!Nl5e2a4EFW}hy?q3$uNq%wBC*iTqGFSLGmp?!e`Hlw;ebMC1x zwky0)c{ne3;b|{IE!eEB)lLS|E&IITUlmJ!;{S#teVD1BgKDW&Xd$ofQbK(Mp7_m6?2d)pPaAlD6xK?S*owD z?{c~*4?3QgHTleU<94n)LOe<4MaJNNyYN;&@RI$0&9=PpEyuI^93|JvSA`BdOho=nzEPYhcqg z{*h`af2b8wd3gxi?U?OL~JXd7eI@Bv4dj#c%%7F_AY3{?TR2R!QGI1`Si;?fT z<$@&m+F4Y3Os*rBG@-i6|53=z(v4G$gX9&b`3R|(WA>`^nD1%MvVJI_q@R3U`bVzh ziwTltz3!pj)b#Yefkkm5lz2obeg(O==*q!^HEFQ=RTazFB1(l5==!O>`d(K*#4_hF zJes|n{QrlwpE~IseP{UeF59fXRben6hTSybBaDc2)d3o3+UNKd>_>nb4&|>SCMjE893=rui&~Z(4ji=HObb zv&!j7B2Us@{O-?qE9}=C>v8?==FX#~Nx}>qt%&-b8rWZyW4MS1KxOPE9hGHe80Sgq z2p3%$hnNJ5W}X8g%CH`wfG7|@-oxD@zmny#3)?9U@%VhU8i;TYMZ2!f{BaQ|;NsgM z*z1W&UR-E+sZX{#PmQ$;TFwhRqfP}Ks`A?96I2r&#sXiPewb`xytP78UVl|?ptoXl zlCKTk5sQY;DVe-sD4Tlfm}W!I(jlx$jmPNm8$NZ?ua%J!&ZDWONiypK>c&4PyrvCD zjbHE4oF^(Youuq8Pbm&hA3h{zzX4O6OHHb>Bg~>K=(@aKPH)v9#-Nb1wY1tBl^J8k zV;|>G^9cOfdiZ1M9|5f|qKbx6H$u{e=jEmWyL|GcX(`HJRAKTUao0WT{6+$l+%lrD z!jz$}9j$zd! zLOz*b>dQ9I)~g4b9m@#1fM651QQ+%I9b;tI+Am}pnQPTTDi}(pKCNYX9vRILzZkUc zxJgte!Z_gfh2_kX0DF5b74w~83AIiX?YgDzV8yg(9K}sTZ!46D*J|eegH83CeSw=7 zrmWHj0Ezz>Zy}dZ(*+2~S)zRosOP-@acsI%2|0-yp#*c>a4cL4STUK2_gLMEu;t<_ zBUq|P5SH?h8Mko4aowsl?!Ad}x`o6dZS5-X7V)vyV{ZdM&XSRGgAYvNnCXLKs@?*6 zJmoT{d{V3?53nO%f7>=@x?BQ;9u7BsUHt+!m@Z1&rl+v8YwAM_=Zh|?7rZh5CQ<@l}wS>vokJGX8(?Vt6O9;zQM-LwlBX#eM zNaWQ02qSKHN%u!?C5O{@`vebD|}e^BP!uZxFLF~Z;QawXbD*;S`I z?Ga+?q{^RPN9wig+?^GrG0V1D+E$zvQ?%2gXkYRPv{NjxqkWLPx~(YI#VREm!+FU3 zqDwz=n$vNNPmQU4>jO!jI&>KFAdhgH5{L67h`>GdEMExBb`H2_?EnA(>HA7L^~O_6 z3Y{#BGRh(IbrU_^OV~&js2G4+s`~R5xocX zbCF`{%XJtAUb{WI-Eh9sX_>0P^+BG&@%?dS<7NKMd5EEfd2^Epy#cBwjbz>EnkDy* z=q$#KziWti6$!#&^LJZlS#9DEN$zWMB*6p>`0XNRwIm_stQBzr-wM4hH`g!cwx=qc zB_0yDm@yP9mqmy$E{0J=t3y`et;ekj>uVx_vne?)Ipi+U!bvoj$=Vn{1}9Ci$7V^O zKMQ&Ez=@Qbqc6p{OD$Y#R}0(t0U!#RyFP73r?>&)E;Rm2uVV(+Ay%KK(Z%!&jALDY zaaD8cyVM0;C)3rQ-_8y3u@Cj|+vqThYgahK?0Sw}UaU&P8ao_rnw==74K@9Dbt$zM z56klo0qz~w5n&^`k7BgI+xA6`XSq1*xh1X$Wlfw0YR_F=9XDnn@6FmN4vl8A{G*>A z1QkB*^4D~^q2wZX+^J3PTWLQxc%Cfz6VWGSt+ygP36RW16;vG8D}N_9_;N(3S%4PV zK;zlp!Xzpg($ZoKpuWJ6e)M*4wRPQ1d!D{&5com46i@t}ok>EY4@}m(sK(UwT#5d4 zgSldm!NZ%ucBbmi(RYLhC-TBH9&x&m0%Eoe@7Hvh=++JXQQYm zDS|IEjg*r+3EPq3aeFe5p#bRAI*kDzQ47co7wbOKhz^1A z4{Hp&F6bSg;|%Y5%}DLe9~mO>$^CYypXVIY*?=D#XuoNW6;MM*EV?>3Q}U1JfTA^{)^(8dok(})m{Fr&scNR@V@T9*9+%EsN~7&v4^=h7Tu z(WrHm39h$126Bw6g&wx)0XvO6@nXljEU(iTul4Le!+7;ymzGpO2i~4k`HUu}v%5U0 z=}b!qf;a)Bzj2RsB?HA^BGO(8ualM~u2dEfr%U5+KWdmjQ0WORu{2fxvP%HnX{*+~ zsK>Z$0=*d46p9+x)EF^i5D#`htDz>of(xKHIYQh4YD1;fP-F8A8}((M<>A%&ZZcH6 zZQ%{VxLa=BrYDz}h8L;ee{(>B>itX|r2AFYBpH`?#IcpZ`Q&;HfZEQ^=a~>lHKR