Return-Path: X-Original-To: apmail-flink-user-archive@minotaur.apache.org Delivered-To: apmail-flink-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 293F9185A5 for ; Fri, 27 Nov 2015 12:03:54 +0000 (UTC) Received: (qmail 41579 invoked by uid 500); 27 Nov 2015 12:03:54 -0000 Delivered-To: apmail-flink-user-archive@flink.apache.org Received: (qmail 41503 invoked by uid 500); 27 Nov 2015 12:03:53 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@flink.apache.org Delivered-To: mailing list user@flink.apache.org Received: (qmail 41493 invoked by uid 99); 27 Nov 2015 12:03:53 -0000 Received: from Unknown (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 27 Nov 2015 12:03:53 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 43D25C598A for ; Fri, 27 Nov 2015 12:03:53 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 3.001 X-Spam-Level: *** X-Spam-Status: No, score=3.001 tagged_above=-999 required=6.31 tests=[DC_PNG_UNO_LARGO=0.001, DKIM_SIGNED=0.1, DKIM_VALID=-0.1, HEADER_FROM_DIFFERENT_DOMAINS=0.001, HTML_MESSAGE=3, RCVD_IN_MSPIKE_H2=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-us-east.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id A0xNQtiKKHYg for ; Fri, 27 Nov 2015 12:03:44 +0000 (UTC) Received: from mail-qk0-f171.google.com (mail-qk0-f171.google.com [209.85.220.171]) by mx1-us-east.apache.org (ASF Mail Server at mx1-us-east.apache.org) with ESMTPS id 6020A429A6 for ; Fri, 27 Nov 2015 12:03:44 +0000 (UTC) Received: by qkas77 with SMTP id s77so35808830qka.0 for ; Fri, 27 Nov 2015 04:03:44 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:sender:in-reply-to:references:date:message-id:subject :from:to:content-type; bh=9bwSs2EN8nAKTuu41TuAJGWQ7rFWqrDVDZ/WjTIB2A8=; b=Yg+l7Sqj/mZ0Ta8HmG1GMpmawqcRyvK0NbBtEMhRdbcUQ77eTIXLDClH3lck9EpYCf Z2vprDVV3Ah8vPxaylZfG+gd1mGSevJibEKVIwVM7Lf5DgPFDBZ2I3BVMlXXTOlkItFz FPi60RsVPhcEDP1sAKQY8wE0qGTxGz+bvgFz2ZJVC6yCSlTWNWgZ2yvdEBYksbYFJwGv hqRz8xBbAkUNOwecjxb0PfKuXeCMO8viPr014sd6E6uWK+G6ezecWOfCGw+FOFInPWeA GLBb10mdb/dYg7QdFpnGVuMUBN5dwY10bqtbsYsPor9Z+e/tsDveY4UwsZU2xWJIm4tq qwsw== MIME-Version: 1.0 X-Received: by 10.55.73.201 with SMTP id w192mr20832466qka.17.1448625824020; Fri, 27 Nov 2015 04:03:44 -0800 (PST) Sender: ewenstephan@gmail.com Received: by 10.55.147.1 with HTTP; Fri, 27 Nov 2015 04:03:43 -0800 (PST) In-Reply-To: References: <02984684-1F52-43E4-A6E5-A2C597ECA82A@apache.org> <69231889-9EAF-4A73-800E-89FE4DD1C30F@apache.org> Date: Fri, 27 Nov 2015 13:03:43 +0100 X-Google-Sender-Auth: q540oppb2uSM4hPn1GdWHnOL5DM Message-ID: Subject: Re: finite subset of an infinite data stream From: Stephan Ewen To: user@flink.apache.org Content-Type: multipart/related; boundary=001a114a889ca777a50525847b3f --001a114a889ca777a50525847b3f Content-Type: multipart/alternative; boundary=001a114a889ca777a20525847b3e --001a114a889ca777a20525847b3e Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: quoted-printable Hi Roman! Is it possible to do the following: You have a queue (Kafka) with the user requests. Then you have a Flink job that reads from that queue and does a map() over the query stream. In the map() function, you do the call to the database, like this: val queryStream : DataStream[Query] =3D readQueriesFromKafka(); val resultStream : DataStream[Result] =3D queryStream.map(new RichMapFunction[Query, Result]() { var connection: DbConnection =3D _ val queryStatement: PreparedStatement =3D _ def open(cfg: Configuration) : Unit =3D { connection =3D // connect to data base query =3D // prepare query statement } def map(query: Query) : Result =3D { connection.runQuery(queryStatement, query.param1, query.param2, ...); } } Since the queries have quite a bit of latency, you could try and run them with a very parallelism, or use a threadpool executor or so... Greetings, Stephan On Thu, Nov 26, 2015 at 11:06 AM, rss rss wrote: > Hello Robert, > > thank you for the answer. > > I understand the idea of stream usage but I don't understand how to use > it in my task. Aljoscha wrote an example of data parts extraction by > external queries. It is very useful but not enough. > > I have conditional queries. Simple example 'get data from the specified > period' by a specified client (or sensor, or something other)'. Flink > streaming API allows to configure a stream to access to a persistent > storage. E.g. some DBMS with SQL interface. In this case I have to > configure the stream with SQL query in a constructor like 'select * from > data where timestamp > now() - 1day AND clientId =3D=3D id'. But '1 day' = and > 'id' are parameters from the input query. I don't able to configure all > possible steams to produce all possible data combinations. Therefore I > decided to use batch part for running a transformation process for each > input query with a list of conditions. If it possible I will glad to use > streaming API only. But there are no ideas other than batch API usage. > > Best regards, > Roman > > > 2015-11-25 21:51 GMT+04:00 Robert Metzger : > >> Hi Roman, >> >> I've looked through your code here: >> https://github.com/rssdev10/flink-experiments/blob/init_by_kafka_sink_ka= fka/src/main/java/proto/flink/batch/Consumer.java >> >> The implementation is not very efficient. Flink is not like spark where >> the job's driver is constantly interacting with the program running in t= he >> cluster. >> Flink will generate a plan from your code and submit it to the cluster >> [1]. Your code is submitting a new plan for every message in Kafka into = the >> cluster. >> It will be faster to process the data locally. >> >> I still think that you can use the DataStream API of Flink. Maybe use >> this example as a starting point: >> >> Properties properties =3D new Properties();properties.setProperty("boots= trap.servers", "localhost:9092");properties.setProperty("zookeeper.connect"= , "localhost:2181");properties.setProperty("group.id", "test");DataStream stream =3D env >> .addSource(new FlinkKafkaConsumer082<>("topic", new SimpleStringSchema(= ), properties)) >> >> >> then, use the "stream" object to perform your transformations. >> >> The output format looks good! >> >> >> [1] >> https://ci.apache.org/projects/flink/flink-docs-master/internals/general= _arch.html >> >> >> On Sun, Nov 22, 2015 at 8:52 PM, rss rss wrote: >> >>> Hello, >>> >>> I have prepared a prototype of the batch subsystem that described in >>> the previous mail. >>> https://github.com/rssdev10/flink-experiments/tree/init_by_kafka_sink_k= afka >>> It does not contain correct Kafka's serialization/deserialization becau= se I >>> didn't see how to do it yet. But it contains a code for running Flink b= atch >>> processing by a message from Kafka queue - >>> https://github.com/rssdev10/flink-experiments/blob/init_by_kafka_sink_k= afka/src/main/java/proto/flink/batch/Consumer.java >>> . This code is based on Kafka=E2=80=99s examples. >>> >>> The question is following. Is it correct implementation of the Flink >>> batch API activation? See method Consumer::run. >>> >>> Also I added a sink for Kafka - >>> https://github.com/rssdev10/flink-experiments/blob/init_by_kafka_sink_k= afka/src/main/java/proto/flink/batch/KafkaOutputFormat.java >>> Is it correct? Unfortunately the Flink's documentation does not contain >>> examples of a custom sink implementation >>> https://ci.apache.org/projects/flink/flink-docs-master/apis/programming= _guide.html#data-sinks >>> . >>> >>> The program works at least in the mode of one queue looped by self >>> consumer-producer and activated by the only message from Kafka's consol= e. >>> >>> Best regards, >>> Roman >>> >>> 2015-11-20 16:05 GMT+03:00 rss rss : >>> >>>> Hello Aljoscha, >>>> >>>> Thanks, I looked your code. I think, It is really useful for getting >>>> real time data from some sensors. And as a simple example it may be >>>> considered in a modern Internet of Thing context. E.g. there are some >>>> temperature sensor or sensor of water flow; and I want to build simple >>>> application when the data flow from the sensors is saved to persistent >>>> storage but a small real time buffer I want to use for visualizing on = a >>>> display by a query. >>>> >>>> But at the same time my question have a second part. I need to link >>>> the real time data with data from persistent storage. And I don't see = how >>>> your example may help in this. Let the input query contains some data = fetch >>>> condition. In this case we have to build a separate DataSet or DataStr= eam >>>> to a persistent storage with specified conditions. It may be SQL or si= mple >>>> map(...).filter("something"). But main obstacle is how to configure ne= w >>>> data processing schema been inside the current stream transformation. = E.g. >>>> being inside the connected query stream map function. >>>> >>>> Week ago I have prepared other schema of my task solving with >>>> separation of streaming and batch subsystems. See the attached image. = It >>>> may be changed accordingly your example but I don't see other way to >>>> resolve the task than separate queries to persistent storage in batch = part. >>>> >>>> [image: =D0=92=D1=81=D1=82=D1=80=D0=BE=D0=B5=D0=BD=D0=BD=D0=BE=D0=B5 = =D0=B8=D0=B7=D0=BE=D0=B1=D1=80=D0=B0=D0=B6=D0=B5=D0=BD=D0=B8=D0=B5 1] >>>> >>>> And note, this schema describes an idea about how to emulate a real >>>> time buffer by means of Kafka. Windowed stream infinitely produces dat= a >>>> sequences and sinks ones into an external queue with limited storing t= ime >>>> or without storing in whole. Any consumers connected to the queue are >>>> received an actual data. I don't like this idea because it is excess >>>> network communication but it looks workable. >>>> >>>> BTW: it is something like lambda/kappa architecture implementation. = I >>>> don't like these terms but actually it is. >>>> >>>> Best regards, >>>> Roman >>>> >>>> >>>> 2015-11-20 13:26 GMT+04:00 Aljoscha Krettek : >>>> >>>>> Hi, >>>>> I=E2=80=99m very sorry, yes you would need my custom branch: >>>>> https://github.com/aljoscha/flink/commits/state-enhance >>>>> >>>>> Cheers, >>>>> Aljoscha >>>>> > On 20 Nov 2015, at 10:13, rss rss wrote: >>>>> > >>>>> > Hello Aljoscha, >>>>> > >>>>> > very thanks. I tried to build your example but have an obstacle >>>>> with org.apache.flink.runtime.state.AbstractStateBackend class. Where= to >>>>> get it? I guess it stored in your local branch only. Would you please= to >>>>> send me patches for public branch or share the branch with me? >>>>> > >>>>> > Best regards, >>>>> > Roman >>>>> > >>>>> > >>>>> > 2015-11-18 17:24 GMT+04:00 Aljoscha Krettek : >>>>> > Hi, >>>>> > I wrote a little example that could be what you are looking for: >>>>> https://github.com/dataArtisans/query-window-example >>>>> > >>>>> > It basically implements a window operator with a modifiable window >>>>> size that also allows querying the current accumulated window content= s >>>>> using a second input stream. >>>>> > >>>>> > There is a README file in the github repository, but please let me >>>>> know if you need further explanations. >>>>> > >>>>> > Cheers, >>>>> > Aljoscha >>>>> > >>>>> > > On 18 Nov 2015, at 12:02, Robert Metzger >>>>> wrote: >>>>> > > >>>>> > > Hi Roman, >>>>> > > >>>>> > > I've updated the documentation. It seems that it got out of sync. >>>>> Thank you for notifying us about this. >>>>> > > >>>>> > > My colleague Aljoscha has some experimental code that is probably >>>>> doing what you are looking for: A standing window (your RT-buffer) th= at you >>>>> can query using a secondary stream (your user's queries). >>>>> > > He'll post the code soon to this email thread. >>>>> > > >>>>> > > Regards, >>>>> > > Robert >>>>> > > >>>>> > > >>>>> > > On Wed, Nov 11, 2015 at 2:51 PM, rss rss >>>>> wrote: >>>>> > > Hello, >>>>> > > >>>>> > > thanks, Stephan, but triggers are not that I searched. And BTW, >>>>> the documentation is obsolete. There is no Count class now. I found >>>>> CountTrigger only. >>>>> > > >>>>> > > Thanks Robert, your example may be useful for me but in some >>>>> other point. I mentioned "union" as an ordinary union of similar data= . It >>>>> is the same as "union" in the datastream documentation. >>>>> > > >>>>> > > The task is very simple. We have an infinite stream of data fro= m >>>>> sensors, billing system etc. There is no matter what it is but it is >>>>> infinite. We have to store the data in any persistent storage to be a= ble to >>>>> make analytical queries later. And there is a stream of user's analyt= ical >>>>> queries. But the stream of input data is big and time of saving in th= e >>>>> persistent storage is big too. And we have not a very fast bigdata OL= TP >>>>> storage. That is the data extracted from the persistent storage by th= e >>>>> user's requests probably will not contain actual data. We have to hav= e some >>>>> real time buffer (RT-Buffer in the schema) with actual data and have = to >>>>> union it with the data processing results from persistent storage (I = don't >>>>> speak about data deduplication and ordering now.). And of course the = user's >>>>> query are unpredictable regarding data filtering conditions. >>>>> > > >>>>> > > The attached schema is attempt to understand how it may be >>>>> implemented with Flink. I tried to imagine how to implement it by Fli= nk's >>>>> streaming API but found obstacles. This schema is not first variant. = It >>>>> contains separated driver program to configure new jobs by user's que= ries. >>>>> The reason I not found a way how to link the stream of user's queries= with >>>>> further data processing. But it is some near to >>>>> https://gist.github.com/fhueske/4ea5422edb5820915fa4 >>>>> > > >>>>> > > >>>>> > > >>>>> > > >>>>> > > The main question is how to process each user's query combining >>>>> it with actual data from the real time buffer and batch request to th= e >>>>> persistent storage. Unfortunately I not found a decision in Streaming= API >>>>> only. >>>>> > > >>>>> > > Regards, >>>>> > > Roman >>>>> > > >>>>> > > 2015-11-11 15:45 GMT+04:00 Robert Metzger : >>>>> > > I think what you call "union" is a "connected stream" in Flink. >>>>> Have a look at this example: >>>>> https://gist.github.com/fhueske/4ea5422edb5820915fa4 >>>>> > > It shows how to dynamically update a list of filters by external >>>>> requests. >>>>> > > Maybe that's what you are looking for? >>>>> > > >>>>> > > >>>>> > > >>>>> > > On Wed, Nov 11, 2015 at 12:15 PM, Stephan Ewen >>>>> wrote: >>>>> > > Hi! >>>>> > > >>>>> > > I don not really understand what exactly you want to do, >>>>> especially the "union an infinite real time data stream with filtered >>>>> persistent data where the condition of filtering is provided by exter= nal >>>>> requests". >>>>> > > >>>>> > > If you want to work on substreams in general, there are two >>>>> options: >>>>> > > >>>>> > > 1) Create the substream in a streaming window. You can "cut" the >>>>> stream based on special records/events that signal that the subsequen= ce is >>>>> done. Have a look at the "Trigger" class for windows, it can react to >>>>> elements and their contents: >>>>> > > >>>>> > > >>>>> https://ci.apache.org/projects/flink/flink-docs-master/apis/streaming= _guide.html#windows-on-keyed-data-streams >>>>> (secion on Advanced Windowing). >>>>> > > >>>>> > > >>>>> > > 2) You can trigger sequences of batch jobs. The batch job data >>>>> source (input format) can decide when to stop consuming the stream, a= t >>>>> which point the remainder of the transformations run, and the batch j= ob >>>>> finishes. >>>>> > > You can already run new transformation chains after each call to >>>>> "env.execute()", once the execution finished, to implement the sequen= ce of >>>>> batch jobs. >>>>> > > >>>>> > > >>>>> > > I would try and go for the windowing solution if that works, >>>>> because that will give you better fault tolerance / high availability= . In >>>>> the repeated batch jobs case, you need to worry yourself about what h= appens >>>>> when the driver program (that calls env.execute()) fails. >>>>> > > >>>>> > > >>>>> > > Hope that helps... >>>>> > > >>>>> > > Greetings, >>>>> > > Stephan >>>>> > > >>>>> > > >>>>> > > >>>>> > > On Mon, Nov 9, 2015 at 1:24 PM, rss rss >>>>> wrote: >>>>> > > Hello, >>>>> > > >>>>> > > thanks for the answer but windows produce periodical results. I >>>>> used your example but the data source is changed to TCP stream: >>>>> > > >>>>> > > DataStream text =3D >>>>> env.socketTextStream("localhost", 2015, '\n'); >>>>> > > DataStream> wordCounts =3D >>>>> > > text >>>>> > > .flatMap(new LineSplitter()) >>>>> > > .keyBy(0) >>>>> > > .timeWindow(Time.of(5, TimeUnit.SECONDS)) >>>>> > > .sum(1); >>>>> > > >>>>> > > wordCounts.print(); >>>>> > > env.execute("WordCount Example"); >>>>> > > >>>>> > > I see an infinite results printing instead of the only list. >>>>> > > >>>>> > > The data source is following script: >>>>> > > ----------------------------------------------------- >>>>> > > #!/usr/bin/env ruby >>>>> > > >>>>> > > require 'socket' >>>>> > > >>>>> > > server =3D TCPServer.new 2015 >>>>> > > loop do >>>>> > > Thread.start(server.accept) do |client| >>>>> > > puts Time.now.to_s + ': New client!' >>>>> > > loop do >>>>> > > client.puts "#{Time.now} #{[*('A'..'Z')].sample(3).join}" >>>>> > > sleep rand(1000)/1000.0 >>>>> > > end >>>>> > > client.close >>>>> > > end >>>>> > > end >>>>> > > ----------------------------------------------------- >>>>> > > >>>>> > > My purpose is to union an infinite real time data stream with >>>>> filtered persistent data where the condition of filtering is provided= by >>>>> external requests. And the only result of union is interested. In thi= s case >>>>> I guess I need a way to terminate the stream. May be I wrong. >>>>> > > >>>>> > > Moreover it should be possible to link the streams by next >>>>> request with other filtering criteria. That is create new data >>>>> transformation chain after running of env.execute("WordCount Example"= ). Is >>>>> it possible now? If not, is it possible with minimal changes of the c= ore of >>>>> Flink? >>>>> > > >>>>> > > Regards, >>>>> > > Roman >>>>> > > >>>>> > > 2015-11-09 12:34 GMT+04:00 Stephan Ewen : >>>>> > > Hi! >>>>> > > >>>>> > > If you want to work on subsets of streams, the answer is usually >>>>> to use windows, "stream.keyBy(...).timeWindow(Time.of(1, MINUTE))". >>>>> > > >>>>> > > The transformations that you want to make, do they fit into a >>>>> window function? >>>>> > > >>>>> > > There are thoughts to introduce something like global time window= s >>>>> across the entire stream, inside which you can work more in a batch-s= tyle, >>>>> but that is quite an extensive change to the core. >>>>> > > >>>>> > > Greetings, >>>>> > > Stephan >>>>> > > >>>>> > > >>>>> > > On Sun, Nov 8, 2015 at 5:15 PM, rss rss >>>>> wrote: >>>>> > > Hello, >>>>> > > >>>>> > > >>>>> > > I need to extract a finite subset like a data buffer from an >>>>> infinite data stream. The best way for me is to obtain a finite strea= m with >>>>> data accumulated for a 1minute before (as example). But I not found a= ny >>>>> existing technique to do it. >>>>> > > >>>>> > > >>>>> > > As a possible ways how to do something near to a stream=E2=80=99s= subset I >>>>> see following cases: >>>>> > > >>>>> > > - some transformation operation like =E2=80=98take_while= =E2=80=99 that >>>>> produces new stream but able to switch one to FINISHED state. Unfortu= nately >>>>> I not found how to switch the state of a stream from a user code of >>>>> transformation functions; >>>>> > > >>>>> > > - new DataStream or StreamSource constructors which allo= w >>>>> to connect a data processing chain to the source stream. It may be >>>>> something like mentioned take_while transform function or modified >>>>> StreamSource.run method with data from the source stream. >>>>> > > >>>>> > > >>>>> > > That is I have two questions. >>>>> > > >>>>> > > 1) Is there any technique to extract accumulated data from a >>>>> stream as a stream (to union it with another stream)? This is like pu= re >>>>> buffer mode. >>>>> > > >>>>> > > 2) If the answer to first question is negative, is there >>>>> something like take_while transformation or should I think about cust= om >>>>> implementation of it? Is it possible to implement it without modifica= tion >>>>> of the core of Flink? >>>>> > > >>>>> > > >>>>> > > Regards, >>>>> > > >>>>> > > Roman >>>>> > > >>>>> > > >>>>> > > >>>>> > > >>>>> > > >>>>> > > >>>>> > > >>>>> > >>>>> > >>>>> >>>>> >>>> >>> >> > --001a114a889ca777a20525847b3e Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
Hi Roman!

Is it possible to do the foll= owing:

You have a queue (Kafka) with the user requ= ests. Then you have a Flink job that reads from that queue and does a map()= over the query stream. In the map() function, you do the call to the datab= ase, like this:

val queryStream : DataStream[Query= ] =3D readQueriesFromKafka();

val resultStream : D= ataStream[Result] =3D queryStream.map(new RichMapFunction[Query, Result]() = {

=C2=A0 var connection: DbConnection =3D _
<= div>=C2=A0 val queryStatement: PreparedStatement =3D _

=
=C2=A0 def open(cfg: Configuration) : Unit =3D {
=C2=A0 =C2= =A0 connection =3D // connect to data base
=C2=A0 =C2=A0 query = =3D // prepare query statement
=C2=A0 }

= =C2=A0 def map(query: Query) : Result =3D {
=C2=A0 =C2=A0 connect= ion.runQuery(queryStatement, query.param1, query.param2, ...);
= =C2=A0 }
}

Since the queries have quite = a bit of latency, you could try and run them with a very parallelism, or us= e a threadpool executor or so...

Greetings,
<= div>Stephan


On Thu, Nov 26, 2015 at 11:06 AM, rss rss <rssdev10@= gmail.com> wrote:
Hello Robert,

=C2=A0 thank you for the answer.= =C2=A0

=C2=A0 I understand the idea of stream usag= e but I don't understand how to use it=C2=A0in my task. Aljoscha wrote = an example of data parts extraction by external queries. It is very useful = but not enough.

=C2=A0 I have conditional queries.= Simple example 'get data from the specified period' by a specified= client (or sensor, or something other)'. Flink streaming API allows to= configure a stream to access to a persistent storage. E.g. some DBMS with = SQL interface. In this case I have to configure the stream with SQL query i= n a constructor like 'select * from data where timestamp > now() - 1= day AND clientId =3D=3D id'. But '1 day' and 'id' are p= arameters from the input query. I don't able to configure all possible = steams to produce all possible data combinations. Therefore I decided to us= e batch part for running a transformation process for each input query with= a list of conditions. If it possible I will glad to use streaming API only= . But there are no ideas other than batch API usage.

Best regards,
Roman


2015-11-25 21:51= GMT+04:00 Robert Metzger <rmetzger@apache.org>:
Hi Roman,


The implementation is not very efficient. Fli= nk is not like spark where the job's driver is constantly interacting w= ith the program running in the cluster.
Flink will generate a pla= n from your code and submit it to the cluster [1]. Your code is submitting = a new plan for every message in Kafka into the cluster.
It will b= e faster to process the data locally.

I still thin= k that you can use the DataStream API of Flink. Maybe use this example as a= starting point:

<=
span>Properties properties =3D new Properties();
properties.setProperty("bootstrap.servers"=
;, "localhost:9092");
properties.setProperty("zookeeper.connect"=
;, "localhost:2181");
properties.setProperty("group.id", "test");
DataStream<St=
ring> stream=
 =3D env
	.addSource(new FlinkKafkaConsumer082<>(=
"topic", new SimpleStringSchema(), properties))
=

then, use the "stream" obj= ect to perform your transformations.=C2=A0

The out= put format looks good!




On Sun, Nov 22,= 2015 at 8:52 PM, rss rss <rssdev10@gmail.com> wrote:
Hello,

=C2= =A0 I have prepared a prototype of the batch subsystem that described in th= e previous mail.=20 https://github.com/rssdev10/flink-expe= riments/tree/init_by_kafka_sink_kafka It does not contain correct Kafka= 's serialization/deserialization because I didn't see how to do it yet. But it contains a code for= running Flink batch processing by a message from Kafka queue - https://github.com/rssdev10/flink-experiments/blob/init_by_kafka_si= nk_kafka/src/main/java/proto/flink/batch/Consumer.java . This code is based on Kafka=E2=80=99s examples.

=C2=A0= The question is following. Is it correct implementation of the Flink batch= API activation? See method Consumer::run.

=C2=A0 Also I added= a sink for Kafka - https://github.com/rssdev10/flink-experiment= s/blob/init_by_kafka_sink_kafka/src/main/java/proto/flink/batch/KafkaOutput= Format.java
Is it correct? Unfortunately the Flink's documentat= ion does not contain examples of a custom sink implementation https://ci.apache.org/projects/flink/fl= ink-docs-master/apis/programming_guide.html#data-sinks .

= =C2=A0 The program works at least in the mode of one queue looped by self c= onsumer-producer and activated by the only message from Kafka's console= .

Best regards,
Roman

2015-11-20 16:05 GMT+03:00 rss rss <rssdev10@gma= il.com>:
Hello=C2=A0Aljoscha,

=C2=A0 Tha= nks, I looked your code. I think, It is really useful for getting real time= data from some sensors. And as a simple example it may be considered in a = modern Internet of Thing context. E.g. there are some temperature sensor or= sensor of water flow; and I want to build simple application when the data= flow from the sensors is saved to persistent storage but a small real time= buffer I want to use for visualizing on a display by a query.
=C2=A0 But at the same time my question have a second part. I = need to link the real time data with data from persistent storage. And I do= n't see how your example may help in this. Let the input query contains= some data fetch condition. In this case we have to build a separate DataSe= t or DataStream to a persistent storage with specified conditions. It may b= e SQL or simple map(...).filter("something"). But main obstacle i= s how to configure new data processing schema been inside the current strea= m transformation. E.g. being inside the connected query stream map function= .

=C2=A0 Week ago I have prepared other schema of = my task solving with separation of streaming and batch subsystems. See the = attached image. It may be changed accordingly your example but I don't = see other way to resolve the task than separate queries to persistent stora= ge in batch part.

3D"=D0=92=D1=81=D1=82=D1=80=D0=BE=D0=B5=D0=BD=D0=BD=D0=BE=D0=B5

=C2=A0 And note, this= schema describes an idea about how to emulate a real time buffer by means = of Kafka. Windowed stream infinitely produces data sequences and sinks ones= into an external queue with limited storing time or without storing in who= le. Any=C2=A0consumers=C2=A0connected to the queue are received an actual d= ata. I don't like this idea because it is excess network communication = but it looks workable.

=C2=A0 BTW: it is something= like lambda/kappa architecture implementation. I don't like these term= s but actually it is.

Best regards,
Roma= n
=C2=A0=C2=A0
2015-11-20 13:26 GMT+04:00 Aljoscha Krettek <aljoscha@apache.org>:
Hi,
I=E2=80=99m very sorry, yes you would need my custom branch: https://github.com/aljoscha/flink/commits/state-enhance=

Cheers,
Aljoscha
> On 20 Nov 2015, at 10:13, rss rss <rssdev10@gmail.com> wrote:
>
> Hello Aljoscha,
>
>=C2=A0 =C2=A0very thanks. I tried to build your example but have an obs= tacle with org.apache.flink.runtime.state.AbstractStateBackend class. Where= to get it? I guess it stored in your local branch only. Would you please t= o send me patches for public branch or share the branch with me?
>
> Best regards,
> Roman
>
>
> 2015-11-18 17:24 GMT+04:00 Aljoscha Krettek <aljoscha@apache.org>:
> Hi,
> I wrote a little example that could be what you are looking for: https://github.com/dataArtisans/query-window-example<= /a>
>
> It basically implements a window operator with a modifiable window siz= e that also allows querying the current accumulated window contents using a= second input stream.
>
> There is a README file in the github repository, but please let me kno= w if you need further explanations.
>
> Cheers,
> Aljoscha
>
> > On 18 Nov 2015, at 12:02, Robert Metzger <
rmetzger@apache.org> wrote:
> >
> > Hi Roman,
> >
> > I've updated the documentation. It seems that it got out of s= ync. Thank you for notifying us about this.
> >
> > My colleague Aljoscha has some experimental code that is probably= doing what you are looking for: A standing window (your RT-buffer) that yo= u can query using a secondary stream (your user's queries).
> > He'll post the code soon to this email thread.
> >
> > Regards,
> > Robert
> >
> >
> > On Wed, Nov 11, 2015 at 2:51 PM, rss rss <rssdev10@gmail.com> wrote:
> > Hello,
> >
> >=C2=A0 =C2=A0thanks, Stephan, but triggers are not that I searched= . And BTW, the documentation is obsolete. There is no Count class now. I fo= und CountTrigger only.
> >
> >=C2=A0 =C2=A0Thanks Robert, your example may be useful for me but = in some other point. I mentioned "union" as an ordinary union of = similar data. It is the same as "union" in the datastream documen= tation.
> >
> >=C2=A0 =C2=A0The task is very simple. We have an infinite stream o= f data from sensors, billing system etc. There is no matter what it is but = it is infinite. We have to store the data in any persistent storage to be a= ble to make analytical queries later. And there is a stream of user's a= nalytical queries. But the stream of input data is big and time of saving i= n the persistent storage is big too. And we have not a very fast bigdata OL= TP storage. That is the data extracted from the persistent storage by the u= ser's requests probably will not contain actual data. We have to have s= ome real time buffer (RT-Buffer in the schema) with actual data and have to= union it with the data processing results from persistent storage (I don&#= 39;t speak about data deduplication and ordering now.). And of course the u= ser's query are unpredictable regarding data filtering conditions.
> >
> >=C2=A0 =C2=A0The attached schema is attempt to understand how it m= ay be implemented with Flink. I tried to imagine how to implement it by Fli= nk's streaming API but found obstacles. This schema is not first varian= t. It contains separated driver program to configure new jobs by user's= queries. The reason I not found a way how to link the stream of user's= queries with further data processing. But it is some near to https://gist.github.com/fhueske/4ea5422edb5820915fa4
> >
> >
> > <flink_streams.png>
> >
> >=C2=A0 =C2=A0The main question is how to process each user's q= uery combining it with actual data from the real time buffer and batch requ= est to the persistent storage. Unfortunately I not found a decision in Stre= aming API only.
> >
> > Regards,
> > Roman
> >
> > 2015-11-11 15:45 GMT+04:00 Robert Metzger <rmetzger@apache.org>:
> > I think what you call "union" is a "connected stre= am" in Flink. Have a look at this example: = https://gist.github.com/fhueske/4ea5422edb5820915fa4
> > It shows how to dynamically update a list of filters by external = requests.
> > Maybe that's what you are looking for?
> >
> >
> >
> > On Wed, Nov 11, 2015 at 12:15 PM, Stephan Ewen <sewen@apache.org> wrote:
> > Hi!
> >
> > I don not really understand what exactly you want to do, especial= ly the "union an infinite real time data stream with filtered persiste= nt data where the condition of filtering is provided by external requests&q= uot;.
> >
> > If you want to work on substreams in general, there are two optio= ns:
> >
> > 1) Create the substream in a streaming window. You can "cut&= quot; the stream based on special records/events that signal that the subse= quence is done. Have a look at the "Trigger" class for windows, i= t can react to elements and their contents:
> >
> > https://ci.apache.org/projects/flink/flink-docs-master/= apis/streaming_guide.html#windows-on-keyed-data-streams (secion on Adva= nced Windowing).
> >
> >
> > 2) You can trigger sequences of batch jobs. The batch job data so= urce (input format) can decide when to stop consuming the stream, at which = point the remainder of the transformations run, and the batch job finishes.=
> > You can already run new transformation chains after each call to = "env.execute()", once the execution finished, to implement the se= quence of batch jobs.
> >
> >
> > I would try and go for the windowing solution if that works, beca= use that will give you better fault tolerance / high availability. In the r= epeated batch jobs case, you need to worry yourself about what happens when= the driver program (that calls env.execute()) fails.
> >
> >
> > Hope that helps...
> >
> > Greetings,
> > Stephan
> >
> >
> >
> > On Mon, Nov 9, 2015 at 1:24 PM, rss rss <rssdev10@gmail.com> wrote:
> > Hello,
> >
> >=C2=A0 =C2=A0thanks for the answer but windows produce periodical = results. I used your example but the data source is changed to TCP stream:<= br> > >
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0DataStream<String> text = =3D env.socketTextStream("localhost", 2015, '\n');
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0DataStream<Tuple2<String, = Integer>> wordCounts =3D
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0text=
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0.fla= tMap(new LineSplitter())
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0.key= By(0)
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0.tim= eWindow(Time.of(5, TimeUnit.SECONDS))
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0.sum= (1);
> >
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0wordCounts.print();
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0env.execute("WordCount Exam= ple");
> >
> >=C2=A0 I see an infinite results printing instead of the only list= .
> >
> >=C2=A0 The data source is following script:
> > -----------------------------------------------------
> > #!/usr/bin/env ruby
> >
> > require 'socket'
> >
> > server =3D TCPServer.new 2015
> > loop do
> >=C2=A0 =C2=A0Thread.start(server.accept) do |client|
> >=C2=A0 =C2=A0 =C2=A0puts Time.now.to_s + ': New client!' > >=C2=A0 =C2=A0 =C2=A0loop do
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0client.puts "#{Time.now} #{[*('= ;A'..'Z')].sample(3).join}"
> >=C2=A0 =C2=A0 =C2=A0 =C2=A0sleep rand(1000)/1000.0
> >=C2=A0 =C2=A0 =C2=A0end
> >=C2=A0 =C2=A0 =C2=A0client.close
> >=C2=A0 =C2=A0end
> > end
> > -----------------------------------------------------
> >
> >=C2=A0 =C2=A0My purpose is to union an infinite real time data str= eam with filtered persistent data where the condition of filtering is provi= ded by external requests. And the only result of union is interested. In th= is case I guess I need a way to terminate the stream. May be I wrong.
> >
> >=C2=A0 =C2=A0Moreover it should be possible to link the streams by= next request with other filtering criteria. That is create new data transf= ormation chain after running of env.execute("WordCount Example").= Is it possible now? If not, is it possible with minimal changes of the cor= e of Flink?
> >
> > Regards,
> > Roman
> >
> > 2015-11-09 12:34 GMT+04:00 Stephan Ewen <sewen@apache.org>:
> > Hi!
> >
> > If you want to work on subsets of streams, the answer is usually = to use windows, "stream.keyBy(...).timeWindow(Time.of(1, MINUTE))"= ;.
> >
> > The transformations that you want to make, do they fit into a win= dow function?
> >
> > There are thoughts to introduce something like global time window= s across the entire stream, inside which you can work more in a batch-style= , but that is quite an extensive change to the core.
> >
> > Greetings,
> > Stephan
> >
> >
> > On Sun, Nov 8, 2015 at 5:15 PM, rss rss <rssdev10@gmail.com> wrote:
> > Hello,
> >
> >
> > I need to extract a finite subset like a data buffer from an infi= nite data stream. The best way for me is to obtain a finite stream with dat= a accumulated for a 1minute before (as example). But I not found any existi= ng technique to do it.
> >
> >
> > As a possible ways how to do something near to a stream=E2=80=99s= subset I see following cases:
> >
> > -=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 some transformation operation= like =E2=80=98take_while=E2=80=99 that produces new stream but able to swi= tch one to FINISHED state. Unfortunately I not found how to switch the stat= e of a stream from a user code of transformation functions;
> >
> > -=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 new DataStream or StreamSourc= e constructors which allow to connect a data processing chain to the source= stream. It may be something like mentioned take_while transform function o= r modified StreamSource.run method with data from the source stream.
> >
> >
> > That is I have two questions.
> >
> > 1)=C2=A0 =C2=A0 =C2=A0 Is there any technique to extract accumula= ted data from a stream as a stream (to union it with another stream)? This = is like pure buffer mode.
> >
> > 2)=C2=A0 =C2=A0 =C2=A0 If the answer to first question is negativ= e, is there something like take_while transformation or should I think abou= t custom implementation of it? Is it possible to implement it without modif= ication of the core of Flink?
> >
> >
> > Regards,
> >
> > Roman
> >
> >
> >
> >
> >
> >
> >
>
>






--001a114a889ca777a20525847b3e-- --001a114a889ca777a50525847b3f Content-Type: image/png; name="flink_stream_batch.png" Content-Disposition: inline; filename="flink_stream_batch.png" Content-Transfer-Encoding: base64 Content-ID: X-Attachment-Id: ii_15124efd793ad866 iVBORw0KGgoAAAANSUhEUgAAA4kAAAFrCAIAAADzX1uiAAAACXBIWXMAAA7EAAAOxAGVKw4bAAAA B3RJTUUH3wsUDCwd1DoLkAAAAB1pVFh0Q29tbWVudAAAAAAAQ3JlYXRlZCB3aXRoIEdJTVBkLmUH AAAgAElEQVR42uzdd5hkVZ0//s8JN1cOXZ27pyfAzDCRNDBEJYiKwIgsgo64C4qyovs1LCqLurAG UFddHgO4gogurguKEgyAhCEPcZgcunumc6h88wm/PwrGcUC+/HS/EjyvZ555um7fOlV16nTV+557 zrlISgmKoiiKoiiK8hqAVRUoiqIoiqIoKpsqiqIoiqIoisqmiqIoiqIoisqmiqIoiqIoiqKyqaIo iqIoiqKyqaIoiqIoiqKobKooiqIoiqKobKooiqIoiqIoKpsqiqIoiqIoKpsqiqIoiqIoisqmiqIo iqIoisqmiqIoiqIoiqKyqaIoiqIoiqKyqaIoiqIoiqKobKooiqIoiqKobKooiqIoiqIoKpsqiqIo iqIoKpsqiqIoiqIoisqmymvWiSeeiF6wd6Prui/eiPbx4o3/K/dljP3ZBf4l9/3rvBBFURRFUQAA SSlVLSh/sn3sE572bSp7t+/d+JJ7vni3//X7vqaezF/4QhRFURRFUdlUUV4TBwDqL1FRFEVRAICq KlAURVEU5Y2EMTYyMgIA/f39f3lpQ0NDAOA4TrFYVHWrsqmiKIqiKMqf5Lrurl277npo/a9uu+3p R9ZVZqYAYNmbTpUCMIaYy4333QYAy1Yds3jxoqOOWHXUYSuXLFnyp4p6bP2T9z3y+LoHH9y+ZfPu 7ZudbKF3ySqMQEgQUhKMBp9+2K/NHnHy6Qt6O958zOqjj1r9vxJ/lX2pc/rKy7aPv+7p5p/97Gdd XV1HHnnkK9w/iqKTTz7597///be+9a1SqfR3f/d3r/yx1q5d+7nPfW7u3Ll/a5WsKIryqpuent62 bdv2nYONkDEhm7Xqk48/ksjkVx1+eMxFwqCHH7w8mUx2d3dT+tKdaIyx+x98+FOf+ZeR8YmOg0+Y v3gZKcyhmUIhaUkp3Vh4kbB1LIQECRLArc02XXd2+zNTz94/+NCdZ77vA1d9/tOtWDk0NPT1b3// 5z+/JYh5+8o3J/oWJdoHUsVSLEkz4hmL+LEUUloamXXjgkMbIa+WZxJxbfeOrXTo0e3rbn/fBR+6 4H3nrl69Wr2zKpsqb6jYND4+fvzxx1uW9dRTT73CuwRB0N3dPTMz02w2Mca2bf9f7/LVr3512bJl J554YrlcTqVSf+pTT1EURflfxBhb9+DDv/r1b5/YsOm+22/pXnJEbsGKQlcfQdhKpLRSv0mwrWMv Fs3BZ6s+E5XR6ujgzO7tQb182pozT3v7KatXHb73lPrPfvGrD5x/fuehJx556rtFvr89qY/XI5Ni SlDEpM+4lCABDIpb319eJAAhKaVOUMCERcTMjg2PXf9vVES6bvhMLnnXxen5K7Fu6QSVPSYBCrY2 5cYWxRpFZS8uOHozZExAyqC1IE4YpB6w1qNwxtjY5q2//a9iMH77bb+cNzBHvd0qmypvkGx6+eWX m6b5m9/85vLLLz/iiCMA4J/+6Z8OP/zwH//4x4ZhfPGLX+zt7b3wwgtXrVp1yy23zJs378orr6SU trLpD3/4w0Kh8La3ve1HP/rRddddl0gkLr300sMOO2xwcPCyyy6bmZnp7Oy88sort23bdvbZZ3d1 dX3ta1+7+eabL7744t7e3ptuuum//uu/pJRnnHHG+9///htuuIEQcuedd46Ojp5//vnnnnuuagaK oih/tiefeua7P/rptf/+pQUnndu36uRizxw7lUMIQiYQIEAybVINIy8WGKGsre2a9ecVrLF6mLU0 g6AwcDdt3Tm1Zf3gfbc4Gr7g78976NH1m4ZGD//Ivy/ozI3Xo6xN3YjbGhUgym4MgEyKMUYIJEhg EqSUEkBKaEbCJMhj3NYIRvKZO37y3M+/e8ylP+ro7gu49GOWMqkUEDChERQwyYU0KCp7zNawBGhG rOjok43nu06FhKRBxupRX8bYVQ7m5Mznntsw/tN/XX3wshuu/bYamaqyqfK6xzlfsGDBQw89dPfd d//2t7+9/vrrAWDVqlUrVqz49re/fcstt1x11VX33ntvIpG47rrr3vOe93z0ox+1bfvzn/98K5te cskl3d3dhx566EUXXXTvvfcODw+fddZZGzZsuOyyyw455JDTTz/985//fLPZ/OpXv3rmmWeef/75 b3nLW1atWvX9738/DMP3v//9Dz74ICHk6KOPvvrqq3/961/ffffdd9xxR71eX7p06ezsrFqFVFGU v1nT09Ou627atGlqtuyGLGQCABGMuBQGwQAoa2utgVgvPv++c9fgCae83ddSK9/14fScJUnLIBiB lAgAECAASnDERcSEQUnCwH4sAiY6k8Z4I+xMGV4sIiaytgYArU/h6vTEDZe8Z9Gb1qw444KspY3V gvaUPl6Pco7mhRwQCCk1hCQAl1LK1nhTQABCSgmAAdVCbhBEMXru1z8efvyeN//zdzRNk1KWPWZq uNVpaunY0Ukz4JaOmyF3I15wtFrAkwZphFxI6ehkshEVE/p4PezPmjvLQbuj1UKuUxQw0dj00OBN V1SmxtV5uT+bqjjlNeH2229ftmxZqVRas2bNJz/5yWq1mslkAOC9730vQmjNmjXnnnsu51zTtHPP PRchdNZZZ332s5/dr5Df/e53p556aiKRWLx48caNGwHgs5/97G233XbVVVft3LkzCIIXP+7vfve7 NWvWJJNJADjjjDPuuecehNApp5ySSqVSqZRlWfV6PZ1O/789QFTjTRVFeS2F0fseeuRnt96+bcvm px++v3fZEVb7HNSxMKETLiS1k+3986seLyRotV5vju7UCHzhpw+Q6kh5z/agXj7i6GOPXn3kqSef 8B8/+PGPr//Pkz9zjdE+z9axTjEAAikRkuiFJZ4RSJOgpKF7EfdjbhBsUjrRiDpT5kQjLCZ0R8eD Zd/WcNHRAGDLEw/1rHrrke/6oKWT8VrYkTYm6lFnSh9vRDpFFiVcyIhLL2YWJV4sQIIEKSVYOgEp OUgEEHCRJOTA40+f+6YzfY50HQBQ0iQGxW7EMYaETqaasUERAuTHvC2hN0JOELgRF1ImTVrzWF/W GCyH7Ul9qBLkbdqMOICUEkspU4tWF9503pHHn/jYA79XzUllU+V17Dvf+c7IyMiqVasAgBByww03 XHzxxQBgGEYrvWGMGWOUUowxAOi6HsfxfoVEUWSa5r5b3va2tx133HEnn3yyrusPPPDAix83DMO9 h7aEEM/zCCGtB209roqMiqK84THGHn300e9cd+Ott/xPsmteYtHRpYVvzi1/z4nvsy0dV302J2tO uXHKoLNuZNkayQifyZ62rpmOeW0JvTviAZNZi0Rczk7suXdm6MaPf3HHul8V5y5xZydptjPAtkGJ lBIhCYC8mONWxylCCKQlwSBIp7TixwZFBUebbISdKaPqxwbF8/KWBOBCAsARp6yJGdMoGa+HnSm9 GrC8QyebsaURW8PNiAdMmBRjwAETCIFJMZcQMk4QAEJcAoBEABgB0W035JaGQMqqz00NIQA/lo6G vVggBCmDTDXjYkJvhBwB6Boue3FP2hysBCVHG66EBZu6kcjZNORSpxgA/FhYlHAJ/W86+7kfbp67 eNlRq1cfuWrVkYeuWLhwoepGfeXUNUuVl/PXua7m4ODgxo0bn3766UceeeSRRx65/fbbr7nmmtav nn76aQDYsWNHe3u7YRjNZnPnzp0A8Oyzzx544IH7lbNixYqHH364FVLXrFkTBMHDDz986aWXHn74 4du3b2eMtV5R64eW5cuXP/74462f169fv3z5cvWmK4ryt2N6evqfPv0vmbaO8y//zsb0IUf82x29 F3xz4cnniny/7SQwQhETOVsbroY2JdNu3JbQg1gmDWpS5EbC0fGMGzs6AZCRkEmDJItdvSuPO+6D l737Px899L2fKg8+d9sn3vbEDV9hoUcwAKAgFrZGDA0jhDAARogg1Ii4H7OUQTBCfsw708ZkI8ra GhNyohkBgE5bPa/gMZhsRp1psxZyg+Kqx/KOhpGs+Cxi0qLYIDhpkoRBHZ3oFEVCWBppRryVhBFA 6x8lOGEQUyOxAEfHSZ26kTApShgaSGhztGYkbB1xDkHMHZ0EsezLmMPVsD2hubHI2VrEpU5QLKQQ EiEUMqFTHAmJACIuDjz7kolyY2Ph2K/dP3ziP3xS07S3rznrN/fct+8XkPKnqBSvvPq+973vnXfe eYSQ1s0lS5bYtr1u3ToAWLdu3bp16x577LEvf/nLAJDNZq+66qpms/nkk0/efPPN+5Vz2mmn/fzn Pz/hhBPq9fpZZ51lmubpp59+0kknJZPJU0899ZZbbrn11ltXrFjxqU99qtX5CgDveMc7br755re/ /e0Y42w2e8YZZ2zYsOGv/PJV16yiKH99ruueee5565/e4Bx1zpu/9KtRD+UdPYx53qZTzagjpTdD buk4ZhAxYWs45CJnUS/mpo6n3bgnY4xUw/akjoC7ES8mND+WCCFLw7WAZUwSMmF1zDvkXQcsf+eH J7c+qZs2AqAYACBgAiGUMokQgBGiGABIxEXIWM7REKDJRtSe0qaaUUdSTwoppZxoRH4setJG1qJc wmQ9LCS0aTfuSOtll4VcWhrWMJISuJStIa0YQAIyKZYSTEoESIIQQsjUEADUAmZQRAH8mFsUA0IY Q4KSesgwkm4sAiYKjjZZj3oz5lAlLCa03dUwZ1M3FjpFQoBOEZcgBVgaCWLh6ERIGXOgGIUcDNta cMHXrbZuKPbPOeJtg2dcNuPtOevSq2Hru/7+I5+44pMXOY6jGuGf7BdT34vKy7WPV3UoZGu60rx5 8wghmqbtu2KU4zh/qkPX931CiK7rrZvlcjmZTGqaxhjDGGOMK5VKJpPZ9+6+7yOE9hsPoCiK8obE GLv2hz/5+Cc+PnfNx9oPPYVg5Mbc1kjERM6mVZ+XktpwJRjIWTNebFIUc2lomDFpaDhigmCkUxRz mTSIG4qkSWe9qOjofsx9JtocfcaLHY0YFHMpdIKlBC/mJsUGxQiASwiYcHSCQFKMEKBWasQYxVyG XJgEOQb1Ip61tarHEIKsrUkpAQECJISYbEalhD7RiEpJfaIRGRRTjFoTnuoB0ynWMGZSgAQpIZYS JAgJGkYSoB6yhE4kQMwEwShk0o15zqIzXmwSLAEkgK2RiUbYnTaGq0Gbo0824rxD3YhTjEPONYKD WFgadkMuASjBGEkpgQkZxK0J/iJva1NunLfoYDmYkzO3z/jzCtb6kcaKzuTITCV8+teTd19/8Sf+ +YpPf1y1RpVNlddlNj3ooINaN/dmU/W+KIqi/NnB9NBj3lyxOtve8qG2bDLkUgipU4wAWufTDY1E TBQcbbQe5G095sKgmD0fT4VBSSg4bp2Ox0gnuB7wnow+WotKSb0ZMoKxo+Oyz4qO7sUMJLIoakQc ACUNTDFqRAIDJAxCEBIgKUIIQcylqWEpJUKICQkSZW06VAn6syYCGKr8YSZrW0KzdTJRjzIWKXs8 Z1MupRfyUEiLYi5ay0VJPxa2jqUEhFA9YDrBGkE1n0mApEGqPjM1bGu0GTJTx42AIQBLp9NumLf1 yWbUndJ3V6O8Q72IE4wiJijGIRcUo5AJjaCQydbPlobdiFOCwljmLTrpxlmbDpeDgby1fcYbyFnb Z/zerDHdjLM29WMhJQDIuhfN3HxFO/WfeOBuNQ71xdR4U+XlSClfxaOXa665ZmBgYO9NwzBuv/32 N2Q9u66rGpuiKH+FYLrg0GO94iJ5/EVdhfSeamhpOOLSpMiNeSwEwkARmBqueHHO1iImDIpjLnSK 4+fHUwqTEgyotax9zGXGImWftSW06WZUcDSfcQTIpnjWjRIaIQgQRimTYgQU470jPlvfL0JKN+Ix lxSjmEs3Fm7IbQ1jDJPNcE7OmmpGADCQs+bmrbl5qz9rJQw62Yjbk3rV5wWH+jGPuYyFNCkSIAFB xETEJX7hcWIuLY0YFCGEHJ2kLRILiRDSCZrxIkDAhQy5tHUy40bdKWO6GeVMOlKLsjbxY6ERxITU MI6F0AmmCOkUR1zqFEVcFB3Ni0XGohWPFRPaYDVoT+qbJr25eWvXrN+XNScaUV/W8GOBEegYuRG3 NTxRjzqzdnjyJUHv4Z3zD1Kf/y/RL6b6TRXlVf4jVGtIKYryV3Hqey7YUCH9b7vA1vFoNezLmpum vBWdiQ3jzXevbK/4cdogY/UojPmUyxwDg4SQib0n9BmXOsVMCINgJqSp4YuP7r190/TWKQ9j0Cmu B7zoaGWfFW3t/Yd13rWtvGPWC5lIm7QRMksnJsGRkBETSYMikFxKBCgWEgBMiiMuAaArrWdMTSNo 65SXs7VqwFr9uADQCHnAeClpTLb6TX2mE6xTxLjECJiQCCEuwQuZAEjqREioB0wjWCcQMBkwnrE0 r9XNyQQX4Ohk2o06U8aealCwtUbELQ17kSAYRS/0klKCMCAmZMhEyqBlP85YdE817M0Yg+WgP2ft nPXnZM0ds35/1pxoRlmL+pGQAIBkw+elpD5WD0tJ/YmRxoqu5LZprydjzLpxIaG7IRu//2f//Kb+ f/zHf1SNU2VTRVHZVFGUvy2u66bbe9/ytd8MVeO5BcuNBAJoDTY9qN35zjsX7t1zsOx/6a5de2ph 1tbqAXM08vwnlQRKgQvgQhoE5R39G6cf8C+/3jFSDWMuMEadSeOdS9u+/+gYxXDa4uKdW2cBwA0F xWAbOIilrRMhBJegE8SF9GOBMTIo0ggGCQhAI3jN0rZlncnWM3lqpH7XtlmEUWsNKYwRSJhsRMWE 1pMxc7b20GAVEEgJTEoQIEACoGbEQUpLwxjjkHEhQSM4YsKkqBEKAGlpZMaN25ParMdyNt1TDfO2 5kaMEhwyQTGKuKQYQiYpQUEsMiYt+3He0gYrYV/W2FUO+rPGs+Pu4pIzWPY7UkbFiwsJbbIRmxQ7 Op5oRHlHGyoHc3Lmk6PNxSW7GfKUSWsBkwAJDW+b8ftz1lOjjeVFuvXLZ05MTKj2uS81ykFRsUlR FOWN7z+uv+nAUy+Y9ERv1nxm3F3ZlRic9XuzphdBT9oEgA/fspkL0An65mkHLGxPhFxWA37mkrbB SjA861dDltDJ4lKiI2VsnGhONaI5ORMAZpoxE9KguOho71zaVkzonSndjflQJZASerMmArA0HAu5 bcobyJkaQY/troNObA0vbk9YGt467ceMixe+cQqO9uxY447NMz0Z8+wV7XtqwTNjTUcnc3JW0iDr RxrtSd3WyQkL8hP1UCPIjfmS9qRG0PYZz4s4lzCvYLUn9HrANk95fiySOunNmKWkPlT2Zz0va2tz 81ZXmj091lzRlYy4bIR8QdGedqOejLllygtifvSctBfzLZPulBsVHC1va2+an/vFc9N9OXO47PWk zalmvKTDmahHxYQeMdEaMutGvM3RnhprLulI7Jrx5hasihfPzVsUY5/FjpQ7Zv0Vnclt0+6Coj1a Cw/pTjIhK0730NBQf3+/aqIqmyqvDzt37vzc5z43Ozvb0dHx5S9/ua2tbcOGDVdccYXv+6tWrbrk kkuEEB/4wAfmzZs3PT09MzPzox/9CADCMFy7du2NN964ZcuWl9y50Wicf/75+5X8ar1GFf0VRfkr 0J0UsVMIgGI0N2fVfDaQtwfL/ry81eod3DLp5h29EbCIt86hox+9e/HWafeUhcXf7yjfvb182kFt q+ekR2vhuSvbL7p5SylplL044tLScMjEko7kiq5kxWd9OTNt0uPm5T592/aLVvfMuJGlkaxFH9tT n5u3Co5e9YcmG9EHjug2KPYiftRA9nsP7cGA/JgjBAVH3zDejIUYrgbPjjUOak9uGG8e3pte3O5U vPi4ebkv3T14ysJCW0LnQqYMesER3TGXBsVHD2T/44Hdx87NHjOQ3V3x5xedm54a3zjh/t3y0kDe nnHjs5aXrnt09InRxoeO7L7moZG0SY+ek7F1+vX7hj+8umekGnRnzMvu3Pn5kwdqPstYdMuU98PH x44eyJ66uPDUSOPGcxZf+D9bigk9eD6MCkvDCZ3smvX7cuauWb8nY0w0oyXtDkhpGyRmYtaN5+Ss HTN+T9aYbsYruxKMi2JCZ0KO1aP2lL59vEzLw93d3ap97kvNhVJe0z7zmc+cfvrpd95552mnnXbP Pff4vn/mmWdedtllt9566+7du7/zne8QQm666aZ0Ov2FL3xhx44dTzzxBADccccdnHPG2J/a+dOf /vR+JauqVhTlDWl6enpoaOgnP/nJtuee2r3u1lJSn2xGto5n3TgWz6/KeUh3KmD8glXdJ83PfeVt 8zFCz403372iVAvY1etGfvzE+MkH5Keb8WDZ/8ztO37yxLhG8Lyi3Z7UJxthwHhrdvz9uyqA0G0b px/f0ygljclGlLaoQfFDg9VvPjCMEBqrBT94dBQASkljWWeiJ2P+4NGR7z862p0287aOABCCtoRh UDxej5iAgHEmJcWIYjTjxr/cOP37HRWK0dyc+ejuGgDc9NREI2QPDdaueWTkvp2VgqNhDHPz1jNj jRvWj137yMh4PezJmks7k997eM8X7xqcbkYHlJwFBRsjVA95PeSdaXO0FvRlTYPi3RX/nBs3nHxg HiR88/7dV6/bk9AJAjh7Rek/Hx397sMjs168qi9tYDzrxW2OPloLUwadbEb9OVMIyNuaTlBrXv+z E27B0WY9NpC3uJCFhGZQPN6IEKDBcpC1aDNkB3cnuJBy/f987WtfU1P196OqQ3k5rZVEX8UncPDB B3/9619vNBqnnHJKe3v7Aw88wBh7/PHHH3/88VQqdeedd1500UVRFF144YWEkLVr1/70pz89+OCD f/KTn5x33nnr169/mZ33K1m914qivGG4rnvHb+/56te//ti6e3uXHaH3Lo1TnUndbM5O+LNjyChQ jHqyJkjoSOnPjDW7M4ZG8Anzc2mTTjWjf/3trmfG3Q+v7ik62jdOW4Axap2XX9LhnHFQ0dQwAGyf 8d61tPTMeKPg6CO1oDdt6BRTjIYqvpCyPanvqQadaRMANk66OVsDgC2TrgQEAMNl//j5OQC46Ki+ 1px9SlDrIqZZuzXnKdYJaobS0UjERRDzrrR+ZH9aIwgARuvRko6EH3M3YmlLO7BkHzUnkzBIxWMI 0OZJ97SD2ha2OffsKI/WwkN70pONaPu035nWawEzKdYIZkJummx2pfRSQn84YDrBAHD9+omejLmg aG+ecqt+HAl51e+HlnclMUJrD+l83yGdSYMkDFIOWFdKb0Q8b2sIgR9xlNA3TjaXtDs7Z/25eStk Yk7O1DDyIo6QPlzxe7LmWC1c1pGQUjoGYQJ2zgYru5KzM5MTv7n2/Nu+rVrsflS/qfJyXvULV3zq U5/65je/uWvXrqOOOuq6664TQliWZZqmaZorV6782Mc+BgCmabauKXX22Wf/8pe/rNVq69evf8tb 3vLyO+9X8qv4Gv86F4ZVFOVvwX33P3DKu89v6+7/5H/eBsd+8MTvPeGf9uX5p34wnH98/+pTV3zk P3av+1VryXqNoE1TnpSQNEhnyvjVxunzbtr4662zCYOM1MOpZpQxtbu2l79yz9C37t/92Tt2rD20 44Ci872HR365caYZsojJUlIPmZh1486UMVKPFhQdIWXIRD1gHSljtBZ2pQw34lU/zllazMVkM+5K G0LKgAlbIyPV4JqHR256euJbD+wer4cEgUSolNAYF/WAA0B3Wl/Q5jw33njrwuKConPH5pnnJppe xKWUWYtOuzEgdMaSNozQ9x8bGSz7M24Uc/nQUPX6x0a3TrvvXFpK6rTgaDWfpS06Xo86U8buSrCg aE01os60EXHQKd444S4qOfWAFRzNjXjG0speHHCR1MlYIyoljYCJq9ft/uLdg5/7za5HhmsWxQbF w5UgZdLBij8nbzEuFpZsQBIAJKBNU56jk4jLnqyJQAKATvB4I5IAE82oI6kDyLl5yx3fvvHK9+QG lqxdu1YtI6WyqfJ6cuONNy5fvvzyyy+//vrrf/azny1atKhWq5122mlnn332gQceWCgU9t05m80u Xrz4iiuueOc730kpffmd9ytZVbWiKK9rrusedtJpZ3/i36YHTjj4ijsXnPpBL92HAAEARjCQsyIu O7p7M296P5cw0YgwQotLTszFyq5kwqAT9XC4Et6/q5KxtIO7kovanMlmmDbpEyONs5aX0hYtOnrI RNWPTzogN16POlO6rZOpZuQzTjDkLM3WSMzljBsvKNqWRtyI52xtphnFAnqy5lQz0ghqT+oVL/aZ qAWxqZEg5iu7kge22RJAo8TScFtS95lYVHIO702vPaSz7EaD5cDSSRALLuS8gt0IWT3kKZPGTMRM ZG2t5rOETucV7IrP8jb9h8O7ts34j+2uCSmLSW2kFvZkDITQWxfmDYp3zPpMgGMQitGHjuwCACFl KamP18OYi0oQV/04bWoDOevjx/V9/NjeGTcyCIq5aE8aZyxp2zrlpkxa9dmiNgday0QBbJ7ydIIH Z4O+rCmlXFxydII2TboIYLwRD+StmIuBnKUTNF6PuISJcr189w/Wf+OiRf/0nys/df0t7sDAQQc/ +OCDqg2rbKq8PpTL5WOOOeZ973vfJz/5yYsuuqhYLF566aXHHHPMOeecc8EFF+y9MOlea9eu/cY3 vnHeeecBwMvvvF/JqqoVRXn9Wv/k08Wuvsniyrlrr8j0LJhpxrGQY/UII5ibt2Iu0xaZdeOIy4lG pBM0J2cyLilGsz4rpXQAGGuEC0v2nkqwuxIc0pumBH3v4dGDu1O/u3Alwajis3t3lHuz5udOnrt+ d31ewTpubhYARmtBb8YcrYWOQQZnfYrRp97Un7E0ABgq+/05c7gS5GwtbdKyF8dctqeMyUaUMMi6 wSoC+JeT5i4oOlumPEvDNZ/ZGm5L6EmDrllaWj0ns3PW/9ETExU/HqkGxYR21vLSrk+wy6IAACAA SURBVBmvLaEf0pPaUw3nFuyjB7KbJpuH96XfvqiwabK5qi+dtqiU8G+nzP3IUb2bJ92nRhqbJ92y z658+/wTF+RvempydyWYbkYJnVzypjnNkDcC5sUiadBZL55oRr0Z89aNU4f2JP/trfPLXnzX9vIv npt+drx5+SnzP3JUzzNjDVsnTMg9tZASqIesN2OCgFJSY0I2Ik4JaoSMYMS4nJMzdYonGhHGMO3G GYuGTMzJaMN3XLvpi2caxZ75l/y81Nkbc7Hw2FPnf/jqt77jjPvuf0C15Ba1vqnysu3jNbCGVBRF tVotn89j/PyhlBCiWq3mcrlXcveX2fnFJf/NVrKiKK9fjLFEvn3p//mBke9ohLzg6OtH6of3pioe S5mEYjTZjLrT5pOjjUO6U7UgThqUYjRY9ucV7GfHmyu7krWAJQ1CEJp2446U/tyEu6hkI0AYyYyl P76nvrDN3jbjH9qTnHVjSycWQRPNOGfTnbPBwpJdduPWvH4uhGPQLVPevLw12Qx70mYz4gmDYISq fpx39NayABONqCOl1wPentCmXaZTxIXUKYo52BoOmLA1HDKJMURcIgBA0JM2mJDj9Shj0SDmEpCU wIUwKJEAGoaJZtyR1EdqYSmhV/04n9C2THl9WbPisYxFg1joBLiEsseSBq74PGfRGS/OWloz4kJI Rye7a0F3ytg85S0sOW7INIK5hGbIkgYdqvgDOWu6GTk6QQCDlaArbWyadA9qT1S8uLVxohnmbP25 CXdZhxMwISVQgp4cbS7rcJ4cba7oTLQ2Vjc/iPsOySWMR/c0DulO7qkG7UmDSVGenhn+1tpmeVq1 Z1D9psprn67rxWJx3/iIMX6FwfTld35xya+KV/fCsIqivN5ded3NhSPX0GyHRvDO2SDmImVQAHB0 IiRgjMbrEQAsaU8AgK0RISVG0Ag5RrC45PxhI0ZTzUgCJAxCEcIIuZHwIg4AGsFFR2tG3NaJFzIu YcaNNYK70oYQMu9osx6TIIcqYchE3qYYQ0fKaEY8aZJd5QAhCJmQUrYndSGhI6VXfZax6NYZP2PR ZsgTBok46BRCJg2C6iG3NYwQSujE1nBKp6P1cHc1zFo0YoJgRDHyY25rpBrEOkFTbtye1HeVg1JS n2iECYNMNeI5OVNKaAXfkVooAY03oryjBUzmLA0AEICQMFYPkyZthrwjaQBAV9rgQg5WAoyQF7LW OgYWxT4TI7WIYlQPeVfa0AgqJTQmxO5qSAliQuZsXScoaRApYdOkZ1AccXlQycEIAUDE5e5KoFOU Xnhkwn7+JJ6QMFaPMEZCQjpXsOeqM/sqmyqKoijK69/N//2T3NLj/nDUTXF/ztxvn5hLAOCidaH5 P8KFxGjfWIC60wYTkhLYUwv3HjdnbSqkxAiN1KNYyIJDmZAJvZV00awXU4w7UzpCkLM1LiRBaLwR cSFzFgUpW8uROjoZqgQAKGQCAfRmTCZEKamP1KKsRZuhSJkkFjJv04ALDYMbcYKRkCJrar0Zsx4w AaAREnNZSuqVgLUnjWk3akvoVT/uyxogZcqkBLfKR8OVIKFTKWVf1kAAfiyYkK3z7FzItEkpaZ22 grFGpBHEhDQp3lsbI/WIYsSEzNiahhEAxELuqYUYISFk1tY08nyImvWZRlDERE/GfMmprQbF9ZDv 9xv8x7d7T3r/Zy69TLVnlU2V/wvVpacoivIal8jkX7yxlTiHK0HrZmv1pVbi3Djp7bcnJWjzlPeS H/YYwQFFO+JCw7jqsRdKw1lLa3W17ruzqREpJUaoFjAmJABQjFMmZUJSgndXQ5BgaRghaK0/Twka qUUA0JbQpJRtCV1ImbW0sXqcNqkbibxNmZAGJfWAzbiRoZHWKldpi065USmpcyGTBiUI1QKOAO2u hgmdMCGLCe354VIAu6tha92o/qxJ93nChKDWLKX9MqKUsuBoTIgX10arGluBdZ/9YaoZvWScaoXg gT8+VKAYxVwu7Ujs/z52zbv/3nvUnH2VTRXl1afWkFIU5S/x0fecMXbXD/fbyITUCKqH3I34KyyH v2xPBEIw7cYhE6+kqLLHCEYvWUhbQntx/NUJFkLiP/5gTBgEI5wyqR9z26AZS9MJxggVHA0jCJlE AI2AGQQLKU2KX/JzFKOXCUDoxU9bIygWMmNplLyigNSaCPWSv5p2I4IRE9LRyb7bCUbipaq6ObL9 mOPe9Kov3aiyqaIoiqIof5E1p506r2Dt+fX3992oExQyOTdv7heMQEJqvy2tjJXQ6Z/IcX7MWz2F 84uWQV8uNiwqOeesaE8Y+OiBzEDOWtjm5OznL/HTmTIO7Ul2pw2Tkt6M2Z02AACkNCkSEhI6XtSe aHV2CilTBmFczLgxIJh2I43g1kKhXAg34hEXbsjTJglikbLojBdjhFozjUyKWqFvb4YWEiwNr1nS 1pMxg/iPgrWQEiEUsD9k95xNYy51gnfMeC95znDHutu8sR2Cs71bJhtxqzP1xYqOHjNpUDzjxvtu j7mkGM240X77Tzx86xev+FfVnkFdF0p5eWoKuaIoymvfutv/583vOOuJL75bnPqv0Jd64dNb2hrZ m8MAwKA44qIna+x7X42giMm9IfIPIVZCQsdCwu5quLRDi4XY2zEppByrR31ZI+Ziby5L6OScle2/ 2DA5v+Cctby0t5xHh2t3bpl53yEd+j65drwe/uCxsbLPSkn9LQfmF7cnmiFPmXTrlHvD+rFS0nj7 osLtm2ZqQQwSQi4YFxLA0QgTImRCYEQJrgUsZVKDoGbE2hKaH/NSUq94LGmSyWbck8YZi4KUpYSO ECzvSt4/WB3Qrb3PgQtJCR6rR3NyxNbxfl90CF4icWpWaurhW2/67keyPfMXvOW9vctXt7YvLNpM AMHopQYCQGsO1r6ZWMdorB4VnD+sbMhDf/ap361e/V+qMatsqiivPhX9FUX5S7/LKb3vjlseevzJ 08865+l7M97S01etPhrIH4Y5YoQm3bAr/UcDHyMuy27cmTYA9h09KbdMecs7EzGXXSmjFdBiLrfN +Ae12xEX8woWBnAjzqSs+azN0SIu5ubMkw/IjdXDZ8abxw5kK178rQeGMcKH96VOOqCwfqT+0Vu3 9mfMi4/pXb+nfv/OigCgGIGEtKkt6Uj+4rmpe3ZUTltcXFxyejLm/KLdkzFLSY3XRXvOqPmsN2sO V4KMRRMG3TnjubEgAAmdzCtYBiUbJxqxgLxDdYyLDqQtGnFeSGjLOhO/21YuOvrWKe/w3nTWohJg IGc1Q64TtKsczCtYtoYpQu0JnQmpETzVCHMvRMZ5BSviQsNo46R7YJuT0HHvymPalx4l4NJgduzB 2//n0Wsv6zzx/fzUcwEwl1In6Jnx5tJ2J2kQLiVB0Ih4mlAA0DAayJlCAkbAX6hs/YUOV8nZ2I2X /PvXv65a8vPNVVWBoiiKorwBHHnoyqnBLff89Nq3W0P3XHzUw1edP3zHteNPP9ApywAw3ohiLmbd uHWCvj9ragRNNKOYyz3VoLWxL2u0uguFlF7MW8sbFRPaC1OpoOo/f/fulE4x9mMhAe2c9RMGPaQn 9dBgVQJ0ps3JZvTcpBcw8dRoAwC4kEmdYIwsjYzXw/FmhBFMNMKMTSebEQDkbG1O1vzt1pkfrh8n GK3qzTAhB/L28q7UWxcWz1nZ0ZOxDu1Nn3ZQ25yc9aEjexI68WPx94d3HdabPrQn9d5DOmt+/Ob5 +VMXF089qO2MJaUzlpTWLCmdurjt5APybswxgoRB8rY2Xg8RglrAKMEdKZ0L6EobM26EENo568dc AiApoSdtNEJGMRqrRwBQdDSMoDNtxFzqFG+d9hLFrvnvuOCdV99lc+/nF5/gju/YPOUKCUmDIAQ9 GYNxSQneVQ4i/nxXqq2TmEuN4KFysHcYwdy8WZsceepzp37ln//xwr9fq9rw88daqgoURVEU5Q2j UGx7esPGnuPPri89s2jU92x7Tj52x/axXc3R7WMv7HPgp29ZtGAgYnJRm00xakQcY1T3WdKgUkJb QkcI7a6GS9qd4WowJ2tEXHSndYqRFwsAtHXaPag90QhYZ9qQAAVb68uaGKOds95oNezNGPWAn39Y JyV4YZsz60YbJ5tFR2+tY7V50m1PGhONsD2hj9ej9qS+brByzED28N70kyP1dYOV0Vo01YwqfnzX ttm3LizqBF390J6ETj64qvvGJyemG2HpkM6BnPXcZPOR4eqmSbc/Z61Z0qYRnLPoVDO65pHRz588 V8Po2kdGLji8u+DoIRNMSFPDjZCnTVrx4mJCG60FnSlj67Q/r2ABAEbQlTICxvMOnXajtEW3Tfmt XlUJkHO0yUaUMeloPezNmACAEHSnTA6w4swPxwee8MSNV/pacvFHv9K6RgDFaNes35+3AEBrXYVL SIrReCMsOHo95Bhgbt4M47hyz/WVR2556N67VyxfppquyqbKK6JON/91MMYoVX+MiqL8pZ56+pkj j3vzvHP+pWfpUfWAOSaNzbmHve0922f8/pxpUBwzaWj4mbFmLGTZZaWkxoXsyxoRE0OVcFmnNlT2 B/JWzY8PKFoIAEC2wuiSjkQ9YN1pE0C2JbSQiYofJ026ddqbnzczFp1uRjNNVnT0rK3pFGesdNbW Nk00r310tJQwdpX9v1tWaq1s5YbsshMHuJQPDlYfGKz8blv52bHGyq7Uqr60APjt1tm8o22dciMu c7a2ecrVMe5JWwihNUvaGJcJg2gEpQwyv+Cs6svYOqn4zNFJ1tbu3VkxKbYo3jzlVnxm63i44mcs jQvwY1ELWFtC9yPOOFgadiMxv2DNuFHapCO1oCttbplyB3IWSEQQ6kkbXsQLjjZUCXsyxrQbtyX0 rEm5lIvb7MlGlDbJlin/oJKT7+he+elrH//vqx/+7mdWX3TlWC3MOZoEQACLS3bEZMak22e8vqw1 Xo86UsbcvBl4Xv2RX26/6/qPfeKSS38xpj7/96PO6SvKqwwhpGmaqgdFUf5yp6w5m5x7dfuyo6s+ T5sUAXSkdCGhHnKNoB3TPsWICzknZ2oYuRHHCG2YaGYtzYvE4pIjpbR0Egs5XA0RQpPNqC9jNkLW mzFbYRQj2Dzp5WxtvBF1Z0w35D0ZA2GUd7RGwMtB3Jc1MEI/fmL8gp9t3l0JIi5tDY/WwoGcaWu4 7MVj9Shj0e0z3mgtHKkEi0qJpE62z/i/3DS9ZcrtTBkGQSmTjtZDQKjgaDNuPNUMA8YjLn74+Ogd m6e/9/Cep8capx3UhhDcsH5suOJP1MOMQTBCO2e8vK3pFI9WA0vDCYM2Iz5eD7szRtmLO1LGZCPM 2poX8YylBYxzIQ2KQyZztuaGfCBnVv04Y9MtU55jkIofx0LaGhZSLizZk83Q0vCeaggITTZjjeDe jMGl7M+ak43w0LMuKs/OPPfz77oRpxj1ZMyIS4Pi5yZcgOfrv8+RU1uf3PPjzz11+ZoPHb9odHjw 85f8HxVMVTZVFEVRlDemTdt2NkPW3dMbMbFz1kcI7ZoNOpN6xMWyDkdKqIccIRgq+45OqgHrz5oC oJTUQyZmvViCnGpGWYv6sZiTMxGAF3MJMOvFCYOM1cKOlDHdjHqzhhvxnEUZl37MdYJHq2FCp/WQ dacNLhEAPD3WnJc379tZXtqZ8CLRkdLHalFfztpTDbK2NtWMbn526qYnxwcr/tqDO96xuLigYB/U nujOmBP10KQYADCGlEFMjYzVgtbq+hpGtkZsnZywIF/xWcbSZtxYJ3hu3i57cc7RhZRTzThrUwCY bEQ9GRMAdlf8lEF6MuZINRirhx0pY6wWZm1trBbmbG28Hjo68RnXCQ5iTjE2NRLEYm7enHajzqQx VosKjrZlymuNQ9UIzpqES1hcsqeakaOTjZMuIORFPGLikIuv3nHPT/uy5q4ZXyNo44QrJHRp/uz2 p9Obb3v06xdu+fK7jiaDv/7+lbXZqY998Dy1lOmfotK68nLUGlKKoiivF48+9ay56NiOlAEgl3Y4 GEHCwLGADRPu8s7ERCNa3pngEvKORgmarcSZgvbsWHNJh+NFoj9rAiCTYp3gsbo/kLc2TriLSs60 G3WkjGbIejJmyIRBsU5wPWBZSxurhX05c6Qa9maNyWa4tCO5pxKctCDXCFnKIKP1KGH4UsIpB+Z/ sXE652g5Wxsq+17EQibTJmnG0qTo5xumTjuo7UOre6SUW6e9328vA0AzZGsP7rxh/RgA+LHQML5/ V3Ve0T7vsC7Gxc0bpjIW3TDeWD0n057UnxqtHz2QfXS4VvFiJmTW0hohC7i0NRwyMdmIezJGV9q4 9bmptElaJ9zHakFn2hiphh0pfdu0P69g7akGXWljuhmlLa3ixbauA/CAi7xDZ9xoXsGaaUYFRxus BH0Zc1fZ606bEhBBqD9rNkLWlTZmvbg9Y5PS/F13/WTG49PjGy2N3Lru9mWrjjn+2GMOeeeRJ33r E8ViUbXSV5Q9VOxQVDZVFEV5A3BdN93ee9SX7nhq3F/emZj14s6k3owFAkibdLIZdaaMp8eaK7sS g+VgIG+BBD/mSZNOu3HRoZunvGWdyeGy350x6gFvczQJyI950qAzXtSa5N6btbZOe4tKzp6q35cx R2phX9Yse/GaJW1HD2S/dPfgWD3qTuvjjag7bZR9VrD1iXpYSGheLDACW8MBkwmd+DELOGRNwqUM YpE0qY5RwJ+/NJROMICMhLQ14sdcp9jRCABoBI3Vw7ytTTajhE5NipohlwAUoUhIKSWTrQulIgmo 5scGRZZG33Jgfn7BvuKuXSBBSuASbB3PNOOOtDFSDTpTxp5q0Jk291T9rrS5ZcpbULSGy0F3xpxx o5ytTTcjSyNcSAnS0ciUG6VMOlILezPG4GzYnzN3zfp9ObPqxw6FX118wpeuuurE1Yclk0nHcVQY /fOoc/qKoiiK8kbgOM7xbz195JkH5xcsilEz5EzC9hk/ZdIdM37B1qo+W9aR4AJsnXAhN0w0kyZ9 dryZt+juSrik3ZlqRMWERhCSEiIhh6u+qeGNk26bo4/Uwt6stXnKPbBo7a4EPWljTy3qSOkzXqxT /MjuWjGhSwldaX3WY50pfbQWJXQyVg9MDbshD2JOMZr1YillxIQXC5CSS5ASJEAYi7LP/Ig3I+FG IuAiZMKkpOzFCZ1aFMdcxFwETORtfaIRFR29HsRVn2GMpAQupZQSEApiISWETAghDIq9SLQ5dFVf +ofrx2yN+LEwdYwRuBFPWbTsRTlbG6mHXWlzpBp0p8091eCAorW7EvRmzS1Tbt7WhitB3tFn3Mgx iBeLWEgACJksOlozFF0ZfbDsd2eMwVk/bWoP/fZXR56x9v9c+A9Llizp7+9XwfTP7xdTXWLKyx+F qwExivK34MQTT7zrrrtUPbwGfeELX7jssste4c7T09OLDzva6F8Bx35wYUeyNTEcANyIpy06OOvP L9hPjzVXdCWGykFP1qj5PGFgABTEImfTqs/zDh2thQN5a+Oku6TdGa6Ec3Lm7mrYnzXKHis62owb 2zquBCxnaVzInEPHalFvxjisNz1UDrZOuylDm/WijEWZkFJC0iBeLCwNx1yGTKQtigC4kBJAJ0gC EIy8iJuUIAQB4whQzqI+EyETpaThx7wZ8IKjAUAj5BIgbZLJZpwxiR8LhFCrR1MChEy0rmzv6IRx QTCqBXx1f1pIeGCwahAkJXixyNpkxmVZi7oRFxJ0gitB3JUydleC7ow51QwLtj5UDbpT+nA17M2Y Q2W/p/V/1hoq+71Zc6jsd6aMWY/ZGm6G3NQwINkI+FNfOuf+39+9bF6ParQqmyqKOgZQlL/4ywAh VQmvWf+/vqkZYx/7169d842rFp7/FdGxqDNl7Jj1DyjaO2b8OXmz7LKCQ0MuEUIEgReLnE3HauFA 3h4q+3ML1lA5GMiZw5VwTt4YroQDOXN3NexKG42QGQTXQt6TMao+K9h01mUJk5Q91pPWR+tRb9Yc rYVZi8ZcIgDHIBWPFRxa9pmlEZOgRsQsjRoExUJihBAAxYAQAgRSSECAAVGCdIL8WEZctCeNyUaY tqilEYwAAJgADDBUCfoyxkQzzlpUSFHxWeucOwBIgCAWJkUYISbAjZiQMmnQgAs/4kmDBkxIAFsj M26Us7RpLzYJkoCklLZOJhtR3qFD5XB+wdpdDXrSxlA16Embw5WgN2sOloO+jLlhormo5Owqe/0Z a7Ds92bNyUactckzt//4ADZ02y3/rZqryqaK8gbJBOovUVHtUPlffF927Nx1wUc/+eiD9ycPO73/ 0BM8p62nkGmGHCGUMslUMyom9JFKMFCwN4w3V3QmdlWCuTlzqBIM5K0ZN8rbWtVnWZvWA25QzIQ0 CGZCGBRXA1ZK6NNu3JcxRupRX8YcqQWlhNEMGSHI0UnZi4u2XvZjW8cGxY2Ap03KhMQIdIoRQCQk BtApbuVuhBBCQBBCCJiQYSxNDdk6qfhxm6NjjIYrYcakOkVTzbiU0FoXlyoltclGZFDSKiZkQgJI Ca0UiwAJgEbIpARbxwhQzAXGCADCWFgabkZcSrB1MtWM8rZWDxkC0AgOmchY2p6q3502hypBT8YY rgTdaWPTpLewzR6sBH0ZY1c56M+ZO2f8OTlr56zfm6Lrfvqd1OzmDQ/9Xi0IpbKpogAAuK4LAK/f fkeVCRTVDpX/R++L67p3/O6en916+wP33celFLm+TNecilHqzxrN/MK+vDFSCbtsGB/Z7Uh/aNNT h73rQk9LWRTHXBoajrk0NRxzYRDSCqY1n/dljd214IWJUNZoLXB0ggAQgoRBZj1WdDQvFgBgUVQP ha1hg6KYQ8RFwiCtpalsjYRMAKCEQRCC/4+9846L4ur6+J2Z7QWW3qWKWKh2VGyg2HsJGInYUdFY sYua2EtExYpGAbuIUbGAKCBR6SDEAkiXXnZZtk15/xjd7AOIqGjUd76fBHfv3rnTZ3733HPOpcGQ HMNlKMGhwyw6jBOgXobq85m1EhRAQJNNJwABA0iBE3IUq5OhujxGqVChx6NXSxRkGlEaDAGCIAiA EgTpwwpDQCjFCEAmKAUiGQ4AwWcgIrE4L+F+R5eREpSAAEEaTdl0uEKs0OTQxDIcxQkWDZKipCVV ZiJglYnkujxGbrW0nYDxukZmKmC8rpa102C+rpLSCxKeHN+wY8vGBT7zKWFKaVMK6nUFUBQ9dOjQ tm3bAAC7d++eOnXq9/hooDQBBXUdUnyF84KiaFFRkUgkepKUKpZjAIDUpISqyoo7t2/r9x7N0rPQ 69wDUdNhMxkKnGDRIAUGmHSoToLq8RgiOabGolWJFaYCZmGdTI/PqBQrdHkMsQxFYJjHgMUKjM+g VTUouEyEx0DEckyNSVfgOEYQDATGcVyswGEA8VkIAAQEQRI5zqJDMATJUILUtQRBcOgIgwYrcEKG 4vp8Rlm9XMCikbbM8noFAEDAogk4NAwnCmqlZhqsmgYFh0FrkKMQBNXLMRYNRiCIAAAnCLEMIwBg 0CDSSooTAMUJHAAmAqXdCsFwwnbYNACAFMU4dEQkQwEAHAZNKFVwGYhIhgFA4AQEAEGD4ZoGhZE6 M79GaiJg5tXITDWYudVSIxb29PbluqfXuzvanT1+mAp7orQpBfW6AgCAGzduzJo1a/Lkya6urhiG RUdHR0dHHz58uF+/ftSJo6CgtCl1XlqJWCzu0W9QVspTh1+P63V0QjGCSdpNabACw/lMpFyMmmkw C2vlZhrMwlqpvhqzQY4BCLBocJ0U0+XSqxoUpOlUiuECFiKS4iw6zHqnMjXYdAARKEZIMZzPoAFA wBAAECSR4xAEAEGwGAgEAJeBAACkChwAwGXSahoU+moMiRwvq5ebabBIh1OCIN6I5EwarMGmvxHK 9PiMmgYFuSUNCgwCEE4AqQIjB/dJYYoTBIeBkNoUggDAsJAFQ6zcPLWs7Nq171hPMJk0iIFAtVKM RYMIAhIrMC02rUys0OHS66QYBAgGAktQnEtH3tSJ1cVvMpPisVfxMmH1qsU+s2Z6U6EClDaloF5X AACQl5c3evRoS0vLn376icPhKMvr6uqOHTumr6+/Y8cOMzMz6vRRUHy5m726ujoiIsLT05P8imHY 8ePH582b9xU29cKFC2VlZb6+vi1X+/XXXxcvXvzVHgVyuXzo0KHR0dHf40MYRdELly6vWrcJaJsb D5jENbdn0+BaCWqmycqvlVlosgtqJXo8pliOwRCAACAApM5CKhv+x3TagOJMBIYhgOIEmaBUJMch AgjYCABQvRyLO7hywKx1HHUNCAIQBGAIAgBAANBgCABIgRMQICAIMBBYJMMM1Ji1EpRJgzkMBACC DHWCSddUgngjkuvz6G+Ecj0+o7RezoAhDgNBcUKK4hIFRgAAgbdD/CwGDAMIJ4BQhjIg/O72eZ6j 3OwcnOKfPA2/FlZXVaFl071d1wEKHKez+Xqm7SvFKJsOWHRadnqCJodeJUbxmiJ5RUFJ5hMEgsaM nzhu1LC+vXtRhlJKm1L8Z6Ao+k2NkovF4lWrVj148GDZsmXvezQUFhb6+/vPmTNn1apVVI+WguIL aaBnz55NnDjx+fPn5FepVCoQCKRS6VfY1IMHDxYVFW3fvr2FOjiO29raZmZmvq/C7t277e3t3dzc 2nDDysvLdXV1v0dtmpeXZ25u/vr169z8wp37/oh9+FB38C8du/aq47Wz0GIX1Eq0OAwZisMQxKLB dVJUl0evalBwGAiPgUgUOBOB5DghVeAabJri3QcIgkjjKwQIHACJHC9/mRLzx6/uKw4YdXQCgKBB EAzDABByjJChOAQBLQ69QYHLUNxAjVkqkmmw6TQYVIoVAABdHgMAIEXxMpHcTINFAJBXIzXXYL0R yjW5dLEclSrwt16nAAAAFDgOCEAaUAkC4ADUCkWx+xdPHeG6Z9sW1XdcUVFR/rZC0AAAIABJREFU cXFx1sucBhQncIKBwMmJTzs5dkUgWIfHAADY2try+XzK3vE1ofx2KVq8Pr4ZYYqi6Pnz55cvX75o 0aLff/8dht87bYSJicmxY8cyMjIsLS137949bdq0H0wTUFB8y+Tk5GzcuLGqqsrAwGD79u26uroZ GRlbt26VSCS9evXy8/PDcXzOnDlWVlYikYh0Fidv8E2bNsXGxpqbm2/fvl1fXz85OXn79u0SicTe 3n79+vVMJvOvv/46dOiQlpaWkZER+QRo1LLqYyEhIaFr166qGxYQEBAREQEA8PT0tLCwCAgIMDIy 4vF4r169kslk586dO3nyZH19faMG79+/HxAQIJFIevbsuWnTJgzD5s6d6+TkdPny5W7duo0aNWrb tm1cLjcwMFAgEPz6668hISErVqzo06fPyZMn5XL5tm3bnJycXr9+vXr1aqlUOnfu3Hv37u3du/cb PHFxcXFmZmZmZmaD+vcTi8V3Iu9v3r479+U/hfaDNSy6cDs6STi6ejx6gwLX5tLrZRiOAy4drhAr uAwEggAEgIBNAxCQKnAWHSYAIUMJOYYzaBAAABCAAISWibmN+7SbW2d3HDa96/hZDBZHLlNAEAAQ 4NAQOgI1yDEcAHI2VAM1ZoMCk8hxfT4TAECmOOMykHYarAYFVifFzDRYJUK5Pp/xRiTT4NCZNLiq QQEBiEmDYAiCAURAAMMJQAAcgNyMxPiApcGng8aOHtXoHUfudZ8+ff4tneVF3cj/LdS8UBTfAbGx scbGxk+fPj1y5IiDgwOCIFCLIAji4OBw5MiR+/fv29nZpaWlUceQguLrsGbNmrFjx0ZERIwZM+b+ /fsSiWTixIkbNmwIDw8vKCgIDAxEEOT8+fPq6uqrV69WLrV3796qqqro6GhnZ+d169ZJJJJx48Zt 3rz5r7/+qqqq2rt3r1AonD17dlBQ0IkTJ+Li4gAATVtW3Yy7d++q2kT/+eefoKCg8PDwS5cuxcfH Ozo6du/efcOGDb17987JyTlx4sSJEye0tbWbNrhnz54DBw7cvHnz0aNHN2/eRBAkODjY1NT0zp07 165d++uvv27dumVoaBgcHIxh2J07dwAAT58+ffjwYXh4uLe39/r16wEAXl5eU6ZMuXz58vnz5+Pj 47/BsyYSiS5e/DcxJ5fLHT9mVOrfD6vLSm7+scarm4E0+sQDn25RGyalHV/z8u65gr9vgZLM0pJC cXmJrKqkqKCgqrKSIEBVSaGi5o2s+o2wvLiqpACtLi3MTHoRc+NZRMi99ZP+CVq7/qfBxQV580b0 vrx4WNKlwzKpmInAfAZCQyChDMMJwGEglWK5Pp9R3SAHBKHOpovkWHaV5FXl2/9qJSgDgfX5jBKh TI9PLxHK9PmMKrGiQY6rMRAmAuE4qJdhtVKMNPSWvki5smIcLf366xdZjYQpxbdrF6MOAcW3bNLL y8ubOXOmmpravn371NXVP2pZBoMxceJENzc3T0/Pzp07Hzx4kPIQoqD40nTt2nXv3r0ikWjYsGH6 +vqxsbEoiiYkJCQkJKipqUVERCxYsEAul8+bNw9BEFUpuX79ehiG58yZM2fOnMePH5ubm9vY2AAA PD09/f39XVxcbGxsDA0NAQDDhw+vr69PTExs2rKywcjIyAsXLii/GhoaCoXCdevWjRw5MiAgQNXC CkHQsGHDLCwsmt3UoKCgGzduVFRUSKXS/Px8CIJgGB45ciQAwNra2t3dHYIgW1vb3Nxc1QYnTpwI w7CTk9P27dsJgkhISBg7diwEQd7e3qtWrfoGz9q2bdvInWosEWg0W1tbW1vbpfO8gcoIeOo/r2So 4tatIAgnanCcIAAdgQsIAsMJlUkciHoAuQ3op26ka2fdu/vBNUonq/kzp8/28jh+JtRv2UhLZ/d2 PYcKzLtoqbGkCryyXmGoziwWyvT4DAWGZ1c1WGiyLLVYyk3CcKJUJFdn0QzUmCV1Mg0OrbhOrs+n VzegDXJck0MnAIETQFQvyo4LT716rK/LgMdREZYW5l/o0HWfE/LBOgnHPKknA6VNKX4ExGLxjh07 jh075u/v365du09uRyAQ7Nq16+XLl7a2tnPmzNmwYQOVgo6C4vPh8Xj19fXKr7W1tSwWCwCwcuXK gQMHXr9+vW/fvmvXrrWwsGCz2eRPTk5OQ4YMAQCwWCxVYQoAkMvldDpd+VUmkynvUwRBpFKpVCpl MplkCVkTx/GmLZOIRCKxWKyvr68sUVdXT0lJCQsL279/v6+vL2l5Vd2XZhusqqpydnbevHmzq6tr ZmYm2UtXbgYEQeRnCGocuaFajqIoOZ4DAFCN3fymHrY+Pj4dOnT4sGL43xFwUrB+ovig0eZ7T589 3SP+78f7T5y9vHeJeS/3dj2HmJualAAdMhAKAGChySYAIUNx5YIIDOnzmQ0KrKhOZsBnlAjJv3JN Dl0hbch/lZ339F7+00g2HZ43d87dA3lfIerAxblzC7/GxGdSjwtKm1J89yhdSxcvXhwUFNSCa2nr 6dChQ1BQ0KNHj4yNjU+cONGsheC/gvI0pfgeMTU1hWE4JibGxcUFABAUFESqw+Dg4ClTpnTv3n3o 0KG///77n3/+WVdXN2bMGDabnZqa+r7WHB0d//77b2dn54yMjNOnT69aterFixdSqZTFYiUmJjo4 OFhaWr548QLHcRiGnzx5Ym1t3alTp/e1fP/+/YEDB6qWZGdnl5SUeHl5eXl5kUITgiAURVXrNG0w IyPDzMzM09NTLpdnZmY2cmBtJXQ6XUdHJycnx9LSkhz0/9ZITU3t16+fak/j60kQGs2lX1+Xfn0B OJqcknb9btT9i2GxEWEGXXoZWVgbWNsV61rQWVwtLl1dU5vB5ihQvLoBLSktM+IQDJx4lldLVL7O kaCV6bEFGY81tHRc+vdfMXX4iKAdVCAspU0pKNqM2NjYBQsWdO3a9cSJEwwGow1bhmG4f//+3bp1 O3ny5Jo1a65fv07FXVJQfDIQBF25cmXOnDlqamp1dXXt2rULCgoCAFRXV7u4uFhbW798+XLdunU6 Ojrr1q1zcXFp3779q1ev/vzzz2ZbW7169eTJk2NiYvLz8/fs2aOrq+vn5zdo0CBzc/Pi4uJz584Z GBi4u7s7OzsbGhrS6XSCIFpo+c6dO2PHjlVtX01NzcvLy8jICEVRPp/v4ODg6Oi4cuVK1a5v0wad nJxKS0snT56MYdjPP/989OhRd3f3TzhWO3bsGDt2rLW1tZWVlcqQ9zeBWCzetGlTeXn5fy7mnBzt nRztN61aSm5Vbm7u7bgEtCH/3t3YUoxAcQIAgEAAAIAS4A1BQADq183W0NLc2kSv07LJxsbG1JjY j/N4oWw2FN8IFRUVCxcuLCsr8/Hx0dDQ+KLrKisr27FjR+fOnU+ePPktdK+/tVxdFP8/tSb4JCt+ XV0di8VSjnQDAORyeV1dnZaWllL54TheW1urqanZclNCoZDH4ymXwjBMLBarqakpK9TX13M4HFVB 2WzLlZWVGhoajdwGSN3MYDDIEXwAQE1NjUAgaCQWGzVIEERVVZWWlhYEQXK5/NM6zOnp6YaGhlpa Wvfu3Tty5MjVq1e/wnlpJSkpKY8ePZoxYwZlaPw0us8J+eCYPuVv+tG2JOoQUHwLymzjxo22trZ9 +/Zdt26dpqYm9IXR19fft2+fo6OjpaXl2bNnGw3tfX1NoOppR0HxfaGurq4qTAEADAZDR0dHVUHC MPxBYQoAUFNTU10KQRBVYQoAUFWuLbSsra3dVJgCADQ1NZXCFACgoaHR1IrZqEEIgrS1tclqnzyS 8/DhQ3d399WrV69cufKDUwZ8TaRS6fDhwx0dHSlhSkFpUwqKfwkODjY2NsZx/OzZs506dYK+It26 dTt79mxSUpKxsXFsbCx1LigoKL4EixYtun79+qRJk2JiYgYMGPCNbFVFRYW5uXlycvL/pPakoPgG oIYRKVriiw4nZWRkeHp6WllZnT59upHd5et1zmB4zJgxLi4u+/bt27Rp08mTJyknVAoKijbH0NCQ TID1jZCRkbF06dJjx45RmfUovkXtQfmbUnx9bSoWi2fOnJmZmbl58+Yv7VraenJzc/ft2zdgwIDt 27dTI1wU1M1O8aOel4MHD5qamqakpKxZs4b0dHdzc4uMjKSO9ifQbXbwB+skHv+f6Qn9/f03bNjw UWvpPifk/5XTKqVNKb7qYxFF0UOHDm3btm3ZsmU9evT41vYXRdHk5OQ9e/bs3r176tSpVHwSBXWz U/xI56Wurs7FxeXAgQPFxcUeHh6N1kLxadr0g7FQjbQpdaN9EMrflKIl6uvr2/AWunnzprGxcV5e XmhoaM+ePaFvDzqd3rNnz7Nnz0ZHRzs5OTVK0E1BQUHxPYKiaEZGxuLFi2/evDlhwoRu3bqpClNV wUTxsbTm+H9sfQrKLETREm01tJ2Xlzd69Oj27dufOnWKz+d/43vNZrNnzJgxevTo9evX6+npbd++ /Us7oYrFYsqLgOKbgjKk/be0iYipqKjgcDizZ89ev369nZ3d69evAQDNqlKKHw8Mw0QiUXVNTV1t nUgkEoqEtbV1IpGooaEBxzGBQJNBp8tkciaLKZfJGUwGjuPNXnU0Gk0mlSEIgqIogiAQDMEwzOFw uFyOuro6j8tVV1cXCAQCgXobPjQobUrR9lRUVCj968VisZ+f34MHD9avX29kZPQd7YW2tvbWrVuz s7N79eo1d+7c9evXf6EhfmosleIHlkcUX7NjQIrR8PBwZ2fn2bNnjxkz5tWrVyNHjkRRtL6+nuoA /2DgOF5TU1tVXVVXWycUiWQyOY5hOEEgMIQgNDqDzmFz2GwWl8vV0tZiMpgsFovJZDCZTPICw3Ac vDPlNr3ZyVLyAwxBAAAYhiEIIpO4SaVSmUzW0CARN4jr6+vfvHlTLxbLZXIMxwkCp9PpTCZTU0ND R0dHV1fnE2KdKW1K0fbo6up27tz56NGjr1+/Xr58+YoVKw4dOvSd+m62b9/+4sWL9+/fNzY23r17 97Rp06jzS0FB8d8iFovJD6mpqQ4ODrNnz16wYMGhQ4dsbGw0NTXDw8N79OgxZsyYefPmyWSyryxJ xWLxqVOnVEs8PDyWLFmycePGsrKysLCwXbt2NVpELpcPHTo0Ojr6PzmYBw4c0NPTmzJlyvdy9jEM u3EzgsFgcDhsTQ0NLS1NQwMDHo+nUChkMplCoZArUBRVYCiGYThO4AROyKRyqVRWWyeEIEAQBARB EAQjyFutSQIBQOpOgiAIQBAEgeMEhmEEQWAYhuP42z8YRhAAAoAAAIIgGIYQGk1bR0ffwIBOpzMZ DBaLxeVyIAiqra2rrKpMSEgUCkV1QqFLvz6tt09RsVAUH+6+f+xFAkHQxYsXL1++3LlzZ3KK7R8A mUx24MCBV69eBQcH29nZ/ecHmYLii97sEES9Hf7Lc0EQRF5enoaGhkAgAACEhIRUV1ePHDnS3Nw8 NjZ20qRJe/bsyc7Ofv78+bRp04KDg/fv3x8ZGTlmzJiGhoZPSAvVhk+hoqIiOzu7nTt3KksmTJgw bNiwEydOWFtbNzQ0kHukilQqNTY2rqysbE37o0ePvnTpUhumHayvrycHqT9t8Y+dF+rzD/WDhzHW 7a00NTXLyiveDtNLJFKpFMcJBEFob/8n//wLDMPkv6QqhWHkf7UpBAFIabAnCAInCBzHyYF+VW2K opgSFEUxDENRDEVRBfoWhUKBoiiDweByORoCdW1tbX09PQ6HfeLEqblzZzWaO+N9UHZTii+ChoZG YGAgjUbjcDilpaXFxcXf+x6xWKyVK1dWVlZ6enq27WSnlAKg+IGpra0NDv6fJDt0On3u3Lmf0+at W7f27Nnj5+fn5ub29fdIufbdu3dfvny5TRzoxWIxgiBXr14lUzsBAEJDQ5ctWxYVFUVWqK6uBgAU FxeHhIRYW1vfvXvXwsICvAsJGD58OHjnSPotDNxzOJxZs2Y1LX/27Nndu3eXL18+d+7cPn36hIaG 6urqHjx4UKkLFQqFr6/v4sWLbWxsyJKcnJyNGzdWVVUZGBhs3749PDw8Kipq0qRJf/7557Jly6ys rEQi0datW3fu3BkfH8/j8dasWWNrayuVStesWfP8+XMmk7lx40YHB4czZ87gOB4REVFZWfn777+f O3cuLS3t559/9vb2vnLlira2tqGhYVRUVFlZWVJSUo8ePbZv3w4AOHDgQEREhKOjo5aWVv/+/bt1 6/Yt3FNCoYjN4Zy7cJkgCBqNRqPRYBgmP/yrgJV//tXEH+ie/I9uhgD0Ht+S/ymD3q0HatzPEYvF dXV1+fkFpFQdOMDFwcGuvLxCX1+P0qYU/zEikSguLs7e3r5r1675+flVVVXf+x7p6OicPHkyNTXV 0tJyzZo1Pj4+VJ4pCooWkEqlLBYLALBt27YpU6ZYWFg0mob0E1i8ePGdO3dIcfb1Ua7d3t7+c4Qp iqJFRUXx8fECgYCc+8PT0/P169f5+fkAAGdn5/T0dGX654ULF5IfyDmcdHV1v8eLoby8PDExEUGQ 4OBgMqPq/PnzT548uWjRIlISzZgxo0+fPkphCgBYs2bNpEmTJk6cGB4efv/+/dmzZ69evfrixYtM JvP8+fO7du1auHBhYGDgy5cvw8PDc3Nzhw8fnpmZGRERwePxbt269eDBg1mzZiUmJubk5MTFxd28 efPcuXOjR48mhWyXLl28vb3/+ecfY2NjDoezY8eOuLi43377rUePHhMmTBCJRCEhIQ8fPszMzBw0 aNCFCxe+kcPI5/EKC4oUCsX38vYhCKK6ukZDIKDTW7vB1GuVoiU+/+rHcTwlJYXBYDg7OxsaGubk 5Egkku/9sDg6OoaFhV25csXY2Pjx48fUVFIUFO9DX1+fNKGtXLly5cqVAoEgMTFxx44dKSkp06ZN a9++vb+/f0VFhY2Nza5du1gs1tKlS3v27BkaGgrDcEBAgLGx8cmTJ2/evNnQ0DB+/Pg5c+bMmDEj Ly/Pz8/v3LlzBw8ejI2NhSDI19e3X79+iYmJUVFRZMv379/v0qXLpUuXrK2tFy1atGHDhoaGhsDA QAMDA+W2paWl+fv719TUjB07dvHixTiOBwQENGowMjKyrKwsIyNj3LhxCxYsUK7dz88vKipqxYoV ISEhFy9eVFdXHzVqVHV1dStNwps3bx4xYkS3bt0ePHiQkZERGhrK4XDIIRQzMzNPT8/v+qlSVlam qi+Tk5NVfyU9Fkj/Ticnp6ysLLJ82bJlpqam8+fPV63ctWvXvXv3ikSiYcOG6evrqzYil8vnzZuH IEhYWBhpGVWe1rFjx6qpqe3du7euro7U+hAEDRkyhMVidenSxdra2tLSEgDAYDBkMpmyQQcHhw4d OgAAOnfuXFJSkp6ePnLkSBaL1bVrVycnp2/n8NLpiFwh/74uCRiGMAxtfZAfpU0pWrw+2qJbhmGY XC5/+PChpqZm165d6+vr8/PzcRz/ro+Murr61q1b7ezs4uPjP/MtQvmbUvzw5ObmKl0nY2NjL1++ HBYWZmhouHTp0vnz5/fp0+eXX34JDQ319vY+deqUpaXltWvXfHx8Tp8+PWbMmJMnT8bGxioUipUr V4rFYg8Pj7q6uosXL/r7+5eVlV24cKGkpKRXr145OTmqLf/8888HDx68efOmra0tDMOnT59eu3Zt cHDwihUryE2qrq6ePHny9evXTU1NXVxchg8fHhoa2qjBuLi48+fP37hxA4ZhCwsLHx8f5dr37dsn k8kePnwYEBAQFRVVW1vbtWvXTZs2Nd33vLw8Y2Nj8lmKoujdu3eDg4NHjhzJYDDKy8t1dHT69+// g51uPT2958+ft1CBwWCQzz2lW3N1dfWbN28KCgpQFFV976xcuXLgwIHXr1/v27fv2rVrZ8yYofyJ xWIhCEJaQPh8Pmme37x5s7Gx8ZYtWzIyMnx9fSEIOnToEFmfdFGFIEjpq9rIqbpRuUwmU9r4P9kb 9UtAAEj1ddHsu4NQ/vm32odaVQ1AIgAZD9VMPaLJF+J/VtX8u+wj329U7n2KLw7pPkUQRE1NTVRU lEgksrW11dXVhb5P6HS6hYVF+/btk5KSCgsLqfNLQfFBkpKSlJan5OTkxYsXGxsbwzD8yy+/ZGZm 7ty5Mycnh8Vi5ebm6ujoLFiwAIIgNTU1TU1NQ0PDqqqq3377rbi4+MCBA1wuNyUlxcnJCUXRI0eO bNu2DUEQExMTgUBQWFiobDkvL09PT8/T0xOGYYIgFi1axGAwCIJQnSE5NDR09OjRHTp0YLFYT58+ NTc3b7bBlStXGhsbCwQCgUAAQRC5dnKPHB0dT548uXDhQi6Xa2RkZGho6Ojo2HTfx4wZY2JiEhoa WlJSQqfTzczMRo4c6eHhYWtrS81lr0RTU/PcuXM9e/b08/NTLQ8ODnZwcNiyZcvp06cvXbpECkcU RVXrODo68ni8qVOnTpkyBcdxbW3t+/fvk5bv0tLSRpVbj6WlZUZGBgCgtrb2yZMn386xggABNfHv bFIHfJq/KfTOhZSMjWrOmNLkS3P+ps1uEKVNKdroHoCgtsqmm5iYSH54/vz5gwcPuFyunZ2durr6 9yVMdXR0OnfuXF5efufOnZqaGuoKoaBoDcnJyaralLQUBgYGLl261NLSctiwYRUVFY6OjsnJyf36 9VP2ackYlMTERENDw4kTJwYGBirLi4uL1dTU1NXVSWNkeXm5oaGhsuXk5GTSL7OhoUEoFJKeqSkp Kfb29qp9ZtWB2vc1SG5PamoquSy5dnIV5AaT8TFyuTw/P1+1fSXp6emlpaWenp5GRkbz5s0zMjKi st+/D9Lj88aNG8qS6upqFxcXLy+vFStWLFiwgFSiw4cPLygoUNZZtWrVjRs3Jk6cOHDgwJSUFBqN Nnny5AULFkyePLmoqMjIyKhZe/YHmTRpUklJibu7u4+PT7du3ajZKL4m1Jg+xdfm77//dnZ2TkpK 4vF4dnZ2enp6BQUFcvm37j3D5/NNTU2rqqru3LnzvTskUFB8fW26dOlSAIBEIqmqqjIxMQEA/PXX X8uWLXNzc3vy5MmbN286dOhw9uxZpekxLS3Nzs7uxo0bI0aMmDVrloODw5o1a+bPn5+amrp3716B QFBbWyuTyZhM5sWLF93c3GAYVraslMJkI2SDz549U83+pqamVltbCwBISEi4ceOGn59f0wYrKyvJ BkkrKSlS9+7dKxaLa2trDQ0NEQRpaGgAABw5csTExIQcVm6BI0eOHDlyJCAgQBnb9ONhbGxcVFTU qPDx48cAgC5duri7uwMA6uvryXJlOD+ZQIpOp5M1lfj6+s6bN6+urk5LS4tMP3T79m2hUCgQCJSN 6OnpPXjwQCgUMplMclzex8dn+vTpEARxudzFixdjGKb0E+jWrVtkZCT5mdxOMiQfADBgwADyA5lZ QiQS7dy508HBAUEQZ2dnVU/l/5b/D2P6lDal+G/AMKyhoeHJkyc6Ojo2NjZ1dXVv3rz5Nn0u6XS6 ubk5juOxsbHKlNdt+aChPE0pfnSURsr09PQuXbqQhZMnT16+fHlwcHD//v0FAkFwcHBKSgqZDik/ P19DQ4PL5T59+nTPnj3W1taZmZkbN24k7aCkSlizZo2rq6uJiUltbe2ZM2dUW05JSRk1apSqSM3P z9fX11dNirlgwYKJEyfeu3evoKDg1KlTbDa7hQaTk5OHDh2qXPujR49Imbt27drp06d37NiRxWI1 m2AoLy+vUYmdnd28efOoS6L1MBgMVecHGIabZkglOxuqX3k8HvkBgqBPC5zAMOznn38eMGBAfX29 jo5O26a1/hy+6Jg+aKMcUs1u0EfsI/VepGjp+vjU3Pv37t1zcnKSSCQpKSmJiYndunVLTEzs0aMH BEFPnjxxdnaOj48nB8vItL+WlpYGBgbFxcV1dXXf1O6bmJhwOJyXL19WV1c33ba0tDRzc3NqhI7i x7vZv07ufbFYDEEQh8PBMEw5HWIjGhoaJBKJpqZm03eeXC6XyWSfk8ipurpaU1Pz0xqUy+UvXrzo 3LkzBEFz5swZPHjw1KlTm2pTc3Nz5Vd9ff309PQPupmqHvyvEytJRWQ2i0KhyMzM5HA41tbW76vz NXPvR0dHZ2Zm5uTksNkcmQLt3LlLo/ymytz777Lvw42y769bu8ZnwUJzc7Mmufc/Yl4oZfp99F3+ /Tu3bwuFQhzHWWy2jY0Nn6+mUCiOHQ10HzZCIBAoFAonR3sel9u+vZXq7dYClL8pRUsQ7ybb/ULE xsaSH3JzcxMSEtTU1Dp27Mjlcr8F11KBQNCxY8eGhoanT59+6cysn+ytT0HxXcPlcskIaPLN2Wwd DoejpaXVrDGGwWB8Zur7Rm/Kj2pQLpdPnTp18eLF3t7eb968mTRpUtM6IpHoY4UpxbcDnU53cHBo QZh+ZUJCQpKTk9lsNgRD9yPvBR4++LEt+Pn5GRoaNi3/5RcvZTqtD7JkiW8jN7zgs3/KZFIWi1VV WblujV9hYQEAYMrUn1qpRJtCjelTfBPcv3/f1dU1MzNTTU3N2tpaJpOVlpYqFIr/ZGPYbHa7du3E YnFWVtaXGMSnLBYUFD8APB4vJSUlMzOTz+dbWVk1W4cM9Cb5XoTp/5Ogn+/0kevs7NzBpiOLxTZp Z750iS858nDl8sXq6modHR2vX7wF6urHjx0xMzOPefjw923bjx09UlBYwGaxvb1n2tnbHzt+bNq0 6To62nv37s17nctmc3wWLPjnn6y42Ni5c+ccOnS4sLBw//59UqnU0dFpvo8PiqJrVq92dHL663q4 hqbm2jVrb9+5k/D0qZ/fyk2bNr91kiEAAKC3c19dXV0URUtKil+9fKmvbxBx82a//gP4fH5cbMyl C6EdO3aysrIcNmxYa6bXouymFG2Pnp5eUy+rD1JTU7N579aXL1+mpKSgQXM1AAAgAElEQVRUVlZa Wlrq6uqS1pSvBpPJNDMzMzAwyM7Ofv36dZvEPInF4q8gcCkoKL4+DAbD0dHxfcJUlbi4uO/IYkr8 6HzXVx3pb0qGgjGZzKCTxy0trVavWdepc5eDAX8AAJ4/f/78n3/8Vq+9desmhmF//BGwePGSBw+j AQDJSclicX3Y1asYip46/aff6tX37t718PBksdlHjhyl0WizZ89aunTZqdOnS0qKg8+eRRDk2rUw Npt1+s8zfB7/ytUrEydOZDKZ27bt4HK5//qbAvD8n6y01NQHD6KLi4ocnZwAANnZr6RSycuXL1KS k/w3bxk0aOCuXbvIoDdKm1J8bgf6E/rQjx8/vnv37pQpU0pKSgAArZyDWCqVxiU/mjb75z8vnKmo qEhKSsJxvH379l8nzxQMw3p6emZmZjU1NS9evGjDyatyc3OdnZ1VzScUFBT/T8jOziaFKZnTioLi Mzl06JCfn9+SxYt37fjd8+fpMAwnJyViOBZ9P0oukyUnJZIv7oGDBmlqalpaWv39d3xw8Fm5Qr5w 4SJlIx06dIiJeXj8+DG5XL783WwUAIC0tDQURVNTUy9evMjj86Ojo0nX5xEjRgIAOnXqXFFe/r4N e/Hi+bNn6bnZ2Xw1fnV1tbL8dW5ux06dGAyGjU3H1seTUWP6FG2PmZnZxYsXo6OjZ86c6eDgMG7c ODqd3r179w8uyDbltV9mf2R04C9TvQAAhYWFpaWlVlZW2traxcXFXy7PFJ/P19PTEwqFL1++xDDs o5atq6vLzc0FAJiYmNDp9KYVbG1tIyMjPTw8ZsyY0WzUFDWaT0Hxo/L8+fOQkBBKmP6oVItkX3mN Hh4eenr6dnZ2icmp71y0ITaLDSMIAsMLFy0mh/vYbDYAwM7OLuDgoZiYGP9NG9u3t/bfvIVsxKlr 1z/PBEdF3lu5YnkHG5vdu/eQ5TiOs1hsJpNJAMK2S5d+/VwAAHQ6nVSoENRSIqoxY8eTY/pZmc9O nw5at34TWY6iKIvFMjM1ffHyJYfDbeVuUnZTii/FwIEDk5OTBw8ePH/+/LS0NOX4eO/evZutLxAI JOnCytg35FeFQvHTTz8VFBS8ePEiPz/fxMTE0NCQRqO1rbmUzWabm5traGgUFxe3cqzhfRQWFubm 5pI6tRE6OjoRERFPnjzx8PCgwp4ovjsgik/l3Llznp6eraz8HV0S9+7dU44FVVZWnjp1SvnTrVu3 Xr16NX369JycnJYbWbFiRXx8fOtX2po225zuc0Ja+HXossstB+l/ZvtN0dDQYDBZVlaW0zymOjna s9lsY2NjPX39vn372Ts4slist7H2AABApCQnEwTh4eEZdOpPZVZXAMCTJ08IgvCeOfPylau3IyIA ABAAGIZZW7cXiYRDhw4dO2aspZWV6jxqKsYUAEHQWyMO0TSPKigrLyOVMQBAXV3NpV9fPo+jq6ub lpaenJzUyt2k7KYUXxAajebl5TVx4sSVK1eeO3fOx8dHNZ1KI9hsdvjFa6fOnc4GaQCAPn36bNmy hZw2hsxrqKura2ZmVltb2yYTMtFoNG1tbSaTWVFR0RqLbOtjGN+3uj/++OPWrVtdu3aNjIykYnUp vi8o6/5X6AB8R1ublZV1/vz5kydPAgDCw8Pnz58/dOhQMgB83rx5Dx482L9/f6OEo015/fq1UChs /Upb02bb8rqsvoVfMby1N0W1SKbJZ7YgTx8f8UDg1l4AvXv1CD13QUtL08HefsK4MSwm3W/VKkMj w4L8gnHjJ7y9nAAAAEIQZM1qP0tLq4rKCs9p/+axoiHIwoU+Haw7lJWVenvPBAB06WI7bdq0Q4cO LVny67hx48wtzF/n5u7dt7+5CxV06GCzZLHvxk3+2uSLDAIAgC2bN9IQGkHgfDW19es2ODk5aWpq uA0eJJcrzl8436tXTz09PQcHh1Ze51R+U4qvRG5u7uzZs/l8/uTJk7W1tUkXz9jY2P79+8MwHB0d 7erqShoPcByn0+k9e/bMzc2tqam5ffv2vn37Zs2aNXnyZAaDYWxszOfzy8vLyelYPg0NDQ1yXhmx WIyiKHm3KBQKhUJBpm0jCILcmIaGBqlUiuM4QRBSqVS1kcLCQn9//7Nnz1paWqqWkxMkvo+8vLze vXtfvnxZdZhPLBZzuVzqIqH4z7VRs/lNv06uU+r4v+/gg6+b37Q1pzsnJ6d///7kvEqTJk1iMBiD Bw/29vbOzMycNGlSVlbW8uXLfX19q6qqoqKiysrKkpKSevToQc7AtH379ujoaBsbm9evX/v4+Li7 u58/f/7cuXMEQYwbN27w4MGHDx/evn07QRBTpkxZu3atvb399evXZTLZkydPfH19Hzx4gCBIRERE cXHxrFmzPD09JRLJ6tWrX7x4MX369Hv37gUGBqpOsvDBo/0+4v6p+HXfXQCAanbSRpqylUbTmPjM uENTmXSk6QklG1Gt0DKJiUkpqWkaGoLamloYQTAU46vxrSwtBAJ1U1PT+npxvVgsFjdIJBKZTEa+ 3err69XV1Vks5rucpjCCwABAQmGdmpqa0tRaLxKRhlIMw2rragUCwfvym8rlCqFQyOFwyHkN6HQ6 i8XksDlcHofH5cnlsry8/PyCguLiEgSGCYIoLikxNTWFIMh/04awsLDWeJ1SdlOKr4SFhUVUVFRM TMzkyZOnTJkyfPhwGIZdXFyauShpNABAYmKiiYmJvr4+m80eOnRoUFDQkCFD9u3bB0EQgiDm5uZa WlqlpaUfO0TO4/G0tLQaGhqU0zG/86Rp3JmTyWQEQTSbx6quru7YsWM6OjpXrlz52InszMzMsrOz x44d269fvw0bNgAqhxQFBcX3hqWlJY/He/bsWceOHVNSUkJCQvbs2ePt7R0RETFixAgAQFxc3C+/ /FJbW7tjx464uLjffvutR48eEyZMEIlE165di42NLSkpsbe39/HxSUpK+v333x89eoQgSL9+/ayt rc+fP79169asrKysrKybN2/a29uHhIT4+vqSbebk5ERFRd26dUsoFNrZ2Xl4eGzbto0giIiIiEOH DgUHBx8+fPgz9w7DiV7zQgEApGpsts4/RR9h8XVx7tx3wfkWjKNkhegDU3isD6iypwmJ0zx/4nA4 FRWVYdfCKyurNDQ18vILyp6WVVfXdLTpYGFh3q6diZaWFflakUplUqlUoVCgKKpAURzHCfztdKSq aYNhGFJXVydfQzAMawg0cIIgCIKUrTQaAkEwDEEwDCMIQqfT6XQ6k8lAEAQAgON4aVnZ69y8wqKi vLw8ggAdO9pwORxtLa1nmVnt2pkEn/1z8ODB9fX1fL5aK8OhKG1K8VVxcXEpKioKDQ318vLy9fXt 1asXWT5o0KBGNXEcLywsZDKZlpaWMpnMx8dn1KhRAQEBhw8f3rx5M6kyjYyMyOm5WyPs6HS6rq4u 2WzT5FCNWnjfKL9CoQgPD09KStqyZcsnT2HH5XLv3bu3efNmNze3a9euUVcFxY9qCJRIJMop5inj 6w/GyJEj79y5U1dX161bt+7duycnJ2MYdvv27fXr16teAw4ODh06dAAAdO7cuaSk5NmzZ+7u7nQ6 3dTUlEzhcu/evfHjx5NTHowbN+7Bgwe9e/dOSEh4/Pjx0qVLL126RBBEamqq8mUBQdCwYcPU1NTU 1NTYbLZQKHz06NGmTZsAAHPnzl2yZMln7le9FB3oe6EFg6iqcv0oedprXuidPRNbqDDQ98LFLWPM 9XgttIOiKI1Gj7h9z8rKYqb3L7W1tRkZz3JyXteLRNraWhUVlW9KS+MexSMIAkEQh80WCAQCDYEa n8/lcdX4ahwOh8ViknZlcjyQHKtUHlvVv+Q9i+O4VCqVSCQSiVQkEtXX1wuFIpFIJBQKhSIRAASC 0Ph8PgQAAYCZqWm9WFxVVcVkMmxsOowYMayuTmhlaRkVHT118sTIqOhWHi5Km1J84O0C2tqkR6PR pk+fPmHCBG9v73Pnzq1atcrExOR9leVy+fPnz9XV1Tt06KChobFz586UlJSFCxf26NFj6dKlDQ0N Ojo65PzXqvOvNAKGYU1NTSaTWVpa2mhcvumeyuXyZtOa4jielpZGes2uWbPmfRM0tzygr8qGDRse PXrk7OxMXWYUPypsNru2tlZdXR1QIwM/HCNGjNi+fbtQKBw6dCgMw05OTrGxsc+ePWuUlEA5vE4K HalUSs4ERtoLAAAymUz5OEUQpKGhwd3d/cGDB/Hx8adOnTp79mxCQoKDgwNpomu2TZlMRpbQaLQP jua3TLVI1nJ4U3Juzdzttz4t/snFufPQZZe7zQ5O/XPW+ypMXh++79chfTt+ICBBVF//KP5xYlKy sZFxO1Ozrl27qqur1dbWFRUXl5eXV1ZWVVZWYhhWU1tbXl5BAILACZzAAQCAABiOQRDMYjFZLBaG YgSBM5lMAEEQgAhAYBiGKlAIhhAEAQQBAERn0OlkFDIMEwRBql5AEBAMc7lcmUwGwxCB43r6+rq6 Ovr6elpaWnV1worKyrKy8qTk1JraWoVCIVAXWFtb375zj9KmFN80XC73woULubm5Y8aMad++/YwZ M5oNCXx7H4pEWVlZ2traHTp04PP5ly5dunv37pAhQ/z8/EaMGFFdXW1gYKCurt5sVBOPxxMIBNXV 1cXFxS1sD5nQ9H3J9vPy8vbt29enT58LFy4oIxA/nz59+kRGRrq6uq5atYq6JCi+6x5sszx48EAg ELx+/drMzExZ+PLly7t375KdRiMjoyFDhnzsPRUfHx8WFrZr1y7q4P+H9OvXz9PTUyqVhoaGAgDc 3Nx27NgxYMCA9/XbSSwtLaOioki7Q2pqKgDAwcEhKCiI/DUxMdHDw8PZ2dnLy0ssFmtra5NeqqNH j265zYyMjB49esTFxX3ORCevy+onrw9vqjsbhdJ/TmD+u2VPvC8838W5M+nk2oIDwKtXr+7euS2X y3k8nqGRUfv21nQ6ncFgaGpqCNTVLS0t7WxtSfsoqfWFIpG4XlxfX9/Q0NAgkcjlcolEKpfJUBSV yeV0Ol0qldJpNBiByf6jMlcUiqIMBoOMAGGz2Swmk8Vi8XhcPp/P5XLV1PhqamoAQDKZVCxuENXX i4SirH9eVFZVS6VSFEVRFP3kyR0pbUrxX2JhYZGRkXHjxo1Zs2bNnTt30KBBzaYIJamqqhIKhQYG Bnp6elwud/Dgwfv27Tt+/PjevXvJbFBaWlo4jldXV5PpLVgslpaWllgsfvHiRcuboWpMbWTdqaqq Onv2bE1NzdGjR1t2LW29xVQVHR2dpKSkZcuWPXnyZM+ePS0/1ikovk0ePHjQwk9kdg4cx0kV+/Tp 09DQ0Dlz5hAEERMTs2rVqqdPn35U/HW3bt06derUtPzp06d//fXXli1bWtPIR1WmaEY90Gh9+/Z9 /vy5kZERAGDIkCFz5sw5c+ZMy0tNnDgxMDBw1KhRCIJ07NiRIIjRo0dfuXJl5MiRMAxraGiMGzcO huGqqqp+/foBAAYOHLhnz55Dhw610ObKlSs9PDyuX79ubW2tp6f3aRkPSLHYVHd+ZoqoFhVqS7/2 mhf6PvfTxMSER49i+/Z1qa8X3bgeXlFRsXHTZoGGRm1NrUhUT0PeQqMhly5e6NLFdtCgQUwmk8/n 0xl0Oo2OIDBZAYJhuMmxwjBs1KhRt27d+t9CHMcxFMUwDJUrUJlUKpPLS0vLs3NeSyQSFMNQ0psV RcnPTV+mxLvJxlrb46XGWShaAEXRr6OWUBTdsmXL0aNHN2zY4OTkpJyrqdkPLBaL9DStrKwsLi5e u3atlpbWxo0btbW1uVyuhoaGRCJhMpk4jufk5LTcb2v2V9ILB0XRq1evhoSEBAYG2tvbf1Bkf+YR CA0N3bFjB5VeiuK/4pPj9CEIakGbKhkwYADZSHBw8O3bt4ODg8ny0aNHe3t7jx07Niws7PTp0wiC eHp6TpgwISUl5d69eykpKRMmTBAIBAEBATKZrEePHhs2bEhPT797966fn19AQEBERAQAwNPTc/To 0QMGDBCJRCtWrPD29t65c2d8fDyPx1uzZo2tre2ZM2dUg7tVK8+ePfsbOf7fUZz+Z1JXV0d6eiiR SCTks/0TWissLJTJZObm5lKp1MLCoqys7KOO9qf5j34FYuIz7+yZ2Cj51IGAQ1wOO+jUqRnes2g0 Go1GC78W9uZNydJlK6oqK0JDQ0Qikba29rz5Pm9Kiv03bdTR0V3y61JdHZ0jRwPraut0dXVXrFip pa0FwzAMIxAE/R3/6PSfpxVyuaOj47Jly1etWnXjxl8TJ07avGVLWNjVa2HXCAJ3cxsybvyEjIyM +EdxWVlZgwYNtu7Q4Ujg4ZqaGi0trdlz5nK43KKiolMnT8jk8sGubqkpyVN/8qytrb165VJVVZWa mtro0WP9Vi0/EHDId9GC1uw7lXuf4gM9489vRCwWP3r06ODBgy2MttBoNH9//4yMjIcPH86aNavl NPgymSwvL08ikZibm9vY2Jw5c2bMmDHu7u5//PGHUCh88+YNg8GAYfjNmzcymQx/DxiGSaVSrDkI gnj06NGECRO0tbVjYmK+tDAlZbeHh0dwcLCrq+ujR4+oC4/iB2PAgAHNluM4XlxcrK2tnZaWRmZk O3v27LZt2549e1ZbW7t79+6FCxeOGDFi+vTpx48fv3XrFpfLzcrKKi8vT0xM/Oeff4KCgsLDwy9d uhQfH0+n02fMmDF+/PjZs2cHBga+fPkyPDx8y5YtEyZMUCgUOTk5gYGBhw8fPnv27KJFi3g8nrIy dXa+Po2EKQCAzWZ/mjAFAJSUlAwbNszPz2/kyJHLly//qGXrpWiveaEuzp3bSpi+L67/EyD9U1vO sQoAGDR48JPHfxMEce/e3T59+v722zZdHd3QkOAuXWxtOnacOWuWnZ3dtWthroPdAo8c1TcwOHbs qOriv/66ZOfOXWfOBrPZ7BcvXsyaNdPCwmLzls1paakBAQH7//jjQMDBs2fPpKQki0TCU6eCpv70 U99+/cLDrw0cOGjPvv26urqhoSEAgL27d/Xt57JipV/8o7hXL18CAM78ecrU1GzuPB8LS6vz50M/ TntQNwnFlyAvLy8+Pv7JkycXL14sLS0FAMTFxX0whaeOjs6FCxfS09M9PT2trKyWL1/ewtNKJBKJ xWJNTU1TU1N3d/f+/fuHhob26tXr6NGjw4YNYzAYTCbTyMiosLCw6bKkBm222eLiYn9//w4dOkRF RX3QDe7zzaWq2NraxsfHjx07dsyYMQsXLqSuIorvXYOS9lSlxVTJw4cPx44dSxBEdna2m5tb3759 t2zZYmRkdPXqVQCAoaHh7du3yYF7MrDGwcFhyZIlU6ZMWbhwIZfLLSkpIasJhcJ169aNHDkyICDg 3fyNAAAQFhbm4OCgHF9OS0trGtxNnawfhp49eyYmJr569Wrp0qUfldTvfQ6mn4yBBgsAwGfTRJK2 mf+PjI4CjXOs/s/dxOXyyDfapElTkpOTLl26UFJSrBwSJG+9mTNnPXr06NSpk4UFBSpRGQQAUKdO nTZu3DBmzJgZM7y5XO7z58/JX2IexgxzH8bj8XAcHzJk6OO//3ZwdLS0tHJ0dMIwzNt7ZkzMw9Dg s8XFxWS+xZcvX/Tq1RvD8QEDB4WcPYPjeFpqSseOnZ48/luhUPyTlfVRY/qU3ZSiLRGLxR4eHhAE mZube3p6HjhwgBSm/v7+rZ9R2s7OLiMjY/z48RMmTIiNjW0hgylBEDU1NcXFxRoaGvr6+t7e3jEx MePHjyd/LS0tRRCkqVmUTPbW1JIqk8l27drl7+8fGBi4c+fOloWphYVFa4QpJpMQ2Ec8pLhcbkRE RHV1tZubGzW7KcX3CPEOAACdTm8qTAEA3bt3P3jw4OHDh1NTU/fv3w8AwHGcy+WyWCwWi+Xh4TFk yBAAAI/3NpnOX3/95eHhcfv27c6dO6elpSltbykpKZ06ddq/f3/Xrl1Vh2VwHOfz+WRrmzdvNjY2 Bk2Cu6kz9SOhrq7erVu3jxKm3eeEtK0wjYnPvHAzKeGY582otLa1noLG8Vj/4ySak52tr69Po9G2 bN5UWFhga2tn3aGD8vomvTUWL16U+zq3a9funbt0Ubn4IQBA0KnT48aNi46OHjCgf1ZW1lsHVAjI 5f/mT4ARWCaXAQCUORaWLPHNe/3a3tGxg40NGd2vnHdXNVUCk8Wi0WgMBmPqTx7N5hGntCnFp/Cx szw3axn96aefyCTzH8XPP/+ck5NTU1MzevTotz2594CiaGlpqUgkMjAw0NfXb/SabM04vkwmu3v3 7tSpU8eOHRseHk7OvPeZqlRJut+I6qe3W69QaTTahg0bNm3aZGJikpeXR12EFN8vffr0aVYFcjgc Y2NjIyMjZeyjo6OjRCKZOnXq1KlTBQKBnp6esnJDQwMZKxMYGDhr1qw7d+6Q5dnZ2ampqV5eXleu XNHS0srMzIQgiOzROTo68ni8qVOnTpkyBcdxbW3tZh9uVPfvo3jy5Mnt27eVXwsKCvbs2bNr1678 /PzWNxISEnL58mXVXsSlS5c2bNgQEhKiPB1hYWH+/v5XrlxRVmta8vmQlsg2EZF8Ni0mPjPhmCfZ JvmhDeXplBFd3/dTRUX5qVMnpvzkiWHY8+f/TJnqYWPTsbiomBwYhABEvuPS09Nnz55jZ2eXn5+v OmYokUgibt1ydXXbvn2Hh4fngwfREASRFTp36ZKWlkpWy8jI6Njx3+hDDMMy0tNneM/s3LlLUVER hmE0Gk1dXf3NmzcAgLTUVAAADMOGRka6Orpdu3W37mDDZDBVRzY+/B6k7jeKtiUwMPDcuXPKr66u rh+M3GxB6fr7+8+YMcPPz+/IkSPr1q1rJD1VkclkRUVFjcQxOdma8muzGfUzMzO3bt06ffr0mJiY lv1rP2EEH2GyNbu5vT65rvDSPpNJv2p0dYUQWlM7U7Mv9fT0dFdX123btg0fPpy6rii+Oz7KNjlq 1Khr1665urry+XyJRKIqXzgczrVr14KCgrS1tXNzc0NDQ8nOqpqampeXl5GREYqifD7fwcFBKpVu 3bpVV1d31apVU6ZMefz4cWVlZffu3T08PJqu0dbWlqy8cuXKH/UUiMXi1NTUnj17fn7kwNq1a2/f vm1ubu7u7g4AqKurGzx4sJ+fH4Igbm5uCQkJTV1Im+XixYs7duxQfl2xYkVRUZG3t/fp06cTExP3 7du3c+fOR48ezZ079/jx49nZ2atWrWpa0obytPucEFJEfrIBlVy86bymZOPgs0Osmm0/LTXFf+N6 giBoNNro0WN793ZGEKRXr97r167mcDnOvfvExcXGxsZYW1sfDDiAIMiAAQPnz5vL5XEHDhwUGXkv MjKSHJpgs9l37ty5ePGCpqZWQUF+YGCgQKDx5s0bHx+fgwcP3rxxc8aMX2AIUlNTc3Mb8uTJ47fv NQQZMGCg76KFbA67d2/nmIcP4uMfzfCe9ftvWwwMDZWpEjynTQ8JPqOjo1teXjZ4sNtH2U2poQ2K Fq+PjwwRzcjImDZtmqOj459//gkA0NfXT09Pb5PA85iYmIULF7q7u0+aNInL5b4vhJ/8oK2tzWKx UlJSjIyMlAmkmgrTmpqaTZs22djYrF69uoXsqp8sTN+K5sqSZ2vfJuejqWlZzt3Bs3Jo5bIoik6f Pl1HR4dKL0XxlW/2T4vTb3YE/yNuFplMLpeTUwQ1lVkymUxTU7NReXV1NYPBUI7+i0QiJpPJYDAA AEKhkMlktpCMXbXyt3D8v0ScPrm4q6vrr7/+2r9///c5/X8wTv/KlStcLvfEiRNkt+HMmTMJCQkB AQEAAF9fXwcHB29vb6UpjpzdnsPh+Pn5de/eXfWB5uDg8OzZM2XJiRMnJk2apK6unp6e7uXllZKS Ymlp+eTJE21t7aqqKicnp/z8/KYlqtK2e/fuQUFBPB5v06ZNv/32W3l5+bZt23r06BEZGama26HR 87PRPpKT2n+CZGxWlTZCORz/CSKVNMeqlhwIODRr5oxr1280NEjIOP13GaNoNARpkDTweXwmkwkA QafTaTSaWCwWCAQ0BBHV1/P5PCaTheM4nU6j0Wjw21lIIalUqlDINTU0yctAJpOhKMpis3Ecl0gk BIHT6QwMwzEyldQ7qqurmUwmBEEyuQzD8JzsbHV1dTaHm5SUeO/O7YW+S8iJUmtra2k0Gp/H+3na TyEh52bPntmaHafedhRtY/kQi8Vr1qypqKiIjIzU0NBISUlJT09vK2EKAHBxcUlOTj537tyoUaPW rl07cODAFjKhNvvOa/T1ypUrCQkJe/futbKy+hKSVAlT21Cj+9CahDsAAFRY9WLXLLaRVTsPv9Yo VBqNFhoaevDgwWHDhoWGhlLppSh+bFqQklwut1ld1UitquraD+ZMbVYE/5BERkZGRkYCAOzs7GbP nj1lypSPfZhMmDBBdUD/xYsXyhSzHTt2zMrKUv50+vRpFEUjIiJycnJOnTqlqk3//vvvnj17qjY7 a9bbGZKuXr06ePBguVxOptwHAGhpaUml0urq6kYl9fX1yq7I06dPGQzG7du3vby8FixYcOvWrTt3 7vz222/h4eHTp09PTU3V1tbetWtXVlZWy1NMt2DjfN/Q/AcladOaqj6jH9SpfDbtZlRa07XAMKyc 5r5JDwPw+Wo0BAEAIDQaWUddXZ1UnOrq6uTcWrR3P5H+phAEcblcCOIpuyhMJpPBZJBmThaLRXrE vdP0/65NXV0dxTAMRWkIjSDQjIz0yHv3bO3sU5KTPH+ermyNy+UqFApyVaDVdlNKm1K0AWRuTtXR 5+DgYKFQ2LZaikaj/fzzz+PHj/fz8zt27NiGDRvel91JOXCgUCjq6upUf8JxPCMjIyAgYMOGDX5+ fi0YI9swBt9o7AJSm761KxRnl/x1rL3vAXJ8/4N2kYULF/bv3xB2/toAACAASURBVN/V1TU4ONjW 1pa63ii+Nd6XIoriWyM9PX3RokWLFi3S19efP3/+uHHjPu2RopwmFADAYDBUO/92dna7d+/evXv3 8OHDt27dqroUOZ9f09b++OOPhISEq1evyuVyVTM2g8EQiUSNSmQymVKbQhBEThnVpUuXdu3acblc W1vb8vJy0CS3Q2sUZPc5Ic2G2LdehrZGpL57PQW3oFMNNFhkWFXTRtT4fLFYzGaz6+vF39SlNXrM 2J49e1dWVY4ZO45Opzdy5qbRaCiqaP3oHxULRfFZ5OXl2dvbZ2dnJyUlqbpF2tratj4w/6PgcrkB AQHXr18/evTo2rVra2pqWr9sYWHhnDlzhEJhVFSUu7v7++6Tj412+rA1SNtQo/vQfzvENt2VwrSV 2NraRkZGLl269ODBg9RVR/FNQfwv1AH5zxGLxWKxuKKiooWU0qWlpRs3brSzs4MgaPHixR+7Ci0t LWUW6vLyclUzRJ8+faKiohAEmTFjhtIsShIVFeXq6tqoqcWLF2dmZoaHhzOZTC6XK5FIyKsIw7D6 +npDQ8NGJY0cW0mJDEGQ8gNZudncDh/Ujjej0sg8UF8aMl6qWfWpjPdvdkFLS4vc3DwHezs2m/Wt XXhaWlqWllbKWH5VNDU1a2pq1dVbO/0bZTelaIkWTHrkIH5WVlZ4eLjqZNlfB3Nz88jIyNjY2FGj Rnl5ec2cObNlp7G6urqrV69WVVVduXKlhTwjbStJVVGaTjmmHT/t9a2joxMREfH77797eHicOXOG cj+l+K+Ijo7OzMwknw86Ojqurq6ampphYWHFxcWq1YYMGWJtba38WlBQEBYWVlFRYWxsPHHixGZj 5yk+TYlWVFQUFxdv2rTp+PHj5ubmISEhN27cAACMHDmyNS20ZlqvRgwYMGDFihXLly8nCCIsLIxM BKZszdTU9Ndff/X19dXW1j5x4gRZXl1djeO4ltb/sXfdcU0kf3s3HRI6oQYBAWkSUBAQBFSKiCJ4 WBDbq4LlZ+E826mI6NlFT8WzUe5UwMapKIoKiieCDamChRZ6CS1AIJBN8v6xd7lcaAHp7PPxg5vN 7GR2dnbmmW+VE6wnMDCQy+VeuXKFv+JMnjw5Li7Oycnp8ePHU6ZMwWKxQmdEmfpaWlpiYmIWLVo0 d+7cQ4cOPX36tMcUKnzK2C8OTH1GRwNTIUydavn779fGjVNb6DE/+e370lKBl05wZeH9u3UEOi8B fwThLSUIAv9xQOIBPF4nS7/wCR7v30o7/CJ8gEKhTIyNLC3Mk5PfqHYbAwfhpgi+F48fP969e/fR o0fPnj07hM2wsbEpLS29cOGCra3t/v37O/VnZ7PZCQkJt2/fvnTpUkeTIzgA6oCy0r839/IqMlNm QU31OlvOMQs/5Z7bwhedii5qgsNLPX78WE1NrR9teREg6BUiIiK4XK6trS0AAFlZWVu3bn316hUO hyMQCEVFRVevXoVjxsHGbTBoNJqjo6O/v/+kSZPevXtnbm6emZmZk5ODZLT/HiQlJeno6BgbG4eG hqakpKxatYpIJEZERDg4ODg6OgIC0Sg7BazW37BhA5lM7t6BOj8/38PDo6mpqba21sTEZNOmTd7e 3ra2trDxqIuLi5WVleA05eHhMXHixLKysh9//JF/Pj4+fubMmUI1Hz58WFlZ2cTEBAAAPB7/7t27 s2fPLl++PDAwkMFgXL9+HQCAjmd6RMfYDqL3Kt95f5DpaVf+/h0FRtbWU18kvJSVkbGzs5mgrZWd 86W+oQEA/hv2FPyPgEn4rMDHfyJFgv+RRoEA2JlXnPAwgSltF78oLiY2frymgb4el8uNi3+elfVJ xISlAOKnj6DH10CIPNFoNB8fHxsbmz179gwf0R2TyVyzZk12dvavv/6qp6dHJpPxeHxaWtqHDx/8 /f39/f0dHBy60eAPTiPZTfUYcQmYjzbnpQuanPYWdDqdSqWGhoYK0nEIgnJzc9PS0lgs1rhx46ZP n47IVhH0+WXvxk/f29vb0tKSr6719fVVU1ODc0WmpKR4e3unp6cLVR4eHh4bGxsR8bcvSHJysqam 5ty5c+GM9mZmZnFxcWlpaR4eHjNmzNi3b19JSQmFQjl69Ki0tHRhYaG/v39NTY2KisqJEyfk5OS6 8c4e0f0vop8+BEFMJhPeAOjo6NjZ2enr63f1sndKOg8cOCBkadqjn36ngM1MO7qvcbnc2tpaKSkp QXVWU1MTGo3uni7zwWAwhHT3Hc+Isi50GttBxHuEnfd7FGR+54vG/xVANMPWe/cfsNvb8/LzFRUV AR5PRVXFztaGzYZKSkvLyisgiPOP3z4ag0ajUCjY8wmNQqHQaDQajUKhUCgU/38QBOFjmJ/CPlI8 Hiwz5XG5PIHo4FwOh8PjcSHo39DgsC8Uh8NhQxAEQRAbgjgcPB6vrKSgoqIiJyvz5eu3N8lvP2Vn S0hITJxo0NTYvHYt4qePoF8BQdCRI0f+/PPPIVHidw8ikXjz5s3CwkI3Nzc9Pb3Lly/j8fjy8vKg oKCuUo8OGiXlAyvxb5gqkraJiutaQelpr0Amk0tKSmbPnp2SkrJnzx4AAG7fvh0VFeXl5WVlZQVB EIPBmDVr1pEjR4S8YhEg6Pdt4YcPH+zs7LovZmNjs3Pnzo0bNzo5OdnZ2cFitlWrVpWWlvr4+CQk JAQGBt67d2/y5MnLly+fM2fOhQsXbty44ePjc+fOndDQUA8PD3d394CAgKNHjwYGBnbjnT3S+5NO pwuyNzqdDh9UV1fHx8erq6unpaUFBwefO3cuJycnNja2V/vPLVu2LFq0qF/CncLoKqgCCoXqqNXp VVSEjjS0t8QU6Dq2g4jg6/cHGqITUwAA6NV0H5/VHA7n+YuEV68ScThccEiYhoa6na2tgb5eeUVl RWVlI6Np8HX6RHFxeXl5ZSVFGRnphoaGd+8/pKdlMFta2tvbnZ2d7GfOwOFw166L2p8IN0XQHfhD E1bi79q1S0Sj8iGBpqZmZmZmeHi4vr4+HKhfT0+vIzEdfFbaKQTpKYcH9Ha1wGAwcXFx58+ft7e3 l5OT8/b2hnOR8/H06dNZs2aFhoYOt40EglGAQ4cOXbp0ic1m0+n0rVu3zp8/v2OZL1++5OXlAQDg 4OCgrq6enZ0dFRV18+bN//3vf/b29sHBwYKSJAMDA2tray6X++DBg1mzZv3xxx8cDgcOWrR3796Y mJiTJ0/m5+ezWCygW+/skQ5fX1/YSBSCICwWe+DAAfj8t2/ftm3b9ubNm7S0tAsXLpiYmMTFxYlY Z4/BTccmpqyN6JEODg497a1ctr2dnZT8xtx8iqWFxYuEhJLS0uLikojIGxISEoaGBgZ6ulgsrqGh gdHY2NLS0tYmGNi7P3X6WCyWRCISCAQikSgpIYHFYkpLy1JTU4uKi+vrG+DftbQwt7GxhiDor1ev jSb2wkYC4aYIegCsxDcwMEhOTh4RU9uyZcs8PT1/+eWXzZs3L1iwYLhR0o709OI8gy2xuRC3L9Y1 dnZ2aWlpb9++VVNT60heHzx44OPj0ytbKwQIRIGfn5+3t3dzc7OJiYmzs3OniuOCggLYycbGxoZA IMjIyPj4+Pj4+EAQ5OTkdOvWrf+8CyQSvCiCIMgXsAUHB3O53Dlz5kyfPn3WrFk4HC4xMRH+qivv 7JGOs2fPwnJTDAbD4/GYTOb+/fsBAJg2bRqcEHLTpk29qrC5uRmhpN/DCPtXod9P9fPy8gsKaUXj xqlZWlrOdp71KTsnLS29srKqoaEhMfG1uLiYAllBS1tLfdw4KSnJ1lYWnNWCzYYgDsTlckUPqQGC IBqNxmIwIAqFQqEwaDQGg8HjcTgcDgTBysqq8vLyT58+VVVVoVBoLpfLgThNzY3q49SNjalaWlql ZaXxL14WFRVDEGRMNeLHSUW4ae+QmJi4cePG6urqqKioadOmjfHegCDo0qVLwcHBw1CJ38OwxmAO HDiwadOmysrKYUhJhejpX0UN52br8DhQb5X7SUlJv//+++XLl+vr6728vFatWiWUm5FIJOrq6kIQ hBieIhiQ0UsinT59etWqVW/fvu04xlxcXPj20Hv27JGWloazg6LRaCwWKy4u3tTUJBQEEY1GGxoa amtrm5mZ1dbWJicn83i8N2/exMfHo1Co69evC5UffRBShX8/rUSI6ShGeXlFVVW1nJyshvq4xYsX olGoT9k5X798raisaGlppRUVgSAIp46SkZEly8vLy8vJyspKSEiQSEQQBLlcLvdvy1IAFpQKMFIA TrIIAACbzW5qam5sbGxgMBoZjNrautq6OjabjcPhpKSkeFwuxOFAEAdit2lqakyYoKOlNb6mprao pCTq7n0Gg8Fms/+pFEAhsfd7CxqN9vPPP9Pp9NDQUAAATp48GRAQEBISMmb1oUlJSQsWLKisrAQA YIR2AplMHhGe7Lez6ZJ4TG9tT7Oysp49e3bp0iUMBgOHl9q2bVtMTIxQeCkdHZ3S0lJErY9ggDBv 3ryrV68eP35879693RTz9fVduXLlnTt3lJWVi4uLbWxsfvjhh9evX8MZ7QV9mM6fP79u3TpdXd3c 3Nzt27ej0Wh3d3cnJycJCQlXV9e7d++OAqNSBAj6DB6Py+H8R/rIYDR+yv78+cs3BTJZa7yGmakp CgWWlJRmZGR++5ZbWlqmrj6OyWS2tLSUlpUxm5m1dXUNDQ0EPJ5EIomJiRHECEIbSwiCWK0sFovF bGE2NzMJBIKMtLQ8WV5WVoYsT1ZToygokMvKy2m0IhqtSEtrvJ6urpGRIZFIbGlpLSkte/I0ntHY CDtHCVaLw+O5IqsHET99gMlknjhx4vLly8HBwXAkCxjp6el79+6dPn360aNHx9TWk06nw6GYg4OD +Yo2ZEYY8B1wTHBzbpqI9JROp/v7+wcFBQnNKbBZcHx8vCApHzXqTgQDCtH99PuM9vZ2BoMhLy/P twHoKqN9XV2dtLQ0PzFjXV2dhIQEnGwGdjAelf0vop/+QD99UGTh1ggnebzh8KL1FsHBYZ6eC2/c ikKj0RgMhv8XPsBg0FgsVlZGRk5OVlpKikQicrnc8vKKqqqqmtpaejW9sakJBAA0BoPFYjFoNA8A cDgsj8sDUSgMBoPDYmGffR6Ph0ajUWgUCIBMJrOZyWxpaQV4PBAFysnKKioqkMlkJSVFWVnZ1tbW xqbmurp6Op3OaGz621+fzeYf8I/Xeq+6e+/+qv9bKcptjmm5KQRBN2/e3L59e0BAQFJSEgaDETSG oFKp9+7di4uLmzp16s6dOz09PUe9YpSvxEdyYw4+VOb6lMcEiyI9hSBo586dJ06c6FSLamBgQKVS o6Ki4LxcSUlJhYWFSPciGA7A4XBCqoyufLeF4v7wPyLWKaOVtCFb6P4Cl8Otb2hobGqCY0iJi4tL kEiqFIqWthYBj8fhcHBUqaam5pYWZmsrq729ncVicblcwcSzYmJiIAolRiAQCAQxMQK8M+RwOBAE tbW1t7JaW1tZlVXVuXkF7e3tHA63o5S0E7qJwYIAotPvCZmZmcuWLbOyskpISIBduTu+GGg02tnZ 2c7O7uzZsydOnLhw4cIoNkLNyspycnI6derUx48f+QsAYkc/JPR0wtYL3RS7ffv2woULuzJX0NDQ yMvLc3d3t7Gx8ff3z8rKUlVVRfoWAQIECEbJ5qHH7wRKtLW1cTgcVGOjYHxTHA6Lw+GwWCyJREKh UWg0GgWiUCiQ78jP5XJ4PB7E4XI4UDsbqqyqamtjQxDE5XI5gvhbnMfjAT1vLNDoXqg7xiI3pdPp mzdvrq6uvnjxIpy+snvfMTwev3PnzoqKin379ikoKBw7dmyUWe8xmUwfHx8AAPLy8oSYKEJMB+0R wF0N09PymGCVuT5djd7Xr19fuNAdeSUSiXFxcQcPHnR0dJSRkVm/fj3SwwgGDkIqYET6hUB0GBsb Z2Zm8gdSYWEhYhzf8xvX43dgJ68n/D/8l8PhtLW1sdkQCgX73/cce/8fe49OWwKCQM+GICAIgiJb 44wtbgpBEByWLzAwcOrUqb2aRpWUlIKDg7OysiwtLdevX+/n5zcKtEsQBN2+fXvbtm18FTCCoVra +UOxe3p64cIFfsjD7uHv75+UlDRv3jwajYZM9wgGFPyE7NOnT++x8Ldv3549ewYfS0pKWllZaWtr v3///v3794LFDA0NZ8yYwd+83blz5//+7//gjw8fPlRTUxN0D+ADzk585MiR3bt3JyQkiNj+8PDw hoYGEASlpaWnT5/elaoBrhyOtAofmJqaIk//e+Dj47N582b+IovMVKMYXWVq6BSosdMv4eHhFAoF j8cnJCRYWFhw+wRDQ8OEhAQFBQUKhRIeHj6iOyQrK8vU1LSurq6kpAQhpsMKMCstjwkWOl9dXc1m s0UPPiAjI/PTTz+5ubn1e4hTZWVlEEFfAatrxizev38fGRlJIpFIJFJNTY2zs/OdO3cwGAyBQCAQ CNu3b+fxeAQCAYvF8i+pr6/38/ODjy9evPjbb7/p6+t3WvnZs2cPHz5sYmKSlZUlepMCAgJaWlqI RGJJSYm5uXlX18KVm5qa8g+Qyeo7sXjxYv7xhg0bkA7pETyAxxE5SujIxZiQm8Kmpbq6uk+fPu3K tFR0oNFoBwcHa2vrQ4cOnThxIiIiYsS5DfGV+EIO3R0xOC6iCDqlpx2lpz/++KNgNp0eERUVtWHD hl27dm3btu3du3enTp3qL2E/HFwMGRh9wxjxg+4G48eP5wtBNTU1Q0NDY2JiJk+eDA/y5cuXS0tL d3rh77//fvfu3QcPHsAymBcvXgQFBbW2tlpYWAQEBFy+fDknJ2ffvn2HDh2Cy7PZ7C1btvj6+uLx eH9//5qaGhUVlRMnTsjJyQnV/MMPP2hrawMAkJeXl5ycrKOj4+Pjc/36dQAAHjx40NjY2NzcDFdu bm4OH5w8eRKDwRw6dKi1tdXS0vLnn3/mcrlr167V1tZuamo6evQoMtR7BJlMplKpsFof4aYikTYM BmJD3VDXf//Ch//M0vD/PJ6g9v17c5b+mxi1J3tTHA7H4/GwIi9Ao1xuSqfTPT09ly5dGhQU9Msv v+DxeG4/AY/H//LLL0FBQV5eXp6envysx8MfkZGR2traGzdujIyMHBHhP0f/PriLFB1C0lMmk6mm pia6BTAEQSwWi0wmYzCYs2fPWlhYmJqajqCBimAUUHAGg9Hj4P/rr79ETJBx48aNK1euREdH8xMR nzp16ty5c48ePUpKSnr06NH69eu1tLTOnz8Pq/t5PN6qVauoVKqenl5oaKiHh0dsbKyamlqnrDE5 OTk+Pv7atWs5OTmurq4QBPGTghYUFHz58oVf+ZEjR+ADPT29BQsW+Pv7R0dHFxcXX7x4EY1G37x5 U0pKavfu3cgAEBEeHh4AAFCpVGQ96o+3TuDvf3fCgvamgqWFcpb+XR6EbUjBDi91J78mir0pXDNX ZHHGqOWmEASdPXvWyMjI2dn51q1bZDKZOwAgk8m3bt1ydnY2MjI6d+7cMM9ZQqPRjI2NESX+CIIg PX3y5Imnp6fo18bHx8NG1TC8vLzCw8MdHBySkpKQjkUwOJCWli4rK+t4PjY21szMzMzMTFZWlsVi 8cWcQpKFmJiYmJgYuAY6nX779u2ampqioiJ+mbCwsGfPnp08eZLFYgmeh7Ft2zZ1dXVYGrd37142 mw2bihYVFXG5XLhyviPOu3fvXr58mZKSIi8vX1paKsrdpaSkQBD04cOHq1evSkpKxsbGgiDY3t6+ fv16SUlJ5OmLiPnz5wMAAKvyEIxWYNBoDoeDEzDUGYvc9NGjRxQKpays7OnTp1OmTOENMKZMmfL0 6dOmpiYKhfLo0aNh2CFMJtPX19fHxyc6OnrTpk2iK3bZbDaitx0m9PTSpUuTJk0S/cJTp07Nnj1b 8IyRkVFycnJAQMD58+eRjkUwUIKbfwAAwMuXLykUSsfQlbNnz05JSUlJSZk+ffqkSZM6ZXJ1dXUv X758+fJlVVUVAACysrL37t0LCQlxd3eHxf+1tbVWVlYEAsHBwUFTU1PoJ+rq6ioqKl6+fAmLDObM mfP582dbW1szMzMej8flcuHKv379CpffunXroUOHzp079/PPPwvlrO8qkAuXyxUTg0NAEiZPnvzj jz8CAEAgENBoNDIMRIeRkZGSkpKg4SmC7sDj4fC4kcdNsZi2tnbRucdoszel0Wjz5s2bMGFCVFSU lJQU0FN8qP4CCoWaN2+enZ3dmTNndu/e/eDBg+HjbxgZGXn8+PGjR4+ePXu21+MJiXQ9KGs50K3t pspcn7yo8ypYdq8e+urVqzs+PiKRGBsbe+TIEUdHx9jY2BH3fL99+1ZUVOTo6AgAAARBf/zxh5ub W0dVIJPJXL16tYKCwpIlS+7du3fy5ElkmA0O+A77QmdQKFRbW1vH/E9BQUGWlpazZ88eN26c0Fe6 urqBgYHwcWlpKUz47Ozstm7d6ubm9uLFi6ysLA0NjaVLl7a3t2dnZwt5JsnKyt64ceOnn376+eef jx8//ubNm/j4eBQKdf36dQiCMBgMv3IhFBYWSkpKiomJMZlMNpuNxWJTUlI6tTowMDBgMBhubm5i YmLp6emjbEYaZCgoKAw6xxuRYhcJSQkMGqOqqlJZWdXZXQn8/e9tDq29qaqKSmNTo5S01Jjjpkwm c82aNdnZ2UeOHFFXVx+SkScpKenv719UVOTq6mpoaBgaGjq08UFpNJqbm5uHh4dgOH0EIxEMzaku VvndxD0VBARB58+ff/XqVVf7DTi8lJqa2ps3b0ZW0Jb3798/efLE0dGRy+WuXLlSR0enUxu15OTk hoaGW7duPXnyBEmLNUw4Kx6P7zgnUyiUXbt2eXt786NK9Yj169dnZWWtXLnyypUrlZWVixYt4nA4 y5cvv3z5srOzs1Dh48eP29jYxMbGuru7Ozk5SUhIuLq63r17Nzo62s3NTbDkvHnzYHcNGRkZ2HL0 hx9+gONJEQiETlcTMpns5+dna2uro6OTm5t79erVUfO8Rr26bOT6IzrPmvXXq1dWUy0exz7rxIzw u+1N/376IAB2lqyrb/amEhISNtOsnz6Lc5k9S9QHNAqGIARBFy5cgAPa2dnZDZNWvX379sCBAx2j 2Q9anxw5ciQxMTE4OPh7yAfipz9os2T3nRwZGWllZdV+7wRJZ1KP9HTPnj1LlizpMXwEnU53cHA4 evSoi4vLSBkY4eHhT548uX79+po1a5SVlQ8fPgwAAIvF2rNnz5cvX/B4/P79+zU0NGbPnl1SUrJ0 6dIZM2aEhIRERUWVlpYGBAScOXPm/fv3gp7dQ7JEDc/0jELPtKv07p1e2FFcKoTp06cPxC3zeLza 2lo5OTnY0LOjXFYQdXV1cOpFCILgeOM91l9fX08kEruvlsvlNjQ0CCVZ/c5RIXQ8CO+a4K+MhfSh Q3iP3/9Av33LbWlp0dHRfp7wV2NjEwaDQaPRaDQag8FgMGg0Co3GoNFoNJyzFP4WBYJoDBoFZ4AC Uf/kh0KLHnufw+FyOBwejwtB/8kMBUEQP2cpG/obbDabf0CWl3eZ7ZSWniFBIhkZTRTxHke8venr 168pFEpJScnDhw9tbW15wwAgCKqoqKxevXratGlD4hb9+PFjU1NTbW3tuLg4JJTx6ICbm5umpqbO lnPNuWkd454K4t27d1gsVpS4ZmQy+ePHj+Hh4b6+vsPcjU8IGzduFBcXh4kpAACxsbEkEunx48e+ vr7e3t7S0tJ79+6dNm3a8ePH+Qxj4cKFa9euJZFIQp7dyNDqF0zvAgAAYLHYASKm8DIvLy8PL/bd M0gAAGRlZeGYqXA+cVHql5GR6bFaFAr1/cQUAQLRMWGCTm1dXVFRsbOTg5zcf8eeCDp9odL/BIoR PO43nb4aRdVt3pyMzKyG+gbRiSkwonX6NBptzZo1MjIyN27cgE3pucMgIK2srKyamlpJSUlKSgqb zR78PvHx8TEwMEhOTkbSjY4UiLJsw5scEI3R2XIu99yWrpT71dXVfn5+/Ng3Pb//GExkZOT58+dn z549UmKKPXjwYN68eV+/fuUnenV3d5eUlDx9+jSDwejorN3a2urq6nrw4EFzc3MAAMLCwmJiYuh0 eqee3Qj6cQCDIIjFYq2trRHFy2hFb+2/dXR0FBUV++b2dO7cuU6vTU5OHoM25fYzZ7x5+y4h4S/7 GXafv3wrKCj8R8wu8Pef1/Cfg3//8r8EBkynTyAQDPR1J+joPHgYIy8n5+Li3KsbHJFyUyaTuXnz ZldX19WrV+/atYtEInGHAcTFxQ0MDGRkZF68eJGdnT3IRBmCoIMHD7q5uQUHB589exYhpqMMycnJ sN0kTE87lZ4yGAxHR8eIiIjeVr5p06bTp087ODj0KpXOUMHFxSU8PHz+/PleXl7wW/bLL79cunRp ypQpjo6OHWlQYmKinJzc06dPgZ48uxH0OwSJKZJxYPTh/fv3sJUtbP9dWlrajf13UFBQeXl5n3WJ q1evdnV17Xi+sbFxbNqUT7W0mGw66U7UXUkJ0hwX54mG+mIEwnBomLSU1FRL8wUe7gAA3Lh502aa ta2tTW8rGWHcFIKg8PBwLS2t8ePHX758WUdHZzgo8bFYrJaWloaGRmpq6uvXr9vb2we5W/hK/IyM jP5V4ncVFh5Bvw/s7gsI5mDk09O690/4Beh0uo2NzatXr/rm7mpkZBQfH//TTz8N//BSsFffnj17 xMXFt23bBgDAixcvtmzZYmNjU1lZ2bEnnZycoqKikpKSYmJi+J7dVCo1Ozt7ZFkyjDgIzR7ITDKK H7S3t7eGhkZAQAAAACwW66effnJxcZk/f356enpDQ4O/v392dvauXbv4l8THx8OFX7x4MX/+fGdn 5/379wuNkKCgIBcXFxcXF3i//eeffyYkJKSlpQUGBu7YdTY4XAAAIABJREFUsWPmzJk///yzYPnS 0lJvb+/m5uax0/NkefmlXp50Ov3W7duNjMbpdjamkyepqioTepO5vr8gKSU5QUdrup2NjY1VcVFR WNjv7W1t/7dyRd/UcSOJm8Kmpenp6bdu3Zo6dSoKhRpyWSkAAMrKykZGRrW1tc+fP6+vrx/kPqHT 6V5eXk+fPk1OTvby8kKI3UgErPrsvsyECRNevHjx7yVojM6WczVJD5rz0uFhMH369OfPn8Nx0/o4 zZHJsbGxdXV1Xl5eI6JLw8LCEhMTL1y4sGjRoo0bNy5atKi0tFRVVRVe8wSBxWJv3Ljx448/kslk 2LN7yZIlsGf3ly9fEG6BAMH3oLf23x8/fvTz89u8eTPQIbMXv87Pnz+HhYVFR0ffuXMnOTmZxWJ9 /vy5sLCwoaHh+PHj3t7eT548efLkyYcPH+DygjblY63/zc2nLF+2FESB58//lvstV1trvLWV5SRj qrKykphAiImBsDdFoVDS0lJa4zWnWU+1sjQny8u/fp10/vxFBQUFH+81pqaT+y6DGCm9Hx4e/vjx Yzj9xjDZgsvLy2tqatbU1Lx//37w13IIgi5duhQcHBweHi6K48toRUxMjLe3NwAAd+7csbGxGa23 icFgtLS04uPjHRwcBOnp5183fZLQj0v5lJGR8f1hwuDwUo8fP1ZTU8vMzByG5qfLli1btmwZfCwm JpaSkgIfr1ixAgRBIpHo6+vL4XAwGMzcuXMBAHB2dobjCmloaOTl5QEA8OnTJ75n98aNG3v0dEHA 30EhnYCgI3pr/52Xl3fx4sW4uDg5OTmga/tvFRWVxsZGPz+/uXPnBgUF8X3XQBA0MTHR1dUFAMDQ 0LC8vByPxwvZlI9NmE6eZGJM/fQp+8+799va2tTVx+loa1OpE3k8Xmtra1tbezu7HWJDEIcjYHDY C3tTNBqFwaBRKBQajcZisTgsFk/AixEIzc3MkpKS1NTUqupqOVnZiRMNXee6fP90MWK4KYfD8fPz mzBhQn5+fk1NzdA2RlxcXFdXF4KgT58+DYmEKSkpacGCBadOnRrowKXDOYYUnGdBS0vr3LlzbDb7 119/DQgICA0NHa2hCdavX79169aGhgYzMzMNDQ0ajZaYmHjyWvJOm4pfdx3ox2Hg4uKSmZlJpVJD Q0P7EF5qSMAXloAg2H1XwJ7d8DFCTEUEoo5H0M10ER4efuTIES8vr3v37qFQqF9++SUrK2vLli0g CP72229C5WNjY6WkpFJTU1VUVGD774MHDzo4OGRnZwsOMykpqbS0tHv37p05c2bLli2vX7/mf4X/ R2HNZ06JiYkzZsx4+vQp7JU1ZoFGo42NqcbGVB6PV1pWlvst90XCyzYWS5xIJMvLy8vLycvLycvJ kUhENBrN4XC4XC73Hw97WGD6j78UyF//QRAEUSg0CgUAAJPJbGpqrquro9fU1NTU1tXVcTgcGWlp dfVxNjbW8Gaj38QxI6jf6+vrnz17ZmZmNn78+JycnKampiERX2lra0tJSeXm5jY3N4sYiKQfQafT fX194d3nmHV44udZ+Omnn/iyvdWrV5eUlLi6us6fP3/Xrl0jqHNEXPgxGExQUBCNRouJicnNzV24 cKGJiUlmZiaPA+We26LiupakbdJfTSKTySUlJXBiyT179iCJGxAgQNDVvAT8E1N527Ztv/7664sX L3755RcbG5uoqKiOshvYj9ne3p5KpRYUFHSV2SsvL6+8vHzlypUrV66EmWs3bXBycrpx44atrW1M TAysMBnjAEFQjUJRo1Dgj21tbdXV1TW1dd9y8+rq6hkMBgRBPB4AAFwAALFYLAqFwuFwaDSaxWIR CAQel8tqawMAgNXKAlEgl8sDAACLxYiLi0tJScnJyigpKk40NJCVlR04CjTClhwul/v+/XtZWVkj IyMOh/P169e2trZB+/Vx48ZRKJTa2tqsrKzBj1cFQdDt27ePHz8+lpX4/DwLmzdv5ut2+VBTUzt8 +HBWVpaWllZgYKCnp+foI1UaGhpCyb75gaX6l55iMJi4uDg4vNT9+/eRyA8IECDoBmFhYTY2Nnz7 bz09PSsrK9j+28zMTLCkuro6PD/HxMR0zOylp6cHAICkpOTKlStVVVUhCJKQkDAxMbl79243vw7b lDs4OFCp1I75b8c48Hi8mpqamppajyUjIyO9vJYMC3o9UpQ1V69e1dbWhp2NQBBEo9Fqamra2tp0 Or2goGCgmSIcmK25ubmyspLH47FYLNgXCt40wCkQGAyG0FUhISFnzpzpFxVzVlbWsmXLfHx81q9f P5h8i29CNByQmJi4cOHCRYsWOTg4dL9dY7PZDx8+TElJuXjxorW19ViYfQZCegojKSnpf//7n9CO CEkY9p1SjZGVFwrBEI6QkZgXqrm5Gbb/5vF4sP13lxNXt5m96urqcDjcQLg3jei8UAOHyMjIYeJU PYKlSmVlZVVVVdra2lOnTv327RudTh+Ih00gEKhUKgiChYWFQzKYmEymj48PAADx8fGD75syTIgp nGdBUlLy1KlToriiY7HYH374wd7efu/evUpKSkFBQcM8qvz37wEGSHoKAIC1tXV8fLyXl9eqVavG bCyIMcuQkE5A0Af0l/03knBrzGJkazy5XG5eXl5xcbG+vr6Ojk5mZmZH4WWfgUKh9PX1ZWVlq6ur W1tbuVzuQM/UsGkO/02Glfjbtm2LiooaI8K/Tknbrl27/vrrr02bNomikhCEjIzMzp07S0pKjIyM 1q1bt2/fvuGp4u+vbfTA0VM4vNS2bdtiYmKuXbuGmJ+OBSBCUwQIEAwVRljsfTExMTqd3tLSIkTp srOzc3JyDA0NjYyM8Hj8d8bSBwBAVVXVxsZGTEysrKxs0ExaV6xYcfv2bfiYRqOZmprW1dWVlJSM TWLKz7OgoqJy7NixcePGgX3CuHHjQkNDIQiiUCjh4eGju9Ngelr+8Aoc97Q/d7EYzNmzZ5ctW2Zq atrnzC4IECBAgABBv3HTDRs2FBYWpqamqqqqGhsbl5WVDUlzZWRkdHR0CgsL8/LyOByO4FfNzc3p 6elNTU2WlpYaGhogCPYtnL6srKytra2KigqNRhvMWPoHDx68cePG0qVLaTSar6/vnj174uPjN23a NLQyqn8inw02EhMTJ0+e/OLFiytXrkyePBmNRoPfATQabWNjc+XKlT///JNKpY6IzJzfQ081vQ/n X97V7/QUAAAXF5fo6Gh+jFUECBAgQICg3yES70lPT2exWJqamrt27QoNDQUA4Pfff/fz8xuSFktL SxsaGtbX16elpWlqagplaKypqamrq6NQKLa2tp8/f4Zdl0SsmUAgmJqaotHouro6CILExMR4PB6b zR6Em3r8+PH+/fvh40mTJkVERIyUuJL9DhqNtmvXrqqqqj179sjIyPRjzXg8fv369fX19V5eXoaG hqGhocPEmrbfladYCRkD/5s5Bz211h3vd9coDQ2N5OTkMZh8BQGCsQPE1BjB0EIkuWlNTY2RkRGb zU5NTXVycpowYUJlZeWQNLempkZbW1tHR0dRUXHSpElNTU1paWkd8+eWl5d/+fJFQ0Nj6tSpkpKS PcpKQRDU1dW1tLSEPfFbWlra29sHLUpUUlLSnDlz+B8bGhq+J/PkyAWTydy/f7+lpaWDg8PevXvl 5ORQAwA5ObnTp0+bmJjAQftHa0Z1mJ4OkPSUz+kdHR2ZTCYyjSJAMJrAGwoM/u8iD3rEc1MjI6OI iIg1a9Y4OjqiUKjQ0FB9ff0haW57e/vz58+ZTKatra2qqqqWlpaBgUF+fn52dnZ7e7tgSQiCCgsL q6qqTExMqFQqBoPplJXyeDwlJaUZM2bIyMjU1NSw2WwCgUAgEHA43OAo0+l0+oIFC4ROLliwYJhw JjabPTjvMGxaKi4ufu3aNU1NTXCAYWZmdu3atYKCAgqFEhMTg9DTviEgIMDKyopGoyEz6SgAlUoF /jHjQTB8wH80g4Chvc2x80AR9A83VVRU/O2336ZOnXrq1CkAAKSkpNauXTuEW7ry8vJXr15JSUmZ m5srKioaGxsrKSm9e/euuLhYSNjZ2tpaWFgIguD06dO1tbWFdmbS0tIODg5aWlpVVVWNjY0AAKBQ KDabPTh6fFhSSKVSBYXQS5YsiYiIePPmzTAZH4NA0LOysqhU6t27d8PCwszMzL7TtLRXRqiurq5X rlwJCwuzt7cfQoI1cBMWn56ymwbEchoOL+Xm5hYZGTk8FzxkXREdGRkZPATDEhkZGQP99A8cOIDw oUHDsO1tdXX1YdISUcPPslis6Ojo+vr69evX5+fna2lpDXJDO8beR6FQ4uLiBgYGbDa7oKCAzWYX FxeXlJRMnjxZXl6er8OFS6JQKDKZTCQSP336VF5eLiYmZmJigsfja2tr4cjAIAjC4dzhFYjH44Eg yOFw2Gw2LGHlcDiwf1V/xd6HIGj27NnV1dUeHh5mZmZTpkwZ5jE4B0JmvHnz5uzs7ICAgP41Le0t Kioqjh07Nn369GPHjg2+EepAh2JmN9Xn/rpBZ+tFrITMQLQZgqBt27YBAHDq1CkMBgMP7NjYWP7G ZtREce+vG0HC2iNAgLwUwxA0Gq1fsgV9P0SSm7a1tdnb28fFxV29ehUAgLVr1yYmJg6H1rNYrPT0 9LKyMlNTU3V1dS0tLUtLy9zc3NTU1I6Bn2pra8vLy3V1de3s7KZNm8Zms0tKSlgsllAxNBo9aG/I /fv3MzIy/P39XVxchicxHSBZEQRB+/fvNzIysrGxCQoKGiDTUtGhqqp65swZNTU1LS2t8PDwUWaE ipWQ0dl6MffXDQMkPYXDS1lYWMyePZtOp69YsSI+Pv7SpUv8AvBObxT05Ki5EQQIECAYzhCJm6am ptrb24eEhMA+OqdPn46Ojh4+99DY2Pj27VsIgkxNTdXU1CwsLFRVVV++fPnlyxchFT+Xy6XT6SgU qri4uB+j9Pd5RR+bOcofPXpEoVB4PF5ERMTEiROHia4Wg8FYWFhcu3YtNTWVQqG8fv0aoae9gpeX 1+nTp01NTW/cuAEAwObNmwXDoCL0FAECBAgQ9Cc3hSBIkOTV1tYSCIROS05ZG9HNx24gesmuUFFR kZaWBkeY0tDQcHR0xOFwsbGxFRUVvaoHgiBk7Rkg0Gg0KpUaFhYWFhbm4OCAxWKHmzUhgUBwd3e/ ePHivn37Fi9ePDhGqIPjNDoI9BQAgJKSEkG2OipZHUJPESBAgGBAIZKni6Wl5Y4dO6qqqgoLC3fs 2PHnn38+evSo05IfrizlH7exOb1qShubg8eiv+dmuFxufn4+gUDQ0dFhsVg4HE5NTS0zMzM3N9fc 3FxSUlKUStBoNIDk6+tvMJnMNWvWZGdn+/n5qaioDPPWysvLHzx4MD8/39LSct26dTt37hzOEm4m k0mn02VkZHoMPcanp/1re8pvhpOTk+CZ+Pj4yMhIQYYKs7pR8HKNmhtBgAABgmEIkeSmWCz25cuX FhYWHh4eFArl1atXosSQmrbxZq+a0tvyXaG9vf3z58/19fV6enoqKiqWlpaGhoZxcXFZWVk9hiyF XZ2Q4GeCa3AfukIw5iUEQefOndPS0jIzM7t06RKFQkGNEOjo6Ny6dUtCQkJLSysiImJAjVD7VnlS UlJ8fDxscp2UlHT9+vUes4kOnPSUSCRGRUVt2bJF8OTSpUuFmiSK0DE8PPz8+fPnz5+/ePHi48eP By1uRh/o6Wh62ZWVlZGATcMTxsbGo37wAN8X0OPgwYPIej3muCkAAAQCwdvb+8iRI76+vhQKZfjf WGNjY05ODhqNnjBhgoaGhpubGw6H+/PPP3voDhQKdsDvM11AAAAAiURauHAhjUZ7/fo1hUKh0Wi3 b9+2srIacUsCFou1srK6cePGy5cvJ0+ePEDJTuEf6gMxhSDIwcHB2tra2traxcVlyZIl58+f7zEY /sDRU2tr67Nnz7LZ7EePHvHzmgpp9kVhdQEBAS0tLSQSCX5np06dOqD0NDAwMC4uDqGncDA7JGDT cAMAAJmZmSNi8Awh+IkVEYwO9KDTf/v2bXZ29owZM3bu3Cl4ftGiRYsWLermwsKq5j60prCqWVOx P3MhVldXNzQ0KCsrKygoEAiEmpqa7mVXguE8USgUIj3tM1xcXCwtLWfOnHn16lURrSmGLQgEwurV q93c3OBkp0FBQUMeVIFOpxcUFCxfvvw/LzMGs2/fvtmzZ/fItAZCuU+j0Xx8fLZu3WpnZ+fi4uLi 4sJkMqOjo48fPy6k2QdE0In/8MMPcEDiNWvWGBsbZ2RkmJmZ3bt3748//kCj0UuXLvXw8AAA4Nix YwkJCbq6ukQicc6cOXg8PiEhAZ6sDh8+PHfuXGNjY6GrWltbd+/e/fXrV3Fx8Z9//hmCoKCgIFVV VRKJxGQyg4KC2trazM3N/f39RQnuiyj3ESDgvwvfuUXvcw2I/feY46aamppSUlItLS3m5uaCqSng ZaMbLNoXbWtl+Co5W/Sm2FoZLtoXLWix2i/gcDjl5eUEAkFZWRmPx3e/xnA4nF69bI11VRXFeeMN zUfr+Ohz6E01NbWcnBw4wxaNRqutrR3pXUEmk0NCQtLT042MjNavX+/n5zc4mcM6ha+v77Vr1zp5 nzGYyMjIgwcP+vv7i0JPcw56Gvjf7C96Gh8fHx8fDwCAg4MDTFK9vLy8vLw6FeWKyOoKCwsrKioo FEpGRsaBAwdevXqFRqPt7Ox0dXWrqqru3bv3+vXrsrIyExMTOzu7hoaG1NRU+MIPHz5YWFh0vCox MRGCoNjY2Pz8/N9///3QoUNTpkzx9vaeOnWqiopKenq6vLz8yZMnc3JyBi0ZDwIECBAg6AU3VVRU VFRUzMjIeP36tZDotBsIekH11gH/+z2iOkV7e3tJSYmlpWVXBYSEpj3uxiA2u66q+EvWx23+J39w sli+ZhNGXBoZT0Kor6//8uWLhYWFiopKfn5+a2vrSL+jSZMm3b17Ny4ujkKhhIaGzpkzZ/DlDZGR kbt37+6KGZPJZCsrKyaT2aP/FlZCRmvd8f6lp12R1G42hJ32wPz58/F4fFNTEwaDCQ0NVVJSCg4O VlVVvXv3LgAAKioqT548aWtrc3Z2xmKxGhoaU6ZM6fQnHjx4IHTV1KlTAwMDAwMDXVxcDh06JFjY xMTkxx9/XLx48aZNm0T3fuPfRWFhodBXRCJxrCXUQIAAwchFTk7OMIm9L5LgR1paurKyUkdHR0dH B86EBItDuio/beNNWytDAADgv6LD1spw2sab/S465YOfyanj6gJbmorOG+qrS+fM91SQk7x2fPM4 NZXF//vf4nkO9m4rUGgMMr4F0dra+vLlS3l5+cmTJzc1NRUXF490Q14sFuvi4mJvb//rr7/u3r37 +fPn3fMPCIL6YE46qAiU7dt1ParSPn36lJKSIiUlZW1t3T2xE8K9e/e0tbU/fvy4dOlSe3t7+OUl Eolw9DovLy8DA4M7d+6Ii4v/PZF1YOrw69zxKiqV+vz583v37q1atcrIyCgkJIR/ycOHD2NjYx89 euTr6xsdHS26A0pXd8FkMjvKjJuamgAAyMrKkpCQgM8kJycbGRlJSEg0NTVlZWVZWVnB54fJItFn PHnyREtLS0dHB/745s0bPB4/efLkgf7d/Pz8AwcOdKpYEERERAQej1+wYAH88dy5c4qKiosXL+7D L6akpFy7du3cuXPIhC+Eb9++FRUVOTo6wjPhH3/84ebm1tWEOXPmzICAgIcPH548eRLpuiFBQ0PD MGmJSERKRkbmyJEjgrP/gE6aHC4PjRpU8xE2m92VFAqFQgkp+luZjZdOH3RfvGaiNuXIvq0UfI0R VZuWdOvOgydrV6/Ysm451Wr2d7aHTqcPE3FLc3OzkACpD23jcDg1NTXx8fG6urpGRkZVVVW9DTo7 DEEikUJCQq5duxYXF9fNPg2mTf1ojwhB0LNnz1xcXLov5ujoKLp/T3Neev7lXb2SnnaUd9JoNE1N TfhYSUlpw4YN8+fPHz9+/F9//VVUVNQVN+1erW9qauri4rJ79+6zZ89OmjQpJSXF09MTJj2Kiora 2trPnj0DAKCtrS0tLQ0AACKRCNuUs9ls2H2k41UvX75UV1ffunXrli1b5OXlQ0JCQBCEIKilpSUm JmbRokVz5849dOjQ06dPReemfTCVMzIy6nQ65Z+n0Wg0Gq2pqamxsVFVVTUnJ8fAwAD+O1LEsSEh IQsWLOBz0+joaGlp6UHgpq2trd++feux2O3bt48fP87/WF5e3qnk4j+buMBAY2NjmGkJzZMFBQUI s+mI9+/fP3nyxNHRkcvlrly5UkdHp9Oh297eDgBAbGxsQkJCR/0DgkGDurr6SOKmkpKS06ZNi46O rq+vX79+fX5+fjfctG9eUHzYWhlaro8cONFpp2hvb4ffDSEScDfuVXxWcU1BNrcyt5UgL6esttTB bMaMGW9TMkNuLgAAwMnVA4vF8ejZmNqvS9xmzpttv3b74cjb97f9tI08bsI/xLcdi8WJSElv3boV HBycmZk5TLwrOmo2fX19ExISYOYhuL52g7S0NDMzMx6P9/Xr14KCAmNjYxMTExqN1tjYOEJfYCUl JUVFxby8vObm5n5hCaKo4PkLqijdHhAQIPougqRt0i/KfT4l1dfXf/fuXUhIyLlz55SUlPLy8vpM 6X755ZeJEycuXLjQ1dX1/v37Dg4OEhISra2tUVFRHh4eFy9edHV1BUFQS0sLAAALC4vKykp3d3c8 Hj9hwgQej9fxKgwG4+HhMXHixLKysh9//BHmrzt37gwMDLx//35YWJi8vHxBQUFkZOTAEVNRIDTH wh/hvzQaDXaLLikpUVNTg3t+BBkPBAUFxcbGAgCwdOnSpUuXcjicEydOJCcnk0ikPXv2GBkZ7dix w8DA4Pbt23AxPss5ePBgW1vbihUrli9fzmazjx079v79exwOt2PHDktLy69fv+7atYvH4/FjRNTW 1u7bt6+kpIRCoRw9elRaWlpwes/Pz9fT0xNsWElJyeLFi2trazds2ODh4VFYWOjv719TU6OionLi xIlv377x3eY0NTV37doFb7r40Yt27NiRkpJiYWFx9OhRxDtHEDwez9vbW0NDIyAgAD5z8eLFR48e QRC0Zs2ahQsXwrHnFi9e7O3tDRcoLS0NCAg4c+bM+/fvg4KCWltbLSwsAgICkI4dUKiqqo4kbtrW 1mZvb6+vr5+dnb1+/fq1a9cGBATY2Nh0Whj2ghpBD4PNZndMc/UgMe2o33Z1qzm2E+UBvUlVcipq 1coMDv73j5UHT7hZU8dPt5psYWrM+/YAULdGKRjxpDW5lWlETmPE+f200qr5Pn7aGhQ0STHjzXNN L+/9W1fvPX6xK4YqSElHRI9VVlbu379///79fC4iClv68OGDubk5BEFpaWmSkpJUKlVVVbWgoKDj rmA4Q0ZGhkKhVFdXP336tMdwuX2WQXY/d4gSXVhRUbFXhAmmp4Uhe3W2nAP7ZJdCoVAqKiqysrLu 3bsn6EUUGhraKe3uhtIJclkikciXo4SFhbW1tbW3t/O14W/fvq2rq5OSkoJVseLi4llZWXV1dbKy svw1TOiqadOmpaSk1NbWSklJ4XA4AAD27NmzYcMGaWlpOLZAW1ubrKys6M9u8LeRfNrKf+/odHpS UpKkpGRjY6OkpKS+vv4QOup1j8+fP4eFhb1//769vX3nzp0eHh4hISHfvn2Ljo4uKChwcXHJzs5+ //59U1PTH3/8ISgKXbhwYXx8vIqKipWVlaWl5cOHDwsKCh48eECj0WxtbT9//uzj47N9+/Z58+Zt 3boVvmrdunVz5sy5cOHCjRs3fHx87ty5w6/wzZs3FhYWQm178eLF8+fPGQzG5MmTZ8yYERoa6uHh 4e7uHhAQcPTo0cDAQL7bnIuLy7Jly5YsWfJ///d/4eHh6urq7969O3Xq1OHDh83NzefPn9+x8rGM jRs3iouLHz58GP5Io9Hi4+Pv3r3b3NxsYGDg4OAQGBh4+fLl+/fvP3nyBACA+vr6hQsXnj17lkQi nTp16tKlSyoqKrNmzXr06NHcuXOR/hwLECm+aWpqqr29fUhICJx45vTp09HR0Z2W/P7Uo/1bT4/g crnt7e1MATAYjBVb/T6+iVv80/zJ01CZ+p+SqWn5quWVsnQxFGaS0QT3jb5TbR1Syltd5nvi9Obx GDRO3mMAANDjbFAq5gCzQkMOm/b09x+cLJNevVy3eWvqwyAZMa6aMjn80gkhuWxkZKSxsbGCgsLm zZtHCjHtSFKpVKqysvLBgwd7DPzOXxWampqSk5MLCgrg6LNoNHr4xzrF4/H6+vpkMvnFixepqan9 RUx7i6ysLFFoh5SU1KVLl3pVM0nbRMV1be65LTxOXwyC29ralJWVqVSqYKDBJUuWdGp+0GdKh8fj +cQUhqysLJzLjV+znJyckHBF6CoUCkUmk2Fiyt9ywJcQicRhTkw7BZlMtra2NjIysra21tfX//z5 c1JSUlJSUlZW1nAz71ZRUWlsbPTz80tNTQ0KCiIQCPfu3ZOVlb127drr168BAMjIyABB0NPTU1FR kX/Vx48fx48fr6OjQyQSMzIydHR0nj17tnLlShAENTU1dXV109PTP3z4ABMXOL4Yl8t98OABbOPY 0tICkx4+nj17JpTGDAAANzc3SUlJNTU1IyOjrKysvXv3stnskydP5ufnFxUVCUoBExISPD09QRC8 evXqypUrAQCYNGmSrq4uDoczNDQc8mCfwwoPHjxobGx89+4d3/ZaQ0PjwIEDV65cuXz5MhqNLi8v Fyzf2trq6up68OBBc3NzeG/57NmzkydPslgswaeAAOGmAARBgitxbW1tR0EjH3PsvzeDxffX0Iv7 R6EI/8WRkNu1baCSrsknjcJMrfxm8b9dy0vJdByVqh72AAAgAElEQVSvLQ0z8SVoEdyg4zxt2sL1 O3jN5SiKNYpixS1+xS1NAggyKC0XUFYHYBQtmWtDS771Ouos1VBvk/fSry+vx0TfDb/0r3kTBoMx MjLy8PBQUlLqdNkbnrhx40bH1i5atMje3l5GphfqYA6HU1lZ+fr1a3jrDJOD4Qk0Gq2urq6jo5Oe np6QkDC0gt7k5GQRyYroHKtf6CmRSNywYYPQyeDg4MGhdIcOHeomCseYAjyxwBkZYOMKmKcOPkkl kUjNzf+aeDU0NBAIBCkpqbS0NAMDgzNnzpiamjKZTC6XKyEhAU+/Bw8ehHO7kEj/CXTd3t4u5FDY 1tbG36Sh0eiWlhbYpRUAALgk7KAG10wkEoODgwVXsefPn/NV/3yIiYnBBzgcrr29fc6cOZ8/f7a1 tYVNkgTnLv5v8cHf6iDxboXg4uISHh4+f/58Ly8v+BEkJCQsW7ZMW1vb2dlZQkJCqLsSExPl5OSe Pn0Kkw0rKysCgeDg4KCpqYl07EAD9tQcMdzU0tIyPj7ex8ensLBwx44d3t7eS5d2bg/64crSR88z vrNNj55nDI69KZfLbW1tFRSapn/+8jA8eMrSbUK+WCAP0CvWLMarwVNPC0H+FUfB1twiJPJPTu5D Hqcdre0CkFS5uQ94jGJQahxK2wXEimFaijVU/uYHJKL4oV3rgP++WkZGRv7+/hUVFYWFhUFBQYKa 0OGTkkSoMUuWLIFPKikpHThwIDMzk81mnz171trauldqxLdv38I15+XlvXnzRlpa2tDQkEQiDbfM pWQy2cDAgMlkvn37tvvcDX2G6IlhmUymn5+fiNX2zWHxe+jpnj17dHV1+R8jIiI6avMHaOXW09MT tCYcBIwICoLBYGCSamFhAZPUAcpt1immTZt2584dmBNXV1c/efJk5syZeXl56enpK1eu/PPPP+Xk 5LKzsydNmkQikTw9PRcvXszlcuXl5TtWZWhomJOTAweh2759e0ZGhomJyYcPH2Da+uXLFyMjIyUl pfz8fAAA3r17BxNWQ0NDbW1tT09PR0dHIpHIZ5N1dXVcLldOTk7oV9LT0+F1IScnZ/z48W/evPHz 87OwsMjNzYXvAnabw2Awurq6Hz9+BADg+vXrPQYEQDZL8OQgLi6+bds2AAD++usvDw8PZ2dnWVnZ b9++wUEq4O6tqqpycnKKiopKSkqKiYnJysrS0NBYunQplUrNzs5GkjUONAZzfuhh2IhSCIvFvnz5 Mjw8nEwmKyoqvnr1akSkLRVFaMrfKMPYsOkn620XUGgsAP6HmJp+NWACE/PxegALAgAAhcIktmjZ SlQ/+Fi40tUOU/mRw4VQ6jPACe48ejb3232Umg1K2Ywnp8cte8trYwBi8iAa1z2N2LRp06ZNm/i2 p8O208hk8oEDB0R3hOoer169srOz43K5GRkZUlJSurq6HA6ntLR0OKRQJ5FIFAqlubn57du3HA5n ONjgt7S01NeLmmg0MjKyb7ZZfHraK9tTOp3u5eXl4+MTGBhYWVnp4ODQMXzBqBEpjbgbgUkqvL15 /PgxAABOTk4DbZO6Zs2a7OzsiRMnqqiolJWVHTp0iEqlVldXr1y5UlVVFYIgCQkJExMTdXX1xYsX w3u/KVOmdBr1QlFR8eeff7a1tZWRkZGSkpo4ceKuXbsWL16ckpJSVFS0c+dO2Kxozpw5JiYmUlJS 8NM5f/78unXrdHV1c3Nzt2/fzq8tPj5+5syZnYoqli9fXlhY6O7urqWl5e7u7uTkJCEh4erqevfu 3ejoaNhtDoVCXbhwwdvbW11dvaKi4u7du7m5uQit6RFhYWE2NjYXLlyYM2eOp6dnTk6OlJSUlZVV UlISXIBKpcKGHFgs9saNGw4ODtHR0ZWVlYsWLeJwOMuXL798+bKzs7OQBxuCfkS/LOv9M8eKMsM2 NTXBe0Q+1NXV+fFiOmLK2og+u0O9Ss7uVGh69epVbW1teGGG1ayCwq2UnI/ieHGjCRNxOJyQ3Euo JCwJKy8vZ7FYQspZCIJsnea4/BonSQDlWovp3If5quWkFjHjAt0mvB6NoM9gQQwWp7mN09IG1bbx fGX/wrdx1jroa1CUeBALaCoFCLIoZVMAAHjVGTyoDaViDuJIvBY6t/gVIK5Aq2p6+7lq2YafR9BI 7TQlgSgrd1xc3OTJk1tbW9PS0lJSUszMzFJSUszNzUEQfPfunZWVVXJyso2NTWJiop2dHQqFSkhI cHBwAEFQWVlZTU2tsbGxqqpqqJZ/LBarqqqKQqFoNBqDweBwODA3ZTAYQiUzMjI0NTW7jyE1VKzo O/lTc156+cMrXdFTIf+trKysZcuWhYeHw4Z6MAUR8hxHiOmw6gqYEFhYWPBf8D4ngeseHA6HwWAI WZjU1dXhcDhBxX1jYyMej+8mdR9cVWtrq9BVRCKRb3Dc3t7O5XKFTM7q6uqkpaUFVfBNTU1oNJof HFeIoba0tPB/oq6uTkJCAovFQhAELx/19fXS0tJwXzU0NAyCtH6AnsvQvhSwm4eMjAydTldQUIBP BgUFbdq0Sagkj8erra2Fjcjb29sFzcRHVheNCNBotJEUe7+6uvrMmTPwcUNDw7dv3y5cuNANNx18 tPOgN7g3VYlVVlOsZKVlRHyXhOampJxCqp2LAgnXBnFbSerTWJb4qlQrnulHtYm17TiAxXHSlb+T UeU2kXzjYwUAAGIcjIq20q/Bd84e2AxiCICMNo/N5BbGARKqKBULkFXPLX4FEhVARROU9hwu7cVI HKl9lqyILt7jo7W19frN6yu9VlZXV48fP15XV7eiomKQ40yBIKigoCApKVlaWspgMPpFVspuque2 tQIAwG6obq+rBAAAYjKa8/92fRvvfXh4TqyiS0/Pnz8fHR0dHx8Pk1EjI6OOYXERYjrcAItRk5KS GAzGgMpQ0Wh0R9PnjmckJSVFqUrIDlXoqk6JS8ffEnKnEwQKhRL8Cf61/P4RtKofZDOS0QQcDgc/ LDKZTKVSYVfgThMfgCDIN/Po9Pki6EdR1DDiHqIU0tLSun//Pv9jREREW1tbN+U/XFkKi05fJWeL 3hRbK0MJsb7PjxVytQAAaDfqiMJN2Wx2S0uL0MmHLz/oWc6Yri2TUtzQygY8Fy6tPVOlZKJk2lxY oWCUUs5abqp8J6NqkYnS8y81ZuqkQo61+wTShYBDk5xWXv11N9VQD8QSedJaQBuD+/UeKKsDajoA TeXcgmdobRegrQEAxspElpqaOnv27HXr1q1YsQIAADMzM1GuamhouHnv1t17d7f6bp3rPLe0tFRP T49MJldWVg5OslNJSUlFRcW6urrs7OyuMv30DUXXDzV9ed/xvMyUWb2dO6qqqoYPPYUgaMWKFXp6 erGxsYLkpiMxBURIIjUi0BUxHaGc1draGs7mYGBggKzNCIYEPj4+mzdvplKpSILfoUVpaanouZqH BTcVgr6+/h9//CFibjcRvZrgoFF99oKSJcrYfjKplqgDxEUVfnR8Bhg0+ktx9UF16Rnasm+LGHWt 3LcKzo2MZncdeS8jrfDXGR++5anzaM+/kBeYKG2xVb/0tuxRdrrxFNt1i+xWbj1koE25fHIviSgO EKR5eCleGwPIfYDSngs0l/Fa6IAEBWiqHyNDfNKkSaWlpVevXrW2tt68ebOJiQmPx+sq4/l/2CFV TmuL0SGPQ3Od50IQBBskqaurs1gsOp0+cLs6MTExFRUVFouVm5vbW1NXBoMBp4RRV1cXDGbEB1ZC ZsLWC3Xvn5Tc+RVqrP0P+dOZ3Nu5Izk5WUQTgkePHg00PVVTUwsNDe0xSdVY0LWN3HvEYDAuLi40 Gg1ZmxEMCRYvXrx582YfHx+kK4Ycw2d7IJKffkFBgec/WLhw4fz587sKvD9UMBivZ2NkPZ5JIWDx Il7C6QCTSVTahxcYFFjZ1D5TRzYhr47RBqU1ihtOnJSZkWE1SSoZuimu+lS6JG75JLnrHytWmCqf jkmfOHEiQdnkt7CbbvbmunbLzodEsNkQCIIgQRqQoHAL4wCyEbf8A0hUBLjtI26kwkGU+rbgrVmz Jj8/PzMzc9euXXV1dfyvpk6d2uklBAKhObOBWfC3Ep/NZgcGBlZVVX369Km5uVlDQwM2OepfYLFY FRUVJSUlOp1eUVHxPVFLi4qKOgrj/90+mTtPPHRfSFDK+JSUsWPWdis1azVJtghbl15ZAvVXEkWS tom89bxOPfczMzN7JKYIRgT4Q4sfhBIBgkHjQ1QqVURpF4KBQ1lZ2fBpjEjcVE5Obtk/WL169dOn TxcuXNj9JR+uLO2VQh/o2gtKREhJSNpbzxynqCYK3+r0/BwjSkNuyuOcmltpldGf6EkFDY2tnOom dml1VXhxFIhGZbO+VsjV3ld9RiK0f/nwqKCulUnL0tbWhi+X0JsVGvZ7TQ1dw2rR42cJAACAWCKA EeNVpYMEKQBEAdwxF/+CSCTeunXrwYMHJ0+eDA4O7t4UREZGJvzKNcknf49JCwuLiRMnuru7x8bG lpeXf/r0CYvFamlpSUpK9gsrRaFQcnJyGhoaLBarvLy8+7YBANBjAQAAKisru2GEaLzYeO/DujtC MJJ/B6/R2Xja+OTTQ9FvH0TdKrkVmLFjVsmtQEbW6254Kt9voHswmcx+9B2WNXcm6UyC6SkEQfwk jYgObvThr7/+EjGJBgIEvRJzdPPthQsXuplMkDylYxAi6fSlpKRGWaKwTiNKotFogoTMkXtvsDLK QMa/Vn0Lrn2dBxDWY7ezsRAAAO0oaP+14+9qTa/89gH19YWiwr9+hTw03sJ1/dQZrtd+D94b+Mfd 4EOa6mo8Bo1HUgaaKwBO+9gcZOPHj8/Kynr48KGPj8/atWunTp2KRqNhVwwhKCsrBx48ydjGAAAA g8HMmzdv+vTp4eHhzs7OgYGBIAjicDgNDQ1ZWdmqqqrviYFPIpHk5eWbm5vhRCPdzH08Hg92/u34 FYPBiIuL4yeAFvWntU2oxx6VRp3hE1C8vApeXkXW3BkAgLaacmZBZsmtwKavKbJmjpIGluIahoI5 7sPCwjrGDO+IxsbGdevW9eNDVJnrUx4TnHFyw/b4Anf3+cjUOVrh4uKSlZXFZDKHibsugtFET7sy fel0OUAwyBDFH3HwRosoZlJv3rw5evTo169fyWSyvb39Tz/9BCcv7RFT1kaIbm/afUnBGFJcLre8 vFxdXR2LxXYfLqqrGFKFhYWdMpuXKZ927jsgveYyF+CyIYgH/c1IVLFNLtLZ74zTAAAgtOEmfpoc xLACXvxmJ1v3JT1152afKTPdhQIM8SrT9h6/ZGdueOjn/xHbSgE0nlZU/LaCOLJiSPUvIAg6f/78 sWPH9u3bN2HCBBQKxRdh8mNIwR/hHKETJkxgsVjFxcXl5eVBQUEAAPz4448UCkVCQgI2D6XT6b21 88PhcAoKChwOp6qqCmaccIoXEATZbDabzYYNPGBfKDjQGIfD4XK5PB6PxWLBlbDZ7ISEhNu3b1+6 dEkwYwKfi4u0QeJA3fu/wzy1ITNRkKd+oZUqKSn1KLBMSkqaMGFC/8o1s7KyFs+bvc91iuevd1CY v1PvIBN6/y7SQy7ZghsGBwJDntTQClA6PpeRO6ThAr0d+XypQY+Vf38XKSsrI8lmBxoHDhzw9/cX pWTPOv2YmJhly5YtWLDg9u3bR48era+vd3JyYrFYb9++PXbs2ADdAOwa1RXgSCJpaWkVFRV9G45d pUGyMzXU0NKpfRdd345u5uKZKDH43zeOQjWHCAAAyANMvurfajYC6AUylSlPjiz8mnDt3duk/61e wqD/x1YDVJo03nCKse443Rn/z96VB0K1vv93FuvYZcmSJSXJXilS9xZChLiyt0r7qksLoe1GuRVF t0W30EqJUkl1s1SSbajIVigZ28jYZjm/P97fne/csYQsg/P568yZ97znPe95z3s+53mf5/O4X3tS SOugAC5+ZLyaTiHwePz27duJROKLFy927NjRuz5UZ2fnu3fvyGSyurq6pqbmH3/84ejo6OLiEh4e XltbW1JS0t7ePmnSpL4v8eNwOCkpKWlp6W/fvlVXV8NMvJB0UqlUNstoZ2dnT4v4eXl5GzZsmDZt 2osXL37m/c0kpj059UJjqvLaw1rBjyQXOdPbWipvHK+L2PrmwuHe1/0BAP7+/oNLTGNiYnbu3PlP Zs4CU4uPp7fi0EW2MQ0owV1bW4ugGLmEfGOScPd9gR4S2WHrDZSYDgMOHDjQV7bwwxJ79ux59OgR 06vSyMho27Zt8fHxWVlZMjIyvR/bd/9RtpI/PJCfn19PT6+6uvr169fq6ur9yuTOJCvdWJteZZS/ ThaWKSY3VoNf1gPsf8Kuuah43XcaLylKtQWZvOkRguKCF+7nrV8uHnP2UPmnSluP/SqKcu6eu7h4 /ycWoLZo1fOlq2Mijiguid6z2lRQShkdnRISEjdu3MjLy4MplT09Pfn4+LqmaYFoaGhobGycOHHi 9OnThYSEnjx58vjxY3Nzc29vb3Nz88bGRllZWVFR0bq6OqZRs1sICwsLCQmRSKSe3L1pNBocEj2t 4AMASCTSiRMnDA0NU1JS2DKKQcA03EPRacx1f2UAFi2YP2eRWS/r/hQKZRC/G6FQlISExP8LRVl6 fEk8v0pn4oXsr+hgHttITk4ezrwSKMYDmPS0L3bW4W/buL0vNBqtqqpqSD15+uU3/AO7KVziZBJT CEtLS1dX1/z8/DVr1oxIJ9Lp9JKSko8fP06cOFFXV7e6ujo3N7fvEpi9fK1m5L5kMBjPcuPX/aaH +csRvE8B5P99S83J044pEnkbc3YuX9mLopTgyyEnbz4DHc0IrV1JQT738d/LTOds8Fz78vFNBktE M7eQ9MrfTyfEnH2ZX4HpaGTGytBoNM4PiR1wnP4PoaWlRSQS7ezsVq5c+fbt2971ob59+1ZUVCQi IqKiogKjo/Ly8pYtW5aVlfXly5cvX76IiopOnDgRj8d3NZcSCAR5eXkEQYqKihoaGug9oK2trbW1 tSdmSaFQzpw5ExIScu7cOW9v716IaR8X9H8GgUeONktOY7OnssZRFRQUdM0VPjCQSCQ9PT1XV9dT p04xFUxlLD0AAGt1J6Iv2rENBwcHmOAUBYrBpYC9GFDhvI36Cw0z3r9/z1G56H8wAhAEUVJSYpO+ u379+qtXr5iZooYHXXOWksnk0tLSSZMmycrKfv/+/ePHj2JiYlOnTmVLW9rV31RcXDw/P79bM1vk zb+j/7pa0PABANBQ13D+0r0XL95+ykgWUp0pSiVhROTUZ2qsdDVXm6EKACh+VxSw40BWzBHQVArE VOGTRqXSjp46f+56sv3SxbeTUrs1sEH6QqPRzM3NAQDW1taWlpacGXkwDK5OFAolKCjo3LlzwcHB ysrKTH/TbjcEBARkZGQoFEpdXV1VVZWvr6+4uLivr++ECRMEBATExcVbW1vJZDJsMA6Hk5CQYDAY paWlvYtDsVJSBEHgWj/cbm1tTUhIuHbtWnBw8A8d9oeBmMIeS01NNTMzY9sP/VO/vElZG3TpVtCe rvbU/iI9PX3jxo3x8fFdBycGg1mrOzEgIADyVBT9faw43N+U2UgikdgX/+Ze0NTUFBUVxbqHi4vr JwP1Hjx4cOLECR8fHxMTk+HvKObZjx8/fvv27V6yTA3KCBlL/qY/3NnHYkPxnhrnhDg9PX2oI9L6 dZt+fDPs7e3Nzc2ZJlIymbxgwYLQ0NBhljhli4X68OGDsrKysLBwVVXV169f1dTUxMXF6+rqioqK 1NTUZGVl8Xh8L7FQHz9+7NZCtt1rZ2bGq+Scp3IK//mAaKirF5vwH0NU1acqj9/WVFZ8bn1/H2mr B/QOjMD/PBxqSXXb/f4kNbYE/hEiozCF7SxKSkrwJpFIJE1NTejmIi0t7eDg4ODgwJrhejxwU4iK igpvb28SibRjxw4JCYmeuCncEBYWlpaWrq2tbWlpefHixZ49e1xcXDZv3szFxSUqKiogINDY2MjH x8fDw/Pp06eWlpbev766iu1DJ1T4uAYFBW3bts3Ozq6nm4LD4eh0+vCwUta3o46OzsSJ3Vgut23b duTIEXwbuWscVb94amBgYGpq6t27d7vNFAIHRnXCX+BfM+pPjrGxYQ0ae9wUABATE/MzK/s1NTWJ iYkAgKNHjy5fvlxZWVlISMjBweFnmjplypRHjx4N80PX9ey1tbV91HRDuWlPBVivrpcyY4abFhcX P378mPmTj4/Pzc1t8eLFz549O336tJSUVFed14yMjDt37gQHB3etzd3d/cCBA5MnTx5ASyIiIvj5 +WHuRvhOGWqx6kHmptXV1YsXLxYXF9fS0mpsbExOTt66devevXuHeeiz2U1pNFp5eTkXF5eSkhIe jy8tLW1tbdXR0eHh4fn48WNDQwNMgNaT3TQnJ6dbu+kfJ46l/5PGy8e7zNHOcpml9iwdNkZCo9Fy 3+QkxiXGXYs9ceWkh/VqWlkygiCg+TPgn4Dh+s9bPO1V1ha/MIPZWlt8DvMRhLo1sD148GDJkiVs zXBycoqJiRlX3PT/eywtbePGjbNnz3Z3d+fl5e3FgIrD4cTFxUVERGpqaigUyrVr186cObN//35b W1s8Hg/NPJ8/f+7F04M17r4ramtrfXx8VFVVAwICul3B7/ZuDiecnZ1PnDjBRk/DwsIWLlzIln+y 23j/XngqhUKxsbExMjLqJaCSOTC+JJ7/GXo6ZmwV/bqQ0cVN4brZzy/siImJlZWViYiIZGVlpaSk 5OTkuLq6TpkyJSAggEQiTZs2LTg4mJeXd+fOnfr6+jExMVgsNjQ0VE5O7uLFi/fv329tbV22bNm6 detWrVoVFRVla2t77dq1sLCw1NRUDAazdetWIyMj1pqfPn06Y8aMW7duTZ06dcuWLX5+fq2treHh 4ayPTF5eXkBAQGNjo42NzbZt2xgMRmhoKFuFT548+fbtG5FItLW13bRpE/PsPj4+KSkpu3fvjo6O vnnzprCwsJWVVUNDw6Bot40fbpqRkSEpKTllyhTwEwmBRxE3jYqKOnPmDNPYx8vLa29vLycnV1dX 19LSgsVi+fnZk1s+fPjwwoULt2/f7lrbnDlzLly4MGPGjF7OePz4cS0tra4rDP7+/gICAl5eXnDO LysrgxGQHEInfmyfk5WVzc3NjY+PLy4u1tTUPHDgAJv76YjYJxobG9XU1CgUSm5uroyMjJqaWnt7 e05OjqCgoIaGBo1Gy8/Pr6iomDFjRlfJLiwWKygoyM3N3U134HCdHZ3pn169fvbq77/+Xmm3ouV7 i9wkOaXJyuWlZVWfqwSFBPX0Zy5f47g1cAfTWReDwSCCsqCxBBGbisH8z4V33pyZmfcv3IpPWvTr /AO/b11k7YrnYj+phYXF1q1bT58+zbrT1dV11JmCBgXz5s3Lzs6+du2ag4ODt7f33Llzubi4empY fX19c3OzhISEsLDw6tWrHRwc9uzZExERcfr0aSMjI15e3gkTJnz//r2+vr7bw3sKw+/o6Dh16lRx cfGJEyd+ONQHi5VSKJT+JjKOiYkJCwubNm0aFB8gk8lXr161t7fvmhi9X/qpFRUV1tbWZ8+e7eP6 DtQ9/ZJ4fgD0dHwS01EHRUXFmJiYn+SmZWVloqKiIiIiAIDU1NTbt2/fuXNHRkZm586dGzZsMDQ0 XLlyZUxMzOrVqyMjIydPnnz37t2NGzdevnzZ2tr64sWLqampVCr1999/p1Aozs7OZDL55s2bAQEB 3759u3HjxpcvX+bMmVNaWspas5ubW1hY2P379zU0NLBY7OXLl/ft2xcVFbV7927YpIaGBgcHh3v3 7ikoKMyfP9/CwiImJoatwrS0tOvXrycmJmKxWGVl5Y0bNzLP/ueff3Z0dPzzzz+hoaEpKSlNTU16 enr+/v4ARX+Qm5v7/fv3cbWYPnnyZFZVbKaJJDY2dsKECTIyMikpKd++fXv79u3s2bNZo1qrqqr8 /f1PnjwpICDA3Pn8+fPdu3fj8fg//vhDXV29vLzcz8+vrq5ORkYmKCiouLg4NDRUVlZWQEBASUnJ 29v706dPhoaGMH9KXV2dq6trdXW1kZERM6MKh6BPa8d4PN7Ozo6j2k2j0d6+fauioqKvr19XV/fy 5cspU6YYGBjU1NQ8f/5cXV19zpw5ZDL5+fPnKioq06ZNY2WiDAaDTCZ3a1GDETl4PH7B4l8WmRtz YfB4LJ4Li896maVvoE9j0KkIjcag0Rg0uPG/lxMWjxCkwfdqIPSfxFRcXHhne6ulZr/uP3Lq7IXL N+Lu8/ILshGaEydOPH/+PD8/H/40Nzcfz3kg8Xi8m5vbsmXLvL29IyMjfXx8pk2b1ssw+PbtGx8f n4yMzPfv30NDQysrK+fNmwdvbllZmYKCAluSm15YKY1GS0lJiYiIOHDgQLcLKBCDvog/4I/+zZs3 02i0x48fNzc3Kysr+/r6/tAbpHee+vDthyN/hj558qRf/oUDo6coMR1F0NDQGMDnEyvevn2rq6sL t7Ozs7dt2wYDL1auXPnq1av09PTS0lIzM7OysjIJCYlNmzYBAISEhMTExGRkZOrr6w8fPuzi4gK/ 4XNycnR1dWk0WkRExIcPH3A4nLy8vIiISGVlJbPmsrIyKSkpFxcX+GRt2bKFm5sbQRBWRZeYmJil S5eqqqoCADIzM3uq8Pfff5eTk2ttbRUREcFgMPDs8IqcnJwuXry4efNmAoFAIBBkZGR0dHRQutkv KCsre3p63r59W0pKCu45evToUBvwOBMwGomfn//YsWNpaWmHDx+ePXs2k3o1Njb+9ttvp06dYiWm AIDCwsL79+/Hx8evXr369evXFy9etLOzs7Gx8ff3P3r06PHjx2fNmrV27dq5c+daWFi4uro6OTmt XLkSuoCnpaU9ePCgublZTU3N398fi8VyTqjCvkoAACAASURBVG9gR9GdYyYrx2AwcnJyWlpaJBIp JydHWFh4zpw5dXV1GRkZgoKCv/zyC4VCSU5OhjxPQEDg/v37X758YX2XCAoKCncHXA9a6HMM5vRi WUSoFITyDbR8BVQK0tHM8hcD6WhGWmoIbeV/+m5Qn6HVSG7plo09efIEbktKSgoICGzbtq33uPUx DwKBEBYWdu/evYsXLx49erR3n1Goz0+j0WRlZeGbhhUMFsB4fEZ3yMvL++2339ra2l68eLF48eKe WCkAYPi9S3un8hYWFoaGhlCmoF/Hsuqniv/iEHnuLO717SumUu1P//6hfmpXegoAgOv7KDEdk9w0 Nzf3Z2rIzs5m5aYLFiwAAISHh+/cuXPy5Mnm5uYkEklHRyc7O5sZyZCTk6OjoyMuLg71Cu3t7cPD w5n7q6urhYSEYBYYGo1WW1srIyPDrDk7Oxsa/ltbW+GXGzxQS0uL2SQmy4ToqULYntzcXHgsPDs8 BWzwzJkzAQCdnZ2fPn1irR9FXzBt2rTIyMjg4GCvfyEvLz+2LzkhIWHav7C3t+86pWhra6uqqnJz c6urq0Pq0tbWZmVlFRgYOHv2bLbyLi4uWCzWxsYmLy+PwWDs27ePSqUGBweXlpbCrIdMovLs2TNH R0cMBvP333+vWLECAGBhYSEkJCQnJ8fDw/P9+3eO6qVRw021tbXd3Nxg7/9rkuRSVVVVVVUtKCgo KytTU1PT0dHJy8sjEokKCgpGRkZlZWXp6eni4uJLlixhXRrGYDBNTU0N3YFG7z8jbCgCHWTALQDE VYHYVPC9GqFSkLY6pOEjaCwFtDbAK4KR0MDwiQEcL+hhFVhCQuL+/fsAgLi4uJs3b+rr68vLyxOJ RE549Y5gqIqiomJKSsq2bdvs7e2joqLodHovhSkUSnV1NQ8PD9uXA5tQVFdWSiKRfH19nzx5cv/+ fU9Pz14YHmSlnENMBwskEsnadY2kgaV9+EOt4Mfd6lINIj1FienoQnp6ekxMTFBQkLOzMwaDYZNt 6S83bWtrq6+vh/wjISFh165dJiYm7e3tX79+VVVVhYQPHpKXl6epqZmYmCggILB27drz58/fuXMH 0kRdXV1JScmmpia4BnLz5k0TExMsFsusmXk6WAmssKCggDVZhpCQUFNTEwDgzZs3Bw4c6LbCuro6 WOHbt29hw+DZKRRKU1OTjIwMDoeDRpOIiAh5eXleXl6UbvZ3kp8xY0ZxcTGFQpk3b56ysjJ0/BjD sLKy+vAvuvUiZb7FmDNMamqquLj4o0ePuhaG4RDwTU2j0ZYsWfL+/fv58+fPnDmTdXaCyQ7ZLKNw PbmiogKuKnBUL+FHy+3U0tIqLS09duzYunXr/P39mc5PBAJh5syZJBIpIyNDVVXVwMCgoaHhxYsX ioqKenp6ZDI5PT1dXl5eTU2N9Y0ymO8n8f+sOCPCk0BbPUAQwCuE4ZdiKywrK9tTPRYWFmlpafBb 39nZ2cTExNnZ2draev369ZwTtj8imDdvXlVV1dmzZ83Nzfft29eTSj/41xG5srKSjUDAja5eHB0d HdevX8/MzAwJCflhqOPQUdKRnRSgUFRUVBRzHa1f/qls9LTswr7eF/e75XMUCiUyMpI5L2toaMyZ MwclphyCM2fOXLt2jflzYCKITCNlfn4+M3TDwcHBy8srKipqwYIFIiIiUVFROTk50KPp06dPoqKi BAIhMzPzxIkTU6dOLSwsPHDgALSDwnimvXv3Ghsby8vLNzU1XblyhbXmnJwcKysrVpL66dMnaWlp 1m/XTZs22dvbJycnf/78OTIyko+Pr5cKs7OzFy9ezDx7eno6pLn79u1zd3dXU1Pj5eWFBlQU/UJJ SYmVlZWhoSEej9fT01uyZElqauqYp6f9hamp6bVr1+bPn5+YmGhpacn6V25urp6eXnFx8cSJE3E4 3MuXL588eYLFYq9evQpXXyFnxePxqqqqb9++1dPTu3r1KuvcVV1dzVGr+f8/wY666RWKDXV2djo6 OoqLizOj7xEEqaysrKys1NXVFRMT+/z588ePHzU1NSdNmlRdXd3Q0DBr1ixYUlJS8uXLl936m544 HfI8+VlOXT4ei8dj8Ux/UzwGjwCkq7+p7gTNK0Fbne2tumEbDR+AyGQM7n9+rq4+EUePHe9XSEFY WNj58+e7FZgcthcw4JgQUQqFsmbNmsLCwmPHjkEl1J50piZMmMDLy5uTkyMnJ/fx48euIfkMBiM7 O/vPP//09fU1MTHpnf2PPUMp6wCLj4/vSSiKncqzxPtHZbx/WUl+VljBxlMROu3j6a0CU3S6pac9 8bmqqipNTc2QkBAAQHt7+/Xr1zU1NdkCBMcSMR11cfpKSkpwp7S09NevXwf3ocZgMPz8/HQ6HT68 Xcu0tra2tbWJiYl1NSvA3MI/ozDa0NAgJiY2sAo7OzuLiorU1dUxGMy6desWLVrk6Og46ANsbMfp Hz16VF9fX1lZ+Y8//oiIiDh16tTkyZPZ6NcYi9N/+PAhq+Jve3s7jNP38fGRk5ObMWPG8ePHoewa TBfPy8sL4/QrKiqMjY2fPn06adIkeOzs2bN1dHQ6OzszMzN9fX0dHR2dnJxIJJKgoKCVldX+/fvD w8MLCwujoqKOHz8uLCy8efNmBQWFr1+/xsXFnT9/Hsbpp6enL1++vKCgYKg/CQZZQ4ozkZqa+ttv vzk6Oi5evJiPj48pZdre3v7x40cqlaqtrc3LywtjAOfMmSMmJsZksYPLTe3NDIpKPv39p4+m+n8N qPRO0FQKxKYx59MBcFPwb9y0t7f3iCQP/MkAiKFAeXm5tbW1qqqqr69vTzpTEhISPDw8OTk5kpKS OTk5bDVUVlYGBASsWLHC3d29W30oGO00tlkpTP1gZGS0d+/eARjmZQR5tKQFzniv72pP7Yme9jL1 V1VVzZkzp6qqCv5sbGyUl5eHTsZ37ty5fPkyDodzcXGxs7Nra2vbs2dPUVERPz+/j4+PhobG+vXr 58yZExcXp6KiEhQUJCAgkJ2d/ccff7S1tWlpafn6+uJwOE9PT0NDw5iYGElJybCwMDExsdDQ0KSk JACAi4uLi4sLnU4PCgrKyMgQEBDYu3evhobGkydPQkNDOzo6Zs+e7efnx9pFvb/D+vLIjC5uCgBw dnaGplPOkbfjBLS0tOjr6y9cuLClpYVEIsXHx0OXdJSb9r3AwYMH586dq6KiArmpn5/f/PnzjY2N xyo3HQq0t7djsVhmwHdDQ4OgoCAXFxeNRoOcp7GxEUbyAQCamprYOCiRSBye4LN+3SbsKJ0UjIyM qqqqZs6cuWrVKhhiCffz8/Pr6OhMnTo1NTW1pKREV1fX0NDw1atXBQUFQ9SSY2eunv3rovuOP1w2 7m+htP7vNuC4Ab8EoHz7yfoVFRXfvn1bUlJiYmLCFnI+DOA0YgoAUFJSys/PX7FihampaVxcXL+C xshkcnBwcEpKSmxsrKenZ1diyhrtNGzEdPideisqKuTl5f39/dlYV9/xtaXzYUlDt3lTm9+9UvY8 1vIxh9X3tF/z/rNnz2DnQ/nJq1evXr169ejRowUFBZcvX6bRaElJSUFBQfHx8dAOQSAQHj16hMfj Dx061NbWZmtrGxgYmJCQUF9fHxISgsPhoqKi+Pn5nzx5IiwsfPHixffv31+6dCk+Pv7WrVsZGRnt 7e3h4eHFxcXx8fEHDx60s7OjUqnu7u7nz59/8OABgUB49+5dLxdCIpHS09PDwsJMTEwwGAxs1RjD kSNH4AabQWucQ0BAICcnZ/Xq1fv27UtMTBwsYjqu4OLismnTpkOHDuXk5Hh4eCQkJMybNw/tln6B l5eXVYlITEwMBtjADEQAAFFRUeYrpqtxtLm5mQMvCjt67wcej3d1dS0tLa2srNy2bVtJSQnrvTEx MYFvrPr6elNT0150iH4eMgpT4h+9WGJjr/qLa9iFaCr1X7bEKw6oLQit/eev1M/PLyQkRFNTE01v zXxHVlVVtbe3//LLL69evfpheSqVmpKS8scff+zYsSMoKKjbjEpg7EY7seLBgwdz587Nz88frAx1 rPH+kKd+jvmD9r2h/mVi87vXfSSmtbW1M2fOnDlzpry8/NmzZ2/cuAEAuHfvnqysbFxc3K1bt2Rk ZB4+fKipqZmUlHT8+PGOjo5Dhw4BALi4uFxcXDAYjIODw8uXL/Py8pSUlODz7uLi8vTpU3h2mG1F V1f3y5cvMjIyzc3N+/fvz87ODg0N5eXlvXPnjpiY2JUrV9LS0iAn1tbW3r59e0JCwubNm5kBNN1e SHFx8caNG7ds2QLVNoqKiob/G3Kooaio6OTkBABQUFBAJx9WcHNz6+jojLjm92gEDGtWUlJ6/vz5 tGnTTExM5s6dm56ejsaToRjd3BSCQCCEhoYmJCRcvXr1+PHjTB0ELBYrLy+/ePHi+vr6x48f9y5C NCgwNFn2Ij3za1OnooHDg8fP4JsMCCsCcgWCMH6+fg0NjcrKykePHjk7O1MoFHTs4vF4f39/IpGY kJBgbW3dkxsc1IfasGHDlClTYmNjWQN1WaH8L8Zwj9FotMDAwKioqMrKyp/JkN4XnqrqHckrMxnH 11e7u6SkZFZWVlZWlpeXl4SEBAxeZDAYBAKBl5eXl5fX2dnZ1NTU0NAwJSUFh8OtWrUK6lczbQPc 3NxUKrWjo4NpCYZOPvAvaDaA5FJYWDgnJ2f69OknT57U09OjUCgMBkNQUBCeKDAwUE5OLiEhwdnZ +eHDh+rq6nl5eb09+IaGeXl55eXlkL2tWrUKAEAkEh88eDCWntPz588DAKZOnYrOPCgGBT4+PhkZ GSdOnCgtLfXy8jp06NDq1au7ZkVCMdToJUQb5aaD8FmfkpKyY8cOV1fXGzduMNd58Xi8jo6OkZER Uxt16KCsrDxVddofZ6Lz84nRCak6pivLP1VisHhAkP75lX3m5Zw6dcrV1dXAwCA9PX0YOnZkNaT6 AgkJievXr0dHR3t6eh47dozNjTgnJ2fv3r2fP39OSUkxMzPrdv16zFNSCBKJZG5uLiYmFhMTM6TK Dwid1pD5sOjYqomL3QlKMwAACIL0fRRt3ry5pKTk7t27AAAdHZ22tjZHR0dHR0cREREpKannz58j CLJjx46MjIzY2FgAQEtLS2lpKQAgPz9/2rRpampqRUVFkJJmZWVpa2t3PUVJSUlubu6KFStiY2PF xcULCwt1dHQEBAQcHR2XL1/OYDD4+fljY2MtLS3Dw8PXrl3LlG7p5UJg8qTa2loJCQkJCQkNDQ0L Cwu43D8w0SWOAgaDgYrfkpKSGBRDhnFFiebNm2dhYREYGGhtbS3NgoyMDJQvDhs49vsZP8bGOhQb sra23rNnzy+//AL3EwiEPsZd0gekeM/GbCRlFK4nvsh7/XyZm9v0ybLngvcS6E1I56CNAAsLiwUL Fnh4eNy8efPEiRPjXGEKQlNTMzs7+/r16wsWLDh48KCzszMPD09jY+PevXvPnTvX0wo+h1DSYXDA JxKJrq6urEJRQ4SWktzSc97yv+1Q2xeFYclkAVldX64Uh8P99ddfVlZW8+fPt7Kyunv3rrGxsaCg YFtb2+3bt2GOuhkzZlRXV2/fvh0AICoqGhwc3NLSkp2dHRsbKykp6ePjs3DhQiUlperqalbxIyaE hIRWrFghKytLo9EEBQW1tbUVFBSWL1/+6tWrurq6WbNmOTs7371799KlSxMmTCgrK2ON/un9Qtis 0YqKioqKihUVFQ8ePDA1NR3VjyqCIFCJBp1thu4DYFxdr6WlZX19/V9//aWpqcmas6Db+FQUQ2ez 4MCoEjB64/R/+CkAxYaOHDkyefJk7H/RbZw+lUqNj49PfZlaVlzWrzj93jswKiLIa//hPeuWelpq rTqZcTQoZBDVoGJiYnbt2vXy5cuhU5gaFSGibLc+ODg4IiIiNja2qanp8uXLx44d40xKOmyAQlEx MTGDuI7fdWBQvzdW3jgOAFBw24/j4evpqAGMpY6Ojs7OTubnJYPBqK+vFxYW5ubmZsqvtLS0EAgE 5tudTqdTKBQhIaFeqm1oaODm5mZNANjc3MzDw8OUwKRQKB0dHawCQwO+EJhadtasWfAWjLo4/fEj 5jqy/d9T54OxGKf/+++/29jYZGRkzJkzp/f4JzROf+hQUVExbAqV40JDqi9gig15eXmJior2xE0Z DAaRSLx+/fqGDRvSMzNi/o7uIzf9XPFpjdXKmqqa3ptBpXYe9t4Q8fctyQki9x69GNxxQCKRUIn+ bp83Hx+f58+fGxkZsXLT8cZKaTSau7v7tGnTBiYU1cdZBqHTSC9i61LvKK05xCerMpwvACY35cCX cVdAPxxDQ0OUm6JAuem9e/fc3d0ZDAYXFxdr4sa4uDgDAwOUm45zboodw53OFBuytra+ePFit2JD lZWV0B/x+vXrZmZm61Z7cHFx5b7O+WHlxe+KV5q5Pbj346h5Li5u/5CL+UTigjm6/NyDvGojISGR nJwMANDT0xt70cEDhqKi4vXr12/fvu3i4qLMAo6d1odiOY9EIsnLy7u6ug5YKKovaCnJzfdZgicI q+2L+iExBf30Pf0heHh4YLLf4ccALsTQ0HDq1KmckIt4OFFcXAznKPixdOHChW5nKgqFsnz58i1b tpw+fRqqNAwK3N3doTvyAJCRkbF79+6frARFT1i6dGl9fX1QUNC9e/dKWKCvr492Doqxb2mDYkNn z5795Zdfjh8/zlw7IJPJt27dIpPJrP6I8nLy6enp23ZuO4eNWLl5lcE8AyHB/ywLtlJaM1LTI89c JNeRM19lMtOl/BCSMgqhf98domvcvHmzpaWlsbHxSEn0cybGs05eenq6vb39kPp7TBTgLruwj0tQ dMahuz0t4vfC6gbFRIHBYEbwTTaACxkieQRORmZm5sOHD01MTBgMxooVK6ZMmdJtJ2RkZDQ1Nd24 ccPHx2cQMygWFxd3m2alL5g5c+b06dN/shIUvQCHwy1ZsqS0tDQrK4u5U0tLS1RUFO2c4QFT2ojT gB0PvY/H47du3UokEhMTE62srAoKCpKSknbt2rVy5cqLFy+yBcqIi4tH/R11LjSisuCzsc5CRcFJ jpbL16/w3LBig5yAzK9a8z8Ry8NPhRcQC/pOTIcBQyTRPw6jRzkcFAolPT09PT2dTCZ3e6OhUJS/ v39lZeUQEVOETlurOzFk8WRZm03yy736RUxZWd0YuB0DuBDIzMZA8H5/O2rt2rWKior+/v4AgPb2 9p07d1pYWNja2ubm5jY1Nfn5+RUWFnp7ezMPefLkCSz89OlTW1tbMzOzAwcOsH0JhIaGWlhYWFhY REdHAwB27NgBs4uVl5czq3r+/Lm5ubmVlVVhYSGs1tra2szMzM/PDy6mXb16deHChUuXLs3MzAQA 7N69OzIy0tzcvKCgICIiottKiETi8uXLly5deuTIEQaDQaPRVq9efeTIkT179qBzVL+QnZ198l9s 27Zt69attbW1o6XxY0AaQlNTk0MFKJBxhtzc3F9//fXs2bNFRUWl/6L3Q1paWsrLy1taWkbFBebn 52tqat6/f39Qahufg4RjER0dHRsbW15eXltb++XLl7S0tICAgKamJtaxamxsHBoaOnRtaMpPLQxY bqYi9vMDY8wMrf5eCAAgLS2ttraW066C9UJ62u4vrl696uLismHDhk2bNjF3xsXF+fr6Igjy7Nkz PT09BEESEhKWL1+OIIi3t3doaGhWVpa+vn5dXR2CIBYWFp8/f6bRaIsWLUpISGBW8u7dO21t7c7O zpaWlo0bN7a1tenp6b1//x5BkJycHENDQwRB9PX1169fT6fT4+LiZs+ejSDIxIkTv337RqfT//jj j7y8vFevXunp6X3//r2goGD69Ol0On3+/Pmenp41NTVJSUl2dnZdK2ltbZ06dWpBQQGDwfD09AwL C2MwGHx8fGFhYWQyeVBGEds25z8pP2xhHy9hy5YtZWVlA6h8RKajMTCDRUdHD9vY6NdtGnfRM1pa Wk+fPu3XIQQCgTNFFrqFhobG27dvd+3aFRUVdf78+VHUchS948GDBzo6OlCUHmLixIn6+vrMROfv 3793dXU9e/bsYCV8YkNH3ZdPVw/xySirekc+PNAPieyeFr7HjPUU9D+QwtDQcAwIS/UR9+7dW7p0 aVFREYVCgTOSjY2NkJBQSEgImUz+9OkTW/mSkpLw8PDk5GRxcXEAwKVLlxITE0kkUnt7O2thZn4v S0vL0NDQnjwBXFxcsFisjY2Nk5MTg8GAGb+WL1++efNmAoFw6NAhKysrAQEBdXV1aBPFYDCOjo5S UlI9VQKzZL958+bNmzdCQkJJSUmbNm3q7Oxcv349mrb05xcAc3JyOGpNcmyDLeyMc4BF783YA6tE /3gLvBiroNFobW1trMSUea/d3Ny0tbUdHR23bdv25MmToSCmCJ32JfF86dmdCm77B7CI3wtpG0tW 7f72qqmp6evXr8fD6LWwsIiKirK1tXV2doaZKg8ePBgRETFr1iwTE5OuXZeUlEQgELKzswEA9fX1 BgYGvLy8xsbGSkpKrIW75vdi/gXPAgH1MuHaIo1GY8v41dnZyRokDsGqLNa1EiqVysfHB7OI6erq Qp1dXl5elJgOAImJiY7/wtraOigoSE9PD+0WFCg3Hcvvg4yMjKNHj27bto02oJwC4L+meBRDSj17 L9DY2CgtLd3TsUePHpWUlKyrqxuKUJuGzIfvD7sSFNSm+13nmSCD3qxB/IYUEhIaD/Ia0Da8d+9e fn7+Xbt2AQCePn26detWIyOjmpqaroN/y5Ytt27d2r59++fPn4lEoqKioouLi6amZmFhIWvhrvm9 CAQCFBRjja3Jzc0FABQXF0+cOJFGo7Fl/NLR0Xn58iUAoLOzc9myZVQqtdtLYK1ES0uLTCZbW1s7 OjpOmzZtwoQJ6GAeMKZPn+76LzZt2pSbm6ugoIB2yzCgoqKioqKivr4ebgyYJAzVpIHeoTEMAoEQ ExMTExMjLy8/pCHbKH4GfVF9S05O7nbxhUQiMfUZaDSalpZW78nf+4WOui+lZ3eK6C5iS/KEYrCg oaHx4MEDCwuLcXK9ly5dMjIyOnv2rIODw6ZNm6ZNm2ZgYCArK+vv7z9z5kzWkgoKCsePH3d0dExM TKypqXFwcKDT6W5ubufOnTMzM5s2bRroLr+Xm5vbmjVrdHV1J06cCB8oBoORmZmZlpaWmZl55MgR fn5+toxfkyZNunPnjrGxcXNzs4ODQ1cbatdKJCQk9u/fP3/+/ClTpnz8+PHvv/9GR/LP2D6MjY1x ONzp06e/ffumoqLCyQ3u6oOExgr3dGd/tqtRq9h4ACrRP9q5KYVC4eHhYbt36enpGzdujI+PZ351 tLe3e3t7nzp16iebRO9o+3L3TNuXMgW3/d3aSkddwjBOu+PMrqNQKGVlZUOdS3YA43CotfdbWlow GAyBQEAQhE6n9zIvIQhSX18vLi6OwWA6Ozu5ublZ/2XL70WhUBAEYVuUb29vx2KxzAO7Zvxqa2vD 4XBsNbOBrRIGg9HU1NRt2rCfHxXjQXsfAGBnZ5eVlUUgEGxtbevq6qSlpW/dulVQUMDmOsw52vtj SZ8/LCxsy5YtcNvY2JgpQjzog4H5E80LhaLHsXj+/PknT570feUXpSAcwk27PvCBgYGpqal3795l C3d79+5dRUXFz5jiGjIf1jy8LGu7WVhjHjowhuFFnp6ePkTha5zMTVGMc25KIpGWLFny+vXr9PR0 Dw+P9+/fAwCcnJz27ds3Y8YMlJsONUgkkqSkJNy+f//+IK7eDAo35XR/0+Li4rB/ceXKlZKSkrE0 Ge3YsWM4ZQ43b94cHx9vbGwcExODvgk4B31x6n3w4EF5eTnT6mNiYgIASE5O7qrDAHVwBubF2FH3 5V2gY3ttpdq+qF6IKYrBhba2NkxnigLF+AGJRFJTU8NgMLNnz4aCDAAAKSmp+vp6tHOGARISEpqa mnB7wYIFnNY8TuemmZmZMTExAgICAgICdXV1ZmZmt27dGhsjg8FgPH78eJh9QKFE/+vXrwdXoh/F UMPQ0JCHhwcAUFFRYWBg4O/v7+fn11PhhQsXHj9+vF/10zvayi7sq757ZsqOcBlLD9S7dDhBIBC6 6iihQDG2wWAwoCGNbQUftdAPGzw8PAAAxsbGnKg1yeGyLFC3mVWxecmSJQiC1NXVbdiwwdLScv36 9Y2NjY2Nja6urrBMe3u7g4NDZ2dnXFzc0qVLbW1tb9++jSBIdnb2sWPHHB0db926lZycvHTp0sWL F/v6+lKpVFgza2EmMjMzYai7qalpcHAwWz3fvn1bt27dkiVLVq1aVV1dTSaT3dzcoMNTSEjIjRs3 XFxcGAwGgiCdnZ3Lly/v6Ohg1vzq1Ss3NzcOl+gfLRkHxjxaWlpCQ0Ojo6M1NTX7otkeHR1dXl7e l5oZNGr966RcL9PvH3PQpAwjJVTe0tKSlpY2JrX3B4CCgoLQ0NA7d+7An3Q6PTw8PDQ0lDXNxDDj 4sWLjx8/Zns1aGlpFRYWDsWoAKNEex+wLPv0d8wTiUR+fn5ZWVlZWVlubm64ISAg8OzZM47V3h9j kx5MwTVYmXp66iUwIO390cRNGQzGtm3btmzZgiCInZ3dpUuXEASJiYmxt7dHEGTOnDlZWVmQZdrZ 2eXm5kKlj5aWFj09PSKR+PTpUwkJibS0tNbWVrbUIF0LMxuQlJQkKCj44cOHtra2mTNnJiUlsdZj ZWUFm3H9+nVTU1MEQU6cOOHl5ZWTk6Ovr9/Z2fnrr78+f/4cQZDExESYYoSJwMDAK1eujGDfUqlU JycnJycnlIByFCfo6Wbp6+s7OTnBT6m+wMnJ6YdlWqs+FgYs//b0OoNGHUCbUQyiUWBccdNv3771 8q+XlxcOh9PQ0IA/Y2JicDgcFxdXZYa1IwAAIABJREFUe3v7iPRMU1MTBoPx8/Nj3RkSEoLFYqEl Ynxy036NkK4FqFRq5b/4/PlzTk4O3O56l1FuOnTQ1NQcdAIwXripmJiYnp6enp6eiIiIp6cnmUym 0+lcXFx//fVXZGTkhQsXBAQEEAQ5e/bs7t27EQSxt7dPSEgIDAy0sLCIjIyMjIy0srIKDg5+9uzZ ggULYLXm5uZOTk53796Fd6VrYVZuCtWhEQQ5ePCgn58fsx4GgyEgIAC5Ap1OFxAQYDAYDAbD1NRU R0fnw4cPcFZdsWIFgiCurq4PHjxgvbT58+d//fp1xHv4/v37mpqa+fn5KEfkWG5aW1trbGx87Nix fmW5LC8v78V0SmtvLT2/t/T83s7mhgG0OSAgAOWXA4ampmbXLs3Pzx/Zr8Th5KYrV65ctGhRampq t/8uXLhQW1tbREQEQRAajTZt2jRNTU0tLS0EQcrKyhwdHefOnbtz507YXffv37exsTEwMFi7dm19 fX23e169emVnZ6evr79q1arGxkYEQV6/fm1tbb148eKkpCRLS8vU1FQGg3HmzJl58+ZZWFgkJyez tufZs2cAgLt373Z2dh44cMDQ0HD79u1Q3HQYqN5Y5aas+P79O0xdOzBGOHTcFJ2sBvCxPSjcdBRo 75ubm2dlZWVlZf3yyy86OjpCQkIIgmAwGEFBQV5eXgKBcP78eQaD4ejoeO/ePTKZnJWVZWZmxmAw CAQCTN3h7OxsamoKWLJ9sKUG6bYwE9DPDwDAzc0NlZlhPQwGg0ajwVwgWCyWTqfTaDQMBiMhIUGh UISEhAAAy5Yte/78eV1d3du3bxcvXsys8/v37xQKpSc19eEElOjfuXPnz0j0oxg6pKenGxsbh4SE rFq16s2bN30/UE5OLjU1tZtJhE5ryHxYdGyV5C+/Ka89zCUoOoBWsdmQUPQL3WrQamhoQIH3cfIm S0lJMTIyMjY2TktLY/s3Jydn0aJFTU1NFAolJiZGSEhIVFRUV1e3oaFh1qxZBALh3Llz9+/fP3bs GIlEWrp0qYGBwcGDBwsLC+Pi4rruodPpv//+u7Ky8qFDh6ABgkQiLV68GI/H79q1a8+ePYmJiRMm TDh48OC+ffsCAwMXLlxoY2PT3NzM2h4AgK6u7sGDB4OCgjZv3iwtLR0bG6utrT1u56XxI92Azlcj sow2mvJChYaGHjx48PPnzzgcTl1dXUVFxdHR0cTEhEAgYLFYUVFRdXX1Q4cO2dnZ4fF4HR2dtrY2 mAlNRESENTlya2tr19QgPRUGABCJRDqdDgDIz8+Hms8QOBxu2rRp8HVCJBInT57MxcV169YtLBZ7 5syZVatWIQjCw8NjZ2fn6elpZ2fH6vH99OnTX3/9lUM6lkAgJCcn6+vry8vLs+kGwBx9KEEcKYSF hfn7+2dkZGhoaEhISDx69Ig1MWPvwOPx5eXlbBFvLSW5+T5LAABq+6IEVLTRHuYokMnk8XbJkKEu WrQoIyMD7ikvL29sbLSwsODi4iovLw8MDPT398/NzdXR0bl06VJ9fT0fH19sbCwXF1dBQQEUJb1z 5w6RSIyLi1u7dm3XPVgs1s/PT1ZWFp5CWFj49u3bTU1NQUFBJiYmv/76q4CAwJQpU4KDg+Xk5P75 55+KigoKhcI6E2ZnZ0tISMjKyp47d87Z2dnR0XHVqlVUKlVHRwclpj8z2l+9esW0+6xcuRKdAVCM Sm4qJyfn7e29du1a+M729PR0dnY2MzNrbW2FBdzd3U+ePAmHuJWVlYSEhLGxsa2t7cmTJ1nD0GBq EDMzM7jO7uDg0EthAIC0tLSrq+uyZcsqKyvt7e3Z6LKnp+eKFSvWrl0bHh7+9evXAwcOnDx50tjY WE5OLjQ0FADg4eFx586dVatWsR746NEjKAPEOXB2ds7Pz/fw8AgLC0MNqMP/Xc62k0ajmZiYNDQ0 wNzicGdgYOA///zT95p37drFNLVSvzeWXdhX+/zWjEN3xWaboZH4HIgFCxYMp6gch4CHh0dVVXXS pElMIojBYPT09GRkZEJCQiZMmDBlyhQymayrq1tUVCQqKiolJYXH45cvX75q1SoxMbHnz59PnTr1 2LFjampq375967rHw8PDzc0Nj8c3NTUBAGbOnPnmzRtRUVFlZWUAQEFBgba2dk1NTUtLi6qqKh6P l5KSOnjwIGvmzJycHB0dnbKystraWpi/6uPHjwCA8Ww3/XlUVlaGhYUxuenmzZvRPkHB/l4cpaiv r6fT6b0UaG9vb25u7vavlpYW6IrUe+GkpCQYw9RTPQiCNDT05rH3+vXrxYsXs+0kkUg0Go0Du5RK pYaGhjKDwcfAIBmNKC8vl5aW7jY45sqVK30Ph0IQJDk5mUGjfnt6vTBgeWvVR7RvORyDHjPLmf6m K1asgKx048aNlZWVrH/t3btXRUUFQZB58+ZhsdikpKQbN25gsdjv37/v3r1bSEjo+/fvaWlpHh4e zc3NFy9ejIyMRBAEfoNlZmZ23SMlJeXo6NjR0WFhYcHNzU2j0davX8/Ly/vhwwdY89atW5ubm3E4 XEBAAI1G2759e1xcHLM9ra2tOBzOx8entLQUALBv3z4qlbps2TIAQO8RXWPV37Tb9oABxemzivAM 2Fd1qP1N0Umpv10HxkMs1IiDyU0HhqNHj6qrq4+6SKPy8nJNTc3o6Oh+0SAUg8VOpKWlewp7Ki8v Dw0N7Xtt9pZmuV6m9a+T+huJj2JEMIJzxXByU09Pz66sFMLMzMzBwQFBECcnpzlz5iAI4uPjo6qq iiBIdXW1tra2gIAAHx/frl27EAS5d++euLi4lJQUgUBYs2YNnU7vumf79u0AAEVFxePHj+NwuB07 duTl5cnJyTFtdX///TeCICEhIby8vMLCwioqKqxSLXDd+ebNmwiCWFlZYTCYKVOmzJ8/X1RUdOhe 8BzOTQdGH7tyU2agMyug3g7KTcc5N0Uz0f0ATU1NNTU1rG6m/UJOTs6kSZOYSS9GEWg02q5du969 excTE9P3HKcoBua8BZ9YGo125MiRDx8+XLlypZf04jCH0A+zXHbUfam+e6a0sV3DaZu03CS0n0cF oH/wiDxxw5mzlEqlcnFxDXhO5ufnZ81rX19fLyws3MuepqYmAQEBPB5PpVJxONyDBw/ExMT09fUv X77s4eFRWVkpKysLJ73m5mYxMbFezt7Q0CAsLAxDYAe9/xGOz1na00gYQM7SgoKC33//PTAwkK2k qqqqoKBgfysHQ5OztOvOYQvA6O/ljGDDekpSyvazX7cJ9Tn7AURERERERAZ8+Oh1lsfj8adOnSIS icbGxkePHh3EZLsoeiIlzs7O1tbWP8woq6+vf+XKlalTp/bEYBA67WtSZFN2yuSNIdiWTkFRcbR7 RwskJCTS09PH/NfggIkpnJNZf2KxWLbu6rqHeQg8b3R09J07dzQ1Nd+9e3f06FFITOGk1zsxBQD8 sMCY/4oeRIiJiUH/3dGFYfhOGFhXc2zDBoDRFAv17Nmzd+/eDfjw0tJSd3d31j3u7u7QhagnQPH8 8fymhNKnUVFRzs7OfY8QRzEAQKGovgQE4PF4Kyurd+/edRuyRiamvT/syispP93vOs8EGUVFRaZ0 GopRgXEYrT/MuHbtGpFIDA8P//z5s7e3N9ohI8J+CASCmpoa2qUoRj03jY6OZuqMDABtbW3FxcWs e4qLi9va2no5hMFgEIlEdJTExMS4uroaGBigvTF0ePLkiYaGRh8LS0hI4PF4NpnMjrovxX9ubH73 StU7Umy2GdxZUVHR0tKCdu8ogry8PCqUMdSYMmWKnp7eeDaCjjiUlJT27dvHtpNOp3///h3tHBTY Ud367OxsqAC1f//+jo4OAEB5ebmbm5u5ufmaNWvq6+sBAEVFRTY2NtbW1jC3BxueP39ubm5uZWVV WFgIAHj69Kmtra2ZmdmBAwdYPxDb29t37txpYWFha2sLBU2vXLkSHR3t6ur666+/RkdHAwBoNNr+ /fsXLFiwcuXKmpoaAMCdO3esra2XLVsWGxsLAMjJyQkKCnJycrp9+/ao62pUon8oQKPRnJ2dAwIC qFRqf5dxDQ0N29vbX79+TaPREDrtS+L50rM7Fdz2yy/3wvHwMYu9e/eOmTwCxaiAmpra+/fv0X5A wVEYCrH9y5cvS0pKysvLJyQkAAAKCwvnzp3bbWYKFOMOoygWbM2aNefPn2eV9pg0adL79+8RBFm/ fv2RI0cQBNm3b9+dO3cQBDlw4AAM5DQyMoqPj0cQZPv27fr6+qwV6uvrr1+/nk6nx8XFzZ49G0EQ CwuLz58/02i0RYsWJSQktLW1iYuLIwgSFxfn6+uLIMizZ89gajU/Pz9DQ0MymVxZWSkqKspgMI4d OwZrO3fu3Jo1a3Jzc7W0tMhkcktLi56eHpFIfPr0qYSERFpaWmtr6yiKv2MbJNHR0dLS0r0kw0TR R9TW1kpLS/+kZlBaWtruNS55fr815XeT/rGlpcXJyQnt6lGHbuXDxlKcfn/x6tUrPT29zMzMS5cu hYaGVldXw/1v3rwJDQ19/PhxH+u5ePEiW+HMzEwtLa3CwkInJ6djx45xSLAz4LA4ffBzKUm7LdDW 1jZ58uSSkhIikaiqqhoYGDhp0qRr164NrHIwXHH6w3MvBnCWEWwYGII4/VEcC5WXl6ekpAQj6F1c XAICAvbs2bNv377ExMTg4ODS0tL29nY4eVlaWgIA7OzsXr58yVaJi4sLFou1sbFxcnJiMBiXLl1K TEwkkUjt7e2fPn1iFrOxsRESEgoJCSGTyXA/BoMxNzcXEhISEhLi4+Nrbm5+/Pixr68vFotdt27d unXrDh48KCsrGxcXBwCQkZF5+PDhzJkzp0+f/sPwak77dGHb4+zsbGJiAqN21q9f30s4OYpekJ6e bm9v//LlS0VFRWhAHUBPdtR9EU0Jdft19vN6nG4zpuvAioyMvHTpEtrbKEY1KioqzM3Nd+3apaam pq+vj8ViZWRkli1bxmAw3N3di4uL9+7d25dUJmQyee3atb6+vqyF09LSiESioqLi1KlTT58+7eXl xZrAD8XQ4fPnz7NmzZo8eTIAYMaMGeXl5Xl5eT8TeTzoi1qNjY19LJyTk5OcnFxVVZWbm+vp6fn9 +/ebN29qamqePHkSABAeHn7//n0ajbZmzZrffvvtzZs3KSkpNTU179+/NzEx8fLyGrqr4NiG/RCj mFh0dHQwX+c4HK69vR0AsGTJkl9++WXx4sXc3NypqalQmR/ONd1GhvLx8YF/M3N++/Zt3rx5gYGB xsbGhYWFrLTs4MGDRCJx69atGAzmzJkzcCdzqRQudnR2drKegsFgEAgEXl5eyOemT5/e0NAwNqJS JCQkkpKSIiIi9PT0njx5gipM9XfKO3LkSGpqamVlJRzAAxBAoXe0fbl7pu1L2eSNITwTZNRotCNH jpSVlc2dO1dYWFhUVLShoSEiIsLd3R2OQBQoRi+8vLykpKS8vb1fvnyJIIiGhkZlZSUA4Nq1azgc DkEQXV1dBEHCw8OvXbsmJCS0Y8cOY2NjMpkcGBj49u1bbm7udevW2dvb5+TkwMJUKvXw4cNPnjyZ NWtWTU3N1KlT+fn5Z86cGRAQUFVVxcxQhYL1BTfo1XZ2djLfoby8vDt37uQcYgoAqKqqUlJSAgBs 27bNwcFBW1ubLWEkK5qamkJCQt68eUOlUjU0NC5duvT48ePZs2fn5OSIioo+efIkLi6upaVl+vTp xsbG9fX1R44cefPmjYKCgpGR0YwZM8zMzIboKji2YT/EKP5AVFNTKyoqgpQ0KytLW1ubTqe/fPly //79+vr6Hz9+hLYoaWlpGIz/+vXrrpVA59Hi4uKJEycWFRUpKiq6uLhoamoWFhayelU+ffp069at RkZGNTU1PXlb6ujoQLsskUjctWuXjo5OW1ubo6Ojo6OjiIiIlJTUWJqt8Hj85s2b4+PjjY2Nf6h5 NNQgEomjJdMjhUIxNzcXExNLTk4esMm5IfNh0bFVQtPnTN1xlmeCDLwdfn5+y5YtExISCg8Pv3nz ppSU1IEDB6BRFsWow9SpU6HQKYrq6urY2NgdO3bg8fjs7GwhIaFZs2ZVVVXR6fTAwEBzc3MGg6Gr q3vw4MF9+/YFBgYuXLjQxsamubnZ19f34cOHhw8fnj179okTJzo6OnJycgAAsHBQUNDmzZulpaVj Y2Nh6lEZGRlozEP7HAUrTp8+PW/ePAEBAS0tra5qrEz6rqOjIy8vr6ysjMFgli1bhsfj1dTUamtr FRUVAwIC/vrrr3PnzuFwuC9fvgAA5syZo6qqysvLa21t3XU5d3C/KzizYWONmx44cGDGv6BQKD4+ PgsXLnRxcYmNjd27dy8Oh7OxsTE1NbW1tdXV1c3MzIyPjw8MDFyyZImjoyObKRQAwGAwMjMzV61a ZWtre+TIEV1d3ZqaGgcHBycnJzc3t3PnzhUVFcGSDg4OmzZtcnBwqKqqkpWV9ff379q2PXv2xMfH W1tbu7m5WVhYWFlZSUhIGBsb29ranjx5spdPLg7/Yu5Fz0xRUfHt27evX782MTFhVZgatnB+CoXi 6Ojo4uKydOnSLVu2cLjKFZFINDAw8Pf3H3Dm6I66L+8CHdtrK9X2RQlrzGP7l0AgSEpK+vn5OTs7 /39qjeHSokMxuJCQkGATFRm3ePjwIQBg6dKlAIDs7GwdHZ1JkyZVVlZGRUWJiYlJS0uLi4vLy8sH BwfLycn9888/FRUVFAqloqKCj4/v06dPN2/eNDIyevnyJQ8PT3Z2toSEhKys7Llz55ydnR0dHVet WkWlUqEKNQwP/xnh1TGJITKaQty6dUtOTk5OTu7OnTvGxsZwe2QpEYlEevDgAZFIzMrK6vpvL2qs TBswHo+Howh23bNnz1xdXVVUVMzMzAQFBWFnMgtzc3NTqdQhvSKObdgPMNqDBmg0GplMZt1TX1/f 2dmJIAiVSoVr+h0dHW1tbT3V0NbW1tHRAbcZDAaJRGIwGPAo1mLfv39vaWmBZXrJ5Ekmk+FJIdrb 25ubm0d1LrK+DJL8/HxNTU0Y1lNeXg4AGOq04FQq9eTJk1JSUiEhIQkJCfHx8YcOHZKSkrp69Spn 5lmNjo42NjbuKRPpj8d5e2vp+b2l5/d2NjcM+u1DgYZDcWws1J49e4SEhOD2jBkzduzYERkZOW/e vClTpiQlJbm4uBgbG1dVVQEA7OzsDv6Lpqamtra2oKAgIyMjLBbr4+ODIIi6urqpqenHjx8BAOHh 4bCTAQCPHj1CECQqKgraTdFYqAGcGvQ/FopKpVZ2BxgoMsyxULW1tQCAtLQ01imaSZO2bt3KTCbc 7YmePXtmbW0Nt4WFheGGi4tLUlLSgQMHAgMDEQSpqKjg5+fPyclJSkpSUFCg0WiwDEyZO0SxUMPT MIDGQnUFDocTEhJi3cOUrGOumTIz13ULVoc8DAYzYcKEbo9iuopiMJheVmPZGsPDwzMeFHw0NDTe vn3r7u5+5coVmB9hyZIltbW1Q+SKmpiYuHbtWgcHh7/++gs6E2OxWC0trfDw8GfPnnl5ed26dcvI yIhDOodGo7m7u0Mn3QGs4yN0WuPbJ5W3/pzseUxARbu/X56o6XSUgm0mGbdoamqCOZ/b29s/fPjg 4+MjLS2dkZExe/ZsMzOznTt3WlpaCgkJ4XA4TU3NPXv2eHl5zZ8/v7GxMTQ01MfHZ/fu3ZaWlvn5 +W1tbR8+fLCysoIzRlVVFY1GCwkJAQDANf3Hjx+rqKjIy8ujfc42hwyRCRaPx8vJyY34BcIU0DA3 GFukMkw909LS8jPLnnDZ9t27d8LCwm5ubrt27dq1a5e0tLSrq2tHR0d9fb29vf2IXDjHNmyM2E1R DCmgqbjvcHFxYX2wB92EWV5erqGhYWNjc+PGjYQeEBUVNX/+fAcHBw4RusrPz4+Ojh7Ysa1VHwsD ln97ep1BG3hPoo85ajcdvXbT/fv3i4iIIAjy6tUrAMC7d++gq1VSUhKFQsFisVB1KCQkhJeXV1hY WEVFhUgktrW1WVlZ8fHxSUpKKigopKWlwcNv3ryJIIiVlRUGg5kyZcr8+fNFRUURBGlsbBQREfH3 9x/ZRSqOspv267yDZV4dWBsG1kVUKjU0NJT5jgNDpiHV0dHR0NDA3E5KSrKzs0MQpJc1VTAsGlKD 1bChsJsOoSsJivEGIpGoqanJumfr1q2nTp0alMopFIq3t/fz58937drVF3NsZWVlQEDAunXrvL29 Od/Zl0KhsDWS3tH26eohAID8ci8uQVFOdhpDMRSoqKggEAjDKYLBphfBOmZGcPxERkauXr0acscf LlA0NzezpnpqaWnp7OzsNvlTQ0ODsLAwDoeDP3fs2HH9+vWPHz+OoJRKTx0+AB2PQR8MgzjDDFHJ /nYRiUR68+aNhYVF76fruvPnn4WHDx9euHCh9xQ8AzjLCDaMbWdPP/t1m9A3FopBg7Oz87Vr19h2 3r9/n/X5HwBoNNr169e9vLy2bt2qpaXV9wNhytnTp08fP37c0dGRY6VY2Z5YuIhf8/DyJGef/i7i D+LcjYITPvb6nsZ2rHLT2tpaGRmZixcvrlixYohOcfv27ZUrVz5+/NjAwGBk5wGO4qb9OmO/yvfl iga9QjZiWlNTw/ZwDRs3bWpqqqmpgdLsHMVNB9wwlJui4HTQaLTXr1+npKSEh4fDxK3w7TJg809q aupvv/3m4OCwaNGigYXQUqnUu3fvvn379urVq/2itiPCCVpKckvPecv/tkNUzxiDww/uWdCHfXQh PT19OFN1cCY3BQBkZWWJiopCkfahwNu3bwUFBadOnTri88A44aa9HzKA6+3XITQa7fHjx10tJsPG TYe5P4ehYSg3RTGStKm/qKioSExMjI+PLygoYOrM9+vwNWvWCAkJLV++XFhY+CcvhEwmHz58ePr0 6WFhYZyWLAB2cmdzQ+WN4wAABbf9OB4+DpnvUIwgHjx48JNrDmODm46fyZZDuOmwEaPBYn796qKY mBhnZ+eBsS6Um6LcFMWo56ZMQNnRvjt9UiiUoKCguLi4LVu2KCgoDOLlfPjw4ejRo56enr6+viO7 xF/77IbEfDtoGUXoNNKL2LrUO0prDvHJqnDalIdipFBRUTGc2RP6zk2fPXtWWFiIwWCkpKQ0NDRU VVUHcDo6nf7gwYPs7Gw8Hm9gYPDrr7+iky0ncNPBYpkDGHWDUskPn6me3Lj7zk2H50YMgJuOVMOG gpuiiYNRDDkIBEIfiSmNRouKipo8eTIfH19wcLCioiJmUKGmpnb58mU6nS4nJ5eYmDhSHfIl8Xzl 9eCvSZEAgJaS3HyfJXiCsNq+qKEmpsM8haH4SUA5Ng5EdHR0dnY2gUD4+PGji4uLm5tbTyWXLl3a 0dHR7V87d+68devWvHnztLS09u/f31PQ5PHjx5OTk9HBMFbx/0HZGEwvxHQQp6yMjIyfXDcbNs2N MdOwAb6nUCMKit7J4rDZF1NTUzdt2jRz5sxly5YNtShsR0fHpUuXSkpK7t27N8yJPVtKcouC18Jt Ic35vBNkZGw2DdEi/k/aGFCMLDjWbrp27do5c+asXbsWljc1NXVwcPDw8Hj69GloaGhbW5u+vr6/ v/+FCxe2b9++aNGi6OjoN2/esP6FwWBUVFSSkpKmTJkCAKipqamtrdXU1Kyvr/f19a2srJSTkzt6 9Oj79+8dHR1lZWVPnDgxd+5cePbHjx8zGAxmY2bNmgUFUEc7OMFu+jOWy59p4U+6Vvexi4hEopqa Wk9vtD7aTQf2IA+z5sbwdHtPvTQodlM8QIGil/ExLMSURCJt3ry5trbW19dXVFR0GM7Iy8u7cePG mpoaKysrdXX1ixcvDo/OVFt1CZOYAgAwAMgv9xpmsogK8o8WSEhIDDM9Hdhrac2aNZGRkR4eHidO nIiIiJCRkVm8ePH9+/c9PDz27Nlz8+ZNXl5etr8sLS2XLVtmZ2fn4eFhZGSkpaUlLS0NAPD09Fyy ZMnZs2evXbvm4eFx69atWbNmrV27lklMAQCvXr1i5aZTp04dG9wUxVDbWSorK4dT+IIJRUVFIpHI adyURqPBjAMcyj3QIYtiZB+PgwcPnjt37vDhw0pKSsN8dhkZmVOnTmVlZU2ePHnPnj2bNm0aUi5O /d5YfHIT6x5y/ouWktzh73ZIT1HTKYeDQCCQSCTOb6eYmBj0Kb906VJiYiKJRGpvb//06RNrma5/ BQUFWVtbJyYmbtiwoba2Njw83NjY+N69e4sXL4aONw8fPuz2dH5+fujYGKLPjAHPCdLS0iP7xcsm rd0Vjx8/NjU1Hanm9W6yHRGMbIf8EKi/KYoRQ1RUlJycHMw1r6ysjBkhzJo1Kzo6uqKiQk5OLjU1 dYgult7R9i7QkdZcz7qTT1altbJoBF9F6CDkcHCsyykrXr9+ra6uXl9fb2BgwMvLa2xsrKSkxMpy evrL0NDw6NGjL1++PH/+/Pr16+Enk6CgIC8vL4FAOH/+PKt9lIlNmzatZ8GHDx/QcTLis8HXr19/ Mp09ayL7ASAvL6+X5lEoFGFh4RGkhs7OzhEREZxzu4lE4qxZszhW8xugdlMUfZmwBt3ARiQSXVxc VFRU/v7776F2Le0LcDictbX1/Pnz//zzT39//4sXLw7uQipCp5We3QWJqeisxSKaRnyyKrzSiv8q mDoO/yX/MP4ABSdg+vTpHN7C58+fnz179uXLl0QiUVFR0cXFpbOzs7CwUE9PD04gNBqt61+1tbUL Fy588eIFTNrEy8vLz8+Pw+HU1dVVVFRmzpxZX1+fkZGBxWJhDaxnXLduHeug5YSc7GMDIzgVSEhI pKenD92qd3x8fLe6UcOJ9evXp6eny8rKjqyXDoVCiY+P19DQ4ED/V5SbohjJB2PNmjWFhYWBgYHd 5hIcQYiJif1fe2ceT9X2//9Rx6OiAAAgAElEQVRlnudMGUKaIzQoitsglNIgGZpEtzS7lWZE0kSJ DJXqdks3FW5pJpShyCEKlTKdMs+Owxmc3x/r9zmP80UnleHg/fzDY9t77X32XnvttV5rrfd6v3fs 2PHly5dFixb98ccfx48f7ykjVEpdpfw8O3Un7y6jj/ZXkwCGp5xPbm4uZ9qbenp6BgYGUqlUeXn5 +/fvq6mpSUtLl5eXW1tb0+n01atXh4aGmpmZ6erqLliwAEfi6HBox44dhoaGampqNBqtvr7+0qVL CKHAwMCNGzeOGTPm06dPu3fvRgjp6uq6urpyc3MzXb1yZgSNQTAGMVhJTk62tLTsf7HFy2toaEgi kR4+fCghISEuLq6hodH9JqZzlLifihtXVVX18eNHhJC4uHi/y/RulUkYNQF+WGf1SCGh0Wjnz5/3 8fFxc3PrF4P0n7rVjIyMv//+e+/evZwc7LSnXjFUAhxLX66F+n3f+wwGo6amRkZGhouLi0Kh8PPz t7e3NzY2SkpKdj6ET6mtrcWDpqzXqa2tlZSU5Ob+/yZndXV1kpKSg74f1Y/r9DmhEugli0ysybqz Gr331ul/r5UhEolNTU2lpaV4eiQlJUVLS0tMTKypqamxsVFJSYm5uj88PNze3j47O5vZdObk5Ghr ax85cuTAgQO8vLxVVVUkEik3Nxdf6uvXr8XFxTgGb25uroqKioKCQu8NlILvfWCg8uDBA0dHRzs7 u8WLFw8UqdfW1hYeHp6enh4UFDRz5szeVsP9mC0gTzl5yKfPwpZCXKihqU055OXieNc9W9qrqqrK y8u7ORTSx9q0m13Turo6V1fX2NhYhND27duZboA9PT3d3d0RQlOmTHF3d9fX1+/HOXrQpsDA4P79 +3PmzMGzFUVFRYsXLx41atSGDRvExMQG3LPU1NScOXNGXl7++PHjvTSC1e8OR8HjKcfyU9N2oE1B mw7orunDhw/nz5/fU730qqqqZ8+edX/+mgO1KYlEMjAwyM7OZu6hUqk4fxQVFcvLy/FOBQWF7Oxs 0KYA8OOSKi8vf/r06devXyckJLi5uXGyH7XuUFBQ4Orq2kvBTjlEGoL+4EAG1pw+MOC0KUe9WRqN 9vTpU6Zh8e+QnJwsLi7+U/06DtSmWK87OjoyZWhSUpKhoWFycjJzKk9BQSEsLKxHMo2jtCn4kAJ+ UOZ+zcwrLCzs48ePM2bMuHnzpoqKCvcAZ/To0Xfu3JGQkFBWVr5+/fogft1Q5jlQng6OB1mzZs3n z5+7P2JkZ2fX1NQEBWDodEd5eXmnTp2anJz8mwI3PDxcR0eHw1c1dJMFCxYUFBTY2trif8+fP48Q ioiIwP/a2toWFBT0rzDtrcIJvWGg+0Mp3T/r2bNnenp6DAZDWFiYRCIVFxd3cAQzQGlra/P39//0 6dP169d/6O0Z2iqgR7Rp3wyd9va4aW1trbi4eDenHZydnY2MjJhN8lBTin0zbsqZ3zt2cmRtbf2z M1TYYrWhoeHXtBpnjpsyYQ6g1tfXS0pKcsJw6fdyCeb0AU7XpmQyOTMzU1VVdfz48eXl5d++fRsc 2VJdXb13794JEyYEBARwuJc4kKegTftAm7q4uOzatUtZWbmwsDAkJOTEiRN79uwxNDQMCwujUCg+ Pj56enq7d+/evn27oqLi8ePH09LS+Pn59+zZM3369MzMzLi4uIqKioyMjGnTph0/fvzr16/z5s17 //49c6k+aNMe/yo53Mo8PDzcxMSkm7UriURKTEyUkJD4naVUHK5N8WNu2LBBT0+PQCBcvHixb+Js 95c2Bf+mQK9TUlJCJBInTZqkp6dXUlJSW1s70J9ITk7uypUrb9++1dbWfvr06eCYPMJAOFNOIyUl hTNdnLLy8uXLjRs3IoQaGhrwnGxaWhqNRvvvv/9u3759+PDhBw8eJCUlrVu37s6dO1++fLl3715R UZGRkVFeXl59ff2JEyeSkpK8vb2nTZu2fPnynJyc+fPnDylh2plfcNHA2o0hkUg/1C6c/Jnb2dkV FRWlp6dLSEiMHj26s0glkUhZWVni4uLY3RInjCD2jU3UzZs3mX97uy3ox8wEbQr0eul88+bNlClT srKyBAQEZsyYoaioWFhYSCaTB3TOSEtLr127lsFg9MhK6u40JP3YDwb6EeykcMDBxcVlZWXFzc2t p6d3/Phx5v6nT5/u27ePi4tLXV19zJgxb9++5eLi0tHRGTNmDEJowoQJ3759+/Tp08iRI4fm62Z+ esXFxTNnzmQdPaVSqXx8fM3NzaKiosxuJN6JEKqsrFRXV2cevXHjRkxMjIWFBfMvQsjCwuL169eH Dh2Sk5NDCAUGBiKErK2tY2NjtbS0hISEeHl5lZWViUQiJ3SH1NTU8G1UVVXhiErMQ1+/flVSUuoz 92oDRc8NOJ0N2hTof+h0OoVCefHihZSU1OTJk5uamkpKSrqMl83hCAoKamho0Gi058+fV1RUqKur 90gtwDmVGsSLAn4Z1i8ahyPu0M9pa2tj2hHy8PC0trby8PAwAxfjxBQKRVBQcIgPBxgYGLDmG95m /mV+oaxGmQEBAaxdXNyrwSOOY8eOzc/PRwglJCS4uLgghB48eICXdero6Fy5ckVWVnbNmjXm5uaF hYXq6ur4740bN/A0l66uLvbljq/Wxx1pWVnZDuOmnD+TAIA2BQYGBAJh2rRpCKGGhob4+PixY8dO nDixrKysurp6wHwtvLzKysqioqJpaWl1dXWDu4HsztApSNjf1B/d4evXr5zfEouIiOAP+c2bN+xT 6ujopKenz5w5k0Kh5Ofnjx8//sOHDx3SyMnJVVZWDvFCwv6ld5CqeOfWrVs7l64ODj7t7Ozwp00i kYyNjVtaWvj5+T08PBBCo0aNCggI+Pr16/bt23Nzc/FCtJSUFISQlpaWvb19QEAALy+vs7NzfX29 s7PzxYsXExMTjY2NBQQEBnfwPAC0KcBZ9MaQXmpqqoGBwYcPHz5//qytra2lpVVSUsL5zmJkZGQU FRULCgp+08XJwHr7P3z1MPX/a99U92loaOD8h1q9erWjo6Oenp6ioiL7IrF3796VK1e+efOmuLjY 1dVVUVGxszY1NDT09vaGotLb3QnmX+bkOFa3+F9swWlnZ1dVVSUmJpaUlNTQ0KCurq6trV1XV3fz 5s0tW7Zcv379yZMnxsbGwcHBFy9eRDCi2ctQKBRTU9P4+PjBX09C0wL0uDbtsE4f25u+efNm2rRp XFxcr1+/NjAwSElJmTVrFhcXl7CwsJaWFg8PT2FhIZVK5cBMEBMTU1FRqa2tffPmTQcjhLdv36qr q3c/9Mgge/tgltpLip+VgeJDikQiMRgMbOz4QxobG0VERHh4eL7X4ZkyZcrt27c1NDSGZsHovXX6 PfXNYre7X79+jYiIMDExCQ8PHzt2LD40d+5cVr07oD/Mn8quvqkPKysrsbkw51RTsE4fGGzQ6XQy mfzmzZthw4aNHj26oaGhvLycc+QOHx+furp6e3v7mzdvqFTqQLSO/R3wzD4IUKA7/JQNori4OPum LiAgwMXFJTo6GuxGerz70SPg/pKamhrWoLNnz8ar5rW1tRcuXLh+/XofH5/vLbEHuqS1tXXDhg3/ /PMPQujevXuNjY02NjYbN240NDQMDw+Xk5MLDAwUExNzcXG5ceMGlUrt4Ivt2rVrPDw8jx49+vr1 q5OTk729/YDODYgLBbCDSqX2qi5hzo/X1NS8fv2aQqGMGzdOUlKSq7/h5uZWVVXV1NQsKioiEAgU CmVoFgBQpf3O0IyNZGBgcOzYsZaWFigAA+KLFhERMTQ01NLSam5ulpGR8fHxcXR0JJPJcnJyubm5 HGUHRSKROPPV0Gi0Z8+e4e0vX77k5+fz8PBcv35dWFg4NjZWQkIiLCyMTqc/efIEIRQQEIB9sZ0+ fXrFihXNzc2fP38ODg4OCgr6559/tm3bNtCrbtCmADv6zML9+fPnCKHCwsKMjAxxcfGxY8eKiIj0 V4RSSUnJsWPHtrW1vX//vg/WPP1yYNg+a8xg7KofycnJGZoPPmHCBI5yrDbQ6ZsJEBERETU1tQUL FmRnZ8vLy9va2j5//jwuLk5RUTEjIyM5ObnfpeGGDRsmTZr08OFDjhWpHV7ZypUrEUJ6enqswWue Pn26du3aDr7YzM3NxcXFlZWVhYSEGhsbB7b2gC8W6HHk5eUTEhL09PR+6qy6ujq7tfYnvI+3t7eL iYlpaGhQqdTKysq+NEIVEhJSUVFpbm7Ozs7mcMk4KBs2oDODKbJD9yESidHR0QghPj4+TU3NOXPm /OzHWFpaunv37lu3bkER6hew46fw8HBsmRocHPzhw4fXr1/HxcXl5+ePHTt206ZNnz59Gj16tJSU VB8v88/Ozl64cCFCaN68eS4uLsbGxpzWC2Laj/Hz8+OS/0NfbOh/XtsGR3UN46ZAr7Qr3NzcEyZM yMrKQghNmTKlO2e1tra2SyCvW8c3bP2zubk5Jyenrq5OXV192LBh3NzcvT2Jz8/PP2LECAUFhS9f vhQWFnb/YQfEMurfhNWSHehjxMTEhuBTP3z48N69e6Kiojw8PBcuXFi0aNHPXkFBQSEkJKTz/mfP nvn6+nbzIj+VGPqWXYJd6JeVlZmYmGDXqhYWFu7u7rGxsXFxcXZ2dq9fv+bi4iorK7Ozs6usrExO Tq6qqmptba2qqurte4uNjV24cKGoqKiJiQnq7+l+ISEhEomEx2K66YsNIcT0xTbIagAYNwV+UKOh nzdR4uXl9fT03LJly7Zt28rLyzdu3CghITF16tQff5wjRNVdtVIXPkQI0Wg07P1URUVFU1OzoqKi lwzvuLi4ZGVlxcTEqqqq6urq6HT6z17hy5cvuC0UFhYexPIUtCnQZxAIhGXLlq1btw4h5OjoKCEh gT0ZHTp0KC8vT05O7ujRo0pKSleuXKFQKLdu3bp8+XJoaGheXh6dTt+9e7exsfHNmzfl5ORmz57t 4eHB3M9gMDZt2iQjI2NkZCQvL79///6GhgZtbW0PDw9+fv6//vpLX18/PDycm5s7ICCgoKCAmRhX X5mZmc+ePSMSiVlZWRs3bmxqaoqIiNDW1j579ixCKDg4+MGDBzQazdHRccWKFenp6XFxceXl5Xl5 eSYmJrt37x7i7xSPpLq5uaH/xQUwMDDIz8+XlpbGuvDmzZvLli3Lzc21srKKiYnBvYJVq1YtXLiw sLDwwIEDx44dO3PmjKen54MHD7B3lOTkZB0dHYRQVVWVsrIyLy8viUT6BX+rsbGxCCFRUdF58+Z5 eHjo6+v3vcdWHh6eZcuW/fHHH0pKSoKCgj/riw20KQB0C3l5+YiIiMTExG3btuno6Cxbtoyfnx8h NGPGjC7TS0pKCpRyfw39yBRDFhYWLi4u5ubmFRUVGhoaw4YNKysr69llSWJiYjIyMi0tLUQi8TeN B8rLy/HGzzq+GSiTL910yA8Av09mZqaDgwPe/vTpk5CQkIyMzLJly6ytrU+fPh0ZGbl169aoqKjo 6Ohhw4b9999/V69epdPp0dHRX758uXLlirGx8d27d/fs2XPhwgXW/V5eXgihx48f4/jJ4eHhEydO 3L59e3Bw8I4dO65cuTJy5Mjo6OjNmzdfvXr10KFDODEWTwih+vp6Pz+/9PR0KpWqpaV1+fLlp0+f Tps2LTMzU0pKKjY2NjIysrm5efz48fPmzaupqTl27Fh6evqIESNmzZo1ceJEMzOz/hpi4LTPlukW LTw8HNctNBoNT1hNmTKlpqamtbUVuxrEI4I1NTU3b948duzYuXPnXFxcTpw4YWlpKSoqqqCgcPDg QWlpaXt7+/r6eklJyYCAAPwvrq+wx6VPnz6NGjUKx3fFMQXYiFSsU7dv325tbd3H2fL333/X1dWJ iIjgthIh1NzcjDecnJzwBg5yoaCgkJiYyOqLDQdQwBCJRNCmAMAOY2NjAoFw48aNTZs2bd++fdKk SdzcXVuSCAkJRVy79fDpo2vvriGEDA0No6KivL29r1696u/v397ejq3sW1tbKysrf9+dk6CgoLy8 PI1G+/btW3cq7ra2NnibIE/7haqqqiHliIdOp+fk5Hh4eAgICJBIpJqamn///ffTp0+vX79esmTJ 1atXGxsbP378iCVscnKymJiYtrZ2UFDQyJEjra2tsQDNzMycNGkSnU5n3V9fX9/e3i4tLR0aGioj I/PmzZs3b97QaLSPHz9++fJFVlZ2y5YtCCFxcXFpaWlmYlaRp6urq6KigreXLVvGy8s7bty4yspK XV3dI0eOXLhwoampiYeHBy9bmT59+pgxYxBClpaWqamp/aJNB8p0By8vL1Ow+vv74yaAGekKK1dl ZWX8NygoSEREpLm5OSsrq6GhQUVFxdbWVkJCAp8+YsQI5mXxNL2mpib6yaW9586d65d8kJKS6n5i 9r7YBjRgbwr8QIv8vhDh5eVdu3bt58+fyWTy3r178Qz49zwzLzRdcPtGBN7W0NDw8fE5evTozp07 9+3bV11dnZ+fTyKR1NTUpKWlf9m0lI+PT0FBQV5evrq6uqKigv0kPo7u3XlItb29PSUlJSAgQFtb e6gVCfgu+hjOX1Dcs+Tl5ampqV28eDEwMPDu3bsEAmH27Nm5ubk6OjqCgoKCgoJycnLnzp2rqqri 4+PDStHY2PjJkyfV1dVTp05NT0+vrKwUFBQUExPrsJ9AIOAp4NzcXD09PXw1IyOjTZs2EQiEWbNm 4RvIzMzU1dVlJmaFudyEl5eXj48P/W9UMj4+ftWqVZqammZmZmJiYvgzYSbm5+fvx8Aig+CbxcqV +Rc3H9hx1YIFC7S0tJjjr1u3bjU0NGRGc1VTU+syvuv3UFBQ2L59e1JSEpVKxRIZAG0KDGZERES8 vLzu3bt3586d4ODgmpoahJCRkdH30qelpbW1tU2cOHHatGkRERFGRkYWFhbh4eHl5eXY8ZuampqY mNjPClMpKSkVFZWWlpbS0lIymYzHX7uUp21tbWQyucsWpbS09M8//6TT6S9evPh9Q59B5hCgoKAg KysrKyvr8+fPHBKt4M6dO3gibCCirKw81OoKAoEwbdo0ZWVlZWVl5siQiIiIiIiIjY2NjY2NgIDA 1KlTsYJECLW0tMTFxamqqh48eHDTpk3Pnj3LyMjQ1dXtvD8rKwt3JkVERNTV1W1sbFasWEGn07W0 tAgEAr4aQujt27fa2trMxAih/Px89vecmJi4fPlyMzMzaWnp0tJSGo2GEMrJycF1S3Z2NjNsUl8C NuLdREFBAb+msrIyf39/Q0PDvrc3Bf5PbwSyAOhLNDQ04uLiXrx4YW1tbWNjM3/+fCEhIYTQnDlz Oieura1taGhQUlJSUFAQEhIyNTUNCwubO3fu2bNnubm5eXh4VFVVJSUlKysrcUvwQ3EsLS3d2NiI B247wKpB2YSAamhouHDhgqys7IMHD35q8mXosG7dOiMjI2Fh4fr6+tevXz948KAHJ57c3d2PHDny s2dFR0dPnDhx2LBhA3TEaKgVoczMzMmTJ3fYOXfuXLxgn06nKysrW1paZmZm4nFNfn5+X1/f4OBg UVHR4uLia9euXbt2TVdXt/P+x48fBwcHGxoabtmyZfny5cnJyUQiEUfQyczMxBHki4uLpaSkRERE xMTEcOI5c+aMGzeO/cDnwoULbWxscnNzJSQkVq9evWvXrl27dikoKKxataqtra2mpsbKyqpfMhMm OthLUmdn56VLl2ppaXFxcQ1Nf22cCZiOAT/udvdGIaHRaOHh4a6urnhtLA7FxIzJ1GFDUFBQXV29 tbW1vLy8tLTU39+/rq7Oy8tLRUVFVFRUXl6+tbW1pqbme/fJx8cnKyvb3t5eUlLCZiSPwWC0trbS 6fT29nacDE/o4z2tra23b9/OyMjw8vJiM4//CxHAey+T+6B4dL7tmTNn4hUqCCEXFxcTE5N58+Yd OHCgra1NWFjY09OTTCZv375dUlJSSEho8uTJfHx8QkJCZmZm9+7d4+fnnzNnDmvi4uLi48eP8/Ly amhozJ0719LScuPGje7u7vi33r9/f/z4cW5u7hkzZmzatOnYsWPi4uIEAkFAQCA4ODgrK8vHx0dN TS0jIyMwMBAPXH39+jUwMPD9+/d///33rVu3MjMzubi4Dh48KCMjw1xJffr0aV9fXyEhoXfv3snJ yZ04caKoqMjb2xvHsHV1dX348OHbt2+Li4srKysvXLhQW1vLvMl9+/a9ePHiypUrvLy8q1atMjY2 7k6msaeoqIhpitdnH/v3wrv3L01NTfz8/My5clbq6+u5uLiYRoff219VVSUjI4Ot3uvq6sTFxfFS ki5hTfxDKBQKiUTCXVYKhfL8+fNLly7duXOnqampO47Avpfhv1w/gGl4l9jZ2cXHxzMlKfvs+tmw 8oMpo37qwVkTf+/fnyrJMG4K9Nto0Jo1a5YvX75+/fp//vln3759rAbsnSv9jx8/iouLa2pqSkpK njp1Kisra/Pmzfr6+rt37yaTycOGDVNWVq6vr2euasRwc3NLS0vz8/N//foVeyf+Hvhol59Ne3v7 mzdvzpw54+bmduDAgR4fxxp87cfr168lJCRqampevHixZ8+ey5cvjxw50tnZOSQkBC9k+eOPP9av X//nn3/W1tbSaDRRUVGE0Ldv3wQFBTsk/vbt2+LFi5csWfL48ePJkycPHz6cKUyx0Dxy5IiGhsaU KVOcnJw+fvw4f/78y5cvL168uKio6MCBA+fPn1dVVZ04cSLzFCqVGh0dnZ2dXVBQkJiYePPmzfz8 fFdX1z///JOfn//s2bOZmZkNDQ0fPnwwNTXdvXv3qlWrXr9+ferUKU9PzwkTJtjY2GRlZVVWVhYW Fl64cMHPzy8mJqawsJB5k3Q6fc+ePS9fvuTi4po1a1ZqairMq/YgbESepKRkd/azrir74dTHTy1B 4+fnZy6vZm6gfvJQC6Xue1y8eLGXPO33TU3eNz3V/lcIUFKBfkREROTWrVuFhYWWlpaamppOTk7f a2DwkMmHDx9kZGRGjx4tJiYWGRn5+PHjuXPnHjhwYOHChXV1dXJyclgSYT9TIiIi2C0i+7ij7e3t rGvwO9QvX758OX369MyZM+Pi4rD5QWd4eHjodPovjJgOVt69eyciIuLn5/f48ePhw4dnZ2eXlpZm ZGSQSCQ9Pb2CggK86GTSpEmdz+2Q2NHRcc+ePaGhoba2tp2Hr9ra2tzc3GRkZMrLy3HvAg+ESEtL Nzc3E4lEdXV1hBCrNkUIjRs3jo+PLycn59OnT0zPLIaGhtHR0fPmzTM0NDx48CBCCE8rT5gwobCw 8NOnTxMmTMDX//DhQ4cfYr3JmpoaIpG4efNmhBCOHNh5JA8YCkyfPr2/NARTmA5NhcpeI0IgXNCm ANAt1NXVs7OzY2Ji1q9fb2NjY2VlhRfAdkltbW1jY6OioqKsrOzSpUtNTEx8fX1DQ0OPHj06depU Pj4+PJvMy8vb0NDww1jkXXpLxRP61dXVly9frqurCw0NZb/gCYRpBxwdHYcNG6agoODn5xcSEqKo qDh37tzly5fX19fz8vJ6e3tXVlYihEpKStTV1fn4+PBbKCsrU1dX75C4srIyPDy8ra3NwsJi7ty5 HX7Izc0tOTlZQEDg/v37nRtjUVHRhoYGCQmJDt7+sMZVVFScMWNGQEAAnU6vrKxsamry8PCQkJA4 dOjQ7du3EUKVlZXjx4+vqqqaMmWKjIwMduRUVlY2a9asL1++sP4QNlDBN2lkZDR8+PBLly4hhEpL S0GYDlkkJSXZ9LT7XaINVmDAGLQpMNQ7oD2LhYUFkUg8f/68lZWVu7t758UQrFrw27dvAgICSkpK ZDLZ3d2dSCQeOHBARkbGw8Ojvb1dVlaWm5ubSqXy8PB87xHIZDKb69+9e/fGjRshISFsTEt7cLiU RqMNviUvVlZW165di4+P37Bhw/r165OTk9+9e3f27NnVq1dv2LAhISEhLy9PXV191qxZ69at+/Tp U1FR0YgRIzokTktLi4qKGjVqlISEhJyc3IgRIzZv3hwUFIR/YsaMGc7OzqKionp6ep0DVG7dutXa 2nr8+PFdLpWbOXPmhQsXtm3bVlJSYmtrO27cuF27duno6OTm5jo4ODx69CgyMjImJiYnJ8fHx4eH h8fJyUlDQ6O+vh5rU9ZLvXjxgnmTSkpKdnZ29vb2goKCEhISfn5+UI0wGIygoCD8JYqIiMyYMWPs 2LH19fXXr19nTcbHx7dx40a8TSQSMzIyLC0t8fd47dq1WbNmYS+VHTh+/PiTJ0/8/f29vLxwp6L7 98PNza2srGxqatqlAStC6OHDh76+vvv27cMxLX+HoqIiLy+vS5cu9Y14AkvTQQaNRmMdsrG1tb12 7dogXigJttIAx1FVVbV169b379+fOnVKTk6O/TIpcXHxYcOG1dbWtrS0xMfH79279+DBgxs3bhQU FCwsLBQVFf369Wvnn2DjSD85OfnkyZOHDx82MTH54ZffI8J0kK2F+h6NjY3MBft0Op1MJv/77780 Gm3Tpk0UCoVGo7FGfGVN3NbW1traigcg8aI01pQNDQ3i4uJcXFw46EuHH21tbeXn52ezlqWlpYWf nx+/aDqd3tDQICUlxcXFtWrVKg8Pj+HDhzN/q729nUwmf29CkPUmEUJUKpVGo3VpBDIE10J9/Phx 3rx5np6eCKHm5uagoCAPDw8jI6OYmBiEkI+Pz8qVKzU0NMTFxZmReC5cuJCRkREaGtrY2Lhy5crp 06ez2hkzIRKJxsbGHz58ePjw4a1bt27cuPFT98NgMF69evXmzZuMjIwuU44aNerJkyc9NSvi5uYm Ly+P3fuzLxi/sxZq4NYnfV8p/fBEjlpPZmJigmNWIYTmzZv37NkzzszzHlkLBf5NAY5DVlb21q1b 4eHhe/fu9fb2Zh+QqZLaTjkAACAASURBVLm5uaSkhJeXV1lZ2dzc/NWrVzt37sSH6uvreXh42v8v LS0tLS0t9K4oKSlxcnJKSUmJi4szNzdnL0w1NDRgHv+nYPUkxcPDIyoqKiYmhpeJ8PPzs8rNDokF BASYmo+bm7tDSgkJCVzldWkHIigoyH6RtbCwMPNF8/Dw4JgOuBDy8fGx/hY3NzcbSzXWm8Q38z3r 5F+gqalpQL96AoFgYGCwbt26devWbd261c3N7fbt2woKCk5OTk5OTnV1da6urk5OTqwhIgkEwuTJ k4uKiubOnbtu3TosTJuamrZt22ZqamptbV1YWPjt2zcbG5uWlhZvb2+m26moqCgvLy8Gg/Ho0SNL S8t58+Z1DvDDvB8HB4dz585lZma2tLQkJSUxfa27ubl9/PjRwcGhqKho3759zNZ0z549ERERZmZm VlZW7969s7W1nTt3blpaGkKosLBw9erV5ubmjo6O2Hmzi4vLv//+u2jRIisrKxzF6q+//vLz82Mf 7KP3qK+vT0lJ+fDhQ6+Krdra2jt37jD/jYyMxAY8ndN0SNmBjIyM73UYWAkODh46VSgzii/6vxFK ByWgTYEf9If6y3xHW1s7Ozt76dKlS5cuTUxMZOPBlMFg1NfXl5eXS0hIyMvLdzjEKkxpNFp7VzQ0 NJw+fdrT0zMkJOTkyZPsVQWo0p5i5cqV2LUkB3LmzBk2jiP6mB+aTXM4mZmZenp6eJtCocTExDDj LX358kVKSqqzXWZmZiaNRlu0aFFgYODKlSvxzujoaE1NzSdPnpiYmLi7uw8fPtzY2Hj//v3u7u4E AgHbdVy/fn3Xrl0UCiUkJOTSpUv37t0LDAzsYIORmZkpKCgYGxt7//79TZs27d+/X1hYODY2llnJ XLlyRVFR0c7ObtGiRREREcw6MC0t7e3bt48fPxYREdmyZculS5e2bNni7e2NEAoLC1u+fPmjR49U VFR8fHwQQqmpqYmJiffu3bO1tV2zZg1CSFJScuTIkVjL9s0IFpNr164tX748KSkpKCjIxMSkvr6+ l25AWFgY20Hh7sS9e/e61KbR0dHMlF2CIy8QiURsuv09wsLChk6FyTQsUVBQ0NfXH9wPC/amAEez evVqW1tbLy+vhQsX+vv747XSXUKj0SorK79nN4a+Y2BKo9ESEhJCQ0NPnTr1vTCqrKq0N54R7Gq6 JDQ0lGl9mJmZSaFQvlcdsz8K9DsEAqG4uDgiIqKlpQUhZGNjs2fPHnwoIyODKVtfvnyJLTQMDQ1z cnIePnzY0NDA6sVp9uzZsbGxJ0+ezMnJwR3IzMzMAwcOIISysrJiYmJqampu377Nzc3d3t6+devW 8PDwlpaW5uZmQUHB3NxcrFBNTU0JBIKKikpCQgKZTC4uLl69ejW+yb/++gshVFlZKSwsLCYmxiqp mcpv8eLFCKGJEyeqqqqKiIhoaWlh7XXw4MGYmJhTp059/vyZ6a5u9erVXFxcy5Yts7e3p9PpPDw8 o0eP/vjx44wZM/oy/6urq8+cOZOeno6nCMLCwk6ePLlhw4Zbt27t27evpKQkPDx83759ISEhTHe/ NBqNzVEcJxa/0z179mCB6Ojo6OvrO2vWrGfPnqmqql65cmXMmDEIoaioKD8/P0FBQaaZOAa/LAcH B39/fzKZXFZWZm1t/ezZMwaDcfHixTdv3iCEIiMj379/P2nSJDKZzOoz+MaNG8+ePRs5ciSHRJ7r G2RlZRUUFMrLy52dnQd9SA4YNwU4vv/Ey3vkyJFXr15FRUVt27atoqKCTWIKhVJaWsoq+6hUKpVK xRaNHXj79q2trS2ZTH7x4gV7YQpjpX0Pbr/x6Mvbt2/T09M7p2F/FOAQsrKysFlncHCwtLS0m5sb 0/0nHu/E2wkJCXiFXF5e3ogRI+7fv+/j42Nvb4+HMzMyMubNmyciImJiYkKn03F80bdv306aNKm6 urq8vLyurq6yshKbcNja2kZHR0+ZMsXQ0LC9vX348OFZWVkJCQkvX77k5eXNyso6ffr00aNHfX19 PTw8Dh06hG8SX/P169d4gxkTlRXc++Xi4mJu4L7lwoUL8/LyjIyMpkyZwuxtMtNwc3PjpxASEmKz CvM3+d6gaVZWloGBAVPNWFhYpKamkkgkHIi1paUlPz8/Ly8vMTExNDR0586drq6u7I/i66iqqj56 9AghlJSUlJKSghB6/PixlJRUXl7eokWLRo8ebWdnhxDS1NS8fPlyaWlph9FrMpmcl5dHoVCePHly 9uxZAwOD5OTkCxcuVFZWlpSUlJWVlZWVLVq0yMLCYsqUKdhHW1BQ0N69e6lU6unTp8PCwtatW8e+ ORh8YLsXZ2fnwd/uQ70JsKG5uZlDvMGpqan9+++/SUlJ2OX+5s2bO9gddqij8Qaex8fhnVgT1NXV HTlyZMyYMT8MPQqStA/YunVrYGDgmzdvrl27du7cubi4OBKJ9PjxYwcHh4sXL3Jzc48YMSInJ8fZ 2bmqqio4OBiPpaWnp3d5VEJCgjWyFHMoPSYm5u7duwwGw8rKysLCwsPDo7S0lEqlHj9+HCHU5eBN aWmpp6cnPz//hAkTsMtS4GcpKSkRFRWVlpZGCBkbG9fX16enp0+dOpWpTfFoJULo8OHDeOPatWvT pk3j4uKyt7e/f/++p6enp6dnXFzcokWLVqxYUVtb+/Llyx07dlRUVGCr5dTU1KVLl4aFhc2ZMych IeGPP/64f/9+VVWViIjIqVOnsL60s7PDUqmkpERMTAzfD0Lo+fPnGhoaVCq1urpaVFS0vb09JCQE R/PKysrCbhaqq6sRQmxi3tLp9NTU1NjYWG5u7n/++YdpG5CVlTV58uSCggIFBQVcDsvKymbOnIkQ ys/Px4HKelCYfu8QPz8/qyBua2vrPL/Uwd1vd45ycXEpKysXFRUJCAgwGIyioiJVVdXOt4E9AcvI yOBR886MGTOGm5tbSkoKj4VLSUl1SFlVVcXqM/jLly/y8vI8PDwqKirM9zh0tGlubu5PhYQAbQoM QjjNTfHMmTMJBMLNmzfNzc0PHz48e/Zs1vgrnaFSqQ0NDR32xMTEpKWlnTlzZuTIkZygSof4ulrc 9lRVVcXExHz+/LmtrS0qKmrr1q3Hjh0LCgrCgaCuXr2KEAoODvb19b1///769esRQlOnTu3yKIVC YY0sxTQM8PX1vXv3rqSk5NOnT8lksrq6uoeHR1RU1MWLF9euXfvkyRMCgeDv748HbywtLUtKSlxd Xd3d3ceNG2djYzN37lw8Rwn8FB1mxrds2eLn53fz5s0ujzJ3TpkyBW8HBwfr6uqampqamZktWbKk tLRUVVV18uTJV65cWbJkCXOAExuwHj9+fMuWLWlpaXZ2dubm5kpKSoaGhvn5+axqODMzs7KyEqen UCg6Ojr+/v58fHwGBgZmZmbS0tKtra1aWlotLS3YjzJC6OzZswiho0ePfu8ZeXh4lixZMn/+fDEx sUWLFkVGRv733394QDEpKSktLQ33fxBCqamp/v7+NBpt3LhxVCq1Z2dmv1eHTJ482dnZuby8XEFB ASF08eJFCwsLVr/CqJO738bGRjZHmVeeP3/+0aNHjYyMaDSat7e3mZlZz6pqjJSUFKvPYBkZGbza rLm5ufcMZ3vjQQbWD/VvkwTaFBho3Sle3tWrVy9btgybQHl4eDDXVbCnvb09JycnICDAzc1t7969 bFoFGCvtY+bPn5+QkPD582crK6vU1NQvX750HlJijr50CEvb+Whubi5rZClmGnd391WrVgkJCe3e vZvBYGDRUFFRgd0zdTl48+7dO19fX4QQiUT6+vUraNNfwNLSErspxTg7O7POSHZeKIMQOnPmDKsu KSoqwtsfPnwgkUiSkpIMBgO7e8RiaO/evTjBtGnTsHXHpUuX6uvrRUVFeXl5N27cyOrDwdLSsssi FBsbW1NTw3TUgBBiup9jqtKEhAS8sXv3bryhqan56tUrhNDNmzdra2vFxMT4+PjWrFnDzc3t4+Oz a9cuTU1NHh4efANPnjzR19eXkZHpjYafzQVFRETCwsLs7e3l5eUbGxtHjx69adOm9vb2L1++uLi4 4Lqxg7vfZcuWsTlqY2ODr2xqarphwwYPD4+2trZt27adOnUKjzGLiooSicTQ0NDffCg1NTU3NzdD Q8MOPoOnTp3q4OCAHQhyVGkfNEMM/R7CALQpMCAREREJCAgoLCzcsGGDlJTUjh07lJWV2aQvLS09 cuTI2rVr2YQeBVXaX5iamh46dEhERMTY2NjHxwcHGv1lOkSWYu5XUlJ6+PDht2/flixZsn///mHD hvn4+Fy5cuX9+/ffq4vl5eVPnDghIyNTVlYmJyfXX/ljb2+P56OHOHx8fHhFPxcXF5vQcRjm2v8f pmSCVeMvw5xfZi11goKCeINEInl7e3cIN9BnTJ8+PS4u7tq1a69evWLGg0hJSWlubmb6a/vnn39Y 3f2yP4qRlZVlWguQSCSc7fgZs7OzEULMWQs8ucEU9DgN/vvvv/8ihNatW8eaktk7ffXqlaCg4OzZ s1l9BgcFBbW0tLAx6wIG/CAUZAHww84Tx/YF1dXVY2NjX758aWVltW7dug0bNnRuhxoaGiIjI7m4 uO7evcsm9Cio0n5EWVk5JyfHxcVFQ0MjPj4eT6FicCCoadOmdXlil0c7RJYaP3483u/n50ehUAQE BGbPnj1hwoSjR4+2tbVJSkrGxcUtXbq0y+u7ubmtWbNm5MiReLUytIVA97lw4QJrrUImk69evaqq qtqPt7R69eqSkhIHB4eQkBABAQFubm5WR8IIoQ4Ofdkc/SHsra26D/NH+fj4WKt3+BgHufYA/zXA wNWmTGg0WlBQ0LFjx9zd3RctWiQrKysgIJCenh4VFXX58mU2oUdBkv5m8ejtstE5EFQ3j7JGlmLS 3NzMw8ODh14oFAqVShUREekyoBQTBoPR3NyMYwT0V6b1TXCa3osLBfzsW/6duFBD/H31Y1woNolJ JNLt27eZY8P3799XUVHp0hotIiIiJCTk2LFj+/fvj4+P/4UHoVAopqamXZ67Zs0ad3d39gstfv/B IS4UAPx/eHl5t2/f/vnz58TExAULFuBZ2srKylevXr148aJLYQpuoQYEnQNBdfNoZ2GKEBIVFWVa dPDz8+OlfuznfLm4uHpQmAKsnDt37tatW533p6SkMB2gMsnNzd26dSuJRGKdHb5//35WVlaXF4+I iJgzZ86rV69mz579wzshkUh2dnZNTU319fWBgYGsnjj/+++/wMDA3Nzc7jxRXV1d58TBwcGGhoaH Dx9OTk5GCBUVFTk6OnKOduxmBCb2PHnyhPW9/BS/b5bK+dTV1WFXZbg8nD9/fty4cV2m9Pf39/b2 1tHR+eWIG/z8/F1+Vgihjx8/9p4Lsx6GAQDfh0qlDrh7/vz5s5aWlq2t7bVr16ysrD7/X+Cd9iBQ gfRNpvVNPndoEb633YM0NTWRSKTO+x89erR8+XLWPTQaTU9Pr7CwsLS0VElJCe8MCgoyNTVtbW3t 8uIGBgYpKSlkMllGRuaHd7Jp06bw8HAGgxETE8PDw4MQqqmpwX6RcGcmLi6uO0+El+c/f/6cdefi xYv19fVramq0tbXJZDKDwTh8+HBgYCCbt9xh+2fzv5vp3dzcGAxGeXl5eXn5b77KiRMnEonELg+V lpZevHiRzblv377lhEqpyxN/6mpsEjPL7eXLl+fNm9fS0oL3x8XFLVmyxNTU1M3Nrb29PTg4WFJS cu7cuampqbjcUiiUTZs25eXlkclkFxcXc3PzJUuWZGZmsl68oKDA3t7ezMzMwcGhoqKCQqHY2dkx GIzdu3dHRUVZWFjMnz8/IyODwWDo6+vn5OQwGAw/P787d+70VDZ2SPy9f3+qJEPTAgxO7t27Jy8v z6pNOVmsDFCR9zu33drampmZWVpaiv8tLi7OzMxsa2vrvbvFv8j6E5GRkdHR0cx/Q0ND4+PjQ0JC Op/bYWdQUBBo01/g3Llz5ubm5ubm169fZzAYV69ejYmJIRAIp06d2r179+zZs/fu3cuqTUtLSx0d HZuamm7duuXg4NBLbTyRSBw7diydTmcwGJ6enpqamqKiollZWQwGw8nJCc+61NXVNTc379q1S19f 387OrqCggMFg5Obmrl271tDQcOnSpTg9DgZRV1dXXl6+fv16Q0PDS5cuKSkpbdq0icFg7N+///z5 8wwGo66uTkNDA7te7i9tmpaWpqio6OHhERMTExMT8+DBg6NHj27YsGHjxo1hYWE2Njb4VktKSpyc nDZv3oz/ZXLmzJmNGzc6ODh8+vTpn3/+kZWV9fPzw4dIJJLz/yCRSOvXr9fX109LS3v27JmDg8O6 desePnzIYDB27ty5cePGhIQEZ2dnBoOxY8cOLy8va2tr/K2Fh4evWbPGz89v69atg0abhoeHT58+ nbU/tmDBgpKSEhqNNnfu3Pv37zMYjMmTJ+NSKiMj097ebm9vj2ubyMjIw4cPMxiM+Pj4yZMns17c 2tr69u3bDAYjOjr65s2bzP6YkZHRzp076XT6v//+u2DBAqY2DQkJsbW1xQUetCkA9DVUKrW5uXlA KLwhqE0fP37Mzc1tZWXFYDDIZLKKigoPDw9TavQGjx8/7vATM2bMOHHiBN5+9+6dhYVFXV1dl6M4 +vr6rP92aBtAm3aH3NxcHR0dCoXS3Ny8efNmMpm8d+/egICA58+fDxs2LD8/v62tbdKkSWlpaVib 1tbWTp8+/fXr1wwGY/Xq1ZGRkb3UxoeFhW3fvh1fZ8mSJTimEfa2KyUl5ezsrK6uzmAwZs+ePWXK lKysrJUrV86aNYvBYGhpadnY2CQnJ69cuXLjxo14iFRDQ4PBYMyaNWvMmDFPnz7FAVGx3kpOTp4/ fz7+IRMTk5SUlP4dN8WPf/HixYsXL165cmXbtm34EaKiouh0up6eHoPBWLlyZW5uLt7Iz89njnSu WLGCwWDk5eVZWlp2+CKeP3++Y8cOBoNBIBC+ffsWFRXl5eVFp9N1dHTa2tqoVOqkSZPodLqmpiYe MsAfl4aGBn6DuJBMmjSJSqW+f/9eUVFxcGhTfn7+JUuWaGpq4vzElJeXX7p0ycfHx9DQEA+ls5Zb FxeXAwcO4JTt7e2xsbG+vr5ubm7Dhg1jvfiJEydmzJhx+fLlsrIyXJ1ibWpsbJyUlMRgMD5+/Kij o4Oz2tPTc86cOT+cEe13bQr2psAPbJz73c/Z7xihclrsAIBJZmbm+PHjcYDZ0NBQSUnJcePGcXFx 7dmzx8DAYP369SQSiUql/vXXX1OnTjUxMcnMzOzwLx7lmjp16uzZs9++fYsQ8vDwmDx5sre3t6Wl JR6UtbOzmzFjho+PD/7FcePGCQkJHTt2bMqUKZ6enm/fvmUGpQwMDCwsLExKSgoJCUEIvXjxwsHB YcOGDYmJicx7vnHjxrp167y8vIZUFO+eYvjw4Y2NjYcOHSIQCAEBAUznSlxcXDo6OmPGjMHxt759 +4YQIpPJixYt8vT0xE4YPn36xFzAUVVVFRERUV1dXVxczLz45cuXnz59eurUqdbWVtb9mF27do0Y MQL7VV2yZImxsbGfn198fDxOyXpxHENVRUWltLTUy8vrzz//JBKJurq6b968iY+PV1BQiIqKIpFI 7969QwgJCQm9evXq6dOnBw8exMUGn/7+/fuXL18eOnTIxMTEwsICIYSL2ejRoz9+/Ih/iHWbQ8AW kFJSUmPHjuXm5sZ22NjFr5OTE3bxi1Pm5eXhdTxjx47tEIkUIWRoaMhgMObNmxcdHc10y1VVVSUr K4tdUMnKylZVVYmJibFa/EtKSuIOKg8PT1VVlYKCAi8v7/jx49nH7RtASEtLR0VFXbp0acmSJVVV VdhuxMDAQFBQcN68ebj/w5q+tra2rKwsISEBhxnz8vIKCQnBtV+HlK6urv7+/l++fJk5c+aVK1dY D3WIrIsQSk1NLSkpwV8ZR680gBoTAIC+h0AgmJubl5aWksnk8+fPGxgY6OrqHjhwICsrKz4+nkgk /v333wkJCdevX4+Li1u/fv3Xr187/FtbW9va2hoXFycgIBAeHn7v3r2zZ8/evXtXQEDg3r17Kioq S5YsGT169NOnT0+ePJmbm0sgEHR1de/du3fy5Mlbt24JCwu3tLQwtamjo+PMmTMtLCwIBAKDwWCN 342rdRqN1l9RvBkMxsDtIjKRkJDAHZKzZ89OnjwZu8NkbUFZG9GXL1/KyMg8efIE76dQKEwt2+Nt PPPiNTU1JSUlenp6ysrKKSkp0dHRu3fvxnLzw4cPWIrx8vLiwSeEUGRk5LJly27fvj1p0qRHjx5V V1cTiUQ9Pb3U1FSEEA5t9enTJx4eHhwbQkhIiLkShXWbc0YiOu/ELn4vXbp04cIFHMoVIaSgoIDF TXV1dWfv901NTR4eHk+fPqVSqbdv38Y7paSksFt+/EakpaW5ubm/9+sSEhK1tbX4jXQI7DdwwXbM xsbGLi4ulpaWra2tOTk5ampq9vb22tra79+/Z4a6ZZbzmzdv6uvr79u3Dxsxb9++fdasWeXl5R1S Xr9+XUdHx8vL6+rVq8wM/x4nT548ffq0jY1Nh4uANgUAoKPyYAw9hy8EAsHY2JhMJgcEBFhZWWFN EBkZmZ+fr6+v/+nTJzzapKampqSk9N9//xkZGXX49+XLly9evLCzs0tOTtbU1Pzvv//mzp2LE6iq qjY3NxMIhFu3bs2aNUtYWJhCoWCRcefOHTMzs5EjRyooKCgrK3cZV4YZv9vZ2VlISKixsRHvHLJR vHuEgoKCrKystWvX3r17V0ZGBjvT+B7z58+/c+dOcnJyTEwMQkhOTo4ZQarH23jmxQkEAkJo8uTJ KioqN27c2LhxY3t7+9evX3V1dfHo3aJFizZv3vzhwwdjY+OEhIRLly6dPHmSQCBISUm9e/cOn66n p4dVF5FIrKmpCQsLGz16NF5QVVZWxnSxzLrdX2D3wOzTYBe/27dvd3Z2bmtrwzuNjIzq6ur++usv R0fHw4cPdziFSCSuWLHC1dU1Kytr2rRpampqERERycnJdnZ22N7U1taWvXMMERGR6dOnr1mz5uTJ k78ZEIED2bRpk66u7tq1a3V1dcvLy62trW1tbVevXh0aGpqfn98h8YkTJ5KSkmJiYqytrbds2WJt bU0kEpWUlDw8PFh7X0ZGRmvXrt2zZ8+WLVt+eAOWlpbMz4HT20UAGGSmkAAn25vW19dzcXF9+/Zt 4sSJKioq1dXV8vLyiYmJcnJy165da2xsvHXrFp1Of/XqVVlZ2efPn9XU1G7dutXhXxxi+9OnT9zc 3BkZGZaWltbW1jQazdTU1NLSEvvxKSkpwROs+BcTExPnz5+/bt06BoOxdOlSCwsL5i2lp6djk0F9 fX0KhTJlyhS8v6SkBO8kkUh4Z1NTk7Kych9nWh98hh0+dtTT9qYVFRUGBgYrVqxYunTpkiVL2tra sL1pfHz8woULcRp7e/uoqCjmWqjCwsKRI0cWFxd7eXkdP36cdS0UZvPmzdbW1vX19ePHj1+xYsWy Zcv8/PzGjh2bl5fHareH10Lp6+vfv3///PnzWlpaK1asOHPmzPjx493d3V++fGlmZsZgMI4fPy4n J8dgMEJDQ0VFRauqqh4/fox1JJ1OX7lypYCAgLCwsImJSWNjY0FBwbhx4yQlJSUkJAwMDCoqKo4f P44QqqysrKys1NDQ4OfnNzY2HjFixNKlS/HdXrlyBRtiYvPK6urq/rU3pdPpXfpJ6EB7e3tjY2Pn /c3Nze3t7V2eQqPRampqmEdJJBJeedPW1tbN9Y5RUVFtbW1NTU0zZ84cBPambPK2qqoKZ9QPc6ap qQmvoGhvb+9gMNrW1lZZWcl+eVPvZSOCtVAAMCgZiL66fqcZSEhIkJeXZzAY5ubmBw8eJBKJXFxc DQ0N/v7+SkpKmpqaGzZsYDAYJ0+eVFNTMzAwmDRpUkVFRYd/ly5dKi8vv3XrVjU1tV27dkVGRgoJ CRkYGIwbN+7IkSPt7e0rV64cPXq0srJyWFhYQkIC/omwsDBhYeH58+dramq6urp2qU0ZDIafn5+d nd369etdXFyYO52dndetW7d9+3a8sKCPM6235Wlva1NMTU1NU1PTz55FJBInTpzYYWF7T7Xx7e3t enp63XHlQSKROqi07z0Ovp8OOw0NDfE6mMePH9va2rJ5v32jTTmZf/75Z9myZStXrkxMTBzE2nRw 1O29oU0h2gcAcISZ10D8Ensj/AyFQqHRaEyP+m1tba2trRISEl3+i4M/YSlw4cKFqVOnKioqjhkz Ji4uTl9fHyFEIpH4+fk7TCCSSCRBQUE8NcwG1vjdTH4/infPxq3pvXLIaXGhTp8+TaPRemkiMiUl 5cSJE9HR0b1n13vlypW8vLyTJ0+SSCTsRatD/FKIC9W/lVKvxoUa3NnYG3GhQJsCAGhTDtKmv4yf n9/ly5eFhYXXrFmzdevWIdJ2Dh1tihBKT0+fOnVqL138/fv3ampqvefZg0AgaGtr8/LyVldXNzY2 dg5K14/aNDY2dtSoUSNGjOjyaGtra0ZGhpyc3KhRo9hcpLa29vnz51ZWVt87Ghsba21t/VOZ9vz5 czU1tZ8K4AfaFLQpALIJAG0KZaPvMq1XM5zztenQKRh9qU2Li4v37t3777//MvfQaLSWlhYc9ZdM JpuamlpZWSUnJ5uYmDg5OX3vOq2trSUlJaNHj+7yaEFBwaFDh1h/pTs0NDTY2to+ePCg++PZoE0H hzaFdfoA0M+w2uIAAAD0JWfPnt22bRverqys9Pb2tre3Lysrw3taWlqOHTu2fft2d3f3uLg4ppw9 deoUQigsLCwsLAwh5Ovrm52dffv27W/fvu3fv3/nzp2LFy/GvrT27t27bdu2v//+m/lzmzZtWr9+ fUFBwZEjR6qqvOzbEAAACdJJREFUqtra2kxNTSkUSk1NjaenZ0xMDF7OHxMTIyEhoa+v/+DBg8GR 1VeuXAkMDGT6Fn3z5k1gYOCzZ8+6efrly5c7JE5PT9fR0cHrPgcZoE0BAAAGTDdmEDg6BTiK1NTU 6dOnUygUJyenw4cPm5ub37p1a8yYMfiojIwMXinv7++PY1whhFRVVR89eoQQSkpKSklJQQg9fvxY SkoqLy+PQqH8999/p0+f9vDwCA0NTUtLKy8vDwgIwI79s7OzU1JSQkJCXF1dd+/ePWzYsMTExNTU VCqV+urVq/j4eDk5OV9fX19f38uXL/Py8iKETExMuq/eOJnm5mZHR8edO3e+evUKIdTe3r5mzZqd O3e+fPmyO6c3NDQ4OTklJSWx7kxKSsIO1L53lrm5OQ7LDNoUGFTg1ayQDwAAAIMSGo3Gw8PDYDAa GxvFxMQkJSU7JGhra3NwcDAyMlqwYAHew8XFpaysXFRUJCAgwMvLW1RUpKqqyuw1jRs3jpeXV1pa urm5+fPnz1iVTpo0CXUKKGVmZhYfHx8XF3f48OHnz5/HxcUtWLDA3d191apVK1aswEseFRUVmYO4 Axocyk5LSwsHw7t58ybOdhwbNigoaNasWQsXLoyNjcVKdNeuXX/88Qd29Ms8XU9Pj0qlenh4zJw5 08XFJS0tbfTo0WxWZ2ZmZi5dulRPTy8qKmpgNeWgTQF2QMzPvoE1Rg4AsAGGToHeQEBAICIi4s8/ /wwMDFy7dm1RURHzkKOjo4ODg729PWv6+fPnHz161MDAQF9f39vb28zMjHmItXxKS0vjuAZEIhF1 Cig1cuRIIpH45cuX2bNnFxQUVFRUqKqqKikpPXz4MCAgYMeOHXh8RFRUdBDkMIFAEBcXnzp1KpFI pNPpnp6e5ubm2H+Zl5fXwYMHcaT7JUuWNDY2Hj58+PHjx97e3tOmTfP19W1ra8vMzEQI4cQnT57c unWrgoLC3bt3sdZnT1ZW1rJly3R1dSMjIweKQuWFbxIA+hdYcAb8QpmBAgP0CGpqasXFxXiR/ujR o/38/EgkUktLCz768uXLhISE1tbWgICAkSNHnjhxAu83NTXdsGGDh4dHW1vbtm3bTp06xQxJysof f/xx7NgxFxcXHFPKyMjowoULf/311+fPn3FAqREjRlCpVCyOcaAsPz8/CoUiICAwe/Zs1MvOGfpY m+rq6qqqqr579+769evS0tIKCgoyMjIqKiqnTp1SU1NLTEysqakhkUhFRUVCQkLFxcURERELFiw4 evQoPl1WVlZJSSk0NNTOzs7GxqaysnLfvn3MqMsIodTU1NevXyOErK2thw8f3uEG3r59u3z58nnz 5t27d6+DazxOrOKgggMA0KagkAZWpvVGzsM6fc4pGH25Tv/Zs2cvXrzw8vLqvUfr4BWYRCIJCwuz Gf5vbm7m4eHB+snMzOzmzZs4Zmyvfhq9vU5fS0vLxMREW1s7LCysoqLi3Llz169fr6iouHr1qrKy 8vLly5kjoNu2bRMQEAgICLh//35ycrKrq6uPj8/EiROVlJTOnz8/atSo4ODgTZs2JScnz5w588mT J/Pnz8cn7t+/PyoqCiH0/PlzrE0VFBQqKirwUTk5uT179mzevLk7HpphnT7A6dUlTCACAKcBM/tA T2FiYkKlUouLi3vvJzqIIREREfalV1RUFAvTyMhIBweH7gtTjqW1tTU/P3/y5MkqKiopKSkyMjJm ZmZ4JFVcXJyHh0dbW3v//v01NTUTJkyoq6s7ePDgunXrXrx4YW5unp2dTSaT8/Pz9fT0uLm5EUJE IpFGo/n5+SGEWOf0fXx88vPz8/PzOwyaysnJnTp1qrCwcPfu3b8ZOqTvtAf0hoHuD6UAQI8MUUCm cWD+w7gp5xQMiAvV9x9Fr46bvn79evr06bm5uTw8PGPGjHn06JGRkZGYmNiNGzdsbGzOnDlz4MAB AQEBWVnZqKgoTU1Na2vr2NhYMTExISGhGzdu8PLyTp8+PSIiYsWKFYsXL46JidHU1FRUVMzJyamt rWVzS5MmTVq9enU3x0p76sE7dzzA9z4A2hQAbQraFLQpaFPQphykTX8IjUZrbGyUlpZm7mlubqZQ KKx7mNTW1kpISPww6jJCiEKh8PPz93Y2sp/T/zU5ATUO8IMPBjuZY+0Mdag0v7ezy44U+52/c8HO 5/bLj/7muQOx6wL8LD0rKHtQ6YI2HTTaFL6vgaVNB7TE7w1tCuv0Abblg5eXvWBlA4lE6uyCqpvn dv+CP6uw++xHeyMHBrHGAn75FYBwBODDBAab9oAsAH65vuuyBvydnT17br/86G+eCwAgTwEAAG0K AAAAgDwFAAD4FXrcjAS0KQAAAMjTvm57AAAYTPUPaFMAAACga3naG+0EjMgCANCXgO99AACAwSNP EYxxAgAA2hQAAADgHHkKUaMAABjQwJw+AADAIFSov29+CgIXAOCTAW0KAAAA9KQ8Rb9qLQo2pgDA aZ/M0HHHAdoUAABgMDeW4F4KAH5HDsL9gDYFAAAAelih9tL6fQAYCr07jhKmMG4KAAAADJ4mFgZQ AYC98oObBG0KAAAA9KlCZTZs3xOpsJgDGJpoa2tnZ2dDPvQqR44cAW0KAAAAdJSnWIB2HkM9cuSI u7s7ZNGAbtGBX+bt27ecfHtDbdIDpngAAACGImCECgAD5VMdat8paFMAAABQqNAQAAAIU9CmAAAA AChUAABAmII2BQAAAL6nUEGkAgAIU9CmAAAAAGcpVGgaAAC+QdCmAAAAAGe1jiBSAQBUKWhTAAAA ABpLAIAPDbQpAAAAALBtOKHtBABQpaBNAQAAAGhHAQC+JtCmAAAAAMC2TYVmFQDg8wFtCgAAAEAr CwDwsYA2BQAAAIButLvQ9AIAqFLQpgAAAAC0xAAAvTXQpgAAAAAAzTMAQMcMtCkAAAAAbTYAQB8M tCkAAAAA9FhDDm05AHoUAG0KAAAAQBsPAFBWQZsCAAAAQLfbfmj+ASiQoE0BAAAAAJQBAEUOihxo UwAAAAAA3QBAoQJAmwIAAACDQ1WAsACgzIA2BQAAAACOVh4gPqAwQGEAbQoAAAAAnK5RQKYM7pcL 7xe0KQAAAACApgHgfQGgTQEAAACgr2QQKCHIdgC0KQAAAAAMGAkFWgpyDABtCgAAAAADVZANCIn2 y88CohMAbQoAAAAAQ1TU9iqgEwDQpgAAAAAAAMAQhRuyAAAAAAAAAABtCgAAAAAAAACgTQEAAAAA AADQpgAAAAAAAAAA2hQAAAAAAAAAbQoAAAAAAAAAoE0BAAAAAAAA0KYAAAAAAAAAANoUAAAAAAAA AG0KAAAAAAAAAKBNAQAAAAAAgEHF/wO+Yc4JxEPrtAAAAABJRU5ErkJggg== --001a114a889ca777a50525847b3f--