From 1aa363cea7d11db387740cfd493cd609f6c9a189 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 5 Jan 2016 21:27:52 -0800 Subject: [PATCH] new quickstart --- distribution/pom.xml | 2 +- distribution/src/assembly/assembly.xml | 119 ++++-- docs/content/configuration/index.md | 2 +- .../configuration/production-cluster.md | 35 +- docs/content/configuration/realtime.md | 7 + docs/content/configuration/simple-cluster.md | 115 ------ docs/content/design/coordinator.md | 2 +- docs/content/design/design.md | 20 +- docs/content/design/indexing-service.md | 6 +- docs/content/design/realtime.md | 2 + docs/content/design/segments.md | 23 +- docs/content/development/build.md | 15 +- docs/content/development/libraries.md | 24 +- docs/content/ingestion/batch-ingestion.md | 309 ++++++---------- .../ingestion/command-line-hadoop-indexer.md | 67 ++++ docs/content/ingestion/faq.md | 21 +- docs/content/ingestion/overview.md | 82 ----- docs/content/ingestion/stream-ingestion.md | 39 ++ .../{realtime-ingestion.md => stream-pull.md} | 79 +++- docs/content/ingestion/stream-push.md | 136 +++++++ docs/content/{misc => ingestion}/tasks.md | 165 +-------- .../content/ingestion/update-existing-data.md | 2 +- docs/content/misc/cluster-setup.md | 116 ------ docs/content/misc/evaluate.md | 98 ----- docs/content/operations/other-hadoop.md | 10 +- docs/content/operations/performance-faq.md | 4 + docs/content/operations/pull-deps.md | 4 +- docs/content/operations/recommendations.md | 4 - docs/content/operations/rule-configuration.md | 2 +- docs/content/querying/aggregations.md | 3 + docs/content/querying/caching.md | 23 ++ .../{operations => querying}/multitenancy.md | 0 ...-valued-dimensions.md => optimizations.md} | 4 + .../{development => querying}/select-query.md | 0 docs/content/toc.textile | 30 +- .../tutorials/booting-a-production-cluster.md | 10 - docs/content/tutorials/cluster.md | 340 ++++++++++++++++++ docs/content/tutorials/examples.md | 71 ---- docs/content/tutorials/firewall.md | 172 --------- docs/content/tutorials/index.md | 21 -- docs/content/tutorials/ingestion.md | 42 +++ docs/content/tutorials/quickstart.md | 242 +++++++++++++ .../tutorial-a-first-look-at-druid.md | 314 ---------------- .../tutorials/tutorial-all-about-queries.md | 207 ----------- docs/content/tutorials/tutorial-batch.md | 108 ++++++ docs/content/tutorials/tutorial-kafka.md | 192 ++++++++++ .../tutorials/tutorial-loading-batch-data.md | 337 ----------------- .../tutorial-loading-streaming-data.md | 153 -------- docs/content/tutorials/tutorial-streams.md | 134 +++++++ .../tutorials/tutorial-the-druid-cluster.md | 313 ---------------- examples/bin/examples/indexing/wikipedia.spec | 73 ---- .../bin/examples/indexing/wikipedia_data.json | 5 - .../indexing/wikipedia_hadoop_config.json | 82 ----- .../indexing/wikipedia_index_hadoop_task.json | 76 ---- .../indexing/wikipedia_index_task.json | 76 ---- .../indexing/wikipedia_realtime_task.json | 110 ------ examples/bin/examples/twitter/after.sh | 13 - examples/bin/examples/twitter/before.sh | 28 -- examples/bin/examples/twitter/query.body | 31 -- .../bin/examples/twitter/search_query.body | 21 -- examples/bin/examples/twitter/topN_query.body | 94 ----- .../examples/twitter/twitter_realtime.spec | 196 ---------- examples/bin/examples/wikipedia/query.body | 4 - .../wikipedia/wikipedia_realtime.spec | 110 ------ examples/bin/generate-example-metrics | 48 +++ examples/bin/init | 12 + examples/bin/run_druid_server.sh | 36 -- examples/bin/run_example_client.sh | 30 -- examples/bin/run_example_server.sh | 52 --- examples/bin/select_example.sh | 38 -- .../druid/_common/common.runtime.properties | 108 ++++++ .../conf-quickstart/druid/_common/log4j2.xml | 33 ++ .../conf-quickstart/druid/broker/jvm.config | 8 + .../druid/broker/runtime.properties | 16 + .../druid/coordinator/jvm.config | 8 + .../druid/coordinator/runtime.properties | 5 + .../druid/historical/jvm.config | 8 + .../druid/historical/runtime.properties | 13 + .../druid/middleManager/jvm.config | 7 + .../druid/middleManager/runtime.properties | 20 ++ .../conf-quickstart/druid/overlord/jvm.config | 7 + .../druid/overlord/runtime.properties | 7 + .../conf-quickstart/tranquility/kafka.json | 77 ++++ .../conf-quickstart/tranquility/server.json | 74 ++++ .../druid/_common/common.runtime.properties | 107 ++++++ examples/conf/druid/_common/log4j2.xml | 13 + examples/conf/druid/broker/jvm.config | 8 + examples/conf/druid/broker/runtime.properties | 16 + examples/conf/druid/coordinator/jvm.config | 8 + .../conf/druid/coordinator/runtime.properties | 5 + examples/conf/druid/historical/jvm.config | 8 + .../conf/druid/historical/runtime.properties | 13 + examples/conf/druid/middleManager/jvm.config | 7 + .../druid/middleManager/runtime.properties | 20 ++ examples/conf/druid/overlord/jvm.config | 7 + .../conf/druid/overlord/runtime.properties | 7 + examples/conf/tranquility/kafka.json | 77 ++++ examples/conf/tranquility/server.json | 74 ++++ .../config/_common/common.runtime.properties | 52 --- examples/config/_common/log4j2.xml | 30 -- examples/config/broker/runtime.properties | 29 -- .../config/coordinator/runtime.properties | 26 -- examples/config/historical/runtime.properties | 36 -- .../config/middleManager/runtime.properties | 28 -- examples/config/overlord/runtime.properties | 35 -- examples/config/realtime/runtime.properties | 36 -- examples/config/router/runtime.properties | 25 -- examples/pom.xml | 1 - .../wikiticker-2015-09-12-sampled.json.gz | Bin 0 -> 2366222 bytes examples/quickstart/wikiticker-index.json | 85 +++++ examples/quickstart/wikiticker-top-pages.json | 16 + .../java/io/druid/guice/ExtensionsConfig.java | 2 +- .../java/io/druid/cli/PullDependencies.java | 2 +- 113 files changed, 2658 insertions(+), 3867 deletions(-) delete mode 100644 docs/content/configuration/simple-cluster.md create mode 100644 docs/content/ingestion/command-line-hadoop-indexer.md delete mode 100644 docs/content/ingestion/overview.md create mode 100644 docs/content/ingestion/stream-ingestion.md rename docs/content/ingestion/{realtime-ingestion.md => stream-pull.md} (77%) create mode 100644 docs/content/ingestion/stream-push.md rename docs/content/{misc => ingestion}/tasks.md (57%) delete mode 100644 docs/content/misc/cluster-setup.md delete mode 100644 docs/content/misc/evaluate.md create mode 100644 docs/content/querying/caching.md rename docs/content/{operations => querying}/multitenancy.md (100%) rename docs/content/querying/{multi-valued-dimensions.md => optimizations.md} (97%) rename docs/content/{development => querying}/select-query.md (100%) delete mode 100644 docs/content/tutorials/booting-a-production-cluster.md create mode 100644 docs/content/tutorials/cluster.md delete mode 100644 docs/content/tutorials/examples.md delete mode 100644 docs/content/tutorials/firewall.md delete mode 100644 docs/content/tutorials/index.md create mode 100644 docs/content/tutorials/ingestion.md create mode 100644 docs/content/tutorials/quickstart.md delete mode 100644 docs/content/tutorials/tutorial-a-first-look-at-druid.md delete mode 100644 docs/content/tutorials/tutorial-all-about-queries.md create mode 100644 docs/content/tutorials/tutorial-batch.md create mode 100644 docs/content/tutorials/tutorial-kafka.md delete mode 100644 docs/content/tutorials/tutorial-loading-batch-data.md delete mode 100644 docs/content/tutorials/tutorial-loading-streaming-data.md create mode 100644 docs/content/tutorials/tutorial-streams.md delete mode 100644 docs/content/tutorials/tutorial-the-druid-cluster.md delete mode 100644 examples/bin/examples/indexing/wikipedia.spec delete mode 100644 examples/bin/examples/indexing/wikipedia_data.json delete mode 100644 examples/bin/examples/indexing/wikipedia_hadoop_config.json delete mode 100644 examples/bin/examples/indexing/wikipedia_index_hadoop_task.json delete mode 100644 examples/bin/examples/indexing/wikipedia_index_task.json delete mode 100644 examples/bin/examples/indexing/wikipedia_realtime_task.json delete mode 100755 examples/bin/examples/twitter/after.sh delete mode 100755 examples/bin/examples/twitter/before.sh delete mode 100644 examples/bin/examples/twitter/query.body delete mode 100644 examples/bin/examples/twitter/search_query.body delete mode 100644 examples/bin/examples/twitter/topN_query.body delete mode 100644 examples/bin/examples/twitter/twitter_realtime.spec delete mode 100644 examples/bin/examples/wikipedia/query.body delete mode 100644 examples/bin/examples/wikipedia/wikipedia_realtime.spec create mode 100755 examples/bin/generate-example-metrics create mode 100644 examples/bin/init delete mode 100755 examples/bin/run_druid_server.sh delete mode 100755 examples/bin/run_example_client.sh delete mode 100755 examples/bin/run_example_server.sh delete mode 100755 examples/bin/select_example.sh create mode 100644 examples/conf-quickstart/druid/_common/common.runtime.properties create mode 100644 examples/conf-quickstart/druid/_common/log4j2.xml create mode 100644 examples/conf-quickstart/druid/broker/jvm.config create mode 100644 examples/conf-quickstart/druid/broker/runtime.properties create mode 100644 examples/conf-quickstart/druid/coordinator/jvm.config create mode 100644 examples/conf-quickstart/druid/coordinator/runtime.properties create mode 100644 examples/conf-quickstart/druid/historical/jvm.config create mode 100644 examples/conf-quickstart/druid/historical/runtime.properties create mode 100644 examples/conf-quickstart/druid/middleManager/jvm.config create mode 100644 examples/conf-quickstart/druid/middleManager/runtime.properties create mode 100644 examples/conf-quickstart/druid/overlord/jvm.config create mode 100644 examples/conf-quickstart/druid/overlord/runtime.properties create mode 100644 examples/conf-quickstart/tranquility/kafka.json create mode 100644 examples/conf-quickstart/tranquility/server.json create mode 100644 examples/conf/druid/_common/common.runtime.properties create mode 100644 examples/conf/druid/_common/log4j2.xml create mode 100644 examples/conf/druid/broker/jvm.config create mode 100644 examples/conf/druid/broker/runtime.properties create mode 100644 examples/conf/druid/coordinator/jvm.config create mode 100644 examples/conf/druid/coordinator/runtime.properties create mode 100644 examples/conf/druid/historical/jvm.config create mode 100644 examples/conf/druid/historical/runtime.properties create mode 100644 examples/conf/druid/middleManager/jvm.config create mode 100644 examples/conf/druid/middleManager/runtime.properties create mode 100644 examples/conf/druid/overlord/jvm.config create mode 100644 examples/conf/druid/overlord/runtime.properties create mode 100644 examples/conf/tranquility/kafka.json create mode 100644 examples/conf/tranquility/server.json delete mode 100644 examples/config/_common/common.runtime.properties delete mode 100644 examples/config/_common/log4j2.xml delete mode 100644 examples/config/broker/runtime.properties delete mode 100644 examples/config/coordinator/runtime.properties delete mode 100644 examples/config/historical/runtime.properties delete mode 100644 examples/config/middleManager/runtime.properties delete mode 100644 examples/config/overlord/runtime.properties delete mode 100644 examples/config/realtime/runtime.properties delete mode 100644 examples/config/router/runtime.properties create mode 100644 examples/quickstart/wikiticker-2015-09-12-sampled.json.gz create mode 100644 examples/quickstart/wikiticker-index.json create mode 100644 examples/quickstart/wikiticker-top-pages.json diff --git a/distribution/pom.xml b/distribution/pom.xml index 04e205158d1b..86138cc8fd3c 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -58,7 +58,7 @@ -Ddruid.extensions.loadList=[] -Ddruid.extensions.directory=${project.build.directory}/extensions - -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop_dependencies + -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies io.druid.cli.Main tools pull-deps diff --git a/distribution/src/assembly/assembly.xml b/distribution/src/assembly/assembly.xml index 769150ac32b8..9e7853a8aefd 100644 --- a/distribution/src/assembly/assembly.xml +++ b/distribution/src/assembly/assembly.xml @@ -34,85 +34,152 @@ extensions + - ${project.build.directory}/hadoop_dependencies + ${project.build.directory}/hadoop-dependencies */*/* - hadoop_dependencies + hadoop-dependencies + - ../examples/config + ../examples/quickstart/ * - config + quickstart + - ../examples/config/_common + ../examples/conf-quickstart * - config/_common + conf-quickstart - ../examples/config/broker + ../examples/conf-quickstart/druid * - config/broker + conf-quickstart/druid - ../examples/config/coordinator + ../examples/conf-quickstart/druid/_common * - config/coordinator + conf-quickstart/druid/_common/ - ../examples/config/realtime + ../examples/conf-quickstart/druid/broker * - config/realtime + conf-quickstart/druid/broker - ../examples/config/historical + ../examples/conf-quickstart/druid/coordinator * - config/historical + conf-quickstart/druid/coordinator - ../examples/config/overlord + ../examples/conf-quickstart/druid/historical * - config/overlord + conf-quickstart/druid/historical - ../examples/bin + ../examples/conf-quickstart/druid/overlord - *sh + * - 744 - / + conf-quickstart/druid/overlord + + + ../examples/conf-quickstart/druid/middleManager + + * + + conf-quickstart/druid/middleManager + + + ../examples/conf-quickstart/tranquility + + * + + conf-quickstart/tranquility + + + + ../examples/conf + + * + + conf + + + ../examples/conf/druid/_common + + * + + conf/druid/_common + + + ../examples/conf/druid/broker + + * + + conf/druid/broker + + + ../examples/conf/druid/coordinator + + * + + conf/druid/coordinator + + + ../examples/conf/druid/historical + + * + + conf/druid/historical - ../examples/bin/examples + ../examples/conf/druid/overlord - ** + * + + conf/druid/overlord + + + ../examples/conf/druid/middleManager + + * + + conf/druid/middleManager + + + ../examples/conf/tranquility + + * - examples + conf/tranquility - ../examples/bin/examples/twitter + ../examples/bin - *sh + * 744 - examples/twitter + bin + ../ diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 7a259870993e..e0c74506d387 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -22,7 +22,7 @@ Many of Druid's external dependencies can be plugged in as modules. Extensions c |Property|Description|Default| |--------|-----------|-------| |`druid.extensions.directory`|The root extension directory where user can put extensions related files. Druid will load extensions stored under this directory.|`extensions` (This is a relative path to Druid's working directory)| -|`druid.extensions.hadoopDependenciesDir`|The root hadoop dependencies directory where user can put hadoop related dependencies files. Druid will load the dependencies based on the hadoop coordinate specified in the hadoop index task.|`hadoop_dependencies` (This is a relative path to Druid's working directory| +|`druid.extensions.hadoopDependenciesDir`|The root hadoop dependencies directory where user can put hadoop related dependencies files. Druid will load the dependencies based on the hadoop coordinate specified in the hadoop index task.|`hadoop-dependencies` (This is a relative path to Druid's working directory| |`druid.extensions.loadList`|A JSON array of extensions to load from extension directories by Druid. If it is not specified, its value will be `null` and Druid will load all the extensions under `druid.extensions.directory`. If its value is empty list `[]`, then no extensions will be loaded at all.|null| |`druid.extensions.searchCurrentClassloader`|This is a boolean flag that determines if Druid will search the main classloader for extensions. It defaults to true but can be turned off if you have reason to not automatically add all modules on the classpath.|true| diff --git a/docs/content/configuration/production-cluster.md b/docs/content/configuration/production-cluster.md index 0981385bb3b0..4b659e241624 100644 --- a/docs/content/configuration/production-cluster.md +++ b/docs/content/configuration/production-cluster.md @@ -4,21 +4,40 @@ layout: doc_page Production Cluster Configuration ================================ -__This configuration is an example of what a production cluster could look like. Many other hardware combinations are possible! Cheaper hardware is absolutely possible.__ +```note-info +This configuration is an example of what a production cluster could look like. Many other hardware combinations are +possible! Cheaper hardware is absolutely possible. +``` -This production Druid cluster assumes that metadata storage and Zookeeper are already set up. The deep storage that is used for examples is S3 and memcached is used as a distributed cache. +This production Druid cluster assumes that metadata storage and Zookeeper are already set up. The deep storage that is +used for examples is [S3](https://aws.amazon.com/s3/) and [memcached](http://memcached.org/) is used for a distributed cache. -The nodes that respond to queries (Historical, Broker, and Middle manager nodes) will use as many cores as are available, depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is not well characterized yet and would depend on types of queries, query load, and the schema. Historical daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries. SSDs are highly recommended for Historical nodes not all data is loaded in available memory. +```note-info +The nodes in this example do not need to be on their own individual servers. Overlord and Coordinator nodes should be +co-located on the same hardware. +``` -The nodes that are responsible for coordination (Coordinator and Overlord nodes) require much less processing. +The nodes that respond to queries (Historical, Broker, and MiddleManager nodes) will use as many cores as are available, +depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is +not well characterized yet and would depend on types of queries, query load, and the schema. Historical daemons should +have a heap size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. +Since in-memory caching is essential for good performance, even more RAM is better. +Broker nodes will use RAM for caching, so they do more than just route queries. +SSDs are highly recommended for Historical nodes when all they have more segments loaded than available memory. -The effective utilization of cores by Zookeeper, metadata storage, and Coordinator nodes is likely to be between 1 and 2 for each process/daemon, so these could potentially share a machine with lots of cores. These daemons work with heap a size between 500MB and 1GB. +The nodes that are responsible for coordination (Coordinator and Overlord nodes) require much less processing. -We'll use r3.8xlarge nodes for query facing nodes and m1.xlarge nodes for coordination nodes. The following examples work relatively well in production, however, a more optimized tuning for the nodes we selected and more optimal hardware for a Druid cluster are both definitely possible. +The effective utilization of cores by Zookeeper, metadata storage, and Coordinator nodes is likely to be between 1 and 2 +for each process/daemon, so these could potentially share a machine with lots of cores. These daemons work with heap +size between 500MB and 1GB. -For general purposes of high availability, there should be at least 2 of every node type. +We'll use [EC2](https://aws.amazon.com/ec2/) r3.8xlarge nodes for query facing nodes and m1.xlarge nodes for coordination nodes. +The following examples work relatively well in production, however, a more optimized tuning for the nodes we selected and +more optimal hardware for a Druid cluster are both definitely possible. -To setup a local Druid cluster, see [Simple Cluster Configuration](../configuration/simple-cluster.html). +```note-caution +For high availability, there should be at least a redundant copy of every process running on separate hardware. +``` ### Common Configuration (common.runtime.properties) diff --git a/docs/content/configuration/realtime.md b/docs/content/configuration/realtime.md index 48b8623e3e48..45b7c19a9d69 100644 --- a/docs/content/configuration/realtime.md +++ b/docs/content/configuration/realtime.md @@ -1,6 +1,13 @@ + --- layout: doc_page --- + +```note-caution +If you are doing stream-pull based ingestion, we suggest using [stream-pushed](../ingestion/stream-push.html) based ingestion instead and not +using real-time nodes. +``` + Realtime Node Configuration ============================== For general Realtime Node information, see [here](../design/realtime.html). diff --git a/docs/content/configuration/simple-cluster.md b/docs/content/configuration/simple-cluster.md deleted file mode 100644 index 783d80d693c9..000000000000 --- a/docs/content/configuration/simple-cluster.md +++ /dev/null @@ -1,115 +0,0 @@ ---- -layout: doc_page ---- -Simple Cluster Configuration -=============================== - -This simple Druid cluster configuration can be used for initially experimenting with Druid on your local machine. For a more realistic production Druid cluster, see [Production Cluster Configuration](../configuration/production-cluster.html). - -### Common Configuration (common.runtime.properties) - -``` -# Extensions --Ddruid.extensions.loadList=["druid-kafka-eight"] - -# Zookeeper (defaults to localhost) - -# Metadata Storage (defaults to derby with no username and password) -``` - -### Overlord Node (Indexing Service) - -Run: - -``` -io.druid.cli.Main server overlord -``` - -Configuration: - -``` --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 --Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - --Ddruid.indexer.queue.startDelay=PT0M --Ddruid.indexer.runner.javaOpts=-server -Xmx1g --Ddruid.indexer.fork.property.druid.processing.numThreads=1 --Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000 -``` - -This runs the indexing service in local mode, and can support real-time ingestion tasks (with one processing thread for queries). - -### Coordinator Node - -Run: - -``` -io.druid.cli.Main server coordinator -``` - -Configuration: - -``` --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 --Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - -druid.coordinator.startDelay=PT70s -``` - -This simple coordinator assumes local deep storage. - -### Historical Node - -Run: - -``` -io.druid.cli.Main server historical -``` - -Configuration: - -``` --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 --Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - -druid.server.maxSize=10000000000 - -druid.processing.buffer.sizeBytes=100000000 -druid.processing.numThreads=1 - -druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}] -``` - -This historical node will be able to load 100 MB of data and be able to process 1 segment at a time. Deep storage is assumed to be local storage here. - -### Broker Node - -Run: - -``` -io.druid.cli.Main server broker -``` - -Configuration: - -``` --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 --Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - -druid.processing.buffer.sizeBytes=100000000 -druid.processing.numThreads=1 -``` - -This simple broker will run groupBys in a single thread. diff --git a/docs/content/design/coordinator.md b/docs/content/design/coordinator.md index a95907fa670b..fff3b7dcfed1 100644 --- a/docs/content/design/coordinator.md +++ b/docs/content/design/coordinator.md @@ -260,7 +260,7 @@ Disables a datasource. * `/druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myISO8601Interval}>` -Runs a [Kill task](../misc/tasks.html) for a given interval and datasource. +Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource. * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` diff --git a/docs/content/design/design.md b/docs/content/design/design.md index a7bbee8e967e..a54df280eabe 100644 --- a/docs/content/design/design.md +++ b/docs/content/design/design.md @@ -2,7 +2,8 @@ layout: doc_page --- -For a comprehensive look at the architecture of Druid, read the [White Paper](http://static.druid.io/docs/druid.pdf). +For a comprehensive look at the architecture of Druid, read the [White Paper](http://static.druid.io/docs/druid.pdf). Please note +that Druid is undergoing rapid development and the white paper may be out date. What is Druid? ============== @@ -19,7 +20,7 @@ Druid currently allows for single-table queries in a similar manner to [Dremel]( As far as a comparison of systems is concerned, Druid sits in between PowerDrill and Dremel on the spectrum of functionality. It implements almost everything Dremel offers (Dremel handles arbitrary nested data structures while Druid only allows for a single level of array-based nesting) and gets into some of the interesting data layout and compression methods from PowerDrill. -Druid is a good fit for products that require real-time data ingestion of a single, large data stream. Especially if you are targeting no-downtime operation and are building your product on top of a time-oriented summarization of the incoming data stream. Druid is probably not the right solution if you care more about query flexibility and raw data access than query speed and no-downtime operation. When talking about query speed it is important to clarify what "fast" means: with Druid it is entirely within the realm of possibility (we have done it) to achieve queries that run in less than a second across terabytes of data. +Druid is a good fit for products that require real-time data ingestion of a single, large data stream. Especially if you are targeting no-downtime operation and are building your product on top of a time-oriented summarization of the incoming data stream. When talking about query speed it is important to clarify what "fast" means: with Druid it is entirely within the realm of possibility (we have done it) to achieve queries that run in less than a second across trillions of rows of data. ### Architecture @@ -33,9 +34,10 @@ The node types that currently exist are: * [**Realtime**](../design/realtime.html) nodes ingest data in real time. They are in charge of listening to a stream of incoming data and making it available immediately inside the Druid system. Real-time nodes respond to query requests from Broker nodes, returning query results to those nodes. Aged data is pushed from Realtime nodes to deep storage. Realtime nodes monitor ZooKeeper to discover segments that they've pushed to deep storage have been loaded by Historicals—if so, they drop those segments. * [**Coordinator**](../design/coordinator.html) nodes monitor the grouping of historical nodes to ensure that data is available, replicated and in a generally "optimal" configuration. They do this by reading segment metadata information from metadata storage to determine what segments should be loaded in the cluster, using Zookeeper to determine what Historical nodes exist, and creating Zookeeper entries to tell Historical nodes to load and drop new segments. * [**Broker**](../design/broker.html) nodes receive queries from external clients and forward those queries to Realtime and Historical nodes. When Broker nodes receive results, they merge these results and return them to the caller. For knowing topology, Broker nodes use Zookeeper to determine what Realtime and Historical nodes exist. -* [**Indexer**](../design/indexing-service.html) nodes form a cluster of workers to load batch and real-time data into the system as well as allow for alterations to the data stored in the system (also known as the Indexing Service). +* [**Indexing Service**](../design/indexing-service.html) nodes form a cluster of workers to load batch and real-time data into the system as well as allow for alterations to the data stored in the system. +* [**Realtime**](../design/realtime.html) nodes also load real-time data into the system. They are simpler to set up than the indexing service, at the cost of several [limitations](../ingestion/stream-pull.html#limitations) for production use. -This separation allows each node to only care about what it is best at. By separating Historical and Realtime, we separate the memory concerns of listening on a real-time stream of data and processing it for entry into the system. By separating the Coordinator and Broker, we separate the needs for querying from the needs for maintaining "good" data distribution across the cluster. +This separation allows each node to only care about what it is best at. By separating Historical and Realtime processing, we separate the memory concerns of listening on a real-time stream of data and processing it for entry into the system. By separating the Coordinator and Broker, we separate the needs for querying from the needs for maintaining "good" data distribution across the cluster. The following diagram shows how queries and data flow through this architecture, and which nodes (and external dependencies, discussed below) are involved: @@ -61,10 +63,9 @@ Getting data into the Druid system requires an indexing process, as shown in the - Converted to columnar format - Indexed with bitmap indexes - Compressed using various algorithms - - LZF (switching to Snappy is on the roadmap, not yet implemented) - - Dictionary encoding w/ id storage minimization - - Bitmap compression - - RLE (on the roadmap, but not yet implemented) + - LZ4 for all columns + - Dictionary encoding w/ id storage minimization for String columns + - Bitmap compression for bitmap indexes The output of the indexing process is called a "segment". Segments are the fundamental structure to store data in Druid. Segments contain the various dimensions and metrics in a data set, stored in a column orientation, as well as the indexes for those columns. @@ -79,6 +80,7 @@ In order for a segment to exist inside of the cluster, an entry has to be added - **Historical** As discussed above, if a historical node dies, another historical node can take its place and there is no fear of data loss. - **Coordinator** Can be run in a hot fail-over configuration. If no coordinators are running, then changes to the data topology will stop happening (no new data and no data balancing decisions), but the system will continue to run. - **Broker** Can be run in parallel or in hot fail-over. +- **Indexing Service** Workers run with replicated ingestion tasks, coordination piece has hot fail-over. - **Realtime** Depending on the semantics of the delivery stream, multiple of these can be run in parallel processing the exact same stream. They periodically checkpoint to disk and eventually push out to deep storage. Steps are taken to be able to recover from process death, but loss of access to the local disk can result in data loss if this is the only method of adding data to the system. - **"deep storage" file system** If this is not available, new data will not be able to enter the cluster, but the cluster will continue operating as is. - **metadata storage** If this is not available, the Coordinator will be unable to find out about new segments in the system, but it will continue with its current view of the segments that should exist in the cluster. @@ -86,7 +88,7 @@ In order for a segment to exist inside of the cluster, an entry has to be added ### Query processing -A query first enters the Broker, where the Broker will match the query with the data segments that are known to exist. It will then pick a set of machines that are serving those segments and rewrite the query for each server to specify the segment(s) targetted. The Historical/Realtime nodes will take in the query, process them and return results. The Broker then takes the results and merges them together to get the final answer, which it returns. In this way, the broker can prune all of the data that doesn’t match a query before ever even looking at a single row of data. +A query first enters the Broker, where the Broker will match the query with the data segments that are known to exist. It will then pick a set of machines that are serving those segments and rewrite the query for each server to specify the segment(s) targetted. The Historical/Realtime processes will take in the query, process them and return results. The Broker then takes the results and merges them together to get the final answer, which it returns. In this way, the broker can prune all of the data that doesn’t match a query before ever even looking at a single row of data. For filters at a more granular level than what the Broker can prune based on, the indexing structures inside each segment allows the historical nodes to figure out which (if any) rows match the filter set before looking at any row of data. It can do all of the boolean algebra of the filter on the bitmap indices and never actually look directly at a row of data. diff --git a/docs/content/design/indexing-service.md b/docs/content/design/indexing-service.md index eac3083bfa06..3978c161ced5 100644 --- a/docs/content/design/indexing-service.md +++ b/docs/content/design/indexing-service.md @@ -3,9 +3,10 @@ layout: doc_page --- Indexing Service ================ + For Indexing Service Configuration, see [Indexing Service Configuration](../configuration/indexing-service.html). -The indexing service is a highly-available, distributed service that runs indexing related tasks. Indexing service [tasks](../misc/tasks.html) create (and sometimes destroy) Druid [segments](../design/segments.html). The indexing service has a master/slave like architecture. +The indexing service is a highly-available, distributed service that runs indexing related tasks. Indexing service [tasks](../ingestion/tasks.html) create (and sometimes destroy) Druid [segments](../design/segments.html). The indexing service has a master/slave like architecture. The indexing service is composed of three main components: a peon component that can run a single task, a [Middle Manager](../design/middlemanager.html) component that manages peons, and an overlord component that manages task distribution to middle managers. Overlords and middle managers may run on the same node or across multiple nodes while middle managers and [Peons](../design/peons.html) always run on the same node. @@ -86,7 +87,7 @@ See [Peon](../design/peons.html). Tasks ----- -See [Tasks](../misc/tasks.html). +See [Tasks](../ingestion/tasks.html). HTTP Endpoints -------------- @@ -96,4 +97,3 @@ HTTP Endpoints * `/status` Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node. - diff --git a/docs/content/design/realtime.md b/docs/content/design/realtime.md index 3f0f5903e74f..f1bac49b8204 100644 --- a/docs/content/design/realtime.md +++ b/docs/content/design/realtime.md @@ -1,8 +1,10 @@ --- layout: doc_page --- + Real-time Node ============== + For Real-time Node Configuration, see [Realtime Configuration](../configuration/realtime.html). For Real-time Ingestion, see [Realtime Ingestion](../ingestion/realtime-ingestion.html). diff --git a/docs/content/design/segments.md b/docs/content/design/segments.md index 4f271b92dfe5..7e03fd66d8cd 100644 --- a/docs/content/design/segments.md +++ b/docs/content/design/segments.md @@ -4,7 +4,6 @@ layout: doc_page Segments ======== - Druid stores its index in *segment files*, which are partitioned by time. In a basic setup, one segment file is created for each time interval, where the time inteval is configurable in the @@ -169,12 +168,18 @@ A ColumnDescriptor is essentially an object that allows us to use jackson’s po Sharding Data to Create Segments -------------------------------- -### Sharding Data by Dimension +### Sharding + +Multiple segments may exist for the same interval of time for the same datasource. These segments form a `block` for an interval. +Depending on the type of `shardSpec` that is used to shard the data, Druid queries may only complete if a `block` is complete. That is to say, if a block consists of 3 segments, such as: + +`sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_0` + +`sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_1` + +`sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_2` + +All 3 segments must be loaded before a query for the interval `2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z` completes. -If the cumulative total number of rows for the different values of a -given column exceed some configurable threshold, multiple segments -representing the same time interval for the same datasource may be -created. These segments will contain some partition number as part of -their identifier. Sharding by dimension reduces some of the the costs -associated with operations over high cardinality dimensions. For more -information on sharding, see the ingestion documentation. +The exception to this rule is with using linear shard specs. Linear shard specs do not force 'completeness' and queries can complete even if shards are not loaded in the system. +For example, if your real-time ingestion creates 3 segments that were sharded with linear shard spec, and only two of the segments were loaded in the system, queries would return results only for those 2 segments. diff --git a/docs/content/development/build.md b/docs/content/development/build.md index 62b2dbbf37ce..d1a8eecaf907 100644 --- a/docs/content/development/build.md +++ b/docs/content/development/build.md @@ -4,7 +4,8 @@ layout: doc_page ### Build from Source -Druid can be set up by building from source via git. +You can build Druid directly from source. Please note that these instructions are for building the latest stable of Druid. +For building the latest code in master, follow the instructions [here](https://github.com/druid-io/druid/blob/master/docs/content/development/build.md). Building Druid requires the following: - [JDK 7](http://www.oracle.com/technetwork/java/javase/downloads/jdk7-downloads-1880260.html) @@ -22,10 +23,8 @@ mvn clean install This will compile the project and create the Druid binary distribution tar under `distribution/target/druid-VERSION-bin.tar.gz`. -This will also create `distribution/target/mysql-metadata-storage-bin.tar.gz`, -which is a tarball that contains the `mysql-metadata-storage` extension. - -You can find the example executables in the examples/bin directory: - -* run_example_server.sh -* run_example_client.sh +This will also create a tarball that contains `mysql-metadata-storage` extension under +`distribution/target/mysql-metadata-storage-bin.tar.gz`. If you want Druid to load `mysql-metadata-storage`, you can +first untar `druid-VERSION-bin.tar.gz`, then go to ```druid-/extensions```, untar `mysql-metadata-storage-bin.tar.gz` +there. Now just specifiy `mysql-metadata-storage` in `druid.extensions.loadList` so that Druid will pick it up. +See [Including Extensions](../operations/including-extensions.html) for more information. diff --git a/docs/content/development/libraries.md b/docs/content/development/libraries.md index 3f4e9b4c5838..7dcecd666f21 100644 --- a/docs/content/development/libraries.md +++ b/docs/content/development/libraries.md @@ -2,6 +2,9 @@ layout: doc_page --- +Query Libraries +--------------- + #### Python * [druid-io/pydruid](https://github.com/druid-io/pydruid) - A python client for Druid @@ -18,7 +21,7 @@ Some great folks have written their own libraries to interact with Druid #### JavaScript -* [facetjs/facetjs](https://github.com/facetjs/facetjs) - A general query planner for Druid written in JavaScript +* [implydata/plywood](https://github.com/implydata/plywood) - A higher level API for Druid. An extension of the work that was started in facet.js. #### Node.js @@ -35,12 +38,8 @@ Some great folks have written their own libraries to interact with Druid #### SQL +* [implydata/plyql](https://github.com/implydata/plyql) - A command line interface for issuing SQL queries to Druid via [plywood](https://github.com/implydata/plywood) * [srikalyc/Sql4D](https://github.com/srikalyc/Sql4D) - A SQL client for Druid. Used in production at Yahoo. -* [facetjs/facet-cli](https://github.com/facetjs/facet-cli) - A command line interface for issuing SQL queries to Druid via [facetjs](https://github.com/facetjs/facetjs) - -#### TypeScript - -* [facetjs/typescript-druid](https://github.com/facetjs/typescript-druid) - TypeScript declarations for the Druid API Community Helper Libraries @@ -50,6 +49,15 @@ Community Helper Libraries * [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness) - A set of scripts to simplify standing up some servers and seeing how things work * [mingfang/docker-druid](https://github.com/mingfang/docker-druid) - A Dockerfile to run the entire Druid cluster +Other Druid Distributions +------------------------- + +* [Imply Analytics Platform](http://imply.io/download) - The Imply Analytics platform repackages Druid, all its dependencies, and an UI and SQL layer. + +Tools +--- + +* [Insert Segments](../../operations/insert-segment-to-db.html) - A tool that can insert segments' metadata into Druid metadata storage. UIs --- @@ -64,8 +72,8 @@ Tools * [Insert Segments](../../operations/insert-segment-to-db.html) - A tool that can insert segments' metadata into Druid metadata storage. -Community Extensions --------------------- +Other Community Extensions +-------------------------- These are extensions from the community. (If you would like yours listed please speak up!) diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index 0e5fee5bac5f..e3af622cd3c7 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -3,113 +3,84 @@ layout: doc_page --- # Batch Data Ingestion -There are two choices for batch data ingestion to your Druid cluster, you can use the [Indexing service](../design/indexing-service.html) or you can use the `HadoopDruidIndexer`. -Which should I use? -------------------- +Druid can load data from static files through a variety of methods described here. -The [Indexing service](../design/indexing-service.html) is a set of nodes that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the [metadata store](../dependencies/metadata-storage.html) that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. The indexing service was created such that external systems could programmatically interact with it and run periodic indexing tasks. Long-term, the indexing service is going to be the preferred method of ingesting data. +## Hadoop-based Batch Ingestion -The `HadoopDruidIndexer` runs hadoop jobs in order to separate and index data segments. It takes advantage of Hadoop as a job scheduling and distributed job execution platform. It is a simple method if you already have Hadoop running and don’t want to spend the time configuring and deploying the [Indexing service](../design/indexing-service.html) just yet. - -## Batch Ingestion using the HadoopDruidIndexer - -The HadoopDruidIndexer can be run like so: - -``` -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*: io.druid.cli.Main index hadoop -``` - -## Hadoop "specFile" - -The spec\_file is a path to a file that contains JSON and an example looks like: +Hadoop-based batch ingestion in Druid is supported via a Hadoop-ingestion task. These tasks can be posted to a running instance +of a Druid [overlord](../design/indexing-service.html). A sample task is shown below: ```json { - "dataSchema" : { - "dataSource" : "wikipedia", - "parser" : { - "type" : "hadoopyString", - "parseSpec" : { - "format" : "json", - "timestampSpec" : { - "column" : "timestamp", - "format" : "auto" + "type" : "index_hadoop", + "spec" : { + "dataSchema" : { + "dataSource" : "wikipedia", + "parser" : { + "type" : "hadoopyString", + "parseSpec" : { + "format" : "json", + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], + "dimensionExclusions" : [], + "spatialDimensions" : [] + } + } + }, + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "type" : "doubleSum", + "name" : "added", + "fieldName" : "added" + }, + { + "type" : "doubleSum", + "name" : "deleted", + "fieldName" : "deleted" }, - "dimensionsSpec" : { - "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], - "dimensionExclusions" : [], - "spatialDimensions" : [] + { + "type" : "doubleSum", + "name" : "delta", + "fieldName" : "delta" } + ], + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "DAY", + "queryGranularity" : "NONE", + "intervals" : [ "2013-08-31/2013-09-01" ] } }, - "metricsSpec" : [ - { - "type" : "count", - "name" : "count" - }, - { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, - { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, - { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" + "ioConfig" : { + "type" : "hadoop", + "inputSpec" : { + "type" : "static", + "paths" : "/MyDirectory/example/wikipedia_data.json" } - ], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "DAY", - "queryGranularity" : "NONE", - "intervals" : [ "2013-08-31/2013-09-01" ] - } - }, - "ioConfig" : { - "type" : "hadoop", - "inputSpec" : { - "type" : "static", - "paths" : "/MyDirectory/examples/indexing/wikipedia_data.json" - }, - "metadataUpdateSpec" : { - "type":"mysql", - "connectURI" : "jdbc:mysql://localhost:3306/druid", - "password" : "diurd", - "segmentTable" : "druid_segments", - "user" : "druid" - }, - "segmentOutputPath" : "/MyDirectory/data/index/output" - }, - "tuningConfig" : { - "type" : "hadoop", - "workingPath": "/tmp", - "partitionsSpec" : { - "type" : "dimension", - "partitionDimension" : null, - "targetPartitionSize" : 5000000, - "maxPartitionSize" : 7500000, - "assumeGrouped" : false, - "numShards" : -1 }, - "shardSpecs" : { }, - "leaveIntermediate" : false, - "cleanupOnFailure" : true, - "overwriteFiles" : false, - "ignoreInvalidRows" : false, - "jobProperties" : { }, - "combineText" : false, - "rowFlushBoundary" : 300000, - "buildV9Directly" : false + "tuningConfig" : { + "type": "hadoop" + } } } ``` +|property|description|required?| +|--------|-----------|---------| +|type|The task type, this should always be "index_hadoop".|yes| +|spec|A Hadoop Index Spec. See [Batch Ingestion](../ingestion/batch-ingestion.html)|yes| +|hadoopDependencyCoordinates|A JSON array of Hadoop dependency coordinates that Druid will use, this property will override the default Hadoop coordinates. Once specified, Druid will look for those Hadoop dependencies from the location specified by `druid.extensions.hadoopDependenciesDir`|no| +|classpathPrefix|Classpath that will be pre-appended for the peon process.|no| + ### DataSchema This field is required. @@ -138,7 +109,6 @@ Is a type of inputSpec where a static path to where the data files are located i |Field|Type|Description|Required| |-----|----|-----------|--------| |paths|Array of String|A String of input paths indicating where the raw data is located.|yes| -|inputFormat|String|The input format of the data files. Default is `org.apache.hadoop.mapreduce.lib.input.TextInputFormat`, or `org.apache.hadoop.mapreduce.lib.input.CombineTextInputFormat` if `combineText` in tuningConfig is `true`.|no| For example, using the static input paths: @@ -156,7 +126,6 @@ Is a type of inputSpec that expects data to be laid out in a specific path forma |inputPath|String|Base path to append the expected time path to.|yes| |filePattern|String|Pattern that files should match to be included.|yes| |pathFormat|String|Joda date-time format for each directory. Default value is `"'y'=yyyy/'m'=MM/'d'=dd/'H'=HH"`, or see [Joda documentation](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html)|no| -|inputFormat|String|The input format of the data files. Default is `org.apache.hadoop.mapreduce.lib.input.TextInputFormat`, or `org.apache.hadoop.mapreduce.lib.input.CombineTextInputFormat` if `combineText` in tuningConfig is `true`.|no| For example, if the sample config were run with the interval 2012-06-01/2012-06-02, it would expect data at the paths @@ -175,21 +144,6 @@ Read Druid segments. See [here](../ingestion/update-existing-data.html) for more Read multiple sources of data. See [here](../ingestion/update-existing-data.html) for more information. -#### Metadata Update Job Spec - -This is a specification of the properties that tell the job how to update metadata such that the Druid cluster will see the output segments and load them. - - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|type|String|"metadata" is the only value available.|yes| -|connectURI|String|A valid JDBC url to metadata storage.|yes| -|user|String|Username for db.|yes| -|password|String|password for db.|yes| -|segmentTable|String|Table to use in DB.|yes| - -These properties should parrot what you have configured for your [Coordinator](../design/coordinator.html). - ### TuningConfig The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. @@ -266,123 +220,64 @@ The configuration options are: ### Remote Hadoop Cluster -If you have a remote Hadoop cluster, make sure to include the folder holding your configuration `*.xml` files in the classpath of the indexer. +If you have a remote Hadoop cluster, make sure to include the folder holding your configuration `*.xml` files in your Druid `_common` configuration folder. +If you having dependency problems with your version of Hadoop and the version compiled with Druid, please see [these docs](../operations/other-hadoop.html). -Batch Ingestion Using the Indexing Service ------------------------------------------- +### Using Elastic MapReduce -Batch ingestion for the indexing service is done by submitting an [Index Task](../misc/tasks.html) (for datasets < 1G) or a [Hadoop Index Task](../misc/tasks.html). The indexing service can be started by issuing: +If your cluster is running on Amazon Web Services, you can use Elastic MapReduce (EMR) to index data +from S3. To do this: + +- Create a persistent, [long-running cluster](http://docs.aws.amazon.com/ElasticMapReduce/latest/ManagementGuide/emr-plan-longrunning-transient.html). +- When creating your cluster, enter the following configuration. If you're using the wizard, this +should be in advanced mode under "Edit software settings". ``` -java -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/overlord io.druid.cli.Main server overlord +classification=yarn-site,properties=[mapreduce.reduce.memory.mb=6144,mapreduce.reduce.java.opts=-server -Xms2g -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps,mapreduce.map.java.opts=758,mapreduce.map.java.opts=-server -Xms512m -Xmx512m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps,mapreduce.task.timeout=1800000] ``` -This will start up a very simple local indexing service. For more complex deployments of the indexing service, see [here](../design/indexing-service.html). +- Follow the instructions under "[Configure Hadoop for data +loads](cluster.html#configure-cluster-for-hadoop-data-loads)" using the XML files from +`/etc/hadoop/conf` on your EMR master. -The schema of the Hadoop Index Task contains a task "type" and a Hadoop Index Config. A sample Hadoop index task is shown below: +#### Loading from S3 with EMR -```json -{ - "type" : "index_hadoop", - "spec" : { - "dataSchema" : { - "dataSource" : "wikipedia", - "parser" : { - "type" : "hadoopyString", - "parseSpec" : { - "format" : "json", - "timestampSpec" : { - "column" : "timestamp", - "format" : "auto" - }, - "dimensionsSpec" : { - "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], - "dimensionExclusions" : [], - "spatialDimensions" : [] - } - } - }, - "metricsSpec" : [ - { - "type" : "count", - "name" : "count" - }, - { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, - { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, - { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" - } - ], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "DAY", - "queryGranularity" : "NONE", - "intervals" : [ "2013-08-31/2013-09-01" ] - } - }, - "ioConfig" : { - "type" : "hadoop", - "inputSpec" : { - "type" : "static", - "paths" : "/MyDirectory/examples/indexing/wikipedia_data.json" - } - }, - "tuningConfig" : { - "type": "hadoop" - } - } +- In the `jobProperties` field in the `tuningConfig` section of your Hadoop indexing task, add: + +``` +"jobProperties" : { + "fs.s3.awsAccessKeyId" : "YOUR_ACCESS_KEY", + "fs.s3.awsSecretAccessKey" : "YOUR_SECRET_KEY", + "fs.s3.impl" : "org.apache.hadoop.fs.s3native.NativeS3FileSystem", + "fs.s3n.awsAccessKeyId" : "YOUR_ACCESS_KEY", + "fs.s3n.awsSecretAccessKey" : "YOUR_SECRET_KEY", + "fs.s3n.impl" : "org.apache.hadoop.fs.s3native.NativeS3FileSystem", + "io.compression.codecs" : "org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.SnappyCodec" } ``` -### DataSchema +Note that this method uses Hadoop's builtin S3 filesystem rather than Amazon's EMRFS, and is not compatible +with Amazon-specific features such as S3 encryption and consistent views. If you need to use those +features, you will need to make the Amazon EMR Hadoop JARs available to Druid through one of the +mechanisms described in the [Using other Hadoop distributions](#using-other-hadoop-distributions) section. -This field is required. +### Using other Hadoop distributions -See [Ingestion](../ingestion/index.html) +Druid works out of the box with many Hadoop distributions. -### IOConfig - -This field is required. - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|type|String|This should always be 'hadoop'.|yes| -|pathSpec|Object|a specification of where to pull the data in from|yes| - -### TuningConfig - -The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. This is the same as the tuningConfig for the standalone Hadoop indexer. See above for more details. +If you are having dependency conflicts between Druid and your version of Hadoop, you can try +searching for a solution in the [Druid user groups](https://groups.google.com/forum/#!forum/druid- +user), or reading the Druid [Different Hadoop Versions](..//operations/other-hadoop.html) documentation. -### Running the Task +## Command Line Hadoop Indexer -The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopDruidIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `metadataUpdateSpec`. The Indexing Service takes care of setting these fields internally. +If you don't want to use a full indexing service to use Hadoop to get data into Druid, you can also use the standalone command line Hadoop indexer. +See [here](../ingestion/command-line-hadoop-indexer.html) for more info. -To run the task: - -``` -curl -X 'POST' -H 'Content-Type:application/json' -d @example_index_hadoop_task.json localhost:8090/druid/indexer/v1/task -``` - -If the task succeeds, you should see in the logs of the indexing service: - -``` -2013-10-16 16:38:31,945 INFO [pool-6-thread-1] io.druid.indexing.overlord.exec.TaskConsumer - Task SUCCESS: HadoopIndexTask... -``` - -### Remote Hadoop Cluster +## IndexTask-based Batch Ingestion -If you have a remote Hadoop cluster, make sure to include the folder holding your configuration `*.xml` files in the classpath of the middle manager. +If you do not want to have a dependency on Hadoop for batch ingestion, you can also use the index task. This task will be much slower and less scalable than the Hadoop-based method. See [here](../ingestion/tasks.html)for more info. Having Problems? ---------------- -Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-development). +Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-user). diff --git a/docs/content/ingestion/command-line-hadoop-indexer.md b/docs/content/ingestion/command-line-hadoop-indexer.md new file mode 100644 index 000000000000..eba792d09f80 --- /dev/null +++ b/docs/content/ingestion/command-line-hadoop-indexer.md @@ -0,0 +1,67 @@ +--- +layout: doc_page +--- + +# Command Line Hadoop Indexer + +To run: + +``` +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*: io.druid.cli.Main index hadoop +``` + +The spec file needs to contain a JSON object where the contents are the same as the "spec" field in the Hadoop index task. +In addition, the following fields need to be added to the ioConfig: + +``` + "ioConfig" : { + ... + "metadataUpdateSpec" : { + "type":"mysql", + "connectURI" : "jdbc:mysql://localhost:3306/druid", + "password" : "diurd", + "segmentTable" : "druid_segments", + "user" : "druid" + }, + "segmentOutputPath" : "/MyDirectory/data/index/output" + }, +``` + +and the following field need to be added to the tuningConfig: + +``` + "tuningConfig" : { + ... + "workingPath": "/tmp", + ... + } +``` + +#### Metadata Update Job Spec + +This is a specification of the properties that tell the job how to update metadata such that the Druid cluster will see the output segments and load them. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|type|String|"metadata" is the only value available.|yes| +|connectURI|String|A valid JDBC url to metadata storage.|yes| +|user|String|Username for db.|yes| +|password|String|password for db.|yes| +|segmentTable|String|Table to use in DB.|yes| + +These properties should parrot what you have configured for your [Coordinator](../design/coordinator.html). + +#### segmentOutputPath Config + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|segmentOutputPath|String|the path to dump segments into.|yes| + +#### workingPath Config + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|workingPath|String|the working path to use for intermediate results (results between Hadoop jobs).|no (default == '/tmp/druid-indexing')| + +Please note that the command line Hadoop indexer doesn't have the locking capabilities of the indexing service, so if you choose to use it, +you have to take caution to not override segments created by real-time processing (if you that a real-time pipeline set up). diff --git a/docs/content/ingestion/faq.md b/docs/content/ingestion/faq.md index f50520446d9f..2289f308be58 100644 --- a/docs/content/ingestion/faq.md +++ b/docs/content/ingestion/faq.md @@ -6,11 +6,9 @@ layout: doc_page ### Realtime Ingestion -If you are trying to stream in historical (not current time) data into Druid and you are using the [serverTime](../ingestion/realtime-ingestion.html) rejection policy in your ingestion spec (the default rejection policy), Druid will not ingest this data as it is outside of the acceptable window period. You can verify this is what is happening by looking at the logs of your real-time process for log lines containing "ingest/events/*". These metrics will indicate the events ingested, rejected, etc. We recommend using batch ingestion methods for historical data in production. - -If you are doing a POC, you can use the [messageTime](../ingestion/realtime-ingestion.html) rejection policy, but please be aware of the hand-off caveats. This rejection policy is not recommended in production. - -If you are experimenting with realtime ingestion, you can also use the [none](../ingestion/realtime-ingestion.html) rejection policy to load all incoming events, but hand-off will never occur. +The most common cause of this is because events being ingested are out of band of Druid's `windowPeriod`. Druid realtime ingestion +only accepts events within a configurable windowPeriod of the current time. You can verify this is what is happening by looking at the logs of your real-time process for log lines containing "ingest/events/*". These metrics will indicate the events ingested, rejected, etc. +We recommend using batch ingestion methods for historical data in production. ### Batch Ingestion @@ -30,17 +28,17 @@ If the number of ingested events seem correct, make sure your query is correctly Depending on what `druid.storage.type` is set to, Druid will upload segments to some [Deep Storage](../dependencies/deep-storage.html). Local disk is used as the default deep storage. -## My realtime node is not handing segments off +## My stream ingest is not handing segments off -First, make sure there are no exceptions in the logs of your node. Also make sure that `druid.storage.type` is set to a deep storage that makes sense. +First, make sure there are no exceptions in the logs of the ingestion process. Also make sure that `druid.storage.type` is set to a deep storage that isn't `local` if you are running a distributed cluster. Other common reasons that hand-off fails are as follows: -1) Druid is unable to write to the metadata storage. Make sure your configuration is correct. +1) Druid is unable to write to the metadata storage. Make sure your configurations are correct. -2) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the coordinator logs if this occurs. +2) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the coordinator logs if this occurs and the coordinator console will show the historicals are near capacity. -3) Segments are corrupt and cannot download. You'll see exceptions in your historical nodes if this occurs. +3) Segments are corrupt and cannot be downloaded. You'll see exceptions in your historical nodes if this occurs. 4) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the coordinator logs have no errors. @@ -49,6 +47,7 @@ Other common reasons that hand-off fails are as follows: Make sure to include the `druid-hdfs-storage` and all the hadoop configuration, dependencies (that can be obtained by running command `hadoop classpath` on a machine where hadoop has been setup) in the classpath. And, provide necessary HDFS settings as described in [Deep Storage](../dependencies/deep-storage.html) . ## I don't see my Druid segments on my historical nodes + You can check the coordinator console located at `:`. Make sure that your segments have actually loaded on [historical nodes](../design/historical.html). If your segments are not present, check the coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because historical nodes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example): ``` @@ -58,7 +57,7 @@ You can check the coordinator console located at `:`. Make ## My queries are returning empty results -You can check `:/druid/v2/datasources/?interval=0/3000` for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists. Note: the broker endpoint will only return valid results on historical segments and not segments served by real-time nodes. +You can use a [segment metadata query](../querying/segmentmetadataquery.html) for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists. ## How can I Reindex existing data in Druid with schema changes? diff --git a/docs/content/ingestion/overview.md b/docs/content/ingestion/overview.md deleted file mode 100644 index 943308807634..000000000000 --- a/docs/content/ingestion/overview.md +++ /dev/null @@ -1,82 +0,0 @@ ---- -layout: doc_page ---- -# Ingestion Overview - -There are a couple of different ways to get data into Druid. We hope to unify things in the near future, but for the time being -the method you choose to ingest your data into Druid should be driven by your use case. - -## Streaming Data - -If you have a continuous stream of data, there are a few options to get your data into Druid. It should be noted that the current state of real-time ingestion in Druid does not guarantee exactly once processing. The real-time pipeline is meant to surface insights on - events as they are occurring. For an accurate copy of ingested data, an accompanying batch pipeline is required. We are working towards a streaming only word, but for - the time being, we recommend running a lambda architecture. - -### Ingest from a Stream Processor - -If you process your data using a stream processor such as Apache Samza or Apache Storm, you can use the [Tranquility](https://github.com/druid-io/tranquility) library to manage -your real-time ingestion. This setup requires using the indexing service for ingestion, which is what is used in production by many organizations that use Druid. - -### Ingest from Apache Kafka - -If you wish to ingest directly from Kafka using Tranquility, you will have to write a consumer that reads from Kafka and passes the data to Tranquility. -The other option is to use [standalone Realtime nodes](../design/realtime.html). -It should be noted that standalone realtime nodes use the Kafka high level consumer, which imposes a few restrictions. - -Druid replicates segment such that logically equivalent data segments are concurrently hosted on N nodes. If N–1 nodes go down, -the data will still be available for querying. On real-time nodes, this process depends on maintaining logically equivalent -data segments on each of the N nodes, which is not possible with standard Kafka consumer groups if your Kafka topic requires more than one consumer -(because consumers in different consumer groups will split up the data differently). - -For example, let's say your topic is split across Kafka partitions 1, 2, & 3 and you have 2 real-time nodes with linear shard specs 1 & 2. -Both of the real-time nodes are in the same consumer group. Real-time node 1 may consume data from partitions 1 & 3, and real-time node 2 may consume data from partition 2. -Querying for your data through the broker will yield correct results. - -The problem arises if you want to replicate your data by creating real-time nodes 3 & 4. These new real-time nodes also -have linear shard specs 1 & 2, and they will consume data from Kafka using a different consumer group. In this case, -real-time node 3 may consume data from partitions 1 & 2, and real-time node 4 may consume data from partition 2. -From Druid's perspective, the segments hosted by real-time nodes 1 and 3 are the same, and the data hosted by real-time nodes -2 and 4 are the same, although they are reading from different Kafka partitions. Querying for the data will yield inconsistent -results. - -Is this always a problem? No. If your data is small enough to fit on a single Kafka partition, you can replicate without issues. -Otherwise, you can run real-time nodes without replication. - -## Large Batch of Static Data - -If you have a large batch of historical data that you want to load all at once into Druid, you should use Druid's built in support for - Hadoop-based indexing. Hadoop-based indexing for large (> 1G) of batch data is the fastest way to load data into Druid. If you wish to avoid - the Hadoop dependency, or if you do not have a Hadoop cluster present, you can look at using the [index task](). The index task will be much slower - than Hadoop indexing for ingesting batch data. - -One pattern that we've seen is to store raw events (or processed events) in deep storage (S3, HDFS, etc) and periodically run batch processing jobs over these raw events. -You can, for example, create a directory structure for your raw data, such as the following: - -``` -/prod//v=1/y=2015/m=03/d=21/H=20/data.gz -/prod//v=1/y=2015/m=03/d=21/H=21/data.gz -/prod//v=1/y=2015/m=03/d=21/H=22/data.gz -``` - -In this example, hourly raw events are stored in individual gzipped files. Periodic batch processing jobs can then run over these files. - -## Lambda Architecture - -We recommend running a streaming real-time pipeline to run queries over events as they are occurring and a batch pipeline to perform periodic -cleanups of data. - -## Sharding - -Multiple segments may exist for the same interval of time for the same datasource. These segments form a `block` for an interval. -Depending on the type of `shardSpec` that is used to shard the data, Druid queries may only complete if a `block` is complete. That is to say, if a block consists of 3 segments, such as: - -`sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_0` - -`sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_1` - -`sampleData_2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z_v1_2` - -All 3 segments must be loaded before a query for the interval `2011-01-01T02:00:00:00Z_2011-01-01T03:00:00:00Z` completes. - -The exception to this rule is with using linear shard specs. Linear shard specs do not force 'completeness' and queries can complete even if shards are not loaded in the system. -For example, if your real-time ingestion creates 3 segments that were sharded with linear shard spec, and only two of the segments were loaded in the system, queries would return results only for those 2 segments. diff --git a/docs/content/ingestion/stream-ingestion.md b/docs/content/ingestion/stream-ingestion.md new file mode 100644 index 000000000000..33d4624fdd24 --- /dev/null +++ b/docs/content/ingestion/stream-ingestion.md @@ -0,0 +1,39 @@ +--- +layout: doc_page +--- + +# Loading streams + +Streams can be ingested in Druid using either [Tranquility](https://github.com/druid-io/tranquility (a Druid-aware +client) and the [indexing service](../design/indexing-service.html) or through standalone [Realtime nodes](../design/realtime.html). +The first approach will be more complex to set up, but also offers scalability and high availability characteristics that advanced production +setups may require. The second approach has some known [limitations](../ingestion/stream-pull.html#limitations). + +## Stream push + +If you have a program that generates a stream, then you can push that stream directly into Druid in +real-time. With this approach, Tranquility is embedded in your data-producing application. +Tranquility comes with bindings for the +Storm and Samza stream processors. It also has a direct API that can be used from any JVM-based +program, such as Spark Streaming or a Kafka consumer. + +Tranquility handles partitioning, replication, service discovery, and schema rollover for you, +seamlessly and without downtime. You only have to define your Druid schema. + +For examples and more information, please see the [Tranquility README](https://github.com/druid-io/tranquility). + +## Stream pull + +If you have an external service that you want to pull data from, you have two options. The simplest +option is to set up a "copying" service that reads from the data source and writes to Druid using +the [stream push method](#stream-push). + +Another option is *stream pull*. With this approach, a Druid Realtime Node ingests data from a +[Firehose](../ingestion/firehose.html) connected to the data you want to +read. Druid includes builtin firehoses for Kafka, RabbitMQ, and various other streaming systems. + +## More information + +For more information on loading streaming data via a push based approach, please see [here](../ingestion/stream-push.html). + +For more information on loading streaming data via a pull based approach, please see [here](../ingestion/stream-pull.html). diff --git a/docs/content/ingestion/realtime-ingestion.md b/docs/content/ingestion/stream-pull.md similarity index 77% rename from docs/content/ingestion/realtime-ingestion.md rename to docs/content/ingestion/stream-pull.md index bd9fd908faa4..59701cb5bdce 100644 --- a/docs/content/ingestion/realtime-ingestion.md +++ b/docs/content/ingestion/stream-pull.md @@ -2,20 +2,29 @@ layout: doc_page --- -Realtime Data Ingestion -======================= -For general Real-time Node information, see [here](../design/realtime.html). - -For Real-time Node Configuration, see [Realtime Configuration](../configuration/realtime.html). +Stream Pull Ingestion +===================== -For writing your own plugins to the real-time node, see [Firehose](../ingestion/firehose.html). +If you have an external service that you want to pull data from, you have two options. The simplest +option is to set up a "copying" service that reads from the data source and writes to Druid using +the [stream push method](stream-push.html). -There are two ways of ingesting real-time data. This can be achieved with a standalone real-time node, or using the [Tranquility](https://github.com/druid-io/tranquility) client library as part of the [Indexing Service](../design/indexing-service.html). For a full explanation of why there are two methods, please see [this link](https://groups.google.com/forum/#!searchin/druid-development/fangjin$20yang$20%22thoughts%22/druid-development/aRMmNHQGdhI/muBGl0Xi_wgJ). If you are comfortable with the limitations of standalone real-time nodes, you can use them as they are easier to set up. The indexing service is a more robust and highly available solution but will also require more effort to set up. +Another option is *stream pull*. With this approach, a Druid Realtime Node ingests data from a +[Firehose](../ingestion/firehose.html) connected to the data you want to +read. The Druid quickstart and tutorials do not include information about how to set up standalone realtime nodes, but +they can be used in place for Tranquility server and the indexing service. Please note that Realtime nodes have very properties than +the indexing service. ## Realtime Node Ingestion Much of the configuration governing Realtime nodes and the ingestion of data is set in the Realtime spec file, discussed on this page. +For general Real-time Node information, see [here](../design/realtime.html). + +For Real-time Node Configuration, see [Realtime Configuration](../configuration/realtime.html). + +For writing your own plugins to the real-time node, see [Firehose](../ingestion/firehose.html). + ## Realtime "specFile" @@ -119,7 +128,7 @@ This field is required. #### Firehose -See [Firehose](../ingestion/firehose.html) for more information on firehose configuration. +See [Firehose](../ingestion/firehose.html) for more information on various firehoses. #### Plumber @@ -158,6 +167,7 @@ The following policies are available: #### Sharding + Druid uses shards, or segments with partition numbers, to more efficiently handle large amounts of incoming data. In Druid, shards represent the segments that together cover a time interval based on the value of `segmentGranularity`. If, for example, `segmentGranularity` is set to "hour", then a number of shards may be used to store the data for that hour. Sharding along dimensions may also occur to optimize efficiency. Segments are identified by datasource, time interval, and version. With sharding, a segment is also identified by a partition number. Typically, each shard will have the same version but a different partition number to uniquely identify it. @@ -175,6 +185,7 @@ Druid uses sharding based on the `shardSpec` setting you configure. The recommen Keep in mind, that sharding configuration has nothing to do with configured firehose. For example, if you set partition number to 0, it doesn't mean that Kafka firehose will consume only from 0 topic partition. ##### Linear + This strategy provides following advantages: * There is no need to update the fileSpec configurations of existing nodes when adding new nodes. @@ -191,6 +202,7 @@ Configure `linear` under `schema`: ##### Numbered + This strategy is similar to `linear` except that it does not tolerate non-sequential partition numbering (it will *not* allow querying of partitions 0 and 2 if partition 1 is missing). It also requires explicitly setting the total number of partitions. Configure `numbered` under `schema`: @@ -205,6 +217,7 @@ Configure `numbered` under `schema`: ##### Scale and Redundancy + The `shardSpec` configuration can be used to create redundancy by having the same `partitionNum` values on different nodes. For example, if RealTimeNode1 has: @@ -240,11 +253,6 @@ then it can store segments with the same datasource, time interval, and version You can use type `numbered` similarly. Note that type `none` is essentially type `linear` with all shards having a fixed `partitionNum` of 0. - -## Realtime Ingestion using the Indexing Service - -We strongly recommend using the client library [Tranquility](https://github.com/druid-io/tranquility) for this use case. Please read the documentation on the Tranquility web page. - ## Constraints The following table summarizes constraints between settings in the spec file for the Realtime subsystem. @@ -259,3 +267,48 @@ The following table summarizes constraints between settings in the spec file for The normal, expected use cases have the following overall constraints: `intermediatePersistPeriod ≤ windowPeriod < segmentGranularity` and `queryGranularity ≤ segmentGranularity` +## Limitations + +### Kafka + +Standalone realtime nodes use the Kafka high level consumer, which imposes a few restrictions. + +Druid replicates segment such that logically equivalent data segments are concurrently hosted on N nodes. If N–1 nodes go down, +the data will still be available for querying. On real-time nodes, this process depends on maintaining logically equivalent +data segments on each of the N nodes, which is not possible with standard Kafka consumer groups if your Kafka topic requires more than one consumer +(because consumers in different consumer groups will split up the data differently). + +For example, let's say your topic is split across Kafka partitions 1, 2, & 3 and you have 2 real-time nodes with linear shard specs 1 & 2. +Both of the real-time nodes are in the same consumer group. Real-time node 1 may consume data from partitions 1 & 3, and real-time node 2 may consume data from partition 2. +Querying for your data through the broker will yield correct results. + +The problem arises if you want to replicate your data by creating real-time nodes 3 & 4. These new real-time nodes also +have linear shard specs 1 & 2, and they will consume data from Kafka using a different consumer group. In this case, +real-time node 3 may consume data from partitions 1 & 2, and real-time node 4 may consume data from partition 2. +From Druid's perspective, the segments hosted by real-time nodes 1 and 3 are the same, and the data hosted by real-time nodes +2 and 4 are the same, although they are reading from different Kafka partitions. Querying for the data will yield inconsistent +results. + +Is this always a problem? No. If your data is small enough to fit on a single Kafka partition, you can replicate without issues. +Otherwise, you can run real-time nodes without replication. + +There is now also an [experimental low level Kafka firehose](../development/kafka-simple-consumer-firehose.html) which +solves the issues described above with using the high level Kafka consumer. + +### Locking + +Using stream pull ingestion with Realtime nodes together batch ingestion may introduce data override issues. For example, if you +are generating hourly segments for the current day, and run a daily batch job for the current day's data, the segments created by +the batch job will have a more recent version than most of the segments generated by realtime ingestion. If your batch job is indexing +data that isn't yet complete for the day, the daily segment created by the batch job can override recent segments created by +realtime nodes. A portion of data will appear to be lost in this case. + +### Schema changes + +Standalone realtime nodes require stopping a node to update a schema, and starting it up again for the schema to take effect. +This can be difficult to manage at scale, especially with multiple partitions. + +### Log management + +Each standalone realtime node has its own set of logs. Diagnosing errors across many partitions across many servers may be +difficult to manage and track at scale. diff --git a/docs/content/ingestion/stream-push.md b/docs/content/ingestion/stream-push.md new file mode 100644 index 000000000000..31c0934b1dd7 --- /dev/null +++ b/docs/content/ingestion/stream-push.md @@ -0,0 +1,136 @@ +--- +layout: doc_page +--- + +## Stream Push + +Druid can connect to any streaming data source through +[Tranquility](https://github.com/druid-io/tranquility/blob/master/README.md), a package for pushing +streams to Druid in real-time. Druid does not come bundled with Tranquility, and you will have to download the distribution. + +```note-info +If you've never loaded streaming data into Druid, we recommend trying out the +[stream loading tutorial](../tutorials/tutorial-streams.html) first and then coming back to this page. +``` + +Note that with all streaming ingestion options, you must ensure that incoming data is recent +enough (within a [configurable windowPeriod](#segmentgranularity-and-windowperiod) of the current +time). Older messages will not be processed in real-time. Historical data is best processed with +[batch ingestion](../ingestion/batch-ingestion.html). + +### Server + +Druid can use [Tranquility Server](https://github.com/druid-io/tranquility/blob/master/docs/server.md), which +lets you send data to Druid without developing a JVM app. You can run Tranquility server colocated with Druid middleManagers +and historical processes. + +Tranquility server is started by issuing: + +```bash +bin/tranquility server -configFile /server.json +``` + +To customize Tranquility Server: + +- In `server.json`, customize the `properties` and `dataSources`. +- If you have servers already running Tranquility, stop them (CTRL-C) and start +them up again. + +For tips on customizing `server.json`, see the +*[Loading your own streams](../tutorials/tutorial-streams.html)* tutorial and the +[Tranquility Server documentation](https://github.com/druid-io/tranquility/blob/master/docs/server.md). + +### Kafka + +[Tranquility Kafka](https://github.com/druid-io/tranquility/blob/master/docs/kafka.md) +lets you load data from Kafka into Druid without writing any code. You only need a configuration +file. + +Tranquility server is started by issuing: + +```bash +bin/tranquility kafka -configFile /kafka.json +``` + +To customize Tranquility Kafka in the single-machine quickstart configuration: + +- In `kafka.json`, customize the `properties` and `dataSources`. +- If you have Tranquility already running, stop it (CTRL-C) and start it up again. + +For tips on customizing `kafka.json`, see the +[Tranquility Kafka documentation](https://github.com/druid-io/tranquility/blob/master/docs/kafka.md). + +### JVM apps and stream processors + +Tranquility can also be embedded in JVM-based applications as a library. You can do this directly +in your own program using the +[Core API](https://github.com/druid-io/tranquility/blob/master/docs/core.md), or you can use +the connectors bundled in Tranquility for popular JVM-based stream processors such as +[Storm](https://github.com/druid-io/tranquility/blob/master/docs/storm.md), +[Samza](https://github.com/druid-io/tranquility/blob/master/docs/samza.md), +[Spark Streaming](https://github.com/druid-io/tranquility/blob/master/docs/spark.md), and +[Flink](https://github.com/druid-io/tranquility/blob/master/docs/flink.md). + +## Concepts + +### Task creation + +Tranquility automates creation of Druid realtime indexing tasks, handling partitioning, replication, +service discovery, and schema rollover for you, seamlessly and without downtime. You never have to +write code to deal with individual tasks directly. But, it can be helpful to understand how +Tranquility creates tasks. + +Tranquility spawns relatively short-lived tasks periodically, and each one handles a small number of +[Druid segments](../design/segments.html). Tranquility coordinates all task +creation through ZooKeeper. You can start up as many Tranquility instances as you like with the same +configuration, even on different machines, and they will send to the same set of tasks. + +See the [Tranquility overview](https://github.com/druid-io/tranquility/blob/master/docs/overview.md) +for more details about how Tranquility manages tasks. + +### segmentGranularity and windowPeriod + +The segmentGranularity is the time period covered by the segments produced by each task. For +example, a segmentGranularity of "hour" will spawn tasks that create segments covering one hour +each. + +The windowPeriod is the slack time permitted for events. For example, a windowPeriod of ten minutes +(the default) means that any events with a timestamp older than ten minutes in the past, or more +than ten minutes in the future, will be dropped. + +These are important configurations because they influence how long tasks will be alive for, and how +long data stays in the realtime system before being handed off to the historical nodes. For example, +if your configuration has segmentGranularity "hour" and windowPeriod ten minutes, tasks will stay +around listening for events for an hour and ten minutes. For this reason, to prevent excessive +buildup of tasks, it is recommended that your windowPeriod be less than your segmentGranularity. + +### Append only + +Druid streaming ingestion is *append-only*, meaning you cannot use streaming ingestion to update or +delete individual records after they are inserted. If you need to update or delete individual +records, you need to use a batch reindexing process. See the *[batch ingest](batch-ingestion.html)* +page for more details. + +Druid does support efficient deletion of entire time ranges without resorting to batch reindexing. +This can be done automatically through setting up retention policies. + +### Guarantees + +Tranquility operates under a best-effort design. It tries reasonably hard to preserve your data, by allowing you to set +up replicas and by retrying failed pushes for a period of time, but it does not guarantee that your events will be +processed exactly once. In some conditions, it can drop or duplicate events: + +- Events with timestamps outside your configured windowPeriod will be dropped. +- If you suffer more Druid Middle Manager failures than your configured replicas count, some +partially indexed data may be lost. +- If there is a persistent issue that prevents communication with the Druid indexing service, and +retry policies are exhausted during that period, or the period lasts longer than your windowPeriod, +some events will be dropped. +- If there is an issue that prevents Tranquility from receiving an acknowledgement from the indexing +service, it will retry the batch, which can lead to duplicated events. +- If you are using Tranquility inside Storm or Samza, various parts of both architectures have an +at-least-once design and can lead to duplicated events. + +Under normal operation, these risks are minimal. But if you need absolute 100% fidelity for +historical data, we recommend a [hybrid batch/streaming](../tutorials/ingestion.html#hybrid-batch-streaming) +architecture. diff --git a/docs/content/misc/tasks.md b/docs/content/ingestion/tasks.md similarity index 57% rename from docs/content/misc/tasks.md rename to docs/content/ingestion/tasks.md index cc9788218c67..4587c984bfba 100644 --- a/docs/content/misc/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -9,6 +9,10 @@ There are several different types of tasks. Segment Creation Tasks ---------------------- +### Hadoop Index Task + +See [batch ingestion](../ingestion/batch-ingestion.html). + ### Index Task The Index Task is a simpler variation of the Index Hadoop task that is designed to be used for smaller data sets. The task executes within the indexing service and does not require an external Hadoop setup to use. The grammar of the index task is as follows: @@ -123,155 +127,6 @@ The indexSpec is optional and default parameters will be used if not specified. |dimensionCompression|compression format for dimension columns (currently only affects single-value dimensions, multi-value dimensions are always uncompressed)|`"uncompressed"`, `"lz4"`, `"lzf"`|`"lz4"`|no| |metricCompression|compression format for metric columns, defaults to LZ4|`"lz4"`, `"lzf"`|`"lz4"`|no| -### Hadoop Index Task - -The Hadoop Index Task is used to index larger data sets that require the parallelization and processing power of a Hadoop cluster. - -``` -{ - "type" : "index_hadoop", - "spec": -} -``` - -|property|description|required?| -|--------|-----------|---------| -|type|The task type, this should always be "index_hadoop".|yes| -|spec|A Hadoop Index Spec. See [Batch Ingestion](../ingestion/batch-ingestion.html)|yes| -|hadoopDependencyCoordinates|A JSON array of Hadoop dependency coordinates that Druid will use, this property will override the default Hadoop coordinates. Once specified, Druid will look for those Hadoop dependencies from the location specified by `druid.extensions.hadoopDependenciesDir`|no| -|classpathPrefix|Classpath that will be pre-appended for the peon process.|no| - -The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopDruidIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `metadataUpdateSpec`. The Indexing Service takes care of setting these fields internally. - -Note: Before using Hadoop Index Task, please make sure to include Hadoop dependencies so that Druid knows where to pick them up during runtime, see [Include Hadoop Dependencies](../operations/other-hadoop.html). -Druid uses hadoop-client 2.3.0 as the default Hadoop version, you can get it from the released Druid tarball(under folder ```hadoop_dependencies```) or use [pull-deps](../pull-deps.html). - -#### Using your own Hadoop distribution - -Druid is compiled against Apache hadoop-client 2.3.0. However, if you happen to use a different flavor of Hadoop that is API compatible with hadoop-client 2.3.0, you should first make sure Druid knows where to pick it up, then you should only have to change the `hadoopDependencyCoordinates` property to point to the list of maven artifact used by your distribution. For non-API compatible versions and more information, please see [here](../operations/other-hadoop.html). - -#### Resolving dependency conflicts running HadoopIndexTask - -Currently, the HadoopIndexTask creates a single classpath to run the HadoopDruidIndexerJob, which can lead to version conflicts between various dependencies of Druid, extension modules, and Hadoop's own dependencies. - -The Hadoop index task will put Druid's dependencies first on the classpath, followed by any extensions dependencies, and any Hadoop dependencies last. - -If you are having trouble with any extensions in HadoopIndexTask, it may be the case that Druid, or one of its dependencies, depends on a different version of a library than what you are using as part of your extensions, but Druid's version overrides the one in your extension. In that case you probably want to build your own Druid version and override the offending library by adding an explicit dependency to the pom.xml of each druid sub-module that depends on it. - -### Realtime Index Task - -The indexing service can also run real-time tasks. These tasks effectively transform a middle manager into a real-time node. We introduced real-time tasks as a way to programmatically add new real-time data sources without needing to manually add nodes. We recommend you use the library [tranquility](https://github.com/druid-io/tranquility) to programmatically manage generating real-time index tasks. The grammar for the real-time task is as follows: - -```json -{ - "type": "index_realtime", - "id": "example", - "resource": { - "availabilityGroup": "someGroup", - "requiredCapacity": 1 - }, - "spec": { - "dataSchema": { - "dataSource": "wikipedia", - "parser": { - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ], - "dimensionExclusions": [ - - ], - "spatialDimensions": [ - - ] - } - } - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE" - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "kafka-0.8", - "consumerProps": { - "zookeeper.connect": "zk_connect_string", - "zookeeper.connection.timeout.ms" : "15000", - "zookeeper.session.timeout.ms" : "15000", - "zookeeper.sync.time.ms" : "5000", - "group.id": "consumer-group", - "fetch.message.max.bytes" : "1048586", - "auto.offset.reset": "largest", - "auto.commit.enable": "false" - }, - "feed": "your_kafka_topic" - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 500000, - "intermediatePersistPeriod": "PT10m", - "windowPeriod": "PT10m", - "rejectionPolicy": { - "type": "serverTime" - } - } - } -} -``` - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|id|String|The ID of the task.|No| -|Resource|JSON object|Used for high availability purposes.|No| -|availabilityGroup|String|An uniqueness identifier for the task. Tasks with the same availability group will always run on different middle managers. Used mainly for replication. |yes| -|requiredCapacity|Integer|How much middle manager capacity this task will take.|yes| - -For schema, windowPeriod, segmentGranularity, and other configuration information, see [Realtime Ingestion](../ingestion/realtime-ingestion.html). For firehose configuration, see [Firehose](../ingestion/firehose.html). - - Segment Merging Tasks --------------------- @@ -328,7 +183,8 @@ The convert task suite takes active segments and will recompress them using a ne Upon success the new segments will have the same version as the old segment with `_converted` appended. A convert task may be run against the same interval for the same datasource multiple times. Each execution will append another `_converted` to the version for the segments There are two types of conversion tasks. One is the Hadoop convert task, and the other is the indexing service convert task. The Hadoop convert task runs on a hadoop cluster, and simply leaves a task monitor on the indexing service (similar to the hadoop batch task). The indexing service convert task runs the actual conversion on the indexing service. -####Hadoop Convert Segment Task + +#### Hadoop Convert Segment Task ```json { "type": "hadoop_convert_segment", @@ -358,7 +214,7 @@ The values are described below. |`segmentOutputPath`|URI|A base uri for the segment to be placed. Same format as other places a segment output path is needed|Yes| -####Indexing Service Convert Segment Task +#### Indexing Service Convert Segment Task ```json { "type": "convert_segment", @@ -380,6 +236,7 @@ The values are described below. |`validate`|boolean|Runs validation between the old and new segment before reporting task success|No (true)| Unlike the hadoop convert task, the indexing service task draws its output path from the indexing service's configuration. + ### Noop Task These tasks start, sleep for a time and are used only for testing. The available grammar is: @@ -396,4 +253,8 @@ These tasks start, sleep for a time and are used only for testing. The available Locking ------- -Once an overlord node accepts a task, a lock is created for the data source and interval specified in the task. Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a "task group", which is a set of tasks that can share interval locks. + +Once an overlord node accepts a task, a lock is created for the data source and interval specified in the task. +Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release +locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. +Tasks are also part of a "task group", which is a set of tasks that can share interval locks. diff --git a/docs/content/ingestion/update-existing-data.md b/docs/content/ingestion/update-existing-data.md index 761744f40c38..c8284e8d0caa 100644 --- a/docs/content/ingestion/update-existing-data.md +++ b/docs/content/ingestion/update-existing-data.md @@ -109,7 +109,7 @@ For example: ### Reindexing without Hadoop Batch Ingestion -This section assumes the reader understands how to do batch ingestion without Hadoop using the [IndexTask](../misc/tasks.html#index-task), +This section assumes the reader understands how to do batch ingestion without Hadoop using the [IndexTask](../ingestion/tasks.html#index-task), which uses a "firehose" to know where and how to read the input data. [IngestSegmentFirehose](firehose.html#ingestsegmentfirehose) can be used to read data from segments inside Druid. Note that IndexTask is to be used for prototyping purposes only as it has to do all processing inside a single process and can't scale. Please use Hadoop batch ingestion for production diff --git a/docs/content/misc/cluster-setup.md b/docs/content/misc/cluster-setup.md deleted file mode 100644 index 10a18e63efcf..000000000000 --- a/docs/content/misc/cluster-setup.md +++ /dev/null @@ -1,116 +0,0 @@ ---- -layout: doc_page ---- - -# Setting Up a Druid Cluster - -A Druid cluster consists of various node types that need to be set up depending on your use case. See our [Design](../design/design.html) docs for a description of the different node types. - -Minimum Physical Layout: Absolute Minimum ------------------------------------------ - -As a special case, the absolute minimum setup is one of the standalone examples for real-time ingestion and querying; see [Examples](../tutorials/examples.html) that can easily run on one machine with one core and 1GB RAM. This layout can be set up to try some basic queries with Druid. - -Minimum Physical Layout: Experimental Testing with 4GB of RAM -------------------------------------------------------------- - -This layout can be used to load some data from deep storage onto a Druid historical node for the first time. A minimal physical layout for a 1 or 2 core machine with 4GB of RAM is: - -1. node1: [Coordinator](../design/coordinator.html) + metadata service + zookeeper + [Historical](../design/historical.html) -2. transient nodes: [Indexing Service](../design/indexing-service.html) - -This setup is only reasonable to prove that a configuration works. It would not be worthwhile to use this layout for performance measurement. - -Comfortable Physical Layout: Pilot Project with Multiple Machines ------------------------------------------------------------------ - -The machine size "flavors" are using AWS/EC2 terminology for descriptive purposes only and is not meant to imply that AWS/EC2 is required or recommended. Another cloud provider or your own hardware can also work. - -A minimal physical layout not constrained by cores that demonstrates parallel querying and realtime, using AWS-EC2 "small"/m1.small (one core, with 1.7GB of RAM) or larger, no real-time, is: - -1. node1: [Coordinator](../design/coordinator.html) (m1.small) -2. node2: metadata service (m1.small) -3. node3: zookeeper (m1.small) -4. node4: [Broker](../design/broker.html) (m1.small or m1.medium or m1.large) -5. node5: [Historical](../design/historical.html) (m1.small or m1.medium or m1.large) -6. node6: [Historical](../design/historical.html) (m1.small or m1.medium or m1.large) -7. node7: [Realtime](../design/realtime.html) (m1.small or m1.medium or m1.large) -8. transient nodes: [Indexing Service](../design/indexing-service.html) - -This layout naturally lends itself to adding more RAM and core to Historical nodes, and to adding many more Historical nodes. Depending on the actual load, the Coordinator, metadata server, and Zookeeper might need to use larger machines. - -High Availability Physical Layout ---------------------------------- - -The machine size "flavors" are using AWS/EC2 terminology for descriptive purposes only and is not meant to imply that AWS/EC2 is required or recommended. Another cloud provider or your own hardware can also work. - -An HA layout allows full rolling restarts and heavy volume: - -1. node1: [Coordinator](../design/coordinator.html) (m1.small or m1.medium or m1.large) -2. node2: [Coordinator](../design/coordinator.html) (m1.small or m1.medium or m1.large) (backup) -3. node3: metadata service (c1.medium or m1.large) -4. node4: metadata service (c1.medium or m1.large) (backup) -5. node5: zookeeper (c1.medium) -6. node6: zookeeper (c1.medium) -7. node7: zookeeper (c1.medium) -8. node8: [Broker](../design/broker.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) -9. node9: [Broker](../design/broker.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) (backup) -10. node10: [Historical](../design/historical.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) -11. node11: [Historical](../design/historical.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) -12. node12: [Realtime](../design/realtime.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) -13. transient nodes: [Indexing Service](../design/indexing-service.html) - -Sizing for Cores and RAM ------------------------- - -The Historical and Broker nodes will use as many cores as are available, depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is not well characterized yet and would depend on types of queries, query load, and the schema. Historical daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries. - -The effective utilization of cores by Zookeeper, metadata storage, and Coordinator nodes is likely to be between 1 and 2 for each process/daemon, so these could potentially share a machine with lots of cores. These daemons work with heap a size between 500MB and 1GB. - -Storage -------- - -Indexed segments should be kept in a permanent store accessible by all nodes like AWS S3 or HDFS or equivalent. Refer to [Deep-Storage](../dependencies/deep-storage.html) for more details on supported storage types. - -Local disk ("ephemeral" on AWS EC2) for caching is recommended over network mounted storage (example of mounted: AWS EBS, Elastic Block Store) in order to avoid network delays during times of heavy usage. If your data center is suitably provisioned for networked storage, perhaps with separate LAN/NICs just for storage, then mounted might work fine. - -Setup ------ - -Setting up a cluster is essentially just firing up all of the nodes you want with the proper [configuration](../configuration/index.html). One thing to be aware of is that there are a few properties in the configuration that potentially need to be set individually for each process: - -``` -druid.server.type=historical|realtime -druid.host=someHostOrIPaddrWithPort -druid.port=8080 -``` - -`druid.server.type` should be set to "historical" for your historical nodes and realtime for the realtime nodes. The Coordinator will only assign segments to a "historical" node and the broker has some intelligence around its ability to cache results when talking to a realtime node. This does not need to be set for the coordinator or the broker. - -`druid.host` should be set to the hostname that can be used to talk to the given server process. Basically, someone should be able to send a request to http://${druid.host}:${druid.port}/ and actually talk to the process. - -`druid.port` should be set to the port that the server should listen on. - -Build/Run ---------- - -The simplest way to build and run from the repository is to run `mvn package` from the base directory and then take `druid-services/target/druid-services-*-selfcontained.jar` and push that around to your machines; the jar does not need to be expanded, and since it contains the main() methods for each kind of service, it is *not* invoked with java -jar. It can be run from a normal java command-line by just including it on the classpath and then giving it the main class that you want to run. For example one instance of the Historical node/service can be started like this: - -``` -java -Duser.timezone=UTC -Dfile.encoding=UTF-8 -cp services/target/druid-services-*-selfcontained.jar io.druid.cli.Main server historical -``` - -All Druid server nodes can be started with: - -``` -io.druid.cli.Main server -``` - -The table below show the program arguments for the different node types. - -|service|program arguments| -|-------|----------------| -|Realtime|realtime| -|Coordinator|coordinator| -|Broker|broker| -|Historical|historical| diff --git a/docs/content/misc/evaluate.md b/docs/content/misc/evaluate.md deleted file mode 100644 index 8482c9c2ba4b..000000000000 --- a/docs/content/misc/evaluate.md +++ /dev/null @@ -1,98 +0,0 @@ ---- -layout: doc_page ---- - -Evaluate Druid -============== - -This page is meant to help you in evaluating Druid by answering common questions that come up. - -## Evaluating on a Single Machine - -Most of the tutorials focus on running multiple Druid services on a single machine in an attempt to teach basic Druid concepts, and work out kinks in data ingestion. The configurations in the tutorials are - very poor choices for an actual production cluster. - -## Capacity and Cost Planning - -The best way to understand what your cluster will cost is to first understand how much data reduction you will get when you create segments. -We recommend indexing and creating segments from 1G of your data and evaluating the resultant segment size. This will allow you to see how much your data rolls up, and how many segments will be able -to be loaded on the hardware you have at your disposal. - -Most of the cost of a Druid cluster is in historical nodes, followed by real-time indexing nodes if you have a high data intake. For high availability, you should have backup -coordination nodes (coordinators and overlords). Coordination nodes should require much cheaper hardware than nodes that serve queries. - -## Selecting Hardware - -Druid is designed to run on commodity hardware and we've tried to provide some general guidelines on [how things should be tuned]() for various deployments. We've also provided -some [example specs](../configuration/production-cluster.html) for hardware for a production cluster. - -## Benchmarking Druid - -The best resource to benchmark Druid is to follow the steps outlined in our [blog post](http://druid.io/blog/2014/03/17/benchmarking-druid.html) about the topic. -The code to reproduce the results in the blog post are all open source. The blog post covers Druid queries on TPC-H data, but you should be able to customize - configuration parameters to your data set. The blog post is a little outdated and uses an older version of Druid, but is still mostly relevant to demonstrate performance. - -## Colocating Druid Processes for a POC - -Not all Druid node processes need to run on separate machines. You can set up a small cluster with colocated processes to load several gigabytes of data. Please note this cluster is not highly available. - -It is recommended you follow the [example production configuration](../configuration/production-cluster.html) for an actual production setup. - -The deep storage to use in this POC example can be S3 or HDFS. - -* node1: [Coordinator](../design/coordinator.html) + metadata store + zookeeper. -Example hardware: EC2 c3.2xlarge node (8 cores, Intel Xeon E5-2680 v2 @ 2.80GHz and 15GB of RAM). - -See [here](../configuration/production-cluster.html) for the runtime.properties. Some example JVM configs for this hardware: - -``` --server --Xmx6g --Xms6g --XX:NewSize=512m --XX:MaxNewSize=512m --XX:+UseConcMarkSweepGC -``` - -* node2: [Broker](../design/broker.html) -Example hardware: EC2 c3.2xlarge node (8 cores, Intel Xeon E5-2680 v2 @ 2.80GHz and 15GB of RAM). -[Example configs](https://github.com/druid-io/druid-benchmark/tree/master/config) (see broker-* files). - -* node3: [Historical](../design/historical.html). -Example hardware: EC2 m3.2xlarge instances (8 cores, Intel Xeon E5-2670 v2 @ 2.50GHz with 160GB SSD and 30GB of RAM) -[Example configs](https://github.com/druid-io/druid-benchmark/tree/master/config) (see compute-* files). - -* node4 (optional): [Real-time](../design/realtime.html) node or [Overlord (Indexing Service)](../design/indexing-service.html) (depending on how you choose to ingest data). -Example hardware: EC2 c3.2xlarge node (8 cores, Intel Xeon E5-2680 v2 @ 2.80GHz and 15GB of RAM). - -For the real-time node, see [here](../configuration/production-cluster.html) for the runtime.properties. Use with the following JVM configs: - -``` --server --Xmx8g --Xms8g --XX:NewSize=1g --XX:MaxNewSize=1g --XX:+UseConcMarkSweepGC -``` - -For small ingest workloads, you can run the overlord in local mode to load your data. -To do so, see [here](../configuration/simple-cluster.md#overlord-node-indexing-service) for the runtime.properties. Use with the following JVM configs: - -``` --server --Xmx2g --Xms2g --XX:NewSize=256m --XX:MaxNewSize=256m -``` - -The size of the runner javaOpts can be bumped up: - -``` -druid.indexer.runner.javaOpts="-server -Xmx6g -Xms6g -XX:NewSize=256m -XX:MaxNewSize=256m" -``` - -The coordination pieces (coordinator, metadata store, ZK) can be colocated on the same node. These processes do not require many resources, even for reasonably large clusters. - -You can add more historical nodes if your data doesn't fit on a single machine. diff --git a/docs/content/operations/other-hadoop.md b/docs/content/operations/other-hadoop.md index 296488e5278c..e0c10859a1a0 100644 --- a/docs/content/operations/other-hadoop.md +++ b/docs/content/operations/other-hadoop.md @@ -19,12 +19,12 @@ To make this work, follow the steps below Example: -Suppose you specify `druid.extensions.hadoopDependenciesDir=/usr/local/druid/hadoop_dependencies`, and you want to prepare both `hadoop-client` 2.3.0 and 2.4.0 for Druid, +Suppose you specify `druid.extensions.hadoopDependenciesDir=/usr/local/druid/hadoop-dependencies`, and you want to prepare both `hadoop-client` 2.3.0 and 2.4.0 for Druid, -Then you can either use [pull-deps](../pull-deps.html) or manually set up Hadoop dependencies directories such that under ```hadoop_dependencies```, it looks like this, +Then you can either use [pull-deps](../pull-deps.html) or manually set up Hadoop dependencies directories such that under ```hadoop-dependencies```, it looks like this, ``` -hadoop_dependencies/ +hadoop-dependencies/ └── hadoop-client ├── 2.3.0 │   ├── activation-1.1.jar @@ -44,7 +44,7 @@ hadoop_dependencies/ ..... lots of jars ``` -As you can see, under ```hadoop-client```, there are two sub-directories, each denotes a version of ```hadoop-client```. During runtime, Druid will look for these directories and load appropriate ```hadoop-client``` based on `hadoopDependencyCoordinates` passed to [Hadoop Index Task](../misc/tasks.html). +As you can see, under ```hadoop-client```, there are two sub-directories, each denotes a version of ```hadoop-client```. During runtime, Druid will look for these directories and load appropriate ```hadoop-client``` based on `hadoopDependencyCoordinates` passed to [Hadoop Index Task](../ingestion/tasks.html). ### Append your Hadoop jars to the Druid classpath @@ -58,7 +58,7 @@ If you really don't like the way above, and you just want to use one specific Ha The default version of Hadoop bundled with Druid is 2.3. -To override the default Hadoop version, both the Hadoop Index Task and the standalone Hadoop indexer support the parameter `hadoopDependencyCoordinates`(See [Index Hadoop Task](../misc/tasks.html). You can pass another set of Hadoop coordinates through this parameter (e.g. You can specify coordinates for Hadoop 2.4.0 as `["org.apache.hadoop:hadoop-client:2.4.0"]`), which will overwrite the default Hadoop coordinates Druid uses. +To override the default Hadoop version, both the Hadoop Index Task and the standalone Hadoop indexer support the parameter `hadoopDependencyCoordinates`(See [Index Hadoop Task](../ingestion/tasks.html). You can pass another set of Hadoop coordinates through this parameter (e.g. You can specify coordinates for Hadoop 2.4.0 as `["org.apache.hadoop:hadoop-client:2.4.0"]`), which will overwrite the default Hadoop coordinates Druid uses. The Hadoop Index Task takes this parameter has part of the task JSON and the standalone Hadoop indexer takes this parameter as a command line argument. diff --git a/docs/content/operations/performance-faq.md b/docs/content/operations/performance-faq.md index e6f6ffff6d39..70c495b9d38f 100644 --- a/docs/content/operations/performance-faq.md +++ b/docs/content/operations/performance-faq.md @@ -2,10 +2,14 @@ layout: doc_page --- +# Performance FAQ + ## I can't match your benchmarked results + Improper configuration is by far the largest problem we see people trying to deploy Druid. The example configurations listed in the tutorials are designed for a small volume of data where all nodes are on a single machine. The configs are extremely poor for actual production use. ## What should I set my JVM heap? + The size of the JVM heap really depends on the type of Druid node you are running. Below are a few considerations. [Broker nodes](../design/broker.html) uses the JVM heap mainly to merge results from historicals and real-times. Brokers also use off-heap memory and processing threads for groupBy queries. We recommend 20G-30G of heap here. diff --git a/docs/content/operations/pull-deps.md b/docs/content/operations/pull-deps.md index 4b24c9da6b77..9c1a87b6b9c3 100644 --- a/docs/content/operations/pull-deps.md +++ b/docs/content/operations/pull-deps.md @@ -68,8 +68,8 @@ extensions ``` ``` -tree hadoop_dependencies -hadoop_dependencies/ +tree hadoop-dependencies +hadoop-dependencies/ └── hadoop-client ├── 2.3.0 │   ├── activation-1.1.jar diff --git a/docs/content/operations/recommendations.md b/docs/content/operations/recommendations.md index d938285256ff..0673b62dfd23 100644 --- a/docs/content/operations/recommendations.md +++ b/docs/content/operations/recommendations.md @@ -13,10 +13,6 @@ We recommend using UTC timezone for all your events and across on your nodes, no SSDs are highly recommended for historical and real-time nodes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory. -# Provide Columns Names in Lexicographic Order - -Although Druid supports schema-less ingestion of dimensions, because of [https://github.com/druid-io/druid/issues/658](https://github.com/druid-io/druid/issues/658), you may sometimes get bigger segments than necessary. To ensure segments are as compact as possible, providing dimension names in lexicographic order is recommended. - # Use Timeseries and TopN Queries Instead of GroupBy Where Possible Timeseries and TopN queries are much more optimized and significantly faster than groupBy queries for their designed use cases. Issuing multiple topN or timeseries queries from your application can potentially be more efficient than a single groupBy query. diff --git a/docs/content/operations/rule-configuration.md b/docs/content/operations/rule-configuration.md index e93e79116d2f..75318300cfd2 100644 --- a/docs/content/operations/rule-configuration.md +++ b/docs/content/operations/rule-configuration.md @@ -129,4 +129,4 @@ The interval of a segment will be compared against the specified period. The per # Permanently Deleting Data Druid can fully drop data from the cluster, wipe the metadata store entry, and remove the data from deep storage for any segments that are - marked as unused (segments dropped from the cluster via rules are always marked as unused). You can submit a [kill task](../misc/tasks.html) to the [indexing service](../design/indexing-service.html) to do this. + marked as unused (segments dropped from the cluster via rules are always marked as unused). You can submit a [kill task](../ingestion/tasks.html) to the [indexing service](../design/indexing-service.html) to do this. diff --git a/docs/content/querying/aggregations.md b/docs/content/querying/aggregations.md index fbd6d5fbfe34..fcce78d45d10 100644 --- a/docs/content/querying/aggregations.md +++ b/docs/content/querying/aggregations.md @@ -82,6 +82,9 @@ Computes an arbitrary JavaScript function over a set of columns (both metrics an All JavaScript functions must return numerical values. +JavaScript aggregators are much slower than native Java aggregators and if performance is critical, you should implement +your functionality as a native Java aggregator. + ```json { "type": "javascript", "name": "", diff --git a/docs/content/querying/caching.md b/docs/content/querying/caching.md new file mode 100644 index 000000000000..74f7083f489c --- /dev/null +++ b/docs/content/querying/caching.md @@ -0,0 +1,23 @@ +--- +layout: doc_page +--- +# Query Caching + +Druid supports query result caching through an LRU cache. Results are stored on a per segment basis, along with the +parameters of a given query. This allows Druid to return final results based partially on segment results in the cache and partially +on segment results from scanning historical/real-time segments. + +Segment results can be stored in a local heap cache or in an external distributed key/value store. Segment query caches +can be enabled at either the Historical and Broker level (it is not recommended to enable caching on both). + +## Query caching on Brokers + +Enabling caching on the broker can yield faster results than if query caches were enabled on Historicals for small clusters. This is +the recommended setup for smaller production clusters (< 20 servers). Take note that when caching is enabled on the Broker, +results from Historicals are returned on a per segment basis, and Historicals will be able to do any local result merging. + +## Query caching on Historicals + +Larger production clusters should enable caching only on the Historicals to avoid having to use Brokers to merge all query +results. Enabling caching on the Historicals enables the Historicals do their own local result merging, and puts less strain +on the Brokers. diff --git a/docs/content/operations/multitenancy.md b/docs/content/querying/multitenancy.md similarity index 100% rename from docs/content/operations/multitenancy.md rename to docs/content/querying/multitenancy.md diff --git a/docs/content/querying/multi-valued-dimensions.md b/docs/content/querying/optimizations.md similarity index 97% rename from docs/content/querying/multi-valued-dimensions.md rename to docs/content/querying/optimizations.md index 65c4fa3d558c..d5dc045287b1 100644 --- a/docs/content/querying/multi-valued-dimensions.md +++ b/docs/content/querying/optimizations.md @@ -2,6 +2,10 @@ layout: doc_page --- +This document contains additional query optimizations for certain types of queries. + +# Multi-value Dimensions + Druid supports "multi-valued" dimensions. See the section on multi-valued columns in [segments](../design/segments.html) for internal representation details. This document describes the behavior of groupBy(topN has similar behavior) queries on multi-valued dimensions when they are used as a dimension being grouped by. Suppose, you have a dataSource with a segment that contains following rows with a multi-valued dimension called tags. diff --git a/docs/content/development/select-query.md b/docs/content/querying/select-query.md similarity index 100% rename from docs/content/development/select-query.md rename to docs/content/querying/select-query.md diff --git a/docs/content/toc.textile b/docs/content/toc.textile index c445141344ca..cd5f46e670e3 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -5,18 +5,22 @@ h2. Getting Started * "Concepts":../design/ -* "Hello, Druid":../tutorials/tutorial-a-first-look-at-druid.html -* "Tutorials":../tutorials/index.html -* "Evaluate Druid":../misc/evaluate.html +* "Quickstart":../tutorials/quickstart.html +* "Loading Data":../tutorials/ingestion.html +** "Loading from Files":../tutorials/tutorial-batch.html +** "Loading from Streams":../tutorials/tutorial-streams.html +** "Loading from Kafka":../tutorials/tutorial-kafka.html +** "Clustering":../tutorials/cluster.html h2. Data Ingestion -* "Overview":../ingestion/overview.html * "Data Formats":../ingestion/data-formats.html * "Data Schema":../ingestion/index.html * "Schema Design":../ingestion/schema-design.html * "Schema Changes":../ingestion/schema-changes.html -* "Realtime Ingestion":../ingestion/realtime-ingestion.html -* "Batch Ingestion":../ingestion/batch-ingestion.html +* "Batch File Ingestion":../ingestion/batch-ingestion.html +* "Stream Ingestion":../ingestion/stream-ingestion.html +** "Stream Push":../ingestion/stream-push.html +** "Stream Pull":../ingestion/stream-pull.html * "Updating Existing Data":../ingestion/update-existing-data.html * "FAQ":../ingestion/faq.html @@ -29,6 +33,7 @@ h2. Querying * "Segment Metadata":../querying/segmentmetadataquery.html * "DataSource Metadata":../querying/datasourcemetadataquery.html * "Search":../querying/searchquery.html +* "Select":../querying/select-query.html * Components ** "Datasources":../querying/datasource.html ** "Filters":../querying/filters.html @@ -39,7 +44,9 @@ h2. Querying ** "Context":../querying/query-context.html * "SQL":../querying/sql.html * "Joins":../querying/joins.html -* "Multi-Valued Dimensions":../querying/multi-valued-dimensions.html +* "Optimizations":../querying/optimizations.html +* "Multitenancy":../querying/multitenancy.html +* "Caching":../querying/caching.html h2. Design * "Overview":../design/design.html @@ -64,7 +71,6 @@ h2. Operations * "Alerts":../operations/alerts.html * "Updating the Cluster":../operations/rolling-updates.html * "Different Hadoop Versions":../operations/other-hadoop.html -* "Multitenancy Considerations":../operations/multitenancy.html * "Performance FAQ":../operations/performance-faq.html h2. Configuration @@ -75,7 +81,6 @@ h2. Configuration * "Broker":../configuration/broker.html * "Realtime":../configuration/realtime.html * "Configuring Logging":../configuration/logging.html -* "Simple Cluster Configuration":../configuration/simple-cluster.html * "Production Cluster Configuration":../configuration/production-cluster.html * "Production Hadoop Configuration":../configuration/hadoop.html * "Production Zookeeper Configuration":../configuration/zookeeper.html @@ -90,11 +95,10 @@ h2. Development * Experimental Features ** "Overview":../development/experimental.html ** "Geographic Queries":../development/geo.html -** "Select Query":../development/select-query.html ** "Approximate Histograms and Quantiles":../development/approximate-histograms.html -** "Datasketches based Aggregators":../development/datasketches-aggregators.html -** "Router node":../development/router.html -** "New Kafka Firehose":../development/kafka-simple-consumer-firehose.html +** "Datasketches":../development/datasketches-aggregators.html +** "Router":../development/router.html +** "Kafka Simple Consumer Firehose":../development/kafka-simple-consumer-firehose.html h2. Misc * "Papers & Talks":../misc/papers-and-talks.html diff --git a/docs/content/tutorials/booting-a-production-cluster.md b/docs/content/tutorials/booting-a-production-cluster.md deleted file mode 100644 index 934b0fbb4cf3..000000000000 --- a/docs/content/tutorials/booting-a-production-cluster.md +++ /dev/null @@ -1,10 +0,0 @@ ---- -layout: doc_page ---- - -# Booting a Druid Cluster -[Loading Your Data](../tutorials/tutorial-loading-batch-data.html) and [All About Queries](../tutorials/tutorial-all-about-queries.html) contain recipes to boot a small druid cluster on localhost. However, when it's time to run a more realistic setup—for production or just for testing production—you'll want to find a way to start the cluster on multiple hosts. This document describes two different ways to do this: manually, or as a cloud service via Apache Whirr. - -## Manually Booting a Druid Cluster -You can provision individual servers, loading Druid onto each machine (or building it) and setting the required configuration for each type of node. You'll also have to set up required external dependencies. Then you'll have to start each node. This process is outlined in [Tutorial: The Druid Cluster](../tutorials/tutorial-the-druid-cluster.html). - diff --git a/docs/content/tutorials/cluster.md b/docs/content/tutorials/cluster.md new file mode 100644 index 000000000000..9be27ed84eac --- /dev/null +++ b/docs/content/tutorials/cluster.md @@ -0,0 +1,340 @@ +--- +layout: doc_page +--- + +# Clustering + +Druid is designed to be deployed as a scalable, fault-tolerant cluster. + +In this document, we'll set up a simple cluster and discuss how it can be further configured to meet +your needs. This simple cluster will feature scalable, fault-tolerant servers for Historicals and MiddleManagers, and a single +coordination server to host the Coordinator and Overlord processes. In production, we recommend deploying Coordinators and Overlords in a fault-tolerant +configuration as well. + +## Select hardware + +The Coordinator and Overlord processes can be co-located on a single server that is responsible for handling the metadata and coordination needs of your cluster. +The equivalent of an AWS [m3.xlarge](https://aws.amazon.com/ec2/instance-types/#M3) is sufficient for most clusters. This +hardware offers: + +- 4 vCPUs +- 15 GB RAM +- 80 GB SSD storage + +Historicals and MiddleManagers can be colocated on a single server to handle the actual data in your cluster. These servers benefit greatly from CPU, RAM, +and SSDs. The equivalent of an AWS [r3.2xlarge](https://aws.amazon.com/ec2/instance-types/#r3) is a +good starting point. This hardware offers: + +- 8 vCPUs +- 61 GB RAM +- 160 GB SSD storage + +Druid Brokers accept queries and farm them out to the rest of the cluster. They also optionally maintain an +in-memory query cache. These servers benefit greatly from CPU and RAM, and can also be deployed on +the equivalent of an AWS [r3.2xlarge](https://aws.amazon.com/ec2/instance-types/#r3). This hardware +offers: + +- 8 vCPUs +- 61 GB RAM +- 160 GB SSD storage + +You can consider co-locating any open source UIs or query libraries on the same server that the Broker is running on. + +Very large clusters should consider selecting larger servers. + +## Select OS + +We recommend running your favorite Linux distribution. You will also need: + + * Java 7 or better + +Your OS package manager should be able to help for both Java. If your Ubuntu-based OS +does not have a recent enough version of Java, WebUpd8 offers [packages for those +OSes](http://www.webupd8.org/2012/09/install-oracle-java-8-in-ubuntu-via-ppa.html). + +## Download the distribution + +First, download and unpack the release archive. It's best to do this on a single machine at first, +since you will be editing the configurations and then copying the modified distribution out to all +of your servers. + +```bash +curl -O http://static.druid.io/artifacts/releases/druid-0.9.0-bin.tar.gz +tar -xzf druid-0.9.0-bin.tar.gz +cd druid-0.9.0 +``` + +In this package, you'll find: + + +* `LICENSE` - the license files. +* `bin/` - scripts related to the [single-machine quickstart](quickstart.md). +* `conf/*` - template configurations for a clustered setup. +* `conf-quickstart/*` - configurations for the [single-machine quickstart](quickstart.md). +* `extensions/*` - all Druid extensions. +* `hadoop-dependencies/*` - Druid Hadoop dependencies. +* `lib/*` - all included software packages for core Druid. +* `quickstart/*` - files related to the [single-machine quickstart](quickstart.md). + +We'll be editing the files in `conf/` in order to get things running. + +## Configure deep storage + +Druid relies on a distributed filesystem or large object (blob) store for data storage. The most +commonly used deep storage implementations are S3 (popular for those on AWS) and HDFS (popular if +you already have a Hadoop deployment). + +### S3 + +In `conf/druid/_common/common.runtime.properties`, + +- Set `druid.extensions.loadList=["druid-s3-extensions"]`. + +- Comment out the configurations for local storage under "Deep Storage" and "Indexing service logs". + +- Uncomment and configure appropriate values in the "For S3" sections of "Deep Storage" and +"Indexing service logs". + +After this, you should have made the following changes: + +``` +druid.extensions.loadList=["druid-s3-extensions"] + +#druid.storage.type=local +#druid.storage.storageDirectory=var/druid/segments + +druid.storage.type=s3 +druid.storage.bucket=your-bucket +druid.storage.baseKey=druid/segments +druid.s3.accessKey=... +druid.s3.secretKey=... + +#druid.indexer.logs.type=file +#druid.indexer.logs.directory=var/druid/indexing-logs + +druid.indexer.logs.type=s3 +druid.indexer.logs.s3Bucket=your-bucket +druid.indexer.logs.s3Prefix=druid/indexing-logs +``` + +### HDFS + +In `conf/druid/_common/common.runtime.properties`, + +- Set `druid.extensions.loadList=["io.druid.extensions:druid-hdfs-storage"]`. + +- Comment out the configurations for local storage under "Deep Storage" and "Indexing service logs". + +- Uncomment and configure appropriate values in the "For HDFS" sections of "Deep Storage" and +"Indexing service logs". + +After this, you should have made the following changes: + +``` +druid.extensions.loadList=["druid-hdfs-storage"] + +#druid.storage.type=local +#druid.storage.storageDirectory=var/druid/segments + +druid.storage.type=hdfs +druid.storage.storageDirectory=/druid/segments + +#druid.indexer.logs.type=file +#druid.indexer.logs.directory=var/druid/indexing-logs + +druid.indexer.logs.type=hdfs +druid.indexer.logs.directory=/druid/indexing-logs +``` + +Also, + +- Place your Hadoop configuration XMLs (core-site.xml, hdfs-site.xml, yarn-site.xml, +mapred-site.xml) on the classpath of your Druid nodes. You can do this by copying them into +`conf/druid/_common/`. + +## Configure Tranquility Server (optional) + +Data streams can be sent to Druid through a simple HTTP API powered by Tranquility +Server. If you will be using this functionality, then at this point you should [configure +Tranquility Server](../ingestion/stream-ingestion.html#server). + +## Configure Tranquility Kafka (optional) + +Druid can consuming streams from Kafka through Tranquility Kafka. If you will be +using this functionality, then at this point you should +[configure Tranquility Kafka](../ingestion/stream-ingestion.html#kafka). + +## Configure for connecting to Hadoop (optional) + +If you will be loading data from a Hadoop cluster, then at this point you should configure Druid to be aware +of your cluster: + +- Update `druid.indexer.task.hadoopWorkingPath` in `conf/middleManager/runtime.properties` to +a path on HDFS that you'd like to use for temporary files required during the indexing process. +`druid.indexer.task.hadoopWorkingPath=/tmp/druid-indexing` is a common choice. + +- Place your Hadoop configuration XMLs (core-site.xml, hdfs-site.xml, yarn-site.xml, +mapred-site.xml) on the classpath of your Druid nodes. You can do this by copying them into +`conf/druid/_common/core-site.xml`, `conf/druid/_common/hdfs-site.xml`, and so on. + +Note that you don't need to use HDFS deep storage in order to load data from Hadoop. For example, if +your cluster is running on Amazon Web Services, we recommend using S3 for deep storage even if you +are loading data using Hadoop or Elastic MapReduce. + +For more info, please see [batch ingestion](../ingestion/batch-ingestion.html). + +## Configure addresses for Druid coordination + +In this simple cluster, you will deploy a single Druid Coordinator, a +single Druid Overlord, a single ZooKeeper instance, and an embedded Derby metadata store on the same server. + +In `conf/druid/_common/common.runtime.properties`, replace +"zk.host.ip" with the IP address of the machine that runs your ZK instance: + +- `druid.zk.service.host` + +In `conf/_common/common.runtime.properties`, replace +"metadata.store.ip" with the IP address of the machine that you will use as your metadata store: + +- `druid.metadata.storage.connector.connectURI` +- `druid.metadata.storage.connector.host` + +```note-caution +In production, we recommend running 2 servers, each running a Druid Coordinator +and a Druid Overlord. We also recommend running a ZooKeeper cluster on its own dedicated hardware, +as well as replicated [metadata +storage](http://druid.io/docs/latest/dependencies/metadata-storage.html) such as MySQL or +PostgreSQL, on its own dedicated hardware. +``` + +## Tune Druid processes that serve queries + +Druid Historicals and MiddleManagers can be co-located on the same hardware. Both Druid processes benefit greatly from +being tuned to the hardware they run on. If you are running Tranquility Server or Kafka, you can also colocate Tranquility with these two Druid processes. +If you are using [r3.2xlarge](https://aws.amazon.com/ec2/instance-types/#r3) +EC2 instances, or similar hardware, the configuration in the distribution is a +reasonable starting point. + +If you are using different hardware, we recommend adjusting configurations for your specific +hardware. The most commonly adjusted configurations are: + +- `-Xmx` and `-Xms` +- `druid.server.http.numThreads` +- `druid.processing.buffer.sizeBytes` +- `druid.processing.numThreads` +- `druid.query.groupBy.maxIntermediateRows` +- `druid.query.groupBy.maxResults` +- `druid.server.maxSize` and `druid.segmentCache.locations` on Historical Nodes +- `druid.worker.capacity` on MiddleManagers + +```note +Keep -XX:MaxDirectMemory >= numThreads*sizeBytes, otherwise Druid will fail to start up.. +``` + +Please see the Druid [configuration documentation](../configuration/index.html) for a full description of all +possible configuration options. + +## Tune Druid Brokers + +Druid Brokers also benefit greatly from being tuned to the hardware they +run on. If you are using [r3.2xlarge](https://aws.amazon.com/ec2/instance-types/#r3) EC2 instances, +or similar hardware, the configuration in the distribution is a reasonable starting point. + +If you are using different hardware, we recommend adjusting configurations for your specific +hardware. The most commonly adjusted configurations are: + +- `-Xmx` and `-Xms` +- `druid.server.http.numThreads` +- `druid.cache.sizeInBytes` +- `druid.processing.buffer.sizeBytes` +- `druid.processing.numThreads` +- `druid.query.groupBy.maxIntermediateRows` +- `druid.query.groupBy.maxResults` + +```note-caution +Keep -XX:MaxDirectMemory >= numThreads*sizeBytes, otherwise Druid will fail to start up.. +``` + +Please see the Druid [configuration documentation](../configuration/index.html) for a full description of all +possible configuration options. + +## Start Coordinator, Overlord, Zookeeper, and metadata store + +Copy the Druid distribution and your edited configurations to your coordination +server. If you have been editing the configurations on your local machine, you can use *rsync* to +copy them: + +```bash +rsync -az druid-0.9.0/ COORDINATION_SERVER:druid-0.9.0/ +``` + +Log on to your coordination server and install Zookeeper: + +```bash +curl http://www.gtlib.gatech.edu/pub/apache/zookeeper/zookeeper-3.4.6/zookeeper-3.4.6.tar.gz -o zookeeper-3.4.6.tar.gz +tar -xzf zookeeper-3.4.6.tar.gz +cd zookeeper-3.4.6 +cp conf/zoo_sample.cfg conf/zoo.cfg +./bin/zkServer.sh start + +```note-caution +In production, we also recommend running a ZooKeeper cluster on its own dedicated hardware. +``` + +On your coordination server, *cd* into the distribution and start up the coordination services (you should do this in different windows or pipe the log to a file): + +```bash +java `cat conf/druid/coordinator/jvm.config | xargs` -cp conf/druid/_common:conf/druid/coordinator:lib/* io.druid.cli.Main server coordinator +java `cat conf/druid/overlord/jvm.config | xargs` -cp conf/druid/_common:conf/druid/overlord:lib/* io.druid.cli.Main server overlord +``` + +You should see a log message printed out for each service that starts up. You can view detailed logs +for any service by looking in the `var/log/druid` directory using another terminal. + +## Start Historicals and MiddleManagers + +Copy the Druid distribution and your edited configurations to your servers set aside for the Druid Historicals and MiddleManagers. + +On each one, *cd* into the distribution and run this command to start a Data server: + +```bash +java `cat conf/druid/historical/jvm.config | xargs` -cp conf/druid/_common:conf/druid/historical:lib/* io.druid.cli.Main server historical +java `cat conf/druid/middleManager/jvm.config | xargs` -cp conf/druid/_common:conf/druid/middleManager:lib/* io.druid.cli.Main server middleManager +``` + +You can add more servers with Druid Historicals and MiddleManagers as needed. + +```note-info +For clusters with complex resource allocation needs, you can break apart Historicals and MiddleManagers and scale the components individually. +This also allows you take advantage of Druid's built-in MiddleManager +autoscaling facility. +``` + +If you are doing push-based stream ingestion with Kafka or over HTTP, you can also start Tranquility server on the same +hardware that holds MiddleManagers and Historicals. For large scale production, MiddleManagers and Tranquility server +can still be co-located. If you are running Tranquility (not server) with a stream processor, you can co-locate +Tranquility with the stream processor and not require Tranquility server. + +```bash +curl -O http://static.druid.io/tranquility/releases/tranquility-distribution-0.7.2.tgz +tar -xzf tranquility-distribution-0.7.2.tgz +cd tranquility-distribution-0.7.2.tgz +bin/tranquility -configFile /conf/tranquility/.json +``` + +## Start Druid Broker + +Copy the Druid distribution and your edited configurations to your servers set aside for the Druid Brokers. + +On each one, *cd* into the distribution and run this command to start a Broker (you want to pipe the output to a log file): + +```bash +java `cat conf/druid/broker/jvm.config | xargs` -cp conf/druid/_common:conf/druid/broker:lib/* io.druid.cli.Main server broker +``` + +You can add more Brokers as needed based on query load. + +## Loading data + +Congratulations, you now have a Druid cluster! The next step is to learn about recommended ways to load data into +Druid based on your use case. Read more about [loading data](ingestion.html). diff --git a/docs/content/tutorials/examples.md b/docs/content/tutorials/examples.md deleted file mode 100644 index e2279bcdfd29..000000000000 --- a/docs/content/tutorials/examples.md +++ /dev/null @@ -1,71 +0,0 @@ ---- -layout: doc_page ---- -Examples -======== - -The examples on this page are setup in order to give you a feel for what Druid does in practice. They are quick demos of Druid based on [CliRealtimeExample](https://github.com/druid-io/druid/blob/master/services/src/main/java/io/druid/cli/CliRealtimeExample.java). While you wouldn’t run it this way in production you should be able to see how ingestion works and the kind of exploratory queries that are possible. Everything that can be done on your box here can be scaled out to 10’s of billions of events and terabytes of data per day in a production cluster while still giving the snappy responsive exploratory queries. - -Installing Standalone Druid ---------------------------- - -There are two options for installing standalone Druid. Building from source, and downloading the Druid Standalone Kit (DSK). - -### Building from source - -Clone Druid and build it: - -``` bash -git clone https://github.com/druid-io/druid.git druid -cd druid -git fetch --tags -git checkout druid- -mvn clean package -``` - -### Downloading the DSK (Druid Standalone Kit) - -[Download](http://druid.io/downloads.html) a stand-alone tarball and run it: - -``` bash -tar -xzf druid--bin.tar.gz -cd druid- -``` - -Twitter Example ---------------- - -For a full tutorial based on the twitter example, check out this [Twitter Tutorial](twitter-tutorial.html). - -This Example uses a feature of Twitter that allows for sampling of it’s stream. We sample the Twitter stream via our [TwitterSpritzerFirehoseFactory](https://github.com/druid-io/druid/blob/master/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java) class and use it to simulate the kinds of data you might ingest into Druid. Then, with the client part, the sample shows what kinds of analytics explorations you can do during and after the data is loaded. - -### What you’ll learn -* See how large amounts of data gets ingested into Druid in real-time -* Learn how to do fast, interactive, analytics queries on that real-time data - -### What you need -* A build of standalone Druid with the Twitter example (see above) -* A Twitter username and password. - -### What you’ll do - -See [Twitter Tutorial](twitter-tutorial.html) - -Rand Example ------------- - -This uses `RandomFirehoseFactory` which emits a stream of random numbers (outColumn, a positive double) with timestamps along with an associated token (target). This provides a timeseries that requires no network access for demonstration, characterization, and testing. The generated tuples can be thought of as asynchronously produced triples (timestamp, outColumn, target) where the timestamp varies depending on speed of processing. - -In a terminal window, (NOTE: If you are using the cloned Github repository these scripts are in ./examples/bin) start the server with: - -``` bash -./run_example_server.sh # type rand when prompted -``` - -In another terminal window: - -``` bash -./run_example_client.sh # type rand when prompted -``` - -The result of the client query is in JSON format. The client makes a REST request using the program `curl` which is usually installed on Linux, Unix, and OSX by default. diff --git a/docs/content/tutorials/firewall.md b/docs/content/tutorials/firewall.md deleted file mode 100644 index ce6301a06683..000000000000 --- a/docs/content/tutorials/firewall.md +++ /dev/null @@ -1,172 +0,0 @@ ---- -layout: doc_page ---- - -What to Do When You Have a Firewall ------------------------------------ -When you are behind a firewall, if the IRC wikipedia channels that feed realtime data into Druid are not accessible, then there is nothing you can do. If IRC channels are accessible, but downloading Geolite DB from maxmind is firewalled, you can workaround this challenge by making GeoLite DB dependency available offline, see below. - -## Making the Wikipedia Example GeoLite DB Dependency Available Offline -1. Download GeoLite2 City DB from http://dev.maxmind.com/geoip/geoip2/geolite2/ -2. Copy and extract the DB to *`java.io.tmpdir`*`/io.druid.segment.realtime.firehose.WikipediaIrcDecoder.GeoLite2-City.mmdb`; e.g. `/tmp/io.druid.segment.realtime.firehose.WikipediaIrcDecoder.GeoLite2-City.mmdb` - - **Note**: depending on the machine's reboot policy, if the `java.io.tmpdir` resolves to the `/tmp` directory, you may have to create this file again in the `tmp` directory after a machine reboot - -## Loading the Data into Druid directly from Kafka -As an alternative to reading the data from the IRC channels, which is a challenge to try to do it from behind a firewall, we will use Kafka to stream the data to Druid. To do so, we will need to: - -1. Configure the Wikipedia example to read streaming data from Kafka -2. Set up and configure Kafka - -#### Wikipedia Example Configuration -1. In your favorite editor, open the file `druid-/examples/wikipedia/wikipedia_realtime.spec` -2. Backup the file, if necessary, then replace the file content with the following: - - ```json - [ - { - "dataSchema": { - "dataSource": "wikipedia", - "parser": { - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - } - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE" - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "kafka-0.8", - "consumerProps": { - "zookeeper.connect": "localhost:2181", - "zookeeper.connection.timeout.ms": "15000", - "zookeeper.session.timeout.ms": "15000", - "zookeeper.sync.time.ms": "5000", - "group.id": "druid-example", - "fetch.message.max.bytes": "1048586", - "auto.offset.reset": "largest", - "auto.commit.enable": "false" - }, - "feed": "wikipedia" - }, - "plumber": { - "type": "realtime" - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 500000, - "intermediatePersistPeriod": "PT10m", - "windowPeriod": "PT10m", - "basePersistDirectory": "/tmp/realtime/basePersist", - "rejectionPolicy": { - "type": "messageTime" - } - } - } - ] - ``` - -3. Refer to the [Running Example Scripts](#running-example-scripts) section to start the example Druid Realtime node by issuing the following from within your Druid directory: - - ```bash - ./run_example_server.sh - ``` - -#### Kafka Setup and Configuration -1. Download Kafka - - For this tutorial we will [download Kafka 0.8.2.1] - (https://www.apache.org/dyn/closer.cgi?path=/kafka/0.8.2.1/kafka_2.10-0.8.2.1.tgz) - - ```bash - tar -xzf kafka_2.10-0.8.2.1.tgz - cd kafka_2.10-0.8.2.1 - ``` - -2. Start Kafka - - **First, launch ZooKeeper** (refer to the [Set up Zookeeper](#set-up-zookeeper) section for details), then start the Kafka server (in a separate console): - - ```bash - ./bin/kafka-server-start.sh config/server.properties - ``` - -3. Create a topic named `wikipedia` - - ```bash - ./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic wikipedia - ``` - -4. Launch a console producer for the topic `wikipedia` - - ```bash - ./bin/kafka-console-producer.sh --broker-list localhost:9092 --topic wikipedia - ``` - -5. Copy and paste the following data into the terminal where we launched the Kafka console producer in the previous step: - - ```json - {"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} - {"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} - {"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} - {"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} - {"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} - ``` - -#### Finally -Now, that data has been fed into Druid, refer to the [Running Example Scripts](#running-example-scripts) section to query the real-time node by issuing the following from within the Druid directory: -```bash -./run_example_client.sh -``` - -The [Querying Druid](../querying/querying.md) section also has further querying examples. diff --git a/docs/content/tutorials/index.md b/docs/content/tutorials/index.md deleted file mode 100644 index 7f456bbf0bd6..000000000000 --- a/docs/content/tutorials/index.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -layout: doc_page ---- -# Druid Tutorials - -We have a series of tutorials to help new users learn to use and operate Druid. We will be adding new tutorials to this list periodically and we encourage the community to contribute tutorials of their own. - -## Tutorials - -* **[A First Look at Druid](../tutorials/tutorial-a-first-look-at-druid.html)** -This tutorial covers a very basic introduction to Druid. You will load some streaming wikipedia data and learn about basic queries. - -* **[The Druid Cluster](../tutorials/tutorial-the-druid-cluster.html)** -This tutorial goes over the basic operations of the nodes in a Druid cluster and how to start the nodes. - -* **[Loading Streaming Data](../tutorials/tutorial-loading-streaming-data.html)** -This tutorial covers loading streaming data into Druid. - -* **[Loading Batch Data](../tutorials/tutorial-loading-batch-data.html)** -This tutorial covers loading static (batch) data into Druid. - diff --git a/docs/content/tutorials/ingestion.md b/docs/content/tutorials/ingestion.md new file mode 100644 index 000000000000..ac85f24d0946 --- /dev/null +++ b/docs/content/tutorials/ingestion.md @@ -0,0 +1,42 @@ +--- +layout: doc_page +--- + +# Loading Data + +## Choosing an ingestion method + +Druid supports streaming (real-time) and file-based (batch) ingestion methods. The most +popular configurations are: + +- [Files](batch-ingestion.html) - Load data from HDFS, S3, local files, or any supported Hadoop +filesystem in batches. We recommend this method if your dataset is already in flat files. + +- [Stream push](stream-ingestion.html#stream-push) - Push a data stream into Druid in real-time +using [Tranquility](http://github.com/druid-io/tranquility), a client library for sending streams +to Druid. We recommend this method if your dataset originates in a streaming system like Kafka, +Storm, Spark Streaming, or your own system. + +- [Stream pull](stream-ingestion.html#stream-pull) - Pull a data stream directly from an external +data source into Druid using Realtime Nodes. + +## Getting started + +The easiest ways to get started with loading your own data are the three included tutorials. + +- [Files-based tutorial](tutorial-batch.html) showing you how to load files from your local disk. +- [Streams-based tutorial](tutorial-streams.html) showing you how to push data over HTTP. +- [Kafka-based tutorial](tutorial-kafka.html) showing you how to load data from Kafka. + +## Hybrid batch/streaming + +You can combine batch and streaming methods in a hybrid batch/streaming architecture. In a hybrid architecture, +you use a streaming method to do initial ingestion, and then periodically re-ingest older data in batch mode +(typically every few hours, or nightly). When Druid re-ingests data for a time range, the new data automatically +replaces the data from the earlier ingestion. + +All streaming ingestion methods currently supported by Druid do introduce the possibility of dropped or duplicated +messages in certain failure scenarios, and batch re-ingestion eliminates this potential source of error for +historical data. + +Batch re-ingestion also gives you the option to re-ingest your data if you needed to revise it for any reason. diff --git a/docs/content/tutorials/quickstart.md b/docs/content/tutorials/quickstart.md new file mode 100644 index 000000000000..bff14cc33aeb --- /dev/null +++ b/docs/content/tutorials/quickstart.md @@ -0,0 +1,242 @@ +--- +layout: doc_page +--- + +# Druid Quickstart + +In this quickstart, we will download Druid, set up it up on a single machine, load some data, and query the data. + +## Prerequisites + +You will need: + + * Java 7 or higher + * Linux, Mac OS X, or other Unix-like OS (Windows is not supported) + * 8G of RAM + * 2 vCPUs + +On Mac OS X, you can use [Oracle's JDK +8](http://www.oracle.com/technetwork/java/javase/downloads/jdk8-downloads-2133151.html) to install +Java. + +On Linux, your OS package manager should be able to help for Java. If your Ubuntu- +based OS does not have a recent enough version of Java, WebUpd8 offers [packages for those +OSes](http://www.webupd8.org/2012/09/install-oracle-java-8-in-ubuntu-via-ppa.html). + +## Getting started + +To install Druid, issue the following commands in your terminal: + +```bash +curl -O http://static.druid.io/artifacts/releases/druid-0.9.0-bin.tar.gz +tar -xzf druid-0.9.0-bin.tar.gz +cd druid-0.9.0 +``` + +In the package, you should find: + +* `LICENSE` - the license files. +* `bin/` - scripts useful for this quickstart. +* `conf/*` - template configurations for a clustered setup. +* `conf-quickstart/*` - configurations for this quickstart. +* `extensions/*` - all Druid extensions. +* `hadoop-dependencies/*` - Druid Hadoop dependencies. +* `lib/*` - all included software packages for core Druid. +* `quickstart/*` - files useful for this quickstart. + +## Start up Zookeeper + +Druid currently has a dependency on [Apache ZooKeeper](http://zookeeper.apache.org/) for distributed coordination. You'll +need to download and run Zookeeper. + +```bash +curl http://www.gtlib.gatech.edu/pub/apache/zookeeper/zookeeper-3.4.6/zookeeper-3.4.6.tar.gz -o zookeeper-3.4.6.tar.gz +tar -xzf zookeeper-3.4.6.tar.gz +cd zookeeper-3.4.6 +cp conf/zoo_sample.cfg conf/zoo.cfg +./bin/zkServer.sh start +``` + +## Start up Druid services + +With Zookeeper running, return to the druid-0.9.0 directory. In that directory, issue the command: + +```bash +bin/init +``` + +Next, you can start up the Druid processes in different terminal windows. This tutorial runs every Druid process on the same system. In production, +many of these Druid processes can be colocated even in a distributed cluster. + +```bash +java `cat conf-quickstart/druid/historical/jvm.config | xargs` -cp conf-quickstart/druid/_common:conf-quickstart/druid/historical:lib/* io.druid.cli.Main server historical +java `cat conf-quickstart/druid/broker/jvm.config | xargs` -cp conf-quickstart/druid/_common:conf-quickstart/druid/broker:lib/* io.druid.cli.Main server broker +java `cat conf-quickstart/druid/coordinator/jvm.config | xargs` -cp conf-quickstart/druid/_common:conf-quickstart/druid/coordinator:lib/* io.druid.cli.Main server coordinator +java `cat conf-quickstart/druid/overlord/jvm.config | xargs` -cp conf-quickstart/druid/_common:conf-quickstart/druid/overlord:lib/* io.druid.cli.Main server overlord +java `cat conf-quickstart/druid/middleManager/jvm.config | xargs` -cp conf-quickstart/druid/_common:conf-quickstart/druid/middleManager:lib/* io.druid.cli.Main server middleManager +``` + +You should see a log message printed out for each service that starts up. + +Later on, if you'd like to stop the services, CTRL-C to exit from the running java processes. If you +want a clean start after stopping the services, delete the `var` directory and run the `init` script again. + +Once every service has started, you are now ready to load data. + +## Load batch data + +We've included a sample of Wikipedia edits from September 12, 2015 to get you started. + +```note-info +This section shows you how to load data in batches, but you can skip ahead to learn how to [load +streams in real-time](quickstart.html#load-streaming-data). Druid's streaming ingestion can load data +with virtually no delay between events occurring and being available for queries. +``` + +The [dimensions](https://en.wikipedia.org/wiki/Dimension_%28data_warehouse%29) (attributes you can +filter and split on) in the Wikipedia dataset, other than time, are: + + * channel + * cityName + * comment + * countryIsoCode + * countryName + * isAnonymous + * isMinor + * isNew + * isRobot + * isUnpatrolled + * metroCode + * namespace + * page + * regionIsoCode + * regionName + * user + +The [measures](https://en.wikipedia.org/wiki/Measure_%28data_warehouse%29), or *metrics* as they are known in Druid (values you can aggregate) +in the Wikipedia dataset are: + + * count + * added + * deleted + * delta + * user_unique + +To load this data into Druid, you can submit an *ingestion task* pointing to the file. We've included +a task that loads the `wikiticker-2015-09-12-sampled.json` file included in the archive. To submit +this task, POST it to Druid in a new terminal window from the druid-0.9.0 directory: + +```bash +curl -X 'POST' -H 'Content-Type:application/json' -d @quickstart/wikiticker-index.json localhost:8090/druid/indexer/v1/task +``` + +Which will print the ID of the task if the submission was successful: + +```base +{"task":"index_hadoop_wikipedia_2013-10-09T21:30:32.802Z"} +``` + +To view the status of your ingestion task, go to your overlord console: +[http://localhost:8090/console.html](http://localhost:8090/console.html). You can refresh the console periodically, and after +the task is successful, you should see a "SUCCESS" status for the task. + +After your ingestion task finishes, the data will be loaded by historical nodes and available for +querying within a minute or two. You can monitor the progress of loading your data in the +coordinator console, by checking whether there is a datasource "wikiticker" with a blue circle +indicating "fully available": [http://localhost:8081/#/](http://localhost:8081/#/). + +Once the data is fully available, you can immediately query it— to see how, skip to the [Query +data](#query-data) section below. Or, continue to the [Load your own data](#load-your-own-data) +section if you'd like to load a different dataset. + +## Load streaming data + +To load streaming data, we are going to push events into Druid +over a simple HTTP API. We will do this use [Tranquility], a high level data producer +library for Druid. + +To download Tranquility, issue the following commands in your terminal: + +```bash +curl -O http://static.druid.io/tranquility/releases/tranquility-distribution-0.7.2.tgz +tar -xzf tranquility-distribution-0.7.2.tgz +cd tranquility-distribution-0.7.2 +``` + +We've included a configuration file in `conf-quickstart/tranquility/server.json` as part of the Druid distribution +for a *metrics* datasource. We're going to start the Tranquility server process, which can be used to push events +directly to Druid. + +``` bash +bin/tranquility server -configFile /conf-quickstart/tranquility/server.json +``` + +```note-info +This section shows you how to load data using Tranquility Server, but Druid also supports a wide +variety of [other streaming ingestion options](ingestion-streams.html#stream-push), including from +popular streaming systems like Kafka, Storm, Samza, and Spark Streaming. +``` + +The [dimensions](https://en.wikipedia.org/wiki/Dimension_%28data_warehouse%29) (attributes you can +filter and split on) for this datasource are flexible. It's configured for *schemaless dimensions*, +meaning it will accept any field in your JSON input as a dimension. + +The metrics (also called +[measures](https://en.wikipedia.org/wiki/Measure_%28data_warehouse%29); values +you can aggregate) in this datasource are: + + * count + * value_sum (derived from `value` in the input) + * value_min (derived from `value` in the input) + * value_max (derived from `value` in the input) + +We've included a script that can generate some random sample metrics to load into this datasource. +To use it, simply run in your Druid distribution repository: + +```bash +bin/generate-example-metrics | curl -XPOST -H'Content-Type: application/json' --data-binary @- http://localhost:8200/v1/post/metrics +``` + +Which will print something like: + +``` +{"result":{"received":25,"sent":25}} +``` + +This indicates that the HTTP server received 25 events from you, and sent 25 to Druid. Note that +this may take a few seconds to finish the first time you run it, as Druid resources must be +allocated to the ingestion task. Subsequent POSTs should complete quickly. + +Once the data is sent to Druid, you can immediately [query it](#query-data). + +## Query data + +### Direct Druid queries + +Druid supports a rich [family of JSON-based +queries](../querying/querying.html). We've included an example topN query +in `quickstart/wikiticker-top-pages.json` that will find the most-edited articles in this dataset: + +```bash +curl -L -H'Content-Type: application/json' -XPOST --data-binary @quickstart/wikiticker-top-pages.json http://localhost:8082/druid/v2/?pretty +``` + +## Visualizing data + +Druid is ideal for power user-facing analytic applications. There are a number of different open source applications to +visualize and explore data in Druid. We recommend trying [Pivot](https://github.com/implydata/pivot), +[Panoramix](https://github.com/mistercrunch/panoramix), or [Metabase](https://github.com/metabase/metabase) to start +visualizing the data you just ingested. + +If you installed Pivot for example, you should be able to view your data in your browser at [localhost:9090](localhost:9090). + +### SQL and other query libraries + +There are many more query tools for Druid than we've included here, including SQL +engines, and libraries for various languages like Python and Ruby. Please see [the list of +libraries](../development/libraries.html) for more information. + +## Clustered setup + +This quickstart sets you up with all services running on a single machine. The next step is to [load +your own data](ingestion.html). Or, you can skip ahead to [running a distributed cluster](cluster.html). diff --git a/docs/content/tutorials/tutorial-a-first-look-at-druid.md b/docs/content/tutorials/tutorial-a-first-look-at-druid.md deleted file mode 100644 index 858858cba781..000000000000 --- a/docs/content/tutorials/tutorial-a-first-look-at-druid.md +++ /dev/null @@ -1,314 +0,0 @@ ---- -layout: doc_page ---- - -# Tutorial: A First Look at Druid -Greetings! This tutorial will help clarify some core Druid concepts. We will use a real-time dataset and issue some basic Druid queries. If you are ready to explore Druid, and learn a thing or two, read on! - -Note: If you are behind a corporate firewall, please see our additional [instructions](../tutorials/firewall.html) for running this tutorial. - -About the data --------------- - -The data source we'll be working with is Wikipedia edits. Each time an edit is made in Wikipedia, an event gets pushed to an IRC channel associated with the language of the Wikipedia page. We scrape IRC channels for several different languages and load this data into Druid. - -Each event has a timestamp indicating the time of the edit (in UTC time), a list of dimensions indicating various metadata about the event (such as information about the user editing the page and whether the user is a bot), and a list of metrics associated with the event (such as the number of characters added and deleted). - -Specifically. the data schema looks like this: - -Dimensions (things to filter on): - -```json -"page" -"language" -"user" -"unpatrolled" -"newPage" -"robot" -"anonymous" -"namespace" -"continent" -"country" -"region" -"city" -``` - -Metrics (things to aggregate over): - -```json -"count" -"added" -"delta" -"deleted" -``` - -Setting Up ----------- - -To start, we need to get our hands on a Druid build. There are two ways to get Druid: download a tarball, or [Build From Source](../development/build.html). You only need to do one of these. - -### Download a Tarball - -We've built a tarball that contains everything you'll need. You'll find it [here](http://druid.io/downloads.html). Download this file to a directory of your choosing. - -### Build From Source - -Follow the [Build From Source](../development/build.html) guide to build from source. Then grab the tarball from distribution/target/druid--bin.tar.gz. - -### Unpack the Tarball - -You can extract the content within by issuing: - -``` -tar -zxvf druid--bin.tar.gz -``` - -If you cd into the directory: - -``` -cd druid- -``` - -You should see a bunch of files: - -* run_example_server.sh -* run_example_client.sh -* LICENSE, config, examples, lib directories -* extensions (This folder contains all the extensions that could be loaded by Druid. Note that extension `mysql-metadata-storage` is packaged in a separate tarball that can be downloaded from [here](http://druid.io/downloads.html). See [Including Extensions](../operations/including-extensions.html) for more information about loading extensions. -* hadoop_dependencies (This folder contains hadoop-client:2.3.0, see [Different Hadoop Versions](../operations/other-hadoop.html) for more information about how Druid picks up Hadoop dependencies) - -## External Dependencies - -Druid requires 3 external dependencies. - -* A "deep storage" that acts as a data repository. This is generally distributed storage like HDFS or S3. For prototyping or experimentation on a single machine, Druid can use the local filesystem. - -* A "metadata storage" to hold configuration and metadata information. This is generally a small, shared database like MySQL or PostgreSQL. For prototyping or experimentation on a single machine, Druid can use a local instance of [Apache Derby](http://db.apache.org/derby/). - -* [Apache Zookeeper](http://zookeeper.apache.org/) for coordination among different pieces of the cluster. - -This tutorial only requires Zookeeper be set up. - -#### Set up Zookeeper - -* Download zookeeper from [http://www.apache.org/dyn/closer.cgi/zookeeper/](http://www.apache.org/dyn/closer.cgi/zookeeper/). -* Install zookeeper. - -```bash -ZOOKEPER_VERSION=zookeeper-3.4.6 -curl http://www.gtlib.gatech.edu/pub/apache/zookeeper/$ZOOKEPER_VERSION/$ZOOKEPER_VERSION.tar.gz -o $ZOOKEPER_VERSION.tar.gz -tar xzf $ZOOKEPER_VERSION.tar.gz -cd $ZOOKEPER_VERSION -cp conf/zoo_sample.cfg conf/zoo.cfg -./bin/zkServer.sh start -cd .. -``` - -Note you might update zookeper version. (see [here](http://www.gtlib.gatech.edu/pub/apache/zookeeper/)). - -Running Example Scripts ------------------------ - -Let's start by spinning up a server. You can start an example Druid [Realtime](../design/realtime.html) node by issuing: - -``` -./run_example_server.sh -``` - -Select the "wikipedia" example. - -Note that the first time you start the example, it may take some extra time due to its fetching various dependencies. Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. - -``` -2015-02-17T21:46:36,804 INFO [main] org.eclipse.jetty.server.ServerConnector - Started ServerConnector@79b6cf95{HTTP/1.1}{0.0.0.0:8084} -2015-02-17T21:46:36,804 INFO [main] org.eclipse.jetty.server.Server - Started @9580ms -2015-02-17T21:46:36,862 INFO [ApiDaemon] io.druid.segment.realtime.firehose.IrcFirehoseFactory - irc connection to server [irc.wikimedia.org] established -2015-02-17T21:46:36,862 INFO [ApiDaemon] io.druid.segment.realtime.firehose.IrcFirehoseFactory - Joining channel #en.wikipedia -2015-02-17T21:46:36,863 INFO [ApiDaemon] io.druid.segment.realtime.firehose.IrcFirehoseFactory - Joining channel #fr.wikipedia -2015-02-17T21:46:36,863 INFO [ApiDaemon] io.druid.segment.realtime.firehose.IrcFirehoseFactory - Joining channel #de.wikipedia -2015-02-17T21:46:36,863 INFO [ApiDaemon] io.druid.segment.realtime.firehose.IrcFirehoseFactory - Joining channel #ja.wikipedia -2015-02-17T21:46:37,009 INFO [ServerInventoryView-0] io.druid.client.BatchServerInventoryView - Inventory Initialized -``` - -The Druid real-time node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. If you are interested in the details of our real-time architecture and why we persist indexes to disk, we suggest you read our [White Paper](http://static.druid.io/docs/druid.pdf). - -To query the real-time node you've spun up, you can issue: - -``` -./run_example_client.sh -``` - -Select "wikipedia" once again. This script issues [TimeBoundary](../querying/timeboundaryquery.html) to the data we've been ingesting. The query looks like this: - -```json -{ - "queryType":"timeBoundary", - "dataSource":"wikipedia" -} -``` - -The **timeBoundary** query is one of the simplest queries you can make in Druid. It gives you the boundaries of the ingested data. - -The result looks something like this (when it's prettified): - -```json -[ { - "timestamp" : "2013-09-04T21:44:00.000Z", - "result" : { - "minTime" : "2013-09-04T21:44:00.000Z", - "maxTime" : "2013-09-04T21:47:00.000Z" - } -} ] -``` - -If you are having problems with getting results back, make sure you have [curl](http://curl.haxx.se/) installed. Control+C to break out of the client script. - -Querying Druid --------------- - -In your favorite editor, create the file: - -``` -timeseries.json -``` - -We are going to make a slightly more complicated query, the [TimeseriesQuery](../querying/timeseriesquery.html). Copy and paste the following into the file: - -```json -{ - "queryType": "timeseries", - "dataSource": "wikipedia", - "intervals": [ "2010-01-01/2020-01-01" ], - "granularity": "all", - "aggregations": [ - {"type": "longSum", "fieldName": "count", "name": "edit_count"}, - {"type": "doubleSum", "fieldName": "added", "name": "chars_added"} - ] -} -``` - -Our query has now expanded to include a time interval, [Granularities](../querying/granularities.html), and [Aggregations](../querying/aggregations.html). What the query is doing is aggregating a set of metrics over a span of time, and the results are grouped into a single time bucket. -To issue the query and get some results, run the following in your command line: - -``` -curl -X POST 'http://localhost:8084/druid/v2/?pretty' -H 'content-type: application/json' -d @timeseries.json -``` - -Once again, you should get a JSON blob of text back with your results, that looks something like this: - -```json -[ { - "timestamp" : "2013-09-04T21:44:00.000Z", - "result" : { "chars_added" : 312670.0, "edit_count" : 733 } -} ] -``` - -If you issue the query again, you should notice your results updating. - -Right now all the results you are getting back are being aggregated into a single timestamp bucket. What if we wanted to see our aggregations on a per minute basis? - -We can change granularity for the results to "minute". To specify different granularities to bucket our results, we change our query like so: - -```json -{ - "queryType": "timeseries", - "dataSource": "wikipedia", - "intervals": [ "2010-01-01/2020-01-01" ], - "granularity": "minute", - "aggregations": [ - {"type": "longSum", "fieldName": "count", "name": "edit_count"}, - {"type": "doubleSum", "fieldName": "added", "name": "chars_added"} - ] -} -``` - -This gives us results like the following: - -```json -[ - { - "timestamp" : "2013-09-04T21:44:00.000Z", - "result" : { "chars_added" : 30665.0, "edit_count" : 128 } - }, - { - "timestamp" : "2013-09-04T21:45:00.000Z", - "result" : { "chars_added" : 122637.0, "edit_count" : 167 } - }, - { - "timestamp" : "2013-09-04T21:46:00.000Z", - "result" : { "chars_added" : 78938.0, "edit_count" : 159 } - }, -... -] -``` - -Solving a Problem ------------------ - -One of Druid's main powers is to provide answers to problems, so let's pose a problem. What if we wanted to know what the top pages in the US are, ordered by the number of edits over the last few minutes you've been going through this tutorial? To solve this problem, we can use the [TopN](../querying/topnquery.html). - -Let's create the file: - -``` -topn.json -``` - -and put the following in there: - -```json -{ - "queryType": "topN", - "dataSource": "wikipedia", - "granularity": "all", - "dimension": "page", - "metric": "edit_count", - "threshold" : 10, - "aggregations": [ - {"type": "longSum", "fieldName": "count", "name": "edit_count"} - ], - "filter": { "type": "selector", "dimension": "country", "value": "United States" }, - "intervals": ["2012-10-01T00:00/2020-01-01T00"] -} -``` - -Note that our query now includes [Filters](../querying/filters.html). Filters are like `WHERE` clauses in SQL and help narrow down the data that needs to be scanned. - -If you issue the query: - -``` -curl -X POST 'http://localhost:8084/druid/v2/?pretty' -H 'content-type: application/json' -d @topn.json -``` - -You should see an answer to our question. As an example, some results are shown below: - -```json -[ - { - "timestamp" : "2013-09-04T21:00:00.000Z", - "result" : [ - { "page" : "RTC_Transit", "edit_count" : 6 }, - { "page" : "List_of_Deadly_Women_episodes", "edit_count" : 4 }, - { "page" : "User_talk:David_Biddulph", "edit_count" : 4 }, - ... - ] - } -] -``` - -Feel free to tweak other query parameters to answer other questions you may have about the data. Druid also includes more complex query types such as [groupBy queries](../querying/groupbyquery.html). For more information on querying, see this [link](../querying/querying.html). - -Next Steps ----------- - -This tutorial only covered the basic operations of a single Druid node. For production, you'll likely need a full Druid cluster. Check out our next tutorial [The Druid Cluster](../tutorials/tutorial-the-druid-cluster.html) to learn more. - -To learn more about loading streaming data, see [Loading Streaming Data](../tutorials/tutorial-loading-streaming-data.html). - -To learn more about loading batch data, see [Loading Batch Data](../tutorials/tutorial-loading-batch-data.html). - -Additional Information ----------------------- - -This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, read more of the Druid documentation and blogs found on druid.io. - -Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](https://groups.google.com/forum/#!forum/druid-user). diff --git a/docs/content/tutorials/tutorial-all-about-queries.md b/docs/content/tutorials/tutorial-all-about-queries.md deleted file mode 100644 index 81617cbcc870..000000000000 --- a/docs/content/tutorials/tutorial-all-about-queries.md +++ /dev/null @@ -1,207 +0,0 @@ ---- -layout: doc_page ---- - -# Tutorial: All About Queries -Hello! This tutorial is meant to provide a more in-depth look into Druid queries. The tutorial is somewhat incomplete right now but we hope to add more content to it in the near future. - -Setup ------ - -Before we start digging into how to query Druid, make sure you've gone through the other tutorials and are comfortable with spinning up a local cluster and loading data into Druid. - -#### Booting a Druid Cluster - -Let's start up a simple Druid cluster so we can query all the things. - -Note: If Zookeeper and metadata storage aren't running, you'll have to start them again as described in [The Druid Cluster](../tutorials/tutorial-the-druid-cluster.html). - -To start a Coordinator node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/coordinator:lib/* io.druid.cli.Main server coordinator -``` - -To start a Historical node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/historical:lib/* io.druid.cli.Main server historical -``` - -To start a Broker node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/broker:lib/* io.druid.cli.Main server broker -``` - -Querying Your Data ------------------- - -Make sure you've completed [Loading Your Data](../tutorials/tutorial-loading-streaming-data.html) so we have some data to query. Having done that, it's time to query our data! For a complete specification of queries, see [Querying](../querying/querying.html). - -#### Construct a Query -```json -{ - "queryType": "groupBy", - "dataSource": "wikipedia", - "granularity": "all", - "dimensions": [], - "aggregations": [ - {"type": "count", "name": "rows"}, - {"type": "longSum", "name": "edit_count", "fieldName": "count"}, - {"type": "doubleSum", "name": "chars_added", "fieldName": "added"} - ], - "intervals": ["2010-01-01T00:00/2020-01-01T00"] -} -``` - -#### Query That Data -Run the query against your broker: - -```bash -curl -X POST "http://localhost:8082/druid/v2/?pretty" -H 'Content-type: application/json' -d @query.body -``` - -And get: - -```json -[ { - "version" : "v1", - "timestamp" : "2010-01-01T00:00:00.000Z", - "event" : { - "chars_added" : 1545.0, - "edit_count" : 5, - "rows" : 5 - } -} ] -``` - -This result tells us that our query has 5 edits, and we have 5 rows of data as well. In those 5 edits, we have 1545 characters added. - -#### What can I query for? - -How are we to know what queries we can run? Although [Querying](../querying/querying.html) is a helpful index, to get a handle on querying our data we need to look at our ingestion schema. There are a few particular fields we care about in the ingestion schema. All of these fields should in present in the real-time ingestion schema and the batch ingestion schema. - -Datasource: - -```json -"dataSource":"wikipedia" -``` - -Our dataSource tells us the name of the relation/table, or 'source of data'. What we decide to name our data source must match the data source we are going to be querying. - -Granularity: - -```json -"indexGranularity": "none", -``` - -Druid will roll up data at ingestion time unless the index/rollup granularity is specified as "none". Your query granularity cannot be lower than your index granularity. - -Aggregators: - -```json -"aggregators" : [{ - "type" : "count", - "name" : "count" - }, { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" -}] -``` - -The [Aggregations](../querying/aggregations.html) specified at ingestion time correlated directly to the metrics that can be queried. - -Dimensions: - -```json -"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"] -``` - -These specify the dimensions that we can filter our data on. If we added a dimension to our groupBy query, we get: - -```json -{ - "queryType": "groupBy", - "dataSource": "wikipedia", - "granularity": "all", - "dimensions": ["namespace"], - "aggregations": [ - {"type": "longSum", "name": "edit_count", "fieldName": "count"}, - {"type": "doubleSum", "name": "chars_added", "fieldName": "added"} - ], - "intervals": ["2010-01-01T00:00/2020-01-01T00"] -} -``` - -Which gets us data grouped over the namespace dimension in return! - -```json -[ { - "version" : "v1", - "timestamp" : "2010-01-01T00:00:00.000Z", - "event" : { - "chars_added" : 180.0, - "edit_count" : 2, - "namespace" : "article" - } -}, { - "version" : "v1", - "timestamp" : "2010-01-01T00:00:00.000Z", - "event" : { - "chars_added" : 1365.0, - "edit_count" : 3, - "namespace" : "wikipedia" - } -} ] -``` - -Additionally,, we can also filter our query to narrow down our metric values: - -```json -{ - "queryType": "groupBy", - "dataSource": "wikipedia", - "granularity": "all", - "filter": { "type": "selector", "dimension": "namespace", "value": "article" }, - "aggregations": [ - {"type": "longSum", "name": "edit_count", "fieldName": "count"}, - {"type": "doubleSum", "name": "chars_added", "fieldName": "added"} - ], - "intervals": ["2010-01-01T00:00/2020-01-01T00"] -} -``` - -Which gets us metrics about only those edits where the namespace is 'article': - -```json -[ { - "version" : "v1", - "timestamp" : "2010-01-01T00:00:00.000Z", - "event" : { - "chars_added" : 180.0, - "edit_count" : 2 - } -} ] -``` - -Check out [Filters](../querying/filters.html) for more information. - -What Types of Queries to Use ----------------------------- - -The types of query you should use depends on your use case. [TimeBoundary queries](../querying/timeboundaryquery.html) are useful to understand the range of your data. [Timeseries queries](../querying/timeseriesquery.html) are useful for aggregates and filters over a time range, and offer significant speed improvements over [GroupBy queries](../querying/groupbyquery.html). To find the top values for a given dimension, [TopN queries](../querying/topnquery.html) should be used over group by queries as well. - - -## Learn More ## - -You can learn more about querying at [Querying](../querying/querying.html)! If you are ready to evaluate Druid more in depth, check out [Booting a production cluster](../tutorials/booting-a-production-cluster.html)! diff --git a/docs/content/tutorials/tutorial-batch.md b/docs/content/tutorials/tutorial-batch.md new file mode 100644 index 000000000000..2678a082183d --- /dev/null +++ b/docs/content/tutorials/tutorial-batch.md @@ -0,0 +1,108 @@ +--- +layout: doc_page +--- + +## Load your own batch data + +Before you get started with loading your own batch data, you should have first completed the [quickstart](quickstart.html). + +You can easily load any timestamped dataset into Druid. For Druid batch loads, the most important +questions are: + + * What should the dataset be called? This is the "dataSource" field of the "dataSchema". + * Where is the dataset located? The file paths belong in the "paths" of the "inputSpec". If you +want to load multiple files, you can provide them as a comma-separated string. + * Which field should be treated as a timestamp? This belongs in the "column" of the "timestampSpec". + * Which fields should be treated as dimensions? This belongs in the "dimensions" of the "dimensionsSpec". + * Which fields should be treated as metrics? This belongs in the "metricsSpec". + * What time ranges (intervals) are being loaded? This belongs in the "intervals" of the "granularitySpec". + +```note-info +If your data does not have a natural sense of time, you can tag each row with the current time. +You can also tag all rows with a fixed timestamp, like "2000-01-01T00:00:00.000Z". +``` + +Let's use this pageviews dataset as an example. Druid supports TSV, CSV, and JSON out of the box. +Note that nested JSON objects are not supported, so if you do use JSON, you should provide a file +containing flattened objects. + +```json +{"time": "2015-09-01T00:00:00Z", "url": "/foo/bar", "user": "alice", "latencyMs": 32} +{"time": "2015-09-01T01:00:00Z", "url": "/", "user": "bob", "latencyMs": 11} +{"time": "2015-09-01T01:30:00Z", "url": "/foo/bar", "user": "bob", "latencyMs": 45} +``` + +Make sure the file has no newline at the end. If you save this to a file called "pageviews.json", then for this dataset: + + * Let's call the dataset "pageviews". + * The data is located in "pageviews.json". + * The timestamp is the "time" field. + * Good choices for dimensions are the string fields "url" and "user". + * Good choices for metrics are a count of pageviews, and the sum of "latencyMs". Collecting that +sum when we load the data will allow us to compute an average at query time as well. + * The data covers the time range 2015-09-01 (inclusive) through 2015-09-02 (exclusive). + +You can copy the existing `quickstart/wikiticker-index.json` indexing task to a new file: + +```bash +cp quickstart/wikiticker-index.json quickstart/pageviews-index.json +``` + +And modify it by altering these sections: + +```json +"dataSource": "pageviews" +``` + +```json +"inputSpec": { + "type": "static", + "paths": "pageviews.json" +} +``` + +```json +"timestampSpec": { + "format": "auto", + "column": "time" +} +``` + +```json +"dimensionsSpec": { + "dimensions": ["url", "user"] +} +``` + +```json +"metricsSpec": [ + {"name": "views", "type": "count"}, + {"name": "latencyMs", "type": "doubleSum", "fieldName": "latencyMs"} +] +``` + +```json +"granularitySpec": { + "type": "uniform", + "segmentGranularity": "day", + "queryGranularity": "none", + "intervals": ["2015-09-01/2015-09-02"] +} +``` + +Finally, fire off the task and indexing will proceed! + +```bash +curl -X 'POST' -H 'Content-Type:application/json' -d @quickstart/pageviews-index.json localhost:8090/druid/indexer/v1/task +``` + +If anything goes wrong with this task (e.g. it finishes with status FAILED), you can troubleshoot +by visiting the "Task log" on the [overlord console](http://localhost:8090/console.html). + +```note-info +Druid supports a wide variety of data formats, ingestion options, and configurations not +discussed here. For a full explanation of all available features, see the ingestion sections of the Druid +documentation. +``` + +For more information on loading batch data, please see [the batch ingestion documentation](../ingestion/batch-ingestion.html). diff --git a/docs/content/tutorials/tutorial-kafka.md b/docs/content/tutorials/tutorial-kafka.md new file mode 100644 index 000000000000..741f38257b70 --- /dev/null +++ b/docs/content/tutorials/tutorial-kafka.md @@ -0,0 +1,192 @@ +--- +layout: doc_page +--- + +# Tutorial: Load from Kafka + +## Getting started + +This tutorial shows you how to load data from Kafka into Druid. + +For this tutorial, we'll assume you've already downloaded Druid and Tranquility as described in +the [single-machine quickstart](quickstart.html) and have it running on your local machine. You +don't need to have loaded any data yet. + +```note-info +This tutorial will show you how to load data from Kafka into Druid, but Druid additionally supports +a wide variety of batch and streaming loading methods. See the *[Loading files](../ingestion/batch-ingestion.html)* +and *[Loading streams](../ingestion/stream-ingestion.html)* pages for more information about other options, +including from Hadoop, HTTP, Storm, Samza, Spark Streaming, and your own JVM apps. +``` + +## Start Kafka + +[Apache Kafka](http://kafka.apache.org/) is a high throughput message bus that works well with +Druid. For this tutorial, we will use Kafka 0.9.0.0. To download Kafka, issue the following +commands in your terminal: + +```bash +curl -O http://www.us.apache.org/dist/kafka/0.9.0.0/kafka_2.11-0.9.0.0.tgz +tar -xzf kafka_2.11-0.9.0.0.tgz +cd kafka_2.11-0.9.0.0 +``` + +Start a Kafka broker by running the following command in a new terminal: + +```bash +./bin/kafka-server-start.sh config/server.properties +``` + +Run this command to create a Kafka topic called *metrics*, to which we'll send data: + +```bash +./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic metrics +``` + +## Enable Druid Kafka ingestion + +Druid includes configs for [Tranquility Kafka](ingestion-streams.md#kafka) to support loading data from Kafka. +To enable this in the quickstart-based configuration: + +- Stop your Tranquility command (CTRL-C) and then start it up again. + +## Send example data + +Let's launch a console producer for our topic and send some data! + +In your Druid directory, generate some metrics by running: + +```bash +bin/generate-example-metrics +``` + +In your Kafka directory, run: + +```bash +./bin/kafka-console-producer.sh --broker-list localhost:9092 --topic metrics +``` + +The *kafka-console-producer* command is now awaiting input. Copy the generated example metrics, +paste them into the *kafka-console-producer* terminal, and press enter. If you like, you can also +paste more messages into the producer, or you can press CTRL-D to exit the console producer. + +You can immediately query this data, or you can skip ahead to the +[Loading your own data](#loading-your-own-data) section if you'd like to load your own dataset. + +## Querying your data + +After sending data, you can immediately query it using any of the +[supported query methods](../querying/querying.html). + +## Loading your own data + +So far, you've loaded data into Druid from Kafka using an ingestion spec that we've included in the +distribution. Each ingestion spec is designed to work with a particular dataset. You load your own +data types into Imply by writing a custom ingestion spec. + +You can write a custom ingestion spec by starting from the bundled configuration in +`conf-quickstart/tranquility/kafka.json` and modifying it for your own needs. + +The most important questions are: + + * What should the dataset be called? This is the "dataSource" field of the "dataSchema". + * Which field should be treated as a timestamp? This belongs in the "column" of the "timestampSpec". + * Which fields should be treated as dimensions? This belongs in the "dimensions" of the "dimensionsSpec". + * Which fields should be treated as measures? This belongs in the "metricsSpec". + +Let's use a small JSON pageviews dataset in the topic *pageviews* as an example, with records like: + +```json +{"time": "2000-01-01T00:00:00Z", "url": "/foo/bar", "user": "alice", "latencyMs": 32} +``` + +First, create the topic: + +```bash +./bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic pageviews +``` + +Next, edit `conf-quickstart/tranquility/kafka.json`: + + * Let's call the dataset "pageviews-kafka". + * The timestamp is the "time" field. + * Good choices for dimensions are the string fields "url" and "user". + * Good choices for measures are a count of pageviews, and the sum of "latencyMs". Collecting that +sum when we load the data will allow us to compute an average at query time as well. + +You can edit the existing `conf-quickstart/tranquility/kafka.json` file by altering these +sections: + + 1. Change the key `"metrics-kafka"` under `"dataSources"` to `"pageviews-kafka"` + 2. Alter these sections under the new `"pageviews-kafka"` key: + ```json + "dataSource": "pageviews-kafka" + ``` + + ```json + "timestampSpec": { + "format": "auto", + "column": "time" + } + ``` + + ```json + "dimensionsSpec": { + "dimensions": ["url", "user"] + } + ``` + + ```json + "metricsSpec": [ + {"name": "views", "type": "count"}, + {"name": "latencyMs", "type": "doubleSum", "fieldName": "latencyMs"} + ] + ``` + + ```json + "properties" : { + "task.partitions" : "1", + "task.replicants" : "1", + "topicPattern" : "pageviews" + } + ``` + +Next, start Druid Kafka ingestion: + +```bash +bin/tranquility kafka -configFile ../druid-0.9.0-SNAPSHOT/conf-quickstart/tranquility/kafka.json +``` + +- If your Tranquility server or Kafka is already running, stop it (CTRL-C) and +start it up again. + +Finally, send some data to the Kafka topic. Let's start with these messages: + +```json +{"time": "2000-01-01T00:00:00Z", "url": "/foo/bar", "user": "alice", "latencyMs": 32} +{"time": "2000-01-01T00:00:00Z", "url": "/", "user": "bob", "latencyMs": 11} +{"time": "2000-01-01T00:00:00Z", "url": "/foo/bar", "user": "bob", "latencyMs": 45} +``` + +Druid streaming ingestion requires relatively current messages (relative to a slack time controlled by the +[windowPeriod](../ingestion/stream-ingestion.html#segmentgranularity-and-windowperiod) value), so you should +replace `2000-01-01T00:00:00Z` in these messages with the current time in ISO8601 format. You can +get this by running: + +```bash +python -c 'import datetime; print(datetime.datetime.utcnow().strftime("%Y-%m-%dT%H:%M:%SZ"))' +``` + +Update the timestamps in the JSON above, then copy and paste these messages into this console +producer and press enter: + +```bash +./bin/kafka-console-producer.sh --broker-list localhost:9092 --topic pageviews +``` + +That's it, your data should now be in Druid. You can immediately query it using any of the +[supported query methods](../querying/querying.html). + +## Further reading + +To read more about loading streams, see our [streaming ingestion documentation](../ingestion/stream-ingestion.html). diff --git a/docs/content/tutorials/tutorial-loading-batch-data.md b/docs/content/tutorials/tutorial-loading-batch-data.md deleted file mode 100644 index eeee9745b3a9..000000000000 --- a/docs/content/tutorials/tutorial-loading-batch-data.md +++ /dev/null @@ -1,337 +0,0 @@ ---- -layout: doc_page ---- - -# Tutorial: Loading Batch Data - -In this tutorial, we will learn about batch ingestion (as opposed to real-time ingestion) and how to create segments using the final piece of the Druid Cluster, the [indexing service](../design/indexing-service.html). The indexing service is a standalone service that accepts [tasks](../misc/tasks.html) in the form of POST requests. The output of most tasks are segments. The indexing service can be used as a single service for both real-time/streaming and batch ingestion. - -The Data --------- -The data source we'll be using is Wikipedia edits. The data schema is: - -Dimensions (things to filter on): - -```json -"page" -"language" -"user" -"unpatrolled" -"newPage" -"robot" -"anonymous" -"namespace" -"continent" -"country" -"region" -"city" -``` - -Metrics (things to aggregate over): - -```json -"count" -"added" -"delta" -"deleted" -``` - -Batch Ingestion ---------------- - -For the purposes of this tutorial, we are going to use our very small and simple Wikipedia data set. This data can directly be ingested via other means as shown in the previous [tutorial](../tutorials/tutorial-loading-streaming-data.html). - -Our data is located at: - -``` -examples/indexing/wikipedia_data.json -``` - -The following events should exist in the file: - -```json -{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} -{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} -{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} -{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} -{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} -``` - -#### Set Up a Druid Cluster - -To index the data, we are going to need the overlord, a historical node, and a coordinator node. - -Note: If Zookeeper isn't running, you'll have to start it again as described in [The Druid Cluster](../tutorials/tutorial-the-druid-cluster.html). - -To start the Indexing Service: - -```bash -java -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/overlord:lib/* io.druid.cli.Main server overlord -``` - -To start the Coordinator Node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/coordinator:lib/* io.druid.cli.Main server coordinator -``` - -To start the Historical Node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/historical:lib/* io.druid.cli.Main server historical -``` - -#### Index the Data - -There are two ways we can load the data, depending on the data volume. The simplest method of loading data is to use the [Index Task](../misc/tasks.html). Index tasks can load batch data without any external dependencies. They are however, slow when the data volume exceeds 1G. - -#### Index Task - -To index the data and build a Druid segment, we are going to need to submit a task to the indexing service. This task should already exist: - -``` -examples/indexing/wikipedia_index_task.json -``` - -Open up the file to see the following: - -```json -{ - "type" : "index", - "spec" : { - "dataSchema" : { - "dataSource" : "wikipedia", - "parser" : { - "type" : "string", - "parseSpec" : { - "format" : "json", - "timestampSpec" : { - "column" : "timestamp", - "format" : "auto" - }, - "dimensionsSpec" : { - "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], - "dimensionExclusions" : [], - "spatialDimensions" : [] - } - } - }, - "metricsSpec" : [ - { - "type" : "count", - "name" : "count" - }, - { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, - { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, - { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" - } - ], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "DAY", - "queryGranularity" : "NONE", - "intervals" : [ "2013-08-31/2013-09-01" ] - } - }, - "ioConfig" : { - "type" : "index", - "firehose" : { - "type" : "local", - "baseDir" : "examples/indexing/", - "filter" : "wikipedia_data.json" - } - }, - "tuningConfig" : { - "type" : "index", - "targetPartitionSize" : 0, - "rowFlushBoundary" : 0 - } - } -} -``` - -Okay, so what is happening here? The "type" field indicates the type of task we plan to run. In this case, it is a simple "index" task. The "parseSpec" indicates how we plan to figure out what the timestamp and dimension columns are. The "granularitySpec" indicates that we are building a daily segment for 2013-08-31 to 2013-09-01 and the minimum queryGranularity will be millisecond (NONE). Next, the "metricsSpec" indicate which fields in our data set we plan to build metric columns for. The "fieldName" corresponds to the metric name in the raw data. The "name" corresponds to what our metric column is actually going to be called in the segment. Finally, we have a local "firehose" that is going to read data from disk. We tell the firehose where our data is located and the types of files we are looking to ingest. In our case, we only have a single data file. - -Let's send our task to the indexing service now: - -```bash -curl -X 'POST' -H 'Content-Type:application/json' -d @examples/indexing/wikipedia_index_task.json localhost:8090/druid/indexer/v1/task -``` - -Issuing the request should return a task ID like so: - -```bash -curl -X 'POST' -H 'Content-Type:application/json' -d @examples/indexing/wikipedia_index_task.json localhost:8090/druid/indexer/v1/task -{"task":"index_wikipedia_2013-10-09T21:30:32.802Z"} -``` - -In your indexing service logs, you should see the following: - -```bash -2013-10-09 21:41:41,150 INFO [qtp300448720-21] io.druid.indexing.overlord.HeapMemoryTaskStorage - Inserting task index_wikipedia_2013-10-09T21:41:41.147Z with status: TaskStatus{id=index_wikipedia_2013-10-09T21:41:41.147Z, status=RUNNING, duration=-1} -2013-10-09 21:41:41,151 INFO [qtp300448720-21] io.druid.indexing.overlord.TaskLockbox - Created new TaskLockPosse: TaskLockPosse{taskLock=TaskLock{groupId=index_wikipedia_2013-10-09T21:41:41.147Z, dataSource=wikipedia, interval=2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z, version=2013-10-09T21:41:41.151Z}, taskIds=[]} -... -013-10-09 21:41:41,215 INFO [pool-6-thread-1] io.druid.indexing.overlord.ForkingTaskRunner - Logging task index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0 output to: /tmp/persistent/index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0/b5099fdb-d6b0-4b81-9053-b2af70336a7e/log -2013-10-09 21:41:45,017 INFO [qtp300448720-22] io.druid.indexing.common.actions.LocalTaskActionClient - Performing action for task[index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0]: LockListAction{} - -```` - -After a few seconds, the task should complete and you should see in the indexing service logs: - -```bash -2013-10-09 21:41:45,765 INFO [pool-6-thread-1] io.druid.indexing.overlord.exec.TaskConsumer - Received SUCCESS status for task: IndexGeneratorTask{id=index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0, type=index_generator, dataSource=wikipedia, interval=Optional.of(2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z)} -``` - -Congratulations! The segment has completed building. Once a segment is built, a segment metadata entry is created in your metadata storage table. The coordinator compares what is in the segment metadata table with what is in the cluster. A new entry in the metadata table will cause the coordinator to load the new segment in a minute or so. - -You should see the following logs on the coordinator: - -```bash -2013-10-09 21:41:54,368 INFO [Coordinator-Exec--0] io.druid.server.coordinator.helper.DruidCoordinatorLogger - [_default_tier] : Assigned 1 segments among 1 servers -2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.helper.DruidCoordinatorLogger - Load Queues: -2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.helper.DruidCoordinatorLogger - Server[localhost:8083, historical, _default_tier] has 1 left to load, 0 left to drop, 4,477 bytes queued, 4,477 bytes served. -``` - -These logs indicate that the coordinator has assigned our new segment to the historical node to download and serve. If you look at the historical node logs, you should see: - -```bash -2013-10-09 21:41:54,369 INFO [ZkCoordinator-0] io.druid.server.coordination.ZkCoordinator - Loading segment wikipedia_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2013-10-09T21:41:41.151Z -2013-10-09 21:41:54,369 INFO [ZkCoordinator-0] io.druid.segment.loading.LocalDataSegmentPuller - Unzipping local file[/tmp/druid/localStorage/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0/index.zip] to [/tmp/druid/indexCache/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0] -2013-10-09 21:41:54,370 INFO [ZkCoordinator-0] io.druid.utils.CompressionUtils - Unzipping file[/tmp/druid/localStorage/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0/index.zip] to [/tmp/druid/indexCache/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0] -2013-10-09 21:41:54,380 INFO [ZkCoordinator-0] io.druid.server.coordination.SingleDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2013-10-09T21:41:41.151Z] to path[/druid/servedSegments/localhost:8083/wikipedia_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2013-10-09T21:41:41.151Z] -``` - -Once the segment is announced the segment is queryable. Now you should be able to query the data. - -Issuing a [TimeBoundaryQuery](../querying/timeboundaryquery.html) should yield: - -```json -[ { - "timestamp" : "2013-08-31T01:02:33.000Z", - "result" : { - "minTime" : "2013-08-31T01:02:33.000Z", - "maxTime" : "2013-08-31T12:41:27.000Z" - } -} ] -``` - -Console --------- - -The indexing service overlord has a console located at: - -```bash -localhost:8090/console.html -``` - -On this console, you can look at statuses and logs of recently submitted and completed tasks. - -If you decide to reuse the local firehose to ingest your own data and if you run into problems, you can use the console to read the individual task logs. - -Task logs can be stored locally or uploaded to [Deep Storage](../dependencies/deep-storage.html). More information about how to configure this is [here](../configuration/index.html). - -Most common data ingestion problems are around timestamp formats and other malformed data issues. - -#### Hadoop Index Task - -Druid is designed for large data volumes, and most real-world data sets require batch indexing be done through a Hadoop job. - -For this tutorial, we used [Hadoop 2.3.0](https://archive.apache.org/dist/hadoop/core/hadoop-2.3.0/), which is included under ```hadoop_dependencies```. There are many pages on the Internet showing how to set up a single-node (standalone) Hadoop cluster, which is all that's needed for this example. For more information about how Druid picks up your Hadoop version, see [here](../operations/other-hadoop.html). - -Before indexing the data, make sure you have a valid Hadoop cluster running. To build our Druid segment, we are going to submit a [Hadoop index task](../misc/tasks.html) to the indexing service. The grammar for the Hadoop index task is very similar to the index task of the last tutorial. The tutorial Hadoop index task should be located at: - -``` -examples/indexing/wikipedia_index_hadoop_task.json -``` - -Examining the contents of the file, you should find: - - ```json - { - "type" : "index_hadoop", - "spec" : { - "dataSchema" : { - "dataSource" : "wikipedia", - "parser" : { - "type" : "string", - "parseSpec" : { - "format" : "json", - "timestampSpec" : { - "column" : "timestamp", - "format" : "auto" - }, - "dimensionsSpec" : { - "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], - "dimensionExclusions" : [], - "spatialDimensions" : [] - } - } - }, - "metricsSpec" : [ - { - "type" : "count", - "name" : "count" - }, - { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, - { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, - { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" - } - ], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "DAY", - "queryGranularity" : "NONE", - "intervals" : [ "2013-08-31/2013-09-01" ] - } - }, - "ioConfig" : { - "type" : "hadoop", - "inputSpec" : { - "type" : "static", - "paths" : "/MyDirectory/examples/indexing/wikipedia_data.json" - } - } - } - } - ``` - -If you are curious about what all this configuration means, see [here](../misc/tasks.html). - -To submit the task: - -```bash -curl -X 'POST' -H 'Content-Type:application/json' -d @examples/indexing/wikipedia_index_hadoop_task.json localhost:8090/druid/indexer/v1/task -``` - -After the task is completed, the segment should be assigned to your historical node. You should be able to query the segment. - -Next Steps ----------- -We demonstrated using the indexing service as a way to ingest data into Druid. Previous versions of Druid used the [HadoopDruidIndexer](../ingestion/batch-ingestion.html) to ingest batch data. The `HadoopDruidIndexer` still remains a valid option for batch ingestion, however, we recommend using the indexing service as the preferred method of getting batch data into Druid. - -Additional Information ----------------------- - -Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-user). diff --git a/docs/content/tutorials/tutorial-loading-streaming-data.md b/docs/content/tutorials/tutorial-loading-streaming-data.md deleted file mode 100644 index 18139bc47e86..000000000000 --- a/docs/content/tutorials/tutorial-loading-streaming-data.md +++ /dev/null @@ -1,153 +0,0 @@ ---- -layout: doc_page ---- - -# Loading Streaming Data - -In our [last tutorial](../tutorials/tutorial-the-druid-cluster.html), we set up a -complete Druid cluster. We created all the Druid dependencies and ingested -streaming data. In this tutorial, we will expand upon what we've done in the -first two tutorials. - -## About the Data - -We will be working with the same Wikipedia edits data schema [from our previous -tutorials](tutorial-a-first-look-at-druid.html#about-the-data). - -## Set Up - -At this point, you should already have Druid downloaded and be comfortable -running a Druid cluster locally. If not, [have a look at our second -tutorial](../tutorials/tutorial-the-druid-cluster.html). If Zookeeper is not -running, you will have to start it as described in [The Druid -Cluster](../tutorials/tutorial-the-druid-cluster.html). - -With real-world data, we recommend having a message bus such as [Apache -Kafka](http://kafka.apache.org/) sit between the data stream and the real-time -node. The message bus provides higher availability for production environments. -[Firehoses](../ingestion/firehose.html) are the key abstraction for real-time ingestion. - -### Kafka - -Druid communicates with Kafka using the -[KafkaFirehoseFactory](../ingestion/firehose.html). Using this [Firehose](../ingestion/firehose.html) -with the right configuration, we can import data into Druid in real-time -without writing any code. To load data to a real-time node via Kafka, we'll -first need to initialize Zookeeper and Kafka, and then configure and initialize -a [Realtime](../design/realtime.html) node. - -The following quick-start instructions for booting a Zookeeper and then Kafka -cluster were adapted from the [Apache Kafka quickstart guide](http://kafka.apache.org/documentation.html#quickstart). - -1. Download Kafka - - For this tutorial we will [download Kafka 0.8.2.1] - (https://www.apache.org/dyn/closer.cgi?path=/kafka/0.8.2.1/kafka_2.10-0.8.2.1.tgz) - - ```bash - tar -xzf kafka_2.10-0.8.2.1.tgz - cd kafka_2.10-0.8.2.1 - ``` - -1. Start Kafka - - First launch ZooKeeper: - - ```bash - ./bin/zookeeper-server-start.sh config/zookeeper.properties - ``` - - Then start the Kafka server (in a separate console): - - ```bash - ./bin/kafka-server-start.sh config/server.properties - ``` - -1. Create a topic named `wikipedia` - - ```bash - ./bin/kafka-topics.sh --create --zookeeper localhost:2181 \ - --replication-factor 1 --partitions 1 --topic wikipedia - ``` - -1. Launch a console producer for that topic (so we can paste in kafka - messages in a bit) - - ```bash - ./bin/kafka-console-producer.sh --broker-list localhost:9092 --topic wikipedia - ``` - -### Druid Realtime Node - -The realtime spec for the data source in this tutorial is available under -`examples/indexing/wikipedia.spec` from the [Druid -download](http://druid.io/downloads.html) - -1. Launch the realtime node - - ```bash - java -Xmx512m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ - -Ddruid.realtime.specFile=examples/indexing/wikipedia.spec \ - -classpath "config/_common:config/realtime:lib/*" \ - io.druid.cli.Main server realtime - ``` - -1. Copy and paste the following data into the terminal where we launched - the Kafka console producer above. - - ```json - {"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} - {"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} - {"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} - {"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} - {"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} - ``` - - **Note:** This config uses a [`messageTime` rejection policy](../design/plumber.html) - which will accept all events and hand off as long as there is a continuous - stream of events. In this particular example, hand-off will not actually - occur because we only have a handful of events. - - Disclaimer: We recognize the timestamps of these events aren't actually recent. - -1. Watch the events getting ingested and the real-time node announcing a data - segment - - ``` - ... - 2015-02-17T23:01:50,220 INFO [chief-wikipedia] io.druid.server.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2013-08-31T00:00:00.000Z] at path[/druid/segments/localhost:8084/2015-02-17T23:01:50.219Z0] - ... - ``` - -1. Issue a query - - Issuing a [TimeBoundaryQuery](../querying/timeboundaryquery.html) to the real-time node - should return some results: - - ```bash - curl -XPOST -H'Content-type: application/json' \ - "http://localhost:8084/druid/v2/?pretty" \ - -d'{"queryType":"timeBoundary","dataSource":"wikipedia"}' - ``` - - ```json - [ { - "timestamp" : "2013-08-31T01:02:33.000Z", - "result" : { - "minTime" : "2013-08-31T01:02:33.000Z", - "maxTime" : "2013-08-31T12:41:27.000Z" - } - } ] - ``` - -## Advanced Streaming Ingestion - -Druid offers an additional method of ingesting streaming data via the indexing service. You may be wondering why a second method is needed. Standalone real-time nodes are sufficient for certain volumes of data and availability tolerances. They pull data from a message queue like Kafka or Rabbit, index data locally, and periodically finalize segments for handoff to historical nodes. They are fairly straightforward to scale, simply taking advantage of the innate scalability of the backing message queue. But they are difficult to make highly available with Kafka, the most popular supported message queue, because its high-level consumer doesn’t provide a way to scale out two replicated consumer groups such that each one gets the same data in the same shard. They also become difficult to manage once you have a lot of them, since every machine needs a unique configuration. - -Druid solved the availability problem by switching from a pull-based model to a push-based model; rather than Druid indexers pulling data from Kafka, another process pulls data and pushes the data into Druid. Since with the push based model, we can ensure that the same data makes it into the same shard, we can replicate data. The [indexing service](../design/indexing-service.html) encapsulates this functionality, where a task-and-resources model replaces a standalone machine model. In addition to simplifying machine configuration, the model also allows nodes to run in the cloud with an elastic number of machines. If you are interested in this form of real-time ingestion, please check out the client library [Tranquility](https://github.com/druid-io/tranquility). - -Additional Information ----------------------- - -Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-user). - diff --git a/docs/content/tutorials/tutorial-streams.md b/docs/content/tutorials/tutorial-streams.md new file mode 100644 index 000000000000..f935c7a82c1a --- /dev/null +++ b/docs/content/tutorials/tutorial-streams.md @@ -0,0 +1,134 @@ +--- +layout: doc_page +--- + +## Load your own streaming data + +## Getting started + +This tutorial shows you how to load your own streams into Druid. + +For this tutorial, we'll assume you've already downloaded Druid and Tranquility as described in +the [single-machine quickstart](quickstart.html) and have it running on your local machine. You +don't need to have loaded any data yet. + +Once that's complete, you can load your own dataset by writing a custom ingestion spec. + +## Writing an ingestion spec + +When loading streams into Druid, we recommend using the [stream push](../ingestion/stream-push.html) +process. In this tutorial we'll be using [Tranquility Server](../ingestion/stream-ingestion.html#server) to push +data into Druid over HTTP. + +```note-info +This tutorial will show you how to push streams to Druid using HTTP, but Druid additionally supports +a wide variety of batch and streaming loading methods. See the *[Loading files](batch-ingestion.html)* +and *[Loading streams](stream-ingestion.html)* pages for more information about other options, +including from Hadoop, Kafka, Storm, Samza, Spark Streaming, and your own JVM apps. +``` + +You can prepare for loading a new dataset over HTTP by writing a custom Tranquility Server +configuration. The bundled configuration is in `conf-quickstart/tranquility/server.json`, which +you can modify for your own needs. + +The most important questions are: + + * What should the dataset be called? This is the "dataSource" field of the "dataSchema". + * Which field should be treated as a timestamp? This belongs in the "column" field of the "timestampSpec". + * Which fields should be treated as dimensions? This belongs in the "dimensions" field of the "dimensionsSpec". + * Which fields should be treated as measures? This belongs in the "metricsSpec" field. + +Let's use a small JSON pageviews dataset as an example, with records like: + +```json +{"time": "2000-01-01T00:00:00Z", "url": "/foo/bar", "user": "alice", "latencyMs": 32} +``` + +So the answers to the questions above are: + + * Let's call the dataset "pageviews". + * The timestamp is the "time" field. + * Good choices for dimensions are the string fields "url" and "user". + * Good choices for measures are a count of pageviews, and the sum of "latencyMs". Collecting that +sum when we load the data will allow us to compute an average at query time as well. + +Now, edit the existing `conf-quickstart/tranquility/server.json` file by altering these +sections: + + 1. Change the key `"metrics"` under `"dataSources"` to `"pageviews"` + 2. Alter these sections under the new `"pageviews"` key: + ```json + "dataSource": "pageviews" + ``` + + ```json + "timestampSpec": { + "format": "auto", + "column": "time" + } + ``` + + ```json + "dimensionsSpec": { + "dimensions": ["url", "user"] + } + ``` + + ```json + "metricsSpec": [ + {"name": "views", "type": "count"}, + {"name": "latencyMs", "type": "doubleSum", "fieldName": "latencyMs"} + ] + ``` + +## Restarting the server + +Restart the server to pick up the new configuration file by stopping Tranquility (CTRL-C) and starting it up again. + +## Sending data + +Let's send some data! We'll start with these three records: + +```json +{"time": "2000-01-01T00:00:00Z", "url": "/foo/bar", "user": "alice", "latencyMs": 32} +{"time": "2000-01-01T00:00:00Z", "url": "/", "user": "bob", "latencyMs": 11} +{"time": "2000-01-01T00:00:00Z", "url": "/foo/bar", "user": "bob", "latencyMs": 45} +``` + +Druid streaming ingestion requires relatively current messages (relative to a slack time controlled by the +[windowPeriod](ingestion-streams.html#segmentgranularity-and-windowperiod) value), so you should +replace `2000-01-01T00:00:00Z` in these messages with the current time in ISO8601 format. You can +get this by running: + +```bash +python -c 'import datetime; print(datetime.datetime.utcnow().strftime("%Y-%m-%dT%H:%M:%SZ"))' +``` + +Update the timestamps in the JSON above, and save it to a file named `pageviews.json`. Then send +it to Druid by running: + +```bash +curl -XPOST -H'Content-Type: application/json' --data-binary @pageviews.json http://localhost:8200/v1/post/pageviews +``` + +This will print something like: + +``` +{"result":{"received":3,"sent":3}} +``` + +This indicates that the HTTP server received 3 events from you, and sent 3 to Druid. Note that +this may take a few seconds to finish the first time you run it, as Druid resources must be +allocated to the ingestion task. Subsequent POSTs should complete quickly. + +If you see `"sent":0` this likely means that your timestamps are not recent enough. Try adjusting +your timestamps and re-sending your data. + +## Querying your data + +After sending data, you can immediately query it using any of the +[supported query methods](../querying/querying.html). + +## Further reading + +To read more about loading streams, see our [streaming ingestion documentation](../ingestion/stream-ingestion.html). diff --git a/docs/content/tutorials/tutorial-the-druid-cluster.md b/docs/content/tutorials/tutorial-the-druid-cluster.md deleted file mode 100644 index 69f40dd6ef07..000000000000 --- a/docs/content/tutorials/tutorial-the-druid-cluster.md +++ /dev/null @@ -1,313 +0,0 @@ ---- -layout: doc_page ---- - -# Tutorial: The Druid Cluster -Welcome back! In our first [tutorial](../tutorials/tutorial-a-first-look-at-druid.html), we introduced you to the most basic Druid setup: a single realtime node. We streamed in some data and queried it. Realtime nodes collect very recent data and periodically hand that data off to the rest of the Druid cluster. Some questions about the architecture must naturally come to mind. What does the rest of Druid cluster look like? - -This tutorial will hopefully answer these questions! - -In this tutorial, we will set up other types of Druid nodes and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system. - -## Downloading Druid -If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. - -You can download the latest version of druid [here](http://druid.io/downloads.html). You can also [Build From Source](../development/build.html) and grab the tarball from distribution/target/druid--bin.tar.gz. - -Either way, once you have the tarball, untar the contents within by issuing: - -```bash -tar -zxvf druid--bin.tar.gz -cd druid- -``` - -## External Dependencies -Druid requires 3 external dependencies. - -* A "deep storage" that acts as a data repository. This is generally distributed storage like HDFS or S3. For prototyping or experimentation on a single machine, Druid can use the local filesystem. -* A "metadata storage" to hold configuration and metadata information. This is generally a small, shared database like MySQL or PostgreSQL. For prototyping or experimentation on a single machine, Druid can use a local instance of [Apache Derby](http://db.apache.org/derby/). -* [Apache Zookeeper](http://zookeeper.apache.org/) for coordination among different pieces of the cluster. - -For deep storage, we will use the local disk in this tutorial, but for production, HDFS and S3 are popular options. For the metadata storage, Derby is used in this tutorial, but for production MySQL or PostgreSQL etc should be used. - -#### Set up Zookeeper - -* Download zookeeper from [http://www.apache.org/dyn/closer.cgi/zookeeper/](http://www.apache.org/dyn/closer.cgi/zookeeper/) -* Install zookeeper. - -```bash -curl http://www.gtlib.gatech.edu/pub/apache/zookeeper/zookeeper-3.4.6/zookeeper-3.4.6.tar.gz -o zookeeper-3.4.6.tar.gz -tar xzf zookeeper-3.4.6.tar.gz -cd zookeeper-3.4.6 -cp conf/zoo_sample.cfg conf/zoo.cfg -./bin/zkServer.sh start -cd .. -``` - -## The Data - -Similar to the first tutorial, the data we will be loading is based on edits that have occurred on Wikipedia. Every time someone edits a page in Wikipedia, metadata is generated about the editor and edited page. Druid collects each individual event and packages them together in a container known as a [segment](../design/segments.html). Segments contain data over some span of time. We've prebuilt a segment for this tutorial and will cover making your own segments in other [pages](../tutorials/tutorial-loading-streaming-data.html).The segment we are going to work with has the following format: - -Dimensions (things to filter on): - -```json -"page" -"language" -"user" -"unpatrolled" -"newPage" -"robot" -"anonymous" -"namespace" -"continent" -"country" -"region" -"city" -``` - -Metrics (things to aggregate over): - -```json -"count" -"added" -"delta" -"deleted" -``` - -## The Cluster - -Before we get started, let's make sure we have configs in the config directory for our various nodes. Issue the following from the Druid home directory: - -``` -ls config -``` - -If you are interested in learning more about Druid configuration files, check out this [link](../configuration/index.html). Many aspects of Druid are customizable. For the purposes of this tutorial, we are going to use default values for most things. - -#### Common Configuration - -There are a couple of cluster wide configuration options we have to define. The common/cluster configuration files should exist under: - -``` -config/_common -``` - -In the directory, there should be a `common.runtime.properties` file with the following contents: - -``` -# Extensions -druid.extensions.loadList=["druid-examples","druid-kafka-eight"] - -# Zookeeper -druid.zk.service.host=localhost - -# Deep storage (local filesystem for examples - don't use this in production) -druid.storage.type=local -druid.storage.storage.storageDirectory=/tmp/druid/localStorage - -# Query Cache (we use a simple 10mb heap-based local cache on the broker) -druid.cache.type=local -druid.cache.sizeInBytes=10000000 - -# Indexing service discovery -druid.selectors.indexing.serviceName=overlord - -# Monitoring (disabled for examples) -# druid.monitoring.monitors=["com.metamx.metrics.JvmMonitor"] - -# Metrics logging (disabled for examples) -druid.emitter=noop -``` - -In this file we define our external dependencies and cluster wide configs. - -#### Start a Coordinator Node - -Coordinator nodes are in charge of load assignment and distribution. Coordinator nodes monitor the status of the cluster and command historical nodes to assign and drop segments. -For more information about coordinator nodes, see [here](../design/coordinator.html). - -The coordinator config file should already exist at: - -``` -config/coordinator -``` - -In the directory, there should be a `runtime.properties` file with the following contents: - -``` -druid.host=localhost -druid.port=8081 -druid.service=druid/coordinator - -# The coordinator begins assignment operations after the start delay. -# We override the default here to start things up faster for examples. -druid.coordinator.startDelay=PT70s -``` - -To start the coordinator node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/coordinator:lib/* io.druid.cli.Main server coordinator -``` - -Note: we will be running a single historical node in these examples, so you may see some warnings about not being able to replicate segments. These can be safely ignored, but in production, you should always replicate segments across multiple historical nodes. - -#### Start a Historical Node - -Historical nodes are the workhorses of a cluster and are in charge of loading historical segments and making them available for queries. Realtime nodes hand off segments to historical nodes. -For more information about Historical nodes, see [here](../design/historical.html). - -The historical config file should exist at: - -``` -config/historical -``` - -In the directory we just created, we should have the file `runtime.properties` with the following contents: - -``` -druid.host=localhost -druid.port=8083 -druid.service=druid/historical - -# We can only 1 scan segment in parallel with these configs. -# Our intermediate buffer is also very small so longer topNs will be slow. -druid.processing.buffer.sizeBytes=100000000 -druid.processing.numThreads=1 - -druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}] -druid.server.maxSize=10000000000 -``` - -To start the historical node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/historical:lib/* io.druid.cli.Main server historical -``` - -#### Start a Broker Node - -Broker nodes are responsible for figuring out which historical and/or realtime nodes correspond to which queries. They also merge partial results from these nodes in a scatter/gather fashion. -For more information about Broker nodes, see [here](../design/broker.html). - -The broker config file should exist at: - -``` -config/broker -``` - -In the directory, there should be a `runtime.properties` file with the following contents: - -``` -druid.host=localhost -druid.port=8082 -druid.service=druid/broker - -druid.broker.cache.useCache=true -druid.broker.cache.populateCache=true - -# Bump these up only for faster nested groupBy -druid.processing.buffer.sizeBytes=100000000 -druid.processing.numThreads=1 -``` - -To start the broker node: - -```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath config/_common:config/broker:lib/* io.druid.cli.Main server broker -``` - -#### Start a Realtime Node - -Our goal is to ingest some data and hand-off that data to the rest of our Druid cluster. To accomplish this goal, we need to make some small configuration changes. - -In your favorite editor, open up: - -``` -examples/wikipedia/wikipedia_realtime.spec -``` - -We need to change some configuration in order to force hand-off faster. - -Let's change: - -``` -"segmentGranularity": "HOUR", -``` - -to - -``` -"segmentGranularity": "FIVE_MINUTE", -``` - -and - -``` -"intermediatePersistPeriod": "PT10m", -"windowPeriod": "PT10m", -``` - -to - -``` -"intermediatePersistPeriod": "PT3m", -"windowPeriod": "PT1m", -``` - -Now we should be handing off segments every 6 minutes or so. - -To start the realtime node that was used in our first tutorial, you simply have to issue: - -``` -java -Xmx512m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=examples/wikipedia/wikipedia_realtime.spec -classpath config/_common:config/realtime:lib/* io.druid.cli.Main server realtime -``` - -The configurations are located in `config/realtime/runtime.properties` and should contain the following: - -``` -druid.host=localhost -druid.port=8084 -druid.service=druid/realtime - -# We can only 1 scan segment in parallel with these configs. -# Our intermediate buffer is also very small so longer topNs will be slow. -druid.processing.buffer.sizeBytes=100000000 -druid.processing.numThreads=2 - -# Enable Real monitoring -# druid.monitoring.monitors=["com.metamx.metrics.JvmMonitor","io.druid.segment.realtime.RealtimeMetricsMonitor"] -``` - -Once the real-time node starts up, it should begin ingesting data and handing that data off to the rest of the Druid cluster. You can use a web UI located at coordinator_ip:port to view the status of data being loaded. Once data is handed off from the real-time nodes to historical nodes, the historical nodes should begin serving segments. - -#### Query - -At any point during ingestion, we can query for data. For example: - -``` -curl -X POST 'http://localhost:8082/druid/v2/?pretty' -H 'content-type: application/json' -d@examples/wikipedia/query.body -``` - -This query will span across both realtime and historical nodes. If you're curious, you can query the historical node directly by sending the same query to the historical node's port: - -``` -curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json' -d@examples/wikipedia/query.body -``` - -This query may produce no results if the realtime node hasn't run long enough to hand off the segment (we configured it above to be 5 minutes). Query the realtime node directly by sending the same query to the realtime node's port: - -``` -curl -X POST 'http://localhost:8084/druid/v2/?pretty' -H 'content-type: application/json' -d@examples/wikipedia/query.body -``` - -The realtime query results will reflect the data that was recently indexed from wikipedia, and not handed off to the historical node yet. Once the historical node acknowledges it has loaded the segment, the realtime node will drop the segment. - -Querying the historical and realtime node directly is useful for understanding how the segment handling is working, but if you just want to run a query for all the data (realtime and historical), then send the query to the broker at port 8082 (which is what we did in the first example). The broker will send the query to the historical and realtime nodes and merge the results. - -For more information on querying, see this [link](../querying/querying.html). - -Next Steps ----------- -If you are interested in how data flows through the different Druid components, check out the [Druid data flow architecture](../design/design.html). Now that you have an understanding of what the Druid cluster looks like, why not load some of your own data? -Check out the next [tutorial](../tutorials/tutorial-loading-streaming-data.html) section for more info! diff --git a/examples/bin/examples/indexing/wikipedia.spec b/examples/bin/examples/indexing/wikipedia.spec deleted file mode 100644 index 0120510eec3a..000000000000 --- a/examples/bin/examples/indexing/wikipedia.spec +++ /dev/null @@ -1,73 +0,0 @@ -[ - { - "dataSchema" : { - "dataSource" : "wikipedia", - "parser" : { - "type" : "string", - "parseSpec" : { - "format" : "json", - "timestampSpec" : { - "column" : "timestamp", - "format" : "auto" - }, - "dimensionsSpec" : { - "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], - "dimensionExclusions" : [], - "spatialDimensions" : [] - } - } - }, - "metricsSpec" : [{ - "type" : "count", - "name" : "count" - }, { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" - }], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "DAY", - "queryGranularity" : "NONE" - } - }, - "ioConfig" : { - "type" : "realtime", - "firehose": { - "type": "kafka-0.8", - "consumerProps": { - "zookeeper.connect": "localhost:2181", - "zookeeper.connection.timeout.ms" : "15000", - "zookeeper.session.timeout.ms" : "15000", - "zookeeper.sync.time.ms" : "5000", - "group.id": "druid-example", - "fetch.message.max.bytes" : "1048586", - "auto.offset.reset": "largest", - "auto.commit.enable": "false" - }, - "feed": "wikipedia" - }, - "plumber": { - "type": "realtime" - } - }, - "tuningConfig": { - "type" : "realtime", - "maxRowsInMemory": 500000, - "intermediatePersistPeriod": "PT10m", - "windowPeriod": "PT10m", - "basePersistDirectory": "\/tmp\/realtime\/basePersist", - "rejectionPolicy": { - "type": "messageTime" - } - } - } -] diff --git a/examples/bin/examples/indexing/wikipedia_data.json b/examples/bin/examples/indexing/wikipedia_data.json deleted file mode 100644 index 261840b50093..000000000000 --- a/examples/bin/examples/indexing/wikipedia_data.json +++ /dev/null @@ -1,5 +0,0 @@ -{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} -{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} -{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} -{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} -{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "cancer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file diff --git a/examples/bin/examples/indexing/wikipedia_hadoop_config.json b/examples/bin/examples/indexing/wikipedia_hadoop_config.json deleted file mode 100644 index 44b13d32d7bc..000000000000 --- a/examples/bin/examples/indexing/wikipedia_hadoop_config.json +++ /dev/null @@ -1,82 +0,0 @@ -{ - "dataSchema": { - "dataSource": "wikipedia", - "parser": { - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - } - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE", - "intervals": ["2013-08-31/2013-09-01"] - } - }, - "ioConfig": { - "type": "hadoop", - "inputSpec": { - "type": "static", - "paths": "/myPath/druid-/examples/indexing/wikipedia_data.json" - }, - "metadataUpdateSpec": { - "type": "db", - "connectURI": "jdbc:mysql:\/\/localhost:3306\/druid", - "user": "druid", - "password": "diurd", - "segmentTable": "druid_segments" - }, - "segmentOutputPath": "\/tmp\/segments" - }, - "tuningConfig": { - "type": "hadoop", - "workingPath": "\/tmp\/working_path", - "partitionsSpec": { - "targetPartitionSize": 5000000 - } - } -} diff --git a/examples/bin/examples/indexing/wikipedia_index_hadoop_task.json b/examples/bin/examples/indexing/wikipedia_index_hadoop_task.json deleted file mode 100644 index 697f1a24fa51..000000000000 --- a/examples/bin/examples/indexing/wikipedia_index_hadoop_task.json +++ /dev/null @@ -1,76 +0,0 @@ -{ - "type": "index_hadoop", - "spec": { - "dataSchema": { - "dataSource": "wikipedia", - "parser": { - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - } - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE", - "intervals": ["2013-08-31/2013-09-01"] - } - }, - "ioConfig": { - "type": "hadoop", - "inputSpec": { - "type": "static", - "paths": "examples/indexing/wikipedia_data.json" - } - }, - "tuningConfig": { - "type": "hadoop", - "partitionsSpec": { - "targetPartitionSize": 5000000 - } - } - } -} diff --git a/examples/bin/examples/indexing/wikipedia_index_task.json b/examples/bin/examples/indexing/wikipedia_index_task.json deleted file mode 100644 index 54fc7dcbd730..000000000000 --- a/examples/bin/examples/indexing/wikipedia_index_task.json +++ /dev/null @@ -1,76 +0,0 @@ -{ - "type": "index", - "spec": { - "dataSchema": { - "dataSource": "wikipedia", - "parser": { - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - } - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE", - "intervals": ["2013-08-31/2013-09-01"] - } - }, - "ioConfig": { - "type": "index", - "firehose": { - "type": "local", - "baseDir": "examples/indexing/", - "filter": "wikipedia_data.json" - } - }, - "tuningConfig": { - "type": "index", - "targetPartitionSize": 0, - "rowFlushBoundary": 0 - } - } -} diff --git a/examples/bin/examples/indexing/wikipedia_realtime_task.json b/examples/bin/examples/indexing/wikipedia_realtime_task.json deleted file mode 100644 index 18d076d59c61..000000000000 --- a/examples/bin/examples/indexing/wikipedia_realtime_task.json +++ /dev/null @@ -1,110 +0,0 @@ -{ - "type": "index_realtime", - "spec": { - "dataSchema": { - "dataSource": "wikipedia", - "parser": { - "type": "irc", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - }, - "decoder": { - "type": "wikipedia", - "namespaces": { - "#en.wikipedia": { - "_empty_": "main", - "Category": "category", - "$1 talk": "project talk", - "Template talk": "template talk", - "Help talk": "help talk", - "Media": "media", - "MediaWiki talk": "mediawiki talk", - "File talk": "file talk", - "MediaWiki": "mediawiki", - "User": "user", - "File": "file", - "User talk": "user talk", - "Template": "template", - "Help": "help", - "Special": "special", - "Talk": "talk", - "Category talk": "category talk" - } - } - } - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE" - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "irc", - "nick": "wiki1234567890", - "host": "irc.wikimedia.org", - "channels": [ - "#en.wikipedia", - "#fr.wikipedia", - "#de.wikipedia", - "#ja.wikipedia" - ] - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 500000, - "intermediatePersistPeriod": "PT10m", - "windowPeriod": "PT10m", - "basePersistDirectory": "\/tmp\/realtime\/basePersist", - "rejectionPolicy": { - "type": "serverTime" - } - } - } -} diff --git a/examples/bin/examples/twitter/after.sh b/examples/bin/examples/twitter/after.sh deleted file mode 100755 index 7d73969640b9..000000000000 --- a/examples/bin/examples/twitter/after.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash -set +u -shopt -s xpg_echo -shopt -s expand_aliases - -PF=./twitter4j.properties - -# if twitter4j.properties already existed, then user is okay with having twitter pw in file, don't remove - -if [ -e "created" ]; then - rm -f ${PF} - rm -f created -fi diff --git a/examples/bin/examples/twitter/before.sh b/examples/bin/examples/twitter/before.sh deleted file mode 100755 index ffcbac42c58b..000000000000 --- a/examples/bin/examples/twitter/before.sh +++ /dev/null @@ -1,28 +0,0 @@ -#!/usr/bin/env bash -set +u -shopt -s xpg_echo -shopt -s expand_aliases - -PF=./twitter4j.properties - -# if twitter4j.properties already exists, then user is okay with having twitter pw in file. -# Otherwise a twitter4j.properties file in curr. dir. is made temporarily for twitter login. -if [ ! -e "$PF" ]; then - PF_CLEANUP="/bin/rm $PF" - trap "${PF_CLEANUP} ; exit 1" 1 2 3 15 - touch created - touch $PF - chmod 700 $PF - echo " Your twitter OAuth information is needed. Go to https://twitter.com/oauth_clients/new to register a new application and retrieve your keys " - read -p 'Twitter consumer key? ' CONSUMER_KEY - read -p 'Twitter consumer secret? ' CONSUMER_SECRET - read -p 'Twitter access token? ' ACCESS_TOKEN - read -p 'Twitter access token secret? ' ACCESS_TOKEN_SECRET - echo "debug=true" >> $PF - echo "oauth.consumerKey=${CONSUMER_KEY}" >> $PF - echo "oauth.consumerSecret=${CONSUMER_SECRET}" >> $PF - echo "oauth.accessToken=${ACCESS_TOKEN}" >> $PF - echo "oauth.accessTokenSecret=${ACCESS_TOKEN_SECRET}" >> $PF - CONSUMER_SECRET="" - ACCESS_TOKEN_SECRET="" -fi \ No newline at end of file diff --git a/examples/bin/examples/twitter/query.body b/examples/bin/examples/twitter/query.body deleted file mode 100644 index 76d3b9baae63..000000000000 --- a/examples/bin/examples/twitter/query.body +++ /dev/null @@ -1,31 +0,0 @@ -{ - "description": "Simple data split up by hour", - "aggregations": [ - { - "name": "tweets", - "type": "longSum", - "fieldName": "tweets" - }, - { - "fieldName": "text_hll", - "name": "text_hll", - "type": "hyperUnique" - }, - { - "fieldName": "htags_hll", - "name": "htag_hll", - "type": "hyperUnique" - }, - { - "fieldName": "user_id_hll", - "name": "user_id_hll", - "type": "hyperUnique" - } - ], - "dataSource": "twitterstream", - "granularity": "hour", - "intervals": [ - "1970-01-01T00:00:00.000/2019-01-03T00:00:00.000" - ], - "queryType": "timeseries" -} \ No newline at end of file diff --git a/examples/bin/examples/twitter/search_query.body b/examples/bin/examples/twitter/search_query.body deleted file mode 100644 index 94d467adcc89..000000000000 --- a/examples/bin/examples/twitter/search_query.body +++ /dev/null @@ -1,21 +0,0 @@ -{ - "queryType": "search", - "dataSource": "twitterstream", - "granularity": "all", - "searchDimensions": [ - "htags" - ], - "limit": 1, - "query": { - "type": "fragment", - "values": [ - "men" - ], - "sort": { - "type": "strlen" - } - }, - "intervals": [ - "2012-10-01T00:00\/2020-01-01T00" - ] -} diff --git a/examples/bin/examples/twitter/topN_query.body b/examples/bin/examples/twitter/topN_query.body deleted file mode 100644 index fcd644a4f1c4..000000000000 --- a/examples/bin/examples/twitter/topN_query.body +++ /dev/null @@ -1,94 +0,0 @@ -{ - "description": "Top 10 languages by count of tweets in the contiguous US", - "aggregations": [ - { - "fieldName": "tweets", - "name": "tweets", - "type": "longSum" - }, - { - "fieldName": "user_id_hll", - "name": "user_id_hll", - "type": "hyperUnique" - }, - { - "fieldName": "contributors_hll", - "name": "contributors_hll", - "type": "hyperUnique" - }, - { - "fieldName": "htags_hll", - "name": "htags_hll", - "type": "hyperUnique" - }, - { - "fieldName": "text_hll", - "name": "text_hll", - "type": "hyperUnique" - }, - { - "fieldName": "min_follower_count", - "name": "min_follower_count", - "type": "min" - }, - { - "fieldName": "max_follower_count", - "name": "max_follower_count", - "type": "max" - }, - { - "fieldName": "min_friends_count", - "name": "min_friends_count", - "type": "min" - }, - { - "fieldName": "max_friends_count", - "name": "max_friends_count", - "type": "max" - }, - { - "fieldName": "min_statuses_count", - "name": "min_statuses_count", - "type": "min" - }, - { - "fieldName": "max_statuses_count", - "name": "max_statuses_count", - "type": "max" - }, - { - "fieldName": "min_retweet_count", - "name": "min_retweet_count", - "type": "min" - }, - { - "fieldName": "max_retweet_count", - "name": "max_retweet_count", - "type": "max" - } - ], - "dataSource": "twitterstream", - "dimension": "lang", - "filter": { - "bound": { - "maxCoords": [ - 50, - -65 - ], - "minCoords": [ - 25, - -127 - ], - "type": "rectangular" - }, - "dimension": "geo", - "type": "spatial" - }, - "granularity": "all", - "intervals": [ - "2013-06-01T00:00/2020-01-01T00" - ], - "metric": "tweets", - "queryType": "topN", - "threshold": "10" -} \ No newline at end of file diff --git a/examples/bin/examples/twitter/twitter_realtime.spec b/examples/bin/examples/twitter/twitter_realtime.spec deleted file mode 100644 index ed9039b0f0c3..000000000000 --- a/examples/bin/examples/twitter/twitter_realtime.spec +++ /dev/null @@ -1,196 +0,0 @@ -[ - { - "dataSchema": { - "dataSource": "twitterstream", - "parser": { - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "utcdt", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "text", - "htags", - "contributors", - "lat", - "lon", - "source", - "retweet", - "retweet_count", - "originator_screen_name", - "originator_follower_count", - "originator_friends_count", - "originator_verified", - "follower_count", - "friends_count", - "lang", - "utc_offset", - "statuses_count", - "user_id", - "screen_name", - "location", - "verified", - "ts" - ], - "dimensionExclusions": [ - - ], - "spatialDimensions": [ - { - "dimName": "geo", - "dims": [ - "lat", - "lon" - ] - } - ] - } - } - }, - "metricsSpec": [ - { - "name": "tweets", - "type": "count" - }, - { - "fieldName": "follower_count", - "name": "total_follower_count", - "type": "doubleSum" - }, - { - "fieldName": "retweet_count", - "name": "total_retweet_count", - "type": "doubleSum" - }, - { - "fieldName": "friends_count", - "name": "total_friends_count", - "type": "doubleSum" - }, - { - "fieldName": "statuses_count", - "name": "total_statuses_count", - "type": "doubleSum" - }, - { - "fieldName": "originator_follower_count", - "name": "total_originator_follower_count", - "type": "doubleSum" - }, - { - "fieldName": "originator_friends_count", - "name": "total_originator_friends_count", - "type": "doubleSum" - }, - { - "fieldName": "text", - "name": "text_hll", - "type": "hyperUnique" - }, - { - "fieldName": "user_id", - "name": "user_id_hll", - "type": "hyperUnique" - }, - { - "fieldName": "contributors", - "name": "contributors_hll", - "type": "hyperUnique" - }, - { - "fieldName": "htags", - "name": "htags_hll", - "type": "hyperUnique" - }, - { - "fieldName": "follower_count", - "name": "min_follower_count", - "type": "min" - }, - { - "fieldName": "follower_count", - "name": "max_follower_count", - "type": "max" - }, - { - "fieldName": "friends_count", - "name": "min_friends_count", - "type": "min" - }, - { - "fieldName": "friends_count", - "name": "max_friends_count", - "type": "max" - }, - { - "fieldName": "statuses_count", - "name": "min_statuses_count", - "type": "min" - }, - { - "fieldName": "statuses_count", - "name": "max_statuses_count", - "type": "max" - }, - { - "fieldName": "retweet_count", - "name": "min_retweet_count", - "type": "min" - }, - { - "fieldName": "retweet_count", - "name": "max_retweet_count", - "type": "max" - }, - { - "fieldName": "originator_follower_count", - "name": "min_originator_follower_count", - "type": "min" - }, - { - "fieldName": "originator_follower_count", - "name": "max_originator_follower_count", - "type": "max" - }, - { - "fieldName": "originator_friends_count", - "name": "min_originator_friends_count", - "type": "min" - }, - { - "fieldName": "originator_friends_count", - "name": "max_originator_friends_count", - "type": "max" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "NONE" - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "twitzer", - "maxEventCount": 500000, - "maxRunMinutes": 120 - }, - "plumber": { - "type": "realtime" - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 500000, - "intermediatePersistPeriod": "PT2m", - "windowPeriod": "PT3m", - "basePersistDirectory": "\/tmp\/realtime\/basePersist", - "rejectionPolicy": { - "type": "messageTime" - } - } - } -] diff --git a/examples/bin/examples/wikipedia/query.body b/examples/bin/examples/wikipedia/query.body deleted file mode 100644 index f90d4b080aba..000000000000 --- a/examples/bin/examples/wikipedia/query.body +++ /dev/null @@ -1,4 +0,0 @@ -{ - "queryType":"timeBoundary", - "dataSource":"wikipedia" -} diff --git a/examples/bin/examples/wikipedia/wikipedia_realtime.spec b/examples/bin/examples/wikipedia/wikipedia_realtime.spec deleted file mode 100644 index f04d09384939..000000000000 --- a/examples/bin/examples/wikipedia/wikipedia_realtime.spec +++ /dev/null @@ -1,110 +0,0 @@ -[{ - "dataSchema": { - "dataSource": "wikipedia", - "parser": { - "type": "irc", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - "language", - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - }, - "decoder": { - "type": "wikipedia", - "namespaces": { - "#en.wikipedia": { - "_empty_": "main", - "Category": "category", - "$1 talk": "project talk", - "Template talk": "template talk", - "Help talk": "help talk", - "Media": "media", - "MediaWiki talk": "mediawiki talk", - "File talk": "file talk", - "MediaWiki": "mediawiki", - "User": "user", - "File": "file", - "User talk": "user talk", - "Template": "template", - "Help": "help", - "Special": "special", - "Talk": "talk", - "Category talk": "category talk" - } - } - } - }, - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "HOUR", - "queryGranularity": "NONE" - } - }, - "ioConfig": { - "type": "realtime", - "firehose": { - "type": "irc", - "host": "irc.wikimedia.org", - "channels": [ - "#en.wikipedia", - "#fr.wikipedia", - "#de.wikipedia", - "#ja.wikipedia" - ] - }, - "plumber": { - "type": "realtime" - } - }, - "tuningConfig": { - "type": "realtime", - "maxRowsInMemory": 500000, - "intermediatePersistPeriod": "PT10m", - "windowPeriod": "PT10m", - "basePersistDirectory": "\/tmp\/realtime\/basePersist", - "rejectionPolicy": { - "type": "serverTime" - } - } -}] - diff --git a/examples/bin/generate-example-metrics b/examples/bin/generate-example-metrics new file mode 100755 index 000000000000..57f78fe4c587 --- /dev/null +++ b/examples/bin/generate-example-metrics @@ -0,0 +1,48 @@ +#!/usr/bin/env python + +import argparse +import json +import random +import sys +from datetime import datetime + +def main(): + parser = argparse.ArgumentParser(description='Generate example page request latency metrics.') + parser.add_argument('--count', '-c', type=int, default=25, help='Number of events to generate (negative for unlimited)') + args = parser.parse_args() + + count = 0 + while args.count < 0 or count < args.count: + timestamp = datetime.utcnow().strftime("%Y-%m-%dT%H:%M:%SZ") + + r = random.randint(1, 4) + if r == 1 or r == 2: + page = '/' + elif r == 3: + page = '/list' + else: + page = '/get/' + str(random.randint(1, 99)) + + server = 'www' + str(random.randint(1, 5)) + '.example.com' + + latency = max(1, random.gauss(80, 40)) + + print(json.dumps({ + 'timestamp': timestamp, + 'metricType': 'request/latency', + 'value': int(latency), + + # Additional dimensions + 'page': page, + 'server': server, + 'http_method': 'GET', + 'http_code': '200', + 'unit': 'milliseconds' + })) + + count += 1 + +try: + main() +except KeyboardInterrupt: + sys.exit(1) diff --git a/examples/bin/init b/examples/bin/init new file mode 100644 index 000000000000..241d9ee3f188 --- /dev/null +++ b/examples/bin/init @@ -0,0 +1,12 @@ +#!/bin/bash -eu + +gzip -c -d quickstart/wikiticker-2015-09-12-sampled.json.gz > "quickstart/wikiticker-2015-09-12-sampled.json" + +LOG_DIR=var + +mkdir -p $LOG_DIR/tmp; +mkdir -p $LOG_DIR/druid/indexing-logs; +mkdir -p $LOG_DIR/druid/segments; +mkdir -p $LOG_DIR/druid/segment-cache; +mkdir -p $LOG_DIR/druid/task; +mkdir -p $LOG_DIR/druid/hadoop-tmp; diff --git a/examples/bin/run_druid_server.sh b/examples/bin/run_druid_server.sh deleted file mode 100755 index bfbc8276463b..000000000000 --- a/examples/bin/run_druid_server.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash -set +e -set +u -shopt -s xpg_echo -shopt -s expand_aliases -trap "exit 1" 1 2 3 15 - -SCRIPT_DIR=`dirname $0` -MAVEN_DIR="${SCRIPT_DIR}/extensions-repo" -SERVER_TYPE="$1" - -if [ "x${SERVER_TYPE}" = "x" ] -then - echo "usage: $0 server-type" >& 2 - exit 2 -fi - -if [[ ! -d "${SCRIPT_DIR}/lib" || ! -d "${SCRIPT_DIR}/config" ]]; then - echo "This script appears to be running from the source location. It must be run from its deployed location." - echo "After building, unpack services/target/druid-services-*-SNAPSHOT-bin.tar.gz, and run the script unpacked there." - exit 2 -fi - -CURR_DIR=`pwd` -cd ${SCRIPT_DIR} -SCRIPT_DIR=`pwd` -cd ${CURR_DIR} - -# start process -JAVA_ARGS="${JAVA_ARGS} -Xmx512m -Duser.timezone=UTC -Dfile.encoding=UTF-8" - -DRUID_CP="${SCRIPT_DIR}/config/_common" -DRUID_CP="${DRUID_CP}:${SCRIPT_DIR}/config/$SERVER_TYPE" -DRUID_CP="${DRUID_CP}:${SCRIPT_DIR}/lib/*" - -exec java ${JAVA_ARGS} -classpath "${DRUID_CP}" io.druid.cli.Main server "$SERVER_TYPE" diff --git a/examples/bin/run_example_client.sh b/examples/bin/run_example_client.sh deleted file mode 100755 index 8e64a6d0de89..000000000000 --- a/examples/bin/run_example_client.sh +++ /dev/null @@ -1,30 +0,0 @@ -#!/usr/bin/env bash -echo "This will run a query against a stand-alone version of Druid" -echo " before running this, do: run_example_server.sh" -set +u -shopt -s xpg_echo -shopt -s expand_aliases -trap "exit 1" 1 2 3 15 - -SCRIPT_DIR=`dirname $0` -CURR_DIR=`pwd` -cd ${SCRIPT_DIR} -SCRIPT_DIR=`pwd` -cd ${CURR_DIR} - -source $SCRIPT_DIR/select_example.sh - -select_example QUERY_FILE "${SCRIPT_DIR}/examples" "*query.body" "${1}" "query.body" - -cat ${QUERY_FILE} -for delay in 5 30 30 30 30 30 30 30 30 30 30 - do - echo "sleep for $delay seconds..." - echo " " - sleep $delay - curl -X POST 'http://localhost:8084/druid/v2/?pretty' -H 'content-type: application/json' -d "`cat ${QUERY_FILE}`" - echo " " - echo " " -done - -echo "$0 finished" diff --git a/examples/bin/run_example_server.sh b/examples/bin/run_example_server.sh deleted file mode 100755 index eb0893ede5ac..000000000000 --- a/examples/bin/run_example_server.sh +++ /dev/null @@ -1,52 +0,0 @@ -#!/usr/bin/env bash -echo "This will run a stand-alone version of Druid" -set +u -shopt -s xpg_echo -shopt -s expand_aliases -trap "exit 1" 1 2 3 15 - -SCRIPT_DIR=`dirname $0` -MAVEN_DIR="${SCRIPT_DIR}/extensions-repo" - -if [[ ! -d "${SCRIPT_DIR}/lib" || ! -d "${SCRIPT_DIR}/config" ]]; then - echo "This script appears to be running from the source location. It must be run from its deployed location." - echo "After building, unpack services/target/druid-services-*-SNAPSHOT-bin.tar.gz, and run the script unpacked there." - exit 2 -fi - -CURR_DIR=`pwd` -cd ${SCRIPT_DIR} -SCRIPT_DIR=`pwd` -cd ${CURR_DIR} - -[ -d /tmp/example ] && echo "Cleaning up from previous run.." && /bin/rm -fr /tmp/example - -source $SCRIPT_DIR/select_example.sh - -select_example SPEC_FILE "${SCRIPT_DIR}/examples" "*_realtime.spec" "${1}" "${1}_realtime.spec" - -EXAMPLE_LOC=$(dirname $SPEC_FILE) -# run before script if it exists -if [ -x ${EXAMPLE_LOC}/before.sh ]; then - trap "set +x; cd ${EXAMPLE_LOC} && ./after.sh && cd ${CURR_DIR}; exit 1" EXIT - cd ${EXAMPLE_LOC} - ./before.sh - cd ${CURR_DIR} -fi - -# start process -JAVA_ARGS="-Xmx512m -Duser.timezone=UTC -Dfile.encoding=UTF-8" -JAVA_ARGS="${JAVA_ARGS} -Ddruid.realtime.specFile=${SPEC_FILE}" -JAVA_ARGS="${JAVA_ARGS} -Ddruid.publish.type=noop" - -DRUID_CP=${EXAMPLE_LOC} -#For a pull -DRUID_CP=${SCRIPT_DIR}/../config/realtime:${DRUID_CP} -#For the kit -DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config/_common -DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config/realtime -DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/lib/* - -echo "Running command:" - -(set -x; java ${JAVA_ARGS} -classpath ${DRUID_CP} io.druid.cli.Main example realtime) diff --git a/examples/bin/select_example.sh b/examples/bin/select_example.sh deleted file mode 100755 index f90111e99f56..000000000000 --- a/examples/bin/select_example.sh +++ /dev/null @@ -1,38 +0,0 @@ -#!/usr/bin/env bash - -function select_example() { - example_outvar=$1 - examples_dir=$2 - find_pattern=$3 - example_arg=$4 - default_file=$5 - - if [[ -n ${example_arg} ]]; then - if [[ -f ${example_arg} ]]; then - example_file=${example_arg} - else - example_file="${examples_dir}/${example_arg}/${default_file}" - fi - fi - - all_examples=($(find ${examples_dir} -name "${find_pattern}")) - while [[ -z ${example_file} || ! -f ${example_file} ]] ; do - if [[ -n ${example_file} ]]; then - echo "No example found at ${example_file}." - fi - echo "Please specify an example by its number." - echo "Examples available:" - LINE=0 - for e in ${all_examples[@]}; do - LINE=$((LINE+1)) - REL_FILE=${e#${examples_dir}/} - DESC=`grep 'description' $e | tail -1 | sed 's/"description"[^"]*"\([^"]*\)".*/\1/' ` - echo "${LINE} - ${REL_FILE} - ${DESC:-No Description}" - done - read -p "[1] > " NUM_SELECTED - echo " " - NUM_SELECTED=${NUM_SELECTED:-1} - example_file=${all_examples[$((NUM_SELECTED-1))]} - done - eval $example_outvar="'$example_file'" -} diff --git a/examples/conf-quickstart/druid/_common/common.runtime.properties b/examples/conf-quickstart/druid/_common/common.runtime.properties new file mode 100644 index 000000000000..23e94bb89c30 --- /dev/null +++ b/examples/conf-quickstart/druid/_common/common.runtime.properties @@ -0,0 +1,108 @@ +# +# Licensed to Metamarkets Group Inc. (Metamarkets) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Metamarkets licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# Extensions specified in the load list will be loaded by Druid +# We are using local fs for deep storage - not recommended for production - use S3, HDFS, or NFS instead +# We are using local derby for the metadata store - recommended for production - use MySQL or Postgres instead + +# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. +# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. +# More info: http://druid.io/docs/latest/operations/including-extensions.html +druid.extensions.loadList=[] + +# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory +# and uncomment the line below to point to your directory. +#druid.extensions.hadoopDependenciesDir=/my/dir/hadoop-dependencies + +# +# Zookeeper +# + +druid.zk.service.host=localhost +druid.zk.paths.base=/druid + +# +# Metadata storage +# + +druid.metadata.storage.type=derby +druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527/var/druid/metadata.db;create=true +druid.metadata.storage.connector.host=localhost +druid.metadata.storage.connector.port=1527 + +# For MySQL: +#druid.metadata.storage.type=mysql +#druid.metadata.storage.connector.connectURI=jdbc:mysql://db.example.com:3306/druid +#druid.metadata.storage.connector.user=... +#druid.metadata.storage.connector.password=... + +# For PostgreSQL: +#druid.metadata.storage.type=postgresql +#druid.metadata.storage.connector.connectURI=jdbc:postgresql://db.example.com:5432/druid +#druid.metadata.storage.connector.user=... +#druid.metadata.storage.connector.password=... + +# +# Deep storage +# + +druid.storage.type=local +druid.storage.storageDirectory=var/druid/segments + +# For HDFS: +#druid.storage.type=hdfs +#druid.storage.storageDirectory=/druid/segments + +# For S3: +#druid.storage.type=s3 +#druid.storage.bucket=your-bucket +#druid.storage.baseKey=druid/segments +#druid.s3.accessKey=... +#druid.s3.secretKey=... + +# +# Indexing service logs +# + +druid.indexer.logs.type=file +druid.indexer.logs.directory=var/druid/indexing-logs + +# For HDFS: +#druid.indexer.logs.type=hdfs +#druid.indexer.logs.directory=/druid/indexing-logs + +# For S3: +#druid.indexer.logs.type=s3 +#druid.indexer.logs.s3Bucket=your-bucket +#druid.indexer.logs.s3Prefix=druid/indexing-logs + +# +# Service discovery +# + +druid.selectors.indexing.serviceName=druid/overlord +druid.selectors.coordinator.serviceName=druid/coordinator + +# +# Monitoring +# + +druid.monitoring.monitors=["com.metamx.metrics.JvmMonitor"] +druid.emitter=logging +druid.emitter.logging.logLevel=debug diff --git a/examples/conf-quickstart/druid/_common/log4j2.xml b/examples/conf-quickstart/druid/_common/log4j2.xml new file mode 100644 index 000000000000..a60ebda9eb42 --- /dev/null +++ b/examples/conf-quickstart/druid/_common/log4j2.xml @@ -0,0 +1,33 @@ + + + + + + + + + + + + + + + + diff --git a/examples/conf-quickstart/druid/broker/jvm.config b/examples/conf-quickstart/druid/broker/jvm.config new file mode 100644 index 000000000000..430209ec98ec --- /dev/null +++ b/examples/conf-quickstart/druid/broker/jvm.config @@ -0,0 +1,8 @@ +-server +-Xms1g +-Xmx1g +-XX:MaxDirectMemorySize=1280m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf-quickstart/druid/broker/runtime.properties b/examples/conf-quickstart/druid/broker/runtime.properties new file mode 100644 index 000000000000..a01ab07340d4 --- /dev/null +++ b/examples/conf-quickstart/druid/broker/runtime.properties @@ -0,0 +1,16 @@ +druid.service=druid/broker +druid.port=8082 + +# HTTP server threads +druid.broker.http.numConnections=5 +druid.server.http.numThreads=8 + +# Processing threads and buffers +druid.processing.buffer.sizeBytes=256000000 +druid.processing.numThreads=2 + +# Query cache (we use a small local cache) +druid.broker.cache.useCache=true +druid.broker.cache.populateCache=true +druid.cache.type=local +druid.cache.sizeInBytes=10000000 diff --git a/examples/conf-quickstart/druid/coordinator/jvm.config b/examples/conf-quickstart/druid/coordinator/jvm.config new file mode 100644 index 000000000000..66aa951e81ca --- /dev/null +++ b/examples/conf-quickstart/druid/coordinator/jvm.config @@ -0,0 +1,8 @@ +-server +-Xms256m +-Xmx256m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager +-Dderby.stream.error.file=var/druid/derby.log diff --git a/examples/conf-quickstart/druid/coordinator/runtime.properties b/examples/conf-quickstart/druid/coordinator/runtime.properties new file mode 100644 index 000000000000..a61482f2a0f0 --- /dev/null +++ b/examples/conf-quickstart/druid/coordinator/runtime.properties @@ -0,0 +1,5 @@ +druid.service=druid/coordinator +druid.port=8081 + +druid.coordinator.startDelay=PT10S +druid.coordinator.period=PT5S diff --git a/examples/conf-quickstart/druid/historical/jvm.config b/examples/conf-quickstart/druid/historical/jvm.config new file mode 100644 index 000000000000..430209ec98ec --- /dev/null +++ b/examples/conf-quickstart/druid/historical/jvm.config @@ -0,0 +1,8 @@ +-server +-Xms1g +-Xmx1g +-XX:MaxDirectMemorySize=1280m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf-quickstart/druid/historical/runtime.properties b/examples/conf-quickstart/druid/historical/runtime.properties new file mode 100644 index 000000000000..40342dcb0ccf --- /dev/null +++ b/examples/conf-quickstart/druid/historical/runtime.properties @@ -0,0 +1,13 @@ +druid.service=druid/historical +druid.port=8083 + +# HTTP server threads +druid.server.http.numThreads=8 + +# Processing threads and buffers +druid.processing.buffer.sizeBytes=256000000 +druid.processing.numThreads=2 + +# Segment storage +druid.segmentCache.locations=[{"path":"var/druid/segment-cache","maxSize"\:300000000000}] +druid.server.maxSize=300000000000 diff --git a/examples/conf-quickstart/druid/middleManager/jvm.config b/examples/conf-quickstart/druid/middleManager/jvm.config new file mode 100644 index 000000000000..7b9f1a47b55d --- /dev/null +++ b/examples/conf-quickstart/druid/middleManager/jvm.config @@ -0,0 +1,7 @@ +-server +-Xms64m +-Xmx64m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf-quickstart/druid/middleManager/runtime.properties b/examples/conf-quickstart/druid/middleManager/runtime.properties new file mode 100644 index 000000000000..cd250f33495c --- /dev/null +++ b/examples/conf-quickstart/druid/middleManager/runtime.properties @@ -0,0 +1,20 @@ +druid.service=druid/middleManager +druid.port=8091 + +# Number of tasks per middleManager +druid.worker.capacity=3 + +# Task launch parameters +druid.indexer.runner.javaOpts=-server -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager +druid.indexer.task.baseTaskDir=var/druid/task + +# HTTP server threads +druid.server.http.numThreads=8 + +# Processing threads and buffers +druid.processing.buffer.sizeBytes=256000000 +druid.processing.numThreads=2 + +# Hadoop indexing +druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp +druid.indexer.task.defaultHadoopCoordinates=["org.apache.hadoop:hadoop-client:2.3.0"] diff --git a/examples/conf-quickstart/druid/overlord/jvm.config b/examples/conf-quickstart/druid/overlord/jvm.config new file mode 100644 index 000000000000..90526a53804b --- /dev/null +++ b/examples/conf-quickstart/druid/overlord/jvm.config @@ -0,0 +1,7 @@ +-server +-Xms256m +-Xmx256m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf-quickstart/druid/overlord/runtime.properties b/examples/conf-quickstart/druid/overlord/runtime.properties new file mode 100644 index 000000000000..7ab544ad9367 --- /dev/null +++ b/examples/conf-quickstart/druid/overlord/runtime.properties @@ -0,0 +1,7 @@ +druid.service=druid/overlord +druid.port=8090 + +druid.indexer.queue.startDelay=PT5S + +druid.indexer.runner.type=remote +druid.indexer.storage.type=metadata diff --git a/examples/conf-quickstart/tranquility/kafka.json b/examples/conf-quickstart/tranquility/kafka.json new file mode 100644 index 000000000000..ffe537870d56 --- /dev/null +++ b/examples/conf-quickstart/tranquility/kafka.json @@ -0,0 +1,77 @@ +{ + "dataSources" : { + "metrics-kafka" : { + "spec" : { + "dataSchema" : { + "dataSource" : "metrics-kafka", + "parser" : { + "type" : "string", + "parseSpec" : { + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions" : [], + "dimensionExclusions" : [ + "timestamp", + "value" + ] + }, + "format" : "json" + } + }, + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "hour", + "queryGranularity" : "none" + }, + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "name" : "value_sum", + "type" : "doubleSum", + "fieldName" : "value" + }, + { + "fieldName" : "value", + "name" : "value_min", + "type" : "doubleMin" + }, + { + "type" : "doubleMax", + "name" : "value_max", + "fieldName" : "value" + } + ] + }, + "ioConfig" : { + "type" : "realtime" + }, + "tuningConfig" : { + "type" : "realtime", + "maxRowsInMemory" : "100000", + "intermediatePersistPeriod" : "PT10M", + "windowPeriod" : "PT10M" + } + }, + "properties" : { + "task.partitions" : "1", + "task.replicants" : "1", + "topicPattern" : "metrics" + } + } + }, + "properties" : { + "zookeeper.connect" : "localhost", + "druid.discovery.curator.path" : "/druid/discovery", + "druid.selectors.indexing.serviceName" : "druid/overlord", + "commit.periodMillis" : "15000", + "consumer.numThreads" : "2", + "kafka.zookeeper.connect" : "localhost", + "kafka.group.id" : "tranquility-kafka" + } +} diff --git a/examples/conf-quickstart/tranquility/server.json b/examples/conf-quickstart/tranquility/server.json new file mode 100644 index 000000000000..0072b64777ec --- /dev/null +++ b/examples/conf-quickstart/tranquility/server.json @@ -0,0 +1,74 @@ +{ + "dataSources" : { + "metrics" : { + "spec" : { + "dataSchema" : { + "dataSource" : "metrics", + "parser" : { + "type" : "string", + "parseSpec" : { + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions" : [], + "dimensionExclusions" : [ + "timestamp", + "value" + ] + }, + "format" : "json" + } + }, + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "hour", + "queryGranularity" : "none" + }, + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "name" : "value_sum", + "type" : "doubleSum", + "fieldName" : "value" + }, + { + "fieldName" : "value", + "name" : "value_min", + "type" : "doubleMin" + }, + { + "type" : "doubleMax", + "name" : "value_max", + "fieldName" : "value" + } + ] + }, + "ioConfig" : { + "type" : "realtime" + }, + "tuningConfig" : { + "type" : "realtime", + "maxRowsInMemory" : "100000", + "intermediatePersistPeriod" : "PT10M", + "windowPeriod" : "PT10M" + } + }, + "properties" : { + "task.partitions" : "1", + "task.replicants" : "1" + } + } + }, + "properties" : { + "zookeeper.connect" : "localhost", + "druid.discovery.curator.path" : "/druid/discovery", + "druid.selectors.indexing.serviceName" : "druid/overlord", + "http.port" : "8200", + "http.threads" : "8" + } +} diff --git a/examples/conf/druid/_common/common.runtime.properties b/examples/conf/druid/_common/common.runtime.properties new file mode 100644 index 000000000000..edd2712b873c --- /dev/null +++ b/examples/conf/druid/_common/common.runtime.properties @@ -0,0 +1,107 @@ +# +# Licensed to Metamarkets Group Inc. (Metamarkets) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Metamarkets licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# +# Extensions +# + +# This is not the full list of Druid extensions, but common ones that people often use. You may need to change this list +# based on your particular setup. +druid.extensions.loadList=["druid-kafka-eight", "druid-s3-extensions", "druid-histogram", "druid-datasketches", "druid-namespace-lookup", "mysql-metadata-storage"] + +# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory +# and uncomment the line below to point to your directory. +#druid.extensions.hadoopDependenciesDir=/my/dir/hadoop-dependencies + +# +# Zookeeper +# + +druid.zk.service.host=zk.host.ip +druid.zk.paths.base=/druid + +# +# Metadata storage +# + +druid.metadata.storage.type=derby +druid.metadata.storage.connector.connectURI=jdbc:derby://metadata.store.ip:1527/var/druid/metadata.db;create=true +druid.metadata.storage.connector.host=metadata.store.ip +druid.metadata.storage.connector.port=1527 + +# For MySQL: +#druid.metadata.storage.type=mysql +#druid.metadata.storage.connector.connectURI=jdbc:mysql://db.example.com:3306/druid +#druid.metadata.storage.connector.user=... +#druid.metadata.storage.connector.password=... + +# For PostgreSQL (make sure to additionally include the Postgres extension): +#druid.metadata.storage.type=postgresql +#druid.metadata.storage.connector.connectURI=jdbc:postgresql://db.example.com:5432/druid +#druid.metadata.storage.connector.user=... +#druid.metadata.storage.connector.password=... + +# +# Deep storage +# + +druid.storage.type=local +druid.storage.storageDirectory=var/druid/segments + +# For HDFS (make sure to include the HDFS extension and that your Hadoop config files in the cp): +#druid.storage.type=hdfs +#druid.storage.storageDirectory=/druid/segments + +# For S3: +#druid.storage.type=s3 +#druid.storage.bucket=your-bucket +#druid.storage.baseKey=druid/segments +#druid.s3.accessKey=... +#druid.s3.secretKey=... + +# +# Indexing service logs +# + +druid.indexer.logs.type=file +druid.indexer.logs.directory=var/druid/indexing-logs + +# For HDFS (make sure to include the HDFS extension and that your Hadoop config files in the cp): +#druid.indexer.logs.type=hdfs +#druid.indexer.logs.directory=hdfs://namenode.example.com:9000/druid/indexing-logs + +# For S3: +#druid.indexer.logs.type=s3 +#druid.indexer.logs.s3Bucket=your-bucket +#druid.indexer.logs.s3Prefix=druid/indexing-logs + +# +# Service discovery +# + +druid.selectors.indexing.serviceName=druid/overlord +druid.selectors.coordinator.serviceName=druid/coordinator + +# +# Monitoring +# + +druid.monitoring.monitors=["com.metamx.metrics.JvmMonitor"] +druid.emitter=logging +druid.emitter.logging.logLevel=info diff --git a/examples/conf/druid/_common/log4j2.xml b/examples/conf/druid/_common/log4j2.xml new file mode 100644 index 000000000000..3f2bb574f37a --- /dev/null +++ b/examples/conf/druid/_common/log4j2.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + diff --git a/examples/conf/druid/broker/jvm.config b/examples/conf/druid/broker/jvm.config new file mode 100644 index 000000000000..a6a9982d44c3 --- /dev/null +++ b/examples/conf/druid/broker/jvm.config @@ -0,0 +1,8 @@ +-server +-Xms24g +-Xmx24g +-XX:MaxDirectMemorySize=4096m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf/druid/broker/runtime.properties b/examples/conf/druid/broker/runtime.properties new file mode 100644 index 000000000000..41a81cde3c32 --- /dev/null +++ b/examples/conf/druid/broker/runtime.properties @@ -0,0 +1,16 @@ +druid.service=druid/broker +druid.port=8082 + +# HTTP server threads +druid.broker.http.numConnections=5 +druid.server.http.numThreads=25 + +# Processing threads and buffers +druid.processing.buffer.sizeBytes=536870912 +druid.processing.numThreads=7 + +# Query cache +druid.broker.cache.useCache=true +druid.broker.cache.populateCache=true +druid.cache.type=local +druid.cache.sizeInBytes=2000000000 diff --git a/examples/conf/druid/coordinator/jvm.config b/examples/conf/druid/coordinator/jvm.config new file mode 100644 index 000000000000..99f6d1d646f0 --- /dev/null +++ b/examples/conf/druid/coordinator/jvm.config @@ -0,0 +1,8 @@ +-server +-Xms3g +-Xmx3g +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager +-Dderby.stream.error.file=var/druid/derby.log diff --git a/examples/conf/druid/coordinator/runtime.properties b/examples/conf/druid/coordinator/runtime.properties new file mode 100644 index 000000000000..d956c851643f --- /dev/null +++ b/examples/conf/druid/coordinator/runtime.properties @@ -0,0 +1,5 @@ +druid.service=druid/coordinator +druid.port=8081 + +druid.coordinator.startDelay=PT30S +druid.coordinator.period=PT30S diff --git a/examples/conf/druid/historical/jvm.config b/examples/conf/druid/historical/jvm.config new file mode 100644 index 000000000000..7fdbb1c51ce0 --- /dev/null +++ b/examples/conf/druid/historical/jvm.config @@ -0,0 +1,8 @@ +-server +-Xms8g +-Xmx8g +-XX:MaxDirectMemorySize=4096m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf/druid/historical/runtime.properties b/examples/conf/druid/historical/runtime.properties new file mode 100644 index 000000000000..56f7f6e0feeb --- /dev/null +++ b/examples/conf/druid/historical/runtime.properties @@ -0,0 +1,13 @@ +druid.service=druid/historical +druid.port=8083 + +# HTTP server threads +druid.server.http.numThreads=25 + +# Processing threads and buffers +druid.processing.buffer.sizeBytes=536870912 +druid.processing.numThreads=7 + +# Segment storage +druid.segmentCache.locations=[{"path":"var/druid/segment-cache","maxSize"\:130000000000}] +druid.server.maxSize=130000000000 diff --git a/examples/conf/druid/middleManager/jvm.config b/examples/conf/druid/middleManager/jvm.config new file mode 100644 index 000000000000..7b9f1a47b55d --- /dev/null +++ b/examples/conf/druid/middleManager/jvm.config @@ -0,0 +1,7 @@ +-server +-Xms64m +-Xmx64m +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf/druid/middleManager/runtime.properties b/examples/conf/druid/middleManager/runtime.properties new file mode 100644 index 000000000000..84945838d6ef --- /dev/null +++ b/examples/conf/druid/middleManager/runtime.properties @@ -0,0 +1,20 @@ +druid.service=druid/middleManager +druid.port=8091 + +# Number of tasks per middleManager +druid.worker.capacity=3 + +# Task launch parameters +druid.indexer.runner.javaOpts=-server -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager +druid.indexer.task.baseTaskDir=var/druid/task + +# HTTP server threads +druid.server.http.numThreads=25 + +# Processing threads and buffers +druid.processing.buffer.sizeBytes=536870912 +druid.processing.numThreads=2 + +# Hadoop indexing +druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp +druid.indexer.task.defaultHadoopCoordinates=["org.apache.hadoop:hadoop-client:2.3.0"] diff --git a/examples/conf/druid/overlord/jvm.config b/examples/conf/druid/overlord/jvm.config new file mode 100644 index 000000000000..2df9a1c2d53e --- /dev/null +++ b/examples/conf/druid/overlord/jvm.config @@ -0,0 +1,7 @@ +-server +-Xms3g +-Xmx3g +-Duser.timezone=UTC +-Dfile.encoding=UTF-8 +-Djava.io.tmpdir=var/tmp +-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager diff --git a/examples/conf/druid/overlord/runtime.properties b/examples/conf/druid/overlord/runtime.properties new file mode 100644 index 000000000000..0b1700743c12 --- /dev/null +++ b/examples/conf/druid/overlord/runtime.properties @@ -0,0 +1,7 @@ +druid.service=druid/overlord +druid.port=8090 + +druid.indexer.queue.startDelay=PT30S + +druid.indexer.runner.type=remote +druid.indexer.storage.type=metadata diff --git a/examples/conf/tranquility/kafka.json b/examples/conf/tranquility/kafka.json new file mode 100644 index 000000000000..b0adb3248931 --- /dev/null +++ b/examples/conf/tranquility/kafka.json @@ -0,0 +1,77 @@ +{ + "dataSources" : { + "metrics-kafka" : { + "spec" : { + "dataSchema" : { + "dataSource" : "metrics-kafka", + "parser" : { + "type" : "string", + "parseSpec" : { + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions" : [], + "dimensionExclusions" : [ + "timestamp", + "value" + ] + }, + "format" : "json" + } + }, + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "hour", + "queryGranularity" : "none" + }, + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "name" : "value_sum", + "type" : "doubleSum", + "fieldName" : "value" + }, + { + "fieldName" : "value", + "name" : "value_min", + "type" : "doubleMin" + }, + { + "type" : "doubleMax", + "name" : "value_max", + "fieldName" : "value" + } + ] + }, + "ioConfig" : { + "type" : "realtime" + }, + "tuningConfig" : { + "type" : "realtime", + "maxRowsInMemory" : "100000", + "intermediatePersistPeriod" : "PT10M", + "windowPeriod" : "PT10M" + } + }, + "properties" : { + "task.partitions" : "1", + "task.replicants" : "1", + "topicPattern" : "metrics" + } + } + }, + "properties" : { + "zookeeper.connect" : "master.example.com", + "druid.discovery.curator.path" : "/druid/discovery", + "druid.selectors.indexing.serviceName" : "druid/overlord", + "commit.periodMillis" : "15000", + "consumer.numThreads" : "2", + "kafka.zookeeper.connect" : "kafka-zk.example.com", + "kafka.group.id" : "tranquility-kafka" + } +} diff --git a/examples/conf/tranquility/server.json b/examples/conf/tranquility/server.json new file mode 100644 index 000000000000..dabeed18228b --- /dev/null +++ b/examples/conf/tranquility/server.json @@ -0,0 +1,74 @@ +{ + "dataSources" : { + "metrics" : { + "spec" : { + "dataSchema" : { + "dataSource" : "metrics", + "parser" : { + "type" : "string", + "parseSpec" : { + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions" : [], + "dimensionExclusions" : [ + "timestamp", + "value" + ] + }, + "format" : "json" + } + }, + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "hour", + "queryGranularity" : "none" + }, + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "name" : "value_sum", + "type" : "doubleSum", + "fieldName" : "value" + }, + { + "fieldName" : "value", + "name" : "value_min", + "type" : "doubleMin" + }, + { + "type" : "doubleMax", + "name" : "value_max", + "fieldName" : "value" + } + ] + }, + "ioConfig" : { + "type" : "realtime" + }, + "tuningConfig" : { + "type" : "realtime", + "maxRowsInMemory" : "100000", + "intermediatePersistPeriod" : "PT10M", + "windowPeriod" : "PT10M" + } + }, + "properties" : { + "task.partitions" : "1", + "task.replicants" : "1" + } + } + }, + "properties" : { + "zookeeper.connect" : "master.example.com", + "druid.discovery.curator.path" : "/druid/discovery", + "druid.selectors.indexing.serviceName" : "druid/overlord", + "http.port" : "8200", + "http.threads" : "8" + } +} diff --git a/examples/config/_common/common.runtime.properties b/examples/config/_common/common.runtime.properties deleted file mode 100644 index 1c69373b8c54..000000000000 --- a/examples/config/_common/common.runtime.properties +++ /dev/null @@ -1,52 +0,0 @@ -# -# Druid - a distributed column store. -# Copyright 2012 - 2015 Metamarkets Group Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Extensions specified in the load list will be loaded by Druid (no deep storage model is listed - using local fs -# for deep storage - not recommended for production) -# Also, use mysql for production, add "mysql-metadata-storage" - -# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system. -# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory. -# More info: http://druid.io/docs/latest/operations/including-extensions.html -druid.extensions.loadList=["druid-examples","druid-kafka-eight"] - -# Zookeeper -druid.zk.service.host=localhost - -# Metadata Storage (use something like mysql in production by uncommenting properties below) -# by default druid will use derby -# druid.metadata.storage.type=mysql -# druid.metadata.storage.connector.connectURI=jdbc:mysql://localhost:3306/druid -# druid.metadata.storage.connector.user=druid -# druid.metadata.storage.connector.password=diurd - -# Deep storage (local filesystem for examples - don't use this in production) -druid.storage.type=local -druid.storage.storageDirectory=/tmp/druid/localStorage - -# Query Cache (we use a simple 10mb heap-based local cache on the broker) -druid.cache.type=local -druid.cache.sizeInBytes=10000000 - -# Indexing service discovery. Update this if you change your overlord's "druid.service". -# druid.selectors.indexing.serviceName=druid/overlord - -# Monitoring (disabled for examples, if you enable SysMonitor, make sure to include sigar jar in your cp) -# druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"] - -# Metrics logging (disabled for examples - change this to logging or http in production) -druid.emitter=noop diff --git a/examples/config/_common/log4j2.xml b/examples/config/_common/log4j2.xml deleted file mode 100644 index 78e9768e4d0b..000000000000 --- a/examples/config/_common/log4j2.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - - - - - - - - - - - - - diff --git a/examples/config/broker/runtime.properties b/examples/config/broker/runtime.properties deleted file mode 100644 index 536533f3a70a..000000000000 --- a/examples/config/broker/runtime.properties +++ /dev/null @@ -1,29 +0,0 @@ -# -# Druid - a distributed column store. -# Copyright 2012 - 2015 Metamarkets Group Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Default host, port, service name. -#druid.host=localhost -#druid.port=8082 -#druid.service=druid/broker - -# We enable using the local query cache here -druid.broker.cache.useCache=true -druid.broker.cache.populateCache=true - -# For prod: set numThreads = # cores - 1, and sizeBytes to 512mb -druid.processing.buffer.sizeBytes=100000000 -druid.processing.numThreads=1 diff --git a/examples/config/coordinator/runtime.properties b/examples/config/coordinator/runtime.properties deleted file mode 100644 index 12aa47fad136..000000000000 --- a/examples/config/coordinator/runtime.properties +++ /dev/null @@ -1,26 +0,0 @@ -# -# Druid - a distributed column store. -# Copyright 2012 - 2015 Metamarkets Group Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Default host, port, service name. -#druid.host=localhost -#druid.port=8081 -#druid.service=druid/coordinator - -# The coordinator begins assignment operations after the start delay. -# We override the default here to start things up faster for examples. -# In production you should use PT5M or PT10M -druid.coordinator.startDelay=PT70s diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties deleted file mode 100644 index 00248476a31d..000000000000 --- a/examples/config/historical/runtime.properties +++ /dev/null @@ -1,36 +0,0 @@ -# -# Druid - a distributed column store. -# Copyright 2012 - 2015 Metamarkets Group Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Default host, port, service name. -#druid.host=localhost -#druid.port=8083 -#druid.service=druid/historical - - -# Our intermediate buffer is also very small so longer topNs will be slow. -# In prod: set sizeBytes = 512mb -druid.processing.buffer.sizeBytes=100000000 -# We can only 1 scan segment in parallel with these configs. -# In prod: set numThreads = # cores - 1 -druid.processing.numThreads=1 - -# maxSize should reflect the performance you want. -# Druid memory maps segments. -# memory_for_segments = total_memory - heap_size - (processing.buffer.sizeBytes * (processing.numThreads+1)) - JVM overhead (~1G) -# The greater the memory/disk ratio, the better performance you should see -druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 10000000000}] -druid.server.maxSize=10000000000 diff --git a/examples/config/middleManager/runtime.properties b/examples/config/middleManager/runtime.properties deleted file mode 100644 index b4668b3e53fe..000000000000 --- a/examples/config/middleManager/runtime.properties +++ /dev/null @@ -1,28 +0,0 @@ -# -# Licensed to Metamarkets Group Inc. (Metamarkets) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. Metamarkets licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -# Note that, if you are starting a Middle Manager then you must -# uncomment following property in overlord/runtime.properties -# druid.indexer.runner.type=remote - -# Default host, port, service name. -#druid.host=localhost -#druid.port=8091 -#druid.service=druid/middlemanager -druid.indexer.runner.javaOpts=-server -Xmx256m diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties deleted file mode 100644 index 4ca5b43c42b0..000000000000 --- a/examples/config/overlord/runtime.properties +++ /dev/null @@ -1,35 +0,0 @@ -# -# Druid - a distributed column store. -# Copyright 2012 - 2015 Metamarkets Group Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Default host, port, service name. -#druid.host=localhost -#druid.port=8090 -#druid.service=druid/overlord - -# Run the overlord in local mode with a single peon to execute tasks -# This is not recommended for production. -druid.indexer.queue.startDelay=PT0M -# This setting is too small for real production workloads -druid.indexer.runner.javaOpts=-server -Xmx256m -# These settings are also too small for real production workloads -# Please see our recommended production settings in the docs (http://druid.io/docs/latest/Production-Cluster-Configuration.html) -druid.indexer.fork.property.druid.processing.numThreads=1 -druid.indexer.fork.property.druid.computation.buffer.size=100000000 - -# Uncomment following property if you are running Middle Manager -# druid.indexer.runner.type=remote - diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties deleted file mode 100644 index ae213d8b19b9..000000000000 --- a/examples/config/realtime/runtime.properties +++ /dev/null @@ -1,36 +0,0 @@ -# -# Druid - a distributed column store. -# Copyright 2012 - 2015 Metamarkets Group Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Default host, port, service name. -#druid.host=localhost -#druid.port=8084 -#druid.service=druid/realtime - -# We can only 1 scan segment in parallel with these configs. -# Our intermediate buffer is also very small so longer topNs will be slow. -# In production sizeBytes should be 512mb, and numThreads should be # cores - 1 -druid.processing.buffer.sizeBytes=100000000 -druid.processing.numThreads=1 - -# Override emitter to print logs about events ingested, rejected, etc -druid.emitter=logging - -# Enable Real monitoring -druid.monitoring.monitors=["io.druid.segment.realtime.RealtimeMetricsMonitor"] -# Enable all monitors -# druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor","io.druid.segment.realtime.RealtimeMetricsMonitor"] - diff --git a/examples/config/router/runtime.properties b/examples/config/router/runtime.properties deleted file mode 100644 index 5d93057f16ec..000000000000 --- a/examples/config/router/runtime.properties +++ /dev/null @@ -1,25 +0,0 @@ -# -# Licensed to Metamarkets Group Inc. (Metamarkets) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. Metamarkets licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -# Default service name. -#druid.service=druid/router - -# Default broker, coordinator locators. -#druid.router.defaultBrokerServiceName=druid:broker -#druid.router.coordinatorServiceName=druid:coordinator diff --git a/examples/pom.xml b/examples/pom.xml index 29d15f116805..16c69f55ff3a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -113,5 +113,4 @@ - diff --git a/examples/quickstart/wikiticker-2015-09-12-sampled.json.gz b/examples/quickstart/wikiticker-2015-09-12-sampled.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..6ca3233b777cdf165c312267b08b4bac7c786df8 GIT binary patch literal 2366222 zcmV)uK$gEBiwFqWasXBU19xd_X>@60Yh`jRGB7bUEigGPF)}T4VQp}1Wn?aDb8l_{ z?7iD}8&#S&eETS(%Q-GFl=NPf5w7{PVtjefbhK45q`I z{+=1q0-xg*R^Sw0;db>1#m^g#uEV(~mziP1J87Z73Z!C+lPquJJB4G|X7zw2hD5>l ze)p>X7Tn4j!l;&oWA$Zbsctd6HqH%U`p%EA1>f1E$O+qbe$1#)+Z4La3hP(P46gXU z+kh{%Dm0ISpQid8{KyI|HqJ6ftp%zV$TZiBr7|_(Rg)FDFj_|7UGxvYu^9Z)zOj+R z!YEp{$rz5mz;v7TWz@tzZ)&_lxXNrii9PlIJh=%H$xJLJY3W1@5_uXB33qY`H{fU4 zlsx(55WFG`D^KmGo+3{;j4(sj>GJ!O&~lEPIFa{W&%hg4QJB@tB-M4EcXT>4(={M% z)7c!y#*3k09H-o9!nppsLt{6=Og1Qdu*u1F*zhnQY;j3TYC#uJU)ISvrqSnzZr~ru zlud2NB4U>Gmmp#_k0wVk#czCJq{>%+nHsHC(YjPu)6B|e04r$+!spEYcG3|q2 zM5csem&kFdGplbM;6zMPl(>{gNV29hVUDe%919FGO37FY$n!kl+60`);s5*9>zoWt z$r)l4m*1`uyR`g{QS7OumaV``AV6}abNaL}oeuc?$dVf>F=kqS*Pq+4>{88TmkNrA zEK8|`5*3lW&Zt403UMr4IVVm`cw1N|dwg&a{&i6XZoP22g$%6WJW7yRotDeI$V|ry z4bF6l+oxbRn=1(pb8xo+sZu(X+y$r(P?44?DGSSj?U(1nZ&qCfCn?2Z>8QE}Y*+`% zuu*a1?jE89zl)m77cJP7cDxd}5tkuau27>&3Cx>C2C1H>hDG+sX<6%s|D^VY2I?Sn z%CI3bnAj#r{YvLeN7)*sv=mR@jc$%hr+$G51GZF~=C}@@Ak+Z09iB!G+5#Cmdf*u7TqUOCRtmZyU4NfL>!9duB1_@P$ahyRiU1GxImS1_-;!W)xkAp2KMV8{pR8+8r zM5T&oQOjbMou));O!Tik}J-&GK_{x zVB)sAp8`YkK$y2hI?qUv;SY;|hO?g<{TuvAk=%f4Qc9(EH>VAolLbPZnxOD)yWylM zQY^je2&y}R^c;Gd**3a?G}zS=t=fX=1UF=BuhwqVuGhY}`E2c4?efi+NbN27a|SMc z4wu~vZ^NrgwLjlHSNpW~RqZOiulEey+Hc)qU9No%|E}*NwU=QmU(~L_$l$#8`OPzU z=^700YZ&`ywW~Lu3k@rN2j1WPp3h0`{n{6Ir7dV63Zr{nA2?Ci&=v|#PH=0x^@Vs$ zO3AG*q`-PQth`aKmwV7-bcpe)0T0$;VORw)GeW~~xF{#A3Zte^mF}12j>|)qw}e2I zlz1xyCqsDg);_s$`}}*~pZV>bk6*ui{^dJYZ>%m|T)Xl5+J#T=oc*$!*xVztXCpML zXa8gM+#grZ{c82xN2}+~t)Bg4_3W3oU%Pz!`oFI(eZ2PN=WEyATRZ;;Z=~N|J@egr z?}tWjP) zZ^#yz6{c?m}>A^)1b$)^TYjwJottNvhJibp^&2zJwTz6?A4<`!LK)I2iKM zK9U#4Jg8ix;kvMTOn4rfnj&MSSkReC_DoHU^~aOR@Gkaglg(91Tz4OfhOMPGZ7ns} z3QT2UiWF~4xnS02vHNj$isuCxf-4&W_(K@e5ri2;r|{?%Xg<%EuNW9p0HjXYf>Z%H z(`4RdcpqpaHa*2V!PhtbFKt13?~8XtgnpdbC2o;HxIwi;cz*c6Xlrnt+qt6<7!JJx5P0iDXG?A@SNBT$0;Tpy{~02{NdfN|9a&N`QfdrfBop| z&RbvKp~*IACRxSF={%3w7l(w&c19k>4{yHxx3@m};f-IDv481+k}S7^)R>Y6JaS9| z?K_^7WUUEKBGFuA?s2eH2KS0-t&x#U)s((zF;^9>=&ap%@!PMy+60z^6K${@q!l-t zYx|-2{;gNOf9_vzUwW?-4r(h7)Bp9$x$1xY(vJiz6ii@&4aEmVXln{!P^6^Vx=3~% z>s@iIrzBaG+X23Bo^?Ps4zjc@Y_jQAHbo7a)8fW;HZCw1%qYNCrjrUxzkn@_%yU2; z51w#1Y+dYB#bg#-;P6Fod6pa8-Pby;%%Q_=D{}}p?j5l*$An&CIIs~IV8PwL-5-aS zO%^~{g}M0n1@q)=->PAZZDyF^?{ z#25W1Gc#xi`h=GMjd%r2Kuj~xDHx6>gXIqQ!*3b_Qx~nwBV!WY0qn@>_PYneAuPb``1m^U=&2379hi4+m}? zt@cNO1$;Q@{{4CI)lP0i88+{A7%=34P-4I z*-8iDhhB#=g{P?k%j?96@uO3j;q3GXnLaQwweyvm76-F5T=+?7Bz|5!wa|p2=_?87 z21{jB*cqcsNUs9r#EGFs9Ya>nIb{oLQVLEUUIIauH40TS0h((SWMM-S1#z~2XmP)= z;gk0*6wtMGV{&Sc<6O5uRPUiUS-bS| z>e=h7=YG9VxH{F5#@%yE#w}0`%?Ta6*U3mZ7uYVgMehP{m?)oKOQDU)#&kdIL z+>rPZ(t736vD+U>7j?Y*_My7a>8(!Z`Q zy}r8i?&{L@@G1hW8p3ksm>+ zoo5D^)?jJEk9YLtOTDEh9C8dwbY>au4-dKLfS|Bgq0HyVNyp&A=fMY~%V!cRfx!Vg zMr##a)_g#4Ko4lVFOzj_0lFNw=7>ehWopb)+eSy%;$NP9d1mHlxxh^4nK3xWL~#Y>RT{v4#BdW0uLY79syI-6uzCP>Bv_D7E=P4 zppdJ9{To|>Io*hmxpBb)xAKmKhV8&%Q5?YUB~z?ovvLmfcIEFI@6L=dp^IWw@hOTu z=dx_^a(N>Pjb~P;bEkaC!?1epcdKXrw0iDe-OPp$*Dn9tx7T0xg=Cr=S|ad%7T2U$ zTk1|=w?oOcE17|Tg@pylI=R4d5;tbWfXOW3n0aOmSZGgJI(m(DUCPgv|My{P6sJYy zr}95cI5{0oek%8vE`ly3Mslpc9GkfX*$0{Ba5xSPDCp~ICN5{RWKzovDalwyk&~HG zWpp$%Ix;kxNhA`aTUUS#PvJdnRU z`-*p=ab+I?v8fC9g)AQJ10D5L{hd>Iw9jQspx%`u?%_UInne)ACHLA${dkYE_d(HZ z1|8FeIXFyDd)bs>?Q}+d=I~R(=6Nz9K>HPU_teCtxYCvpd+)_=PYGFg;%sU$GU*gy zt5^u{`R^cS`<2dtxh=s;N=Y^H5=TMHOiSPgd(X3S48F@QiGnyq4mxuzCr*V%VH_XQ zDdFhTrel{z-H>;~@N1%>2DIV3H-H%$*E4p1?))%@>7v1US7Up^w@5+#YjR!-A*Ug%m|PJrUU@QDJkBb$F)l% z(Sb(dFkOI0c2<~q<_^Cbj1r+uwlordsB+{a0?#BEneu$hLnfn`?t>HH1G<)pu^wiPG>5LoW-si@<;l=i#c&^%ri<>huofzVF zl7qVObV+;+z>NY7j!`zzl-0gnvO5<8)ish(fV+O-izU1hq+Y#&A zKhn1mvA+Ev&@d`d2Q95iih`DYOp;T+0J!WcAaIWZ0vDHJZ9D63a`fFNN5?&g6eaYs z>|S;>z?*`nB{ID5Z2+C+gDpJu)Q-0Shw-QS)+3%~rfi8E z6c($L$fOGds6rrSO9D$;V08+ihgXD5Kw-awjHSMHsW#=QdmIcnFm1F%t6jKhWlsmY z@Q0`0{Oe0^{P5dN5+Hh~q~~w1{o#Kup9u{{98(-68Pv)|%4eg;ebL!?U@564B&99Z ze*f%`X|r0KXJkL6^Gq*`9!ZXhL$Uq2yZVA(sC8%BAsp`nr_O-bYKyw#iVo_gZt@H2-n%y zudcoN>FqbqZ6YG4kGF})Y04LNi2Llji=Te`)za;^E{BiAVl|B`(m+Y^OUHE6gP9KO z!GxQ{+L&ml84!k*4{tp~R{pT^#jU4T-e0*yR^Gq$^sQ%CF5NocV@qP?%B|;N1fQ;a z5E_qL&%k(}S^2~r{K^+A7vVk^Z#{okik}Aio)S5s=%I3_OjXTJRa5uJi4vEkcpK|k z#SVy5);3wyA>@K%p5$1;Ku=C)NzI>&?^I{({mfsZ{Uwpi;;+W#`K51K* zTIiHLZ_we*Dw7t4F%AYe7;2O54H`;e6E`(DaGIpXBKs?0_c<=iIm}ZaxY&A3*kuia zk_pTju_9yw4I)jnPo49nzXslQ;z=nXM=qW*Sy_Ne+D{r^)??FOb;JKt3}6Vq4mivg z-N4Zu?;J*sZwf#>hFx3I(Vzbw2Vjf|v&hJh&J6@&&pgjFQfZXphObEK7ABgMYO9id zcw?AF40YQj$0B*^z>c41Mi&E_;Ym&rp`;XxZiT|G;W?|iN*{M$GM12d{xR6`ywZ6x z%qrBh%k{eT5o4a4!oX8>!wK}1%)9Adq2W4cCDPVO-{ulBoe42YNoln;o#`PY1kPbi zdlJ$aww`HYYO%#t!ORWjVYGmOV{RgIyy2p-jypJgI<8 z6PS1tnRsAml$aE2uSN*5V#Z85k3DNfsjgdOkHsify(@crA@Op{2x%6!erlA$Z!5kJ zHjqg)w-xs|AU3L`wetxDO<8vJFjQbfFF?_1x?^o%*>Pi07<@X+^CjKYEUnhO;@Qmt%6)^3I(g&;>TA&prdSysMMh)=4bD*tq64*rS(8>O%;lx# z#pMq=*`a--?OuUV6SRwa!QUA={xwB8+5u@C%oNL;3=p%ueBst?(saCUBVK)vQk@hY zIm$Prlw$d)EXgrhO8XRt{!LmlY>8b$iYHpJ3q^adba8d*?CRMU4o_#-KDlw{(rb5K zz4-lGzh8UiqwimP0qLa9zKe_iXaBr<4%w>7>X|e0>bd{C{pR^Q7hhPr@r+BF^WN&x z)2mB2R+rvbUHW)+>HXWUe{lQEA6A$C`0e#yuP(jqGVFZ(-Jf5*bLrL9r5A61^qI@H z10Q)GKC~Y^@)}M(c|Hqv_UHtxc&$dj|!0ne_zWvVm@OvgaYE_?tuc^(w(mf8i zqSKO+j9e*jfGn5-)~*c}zl~LQ+Z(SAzXwjyvgNog;E)xcp!Qgr$*67!oim@KNNXCs zT40+u4qAO1o72PE!YfBDUZ!~`f^WM^O}r2hRZ6#|l|H?xECyh zA~QJEd(0LdDUc~tIcWIQu2E(hoQ#zZ}|S>|kH z(BPIwUG|Ryq#%q(Y!JANFfYxV6^jJ>R9O4!ujOe4I~sC$OBv~OK|Z^4)yWAtHrVEb zthb*z1QNX<%J;w-86kM4Q>!QzIyfH>?LXY=eZ(JbVaBXef&bL!wGMAemE>4Vl9QU0 z@^u*P*E+YM)HZKJi3lsOcScSRKO$@bYBlPhf>VXniJ5Z5aXV+O3~n(+`dXyIOD z>W2=`5Ayy(=v^x~sOTGisChK;nMOOwUg)&kd>njiG(p!iCRe0)!;y7;9joT_2mzGtsq5J4r7d_8xGt*B zyyd;CcbG9ISfxVA1P`BTbe!wD;cI6aZ#v!L0m}wtY`@Z8ejeD3kTvrqK94GT9(nR- zLHpYvsHQ}IuHsZG45`%S85r9|>UDrVm>oGuSvnn)x3~G&UFYrv8RCF9PvNE9U@oWl zc+yqX*FwoV35YT#UPSKHvXS=Jxx!+1TlJafdKJ`n1VmC%k3s`(V^SzJeWF z+G(>ZdO$S#duzKN`tLV#CawOTPj#~Yzu&O_`wc_Pho1I>)Z%O|7T0_`Of&W?2aNqH zNE4v68NGp-n?Q?Oe!tGhgnh)!GKKRmu*QXziOFWIUdwU?^%0C%gOYK_;2o%qf6PxL za@B&9pS!EoAj?uL*;cp~R3o2c^USn)L0E2|nMO;m0xHZ?6_)2z&kPBedY*GUv3yum zR4u;f{Q(Vo5B9NXc#`WcVhcuTW@azGh@_8Wi~Gmmbf>SpdOx0e2TATV23VJSt*_Mw z8-6j)<$PL4f9K8}oA>QN$Tcb57R!BT6>rmlDTqdN(=FW_AW2GU+ny9y_nyL*gJl8w zfJxo38kslk~k$dpq-`YgZqBH7t{k=zO;OoC`D z$}pYa0gc)_HU~CzI&qcJ-LX znv3~r9b*BD9Thj_A>C^61`beV4Ff~bn8<>Wg|cv4-gwprk>yl8A;(kk#$LGmL5IyS z2?|k6mgKZ1#gmI0&RVy|jc+Y(H(Q zR(OYdEI<8(QL8sgrUc5&?)DfGQKeG=p}4;w9^+cm=nwIfIYE??j^%s!+IFa z26Lms>#z#b64G3!B{e+z))dv*Mol;Ei>`Vn!5~#tDb?QnG@{qj76E;prz2HtByVIC?iUqj>aQoaQG@&jgi=CQzF zTZ29d$YDuIweGfo%P{~Uw<6=}%BOhn{K|i>ynpM3-pkfnxwvxW*122fVFVxIK$cdn z!9bpM8w!LpQRv(pPy2Wy<8fc5Lts=iDH(5dlm_O6*cdQaJ@?zyb6>BXy}G*e@tx}* z-+t+L8SE73YB?4)3ORA=si(+4|6PASSzY=Sl03Zg;_BJ+t7o78_Ua3(XJ4taBOu$u zd#g(?xm*yxUR`>c^!NX-dsWKPJ)GChe{kp3i{G7jrU~}?rOfovsr?5=kBuK7N3vtt zqXYHn2(JKdyw6>UOf3yIUSyH^KuE_THWjnVlvp+xeJn!RbXTZ!73?&jDv?Vfyk&TN zaA10Ix>^wgb}vv`W@h#g-zzq}g8l#CMFvKF2F(U+f8;g-L{=;Jyx%+DqwL*BG?i2- zyaiv>)PDCEUg+J%Tjut0 z!K>KaLQ~p^VuNmA@CMuKm~0Byj{@J1f~k%@u(nT%$%+(fV@M1v#m2=v z2(fW&!VLefR5&>(N?1v4y0yxfN4KBo@Zd=#i@(gRKF3!&e`FM}D$q@;0d}xnjg;Tm zo?{q+AUWM}wK$9CIgQfdhto2JmFin*a>6I`5`kBm9G4PuWJ0$GO^f3Op9M8;J9RiY z*&!%?7x6bhS4$9 z%1v01!vYMd4i#@df;MIXC~*m!(WZS#nQF6aS`An>t!5N?Hy+;6#?@)1A`G72=2}re zNR!g)Ze!XYG;tH6;^@>QLym4cW_UfmD_n6Z<}k?dR>xt;sLDCFz4M2}Yxn%&%Aefu z#HUCoMQ$zKdY)i>;@0_<&)~n<|M?TR-gAMMThFZgY31U|$IZdGL+M7~rC;fYo3a&d zYD`Khk@r3?85%T5m7wWtZMEkmRY`@11pc$wJZ#JPYQCwFg<@kBBWxV&J1l%hQufL- zlQ9YQTB)8jJ$-y}`gjjyO}B1~hU;S(?#RymBhSFdDcfXawx3bkU#c2m0jn_c(K8et zKF0~zN_p%uVlWY5yX=bhLV+Ns}mvHLO4rMZ0R35SJmdesRyxwE<= zZ>g_<0hn$psC!u5lr-n)+v%p%(o$00B_D1#q`8qd!TO|8JRRM&q~nSd*IIoYfy=Ch zkO;;G4LpytQlxlU5Q%6~I8LS<7+83M*+F3zU58o&mGC{sR3x5I zQ@c^h&6Vx_Os}|l_V#Gj*wrH)d6#+M1Y3hu9Xmmr<@)%yk5XO3tc!G88;QUb5?O*N z3iBq&F=}w5LjCC)k?mvm7w=p<%*BvaRa+p^ZHXV^(*zvkYFt zw2QR1+r_tCg$5;S9%f*BBT*WXg(RXUr$2ZL<|OiwZ(p!pLn;roEeAGwPvu5|{g6)0 zIkw(cdE;fWr;#_(d%cyXbjB(=vn+Ci7fZH9_KcadOeUylVbwm3)<1sA&KefeeRPOQ zQ;$;u>v2c|5*^HqRX;;mmkeeRj+-oSyVR)p$%#{hEF#|^7)Jeh@>BBn>2%t6$%0EV zp7VnH@&ZT@=AGh8-q9?+lGAB^4$qBQ;oV77bFsMExa8g`);OEpcr!Q|pzC}#><$h$2~LO)Kvd-|WKxbKk)SV^Y2 z*&WH<3WoL_bymzXvHhJSG`HDDTfWu9i%`S5)|rzRd1_X++qzbeN}YY*H91<$3UC6-NfP&27%!1%D7l^U1?>{YG0 z+UJu`o;YE%GF-7)=IHUoqsM1vo_eZxi&nR6h$JVx9A!D4k`yJp(4_OcaF~ICGNOCz z5YhdQ=)u%lcdx*{;bCUYGHz3Z)G1RU98#v-!t}N}^=c=kPip}ErJyzKdpMSqFI-3Er5DbG?MMBv})N1grkI$!%Q3 z2tI@B|2P1GZUIo;tYyZ|(s#C8$Ok}gm=sp{)SL+hcQ4a9W+r@2>SVI%rFM^ltzk`- z;>pPIQ7mVI?bQ-x<~7cA1+q&}#)!>M*}b)<21V7vI;L#SA!8+z@?2G4ba113D z^K}YpDn5br9!$S!TUP}Z*kN-3BZ!8p2*Cbnk}1ZpBYnPPTO~0IauOw=&GJ|fsc8LE zwqq2S(LaiP^LBh*@0JSD7_)GgFvkQ;3`}HZ#z5AXHQ|lnvk=Dkti7?vklbXNbB_b! z31&uHQ^#F11NQqxOyfyrli1UKYSQx^r3mp87g zCTxLG14USioEeNHaNNv09PA_x;;+IC?B--68xF|tznu9Me&OEUD}6aMM209}RR~|_ zFkNDUm~B73zzs{*ToU<)g1AZR=PdyO8=;Nm#t7OFgBy7rBp#b3bk+t-UPs}V%*50C zNWlT4mr|1?Ntqjt%?#YpK=o(lwrU-r6H2BWbKawzoZ{gd-2z<0u(1iFb&Jj4^CUr>Ki9 z=dIOat#9AFRC-70jjb(Gu$LmCwrj2c2njOfz$BXFs9r5scs`0wvSD8CZaF5UxE9$o zTQ3-t-Gad_YvfZ}-G73mrOINv&YlVXyqOpykgnlMAsT}NA zje5&=7XrkQQYPI-G9TEI7z71$o*51!e|LW}79vAY z=l;f(4uS+MIBGM;w`w}So*4(3YcNoDEKAIZ&{!Q7^9+-0d>|$L4a1T_TZYAv9!8CI zuQtGhE7;7JP>=lE%a5tM|8_Ev>#ma)U**Q@t!24&igRmVNR%rMx@iYoAMO;Z*sPp` z-6C<}`o->QH+Z^i7vuA6le+ev~{m ztpnNVEIE8&m{eeYCMDahWCjKn78WE9%DW`Y;y}g8>5aZQ18$^rAP=MMw-_!`iCUFY z9pLOcHr3YxCn(kKa-Xv6ZA|Vx%TFq%ZUyX7`y^kWCO{%cmYWq}b|@D0T8R5W1+7=- zv-k|)9M{23BH53mDqib`IYj>1-VT$p-+Oq;a82r?2z@RyB%HBw{-cl`0X2T;h z>XJ>9f^d-PXMNjqn`IotMoc^2?YR-71C~z9JWt%;PWCiQF7c5IpRt>r=lPBG*PZL; zw1yZ0EDknC1yl+%r{yJZ>F~IV8AIUMpu}|Ckux!^LC2A8e=T19ihRHbwM%PJ! zGxF&2f9i7#WHn|L6xb{W9obDhEQ5J(82iXP_OGNR_%MDDzs~;=zk&xJ8DyuOBB9Q# zdvB*)q(;%QDyD>C`~_wd98@}(_mg|A)17aAtvQoh*hH6%IHssjH}xsP!(`b^hbi~> z=#HSWYjH_Swl3y55nN^e+pmA~!<$c&AD;bX@5M>~=gV_{z4FBm&wuo{r?39cm9ImC z@YnY~`s=SgYRUb)}F#~LG3nn3N<1)g9h8hOg<0noGnzW#^Dw(p` zJTu4yGtKIQra~m*lB^`KEiydHp}`!V80Ge9Ii2wJhDfEFd42A2u(hP9Qe0{E>INqB z`}WTt-@f$j>Y1}U9*yaGp>`9D#+A&sSHE1l@#5O=uHX6Nm)}44uiu}0XLaf0wXa`V zd-j<-7jAre^~`tgy&qoCQC@_H_aHa@gg4C>Hd3RI~wCDTlIiVkK)pv|ulag3YC(H6Ca}ql5|3=P z9TvqRO3^xL0ak@sUgysp?Q+_>T)7NaCNXwLbtK@|MKLr)E%)%D)HJ-bz-E_{mSSyr zet{+CrpfTIu_}rtDX}zmy;*EIZ%c-6dHc6t|Mt5NKE-tS)ulhKo;|;M_62fa7=CU_ z*05C_NVGe@fUB>ZII%x7`RL->ORue-eg5`;o?l&heP(9$%$YlHT)*@A8*4AVcKekJ zYrp&bj+f^fm7}#^zJB|Qr91zACG@j#XpVBzwd5PItW>h7~2@vn!7L1SV<3Vv$*j9|*-}?RRglefbV3Hn*?*`p%7eso3<(oihsLEiwuc zNaUinwfYDX1ODOdH-Gs3zy0w3)4dl0{^9Lk|IgJkKfLuD^4HIP{lk}UgvQ}-zy8f% z&;I7G|Mr_7o_WD1y_z@#B~%0)(o(9;26%X5UJbgw3N^V^;$37SizLX9WvH#LCronl zq=Q5pV%9A;9HB0s&5mC!9V+3nwoT3cERx$=Rg=>JiyDQTI29V7)*IcWk5N;lq!w8r z1c1JK?!)h{{SjsJbJuQv{c113)53{=?t|5H=T^^NT0Q&j?N8sn^XldBm{{~Q_Zd*K z+RUVIj{{;6-=#b0B+9_PpJnFu@{T4jBMVWOfl44=Tx!%Z^l_6Nu5=`r~>PR)}Txh0R)tFS_bv&v@7-^I6c z^79tC&k5`&)gD@W0Iivg^M3o4OLyMA2nNhObI8V;yNr7raK$9&h%M_7O{hRk@$*vn(@qWccqV7`D=2M$eJ8?stlUj5_&}MfQxtJcEYK%<$Da z2aH*mdas&!fwKZkzhjqazT~(|qQL~(6s5_4k~E3E#N^!uqb=#S*doBOh-LGNqoWg| zd-Bxh+Yaj4PGX{2qOI~LFyS(&?GvaHa$9Vsx9{lAd(#3wC&-=&*w~Eacfv0pISmRn z)*i|+_#5*nf=a4$xd2?>3J+P9VzJ0()MniDbwXsf6n{srnJrmZH_Koer(lDi49i;s zKI-T2YGg|SlNxKLUAV`=mXxeY={5q%hgC&kv$KvRcp-{u)ioVO2Ay8p-HFyCh;vpIi zvF5#-sD-XKOd=IaOm1CDrW0J9tNmN;YVAht%FQ#i%Qv5{UA_5I?}aRCzp7oXU9Vli z|L!BTH(?ad!12}ESIv_zYhS@IKZCR9LSuD=&k3DY$su~0&iNNv+PBIA<{Tw4=P1$E zayc-!A3%1T!;itFd@_Z7J7#+iu)|n*?$+Tipat(8xue1!mqnxmXD~D2Rfut>mwX|> zW+`>RO6nwL{=}jSL6;`A!c4EWYW(QbaQ6PG3*ABoZpTcUOyz~$fe!4)GzFCZ1qKSb z-$|b8>?KcaiG(I46Ro0ccRT8|9l0bBsI+3|8)1fLZ11!iMOi#aDXvCNdByYw_!??4 zw?7vQt}I+GkRFh4V@I1{jUCM#9VXfF&?p=dRHw3!EkBYQPE#bct#~(ZKLeMY4YN}n zq+QC4n#{@zOx7JSuv?n72sV*TPq{*y=)xk*yFN2Rsw9u3X98PTz+|)TQAykv`&FM6 zh9$@ZE10m)lPMZLlDj@zKKnInLka*;>yU#I*hM!gFc-$_Qa1AFqUmyY^8!VL=#RBV zYa}oy(-=mUbXrF&3_PtjrE0MaA*NV_QTp?-Lb?% z`vp^&rIAQQ?n{78&8W$k*$bvuON6&-GF>MJnK8$W@M^-N)x0jJ)0)o=PBf?AJq`#! zTuLNbP2s@22ACe3H>v%I<@T)wJrs;CoeY^YKi7Y(vfZ!*h`W!moayj6|DsvHlxgWW zHgHI%g`6(tB4~BpYF(Cc+v3C{cOkA*(mvj2mz;9$@jfzZiZZASvoMdR2ru8i+TCug zq)`)Fr)P82G{RR>|Gaw2M~kHT9OVK1Y34>KIf38ASuf0eOLfu|Ia*N^VzQ3#!2*0@Eil5q3_DGD?-K{z12W!@rtgh}S{R#i8V!)CO+mo%6#yO$e&f8ajO zb)CZ=GGX#Ugoe$-6b$jKX)xQ*9NTo*C;=TdDI~p#wM76QUa$3r9Pk_rvP8?gyS&Sh zwk8#QIgw-Q8+n(9Oh(s9mlwEQB9FlT#%U3TNd~bv1E$S@OJq+@mi1#b0X(g7?l#Hg4J;$bpdMw^K#0^FUy)+9xzx+Yy2kJ3u6dmes zB^M#d2cDyZluSpapgd3pIWZ5i2gN~w*_7)j3x@EwalORKk62g?hKUIk%fa$xE>Jw+ z+c|)OG6o~wJ+|AAF<};Abu`BrK2w2F6I8kX1Ih$gd~H;& z-P=^2eA1xv=uBI5%g)J93DT&-V9BEg4!}j5+d5u-=4|ljw9hE>a~uQ)DJ3NmkwH3MXCU1y zS!@)9prwJl)(xR^W-Yc}0A;M7{Jpy4H`y(@OIrsO-V3ig3`8V3GS);h9dA;+xyQlq zjK`#OEV3Kk-Dn>h$xbff)>s8az#yGOkEkaD%%KbqgK#Q!i%NK<-$*Pn+xWGCFbI?~JbF zaXGmaYQxxeVfxW_H2RJV^=(9>?}z}+0+SKo0X2Nx#uF*mGK(q5oa;A2Iwy-Iwn!FB zVV^K9I+YbO`o{))%lbQrafRb0crM7EM~A{!!4K<7%3?7V8aLZhFHGgJ+px*yjzNCqtsbbTG<819H$Y*qAOvIp%ALEc+S6f>Ma3 zGqISYwxtd^b0JC!-?{l(?d!jsd850vg!g4-)PCz0LM~zP;LR6n*I^{r+`7Tbr1lBC zbOT=cs&?7C@Cv+wRfV6!yT7hoy?F-S{aNjLEj(l55uTspR)x_yAA7YDOZb$%;P|Oh zQr;!B+Tenx=y@m9F`0n%6(xzHn|%+{$ifnj-u}7=cbuO&jDaPm^V2lP?9w9^8HJx& zaCwN_wsXVW!W_y1nQ(~)?|Ms$P&$>y6tGw+skDQ@PO9b z*6Mh3ljb;B&W=rnOvdK=+Lp~JY)4C`s}=aGlcQ$7#4M29n{Z6+2rDNQ+tSJzLJN)z zIi}qKsJ@|WtEDr9{IS9`xX;Q-#w9f&$qBG>G+#nYs%hk;0yQO9gDWs^AAmH* zFmohdvU@NcX4t_I>mq#MbQmsIEn&&&l%Muh?p%|!6#$AXB~z_?{;oKiXQn}`k@EDq zD$W!TXKl^GLz$zFInU=ol|lC?NH5SiEo_1}?&vm0?&K*(En)P8TF@cobsUyHWqKi* zh@9-&yoFCQ2wNYWM$Vg*ayQK54VHmkGfuMx#(KOq&BI{q!+!UeaO_e9b8}ATzQ9j^ zXRp~7PP24IjY(ScJibv(SOYo64I+Ca;}{{a{NQoDm=@+KuMZVugck9O_mOvzy9V;KDqkDH)BF z(I(ZZFjlA%_bbt*h?x_H=Uo)DfLUt-Z8z!P@1qhvR-q=?N_=2wYM4Z6pW3*A4>Z-M zf~$d*a4jLFq9<-T^+~a02vs^EScan}!tE!cKBzX}G zAr0NtccV{?oU+mSZdA&E2GPIc!RW9m250dsvRUeV+!uFCH#ZsgI2ao7m=uqCsLf#s+Lm{2xA)HqFB9oJbf#u>F86J#0Ij%`-Z+;?EQ&0~GQrh4_h zc&dL)qK$FKs*+J=nxJ3%Yh^k(t{-i4T!T6^&y74b4ddidYBGy8J=p2Eq^NFVGfDP4 zP(!Csr7=q<9ldvEou8qfG`?)7FR|9f6=iaW8AgQKJ`R&beTC)yO6UH*$`<{7)0nN3 zXmcjepweyS;;r*5AA%ls>zS1+D<7_Wyz<8Z zKE3rkoWPi#U-=|7&Oe{q$UEsvEtP|&mZsfyTW#Ev!M%EJYaz_(-baopDxsB1uW5G^ z)X~X-)2G8zo{UqprgR?!37<}v2%^==89)I|i7h{0GfZ`os| z>Lm$yP+*zt6xZh|+2an~RR*Ipn^R%{iKfIdN=nJdv6z<8Vsa{z1?@YNjVI-d8Xxf! zeKgl%peTn6x2#C9=-rndIPqu$UIy3mf-s$onee2B&!y|Gu#`b>Z{v*&)uQA-u(8L@ zXE6`=<}YejYu7+t*50apL2jO_U4z&D=H`oV z7_G61BCAp=Daon~QdjYL*?o-FY9PvL59yQ?k6r*gjT8>ETxZA#0V@)t+zo&3zGhR6 zp{e1&AB``f19UWwOyt6>z_)?Wp2biDjBM}DckPaP3_OkS7o24#kiVuROhY4sld zYpzu-@LE-3QaTyAcfc^SN(??obc!4fdA@wbAgOpPp=zm^N4zj%wbT&iVLZs?kd8_2 zKP)0oEp9wTl-FAlV5V>$q&3_-)AA|wTu3uB-J@c-PquxDkx4cm9&^W{WlBI1V}8Qy z2r84Jgwba^i0DSSzU>y<>$;~DW$SE8J>k*pknx0SyE$j@V##wDg|!D;J|2nA%)=)k zX=*DzrJ!5r@^4QPp%%XE0i)PwG5lXYey zVY$6-8vSjyYMqiwph~4-|A9sSNn;ah6qIjxV`Jwc?%vtpELz_~;rm*JeNUw=7OG^B zGt;8^@D$IR>~j5b=Z8Vw`e}$u@#u|VABc3gb$-+wvz>ZT^?m0@^}tBMqu&Zfm?kQ> zzxdPIh1XV>K3+Zd-s-upSI=EqJ@+g3;QZ>@S8l)lhud#{^zF54w=cbT`xh^)F1_eo z`R>Etub%zm>baNkolCE;F8%xJ*|TtyO@!^ii8f(-kXER%BZO@Gz(b>noy@`fOM)Cy z{l06}=erI<3+F+KdU)i19S@u{rU(X$WmAInT9(Z%3#5AYyfKj$ou*20rPW3cT-{xJ zyLO}YY3;_%vl)UuhuWX9TpEtPs(n$rUb}wtB~tsdTM_-)&DUyQ!lfIvE43SV5uFzJ zp35CR^Ot8|0kmtc;Ad_w-F&fLbA6+~_BFb0>X)w8z9cuF#)|8kX9@lixbFMF&DTlo z0^IU*_p2`7eD>yRH(&3us9^1t+MBftIJgU;pOb%iVnc>4-~e{BE>bUMJwS zMShOJDE5_j;*rEPA-q3 zDGc&>-vlpKnDKZY8P6ZWWK?5dEGvgzWRG%_7wW#C1_OjI{URAW{Q}d>qSv+ALd+3t znYRbFL1XVe;uU>7;RxHyMlVxyuGey6hIIJRn0^MsY7lcr2nHu$f>ZEMhJT}}WCGTO zn#`osR5mlD42@(I5N^JO);UXUvm`YSyOg9PTCSnCaXDsTY90 z)eC)CFu$BQNCD>yBDKGk83yHxMc`6VVj%Zo2@rg+-}sEexZ>jqR{|P^2#lm9TT$6G z0Nj_=?Y1--P>3&so{!Pp%dS!J83umQ8n{}!a`W8HGXyOQ0=hpEG+nEGvB_3;X$Cj~ ze^|KF>lsX=Q4|St5t5-1Zfm3<$5Xz%VfQ2$zOpQ-+TF01J<6HwP6xx3M*|JSYOgtA zwj^<$fLSyZH5D49(d7?qMkY#*zl2;(H$zbFaX^?KD)ib~R{_SUOzjpb%ii`RVXPEPxZ$pU)KWK3^NMGq-Ha3K%OJ{9FE=sgy*Iqdq|SYPhp zla$o9q{J|-0KweLmzF;uCCUkGc-E|2xGTIasRdk(!9@Efa|FK+F~H!Y=-t**6)cl9*_j?BqZ zy;`vaFGS$o=w`%G#^gdpiQH?><-A95jtHCQSx!*1Kx|R*`km}(Y_^OC4O6S+4rX9K z(<@*kt&c8(k-&;#gXO3Adp8DZ-smBvNwK!ra9~lH7pCcnJ7geotnOtg-6L<>#}=Bu&1T?X(_aV7=wAIhw~uL>5V& z{FKP>w*Y@r@V5$o(|DcQaD?ympu_%I!nEfYzF}zn8f=0;#U5E;=cBXur$jLd{dqWa z;AOAVi0>XxJw^O%NOCIXe|ctRTbpSGS>TPI?(>v42b1=VCG_L7-zkc;pZ7@iHMVe% z13p7(Nr_Ib1Io{FZX3+9yl!cD*>b~A^Bk-}Y*}80J*M0f3EnM~2PNUade4_w*^R7G z+m4c)SdVGMGASABUxw0ync-AY%Zw@`iOfh!PG^SIq_4uQU+$cpFK>yWsz`G5M&dy! zlG>|Kc1e<&J{iADCa_0F)uePAgLx>9OjdB)*^(Ug?)LFyx-$I4i3~F`9#!Un!-p45 zh9BuUnz$)yz)~D#WU6AqgKF&~8@RATl4o)dPtz`QJsEYgG}lQq{oTp~OHdV4(cApt zLG#B4F}9J-+YZIf2S!1-jUs9HmesvUfT@>SZE$?zh0KtE5e<00UAJ&6G(PSfd;&3* z=qM0Vi7i1%OX+y4NfFqSy>3!q-~oRYHq@HsMaT4{A&yy+B(le`T*y2Y#zokZi#o$3 zeX>Wzl-b12FEnd7tlM4KnQ@V;7M%RtK*=4KD&?*4Xf|%r1T=K@nViuyNo}J=eF&5C zE3ogP-61M0@0bxy_papT%a{w8Y>U|kru(^jo}Z5IAbW28$E`o~9MjLzt>9Uu<+?r(=h9-z+ew+J_pT}9(bY0vB}O| zvZ`$$HHL3cUCL%wp*n^V8<5Y;{1a)TK!$fKe>wB3O+WHr5FxvBk{ul)6DG4jt{e6v z78!CXp+O%$sryvgW`4yE*OqZRZ0qvWOd9- z>^bi>W*xejq#zY^Zh1%Um@F*kS^mJ}%!uNhfn`+kQN=1vYhDRx&MTN^sI_~mH?2|{ zbKqw`;c&gL1tgeJ9H!|JF6d+mRuPDdDdAQL;+2uSgGD==tZyTJeqKeSe8cHP;2U*JDOpm}l8i9~)tB_G zG?Sf_fVoZzQY$8tyDjXRqOlCyHI{J_YGAk2ZEY;!8CWQMeh%o zYn$?yY<2_8ot3dDU6y1yxw!GP=M23mEc3S=Z`9b}Z29ZuOK>Sd(;{z5eUF3MR8rmJ zfDeQwCAG*}Dqv-ya58Ly4LgXJ`zDd$=809O<{Ljcl^M=XkBlCj9M4X}qmZk5W;?ua z4 zqEjaMk}gb9DOGjdiNF)x75}+{M2ac(ZgTN$QRzLsbdcuf(CIrS3U#%{t?@ZvxNT-o z;Jf_jA#?A!Vc;Rmh(D2HlRE4z3b7STMNGHx=mm~y04j2fxX%3dZ#}>AY0vc!_5F18 zNQOpYtNY=qFsnO1ORNL? zMAku0E+VJn>Ch;RF)=%9vg))Fi^ZBiC7T*tGOz}xVj*>81-*M|zVY2O-zaIm@$FCZ zRgl(FPDI{JA0R7$SQQR$jnvh&OPar$lGOBWJz#4^5=@r>X<== zVB1ShsbD5xJQmq%_I5uneRDhalmMki9rTfD$29tf#e#tp5}|Q83>Y9+#P~*@2(`JL zsDXDAEh(j9!Mj^c<*AtyCNh4shH#Eek)su6Y!aXu%C-e)hTs`2?2Mi(E*;CH!1&-O zD=-r|1U#>u#i->nA+&~Ojvi@^%^WrCdLs3D$CZQ>Pf408$-aIova0!fy8$c26wKgD z%B|ae;Cekb!eoO{X1iQ~oe;W@4ah>*>m60dabfD=O{|n6#Z@USBfXyw0$;K~;8cOQ zA|;ZMGnXH@y`FxjWLt*4Uzw(1PYaskmR(F#7}7|W=cTCOm*4bq6hvTBm=Vy&tfD+xhv)+Adv(_ zBiDisHYHrT#Y~=J1DG@DEq2%eeTwvNXUw$wxvTnOi1W7KgSt4Oc!8Thxf&_VbI+E z1MdDRrliE>=+vP`|JYz}#a%}V3&>jI#)`KEq5D$_ zCqS#1$3=ntwTG84z{pIGT=D+PBL(@XBd-Uq1djBR#+UWhT9}6E1LNVKC>fS8$exo9 z`hz1#CeRfP>RT55N@oIhWeWy(6*F|=ZQZ;>P*7E1Z>un~;Mk1(1X;Cc&*8zyqg+Q7 ziR||}-h~uVr|D^?>*62E)GWd#<2BA)x7K}*##BH@ql&~}$z4BJnbc;Wrns4()F*ee}b*3!S_vhuXaEYH*B_mO`7o>?c`%{o8KX4w-CQtG z>@!3~6g#D6+*fTJ`OMl|&W9m}pk=OCd(W?yEtni6HW|aBben~*jXPK_s=gL*5qvpX zLQ2MW$s5`h2W1%03$NgHd1a)*jEbBa`yhppX+6NYT?~IJ<}N919_UR;H2RL(HeG1z z%Qs5%u+B}lMBL;SsG~a|*DrJ`*)Vs@-t&&mkygVY7B7~?@+XB3>m^8%pPbYrSxrfD zIw8pkpFq=+O~-+I91tq(${ktHGRvey?Ru3qsKrOJ&p@R;(@PsZ-o^-E<^c>`Jia#qUYz3)`No2`%YqI#JUW2|V zign3Gp4L-c{Zb5x47TtV+}9Mc(9&znbdb)j-pzWPnMFr0Ax*W89I$#L~SadS)2aI5pBk@JKKI-k^ z7}YXzOo}C2wZ~G>7{@oSeskfQx4(Jgn^(yi^=#(@p7-$)_DkM(GR!p}1z89uCRwhVlZj=NT(o-@o^2;Co-F9zbfjr_~ zJZj5el|NF|Y;hct`3E#Qn7E%gAV=LeM`OjQKCM#?taZ9Mcdcs1!|NL?Qp=`!&@0M3 zZz6rx_GYnYnERy#r>X;k4jv1VHH=nsZokMQV?}3E?$}hDDc7o3-TT~?L@d3fx{kV_ ztM*OV5;-h7G6MCDA8j??#<^t)$4zT(D193Cc#vcb^H_X)R(MJo4U@o9!M#v@k1{>i zNQm&O>R6ymz!YIF@h-8e6#;hKsgB-{zTuHJw8lWAqQ<=_K@Go#4w%^bcktx$f0}c; z(EfH@UTveP0Okxut~3Tj$PwB=2P zVc?$tJp~!VFkK5&{IX*?I&A9}VFsCY3~upFB8fZ75^7qnoh;hSB-1%U;Tu$VSq3Au zxa;0`JVyUU#H5+;ss!x$rDZf#ifOG2X*bjIM=~w3YZEi)!C0TK_gTM-{K|B?8DVsf z10t7@;!0#f$A?OpJ8JU(pS?G2Zre)NMgPj$K2;~~y)B3U2u^43b4sH}+1iq9?Q~in zmdGV?nIOQ8K{6e8mB++!oKDi2$F>qDv6DES6UReR)l6>H{U%klI`ZvL9a|pSPMyDS z-xrijh@jK}L|bB{?#LoQ62e+9)_UjX2}Rl77-J5Z6JxcJy}ddATjR_hp1XbR+MN^Q z_uu$_x%a@}O6vBNGc)hqxPA5NopZ0wocwI&%%}I?IXrXzLp*=x)8Im}OKgPkVXC{6 zj;@@eU$H2npm@q%R;vk-w+qw^-dff;`$3UmH>=e{K{9@QI&5CEd9iMA6EKI?rx(y@ zK$1a1(==0|c2`Tpw1>f?&@AqNqoxr}Mr*cCjY`;qr%93NMKwhJZ^tmx3~pJ@M*67M z-^F}ZIq)ezMUbMQ%PfBtD=7;P6D6aJtE#T?5Lj0YspE@c?NL%(6EiV6i3BR28gmN@!sigqN~B$MjT@nVS@nJ31W+aPJy-hNL_`r_?tUoViM7#Q@#yy1#t4pq$ZhrI7x`uxt3 zqcc~1HPcq7;!#7bEQm=(Kh7QQFC;}s#%c|P)w`d~nQ_#}mXOU$H@&~I zEtC|3(+qPJzOt|Sa%c!FJBwy9Fh?s)$+HyR1B3ep28V_kN80^_8aKc{g7r1V4Smea z2RFyP2#nEc_$TT!{G+5KQHUm%(v#Vvn-)paViD<8R9a$2LsilfLfumNDB{-4Qn5Wk z^(5fR2ItD_&l-$rq7;uW$&!;4L5hb~PfZTLwT;>aCEH6OfS&3gZbj?Pz7$<_`s0G) zsk&Nip&YZMHmDaflx~qDf+8-Z0ixe76sXY@E#uu>W!1UIqg0=6nmy@u+t64^*j)^c z0@&jaXm`-O8_^fv@ zx-neoAXBM1+ZqKg3gvM)f-54(e}$O6U~8EWICai=ZIjIA_Ld;FxDb;Y#Gux$QZ0`K zJGlyx%!*%5HCx%RP5Qw&ff1~SrBxsqZh#x0hk%;xr+ma#N2rnE+LH8pR1_2s!QF1Z zAy7<+zW3g}>!07dc5H!w*_K{Uz-$XGu9SG$hhN`4ef8FXuWo<))~$ob?!IvP_Lcqj zKK(4PJVY@nh%xXI1<6Gi7XknnB!d0qp#hl3mo7XB5j7% z6RI%_Ve6c4l7Sr45iHfM{Kce8{o{d8i6lU(OUZ}sR4uS0H+7PB)ow9pQ&6061fl}? zlQ3x+jdFD!t!2cOc8U1TyC{l~kV4~LfLWd*-Fy!Q#4@^#)lLSh=tGRIJo8My#Z;AI z^DsBbCZ5fa{&Y^))h0`l^*AnMTBSywk=?e&b%W$^G^R6};D*^KPtS8{TH+$o>85$2w8%zh6D0fo3S5&~jvymp`Ak%oh4g zYQUyifor*l7+hRxF-HroFhDe3U2M*4KZ`9^NpjXk$evuLx+7;B=^U2v^iz$rW4^)2 z=o?4}H>lNQE3(VXs^!xdzAjS3(Er=!R7FP$MrKupmMNbh>z$zm-((w+(*|)T>U1@% z)_OSAh7a-(tWJ%ph>Ky8F5;=E4`Km2XsM(hZpdoFWT``ghNkt?x{)V+;J>!RHx}mF z{AaPzc3p$pcco6NiBsPak%gECHz7zgs0)T#K2MuZ82z?sam!Bg@&lq-pa$@UN|Hu#`}3R?2sIrY>Ra&Zj-0pq+*df?pZ^!t>~ay@ILJ(%s3xgs^B8 z-pf5rkKhJjsuMy~5yYemY%uPkNAtbVBv3=1K!iE!f72Vtfgjmx=tVFDOQfwmWfJrV znQCIz8zV5B^p7%wXBUEs?DzJ3_m9Dpt&Diy4wa5p28p<9MHXB)$#Fqaz&!#NNs?Wu z(`dD9BjH%hPg}l* z)mTcF1SJ_73t?w5Q=z&WocnmHn^DU~DuUT69TUVuE!gWunf8=Lh8fd{Ug=N4DIvX2 zE}0Zsc7v|Tm=)lX2OJRbr>$dt=qUuN3kW+VB$V3PQ!M};Qadv(MqsMVJR^CS^AWBV zQp;_~2Sx^MD-y3T_MyJu?3XdsCY__HELw zj)unEAJ>|l8({VEm_I4~ZQb3ZBS#BGgwf6M;EX+;yH!{5i{vU!@>y&V!&c{n2e{$M zlex?@1>Xc~NatG?Fk%*!LIRdetu^VJ*BO>g zCr9Y|VS6v!ql5DBE8%+HW~cKdvI$&GX6AH8)}yNl&YCC&&kX+)>SuS{#Ao0n?lcc7 z{y|yW;qT82;8z--Hn$D5k!_nh!RRo^4r@0bqsYvllQwQ3*eS{}MRJx^OtrR-jg1MM zT2z3|v9&@iT-hI6F{WU)reQ8xOh%1#PM~J79trNsfO$dZ3_3whwD`( zRv47#R!x9y11!LGg1vBy8k+NG&5 za&w=F>>GP{r4>*|gkI--lf{tQN|X%HJT2%}@Lt77ld^=!Zye+J5!rV}W!)ev4I4Z& zn>Kcv*4ZrFDVU+8kD0c*P`T&2jqMBhKwCkBW%R-maf1GtV5CSy`g#9r5t($^ST+Q@ zqa%l#o0esy!>$vvUDx0DbgeJcZyN?HRsEm@GOr3TSx77eoFI!{i`2-MT&}4L%r)v4%{7iq zTFF@GDC|xx!!yKSvyzD=A~8`+CS@{Q!nU9yOLMBGI&{s6I8{Oq8^Q2XGdjU2^4@*( zj)sO1qNQp_&HpH#8XEcyb_KDjRqYBAvpl9CRtyp^(In? zz?WeasFtHkTIz}-L_N%fLHd+$9r)_j!DF`$oNERHv!+Y=gYz@TPRzV~a^{;4?tgY9 zumJA-@yy+iPtCk{1LolV3y1F?cxhc3Dny(=K)J0hjT z1tsR6eh6c$?S`)Hsnwo+Yy^wZ(rh8MxgtV`N<>_A12)p=ZKW+6wi>Lts$@YuJUzeSyAm*_uuJA(Lq)$^ zA0as~KjEWl#$a^|eO83b60yW$GTWll|F+9#G_uj&TPTm$DJSt6<4q<6DUuM95kXRx z#zUD%MWRB)gLkPnqU=zHh=t26JMx_$NP z|Gob1?eD+3bLQijmyh3n{w4Tt=BsySUU~K2^B13a=JvJc?;JRF_t@FHH$HDNCvW!F zZ+sryf~akJc99?Z6^-yTQ9>tcn!LDp6Vr?m*+?}jcmSmi=%Z4< zwp&?r{*yv77JBD1i&%@N+ZKtg_^DsCr{&Qr($BM6Q8%sDxv7dITXz&&Sn$%dUEJUq zZc$f;KXCJT0yAHcNN9*I#7_ac+-YIPZV6Xq;ffbn{Ti8B!e9z+FB@ezOcSGZOY4>x z7~SJCA>vgr#r?}k5h9X*cXyEH*$8K9rZy_k&lrKp(W~IQuC6 z47=dG7zWqN>BurI^LOi-Hw&;s%~aruW=C1aIUd~FUF#jYr*n?+8e5?y?5&kzF7q&W zZ*2l`B}GrfDZ~7)K`{}DVBLKpnsh2WbdwcxV7^;>orksA)*&OK*xStvv;4KDf=Q7F zmxXBr4XQRH!(fWS^vJML7fwg4>J-O(d&SX+5EE-7xA_45eh-hG{N>R8haX%=F+KS5 z%7Y7UJpAA=`Nzc*55GOtWRm*j_y^?S2ge`25j=|9sb&hMr6pr7-tlnuMKmfXa%lU) ziEfT!U`ZsIM|7%SwCe@pPPA;3WCaX$vc7%CK;v-nACI}dXypwKq_r|?ETIZ4xDM6E z)&{b9^JdqcmvB=p`z-iPAb({vvJ~Q$cAKji6doq5V7}6GHK2V3lGDdpkl7$J%F|Bq zcBScP`;LK4I|c`KpfCq$7W-VZ-~YbZ(peqUR`Zocfq*8X&NF*-7YK5Q$ebD22;;XUHR*NGWFTiHS*^#{$10#mR9Wy z``;w>?`DCj>i_2D^n-&<~^d=|))geJUHV_KA^5anHWLoS(W5AG`+F z{~9klE(pS(ucekV4C$i*`!C01%xkqd?Y53E_2CWpy;-_s5bNEGrfhS_Xn-pY29EhR~L@$DU z2)Al6V}s+bm{=n^j5lLAO+2b@R!U*0VXpX1XVF-~E|@7~voG3(B3WMmvjx#*H3oMW z0^^ZuuIkxXHMkw~8Qccp$3P+Y&zBEURkJP90d97_);aa?f34gUY$U#D!3l@?EfyK+ z=jtfKhO*E}*tjCdSYrnMW>j%2j#z!g5nD`FVnm3Fp}TFsp|tc8K6@Evf{I9WP^)}8 zs~eo9uw#nY7Dto}&2m0KjjUY>pk(Z#%KzIgpN5y%8;|iKWH_oIQ55t{`IO4`1^oQb z5TThe>BFDi_~{+^`JBMZ(?6Z~>2grC*WWlE(Zrqrv0P{f%|?R??3qSpsOrx zt8TVb*_A_1w@N+jW}i+U3_CFV;SPxm>PCoQZ9)BK_cifPA|79^`ri-Ny>lXf~j& zUfWFbkr4aU7N%+VMX>GD?q636(tf<3T#~`(fra>hOrRbEYwj1UEzdY_AG# zEQWk%Gt|hH;Sx=qcT>GO7@HZInuV>+(BOMtg)X|A>xP|9b9ahBc4Y{BRxc$n*BeRH zCJk3_WWwU094XQ@H83I(9IZTwUB^j}NxXXHlBOo8paO1pV5T)vIAN@3Fg_IjrJ3un zPa5I9<|S23h9o9wMk)mowo%WXUKuHsf6|Hgw9Yha&MTibVeqU@glJvIfG44Xc-Lu* zr=7<4xkkjekcbJg5R;agPEp`8$iptkYk7;O9#fdW{OdFZAH+QToj1{4$jixd$Q;h z5DWg?EL=dcWVw+?)D=jB)Kp1it&+#5f2@6_9OUcP$k;PE@-M`xZp zLaajZ-uE|Vj-6<-t?d4}tG5n)edoeQ_s)DebM{YnZ(Im0DhKlB^sPgOZXG)6l9Q5f z?O@Js}F_jnK-STJUk9A^{$BQsvgeQZs z!Kd2f-)w-5lxpDkIp64}rqym+madumT(6&$C)>#69)^&~00=QmksLm$={;)2k8y0lwsT;lbKh-OFL z_~D%_n6<-9HS?To)qi|1&GjD-Qqqw#_A)%$4H~|k7Oj$Ll`oEwG1Au?SX}MQ&GucD zlYc2ig=8|crRcX~Z7Z{KhHW;!X>HMdiO18lHanKNzBQ+FzO|nfEmjx~gG^knhfP%G zf#dr`$47C-g;+E+XIiBL8*BS#%}s(^OeMMNK8vMox=~vBjcRq!UfS%eZCXT3#v*}b zwUOx~BL*uC%qNBk&NvN{ii(2hu|yZFeOLz*W!Z-2VF~Zli$zXB8l0fR#;L{^XjE9q zMG=EAo-wdka-@gX$M;^h%@a+BrxCWEG)KFKY{OzySDyk_ODKXQ%Yvl1+In66ZIVAQ z9uyVQO4iog`LA79NkpF{?9!@wjOk;%{dUn6Z~w0s(7ufX>H_-b(9 zS}&wcH!5*J!W@u^5)tJ1QX)-#tU_r-IKsY-B6F|G%|i1@X4l-U2Npxya9R{48C+x6 zP8GDA4rLJ&qMmZPMKDP9bRIN^UcdS6pW?-~?eyrAN4RiLBt!d8gR>R%FVdf*MxNEf ziQToiqgJWx;%QW-E=?jL9tv(fF$FZXE0qB{p=zVWe1|oZ1YD}NqJQ-U2CZDlC{SXt zA~Vx$nUfh}Dk6f>u$0osO>`IxF>8TEv;DW8MKhl-as@0Kwuh%FI&qA(VDTnHxTWBb zZK8zs7pxmW6kU)+QdKIFf2n|%>Zv^mf>D0={o@GEec;rs18?0r@F|9HKRtNw(*w`J#bStf3? zX;7wehh^HzG2#J%4JMjRjAlSO6{674Ap9T|8FW+l#9L3g6T#{;Pnm?y62~C3mxE>p z6HZLs#>%uNQ$0O{gV=FbC{nEy!a#$&&sF|cl}!s6)AdW~Bhf|0BQIsqAgp!;3eq{V z7{MBbgW?jH7zqS2+&BMlXo$3s-{I{Ore$=)bXs?@p?4Q}>8k3GvXIRN!{|X>9OTEa zGDGi_i{gm2RdCi!hxN^o5=>8PonZ!kF8aAPzgb^OxC$B)K2zl)lD|e`wQkjugI;Je zXgXJ+hoU5%GfJkOWm>7(`Y#q5MqsgeFc^a4tDsO5Lc)XCyQDPi=H^YqcIpML@igg0 z<)Nx8DGMv6qLPsG5A0XF{QeER!FC6KapmJJ)~Q_kZpgz_7*1{{bGt}+46}J z*lOi*2UEC-wzeX?xB^)8ui3{!2noiQaU zB$k9tJ6n4N+Qlhlg*$2;*f&rr>SV;BR~{PLKwR%w@XmVkH~1#Y3=O47MfGar=jZ+i zuZ_yVbEPGk!+Y?G?2K)QvF;4sBVrTU*X-=kZo62S+y+yF#D zn{q^l$yEv+mjAVFk}xWG^nzm`HfooYQjv%dOVqk}X5ObfsBP&5P`l}Td7RL^Q)R!m zJdTFYdMQDQ;IsVr9cOuxCz(h%j{rrz& z*}H&5_Y|Lj@>LZ^L~VX&X4>IKu6W8l;Pw!7Fzfk~jsK#e`U_c6)tt@v=d#n!2_D ztU=AB%@!rA+&o!dx%p)PCM>E7_vXOPwe&PA=BR21UnCuyc6PMws0tyGNLHczDhK`qAxnbnAwk@m^C^hi z-+y)I%*Xd%e`n_C7c*zxYO<~B?pNb?UwLlk{QkhxKlAcSGv{8q^U38qUmkN=XF*9g zaQ+oRl-IBnYc?sEI@qTyTV39Gql68iNe`K;ZzbOCgcMOeSUQ1Y1UF}xTLKwm#LVfo znju+c;YnT3gS9pSv$+RLEplLhto;7a8sUwAzAsQKN7fG_z;Tv1P4&U8uTE-MIWWK6 z0mBeEF9{i5l|-%d7ObzX7oo@@b*-Tn!HO=Cw)T`s;6m`nP2{^kmzI-$@RfPisgmm} zpR_Cr9^VLj!lI*C#IT`yHfzuV#=Pnw2Cwi!?Hwu|t<1(o!!8y>F!sVqk`(aykz-kQ zmCVJedMoBzYegj?;Q_<&y(~LfTIEiGiD?fPxYKl>LUvKoLi!(_AqJxvlc+q81lH0X%sN?7Azs)uU3VW%a%y1tu|{+ywY1P@&8 z03j&JSh6ZuS*lhJIj?<+HYy~-E5u)0m|s^*S+xX4;xY#Mqlr3{fWB$w43^Qg3@Gyz zzc^}(IqWO~!t@}6W$^8;K1-XpK=f|W;2rFmaSy1bZxg+;QFZn2`}w(7hK6RYUA_Co zi!;w1nfdD5+gGmsy#M{Do_gxe`}=R-xH$9j*}Jcwy7Pz6W{!S+=h*&x@&~Y{q7Gl z@4q-S^z5@L+b1TIk(qBV-TCI!%<(txp1ykL+{bq=zHw*#=)Mlem~G>RAP7J2|KRSs zhwr?3?*91Y`yYJNT+ZD>^BFvnsH-DsS2d;>k=@YKvhPp`rp-M?=0Rdn1B@0bKqeW3 zgCcZ_nKxpthP7s@#F#pUgyH6i*9XVPk zVy+{{i^0=6C5sN*OGtl9PWn3*+T^Js4M{GfA*^h$98j;hi&RoH1GF$qORyMwb6n-c zBG=d>z#ch)*_!ngwX%{G>ijXS^VZBRJl4SS-KLM>V*(Gp$+HDITnk7PIC?2#dR+=n zvhb_3gNphH)65V6n2oZ7WbC4rJvcTC<3kJ>w&5vsb+V>9ZlI+F3S1o?;rV`G2i0h1 zg@8iPY?d>!QD-Ir7m1Qi&gx{y)@|=!H(w^}w(FzdT%>ht9ykOza!xB1%x%Ku><46& zLE$8XnCQut`3)#%^rEW$@SavaL&{e(boY;EH;^CSp?eveX)-<5&-TKbB=6L>!tJMX z1lCY_++uk$%E&I2rgQq}5ATuoAKzhno#6#rP(Pm7^5fa6tdfap$*c3)pS(m#h{kId zQ?BISBKgbtsq0hYQ`e@xnEIMbor9N`;OCpEGgIS#-R~S>UVcCI37j0C`gH12 zGi21J{xJ2$Uk}6WU!D47>LmOgugcf^UTW$M7~FSL<2Vgh;2gZ1`l|IW-@*sZPJN9B zQ&;|a5MIF~1Qz#S&O0-AiA=o&7ry{Mm*D4XnBDQIud9ziG+te_&TF5Kfg(up(8OG; zsxSfV6W!E{xe_)osZ`6_6|&%^S0_Xx)#$48+Mn6RQg6y|Q`0t*1SKt{h46<5EF;t6 zrrKm{S!z6vCY#_g=%X2)Hc5N=bcpOwA}L5Pg=lw3E;PMpHFFpB+0X`}h=EgG8y9V-bQE`(Oish4d2l@H89P7vdw^P46FPz+{C(KuwsH|h`SAnWtw32xr6Ic{ouN^s65cA zuMMtKzsU`d$Mk1658pg6eg5XLn+KhP>5p$7y?Gepqtl;Ee+_32OrIvxr>4Kb)6<_# ze@<>5nLaap4i4d+CNqGX(LM*GID7Nh+-RSNQ?my!3-I38aJYW@%jvVzr-O^O4Id9T ze~(gKXNvDwjT{q_${I)9BMdK<8QdgUY%MVw+QlbTtysyo5gUzw5nH?HGAszwVn0vk zEU-#VY78?DCnN$#%*4}Z+9Do8o~av%q(sQ5t`S*`5e@e6y>{g@T83$JpL$XpbD_oQ z(2=NBfmV)FU@eU?zPEL5G6IWrBOkG9_!t-3MHk;jqq30nAQ}435yLjx4MSj+j3P(t zw+W&&wcQ@1TUR;4e4AM51J9QoqWXEGq1#1nkbiM4F&9FfKj+NkLB5O2`Nh7M=Lx#CH>yCvL$1&rQBeCN51}o_u-o zAe_23c^poDH+f+4;NR?(aw%RL*9(FcxQowvjGa@GAGD2byGRRexV_oLcj2bOn$TFuJYKCmr1<^rE zAvwz`rdnIa#>Rv(3oJ)X6+JY?-;F6my zCA>H|1}$TF@fOWtp=u@#h?6*=+Xg%l$mto}C*Q1rfO!V-37526ica$Jv3Uaj@gLRF zMqMLMJq0F~mM5Dnva|TqQ)JlCX(mlg>p%VjrXxd3WHV8zjY7^4TDcH{hhnQ57^x}O zw}*xZ%h+kBE|zeOTO3}FGJ&+IDtrf~wm^5oS2cz-ba^-J`Rqs$KUffi74MPM!47oI zLvS_AV9AWpQpRN+x!{f!zj(({L=#ZN+5nsHJjL{>n@6WV2dUmXGJSFS^Xadq&)z(e z>QaljRwr)%`SfM@{oLP<-#k!BXdIdT)(P?-o<6U@H%mtwzZvb;0g|Cez0 zN6Gril)(IabMv_{35(z6=Y^(9fqCeI664)C%(fA!}-ZSw7`vBnYh|A@v0LU*f;T^6C=0?KX}#I z2*}U-KmGHym9D^Rf*(Q09{AzfMb!*9o?cim>#d1{gl zyGZ&$K0Q<;12pUSiZ}yWQ=o5!V1@Xn;|?s;Ei}(|yPRV&?y`2`zO5aEbyx$#@#I;? z!o{%~nPCChAW%_gB%E1pofb0p$5#5Xs)6g`%t$lRZd)n~4&~X&VZtbNrIKALX#zdN@sT!|~$f8t7!1W~G$ zi|=xcZXGd`;Q6wjlqt}tMX^O6CY@}=i3jQ$sSG5rB=8hysCiX0)&0r$_N2h}wFqQ)V zS3xfL$ddF`pRHD-a$QY=hFd3*-)33VXlXNOftX;?(N>dcnPGiT@a0x>A(z2j-7-a= zd#2(58h98Sf(Qr<>=EW5iBOfmfY9u&B-xr&Qti~tGpp0iuxK*0l-w6_MIowbAHQH| ziUZ@HsC99l+_nf4&Q#Sx81QYn%}ualX_IY6sn zJi1S56;@!JjC|$u=2OIE75zP{^Hi?oF*#p};;i@$wI;o$SEhm*8PEv9J9?!UiYXzd zI^>89FFM;(F{KOK;Eq^FD&F3%q!Q7HyhIf!ChO1$uiE^YJ;E(y+zzhu8Wq)55L7~v zLYILy9lthQob;jFNl-i68XE2hvp3*$LZ!j3>cDG<0Q%^mdw2mW#1O)mWvS^jJpG}A_UgfwUxAoVKa;;1u+sAqVOy35=+7r?eH1wP>{P`iiCEI``{DcWLXvF z`sT{VZ=-oe`gEEh>$h$0>m#P+)D(t+h}Q->U8Z>zP&4imQB%Rz|F|dLzIu)Gxoned z=-3n)!eeS=!b!pOZq!4H9AxwAWM0EW? zoZ@v;PVvP=5QU8@Jo^Pk@EA?c&VQ55z61PQfEB)ZbBZ9RRUc2ID?>VUWI+dQ7$Qrq zn>m}3|7yXEyX@`~c6TNEZA(-zB^#GQSB*A#N_MvSf%FKZld2A~+LE)f(fV20=%PYM z3UN=(!djI3u|&D0s+yO4t9eBhl(iDtSr^|-=eU|##v)n;fy|T9wU}tF*<~4^V06$p zA>hXwX?Vp_5lM*1Yj$xR0U_q9N<+jl8!VuhAjd+JCpv*^m8B~L%iC$XlD?}SiRtS1 zc=P?m%KR+9GY?K6XDaD|k70vkV5uY$f*2E!luC4Afr?ctK=c_L1CbS)4@_K`xQsRHPB*~I=)6oG!h-e5=i$udN`3!|HxX(o@jrv196%s~D#ij9 z;XB9*@RkQ@A$Yu#seBgMajw;_+KQ$4Y{iPDL@*aTfPf1wF8HfjM&Ik4oSzxzTACLr z*4`Fr`HYFb4LxJ0E!|+|0;{df8#`+=vY0y3Rz|!N9GAUy;p5izteYPHj}u} zWI;^Cgjl31rNE7}F(+%|VMJQJ8Y+!nH92z4BSpr4+qAIU+0_{4ab&5sfnvAc(gkC5 zw7K#)_uo5w=jbc9ufH33+HZe*xJCENW>;Lw(@V9f1i)FSZ<<}r#PJedu*z~38&&c%I=}!>> zJFqmnDbDnKzLIJjFzPc77{h#0B(|m=UFy-cauk%vFkcY`c`<~APAZ{8>8lC>u7?^3 zT%q!j#xMYdaKfUJkc@{;4z4O`J#DvaFWfi0x^5MBJ|8eEqFUOBR`0&+#oZH{*rI5e8`3a07us9S%cNRo3A;eq+prK;$ ziY!FpYf-^k18+p7qa2+6WjF7XD6AMy0A#_)dOw15(X59;w}-RBu%Ry$L+{w;Suh|6rax*|w&|um(@2D07cm79_J5Oa<#KHxem}MyZkt{1vnzxoWPZpSi z^S4YLG5GF`L-#~&D>$L`(=aS%4(fw0i&K>A8l0lMnA}BCP!hfg;1Mm!LX>+eS8H^3 z?cvrOS6Mx~bEmhGtu{6I`0dXaaycw4m(3cok>TwSIa0^+9QZGf<;nI?vubz7ML`k; zQBed*cDd8>s?{0yX>`W15K4@A_~ZOm8TP}2RX$HMynKn8ZT`FlascPxqt}a+5x{@W*4r)zRDLh(a5FBnq8Y6u?6QZAb z7l&EVRXy(#OG0+{fBLrgkqAPr-SnOutP4Vus|;fh?eN$2bbi=2nhom8C0ExV0t?kk z8(+00Mc=k0?72}-YI$KLwheZ@Hn1R6N9)v}2E#syX39#|Vu{G2TCDnbVg1cUPBqIU z>$~}GUdP&B*U?t1(>kciM|4BuuHsQCg7|)M0mH>fH&UPrD$r*|M;wt1ymWrPvtb*t zDh0MeO}AKzv}H1U4zeY~MzHb{mKauIS(ww*(GKmvo&i{b7TLtJIntlb>AFhRSI7CA zp&&QO2QS&O!(yh z2QkCr;%T#AgYcen`E+AB{XHou#629hMZf+Y2xs#sPgAl(ovWArgZSgmn0%IQc_5ODoo5v^(>I-StQW|QjG~wPg`sdwu$@iUAy<`n={9rzjNmJ zo%b#@pKCC4^8CynuHC+I;_kbL?;rg5_LcoJCr<~KN)Of2rET0rG29|WWA#f!qOnCK zk`!dGv$miW*{35tpT&%MHl*`sNs$>`uUDtDG7LxspV);3gH7-Vy)bJi!5Zs9dh$BP zxZ*Tl9-TNhaUPDpo4CWJVGX}Pn?5Kem8NIOuX%U>C!Bn)Ro}-BI2ch8PbP6?fPg{ zqOR^Hzr9UyMA?XW=nqX^D5`#O_Q!N1!y9O-KcRr)NEXHmmPUwhVxK)c%*`BFC5570 z(Gzpg#OkV&d@n0S5~7|KXFDHYS-tB#g$I#tYvNoZ?S&`O;WzOSERKVd#~r@43uNMt zlW$I3t*k9F@!Tx&+IJzj)VK3=o|{ER9X&8tmzZ!hFeiL=FoU3CU>|w{_d#I2!N+qj z0kFK>VHzfBwt>u+0go|mWQcC?tP>G7n8~P-&b8p9Hq5OiTVPvxfppSQZj$xR4WgIA zjC^qGd^@u(gR0z3`WQlaDM=w2o_Flm5TE$rmA_wmnf&moUB1Lg8g4%aKI7l7o^^&1 zuzu0rM75TA@m<-)&F51n2}DFmN^6y+wf%Xfg`0q?>SiOG^g@J_5(tSB^>FY6i&3-; zgBOp^laqzjm{n<(O)rGju#$JRrzv<&!Mja5sKJLjL8g|w;h}!N!y@et39ROjW3}Qi zNBxUqYxyKRR|YDdF2gLJw2+@SQ~o+b)bdF!PYMn!dY+Fuhs~BI-o`Xax-n4HvJqw@ zxM{bIk!`fdHL{BtuFWk>Z%QI1N`f4(%|@=;N+_$8<;xcsPhiSEmJHoiXBSs~ch&#{ zw~bn?I`?yDQrn;I=qJ7Kazr-@@Q9am+ad!nQzq&EZySje`YVXRfd$l6!7^%6@iCL$>!$dd1xrPUn=GKRtSF)cfTQE?|4uELb@5XM|XqC+4&S%}8guogVF z9B32WhFEIXD(*X)6d2MXlZS}o z%3p-fP41sOaM0AQOk8z^ENb3KRk+UK z=Mwx}4=iTS*XN{MRdlj%MJGqra6s!WM3&}_^{7HELRk@HPatAV$<pEyx>)B#r}U!J%DubdF}O-LTe+c4&>>np? z-5^ypaFRL}Hs}1NTA47208N)6z$%QKxUkXFX{g zv|CINW7waoENSmW_+U>l>6#g1>N#4XdCowsEVCt}$U?%?U%MtDGtZI9BzSpYV7FmJz^}_b5e31o%!YgJZeq} z4J@Ayr(wY+bhmC><Hg>G+UZnr_T$HMFvxTO_r6^)xJHOU|yGr-)b2T~qO*4hSr zx$-5ARqcX=!#*m+5`vqb$<^iMx5+D+65%ppZJ?1ZB%nGQ8)99No6jR~#R2S@pZ;|E zESdi7=3#7~pZAixY5#W6i(6&(Gr|SIO(L&#L!gIt14L1ua8(`VK`Lk#YzGf@g_097V(Ix zX!#V?W(_k_wES85V;%HzS_d6%QG&mDmSQF8};etCNbw3i-)dD`;}RL7Ug4gX;7Y+^(RFIF}fCY@c6WR*e+3R*@F&A z%wfvWTI(<4w^&^ygKQXsb|lNnC$&ta%BPw9+fGI^PC6j49NKm8e+#ZM=oYblJC(Rd zH&wFIU|L&8^8|Au2E*FHqDe-XVc2Qiw9RJAiPRawvLr^5*n%3BYK6HNDkXTkH1A>M zy8rIScmD9ntpjh}I`HYO1BY)N_|v`TKfiO~+U*;!H6vQ(-3u?=xp49B>sRZzx>CH> z)&23fS(%bYeQ}E5z$(OIf*6mx?1SoZi~8R0@ZZR}#K@F#i6- zHxEBJa{b}4lfNAKpxOI{AAb1C!w;@MIKKbkyXRnxfmc#Y6eQ%bjSG^SI=UK~kbH+G zq7fnD*&Ela2a{PmyX=fJNvF?Ls&!F)ucZf{9r)$Imk&O!dIgfZi5JcJ!w-+X z{ovc9!TnJLLQ6;$Xz`?L!MK35eBx*&kv}2fF}H&xDLk3jKBt)~Lo%#rFti#nu#g_r z!B#r3GSs3&Lu8CwxeDRe_O_lbQq+sK>U`mU5(hE{9=O(x-TUTnX~5TFU}`M9I>~ww zd>-|fiYiTMk%fH3^PfN8n8WC z%bN-Y87XyQOuknZ?t2-cGA%m`4%#$1jt+fzST{|^9Mmvu?a2oZUv%r)Y)R8+Q^b!t z9P0NsEGh+XoTb{xMcOZBaxX`wuBu{^lI zWS2_QIeqkp_elGX@36h)Gl6CFg_}0++q4nUs@KZ`*uruXU<%sQ;5x~0GYyIshV0HN*1iXd1xA$w zQAN^w^XqN&=V{R*80JLo6q8vNgIOVTl$bmNChVWTc;=Z@ zhNk9-TKeFMXy3LPv!&3j2}w>UVr<_QX0Squbm+!#NgEpa^R<<3%hps`xoK2 zimjH}+yz28AM{$l)=Ln-~Z*s1wedPd%i%aT5^hSOHNJ- ziU-3Z8-NhyznBiY?nwDe6IKy)Fk=)Bmi*?K+h|XI`zd*s$!u3ulhJ+v` zR(UU01&jLc*FLd`ma~@&i5i5VM1)wh))#zoT@78-(zOlnJi=enXp{9bI5$t$&)$4+ zgm+IXLv=Oz&J=^O5K-2U#9d`*rfpVAS{qq?7MR2nMWcW%=bOZ{jsez5HBNd}on^rQ zU^v?t9<7-FRx?nKEXwJKK0psn0~yFM9TAuU%d74s!l#cAIVR#M#PwTY11mX2Fm~zG zm87~2W~OyG+1jP4FdgPT6KeWx^w*k&vWOvk!V#{DnezI*0{nKPf>IWd0g(CJ$T z&)quo+O0#^ZykE$)}ceU4jsL9=zZt#iAV4>YQG>%t1`01^K-`+ZS(Ye+O zONxbxh}5IR*1Xv{Z#G*VFOfY|3r^Np&!Y_tlZ66Ngs2Cd(f5W|X@KkM-}m!#uM7>% z)sG+pDhw9!lb(5|pK2M?LOzN0oz$?wnnOZ7-E=b3zjK0aoxU5ktOH|yb2|jCm&`A=a z9^SGA6F=+vnQp7|0@@GXKKaX`{n$(VkBcWBetT*IdGO_x2N&LW_`%`XGfkcT`Q`P) z4^A9?@cQcyKX{w`^7{3MAHMRBbNl~s;dq_UwdL{)ZD z)YYON^}R1jvJmr9Xf25ExLqIRq@QML^AQuTPFz8f3kTKu@WhqLw-=(`T-WE#PV_Tl zm_H6#1>X23K7spxd*T|>bBxcjc^q(tbGeSNcI~JY)-HYbsE!!YE`cQyf)tGj(J20^ z3asiv`SzfYu_Rf$(AN6HpWx=Zh2|OAW~XWJ6k^@h@cKaxIKS9qAV=z?N&-024!>#( znfr6iXd)-D8(GIes8E$ZeRlPSBLl2_+iC_6dthF? z9gk8Swqa2xc9Wyr2rQ4SM#;3O%9yJWT#>7lT+VBMdQFZCvZr#$w@lzJ49h7mhACU| za#)y^u#;yYk#tfMg%cgThih?R%MKiHdjFR-Y4JRJN*3J%ZCHc`t5Iz-1+3P%l(Zds*$D2 zfzKI_3Zfi3%kisofxA?erCRVbVPk3~KpZh484q0rysALC2MoQ))39x&OKDZt7%>?& zLtu9{g3k;jr$mH!veuA$a#b!pC9W5k+HCbLbGIE>YF>=as`V}U4ge~$AjLwn;trIG zI=ZGS-cG26xi!-UFJiX<*~K$VC;b)D_`pJtViDBYu~bxq7ww6*cq$=E9jW$cdm8V3#_oMtUC)pa%B|ec5CZvF z=3>2-@u(1w2~j~3qx)R1Yr8DnADeBpOp(4(Vr!;u8)=qlwKb+WsvPe*{n#)>o`_+O zTP$-rM@_QcA^RHPY9V;SYnyIxxQ^lYqK^W73j~_P#Uf4cm?(nmK@%(nXx8YrN-D{# z)GVdK~*m>UR)D}CFWhN%9UG;&LJ}uG#dbUE}SS&L5gIMKVK{moSa-@SBX`6@O1Lp{C zT%fMVZM51<81>mqC^0e94wp1SXlG87HO}+OrF8La#rO1b`eZ!_^pp}X3d@LvrQ9M^6c4emSF~U{J(x3 zO*$;UoU3*SzS6f2xC#fVl9&8TzMksy9rbhBnz?>I!FeNLmwMq4iNrY zkw}OV1*VZf8lru(Ziz_!z&4y;^Z`K4TwCAoyQx{8Yic%|x2u`yMwXN9Jg=+z80m!R zG}Af9DPLlHbBHE$3GsU*c@2Zm6Aq;rWF=qTfT)BOJyK+i=I~Lwtf3E6lbd9erqf*G z%iA(5Iiz2nlAR#+#h}n0o_Jpre|h&f#`YqK-!wa-OjmoqQyKm4r3Xi@JoxZ!^5ES2 zzr6ZB4A3hpKQ+Y>K~@AYfx+I0JGdRRzDmM+kob~1%eso()HBV9iZuX=f|_9JbUPtn zldOm7{hf)<)T&&uO&TD^5 zD=G=eSZJE+PG*iUZqfDTZ-`x>M7K$S!oC4tEPqzM%$yoVPZq*#72`?d0u-X~%tfSq z^AFMJ`%jx82EE1b0t1`-x=8=dwtmuCes|aAPIzHDZ?m-1WJ5Z;`aAl%cD3P$2!7!I z{APpmhGTN}we52fCW!n_5ET|4FR(cq#~SjOd=!M|FP2YNWH9@LRnMgcxWR~N!}FFY zUmfL{5EaQ!gNIrsNLUtTo%EzTV8!H0jqy5y zbC%(AkQ9>f&~B$w8QsnIplmX_jYUw#Y^PnN#jYEk{PJm|LTfNr0ts7$P5MDD7B>r6 znQQ22n4Tsps;LYB20q&-K0pg}@BjO8@FJVGQJ$VBOR4HVO1?8&3K#;3+Q9B=%xqb# zP~wz~sGv5AA#mdwm)24ef)ZZ}ZcGnNx555k8<;)74HQ=q9iv7Dwo$mC(S#P)916is zjiZJ~q69cX>}y0cBO^3zt8k8L8Dd&=WF+|BsM;)JXV^S^vk)a8GGdd6{-OVB<=2LP zj!>{_>>_M=&Wzxuh~v{Z;lnv7ZhgcmF;KyJw~_8(;jvAl<9;0>r? zw*N0;%z6gTK5J{UWz>8x%l!WLk1Tpj8FoBM7dF|Z*UqzO)-teyRbDj74m)@TX^W~B zBCamu7I6V=?M1+~WlZc#(KT@Gxz%OTVO!PB27zmbTUbd!^z{5L7^qp-N1Yb*c?RH? z@>LZf+RCR*YG5lF0shHAMw0<04p z1>0lTX-b(DVgeTf0yhhBSH++hmt3K2pR`~ySn@(NJP6wANxsidx2NaQ?Q}NXzPpl} zheMrf_`%^4*}@A*e&mo$FH`1SMy4FeLrW0EO{>JW9EgSyh84TS8e=r?6d0HoNFCw$ z{U%dwTX}(WV(68ucW$smz$r2E!Q6hYIcF;7UOR-hfmf}K7 zv2)D9((4pR7dM%f)5n%^Ny-r+s;tTRamaa)8V1wIMuBVsQSlHZFAWp2TnGAfZ_fX| zJM)J>-F^K}cmH^9=A%Ok7(<)3dW@k>Iy?|tb!XtqpB27y_Px88FWkR=B6zw{k_9m) zh|#DJcV!h_JjQ-$j6oEN5R+;}K^KCQsw0Z&Q`6^f9-BVf4DraD&)qyY{n_*-GX2@j z!#9siU!4AY`m5=)H;+`MvfxKj12o4BDH4eUmejv=6=4fSHD*i&Mzu3#MGNi=h8g_w+>!*;CElUb@1EZ zxf*kiJE)v=VOcr{K7|7sP^twO4Mbe~%EYCK>kjMs736(&AQ6#;{Q{i%4uJ(PIlzJy z-u27PpyX;W65oY^?T33mUl~|nLG@BQ?^;6D735n+h^81C0B>!lT+E`^^^Bp9kwLD8 zLS|{u7O99rG8#Ht+T;)gXxxD`HtD9xcw;~cg5w;@SOgN2mx4Ch%e8!o^ib9L-30sG zplf9AiaR{@h&8^_>yxQOm_Y!o+w^R!0~pZHiWVyjGlRH-$_)U@7G+n}wCuBNTE>cz zgr^dBRVHzAB}z)vH0|9yy?hL*QLF)pBtu*H137LYCBUM^x>P{OB8 z_A&=!e&sKe6^q9ks}}CWd?G1FvAiHIheqURGtxOeiXO4-GH|iG$Em-sR+T#5B_7WD z&7`B0261OwD_3Adl!k^j1pJB&_6J#Cv9eaOZks=*I72zkfW=*p8U7E33^XAG;nD(o`qK-u;Ut2QyxWX z2*p$0zhIwv9R~r4ovxgF_~9qymv`QO@bWv2^c0>5RMEq?UU~58mk(Y%`QYf`hsRFB zc;Jvcd~5t6oEi^aRxm)tglxGYkuK^!-%^PSF^^P&RCe6^=!^UNKfZh8!tJY9v7qku z_g~#P^YPs;U%z+uY;%bR)BQ8wUb=nd%)OV--Z}T`%=<6iy>jF3<@5IsT)B7jg%EL$ z0^{1!qys)j_)76#+ZL8974@t^3k43A(X#Dh-~=jCXt0@InM{^X3gAD>o)nhfL~rSWD6Yy&zW>k)1u|`dT`Nf9@aIoQ*doq zQrs2Wa1Q(lM-c@jQCouXb`#Fev!nzg6sm45}JJcR|jV{2R2vanidM1ypdpYAf2d?^psVs#z>59TClxZ zIi?qLJOrOmqw?~K1LW_mHdh7-v}c#UD*wn#txlOt^=ryGa{W?lz;7{R0Pj6k{(c%2zY{kehkGBbw=DjVP_ z*6nN#Ou`%;=Ch@Az0}NEwq|Z1+|-^zhDxfcCGxvvnp%x#!goK9S^UEFvumpgue4_420MJb6dZv9u1yz~B1QM~ikC$|o~c~5O7(Bzig)`KEtjil z3u#1Gh2-kGL2f{0>5LfF)3;ssHIC8ldyiS$z~qmMdm`k@%&WMqhoq+(JV zILk_RsxzV}skQ_h#=E-YC8Qt~#4?B|OM$X_423xVKYMTf9mRF#jsGimm~+N*29>(D z-sYQkzEVqKlRz&LFj+WrZg*dGUsPSUbE`_wSUv}VjrZ8e*iIbB2_#O8i5IYOjKRw} zQRa8?lf~pGM+Vzt@;|)KQ!S~h7s*wn21y_%NL|%crMmaI%d>tycgZZc7kN01G=Z=J z&{Rd&VrTYFFv|jctqwJftvLJaonOw9o;#0SzVoX~(Lo#3lnFYQp&-wDa$Q@8t|=)aeD>hD zOZIWA*an5uk1H5;3+2^drg!uoY7_)sGMJf-9uoq-9wdASxB60?vlOlnSIihu&xOq@tP+G*kIfAgT&dq1H4lLSM%~ z)9*?|3ZcZ6WE-7iGzd|fmbN5NDE7tQPyLsgB!{LXeB+bkap|~Zgwc{0okw{jblInYt^8e3+0eD|Jj+gd#;p3OiA?Xgz0k%axflkDecZ)@KWskr8f3A1Z_Wdjo{p zF(5TeKbG89lpJN|jx$D}*{i_3ZRWo3w+6j@3lw|@aweD%pYi?i>1eetu~ zFJ0Mc;yKP7QDG#9HCQ<-hN$-KQwGLH)kJ`IB!*&WqK3Jiy=`O?cIs(nJERBKI%J3; znz8Anv8E3$n4G-J14S4u3yiFip(56fIDG@OFlC7nVHRJmZy-gr1 zT&9yQbMl_7k?}%Vi0FRfF>0}#B|H%6kuvuys@AdfCb4Wb)kb0X0D8$7h{I-}%pj!^ zT;pz>lMIBCR!n2tqW3L&*f2eJ!_s_{7&026Y_fICv7Y1f;Wa1;HoWBi$hObV0qQIckL5Ml zlxdl7u`OzlJ79to>in0>ZRo-t6@Wwx>6iuyCKYvEi1y3YUzRlN>z!n*j8)Ei(9Kfw zsypFR$%ECBaQ7jH;Je^zSz1D`{p*N)=W6qC16AFJ0iOKm`m59$7@Um!*PiK zq+9(Uq%<=f4Cn{is=~5k?o#Q*s1@s;VWn$|-d-&h&%^Jhkiu#>WS1=Nlwl2-q)V|G z7)V==7HPCF!t=}`2#XL|Lc1xKH8SWG1O5mtkVY6@K!&1j7ge+uRZz>~#SJd1NFs!} zs<)*ShRxtdFEKBG*%noElp`)9(9)IiB$Il~V_vzOW#C!Hn zi?eSo&Yr#f>ea>B#};S*xHx-y@$_?WW^wk}#Z$A3v;XD)|J{`U)#B_Mi?gpR&c5S+ zcy00Yv+$7H?|rs7`}@V&-}=va?)J+cEuQ}Q;_NTrfnQ#IW^wiHu7M&{lSM{{q&jJzx*vcH{fvB z14xxY>lx0kIUdqVs*OY?G;_zog_}<KS?-35FB(0F$3K^3%A$O9!Qt65dxS{VGq4j;_Ti03BlCE&BPiX>O2l`3W@lzBC>koF53`-;0RwWH*VkeY?-lMrEc#c^uoY42{~mfMB5BazzC zEIcq`igOnVWo*MTBdwuiQi_JD`bMA>&ompmk^ecI2vt?l6YFt3^^fJ8zCFW-oxX9e zS{2eI)2v*ttnMtP(f&3ujS49W1+gjYv=Q$3JSpra9_0LgNOQeT;XQa^E4ardyY5F4K*f=6?=3(^5)W-_D-#%P96 zC}-m9AcuSsk3g!_rk6mcgf&sN>$SKJ={eXx3{!fN7nqEm z2(^z(4kq7-m;&1_;KDE){#-L;P%MQ+%%<6z4&{V(=OWFBeT?lSHakWgTm1D2x{>Qs z7}3n1o4+*w;r#papU%HG|LKiWWd2if<4HJodHy<_`{c%%hTy*Y+U)#m^OtU%f`cna z)k$tVKK}t8-uUJG)!z9(;K7Zv^B*id0{K2KMF;%fGYy8s;r^fzmk1bfp&)T^ah9lhb1eoy~t@yyx9(^nSHT*Ugo&px^J z^tsLM*lpqZJ@XsCxy0; zkW@korbdO96(|yIt7YhY3>~MVu!W)W*W>WYQmxpEiS0wb1uX1(Idqwk2A@cOaT=cG zNj9^}`MXzEbWcEc#!|Z8H*ustvWW4XEd(jDq z@7`%Vk-eDxXeNy(rEIDD5Y~*Qk9?BSyHMh^>o$r!%ylPEn^! z4pW;G6(hLAqsS>uiKIm?h~w6i#kq}RM{=7NDX6k`kxYm@Es3K?#c{iiaqN~^ui%+`VgN_b&fXS5!sSw-~cSzqAn#!;j+2ldv?FN-?s@_EIT4s~FtS1E^t5 z8@gt$H5OD=$)s90k75LTq5;^@p{@1(+~>0u#f{DFi#$IM3NKIjiRh6lNj+xgsb9RX zvV&&=TX@LS_B6)fO)d6%jry*ng(-s6AGo=8&YIX1;OyTejEORrZjBs>fs>{hnxYyh zC7D(jVyDtRP?h?_pvCYQal)WMW*4I8A?2u56cV(KqGJRX$K)z0 z=*cxw(9;;TH50KBF18oQ5DO%&c+4Vu@hUPTr$O*_D({V1D0N-Y+fk4Rqs$w9l(Z>XO40%X``dvSZKqj%I;0dV=IA=44qI9Fqv&i}x={ z3TfDNMNK3WP0Ix4RRD@UWSl&Waq^7XIv&A&Uzq=7{t8_90sQYhX!R43Cm0Na zpc4?}*IAcOtSK;40~nF>-cjK8Gj}z#wOK|@u{sG=O|2}++J@(1`rvxvC2Ym{+eFA^! z+Kn@$2fq3k9yE*Htq@cH`>7Yr^us#|jGEac5qtuD3r{5TpW$DI=Y0TOjt{~Pf1dp1 zPxuA=NF}hw~rLUm|exX`&iRB8BWGI#JE6pHs4>hUK{>VL*rX zqM&-`UtGy*jF@KHe>fH75Q|~Xn@L4)4<$zdELy4+gJI>=KW=(?tO>S$kp0AC4%oAk zLK0f6aKAjWufHC1jVz)%Lc5rB%WTOLwY;VWcq;wRA?20Xc9ey5lubZ~V$(gF7yL+Q zbi=SKKC=wQffMo3+U^Hy@P|Sm-!3H;Doa!{mqrgPbf-iqgSF1-wzBjn3~_gU^TPjL zy>#a{e<1&O<=5Xn|H>91ux#Oi`^5|2{_(AEpLzY;Cm*}>^y|EUO6ysA1X}Ax*ms#;NsmY_ETFUeQ6gUcx+CPAvjW z;7322@T5%!#c`VqgBEn8Ox?RSeeO=OPkRGM52$aY=+2Xzn4Dzn2zSe)C_T3+wK$W<%bMh-%1yDhNBR zB--+}x3$iSXSw{CSK4r;Q-^i34a{;|KE8Y=HypA#gVpKD_B7^5bq}=Q=c#|MkC5en zT$S5!YGZ5aVnl^_1u6Eh{=k318iDaAr=lyJ`0mbOVUZ3jht5hb4dB5LC=T{ zjdLGmBS>zss_Qu<4KIIES^ZAo?kA7VNUt+6O8F#lna9>*XzNS z`K$9EY_aJx|MvV9j3+@;KJepl@6CU5_KlY@+ynb+k;%NEvKU!DgJUD>cBvbn zwzWBNR|Tx>FoOmeCUMr|0$UM=@-UYrZj*l3!aDXsWFe#zil&;1hIk37_2AEEFmSE4 zvo=If0d|8nLiUN(N+ve_ycj@UE6?p)!dr|EAG7p;pj$AILr^~mDT}0%h>yP(#dei? zO-uOy%>+I_7uAdMs;kt?tWhwN!SHaREg#&O zUM)RqLNHk`6I&huLKY|I{%jos8Isc6pCu_#+nJ&St7(Zd#VyOGo#54Wj1RRfmgC16 zJmxMemeB>aYwiyY%L`d*7tW4>!5z)Q*|A?p*xFpzfB7XLrP4?at-}1$5=Q*3OHW{> zcsqP(l!V7`UjE%Tr+)vyA)7NdV&Y`=p;^RlGd~n|)D=Y88hOCb7sO-Dtqq=)-5F z!y=EI1F}}~0u@ZEt_M<+myW^>8&y@bRBXcLA`g0{wriCn^3A-gE?IA*A`gR< z92$>~rlutmX0{(R?(DFdHnW2&eA119WHy~tO)$x{p{&tAFcjD%8+&AEA(=|jh%KZQ zV**VHr=u<5^g%OH+2Mf<{4p(&0Tp&|Agd2(2AneovqMHs%bKZ-F%$B-Ruqe{kH?D< z&B7JL0;Qh7n>#rB;ic;toLWLlY%LolC%pn#VJEN|Vv_7-P7If4Q{&?n^5aMX?Kyt5 z8v*!8b?!V&8Jb-J^6rCIo1A;Qzy&PUHmG6<;Y}ryisU2~idf5wdWDbx%6HQxJ4ud8 zTA+@ycWl?n_5bP83bo4uuh$ca>IwJ;l_0QTSN0aH#Vt{~+92q-jdJt(+zFT7;#%b+rN;1pD~i z6G?YHO{>Ci?vcVS!my7llBMJxezNZS?3ZjH$n<2U0vOe)&}yJ&D*84dbui2%@=n2% z8y99mp`?^dGPZ7hgjsOFcGg{ia%nf;$c1dZ(6s}?W<^m*8POfIv{D11WCBHbsusZ1 ztRIAvik?t1@o5A5{n8ECsnE56!3e)l#kZsW@5ta5UcMp(~;aX1vTNUF^qF zfP+G9M9S50fpQd!p=fO?%7ay;!J##h!`cPk#^BcMMtuefBCQy0R>uRqm4K;Kh{>?j zz0JhXlw>;g{J0;*GEMD`m58ZmZLYUUSOm#YR>^}JI5kCW;&@1}UV=};V&zq1e*0;e z7sq`@$C<$AR--$3*%*^~7`hZ6jt2G=tD@cH+t(kgQ@+V6q3%=e{KH2 z;4tM@V|d_Eb^=;EML9VJL&YwUgoZB0$m8S(!jXY`(Oxy3S|yh`wdNp6E6KJnN2FAv zpU;2bS9@Q&@jUTsw=ZFBHbyo6b<-;w^S_zDd}H>;nfX6sQc*oC3hUCJz=HN0r|UKB zKC;1b4pO)qvV%X2p7!#?Kz!e5I8pr1;by$4DQdda$_lMI;wK7<{7<^4#NQK7)(E#A zBKvssOxK~Gc<`Dh;wQIYmnRifZ;hyLhn^~|EaTMhb}{y(VkBK{tEF70DG3_L+JVs;geV_zjI46fcH<9pi7nkb0X%Wc=bbrpuil>IVRqdPKG-E;6#;b5D987%bzRVBRFNoIH~ykQYkPbFeIGYNID-bTI{* z5}aLeMf;*cH~dwwYzm@!phahDfjk?M5IPp@Ji7|yj;euPb*UAVk(Vd0I1 zi&?U8;pP*6KlSXw<(p3~yhq^t`wMT}Jo~q2$-?h$J`R^$CJXQW?b-i%7e3s45T92JU;Kw_7(1)FX zA38KSBHbD6Nfo}XDj$@!p_pcyzay;vIUwAu?^Udmg@O9!6a<{?8?q7U+?j!`(4{B1 zPn&}*I;?*F=uuKzVbp_EZ&>oK7c6jg_J z5pOw0)Bt^2ik6HGPkuj(2h+>kjV-80`RIzSu7`x^kZtjbPtY)CQQIN^N=9J?BZJhX zTTg-9fJE6|wKFNFx;iH(ip^BCHXMYLeYHx5Bd7yXSZLva{JuJ zw?2M;@zm_?pZ@%-r=Hwm(!2f3pWc4?-LIbd=+@7l{p!kN(PO0^Jy8@v&Wwg=v~a+J zD4JAH!>fBl&^cZCzX{KfU>Y%*5n7ROSCt{`#h4bQM1+B2-D6|)VS zB0}wmA1~gGMVOw1w>f$=OZE#aJ+0%tt`M~EQuUeK_#plU-L5LoVZ^D0X~Rn;;Pp!f zC7pD$Ip~<8WU%PKE3j#}Zh+dwd*vWm7dGyynQx%kF?fgtdv-FJi0&*|>EnjP(^^9} z^goA_xsgy($<{pH&`wNWrKcr$O=%0nF7_RnMNSIBb{U@E?CQsw;L~HYgzmc{1@pr$ zVA~|?!yF9ShaE{A=7L$3huu6AS%#z3lk3*)4OK~LZMlov2HAJ4zqq7WuxoU~BI`|d z6kBplu^C{Jwd(v^K2vCPSAR;M@qcc9_jQLnp#jRLNY%4zFx|N&rZz53NX1+-vn}Z| z^sOz5M-CAu#=>}RE8Bg0EDe2 z0t#y)w8m96YhXwHqNER=w$Q=IERj-0a5g7 zoy|IJO|#C(bn-xM6D)Q)`F}S(2~q-a=MZ*7(&+KLt`IDOz)f9Ax8pN!YdGIAX_qPn z_j+>fyn_p3&!nXGPdp44MUNJBPs~&%uq$0;?dRoz0~1s zDq%J;0w-2c(6Yo%Q&(g_tzIzUfuoT z+Idd?ulL2Z-@|sfpWD&tn)jEy+85U!x5-~$_~QDTh2@9D&xh2Ie@4q}BTlymVDg}{ zF*`x zVZEog>9*LR~#vK$3?8FDMig7N35czQ(bWprveNqsgM*YC~`wdwh=*vPs);o zGM~n^Q6oj}6g&s>j(ePk>(n-l24k8iyfe4^>bEe*ce3{NdnM0-fwD8y)#I#+E?Fkg zXVp_rc-hP|ea$hj(#Xi-9}&lyEKr9LhnHX}7cSYeXU{r(%`gm-if+V?4mE~>+A<6# zrKofk?R_P}j-$Hka#e$PE^25z7Yz}C)HdTRG$K&l#pQt*bM@qBPO}3PI-pD)SiB`E zc}!yY=)VzK1RGS?VvUOn!k)CaD}1UW=s#R1{GiO-n;4ZDFGyZ>oxa_|g$K;ke{FtW z%SvMdaftTz7vW#Mvm7E^*w$>(JI!o%5|l9-y}EgX3C9YiRYy&r*N`F0OvT2>01IZ+ zhvgiHxo>rs6zSerlzyb|&^S3zWj5&nJ;dfQke6}-jMBqa5ZWik8mt*gBE6nuq&oc= z=usnllC4tubEv)ar^x}jqeUzmQUY|q;#i8dk2z>}MHUB^|Cl-~d8(qP13~l#d}2Bz zcA|y}mMJZ^P`0Z{+J!qnYc7%ncBz)cXqcTwXtPsS6*JSS^+u}ha&cu}WsdEcXq#hu zU8GZ*(befHTrgw(Git9ZGy!fFgqf$fMMsGN~S&F7NO8_FM7IH0V3Uz zJJ8noIbn%P-hb}KuWf_Op_6lO3xfTUC3|2!pgW0$SOLr~P-)Y6Rj6t@slbA#CXi!7 z4=9ZRl`-^IWMHv(>_Yte2Y&_{k6Zzukkct@(dp<+iQPh=0hz$`ORWn-ee?ivFoVQb z)JV>va)~)2pOMQ{XBrwM$veQ5vaQR6A^1V_>y5?!}BpE@! z3{3xB60^f13#Z@qs1#Hk$ZWdBAEOX&#PV`gF# z6Xh_44m1U?O3;rs$))3xDwVV(X|;@7k;`Po!TlG`R4P<%v4w3<2@MF(A)__Nof5~L zsp$IM0hii52w2x^bgrK;)~KJ*F@R{KVrzHX^%K^2)=$7__E(zSv}QZUg><7Qxpt_X z2ngEGV z86_FNIt|Q%12WHDe?Cg#)?FQ7zVS+l*n(IB3klJf!zo*H%e;#U%)JrSetv@VRD_6u zleX_Tt?(&Yf}g>qH`95Og!5|&O4k%kiw%5n-)5Rylx;N3kB)RRsWuhn!3<||;d0Qn zq#ehH->qP#j%XyQ_`t14sI=?Tq;o~QmW02erqkI(wtrwKnKcJGqO3& zP-jA11AA95`DgVm&AhJB39jk*4bjZ#!@%+wn15vduD~cT5)Bi)eh_XX7`l>JmrJ+} z!WXwQ934l5w2C?E`d=XXWZmUO)Qp*MNS=fCw$+g4N2#15qlHlhm&8aM@3(n{T3~&- zwT9%(#{4xy&tC&<_xSU-z{qhF$`9#bu>8gKH(`d@hftec20aJ`ymIG=!pN{S!ftN0qDJNPY z5#Zc40$KJ(*d%)$a|Dv{(F+dxN%lojWL3!uv~G?~unL&rF3Z+4dyBAV!=B`mwF}#- zk^TLnL!>IIp5@>0FZEz2=u3UOhG%vKefU^mQxDnv=tsH!iR^(HAyFNG$GNn$y4Fd0 zRMjxe?%8km1_21UbOqR`!XZCI_F&0rkwwH3I^v$yERglWV$WOeA+`MdB)i z1yc6Ahj1I-%BAFyM{@8D=s7S<9(|M`6acd=i_X1439aNojD`C5+2trt7J8cuda_I_ zI|+W?O)FKW%q83OQSBCEZ>1fPWeHP=HD{kRLN{vY(2bcinF58#NLLpb#$ z=_SX@yif)Wp4mmb*ZKD?Cfh+aNh|dTQ-8t5`X)WP4neuBfr?)YGS@VuriK=prYU+` zwakON8m{bB)Gn1B&Nd$V2Sj9UlbkO{WgI_uLQ==^tPSKj2cckwcI&)37S>U*c=}I^ zXU;C3zOs1c;^OQ(Uw`(=t*6gzE$c^1Br;@dFh_E``$tBG_U{@Z69*0*L|5DX=$N%X zz!g#S1~a#VeuuD*2LmgXjrdF@2iY`}=ps}JTDv4)TQznDLuq=N+cQhQTT!H!zs7^3 zM<33TL191S5_H8P@rfe}Q~sZD6tV~QGK7?_5!e^F#bJ~4U6*bdw>@~GvXm?`ab#xV z2s~_y?RVq6R6)kTVadE=iwV1$cKB-eq@s zLxd|1cE7O>wzaPxUR6vcqp!iWJd-|VPeY86(=81oP)Q_Pcg1Z9Mv>qFhszk={w@g+ zYv<#8Nx)qA>kFX7lu3!tohOB|SDU+Nlgiwc?KoXaoUT!36SBz0z+?lxje_*SwF;oq zs$hqD)o%Jm;%J?QsiZQki(V-#j4|S|Fgf=&C1_{oE%xtQVXA=Ig?t+nMw1U<9*AGI ze3KorxI<-89E%b5;y>}YD}uhOOn?d2KL{x!@CA2qP3Pca9$2^u+*&9B=MC$f$=tOa|-FhtPsx+^p00Sww5m z^rWJyrlOhANut1@XEwzM)e*!VlNNMoL&h1ik&ESj4kuv~l*jP(13QjT%l2FXx+|+^ z@&vI)Sl;38YJtwpfkKu>(C;q#MPzps{L zl`<8SbW&3^BS3c_15L#6VfK@)X5fQWvc_zIeqoqNq{2?{0<~i}OuBjwSgpm^`RuD8>LLRO{JPqCcydrk)+v&7rKGG}Bp49SRWgsDWCN z(2JOs0P`huO~RVyOIx1crq#y2!-(mZwy=t1l!1|GIgTiIC>mLx857w<3W9?DN?(r_ z&;)gJ*E8quv+VdmSVxQGhh-`v&*Dz`@TS>0PGy>F4gsVCDf5VrbeD>9g7w*wS>6VykH{jpv@cyrS z{$fKyt>0v!i5H-WuYCR*`TWh#Uxe?^!-*FOwDUzgck%O|eSQwUzDVH1@8I7{aN~=8 z@Z(s=1qB^u&c(=VS|a-WI)oBJOhlqhp5<5UP~E(vfBaCZp-}H7VK>YKH3Mdtt|Zfnss&V)$dtk$ zFheooQ`2FYv)}oAE1KIYsP-#O$k&MA79w;Ny@Ffs@)$O_mR7Sl{9qbdI-68=Et^c_ zQlQQm{n_FEq2&7J7}9yh&bq!IJXYvyukvbi+p1d;DI`8>gVc_cKZfC3FqDgu9>)Sm zyU2@l)llwco{UqF-!#=7(Lf-VsPImTHTM`M0C-5gXqXu@nK092TG;IgzE0m>;S7Vqd~#;_!_lKe&V5{% zD$Tuamw1VkScMkK?!OVoE0jr|BLdo#upMS^HKdeb3))q7El{(ZDajp%3doQrMYX=u zMfZI$`zK;ui=&wVoiTP|;C<-QKPFC6dy2&h;CIWZpF|S(Oj~)^gDjRQ5Q?zlB+zzG zCI}52+tYDB$OH9=A}xRz5G|L-2zzxR1#hfcVR3Deh*oZ_1L_IxDZwI~C4WEl;{PE( z-Z2c;V6C^tsO%M1@y9#Jk5u9(ZrcTR6t3~Thoc9^|9QWMOwZ2Im6N-FFt&SWKO)&U zUcMLpIrTl9=sQVOa2t$F-A86fZ?U~<1Tz^%sh_HE`_N@QhxU<)!vw3!SqwhXBXk-> zYnOa@+PGN(NyJA_7~3pxftAtnN_^w80+LA6ok+I=S>8Jsp;1p#Ogoj#<&z94pSP4G zTVFr2AiIGflte)FL?$+T+x;-5U~}V`zlrd}JPu)BcZ}s^R?`(d6Pt<@{(jacx36dE z4}*bO5OA_@$VTvVXU2g|i6RO`kwIG>oL_hztgadu)|EULn8Rvgfy|bY-$78yb1Y<5P-HrD{iPGTe(jNbKl32CKDdC5llY%YqJf;+VDz&IHj9NuDj z#DOZeQ6H+p6Htpla)mO32HF(*!6uWe=;t*wIX|LX!iUh`P+98Ql{|pX2C=&K^h<;3 z72Csmmk0}Xk|SdST=@JHwYHlZ-)vd5uMWHhBR=Q~8DYb0TEd87g*b7Vc4h-T$hH}F zQPy=uYXj1Jkk!r#b-*6qK($i`bKGpBfC;-z*1-pOQ8@l4xV-Vfp12wYfgeM{Ad;#i z(=$y+t#|~@mlnBGP>vpbm{4TYp1Wp2izwm|_>nyPlS4)BQ@30E#ActT&EsJoEK?*W z@-znS^DfGd@xZj~)jNX==xbmLBoVVc-3HDVdbRF{)fyMl3ax_TP|vGG7Lkg`bBmGQ zUQ+O+#87zsyP_aASS!bcoFcHNEnQah{2+PYLBd~#3qr^uQhCC35eSu42XWl|WY%j` zwC`df@NG1S|FCxAcp%9k5lC`Kgr4?>s_1Qbnh)|qZ1a*7(wp2+f3;yMdRvv|11%MS z7;+mML@+Xn-sU7-m6dwOIJCtDg9DTaOLa^fc&-m}#PDi2&PD zwy%Gi1-u$i%pB4o(~x9vbO17Y7F0=HdYS6^j;`Q{JHY z*s?lSn8eCxvd8udSx~J8!yhbPgjj$M>}`7CXu?(#V4&$KtOn6^V||k@tr%^j#}95s z^D;_fN!TR^I%LB^8dC_lp4l#{YL?TGv=PxLSEjH7cS^*&#}!o|B|}NJrAkhT@ayJ3 zo`34jYag#7lYIx(j^aKc*+!`$Edic^9QpRCD|eoL{U1-h_3d+)?!0y- z`sxN!Q8R{;PAaK&3MJLSdjz`jFl zC;)J)g`SsMLdhi8M}0SU4UO}+3uTys_k-Dfq=t|_UC9sB8nxV7NPMLfvd*Wp*t~ZS zR4H741^V7^W0lIO=#!{JAzb9TsuVKEYwb>!RSV(9DV9u>M5{SBirwPMxi?n57`sZL}A|qpS*U*#^7#<8Xw$-LPINIivT)Md46;)wzmj)~0(1b`U zYD!mvZju-h(-SboS_?!X!DcZ~L(uWH)~k3Ed`>T zpLn#m^`^`OSV5GmIDMs8%GU@Oj3tbnvl!Hw<1U*-Bvft*hfVWBLq3rqxKkKZ)>5%q zU+!DOcha+X3CYSf-1IdJMa{IT2Peznr_b$QU;OgZvtPaP@vX}*ZN1rZ`{hfwU;FH< zr#`y%+^JhHoxSzzCnL*aAD{9Zk592+%8ilk=p-`IYsw<2WVFzj1s#)HVJVZ8z%-d; zPJKb>tgQ{TdmhHC^`6;yZEcVVo1puy+R@+s9r8$n5|&g<#JCJ15=5sJOJ(69$@qA` zD>OVlLrzB)9;XkqMoejgS(r3uf(NasZ%r32@$sP9WMMgxALoNkgTIkdaY8Gaj+Nn? z-TlzwX{tq!e_@etv1SJd7o67CRkP4dU6DIBRRaRVWNNC;pgYdSD?*xfIv#}l9$Su z5bdAcVjCF35=>Et9w+;kRltL+fCN*x9`x5^b>?yDRuvY2;>4n(@Qg+i{lf zjI#`SHs)-N(BnS1E!ic79cN~CCDxM05Ve_(O$7=Y?c(f9U!Q+=>zROyr$1Ue^Q*f?|C^V8_vL4&zrOIxFRwkbIQ!0*S5JNY$2TI2eV6l2ZNO9@jzG2K3 zvx@?ZSWI+=Klt7EjCy%0=@i*Vt3|%~ok2$ef?E#fMUqi;=l8O#0%K84bS}Jw^J|Vq zus`+n@S=}C@N%vo*-Bd%_fy&)ly`EEIr1hC#3(7 zpz}Kzlt_mSOQeyoKHX-yMs!l(Y_AH(r_*Z?wyol_NF@|ClYxJ_qNQe1UC2I9yt%7Kgthr6)qz$!r|;Pt1Roy(W*-!+r7JVan1`*$5Z zs*oN{H%v8?e3r~y?x3zR*+x9<)94aYAOKu#7(6}q)+#g8K zAhTifpL%$UVF=YG-kJ`zqf+6sT1@L5^XGjLm8E3DvE9 zgEZ1I=z8mf+Ky;XS~Slr3mJAcIHsM(ln_0&foon>6h+m=K&-`GSV2&k-f9b9r#5t9 z9c-kKs3CJm(pe+XLEG7$ag*HQCweDYB^F$0S0|jIDT#DjQt+BNrUV&|n1@-IW7HBA zzYMv_Uoa?X6}0afsd&d=9IaMDxVbYKKqsuXqg_Hf26i+HXvYAki_8*splC3IrMvWG2I^E=kPc?FXQz4UoK}hKiyd~R<-6UrgjyX>44OcHrRnZf% zX)o?ayC1~}&Muz*{o4NnXg{Fe(TKhi?i>1_1O8_FTY)9y6dJ-`<{3szx?EdZ=U$+ zSLa^*^4hcCocU>Fxh0*_u`rGv^&`&BCrQr$mTNj?k`tB(_tI{9Delcawwp>>Bcp{u ze67e@`o4ghDMn>P+UL43V~b4sl&++gEdQ~s)%T-RPLc5`cmoz)YZ!zL&89(URjo6_ zsqgm;2(PtRY4X*T~M@t=y9d<_5a zQ9s3J|9t+p+1~-jbPj0*njsA50J?OUicj2o~;j8P=3;v0p z!#CIcj=c(9a20zJzWAgQ2G(~BlKs3e#hogneu`6k?J|QP6`3}$)qCd|*+nHoi|-)c zYGCq1VDuoWX#MTy&aK}SqBT-Nw2&MPnXH;n(rxTDQ5V{xSG9Pdjx)KH7TVpyE_#wW zjbbMcv8j8t1ep}V%~Q*YSMfB=CqWXtP~Ur&7vf~+thwac0y33EI(B4kJ8kZ})8=wz zDlri#M;&wzjdNULB8$9J)?9Rw<_soZBvP#l>fPRWVVC*L-%KvS_Ord)-izJVY^cR% zNVPbF)E{Q1Er&L2J=fUW`4YD&$y?Nh&6O5uC-U~xbH#qsh>fRkCm9EaAxXc=EQ2P-x$ zI>h2rhyt{9L64u!)IXftJ%c}BvL=+wy~=!8UoK2@8ccSVC18u~9i%Sp9Xe59@G!f; z`W)D2^5R5f`5pi1st`gqoO<W zk`>B3Nx`Cy^L@wl@;eEyU_}uY`G4OM$M4dt>ROpeheIyc=ChXUA+m ze{R{6sRm*TYUtpCX?CFgwylsN+G9_9bt1|tm2d^Jw-DzTtb3x1E8RUYnV2={It)hK zu{?{eEgTdTcwF<8iQ%&{5i)!`&%8}bhcQ-ld0}JS8b%`P)1tJh>D%&Q8(si_vPmkb z_*@L2&P|jV$s_9rjI=4{qTV(xDu@^a+7GjfC^_lbzEc=0(hrYq{>;}7`_3}22MonA z*n(=LDw)HJ41sPfH~^(wa^%Q`9!|zk&+C!8q2h7a+{VkiS`F+`i2(X^BAgn5lFcZo zbZq$dZd5)ts9m;;AHS=Ix>^6I25V(Q254cxsG5>)iw{s4a=& zl1kca!!li!N$emml)=OfpmXBNouG_VWoqq4sDLHZL_54xn<7y2HPcw+hNAG z8U3L3#Kg$V-_O2q^e9qVlrWt5Do60kUqAB5Kj*OoJ3A$2j?rFqhS~7rWVtsnGtGMw zk3LFvgY7K-o$Pt2LiXZnnlv(oLdf^OPnytN@Wl5(^zfkUKc4>R&Xd2){_oXGcYg6! z-^yJxJ_(>rRun=|g}+C#%+7Ykhrt{Fe(H~ZKlR#1I_E-ctP|zAb9R|b^SO&nOmdC0 z*_BR+JeGoYu9V0Gp!$sP<0z#hQ|n2o7#E%__+;8HW=L?Gk(zJjZu;f8Lt@eZrXopE zEeqRBrp?$HNiEN5NGW<^63Qr}B+b}lxA#M+ub%LL9&KpD7uPQr>ZsD)$a~Nm4plvz zM-?QZYl^xJX?j<4v#GSmkU~ZayToxC1G}~xS=lJ*q@uSm^@Nu6Fh4<{7kpS@$tEq7 zA|-ihVT7p&+W;6Fz2rEn=ax;`;y5{8rY_+QaVRhD^jB5?Oqo`z%!Yd;%lgdYc_F4A zf>E%9&%Nna+ZGHvDJD63^tcTwgD2}~=@BjlJV2sK^hng-~|qSfm39H(aI|5E1PSz>Yy9$!v>6a(G9=A zjq92i$OyQNU9@bDvyvs;7(i01n&HtCU_mHmLdm4tswgAyEcM&)!&M<;|0CyE?3#`H$27{%{hkx+i|emO*OWWiM2(~$ zJzUNYiXy3)ZS{-Y$Cw}HR%N3B=KJXsb;{G!=1WZGp&;hRE0VkA`x;yxh0~1)ZEvPC z@m6Ma;w>D38O3Ph_}-?3=|l-Lm?N0fUWifd+yzn?L(L{M#ccPWgxxq{waSwIg6mNW z?0lqpkkp>yU0k|1Vrm_%w*i46;JCn%`GfRuyI^@WH27WT;pj#(t2BwKSMN;Ops&u@ zfYM4Ss@b~DK7c0j5e|1&kk)|g=gh;{s!MwIL>Eg(b2%M|j)$H{VC2O|YEOYzeFOpHDZ-IpldkCz~0-pqv-d$>3Od!VKg9C;_L>_jKlBy}0HZNk4 zg)i8_%sKxT0+rN1ZFZ+_3ktC3V3bc-laZyMGdo~A8x1e65z>pR`pG?d>nsUPOub+FN@5`%C ze|h!P?YGX~e)S4`7u~$$)Gk5C^|tIg(Ot73x0I4jZEFcl@;c;rRC3`ET5$Qah_1o4 zsRY_86w}%o9Md2iEumzL*x5VHlPZT{46xs-D`ct+Lgsluy;2d3$Cb_^Aw<)#T)0+aG2t>;eUYhu*cF!*Y@mlN($Wr26?EdhFSF z|Ki^tp-vf^<3iKMkx*O64F$nRNtx?m{J4Jt1S2#rX;DNc1?~z|3T#1`k0usyP$mtq0- zTA&yQjp6hv#&CK~W7tS4TARiuz0k(6kFPrSi<$XPZ=9b0c>V*#8$~$Q&k(qD{yjuz zB`IA?8)m{JH_qI60^v;I(ogDW>8tXorqYUr#6PN{B{DNjM@w8>EnJ4)gl_&FwBsVS{pMK$9d&u(%FQQlK1M(WPZBu#BOE;rUtf;ySP>HaGO!N9 zP)P)}t3)6i=O2U_U|Kez!iKl5#MULcd9ufzv^>O=C1CZUI=LkhTw7KEVZm!LCP4)( zF;SAVTIL19?MWdk%k>2j_tAVoRW;pA`{m**kgsY-R+xe7l*RE1 zY^C4pBg8$<=+vec(V8UkxiXuc=F!D6EE(Hpi{tPXCFVz6L;&t9pyGs8a6mk!Vi>V4 z#|J9ys^7}T(096$a;0Gs4@}0mNeD2FjxagJNKe1#9H(Uq%t`1aGUS6)7RnFf%-S

UdG*7f5Nia8Oq*V9myUCRuU$CYxRf| z7Ex;c-2A2a59dFb|A5S2^@)6zT}2;kDf4swck>^>>!Zu&ED{o3BJ=MFYned~5dCx8MHV;_N$% zv%g(DJ&UBUXWm~t^NYnZZ`IjOzj@{LTR*+N>9x{jf{Z})e3>$1arUvr+1LAuq7Xet zLXu;=QmF-IR5I0ZMkP~gNeT(h+ga@*)ME|Vm>WUZ371KGOT;r;k(nJA^e=%)XKZspQ$T0f9sEOctp_haq^hZu0)4J%MhXkI*QuoH2}MnZM#*ax&jG6=CETI$w!Uvl3A+m- zh`!{Fj*yi9UHf`K&voj_v0RU`de79`9-k`uBjWITWVUV4HglBvMDXr>$CR{wK2p%}1H8KLPr zb{ysfSIGL=pf-{7EZ0M_y+-_{Gvv!=Lu_rOmMYNRoUmv;M}CA=I>{yyN~@NDVehAN zsJhk=S9F$e6qc@}GwVTU-Nt|EfdA5~COTI`!TB|%ky0|rRvj-iRZ+DpVAHAp)lIHU zv=m-gMKa_Hn^W5%!^`~aIm^pO4}Vpt8kiHw1{`BDBpjoLm@j&|Rk;hD>IHLR{+yq{ z^~sHA&~^8|@3uql*XbM2tg?n3Jk+*^9rPTBZ;mx={`~xBz6bKt`D?`I8$UDuXE^aZ zxiJeZeH`h;;rD55@r}oc@7wzbE{BWHA@#P;%k7`QFUb7s^BMB=6fTPy`9CFAc zJLE4#**7sd1FE&J6jVUf;Z+L@Bkg+m6~3pvR42FvcGKqNV$W^{O2I)|9FvVnWhwZ$-t|_yOqP6Kd4{!+92F;r}q$;eMpAbZ43@2AN zBe6PWZKbsJpjD&RY_nRHjB$n;Ta#pTHM^m|eb4Rp99J&+`Zj*s9kXC@caiGPsvQ@i zNDeYw49UNc9EC9$y?#Me6AD-jp!2KgfRdDIn0cv?Rvt*oP!j3bNkQQ^MqEEOI%d90 zG8|0Y_JMg1?CKvP<1plXEPUqNy=2bKu|Az{T%+nj$CRd2{5s5WUh-+z zmd;?s{n9|$g=uj`NlU_>34Dg3)JuNso3&CD6@rZ=$l`%=UvhUXWv_j&@5ehfc@(}= z4YZ`w&|PF03@nHA)Jr#s8r}6e-Ue=@1>haEkTF9Y;f?h8dX%hyreyaX9N%IZ(lGVn zfE7(m`7C=hFspQ~L3%5_res$*U zZ+>xMvrmfE{i%nlu4Hx$J#{Z;HuTb_SVY6|5Yq1eLNQ=>ZcT%JaP`yWLZX7CjsqRg zMJV7HbwoO16=`i4Uk|Q$n6N~lEJcM9YUTN)=WGJ|K}#rlW?M#`WCZt>YYr%N8!jVF z$r#%xu3^dvg46}5f`Y+Oif;CGDldqf-lhADWbswCI`~P@*1*tm-P;vBySonmwfWIe zI%wB>J*!64Oj3(1BA+Wzvf6}}2qp@p8p>TNBo-pbDXpl9?Rw-kH9|NpwW9lsyQ@U# zs-m~$uWk#cAf&?!8(b<5($N*OEra!*27>mpDz&ylKx=ePcY{9xqVTX((mHa6Y2gWr zTZhR(R`D~&imWPG!AHgX(T`S)&@m<*VHc^3M45hc$VYqG-#^>hPLJ z2Ksgl&+K9_0;pAo4TjrRT^&e|sv4%*y&-Zh%16dsG6wha0@9%k(i-W3o~lK6Sq)Iv ztqru~3$Bu;kz6B9Lq{kjEk0l4{k(`5`J}M;hOgs>rld2m!$|uDvEWsh#C%S-%pD)K zR`ChOeZoFkty)_^Lv3T^o90D7kFG~Di-RCMhu~6#&5k~u#>#@({6s=mlYtp!bdJj! zYXoLNT!x}2whPR|ScJotki1BVAGFvYat|G;%8WJ+zXuCa$NS>?&)hng&Jts7`HSoS z<+oxZ=$Z7asVQ1IHp~&I4&;82OMev_FQhE9QiYN-tyE($x#!9Zw6K3_2Dp#45t&KY zh+-$GHw+4m6uDEtj6h-cI1l5}7BDI5h#^7MIaik4aXGc&*?s$k%L^qX8jL41wS1QSLY`ge= zP(KcH$%G9GdJI2vS8%vN2)bgVw?m+9&It?cT!+VzOc#9<24ccwlIy9snkX}}3xVCp z9?SAjBgAU^0^9T}RANIlQ{uYYO&@{Kdlgip!*Z=hM&k{CG4t++hB0j88D;O*p^g3**@8#WgYi%Nl7 zilp4#u8M1_ifimE$_c}B)v z$yf{mN~<|-Fxe<+PKK;#HZTaQ#V1m`kLL)rOL@v2_|0r*qrC@4ry_(X-h+dzz^d+& z)4>1p#Ix(HP(?o(mz}8MuO+xD0t4be7#UN@ zfG45TEka$giz;M-`5^S`@mZU=?8VM%Kggz;#DqMRz_m*irkyptK)%dO;BzC_bBQoI za2cNPOO*Gx4>`-_$4d-uNb^hUBf>9MruCJIDfqY}W7&x#GZRO)9?4g?vy{eZg^W{f zHIZP%n#CB&4_&pgV;{90>X00q+0inc4^)|4f%X{YCy*1iR_zFlx+&l}lWKC!kOy;I z+QIv=WAwm|X2I+jPEHOpK$&HRWy~^><|oxAw$R{nez(CM z1+V2&^2j4O_y(4^0(BpK6wES)UPFt{y@8d8dDtWi_3yKo0ACWamK_new>tzg9MQSB zrDk69%ai>y20F_alg*F{Y5ELm^)O!da!eDc!{!!WHVY~TR?4gHkGtM zoeg(Jfq+aUp{*xCKjd>uP>WdHocVYlj<- z9!4<{lOd_DC;4jy@M#O!)EzX71F}|%0VT6?+wwvdT*<;OWtyg>Q`?GOC`Au7@&YW~ zIX=xTM1}8)suvvw5u}z>RK3mP5rIPIAmtU$E($WrXnqxRz0Edr{MdF-TJ>NzBGEHT z^o{a@6wJwslOi6RJv=0^?F3P1Lrok|75D}T9yi82&sAVsa5yIqXr-psQ>z-ALW+Kqm>1uLSczSuQs6woDY`L zzme51bX@44l`B`Y#v^0d-MN|F!iC$mU=cwcj7&003sY7MhTH*KsllkoWCHu29&kA6 zAw5nehLw#rResf#bjLVMFiuON)gd^FO?TzooAzo92P186z+9R8SQZ+b{@ZN)v<%>xSL8@BV)pKS$CAi$VpHp7Sv~wt#(!v zZa9Q>ntNaR%y3(5#^B;aVYk7T$Kc~b2augk*o-hTitAo<+cz(Ho^dJEhEN*DuNGP; zrjpFGnO_kU`9bE;$+@=$f#!J_$6RgyInciU2um!n(ro+|w9*djqAb~CL)&)}ixSD0 zU7*sY1wQwFbWH*GowT8t1_r6tH-FNKX|^tRlVQsX#+ZvLIjM*up9I~m9y}}vr!x1Z z%RyOnb`r#o5u6k~QiWA@ynVKG#;J6E2-`89`ut|ZzL zccN^1dqs&B{eaK_5`ckLJSNdB?cx@AFXrTO(eFW4g12|bVd|ick02zK=+TrBkDp|A zslp0D33XN>)irfUGpT5)*g5n+_FG9{MO~h&Ki)yOQ=e!D4jw(KkbY3PDpj0+pjX<= z^Fy^KneRlQb;t2G3Q#8$pa7>%Q%SzLra04zo`{_U4AadORdjasTbLPD^P?}U_-|yq zf^|~}5CK~z3}Zip=2f?vhigxNuO1y(bP&`49++v7)y!lzkx(-2YGwqv40Ed@e1@6Y z1}}ZldmY_slG~JwBSa-A{m~6($4i<`+1ZWc?9d|66}>GrV@iYq*-E&+Iu()-K9sHJ5rEpiskt0{*v3<5U4zE^XIV_3~CVHC5hSdDe z;bxnfP;|XjZZxd1w(!Qn`wJHrE-$0FMqnZInt1`t!`v?|YSr)vXnK;quL= z7v3d3H)pZaZ$6Grfkbr0W`dTPY%Z(mRaZd^{eSGeZFk$&nJ)ZSb~S5FoQ?$%0N;C_ za~368ewD3}6ni?UKWvg)uD#7^SWNt|@8 z(V4Fq>XYp`pPndo(#EsqFFg0XL5UDSsQ^SvqOGi0B0!J=aAWU%zg+iqZ9*2B_V_p+ zaad~EMfqa`*3X`xQuTHQMh}^SNh9w@GCnHx3A4=0F#XPPNw|^DCeo^&*0P%+{y;M37>aIiM{y}T)^e#&F0nR3#9o;^D0?hY<>)v^E7ha{b>2Q<{z7%Hs2+B9=71s zjpirI&k(raPhwEVWipB?`Ev%q8yaa>J z5oCrsRfx*K7~pr!Fw64lMG(K(ZmHU8+qTylw~5UdhK%kEdOG{?J0JfWZwzDDh~LQ_ zqo*Bk+w+LP*O8;tDpR}4X$*N=>N0;fNcR%bCKV(wc3Q~Fm~N7tcQi}zv1%|Cg ziB;W7RrXA*0m^*>n|jC=>@*@8lG(k{($)PJ2A|t`=RcM0f?JN_397kV>hlXP!mlh= z^qTL3v-B}Ih2ScE3@<+b&*@`uvMw$^yPv@COUsw=r-JX_Y+eT!>=UHr{geFE;_~<8 z;pkdE+lM~o&yiRd9K}VrtdE*E$YE19&U$W8(9i?pC)#|St;-n~c1fHiCyLneG_Yp$ zBr5btIdw(VeI=Z>QxUQ+D}(TdbaLlt5ejylSru*=!sh+Pim-a_|Mo{b`^Y0VNQmH5 zYYbV<%^({|R*#(%ez@ncd3_>N;_@4X2;I(^+`xy@A**WeWFhowtE^&%;tR+8?M6s78#exuL+CQxGmdtCX603c5ctQRhVO_4r!Bu zZL=D-9qpSwNxTYW^!g-r4~q$52rcEu8YH{NzsXGk;0$t$=jw4O`f1F6+w-CUfo7pnM(9 z7+OL}W;&}a+vnPiR3YjtTeR9RXrFm@{!{OPUl*O|Dud)B89jEa{Lr`zJLV~9DMi~7 zEc^hq3`FVA^x;f430s_5CO70_f0y~5P`Y)LuBVDYfZV^Sv-R19Rk+1!D=!~R}|;SSt?23gk`}XeH>K<5T8cAGvHL+TJbvr zWX2M;wWE0rZ`5wLYR3pJ7OBk}4Q?W9ewoK0nEl7bxwC%1yt_5^O|6+qY(@9gQ)=yL1y&4}RcT2Uqwo)Of-7?7ikd7~@=*fccvaqFK;;Kku znQhoBq~;QehNAQ-C6$<8x7FGCv;NRBw@V6k`cWjjS^CI;E21byA}Me1PaY`4Srxa( z7wy$UFvV+l_M=Nz==sRcz%j(zlQAwVw^Z_Jn#U2K%^)U2-;1c*?vR7rp}b5{{B#!K zemh@1Oy!hpK*Q&e4&Yq)B8GJ8_6|nYS>!FqZB?xd{1-Augom+R)6F4&M-B;7xR$-D zRMuz99Gra_rYW1O;$4VqmYFrEX2{H{*0aNCbYEqR&w!nH$?>NWV-cKYu`Hhsc%V^u zCl1ceWO8V(uiv?G{})ev{c1Ou((W>oM<((0)oXuw?`GRv{=ND83vbCOAUdY=5{ z{R?0J?k#fvr5EqNdu0gk-`AI(BKKc;Sm^=4$--T&mxzg)fW^{a2REtg5R7cKcZ zoaK8yNxkW#VE&6JpEcxh8d}{?TyUxoT{Pq}mzM1DpKBW_*Dko_LKfV>NEwKrVkeazhJ?~dYF03pWX*9)*p3xvUh4^Y zR~8%a)5*>m?aW3<&4Fn#+gLBNvOZ#Z^2wEjk5?A|u(I&(%HqwHg*R3fe{%Zt%EcQi z7vEl4{Qb(}g_VWZRu<^R#m(N0s%@I&W-4$O;38sl$xph)Vyo;|cRfLnHExUUuAI85E0@eD$v{;y3LbrG z>4U1tFnLL)mp&*};hezrwq&W*k=kPv9ByzZif}>|W|CR>GyPSJnq2yzR^>)Hvaxh# z`pQpQdlk|`uREgoE&xio)uRB0+8g9-7=H=-w<=rE`I- zZs@0hohun`S$17#WlDw!Rl*+o)lrm`ARXlnV< zvEf|4FoHiceJnpRmQ3YGGMcufY*Y%C&}{5T>>+U?nPH&D02U{Or#GUdBOY39~23KOYP=w7D`$ zF1M9($0Yn2YWHEsⓈe1`58Jb}X70SZp+im{zYp#x`0<5$ol%QL_vep+^K3BN&-0T>wiAHfMo2p)m24t*eg2_ZhAMEsVfyaP zBBZzdYJ3^e7#T$ml3+y;QY?U5<9H&2E#N$aM13&DZK#VrhCDnd@m>?A!)!G(bc~(l zrac=SksJ7`wy`3!@qnpvN-iLz7@p6O&Pmx=a;z|U@PMZTHR#s8U>7m&Rx9reC%nqn z*l#fsN}`}*O!Tbz2sJp|0+Z8Gi_w}rCi7Ekp!W^bFZ9jiFLHTJS5m=pZ|?vSb=NF8 z{br*cLL{OK6Xmdeo6N>;Lw$JbgEhfwF}g!~*&__b$tmghGo06Xz{=PG;Mkj~lcckX zsw-+vS2Q&@ziz9u<6!-vElH6)8Kc~AEtyY}DZ!D$(+B3^)EImoH3$qYHMzZcwACSY z=-~XJgV?UH555pw)oscL_G;jsWL8la4vU@yJi=?Ls;^@+*1J>u!2Zpi4=f(1Vo#{x z7b6I!;$kirV`o}+4F{ElnE__|{OKRTKDkBq)tRUp?D4+qAeyA5Dx818uEyygnm9V` z8^O_;NpeUKpU8B!YCg*C2o+vI-0h4ZtesQ9$YEo%7GK2P6myi+mwr>>Ce4!rwnIsa z$%j)fpUz2%StWtJvNrjyOSw&OW}IYj)5#MiFAG@)$tKjny2WhxTJ|_$wgXc(S>2%! zk|&QnhJ3WE-$v1G#>|hJNo4kuId+D^x8_Gy|9F1HI}Sp3VLc%BK{@I~{wuT@YF+bfvE*6d99*6e`^ zHp|U{4lj4awJdaTyK4hHF%>`?z!l>%KjmO+ZzR+~+-WzRNo`81JaXt*7ofr6_RiLs zA2tj)EeR%R*&&@)a!EzQZWA8`p4K*o7Ty+VMGp{3y0a0K8h(v*x*iK}^ONQ$8|gLd z<^{0%(((o55XP%meip9br_GO0ouPS^G~a=@K9yv{SG^sm)_}~z@59kIn>Us(z|rr< zl5IH7OS9arGdk<5dS}`RnM~+}Oe&!y^*tG!K!9S@VYvX0`y83!20u4A!)iQ6EufcZ z#vB%<)j+@!X~?|@kKKZl47muu-)(*>nXzA#4<$UM1fIxGmY;&h_JR~3d3yP$%l}N8 zf5Io3Na|=whTU(2^m!O$@541*$DvA^FT+*52ODq`@Mm}<9V3)?DQxim05(K_?Xc{lY9%{`i zLsKQ;psP!!Eo$0`(~oSu_Ls-U8>A9QR;~pIV54NR512FPhtL{oBM2M(P>;p{1H!{* znbpDV!nYX0@ffb;;EoY^bA;CF$Y2Ea9r5U+DEMWg%m&95-kuhg$pk6XndKlEsAkrtJs98OJ zxfUM>1Utme6||^e2FdCl+wBuXI-+-%FZ2l`rYLRVLa2>I3eH+xW-3L>4 zGh-pbA<&nxeWcANDbh4lg#{D8JP4LKOY|9_sQX1kPnB+6paTt>RFd&qs2?dua~yg1 zsMR2Jj?)(D$&UD$YDUQ+HN7Iyq9vD&)!T|8wX~|_)SrApl+_TN< z4rjHZB{7Kj2m2u{=Apn4mNU?S48wi-V~-Uq>ET76`Kk%sKhVDOg?f&8~|LYeo8D%=K>A6W_%G_kQ`W_pZDT_x6Vf0NwfQdGQ-_#*X};|>b;-5^~HzJL{GKK)Euk>@XHU*Qq_zPVIns|8YQWu z)vYwhOf^hV2bZTG-0Lo}>oMVSCdy^bhHPqWAR=znzB(q)xr?1)mn!Co}eYhx5kG{H7YZfN2ik9>(Ano-d ztaY%jeNSYf%?M%fBD1(jVL`D=e1~-k(LPBfkqs>GVKWE?%UQZoat%RB7NceOC1jD% zV-6T8%wtgJ$TC@(5xRPGFZ(iMWP~C!txU6$$>Gr>qr8G-Ojz@S!1ZriLe|V7(;oCUsbyP}8=+afIwR?mHiS5-w|) znP+%)s3t5%YT(kj(LKTT*Z549^UZ>}kkg=ws^iH33&f+mLV+j9zU$IBRHD6FoYcIk zE6Ml@yAPpN&d`!5ax*54at{rY?t-{|bc3hW%;?h3V1{;Vx3<$1uV|v2lF4??Xy>aN z^@NOoX^=*3lq1h($rkPD5@-)$F~?Ql|1fHG?CY#yz5=L z^+V_=v#T*eSHxjkwj)ed=cQSHej$zxaxesA^AVQBb`6ZJxz8QwpLSpTRGpq>WjNII z8ZM%l;-CR;gf&j*n2%;d%_e+=LUJRVDRfoI1O;UxKz=3+?9qiFRib-_)O6p#^E=!D z+ZH8G&%;e;WP}m~G zZh+%xqjEATB!gr9xal~!#c+d0H>m9p_;Zp;5dP9QpJilH*kZ=%FT;-tw_;$%4(6iSIJ_w1 zNGAce*e+pz8^o!;$DsQ|u=yj9Q8cX+d}q0#1@M|bYW_O^`P0vPf)1qlORvFw`8kAF zN+hKOS3cdmnkWDEhvo~iAO4!`A;wPkPnz$MeXzAa!vH6M+Wims#Q-!B$ZGt&S9S9dKS`lWR z$at8P!0EN*qK{e(1_$pMxgT3pSSOC1m^^*@TLkX0*pI~nSUMo$vf&Qdgj#G}U15ep zL`G~zZD9@$@)^<)E-8U8F~h;LqpJ*BPL;w@v|)Q{xMw+-kJLRr-MS^_Ae+x5;yM_kJ-xZ%St1gB2)Bx|8YU8}Jr*8*E_)z~%JhcNAJ&~rOiBUh1# zAec&>30s)G{rc@UZeNo>--VyocKv9${ra7!{_~wXm)oXT-IKrl4!nH&_WO4(!B6@6 z|2|9p^UXWoBe4DYolE5QZ|^+)`GsfieDC(_1P-`H?p%gzzH<8=+ylqL#(QuOzIp>b z^`CFz9U-@`-MM)C_jkS*J(DLaqfAEJI;ISK%spz_&x)K5$%>4o&MMjXt$<*b5-VZj zNSPY_rhIms!%8$u$vCVF(Tf)+>^c(L`%U1(+^0ig?llR5NXHJV9;r0=Isz_ITkXtj ziBs)NR$N0pg>FcD;iEC-Uj6s+rx!#4HvHERaHqR_K?SI zV1tI(lT;>l$#0(E)e=Jf zAM(WX2t;|@kVROmU>r8fE}|c(L8cnE!)m+Vg9izO)(g%?D+q;A1o1HYln=eT4Ekkno?OCLE!VGwFn zF!>n+-FgBm3G#ERv<{I)iP3aPFkPp6w~ZiWOPn>qzddX2$LE<-llB`-dF=|wzSb2y z9=(XwiByi_c+oQ$G>TXd|x)J0iY)Im4UdBNQ6D#a4^D+ix`!OXEy*vrgToD3> zjOt5G$M7+A@G%2~9-#>%BnvT;o65SuyGVnI+!6gS&34ucmM<(XHg7hsHLovUh+fzB*I);{GPJeK18*NojjnS)aRKH$!EHZ+-owswDNTpq8d&NsF+V8L&=Y_=#Oo7spXxV4PZXRyaeF`D%ie_zW)*r%|s8 z><*Wg8QEC=FK$~Zm1uht@^wgC=}I;}6ccPU%}fX}(MSdSTVx9g$MdEcP z40=9#&QD7xGWoILTq3UyXLI?n14#{RFfx))W`^^bLO}<&E#(vGOtwvx3~#Blq6X@V z;RSqzx%-jIo5IJ&7`0GEbJ}v9>NJCIjgFn|{vGihwq#Dxv$2ViCnZrbVlcrQEm03o zu$Bp!kb;oll|roKDGdtKXprND@dKnT>aL`1Cx6?5Aee>0P(G$ox3t=bYOb-`yU zU^c`IVoNB*T@|+bah>b!FfMSa6f-bex=svs?BjzH7-jo$JPBXgHxWATo62+{^?i7W z%BR=UiQ=qyj3O=m@rEUa3RbDg;d<2Q7U>0Fxt>p`N-7&DSUq@AcG7?dzE;HCA`&!M ztalmU?U>d^1o(4PZj99|sLej8cBeE{H(@jc!*yey+P?JQ2)C>%H)zpvZE{9fWoq(1 z%MW(DgstZXV=x)$ST;@v1X#)}G(r~DHb>?UjhsH6CkL#`(re~<239-Tl2coCOvt{Y zoEg1TdEAyn;6Y*4$U%$OqU*IDIfhr5_&%pc%fG!C3kknK9^`u!Yr2 zCEXb53oubiNu{Q`0?~nm zz@wULantQ3`QNr+3_H)i2X10?$doSi;g@Age`sf$ zx5W_h=kMM;%)H$xO5H+vveN* z!MC1wt)-i#*$T7q;QkuZXlZz4>|*pjv%_bZUG$CNOn2^v%qHXwIGi6)7QB*_Z6v{(VEv~(BndTlsV&cCA*$L9BayVV+BWk&sRFeAQKFl}nV zAEDLLr}xA5IENS1>a4IO5e2cli zF*ocqfa)lLh=oRgv)=yW?Q3^_aAyGv!Q3`R8GkTX6Vo(QPT;YkPQ>&jH3{%w&5H- zIW;oM>aNp3N-JT_cWmPG8HCW5NL>q#ugr!t+0vG|{`sXJ zc+AfKee*K>x#WDft zYZ(K-x5P0m>z~uQ&uE+7e(c;1G_ytO+;+izEE!9`u~A{n>v*2vHDq+`3m;*iN z7k%1e2A$(@nkL*|zr)ZTEt$}x1suw3ZY#^r6l0P>a3d=2Lck*!p(ZWw4kGR`OVm7a z=4cj3W|UM)QT4Q>xq!aNW{*O{=d}@}j5>-r@;;h8=bLm&^05&C=*At2w zke}T11Q2Wj&L^xUaM(`!0;tcPiaokg#A0<}IX0$!;A+883OOzuUSjg$Lq_!?EzKID zQt4$*=)qLLH%$o@)+K>BBKPLJJP@H3xO8NghJlA_U(sEQ%`-z$`{lu^X(buHY&$UF zmZ}(p4pYY^qb?=G;MUolq3(fmsE6taz0YpY2RtzT?Z9d_P7T_KN`CyH*0Tb)NF6@y zuJUO3&aT5|18Tc^*SeG`B&4edoRQaJU7X}#l}qGMgWO+}>0XlA)NEO-Z0}8&tJpm1 zR5^vETXL57u#Y~5m~s-u3K9W7aL+hMhZRfWu*cR(DSAL2If|EFIEzh?(szQnP*{?E zr&ML7S+tkMX1+yiRMsK1R02XGwcszn?Qo=| z(XKbgnkUy$Iao;Vcnt|zM8;PpB;W%7RK9c)=p@i^rts|}}G<0W#iQ8OtLG{Oz2(pnwC2vuH> z?lVZ}XHrS06+eAuR7hM3>$9nBY(BAvkYKxRwR$Rcj$#|Ef;IctVB>kPwejzMFCc#M83WQnB26eWS|6;pxC?a~Qm%#sgSqBA4){8iufAM4c%m zmrydO2kK)0i`5(fhR7;fU=;~HU%B}w*9YcdQx`)sHb#zzVG zY()XijupLdG-5Dn4I3_-nr&o#Y0${wP6`XHx!hLPlgU+aR!j!$@qi!T=_8;k>1=Gb{gEmj7FwA+N-ROk1O-)XtnKuqde}BzXpck7D5(IuY-rOx zfgIB9X0UWNkyh1gBDYx&nwrfhY9^(qC?uMmU%%Pe!qRc@Y8#rICGeSXO;7NzUpY9Z zILvI=!{8te)mpkjy$%K#(&r>|w*yv`n6o(C>oRC1N6468=5J8jpwzYL5ZA#o4M@Rk z2Tk4Kgaq zG4MTImS|1Sb|1cac2k?Dm29BbK8lxi(4uqjJ{hBiVUv9xk2BHwBJ$s~m7!ocHELF) zr(_P|Lt7RExk=9ZhGo*X3@1Y`#L0w`);kNIQR2;EUNpF0<-dRK5r>57(!jkkIzts)GQ z&+e>IQc614**m#?jCPY%T8Dt0m)u&PaoPFKJbFMLV6lr%F_1##?@Veda%dBVaQeY* z?*vvca+2GyUV0gl8E({~=R}a_#I+L%KZZ2b)k#QgY=bm5gsa*fOtB5Rw}h}ATBW>F2p8hT-zar5R#=x5V&r;v7glNmdfhu8Q0|S63kcuPeD?Wk&@JiY3;KynGcVu z=@<8WHNv3ECuCLXHACX5U4fj+V z(ep_)k;spwRQNG^U?hJaS%80;1MuE~13sHYh0>)Z#vmXrG82Il?-*yMP0q4nKSZrh zINabib(rf%g|v6($AnvEMG8)gX%Bf-8>hPT2D0Z2jg^NZ8_@rK>&BLUXoa>_t_yL~ z9-3>rl7z+_HNiO)r5N?dt_6_p(lW4rvxh*}6*aXNItP40gO-qmI8K$Lw_c&1QPkvK z1RXFCH`yF_XptMeRBexAp(G;tJCn3_*fp>DYV)J!$IYwDFUfz`$@0bJ#pb)s8_SoQ zpTN&c1U5feUR+*Sz5qvDhGTFi9C5w*$L7^<5%~D0@akss`tkz2^*z|WPMSBoUGNpx z`)XQj{t{05CR`|9$5n!N#(VM6@&&jv3-HbbxXPz-*GuyHZZzLpzSz75mw%!8UgSri z;5wqX^-6LYiTyKyCW1X0#O7@Rk@vu`81pB{sy>Hwx zZ_84li77(x5YNQmp4b54s&}vG=$n>wl367a(D;ht`Hs@F8+4TJE+cfot0U}&iE^2< zA&4T%MR6{&DcWDrHbi9hf^2Twm3{>BUOvr#Hli3*BP9OwKWqbm2@Dwh?Kmk zHsERuQEA9}LJdAsLZ-}b4v}eEK{9d1D<(k_As21~ao9Nr_PTJaoQKs zB`Kta>6~02U8}T;2^=OiYIWl#+s1lA2DvYCLBiU-Cb}+7bd#9sCL?YfqxFP!GlS@o zT>Lt@nC(X|af%vqylnB>HzMElbst@I)4TK3Z^BZa$*bMSP9wFkz#9Zy2Xqz+vJKo~ zwre>3CS2$dz)-sigC5pwa$FMHz}FD3Gbgfz#>*z1^HdO5XEEJJVWEd?WC0VYN;=39 z7lC;DfbCe!6t+kIY0{FsFt!mF=I~@7$uVrv^W4}K8IDO5`fqbl?JW;9kvVG78K?a?$j$up`8f`uN(_3yE+wAi2ITn3 zO<+G6i7Sb(tXu+b1GPr?6KYtDmK&&K#o-Zf5T<34Rh;*Jg47yP#em|$5P^a5NX2jD zhb7cLfV!xo%%Eqf#mKZPZ))Ta4EddXRa~vjZ;LO(&A?hhN$%k*JKcPPvL2(7<9PX6 z5+?F3)z<|Q^=_~h9t^I92Q9{AE@0=^!lS|vwhasWq$mrQ3@al_WNT!Q06MLxDHI>` zPlHhvLNW+TN$Pt!ioh^Is7fsk4O&kJimrimVvD<3Efk7(SReWIM?7$J!7#|MK})kS z1l+jkQd7Xi%RUUFZ3kZsGaK5S*HcP1fDaB${2^VtCytMvn7{X<*Z%zCh0~|MeDT_b z>g1Y|Ls7x36ess>oId@3RP~ zpfc(|a5K63W5VJn#Kz7(FtPgWnJTa$Zh|9WH+5uj<{)_8HgZ~GB{D_Jwf5K z=a3`YY4llRw(rzwbUM9YYj*UqX9h7O*#OdgV@050SdFRS@prB#5aC8mNG_1ZMy#>N z1CbuGobL|#pQA1eB_gj5uA$yd_2+0|V7)a4j#A5n_>J)9+lP*ACgGNwN-bJ4_dDiBB3)2Oe<`rhy zeWz4&^u)2Fh3NvBK78=Nq{^qQXWjhDwIMh|1?zXki_TEb@&c zQ=M~SnMJ63FMZ%R9D{%IgvtGzeSlaUW~S?$_pf2Ad-8{kGZaI5Ham1kl&W^A>KgFF z;xi75ZoBrk`5eKd&yP=roB*>DAd=V$kad!k(9y84#7Yee>q{1QI85|49AYPG+(C>h zvt|qztU=BSR92f27D0R?sap)$=4eZS5&V1jbni$L9?-Hjy>2k#R(@C^1p}s)Q`EqP zJ+e(l8|H!Uj&mQdVNg>yph{3TDHvpt8$%12v>ukCb4*yY=GD3<51gF0WDbm%DOS0u zjft?O*O#s{8>PmB*|64|Ljc@BBfn|}F??x7l`59$`HpS)E~F78lq`K@z;#eB^|kb> z*!*ip3|8dgj5WXd;`HhL_z5+aPbBseS`jv^y1KfbNZ`r7C#FZXsq)^d1$WF6=V7*! znl;f0H;%yIV*h^G7llNXL(1k=P0=!2n-TU}Yq!u^3vIqqFKb?sF~R5 z(T9Nuv86OejUq3>m&DOD?TOl47L{cZ0iIVgEQ1H1C8l6CQe_S~zq7Ovk#&8J@lw@+ zCEzTx>M@i<4p@A4j+(w)XXFSm3yZCiP4D5IL#1T6jbc(IVsj@(VZJAM88BE%QFF27 z{2tcQL7lpBhEOl724I@3k`9ceSrOi~_%?%cvB5z1ufFm1GjEXlFTHsG-7EbN8{U8U zssCDd?dz-8{_@_u2Pd+)aTEr<>@D2xKVv4gYYJO)9 z+`U61^M`~3NB3RQtYA6x9H-#)Hc;ub5q;ruID`8;D!?i}U%nj;>LE#uvJw}Q=>$`U z4h9R@vM5Yyma+TUv@vN*;BBOVw^$i5vlyB|*!GP95qq}Xqd46-vcn8r6+Lq1 zm1W3Om0B?%jNR!?ySuEge^Xeqt&=TEL{okT^!Zhs8z!eMlE%m1gHD*bHyfGo1B0P9~Aw;#t zJkU{fTE9P!o2^NotX=3#>q4WiYiAj^ZPmwkpGviJSMqf@8>TRU*;#~oB!xW8ieTV& zkeX}2%El~u1m|0jfcBmbFI2{pHbGj-l)5rd#`H&7Ffev8E zq>|7=>k$u3fi;c@d+E)Aj&XH(1Ki|DutRPHWiiMkEV2W;0X%t(Lu?^t2t@3bFh#9F z&IlK~uDdKS15-x^)>{RQI>)i`3Cp*w+k-|~Z75Icikg7{Sw;GCU*@Xq5u`&p1YoSR zl8aAH|1dZMSj-Z<4DlSD!351t6|-`Z86)7FZ2~<;EwUM5mBE3wB%f_ZsTm$Z3PCA8 ziNlsCxl3=tV`ywd!fqeV&%@rqljMqh0v z;meBREGrLAVYUQ*)w#v}8Qu157M8dqhm;5@wQ~e*U+YD|5PjBh0cRZ9biL@a08{0Z zY{~{CorCzQE7numWi~Z;PRXP@2VfL)-&^?opRc^Qa`El2-ne=1=l{HN@eeB(Ut78O)XK%* ztz5X!594uV@t;=~Z-QC;mTK+FuPpptzIrpVg^p6kvE9LQgEi)BqtSA?b{B-V5pJeT z=}Ia-Jmw*)BZA-IAUTLIE4(DEIO)S))eyldX(hWQ5F%1N%Ko#$_$UFVswl|FD!0LI z9yIICG9rZWnEBY>yPU=7p*9$`h7FfZ%{H<=4t>o>q!T6|rR5Wed_q^$WS|rhfk^kD z<<aNf zMAnDj&1!y%wD6WnE9n3$^@iQK18_~eR9%;HKgjG-6`L~bnWfjTfz>rDL?OehVlCY? z%eZB8R%K+AGkeyxvDv>{9)9#dAP;|#SrBG6c0~ey%;F5JH786mRjMw1U=}Uc?9!$? zF#PS#Jp3@7YmxM~td~<2J*jAkgrX&VCLHule(U?PHnX6bfXT1ZReZFfVC(<9N#>Sb zi4i~bAPn}5kW$n@IWhDKRIp$K%^_k|BV`X245z!#e%#}WXgBsBuqv9BgcNilQ>^Z=S`i*{b*%|qJ4mcl zQr#}z#@J>#<-5{Oz-Rs>Jhuq`!VbLyWsZG*@I*>YszQmDS&f&-5f1=DsO9hyDj^gb zZrN*Xjf#cr8fjAE&aAZmM)T|1-BPIknZD%2$LJ}Y>`wF0S3WGGdEJHxi&Fb zof`2fWL;Audb>eW57VFN86^jicqhD?hDG__e&f#N+rNhy?9Qdz@7@0W?YD2gbLUe2 zfKjgw-G2SfQ?Tbc{C@iXK6~dP!5whehqqscgPywm=Dh!iJ3r`h1T2H;$NOx~lXv8! zJ5NV8=|9e{8)3hlp&jKxj7>UxX7+Uj?fxsEMI+2dt0 zuszp@ubsfoTZl<4`MJMa_qmfC`}|~FfeNJ%ld*N$qYnx?zCD?oBZ7X{UAkhzeX~k% zxnzQusxcY>3Z}F4o=L_aW-V3yMWA%|8c=$p5|Dg6QVAue?Nt_F^@QiM!h}_U=Pr8r zwI@Xd&mbu!rN?$>9tLChT+7=3_wjSROR=SWR1+aR85KoFwcz~O%V3;5iOK6(q9Jlz z+x4yLv19WY?8*VfW9tln3QF4FtsDdTx&qU zN2e`Xn#H_=;+3dIhQ}u6Cs|!s4jIFH4#B3)9X`kY7kqXaPJY(jzk2BLgYzSXi>Y98 z{NU-+3d|RJGNq~M$H_j}*=JO}BNp6T(mFMQ2!S|{O#6oTR_=77?NNCf;T{_;p=9FI z$gGv7^XoDIMW!=PqEPW$pq;97!$lmdydG2o+C6}(Y4qDBn~h48*$SxuOJZxOl{;8H_8mASr~w^ z8paMus-QBpEI(9K_3@~MX%zCzkW|#%o|0S#-zY2@CZAzeoNg6rKH*>z;Tt~7M5G(_ zER$A}xqvSemirJ+AWvuo&eAdkvi9Q4K*}5`(ZDZ){kz>Vo6dC;w1R@?WlA7uk+Yt~ zhR_|L5@|$i=HL_uer_(Z3CF2xiQz4k?N!u0&Sg7MIT6eDwB$&~{5*u$;QTmXzsJ_x zcK8aE5~^DQyd)~MX?C?aCtzW6kQ$(E&b3|nWAgGwT+5-ynd?|F_2mkdN!@jpOXb=H zkX%^Xp%_}GGk>=?@}iEs%+8EF9UP1Jz6aaaN%Mxc3%=rdU**i3zl4*%2^Wgjag{Xx z1Q+){?D!O}9&g9fxbxLPEfqaDE4a@3;4Fr%QC&I^6NC=U#*tRH!idEb$(L?OS*=mm zY2?YUK})mbMBOHzU-&WkM^P(sS-bZn!VevsKXmZHNrbN&@4+hCsafzIQYl4CYKoeO z;eCoTf*hh|xoyI%mi9HwgifX=lS(!hJHj6NCRtC?LcTdv67>dZ$?qrBuox{j2sb^| zNMuzu$tr4eKfyLNoX}v;eh;s6YM4~~NbrY674)FRm`TR*)L=iEcIBOp9MRX{!#?ug zSZUW}R!Js0=jc5IU2N0B`M-7h-P_mT?BOCV!)A0!mrgL9uhtwoFN%Ph_+UC1$Wu;jaRA;DKdyKORVlm`V?Jl>5p^jhfb8mt3JJ$5}BE!Tg2rK{AMmTP!590LV4VcM@_?!qfL zw$41VSq=)zvbX{6v9Isf2hj;<7X(cy=^j)XNUB>CT-&QJlxs*==~yUFYd?mgikDbX zP^&z2*evlfd|xX&pUq|aq%UsMI3~;$M~AoeylYOwZsJ5THi%&3Ct$`BwaCwa{iv9} z{jm4^8PK!2luu&F+Y0$Cc-U8yWhAybS_wLT!GN_q?dJ9rtbaHFy9H z@lndboq}6$pau>4SSTHAaOr^A!nI1Q-@GiDotmyvqhS*m`VawOi(m}(SA;FO_Bs~K zLDkntv<@Gpr#F3ubS0(5?q2|}t0e_V`_K8BhFdhV^q%SQtVfgyn6YySU)G`R?`A_f zyc*K`X0<(RUL0(mA(JhLJt8&N=A{RU(9?76HgowpoO!i`k__-zM&lYNS&VKh#4%8K zVxYs60|o4gg3I3>$^qlTVh)`CFpVC*sG5r6_DW7uw5;D4K3jw~g{Gv_oeSloh(2$~ zLJ(XQb#@MVu3^DH9e|#1U!VRV2HtlCwFdT+?0TP}i!SyNVONhIICWIb#fZ&1J{*&{+OtAdQ3Dcod$^guj+;T* z4PjAhlFdYq-h#lH!R`cKoA7&S*z~QXI!BP@hW8Gd&o}?nyx#n{`SJ2|Sm$lN+q}B` zbo0jYbN%OuH-Fx|3FqDX1TNu%SJQ<{f{R;z7WREAE7lQ>1QxGR+tZ2~{8{l`R{v&#n!A_QHQ$fQ?0X zMcLbD@Le6y@?2263z3Kuw&9m2%0;n5-sYh7D1g9csTjFLz*;#1RCG%)8uo>9U5d z-Jc=2D?@lMqCWw?TikvIw02@s3!T+WL4csft{Ohlq4=sAtOGZ?l}db&!MLjWIw-u2 zkS5Nd#9W7o>tP{vFqt0_6E!$EbwVT+#U#si4Y zy|0K^eUz*r4#Q#uwTZcQWu_oj>&nWM6sDsKHLF>;;}CggbR_fILIFw#^C^h{X~W*| z3^P2Dv$*zvJCl={?$ju-e^aZblx()s1rAM*zfp-o&FYE@U#(`6N-8_QezUU>(Q)t> zH(&bd&1<->BGvxF<&}kJ$l=j^DxFo+IXwe=-~RIZu=ht#K6xlVd2;^lb3a{K_`$so zF0WktS>F-neD#BCU%q(t?xpWXeh32k2)?A8-~M`TQ}ct*mE8(bXH1+mz>tWTTYXuc z6CMxi+OlLLoacWboX2EmiOot*ccSXuWvS_PfJxlSkHJB3HKHEfPomcBj3p{s(pRw1 zbRSGIB)Uq^s}QxSoz2SjTu7UpbK61v;=141AcqZF{f`iErxzj(6PI1$L!E>O_ zZ8(3D$&)EjV`PGfy1~d1*LKLLFduWsQB$17D>pIh_XjOM7+7!KT`6 zv1AH0(>FHcDY!w%mbcGl)P}V)`FPNf0|yHO>kT<@(4(*#Xp_EY6-}zIs*0bte=yly z9w@nyK9Fb6jFJveIz&Ptdp7Jt0oHMj!%8B8rbSIy7R=}#usb5^4G}eEi&lhR7zToS zpY(SsB=$|m8eT4b&E@3UMBZ{{`Ox!yg=^1dh`A-e~zB^{l&}p-q+8(vFjJ1`wMT~f9cKpzk2?^o_^)) z-@MYcPsbbisWYyb%V^PUb^oU?e0}Np|9bY>ub;W%&rO% zCvf&n`#9TIvr3>8gG%JDf!1@0wngRZkS5iYOnPfDygkXEEf6_jQ`|5!46<(owZ%Dv zG&4dtMQRw)egCc^Kw5X)b5`5*u}CTDK=L_?^K<{T7yfenN;7wY%*&1YIR8;Kt)YT_b^7`xP;tzGd&Yn-qF?^nsswDMJJBQ{s!Q4ML zYNBG%8Mw+iwa$Z|$1wCL3Cm&*m>UxZEtBK06}>6GaGsurSyudQjaro| zv$)Lju8zd^6qybkYoyXjB2X}o!Y^B?(z9T5Xn-lEt!CE$V|!y*J@vTRJ!2{Ic!FL0v9EQ87=$rrt}W0%lY%PJ{tkDP6d zoXrrnElO1b`$P#fio&K9?!;kdP=7q0b z{mq~M`R(ZA!{B6;V*3uGuJ88ySDf%mj+#{TROcv(g6oso?n~DpysryHlZZViU#G=3 zg7T@FFUh8bOvxm%%$-Z^rDS$Y-_>Dc3cS7O(3M`LUy`ypHn$Za?ol?!3{l4&@C>Y3 zZ+d>ZQ5R@AFqu~;Uwg-acVsda%vwHuI!`i^#EE?V^p8kpP`-Rfc-D>!a!`me+$4Cm z_rddDiJoekt}?{ZpYDj=r~$W zD%pUFUFe2eShc1tTIMCb^d33mvN=g;LrZYZwCpvj8x&Tjiff@OXOQXhaP;g>hX#>- zQSY>QEylbid4Ucw1Ll+2hR@4Lbzg8&n_COpH}}92aljH5+yt!DJxHiTD%%bof#*myIv5KN0KerZDCA$Yz$xZKmT1f`9ruXK%F!P-aUuue)q+S(8^kMA>I`^eh zZ3D~K;l@`A5u349bRSZ8-3ITlRJ9s5wJHsZ@1(yCom;eo-a3OP z6dph1nuO)*6f5quCm5@wz3zIV(|{F>(|Y z$PFE#rc14cmr4xvAypeXQE~*VpbUHV(QU2o)P zujf!DcnozVn~1H?JA!zm5{@qi91T)}0cP3K&%!y~>pk54G`5?}9%?R;Ajgj(S*5Vb zWX@J3t*;r2yL`NC(m79pU<(FIb>rO(%V>MR;5BYivn((fgPJ@Nc(Zhc!V_wj;1T6; zAabKI!fc0@!9+;cZDXbATcogbjp7kqK7Nv!?7T3jDddN#{4#jn!F^f(MH0n?Kv{>a zV@K~Z5Yu{3;J_Ru2WICW8jtP*SZBF;WD8IVcqTGRF3{>1e&t`SFuHXVy1>y4K`N8+ z12@6Ywu2x00)2)~xCDhJD6cBm^t%wf;z9|Pn zrGhidO?y8XIdB4x9X>oYVwbw5iw+KNlrB07XHyonZOIq?b!QxIm4%ckq|~}mk!_t! zE1Iq=;F2q9R`aFPeI1B;*cQYTf;ZFv1@>M*OUaCDmw1Vrr5yw}r6f=z*Et}!kEeI4 zVU`=B#4Kv}89lG>%)#dqv(dY%11!BUfS;+nrYfma?0$-I0SntHMh&OhAcOz2^%CO4 zZR(8ZneS0))E%;KQo!og0*;qh3`E3?24CRja!LPE*L|Ve)F^4B3)w?LfSX-`sjdj$ zf~DOVVZ=z@^$H6CX2~gu+}>IgT#M@9wDYoS?Bq(Mt4boi)J*^{w(BGAPSz=IGMf=_ zDrYbNiIp0eu^qSve&Ng*!8Q1P=>T*yG-n^!;fqgR{^G`!m5Xn$EL>Sxyt%UYj{N!k z4GOVGCIX)I5n89agADB6&wnj1Nm*B!otdhmsb{^UAg$$%Hq#j|1bR3 zyU=e$HXFijwUmOPk&iBZP)((}R|r$7P0feyd+eNQ7sV(wnbZ@TDMpSlWEBj=6-SlIVhy~dPHCF=(OMT-3`B~ z*>hd$hM~>=zya3#iM$|8mldY-PaKWipN`K}NSVdE{7NO$G-=HoNWon!N5Bkl4~-QsE@B zCACCCQIY=-dnXeETZ_Y!Fad8#3k(5kC+2X{G{^)k&EoLtJ2!R=D@5b2?Z6UFrcR!S zKDt=NwISf!qBT;Ql1O$26QQ+NnlCgzU0%qO=GErA%L~mVFe$&Kb8n>Us(!1lZF z&L_=JmY->UNMQHP=AYmL7nT>u@+tng;q{H>i}>~U@!cL!HeZID@&586%DA*20{`ATqA78{Em32rFg&7V zqKBBQ#w{_oZXt{j*C5%!mRN!go8;_TZ*kv^Oo?7R#2RtRKUzAW=F`b+J_~m>e;}Ph zoO~gXFX#m|Kb+CV@*^V|zf!5*oh@G9ge*RV@(t+#^nPR5?yqnruq5YeGxA7~r2uY&y~k#Ug8vNkO}? zU=JKW(mAZ)RLBt+hOE&ZOzDi0)s=Jtvp#LxL?-4+V`+X183be|FT15xE!%kAy5rP_ zsOs|zzd-m*#bQi0?$+eq$JX}k(;k0Vx_Bk7f{ZFCF;q69cWZCj!8_Q!!(fvJgEVS| zs{wY*NScMe2H*82>J=_OyTQPm3~CxrB6M?i2pTtk1J3Io;UB6JzPq0^f6{ypoqGar zpqKt0`03Xn0JyRIz37?VOcup86dif6+lJD7M8TmaQdm4Q$w!>poLR`L&uT@=%s8o> z-f+sMu?C+AD1APPwh-qUuI28aAyP^rw-t4bDYtI18Y2dyWoDuKiXh(6iOHjgEAxCs zTeue1DY!W!uyx}*1NOR4eP^K0{6b`w!Atc{-^Q26h2>PqQDMbs#UA6Pi7!1Nqg6y_ zsy)fc*C7p>Rn%nc{`iLuNKrx3F)X|-8_O_&AT)OL;Rqq z#z>o}IaSe;r~w5}v0nztrvnvI2r7-b0DIUKA9lr*z!fC4jJ4i$L#;%o!%SnjUDpC~ zX;#=o^Q+B|njbf>F25xIU6<)%>0Dk0|MSN3B?6nDEH8pLeF2WR49DP3I6@|izeV8V z=&jxa*K`40QrNyuWTF~&!B<@Gt9h>ZOE_uFwc~YM#k}~Z&G%skxZm=2JdHbF9n>P* zyWl!vy`ZIEonsxWPu&FAm&QV>p6N7q6fVHK&n3Gv8|L&TxxntI zBh+Nv5`om-D(d93S2hwK+)j5PjAb_EBv`#3uO+bEraE*?RE8w?f}f6|OB!%8kOmCg zz_og|s-$%MUsHYQg=D&2L6xt=Sv8e_m93LFtQuCV3oLo-^;^HV_2#Xg+8E(WaMz>-w+G;YWN1sc@mWe8fdIJt}_7iGYjFz$Cz09#q+`yM))l&Nj zvDgeZ5lDvzn`Ht6k5v5N$%m!Zafn$0u4dYmcQdM057pqye8BchJBH4LjG;4FeAMD+ zlOERjlA9&T;ZkyUZpzHa@}t)2M9zc_q%n2txlz^zsJVf$pw~ENQM2?QK#h@;OtCaV zGZ{t2Ze1;>_+b>PFSJ#|3QvLoQc5<{=}bfs?3rYFa%2+o(+U{d;*0Za8x4Dyp80Y_d^ag~9#nf5Mlp^)^CUDWxP+ondX165+5VXt{)hAbpp_lj(+4S?YZ-otfN@ z-Xwhz?vpiT(pjdae9KN`NNxtJds#hp{QA(P^6J!@+uXfrrm4I!Mjh8;gBFrG#Rr}pq8_DNm*n1GvFbs1 z=Ljv$PWV@*q;F{oNh=Cgg;_5bde$2lhqJStKyG%zpbZJ4hA**b2|J*6Gx#!#N{Abe zyCZ{eE^q}WdC7rToSdBMl6@U1bY@>ir~!l2GX02VL^CV7q@wA*jxZmpE3}!iUBntf z4?gECqqFv|A7ksSfY3kG4)&OsqGx+7VV5jk+*-p>CZ7cRr8^sj%3-tJ6QhH@oZYr1 zA9KVtxN8$^0z4o2woe&LAI#R^M)_`gdty>U&r8XSl8R3!-mP0bo5<;#b*uaF()0d} z&R+F(2ew+(`)q7MTqOCf#~X;Ch9F18P!65o5N@5H^_6W0yK}v6LiL*QWS?Vqxo$$) zIL>W*wLL}r6@JE%)_P4PJ!8iw9|}gs_QkfJ0^bc7gPUl0;&G@xvhEHiM8pM+(p&Si+y z!WSQ7CO64ZQL{x&7~U<3J{;j*t*d(_+X<3n==4Y`t)w%Z3-TVG?SbOia?x$wiD!$P ze97h$2_+p++1U`vHZTGf#+u}6YiDO{c=yr=Uwr)24Z^`gM+4#DApujvQ6Y0RJKyzl z_tNWMzIgS^mw)}`@1m>SY6)FYv)IfE-mH%~Fq!n_^h28k;ca_0%C!a2qP1Gj6hET( zlw^@?T-RbF0c2gQ;u_SF)qRIClT_h;vYJd3^<4^iwO3>U{%C^?^9ts}VGXlN?O@d~0>27=I= z=#cs>>aAx20N>D|;+(4JX);Kp2Wq>v!yUIFqe0aEt{V{Haj+#~Y?j`Hg^a?nFio!R z&mW^^Ww4A4{X;Nvk@Fos>e@2)gySajBd0J^fPZ$k`=A|DsR5(Y^kfab?N)?s(76~B z>p1?!^`vBx339UtHqZin#(P3$A%x6^nN`HR`-9Xj8|+Rpw+Y2u7NH2aOzZ?t&qi&3 z4M}YJx9qa8W_mCVV}$wHyrwDXbZq+YM|vRgqQQj}1VKHm2RR<~M9!zZAU8rG3( zpt?t^FbjQ8 zrV{uk$q|9Oj^HF+TI=BHp9p69BsplnR}%QEGld_7WX8&0lEYR;!4y0SI&B? z7BYhyb+=l#{pPEZuiOl*ZD2DQDs`kjmKwLZmA=@B@q7g~Nyue(GLvL-Ba^U62FT<> z0we^IOx3#fcljsG&QCKQAQ?#Qzwo{9Y1?W`veMlawgs*PTiv=?r{6y3yqD*Fo_K@r zv*oh3?$WV2_@@xQK+z_TH7*G;F)|Ut!{O7YE~=TS8^*Mg;!5NVVCQi&pcU@VoFVr< zx_1Bc*$i1Y`TVwR+y3$8wR@4xmE`Nvl$@!;B}`+s=)-V0~$z3@J~ zYT@Jy_nvbdy(fe<|PVQa0c>kGma0t)ByYBtw>3>{($ve6D zg15iE_sT2xUw@z6|LpX=(=Q=3pl5{q@Y}QZ-#eY2n@?qhlQ4p;%T-CRlI4O|(L(@gNeMwZVI%_cMlAr;KnGS@gElSPxL z*rsZj4kx3eGl8I&t>!Jwgk7Y>@P=y_3sZ!M4D5jgxCUnRt;yATa+P76dTmM}BfyEA zYZb!7q1!aAP%pcI*EzVJFSBhF4ziel@okY>=+q79u(jul%<`g}hpn+#MDPy;Xrdwu za;gncNjjGFxi0~$ZE2+Bme-_UdeF`3THaGy<6s>4f{1{&7s-c_EN0XDnDxZaOJl)0 zF@mFb4TBt^dIX@@Ahnp|%lgVbIXwP2prO)2LaFa@T|cl`FST*e7< z&`MN=2R16_sb)7bx|!asYdMR*uB_>-yV6~PJp_D#Zp%DIg=6KyFS-QSS1pbtV+L93 zhQN)rc6Z=%mf6MBNv(kSgVwTIMje@f#ad@|vW|weo48uh*4c<})Pi6P-_OYpM zdb_pE(PUNFfv~TJI;LfB!<-{5(m9h_>dxIpUQ@AIEPPhZE>8MOFStVBr)9z$FWB%iK3jE zTKrv)+*o>g8L}}vkrNivpQA^nMvrtJpUa3=D7l781}x?_Q4IYNoA<+oMgdJ@KwA;` z`p7O<>||29rPXC2p04*R0`v6_&t5F(FetlyAgcAjz)*`3b=#p6vlqheRr`BVK8e=a zR0}@Sl7h4*dAt6ld=RETmb1z}@#6t%Eo#4Q_wuyMAn?aJPVRV+PU^5248zU}LGpf{ z6oa83q}F(ZywRjuiN*Y4)jqB2zy)w6EyNlt1VMd_ya{psu} zyG)Ic`yV!KYH5Z~;8sfdaE5_QAjUE=QHYCMghM_L9QZrJ&?JhBUoX?n@)iFr_%-3L}aD>5XhRrP^`McfV24HM_Xn z_!-(5QX&@1$f;N+6;paLvLwecJ$+cxO3Rsqlr$j0^2FI7DjST_-Z=Q z)!bpJoP;&SQ*mED+VQXmn(QsQb}?s~WV@J7DkR#BY^mTL7Geh`7eyf*k32Pb_F{lr zm#C(T2zHBk!ZjY!y~i>q4Kj-4zwm7#6hU}e!2WWug(|f|87E#FZGJ`eo|jga8FyOV^4F5@V7-LnQ^vQK3mwf-J9P%iN{e zoTWPIYMixsUyAMO4quOb#}CblybLztu^u8OMvE|o;U?*!*tcRhi*SOCU8AL#pfn(7 zL%C0-GEQfYWMYv+vRt&N&B!mhP|S&06BM!S9;Yp_-8?J+HW@IF|DvlCJgw0c(|mL> zod<4X(Hsj8HQL3j+z6XoQNR&XX&>H*9|SZWtj!JVRAoJ&@pM`-UD#unJ~o#8+sQww zWZ&p886Mq7>>?uN3KZ6CW@($=Onnu5+_jHUvN-sP?f3{!n?0MkdG_Xiq97>Ys-SP& zeD~%nH(!FsuZOm3t4~_0c&BAw9|}JRXk=tTjjVCM%XBB0MN6b?nK@+LvWY>->~$Uf zoR3t45k5s#+^NUtiP;akb!zV*$H2BG$z!()EYS{9$(*1|j31PgXU`QhgEk9f*Hsud zbOUNH#nl%s#E@%f;Th+}lqg7|EXXK&=0ggJK585>ATthFSQdqt95^)U8bN~{q^{ne ziLu)tI?PuWli6gl$SeqH)ni_35w?+8C93NcG6t8;oP=Z7SWPoyv&kB-G~YqfcFLx$ z@q2c8gpFqG8addt(u`drw07;5teH#;>9mkU$t+(gqBdLOf#<3m7v$)KsSiUB%BTq; z8U!rGjp73UE5u`hlnBfR&^5wLSFfK+@S!ns|Mb~=zkc=p7iY0m{r(?*d+#@I+<)_x z`|qCJ4*!%N-hA<~Ri@A0#_2Q6${wN~yIM_)Z4M@zRY%xlGWit2>#p3=L_#4#B_W71 z{*L?CMc=X**ov}{ZU76dA25D|>JtcoV`K!A(b$)>>9Y!T723uTgr1UZK*pmfld!?n zyD)*tiRM3;{{ne?<}ZSs=0BXjdh3~6&)+(gnZI!B^!!KgaAyAe(W6M`bAJ9ZJpal3 zr^xyPFTQmKpS|@wyz=?^PvPJB%o5Gft!MBfJ=UXBx6aIecI)}67SC<|=(BcVYIlI| zV!^mB!}a-S{!`+=ZlQ08R=h`2ZLyYuFV<;6X>fkR0A_lJcS+e!2qf!R-hcPvcb8te z{hKfG```Y<3wK|+di(jeZa;tFyKnw@_eOX^s$r9=BSmwvW{yNHOP>gwrdMJ@Dtc9T zod}Qn4Oq+=(ZJKyy~SK74m<9coyCIOZ#%Va&wZNV6lr)d@8*DWw>n@#$(MV#ADcJk!N zSo=6*)+Nt6!jM?9k5*8F*%)Igvrx*%3P0Tg%iEQQ9OK z+v1rBy)depN%kFcReM|`6C2dBfab$2Ph?(3?5Q&y*rfJDY=O-S**vcX9I=?~ATtr! zj(_oQCtohZ8I*KiC8FHSE8;$;ZR;bWv>YCEz0cKbf&#Dph~J+J4;W&I90t)A?B3pj zWN1EWxNAgg-CUc-(IU$fCbx%K$(&#+*cet=Ozt4Wm~-sGT}8M0jz?;z9tRvWGWTk z(h&D7JnZIB*Js45I)$ucG)jkyt87dy_azCIwM2b_R0|5nz`iFPd=unbf~!FGnW%|A z2ri0c;7dr>VsxX|y3lY6Zxn^vC@78sM@Y$jt-!1(+HRV~-`~sf{^&4#`AN*k@>*!@ zwB^z_;YyXo9Z`nFI2xm&E~H$KRtJH zZkl*mTljYsp83g7e!^v;=bo!x~OHsH8OU!}*G~okXM5xAs3F3kn z3tU#{8leW+W7@M9>)|uIR{_TKHUeXMO^8~^Fx-Bd0ui$$Doky7i(!^343O>T zs!zGpa!|Lp`5tv>q+yRbgzdp9Qe%3&j1PXBQetB3Tt~S~Ne_da6ZZaQ@;6<*qxE}J zuL)N!4@P#K+Kwbkf)s-dDJH^iA0>$BYnvB?_RWikONccvUk08>(aJliIKj)pi#n-z zH-G$)6mqb-vLE7VjD&|7_EMQz_*vMpMiaqNSJFZ%9=u`U2>P~t3@g1TShHeE9zlhZ z-P=k;Ghj+$@qtSkVKtmud1M&o%TZpZw!MQOq?^CgqAAf*LSNeR9l6h*1|{rqtdlS+tlXf$bHi0-ZF`OCMS zCbynJh0XaZ1aYPaxy_JaTHf@#V(n{t0N_Os@vWz8wp#TinFB6WH3s7>sc>(GPx#lr zeL1J%tGOft&P*_4Qb;zKF)(Pp--e0jXci@~+-N206p-=5esY9a7S(mrC~yvOq=SCS zoxMQxZFHF}R$UW=IDAx*5@zQ(H8~HSmZ`~^PHa|~{TdUCC0N4T3~53!{;%*N)SzCX zA?M8E@bOcNpHn*nf34^^<;>2VlarGI3>_}ZxL~l&@4nT^pPF`_N0~M2gueY&=O9;_ zi`XiwtJvc0hZ{qEJQOB5K7i2~ryF-rn1I2pxPA1znj=<1m+uD zDs!XD6%jC_uAx#ZM~$^pv5k!}c0_ZEL-2Oy_;p)+1lIx4_Y#tAq?Gz7^wH`c)ZjVS zCZ=65%}5Mj?dTzd|C55OM5bnYi137FsX|GfmdfCpdt6-`11FS_sv@pL4lTEUVQY;< zO(YK(1W$v4BeuQBIbbwugy)%R3KCm|_%*dU@hjdEC{Pv|E9e=oe`M-!&%AW>Xs^Y% zQF;1+n|#~PenxuT99S_!vfy)?W$Z1$p7VVWobo_)1a%Tvp+exLtxt<0ab*i)_s5ihYFm&+?4zg3RG7ljRW5?Nlc9$~;&LsBSY z6x2;~9R3CbzuO+r*85* z(osUFN4yb<8cNU!0XUva&;S@XkFdI(UiX?k1^&Ku5pY*pZCqVST9J zf}2AzNBAt$YNHg)daM+KE-lSo$ChKVeFV01-K62yh~>-W+cd(m8%Zbw11s$#?@vL{ zjB&K#Ajn(WIHyXE3rQ(5XEfG%a6Qd?OElu!an;CGh_zV%+015XlKbmHr)cL=!P&IR za8WS*SwjAnohq$#$HMQeb&giTxAb`j0Zo+@(#jV6IvfDiG)^$fDN=}EdYGk0z|Xe@ zdo5++5?ud!>pim3i;CzbBfo*YP!cM-G{I_Y&8L=799l3*7Op|~idVbl5vJ1{Hmnfc zpFaeT@%;8AQQqb>MBOt9SOYrJ6Zfj{@jL_>Z*;(7G$z2vW;hFtuP2*&G-(EaYp9 zWDT1(Wy$aQ_r}uy(sl4r*olca*8BxY@|i_U_#1E&K|63I&LI(V^gzc-t$(Gl$4 zy$XhK6Ibi^XM6i_2*^>*#S?qoa(HLk(j3#Uhc!$e`BEh5=G_YE7TpRGS7~Wz!U~r& zE79>)81XkpmHkJJ;yQVY=Es>u24Fv=cKGx{9J_jictTKo6Qx^Hni@n>S{fuRBw}l! z!RH#i7Bw*c(|MH_ydLC;Y3Ztu1EIG_OJ!_kDr)5j(xN5oV?Lo#-B%F{4Qi1yF-b_K z8`P7xJhUJ6&^XZ9GS~)1J2G-VBpMdEz|ju61I54IxCl3*QT*%mkB!coRehFeFq>?r zdd@A8{9kXJM`l}_g-^S|*De;RMi$#Ym(b~nS`vXj4rrIOkV-_(pL$|^nhMS*W=EGY zqAFXp6%y%qW%NP>amx3&z7f*@GE{9APG=Ii=1|Plr4vACDWYn zl>Y6qTw*Orkb3@yWJ(lMsZ>klVNsG$D_oF7MM$NmRvy;7K6Ni%WPt$tmo&SK)n}DA z1#Wuxw`Cs_K3c@wz)Us^7A+SmJBL^)7lq$y(1Z|`BUXG4P_lV^nQVD)$T2}|r0m?9 z)ju+;HA+St?AY3{%-YB^-ClMC!9Cs&)Cae z{k(d;dbOhr5wL+>3T>r^_VoxHM=}z3B;>V#qrqcR-MoXVAK`x;7meM<1nMFWapgXm zf6CQRbO~HjTskd=R|~PKeO#V3uWR{n@VHb|iH3`2b=WHEjiMJv5!#4**~A!~sM#>? z;}+wO16naDNa^~i7Pt!&ta$6``H$znoIgMRA({UWetmZ9`T0-a-^I?`)D}A}Kf5(E z|2c2Fod00{GW*-t1dnf*cO_{e&!6E-Ke*s>723`*@3vIzx!Y18fb8vb~Q>KMf z^a)u~bB49Hwx2Dui)feQZzhPexQO@15FZI`BDdG~W8G#)}PILlo8~vb}7= zrUE-olzy&*6s}Cop1z=72^bs3Q6@!dL|O*5r&lWmlZw18pjEy!oe(4`El4q#^HOSR z{*o(SG#E)aa3RGVHEP(RdG?&|vJnocu0Q zqM(OamOoRfG9SqSF(@nQeLp(%kJcogyvJQuvV4&`MNCU3Q-- zq?iwI8w(hc#1X_HMJGl>5f3iXi^vpi3~;i+DsEXtl_u^Cq5ETz3Zqv0`rOvXK!cj4 z+uMletMRxP@g$2Chww4vxG4Dqh(klz;u#Sf+XniLEl%By=hRJvPYbn$Q#X!`5=wBE z4{Uhh`X3`@3tXI)R9zO|WSbagbKroLSiB`3O0!g?7czH{gUm4OiatS&X1wHG2OD^| z>fW$uA#@Jn;25MJcvzc_Qi38!USb=3*@Mh1>&)vpf84ajw;a|VaaiS=&E&vFlSDyo zSZl-iD2tkJ-p&CdgJW#HgG~}^6`0^zr^fGgS?GYJV!t=U!YZtMPK1)hZbM~ba6t+* zr>~&>mwY+#fCR~Lj-zSF!L6f&c+oU@i!yb%=9ZXBC&-A44Q=FrUMZEeyxn0j|EOjm z=M&ju=4}|m?8kf%MBN74hUOyt1_#-OoIv;*NF4bZJ7&qt%HbK>ztO_(<*LDm75X%JAWef#aR=P+uGz+J+0SF=mnnk*5Ifm>O>cLk2+z!KHB zdEvblv|T0D&&k}gUQ{;sBB{Pwz0Awkmq_&-#4caLzi+%SZiu~djZ$H%@rp3o*ljRpz)aycoaR`mQeZ+vKJNLh$Af_cNhm1x<4x1wJ-3_~|P zdMU>w+5NkD!U0WS8qIi+H`iQi%|5@(B^#)+}QZ&!A4(qGAq z`+WU)aA!Z>w6}Q+xzuVp9;7&nELR|XuEBTGJQWv z3p;Fba&oHvM18HH;SI~ZXAm7Yf1wWFb z9O8=Py|CpqNlWf;(3)Bi#?`0_l4$`IrWC)JM_B5-I2PFB>9xpk!!Q`EZBaGbEoQr= z*^6A%!&fMjuvO% zHHu_HoBeP_e!q4$Gr~urId=HU*^0MD#~3NdD^g_VRc(%ANJ=SJv!nc2H~Z~NT1sSThPu?8cE)5<)^G6-=$Yi=MEC%4;&AAp z& zh9GAi_gJNOV55JYc)|pZ*2dYiKIY8kbi`nX&skKC``eC1xjBImoNs9!MUbNFExU$_ zR03Wn4@5}j7|HYeLkaPZ8r63oNT7oG;0p3EfO9pbSzaj@4^lGRq^UTu83(|?DT|RD z16wjrEP^8qh=WoHI&l~+2?RYfoza2|pJnQZ6QW}X?`r(K4!aRXUEQYSfTp92>vl1n zi1WdO_c$L1zoi6b@JXMEqAzk51INx%Qb?!Q(t(NjW3K+uajWc5h3wPD7(xsVSQr3= zHp5c8)9w=-4dq)$Ys+X*L&zdb)5y?*0Zc# zNy_6PMDz6aAHZokIJEhTaP=*D;J)i5JI-6wLYqxW#A4vN!$t3jNm3>)ioKZuMHV8R6WHT5PeuOW_8Yj=Fq3!aG3}eR#MAW3C}5d_4vJ5_jMVx57~?4ZR1+F{)=dWz+CE1xbHt2-;{!~>mG)Im18+S_N=Qgs@^Rr*$u+t`9G8hqOcWrNk2GlC2*Rkzm(6w-!mYE;7Be#4U00SxMMB0ZnsfFcN3gL z0dT^!ok>dm`^`yM5hg^sLXT%HA>QrvE0_g zX=Sy;6oNFBh6k#&DOQkVA)SgGfA3=mPmstYwG0mqA)rlMo!a9X$>yobN)>OLYixQ- zZ2tXed2e3n_7=} zu=~>=-GB2U4B>}o-o1D3-J0#{p2%>W8(6ClKM)UXI`4`tUa)^$dg=c8lLW4%f9X_w z@f|-1W=~lZ5)C?|TO@58PtukSUlVG=(dKJd2->fZK#WUcrEdK&F1vFGhdt3@e$e8# z-VjVn0#!uQ4a;(1RiM9L`}Ox{Pk#5uQ+J>H>h_rrZh!sdcOU-w`|0;?pM2r=rQh6n z_1f)^U%T`Ar{BN*hwo2ayZ!5v-+g@P_O*}y_v;t$zH;Ht3m<*=#yhCabK%B!=ij?? z;f=fRU%!3nv)k{#yXn27Ms)V=uddwr{iWN_zkB=bbGP3=55o;@&^xbxaQoL^-g)QK z+rNITX4QBNfgKNsz)oZkO_Pkwe{*p5TtPFa*YTwlG!96k`*48yG8+C}E;;tDT&kw66# zQHgATr^-)l`pdo~>)@Xci$V_w5U!Xm88^&HVjv=)dO6+f2@Q7UVR|e)C}{u01YAiU zJY9;Ve7UZ`Zf86#q~*wXv8)BtdjjKi$wxVa4f~Es{0ZqkW*C}HdMtC&*y+8C5jnv_ zc&|FLnH`Ll!MlHGq0P9@oMcv>+Kkv*p+wPP+r4}Dn$|i+7GjOc%wZ_>mc%h5lQejQ z6+}hS8pf#_yffSE#dACAPw*F&>drkRv<^Ix%Iv&JMjWjiKBL-_&7tPa1a@W=h$>={ zxda}z`fizyYq&dVuy}1?fftc6V*RXrgpFcNxcbHw+>;#nl8W3Kdt@<--Kv-|NQuDCO;?X=89HZ8 zTCpD}iO}78#`W!kbjHv?+qKCHCLNuZGMWwqwJVt(Tg0o9xT(?^Oeacv zY%_(z)G7*vsTL%{NyJISqEo3nR9>T2q9kux8oa;_k8y9G<0v74W&NyNznaZ}3m?S! zRFRN2>%dp0ou(!+0KPKPi7kHr2;3{}sO2Oxa!e2#a%0+R{mJ#FgG8O;D&ZFBvG_e{p3A8#DLxrk`_l+7|C{6wcSxYjeLL|MjQp zTQIcib2D?N{_6%oY!0d3fTL&U&Ts$^4Du2l;uYwgJB_2}*dGM?T=Nt_s@FZJ&q)tL zRQ)Ej(Fd)=HG@ZxP)j0(K_V6zf4$Eva5~ZvR(4n^$1Ea>J9vi(*)9zdONon*ktgy+ z*BFOqFeM_&n?5nnl7;m!G9|!vBMLFnuWAlD!8#dbw6wEU%GqARU58qg#|T|wZOv6+ zDWiNdg1H{r(nk*RKe@tXjz9`rKEi$K0Th{UDLDw$-ET6yr_w&yVK#%#M3MG|#nQp1}3 zsQ1g~$;nBdFUJpp-3bc9oa^s|RXM*_L1N8bh6%9ISggCs(@;WmT7 z8OFV#i?O5#azpiU{pwbO%tU>%-McYww!6eE@V+~&cr~cO5HuWux(!D#d%@AdLy6l_ zyaH>c1f@YME)=E0^q0suFnwuZ=1&`)@rZy7IJBD#3?JA(s`ZlZuD*Zw+i&({5Bi{` zffJETHz(M*nPdJE-%{(9L_^(-c_Igf7E-@I{ehR*nVUoSnf}8VLXoEI|!ZmmB#n#qL zf-||5YgzK}Z4F)%sg0t2{=Fg!u?BYCFa@wZW`U|WK}a56uEKjh%rGeu6nyV>BKvNa&`HZk zm{%|p@l+-y#bOybrNlA=Vl0*E@9$4$`o$h46HoW_gvZ`AQi7*_h#USm;Jtw8=E&?D z4{iPl8&T0m->~dc8rj;Dv3fYcs*I{#Bg`}kFtTng=~gq*{h2e1#Htzc%U}NR!bx)f z^!xWde0q^=_5Q18?oD6$)vw6^`KRt~a{slD{&DTn{nuV1@Iw4`@5Pfpyn1%UTac0U z{vUpO|MX`+y!!h6)7S35`HDcgyZT+=uPi^{wf~ezkghMZZ~?>P*jvI_LTgl~Ipwmr z7IN1WJSZKOj~hm6*Am=kHXWzBIT60JuIXxT5obm$;qzvEWTJr`=(r@r<+T{1zr-xQ zagG$>rieTVdriY;PazS8Ug5LQf8wE;hIozKHnT}an0?6Xn_P?F_`#y4Q}|ouq6VRD z-qP~p%t0Tihv_9`0_p7~<)Z1dPcvklGUbbu(KXUFM0aA@#v5Yzs$hwVlFwGL8f6fF z9L!!yOh`5a9AWG=1mB(MwWF&`K5gX{pfikl^Lgw&(PXjr21frQV zXJl0(x5I)>V1ZlQ!pIoSV;`v2-py=XRM8MbQObO3iYS+K9F|!SY3B zR5q#%h89tTL<7Qo%nGc|3ZsF!s=7qinBjv<4`@2e95gNd{F5qsazAy7uzIm4`&iz@ z`%^epE=>LCD{wyBZnU<%^MPm$ZB$aT!Dn=$*4&GCAoN}hK zDhW&1mn>c1@E9NOV)r2_*SNpgy<2YHTP?5NTaEW#T1ZCU)PrzbY<2@h=ClC&!7~q% z#Ud>92ka?|LMj;8g9!*8{fZk`OXL5EX=&RFnw-e=JHw!X<#Tx z77~s6=V9m)yq%!Oay9UY#I%&gBnp!BlU`U$;ZsXx4BRnYR5C5bCG0!QlntXA&~I&wV?w$??IhIhXfZlIMr{WP@J6+4eYX!QGq4Bw}#WCd7C$ljuX-XJ4#8qh$LMnQTAY$f@3B&l)$x zcw9)wBa_nBa}B|t-}E-x92`SF?r%9D`8QfCA*7-gxt1Pnb_+$Ihume_Wi8LqFg+Aq z>tWL^gr9Rgj9QxC$&`G=zfwTvU$hfMa*4=P{SOSfu`YECxQ5{;&mgtRJPYX0PBu?Y zwz}>LGN;w&Cf6%p(orsF^(=hpD=>;TFza!RGgE!*QK(lhleyEJ;fm9!w;>zib4FK+ z0JXFwtaVELTDN|LF~f}NB%7z|?1hrXyPtbq-El2S4pSXwCM*-Q!w6)83@Nm=+rTFy ziJ4eTh{P62!(Da?tg4{`{?H=0ChO4Zly5 z`SbIi!y0<)X*l=gt?66Ol3Qo~^5Xxz4iDh@>G==mufj_{{mYAF{sO$@)A`TH{D010 zhV$p=Kbt?lZZ)EIO-}!p4{gI;?lH}0zGTv8K7J6)d}2(9C$`G45z4SWIM#zK#imo~ zTI}E*1hi~iP#UP@x188!PV8%3`e(78$8~`|_4vt_@%(#L0D_okq11fsf2-5di-Oi?FNs zD5N5Vf=F{It)*qAhvvueo`O}m$9@&>l2OixXe&zKW;~k zEOa*w*b~`CQ;~t14-s}nr3MzcDaj)?+JafuKJG)=rzjA3rp4oe)Zo3Xy6|=NLj370 zsE7r?GVBVBc7=~~f**HvesmNX?%mqH~bwzAOU;3AllTvM7GhivB@ z0!>!JuKjzuRvM@az8Kj9W?`7HT{Ki8DWqf}5fdcY4^QxQ-us<|c(SFblHeso);{N| zyJCt{9GaHJ9LMC|LWk{D1Dak!dBrha1%PE7)z>K3TMQo-pMbpz4E`@*>;lqvabJDW zMyZ;=oeJ!4D`@C?69vvF=a8z`g232vG?m%}-mpj@s}!FYaPhCfE8?=ODDiXx|0?VvI%_8r_d4Ar zt~&dnk=IPZ$Ov<7k}YWkGyFPR8~yUkZJs!Zw@jQUNHEp_P7B50+^=gT%|K2M@{6t> z3bEyr5Gb2Z+OG9v-qg)fPNPraAI2W0Uvvqi2PQd%S2Ls5z+|qafr(Sn{aE7?<-B1Y z-6G7Y@f{G_>n6p1z#=@CqbezSP1u`0M_addmu4?o$R#}@kgO1X!)|9IlgJCq4dKe8 z?N7OR*UD)QFS2c*Kc%FFbYd;l;1%$hVOBCHz(Z15*iiA=Qp=tW4oodES7Q1UIo?-IyOHgp7*~|`P1E(bBYMmn z85xJ`6Z#J5p^j6jqjyhWVn+~z8QyJ5k zq1J2=me&uizbFZELytvZuLEYyzqxMx!H4_FjeZvgUm_NqZM`#{y>NQ%6r5p zL}FuZc)P&2FBYkW6nDPdfD)^%X#8tm|_#FwjdJf_-u)oca<+j`*DF7v8w%eiPnn(ALHbAz|TLQUcGVt|3gRF}}C1 zm{9}FcC?8W)LmV@*@mokFEywdj&=J%&8yucTOwK4MmYkl8CIn0-+JrQ*#G9T9GB!- zG)6Keq(mX52x2O^Cig)?_bqaZ<|7UJgKKU-P{d!&9}nz}%d()vw@{ps%xQ3akZg(Z zl;Q)0a{6#X>7c+NFm6JSqW4yYko`PREZNv8@GwYzE{$bb?vrzah7?8n5xGw(k&u0W zibUW5K~BLMzQvWalsYcB2Gx#(weq~vxm`?2HCIIp?5o5j@Kx5*bUp$uGh6|4v5lFj z9`VZZ`#l-;Wq4SJ#Ri8pHreeIEz>O&iA5u{qwi%CV|2oArG(E+f&G`bA|#?yOzcPU zKUnQOhS4H*kOq*G7WP_N3BsHsFtN%$8bXvaFs(BDDKIFaeJ3?tCpktIzb)41Y_1ld z(u!Ahk-mQ(P&WUBY`K(Tzn@-_#L%|xsbh5!MW3ztK|o_C!%ziw?WH;`(G#Y* zB7%70iGJAIi%o5@N$@bZfvi6FM&OI+;Eymerc*&${ z{rZgGSAnzNeRAQuPp;!L&)t3V^xfyq+&TM^oD`Fy7>ob!*Kgi_|M%a0`tjed{rdLv z@813Vwe@jk+O^Prcjelh*I)kr}IMdx6E71ZfAz>T5l&-Iz z2F|*`W?22e>;Hy-H?Sd`lmA`yaxQbfAzn5Ho5NQL=!XwAOv_>PUpGfU->=y@3psql z^W^X;n1U&o6htwxMk^%*xj~XVcmw9g^OtU&BJ-E$Kl{sz^IvUxe*_yv|L48$&DO@g0}X2<|7c!#S*pQoigogkuG!2cyO?V+>Kl#Lu&53%0Ro|^%;%%bSyr6KsdX3DT;Y_zT z=3XGxk70B3IAJ?;bF02ry@D;Tuw_rfE`4L}McAodsD4FYRM+NClIqJa{IBr>V3%+c zn?9Z$t~cuA;Fs7sov_DIn&3bVnHskFlG{lkfvK*zAf|mumQ*bt${z=`F{+hD4iJC3 z6vl7*{jgqszRkb{UuhxuvIrBjq4E6D&2B9=p_Z6V4OT8%8`py>BWgs9-dg8Rn`Fr2 zri)pQK8ubynS&I^N8tV&hah!o1-lxTjl@ig?dD1vPd^_$>OqhhW=vRK*D$Z*scdHl zdUs9X4|q*XNUvAnVJ#r87P*syYEd)nTxgRY0NW_(MA~m}AH2>x2x#-9kc>y>#eHa} zBP|$OS*HbUBj=+e3b9zd2l;4m7E{-vdF*$!%cj0zS~y^exDZdS9i^~;U?1tF7K9i) zb=a9uh5kKch#D?y;zk4jDM%uCad2lVF+oiFJoV6VuZ&XDiDZ2S>XBc&bk|{ap0%w| z7HPVq(Nm8qGO(OIQ8~WeTr|vpKmOjdS~I(4SYNePo3dEUxS~&`V|4Q_b&C4FHHZL zf5h2k1&4*{3;gFb_<8i`W|ya48c6MySRUHyd`OZ}!=y zZjSTj=;dOLzm^j?HjqU@O4i>tDhr-cuJxQUvOVkCleDNmw#p*g*Wa+n4q_{uQP~u$ z>>iWpWUpqxDyQU>pP6)9EXyxijtU(uI_orN~VdL9vsM>_4B8yas z+Ibgz`c+zHy83$??9!iiscKrA!Y_%&yC1#lU>>O5=4mr^#OX~cI+9A2&g4gog9mMt@lcs%whPDzO-M^?p<3IeVgDd8*d(zbV9*v>RyOk**)k2a zhuJo*Ff1qKm=m6zV?T8ePoQf0q-pW@FUJl(3ZmfIc_l6+#rg`&>cRG?X+gN%P&4b* zsaE1h-BGP%I>idv$Ea25$dG#ozxY9L8j7YvhFGN;sG#e>{(4X2fZNSIrl0sTp%Pbl(U8lH=tRCjyyYu>?me#xx&XQW}R%wM~ReX$3*E& zPmMJ^#yHMZx1*OPn=wpoQB#CnY!TXvlNvR>vxebfu;!tGO@3~q$jac%s^RaGCgc^8 zZw{{zKogag_eoMv8t6+N-ATNZ{A6z4+~7iL91E$9autF3-#ClZ6Hgp+;E-hoEN16T zIJXllh^Sj6ByMJWp&W_8Zw>m_!PTH#q?mM_y9(p_<6#gC`! zE8xM=lAq0Am_I-NA*xiIx%F)Zjz2|w;j_1%p1*wSng99ppa1gtU%r^~PTzXP_XOFF z&tN?GQ#ke6{D<=w;M7;bU#>ouAwMO5`GRL^p*I(_TR`~~=TncRB%FE7qtxpg{% z+v8Z&uyy1|^@n|FU)&c^^8l=yXoC8AB$F{_9+pU|w+S8@ZH-I|@diQOz`4mG7ni(W zbZP4BMUDQV%j5RJV}^%2!1G2uqY@(~G0X{*(7b7>CV}|WELgMWV0|=&jm)k@5ce@4 zd?nk*G`$po@cDr1s`$|I5odOsDA!r_08&~ z>W#VIRIgS)uU>kx`jSxno$w?IXFQ4UNKYq@D~Oz~UYq-mC#%0hfenNd@~8QG{vg2r zPgc)zI3hlaZ@T&<9DW1uLWt$paQYcI#?M=P(o^j?<;j!43sIQv^Xj)k+2D+f)%W4- zSJfM+VseFl#ot!HA#daAi~DZ@~UcAHT*)c8jWIvU0I3? zvBp$GP*P5OPXR(0blkNPEp(dZa=(SCbMS486;`5pgy3|T=FGj+%F*!^vZLMND)Ybf z0+0~msd^;LqZ@a4$%DMBb8x*GVM}da>Oe{fiALexpz{Q2i7dBlc7hme)?mkdWO)2A zbJ*0GmNRuE($!%Ydt6q4UkCKrk4t5+4QDhQVYAnDBBoQR@I^t}5XPCo@d4564{_8G zk(2dqXc&}3R=3Fryw|nM8vI48GXW0X4$&TDL#k$nRB$^eO+3&xFlRU>=GU;WvKu)x zdf~%eG6GR;Nz-YKaiT+bG@1|`HW^lEGyWKRMg%uOPBn28)=|sXb>N~F*p8ZHYKN(j zERBF{UmijDK%+s1q$Z#cPq@JujI&ncptNi4Bkj| zu3(VYW-l8grNk0(Ig#*m6Z-AC9)E1}WA;uFm-|#&5Jmh3v8lRatTSNuze{U`xcKRr z`uE{FEx@|F=_7!q{>|X*MT0ki!nG>RUbnGq3U5!R)|wZH zCj=$67B+P;os6ODNo><{c(8^IfMkqKA}NkZ97aowP~Gg@Tb2hB38nbao|8+5iSF(H;*OM!ojAAc=HNN-Lf zily8pj{N3jMmXQ{E|3LTj2u%gm9o60*;mWzH*^@a?NzXjYkW<^n2mYo!DtndMjwyK<%u{@uoP41i5iLOd1O9d^sA$A`JO z!~J0nWt9H z#I&1l`>18K<$x(|1#vS;DKgK>!?>lFXx^Np#)fX{Qd~$haG17%D7w#Zm`=LM_E7|G z?6w~xqeVvgEQksaSKdDXdl9yTU=fYjJjqA$=EN4Ev_77}tn>+XEJqC*0bgHFDxLNd z5u}6P7%3svfZ(>RppruGRj(sTZSGC}59NgBW;k2k8LqSQGy==MnVa$OXLD!ZHFy$U zakcv8rWYKl!-$=!UaJ1tL(##guM!T=;}5RSo#YTXMCDzZdxoFlm*o^bJdeORxDH=8 zBMAJl%7yEgE7hJwd6lAt+@fruWaL=0R>Q6I=8t8kvAsao!Y8BJlHF8L(Iy?Q&u%`F zyQxKO+iaVVa-;x2POfDW1)mfA-cE2faf@2HVP$#{EaEylZP=u9=JD_%GK(Luh6JsG zEFX#-nYM8y+0rn)-h!|goA;5rfMc14%?i~?4>Jmw+0+Vf-#joMqg`Z3b;uSzf}A%3 zhmRoqgfYj)7(1dl#UW~qGsj1FB>SX|12Qivh^9y<>htQ47`q?X0~O&ZIC~JWMMz?_ zgWV5I0a5dsdTwDq94ALuF1O7lBZm)k!fIBR>htPte;79C#?u!nfw&N)%;55@UkM8Tw-3i-{13YP947 zmc(k%TmCqpeI+5)K;0OaAP{EU-7hbE|JE~qzxMIMsZSP8J ze4uq-m_;?4CWBiw)kM5!{jkz|i&V49uGL`x>7}n7+Hwt`=m@+m2~-pjLnxi~Qis~q zg*$FVnLf*P2VqM@et~9n`pqGC7K^u*kw<=I$5SaCWR6p@bFS57N_8E~cC9oF7{f5^ ziar58CtjJQ5`H3qgv5f97DQR{ZE!wS)_~J1fv}3Wyd~jTSF0>o;D1q9Z5M_;ysHnE zNeEJEtBTxc8GD0CqZ6nMA~p&pg^0oi8SJ~$~guz~j5%fLdz%yLw zSUb+tqenAjmpO@GK|=oa%0g=LXTKs+MITZ!Ck*(_1ZcFPPKedri6;u#x9&XM3eIE@sUYwPrhr()@3 zTx{Lxb6CrR3w=<7$5Xz?t9JPKU*9p^J$0#5=U0SzfOM8m~ z>z8d2V+r3P%MXH0XoR20ZRmBDoAo;Tm}E=z7=${d%2J!Ny%WF6&gJmm+k$`m4OzF~ z+D?bGNVg`}T^eEd^i8{4q!WmDNu}ybMPbrz-=BQ%?iX*~dH&VA7yi63{kuEY-r{Mu zHx^F)Y2nnTs}K?F9c~CHmn0}L&AQjL*hbOpw#@C^e(A-9Q?Gw_;q|{?e_`R&wS`lk ztTz16ww1u~5z`QhU~g*8r12VBB_0r~lHgbsxjsV|#;6O^R~Kg9_T*C*X3i~4|7jIN z?m5_CNDkiIAmi1c_usho_iN98|N7_OoqOy1H=kcP^)1f}!*FI6re6+iAyJY9NfBW6 zPNw`@Ypl5lGn{W_BUub1QeST)M3L=)wbtChkzr%7f0A*-oHQ__NKxt*F1)4^yg2P+ zHCrFE?Q%;KEPRV=)+pukbZl%5RVJ`XTu>S?{f}4)Healii>Be)#C9zWUg<{30koqI zn?SXhWNa-h$NPIRI?azmm}}dS^8l0}6tl>v$}<7fs2x8f$$2<>s12Qtz*8|x@6$n52=|QTr8`${+&Ovv z_KUywk3GDPf;>wQ!Eo}7gs!^8Lo4} zj}}`xlgZloqGd8?XAjdHix%t?v>0BG+5cwdqx6dq62#*|GCJ4J!<2zz&WQrEG)f>e ziI@y;S7$;3AxUxrKw|w0w4+6fLEH(xB{FA1d6+vRpDijb57Hvw>QUpAqstU+?9 zhaRJz4j}wEF2%~nSMe$iJ>FPiU+z=ec)y}ocXMPv#hg6VF=0QjNKVwGzCLuIzP{ea z$`qCzyeyn_W5xAauC^zI1ZQ#-WxvRw)I7SDS_aojvPp6?Femj-MZ#&JKc9PHt{HE^ zW|~6PcaXTcdbN6eZn|bJ|Je*bxP)xhJF9Ol6IjEO*m(3Gq376s&6=PFT{+2;KCSs= zK)`bn;Va1o2k_A?y?d;Fh6d5M?P$43>tB1A{lGMFAr_0=o%(QXM>&hy*qSSBQi z>OS#!V7>WzgRJc7=tZQ|QbPwPSKIc>Uur1XmnV+q=nQEvfi zkDkoaGVgH3Gca(&Ey2Slp2!;Tw0`s`ILuR|T&(Cz=Pd^!oU*ANJ&ModbyTRa`BS?M z)Jycvb2$b)xsR1K)x_(Kz`2 z5{A4vGF1Cu9%QY2lZCXar?1{|+;3VI{B?(a_r=SfObJPZy(Yj{jIYfK2|;dvmxU{n z-B_5uv@mmdVdmt*%#DSaXBVd5TbOxyVfxF3=@0Szsq+gnuPw}cgr&4ouPvN<=gw;% z+&TNv->+Z){?xTQ&%MxzImU&V4{C7_mvBG&U|tA%M0Iq{psB|Pv7~^x9`67NhfScEdd69zy7=1&tJU#$$#E? z_sq^apH0I-XoKxF^-9SH{*rv`T2fGUtt28t(#gmL>S0b03O*)cy?CEX1BIu(Gm#Kd zNzTO`Pfhs`me74mkeL-P#50Hka!ij9dSbER2L47LHIzhW72ZbS;*OIcM)rYA9^N-> z0EBfb%@Za{i-buM5hF^x!Pr4v!Xav-?zxB9-1HN05j5K6gJyYb9YdO>ySlw|l){IF zH^t)HYoG{11wP|Yc;|2iUIS$RU=&ey;u*l$zDzM zt$Z~Eh!hY4l0*@hMgr7Fvi#-p(ibySSJ%d{Jz&GCUrCg5Ni_gL*01{20~?LT3^I($ zaUE9c8bOUmovIX5?~4Q3kRn$SWn2etn)bJOtDvmU|UHSZLB#MO@4 zBdc$7VPF^?jGvF>&oI0zC@Rbagd52U--PEs!;yx*1sbUe<>u2;ZaFU%1^WfI=H<%1 zQ!!-`LMt(e*+$6BZlm|Ab_dFgss|=2c>11N_7)5SE zg1sdvwobcV53>wvsbq-S<64B6pqZvv^Xwy>Z+ROegj9o)L*Uz>kLAIYM#I={uB7o+ z)C|dKCg~<4%V#k7qdC)c2<1)3aAvY-64iuT$1ojIV$8t@G^cSsrh==f$Y={sdmYHx zf!c#V@}Q5{4mpRX?e28EE;6{9-1G23<45UorJq!NB9r}45g$K z>6-o1wIv`Od<{rHU}TFAPl9U2QyhTuGuv@rSx}_N3#~n7YB!!W_cu1Z0b16#vrEK>Nd3VF1ebp2VQm6g zSCNVc?sJ!&h%iMU?xViBAtn8_pMb?5uuxiv#kZ1ObwM>do_DPYx`ConDFlzC8>Ge` zUCVfBNX#Om>f*-bA;B<;Ac^&TX=6e2kfqsok9mk-feh>($Zq=OZKWM%pL@!MQ4-Z9 zBUGP2oh&XfyJ~mLkC}(l)fC6(6Ap|cq z?FbRz6YRl?yLZQZIXVAY3g=sz zRuNLE$Pz;j={9Gxk_KDdMg~#wl#pm3H`5&`TW!EvZ@e-_&c`{i9Gm#>-Jr-mHE;5xA znZ?K?oTTK?NdNHS+yAM5cV>nVz1We&3!aOX*c`Qt6iVjtj|#G)NeGk7$LG3v0`O#atC_J`7g`rXKEr zj!Clncl$VyH@y*C=jYXuX24AfK8$JE+pr$4?2B-gUz z?Sdu2v?`r9S@s^>KN$MT_Z?Fi9rwW>YUqc+S3DskQd`VMb#WHB<{F3LY*?g~Mzm2P z9XSGi2#X?2H;n>yHn1v^Mp#NLI_2p9XYXCV+o;aI;eTb{?pja6ea92cNeB1sdp|zJ zb`DNb+az}jUR_(GZE44&+3FmeD44ZKLV;3RXiHnl;VJ1Uqy-uZw2%P(;PQSp-{84F zQR5tL=>PCu*OqLJC0U&r%Lm2oDzQeAWzAf(uYDbU*Y9`VEth&%q`*)UX9v6PIZ2Vy z0?gcVTlxeInETy;7F_D=7M|W5VpY;D!gc#KqbM~aH}Gg-iwi>`U7o0_JUNMSNhNL% z--o-?Y*3^Zc5r9X3`NxvhYRWjsk@I9*fBmN+3Y?bgam z|L5LgkF7j)}|><=6F$s^_#!ZuzmxyFON2Z|s$qU;E<2AKw1OU+(?lo!c)z zcJF6@gHv01?xio@yS4JvFYaD_Eq+<9cEv)qZ!KrEjS%7aj<$#GE0-8GdmlQ>bAe;A zZaKDHd$$Z?rj)d~8HNL2R1kCE*0~KHd88cJKCjq}aL)^v z_voXfpE#m^M5K)REN8t23_8iUEpb7yEj1>&OKptRoW`A@$$} zH7eW!nWljoc;uj8;&4=y6r1QQ;!hAwvSQ$sScu_V)xu0E(Wp|;E|3+a_Nh`DeNu1- z%<*jM6u7l>xDvEXTFJK12=1dCBcL2dxMzu?ODm=77SE!qZFe$5?P%9*n@RLdP2~}n zRqX1ncZfSMe+QR{9e2ce_>M&;DN;p<%uo-pw(1dSTfu-j+z81V8-uP@;`122tD)h^ zgWdzRA=-x?TDvAS3=tmIqkJ+rTZF#|SBW~P19z}?H|}OPyZ&U_7{2349?8RzFwIN%VO;GRc@Z1^`pbFL~N4xOlyu*menbhRQ-R0Qt5nVCZ*^okF!$S{~+V1=2P0R;RJc7c`=@$-)- zeiA-D1iR@2+u)41Wj!95hj2);>Og2zrsXoV6O_VBjC?cT7cgxir-e`baJ#9|ZHj7e zQ&WswbHO!^Wcj_Dzqt3Sw^wfc=JrqC=)H2??H7J@=h;{9y>x5k!?$AFVtme?o2xeq z*MgSzF-u1in2KSvG{~d&Vqsq%Ah@&DB%UD?_ACKU9zNpOaK_#OEFPa;kki%>n|W9R z$LC=~tP+btcn;6OX9>59ddk~T%R9p%nsLGR*<=;kJ-T?)&x`nt^|gx@?)im!x9b6R znU2QHDCu-!`Olq+ai-uG2Fny# zurTKOY$wase&dDbJKh7PV(Kk0s2KOaQ9{eGtBN_a4HkqbM0OPn@E}crI|R}bMh{`} z-YPL`PCi>Nn(X>WsbhU;n$Z$`8+Q1c0rTXLZ2Y|C$P{Nq78ZdpGohAqM8K3P!EDhx zXno_<5xg1FF)b=Y(>B0zoH^Nolyk@^s;PxnTl=-v$wq1lk=L7!i&PmIwGE8w zquS}LqCwE3=3r$ngb$jdqjhg=j*gJ9bTvFW7I^*;bKR;^*GZf{yD)tg-rZXf(rDlZ zE+YqBTB$HMepIw}8A3NgrhSZ@=@25TsGFy0%^`A}l=JSJAKiWPYVReRD?fSW?v3YG zp8e~c=igm<`lXepU%UI^@9*4tJU%K!(85$noDVO__H60u#$@Se2}3bjc$)VfR35(l zJH%c5byr5TjS!h)G&V&v1bEnNvv-U$qJ2hswQ_#(dU>bUMeM=sL7#)OEeir?y7)Gv zY$CQDrb@-d4_q=reQ)uDVmN8bHfmzoh($3KQHW*>C(?dzFhXzeh~1D0+sfu(T*gu~ zDsDT?v3}%{t%dskcDv!u2-ik?HOgeQCt8Lc<8L~xAFzpdP&1UwX5zsu&bAK`c|!1( z;lb&FQwzealn{&qPPTZzW#TB{OGen3OV4*D6*t?DOThCTZ-|Vd8=Dy;Ung2HAqsLv zJoGGltm`X17BZq@B$@JElx{z$rBWuD=LIH65q=|n2tVjRx?N!TY$%yCvW=lW`8b+4 z)l>U~w(D>fIB`&Qt9p3QnRVe+=kO>FvymZ%AmqFzrYI_<>5?eZ_@&(`U*stlUMLT1)aSF& zsAbSfnv%_IE=a^R@^IhEm1kFO{CMU4cUFG(`+L9nKliTRP>Ou^;l4wJ`wj7;`j@H! zwCl5DwNm5ipqL!zo~U{xqpALdg|$Zz`sE=Gg|i=kU#wJJUM%^rRSR3r{cKm%(XEJ( z&w-yIQx(RfkNK7e5{OyVFW?A67sW!*r2cICs$sTc)zB~%y+yOpi>kuC^y<=$rN1tH zxb!Yrdhe42yE-iGI#dCYY$3fqw4g~e%^!!|c$b{voHya>K zESt@Syqnn`Kwx&n512^R-poE6d9}D~w#Tmh(IWgY1d35|E`Sh`IH<34!3sPYe^Crm z`0W$){IS59ecRXaP2jfe$MdguGeT z5KTzjsHiE$NQI4*+Q3Yy&W19TteO};ILW>%v2(EDc$jKeYXu05m1{Vo;i-|Yjj8Ga z{3P&mX6At{Lm+LBDTnw)4HA_?f(UvX+zu{TN`?Y!j)iGz!h9%^onTfm^~9yvLD?JR z&a80D-OXdR#}32b!F@923+6fr@VC>AjC*zs?KV|0#@(-izrS31Ah4LZNs4kaQDwK3Dl$AanFTFi?Jy|xa8un#x~z8P?q zAbo}?$cLbz5N5{QOmk`y{tWVP^jtw9N9rOex5BE)qDwt`|gRA0* zGPmGhN?70ts**ZIaz#GeM<9NM9L1)20cQ)LA-t{36{XjJELtvQkO{DBIYAD5j8Uh5 z6!PfBYXU9?c)oD_7rzy+A^a?tOB}iHX@qCsiV9ZCCb;Da&xe?4Yh`9tH5ZETmoG?oj%1q81#@DJ!b;8d2xV8t2375(V%zUuv2n%jdEiX7qHBp8QXve$ zH+d!a&I#D2m2;Kie(QgOd599dzz>mQoH|7>{yap7o`1G|&p+GI_PS!UjA!oMqJF6? zJl-t;n;P<|k=G*ZnnvR2bWiV^tK2Wtt&%)A2T|Nw&eC)UaFvdVxYCdnFRLb|PrHxU z3M$O=>tbW@q46`2F3G8aNeopn())E&Xcg z-KBSy-i5<=Ke>$m!qEr#0)o>%fRlY;=|*g`wIb3IU^X;Fq-9z(@b;vOD+QF$?*`l! zPJ*zadmaukXPLx(fLg^LPmAQKHTQMW%N5}%Um=4wyFi@+#78IDyf;@R!xWre0lN?f z$05OjPk>j>lVfoCSv7vaa+JC*Ee8H<5Tf2NGucMHvUU*Bc)Fq`*T=n&wDa>NKCZQLqVT!C#!y`g3yRLLl6&Q#J1;e%%CjdgFt;))_`CbI7~ zKDIDECJ%K*RcZzK9>RzY3ztzWCy=QierNR2hV@; z;ma#Oy|jGkkIRoevwY=O%UAxea_x^RzrA+*soyU@_QU1NKe_w24_BUkwa1+6?N>g! z^Zbo_zy8zh8}Gz6&6E_^acPBz%#&$UCNmN1lKMbORdTtOEPCsf(8`bf0@qqHQ{n4&Y|yz;BxB!yXwH5Njyr*pWnRN746aZ+T>`8 ztddS|MtgLW+vN=~@_nbr`k0wIN_g{H^T4Xk!LB34_MTwpB(P$A`(tfn^La`JIRtE82zv9D~= zST#PZlc0MIk@y)F*aTv2HF4!)pT(wo_rnM{n<*c0L*3z%H8%i+n;ih*IxJ@;vtJBp zU8C^!ZmIqzS*Dt)7`e@afoEM5%ytlfKIIDsL5|6^E(}uUPT<#sE{y*A!F?idF#$fw zAqPG9ZUx_^)?EBJv!lF4>|zL59vpP4InUNpBi?8+IpzJtzg;(NanmV*Z z;ZbaBj8qFE2rgX6>7nQ)L<1lnM>B+OK>pBdh{#nbFvRWWKf3qWrQ5%}eDCK!UB2|@ z?H|2%=i05_GYS^b{`~gxmG3WKxxRem|HL-Q^5u7zufXp=zkK=En_3w^5mO=NhD~Yy`rVE4I9-CqM6A^ zDJe5DP_=KWf~efIJG6jKxqfL?E%)CKlT#-~2FUTnzg&cfi5xg^aN^X+k;#(- zW3^`YJPmXgPV4#zljN&-Q*rRZTIU}@tOcFHc0K?gPt8&kLGz@6h>8}^#M zGlPBWP1tvayG34Wh1Lz3&cbZ~_dzS{5y^ zGRN)&N0oSs@g7sSZ=AwOdJ(}?HVd6h>}A>w#d{qwAUSg%pR*S8Til`KpjF}tqQ5R7 z6~QR5aMR5okH~S~B%CkhJ}Csm<`DKbR?{fA*}xIF?2FggcM9w~_JJevI(%n@+FUty zdTgQ7iQ=**w{l}l*`f+^dWVf0>rmmz~boDm$0dOlo`daVYG;DJ00GxI+V|i$!2^)RSIJa~ahtZA0q7f@Yzd`s z!pUM=5JSbL7f zxC~qXi$B*0KCV>DH`cEG1{3rhW>gIXlx!lEM-?-Vc%nebdAQGAa!~&GkKt3=1Efyz zRbtfkOV$7Eg?dZJ5sfv?KFAxqxx-TyXEm1_m)|y+q+vOVM&e*ZOdJfP`7v9-<9j-B zsNp6d6k|Kr{x(of&TN5A?sdDADH^6*O`Jq#$tt;)3i6(et@EGX`W+{q-~6pZ{{GzO zw|?1;2-|3+J4QrEf`LGhOmZr@5h}T_%5aW8b1?M9zO*-h#VzO2M|;XNau`0(Xd(Uz z8ASF6HgsybVT@3%IBKQXKzxW1T~ViR2zGQ{7A|Y96U%{23sMnAEVv%DvJt1tZm9-q z|3Vw{952AW(mzgJ+9nz8n;2=%VASx*nM+RaxaxG!l{!nv-q2^H`3Sfi5TF^W86l_2lU%zV9p3T=HETT3WXzb=pq@m}I&(D1^z@Zop2&A{Dq zgp();NO6wqU~%<~<98{?gQCdgat#w}Y6#CA*~DojZ6szOlOX$gDGQ-#mZ(&bSN9M^ zy5Ok`Rj^}$yRaH7#V~I8et{#NsQNY|(}BFWv5kyr zX$Z7$G-~4bFq(~xlwziu3$u~^E7*9tM9XBZ^!d%-pyu2CwE3NclX}A^LvU(c+5Zwi zHw=Vsm@TnL9ox&{CJdFkCxo>;mGkKS!LG0BJ(6NC-Eem*Qy--r+* z1_&m0*7cT2<&A6V!#1DwMT->6G|s{ho7c2Hc5^K~)e%`en9s8+nH03pri^~_SaTV@ z!BNyGq^C1VR#(y~*mqK!a%o^2%#jlZZ^t1CoUo2Me${1?2G$X^x+Q{XByR?$O=V<@+h>qj*^-aQcdWSRlOrKp~mUL+jiT<;=A{qxkPw(uy5(^!jdl*#QK1*8Nd z3xt>Pjo(;(^c{=V=qm6TIBO>;!^4Lj8g$@gduE1Hc)n7q+N;mIe#sRTQJ9&*S1g-) z9(Zhg>8Rtuol<*VxCJ4fJi#iwAS$Q>F*5@%%+fNq6-8+`W7Um2Dbk2sf)Kh|#&>h9 zj+ZHHSK!$aP~)EEHn_nuj)}m-(2!gv6bEFY2Lh_8q+0rXdpz%i7)Ds!b$1R&XG*9; zOR}@)?GjkjOM=(#o?SE**2}(B?s~f*E!GaAPEABBq_pN~m3=DdrCV?yAKd)pv06Pq zRzRM>r=DASTb2htil0d|b4e#C4rr!gCJ#N48f4c8k{yvjgR77q#6K14(xrz_&@y!? z`KI_Tr$K_6Csqw6@bk%Zi3tcc*3t6B71K450Yps>8@}(nonK;3w@}<>$j_OP9i5IG zNs-6$+V*uI+-9{}OMwvx;ORx2+v~m3=a5^4yK96xoY_h8uy*_`YG`m(K8|P=T}i8n zMK~Qn90vK9FNiFJ5fS)U^zx;2JL=g@*BP7oSUVGeTVSrNY8<&xp-!QujrE{1r99k+=6iU3*%_Y1V#vdNnbns) z(8ZcVGnd2ZdxSwupO6kaZlA$VDBl7G- zn{}n8(`i{_%NV`O7Ca_=f7a!&`0FJaa)R-^!{Zlr4IKPV!GBpr(?i4GAu5l^rZJRU zOI{QU20~p1%Oojd-zYeC=$x9tQu+qY?@ zLQhS=4H)WsHf990h%*9Ug0zxLF4cJ7`A95mP)13a zwoIOFox+QjN~w@~WU~X>D13vQW)2VJa>n4mur{a;=-JVcfs8gdv=H%q%?dG5KmPtw zkU2FEo7Ft8P?s0lA)fxl-P9r`lWCNgGZAad>4B7{s7B%}alc8v<4y81JtyKb(r?@% zPc>#P<>QFAk)fEm=D2OoO`R`Zst9+kTeIekfgm;FKu{XdO05Fgdw0*S=QSSu&dy~u z=>b($(%Rkz$1h$Lq`dg5Ace(Olgue}10D~C&>_8SHbljb$|u9TSqA$VmTv-Zy&^zv z8IQm(E?!%Fm$eU}yu-)D;%@{wgVYA(H0AS@s0(eL!$J}Kv0AZca&SS0X&%0f5jxL3 z;myo+FQeR+5NFJlgwY3w8Fd%0#xMJvf;a$$?JanD@v6%Ofnb6bLbad%(D?JZmTk7< zeE{)u^6}^OQO_i+*l_Lzte^Ne^tBrKctPIt2P*ajm(4B|&wWGJvyVQyRl!F0EQn!w z!-BH%aoUCJI7_X=9(4*e&EuFv*!2$x`_W9OjVKSIEn!VlvS#8)vh>07uj7YzwbSdU^l;SoEa9?hKDK#|PmP8+ zyJA)f^va*BSrQ{THy)$vfW~XN+S*;mD&fbHmYhALMhKjs;&wc z3VGspPRT2c00=F!8BV9EAm8uRB0#?i{>u_Yhp$Oh4hiZO$S{`QR_DMHHx5xItO|jZ z>>I*CzdX24+J-g47$k1QS~#@{d|C-Mxnle-es_C^aEoNYtG2Kqi|Dt=kvR{GQ0Iuv zi`*9H5G-?ol85^yMHS9;5YHSr8!b4P5*BAZP6Co)>d*pxxQ`Ts1Dn2$XcJb%8_OnT zt|+}Wsi)<#DLAPza}#tqj)5QSM?U+-YXUCDFc@x=A&dc=rI8q*h6M2Nh!zoZLVV;H z-e3}UcbOy(ne{|tST(l2PEwbaLMGBe-dT(8o()YgT5vRRu$|-FLxw@rS%6gsA%WdJ zM)+-Uj1Qv9w#Qgr2%qQ~J$~tzx|OipURX9PvKdUp*k|HsPQt)%^CES-CK#N^i)9rv zm$+{pp`R1#qD(`2vmo0p-|+k(zrf3=9DzK`->}8`t*do(t+6t_E@i``fvxSXB%tw6 zBPJ%JB@RjVP1fLCcxL9& zM`h}KoBYyXCSuF2z@bONPso`!F#xh8^W9o@9 zO>kQbkRvun_v{pKj%1YC@Ox)Cv-_C?e@(4HKNT`0T$jRInA1>46q`{;RM1)0#Sc#& znI1lM`dgDn$Hu1@+CW0Fs3Qj}2m^ziFF}4qM2*kqJj+W*9XK|hV|Eq2aUNbh?~wDt zog@8(JXm)I^*?rIba>$m_X28<^4@X>HRBMpPW)SLV@bvKtb$qxR>2Gs_2|(BklU4W zELtO0xZ`86}-f%yzezRZ7N~ z>`QE^LP;*<^K#Lho|aNFdg8rpnih+)AMaM+x51HsN&079#_Gu3FfD^Z$kAB{V{qzP zzE~$1tSJFtUg@l*mUaF@hh6ZAoc1LJTJ_{i&Ur|zG3cqE-mV{@LAmZwRDFQuy&%LVHBTHFtAcdz@t30KE5^@3*3mjz%>KnoWxx5_pkSbE9?h0pJiR+ zvP$B9wX~9L;j)UnQXfR&paul3$Er0)ki!}|17(Z>x7mXZN9ZG*ZfuK8G0SB>zM!Vm zO+_mRo=LZ~c{j>O`z98z+O$|}CaqQJ?Be20hZJ3Eap4J4qWQ%)D5(U@lT-)ekQ72j z+QwXcM_TE6rz*9pw)1n`6Q_@i^&8|gICMs&v~x3A@5Er=dLzLn)GtrnP+3U#HB|0a z)KrM6x4%8tY6q&-jD*y*WN^F?rzu#j1P<;{c2IWs1_Sm-M_NpB)F}vud3-B`yJKsr zPfZMxlMH#_iAs(OpCSooagN6yEkYI$uni8_=QhWpoIya1n%>lSeTiGND9DhFB-i8g zJmlDFPB)N1#IaeG48yJjpIxpPgyT>ryd8!2qsSYQ zo>OvWGomvNUED`YZ!LZF$>jmE^bS0JXX(Z#mzO?5>D5~K2GX5{L2&kh9P~=OatLf{ zXb<$5D(jB1$?1(LEj{9pRt8B(j7;L;`jaArS(DUD4?Q#xHR7F_!GvHfk-$<{Q>c$w zK_eN!E?o`$Has=*wNZ2ThnF2Oz!MC9sG7Rjz+Y93kR+l!h-i9U$+WaQVzF5Byv+qf zGL=%596W8KWii2H1bO4@y++9JWOjF2*xV~U(0ji4XwiZDJMhRv)$`|&G$_HWQi)Bn z!fEE5njH~R%V$&NX(OYgTJ(G)TkdE*0S7-Tbpkxm#V*zM*kn?`wtp(9&((Yj?lc5N zLJ)w@O^YBtJVQ&TQV_wY+Q5*ijSP$$T6!Rz9We(o!$YZoK~+N?mEn*Kvl<%GSEEkp zXHc(5OYKWBH~NePUh$~CgJR4Xg40t%K8PIjH%%hJI#1t~ew&lnh zE$2fW#nq0_0C+~>Q0{@R6tiU1UA$gcyzX)fqFhno&USTmx@1~;HVjYZVD0u~i#?9pb@atzLvd@~SE;!(@B zPuN89D~n)^oW2>p!pXp{QAc8>9}9dvYLG$Sr*8kC3oFB`x|}AqOY;YXJYFuu^pK0# z+sR7YLN&`L4dt`wt_U9^;3BOFatyt?zg4y-4>3of-53dpV201HSy8v%5l)2+W&?uX39=tBe{UpcUadu81 zu);wZWKPM@*-^>0mluVq94mr%l~AY_pQ9f0VB^N$%u{Qw=prt9;A^9!!^64U*Wn%b zMipNDce3ttcbc;=LT9!Q4-`zlO2zsxRJq;r0tZT6yYvxJYS)tCU8>GG$gm zPm=50?q%fPIHNW)4Lb!g%YBD=-j;#@)m%77$VJZ*U;{|C6rwi{OR@a*E#b~JhojP{ zy{j%gLgyjnonkI?1vw}x%%ub?cg$_$)%#99mqIm0U(Kw2+nJN)Uj3 zj5dwgy_UsM+ol(|M?5R=cQS|5!5waD9%PL*0l_Gek{)$^S;b|R1nGswFi(ST5s<(7 z$Ve;uTf(uc>GRZ+zHB57xSdqKMLC+iq zv6y5O)y-M_=H3wX{pz~Apg)X&SWZ2&Auu--GrO?< zu(_1h^yV)k)r_uX;Mq6td>@|u$wLo~4@{m} zSb6s6%a@=5xbndZvG0O$H>eiFG?58%9s?1I&AaCI zwF)PwJI8D?6?l}Pf9R=Yy#C9R6V#gv+}b;~zds13Oe?uf$(kLZF8F0ciOXn@6@oSL zF2bbF8N+IN-SPL>N;}Con5xYE_{}B(dyn+E4Fjbc21-XaP)^C|o53edvhO0Q8mxf( zEzv{yNIh&NR(baWC-Z+@Fo!phfgV3n}&MjVd7q6E&1Y0>yp8BVB4E zLXnT7m_Ij=QWbrlWMn-N7Y^IwFzrqfape#}k_bk%f;cS&C z@Wt0$)?=Y$hcoFyG#M(HMl6_uX)|rgX?wL?GaDeibuhbA1)CgVAk^fVfupuS{UvhL zkt`k<)Kfb&fmj1j_t=JY0y`{X7V{KWAmWI_9=$qu-CUTB4W5RpoS zBc0;p9mCXhz|bac@&GP94a%z#$q+t{ZG;o-q9Y1nNfaYQBw0I%Xp5YZP9XeCKuqzSa%zkyP1r;D#?irz9=7YDI)=iO&+zwu7*t%7?$ z`|B6)e{bd4AKZE8vH10QS_RKVvRO4WUmnVUqu%s%{;#zPX zj?N>8=ZGgUnr#zVM(_0E)gqE8S5Uka31KVLrL`9Kk;w&C^f8@;62yq8kc)AHu_h(BodCN1Wu4MPoQ&QGf9&3V_TqQnW>$#-~gC z*P?k^NPA6-D!m3~*OY8aNo!yDB(|~jlx~n=xX&3Z#$x<++*$iX4}kz9TQ=PWPuc~N zmh3uWli?6a3n>f96)@X9TFvb-Udvf^o316?;Evki&LLfknoGQ0N|X`?0uT2M&d$O& zW8aw&0bJ(qF#kK#!h$dQ9r(?_t#K$zs=gA1aDgFTN1ec9))nP7MrE1BCnZ#d&$TFA zrvb8Eyvn;h${TU%YzR#(4+m-{!p>XGMDdG58!{=YA->6VkSQb6=`J)hC8Z`V#eyFh zfsFxz9D+0fJ;>#>T6fJe&+qq`Emfe3+_edU+Y{it$E#xQyl_(Kux;AYh^QTCL~;mb zN~aT#R@e9i$xfX5zH*NnVcLzE}yk`7lakyr`F&z zJKcEfT8G$G$b@j_q@zKnGs0a6UcgbsZ;K2%ehE$|Wbt(SA(3>48Ic@nG^JZccDA=( zt11vOW>najjD^tS7L;S?Z5@iWBW_SA@Z#>5gKg@S`K(fcZ4!ebo`CyxeX+LDuZ-NT zG%^sjC!X&Jg$7_;3vAyK8Yst3kKKL!vn34z7+;Z+zbRM*Ja>tbF=r8uUl3-FV`PuOOaUX%H1!_YEa zv??z3XaXMx*BF@!Hwgzaa}d8Yht_-Dvx$73TP2&h2?!Ls}olndF;5qWKhe5xPL zJe9*zZ!J@g`tAkh$e7}&PtH?sEd>cA=Iy1}*PjHJl-u5+y zy5t=iS9S4?*!LaUs8G+=*7e`})9LorqjbmHF0B~meozxqi?L~$Q{qskOH&0QPT`1@ z!eADyohZp(2F7}<4F1}gwF8zw)vinESTu~L7}_R^Y_+25zb>g}R3)8JG(%TZ)mT_} z)Ld3w|JK#QYAxX#A6pn7lZWss+)~>_641TmofgPJd_6uP(^W9_iZeTjE);Z(uE8^B?A2f@m}GK)qKf=bdT7_F05vMue%*{ImePv83VwNI~o`iBAX=^LNk`0Nt-?4$pF=Ch|hd*ahS zefH$1*Z%t%^4a(P`%$YRJw373(XuGc zl^plkF)9wlwt906-2B=ui+JJJ7FM5KduC?lB;+%vPUB~fx^zD(uIr4ozwOM!Sa=-dZB*a5ox^g{ipB!^0!}n^vb zZSJc?H!@E$K^UkWZPEIhwS!j6%FV~E782~_4WZkI&bl3UE@HU6YaRt?j9ec9&$)XvB_2#L!WdyhW|@YBBo# zM?9YvxByo$>#~9%L(HYfpS?lNdT_F(W$7=^YE3P#0hRVKz&l=y;OWH=Fw`QU?aftLb!1h2BxXeO{^Mr0TK6sySS7$tsED1&we)H)mZEQe zx!W&4rK-1o@w=TaU2S@K-Fx-Yy;pv{^2Q4*w|;&1wIAL6}| z>rclv(nQWknW+%#n-*%=YEc`uDxxS_ImRk>bQ7Je<{>0u2!NmS>73nz>{7>&SN5GU zgXk(g_rnPa+d?s5o9nNt*g4M_i3fQHU8}@pZ&f1&?pc9{&itS!$NIt$mE%Ya><*WB zDWLHG!?j;zMAIZTrMBjGv%l*YXEdE%d}E69%(bWow=*x6DVCCs9zEJHZCqVnv};z$ zWj2!nOtE6%6et-%;%ABa9V}Cy1wA+c)OsofMjmv7VnE|(0LKFASS6gb%L|9Oq5fqf zua2Bn>#CAz>9WL0t0$^{Nk9-(1*=TW7H7TUe7gkgXsd*%jIMS?ayICBQpz+gf(SfU zRxwp2B*_$Vfm-CHqMHz*CzgK!Biy_8+}#iUu=46V%a`7~{j+ygF8{I@btrE?|IwWf z-@N6`(nTBtdQ#Erz(-qUmWR2_s>|7O_R=*+D4Ih0J-fR1LdU4_C zy!iO)9RK)jGAAl__3uCA6epk;K7N(cz=r1+jtPh3TSVg#ap^hXP;d3ODPLG9!}s+T z9(w4%&VXAdRGFE90~C6m#0dnBLfumc8-|3e2VUtjH>(z>V z51m0K36Gy!Cz@`CRnyJLH;@5IUJJY+dSrut5blJKhL_7`dAZ_>dA87aDDCCbA_hG5 z1hv;nDC^BQ&ijWrES*Va!Fq|0>^#d0Y*vISCkfyODzccQGMkxJ9;{Rtg$%*!!Ng$n z=$RpM27kSpr?yQdm~+4*4{B;%n01X_tHk58jtsh-*`8g+B0LY#QDrl!1}$gpAfl;E zMQfS8+%IrNE^yH;Eo>!79Sn%7a*?&|n+^mVJ7d!e^@qLZ z7e|?Mo=eI*JASqfXFWT~Xd%q65hC6)B5!IC?&dO^(Kj1|7x3o7UfXB@=QAWczl%&e z+cbZG{$U!`vd9pvl1WljKpxxn&q#wxqs z4jp~nV7qqH;ghW724t!X+wmL@u@ra^Kiz}VKuO*~%MYj|1ZTzF~+_;1+P?#axxh&MK-3$+ebIvY~;lT&h87Cm%<# zqBfAylyoYwQ1<=SR{!44-zx0*PJWfKQX~ z&NJMx;*)t~bVbb}bR?zdnUI@nIJwBC$SLMtSCO~pIVC0G`dBa8DW)3@Njpq6@mTP* z%i%QIt%uze6f_n3$Pn|%M3tQ2v+;qy--0OD zfeEd^_5+t~EwCor*#G6@XzNPT6isiA&*O|AA!AXEvZy97A*ClMnZ+jSEE2izbXhR= z?lTODDiaR%1sQUw$8E-4iPs(te3lnBK9o0bKbcy*?s7&(;x{)T%<#@rdmRy|mJT62 zA{)?v_w&t8LMA1|nxgO~H>)LBnj=(2Q=w zOdQHk@TpFrUP%fq#FMJ7Z_o}M8`>!5I}X`(L6moe?N+JY9XXtzhwCf_HrLL!*=|Qp zjx`(Zh*Pv7O>9^x3!7I8+s^P`7)_;y2Wg{D(t@FKijlnTw5Rt%mwtCk_rHF#eRN}X z3~qD`DN2M&4w0kc ze4B*0?`TW8qW;FZmCTfqRTWLs6)i;Cmuqw>a}m3g8mj5!qHl7|>&^o&;xfC=saW7H zvmm(y>!8$R#bvdcQ|&`~hp}mj6;SlS0#U%mL&SFe2V z)zj~P_2-xW=e-+WKK*?CDLjDpefh*szIx>y^5tJ1`|@}1;HzI=d#?Yh7k>NY?_MBZ zJ@vv@ul}r^Wp_2sZteWq>|JuCY~9{9{qYaysvrL_6yT>IHbDA#v}#x_$ZW6HU~Vkd zfF{i<*_NvAh9+%Zc3-~!-dB&mEba9E<6ph>SdSI5zIx%(|GfLgm#@B0zWUkYU;gl& z*h}ln%P)NO;(N7!LzYn6hkR2+gAcr-9A zD&(LqgdIPl8gi=^D+R?83)?ItV-vCYr6VbXsci;~HhmtlkPZ03`c99x1aYTJ+~$=E z@zvk!=GRRZj)^ zqc-(Qelkfp_E{Gp)BA-N`Q!!g~j^ixZhan=UNY!z;y-y0az&xKl-1ndV_Mg2F+lTfXdFnb#-J#PbDAX+RG89>=ZDM z5exFn^43o0WgA`RY{ag!E+^xYEBWqi)6Q?LJq@)H;K;|(ENY}6X>XcfMhMC0|6-*6klR;}_rn^4M*h((bhO>+8>x|51W|8-_uVd#qWZLz?wQXt0 z^y?e2J?ov=o^>2*PPGhU?yc6RkG)Sy%G9QL>T)ZI^kg?CT6COY&hBw3?Yc3TTlGt> zs8ovym~6Ucj;xYv3HJ7+Ir7L^!goqU+dz%^Y&vm7P-nsz!7uR5fEM8UEQAkcX2^qs zgAJ{^ye+Ib@Yi?}o}4kRzi^&%iAHqUcLV16-ZqsTw`TvsYa-t)w`p1^a|iY-m-=|* zBT5+u%xkubXZ7&44VRwU7oIxWJ)>mxmJlRrSsz)sX+aiIZR(+F1a;x()QQP$3Ou?M z3Vxe8+{eQGRXl_EmHT1p+QKVvPZZrSy_c2=(R;~*Xv-1h(pw7n(I+oU%pve+4nQEp z5OeK;o!}(}$(5XatQ(72@^Igv-9P9Qs2z9@_aWie!+kXT47*@KpozmF z-wlM%yh9NE+YB~!cP^;ZhyoxHAd+UT*HjG}r=H02a2e+|w+z*^qMM0@K#}=I%!9KL z;7te5BTYpw)O^?5Ae-?^WYD4oR<71hB7QlZy9Lf1*g3MH@vsCOZD0}9KuE0Lci)62 z%(f|{P}Loa*$r%v+FR_!NuEC?03V+j|Bh{$>27TOqWcZ_$Ls@fT} z;|0`~azL=>`X#omdV^6RwdUIRy}mOob6;n;3p;t3q$ZbD)HJv)xbT_GCT#~FlS&+a zz+b~y;E<#awR(+yZO9e0aGqAl5Y6)hLqo_E#fu>l2rX2j)uI+@4QxEN4h_ZATbvgb zBXB)e@x1)v#aiLHcXpvcyTZsYFXt2R1A$9H%Kpu7LAs$BErboRn9S)pNcwC>i$M)k z)2lV}<1Y8ekXxlr{H%2dHWlWE*@4W4Ym$#6S|g*F$#d3829dJQ3&N8?vsI;23p9U>8DrlhJ-s&**#uz6ok)LJ#tNJkKdLEP;Navpx05&Vh-!Vvj3UsSlY z>+z&*#!`mG>?~s?_!jYzXyZXK91_N!Pd2bwoYI=jxu*&e$LSoc#Mg;!Q-{blI=R`1 zI2JGfieonKIdWc~(c3N$soVKASh9KT=(+L%qVo%Z8x#0w` zh^l1L&H2E7p1qxUb_28)5wT{X)V8W8?ihpBj$Aw?_1Geo5qnLUj!a^iNL)y&?>5G978mYWYKPboHMJ4H$j8x^u&#mQP0ZUeO^Xs) zD%=YEHk=k*=KfiiF~ZA8RTWm%QV2OZTk8hYUW5k@WV$u|DHNM^*7Rn%Bdb*oyWqoB znumC(K&##%!hKlw5cElE2YGs1r6}CB)j3)V##YHf;Amh%>aCxLdmyMN=PL0Nn4$GM zuF9mDnlmx3Izt+@Do7o`)9E#7oEtdp}MfXy(( zUP~-e!PD_VKUa&SWyf7|&_<$~QF>TL2dx8cov%x9B zmxC7$k<-j^ysAAxP7q3A;Q_h3UtlH}X zs|Fu9D&UVoSUbUG1q-E}dW_7j%^AeDBxLDq{r1_&Orj?yW)4i5n0qimaeaM#aiQtB zrMTYo_O;?VvVD-nn>L&m#n`Z}g(4Z}6b)iC6}j$Z0kenJ_r{gnxMlX(TWA`Yl(U=Y zpy3>3p_YoS^{DGY*K1uUNXexK_nO{!_=G4PmbrVV0jGWK!6}l;k|TBv{5CVA)O#w^ zXBVc=_JSyO6U2Aj>BNtc=zLh_S~o(j+lc79(Rn(n#i<+Z2j;pPqCxd#wYBxlK;bvf0@LnEr0jlcwc> zdUDB-nl#@dkA!SJeCp@|{5LZL@ieAG>t3V<4A~ZHKjd3bl1vtc>s8^N-ebpk{}87h zEDcK}8FN3&3v5<|wpp{$i@}ExS9VTGw+O9o2ng#N22Yed?*(zx-2dYiMdpNh>(_rD-87f~Fd| zP#4CCY!{SGZ6Tc94}Ky9KWNvj75r8+Mq_lKb`a6186}t5%rMl1oGPXc1P3Us*#i*{ zCdhs3u(AWEZF-T;9rnQC=SBSS!P>8fYw;m=ooqy?LK^#=MypQs1 zlxwR8>@n!D3!&*jc{tD@%9p1B7f{mWlPP#sGI0&N$ng}vhBxRG16X1jAFJV4sq=DZQ)R&rh36hO)>_;b(kQdtRQ3Dp@dQe)Gb=)!#h)GUy8bD4A|sa zThZp|9B^?&jGjGi5{(3x@l%3^(Rk z3Tw_q%_~n?jnJjtJii?=%gJMX>+RGB-<2GL+kqiDaNS<%oP=I$H~?VgHfOW6l1(Mn zz}j?QV}h4O-e!)&1Gp@3jVpjg3yD3muM7OcFUHF>*?(g>5XRK;YS-!_y1X z)r!Dc2yER5a16cT!z*$e3pu-IW(H7%-NR9e01=OL3B&(xO9bb;aFfvbZ1cF~g zHqL|Ppa4G)5s#JS`jRmT-a= z(x$jPy3@=mIoyH(Le&qe!-ET_95!roYpy<4=+4Z*jwBxzLQTvOK0A9jolT`R&CF&N z8o>#~@IHfpM22947V>x^?1Gu8jq2C7Q82ctX;^BA?tl~N^)F%8)TFJCl8)9Tk5U)f0gAh&+awoeN8CZ@9 z0{=f-f?G7UC8k)Odb5F_7j`zA4b@p2>8lxW=rdg-Ip446CCi-Vu^=JJ-14@zu4jbw zXln;ijGI9`Lb`>DYR{4MP0yd2ieDk>4m)24C)=~h)ok|m0`olLLHGcljelT^wP)i_ zm1PhJk~W%4wcD5Mo~rmV3@dZlWERD0&!xHbFAhQ4;-BT&bRolykKxM010?@-7sVX@&X zzYCWnmFGgyT8A*^3j`j|M0ilSD%W%<%xj077g>JHWq8-wVG2hW@2vPVbmx%$6dl! znOr!M7L{S67wvfP`zHRYr9n2qE^te1OK0p~+2}z>#v%!D;F54Cn^Dr~^$jnh!L5W3 zquE5)6*D>g`kqek!zqP6A(aX3R(11^_&f$jHTm2z0nTK9Q~ zR=s~b_Iw>>36D++-(_S3zi#Zv#j-LPU2j+@-9RFyBjOM(p z7CDahqmCp|htCB3AyM4gf^zK6XlB6FluV1_aO8f?u*+Z(2}e>6*|l>Mw%E~qD)#al z<?PR7!i33>yB!}-L0Ic>D@nM)N#2Eu9F|;j?dirngTJQMWkEsS20BG z7~%g_tOB+YHV;!a=nasud>qmKX~k%_iEq<1$*l{OPRBqQYo09pdgM4ZT^`2WX zXQ_=$R%D1Vw;J0LCm}ow9CpAX(=IPCXLCT_tkhFiHzVY{{WPfeS#ke}%N=qmPvh4t z+fGYwAfB4-AfKAWq4`#f{(kBZC3WD1g3ZVglnrwoQszkg?N&(%LKm^c_>^?0IGv7K zod#o@ijhmaW3F$H-~RcH{js ziulx-!#hw*nOdeFa_eKK#}@Qds&ZjwMp2XlTXZ{iYv6n3sq1&1f9>vL-)r2BYflYK zy17Cr@l2}x?NEOsol7-}t@Wd5_mP%T^wu%QI3vP=?ej_rwPPe@B&;da=-c)C`Szfz z&z4}5EFUchE-RO-b)j^lh#((Fv=tVZHUaZbGOsfsed0eJ`z4M#6kP^R5XW}P^63MO zC-0Nlog~B*qvKdz2k8 zRYf(Lo!%ZpL{O_#b-@WEBXTi87}p(n1h9pv7%hxvR#eQTD9q7|#G&NxKeWLJYz{UN zx4d}OcK-enA`Ecu9WP6;_J$tYQl{8Nc#+IOdLOvX;u}^8&IwM?hf|#=!zDW?2P7cV zAODi&1)OwjJAVA?v5)^0+GS#^A7-&_q-}ym-xeP4;_}&%O2w40<|;v@!f@nlo_Rhm z(ptNDlp((ed{elReqbd4^kI*~VU`tgdPYd&$%r_HmBV>eJ(ajyRHssRiUc-OkLPXH zXc}YK*3xhH4rkMfn#n4vo`oAEwXpuM#y~J%F9*X1;LDKGX@;(3G^JJ!86Fxvy)ap8 z>W`w4Aer~L&*#~H!tYPRY0P$eSjQj#AUo zDHZm1ePCF_2cv`ffS%QI1G+Y@3&Sf@(kgN;4HGqMuPA@Fbd3abR8;lH=D>90j zKHKZ~$F`^0R?;PYeS9nwO(js^%pvD@N=@8P#5P6=vRuoD?=Qf?%yV|W_mEuUs5h0D z3lPsFemTNr)jVgR`Onw@aj*nLQR&1Dr3Who>>nMiHzo4mAc*Waa!-C`~ ztg%^cGrTHDx7B!eL0AEPYOV3K^-bOFvGIiKkqK*zJAUY%%Uw zvF&v{Jr|asNs`*=X~oQKMxgk?GQnbuQ>v8%k#lSD5C7H+F7iPySiFjaHzPG6vf4=0 znps+292gm#JbH>u9XWGk@@V{>Z2d2_0+_Yhe-o+?Ai%-o0j5H3-KeGN>N4BX@)9eDqFUSJZ`BoF1$eS&5hVQ0DTlVKZn zKG-CttdbD%MHMr}!;x;8^6eJc!*}&hSURVy9z|)%qG|2-@ z&>bOxOOWvNfCufEy^S)8>1bh1|I4CIfn%)g9bVlf#4MQ%x&3+IPPs>BA+1*knPf%k7LYO(&07-Z3Eb*cVqr~_R7F6FaC;vj+=sLZ zK~z{cc0^!H00{ThaK?FUn?R2;!ye(zoJ|w7$&S&gD;!-5@2*;V$S^$L@&3pvxm@BP_(Ak?jiRe0cTbra zy;Irv-b=5oy!qz6XRogO?aD&9BRv?Q2Ew{D#&wQv)fa6KHOO&WXtW{GU(1Kkz^A-T3MkuOCjU zaHA;MjH0QTh48`5%(}w;y7&F_iC4e8`s&Qg17E%J0#S`@O3mcb|9s{-^3NZ=_tmu@ zfAz}ahYJ!SFgPD z<+HyeU%mMJSATxv|7Y)Ao7*_9{NbY=NXm-6+u{ZLOoIwwgc_ zXmc=)%!L$WrplIlQEX*5$$H~loQ)kjk*vg)Z%KShwrZKYSKg8Ft5}g_ubujTh3A|e zkZ1xRJu@UlLX=A;X9k1e^qlF_=lc7}0Pg2cJxK~*{Pg+6FTx}Kqtib2!YhlrQBas0 z3-KCw@MS;@fzHL{Pv|^efGkS?)>)hXHWdwItV>erw2MjFfc{={El_K zuN8xAKfEUY{Sx~SpLhd?bn}TP6N^OGEe(M#vsMQVVB6G+JH6marn$*RcP=Uj=-n4U zaXliigUINO?y7**D{rT6TI{$^TP)djQCT)#O&JO%t7W_T2a*7p-udY(pZ@BHcfRxf ztv7yh=lQF*Fa7S$@2`IP(e*n&c`{-Z?gw#g=)eB`gCE>__O)AY{NnaYmk!;2>jLZ~ z7TSp;!^#q+P(B+mIiEpPh6(n{WI$ZzgK(-SC+}P62`{lu4M=|d8W4OP@>;G74>yT@a_2X{`TGZ# zZ@>8STR;Ec_KUyy^y;NM&wqIHiC;%h0=M4y`RyP6;qULg4$J=EKX~r;FJE8BhS_Dn zJ!>U)=lQp9eemL~_b$Ld!1;xZPhWoF){Af6e)q%Mk6*cU`G@M*6ARNYz(&+zmV;em zDvqSBw=EmF9dCl0hE_0BBgBJ5r#$LnQM7O|4(NkSD*do%-X$oX#NFN~5?d=6sm;x( zE3!hOzDHRo0HX0DxMF_=ao=Mz4tjbkMIXCfUc(-p36!iB?iMH6IRdBn3`U$hadJ4I zb=ixzx=1QZWbFA<0FBt0HHqzd&r|fES+J*FWnJNhi5LBb=WdyrnZBcYFV~zSOY{+1 zZ}>ItZ8to-p%6C7otVPS!lKmWEW~ZgYxN3MgbyR}Ng10Ag=A@6FK|5qYZxAOz&oW2 zYxX*;m)D+J;eJ`eDrHI>N)|tGU__Ki=zFA5fc5V-utfx9s$TUEd>iIn3vp6b4_O73D+i(In5@Dc4bi zqA-xeR%3Q{Gr4;Z$^=8sYl)!@{+7e4wA_c)iY0Oq+e-;p!I)S*g{UoL|AQf8Fs^jz z)d?Y?!DoH$I7*?mS!mB4xgAqZVBP2Xg+J}og7A=O;@)IVW7viVmfiU+34djKBMrdu z^AK;rXhu|4k zzBxdS;=@TnUDKm=W}Uj1jm2KjGRA#GsfO?a;b1AEgwSz$n)ufi0RVthz`xCOp-%gA zarNTjt5vM93}DHIsWS!!bFmc8V$337fgi1qyMF$Z+_}Y9kwKU#a#}rBJjuPf@DELb zx#leXGKD`7T|mjhD)6!KdJIueSk3OJ`(`Od(>x-4QKQI7=? zOW2x59fWYAcz_coQA7DUiT0xk3To?`M*}K7>m6tFI}w7kU@#s=*t!KWP&zrVRv-vS zJjXG7Pb!?0+_G@-d975=7@Cdsf|{rnYw8ZxEalIIR|XesC!I7V~8Tz@buD1q)9S_CK$tE3h=9ZN@1DFDN^}!GxDxE`1dO_Ll!dyKa z)Ok>@F~`|Tz|}%u0~1+nCm!n z@e0LwGR&}Y^m!4pmQ~?3i$}dbzsxZy!u{Ia3d(JfB*&@eEx!G&5Z%6gavX_;IL_alJRW_U7OfO#`8vR#jG1ep}co4waJ;5quP>AGBBu5=F$6)Yq-8Gr? zgwQ`A0zISU(^n+HG%OMX8{soev`Qow)6j~VYQf`0V4I;kjr}rU7!*Li1&Bkk+lwB0 zf!{SN?9d9jiq~U&$t?xA;u=371-lQ*5&54}1H!I_lo|3EUB#1;#W4FGdklTj8lx5F zef!(Ho#=CFjY~Zxz3B)(CWzo;D%@|eonWEBmcqvG@9x+Z2qf4rYf83nQZytki}@BMe9t!A-y`WAJ{->Xvf>vY<^Pu*2@>ah+;+F|AC% zl`6Z#^=Ys9$>aYq5z#xB7{*V+K*lp4a=G+IQ(z??qvuE_o69LW)aXrSVXV()fs)H= zR(G#g+)jsKHof>728CcAx_(H#H4P68#EAtJ5kx08BorsZ2J_0!;>$iS3q+13Gm9St z)50Ott71CH6$?&Sg$tF1=fU#@I}@4YSRWK%bhvL{KGMWOKZRVJ?q;&^22!~i>4Pyb z7EKPTCNIOP1J)ks;>RAFqUU6h+d`|b?dvOB|5~O|2{9glLAzm{APINLg^!)HXF<0u zkVdWPtQ_}(8nPf)&Ys0j$`19xA(kH;VuE|3{o=gv-Z zcBgognsq!ICQ*&mDVbq)?s9lm$rB#jsWAtE5fS0vX}?A}r2{S(w*}6?)?K5E6?olT za8EmWPLvq|-Ep3h(1&xk4xj6;)C7~`%u8PPRzfU0F4io&3d9<2#uR|9>0Mne@f&qT zqlru$2UfxNxEtXrI=D$zOr-+^yLHi0f+GiHy?^QSAgQXQ;P*=T)M@J1U?2!oJH$bK zCpV~$)Zv0Ev|?gX1QaZGnWb)Dx)<|K4dFE9){Z5f4B7JA1;ezrrP8eL7?&&x#MTO3 z#YnL+2*0>^iA1P+MY3<-KHU7T?;)Dt7cad2#f9JCp=X~bU%c|s=P$oVqEA1+@FVj1 zbFY5>%lF9tzW3%oe)Qhwzx;^&{U;OaBFMjwg`N#KP{Kt=;#LImC=8K;{ z_ZtKv{Ng{K{Ng8Xe*VjMKY#7b{}*)W{}*9&PHAMAx|})w0rE{*ZaEo*6$$5i+T$|i zHQDq7R`e9VbopGRhE^Fq#<)tsbdC|OA>_VQ_G zG*eGqklM-~A8a7Wyu>2L#dIY=hQ6$BC?z{GRr}RL#AWjLpf&ErS9^3CHl?(jwf z7GQselFmJ1ONVKIa6q(5|DVJXL}WZYlSkuG%jXpvD5sDu?ixMTlp`^+^x=uyL*uQkmkTG@+oa|jukulD^ zI&~QWOs4`Wz`6@Po7eQ5q2*N_r<8%3G0Q*=Y#+_$V><`~N2mvG=5**qum4&ANnhXv z))C4>lU+h-0m)5DK<~RE8#|lbaF&QVBcpVVk&@G+`FdcyG_Y1MAWYZwo6a08(n_6f z0So?g_}_-*L(yJ&1(z@%GaqOcGnSUkrOpTL<)Y-%SPbri4Pa?>7iGz$mI? z0XAIFEz=;y%CuE}f?0&wn#=C-Vxk&zMVXgt&Q7&Vn+RiEFjHrVPE&aDLXdh`u3DEc>&49G3wFg3{&H`Db$o+(CjX<$8~`;2d2bqb8Q=sM96;U z=7sCM%z<4Ek9qUr&u?CQ1#kZ1f8D(JQ4dQ%H#;qXsXu>rw)yAp4k&h-A?>t6K~rFE zbGi~ok1Gnp(5$Xf$F0oB;A+~6%gQ!-ZxTyh7xb@_HJ_0|j!b6YOOORe_9T403Crq& z%(Hym55j4(K*K;pM}4eH^bgJpZ}wo-6JaAdjG&Ix|N8LaYd-fDuO1-B1X_(U5v2aD zBa{a`y7W$GWYQH^MB`c+XW~gbN$2L8kJ9q2p;T8JmQpNh#jKSzu_w<=-&l)r3^5bT zBf`X250Mqjn6JvLS^ziT2k=tWBOJP+8ZXnTkhGq|(ajwKVH|WtK(R6#ktx2MVtMU# z`%QhfDxu&BHZXeUV`L7|gIdpq$ZN?qU1a1+gHdu4rdEwc75#c9lTniVrrEnt%G~e; zK@_3XFG$K>dlO_Xf~eEpJSeyCktEU4Q^Uz^s4Z=#H!=5`?@Hsx4t>L%|NQMIzj*dl z^7&KmeevW=eSkT*Rs{d}_wW4U;?KT#;Wz*L=Fbw#;-98zO`*!R72N8W)oZ5Y=SdoWDrc!dmkbb2~@WkgRFh~6YsaD2m;Ag@{GeI%I_G1GAxxO7`Xre?P7 z1Iz7r^V-g6g=}m%y@APw2lHx%GpDj9DlvlQCqZNM|Bz@Kxw-4KR|)a%NLpmEDro%~ zcWO2e@QTi`UJdaBM^AJHDvn|Uy;#%9o5^e0oTg{se?y5?q%b;dTeFN#qi2uBQJGgq z#2WBu@eSmTBf!|%skEz!89p=LiF|kWOq(g@d9(p*9uXeG07A{5r@{U?(Pc;NYrrrS9!r;t?_+Li2u zn1$>XHiYWwXWTsX3HON~&dl&KkJy)Xw-3Ko3MUz>lj9UFz{wz;POkQj&_<(qnj$+r z^HlJ%gX;tmZ2;5BaNy5TMPAq7WLHLSA|J7r@kE&3!(W;(?1t zU<8aJvkE&PIpl*H_xi2Dy#)sI@E~~@uhE>Qjzh+n3!{9<&|B;&8^gnViJ(g!7OOf< zY|-ON1TC>CX<_~`o7s#XKNNyg-GEF8)E@oU5kKfMPL43|yu+Kxr`#a*XIX#}o}sR; z=s#USwU*oQ#9^44)n!H|;Z?kO@tKp zUW9)?mFevdQfR{_!m{V{x|+A?&33r~*QJG7n{JsTnIcCx0_GttN3DtVA9!2e9PsO+ zdT@s0z=5G~h)Ua*XX)vXoEu#kFO5T*OJeeMcxSf9y% zoSr9RVAwCd?xx`3IPAEr>WN2%ze!o!npx;9*q~ngiMjmL(r=co!O!=}(v{^Wm!DdG zX8Gx*>r3w}KfC<2Jhb%trK_-myZx4$TN%sJYa|*4ZoCddxrCdEg*F<95Fc|vEwfSD zRbw0ZRz}NNsR=uV4)>Q7#*h)jK}gXR1ciiZk2C4)(gE3N+``UXNu+G(5bQFiG?B2XnK;SZ4REibnCIeh@C^+ zctnL3p%?k>#E~G0(KbfvxkOXecKzGfp}V=u(W&JTZpb)%$%CJmaP}lE6q<>LUFKC6 zUxy86@wIY*rDDkyN+h3Q=5TjbE)pxRU!iy8rm$h?#u7qBt|Et~ zU+sK@%u0p3D~+ycc-FGUeRZ^*RVar~c7e(tcB{x%)viLx=QTZJYq|mdXB1c7Xjcmv zF^h$4Bw@@JQX3j=$_>4PazjG7s-zpoDl-y>4vevTfnGxy&I*MHu@g_aFXP!T)EtBn zI|*T9n`R+>E3{tdZ+L2l&N6abl#^@0EkUNXu|3mD-<}zx4^6@2!@6;$v220Yv7HR_ zuU~h>{0^h-rYwW+!0Gd42auP%PIg_GKC4|#WW}r|0;%S-Y*+8k-VVSauBgM@3UhZZ z24^E3X0DLhiUd2OBvtu3#5KEF)s2N1l8&fJ9j8eK;qr+EVPxSkni;D&s2?6F4ri@Q zad>zrR~#y2hl{3Z6pGoQ5e1$@SK55^xP3l0;^XSw2rBCZ)FQH>IbLC1oQH$-QE(+? zkkrVWGct{H>p`hJ(rqJpVKt=N!FiY>1?z;<h2BM<<^6ebD+>R$eQ`a(frvPJDHk>)LitK;ud*8kF z@2}o^>$%Uq^WG4Q?qTR zn6d2?GpAX4Z0jFhU$4TuoV3XXlT>ptAlat6DZr*=`PdH4$6jb`4hzAo5@34po*Y2~ zR<3g@u0EHxhh~7bvkRJDPzs97Ogm;Jx8n)Pf(hJ3(w3;8=)&7KFMf3MBF3!<`Sd%# zx(Rdc!sVM!T!EQ~MMj9<@WRcDPh;5;oOt2--#`3e^y!^nd~oZHpWM3ibTljbY<0Zz z^LPIK-m{c@#En4GRx5S!z|2RAQXx_RN3O4hs`(~k?TXJfxzE|;3!64$-4 ze>pS>6U|{b=aBq`mzy$emX^ic$Ye76d>IPE&3guWc*Nn2gENk3kXksswV|-&EIBr& z67yZ|;9YE*r4{fYT{1b_Bu9M=CiPp>Y{@&v8;e&RS|xHYuD=!Z97+RpY#Ycndh|ah zBw&P@t49c@y*L+TMV^0X%+pE{Y=vnos5>hqX)BSaH7DYcvu7VDl3~GocV9s27z#l~ z38uA7s;ScpQ>Xi{UJWPZkZ`8q`H_fLGVvQw!tf+J!K%zv8sQJFQy#EUbpX5vSuLAS zT>>@2oJt*LB=?C316cur^_L!tqQ$uFGwKUB3iwbGCGGv12p#nIxcg}6HTwYJV0|)| zBXA>#4PWx%`#$la+H;rAO^b7hB|GU*zcwPy>6>ZFv@ye*vv!V`@d)<>h=Y?qUk?0m zw%=O+I&XhRa=ckgJn6aAfdBVGgl-ZUkwCRLY}yOmkmB!tzS++9E5!{D;c z;_F=w=HWSvajZuxJ#?zer-XGv)Tg(lF)z6^CWcOuBf~}yN3wKss?(D!xv=^p6`2yl z$?KYKn&3)eYn|GkmozVt#f~t&D=9k@x7>P8*UT(Ai0(mgXmX+53`YE>8_;tclpI{jq*CmUul|8U^IXU12s|9LReHuc?EH;G|x^O+bFs7vCpCd}d?#3&BdFIBOe|h3B z7jFFF#(OtDxbYsj@l)7%`^Fz{{NXQ8z=1#Bc%R7K_u;^E3S7$(<}`|e&o@dJE3u>B$KlN+!8<@-0@z46DtJPzM@ z`^I~Jc_#6D09X9ujo->g`tFT4mDoeOc03o8pCOA)X{Om3=-Xq9BCLs&`1ot5Feb8? z12N+BWJsKK&u2l`invm+i`Ay4uVFR2rPj*o%~z*X;ae8rdfqBQSoko~Kai_7tc~E4#x_81V!$ zc_tSDvLt|q0H%FTQGE)9MqCDsEVh{GcIxS{r<<8Vcb3#aNtd^^naQ%+ArZM9Q?6Mp zV|A7TCTX*kjkNE>&!?83T)u>zwO6Bd+okuHpIN%ze}yw^?p}`WmrL)!PaNU1(ScW% zFD<*`0_TP8e%83%S1DSKdE zRa3O$Zj~8tn_Ax5+vt5Di$;xBDVYP80_#6^iVP)-xfsj1&zH7Md}(H`UF|2gV+tj& z6}pz3B*xrv?g+B~RHMc1LY)WtVU_7oQ6vqZaau+sEevng1V&6{dhbwa=rAc$r<@`p z5nTu_uCQL@ZN=)dW14+AY>TrqsnyxNQ`p=9Z-A5l(m*Z09qP^Q92GA}!`2F$3O~e& z4(+a4)7i8Q17{8yYr+#H`;Sn^5%Y|>$z$nL9&1i>rr@iYN(o*}^Ero2QhFwJvj09( z`sQATQ+}CxWCCFnoJR7No}O6g+1_Ene0HPPK({o*+Vd88WcB$0_`WG{eS6q^X1f>1 zjM<9=vdC*zHnn+int~r8r%3PODIsNJBTHiwxwv&*5vrBtK*mY&-s~RUKxz} z)^HHK6f{t~YcZKhQ3`Vve3q`)cC^ozh zNg~R@)K3vb7^lLcilj11`*SAtYGzrQ-BoI^N4>`ebN!|>N8L0~ds~w>$RYJvE}xn# z8bQHNO%XfO4@(R>a%tVO=e>S;1!S+{^cGml+lc6#2VFUzt$7{Y_0a~z)S zg0dau%dB0$jBS`OQ^vC;S+RoB3tfoY44ucmFWu;zaTONdz&OpOSE@+J=*i)`Gj+DC zkc4`(LDgZrTmv&4ekR9^80J`Wp}<8bsf@Upj67yCGMUt&hA+hfE$+kr&Ya1eJqy24 z;g_7I{BaC|Ta7z`E1yUzpC-AEr{i;O2n(!-!M`-|C>UH3t4>s2O54pprxI>qPg1gJswZ;m zH(4>^oATIVX2lOsbt+=R^eTOfi+w~=1Hi1GsEIVBS-rrq4F#V(_AJ4)RkpL9axaFV zm|NlH&;gId1%>8pQ79@!jcXox%*^!dMfajirYV*V_rw6bdr|>?)t%r?P!(~w0W-eB zg%7qJ4J$ApaK!qsCVZGlUy8udN^*f5=d-X91s*mkZ7P6nTS-_=QXBGxtuW*Z_YL_@ zL%y&(40#)D&93z?_FIJLfYTgoMp@qm%Q+r^^}OuyhRm1E!vwd9D*}@1WjXc%CP74# zNJdTZ(W@-x^|{PfaQIP%Ow zbmWuAFSm~&-iO?~wtNA*k6~x&ZF1&JbZQay-hvO`S-M8#=9BW=>&ur2ZbqM-JxkzP z&-DC~dRCul{-s|my}kU@(v^=dFTMNM>oD$T2zH@=d|4S3!F}1T{S^L89^)3CQz1iP z1M>tb*Sr!nUDGIHZ}`^OTDFjSdQY%#h0G5yaOxwrGi0Kdqn zdG_oBeFyA1?9rJ(S=w>8I3_~43q*jM!g|Hz41p)8R}~G1o>!|Jl}`JZjrIkZ5;Rh? ziEz)u@CtQ<16Wp0#Jszbia9`L!67CblNbd5I+?-x<6SSW?y_O5P&j>jVjKrD#KEL= z-N-9USji2mag=leT=Kd z(yu^Iz7~O3z|w$=$m^i}uSQ3%z|J)o-j$`_w-7FtpI&~Vz4O%4tIHRct|w2$)7e>G zP;$O{R%sW>#?1N_ki^IC!k$UME&>{G^3>4rvuFSBM?d-dd*A)+iPvsjdgkU6*Y5oO z>SsTFy05&@TR;D|zrXj~o##J5euj&WOMZrDZeDy1xkD~ogWbeJI_l1Fj1Fe91qCA6 zESPP~tY{~utV}RqJ7pCcM|4OCuOeJ`@in#rx)2dh_g8jlg3s~As~%`W!QtnAZ)JZ3 zo~}qyXZ$eZj#JfD`6T9aDImmP-cD_^g6Vh_RKzv1)_7fh2K2)_%TNFH`d2=F`{UoW zEZF7el@G{%{Nw?!LcbQ_1T9_p8hrNF(zT_lUjxna6c})C!%6RJAOH4&BKca|P6a)5 z2|j!WEm~>eJ&i{0^`)zD5N`#v6xex7?|gi@&vN&0+hE*F?fL{fflEuT6Bq{=_8(tP z{Bn$7KH2ph8VH45T*2mu3(Zj|nueCor4E9_pX;-=aWZJ0BT>EV{tEYLeVSK8Sz4NW zj!sV38TZ7zvfL%7ktiVXQogJ9+;tMf^}x#HkNP`a@0Yl}@Nfle#^FEzk^R+jfTBj`!#-T`P2@fcA8t8(fn*2D*h8)^{w{UBC=PHrG{blpxlf)Bfig8Kn*_ zf-4U`7Yo)yg50te+^thtC6}EOihOXlI7m5mF9{4wSjhung^Xqxwq|6qnyx0d(}gvx zLNA!D^^5zdTcwWhSZ8YcF!h`!arg`(<6=cv6pHN<($SM$LOSYJb_uHgBeW9EF>KU4 zj&$o-91#SR4Ap3zdh}6BPSXl+(q4$h$4i|#@8cBP(jz?F714vD^1(PfR^ zvdbD92@etL^wij@tDS;o2cAk%qb_(ok*ZG~HSeJuIAAe;&47 z8C8nLnOIH+7CL4u&D0E)=rrG!RX%1@MgduXGMj43kbLj~D+i&+`WF{^q8V13!mW}c zB49m=tiEjV!G%XRG(?9svWnG~uwcY2W-`T$saaNR*!&(M845j_)pWRwtjluNuURB? z;chFW!TYV(`D6*Ok0)+jdg0TTuO^m6$*p*7-qh7L^@I_#ND^7iGCR$;`^@Lva*V-ze&^Tk zfA-Ob2S2;=6g-UESAVQM+DP7%wpsRbDODq*ndyC^lOjMw->5}BsL_C&Ia3_r4l81Y zG-3dQhaF(kozNOc_agZ|y$1X-FgmbjcbkjbMbM^7A{ z><7JiH-JBq)m253$ZPp*r#;VuBw$~D8QFZn5`{^_&m$Ho;SSMtm_Pyc{etTydw%;s zNE>$hj?*f*gai=~sas}_Lp^rrVd_M|O7bwTf*X*$^x_dwtv3C&WUC6SQ8xC{Xqs9f zb01PdG2<&3OP>)A-$_Fr>3}kuf#08$y+?t-A~d9TsFE`BeIh<;Wd+H4*Uzku#xYP5 zh>d~;yCX`vDR4mEunBpmDg-rr~&}ztFX`Ob~i81XiN%UBVK#A44r(4z` zA8esCwBM^&P~?$_15V1qLYUh0Jz+KywD)aK*l-hP#PNCRyPVDYDTohxb%iNq@v*An zBRL?KT<4sK?PS59LnYOyi9l=}eDhr?B)aqScRu^!wAQS6BB*iEs4=(SU|FWPa28)mEWU177RBg}ttsuiW_F?Qrsg9li$)W^6d*u4 zVy!npvK5Zh?hbCQ_L?0a#A7q)jKGlt@)a+g9wb$jeB_>?sdcLiXX{ibA8+DDu{vB$ zO_8+C5+kNvqGR(>HkUf-6-MyqbJPuK5cZi>G8Qc%zC<#>)E2h5NmOrlCGk}XM4gw( zVR-pK-@JCI zxLQuINQ3y&D5$JPYYu^(a7bDOilJa6_mjoU3}dHxP(z3x7ARBAvK1UaJLZC@W3I*M z;*;6aVT>6^AXJxI*|{(Vh{e_lU5y6o*PM+~?ChnHx?4jGz4EfH9~0bls5cuE@>rUY z$$;ak^_V>a4|d_-JQ%WRRvu7MZXRmsS`U#EVxJUlF7t zNWjhoLNarZ$$+70NXFnrba&EzNL=K61W84N@3V8ZPFIxpe*Xb7pF%2{F&SGW8kpd9_Z_VLE#BsG6}XC<;;T#FS`GD|GdwCu7Ayfnxfy z%%%nPFv{ED<-5gswLR|nF!gJ4Fo%vXr-9%HE2B$Zzi|TGoi$5Wwk@e_Y&n?-aUb3`=rU_KnB;T6_0Q za${=lxr}C;$TO6bJeO3AA2*N3$8(opP0L(g7;3aWH1J)bK zRJiLS>I>d(E1Ypwrri6A=!e=jC`t+U0cfKjX0KsZ@J~jHB ze2@g=J^q2HU{mH3ebk~7Y!+kd==S?HRG%27twq$>s932p9o8Q>Kmw1JXNkkyYEZjd zVZp6ZuP14P1?0~!i7>JJj(dK<%P0A)a1^@NY}Qtye=%pLfgq52E|8 z`W`iQ^N0qqmzTJ^Tf4_N44pY-LL=iC4e!N%sOz1Xs@9VrlQo52OvnCiJq!QmH3bbp zR|*aFxRr(=o4l6Ermnu-%cX9U)S)xmTFlm-Y_X>r0(KVmc+)EQGmVXp8SfFC-ZitIbMd(VC4S71ViRn z!aPrScg=xIEyoSO$|t0(WW0WL;ECmmPiC#L*>2X4+pBM28Af+)aFR~s&n{g5?BaW$ z{{H&ymtMYg`GbC_Z|Z>LeD%AxU-;FnSFhZ9{a4BLoRu$R@_CgPvn_=}d?^$(JCoWp zG&6nOsBlcwTyh#qGR3^>u;jIVTftr<#b6jP89M7S@B3Y;{H|Nyec{#{|BZgv?JLiH z_V`OX=XV*lRSm!}s})SK?R67;xm@bXm}sggYIK~5hQp$Yv1AeL5JW^80SO!ZPZL^CAVp z+P6YRX9s0B^opWY)NE`eYnhszPYthjZv;hn=xq&rSE^8mmDRFc$QZHljAbYUV;op( zBZ?y)^UGLSapIA}(shpSH`=O0`P@lX3oY^TF>$(ztu zACHo;?wzpSAYW}D-Qp&qS9{vxWapxib4(h$@dN!0q9-GHH*(*d$ zNt|lP)?^R#BO=r?B9NKGZ8E7lW}Oyc{VrCLgI+kR!r4UoMR>fRBWl8t>+~czC9@P{ zm7=|s2B&FIuB~0qa=n4ybL-*0gQ&1hM9X&N#**m1Ks3-mhxHZIJ;V9|Zv{M{g1!F; zuP$A0>>;Rwt6hdA!4I0?k@sg5Y;qPHNfwa((;&d)?(}KuARP1P;_o~x^H${7n{#13 z%7sXJCivBx!o7}m`e9Yt(BUmr5l|tha33rF&@+r)tc3c{tFO_0@Vu4yf6Qhc_-AQ_ zE~1jX=314RR}R6-3%yyMg86Yvd@T4^&!N3(NK797PuVi?%!s-nHLqDH9NYTsP(tGn_=qDiC1@?Rtx$L!aFpc%XwuZ!_H3 z&J@@u}d_RKze zNz|K1#`xu@lF!n0x#HLHq&IM-O)Ms{(nlO6lv^1zZ-f~|*trHcEv(;ad$y(Uz{)Dt zeY6u#cN(}JrcP#>R)3dSfoorHk|9Y686-DKdR1Aq%Rge4ewYXlZ)Q4Wu&3d;kcZB# zP`8Tc;wd`%wg{5TESee3>WWFwxE*6l*Op#~HRai*Uz1MC`rj@;0iPk^^U_txU;i{& z`XwrBskEP6zPSACZZ{-X?h}^JXXPlbw1wIISrLZ(tX!*KJGAbQ4=ufsSlDCKqidZr z(_V+Q(B9BJGfmAjIz#TURpHkrMuv91`kMkT*omy&mDQ+qq&Nh_@L<}VXB7qgDzP+< z(WWO{Q^i?RFbeg!7=@;U5a|N#OKwQF#)~%1Piz)`l7<3GsL_27RV`}WNPu{a*cab zzL6=?!d@Mz9_s9TnR=T6q%$P}Ko>H{Y}^z+NaKIs{P7n*dzXCv&b$Bn-kbd=lh1$i z-WUJ%y)Sv;jyyp z_dj!H)tns%)gi!z4CvXjWS$2#B3!3QPL{?FlZI%75kSB{wSgl?M;1m^aDSzgz6%?Y z><8s<+q=EpvV-E9GBS#F#>tKPK^g?j&IDNR$S@pV{}r$hC?IR@9NHB^{m^M%;lk-N z8&kRS#NzA5y0R6IEZ*$35Jw>2y`&{6EEEl5*d%IME|M|f2V`7MMUS+r@k%R;c?+rY zG-I=5e<`enaJrennw+-1V@KxB0~ijLc&{sHAYMy3OyWHEqM9AB8_G52?eobnZSHyv zzM)~X=20eoA4Ub}H94o%37s)}Pw0%f(OaRLpwl-)GkTbMpqCv!(-+Kc9h0&KHxIAs zkOnW;d~&rart_I^DG|zcfHLl+)U#S9cfW=^l>;#o%u|nTtM_0iH3~-DA_WUh)yz#n zwp7t@@f(a)no)}wJ_Ic4j#9>`ThE*s3u_d(aJYUqk5 zs8dy0?>9>KShX*>OHE~Yd`8oCT{FzAW-E9}2MawHw8Dnc%4>SIGg`JT+`+)9S$dw%@-^bR{ljPO{Q9N8 zzxUnSm%exB*U$Bm$+7V+nH-}HX_)W4L~dUE{>_W;+`Mr8=EV{l56VLH;ftcI2I;a&ZiS>^!FFzrR z3*fZ(vCa)XdTl{HWBJ)0XMm}neW1^xAxp2o2(H37o?ZF?3kl%&wZ!86#_XD)h@})z z#TKkBO^^HjR~r8-3~Hnl`m*YP5>LxT%( z#u)2Ph>=9n3cMzk#_{Z|qDpNghHS@8nu08H;2rO6u!nLJ#334?Pw31HcT<$eZB0Lg zjV+fMYcuh`sT!-ddMW#1dUWJeM49ZLdYGiGNdZ|8_Crj z!j^HEaRMGoSm)&kSRnoE&}}W(1+KTJE9VgC#5F-)JUDVd0!o%n4IrRR3cPcRkoUEq-Z{ z!*J=5Y@e1jtQ&HNxZhU}06&ue`31(tOux(5S=DQT&{dC|Kqyv|qCh+S?*;D)k z>^`&EtYD#S#TQ~*@m5~T8Jmg7#;o6~P(=L?iI}E7?bB6m;O-Rf8ErYr?RbK;Gg>Y^ zzw;!ox+uujk%-?}j&g{5=yiG%jQYY|Si5`nEICNNi7NRtaskUGw+fcZ5SPub;7opV zV3*I(sWrxnBrZEDD`XPOvqe^|XkAmu;F>XWxVFsUZVDNBP5aTVpImnXbtcyqgxroP zh=OKy6|L_zUkv+Fz4(_WKV+9Xkl&HO8nr55_$spFI78_TGVeB?fgOoaKLhdG=2) z_+;GUehTw*s>aBJAwLJcZ<&mXfCiiChy}UlG`lO#7#rGw2EkSWfZiny7?$D5lVi+v zIos!xBaEi#ohqHD=b7V(f2xC(QDfNX+!jQ(Jq5F|r=XtEOuf@syw9-LEzTGWd!JtZ z-R>K`CH0Hx-NfYdl9Z$j!|r!+_s zZ~p*+6}4PL9zad9B+qVYL@yEB0b+m|7v%nW>8n zT?+zRy0&y>>8+11FTL^Cr{|GmKZTS+6#F0Zi-Hs1+X+7Hi31Whq{1YZ zCVqu_1Qrqa7`%hU*THm=`bmCG{g(U-6)d9Q*dOI;x)48{SI zf6Kw^kYl0C-2?CqQs?Iax^PaHDyki39-B1` zU2jXr&f7{8M@%%CY|+wT+NX{I-ir=Wz}%4g4PM?!4++!J)_p+26uksf)CYHCn;QtS z3N4!4jww|{BTVi0mb`Ke)=;+!j&BZK2(dOphG^4ajFjA<29i4z zeV4&|0RAM|9}Q+V&(9wEa-Fdy&r2e3N11{>;PYyoB9z(Dqet7qOQ>;D_3J&O8~&keUNa0VTQN0Vw-(m!o;{0i#9V8Rzy;uAB+fRm zheg*~l;ndW;|mdR6&XQ+ka6Mj0A7-RhVM_oFlPM&E2o|uU07*3JUJ>8+=gk{hHb*T z1P6D0$a1Z=A8W2)kdYD?im-b`)EPpUCfE3@Perkl%uMfB2##;~64{z&26EisvDmt|g+A7-UrJ^_Bx;-&EI0ZrrE1dbO+S)ez zL|dmI_f-eH^C~=7%ov*4)xwlKCT)-w(zT3D`w?|1#tcCdXY(0T&*w5a>ST;J8=0(@ zx4_Z^+xa2a7ok^X{l?ug>aivnW+%Asu;e8rCm$LeeQ0b_$)PH}=z82(7jdfrnL+B1r;)m06O^I@Ab=oA;3_sz>&d2-!wXn>z$@QqU0QFe_ zDa8_7DuZL}%Xb><{OfcUT!O^H?q-q!)4x5Rt(c*v9A7Lxr@uR?}}>&gWCN^q(jnu;qk!_>0Y{V1>!1)gWHMkc2i->L|34m*&I^n5{k zIs%N9h5ok2e{;6kws@kQcv8g<)_=#c1bJ$Oe#d!-z>GR1jV8s8GQRTAB*Fe=0=FOa+hnXhZ2DX)@E&7IzDUwJeohS& zD0Z(E+wiq=n!PC#X&wJDx5nox_qG3++fM(nU0)p&ha!jhZ!U~)UAENm^G-%Kmv5m1@JYfATG?9^m&=+xL)Y1_c& zw{4afI#vyG=2Rq?8C(a(GyzXv_JHqy$mO^&z&r}l)JGn}KYSuAQzZhRhub;&G)xpuVfF*4d* z{GD5cAHLu0H^PqA00i8>fZIZ*BRwCxKcrh$>i=3FCvSx!VYq-6Q^ zLLrN-0}bv(eI?YH4(lq{EPB;LtI7RcKixV>O6$`C9*RpQM0qxaWmppxIM1zkutqAA zO9^_$4S9m-49(mW2+!UPWP1Y3&>d24mL1`4FGXTD+eSN++cBkUV_j+5%u!^JOr2N& z{dxB6lt*Fi)&KV4PkE7i^7xC7J$B@8AO5(3sib$|gMhoF%K!GkE1XO(z5@sT^!>$` zk(*YK`r`Yvb#Q?Mi`QK8b+T{YzR{z9``{Nx$;d-flcf{n!H53#;rE8f=+P2<1n0Sb zx>$$vj7+Am@~zq0;>%??_AqAwNRZ5gfsfB$0oz_Vzqa@rxAw{7zxw3ypY5bpk4{Pw z+39|Q;4W8*TU&gI%q_kP$Nu)=bLASmV1N7Y`(=3IR1P_Lk%%alX}MMtWo32F*p`BH z812P;(ej#^o^EbWtC7&<{x&y6=((%{>?L>NiPbPQ(@G7{a4%$?C+Rs}77itkGOr@s zZ9frHJgm|5(U{$od6~>ICENEOYxXhJ81I z3zc;%zTaY+TFywF=UsYh>8h-2J9B3F(()gRuz!8|8M6EgB91OU{nzVX`S|URf42}F zUVdKrfb7Rl9sq~tYjEhTrE5!9zqWJ@v#d*R!)NbnAOH4&BKaCzIQnvN`RV0L@F9G8 z>H1%rzT9t^q_4|SU&o9&KFoJNzMS}l7@?974}3+T z@TAyngV~O4FC%MfVKdR$Ks2y)b=m>Xt}o{2x`Da8jhJ_up0Tt-KCw(jk-~;5G`hNC zfW$UGEM3#n_gsy(s>EFpup%i{!0u}4)^MepgMiV=>kaVanCCP}Sq7c!5$K@Pm-=>H zc|OctkNHgV12zw~re-G>YIi?KKHFYC<#tTD!n@Ut!W~=h_BT|FtO0&{>DphfOZ4eW z%TH}xMUB~;7|+~*88450&z70Gq6X80=U~DhV37ls3dX%_w7S+k#FwiOV~H@hhk-bq=01+N{fzjB?! z6^hhgfrGgR79<%vHQ8sz$b_U9uh8TrghO>NlW$8TaVt_mLi(ns4d}`mj#3XREj;kN z?k5c6B@9(sme^9WG^2~9t-|8ZqH%BWRi9U6nU7vD^9SwB0|&@A<{6g$lhbf(vmR$v zaMCK|cmy+gTpob=R+^sfHw?Hxz3Ic*Y!=JQ28YYrFlrQwlL@J}5m~Vrv<plBfmjF@&>fw4kNP9B1pJ1Spwpc`Z$YQdrF|ufHu?H1;^~u*obHy$z|_ACtd+>X z6gwvicw0g#VuimWluJk za4D?TBS!~Iq*Pyg&EsVW_$%WF$rrm2hgD@xMv+Wgt;N=9rmYpax=YG&1H^4wq>K z9+^+7%&UVr*6V7mYe=|X()z>j?dT7Et2jx){e|O>sM9`6(M30n!U@Z&C94vnzM&^dWc^K?)yG*?V&La(K!r`-7 zW7p5K9wTVW@k@u-@~sY~QrMG?HFGjsD73{J?ZlLCPRpCI=Oz4E4`$wsZ2OrK^Ri@f zjh%ySF4J(+u5?vE4+k|QcsUoWAVCp4QFaE_sqd3o)0fHG0|eZd2t~?gtS*uL;Hr!u zc>!F;74tQCjSi5|bwC-coLO#_m-}L!>-LH^><%N#&gdH!k~(rRnxJ*=^&(v!IC-qo z+`@jPV_+?_rebqtHN!GAM6Wdz?&WOT=*q@6y6lWr*beZ{8Bf%S+KoR+>jCxpB;v5K z3YQ~k<3n$G6hb#tZgV3pw|N0mR|Ru3-X~nqz*&lY%@_;(>LKzd_&u;juXKB_z{1I= z(pS0O@e)NK1UW<2Lc&mfCvHn*ALvePw&wG%EG7G=p-Mvw>)xcs>PS&lX)K0q})IS9%pmdOcUj_-3poI-mDz^TxB=<{-7 z`5DD72?yA@)?%!KAS|6Z7{AFX9(oM20RI z6?4pUWy}6{7GE5+a=n*8IKH8nvzy^*iHsDRd;^>NPB2;x88Q1or>>oYattShhuB9h zIXQWnj0h1-Q^x@Zh^C=g9;N}(xRf`Eh5-UHz$NGk>+Cfn1>x#Ea68X&(3M+Zog}Z0 zK~Y#*zL45G`=o zBiwQ7!oy%MjHh{k!0o!s7eVq8r^z}@#%a(pN^M4=ooOn>OEjSjb6U0wB_SbhA>tOl z`^n>%mwvJQ)bgdJH?UAyHk>R!we;@tGjQmw{?magi*7catv;6zP?zIW%nKlYz)z4iU8w_bei(|3N6SPX;H*-RGxsamwR65U7+*d&&| zDk>tXC=yIOpe88`j4spqG_QtHp@r0^6~Q9fcr*c`6S)uPw!ZS!L!|700{Z7&FS{N! zGtn3lOYdQiD%cUG0_H2X<4qzft7W>Nw;DlQs}nw=i&qw35v1f+JT^}zYt(CygDt8) z?(hn&NX}xoXI4kFwK3}XH7Zk^J142ciySUfWR$WXML4v}Ph z(ehnbSxM}Okq{9%Nu5h)sxNRLI}in3Kj1+aNOrDCJSB1_e8ixlp@b|YlV#4*%)OCt zHmnPkm?3F>Aj-(r>@M1sJ=jne;KZRM2&Z|OyE{gO4ieFI&CH~BdC{V#)6>YoiLr}5 zn=hl>k-c+iV(1<1b1CM6j)cUU(O0qMoyL{he{Kq;^88nJesZ-z?;lxG+lCbn{5FlZP!5o<+Xf1HEq|upr|JP z_ThiHRpR~a!ylK)AgwxdhLvlVfcM#p%Lj8g#Q-6)sqq;{cstgY zjZy(S+TKiU1-SI(TK}0_dZAz>k5@D5l5rY3fLqm4Sbe+@m#YQaPc)#7B0R;6+vb&Mg%*%zg4~DvU5V22e?}t_e9mBbz0_&@3@o~Sj(?e#hj=-(B9UqeCXDjmp=XI`Z^kP>U5U|ouYhR zRuAmF2EFxz%eSvyxb@?!`18ddB^Cg<$cABSW=>PJXI57@a`c#SbOWiy@?A?@Y=^a`AU+7r?=M&v}sF=<(|r&F)&h zVpjmUVvdr=;>%TVE{o)`$G~_u78+n(2#T}bFM~$M4N{IOfzF&MIjm0I8eO2UvG@vT zL3#L;@6u^6{8q@$p8fW>+qiW1EnM#>Wc?ud&z7-7JrC6bYNAMYr9?trwDMbGt^cDFCB(nNBG;!DlN6q zPE5HM5V9wB$nJ!2D?Pv_b#t9o=Yi3|F4wZ z!30koTg2p2vH`Gf3roR7h{eH-UM!UgMzN3|F^aiN_Hfa%2L~78ofcu`t-;BSb8$RZ zM(MX1XYa1~%tPEQa~GF1GQmE*Ti15Il;^900|g4RR>7Kl;9&8UIk;K z2f#uDn-l!1KvsRuA0qY833vm93`)ff;0#P4hHw@-S*?TMD0?ET*2wt8P@iQ)r{KZM zu4lxyuEB=SNKN+kHRXdT+)v1peUQ9l@ie zpj>w{q9F?j?$cSP@pO5G2~veCZ7-v>srb3kPCUulNIH|cx^9YAtH}Q|PR2!pNNB6w zU*y%J>|l_=IzCLBKG{F&QA9!>5>5aT^@Uq~Ah}LtGNX@xYf58XzO54Tu@$0if!CT^ zK3HcE?f6l#$cQ(nO^xENHsknct*dT#nDH%a@j}6_>6p{dW1u`^M~= zkikK{cg>u>fuPJ{1`OCP;HPyHa03rh2W<=<-><4w1{JkgJne@E*c}lnd0Gm$tdPCBSVTz4W;qh zh-vNPYNchnP`GB|I$=(;wskh(-GQeTP93WvSU4xZ33fOS;3u5ITyi*I&f=^8^LiZ{ z7(50?f-smxWI~U@?^zg5a>wjgT`dx^dI!2Tln|29ZvkIjXF+kBuMQL^tYLLhXZRO* z*3E%a}7ao3 z{NkxEe)_#He)c0`et!9J^2OUPe(~e~Ouob!8Eg$Q%#3E}1!XN$%*WXFqmk3BT-0uz?Z&eUg7cIIxUkLciQ+cI4slhX#|!4}}Mh%;9PffVMMv?$=X7XLQ z80$3^4vZlKTcx{cf_ffI`*kME*uQj5mN9mYODwmYpCJ>qrcX}OStjXzdiYn5JT%nl zU_BC+X|VXRtGZS&Hwfvk=yYmh-qq>Y^M@iH{OoqyMnI~iXTz}MnMHj zerGugd&%YDlU;Ion6tp8+mRYiyqx^$dyDVF%`ZODLnI1W*h}|&z8yfOvUdv}vFHn0qv2q&U!9nI$;lKchKzF~kW>yNe zO3)a$-ps_dY@>_nZzpgI?urk+QeRX79pVQvPhKJ5W7XkGiRI7($sln@k%>u5-M9<& zM0z2Ee@N4_7ApURj{VLGq0ZA=r9ee8Lb-#0dD00#Jzg9-I(2vfZV-DZD5%{QJGC!D zugrQ^4q1IxIdqKI$!X!tBz{9q`L%gkQ{N9`oj1hT_#Me=g)ZlNCT>;+!IBqWgZ&0p zmdL1IomT02tf<)OIA(HToW_QrLA0=g*)3#L*JSKL%>s^R2u}i0ssf|rzSshLaGJ0~jzm(KrFYsvf|7Y)Ao13`O zwc)?Awx6me%pTlUx70fAcWTOK4mNmfC+ti@O)X2y(n76Xnm*W=IB$Us2{{ePgq)KJ zB$)(qAP^wom{e`gR6QT%c|QQpd>dl}lkA$m@ZR^5Y_%oX>6VSL*klIiZnavy*1Zn* z@w)5`vzb$eIrQfy1{Vca+Ix%f<4T+*>^sb&N0e}^VI+?Lxv?QIU=rwEhTSAKx%{*y z*LC^L4^>WU2gPjXG_$k+IQaSi&0&zo#7$ThZmd&|t6tw^&{s7X^q7(eKr!4&O)w0s z8dplU!I#QV9I05;cd9f&0xQPZx)7m>43x-I?Gk^%AWsW3%R=+t4yighQu z+M{~1yS>v(s1dj#JAxp~>Z2OlMSTR<6149gm^uOiiNj{NCR!or-*hu_7j9fSh=*V@ z=2LTzAa+5b2E(I~5<9h0Vpm;Zm+Xf1y#S+deCC=7_pB$NzdT&s%bR5j{J50)oBB^ps8jU&h*sCIk*Aju0X;%__Q_~cSSF}8N>`!i5Scyj(c1VKU`XtF)idwLf zCW)GeN5D9NpV69OZdDo8jzh@~{5ZI}7YZTDTRa&Jh1F!ZJ*jsmJHp*v$(|myJJ}xX z_DCvl&M8O}v1tCIxf>$uIoxl54+xPe)sC&t$Kcl5z=ut_~?B*`lfuUEd%d$~ujK9U^Oz zZPdhqn&8+E9n^6qs?zyADay<8`Q!Sk13*`F|8by4n4K+a!t4y30U5!fPd)r72;N8# zd=Yme<&dFNRtV$%qUkguTo@mZG91<_+$V&Qq9&5AZZ`48l|;10K=ex%Pzp@&w6LKK zHfXl_5iaZ!0^v#l3Z$h|u);_SV}QbYS{tBPb~EO>D=S7Q!c7*Tm^oxIO7!4mjXS6% zmPMpu(aOFmkNt^W*OYiot-f!>*5zdF3zv@EIelgR$fX=$q9kfmPoteUz$QSrNJ2yebz^pz% zD9KthMa=B|VhUb2{LU)BHR6ZuBYh%IbBz!HNLUP>8`?SKu|qT> z_yo5y+?0J#mPsP2Xu7Vbx~fDpk8K|~zY`>ak7_s!__M4k6*cm=G#!mMN!8=3aGbIs z6RT7@Ct~~KK$23*Z8qadVXj7JQgGnynFh=9IU3ExNb!-{-qLAq6z;AN4qdRAXno$a}CJ`zM0p(P;_-L>U_b$ z5K<@XNU%|x^*q`w{WWV~KecU|%7R23$8PaNF+f{T)EF#<-YUnS@d_O&E7jV{y2hZ2 z3zjGdTl%=mV8s%lxQkgV2O_jA&K@p+V%aiHQ-fR4^=g2$gkDiDN+&*L7=e+nj%nz2 zQ|F;`GmQ{jgTismbnviI-BfgsMz8wU=p!K|T!VWTq{jQd-#GQpt8bGZ&tLfGrL&J} zynm*~J6h2Kqdv1Cbcsz^<6yR2|1v6`EPfCFJ2(46@nZ3M@lx?3!BhBa_9&d4D_(&s z|K3K57vS;_@aHPrRlal6Pp#sgpkaq*r;Fdh>sMa7EYW(OCX4UR9)wnYGkXj!e2=|x zp?J3V4K%y*{b2Hj9=-_AJP!}JOyD>1N$2q=ikD`e_bi{Dqx)^B6|)H!uV*7O%}7YrZWJ zCeRgFj$m5CY@L$R@-ms7f(vJh*Jlsmsx*6Wb_&{vtJzt+4!7Y1YiP1!hn>|luYDe_ zZ=i=)wb1pb5?vFMJ!q%0G$Df(=Q3Rl-T{;w*b<4k?6W2lXuk#K=^k$6nEe>(;wJhb z289UDz1PovO}0Ol$KD{ICm6;A!TNrWFVo9^;&1*%lOfERX&8aKiz4_CLF!zGnbb@% z1Pic{kF0xtftqPzm(uZ`eRh+3Q>v1hsGTKSpq)Hr2M2|1N8T6Kv3uCF;dwASeCni8 zj3C594SShakA5 zk5RK9b#C5pAZR#-BlZ#OQ^JG5-&EKTsjM@eZ?n~|yV;}|F%UKxR*uG>ltcr_vnC;Xt&fsEld4j<@BqwMUyeZ8m-c%I93}f-N>{f$| zBsh;Ur5n&=xK*l!Als-hG2&WIJx;El84Pt3rt6ox!u8b#17^(or{YE;?E+hl3Wp`h zM_-+P{p8*A&o8|BPE(R{-+TL;`RhmL&%QQ)`GQYU?n+lIRf%hg8i^_RPkb#&(jb%s z2&Kl$<#)|AC@EO*WQ--U5!iWQLr0FIJX{yn?}<%EV9tVN?M}djR6W@G&9$z!n{2;m z@nm_yX|fws5)-~)|szbJq=rT)v|$k7lC;ft9FHKTsc@j7JQG0{bYC?p4e<}( zQ^K)obu$=J^YMRP{xkXK`4c~W`eid{oz-!>H-7ry8rkyG;d4K|dp5Y|Y6nQco``9l zcxr37ekoRXl}fA_x)s%sZ?|I00JUshC{S~vTDcgIYNAC_OIalZfm(QqA3#J??iiBW zZukjeXT^9u-D8tvoUEE@2RgOE_9HKWCB`W11=!^Dgbm*>Wa~_(mR&UrE%15-#AmZ* z&^4fpgi1_}9t7dQM2)ff$;>?M*WeaTlJtXBz^oAp-yj8v=E>nvBtyYKAUUi9*(n_4 z2{)*jX}Z-9Uz+4Z1IkGI7*8!*`6~0xXMqNYC#J-r4SVsI7ie@^Oqylbw35;G7;I;@ zJ7POYcf7UqLMm)(|1bRa*v2?e)>gZ*q5`7E;tGOvYFZexa)O6suu7l9TckL`57^$s za@_-~qWetoilMJkukn{Wvd%YX8gFB!H9>Zo@BkzT>>rT%0t3QwY`%As?(A;gwflD+?Yp{^eTB@^En7kB zAz5mZ5oCTTzfCCuQ8qM6SdWWVg)CC*HVkf(4rtDSNzXA%&JkD*py^5Ya$&-fMDb%> zln@WM5}V;}oJ=Llk52aS6atf)naRC-HwrLjou>5e6r-j9QyJ8Z;Nzm)0bR(UoLf$g zu@VUJs^cT7#A@=ZeN(8Jpc<0G%*hNlDMoR(o1>zwlcx+xOEzgD&GoYXenob=mP0yS zC&vsJS!5DMd4^TpQ@)!2okL*`WJT3g1PVe8t$dXhpfA#*>IwMUzr(at=mt@_3Ysj( zmlwaoT}Dyv*VwsF1%=CdFYw^$6^1Y~o}^(l9Pwm*%M*X=3hM8%>f}@_fLS}t3$De% zRDy-m1>@&0)Boo^PT-aikc;U^vkG&)la@%rX5aP1uE`W^5GgQ_2z4Eir_Oq&e{V?D z^x$8n4~z)z8f4~-NmxOoC^2aPHa}raYe&7_Vefr+&z!n@<_g~O!ku>y-+AHa!uwx_V`@y*BZ*C~ z=&fq0d0H9VskQ98OYpt*17^`x;$^XYJW6b9LtQVYt)4nP{4X z8Y6Cgy@=CWSHM(WRVFbd6s}gfedmJYIdk`Kc4<_Xr&+?BInBtHtjOc5aMnPMVg}O# z1?dC@z+$jrcw841$A#ZTQBNpgt>MbFCD8h?4+@rO6sDktU@w{)nOuHaL%_29<|Y1> z$ITQx3xt(&_!!%uD!2PJ96H=|hB;ugNwO}hg!9g&Q5+6DEVhHvKz5tbr`}(3$1iP` zArHS<`OMGgh$ll4B@}HqIb8lCjEBxe{RLLw1c9c~5wLu+c$bas9cJ)oSW%eAErr-W zos-*y(}@aflL>af;%O-eBxD;x|C50k)gACx@I6pu;bl@egJR)}M+fqtoA`w`iKDwY zTvNp8TL%JrWd6wc+gJXuaO&jz2X7^x$Y38dgD-0JXmgKjiF#szgG?Hx5U=m^K zcZm`s_|~}xrsb$NcOUu7pbq;_d4{+&O+~)7$lP({Id8ot&HcWN!M4 zip0HFlfyJa^Sl){1(d0ZPSfDNde?C~J3V5pumOgBaskGZT1ZiAB5@mM+xI}Xrm4|n zW_|gsABaUCN3ia2O-8J5E(7!oZzZ_=z{F?h$`O5n78#%x$DORhwV^?^2J7wws^bIt*7X!oSb`{pDMS2ws5OF5mnNZaRGP^rZKWTQAnR12&N8FPdzE-8_G5 z3i{+ba`PNC=MXgJG`V%?){C-nH@~`h?$(iVJ8wM3G$EU{z$^Dc(h_PySb)y zDZ#vpj&Yl8$-~SxnH5~kL0+xjxO~3=b&3oK*jYWHjKJxYaRld%$2Sl^HjY$vF-6lO zikgT{EI!e2`U=4NT97ri7wvqMAu_Iq?WODJ)iiWyfj8 z3=>%Px|p3pQ?cX(+bUZ$w8$)jzNW!4J0zy+53&7$#kMm`_wjVr<8%$~`Ho}#K_Xh? zb$yf)bl*~f0%Nwt5j96KDVN+^hX8KjKnM{xL8=gN!iH)H!*ql>dPl_XU~9 zx0u}VE7I*ah?+5!5(Y&>xcrID58$VhM0(qn-v(DlxaDLqPXj-D=9vyWCGr$UKYV{u zO@!4+eNqigrbmA@!E7)|7r$08Kr-;Xr=A+N*ceX<)V#LKl#flHAzH&$wA3qs$4s}y zbEJpbPM&&l1hnXKv?JdRB3)IYYXTNnmJHft)A|W2p$drso!tNE!%UMm z$VxfP^NBU7EqX!;C8}lP+ldOlhB{awD;%+Yz(O0M%inqJPj_EDKmYQTM%4ImYUbFo z+_ZSSQWYYP{d9jK844?r8u*|fk)W<8x|*_Pc;V=4^9NtMbMe*r)9>EC@*KH+>CHQD zynN?{qk+YtK_A9c$uI52v8?z|e7aCT7*Qn_YFLG@)T63D2Xq1kH%pr`C$Q;@P#4>> zSXx`+V}*SqxW@Q1C8>!}CFMvS`;#h`goM{(FPAJAmL>zPjF@8vlCW|TWDPm1v!MDz zZ-5#}`gadC+0N8Yt$o7cp!{cqJ@Zv?sdeV9P<#zrP(p&y)D)er!WRHfr954VvU(Bl zq(xZ*WURj;wm)98A}&$bKcGdiPM+8RL+aAqj|~ejDr?2jGE{?`Epr=)w<$~YixEYQ zCg6{*s1dIr7ScSiai8@W2}Gj_*C=wfUq!BXqIkCW9U{V#;w6M)l%R~?AZY9+UjVBh zCXd2v|4_!0yiz<%idW%89$c9{C}AfNSMt*Ab9m`(i5mOeZ18mHcGnTh;6VcyHYO0Q zMvt$BNcT}a+!{v8b~q#3n3*nl7`6}{N5m(>&1=m*{8}T|D zqi@w8#>U+A@ww?gV+O(0wYjM?cszY=Zu*Pc-+z7Q=*781r{<>Tlx~wia$yo`x8|p ztZ323frn&5?K97Kw!Wb(wcy(J0m9Pmt$0pb8B6 zf(vrn^h;z_otFVk+38wo9#}57uUuJp>zKz>@C0pqPcU6mVp_FWM!8S1*qmDy12u=_ z@m`y`7CeMvmG0A(L{*8*EbzKbpKc+o!seSFjBVA-+kX7pocSa-F*Y%Q1ilRvApDl5 zvL1iE8uQc~#C&_g^q7)Zlh5Rl*|3(Hdpkuf(#KedH+LQEgZgSFTwiezXrW|CRbn+s zgh2*D4>v(Z7W8ns8MMqg2`4?@Fa?PaMZf#vd>5l=BtP)i=A5mWRU_0l)? z0malSVG6fW2m}kxZrh|Imiv3RRP#g4G$JR;1!?9qu1G^Eix(Wbtt#(*v#Tvt2@dWM zDYbOwS}NDNjbhSnD0q#@lhn+()Pw4*Rm72Z-&XhBp_k^Sj?EopEEW@3Q6*j#*8YtRo&sa1qq%IH;sRP7=oT|c8TORh%L5F>^L4#y8 zUD0ZY0ewpbbbiZ`P9 zaG6Ld%bicAMi2;63HpZ?t)tfUa*K{&(wi0vMVmpft0S8ysFCNE8i}-_)-xi4r=Tu9 z!6S}}2P5pygMs3RDWS&s1Yn2RJ(TQfM^mS*WFo;yd*|TpT~GEeyEfR}*FN0cC9heq zJB=~RpoNmb_0&^;eVI31efRF*&W#RhBE_<9Cd)i(T+om;2rZ&2@y78&X$odB*(bQk z(wn}|yDU2DR61RrCNYH2$t~D%$QHMNJYQgbk8+~{Ad^m7a4<~PZrjubGydNyf{7$5 zcB?%0DI7$JTO*i;%xV+xKA5&QwxrY51J*wsPi?{CTo$NGREZ8KwdZj`M z8>_DAiC>T{aBY0gM%0RU$R%j~iER zybYmI=@>UBW==F&Nu>AH0}F5+3Xf5EpjsxXz08lQTpJcCy#JT*?&ry^>08f1Lq3H= z+0Y9&&)u3T9exa*cK+sf!Tr+9OI)KBc}IQ9TSbSYTEkzxag(*rivQdiWNZ*#px5>$@yG%E#o7A3mFwH;D|H?jrvMHYAM zAu-A+NefmlAHC)ACtIrI)AG3GW?YIchzFk)U3&RWiV=}o(X{mMZdooGgnc= zc9x6_D{T|zq?#?*+DBo~A(blGGHx;3F&ath>L1wA+O^Z;TMl_@o;6q6e}6CzqdA*M(QzTt<*Xo!Fo-c|D482Q&9kup`LREj!rgsKq9p zZUJq0oEb)QDYj2g6AXu5Y)fzqotmtOpe2(q1~jZGsa{-Hj1++)XBZRXLZp|it)AHBLBPi?15?;rW@51p$-F&5yv`v}aCZo_iV#GI<_0dib=?JZ++Cq9i;H5pcwXxz84+A_#fS%nyd9?*V$epJDL9SpmN}=SJQEsI6(RpvJ*5EyH4?9%kEv(GTLoA)Z(69F9Gri z!>$)c|FahsF5+dziumNlLPsLt=|A*z243EFI)$jFufg@EBKWr&Zdd`4}NgFBq%?jPYqgFC)Y^U1ZG21?!$ zd|-^49UlL4K%O^{iXH*|yWu!A?&0aIYw{e%DsyUU+H#r8nlUUb*wdOSPXP>%j3-3rD}bcjD08!Kr^;JNnEs9m09` z*?(Poq4@&f%bFHER#)}yCBxWb7yA}>WG$(TJ&1QlG8WC}skKS0q2N1KXVwDafSZgA z(}_}MM=*iJbg<(=tB!yM=H0wn_4;a@zPdkYjhmS+DT3g(gVFQB1Dc1AaYaIaSC>_t!;dKTp2BYqQK+Y z3fvr_hC!_qO>cg1?piptfztbv_QeJS1jvYnw4|nM$*xdmvLn$GN_KU1_9Q!1tt;6P z*P}_TJKVJNq8b*`{{&Emkdf1hiPkum~qcxs}2s36i1~>|bqZ zkqSQjCiaheTo#p-NY!UXBnVtn^u|5GpowmS-H z;$Z=a?RG6;GaB5^SXW$F6tu2HYse&4L@{jXM>nYS^)2TKXU5D|$eX>A$G87KYjm7` zwx6fOrciEnP?I^NKQ+wiC0VvS`FO=wRet#=UD#J&~rTcP29se z823p!2yzLBU;;KwbOs{1ed(+DSN?$YcxPXmKXPvVk1x-^cYWdLt6(|Izxs!!tE9hk z{jIyNe7tb-^}sV`{*%w=zj_t^zWu`o^KX0#|M#qo-pmodsUunmOnEJ6t(_L!HNqv@ z47Y7p7P`sOdK(mfxJ0D(IYCBHJ57^)F6iE%wn&hgO|OI0fu$MBiUJ><2rN8U=0|&} z3|1`BeRzl=bl)a>M1z^x@P@WaRJuP&Yy_>ESaM2u=uEI8;i;$kT%O@1!=Vj@0tSp= z^8%Y;gxap|SWnx0#Zr!b>SVc{6CyZ&L?5*>0<@n4o*gV4(sfT(y6$^QMxshI z)UXOU$WU8>i6*lOFu+X9?x&_pC%_OMjKT~PhRDE>HOMHVj*H;JFg_#UFzL2LaJ{2z zSL=wNR@zoQTU)52?8{?+qSA3|tM+oP2M*MK5cvCh`0!fs;_MWB`aE8R+e(-4I~%j< ztrt|(NoW_g1lx3W_8=*Kq>$nVaOo&C?!CY=tbv4T4cFHg!>h|rS3))T7u;_xX@_u< zWC*EG@|Z@38zE91le8!lIBa`QJ+*y+ris>M8FKj2!Bdgj$9QUawY64XBHAhv38Ozc zq*qHM7%Ogj_t4}&o_lrg-g_r6-uvLA`R9(zpSgVZqc0ZT`(fdgk8WSO@~>;}K1S~s zob-K-crLtl`tHl$EWGqW+fakmI!jMjgQ*opH*%sT_PpNn1~d`gl#_?_%Qs(-DO8^q zLtB{_1BH`N5;bKXz6<2b4v(4h7N;v%z#W4%7I23}4{+l?Sir$UpB=*l-p1y^wK(WW zC5%9kiXQXsEV24KOKdeEsDY?TTRM=Db5vMq(jzQ4U+t9WLbTPD91^`9D+OI-nuEx1 z3dcmhFw$ZSfrVUdbF&_xXV>n*YUe~3(nvZL*F-$3s7UY>iz%`2nwBs%rl^S;Eqg_8 zQM*C+A^$_QWV#KG78BA5t&n6^J6SpfhGzyrqumse=@j`9q-WStumr~TXbFqmU}GSt zGD}hAmFr~+)&Q}Btvkle6qk?kbe(6SsfoCvVbzi5@nJ+lo{Ti#qC{7fULC`JRQ}R9 z$=k=!f^&;JZik?IzJ<0u7uj=q6SuPm+_2p#mc9Y6InIEKyA8@S+EXI5Xa&OmsKM0 z`S&(vS!ymOS+}>-0cvGVlO}9^M>l^b@dv3ZjKHxu9}u#Bl5FppjJ<;?69$ z`0rir@$@G`il!zMRShXIk2|D+{S*|ZXpO?-TkAAIH6VqVlNs1clcYpR)Yi#U2Bs>I zNmJw*S@z$rNI4IR^s#YnFOn85zomRN|2qd6kFuibs=+eMOw+}yw7?9W7FAEcpZ*=Z zXaPDwRQ+_9BrJi21K;8PB?;aqhN9&ixFuEY?cgOly}}U7I6x3-&a;vGb$x~rIyMoh zwkr#Mb2(c#{VH8$s?1zVWf}D9K9h7xl)wYfFTt~!cYw9Sav?pkrU@L;lz6CGi+beC ztmDgk1KM?cc3Q#{uhng-*UuPRRZzMT@lO*`M3k+bV6hm=!7srC&2!> z27k+TOZ?syum%sn0-WOyoRl=M&!tc9(x73K7zE*>DcQaHnL`cI%cw<$1njLuOTqzs zCPqmgA7PdgJdCQvLy=@8tR@q&ke-Zp#gckL?MiArU7=)myj@GiL$V%cQKUiXFj1gL zhq;!FAym+iOp@f90Fa4sYCKeAzQ)^P3F*IfNuzNRA^m#sk79C8vDn= z*CP$kz^#^36qE@|izGvuqSnA_ZyeIXpg`8xUBVit6HVqsSbBArWrCLltp_ejMDm!` z6aYUek!!R>PW8Z0seWOoBAEQF)x;wn@&xw{a_rP*xJQF~>6T}>heL7IleQA~ndS## zjw<2CbKKEW2Szq?BqHG&iWk9@^jHnMcaZ$cyQjWdc=LMm31a?}&+Z)mX5q@`ciy>n@4c(@S5Msg^vipH zdS&7Chj%U>4lJ8)i>KmxNcBR;1h%syn2#QAJbiBw@ZS^;OA<*a0B4{?r4aDObheTa zgPJ+i4<+t$C;Uc=5eM|5L0SO){Nc*k>rTN+Z%=@pF(y*9Zdmc>HO3t@l~X;%_%YoB z2&DVOMTlUiA|9=FpgeM31$SNr=hlYa4{M56Q|7U8n!2tdbxz9SB;90+y64RwkeVLW zm1wwHbU^?`=8sIzO<#rOe*U{Z-~IHZJ3st!?$F`p+pp%PKSfTxsY`+7@ULrM{Oj7u ze_gxsuWJ`Qx`L-zG2*vaF@nG%ks5SC-y2k?u&feG@Gv|d(^>L#gLs|SZD|t|?g28~ z-`=W5`-A&$`YY@$g14&0EvfQY16P(CV~BQvy-- zSR!BI9SAlNA~8kRK>$%j^~RPH9;QN{ySt8Q!O_NX3rf(KON2iYNC=xC3{*$SV~%&( zm7zMJ z50EqsTPxiVJQTb`WZ{idIRn%rP^CP3QAMSW_|&L%^u9)GK&dU^o1XG7({6@3lA*G+ z4<#+6W8>!kd5=w24ka^Xv$4`t3AzMnL1F!d-B%Q5&NyN$u=Ivz&g9{>xM%YUSL7D< zFSnQy3RSO{K|BugmoDA?^w{0+zP|I_yUnM^ExdX5&Kob!zjAQ?$Y=A%UcCMNbMxOH zynXqDd*8n@|J{Z8kH082Z2st5cTb)QE~H%vRgIBYf>gbr-{W%VH*;Sha%Cas>SQ#rMkG%*lh_gl3`PTABI1Z%g(AHQ9Q>L9rVNi!}B6enx0z$joO zFVa|~3L91)rNx`dpzcIK-}Z4DJhr{mQJ$Ht)z1*oR!I@j(G?YntR?WZR?G`$To9Xq znqkSwtdEqhth#P5Ly?_aehPE8 zl{}5(rUX>GSaOW1)FvB;kswozpKjUY^LSNPb0-+t+;oH28xL`Fxsyzbdmzo@zO|}w zq7Ffp?>wVX#vQR(YlmA<2`axpEiAa(#S9lzzx!t3+@}=Q(ra^BFcs8Gld+g-djg*g z;OXg@i>JjKUTEcVNze*qTTAu&O{U&-z;rk5RCb*4Ol!)3jU3hzYm7NgaCbqUfv|!( z=Hb9pqZNP*dF)dt@j5_2zI7ML*KFiVwE`=YKyI8&0_Wm&2-3wY)&Q=kFl>=d8=ff< z+*hVc#(TAz5iaM~W{1xI>4Ud_I`Ibi?|0t+@z^^nILvm|7S#>1f;(cst|~^!PGPal z;XC{B>>K~QboQrj-wv+lBR?`2(oqDN^prZbMnd&T%;EZ~azQ5hhQX@uJ9GV3vgD*| zA5QL`6jlLVv!rI?mB;%gT{F!rTqRM(gjF+RKV)?GU0~_hq}P%@%D@Ar4&QyEzU`1$ z#de4$Lu>L0v|FhxA493AB+cyo<-!nX+=@^o>){hEHyMPM`&!-*|`II1cB>WnQ24Dv7W9}NV**xb zu+!8;Ovf;vX|M^>=@tk)Jzpp%sW+KOYuX}7=!2VR$R~O)@waIQ{<}1)aQ$l_LgQhcw%287_5s$2( z1`TxAs6nvGyAMbUYB$+_;Rz7oX|l2Q1|A6##{*+QA;ib>c#Av$a+*&@ttcV^glho1 z?Yc}$1XkCC{=Jo!t%>+CEHEO>AYsdPR-*_ z@NJk%wxYEpW!=8=$-RRsq+_*g+gY;~Z-Ze)b_&U5^5D(V`HQd4pF2mk-23#Mg-c%t z&urAdcvKUrqG4*b2eU4Cj3k=W!(h`kto4@p)Vg3_anR`DW*T(yNl@s{7JmTk?iwnH zaC{y9y+}|g{!T)%pbmUg;;megYB6Yk-=mtm>FZUBZ^JuZk*e=mgpi&+=+TdBK1&vl zV;6wBe-IjanH1lJW_~OCy4=qnjgD!#jI4iD`=WfGfiw!$cj8zR)iWOZl3fpK?V^|H}k9J^KI;;R-Ok3pd4nYtXZOj zD7f&OwnufX=&inG8&zV_YS@A`udv_Dz9fB8xSC#-D=93l*KxxsEjE{0(f0z|4Sw_; zY&y8YezUj~KXmKtr^Ppl9~3_-o+$pQ_#WKw(Nv(Rb+yTM=4~#+Mu5fvYGeapLWj$KTXHCQ+(~{XC<1?n*HUv^iFh_z#DG+@ z_#hq+1i&{D0Q7d`C!^?RRw zx$yq2dJ~`E_F65Hy_|B`(&42f21e^IPI8cikk;EsejvP&%z6_x1l{vq>3Cpte4^;YWF17bVZ;io%wy?IuQjfk;m`w{shQqAx`*2( z5~`rZuvK4Q%Kq~tHH3WTG{ms}A*MGh536k4c`!t}@Z)k)vd-#yCx5 zdix%p%4KQrb+oRlq`pQ7*H;?~ND@?2SLu5wEEbv55#c(dU4ry)?%C*{Q;Vka+_ba2 z(AHiWzrlSbx%9Z#V-kycd?qoUZq2x?KBysaX)twbX|IFUcJegr3^H4-12%{QZ1Y!& zFTH$4dX@GJz$;SHE37mnTS_fj2i~Kw2RYkk018BwP{L~iOM2zt{D&VdynAZ?46oyPoQvZeSo=HJgw&rZQ-R~kv2!T0h^t5C(KB#-?~EG?!aYG^@y zuZ1Oz-(9u}4$hoUQ&a5S`ya64b41Y)b#EYBK)0cmLuFR!FQi67&9p$J1%G_zH0~fn zDT@~zyRBp@tiQ!@pS)PBuB*Ll5jO#Lioxv6g$5A0pQL8SrQWE1a8Thm3=}1>Akp1c9%tU%kQow8xj5;>kfuv#ybA8{X62CYu!8oGU%SBM(oIT^{?C zNEj^eh9%O2QJJO?z(zYn0tZ>*Sm3b+f4i3$r_tEjHZjh|pkrigt$eceVF{~j6Bsz! zMy)~v#dWI&)B-0s#nF==id6%$JTl-n(17o!R*DRAHaZHO|DZA*;s`39sEMrlR$vj} zt%F)4)S|!JCAg#PD`cK-QAqI3twT-36d4|XU5}46u#u4UZy^F|RA`_30MO0?(qMe#dB|CTbC;KKlMPb6i zNH!Vc!jMp#xY!s%z`!+lj*;Of+QxXA31X)##=vG54%w2Y6C+v`QLWDIC|BufT? z9#C|#3p8b?A~JcJ9HPzyB^?&c=IdnMwCrrJ4rA@42r5f+oRot_{t(lNNDtU^fe4-~%gkS| zrvvFe>0PV3Xxm*o=bEFzF_!zS4%afh7O1Bn(q}njBB^SM9$yPjb0341kYtyjc*#%! z`3uto9q0(d!HDb&LNRBFO%T}4Tq8|J*jD5RgO!mxoT6_tt&r_D;SOiGaQxS5o8A2(}=%mb7LKGbvI@S4U;WHFj7asERba^ z!Ciy{1~r*8hf zZTlOkv_k{2Hc)zXOBW7ewm;morFasK&%yWLTfwE-?mA+58HZZGj6+?ej00pDS3))E z0*@@+4ID@9X->9V)C_KP)H8GS70g_H6*D)1Y;uWM!=qq13PGpEqztinB?&+F%rj`Q z%I1?aYV4CGb@o^7rF`N&`~d8)lQ1gSubz2k$Q`krWas09gWbD^`*-ge-iD*&*=I@B zM^ARQ4?g>>=lU+!nEZc!^{cdyJ9$Y{hJAsf8EeA+4tx1Hhwa;Dj;s*G-viCf5qU*X z9($0nS)Y`iRc`98@qMvaINr8HfH4EDC4FcI1`i#T9^=^*zz*`TNydEUy(E$m6_Wus z0Mw`xzRJw77JnE}jg}g-1Z$+-2VA3=C(vFx@hQ`=l1S9#G6ldj>U3$^T7kuFb17VP z(TIkHWio-Gx!<@mr$<%`}YIw2PY)gC@iK1>ed0X(|%+@FS~{xF?Ur z|Ky7)vBoFgAO;g~*~XZiy&9{98b{0|RY5unH9oogw3?5o>gF;bQJFVvTkqtyUU?c; zVC9Hz(7wH18hQ8VC}|)d+=&=#f)(AWb_Bi=660&DI*XqZqv{eU!t|?mM zF~dF>2nMXccNtdwFT;9VNz@Q!=W@O^ zwEA-yLF@DoNGgOr&&lPd)pNP(=GiHf^#6DIZyy&g6fc%a(j^MYn`qmCWmK;7$D9{g z)qgn=PsAsepTci|W%T{uKK|S5voDe2^}n5)J@mKtWa#51S^JMC#mlnx`(W|H>`TqH zL=@lu+t)H{DIoCV2ha}(p*y94c|xLDz63q|t;`zwzW9yT6Iy?X2(LVv!XXLHQA2^` z+Y<_-B9-hNoa`OM5r8KcN$PS721k2ua_I=;NKZZ0!wr^lVic5nI_%&0=Fb9GS(+@Q|j*JYMPy}M5RT3D^U~S zL~BGR+q6(ZD?hM%KTSDASj0|I5MXOsjGH9A_;N}4=E;D+_m6|0HrEh@hm+qjYRH!D zN8!p+IcuWSz`$d(UWCz;c2@C0g#G!Ga5P3ib7>eVW;EYqRN1}4u^K86XBMpBc8Wv0 zx$U+3>pOl{_WMAetMzDN{j*pWr0u*|fjJ^XjyPo(4=Ev0VtGGw3T_^Pj9@A7D9kkM z6?Dl~?oNx8i%*q-=M6|^*`S)B2~vhGBJB;(Ko7T@P8OQrUFYkUAB0z~K0qlal#tr6 zlmi&#JE&z%sG1&n7!2~I##A^-f@^<2Ymlp+63Nnuzts}K6xUSj-)_PTHA+Po9s?E` zMQw+gDM>-t$IV=m&AB0Fjd6H;D%%%)fd$PNL)6}q5>G*izj6|kPh1HnYIZZf4wC_4 z!Z6HJR6H<(MKNQ#US3B3f7aw>^t1guB_2>>M%5nYGRu+)&F!NZ5Qy!}oHWT6H4%%q zs)>Xa+)JXyv3NVCM5D_KjXh;qK4~FAMq#X0uAW1_)7&xY@p~6;v~ibnc#0RO>97aR z1rr!$x0B`z44ubo;H3VHE>`_lbKxMh}z1D?6 z*ohiI8^3dDkx|)psl|ndQn}kSpgZgmk@9{nOSBMaoRzL`dMLT&QrvDMP<%VA7LIGu zG!5>1s>7iiG-(!u1%{j+Yj|S{K!Uc1Wyh)EJPfKvxheNo(eOlbRKL+2lxJAcYfOay zd13z75EmJde2I+UU%(=+Ycgt&=zM3_9=LC0s)1f#gEZrNs&#>_1@kNq?R3B*LyZra zA(42XZ}8b?n@T)`Fl#KE!vel#gqrCF@+dG1mUZkN_QXE}vl@d)bX^InYe}T&N6L>9 zvGIdeBpRwh)yW=V8ELWyHWSCd0oJ)ll9%O$;G5F!JhKdXfKxBKZqT|N6ih;iMb;vi zVJQ+ZDnnynGZidRV4(J`r0}(L_tvQ9-I7R^Vcj1v&e;;}|fZ(RDt=Ob+9J8_#6PP%0}#a3j0YGhWwL z;ACzbF>ytUHe90h$btFu4$L8zvY119sg-68Adc6#Fe~eV?`1&^E790m08MMM6Bi*X z4@hyff!>orfjuNel?GgqWR)RkigxOV1TCy91)1Uo35uWNXOyr3_@oFJOtBEZ)3`YAX zn>g4C(1o5z(m^egIl+r~>K4wfkX9aFCB6uw04o@)x99b69{fki4F66NlupT-)+_S1 z(k1*(Qwgk&f8jG|#y8O7>#}`sH-PwGTO1G^OAv(^RU$Q{hB@I=h(ApqB|m<7>8Hc* zqi5l#Z{PUm#V^TEhfn-;^lkFfi8p>ce}PQ?^udK6KRrx-MSgtc?2j)U{prO|$d505 z^5f;hKOQ^$-^UNb4O3tK^cmi;c=X?gzWnEPk#LTh&(!R_@{3!kfnz`^Ne9o zGc#$qlf~mDPNvyIUhtiZ*uyw>h^*rHX7;7x1P=7>EgF=?Ocfi^p)Jr3tOX-m%0xzGa=hSE?+NGxZX0yW^#ssb!}UcT zUR~mGC0bLh7X%RY?x*MHkGwGd{@Es|ZC);sJ0G6E_vXR*mxE_u&K>$-Zt8`(LqEX( zJpu@vXr*FJOspo<9IVt;6r@X-xYB7M>yRxrYJjqeunvo)48-A18L7N^=1dnI<7up0 z3Z5f1C{nqJ#U5>|QVcrAy5 z9$LnQ0mBtk%ip^UQw$6`^BE;HKjK03Ni8Cz4`EQQb>;CrqfVy}j z*=-cEZJ;F{hAS6QTm1XWw+<1!0JmL6%@IC2J?XvU){AxSAX~KPFPco_z#h4ESZc(# zUId-^Gt|vdZw@Tw-{zLBe9?+=?>Q?7r;-lS@Z?_kvov z7>QQORk*`@CWrT|r~BkS+3KqXx)6phq#*b}aJ>VRElxdPzaDp`UuUI;0gObXT1yO) zBHm8gZCj)`X!|oC3$opmk!Vckrv^!bRxxbR5o%;<8g&c8h#^ro(DX?S9nqS$79!(i zA(O~(4nydg3v}N5)#$aJ3Wv>%LMFZvMn#2i;H=H7{1XWUs z>i70cW`szSQ3|#$#~P=BBUR{o(rj zizn)tEIS5jOqLzAut{u|`Lo9sPQE`kb!u+<@Z8k-z{1eL=u;!o?p9RQW1z$r7W*~W3R*ZS1q((i&?@^y#YYhRzV>Qh01;cMa zh>#3Cxw0bjmv_k1ExXePKwCY)Oip%NY0!4^2!H(7YZHW%e$G+{=+iCa03`*H7pcD- zDJ{mksCD2k@1zfKdfym&HpuaraY1eU$|U{n%sFv@^n&i0X2JK&9W>{Txa7{+{;K)U~JWAu%6V-;N0&>y7cK3IHAgy;+` zrtO60$!=3t=k!x=vbh@f#)jg4qeEfznMUF@wqXFg!)-ICEdhq%1hK`;xw3U%-*-CD zU89H$Fw1>FdPrUP!)^l+(#IF0Beu(6J+za=re?;H(Y!ogJlO&-KhpqrotgkG1JT7I zVMX)ckT#O<5=Lxsy~aEYq_P~p@y8plgLd*c9)1qn`x{bG`Mjjez}An~Ps8@m zPTzPRuY4r$d>s!@!AGazqknHB@PI!`jpz7gxBI1v_8I*71X}-&eBVWdr|Od=wDe8c zC&3k;8*jl^PeHG|QxUYLdtu0Z;zos$T{5IL9LRlC0`xN*D3$=|GVied_ zhfFGpT9ZfaTgE~Z?5(8yOOnVT@20X%CWlU86@-Q3tRTxBvO=VTn}yMknv6zb$%qOE zk*@Y=G8~GTQePRqYU9cgU z5q#6F12Gn<@3o1nD)C52iAL^6p+{4N9*cjHL=R_?%VG9iY|Xp!1Uq*b_;=1MmZ%}IX>&U$UZ+~bjDV{7L)ix##6Mlr+ z*__ZqArEGE@a&8*0(NWRYW2nN8)5t<$q8Yl*RMonL!wDLUF)wW|MHgk*Yl*0y4Hj# z$o4UA+Bq&rhUZOsfbR>Qtn>tK0XAReWog7z$IL*z+6BFFxu3&qAG5egn=I{O@wEes zZ9URCBgs&x9)rzF2761d3*DWk29-GeE!FvE1#W|~D^{}DdxvW*_N51ICbNBrWn42& zr9K3@V_HJ-^;&ccxeI!zVbpPDw)8&UvRqm%T^>qy`2GvYSHCcO6x5)@l9U(Zf*R40**D=7i%O?vr*X6emeCM3Cun$0 zBGNP^qBV@4{mAI@VRSicY+X>C6Lrq(pG_iiXbUTB2=xi`-UiZ>ZzZ;J^kNrAKkmRj8yRB$@j^Rhu{9^H>ZC3_}V`& zo!#tnXSrzLg@1hU&p#bLM@r2KER-j6*>T~GcxsghjOH^LA(D(BZBBIv&36Q*W78w_ z1PPR!xbe9RAH8$q19Ic!jkn={@5q1O!Aos7{&eHC3^ARSr;(7_s!P8@z-*F!Xwj1% z>$kGH`307bVCAFMIH>4&7M<;+6>)V)p4$a1qe_!;VWpdFzwe%q@np;o9p;$^^5ME% zCzS=$KF1~!%e+K^v*IF1<`q+`qnZEn_MDI4wXY%C#09zGTtujxb=Y#$`VvuB7YTxr zS6f7nmNy^ETNIYJ*g7n4G0(OX^S>`45^QuG*ifng%E2h|CJaC6ns(OZ6>PSLf~BI!I`zDWl+_XSPc=B@0{=Id~Mj$diZ{3O%_DKqT^H|=FXfS$FjYG zb%5oRy?e_T*F!t;Fh?C_tMmpsIo@Pq>vMUAlYW{pKtjPCdv&t98!(i0Okj(v4LfcJ z1wqD;qAkH-9zih z9I-K@pwxi_!Lf6@(`m}n!n*+1ckF7bcy@J!*Gx1{m?QNVMxfOzkgKw}3nN4i^~$yz@fS>Q4+!TU9DCC04UAu2>7$GAt%& zp0?UqULDWBedVRwSHGP*bmq=GSMD7DaD|DosS5(zcXmYL3&&0_yztWPYwylaeK&vR ze5qM=LilR5I)q>98&A9*tS1d)N*aQgcz8R4gB$7++%R6P{quBy*#o-~^n|$*xJ*}# zV78lS*s^VYKNhjTm4NMThBMMbP0)kg2Eb%M0oOo{i$)YJ;)wx!imJoD)xk(aNrcvd zwKQOfw40Kl+h*KPcG7ef?29hpAXGtMSwwNkfmsn&)QD$RXdVzj%`b`|2qUiOHCeGi zgyDz+R*#^DVyq_>>&B4A2%G$tISnuo4)Ki1(gzNBK;v4Y^=+f*sshkKwcZdi!d=u( zQ40aG9tJO8r8P_LOMTs4erniL*i}05H>>oRqSb6S!Eqyg9w8!z<Qhgx~kf#pEK=XSr?hbR+f+B3IT z@LR3swu&MmWF%az9R|XYfB((9Cr>q>!hY|Ct9RZ#Jpby;x39l)=fa8L0WeLECgL%@ zB7ln7>a!FbdOr%|y*fp+0Tlo9&)+@uV)Myh{@W{ekDOm86o1uY9?4^WB2dFhDB7^! zc1gQvuBB&vm5TawD66ocK(B*N7pvCkM*W83do83UoH=DN2{`4-odL_IsU#6s2$;}d zL)(kjJ6tPA9zSrvK!jn;uT0IH#ybh=qn0&+pIc&l{V-`aX7&tZg;NT+aalZ`L7w;q z9I%Xs-EN@sHq#C9;n9Lb&j7av7Nsb}Ps$!C$YY3a0dMqNjwpxbEyj1YN6l4Hnkj~X#nqN5J1tRp4nL`0{jVD>Hk_GAZD6y>q|G?~|n3#tqg>zU1^3(1c9tqN|{!R0+! z?y#LE&tyGWRPk8F^Dd8lj;OE_3HkQ%R*$ILmk!>WK6U$tGYemR++2Fhy*Iv_zwn2- z>38R*&(BSrfxj2dURZeR*u77J^IqI}z-4J?7E!UF3FWk)z#`e1b<^CUU?n3SC6#DC zi}VMTd7}HCCVEIo)OZqon_Kq|zI*4xKmY66yZ26By!XLJPdzpN-G%wH*T|MTFTM4P zy?f_>_~PEd59g1bxcAN{3&&oafAy6Ixyr5Cd!>a!YO*^TPlACONya+Vcv4d%v1CV2 zmzL}ahdYz;?ohaaa6kNhA{GjH!HLvJqEfF=I`Jt`tXODRpfJJiy?gG?XNTs`yfA;H z34Iv7LSziK@YbIf-hBS<_pi?%`SZg0Hv)@9@y^@a>anOf^O=CR-$}E<>AiO%DwtIM!u)&~U<`UGC4AYbW7)qZ0MYIKph`o)a$9V?ri#^oN!t))l z2kKqiyBD6DqR=PEB9u%eJ9qad`zAX@VZuT!n2d2@$a2@cdy|AMvdx!(p|=tkp9W7M zV!AvoFe}fchv&Xg%igB~4Cr(+EPFlhKq^b2i(wqooATHitDiI#TP0~KhO`=y8mGVm z)lWADI3+tF%#1 zX08~}23m^6Aq~f>kK9Rfjtd(iPfu13G1nEowb%!t$Z!aWt&*xvwmd%E*<=B{0qPn8 zkw8-1Oa~uGPf|1EQjfPaaNcq(8Agky;TeoPPO#m>mXJc1tLYHiEugyyR{fEQ^2%#( z1&J537L^bLqQJ^HzRjd4579igNH2r4;Le)C_E0?JwN^r5Z+fI(8XORou7n~Dqw>RY zyKv#^{7Y}#zV_O}S063BGc|YU%-r-zbbU;}IyZf->DJ-7sW0ZH4XEzEQPz~XL9>HjUw8q6FOMuw66%(tnmgV4;WEJ_iZQ+!*(l276t1~fNl?q0~wQ6EOH2f1+ZvFwol6JPHEz? z((PbPZhCOk$8C4!q{YP+`U_c$4to={z^lE~`#ibK*j>8~7u6~*0D0_FwqYe+GvPK4 zEwz_fd1_7&Q@Y<^%F2$nNLHkBY=St9=9>&iRl*Xa_{@2dek?1X4|sXV7qHISCV2`Y zz(6px^keM?!`80{=&A>()3@wZRSDHF`1#gehEmy?Z_Uw}Pb^rSpp|Z(3{%IZpp=?~ zS>ud5=))d4Fivv9vUr9w%#hn|W|&R$VDUp2BitZpK0|mf0WEgsTO0HS0vdXnnOi}L zB|K}9c6ZdVxZ_9->FJgMVQ`1%P#1i^rj( zB#$E7IM_C3jv0Z6#Sn+tWx&ed$xaQ&Dr>$x_Bl2{vuzw^E68(Gl0OVGC_8=_ji+mD7c3F!v8K)x9;g^ z>+PBBEhB@B^8L)qK36kQ$re?O==uhE+}CMS_oONhz&4hqJoo_UppG-KY}1Aby8`1t zVinE-kUdP<8dh`PaB_&|{GLRkmOK!Pn8(d$-xfcVE%=@kKboB`UN4@9zrPDE7LkKV zzSAR>`g~=p(3Qn8u~w@=acRJn8?NulT_O!kcBi?68ztIrf9mypX4br4Ex?CBMBvuSc`x5 zkyrk~dgU9kYjq@RZPd_Aykiv%zcb)EJV=$B+*mYuPIj@uYFZd~SOn z5nYL{Ng{)6lvxN=hBA3%lmr_Nd$?h6nkNHUUMSdPOE2P#z^~(?)FJXCbRsyRU_B!x zUOy=tuCF>a;%gy0UxImXL_ZnDx46Sn*~z7&YQTreEoHQaBpDnV zQ#OW>`@SU((;OomhC4dCm28KBVG16++OW4Yq`XnQ&it?5GHrT4${*((cg9lL$$>g~%f1Qy2KAFkj1;qt=!pD%py zXOBkYF;P^%7D@uF8L&{RE5d@f`35X%BH^0h(g%)`7r0oWTt4}@ET8PP=z-v-<4T2- z9?(e5zcivsti~>Bz+aFugbj=2e~xz9wY~y8D(`fpYq6f)#^eSw)7B~S1=o>)M~jPN z5~P&ptYi0Zo1F!1cXYIE@oPh+Z*?NvFS90@WeLmv^~Nod_s7fNuY=lX$>9)q$SifI z*PQXNXNP^J86~iWUJI`UEBy)1#+xr)z8|APL#QBf4F3e*&UV<2JK~9u1ZLe$VCrn7 zhPbN%3^*_rImUZZ7L$sqEwGlPGs{h8Gh#B9X3a*i@;HbETGE4&AF@YSIZ|Splx+`Q zU}Ers@KAd-KqH!vJl1QH){rT(Fl+5B>BK&tAe|CP*#te43|o{NC`-^tp;y*38U4Ox zrqr15L-1y4W_4I@inR^Fwm2g8w{=dWj8by~#7-!HCV(b_bKK4W}j`<$$KsF=ZF=@Zfk zsx}NSW*f3F(qc&c1tVrF$%+CaPL`ts+sZW?WK|!xrLz_kSXi)ZJlq9}UKubuuza>r zGlTlDYmIppzC=Y%3IB3J!nSHT1u@KETbf`wGUD1KCmh-auLl1TFm*T?##&WOtLuPI z2ZV+3YlDu{bkkXI-k0A>I#E|hi|kf1jM+|Lng(Cf+D#bP9CUn(N4=caArGu+$e$P^ zA>_OcH4fbD+iY9(>?|53lXY#uwb^vZWmaT+NpZC$-NYg+THAKeir}JSt#5+nrn3SY z_N3J@{#qmT_sq!3n(?4ou5> z1yIA0cn3@;gSxhBQCZVVwhVxRmvI|$d2gc!=*&5)dho&{m4euazlvBCR8)zD*JOJ< zA_A$b!j(1HZ~sdI_38?|AS9dL7ta;HpPfn)d33dS3Hd3D-xSZzzFWLVwhwjeYAvoG zm+*!#ZoUQ0_g-KJh)5Y^}esJ~>@`w`r^j!6|Z{W7K%O|tb zq<9j(c&>P9_HDR(i<(d)|2TLY{)q>lb2TKMK`Xsb-ntUk8s}Yl6l@G;LShQPO@=3o z21=3YMip0JZH=!g^suHxwY5Z?F+WG-*}pZzZn!#3S~R3WZ^e_LWP3+PcNs zbSGnaXC&FFMbyA@+QqoB%j3ui$2>V#{t#bLMsXz^{r}l})8;m=Gk^4}oN%hHl_goWY|HV!#Y-$(jwHvjwRn*wFIB_; z+?ThgSE7SMhk-2F7On0o+=Ff9WNuY}x!g0!#-lo^_?`Fgy>q;FqLhTa(!SxAZ-;V~Jo zz-#VwC{jvb?wx|!-+uD(k0N>mjAk`Z=)ThZs9{bNM6yA%T@+h>fX!dTZgEqR+hP8j z`du)i`us%>lTHoS$Z1|bGBzq_wXCk04(!VoobB?OFDiw>qWR z<(Onl%b8ol_BZt@k~3|`EroIb{1Z_TNFpXEsvyM#Kbe`2S0N;=0?H&V!~%_Lk(e!B zI=*uH4b1Z_J$d}rnu)4fy5n0Pc41&)s|m+b$l1kpIkoj;ty9| zSvmF6%F`E?j=jEeB05m<(&Ha3oqKWV)1y8+PHr62kekLd)RdqEf@Sv9dh4c#*}MX? zMoc?(7j~j)VzYywML4vV(j}_vInTchI?o4Lu1LmM*#%{j+ae3ea$EyhdkYO>(j7a7 zJL(q_Cw&CloK9;s%PA>{F-efntyX+ho-MQpDk7G71`$@Zu>JpF_hUryA%yv2>3)mZ zITOxx!EWkM+txtoYn-QeE^wyJTEF2>QE-to3mwL70~n#UqMLqeGgK8@RD3p+SyG6r z_xg8l*Gk2@{^400hOaT6O&Zl4$IiM|p|}~zRdF5XxT(lRS%@nw`^J;n0c=kxwwWMy zTRPVEkxYqN^Jn!IBK18b%MY^WvpK@9HRKdh1Jw%=%=(OjX$J&DX#=BvD}f9}?;`8lS%sSslkpT`wgaxGW*6})&QCMS z48xw)r>UV)U*r}E9(y()NBqt#%yCB`qwobPp`~Jxn_U-Gb6j|a_c;Wl|9(&B6 z=N+InOk~;2+rR%kseNmd>Xey=W9nvM_V>TPtBgTmq`vxW*9D4ePNxT9G_a}}DXa}V zvbaZKqjmBY^AiR{=0ZV1r-W317CyZCtuO@~jMP}hY19dQ)D5PXjvKeYBneet>EI) zM31m+nFnS;-G2Y%ORGoV&0EvVYCC1y4NAkvF|rNRE0)K>!x&wk_L@e{@;6^aZ+=E0 zC52e3<+{IL`tov+Ko0C22mH4|(=xd4Hb_g+t((A-QmUSV654f8Vb=@r2;Gxb1zaEt zo&!5jq#PBcRW7g1(0vlpRf|FMB!Z9kFw=E&1`O;hc&dmh(FU&GL$-?<7=QYipGXgs z8@u}f8pO_F-c|(xQ zyqJn2_6Em!AV|E#5aDTC38(!2e?@X^UX2A(=`>7q&)W{GSqD!TEQy(^?Jz#wv}g;& zEW@;5n7T&e311ep(_i&bTPn3aN*u^dk~dg})1$lDDui~M$)nG9SQddB;r{b1y0`m~ zZcf%D&Q!Ir5LC6P5Mc2H2mC8}s9J<%%-E4j~x6Ru}l^(Wm4jz`2pvW6x z?Y#%5vk{C>6BPKcYZU0T79EbaVTtQHDkIrcWyn~q*Rliuvw=|fA3DBSP1ngV%THht zgs$iC4X^uP$}ndP1UU0uiEnB)0ezY&F)UUxNXgY5t%}=Tf;-E>Z-mt`cSr_Zqca5a3YIs_ z%rvJ(^A~)^)R?auO%0j;hleO3NU4@LfBx9u+FtGV^1+gN&k{C+J@QC5!(PgMTGI5{ z%_M5LJz{IRM>KA8^A9kxT>B27_2DqTI7MFpFFyz;P9-KJ0_;-Z!$j?7Oph|d0(%FP zic>W6ktco-6i(JGWqrChnP{krlZmFYj*F<}MDasLUd#Ky%aXBpEEboWf|tdSPbvvN zsvyb9x!S{e7_+*ID_Ca4+xLlUcr1!}u3b#0$gCO7#`9nx%z{0MBgi4ek+ylA^>^m+ z7{(YBUt6<8j78t7xvTcDY5OaHJJrUp&+7WP&+vpGeM_L{FG|nZxrHwmzTy2>R}U?G zhdA5|7qH)|tt_X7mlm$TjeoXqY2j-a$m0Y~V5`|t>^fVx9NnU>?+)u+#{q6C8l)I* zc?Y})0vn)up9kn8=&JXFKWeq1|f?1XZ z7Ae|3s{pb|Nf5ETIIaq+RAbusEt{|+6v42O>uVN^VF9|Yy?E`-Yj0e8_1fFy+Dq3? zUpsN_6(S~-crqzTeDST2 z;Vl`U_hTQT!f!*t{&dFi4++yLB<3@EYLt7iM&>y|2kK6 z5IPRi*<@HNSV&lDkm>m^^eJX>7FZpb5~diYyaz0|TxMh%+#&yG4bI;N&nLVE{x&5B ze8k!EypQ$GAAIo9qXfEU(=0f;YRX?rlJFIigkE71tgt{K`v1Na_SV%S;EY^A--9fi zKo|=|_JC98?#eooWkHJHlgdd{btsdohe)pwz0e@&6*dm9HwmsI#jubJSQXJx>A}3j z1h(BQmQ#alGgos|Z*bOLx&}&*ej^Xb+l;|ajz?lOn;0fEMFZ@6l~K2;w<~DAX_fzV9Xt$c#dpwIqlIdR9n{v? zR7~|%JczO?`I6Xb=%|b;3MqLbXd~lD2ApZXk`oR@*FUtUo9w~wn9Wk26g2L$5gl94 zlODGmSu!K0fs>;-uq%{(24YmaEE$0jx9gPCITU|AsEW+ZHSje&fF%X|GZ4b7^q$xph+gtc>xa@Ii}r$uCd>B zGzgq!M(P7+pslAh155yt*GLBB#a--*0F#oSCIk_-c3;J7nWGV-!gun<{P>m!(x5y)7(IoJ&^Zr8XQoij~+%d5VJ z9E7FAZMp%g?451&Yi^>3WiTf>&j%U`7?$2GbHbl9MyBK zj`5$@n@Ilgd z0CWsue9WITreL&i>aoXizVKvwnR*ZF)0Fg0Wpx@oPv37*V@hNFK1(lpYleaa%1W$0 zjujf71mjqm&eFImX*8o_g`C-|W!!>m!=A8>9VnZmQ&I`9$)diig+~QayXW%9E0zrdnmPZl2Ti!zsS#M1+4ViOYI_fG>)IxL-953 zD`C6#6@+(H)r~aZ^k4}ZB?YR}G)cwoTQA9%2?B0qtO9(EX51~P-G;EpeLBI2Be9F- z4B3YBb$=S3#Da@(7zn?DHR#b}{bCB4!VrrKO%`8FQWdLma*rH>1z@*%5IF-(;RbJ#6@*;z+p~!+L`ta&I zrFFWenzT#MIon(Cej0N^tZ{!6vZe}g`TI!W5Risu8?}rTTD_-bwBlzM- zWn}TJey&*4Sw4zYE}di#0_lMLR=V3_+<0M1CYI!wJ`w9uRzJ47Q>j&Rguf0aC?zJO zS})`GV$2u@ESs<&XLUxV*96lj+-!FxR3Rndf1n`d{0IB?VJsMPYG_Bpht-&%f*whx z1WE8}mU_DTN9V>EtYL@r6hV%KwsATp zV6)F@V9=BU8|D~iVguc5VPK+6^>sGQ!I8kGIcOSraJ}p;v2B8pW3&15ye6B&OXV~M zzZ&eJ{Q+=9kByFcu^rUuU@hgak(>i29`xfSipe;v2%dmR)pUJDtvdu6ri6Hat8vN< zucm)Idg-Sp-Xs4w_x4YJe5D;!ajnFDI`+no&p+{xFOL7?;(I?m`P@$z-h;!)tLmqB zpZMwBkN@%IdmG5nC}~WpC0obBz%DrAmU$H&5=RMMHg3So7hyXuQk!535W8in{WWl{ z$uR-fs;h!u)}Mh50Kpu?^V}HmXSwYEjRTm zJ~}c(F?DDXkgPrXt2TS=q*zQM1MYrIJQ1**o#;uB@q7tCU1S@3<`93S<@86j<5TqY z+-$S&U+wx_ZQ%0^os8=g;c;_jz37CgV*|i8FQmDZgSs`Ns0#6C23J7BXo4oIzanw#$pbl&baIRlRJE~x=I>eC|;P4F9^DO5p`WJoa z<&ar0kW>ns;Eg1gGvL*0JgorFlwoEy9Tg00tzt?{wV$<>aczfJDX$JCvS5A-mat^! zeJiNCfWk^el)-P=$olD~xv7F>Vi{<=g^(WV?;-u1EOi9`O_MP*H&rzCJejry(yiI$ zR_I1{u*u18%QegmAo~0Yya!ua~M$N98K}AoZBvRPD0;Y*9h$-2Z;PkNug|;%Kgb;5%@;k;J zb(zg0iguYXa3t5d`n@X9p7Gv)$t33B7s({RfoSvUP^_BWqZzYcN5UMqMk||QPmx-> z#<=G{sug_8%_qPbz7x5b0A*1~i2(tI&0Y2~yU00Q5oA5DCsK4qq!v5e!$V1g^D{O>#(|Ge>wKyfsqFVITlZpwT0KvZ!{8U8AYK9~k5<6VQ z1W7Vk?`TA7S=00q*4j;arFIX9TYdr6*^@eU$?bBnz@~GI+AY8h?J(_G$zSO0$D|1h zNuwfiF>F6ABMenz-#kW*G4d@hil#Z|!E%}ErO47)`&g?HXBCkyz!bP^ozmk-8L~Gs zKH6r@(QEe|S!Tg@=;)QC@pMcQ)F2t?J+!|&BYke+!_2|s&a6q(ieA0i5b3szp&`SFIY~0MZmo^13e+K5)vbjM=8o||f{I%GLz97~neMCg|Nj&axt zY1y<`O^!VTKbY-iZLn(kdpJjj#&^-KJh+OE!De8+{y)F!?Cc~%m?(9q;cUmWD0bp| zH8@zY^qK=8mn1L5Nbn}xF4i!)Yzg7{(DKeDxPMSh+nFquU) zE+`4XPuQ986uNmY@ASRmFOh5vo| z?3YW=U%1qoFg~xP~3hH+>RPLR_tn3*I@4IRh$o5-qI~Ef}0sxy{Vv-~gicBy0v= zU)+oE21*3B5XIJZ&O7f+Z65d29Wko=N=Q_0k^yRAKQTS%lHvI?nAnXT%MX%o#j7Ex zcu26EIEIj=V1hbIZ&?v6UoZr6It9adQmgQnBAP+FfzJ=a-hfONpoHLYz#-O+Dtd%z z(smG6*yjgzDvi4~l_p)AiF8bcUA7);qIF6qGBH~=+pJO*3*-3yPj(>?O=MXNKRjMb zt`#+TpWuie;T_TjQW)0?27j6=0UgcEuS*DdCjoR?skNE4Of7osJdqL*6;VNqUtf#4 zBvnmM{yL2IN^ndLspVAU&$e7t1J1*iK*k-5=Cx^XVqE)|5JayZ(^@25k~ck+t|-2S zS|wxzD1kv(N!|#P-8iGDF+|eY4sIRFrdCqvo$z?(=~N3V&Hu^ORVMmYnHYMNNiiYW zInQ1tjWa{_7zpG?XVZ_-}aZT!0V9n|DZe1&wIdD=* zVQu8m&3+6p{{~ zo&(q0p$9a3wWXJN=fU@$1>2;Q)!-a_m+1wLx3Xfb=c_hY2~i@zbA8CrcqSlE^CaUs z^A`>9=WxKIF3alC&6c%U^qY{4Ey$*lO$lo2+;Vx%akhSyPOuu7`KJn{&Z(9iXl2zT#Mi_UTd_o&NB zS9fX50B6!c{;esE%%91Dh9LZVh-(lX-)6kehP9a#$2l4wQBbepC&;f|*nr&VMQ~%R z=-7UqylTOP7_!_Vkyb<@83fPRNV5}pK;*A)7qMZR_{e^zEVQ+Vu3aFVelK{q32h99 z%XHFbJG@=Dhh{BiMHg|Sz+p8;91j^IPNY?E3KAPZ<;&P$_1a+Z%_5{&<#ii4X?0eH z3EG=CM`!XZ=Tfgnhi8GejaMM~dNpK%#PfT=P&{bD#z&)DBfDI!gxuUU5O9%+5E2ND z(x5dog5dVdpQR`@B>pwIEe~(IhL)=gzU^fA5cW|QXbaV5m~J^}(UJzgN#@hm<<7EYBKs4|gA?7>J$(Zp8iSG@%(hwd`1=Uc^bwzR z;ja%1TOgK1tPeR5e2)S%o6#Lh1C4E2u<_U$4+F#IhM?i2G>;lCaI>08 z`sXh@h-8VWpXipLW`Id=G9%|fviOX`s);UzG|@p;Ng*lTNo2$k{0JKM7M=6G4&Wk+ zaY;zY;6H3GE~3lpSeS-`Q>z*%uu=vU1 zmy4$sPk@xJA6q<~zW&tWCokW5^V)Qyx2bZdh@U;VhXv}@BOTJOL>WLfOh@@&#P;8Ur-j%6uq$8r+0 zWAPv`dh0CpYD!8ZlFgXS5PFR5GN;+7YfR169Mv!0dnD@()8@;2+Ed8^!~lj9z)Sm+D}5kb(IxEM0RB`pnd*OoK^G+=zkNyG;*h z4mqsEb2nfgF#YpLsa?-mS1mHy?1KgZC8&yy@lfBwvdgPxt@I`x)K z$}N)!uP;lXSAWMY#If*{t=|N;s9|$cCuhQUaG|dU+cVG|s7HCo&@h?HQ_W^+OBAN` z>8xBt{=@$ZDXSbX6&(b%vz>J+xH|eqsv5j9tUjYqNlvU%IVC+%9|_A={2lbs`;C%_ z2_pq~5u((oFP(ht){)bAfJaY)Ubb&Zt2lrGV@ zZy?ch87Q#n#9DRE>oR=`l zZK|yA3HW-`Oiz&qJNlAfHu8jjaDzTpdbmY`MFwvA-|)7L5S#SLCSfubvvgaFQ)CQN=_-|ki zQlLp8)_M>dJKHRf8G0>aEXT@~=EFwutI@$g{A#qQ=~}sr9gJHdcr{pQR^R0sE%0}W zDNc%!5d82*Nnda=8Mb6Om6k*yrM6s(;w5ZhuNEu4_yhfO1=GxvNvb(T*sE}%HA^&z z@b|&{lSD;G#^!2{YF+X>u`q9Eg?gq$58`0P-~z%q9gvKeh?}`GJLSf{x7@_}h9>}= z@7P8t1n%`-`FpQArN-%2sY$*y#W{2X?b)Hh`VA~CD$k!NXa-GJgI*d<_h>mx;!vBI z`uy2aS<9I=r*kdS`Lp(1H`BF)nDX&|28>tC6a+p7W^2RitAAtdiaD}d*SwEy{cUt@ zB!LjxdK0)d&d3kUfT^2C!G|s05{PvwEvtgsdZ!5j4|4XnN@_k$`tt6hE^Uj&yROBP zabdG5xY1F}cCliK8Zke6Z%T>#jGWNBshkqzV6pVZ`;;BCrb{`KEo%E2bI3S_jkast z&EjN8zl3Jx+Mz`k<D#P(#T}p(o89tNwCzs z(sw-zYJyqt6WwY}rUX&MK))dRy~F6$Nhw4iiYXiMC5Aa}V+jHLzzT<7HnCXFwC*}N zRdnMLi3FxJ1{%CTt?1U|aAv@r^527JmcSvFJAEC{czd0SlxVT@gunIuV)rNnyWe5%s7ABrc%{v^g|7o$wPU@!I9b+ z@{GC|>AMJj9X(c3z~Xf1ZOTgqlU zXCe6U04qgL7xb`MGdIP2$vU~Z?a86HJxK!F^6qVW{n%E7^kGPF1=ggAFiT{X^b~1s z{$jiBf@`Cx4y7`XyR;ul2|-ae!lM%C`)@C0*;D-QE1wc${$T-KPTe^4 z(aMR_H=qBuk#(~(=vt4lvWsNih|IUswAkIk+OCyy!wHJ+W=o7CEW|GfGJ{Q956AO0$`%;MnuaVTCvR(+m$)jUmv zHc#cGpe9;Q=j~-W95A#`)`}#5b!Ym@H;bI0X>d`B$Ha6`cUpr=~Ob& zlUBRqsdQhTn2Ibt&g7^Wu8-nPl9ImWS$+^se2O9@e_HwE^vWlf zmyTY%_4?U=e*H&(z3Z*_4&8d^eZ1_-`zxpau=1DJx>ioVaqIXKU3e#5u~b(~ZM*F1 z(sQpaoqK-iwdbR6j=Ib8EH~v3MmE-xN^_|tg+%Mv$|I(;20WZ54`aLBBae8b8eo7? z(law_P-`yZk&W2AoQ0z+^1iJ#A`@^F*LX-^P{>n~uPoh6f6w3?{sY&5SJ zSF^n~v51gGW$>oF=Cw6r)mJ-~2ve^lHu6?s|3=WttL_ ziF(W(%MFb=yztV(pVITM&nn*Rw7v9BNzItTgi-phlq+MG0y!yN}dN;lBCz!g&Vd^d|dPSPU?bh*joL_|F||Pc!@NLqTV}N%bka1PEYUm{KlV_ zjy}D7`tZ`*FL^^+I`;a?iPO>Je_;D!?*s+`t3pz$MIB1GnhumuQc#jDdpEt@2gS8i z^-;}hfr*6!^d={Xp+1Ms~xE~OIQj9FKAZN;ZKo*tHI4<9e=y} zwhh@cd+-auaUX91`!MLw`Z&syq4hLYlm?9JD52i}dG*bIeE!0{m~OL}ZgjE+kc<)& zl5%LWhK^w*D9G2+n2vh2;hdm0PZ0x#{O4aDd#qo>9>qNNl>IcNT? z-w!{c6a3$UWTyt_CT2|1ZP977dRfi?9)zE{`Ljk|%dx;abG)#6kr(3VQx$^0!H~;v=($}xveC0H}xOL+5TkpI-7v`cZA1A35@2#g^Tz>QN(noJa z_oERs1E#YxTVqK@Mb1E3ri4TQT;-nD-dwF66{ij4qN)u#iSI7l&>e9&Hxq40T!-*u zeXwAh_>>R}gnmQElVIW(-dOnN>fwdYuRabxhv4hombJ9zox z)x(h`Ii4$8+SCRb@l^?FFx3DbT-2!G_fNjyagmYAiG@FTV#V>U1qiQr0j{gh&oz5{ z;d2Zs!W&>W!MB}(4^(1^1f~fd0l55Im@>Hh>Qiv~t9|<#+ADV~Ro=q$;2#tW0_Wlz z7~C;9@f}|WUqyZvGCDgzjl5+V+LY-70#~XYiW1U8kq}#2l0tK%cJvgr94(kNc5p9a z2Z`Z2jcCU@9vTnqb`{*iaR+RjJJUHHk4eo-!T6v^H_Wn4xZ7A@WYl5}ilRG)cGX*# zLpu1*re;^I%XnN!CWHi-n5vK0poK`E1d3Tmr8ZKg*KfL(Q>;W{y^2^dl}vp9_{(A{ zrM4kL`!3CP&`>I3QO&fLXC^7Y#pr+8rp}_5%l4W%$Fv5gc?)yvu^kD^o|B1|LHQ8r z#L|qMRtEpDr${ZDbC|^|pELP9dC15!YsNJ5Hjk{2Yb7KqGN&1`0JfP3z21En<`e}v z4z8*-)e4{4c$Vsu@O)S0=Ho(0gtRLqgliC$`i;>Ji~kEd+_$Sc+toMu&h|NhP44rB zFBd+?#_@%B;Z2_MAq(f>*A=+_BH=c{6`oi7{>hi{vOjVo0l(n_rhyhNdDm1v`C@}l zwwc%$-iJB*3^v3!V9Q@P0fYGlM+c+%8V3FbzKR|@H>W^(@7$o4mEuAy)-rXcFQ1~( zHl$(dhau0BKA5T!jUFl;V|heJCZM%)F7`&{eVzE#&|JvKoP>OjL1!bh7WJ^uiBqJT zS5k0iq%h1hyGRC1JrBQI*u;#SMSbOxosD6!*GI+1sZYSvZ)#7#z0_i~hhWsyD}YR8 z&YOaO;;==IQN3IwBQ(#Vr<@yP7p2%ba4NC&JH+AfP!u5*+X#O}wKA{mbYvAK)3lrx zTC(6VXM1$&!?bC^gU4vKTA$txPwjI0`mx22kk;ks;>E>}7C%}1^!m|sA3RxI$c_SM zK8N2={_XhnLmpkq(Z#P8Pr*e`ES{P3Uvd4J23L@6QvCil3pOu)03$dL!#M&!$F4s~ zU|zn0N&3_~j4b8fPSvb4xoV2Yp-mB_hD>e@If}tvY@6-3#^BELWE*UM$Yx$}xwjg< znxe;?oMdb->?XtDRxrmGA&Cymi~>mraW!;=q>(@zq}#Ic8pkTbQuNv(f~wcG+$^s_ z_uTY$5Zzh>L|s{A?fEnwR<>r+3wLBw*NUz_`(0yd*ff0gZECEFKg(Z-5vGg?v_XE| z(1t#&?lf5gicPPv%Xjq|=!M&JV2dE*3kS2(O&g1v@qC-9MX&h=hK9o!$|6%C*+#8{ zW>$mkFZzmHo2LzAf8?|n6Xf{4nzy_3OrS0$gH+dVTg6+VF&!_QZ~9ckg?Q_bwrfzE z2&rkW!-gov;zA-Wz>Q${=UnY!J#$yx#jDg^tKbAa^28C?4|FnK+i+qYhO$jGd#CgSQ`<@dgQ|NQ!xU(~Dbd(F7`_R}~2_{7bp zk1fCbv78VSVp2|RdAm6fR){Zs_1O=X&aOQF>GHW3m)?H%=3ky&zWD0WsaKX>K5^^N zrIlk(M}7t>6SXm+BUx$aWLC#GELB=IE!MHU-W}z2^0r$lVb?_CnyB%CK$h75g?GOe z3L$1~vJK&gVE1BnbdyH}OGQ)!MG+)FQez7)sq7)5ZX`xPClQp9%rJO|rxSJX=>*@7T!`L)$lqgIJyBl6|d6V56=FlzofoX`|eh)eZ3rYKc~ zKfV3lkDp%p@rjeBRYa^@UBV!mBQ(8rsKHR6!0vvxEi-{Qk)ohFK}=!ip(U zotD5w%7V7AHFDo*rYF-+u<4m%9v>G~k6C79LMt#!*k3LXa^HRTjTA^vkufsNi~<}I znuo6`5>FLkuP}Vtfp6W<@JkS6)hJ9+Jmco6MFlbhmoQ^L&BFKFEK3Vus3*xzR)+5( znbnI&u?HK*2QRm+efRxj4wm?Rw`q;vo?3AKd_{h29@42rEUl>dqDl$HS48C= z?n+_EH-^y|LGppo$K|T5`RgzXT|pF#RKP6O!UC7C29O;%WWM+@j~j!B3u4oF?tCtA zrp;o=J!Xz}_LzEfCY}b-;!d%#10yyU6?ib>MyeU^1&lXV<7ei<)&%Qild2ZP-}4QeCFm0N0we*hofL?g&FFg^1l4+)|Edjz5T)R>mRN>esTG&uUEdf z2!r|IyYoMM^}?+q(cuP0S>7_iQ7hojhC)RkoRpA?ht7|2OYmXSnyQcDb<^Aw5}SjG ziDqVwT5P>4uC3AXp~l3f7%o_*XywUXfsDAeouzIAWVMcwhwCj&-~6n?;3q4lB{)!V zKLtxdQ+SciD%WJMSG0YiQU}dba zM5g`N+IKM9W@LmrnvrGIt4pz%FU-{`B^o1%!TDw~QW5cLBenkbf+ri{%*xMb26*XC zE_x@>W}(R`$Um1%1}dwsL$fvha<%X= zvTSy-et2$kd%z_0?}&{z#Pvm^reiT75p2bY(6Bga!8co!6fiT`*pJHu6UiA15q31= zZY7X220LI@rNa-?eL4ZR$KXU!WE)P^{b|^@3*f9$;((Gd!5D3U5k3_YBnh^vSX>ZQ z9}=ACD^ujJ!|Zx0hNCxDv@u}j4{G2tkZqG#pbfSQ!}fjBTLGpQ$hK82$QihY;bN^e6V+{N$+wp92e3z6h(^<3JA{I|EO&_UB zTN10)9{hDUX^WyD1;AE9>&~2Mm*&qn8VqYYKHQ}e)hu|4pVl>IG=kc=V;+8Vu3Oz9 zIk+-<-^VVqGzHJE#T#N*jn25w=!}OnIzb@d4F-+Q|1B-v6FeDXlAY9W%;?#e&F%44 z5pf}=Y(y6#g2cF1Fv_Owq)99lBz+)=%1E@W-v!>Bm%{{;!*&`W(fpjqe4lJJVFQ9v z1u@lf5%)tzUdw|ZrZwL55KqR`cuG{nhK&}L%9bR?QxyTh$i?_v%~5@2OYNnVR*Nk! z@45;WEt^FGnr#ziiHFvAFw2^4p~p%?@P_=vo_=ch0WFnko17A|OHNJ8qL7T$Z(i&2 z(sfLjQ?s>@P-y#Wc2551*T;H#ZydSO_nV!%T5`pKh9 zpM16Q-8W0myw}JE-7y-lL3hyd7QvTBCA4Q~d}#MbX4gZ%X^=k<<@)@=#D6_EHTz%B zb@X6ofC~ea9q+$uUs00F{5zLvsoN1V) z`Lk`&Y+MJDVVTniltn#iSi_;D$8rz0utnx6Jo0{9Kk2jeL#s=y>E96DakqwvyK)iR zBUbq{+Ds+`U~G+38Xd!H#|+bKTlA^ER;s;J1nrf?`YMEcgwhh+LB0GbBik~rJwuB+ z_pcui+f5eCDq?cG7yU+~g|b-Fo|P2evhb~n(2GKq8&rjaUKEB2X5(a=&HE7Sbs9Cu zxQV+a!W52zewmFt|G1Iv#Ch-)6ErS;Q=3%2#W6@IhGzrm=z%|pA$73aV7JdX^%k91 zptJPCb4#y%0*2kqS1#QA{0YyDYh=0&?hKf2gV;~B)8s`hci((ldFDL+$BT8A-#!vq zOt3=;k{Ux+VA)^z;g1;AZ0KSWbS03MA|%CnpEx`U;d=|`7QXbb2QMLG)*4mU=WUhw zSa=x`q7exC;wmyAM-t|cfyWl!j-I@99kUh@K5}7B90}FrAZ$p49X_+4I@&bSMEsKS z%9$+7F|eUnD_cy_rDTU`P^VavQjV*>=w8TLk9b-H7b;mVpgUKpGtNqy>1erjLwdpc zp1**AbCD&u88tc}H|$GlnKlC!*7Dgem!7}Sem=~wX;Up1dDfR-JhO8A!t%3EZy>4{ zN18A>cJ~U~#x-Vn#HRx;wm7=wtYJG88yhFXnW0_XyMOabfZ1p;tW_Wp+v9KF;Tq9}`*R>1DIa=SDlB?g9HDX5 zc)f`@twjOin3&k*@i+XR7ykRos~Rc(zvuq@%3JUR{GS*9`|@`N$b6@!>*T(^63OWL zZG)J;ZQM{g@$NkOdW0Fx7)8+!&fshG58LMtLP~;i-AJM_V-;LHYDWY|SIcRj7I%V9 ziJt9!7&ZqjQNvc#I1toq`W0D-wGP6+zDBHr9hxYzDR|nLRrZQ1*WicbUxHe!AKj)a zz!r=aB_${UR<0&})|!#Gd8)fUEk*Ow7&9)gb@KQfJ()n{qK6t@4bj~XTa3E!3DUQp zHzksZqUG8S$+!isG8&}zJ9Y>9q<0$#@?uuAP?9332|*GSK@@$-!%n4fYC>s}nuw=W zNr(lk%Sg4_m?zr8y+uD82fPoqp%KidkzFRp>S5E3yns?jOo)gkAQitvq;;x!n^X(N zv5->GghWk7Txc&BvRD?E`J?H=OqxXRtYfP-8n~X&@{`;+K!Y+SVL*H znV06>KH0W<`_YrN0Vl0W3lx$SD1osXvPa%?99%(Mzx^oUfi=@yDgG9hhDT_!M?D9iQ0 z5m8c!1Iz|vFjsWi4xLEk>kV3#HL(99$UTUGKlP0_|i|W`6nL``TaJG}IPK_yeSE#hnP3)bU*xL@3 zl}%k*DIOEkJ>A%+D5q0OES*r3nRF_d=t-;H@l?96PfSIY@wFz*=XUeAfbxJ}+H-6b!oX^%%HUjp;o1}hy` z;C;lvd>hM})XH}a85ngk8u9XUjP4A$zleWWhom6cqD08&8(2 z2Bvoqa(N~M*apI4y~48uH&m~3chLL{a{?H^+gq9k`PY_O1uVmu~SJlu8)HA z&E%y@LL!j-Du)HfF~H0r<@pl@&7f(rhvuk_R7573KSNAXoIjhjv^*!gaZ8j;!}|nZ z5=XPBVOFlS*nvjU?G84|c3bqIrbiO(?MaDMM;clGiHVdT;)bZee(1AtR3GzF=+&ks zgjk@uEh2!2j}cija%K_R@wR?{SAc80%uSD8sz=W&4{Lhf3jhsjxt#0k?dn_y=AjwN zLlWbQ$;3ub9x{1XcZh|&+v|0aKNMIT85hks{kEvSRwQD9SqW-^jKQ304fNf3re5ET zn=Y7F{&B3X8Q7d;?932Ee@VOOYr_)5_F*NlWn7i&GbvFZ>iwf_S#4vBNb)xA9h(aof z5{=paO8)b_A1Tbk2-~;s`zK%WofcD0u`>sscdc2{4c8#!^UUhC>tKkr*#_5F0{h8B z?IYXl)lP^O`s7TQ4DN#YgzRgB;IBgpIltP_v$KxHW*&W%zwYASow040p*i2{efu^KD(;SC4SJB?6eAUT zv>a`L6Hd!h+izlauHzvzld=T*K(4nijj)^L1JrdY)gtY))&}N3w3)%O+2S;fUhy^p z4(pUJQ*^DK4a>KpgrW5YZh0BJvrPp>0YzCL{u3kBuNYq|}(2Xol#3Apm5< zL>>ltuI8w|alH1@)z)#$7CrLF@b2++PiAzy?RLbBO@_!WI!k(IaW~kO1@|xdez=$F zXiW{we_`pGbI=!?u7;*VKLfI$NE>Nq>c!D(4jBW}b1&9?3pjgL?g2b74jO zrpZHaZ#f3e7y)yQpC#}YS@gK*x|%q8xq1?#+DWzEYQ1w6elTX( zUBU9oA)T{9EseHWkza!mM3!i<7t0q;2pvyQlY*><&hj4! z{daD@^~uWne_1(l;)lzJZ=Slc^z5t4Z+*7(!gK9VHPbYe|HI{%mYzAY^vV0n?;Yz} zK7SYvB8%+io8K&5I&$lSHC&fw)yQr1BGNY~G|BpM2 z>0(OpwPJ~JwOV`Y9fXrC26_XPgrOxnV`5;Vz<1a!zv$|Ksx-aG^b*;vgRzIL2+Yk! z&p3@~laqe<{%ZY0$a=vPhTqfxscLw*cMK&`omkW0W1uH!o352(w`<|tE@|x1bVGA( z@R$m%fYJ`K{IrH-vi@omDV}It-5gIelZT`TN+6LQ#rqqz5Pho*La9y4X7YKEPKkNW z9-oVU1rK8yI|6xqx@UgC)!RW47BsL_4R~pnH5gaB2^7KqG9?$L998)^Em3RA#|gzgh$$^>SW&@CWqd_WKoDE1LQ$rxr-5nOg9T8zlL8#hPi&f%f?2CJql?p3a;TZ zHCE50`>C;CBcKUkSkWum>p$IkqdrI{(V%Ghb~Nh$mgc4q^(&VAC3x%A(X2%wq)m6- zrC6*gQtu#~P{CCe14Y<((0Q8%Yiq=uX1Z>Y%nY^i?S?-yYO#_={NvaJ->sG9sU!te z4bDhqOe)9}tMyD>LAf5ssF%rfRr7YAF34a<`(&%ijH<#qG6sbk^1m&n(6O+m0ow=a-p9J1J!Aw^#uTlD-PRw*Hn zJGPYjU_E^>t74d#wUUm-(lJ$#Td$=65j_mHu8*FBSgWCfUlK50LKc(+Oq3wXV)VRx zEdhzFRNVpoI-H=y80=^*N3;Xlq8I0pEAiyuo3n|d- zeKy(8@O^MZL1qPl7r}ah?QFz_Z?Oc@XW7&_2+lU#vqcJ)Yvk?ulg>e1%N2um+H({V(Z7Rt}?&lF(sHoS^#%TnqR zHg(F#BrR$1{8}wA69r1vqNa?zM?)48ftXDs8`nxx{jvLQaQ3T z29YTddm^xJnPhU<)CsB(qmxw$rK%E2g_lrLNXA+=NrUB5q8f{dP0OYFuu_`o8m!(D zFQa7#=Ng==Pu(~A*lGb4#wvS;GGxZI^gO{TY?3w2DJ)cT;RJ6J%`#8`Xd>H$dQ2U# zEKIB2vL0hF1^AF4iq9HoS40pW%hy*E6(NB?G_oqh)QuR)HRP#VUIO?K)~@GtH=Q++ zEZCZ%urj-BFu}6sfv#Qf+teMB0m}jDN3 z8={E2M?kzG3K7#PFtS%5yRFH|7M5j$*7?R17I41lV-Oco$@<42G>_)AD9CZriDF6j zjL-F~bypF4)^O+nB-lg=?a~|umC#F;I7M;g0BQr@iHau}wd#K`dcnLIcJ(Tf9E(Xo9?Jz5PUm#->r%~) zQmb4kn(;05>&F&9zWx+h{Alsw^`n0~4lmbf!w)tR3|Y5@OfGVt_Fq4}{^a7P*Pr-X z17N2PF;?%n?ku_o8SHyT)n`p9f|!)RVO0dh=a&Am;sU6wR=YZSnO$V? zK`6_;--Z=cx zjU!**IP&_9BNxfG8%LhJapc(>hreo}fgBMrloV`Sg6u;pkJf_hi6$Y4YRkomeBpqV zWzWGs`}U>D2*NVurA3hldQ!k21oAD`no;UAVUSy%CoFR3(>v$Zj0uo zm{XgiRQ%XMN+|3gE;mF7O|bH5L4!2LM-w7-{033sK(_5M5_h zzMv{pRrMh%dIuqe8W-ZNr#7)7ng{3G%>%IV9%xgvdH? vd(vA>@@?Q`_23Q!8I z0i_sNs~ab4Go5AGirO-^IV6HKQx9h znmbi?vsr6yfEgwkFM*nmERNsOa+&TPrjF*%N=70y^aK(Y5@O5CZc;lyij3y5?i)+; zhj))}CvXcSYwEmRqGEJ_9|M~hhQlx%uQN^$ytVhJdh|4X{*0xu&J4^E8Gtp1J-qq* z`oQ(}jH!rK1tlxyV{>Ss0tZBw5)$#2w<#V7MG^h3WB5PD|082i!zqHjo-a{@W{dPc z#;7UmFBg8-v7KlRDN>tc88b-b-n@F_O39p;#agY87ZvJ#GUHO)D?<%FlYAH}xzoxVFFc~L-+as0Ro7>E+^KCR`>&%@2*K(&Uw@hu! z&@&j?6f^P`ZQ1;(%ppu6m>$g6M9?WWcKPL|&z39;Vu0r~H1d6AH)fU#&(|~vR3@3M zohure1BNXzXs%eZP49t0BRWvYwM$IT!>20Sm&d#oOLd7-n{U&>Ax0MQ1U+CHHEv7f zV2+3x$y9(mX_Ewdn*>x>+I*+v6MO_WL1K!StR}&{gOGAd334zH8^LhuXI8#7bLh6KRO15TONz=|%~1fyq2|)nV0b0eIyf*lIKU6#Z9$aV zEN8U~3*n*%#dlB(i(RU6mzB7$t~LzziYN7?5K}gzkxOX=zEagp6O0yNl3CZU(I(;# zz#7G1Qf-souBN&-IKt@O>866RaQUh+Arl6qysSf~gVXCEmIYuH7|N51s zmrtymINc5#W3335UjAt5g_l;2J+=JdGt0-KV`z-Byk$DziQtcKLyXwEd>5^u7-114kF=wvt*Bls*s36N;B_3CV>?odkZFy2w_>}c z618UAE>BsbG&L~4N?Bh_uo@B!_q<5ySgZl^cr-Oj z3viE?VS?zg7zzcUdN~w4-_~B^C@o<7Mjk%H0xzh{&vzwHQf679cis znj3Q`CF}xAw%*N-Zi$OP&}0`Yv_*w$Bj^Wu5Y4zTR45YJ)>G6B8au)#UPaY<6Gol~Q<=fTsq2Yfi3^~H5Sn7H`m+8J@Lkla z*;8IX0pkPIorK4L5g5-whhgU^Z$k3^pjUmNIWR!D|GBkS@a+d(YyRupRDs#}!S)Ic z^|KP}!v~|Qi*d)yQqA;L+lg^s3dK7J`E106Q~(EZ{jC_0flH>5ui1?o;cUmHq})td z^wMc&BGr>?&-q^0KQI2vRvX0fY*NF$T%YC652y?_O$~=^2akzicm^v`I2+ zx@3|uodX%7E4B7H(<1CZ8QJ&Hy_&_Af1GCz`LI3vi^t)ePMaOT+_t6RIACztK>S-q z^@bhh9eaj)>dlQkS`JLC8pfcQEJ!gyQehFNe3U_*vecLa0nRsj`eac^ww@kFqAhSX zKve&CSC8iKjf3Ri88BRrwyNN7d)W>W_lnojWX`qp2aAqVPIq<9%*+V;(UufUtI%bG zf2Ol8F)7DlN{oE} z*_~432$|F*LM9>dm@L)TaTg;{Y44o>;^6!jHko2zQvKJ9lT4rg8vpWC`+06-tZW|C zocW8E=9VH$B!j7Xy9j^1Dia?eXfkB5OiC*;K@66UMj_3hmDEvt6K3>)7s^a9^*Qn4 zDicVrhPxj2jPPK`wzXTopLCLb-dG|<&;4O(KBmM(q!9J3sc1|yp7`QIY_qC=YoLQy zD}VQn4iD`b92%dveO0}i1{^j~Q>%?f=_)?x47Danp^Oo=m1@ zPz}tOi2#B21SWlKo3W}HlS&9lQQ*|seiPbbB|O`}8bfoh22K??A@z#whp+y8^ZYv00v&@p1Db5m1dDyA z$umfI-#Po?(%Gfw{_w-)%gg6pTsnGw>80aKZ+)}!{$FnX`Owl+mv26O?B>&FmX5u? za^iG!yFo(#O!908pWz_-@#sUr(R*T31U10-5yd^*#R?=vwqY7aa}ZG+{i%r})zAsO zzx_<|+nP*Ksu5o$WY``=loS$T%T0M`oiht;hZ3glkoFYD$s1oQK^Dzihb4Y%NvUZg z-tfTiL@E{&liWlfm&j<}SpV*^;mk-+AN-RU=o{`EnP|DHIT4c*NfG~uPn4;uEx})h z6Q&p!lvp4yww|li(c@CL=&+KRt5J{x7A?)6aI_rtBI*S%WfV}9!@mhA%3*du%b8n3 zOAb;42?(@Ykr`Ql_^Tn)^DNu%pXf6wWU#c+$`FafGo_tc^ zOSRmHve)Y`UpFKwA{atUkQC$#R_1CC_w56J@?XUEUWv6sioDW#bQf`Z$;g1;zmj|v zK>Q%wT9YIp5r{N}w%h&1kGI}CbnBh>m(ISibmje((|=g`%WGXLr{B1B{E13gcCsrb z@7wp!umAYNckkSM_026W!Uj(A%@5yQK7E*MyYOL%o#&g^2Pw25 zZibW)`ff(IWtvm^tWob?APB?dKd+p*xboe(Egde2i6QVLZhm`o>F|sH`Q^%m zD@)&gx%BKyH@~=W^MxZT7r$9L`$2!jtO%)SI51ZDMFMyI1L<9;}Qeb`h%$};z z6INuh)PQd(O8Df5OIKF@dq*-Ojsort`AZl1sR!{v9E zKYhiwX#DZUumMLIJFeob6Jw7d{u(v3g9w<2Iad!KiIk~DLGjI+7Rl)pE0@}^rB=ff z@irt}FU?<^*7WF`->}KD;QP8-mYJnae{fh4eSKD8Wu=HhEXarzAs^#qCB#@etd^h4 z#PsD{%5&5P&&u@AUv{#l?&V@S!F{k@m}tyBjaQh>`y5)19(~N{SrhpttobwGl=uW# z4XY@jqbiClqykMjcMv5o@Xjw0D$prv)FwO4gSt6mPem`W*)Uq5%o&pdWmlo){QcuE zgN%EqGk>N44>(KrYkCP)DpoaEZ0j3})myL1rG7a1`>0;l49zY@Cwm+)T+8%%6rDTX_04A>WTo1S^OD~^Tdf~aH*FO2@*T-(Ya^dFZ zPyF-ir`t9Avvl&*50}oad~p#5w*2;y$cqbWLL|^y@suFT8`z%|P9vS%NMFpjM{au$ zvAV@{?4UpfT_;!6T4*M22wI(Nyk3*ZW{&}^eYtf|vxs$<1a_H%i{(QP5;)@Jm1LWI z|1TvT);Bo`cW)!AWez5jW?P^@YfwumH>UEDn^5^ENM#+9>m%HDSo?F~y@hiNUoL#J z@HttyboJ1}cUK>W6BiaPFMQrsOKst;g_8>}6By373s+zS-zUCZc7Zj3y;TtxeAhI#{L%aD=BDB5l5D2Qx)4!cEKD$zV*V;3+a79G=svpUAnI0ReM z->n$0Zd?3yIDug6fYh?}GUlPT;htlc8OzVEfl^i)&U(gs|0R={gI^?**thTgHg7~5 z$58Abf(l1>=f*OzWDO9YuTC;#oFBx606+EX=bqvVtN13rfBfZctA#FrfcEQb93^QI zBvG&hAeOpEZ*!L{m|k5nPlOy$MV_V|9;Es&9%CY zx+<31^JzS+JYHmYmKr5$twCNz2jLs1Gp4Ukr!&zw@{(v8cu`|QBHl8@8}H!H&F%^{Ovfr z@Q8-U;%AGWlIzFFB7gf6c>7bbcxv$@e6e`(uV={O``4eu%TJN(M;E_>FE|9B`s*1u z!})&T;=dh-7a!vVhw(c6o=>ko^|#~rCjZvU@cR7fg`dMWpI`h4zU}(azn;PGKDBre zkDpq+fFE8w1?R3mdHoQ41TH-d_k9YPkt(-#zW(pt^7Gf{Q480^aT`Yu;Qabep^91tYj{7Z-n0NQt2lni?$rQEJ=FDDd8VG7dJwEEo%>>u~tG@u^w70E`73cMVO9N!Q3ZZAQ~Q0)K0kWFB?lHyPr!6+A2sPDw_y%4Q42k>hTeSsBaGaylDu zXR9(*tHz-kvcgMEi?X1Ko5u2c&xMQjcuW*c*a|V>lR-8)r-h_?BR4M9cuRrvO&^7n zpa#s#&=hMo-@Lr^!G+~F58pieRhwzp8%LhMapdfc!+*YU_=6iqF5WnD<;Ia`ZX7;& zJfhvWlb<8>x!uT?0zD37w%DPcr0m(?$$YFoqDFO5touTU$^~L0@}ds(Brs?J;_- zjkzmRH{{3?9YD~*-TEX&@F}0!(*lDuymp(YR^u$NMBHLgXEkHZbSr9f!CjDR;Rat}%;?D0zjk3GcYThY;_JN$t?QJekO&l6 zhEB?jFc;jrHEtaOB3etqRdF8MVcpzrR-}e_jefYsumV$MK?)GiM+@<>B3_e3xmlJ4771c^R}E-1;uvHCjlZHKc!tHHPAX^G)Ta z3W)$^+2%EngJ&lx2|+@LMp2!sJHEXwq_Iyn~Rd5gh3N zFXg?OhHy0vJtq79@t1pe2gimhx~dgKiI4?Eal|Bw2g;xCwdYiZGf}iO_%e%>?b%$> z1RDh#%c2|e^-VfzW4)<1QJZmIsV>)J7)HccTYl}Eb}&Wingw2X?$&EBFQ5Bt>6y1~ zzWMd7x2`N*c`tfaz|wL_H73eF13q#x3Z#${63LC^1jZd|=xmnsxMeUcKoM|mv9=S6 z6pO`b#?r5W5g*;IKrC%!rRp^egBrg0epNPVXyJ&GkO+_CT`wH4Cv=@;tg@z)2mg&^ z%CPvcEPtyjvEV*~mG`@_;b+LkhBjtJm&*hqKs#o6LyJXJg_N`rF3eHh(+tk-16v>T zr?D7dIAzo37-F0^0wE+ww>i^73F@Gg*9?1V{+kVD!QyF26cPdAnR^ynIE$^9PH%#( zCab=7P&H&fXdHP8g50wI`?PW*8B@Xb?KmY#ie`K`~u^l1mvplRsnhs!T5 zJ#%L1llPb3JJz*){xBRwme|cVzgfC;~pu~h!0LnA;=7gCic94~_Udhb^B7)Y~WG}TG*q>~jYxLa*88twcw_E$l2%ECd z2PKL?e(l1LC}OW|&iy_^EgOjOU>V&*NAS9}D1(Z63D%gQU;0Op7Qn zg*-?mkhjKmv%KWIU^=9v**1JY^C&@K;;bq3r@->UV@}s~R{{B>H<`hJa1?qYY=fV3 zrr5Cdn_X;xjpWDuIbdH3IIL7TBuKrH7-f6A`Ni8SufKEi)5l?BYll6o4wCHrd$*4N zW#!v*k!7$`o1(Vn^JGOIOEP~QQUL1z&)%E0H*sZq<6mVr=Xp-(_uxWRrTM()?Z+dd z!N)eC(@8G2rERHjskTysjfis|uz}EoOr3e40|^O3LSm-GjJfdX|FwJvr*Hbi*i6oM z_^-7kTV+XBsw4vz?e1V#l}fVqUbWX=>4|DS?ED zI9DcEfd+95xUSih#GB3P-%jDHcQ75EznpK;>;O+wrui8SXrNps6auLg#1)q#f-j7c)~wg#7SPRcHq&7;Q_!O&HxA_U?q zboJw*dzU}Gcli+Vw|w(?^BFG>KDjje#__r1{>}S)H{W@1;{4q4w_H0?FjBp}2(H%> zgQ`}kdCqzzVG^N^Op%A_X-mMrm&>d|*lg-6(A1pt;qNXJp$WlkKt2|D?|_q_hX1Pc ztq$T9b7oj>8C7pg2-fIJh#+1^BwC(NG<^_p=HJiX%3qrv&)>*j$={qF%TMI5Pan(Q zgsWFb{#t$#51+R1>WILf#^^f?=#A+k)9>alZXo%;2wbVlFrpjz$?5$h{~nGQ{tp*n z#8>hc;qvRgWhoG=Zs2FSoP*M8Q#|4gM-VVttzbO!6IaC2mSPrYB?b0Ga>CnmjDq&? zv`*H;b_6F`4Q8zldweUma*E57_6>F)5nPC|aTK&^w%n<%^^*!n;g{^am90g8kje>byXN@ZyXAJpp#GG2rN=jsEQm zI`p?MSQ_5eiYru-f=F#br2Y5hu?G{OOhg z{s7rZ=|CL}$~F0bd3N`rq zum-=4X|_3aD#bQNB9bZa|3aE9rT6a*<)tI5koM8yMOq-^xIPCblEJoU?LE$3|(rKi0%}TEz(%F;H z;_XyVVZ>%LhUxut1jDz_Xsm6givm+ANl5W7B2|i2R2KOnc-t$WH*~?ou$IfHC+3=D z7Z4kUjx9=1QOL9boJtiv2E|<0?Ts>42t`RDV zB9mscqN(klCB6U(a{$ScdZ?pkrf%RP_T(cAQOr_AgXQqLp2Kxm7&|>CN@J(T^0%g6 zhYOSWYxxP%0vAc4ZdSyz^0%7I_{e{lzmmT;eZaQ>h(LG0P+rKmW2lkpiPuPtylWI= zs6I594}+s~Zx(45tOD^@vzZ#)un;LUkt2l;{-pw^!K9kTnf<2)Sq;fNhXM z?217Oh6HT?Sb>Lq56iXdDuJ-11fxb&)DaMMBLy;RhGnCVfsr)X2C%izY5G&t}%?vOMcW`vRq$Co8ZYKC-H&G!1Poqr@D|M1n@aEr1n zXTTYRui)lzR$YAxs%OO<3QB4PDB3??v(FdP2SwKOFzI|sy*^RIIEdAcPuHPY2pf`e z3bsr^ciJ43+GeKdT8bH#O_MgNG(nw+e zDrzGUnSPp~CbAFiwYdTQ#Q)X)A9$oXO0ZSe$#&R8P3!+|)c#*HD9@$x>7YfZ*E$LZ z!vQ6(Voz4Itu-EN4+r9H?Ok%bt5pug+d3kVc&FOt5?x7ho&^_{!qJL_`}m=Pno7m? zBPpiNkdkhNaDn`itHdxY9jOphz2TGft&TN9)B5k5vU9vM1uCM=1S+!6sX2J&b2{ z_8eA5!4#7oK{2$x51w6`I>23 z@la&DX_D1f+jkQqSzK*$WaXEeTuS7KV{fulgH?D)JG8?*e|y z5NJV>*5A%@$$AL*w825$g>2jst7>J$Y6>AD!L>#;p2iE!=0^j$nHK^cd-owmh9MNDPs3qZQn0L)>!#*~2UE{QAjR^6STwj}A>f`sTa& zk53V0{@~=xFOx^#P0kBo|G|xiSH2^U#xBFr{E?4;+5gtdFV7!8xc<@CH-2Aw)3&yq1Rm6f zKTVcyefj0}^Pk`Nz4(-mzkhgje^beM{_x59&)$KV@Ga8$4-U?Mb7X$(JD6B^3}YEq zndR4<96&VHP^i2J#Lt?$mA{a`H9Zz5SY!bICa@A=Njd#!4@}NJ0esbDDQ{OKAHRL$ zYfmYaBPiZFSy4pcMnd_!jHH$%CL7j7y=pib2rH@_4%W(W%dvo@D3T&$b;{_HGYt8= z{jjCC;-%&z*#<8fTg!VS(qmHFKjYVO%ois*+%ZWz zHw|W5BqMIxKrHUq=@G1JHVv|*;I8-qr9b>HCZiTSRIV=|SVl}u_$3DU7X`mgKQAVc z3a&mS8AQA?QaZCN0^Um@9R&{w!4Y7hi58ELg09e`ElEw5DC&k)$M!iH3KI-pFSh}H z>l|r%{b$;FGObzg^p4Qd1E`j)Fx8tChR>fOMS|sSzvi{OX6^ZVYR{i<^WQzSPxFBb zN`VTb2G3kSYY$u#DZH8SF9m_$;hC{(Gvj}KFn*G(pBcM^?0(~CXU6|L`_o_R0Nh!* zClR@C%C}zAtDCd-BsP!g1MftGNpp14M-8&op~eQX%cfP}M4#Nl7e+{;dhI?flQl(M zW3i8k$YdnhT*{$GOEGvTv&?^AakrBh;4ORbCo(F!Lb`ry@=*{`?Emu&x7)Anfybu+ zrR%PrF=>t!df%7nU0K~gyW7BHG)Q}zIwm!$HT$=a@im4?z`Rxk?If2_5i8X?6xXUn^t|4mxs=2JGWw4bF6s(z84x(Xoxv}Vm zFHI@N`}t6gr!-TK#5P(WxBnO#czr!YO_#S{-nBm8!rW^QUwxk zO@fJ_A>;_=KCwL59ES0su|QLxA1s>;FYxO5eyl2{)P=j<5{y-^3y15q9xqjq0#VP( zy8Q7f^nz$fZi!;J1+Kg#tI_-skmIr%#V(;}Al}~56^^%tS_AQz+8K(gvDQdj?FfWM zJ>FLyPV@TlSREmunMo1Waq5~5^FuM$zKHhNo`#(yBrDBj19a0w+95q!0)F|A!&zo> zpYDF9sy|URpC{Osl)8%^M7}+(4HU^!{?A5i3K6J*sJ-B!C*3>yR{ge8R?Q6SJx6-M z#4T7kK8gD>rht2|f13}=QYc!Up-5}XnPV^)h&`Fs$_=uAocjJBr^fzq>YIO@`uQKH zCYCYEw(O`d%C;EE#{$3P+iCg7sc-*r>IQuGe|-I>_z9!=P+Yn7k5k`?pWjq5)H=48 z+i5!}KA#lJAF{~7OZu53oZjLR=L87NBKpo;AX59BsEr$^u~!t z=MK$Z^iNc~as^4~{7S+4t6dU|2>J%vWU=qD|^ zUaIYdnQ^tievOG%irqg@a;y+nAM0X zYZ`OKHRcak^-b2xv1s7Yp@aUj*DI+zl`xp9jkL$D(FrxwHiXPNu;#PWf3MIAA*H}W zpvqFP0=>%cy3qekXW%O~*XB9(v1zbD%^Ir1o!CJQI^>G0_($r;DUV2@a3kO}?P-dL z!%dKKEQdDKKncKhw;myYcQZDG@Mhdd`PYMg+&Sp>)MXcPn>h9+iJ&6Mm6TbJq5rho zF&*n!uu;T77Q0W?6*pC4vf^q!^;iufBjtizk=$PkReB6nGSh!{wo0tBB)PC>!G#p? z-#k}kER(M3LNIoM(Ckk+M^i_vl7jlcAHRDHsTc*+TQndkusWjd(u|U-^em>R6sz#^ z*4%-H-GNJ!g(lFaLttG6|R&lA$Ss zA@08@7Ad+95@7|X)84Kv;#h}?BCt6)_QD%28=m}xL=rkRwZU9-S)A~<(2;3@Nhvxy z37*X|vjs*2$5>M4KU3b*>0-bRmhwbFk9i_RKoUh7sfg>snII^*EyghHnPALeX))el ziOapBq8OeGi~R0)&0w$gFEM-YEZ&2UaF@<=Nnzx1!Qbc~&9;s0%kAOZ#cj}{R9&u0 zO_4=lwbtGI03v&E;Jy1l{m=}eSh|E}FP@!!{o3s0nD2@2Gz{1(Eyp$SYIT@rb$OU) zn0h53sTGY*p0_}l`iEEE!QT0g-XV{U?T0;(Ji2=D*JJNs+0Ub~GxNtkCi7n&!kfOn zQGDsw4^EKzq$aQHecCd+bx4Gv2fc0L_M*|n0bEl(wH`&esw7Ye7{5i}QWSj3~> zs0g4Vd2RXCx%`~$3M{->sa-wh4SEse7DN_x?{qKd?{DWul2@=}_Lw~B2x5fQ%=H3W zL``5)c+hrGY&B>zxt+$U-!`^sQ3Ttr)QpS`GkTPN|loJ!_bXAT=vv?3!Xjt)H4$P`#NbR%eVnM z9qD6cO3PA1^rLk-Hbb(`7ScC$7M3At^gD!d*S#>q76i&RYzfhz~GTv9h zh|@$`H1krE33jad$S%7aH?1X}JyX1vf8x45b=cd4}15M_@Z87Ce!fJ#_+Wo(46YTYPZYZqqIOl|gf`aC47g*Yv$=`-n~HJn~6I8}t1 z2`P2QDt|LmD z_G!53;O(=c?mO-rt#SuhuY`Z!WL5IFzq)e>{(ls{{q3)ApTB((26Lt`va#D2;N;4k z!?*q8bpB~*Nr_RgB)D0o zrd^1i)8^XkdcfDokzS4aZ!76^GF;EPc^>^1j;RTrZBlM)-i65EF%qd_xn?L*w>6KE$i@PYI|Kqxnu$s?j6u3DY#$B84iDleDB~hR^{=pg$TZ*a|YZ-R)r+njV6)Gz=0P8S>J) z7hjAsBffy7)C->q`>VZi0gZfPIH@!L(dA?HsD+KT=8n=pX4N&d_Ht^B3@Et3BkeoP;n z9-kh=8}iqu$NgJbVAwREnGx|tl9fg{zg8+chzwgf2HO^{PB>{1C(p>j?52CPB-sjk z)k<&1bu39aB*lDS&URxGJ6VA_Tkurr9%hjqQ!^4;mg>tPnXY#DSxx7pe}p3g^>7tB zgmo|MDw)p3um6GrzWdZkqf{|dK%14NAf_i=u_M=B>RCcjFqi6PwlqZKsE92~Qra-Q zv&mFutC2#&Jf2}m|Hbkhn>ID-?+4>bP?9US5Iyf(LZ{!FejORY;NRp5(te%!FIJ%ghAo+^~M)?!$Yp}_Q5l_N3xcy50 z#`L@Sn^^gPN+=4%cQ-p<6l}MqwtHg zMi%WAyt+x2#bx*{(3scv?`}5Ysg+ZyN~vaE@mMntqJu51qGsOnA}iP%nvrxYTf~Ex zsXOYZFf+Db`7g6p2dFA?XLF01+6K~%!5RN3gR+eLXCVnIp#v_%-(~rGHvNN&6s#ax z^p`or20c<}WWbc6XxPk9Ec-X8k!Z5iE=~~X;%3ItDcK3`h$Y77-(DZlOg)!n?ruG~ zdb^%nqhBwGgtBV1Zr|+17rS@%#XDM`T-U)$31Vl@AN^-~%v^${om85+D;i3gD&$#H zA#iP=d|CMElAIa)c4qAA%-GK}<0k|O(d#qg-!7xEw)RwLtgS^%pQrB^&5T_ULpVD# zc4}tq%*^=s%-E!F$td8>1VfUn!UoWg-x>-@fk-2j4m-Fl2sliZ;AXP!NCo)gCL82t zA;$RZUgy+J_a>H7=ZR>F@)0!YVo3hl*QNzFFIy(C_21cSwT#=9)87bs3fDu9* z<7lLAXrv;$i92=Mo~F#^V9MEAlCqay{&!N5-CexpSxMwB4u~YV&Mn8k`_+R9z@!OZ zvQ%|jD!KJ6OfWZZ8*!d6t0>Wi0fx!hfJS zu$|Cmg?2)?tv#*jFzKa^*FLt_p*rc@o7Fio!HI5IBe`(Bb-=OwLwGDl1gyqN5{`Km zaZC!t>IRi7)Nzt6#vn}nUNXS>5V4CDO+?GUJQC1J@K(&h*_;ee*gFR`U9c32;=;wd zlRV+zrxqsZ@o%V4<{8=n{*hokG(p${%)kHL-)h)t7=+uSkgIzXaxFQ_;5Mq|o9SBP zbA`aFHa>6TL%-~QZyR3;xU)JO_7FBc#44(-2Yq-Zx4*n|;LcxeU-}d) zF_88DLH_3lIC>q=Hr~E;`@)?gw|~IXjdu=VGr*lge?NBn!tcqQBLv>~*6qu;&;7Fh z7~X*I#a4;ir{Uk_;zf8De)8?_;NQZH*jjN!^b-8=`{iOcy_K;W^RYNWzQZ;!4Wc8& zx7;J(z{eyxj8rr(4ebgBs9sSDLkJ-g4V3HXwYHb~+g@NsU=&BmdN?}T0xRRF7~Mhq z?Z4W%yxpZ#Y#j7hZ44rgRAGwnte&5H?oUo)h?xUZH&8=A_Z%5<%p@bj%!G43z?#kU z+D5Y`w^$t!IL554E?M|xj|w5gOSt~73N(kT0y#i(^?Q>`Cw((B%pBP%kv@24MyPJr z0d!nJAmwWG^Xx+kpyP6KhJ9)8yWPEslXK^PdfKo8{w46>lS{L29G^SxAKLNW&37K0 zI6rs%tp?(3p?FZ048m7BMPw#M*$ zi}bg;Xj|k^w7Me@iq;fzSW+unq8lgz>SGDVNKz9yP1lNm%8P*HSSj!#jy*2Lm=vra z(esy!xHvO@V`lsW#yRia{CxJk1NVQvIeYZP%=q=0@pCidmuAL(xcl?>bLYQmGA%j# z)))6Ke>D5{8xLOhZzavANGK zHrd?YyK!#r!yoQmedoc)U-+MU_-$-4hOuU_fL*&Fg;2>a631So5RL~_SaEfmY636V zpbODF*s+k8Ipq|t;CQWnCnMBE99j}w^>V728Hgw_Q>qjK73H6&y#rCgOn1>iG+L6E zIQA$n#3QPRVtDmqMN4ItTZ^p6&6Qi*N)7TnxBQ3lTQgH&~1Hbv;r)h=N3{HTy;vU=M zDT1xJM=p$|j4Ch?TXD}V7Axfd#IZN6tss_L-JF%Jg4~$A`f4`YvIxZc+H1uE*#6z4 z{kuVmO{RRG*VsswYp{P{%!=#2MYV%z`i_!hTz05;GgN`q7ArSve1)ryoWFm5qWQvA zv!7nRcjSuy+JrQ>c}{bg|AA5ojNaW2iufsCMPh(Vc+6}H%W>dkQ7XKgKx%o@^iRt~-x|G(nZ|*D)4lbROqKH6S z$uh>*qf*3`5>{Ny zXC55?a_-1Ccdz}~_fmTSd%k#!Yx@e8L>Tle!cb6BDwxrp97vcjd$W2=aMh>oi&dU( z(uu`OFbs#9nk7c!-wr8936n#vD5PgWM!_my1G!&Wko%8M%^$t`>-cf9zBmW-2T#DL zo+j^gBpwKYy;*nHln<|ZE2SnkVxcC!`G$P{F6;LvMoPz&! zXkuuSV>_Tc_W3kKG+Sm?jWn*U&`CUnZ8_0Kq zWeBV#p`|crZ)*viSsQnAbI3|_|C~OX;FYZ~p*?P2&lNfGTG<#uxQkdtuzGFtYqHm-HW{!R9=eq_k$)x;us2B8RG@ zm%#O!q6Po^K=1<7ecMZijR;<2j4KuBInRdmU7DJGg<9 zquF4%W?k;R-x^|&3m82WB%i)$A`NYjn;AlN^mFZWrtX$yi<1O$5hX>-vc+49XU|B- z^7aBuy#TV;>OpK+QG~S!Mo)vmMtYWaihgAHc4(irbJHv!^!ITGU%)aYD#i+8L@XO! zM=`#%&a#~>GhbSVO15yuO$knSaCA1Yy&2xLfe2kUt}OhP=65^i!G5YgTzE6U36vfPovsTpfF!?;Cm9WE3*u| z&0w%6Thz?qU0L`($PS8*lV)id4ZebtWE+}+1GT(Glw<8io5@G~KYOi7ni(+Bxozx` zi#EcP0*~Aa_-u)49U;1uhV7UQVmfYyVWC?rv`+ElnSn`Yry0^qE&s^qD~pg2iH8DG zM6JIfpmx1}DTdR<3~g^0qo{$(sv}yRlw;Y?LZHe$$Q2J!wFF(U!g-+B4vc9V(0;b;qm zjnBRMs^tt=cH9|hv6+Q=w*P|vJ@=T_k|*%^hIO7ShorE8JC%X~-=gb=DKf##vl1GD zVpy&?L?~FQ=n=;rMHrSM^^34qa5{k0&^EQuQho8o1yX=^4#Lm|9AV>b#BpOHUIBAm zj)X_c&aw8?z5kWpju*w2SAdz>G`e|{I8`NClAlz;>Pl(XE^ab#2@*tzG2wp&SGEWE zSFKS*BZuj2pos3lH=oQ+{xEm>*6iC8bLWrEe)bt29C_#7pxE2J{`Pz@`jn^$Qe{4JqXpoA!qQeT=l_9(fi z6s+j5S(BVs$(jCPSt~^sDQUlm$B1DxuE5?QmnXv31m{_B{(bRE!Np2TF6Mc~DN!j} zfve=5r*;7pEP;fzPF6%=Qtl3!4JEj{*8cd;QEq18?|_8@uEVnuPuXO`*mZ0a({+qQ zWD5aB-y+@2XmWsO21+7TU3fd5w?E`nt!TYogWIWFyzu?MyK@L;Jnjx^eB+>|z>gcxr7S?539DRqoqnKL@1wC`vlnbnz=BBP; zqXZ?bQx|A*0okH?WYvdof@!LeZkpBoduyFZ1a99;hrv7`DQ@heI@?EJDu=0AD6*~J zZvgi=Rsqv&4SSEkl**bG44NTg(M*OKgVeUM5x)?g*_ladrj~+TGKmq_BsGYw!QKHz z&K`Ki46;*CGG;tB<7UO8GiB)K;#+SZg+cfh?Y5Z7WZAX(l@y}tSsw-?QaId50uxKo zSw^(04>PjGvYh3(p;oa)(ngITgCmqzv9v{KNTi=g!e>(d2?F@Go3$ZoIsq3FdR?#= zQ6y7U!{r(QeDabr}_W zrWNbspN`e=3Jl~*k;e)7oG#~ogaO^0J|G75HocsRI zGD79A!R`LLN0;&*y5M`fqW8p*t~6AEr@5?-!{^V|XliNcH)=Y^G|}YCn&^pUGU$s{ zZ*gnizOGD|>kuj$ckicP{@uoTrobk123*1BP*qagn^Ul4`UO4OelaZASL(+tL6N0lVDM$zbeb4S(~%*IvG5QR~lsIi512z9ZMG|P2Io6-mAeRm`Q@ceK$dHsI=Zfp4XeKNReAw@S}Y7Fa{KqGM;VvQShtw>f6U ze{*&p9pNr&jh0|_TFPJzR?3*bfUcj4LE!Y`{?yy`N75@;yEYIR&RVSvBmlqLEX-yy zBfh&Q-qYFIzWIgG9!3-C98shISx@zBnv&@6pBzF+Lq$GIbKUXw+yFDPSfUe-L^t*h zE=+-MNs12=#XYWYe<|AUbuB3&DH3gf%+PC^TL}l`csLl2M?2v_iAAIF_F!u;-X05u z;+<{LP(0Yt)iLVrw(_L7#}6;&%Hst4k?;w`;tA^+Nu9$`9E)`BV|BPBm1*-j*cD@# ze+X5?FFj-G4a?w4!v6WT)%NVFstiC}L!Y~D88Ycsfcr)j6rc+SFB$83USq~c6tP4+0X zQ;QDh)JXW>9(q~Ywp?JCE!9zdYhe1uu!l#fpEepStTqiKu^fEwmK_!D%N8TKN&q@` zCTtE`naKVD0OY75g%J`HIqF=Yfl$dp3VAk=)SwiU>)xP!bcgrSBod(->{$R^TO7<2 zKwVQy`c)izlxs|iRsPy@n>Hmt#bPX*IA8 zz$Syz)*jegMFsxKesZGt$(?&?!p5Ew3zdJeCCNt&l3W;3{FnXTtac^7P!q8v9JGMJp(pJPQQ*lVJhcoZ$V7J^HI& z8dbclK5C>io04rH8dKib4r_*qx)wG#2*xEDVFPuz(5}zaTwz2d5O5_b#fi7|6ADPN z%Dlf1@W4LoIV=DN?x2Z6#=Pmmg^1+KP0am^j#`*HGB|b39^g6{^#zeMUT%~}d7+uk3HYI@l z8{MwgPp?o2RhATK>Lt!IW8mRdF}(JlrSmjBWZfn1PDP&4V~UQk?VP;XgKlvTJ2 zBb{eH!rP~BU%7J-ex4WgZU6Jbog;VtOzs>;GQ!&z?!3P0rE=#MQYGFwjAV%=>A0`P zyLlSkkd%-TRgweJo~-}c;yZ_KpND@x+&PN%fbO~hzc_UuGHkGj)?L^k>Qjl66`ZXv zzSzC9FW%AGd~~ck`?mX*LYtXOu(Xp(FGcLA9>kZI58|T?LQ*8uNUJu&9S9Z$tC0Fy zXvk+|fG2Yc#VSSX5mRKXI_&7+p&@#|Cc5;|5-XfxcF6(PPKUp^>g8CCY|CV<#pF~Z z-qYNK0fE^8HwZ!raJ2;a?m{}-x7@qAfpEM7E(+~%tmLVQV~(8j14EIoc z=Dl`o{iBPaHB=IB$g_A=r4gRWHIADG`4SmSIm|=9Nm;56u#BoaH<9Y*R5ViaIf#OO zUL&=yQ$N>1QB zjlP9Po)&bYL_H0pw!csu>T#~kKJ2k8%)eAVTZh}r5rd{W$0{LT-4QfU$ zy3TdG`YI|*cELLk@X#*SyFUI3lTaX7GsVdEYF4IFKj8gUfCVF8lLS^Pe1?|LTVCCAeZB*@}$Z5@Dt8 z0WIv7>3Nvq*@((Z+YlF zkccdUsZ)*u;Mqj&%W@;d#C=i$R%CT=Uu7y!eET*&fE7R^G?N+P*};vSSfH%*-Gia1kd(N~!yZx5PclB_hdzMjBLmbZ$3YL+W<0&Tw&t8?kl+9h-NGJJl0T8ZmA`-;t=O1aN2bUc z;=`KFU9f8{!Rke1!R3p{P&BbdIzuXIAo?^Tnbt^_@vP3Cl_H3~G);r<+m@!UP=mA% z3{b0%hH#$JkekA_l-z)8UY8O=1w|Uc=e;|5{_f<5Gvn9q|NJf-z-mHLM&vF0VTE}a zpNw!GjGy!^iw9qQ_Tbi=v)}(X_xYLG13#8TQfh?oh9HUpiAd2{-3#b95V5;?5@D)D z;d&pY-z_*E&=+ghiQFVW*sg(;M7P}Ne9QB<3i16`JA+t*iWCafE!KbA-9y^&ZRYAP zsI5}xu*uQ%G&8}w+s>%}ls1lVX+9JCXk>e-k&` zN!W7{6k;NO5iE!+t9`93H6Bh(j~7rbNDqdN&X8szlbjr zxEY}`#>AV%h5Z5mW8A+31;+YPj8LBlO(4XOECnjU3~PB8Z+RE^6Dv`OQYsvad91gT zku6x3%M;#fkc4&17iR=UArJ^*YQ#{E!v7EZ7H}B~q3Vjhd*0ou2H$--WS@d+xe30@ z6$I6p9&1jf^}pnQntmGwF^QYRyPzcxh?DE^Z_>A1zz|f>IF}TWTxmdM#w4|pzR*u< ztq2p;r-8y0S)l*3(F^XY$+E~~n2eKA$JGBLZQI%S#*HH*Bhntr<|c*}H(CYzQ&oeJ zfE@gT#a{W3uxMQv75Dr{@r7rlfnkeb*)WR*d`fL!VG8*m0qa{&!Xy{!bHH+^1IQ_* zF@f5#_>4D-CflM`r&m;$=T}q|M4O5#jT9^Oq;oolhog{GOXw7aGl(sgg=dMp`l?mz zUH;!nO zo+`!aPXqkZTTIqUl&biSN-Nmypw?d@*uyoWPy+$0&0Jh;O}2iv(gapjknJ=<{iBI? zW;6q4qc4%>1;EQv$ECaB(z8HRDHLs_LS?OFvw=sUHOax+C%qDBa}q;vpYO+^YSHF0 zJuuN4O)v^3@G8ks*EFg}jq0of(?O&pxQIs|RDu7baQ(T7rHBK)88B~?*qu4lWJ>jU zEkkv>(trbM^@XI?lt)yGHN^YbXFExagGdIPOcO#~_EJX_zWI~E3(8Q!r2BGv_z*LL zZW&as5vjq?2r^I=DOy4I>luBk9&i|eT*}Nga7+-Nb2HDdO%5DkgXvPR7kr6`*NDaS zBX<`k)*vW3j z{68DjY^z{(6gaqE^MQy;A-QfV+Xo0)J8~~@qe=3eH6h4~Ls?AXMpiuGDMx}XJPdK- zZG8t7Nmc7cK-t0VMb|ZsZ0;LUH0zgkdQV@vNeecp^o0ZdO#*Z-*BgWTS0D zh{irJmJMx)k~RNOABQdhR%R~nFY9t%mUZ_xoFb_^*DCg3IF-1~9NQhnANPbhr^pql( z=Qr1&oR)=hdfG@%AhD1X3RhHOd8JwF0sr2dI%V4fnw}K2X*88w^?Sr zHscC^D$&yHi(`+ouU28=T2^s^IjPeO-N%>GIKB8H!WfEBmPV@+20o45r6N2P3ad5i znJ(QKyL06BPvrJ@w=dlO_RiSfkKH*)ZvSxm0=aYOf6fZkR=&-2Zz=k*FICieDKC_jpB+*(TYU+iW3!d^W@YS9f81fGRVx#)Y&E@64;?7hIDF? zHfAJkqmI>8PA3p7S25_ zCbM!8MIBu{dimugOfRK7em#8R(a96w#m*o9n8?9sKv5&HUyr>*etm0l{`9-^#}AT6 z?|l5|?PHI=xv_yfI=g@V?8#sMdW6jX^b##Izb*?KJ@5~BXF{ENrK1qwxCSv03 z{9*siFn#G9=Y|X6xA!n-<)9)|WZ4cY z<#iZqy#NjO0>JO1U_5`1e!p)4m$d=NYAJmuj=e45Aeh<}MRJDM1^jP1Glo{qp?o6d z?q;gROCnT%{P24Q;l5)AhdyI2lEv3iBpXyS z$%uwjSC&<(gaFNLz>{qi0!lBfTjkbEO@F1|1g{NCqWzZ%_kokMw3pG}AUPML9$iIOSMchdK^3z_B^%$6cKZX`FDcH$ieT%^C z?Tfe16fU2}eA%6YaMz{3AM-7@R(*h*`)JDKE`AC*=&C35EV+=RRG_>1iTZ7Z=Wgnf zxfb&Kt&pGZjwU3;l7dP_{M>V%_5wa^x6j3iJ7at9_O-tsgU9XyI)nHdp3Fi&G+xH$ z=+{BXicIcd^cIPDqc~|cC(k_w7@|1wEzdu`GEi2fxlEB}EkPm1^{r1+f@DTv>1AmC>=62^RAtw}=k^5;QZQaWoQP-SlNy0*tD7`vPd92ms5m zfTT#WqDrB_sQU!-knRBak{dCYE@5Px5}lH%YdQ>vB4BHQN)rAa-j00UGWa*q=8ERD z!(5u2Y;}gQU@1w!(}SIl))Cp+n#njRhk;$=elSAI6vc;X6vYQcQGETxf@`g&{+@w0 z(@8MWB_Z_~^WSz{trsG^OnY3oMt@vTa7-xW!PI57@w$FKlv`RG0t)&5&2R2~`w1d= zo<6cn?cvs*it3Q9*yr~cszUq=WA@Pbd+%Sod+plnPhZbo`tjaRpH?XZk=61-kp92D zIh6a`o25Ds8LUqQgKLr;b>${~Be?~!;5r;Cmx|{ykX|svsbdzxYDs1-tQtFB@d`B( zCj1&OHEZfDvqsC#n+^J82S;BbNi%gYwf1mFcRPnI)z?$a)f9qFNo@qhqz`5_2}3Bv z%A3r<>A}c;m(FuZkqyVfmTcO8^JyEy##49WU&+$(n6C(8kUJPCugzH&Eo$BKT;G?r zhO}I{Zp)t}`ODM$r^oY`@{{=+`AL$W5CFK70(facTxv=nspBvn1lzj_|1NGI`FHZy z3&>dT^D4>z1;%z=fc+kW3m0L$)ym_Z7wtbw)FMn}3Pqo^hG>qGodY_x?B_~aQkAhv zPLfqsk^`>Z;#etI8uQqh62$U(CFYq=e%&U-`T=um#3+|@r;$W%OuVqV>*ul1VR@-; zry~OXgKTH;q@|F<)Y6hLD6q`YT+dQ1Wng!)U>&b0cI+q@W;Zo+n8I>xauHPuhb1|T z7_uel1p}@Mx`4+Lx)9dY$0{~DXUHpF=4E(xFT;a+Gd}^FT>eu8jfAK5QvMb^w;xWw z9nXIzUcEeh1YSA_FW#K~u&{y;!RrMmxG}stKM9s({wE?nvWQtJKIbj`;J@U*cq~}Z zPh?ZffAQS(`_p4E##;r%xB|8&C@=rfZ))VItD`JjTC-t~Yc_)S6-ePhC6cX9Y=x`3+NA(egrBtLn4Qmx0 z=(LvgBBjDCk#5HT7db1!wMCIBK~S|ZDFQ}m*u`Psy9XwODT^u|qSw#?2yN<9^y9fj zQsC@@x@980M36$54TF~A&1dp#GC_JhCd+8tHI6z=9%B!sI!&5O08gqrbTwrZlY-vc z$tvp^HG>v`@CoMa4L$<922KtcO7r3?O(r7upnMBvJ9BFg}wL~094whhuD@f#6 zC``IHE3tqo7eAwOFHP8lo5X?#ge6;&U{sOh!Z(PEuNQCVPc+vqyL6)chj?oupQt4d zNcHFfh2m;Jk}IK)mebekx~I;XT9TIUv<2m6kz5fAApzf?EoUNk|FL4r?Phyd%Wk_( z(>duM;n-n6Tt!K*dtq05xdHrw1CR66NmsXz9F0keB1^%tZXeiQOWEYG=Uo-tAf>Y7 z=Sd{G?i{%DI=ufm(3YgTeNANDaq3Wi=j+ zDxITVZYs|odHP_yQ#4!@*nKvQZr&tLRY{iQCzV22u{;tVVx*1dEYiyohza2P!Rg>g z_YroHUbHCHOV*ScHESV6;E9Clu37W`@I(Sq7-1JAH8$!xE`Jo@JL&twGvkyVVz zG%W5=^(1nrMh-cIXgz90(8ja4?X;5_FsN2c?TM0LA)NlrySm%@w*0BH*-XFQG~8AX zc={Gyr+-3h0e9A+db&uhsl7u1Nr{!`9sJZ(-1}_u!AGCZeR=%eXWwI~=j@k%ojrPD z?##)#<3BfbGkJ=Rep7!q6+6%@ptS4cOrEUcq9 znS`l3%}#@pr0aY@b28v!=?PZQc#l<~Efv5*HO7E<?wzN4NK35|(F!BFOC;tQ-9;3x=!|SKG0(ju)Ud@!=17nT%3YA1Y32Uwzy530 zrs1MUBwHh{kEnu%m}OC9ktCqQ$u`YMC3yy0#PGD;%vqrJNZ*iCEnu{6cLmtrlF-ls z_()Vzz#v1UtDtM!YLS2_x39M9|W*oFL9f7uO5INLj%D9g^S@yS2m@##pbnM~CZ2aG^BV>3NgPnz$ z{+)qsG@-c|L|j#zA&;%mVML^nD`>bhue~YVJc&@apqgMHsgwru&Q?FPNEzsv2CgwD z+qD7HKi~258YrrE++JPKKyt*@r5^Dvil`I`)D3&)E1(vNcLoB1cxy*zAl?;%Be^3K ziMIumu6So#YgarP4a7!0-d828=IP^$ah5pgOp4Z50y{B$@nkp+0C?lY5!XHPt-`e^lgh$JtnEth)IS8_jUd1w$+3w*Ndil7Ht>D zSp6*6<@n(=N2o|rckSL$b#+Xb#u5*&4~jL~xs zM`FQfBoqkN+5>kmP1j5Y`fapySiY0FZdynR?BE+n4yFrBT|gxk;1XMiV-8S*WWkUS zz4TzBQzT>>GO)w}X1MUWK>rYv0GU8$zX?7ZH{DkqY+PA*zeIYMoa5(ZnK{T!j6@)K z$l{-C2I}>t58bXMQ|0@a>xJp!8MJY@DT9$wdhirifttr$dKkmtu?kf4HPW;SsZ}LS z>(~4TlPXEmR9D6W{WXMaiYl})=%Ox8)oT(PH(#4Yv(JDmKE?F z#YoYvAwe(&E3FVXBVF7~F#q{pFy`fWIH1M@prpfcJQD5fh^s0X@@lLTT$Wh7s~*23 zNAFSupQobzCE>M}Ma&}TVfd~#C9l5P4zGZA2czh<*T^dlc%_D})2Y*x(98hVmla-* zqt%q+W-i{#v6i;oF*UmCD@@u8Us_Dbf}62$Dc(MH4dEfgc#^yUVZVscvXO;V*um?F>a=OPo*Uq~Xafr?g1L~eQW2M3 zE4_YPrPrzpYfJUYLuSOpbMMe(u88x*9{(kvB<~J#`R=`*JEmX;ZZab+SD1d4(qR- zbDa**G|d(|Ar{tG$;DSZ_Hadz<~SH?#5(9@36?Da%T-%Oo7$Gk@)onGt71L%v!Nko z5dWB0QCK*UP@zoG%}f-EsxEC8_1H%l!5DF{e%k3GtP3oWEVn?v>5L46jzIr6Q#I~l zZImSyHai8DerVKnjHIQUVEP_K3vb=BwTiW zhQOnsG0Xo}S(+ACpiZrx?^kQ&`y&W{9}AQ(W6wAfm{o8F^dfoYiaOd)L?)XUH&b8< zZ$d6Lfnuan($5`}*|z`qX=`Tctgi7whm+NfC833ClzQ{Y$ECk_?MI%mc%&5VCJGk$4i?ApxOw=?61XT~nijGw#z^Uc|#C+?m9b8`u>edm@P zb7#NsZ}XdUh%uLk?CR!=c(yTvVbHnd*1qSW+Xu&fxPR`{-J3t(d;iSbnUnYae5A?D zk-5v~W)EGycj}h!>7Tp!$=r<-vlH*mUcLO_)5G^}y>;)%Ete{bRbMKxn$id`Qw~OYWLi%@>MEA+fnE^^VEqm&$Sx07~R|AomNmW*s zlo0yIT#mx4QaDg|cxWf8VX&F7#3N322^G70;hkiVn}jAs07<+8r&*I{I6e&$S>M0= zarLCFoY-nMrFCYgrU#RZW$BPf_xKiL$0k>pCKMC6shcGdlq5`QFoIo#G;8^f#O;C=YrzI=kUqL+4^6tXXC1YrQ3TS7u9cP5Yy7a4 zC@YJUN@x%)RIH#3?X0Bh8hY4nT#7{z#ZELwMt zek~TzU{R>4B}lhKx>=6Z5&5ec3Rs3gBwBMf32goB;ncC}}mpjH@Q)_KHS*>>&le zxPTeDP%x!R3dT9YK}m5zD63&NwxdVeCWa6tp$gs(&lRDd2;zbm(xJ%Y)@3p)!QtY@ z;<6kC)oIhcS|KS8tCJP3uX=Z{e0?GZD|oHj8TSukvN$Vlo#_{nH->Q6G4F8K)waz- zSv$4U;HxIUwzkqdE2xx7q3{ihlaZT~I(s!;!}U5)ya!`5+d+J-MhfXXo6H$Re*&eq zIP`^KEwn|HsS2OrHAJAbIcQ#&RTPn3#c&?$eq$oYS^sDd zVn0f&$vDgrY>J81-;h!cnqB(pPbCZ%Ln!l@TJF8CC7ByeGEdx`KX`r>C7ENOX{A^N z+{{{h&0Bo!g?%d!Uh^Fdh{aU}%-BX~JZIk?zrTO{{-xLFK5RxZ&l@)%oc(Tg;^4h& zKl@J!tpv&>h#(A>WqEPNvgM&zJ~U9_htC{%|NiUe?qBrcQNiZt0 zp($;8GlW(3>C{IUp&aa^rGs`BYmz>eE_2a!``Pb+=ltH8%Z* z@OMyKz_V?qBO9=GwSyba*`&v@!2s`d29TN#M*9MIOJp}vs@A61xn`(QJzF30A^n z%&^kjZi24(D(Ljx+|~v`;TcIjLb|B|FZuV|I~{KF^b$KI6m-?U2fg|3AQ}udN9y+P z43GjAt)v;^TiDG<`~hElGc{63i$Qv*xd8C*llY^fV{4y#h4`lXKtY5gwZh->EC?{< z5&a^|xsFzcog=R}++O{`#ws>_o2eme@y6}cBG0oF{Os)L_U{N6KE<^@TxeGzk8oEY zmS8{%hwJY0Dlg46IL0A$%@`701ch!1cm+gP3?WD?UbNE`Or`{;-{1wd7i%}r zmIM-51^4!IKX@`{dqR zXXh?`F#FEIduP7C_rW>*`RT#?<0tR^`PkeyWB1;AeVOde=4}<(oy{C1yN!!j(9^HW zoc-*xyVtJGzIkBw(D~ULR~}ruK0D!`R#lOsk{pvIB`Aem83MoINsw1ak)joex;p9L zc2nE7Hn^+0mLr2(!65QeX}k-ErDy~k?+{2J)^Jog8Y?X{aqLlaVfY5+-nZwaHhbvw z?Dtpa-n%?|=+LScM%fxVb#C_RrMYY8{F{BImWYM|vg&GEjf7nYJ7JIenv7Mh(QtX_ z(3f-c)s6Xs$D7hEdU*NWM}Pg8tbh2^)%m}^`*33aQ*w+z3ZMroVNIVPth1fA5@}6O zl0ll$^xUeirK{O=S{jaTd9FZV9f(s_rAS4l+OtY4q`5l#Mt?c{&dV>)UcYwl+c#zp z9|ARZckMyC(g}%e(m1-S7wi%nLY9mT=6Z)?O;HrU_M)x61V#<0zzp9 zrC6};s6}g1U~|9vYBt-FHNh9d&S6-W1%sl0_h|oaFesW#c_PXp`Bwq9=vy+#60*Z3 zk|k_F9`h<7l!z2+sOBGC{{M6sGmtF=0~edn`8V)}Km6ej&9I+U4JsMn`=oPNi`42^ zwn_CmNVEHfa%LgvyckmoyJAY-io-!hiY&z{Fcth!PGIXAEikf{n0^=Q0QyJ!b6Jix z0PuAStd@(f*zgJxV`70o^5vJ~q?=-3iX>SIK8E~q_%FpP&soK58_F+;n=)nBo)H%t zhRU5JY2zk$GWAg17uJuq7pb&J54Q`ATXj~u;==;INJSN%XQ0$JjTytt09)sGPiWoN zZMt+U^0pGk)^yDRyTx$@zpB3uMvajhrsfdypC8)}nhm^I7gDNR5|Hd&LNV~M*D&c; zHbeH;gO4t~JOAm0rn_+s-A666&%BGc@MyvR$SDINV0AbojXI^e=}UAUTkqG#wE)~sU@VK+z2ybH+%~M z^G$Yb6Mfz7TRMr&_S#r0(n4NZx1AQu7Wl4Qa8_9_t*aI@ZVo*EP3@gzrwJ|*Hqma! zL6P-Kzo!2nOcXhpgHlKauiGWdP^_vfL$R8&3|1i(_RQ}ry1tRvnVSYDHXUHPPDxTD z38c@#>#Kbo?skoH0hoBXICtB02*?!H5jh>cmxuL;^L@5vgtKY@wIC z&E&`yR;7)5-LC$1OC_^zmrc{_O3gMgMS}m7Kv+^jt|jId*cc^*Wme_U{xu`AQ6qBx z$mhR~fBsaB$Pj2@r5sdmO>_FyoWZA~IkCSZQZ7Af*n6zT=GkSE*?pZAsgB;);K`;1oQE%R*5}4cA?Q1tRE6q*>AdjlcQ} z;+D!lECJkd3Y#Zsh4s!5TZP*LRM){ArWxO&*=b@1z|JK&H-at%62Ih-s)9W*iv0A_ zbDgGP*`&+VSklxIX@aoD;1jI+BwabYTGdgkK#H!k*kq53O+m~cwPLsW=L>m!F0d+Z zO&=lok1^0O{aNFIj(W@oMIih{8!51BrzSjGCZU>1MhbMegmhu_pHSFe3rJ}rU;#5? zk8m(821P-hjc-VpbkHU+iZm$}<$?2!M)N{LchQC()J)5kNGp=n7m%T-ctZiA+Yko! z;^#@trG$5A8O`3;lF+GTmF6T-m{tEdB8YkxO&af~ADK>CCKyjTVhxgARICK>)2_BsF8K8 z-JpxKIp~qJ86^~!f|2qdwr2_5|8{cr)Q7X5-JIEf;NG9Ff{9Z%}qT?x@f{CZP;H4 zTRXxr7I*bkFP7h8zc3tztllIe7`mB%ys1tz>#~OVJz24{tP^86ZxyyNIt#azk!O7nl6;!U`e?JJu5f08vLba z{sWZFdoYCS1?fz`-3(P{Wy1Limiz^HvaZ!Zaj+BgpZ1DNamg;dC3{v|YNQIq^04Mn z2!q;Qvi*Orf25K2J~>W|)YOR__ zOzU@Lg|nrond~`i77R+E`nP@$cu>~HHcHb@k4Xp8bkucR?wYx8TY!av{T5tZ+fAh& z4oUEztVAU>(p*ZiS_7+Z$(Qx7`pzOKnB9{bBCzPpY8HIe<;V(JD^m%@tZOR>mvm+{ z^NA&2y@;eWL-%1Qsx>(bLv&0gkhQzHnN9naPAn)XfrzBY@GIn2ngN&6^em~6EXBf& z(2lk-1}mumlHRU!tcu-0V1SNplM!kW%itq$Zc{TQc)(Zpu2wU7*Nc#W;A#}m-jj=! zNE^5FR3UxW$Fi8dgSF?sjjo>#YkOVt3`AT_3gX10L_<}`(!Eptq%m8PBrjo61j1In zv~DqIsu`dwUs~7Ym?FMuGKwt?r5jegbjuc|3E#g^Q;sdveTg*Z{u#8kax3jnf(2-> z?Yh{WTO!q?Sdp4htWZo+D*74x^x-|cc=OSTuba<;M0k?%1M?>*5S}C$STT5#>evf4 zpchnxQ40s^UO{U;{WayhTeIt_U9V{7FJZ@e1CB+x#-u>3?$pw6lc3?gmqqrLWSoo= zE%_ht6I(|{Yd55T5o@tm9QZ3Bdc5C#VZ*Z`_a3h2U@`|;omK(Bv@;y zPOh~)wH7=IvLdN!`J>=D6b)*2?&A})<8RL%`|$r~?_HbQsIvXxud<(ZmU~r z-Oj^*%C}s|&0{+R$f2IvlD4IdTHTuNmK_CC728QLBoJV@6Ci;=5+LLb#<@^+Jo8?D z=Nw)QjxQNP%~$xZwIxSwNjBZG6I;a0#O`jjCGEX>uf49n-`!g$XZ|+Sc=~pJ^v9XY zm+yUb&ihiNdEKybX-7gpc0^5Nw_y{ILRj)udF<{nZm?7gk@|UG9+ya;X=F(I114S5 z$QqWwnuV`|>y`?mTUH!anlRd$0)(L($*lDb4qF2qr#+uwEerHL@ti3#+1bjm*Wvxc z`!4qU9Z5NH>`uhUw&t(gOCP^j@>`|g-dN!_dDgo}Ufo3E8J8Pn5t737`+6PRqL~;^ z=BS{aLj%Xc;K}~|+Zfk4?1Yw06E=oiL@7kP^g>qTLjrS6HgF%n}$PYLtWYj-TZ15DlcPZ1~6mRgRZier}oR3(3)im%)) z^8q%yvrN_ix0ez>X@a-+H^Yy>lS@U}x~rmWttNsgaGkDEXsuOe2MkBB8&9LfU?DdI zhhk40>~iYekC($iNmf)zMrKb(T&=mb7r%@X4|0k~p(>gsk2P#7X>o5wevwiP4jl@| z`5(rgXw?%Y<76u{;E}ZA-n;&mZCrO?QMg^40YwTZb;rE7aD+A(peB}|M-lNr(7K|i z;Nv+Mt5^`-R}2vlW}1d6LJIr$*Tk%Z{jt$X&6X{Wa3pS4n$8F~Dh~s0OAkT3VJQ-E zPx#xakygIAnQX6~;0Z_JVb~W`!m=;CT;bu?9Jg3D%RtC}GcmaMv@*L@adRob)#&+? z)-+G?v_ZBRR*ENBwfHx+KEG+P_&2r2a-<_gV|n&RlE4WmF?kbMDmBL*89V!eI^(e20sg~!Fw6zDsl-w3QxB;-%fhK#8$r+Y+E4T*uQ^*lTDxSC$ zR7g?-b$8*Qd&_l}p=kLe3{;FyQ?8NcU>2!J{)X1Sv$AZR2p*NZYjEV?N=rAU^DMNl z7gE69SDBdyC7R-TA_v=wqs~AMI&v>Tm;Ouu>+yY|4FHB!24MK|{+UDfKYr7*bXO06 zA%HwLY8AA`s;0S@i#-KM3K^U*>=bAghkuZOQ#W!}S`S#cVyw@=7E&;9I&-Fxg~=ja zps976?CI;{@fwb*_sn*E@{dSzXeA|mod*QmrdG0k|FdO-w$t0AQVdFxeG=KlH6#^o zdNG$sGVcXVwdgy;;o8qcIK13L;jdNj@E8`y$9a|`j+zd-ZIx=`>*!<-AfTV$SDDIL zHE8g;?=Q@JHa>G=H&7 z^2-a?UwWx*$iNGC$NqNj^AB2f)Bc!o05?Oowy3-Fg_V$Jq0OB6X!fo1(?j3Pe)I}#X-+TLA9A?Ce%HEF zMZnO?T>YcNblL?2Gc+UF;8ZIM!>EI0U^S>#K#U`cv)zE1jN3e(v&)Ip z5EvUiNyc<)B;t^o!wi<)+QBOJT0{bs&gx_6mzX>uBvnUbaJ3qBRm_Wot8}63$bkp} zJ!lq-Q*gAX#t+g4Yz=EOxy`K}dSo@k6FL6uv1%Vys32Y17kM|S71-h2fXp^9Wu zN)>vJr>cWb3fBwQ3b#dO9#ib&g|R=2@`GE2+l4E|r2H`4b5o?;t2AFnbzv$CFDy^{;cvgq|Wd_6xz+es1>2Z21tHYC_*sc(&TwV z_g?fRM_x?i%sitjiVdnEunQ_}@yFURB8JIVt-_}+|45CjgsT8egCYPMEQbZ4sA4R= z#YTQCJ^4q80kv@xTpvyIo;s`og{lL*HyHK#!8ETsviAU4^kQhi^L|D zcpwrD*V(&R+|SWWE&)?Zb{WZBA}4SN6w=4FIv77IT6>%QwWCV@Wui(!%zF9My1Rk) z3W~@FRc|qbrMF9^v4m$}%mkca_*1Mpv<(tNST`6g!x_RNx$@l!Bc97VC@_BbtNYQ& zyO@mbr!N|$OK`x5puk=ee1}|yz*nh_ggw^N3&%j@mq4PwJlwcuUXVBY=dyV(n0 z+#NeKbLd#x-mU9h=gyuxJ@d)?Ww8{`qn#QPOlT6E|#oGm$NubmY>kh!L)5gxdSh7HpfQ++5U_o^Z6QgUx_* zVv%M{DnBU2?5K5%XwwnzGWeX4Lo!;DVp{h9X*lbmgIRH=!-imse)EM?%J%Je)~P3v zr_8%qi67Ux%aKAZFHgn&`V329Uu89w>wacp$hmIvaityBZ;$pzp5 zrF7k|qHdO;=^~oOgWluf71+$d+5x@UlI6@w8~q%9coxgTo0R3^C{uJ>r5F?>FM%q# zJa*8&ZzD+6XNq^a;G9ACw1+Tu(UE$eXM-yi+Z^YqtZ`SHuO6F>5@a}P={Sm~g<4xk~N zHe!Znh&6HACT*0orzXbZYb^;E?)b8uyZ^j#4mRodKW|(pqMwu4#3>Gk{~Z6xCiXvX zyk)o48g?DITL-B6LJ+hpMb%|ZWGE`jl255jB5df!73;PFdLqaE?_ z3vk0!+bN2`8*xLAbA2t#T0kffDWtB1$Gw~Nv0^B5tyQIWS!~se3^Tcqgw76lPdOk~ zCe^Sy-jTX?R2~Oh;ueIWNZ zCF?d2V8XLpwrd)Fo)>G7>MhR?VZ=WYteXO$ZBK7=bFrLgA;&}D8wiR@Yzq{XMNy_l zYgZS^KELFb6nrj1fx$&*;CtJK19?6a4u`_3KS-KVww<-2EiD5B15)0|*|`{#;2X6J zP&=ObbN^HRTs#>3|C4`B7|H*)k5TODcuL$S!REdHzyCW{`lJ8;Z~U`{t3_(HQrPDG z3{T`?Nv62x=jPd+Af?3sZQo~@&}12Erk{ill3RQ&KKL#2hf%VUXXqh4 zFRq}6R$NxUXpODHqZ3h-@=j1Zai@3uK`Fe^7YT-vq46B{itAuutf`e0@JT+ka(8xJ zT852U7rvUm@YTJMbN4>@tnuumyJHt-UVj6delNW>bL8^O+s9`@tl`LadJiAeKt>JKj;+G=Wn~?aeBljbR8XFF3K8N- z+~hj%GwWgvT{Q)&`fJ2Ry||gXaE;~B)agN!1CMLxCAzj%*J}$UE zQ0-6+ov5bPoP`*aoO;jS!HBT7H?A224j_Mjd7b!Ou9Gk}uY{^#{FhGL?V;&XMO;}v zXyeE7j6|E(ZQ3sE3oY9ZZkQ%V>c`XoxH1Q8r0m*TD^qsuAi6%o)Wr-f6zAf;FH4|% z(=p+P;$x*wt8sZ8V-nx^#E||FdO_fLB9|pmi&zpt7u(!158? z7f+BaxuQL@$6P>%s}$It9hCyx15YmEZ8*dgi6E$gETg7dNhd-WTS4S%xLps(cO*fr zRD)URow~{YnqxL4t!6*d46>8LI_j^ZkUC@foLTPL$+%^5abbi!^9NT3VcjaYG5JBz zZw3KdTemTwhh?$}MlivpBRmwRs|QlOyqM~?%(@o7yJP7rT#9Z-249;d40CsBI4wQg ztOIxTX_i@Jrm5v+qq<%GT zBVSLC9GV_JgLSdP7pF%?*V20M3|9@l?%b8z_YPetj>@x?x|wAgvKn%dN0*x%6_*eh zVN4fP=hD2?8T@+n+WnKK?q53h>u0YvUO)n?Bt9Cu|Jskgz3W{Dy#vOJ*-$`r7}3p2 zZQeO$8Q!_D$PZNI&K^To(=-w$&E&wqG!m1?ZS@>Z%ds}A<@9vV+^Jx>#~-{*6{Tt&*e zc*ONC4*qGlirVXOUK<8dm#$%^Cydxua(DyVYPHMH?tgUY{^3tN%XRh2&%zjjR^_^5 z0;`Gm{09O0*=o`h&5)gRKr_7a^(>3=?73JcEHJYIb;G{2CK4c%786r_D4d0szZSlV zBv_Be^jn~V&YDINw5mm7ra=>e0wtCw+u&6gm*5U5O_cY5_ZHLO35T@GISAQh5+V}y z$&#Y35P*Fl)X-f@M6iVDhc6by>;yR?ISR39maQ!-mUkhxYmD}bp}?2-ET7$sY5agA z#85We-6qW=81i00(!4%>Df>vC(IVZf>}wXse-LenNl4O@Pcw@&@i-$XtZdACUwg~y zP0yk72!i%Rs@T61%%zxP%tjzA0gTmPda~8DQ#q4t!&Yc4;qWYpU9d-(96mSBVKTUr zd#=3RTrW4pCn=FiUtq)VT(RN##m(P7IJ}02X9Nknef3)=C9ZkYfmW8WM4pI`6*+66 zO51xn{-ZTC2){^c=!F-aY;ZimHp(@YCrO;CIzm@q3M7oA<=p+`vTH88bj?9%K}oHG zX7BR=&3@RBqS8E&@j za2Ov62W@x% zdD>Xl&Hle9nSb1(W*QNS;AQPa&yopY?Y}B1{s=|@%MuC%%h3pN>`q1jpA@Mgee)ut zEnF}!QjKhJN8-YofOLqDs5$i?*r0_Go8&TU)d>5bSWs z#w)o*1j*IIfw~hP9^j#OQ!BL#B$qPGNO*A3StUgD;#*4~(HOV3Q^LkWjPDq zcJew1NmiFiGa`_ss^B?1HF9*4?LFkV-o4$=w|4fnHiBKVY}aV1m#akPVYQm9je3Wu z-OaeR+aVYSbc*5Ia6kJq{CO{oBW-P7xOMl|!FDZ&xUppSRzxFdQv3l`iO5fq zCb;;BT3c631$0yTVisKT=db|GyUF!nBAhB9Tw4W%mwRkvpA@R1;CuY6V8JMhOS+F+ z@f0&x$qM#fQHdf!O4zE5fm{6Cep&Jd1=K%gbOzfE6AKz}4O)5FM-O0U4~7kP8HtH= zu*@^qKF-X{#CiCmJtpicNt0$_XJWQ!Didd%HQ4fMjk7$}>LK3iW+~fpxRzckh(Lg# z)csXG6-#fc&lvDQq?;wMGr4kW-AVOiPEa!-BvT%QV&$@^Fky=XrMlHHVCN;hVAxy2 zV5+^%wslu+v)~IH}aRkMsGq_cf@m2?E*?79$6M|Z~_BC9INWC%5}*NmBY zlFZdId$x5~N@*K4^Hg{CW`-3hhy=e8Dd<>aL5Ix~bZzq}VM*~-ZeFfSPP#gp`$T%T zf%Mugc*G0>8Y9aKwp2hXdUve~UI$U}OkeE1jw^sat8puOlq*Y{{ zSwe$V^O0{iSqV{K?R8iyH#M#^LN$$3J?`Bo@XO%Vgrp!o@hdV7{6WM7bG@UqQ9YNW zi~a3lzdu&wW~f<97!3ei@*G$-B*B}z!0F6UowO#6T4ax#{<62Ta!2_~jvW*}_rit( z)|QHK3~)Lf&9Gh*dLWyCPYMTJ%cjN(*$IwoGmSSUS9LG5tkgeVCtF$33iY-?R5j_W zx{o*txg+i>xlsU!?X!{%WxFm~$x7Mo9y^y{$p7`9E%l`aw&YBNW62mM6WIaB`R2_; zAp?x1#S2QaIQ)a;3-*|7fRc9n`{XL;3F^?8fuN*Tae+RD0Seen`PwzKfgn^r7~3nU zL+R8LhG_{}*#@ja*+~ZkIyKn|ijpTu)?^k({lY=j#Azl=c~S?h;FE*i*G$KbZqlCO zahf!U>~n&0)#J9R;T7u9MNr%usD&OKX54%dWLHi%4bp_Q4sB*mw+47T4Kt!~0nVa( zciUB>hkIrqn%lvMf0x? zEiLUZGNtyqn%lMxZrdtO z{gN!n!A8r>V5tDM6(eG1+(izv<~q_rE`1q_SJ1ep8fe zK=K7D{YdJz5Mk=vozbbUK}vUyOkJJ&dg{BWi+7GhJGE@8W$Nsm!*Jy~{66~g@jF8V zFTia-PMw9D4o{sQbl!32wHkMT6A^r}!FaQAMqmu*#bBnsn7TIgEw*u%PGPpNEArf( z*QUHf5k9||F~nc)iuAi{iUe^7U?D4&+H_+fuXG{*cH-6FJ{Vimg$z5bvciB+QnaIB zb4qENiIs~bc(I`f?HxjNHbATanqBv9%Y$@nMJ@K?>(OU8{1P<}n9*(NJdnj8hxd}~ z4!V^eJm_Q)lYI`iUYxjFp{h?(!j&1StnFTodo3`^Ud)*j7ZNxCSB<$6&2mb{&|h?KoTGZkYZ8JI5Q>8ogy%!s;XWsx0X%eRN+$Lhr+GGby66c94h=Yc?d3CDU27cH`L}-I0;|>Na51t z2ZbAj+obSW;risTV5GkSFMLdz3O^Ob@TJM&I=C!ywy`Mb3CVi=1aq$n5{@uZJvQHwrXuO!oSmO+^g2C)y{qZ^YHn2}% zhmoCvX8_Uc61h0mW1%tAXXH#WU~+69CIcxRPZ4$?&X7<8M!`!ROnsbbNS2u4wzTe* zZ$-f%LnCG@;-B>x8c&dCq67uh1ygCmuF_S!+hYvQq;`rt>o7|LwJEj&OEYYtU{%TI zTgi^ugOEnrT3a#Rg|d(4F#?K@ScgfIb##Eqkrxv=GtV4WrK4@gZP$>3tj*yn7vFT7 zqQHFucD=F5!{Ya)$)j+5qQQhwI9(W@9D!|a9Jlew!;{0X@!iHVxO#On5xe9oFu<#@ z$=~w65q2gGNA+g2yhO@&nT)N1$Y1_YWyaQaYp--n+&l=Mo?#?Uvwy#=CH{VS{j0Hm zd`e@QGvivi5bacn za*dRI%+8Z-)QlNsWjdkDFw@3n(v@Ro+z^{ok1?>G+b30j#0`vWM=qWw9Y!LFTP;kJ zN#nIaJ2q|ES(#4Qq8Zp)QBER6f|44MB2vKF;!#$Be%A`nuSjxeC83edzvg&9)kIMf zjPhxkML>9l1-kHfYOJ4WYwl;e_1Tu}O)uvOR+wLi$Yp;`9J`cKPzw3##WaHMW3&Ijz^EGAJC9+G}08DBVddR-d&P@pw+NTYP?>U$|vdO4%8$!Q>1Nd9jF@X7oD3SUXaT zCz!+9aG2!*_i_cVJ5V>EHY~PnTu*QWbjUOUoyW5MJa|;{=ZWaaYbn8~!WO#_8TK9; z!Jashf?3L0Hg$Ai)e$05eUiVbX<;L2vodXa>D+H0zH|S~A<`sA!ogpU9QJPYuU`5F zWUNa5Naccborg{FWUAR@mZ6C{q!Iv2)BgR4I%eorbm8|_&4^L0n`Jeal)*)3q~9k6 zC12yg!%bbh^V*#wcZLP_><^e)MHJm@Q(xnSJ0o@FT2~RSJK#sCoj{;62(qz&S8oAF zd~3Xb{Yb_vt95slweu4E5o|V8&t-*Am~__AZx*eyFcXO=8N}Ap47JHq1pXToq3S`> z6ne7$UZ+1wWP2YjT!EjU<;5ggR9~1|u9G&V*AWZd#%LmDWMYnIB(emC>nY+!Wn4b*DM{8TOzQ(;MO#(HhBSalRS;=lA*21L6#i_WhAS^11{V**5Uq z^%yggY?XP4JX>~ttRDBvu9=TF^?*tFv|&x0UldR3nmC>0Iwf%v9#i-QPx}9S^U_OO zIJ_s8cqRn+66|4=4C?R{!|GrB-&16Z$r6b?*}_bH;#_)jd zO!RL)5q$#x+rK{ua)kqIX@Pyi=CC!|5}d#E(sP(j#>*vgJ1{UaLYb3z@mB(V1>tud ze*5sUut%M51Q#vXdD0ATabAT5Cf*`TO}y|FkuAG94u?7T9OUt`U`u%7Aw6!ZM#9!! znwd?G){5elXwbK4BD@=%xug+^L>$`Rk+M?U=B(f;@=L)g-aOaEgj)=gW{SpScjq&M zW)VL!L9w5gTG$LVadzT5vv41)Zip(q*n*%cvJBo)5cV}$9;`Tr_d=jpLdnFq!2B-u zoKAdUEdRF88(JdI9V z?J%@7wX1+Udtky7wn^jZMnr+^p&F%`X4bsjO8qrTT1l_yYZeT-JB|KYke&P5DxKSX z%(5}cmK5&Nf>M)xlA?qp*!jawY_&2Vq3n)R043#@)T%xg*T}g=04Y%(T7vKr#Ky^D zObz>cQ$`8PzpX7QBwh!)shdmQyOW%cHFt=`m-q}I<_={5A7Gd1uSO7#MX z15(1^qNo7zFyv6E#pjFo)QB7kt3KK9^TGcEr2skHiN3}%_cf_cZbttEeVc`_`X40t zxsM|-Vuo}|1Waii-p4ykWsNwvqSiJg;rh~#Oh9By!j&1Z$F$gS$KNlTY5Moi=)~D| zRPY!;wt;|JH%2*V+E$&$Q>LLK969-OBUEH{a4TMk0#v>h-Dj9u0x_&XM(qr1c)aUd zu4kO7MzwaTQC&_jG8iktO0V8^LHGXsg|`czMSnT^W-&TMN*6?H(4oTh!mYxk2klNb z#QE6diNWPwE?kGV!57H?d?Vg%tZ+^IyHvO_d3M5*cC?(B35!rLL&QiknL0Ohb^(%M>g=5(h|w@QbuM~m=*}o25$(0$ zyIB?)K+do-ag&aChlD=3(pa#3Qx{;2h_-+z5@hPSs07n82;bX;DXWo2$ zdid>`Gq28md~JH@?DWv<(?hqXhd!JhI)=^;yvy{^`_n^*=Ps^ZxzEE|smoSv&WxU) zd*|Hj>Cu^^M?8zPhaRNxNj&f9tqQAUv5I4tt2ZPC{gwTzeQrZ+t?4M)3myiscz-TR zUV5o3Z`paB_VXk~uZRp{cR}7-l!KSUJ~Bvoicu}c9M}6fINw~iC(e*Ti(33n}aZSTz!8Nip16CO{EDOmSE43fdMR59 z3l7enJ~{u^tFt%1ogRAY<(C`5{9bm!Pmi3Q9zG+Y>>pv2eRvF%+1w|GW?y@E?#pwt zuO4c7pxo}AIx_e1cQc1Rod4paXHm|a`Dpg7^Qc@udgbo;xI?J^a$BD`b}7<;6saG; zY!__OXpR;6)H>*ZPLLmjkuHmBV0si$1*;(gR}oP6AuY;fn~{{DFBlE_e7S0Gcl6|E!F+h>G+{jz?xYw~(@tIEMjeoG5H#YW%2v{xoidP^EbYiBHu?WBEfot{#jQM6 zlO@}GLNQcZrlDnJDuYEBs%leTs*mC^4}iDo?114cAAr;7Rxnc%hAE0UHPU6W3^&Mb zBL*i3aY0EJtjHaio(YWpWgJAsJ`^QT zHRug-@+7lx0mpZ;Omj?D@Zu1NSa)YNZ@5D z$`M70k~Y{qNH5nCxDYMUlRJoSMs_mfk7z)CMWk=i9;l5T2G#&+vYD=Ps}68YgN7AZ zarl2;VHH&*Pgbi+w!L8L-2Y(o*Vn!**|1=g{`%>;B}yB2RQKw^Gn$KOJdP8x1NN5n zQRevD2Z!%}`R%XAMmA5$LE+LX#&sB+J7u9IqS|IEDhQr@I@ zkpOx{wgfwydh!XK<;gC?HcShR12EamG;FGv!tKeLypBG!ttL&u)wsiMFPnZr*QQ?} zAO)((2whhaK4$J&Mb-s8WNcT?p)_qyY?Jm=bh@#TxvAU8(_Dr%@71|?17jKQtd7cj zka9!{s4MZ$keQ0Kn?^Rf0vPY0Xdo4lmSD3u{DXyBwn?PNW=vaU+U7dE02RFe*SMOS zo}oHSY(J|)L+v~uAl5a4ak8a&)!JLDjG1 z{v%REsT;Do$XAP2Pu8$39z&L6swXH|jhUM4wN?u_y`hl6Wbpe3ohL86fc>APFggD5 zLJgHvG{f54wsZ{^qmg6_&NNsS7VK#K?9cG0z3}bR*5-v^&I!U@8_Z*V?-~Twm^i!I)I2vLNkotiL%Z zn8NyaCI-IE8kR%_F<&J|Hp=z9Erguyf~c(<3LQN8b76!>>JyLcu2*W>G;%C8vv6n}B8+{+S>m zjz!c;VYueY17R#QN0P}83b!Xm8!vrV7=`aW1cUfV5CRp(3pczQeny(d=?>?H9<>3)UOV;D zOZ)ftMs>r^pbc+web90I!V53IJbUVu`R^{xj9!}kV452IKZi)aR<#ppzL%7Tzg%CfTUEhV(^3qdRMS@sGA?fr(B!X#d281arUzJN6j2 zJ4`5w2+xtqaDwZ}M_c+IAF@Q|?;qRb@2_)RH!PduBEf=>^}b9ha5XifpXpY5;xvhI ztcS%TD{E-%@2}&qJPWE*TLdBIwfGcd6OsZ|k!eq0G#{Lrd*{a8TPJ7!Hq;14Sw++h z!J5CB9vPh;IX^w}seqI?Iz9Zw^zgTyWidT6G(GZ#0H!!KJ?ub>X!h3zM)WU(ix@&I zsGwR|DYubPUs*=|`*;3z<<|XU-bqwfFQX2URiseWX0&?sH(1*xI32c_R8NzqC9;#| ztT?tIAWJma)7Qtry=}{J4Y8xet%O^I5;+ozG{asS$9GhG0n%r{CM>JOvXS;-|9&f- zx8f!i<-G6$Qa9}1pD+e=8~%}Y!F>!a4ZH##w`iR*OdTu&qmKwEU{QX`4YL8tZz{Qc z+{~kQHx6tYux4#1U%VW2G*lH&-0iW7Dg__QTIY5Ht1`$oZZfhxYYp&tY8@0hhcJf{ z^luP7uB!U}?SrxV?|(q<|MG_LV2f0!n$siDQ>bE-Xc=z;La8m*UV6%Gbs=#Qpns6tiVK z+Q!VZl|tRJjoK+XK=ZJ@Wkg1;+}g9vu;bJoEdFY^T5_go7TH40q<5&hj_sY#b~;GS z{AJJF@3L1Q1Of)B)!qKK*czI-F+TUrv6-VsX1@FJ?%4P*hfcoq(r(%xGY*hJf^&dq zhVyT{F?;eHToBdBaAE)c;-$H(H)clP2LTU)!Gk;eQ{<(WUJ{3h-eAOGl4;grFTeaU z+!8Yq`M5?ctTs+z69yI{Cru-lmEp?l*x2mo3Alo<$O3WkKfx-44+8V&mqVY-eRO#C zy-W8_U%&Us=j+@U>Ma4lI(FaEm1uF^ygxUT|23B_+Dr?Nj-$`S@3zqd9-a`G(v?n9 zR@+Du{r;Yd4(5q@@b5I(|Z7*bk&)!;j;*2q<07>N;=33kWf78w7kH+u0T54@ruGw3Qf= zY%FK^IU`4UQ(RAb59Tt{%(rUnz)JGeAnbq?s^WTfz219r$t1IEY$OuAJlEmrz5$Q% zRrq&(a+pjG37Vd3lOvOFOumX_Jp>noh&>%Ij6FQ~{#7$%3tvpW0$&%uG_K8Y$8xWD zCE5$CaDb+Hi(^)-;vv?Nd6sbbZJavY5gRU^xLnk*j{J2u^&x;AW?PJ`s2`~pA7Ps@ zfZBlwgoM(`=06HX0=D6UW+Ho<74M6AIdlZi*jr&_eDPJSYS)3u$7~mG)%z#TU`4wl z;8eZ_%UVB3bxvC*-org-;*4$U@W4{DCEVg)f=N^-C&3Fo%ZMG!WTW&#=}3YrSk zl%30igJ6)XVGvnpJd$D1G%K(PRvh z5g0s;85*|{Zd+UdnObT5_y)kytFT$}$hk)e5-m`ZdtYFCvNXnFb0lyFLOhg(qbQ+r zy>s!zrHH~(sH)<*&qHYF&h0y+f_@OW_g>D|jDXK6_{M&f!K#klz`_(foY;mqW*4tQUVi zK6SP%<{B?X|M%k56*#zvlLMnh>O!2Db5oZeyz)xR(+1VCoT}A~lYRTP6L{)VM$)9& zlp?*D^KZRAWfQvlP$qP{2g%1{2j(LZqR)pX|ac;GKxR@?b*4@R~s@~_{O=p zH;>L-Ix+usjl$7Qa@jIn82>+BPv`&fdO0*N%NR1je;Ay1#0`V~GBL0?b}5gL6qGA# zQ#S0HWu@57;Ik0qep^mYniCf&@>D3kW;qfF)&Z)xI@XSgL?kQ0pxr~$1BQ_xt^G{T zF)WULt)1?WTgNIUh*p}$#1&Q1~2pa~1BQ+o~5&~5K! zE!exI=Sgd>EIG9t=22Y-c~n1GH+4(S%4fi?S^um`6$U0eS;-s~TqteSjO7w=Y>RSW zQLfnqJ6LOmp~}$7R(u=pMh;9SE#_^VOUuEsDZ4g6)u5zQ6(2o@eRVM{Z_-RQm9t37 z&Y1t}a5n<9n7Rqm7rD}1YoCeap-;MQP*w~8g$ zYGpN+C)6TcTVyi85l*wg7M8H;c6YXJ*-0!$>j^3Lrr1sz>!!9N2vJ8Rc?3}wLsg9F zkMlld(SGFoGINjxcyXAt>lVmFmgB~DOk^lxwk-Yu;Lu_UNiX* z`5~>TM?-*KF(mJztbr{n`;{^%Q}M)|G!@vs!W*=xtJ~HF({ROmnc3M%;x1;&UNKf9f zSq5C4bR%TSDw%D}7|bDObu6@A~$5u^_DRkb!emZoHunF$z9 z(&QF1k~t0h&b5q11p+nt!5(M9q*#6}15ts96s*GO>~kv*Wbw%wyb+Qd8W=K;ip0`| z=q?~6`~EM{on74~4_nvquT<-t8v zw+*Y$Vae5%qmLjFml~)Hth**d#CmQ~{K3~fSYUz#_gP*Bso1no~J2|AYo&xX*VK+ z^}{g5yfO=-T!NCnYQtYLUcRZ9rZTMPa5`l3mmG%03E~zQCQVSxMF*FnN`7`zg-AB- zVh4B}Rr=Z&8ttUy8RMWNNo3c=`O>$ltTuXXZxyrPbGbORi_!CidZTbik^>=0@dXys zBzv`%K}K>d2#?ilQ7Z-e!m=c*suY1w8+063_Nh59;Z~M=53=vEQUf&Q5OYAT4N3^tmlZX}AS*@fx=}@V45-g(|d>4sL;m1pXf_=Gn0h*3gV3R*Wt|S-hu8KJD-qC6eE@ z7tz**?_a~5Xx&`}E`n8l#$R)b7>aS%g$WJ?C0|)efk4@u5XbHoi{B6Wt1|rN36=P( z;rXwYz$0GYi)Ci?{rMZO&7J#hX7tNC7K;yr5SGGVkU7?hBb1=J%ci3YkdbXAQ1_xb zx8d9#zt4wxb4KD?F6Q0wp?!VKFFVTj>u`AjvgE6(HK870C8y5bIXw0C)P+04cV4?Q zj7d2-yAJ2COQ*qX!MmdjxS`j+qCU=E#WDZZ zI}8!gSbL-Y-dEq=y?u24%b)I!e>(Ho_}nL7%v^eF=ErOEH(vL=a^R7dvGdH2QBa2y z;Moub8HFXas;75(c+Texvnu-Dh3)w~PPNU%GH_qpEH-kJX&lCovq(Q@1M7NqsXqC4 zyk`^4mNsVU)PygVq60M+*zV^l7nozhO2P|*@ItBtvQxqivqTL!D#cu;v}CM@R_o`4 z5FDmPzkyJsunp~xX~=>BJ1$sTS;J!M-_oo>8L>ML24%)Ou)^~>E!Rb}RCTxqAqQw` z$fa8?V`w&9MgEdYd*BRsXYxG(;)XDWHztn@xU(NgVSI9AQuuns0DIvQL1ux%&BCR1 zFQ~$&h0BGj;#=Pquyu&ec!D4pqkv*W(8duQ5<4Rc7w7D4 zsc>5I{4i_>*Ni+zk;^Nd&O0oWpaWtf=rS@aBWhGwt{V%^WXh>8y3-lVcu!j1n zRb*$vY9xy^!6*5DIrKO96?hVr0e-p9f^aR;y;<<+FhSUw z0O^URIG#t9UIMaDL(r3dfM zP`xM1`}*og3B{NRUz2QKfiIyT?|h(&lF)U9e+h)$jy_HMX;0kbS-S<7YMfau#eA=| zE2r6f(0KxuNR*KOxm-*mPn<`Cg+)6s9FRcspIa8y<8TDa*<`xKcv7D@&q&j=&oW~1 ziF2^oz(dgde-{1y?gxLzbka9*z8@}{M2@I|hD$jko$th11Gd+lxj3h~_o~3BGt|lR zpe~`Wmr>XX;=qKJx&cKNdzsyEe~GnOIGVPBIb`_cQXWxY zTIdy_u%iT5oVb)?1ao8cO0hM8Jl{kHF;eqc-akmq1H7Lw!86@tJ&Al9K6o+4V{sVT zVClDbMP$72#RN}s4BZ(qWY-|Y7-b(Q)4_zn#jO(Q;0eNQf@lH6GGOAt97+BeoSLj* z+9Z}sTAK+oO~bT`nKTEj34$#;i$dCp-vNPK8w`xK$?uCO{>YOefJM6Z$v)4|j$`nS zhsObY;xdMyM3sOf%dS&t8$^W_Vs&Qb?G)&ZR9=U< z@UDwjsT)GU0`JJmQq#=rGt3NHq@qenApeUT7HC8r=0DC74Y6x3-Y}y*0}OTq(rv`k z7D-Vv(E?U?B4!-$9{j0%Cx^&XhYnO-FC>3Z3j1Bh7?zCxY=PMq8!4tEidvHN#8ZYL zsuMfo2F4rW>svLhaeyeWMDN}*-7gAany9|YrZ6wY56m$o0|t!9PJx**015?76cA;j zWf&htgte>|U2DaFUkU|Q!V0&OC7GURgrb8l)$eSw+lBWPfQ+JLcewItRyAjx`l-7e$fFZPoku5(KV#{II9sQfY*)!5wzRIyJeENWL_PSu1 zy(Rc%;oyq6{G;fM$Fju2PlekA>k_d(5exMSwp6=Qlz|s~<9|M{?LfQMrS8={_&=fK89wm>s=1`|%Q8DeHLw z&ELLpZ}_XZcRusHc9ejxEh@{=K&#IO9;Ywb-k}Dga!~1rZV9w+iFUSy9C3)UeIdKF zFZ@wOL2bAa>704*-pIM>k*}vmu1*hM#OTrV@Hf*VN2iCc!PWVnZq2-Qs=)*{bL5-Z zljmly+?>1i&dh~3J&SGbol7%6j+G^&mi;8vrK98*^v6|ITaRI&HNn9Pr$nO}u5;7C zhP73KO!b$=;&(5Wh!ly`9ru9+vx{qKgLL!0KIDgo-I3`=KaDq@S}T9`xS_{IN1^Z{ z;xv=>ETG<0Dh0EYp_UWVTMjsMTEMkZ^9Ln=)zWc|KKK9wp#6mxqNG!|K~Oq#6s)r7 zRu(f%rV}|5R9|@EzeuNL*0A6`y+>c$_7eD}f?Rb$OtMo@vIkN5C4Uu!p%<$H?S~Pt`TjQd?$>kI8-k>G z@sfF_bNAjJ($&4^=}w2m?N$`qs7`H~p=rY*jEFPU?%MngM5J&vq_o?bIXM!J$V7k` zYv50iCPA6Ci^CF{I7MDGa%Ks#OT(TxIJO?#Ow5V%xGPws=}({+pJRQX2zMFqrxO=h zKeyv4vIR?F+4?V$EY=N{g|wC+!SJ`{n2Fr!aKfp^T3Rb-z_a|HpEzGZcIzFW z%`d<+l*%y0V`xDdiBgV!?RXzi8X-xos+Q8-QjPpcW3Z2|9-Y}Eu^YN_Cg=r9k~~Ru zyB-GA-Jl4aO3;z24Y;GFRRdW19rmq+j;GhqSxX7%2ZQ7#6-O+_G8U-5G$WM2ri-1j zLa{Z#c2%RDzn9TWv7Vr4$oDX_pLxfE+Lfj>;T23(g7WSxSI|Z*}>Yb84jan&e@x(%kB7R}RNw8<)X2J$1Sz{*Y>Z6H1 zj{b%P6WC{amg5c%54>&VV!1x}I*vuyyad?sa#+U!upcGWS1Hd8s-=}tEq!?Y{--dY zQ=WypYE(-o*{~FhY(%PbV-4+#KveNdfhvk`*KN|U%fUM;jj(f7z#(5aJ$r6==H$7% zV~6Ii|4@gPOH+$GViR?=*andCFH%*}yrgp5)WeLNR+*VnKtn1B%#7gHp$Pfc*tLI+ zy-Mz1I``|bbL9R<@7%v~0RtBIM@Gr7-;Tk>TA8aI-Bp>ZZkEl(G#*FOWrtDh!dr{t zm-{CV|2BN)*GrfGb@gUhfX=H?P)PSb0-g3I`Pa2WpsP#6Zovuv^~hmz|I|r;=>8Ws zYb3S=XUU?(*1{|=U@BwhL?k5n70C%HEBpNA?W}m>PUilAr1&d0G*3Z};UA|*ZcdMU z3P1mSb9CYKtp^vUhc8c$j7*P={`=;ujkhxI-gs^H{V%77j!h50Iz94saU^13SL;cH zZHWeysN5M2Mdj!g+20urs1Y^V+13`0Dy>0Bb~&IrQu?aPm=%aKEQPAPWpCoVB1AYk z^UC<_&3C3pZrpqC^LvNBxO?m5%-@EJiLo2g!{1JioEPb`o86ty>}kmS*!1wN>5*e7 z(BX^I!#@;9Gd(=+JyG`Mn=?1REQ>DaKnV`Ig$WJ_fP*Rr&-K=~xJYW|#`xSf$7YTm znfdO=yJO?O96I^ZOBiewF`u|ip5X~*kR;B{Ac=#|V3DVYW#?jKkZ>f#u|;3!Q)H(x z@bb&Q9QtJLqrx3{Z_J)Nw|%>3(RNV&*IbqhBW^*u zW#cmFGT(+ZaFQIZ#1C2lvgOrx=SFYOj2&-OaLd_~hvu#i&-{2}cI4`_+qZ9@yLtmb z?$5qG|LPSyIDO8us07BPr~{DXfTQNxb9HYZD&DGcCAufH2|ZwZDHXu3XY;@CnJ7QL zUKpPoF3ZPxHCp%t#xjH@e3NepJUt|}Df~1!0w?hLRk#gf9>OBMrI>G83U5vx znmhuxToHy)v)_9htp*oP#8H+Iaa)(5Vl*vKmD^cWKmPuQV{ZNU2e&r1AAkSHYrlSV zll<$_puh1sjtGoE0^=^+gk}DHsTTJpfgjgIHSTSA zyIbOtz#RPnKIjCN`Fa-kGa60cE=T)y?yQ;c-jS;!HgTw`s>OB1L{~?1pQxN}pxi_| zH8s*hwZ0-g`O0e=Ja;Im2r29V0&01?wPtZY0w!j%oRKDr9e8qWi~7Bj&N{z5p>f-L zNkhBQ*T>jCZl`upGtF$rGEp4pvaSWD_#|Hyi_c>i>9kXZ6*sbL81Gau-YKuB5U-nA zInB0+O=B8E_q^4@2w9{>mW7$sVB411clP$|Bo?EV;hocRt7+uoDWfY7!gYp2y%$NU zMUa5HdCYg&{r+0M*_Q29zS$P8YgA9H3(ss93{EGYWzspDQQKyw_hyN(ij;4Wu%tT6 zumTa;Q7;#9N&Kc_gi`fYVRgDiM?W}mI;}}$Vmu2b1|q&f(C9s~g4F=6L1M9YuCS&B6qi;XB_Jmcsy1Dw2xG`3J z<1T8!lfL-2OvSYYZmm)b4ZqqT{A;u_*j@AhX~QOJYcqM;^p13o4Lg6y@dJoCN4gA~ z+K!}pIqT#y>7+&j0WWksRa%x=78;5slRz|WW-iW137Z|Z4=}nwX3TuD$cyz**`6Pc zF}W;=mJ;;IQKj7%4F`PwXhiLV*$m5DqH>4a9^K+^Q=>uI-{IXas6eZfeGbU$;)zSq zMkHU@b;%M~RXk3UMrE+G%dqU`0cw(Life{tWK(&xb4W93H_Vc=fYmNGw>7~NhbNY( zY9w1JJPKBd4v?5(q$Sd-6GKN>d;*4FUI3~0Q?3cyDaA|%Z<@00Y_z2%jxrK>!IqTC z=~A4w#1okoll3`XYKiMs3s1DD!C)v*YqMV$9eBUT10|78iS$#QJYytl6xg_8fl`Vs zso036I5vInkisf9=qvup{eB}fw<4wguS@U#dg8M z$B_^jeqLJ=T~_3yCB^5Hlt6HB(OD&j?BZLe&P`pI`UWlNsdIN;Ayc1EeK&Ra=i_&d zBA@j5&&Tnze?CtB_dL8N!Wb@!o5U;ErY<(t(}V(mfn4_d9JR5OnLFsTp(3@7sK~O;OXoVo+8*i2!}fvWc0~1$J>(W7vn7Fmh->gI66-H&{I7?%0L-6BqA} zkI!CuYv#!1nYWM6oIG~#&Fk|&T)a1QZ2sn$XYrLLYOz8Nl#Qf-OZO>=$i)8O#xA*d zm+a>zPw-?;TRW)IporRs>+b46zLOL+?ImDa*0E08-bc2R4nu#!CfjH~LuxV7)Jrqq zoVS5+Y(poOp(pFWOia-nax~}~%S!_*h$jURaU$TWyTLlsVESu|_V6wJ-F$EU_FMnHd3^0uwe>1Sej4IOg8D zK6C4G!)eE!-P;0T?~|U*GMgV{X4n^YIu7L;;;_pSVwm}m6jthvrNhi`=elu{Y;G5bHa>URvvtymB8)=k$Ny# zKP%T`hlL5whY3zb9qMai^xh7i)J!Z@1wSjPH%DL?a9E(<4I;B6{y7%FfAoQ z&@Gygu+m1-vL?RNlhkC5w(`1Cl!3RU<+I!(Chvz;SPxiP4@?JqLz_9uaO&ut2nz7v zNZm9`NQ7=NMqnC@2D9y3b66Lvad@`c^KlIWE*RtYE(#b4B5TGqW58JxIhEwSQ$wX2ZhSs} z?v43x-fNjZ_rbm6hehdOiw~?W zcq>uM^M_=BI7p7%avX)CiYx{Em0tD30OIufWj8pSfrf_Rd^#sAnx-9;gPgpFidPAV zGlXatKD9D%sk_C$2MxNXpWy;r>)tT$Eh#Ck!IQ(&#Ct<@m&tW2$qphUlOs&;$Zsib z6I`LFUy*`UmE9?~4SMvTq3fW`8_bsIH8P}Kqk265ghjgHr5JeAJ^TRiN8z*|mUvH! z*VE4R%*6RDcdQ8q1dQw!0|sOQJ76_Ft=oVH&6>sm4to{W*$YHvEobqGt2zQ~f-{rg z@i?AJWDA9FNpd(}ZC?ta9De6ZT#! z2fLLcc2c?!E$Tg44nI!7_2)DW7E|uqIp=0sTQ&(;#-@fDVL2SXmyhGaV_KB1m9}X z6kLt3$gc_u3=U-@6ZU)SrOhy6MC0*v5r*iIM-jsaq!F%PP%+Jjr&(hC^T96LYvm6~ zp^zj8VMFi^T=O)|n}+UpbXrD&W$RoVyOV?rrdgGBzTvah?NWtMTfr2yQ)H{j`}(+@YJS?# znKfARYGrX$?LWnrYEu?8MBEn!)z9(^6yXk-wEE(|k&qPeNpeIIcnS4G(1fvOR|(Y( zhElA!;d-C(U*!L?_pZ%NTgk5 zs6Bvd3-txv#`+>i5ErVx-wYnEG{K%K1wr%_MiX@=l;2rYHLWxOn+=99$E!g+*`v`c z0!FUFp8`2V1YdMD>_LONc;H}GVdqX$lM+owApidU^=}K7 zr}MLlmca%D?_r2PS&R;~Ypb=v{UIR|tvV=&EeBu+fK7nf3rSXGwFv(TE}$wKDYi@& zcs)yH;Th|udo-PNF`Bd(Cl};&{sjIX`^QV(g=XgUr7KaocNkS_A{Df$9T! zI~nXiXokT%O=Bf%F&Y6!1&kR#a?On_IV!gfm%er0Dtzo%8BZk9xRWZG6g&?@81YCl zmFZNeBHdU9hS3rcE9%*ZxT54G@Egig$SJoeWLUjFy$k=p@Y5$ho&D+jPw)M70rf4M zIrq~CKfU$SyLGm3VHhW2An(9HyyN`-^d^k@ugVz@W>L8W2rH z@H@&2pUN&f3#%<)67+=kW};g4)J&;$4QFrHy}+It>kN*YvnX zw#v!WvcU(JQK3*skPy@e=h@ct<1cA7ze}waMZ^ zg#}-z+?IIC2{y)aH**tDm^>pKEqP4*wL9@4NuN|6i%NJPnR+2HOqHa`%MY}qP6yHI zApa)T*)G;2E1E^xjDdS!4SA0!rI=2QKB~fkvMr~utz<9kGQSbUTIP_0xz9^$OH$m&o{fk~c>DB6Mq*l0YxsVI&1?cL$uBo*mE?oS58vW zP4~-GH-@UngzS&+X9N0vX6(Fh-YW?XnGa zUQVTM!9GS)@NHP@!bqzz?e~&yycvSyYPxLLyH#xws(||tRZtSb)dSLOO7KX)_F~n+ z`WDXPEnRP^oOD_t=M=?&Z9uF#e!=n42Q@_dQHPjb4~0mW)ofZ)x*6+%d4fS0Su@pY zP&F~Y^Y9+a2TS)T2_$4^GYPq7GXbeYz0e)N0#_4;wzPndhi9?5$}FN7Il{C&yty4%Ijl=tqP;}}fvaWKb(MhB! z{WAAyYwqaOcslDpXaJY*3KIat%%0uGrnX?;oDNGJRqC+v!X956A8wz5fbw z3EV$&|5z)$I5&Nrz>l-{4{@mev-o=Y0%G|eg3sOHF#d<8&yP%BhO79F3;q411+Gtj z5APYBK2PBCRk(T#Z&ZGp7@59te{A{^5u*_w`M1(h)JK{FL0`cC3~9c9`2G>XRD$Dq zQcAu?56JYD`^U)im&la>KRiDr-Be$izL83~0?AFPp!j@51enUQ=R-~`XY95s#j59} zRo`I$ny@zyQG;i0U?U%(aD>E>232T^zcS@YYyeH$Ivp19HZ_4=N%G@xP&}u#!K@$B zA zv}Bee-g1Cql@7~ETI_iv#RyY9*`u4qP=|NPD2(duip6?|vXNCZO>*G`BtZ(4cjviA zsWfRai7A%@2TXBfn90h^FN1Mlrg$+~iYXb?W)-Mx>S`&?o_%Zf>Sr_GjeC|%OE9xyoD z(o6^6SS4HVbDo8=1169bYfo*#Xx;p9dd1OX4&=08b5pduckW-HLt^I`!Ttf@BkH zVWi-5kmepbBpW8|4c?227iZ1xZdGQ`6PIJq16hg)buO*0H+t18c(bMxwvU3HStPBs z9sg=*YddASpQ+w`gW4`#5sSxzqU)`+X|uL(K+`oVnJN#CbYr`_3UbO&mG?EwH`H)I zOqeC`x3^)&gQ@(aiWkx>J;3w@^727s@zr4~ssx<4KALcXm_k@ys307Zl1>x`=}1** zCv56=wBU48!=UjLh}Sf2%;G_xAcp*c7#2h+Y?E-wEf-z7Kt78P~ zQsj9qOrw~W>e5WhAYzNu@~cZliIM~o@D($;-ubPzS_(YfwnS98y-EUH!Y{@Bh`%iM3TXIVHC>tzFQ?^YO0YiX8RePOUa zJPW6IFIz;pv{6bf-FGo2Av18f=8U^-2jDYBWEH*;D2oFJI^c??8gXiZAO9zdp^>A| zduJ&D0|!=Gym&pffGr!_PW7z%q;|a;89X~>i}3I5Sz_S&uqDP#Rf-ThIDA#tQ+6}Z z)Ub`|OaOuOWff2rz0^n*!6hCma>V25Ygl{OjvQVXp2;5iEh>QBK!#9X6XFEuf_xgLp_OJ)0*3$2#@8+@wnEc+uJNX|Rz*w3I0O#b)CQ3W-XHEl z&7^ndVUq!}TB1h1fthh>`Naz4N{x)1!xxZEVw6U*CK)*qGjdKPT78Y@!_(IbhhLq! zIQii25mGE`+e}rH8d>uDZQHiZp1f7~s$8_aZ z;phj&G2x3(y<_+AdQeG_u&OK|+Ek@vXK;~R;W`$j8WQ}@Vg}Ro5_|zms9CIC#;N;F zbfox#L z_yxa%7}ZmQFx8bTQRKmzwn{w_^R8PU+vQ}+B)zFDO;d$>_qy5?g>0pYqV3&akX<^J z?Y$u0z?6#O2n#+3rPx!LN=-3EgK<%^7jCi_94&o~pvSmWf}#-imz&=W&4rpPm1c?7 z4ywduip@J~p~8YeAsDQ>01gpQv15v}RO?yN7U%O`hfoK9poTS6MZQEcQF5kci^Y2G z!;uhl%oi@--JeFyrE3WtHP-+l`Rf|G32RpgA|b$--_ajcXJy|oC|>9t4ul_Dq4$$` zI_{6d1Rf&OpTnB{2C1Z3h^FQbd-UMHljb57KnuXbdq}_ z@1`vlQ!-0ao%p#VOOTXokQ8hZrnXnzNWj_OOAWGD*D5aQb>5o_TLk zAv&!on+mHl2}N5)k@V@7nIZ^GnJ)IBdsJhxVPvI1$kw`Dl96i>NwCHo)Km?V9Zhxx zyeV__5ul)raK*h?f`aG(3-%Zq_nM;W(V3Bu;SM}Ug-3D%kA5XdsxD?CE{o1cJ0o4N z>PWK~4f>HIM<&nV#|TvhtSadIR?K2~&WnKXqp&8cu6|vbT<&B$X%_b5Avmk4zs>}+ z6oN_f`$Dny4pEAAwnu%jU`K~97VT{BfXVd3;bz~aRcD_?F={gu`GGrmf;|)RRZX7s z)a1ErooBG8uSuFq!ZcUVK{V0W5BYvr1Qg&%nnS5(f?% zr1{_B4^b;=8Row?kwINcBcUJ(MSLN@6!uAV))>9O*z0Ifh$zghc^KAQ-`widOp%OW=^77&H7BdgT_Ojs>o!q2aL@#yrMp5hxvX;#3%$F?4@6%{$|T@a^NE75dLok6-;%xsq=Gcl zli+p!wiT?!b)WTI#I`MSoR^-D;o*E}XOaBT`*!8POOqlcY6Sk;rL4gJ3e|{$q(f(P z(NX4z1{=tM1BNwVm@$rS+NP?MY}R3bhD?WAcQFNS#a@2-Kj42G*1Pk(Vo0Qmsd2hR z*1bz|>73YCo{d?dc;Hf^5g}YX?hdTJoh;5W=3*lt`IDuYOu&Rd27yH6{S29pXkwVE zaaM%fO)&zuKnG}0p+*XRYFYw5vLRh;E9A66_%4I-vVH+Mys!~1W9b>qs9`$c$)w$O zQZt@nCiz#=p-^=wMz*P@#s|2a?X$e=-cHO+>M)?^e&1;isDx}06_@Q7ery`_Iog|^ z+8q(=9H_Z=R;VAdav?_(r)#trSdfJiR~}3rn>+jV?5RT!51yL+_{*8sjueg_DZKdx zp84S0ho5~|4`o3s;=w<-`F7#h`NFrK&3tmSb>`T?nKPHFNepNSO=mJy)0$*kNFf_Z zO~|z-l|&&NEssqOx-C(`p(T(+T)|+Gl}XSdQpft1QKxLgEdyItN#BuwcY>hf!0HrCW-eb-*=ph@8q@ zDBHF-l}$c^F+LU`#_Geu{2Qr6M_ON5jq|i_CD8a??M+KX(zJ`g9om_iq-aSl6zISX z0#hPHqk}tPp`h479BJ#LqepvjR*9JU);z=_6#~|#oRZ+392Or z!xN$|CV$W@(wCCeA^xfZq}a*L+~yaAUIENkYJk-$O5T&!h&|V>X{upgWG#V;iY~CC z@Q}=|e2qPY=AFq@Es?E;a>2Rn$j5XQgO7wS>K}RH(yWyIV0qQ3zGfUx+wkxPy^2)j1(N{4$8n;+meQy9Ui( zFc7npv}r?CG?po%xqnE9txJpsRt2#RA_)jl-)cnCLDLxu8ntSe70?5Wk-dx!aifDD zw(!GWiN(Po>TnN!QD!E*4a=Fp;!+b{4l=Ab*3wmELEBV1lrG}BET~JHwsfg11H9|dQP$`l z=eXEu0cL%gnuOO-%Lbvang$dtKEyLJy!sTEr>dF!QU-2P%+(;@s{vEM7jFCS>Tbzt zak+?C56ezxiky_IAadQNDD)*6e%M?*@5o~To*;EFDL2e}yTHJv2WuXiAqzLIKA1c< zbM5+r$@gd9J6?GG)ZABhX3ne~2t3FyBm_gN>51Ab>xM~82D;z)lSON4tvs2qB+kj- z4!$FX$R@*N$&`ZNKSIYsILotex~=T*(^hsBUUMKOf;A~NiWr+J6fuI2b?>~frr-qu zOd_wMhXR((cG#i@6h^3GrB%897ERj#=cz;-;mubuZBQY^=RlVDVI}XJH&X7+f5jXQ|c# znHB^a7NYJ`o|RJoD-_TjI3Sx8*)Mq8HAwrwfj+tq=5`+$A#xhu`0T=~KgMaXZVS)A zuvIG!FL&>5Q{l4G-!D@*56+#ka9+nSUPc2?9al_{IGA zmHgO*XQAXr@8(B8&yT*IAA2W1b}T=7$Y!NRO4)M$+NE5;5^C(Km3P&(zy=H;U}I3K zn&Ub5Np2!{J$DCt7bda40r{JA-{o#TsqT&%Nh9}mZZdasY7E;mNbVy%fbn3T#vSkO zwq-7rN~qLx7;@Qi7(iOQuzxk-=w2|nkxr-X)K(v_{LPq_T17d+OKd9}99(UjDi9EY zj#7k%n&N88Ou-jw?O=u}!+&d`vI5Zxg(3~<6s1!|2&x7mL0sqHs#_Q8~SClqL>YLX66HIcGP!U)v@lqVagKGc~0g{STV83m^tCWkeUDF1#b|L}KvZ?{V3X(huQAgm(Q>`!9Hamn} zLWj^Mkj-*k+Gh)e**z&)MdC&zvD?h_0m3q}fvtC*WwS$Nz-0CfP2KPSB|-tHv~r zp367jzpxNkRkL(2_#H(y#`(4hG7OJ5az3ZYUaG4ow!peyuPb)fh7p|)X%&S~*t5Vu z&??hIwoxWuiZ&X6osgRk)?sCvsy%*N8_Xa_;3=z0Ykx`4Uk# z*e;v#6bC)%rU}?}XzP|X(R+1`CL3%|#wLQ0y&NFqwjLmeq?tZPXG&vn@WoLDfp;yI zMoR~+6iq!_o61Dd)RU4G zIg^pqC&X<3Plckx{1Et)9BTo!L^oAv){raYZkvAhLJ7BdemuXk*nS?oTIfcH{XG~E z0s%n`*s$g5e3PNK>`AoJJh=nnvOulPz6jiPh|0E>R?UPxwZFe`{kxg#w+hETu^9_9r*F?(yIweZq~5&L!kNzt*RRgJf2nZ$ ztP{y!J*=^67CZWTIDw}4Ep!&P_ zeFAo-A*Ry=BPQEtl0l}>AyZD8+zH+;;VXuRc*iQ8Us?*js6FJ0dP$*+(v2sge6-469u!HJtaJfp@rsv_Q zpav(&U1D`ie=~iPlLC(Z_4xGJFP~<4UDo>EirO;BoR_AC-4vWZ73-7}f4M7bdt}ljGt(Nb!a~%B zvNkMiF<>hL47kkqhp=BY8YtIp3Af>;wjtBP)S=Au^&f&=x?DEvj3K!jfn8mu>VD7j z4=Y5Bg*QcF9lpS(n12)R*AhEp(V#yPYu^;6KYxiY{KeUaB;r0ZjlGAUum~04$ZS*YtgF;LJd`D&}rBn-1g8_swdzJJo&V62s2dy2W3M; zE+`k7CTmdT?m6iJGLeL6xIF3c^feiN24DLs4B^Av1j&7h)KGAEjgw+cz)5)JeUqv| zvmwYQ2v2vpANBOXAnViwlZX<9S#d3sGE-DZuN~P%2zzs*4Rq>7v7~7nyFb=aA^}kd zM#@u88O5bU_GZ&ERVmq@*z=z!<1*by61p~^a%!%u38q?nzseR*>MTKr;@LfFy?M4h zx`iP9GOdD&OEXMOjoLg}8!G=2RCj4bF$D#wdLpi!!^QTF7q)l4u&KYFuyiIx5zIqV zix)a}ZtC2$y>DAvSAV}zXTpiLiX!RLE%5X(z&BqW>M@B#Rl!VY=_1}))gkZ@=K6e& z<>aMbqg~^MX0rNT9k?gKuUBl-fuk0F^egGm`2*EK4)cgxp5Vz@v$hpY4S#8^_}wp-sNf4X!h0); z?BU6yg^3e0AAY%JvAr<11cz(I`S~ku;O>)6GfNaXJ_Jvt_u}#zbZ>_-n=3{ek)aCP zPY0L@Yk)o##sseDz*~2%}d$zp^kNf+d;Y_o-mayW?C^`w9+j@Im=-bpe zGDzcCyFr+)Yr0Xq6&Cq!8QHo8K1rJW0ygG7Q(xXdho-}rbjAifKN-%(@0LJihcB># z#|MvuBm@KH+ofw7&~}-r!%Vdp`IjfN{Z*}ROEjMZ9#Ta%sL6Quj)Ml&etpSv9S(Sw zQWtpiU>w+sI>UZj*&sh~r@+=_+B&xWVWfxAcq&UI!AF{@l1Wh#c?R~Mb#83-v2+HECuTGM z=g-9WU=18=D|kD)X0VWVHP?|HU^<%K+g1^d3S$Ft#K9}m@CWAo0j;%1N50+68elwv zY@$LiQf@k?+*aPU{+9f7;it1dz4Oz_pZ*O0KB|Le#-$SZ>0KDW`JX=HU;d}};N=J z%F(gb{XX?i_}qJBa2FW4_X&Y}SR`SQ=PaGJ@{zNJk9mw!$uL!{hMdv0JrDzn1Irqs1mSicKffZ?Z9{i^4d~d2j;9i=}YL_m1 z5_++V7U{=HbDY8mLEC%pcC5(LSvjdGu8`CWW06KMS~WQWCd-(rX329}wjNp)(Qz*c zf`@MUQf!M3R|hSn7bA(S22~RS+P-S?T1#$ZkJQ^>?R_p`?Za58;R}@;=&r+9AezFP zhi1=xJNw=R{CVoo-1w)nuO6TIYHaq6SC;9u+WLG`nXsC+YG6%1ukj%M+CwE2PM>~o z^Jd}o(Zb=2h1=I2-n>hU_;54gCJYP>)oeDN!Q{V$M33@790oKp|C?Xou%-CPgq%cH z7@`eu)=rt1t_;FfK9bO6{#F4z(FBoAl9mmF7%*jK3jXf@Av+neIg6Pi6>jAB zHLPS3TZ=W-Ak-xGFl2l2I}q3q>g=%*C7(YKCcPB)_9`$NYy-Dhq9t`3a;C2J_F zreyC$R?jn42iqhXT(V%KsLett9=KaHLCjY)AmJ-*e<`|pXvdatWv?u}qFaWX<6_lV z7YJ0x5(JwgN#`bP{^r#kJ1Oj8xm3Mk$BUy0bWB13Bt#^MJlZO;RH* z0jcul1m{}nUXYk52_Ytpb4rdV5^-X z6eS_*NRKp%b(h2g?@SbqzrV)4_z|pY2}R5OAlF+KXlQU&(dstmGulL@!*Y^W3Q~m> zqtSJgXZ-z+9)uQvzZK^d#s;fm&rulF+ZBuT4rL?3)vlzd;0rh`??yDC7){v9kcj}~ zRl9IP$R18`nGcR2FG18%_WtyBbbpe&o4W%4zRyie9fX10%uSNqhq+TNe(x%-2A$jy ztZ4~2T%*QDv;-99M^qF7 zejySNM88y$Qg~5FVb?P93t>mL<0*8&be0Blks5}a#`5(w^u7EcA?m1(ZiFc?OrcOK zQ43yMTdN0u?NTsNA?h!8wOpg8BGrM+!0w<{JXLq~n_cj0fPuxG@@}fPGc_T{wSY}g z*#dZy>sV1h5=5sf)mRv?Ff?7KiPbNRYLZJ4L?;RP8~yh4TgU)4U;~L;Yq^C4kp-mc zPLAr3N49EvvB8mW;F0-*jbtMU_|nnBv+g%W4mHnv$-(ppldgZD1dWeoE+ zr(xcvC$SPuT^|)ky(T=LWCu&IA*K&GrPI{Wv{X1tW`t}OI)x^iHSQBcUl{fg*rEJ3 z++xpVoe?ad2s*$^TrXWSB^w0(LnLVm3o8TH$)7CEWLWlRFjic`?Z2zHCDyB9F>p7{ zwkXwLOK(#Y`jV{hoXRSYxFT3c;S|rZ@u;Rg3TNf=1vGp8)Pu=mbEEIie*exg3#f0G zV*$av_C6N!3GdP9{6Ee>8rS{QS-2qeEw6P51 z-ah#E$%$C;)X2y`PTziXFK7R6|Zm#lc;PLLw(Zw?E8k6XZR(0DWN-hFs* zv~c-gJ+$g9U03rMauQ zySX3W-!058;`!XQjU;z>>Ja=xpygW&XY0$J%e`G3EqwPoQzu}oQ{ym{$*Ci`3DTVV zG&cbkZcdH0@R=FovvYZB9DZK$-m%!Gn|(UXu>Cgc&+jY6%K2-Tl8y>K2fe0;R*i0! zpz1PYW1S9*Z=0Hc=XZqO|mVeutBDHZv*J&JkafoiP4aa z4z3F7^C+h7gORG&%1+MLl^_Y0(OI0++V9@I2)j{K{a6*#-(Sq=YpRkZ+uOP~k&Ko> zd_nlYzm@`4TQ-kuVJbtsD2!NPEBCU&C(Z9{xD{**#e_A9m`UC@L28=W zRrhp95$3|@pjBUXCv0j{($vhByT`fHu%B@vfvML>?k4OD6H}wGSxmyFHwniRuuDwl zZY<+DKmU?Lk3LWL)1lNn@!{Gl(%jpylf9n1HFX?zwh5B^5G#iGjyIW`TqYmf)YjEj z9+4`}9{C+rsF5vs*g-+U=BNNzbU~YcE!s%fMYk;MQKY&EHwaNHo14S(>Km=3kgZ@X zno>N5INIQ;d@`yUi8zBza6*zDIQz1tgLZTJKUY=wv& znZF9mtSG0naNznhmIp2mrdCl#Q3G!*02)hL1xjbb?(hVJtSQ{&Y}d8DRZw&~?T%>0 zRTy1S;z1$c=vJt@fxhtscujoENOFQHZ*&hksGc>^P;?4W+)Iyoo_J_k&$>rBAUS6?9r*!A=GhkpL@!`MIGJM;Hz7s)@~J@oUZXUL=X zZvXw-H{{XDNqF_>=!bth_(p&KqmK?X|NQ0cXG$M>zI_Mz`KxdKetoj^*8cwHM`vz7 z!|(Fpw|~EJu&$&F-}lic@4+xYYeb{s&xa3r4~;!~@6e;Kjy@Xu2Bz4)?!1=)MUi$R z94s#YcMYkx3*LoVl^_a0v5l4`)_wTpg-Te+wv(~8ER7Gfc5AAI<1`4h_K~!vo8Hx9 z7Z1(IWo6m6$c*SRG$Z5>haL9!v4xT}#zQTT;O_0}qgqJzSEL69c+yiVgLLS@ul;Ie~eOgMq| zLnkJz)vRZMbW(jtXLKotSF~~vE4q9T3q=wZq7DP1(Q~xK1wZrJ?ANc&y>fGQ^z+%X zZ_OM#Soq>>;o_wC+)Qoyk5Fn+Arf+go83|w{`l69x8VPmf4uVJ^-2+uEzW|rEjnXy zwk{$fKc4>a+J7DXuR}jx{IAg;um9H=`SI-^e}JLf`0+BF{Qk#p|7*;9lq3=n0)i;{ zk%}kcSqgKnop^BPlfqXw{=+5*pPjbO=aRP$V@W!K0W_i`9yKd?5PVxF27~bvt_|^Q_=lSu`vSEAQnT((<1L`l8;rrc{;e$XW zpAc}AO?it;#lfWEKnA~U?b7x#oy4iZIIxa^)IrQ3MK&nHX9}^?63mF}@&E#4xV8NKn8&R}}eAsjAuqC7ClY~~?H z^1N(3KK8p>ia=3{pt|!?m_HG_cgPrGrg`rRFyDtPyrWzNrL~h;#%q)bC4wA3Rl^UW zTgwAx@WJ}?pwJps3H(X#DakHPO?JUz>ao3PpzTD*@oFp8Y~xz2TvlVTd{d8poythrryeZN172AXzC!GyZTIj|8ELM-hTMW z#r(n1-(dOD)EK`uId!7&>g$Eqk2Q-(j=qujMZ$5Au1t;dH%;*BroSD0* zsV|QM>;F%1&*E%;>?_Z5&5yqUen|1B-Ldf7Dv|ktI~fNgSig;wTFp~xpWl8o@eOHS zn1e%~z^K+Rv4)by@&*SWeM7~9iXEbu6H}i&bSa7@A_PNKXKtPvru1HmbEMlw;8|F! zXG;fhVF^O@&4>+sxxBvo$A^co1R)Yx+Y$tnaYP7&%lE{FrbJDtR7G)9rX=m$%Py&x zMKR`+gn+NSFl1SLn5L~1{FNn6CT6++=hj;$fyv4K&#hO~Wy+Rc-0pDaUX+;$v-B7o zInVavf4+I|D(Jz!d!N9+&$8q{-~69jC*u{II2(iPLh&`sTZ;6gG_sYF?V7Ef9x;(b zzaR#~U=51)_2apH10@AMp;Vp;^3vz~;O5QQSI;jap67ZT;(3l{)=^RUnR7GW-JZSr zZsFpap4W;bf|ig#3J3!dC<($=6XJTkM1-h+6?vczFp83zo{i0Yet7Qo1^7Ap?ss#y zj`jEF$HwMS!bMz}1kavZq(;>!e8qGU>Imky8g|M$d z*I}3`rZR&JP%yzgi=swgS11m@5N$NXSKRc|RJEA$sWxz^%H^QISs6s_M z)_tfHmEy16i5+3oedU42xZACZPY%NOGu@=*z<~~)+BV|UeEDVaCyOBuvqJBkrG%yj z;6W&+zKvLXVZ&>ua6N%KDN8o%_s%Eoo!8|!=}2iAS*^3h3}fL#X)baIlS_g$m|_XzLR2Hh-4b!W1knUdxx&7oVsf9S4QmfFOS%VG6)M{ z2SkfUu%sd$#6<4<+~fjHN$wg!1f|>!xbzxFPC~q-?>KVM9r(Zr0`IxA2%E$)B&JT( znYoucg-B3S$KlK0g|Xcw%`lMHc~pi|5aRp02YKB z@CjkZ0&{IG#I;No-ehhBGgfCbkp)|@m={s-C0VfZHWKg+2EpFNGew5-wq5NcX>YTw zUu@xp@J*AgJ7(U#-AQF-FNLkZ!nx(JOFeW*o<}zE*ua#AN6TJ5Zu9b7yLrfWAFSFs z?A3Hreo|Z7V-ZN74GSx!?KO~Z45iX;=>t?9+DJO6A`fc1T1E8zCG>7HHUZf@GS?1S zASC$3asj#KEZDt!tE>#N*h`i{@!TLa42zSSCdltB22-g126>)Z%HA;2_7XyZcgl?P zGTCHgucar@1F&GRCgF=G%^29DTVRTr4EC{Z!h&r_UIa3NQ;1JRONM4(+n4t<@+_eN z`xR4*<2m@l_Oo$KvC^smS0n-^E+U1tS;@a0{A#`DOWT=hnfvXGstadgFv%e2*7ke1 zah6(PJWNf22|ol9@Q(6SyqkT=hXS^ik>Zid=PD{hd{uXlzOcZO+m=YfYQbVVLMLFeR=xQ{Z~p-C?3Z+3ES2GzBGOI{wsewc>Mk=(`WyBoZLSO zm%p373NQcr$o(VJm*DW~^acJF{sufl?jM4;BW>K-`$xQ6Kup!ML!@2P>0V>VhRkb2 zh7Y(7(gYE2t!lVKeV8cN$Sn~xfQ|KT5-nX$+YE)KF>*Bj?L14OON)cu4m+HkD|vX6OowKjCu$K z5)u5N)j*4E+GjEyxu+Fb9YQDz_zH`j|F5Qpu}uXhQKv?V>Qxj(_A;HcrKxJl8YX+U z%i#M^o9+zSS~*;Ha!3gBAB{AM=PKpVhr#WV_pCaP9xjO`>|QLA5UQRKsIjGa%(^O? zmLT0Sl6ci*Q}bSi2OXg85UG^TB`ltI@X%z)s zI%E=}F9bv7q0gt+G&JH0ZJH3 zz`9t1TM}ETDpy3dJ8Yzo=DR4o32v8}md$vIDYhh6#8(m@f9+C$lHhj`EHv5$SKBW5 zkMT2qpYRUGRl6oZ35&p-2sN&kW`r|Vgu{F;?{@gVr#PXVQIi>*chIU2yh3B%L+$thV9CV$Gtmdk0lBm5CxxO z-*YX!T~t?#;;G!Evjvo-P89~~Gp*@Ks*X6$fN9f0ej~1XX|8C;I7a2&>sBo6-XPnz z*cPK?!?P|PxD=`%jG3wBD%{ zJ~N~9t{JZq0G-++kl$IdYQvKBTqp%%K&N^&G5Dus7n3nAvP)JJnWhcWkpeq-eQabw zpWt%@8|8#c0fMUQX6XTongmRGib0abrA|PL^(BGk0u}HoyBwTd?87 zFXzMS2&!wu*8tv5!g|&~^>*k}V*qus0c`TY9+dT9_xKTQE9hWbSsvCItOff2PA|pV zQencJQfS(_qp3^^O*=LC`uvt_kKnXL)4+Nlo4_hdIYrU5#Q}}{oRwgvrJ%t?O&g|G z7*$6|h$29;jRGQ6N_mG|ro2HKK|zXCy^3I>3D#<9S+@Qo$rbjR9+;s)O*gV;s@0%s z0*q$Qi^O)1k}RSn&pPVTv-Tsem*~JzZD=&79gc;7J=8QcYn`aHsNj#51EN*U@B8Z% znyeuILr0401IS=O*NNyvQSeJui|*GcD#Y?1V6euzsH`VQGquc=_85vuf+a60=rZo* zM>I9bQ8niYBIC`YGe^HF+&x|o{!uv<*sC+|-^h=DUwGr={OGqZjQr>!&tl1sp3RTG z1OM42VOu*s_lqYagdKb~u07V-v$r0e{NnerHOSQDAGfJU3&mqh#ve8Hj~$9UIQYk2 zOCN-pfHQ-C>_(aVvCH~nNZ6BM$zN_CFI@j_?z68;I9q&A@6H+|EQ#j6k;Pa2{e`0! z;BWnM`&5UP$rh92!?IRP2;>ir)!B0}d-h=A%EZiD=L=Wg_nx>ZPF5#FBY|5)3sH+u zRWo7x z94Y=<|Hvl`gDc5)ak$&syFJUVbK9ohZ|brM&n7$fq6-K?$NF<^z;wz6O%KRP3ma)L zFx*2G&Tq^>T7Nt8HWKtvY`{SS$98SKORBspV^D`D534Qrfl-6`5Yo1Rq!2Llh5viK~PICCbkicao+qiV~#6Sjq8X!smsQ_^UN42 zJFYjGcEBEn$y6K~Gp*t;I3fy#t(_4|MfycUHo=DVUSZp#S&C*8Z3)`OC9(|&HY$W1 zp2gGG-od*&h^oa+YwaK`{D^lbMOK5X_u$s&GpB!;J@@VGtDipn>KjCGD?0@idEtx8 zGw*%y;O=Yr(XaF4ALqxe=f}s3NU)5eM9FBTii|w5>3iD%q@ShIjap#}eyAE*1wCNuzXR)ukgJ9)e+ThN-|BGfaVu z147haUaVjZxs-A4)2T7UZRJR{cXAVCfjV>QeK@?Cn?O{=BFJlQa_VUA4xGP1IDYLY zeBu`W)FhtzVQL&dF*Sy@V{nhj+z-V5zh}w(C+^}K@D91_@Ruj-Z^?ZCXK&*jk)AU* z$=`W5cNso~0J~#T$8EREeF-NI3WD&Mw8Ud7=`A0l=Z7bE8m93sm&8>LJzSK=)El{* z_R1H@7F*y4E>%WE2s-!@n6X^4zTIM~W{@_y2pUs*uzK!&nv_*a;s~2daW^Ob_v=4s zMoK0Kg$o`k%((&nD(E{ECZumoFu(NJ4FevV_GQ4k;*dSMDaRF7XRBt5q8SFZ!NTK(K-loO zX;q*>?i83XAlM{QidMJmqyDmRU1{MSXFxOe6dUE373?DO#C2|std4wl}_w) z0(o8_JGH$^5eR?HagzdAZ55HK4k!IOr9|UHGS*`=Fp+3SX(LkYYF8!i zd9F&ut2Sztuxs5Z283Xsd_Q~YSLL59EVE%`m!>6*DzeRwp@c4g+?Ubnk-(kYm%d;q z9A1(8GK!o^f*27*|A_5%Ip?MQZS#DWWV=QT&C=t@rK4h0Kpw=rjBJp&LgjRwJrcc) zl3w^~9QU%hUCS1a475nw(lEF?`9k_Rq zE?h=BTNQP$O5}7#(@hu_d>F&048fJLsEDLs7IlLW-s&BPMO+Elp=ndEBShMpDNg*_ zcV)F(=?quK>3cVg%IdeNRN7ZQ$qSKD(JzPrzYz2ZV#tOX)e@|%go7*V;HVIE;3axl zty^%kq!~o66bqv;Zo_wLm;zY7^O5tI%)|6GvxO^*9bT%2IF>~ zN=aruXHQkOkCFs@%3Sz(LL(iDHNfx%GI&(7`r4g|@phf=mzBq0_IXZ=mw$V2!WF|| zAs7~JgNjZ{F*sH6@@HTzkBQjKyu#> z%}Rd3fsd z?D?BBlb1Y8V&>ZQ2RCh%ostcgTyl#`4uXgX5y$@Z6!yyKay+H!s%#M58e~SCsWobp z@%wD`w(g`7_`5P4li2cnM`xtUr9 z0~!fU-Cz>Vd_|~1 zOeWjX$dQuu7IA651v%AgsN8&O>`!({&7HekxOM&Ei4WFre-cB26kLrW?4UZBH1+2J zVVkT->#9qU2t4y^6Ga9+^0@UjaR`U z^G^+ml8l1xWfT&8(ej$(2Ab7BG38Upk&!V-$U9$J4S=kI$PyZL09iGRizUwQt#2OO z_-OX@)rSX9&D}b-h9-tztPo^vxWL>6)@U3FQC9B-doFqjA_k5>P`*9s)^f#yJ2k`9 zhFdhXrE?w=#byQOK3wSj&rMF9Ai3MvU_W&f{*B}5+;=2*ge+|43{!9?ymKF-V7neP;4)6N~eO-rRYKc7vdL~$f2 zfJFRD)5I;Iik4+)MZpGzaP^og#Xg!{?X7(yec6mg2IYOAEc^R665ACMt~49Z;6;|` z?~f705n}q^4xYrcP7+)KjtE#hyJ6wnMuJa&mQkHwg@YD;(CmNa30OE!WNX}H49S1_ zn8Ef>b{Jr+BC4k+;pbIy|0u!Z|33fM<9|DNeEQ1t+3D~8dc4jo()*+LUx5)_nZ7`# zuiPIk4)6XE!pAuO^3wEWa{tKm<@-nf`$NwX+)C4OB1H{@su5~#fL0xK@PKqb(*M$oxJF=~?P%p}-whz1k*QHewq zUEYtO93;rO6jxQ4njb%%AOAEzexY#v&vjUydXiVDew`n`jfj<_-##3_Qn)t67{3g$AA8Kg8cmI)knw9c@}Ez@Z^G6O(i+I&l>w%g=PyStCF28t7=MOU9=7c zIqFYnHT&Ab&%c{_|78B)=-i2S3K!qREbH#w`LWCSu{#{M_=EiTk^I=_{r&mTujkGk zESxx2XL7@BYV32^xbx%h!iZq7vu_^pEWDjG9@oI8d5&hN$_!Q#n^fx7m0Wh~hGITZ z@P``EKr3x|J(?ne_q+xhUI4sZ*i zWthsdfN`d>|5GWL*3sq&rgc!2Ce}wVty@RTAbjYW`(7k4^D;9kHP-IfULHZ)p_&NJ zSPG$qU;~mOLR1t)pRM>*lx&id+{-Q`1Oip3#$CYC$R<_5>o#RmVK2W-4jkyC`#2YO ziD-NN)gMcY+9TklCsivAFL&>5Q{l4G-_QBFGpVeya9%f4x|Y!r{rz|)u292}4Ss2x zYQW}MJg?~qji2md89BkdWAyn)$RJJ2%370ddw-OAvuS*{Y{XM6O<}SX1%?$*=}b|) zmzeL8l@wFb*w%!c@TKP|>@RJw{I$F132T>p4J@i9YabDB5|nf^vx<(>+83CGkTNb3 zE|1YHYX&v-rsVyWHbl$yb$;|Cbct`h51M-J!WV^$A3XTs+nI}>S5nW<^*GcsHxAZ? zcFvE#l^?sAAAc3x{+B9fWl1X6%C;$)7gR7RL_$I|EcjQ`{}#eprAVOML|r!P+tf$l ztXwXchsO>UZoF5x^V!_xPiIdZn!EGr+_%S;$wKw*a!3Ylx_6z3rtr>}Gw)xTdF@p< zE2lIY3+W~ad~qMBBv4C!yXP0J?D<7kluT5J2FvsDPhX1wq8Mq4We|m?)86gPTCr$^ zu=26$rhaglk6+*5}1t|)y zcX{bpqX=oq7IplC2RFYdOpX-}e_i!faMn_B$x75u2qu5v$W zJE!9W`=f(qFj7SNn6As5TorbPg(+G9l>>`U42Ogu{0iBsOai5v=zz#Uf0t1~lQ zj70rJR!N&>m^uiBsxSvZPy1BovmG4O3u?%S; zR8Ly?VPa~M%m-*3fo*TncrUZEEJ_TT(m=(@_M2MJ!-^iZ!*hOG|Q! zEYiy|CQA>%tcap#$v z=r@~wTfI{AmC|4QwM)rV3I%&B3V#J_%f@cx$FAhZCU}iCR-4ryu0j`0$jsP4_`5xq zT&PMS#bBk_ysj$neZq)7B8pX;fMCoHD2(i-^QJl2dx?$kq?trzafY;jvXTXj2dV)J zuq^ZcL;3?*O|T4uA7BY`?ip}dFoaWQ0_;dxdOU?Ord|M~lqsYcz7F|=rVo)0m{ISPG%xOFY+#RRv-+zD>4%Y2DiCc%ttx3p ztn>1Yw@|JdZM4vCNuk5Y9~B9f7dO~rT*Ml*eut5{Dd;f7y_kSo>u_U|4HLCm znkp1&!;^6A-LUJH6VzBzTxx?I5#8dBgfXcvO6A&W+2;ES63Bg&o5)?xU7I?XyO#St zcW3Ig+%=M$B)K23^&b1u>p?`h&!=AD6!c?AYtL!u7kcc=uEhEbjhKeknmfw}eFsqm z=0D=GrMh^dB}QM^ZuPkrBEpcAH@tX>3BpoN>adN<48uuGO~Tmu)e4!d%}$K`%{o=% z>@lRaJY(d8zmt?n*S#wl=Dm8W)Av>rx0lAr@;c+hQh2*b7(e9APfJ(TRUG}8b@FB>wg2R&FOnuTw#GLeW9a3wd0yH6Yc8Wr}P7o3cIOK|v}NM%JhB(2Pzi#4fTjFEOrAF2>{XzK1O zudnFVRFkTsA;=rrI@(cL)4J z$bkyr+SG2>%oxf28BFb>#d!%FSk$4esZ<& z(TVjw`-?p2B@vXU(Iv^p8D_<35Lkn#dbV9uw501Pl9#l}R9G$GX>c?M$`I{dUUxg)^}>rfYJdwd;AZ zmFfeQo3 zZba}q9JofPInBscS=Wrz8dU5?K!6n}Od3K#p;!YRNN8c&jJTrUdcg=KWXd0hMZD)=E#$ z;yf-+)Z&os<1LQXyDA>iB`FxR+fI_rcyd{+6~SVyfMc~k*wrbax4G_lnbkXPti^Oux`-?g@oeBfv{$|5Uh@NJ1T$lroMqza2{ z`TXrJ4F;@fY51jSOOm)eV@ue+j1HzQ0b5>HiL4H4dV0CC=Vi3{xcbE zrTdvOpljL?f$<;-Co#;0;gsiexA*QUPwJxO->X^RT<@X^&rQz@E&vZ~M8xKJ!B-NW z`iV3$T;V_WFGug%u^S@rz% z@K4)3!X+$Xu4%*~SQF=j(rtK!=oJp%`*iiaa#fg_u zCub%v!QMG@<-M6R;}1tC=Z?ApL4Hh%IEc1qBXdzbMg4h>F;KazBxO2xBhBKcO87>LJGhTcY!DxK`)qv5=d^I+A;TzPVa~Hng)vs6bW0!upeRT14;qcM?=%vEM z33%Txw_mNVU^6f2J<02QKP!B3e)iNW-gVdi~t!=egAJLNPin>-^l3xh9r=e?F0QV~2bISY2EA2b$ zqIEpz6IT+@7N+S*hS6n}>hQ%Y5TuCBE~s4pyh8Oe zh$tik9UaTokjv$h`*dmyyKgx%(4E{wv6mLRb>D}>o4JV?IdCBNHXPl6E60mXx!C_J zkr4uCZ*Q17LCVALxp#|w&+v&iayLotZQk>H7n_B1x8d}`3a5(qoq7ds%fHQJ?h5}3 z6S-Ub@8BJ$-d~9C4_>(9w}cTMDqD;Bt?>Kr*gedVVzQ-0W4a{#b8uz=}@SZr~w|f zcMm)e6+#ZgSJ&EY_Ux^PC%@QE)gh`T_ow7P`h|>|{N?uX!u9XwKKpw2?jj4R5#z-a z-P!-2y?1SH;>z}izsmmmR6US6xUIhFcAoQ`@`XDFk8NOP7}wNRYg_8L)mzgS8xvCn zh7fX3CPQwMOdyklkQ*d~gaApZ#%IoJ`PIOgSBbHKKWJovIP-sfsAlA(=Nr}lSK*}?XT>!sIB7{QE>eK zezlpdf8?6$u&ag*MOsIGvieMZaEosFyh%|^CS_kx);kLEa7i)D#LEHL@0QKv*-ZA- zmFPC@YRx62NxX*eZP5g2qW|NKsZX0ila!EBs*^~Oz}a8J0A1+Q z_OW!v`@hp(V3tfDpFTD{>AiJz`WO*YY9e}QjE-8m!AA-9s<83;>l}beA|kBTUH{$# z>fX$e0!}YvCbsYphvsaIjd$zimP2AP20!WW;dRAE19&z|ELLQWUb}Ph7sDpJptG($ zgFY9%LLVtSfm9(LaM45CGQdV#Ew{)?=lbKJ2X7VB!#^n@xvC)`m;#$ei_@9y1tE3Q z3q#r(FEWfn!Wr8`&zWyfB^&=ud90E2yiXNF6~LdR06I(0r~ND24?4>vV-`+VZ2Z8u zI<%N13sL}qy5+Dpz+wOSjprBYF7Lg4xrsHGNM%(}64hGp(R1GQL%M4<4P_kZkP_rT z?dyHU1nA`!mPCJ<`sEG4v%A(h0#0zPhfk4;DDn)k8ik*EAEB#WoY zONefLGK(!ic@%vR0pSn=kb0^hD|L@)g5N;&LD%$($kr40^ia~zr~wNj19@Jf*6bHH zelZ5T3nu^P6YyJpas{r(V8&=9SsYACrx< z&z`z<>f<^ZI~r4p0EAN{A(g21szXEEf-dUZy?f7kaH$tiY)Ox9kCa>6d~#r#8mQa&3n71{_?;zC@m4ji_aYDw4WUtgR3{ug(nYVd_TuyrIDH(eb4!<|kK#xo99e{w%U{FCFL(o>mKkr+)*o!P>PG?9 ztWK+_bD!SxYOs%$u64lPEwdY?Zz4YvVk{+y@uVOrF(K|N{*uLt`Qp6}`Cx$g5=has zSa^x~a{TPv8&_tJHVqmB$&?aOYO7DKR(6xyY`CMF8k$a<(kv&5SP!7oKimwY>mJSF z2rsa|f7g1jCyMVX%(Jbk+qT|JgqAWkFwTYloGR zE0RHs32ID+O;Is#<%Pj6ub+Jj)*PcC!(YjxHrIau6>Bipmv3W7QpSaNqV5S|;2mHQ+C0x~v6!uKIM-o=T}W+PCt0R@(`zRJ zXVw`rZZ<9Osz@(~h1%4N?gTGxj*nix*y@ssveZfxQ(}VD!nUd{Am`^otWqG>F;W5P zA`5Z=@4ZErdAhhw=-v@A;hRdfc%>Fiz@^~Lp-ECms@3osEt+^Vxp>HC(MS6>fGDac zq~z*ULyIUtl%3RSfKG{yVu?gi1towK8=+>|ixcx$llIQ zFBXoGu6DAY>fS5Fte(HaK#^$<*ByhTANygH%IjnH5 zudZE;5gY*Lm9>kpcJKAWQq->F*x+%}s~d*BiEKAvSIq_&xJNFfmM{}6wsI&n#WREL zaPl49%tq*@CWJ$PI*Opmb@Sym_}SbV_ax*4%S6`CFEylOpG6&gL`|f`jvc)q@*~rr zasywWH|{GuiiebuK#K&Xm{*e;(Q#x0A10XE0-xPkf4PyJx^3&QB({lcF*wNj=mKmj zwZ<)KFg=$qa4ScBV~*pMkyi0=1Fe88q9{m#oP9*psa@_t*BbxBHg3R*#{baAEr%L^ zkYlmpLE-V@BMm}z>cq{dO8O; z5A5D^-=>S-FEZ@rR)O?VEsMa**~o<0jgO&?*g%EV#B?PDSr7J(EDs^QYkWSrwIo4| zK-eTnHsalvX$gOz<5ugj<;qhF(3Zn&eh+r`S!CC|ud|EWq|?m7j@pD3bsO$kBOcNg zPqi(jSzCX8>Pv6PgRgW}5+y-N3S!JZz1pAK51ZVVJaG0vGh>u0v)LDq-9B;*`xkG1 z|Jj|teRun#zs|n+{yO8OTVK93ck$g@Pk)0iUOj*3$m!cBznOjVli3%e*SmMS^kA0z zRaixXg{F!uB&53gxDhNvhhGEI!O|%iw0^kyteGdf^Z#>!?997UA2r*^2$5i3aoE)l zS3d-n(tf!5Dd^Fuk4%!8I<3RYAFh1w5a)-hFFPG8Z$Mii|zLGZYLYjm9J;A`+zNqzUofjII@t_$bckr zOppTYb1hF7Jl$Q?ZA2teu5`}HAsgo;1Ur`lj1p;b*vXe}ef(^*rCf9(+&=p1&1>gp z-+S}c``?4=x%1NH*(Z#DEvz$^TJqbYM=1h*^_8H$7&53YVZlWr(1Y3%a46td0|2q*O4P`E9gZl7AD>)J zr_HYQ2^+uQB9(s~qJ(9}$=04evJoql!HbP<1Z?FQ>S8FCt}5cKsDhXZJ$tW~X~tM~ z9a5x=W?|zUa96406`p&A&YgzQ-Z_jYb!01kG%{7T1EU$D;)}0}e(4one1VfI)o(IC zEX_ANsL5iI+|h>(5MV8Pe1oA6n{0zSMkY_qZku*Rt{s4b^DB`^6@^5gz#>wbka{qI z;H}x*wKS9p3@0>ay{s_K$j}V-U+aOl2A7}%;!*l{eClJ&&6&|n=^nS>9@4WuTiS=L zBZr070Rn*qm#U5>w2=Q+i~I*WaqsOfBU80;Xn#Q2xFD@+4o#oMi`=G0nsfmZ0=E?{ z(5GHA67pWRivwzQQ{Ca~f&Eqt8Fiwf!+r!TW4AZpBzRq88BVQi$97ZGv#=@ACFwT( zaoy6Bz6pPlnDEU-_YOj`X-bI4TE&C4EOp&pecPhsr%E`Q$XEyw55Td9X0SYWm_aON zV=Fl+P-_Igl2BA|m=Z7)qsucuY*Pi<&tX)=9SL zMqxc@t=cZ~E^K)wo%2}Xq*J3=Rv0IpV>p&}49rQ`d;*ezc%Cz>!$;ep7c#udZ-Uge zKvpDOTu|k@Q5%7Dz4URkGdNI!YFdq;G7!po|p&i9k7b=oqor)zv>LU`d)LAPRn^*w=e9^o)xpsof!_ zR6&Ag4%6SVGO_fqTqTVDav6=i>f)?!YXVjWQ!)y?BM?zdsB%n6t@MTP)A|`+&aDaG z>*}xaU$*!?2f%SxoBEuEY2$@oi*RI97b_22>M~CrW7#B5pL+ffnA)lvUuBU=Fr$18u}0N!Q0^4Rfu#v-LaigNA;}>q}vprPB7mW7~LOxoG5R2D6 zIi{?YHV;hvc;cm@p>B(Ld3oG|GKSm+f>Y$Pe2h4ZYI#&gww7lFwqcDEv}%q_%{A0v z(3?3*2J`Ssd;$A9+Erw%rjP>uawIUww@Tm64{Cprl)gi_M?`eIG(B0mwEhgo((4`? znA;&NBHZr5NpSWSBMVFI0zvgNUVgG+k@<&lL0#p z{AI^&pqF*ONV0+u9OR}>dO7TMl@yF_y)E9!SNxKw(8hZmj)!DfP+RC1HDgY$p3^(J z!P~;TWE1*4WW;2bPh>!&;a-Pd#_*%i}jMe>(ep9mOxN ze?RxCKfaz+YRaypvJwRpNl=rmA;%5QoSeJ&%s2P``biTrCo<;yR{?3(iC;%-loPh& zdYJ=^+3a+!Lp5m8*8RJ-Dq6I4A7Wab^s}OyF?6l1iY`s5U*D%2Q|Af`i-<;=R7?<) zh(v}Ju0HJ3L?s@W2ptbpQbHouDp{(nlp2r;sY4a1QjNHIOAxMF5S~7D@68KM6@-{V z4nT50(istZ+!&>fuGIk{QyW6CLKZo6L{&w2{JI0{4@=$CBIs~8-beH560pMxw%aX` zF0Mabz#6E1)FfTZHL;Jt+(e|9lPBG<6RtJhn9`^^fQS{$9jgKOgh-85pS?Es+K21U zaKE2L?7k8;Ii!~hlu!jFF#5DL9b$>I^~S}yC(qt`@tNq6*al69P;kz=qJ%CaZ0kmZ zJ0KD1hI4IxitWPA;w;e?&R3b^dj|sM_+Asi5XgW7g3^8<)_5`|twfIvowja)e~_yQ zrry@w*|wA}4*0S9h0Mh(NCr10T$Df$ZtsRJUypI)Cak4isnW4)35>*FfEp zK@M_1x);2oKo9!t2+>zb9h#JiAg{`z6H`fX_M0znUitIfTYtWF{>a>)ug<;l%*`v$ zE<2`WEz_f+0op_|?h6%#=dPq6i7lk6WfW60P*4@@sa68m1ub)X34+c`7jHlN{_N%F znrLogW3SSxH?Pe)IWQWcgD(bZ&MU=EQpidtpi4p!V|Azf9u4++=ILACy)gUIUz*51 zIJh!^demZ{!aU|mM?W)kAcEb1ChO=bNC8W&#X~DwZ*Ttz$C;md^3JD!p1u5WGdU?G z#NyS#g-0+2!5yAsmTnVAw;DFUsE&>dC9UF6-d4bLMHTezr#G*>F!x6EI=jX(jw%bO zR`u4hDu@hXA6*mj1hbe&4pogcI!dZS9?|oue7dQ2slEGo+s#Vy3O% zw^48REAG?d4bZ|}$THID4%-ejqr;C5IBeL<+ZwPvx{upZl|Yd9IwUD&A)aWN0R&TqdHrMz+v8W(0-n^^c*J}sRr65I(XY=GXrmo zYgDsbAG~hl0tFRERHp*9BP}UbmSRp5d?19h5n=)4$!Oz)3Po<~Ue({PI=o(gd!2Df z7yK%Xk&WAPIgqi3sKF=!>g1xcN;l_5fm)g`xe$GnyXvK|#kzZ45itsk|Dr+1y&>+T zkFGnCEC6h5wGcgHz@Po1)@?=Z+r>2>q(XFP4Hes4 zmB7&BmP9K{v{AhQb62B_G&v^31Jiz5M%j|M=#x+1Ir7f!KfQ71?RRf}e_9foXq;hp zYqGjHg+GJhW2;DH;=3#KD zX+;#`0lT23Z9Hw`=ti~)S!@-XRFhIGF$%20iAIYUEN`adQj)Dm*|I$1Xdc+{*AY0KYPjNo(9koX7&R%x)aqdJqp3(tJxil2^F}F= zcseEul2YxH(6H?w*FU^*^7=>Qz=0c2UcY$#qwAktKY!!N^k$=&@3?;U#&I}v3I3k= z*K;?H5epMf&QpjG@I1XoyC0dmqD~1TZd+Fwtm+yQReRgc4LIZJ3N~c;4 zjWzX^4m-k(rWFjtu@6`bL2pl-jISPk%p6sAm z>Y77C0z$g(?@JHvpBUUfG_>}?)@$G;bnYAwQYlsK=pN`H2G2!ju=yMK=;wp45X0hE z#Ia#GnW%0}4Ub>J3=N+}hm*wt4*%!D(5F%8nTz1SOG#A53*JI4n(w04AL&?h4%%l;siX=@tlAi9DXS&Q>a(wAuY5Xp>W$gUA8*?}*xR|A^l$Fk)4%)w zG*NqrNkItitx{UZe$|vr{gAjoQ^Xe@^Bq9PUDy!mjV_@i}#iiNh6)bA{hWnB)ZSmIChib zf}9Eflh{$LF01m4}yG~ZMc!B?jBi8%kiFAT1_Ql>8{PR63qhRahr+ghvCfPr(8ElyBU^IF~##Y@h zbXxG&$wY@RjN^#o;0*Vcc>YDl-(?wtlY)M*F(O&5Njh~!%B6%vAfpzV@3%6OS=7Ku zek@J)ni@CD8V5(^pez6b=>d85p}fYhVtZM)F`SrLJsT!{u)WE!xV>>~x{e1X#fR>u z+U;6|={cY&nL2J}nN{wpvZk(SBe|*f%$%MhIaZ)r-q}PBx!!sFOZLr6Q6eVOs=5jO z2w*i@f;Y*XLR#@hU(E`3@0uJn27Lg|arRc`>t1yZ_P`gZzw>AUG8 zrE8@N(#D`f~ck(lz|}*Ko&6@Xm$l z$?4|^eC8ya{JeB!`dEzzpsjyzTh#;5){pRE_~V-o#a;R{Eal>K4OFGJ-aY%mhjczeF3K%$!iFPHB zXb(MV&D#~CoA9d~N}hP48{Pq}sZrbo-sk4-TVFOl?R(?MX7qA?+#?uRk z^%!3GtqZ4tl8u`}$TxwynmoQ9Kcdleh9vn1|wd zxHfuUadiSJ;lq^+NtOgQh9?8Fr9y)?cKnL$qF59KhFg{!+~PJxgw0_J>str0gX$=p ze;twO@f$j-Gb@{n`Bjl0#W`fQBH^`o2N0K{1aP+LxjeI-_8ct43w95q&y9U778_n? zR9R?a>7ye$HsI^?i-A+PagoWiw?M(g?b1y$Kp%gcW-HW^;=ak*@sL^E1ddvcwTf4} zTr01eO)R)icr!sT0LcDwq=F0xAf+7FGCN=!<6zHf zhHLAi(c@Tw8~S)H`pgF)VnT?Etuoje=G^Xbo?vyhoKoQ{B{XAUA97&AO%%o&(DE$n z8q1h&Yh#?iFOdUwB}N9g>*N_`9(r}#=B}leeQv`l)#Lus#Lp0t>n0$qBpO6GWY6*J0HJ(=g2#=PkwUy>+?;^kS8#= znyP|qo{`kAwAN{qNWReA?{;5*f{cnIaTE}8=5FV&5*8zui*cb<~Hwp$_ zZ7c8>Xo7a1SHYtQs!!BKu69k7r@XczKs8MUvkAZ@3G9VU1~7eE6hVq`t58RtzY#Q% zg}B<{k{UGGW3cFT-i@FL#&_zj@GjS&YTQJ+ta+!jDy9QYBjQ%-TD~5qMvBM8w31ZP zNih~nt8yZiRunOrR<_8nsd79mZ`qPumG!KoK&1oiAHw!7Tnu$WUF3%kMXy(|(naT4vzyECR)4#2GHrnr7)k68$VzK$$v!Bdf zJA3o$>oZ56zy0CsH?KT9`_l6>M?brH`N-`L&UVaQdw2HJ*P^4zY-K#BvkIBoMY(o~ znY<#M6rfWpu2W0>*c!D~t$(Bn^GM;DCm|%0bytsJ#B^T|N;I!PtaE=?$Kb@^c#$K} z9C)KcLz{^29S7doXg}tm&}D~)(pamMcgmd;Kc4tg`LKOSBZW7)O>^)D6Y$@IB6OqF z+%{%$T_c^aqK(u98bk-JLwD=fu6_^)){#CAx`3}ju_ZABD8@r8wi*=gXGjsyOBVa1 z%WTIc1FYzvy=rQs@zk8?O67RkD@=q6z|8)7nGRDZx{(v#!uUgs3Ew(oDQa?ZU z+}*dY-Fy8#a`&Tich9_CM-T#RkQ5qw7;#PbgG2u4PM~q5FfyL>l~DScTf+9XB(cX%bg_U4J=VY+^AOP$MP80>xmhM6I7xw=aHl``EX)K7D%j*f&kf)h5%jB#4Pt z{iMe?rtaw3rt*!f2+3I8*$%8drl!X_J+KwTE*jXW>=u`+r`#2AVY~CX!PbP^>R(Cp z%k|gUAsR5Y~)q!_qyP*vgl#-Zi7yJ$_sfizzwSV5NSlE5>WSE zha^%;P{bB0UQM^iH9ESP?KEUeTOGN=WLlAglv-Vu5;}N#_sgdU;vC%l>iFIFUr+yh z>aCwHeM)}*^YOc9-Xix-UHj?MC*Cn)|dv9HP%)86$pZ)aBk#$ApdneA^d+*4-<8PCV3wv<!8VOgV`;`*{Q-jz?mOnyVm03)TwF{Ta#S{CA&Vh_ zrxT08!x5RpS;3eyJHvJby(*U^M9+Mn-<>Ag0(ClI1?oXB~w=n z`27@7WHf#1qKRLygIe4*vTHpxT$K=^g(9MeM1B^2%d>jUm#t4JVlv^gTD+q$6i*@K zxtwf`X0D~qVE3%1s2Mh|Mt zPcUfFVO?vahI=5=te?AHj=G$mW<*r#CSp zi78*}Z7QrF!<=rcT9CCcI~s)9p;T0h5?Z%N2p-ka{IxhGuN!)?*o3iiDa^|{^Q$6o@`n&Fy##c4#XTR1Zh9T_L@~p_Cdo;CXAtZg!vEfOX6r^O`$w43`5-kQ~qt(Gx4iVnW=ot-`H8KC!d&et>#ZPASoL4x-GuR$E~ zsX%)4ECE-rpxZXFLHIa%_}>2_{jhnKLo#m8-b9#XaSQ(9eQh_F!;}#=&QXW>!%ki0 z-$1|_TxSAUOeSP>j%`iH$scuy-o=&s1c0x!EVKkxBtZ<{=2-pI^icHkNdX;xW>Q#+ zCk@hI*z_=(C5CQ}96r3RNHbv23@N&Glx1|{bS&1jXRtz&ienrSH8F+rip5mjq1{Io zhLEJxUvxNuY|p=9};&OA4t#-DZo8fQ|a;Fpl|bV)99jVQMfX zRVOkc_P6QgFv8M!-Ci`qU5AY2IoVEV!85|g^5f+Lcze;S6Pn3bA*V-HWvF`G5!U5u zSTmTf1Dh}K@dctuzTW%tK}gc6LQHH8vTocUYuJ$+C%kdiu&nE6BU89)dbLt9;FWL| zJP=ik36fklKd$w$t*h2(V3dgo?6+10NlZniP%p?MYI)-$7Pv3vE2=0Tgd~&<8mKzi z)hY&ET`}mhr|-Ra?%p%eyM7v0{E3uRg{0aF_1CmmrmjQ}Z8X425z&Zjk)#$$SmD8Y zVn7i|0#B?uVBE^}Hz?O1xf=^95(jgzIv5*z_#04UC+rz^V-Kza_?<3hy3QfLiN0_q zJ;<27BQg;dK2_)V0N8(7MBwd~jH)H0%tK8PQnDh*!GXhJ!{MEp<5ELEL>CK?9llWZ zoCSy7CX{U7y%}Mv6oGVe%VLXHax2SB%txA7DvjZBUMPa)sli&A?d7-}eUxGdoiKwG zsHJiI+-BI(sn745cYMDj;AV{jrKdji26}ZCT9 z?42{LZLJUlh{)fwiDF%ZbxsP5?0w8_SdLA4S1O*x8)R(K`c^Re|NoxvFd^j*2#d-;$c?+!A{8prsAbMH`z$oU0F=WBteRW9vjm(h`tDT zB5hR!-)n+yHOOX!k6}Q)+O5vLziF!?OIt5HsB5@{=C?X8KzA%%OAfCOlBunGPXJFI zp8CA28%stACIm@R!F^A{e^P}~311&fB4i&7QY((hiB&M>er_3AvLE-GBdx)>8vaUZ^aT&zT8yw~BVff==UwbSwF!FWx5QXrWhu0C5N zg&(fINDfjnItWpN(7cc+st^~eeFe)6&myHWrE3D2o-AD`T`7H4y0nZe>)#uYW&P+f ztquI?(p#kq(?_R|mcA=}0k2*prK_dOu%=7XN8oUB`WXI=Or?|x(hMJZA>oUps1@c> zL-UR#3UUB0Jha0;;r;IcNw2#3WFH9I@?6%1IVI&2=`zwmI}T7m%%SLWW@_A^eiJ}a z1UV7f+H32x7?u>#Y7?M~nI>8`BedWy!!y4N&JyXN3pz0Qj{!(psX+=_{1T|0^5L)8^k#^_*=fV$H{|1)yAKmdE zt{%kz$p2jUpEFo3|HG9tIvJfhP3#}8{0-l5f4KUF21i%lA^9J!o?nKl56)l+QCwA( zKGQXe+kDs|1CE>3>u7@+C}Jao<@I#>NT<}Mow6eFa6)4ME2CA{vAVjBQz!4edim#5 zZ$zeS~Rv~)of`RAu0QNcn#~%(IT}R6U?CBw_(f>dfHcq3z+TTR8M ziOhVR2DJ)gcCqYKbRC|dd9%n3{Sb>@oVwZ4M@FgHRUuo29p$iE zPZ5Pg3lFho;E7|%zX^dSAWAKSue!ib*2SowS_mlFsdFd?7_JF%P$V%%HukyW?OE2| zGxcRXqnfH8UnM0eG%LFr;O{hkNIjEb*VUffJ!Osm)VA09tQgg4<-o>5&v&tZc zH?y$r!3&QPp|B!BZz`!0VD-1`jGA@^Cb(8@X84-NLiUcyQaTnBk^xkL&{Xl-nGq|8 zTP7zC%`n5DxGrz#F`i%Rp{W&z4E9kk^O2=Q!`^vVo5ea1d6RZ{gA{G%;9NzEk!17; zJaw(5EzObjl!QsEWNVmRO`dWyQ|BK16nCUaTu7zrPKG=xDv5(H&rDm79*Js{Mv^dX zr6|=-R*QQ)?s1~0t{T8QqAVl=qrzJh0g4PzW0Yoj1Ar&%ph}`ucE18u99~DUUjxV@ zA$msW1aDg}2mgAkZ7FF=e=5C`o_cMf^xgDi>00Sh>D$tUp&?TG9@C5HPo} zyTi&ww@W%rCl8AwokiHXaq6X77m4T_Ruc~$qriD7kH_HM-&!~NW!GB z6cC6;*q$CQm`MvnFAw?9olV5nz+z{nO*R}ju!Y*1uDOP5r^z<%Fhj^PVN{3R6X4-= zb&Cni5xTt>F)&wm^lhDR@@^qBA*nG^JUB6bRXcOqy{oWBM{_N(xC*YRYkHA_)inhF zBmS@IrA2Jqg-aOO(vLrmF*uQBB%X@NicmCj^Iszf!iM|w2sI2?r+O~Y(jAHX*lc6Q zD05(|Y*N6ev+4TBUy6xJwMY3N*e0$e1}EFkQh!-okt5r{W!(5bVO&_V_kc=`P_ zT)=YRRir&gSBGl&d=wc6C&6B4BEFAjKseB;^RUq9o-Nq$Y)RHutTghAA^Z4cgann6 z)s|c_tWRX}QA&zaXV9KQDEIB=Fz@KrH5^Yz$%N-yEcxGmk}WI?{HJl!PYY9@TI>*+ z&f0w(tjs7O0u$Rc8&B1E!}7XK9fc2)(lO4*2zd(8{hiU+(L{m|I`ATwn!} zpPa4S;HH1tq^PLAXU97T`ShqlLJHm6-!{lc#`%)CKX`02lkd(TFNW+p$zwaiR@Ff;ky%&})@j$NFY zyx_h5WaikbaEOtg8d%M3qFf!UANuZvZ!xN~YqcMiyy z5Rw6ih6g9fe>-)~poQ{W*-i_L2)fP$d2j(ulIEfa{Md?c)rxRA49rm8(N{Lbn-JUr z>cj;x9y)ssd{tmoNm4MJ_jFGDc;cm@q1~>hdudtqZrHHli6?9~V>{`-&aInwclK^R zd>BrWsSgQgIM96mxtlxeKmU{bzuKo#C(EEfrQ>Q^JIIfO& z07B4FR$JS)Z66GX_U#Te#v_ULMxlXMg--mtQA1ew_f`w*dAcn$=qdt^&PoWP;seu3 zrYdfn_d4VgloXTzrdDW2&a06`NCka(977}?TvGRf6z;Qlk=tNQdq2g4cel}b5~b=~ z#YPoGIyif*fupzXk4&fm?A|?|WgQE+5}IpS=s6av*B;$f$NMf35OE)XLC79#8Ie}w zi5B)wZ2>;Ynwy@GF<&kv>h8vPXy7G&xcUmwrrvdl_QTavWF*fiNE0oOXZrHd3DYQ~ z2^pKQgCU8~^0B^FC=6N=gPA!e?{!1rar^x7L(&k{2^++kj*KKD>TWzx{@)F6qyYJNZJ59AaB{5J-s{`Y^Xl-3O%u(*#I*;J{ATaTu;0zTe2Ygjt8C-pd;fu@MT?MAYjnv(Q+zN$=Eq zmX?Q;qhJN~b7y4gy<*Wjfvt>(n6Bkb27YZGKWOtDwa6GnFyKS(&``5+7gk-U@oPjo zcBbBQ;9-D8TdtYw5GA3?)QimVz05L}xUoseH%Lkft!QOYF{x=kggl1{6Jm|cc8L1Ev zvL1^LI~7`ahv_8?A{$XkMlVn6EW>_p1M2$>%8Ci0gtj15Oii^UkRIJ< zZ=8Ts(R*lh4%agEbf*cBKS~*ZxSDk$W_9QOdV1#QXLFx^Is4)l>ocEw!pqaXGWXWo zbFaOB>*Cefldoav?O)!Sd;Qw%)6e7TJoNcnr#`xM={T&nzDzC->{V1Do)T0)5}{bT zBu8AKA_oYk2y(2I{;S1GJo*gRYEDurhPo2cPzRELbco7p(qy{{RxFNH+072R(dpAe%^cWxuD#@!T=d+fjep7$U^d4I*v2I;H!)^Pske8$>1Zgc8i1Y2gYokTQ*i& zn^;(f$xz)SyI~XZq_e>0F@Pfm0lf|Zni=z(@V=<97&0!bV8ui{0IJneXr4kdqeq{D zA;8G?I?yB~B<1QxfRv9%;_hdIB;Mv?V5|VUD*l$!O+m*!*i#fy8YRO*^?LM-ad2dFQ2WixcjZA z&)s_I#O&vjk?FKcr`$~WMk6Q{8Y$svBnz?NY|;B$S0Kj8&R(ygK#c$W^hkkhlpbp~ z?=^~^y*f4ZO{C8?AE%kdmKFHW!o|>DnhXwjt8TWIow1J^%&G&evjJf!%rw>y2>Eam z#nuh|@um<(7UV#Kew5hYavg)WpZWau+h5&z_rtljPTu*;NANG6`SgQ3$3C1`;k|hr zr^VqQE8XViwKs2{`+R8VF*0-Xpc3}%+Xis3r~8(g+$=QL{gA^S?kEv zTMBl2CzCB9TZ^Cd5vLh|dWzT8y{gbj(OU{Nh@lUAp^^D_y&=6AF|nNuHW@Mbm`Q9b zmGj;#ck;p&zlUuDQ^3sqzHK4Q0=KJG4!br=@Ir-Gw(&zaw0dI!tTm8yX7n_uh+xP< zN)!}dU7Zw9RDzi0gOEg);Z}9$vU_wJ>j-i#Y;qQh5f)1fw^}+_s-B5jb(U=0YBB1N zE=zad$$#v=Wwh^`feQNsPo=}nO{50{WYPO`d$_4tv{0d+T#=b_XrYN%<>G1>w4W8g zqp&@O)A%T`>sl^x{Uy2#Fr!>Z(FuDzP)io17G8O6W5mJsnkG~psMt>)4^Fub zJCAL5xg+ww-{xe*HL)jhLwDZ7em828U5uM~YGui;&3n7I4PtNPMw8kj`hUNTC3M8V z{A!kh#GeqQ328!%E9&9Hgqgs6ot6o*EFmtthbbYzi8{Ob*WtU

#`Enrw{N%&SaW^JdJ7lQReGQD#b!DYH)>9rf|G z?90;1VY9SA6-kIIE&ccl`q7}yIB3XbrX*BMs3+ync&BZhBNh*ipSKx$bh1{{3ETvK9ldBUgm&_L2HOL3V%h&FSH~hks&0YBsu|!Ky2UkXtEw1ZUY@4gm#n-- z^kvVs>gda!yr*{O>4^hslPN(-36ktfU`W0?Bq?kml8UY4@p!dRTsCl5LnzKnels)q zJp!8?JGCC#pQ?gkpI)6g`q}NLUYUL4!^oWFLAjitId*zg4uTY`F-fr{0BZ<9jT++z zn<)PzSx70X;g#7_CuUzcI&(B4ZeR(x9 zQFgEq9h8}w&5vhUg)Gr0UaHu9pHl0dum@Dx_QTbufNg%b`ZAV1{c!bpoJ#r-HRZoS zyv}@c>PzppH>U|7C+rC+xs|u8;acz4jXX0}Ux_SAK5!^0Bsvs^6NPvnLX{8OW&O{y z|9s=0=l*%>pC7{Cx5+=>`{!wR@mKf{4%b)t!9QQ}e)oq2UYz^qTaoFpH{)hfK2NpX z2cPE`XR5j)Ig)}B3k{xEO^%!2e>V5&-`2b~8Ds@i>CBfRnLm?8u*eKQNOX;nJl96p zIBNfKW_8%0FW{ZXblKUiYlnQ;=DwoSur;SDLeR&gy3MqQMV)k0jXEwR)Tjf@uC^|b zd@?IEF;qdWzuUZ@jWVyAy9_B^3$NJjz==mO0o!om6Ta8gg$4f0@C23-IGh;Nb_ixw zu*e_2U@KTVx|j7?NAj?b=glIE-UL`#>|)!JSi5aWT#?90A-Sq}9XS12-7Sz|2G1;p z4%Qp4z&_X}i<93`C zOK+B5D!p4C=XRxZVS3U(%tnU(m7vJF<*emkfrIrsZq)%9 zvFgNS99yQvWVP7LhZURM`p5{GI%hdk=SG+@bE@pGsqp34? zn%IY?uE7ub$IF|@Fo;cV+f!$9X)-eP)ge}7HoIRLs)nTx_vwxgkP45|okcpVw~v4& zHFeGyCmZ{xE}9e;dTu0o<dDLAgFZ@X>Ad6cbVdw?YjdBjqPT3t;`mkpX0`d$qsl!w`SN!65NWVCa&Cn^a+7)|no$R~xA|IvZ_&U>23qLzL(E=UpJ|+pxBVrtkU{@du^H4s(ND zoEO%evE7}gmci_W1x8N>T5J@I!JNr_UFa2KJEW~0_Xb(V>t-OJ(-TkZ(sLl#rY^y1 zSlJ-3LFTQyOkk(!fL*3@@;i{w?&Hzx19xznaU48Y9!Msjg_+^&U z-p_KTK6TEURT^Ef?AhDX-#O5}ZSTN9g&uXxtB!kc$}Nb7U#K+Uo_7*mu{5a4B#!w% zQ3|x>ghd7g)q)=4sGfBxD#`-2HP?WBT!)$^qk~t#5yEsc)>(6#>@{`#Qx^vHuB@G_ zkfsjs$T)VnN&#OdwASAOhDV{?+uy%=--P95q4X?8R90$ZGHTu&fCUGm;PY>Gh{=Q? zB?K`hNRm9UTi7ITkb~)ciWkm|BL19aJ?kv)mQ;X_`vHF z{`=cE6pUtZnEg zSA3S@f|v+x>9@hYkX_u&1}kcDiA2ME;~H+@Qe}B%Wy#2HIs(6`%i=il*Ct))yKEfX zM_}vcM+ybT5e!zxmf*=Y(Pq$GrxIOCNXQ?e2yBvCO)pnGt%|F```BB|QU`TqXBOEC zci+a;6*JTGlW({EcA}3_Ek8~qAx1V*qnM{8`Pk6VS|6yiLOF-GSZwjNJGPE7lWe7R z7(w0R8D?RmX#48urwFNdwQ3!XP{4WP#Ep|b9(m>Z#TzHCfAr&#=l=Kn_0O-L{nvB< zdM-_FJo&%pu77m>LjvzkUO#*Nv+HN!@~76H-o5b@tnPDft>yKR8z*6P*S~>9U4l1o z!Fc+{iO6*H?)kd+!3E#DqK8wV^{|>0WU0F3Gjui@=veyGLMC|nMN)dJbYc2P>7w`V zrRhmhdb9N1^wH@_yarnnrjNnv@1~zBeOtN&|6TSnt?P-RrFSqvJUuzTK3EC-nSPd( zUV<~2L%urw-1Ko+7%VO_O-rAZu6p~J@Fg%S9qA$4(G%EunTS;@#F4N~es{loieMM| z-LH<{egF0J&!^t{>C&g<=RY66d*&^2@6@%QE`37oy>S^{-8=dEk4Ih@8oKxP@r`%i zzxG(=Lp!_nkh>pz^3&IsE0+!pZM^r^wa2`>y#Cow-yB)vl6WcM@4fZj-HR{({Q70G zabXXRpN3UMk8j>Pb^P83C+|&u0vqkG6{szR#g!?Y6qHnT@GMG@6YI7}k?ZC%HqYAP z#TwHF2OI_$SYYCeci>{cD(HZFP@GDw8_bH{jOV+fk33TCQVwZ_QjP?6aPJJkKN}P;i(=UR6DDigh^+N+7Ld_8*%{q{$OG)9yBDQSb*M(>w1{=>% z!yvuP++ZUJh><1TUax30^6cg&?vCy_Af0^iyr>mG2}~5pf$pT1RkK>F=HBtkKV5vU zsaA}X5Y*HoKlHVmVi%vq>gh~xRBu@Vd&|^73sWShcbefAv0+nkIm0W*s!#9rGHQANykYjiOyD}t)DQd=vhwhF9}0S^JL zXmhrnVfFYGa4+X%*@h- zv$P1Nc$Qb)0l)hX=vTuo@Fm~9%hm10A0P;gt1q}uCDK)G2TS%Z@{_X_B)6IK3tc7Y zgUI#{LOwZi4Xn_X=&cgH)I#29Vqrl7`|0FB_9GJh6)ot-daP9Mr!VN$}rlFpm=my{ZI2uUcN=s=dO$egZquRIgA~g;c85CQc2FTDohO z%c<&^1XjBWvC_38hwR!(V6re77$n-#PfPWaF>WDR-uEEXnl~nlp z=2Bt%=769$k-qNYrbSE*v7DW)1*);o$sbDz67ah!NUAik)Z}4hMUtoB(W~|dmNi4$1{inCgDopf z<3B(1P5P(H7w*0GNr%_JiN&ZAJk^QRMAb1^mww=s>Z^Y~ar*9=(?dg#-Fy3Wg8rux z6Q!S@dx`w~!sUBsUj_@7+_k(M-|3# z0SwN&Z(k!D@4fxpy;DyTFv9Qt?VDO+T@~niKQPC?bpILG%mkAj4;+6Qo;iojE;s zK6qp9^7%U-zkch}r{^x7y7R`nvoE}J`@?tO_3V}CUg`W6EcJlu7&e|nj9)d-tQeZH zd#UAB1`K0Xix!I(A4S~^W*JD-?6_yWdB5Yq;h@(|ur0dbtKPw*^k$&|Tfp-=Tx05* zTM!;EMu*8n7F*JmT}OXO5{m_rf#5+DfZ5w0d4R01y}$nIIFfU3olL?nkG%DpLGO^5 zZy-{tQhb^US$&$sY`z$;yXfs<=MmD+ZJTsvscw-jBby^qET%S-QNVW!gtDVB%B^S4 z-@I~V{c*|cXHU(2adr074{m+`r`a>#&0cshKKuT~*(YC``|Qf>XBTf?dG_YjuObtr zr@NPQ+C^sNyo$wDcoblV%1R5&eC1%MT%~UsSP^Me1gvaF2(J=*Kj_R&3d)V!R zYyrcx{Rr8NMeuNE0^DU#vxzo>q6+b3Yp~ESs3I&wa+)}DFNtLdiNF}rM^_mYsN2Ls zuq5_4#{-=Ntw7HbBmDW-<}RPVef61JAHUJWu#T#N!BBS>?0sx%y-!J$(%<0MUho=o z@%LV-5TyE@bf>8oTzels*irgZ=}PHp=|bttIw&=NlM0XV4dtJDui39f1% z=LXraOg4LPTOjK(n5X)D*0=e|4%m)(pgX7Bm$Ma{6-l3^M<3|m%|45H>7)2+ve+ri zVk>HO(lfNQ!;wz!y?W=^hj(82ZtjJ37<8LE^8VbBv$wx{Y4*bLTUW2mzI5`|YfnWF zAB@;}Q4xKUPwL1&0WA`OlB&Btu|Tto7+_7444dz~Tu+qPrP~hbMmmNdlAdc|@&cBO zVqy#21dZwFGl0tMr)EaC$WB`1b)=97SOF|Po%B)H5B=(!3J?!Fha-ip!csi+M93QK z)9U075~x-uw_J(E$swV6Q#h|v4h*x-h8^q#rhS@pjyNusUg*qXWIweW*UZ^i)z8SU z2lnsj-??l1z#!Qusxh(|h-gi{mtzkqyIQw3w-59WlCT0dUOtij^+5a7TN#|%QqQE~ zUS=6fp?Ou`Y#TK^Q-M}fS7R~kLMdo+huChh$i1X|X56#DGd%0zb>nLwrmh(b0Zs;Z zfgxVLO-C6iFfwfM0$c;$h($9vA|lV>!tLa77laf%nFL791wkMr-QKt6xpr_#0Tv#m zw^)p|TbvfqSIKkBhGkO|n@V64v9WD8LQaxS;KO=w!g{&s!0j?Vj9Il(x~hgOT}`DW z5nSWYHFo97L5rJZ1oK*e^M;NCtk*J5SWH|=ODgT#_O$oveS<-gwC( z@*1+kBkENXM2rK&@1mxL;BzQEeaE%fWlcA{^w4oi20*ywNf)dGY@`4I(s^M2FS)9 zcoTGtt>rm4>~&XT$GDYa4j(fSW!4+r=Zup9Z!u^x^^^;4t>~EZo2?ut1Fml#USz=e z6sBxb$v}NdSX5xomdC;|4+XK9V! zl_$DIBhTVy_*g0`06_|~x3O4t_BN9XNuP}e4ruAE{S#aJaa0{p%1f(t7RG@pHs;WV zhCD!t`8T@3Va8{2{sOWaj8u>*Plh9?SomX@8hPQxGrc?_=h zfy-j&5o~Kd_qv{haP=}G({KkpNMiAXuNtr}$4gO!SOB9UnzZfIyvWHRYG%0yjbIBX zTs3eF1yYzgV-t$4PPCSBxu&8eeV8HE%lJME^Ys=^rTfa55B9>+7y`6cFE1hqSz(EN z6R&V#JUkaB1UOYM7kXa@!nS*1%2X%9y@8g%UK-DhX<`Iyw~vP5b)U>*ue3_^;>S??;F+ef^8;A0n*uk4K)n z{^gAmaQfoEo?8!jfg30P_1yoS``2^u!*6_d{XDLyLaQ2Yrf)on5aBnD-*^hoK81+d z1lIof|DM181zd3cM)V}yHVW22p4v7wbBT`=(KFkFdFtdW&PT_!ysnK@h*42);dC1kLKfmJ zF!}}q^t}D$-51~Z`7bBQMlqF4-F@68)@f%erF zovP?WjUv2gT2~R!35$UOm5!muJ@zx(p{`Z78net^AdQm_yhBY5+*6XVbbIO?vnQ&K z*Bczj!2Fa;aI`hOXaKwE79&{~{cxM<;DK+TI9>q#@X97kopWf$&~4kRS+GeN>cS?& z7B#2NQViv9VEFl*S*~iZ-Qv_aOJ}Y{iU!rD&e_C7$?_mUJXyvBP zc?p5#ZWQ@p>*KD$eMkgR1MT>fDMB(;-PX~nPhTPw+Er=xrRV0Zd_4R4d$Xs{%>M25 z+uy!(Yx45kxpyLytU>kaNIB5AftXxq`B0uoVd+yJT5LVU1gZnuqq<3Y=x7~T-!7)x z8SX3Alv5S)?7a@T|JXEL9j^(UqQa?JXG)h!-(!-cbglFSCSJ;uAWIj8W>X|Y;%Q+B zR_Q|NsxJZMJDV=O3ro8)eQf$^SeloV`rJz@U7mii!@KZk=_*|DC3YDQZ*bNZbq>t} zztQ}ozb8@jRlJDd@)ST)Nfu&(3eT2lwA3^j#<{ETYPcSsvSHSdqzVbS1;Wx+vw^f+ zd_lI)caI7?`nE$WlnkZmhQT#31M8W1)G$txV?r_@ z^jgT1;e(w$xcOa%%j_OE%fwVnY8O+Ehou|j&W6m-3%95#Dy+3x5N6+10DLLE+bQb%VM8cAw}m%P{EsHDV%cmSe2N?y(b zy0q9IT?CD8C-4g}wqt3`_Eg+H-PCmA7Hx)YcJoblSZ3zcj*aDYEswCB8ZVA}q%Bfx z@N(EeXU)^IMsv>qv*58L`@vYzX`zlP>qDISyzOMAmo*u#mlY{03t}LD^eC38a3>G$ z7g)uvR+j_TpmG%?i7JTo_v<_&wgto3FWkknn`+`K2|<O3h9RK zP^(d-ie4)Oq*4*ayA^`^1-7!roxH_0YK%v(*lQeHsfv&cjL8gbwBP*p`&)nhuw3W5 zXRy0-aIw~P_UkX^zP>v9%xPcA?CgsdZh!FV?0c`?zIg4uHHHE z^qqGT~dk%t4m~E9YH>iv`Iy8eK zVxgCuthNtBLoaSv6wo{`u8WE;JL$x!A<5?j+(lSE34Mjz=&@Jly;|p?B%tR@qnJ9YTPKj!og%)_5t=#<@&wAggdK`woo^0%d z4YG60*761e$riolaTnuQIFR+JoJ1wM?!694gOm`DSLe1u+Y$Jg99R-yJQS$o#fdPG z#Ijx}Y0Xoo26j_tj#{}?M}b=i|6+}}#psb*?-oAZkSc1&dmWB4Vy)@R^;>v-%tSa2 z?zD`&Mj3g2h6_zqRS=cByGXI!LbvoB95JuIvYQu~z6^NJzRhh_)N0$uOm@g+#@aw6 z@qT<{vWTgekVqmFiR6bDY?o@5drK=4BQ7N5x>Jh}eMkvz=vpIsGtD{?)KdV@_EC(P zj~5+o<;P7e@0I5|d0J>1A~etiQOII-2V)-;b9-JGFc<2pQ{C5HlUS_D-)wa>eQf~9Uae`d`#_Rh2 zh`>LeFLrzr{O5nq-#89eUHsQ`(amrp!>vO!XDw+isK6|V;c%9aa4A6vOks(Vithxe z*2~FGFeKLjl?LYvH+(7d`7`MO3NH}V!^QTJdC*YAl%qwQrD~E~dH5!Qef6qBHhIc5{K;iFAdPqHGYQr(Cwz*viT@rRb_ z|7eYNZ^USKBvnF4q*f&Z=b?oymf_qX8~^JuEIY!gB&;$jto@WkjjE#Vyspt4Z?{2G zF;e6Pn0s1u6JyJGHfsR=b^lZeN$~}ZLg$c_m=H^~{3SH_c+mk>gq?}Z(sW?=dTJsX zXZ_)@us|9q8`k;*&H?S}7B>O^V5cU_rpd0Z_8jB3<7PY@z+JyxJoJVFSZz_W+>MB5FE-@)%tWJzR^k9Wnf|M`8 zo(i8~SK@*c$goEl5j*$V`?o%Pc0CyJRcy4gOG%QMV;{{-o}ZcgaAxwGnPVqsCf}Vo zc7EpAwVBDQGslkM8N9%I>)0FR7h+6S>oehZ^Nj0IpY>ixnTv>n;KcediGgB!+;uTU zBrqXKAzhoh$sn^h8K5>bDcK@)3t$pbyN;CYe%&Gc+%)wHIpP(07!NHFRY{Q5x&fU@m*gAFqyI<9apPD!!47!yGTy5l1&PLphSZ1 z(rua@rKSVR^W6FJv;Z=A_Z8Kv5aT6wJYFR&CiC|37Ra+3v0r?3alnF%}{B? zl7a-LwWuUkWlyLnK??Z0q4m#!1B-E(naN8iu_r$U>3!$u<+-Py9U4M0t;~M&r6|t5 zby7-zsv;Zb-umz{BElEPemwHR#=V2xkIhVi+<#`x^W6$DsNmW=Kt5(Ob>Lp^Y}d6z zJ{6@Tk_lgfaQP?3r+cpXzg8Wnc}Vkv?eYo$eKp~#AP z-EeJ^bAh!Aw9Y1vMtR8+;;)H`Y6By*myC0+Kor1wnWhixBO{F1BQ(Pdj^h7U9gmI| z-37nfU)xkZPIn7~dDmhc1F*iz&U9>LJZI72=$Xm}c-az`pv6`dTEtp~oaK4Z>&~R? z*AY}JOSVxDoibv1Gq>RP?{H0)^hFLM&Z7kiV5wRCQ<6F@Ew7KFqS6eGNy*;~_t@n& z(gO$fFWe%Hujcg_>6wQmAvU+X@rc{NFy_`cSTP%MBncMc;iya2wP&!y&btmD9`0E9 z&_MY!-N?+$nqZd2E&IQQhSvOy!TrANoF{*@?dhQ2RR{)U{1y0A~ZS==|F_i$;tKN^Dhe|A&bz#(#efL~#w5k2cfH z6q76`a&P11bTXl2-e1-77F%TiAK;Ygdgj4|HP7AB5ybS>X2fxUn+;+W*hqyLLxWrG4XHWlx?L#+jtjb?Vdo_<#OM2nQjMNrK!Cll83K)jQqWRM&3n zkkH873nVBio<>kn7+`b+6qKW40B0?O_v`R#!n$9fCIJa+zQS|u>Lgw1ZemrXrAg8# zq`JDg(tH1EU;8@#E^PnhFX%;w@YaDk9zTr{&ZS_@XvO!F+I(?J2exgUa1V4h=EV(o z#VsQn!w7jr#@qmc?_Q*gs!)u1^mE@hCWJ%e)a*a(mo{k8u{;Om+3``SF!^1_nA!g6C^f zi`0ToC7qXCXD<1_Hc(>80lhHr>;6n-$uFa_AWZKQ#Cu!Ot}O zjKI$V{EYMSpzS}i=uU8wyK;sIzr=f%8pT_IqRUS_gJd(^s==WHzo-DeSrCM$p7a!; z`5II2N}t=lp@* zP$w8?bM01xLDB1^h*@xkJX8&#Y%@g`lnQ!{2DPgfw5j(FPMv;b>eJ8f?|&57j@I1VdJtGqBe}*Kq+3$H20Z7$K;kRYNmt8AOwc;xWdu3QaUzM3&zxL+7{ye zJ7Q|YCG_dz>Gv9^8i6~5;naiScLc}14-83d1P>G0>#Hp3B8qQaL{)@HMG|t}FVz>% z3AgC20~v$fI;z2wJc7X3GAMZgYgy1rh(5%u-8#v!kpi5csJ*{uE^WD=IVS4nJlLj% z#7HE9nlmOwd05ctf}!ok#geum+rc{BjkOo>3VQ_lS#JE)52sKEG1D;kz!-NE`vqsq z`py!-iINJF@U6nAyiB9qiRru1YRSF45j4x>2As|iyR}Gbw7_mOdwdklzI<->qq7f> zT$vp`_A8B9A09mCwpv|i$vXS;`Pq}l9v(T1-`&vB^zg##n zd+Frt$L|Dw6t;SezYwihCC+fFRHTLZHaTFK-`{^`>fkFgZ(f>t^R0V(53B?;unHmB z#%vb!A(K!o+jj5G*JjTAGa?4}qLgmpF$5zeSlnYjt5bGqO2zVRHwZ5GF5jFw|H;hN zi}%hPTnV65)aIyD-(I=*>SZ{J@}oB{PM^GS@5A%=zP>#3&i;E}ADFrF(Y<|N!7u~k zw2m1(*P~DZDH;(ybvi!N(E%h$5ToV3lSas!ltSLr;5l)%YhxV3ts?V8bVV-DOz@o{ z#Bzgr!M2Hw@StVT+C}6Z%igkurV1g#alJWedTHO(s#}Y? zbQI0(KRC1Ri~Hwau0*Cul$#bK)71aNn?r>k-fSvSPDrXC2~t8(J$w`1sv6%lHGmUn z0P=DY!@CJ~i?7)r`g7BI6B|Rs0PUR`e{KTIiou~B9c}%4`U`o62I(%icwnFfdET+% zoo4H4{w7He3?xyzrZF=O7wQE!2ha1mrQQ;EUTN?E+ z8`~c0%)mXKWuBv6N$|X8y=5A1LE8rF=6nG-WI>FD&Z%2X84U&9Wg2m58O;m~-Z1J| zLM3yCS-C=L1|d1P|Nh_a@BQTd$pbSVf4LI3wx7q&-QRz6`tW!6_wEafgUAaUq9Q!p zf+%_Q46(WiyCf2zXH*oktyz(k!naQFy!>@E!|K;`S$4DXBju8-5z6!e_9s!4+C)+Q zY41KbRNzpYK6h&B;0O0^epa{8rp$Bm6S2@_AiZRmxYblB&n%0kbb^Mlk;Xc;avZ3g zJNtOqDAtpm3=p9yt4)kX1u-rPaj>S9=$^$d2L^b?KHQV@jPn~*@YM)KMY6MFeb1g= z!aFXY^&AUOhb+n|fc_8MqaTJcWVOs+x^=^zPTk>|plIucfdK(EOR}QMV(e+u3@2+B zRCp|kWDnC>3)-aX@H@k7iVHL05yVP$XQHz7qazCmwcH=#+ok;A%Bv{(D=y;Mqu1bQ z_TYy!$`3t}YNEYHI>e{R|pkG4h?Fb%1NFbH+{0tMKCIm<3Fb1iN# zY=j^y%N#vzJY^%eO6w37C`KHKTDG3p+_VQAS9F^NuYMA#(fB`c4W0EzhrBnr|k zbe+&zST?YW;0({guVFjtaBC+aD3Q>{XuI25)d$K0)XsI_7QbKw{w$BjidmdaE>HDC#sw4&Iw{Gt1|3-cxbYs>Z7)Da4nA zgZrt=O<`&WtXSjw3-J`JxrJ($MNyhi#4Eb@RGa~OmT<0AI!HkqLOBiuT~TWa^f8Sx zp2A#=IxoL`O|I_CSPDN?6k_2kj8|Fo$4KA0XlBQ66vl51wV?4=$xvMhvj81gh$PC3 zZ%oH0IuqaB9)-VqCvHq!pSU`4bK=^>)s^L$pZEa2^9}dAk4=2nf+qe7Be^hfW#ZcH zeeTgEG;!?qzS}RCPE33Uf3MwsHMsvLjfV@KR>iHJE?kQoy@@3eQ-yeV&(CUHPc5k5 zv2k5jQCyIu@>S$Jmb$n)f9m?!%olG=y>ejc%Wr?aHumG*V^2Nx)X&$>PaS*X=drQr z3-8`L{mJy<@9tfH^ZwCm@Gty7{lN!t&A)FPn*Q*sojaGfc3@!UqtTr^iys*n_-XIU zKkhvar>5V(aR1cR`^W!YPnT`+q96$lzcwj>pNN@#UrZf%ZTiBMsbeSZ|NYGLM<0|P zj6{h~DB+V&Na0E1qSBz;T0-Hsc!O%a&Q#a{6zqf1a;X}SUn6J=r7WWY=e*_Vld|0E z9yiNR*Bjs3RR~04sGGMOu$Ya}2BxQp0YCCtUN<(#n4QA9j%X%A;7N+ z5j8aYZNEo7L}z)%$I638ys^hzq{zkuyBvrtUvOI;B^T0TFlt2xv*BO&Ozo=V9#2j~ zacsf0pl4~$W1*J}bl+IT1u-7FUVJUcRC|QLJcQ^eUpsj!ih>lWAT0@&qeRl~g0UWz zl_a^vJp!k++2egtV$Uvi-&%AbKQl1!G>?qy*s{HeOJW>$YcN+OIVQxVm9-i-=nSMO zZ69!ZqJbe2d_7Hy5nb3%6f^ z*Vho3b?!|r@EhEI3&DpjxM$$13+^3W;lKRq?RQJx;A{9jb9Y~1ZatXOOYQ_*z5PmY zw!Vgud>i=bnD`PVm``pj;`IrsQgyG8DlPe#sIbsNyII#^H-#|=k&)8Kk}v3KB$4i4V3?%5nOVPj>6BuCV^ry?R+%HN9mOy2@XBq5e)_ywwp z!SPd?m13qfemOWw)fL#_48jV}rvMmpgY( z{B`0)a{Plmpbot}=l8z=_QJ$JUB&Ox#26@tFVoU{X^Y*KtA{MpHYw+HMN;|~i z4Qi~m&Mb&(ZD`Nj&p8vMKRJKCs6{@$<~%i19Y2DBe=~9Y_S?7jqUMQHFk_cMjoi!U z4HUtNzun#kFL`w2?cfn&xZMS~{S`Fv72FXt_}d3dkB1uZtlQ!Xe~*Y95!8wngpb_r znzpiHU+qQioc0lHrXRtW*|ceEQyE;aNfz&0|Ej-s;{~i!)QmOTBo^4t%ruagh)1iy z)3m*(X)!!a+ie@0sK>-t2BqV3fH;BLq1H*#)W@H5NQPZf6*< z+gyLB#T--P{BuScW$`dUDMAdCW`(S1S92-yMTIhf-QnH-e0D8 zBg<_%KbMe$;1I4DE|#je4A_`A{}Wc4U|xp}3T8z=%@Fe`3vDGdlZCaOWh{hlo3U&* z|SM z|J+jdz=t@|D@%}zD?B4LBA4&Qje^i+7li)( zz=O9(1A|?=f>4F0u`01;5yh+yW)cH75^bTovxy7XaO%fP0=U3FDK{wOmjZ3(tvBw! z_vO!HUxsjm*M?+4;c2T%)OUCSR9~Zl8x7QpL7v^h46K74#`P9Zh&fHxBMi&!eHDh~ zcHSs4jrw4tcJ;I0-oT8D;*$0y=;5H)_t~K98y2f#tTb(XuH35JN`|p(WY4q9MB;)Z zgGCut1u^0YCWX*IR7HqZc!buroCAX9zb}BW7{B5^bM-O(Ri3ISRkWFD>`ROGr7K5gzPvbn@WS-j6Fcg*H>sS}HnF^t ziq#?V7}Ca1*(eX|=fAmVxb<$RHB>8!rxw!)%asOaD1bG5`1bo?=Us74-Rrjxxp9YY z(Zty8{kLCU2_(@Ie*Bh$nyB*B#8?Gm|Jw9yryMhQ zm%HjkoTZ!x*=Np!niQpK-pBw=)7A^9Q$QWq)Cj0kcqOd8atHA+O~U|Ej$sW2PX%oU zV_EYQZ^riP_u8qzdJ2ds`*uAkYe%kkC zks$l$YkTi~b~-Q=>-xL34cHuZdm=1sPaf{zxKH+Al~ZwwiDgm;aw0HRitI!cMM@|M zIZ`!EcRoD6z+ktC)^Bm1uIHk;EZrpP9iDF~Bs?B=sg;(X+gd&Oa*5PScnT7NT=5h% z472$b=CBE2u%S}~X4-T@6sOe8CX+-#O@z*o4Hr5SBfQkzv56Zu6Cl``*a1}v{G`PVYQOCl4S1999gVdMd2yluCYDJ=YLt4&DOB=pG{VF$Ap6(S_ zLbv!?RW=v4i`_%VUuE^(w`56>;|<1lz-0Gs%-e+86f>PIN0Kd5h{=i|tDy;H!4_M` zV32(0%b3ivKwZc<#h&6w{KE=U#kWw)F1n*LSr}8RJ0kwL_240R3jl-R>AB!YZO>uY zv^uDP`tS=cU=P)Y81ZCEMto*Us+V1gBksqZJ0* zvy7m=Y5~VzGnoi8Gh|ViF<$cS#eF;X-!+Z&4utvu(8>oOl%b{RJ z=NMl`|AClkvy6!c$Iq{@Qt)T7#!{#~pC@_>huC=9b)C1|1#M5qULQx$?nv-|L`XE0 zfjMQM1@$wluDY2b3zE{%1LooAW4XGbp$KAls#XwSeNNu(-T~6;58@NQd+hKGYYvBn zQ^{WPq65OXm>Z7MIciv0ns00GA!gO2mS5XrOiXfwhYM^#!b=qzFGVlhW1mn(1-XL% ze@!l&IR)BsQWzkkHrW-L9CW$MCm~2uLzvAAvuThF{80~Tkqif6a-J5;5Xt`|C^TO3RPtxCWUr|u6h@n$)7tNuC9?A zG~9Yqg;)inPXnXH7=eWyqD5@&Fp3LNrO_PLqc-H>ms-nL7()5n)PxYLAeCE#X?l5+ z<{UlaSP1S@@5ZoEo;@j65a?plCoGd^?i`%_99GDk1Cy60KcD<^@~b-sl3kz(me&#Z?f?KdcH?{=BhD0NV%w&}kI3e0 z+SK1vhF8-j8|(c0*ShW|Fxl}2tn=;!1Xyx%zpgN!^1abU;lLW_=j_2L=Qec`{2~^z|e^zryP$6 zG5Dc)*r+^|@}kc$6Cjrm;uXQcMkY#8lPqSU3O5tO)~*Iyp3i zox#BzX#e%2zm2-7&++;ZDRyJtz|VsHuGqhHediO8=aN*472}yF5AW2SRA88E!KxVL z{H4)ExzJ3L0DLEomHGY(3`!dUu&v*%1SPVA7>>OgU|2`3A;+z4_AJ86H6y+WA|o+vwj?Y;e@MZ zc8a3AV`mN6=63G%o=tAeICrnVXK)1u^q1n5M&|DIul8&#pe^t+KQQp)-s8W~!mM4( zanG1aC_i$fHoZIcPnfLvF`{Oo2d7!=-sSDbTS|S3`AmHR_+>!}K&jZ1r8}wgt9I0`imQ7*`;Jn}MC&;D?W| z_tx2kX#WNHB?MV*#7vhm*)bP3X`TjKztp&BQU?7dya#MG>6?YxIk`$(nCz1nAApbr zHGGoQ+F3DLmKrrIj0?AD)GBNjzC9prXXf@H+rc`uC~TL!YntsC84F(91pTpZ$F|Feqh?V;T@>&b|1?9M_Y_M%%MEAA$k8 zo0$4yb;BEJ^B-!HV}ew{|2!Cgvf<9?{~Vh9YUQ~|ijhzrvPP2^?;M=GeCJ^BkY|pX z?VC#&D6!J2a*zFCphSd7MYz+@{bR5Z>%#;MQUmMc5%$&AEfx0FR>D~pbZn%53+pPd z7Po_Y|I$M^m#4gL4yGcHh8f{*UnC~n22=qfW(h@5R3Qq+mK-aA6gXQk$~lh{O1U+> zMwNwhwpQ9gaF(AauV|2>sX%Zpbga11X5yx z94+5=0@Rx3*f2>07M=6%pbdoOuuU~qJ4ttJ+qR9KvD2|_+v?c1 zZQHhO+Z|hbuD$lfxj(;QjxoP?)caHwrsA)-l~oBQKx9Pr5oR18%)IxWtweb|Aq94n zgQPrTENnY`%kl*(xS}x;QF2YPXiK26t5mC#!&3oFsk;xns9b-k|45s~3LnvUTLoE@ zmacg%5rYBO$MODR-GpLJb~g97x}9d5sL`mr&OLHOAbNa-fP35#^r8$Zexb(c{Mr#s zDw8ZDehLns9045a%q`?xdZEhnGbTf7Eb);b1El`Nq(IFMkE8pM`*-p8Q{O{f4A{)a zLs`nTFAMwUlyVFtjRfOPLp8KKmD8p9QpRwbCWbHPoc|jh4#JY6$55cKNPCHG(C)Ts z0gGYDxA#H8iR9K)g7^U$-ps?AZ27M_T!!A`sLPzQ;xpeJPBo)p85a{OEL@_-slzmK z&*JXakwrh-;%zU_^T!4-uo|$(8L}|hpfFRhVE~Z^YyQH0$&!#&+$b+ z4p5=ep2pEyhT42J$W~BohKvgF@D7;0~>D7pnzq8tl+p}a|8-E-hA@>xYBl1 ziq?&vk|1^E{h*r-r^;L`FZ)s_qkR>xX>YDdFKKT~c=jaakI&5Z;Co+{5|HOEl|(;^ zfXa2vP@S;s5#jV%br+J-M!) z?)+Nnaus-aJ+3=Dcvnp|V7ia#k@-AWDy!-4c0J=Rt4+*QCQUjXYomzvQj6*+g;GcBH>z#PGGswu)3dBiEHumS!OY?jcqurd?_4% zr^h~|{&6VyZ)fMsmHe^$mcq5OMnzOjMz7#flYemo?}ICM=9X_MeaUyMWaKC=#Lhm zV=X<c&r;5IPbBAxYR>if7zoR|t+_vbiSnquDbzh3=BLqsVQKJ7B1d-1C$7dObbEXO($M|5W{OX9qO*G&lozcO2W2tEn8c3&>e|Q+ zKXSLw-d@(7MwOBP&Rv6Ej?Qfr4{yT;%67}hLNJ!*Z3$d{s{}ze=j-CDY(=2jcy7v2 zoizhRTxm(`-*!dLPKSs7TLX>{Xo|#xt3Q|dR}K&SO1@F zexv@ycDvjN?jq`Ia!)r2g;2fjQIy9aN>^k*FPc4inz>er3955~ z_tM;OxN$PINLl&_JOfbKbg=gn*^5;>M7uDUIky0g1|yUka>cY+kW%A(q6mMDxI)xI zumzwlWcuy$W6H7{8+YQ(4QZF)ep`pe;!njio*Oc>wC?W!-Y%k1g! zG@vDBT89$_Gjpw{uQBueC?f%%jRd`-++@%|3iPLDsuumQ*F6{5@6Ur1YcbQ9{I#>v z%fKc5{Vk<$xaQ2>NS};gto8lVT(MeMnC_xMWI^1>@z6=lWcTL8d7pjHG~r)l?UjgE z7DyVh$qspJ2K$1a?PNpl4#rA8lM=$3+7ROC-8sf>>cZP7735K+DEty)YScv`XL#BR zmwB9WlaZ8p4WDjkBN6ZoVL_stnv7m!05My@k3><#11vp~MdH%l%#wM^h!Ch}?kK(K z#9;xRcw=tBfX!#uR8D*(n5%v?+_m026-*Iih}MBNxyowyGeu=yH6Y>F45-zLvn)nM z15)Z=JiV+{^=pkMR+Py1OL9m?DM8+C%GNiwZ^i7>_JTR5EWSVC#HACK3SAnpX*;m` zTz}{^N1ptCNNOR{Z2CtDwu%JV5#V?DzBIh+db#;thn0by{PVt7w%b!JaA@V6=|!b^ zYpf(2uzXm(6kC~PNU`ycWAi*CP5(RvJ$dvzIJ`V9$(+7^34w&IfuPW-2)iOHQ?_Q1 z&_*Vah#fegt%Ae2KGwvTBJ#TUi7%inb_PszV4c@Hq_UP67_2wPgK^&HWne*qsamKa z7KfZvh4J`W7UA>0*tbE>?@*HA^|lZMF6=_p3fzo>hi8J9a#MZ@5n~|%h~?_?<-K z9}D+Ao9P=G&#j^i#wkyv)92Snjuyq05_@%+$=iIs0g4X16k*}sw4rX7>&7Sxz1$qU z!T52@cRwR)PflT4|FZLN#g2E;y47u{{GGl0VSFeesDMwKh>Vr#d5HigflS)AX&?TX zj0cLHkOKq?KP>JZ4l!sLz>&mUb$T3crfEKQdO4j2L=!&^Hv|42Qgy|8E$fhK*{qrk z$Z0T#hQ<+jz(E%Xr3vv`a(aO1b!lI4Sg+NOkbVet%3h6K6SiRjJAo)N zX|cgOZM-=wLagic>_yQ`@|_cNWv%Sr8W8Pm(+528GhF&$iVp#bcQ7TmNJ_@)ptebL zOe_=g(i7*Ls--8)3B~}poWLe%j(pZRT*p@Wu-CuftH}l=2djqNdKN1iE^&tb$hNb^ zbQ;90SF?cP5F6nx=^zR6^8I6xn?X*}42Std3 zmOmFxEYtIVlY8@sJmI&KL=mr(iayswqb2(|(1%mWUq0$~ZoH{v9yC@r?NI**Ij zjd<>;P!&JAbBKb8S^G#D1x&f1mlveo)72XeQhN!Wub7~KsLxWPoVXJrbwkV-{m04H zj}l~9=OFIo*(DQiIPg;M5TQ6!$9HI0LiryZ1*vdOsvR z#2)03dZhO&?}keA2^Qgkp^N6@9*8w`Zy;dgFXss7IZ8J2=*iP*mhqxX{IhRrdJ@fqA81gl}Q+_68&moAH$MSy+ZD+0Q{zQ75j z>{*u2f7ImQxR_Zjo&D~qH6yJeXfen9<7;kaE8BzOTD$9S`n9ju=eFs!T-TSyjjW%1 ztqmC<5;9x^XwQ2i9s&N5g)5NmV2i#pid>P)I!^gvd4QOcmjscS2hs_;9VWy42arVt>6h=f{7mz z!yt7WH1NmlR-9aq^`UyE0K2hhl>3Ds1)`uJ9J@5DG(mWa%81s7U1wzgdp>zALEHJr zv6ke!-((|urPUwQBk9tk$84f$eXrevF`Vqb5J+)yxmstgsMmRHbHU^zrAGE)_{&q- zjXD_Oq#AgbCodtcb0JEe`8Hi`o{lmae0dl>y%^FzUp504Lh$k1n0Y*qr?R(s-oH9{ zky$;gp9%)#ctayFR)ADU7GBrOM3E&Mv_aE&Soyx3EG<+bF)4NYy_X`vNYH3PH<%0_ zFOTEPWLFo2BukGS_3J`+jPLsMmK;OR*FrbP#$oSoH-T>2CO|k;{UW8X#{+qyDV7Og zh8nw8e{h+#A&)Sq@l~Qd6_Ie_oij?HkOR!r?(M>XY8YFNx2|i5l^(rB_v1|fk69yyya$e~rHXD^QegN04&m;lscnRx z=?~6N=6~sg>A^AAms@bC&eybGJmRoGngl()gx?WDywHYN&H_+>GFgOQPt=fD%% zUH$Y}zzRqnL50y5#S5aQYs`nir&1@2sWeQR)Z~GZF_1kx3!#lKaGYUE0Uv51 zxJ+RnnxMqNj+jdlY6V%QCVX80jgql9X_uzZo z|4)ZWYlL;}Ye-SE5A^kq0G`1J0-cci*ZzLGV|)UMWT{tbAY4K~q#Vk|UCBV^?(84c zC}Fvb%8_{*1vMAWyZ)Dssr~Kg<7J!&3@nShk2i55M14}u_uTipx~U1R%5Yebef$<4 z)6x85(E;m+Z^-V|_jOTve{;E2CW#`E6;WIx{fd8y=t`b(per|s?Ps?V*B z<|o_D_3f>r@L&;JzMZA*cZ`>S@E+$73F{V_04`slL}G3ns!xxXtJ>Kd4RsoCMO|N2 zhV!lEwl;oO408GAY=^Zc!nt_9vk3@W02z2K zj=51B?h!+_XgYj8Ua*0fYtA@AHkgQP7`h+B zD2$eXBmrE5g#(wQ))uYRXpNN=e26OBv>4>(ABrRXTRAOt7sfU-ae9tDQ<%hWh$L?g zypO~bK~c)$Yr%7BHmoMn7D4)OWl3F-zeRxHis>ZvT{PVb^&@YU)QLU3s8*=zBKzC6 zA-LnyIIoyN&sA45M!KIL($H8Xu%uhKyl^hCURnQ25I+@wFx( zYDPxgej(1fe!^v~R_l2dT2l%KxP2NlApzgl#)euwp2~LIfWyo)@3RUNS7c}DL}og1 z#t6f=jkEn!3x_li`6F_l!8He?no3xH229`8UpT`PNGgVXWYobk^4ACramSvqRq;5U zm_<-l!Z6Qq5qbWbdcpeQ}TAYziK<(&Z zsQ+xiA@AcpXJ+1nA`!@EFL)Qo7>_kSC+{d7K1;dKO=ZeS>~y2W^6@4UmC4!w%EtkV zi(_`nc{bJaf){(Kgc8&#+Bd3qmDF-1%+IS78sWLa(*piv^*tOtS2@mr}`<4oA=Y9{#OG7u}|zphe|!$ zh(yDsKX({ZuPf6#p6?a!6!daA?yo=uDQAQ~tGnIyd}?4}fqr7XdFab#~jpjnLnE%@5~FD2iPaCgr51w2^JqVFQs#vChV*vGqsM-cG4S<|RzOuL=3jsJ;p_x0JrAs$GB_L$DwE=!H~_-zkTf z<{?#S(oCO*^z)d(LLH}XxTv!Qv(AuCPbERG2q+CNtMQx5KeI!@2EMEU@biW`>%3zi zK}CATr+t-$le4oRRGUmkC=r@#QhCyVIh2EsS&FCsW(%ezFn08qkKxSU%IQje2EV*- zXstPt3A+y1ZSGr2ofZrSCIrnDBM19cTnbOGs;mFrT!rq^Bmxcxd5&F{b#T=3fH)zD zfE94@4!wEHrBH5MaVP*9&SQCxO`E*otaYW5m_GqinWR6iCi>7@`8=hbRDK&bIq>rM zsVNKc@Gl2OI-1{GG+79aaVFUc=aqT6{M%&m)s82a%Bvr7QBkEfn3x?aXYfrnXP!>Cs0r$~~0ROog$g1Upm%tl|T&}JhGrgY#sn<&eb zc0f4vC|d@b`LHh6??&Iui9H0aXqdgG%7B04Bg;Gub}xt^ivMGnv&J0_i_e>Oq?IRtSlGT{+qec*Tr%^EWdZ zdo96XB$MJ-W(yIjeqUxVzL`uP3Xxq@79=e4U>r!&j^ z0jWX{C*)uHlh>ssaYg#xMMXPSf>&+GrVgR|v=##e+d2<&3_qg`mjZ_C$mXJ9>TYCW zRN~@{kZa>aYpVkgp#b6XOOo+Y%B=M>#MogcYohs4F4nc$XwsPz2S7^ccM4_E_$WV% z@ejyZGU~sO#E=^xK{mnciYm5S9%)~f{Jj8*(a+*#-Yic7UL($#-?uD(f-wUp-1?hf z{uYvHf%((frO%YgcD=S)K=CdKxXA-5Z`fG-(R~Kc`6lZZ-t@c$(UE7Cln0HkBr8O~ zJ(QkMXZWjW=E4RY9hTJEAUfj^8cG{h>rhy1b24B5RppEf|I_W1Z^6Y3qc|Quhzjg2 z`lxcpq?u+6XsIdJHIi_6HKT!CT*=0kWlcCL3v!zW3*e*TBy9>;5${y>BAqvLYBO@} zKESx{>2W=dC@6J5zk*Q=;ZJRUa=7u2TZiK1A{JLTUHxTt`AHiy2szG}3Js^Enc&H| z?u=*6d>erp6`MHp`GroOTY9SVG+6Oy91Vd9!Mw@K_-6$Is{v3ki*$ZX4>Pll z7W-pf9w3{@AejEt1G$0((*lRI#l*C^i-e}dhm$e>+x&b?A?x^l(e5*?iT(f@S8Y?; zft+23P!#68SF~?zZVp}f> z^6_po6p`Q~oOXH0(Lo@8|FrPcAaJ$wy)c2{Ft(KbPcBjF3u*0HLkcnu z+Ip_V_wfJOhM1+-8iaY(i-$i;=Uf$PAWj{nK`j@FX0P>&et944PG{M~3!5g#_aO;x zNL!&e&Mb0#r*hv+EoRutb@{mH?e1{sc<$F##;b5X9z!gM^<;B>{(HFa1OsdT?-GRJ zKonC*?$=fkF-FPW%^iJbnd{6Zk3(ZLKD{z};b}d0S|z+d-E;$+6U9N{xjG(&y`Uk` z63LV%wJaX9)9P&S0lqq=y^Oe&lGTg^AD@aCmQ-T>LeBPUN{@tnz8x&LlCvu}Wp$HB zrpXEhn}{6S|F+kxjNv?Bq?$qia zwrt!ktP7hOi32k+bl6Q2X9W~8Pey?(JY2dP2AkG%911V80?l{aPkJ!m!paZ4Z6J42B`cjYtJpZ;C=s985d3L0j9WV5O>waX!c%@2~lC?|Sz# zm>AFM2ps89(TFA6pkr6SoH~cuMGc72KkctQ4T$ki82i2yJ5&^6fJmT>^4tL>h+(97Z&K>_NKP zg;S#VY76BP&>tc}X8D@{^wIf4yknKWRO-HOqc&EIns;m>d6FOlkMr#mt?6)`JcCpHJbQ`W$^(f6$f87ZR#>3~nhKHQV?Pls^K6& zcgi7fiQmykP$i^HTGhL(wwJyUX0Fpcj^C$K?;|^6Q_tVGE1nPtTN|FaK8-xk z6oME7k6tD-ktY7Xd*(mG{hVXkiRD8KvN0?t9_ zYyB82`TygkiBiY}mRb4D%o1nV!572dCW=SB@AZX?Es~)HM?l|9I5ERYZCxEq)g0@h z=tm^Lm|{Wxc;V+#I|v{r8sgun)gS13B^laP!GsIA5hrnsmJi7_iLg5Xfi=H)o1V^2 z+I>|2uzTzaoOb4#^-q|#8Z0!#O5=yFkq#XhXQSSkAP9V$JVSY=zs`BxKb_&&$!vQ) zA`usVb7f5enO%%I3zZtu`evVBxSokKm%-!YaNgbTQWfZNJ)iBmsF{hnx_eBk=wtB} zmPaM6rSCmW;8lJi<3}Fj5Ur0brBndM`-wpMI9j~*qq#v}P$er9sUX>VIO#s&8?3Jt z*Do!el!J+T4>DWq=+s4`t4-yECb90sdgCovA>Q)T!|GGWV2xk2>fM%91ep9rVl7Ro z3M@Al6A5_28>|mAC%k4n!ALEH%twZJB^wPl^-HQv_%Osk*;g1~8Ssoa3>|a$#Yhj3 zKT+)QI$#d0k(LNYB3h?R(~K0~9z;D9lF2mQ`yKa7onDtbDr+=irK?3J$Kz=8Aw=xF z6(uYjv1(Kj@1etGJtW=?h!h&l>4!jgG;>;+m9FYeem43+xAT9QXRlWVN0{G&D>oDy z3^81<*#`G>3#Aks1LLcH~^IMds%Al#M%Q-{k}%fw~BNMH_k&$B{@lN`0sxT|@P~Xjt}Z z{dZ~_2xNK-j5p{$9ErHy^Ma#LbdXFjoO|t!QQ(U-Bna&xH7xB!VV;IG-_m|wk;WFQ zQbrc*x*vo-uUUo!OEpI)Lbpx(&Y;GPSGC2UQqd9ltFBEsq4EokS4-Qa9=j@b^Ttg) z=}N}nJZ7dPSeV&8z8j?s<$M`aW6Azx?+@a83Bk>zQII?uNduSk$@*3yRJId6gjqXC z_|#Xdd$%XBgjLW`Cg6{~0)I>z3|CR*E9XlzgLr9Pn|HfOGsDB)xnRXNwL^O5=T}%E z!NoO;M%|GQH2=Lk?)|7Z-S5vhiMs#w<=0w?b0ns4j>*#Bg+$_#gfqM;n}`;3NZGHwtj^rxKAxuFaxv{$x` zpl3ehlBd(S9Hvt`BC1vlV0Xe_&Csd9Bd75Oe5KnmN~p^@8rEI8_lqo?om zJb*fyg|R%vLYCGzj#P|&FcuNDpTyZ(fTVMae2#Mq&>@mS^OCWAsL)>~6KueYj|Yd$SOGoh!ab1Krw#I?8UC)Nw7Dj#30% zKOgZsx!S-C8Lm4(2h%Qrn=X6W@IF$0V~|%FwNc1z7cauIh0zP%hvPei3B-D1vIj-> zq+xPb75j9gE2pZFcZ$v`)y_TQEx^>Azk>+-k(b?J4AmvZo1fF3NFmO1@4atr{4!{M zgLFKDcE-)%O0cg<)A;Jj@osgz%o6?VM4#A*A-tY8Q_`cO_UF>8IsAp!8Y=H}q;MnZ zDMR;7K>oO6i=l$hwe2~kfN{aUj+LFpi{ri@jGUgDr@|i7+{X0)19cmE?)l%BNV0-x z^lro>e?T&ZTElQHonurbMvb~8=Bw+m4PKYnt@7_^wdKv-B$6Kxbzj}idvOF}qMScX zF)Xq@4n`;A7|vC%L1X)-R|wA_(x4WUcjlYj{M4bxm0Z$_inM#giW>h*s&@< z+ttv2gi{XEtPU>|Q^HLA%oxZcA)U}Bs-045lvRt++a-`iwIg!k^?3#8CY7v6GVQ<* zIna()cm7*uEq41wymN9z%OHpA6v=wL^7kA9CEnP41ri+bj$3S+Et(Cs0HD2<5gru2 ziR!NL8{P}5HjT~-eHM~sfhOv$e&P|Y7IzvmbBREHfc#j?bEcQm^~vec9kv3wH{SOz zioaT%z!i>O!jInd%pI^=rW2l|VofbEiS@bTaT)#GWSJ{smZXcLA9M^UEpvp@;1)4e zKr$k>{Fi70^|Kn3;!f;c;tsfSwAjF`eY(9@z4I+e1@E&P^Y znKpU`TRwM+ROMv7oPtRbi+RKxIF6n9-k$}dM=0a+Od_7VWJ|y9EI1xlU|K!@TG*A! zW;u)JOh3{yhI4~{icpX(ACGIlCZk?MY^U$n&7xn1Kunj*_bS`#*Y--WuBV~i-vu7t z^YpZqX<6^rM_ZNA$Bd#nJYUPVQ%}VdhTP+{0o8~A$nJmK8}Ef}0{iuwlJ%6MbE0k3 zqk^-sPO_76{ou^?t|PMMkEkK_wBin2^n=i96M9^$g_DYtv&MEwbJWR^dft>n-OV+F5F)Sf9(}TDbU5`FwrR}&jyR95lgDXe4T3HTI>&ES0Sta ztrT_wj>0++&`1@ZFqGz@xaMA*>z%gWJ-(7EJ`l1uk&p7)o&wMHJ^G5%e>Z!B(L zoSz!2k`gHZT6b~Ea%{gGk8iLzx|{4ZeN8>Tv!f=#gADcb&NHM~N@{il-pW)#USHli z*KY|R=~KUs-xe<@_%NiqSUh}pE*2mdmb7?Yqd;};cRGJwD6FeQJtX!@^#84=9o)}b zG2FPGdq{gOruS*zw=lc@Jjh|G$xeNJSzt8p;R|#vj8zf|7dFT?R?#r-E+J^9E0Ul4 zQ$JnL9ncZ$Rll=xh1tgPlB9Rq@#rHL4-5T z*0JxH;iXGwPl@LjcgXB?2u#ZqO1OO>%r;A#xbGFTJNi8$n+~hVKtB|*@*`J~0-HY3{7WIKWj zF^BHY0uSgKbdfeJs`s)P6oNH9X}p1C5X~ zVYo&n(HL*HgORLrkj=aAyk{>}CEQedBf4Zc1=vB?v+Gcnl5Ru<#HQ_d?5zeVb-yq? zSyi^i@T2I$RHemBCnO&>_idE2ybT=YY?|_)3S|xYb3>z%tm@$5_x@Z@BD*pBwyetF zSjg>{wT#z$jNcLefxEB)sBTX^LTT1Biiha$F(Go=EP?v2X?V3sLSuMHM7I=K>4b!s zW1B=r753>}?E>>t<+$1qeeRBi$ z^_$2A(s0zBux69|dow1&Y*!@cydY!m_J0I$f%=S#*`SJ%HBz$v!*35i4FXBn)Y0lT) z3E#JFE|xEMwv)oj1E6|LRN^p!sQ)@(ee2>R&inZdsU`jH-=pi^iD>ZLamNZn3n3M9 z7=vbgm|e=egT3yH79$iulGVzG!*&$=+ar|2YVG&r)+6gPTG`y7bse%Eu#!sA^M}!jpw5^wX+gMCo3E zCMZ{1jw>0!%ncwv!fudVMx@=+!aDy}GM(z!UibGIs(96I^ zDl~J4PC9#usrLFF8stGyC9js<%xtyu{J!F_1;&35Kd-3j6c%NqVoRV@GzDOGfPpdJ z-d=s8q)>Ecw6Zd>+gtA!i}8mk7bUckQpO`DP4H>E_gsZ-;`%F9kb*&Sg|zRrgu`Y= zZx=I@rgKYoW;pN%YLx7t+9LgkFilI_vw|xgcJtn5&w@v>Fj$o znU9Mje}w9IgQmFLF2?u?_xyb0{($Bh7WSe4k^C}>H0q@_0&S9Skg-kC7q5vE!#~O+ zOS+sfMd`5zecGZup+5U#Iq~Rc0mcNpirNPN&~(==sR4@@KBfn+%Bh1?By zBOgQRF~Cv}ZlDzp`ge5B|>oKXm@*nGPz6_$2ND20e~SLUTsqBEHhXbCe2 zy*vOF=O0eY`*5!boyWi61XX*0{5W2w`6UV*O;5kd2s$IXd`dR-Ue6>list7kq42YK zv3Qw#u;-=ivPfAtcvtbt-|CgHE~?@CH{m9bVW3rNz06||$(lBrnRk9LaG;%3`v<=L zAM=R>z1p44ga1>P_5Q!qWp4M^>5CiJ?=00n_lF#e@iXKM;0RzJ=4ME3@e1o9s5=|6 zs7Qz5cV&cCcpOKZew5AD2Dz+tvyPEPSVB0P(vDa1U9HQ`%B@@Mh%6ynlF4r=U1Mi` z+J1Yx+idcoWB!G1p1p3HjP<$#(vcLp<9ln=D&DeD(Nm~jdp~E&uzZt(#S^NOX}O0N zn3lqsvxs^?8X;75L^9p}^{o^jeBJ8b?Zp}PXz|7150$lvBufFpWpGCAxE~$_1#`dk5`AQ2!Pkq z{vpK@!$e>c19N4Kq@W=OFVp)KSGtnp<9cMLRgkU!?$O zSAecGmr1n^R9;;nE>b>PAw(3Rpu6o;9m4rq8t1SgAY~8#%EisrVM|b*7nTum_ItOl zAkAP0fI)0ot+)(*k7(`0to}f|9PGd;+y&B8btwM>1!k1g3#`B&drh1xGv-1sp6bv# zyvak<`7kwCCDDi7Xr1_Yhy|l$yMy)qDaAR_TvGTCF)X>!b-lCf#C5<9&hvxRR_(om zq6f=bU~CWB>_fRertT%e{XGcdK$M@S-@n9oYARLiXELXtQ4XsiQ}%I%%FQqv?beY- zdxNjQ&91@?ccO+yyQEU6O6=0x^3Rr&h1Fx^`Yc7{wg*-8Yf9t^-vk-K@*~hl%2r?~E$V3oPsBLLU#OvEjF4-!Zh^GwW!0SHFLuxX; z?9NGZxjjFGNU3i_iOcQ=gsyW#Y_&7^KItn>A9k}42s*ui2ZJNqjjPiwi@(**=;;M+ zk(bBA9%wb`1Crv`J@31{JO~JQ9yi18GZsk|C6^3gE`p(K(0W@B`8BMI&pRcN=#6Q| ze@0%Cj>JX;%~{Hog8ZxV9MPSpBk|FbjyQtC?8`N|ROqq2Ls~YA8Lxzz1Dq}7tL6Un z7XPmeDxmsxq#JL;gTr=haU~|*_?NSM>i4Z2%b`70S{`7159(xZTQ4A&)&|!u{yb4=CJ=A8kU>)pfCcbKUn>J9qlbB zZ}Gv*tKG7VClKV;K+7qs|B&za{MLaz{%4X-m5EUSC#~4eAj=s4fW8(QOHNdnNF#2y z4m~OS^+&iwaM`qiJ|>|hq9Dj`R6sf5t9r!Wn{vGCC|JRT!OEm!4CwSgLe&Q(D785BRJ0^6@^SerR0;V#<%)NUB)fcI`4?en0FRA7 zn5N>>0%vJ}zgAxga$&~OB2R%L!A|De-9=dsemb{{{`u_c!6m6%Idr|&q&C7DO>5G9`tsnd$WyOxxA1qSh=zdT8dcY&Jw#7KzLl`f!50(yEH{Z zeVbmN1>h}bJQgyIpUkKaDg;FJY4o(#0tLDBtM za@6xWqjmd^UAG1xn?diP*s2t)4-?Rj&(m@aDJg%-(B~1qi;A@?c4V&dO(ZvrjErZwUcD86_ zBw@!}1M0mE0#8p>Q1sC;4#5SgNBs3OCvu}nU+uF~fxfWHKiQu{`)5#gY6bm$cBSHi zjs4I2JOYq^UcU-)2-oY3M|yjoDZUBM1HNxRC2rpnc*G@3hG(BCSd2atNo1rwgG9jE zskG7&s)=1A3p2NLQdUXkJ(EZOh22Zvn^kK0yiz0%yIUA${UbLIoXg3`{&rld=8(y{ zLeuo5V7Gf4Gwb5nk@#{iaR#&AO8&&N;AmPtK^#+ru|d>|EJGq1cXSp3NYj>_*kqPO zRNM|fF}KW^X7z7fyu%treeY+-D?X2h%n+E!4>%URPt>n>&ROI!R{}w||RiPZ{W(=AEv|D8Vu*d{-Bw0c# zDT%XEvoo3u|q$xQ(Mw|wIhp<3(cixU~iDDO>>2qGoQb+@dbU+Y6J&{O>#El>a zG>7tTS=57TD1VaBr}}(dneRk#RmQJC_B>XW?E1LjAK2**RxPf-wc$_5`-sDp>c!S~Bafn3B%@-^+n0lTErx-gS$Z;d%MgA5e4zB!)b*dJ zzIaKhc)l$Ufx-)gDRn5*hYON!GZsq?J2m&F@zpn~z>Nu=G3V2YTaf5Z2SAfJj|ig; zx{=(DL`YzxO9E;bsF{f3DWCaXkNF%-!R5ialcMIN7@YfgCTm&$FO8GXp!a52&M?rC z>bERScTOOT9{QvpkVw5}_bheKC^PPRn1dvj<9Bv3y}j^^Si+E(fTt$`S3VgM#v|Q6sd6mMXwh;#F~F)sWixC~ zSY0}FY-0XWMIygz8u+!Z(RDZSFL8(n3(t3?f$Ty&QKa#3UH)J3-|B+d3A3RI0zEp) z)vrI9T-+3W<@)8-4@Bw5ZI+ZgC&aS>xx4gdi{;O4Zm|LLmZB?gtHO~P2whniM`h63 zCXcHvOqtv6-C1}x4d@Whr7kE=MvG26>w}lNys?yQ^X_e zTfto-X7RA>YQul({um&kb9cYcm?hU>Rx*A}dgacWl#WhiXkD8LH5+2H>I&8RR1}{4 zV&5C8>m%;Fd3*Kysfx&#lZoL+26c+VATKT+HHbC+tQ_%gu8HrDsgvx6Ol!W}#wkvQk)H}^=JE>GYKJe*WMX{i)h1o$K`xg}Y8o`SE_{oBQ< zysF5380X4>(|nTub-7h1hdH0sVR#aCPJ1FVV_kJ4$|n|Py}RQNAgdLCX0?rUK};{- z$FmBjKKfVc(@P%VQ4ec9XjT!?yD3B{Sx+d^mIj+Yqv$eAJzS0vJecf-4eVmQL4QNf z#Wn}HkIY`8uO9YR27mW)=V4zWb{<%RIHxWiyW>y|2WPvDsjJi&YLo^bW?;$nfq3~= zYg-302-i!?71&tyI{)MS>6-G#GSEV(lqG_1Ux}{DWa--?8cI^+$7IVuq3MTGus}+> zgk8F)7u>@q21D1+;bQm*#;LGpEmdgB^zV1qs;bfvqE0fZv7<2?E8@tT7ZKPF^7=9Q zRr)QS)kVAiM4j9}6;+$&bSTk&fW9Ia$v@-|xaZIrWvtHcrKE?RdbQo_hQ>uBF|n4P zSn(Nq`rGKbmvwwr$yQ2Nvd%x(J3sCOh#p4fI?ZE4xh~K(lVEWS`tJ5pG zK4?AUj_|>ov{UjRKD1AU(Bye}pSK_R5>*)+AAw!mG=0mCR_GlhvG1AH@`?W+gNJU8 zNz5tX2WiBWjxR4o20S!X+J#N^ZPW|)KL`Ps*6?_Lv+49^my0}pXy_&FoT%`kEVzysY1pDC8Y!cSl`$@{Go#Pl(# zVsFlHk%2H31`xtpYF?+{!#NYaDxi0PH3t{jK;NKKS%Sy8fe(;^f+DM>0skGqGtT$X z90AfWyi2ue9{7+t>HBX4y%;@oFLGk;^O(H8osH154jj3w4zoroKE0oLf+yzm{aLC zyC6jv5N!+vfA?Hw@y3OJ6|L4zCxBpOA|K3{FG((7+{=lvY<2sboEhnu1J5i9D#5?5 z*&~8-xWT7ZH_%T-Mf?!8GUB4=owZ#z3EMD{7Sq!q`RBsmR}4z-7tG4dt*JMLaBo$T z^#R#a{ky@0X1he63;rCiHzeLXW0_Fa%a6FoLv$!6GWVrg*{;EoP&$IPw^&X4PB zPoX87#20+yJEm8zKY9HW&S*tz#<*h08`Z4YHsgIuNWclQmkr2^}Xli9vEh zZDM`WiOqF>qx>HL$v`&0Jg=A0MyA_NWgVJ8IxC_U|NTcT(~k$Bv)ID$agAo+Zj~>c zqRHnfU&5wykDi|YXYM{4Q?b@~AAXV+`A_ihB{1ujh+&m`H;XB_u^@ELR}~L|g1!rtE*}#u|_5z`NS+%niC}WK_G&g<@#D zT{S?`px{I;lp5m8SkKv}MR|Cn1SWQv0Lh{tCL@mW)?rN0OKF*OqkeY%I3CwHVPr^j zku|L1fK^n!*C0&rDVY5Hn47%m|YeF`qxH~r;=`-+>dx3~hWkP^>6fA4(k z;rD+F00Yjw_)QxYJCfqjRYxm#bIZR;`047BMFLmO9z>9Y4t$N{%TWo8tRX~U=t zRD?H7MWe*D=ot3TtkBehg` zXEl^gxzw0?NrvdLA_EXp1SJ_Qlnrxo!PZSnBSt75Y0>E&CkIYn0h50XT7yC99|IV)cag~5K$?EVr7 zg{>MRU}q1~SqmGjGtnkySox6SULr4nO*BWI+i|Tem#)I?JR*nkSb3-1ZLQ1Ug+jVzR4St$cdR zQ7s&*)EXoSA*D2haifNL|8il<@^BgcH1HqNHLevLaM3azvQR2u?@mx(k$k^5?Xi8?cdVX076&4fyXI# z6^GucWIzK82W1)BLW=%1HZhq5!$y!esdmiW%s5@2-<)l4Xv9S!(H<$r5|NHaYYxkX zmt-sQg7I_{S9(;`Kd-3&^xHFUesKTDuDf4seO&#A<9>{k8>~lR4qqUTl7bKHl1Z>1efqb(Mk|fL0SjcN!jmx%+a4mB#~w{R&if zRiH|WYsyr-izuk+P2pG11wnm=nx|uw7@R$S3`QuSzKp5nb!u5eN4ye<1No>_=p`64 zubwU1W|3%F_|(eUEmk@WEMn=EqV&|-<^`7E1v&Vehv9w8M2cYpm#00<_Bwe^?l`v zWpV!`@oz=Fz;tYG#ARZn)2?N%p1>u0&fkisgrwN$;ER+KH(Ga&oWA$g>yBmuo8fCF za!gQCffKlviSN&#uo1eORu+Y1EO55%@T_z>67|X}+qVysVoAdm>D|Qio>7AkWRR?_ zlI%p|fsqlA#1)SA*#F}21pER0vS+S4?{J>5=S;K#^b?IB?!C&wdXUk`N`shmoEQ_c z`%jODl%5z70`wDpFdJXd$l3F$V${>V58?i}ppk}tE zAIdT){!L8+gQvqQF-3|c7fOd06QaOd6Uu@ZmnY`!H73H(zjSeODsLmEh1S7yA_3Up zSO(@Gm&Ez!Q5HEnfYemAwHz&gxd~v7on*KiVKWrc*08JN%WK8LuHw&2E%~hhXUZV7Jq( zLc=?KL%IQ8iFxX1k0d8rN*Ekp5YeQNj;w6A$JybE(r7UglFwOA3sRK7?w<8ZL}w(- zX-|m>aj8*AB1_wi*8LC9&0PC&=DUj_TGxswq=H-5%ODFkgMU7L{Y*F7WSh90q2ZCM ziJTZiZ0fomiIf|&dtaQn|M8Lg$HQaHc3?_-_kwh-#?-_b&Nv-*=cfr^FeX~32W(#7 zlpe4d7pQ4D?0=n35b%$~5XM=GHh|WKhB!A*GaOx;`@81D(Y4vuci>D{Jy9S9)?g|j zNRlXsF3)dI>&|xzG`kJ(Rw807p`Y1t+mvLU7boXsWSXh|kQH4p)KTyZg51tjwM-&{j1QNY;e`VwPQ4Lt zN;8VWfloLbI0V?}xMvN#0OLJd$QUOp>lxtwQ=t?jFX{0S!Fau$d)#9?cZs)i7n|4} zm*c9&6>+BHLO0lXdN_JGpurud5j&1c$H24*);JAO#}H#eG8MQdcrN=JG1a&) z46b@6CQ4Bfq^68)&1)YfP&GF(3!{FbFT#w$bvkvbvwef`0@LB?x*T}^lLRHsA`)v# zT3MDY?~2SU>s*<9nUw-WlRDM!PU2)Cv3OJfys*koa|R8}SJerHUrY#zCQ28>_C93| z!s@FLWARN`8;Jo6C^36rzV%<$QT=FYZ}nXDTJ>!8QgyO=)uq(=trnFycT2iG!owvt zd+nrJo@l%R$-lRvgELrKNHq1H@SF)bn?IN>NVT-nAPxldze%^UcYbgCE?x`gv&KREcx4 zi0yGoERp%lq+rPfFiKm_VS&5_k}QZ#sf>{j%`5G__Rh?q?|+^AcIMm3yI+48rran( zT#CX+UH#ZB5MzvLTCFS@)PV4OI}+zTh@!0XKq+E1Z#a$sh&oR7sOb}!3H2vzwY9VL9W6gHbT6`+ceGRja z#ZW&%1H4OmfSkfXV!|p>hd`>kxg^hSuDB?~mEMA1=OF zy#mHBeB^5N0-D+l|4hOWxY5+^skc$}EdSsn>>Yxm7f|&m+-^_x!qhG}4gbRTU9O&< z+STGE=-RTitKlW++G3lgGq&)veeCP_<}EqBqYQrv%aBy!YDRXio;Ic*(tos|bKaeJLWm_I;?Q||9kvas+FE#G!kdgV8bDDW$-vl7pS$@w z{Mmc+AgXOne+5H2>l@sL4w7Qb)z&BGv9FKBnLOm+Zti6g5hA)xRcvb^68!^GX(646 z=qu~Rc~oO&8%+v3fr%_d+1_qUcETO>a(MNU5@*UtrK19s21T2Mr#{GvkV-`U(*{7Y zATA~irEQ$VsT7h@WBeg1R;?-CyjV4hi@a8KEh*4q9*0+}3Q)CbUJ`|h8p^-#962!a z)kh%)iIkWiOHr~&J^(9WYv)0bX-yG@WYl%Um4vNjSjNDrMM7KrmAH@;qs6~grGUqH zmBJ7nvqRxf1vF$KB{d3&pS;5A$JNtQyPZ(M1;^2TffwObuT{^$9#6ZgA9o@cS9!o< zYJWGH+B>zU7Qm=oF!grF;Co6ThU4(_cTeqs(erU$aJDYNHjMPAsa>-tckwYh(Uh~* zYrZjFjSZ8I;|y~}q{Nuy^6E&Qkxqcx(5T^gZVWso3h_k7DOuRuJ1{gc@TiYAGRsb% zl0Vvn)6NZGKD|`FFo@H*MCFuqmGc_uT;amg5YE-MQNTJ{&x}G5`_DS%!wW`+^O&~ni4_W1(Kg=5#{d|o$Ezv8NN(L<3&${f)hh^&!da(q*pFbeqNNlhR zCbkOLB_VQ436U3sLK0GKrKM~kDk;r1rdL;zMW!xngS7;$SeMIzNj;8XaX=>1bN=bS z2N=_e*ihZ`Rco$AaRI8uI4|{=Q;}&@j~(Q3v5ltm?K`Rgl0*v%{pAb^v9L+4kb9p4 zE-4`uoRxH$1Vk}BineOIJAsrK6Vg&6FNWt{OMM&GY~3`3R&Ckx;t(2Kgq{3It8=|yXZ zPR20GV3TCgFR#8gd!WZT(2YjnOA!CP&N;r<vW5E|KpY+*8IZ{ zi^zZ4^krsI6>Vk~wp{fWzWad0ooZOAF(qr%E4Rqaga0@*eP;R;{CytXJc#)I|9tt6 zL%-}gG<|OR)b!bZ96~qu-#iG{eE*L_i_8LsAN>Q2;r#Ten+JU3v=*$hY#y5+=v1sD zl&I??36JF&abEWtPechpP6+DynM~zMxpIYvp{O<%IJ2S6Wlo9<>EO&vEB_s>Qkl+* zW$qg?ifCt^TC;;*R1?}Zq7@tpM}H8k*URHNc8Nt07q+Aba_}^{Wsm|i+xC)->i($~ zI+gwt=XI{l##E#bHY$WUV!%ofy6Z>+13D47MG?$vLw&=@RGHzlvk#^t)^wwuonRt6 zYG7^O*>2Fqy{jT)?o1~;?{ z8EzeF3K?$IDgV$A$d}XI_Q~^SB*f%=IOe#lU1BrGZVg|o6O&UO5AUS9i411e-mgXq zjcx0?=3~{hje_l>n-0>mkR=j=n2rf@N)V-lJF@7Pc`(ik`{br*@3IV54OA|n;;l{Ne8TT{Ayjxc|2=!sL65^C~hW5&3aoM!SBT=o&G)(VEH+MwYZsC>Th7l(--% zQEL^~3X`>U)z-r7NfqaPKod9xWU^vNLc;T@79# z+YxC>3(4SZIhN8K$Jd-JEp(&J#2CXy`;|li)*KliX{Sxf62b$URD&x`q~?vX$%4?P zcLoqn)#%-Avz}>%nncfYrfnqHv}?iDR=%2bo0@zz>-6k1!g0C*>sgc~IRj@Mzicay zOj9TDZN#*A>#CfSD>s+#&HciH7?%Z6k?Xif_2ycV$L3lxFJh6KnCyL3l>c>V@6;}* zBzzZVsO40yHQLsxcb(euv(-u7%*CN|y;MCt^=e0p$DfM3vwEs}p?anI6O7;*jPx|8 z>pfgO%d5t#KTW-X;LJ7T5d7|ffuE^fKw>(U_8&OVGDtTIuBFI-yEK{im24v?d>JGL zX|x-yGR7(B4B-(2$-MYQj0a`|TrM~POyQ+($-MVqn~u08f(X~^v``%hGA~R@DS-e3?TL6l`z*FI;*ds7VP;U*%O%V4`pZ?@@%VD3VG(JJLVM06)Tx@c z(0_N05yP~IS|Iv=cXh+eL-GWuUk^XTZPSHql?tZUcL*v4pwT%m#1eMuJL zk^T8Ep`u2p)~A#Z(os{qSqCYxRog_S3i^wtVIwRbsgxFC&3cd5&TA|G9c`ed#Ummm z-h^2PWemomF#}b;)6p2k|KCT~_aa!i=qOcThjRCvUCP$d$QDzIXlK?FfDEo27b~#lOb@D7{+Z&W|QAjo!Kg&_P=3?G$_nT}#N(ZRKg+#20 zZ$|UVH~0 z`uDfJAPkms*mSFTkJ0iLhDQ-%$+i+imx$2dpO5F6!N$8$Kf8V$kAqTI4yV*x=FOwh zP3?tLuM7#WD6OVik>w0L3e!Lb8&p1}m4;asK^ur(_ODWJ#$1^oZn5xPRVDNGPRXYH zTwejYW7P|@-6$s=|Hm84IV>l}+7HXsNh~~aSjfC;A>Bmh?5B)m6E9-m495iP$U%b` zJt{7iFwN;a9*%Zu-G~RG;o-&(ob8Z>N9Z zvanu@q@2ZiHyXCbHCqkS)gUW^)LzB*Qc08Fl!JkEtd1mBSFW9Hk7y)?cp}P^UkIvu zpZ#?I!wbJ&eB;i+)4s9lR9>mfo882kii3U8_hdovXUUyg0Pt)lVr^nH>wne~V{w6( zLWHGN-C%?wss<-h$41lAT9 z)?5|6QhmvgJTDoFETo(IJ^Fd|AJtiIAu29$#Vh;F+ceu;gPW`1eH5yi8<2@cnSrv@ zRC!-`rmx(!$6*= z!#@6Xps{V={%BSl%3}kLt=o)9)}ENRH!=b$oi%%*fL1spSi{>UhPRE3Jf(g^!Kcuz z@Z^z&wL%@pM~8YPt{Wr-p0^_V26GM7G>VHZ*LGc{i+fb*6yC2xPDfq97cLdDsG4W+ ze2CSdoYDYM2*oLdh_~a>OQ7m$r)L>YCwhDFIaRBlR%{xqzG~o<=C=xE>w1_NUNOakaHo>GL;mRre!K} zEx3XD_3Gi7qp#h+^Z{CN=i5VfUVFnoYf`5i41iawe+Fgkk_HDWU!pA5 zO=Rl`mr6wAZCHyeTlr$KHDTRA&=y7!(a{cU2PoQlY!}v4%b?|B7U;)@0fRNUYoJs+%MTCav6(O8xzmX9 zd?i)0xgXrx$!g8JIMD}1J8aWJ1}U;J;(XL3DuWYDCjuIx#gbM=CN6W~gtG=gJF$tf z%+6^15PYli8Ti2{Q~NIAS$B~)=jiicWab_ue=2!`vpbnlaI(BM;{wm)_pj>@Ie{+J zr&&cn!=))ok71=(7-*eYfDPyi5kPhy~Tf&%x)dx@E1x0eQoophDmFiV^2v$RDv9)U$!j^65lrP5` z&SffJ`bMIk<_4K7{;kyW>lKd)_6n~ZN+?k$WUiR2dHdGo4=q&u`AYwf}jS z+PX2Pt7RwLJGC?+oHJo^M4*gPRU=O-S?DP#cJLUA*gvUGAEQ~9d6TMFBBwkSA}hVJ z*rLI46ghXpbN9yzxn5nll4nSMpisLcIA2JE=Rxn z=fz!%O8$QP@Xd#>U;O8f$9#+6!^8U?etz)bo^Rm6x{wuH7oUp@3t>V~ns}j>rK(a( zmSh>uaTOk?Iv|n~mVnRxcM8qW`7Ekd| zv8{zjYohRavSj3}bK_z%jhHIYLq$b~=__!2eF>q0e?DG0g)Lm*2@hF({df^)i5^O0 z2!&UXNliUfmoF+tmcz0YOb1d936PW^HTCn1(okv`$98myErYg%~8C<$IH zmKGz~aji3)crN5-k-YtoASr>99hXVzSCLwIiuF*%M;21)#`v9QZ)+_zHjC`iJfh)^ zShump-Ec!zStBbqIGv^^Y7M7HMr!^bo?wECnD?1BJC{cCbeyNdd7Fo|dF@`ozdAR_ z)NR-@nJ&9Y8(hrM9wXujUV_vHG0>8n{OBL;Fy}onCdk2qOG{mI*ugqf{~y3rkl>{( za_|KFrH+k(sp>y0+d*_V4{9SiJA80D-uwKk+q)0lIeX;pS^&G1Hb4K~JzJMqfJqvd^CQ-pYM7Sv^;&cRoX zQDVWBv{MXp!PcOpAUA=fdu0^1sby*~G96irmT1>*UIZriWq8IfX z3&1t+Lb|E;*t43wm1kmcQR&MTsqO?rO`d%~^xkco zdpED@8(8~?iOmG7`7)A(7+Qhhhhvm_{;6@81Qw;a)C-qxJ(rB(4A#j$TRF%d|9{#J zk*3c;LpehX_=#v;9mUp>xCQZ(L_rnG}hV5A@J`R3tYzI^z}J9W4p9qRq*worh_ z7q-l%1vwRkbRVGt>OR(3F@udP4I7)dC`e*NF5tq(qO1f4K$wPDJT6G-NMPKcbgXhc z7$Bf^$c>4|oz*%{?RxOlepqZ_<^5M$hjS^KyLK^WOaAlp;Rom5efY-59n&OQD-sK0 zn-s(*qN2#VJL}!GXc-&b==$lR;q+q+G2|Fe3sSN%xx!yPt!f(v2L^6nV3kJ@otUCo zAeNeU%4`9%Eil(u%;x3}S}{ts;`58G%jtSjM*oI36L_q?@p^&j*n;OfxH;vfxvmSA z=&_6~!TZE0QO@gZ&g&%$hH&fh85L>RKS^?@s@M`vPndKx-7PfTX?(_8Q@LW~c;CrV zhv;y<0NUb}GT zo7a7Vv@tEml$cxxNQx!v*{yur8$@DEh$kCY6wl7h?c4b_)6L$sL@V{K!Py+Kde$;) z09zBae;j`%s%13&cKH0meaGs4%?GDux!=HOAZLQE^lgZ=ybw@iZP@nHWuza2XC;Eiux4jM^m>7kL71+kR`Sq`jVgZK^I zx_kx|=&j2KcwgqF?d`05O3|=!eJ_k~ zh>G-)u35A-i|WJ}>lAlEFjRQ4PGli-rxF~tgSz%Q;Z)+SW||7zBc@O{O7{=EdGDw9 z@4o*1-K$^Sef!$XiSxek>J)2ek{948(aQ6kj*^NdIe^8) zI@bA&P$Y00+l8NTeb?#1lfIiHJ|8V&(z-PW%&Oz z4CQP$s(xC%3`fpY&%pjAR6SZfGqnp&o`X?dsa~s|txirI@D1E*Y?yQ$XPDbr?YF%> zpe6~4h>qIvw4r{S?U+L#h^tKo6`_f-b<`{W+Rbazr>4(MfA`GG&CP@VIP^bX{^QUu zyAHu`{{A0_rcc#zXmPUlf4-kS3mf0xd==f?e{;|Dshj(59^e-om_GH7LumTDn+K=A zSb6in%~z+-z!lRM;Xz!0O@4Oz3|#)IZ%~J@Fop|Mqu3>0@w5xow0Bq&f*hP5d8r#u z8YyE`!nR5c5WjE}^W>C}Ot(`P`w}+bjjfhDCKW*q`=|!YbQfW7Om) zsjF+FtLcvD+1;RkD$zs}8pRykc+FgU=l;>~x|1cH%Aw#?G)sO~)3BW-pq9g|#@ii+ z8D~)9LOSZtGtaNB+9iXkEHE`qyND!&SW~@f)~lkofo3hloYue&1p`GEDVP?~yUogH zypO0!bYc)x`OGkRM1^vKcf@l1tVtGIBjL?dV>aqDGnLQ%H`wkg?!Y*sx-2BAUS8~M zdju>cB$AE&{g$H$ks_zGK;&gkUl^XMmf}rEM9Fh=%oeKJ8pg7q4PXrv1I}Ec+ZIR) z{A`|Bhjwb+Fn7aqMJicj0yCUvMQoyt_UI@H(iV=IJE>Yg19@C>%Z*EHJYysYiqxpa zf!gIgbJaHa=Xje>%o_psY@@^|F9zj$!7TC(l^;wsUx2rc@=07@DT)FbA|;CyGcZ&0 z=ja=^^#;vmHJ45&*ArHqZI6)2V5>G-K%<67$o>yPzXP>y$Xn{%kZi5dV%kgMci#GzS0F{3%|blx|`#@iX0R)^M4V zH0{hYRN~uNGzzmDJgPiWofg51K&s{=Q#oQ(ekd4N&*5yvzbUlEw(>(38LPDeoCPNl ze*_NS^axH$kmSaB&JQ?Xc2mO5P)2nE+8P2IoU5r05jWurl2IJXB7>9&=DnEtC;3VR zcI?JJv=ek%u3T6QuE@b7LcFifs6j}U716&T2R(iPd;%~hU;v^L+sWSs-b&rF_(!OP zh@}c3za?x~{LUt)?9o*MSLLyVFldxa*I@P?1?vXcrGE!ibBsPZU$QNDyWr+|i+|B< zhxV20bR;IzapxzQaGaW)gAir0UCPtNMrYrMlkaz~U$Y5X2lIZ8)!(@uNshO!3z6ds zy>=q^l0^JdkFmf_6U=WF7fTo}4%4j&K#4-SDUAtxT~w#PxOs5;D}dzY{^@hmUrm2E z{r%1Ty{oiRekJ#W!;uTz#r=;%H+Lh)HC}Uoe-Ed=oN!-p^Ys>2pcPW$sj>2P2;J#( zH(%#}oFN@V)0b`@y!q<%S!ZakPoL!>fK$_FroXCPF@4rI^#4`ppI5}H)R$~$+av6< z5KnmaCU^D2MOnMl)vz|eY+;+|B@7SiF`94Jx=mdTKy~pshTp%_`8NSX(O~*GOq#hC zvt`%DwO#XJfDdy*Ni%ObNg%U$h=L>uqLLQGSfUP#?Mf+5XpulMfxzs{V#vTEkdlQVCBdjIgTnZKQ%Irzc7lVA8x zDcYi(e*}1!Ky%AcV6p%^?@rays zK{L%MIvv$mh>5=`rwdL~XM$y^GU!$w7bz%qykCTog{7!KnU)QlUD_@%X+chScCU54 zXpmx~5j9lSSqTrj&swN z@9us3&TB{S@A(|9xO@2Yuaie+K0EO1rAz)<4DNsX{oS`+{4fFRoVX+?anA;A*AUh@ z1umy$uO!wi7n(GlYr7MK+0NLpxtQMvg&z(3#80Wr|mC@lk6 zqobVO)a8FjiXbtPAi_)(Pq~Al?x?8euxMOvhtOwPC1pOgwUG0-N&K@4zC<;xj8;>NO^QY5w4N@a$z2Xb zF#I}HI20&65>E+lum8wnw1Sg3{Dqz>k~Z*U4{Z-a9n&=B~#?&xf}) z;WCWrPN(oM{P6jmBd2E$eK0e*XJ-F5GyBinJMh)LlZWnIe*NyL# ze*NM7%bx@cNr)mCw~8phbCz6T2{B!_ZJljzXu*w?WLtEKONOz>to+qR10-YnhNLx< zy5h<~@eBud^EEC#o-?pF5B2~ePv^;ZN?u;T1(rcWmVxccaoxu1@`5@E6EuC=Bvf)3tgBG!l)-$`n{L4T;%?&bF9cdi| z?-(QmCEZAE;2DDfjbPm_3D5fL-(vahX9hLV5FTYZ^WX2UFN=#=tdX+Y(W$r`o!)1k zv>+;hsdSdOy|^PZk4w5YCW0Z~m>p#oyw6V6PiL7+I8)0-RQ;)XVQLrr**|rpdLFIV zNL7QGY}7(SMXYJ4ADdKt413tKHHB5*gArVUKQN-|`KcpQZ`UzvKG$14Ikm?jeAzQ~ zq^EkRdKwwHd21axc+cu3L$J-uItK2UD4DmWZ3@#nrD%^4hQ~%}Z30O;Mpp zY~}?dKK%afwYTnm{UKU0^Y(0P!ari26Cg6VCLP-3*7tsQ9P zkrY9cqE@<(R5Yz}u|vId1L9`z z7qH6nb~Xb@1vVBYOuX|+Q&o_kmKp;F$r4@xPf(UwmO*ug&;{mbunu~WMi4WyRLA@h zc{q{R%=s0ueN zVV(RAtMHOP-zwwbS?<@dkr_p%Q?tl?gANN%GA4-W7%wxf!`|acp>t~$@x+3-#|1GJ zsTHnylLKRRtP^=KVN?XUDIq_aGhK6L0cWqD=|*cRKNKpbNX7}`hK5#Bh&SZ|dFGOX zsdEECzhC$JS6(4tN13$pg<9jn`W-UJ=pwUsH`-?9OPJs>=k3B8XK97Go%2ZraCZ;j z66fq%3y3ha;8GA1NmsJ4XQMZs7L+E=KhH*QjrwHO(8MpVzB4j%|G=9w-~DCgop)yr zp5DIw-nW1G_451oKRP?}<_CAroV)wuU!GX`N~g;I8%oHX0sK1o+RUFns|~<6Ser=+ zCK&42FYPGuGVjA0n45kzk91{uMyHH2oXXAMe43ozjdRsFPKxc@dx_o)U~XBpdfkQ% z69(ZQ8W}+=S}76FvatPueo{R%wR>uB^@r+tS7)EOl!_K-d#lH* z*QRz?uT?Kk9fFg4VC2_O^`!Iv2~_=QY8TI}I0+ZMRsFGgb?S|&{k5Z~okWK{aHIMs zPfzXl4f-ak7D_JuA<>lz>oF5H?R5P}L}A@WK@no|+jyW^qI}WZm4kU(oYupqkZerr$~iU7IhAUoay9yom&Gx`%{WCm2>dMU4OT7 z66J4Q;>vLb^TAzyS4BCn0pO|1zTsQTR28fFJaf6{Mb{E2dS3$-QHVDs6h`j(dA-PY zzkcW7*}HGO@n5Tc-?t$|zq}F`lqLo?@1&QX-h6fX{Pd>|U*+_Lo4cpKo&FL{fAO5N ze`@;N&HbfKuerHr`U^o2o?BG9ZtI8V{|y3N zOT(Jvp05+G1=QK`_zEe4*c6Rhj!Wnqi63uYvIDI8P`Tj%m!uGj$Q7NCn+D5;LAp^t zJUIVho{nfXN=isZ5gv|)q2@^uCXVuO?W1%L-EdA zlc->@6N9_H;Kv|Zgz^sMylZ0e&BVj3uQpPr5)q%w)u;0I3X;OrEy3vgGBTYku z(^{DU*>Z_9)3iCUIUC3HrN>ujXDI9=*o)n3_(Ij2dDj5N>ZU0z)Yk8 z*sa^nPvA(g?1`L6kB^uU(!Ak_1s(zO- z9wkZ)gM_3Di*gIfcR%DE={v^`iqf4AK3fdS_kvB|?w{Cw|Kq>Te0y}}@?Y16dg*)H89@JA;2gombD2FE@$PtP7f?~8@Oi0Q~_)EUMYd7CX@SXUb zZ%*=B&a0n?6f!X_q?>rDqiiH+@xcXRI0WKjG-tPwjNS^|#GI~KkgDjg1;uBw}6O=3O#`9anJEyi~4=k;E zl>o0aQ*<| z%^V*5uH$T44c1d}4xZ5|2jOf)yxJedX) zFVZL;#jvz8fYMXzz!9d4L>7faQ2Ku!y(C;`v44%rq7 ztpgt5p~Qr=(zptF?jJBrEux`SPGkP;^vU}yY$;rM3_s^(oW-n#Qz z_Uye@Bq7$+hJ9HIguVk8sZN0JMqT-_NfW%6Ovy>VnHQ?MbMWlU z>j&|n)%{NMH z=FF@254?N-oue=$FDTWY(kxQEL2;5-kO?W#g;#x17d`uX6h)Am`~aSPbK9_?*92&& zL{y5k-T?*=o@Q*Go^P&{5;3tCzO#2A7K`;JL@CxgIM|=)RZ_`h@9M#Rsdq3fCMLXH zHZp>G(Ek?0^zC^8d@Eptta*JW9v$lSHd6obC(S0XprK7`T(5)dTGVPg9$$&1AP4U_ zxs+yABSUzvke!5@w4F@acJ++3ASWZudmbQs+6fI3k@pBm#Ujn4u6bZo0=+}0p|p|Y zfbixUQ8ny5K=PO{D{nYW%EEv|5mHT^JtMbJN8Cbx+x6RrM?-T40k%n)|DsU)JaNMW zHdugs+s;T^T8IZnWZ5JRSNMtR(bXtfqguPO|(kS5;Vqi6wi*n z{FvsggR7cs;WdPptUd`cQ~$W|U+x=*WKw|V2iC7NuYIoUYM(26BlqJ-NP^tt!F#Or z+XYVpZD4wq>5U$XA%ac7Lh40QES7o<99xH);F#@kJY-?Z>52a2K~XrN(Bm=T38tGs zUb3j)0Of%gChF&9^ecv6LI40IW{Vg^*T8UBgUe!o!Q)gds@hiFZ;|A=<^Uf+m`>Nh ztMi|DBR}E5#f6j@nM5v=2__#ynrj<(=?Z5>#rjfBQnhbC}c-NH3 zICbNpvJ6k{2?>w1kd8H$qCI|<*-Q*8-?+eVzbfoJW@YKy+O)Ft*`{S+&AEB_OZ^6F zTD;(Kkg1@7p>27Fu$`q0l{8#7dHG{M&7swxNSpZ<>&v#j#`R?z;YA`kau0K^XS*Z| zqMQQbQ;5rLtum4*q?3(;6jgxEW7NBNb>{ftnd9%@{rW@S($%SIPe4i*VoC2|rvayn zfQAO)wKS+|HA>secL!#^`|#JvBX`g5`oC8Wf|fQ0nh<<|f6z9{-Ds;YApGAeuiZU& zdFJ2;zg|5y^XJ`gV$m`bD$1GHWw_#UbA)6VBf}HC;xo^uSZwbc4kL8 zdHLSwyYHQP@7`-??|uL7%sX!eY}hJGLOLZ#k|aoR7ZaZE5^a%}FG)=+U2xd%oI8B? z;I)~_Ln~He?YRL;9@CZiS;VxV57n_XKrBhPoEF#}&eKzJV* z9(gde9VDyouo2bq^*6!joc9a(?Rq!jHF$#mwb)8V;T6g79M68L(d+&hNBNN3nq%YK z+Y(t5ysFCl`CYSe(#VGBJB;(RkH-2OKV{%+VeMl^dGRq?Y#@0Ztqnx>wgne68YpnY zO@ip+K=dvPiXbbEmYlA6eEKgDgV|YP@b)IuNg`cDB6UaSXV3BT^msd{nxAM#n}s`Y)=5R|zfhx}}ud4htnf0VaHrnFw&% zjB`CSLvwDI^F)jDpeh6@;7Un7`X-05n_-mV3cAO}GzO0SMcpC`(~ACDyn_5Uj8K%klZX1;=Y9R-r0UudhqP-sHHB-%NIOqEndC_Gdrek$asI%?y-AHh!YP9S= zve>h@z``=v<+LC+HKogY71SGF-T36j;TtDzyuZjy0QkwL;D`VD#`|c+jrVT61KXe6 zIC|rY8z0_y-+yV`K%VF(%up`TsJpV#o?TgrC`e5e56e+7=0ee9SR?wlO|-!DJZ;9K z6-h|Oqp7N1C&!B6@<5ADSx~}r2(*QX@CQ4`Obw60a2B2CVh9^}2O0CzHeTjkL z<8rpwAHMAU;VUipcL2RWLcgL!+_rIju;(a@=HZ9?d;{CLL|i2)q?+0YMlb_fIeGfw2j8rec$tor z6hui8#AJG+VINGF`F`s8SO50vi3cZ7jEp@0@Z+QCx3?xAo_ZI|qB*xTTq}#bjzx?s z6Yd>uJRL<3P95ZC(y7XKEJZKoYown2mAM-^uv$dM-M-} z`tajJ{>79KT}-E4IcHK`MkVhu8W+;RsV-LjJKAV#7A@tmNsvK`Yz%9hkYo!fY2|0$ zBe7u0=Sx)6IGJwAE|$=WlEF&MgnI^Xsg^-=DY2czNF-BrxKgpYEIZpC0R#0T75VXa zq@RtkZe-oM{Fq(IqEPy+yb)e8i65?HyKnaN$F`e>XYBBVJT{&rx@$T8gVmPvz zWQl>>?}Zp_5ljp5$p2jn)u}oua#M;JMXFPzw!;dPk8eS&NTzsL3ew+n4uz2fi%W?AZM}zt_t*39)pejad_(y$%vW~$soL3c~ z)cO1rk9>X-FXxDhk(9VRK%69zcA_pIkX0)sgk(DM|IEV!t5~=4tdwWr&@D^6MUWh8 zv~i=5&%BV&+oxyV{NUc@*YAG)AzE?&>!bJIx!~WL=~Vbf;=O`or4cf_@VGj)AzZ}H z+cj^^sh+G(Rl>n9` zNKNJ1%W~G6^{^7#ne)PTtsQZo$vo{a>KptRc^Rvm z=#;<&LnkdHq;}k!^&)XZUM_L|zu$o}FiH%ftBXw+*kb20<(__9&k|FkIsZ!80M_7f zkfM8`iHmi(8ZHZRg_iKM!CXXECh%`g11Ioz)xE8k4Ci zPoyBY$aNhx4=WS*L>Y3{4sXe`@dUooK>Q7UoUWd$p5_O+uj;gO;L_9)XZtMQ=bxH-+o>PlzP^7OSb-@kt4mDx4UiOW>aPrVCguRgk* zFPc+_rrw^g3lkf$YOrx!q@cZFHOuL}c@yA?>T&lC_>x+^hN_d*AJNoaSTy+;PQqfj zyLySQtq6YKBy3Ld?}f`JtCM`0y#`-by*9P`mtCL#{qhr1X*<^7tNKax3_P8^)gNZZ zJ6XNpT0vW!?S0I{^S`!guOBPB?3;9`!)IJTAyL_Z(L*N81r8jkyZpT78l@^2j$h(T&qWH>5~%| zfY=DjpxjV8+u?AdL^xpG$x)tpA# zx8L6Lx7&NazrFX|?Y)<8?|t|7-Xpj7etUcG;gONsyC-k&{_ghPuWs+%Hyv*vrIlH}^u|}9~fjOJ2e2Y<*%#M@jJWojW zjoS+bdSOEywY1BUN_)4Y;;|N`&Yp$P`97Ep2e3tQ%qWBOC(?+3=z)T6=J{17&wJ+8 zYO~FA*TejoWyO(^M^hFED!3CymH<+FM*3o5r23sJk<|h zZ(s)w#OE25B=n>^!Q z9jWjGdL8Ow8yl4Mt~Q9NGB~($l@4noF(&{n3%?AnBX4cXxChcXzR2+E%h{$?`Bq1L zDl;vNHsG8cq|@!A=TisGciT*WEG|e=%nhA9eQ5-mRt*-atOG+U>Z zStz3wdArOj;Qbepb&8WX&O0GVVno~JQVocvv7K#5OuNF864LE8*jXZgJPHrwY|9Fi z!q9>EfXr+B(&@;Cbm1||3@aQ46+j~{B$_B8J@;A$H2!|V33P7Tux(`I? z?EM~L zOA-!)j8WcTgodTxi?FIh#eVZJp?sna&*I`mWlf9FvdmU6n9A#B2*O1?&H{KOI+esP z^X^Du6LlrEJc6P|yBz*u!MrZ%L9O=Xu>3XfTkzkX(T>@3)kSH>wMM^7sc~e0Jr_nyHMDi8Y#a zGTzrvLwn9dYq;Ft*~Fk02z71zeOCiQT^q^E)X_`Bulam|m?rPO^g z`F91_>*00JC@Jky`7gQUit3$8ObxVFNl8JH8+{s&ZS~VN07w{En{7j=){Ie?6y4wT z>D^0*W?nz=7$I5PTp_)d!7R9*IT(g7C-=TSaOd1>clQ6_8xskZ{Ukii7_TUDxdr{# zJ;+W~lGrFGo{Mb(hnY(s+&lQeox`UegTo7(n{apmm$3d!U~qfS=ePHKaeMFAxA**b zd(U6*9r-lmYmVXaR_EI{1uv8@6g11(JNM(eQwx^_@@rZx>9 z-iJvHOdznt3uQH&;Q+0=ew>C^EDIPZLb}PL77^^W4e)Qf&%>bLNuURlhaP-7`QY^X zzESQJy)K}f6cS=!N1YR493I|KBZe4;i6w`Mw=RD~d(khuj=l2AaQ@ciy+~uXE}f+4 z)}=4?Km7^dbsz2Bl(TtL1g;c-Z}txG?_vzi?uT9G&Wr^al2+0}3$Db=0kl4lW53kf+&iE&xM#Gi8+Uh)!TFi>#9L`K#iJ}FNrzPu>Z91>sUGyH4J7vXQEZv zoPWlK&CJ&3F@4v0zC8n%7UcG#P)kITW~Vl%sX8m=%bFIFJ6sWkL=#W0XEk7JDGR~^ zfb?aHR7XP=woJ5Q`*x!Dg0{YSRo}qcKTK>USk0G_B*b`{N-2*~>iLn8=bv%FhDou+ zvrk+q_Uw^fXRsz8>q^cXN(1y)EAu6dS!i7yKHqg2GOx=>wAUVb>3GTEz2b>bW{Hl6 z=_XecA(d$K07kBiwn2XQ>PZ;vQRr&gTGfGEJRP=KSr47(d`9k=yD<%g0{xe}(NBn3I@ZdNa)-C=8?7SaMDV3S6k zX%`~QB&B@NHN6vvh%r9}6J47z=py;9hQx*;QnCoAmgEGa%44z{whlFAH_S45a#Pl+ zw}0|k4hm2A_}6t21*&QAo|!09MjGCUh(5{;mGBY6=~1eJ1=B!BS{p=n!?S2cwrkTL zy5=L>1tUU#U8)iK^LKd2Dr;or23%&gHK95GjEp$Z7X}Z*m?d2^7aJ?lYN{2HsWQXAQ+1H3 z+puLaU6x$n`6nF!=#qjMoO5-_>w#HHEYw#F+-|9HVe`+|rSN_-a=J0*n)h00edFkj z!#DnR)nX6oOTmt|B3o0+Qg@L(smi$zj%VYmdp<2(P zb#USyYXcPCk|y0`9y~S^y@l?0=cCDcUmd!$cmK?ZGmB4!oH_mW{f|ER_2L_M4*Jh^ zUTunsB8o{D1a+5F+_TR~N(!-9L>%q}p3D-+b*Fj(1@b9LDM3jGZsiC9@gg@LVd+BK zNX7)qv6Djagw~8f1CLsDD4gBi;kE0wY~9$mX-(gTzKv*j)yB;mR{6%|C88IoRZCgN z&Ko(4O&uD~$i!oUl6aa8NNN`UQt3akUZ=S{j1ery4LB1(b$@)5k@~`;cwTtolAy?q zXxd)eK~_&r?WvxvUYpugy->YcJ&mSz!vWZxIs&_ws;7I=bI(0jeYbkLdU2Iai;D`%Jy@ESB7Yibgds@p87t z-MgV>bSS(@N}|aaFuf)lR!_jY_9KYa1>{H>7u3@rRhOz478NwrgD_R?h6&;)R6Pd& zT&iC7jY*$vv3Yp7Djq$fkrw25;AqW~H_ZyXAico*%Zj=ZC z{ZbE{JJS)Ac%eZ=nEqJ8bXa;2dHMfm@7@TESbyE^2B9 zEHl(VND`b}Q{_4uJJZHah7?($$gl@bii-#5-&q7}QQ!>3Mxev;4{E4Vqj6L6mQ%`%QK8BTa*C7Wz^l;$o8&A~B+Jm6q06fOu%<>q5@)Fev6_UcMG47- z?xR^G%L=U(!V8*?J0&B8nsNkQ9a~iEg1&hPl`Dik)qck5ApCyqrsvz5S8w#)^ILJL zK;2ZL{ESg4{G5F*b7Bo!?4leAY;wZieXe?n!wm}Q3pPE)xn#WB4&WykWu<|$ zx~}mFMVSw=DBE4_n!uwH==<>PQ{x9NkAL#n_^~UGzIbhXV2^KPT-{MYMykZ9NFHNU z;w-O&k7^7}3>Wk1sG1&U#S`TEDse3!N2f*AG)n~lMY2GHNWP8oaf7#FEmtxWtWhNt616zXxe)olk#`Y$iorpv4VY zhVZDwSsjL0lma?6s1!^LTP3DKv{0&*#9M?I$Jg*EdmdL#)MM+a_A*syn?al+Qml=r zhgvzbWarK{3=_F-FooLEi&W_=8U%2Bwq+{TaywNsU6iqXQLT@>q^Nd=G?!a#K3gK@ z)G>GtO3H_v8eOy81G#=OT|y50rM?-j;5cr{ucQbxEYxnYjV_sHwpQbAVqg=sIW1o1 z>IdKN)*rEwXHgTeX$hzm`-QXZ@eLEWloTn`4ilMy3sf(fF!I@;w~Yo6x&#~bG8c=g zM!ETQaFHc_cLfi1s27nqm#WDuSd55FjFJp0;S z!3Q=IS~#0!!)WJDi|^|uRNb|!EXlCUj~r2pEJGqkGlU~A5hh)fRkx#}x zdHvxRAE2e<@17ew^xoK&J^p>R6D^-^5s@l7f#g{!5CtwSRj)>d?R}69|FxX%ap#rc zOHQr_?9IYMvi13PLPz+T5@WUY}2z-E(ZWn2o5#n)HvLhiy&i5(jd#(gcEAut$u-t~1Vi9MFFnp?7 zf%VK@@xlvhsoF(a{%D(+9qYoTX|pk;Wzg?z0{x`@D@)gDuDYPF^^cvMEodbrsGYD( z;U3$_uo|K^!dey{JZrUpn8Hk%Aclc-vTjS?ijL-1O@)rs(%9~MUbBdy5thBWYgd?7 zj`Brwz62|XW?Q`uo5a+MP(IR9)!Iil<@R zc4X?=O=Kz*T1LC*i*|s?bqi&fS{8BF@oN@mW_lLqpn8t8)Tjg>c?4VwU&hcC-72Y# zj35ZK@JTY4h*v`sF|60muzuv&&u<=n^4>+?*fxq`JSz{;)`OWoJh@dv>Ns`D*x9R(&K`Yy{1W4I zTv|bRt)0!Pq;KsS=Q!w*pu+t|aXpQa-^Z`)A3t)`KfXlCmE2wx-_%<1a-p5bHCd6+ zZHMA2RCA%6;F5_*YbuLx%h38;1HHEfRF+2-hVYhP6>|9c(%BY9u(eK+;{3Tk_zueF z^fIkqIHA^n3O~arAzq8++&+H$=I!g>_kVxr`+Y6w_S?6w!q)EFSH6Gs`@OfnMz=4* zf&SZrx4-&+54t^Y`y2Ro?e@U;`)*(Q{?BmL{@dTM>)<4OT!N8y`%^e}87|)kN1Z$T z<@Qzh3cJg|?Xz&dORzuCa{JTUU*EpYcD{cVKG_|=gx|e-`=;}oaNrf^BpmvF@Avz@ zfBp71%WuE={ch**S-1-NL0T?8^&{D1e^g$tU$F!Id%wG>vcmm>HVfj+6^O+--%yt8 z9|gxv5Bo_^M#|OU_^E4sO|T3=DH-819T!FqXoaMVCs;ivxjy2?1p>*G;>WU+>nQ0| zFCIxQQ4>SzCp4}Nn-+6}{l`=91Dg&2AB&+s5Nwp$a`kTiB4-=!#)hG5t~BMOOYBT~ zmpl0wC-YH!$K3H98nzXcgd0YXV1<}uG0MlBQW4Cdi=iq9yu=A|J)ymGR6E$@<-rM=$>U+&e$NaUe9eWTISJtJ;o& zy+&z_d^U1<t^t& zq0(2C*I`=!L38$uTq^h8m(Vikk(zGw{n~7_b2N<#STl9oRYypwObU|sNg>I}$tWOs z)_`P-<(E3wP@(Q z#54_}xJzr5mV)Z4!WEuZ=1+?REtPs7I7w*_=X+OWd>6)I(I* zA^-9#X?zXKxt($@RzRuv8&{GvHf#;Sr*9B8bfiFijJe8(3S!(W4r9D^K0O&Hn5gQ zYl76`U5%Xz+iMI~TD0VLqPXOWXn*U6R24W=4P;|lbj2Bt-nTAv`O7fnKlA_4PYp4_z)4NgfyKiW>Zq0+T zk~QzWhK&r`KtahNTACRCO~czL>oM1A;u5UAi`8P}`JEo^Mh%NC8DUOFi5cQcqXlF7 zf&pmt;_$s(Vv3U!^|17gzju1<>ct0FUN3LV2g;PHNZm_ckDYsa?D)rH`(GP7aQfS? zAH8~G?5zWjZXOuB@%7l1cg9Y<;TxY7ST9i5#O9Kcn8e&*2ltFMJe36*G(V;mpRp-h_?^LAK~ezL$xas-Ck7+u7T$P*5R zh7={GRt*isC^H3xg-D!}f>T&z?hn4E zRYiI?G0=}M{{?N(hmKLCF^yIkx&Z6Y{lUG+8v3%pHm`kFLiYzhSs+lavWu|Pvh+}- zVJ*xungDhZhF_FQjKXp#`l{X@SDjo6*sKH*LV?pSI}=G5_?UP5z|me`pdyNN?9bi=@KDbY|H+(*|;RMZUsQz`GYuvnUwl{st_a3)v6c{7Lmrtfxka%gFr zUMv|jo3l*RO&JXc3o>bj`?M%66E|+zKw0escvt#5I}w~j&riHK?3&FRTehuk>)S?6 z8>_3SO4@RaI}dK&xa|MAjfxF=kDi0~0BxZf4I<-Abk$0tQY}whftJmDOq#hkUQC+v zg^JH$<6YFFn^?H0t1#(Wfh$MXJIIp}pHmeka1X46X53A(+$<>;#c;VH$?GuvhKL9w z8n2$Xyk{4j8a*)bIm-;#Kba~py1!+WTFfnH(f%x69zI|G-$Rq(|NBR-j+}&x_KlqG zb6>$?>T9k*ONIC|^9?+ow| zxmYZ+N4Z8wH)W-EL!Gt+iIXEo)-)N7gc$vjWJyPSvK~g4twbS3$4d9_%Ud%U#x+l& z!f7<=k`1{8mzCGwaaD`_gR3gSnJST{;3QDd@6P^)Daq-|4ju%5J zlFdBSCBxK_|9kZFJAfa7wRD)5Nj@sVM6 zjHvfq;1UwSxRmNGpk}-2}yDmlU$0gHaB}NE~P9XriId!Xz0J;2fKA@ z{C5+|V2zat2c-Z$^`WbXswQHIX_F{EOY zAq99-3EG0YiAt4uwlJ+WSX)cFoz+cN+MZRff=0CyXgF-Hu~`i&mcs%4tNOS0JV&tB z{2W$T0W7qJ=tkBNL?%};*D~gLo&@nR7LOd`?orvL7wV|2S?92n1duDLXGzZrt;09e zx~jeJ$CnRxcD5M=!!IJ%#B${SlASv#8*d7-sHKu!yU@;^9rz{a$(K+cq6K#0vy)qY zQgB%_W=Eja8c4y<0?@4*oK`zKp{x233u2u-Zdf@(FY1}jPIg97vB?tn*{OA!39m-^ zxNc;0=imlXq!~w|#+{vTq6-(O%5mJ1Uun=cjB024o6})i7{Lu_5&AVy+Ua(x=7`Fo zl@M&IEQxiI(o=BNSC>c_U6C<_H4GS@-)msPUdLvMg z`G^{fOmmwUG)prfX@0WE#pFojuLSwQx}Q<&$v>zU5Q$5O)dt;0%KR=#y92BFK1Vcv z*0k(&A1ydRJeoqDRrQ`-yPkUCyJH%~CEIKH6(w*N6{6(XpCZS(gcKN0kiR?N#_&?I zitA-`x35^<(brKbGRpIpD61#947tu&aAv9b3_D3OjA?c?+!J*{pJw4zV3~L9K=b5* zkcV%hME%y1mc<6vn75O0F)>{g`zOI1d?G?z z-=M7~8LYW@vv^m~pvQ5Vaw^~@0@r0-@ID+1aLi-1K|X^3K~O1H4KJ0do=ImVH)(1o zx~(=<4W}nZul)Sw^;rT3)^Du|9N0>VcAA;UKxJO9!*cpm1QGo5;h`t*z6p)F_~h+V zwTLwFeA9G9raEIXULYxO0xxm`h<`lq3K;P1{7kX*JR!I|zZeD%2$n;qwV>IP4H-Id zE>@Gu6Y&Byf*0t?U-v(G@1k#P8^sHhVwrP-RPDR-69F=Ic;DlXULJpE|AT9*rG!k{J`7@FWY}HdF_WZN&YN6A@90@mh6g~{ zP;#`JK_Vkv&7iHTw`>8ZKy1sHXz2?t5UmB&}=FUO9)J!I*Y%_;yqN{gNOzq zfuMaZe}uX_9^LV^?G zk;$VRJB+p%m>QIzHfZrOw2sZadZ_jfA;~3Ef#F&3qKmGSGz+uRkVTa=sBl#voJ0^Y zQ-RYj7d%dy4%bkOY=fpr!{XS2Ati9B8d}Q8??ozpFO{eP@>&{*kjy27DAKSJ+m^Qr zSg$KAw}ME3;Isu)d!M!3JACihy|?b2 zy7&IQxBmo3PTu_XNUI-x-eej|G0;8Mbz5j7i! zKGSf9QIbqF);D}BiBZyA{8FxvLhk{Evcx6DYGYH*_K4ijhdWq26jqlZ3p1AH0&YP+ z_J6!{=k_hDH@B_Zwd=?JqrZ|9q+z*U+oKXzb{{vaon%0?N6=O8CKvqX)to6 zgj!VtFODiI#1~Wyd43i8NflEfpsv8yqYvH3Jafyf# zyc(6d*ryu;3oKfx$WS}0athKXRmd{5f@tKs<9c_4^wUZd8Qb&-Rf0>^N7}PM^wU#1 zO5eXWs3Yb6;L*ltr($LO7d@~4e2hy)HPhWwTA@IV9PJHrLgPg)R+Ajy*<=7p&1NH$ z*7K&4BVELpf2jfO-JtvDF>1}@3@JF=q`K20-6B)1W zrB#&Kpe#xZ8wN^VHyHI0F|4QEL{mLy8Pv2ufkx}Gp`uj(Xj1b zjWILjewcLZ2s8S1ggLQ^!&s@#F{GA9Y+5yT1Y4Mq#1`}n?V<`s37L;2W3dFvP?*@- zSygqcwzM(oT$;97UYEJN9jo2g(xGhzRaIRS`)ZDKcC!38R<{;ej3uY`D;a7kwrRo* zNoVIXh!_T(WxonfSfncMWoPEw;GJwGTH`d+|68FF)Q(NlMFP{z$7W8_hU0U-90^W} zc}~-AY9ovsSw$-;D}1L|)wZIChxd-Xb$smbpU2+)>L1q+Jh;;T_``$$xc=%asS0b? z*BFh~>M#YY*ByB!d`gS46Mr4wd*SiPn`8SgKe~JYnm2xAuW!5rKFJBp@sQ#~v4WZ3 zbSwr}ii>$S>CLe2wG)z=hnzhtaHc>tOV<##91E9||I(_e&8=O`MT^$3ulfdN9ZVi7 zFY&JR&v%eHpdv7mp1?7ei$T3&uD@=qW{BCS+F1TQO$;c3<7*gfBcQ5)vJ9f)9*u?? zv&7I6K^3kmEOX(g!bQ@<7qTNvG_-V(P)!TL4>56({RW_flkxebMXUr-JB>|(;3o@M z1vQ&V)4qXU!xAfTp#r)fR8$9N+av4=P7ovON5%Sq3%W{*;RS>-Oj2AjS?&7ptZZ+f zTE2Nb=-4E0IpvS$(coCkq4gZvj)}s8LhIM2MSf-$e15L(7kMt0ii)7ShD5r)?2Sf@ z2~psZv54#5mBLZLpeOYNm$&N$WG33ANYt(pwM|-}4T~-gBxH+mE-psZ2&N4j#zYlL z%`=8af=kxm;h9B(F;MMiJP;#iM+T6?ns601&yJi%qx(kpF$#r|lcNV=cix$RM$U|! zg(v^Q;kIySsh}CYG1{2My@o67k(Wi6G9-FNDaDDYs6N4z4A5npiXLvJ1fU|txMWQk z=O|O6JiVz3R&Na~LmecaC;6evp%MQwphPYvMs~P=CLCbR!Ws@OSDgVRaf#sc^9yL6 zqxn25i9;-CPnW72AxQ^PoJ-YYMMsR;^)qIF{MnPg?*HYZ1HNHx5MwsOHpwM}GuAJb zNwKKwDhYK?lzC39!O64vEln+ppgduvbGDPTO0~R+3kB6~pj+a%5y)FLi$L?y3S0to zd;_vlSGcv%4OXvv*hn*v8);5JL@tr44nFeins9!O)z`{17lIBAa@seq?4r-HGiV#t z6h@r5QQ4>)hTh{Fl2x|Kstd!}Zs`~bm)tn~@apxkuio=- z%5GSGi=5!(sMOw+h{tkU>WbH|OTyz>x{(~55pMy(G06^Z?S_i21O`trz0=BGIcZQ47!&o{PJoYt?UKYK6Ax40?#QH)p&PYx8G`o z3<8Pc`DitBd0|yjv~&d<#gJArUgi=t#NmD-?c4N1kri>!H5P9V?ZFvjmTMk8^@bi6 zYFPn=?X|SQMMM*YQq}TS)K?&(xhw4 ztpVszvJ9;mx~viZ2D(OBOnFvBC{g2=&3VP?hZE<2IC1lb6MKF*@lL2Crwq-UApCIR z$`2?0#{Pcx=O0d-ML(Rl^uvi8Kb$z{?4N+`3Pk_>+Lm^^fN2T;KRL>}YWWW*K6lQY zgUf&T?6n_GyyqO*`@@N^o#V%RV?6x_>RQ5BOm?IsyO*%MfuWm2iNE8WRF?=2w|=Z^ zI5d)R28hJP;(-GqGxrByQ*z6YL5eCgGt+`Pas-(yg@Tw!!5T^v^yABaK^yd;W3ZyJ z1x*>c0E^oF!MzIv%C)pubYk-`>cW=YApU`7VdfxkVU8eho~6Sif~UhU@h}JT1mjun zVtF2#LYR=a6=$f9{tayocean`nn^S_nQh-NI>=u+27X=0FZZAdXFZ z0d3a}HM5MlNHo-1U}jTzF*@`jl42-BGwcL4klv%A62V3@fp03#b5ITOsU-g|PrYPX z^+vFAs+*ySj3pW^U8Tay4UI+{oc+!4t`HL@V+2BY)_=XI4ujkZ_A{oc>aZLZlETC3D`eD<>qXb6j*0zF7rkuhbiGPg^6eBGE#5? zx5W}yQNX1T3)cWlVvI}h-U}qIWR0XX1Oy#8YdSgNFzLaBI#Yhr(?L}!$K+a_+evSn zu6R!nV{CeU&6?*ow%6j*v~gY2bbOjNQdlFi4w2^sN_-47Fp=Zq0wY^c7NxF%u7o9qf^KMFePJd2cVxkqa7aImat(k6UhHa1*w5kNcnBIlf65Jg? z<9<#T>^!O96BjB2$=UXZj>yI1k?3CuPFHnBp3b=A7G^1ItZ6(h^Ky|YfvpmC4jL&2N=~)(te74lDbRwBa^HPGzdLT-`sk@&PpTO+ zx@iVfeir^>=7+xV?~ux2l@`!?qT#gR=0c2#mGz{&;}HxHj}y`OaIGlNX$DrXX63>f z9}*|lFh9+%UA9B_nNF)?~WW-^0ciOph?X=-0nqhz{o8a}n%Mp#8Og0^4- zXBZQqZWqy3QZ$GOjde(8P#d(KnbLj6^UVU=x$diu`&obq$-&!-}!GvO`RjM$f$ar53Fa2jW(jimxxYSY++0iiGkuCgdG(^%S4%Z$4P=c#Eq zX5H|I&}wXv9BNf5S`XcUmi`X+^netN*n3Be8GLO`>)NXa)l?sWMq8v7R#68r3~K2{ zsWQ~W6zH0aUX>GKF4MJKNp+GvraJLXxw2$L62(%Tuv)hq9&8xRl)%0iA(&E)upW|~ zO-{lZR=x1#N*d1@&DNe|N8IVM0a0z zHm=@1&JN$bjP72%`vq*=aK4NB{?7&2*mL*N-OD_B7VX?w+3)*5*Wt>)vM;fJyLQ3V zcl%-U(%sMCj-Ro6v2!<^zwApF;SP7-fOB8my>|D?-7n$u3v~BuI56n^H2m86yO;lG zzjMW9j^n-^Z26Te9iEB{dWa^R{>seJINvW2pv8c>!n(1)ya`=+8hV#~UWU)|U8mkJ z-u;I09^Wy6Wi!U=N-Y>W_r`-Oua52Ci&{`THg@3U`N|~!;M$=F*RDOjHaK?fm51+q z{_xaSV{aTB`|87oH%>n~?mwr8QOffvo)aW4F1zOB%9QLGBA!du_}wFAu1cB9!UszY z4ow0DphQltDYn|>gBh;P7&<+4n(-SLXbW~KGQ((EW8gvoQzP6e8op=4HVT&xeW9V1 zC9L^Z3|7FC!wrhLq0<@N6=CFCffLGkkJSSx%6qJzljqnwzr4Ti=W|DVgWITtM}Z|g zCTdDkN0VdKHYkAZE|f}d@WaVXFNBM4kV zs1C_6Z14Kd@LPBKhc69Z8@|yD8&`&}-`Nd2=kC0Lh6jdk-08n_2tLoDrEp^J@Z~$N zz$M?@dF>e%iavZEcCHOyxwFstJa^}1*nVSvDLBI?q4C#;Z$b-h41WV{*n?pA3UtLe z1V?YUd*Kq|Q;sjvXRf`PMw*#-`7kTVH?qgn8-b+}*JiVMR&;mnWNE`~7&;2Aunmn^ z7HZGwJ-c>=YW?$%*D7113PDS|s9Nx!*|~w}MdHdEaOK%}O|tkQkghc}sVU;df_-=TSg#xRCC&sIFTF)s5oAmCX z;|N*-2<34L+PQO`)}=E(gJz_gdK&NAg_bN?vPz|BHC1U5ttCp{RC2_CqpTb#EQCm< zrll8grn3{!<&%k2XD2h0PVy)Z z-KY(ncD+3*$>n!;c09g8A}1xHU?7*JO4SR*z*b0BDpBTy8V|E)+0t>Cz`SDi!5`)& z*`!-cM$@}cQMce4id9xzEpHc5+8{Vjf3=baRU;@cOmpVsvmoF&3r!TbRKmXl+Q3Sl zSku@$MM+G!^l{O%SxW>_v?k$b_Tbd#H7C0R4m4G1fm>Jw<*mfDRcp4ymZyR}AsI`| zHJ}_G9MnN@0kzr1T5OuyTAQY0)6|Al+NB$s(^r#0HWDXuyufq3D7dCO*Vy8H46Y$&&#y%YD2-TUC)+vwhrdvD$Q6!5wC zw0lN1wbj;`X-fO;h>fUL%*T?tXuFD@R z@tnj*iTf_+V;4xuuoZal)G0r(l)q%;pl@11Gek3y9lqg1l0c%Xp2Jot?LgU~ z(|Xpx#T+H5x%s~wMT(%z&}j=(RUJA#&q^h2C_g?y2I;nmiA;qu(X@-cXcPZPZK1?4 zh-DGpmEzjWOUW5$GdMmcSTZLCuZp>VhR@V|M(ivBi7aSfS-sO)oQo$SALMKjTMyNi zp>?XtJYS*CaY-%~3+xdArDwgK>%j&^8?ctOp?&j>Ua4c6g39(<*5`C+5>h8x31mYQ=6ql&+llY=a#RdVT8&H8Cf9lbwA}66r9v04xx^57(6f2fK zQ*z`#Lqk>3TCA#i&k_@DH!!twE>v8kSE5(eD}hz9tmTKEH8Tg^DmY53%?0D9KY9G& z!C4Fi9ouUR1#p+1rzC;((HrN-ZoKyB!dv5Ce)?$O@S_i2e{|q;FUVB7Vlo*#A#;2( z#tA&ns^hri0C8reMw}ZMflCG_k69Eh5VR3z^scVPtU0L)NGZ>%rz~-Djc5=k8{E_j zgw1Po-ll?YvP}b9XnIEY3^un0CxBP7IoPI0*y3EQh8fd){VGEg(zPU)$UOQX9- z&WxN!Bd3FEBE0jDg~L3W_}VmSNx$a&{>?DEmw&heSQxg<rCF8N>YsOMRWp50 z2OI4iO``(VOx-qILa0m3;k}DFEVEsXj2}}`al|NDIYZBu8byB-NQaYCur^n(rquE- ze6Gp>$v`&0&|Y{*I{#>jH(+F8BkRBS*EmKEw>?8h%xupP60`Tc$dbQPd{oM|X$8%q zt-6x0TU)M6Q5IPlTe&9FJ1WRMtswXQGY_uxkALvtgX$O@Ns5a}ffMH!x(}Xe`S9W-MV6#3C9Bm` z&t(m_tZar+^V@WJqg2s$-GZrVo33IlfE8!6mM3anC#s7Fr&*X$FA?VKcY#X=XE9hn zQ&mO4GO%e<1>0&kOgN&Gms?|K_LYErgk>GG#OnEHY92l@{2}`Gi{Uq+3H^6=4__J{ zET>U_JBWtQBj?E1&JmV7ok}i;KmHS_hmXN+E)Jg?zIJC1OU52P3j0@wud^Q=K3Bmn z!6z$KCdxY=F^q!|-^4AGV&B)NDjuu1j1J$3;ry+ornYW^=| zx%U4P!6a%C=8m@%rNmB89Z;0gj2`_BwH|%8q-357e}22JQnWGv!$?Zx_ykLempMLO zLCbe!zv$G0ja3_ZJPQ`TYe)Yv&nD0|qg1pA%Y9w-l0`J=EqIAYf!YP;fSiaz^vSxX zE8JQ_HLKS(a9orVZZKqdPBa9LmmJ^jXbeT*QjtMM|cbx1PB$BidYR5mIN1lfgjE?%JQbFm1OMjzDK7A#tyvq==iBw(uvwO)fi3M;LUtGDoT$1?cMSI z&&OYV?eU=*iZL}wQqy!e$~}=O7bluPh1KJdB~Idacb-G4va+W$g%{f3P*yie zEn9SzT2#R*(v2+EXfG}otWKBE1WJ8Vm8hMhlBsWQo_;+x;Wwy;X1RLecT5)dJ9SCl zw0po8%9nE-_q6ArOt#DQ{`|8Zb@z?$SF2SeFJUc147AS7!v=Pxpa`)_z`e8W@g0*u zps%5U@)0(C`@`D<-|xHq_3aC{Z+^e~_SxG5Ew_)|zIl7__Ek81_4~cIuYZ5&`#tkP z-oO2q+gER2gJ5CU4^+UN`FzXmHz(Te-&e5`4wp>Zr4mSqin-O<_6Q?$ zLseUoA~#h1WuFgT`T5uZR`vNcc>cYIe*W{@PY#|4(I+J*xMU(S8&29p3_DE~Tg%vu zQZ#}s)G5&aqI%r3L>%23$b%lJEkm1ZBZF^!a%&(os3capQVw2Bc)`bm=T%Z#hSupl z;py#?Q?)X>WBi3%~2rCB;vNf=?yl&AN&5HXJBHD=cD?JX&QNLl=s;84eE} z-+`MmLzgw{yEpv{@Y_1pqxI`rR}Q`9n)F=kIv&Y&0H`DU%`N%l#8$oU%Kczn4GBkX6c#xHyxwG48=EXbx zGaM>^<<2Xt`iv7{cmU1}`sY#Fhz+V?J{EJC*D4xX%A>i_Yx8g}SOx zHbT(g1&)tEsVGyJVvB^!KPs{MpsAWHCx2GYwzaF;)^A_8eqCoLB8H(GtPC&9L`DCa z5;^1q%o!S15zB@sTSGItomL6TSyr)S`Es;UXT^Lcf3aoFs!glc{eD$@UzTXZfG@JL zzRXlfyAVc2Lx=Ac5P6B3mdT;!Wyl~!6?!_;GI__z8?&0fliRbtJZLG;OLFX)dG!)? zFvgSbj@jA{dg~S6@6U=gd}^A-ym@9m)H&EtC6*wUibnw>ljFGvtx-cXi~t}>kUoQx zXPp{7F!DK|F}i=`!pP?%UyYm{-QTiGE#{Uxq*$MiT!PP+|M$@7ZigicF1tE%5-!>| za=Oob1!IJ&xdJT};?K-8VBz`CIK6Ue39a4ZS6&}` zKg$8x{qfP35%VMxq{)F<1@~L@wuQ71TGfwD7nNa>?Ln zsS62`iIDgV0nJLmnw1MBCLR!xxp;@g1CF30$zOsyEuiY5_ z+DSb7hSfZU3s_EDKkQ$3^zr>H_3axcH;pB}UHNv9sp|j1KlQ4)V`aGE=-PW+D!S`v z%kW>Fx~RSrE93$ zH97jux0G0hU#!Oe<2m1QgTbK{;ZW7Ev?E(k)idNX6J< zwT<8klak5bXE$nDrK+VK+*khit=^2lk(krpjCJt6EXFG8wr!&amNMTj##U^mv1@_^ zv}d6+p7YH}aH3SLV0bov!OAuA>B!a5{f_18mHFp;9o-F2esJXcr0I?Q-8*_PfDg_t zkPHGmO>EGv`j)LE7q6K-{n)4;e>6D$*`=|sF3(2;)la+M`f2y^pZ2`^=*^EFe|lr= z(6tBG`p1vFI=27Iu_J#Od-c!b7jKNca&7$8{-1V#_3(qiv11n>T)X!0)zbm3SuqZT zY@X*UkV%zsBzg9YOR*VBG zq5SCgXesne%)hC911MA-(6y!F?DDmKhn8W%W(Hjpk0>0Mou zZc8}SIPd8HWst7_>gnm>9LK1$H}Nyh!KK183(QNpk(wr21>-n}7?=(4lg?6TVXit0 z601`vrXv5qq^DF@X;$x2^-%2`60Dd~H70i6BPPSg@9Y^q?}T_=Vl--Hi!|)sc^7uC z4WDa4J9iF$bmv`WOJG)n32Oo~AzVf5yKt%>p}ygx!{>&-hFhIO)&F~zS*?%5O(%ZI zx%D@9-eC4__I0@9w}aDfF?`*L#JX|kbyN;N8@}Lt>)i0*oxO0wojX~zOqu6cf7q+GS4Mzl+(qU z)_LpA``1p|NOKsSeFM`dB4+>vl=B+5g1?|#Ar^}@6B7-c?k0J}LUV_%?MM$@P$}vn z>JHl)IzA8Os|}>YOp}`pR$v*eZu5;%A9Nmtg)mh`#7u^_bv^A>GB37x@fa$s-{CbppWv6sqJYxTuQgPENEEwVGw=&BZ zjERA%|5;s4s`U~T*2-KDp(bIo@S064=hvjW(x8?Fa{E?7m7IU^mg+@Y&A3F{%A6{C zpHt(U6giMx4rGNnTCoz2JzaKE|6Jx6j}y>8nIRvtqor8HJm zv|WdWEiuvaLJYPIHG|r6xKLz1@ErATP60)kZ{fvcvPEoJA+Jcah*E52%L;LIYfIZo zUTjINa+tOzWt57D;uSr9rtqZ2XJLWqOvM>H0Kavk2(x6C&nHHH!s!!Rmbt9zF>Tj5 zx$z>G;3G^X#W0Oz4T>98-#pkTk-2!HI!(v(a-*mOYsfAxLuG=w?JJgd^jW2%zSMli z(Tg&joh@i1T-1}JDsj}Fsl8=PHB#V4YN^>dGr1Qw$cj;`vswy)~UCCA3w?8Q(9&R%$6Q5=2pSg+=f) zTlo38UkQPW3z25*1e)op-rGxc)rLjLHj$o7;}QlHtO13*rOGNMc^<7YC)ChJ6VoF! zjQ%#Gvk+2H^fR_X4WwdJ96HWM6#Djy7hY%~TFd09-3s>u$;4VdIlN)yCS%FC$V+`2 zX`Y}pm?$||);l}D9Sk#M!FOczw5RtkljwGu<_p-O?i_{$A9Ll_IXfO9jdOAhH`QVs zZ54G5ZedxC7-B(GH6g{NgeWL%N>GFw&@Nr2VJd4P&!vLL?<{m`(L5|k%g{!w;@&JZ ztT1sBxMVEiteiWUG^^sY(Xdi2!ElnfM67zv@$7_x2WUa9dCSHs$OXh8Xgf$sww5(v z-y-OD?b|nRUB6*ndk0!7GJln244uf5#(8n)mc;|(wcXYF3D7FuZ?@&UfO!k8BpR&$ zYOzZtS+cy{@x=8mU#;hD|LXKcVyM$dXPWuM%;}JDd`?Ug9G?ta`g3N-;(xSnpJxh` zAhn?%W}HW&Yc$D*G;IpYgd_Pl3DC>L@lxkI5}u7M_wY#WZ0@% zG&|3HzGDa8e{}gETKeeng@=dtKDv2$?BJ=fH{bW~>zHW#bPuppF)B!&O$t(+6KX2) zH7r~(#=-@ePL*bM+U&9@M7t)OA(vPB*g46_^f|%aSOhWG-hF)OjX$4xcPDbPW%%zF_2CzQG+n zJi*xwT{`^PokMqCFL&1PH!j^kl4h<$569;MU5;^*T$e?EVMOO>imEHbuyO&cS&a+O zkp)hwDIdIu*NmU@h#3yQ9h9jFK1)EyLA z`XrCddb1Z7;a}h6MrUW^vP^%Im=e$)doYL`xAyl2smrsG0Bl+iU|=9^*xuZ4e7anrX646~p-gK_;f~?O%%n=2seMo3KSt zs{s>V8ms6wzP={cOohU15yhAQ3Mj#9JG5Ug`0 zT29<$9n)Tk>D)Yg8iIB3@D7hDyns1sTIIUzRcJIlx4vmQ8ci#i+b4(C=WNq9(Ymyq z*DZ`HJv0Fn&+}YL1}RzKlA;?g7hImvN^xQ=upqZcf<;(JuNi8pOuPj!Pl(m>MC1gT zR`ceA3*+y##p!g(%<04x1tM0WKt#{z$eb)z8-1feG)-E!n+0Lx&@Paf{MGJg7#FozWg6|pFF(O(bXXihzA8`A^s;(Y8d-c)TqwK0zAAY#+;q`YP-FWjK*FPHn=;HX>SI0j3c(xzi zK<$#Pr4oIU8=al-{qZCHV~1aVaHapjmCql(dSLv>m-FjoS+^wz$|#Dr`;RSAv64?E z<0(GrVghLvXLO^8bG|AfC&jAO1;s~kfwdsX4V^bAW>%E~DWnZzqApv@;EZmV<%%7s z4bzt}+up(~NPy^k6IYkC!quwkw4sgJfDCA+Z`gik(?ZdvSzf4s7NBe?V+!QranE_c zjO5Hh8*m<;y-aO3J(L_9Y6+>d%Q1{8GnF@LdcM8Jl=8er;XzTm1@pG~o0wA)tP0-j zAgzZ7VU_RE8a=DqGZ<2y@2EDUJa6Hg%U%+XbBP!yCOJOkV!3Y?q?ujL(wq>13K5S= zwfz%=3GV6&k-!}OOK?(6V8CJ&PAlR5b!!R1fJkw%cwo{@Mkr>G0tK;SGpwi`6dE&& zRitUtA8V#E=vnkff>o?#o(j8gtF7eO!=SBpHeH$sKt+mbrxDzPxiI{%JyMF3V=%D^ zT-+rx#9b_vo;5l##--#aByLipqo!_JG+V%y4hO=EwurNg&P2_cCx}NB55& z7`ci@PK|s$dSGt@3T%3;oz7`urP#M-z=CAV)#0H`l33eF@W-ep zD@+(mMae#{VPWB{4(NnCFeEO;S5H=cO0JH5ed@urYvUj6c`*3te5|_q_?6ek-uU#< zp*`cLuKJg?oI95`zScbOYeAlgkJV{gk z6lsmqKcys>s)^BzfT{qM#%a6^wVEo{V5o6apGk_xIoZ6>J+ebqG$}#MGT@TghObf zUXCN87V5z!dePS5u{d_??OR$pw)b^xhx^Sl;iMg_q)@btWvCq+*nc^y71^E~wOk=k z0km0}=TcG7#*{W|=q3p%d`1?zWE9a<_D~S!X3mfv(ru8CObe{WVogOL-%RvjW74Rp zigCx6&_-+&0+h|0kuJVDHREKJniG=*7fS{pXedXbyuYTJ7#iN_wj2%2S&xGf(6DJ>keRb^P*B%}C{Nd?q<7fZ; z=+(dZ#&7Jz8$a#->hVXPKmO>vOJLw#gg)gRk4Q>ftiFoo3uHuL!wjAsoC0IhPEJx#Ja^1-q zxSQyVaV5mLj2Ah+roy9VD-JC{ix^MpGK5VwUJQaLt7L^VLNX5TDn=Q*s8WS7ExJZyosGEXepND8_^vFaPH^_IcM+{sL?R9xO`E^dP5;_@})#}8Hh25J!lTejhl zljtiH?!#_gP8{*&mMhuh%1agMOCVT^ahEFBjg@vVx z*6M6~#4FATe6^e%aqR`?+IzBh_mh+ReB;|VuDt}y;pKz#AuV{b9W+DlHuEL4G-sFY z55A)L4qhb#WwCffZ~h$y_Q`9n`IOt)}UuN1%*e7&@-x?O8(d z+G*U(HY##LrQ zt0nBo!lU&_I0=+Bl}RIO&@^(6!(To;+5c$Z%ZIPO7NS8&jB#=*$_$@v)yc9Y$L{Jf zVDV9c3vP{Aazv2cxLZI0!$w^VpuoD+z z=o+!CWoSD!6&=l z+(3#nqZgS+q64e>KG5V$^I6lf(|xp1)D4Rf*gvc4J-c=-`IY<;lc3eDma+y^KxBI@ z+f4q2f1L*G;`~QAn-~3~P zf^qiq8y`IR^t$i<|M}#BCr7XRvir-QKYkyks$ULXK{1Z!IljKNP3Ac{75Q-Ive)X0 zVyhPP1`MV!|y z`q_asvw8E5je&**koF_!|3L!O2z3Snx=PlVIu2QVlnQpGj^r3t2`W z;$&^7VE~mzdH_2|b$;q6v?!k@g}rs6z>|3TIWaTrN;mr2(?jpVM<1+6H1E z(bBNUxeQUR`e~Ba0%{4C`JAdLyWxkMb>!;E$&tT}?i)FcMm`-mHF^MzoMGgwqx)xx zW@&G&iDqfXc7~!>1E-&o3V0(IN4^@lOeyW1l@aa;i+w%33)v;=foI ziyMb2EtTK|$yNTYB9nU*$&)N2I#okXy@*#S3{6wRA@@wOwCIEo#Ujroj%QWdf*%br zb|T4gp7~VdYn=4|63ndc;RT2#SpnTx6wERONJcNv`a)S60!kv{*&=R_JT|f+DL_e< z;hd^bfEFVd*kCCwWp^QSRnrC_B3A+NtQbslcPmsQt-+eSfOe-v3E(9G#g7q zb5+&wFm#0kE@4zJA`&Oo=#+~Qk%=Xp6sXxKp*PTS&V!7V2wZq^Q3Tx)8>#I`{dyKzyg!DcxomQ~}_?`w*BP+^&KGRt#qyX~1md60bc=*J6T;B|Ha5 zH*Gdp5`^AVHDzsd0=-p&Y)xT_BUH0s{`$WWTE%5Kl;Gm%+X2LL@>0{n)-!A;#wC}U zCDVH5p9dN?J>T9`g=Ew78f96u?H0@4Yx*sM)?`L1?*_Gl6pAc)uqx_s+ToUMGf|{A zt=`Zy9oMGSs&15N&nCc4@SMbR0w3de$yF>d?qa!*d!@NgvYc``S#1vT9A`N4`N)~k z1EU972;OI-`$k`dAvp3S3p^Y-&jJsxIVIuGj=sz`_l+JHIfa~h>bplS!&gT4kM1A& z!U<7iv3&5gtE`|pYDOcU!_~V-&cmG#jK18!Q9K>-hNU)-?r}m7PgTI4Jkqj8S3xXX zqv~n00#=Jw#WIL#c5H(OqUC7%)l@AYK9%Hu?0<_-rBc3!JL@{=%gF-u0LejCufFnb zq$=WL%Cq>$a@!*UcbJrK4XC#USg9RWM=T$rKrabgqNWPtB1I%~Yrs%$4QTnH%cW&# zEA0)h)Up?R(fUPeXIWu){CWQ@N!?BB zx7H|6lf<#BDPmaEio*X^&^SSqZ zBP0l{(0l@Ps3gbB?h*!-iW6Q7P5?>*2eNxK&w*4nbTZG1mVfsNt1&1oaE#hUy)&ZUuuTmzq9+!p5YsJ`iHOFd1L-5FvFh= zUmE`A&hFtW!-IDY4WA!AH~bB2Ni%GmcTT#whkSw(S#er*AV--v*%9#oxQ^sofch#!{^cV?W<<6!b-_% zE9_LKnoIH+ZBDD$w3bwCvH}l6fWQe7E2bj29qNAhV^S<(RjiNWVxcIrOp`5TS&CP9 zdTh$6#w%6hLA(}{XjL4KVJ}wb(1jU&7_GHpc&m|hsQR7`@tYBAQd~f-io!hT*ngBH zpX50q#qo&*ELEur9=>zyQ>-j+JUA!u!jJhP%*tRQY>9nj9ds)kwIx& z$!S62@f#(2r&tkKDHfQp*dRqr4UnQ)sKZk88D!}wLrs_!Xx4TDzsu~!OC8eGm0LFS zm2?{^SaZxmot@91X7n8ECA3j#!-kbB{U_tZf*V@oB@30q#~DTGJQE(bQs_-1gP!Bi z8iNr3>@l0Dl1H2DLfWAUnvxdfnWaT}E`(VQY^>TBK{MVZ`He8<@?i_`<$)JGPIp9=3tOD^Jtp-W2fHl{m-`G@sRkU`(nbMSpX_Wvz?fA9lT%+Uf` z!v6Qd3$2!g;l+bt+d(ZDrAyh#XZorRcXm3LBkTU)7x2Sym)BtR{D05gA3RD?Zs@NC z_ADPDC3k=DRb&}MXOtYu+#fv4{(L}D_t2N{+duXnecGM)9l870KGl(G-@kUoHMpl8 zZn<^%{XxD7Sw^7_ za~BIy>BSk0in@Wm{i0**<_#OZ9enEN<2fe-wx#HrVjHHud?hh)*U%ZA<^Li$?a1Y5 z`^MI-%UcZ#US3<({mV=%bvsS-Ecpq$;uL3&rk)!}877MAAjJkyojYZw|1~CZ2yLXM zBUNoBT`Dy#II>b_H0M+OwPw|()$4x0s=W`SNn&8tS))z38|^}Pk|G_Em*C4L2TS^_ zLK<4i^O78UW?l(vZO)KH+o;(?%pP1Gopo}gixs<%=+Uu|VhP8|;Dx^yko4?!pyFbE z8OC*LG<=Fn1+Q$p;0sED)K!jK&FET)rHfK5%Qcw@9DJ}SaMEZtFcC>ZAQHK_5V0gq z83$;VsCsvZx#206_nN55)Qg0V-%|L{ncWxq6B^#SjZdUv^$Q<-+*MZ~?pb9O;+!Bv z*|l?6T9={UIZ7egMhtkK`m2#u+gvMIPNGM;=M>AvjMW&(qu4GtdgGs*eK7dh_&edT z38kE>zCyKdPc5%rB~8&mM}clc3Pq-_>4?2}#OP+s925~`P-abKWFs^AoYr;J_?*_k zlvDZ@{7oJ#c~w&&oH1C16$<}~wuLSKOQOv2@g(!ey5b_5<1;(o<8wOSMV?E=B7Ho> z(Ylglys8s^gQZ#cfcK&sWThf8{M+1WSI5dE%X^+=MUj)mD7G17SF7YyL+YYg+h~MV z9!PE#B`#iLU5^IoZ#hUw+JYqrzCM|&S?916DjZ6JM+&7BD_AVl5W@ILuxGw61u57d z#0c937HFnKo2Zi0RsRO;^)!>w1=n&ZRaQ&Mv&m9Sa$=%7n{+YGi`W1WCj7dNDVD`4 z*KqL7DrZ77SVZauy^zL+lXN_7cyz!|b~_b%SizoFP!*u-^-1ElUN}DQq|L3+4`>6` zGC934LkeNj*4C!!5H-P@*-bTt!m>xvR@_ZA&6S;BGet^2RSwwc$ozio*3FySnpbc1J#e<-Qh^e;i088P^FF+) zEUz;j*aWhmrZb%h;UMZ27@tFzs)xg2onm!r<9rky%M^PMw=fKCnz7Y}1;_^RQ%Xrr zh>G$}8zm~$2p|(mdS{B|4aRDC0z5Aq4$)B?e9E_HQYQL8tCB9OZ`vN{G?qT&msY76m`O4^+T=tJg4VQJH;8HV&31yRM=!ikso&Px zO%=8Po7C*vN{YH+p;b(K>+GCg`qOOf^>3ks4w+U|xS5~1&L%%cq0JO4VjGW2m`yq7 zKq1te`8Gu7oGfx;jX0R|k)pNwyEn~VVvwTx-J2}HvjfCJtio{4>Ak%}$8l!pxQSGVW_Jdw1Ejp3J3&R z*7ncYQ)?%zOr?VNsWizYq`;AN3q3kW&*oU5os(uwyL6R?w(5y7mi#IUk#gZyF0f)3 zs1{BLBtt~t;{QK;@A@6Zajy&iE46h#Y&j97>Dyd%*885N3m`7S1j*pUn-4WTrI})S zx+m4$!lRIPakVjaVjMe(9dLY)!8SG+u({b@>j>}9Vy)wCSo;$J2^5oc{)P8fEs1H( zjFj%yKqINMVy0)NXEa?;SJm(KJU=;Ud*?$WhRCkKLB&#&ff1@h^jxQ8Pd4z@v0E?R zI(qYqq&ak$=Fnja`~_@jtSE^Ze!JB0&%gWNub=(%t?-DpO#xqEn!(v7(nergf8wxN zpxhPRWzm>r0#?ISsxxBrh-yI2@~#L8*xiHn*eEfjY&Ns6e%Kfg@tril?6PZ%Y>i-E z5_c387&u7O8F#2CSrhNf{Xy~ho?VO*ETPcZRnLhh#mzrM6 zZ0=U7uj#>;^qkP7N^19((q%h!Zt9Cyrd~W3J~Gyx+C6|tO5wD$oEBPA%IlqFIua698jfPU?F-cM3(qk8?Ib|v&yn^ zcI|NFBn4-i6sFd8JS04{YXpoLa7`j7{C3caHAV}tn2Xw;yw|IN>G0!`P_-aB%rP7N9L<)Kc=5_EB(exF zFohoU_ZR+v5bDf4^{--;+RBPmZUsY87)%8^>YBQP`eDPfgfuFr1_Ll%vHRH$b6h7v zm#1G^3qjAc1U<7P=vka?(xqC9aTw+>uj`iS#!`O_P{=8qX(Ds1T_}iKh@?H<9xP;> z)|RDY`G?7GC$HW(I(c#O+?-P2rIl9;yg>X==spT{$H^}zzq#@9jUx!&yLe;2_wF~7 zXD7dcYrQNa5-$q*PPkln_;&J|r~N2Y5``}0>yu~U6Yw^ge8-bg^mHUWVaSl`%e^VN zCo83LJ-Jj?m>}7nfow+ZQM6Rg22~#DS+Di3_gOV{envVNKSSIEDj1td=jpi#NrbFc zTuWmbXOh-l50UU}emyQMvN)Y>3_XvA$QbOHI6XRXnp0C>i7g1Xx+X!}NtAwSd#zn? z6Z({Wl#EYGG!r;-qzHZ3BzGptsz`W;w>3$+?k@%GdM0p?aY~CEHCpxDc3P%R7uw?T z@85Av!;LX&q^MZc8n=%%9yQ`rFF9xvMxA}6M!6ZDS=1Ogs(YWo%G(4mX`ISMW_qoW z88l#Bf5**}8X3QSE_`UHjk73&WjV4A@2U@y8*aG^^tEaSbLGJ@q!l`WIeGn>u@rLl%zXT`!1>v)MjU5*4$Dp-B*}d z5NqNl1V2TfWz+w)!o1-jT0}!&K1R^GC6pV!%}!4FB~(&@gLG;dt4&e!q!KP7o)7+f z{HNDX{^RWapFX;P{&D`p2L~_w^zP9I$3BFuIG0s^I&wI?pU}=4Il&Tb^5oWH=!0;R z6ypg1!mcMLtiCx63KWbSwAZo~bxxWi5>}O=+eziO9sq%rr#H)e-n1aTZ@{qDN>V z3cDmXb%#9l*O%|TadGO{iK%0!?tk;Qdsh#~=CVqH_}FYO2hMy@zLzqG^7fH79*yM{@{L2ah;>VAF{NsPsa{ZYNH zPgs`WKPkN+DjinmLI9G+n$+m!ham>vI&$mPo9Azyy7ihzo&98%AbjNJxtm|Lv+Cy& z(EuAr!GiVs$*mXb6yWoM1T0v>p{L6ZQeoIGG1DY_nQvQ3)u)Yj9B|%bvD!q4S&Pe9 z-Iv@$4Z9pqG)5AvK1EC17C#;(i*r_p#j%$JvrkFdpDn<%qKA)H zea(+vq7~76n+pkU-5`-C&azu?3#`&u)e?4)fA;7WuT}Dv9oOs;jd>bpPhf|BlQ|0G z^1NvH``4|+ER;EAf;Y+v)I0q9{*y>B)=0%M;Qpf=iSKz#tjcy7Yul(x%`y`+$wZx7 zC$5<^@A4vCjCY~o6PVlWUt)dGsv{@+Nag`K=8|ao*evCjl+VphFQa<>oJn{W8WL5G z1#E;Rxr47Sli8>BLM98;QUB@9NwIi^AdJXz~p8L9mho<~Xf*RM{b~6kr!-D7D zR{GCl*<0rM+`SwwMX3Gsf+@{(Vn!%{f-kUz`kPO_WZfc>nkAvoaIPsUvo_Z%vivnl zsTXi6!J1qO2r@X^R2d(b5;ug_{R*u^Bb=GcnK-?UI_pNPJSkc$Y=zqDiDH43ED9Up zn0vZdHtHQ6B}_b-IPGu(=F2G_LF=H;Di&bAJ!4Sjn=tkH64(xtb683xS6n z>blFc&J-Cd(S>zayHjI}KK6i6d_^0QXfl~?$~!0Jfxhy<^b=JcC<(S|CKGv1t(5?2 zmAHC4<>(^CN0oF7ErbNBCaB6_4lYNsqAgfuv7D563O|W(w}PrQPAXB8r0v+4)D%u- zlPc(WJivwFj6sA`a!Ey5pttcW&U*>#yo2&|H1DwrZMO{_|NX_6> zCNg6R;M>UZ@Z%i9kxh&-7z!a!%Z0Yn%G0wNBH-Ee!??k~hn`jN%f^{e@CC5tmH}*e zNjP#?PA7eo7L6fPEVz2^+NePG(l*%#kt}E}g>$(?@9JN8JBDQzz2VSBSD(mLR&4=H z2EZZ<#jP2@M4R1saZ+2ff=E&lez<1dKQ(pi#J#t!&78P8b@kmC(-4~I9Uj?wNEDWq ze=q#PrFp8Oei-C>r{h6OFVosw-yOL0Dlxs*M0plIh}$f|6s6!x6Q%i$l$Ip&UXQ%z zo-AU%EZCAtCMo-xcer)JIed-ag2(7_%c57LrX*F!K3EE_Ub3Q>B~1l!`eQPQdw{`)g*84+1Vv)ES}P2 z*=whlURHh4>e-+iVv1&|;){xc2^T@-r+s^s43>k^2+ZOI6ul-XQC4Xzp;%vKwpCcP zbmE!erqYRLOa?y=4LfAE*XE-ypU9?^CA+Y2r{r?fX;WfbCX#Y3*cBc9O*Inz#9*Tq zSmgfV2vx?CoWofS!-lhr>Ihj-`{Z%^-SbYx? zXS=3|-d8#7rt?)WU1*FIsMUq6Dm4-OREP2|M9c!R0XUQwKZXnLrd_2vBL*7y<(4O3 zFMXG9*yeDr=W%wzK#B5^s#~d|&I+@=oUuva7lvorY_>`!+6m)D%{=8r-z=5t9VEQt zfH#7MGfnd}smqx4!eC&*I~Mw=sI%nOZeM*dLIHy2}cMo zwacK0vV$pUaMBv%QhUf^PTDq7v}$CXU0Nv03P8@^pI832$n@I0y=i)FCemnYjM*|Lf5rx#-?TC`fF;Aj%Wt-Nc4lNq%u@G_HBs?*e+F+IHTrz&%e5)sH)xa5B zEsM2O!bpp8E{~L~3b9ciCuRZe{LWEkR8T=^V6wdHSUkL&)kHy){Qa?mw#QZkrAd_n z7h5#m8@mWMXK@;y&AIpVTjuOV!Oyn4B`Jet6|A8Y*7o_I0FLX>|BxQhoSsK**;D!% z7;FTqaJdfU! zmk%9b<5UD6N{$};Tm2Y;S#$lIAte5PX^j_&6DL$@@Trr^KuZ@j-o*&@nJsGpqd}3y zN;c7tU1DVu4#Ijq&P<(zFII2goRMG?R0H>CT&qBBCx+W5kbIUxU^_vQu^haK=IV`t z)bmd?zMiH4hl0~hSA`E=wB`eZ@1CJ} z6sYByW2IkFQx9nNSg(pX3TNXJ5` zzyyTh)Ex#$cRa)_BTQ^8V%}0NSBEZiZ$JP}2qY#_9S`lfQPHq5J{@@gB#X5~J7t04 zpc9itm9tqU8cM1~<#yl@ae1?Z zz5nRQ(*WQ=AHOo;$omck-+@rgauMMaRY(CZ{NI0cwAz?9ZEMnma-~VR4i#e0=ufz3nG-jQ%r#= zmQ;gv^nt?=fvMNBcvY>NaB5&2G@NNtz+Ia-7<)zF&J^SS3l-9*n8S}if&T_lAfb?j~q%OT$fr-{z6h26W`SJd-} z>On>pl!Az7J;U%edfF&}b1@cvgBqfyC2PyX&g+!bIE|J;W7!3exqzYJHSGPx@UEwK zY}vYLcm#DyXgvcQNbyl2)wT(?q^yQG+CEFsL1HG3clUYWZut7}In*J_jXr;nd!3SM zEj0&Hj^7vCM?HK@H z3&H(q5@y>XD24zWn4hLiS<#$0O;P8w&r)QwiBq~EvV-gX-vyuF_V9DmL>0o2luKt; zTn=%BjgAtthDOL3^)$=FhZHMbtbTr`hVEym2ntm$dQIypGA&DRQ%9ghwJYeM3i692 zHwHb{CK66H&iIomC@fM_yO=c!MVT0JBBA0`u5oz^qoOu*=+xBF7w-M;%X`23{r&w% z;!ZmS<$2XH@IsY?rBZ=%F4~WD-~Zi9_fCD6l0u{6`w|~2epTe4vdF6F&LcM|JtUfu z{>d}_&iSb`Z%_a2^Uy%GOEyv!9ed=aRAJ%-&pE+hbx~6=+OMDECMn0}3#(phpQeOI zg3}yMwRYJ8;S0W1!6iy-kqakx&@mzQ1~^nLdp1r1W@bUSwH2|jHF1^gZb=p$1hJIH z@;=`&I8u&<;M-;!h1QS^zf7G~>66nK%`)TrHf-OwgH|o>pbcWE(RRyb4jZHY3qC&r z7f`ml=1<+WabKPIyL&fAowA~8ijsQ@b;65}De}=SXKEKEUaUvdnCS+qkO&>(Tk)XR zsai{~O4XLUYZRPH(kDF~zhb6dre-{CHB`Z(k)sQQ^zN}-2XQygJspt$=qP~0?uz-> zviju-E?(M7ye4)!MWo1?ZW-a{#;*u3SmRlLv5QZb)&6=E2WXZQ}q+18$(o00+RKe1aGLeOHSGgTn$umx>PGr5@cElrO1uI$`t6Nq< zT0;;sdkv#VD^xIk&=%NeV9PHN*G9Wogc_E{M8~?;2H2AJm*8XMl3=Qc;DNWn3#1fh zJXOwMEqZ8s)h}zVO%SP&k}GmraT)@m6HHUj?ZB%W7y<)CP_+@Kw#fa7@JfNAA@ZBw zOuX~k-&+z7K6VZ9ut+@ZeS(n_jchVC5)17DG_ zno4YsB`2v_6pH=C1=~H&d8#N_i|#|MC8%thRboX-!|A40$7|CLDR2T~V30N`{6;cF z6!3I2k!#LigWExMI5ujCIB3V{fN1atKjKM`lrE_pxt$1R-g?8Z_gdU3b?>rxxf+Gv z*6>AjsN_t2>UhTi!OG!mW?6)s)gggH4F+ZbLDiD5ETSl_w5uJiDw@V<(dp7_V&};d zo7tAZ3UTf`Dp&|=!yxo1fhL z{N@+8j`XZIs-^CmAKy9*uUv%37ykQ|TL+MM0ZzMo^J6&a@Xgcv{Ab)c*5VA*DW}^h z%?kiFQAM&fRG}=9@JZ{W9SeM1Jz_)mwqyg< zEY_k&*H;lDj9YnD7>7YS64zPeB}0(Ggag) zm>Baf^^8~(@0v)zedPnQBpyC?(e>B<^X%=bZ^Qj&-@fvNgZd^ugZID@|A65=)$odW zzi_EpVh$(10C$~_7Eo+j^=K|yV`gJl5avV=sx23c43?s+39KS8 z7&lnC3-xkRHm;wG35+bx z5MluCs3BfPqZ8LS8ls{%Uiev5|J=i*PR5ti$ppntk_2U;0lShL9J^k#Xux8y0gf4i>>1cOvIT9~)H^WodiTU%_8^^=%QQR<$@ewBkY#;h1+C52L`xR6 zNxLgy_4yaTsW*<_yYTM4o-O|TlWP3?`^&_WXEFnqf%%)td zsne>?CeE8^#C2R!4Zn~!^~wxi-l_=_MOJQfp2}9hJK(1aFCwA8|I=58fBNXno`3%F zxEibKdx`Sf&#_Rl7`sQ_wQ92^t-J zB2836V@y1vAuqr;aR6Q!o-3<~Cq}LbT87-T9Bo3Cf;uoB_bMHB9zd`P)R}w5i$SArK3N6aqYpE z@5UNEP_jvrTF>xb zHaW?PMZOod3Yi0I6wUrNM(tS*K2l{cSM1pou<+p>OUSxD*-zNlIZ)W*V!@ zVQ{7eCo0#ZZ5IZBnU9Z5fBf>y{&xX`sTYn+eSZ1=ci&7Md-sW7#u^&^oP2v~|4VQ& zsB`-8iF@ySbnoOV;cM=u-}!Fp_wP(!Iez!jOEZUl@0%Qoul7p`Sp1dtTFNw)L%tM8DMMo*N?FKQO0*7B%CrZ_Y;KgAu^k9xAZ5bC>XtoF%Gn2;XSEdC)DwCK7 zOPdCQC}=E;INcVcRGiI3&h4%V0Xze)HCBkySZiuKlF-;5{S7c=@uEoH!VYcF+VZn- z5o|{ZtWUWye9eQ>fG1B(Ub^w(L{+&DCO z6&^26e%Xa4k58VPedWgf$+I_(PF|Y4fF@s?Jq0clP7)U!-qg=(SW&?a#MzWD%`UeF zOkNtXEY2l)(*VUX{kN~4A?W(?i3>4S%cKF#7D~NUBe-|=be>x#LE9~T+^NBXVV4=* zBNVR-us)hpR2tVDi`VpVlddo;f01k3i#8U>_lNDW8ydLH#B}ZPGTGy+XYqxff=8cZ zsfAa)Cc(aJgHppMzVP*3Ur;=I)l+0{U+b#*20?t}ez|hghs?3Cw2pY4Das`H>4PsVGy(mlxs41Ej+{0`x(G>ON z{|Jq2>o0dB%HUjr#R4qi6c$Sh4x-isNaA-r`a58|W$htokQNJVD1BU)ZMjipmqK2& zcv2&K+?$20R1Ni8hUIb_^={v^!mPd_QbU8xFofq@rr~ppvjZ@9{eCmg*KaEqu-{f% zFc+n!R+34C${(SsSe%w95vAsF2C9^VC(;m%H^~^W+g1Kpk48v=<3~ywD~Won-{jUE zYFYpfX+!mJB%5214x_gQT0M}|XGsNCu0pK&DkX>0sYZ6Z8I;`nbAy`Y_JwvH3o)sW z!*y0BJ_1uaypgpH_L;KfOWoFs>*T=2b)rSq^16(yj<#inVd=z(Kq_h3tt`^*uPRuM z-a27Tm{{;;iJ8kO3~t9drD|A;ULCw9^6J*}HLGC7o3T~XSZneh6Rn7-S`nA`|Kr?+ z2Y)$>I;C7L_0y5V;j6~lry7wK?XfaVjepD#8KV)47U>xLQ$VuPoyzv00ZI)dd0kRR zi*zsL<@WKMA63j?Ny+UIit^O5xEN9s7SAFx$-mp({ws%JO=V>zG?W93mv^rB&Cu|Y zZ)w49%|&mfwKjGL;ADiBp`egf0OM7=SUpuH-T_ViPNTl7KQhZyCNoj_U7R z8nAU~TF@uurak~`ad~1^wN!HiKdd5hTCNc}nzPK{nwG<-?VhF0ZmA?kdhmjaek-1h+I`gD7$`x>!G=j!h#S4Flh?$svZQxo&q6 z-RxI{82gQkz_nOlEg6iOh42t)SW0KHB!e%b_{xhTbRGgE(m1Onk)^gqmS7lSj3;G8 zn>IC_Bn0bDZi(@DJlu?6gJz{fli8P?rQC9u9%FVkBYNRw)W{QLO=LZUYA=jMe7Pw= zNx_-u#FsS@Qnf|x32}M!0A5>w(r`MPtZ}}m1$Ymx?X)3g5e<(FR<*`_(u&jT9jGw) z`TJ%cR=8?!pT+kOw1ILH7NG(i4}D)WN%q#mivfd+a9iYXI(o0JwXy3Fotccpx|+&h zoa7>Jdus(I8l%WT#v`67N%I^3jz^ifm+*oU5A!S|m}lvx-n`N2oA-YA`R4!UK=r}do?w7pz(Eu+V|Oo|nf~GrcdxvB z_v%-nA)CAMg(aG5-Evn08}6zEhI=DqfvEzV7K8)_?E-hbDuO759#pVktsSI*oNbw& zlNnge-(>3*|5@>+-~g>)Oe<Hx`L+EE(WXRjecn=#r>S zojZ5;(#e@G58OZW&Gh-R@bvyy=kFaqboaY+cP}5GIdlpJHFf{uiK#Om#JQ}daV8bH z!{sJ(6vpiN9`yJ9f87c*t;{L2VYvk8yu-imKZ(SgBo)VCdWmu*z6Y=g~p!}H~~+rPy6ppW9cPHx?FQeL9#WJXZsRJAFjXb0`pVL3fsdeJ4ESv?y#wRH<# z>|Sq*-O%0#R;EM1S3m_=MlBoB+1Fvmt07(JvcqfeT|%OKMa@FdH57jOPRa|^MxE4l zh|VU?o8fVMIA*>bFKjQ!OplhrN)mq*CP>Y2xoaa!cMamWae4!qyc}7iHIa85g;m^z zbs^3Gn5MQUnNQs7IrqBia)L~^z;w%wHH)vMvD#Gk8Q7lRMGbgDkz=8Up{{=ul{LGz z_l)e?H?nK@?)C+$7Mkz#SS+{Z{b3DI9iMtx(9;=aM+hTtCWKpN&X<;yR9^yY_DMi| zGB}&)SPACh_&BpFHD=`*%p6waF;{(RIV>ejiG9WE1=rByX&#vsbUG!soF?zHJe_K( zCZs5;1e_p;n)XUlE#E>puc2X9@lx;MaFEj}t*1AYO2P63gQ}%_(*2pfp56^gt|zx4 z)zg<+-@lwP3jj&R8F^V0)m1{GcPn{oqZy5=Et@%;*BE z8kVTKiZPR%70frKY1$;gy9L>g6W>mK{?*i%r|*6GR&3F^tmt=`RuXnyi(X6h6T62m|nIHeV0TP6?R*-1*Ly~dzYHiL7%b*&DJ6c`xM!7#dKv)Tg$oS>pX zT-_4JXq?-RB;j0>X}ty)HE=)~hY_G%Xu!Juj+-YnGJgGBe5w>#k9e4ERHd@Vf_pvN zCO!hM2aVD87A!w5suHu&M$U>B-$Tbj!!SfkuqgVPfrm~HWCfv6YAW(eOfx@BcQ3y^ z^X>1Zez$+>?9qE)|7Ggzt5aWJ$mE9LI7UlNO5=2r*YV|b;G=r1d7Z3agJv^{rkKxq zuH+3GK_0t(wzCAbbCb|svc4^FJ&zrmIr4{lfBb6t%quZ2XmScGsm7fsu$SWv30jAS ztT6!?tOOKx)fyUNLUI!h0H~*RN=!yXs^_`qdiv*1>3MhxGv`m)y<3#2q1kfj4Fmh& zKX{2a3+kcgzJO6!dn}F$%+_Z~*WFvKVz;odWl-;`@Qx!_&yJCJoe1X2E zA%aotb!|cCiz0eoE70XN)y;aXBNI}>oGo7@W*6FG=>*`2aodv>ETtRcuz^duyh@?o zvZyg+Hlv)PVZF4M+79Ja0a4j8m|0#?;PjeB3cKGYS>0UZV{IgAM6ws<43?CHQnPqS&vx?$tHemA z0gOaX-F%)w+(Zs%U4(TAd*PFhihnKLO!P9^Lb>Ge%HP>MXf6(PJV!ldSd97p3Aw z0joOQxF5CE_1b1}r-XVdV0-IJI$#SYOLpa{Qj6KVO43Y^axYm-cM}D zps3$+xl<#&uoCp?7Ea|2#B_*VGr)ui4S}3a$vuh~%08*DC#$Sa_4MVkT93RwH_#(( z&<1+cY&PxNN23%}t1?zokyQkf+5OmOcL`j0tHsF1Gs6{}%`B_K;40a%=jlW`OmtR= zr5i?8w42tNl;Q&yK6q2Dcm43bzrS#N_wJb&e>e5{U#5<~Hg)vubI;wqbl~olD^mww zYa=cZh>;Nrxh@@;e(Q_r_YZ_dUcWTK0Hcb&+PEoS;EgEQxwF%j-BD4wp@4p|*z? zrA%60-jb$aNliGJ9Xm=j%PG+cvrz?>uWFp)Kv_?9rm0Y9(YeyG-8E;W80K*qf(Ma@ zFL?2}Q3lo(Dv%n|#rWvrE!=9-bZpwz(EyJpIVS#G{3Ojm$}QCCnKGlysI-VwcMJ`7 zEW`)CqC_~cOODrPW)3K@^k6B%l9a=e=8J3jr0nG2>skh@sm9$VF&m%UxE73PpJlMI zn2O9bxcQo_lhL3n?vOL4w3@jaHZ{%M4a}U22FE{lM_le0aFhLg|LdsYI>dAwxLl*k z%9d$H=%SLFQ47dX%=N%Jprj*9<0R8J&am`?0gGWW=Y(dD)MV5q z46p@>sdbVzyng!2mu9ZMIdl2!)U{WqkG?tmr$0}hyb>C+b`in=I0dWHvV6qVA+}Vh z+Lm2q;}ND$E2*Uj)-GTne2~B`-`3R76IhF>syCp9M7JKfO5h?bk4QQ0LUEXYU>OAbg!u8)s1tYw09D>5sVx zIs|7i7T)HB-cnjkgnUFzVBz|S0ys1gvMjCP$kgG;1cpP#a?-QUSMZTHXthMR1uJnZ zDzX5Hie)LGl=>iHU^<)8L&eL*L&I7sNk=Z0j-+S@8xu{Fb}?JGOq+!l{cUZd)w}5_ z0lVqtJVm8k<8oFCDr3<7GxX!fKfd+j=^y|2<0*K28~ylac=#JUeHR|Z3oA{Z3_vNN zUs^8OB~%#(;h)gx{P;dx|Lf?-<1iSn`{t77D?Zl(ThUSm=b9=E2@itJihSXuiQF-#0zlR<(nVFNr!Kq-seB# z*0C06piVjc)C!w{H_wYpxpnl`3kV*>^_@eV@Ob&=DYzghm&=8>!GBd=(6PqX>`M(; zDVGzXO-giOgw;*LzkTHc6AevVj?oO4Q#jXDS{kO5eBt2mYhd}@RWp11!RlPt*t(-(a?v*zKz? z#@E3jCls18NplhZ2E&PM!$F)IjELk%&bq7~OkjgCBl}d#ofjFLi@tQNA`ED)MrsD> zLc@e#Kc|y2sm1B$Dmkn*^{g_D;KAFCoSODtwr8A)nj~tXpLLJ0Jq5!W2iLMlD}

lBUl@qEtayGKm z?LrlYeol#C=ZVXnEg$>$-~7gQ^S0CD?p^1aw&il27Np<)7X5$lzkhEVh>u9zVac}F zP5hydt(83SSNIV+6Fe>3w;E=0bs)XtVDnB@a7s<~B%hs7tGGtnIDC=p1XYC_B(f$I zuZ$s`RPB0rZs_UZ$Aw0_a{RqI(HTJmJWpm4$J6(=s~F{7US^IHLG59E6{ZWVSoX3) zYpRCR*~n(~8t{CEso8~gPP}iT>&I_jeLcXdR> z<)85O9FDprAwGTnQ@Mv|1((sC-7hW7x^ z;p)U`Sd5IGehW}xdDlCuTk=GZMPCLiWwd<@pNgdYbI;ZH{f3hZyLUf@)}b8}ryY_v zm~GRFr%OP&NREhLs14SvO98BvwQdiy$|&VlHG(d|M#~tbHvA=@P2)3Z0x!{83TJak z%+T|);cAg#nMJtpwPHwrs$xA|%#&V`{dkfsWtevrr$3ycUB3yTE zp=raYUW(!vkJAe5-{tzS%aT$TN>VwqGAIfd!P6fYv5s1DXz3uwbS zNy>z`f0@M#U>Wpr*Yw%ITF$3Uk_(QTf|aHiQWB`}3}aZyL~U}X!){+aWA#O7&3zb1+ci9cvgXp;%(Mm3TxV^#VbVLL(%wNcKygWG2sZo zN?ss)VPuh2DD#@mXr%=%9ox5dG~m*)-Fjkln066^RZyqxz~vECwhX$*+DpPun`{ne zQdkm+5WfObuHH^o90r6;G-FAUx%K45D9oNbGvaGiDnQBMbW<;uL{q|#*ZtQ%m^plK z`s}NrVQQ0<1i&PLVW%a6EFw&nH%QfHHmX{pggc(yN0M<`YqYZi&l<3^r!Rav_3GK_ zGp|ga_;UK_@q3rPU3r~oW)7dZ``xjrYi~wzG{L{fWql~rK?sb2f^%}C9k!+$#)AXN z9J|6ykvrwgt$+u|5jd7YalhzLQ?za(@E97frgrj~mNBx-l6YxYN%|oySi(xwfr}(9Y&pVSO4JfyKq%8W?cowU~y|n<@aJ?1x}D5BsxqpvEfSZS){T7#3Q*RC4W6lQpy6PpQ%i(9_R8~7cep0%-Aih-^L;4%@CzzF9VMogsY8&-`D(WV zLy*Br^jg%_Gb_Z_Lit{r&ohFkD0d=lh~h+>6C=XHdWDymFe z8~2Ko!VUW{tud$9HxY7by|`)iNx-33z&A|%5#bvO#$y*6u&%%3=HYPs`njlt?d2{| z8dj4|Y91!6+=iXIt`?^RlZz7=WN=DpoELk@VtvyF8Mo}xtnBy#v-ZxxGj|SsaOdFV zJBN<8Hf}pMZ*R);ZzfewCNKUp|DA)E?i~8)&Y{zH4t{;-(4jjAzqoVg>YYQc-8pm( zjyG-E)C^CDqIN8VXTk8yp`>N8lvS_n*jGu3vtw0Vk0|RflRO_K z>)ej=W`T%;m8OXkT;-3B!6ncFf-@ak#2W=`jrvJzcDI~P$?%KPJ$?O>+%wRZOZ8~| z{i&YZKwp24oKxWGhSd6f!A@)Bw*~xiaju}gITkXAT_?Q3FW%);n4_1dVLYCjsBMMX zbO))SA-zm(+Y0aIK3yewCI}~R2AM6~dYGW|RsJr>(B-5hZzUC{v&p`=z}nwf>?+hp$M^14-1?W z-C<)CiJ$N3W#tWGmTBR)zkO7)SnY#NI16>y#Iv4lWQ>_MwIo>y4c!2Je!${&?81uq z2;?-JO*NWjYrEc;jaw!`{Z`F#A}B^Q7WGEKBm>?{?@VS{6|a3<$*si9^3k$5 z(_}n{k&t*g&&92_-O|UM8axa!B!dW%(M}<{={hTP< z-RANV37;!(Lrzxm`14H?Ie}*i{0jy_q26z07cAVSC<-0kOADUwwvy(~22S@7w1FAn z<7Y#AY<4u={X4h>7va(cVLlqk5mC-??i3?gF|E7%AyVjO153d2LJQX+=F zOGM1kkob)xfdEl>4eBV{OB$z}I%q74QpX@Otj7F#zd>vp4G~+n z97N0l*rJBT7ujz*o*8aRr9WdbkqH@g$gBnX=u_$$(Lr)?u*6lOK~^jY|G;V`jMp(( z&#KQ44cSXPk<&S-GBTFuB?PmoZ!*<$=Rs?L0HzGiCCz9283LVDs+PU3YH`Pkr6x`i zw0T8sWL9xEASjdHPM(|m2LAnW^6KP8IJz`>5lvp0JPU{4+<5uMkrlSWO`a4#{l<%v z7jGQ6aS;C9k0w8dYdJf4Sv;M*5I$FH@x9F3f_p``F<+idk<^xcD!jfFU{VTann(#@ zwxkiOiewGV?m&yKaZeUXErK}1_82wgZebnR;BONLZ9*k7Mv+Y;1gU4g&9@8Xd}WS7 z=aNuaU7m#7s#)M>bwSF!W?0cA+}af~08~_*Qj^lthHVFCKG92&V>!#kM8it-aNFv^ z#Fqmlrd1&^_SX`qWU!(n%$|j)6sZx}HfswgS)5ZFV`pm+ls>|lEp(;n7}Mcf+OTv* z!b6sDDwS*}5Ak4?JQou7u@K!QL8z88k>ObFWgEIzUBkT^!3fPu+P8|y3f2-boX1c! zm^DVaP(Oo#$cv|O!UA?!^hT$vKJ973D8dpqMxSSfL8429wgsv*mYYC?Ryok0jXOY9f&|Ysj~Q z{#sX3Rst$10|qkArXv@Tt_?Y)tB3C&J2!RW%)L*;bNAY%g(L|Ae@0GXL*8uBOD1(I zmdNGstVtP9OP8~)GtqVg7& zi;C#Im!Q%-ZK~V?8$bjMRPpCt#jE)7y)lsTPPM z&=cZaBk)TeQ>(Jbn6*f=KD?}MxT0=x5upL|kIeEcwj53;p;ytKgICG(s7N@eja}jU?d9T z1SN za&S7jZlrhPI+%_t3<3B7Mapd3wUI%Yr4v{hL*vlT$#_l5W>Y?Qwm#*Q;8RY+*-T@M zmIM`;>;4(K`SGn6ZeF^14&D6V*0Gyk-TZhZ6lX021U%+qwR9ws9o1fk~cNgW6-Ze{IXk_C3 zT8!SQmcl71vToK|N&lIw$d}Q{yd6fiCgDsrQ9kg}=aj<`bfIUyefC6#W0adzTypsKF^#0|I^!fT;Y3q$wh5v&s5i%n&P;+F zTB$HoFGnZ@YM0`iBpS?Tl7Of$MQ&1NaLV*jyXvzd<;b-R&Mj}MtPWUaHyE9Ep@|m> zhb^X(kF)4U%i)}qG&Nlcv3u{FzkmAO`@erFw&bWRDD>&3Y{FWcJ-#dtax9xHCpf*V zsUr&}y{0tAV%D~OMtPpFf-S~_!M~KF5n8PRV#_wp>TJ1yWI0(->$P;E*}PQYwrPde z25#5hBMZ8As!8B2>8rVnzM7w1`sdr9JwbgnWzoq#*R)8k!Hw&oHEva_mdzZmM__!U zRavmSla5Xu3rMSQY1|qk@J;RLJ{M_8lw>R=g39w=QWv^@)}(pM2(K&FzFkUEa8?bj z{jw9bRxd2o(U9m_|2&v>CNbvn*{u^k{k=^IpMGKzuMj!@gwF;}%re@@T-)I-)~=4e z{*Hx^bo6mp>g0RECP8DwL7+|m_#kry6aHgq4bCMIC&xuwTK~+)i9;bCIOIXz!g?7xK zuIbOgf-JXB+x4+9rUpsTvA8+PCoYY#vJrvexD5=IvQH^gtH*4#z(E^X#<}#PESnx7 zcG;_)HUvWqrp!9Di!kqK)Z+78xTi-)nV{ee_stUQ<{5c#f9mD#`n&tQ7v?(uh$cl( z_-y-^_19u7LL=Y0{#veA8Oi|(uCnNeo^Ga6F4nHF5&s?30V5mPm5Gl=>7G_xfsF5;jLHcy{H3qamtdCpS&KB~B;iwAwzbpGnp z(Lc{#lQ?+qRA~G}hfht$vILK+uS`*s>o&S~91z4T&Sn!4zxhJNiTBMe)OY*J7y2xv zA)dgSBH>h`Oy&cn4ISme^R;Ec0+teiiv?ln5;cou%Z)E6t*BUS+I7|-CUb^B7aDeL zmm6+;#i4*n8mHvQ$?a8yiN7Gk)~%`=8$(_eTx?lw%&DD%m?8^GL|V??$vxVCF7eanZR>J zm9Q))0)C4`tC?9~R@G)UVsObka;waC#HzIb+==qqn;LH#qloh|%sikqciNurG&U3bN;rcmlTt8RtLK`QpRVJ=|{|*dNoIy4% zi?dpzeGs_4`L?!=|09Yv@A6&7910I#>VdndYK_l3_O5Z1VIJU!Sctd({?)vM9_zE?G)@dk$bFD^iA-6`hAX=@$5XY!2p$HzfnoA^{VHGZ09f>oS*^t^){P0Z^9Z^2K$;JF+hLc|GLQa zghpj6DTBjm`;wudN%D#y$!Bwk{>PGuxi{aLy72qyw_lxk@s-dZwMzm-@tT`2j~cp4 zIBsJ?xPaJ9j}R?rm&}wR+L~t*{U)(yfMPi|fm(z*Zr%B=&Q?kt>k`R2!P(&3w@_xaDbb*#l1 zs8dcqwZeeCd0t$~tz$RO33c6DN0B&yarrnj#=j~r;Du!C89H$o5M2eQlX|!L`rPfM zi7O(c@FnV>xByorR9@qVoGPOCx!Tk*A+aA8*bgs%J9Xl(=>BJKPksK;4(Ke*F4rg08>zpYNII zXWgqivUmH+2XGnB-@fu)6_svZ5m(m73?pw5UO>8g`|4|O-PaFAO#(LVhjPX@H@znT zF->E+iN+duc21wXGWFJ}yH~FCpy_vCy#M*x)ZI&;-23>8so%dmee93-K05V;W42^4 zaOT|uGjG2?b>_{ftMA`G_51su{;B)^sW)d{IovHqu{)LRPK6gV-^?nN+QPZGPHWw4 zL0h8Pf+Bjd%bCa>C7ZS7b`kElX3-X;SBicyyzA*5TefZ*9zmUwhSoCxYvRKqeOv*< zM}w$1+CD3thx%NzK;E%p5Qt}xv{+7LW-dM=a$DXT1EaF zcSo5HUlO5srF|}(EMuh!k;Hac&>Qm1t1~A)nZEXEjE0GS;8> zZ(seWO>U@G?^Ej?2gFLj*<7O|8YwAv^0S%uE>6Gs#oenPO?|un{^zIee}3iOUk*>5 zdGp?hH=l5jg!U+Mh*7R1_3))0Nt6{xIESTFrW-D2`n~W97_+%JvQX1J?W@ZVjF*PB zrb@5G;9jU*a(U+Di}%lc5~H=PW?-*M%JrArt8~t6p^z2lPL;)K(w@`r6uG0L+yx8O zijl{sc%8Zv&uBgnH+1{zWv_XawZXoOMTsS+*T5(bu5;3e(R0u3+TPPYxW0eOu1#Au?cR;xuCq9TT?`{MTr(Z=ym$hW7@+|R zP^(v&ZCt-?!=`7~5AQ2dQv{!8&x5aEqewI^vGNA3z*J_o1D{^EIPRzuyc{jnzM8wo zfj{7&{(b-Jl7;|7d#|_^wsTs6S&Oz(Qgp*p+xs?3xrTzt*2Q(&()U9eXS7BhTi}K* zun6~Z$7_4ig(|Sja%2@znK-m)adB~bXc3hN7u_2R@G5sub>buB|9))oV%+;_fOZpo z+%5ZCfXqJc*uOOSGO6p#LLuy4BxA;5)wI`LrN>lOp+Qj5x zu7nr7#;x(BOH?6Apec!Pu@Ek%T+rHCg+-|$ZkHVbh{!l6Mdm;Gg1Ps<_!Kv>) zo!S4+)X{fm-um$WUOPH#rwkALkEhjr@PM9Hcklkc*Is%|n@nV|`e!%JS{<s`vGu$0SSC9`kg)7`s8 z2T}M=8!lOVfh?zlR1#KlUgJ45YYO&l*uHNE^@t7|#PFl-mdzYCM*o*+Z3@>>w!7v} z-L`RGzu{6?W6-vZyLV%l*ovwtD$MOpc=0g>p4v5~H;Z@B4mUy*tokVUqD)Gr??w8FfeZ$Stw@FCUZX;uIB{iF=uxDTK$1U@#7Pk>6w@iitNo_- zcs*$oX4FF6>jLFgkvLZPcH8aQ1T#1zMdl5{O172esgV0X14fhm`kl9g}H{@;50>RWmj+Um897WZ6C z;U!tZS~hsAjA7VAiJ0+u;#DXtSO&M3YBtmDZoNb+qSOGb@WHeG*HTc&<+cSbF4`mMRND8tBt>muV^u8xo_*R474I3Pwc&Sn$O zxo*x00a+5gD?>Z%I9YTtOH6EkSe8>Wu9iQlkmJZUaX$hFGQ2+rSUk+oD7>()!eCHL z@7k#Ckh}q&n>gPVM62~Kx2kma?uRh$+TJtRzi-fT;5!~!L)!LPa!X{V{^hmznguIL zf!YYG_@-UDE=QjZC`YdfiI3*`_JoG8K&|NVk*P0ZCH<;G|09=lck=C6doAz*$Mtbi zj&;{kQZ0IKu0dR8ZSXDZ_SFNT2WQEec#om}(!~32JXW|OW2wnE3cP!{CbBJ;>#&6l ziMl*S3^f0DpY(mGA!Su^^CeOi5&RM{!h&@GI3qX46zmXP zxcjW{-!u1+>%S{o>q>l#IChyK+bvu9xXUBMgR+A-$b~DiO@|-x>ZJnaU81b7YGw9Gz$v2wM2V8o0t1{0Vb8cMa3zg3C2}#kK>!PfR;mW9U|k{& z=-FdcYQg2rL1lxBK}}9cVV2duk|xn+;_9Vw>Y$C5;|L>Kbapb)Rw8r^yHMY*ok){H z1Gka#0ztjtc@_LykZ(#!A!ZcV*6D;z@YpACfirNmC>o|kH+%-9i8>IWG(!P%y{UXA z38DK#=v;iHZA~U48Hs8NoT_BUn>XIQp!r+`{aH6IeyS=|N2I3W=QRk+)7&Y+*i;kJXW9~Ui$Vge+CbE(J$5360{g*t)~ZU0*Y!p)J`BeXPP~ES)PuLL?cR;?qtZd0zlWA9QLyl)X?@+H@10*- zoO0&BcYNjLvrhuvKN?n~$Z7aBq28zn#uS#oSku&))f)k?ObhN6cZ27%Cf~eq;Kq^3 zZznI@c=g6hlb0qhqZsC6dv9EwI`rxanUkxE5QeZl&;lhM6` z!j@#gB6CShG8(?3pjkLeO|>rgfY+D20~9Q0k~LG56O`aYgMmv6L+lW}#7KCyX}f|b z0E~uH={7cj9o9>Gsqhb~vt3dJ`E+F=!X;aQ){ z_Yfz7jM)}mSQx{vhwuZ#l?F?*F-ggs5E69NDvUW>36|ghvvt=NTy> zw9O-b#!%=1u#1Rvwu!Bu1XKLMl(d9pPXi=_({V>r z05BP6+9G$issT%E$F( zq86~V1redYl<3_0%IOP4XL>xv^Z-W|t4$TTqdbHgRUlQzGc-k53ia->Tn8B<1t;XS zkZ8FwahfxVI;q({8~QQT(px5RU|7(w!--j>h&pJ9qiq)F)G)#WM)@c+{3+G6FNl~9 z?D|SM*kvLwYOB87L&RK|RkfLo@-^Fr)8ebm(S+`WswC^3s5HF(4w?A43k@=3Eb_v#X2)tz z-{$t7ecsZ|7i?o4%h)?{RH-#ROEb&U(`&N2TlNU$EqnnSAY76VAV9$3YQTw&ZSa*; zZNdJNJ*s-yF?IeTLN;Ji`zxGh^+=l5j4Zl)2DfE3*w%DUPixlt^;*yU_qfFJ7-*M* z&MSwUSA7<8QZCz`LKl*GnG_LR>Dag7TFwZsDg4!PlCDoLJ8FL%mR~APiD$uKU?|#- zpOINkrzJT9TfHJ>m9>R7sJbGWO~PAtIJO! zhkteHEh~J!Tu4b7C6|PHq~pOc=X84Tyo@^pe<-$!tWL=V8OFBLi0p6wblD7{61yV2?l?B}L8v zznL>^3T+soZV?yJZi3D5l5c6!e4F9QZBkb0lA`BzSxR*PS*{1S-2Ts*FFyX_)E6iI z>ya-`e{tfAH{j=m|2p!;$uG`(KR;Rv3H!@I`ad6jap8-TVjN$5^u_z=i;v;uWfcHFUz$7e9~c}~d^@U7De+zh z#8Q1vMNi&AGXW+Fl!zJ1WlLZq~FH^%U=~FHg+6hxlw}Lz0*$gw3JInCRn<6 zKp6z<8dqlC`Sgs5`9?JJcwBj0dR9_1Nq|cz@dh;o2hC0jc72>nBq!yP(uZYvATTuS zuzr|n9v54BI-imyEfd)_anJkmYYw$AnsiOajnZhSN6RNIh#}G)FeMzaZApl!`HSbS zpZ&}Hqo;42yfpvV%kwXP(l6`U{Hw2Df92BGm!ACka(J7nSERClp$1DWQQewCX(QUs zCT8BjQ*kb;w4`(xv+FNYlw-pYw7v-Q5)6$*AoyJ|Uo;vf8A($k=Uc23dE{_f6O;}y z8a-)I!!LIv4z9!PY*(#JS&3C^WmrOXjat+YU0CB&w$q}LU+wgkhbW8~Iph#-VG{{f znuJfPi>G63&$fRsQjXgeRv&mL)9ZCq$v5^UKYH%>SH-nbwbq(S5~ z-dG^dzB>CPYFuu7()dWcY@A1pbK-3C;I7yF+L*Fm5FFr-3!3oRCt-9jm@6J@_A_V$ z4C?aiQ-UkJaV30;%or7Il7<*Fm6TswPv1sePp_)4E3%|?5RN8EgU$VMS<+w)7wH5u zX?$seX~7(!MrV>*Uubb$5{Y=kvSZ{sObem_ExP-6ok2E9Crs9>5Lh(0GEz!Uk}O07 znZka@nI#I1z((Rmtm!XlK_`$+V*4G)2rdJwNTYH*^a2?*C8^nj-*Cas9a?DB3qSwj zqS5lqrFd*CRgvV5a*)6_7AF(^dj_`j13GI=^WJ88E-k?hf=yzh&D5+;5Zgh!%p$Q0 zE)Ue-c8hF+af8+(cTw)iGRm$%zrU_~nFv}b zc+iTj;e-dKyc6}7>yF259yYNpAcFn5zVA!pHmTXI3E`Ik+| zu9>|P9(P7Z+W~gJ5QBD^)(&!FRBS}8#UaT#FZ33Ya^wJK%v)89tzue@VgkdWN~sRQ zy7jm`z}Tk_Mpf4~qKCP=T_c*5l@rk}@>mPFfH$HsItfxRd^o6g0$oc>*^a2)`h)~e z*{)Z5R3~Pft{2g8H`5-T3#?;-B-xD1e2Taw!Ay*H+%b)+SAt}C5N#Ve6dHg*S}M3D zMfSzdLRXom^Lk2>J3Nx>z{B!@TgGSyHchuuiNN~X=}d{B%EH{|a*3b5%^Pf!>RsTe z*A*#~Z8tL#|H(4{$rER;UwZGx`{50c-Z@WnRmy6KHQ2YysvG4c?}=*}*ooC|qU(~{ z(XBG@@&TECN$ae_#2{ z+GA1p`qG=%&mV@7g$IdNk#jJoWV@3+87b9azT?}qE9%=#>$q6%YXj-pJ6|5V_TIHq zUmpGPi7y|$P3G-bXXfn~c8k>OphfSF{M&1P`||j;GhaS|uDx;X-D@9A4}y`ksLUoeqEXC? zZh;z6d0e~2BPS_(LSeHSR5$+q`nNBC7^m+kBg(|PBpAF$NKruAFxwzejICbLp#U0+ zl#R|De(!8zeq&2m!A4w+X&F)AE$fjx_O2TZpd@-v(o!`}6+qS{U2XT*hUs2DBwDpO zqCq)~04^)-;oIAW&ij7uiifJ*h5mEb#^O-);&cI& z+pt$`?ew98PB%q3*x|*y!iBz_!%AT*aQ5{5d~3U-d}%5cFv(;jJuB&8$hRQTTMdoH z%V3ZwDJhjode8|=rr??wO%X@5KZA*LB3D;+sRSspq;_b@ffGzBjPtvnFsU;!lpxVY zYrDsWS|nDbI`gIpC#Z#nDT!3S^H7mVgPa&;$7_eW07HQ#a`X69xM`|cp5da(9FqSxo{V>R2z63g<b0)WyhRAS z_AuBNHp7>9u)Bi3kn0GnkJvVYne-_PSxHghFU+-~`4S;C*{8`h4+4UgmQ*bv_g>vp zt>T_BheR?`l?>QIiJ!V}kXpsamQR;PkCKy8>2?fiW^ifrAjKBNXje(tI;g^`cfWeK zDj()3Gp0xdn=q(Nm^<}~``MBDCNXdMR4xyb)*5hV#KtJE!X*1c*Q(+RU8{kGlA4mV zbix#PT9nf;*#NQoAg!_sVKVT+V7_e0&MceWUrUL(^NC1NZ15mOqvXu{ct{(HeB@ zL!5!N7eVo+RUgdhmg0~Ot~P*;l}id(#a3AO?6>_TT&1Xlf710{=;OIe2X@?PDM`*m zj?qoZlA1>SXqP+5o1Cp{s6=v*U|y^;PS8GSP}&>bwS)NtX-&V9X;EQ&uLA-gOIfX5 ziB3GHexSa6>Fo7KUY)=CW{eI~O_kK>l;!t)$kg^2_jr*w(PKhwDui!2g;-6Rlva~I z4XeA{CQLlp%n$8ZDN7_hTuZM|II$iTfdvzF666u--ad_ng=qZMjzxt9#Ny_;$C0Qz zx%tttn{U0C|Ni+`zB_*kegB7JH&48RZashHyYnBQ?~i|Q>-dZReB|lz@msGR+i>%( zEBCiPv3+nCy7~48-+g>x;kxnh4Yyvoa=-Wa7ytU*#Yff@qVRoh9Xs>=%g51%h0)(S z_BxCzeEjIv^T%$zef-vu4`7O05ckU~S7*wrW1N+;N@T@*y=;!6CMr&vfQ+DU$adJl z>u_IO?pKAEZhVX~8A0Blc91u$LsYnkLc+LHkx*+Wfp=FL(G;YrwnD-PzagYW(*-f}dyxUe8N+DbUfqrKQdTd_&P z*D($gUbm=kg=A!3ZK-z<5E4Ox)ou-i$?_QzSw7;krDpE+L$x9XDS3z72ffUl6Mm4b zaECg>6(LK!%yPoRn?FBec9~`~X`eqqYk5Jv*8#!FNs88Mlw{ub4HBW&Asbu80^y}zc~~Q;d4LJ1c}jvw3zLeCdAyNx03{=7 zsmQ$9KMfN!AFwGNAOD{y)Q8q}q`h9_QE2m)1=?jf=S$HGSh|!IBn(+S@?u>tqK;V; zt{rPEB%7C0Ql_Il{CeD07aST6$HJPiYt)Fv#grAb|8f$YS`u^|*Q;*t`l&?{bF$iv zV|-l%;meFw2&dSJkEJOq3jZ~^n>HQ;#UdJ{ti*AxTwe=3tV^l;$iZf%uQR3JJD=Z6 z;RnLpp@vE9O(TbDpwg3KXsC9R3aA)#V+bC*Gw}rlcfYNg74|fI)+~$$F&#M1x6| zvZ;2M0>?JS+#07ca71mAqHHS3{U zy<|o0zY;wRxK6}Uf!d%2#2i^r6)6=xh`uhIrY2kmdihkW&TUndl;{;^_l_{x1*#mw z#5T&Xf{2L)*D#M$uUnD2A&fU0@BsNz^)e`j7#ID;#sK=Z&Fv*MP1nw7jGVU_b#d{%L+mVE!Do_LZz8|l%|4q&@*C(wvG&K-?wf1 zHqohybH)*@N(rH{y5-hkj-Tx5&9=4lT81G^>G)W-SHcw7n=H3@iL%V3>NXxi@3CnFKCGY!y>3x|VTH zOEV|qwdX=JSnd-o;NTe?D!beetfBC%rIl2_oX%#$FYrOrUqKRe+TIx)g~1 z)|+HqXlP}}spU6qI(YD)L@3NWHp)(aS=6#%)0B#26Sa!ukW?$zHaP@WH`z4`-;`Ix zeb$&qeq?ccXe%{~#4ZyW-tOLsoheeq4y|*CM`Ql#mAS+3&K-GX?(l1KhhLaG{MYMOj$A)~`1%WPee>>%p>cVD zl|@B&MYN{#1rA$r!=T9MvXtp)jTzVi!Uve^2!n^uoSc|B=`~rf(NEFB`TIlz=bA_{ z1*?cz4mv5$rT@HMhVjb`UYoCnRbP3M7vA| z-y%ZhWqV+`MO-ciBreYhZjjCvkn0hRCC^cTlL?a?>dL3r?Gn)?rGtkeOe3ey{$#oZ zG=;{nY1R=%2kQs$R1s~Eb6NN`xtwyp@F*2ID}a^DNfB3!`T^_iPuyGxQ`c>$v)_|A zncv|WmHMFRl1)P<<`sgrU=GSQ3!hTEbSK{eOSQ~c?{z>l(^9&_Pq7}CShE7YN=lrV z(fJ5^B7?G$-a*rx7^+K#Y8i7j6=Od^Kqx0=)pjl6w#hJkn_;7Cb{XySBefTcHc8a3 zBG)Ri6=3LlXsoXdhQ5a^DiRAtdr!|V_T%^O&ZLyp5P870DC1)H&S_|Zn$;E5?7n^5 z`j!I&`=V_y%l7WH*qc+9a=MgLrA(@YPyfp4fvaQ(<^<15R!=J4m-aM#h*sG_YD~pS z8IcogD_S;igWGmo1}U*7iRBXXlOIVHSe*85Q5c42+qG?xYvwu>RyY>U=oT(3zn|Z^ zZMU2WuL`h9BKQpaschE$(fmTcsAA zAiXLch=z$^SV`Q~K6f>I*6?i9cpTO_L9voavV&IN%9srkj>EGP^l;(iR!9>LlTLugNr%ecfIf6A(+}WTB)T|c~FcM75-{DN!LTeH4IDA z29wEC{T5^1T1!LlUI)WVNv+U%E3g7L;3Y5o4`hNOXN!j3nA$e>G8>46=~eErf7dA$*0Wqk)+KQ)Yae_1x>|9gF>A}8>;Zg z85CP9d7Os^+C9!gFb5NiTi%o{;FQiv8vHNoN(-jYY1p|uDCctYh|RsTcov*870OFx z+D*DzFIJA=L`k%OWjopwvq}YAkB!xJ*-a%x!U!cIrBn;oO zVl4T{2`Y+IvRXuzgK2vG-!Ph_92B<+lSK}0cDaLTiX>}^PhdfV22BkYyv5Q5(^fA8 zi>69BEpp`{sLeLR2h+_$e^{`TE4XIdef}i&Dd}aEq=yH@YmzNlVFPYP0PSi`9sO&6w0?U6C>! zD~7uPF=_mH_DJJW<8tGZ#?{7o`1=WJ{JrtX>|r={zH!bwc^=8AaiQ^X(9+ZXgYog+kzL*y{jc#S zZ%iM;j9h43M2**A);{s3NX*H_(8v#0Mb93z-K165E?4~NIdbbEkb^%2Dm(=3_H5wP zwufp(?1*(aP^?nRi@(_dUSj3@`JNfO7Fn-W#kf#=2=paAQVLONQ#O3&0md+q&CUbxD;IV$5aRs_0_bP1DhC zcU9P^(n+D>G6mj*k;l@*65s&UT}wjVE$=qmrRLeuLbAJ~FhktCY zZvlx&v~D+d!w{GL+(lTloa?2-`$tL=-RB!qd#CaIj+u{aqg(-=tW7zotr#XW zN}D$ipwWIg?OQpT(ke`79t4C*m$W2sv|JJq3Uk4WPUQYm-7cN9l+_Z7dEahEs%{h$ zvxmD`R>~xh_#v6#)aW7{(GyT|Qcg?KM*ViCi3`Lu!OH1Hhp9-Q$g-p>-DHi9@|lyS zTl9)V1j*b=Y*KxD_Vu-4(YFWFgPtDR>&*AionwYzID+=M=oN}k8*PxY896+sp_-Ny zRg+{{5k*ohT>6!j8;q|Ciz;ayJo4*st@z}84b;mRvvve1qJ>GKeDys@T;3{v6J+TY zwc<|OzChx0E@I}UC(<@51yaV?JEqlEtT8n~-Vxyu=>nq0v1n4T&x^4A7|!I2M{xh*xZO5P$IEe#d-A4NB;cq!#i-H zi22N=(#%;ztqGXDLueW~#5_Q3WX+tMM85z26^#@{+iW!G!oFd_h4$3U75MryCvA#~ zGbg~d_Feq&!(guxr^0wYy!h<7{^k``Y);ELB{Qu}E4g3)y6rBLGbdoC;nVx~4{}`e z5l&v8SEP8`n`E%! zK=jI5R#LU;Wk;gL`u_c`!2j|~zrOOPZ%&>WAHN^X9e#iA$nm)&PoizZc|Dz#b17AZ zvwywu8#w#B{rk7(cki8^f98*KN1nL;(eb&%FZ6J!-kChsZy$dHzWm(LXXlQ*FM1Zf z_f+TyVGHKCQl+l1rb%u&bH&SmkAWtoI-Kn5eE(!M*D{(T)@v|}>@_|=gBs0QdT3!Cu#P26TfyE|aY=<=!zjo3BOKJ2X6)~s)6S36EaOr%P9 z{f`#=0Hr1PlS_$uAD?C0GHJBHCXJQ`yD~}US|nyPuxUnTW$ua0&q}GJzov-nvZ5$; zs0vm&fCcxAD>D)-dP-7KNvT6?oe=JtBsSUwfru>W9o^~!E61EvLEs2FY|<epTIr#S+@`wtq!ud;Z^+nWpweeiz+0X!w?HkzBpYa#2 zsG6^E#XAVN>NF|S(bzF?ebEHt)r}qpy}h>9+d!3~?aVIIDxGq}hg_M}Uvd3}2`x)h z4jhEg(~{bus@-<#`v#fd&TUtx#haGx@R>JDgj2Z7Z3?~mhDSOy=`Dm;u~ok#I`j_$ zVmZuOsgAhs!F}xt^n(ocG_<=6dwu^tVmq}mHvMK#HY>?$N~qLI$`>LC-B?p#l#y~B zSr zv1u2*0J>TsvM3o&B?$RT@pUM)?^Ik zu7%)VT#uj<@1zxib~Bqz_`vCxQ@3%PrB-pB0X_;CD~XrcCs7LV6bZ{-<#tEPXTrSf zq4U+HHyGM6b2T>azbtCF^$xPbz&n}u>i~x#?2r=U^}Nc9{Z%S>kGy>v418Pwi@z+c zw-L3_h|5`xjE^rtxNkJSb!d7kbKo9pAg6Y5r2Y^>(6Mg?g&TVx4T(0b`e07CoCWE? z-U2x-X}Ls&@cYw>*rw)W*}B7cqvdWl<-d&UK|w z+!OV$E;3wH{p#{FXmaLt@5}d~_*HAIA1(0uQ9uLNk`gRmrn*&f?Pk-n7)uCbQ7N7a zo>$loj7U_QIa|UOHp&j#0sG><}EeW#u*AS}#FcC68dbobllH3v330xuv>*t%d z|M<<}b2pxT{5ApU2Sz&r(huO;k3u~bp+fPAC+7~oHh1{z8}gwC7M`v z4K-kzWbZY2`-Wrg5oIubJv`pzrwsq%1-zq-C=TQm~mgsR7qu5jG4muRk6!(T|Nb3>E4dq&Ah?T7=sI3v;k&P4WG)^!$p^qvU(RYx-SEwUw7^34xQRE^rUjIcODCF?&Mr5&rsO_4Kt&0l8z%W`w9dr=e zVg(n-1mgtcK7qyT1; z#!0g2GQy*yr@ndOZ}X3S6dI)7rI~8V3TdY6P@nY0T|rT?q7bL{M4J@F^w}hA1V?5G z+cv9}O;Qk)Ixu$lpyfobFX=^?Rnw)MnvjHa!h`D|l4C_CBc-!R0*T)+rUls1m>rj4 ztmPy*oe+chqIY&p6w}9Q0P86!)1iy6Lr{dg7p3)tV>C&5oPBBl6h%@ynh8)RASecB z7BiVpT_&;VI(Q1cD!uFd48znmT-%24>T7`RJDDFUQ(|LOq-YFVXbZ(KWQH&rGnj2- z15b+nLg5{;*yRlEVN+AoCvnsk^O5&DAa1Ip%kAEz#Ku`xPas{=%skZ5&;f zQhMSuUkbR^>ZK}K`@fLFeH}W%!zF< zIJs%WGGX-gG+^f7L(~@S7fog((mX#tj_Sz3)dGcca4*Km9io2eRqimuWtYPC2;=%W z=D5VB!f%mF)78RoWiHZH5LEK1l$7nL-doQC%jNJV9*G5LC{j+1oLYZxhnc(ru{~w? zt4pV>jc5>~Jv_cbWJQoBsCtqx3`bxTE^sQ=;1%JM@%iQyk3XSuKM)3RspgWu$z-2 zI25#wgvWKbP?mgi1#Du=jvV#r-ZiHP7KN0agnoQd--;&+rwICltlS<<2yCQo`WJNl z+$UhgT|f2I^_M@qe*D=R7d}~gWkKJ3@ap_ue{?cP3!9^?|z*Mb-sWuiGO8m4xo8Ov!w z)$vw-5SPQRp@%FwNw|&8{%sZ*H^df{oz05X1@Kz&oPg0BfG*q3a#}Y&PQc9-8HUk| zwR$0kl$4ZG+D*zNjIg{JIKfr4;l}x6^UofSkIE9Tk|m{sN_-v4sU^dPyYNdRHy`Oz zLMfnEM=ut>x80eTacToVWopt|jmFp$P7&k>n%wTo#J6*r_3fs0TrBssX%pAp`SRGc z_pY7#^5~aOeEH~Y3hu`S`r7d58^dmqq5+Nz-tURO?*7`}zC3>I%$HA~Yj0e8_u9u| zTM8-*Tk-1~`Iw`t(l*eE(glQ!w(O^$C+gt*qMe?XmAM50;X`$Y54H@#@Xf# zW}nHcxm5q`k=Z{Y(GdkTu1ctJUi#6P<`&WIdVO||q=o&8qs%QEVCbiz@iN@+lg3q; z^mC~3TH~~U0+4vRajErmWLrhG*?={*iXddjYLQci*Nx)u5Q{|4knD*>%t<+=qY>2Y z5*zxq&;v7Pt#*kx$OWz5Z6x7HXIH_oT5~?)7lA{VbC@OWWhfYw8?H5pO>EHSDBGo9 zM|;5m+X2H3+bzkL1CghvrK}1P%ZH7xVNk$!L5kpTNGGHweyK;B)haP&dnbOE^3NgBqSR${457P>!%#{w*N}~NR8e-J?C7bx=;`_h!UC_IF z@H?Y7G{?XR+O{n;WD8czFcICa{QmWpju2Qy&{9c5C~qy1&t7xzXRonOUo+i}Xgf7y z^`vXEl<8=Ads|O>-w?pTn;{(7p$(^9zn(vJ;rc5_=bw4vo5LS}ef77uQM9*i?@+Y2 zGQf5_^PKGQE8Ev!J@d^ofBX8vZ@zu$FW-Ff^4FK%n!kAN#)Ydlo_*}=3#VX^p%K#* zNs(j?RBk`xquk2gk^^UN!L2eK^x69iVA*%E+uE*qI{2v*a!YE5tmVbN_x z#H2)gs#XRQmY`q2m6k0sI~_VnyaIU`vTKwPw$WyG=ofu!Ay&Kxo0S%L;wb+Bt)gvK znN5UG{@O}-VT>Fkrs>l%qHwAzqS8N;OL~4aH9sPXWJTi$;!Z|(mim z33nL(l^avTRRBcraYZ&{@9{ultqi-dQRa2Qkx@4z_sZ*u6v|22=n>z0XXUhfbIi8D zgpJiTr^}L}N3JSbKRO(C+*pGSx*};EO>-+OAWr-DfB(eA{C5{l-+JMLO^T9|<+&u9Ly7}yjH=lXx=G#~9%mdLYa}1{M z*0D3Uo_yuzQ;(nxw_d(@>&OSUUOo5y6BoiKHZ4w)Kh&RFIntk7^~unrR4U-UIJtXc0zi?aR5~G|T9s_~!rSw2p1kqo3-iA{F@NFcw@1S_we8^| zQY0mvSmaBKr~uc+yMRodKX)Z?2#*q79!u_aCRfy8MO#TcEg8`KwERhfiL9jr#S#fLi3_%m>JvIfaTW z(IS6jEm9R!Bbn$?g7x5(HnGdSF0Ao+)c__DJ4hI=ngWmmDpSnDJYGn0Qm&&l&N}QM z{nh2?4EX+ET|Q%=Lo+AB#|T#JA+59u{FQ(O{%T&9rL@+rCI;pOUS#rf;E?!;{5%3b z+oq^UC=y%lbjKNxS_D;DkxozpY=Ekm|K2eQ-w7PI!n6~P#wyL98}`631_^e|W(%gS zS^fIe6+Oo4s&J}Os$+!(c8hs%4pbah_~zYMnJ zR$MK(yo4Tbt6Zz>F8c3~j& zLT1TnNmJTWN|OyT2$(a6OU%J&JEyiYb9SO9dt!tznkr~FluY6aY4SXkShZGmZDf<8 zYdCIvdBK27PLh>&M=vph=PrK!{C__3`9=79`SVks|NZl;=<^RhzX*RX!@qz3{Oo@o z{`|~;KKA+FK0mV-@)Vb2_4$jRpNBDA7UO`?T>AX4=<_%J^KtJKynOES3!i`d`PKh? z`t$Hk#_a>!_v{+;*}%RG;cU@j{2MSn9r1!MmY$RIjd2M4IUW z1^~ipJ);qd28t%7Ta@Ek)6OVj*>d5?)&)<0d{cm4>?E3Q;=8wPd&53to z3}7i~DVIrVxmx}^a!52Jiq#lWvS17)3G&Mo=^ExBW{4@e2Un|c!lS07l-ACidLU@w zs;I1c?9utZUR-<1^4#Ggb4PwZclfo?lmD}awr|_D)29r}IX^c=z{*oqkW(lfy@-=s z>-JphDzzw!bqiBfP{)$6P)n>7IGKczTULxE!m6OGP_?dTMDB^uP#s&;a0mynVzgfO zCO)LL;t6U%;wr#JmvX6&PESEIoxNWNOJ!&w)j6;7Vt>^u^YaX5Vmr8CQoAg!w~@%B za5<}y@$scqwr@1Qb!d7kbKo978e>`SqtDoTi!R(%UFtt|m9 zUIqlZt9BjhM}T2RRIA2F+o=kEl5|4KzRbDY6r_j;HG$^Gbc*a z!pPtR7X<^3h5r5hhacWd;btBin@f0&Ap^dO?GW^T|D?YkjaUXQkSSbbsK5XJLDLqi zf;30B`|n@T!w-)RY#DiQVCTrMe~qST73R@eCF9Tiy7W6wVPF4*ake=yvPTyq?8sVymt`r!I34sqj%LhT+hp_ z>H_1mQuY>gtct0oN@^}LIO`x-Z^PY0y)(0V#lIU3Crwhi%PqTCKxw8|TWpf`mpEuD zPB%-aTU64Ll1otWqk>7lf#Eg`6#ZCwkEw#^B%SVvyNBZp5?xj-gDf%z?O+CI#-OyU z0pFoWs&7H#rPyyFY+{0wI@-xf)U*iSWMivXV245j)NvDJ?Csc@Pk*w3O8o$A!-kS;mfoy*vOKC-;{s_)*e&M!b9Yl|vZfW?XuJ)@j@DV$O1X;i^9lF^eWbKj0Sr;DA1O>v{x z6hckVifE~JOadoj{uE?oGp%tAsc+k0Um`{XX2_*v&BYA9R#J=>FegS6lp&d1vIv$f z0^;(zC~Jyy9jKD5wi`y=4ce*fLgRAd^6b;1LD}phjnlJ_H7`5ko#w!L5XZ{Uj6-Yv#?FHxZ1bT!fzshgFPi zhj}NNRdbmsDj$(eEM$Ta3ROouxg@Lt(a~zN~_ux^U$_VHUsK7h&ft+@yl@fy4<=}DN-C+t{E5nD|Vf@Gt&BfHnb zO%Aoalxb`@B+gPCO)!1PxuhHBQu7dz>QIbr(P&aOdXmsR2nH=-%jrS?zG!HqS0MJg zeG}iE48H0lEfTg&U*GgEXo#7N6B~(8AO;&A%y&@7F*4mLv|;y_!TWF1dhDT}btvBt zaku2+`dy)Xd&629)Czj{&W^^u!$dTR!eVNe*VNEIXG=})(~_>i)Wh`Wa$UGj0l|!< zX4}_OVDI%%^Oaao%tE$Xs8ZV&xhWgG5Z=J%p{>*6pYd^&l5%&70x>#z!yHdIE%>~{ z{FuQBJdN9|Nn$%_3ukUE^lBSBD5%)^p5_l`tN#DQ5Foo}b>e`tvk0Kb;VHn8!=-Nf`^2;YevlM{nWay$%M1CU-+kbPw)eml&vG+RQLy?j6OuI%NxWf-T z6vLzhiVmKFuSt+Y1UCw{2o~UDOZbEl7c*!OFAIgBenHk&^nYjrJR^kDN$k+OKOJTH zy52`klj17iY87q9(FOz8I1ZmqJi^fsJPX`cbk?t}m`bdz3T949W!keI60ZMcJMp%m z8-MtC{^h@f#;8~1{#srablHhs;#28OY_M9nZpNk4>xz_(PBU~5A+ivQz%Xr~$j1|k z&^1X@B4^Uw^GW!Om$6Zfkrq-D)EAnZlxOzspao)@Vhfva&B-`TT|FaZ)9q?pQX$q= zh1mBm9~Tv3IXU(UF+B(3DNz{wxj(jW0P^@4uk|LuL(&dNM9fXEmxjaXgb4;;oS3MR zswaBfl4fnXoEBqI<8;B{knM;dC4TaDKRI>?7sW`iDG$0R`d70_0kYpu&P?0sfr~sE z92qIo;gVa2a173zGO1B!likD6qQVd_1HzD%a?$(AuAlv@Ci)b_xqZngDVrqx{xDD$ zHDmM|08WaOQ4%X=nV8{pidhaeW8uQr1&>0yqdD+8gk(uyW)=vXqJShOKr|_rkjnZ! zYuGB0Nc2l9Z(LC{0Xdx{kuJ0JZMRS)hRZqWRnLKFizOpv6P{TwYEuH)z7c))XUcK- zT*aPPEpkp$qA!y5qlK9T3d<%=eOZyEbkfJ%w`{gz9^0}}6e*qUCeM117ECI-uh+_T zk#vKatsZ{SN3CD$5T%q%Ae&VQs@9z{HM~ZPj?0M+*ZlAQeP>|nRnpBsky8Rw zg?8e)$2A)cw$0R>If+VcwcyLP@pT6djGQK=qnn-g&NA|uGj5gI4)3BQyGw;YNlU7l z^q~(RgNSYPOm~a6?`0)9(^cX8JpzjmLNo7dM8ja?gx}11B7@YFAP`Ql*g;%_abBht z*f?dkXful%acXL+EM*cJbf0@h03^IKyqW3*&wy9c&?Wy>2_c3Si&K{8$;Ath%Yd-J->65j@S!8~9WJwZ(qqy$%SFCgs$2 zfUKYZO)%FvAkv+=?GUrrGgR(OKDsEn9AbqCIT*u~SN4vyCV%BLrPU`sEh*VVr2Ddz zT=6Pah9X^*T+2yt-|xI6)R~GL+wTb^IZ4m8i}k=gX!h@K{I2m@e&&Vg#wW8!8dnbwxK015j*`v*WzkTZM>+ik<;|mRz4%&sTNQwehkN8!X_E=M!3i!KFVV_S6QT~5#>m~zcu2RDBRW+%Uk zlc{hL4JT2s0j7ISKhRn{K_S^V{aP5 zHrX_aO(K{aNdGpQ3(|`P)(0D@n?^*}49g&Efe1=$6>YQ*OQ-T<#BoegjSw{+HC=}R zGCpakrdo6!-s@m!>awJFG>eCI5*W=Vy-22ku1RXTJzFVoz3i<&p1b+h zi?@!yc{!8AWM+Bn%e`EDF$8ce5g&IaPAW6ryPF%!w(G zrc=#Oq6{OmMIi4Hn%!*u7Xu z=Tj*u*U_XSaP0A?4?ld+{`IeaTC`C=8Z*H70c9Dn8iFBH3#0q)d%!IU(`XFylGo#n z-gn=RB!Ai%q$BTmyTJT5NePumfxEQgHCkIhu;ZfAS)-#RqCl{s*aHNb?oI8vW$BJkLEGccrR1Rocz`W9?jS%O)- z3<$F(rMn|>xHtN@7@&`rsf|kR0c=Gh2=7$~7XU?<(jA&e7*4`1*lfmnX|*c0P8zpu z8(KBo!^zYXwiv992+rTl%(j^Gd*&?lqN+)cYpPq5~L@ewkY0)V@ttB;qeLF;$s>G_bvTF-&c8cS3 z>!d}Ac{-Jda}ejOW>ZJBlAx?scFNd{z%bK0i7soBmP&jJOTpcWYfX_@7}x;|UCPMq zSOg9mz#`3~eXeC2p8n1_I!JjD6^Mn6GN(q_p(W=o$7GOLt~2E~MHkQH_sv`>bHdSf z6T9J6;4O5*S=P5GbXi$i6g4XyQJwX;RH%vVn%jLzjM_5))xhSzoRySxyS{bX6wN-+ z(j7M0&#eAoX7#mMBgz9m@82?Vo4ldj5B0Upf8TD`w!L2vf6`Fn4A7}IIh?#1s5mY*sZ9wd!ZCV~*bd<}flnL;%Zem=d7EV{G~fu&7cdoxrjj7MQsb zjNR%PJMr7h$TBr>iJ?KbF8ss~3*sgYU;lxB8*a^f%SRvB zMX#i#R4VbbEME>qItlXKz)l1!rH4yGu+AsYEQiKmBAgYuk9SXS$w_*27P))8g9gVI zwOIk%h|?l1xK=UNH6+LvI$F;pf#x42&^(4MB$BQy)@`i0l_Z)Dm?N4NYz}&KHiva% zu@cLr94?I0W>rP@M^{d`dm)an^Q+6xApX_mm*4^W^f_X|KX4|-FoLQ{DK*hI`gUNa z%!xs5LFQyr?8He^lNQ7eaw?&zX;BPRF`yD3!vG*6m{B7ma_=jEb$ozWpFJ1nn#oD3 z+U~v%T!66YU(l`NFWx-y>UU>P+$ONFeRpSIfz(`3hzwCtx1Divcf=ST7}Y}KAS<#Y zXS0$bs5CM@FQ~er7gSvh2|=`w?W*wjUXb-ebqiaLuwP9OC!C2A_&9R_HAPa=?WG}s z&7y~DMeH?`9w=6Y-tS;A05A+vvzb*DrJN;l00StpL)>J4gxQr z%nH1ZfeE4jw#JFs!?RB|E_+l^Yp$-h@w>+P*+;~87tB^0QR6Qn|6ukhkG1JEYCPBY zM36Sk9u^c%;#BBW*7&{0`gFE=Bf)(sH_17vZU+nNpcAnoPXh5fA!3bk3K}BtcD)C@y;Wmf$9|l!u;5PlLwP44S88IM3AlXQ#e+gdj@{y3xt;< zua<5XN>Pq@*M@9a;F_>siAEI4@dt7C4!t6U>aF zp!H1jh|>M&=_Md~22hdOxLTkkx7m#a)==9SG4wSdhL*Zgw6$wkem+0EY<~>gMj&Nm|M!tPQ@{(u4@F#;Gx6rEDh2Y3$odOuCe|VAm+8rWtG6f|`;tS^`YV zvok?W7*rM8=#T)&5=u^tEwlk4Wd-v!v5VU5B7<3?|Ar$nx8np$leCVQtQEAd^1(W) zk|Ltk1Va-XBVzqX8bP4`8WD;qaTQYoBiAYtj`r?E6ox@*krT&3axSNbMrNeQ_(XyF z+C*k@t;mA+Iv_B*l+6ZC&*~ebTvSn&akbs>8O0pez<8lpSf^-cj4!&cfGrNIST*y!#|#duMJ-u z!ONFm6t6cALjy95K}C|dBO_MUwJ4vn^d#R2+$_>KfG?%orq-NWdw{nP5 z^8sXnHp-@ugSo?|X3ly|!i(5}5zd^dRtTCoX(N-&oZ>SlY*cj}?7$0*FtXulgH|fu zg#=*}gORdU;!QIro0H>@HF@Xq+nHfkG294ksD5_Fp28J+fB4)w;f+X-b(N0 zcU5_W(BOk?(g&5A^R;u&1*C_}c#HvBWOaSL?3zVf#Zwgbs-r=qJyTCfO0HeL9SB-( zJOB8l`A09z9scXw;UjZLp8V$B7r*)YX*m4$r8noFzI^@o<+;Nz%sq1Co3{_&ICt#& z2gl|PzXR`n^WF#7UwnG*$dMbzPQV3s{n}d&;N0Qgi$6zxi#8~!lsbR$oEZPn$L5ZH zFn9F#b4PzOcl6}k(LaaZCbwXYD^=<$El4$=O0Nai>2-O9EcNy7#ek`Zu@yo_5Dmzg z_LXMbE>&U$@wJ0hut}$2por9{>G5_@Qo|>#>Gn?+%i36x&$T# zJ-B6jTmreGYPzcB?nfKoeWIXD}VM|i&&_Ntx|+GjY|{JdOrG<+pxy1LK+7T zNM;36(g_|kPj~ILIN6L3;wc1Q3R`F^hHp|t1BD`d2$#`LTrP%}eGgReA>T3$pE#p~ z8PL%#D-`$JpJ$JGyk=J#mmB9BR~r`^=h5up*&_>0+%BTVC#Z35_UP=R9)B8K;=K#6 zHX``<evBHgH=b+UhGh2cg0}JJ#u*sN(Z)xObFG0NoqZezE(R!0 zT=K}`7J24^eIP$(lt~4?`y?7L$stlld#T|t&L*i*4*h`qz|9sPoRs3@%JL3^JU&@L zf{{@o*Q7Le*R3N%+xKnTz74ir!a3teXtyGy`hTTm387KfGMt4R;SwV@DiC;+pxtDG zaAFz6me4R0JN4r4%x@WaaPzi*8yTA}i8?B5iaQb;?oTHW7O^srCRIcZQQLtJ_upRj zqYbjG>8ZP&Fd~L2Vgt= z=VQ-~j}LKzMXre$ZnLoIz5@p;rGofVELH5%fl8$~fhsl}O`rqzfeIqU2@{zU#rsyu zX4%}y{JxoE*q1|Pu>?v`3Xrk90zqci7D>hNGo?Go0if! zNd~i1(S3fO@Fh=K!QhdL&gQjVHbawHCDBJV#x5gSLF}Q+iG$9+9WtvJb8>7}uzP47 z!QR_eGp%5+jA9cuND(AlxpZlub2}U0phr5*w#M=2qGg;z9h3a}*XZGg_uxY?pNG&iqE&I>zb?M|rGaan zQ&^mVp<7_1{42mcV8Pqw_&CLIzE-ZAi|4sh=B&nwP zFb~8^S&6XV?t)_#uIr@?Aj?JvMang;5VWUE&>&-0kr|cnl4mon6AL?FlA7Wcmd`EK z5PInhh~>tr!>Ji5*TDc%3Cg*1^W5Vo4|v@C=-AD-Ud(_0{43v`KZU;k!?Bwu zUO~5>zw+Jr5776=Ke%=L#eY8X^!WI#SC4JD`PP;DTc6kt&*#mzKltwB3k%nck8im3 z%9Z=Q&%gNB?=C*FrjUj2`~A^ZZXJ6SZIE;6%*{uSgjk2F4MT=CM9FU&sPxY{_6 z;KWrgE4i7ve0BDj@o^+lmm6o`)P>o{W}mvtv26Tq_K%HA@b~X9f^)NfoPDN+Z0CFV z#%r^Wz<|%g=_C2ZbMW3LFwjpLXT`gl8n41TfA_j!!xiwx>1L+#>>rmhy5avnhJHN! zXtW*=T}!w0oaRBW^)I+f(g_355-I8gvz+L5L6@7crbt>s$K(UUB!}GS?x=3j04tzF zT?v=uL5k?7XbZ>HD(1*26QcrWn*Aa;2Ylc=-0dd9a%4t1b#V3nXYbvgo4E44;eX{k z-l{ib9%!p?x@}Eu`Nkz+!eWx_3`@OrBppi!wYrtl-8OPa6$V1cWfC$;CXkSsu(<%4 zkU)SyfRL(jW`E4eRZkZ4%uf@?Tn1A6{t55*v~1M6;B>bHjqTnY;#R9A^*NvZp7Xta zzG$NX_L_fGK(l59YpBmv?j)t^3zu{35gbKOrAU`_`<_VIvRPn0)J}yZE~s+ESoTY9 zTa9Kz&k#@&LZYR2sGqExA*-MpdCp{5S$aOFr>O?wVxk)5yl)uRliV215LgCXD7ccJ zJR8GGvcue`XLtu9J--Cqe#6+Z3S=CO6!7S1fTnm-JSL^(n3C?7`UlhTgp`5_RpRMH zBAH0X#X%)45B13|+o^p)?IQKGNlp|JZz&at*0ZMdtU!5LIbma=E@-MKC@s1`5#aM{;0$Kj&=GbeYuueknln=8;NDgO8pgLdY_>j!2& zy?&4n3C*0n{xS^ZW#9Pzw%Fi!OV)LZq{lW=NzUey?5xrCvLK;_J1(H?-!Fe`p)orE zc}ka9Xp@|{Lk!+MZ}}{KuD}!{y!4p{>Sf%$6)@FFiSP2kxYP>|E)0Pc`ByDDF9n#j zI@%1&AgTBUb{Ja|u!LH)=#Jqg)b>H#gt{ocija&&Y$A1I2{T4<&}FbwVT*&I6jfWe z_auYp8Y}@c+buSO0LlUYlK`bUaJY^alUnmxlo6HI19@QPTg9Vwl=+Vxw$Nw!&+ANa z!i&~!6OmK5$*DVPvuv33S*f%b6OvK;qhG|yV0j8hdbc>Wf?*{Ik{WUQxc0+JNjI{X zmBTA8090f_79)>_yzMJzaV!b&O0DPdO3Gdvn1Js?087m{k^ z+_-0F5mw=4{yCl!8IG|fCIu-LSP@u0h}_#;3Yh3c5~`4-gk-Y0qq6U;NGHHPp9JZ& z>}4g#lv-jwqV@2}3K|(F#0rwd5(7;)mPq%CLpRHry@k$hRrypoV_c|OfEStD6yz$VllvOA31yby{~@y`uz~KF*zy3 zqAn-D6bj8z-tZ~Zn6;b|q)0n!$c*HzXR_GNg%mtcL_v}w&d*Q#K;+mqEvlo9QtxkkKN&5_T7gznMHUh*@*8O=Zje$j`PP}q+mmnIcQA(4zSv)uk7I!~4ws>4W03UctM5>He%mL%=NW}G2J>%9wHbyNI z?ww)&$2YKpC72OhN3kwfp2f;x?1OR)p=JuJ2_YU80oG%FIbxe( zX7+TZOfaYBu^(b&`n@OZKRJm!D(>I$t!A{_(OJSL=+$N+U~ zWTDp9)j*LVV!K}sC}j-TV?yk&RD%kXGO^hKI_i}GN{Oiw;v6nnjhK(4Rd7RTG)zr` zranMB4VzOtAdTnHm==L!^WMdXO`QbAlsBdR-+<{_JdP(2{0P?3dJO2~P+ukozlIBF z7#DK%(QivVfUT*Md4o|`W9s_3^ftH^q`7B+*SsL@F`O$=)1(=yQMecp)8dmiHg%HG zQ5t}`^0)9wqk%uFp7H7>-?jEwsDd0-T-GvNNJX2vqOC?7?TVen#W0064r)l?I=|(o zh7_jGQdBPd=js3C`$OjMm%hk`sfr|F*K0PUJiGX;A}VT2%-gz=z%Z=>>J!itMnN|N za2>Cs88kxlQIx|Iwp%`wI_aHr>R31I?V6Ile$w z319O9=dV0iyJF@@tIg}`=qS}gHRr@Q;1uOSyE6oBv*4K751-4ehDcISQ_YstPJACkdNvxPc283* z{^pU(b03`h>C&Z}FP&^3lJ9w9OHUIJJx}1WLoMBa#lrMHR5fvS-+|eE`)2pOgjU@+ z{K3t)F1Kk|*CVES8lmX<4fh6M)9LBH5Sx!lf|yVQPNo}k6|Rf*%|`UR*+?-VE;n0K zp7nK}PDJ%_P4u`Bg7^hR5XL=aeO;&aWfXgpU&cTKou~7FM9v1gV1} z)j^TGgCYr1L@MNpM$uB45z`DSL|6kw5yW`3tGGd7xPtX?L5T~Z6qu3M@znMeEt^+1 zQdq*!NVYIeOpCBG$Eav^H6!xj1PY5h^Nx32M;r7}7y>|G78%xy1w#Fkr)*@o2B(lz zcM3_KokCIqcB;UMc+YG}Z`-`>k>5tzkUDI{8On5Az{OsZQ#TvC{BJ7bJZiwU)&96} zsHfjaUh?m1I3xun*^%pXxh-LOmWI2l$T2~RM==IBBqq)4Ar)-aAQ6RBDsbR-rKHlC zuv6k}XcTERq(CK8qxH;X=$CMo!lBiL4hex=FLGRnMfqCwe61{_W*H8`al6O~A)z#T zlB06JkiFVE&Qbpv4K1a9v9_g3i$4!3K~6>Mr8`e7L%G*F8)h6(5rvo(c+FfHVTCeA z7R}J`7>H^}NrKc_JGte7N`t<^a9|yMOyH2qzF% zso{TQHOeuZM`gk|-8|3Vfm;XxuPz^+zY3sEf37?D63d zg`^a?VcR33C#~i*CB_)ZA=9!m2sVV}ra4gq+j&0e3k@1N)ScES?YC)V@rOkb)Trpe zy#Z*3H32(7Sk1p`sKkX>OG=w(?Wy-yXzqote!BdJ8&}?%d-K@RV>G$()>pSadHvSw z0W_mBn5t;ZwT9MLP|uT}5E3orVJlE1ctUOtmsX=;tl^0~W&YC%x+oDTL5er~GCgl< zw$mQxpXc`!WWyH_-b7;DU^%3bQ47Io*yNZ|f@?5iSkDvwS#D=I)A6VaP@8mwK?8=# z;xXT-xjuL=*U(j&B6+M#5qY6{Yn=Yh!Um{Y;^mlYFWGFXOMp+=ZQ-dkO1$!De3{R$CONby73zR=9eGH}o{Z)iX>rrx0Wwv6L*xsvyZpAr_10g0Q5RH@|!Zt36?$(&1+EzkDbumapQ4m|$i3+wy5t({-Co;4g z)>b16GgyL3IE{YX^T+3&1L6Pc8_z%gEF_)@ooV zLcE14cV&Z-rFsY2S*Lh_4&51V=W=14vFysRGj8F z(}RSqC%Xa=YV|#9Q%-U0E^U=XR}GSP^pdd6M4hr1?T2;j(%jVxbMIaK>FZB!f8w8w zq+97AB5y+9%Jm&NZ2j7+WGB{h%s^|wT8BkHM<&qJq&9{bnFz~mD~UoZ5fM?tD#^2Z z7|}F}5zJg3o-$5FSPF?Hyao;!V8PkkSB%F_!?4bguz2@tNXbH~h2_;Vbtvc}+ZZ0D z*`=UtG{UgeusNIQ)X6efYQTeu^OhDWUKoO=n6y1IM%0JG3hJHTy+1-POf86uhjvaU3-@oe4 z$5$S&e_~_*ljzPTpZ(*)#rf-Y>{xa8=#|Hv&maE$A7AfTQs~0{{(SiDyC3aAtL8_4 zcmD@4D*tQa-M9DO{p8@?y`RApyB)IaLl)xV7^eh9Y~HiH6HlJFesJbf&KLQ@0)gc9 z7t%voxzNiwAUPE!{5|luR~JYjUzquN<~Urmf9B+F_Z6J3vgHc2N{TgsYrkd&zY%WXl`+(99-L;)&EA=W}kbYK3| zHeLq)jdNe0OWinnP!w;x^D$Zjr{M36Kb*ex$?J0;oSA#$<)z?*c*Zs0d8Fz&pEo~0 zcJt7ITYFyk>B{@JPVBky{qx4u_=oB}{voG?L`%Nx3YV}@g-eMEaiv)q^X$SKFcT-DnH1{H9~%U*NOb;HHb_>V-=;uiHJK*#tg~Hm}>ULqMw}Sy5$GuC-BnNJST2 z3yJU|F*Ixk${MTBgUh{_Xz$)jEdGcj1SM)Q+_0RMsU9LxrN~(|q$mQicmyO5X0OHj zhv+#*#?27RtpTr;ph(f~*R#$jl?%3s3Y83_IXle2lo}HfEn4~tgruLCnXW~1gSb&u zNl40pS=V}hh1S~4DiFpU_k`(KL!42|?56kOq3rhXTNchxvKoyTHt+vun-&92D!|rn zjOQw`nCRO0lWq6?tNV6>xX3nFU z6qJZMyuoa-vxZJ=7Jv+@i?Dzy+`_SM7d^sKJvP0!`epU%^q%VZ>NnNXXnGP(z{}}B zz{^Y3(`oeFbJgF&t4r|jTe$2hTzF}EZ@Rj_dbWD8`fc_6^xo&6e?YF)u2uL||5!Z( z_j}bf)^Da?uY=nD{dDz>>P2UQP9taV7g6;nym=Pp;vD?{RsU|;ENtXvvEl+fQ3pNY z4Ri@`7ZNQMHW8;sxV2XWqU+fI2HMICC$`TSTl9gD1+}HEx1TiU^*mPKByuc;Fk|D) zZ!>I$m{Gz`3K2ZH)HFfS&jwhRhf0IQRYK*Pa6sbJDyfzxw2^3rESl6ZN*RNV(}^w! zXppBOE~xS5r^2&*{_k4-)znE|fJM@SQzx@iCrm_iq#I+TlmYC;R8n1PZN0bPjAoz_ z!r9vl->~%Mafw={opUAo1*oBuI7LXT#rTL)syj%je(&JVhcCiPkxHp|Uf3UIGPT4B zN91@EyuMec4~xYZ&2nE%D3$sWz$L_6bXU*0wGRJ(Ejd^>kcBf&8gb4^Ew%_h|B#fI zz_2WuBDDqQDc52!C+p7gu`$xiW4*YSPb4ylWz`mZ#lP)=!Bp+eJsnwRqADa}fsv^tg^(PA zD=pJ=8px{c+c~Q&tdUwVz3?`Fv17;MJV!*srsYl)88-A{P0d|GD&z>vXP(hDn=o3% zbz#jv(!-cHWQJePQo>8i-G%%01yc0BLMDWS8ZpsKUE)6Z4|P=f`=tvO(*J(x7~00e zZ<#iPfWH|1X+upT*BOpwj|9gOCpSsPqbT*g!~r}X6b&zPKuSw7K@HwtZ~0*23deC3 z59@GI7y85o(j_EOh|5u#N!@DV-_Q-x%wQ4%x=2f3_(x5D5M!LMRpsUBLNd0r)QEKk z+Klr7ocD_;4^mxukUT37oUWuLc_$*@YUnj`^vv9;!#7U;$-l?2OSD@O=P`+kQ3`Lv zmbHuGpo#aLZdM5)7FBGyeG*27*KDC(sATIj%<3(P6PBbR${&};opC&{fKQk@w;b!F zjpT|*cqe5pN-#Ng7YiVRfh7rXwI#Ukqq_L|3)c_K9G^Kc^Ys$Q;{CrT;a1PU&7P@4 zQa_jOD^b?Bv3KVD^~sr2*Iz_4pI(1q=FHz-y&iyu)A_18I4QC2QgDtv!Xpcb7N3Hj z;_sWUU!6O8Vs`TLxmVtvJM_lVOTf5&>CA1iOK0iG8RksOG-Q) z?}H;*RpoR%5s#%6ZqU!7_;_BhOFWq9xfZX5l3KN}R?b-qP}P zMWcZ=QX#t8g{W+>Jtis0EkrL-5=1~)@9aJNj|)eCK729M)CVOd$Sn;cJoB(Sk@mi# zWpnp1C5)hvY+;<37GY&Bb482S1uZF9`mtUl%tV{$C_!r}(Q*a27ynl4MwV;vA=JyX zJXUD|HWJvZ(RNfbi@<8dsD!f`&K5MRhgVrDajuJG3%}+H#Kq;=q1?wClAge6E?Ts< zN6quStw7d-D%CyGzlXzyz`rhyM(i>Sb->UJ{@)VKcGCx5uWj&L+MvXRSPNO^N)$kb z^9D}?GvOm+l?o} z!&75|6nx7XDwVB@fLN(@tFj#uvv&JcMl-TS*ygC7Bcru#&S@y#vNG>(IRhV`zj%HB z0P5TDv=b*J3;^ogMDj$>?OxS4(Em7WeZoWXNXc`@SO)L03iB|=rvSEa z9{JVzmcBr>)Vr-;^8Ez0%%AGA-{e&LZV7OH(Pv2#q!vnCUGw-X6)dZq?(H2n43;Ch zMt7Q)V6c4egh7hE|MKjwpR{vpwi+b@ga3cVe?1%jYh*|V+yh(jd9+((Q zD}$nvPOjzt#=+!3IyIn(>11MUI?*Sub(s-$E5h>hO7z(X5Sb. z)OhpK3G#EF{?HqpmcuQhGMLp}E>S=wz0aA?Pl0Y)(QbMk@KGCm)B?m4b(%maot z=qCIzcV{(hn!nynOPv3eFD$=){(01R=@zVE0v4yC8Tm@fy8{?3(G0=*^Upu_fDDGN zk-p5gEUh?>T$l8^Pp8*%qFoIPLT7?_f-Fy#y*EtX56B=b;T+8tT?=hJ4w3R0hk&Pp zcc6((h`L?^DhbVoWJJiEUqx(byHWO9w7ivAyz*)4_|%6Ft30_|D#(ClX9&s}V9S*J zb2R+c&~lXqVenmYEOSPc*1FaeC<^ltfl*tZ2wsWOm7OatNG&>gqy*NKz!65#Caf0& z(2RCT2b4LZi6}48{lwYS!p|+Q@<_V(3UGwo|C;8@&xW*BUyb zy^uL+Ng@_e#xCqZMT0+zA%@vxPDG+8fuo-*qk@6kx;n{gFp}_ku*;m7q=W7X%SU+b z*)nBDvmqHWoQkAHck@%MF!#|PZtpuWyYJK4eP?I)et~|~i|)Oc-TV7n-+nXq@&~hf z_AW6yXLj=S*}eN`C*PT!g#W%VJ9%>MqvJnadS!O<)1NL~@{QGvw=c|{{c3LV zaY>IW3hW$v#4G{oTnC;aC%iw%Z{c5UoCAU0?S8?#{$R6+p6csE8dobo($=77Fm;Z2 zkZb-Mo~9+N>DVd=MOMp_TS!po=lKV{?)P^)r{?qJc=p{RE~tEf^@pKx!g^t5dii|# z_XWA8x&9PLb;^3lBWJzLiA&Td+2@`DQpRkIj%o%A(+(C>LQ;uJTk8o}Apd!6nlu}# zW+=skWJ|k3KMhc}!p~rnAg6yKt1P{GynY5R$FW%;Ikf7T0@Vl_0b6lIBcyC1tYz!~ zjA+A*RFssOaK+n2xqVfNyeK@95brF*e%W}BrcM}Wz|M5bRpe@g;(1?@6hV&4^Z6QP z!Ons;+eD_Vjq)CWfeZ9IG?%2fAf}oH#4}g);7+P@K>C*2)0pHn>uINj6BXYxj|}XZIc%?r02h&cZ7eCIOldhZBS78x|d`0pq9bIJ83#p%_k`p4?!=~t^~ zd9#YcyqSg5$l^<QMZF$4f8q7aVM>%axF2@VV!d&Q*Lac@4WF^9K&k0L%VHUswA}Jvm zP=l)n&yb8sEl>h@#`I$rD%%;2 zngznlCAR->A-IUBK5DR1t+JBUa>smnb4*&}G`SbEXkR2KE>a~ZJ2W&c36&8ocLN9H z+ZX1}oVoSR$G#!z7A-fx1S(o0%6o0}8<;%nH6)*bk`Ur@R9)NP0A)so=mALQy5a&= zQi9mhd@stHY{{C0Pjkc3FvHB7RAYiDN2E&)7H7Uv(j8SgObtzygjf{UX|Qlu)(%c4 z+!ah@K~e*6H7jRj%NqKq9qzKG2&tB|xh@i9NpYvaK|C#|60x+HQetT}IS@<7Qv*Zk zzEo;3Ee;JTyS-iJrq>cBR4{C2J^+TTC((){+4B7aE=sT^C6jIKIyk@RvlthWEw;8f z<~;>t3mDdhPT0~*65KfTl}__$J!NLL;2Vs7Vmiq)3k!i9t7o6pUU*y#l7brXj(3fB z{Dq5m_n!%ucUWgz=co5nPfzci-pBtgJprq}Q@vQd0{_0LUPY^_ z?@sTTex-WFf8cSWVcITH^Ig5Dq+_BGiwBO8EPu~0BUXq7atbH=NJhbWSAkq6W9Z>f zjw_t%Bi^zjc}~BD)yt&>am8oJty=?Fx8S~W1Z_6PU<7Nn7>p~kzTp@KwLFjEHr#H+ zc-{Z*9D4*s6{J)oAq}n&P=a$d!-1Lly0UU51hu8J{YuBhBzDfoR!qw%b6YUHG%$rT zfh1e_`Z9Ts_*BnS&sQ%XP+>0c-0|rbL6x~UeV}?V)D_J)RdDsK>Q|sh?yH`zoH6E0QI#>-Q7gwkF(l&kJYLeT=MmhFaL{;4)2T1mIX_8y`|9|~2FFL%Yj_uI$8cSE?HrY~BnqMwC2lT;*saSi&%HT0cj|rrDt(s{HvuM4 z>4T@HFCXb~C2m?+=M-Goa8frNNK!39JU=?xW}`%PM=R%p0fqWLJvjQ;^)!Le({F2e z%$%PHzX`|F8@8Z+W`M{ult8N%?#+K3FOd?&SWFP(yl-F9twFc%4JIzq8Vv9N0zv)0 zQH6Lc@*g#AIHR^U7G@-mQ+l-c!=wGi4cOc(f1Eq~(v3aG!rX5_1XImki4~Zk@Gkd- zProvK&>>GbA8wX{47ysx`p~2VFg_PWJ48i5mW5bL=+%!R&}X7if;rVx4q?VqB0*A~ zD(9SzHGRC1E^m&ZqcXw8B_=>mU=x%ag0>ZmQrQgPycsb?QR+-;6d^7(`_iHv$BojB z_g}m5%GJ3KzYcN7k>i3SH3u#HWR+~eS_!kM6HCzJ`7~wlkWq{_=dqp-pq_zrl`7@D zD|95v?sQX+*`^973y@ms(V}eA4zf*;@4x%*N#BTeFWXe%bOVW&6o>nEjBF!W+on!g zwq}^9Y%oNgwQ1Stri?9$MAve*j50=>)^6!Xatz_AGlqX(dR1nFU)i z%p@RQB`r5o-FZ#|a+)>u6)l^WIZ?t08p#&Mi3ze_<`5Mvjq2UAbRJUtlkwJ~If9<% zrB?xR8aJ|BqZVCPqP@bR#snqR42x$=w9hn&X`*`T(??#5-qz5zl)p+3*cvyaxmC8# z&~j+Nz^1ht{i{ugZfaQ1`L{p|bkySW>CPaK>*&bd;gJL}+3Zw{EYSv%l*_N(c<-g# zNBrlVcFz)3WkKmEuyZesN;lDX!9W@<64Y-Pdd|oexp|6oGLCpV-C8=!qb+2-GPkKg z@V!&%bqtrVW{mrWYb~x29{kE8jn9kAF)AnP(H`g6BWh}!K=w*j6F5b&)&Y@9H}nJ* zbhqHijUY-R7Pde=;ExK=SIlBfLqnsZut9+sMahp3;sIK#8I>G@u}r}FRrIgV$Rh0dzW zaB=FZ%4)Qk7ER1T^I)jFR8~wj?}MXW5ittg>%aa`{CfQa9%l^=W37}SEFT~$Ytb># zU^>e-QF#w`QE8rb&+Ut0!Fn3TOm(~zf*19MF~q$--8-2cCfca~jbnh6%QRd2(gmeM zblF*=M?*{H{nuk+VEUe;rrrxBx`ZMXz$;%1z)d+uQ+^6G^Yn6^p&rw*4?FE^JwR-3E5S$iz<$-0d#nrB9t zn;O^((KD;^^2}I^3vfjuQ_m411)FHuLg02I9cpBvB&6a|Ov%OZsuN?3rw9jTX6RH% zIKf_&8r&!(MdCE34NFMkLaL?Ozu-0Dbo=&=yxq&xS$L?@Vk)Vu$q*fx6H{0CZ(9}| zlG4;ktwM^s7k_)l4phANRj6Q0OiUfaxiO4ZVYJ?`2C!uj?IAgIx}{JIYxui2u#TR< zx-mw-d(ppUZrV)4F1LDJ=oIflmlTrms3fmZ=t`7@*0D_GjiAMdG|2JZXdvxmSC(kq zVtx&@F-fA3YDv}d>_zC)`5Eph*9a;ZIXX%~>p?ksUnW>NA@EFpRT5-%xBCV6UDrri zYjK{23v+c-f5(o;S0hH9h_QtcCk01iggNh_G9y`%gL|952Z!UePaB0_HqbhdfDGQ< z`0`;rH{DuU*D+XDw=fW6118)uvNWqv&e39^+++7Lw~-W!C4E0tTd1bST>@SYWr`j{ znJTYQQ>19kt)-d)N+v2%o^#f{-h;u`T~&~R_gP&w9p3+b=?e>O24Tqr@R+yn@Lm@Z z*}IUy+SWq1y#j@#K{0`OFijGsZjuCppoMsvmcWL*pI30boih!6jKTyocm}s`UpF7f z9fmcUnQd|Dd1|D^C-{_3c?pgagf<_FZ%~cPi&CQ)wa2~L1vtE+*#$TZTR+WX-Q@t} zKGu}V>&OMsy|egEG*g}a!@(Ww%SdEu`+LE#3mwTyH*`l?Y7_MEoRc}q2+=A?qgkFZ zl3P&Xkgd=EI)WDiaJiwk>yy?qG}IzJL#Acuq50GYL1w8dJjGw8hm8r!Rn%<-J}#D_ zEQe&Ftox4}rnu=O3SulJxca8bbzULaBdw6i>jzaea;Ddv=|;w~!_041c`=$S2j<_3 z@{%4%UJ~Ou$tIXqgO6(rK!@7maoc9>zY&d{driC=a21^v}rX~J!_xpsfvz&*Ew5gN41rBjv zr9I~UW(?~H5rhZT7-!*$=`Q8)rrHqJw4WT`Ptp)CkS`E-!UNH1v8{!w+bWVEw)n9;(~)f7Uj2RbPwA<5c2}=X@2y^`p09pWJq_Dn^;=F$ zQvI@esd^D5WGShtsRWwdH~nJu^zIOyEQ#p-!fJyiYE0}|c`<@l@~t{z(>`+gz@Vaek{o^j=! zu1U4uhf|Ag#EL8=qSlXYrMYNl46KK$HY=*2#-bL3_shG>762YpvGshYWf>~(IxI_p z{m&}|r*WIvI&U8mqFSq@1Vs!S4|M8mdDWo7bOV-MY&o&10@r7C+ame|+&_K9(b+~y zWv$lOexc5D{!Ds=f&o3@gc>zGW-u6I*rI#VCG-WHt1PH!~Jek(Kqdo znpQZ(eR0V*KHaLp zRe1~CWYqk{t=_Rb)9>>)pIbM7x*d1?1mGk`8(BfV0&MwUYvg2 zH((>6J{4hHW2jGqHNJwLrdbPAk1^`>2wt#hm9`3lvM7ovL2W7A^HVDgLY1?#^DPyl z7{r5+{B?wZ>P!O|*{#tBsXcX=voM-&!2f}GyHcq+{V0VWs)3x7wE(0vJ-J@Eu%SHK?@au z9SXWy*MC)y`fr?<6Q?@b4Ax@P+OlQoZR6Vte2LYN$yt{WHcedd?`6hgG>5hzxbd8; zm{=5Dl&GRdO4K+nCQi1b2NrxZJiK=D+L3D~uDyNj(`#>|Yj0fpet0uyOb`aDdx>flB&vngX#SEiBRK-XXd3mm zV+FKoqu539c{mDweVnUGG|LR^P^!bzXfxu!sm_2#FzeTBGE%H0Ro6}!q{C~R*C@A& zn4cyT>tWRc>-bswKF!=7Wn{J)HbcxPVcp>|-eOa*zBJCr3(=RmHS!e1pJX6?2SkEDo!diwIGPAZ;<|}}?sOy^iqO(a| zTG~XvkD4viJSUN2_ zQB$UK6!z*I@n6?EjBQ4<1Px%8F?7sa%!IxR_2Rq$Jl?{ix)Xau_NR-d=05s-?xW*# zr#_k8_vY;0i?jQFH@o+1=YLu0}WC{`ltc)Bkhj-R;{4`bT!`_@65;%}#zXyJvFycK7e?*fBf#>a7!} zZhrIm?BqMMlb_5^?w{TJ%Kuz>b9VBh*~!o6PVc?_hds9@FVrr-edIsjL{u@OwJ{>f9f`y7|FdH&1?b^Y|NG+#()EI=vU9 zd_RhFUtgS^e2>o^Og+r$-o3Y8{B-W%0Y3ZR?gMykoVhf+cM?!>snJR*?$V@duRK1h zsvyOqI&5y=a4u68UR!X$A}%PZXFkSeEPi9ykBt#9UNi&qpc2+mt=7d^?sa3K%#_EE z9+HwA7@I}u@B749BB`qW$vrmk>DX4wCTqwvOs48unRDzBucQ!51U5uDB8J18nPYe~ zTz_hykP#Z74D&C<=VB|_{GqMDOx*pj5-Jz8!gFzy8qzWt_f*kGdm){GJD&3AA zXN{C>--`jxFA9e&q+-qWB3%eXEIWICcA>P9N;kn7Dv-5C* zHXAc)q}MWDXAdWPhjb0bY3>G;EWC@F+%d>Ifs)#JfT@DApsrFl2j68d(vBVE(FJS^ z&N0W^sbRDQz9>M#&%Md&5Ws{WMwxw$dyr*-1wnHpv~4IUNJ_I_5OqTi+70>rD?cB8 z`|k1mzQOEXH{>`k?^jwnTd2sf-u)v}|5)&c@63ri;Bm_({aw05SJsbfm@5Ha0i-2yV%?$ zz^a(w6a|VLIFPYYHm_%GTr$EWIZyCT;mMYk)sdsLfePz|6LW7KyS-=Mt@EdOishmG zw@#i3*<+gE-f+>!XEB&bSMMcYty&Uhg$ z&SKpvgu~8~;GM;lmg43ph8qyve|-5Wj0WBL;NabtKMFC9o8U$3@fKWz_J?``m%;Ei zc^ZhGw_&JDw?)*T!7wScb6a%QB}%HViIV3vQ4~Q-Z%!N=^;&+?`zv()>fc_ye(?H% znJ>`HiR&-IfByFB%o%=W>A9F@KASlX_dRp`|DL?Qf99-j>3POrS`H1^Wd!pabIFg~ zo^4KEgl-O?k%d?yDz0qWu5=SdLU~ScIJ=do2uc(TZ$(8nxPBKruGA?kaY1TH5ckE+ zXqXmSYt*uX&?Y+CRTud~aqw+5a=a+f3Q*Hnu4E_7%eO<~7Pu5kiq-Q%+O!5AA5kJkN^k}RHbBltxg;wS17*9)tEB)6( zwer=puNzpTY9PU>0@Qe5gLtJl0yzVMMoXc30wq~UBs%Ixy*yB;JrtMTFQHpK1asl^D?rtEotA}PJ3S3f z`^GmZ31WhK_9Q`;T+TgLU8r;H5ok$Jqnb5`vBShUFfJ;*XQsnT8(sn|z@iArC=%t) zjWD!|X|ParY&k9@BaVUQohom@OlX~?taK_XNkLH}1){<9(Rc=bNS2Qh6U1axQQ=;@ zSh-MPM!BGoOpwkq-CM#GNr=f2L06-!*trhUxH<(y7SxW&IQuve7os{E8Y4_IjB?aD z>H$478+5RGOPn9kne#4qWZQSmUZiPuf){m)QWRZm(m~)V_-{!1hY}a$mfn3U5|V}e zjuBI%-D_kX43&hCOf@g8IUhZLHjxa`mqLn4KhbS#!iniF_*du8r280I^9;4H+1tk~ zT7-$yt#O-;qCQ>cY&n|$J_z@|((QV1c^itd7@);kt2)x2E^`sK%mgnujYXBfP3nl#V$oQGweYf?R0VXB zNKNE!T|0{z#VA7=F4+J{(6qp=ET?fnjYV$Vo&i|cU8F1w)RhqVCK9!Z(qN6^uLMs=F?uVUuMd1Ek_m`e8{ zFJ2@IG0Cf3}avL$cP`g+|H- zvrM!V=V+UDI6cqw^)zDA^9(i%@H|?EyNWxh2#KU1$+93yaz`qH7&!x5l0NJ#AdQPO?AYuSr)8XVxbJpXw-n`tX!a>bF2C$4jeE_8EkP@)vc6F_&0P7 zo0@BeF@vEG^ljL*sqd*lSNedf&%0+Qp%iP=B4Wi0jgf(s-|SL^E0D}ck{~OcZEbz9 zBmH>b(2gBj;re-MRX$q2mu=s^jv25kiE*^m&VyOCiQv3V{%IS*B{N($PXE{q!}?ai zXrzEy(Z440gkjdKO`VKqvB^niN3pB-ic;4U70;?7Z#LaRDH9OB0R~$B0(OpV{Md8P zaqVMX;!r;*WBT*YKlZ4jHo#k88?4gQL$={`T4uR5ybhe`j)uuSv#{cu1wJ(<&(C#6 z>t8Zy$t99f9aG6WrmBz(uKururBZXO=;dfYJ5=Wu#Q+cqwd5`Dl%fq(P3&@;0mMx%cf%1R;<)p2ZCD*Mi*|U|! zOTJT)W2j*XvqEjOnB>*Pk`iI!Ub{%y(5W8k;*}KCWV55&bG&x@_Cac9ZPSF?{C{nW zYShFQv3U`75$lFY&E8$Lxe|7oyXOxI_lHVW?ZP?K&12IAPZ)~MLMat3cM8W z6<(?=NU3Ir%6(I>d69)ddzuXR8}EF4E$IoPBBT+&ec;{%P)mbFHxS#Fd^#SbDYcv=*CfQigry2*m3m8TT%cDlCJ6ZSm!vBcN$m zw=gv+Mx2=$vm6|D$@~bU>8iZLSL8bB0kJ9J1s(y@80X!kyHo}d0GCuc20Ex11=`$n zD7I!;1x9yu9480{h$tjl=&e2PVPFvAQikSjp2x=Rymgo{c4aw;p<|K!M)Wp)ar_c?h*uvIAi>9tB+N%zSE zE`?Bx)qRo9u}7#B0nSBM_TpAHo(R=H2{5T`_~WsU(uJAh*Z0qSHuL56muEhiIX?4g z?ZhQ`c?MnIJ9A;?1RS0WyO6;@My`9mzVG_W*I$6c<1?S|6TaaZ*4NNd!LUf#rG3|> zD(+pV5`wIDq|Pmms7twE5bkNUER3=RRxu5pJmr61Y9D*4?@Y4yohd2ATPXh{VyK|R zP#+(@^ZJpW4`1|+X!ls3QoQZ7CzITCE9alsxhv9+r*7af<}V(Q8Q@ z@$LLdaoT}a!(l~Qa`Qn~Ml3rxOMW+W8c}XTHMF83DPUrC(BZHvpeRD3rBRqa3ca(O zcy8b1?7qvh`wluJbf3-c`*i84ma}_LIYfGJ_xtBQI)3xW)tkrPn4SFm=8?;DADsK? z(xsa(`PVUj^VG%-PY#c`>^4`(Mf4hSN%10cQ4L%)v105tO$#+Ln&LI*V${s2(SxW{ z&B)ps4ynUKszwCRtPB*qc0=60eIt(pO`T<+)4?NRam{h+q{T?Vn1Gj{F6O3AY88q~ ze?3D=@K!NPi$zl0?dM|9@{0F<2Ekybj*S`G)G2EWqg5EyqG%S;9+DE+twpup(6bYG zO%aNErY;NjXrHCm=R_K(J9 zn#~$w(k&;+_F|G;6pEw}m!qZy-SbV~{o`kM-w)R}?etxbRz6p$>C`)@Xu|-R7MS3) zL+wT8+@?{FBbV}Tz(j69o~~Y*etmlH^s7s6C#oKWyT3gB z>hyu?`RbReQT0&uo9gN5z16GL%hik3uTk}I^-H+q8#wW0_2Tr4{?$1npToK_wkGb% z_^#`7l4qb)FrFj%ahJ~G zp5o2zQ!NPyD^qypHy?Y{Sd+ml)4l5TU?P+fk}a*=?$3C>QN!~B;O^aB%3#yPIfGF! zyJ2$f*8!zx_%}UGxWN2qxMDiL6Mikr7R-vM-&j+DTWr$^rALhQG@{Zo!aED)4W8$? zP`i>)1yNB1QA%)HsX9I>xz4KW9D74Zi3!P;tg8E08EwOGwx!JI$0MK2U3~xcdoSF0 z@5=2%=Wkv9-F*<+_Dl;x@Qve6p;h?!%)NDJ?$ArOPJA_Y^}=5+%zgCkPv5+E86q_h_E_I67QYo5~eyyhwPD}GR3JNVq#$b`f@Kn$>(|HiD{8> z%yJ>N15R;z2sIH!C*9G>F`Uif;QUk_y5nHEOeC6(pn})-Ti4#YcH-JQ*WS4H))F&* zuAPJ%KXUD(Yo8*x`%kYOK`@jf@bB$wZ^GaA`Pnz%#R<6ZE#JsoJLG)(Tj<(AjN z=Y-4riMhziW1h1w6jeyHWX*f_nvOaJ4xn`}F_cC~*3ffk#4f?NU>%WNe5^dk;2V-b zJVtq&7l!SO%g3g)b+IXnLIKJ}B1-J5E7T?~86n39m86i2%H|q91X{(&1rU_3xChjj zAhpmlMDBqGh5DnDKV3dDcj-J@HTUW}KVACl#?kk0A3l5gofH1MadyvJpo)ST^&4yo zAApvIJ4uu6RG4Hzjzu~%7&0VVC>g9^fk#)VjCRTs;m5DwB=5Ct30yEtIQwz5el_kXFglpM5FNv z`Er{vW0=w~4H+@U`9BgZp-w;9Q3Ez(JTc0A*2cM|7nEMR5p6MGZ_oM0V8g%KSaVtS z;(fTD?@#v3r(bLnEF-4(=^xTjmJ`U~CD4feULq^XIlPs0>Z}WicJHP~U>dVoe#l$- zVP;L8%G#P?qKwV!u6uQ(M0K16gaRl4!4-a-K!Lj_cTs2gUZi<#vn zf)h@4vj1~HR6$BcZIBx67pm)aIXtFr4ogB170=Pl73rh7edO#%%qWH?Mo zL2hY`?m0;pwxoZ2`OV$KlXs8rx%=LsJ7+K6`D)L70?-?`wgjNJlCquAXqJPy(O~4! z$Q7ui8vlIct-F8x?9SP9KfiPM=fiLO{O+s1Q4-@S?*bwuV?sRP(jENvVCFETgj6c9 zk+a+b(78Mr#};9$(FS9@ON(A0{aO@-q#C6XHrk>(<=ua=Rt%+4B1c6MQc9$A5g;W$$TU=fH@U@>Vy)pO2*SEfSKSYg9 zObGGL+KVm^x#?#(2ftRXfVq<=+%XWUy(1=tM51|r*r!QPlfktOer}99jUV9t=zUYNM_jES+I(mDcGgyS zMG5*1ZkLh$3z$eTNf6_4K~V(Rm5D9Z6R!B7H_4SGK~e*AFmtCkqZZBb40)0zWk)=h zoM*gGKtlp3AUb;5$ons?AGRlG@d;ZmR@`FZvmGMGLII08GN2liLXs!&O!+5V?uuS0 zse+h@GIrdy6T_rru;5&S9okJ&Tu{YMVt9Q#TOaE=M>*euv%D!-#1i^|RS*kilUlVx zsvI@TwiO_R)vAdo?fFZT`_q^3CnzB#lYv!Lc!GxEW|@wLu*qpz&;Ygyyj3w8DA+7p zxSvMyC~tp5yD;3jZ$Q?<4A4=stB#18s@rtVu}7g!fesRwJ9LFmMU{A6KMAp_AApi$ zf*cF1pe%bwJkL2o7Iyld26ilV1t+i%S~872uW|4j2zENmr^0E~AuO@=Y&~HH-*z_P zjA8p%9iJ>)xJZmK!d!ce)SfU#S{w&ihy`!b#>vbX9mCnm(%VvMRv|S@9EY`VMaD3S z{{4m}!b!FCL7Ypvx}&)IG<%i^MNop*ur5bUe(D{>3_EZ`u1;4dRZ!wlg2ZA_-F)xj z&9_d^{qD`5F1{3^$6bztz8u-WJ^e<7x8Xu#@RMekzIHh&C@rM!p1TLloVb2)=2Jl9 z`U^8>XFi?zeCCVmFQkXGa-r90BJ}CZdH8$aZ?9gTbh-+`Wna%6hl}>loZRic;`+;N zu0X4#csE;72Pvp-f&337{r zBeI2>Y$2LuMh1tgWGk^YTUcZaH5fx?Rs&sJCw727QC&nz1f-(gBn@s{m~+t3L&~>M zWI=9WIExm!rbUjFG7Rf*O{hK}MG=xMnM(db`+Y+ryMH`zXvdBLMtHMU#D`i_*dKfD zxlN=@bG2S_+pt#Loz<{u{@S$c%x+rZRM%L~l3#1a`18*{_JFM51?W09g82aIQ1Vyl zfdVfPSdoX!b*tY zOeso4HPEQYnV@Ym8kQob#swwOyuplmb@hG9NXub6`LH z+9Y5tfpJKjel5>Gj}X7=r3|*S1+rVCd7Z#Yk%!5*;LYcr+n|raT<=1goOgz6?;x~W zH%h4C|9*||7}hORHcX44oDBz@?!ZEtVN(-6K}-Cm8a@GMwW*VNPPrin4=PgZ$v~@J zELlHU6i`ugMu6Wt8$yi7%dmIu`NQx08jY^|sl<-c2bp@3G zD;l683!>P(qD46v4OKd}&fCciV;xZ)tv6u28;t-JJKajh5IF-+tCuk6l~cU;SEyf` zx?&NXJDbp2HgzRi0Nv+Ty=d#h0j66%rF9IOYqn^-Yck6OpcPwhFi`Z?v8j(Tx;xR& zMT_JalDHtXbX_cK9wGVbB{p?7_t(9+x6CFgWolN?s+_@lJE8H)IK??MfrhldUdd2m zefI*|2CIhl-3x0r8(I6W`@2^T@Q^ai=Ldpo4IEM1}`h%I~=e)86bb~!DpHK%-lS41Hl zk7XQ6=_RCN@X)AvSvGEIWg>4j8gK zbkT{RP)D7Y8smtR^V5^ld#hi<@oVtvV)ZP7#7fC4UIv)~fQx(`eCoOB~61St_Ihz$;BTPssNoU#>> z^9m&s&4!idW>|O5y@=8P#+|SB-}(4(y4FSt{ruMcJI9WqyKi6l$NA6D&j&xdd+_j& zdtTeI+WSKaye%H#Dpe>f*1N|K zp*zPvg5lix;Nac&{Y%;1xA))uCg2*pLARbpT>ceIGU3#-V!>KRxs zPVO(@?E|L!jgD}FZHWY^(|NqPglQ$f9Fv3DEv9=tfS}o%d7C> z^7JeIg_5pa`{Zvaq7$ee7j0KANWCUV@>mfhat<3Q71+1D;jt3pHDB-w^cJmx!1}ac zN>G8IacrV_vQMjkEd>ALnTz~JR-^F(%@)wV_rOK}-h*hVtWk%$c`2dI=`tuTp$!{0 z@KnqpJ+EQie-qq6wvqCrCU*gP0C%t=398tvh&(Y=TECUgS4JN>RFs*ArWuR+1`o0qafcndN6t8osR*9xfd_Zy>Ri? zx8KaY{J|1KL`)@Pb6=nP>GFrO`!0*A)WVC&&!5;jyKm3z-j8PYeRSikuWy|C&^J;~ z4XxSoQ{!&tY8 zhDNY9hI0l(PZ*}NKrEet;`@!jZWI*a0#tMOl$zfz`R|mj#0_hEL!H##xOhE0Xium zDMn$EdYHt*l{~RJNV4h1Mx%;?5{u~E-O1LRS}^$T-zgqa8&2vd64De2!Mo_FX0TA> zAgTc*wfYVBrcdvg-dDXeeV}@=yRPzwQrlB~!}0TfQ$1h(*71?oyjvF$ zjEH;4zlL#f@76^$y|;R?`Yrl#&mXHF@e}aoH$Uz<>R+wX$S%xPJ7G{*&nZETu{xHC zc-^n(9D77l5!9BxtC8F_%bgW|{85by@pv;*zFe?x&WEPvJ(TBaPa>~Hfj9O$Z(gZh zhUfVk9^QJ?rLp?Q>KP|K^%abye$NZi+2-A}d)Tn%?45p{i?44?w-D2$C6GTU7IQU~ z2~ha2CWLsbBhCFj61b$~Y>?Yy*5xFVD|p6#^P|fzAVb#SH+YN zYr!RoWCg_c&hOshL%DPOqr0yhz5DVfa-6FlPM8)ym;eBXbMCRHIM*H14c}-D*(F0O8*aChi=rXvag&oc5mZv{sJEc= zwAD(9G88Wf$>J0yb?o@7^uO9TTU=)|yLSlIKlm&FgR-#vF=7R2Gkov+>Il{Sh92F9 z66b`9r6R^Vm)nr&c6c5`iE}(DEiF-3CLjwQLtD!hcw_ETIOGfxCvl3Wnl~KJyj!q@ z&3*KT+xt%3-t*zj^M{tg^3@2%jf01Nx_D{s^y|02IQP>xN9R8OYVOc+-$<;rQNi7G zuAYmjdd$THAW}j+9)+Nqg(JA9w!_6r6oq)QIbW|(@h%wu=i48S{lNd7|KZO+{0I8s zwiy5SBsWBD#%+Ks?cRab>)N-sv6gN1`I6TGC`A9!NJ{ zw_f}6jq?}gPWs=cy2bq{ar&S(9qGK5^1$l&mv8&Dyv-?nEy9S2a|R+hum7VA&T+yQ z>{Ryr%MyrcJ>wemy?oaN>W2Y5I(4$Qk7WyV%;@c_*g4$W$N3{P-|#(=9i_R7YhP39 zE(Yh=BX*p5GH}3ZnXY@VKrk9s0+Pg8hQx@S-6_>BU}EHrQn>)Q@G2)Oa4}Bjxkpl3 ziV0$r<>GcCz&2XpnJxxwqv0jGN}PWwB}V;O8kVtIm4zABDkjN7qJ`;kMM6@yVOfq6 ztcPXAN{XPiEMMI&xgLP4nSnNg#&9@xBfBqYUK2F1IDSio~a zkn?jKQ*4pE!76E)<<^uyMIySRFTsd$Mx$ndUvKhmczvVHLG<2&`plN}`hnf+4GX@b zi$Fi9_~ODyU2uCI`_QvCwhTt$nOJ92odl>*q$Ss~o2Wa}6P|tPaw2jhTG)H(vz{cA8K+_|PqD@#Y zV%|lQuv#Ed$L3DHOAy%oc>}WlpS?GIZsR!jh5wblyj5@H9CL`-u$`-WimP^uN2H}3 zJFHY=&;(i-Od~xrq(ISCT5Z|RiIaGVoj8e|$cy4lw(LaStHR`~4^(xXqVD;0ElaXu zx$-Z(PY+08fC1^50c3E2Y>6|20f^~ecR%~@u}iqJc=!OK2sTQZ7wL$wS^7TZKJN|i z_WFmcck9IJWmH*u3Elbi@bPzTqYcH75N2YmK15@3tT~R&LlAkkjOQ7j%~5|hPuJ1gM^HJBN<7Qe%NRZ}HB)rv?IzMgBRAHH__^=qe}ymspM*G@lv z?evSAAqT1@VesO+H&48N?c|wjC*Qnw>aoa3+JJCnfH8332Lml&3>+ZGVAmdK0c3TIzti38Fg42?aysP`@loq`6~VZZ3>0TX)IRZws_Ue{S5ABN z)$a(pgDWRRVKp3`5=G{hy*lfY-h~r&{^Z(4k3@K-6dgVQ)m{*E!{Uv#_dPo0PXy8M zdBL4orw0`T5ph_CjUp+*u*@MroItARFg3vBW^=I7`duaIRC`}ZI<@W;5S`Wf0<50% zR05%e5v~vi$JXzlDIb-dHn3fpXau`mQO zn%FZkjK*EgDg@mxjMG6SfkunCT!FE+i?k9MjWKK$2#4p|`-8CtCNXLYb{oB`XKmPnTzkK%g zFE9S%z1MI5dJTq%fqjPpUei@7CV}lf#=Z28-aAn1OmyEo_Y^$Z*O%UfNBid4pMUeK zpG5biCUr?x6)ByNl0HwW+E`qnfl5kR0H}T*b8{XujEupIQnwWB7>1B4s8$lG*!h64 zq7D;m6Xf7>jxJrW)>ulT!KF0HYzL!5)YuqnXbjKMJlZQ>kBkNksw8LNudGP2o^Bwb zH)0Ta$RLD=NJi?GU@CZ-I@mOp9-v)`z;FJ6HPg|PG5HS7fm{;tl!n7TxQI1+63(eMu!KQaTyrJ z@VJay-cYUJlM)LjCCMPyQ@(CMa-(sR9I|thhfq4zs?LG>6?!MzrKHdxnkRz&3LO@0 zge^zFgYf)@cT|s2yX4w7jJ9t}yK^nOd!Ht!qDR`8g%|wmCI%lN;377~Ru@j_zGiwx zRaGRMxQ0pWd(aLq1^oa8nX~i)I?7zWl6)IgXo)l zs)3FFYPRP`TA*n*eMnHh@VdZk!3SFdxY$XSKuXJ*7JhKWJhu)UY|ST;f<*Gsoycc7&QmVwS0ek?29&m_o@+ z!(#M~Lrx+v8B?v^mbHzOQOw2TJD#%G6xH0YmPk@j*TZBN=2axrBf`;+;WFVKar!J6 zCpm1IZ9J9((?4jnPo~K+Y&V-GDKIHAlAE+#6XT<3mWpQC#+}zUylT@Lp(!(ii@gm^f=y9H zf@Xo{MQ?Cy`X1|E1cRE4v+H^7wx=XCg&mjs`enG_PiDwEUp1@Bopx#28W}rw znR$uS)G}hxMYiXyi6f;{MN$HZprLogBbZzD&26ee4A5q?qYQ?d=3L?Mu!od{BFSn> zl4VtrGs(r4bJ!dQ(f^b;{8lLX9w7&tr3Qv~E@fMuDLR&>OJLVQ_wDjKA{DEBO$e!1QU0{;F9!JFRw{3W>fnz(@e{)pEH!#n>yyj9!+|9k>}Ulu3#Yz9wt^YZ=t ziO=7Gul}z0ROjoD{r8jT?+<&U^eH^%<V;<9hzBpM>qI}SN*bSbU%=@HjYD=LT~uXu+Q{)09vea z7oM(%Ho2OX)Ig?JXr_$jNW+b1E?ztNOF=ks`d8OZe{}8C<*bM`?3tQe9G)89z31BL z|Gakk^tIFH{`t|l>*rs&_3R(7fAIXu!R1r`{OHleBD0;Hcvg`$IhmI7RTx5QNR>)O z!Zt3_L5nyNT}Ll5;dK2X1DUBs`SyKwWVSHC)O_KOog`|_h-eRbjO2L8dtmxz3Ax`BYCX8FU;O#?moD9S^|>3*zm9g?dil8CupA8$Uh;Gjj&m|$&n>e+(P(5U_T%IVsr+9$Pl1jkA3 zW1pTFy-GmSrA(r)CHl3*1q?&Md#EOADJheQU97fklml!SI7a42_jCh2Atm&FnSieV zmzK-eaLpdWB1@S7RgF!{6*H`7-zYo1GO`2*O< z3zF3QAZNDilPW2nTuFtNEV?S`$v}&t2%58!0A);MkGs6OhO+#?fq=67fGC677zIru zoWKC7KvuugM8-jw911AMX-U?63bL&F{B>%07<5U|`YZwd#HS(%c*=`(A%^nRmHa0? z35H9nfVbyDVp3WuJGDjmA!FQR1$&MWPCc(rG#K_X=P2=IK=`szAVf_hHiMf)gvy%(=hJ>-%DL^MfF$IkBGE*_HtC&VJXi>7FTJVsB)P9ZWxah1SirP>xT3(Zl` zg$u-su5e7vd0a)d<3(6RcHx5$-UqLcJjzpB@W4AWGxwt5mA8ww$f!UZubUy8gS)5& z@3kDWDs6Q8zZv8s*dhhYtb0-YaifPc-7>aRfXgSbl4=9u`bkKTlTtd->f<-UM!Bcl z3a4e7$BhhHinE8{i{}})XJ-Bt4bO^`_OBz1;376%t6I;e1vF7Tk9Lz8OoSIGIzWHy z5))B$SoBSDmfnaA>?C##+?@O87kZ&F75%LGYP!$WX%Rlh?B4k~0h3H8LT?LU;xnCC zndN(>gvfHAWVjU%r^xX!Y%r5ldQ)tu3G$OvAjv;8`N^CtsYxWrOD40UQ;VaFuY_%p zU9}v1Ol;5~Xm%W0x-h$RffLIhh=@jLy4XOw(L+$Y@< zNW&b&$Y74}+|sK&hFfAknV?PVCfXYGCfYjuCYoTt&{F+H<`+a1^m^Nf-&V03vPjmY zOh;)_4y%Y#-r@Wl0qxv_2nL62c#lPku|;#liX(sy9Tk0|edV~RR3p8{I}V9MN>bE* zC86In!E&zMBlor@Xd$%tRk+(ETb>-F1-FO&AvD1bkZ5H zmF)gFF?-1D z0JtQjgce&b4#6$vWAqY9V2d6F*iKM*a*MPq^7AoRq&rPE-{T^lm^p0?rZ0FWGm5;6 zOkbMbNMP4P11hTOVk>l8rYxNr+~uMXj+w_f9;*Xhx4KAZg03T@h2~!CFS4~4nMDMD z8;g^6(%ogICL~pEb@LP>)QJo)AY$5o`?&}ZF1=WQC6;4+=>u9oJLGga^TmlL<&3Ot z!jkg+jN4T-&aJ=wJbIK5!&ul0XfJFy%$MOS`&gwzuR2wh(w(UsTWcq@^0(sIEeGv9PiJSTLCxq)Hr?|vsG5|@#17YO6IW24lM1%#>5o12 z2nm{tK3`~)h3i&`;AZQMfvct@y_`OI}%EP(EJUVBKZ9ZDv2~G&g3o9)$D3etD8d>!jG|@XZ z03wQ{rdpRtpR1sWtKj#iZ$AB#n}7b*mzN*A`KvS6pTBth{Y!BA)hoaF>d|+upZn!k zFTNTXzHU($Xo77b9q7H$&oTDYw?>Y!!En(9MZ2fHku^bUkka}A3V$w^@KT8+gWl+G zS_&jm?1CcyjS}O9A#paen*K|&)0~wn#Br#7w+grKQpwQ>vv5VEvrFb&WMGuxgaQ!A ziL5pTL!6#SB(myQdNi9E)#PkCHIYpXs}nxp8n}gqEiEYllE~1C#Be^3Y*r@7c5_97 z^RS{eo8|!GpK~&}mtYGWz%bVwC;AG4??kvG4GS9N1{}#o4Qn!F2`d1Sk+gJ2@rNz5 zvH9$vf{AloO@fPM3h4yjQnb8!nTl4e1}MGKx;lkuKqY%z z#444s>2<2LhVG*zPx#WG?%XsqLKY}5iAD~D=5qFsXoL_Wx1qKFL1c1EfH*BBI!iR! zGV5>Ef^~{j-m?;OJtL)**jx-=lF8`q@zH%p_Uzj;GlO8lF^&XGpbeq_nUp1jCSA*b zMd?)+6&SHmj=+lq9U`;Dt1Pl5G{!`Gm^+hw*Z6%CdwwuJwOAk);n) zaF#QPmtbu>@aDm_=LBp3vZg0C`pR#f8}Mm4O3*Zo;nyA6C1;w}=%Ctmk*aG?3SH2o zq}5ihM(EnI;d-r$Em*=Fu#qdG*_&t$WVKb*>F>L}-gjZ=LZi(3u}kde6yU+V zH{EvcO|LsKl9HV4XFBOfL+`5Q|h%BYjt;w`&w+|oTjFrr4bMx?_@1y&eXs<{-f+UbULJ;L$ zLqIi0xv=-eD{FA+Zipu1dO}_gu)~d~2;www-+v4=<1_%@1^ox(BH?`ntz=q~Ga{&^ zN?M`;n%=O`^^hjIE|?|M_^OFvikOmHxwqc$6IRdA^rGO0*kRw}sT28`nJgL>lty3{ zxTb@?c=#9o^Lu-}3cKY2-78)}2fqjU1pHF8gS&-X6}cAR7@fs>4A0TL0dO#1s4cSp z1;ACMq}*38TKu0}G_dJ}m$Y;5r!L4QI^bX2CUG{yEyMG4QX||IsFIQz=yMf1ssj@3 zt51J^>(Mheo;&sBhksZ@aP6H82(G=jg01gFV0{}Nw_bbp#&3Rp^QE6f&+d>jqC=9f zenDGHHh{?Z{2H0?ZVfplr3209LQ5-NM#qmI9)9Q{k=`oU6j!}C@W1G7S}&Mmvj{}l z;o+h2t}U9o1yh?kxM7EwG>@bE5=U7Mv@X9rn{02;CfBuS0q~&9zr;c!^!2-spe*cP zU;pXM*DpPr{pRWC|MA`%=$og`eEs6{==Ree{o}pY(KqK_zkTl6FHZb)X6E*9&g}U5 zrH}4uykpHncmPrdeqOQ;dqewatdm99abo@BB zj3VQUqb}#3H9-{$K&0YL{pgOsRc?6GO<-~#l?<&k1=cmMHE@a zE`8lD3Mk`Tu-3-TIPQk0OVK1%iEW~{G$G2OahJ0S5kz$ds8IxM0hAxgH(fc*ZrB|d zuL^YyHwgD4!xSK`19jEcqsTi636m~mQmw{zC2V&10c>KvFmiA@i{@oizITLo+$AIr`T} z(2f$b7pf%~ia5Q$R))|Arq+N`3k@U~8G1i!Wz}m^wS+GgM72Y@>naB` z4#pQW)dEoTEGsh72L1a1sWri03HdoqIgu zWV)&;DH$lJ<`D?lYA;?ndFAoihoWcw8iAZc_Xh$wheVWNV|4Sa{r1YKE05IPL$!}< z7q6U)3`0s#vx88|D3YxB+RzQA+MNuQd-XAth3qeK)-q zdCx7HYzKF-TAQf3Ah6s2eqbEk@SOu;OW^=t z2k?;W3A&(#k+pu}Yd~(=ZjM?GDfHBpq9vttpnqEEEpDvd8nleXgII#<%M(dibo}@M z!`fudefWwFC-6YbyBgK&$jX#7`SO*9&ru2ObJWsOD%D!?xE&XZV7+7@Y%nT=#Bo?m zlHabF>N+Tw0bbJCunBd`Z!BNDdJ-+abM@@;>1su-&%fU`Rz@n55TuSzkF%= z($z=M^6#&ny?So>o#l7Y^84^iXa4&qX!-T4kHAyCg~Y>MeR%oZ|0{kzB3^mZyZ6%a zAH@ri5j?`UnMb2;1pzL;>~UGvePPd#Bx+uYd(3_`xXy z?aij;=(bM#HVqALx*9s+NeA7!U%cF;Tib4-KU*T`c%&_AkbBVs=F%I{eQVuvMJENXO-5-o{x@5caH|FUm_gy0sb#S_ zZ`e8y_eHKQftjJVn;CYf=4J5uwrM!qBHb; zG(}vF-I7g2$s{U)%&}0}En#f^+a~i;-$x6CgYdxAE>V7zkm#&v_w%FJDq`9ICE1W3 z$^0&GI#$Ebq@Rqh)3db>sjTHsh^4B7_^EYCzXYwz>+ zMns;Hja`T3aM40aDhSNL)b#FwW@K8%Wd$N#N+l#Yosg2geo%vHy{)84>+8a#cIueh z8rzsJnm|NRZ~WZ!jyymBh$iXrMa`B1WVMpRWLAOZI5r%DyVV>ZmKq@IQd(}srf)3u z)>v{dpMuXaN@#~ounpsujGm1bOiquXw#z9=?X2o_Yd8xnR;h2KR-qWP^uAl2r~#C; zl#yFOq0U-*$gt_K3bqZ(TGfzIF(L&KXLcwef~()6YiJ~UrUM$u9tWG%yP$|fZuPZ` zV9{5A@}+msBuF~aZbu#1e}AhY0=vZ=wG4_;o}&G(;kkfUYYH-wqDm<#4bO6F;F*~_KpvUQ?jBv- z%^diQE~0q1wDHv^ZD1Mx*#)!Md<>T^E&Bvqj1HDb!2pwrp9iyr*4?Vs;cz@hxJ`RU z{*n~rFKMm68Lp=6;~ZyJj|ugY9{-CPh~uu+#ts(0$lA4A{d9+xF3c`nSgA2LTocT7 z0>iULz4^ezp@6S$f{VJ%jo_?1z{tI*%tXsju${_F7sSA{>p7B^Tm5y@nEN7z>7=BH zPE|7Kb^hjb$p%i|aY&$2QbuoGJ=?ZP^7ThPzIFc9-Ze=EJTj6NzZ1pQ+0Fbm)U3;m zt~64LwUueG&;mR~(ISPz<5P=YJp8knnNd!#$Y$_b8+fV8-#dPMKM~Z$W}qS8$A$4VJrmE21WqNoNGq+P#mFVk(gy_ zVFZU}15d4yCZUC_kp(;zDbW${$s}kz%)%xz=`7iNuJ65~Nz`bOFI{jbiJpYF(y4I| ze8?_Nnz)*MoB(oIQh=nOJJACjZ@2l52cO}@R;oMQ?{rD)*Dcix$%IR_>M_q-voPy> zCW4YuI@PbgSSx4sZ=YbRiuUa3j=d8rZ&gc6i9|ou%kvCS%nKq?WIJG~Rp7Zy)2ohk zyJV2Y%Ei+%lB)LsYLYS)(bTg?4N8-Ot3=E6<&58nO-#`8J87`jDI~I#2_&rM$y#2U z2V72#Vw2NmALE>q9GBnw@2+swSnd2Yqq_rsnkg`b76>=7mA?zln&_bHS;-rGCoRHIpf146CMynO3JO(Lsr->CSa@VqooqzpitVTOMA}K7{Y0G za1LexDWC$$ahJk(Z(tI5DLX=s&R0t@C^YZ9?+%+;qcw?l91;ilZd;W zMC^)ok_RxS4F0W8piT)^qxY>z3fdaEU--8gmLR<96gjK(R0q_PQnKGZ?@iz_ zz2Bi=vtoz1KK;XYF$)80T9^-waOQ%28>5(cZz$Mc=hGXR9Ll-B#ILX&R-2DcMA)R72E30|Uf zd=a#@9j=^IEs7Cmw!?c_!SDX{CWls zIDDM~$CV=4TzJSjI_gpj5ewNO%_YX#0Bd3w&T+uhw_`Ontk6JYq_o=aG3-~sgeTrL zd!>KLvXlr|n|)7Bi>HP=)Nsw7dTZc+MM`E`oob<54lMoZC}L(FoA-i@=MFZ@IE%hG z@!R9a_v2h1^Q8|8OP3I}W?=}Ap+)2n^C+>AwRB+))&K1Da>7J|_T8BCDmq~7@_sHN zdtvD#c!H%1HbwcR7h#y{_doccoKaFUGZGvfUHSm#yN%?8G&9rkp1GwLLCN*rFy*ik z8D!SrG>7l)yYq_^FMo03H+RXO(Y36+mI(yob!V7%n2m9c&b2XiczPe&w`X{4=}F%- z(iNpKk-U?TQ%RFlwXe_DlPGf7fKjOT*DY7NRbdfJotJeflkAKuXZ;M=JuB;*Gn@yV zz%smW*WwB`JIBWFM~mXMA34?m31eIKX}Fr^E7Y!^gq%G|m_4ly%C%WrMlt0K9bm%tV~#_b z(hG*ZkES^;W5Mlb2wGC?Fyw;%2xDu}GHaJ@u#p*D#^*5FF-y&|X#XR(*?$M74!5Qo zG_ki#;W=jc=vy+Xq@`iJrA*3aoHT5c(4eJ3s>e=}?nk!omM38Z=Ww2}=yq$66-#OB zs8-gS6-q)t8w5TWTR(QmLPvo#MU%8ZJBjVMKQ$W=!jI0& zE6?96BSJ|^TB>i{t;h~h?N{ef0K}M0(j_Q0JBDySs zeJXD;2=l>*sz)ZT58&2%Acn5-d82 z_)BJ6B-hMx*P#Zpd#SLgpiJwrlQ=}3Jew!nM#3>WoTDan=vvGz8*33YLox~m871gQ zF^+YkBw;KkMluNy=yC=IKbgvoXGW9R(TUMaHl2i{v5}OoPDp9&bV~T0E-5MfJ_F4a z+pgK0K9e3A>11nGM~M?8P=ErG&2emQ2IFt%UY(t7%C|_cK14>>X~AUNv6#0LcXC;zLWD z{jA;z)UFVcuOjbPinxMEp0ny91~v;|O;D3$cVP~CU6B@wUTD>|rK6RCNC(*A#Vw|f zEKVPpndx3^;I0)Z*#7EBah4q+PVqkM7K+Z$=+qc8Ss{9$;Lb<*^astyEvALWR7*(d zJ|FOE30&RN94v_?il)NC0+iBM!4%7xz`YjkVl& z*tR%q4K5nL_BLTvkZC3RTFI(b!rgM2S^1tupR}~3r8?_e-ly2~C^lh%**0^)mg_Ce zW=arl>6rkg5!@i3{o%jPeD-IAK704G_dojpj^BpApZ?d$&))j%ZR8!EMRF#Y8k93B zZS(mRKYRAG_dfgdvk&3#CGmity(ymf5$_Tl{`|AcpZ)o>kN@kZpM4w|$bG~6rVmav z@XPqTika|kMO~KE_(d37XQTA_NH{i%)k?8y_N+2S7r~R*_M6C`En7XS-w`;gIcP6<50N1E<&25 zD;+h7*c$XD|L?<-E;8VY7g5o*qC29x#a)sTygzDyMQvMeWHv38XgL>)Q>HuHn8-Kk zcT#E1)U(~3O>A56%=9qw3*e+W<5k!~J6MrL*QC3+b!DWa9GYQpU|di=4zx)8QD!o} zR(s=rA7~lCpIf;U4tkL3?CY3Mq%K9Tg0O5X1wMv^xkc^OO1-2yq*SDiv@-5_Ik-06>9v4r1;+L{uZg0CK4UCKh z7QzU&3MS4IySO11!zeE2IK`HjLX4SSCfca@v_gzbE|}JGIxQ)Jw%k`1tE!DX$~z8; zO0tuZ>LbL26M`Iu?!-|4$9I#y(5-@Hk*_5HUzo!GJ2pXlVzrTL%j#~roSihW0q^H!PrGj#i0E^OmT{gMCEo;6 zeLY;%kV%1Q!8)V|dW3}5TtIQW-zo?*-Y9}jPaH!+Q_ijtgxy&*MqCH3|Jxc$(Z*imn*r}Y;?*l*%PCZ23lsqCq-qSrZ0!r^kqd#rTg`sO^&^C z#iRzE1qm3}h18L2FD+PnlJWC~wnHWa^LPU)Rn96!+QY1swBTk*Dt$r+{{k`zsnFPb z4jp-30gsfVW%`vItGOv&7x*4#rT`!rNm2Ttqm`69+aM^HAD94#$6lt%!!2?=~luq}n*}qjKu&6`xz0?O3O-ctETJ?bn096LN z-Y|$2r;xgfXeA|0iS6?6Mh6X|DN!>;%_=GjUbzA?Rt@xcePt7d(6scSFE5s*u!>RY++;Dl7U)ma_b=Qf{C5+JQ?x@oUGzMfu6P(UMN+iZ zyyhs0v)9i*bL;f0w@&=w=6gTed_`q9PknInlb>FH>EoNfI`ie_XKp_Ilbiqf;g^>m zyZO>@Zod4)KR-Gd8I=ha6+4w+nkvbuzMkjPD2iSa-`*Iknl35*x+Omv+|6-1YqA(_ z(N;wC#-62zY#IfKq@+YZrPwar546Z*-lqG3oX~ywmf<%hkqjL>cVzqMayW1GSdGg_ zDLpp3hG7cwV7GEF@_sQuGdet-y?_6rYvl-yj$OCb*IxW%ml8C>*?jau z@-aL|^9BsFS2VFopwY0Ew9vMaF0xTut1zS0(TrQpEa4VGs?d!^<9lKw(ZbML@J_77 zdGBY_wN19vMosM6qQZk%CA5pr&k_FwLie=`P$C}6RbW2|0IJ*4UCNMA83YZV1$@p5wg5p!c~k8 z=1gomA}fBi1)40_kR?S`M6qY5i%^v^0h1_ni)KKtjCr0SIfX1XN6I4P*V5@+wrjoxz#;5m%>?WNN>=Jsa zqYSPs#7*?q(-z|Y?}N82bO7gZ0ow-V(L+aFA}a!HG%2OU?%^=z{f-U4F4nJ$KYGui59FaFynkyHqXuG*QsOA)gsE1w&S(h^DeruqW3Jt^Mz3efQ z5j-ABz)R5&gY@ymF~AIn-&13|Y^jc0G=;+0L+Y1| zpnk~&>aEud{lmj>n+U?1*8Gh-J|Tm_ryN_W2ll|kp+E|UP@fHQ7g9Jv5Og+!GfCV6 z0gFdT=xvfTWHpeZ;Z>Q~gL!xGP;u#9i!|~x(u$;rvgnN9Qte2!R}xYxK%&}TOs$KF zQ$zHIa=X@4Al7nH0QP<}*cLl%78iPNhAFC)38d1j-D6-1lm`R1?s9S_rHW#nOh(J1 zam>vs0#Qpm&Se90LO3WW1qyq*Rak2WyniaN1Kw{@LBu-cV8=^ox%;_ndNQ@HM*|Bw z*zq~sj$UzKYP5ADoMJBAKZ@ZOQm-f5Dxb6_sj?)ilI&w!k4*2K5sW=4Id(4sFTdgt zS#e&CPK+Uklq;qfY){pMJ1gF|K)AcD3GU8HTXm~7;JV{z8>2SJUx!*QJj@*7g4J9| zWbygPuG;Y2(Rasye=2e^Mn;hMsM>a|UudIDe9Mq`t9~IP_;l3xov^>v0o*j<1eZg* z8Q1C#4l_%)gkxCTHMgQIcUtKfxr0ka(DEb$gchNB-9g+fJt}_9=F8)lSh`?UVRPTV z^rF3BE?tT|{JXI!-ir2uvdnyGyh>y1Q$p{3YC=-deK*O5TABZWZMW0~=SU1u2ntb$ z>`YrFX8nkPrra`oN|$<%DGlpd32k1h3c%5CX}(r?#xc4@fEy<^qiT{8;C|eO=Jblg zpHq0bYd-&2X0)nH8MPIWD5XNXL7*_YIVyzicSMg$P+MKzz`}PidzkPXIW{!HnG0%O zj54A62l&-IrG5DUTwfQKl$6f&{Z9VG8;4e?)4^)#&=aU;q(m}6D`SUWD7y$Na6G&8 z242IFG`c_FNE$6t^KN+3qXPlc2rNZ|Vw5Lnm}A#qk#UeS2}w?>qK`^i?#vVtACZc^ zKDNA+w;k6;6#&WOxNv*vLDv*X>vPH8Vf?EQUhY!LMJ#!*X_DGmCBhbg!ynKn;yG$! zE8nfiZ9HU>lCJj`N((YX@O9X%YeDsR(4-|LK#Sm()`~TOG0QDF=h(D0-c(j5G#rdLl{gII#!wY% zzlFzpulBKcwx8E7*50XIgr_@s$iS=_jD_2@?iu6RB^aRo-e0zG#pEZ|`a zUN&cDK-*eGm15Ogz0RE?XBC#8nGvsejTWixy|l*?eYoq_8P7BC;(j7zPQ@WykPR-P zSzM;3BuN{-8)K{6y61)1f-v53xGAKA_8BOP z2;I~G_JKQW|JP3a{@UrsubqBzGnmiKs9gWWJGUPB>6f2A_T`6)DP&fC7;a8znO3iS7`iSg zso3RQ+rVmOW!JVbDibb>V0)->>M1GFZ!z&LNrDxdxOuM|DYKZ<(>y^vBPp5Ie3%jo ziPzWfK7z6U%-4T9^Yu&5X1{s*`G36k2KwfyGhe^>Ji7h#NB?;5b@a`-*KePD_KOog zote4)n=?DUe(9ro8t>RQau9v}^6UTj^W~NMW@dKWe*U9-y!Su*=0Dy)v8f>b`r^~K z&%FK3vzO71m8ZXb<^_1F=nT`hpFVT@<#V@By$(a{_c*q<8|1YyRub8SEGfyprco!L zsT7%k-TFg!D9IX4wAFAuv;`Kbl<2o>UQd#V)M>CRD_%36?gbg6?~Vd3Fx~r3#}?OC z$+c|9B)n&8h5#lhrS(pTB6lra5K%3V!J8rg+)6RWu@QOnC$S^gX-(t{VJSY-HJs%t z20txjlChJZVeqGm1j(6nLS&{RK`Cc%cmVFVi9hrXXW2cu{e%sTGRq*`iLS$WZ|`(} z%CuPqeQDB=Ce&cO2zIND<}opIoDrkc%Y93hrDP!IF7&=7r}Q+!D+O-%pdDU4!2xO$ zwDbZx%3MK$giH)e%65qGrQEq0@udp}e29&93<9#dD4P=ne`u0ZYk|(p{44qwxV)E% zFNH>nT$EU23yla0+L@V+o>I9sf$p5k$3{1|=aSlC6-1_RxX0{q<8%-G1!+?T?@O$Dc2M{o7~1dFrDz!t?urCGm$y#m$MJ zFq=jD7$+N}!NKjvpTGUenaG&PiY&?Lw4|sRNl(R(#S2hmDU*oZjc4n~^_6^ZR<3Yj z+rmZAC1-ZLrJ};_)(_k@yKBlUPTgM@LD3~Gkbk@#L8*J6tENbbVcaP)&Wlych#%|j zPytp_QZjzt-ZlxQbv*k?Zo;4x=sRgmv8_2EZQMwGRd#cp6P99PL$-v zuZs8lKcn!*f>#R<*O_I5f#msA(JW_WuiU+Uy;1f)?qCa}9c5Un2F2Dri>((#gE_YL zTkH3Wg4+il6rNDzV6&7Ro}&h_ENoMI5!6Q5;n29K5Z?Iiqvk2nIvA1XhZu+^zftsZ zP%$$^Ke~R`kme*LQFT}peY$%kGgZ(qGp*sXoo6+<(&ME z_VI3c35KnFkz+K;ah}*kG|A>+40d@W6JGB6DB~cd@CwmV#Kdf;OdX?0{5jc!TKkGw zZS8_tF-7z?yoY^{Tsx1P=`JvfSa!D(=3<$%YD!Y%zB$|MW3bp98!WigA^=Y>oee5% zYf9fQcgHVhvU09xtgZr&jFb)}Zb$1!I7JHO@b}tR_;aKF9yLf znBxSO1Z3U-AZ&QuG=!?kK-(dRS|Y8@Q{!MDruHE2iiDTDzmI=6u`WfEl_c8b=8uA+ zA45|G_16GNjR#X{pGTsxmn)(7a!qT)7Z!RihpqSiio@nDG~rsM&6jgI!d;6YK|4M) z?Ai;28b!;sV9%+`qXRJCN?@*)md?Ag(KVpGCPNc2^W{t?(|{@6Sb*u!A$~0>DVf*? z#nw@QR#J`}QU-C4tvoAYu-Ce1KEzUpw6vrJvW~VPI;#pQ=0+#nNL_&>BV{`4-M2L$ z@q1c175Bxr9|k+N7~rLvGK!3uYjrr8+&9ny2<)#VYCB$+=-%hC z-iPOe?}nN1HG*giOpOgRqcJc=t$cCd4j`HW6Iu4CDHYY%+{w2r!mbKgREa0_*pSI^W!r?2+FN+J2@i<@&(sGeGq|(4dZ{!u~A@d4lQ8F$kTk{J0`NSRM6aVVM z?N`ohAi^IRUX}Dz>`vlaPB;~L)h4}bRv`cZ6|OVgdTRjEq{(r5U3o@#=Sv z4A>MEj$CY(J5ncPT^PCjHRRQii7vxMj0z^s#b{d9g%}Osk(5#aMPfVR;os9>(Yu12 z32aomw-P5MrPRLDpxJ3)x<=Oul;f=#P`>-FoaQ~u%ik)Kt}rsj!9+<&s@AHTMTyFc z;)>&Tw~yF&5~I8iAEi0C2N%%;cpuD zbG^+~0(cZjms@wM@Tr{4azrL7*<@auNs2AvP(+S79NWwUBt%c>SwUR|mtp8k3>N?` zI9LD-h|#e&oV)(&{<~oOwUS7 znM7>bw|Y6i{Zr^*g;;2wiR45iFfz%;Cy$+E+`*<##CYe~QCDPAIEc&%X3iloVrYs~ zKn=(dj^r3+Jhwny5Z@L5i~M1Shs3 zI5mgHEfa>uUIYXke2m&2aVfJAsb8O&K~-ena*o0^_$<&OzM}-dE-k!KW(tsOw3mp~ zMl7BT9ay?>jK*j`dS}==@}H@Sq{^|WMi{TayGVY+ueFJ~OL;Zp49>DY@MhS4R$$W6 zN+G(PgH1e#{XDRZ?JpI2`%^_-RP3b8t*Q?&K_K>FCf9JgL@8Jw>0m);!*eGa)p}Fm5MPuNAm^@Fthwp$|s~k1Ex#q04Yn9 z!nDI6k2xhdv}3YZwQ1>|&5&zw#P@V7cr=P)4+&$=W4Wk5(a^t(%obucxV0^lKL6SHceDcCAtGhFC~Mn0HqtCAwT3Jp2pz=frQ}@ADg8Xtfa1yH*}o(TK@dzKT}PcHFNF?x!RlJ%wtF z63^G+^QJb`2_>{Tp=pxbZ~C+vm2kY}Z`r*RJen?PfePJz=j(1P?YrLjTC};0)8caN z*4G(dNn|Gc97;;*RI62Ih7D#7JAZo-yCvbsv9JrPf*oaRvhfYOc}`s$FRmWcIr#3o z^G29LAaTps&Ngmcy(IYd@|ah}uPPB3I=F_kw46}9cJrHQd*5ebTKVfQ^PnXh{o8B& zZtUa)n+{x7POH~gNp5rQ*vWBr6iNs*EP=J4&QrQR{d z9gZs|qxtyB{v9f+x+W=sc5e&C@Dk?cpPv8fm%q9B)`^?He)FFnJ#q8h6SsbH=H{c1 zYzBv9GcGrN_p9q~{OqgepGP}xo&C>Szq@q($rq!WbbCQvQvIdPvfLjcc}hnCn!I2u#tnI0{h1?G*Q2$Ztbn!A`G?_ zbFXm$R9NJ{gIY&jPKkZz!YmfsyyIQ+E^snSK$aN9yW@YsDY7<$N!O<5?E}~-l>x6r zddM5U$djk>0e&qd;mb|%PC`ziw3LqDBVo(OR7=#$Rm@@T*tDb6pU!i#lx!j)X#oR# zJ1Q)VdXb871=$s9^s>XCA_yily`L?&S}uyqF7F|gfg-pi`f;{4E5#|Y1tR)V_jIu& zrGyrHv22rQb-m=ecOe6SDJ3Q3*8J?yie_VcZ`K_yGv*Tu^ zI3!3Z;L3aO^Wp#g$<>o}mWs2>?=QasH=S9&u;{SJy0Ks%J=J(~>Mn@bd&CZQgc)F=*e`7AhAgF^ShgzA2bJBqtP43hawm!Qpr}4H5(R? z$))pjjyl9yI=^(@`i>kOu_0JD@#-+8B`_CD)U04?xvkW6>B zzon$4)+HD3CiAtAVt6kaEfT9v9UTn=5e%NkzVP}Q(e?F$>x!T@NCf(#^&9`(wM}z^ zEi6!iMhK4Xgl*I6M{N(ex*p_;EXkSJHLzPqel@gk#yPP&kzO4NE=85(MC{l-Y=VG9 z-Zi|C$EMo`r&yWH7d`S$I4ZJvv;&(tw;Ww8{RbfxoBFYkePD!)(JP`ZKdr`Y{~995 z<;xZn6yJ+pdf`snadV5bELw15%OE>VHviB=8$R09hOxYnMOZPh!30lCbf40iiG*51 z%xNLBmK0IVAn2~ita%t0K!${e*OlA|2n zkq7h<69H!nn`RZwh@bQqBrdcTXQ*Q>-Z6#lN-ARyYlS>+jGDxgbGz@=WCiHcm zs8e7UVZ1;w+D!|^l@qDt^r$wesgS?0g2M<0t3X8PBcr~l? z{9xImj`Va3VmlZN6uT(yx5c_2o)t|o0K-*uMbGZh#of$-&)5Ww4p*PlZ^(pw_9-`y zqXRun%RbjYvSDQ=Lz|guLdpaQ5&Cw{U0dr6MA(?OY1>&(XigD>6GzBRVf>&2;!D@x z(&?(#+sgudU11;!`?Xdph`Zs1LG9Vv$5&3)KCQi1`{2r{+9y{|TzP!+Dc`l{;hUeV zeNg+P_Kx@S;*}>*?cFPfMataYP}xIP_?lE(o7So)(r~6>CYbt$c6k(I30mfya#uU!a9>yzMR{ zX@Pbdq5T|($Q-CPB#&eS_c|z7U9(z#YX6A1WpU+ay;fcnn;=KT09IcXCo?nmWYHc6 zQQP{SBYfKVdp+m32-~APBXy6s3D=RsJioW?V8>qZi;L_M1mDFH-wowyUJ}3Q$Ygf< z$l~-7`0Pz47biHj3~GByqo9(AuG{ZolP-td{667iQM9IbaAu8waAv(pqsx*W;F#Qo zTkx=5EmUKtsdufFE0FGGW_~w4*ts+Q zKy(LFnPOj%P;ZDmJ+#=u?gP zq78g-o7{%$6I8e|Ou|YLbJqr2qKr*k#a0Yvo-QbL01GH}0p~_&rq9Wx7e(F>GC0^q zqI->n{_@)K;}c$E2rxGdm+%Th27DD8jP3vRKL!WUxMkoRSpboN1_%E)TC`X>$5*)# zf`4|RnQ#+uESj?R zw6<6kwZ*9bOU?G9M00a6yQ1UJmH{mu4(Sd{Nu~&=-EHyv-MEy-IknI(j;oOeUr8j? zY%(j$dL}zO4F4q4pzNp8x}F`LP{*?A0Omj$zr=(uDw%4e-ljsP-m0R;G^zLFlFb1a zFQ^|sJlUb)AS;p_fW!!!GBb%y6LVstl}jX#Kr#8@z%<8s*D&bPC4^mvm6u-da=>9h z370tNAQqd0cU9oMFot!1IgChQ=_6wQZ~*PtO~fY>Yv~p2Ai+M0IedV)^PuUGJ{=

wlh8P_wQpZb(-peHk?HD6qj0n%_(yQ=B6A5k>bLoY9&=`z|;V}3P;tQZz46hbk z%VR}1@zN{IuD3FUEBWu$eGv`f=fn~_;(9(p? zLan`1yIlLAcKON~@8`uU53eL=ZXz^xGq4lU;fX_$5f}q$YVT(i>T5)#Yd%{mGze)a z)9)PM`A#h2mO(;#;7+z)1j(-sqP|TWR6ixm+N_R6W0YZC!MeQcCZqW_Vp~U=q%nc#d*+wwLZcB`xV%e;=&NCKFeRoMLp8 zVhe1z0$2nEV%gp51R4*ZjHIirc^u(M1YcVI)AB17`2r`n|5gUt$01>nwJ9D6WqC<4n zBxNsp+s|x^Iwx8OG4i}3CJKNs&3omnLb&{^*W(n{ z%jyMYq4~;YSUq=YYa1Z>%590`faVs1Th0H@JDCi=yppiZ8r7_xkYu&bE4&)2@UqH| z!VOk0ec`$+r2?hBeU18CHHM`(7&HesPKk14z>hYJji ziUF?n4dXb5F+7%_c`6FZQrn{}^>{4z)T#G7HIkL~JWtB{OIpxV_wB4zr#dy5$)eM3 zbkDxhlaiJWo$T7*Vpp?cX^zksHCFBR*Y*{Y>lRGvX-NxI=0)QFe6jXv?edkAwNGjv zpxPz)@vdOZMB@LSUpal{RPAEz;Cvd;7|XD^GY$vm(R0pRf|P{4_W|dROl_q^Q7}y*ckhyCQ6hwy-y+FL!@f-h;>b?7bdqFYZWsj7zeP4`NGJ-=`30h zB;WItSd^dys+hF2bZH*V!!hPgwdDTo6!s{7H!?me+HTc$g3STN#q6Eu9m;WBa*zJ? ztU*wL<5{>TMgVi=!q^2XJ@J7EUx?n`-v}oWRwbmEsYjXfs{o+=wCOyvOYpA*IDp- z;jFH!l`Gl9jGJ(22)n{sETcn3A_%lyQCn-<#EPD5vocuzP*>C(c1=`(XY^Jn721_$ zm;_tMt=wDhlmd%wXnN79R@e^v9?uz-pP9+R)-R%9;;{>y5cI{vzgW5Sy*t->ymY%2 zH`_}%BNl@BjZ2a79K&-oZvY6)@9%GT`@^nUS&#dF(L63J9B^4=sXGm|09NDw#XCf)$(J30e`R zg0~Cy(fVZ*TjH{pohmL?m`&}Y%<4IDSN*0a5j$39RkuXZ6fWWDlg1kXG{Ne~_?EKb z%Oa_tgq%PbDc!GNZd%SZE(>WNzxtuWH7yF51J++?rN!&yDiARm{qHvN@>r)$-(-(#Hntba>y*USQ%a=4Oo(m{)^%;BrJj~CU$y9Gcy#!b(qKI>UHiE zIjb-*3dJj4XJu-8FYOU*`%e8j<9X&?^tECZ@(<18G8F_+8@}&yYom&C#b&nfeb#HF z0cf*}O_x)Nh7|OULr$cWq${mahA3q=nH$faljR$~eCgWB$FH4wRMz2eGelI6_#Uu1 zorj*B+!#x!TX`j#s9@GIvGaVky1iZ|f_9+Z%c@c|Nv?-Y5;f6HR+eLvuuKZA7yx?J zCTP|puoF$=-JD66E|AUTo!sx1u#I-)35Z?GYlIZt>FljWFvId_+7-#kK5eF<%!Ecp zmvp&rhqY{R70l=8tO>HDhaN9AB_#qCprIB2qT^JuLqqfP^Ab!nCQ-{7GAi~Ev4+IC zpbQuxnbnMx4m6AExBj?myXOJMD@71!U8#(2pXwv83%Dk?p==Hf`2Cf9 z-FKBI4yN_kTb|U>e(=F*Q9(2=p@WjJ`fO$mdBS3j$TKq%+A(r)`n!s`#YRh1LJc(4 zMP)}#mw=E$d-m)Z8e4trJyxC?#2#Avks|}_`#rv8?W0PU!*Zo-qFJpRNRbR}FMsjy zFRq<>|Jvzu*G_+Q?erhv-wnUtHm|ZTKYa7nZ+D{>3#yFLp7%gmhxb4eb;7#buLD?3S_V_Rm+gxH4@FV~ zPWou^w{gpvCEOxN79kZ{z>dq&j_GQJc}eo*p2d}mn+&39QWnfQPK>3QQp9AkA16^iF4pU zkpl#0ihP%>UE3$YPcOKVl5q-8E*s%%*TU;)khoaSwqwKG*$?e8u)?tZjk|`GRMf^R zk_|U%=@zW2!(uqE+FN-*M#+P!cni!=sMl-nRYFJPbiE6E-uvO!OrLo1LAPyioL2>1gRjhSNI^CoVOi zWYwhrJvst`9En{5M~qpXo3gwm(G+Zw6fr3=f{UnQgC5#gWQjZ>?2=2#6Yw!k$&c#T zm07rTH?v6ZuI^C~0 znxQ>wn#7G?baa_8i%kFI7hitz-t?Kbr!T&F=lxH@$hk4kGqTpOZ&F85DOxJ!vnh0+ zO`(UkDYA+h@arOhN1JpkPmMCcRf!9;gQ6P!!5E+>wM%5MCB!0{ws6i_TkJW|5{jDW zgj;6Kn5r?Kibw)qYz0&4Jp@$-^E8`e9fk zwNEDx)-KdO@US@pSU|3q(pg`76K=T<|G!zgaln6qs|*oYnf###u0H?9o3)FRhxYD$ zj3Dgi2em7c2PY5JuGX#+_!@84eqQ@^S30w@f~B>ed$aYS2kChOMuTvlA0xo&mD(pT zt2ZYPlG@9)58$qwFbfxIAJ^}!U5Nbr>|v(ag*XTjK@oOV;pu=a&%e4CUfhd5Nl8UZ zc9KG~^&~a}8yKJSx+B>}TUbje=|E$5=%DH#*11&>KVvS@3W{_WeC>jeHfCUZhVf+K z*FmeoMQXa%SUn9(Er-<8uPbYDJ|Sd*V@}zbI7c|y2wPZWkq%JTpz|vqd?C)z0ZvU< za$0O?!$)tSfmQ*BrqE7z6C11(o&2jW=8QdUPF#^JRT?=(S3?J!*X-g*o(j(KJU0`Z zjzPyQa6vpWdO3CmjSi!uPgUqqJ?%3lL;l8jST|79F5SevW|?u zf;8N}Z`n#<+OBwL@}JZV%i6Wpo0ge{!{3WO9~zjJ9jc9boV8jSv(cg?Fc%fdY$~Zy zS_+clXty<%e6v01ucpB5_k`C*goZ>K$* zOe>n!NuSJYxO@qwY)hn(Q8HRb{rQtwu4zzf3EAJJs=FMhyG|V2=kHmpB4i815o>{@ zR7DFQm~}T_F zmAr{1yv6u*P_6K#QfVcbY%XApM2Yu?mm<16c?9!D;M98okQ zS6U|X2URFh`!xn^V3Z%$2R8GiXA^J>yxLv+`TeG7@;RI$3<2TvenjAHS8E^P+afMGG)Q-M2V%fi-dW!&6`V;`O`d58i$4{r`9Ug}avze*OH> zyFWg$@|MU~ub!B`{?VORPDB>R6JnnL`^~S$(>^U8UKnX55kD&WXgf}lwUHx8M}UJ& zd_WM?9aYZE5hhLQ4#V2m#+V#u+$gc--?r3*Gt6kwWe9S$jTKp(KHk9+x8Ru^WXOY| z`WpbFf(+}3OOOi0fyyGJT)w@0f*=R{w;vw;_N`aD|Nio+Z?2ppe}C!dx2I2$doN%A z=E{5I??S-<`a7_&u1?r6q^b2yGDD217%7(|x2u ztE87Y7A4#xe%imXQg%HKmJw2i6kL+2R~1(Bk`XPD$dHLQ>qCnyq20VJT$B0=HyS!B zq=rg+s#}mxL_B_#F)JFj-vo%4ku3k8Kkl+D55N ztpYDlSVZepO(BEz1;ixL>Nq&jWVOZIN69uJS-ByqRH`8>?>MBeu(hVsr|!$@jdHV1 z{KsiT4bTGUdsQ=}GzwDWEk+z>lq_zHTgx9|eZCCTWR?uLpB*lf9_fskOmYqeUAfHJ zzR2R(3h%D?d&aud?7}V2aD7pcWR+~7rf7985N>ixs8w`rhYt(0RU9iZoS!t}17teY zI`!HRQI!I5V6c`99{YAmMN^WU&blUTViuSsM%#mXl16lZbh=q2QCQcepMCT0x#M5n zeEsV;udjp%+5oHqYWl*=-g)=*nX`9ZIWc|ick8Er^3wFPuhj<>Jx8`6<65dwlaLBo zlaNMqfMlScDoUJl`rOs&Q$PFq7pK1b^zc^)kKLy=V0pKRfBoAx?wow%%a5P?`t|7a zcdDu?nx0hDoTlXbq0Y$F{~(RDl3E(J$u=+0**B}hsEw=tubxXA|B2X=`|5@ zKuEHu#GUAD$^D4ZY6_nv?DG`;;z*?zl}AoV1PBPDc=a1u2~2vjVb`u74i5Bx|Azy+ zb`I_A+qvzBaAoJNm7rM`_47AMxMGoU7v$u-FPf}2>^DTcO?OV^`v@|!Aqg_lh;Xok zetIn4d9!w<_R-`KtTCB9j6F<~&k-!u!OD^glP6{iQ?TC%%W<$E2YYp{PQHLqo5>?y zr}&9Qi*nWsYzt_n0f9K2%R)2Neg*?QIC&Dgw!Aj@n+psVF0E5@sZ?}c!>p#LnGEtc zffe9eoRNFp(@15W%Eoq4AN3p^jGq)-6AQK5M+%z8{&!vP1ls0mZ;+CjSd7^`KHv*Bj->e%cA}P4M{|@(@Dn zU|sZKxFd!OoXLU>p(`rl+xqN?RJKvH>m7#^e^ONvol^WpwA-MS>JmDNS`kxCYDy;H z2(P-p#0JBJr^&U6L`aVq#4@dmOkksMy0b=nLGJzB8sp5eqX#Eik~K}Hl~kewOYG%> z#=y2Fd9bgCU2~+3{7Y%<7tU(2!5BKYno*@ch~WBGNsvvFF)R8~41#MW7q%ahi;F#{ zs2w!wxpJ&wDGccr+FFp6N+`Mbbm2YuyksMsBA65o7t3t74|uDv>gG*ukWnGawnsoa zAYf@d95J=Z^BpR2SwNChZZVH1_X(v}*{+byOgiJD7+q{7>X6;Mz`%(si7MITjy1&R zqe#_4Z=@+O00S;rC%iWs-eVOu2D?2mU4yp@elOlK1G_NUF+&<=$Q%(QjcXXT!VbYk z<^WA##}0XLc1VevC0LGFC`OB|utQ>1>?7ZJpe&r(_k&;DBh2BMz88PaQZmD3mcij^ zUN~6QzabE;+QejqT4T#1iM5L&cH{=vF2g}k6l(^D*31{hnnAiBzH(QORiY*eVMZ!B ztg^@|Y9gg*Nnfo+BRv^9Iho2Ssa$LfS|*CAUZm(arED zD3!v>`9ZZ)86SjzcM?)W3n52_7LoS{8?+8%-AP&Ks&JSe%Tv_Mc4*$@b_wsd3Aad( z&FeG|^`!=)+}wAdPWRBlb@reP_}Otb#2J3%8>sD&Azl$O`ikTg-%?CB?D}+Q!(KwR=sn0xcV3`&L}{8rUh46J1LrBG3JR54<>X2Xw}3PS18S#6`&*hZ5-yGuvQR z`%=Up5wP6@^8Fjq;9= ziWVR<3=IeM^wYgt1`ceahE145V7{eI3Tqj4pLwRM>wm-l+8J>OBb3J)wB>DBDL-IRZ(P+nmrL9wV+{_*3?^5A5Oh}>(JC$kjvD&Q&(;szxC{` zL*2KIOyc*FZs=i&Xw;N9?wcgNH_ z@b5hQd%GK6fzQAX#lJzk(?hpT;%6OL@^#y><$Qdz`>8 zY(&9g!a~T%$4TC#RvD!Q*1%Iw^|-bJ_j78Ip1hE}$St_vklZMPE#xp-@#5H4 z5&OXLP|c>Z9^>7z$;pI~^P))^HK8hM8r&@%;ofqQWxt7=FoAxM+C<6+6X2bMl)a{8 z1DzAC+Cf*;Qrsvc<3)?)S+PiEbl;+BTTC`g(qw3hR7)w@faw#u2^Ri-H|Z%mE)rI9 zCYjen%4~o z%C!(qA&+FQW7XL0MN4!MG_d;uP&BJhAZR{-Hf0>lX$!>SxU$&J68H>KXs1%mJ6Pzr z$ohXFJ%tLlIGPApBefFx`R9wIk6I3V?*ao;$Qt#c6cPO+Fq?`L{=7eQ=!927u3mf9KYNM;p10IzK1SWpVF!8J_+!8#R0 z02p*Y)#4;P_}<2JLi@+CxC5wBn1tWbS%6@C&6XW&=F4Wy7ToqcyKc6mFp&YXK}%*7 zHRmIYh+(y*Qc5zhlt!UJ$Hjh4YUbU_eL7B;*S^+Nf{PeUk{e_jFP0p#6IKsHU!D}e zUK2&+R*OLbFOdRd!?vA1MU!kSAMZG%c+yHD)tu=V4hw}pb>-HfsdKkZ{O#cLQ*ZzI zc{26x)Z0@Z+&V#~&P~02>m*$H&)+}&^wfKpJ;kdZ+&VsW{=d)v`FSr}dJHZds2}}s z>hje4Q&+zG-{+?;Vz!ywI{Dw{;iZd>8!&MVpEz~#yPKFSP_l=46{E_&YLOh6diU1J zsSn}RbN(C20r-$Z_@yxEJat}Cl>fX&@SDO{dVA_T`R}uMdAVEu^B<_d6h(e|rathd zU?!V=5{2jwtfW%D&`K(F0yC9SlAQ?TJpBmkzRlVZLqh?Vxs|K?)=ju}aq1F0k5iX% z(M|mZPTnC{RsxIiC_ZJE@Hr!YzWC=$f4=(Xiv%8C`0?(o6L9m}SbE}BnGl8G^LUcn zI&tgRe;@Z&{qdQ5&v_pJx4-LcBY4xXTZg@xl&Rm;m-emW%W!dr=mmf;iSzf4w~S|3 z_}sPb`%G`-N9z8cGxK=qdZVYqI}Z1N=?Nu~h|K{B+I@&ssuS-z3`rD|DxqTHn&T0` ztvpRXM5WcmoUTpSBp3s1Y!|OkzvJjjmxd3q=&F*{nq9}xA@17cIcVBz9!l#4M7O_y z)jpGlCZ8wxsvor#L(sJkFw=bvjz6tkA%oj`zQ1AD0+L&H5VlQaK);zhSi3TLtakHp zQhRCgCv~{q$TG|(luR1;T*cRilWr`nboiB(RFVP4$mP_D^EjF7LhTvqesTx=Jm6Z! z46}&M3VZiHKKtS}c%@Kp*23$=p+)-<>*N`d)UxX-I*(~-iM>32Yd*BP*L5WkXyxe? zOOSra>;kj8wlhBQ23vZ`gxaXgR6|#0VpD#$!vlxCB;|<@#~vq}=~$(0T_k!|Nd?$o zmecn7h;5JaLS)i^@Nzz<{QSOw;+NYo1#;Uv^s<3&|A!?Qn7$&vt@4|MJKfn<`-JzDjzH5rnXP#NR z+yf9p^NKd{th-NA8|Fzk+$Od#1&cu>vxl1{W>&aG2^^XYk;bTe&tk{Fp6HaMHR2DZ zs4}~K0BTueq0g$Z*Vry*8?IL)EvxW$SYMJ2+hpAj)_7pfKUhPCrKrHg5t7a(bUmx- z`h%*0en6sv@90S8z+dFijS~3s70!Bh4*Bw0xkkyecN|hq`T}eUq2=U^RR!v@l~vuc zOB9(zEK7!uSdUd?CO0Tqx5X-gu^7DqX~U>%R((n-GT%=YiOv&&f}_>Afz9RNy@m(Z z6tSD{mXwFjVk^F$YtCIoBK~=?cC~i1b_pv3Cy&lB0j{vHR_%4{i9*^#?qoX{Gp~+8Yo=kjTKYo&2>QLmiveR3qzw~mV5D@pJHAbg48j^_Fa#JJ z>5&y7$H-tD_UV7;#c^KL67x8XGrpLjcM?)kx{~hjJo&s{i#xo~#-w`|(Hc^jPM~+L z%DIAF&m+9hUVxe`LL+3;o$7P1MGhgHC^1TM%ocW4#4dDhga$Hz~IFfz#+8<6Vd#f)%TSi?{{=+W|<6sDk)6wV^f`j8_ zj(WTx$}#+ z?woq>&Z}?Ued*erBgdxSym8jgDPTxNK``d>v-#&cx_K~wNq^}Pi|LUy|B8&I-p$~5# ze(v_6cW)m$>RW^fKZ;A}K!cW3a)Gk;)wngz*x=ao(r)`ZsVd1-Y*+Qs-x}K~8Zhy) z%^TvXOLY9IcA@ss$~=N9gL8`!+2NoupL;oN#i zop1ker>ii@4Z7AQJJ^1Q42de)4sSHcI%CW*8PTKXbX8STv_wWx!9&gYyr_k#7Z(>( zO34Je;8vkN@_eW|KN7KBvs!W`YacjI77+=O0UGF4D3?ag-J}j5mLdjKNJ~=g$&8Xo zcZwhyh8xIZ%v&*SwGVPiGEfN7G2LcNw-L&1jwnA4D>5nTZ9MG~RVtw*vmMq_|BfcC zA~&Sl9yE_ELKGwdXdbImBK0jz6hzy1v{X{j)Xt8!_>Pum5-aey$NAe^lY3%DS;ZV@6(Nj(3JL`@sEP*t?hf=kEs{DxLql_Hg9d(!%}kGuJA@ zJ&iF%PcHT3!C|b35zhQ248A~J=r~e(p`x#hFsF z<6f27$Bz@3-)k8E@XG40P9DSy7rc>Pn|!`@C1%lQ>){%+2tScZcG^FD$uOI_1yP`^ z5(6Z8`fBu_9b z=%Xgt3f4h^uDpov00QMnq+O6s$(4_jo&tvtxk}tZqrJh7hQUwaR~PrLL!qaCpsL$bQdXpI``h$&vC|yxfW(qn4;eHzgbim z+K%1ca7}=m5<`}Ez8fp7V%qTag z$x%=DQIL=WW-2Lgk7Wa&2|=Dz6lFzi61`OBVJM`ZkzOGkVMfoA_c)H2N4ubA{6#;Z ztA2D%$pwJ*R^h(wk62e>=Zi<9+`5Rho>KI5r{LEY->C@6o$)1js7sZIu4HntBW{m+ zD8WS*2HVY7vA?oa#Jch1lu`8fmgs7c6Ss_(#d_k2DZ5l6v~8Z4`UchRd9qqj64_7z zBb$$-w}5CNMvN>e*hx&<-|3oZU3tl< z$A9Jxo=`9iWdY9}0m5QlDn(i|U`k|g3z4qL7D0?!VSYz4tL4CrvSK8P2H;sRN}P6O zvOW_pIvx=SRa4SSE9-v5)>%BCyoIYQrKAF}&<=VR_<_t8X7ob)7NB;4P|}Lp312mm zcQ!YxrI<^lRYAoZ0yAiV0`Bor_!!n#C*S@0+OJn$#A^DDBX@rL{`60N_4TWtPQP~J z>t7zbbLx}nQ_(4F2bfiKxkUy`ydWCV&>G<>Eo6Kuhwux@O!LNNgx$EXdh}P5M?6T0 z3kW${yW!!CV&VbY*e9P~d19))>0uCHp5$X3&L{upAY3@e-~_PMWLyY|)ZUYUOO1h%M5AA5WH{g3bb{POg%V=GKf)7P)udH($L z`3sTd(929Q!U~l-f$i*qN%{DzQX#>Wa|nx&4s3Mc!IfceLu`&(>&gb^PyPdfJ*b`W z(%T4+eDHBHlLo(8yAeI@>$!DRA$(AOdQxwMr|TynWsz3&KpHEwwG2x`)fMstEy>mQbRvstGeQNCO>*K)-eNxH zq{yZ3Ic&R6#4mkc6=AmuHKeFAvsHzoG2(6@r<{^aHHWcFVK=w!tT=H_f(u&a%U-n% z(!44B?e^Yz_ht@{*4X&OB{D25N7%&ROxr@(w^DnDRdNFxvs-mWw<2Nh9M`U_+0XQpnG%Qfz85!Pgnub(%Eq~qL`{mU`cMczm zZl&kbi8TD@+Y}=ssDcFa1B`*JeE%#dxuD$Kya`NXk{8LpIGp2z|JZ8;M& z>KNz0zkr5HjO?Ig_)fi2AUUH=kpG;l8`@1^{`rEC!)yA6*36d}LeN=Y?{dS%EpA41 z8BI|WimGOmWO`|W%YmZn)X>&zHOIitGvik*EHT2JOencPgnKoLVy?``HcEV4IIK!z zHx4b)LrNxLUyIMJbpxZflg~{a>n7^sWb#PuGRA^D6x*9{=^?3{*m9Xcn!Qr{Wb&{t zIOMz9T{{J%LIj0RY8TeeymHYS;`Q1MxO|0p0sVtsod4?N@fi3CTjjjs#(1t%GNtHR z>?Y$?5t4vOK-PtJ1|*WYqNifx0v4pmTZ~9npk-rhpU20yqD(p|h9ou0Op?K&$Wj>C zw4}TPcHzgz{~h)Ux3>?HJp;>r(ift(d2lel4QAbtw7NO3^**o)I+2415jhx>#j$a? z9HU)oUi2W{5OX@ToKp!U9iWG^!tQpSpf(L2*uj}Q%8-c)S(PaEw^S%CgXjp%8}d@+ zW%Ze7$UhNP1IMFUr&>*D-Kiv5*SBGC=rOVr-ZRQ9OfBT$Kzen0qB{GTYuD~(78`e& zN!PAzD|1x{cHNyLOq#4niYzZ(US>WB_P|Ts*`$(9BYmfujQjvRLG2RV=L-=w0!W$A z0i=|sq>~+xAZ%e2?dvwrVYhiIK*SS;)Ob7hu>J2Bu-F(3oo|##2loIX6(JT$6ZY`L zHOZWbvk25F#R#WdMpu2ggrZ{*D5Geptbb!n`gX?9g^{VWq6aYU4LgE7gd7&8C~`(9 zDZ8e_N3iVpU)B?>iCl5qRlNw+<}h>O(l`u9es(^3xO>+CE7Q+@>02m`4NDEbVP%wL zz#(3Z8`fNq!Zlf3VA~~nOR1cq#rJ6+Az6U(FzMs+ClNIm2UC`s=ly7^DinUdHF|GFC zAzF%1k_n^=Po_FT@%b(Ld9}1OP{yT-1aZulD8H6;k2UahAOScAq-! z0U@1<)21j#mce!^i`>zDQ2t%n1Gan+ah=6xZBe@U580P24E4oRZc$ zTiKe4L)fqo9q%YqVX2t0S<#lLQdCVz>9O&Z zVN2W0Z3hb_{$;&~vy5pKRzNftDs5&_bJ=fo4~^-$A)fHkNhyg;r+MTdlz?nwd1}Rh z%bj2U(J;5uq0McN$h(tEYu+AhgXX1Ya>wTMUh3YMQ8HTWSoI?$7>8P8`$Vo#fl!z4y&WS7+|qyLa8aQ`aB!-v8?R z-+X*oC_zU;9^? zbccnv(%RF6f`L~6cmIaCx!T1lxL^T7+FGE#wi z?e>)(pgb~4E>KLk8s*VYG$UM?a0?6{{GvA5bQ;)n0Uq1XtQ7EzR`Rq|rWOchYu(@D z-PmG_YUW8q_$f+|LcuG9w~C-@X#Hlm$AJNK*ZtepySWcY0rfIjrnYal^Uk|_%tA?6 z)>zk2iB*POqueEiyPg1LyN`Fx%tqH1Y_(?K<9C&euBvO5yTIu!zz>@hl&+*+sdQCK z!Vxz7=*o|E^(bAUT;g3@Bps!e(v?c;`?Xw3*;kDo(%nO4#iXOYJvA~?Dm*MnRY@d5 z_r$E(MqSD3A*gx%62037kURpXl~}?dBMdxE&I*q$ApB+0Wv;4=t?BIo?5SQ;h(Zzm z1%VT}k*2EIqKd-pCR+~u@O zds@Y5$h3amJ;)3JyBOKzGAU7yVu-|K>wB7q96d%kaaPV}4Cz|9r}q9WwJ2*{(F44_ zQ6k72s5Hkwmcv+=-{ZwBhpB;Z#vZh!taxbP3(>hOkEQvK^zANcl$c3+;TeTRI>4>y z(PS-=RlLec1b%9WtC0~4oe)c=m8{nB5oU~1hgPM?w~^rzB?6+&#LmEY>a1_`hQ6(P zwrt%3N)3B=Btb#J(jotp$|{8HVihsMyq1?)oPCEoCZg@z9JUtk0^9E>q+iSAL^3LDs2iN>42JEx4&6%B?ck zeJq$i$CWX-e|k5mxq0O{U6}><*UVzi2hutR(E2+R46&uErIHC1w5jxRlP%z2&>Bip z3mqItsY+T8aPfte>L7@$Dx@F<O2`S*D zl2v1;ZoO^22R7cn9slXxy?wA^(Wt>mA*0+WVLSIWR^l!tw*6h(aq|aw z1s)Cu`%1rSiqU7DX=}4~%od`4Zrdp z@ZV>rE@EYb_gr!JGJ_omMO`T42y%UYixU*lvtyvTA&IuPTD#)h-OazI6ZGEmW+WgHwZn+!>2S>xV` zbzpRXi!029QFUF9EURW}oXCXRARt{;v`pxIaLteuV{4jL)gH#mB;8F$z-{QqA#(>T znEL}KEDJ|pPic!9mI*aU`b;+B(cdE6<0y>BW?;80P*=2O=jP^!+|H6`vCLvSs!ArO z=$fMYmXw6L853&nh8zcQRdI4-7 z6WnJv#K_9(DJ2uA;k$2s1BPKykC$gYjs>iX0Vea5WA38NY--XHwH8_`TY~1oT-^Tw zER)DEBUp-03H(JEfW3f8w`Q=Xd134TYqmrNU6YQ&C>l#dMOz^OjubrGKBDcY_@^kO zw31p{%gdwDgoG}IMfa0ola9nBJZ@bdlAcxc_}#1zs|op`)S0xa2e~V^;D(pGzt~jd z4Apa?Aq2sBkf0z*A6yCSdQ8qbyjw6Ry1Mu1)FXBAPD zba>(%T?ySz^ViyzDmUY>du#`tT%=O+qOWTynf0Z)v*9I>S^yJpHKu7C12(;_0)NHK z(fEyLA6H%?WT+Nokx>#$qsx6XWPxRM6$&#MgOR#b=`t02FmnOzvI?cq<4nB$hl?ha zkCR?dlQcGi)Dq>Ps!Ae2`49y%aob!G2W+0qu4D7DgNcoMgWO7k?Xi6^A96fsgXtG=Cte*rI6Id<>$Pk!UCYk%+cgfnd z1I$Y>f_;NbjBD5ae|bsP|IgX$|2%UQ-ix&?VB0m``_xlA7&poHLDoh1n+J#ix5_S2 z?PAtw74Lv+Bj5`6n0dD{`>yR=N<~qYJ2q`oq+;GAMSgU?U$OiTjh0Sied)+V_;s9u z=cZE_;uVHqi=f3^B-Ux8rc=|38XrLrcGf;uN;o0*k6c<wS=N|FwpAl zw9M;vp7;eZC^)qyH>hhyS23+>iBYlFFRL}zx$Y6DSXW)7T35x!zI344FiJlB)=^A-uYdsRLRoKLy1S>sWzW|2 zQ&(;snmTvu1etpG*0HIJe}4Yf@yL>?e^f&D$t%1- z8-Nd-b|V6BK}dV=-v2>*hVf+K*9oZC0XlKcCc7!@8U~+u)5C$FU_R6vW0t$Pn`*Az zVfbrgkeMTNrgC0JcgqH-YtVW6C^}d=3U(n@#MP*gW=w0_K*mZpy7|^R+gi&iS$!$C z^(KY^?rLdldv>4nn8eR*Yt*9Mo z8@9j}u#~`ZeXOdwJ?I^PyCaWl99lk-Ewq!JwfHLPccO&K|^t6lw|xJ?-4SS=LK$rHEgQgA(m7^(bZ%XMs~m>c zACh4mTR3}sEe_)4E`h+Z2`hnv0pC;p4C}Rkylfq1*<7WR3W7+$v^@3 zDoU1aq#ZOG=ix|W%sf%ZfdQyNuOk_K(_4eCcP##%4DNSqa*qS z*uT_7siNPx0zBqVJ1kn+|o648KUwwHx4>#gZF*(7AKjadji zt?1d<1v^D%PJC!1E;5-#=46D`d-In+2~|@E9J}Y!%c330(AX$apPilKd%%_ zw!T*i7qpD9qDQ3bpWsvVK3yMD*=xxAnG2BRbu!rmFOIh(g{@)lX*$`-iF&1jqAHNn zf$eAe+Sit&A*E!H8f7(#BtDzN5+$G}6g_}0zWTsgM71PhAQvx)&k|1#nvw`q5=McE zsB=g1)s7LN`Z2x2LV9V~v8e@6IyyebJfDNQMv2LG`W$Ln(Q?hftz1DWiJu?v9V6=YQ%_<1l{g-vs_(WD?$Et^ zA0zfSlVxU)I}>MZL`hmF%mOcT6E&@CSv94%4zT+{t;_pRh4&EnI|j88<$MPDXJa23 zVdKpC>_t`>|Ln!+4@IBtN^@QZR%+pJbZXHDBC8}5fd?W2s?}f4eE65Q4*liKxxbuw z`7dW~{^iX3e>wAxckser&KyGg1n>MjT!N!xcrtT4-13WMFVMf7x%!tgzr&wz{RBn< zL%8C7^9$bj88~hT`yb!iy`7erg_~L_x~kmFt%3_?j>S!C6e<4NDn?cE!99oWdh%#}!m z$O?P+BF8me>E7%X5L$ENm|Y^&A$#_0LFzv63;uTeXA9lqA#Eb`6VgWzEs4y%xK0~y zw9x?>230p!U5Kk-S%ux_KM}rhMi%`J-VaWkFS_37@7F@M_iG;gXs)BgJQ%?et(w9x zX#rEX@UT@&V^UO^85_i|C00gGNu^`2%{sl(9Jaz(cTyI*Dp)jNd5LO-uFb>#TB>;T zgxn%MU|xA33w5dP8R*{J*LoJx2j!;Cj)HVxifKP6_POI^XnRjrBDp>C z%si@gr0PtZE`N4DvK)2| zurmGZm%gY|)bFDiH0qMo_cHQ!>PA#9hy1| zQki;p>dLL-x1POqsC(+Sw+>IeGxhe=M|<}Y@8A-=`u@~;xcL~|dg~1}-Mwqlgygj_Yx_e~Nt;gLa-Xs%e zN4UczF%j{#u-3Jh2cb4`w%|&n;2^eJtxD$c6#vr`8_15eJzKVKAOk&v4=f0nP>ddKuipj9SM~4=RtT4leHrP)J z2<{+Ap5cB>;k1ef-wq>&sE#fB05WEu8lv)YQT-)aDZoEEM~iKWq9B{Oq-D}iZKY~O z_YDTANk^!^4bq>!37@~wH6@$wl;UF%0yDYi4S<&#n+s}*8dFax=~M@GF_7`zAz~DT zv?R5y>qj+}VM#)oW#5!ih}m$k5E_@NjXPcUkcjo>Ut zXB{4(u0y2{XSopw3>^p5GfE=R?6@xi(*K>8KK}aPo41c#`|8@WU*7ono!2hkdF8^& zE7kw%(-*(~@zHc5k+^&5+Sjk)Xrk*7#`7-CbIVu#8yQDO!}jf5pcZ@;s}K|QM=LT- zF;qyhJE4Np)v03hGEGd5IVCBor7_b4Eh3rW$`LLxp3%l+MiTpjgVYJpvh_V11`qu0 z_)qul?URgR9{>)!ZbYwr`e}3#X2eH&DphdlICyj(9osHeb-*gM4;Uu3?e8Ao6)cNF zgs$&C^9-5))&VdwnXnuI2J2Yh)jEH9PlZXa&!2gwt^9ORYCF8Xkzj#Gy`2Px%!I)# zbUo`6g*6WIh8p?7>E{(%;6~Z^)c7Bp2+vJgNd}09LO(Y>UM7NWWXfsG&?M`!&WFvNkLGnt=p@bP!a*I zw6X{(pl>gqAXsbq?T1Idee2ckzrTFyn=9wY-(NcV?denG-pkj&x$++Q`^oq2oqY9g z2mfR5-g~bfUH9!<*B@)VV{7kD^6l^5`{tvoGxzP?yYAkp>yLTwfA#%uK0dgj#C-4Q zse8|U{P$nJ6j`G8UOsy7cPH;1eh;R!A<`~&VH;CjT&5~4y4W$MJycq-zu(Y>vz=^ zikj9HRf`T~y9r75Jhau>5=9>pemX1^KFAOu>zaMH&@z->{UP$n!ruAhu5HU+Bu!AZ zlI^2pBR8Y(jNA9waO(O=l+|2j;YF0oEGhwAN#~kXS1AHh;h*05)7d}$@=w41)4SwP zuffS{aPm4fXa4EkKfV2@Q-AvTpI%z-;SBilXW`qz&xUXs*Sd#Fj10LFJ8XAwYad+? z*MAA)eECmrkpBJsk%hN$e>Iu#86A-wY>-+;$poY(BUIqjUawu4JT!T@c60It_<5yv zt#)=7vQgLYT(!t5+qlZ9@5-Y$Po21th8vuu1eRv*KZZTuVLZoEaHAnae!#@A$@sR~FK9Y?E zl?=V2G*!`a%~9RZE6PLEy*BwD-g9>lp1M!RpVuNwu=XN+$idnN9?miTjSpu3q%V<` z@=?8~LL2>BQqlF;Wr`cj0+nRfJ~~ceGw3Gy*7bosC~@8gB4RU#Jz{Kp;u0Cg$c+t- ze!0xbq{L+bQ6ij)(|L2^G}t0^oNV0P1;(%w2lHj8Ju-3H>G8R0Nnd$sGGuva5}Q1O zY~!oYoA$84YzN6VO^{lJTJR|H*kDvkuir^xTW2nSNELdQeCjFKV&3SU_{9NG*@s~_ z@vzPJ?j^NParb~t02V&UB(+Q~k}?75I(70xGyq{n#w7etfD1VC8-~n zcQ%6#Gt@o*9@uFvP99nbeq7Tg{VUj-kJK&_+%jQqUWC)| z;d=DQWQ99JlJEC<7Lh4`l1Mw7$TUAEp=mI{J{_P)Qo?NB4bMGyZ9|xXh?)L#IL2yl zQYro0vX@>zw++`TH1ZXJ_%3$$*j45@l(=3oyz9hZZ)QJoMIn>+t?iUAjF}1@#!Mo( zZ?g^5iIOcLO7`~Ad%rr{)+ib5c6w?_%JdQRIfS%m3_aR9ZzGvh5`oxd$Hb{1jD=)l z2xQg1GUX)p|0OfAok0w*ya;Oe3aDuKS+{Pk&X_w_JX_|764aU}%Mzvx zmB&13%?jY*8Xqs#PIYN&WXa@IMbkjJ=$fJ}E%24hD7xAlLJ1u(-XR=rux`?WJX2K% zSCAt}f!kG+juC1RwjZ89Bo^ZpOr8r1yVBw4AqD?ime~G>v41t4Q?!(l%`H9juO>8K z8J%|$QkGdImk1PtM*s-y>+ix{5dIcggJWfz6YyOp&P<$H_GaPyb+w&KO18R|MQzg+ zF-*Yiu1%4#*&j3Ug-pUqB$WWqyLok0&9E)C)KQjusK066sWiAnwn{O|jdJuBvH@l@ z(`-z7sxj%Q@RP18IlVat7P`V0mcbsD$0}ZX<;Cx}Bk1HXB@?fVI}_JsnXKFBPP~z) zrfF9>OrDHh)3JjaVssvM>B3kU>LOqnNhC?l>dpRZX)L5nzBu~D=`XJS*WoWd{NgP6 zzt_Kb>x;`@T>auh0$0z&wbTD~@Qd>+Oj7^%;ukl*`1Fgn;HzK#;`d*ifnnhB@5z51 zf>C|=#d)$WnMkD)Y8U*{6N%`8+f9qK%dc0u)a-pSn%PA&nkjt&62~YcbOS<|r1w4+ z+R)~Jg*H^8h?`ni7ABfUq=|@27TF_UOh|%ex_Po2KXD9~7U?U|N)^LqC0>mzrWDu> zNl+8HoT6s3K07;VGdc(=qomW#8=nXcopz>AOl z(?&@xoBgoty*TlEtF4Q#{#b;mOXR+g(cGo(Ct-YK1mwHLDhQ**V<47lJqg5TRukQ+ zel^vd-Iz#pC;PK~-MPM$+MUg8?9TKgH~LJxhJ6=WxY-4S+v&)~GwjGkP8={j4A&+G z<+e>*1vL<>JWhuN-6@Z2UEuG{d%wEy?ehoky?Usvu|Mp>Tw1-sqq5)vWvO{?SuJBV zj<@DCB@<|cj$)>4u$U~45l{@&92*Dc9(-VKjWFA(c>L~;M3u4p`(qcS4DaW56~Wph zvxVs;`-tTQaq^1*=3WB^p z)GW{RX4sCK_xER*Vsr(xSq#k!MB!QYh6>CQbxlgDFn>&n@i-Jed$?I*W(7DV&$#e$W`(580(s<=?9xi89*L|kHV9NpJO)xgtH|S0*nOMK zbhvmI(kjRmE^T4))^&%)*u7Fnm#hb>>BDytF%wQagqk_K^*hTbUl=0Zv}n4pms&{W zr`VDgN>oO-xha*x%K2eOQ(50i@=ihuDzyNmSO*M+3^225^5_-2&3Guj^i^n)=t?e7 zWW5TvH-vhSgn~yNH>j{wJ(E=u0T9Ygz||LU71Ahio0O^R*yY$_YK6j46N;uaJ2FvH zzS|grCog|M6C}58gb?f00(Ol}t8Q=~( z+^T~nti0^|V0lb}`?$sS`ki-p!=NUP5aUacW%KW@#-q7I&?zeNB%wP#IV2Ta|!7U+d>a*Tc{aDji0ONpc5_a zQCSq7sE^7#c+kU4!dF{<2t_Hc={e8}6#CiO%t(R5(;_U`#~I1Mh%DiF$ExO(mi=;{ zzkBy%l_4YBg#hPs9mT%#=!H@nOuH&XdCZ?3UN||tm{>GbNdy4LRttMdx;5-6iKMUY zIb_)r%)F-P+1R~twjt0w6QCYM6*fuca4b+YyaP%mcK^H1Zyq zka>?3_9tsh{=w&P!6WBzgJabhX@}z zVQ^44Z3@;VHAM~d7p!7QRJny=G#f_Jw#AWDasgQPC^6XqD$AhSIg>6(rotUk7FH4F z$#R(WII*}v0@eCJR1DuJM8>zN4{PEyxDI4cP#L4ZYlF$kB_B4L zA44}}zW_)ttz?wB5@a^(M2=puwl@y+d1rkY=j%WWz&$48eTqWFt=l;sCpSSN{9zN(?Hoxz)lrb z!5bPWQQHeii{W98$Ydr0Nmw8zgB8daYaN%*!x3e#q~qKqv>s!JnM{*G7iL0|o#t4j z%8lsu!7e&hq0-3H!ni+Xt=*@)S6igJH-&wLS|B$ZI(llD$W`Q4A1C#^qb0iXlF>5m zhu(Z31rw^NogT#4J%|_YojFL>)rWrX^{dhIOs(rgOkpQtIzFf1qh8Tr zVMAMM%$FG50wA5Fk_aG>bn2gH>Yulh?^%WDJLC2-f`5YWx_MBbBg}97r+q#3-bqMN z=}IC%bRAk$#`4E7Zt{lrbJL`9tgf6R%V5j6Gb(&WhpPEBlNMSv>1ibwpJb_1;S_%4 zgTi%6WUG*I=vL>|Ii(wQhw0FThk8y)BsvL_dJqbXUxN~`+>1OGT4t+S<#lYK3Ic+M zX87DPIB@&mp*xq}xpVYEA@a9M4^5vud-tVxzB+j9ZwG(+<@H~Eb@pOICc)RbRZ!eo z0+V_vC7q2dpiNZLa)rBELp++QHfmbxCm}_YQWELraw4}J0WtG*?bF&-B=e|!GD|k_ zDg68hE*_tJrS|d4i;C3Vsa>dDtX-RYfvkh0Ym?8z8H_8sDL%lYfN34`Gmu2CtJbks zCmV?3cC7GOE4)mJOLkW2NOT_KR@ve@QVFV^JjkpH0^^w))HPd$1{_cQD=rdfQkx7+oRw@m^5j3^O35j;HqTQne$5hg@>OjCr0mda>v}4gBvgWb zZ8G~Yvd(J(-Og=BCf+3bgeysbAs;(Zo-l0ZnY)?DiL-`hmah9(SWGbU$cplgWZpa0 zPW-sw8eUM4tlK5r3bie%s;4M1N; z3>OVkE?s;1hwCidzG^YLG^j~OsN=UBaz4u;WIdSfG5Q6- zA|jGFTm^sal1J!w6bb!S-g2qEj?7t;N08sMcH?n$v#-E-F4e9er{5cJf_#NHYahVy zPDIb_GTY~os%p~b@pva8C6$D&u~QA}^LbFrg$pR35EVcNo}KfR_Pja zsGK-WdWFesFwWX3x1^E_gasny8P3-}o;(ib&nLBOldnu3uJaia^=}7XO5nehVB*z& zHu(w|b+6Pug>hVlQM}+Y_yVtW*M3!JXojI)M(XC;RWf;mz%f#w9-MrIcoeJf0{s7# z=(4rM|N37(V_i%7taa}sq-1qPPjxg}L!%kAQl(bgRa!>Lsm<|*(3DV?7-eL<_}O7$ zk28xuD1Ww9`PW6E$lb6+5Z)=xaMxWFRPBWTqHt6)8ZB zMgkqVz|y&;kOK5lqg<3Q&nqyrw{wI`PS$OrZqemd^elOwE6p)7)LNX^ZvByS1C#YflT<2Kma6dlt+M968b@>0y+KmJLlNmO181N6h z;>7tk&T^U)tf4@D@Pm_wKx7T~W8o0>!{!D>~Cd_9Z!FYC#_s>^=CdfIT} zY*0gb*SdQO6)#VU%pu0j1Ef zZzdaXiVZso0+k0_MpQzHhDrv+v+T9Zd@*bhWL@PCuh|plj2Jdyslu4Az%?>WrHN^vsU zUz~OK_s>m#^!lAEhse6=^T%Nv3n+^^4q2S+W8{E%nf;I<(M*aLkC<3%=!N_!yH(%4FriQn$^2yK3|_A)rac8OWY zIM)t~IgPwOY9N`qswKiL8W;Iusj|=%Jy75nCDpcBh=LF+&oJ92Ty7(tAldDWAoBEk z-*(UuqUy7a8-&YRNW$eb_V^^yv4g>noC7(-j%f);MAsOt@*>m7=aWlxiWu)_pD_?r zp>-XpG0Fr$L|H`(U}ddlJyN3P@UX*T+s#YVB^vVib>xANoE3)cw3Ll-i^-9;t`bdE zQUT{bbbfx~tWgp!lPiqnrk5iEB{uoN1LcV5oUjAbb*L;f=04S^ph<-+p-CgjO*-8? z_n~1L!Mf^)IkTxN$*wVXoR;~0LXumC0wKvQ4mHOfKu9vOH#p#tThn5ZlL9B-ce^(6 zuAQatM3zV{siY9-1(6}r4Y_nBvkTU&B(sYO2DH1{(e8_AcZ!5V?E~9RV{ea^YBr0b z)D`t|QIxb=;@D?R3?U^51&fRd)LH&5rSpaI1Om41rY0?L7M=UB->ta3;A0hvTnz}~ z$SH|bb2&t4Vu*>ONOJF5MP`u^VcoaluOG`$O>to6ccad)4njEhGPldFZNpoz%#XUm zNRLNg7~>8jyJ5pYG1vbNzPUS#{5vTnmsYf-dSKoe{E2Z_PFy$2MP}m%tSm)0Xp|&_ zeF(9cw{ez_9!8j%Q=jeEF#WAE!$}0@{U7_D5Yq*=1nzpAD7p=^r_djexr5;{@s{-P|$uY?ArR3SI~SFyR9aV zz*oQEb==REJ1_7m){tL=@m=bw_o~6`S8CCdsrUPo842nQov4RaipjK+OgEPrlp=sD z42hA6)A>9n+$+kSIBoEWOXZapP2En5V^j)hk58N~60lRM&cuguWa6wf7F{E4N3cyt zSAF}RPbY;}N?C9iIt(Fm;RtHDl@?-{#xm-YIVF>8)=8l;Oust%t2>w8xqJTLSMQv= zd*l7v2M^ir78)%Q;2>aKe#Xku1TEeQR|GjR8r9cWe8EcU~mfD{e(HBA}qI#Fb&>tapJ5n z!9q2;U2nW82oqkk+q#CL6SjfxI6_SvYg7qs<9Q}cUi7@_Il_)c`p!EJDIy&_>bKCvZ>hAB4XmKhfeS>v?57Un-A_d4q(3*bzFI%msXJ7_Hp97o2X>n57xjNi1&scta4QL-HGf{Q#n1YLH?Dqhq>&1E$*!sNtHD8!f?8OYo>9M#(yA=G{v4W|r?WR<(U4v6>I7J+$GYClq~2 zTy@*sN}k>?AM?E+g6H?Z@RkoC9}}bU1y(%~7&5Jzu*e;5kP7#Lsop-Z#VW+W^h#zF zO;r^wkyO-dt|1&BeSK)rAP-Lml<2-TX;l<(2e27DP@+X8vaxJaU02ljIA4#BgI({C zM7T~1OJatF*``j(G&Un;bFtlpr=RZKGH_sk8(;%)_RKR~UH=>Y_ka{lD72a|!srJB zrk)HUuERZ#hPghH?uB3P&T5L5OeyIc{7Of+3QF)v)hZ)W6q%n}$rndd!v_*|_!gN? ztZH^Cad1}KxQ;U233xcP*c~&gMh975>!sN$*1lMhN-27Pa;GDyW=WM{ywu`Gs|cdw z$tn#7Lvy+-QWxOWBX?eXWBR!hUmZNQ@?4<1uieB(K=0e%Ir-D+pZ?_Q=Z{7f$3~Zw z{IK9rBO)M%tdi22l}%{hryo29SR56Kl)x6X3`QJUw7bdEPk+yP<{9|4r^JoYzj=q> zbCK3_c+5NLtG~QcdcQWiGxzK%F{^&mJ4QxJq5>x?YMpI?IjDMGIe1n5RE~0Mt&Ob7 zb^5<(;)StPIP2Xx3_5h5;Zc1=lfVuT+$e2KewV5* zSW>9!i_X5T=*cCmibtaJ$<4clAuxt-!?c^eYpo@Dp1td@&VHO*AF{x#@(-T#5N-o>stZdj@RVJ%2B-3nH6h~7P zEz=wuh~jVFec|Sp*RG+j_Vd&CN%aiv38Z?4C?8!Wa6n)G_KiCy-vF=k>(@_4mVt^1 z9X+Y2IZg4WbQ<}#Ovrp&2FZ=oY^UdcO|M`^iAhQ-+>oML8Z*uI)#HI!Qc6y1_7Q2= ze8JSaQ|E4-y!Gs@qh#u>TSssG7{oJm@z#l{kI1@P$EME1E61nKKGs4JJ)BijA46oZ zY_?0Z0)LE_q~A1-jHwJ_Nh?|aE^8I8mWC>k!eFvJ5PunLMau*bP9hhN}?cS z+qC*ZWi+9saUYNI(UFj(MG6)dfg7a51uw(^UTU3DoJ=Tcro*{$KPSeuI3>K|0pPb& zN~)3!@JU6fhU($c^a>4;lNO#hhu4W|F)p@?Rm2D@5K1Kc#Aw+HGY7W)|LnbMa}!s( zHvCuC_Wlr(!Hv4rI_{lkmroo52F7;6?8)nzT9THfg<9R3J{SbfTbu%9!X#t@2_z(h zoPZ=G5FjzfRK=O6o)7N_x$4QbnfbJb*pR@~{tNGYw`{8|$xe4$APXzRc6Y1Q(puL# z+{f$YNv0+_O=)DEWtenWBi&;c=q|EFPr>gE7?L?xhZlMxuj$?clDBQo?sO!SwFhfT z4Z%fGlxmbsX)TP^3nNQ%&A^biobNrXF0)P+qE&n;u8q$Fg}l~e6KSo^SxV>04B1L8 zr4%FW-MF@y=X6K9vS%)HBncK`Rk`dd(C(blwH)j}n0e^O4O2g>b1))=LzRGd*?)L< z&*lXTG`+v8GSKw0ojj>63j@sn!zHN~=79q3^9+F$2neCDZJ~)u&fOy{*4{tD;-WZ2 zg;-SyYA^1v9@PXLA!Dg&Bh8Ftrj(3ON8i|EAys%RNd)bVdhYXFX0gQ(k^@4Zit{Y1 zxkTu}l|v*Bd-;QJ4m|keMEsw}PyKTB0{Q2$0}sxeA`g$>{N?HwSZGP%G*5`oP=Hqp1~Dl&nQHjpwyU4qAR~=erHj<qWN*Y5Z8weO}!zxw&LFTESUb;U8sLb4+g-er;; zKpGL`K;;I*pqpEkyLXsmhqK!O$sF9)U5#GabjooBvPS z#5n>@I6iTPteR;GH1fp6DY%4fehYq%)hbEV+cqD_wl3%}ohqao3+eJCN{TbEMrA<^ zpbt<8%4JxkvISOhZGpuMR=F}NrKw9?cPK1tP&K8KbvjRLdVvi$m-GpJIa<6qe#j=Y zyI#D#niNk#%dgw*yGfwON8x1OOjYc!i=#GuoJU{Fua&{BZ9H4lbQTH=p-`psrKyKf zOu{lP$dM{U>sOQe@eHCf*oR~l45aS?X1Ub=9YK@7@Lvno;%0W=)5S9XSRYH+|`)ryh~) zSvLlOu2Oxrf=x1A#(&=Bcu-HZ=Quo6RWazC8756AAn27i)wnvp)!wy5o^z;)QrS0v zhb}P|g-EEy$5D>49m2Q_C52U<>ZHAUkFkwnrAIkZ8O+(#tGXBgxj7brA6Qi6CR^0s zhYq_8&AbIo`4)QdAT05(!}1QI0QTf4EbV)tdq-h;hkNYTu{%(JDDe{8OZ}?iCHomy zuv_qo{ifT+s{}5AR9gJT-X=ziS090B5scKfIKcHq2VzK&S~k7eFjh@TWoWK3$z4&b zO1T8Wmz5K%Wg>r}PSpkmYDJMSAzG!|t-y)XsTU|oSUYWMp2o$iMlqdUY~@D)Ab&kM zOjYk;L-q2CN3nirtVObpQ^VS2QH89^W-Ma<_l-;BkJf)*|48+|`_~a0j4y5N_vphb zn5A4hP~4EG&wnOST~@C!$YHI$-^JdqV%Aqnplvpa_-&rY5r_Q~bR3-I?(lV?3+(g)hAU}U(e(j5-^>vnO03Yg+zB`~9yksqm| zK4KnflWtU~VelcIrKZ8P`f`##7H~RL*jPCCo~zZ%6t2)@8H2J(-nBtq3e|}JSh)*T ziB1=EeP0o!^;xc>f*kWrQL&Qtc>pS@)r+^n7gktsvMTVm6%n_Dh`5Wl{&~VX8EO3@ z;-Z*@9FF=f$zbJEwo?)ob!)p_X<>PXyVb7A(4nWHIj{`cwKWqIo~PT+#eSx9MmoF; z4LoZy)0{`uS33YsTqFVxzyWC?T&V=PCYJBmu309QGaoyj96Q&9A=zR$7$i(3R5uNh z2f@eWf3$pPrO(k^>AAChUa1^vWFNa+{!n;m8Kf&LJHt#@kx;Z=#vNbY-e5!sRmJeE zL}*HKotD>z8|7mQ#)MGI7J#!PMKiQcRSbjWY?7eM5_wU~T8%|pktuB*h13t1)FfR^ zI&#z~_?4iSI4@+W2Jb{m(WzE+v}D%sy?E6_&5?sbw5mL-YmxXlYZ&bt8tH>SAYYmp zMl298iK!7Iuh@s0N|HQHb&wI+qpJStDo)QuwgJW|m65`;c&C=&Dkc3pFr#!02EHe> zot^=k_Y}ucCprWG(X|91hlF5sN$es|$W(%D$JB<6Y=NxGScU(-aR|oRf8RJuNw21- zKrEp3a*mc^@_-WF9a_^l3QwLw3 z`tsVnvp4P^y8Pmc_s)Jg{q{H0CtjOAb(pLw-SqU};Q!n_GX2_L?!9st4&Yh$PTrV0 zb^WL7*YCY@e(LbSsaL+n>#u)0b@-c~Zk>YbQ*WHO|J|XVuD^Ewt55IUI0&!)bn~72 zd-mTy`Nq`Y4}L!RG2Z{m+3Ec+Prv^9{qHVLef-(f#W(O%PQ82o%qLSXzkGMkz9!ST zoL%R6p#0X@83_f$v8ZF1ghGxo^7e`Ap%Rl?MBEGUGcby^rOn0$*~%d6imb`T z&yeD2jN=?XJifmaQ+u^|#SZ4*XGhnLj_(^kLW)O=C+wh5`&aQ>c%c+WdX;?tdGQlF zspK|>f#Ul}MMA*$H?Z)w9UF?#r3dW4GvTMp-EUHSA47r1_ZPpxfY(x!+Z-Xa)vv~X ztvfuAm8}d3+dz(y0)cROw1mt0NZ7SL5({3;ORi{%hK65;Bq0!NafmLm z{w8=*(^acp{5ZZ=;80KqS24W1)>V4cl%^Z?OAvcJB8pw;p4z0Rs9beHM;RuE$9Tq} ze+Nf{*+A)!!$2H%g>E4bClIL&qbWPqM<=~o`kQP9^)izxq}ResIv)&_IX&Qz@N0V=7smvh{=O&5 zijoA7rSk zCg6)vl7iE2s<0gCNLzr~R+@!BJE+N<&$Yaf8J2?DGqjNFEa8LmtB(~_>hLOyWlm(# zB`ITT;f+bS?^dtT-E@Qv_QQX2MO<0+SWunpBnwr$$5xotj_ZJW57)D+doz{G4TlP-f8 zCx|d##E7FTmS^E3hV`(6u~w`APB}_;P(|U%OuhVVK6Ka^CVEN%-x*UYN3yUI)Y5sf z!IEGsE(V0)(lXkg3{GAx#gpEXJZkdr`olpKqE&@_7v7wpGsq@QO;wZQ&G1(cn$I(; zV{!Qff(6^2?_X%I={IS~m;LCsnouyfD7k-zkFmAK#%~Py%TLks|H>m1_pE* zwW#pA1}e9qE)DuLX~Kka(BMU2{|N}9h%_9IZOE7GC!+1H6`zAHYi@8#C@gG^v!%wc zFm_P%&DI3-Xf~dh<+%Z0-L@6=l#WRx+GHBiW{vb)$qX6D;25I@p=2#LGM@3-(BJKt zCxL)trr0O0v57`oS=CO{DpO!pGaBs#j$!d~spTkI&`~sY=7&FP!*vqZyb}_{Fq1kI zX?Z0|`c9TZ4^X|K`i_LvWZd%_|0f-u;c#^F8J|Jw;)!z;Cnkk*_ei%GRzVN>4|aHRW?d=xIy?4FVKwQ!63jzH<*@73{Tg3A%vyq3yRFRg2wkLBY_Mg*kLq zRY>u?qU9)PnIwm`F|l|nSIGu7<%8-g2$Gm63Q`~_h*5`jT3)~0FJECVx=`Cjgrou- zSYeY|TBrF;VL4k9qF51b5mz~d*^;W|9%|dxQMtKnqsSDIHtpMIEd~IHC<(!^5Lt@T zJcjkqLvl+6nzy6$S~@kAA1*a${fY@vxN<{j6{uKF1uEKWmwQ~K+GEVF%iUtvo0Ai` z!qk!{wn1V-P!eKQwjX^V+KILSs;4coj;d+P4h*cEX}1yC^i2d6y!hVL_BapbAA{ z;|e*Xh(LKR*~c!?lZBA9B&qw!@D}AWG;J9qN#P;f53_E)3OE8JVIf>)5prJwo#*pI zETb^}KOPSR^^&=}GP;&SiBv5W#qies&zp?&=n2Qx;<;W;3`;#ns%)jMH6HjhJL1{a z6kvnNGo5QGLQ6@eOiiIiVZgmiO*UIlcU?nL9T{hC=Y%W@fq-uwswZdDM+XD$f3WW- zxWAejbu*O8Xt41;Ti}(spsMv$Hx*VOLo~;gLUS!!@HnQMq?aohnwxEjXH@!mD$kQy zg*h;BP#D_BE>XcU6fHfpJX3Nssh8+*8ai1(@sU~!SUH+NO;Hrh8@qzVgsA^r(_a`%2GHa*7 zc>|zF8RiBV5FGMXz)>zO;F6s+hAF4fs+vvpESClnfx?r-7&vqkvk%A00*Za?au|Vt5@_XA*aFHBQ*blPTRQyj!x&V*s9`MC8mpY# zu2Yg=D}>2x7V_G^kSpX)h-QIfv-G-e6o?Yn0RSHJk;0Ly_WW3srjo^>ami5?oM zc@@;A??3PQ{>E}Q<_^rOZjER08H+CF(vwwrP2nborO!vzx{icqnK^BkJB7fqm)4z+ z7BCe9f*ffTcBofjhgUy%c@MX{Kgn%0y?#kwLKvLIG5 z)GsJnwDrT{n9OEi<1GcEIO112EMphr$&6~LX|_Q9r?&2%whB1f)-gR-DqGt4@Yi0t zlRR%`w4tGDG}<61OQmpU5Q+r_DQFV}L?cUZ&?8Lf z+-l95nIKDr>Z{3mTF_}iQ>+F}oJgx=SA{jtb~q|pqG6sbgayPvx#e!0-ClH9kTsQA zdP9OF0S`$O0^VueqA$QTpx%PQCqbh7CIxBRt!N_9V zpo0oLRvs8JcfsU1GJ-t{2;kr$+-#9id zgeojVs;aQ9z&SJ<&zzu|$(N=?)diM_5DEK^(FCz$twM3iz+RUq*xu#-g3hT*w0vGO zxE*u(%o<0@?=4)T;Y@ugpH}hl`GzT-8fhkLz zrMKGU+BK);g=w^4_Jes+hSC=w8) zs36H9AuN@FDc7=cSIP=u0nuNooFuLbpS}2E@h`>G@#6c%4~y>@cum&2Il3PunYcImt zXz`}!8M_Tu(Dn`Lkf!HgB@}=MRqiUSfW0LI?0sw96UqR_FkO=hKdjt+BBh__NBX;Fo2QZyZzW;~;?p6Oaj zGxH|mXgPK#Kf&lB(Bxt*zy7Qf`G-r{)ufm1N^`vt9J>feq(&H$>kOJqZZ)yg6xl(s z44BH38IxdM09RL&gpuDN)Q@rM@l2ZET0G?$h@KQ0Hn-7?!wuCQiqzIUBa5OD6(TX; z1iQa+_!fhDsy)Y{8&s78G}YcWwB&APSZ>8GOE5V2Z^YC4OCi1;u(b0ero_+A%WYfZ z>$^tQYbHFZj+63H!M}*-@>jc{%hF+6pfzk;zczDu&bgxn%NZc{v74DI#sh(+5SFh9 z@q;B1xB|;j)36%n#uPEJG+5mAVpVN}06PBK2Ez|ISKHpVo8~#r#Nlh4|Z4T|sE@UEz(Xb!| z9QNU`gFo9nM6wX9vJmgexYwKS?>{nq@~e7QR(&ky*q=W=IX(K-^xL2ObnDpk$$itO zPJ5mwzyE#VcfWINeYJPKMK(SWlSV^9-=R53Ghrtg3M8rNdj0%~j|Bm&f7YL&DQ$;= zfiklk?A`@+s_O;PMb(1lta2NRIEv8NCoU&MRESpP7_LN79sPBh#Or znm&B=-stV-t7AX?-fL4YzdH5m$J2-3C>?l4r6ZqL7#Vh&uf4`Xi&$h%3RL=27c7NQ z1C@z06GtchVpqRd{vz*8u{HQzMxuNzjuO~_g*?4vGgr6MHE{jp#JkYA)1-T6w`YXb z?#u@R0VhDU1G~gU0=q=a*o=w31G`p`fcBvFV2EA6Q6sjtgcsAgKs z#F!W&zqOdTiz>;n%hdEvj<-Q0GY$sJL5WfhNN^dCAYuk*FwlxenA!VHmS?L;kHY?X zI@=)Mf{5kfg275inmXtX3xmZ1LO3Xh(sJMPDx&V*fd*VHehm$|4oBC>_&yB3E8c*A zSBux6!TZPedLAaf8h}@O-&=Islfr^nW!G-%QPWHHWM)0p8Jb_3Y}lNJj)NYX88%<9 zCeX7uVlI>7F=+UxJ&f#;1;>Y$#{h~7vVX!eB<#^mo>Z7ISGT1JvL>{!bCz#C0~R;f>XLR%Modm4hNe&*V*&IWRTc{M++5{PoGo|Tcz5FZ zLQ2<=R*fY!Rm*wznut+Jz%&z47NQOxq9YXx4_(fxkPxU!f%G=3y4Lq}ZQQnD;|7p5 zn67I&!Q~J3NAj;?L?C1~I)S9Dyuy=q!6bonic^lCI3yho%pSVL<19wBR!xx^fu|_fW_}Kxz@u*h^QAs;Dd*79D0b z6?7w4uaFRFk>5Ej7?@PoNI)V&%s=Y|>cN?@>nKlO0LA{9HBSygooHr&SwnMF=W42v z;W%LD&uKl0U1QX zR6HV4DISf)B{?F-hr%=wPYy}J_)sDh9&vNi;NWTkU#FDl)7LT5$B@Zs>?4gZLSakL zjKRSsi@)yD^t{1H9xi%!l=myNU}Du`r_U-}JDnJ=FFG=0L9TL_w@gLVrXn+=Q_C+8Jo}ZLcc)zG|OZ}mO`EZfp3#2IK+&y zja78X?IdE-q*xUv=fUc3ZxP-21%7MM6pW1#=gHt88K6d%Jm1kl6mHmFlXf!muHNL9 zoeC?I;sRzD-A+YIX4lNTG-96(4lZ{S`Zh;r&6bp#^Cw_i?VD{!1Wo0*3O%7y6VFT& zIug)bQ(~g>hQG6?9nKfWl$71hNeMF|1F~-h$|qkZP@L#lGjOtc!h?VJs6z=sI`>UDX2_ zT~lGEu7VD@jLlk200Y?v`Hd76BC-$-3u44cQdzqu@B}{(}2U;w?E+iX9b(&|Y z;lut|C(%a2OxSQ${M1TR+e@ov4j$0Js>BlJ#>z3@Zb@7*ciQF46_8)lEQ*`0NQ3%} zkX{b6S7Ebs8hxb~ml|o{Al7lNu)*M>gCr_|dfA$^vj*xURSHSXXdEHBgz370DigKq zdAn|ALaCa`;^cW7oVj^7BZm(&-Q@Nwa0-(xr`bH)@V^~^r^C`^<{qW@*Q&=z~uQzz7QB(#U+Dj9TPIG6OrXX5{%USQ@d^9@B=o3O6^dGP#7cTLY0w zr>du^@+PQpw@Yfy_yHVfXiF>J9^V6J^R74BEP5N%QB1SA0za5raU1lxcV-eXKxZ9) z-A*nkW%3lS+uG|jALYmuI_!v7@SaC3VRpCVKiQ%KKaK3(?WicCwlMR8bno>w3^}mKO=n36|&V`oxV7u7H zL4lz;(gEUv!`2!s>motdx`-GPBCR6jc9x~~!|9Z@nha=J+K~K8DJ}(sXmCkEDNn}k zW@(0RaQ#oOX%~j z$MJM#&fC`7^;$B^yr*mQft0QqJGknQ4Mjb)b`VNZIgAA|TmiheMUY8K&-j9=4DS(5(WoE=0|MyQ zLP&H>=JHH-japcUR7EtYihI<$8B0msVgvK)ehDcZqEWn`c+ z>=LndW$a@$i+PJ6*%GRzbV`i8KK4m=rm7sPRV{0I?V1%c&i?j8G{=?go`G3U)wH2H z^*A|JQ<0Nni((QJ!U5lLfmkR|(@eBhHhdQO$RcGr#x!W8Lm(Z7sp)C&YSzXK$ZUl) z*Of97k+R)5;@WNui(;&*)KT8Ua<(OvkN2@SS*Pi{vSXKbpixBCcEBWI+2nS*?6K)C z7+K5MPRUG*3x=8HWw2_#9N(-Bd&g``PIq#pO3^Z2_OVOAf`TMfN)4`8GXQKotYu|! z8J4p|f|>@Q!z5v))9_-70d^{djh3V+Tu|wMG{YoYG^l9qXb9Fm5RpBDw?S1k+p&>= z5nz{tb|4uk?@RWvOC)6>Tt$np63rX4UbGAwrb1gT$~LNF@Gp;jSUO!r7;>~-A2px#si({3fKM;nxUyCat)n-&&jvswQmf*qIE zM8VR<_n?K}LTfG-Z`s}|WM#8y*uI{1R~4@mFM7A|*PTU6!hG#;l_|t^VIWkhIU_GH z-O^3Q6$=CQ)D%cMLuNBbE}cxZ%w5hzqS-6)UaDucU2C3CX{0}y*&%q3JFp+%kX3?Z zc+yQf8>Hz`iFhW7WVMTVV zsIGwsgFMl4ZI^XFh2^lAaYLoos1UA_URNL{vwK`ZQ*1*=BQXgDgh17TbI|pKmS}sM zOC?-yQvZ9C?**t?ZfFQ^nP%y~!bmsA#2B-fHkNQQ~u z;=pl~pUNXh3iG=fFLfaf33AYP&=Cs$DNvVe|2?ttDbzL|lV0u{h)2ckf#FxMtI~N{ z$iaZ@h-$V^TmlmoqQO>Wu(Bix>ZVRp8ueYOtuAs)v<3lTQ(HnYX4-u&^CAFup4`r|hQu6_mA&ivP&AHQ1m2GRK_{@+VK-um&oA3uSwKKkS5KYk3& zz~j%!f9-`g2WK~d-f&sA|{>XuV_tbiX2lb3a|b9tU8$ItJoX7v7#eesSvCukU~J^8N1)1p|QqT)O|KefQouFm>_upKhI+ zdUNlHtGk*E*r{9R@1MLzR&{iAOuhBx^t)esm&l*vxm;m>u3WoVn+!@^gbgZ)N%>Ns z(y7w|)jOy^U;kw4PaoXB`t8q${ycsD^wgfeOr1IM)Ad)UkH1_WbmavUu0_$61v%0R zOKIyUFx6g3VM!g*6uxx%-bYYo5IGiP{|p!_h1Q4I-C`Cev^l1_dI<20+AkGJVzd$ zIQ;Oy2c9wNr)qjTS2H^tSyG-uM1shEAXX)MEa-7*>u1z7XB?6al(sRJWol}HG^w`d z``apzYI|Pg7z);JQqzv`e)J(bQ8Bg{W^1*bq(86era}5JX}21Xwrw44^8snwMvV-t z8>XFcGgp0279yAkiWQZE4#moA?rzX#LkLuYGvuMb(BBT;IS9Jpor8ZnKl$06L!kOj zeogKi{`--?o&Wohf9yE|n&Z{UZ}04#Jc~9!c=akM#Ahcjz&r3R&?pZr_nGHx&F#sr z;7gx%sL9nY#{c%!E|D2K z!BsfKS0^u_6~Vg=AMna)pbri$ZL&(D5RNU$3-=_jW31&#mclF%Q0(%W!F{ON^&)%% zNPg@j0}N{bVeVW#?Lj$A4lVIk(8?6s{wLo+(f zwllJo>PDrc(GR#R$Rg9Wp_+tZgT72cf@k?~@C#vNR1U^i5)Nw2k+wjjQqtIh`Jm7z z11wW&&}FJ(6< z3Q}u-nS~^;YZkn!8VzKx4q|eAAmlq-*vO-juGqoA8#P^LS<}*)y4>aLiWpC-=g20S zrqny%Rev(0nYb|}nChUXZ5L}VS>mEg7GM#J_%2#g$NZ!Ym17#tNK&DO@$3lCp)N_) zBzv|hDXAlu&+r5;{mAq}`dHrN3Ul?=nJlRCt%4vt85vEz%+PjeJV}g#O7jL!J(7$j zDOfJYV;{Sm@gPuEk>L$_ScM-EM2~8EZ^Z$qur#tRSX_r7{}-;uj}KfSy6z}>x{!S7m< zj}SGtxt zv1?OzrIg=Al@!xc)V8qB+7kl+R0zbtAb=HfRBH50D;KmYmRIeTD=~7g3-%pVOH#cQ zZQO{cLAyk`@62i&3qT>MY1xn*l*DGBAgGb+FrZ}%8ZwMUVzbE(vUkzG{1At&qHL;g z4T&s7Tcyh99KBpkQbpmu3ob8F&62}HP_9&cT~CnTaAn!ghxxt!9HVM`hn2PrleKqi z$lmuyi;)d#+7AAQ99efkZ#vv2a!_V+A=^hagG{Wfk;GgpByvm$ zSFw$*#Fc6`x=+`r*}%O1Am$(ks;t~A5s}i~u^r1uGHdC1q`tO}P*4bjTfwg-2M|v& zdxEGO{SFP<%tNvetD;zNy_s(U9ixQDI;QDqM%L3@9<;1F!tr?a_%5#N87+yDyGcnx zr%97(3iVTsae*2r4l61ix$P7M>AM+nd1jnB>= z{UJ&7HeUc~-DD=vrP&4G!WfwqcYT_DdU71Ey~2 zx~8Wz`{E{+=P4}@^8y(gMu^Q&-3(ULiV`7to`VhnE!-d`OXvLYguxifFa_UCa5$Am zQ6x|fPqB|(j-!YW4*M>DHItAGqo_2JNbx;8ZT8~$>jcXk+xf`IGI+3fF;2e!{NGg# zR20QZvH!Juck%t=6}yhq0Xxxnw0L9u$oRqWS4ZF>qs8mv`>}Le@fHler=EJMc*ai9 zyjvl zw``m5`R23v?z?<<-}SruPv70Y|L(r8?(RQ+cmD<3{(IOnRAF=~2Lw^V!j2A;NuZ|T zcM-s3%taQfDuT9xvGHtvFm_&NC=xeLMUgR2S@Mt76Bj;c(0o3_eI$d`jvg#eo}l~L zBMVa1a@U$YwPp`xil(L;n9Lw!60^9pEY~M%{;(W3HT{p4AIY;`5x9T~x`-?1P>a?1 z5wJS9*G`RGBr{UBGsB~mb7shG;=c|}r`#l1FI3@7HAr5wbTXu~l+KeGlB3Bii4y}Z zCK=#n8_exnlKR+rro*LK4n1PPRmI{2biL;CRrW$7$3_d@=h!Bi;EHCMjv%YD>Bgny zM#h@&p(@*e>lCgBR}K+OYkly|fd`+Qi2w8Wsb8*MApbme;K7+w)2Oqw7VAX?9Za!Up$HvYr7esKAXhwmOGt4ghZ_}-}Z#AVDR`}FX`eP2NLI_9TXTYwiA$5#@9ftIaw##U3^*}Qh#8^a-;DbZr$o$j6K|%@0U}M)6_`a0D=O_8tWp%L!cBrPM zkJ!JP&Tc-ybfnHTHEdw-*!i72X&;KQXvp($=%glmFi$yF&e}X|i?FcCc4~7KgWgJ< zqcbkp6g3KZhVRmH)btaqTR%Nbj0!Qag$(9QBsFx#Qaz#RtdWpNff}(SEkj2QLkCw? z+DOg@8MBO|Re_1FYY#hb6f`}(2G7=XvR$aY?VuD^i*$zZ$HmKBdjs-Zlz59yvy5ZD z4$anx<^2_Gq>g|DtCARq%iW?JkFE^_;=%4{S3K4wi}7e=Z9LKuTu} z(S6N@pQub9u1CVFFG*3VB`LagV%>Z7i~EN@yZ6Ddd*{EoyZ6ic=T6-J?xH??chBCt z`@g_MhJ7F2-S_F;{fB@0?#t;5A2pdEXD3~pxx4q`-F+qM7AyZc|c zyZ`*%{qK8Mhu2|HdX8IC88{#1001~-tEm(mTt_@Q=rd-+1@F8{+p!` zn7Q1@nM?8ed*Ax`?GsZUeGYQQz0)6pEOqbXjj0c={B-^Ly;shAo>=#;9=-R9Bk4cZ~er65H>ti49YO;XWjUa>a$o3C(bPPPMIa0siV?9eY zs@e|42M0+%wX!fCqtPX~wnJed6slz9S=b8QouOG;X@h<~pF~2H^a7^n&h9ExbSG8Q z3Qd)g8XIqh?$789(-hP!7EH*HwOq;7Vh$FAmGXN zPIQslI5KAZMy1xqKdY%2$&8hQerOcYf{Yoou}~`zq@>GhnL-{jATwM<+g*IO)09C*Bp{ou?*_Pn=%D))B%|uu@go zQP;c{_NQSh+toSRCp`>*dBS+AM! zs3wTLo^5a$O%GiFE!Sz<@-fwT&LKcY%5X>A<1Gu3s$9SoxcGHyNvdkYJXt4g#)tGA z!o!84t;5_A{0&~6mH-JV|2v!BfyWWF9gq}DtaS|5f6BEbc6K5tq zoH$2di9R-Q4z9rxkGkf>XRuKnhh=}+Z&mOuPfnbLpO0aSI%$93*=?)hl{hro{E6oR8)L4KGpl!qED$~IR zwe;A=TMY)R94GKh24a~Ge`**Ede>JP40a{4cbE+%DIiEvNDwh-E#$Bo)HWI{ zVlxO~c5A4LhsJfaWQXMs9W0gq9DDRr%1MYYa_o zU~E7L6HQ~$$_eC}Z^qv0&9U=oP3J7W73oMVlPlbys`-Z$Gyag$RCVk;9fDU0h2)gYE!_G=-2_59fRQg_TiS-uwE#7HMM;Qebvi5M(zvRne%hMn!* z2ZTOV-qBUYRW6HhvTHF;k_0(gWyxEJeE~*kN}tIwbzs|TsXWQ^NW4&6nCq!7RNbF1 z4fw_jbDQ96cPL4%W;KAe-fA&!rVD0(0NE{sf)~hd4zG{rD#QSVEQqaC(>a&V4xOhn z{<-IusNR4`9t&5>(q0*Ni|-ddEWT4bTKsUNczb+b@p|#%_&(&{{m!P`{knL&czr=y zk>Uq7;qFbi>DGwz1Rk*J#TT*4%OC6tZ}V@wUA#2DcX04&0*{7IxH`UPd~b2II7<4g z1XoM;u4QdQs-V*Sbo|xgcf}jf#Op{}yqXk07~em>*S>O#6i-77k-!-Fj7P^0c^)j= zn4+|AP_e2qOOY*k7!R;QJM=KArm!h3I(8y3kt;(R9~oQo?%l;RFt#ohuK>(21jq=y zda?K|z6I|qUM0_xid(+_y!a+Oa8L2(_c!4Fo6w2yWO(U2__+Z;-xROTy%#> zU6RMcFoG?mMT9fAAMDhU8IUoMoZqkaWvP2YtOkLH7yHW8#l!b+el>mN_SEYar!O3t ze)sF?!$-D>m>41`bijS~R#&=f{7~^?wajL$>CR zXbPVRAR!Mhmj(4O=1wdAoQGw4bAJezkZV zG$35Cu8i-Qsi0Dtc<_?Hxz?ekbZaN{zXLDNl|Dn5R~+zf(tUzt))_lH!+3dUTqDiw zL5y5=%{EeF4)tN)15=X>&3kBePq86tDW;+6=E8W9RiY?|0`=R~9tUpy!%QU`buDX{ z`q($#*#bM)iLrSxMeSluc~)EuFj*AjDm|m+7peA(Yq)z97xq@z5 z29{7SLc|hna4`*S1UJG}G48 zpM`HfJod4{RL_8{r)t`;vz+E8N$vPNanS(1kSGMJVkWui+A$PoY8+*PO+6i(d-_JU zZS7J({9!sQTf^|^Flooat=uxTcsDn&6hhjfBjrQ5FRAG|GfhrHa0p$QrucMjzq6B} z&OT%*#I7}K>C6}$Ty4Jqca{#j3c59ObraJ{2bH(}veQ0xIe(=ZgJmVIsWYlH=%Cup1`7HiS&+q62%!@X z-AZ$nX-lFJstYt>K?+m`F_~^1K)(S<{`x*mD4H4{h6Rf%?V6IKk1U_CZeHw-aD}Qr zTuT0MiNIC5y$n*-OYv@f7t4Y$qK3m^?|yBjBL9ULi#RC<0%3<2)jn|vS44;fs>FM@ z6@Q@9;-n*KT2$e?04|MSsw6cY6B&o^S+vd3I%jAgUW(%pZp<1614?VUd6L_)fos-V zxdhYU`X)uwp#vBi2*?2L;@ihAdzWN7w+!q8iOwO)_;9#}VnQ%bxixtS2JL1>Qs5Us+$aNASsLVWyY?ktjXldRk2`He41=3Xx>rjQAi#{D4lW5DK|L826xbe ziL}z!W;C(9f9l&M#fpTnCUmGOe$aKSh)L6FL}B-CQydv)TzTmwvV)r$VrCf0 zFeQ({7a5*n0^7+5DE}QGbElgugZKXN3)9EH`1#z$siQ}xj{dn6LE^)4Ww(5OCa+vx z)MS^dS{O+UVpUQ^%j{llcE^Iz++dBYrUYo!;*WGKM8F>O!q|CgFc2O|d4{DcV_SIEGX;igw&2)<$^-?HOM-$h5){Z)l~0o%p!wkl zGqPGe5qT`J%A!Dajh*@H$*~LG=Y^Jocb*|z$Idun;iBb<5p_Q?u%2ZrTY5$UFp^r{ zG6+@jyfG|A;kYOYfp9A(qOGH#{BUVC>F2|=K|+}r6asQ(5TR?olI?fx=;<$o<=ZAw zmLk}=l5F*yg5GL8JHm5$O*g5UWY2=2 zS8;1DpWz8kRRgW?q>q8tq%c=+?WMW-pCSmtQ_J0`G0TW#2Thg-&%Dd=OIrp;4|^ z!CHSyaI{7>7}wOVBbEO>CXwBD_rIX&3U>V*0y7ulrMl(q{U$ZJVe+hJNNPO5uR$-2 znfTFgOJ;2A)Dmj_$ld(R4@nXNRpDeS5Qz@h*GuA}Wmvl!BQ8ooAyQSE$5Zh6SH)X) z&d8qeBh6=DD4r>PhnX*Smh$+1J4^zn$bQlwg|h?%mnM zOE0aS35%J#%gX0rZP#=%_OVVl`f3vJa(3)vMbVlqTBcVsHGO#O<23XICj-!Q1zKB= zz?opMcH~SjxG2gIAt<(LlgzhjXbB@}`DTGz>gpI3qAf%mT8>WBI;iOlRC9vp{A<-z zLNSi&CSam#3Cn0;@=X}?R^=*Nz``_%SRH|)r@w7s%Tp(;#97#d&=;65DHqMfp7@-W zQAr2vjTxFFt7sB-ni`4YZ5un<=0nuBk?JZn=apSRTK<3_ia|k)Ip}wM=s-bPhy+?O z%=32`qo8wIhX`h0J4`T)6*c^tYoE`jsGV3d1QJ<)iEInRJW*ObOSrYHEeF3EE;MW9 zowWcYjWz2vli#Uur?dxg5yJ2A~O@o@W3nru!5U&XuVa&^oRaKl{fs5Om zWMIcIBnu6!As3Fz0U;*0`~(hAxI&Zu{j_z#L=wWGN)g(1DDjIgIs$L!h1`xfPX-5x zNp(8(!zDFERZ^h&A1=YWh!_q;L|K-IK?`u@hf5^IemKqldYWeyo+W9fbAnbecz!|C z>lg6)c*H(YB0U9V?55%w-kRaZ?P7F+g)qA4QYgzpAW|t+v?BAINKK6%pZfO7^sBFY z|LxBwzMFpg`n_9kPoH^X`s!=Vx zs^vMRhT1uzu$VTNWU+~lol&)6n-kZg<7g|((!V+iWrz-57MH{FTL8#M|emu3foz?DX`N8%-CO`SZR@KcD$(`s8)*f+giUD6&sJXBnmg1RwULkSK^% z)j9HRHR7#fXHXL)d7F#OgaZ&DOBL{X`bL-n(^jQpXU9I$h(2~E&r_J}y9(L`cS(V# zSA(ix=Stb{SW6AF(=tt1Mn@dS%~nIV(^AtKS@j%7aC>K-?C$8>u!Z#ZywKCP!Lb?2 z4the_B|Tx-PS%Z7>esHrTG~nH#!c;;Kn`gx05+&L%?9qrR3ROPp}{_FCv~CC4JG2O zpw^g3pl5=5iZAphxy>Zy8LsE3niWL{Q009|bV;cj#zNpRu@a`HmRh>t+re|Y8A`Er zipj&UukbU2T!Lx@H--U5^eVNe6hcT-??7zzss%&!O$uHQt6?Z6h*gAD zE76jxYda`1v=f>{`d2N{xLD%|=0b~A>#ir_UuYG@`pZohySEur0y)DZN8RM!<0m_m z_6}siunec2$RTpMmAN9A2`yDo__`L2kt_h((Qi^K#nc4NsD$Up?{_e6GL^uxYi4!M z(#?mEqeY(4kpclR-Vqi-@(skhV{jrzrAT~jXH<$ubn5Nif~>0GC z%(^v(!d!ZA@%+ORd&z^(&ObQw-U1cYHukku0MgdS@>W9ONrZKyrnBV{{Pp1CrH3EB z{P5kQ58pdMf)9@DArHSk_3-3z&j5*%EPy^Q21P-V9fkqVrnv~#aF13MmspA7o0cHm z1%-Owk!lKUvSZ_Mz4M3xArz~mS#vG4*&jy?&C-+2#vim+XWJ=R%N6ge8ok>MIGrb} zh01D-eeAOHK~|Gn9Cvdu-DyovHMH=m4N0^W(>)uOYG!CoMQ&Yq4!5wlg4GZ%0l+0x zBazh`mq6G0p017CHf-E5I9MtbLl7=Bo&Re{5(w$G-8FMTpX+)yuifywp8kwpD>Q=O&Vrt6;PN-aNVp6(Y1FD-w39XN2_RAUAwk7 z7wiNwt`2FKFhXoD;$_XpP$3~OB@OGMrKXrpUg#wpTETe=DTOg1Fy_B!<1F(T_hRhx~5wVPul_=#|)QY!jd%n7| z@$)Ljw_{wk?dN**GIt~#Sq!wT$Zbk`j3mZcRwLB5?LXV*BhuRqEpX!6T+i-nRxMD)A$~LzGx3x}>`DuLnJLsh(l z9;rBb?a(oIW3G1W_+dD`WEYgZg~e)%C+za7H;SXWh%NIGiokPpe$ItdJG50=B(gPv`RW1xLH;YqVZ@=$y3s)T!Uz_7 zk5n00TQja|#@(`g{Lq$1Hq#dH%r3fnNns(@;^w#geO68zUQGsIdTGOT+QG`4lcGz` z$p~hnhhi;6%9abA4-IiM(@@(*Fi?YUZv}!fH%&H=oq>7cokpX6l3q|0ho%f zr-;smIeNxOI2s5^(O@)!-lBzKkJf1kE3MfYk7oY!lEV~=E6d%Uit#0bWwhPZtO`asZROLeqy8ye@v4e#oHFvO3WYKXT39+T| z#BZ`?#jbuaK2Kn5s)@N)N7YN=Xj05;Fu0Qd6IFfK6uUAEM9P1e?=B3MeNjQz&ZuBW zkg6EKhNydY;hnvIKY~T*|FP%DriJtH#t z_2gN2^H&7kec;aF$*XtvI*d$ZWTNho3Bz{Qs>}N8`c+0#c0sF{kD8u!6h0zCq^d}X zmz0;Om%sVx##_xdaqhi)`=@KKPQ7~Q{+^eo4&vg&JmATY!(5t=Rm?s#rgwG=fFRc>1+uHV;#knS`suSj{_KlF`>}>gXRJ z0~>mK$dXFlER&Mk$1aCVP>^H3Q=e9f!SM}Daw%T6lVBd5ao>;0 zQ4R^AK&9}x;s}-ObvOen3OsSdOW!K;k)Y6dLI)H^!$Ay|sod!W)rn^_wmT!So3oL43kYYo@l7)cmIx?yftlwZ2_fqYhcu-HZ=Wy#)Rcss@qZ0}@ zGI%QnPR%dCGdTEf#M8U`HOu+#xd8!+!uqa}^_mHfdSq3Z>nE|yHn=<{+;_o%&Q+eG zDZAFwfR>?4WlFyAG$rob_m>pJ>R#7P;(Us)&(;kqbI%o^u~Y%bq;{ zW4XsA*#1suY#Q5~k}61pdbHK>{XGA=DBTu^tOd55RAJP=wxD6(XIo)CvQF_v? zyT9K;4-rU=WKPiL6jqERNR>H8DCrx!ZopoOvUtZ3w5IY-#Yz|?Y2R-$gXorFXof9& zBXL(r=58(%VV`BZz!dLd^LnOhDb38Aik4%R!@}g1JpnKc3!z{uXKz+$NHX8#JWKME zMKQ6v+Ue1!E-AsnvU@LltfJ-Bj+!~vEE z-yC@G$%*(skDvPG>IL%8V+S6bIYk~Gzxm76FUUU+fAR3}iGS>QZE*16dk0oM_~ho( z<#%lC+(I6F`o%Bbj?UaSIJoNJshdyR??3V7FW2@o6~O;Iet&nAZ-ZK;)9zXE# z)58zZj`v(pIJi&cs@VG zG73nl_;Qb)9{X6`Wj(F{qiG zXCJ$qia{aRsySMg_jrP)twzgxk}O0;-^@-hmHKEdkEmFAA_zh=B#_JnGuz?ZZPh`} zr7&-2+0hM2M%HV|EGw)TU^HhlPr-j3Nh@!uNz>99oRL*Sx|V|(xCyJx5(PAS3vM5=PX`BA zlc#9XGT=6qv6RA|Cchz1&Agq$RgA`+dDzqcuCt;4wR<;wJ4Uu^mabA|1ViY>XSWtq zHZsI@Jnk0ki_)|G8M{PM>ALjnW`V2|MobuSPd%dX=899Qbdy&f=Wi{V%&xO^YfX;= zqf6IRo+L@0rpPP><4W{Aq~@OJkZohJs8hb3IdwU%qCzm#qDDj>KAtowp$gBU6{|5C zhZGZHRhGsTn3V)0ircPCR!A}LSk%s14wg5~pj$wYg*VdjFPEnWYy)tZ$4I}1FjPh{ z#XszSCwwePVsY~PTv4*Ft-V`b=ouV*WVvCFCy;PsM%zJkSRV5knqb~Vq3VP$Q4ULG zC-lsj%OM~O0jZ^5RL)An;JyogFLhF)c-pEIQ$_>}W2QlvXb>C`A+$6(;>m!+Y1>yc z5GwLMO7zuF1rK69r7B0V+v3jf1<*qIykWtPqs7;K$2KZ60X6sG@U)BO0#uB@i`0B=p@(33sh0O2Xf4Hl*1SqtZ9o3E(V?) z5MotMOxLUryLWeP*gCS6C#jy|?4_65+y7tquaO$98yvd1kt~&qAz1ecO{dIqD7m#? zk4qf^G2Rml#iJc-gYoEEIS`M?Yb98gqTTV36pqC^f|0eJf!vybkI3g4(X}+&~1#7q7Z3u^}@QM6pWl@5NKN-ri#%}zo#4&!u~_so+MB#ljf-oGi!&=(;2gY zS*%ga(hvJ59a<4-;M2#h{rukfsXy}J7~>~ z@gvKKRw#;04$&%2yM4Gii@V!h9W|s3dGhFteArD#vs3-UmHvEHaUU#BR+Imvd7WmJA1?Wvw+^dsc_o!xFQuX=M5;Ig@@{v%l&&RIMs${< z*<@k6y8Kz$Dq!f`FyK8VW1Z7Z6s>J1iZ0sek|0-QI%hnj_)WYqadzVT#PNyG$i({- zZ%>?r(_@X6IoxzVd}aKuCnr9hcoTjfD??HFN_>7B--qiT!u3<|^PXqOCf*e$j>3D7 zPn>qh?tb`GW610f5Pe6q0eI^**scdC2cfr~kUgSpWyYpVN^wto?6NZpv+A{Ysa>7{ z>eCEp7AAG-tl*GBV&#f>_OVNdLP9j=JIrYX*(p*yjbXbDwNZ#+AsDOFyA}*GZ0j65 zZ)U8SoyIY%I(b6jT3V<13}?0HRomLWwhB~AWhn4r^N^YC?8mKvk#1e%H8Ks_!cyCO zFxs|KqdmpiyC~9kw9VlXjR`SH5TR4WaM&rU1!`|paD7p5qJmiE(O7}` z24kQ>+58sETi=}zH>zEZX!w8vkDMt-}@HlbjkMuFCqx(QJ-tSMRa z)Yd6A$Jg!?6oqiOmCW(06Y457@SEc8cDtCx4Qw3DDrn5|nqNT>@+yUN)vYM8C7(xL zAxaFahk$;n??2zY8=Y1OEsG>pW`Zh;k!K3k$q4k8`t0uA{TsS_Hh1)He(5DLlHm9) zpMCMgA>?)5tm!<%b&YK1MusDm^N17(hQ>xyDY&*4{u)d%Qm{P} zElpsrXZna3iv~tQBeA(V4U?we~blPg;3Sr>e`bs@y5i56KCPK?PNG+J06ZtoJRM`n-d@5mGf}r z?TKS}`1ZtG6XzyAn>a}(j+So!7|zi_@d5fI1ibtn{yK?IKu-p~b`D=7aLe(DGw=rc z1icyP$M_T^h&X^*o((5hf7$HgRPz8S9gxl;!QgVaC8~vNC0|KArfh!=LWrH zwZ+RILQAbkZae5h`Qg%PvPChiC5LsCAmtBU7HYPw}KHswT) z3gJNIX5SLhDulE%#qWw&OJdow60t)8AsVQZ4_fF|p*L#BqF-Nz_UaHwc%dc5Av_6@ zA?v7;uoOztrN0e~3kk_Ws49)sOStF&)00f6q)$^*v{b@FNwNBQ;2ujlLYJ1O1UiiL zu>x`s42@kk6PjXEy0nX7&-ip0rln^&b&~6rr9Ic!gaxT;wOg@sMPXJ_v*SSax-Jw7!~+3ATw3<<69q+r{&0FV*~;``syD)1fQU$-2(<#+-6%r2 zJgBbJw3NnaLkO5$Blcs3?l3&%qj-3ovhEQI-0WNYX1}7^{Ke+Rx;no15f-HOte7Ht z5kZbtiqWpK^Amh~n(bhEw%KTT;BibhNhj0!4~M)v+bvz^l&b3;j#EcX7};e}80`M2 z5S3e@)W14;uEK=51TDB_huxrg0GX}Yi|^aKs<$x;r%^LV1Zl3SLZNF2K@sh?3={QF z00&FfUtac83Gf&#UjP5ud)Mzasx)8tU)kgN(4i+Tt4bSG zAguRac%P?aM`g(|RV5{gVm0kpDwSmI{p@G&=lc7>&nJE<6s)OLvTAy5xb|$*0$uve zkeVfkH?VYM>2a){T6!Ek)>JGJ5}3VwDR78o8oiDF<8YdY4cmGEu|Q70vk4d)lfNun z!fav+FK>0OS^JiR5GEu{qRXkp$Rxbr4*CsqCL_D6F$$^o@7%s;Y~S8J0j-KkkbuHS zDyrHXY_r1>@PbZ_Gm~2Pu~`mpmN#fyymd+yxfpE!IN@YQa&BT39#M0-WMoN9+NKUnZGkA#}!IXRn=hn6r^iQCL#-QI|o~XmfJj> zw4|c7q)Lm`kP^If?9$o49U_-rzjWSj0=e{?OMm&>(Mx~1^v0zRF1>Q;txGRmIs=c+ z{_V)6(+@oG*Ym?SNdr68c5&%R__iNhdg*V6F1=g-)(yF}zBc^-z|bz8fdRkj4;QB5 zZ^xqB;>;<{&^)|O1F~#H8}MOBS*DasDt6892y(8L>6pput*+fV!6G|ETe&M?KZ}`j zD092GrR_ks21a+LpCk&ABr4L9gpJ&Gx@u))l&sopWtd@Gn6|?A=ZhEeOjhLLg&D7d zuF=ghEc~w=P#JtWJd!t*ivQ)K$zhi?8}r`?sD%!vX}L8D-XdBJ;AWR@5p71Nf@x#s z9F->7E*YI=4%sw55k0drYVnyF0iQC?gPnYV;ItO{xJar>D%qT0*oY1g_@%CeF6pi~ zWosRAMw3Y;+2Z!}b3H9Jpl9kx!I<9pUp^<)Nfz}^n{6CB{^O8ZmR1sp=FsSdJN9>rf*|{u%p>~GM6v4d61BKv6d9Q+GD$0(DK4LB z&n!?IW|)Y;RMv5|+ zj>Dt%>%*h4Hx-1e4p)T3@%;Ol#b(@vRnMlUi7VE8)0k(8Jd%sUP6qa|%?;04SnFbg zlb{}h8=2(>vu)~dXJH?!2OnwG3fUqB^6um%HA>7S zXu4XgtdOm!qzww*?PGhk?fm)Jd4~}b+HJi)_!g>K_-i$LbiKBy!<$6msDEGgb#ti7l19D?5M|l8UleE z*D9_ZxF=x!EPuj1Hmn`f=gabW*g6fp-YOGa&#W7@r?cDyk!-Ul45w8j15)XNG@x>cn?v-X=dh zdG!02PLOL)UHIFG_^PF%RlfB!S@eD~3zo|5ae=ia#X$Sd&8U-|yMqu;;t*7q;G60 zyakgRu1s-<1r3b5A*D+T@D(MF)&@5YJUe$#qO($_y_Lurr_$v%d^KyFv>q;lJMyO>YA6}lp&G;#uP8?l&T2Zsj zcAi%XpAe`W!^lwW;?kj|r!gw@-qO>x)3tLxsO%AJk-#*8f6G~);{77t(co>Y2h2WR zdVF(`%kdL<^Wh#d;cM^KK3Y141dA}vr~UDM9HX(P_D6qC-XE-giXSFD9ZnrfF> zSH>AdPqtNXeP^hhUj@%l@d}d9GU}OK1FS}=WR;BGoQjKKG zU!(Wtr8EBT4|;(0Z?#k|9reHX$xAO?It~B5bm^5#ZxFcn>fet3?WkW@xl*4PkzRpT zOA3`V4}T^gt;oS7kQUOI_Pc)q`yR_9WlX1Hz0u=JDuPvHlFc%iFWxY&MByqQXwsr@ z4?p_%mma(N@ylO7^ym#({iY7IM3~^S=6r0Rn;%;GF^OC~_4wD%pZey*XTwOMF$pU% zC9H`?{M;-uwzQ`rDMjm}@Pb7?@6#<8g+VuyeSE$v;5r}_P<0I$YT!#<{((= z=4UxaDGZfG4>n}eimK^~n$9Yk8q8n>R7wl4Q))s(U;gvucP_ts<)Ps*t5O`g{Q8xnaODjAKK5UaT{%qf0^Ih|<=5e+qnF=Y z2;On!ku~lho3!*@J+^7W_kSCH-X)j+1cQ0x%0pL<6Bx}S{@9L37V|Glt14eo4dqLP z*S(~YNX4#oyZ7GO?`kg$*Zx?0x%O=BiQ3BxK1=Ai+R3FO)6>ED-8onXlNDxgY7LJH zkPOdzxVD`kpVI9D{CBbT@j~!PqweY7{ko^sH(sorS~@&EeHVd`u2)zu9j=|Noh51( zjNr7Uq;(~k>MaMl_5zHfUQ_qUax4^sqlbM2nvaMN4TRN&hsY)b7p`4cI$AF+hube= zEH!!{{{XYBK_tSG$cnYM0_%i3!H`JwHVqX)Hrj-2YR z5e=dhnE1{|AiP^Ji|ppMSz$7IsALTU!iHvrRxC|b^cM8ND1L-1D!vO*bd($(W{UOjo?>MMVN$7IttzkluTuYU6F!;f8k;hcZ%P;`TPN>jq;mAoV=@zuEnXq|dI4$3Rm)|B}B%J%N$NuZF z%cs!Hz~AVog5h!bb-YZ#3V9f9kYiwffE5DP#xa5)184AYJcEzJr=Esa--X-!H=erk z@Rx@k`+wiXv$@CYiyI9R#&mpTkXN3(K1_I6A7+Ur{LR{oXsn@?%BRDYIzLIGqPv<^t1w2YBHt6 zNbe$>;Noo$skC}M?FI*%h#9h<3(F^^W)<$hJELbb8j}*(-ZKr#TmNy$m6KC)*?txI za$W-yO?mJU?e5i(t3KF8t{rn$)yO2PxU8IK(x=er1w6TR6Bd)gC{?^lg}H>=vzWXo z(0UYc_sHZFnc6u%M)qzaVfc0`ukxUrp?rU-Hc=>fJNJse*U3J#k)`fLsqfn{ky z5C@EM7tGnN@dcqwQvE!-9|T)2G%izVVClErMcxw?(JgZQZ2BsxU|qf2z4~tz(GQaS zl6=47L{j2V3@Eh2Y}=+dQVAu|f_Aw9#Q}zyJ;!Xn!Ot%63b*(Si$OuxIYmjSir!)} zhwd!e$14&X0hmCrH7&a0EHRSN6RC72r4tkO0q_o1{%#^mRHPCZPGpG5Nxh?oj8`3Z z@eRp{D`}a#4CFk*Jy#4*2r!iJMw`xZWG;u#UZ~*34hsI(1S!XS|A^yYCd3@rvirc# zgy|xa%7n62iM#|Ry$X{yj38LKlFJ23S8}2Krc_E%Tkx4Sw0ar{<@&!wMoF?IBN%QbEF$V@w%FXJe3rl*+vobnxB3{nT}t4R5x{Xw|T&<8ULohxy4D2iw`P!3ubE+&f^c=kpL2{wF4TYBstvGH zGKI0d2vhL~n3pGDQeOJ}50NG7157AVQFB_ZA>mkcU3x;vX#El#wBqLr9WdP+NwZ|= zq?xWMT1#DPXkY<2gmzwg>EsQ%S$6FkXqxbWeXQc;!CS%D@-89S&8RqzE`WQ+wvCW$ zCtgH21DK&Sx%SNQYe!#gF$IgVX-697w+GL1z zSEd~NkM5j9dRj@uCw}T#`2QVTQYYS~)Cuf=75l%ldUIj-28~c7r+t}O)E1Q(8MuxG zsBwSGH5s4O?S&Ug=F=1W`4hlT=63z{d@yYEi$9V8VyMKI-Q)8kCOI+9lznI0y zczuzL!suM?ddxPekwvDeX$6bKwWN|x2Q1x&p&MFanxeN9(?;=RUhy52C&}k0KL7pa ze?q6^_rCT1$It%^*H0|J{(G?LpSfv)vzt`E1mFEB7|Rndia&h*LS!NB;$=QdZ5|+P zNHy%OR7iVE!;WITrRhHO&gL4u*A)7cTi1ct6)QVmwR>yxFrFJ!#ksiDO zrSrP;uPeG58$up-v)ycPoZ8Odm@Kfo%^e3BXwk)xN^i;LZUG}evv&(E$at-FW0IF> z&;?NgHlrEx@fp32Q{jRz&%eeKx17wc7MZXBS$gB!_fJF`>W(OHo zUyZkA3|gk4D#@?;6$mS+9YSc|YYvE0R?@P#_RbKXgUS zHam_RR~Ecnu|({;fOch(0VjUr(OK@zGj7IQ|1F0^vf%|LL%V^>a9r=N`MjuT5Rh5{jOQ?e>A&F-~2hSO@jEr(#mXf}i0wOvws+LoY_TMz-x< z83oKWk5TkLRb;zqejjy%&30;_S5*z!I0Pcnlx(tJ>gN})6yWfXE*u~(v%OB%uf$WH zl#*^~W!-?1xo-4wLkdSoQRybGzlN=Ly4t*{MTw&%Orjzp&&PN7fBkD)s$lx}N+XgGSE5PfTPeI#WZ0wHH zlp?oyr4kQ(y@Se?YzPHz4 z2;V(DeO<{8ObqWB?R=NY%`ysBh8l2HLJuA*O7;k3i<36CoFa-T+38r+HeLZ9(9$BPKh^mVVYn-^rDr~yM_Nx_Tv0t8=v4xvJk=q(WkrGxy6wzrUMSz@=~)$qQ-18Ego9#^Qz>-IUqG zi(nF~!O)1FT01nNr`DBcT1jYq*KavA;=m-pc)H{9=^13RX(PA0je;%BvB2Cxs0BnQ z`RNcDyzRAR3$s0Up2sixtUy&L`?iLYy*Zf?xFlG9@QY8y&i z<*F<*Ltv0QMlPs>BFlh{(=S|WRDODv%PXF3xZQ+BP5NDcEr@bayn|hmyS&sJ<(;cV zF;!$_ix4GjE1eXc6~kQ?`FwR>!2Dm^S6;_cP9jTariEB3G@8=)rynN6pcLPKaP<3E zpBet)sT1Fwd7J$3dtFskT{ zw%49IdhNC2*N(geQyg^g*5)2p*HtoAQ<8eKlG$NpY%#mUW_ZcDHInF*dhMsS#S3J1 z@m1-ir13Z;F)dd1*wE(lryEs z+YGygtKjxjjd{D*%Gjo;$vkJC>_WsDYP(f3CCb#9XLC*|dZ=)_oM(=j3&PH%Q;kfc z|2U*vGD<3bBIM4|l~vIwa5H(io3SW0r)1*erEe&g$ME2+F=1otroW@9iq--GeCNk> zCfuq~6lJ^X5gje5Xf2eBcmDkEEyEW-*r9ok-mwnzj%@-hnZ))GkL?Bk zbGO&IzHZkbK3d*i$xefj9l|KF>d+LUWuIBqAxKhAqjweVqqEF*=2rD}XhFEruz00O z>?+jN*ygccIXE@7yPoElk`&IA@)zfyY*0wFaWE4q>e{t>;o49iEoPni$g8a(+ z;{0z7Qs8u92mj*y!#4T!xA-=nb1(2IyMpg5`geYL=(R5oJ-@~p8`wHF(6q(|wlffw zjU|WSAq+$1IzlLVa3H}T#`?oso`~jYioDv?L{7=6N;0cxsYKvx%rxj6{l_5}Tuw2#N^>= z%qcECN?=~#*@v}L%XvtcyV@C?s`|X1!?~$X&(a~7o0Bjzj}FxZnHkhF(T^Bg{QT#Q zg`+1LnR5SexP_Ahvnt(pP6kHi`8>^2cQe`LRXu~2y5qwqH6@*Dj_OCvkk|fLdl?q! z6SbEYV6h#6r3Wh%^W~rTOY=Qn$6H8Ry*fc6DD%nM1wYOE@j~!PqbB*^{hH*}H(rEw ze|UQOE`r;~d$?09?+24bQQ*?+tx#L-4VbixxCxRkk3LPZWK-<~OyVKXB&0#ID{<3%^Fc6X@o20!@J;#!$UgEkEf-13)@t13(h_DAhh6 zcG!Sp$V!W^v*D@53$nzkjKDzPhky10yx-zPShQ+U z3F5-P{1cavl{-Q299o^B&Znnr`Vaqb+iP1Z`degyZDTf*7FTnb0JFhhZtW)g+&Vq@ zN$fFBWZ;ZIO1*+wWf@q`bxD%x&sU(Q1Uu5^=qGGn!RXL^ly&Ekj6S$n+OO+$- zr`DV*H)%+TOqLg4l^ui?y-6mMHEa0KhkbId(yn|l^3(_4c^!F5U-OJf>g!H_ zTFGhs*8E#!kbAS#^4hh?8>U1$bZ{b-QF4hs;`sFvRht(i?`m;sDyJkfeYjXZE&-mx zW-`I#XD=X8Rd>!IJ)x+%PBg9eK!#I1`XSA|Qf9 zcp0clD&4Oa?o_?-vnRj%@Wl7;UWhEyPSywO2yNN7xcz<3VsbN?ys^9KUMf<@(VFvh*4+Pfg>e_E#8S#vy${f2sbB$u%Frt6}Fls{4a{a78<3u-wz09B7O?^?-HE0<7#RROmm_T}Y7Mtn1GZq*&-}mbY&=39e zl*=k?Ga0eyXD_fKj~+tnh%>6D)<|?l>b`lQc9CF%4VIlR9r0U49<5y<@Zgx=d2T1tfbaoLu*Y=v9G>1tRunNf0?=CEv^QZ7EF+#6RO@qbRk&ncK47#!Jj<)JG_ zU{oK%*xn?U&s{lo`Hjo(k}D6wtHw%IhAP+afBugANnbo zq_#`JONJ}lpkzOHUGxq+S=b+APYqEev6VZ9=QuF6oOmvbhczVF`r7b_ZC0fyXQKBw zC&(UQxZEKDw5(jCj3O7hj3Sj#a`BNr?>OtdK&?E&e0I@(#|GI{I~a#!2VF2fsO1y1 z_5{S?YGK?hoW(bhH+S)EN+zgXpf(4gL=ORh#bQJ%i~rCqBLIydzcI#;a!B1G_(sgVWO?g3=VW-6dM`rlC_1 zu;4#?c8-H1y0z+*JhG`G9G4oL5#}1oUA>?Q>w?_wqA)ng>=GRsk#4o*eRd>zm*j3b zBV@q~_@K#b!(j3shm=)P(bT@hv|=&c$D|1+ZD$Ooc*>JfGP!;-0Mp!M&moRyo0J<} z#HUaB1ikTblkOamjLZwk;+HbED}}70#y`$CPP9yLYP-d*hRgJvqPK83-T9W+&N)%= z%A$LwmyW%2x)!^4PL_#h`$X`>0d0UUyP4oSXR3q}iyA)t8#`EGk~7<2cuZ zSy9~@2W9LalLmJf;kGL*CMk?4vgF3A1pflz4zY!c>C%8n7CD`xV^3Tz)d&5(e)_x2 z_O1V7wTl*jjwGsTx-FyZjll#V-VH-xAaPj;u^?8<&v-+_E?=J0g0pek+zu88G4gdshV5C=y79tP#+$cPrHb zhR}zwiVIGVD|HBW@9gi&0&Oun|NO7yDo6oHqD&@c==ohVk>pjpECLL#W+AlE5 zC#b}=dPBUpfywOyO(I$Kx#20AZkGOyQ)8kCqY*a2FJ`gj;gM0~{N%0&9!K;zsj8_; z#wTOa8d6I&6fqTE5mQPc-BMQ{p$uRIJtu0LTw-I_ZeE~^uP>fq4(zSmfNxJtx)y8k zz{uVf$&J_r;fG;5Ew&IYuOG->e6uyU?Oz-APXSjrLlboIZHGBS_3t_~MJ1b|(QOme zH@&k;Qb`3QIH;|i57O322HJH7#N)fh)8y^%T34D!*u-6_69my z{J-11Shto|bUn5PZ74bMa#t+?Evx7)*d#ZOBmAP`aEsbL0PZS*F_XB}idf6}yl1k- z`Mk&W6NHm4b(wQZLI-+6Uw5l3Gsz8VlL_vUEilz&ib=U9=<>k$&VkjUL3^SIzBqo~ zvVhWQC7V%H@D|g0K$oI|FYDO5sAgk_#Cz648}$%Gsbg@8arUsJrl`5*8s{h_x3y<# zAJ)zBS^S)AqlO#fJjvt~H3924ol`V5 zz&xil^gs)*2uUTQHV3hzfW(jO*ot7aCbJ3S_B^v}c(i-*729Oe_QPq+bzyHaZ~G8H zQw@SAMFn##J22${*T_~{jc%mvqE1CiBm!d68)DMKi%C~f@hhzEK-;-UnXs7`)5+}2 zA)icxP@2u=C?PZdp!PEZ`6`)!S)dX>DV0IJ`6puXnVGfX&bOD^tXok%BJ&)Eg(1M9 zee72rb6jU=l*S-%3{Fo#Zkd4KGT|kcRdU(pL+HL6G0_c7iBeTGr+B`3>d)VtIr`sc z5B>eM*KcqF-96EA0^Ln3zCqLdtj@QGPJMmiFJB#b`>O{Z{OZWLua123)sa`edho?H zOfxmvY?@7d_UlshvtKtXGfh(!H3O$o66~|I8jw&_Bpe-E1vC9fX9zck;>pxKj!lr1 zrX*X+uR}Mi(ffqw5?;o`^`8h{7xwMH->nwNR_PVQEHxqvVMKDr6)p>;&QvXcPp*aR z6VZ`YGtttP5V{N6$o1Ha@8d9mvy|A>h2!61q~LLrx|ECy+ht_Llb$^yWDYSG--PL(WfDAj!e(Y*%kmcAWY~2f2@Kl7 zGMFCdw^}W`N&MD$iZ8w?EK%TK5P;`E_6y6)^ElDz2}w=9y4fn8MUcv_L6&vr2LvYY zI5DWK#H>y1OiUn^Q*xO;-H2Z;T1FsO2TntnxqO!jMs-!m#YexngXA&qGJ}G(CBfEm zJu|kg)vi2}?c|g1NO@GMl9ut@?^aQ@t}D9U-?LWQK`IXOOd*khq^s>9dPdPQ%^pdc zRn_BQ&t3~c-|g94NH7NA1Heh%biFP(x?CnpT!L(a?oC!* z)4$28>q;f3WLpg7KHge;cx!(qm;ZF-AsExK$Z~DMQOh-CnhTj-Nc*&onHD6d2uvFH z?3)_fGB!0vwv6o?*|YVYk?nhL5E310&00@Z>J!q9km%UFOY;=$h(O|J<Lr#5_5|8YohX-cjIcsKOM=%c!ij7=_l zdF+Yl=~2lj)(qg_$sBi!Nr1J;AI4k$F|&`ElmCVXH|G* z>gR)js}DTzlbf{Ft}Kj>QFFp+9xVM`br}qgbY=}|&M9^Owmfeb?;9AgTv4Hp!CNvIqn_l#;qKy#H&E7{Q!UE66Q-qlJEYIIf>gN8 z$tD*j&4$z6qH7$A1H12OPF=vlp5>s|l$f0C_6(mirIA|Kl1fI&Bm>8!x^_c@x=uqw z8W{@vlqkN-RxRKM77xi@PcDB->(@?O*T>x$&ln&X>4LlYJKJ( z*n8&reB0YiQg4Vr4=sX}qUHKvA$1YNtT$|52>Kdf>^|7x%>_GD6AdJ9|8YoJfzg%i z+e?iCH_=89RVRu&(%8k8O0|WpcW3la&7$RsJR&jJ!8(Q3o%TE0^!}kfuy~|QKK<<% z=O3lyS6=mtbDy}xT09K{EC274U!4DwOU%U=+vnhkr7#-UUCF+P>S=5)D)Fh?+I057 zhu4$M&PSBO+ymSm+ASQ1VBiPqY{)$o(~pyFlWcIZY7%g+mUBdKg;0m&s|Z&|X1ERW zMdlbQ5$(1&5LU|#=aGzT+f5us<3Q_771?f@-$(sM`L^VzD{3}0Gr_>fvZEWQ0hOYR z0Ek7~o2Al(4Qm{V*7yj0k;{6-QDe*zpIYQTyMZNzD5B^AEwty%95P-)kbokCmeWB_3^fVbH|ES*XVdr~b;+sOw*Dnn>4Jo?0VR;NJ zKeA_gK$1zdL5)y<5>lF3MQiDypD)HrGh97??wh|ndiB^tSKs;Qzt5ig^3aKYu?4pc z%QM-(lKbyBh2h~F{Z4@g9w7hn4}*i`>hWXWzVyedk3aJF58wFagSW4~aPHg3kAMA} zSO5N(N56jZ;??6Phv02j|Kr_nUp)NZXWu4+ga7(3Lyb@Um*oe-wyOVn|NZ~S?O9P` z!(fq>F~#jHNJfqNpZk}g`n`&xe0k{kZ=OB+_0uQ6ed)}%&%b=Tucj$hM&7;v`_Iai z;j54T`s;`P*@fr ztihf3jF4rpoAy><*>ERd>&4*AN&EezYIBe$O2PFccRhdGA&V~(i;@`$D~KC}QWL8P_E8|y%6bQC_w1wW=rc6=^MehT_U$6YNK|?HN#}D?jU-p~EMntX8 zMIP9}T!v6=UYwq)&h1{Ql#*<*%{HJ^mNlg;q~wmUyRIy0CDl(Q1eC?gV?>~Ou$!ua z!Ir)W)=(Aj6`5@cIWxm!@#J<+UG#*KNjBSHk&tnDq39N2N?8w*Hmx$|Hg+*d zV^ECMow72uEHB_9u3c2NZcUl4DLK74trHrn2v)){8MECPCc&ZCKvNDdZWKW#Qxbgq zdH;#MFz5eXb*<-AXz@a!5Z$NWCCyc7>get7X{1Qh`*lx^ZH$5yW^{Z1N-Jr#Pg@x{ z?rm;S~F_CNt85iGV}BE0v2G>{e^#rrn70xNagku4i>K z^^@^C))M`Yc0(pr=?EBLH)Y~?we*t9@%M~dq z)q@?Vz>X)7w35_f$ANmbngjk4H@XO~pTRz=RLjJN4lQh{5=VH_SgSVr_Ogtda3!%a zdWO1WJFRju#VoR$+0i4QThVzhR^YSp0gtrdy<|gHrDcY-gpzFOKyboF5^ASw7vTrV z4Nj7k+P&KQwToE4xAY8Ly!YPW+Ow_Khifmv!%ymEeZ;TwJL|t6USC+}HNTt^{^kYv zsOOd*#8+3UE6*++tDPY`ggZ*zTce5{D|0!7e=x2}iH1Mt=+GkAqFyPJlEmQ>O$+iB<;_^+Qn z`Sr`sfAicsH)%WVSQ+_hDSY$#Z?9hb@asqaBl^V4YPp(cjvJhx>!AWIaG?25LdrmI zr)!By3x^hBfzIIZw z+pr9lxZRs%r-|7rDRMowc9&@pt05{d2Ih@SsYvV^6!m_z! zFdtlh7INQ3rpR{md5a~C5@@Z@S=DQ7_Ukqok5(&1!OgrV=c7wZ!faTu!&$hQH6)cz zH+EhBaY!j;l}!A`qn@=+z_!hEx7evbVGNMC40f@o+2#mUPw%M+uUem%UfMxVz}U{$ zyK-yqZ6>wn{ixhUzd!e*j_;_^J2PRWsEXc##<78YI$O+AHaH9BtiwCN2olTBP9>FO z-_LHyq{5KGh$Fjkmjt4>6V$zP1hQQC%xxyiU2Jm7_(W`(eu+^-Af_kVV9V)h?!Q?hz<$TqZRKC9&713dKX#80@~60QU%p^Io* zS?oq@V>jyA*&p*<5wokdL-U?g)Ku)^{GOfn-C~Z(E^aqj?B>d*+VWxU2%-QcO0e}< zlsHC_BUMNYZCH~pAf8Y%Yp|zoT$`vMsZHJPA}@hZa!NAUhsOvA#o(63%UujhW`|W( zN%k=ouL}k7Z_TcW0J``&h_`aTfbCvn|w+q2(oK2|l4cCJCRwpC!3#fHe7+Bd)lwotN_+r{qeV90yk62!SN z=hg%@;Ru{-Hps>fx=h;CBDi7Q0PS?ZZwOdzgxzCvw9$8g7)?>W3D#J2X~0+VH(a+$ zMqzxCk;$TDb57oqYpBY==-xfeX4t5Z_8h2K!xGD;6%|oTbE^!obi;g3hc=%zUD5mX zO@YDzkD0NnG22fI}qnd49kj+qjV0R~7N zczZ(OuYi@9(8RNdr-Cb=ljMfq(C-qD&w~bcVZVy zYKoTaR|TRi_URLgZ`sAoWXl)l|D#MM|MyEV7*RW6nI%(-mT9g>+kheoIQfK@9HYn( z%(9J)PP=l^m29@zhYXESdhfkExHZFuf9W|Cugy@$LG(_~Hp#zvKKsExj(DyZY`S4# zhVr>>k+RnP6ehVp?QiwJr`bd-&fxDp9AZGp! zRVju9Xhu#~)MN%c+<{v1Xg2LzC2Xe2VYN;;UqQ@m1R)zXZz{bZ+s3xy|4J!vRmmsVrG|3g#^? z$;{#h6640DS#H}FEILw=q9AD*CKc`F8-$o9$66F{a*mnIzAZvaBbMdB=6E^&3d{(_iYF3P$+lP3dq;UV+(GDLi`(7vaB|pO+Db=w=NruXctMzPU6od#%IK~N z%)Ac%emiwlqgYgB)Y(dZI)v)sureoxu<#0eUz1RWfCcHo3rPL~GIF}Tt98m)M&xan zjXWb0RF;UWm--Sgons|Zx6)Z|j#8gnCoZx|4trD8WOI>8^inx$`Ka2`M7fgd#F)es z%&~vs!izIZl6;lSa0$Z2N$N3IXCEwG&R_*wtW2XzYqTsA*}!4cIEz9{DyO8h=7e`> z`1*aQPOt+Lzaww=o~;XhAA?P)1R)6tj@KN&4~AZV&3$^Bz|6p{+lPNSzVJ)tIb~sb z&h+$VxVA%>!d`sahMT@T_Pf9ZvPqOAbI8i`yQ1IU)in`$N8Z2qw)^RG^NvMl$$sWA zYw_dA>tbiA#B6SqOp41ZutDougU?6{yGGQ6l2QBUJFDuG^0ZLwT4@r9L{pOK<`~lr zdn;S4mDySDh?}Gq9V-{w${p~Wupx>Q&xLQ!8lq@@ZFtO<+zUS@`f_D>vMBZDG zt(#gF88KE>19mBO!*+V5*;Y8PwoBRu}n(WR%C9w!J| z`1I1@r6WEc2wv%6`^T-puz$IBwsyg%mE5%S(9(lTM_?F-qBnnFhUWe)XM?){w=()PaE}S>UJ4(c7cE z=0fdR#Q^&Z?Y}d zPt7XX_#j;yB~wi1xLJ^3qjwJnH-|madamDhaJ|FO;d#%Zc6YhXK`fe*NHx1#pXQdmJP~OHa@FcgXO|}S&MyHF@MiLrqLaI7EswdQ8c$yg2vzk7vsbk6E z+?K7G;VoOUTie>>w3L#}HY-k)@K%^E>f70b+X6dR!4_oXS04P?&16z4kp;C2Y*@33 zVE2LWL_-*QcxmX0o@%xRm=l%}UAV&p$%s5ExftFinPUjJI5cKkFmz`Dwm6p_?vcGEdHc#P z=U)!glWolp;@6y>zWVkvSI@oh_jlg-`#a~pe*B56PyG6}KZn*OKHLSSo0m&Qme!v9 zjAX1B5LYVDJy?GdT4cHS#TFaJ`@yL&m<6k`LuDAT4DF<*1v{LcF^69Fm zusWmJCyEli1#y7&Qx&~nYaa*3swSU-2Ua z;5j(A{Ylf@4=W{NXv5dvBBVQLF=ueEEO;?QGEw9fuT&V3qQK@GQpq$j>Y309^^~e; z@sE@{M_+=TsX?7e*Fpt|7SQ6D_B6HlerlQA$!I;6mwgrn@M!G=7^2TE4ud%d!})0Q z3dP`&+F7`UOylnnnCbIKnFkl}_R}!SNTPRk>EXzW=*LLN8+7#{+3Iu1cb#hKwT@H- z>`5?&UBAp1l)VRo>U;c(U22hiFz*hAYsC!N!F?cdgG~5Q=f?05;onurZtw(T6>ERF zQ`}5;3lCFSaVqJ#BPp{~O>f9iYj^-!Xa^vrgLM%?gLwXF|b-Ha175bcd)MH<}+DE zO(<$sRa8CKUi3YcQS=rR;LsM1Uj}8t2?~xUsZdEqq)`=iGJ`cydxSTOux7-m7vtad z>enbD99W~UaCk!UGN>ftXq0_ZcO_jH?i<^-ZKq>*c5K_WI<~!IcWm26$F^--C*L{0 z;M~>BHHysE+Mo1p%N=%x}vXFtH)hnE3 zNOecjR<|50{6g<540lbuNkow6=oP~fyPtfXgx2VT7XBS5%z>b;Ie)8^2vCmReO#Bj zxV4kY78m7y+jPcFhc7k!EukSFL=Pz^K*H}p%Y*uLptPioyy>|{fxTT{py#?Fjo1gr z>`%t5Daq*z={vQ^7|3?RIzhHnKsbE2032s}PdaRx=bD7UmZ`GLE?m+T=ka*+aWG;A z;UeR?wZ#7CR(k}u>3ILxn5ddb$05R%ZjcTn*-_bPglcPlig0ovqJ@&9N98nZrA?DT zEKC%g_E7_bn4fDHU$rx`eao6AgM*H?V}pO^K6*46YHg6D+?ZkJk(xo1sG*6q@;&jthjQP*#>_r$e>AO^_{ z#7+lu?z@`6#^`!?7$$Q3vGWh=?@*0rUmclP|G@E0D^Qa?jy_A!u!=B-0 zX5n)+Fg)vhc+llKWrtfFBkFI^YB5{BULNcF_{n%0tWCiNz}qX%pYF=ot)JAko2n4W z9g#+Pcb*0HAz$s-paG{KZ){Xlk)XHzP~|?XM8;0>kVviQ(suguaxl;|YcKyZ_POKu zAqXa7RY1u3n`e>GYBCDqPjlM zD~s-WuTQJn-k($Hc;v2j_6JEdxqm>uz9$=gpOZzJ=yeS>rL4Q$Iuls7EAJgAiMhP9@sC0boTvr+OEp`(to`W>>wC7>T|*l;ehEjl20 zEG|~Rot+P8w*kug$tbiZ#8Va!|22a@)#`{U{G>J{{-ZvXAu{$=<@Jvc+tF3Z%1VsM z1#fj848X(5bd^%AFFR4~;r@1r{*Z|h!sSfCT^p7vci?W?`75?EMky}8G<|KLlISS$ zURPae8jAJ6VcfF1;Py&l z6ITzhAE|vmF&&vbyAqvkIpx#(?T zLe1DvW{VKbK4udY_~mwbHtX7PF=2`I{sb+z&;~|Phb_Il9hS~i6U3~%6`X%LvLP3@ zY}?vc`ppsdL$BM3WqVl10wfS$L$Aks97!z)5>5z)2O`p7>nV#XRNaP#`cdkq%>+wr zMmXTVoQ((Dyzk&@OKrDxwabi`mQ;{zbpyzgUGnuuy!l^F=4dC~QPdk4ladlG`^*wa z3l&OVPlNOVNd1N$(VRW-TK+|^1dg9tvciEq&foD$V7WnKU7c}EwvA9a!=s4GD(%W- zu)v?vQ11FS8MuMc7IH+6H_i;1mbvs`84T6XGx#cO%1Bf?U3`^uW`C$ijG1Isolvhn zZ{pv6>&oZb511Uc!rR4|H-u%#!85OAFZm>0U?wJmdZ}0M4!^{SN)eZspnZC$=%cBb zpzkN#m<3ITza-SiWzn{m7rxh~R~!UF`793wKGNdFeS6|tH^HVj809~@7OP1wV>E?; zc!9C*j9Lk=L{IoH$7&qW>wQ}N=5{!98lx_IVL0!qKws$XPfDBmT_H6H^Y8qZxb0so zgvWUS&sh_w!=f$UuKgtDsg&JcZc`-37Pz5itCpF{i1N`pSF z*>Jh*M#T)Q1iF4CwECV)Q&bhA)xCmu`@S74GYNiAI$yiYSB5jD+UtE^E|=)XROi)e zRUf7ALsA{{u)6X5hkf{yqvL;p8?uMz@I8oT`^x?PP&f%2MWnI$YvyN-n#qJf#YJF} zZ`)@!@-#_VqkMBSmEc_UPEm|I;i{mkab`t1+QU=xS8;#szX=~I90v`Gq|btZy~6Yc zs`@G)us3Mgznv|oRM9>fPr>JZE92k|bm|rX0_fH+$V1dJ7_wFyCZ4cg!?c~_2Qh`g zV4d2?vf3wr2!AU--)`h+E?5sK8|KSKLDv|}gdyefX7ne)_Yu--QJ^ri2T{|HK%L3x z8E_fe2w5~=&FX@;>Dg@Tw*tFH^t>}At(59jHC{Y;FN`sxRFMrPpya;#{$m=T9bNqK z!AR1jzwA(m3Q~9lkJ~B4|6;B)&I4f5?VNA?xcoY{UFtGXp*qRYn(2WV2_boAXptGp z7%qhM0mX0(%cHc%BG%$=2*KY1kfHX;i!96<>2IZm64~+{yc%%j+dZk4YPjZ{Hp+*4 zbRnQ3@ctMgSE~X%qmwy_Lv}q0PPajtL=fJPkfyhwGyxr#sB*(us4e)*4KzBmsm9o)QQpF*vX`xMPjQ=k!AxNQ`&*k-V+^_p% z>d?K0kDbp}mb%FK|o(-3WyG*z~-fF)3VFD9iN_WM` z0o?Yt?aQB39pA^hj#PQwKL>a3kBjaC1>lk0|y^nnxbSKxGysc z2kl&r49p#ADQ)D`qlAgXv@Gm?HypidOGDWiP!FU*u^f_hs4zD)pRJrZ8{qtM4S!r0*uzDo0nKH!$1%y$#&;W_rFru@LxfV2jc+0 zl7??K3TRvP%Rmb}5mPgJ9X35nreBI6POQ#{x1BJiXRG*dttY z$3O4;V^ghjcdm!J$_=z|xzL@qDbW>At^ZXywkMTaq_lZqOanM7Q$umA1z{V92! zhDFghyUEu?U(!!cI|e!t{@L(;ox9QIYBN3hlJlb)f%_Aye`juD2r00tyC0f9)KCnw z+}vj$risW68VDT>BT6SfY5EXyA@cN=ZSa;R2*y@Fdd#nwZ$PqCqrEVE?miFObZaE; z@9dIWiy{8{;}&dO+{%h=anwR(hEU#dNzKlP8`sp_vf94g(?W7Cj;P zzT5WOQ9neF*VaPU?A}C$gkTM8ORaNDqgxxfS?`d=Mi+Zaqijk1qigbLOXW)+j}L=O zw3lm>hv2igxv&w3oktnLITnFRFXjU<)1SlQ1_?rVvXXe;+ z1javr=q7HwihCg_EA3ex3g1LRA($K2wWC$vCOCVE9@Yj7J7vV^1O%%N~Qj#=VVa9!M>^3(ZPP*XaWq?(@4 zI*Ic)dr&)$eZ;f(fp{q1hh$1#L|g8Ce%ppbC5Jkf@BtCATP0nD7y9 zkg$zbtBa-hoa}cPL)CFN8Txv}NpFdM}jnBvMnH+WTpkb-|Q)kCIJxNl}!4)gC0#E?(;n9Sb4+^j*x=P18+VC&($g_Bk{yCou2b$&dLgA zY}q|cqY-7#Gdk${DHqS&x>GnsHK)zEm`3lsTvAN_4nEAuwHu#r4(rCVkraKvV1*Zu zW=58eqt#ric=(po0pbew4AKL5w!&vOah^Qj)Jt=w!}??93R^6NjYZRU*}?Rr%FMP`{gx|_76krb}uZO*ASU}{qEp8OOz}Pk^0pIRyQDbp;ljg zd?-{M=Z&V%v9CWdjZC~2T$94Um30K8Ij;V*w|y#=ofnQgx0Mg9c}(Tc7&|GM7@);|h!`6)rUMoTI*_%?o$&!=b*+#+EH|@K=2gQ^c^)bC2p$ z34J=?7N0rwhLD8XIJ-{K8_=NwA`{Zt$JnHaWe)&f;x>W}C_?8D_|I%Q#}N&mvI!L{ zIS-B53->9HLTe(5+T;c}jtnm8v3@DzVFF`=f=Trfr|a)W8YPU^=OU}78ur^&Oc_r& zvc9qy$@)ygiMz$r+RS5o>0+t*LhvB^5=*IStCC*9-7=~DkhjQ5O5!}1NS4&GmATYB zqS}GZdXzZxG>H1Uv`SCnWvn_h@PQnf)(ztwR-pGWTHe*8$P0*_lIo7kYyE< zwG6k_l|1RuyqlPg*LD75dcQ@Ja2?wwCw8nyl=fU z9Ety8pY`D6`M{^oFpjo{0y8GFGI_r0;6uMZM4BRn|WQRbX@ST7U zPsi6yv<@g+o-v1(eXiI>ak@d{iM&C-;xGswASmWbViFPEFY|kOvR@~m1B9`ouZbjP z34}e+)nCWTXX{ad6{}cSYvR;sz!UY=d7YXLG#7rB4d(>0k_C%!%Hg!tWLu?Wv*E;O z><~R$Mx#-xIp%kbN1o~Yz&F?6>`lBV&nUoV`(JaTeR98>the8rAuIga?I+al9>2%* zU7{SG+s*WrF(i4y&#xl1gEeKWZr*#$*WDNGWj$Ws%TaCSq5yt*-Sr~1l6Woqw*MS#UzG-qU-myu%0YgbreJ4D2eD^6F z4$?_^d4}%p^k^0HvfIV0%IjaOMnS=ol<;|xr=XXPjqk6C$q zv&}guuO`>_2*3W|x3o!7r)7@5O2(q7o2ARNn(BIDr$*awo9(%@u@^ry0G;kfo?4h< z`fPsMGE?$`4Snq1zW2uD&B-mz4x>8$(4Oj*Hcust?X>fSFQ-8ZuQ2b2Tmp{Xl%IXN zMK5a84uoy4%Ku2uEJ-5IEQ<&eh{d!p-QxcGEy(p&@@u@Wj8MO{y1|;=DO|dYJaS@* zMIv-bv5Sv4q3wxV?Qb=Rf7&ycK~E$E{Ffs_$`iH2DUfbxYQNKl-gPDCH1 z2Q~}dmFDARn%66!_idOmDzK)(U20eZD3(RIF=z3ZXe_MTF#@@zY9hN zJ-C3CIuL#$M-{VPHO;L{J|OZuPH;x3+Q{0Lbk}g_{}vnKje&6bgkDCTZ(CkHK40$Y z^4V`kgGAr&8^-hV`UC>+i@WYWCXH!*evgw1vKsHl9qk_%$8!Z#+%`GNCceT&GD+8e zqLK8uHHi8jXE(XsJRj#Zu^(R*WO3fEFBaNu-{hQBXAG48CSqig0%)u>H>wBDAyrQ7 z&Qip1^gH^7xtKXQXUJm%ST26we)rO*S5CZD6$K+>CU1}^zo6WN!N@37I6x+*2Q2pE ztOy_a=%w#br-&zsnhb!8oYGo}z`Y!nHN8eDog-rhc=ra&aD^3#T-yOB9@|P~j+79U z!8$U=5E<_kIcBh16sH~esnXMP)JBULyh`jauoP|HQG2{ZJzs1*7VfZLd`ym~vPJh7 z91R_`)kC}HIotZO{&IxkYeIneQPnDPnq(qG2&D`n0-8dy%1b6^0eVx)P0P%Fxe(G1 zNrLiCsRoFC%J0Z^&2s%8x*DFHPb_7hFdNcYGR&%}gyP&)CrLAcW@W`r2!rh3jwcjJ z(i@7>(3%IGQ;(Ee85j?sQ`j;~nne6>f7e!vMm{~p=M4Q+i28bt&t7cIQyo{mdz<@) z;RgtgKYx#N+jYYuhpZv3WodPHy1<*mGIk&TcI4>*2h#-3j5C#;-0qfoaSHv~Wu8X= z1BgjWh@LW$O>9q8dPn&=QVJ5dg)6ChVN@?AAVUWW+ zf`3m#-^o6Ffg>w=sQWRnrdzS5`qInM*QXa9dfP3z-uL4U(&(wTa7;O@=8tm ztZP}%y5`t?Qm@0|@~(+(S)j|UI_l$0y#Q?}UD8A%#hjD{ka?PwfWe)ixZ}s5^mQFU z&y1%VBu5Jo`X*PqW+|r8Zi~VR#u861qHi`!+U(7 z%tkqVWb-lDzt84jt^)DubBT2DBX_6H?FR{f;&r|}YjR%hCSKMH6inScmTq;~WePK| z&Q2vSD}t}PYXz-Ing!(w!H}2Wi=s%mp~re2JPrg%&1!UfPCC88uNWXDT=6xDp#SA$ z^pl8@jKF1LxzAKBozHi$xpP)QYg$q=ek&CYO{b~lplg_uGr;|@Hh&&nY47}P!2Ua# z(_%vK-1 zafBuqEw%qHOU)Hgj>gvXxAzqZ>p`04Z+pt$JtF&L2S6bgs z1&|{%nk@k3LcwWSs}4Zqq`RAYW-Q_~CMbp)@IX?hU!c;?xMpTZR?+4IJF3||u7e}2 z8QA}JQJjP?QFm-ij>Gf%D3CvVr%?kP&u31#I%%<_X-c;`F$S>$2!42YPGn(6`&|~2ZymNkhZ3bo2flQ!X9q+v z#3szT?d7IdyvR#Ws#vTh9$bZp1CgjZAB<+u=h2Qtu<)iqChhJ^A- zQJpUP)oT6~MxYmk=EktLOwK5;bDS4M;x`Jsj8es(p!{XG8kiYvf}iMK*#@LOhdvc^ z=bV8b7CG9YNO728XESHb!}%Tu3B#+NzKEa-KlDG6IRSAgOn0UlIhMI#u{{ z067E_^!guPsd7A;h@X;Z$|ejNJ-jjvS)=^x({sV_ zu~^!~@v-2$=OCuBF_ZeJ2W4fZ6j^fg9Qysj%>b}Nb(Kn#Xr4J3Gk$Ovxnt<2@JaZn zghphu&{XsB#8F1ITA09IgY-@ZEUXEV(*O8$A#0FBV}xpHQi_>>2d^XfuTb{51W%qp zkn%_nmL)8Zk;-#P!~O_kD!9Q>d9o`S_;@@C&o$ckcnIBCC)`{>7wUBO=;1EFu`L{F zFn=_q_u(N@(TYzv#wvBaTx$DrxJU}=O%A@hFa0BMB|Fc*0#E5s6_)#vuzr>Vbzh(4L*fKMZ~EyEHWWxP_e+_58UflFX+k{+jywB)ntfZp)B8P0w57V!`@ZN| zZI*ZBX-Tqyn&sBQk44v73U`;LmVIr6# z;z}YpaFHSDeH0D!ir^mN#5#l(@+KSgW$Sj--A9$+>Zs9L`drn?>!}oZ(7PCI`;e3I ze~o|+6cbO_{Ksg({R)k%8v_&&GL`-O(1nl@7FR7Q4gC1kaMzePpgomQ)BgHSXTKrv z2eH8e3(2R|M6Gb-f1yxAi>_mxh2XQ-Ww|_jBfmjzE;1Kg)T31X=(a+fS4} z7t|H^w%f*zcR~^>EH$MM{XaNa5JRl{)&aiqLcbq5n50u1H?3Om?{$k$)#6{s;dm1kQcm%aKA}Jk14H2?M6BVxhrL}UOeDWnZZBWQr&2g4 zM^neAyUFM2ZNAU-CAcL1wGL;;r|h8tqSpOC|2{vp!7qFN<~WW(iuw!_++6HkR0Pm) zogKi%jvy=ZpHi1kd!rL)lMXc-z)iDmiL)7-C^p3kuC!=z*Dg%m(x?Nv(PDX zJbWt_pC&WbJ(v{Qj?X^J9u?9{9Lsyd|1rP9(8ubBVpZ&XVwM%?lRX-27~Vv~%F*b0ncF3k1}#_N z0VB7GV*xHtQx{&+*!wJ3E>R>YtHChvVa*W-GcR@QlCde!dl!x8ATnYbl|eHs94s&~ zVX3fblfo2EV;u;zEf|^YBxcNh91EG4ts0koyp~N5G>n{~`nXb?cTA*3MoB&-H z$e0SKOEOiyQF2&Wz8Xmv;MO2*ZmyCnxJk>Q))GI_r(VJD1Ot1Bydy3!ZmN(puv7j8L4Hmri#K$pus)aRXqdjT zi(xNZYcxpo;4X%^LdOx$)Z;!dbL{5!iC~Iw4gaa2{1K7Gv~9H#c)&Sslj}PfGb^`V zQ_{pD&WKb?+E~dFmpW(I{%0TRU`f*YJ0^}SoU}XWsBNll+aC~$k5`xVJ>oXkb~?5? z06&uRB%;iuf4auu4iN(vl)fI8E@117iLal3hl6oPP6z8U=PnmZxg3ysGucB{=`UO7 z@7^&Y%R+vUB~D1Ns=M?Tc49)AD{~>H3)*-d&9=z<@~}940G9PZ5gCkJYQV{g7Psd5 zrm(n@3aQ7GtIh7%F)iNA=uh85MIx$oJg~HDJt{R=gjnNCBk=43$Q4xN zA(O9Rpp=e%vPrd2xWJULSz_ICbf&Fau!N;dMBt7_Njr;Igja1_6?FxH_KOZjSn^#8b-n zxdF0{bxw2rM`FAdYrcCCDMr|}sll-@EA3OFgPkmFfscH9md<+swY$uU!hk7a@x#X* zm^k`Wo<@TwS9e;dr}6}l_xU^nW{6@DCI$uLK*C!8Nbxubyw;HH&{E}DkCVFtiPmUl zEur@Ney2@w2cMBEz0U59_>K|qTcblsCTzM_^b+z8m5=_Eag$9hiI(uxR;(qNVxIR} z8bl)vzQ`mMD=qORv_x+u){)1va92@af{Ofan?-Zo#864oTU^i(%3O-K)MZTXt%Gv1$Bp5 zEFb}wehD(t;?Ngs)-nSFkGgO%9LktxPv15QaC-om;;?aTGW38c*C)nMEuxLuIr4}9 z%u}cMXLdXrE;4!X6=TSJs?WrWE{2#7AyCrcBz=S^7Nilk&PUaDL}C-iP3!wFlTOw) z5+S+l^dR%ZRGCys$2O?PqY=)72kpV2i)DAoK}@kw(&ZfFE5@K4ZvK(dEZ;i(xnzM^ z3|Ua?d!JDv>xwk5IgiQqKT6HK%s*-A`@J)Lx)U8v6Qk%28w+%eGqmf?M_$Wn>@-@+ zLuiE|GyVW7lx?qMh2z*Yg=0jt%~?F;ZTKzO9NE==;`NzJL3dV<(O-nA2jm}v?0_iG zG*arF{7`D|+YEna-qut!xUL%o)#x=J1jION$fl-C@TDEFY02|dT{n_0H_pdTE$uyn%MAT z9bY9+G;-ihV9ZD@0BhB9gY5b}8iM@`jpc;M63M8R}e(&{_Us@O+V*iZh;ehs=QV~?U9k8iu@Oi5F^ zf*zkC4s*_Vc<>-`Ec9NLhN~^V@D~e%adB5~i8{;{-`6H%8x1cpOjRVuA#{Od)T1WxaJK;#(owymj3|wo_dFukPM2;-E`)9tJ97-H&CCj56 zA9+pql0i?lPhY}TfYHhPmTHS-%s)JgoZlAf%|R+|wU?@$K%P@i1D?~5=(kTVMido_E(fBDI_sv? z7N&TIQ+ZP^bRn zx~+@^*`mf!>FV8A0+stkV#6rk-WP2rGvymTr-g7!j}~E-oe6ovvE^|$ z3?NH?VDv6$?c< z13l?H_Rn!3Pni(%G^y#0+r+^~md7$t2kqN-Uy*b4V^ugn2`_M)06$Pk6k3V`m51$A zU=Eqj6Yr;lnZ0g9H%6M}&$($-OuApk2mY)2v3;r>p6o=g0-92OakCAb)1YqlYHgb= z0k9-fGp_8XXu^bF>xZ>Vi{V*$_41OJlU(4m7?PUXNX3tzafQ<&E-N~#$V(Pb9fTmK6YxM zaZhUZDD+&+NoTqKK{_Q;J|Vc`LbmZkmWp1DV#y8rLyhBoR^4lh?P)i1vmK<>gq5D} zKCcC~c1QfCpz}%*Ccbl}mDY@a!xYm-11DXzqOdS1k-~tsFX}g|}#!NgkQ!BoUla0DjyA(r8>W1|l*95Hg zQKxa=S_rF@r868QhX)tLO!X6zq^L!$FprFhH!R{%wykJk_=1s ztrTavCYO;ce~@Z2MdvQD2x+Z?ydcn{&gT^8t!qq?SjBj#Q0@0tEOyj8Y_5iIX~*rS z8$xu>d9MBz6lH$fG?i~WeG#bzmkHK~|ERYjBny@0$bplw#cwlvP#XVSUV>ZVRyU<} zd8sJV`sUGa(jccUE<0Q@0H;Qzt&y#FLA2y|55BkSZxo^a_r}h1V z*;NLsIhwUR^fCp{j!H?EqBROX|F^=iOEUWn9>2XXHYodC?$3e6I4mT>&ew;cOnJS| z&&9>e>F$nGuIyx!wm%mKlk##~Jf62B`|^;K!v&w&192CMMtVu37M+P3=Y;R%5wM^Y zFl$3gH~*31eicjNK!_}}4XhFO`NCNCViaVg#8ieSFT#Gej|jYSQI2t#PR;-Nh8?-A z@qPY06x{T_OV2D};?$;rCy}^q6NJ9nMt<|)InMT;-hfJbD{Q8@3u=}^?Uw`Ei9!1A z9DwHefHqIg%%L|)+eHs__US)3+Iyl_9kdG+J4;D8+j-M5LrA0p+V9hH>1%bz2y-xb z9t&ID{ynyE{h9ofjgU-`JfUAsL9j!(?euT6Ayug0eT1o6vpwk$vDqDrDS#^x0^Ra( z+WqJ+3yH&;+VH5aAcC&%(y?uF0QnMlyu9Z^WKfUwsdK=$3x?$^^OaQsd5q%{WDlIq`~R%Q3?tUByy( zWC3wCtuWXy$_Mw8B};xb493X9#lloOVZGDWx5u{m`IvHv3hKyudwv$&gJ3uwCYhh#MZfeS+)0bb`dolzybs=LTgt%^+NKfE)zb{o= z9_ID-aG=jzch+c3Oh@}?@R4Mx%2o@(xM}W0$xK%rSHrG07lp*DVY zb^9Ijv2Ix)VIS|k2v;H97aCYFPWW*C17esGOKTxGb2{LNk!f4wRL%j}myk(E_yJTu zpIc0=P^=@#bF3@;@bYGTnEN+a`D{OozN%7x%k*7>X3J3UwVvS_j{5ON+dUE7#;^-? ztVCkYgXZ%f)h!1Nq2++A2fU=25*+p~Ie-DS$^hF159mA+4bQ<3 zv5;)78zeH#s(iNmNMB?&%m5q%r<7hS@nAzVswl)CTq>3mQ8N0Afsr>=3Y;#Ocp+Mc zkqYB$QuG!OOUvGRAg_?x31pi)Pb*$mRfqeQQ@u~xDJ6?K?W~Eb_Lby;+*TY6zHXoie=lZ z$Du!;ZDiIAU8!vVcBUO1ehEWR!ZOPQ?MesA^kyV;@~c*$8njVNz>7e(M$qo-Ohd`< zDH)-(Ndv^%JqsLWDh3O5VSXiENIm`g)b)XpJwH3Yl2$2Ak5=!e4fkH5L)mrnPDcO+JoCX_bopk>qL`Vd#?d^c0_ zLx)9p*+z}&9fHzh&X95c$;}ukEbS3DTYb>@3M2Y< z+iy2;o^M@8&ZoQAOCK@QJ`Ep9L3BoepeF5!!7}7VZ(S04i_xtrhY!kJ9 zBMq|!VSTt3byeVfn;LX;YhLXG8+Q-kJ(XAMRmQLUH4odXOSgAn-MeET3n+a z#5Br3B*3SuHR^g$y)58@746o{m0JmJ+`(?2M%q zgErkN-9y{`2cvCBhJb;&;vwC>e9n$Z})o(9_1TZ`j3$W&Q%W?2&*ql4oDy6evc zVFNoONJFt;^>z$OEp&f9(ZlX%^!514HTp4rU(>hCdq1-C(Dk}J`k{T#)IaUW{l};K zRI^KuX^ngtu3&nYBjvBss3SwHui|BG_5#y0GhFet*q^b6e)DQ;j_o0+R- z`Ww+#4ubNLDOmP?sgSu0>P_JH>8^Y0$b~96GE_oY)>=zKY~tCh>}4t*5;mnP(;3?EX|mZlbLd zE8+L)7}@0D-G65i!c|p_*{dQq4O7cC1J24uFq>&!Ow2U60epDwBpd~cVip4=A;q+O z?^r)X*Qp`BsbryG9|ZP(f5hGCD_#TKu^k^?+J-7L;1+g6bc1R|6yK1;IKl1m@?ZP_ zIGggs1ZP#9Dt;Z#{w*bY^IKad{0eJE!Te7WvVVV4#?U(^so!87eK!hmOU6iQ$ty!> z92sjtW#}+SXB@WEuMtwwlB}bCO%TPzM6LZ7yC9@@lR8DUT1`T7S`dn0|3E5x14%%H zdb?fw$QN~`|6g^Wz2A3O5@giPj^XA(3=Ay|Y-#3YVVDyn8bv!B1ALC19;h;Y-#h6g z!)ONtOSTz-RsT%Tcz4^UD|j@n@)Ybxw3KxV(HXG)S{3r$EwlgZrBu*xTn!7gv6Qp--@)c5`Q2nTh&^&X`D+($tZ61kKadBV(aa5Rol<|J&a3m zix`#RIvL?;R=elx<7igh-sdq)I~U@bgU1Z>eh9t6<{F)Wt_G#5zv<_aH<6UQy-~cc zGP*qPA9aetC?#xv_9mnMvQUzeD0fYB?7{qg`W8OQp>crANhg=Y!r5;g8*K%|wCyJu zGrcx4!-v@zLrACWeX}G3ZOtU;DGwLNpja@*>Z6ExzS8|qCX?^ewX5{EsNq$xiJuG+ zLuDe81O=$Y-B~uPAa&vnOC<_TI&4e+oO65 zPm$WS9BnEjgFVj+HSuc#Ar^_{_m=4H2-(B~O3LKshS;gCEl$|^h^TPy&uqGrIaPs1 zQbC+=RjH1TrReZ?AZ&gNtI(> z53+QFzdqLw=UpNUo-6Yh*g1xs{5l?}gkApcnr=eR^<14m2K2s43X`bVo^Q%R!%7fM zocL+@I4~UYyQ|HJN98!Q5|-Kg_Qv{t1Y^qOguM0~BPMSF~AGB$cawt0&4T({LgEGNWDR_xZfJ3unA|Q=evlW9$3u4pYwR3EaDjKEL6m(CTw4)2 z>nc(CRnnt(;|=gPTh#X-mx-}#ne)2ua#KNkz!kqVW6pc>0S=`k=5{-XiF#7uZ%tQu zZ&X3XGOZJ8t?{J|cx9j*RzudQC)>Fl`(VAL!g{Vhz@zIBbzaY^p^`aD*VrX79ddY1 zfycD1@6;#PT!A8SQ`jCgJu4~y3wVRu&Ya?NvFV1GlT;FL!&&DXO@4vDDmtrCz=PBK zi!#q9u?w{O@h2Pw_Ws#6GX^Ut67LefcF2$7cHN)|ICUZJ|9wx0Wg(amfZ5|*ft*HZ7(NM8F`8>VN7d0#+^;M}$W9>FI2#w8xN;{XkKmL^8(u_>@Z!(UcDJi8KgwtC7 zn}}os$J}nYo~^4+eopG9&+hPr=FUi(N@na$#i!M6`@;X|rbl_{hbxkP8OqEVQfEw> zlV+}ilE#9bVwxb5xm`u(tTToa4T7RMrey*)PbeAyuU09CcDf>LNYy zs9r{ntjbj#>{jfK$DCjpN~4^jOr19I*tkOzi+%#mrNBDz2ry%hm%u~EIo(%lzn~)} zUU0PwIJ@%LNjUG>I*hG@N^%z<0abQFSIJs{W&52>ceQ#KAulytq!E*6z5EekHurPk zmKaboN5|RmH954?g!sS=nk5}_%JjjWlohBZBF`Rvnr<)r700K-(}|_(hfbV z%YVHb9k=-U9W7BEvk>2XsuOMVe3lRic)mWRvw7U@znk0V{_?pGdDfO$Zhb6?v+ue; zo1_wKe_e_)S-BV7e-LAtY1T59O0IC)5fI<`E9E7-j zMW$U2UF`a`5JzTV7sX4^Ogf;q&xsLk-;8lv$Jl(ACku9Z>Ku8nwD60D5yVCpVbfdf zpJ(fWS-)NeGKs!>=$JlkPWy;*1@8vfIYdm; zTi%CAEqY!8%!=p~v~BY=V=VGv^hlDa_W9&R8VC>9OCVvJwjZyugQ1HEwbAq)fScL~ zM)B$b8KOLav082~PsgXK{{z!NEWgFs*~QtTi-$g3JbZNV@N->6xq0xp#X}!29{yr+ z_9OSzXBH2?xNqOhqbGlO`NZvmPyYSjZ@<6t#_f~mx|B#{djEd+jnDpg=r=cia~VJL z@Y$Qs|9+E8yxPH3sJqHQw|hqzJgsag+tXaRPemdL#Z&g?p82G`A|>M09>yhk`4 zYxdnAe)CBaH&-F3kUL)2rEIRG+BO1JAm~a*T|K zBGt(w1s0+e=m0Y_l%PhES&@<%^Q2_@RwWcoipHv`WB9G8-?y(46B=O5AQs$aOi~-8 zYcr2AbBZ6ZY?2Ye+`pd~l$xLk?iQ2FoyuX3x=4>>8mJ?V!np~X-z58|4f8npDgh3x ztlu5caJ>=j#t{Xk@-Om+*MqKAvX?ZU*-HroVyl&hTUmS0a+B(r(B6ke=uv(c;|`k8bXu4hJO>8DR4Z5YH=^AKb&^P6CU%xV;%V46EW$fI)t@4-dzvNj0^XHSJ-c| z?19*K>oLHBbJb;w3+kxIUayl>E2_u&M34MRvP8 zTyKMcT55Lm^zG4BgKe=#j;WM)E zDGcJ=joBMVJK+i*KHIr)3MT8rrbM1q1-tLIRoRLvYtl(3bY?WKNefUVDkZmISNLJ4 z(9{kd3B+nh)W8_t{}bN}3OGiP89cwy)F85oZm7`x-?xGeU!b7ZT9})yH>{CVMLmf6 z)(^CZ6w#{9?faiq23d+4b}n4=l*aIgCQHd=wH?)N9J`Lju}99`dG%^wiZ-rYYyxY$ z)$q14&B7y^I7Y*E+cevqCI_n^Pg~2K)SXJropeFaJN3B}R%fp;#&sG(6&t2mZqR9F z3J+ldgGeW^raPv#)niX(P&xbLv%@7s6t+_@iK zKKsL4@7((6y&q>m;4L1yi0K=%rxs@qE*?IHNgRjIEY2QnsVL*}sQyqMRbWCb9{O_e z(4Q6${d)26ON)n}T|9Ke;~uIIP4Q1O%y??I##@o$tfzQJbb^4J45@Ojla-QKW~bC> z+d<3hO7E2ZSbC%M+tMG)<#v}!XRjaj7TbMUy0mI6rSvY`as~cFFH z!{;$iXRUZeRK92~^M%>l zN1wj+@>jP$x!ih0xb^9&TTdRndFf(c5{$43wvIFtnE3>#RMn0l-);Jag*DC0X<-Or zo9yd1|NL}|&HMl}7#ZdHyf7@Xjp`{UA6yi5o3>-7M2dL~LnT@9(k1;8Ly6RZYHn$N z(8IgrL?qT)+f(U}h^>T9pX)Ec?0s4KLQBQGZ1@=C3`L?Q@Q^{72}EMs#konyo?GGvhPJhk)in9OK7 zBW*Wu7UCy4Hjc+6L*-!@G@T=3IR>K+PN*DwKk1vhGVO8c28T`qN~s|8+ab=CJ`G)< zpEW0Rk!n_KMu|zWnlNWOTRt>>6S5j1gVfA0(+a_Fjmne59m5jQd1cszr3cK7vKAOZxpd=^ak`i0jjuw?+M{E0jE1A+d%@h&q zxp@_`Nv!!()%D8&qWSjLM0rIiT{{3B~+euGg5h_Vh z3&iT*pL+4`rOxuD`^raV(j3h%v!r7HIUD3C3&VghDN1khmmbQR^oWCk5<2p7 zx87E9>us&I3U0@8)XthhOkxB)Juy>tzJEXA8TeE_&f)W@MQmo-+b4k~VZXET-W-C_ zF?hHc_jju?WOoW?Xg6bI2=gP&;GR0m9MQaVub)1kteHN5n`*L}6Mg&abl$)! z=X;~B{KCm|GDK|)Sgja5s59zJbB0s3K88~xQnaQe-d5B=yq20xd0K?*KC*5bO;w~w zjZWy>MuRP>8|DB^E@5)Q2qG(&1zF2&QQej$KyXjPTr<*3^)we`2yvnfO18;KCDAD- z)ktvNVGl1*%hQ3-8{P4(B&wKO-wscE!qXX4$t)3GcV92xpeh%-&7eO`eNarHia}f* zed*+(W>d1U+Bw`*B-^NIbE{DSY!8g0Z(=5-NDb<4E`Y0d{^j{M=TFSPOy*yiKQ;eb zIDCEnZ7lqs|05iI09SuUI?4QtaPZRnTLkWS6|TQT=HG(b-+`;|;u{GZ{?w?Y9;^U!az7h47No#vp0 zHe1IIkr(iG=`7}@mp;eThrs0M5feO3cX@~ruU`=vU!x>4iG5C^HK2sPjem^nvT0!u zwaZPf@04S$M+gk(QX(>GZcNObpCT~79L!NNI-Q<&%p%cgN|cyr>o&4XCw**E@Jtukfk96(?uI)1 z4VEtkCk)l;B=zhjmCeNeUXrZSs&w3Mdd!WZ-#))^Zs9om{&ZDusfIAjk{p*MC6b(3 zaaIjaxboJ86APyoKEy1lg%dZPCJTRD_;}&mx6j{riY%O3xb*Gw_}$+=PyTumUMzn~ zS08&E=r&XS1D}qFO$#5v?3==B z^zcV-9D(WhWZ^hjqR08b%W1DL^$h8C3MdFYxY?b_x>7m0>avm}SjY(*?K;#~cq# zmGY-lNTMYxB@phhPqM_NsNBX>_M}%hnI?kJAw`l>JRaKYbzQT)gZ;g`_U_!Z6I26c znxIc2p?;<$Br--*I>S?m^a-+}IY9rO9XtPb|L9DX8O)@*>(aI06EN+<_5H$@$;XMo z7>r0FovTxxY?EbGbK?%1n@d-&fnB&8kr8^JNXc&cc%GvY8+>Q$$xh|C?J3XbRO+W) zC~K|*S(elq-C}EVriC-nyu>FYiXtUzI@7zO!h7xw!6-HV>iqBF%`ht-O|YVED8AJVS(Rcn zA>v?(@W?zc9b^1@M{VYXy$G5KLCv<%j{O50b1!58n`z5n3yPvEI0Lda=Qh4YDzq=2A5=&52;Wk9;=8l z_t+<8l9C#0S0Ge2m^|esYtZfzJmLWq!Nz!6^_CwZ!}jpZv{d$2v0HG!@|&NC*nF>Q73;v_SJ$D;Z<>_}v&XnKqME6SJVTlJmat+*Y8)n_5kPEX8Bjrw>fVG7{$~tde<0Pb(DycQh z6TH%5^*Re{`uPo15l-$>IGo2D5-Iy?UtnK48@WiqnbfB!;$^{e6T zf&M++L;X)YL1x^3yT%%~FW>SXHqFzybE5t|Ix8|GLnTT2&&wNRT>C;?4!&l#sgbDq zAbE^iIV1S884(!lEBD@5-e*DD-}{K`)*RgJCYz1{!#AJe*F%B2mq|Fyfw-;X4yYO-M=&IDC*m zSr1R?Tx7Ak^C|!}n~OdxnRP@Un1>)^+~C1+6?Wyod?{jJY@H4>HCd zH83(WFoNbep7cI4)U{(|W(U_Xn-GWZR!Q#LhgZ6KDYHO0b&AJ%hIKv;c712GuX8f9 zS1K6U{rlY-$~A8t-LZ6M<-2|c#wV!UmghTBn5Rt+YaYD>Ka*WV726G|3X`MBsuQQ6 z3kur$0~T>p9j(ppjIC;>ICpY-d4dA3pFyVQiv&@Z*QW_-5h+&F*VT8Pd3*UZEkAbQ z{K6;SK9BkH$NzeA;rzGHyD9QbT12vxjD{{&fSusRlQ#~#y_=sVHx4bFyYcjmr~Z28 z##2}yfa(9o%he1wo@(|k;e$%$;s}BT0PxHw7ChLSH7|ATG;zA@z=W&3Z#m^CQNamu zkNs^P8rVTK&^(r3IO{g|H;wyy*)%@@MuM2Inz?zXQdAC|`Rm3Crl@Hk!BFpaj5N9n zOebygN#=$j$}b_V>xP3fZIJW#ZbYPGMRrv=NTMhaNztN`90Q}+ZA8`ptfpk7qDQ*N zK4n3yPVi_8dV?#G*DM&~md$j%iAf_`92Q75iv`NCO6TT2Gg78wARIE-(I6Pf&s)sa zT28hyNe{(xhl%{$F-&gd1{s^CYge3|}hOvb@nt%?uX>1{zjK*Lj0DBnDRT*4q}wXqrw>vv3ed4Vw9+ zB!^dJTXyDqsfDRIAwe!1Gxc&rO4P)Q?*FOpag2uPLE+{dQ~s-kjqlcQa8%IePTF%P zTg)&RHeo(UK{zJSAHZN)i1wmGTxVIwlejD=z3`z)p8%pr1XYUF_(8X}32ji|Xq`9B zf|jvmpOR=3rBXEp-#7^~o(>CAxc;(a8H+4q;gdk_Bcgay4-p>BXih1NjK!6aSm>Iu zmfg5}@v&K+HP=*7B2qjST4CKfIZ-CfuLs3_{RvSr)>f)Dh)olIOU8cs^l5_eu%A9V z^3(gTb^Z4%Z~fPW)8xNjKJwFhZ;?B%T=}mHACdn)`q7=Eul@btZ}#uM^Y)Q#KfQnD z!OA;!^*lm;`rxDg`uyV3ef#%syYtqS2i^C-_VIsxaj>PRd*|&>@4RvH&e@LwQ}oU& zNA7%Z^v>arU`oAFtojxI^4c_2fu=Q3l>y-S`#8vP5PeuP(Srirm^)5a>A5vL)=@=^ zj$vWi4(tLQRTODJ=5AntSjMc(P>U2`H%jvY1vPG*!9|NOVv0QK5Qub4JWm~rbO|ue zH3dv`_>n!Z0I~SPh>4_-e#LbWWWc zWlT|Tjl;VWB7?~OCdFP;d}4y<-8;#Fxnugos*OK8`fD@h`z>1-o21!Yd0a$R$Le(L z>4?TVR??_rkeUaaJjI7!qEs>>$?&77E_YRkrFgPKd?ts;Sc|)+kb= znU+&jmg14nITvm#nUs^q@}Nfx88u|L?T#oC)5Vq4)3pkpP5;ZVf!BGa} zrLIf^J7%&%WXL$>b`J4BGu&RQ$zw7hDbZM2&xyomR-VH9?{T8wMv2Zr__)ACUZr?l@X6Ov6xlH zV%B6bbfc4>6G~FBH~rjX<1NNGqda44EZKoQc~ifTX}<#JHp19$&55}@k!E~zu(zP ziflxyv%nC=GS=J4;ghWHAKlCwJvY-io&x#QEE`tC7SHRch96~lK90p{tMt|B7#^$* z9DzZ|3k7bt$u-UQV2|eL1S>vZk;giBGn#c6krg`!L&F+9(u<#)^KaoFY~%bMwxYN-rw%)c~$!sGqX>VuhSYk-+4SXQOTHL#Ka zb~6vQPmY@zA_@qT;+7y93_D?_TcTU!n$^3#IclP~7@{@^ld0fBbCBEV92w+kS4MTG z(X=9?6h77y<4~iOK7a19AD&aNJ}XkABewID)|IF9-)|lz+m=Ru=g7M-ss{tpym5I; z3U&|F!uvCTOM7v}2}I*vMd#c(QloPs_Qa-8HitNHOSw6wQnsjqQ6dTIpByRwM7)54 ziTeKX#2Wr`1*^RjwL1R0s_xP;20U)qqN9o+0|-SyESqBEp6+ax!_1?aGA_(KorZDn zx>Tx7DccDWs<;7U3Z73UU_jg;143l$xh^`!x;s`9sbdNHkyGrmS(&Ki>@h^Gm7-G8 z6S4R7J@(o57%3K);>jB1tktxVjzP-DX-Ab}vZRR2zfR!SEAwy8|Di5@cGuF!GAY{> z34id|sCK+whctsVq*UlC0y_nD&cC1Kjid#ml!p zIQ0FcOFukwve{5U!R6TxpB}mO*6ht=$G^Y&)~%ywZy$g0_A{T{Jo@VG_f7UK%))P+0|CeO`%tRPI- z#F#-wc(&%{I-fP}7#W&@Yw#91nUvuzo;NrKpF6)A?t4dXt-{hPFr#^-T=3*y-1;}>~mE`rFf0uw`zuN z9T_}C&Ac#qf$G!-rYDx;7zVd(raR;SB_^GKTM?vdwJhDuETUHqj%*ubx`n~@;AX(X zpyJVT?(W?*jrv&{Yf?*-Vm0U|reE++1mSq4^vU%nOJ}biEM2^Q==x#!{Q@a{Lg4&t z=_>rYaQ(^3Yj~{;cu~3lcfku6N>||LYUyn2(X!sq1C#MFYH^((=<%BCagVv~6HHgZ zf=Eqkr`veaZO|vuoM4WE#gIx3mcQ>d3<|nlHa$1E9y(kr6SgP7S-IHNJ6$AGoLSS` z;YvYF7t;rVM{7nwQJU~{eGeR_QQYFl8e#-73vQn{ee3sETMlCA=9%Yjoj?2I;o#c; zUSJ(ioFX|-CtObwpyFF9DKcmS?UsdJsRR!<&BB*}tWz*4pq{lmdsc-kd8Ro5O9fkc z#u9DJOb>tP)JVIH@S4RSaVa5(c2#Vhbj&=14>hc?Z6(*AjR~YmO3CEbYr@k@)56M! zu*2`bqK5=;Q(1$nA99cZ{=Hj6b77rfY zAjWBw+j4RCH;c1Z7H5xED0lZ#SJh|(ld?~k6kCxMbZ;X|IUe`GoVq7I>8VMHHkwqG zR&0o7CBrIM(3w=K6BD)|K|FdfbJC(f3xy`>#ZgeU!sBFbA-$NUb?Ss?oJJ_dsw^dI zQc1Qp_wW=mZJGg#B_!zix*14~wuw@+$JjZ?Y}Ov5!loaqj+pp1Vn%t^?K`t{rCe~1 zshwRz2Q0YPv1XPJ_U|X1%prsN4ddPXyQsd>G3f8PWn;fQdAC&CIGhR9y*DYf6ScZX@>d5d7NZovMKBVMd&zo z&NNvkCOWvK0Kzd_uJCMdXq0q2CaBG18)(My-~s1fOqjIbQMN1L0N>F7H32%rdXmCz zQ9uHXutLtHt~AtvD6K{cVOp$nTn}DD+dGxdaWH)BoHy#P)erk?Yf(ar)M@%@_)YHq zxz3D<#&N(s35M-(^bD+h3kPA2J6SzA2kfB+^@bunGq3nIHQh45Y945()vqJ1{>|Us zdG77ywE8=5Uu}R%i%ao# z!vjxZun8Y)iYaTNiK)R+w|cLQ-L%4iP|VZt>Due!5+H{xC2Q&$eCy3Fh)M5YfA6ln zJ9q6wfHV{A0D@HdnUavm7)|L6PbJbPaL2i8^9}Ux*|GC~_m9qGnZZmjlHev69fcYh z;#NyL1;WRP!5EB40xNe_vV+wjt6Jp27Sq)c%$kUCEZ5Yruqx$n9gRQ>E0Tuk1Y%tW zFO2TVPUXO~^z2$ut-cly*VkMaQAty3Fhh$z7sfY7zIpGPi~slVH=liTlKh`5-@O0L zr{7%s<}(6U&%m|!{_nwW&a@c4{?EB@u72~?H}AuvU;O67Z;rt*@c2XWe-FW^KKtej z*`{c*WJCtDDISSLg6oQp(Jbwxd9TKy)YlW?`kJ&ut)p5|Ts8H&V@%|&iH<6JSR>aD zl`fP%FMU~BB|WiYUm@^aUb@8-_A=YRU(?49EWSE*d@kmYL0w>&+|bze$Ul9MXRJIw#v$-oSg5+FE{Uvh6W$!pi3?xOLRW^BLV^*oFv^M9al<)zT|SnD7_$ z0jdjw!2b`kDQ@Kmd)ziz9<23MaB4OeS;ki2Md4XjpOdGAnwT_AZJVYif=e^qu?5Vv zw@*o>(9|z&u$E4(cGw1~rTgKTFK!-v|A#YAE3H6@DsK>j?VaR!HWzsIM;R~D0=(by zCah`|sipaBs}#isf=aXv-dgdQTj`v^cm_imP3)``#V&wKjfEMAH(ow*>*!}U&wX_3 z;PF=A4or8d(8EoB>ZUZ2Q02f>sJw4^Lyy(n&|_;7AXX=|lv3Q2U0?W9#KRbK^+f zM_f{q)#RW7{5?0GyfM3Q{KiuYr*1q=ZXCMtEP=Bl3t!;x8&8%`-d{KmzmLPM&#nfE z`>!>@hMgw;<)(-)EPR2(x^c)I%Z*t$#IZbe2C8*U#A>?1>ABn8-V$*|C|lBSOpsA=O9{CfxK9vR!& zySu*?HnBS1*R_WicvV$nVJxh6Ew`9fzjv@AakVbWbQ)0U%dAHeW< z##<+sggwr6!80su7`zcmxSFV=gs1(&9E0TS`}3OzFutdAV*(7N*3uY;zCLq+q(Emd z$)8`CdoSfe#s?$C$S|9rU%&2Q84t*Y@1UNVl#(@M2!u5I-u2&Q~A$r>j#}*O_t*R(^Biv>9x}5NSh`6-?P_WAlHwSE|xA|e;x++W$8S=gje8&7SIN2 z$c6c+S4rs&7~dx@bn&N{oEkhJ2zyu`%T}mW*@t{@Lz*fjZT+}>lt&Mo!-%Lp>X?Hox_BZ~+DGz~Ab7QuR| z9z7t5{&(kqV>(`^Lj5Whd5t0#6^qRhiO|jcfIjmt&!3!seg2*KSKPmEllkAxzXU%m z*utg!+`(nnG#xj&w>fhqwMX%p+oNK~w?wo$h7g2N`X6_@Vd}1`+!C2qRS9q`Lv~Ow zS%Q~z?aDDDQ_OPH6JXOS!bD@!dU^*wc#plikr`Fwh@oe`&@i&1~nv{&RLFy_o zHyfrQE7pUtwSI$N#kOp*nzn3Pnk;#thjd;yh$0Boyq-3b-IfdVgMeXJkPCX)ZVG?x zoC&HKd;kc&fA82y_QHo#GQe!&HVt&OvtJ13-#bW-TEt{Dox`@yY>E^Zg$+s=1SyTE zR%k!imQ*iZH_%fPuY1(sFlw?3-kcHnO%b^pg{_x~LQWVw4bmMw^jVV@a=}rCVu}=% zB}G;xWj*cDRktuGUafeyp>Eazb@R?gKYef|FhLs!Z=;qgi>h4?0c-N^y!*<3eGW8u z?yWl?Jm0R!`v(~N&C@XZV`G#^IUz-B3MvA5 zCzf`wN`&GeViyadwLB4HIYvfvG$W?gw|Nax(=3u+Y7|`~ZarFyA_E_WZk5xr;nf}| zJ>e8M(?b^OHzLBSVxOXjR$~SSV2hqrHkR&MJdAD!Gqd$g0*SEfW)IVq{IyLcLv%O1Xv12Bdv#p2bp@txM_!Vz`%kIwx4so(D3zi;2o zPu~0S>EHbD);qU8xx9b>;-QO+hdy4MJ+(M{aPjc5#o3P*51%PF5on!^?d2i;0frQq zREvkcTs-ur#Y4YdJp9t);b#{Q9r3vJf|paOSQQhgiRk$*bL~DL92XzjT;o*E6*@?^4iG_FtZ>?njz8|BmeWCIy*_fk)|m&4SE^r?EGJ3#t`|G$;fK|{R{ckuZFt^ zut=vLD{punrq5X8_T^jt!ve_-G^@GsWTPqm^2SHDrt$(@H%Z{q!}xz4$?*sV>bG&w(H!oVc2f4 zt~qvzs!5TWrs6@Ib%Q*cLwMz^DJ(bkK15Aw=QtVXdfq~B7=Gt!^^|*RJbF8g%qH{Z z)g4e^g2}Qf$-uwys04P!x`MN6td0{eP}?M7vsH4uI{Nx+NHW08JT;1&f+!;*JwYCE zDj5PM8HG!hdu z6~Wz<7(5F*ZG#evLMf4&0z#}B3e9)1O6f}Js$0Qvv2pK(yQCu2ltk}EwZtp<^^m(s`k3k_cj3mWcCMx@Y%ZGCstS~dOqv@LbLXe9BO#}{ zL;9vUn@Ud?4N3}P+Mov*`TET%PJ`DSj~1wzmzC(c6bB6y2RXFjkQTZytrnvUC~rmG zErq8RX1J5loG=*~q*D_Jv1WiFk)qZFTLxGPh|M6(i6R&%@R=Z2$Ehc8LGg6N^w|(o z!+Hw2hLzgI$#}M+IGGlB+R>@qM2ve4Yc90f;upU=*uH(csLir-=S}R;I!)wQJkkkT z`P|7#Hj!cmv8Ly);A4R9WKFaD+)2I2CT4u!3D)++-7g?HF+Dx^9?eWrvW=2KVfSLe zsouDrKKH~8(`(ILn&za}Vmb0KtZiYDZNi)sR`8v$*K$lZOJ+GTQ&ILcpRr*L3qF!+ zXybWhHbt+=C?+Q3P|L>61DIH~i)LUWF^LXlww;5iX9QGgi`$NSH?^1Vc*tPX1TNnt ziW~)A5f%V9+zZn=J~0840#`&C^CK;2*|zOAX$ql4EJme$@_vF}ndT-kmq{!ep?C7= zr!hgAh$&{e+1Rk07(R|9Ne{GCa>w#y4hwGBWcS@E4ZM2Fj^jLR8J-flXrv-V++&}M z2sY)K=Hk9V4(NpKkPOQ=E@$xG?91gIKE~W>Vq!$QTNfeV-umcddfLOpooan3d36me zdCe_x2%jvtg*IKiz>TKvxH9ax(Y6`yn(cIRg!ChpMjiR!mt@O z;#&>fzhh@@HNj5PRkWM$j2z;YMfM6)&xG>{C!?tDmwzIjOUzj&tgOz$7Ae*FDc@7;X<(vPox8khh|EFyQsRIN*u;XqFG?dVb?3AIb@@9XbU zwEliDnR~k{gsIqczD+tgA;~om+N-ii$Ea}Z9L*POHwS(Ng&sAq+cI#B1&iHHFkXda zTBGW%$AR%!)Uid1<_x0L0iN#|9UJUeNxrHHTB4(dl{a!CBE>uol1_QGWSq6KaWw3c zp(&wZbV^@dKjPv7T`9xOG}J@6K~`3)I4A=*J<+h^Mr|cXJ8^CuiF5D1^3w~i2PSOe z(&aQPVo;;iZp=V9S1*;$!dfmPlKtn0r5CS1=PvEL3`_2|ExihNe&r%!eRB8ZFaGng zi(`hJPOrc}xO)9?=`7p@uWkD6v-Bc7<|X*~s@#P2`U@UbuX(MjbPS~#qML0m{b8w- z>tz`DXXVLo-yJkDpFK_n(B0uN2V%(zbC*wi@^?&%)--PMt%2@9Bi`QhB&(m7uzU1T zGRCvav?>kRta4UOnDBy~_arLEqtUzMaF2abBOyg=+U1#krL9{LnNJDbF*$Jzu466` z_RIqjJn(;#Q2$m)Fj&$9$5P_P*_{yZC;aj@vs8K_VxFz%d&bqKZa zr$HL$VT<_;0&o|oIWD+G(hp$zk7HvV*DoCmwdChc<&j1t$DGZs`t)%tja>jinRa~w zc79OLoa*ut|QIXVjflA*^GifpDH!AWhB{Jxb z#`K@Sq!<#DoQ+dMi+gC`@%}6R-TzTlQfly?+Kxl7;}EOcZ4M%x3$v-N!>A_3YC0W{ z2k7UqaO%d98%Mu=e&LgEpTBYR##61wIe$F^Pk-v$=gEyH7ruZ&JoWAKfeEtk$-?pf zpS^eeZlgN)$N!am+_k=h(|D{I-POZ6OTOf4JFe}JQ%n|Xd$cX>Xf&hF#g2^mt=LIO z3IST6K)IAlLm@!O9dd=W>hgW@eI>uDQ{GUFlTh0K;rBdSwl&tsW=3&hi?~gQW;7aU z?`QV2pX=uXcR5obnK(Xs>E@(MFd-L+LdYkKLKHhF!X8pY->R&ijWQEs8;B)jk>L`o zoq-}#In`1aT{QWy)fS6OvLr{rv5!r-Uhm&ewjf#tGl`8@@$Sp;1eQsIn!!v3fo~C8 z0enFZwMBi0rE>TUV2y3`O|CWzrt6qd@WzZql@+KyP~;;cE@w^&Me2HUaDG+! zC#0lOi%VnqHAxySYTx;AtD4gc(|=uiy-)mPOWOk*xnJGjqZB8JROTq~W3@IV?a@>O zOkAZl;_Bo#pQjDqDp3R2{XPa7L{#M>nK?8wHFFsLzCv)~*6-m`>ezgl0EoBSnWg5q znzPv=>`#-K5xs_K=f=?IOLG%`e*=&v$n2eQ^+H9e7G1=*MHdN65f7zN5D%yyM%R2+ z9M;H!6Rshh_y7Y%0z)JNh=w($lV~UuSQvZy|G!;^mE$bw%AGX869E#5+zonX)AnR` zArDhAw~icBF!gaMp={#WipmJXvJH(A@Ht08*Rlo-+_iH%!IzN3wdp6{+XeVGDcCSZ zXwHN&jf_w|OR?ZncT7^?Hu~KWU(KyU-%h}J=+4YK9;nM~uLaj97%Cd_!I`9vZo zlghV;UP^|^-mYzF4C5AxFz*)WW;E+G0FzZR3iN=hj5QKhsuAhRfluiqq*!Q^S?$Nd zq44^05t4Wj_Drv$sEf_Rg`ZwH%`84)oNG2eA0uEY$d}=r9EJ~i-S-+>$z>naaceNcZ>bBB5fs$ zI_6rrGU00SIPA0h6Qq%pq8^_!7<`oli=fnm>c_@_jFu-`kl~&eiwKu(FA%FCJ5tUV zYXEM?DKaxlhlfk>dFE15(PUjM6-}-t9YvO6;aW8nq}Z^X4{N-_C&5L&OwE~C2Wn*D z@aJpGm2rQ>wtf24(s(Qbt5kxAYq4LH0oKtlXNkof&49 zkwIh3)0)($(Gu1n50{8eSLsaZ`IX1e>Y?yiYlA?(iR2oxvd|);goxE3;A!e5Ia@;U zmgQ1v>0>$-Nc)TZWg(d2Pz-%M&{J+_QCR_8Oj!k73?v5ozFg}r`lfHV3;f4n(qSXO2`|}KkRom>C>5J*{?mE}7IJz#2iAOWyvDSt z(@{BDo&03ARw3!LRw0%O%Tic=@L;>Qo3v0c)Et8ycl{uwVfw>T)RPzg=}&(bD4g|l zUc*svUK@^iHtn=`P}QhZTCVFepO9n^)N&A<#3gTNM8wO;!Gi~zO!I@&s&nhBDLO_5 zY;(Ny)%DU>N31*#?x&8c<80U*XV!R0uiRjoZx#uxM=@ zpxv#WhK2^IsZU=Rs~g#D**WNm1?&WG@m|1;2NumBc$khgPI`?onkQQw`8Xy`CKgqTTHS^qN`KEB~()#H1NpR>)=63KF&!u)4 zT9%nom!@SYH8X9yHJH`lF^}bUxvOP^8PHH|P};ZC-GP{@m@ohZ$W2%@qyP`2jd4E} z=_>D2FJQwD@l)tR)WNA*=J}Y%oCH| z219Tds^KQ3y6AGOJcS)(r)g5Vp4-?mFj%|w4A6t{qg&H@j%s#w&q1KMWCB6vBqi($ zowz8^BK{Xp?17}->T-%AbJHMe!dlpBaV*{R(uh8odp;{5Ix%%js>A9~*AE6WE^0JA zuCnb67WPIgYr?2~Tyvipk8D-6g{|YB%}$H*S3Y z`Wv@De|?25epjDoi{C|y8>F!B`?vlufBxOur>@?*e5TIs?qY>onu4#M6LYZ0DHaZ) zBZ>bgiVM#su*nt2GRuU=b@6Q#|GZdP7{q3Uq6T*4JMwY=Ni*Yc@B!yE-? zACt}dsA=*B*2VzU=%^dkE>_iGG2a?Y4oeZwx(LA1QhBw}w~u|U(JY8z=~OCh+FZrz z;}OFoV%A{7+Xg^kk-_QfW94&ivSJ_M=UM;;8uYJlVDn&i;lfIh*hlWy2 zPwneVb#!;7d%OB4_VsmWjOxtXzn_fCQaDCh32Z|wv$X3}=cOaZ)I3=_M*Nz=g%u}A z;s0yc{BOc_1Y_g(iQ8kX3_3kyGD^0?vdhp0nhevnZ7h#OyT}-PZ%J}OQW}p${{+oz zaClU%!SL48yv8y@k5qhDJ=%-u(^PB8SMWSKt5c+WX&Ke&+V26W>oh|1g=GdU0rI{`jfeFMoLF(9v%X{r0=- zuirU!F4b>1Hj`uFHn?j3`D^$!Q)lJ<`{yPvyU)y@cx8i&-aEGHgRsiDgD+82&GAuY z-d%Jkj%ABc56pz`p8BDo9b6k>smC0P;<*uOSq_@`dS;=YUjJFzv5nT63ntQ!VL4hS zXJmkJ(6?ibnPEg{4xxG$et6bSktd(rsgHm-4w5~lO(xp=yLJwCk%j*~LX^0a2t^W+ z1exfbKGM5==`DCK5kns`8IO{Q-gHm(9e4`{qJmWd%AK_R_ctd${PxhFH@baQQ@1Y$ z5Zs*7cp>YQ+&Vp>xU_o0XSYfm>n1~;@@vP4k1TJ=C-qhdYqHIv3D(b}6uYM+`TN zXu7wUL5S7tVo~SF8WYSS&6tmm44U@5d`iDIBSde);auyYuPU`5V8VKXYYh=+3(*=AZrj_wW4n_S^68 z-+%k{BlBN={QaS~=a0Yh{U1)=df~V8zkP1QyES$4E8UzLdNR@0G0+Ld#r&7A1{T%A z9bEC2P_-RHy!Iz4EP9OBF0sZk{oN1oLXoqqL5w29{KTjMd!Ys&*lvQ3WMsR;OjFbW z^qJ$VQ2Oc(t)8yj$W;K!U#^mVmd$b;wn6ezULxH%ILlIV)7R~Brjy58+70;k@s>IP zoVBS8aKS_Sma`%yJpd0uKK-*OwBylI?WJ`ZfD^2j=R_1;+fPA#&v{1BBS6T zKL&Cu=9u2r!%U|@w)7j!1WzZ^NF9Ei+ulWHI*xeKN&i$qUc>fB=p>(f64jBIFp@Fp zh&^E$j;XRo7NRZt_&7B)6R?l6N1l9gz!|peRL9=_{;uA^o;|&TJJ2kB>M2t5)~~wK z{ZBpRI^XMP6aV$dBN@;KhD~rJ@C)o5$8A_=TM0Z*V_Z-bR60BG`v}qi6`FugvW|N1 zG;~#P3u9x{wvIGx$AZwT%>q>cu0Y$;1)_C|Y+K}N=+K0zbFkv}PTjR?tW0zsz$M5f^+ox{g!`ZhQU8R1s9%2J7s$+p z&75#+_=Za}hh`?rm&=z)p9PkQ(dUe>U4Qib!F)a==%)my7V>LtB`~4LyLEM;IO5{W zAyR%xfI?po#8fBX&GY5UGsglyKl{L4X~lS*SpdL5Kfi$GxTgqDMAjg}%hzX)%^WUY1#9VL z80F=esR|u2DgUm*p#uZFSGJPuD%6(!S87KSQZm}`y8mevTY;OjmF(0s_||nqn@*%c zvLuHZMq3>k+QGD9s)O3Cv{f_giKSOqI)lucAg&S9D1gis8|hMaPQCoywUe-3ZNq!| z)+@(uy>YqeB3rkf{mtzguiiR+Z2s)a-(7ij{*8-)W%^4@_>212LDR8wcj1{Nus3MklJRD(1(ZU{Z6*f6! zaG!aBls`u&8_vVNFmnPabK*+PP0TajAS+Hs75h(++dqY_300g5q@BA*WUiS$h5&9a0w%|l@K(5 z5N1!$p1XN`_QMKK@lp8yk(-BSPl?0XE3@ZjKfL+O;^BqaD{$!V%@vFGJ}*)YBc{X{ zwqaa%5k6fT%R>oCp=iV9x7Fl!r3`6bd)wdy!ZUBN9wsB$&4eC+#z(LeNR%e&C`ts6 z#gnbBlUvD{VQLvN#z7n51>z8owh}Cx!2yW#aL_6a*7>nmtEIfl$mx0s_76QD+y(5U zqdcR6;0#v{T5woccn%aP5ZAGnHSI;1v?w(Jg|VfnUSxA zkr!j~?nI~GOee@UB1JsDpl=WV)+=Xkz4zAbH$J^bvb#qODD$WPbo<6YETRHG43Gnq2n=ZkRxCE-9uOU!DLoR`mlwyg7>v9o7 zUezea*y%R8(!9ISEstYky5(3PJ@6QCcgvWf9fvS1{en+$oxnS=yIMBEB$SDTVq-Z2 zK3gc_kN3Qd^4Dlld=j>kqG1X^eq%(Gl#MZ#XL<(HSh``;VU3Yi0&_^sjE4P9a9Jhs z1uso9BD!gGwS?#Kr*VO9zp=G;4Qvn1!Wet&-03{p2xrR|M26zV%qxK<^C(ws z!)zUJ3a%WM9Cnrdg?+o4;Rr15O&YjWv@m5lC8l9JdWNo5A9zD9N;oFPVhwMYKPd;2 zXG9N?rE+6E(?#JV7?VG#1Ta=_M&gp3l%;sQx%@};u7tmJ0VeQJ#p@Cg|FfuczZ7oC zSy5JdcI{KZu6{(H?T86cPam%Zy3zeo+x}c*dVcMIhVF}qy7{ol8_=P>RHu1&Jyx~O zM)3)=oxm#hM0}H<`^R>=0y7%{pjkVm3Mwcf*Di}75tu zhiN*~abY-tC9(&+Mn`j8G(A?9!oe*;Ul)J8FYf-ig}$U=ZTlZJJZy z`h!e(D=SL|eK&pV!|b)bD$n-mRQwH?tPyp@a^+$0pyfSPPyNl%|L z4GdG03?uuvnrp;E!svjb87^bb)xGJrc@yLjmLeXGuFbVmT{~F=EC6D_T(9T8lN5!L z@!GJh@0^TZ8V`sX1|9)cFpgYF6 zg?zP$#I==RcE^p-x(LU*0A0I;&(+p#WZQ5QWEa&PYLcC_3*On=PR9?pEXG)UgE6*>y$Fv8g1Oc{BEH#phPjquYuAW*pH!*I#|@pb zykVNbEBJb00WeK<*P2&7=9o{HIf5OPGX)s zaIt`?&MZv9ucurh;MUf9P(ccg2b^M2;|S`-h(?J@HAa|C`lyAyt`6gbOuD2beJ?;ug9BLorvk zbIfPEa}vv26{Xg6{E25X^h^UP&6CWiZZeAhO(t00+y$;7+IrqpFg0Whi|hvJ%h|ZIb$n3eB zQ?sAr5N1zfv&HPG;1Yf;zibJ*x-$GWWhAi()l2>xWDRz*QKlJ1Jh2iSdV5QaC|dUF z9M$b^uqQJHX7<(}6@3?CC6VnWFIYJ`7FZUEm?S4dk^=jZoN(dMwJP-$qq4eYOn|Lk z>$(O4@a6iLkpt}rXU4di*O1}>Y`=ZXw6Xp?(q#IvhpKs|BT+fof)LDklY#l(#nqfO z!n6j^dk1i~%pAhAxa{|8m_half|11X&P3y=y;W zEx?WYvITUTCX@Q3+@b=|pPLNls}fY(?Tk@wa{`mf#~bq9#tgWRt@CWow%kiKw8B~q zt-4slVJQ}AScO)|pEATLZgk^+u*&VWRGG0DzocdfL5@h5z$VtoKn{TmchIrCMPQ&0 zfe}irJgWd7s|HnbM5m*TU=MW|X1-+SSPtpD>=C-G-6<5SpI8pXRuy1ek{bsg`e{W* zKz!>N3<)P;E!B#`H+bgshs07%ZrcVyU|3UhNif1SVi_88^}w@7ixPRfrTZa_j%2t2 zE+4o?S@4sL$6K}v5;cU=HW0mX1hg(skwlnSoWO0l3AfY@!U)ws9g|1`9S*y4Qbb!! zn$-|PLLwbGMn)ZjZzWVm@*Yl*GUk8k*+(0h0->GSwipx6#~?LzV>GzKkL=S?S;F?; zs9W8xWE!n*8%63t)C93ucQBd}jV@Wl%F|c2TsPL%SOos6y>;>5ul@dq^DhUM<-dM^oO+Y!8EXM@}-HI<3)$Vp$OvKDRSL-_f+lYFbyu%cRyML;dc#c zX0alK^$!Z9Em{}`1D*6zwF1JA`2wwx`f%{XBJeE0a}jYV-qvuken>94+5-^{NeQJk zCg(e|jWwuTxAK(aO`c^o*&>4`Z-xIPA8v+?Fs+YHpVTwdba^7xbE&khBg3~&O~JsT zO~0RMfKGPlCT^WtUsm$!vZ|MMUC~@=8L$mCkHt_eH6lacbW!PLk^e(FkbaO1z*;S? zNJ7{YQMVqnzID~g|%#+7DI3eFbt6m9n`K%<;vk|eo4Ic zr*xwcDdtH|1PQ_I733W(2WK$yHdrdw<6>y(tVb4U7N~wN?EeO;zL=TKRn7!kL6Mjg zg*_0A^{~t6Z@|!u#-vE&xZWbu*Ri9kWB0zDyLZBl1x^|^wu96DGbJH`PlGM+a43;Z zV_7BF_O9M-JO8(9U?RJaWkMJm3+{k%4uN;$6tRR%XAA};k=B)?0H9&zNHo-J;kQBX zTRTfSwd_)g^cyNHQ?iW~xYj@qKznv9$8F};ruDI+>cP#KTK?Pu*sl}yA6%olx#6fwSg39yHhC0 z;k(-kln{dAM&!H6#`GGX(6wPENEPw1TOgXIdX^5u_JG{eI&&CiG<1vF<5>pVp-pmJ zA9u2BWcqsce(~3?ndPk8ytQZgI(*)2-C-cdVqo$0P;E5CE$#~OCS3q7NuLNVD8Yyn z@qk)wRts;Kr)3~V$8<`%G(HXsLZBVrRlRWia^aP$h4B%IOEC}fZc+2U$iKQ)QaK|< zvouC+T~ykAb=91{kojtgwiV6sQjuGwCX>s~G1#ZbzQ6tWPM$o(2HY1=iwDm_%d=CRK{Lh@gW4zoQr;QaDmu zB^ji)xP#i$r?N2rOcdQd;n-k)T2!~6$S~`PjG^n(r|<=P0yj0p2YO=qohQI>&0x-6 zSlB!}eQKi{oj+1<+|5Wkn&Sq-Ia;h}v%)Zpt|1)9RSJ*gWL3oCwND|#&J}7HqTtCV z%kLtBx&RkDPs%6Duaoj?0$$+K%pt@gn0dN<89@swxRy^n)nt2E`AuY$6I7BX#VY|d z5Lj+7OP3L!00Dz8BclA<<@d^estPYCwgdYI&$it;gx@r@2D}K|J$746cViGJa=gBcD92Y78Bze(x`HEq z&5)myy;|NVItDVKgA-wna*Pz*tA^dRB0b;Dth@s|ndVeQRy~*MHHH}sh~&9>05PpC z$AM33c^Fyn#=lM@!%0F3yGkgdao5_4`(IsQDIVE`S6#(oq?p-)8&PcPT|e3rlvP}c zc$$)p^^EifS{Q_4MgCe6mIOy-N5O=pXyb$>*f*`_SjG`O z(amTg?Ck3xX*&l~#Z`;!6Pzg>Ia(;9yOiU_;APy~Oo!V&hMs3GSnXt0gC+gt>`^vR zNmd%}&c$jMt#x;f{^QD@{`&FoUq5c!Ijoaj$D$>(323#JI&fN+RqIxoat3pb_G@bD zv=rQ)>tr3&v0PrP3yt2lWa3iP)9)9gbZq*lX!LOO3>v7*>d2a49j#aGSkSr`lvGlR zdAd@3SAC+SUf1B>hE-{C$iRydDVoz44vLdZR~@+2G@8&wN?V4?X{D02lL>~4Q3C-l z8VERaJKQ2UW@^mU3I-GJLcWRnge3r>VVgtPf3@EZCj*Y%C?az#%X|zC^Q4qrCZ{^-;P45ukD7H&h?c$HirQtBBKIIQ;uT>u7G5sFK9CaCsxqt`r#c)H zdsGP<#s0oZ&bu1Zlx^tRvF(adDtejER4joog}^vD470!z_@6o@K$e zx7}uD_^u!&P*PD)nze-ykh^%4s|G&BU04UEEhyAb;4%2-Ow@&b`W@$U^}-7FOY51*Y%NcmeZM<$7Fj}t860eu!kVx z16!uw{b5ANnz;dmt6h8yhI^mTJ2IP78d zF0M7^s!kTKeXg^(6iRq(cE9UvAJas?GAy>GjOc%0i`Blq)Zo5}!F~4*!C|dZg{%9% zV@A$!Obd$`gCn->qS$DibTt#mkwjHWl>>ibl0(>)(5RRk0T#C5S5s|0u1-NMSm16n z(!%{K3R=Xp?J7<~Tc$jJRT_)rP^OziwX?NL-4^ zo)Cgx2#lejN5yrO^784=rav<|w``he+hN!p)SlSUfq;iU9~#;XN-S$&X=SPl3vEaq z+eI9c=S67;GIsx>_PRm=@m&Z!cA2KDF<<=gCe4k~0zD9133gewffBH6*&Ql!1&aL2 zSP4lHrPdw~#42>ULL?;UfRI1uZ1Z_Zf*I+C7{^tzMR5IH!7~jkJ*4*;S;menCtQ0| z!nZb56e$+1O}!3heF|oQtq2(lFB0T3BuHZqLLFrto5C_g2E}uj0 zB~rdx5!B4nqCE66lG0u-e@VVQ^vCk+f@|zDe(KvpZ#E=FifIL@Q`%u-+4rc+AWz6H zzgHailxRdsD4WT2cm7tTCcM%u{IhNeTp5eLVxCxe&aY%Z34U|(&2LVA_|5yzeRJ|t z^nSlNdFh*zUw(7)vN(0}o0DIBbMkDHDaCKz|KOYV|4eWYI20Jf)Hf$Tsf;6N-K%aK z$mg5|Hp8VBNEcas*k>qDPKA-DW)mW9*(x%0mFxOC)$uzbxpnDx`vdFKhorFgQ|CRjO3+pZZ#0VH9I zcw{~3PC~~<9F5p3V0>(hjI0u0O-dd3!7PWLLsZQfCKjgE>!glacJ|h$Uw7(=Gf!RF z*GN*5VCP{Um&&gGGTHZPi6~MuQM+0;ON<*WG5+I`e}4Yv4_A%{7UtTO7|U3or^GjT?pAJ& zQA=P2Si~@OYCG$Oh>Ma!HqTlsV&zWig+}65h6R(_&Dci04WpBzR6_{b0jeXG z#tN;sKiXKG09A$sL5LLQQjFCslJ68!Bq7BTcf(zRRT>>|!U02|)O^9aJq0l1oZwl! z9-8HO$*Gs}i`ZdpxO3BH5rIx5`( z1L1|T4J9quSZgHGD&GYhFv=rbCu`vwxEtj0C@1(qgN%YC9NSoLSp-&79mu(ug?T1h zP<1v!*|UxIaaHB+&AtBKzN#dmQryFgI-c_>iCfRVIRD${ZoP5w)+@~xI-5WG!JSu6 z-hT7#`7?hGJoWp))Goqb*W5?ydn=Ju-IXYkj@BAhLG)5TGWl(hlkr$vJl+&|tc#U7 zEuLDP)PIn6p3ife{72i)0y>mRV8K*xQ58*a#yCYm6gS0_2nf>o&|9Jch%c{EQy_YB zO`ATM%@R6Fth~vG;mZ3(9@n_ER)t^anSNiV8KzrAFCoiwjyU%8XL&faH1f8Ei482s zPG**Lmu`zyt*)3)vr9pKZY9((+0H&{>J=BIKCzChL?uM*mSrg(npk$Ue?Ms@Ka)3l z7H`%1t7QWUEWZxTVc^e@-tF#eLNxA5Cq#WFz(I~tFc%um$~`##r)ubxjCO%vWU{Sb3yy-$(TOoKQmOGu zJBWv>VO4bj1g#RCjag)j4O`s4Z=9VQr$m3il4Zpe23M-aig@i)vWk@Ov`YlTR!(;f zOnm$FZ};!-Kv3KT4%G@M=0i_DiCfFUawHYug8<=%6RJin>yZgAtOCT*d}KnUcFxd1 z+E(F_60_XLt*+~ydWtN+?{Pl|OO0`N&y!QF%gr$W4PPf`eXA@HOUaOdGCalFCtfvXy zcCs8zwIOR;j!PjAUT2UZ-!3d#oIVW-L!|<$yNOGD)8?-op5fCMRA#}$4%YO5;27%^ zX~p0@t7}vHt00MXJW(4e*tEi2Rv74DI|m}OkpF;p z6ktjrfIKd#O(lG<7k)QmIxKS*yx3v63U=Yv`-QDn5CAU{YB)vl^WuVeWmjoGH?%1= zxXgmWqf`esRZ4d1BOGnc;5IhZK+C#1lH*lpH}2cjRbrCjL38pQ5o~SW-Q8T9T{m)w z5<~|XP;-1_r1Cm1IZ|FCyG_orR)Lya556YS%iE7!cB_B0Rf$VUrPh~KsZVM2!xd(w z`-o7`Pd(G0!M4!4jl);pGv#!TO1#C74!$OO`dkulL%I8wKvGJ0fcDqU=OhjO@1cht zYB#iu;7Ebj`*PIAJ&_V{v(!91!T&=Zd4vq&foKAuIomF#+S=^u8Hwv{a1Hz~O}6(9 zR6dCt8RmE^H_Nb5Ns`vpE%n*OxjcZFat%DV4YcgDORLw_NnJVc_xyxqDHiu#yytN| zN0Hz+Oa7-G74JqiC9r30pBa3?{;y#%X_Sut+F|1^r@!j*`<{1X@vV&&ZI*#GUIzBB zBk%n1!XE+)bnVK(6oEA90k&yS7ccCMOX}FJ(5Yz}rvEtY@HQP?PPa^O~ z{PYG2ERY=x5x88KdSMs1fbY~-I3a}+wd-%wS+C=)7x~(;pETsEW6g5o0Im;|sHmf^ zC5Cci;tESqFVbF+TE4{yMy2paC&MPyGa_pF^NkO?vT8-Zk4JJ`Xl-&}A@&aJ2!&jE zEX4&-uJ}bLSFoHp9-PI}tY9&E83pu>D!yPp&U7L7+5 zE>T;KW+<z=BtEdhiSbcRwju<}53 zBOLg;qZtOSs;&L~B?|hWKrNnw&B&&Bn5V(9Ci|&XBIEE)ZeYCzGHA%|?&UPykXioxWPK6qr;+{I$mpX-$f$jx{d*&*6kJi;59WC?n{tp>4)E4N+s;B zP!(crm_wCzr5xB#MLzxYrfBLs@yDIV8nB{63pO)H*>_LFA}s%=L?*hD9ZLc#X39bu&r**fD@6Len`k?bKrN;Qdnl zM%H1&n!^rISL!IRj~YgV*EJK(-4^*jc*Xne_n>4yQLTJR_^f<_#~>j^?ur^iSh&+C zM_?6BpR{>4&v;XCJ}qC3J%(jbHBYt~y2TnGz1?M~1?I9kqOMYksLx`Gu%P}<)Rv*G z7+7fOK&B?DA@;p(a@B6B4Kqw#xbxv_D`0!<*xgbijFue+?1H;1?aucedFsbRa{Jtc z+sD87{+*Zp^7(f+-k3jr>ekuc1(u5fQ>sK_lA=W6-|$^&Ohu98NUd~yqrh$%8d6i+ z`zN;d@86FFENZG_Pfu!F{{&Xt5tHFWWH7)tG=yhTgC;M+uWE%E%at^yv(|8D>u6@5 zRMfNk_mgj*esR?+2evI-SpBA-nSwCba(yG3J*vrJ|AgryKa=tM)3#uhGEAPuKqzsF znn5$nMAj+xYYd~UU0+=TrKdhzkMi4yvMWG$&y_FFOu@g`s?SxucT?MFh8C>0^84k> z2%S9hLiOb_fp9xBIr9u$d9eayi>L6Km*6F0EngBh6sJBaUnpOLljqC9!8vy@txzJy zD04v;2M2oyV~uyxL+2Mnslz^;HEE$h3ZTr9YDPyDLu~}JP;)l2)H~M5^z|G(MM(Nr zxu&%bw(Bw2s!4{znW1V}M}`L?&WbI-)B7f*iw@}d3vZ@+wQrT+IC9Xe5Synp|9 zSD(kql5QKMBX>cq|-MzPo%H&WYdf zXu7xaXWi-E?Gv}JOy2(V(5;uR%s=zteKY3Q$n#p0t*RT$<1C1$by<3_k$oyDh2jnO zVD^K8#{hj6w0*`=S*7-w!1mB*a9yS5^eKW=pB>IHw+b76CD)jF6t*_xa8oTY2--mD zY=@y`RCIW_;litmB;=~1t{nK2R#=w8o-(7Ak=!0LGz#jOWh%97Rk`&VVUd!WShxZ2JG0JS0pX%ds>!@qy@I9QFtCXMF6;VLo>2&tkHNfwagPTv! zel&Y=_Ji4TD}f^RdzX?aiVfC%CH?)>%_FlH$?SVKj}v?iV>usK8c5O&o=!rRj3jU!-(K*wy6$#nkym{t7dRxzB9OyuoU(% z(gh*}Lh2cGV=pawt>sH7AMk*Wz~4jg&b7t!O(rn#y}u}5n>itvj;;h>OU2UYD9`6z zLG)H7T)z}AysC_%Qpls^d|TUlku?x?oN%B8@VkZFj!aL1chuLF?%46Gi9UuKErB&l zwot8@qa^Y0T9Q8A8yz@U?GRry^cD-^#b7NQYukW98>k>E4Kovd9oczTjuL_>hiXOV z+gh0YaQ0I$6Fvt+;pXJcso67VJCK`4VblF^_6mV5_~z3$kAo@k5uAn#&&*zdD~`>6 z@bw9}3O@T0Y|xW``S|7+U!TCM@IH$lI7Z-~Gx*PkaOMoWQMvuiqvY>rX3yb0zCQ7n zYvN0e-8}mB3Ao^MxbhrdpFD<_{{3nA+|A=~gJbyJaErg6BD0^vFPZ&h z_5yr2T#VNdJbeal^MSZ0{3!e{p8G)ju%q~kj!Dw`LTd#!jL@LF`9)=dP;4K;l!JvA z_`wT@LZP1Y|1Q%R_4&x*{Hl*$5)9Sa4LFcyf2Vv2*|y4G!hbK%yg9-E6K=<1yGXTJ#IGudDpK6z<^|+_C7pB_LG6&pB#Qxd z3yi&u&y8KDxN{fXPTK}@7|GQb#{P#5YG#6`DEqp}vS3ytf{B0^-*7Ms@olK7m59bi z5XW1^k_h4yw^kr*z?EnloIrk-E!M*X9F9r`W`{AxMwv-Ol4apw>w5blYqiXF85S$% zNV~~xyMYq5l7B-{x5mqD>Qlvz*R%31=GTa3s!v{zH^?LAR?%E+TL`2DOxZ#rFpJXJh zNKJ0#*0I?Eq>_}v@@6gSkE%u66=ZQiyNfct3yDaui>T-F4}~)ZXq12f2w6 zI}vS10V4F%G88+(k3p>XL1t?#I9gs^d$ECb7LZ>=ig~*3d>6H(9ZU2Z>yDnaMp#Zm z^f|-CA}0oJ&IHJ7iJu~!r)_nvpV+~)5%^dZ96L3)mdI}RMV_71BHLMx@q*4R;uJ`R zGcv$n?)4(Gg6F|I3otz{=&rzYSrLQ-R>HMN00&kIz=`;whsNyIYwye-eqsK^E8pFC zbN;2t6_R!vf@?D#4X-Ak?~na%{_5pA$?x=IwaM?a$;XXl`EEsm*_D!DsE1v^8E$Gv z-`kp;sAE6|0`{kth-uds#rF1_0d*D+jgZ?tFnptmkoMSwN~JYTTUI> z%pHDp?(pgPqwmj6Jv0BsiKb%^zIFNN?U&EqnL0oJ=?j5HkhTYCX^-Oy=>`t{fd~>( z#6tq`fIG7ms4xSYIC2K~O6#e-!XYW@iS-3hUf9@{!Fthwd`Y8uM!-%q4Qir}24Oko z3WS<1{d+xzK|jKZ(G6h`bQrqj6u|u7tJx-P?KZ~9qXtZKHn<{7L?s#i$ciL~?$$I2 zhoz{e{J^(9Ti(VuG<56D4{yEoyIXG_-oJlt^77o|r*nrtnmc@GZtCRR;ZNqK&deP? zevh`DwE`)wQFdZH-(7z_u#Dy=znGi+)7<29b5k#Yrk|TU;!0z=@>ssz*Kkydc<{A+ zudB+||KyWHLrX;z{S^!#stNuEn~ySLsT?lnBglgVx0(3yOZM-7>M0SlunnyD77eyd zW>^eRZI}x{?jjqf8V{~K7vX5rqs(;Uu~yY**uVOWNy%`-tZ18JUd_ii3@z@BpZ(vi8zFIO;zcJ+CTp1I4*A}jQNw&4aeeF~;uK!Siri}_kARu{h39E)g1Rx}Vy zzf;Ij@{88{RL{ItLhLg+HWE?|Y5TZtx9-9OAlkDKH5-chPJGp~EPiw4`!}ARJNz`-6o>!Vj7G%#*%#)|oxAhO z$Mf$@flc)7p*L?`K6UH&M{mFV{;gNe!pUzBy%ktQ(FEpOBnAG7Cfq`D#VHfMcQO!D zJXUK{1W}K&LaYi9zR8Hd#YgaeqKFc!b}&`RV-C_t!5)T){1zX>frkHsRC_7YedyRg7Sk!PB(!oH8`^6f%b_=HtZD^m72c3pR0eb=q!owNr z!UyvU4;Dp0UnR1+Tz-7HsLWjNqIr;3T#9%QdN!GvH`qe2sgYyU&VjvRku8Hc(-_Mb zV?z5nh2T#5a;|2V)#BG^i6RPd!c$<}SpWCTvGUmpYFYEyE#=n*$QpJx-YB1i5q(iU zKQmdrijCg{UR=UHaQGKN*DlRWkuBx-VX#->(n%a}aM5k=>V34UyKBIu0YdRAx@P6T zp9p1Hl0yyWgEqCmdOUBt3XE*&=~$N~FC*kb?epS$$(P@qnVfmHd<`Fdf{TTeuflpc ziztUPhe-L4@YY445^gkCw5|MR1?>R_g3Zla;590P8!*@_%f=X7XyxC+jV=m2NVwH0 zyh-_-&>x^rTpGZIBkvPNJ}N+#6%QVf-vB~8XZk499Y+4+{#0EwY>ag-=?dc5T;mof z2FKM^T>t&lE=RMn%rG0Mq3H$k}h!jsq z3X(%BF3{a@bq@{u-a(am81lN`9du8zXfoTT#NruB3NYV)`S{5vcj_ZBb9vn8>|v^D zR*`AcBoi<<`Xf(1Ik2;{t2f=#`_xlpVwmHPd}L^71hIB^8+lH8__(7R(t%=jLJ5V6 z2Vv`*7=iD#>wJ7<`f?@%=eEJW7SpT@wGCoq!KlnIb3#rgq7%u9*wU4jO*8Oa13CB# zBTx75mn7*g*Y3~iU!$5U*AxX_)gn4gwiy+vxXmwDv2W$T=fe|06tmig$8R&NULtwA zhH%FXOurxPyStJq z1h-$J@Pvmj-62JiRROiik^0r7kyRxWmy(_yyT()ZU7*f}VYkoW1;SJ5a!jMh$Z&}i z5GGC6xl?G~wEp5itI}O%=m@I5(BXt_9q=4vI4sBIa}hjV#S5-LRje9TjQI{LhC@=! z!*vvBS#p9OG!Xy5a$tYk`0P}*wcRF6Fw8L7QsL?jz6!U|KCY_V<^5G{9mTh;g9cNw z;h3SIVZntYz6_1+st;e`WZ1G7_0R#zT}yrDxB+T5n}(W)}h7;*u=m?irH8vTN}M1*!rOC!}bk*0|YB&Qy~# z@E0sMix&)itx!7}%F7@svAkI&^<#np<5VE}t`U7)W1>PH?S{=!osAK-YN2iDooy9M zvXfhEU5rUN;wqYo_%fxx=MN*aaT0==iU{!5O>c=ES3&T9?tf%FN z@UY2RZ_58JhK#@h-EGJ+nbFejVVhF_Y{va=}AzGE5x0ka18w zE)L6TO5SHiK89GH!HPpw0SHcqX=d5ZQeyR(fYwF=YgNZ5v{a4KO10y7G}2EG6djxF zC{+jdqatY;WE$18)D2WDC&E$!f$u_+8}dboC8}Y9%7IV$#H47%cf~|Y2S~%D8tFv3 zO1^-P32lEfl4Z8{!2?z0v=i7~-}?Wx-g)WcMu zT*FwsjTs@+LVzXFWtU+#z5@dg2~4b98fBVMbmcj@MG92x1lX=rG}$Nz`4IS8Fl`^Q zh~;Fn%)-3WFGwdV7^+Ea&4-u2OUQ6k?i^4rdqv|C^vWyAC!+54Cd) z;m}k|3%oQX z!gMye_{y<45!DB{JtGFBp>s4)EJKIuw-S&Ff%63J%WRvE(t4K2h7KPN<{0TUz{V2h zR=c!Q+Ah`E+qP}5-P=IZjKYI0BKKrzVi$R8l2Qcya!Hn5<+iPIvVOBlay1!86)EA_ z+njNq3-IxlC!gHr*uw^P)@LagnWL6d;AEI`QG_-J|VdMW@DF0CGe#=v1pEuVA2tbt+WiU_gw9L%=29B9Uo?S@DBrm zzulV_K_N;xMwU-(@F)*jN;0-gBqcfKg3_16F&A2cIPj-75dKuIHEM(4^7Rf5-g@iY ztykZ__2wtdpv#xb1%Cb9)LF#!MO498gGZ+wo;Pgf>Vm0OF#0TDjA7|VGIBQr6a(-z zs!h5%IE})YGGUpjO|rSxjlp$g#=t)&{WKfA_M=-3MMH5n(L}Ur?L>WAJMmOVmZHkt zNTtIuO*kMU8e?`-Q-dLOHn(yl&CM}d3hpt)VfPCmZ3ieXR|quX0?_k`pa)_JONof@ z#-ci6NmKxIZp=*0yih(jbEqltf!D;=q{}hmvJ~+Eiu&e= zeatbZuY*{qGa`647V!o!pvtWSFuz&x&D`Z-E^4i zOc#=>m&x5&4@nIkNX>5#2&v_$xmNqGR;wJ5LgCHM%+=y*Jj0y=u}THjT0l}{YaOc| zKwz+2Mr)0l5x-v8?77(wXFr(zC`D#Z&z`+GNp61e^@*FuZ$3SH=H{{4557J@ZXW&m z#Owz*pZ@j`od5d7-%r80zn{8!jLe>yJx3N#!Mkw$0e%WTar5}>C-CM#mO z6EdtZ>^>YC+6zA~)lD6f?f^4sV(HcX{SR4$QB%X<(Za{A6w$z@Lb(>GULqzNVJ7m- zT3bnuXLDMKIQlRZyek>f`rm|F7PG{2@OVqd<1G^DI#|@O>koJQF~sJl#UgwOp|)s* zL}N#ELvH+}avdvenuN}Rv(ty#ee(}s!EJa|Kk zo_SB7MdWxq^6jCQ5~1dW7pg%eHF7%nm0{)?*-|t#5!0f2aHJaWn@Ad#2s{WFq3`9Y zp;1@Yv8yl0Z(C3tIep_Em|Ooqmxi4pJqS%Z{SG-`IHq6>qP64^y%_gk%N~6tVR7|9E+f24{wHV2~MbHsc0Lj$X}?X zR$f4eT(qVh(TuUDp4#wcL>+6eo!hFx5otAeLP^Hf%ET|D+vD3t`QQ#P1;d$xi5AGK^(abc@mKdrx-up-DbQ!6ar&8&iS_3ioJK6m@$&u*W1^VXS{?_9Xi3}NzB zTUful`r`aEr{+I>@76oV+vbmc_T9Bt0*kGuWRbWc+XwGzDMcbuBvz{oH_ZgEnNZ~L z3&&~jqK37z6C$v2B^&PY`bo9wm%#hQ3N;YSh{1HOI$9&}SZh=}7{~Iou*dHW1i5N} zGv9-~ojSJ)24R98gcM+BTB&k+V-=fK838zPY#5izjPSmf+NM%41axW!Hv%j8$T2Jv zauq>@1NDRx4vn@c}=*2W2uIY61~oVxGqP7590BWnc)qTIum#*5iSJdZf9}Jj4qH zo~=M+BH44s(Dmt4%))**n_J>4tONq9z#>Ij!tO|-Jye+*GN9&|o`wAgp?R4zilvlA zYLV%WY;&;@VfvIgeG2Y3eX59UCE{1qF$A~udkn!!c4&Pp!Q~>~S)C-W;9FwVw#UV8 zgA$BX%wy{wlf#Hoc*Ty#I!497S5wYc*M}(? zBZi4>@oJ6?v+*Kl>NxxAh-Hsalz+xFhC#|UQ*+>{jO!bpEw?sWJI2{pQ>2G8YrK>X zZUX8VbDSM$hkJZ=y#(`;cNLTej#(s-ge~N$AXph9U^mr^j_Fy*8VuFKDmT7;&oA2g zhJK;M6S}gk-DW1URw`)T&;T3lG|$HB(SEn=^)#&RwQXvWK6F+Dh~Q!@P)0K^Bqd~Y zi|$TJ2`Sz<*Qp>nMG}OI$a(k$DZeda`7Jw9O*cCBvjOk)aH5u60IdrIP(yCIi zJc*8JR*(m=z(8#VTgXM4D9mbiTS)C&sWb;`0ftLe)d5qa4THx$d-ik$mhYp~rWwP^ zIquz3agkq&1E11`=OkIXrdMpIY8f>0C8JjH-u~nhIW{--`P|`;uq$cmDrsFo+3xD| ztZ2a}w;>kv+~hNJldsK99iE$d_PfiM=O#a$oBD9>@XK>kH$=15Z@#;H=+2)%sH2SK zaIG>P{OY-U>8t0eYFJSeNshpRRA6mJs)CDzs*6&*_9?cQluR}pJsBF>!L<>Vdd#sX zo*SW-g%#6|o*}<sS1^!tzS%ZOGWzm+pL`la26U6R|aH!$jUj@?UL>H@Zh^;NCP& z_A~33pxD&lMwR=M3r~lO%E&L1Q6d1KKwrNVmZfmxV1Ay8t+eJ8bS^-P472E1q$6jd zF9e4$MOn6$bkG8bE|X=ObzH&fxT4jyAN9TT!(c0W;GF}t`BB(>%_W~!T?tGp?KQvB z4ie628^i+JMNPAmAv`!!Cu~TE8@FmCCFHRxdA`QL<(JKTs8N+Eowp+S*CdHZ?QxaDd{m@nYFTcW7@DpF|4zL>99^b(V}iv!gp0{!N9ba z2m$TprkcwcBO@!Uq%JSB^gyvPnd`rbU_>>esAJ)5-&@FX!`u<_x&Og|$$yjZi`#~f6k|T0#k=tvBp_NF#0oyz2Dij-`t*jx2t!9e+l4mDU zQLyzq97I8~r~`(cT6GaM}$cFId(LM+;t@-D?aFm)wYZN>ycv*FJ>- zmRY=EVTcO*F5Mnsrp}0rHZT{C*llK( zx2!4kOX*h?R6>d;?w-T9Xc{)CWN?>87rKz)38WS0r*F?-C%8k+f{4CKp)ii5I@GqM z$&3Wg8kpcGf(PNU43aLbS2^&r)RKsGlJuYe1#yD5IXXj>;U;x`(j^nYM+ZO$1TVLe z!9WAMIEGbME(^PAU;CCqL<)P5w|%GEelBnHRnb{O0DuFOhykGimbV?3g;wwFRL6J; zPB3e{jpePPF&=xJ4f{=vCNW(ej(SKZSMr2vW!1^797EJJ zaE{<&l}}&LNg^7G#gj@1orj^JF005?PPJ4Af(Y|rM@4(WYLk;8DTy`7NEbG->o!Iy*;eV$ zSPa}%<92@6pB#~5;f6b0Fl;w{(Z(8;oudXg8_j2?FGAmgufcd$C%uL_22&31(>89x zTZ^MO*y}n$@0%N5yP;1zTa_w&4wSnT`BxdZx$=Z#?x6z%36UG=kMU0Grl0Cpl zBpgl3iFlY$2ZVv{>p+bmCv;dF*kei*6DDCvNlYv|TD0Rp=?Ma@<(Fm_%;fqO+yru6&=7{AWhP3T~Z=9dLFc$bhST;AVIsjK0fd569l)_5GuvtyJ2)9uV zzu9Sw2w-g%AxJM7Zg>L%6Je)rGqVUAX{Fd03~jKBR7Na5&bEr&#R3pclWkFZM!Ct! zmhIHZI=q3Dc?UBD7{{XhRyE&zizY556L*sz-_1&d9mEf38KGJhd^kH-u*epbnKmEc zn8#=}bglGoGuU-CPN{%z;#MVAIoLfwc%i5<9K3iC8{nV{g~F*wM=I78*_MiiLZMVT zk&dL2+oH)-XCm2^>X7e}A|;uMfG6t-to!c892!bhpd327yVAW~{S*88Iy6RgX2QNT ziU}0bN-9KrOVuk&ecc^7rsm1gG46EzOR!Z4W2I79$#8>8H4Me?1oV0VCyDztTsGjM>5STxr`jmZ{uW)##7;w)rE`I$Q$2*L6R` z-VG79w_p^RQZ+ZcMFi|S20DM1mO~TpFM)r-SdkXq8QeE9xNj}PPxoi3;}M7~%i1b@ z7&bnwaEKE^tP(P#2bN@yx}9Y*Zt{;vRg@9`qD)AMM6C%M1jB#+^H1lGzkloHFK?fI zWp47*ThD*^{h{;CSaX_;CBL71cK+Gl2M^*SLkE^xM5Khfd?_&XUlO!7+`_jw>em1- zEB|dmLDUrgdc|=!MZRI#6|wibNTL`ptgXz}{p!0tMKmtKb+LGck^<~)fBE>yCwJ;2 zFl~7Y3`_wn2&zz{CYgYF(I0v8$$_1nUA^g^-lv`-6T=*Ta6VZwA(v{#wWgZzC>d5h7+(j!V(anRC;xWLJ*aT@E zF!V=ykY434*VZ#Z6xb~pz|v)h{0e-kEVpvOF+RmKkZlXFkJZN~nXD>V@!IDD6qlmP zT}7N1_#kuxL+DKt$y^nuXB(roW5RMMaur_?u-}kGs*!kRS{pXZahf#^%N0$NU3H4S zMU@17QES@-A|Jr!s>^~_;TX0diN2yHkX21f)$8Ri%Gb&l;lJmrjZ~Kfr@$9Dz?l~s z5E476!6e#NKH`eEv_`}A>wLp2$|)SevV*v%_IhPT&95e>FJ!*DTDw=mbK?jSO(#=2 zptDNn(-``ic)s~!WS9p>sLT;g2_lQo)*dWN(%^`K26?DXw6vwWvvxOKm@M+B!AA}3 zVdTZ?&YB2IQLuGFk}NA#Au3g8Qt{_Zq6{%|UF%E+T5IpZN;?M+<00{keo?+e;OU$a zR2bKm=rGE^Ex(1A;{V@i#-%uO7{+uK{^F?4!stFPU)qWzI$ORjh+J;q!6i8OooHJ; z4>NKs@bYXovKguyqpdNQTxECjx6cv>lx$RrH?Bp8hRQFN-%gePSbh)m&~MA{O$e!? z5}E?rr2OSV`zkWntQaGgB?X}j!(ao-k%{Gp)hPKBl_>f0iv@ekB$8CX7r^&i63-Vl zj6Mn`6}RF_b7kSn?~3OKS$=R$oP*bg%h$K!!nlDQucZ8vc;f~>Tr(%&^?7_9_zBvh zvu>epWC>!L<0bLoME#af+IV6%qtY7f8ZQrUULB zWoXdma}*7@R*m!1$2R=b*RB2^aK>@0q>Eb&%v9DuS7Ca$$t*@qHRlT2RBhCVKN}SV z5S1c|2k6Fcb#vOWM6t5&=vixoMY__fW;wp!R?Op4rZzXoOvk}9#dSw@R;vqsf4ANr(rvoM*?Gb79NkQ zU@bPCtI>yn$`VGKf~S_sZ-7yGL3kfmQJyaIFKERq3L+*YJvEv^@)BL?fx+ez!1o`& zHvja~^Phft>#Z|;Tjj*u)TOz@hv%j$9K)5M{#l%?;5wA%X`YviP(0)+W|rOERlXMq zmi&284gA)xi{`4OQuEX{u!&U6Nw#(A0yS#F;<`-_@(QAxVG&k24?bIosI)N5;T+t{ zSZoPymK5$zq(|3D+QRb9=?ew$NHFtBI;T$uFJfDoHn*3xf}6FfpA`;Ek4tEeRuiv{JC>5$l%FA&}6ZQBq!jn>`Fv6;CF>%lH%dT&HJS?0a5*M?$1qJ+1l{G zvX7_gi+K-j)HmJUn(ve^*xU?0wgZ!4T&K3AZK+YKTc@T63=5An{HuPNgBA1E#86LnZIOPnaVJVOGJE zvhZQO1+S;0X2E;f&~WB$2dk&d+`3rpu*JTY77S{okmlAj^}r^HhB?u-cv>8W$ZSUoo0^*mwbuO#?JWMvYb#VzJp z++w~LH)Nks8)lGi+I-TWrKRYzYnV_{6+x;YZFB-qay~=<>jRo=YtK4UH`!86@PoC! z*h0_I+Am4%&Is)A79IZ2OGb(sq(BSs2-RT?WGhHpG^bMwo+7$tJIoFa$AP6vl0$+R ziZ1fj8cLoc7AMrOL~_&4l+8M2CUVRsKM4xdJC4fzA7#u9k;Z09NIz^#BjrVw@mi1 z1j;DcB+X#IhU;s#9eZm>FX=Tg1=HIASi;VjISv@G0Q3C#h|7M{&GBx~k1 z>{vnW-tALg8rTg$1~f);npHkAgQZpDUz%(#VOY`wuVQMbi%};Nd|~s9UB$GjD8@W( z>3HDNyi%P;rN@724LDjpW10E2Ystj1$6&o-fCw)TV6(3_T)=H|*fMiuf%mU-0VK>@ zGi!1&VoV|ofndyD?;T!scpf?nDYj{zo+G*JrEVH<6a1}#aX|1l7V);Jo_w0>H=iaV zgdrNL&Zj;8Gpv_p+y;adS8sMnq7eQSu55$UwrOhZ_#SZN6+y0{aq_Ldv1gu|%vrs< zWdqGLz7TK=iW!yhuRIr@dPQfX7Zn16Q#f_fyaRUK0)xMZUtjgCLB5wD3Hxdq)?N_j zLzw1`V5}VO5eAqB{I7mCV(JYv1=qwfZDRe_MRDogMKnfCh}O{BJ^t%bKTGGRW2PD) zP+h&1NtWs$LNfvK@VDQ(ee(9%+wb0f`}TQw^wwHe8-17ozU)c(y6@h8t744GbMa0$ z!H~okW;NKWEbkZ0K0Q4sBI}54gj&3xi~X%=OCH+;?B{T zkMBJH{ecrRpU-?ebBWwJdgl;4bL7qocitlZ7oLNEm;Q0$&M_>cz!~RfF7cmV;Mwav zZwBA<;>=&k|Gj`;7QcGMtTividm&V8OVDh_(Hhu1BGtbgi;dp~uV%w0lysa^dbYE#%v)bk33;jHX>iS{J@&oeFZx?yVK&t@3s4){@BB z=@B##!TUmOa#h719d9{V@zj%jyAE6u;nqQIYunk`!dnh;z4G=umaT)51wTMfr zwSW;2eN&@*?K@HMG=|3nAv1i|grgxf7LG(kk|-?M76vzv7>)=M+(gQ__n?|MZPAJN z47P(grb$MSbS?o7aDv(_Mc@yQW~f6jPA9FJfmwZN1A%c$HQ6080K=5BM`+q0q9g_1 zEG^1%YzIrHnd3zz1{I{o+d>kg$VzyldRfl23dGKUt&B8ohF$F)>wW*LX1hyR6t1Ba z1sh$&9RuNc*ahX%nqO zmKy|&v)$5c3VU_dGEr;IMpiXgU(@Tq*`!V}reeH$m6}1g&q!P?ZuW2S#RfL#Q!>ANM z9X`QQ$X(8iHOEE!Fu<}jIJ-uerL!~y5~IwpgC{9=Q=`MR*rHwDqb?ou4MtEAWcBL< zM)hh*%S;v$Yd0OFcqjxuK*I&oeg?M?;*{Y`$F3Xw*qcNVoN6&7$SAg{jxRo}4t}am zu7p2CM}6>ljV1U?J58i7d_)FcEpP4_GZoz0l`CBcdF%IEiOcj`fvF`S+_+ryV~}@n z)26M{(Nf7`v&LnXSwYv*l!!4kx=D?#JR1>IKnd-FwLRYyZ7o~elx-R#8VtjtSclpg zHRfp^P+Xh*7G`6+ZJRJ1IYI$T`R#ZX=`WuKv#%c9r{(Edtj63G-_B-{3jAx zI;fG(n%U~$wHsAymXc;3+)`tt!3@>b4I>zi2`SMT4j^AX89?csIvGD0`fBr3#-nVV zuqeb3hOHu9k~eSeNt^q&4FtTQhT*4=`$8C2H-*D%vJi@*AlC@GxY-~S8CN*(;RGyv z#rMj*WtWlfVfvlvSL4M~{M9Sd$Ka(S@Z#6g?-0&TbrfDN@nDoW29fLQE5bkW8~Aq< z{^94mLdpbT4@1`e(GYUIf$WNSvC{`g@pVr3aRZ0pO+H9R;Q#M%28CeeAD~8odOEs- zOD>`PWg!%;E+zZ!GXCkx3k2KyfBN#sPk(tg{`1?XvBUW1w~qXD_B46$_Rask`YHMO z*ryMUz5D%vSB8cjynm$ir@!3XPE~0&5SbW>{B-n4@HFm&w~svd=-7ippTgjJ-Ou_hCLGl*@6gWk_1umWYv|*lu>-rE=X}HB&c$A zqSn8{nU>5Bn#kAOPm`LFZZabNAq}wN4!dH^=N3XYai=F?aB4U>n^3=;V*LPR+e@ zvSNZrrDPxRM`ex%kcEi2BBU9j%+B0?^Y&}E&y_G|Yi)*gi8X7hkEd7$P9dQlomfK< z6J&FrdNFD0j10qWJ@qlwY=+z9{{42_`0uv$92NwT6T)4+jNhY~IK8etb%CM& z4@ofUKHEkuhcis|Qp;xLG)};VC1h){neOiMSS_gL2U;yGgln*yfy4puz`}GdfdGHl zL{MUN?@$QHmSGp829*#z0aU~q#6OPW%4u;5>Ww%h7s;3nvRqzYXzEHWbX9EHVWu~g zvvr1x&>vC#&HTyn-U7D}{Dpa8v_5PlNjmZNJssolZxvG@k7l-WA{!Gfy@7@WI_6n( z12d_?y`c(Htoq&vvQdgrlA{`8{i3)f(tW1ufKB2s3#KlhOd*SRYkH122=tx0KvJj_ ztW(knTc^#Mz*hY3@{nMcj^mvgTuqK{+M?MA&xO^2Z>dRZ>Ox7CW>dSB%xEL<9F9$3 z%l4Uij#?fK7YOA%`U5_}H4-wig~BUA5DqYk%8a5ECj)t_#MS^)vyBX9T+2?;0*MN0 zh~$Nwu$rs-BgsB)cas^0T;>)VHFY<+FZc}XgWEr;G0!ZND#cKI=+8`wEXa|^P$ptC z7hP!^a4e=Jnyos{MOALV^-PfBYj8;irVnFAe!xuT1?w=6JdHBZN!)N(}7w#UqHZ(N*(wW(#f4O_$(7Ior z=N$vjemd_e+&z36xmFJ!o4>+O5xRQ##k+^T<`~w`;nm=!1&2``oE{I-MXtm;vOh^+ zG+$H*)g;p%e~=LjUk?60e0!r2d9&3+8maY=M!w^iemKLz2w>_mL3#QPi;lQv+IVOV z&8l@YFE&R9atz)K-mp2)+vnNwJV}4wS$|cA@vp{cKdG**pt^SIHPTwX^an@Iz$LA1 zflXBfaYbCfn@8XwrV0iNd6Nul&w}&05nQ!AvY@BpMAx!9E0FJ>e*+&fz3>Pk!bjqF zK)@jKI2$7fhjC5Fz^2@0X1JbuKclHqsiN^HQH=^>xM9LzFr`PH9D+nS#ATk>w_j1y z+n&}K215<>2y(r4u)#1V=xnr5DkEZ2u2Bi30fO~AM`teIIl}*aJo5>e`S8y3caGk9 ziOhU*=g6IdGhf_!>CQ2D>ZO?rYn`AAZlRe^XFkSDxy*med;&)Y16vA41jYk<%CW(G z=F**G9xtylab%w}M@DKvrSaSwE`rZGND-%#62oBAnPmX(dz#16x?z)U&1wMGk7}uu z(?)K!v+%&%sIF98Z^ftU4MvFwF{R<8*JH+B(+3}^B{h@mC>Y7$WOb{^VgrVW32F_k z(2`QNwIS`EzWBiQp`nc=YnoQ7U{I&9k)$j$!BadbnoXj%?M9Nf#tM1OE}%D&vNPIn zvq2_z_H+_eiUlW4?bEU}WqKx%>|wa!kNsIn1n<2DJ*7E*>1!>gtj*h8j@cm9IlQ|8 z=HSh4&eZcBKF3PN$7dl&MmE8yRGlRBJw0#((a0DZHU9IYV~rX0sQ=gVJwIq#?tdK# zJn+5LVXi%vhUI_{(8Br@^(;_6vj$*-m>@-deLS^0n45HUGjBJ560DoDkt^GS?0b8V zMA#mx@d1zj;BPVY9MjXLv4)MRGJ-Qga>K#L-n|%mfLH$6%;$HG#y5j4HX&aVJaZM9 zoc?hFX`PV3>FCTg@Vq~VmoAL|;u)m%&Y$o6Wy7Ni@K-5WbLaV)Nr){l%(`9Z^U{;Ds-#0q3ocRRUqwS@ZEw7 zF>Fsrswjs%BOmghIQ@>h3=w^#asA>lueMFo4(YH=Cv8oAIrRbi-(-|#EbOMy3|&iR z2+NRRc+5x#O=d@_g}4nyC=47)Y0PG1Cv*4VTWpF2(T#*AAlfoB22v@GvAjdz!$#q$ zF_$FRC`0AvCi5QVuP|Ze^TD&-+e`0SfK5f}zTfgP7`K z`iMrJfq1w6;X)Pq{SY8Pp*qGM9{3M zAEzinMD$&jS+c!*@7~Rue?K%t1VJG1_}QfH+S&*sENL2MPD_$8xMAx0Wlhn|-A9w! zFkD4H%b5-%Tgy$|y-1@5+a-s1QJwUfhFc%xzOo+gN_6BqI8q{kx~R4>)7a;>R4sWA zUOM;l^Jjkg=+kCzp?YxW(+A(2!7i#l{pG>??~(^^pLuZd2>I`?CVx75ppLZ^wc1)R zK)Oy7Zgx>am4wYyVpxcT1+c)1_9rzIL}9&UH`BE&(u|CxP0LJGTdw$eaT2?N5$QF3 zq&PYK>JldGucc1y$9(Ymuv_u;^x^5F#jDeYidRrO9aYpflHyy%iQ-MTr~{<<20Wac zKFHbeH%C!?pRX7bWcuLr zAr6QA8kUdeixY9Oe}C~T2dy50rSo|*4(ra%;?>`hg=Z&8@q_8Z#Y@E-@RoOrSK$<7 zVYu;IGLEmp=N*QZCh+heo`b;K-wwdoIiwHwM%QkWgnjumffJ75D#ll~iGN!-8~)fh zr*?pa&HL=4Z}9zrkJhr@7}(iea`=MZK^piMp4_n1s<$zW<2vaQptKnd}h5AWS9ztetps3 z00iD^sEwf@?jcme^d5*!8pY2c>~S&-#-RxtD+icKo^g%fPQ?IR_ke3~*QikdmxB0) zSg7GNdrBK2%pkC>fG6uFof530iBf`{rjfR$&hsA1ex~Ju*Aly^VY&$x^*DA)#SV&Z zLx_?Ptzlgq%lMxO_fOrJ{rXZ9PJpdFq)(E%E?t`a{QB%?ul_JuCmHk$ zR;b4xQn5kAzYU^7w1!^H@VguD%I7WSIF#60ijo{lk-zxSPp)9n;>Ok$=9 z)lFpkxB>tZXZb&By}ZZ8NJj=W=ep3{^v2mk{cDOa2QB!ZAbc}V*?KU$! zXunFm-Y5H`f{L0EsXF(!bb7uWmOq15mB@Hk_V>@fHV*%=54)U$ntruT7V2+*Exv|1 zZcHAv!Gj4rr9_MQP4O1lV_JF&Yb<-Hm9*%vvytbZE)}n0qMWmZ-k3f?;Ei8%a?r`? zqonv1{DJG!2d0n1A3F|DVA>r1>Q(p>UzffL{{7;$jqrx6#cP=Mnm(`*lVpeRo!=k$ zsO}QovixOGL)p1C)SWxbfTW(TKm9Yb+MpiO7gE%r*^?p zBv24gAcV2<`GUQF|K7a|&CdNLd2lXcX22vGVZ=^q za5}GJ;!utM@+m_@u$?fI<3h5kw z?9<+1AuLuyyn>OfH`6TcYaW@wfbp0dgAwzB4>C^?yV z9+^)3+_Z)oft{vN?g|ZF<3m}FkSc@~A?$%o1a56rkRL~>aibsqL_{xh%)FVPNqFbl zu`nnoa2Ts0YVYAe&K4xB(5H#l&%nmsimDZ4(p1HQ4pXf8`v2 zzm%lbnx=16MUd3$^sR5|;oEQBe*5;>+vje70u$l($G6|T{l>i5ef;((w@=+Z1BV}w z+kb{<&cnkqw@<>%x&8X>*KYsm_DAsa+ogB@`Sv^T`djfHno2XLtt3Fvq&gIEWq-VV z8V2JN0>g6wuKZMK5Z;DEWM}>a9{mwt^Nb3C6HgNumh&(yZbS0^c0Zm~^Dy zGb1Z-CB-LPNkO=hTtjK(8{e>#xvV-r!oF?l@|ewrNvAz^Hd|==Y}3CY zl+Kwt=fIYc%y2~ACU7LdHO)H^V!_onMAy0*DGrRZokc!9vjmi$pWP7;>@4EnkWnF` zHq07<;K2B%O{1fu0<|2?rbZilQOkum(Q3<6OS1(y`C-}e;pE;2<}U!bb{l9yZBL}! zr0JnwQI%+A(<>eLv`tKi##TaWv;&C{OA2UBFrlhp@m9&5E2Sg%P>XEc8rT>E8XL}P zMwe&36Fsb%qF>rfH6D@$C0d@x_FQ482^e=O7zdn0<;@)eAH^Ca7N|?_QF|cLo zexAofO8s({vP?Zo1p~sgYZw<@Qfb9FERU?$37wGpYqx1ROmNV}HTB`~ z#fRXWI;K~O6+-KOMYeXK6lHdHboY(-Gmfb5MyrrM)7CKk{V(v=LAZjfy>b53ojv27 zy34>vCp&wFh6K_o$%-n+qz$AMp4`w}ZtWm=IT|&bjAiBvf$yElKzMGLNc|foviuDr z)xkOhLEdhmxrA$_VgKI_J`G#dt@9ySqJtT(GX~$dOS7}pEKGG?j1NBDF%Ud;Z9gL< zw|IDE!X9vR*e^C(ML?EVD)AGbMQe=?fvst1~eTk$1^{lb} znt|%g8qZ|egZoEk#tn|A4oiY0DcHUzL_PcsVcDbID*I&4P?3{Hj{f?z;s>biXu2L8 z2en#BFc^77P$DY{#CSSjSYs*TFq&%$FwJ5kJiWQQoose36Y~~f^6&h53XSN}bt64L z-(N!>k-#>oJ}tep3@^RPRh*QDox}DNvego5lweMHrRd!yO~clZ0+O?s8EK)(Of`_9 z7Tjn#mNTv3j@WiRVU<2^AzY4l!eyVBCKYLZ)D;PAJvMBlqzf#O#57Ejv?Ji7kt+0Z z#H4{s(JG=vMD^FlE!SgECu8QRUa;3v5kN&Pq*x96=+ev>-5RsaDpm}bB3m7bw-edf zKDdk6$hrc{S$#0a*1;N|3*Ws~9v9{x2+F%GPD${m;ye4&Zk>gzrMrgaqR%o%G_3QQ z!=zg`=K&t`$PHB#ln??pMAa)B9Ibd?{IO4yVIM(m01d|b6Iu{#>9r#*GqUz1Y&aa_W8Q= z7N+MPi-K7h%X+nV4O^Ov*Ggbh-tZfE3d11c=N#ZlI*rq*sfZ>~Ytjt9ISOaa-#Ipe zmCcn>72bKNE>)ovZZ@LAnalq;G4nZrfq9w)EM_!2vVhz9}P^HT?vsp?8 zdgW&D6ze48+YLu+((Wk2 zbA(bO%|={3wUOj?N*x=^Ap|@VOV{?Lm_h!TFdla?9(F-E%;0?Zk-~OxJWqolc?7eM z4(FF!V{*>m;<_UYv25M0<=72K5cFxx5%y^NvdqhC@4>XmenD+2BCEoY8f2AUhq8T) zOkEfmHm#JaPhHNEVN0{^sf*6k1)C8VG4P}x4eFxd{hxznrY_V1X!vvSsTVz0mjy51 zX0|hR)}ou93T-4$shdJlTPS#uvfbfdyMlg-g~A?4V(G|_0mJc-D2Sm)C?T=#*U`Gd z5e>9R)Cfn&o(U%Vw+{qG7*;n? z_53NDjG9)KP@5!m3V$57Mw>BGMz-ZGb6C?^A|)hrGi~QhhXdNAP$;%3q-?Uy9Bbp5 zR+`XR8<&f-+ccvMPSSLpw&7R$H%SsQPVQl?$E#=kZcX=So8nQtvveI~x0b|qC|Yi+ zVPwG49E}WQOgEJ=k#Q{WoxyQk3Cm4-K#bmr?)%md(Pyppi+pEhkkAq{F<&9YWAWU;5n1Cyp?wX$uf)A zr#+CKz?tVT!ZRYF>Uq30ckBQ&UA<-TzWMV@*W&*@Iq~3+pKg+*kPrkr;PLVkMf-Jf1P@zX~) z>*Z3LhMD%KkFWiFxQ>iXi&*iSs>XSqdsdPKEMAK-!PE2B=0Q#_itXiPEv^c34HaIH zBHer_Ql=w-Z7A(xt!~i=tm}n!0T*7C^IrUX97PtAdb!~!33m}9cQ#7P7LJX3o!C2y8O=Se^?le}2Vqv5MVdKC2huIIMrM@JhrX7s?Qw z{JtDv+-+)#y6m!z8s+QKmTB{Achj!z5GfYehA?ZBW|HRFqE3ylhsGi7m&PHCEIgr_ zopac4%{J?ftp9bgnw1$&b;{r|mI;<0;{+`DjnZt`>T<2PT=zI0~Yn`rS2yZ7mt*)PuCJ$Uu* zp%Zrx9=dz*Na<<#5s~1S# zwQQHAnqdl6v(e^wGo)!?fcUhbCE;y^8So3PFgOrthpXA58j6Lyg;%jY23%YQ1P*qJ z!q!&pRQS%FAb_QKWBQeOiZMj5OuyP>#wmUPU;P#)HcC1m)5qJ2Z(_z{J{1(a#$Zv{ zC_I2~kH_wJ0x6;(Newqx4^EwPv~-qf4%u#D7jiO_gB>iz9EzM%%z_PxPAf{cwa&9l@go|R7Vz0fJL5UpV}@$HpwcXOuWxFpBn3$9H> zRSJb;vbvG%GmR`I4$Eq_U21NLi7Kk1I8ZDqsPeey@z4-il>G9ZmSwn&J4|Az5jYLBP9%v)raoJPnNb87eVhvl2MPtOK);?N?h3^;ci>(kZ zSeol4;S}p9y~JgcUP@SyA`K_Kke&-R2Axx;mI(Ngrtap9X2YVl!zI{QHJ}YBGY;6I z7g;37JjH>SZ$D9q3ej-2K2i`%41zNc3p#g2lFZ;OXZ?(qdQv1|nOV5yT9uHp5o<`c zn`aihz)~BC#fD3%%humSnyi_#boaGo5OSV^xuofhw0i8J1sE>solPF>tzYPEIIc>9 z*f`k;LSFLJwpVYHdn6|4z$RtWNVg4QKSl#v+>WWsIRm4A$$SgfcfVdGD=>AFdspJ^Er`3t;&~bSu!0u2U>f1PBWm^j^@Mk% zQOlm4cyQ#>gZCy7o*Iq)bo5AYA93{|+)a$PS96zR7ZenIg=CTL11cu9wqEVd}d^ff9{>AjA_nMr@W# z=#;ny1z$9mhdB#A(bQ$4!M9F=mp65$B>JoqTY~^2#|5D0|AI5dPZr33{KtQ+dl$B= zTmzcAyeR!yelCs$34%PwG3<^#qflzHpT1q9Zqv%qCL~Ls zdfv6tnw`}wOtH7N>-I3yQ^BY3JW4%7-K@7V=PAtj3@{>yZwM(36OR1n%Mg)Iea$(v zVcrl!OJ*FB(ip+i21{l%UQL)fonV&DoG~mRjF<-51;_c}sdGA(R$yRK|Frn2-`1Y7 z?kC=2YuEe!!K}P<>H_>dmh~*sYW+o8UEacBArf5?!0dzKEd-Q(!_$g~;1S}ozDA|t z=1XQ5&z96fF=tB3dIw*WG#{{q@j4gcEw1Rf3B_TtkgcxbVx@B4fdJJA4jHSXu+zZ)CBvH8cmc1UVKRjikJ@?0QU${Q zDw2k_me@ZBx_;9xhQ{GHf!{+z8-(&JgL}pY_cUK|b~`JO97SvLyWU2Io#46!+qbld zo{C@aF6juW?8stuzSMU(&NsUOEp1T8wG0ZexJA=scABIYADI}BqL(6*4y8HtEuL~v^Gx#|4x)vpUV9o~SX{U+Q zmy32xDdee)gnXM^!DLwVT=nhWV6lBf)7^aDB)B4%rY(4$Im&tNu=+bTRu4S=C|Q*n zpiP9^ElXj+;Z^H23SpVVfdL)SY;D90zNW;M1i_<9NN&MAv{6zg*)OUrO3M-U@XtI-{V zk+(BLA#BFo+_7PkN216)TcY0EU0Vh0w#uYyiCUBB|ZTd&Oh>5umhoewMoUnq-Ge_fJR_GF&@y&+ya1LpXWA7CQBc$U{G zxb`%PW0#)40RxW;)sxdN1mAjTijG)PDCEI;Hxy~AF(DT5t^QrF1o0-nkS`Frj)PIU z=Ppzj%n^D!w2l*26%yoBNXIB!uu>?!K|-0!0Z^@+;WV2~$JpA|rT-n+22 z5<^kst*IrG3^cpX!VP>J=IQm)%tfs=a8ziH3YJUS$n?uNXR+9dbAAGOOp0G`#Jy+! zZ4+er1zvsq3d_!=`@K|-z0Mboo4762ZQE(-f4at3@5d~_`he+`I(K+U8L3+M)dx1V z7*jE26GfHhSjE~(B@?0eOhiPHi6*L6FXg_Ak2cdsm|+yaGpD&Wte<2xk~OV;>wXbl zY?gkOHFav_rp`eil{H=6ZJ-Eg=go)*GAm1!RG{SB9EzaIzH|635X6pXdM(ZN(~hP? zbhNaL-uez{|Id51!w+$4%i`g0*+N~%ZcDMY0cuk8CF`E#Gup*xRu2x{KxVHVdxp&`XiD6aHpWgJIQdPv$jnxD-Vy7+L-G$%U8aA1cuJ(@Rnh9HxRG}?znFu~cNOI0Kv?RZv z;M+~L=t%>b$Wc@Y5^L0Go>0O!P6>bh&Py;NGuMzyDX>kenBW`GosV+jmzpN*fZ~&| zJBb9&b?WL(mOFW+a7c_tL!o$7gdbvOtRvnPjVSSMX>&NfIV#5DU2;Sn_jg)#|F6#v z$8(2tq!7UmFmxqy(XfjP=(+9esWK?r9B@WxoJ9@Y7+|}K^wR3Cx z4!BD=pWdiuBi%59>^34M15aIG$C{@}S(+Sc8#P>N6^N+*X8z=OZ-Lvvqeb%YiU*J8 z4(ia+@8Zr^E24RV7f(Cr*gkK9$de!Pd*jGhTUtpc^OGCrk>5BwG@X+6T+VEumbmzq zkt?Mn*=H>YMdXEWtooLD{C)IJYFRXwFoUC?SIt=f)d$I(X&Ca%>hIs2ewOHwS&M2xoBJ!1yQWdt@#FI zby`yw<`W^0Kmlg;b_*|v%q8G7r8qWx#MH^u83Nk~ayzMktE0=Ab#sEFKiKyS8I#)DxGKDScO4PxH|jZwpP> zD8W5SEhji`dr-@HmCS-Gc7UNGf)uR=YzH9}x_9!%?3pV+T)%$r#S6_hd1k-)>+GS| z?;g5!_wYs1diT(u?;bvW_wd=^NriSZYc8%@hbuk+f9%s9F+r}u|N5RdJCNhdVtJ}r zae~4yWH&Ri82lq!^@3td+y7hrT18EUw85MjjWZ(-6RUqm|M)YeOJE-*8K&p|soA`U zb|aO6i`$%XOZ5mcU?w%@6yW*=GQg5#Aek{uJ+Qg=Fx}3qpk^0CgppK3gB=9u-37C) z*Gy@{ai29@7nloGf5M!4tFj?k0WjB!-1_LMpZ$K3bB|V${xgb zV2vxVjv55--Ll0^*``qoS=hT5DGPgCYQbBKx|65n>46%nJZ)%Ltl+q4`;nv%)Uf;u zkZqN8xRz_+b-;aQF}qGPbxUuJE_kn&%xG!O&oghNP*gyqk0?h3QT2El9+;1BGew0^ zbR`M=eiO;z&0!)-O^4FP(fIzrn_`$ey}7%c>_P6Sr|g9W5S#2^H0?If4F*G*VOpA* zR@&3@UU36`qJCls!V1+8e&+plgY`|FO=|`vdCSb3wgV4vpDJa+`^TBCLD)^FKA!r} zB-Yf~yp}?*uoW%~&XPhKQ2P%XuS)O07XA*`!Qf-u^LYL5XY;Bk9?6u2r{e9F#o$yM zMH)+tVaRg8bd#h^n{0+b!{4x+z^}mfK`({9v6EVw$(02b>Ls4WK-Kq56(u30R4=H$ zZ-w>|%{GZ`CSY@*WRdApa9@;fls~jAgbV@4W@zXSEsx3l^0e(voduhbGEW+eQRU95 z^J09PX;7PF8)&tue}jk-&mB@a@Og`VLsQGBN+Fx4McjcK?a_CTLtYDF6s6O@ae_jfXrHM*HL=|ACRXp%bv62H5eI~(T zs9B}dXpslmXSZlJidNz|%@-u@WVqZonc$iEPGrx`(tUPL(+eK7LM1fx85qX6=2Wyg zqmoGbZHdCBiZ4m=Qn@SYN^t^nUQK3*;+v?Q$(x5JN%14z&c&OGf`{u_mdsMi9PuQ& z>gTzZNpr;zXBH8wcQwOro5(-w5P+{<&-5d4ItS4c3q0IAsX@JH9Zzet2*9v{zz1iWcVkkf}}(%wh4^pk8vncjkJ1@9_87KlXGXz z&VBLC{r8(OfqUz#`={TTd+ETBXWxV?xqtG{foBO~$qH&n@hV(2#04Bfsv@~&Kk)4* z_3=QLw@Q@cXG@5!dAg!9u&XcL*T1W4XsF4NpaaZubC|1!D~1(0cnd{8cw|U9%sh2O z(ZgCJ`em(&q4ZWvtuBH*{yWPc#L9W5HJ}HxdK-%(B8|Sr|AJU5D3lkC8Vs{Ts-aY*B-NTpf9=>t+@T+$ZAHI9|*!*iU(rK4}4lQDuikX3|Ig-L=QvPY*?es=LvMUdYW2|c*z!X zv2x?tN90ODPxf8WQ$vDWGx)wevAI{?n0@W#xl>o>-f6b5{Db5mBd%la8g+q8&7JxJY%_c0li3sR{Ljr3v!A^?_s+E+CNKVY z@$)7#SbKXrX`Ma&?)|^M6&zkPcl!L?yKl~(y8u5u(@2g)JUmCGBcB#h>JZ!n;u(a2 zpb67j=v!V>a|CA>M+wO2fMYTQ@(NM;d2u5*+zu05b>;OVyy0>CdQVvSX2nrln#RU06nf|OkH4Y9dLEj zC8oRh8M7Jop00(4=!)$s&&3yCV&I9jE9uY?CT#KgY#xWr_uPsJN{wucZzg@NSVye{ zgVo|;H6lH znPabIU0@ZqjHy>pL39se!Fg>uY2E=_Z-JpVlO76f52Dg1o)YXKhXh5ep}-8Vy|;>C zS!@%-ir9p@h#eG0Qb$XqT=Ki!TemV|Yg6ZwdT>DeE-gnB9wfXNkvsrbe`t8L2W%lq zb+dwRZIt$OT7z*mN#0^P%_QA0TbP}!r;51V#Zb~NR^$q7?Zr5G-`cO9(8F^9I0^b`;b*{*s@Pr@uTLTnXP;1%{pQr^i zsuu9!&7WR9`QZ5bfz7&VY5{1|un?(P*L|m4FgZpY$2HRSumzWeJ+mo@39PIQql6-a z&F4{XVVYq<1WC5;7B-SWrs+fsY@&7}Wig83k)?ZG>X^r+j`?<}m6#ycWH5us4fa`D z&LkG^0&Y5kpll=Mo^-E?dq7Gfd8+Gl#Dsk!M{Nl9b;h0Q;0q*-kU@uC1Ke}!g{FDx z&HjtcDmH&M3}==xV>t*(+Lk%WWna4}?~1{usSE}o18 z#{;}~umN~`hnvfJ*%X7e45Be(f>=Y2<=dD$O=p2yBwnMPY?!hTV+48_JmbS>Mf9Td%ezzvRlecn$HF=KWMrs#=;(2AQdc9$>s3_KgJZ}AyJ4% z8F_| z7Q@*AaSJU zuujw9RU~Kw^oIk<%qUB-u%~P(dsb}O|B@}rf+{wQuK;WAVTt)Apt+pac8C%a6oUHJ zc0-3vhFQU-1gS z*t@s*M)8CA_s_pJ4*y^iD!$Ka7!%X4Rw*(UC*tI9f8D>o_%2WUPfQ<}K1#+(ak6+5 z&aL|2Z%OgB;@7-X@HJ^gJ>lYw=~wY{H{g42@yfwv6yHQOXZ|6jqDlGP3!hv322Uki zi?^c>437AlO_x4k`LveRC z*wD~MGGbae7#~=&QLp&>1ApqGj*BaJf@R^)Io}^R-DJkw$+8v}7OGe^0-JFg&1sg$ ztrNpuPDtOefgBNJshW@NNk)1RJJq#s<}STBJ9%gg)6sHFki!kj3Oz}3uFWz{S&ktA ztTuJU2~N7O3JFdnB*c{Jt#wIgMauvk(b76>EoRH&3;^5*yV^_P|4tVIN&=Ut88l9w z!vA4M#tto=ajP$(Wa>Q8bOQI@m`SLfaItPcy66ZaJruqY-ge3PCiF2F z2ZqfML<6V9)onPUPS;>)rFoX%kRXa-Ar=)R4-=v2p}-M+a^T=yBMH$Oq({mxrPQFN z4^K_*V<`r6*pmN)px3;Wu0mzCWLFuX83KC=L}$o8-po`TrkbX>M@`hPFpWUsI*}x%Zf&=@C1s6 zB{NR&B%;*YO>e*e^sAHF%;yQ^yr{bf}a)W&5v z_fX3y`Je-N1{cF>FFJy4t&RLfT<_(EWr0$QE+}{* zQFmE~>5!d09!ErqdMJ+gfloWCLQJf7U>|R8c3X3(u?p<$jl>Ho_>baBY6x@#*+(}L zd+J=y(Gh0OG?XDKJ#GSQ_OfM!r*ow?fKA&i=8nN`1Q!+@@8CkboS?StaqUD8QK;xM zZz>@_n_5Gq@4Kwa`Iru3CmL@|n?5*wxKcjFvg#zKY=URL;8GKn=gXv2%|>upd=tL* zO@7f=i`V(}P84rUpTOR`;~f8aVUuog*`!+<9T<A6%Zf2)|#N`8ara zmF6_T&S-`~M?LA%m{=)-^T$5z0(+yl60N!?dALVD@pr>@WlgeG*Vu@?;{IZ#a^EQa zZ<~k^4pn1RAOCu9Uk0u%w}!Mc5?3!&s-w0b6Sgx0fhf+cVJt=AoJ90Wvbh#il>gn% zSf2DUvV|HDi9SVk1&2Otp%$zbozy6Jd4PjzN}^R{AyU(8`}og?Erq<9HgoyR+Q|Un z;HyG3T#e)MO~bf<|K7a|o5kkrILi(VJ@*_0y?Kj{rp&yVG82Zz2+P_qX$tr*!cv7~ zVV_ZGu1WKMjs)K-7Gu91VrVuqs6{ ztF%-`1J#yeQ4|>z!t(f{Bh+KvySI`ZSbQozH8ivV{u+a2as(+2bj{8+n~T6~hc>WH z^>p@ma=l`GQd)5tR$7U3($(a8pLmq(`gVE7`+VHC%fczX=?BZ0aWbm8lE1M&Jj@P<`sGZjMcz zg(o;!^3-KZn>w4#!rG9f`J@eZtz+6`A03&x=)h@awyCNMz0?@C;5Kc~F^lQJXI-c5 zS~^|qKY27Pd`~?YZlhQY4n9D2?%OkeKlt|-zn%R1(Z3(~_8a(l@Y~DZe*W#HW=N1O zHp#cIe|zKGi{IY*_N%`iBHy0-`{8dt|N8;BoJ)TX4wmZSQuSo>NSYZM>@A-BXeB%9 z+eXOrTD_~&w{UG*je1T0$XOA zX^*gw>eY|0)Z`?eH94?}D#$hcd4WjQUN25!>Y4Wz9pTw%q*a^5E~MhE=@)q8&vl-) zZ6?bU-{aj*x8NeK7q6no^YjZFkw|W0`q1===~uA>3Y)GbxlHnn;sihGYo34)2rFUW zuH|6sl0qR*zoi)R6z4)di$xMbM`P+raNC{Wv7+W&H!}C|=9!IzwdJU$gVSo;uxW4v zH}^}yg>|$M4K?A>=m?$_IB?eU!ozA9d{~Cx0?P|J%jIzhZcfV_28)6#zOCP5&6r2_ zv~=Llo=QkiYp7;@D+zAximMQ6#-xyP_uyyuUOaa9;N-ZmB>d70kXRiEk{rbHZF9fd_(Oj-T zb`6sb-SvRI8UUzB2-_2*)qY7p63d$cK}mU7X}&C8T|%|t>GqZ?8?`*0#XgXSq3j&o zPQUxz?+CA1UPp4xTcr3kT+_=!VAF^Z)Qb^R^dvltw33&m!f$sT7)Pl?fByJqJi%u- z(`zU?BOwJ~s0Oz6_z(O*!JsT>=GU_Jq98L%wS(XrO|p}@u}1B!9EZRH#h+qj zbYn8(EG0ePwYg@6>S0*Dc^Kr`epaY1&Cb>#n=d~X--Btf&Wsz}1qe&9Iw^vL#8M%z zf9}~pd`f78!4Tpo*A&a_U`&k6q6Bk)rbSbzKui8NF)fEH(oS7mJ@KYTQUchllR?)q zOyus>De2ej;7eHtGxj032D3cqtl;`(5^AG{-uNKz$(B>7GusZl|xDUO(_ZaG3W^4f~ zdA2+^)cW%{s&rH|8{4M4EED{uu|gd`qord{%VLAHbkGzt+V)J{w473-avJQBF8MHJ zJH;d+0^SEKVv5K6X!FnjENXg*ENdoFh}1yqe8Up>R2$fiJ>yx^v{K;yI)#lSWtj=& zXh8xHgdkv1(;6$}HM@W_FlA@7VW-Iw5j%T2i7JPr;0vRcMJWPLVqLDpviz~1l~G37 z;YjF7C(N=v<6nF=cj<$`7Fw+a7#R_Vk(!3_CpPlhXzs0V=DxTw`|_)^Z$JOT&G&x% z?6cX=KAF4x_Kz>WICu8g8rn{TFf3MGyn150MJ&OGj@*0o%-orq@E=#&5ofPmxqs@$ zj~~4=`_c=+ea2NZo+QXMY&O1)H+TB_?8);#T)!SC_gZ_ddQc`_lOzKZix= z(A>ojXHTF1@iX}E`Z~AqJY=Y1=LHZ^2b|0t&DfNK`@mraVwr-wm0B)WSZw3K z&z(883Dy=kSdB^8CW%Y5$&1VM$;*h7mBTA(SKOgF5R|X7kk6x`{B_f=?T`-2L)Bmw z*nzM`E5W!F_WWK~+3NhRgOVtSk(Km9Ic6U7!&aG*?GF&37$&lUCbKAtmO{GuEWDr@ zsJDdgdswQn4=;aQARR6 z*QjHHLydb&(zMXC#vT_eGT2T5{B7ot)KjMAWUy|M&#WmyL}1&MFWNGLH?p={=|yDn zas~Y?`$Rv3)xt1ks*``206y34k8hv6ec|@ox6i}x_sQ))!mp3u@C^LMCz@@8x&0b^ z?N4t1>GpZTui>NH!SjB%{|p20CV`LoG-cxbvOIAi-r;IG7T0VJvPBO&34svf{bE~Sj0)u!6NH2yNizS7jkq)93mGz0@dr>C%y(F2DZZz2ND$)k81Ih+c}+fSQAB zO(+9ja>7?NvGA~F7WR@>Y=B4cE{oArhU%uZ$%cJ2(SW{aZ<1qrB- z6M#Lt6s-2q*&^^3(?K1o!Fw^r>jft!I4#?tiCO6mvrw@?1;pZ$l1)L{E>&F#acc+D z4G2$LJ2r1vYa^_GhYe5^|Ek`rh2sLa`^04TJJ+y^kzqEx4B2$!Tb%Fd> z;00UMM$MKrWW!PrSrU>P1}FSFkR>0ToPGbL*%Q~-3`3$I!XPZIMACSxK}I2zcW}++ zd}UxD+jOlYBV5ic#R*)TjAoOhPPH6KnannxatIrNB^o~mr5IAAhZ&6QV6ZFcJWG&A zRv~0}M+}BFqOnn%*tEbuAB%Wol-dNxHbvUI2RHWrEx z?+F_d#BhyT{F97pGiF>B1U5u#8U|Jm;MEN`v&HESaKGG&-~a<@CJ|z9<5QjZS-4jJ zKlrL^gI${p2j=?{gKx6c_%Nm7gety$P)QNwnw+|S>g-#^OK_8-2=a}x-0)TGHAflb zucwbsA6@sew0IuA>?Qd2w~AkTW~=Afc<}?ctQ&mlPEH?~J`5LnbNV=5D6)dzn0_5* z@Nt4o_NQT%pABrt?WPW6pA}=?kRmV|Q$Z=!h`JJGv;meh!L`|AT6zjaZ<`_Ru8Q$! zP8%ZwG^J(T;0vdgO0!Ng5BsZ!9$WSZjmv(O#udan?V6 zqAO=^`tOF^`s8%)gx@f?R_yTXK19>7YhBTjmAyN`q?#49gOs!{C;h&|xq` zHKfRV2+>3E8CNDI&4UZV#Wzrg9`3$65rjvnt!6yt2^T~m781meDoAq7GY2dANX2(P z5=A}bhW*T6d1W8TvyzzOAe}6Sc(&k5$aTv49=SHjcUBF8$e6s6bj@zq(@c_vml4dC zH|%=TL|I@FIFZXisDtrtNbpQ5O__Ws`L_h(ZuSsk1ESzxz zqTnxz6B~wx{&wRb8GO~~b*qAV&(|lF>k4f69^D+F$*jlw42zz^f^WlzVP$T3HHAnO zD_33Ap~Nspn{4WBVFoxI+gx2^!Sj@zt_54dplfBBr~26@OR>7q7(Bn68Pt#vswqM@ zNKD;d&YR40Q}gsmznQ~i0(CN`t|h_KGGYH?7#i%*Y^u}Wl+pc7$I#ji3%|p!X}6XH zCla=EysDk1JHp^I+u9A3Lk?`-ZfZN)FnhLQ+E~S~F`rHU3gUO<8mz~Xspy*GsSe=+ zaq}Aq|HClF+7cF{S={LOXp|}H@ffU(0Us|h7(9tYbuzG(SgsB}%;Ul#uS(Gf*}^nC z(?HB_!DwYP+7|JOJ$vE<-{FC(3elPZLl9WpX3ZF5xn!&wnJ2i)ufys|Dt#=bExGS={3~DQ4zEwD%BZ8-&%V!=kFYw z`2-fEJ4a_O&wMiT+05s6j>b2GMK+aW>}IaQ@8|z<;?6-XWe2BSoB0?{Ix=%%+^x|(A4CNTujLKJSNs1Oa0FFLB06I*<0 zSxyXN03G5Mf)Ni^k>Sro!h)#6c`^7Gel!c`szr9AG?mdv($(w|OL{q@D@tWVdC_&CnygK{U`#+w1<%es>8{tfn zIKWU8WJM4o5wG{GdfaE94dNq{sdUFm^ND{&kK?w<>2TIc>N9(sr!1K%5Nfw20jh%dVzryY* zsR}{xLQ-e;2GYuL+`YIuPMsn9OxG%_6z~Qbcx*46OsuI3No1!Zt^W**9paqxaN!1p zXHqW7nMR6trjD38ewLlph!~5it0{r@aPohA*>+kyJ#2U!=LJ^HVw>8l2OxMhaO#3( zE}Em2gp==dldK9+rP{+Z{89^;xh!>7Pdf0CBow+}aidn2xR_fayL*X7?2)O-Tn@Ke zDHIX|8)CDDNY36x{7`KkdE7;$0OE4ifaxsD4a?9KKU`>dnMc-7AA~0ka#=c#G(mg~ zV(%v*rhXoNHeFN9U*JlvaFEc=;>7gf=|iNocoTjf<-I=>m_i92we4`9MF_6p&8@2u zkEh|YIYLFkp=h*vA$`Ks(I`{L)zd${dh+L!?*_K$>M?bwD8(kJ)j@~v&9Q&#LR!bm zX|9PjXwlp#E#TJMuOZzn3JOgHkG^+mSu-37g**}`fg4p+6jzHhPIst^(byC3#E*9} zkkuZFVU5kvX>wYW?>zVM?72_wo%%4a zp;oKIDU57LivP}+#>$*xq$Is53!)qvUwl|o=v;jAGL_CQnxZ=yTRpObM-!8n@a=a~ zczf_pjG78(TYc>DGRR{vxV)0KZQcpW2wb*_%{1`Go6OtUMw4cOv|BnyuLsm*!1rqQ zzprSw0}H4^tTqJ`pwx5cD2I)mpSjiyu#YNu)UAJQB6x|?uP?bj z=vtal$Y<6N{ToKDpim7J#FL4#NGR{~w|>0*_5F8_&whM-EoTWLzp7)Y$A6Y|QkDeQ z%FM4Faa=eqNia*Q^GCk>MA}_@jA_Y?;Tmasm}PYEiL<6vWQg3I@$;r8jML= z2irRaRu$9xNX%Ecw7}NcLJga`cGhIN$Eyp>F$VUD3QEoL_hh45%-#6u54WD5n|OWh zkFValdg+JB19KNXm|N3LOO{~M^3C8^ZM7$Dlhr1Uh2$uKy?`N@9!7JmS|`I;b&g04 z=a(v$Eww!F>DV+l?iBK7tG$5?YtORO&`_L+A(Fw|{`b$n0gv1G<5p?IBe3S9Syovd zU4cQcNj9)0yXc6PO2Wu5QYHx3%^QW6HDpu>*T4r-ezO+;QfJ6py0*7e4S7o!<0bo+ zbpYY%aZ{vq7)IVHq_hS!TB8xT^94x`3$h1=E_)cmd_p(EaRol7W_etio~YN&C1{DM zW)VA)o?BK$pk?dM8hdOli!!b~IJn(#?Fdq9r8>e$p`z4i$LIaFZ)MADV6@40mOcd3Y`5q>Ghxv&bF@Hutg-(*nF^kq z=rGe-az(zL5)ssxZ?zg&M;)8%z%H5=dst!~AzXKj+FUaydYF}?#=~fA3~r7Nh?v|Q z>Cl)C&U4#;-6)J|ELmrQbnK~~ARRQtjJ7>fH!a6uNI9A&{rvUHbcsnqL=mJg%$bmP zvPA07mdJ9(QbU5O`j%E*iel}%$y47=e)!$w#CMY~eK$Gz-Q?NtCJ%l$`4@bA?HE6t zKjGSe?}Le(~MpMf{1U!Qs{KCjb21jrrklQ=>vu5LH;^R>JxWhKHM|M(yKCQe@SWA|}7{)+8wc zH%T0&h)J2P z={mJZtEfs!a3^z6%l*>|ol692)bR2>{t4rLxL7S^=IzxI+-oRftj6lhoX3CgVMZg1 z7lVslrNIwgh>su?LRM=+TE9C}7lXJQJV(9)?|OC^CUX-ykfkq9S_hIDHxqpQ0S{3U z|9|%0wYiBaT^s%@YkOC{A$xFJ-8y@}`+dqM4#9xOHn3;1sa;Fbvb3UBFU{(18#zoB z*dc*H&Y4USl1a!U10*4V014+*Z4LE2pE7biA7W-c?IGqcFtz`}d*3Y^wbinxTQ=AN z&jed)wPf|Wd#!aJuKT(W2@SE(k}^ecq$|;q^_ik4xj#V-2R6rY^(VxUoFw7F;V$g1 z_kc@ccykV2I_0F7$}u{=prp!bPCESDwo(q+tj-!J7Pbv)qMKAafc?vBa;^l#WhQG^&%qa96Z+<=jS)@^Gu65 zwbSQryo5vxNL_(M3+Le1OAful*TKt5dP>M~nw7#uIUUz(tC;iHrGCK;k>*#!KtYY)>mZF24ME~X2gJQr~Xp5kHG zF-LtGgpF8Cj_4)J8sU400Mj6EdmaMC7BfXRg}9IBsi~Jgg(yC_^g-7sN(V8w7;ezG zJ&qDiF;Ba^?_xR3j-M`EzVnCucaFb#=g>C`?C+brMV?IpX3rMkl+9OukH35L^2~ul zcfLLF)1^x@uYG#^i^I1MT?}jw6}(_AiX53qMblzS++4H>cntM9Gv8fM8M=%DtdGPK z9;D;KE{IVx-=Mhjk7h!aTJTko>zc}0){?mu%9XD4y=7p!JH8a1f>~zm*4rP3c|lmY z64^whY$7Z2%vfHG(|l5O2ZC`|BI~JKj3z(>JwoUO zDqVHacC9g3Lt6x`5l28($UkDB-Zb;|1$r=HJtr>UOr)ote;(P7W^pyM`T1Y}det+< z%~cqjR`*~nvY!cG6lx|z*b5v$rv!)5rjW`nB#A2RAXMkC=jg2_MdqT<< zE6|h=WpyME8G@r$`v7NBI#XsNGvEmFq%=WOtrUd`Bbjkv*|Ua8)V7j@Z}&SARpN2a zWHAE?BidO`Gh!j|Ti=i;cQaJ1j?l{IZMsaZ3zD;8A>$DF^WZKTO51C=A#3nki?~*= z+S=vY81V*KgU5eg?<+Dc1;o-T2cm$nPb!f_{R$O?JNf41?`}?BzB%#T&57dnnECYNPZv(z`s&in{a?*od+XL`r)J(d24~)V=aakpeh*LY z>^sm39>RP>-rRrf=ERY}Ms$Yf)Xj;naR?_a-kkXK=H%qf$^A8p2Ihc@BL`H%OT!hc zw03U#D-@PNKJ{x@hx+roH(qMFhPvtRrq4{DBGaEcLm&gTmifL1l-{LP6o zxMoej;@6;1-R6Se%#FhJSMsInue7-eKM6$*B~hJ4(_93-YQ#0SB0gtVqRPkg?bDStZ(7X}|e3*~R|8tEt*f=z7dyKb6d`j91zdAQI}`T?oEW!G%g}k2Z$zMi+tb zCJ@4_c@lSPlB>%lxq9Pe)HNe8PA@{3jCxSxp6z)Rnd=}++0vk*ZygHEkiX00`z zXg(WH5~wYi^q^AL3cE`v?0)d^-_D#{!NP6=@!&BJX300w_-8-2S+$zbN=xiUiABWr zL6&WTW2TQAS!!g>Jn{sK>cuyYmPnvfqZ(fd)VrU7TkNpfj;(q-^l~#_6&b5pqjs#b z^83o`mEX2UqQM1ppApaHjORqj<>QB3b;IftpB70fF^_-fNQ$x4c{(jemVQjKo&k3{EMORF#CLgyV`Dafx6@BQ>a}^Vr`#qNz$U zT<@RyX8v-r$Do>yXMF&!IAm!!u@ymlXbIKfB%J#;i`heNSD*G zaWoZuN04njRJY;IBAKVUG8vD)UzY__H8Y3xZM1A|9`-Don=PuZjztp(=db(TQe&i_ zXKh9rD2TOnZ>+Bw3`jA`jWoxNi*z%Yr3FzdASB{SLV-0^a}!bpZstiK^QD&!X-zS8 zl{s#8r9hF+8{Kfy79O0uWxif&6WFcY;gEn$IB}Lk4jF>Zn>XK|$eQGu*Pzh7>bUFBL z_L|?SGIROT?Jo|`ym(;dtBXHfxb)Y3A3XoOAuKlBMfPq~|Lyj*S7(l1y>;b_+izbc zdx!sRJl76?((JtR>X}>bPTan9cIMOf$-0^EU%9jI!0pd|clX*yPyZ{NV~us*`ta46 z&yL?d_loO`yVpK=`d`Vfe@&i$ezP$GR@#H{Ay1nmjntEAEJf|*1sL~#BeM@oKFjdT&;v0|4u%Tn{VpN( z44Ko8G;_QOHHr<==GIo!1aPI`w3us)i}~K;A`rTHs276(V{e^0H}l#8z^>J0o0)?r zZ@+y4mZF-09vtf2O0y*hE`GZD0itz3J{JjWmFg$ByzStCJN!=x`v=@LFvrcwlQ$=? z+?+gibMkCUqU|PMyE*ZPn-drB{{EwxxAq6Nf~JNPHHw;DF+~fx!FF10>u?_XG)PiO zhU<3@-z5Vl`PgSzBM>z+qGEuH%%c$04ohYj4h<T{qHiFkq$J8Y>y{mt*`+#2A`k?iHQ%k+9<^#0~@JFr{znBmVd@3v_vxt@;qhxNw!jVcw8Qs4 zU-^D2coWhusWmF3;U*nV)Xb3ZZ-$5xi!2F8OeJH>+Rpz-+QNnrwO~@Rl}X91;M{RT zaLru>bkzhpbK7_t_7+znFSz#wwm=r~dM@R)oh5*<3M(>L*kD%~bGQ_;JUA8FGVLpf z8TR0If>b8@XCniOfEva(b| zp6HBdaWVpLmK<(aY#t$<9=QZfbF_RZz(zw_3z`X-WgtSIIm6;&2 z9waTWLfK*#K)y4*WRjsYX9#o{s0z{rvw@?#NBS~Z>ZWL|70ALq>tjj8wR*`AgG>b& zPy}QUQ>63%s}&{$Y8TZ}v^_Y~3C>HFrKK>~7!^8KuY&4c^Y448cHZ3j9(N zF7D94BrHjUBWfPUWsNt~ZK&IauHHIyn5=`bQ)e!oyLr5J(I>5zAe@IV z51)Ou`xo1XcJEfmItKzWyle09uHCzzUP&m$lJ~Y}s#I2^!6W)2l@=X$ThG;*SbgHp zf?>qtKY4BO!#3i%RxqWAT(2H9pj-$^-!@ET)+QUNkwx8d%OkwAeS{e%AEl&dGCL#w z&wCc<#{W6WI0niE#DJw4B^i;1M+TLRRA}1Fa4=-k=cdp6`Q-HHWcvJ#7oF?4Nd630 zIDLY^Wx^}Rr!P&PMjkJ?VmRY#l*YO7VqohI2wRR*BLf~=QAkV4u%f$nHyo}-#nmT1 ztqeDdr)kM|f(*|ZX{}_@&qt}w$`?w(=T{FcTJTYuV3bALmGF0Uc={dnNJxpR_1?Yj zstYWNl}{?yDwo=)4py#BO)d~_4SKuw2C+9`rS!-uA5HCpZ+*UUX=c0@bh%#O6AJbVN$t3rY5Q(n`()DJ{a=ovx!MUjmY%=cp^E`v5({Yp=Z`YD?YC!Juh zF{FkuG91Nvf}=!Xv&zhiyCbz3aKxG|d|M(KQnX0DC49G|q;*COf~4nF zF>OP%rzE3@J|fPA&o=Jib4xgPAUB+DKa8iTPKGuUiIK!((^;n_wP=WJVtT>03PN&g zJh<~HVbF4%w#O4Lza@CDQ4$GllU}CoWmIFq9j#~=i_9YFSuv9ql4Eg|Ugjj=A1s)p zc4-cduD@$&m<(^;+DEo;Abnkf! zC)ah36@ZHW(7Cdr)5*##zwDJs3aUbPI^`nZ^VSV~H z60vgaa~f?Z$)qSsE4w=|uJZ;W@U67T&WdEx6@oO=jE(nER0(+lLcjOXARA>8Zykp> z3$N(eXNQU`4bE1l0|?|Pvm+jXSP9#=6je*?o%d|_Zmj0jG8*$w#Y4MyKaBy5PGNb3 zzlEG3_-8gh(@k}~wQ{OMjFDcNA%k}C2Hq}ANwR*HWrm9hyVeonuOS5O9aG|--svFQ z?SHiO2$;v)M+oVX1rr>j7I0zb8)8SwzJ9EH%3#4!UR!CFr%mc=42`;CfB&-%DV2Rw zfwXOha6w?SUZ08S<{~R;F~=8Zr*7LY=*h)Az<91eMXx>=Q$Cp~pOidJx|zXgrhyaM zmhdPt>~s)zwQ`=Os*0M7!?NL0baGY0eb&OkEDlfaQUbdH37?Fg-g=YbfdeY>mMXwv zE(Oh`aovUWc6etr@Lbt0d6Aajzr3UGJ7<_$1up8Z{)UE$DDha+j1OxOpDW_ig^}e& zzCw^8v}HnxYE9F>z$Cl#{Tn}B{{75>Lo=r?wB9^(zV{yj5B@Hpk2vH>&BD83Gi|M#ET z+eu&V=3&y^*S~!i8Q!#ci1csX)<@dg|39)#2#Ek?SQ4Lob_3U0XYI&1-D~CSLdu|A z-z#}GXMyKFPFFMn^KlTIn3wZd+tsm(KQ(jnvM--*+xDIEs0Yr*k=B zq;qZ}GZzn)Z)Z_UK;*oZgBhkt&$# z0XC{~VfXHTC7z|X@;1WFrY0*_5R)}E;Xtj9z@x*J^9$jy=6~pe1$z_Ii=7*3Cd;h$ z>axB3tM?;GdR5+qE1x>No>bnL+BY>hHSxdxiB#UiL4Z-Z00VKga(e1D0uT4Y2wlUG z!KdHDgY%Vh@R~DhQ%9VazX^PURo;iUUqnRQQg5+{67{sKtP!6#c~{&!{OP@mFWh?z zzWmj_?oRyv*4J+b zPhu}m0~r_#k-?ru<;Opg-IAl3mE*y|aStmAj})};a`3X&f*;cE^$%*nY_|H#WN>Gv zQS~?CYI^k@c|_t$I8yJi2FakQYFLSC*tw^uA#_At53SLg4%es$G5ql*Hfhm|fe z1i=K&)3Ihc7o-$cp1USVIOfu2i1`DEfq|l`644UPwu?Ofyvfo|+hh4-;`EMiCV&0w zRd1R)&UZ$Q_I_b=lWgP69!m*Z21iV6Pg``9DVtoedfe5f<=Y^d0_JPhAPzRsVV!wc z%ofWAIp}?$tWmQ>?O5k#ljcokENf#=tY(mye}gOu!q>7Jcn`#yhA8Gr*2OcD8B(9X6eG&_=0sK(mIeC|HbGNe7v>7( z8dXH?MPv1jfp7nLdm+Om3LB%p!*%lPv#57Lh7A0Jp8rPD+llwgu>XW)wN0;Kc^Vp}EqKf9B(SQ7Rr(O=1YjSE>xs?-2!r@OS zTBLrK1cBjfH^`15!i`a(k5)MdxdA(pNx;KrCUOVX`lqUu)m_~k!+Wh#QLHncCL??g z%k19WNy5o$iS@59z6p=poyQNqU+R(MWw-DG=@nL51gCFun9kB*7`Hw`dt+Vng=Nvf z&T2@}nrGv8IR6aWBpswnT0GBmjAXabbZ$iAHLmt>=^rTi4oJ!h+%AyKg`(5F6g=Uu zQ6OEV?y`~=az#%e0L?%$zn|W^isX!{r&n=JX>gU@P<8P)EfHi1l430IjZ-HySi5J! z{%@J65{s@evn{Htb@n%RP92&#aPaQoH#cwIyn?ZYnots9&xX5TS2$wPn9)`@$2Ulp zwi&9PF4P#_+t*fSiMEYQ7O1g0v|jscGfy*|Y?erh+X#z1G~71d9&JN(4}9tNE+a>E zw89+2AuO25grbHbu*12VG+aFzKEdKD!uGtCDc_yqJpz25SsfQ)J-%+4JHNY$qD2zw}NkqD%eIt2VYU+t~D;8{4Sa$+#n0TwkfXaO>qG4AJPje zR5!KtCGLyOP3D*q^$4ghhjLC#O|%{u`r)hQ#A^NJF@Fu_XrzRa^k^ISHqy^rkf0$1 zVv;dbZD3C0nj680Aflhmhi=B_pqukw87gwa;KoxAB!F=~>g*9fJRc3)n2n^{CUryv zcoPKZjX<4hDC)}n_@1C)6*h)?+xremAK0M>^9Ox0TmIA-XJDSWDl+TbV(XSI5zAU+ zhRiW;!0!5VtC?(rZRD8&H!FPm;AOpyaHv3WrpIC~_o%5zBaF4|`bfj~WbMj0964+c z=ENl%VUkt5<63DM-9&ARweRE%E`n7rdW)q3UXTv|(N?x~LgqN+Qv8kFUSf=r0*{ zco9<6WEm$xI(wwOdgSp*lM1p9oW@M>Vno9{X;5htG-p!LB1@4*8D1bB@A6V#cG)H@ zg!+oHUp96RM|gUcwye*VqKNlr^=tDcTDl@hh|QBqB%;J!{lcnNYo>P|`?F0%RU*w# zHDB}znhK$6B8-d(iW-XTt$SWip-_MNA|wiMH43cmaCb!%H5zHHnb|FE2#PldX_lOw zD{$ih5^s!B!&EgF0cN|lfE3!oT4;C|M3iW#euftHh4N?J?D_ zLPXwfG+hV!J=1kS;F23?u09-o=l}gygLGMcZ+*J#`u_6S%=JrTr{JS-4hp8%l4kL^ zl0Z>8>_c++nvj#QNEg>$|py%=DHcjU@ z+S2am(_t@`!r*C==20G;;qc$)DG2RYIB@t~JKH9Q7>!wms z=pwa(d#Z*Y&qTC-CO+hD#zM15muE&!DC$2Wd_@5kT#_~wr% ze*6?3z1<3-_I!i<_&R*=4}ScJ{P;FJc<;w|fBg8z6HT~-xoY;cdAd9e@tTojeY@L& zh@rM_WW^s9S)GfvI-sdS&?~z|7z(#FtXVu|yF8?sgFcgh;U(GLw|p(*l*!;du?) z!;W~Bqf%oseS`a4K!xb;Q=-`8M8Lej<^_`G%8d$qhW;$nZ%OXs|IPk31S*?(cV}-PvaZ8pz z&9aU}Qc)ujM^s+ZVteO5eN=&O(;L{L@^L%Cv!^;Y3$|Hcdd6--b>iQDHNQh3S~I5e z*ymirl3IPZW#@gGvvLli`EymNNiZQq;8kX?vqEtyk z-x24%-^0Hv&e3aAui_`XS^21S0gsipP@;3{1-y1r`2YrE-_$`zgfqC}<`6YX)D^xp zK`8^)jVs|$eH|!BE8Z5#u?Np+y_FIJ>24wN2s9ZIwjMmEu}!2>rNNp7;)z<&<~;Uk zgQOAxr66m2+B;V+a`IAGI$=?pbfhHYm>^SEGif@tSJd?DQ%N#GNHDs z$z;fP$1<|J6BE1O(VIkA9v*3baO5$nxIJZ(p>$3NoviD!(~e5hN@4UgM83v6^pI>J z1W>)HfA;LNKfM3LnLoYo!*@TN{o(sR?f>EHAI^0C@Wu~U;n2k&F8y#G4qSnsSARG| zwr%X0Z4dDOp3RHB<$ZW?@lQwoG(pzEi<9v3dFROF4_EN$(~G?Itsl<(a2ftxgQw^J zG=W$&{F?;M`yTGHAKrAn$;ChIgLA(A;T(LMi}1F(y2R5hwqbw)NHQEHaAjwHxCmoQ z)Py6o7I5D#aw*Js(xsBECS45TNuLOK4e`D)51CRRo%G*E>F%k_@i;sVKs$Udyh^^A$=*qEWY6uQi<4Zyz3V4?TeZ~v~Z1Tip1@83j z8kjL+8A45>BTCw2MJh2l|0ps^k7SP@glBO7k#&8h$@V-=B<1=TEunW~@g~wr&Q{+; z4HrF7yV^Lfp)Q)(z|=yxKC85ruy2+M`ok;tKRkE;9k`meuiSs<+`ZH1?!EQFil*u# zF(ul(vO~)*n>|8iO1el>SzJ;441$827l?Wys6&3bynp8V$vek?fBTawfi1OMydfH* zhgDD2KS^gpBjuf@ZJ`1M;FcI&R&vj zWIN`ru6p6Y3t&Tbk;xP_U7hZ(&K1pPr@Mwy_Tj`*=m>^bS{NC!OGq7lQrQYds`oPr ze4Gr?49^G0QV;MJ)`WbUI3La8`)wQ$swq>S;^& z9h5sk=}TA4;jtvjvk$9?F+=nrH1h zV2-WJ*i>&%3ke&W8X-pE=>;HEgWi_hK?K0Bu72FjhQ)MzM3kflCLLr_^s)jf zLXRw@F}}K{MK?2SYuw!eJ9nhBPv^=&$1GRu7?XL-Bn3yvI3MFmF=&k$I0L6 z9eAe7f*s@aR*@{DtS8(U5moil7Ru`F7$OKwss?oH;JzY7kr^wWea@gHFH)RY!HalZ zMn;aa=aP}7?H)-b>cK1d-txCFgBq)TcGO{_B^=I~IhqMP(n-c`a0vGnA(}> z4uv|^SWi!<7TU0(b3>o1bw)zbo|-{w*i4_u%y1_-1o7s%HDKi$VA2%T-sF*%SiI`? zbhwrh>1T{QBa9|mZMJgULAMHNxr^Is4auPTL^6aCqn?Z}g{ga}WTbOQJYYJ>`_?nY zkkgwATY63}zz8%qVG=3~XE@Xo>I^61J)J$hu?-MubSFD|<65k6~F3S=e+&%2y=WYMVSi3fq7A zQ+p(M=It3>kPXxzYBK3!u4|7rZkmcN(lrGJODeHYefG`ti>VYTSw1!{^6QCNz9ebJ z;4Md7_e(JjFOs~iTW|?RDV-xlTEHfwJQ;;>j1}Obf;-UtcClK*m}{b{!78FYQ5E4% zO;h05Qnnu{MwrvQ4qH%~O2#ld>eL}p@Xe9XGYgN>6qz!R`xNCjbh1OHAZVcZMp4i5 zPyjtFYi^}7ZJTf%*uNh<>WL^?G@_{Cu%fvMfTA`3I_f*>(V|MoLr}4nPHi@wTFjL& zl(ljzluivJKsn~=v<}qm%MFfz3S?a4DV>pZ8r&h+lv9+kV<_uue{E1tSeCYE1Lz6xVEM@vP@>4 z{)<~`X@03awwPVRA@Et6#>iqPS&F9f^C|Fps6NUK*q@R2Zk+>{9p1Hfc-L}54Ifgl zyr{hC_H8?0I}|dw_xWH%-038jG*;aAJPfN!!lS$#h>Q}pKT8x+PNouWNsA1^2H6r3 zK?7>FT`a3}Bf~mO*o3Rem|&Kf`+%jol4006o1aJiz~zxWuw;^g11pb^@^6XrbUnd! z5Wko60z7HQC+kTJ8N46>cMiweXP-PK=(H_dnn>y1182x)(P;I+^Coic;dle4!!@vs zSGNAn%cpL?@p*OEpSgIhnJnY-z-GdzULQ#=33O5Ah=o{5S_Vw394``h17E07wQcjD zw^|~deC5hV4V#z_OG3VhGD%ob)sPZSC<#}iu&evcw>>l^9;wg#2B}+Z6|#W2t%4wo zq$M{k4k4UX{@984U>POrz{51k-w<73TlhE^gc9(;@PXu6N-}V6Q3~)&3%rq&va0J{ zwaX7|!zhX^C^3{va@Bs@)yA&mqDG81XgBuFK=b2oethr8#Gp)55{rDDq z_hUc)?#Is^3Yt%U{1|?}_2cV5z8lyeTX=zw!p*C@cBPu5sQLC2BgqCNXhpwsYJ-rp zQ1wLy`*!Y?4o^1EP!v3&Ceme>kC)HGa+hIV%4~b~#g45sofAD&a*?Kc-A~|C5G@aL zuzX2}D?sIkQErxhFBbp6z5=z)Zei`6d$4==dVEJn?F>iPV@)eeXbRgP9bI;qS4CZ@jJjwY zHL5Cc4}Nhi&l7C<*v8U%QW-{36_W;_Awl)cYl3LFa4}a zH(Negl_(n!Fo8I0wVccbVM}yViULAm=g|Pw>2tw-l%5=gZzkO(M6K2-sQKi~i6Hha z?$HrkOU~4ZAm9G>o7;!qyY<$~E0&UpAen3^UJo5kQ@=VsI3%!y!MCy~*sJHkx&|iu zmCXndkt&2JZUAzXcy zdLyt|DT%cV=;9gB7f#I_n7sA=Co4F^5Y4AHtzwNJHV1@ETWd?&$i*mJtwSLTj%?uZ z@3RX+FS=DtYcs_6$VZfzCl#|6Q!OE;djI6TSAMgCF;xWFWMh%~GHj5Pb(d*U-NNj% zDRdj@pIe~qw^tV zE0JU?O+#>O3A5E)3GW3ggGb>aRPu@Zi*#xsCE`g!F*8U~5#cB{COJDdNM@!59PH>b zbhvDDuZfD!d$^NP)rEMnF$SXou1Ij?`#5l{F3H%ilpbdnJ*tBe|&`-hA=IxtzN&% zuV3TqiNR7OU6B81GYy&{BK60k7QcSX`s2~Uf3y**Gm?RSh4Lxmk4KI2De&h_7$Tm( z{xw-Qpb!Z6dvcYgW?BuXN79}60@Oy=4XK;=k3U9?W5*xPk z;Jt_a%owNc(rk@iNAt2!*ej&ZNi$yzd`&9&?u~D_2!sQ|B zzWwWLRjk3gD&N2<-_0^yUY$BzIZrBA;OW)Mg?6`o+LNFqceHxlQ?JdDtxrrHUY2>= zRG=6RD+y10u?AWQxRY|3rVOsawiw3mE7X}1LH1oW)^y)^6>r@)wD+$szP@|+Ae=8* z8vZE3%fx~Zuo!(MMyh%U8D1T*BosaNzwaq_clVbTC1Uy;rxk%=zE@=pR6#GL7vmlbwb&d+0 z7!ffY19}vd?(FCo8yiz-Hmi&FQL1Oyh(z&tWkhz+ESDWRd;S;ZpOn=R-br=L zBW`d;D0n<63fywRFq`mtBT+^3#Pf#V>QbIsN^4nT3q(sPYOwact%MVEw0xmZKzCV9 zCS$?Nd)%OsY%NJw|5p=IBN3!5jwjb#%B6CA=zI?;@Dfe26?7;g32-P^4vT^fnww1| z6+q}iuW+)-TLmnhc8e91`k#UghQjg)OQicURtTsPMM4RW8sS2T@V0I$(@Ylztxwnt zvjSFGGL)#HPHZ}ewa>P&FC@&{H{{9U3>B+GwGt_uE`#k6t}-q@?d&UZQ)ICA1>0Fa z{3_CngzsUJLit#hr$qxydRupWcE>$va|xcLqDI5m9jjKe`D0e*_`w zkyt&bZw*e-Je=b1AAY%_Sc+&|NqUe;z61C{r#A@#b~w{A5h3R3A2K`Cgr>ycK1?WD zWUu=PDkiz93+A1ON07q65=5t_Mc@x1n+*Pqt?2?%Gd-d}blCtK-8i}D-2T8ue};`x z*w@^w6Vb+L^XQ^!^Jo&BCEq2e#pKa<@(dI3ZPU3{4 z-q=u!*eQyb_Jl`2XU(Y%2nT?hvt)v`OpA`_%w85X#Ncji6xo5p(Wd#ds$~vS6d2Z^ zxz3Hext`1wO%SYW2t0_)@7eXvVhqPez~?hd2)7~V&oU!jO6vmuS{hbWgDJ}LMW2r* zhcsLQk2c4O!Oggn$x=}Y4*n5}L8e{itj*1^Pgn<-GYY%rTIc05O-^a2cox0g6^c5e z*Z~pI>Z6m#KRXW3WK+TZa%IP(*uW6+^qxHaBfq^UinIwx{4x%_rX-_F5xpLU?`u9s zbCRUENfS*hd*wIxe;-^@Z27|RD7H2vL-qL; z-_-aSDN+U_1#os?JuIci4fcR3e?8Idj3KTc{o_k`%c^Aa=FRKL^-EIeB-@4ts>j;N z#vMb;X0}J;6mF5@EInRA==}AGQef+Cg)K1emYUF#u5Q<)&vw@+HbAJ!l}lW2w*ikXS%gU`BvM}C zr^{KW+Vb@%J*?S1JbPiTB+BfO&du|gVkqX7vSb6xuvvIT-zW}G;5%a5(56txmHlj@ zV;XLSgc55Wr@w}p3XGZx-p&$5(0ACQxe-X?-G=B8CKq-j!Q2x@8p0}s-3k;7FZpVbzn3Zm?o-(G*9cT-|19vhOt2vZig2DvBPjG1Va|=1Lg_= zA?myH45LIg@pL};h7*e`k%*#&LyD@o`q%>#N=K1UI_{A@F-Lt`VB>Q_LI8u#L6%8e zx5A-lIF<+{2px4*#=Ch&(rm}BQEnJ}ouAc8c$YC+{?y1&nc-&McEnRUt0&bCU`bqL z@ei%~i8tTC&w!U&6E#OmD$WPnX_8R~<(wjFc_2gc>BnBzTjftU=@ZKU(l~k)k@; zz#WdIIRj}frvK^_pGFERQBMx!@dq&U>C6UySBf-D6gzm#wtPT_C?<_PVhestrb4d% z9;{g$Ldmr9UUY(Uw6kw4PkRCzVeVZ^03k#Z5O_a%eZ`xYAvuG!Rc~U)U#@#U))@*Z zVNW3Y_>ca8$?S~KS&^sVC!9fpcYiLU5i}(cuP;ch#WDf+lOj)HpNDP2CGbMZW=544 zeg%S=7{axaiKYvY{saZBb~DGN42wi()3C~3oJi%0Pa3kuY*C4PP1g(5{<=*jw ztq|$dL2eGdhe(@yUtYNX_6f4?{=56{A3GhKS?w2jJrP>kd`Tz?51YZ-MiwxIpmSvK z#R3rWGwreAx3OXynHo}}9z;zL!pR;Sh%CheoG=p+BiQYSsRd(1$PC$GaHloXDz(_O z!&d)Y+(_q~t0KEZfd=2Iy2j}^!~o*|DNvb(aCjb}i0aZ!QT+<1#1NSu@rXG-0ZzTn zY$?Y1O1txFM2UM=!5~yXeFn)0JB6gXe9uC(1gm*wY^BoT1|^$d8Ee7^?@~6}rlq#{r0oF@4R()@N5@@aCvl&M zyqLrP9;?qr2ZBw$>qH9}_f!FsWNOlR{t`q$3r9h#%RcoK#>jdK!4Jkd2g$n1F$n0c zRnEY_bCvHLTGRcNOK@QqiF4XTgjojnzkBVJFik35OpUQxAur~$kOw9VE1}q0Z}Af4 z7-WfNFgZ&ostgua%mH`06_!FQ*bC7Ef$K=yx@Ak$_bb{u_U+=_?0Qbv7Rfwi8{|1= zlTn^2pA;*ZD+=uOZ8p+EO1o(Bl-`8$?hZ>51k=rf#k#1CoX0*b6>A`T;=9_qO~?Yu zG#G@*Z241T+-j|qU6(#0;70o;%oztN8_=Z~80Z^ZQnfpV%>1!LeYVawfT1gGA^D~? zP7RBo|C&bC+q7iVLXjE5bdzMLl*qUlCmAH`Ul$7{o_<8dzowV90Ua)gSF2190;SnQ zorByeusB{!aMX!Nf&J%2OMwx~G34NndghGp^n$ZjF`XlF`2n#B;f10UMaln_6hsEu zR3T2s5Z%H@tK3?3bIP!U!3Eh!;q_+Fu9h`l5lD6UguY$Eu-Zt_$*-F}Tq=r=2W5BaC7d#yq+i(Fq!2h_NyB-9)Z~v#QKOqt*nhL&#9a9!%Vig6~3!xS~P4qekLNBI+`T3yS62 z(+-3G5Uuwgp7ZNJfI()CUcLMN(c9m>zkn5<8Sq%)8Cs;P!wR?m@Y>8b$8MjzbnC*` zfz6;HtZFnFK`&*iKh89ebE+oVdBs*sL9Q}uE+JH#qQG<;Vj0}s+ZkDAx}RWv7GM3G%a!q z^+PPuBSdkIAOaaivFUNMF?*%K_p{pQx%M=50U+`sNMyJrk`4U7h4lb;Q8`Y*c^J!6 zG9Yv_@1dICR6-%*qJ>(p3rPeH^m}|l_%Lc@R)tfDV}$G`R9ZQ3p9BhP=?rdg3^y83 zWwmV?Xq#`Wwk`H^bj-|)_5o24(#&(0dZ3y}fa{Lp8%jd0na@@1^)dNaq{P69W1@fkU54r)cVt{mV|hwd?mw8#H^65 zU9B%JeEa}<}Yw}59pYlmDY=TL6_;44Vdq20?I4mDMbD$wN zt0XKZI|j%Gp3Naa3E4`ez{GSnHS^3$QC%m)j28Z7q4~JUJ0EipLKKVKSS3sc4X&=i zObmtUXX3Uwrk;Pg$iW@Ns0whPHz zve8XoUElx$yMv(zqR_7VacCfBWX@-{1HkuvM3ji!tsr&&2Ba9M*DG!A$w@4BpEn z#>yE!iYyP*TA`E`STw0<;U_Qc3m$~YkYQsbw=s0hpFm9iMke%9x(Rs7GLC#iNoe&W zzqU6uzc<~V+<$*!C0X+0*ccFr)~6fRl0{x-d|rg_U#KRE*7XW$q_{MKLz|Z`RTM{_ ze9Z&%d;C|zoh!-|5l3EoRc$)4;54lhY++xvF@H44wvE&QO9tP7HVJBEbzx)+++A%C z%#Z>juR(D1(jaNNi6kn zWN<_AQK8!@t~W6q1|-e!tnK1AZi0L|ju`uRygme3SbTI)6j*(M;HiPZ&0D)j&!#>y zykmHHJLwzRJT%-lG}QOhFf%QI^{%W$npq2;w4N^MP!%17%_~pY)!ynx(xii1>+1uv zgEecgWLSy7!V^){aM-ne#TqYQvBg)ggc6I_rvlP`p^kYnP75LhHhsF9w5kWjL-xJ=)8@ ziw@IY-FR{Oi@zNC>%Jq?XQz)(fAg0k#;6Vu0Uya4B{Bhz2OvE%UlkHg!K!?!y_oQLq$Z@dI=KY-Wr(v1VN zZ-q~U@BcX*!e8*VugTLZ>M8|ZRBr5t?|#O)i0Knmn>j-O7Y4%+_%57pgW7!2u5v7<{^7rI&iiKDM&9HYa*Z6kWwsg!&@AA%e6?oD-?uJr<Ft~B6ipGHG6>b%x7tP~$Ah(M$vPN8ZRX;+;2Z05DLKRuv>Qv*yH0_~Dc-1Ds9dX@MVf}G7v=~mkeGs0Ua!2n zo=i<3TLqcgU%7@qkmlhVJh-oN9)2FFoUfdPSK#=CsTV7!3A_j&cohlo;FNEsj!eDk zd|Kr+JeecGz!xXr=(Wm~sYy70mLKCKhYRD<)MSgL1}Y!J)qUp-%~>3t%IWo_@}a{5 zcp6y%VN5HhDp%mzE;|GE4P0k%Rlu$kFNxATp_TrhW$u($#fy^8K5W zU)-F$a&!Mzca9yo_4fBOM=srd`|{2G7vRXucVFGy|Jz#!FXPX*FV4LF%B|O4zjX{* z1j(J3KA$=GBAj${;_S`IV>c(^=fRs3Z~k;)-`&X*cTP;)I`w;a+s*ygoSz@v+&8i6 z?}eL_ui{Wl9<1FE4-R#@hwAnR!TBJ6ZgZJ0!Ox6g8|h!vjIYIOfv)q|-+B?%8mt$- zy`N{Ft?75qX&#ymWE~N-RZ{tmev$2PP|{s?);2Aoh5q@0K4Q1z7B_RGOHWY)N&J@4 zUwSlieT!sf&ft0Sb7}=o#CGX4KhE7#INW#!hZi+sSV?M2(V6e##yHarn$C?eZnhwu zS~oK-&d4Y#YNGm8U_)%BGB3uLi2iVB^6QgEftVnvl$^vlt93l&b8;k*?^|siBm(BCCi3pTa}Jf1@4xV2E10NG2>P`H_x^B| z{QaFn_m98ux3j@%1-FmutT61BlSnk&E)t6xBBI2i^6_mGTbx=^%F7gw!__v z_hL9UbNaPg?@g@a+)FBIxM>B8mPJMk2)$G^S(;|JU05+}L;~5TRS!_;@gM$e%tlUy zsub@^4u1sJy!s@Aoexl7c?atq@c*|f-(#U;fsJKHmuF+yk(V$e55sY5qQdcpQ`WCln3#c1=+u?rcy{)R0(?!@V?_A3DX=R z92lGB>*Rh!*=X1apEAkz6b$Gng*o1Uf#e?;VH2a790#Y|1*2<^G!AMaiv~4XLWz4| zTno}SwsljPW;(bW9v=hH;=Tix za72lQo8Dap*{^IYS3%5T78%Qwi1Symfq9+NH`g=N`_CI=d*K(^7~8%3X=V2K@UFeX zyWsh%x7=J_ux7o5Glfe|xT%`UkJEfRa^aW~^-pEnHZ1ZbQNJ1-n46N=cU>u* zP$J3t)T!_6!OYR$-QBluA#AXkfVDTARN_cYlh`}|X+1W$?&LdPow)PWC77-=``@2= z>C)}@Umz;E_2seKSKhgE?T!Dra%AS<>DzCg!NG{fl(3RWJUDUY&?`S(*!R%ArXg;_N4Yl5u#2kNy=%hp#qNtp2o~w4 zku6vb<|aDOtygC3k>wKwMqr=6td~y)XLG^}fI&^Wpk;w!ya^=YjYaASYl4*H`&nvv z>+p}{#-C09lAW6s2ad7lJqQ+YfLi%b5O!6 z>72rKa+4Kl6TRlk!ZMRUkZ&~d>~fAJVVCLCkSh zD4*0zEWfw@0C;)%2d9&q7#}YmqnS}k)={!aSUuFTn7(`=*`w((G5+`l(*?QyF|+c) zC({-sgRMSdEa@gHsfnS7eX5}+9Q3hAI{Wco$hK0LMPud)vSq=45hd)^)(N64y9b5J zsZLNl1Q#^ZPMNSoj1c&)gqsd2b~-I&2H_^vlB|=)=C15b* z7{ZQmeII4(NfAP-V3?$5s9{s^)zxSXl(eGmTUA5Eah|^ko!1)_r!3Q7cod4Fj^PE22Q`4W{I7FsD zfiusL8!y6}zM4LX?Yjq^Z}}ZOflqdSnDeC$+<4jfGjI{7@SERwdG7+pTdm`B`Ye8H zxB?i3FJbUshCw+Q_#RteFkJh*x)88&ab#mOp~MnTp0C?$=ct}xE4M==9X@7HhrWMN z!!btlgpw3rK0%I0v&wGNaRg=XyvY_ilBcAfyV^+Fqm6I+=;9+ERzjhAwoW>Jut~`fNW$wRA@e{oK@HQn^4X-(wH0(-wMRw&xZ#)lotH>eS)N zMS@SKCaT@B*r)q?bn(>vy_| zQ7eAl)AQAn0-L?^@&7%Ef9e0eSGfYiJ~auWco;veb{j-%>toda`WRP|u_sR%bQ`nb zy_6X%x;G`2q^HyF@gMsE!O~n8rbW}a6@3udj8(omR+##r_6S4-jKH;xTa4_S6;7=FaR>4?!1VFG z_yaz7+?M4l>3o)%czSE4G~n;sdD2e{1>!sm9x0ebF1f+;F2%%XZQe$G&s$9cLo}_S ziA8YKgA^mAtH60mw5KHQ6dAH@Ll5B=D>$-K*u#{vjf~C++mqI9laInj5|iQUmLMrc zgaMoFoRs8_q4k8C&O`7dj120GGD4ldfQj3*EhT&bc!w$UjH7hL4LDV#^YB4r7u5@} zYb$BmNphA|?Cj_m8yi!`9vtpqMh7o;6kw#K(GCZ>Xs>>HdpKm}+F>9Z&Q&T)?J(Y) znH{kZM0Lq%I3ABh5)E#)w%)$Bx|C_!;@z*^lr2;mg(wc>ea?g?mT#k^85w{_XU>`@emi{O#gC zd~)IR{oft9clhAF!=DB=%-?_W?){7B{yw=c;zEJ8tI5V?g5+Z60jDPtT?%JpmuXTR z_DacN&jZ(Mv;|Y9a1C&&&WePS1`alnqv?SWusAhDEK)N{#HDZ=@hfqf#Lk6;r-Bd! z0@!7n5N_(cHTZaTxh`wgB()kgK62{Md5i24aEau>YowQ%JZp4xNsH&1juPO(iw}KN z>R9l$T-Arbz1T@?ZO}sX>)hj?Sla}y3|~?Cm?U;2sG9FytG1ogfPoxk)p2$MGw@Cz zIv-(<>plXmx4r<1xvj_BX4*pynxU)1tJbMO25dRj2s_)>-P1PTE^TwoB|WHSo8qpX zI1L%)AudKa&bZ4?K|4#64sW@KE_#iuU|wg*i)Usoe2cue@1MK#>Z^BNJv8&~A%`7T zT~T&iFkVs#Cz^&qx9sY+QBqjjRuXTP1h+v6N9%ooAO+JQ9D6g@9(HP@*QWMOy)d7r zc(v!KDsLj0G5q_!a=vnI_8ZStLp`r`R*qKQs=QJ8q@(hYQz_l&yn+q(FICPvmD6u3 z*8&@~H}}W=vcU7y8l^Rh2JR?GIxOuT`TDhW&^_($9DeQYi*J&3w@w|ob?ER)u6vQF zqCJ6XHa)@+MX{oUElH{K@AfZSV@&T*fd4~x8 zd`IP2l{5t}xr&#W-2p0B5ZiId8KToTR0K}_q|$(4v#qDE-fis3v2@NTx!gs>k7A!k zR8eE$B`uTDkP`A}DEJ4?A}saudO&I)%*PG~XZiW`d6ZE3%aI%V=RmL*r;noy&GgB= z?lW$@+~5pw$zqR8aCMp1R2Y$$5lT#^fA3txnJVBEkDi(S+yR^hEC=T2P4P%AW{&s{ znxo*`cu0XB|K)WU!G(ntRsBw43rWNycUaW@`;~(tGtLc4IJthMl;(^SWoe9B#|Ej8wrxNtBnzL>Oz1zY=ppGh_G6}%VY!W zGDCI|Ih#kBk#GQ@0%&gB;X_P!3f)z-}L^ zw}bB~)sLNujUuFBFH82YE%R)3fJ=tUS7aX6f+1ScX>hOt(#s$(O$?$LH@GxeYXnUC zX#}hqAfiM)gmAtKt*~sZc>jqPc^E99tQsKNfXTz~ z3q;sn$Z*NCOd?{C>##-OzMdlkRA%9i%ZJty`;gS5=YM;$E>pgcA!@yd=1yLN#aERwU}_);Y?XC`i6IOWT|E{2MGt**klkw=YU)`$@5DQXNaqSv z2j2rb4^qrDXS*L({iLaD%ojy&1mC}U5izW%V3)96*|d0V)r>&a8mj*!+KN(3*)I^qK^_ah#5_4dy`^J&0@l3Ws4^$KbS z;HPGTW4U(;sb^BsPP@~X(8BGiD+sK$gl67fXl=SIV z*mf4p<@F0}m9?l8yD~hg3tY)yLpJ7YbCaCtc14xhPniqtpyTiYpPleRkeaCuvaanDVm4PB}itimou^f`?@$Bk~CZX z7KuGtZma1=86@T~ko1|&Z1`I-!|0}4Y$B$KL>lN+EZlb2Pi@!?%_?ia{7rv>Y18F} zU(fZmu8ualIgXJG&J$`&l4S;KB!%8K?38TgtNT#1Kn-`VovJ~+so_+CDU(#)Ra93I z)%WTagAaT1kb_-&O0tQ<+VUl3B1t70Z(3IAiRkaS`^lNRpPab;(jV@gyX>?#Mpv%A z(Xm-nZNN4Zh?MjrY^9ibZE9lb2&r5nm7`AU#ns9+_;=oEynB<(^`GG>Z&$um=AU)i z;VrFvSNW!@4SUIH(3^m_>?75eKfuRue9~d{ys(~B-*bNU<)c%R_)J;J(p~U-!94~J zjp?`eDDb^dc^y9DoYU)fkyK7PBXwquBGnZ=HB&7I?1 ztENO0H5OGgHCi)L!c_tHKZT<(g_~Y>!8*6U{4#v&tMmVJ<>1T*AKlq^cILph|8wO~ z>-mPaez*VbzK>=Oy>jRKqd#3ZHFNap?L!BH*NBGUUhQU7E^#-E#*}39ik}!qy)VPY!?~v9snb6O+30cTc ztL5+}!#Po)CUK+y$4s(rD4pXYBPNyb+hTdE34e27pl`5djA(6oMSV^$6=C2Z4>l+W zQ{4t`lwes#mvSqUX-;-h2@4A8Y+v)BWWZ)#(IBx_3cuA2qQ4DlC zN1n5vv(=C*Ia}Mrd`TWxjTS@Q-6i-IunEt6_3f<>Cud$hdh4C5Gp`=L_0}stUHzaH zHnY0W?dW&6&z$|Q@9v)b^3IzRx8C_Wuvu<>_^PXz?#74v^z~GPhsVS9;aZSM)LTB8 z*118sDU22*QQwJW3%E{V3ELttS5Ww?&_vc_r%qj9$96T^I13nE%ou7&i6`qPN06Ls zA2-ICZqRgYjG>qrtZe#Ni&>2q5j$s>xWTQeKZNe^Hqu`+%1~s{>jY6w!|C;aZUyUl^c+)FaH)jU8 z`8Ff}^pNJqazRaNWb?$fxXw**AZ_WIFAZPs9OROZ=(ZWWWwl?7f&omIebji^#aHV* z@oA&D5>9%Gv3?_unx`vgN#$dg=F`!@rs)=Wo~B!B zrfH&&RYlBi+|!%ydjn-t%e4 zF^~yU=U;fAr)8tITK4Xijb(wSCRl2sinB_bY(1#XeT9kHew5joq6!w9-}5 zDcQ!f3IG0k*FhomJC=G`QWzwSxPJy~ zUaT@0Q=-v^cNc79oIQp8J;mJmmi27RX2U1=?5C2E5o%4qA+k*_9qEHP6}o?ww+|DM zS!XGUW8?9@@5I5yfAso%o@P{|*ihQM9v>A;ssk?EkTmr;_x60iMh|yoQu53*-SCOf zjWjiX{cG|oa3q-yhRN(Fl+Z#7{QPv~^CX~9C&Cxi|cZ5UbEmP|OHBoei%(Q{|xeKx!9f!@wNy~ES{ zhq^IafDl+@LRCUh(!SUyw3Ixk*1jNMbiT^pgB*CnOYN%$DNr5e$7FDU5xWbSlLlL67Fl>`H0Cmx6nqs-N;gfK!B3@OAduPL`mmK9ry5d8vKz_5imK@^nRA=9& zW9^ZQW)Oc@b{G|m1mCi#UcwPCbwKkZyh8J+Ncf-zA3g7Bm`-t{sjgEvqD14h!Gwce zT_-FFeev4DyPvLL0`3}enSi@!kv6NTtAHG`wfm+clPk)d3^wz(M%mH5@auCL0Y_ zT~w_t6HH}&B|y)Yza>k&A9K$m?}yzcat3)q=8oEIA4rRGqGg-;x~xi|$uz0iac}jU?bAetP&-;2=#)k+Sp#k_zjn z1fxp8#U`9FpQ=1=?yK=uybJ4#4VgHuLo|0uu@jQ+7uPFl? z1korlwbonpSW(fI^q$k zBb24qEK;n>+$WS+wBf{#9VlRAWN0TzQ-cx2e4OBHl5C5Hi9i@{JZwCUsPVSt)mfmL znPZe}gNc0n7dvVU+h;+Va6^jguGbRwtCu=vm86Dj?{Eoeg z)8apeuxz>OEwdZwA`Q;jNdmjHa)*BayIqE)>^L(T>y1ZXj{dIW}_&b9A{ zsVs^-)zCGF_*Zv&Mj8`hdEbwt><Hu%pyfqUE(*$kueYUs+Ugkpwr}NMk0qqgU5p z?8da65kACiY&}gFSapZT$Hqu$5u@55a4i8}Y$@H=MOpr!e_eKef~rz$R|6~13kFu< z7YrtYsuGIS29&J4k3jh!%Gb)5%h%>+%2(%JZ@svl4SaO2e0lEm^2NECxfhVjo8a3D zbQD5J*?>`Y>>o+Nuh>!WGromZez|<@|9u&;shm>t&j|@K3)SgO-7+iIU{<8Tax^7R zn>I&EQw$44xOa87ZfVf}Hl!hnlnh$P45pDj217tfZ5U+V>?JcLG<>Nh5{Y%uv9bCr zAF)+PK0vThCE9pB(P)lIMkxFjj8^=+%>CA1y#!>Ys|ou z1hJ!*)KRY@QauvsP!BFIGMF-s>VH)V5aCj7&J zPQZ{XSPKMP?ICJEcMWtmah;haBR$L*lC!6r)~2&%&1=tEFc}Jf+0rn>H5TcyN(zeG zCF%>N<-kJ%^)NuwChQ@WL9$vY4ckJYflzT>0_+BsVJN6X zTm(H2zqdWWj9*zyHkBPIXwx3VYjGSrd)oJ5&j^#**^l%ov&Ghdu(;9QX=N?LWbGA_ z9W*9(gHy??iRITBg}b;=-~s1i!lZc&~wcfpA{YlFsq1J zk?5xH!@_(wvCD#q4M(@g1>=@lTOii9TxxAYOs{kiELGzMLW*ib(P2rh!(9tF{I!5* zhb^oss#@!XH{4jdXf=Rd$ue1>y5X_EbYyg-Js54V=Bvt0`(AqPZqi1!wQxvDxUg~Z zUO_-$ZT)=a)xTf-nEd_R2S5M$^VSP^-FxrrpFex{AMbsB@4Yw4&wo6I|9|g_Z(;o7 zwKwlw{Q4h94~HGyjSj$E&*Fe>ELbZJKk@wCpb)YKd^e-nw7%#PBcmc`D=59qESwRR z&Vjul$!-B7urr!@9G`FT?ADklq@*yN%=L|kXH9U%Eo*`ArbUr$V%tFw+YaJ^TI>7a zpZI=S)P#W$Kc!|?C`E!wJkfBi`BsR%BFj=yV8}U5bx>w$OQZE@WH%S(HfNnWP?L7V>`*ro-$BS9P_52Q8@CB zVk=iBIgPp1447PFV`TONqEVX#!v5%2K{XML23jm?)TJ?t2+NeL&;SPcQ@oGy<>~xj zxUM{d;Z+5iP$DjLYd?Er*ZzGZjC-OQRgt`il-T;EMjn)w!%}i$QY~K<$NfXe9ZY5@ z>_|+{_}|THzPuDdcl5*sy*9;#u;ALw)cvR_K4usuY^tFA) z+6Q>tXpRf5L?#IG!Dbm)QTf@c=^QC)C7r=vSUs4XZ>@}CWl+9lGGOV_I=F@)UI=TZ zL$2!FE$7!MoQ03i-TD1*@4S8N?q{#vdG-CK`M)77r4BxVV6oknN%c$<-QN&G^C!Z! z%VguAVn+(NsCHr$Ak$)|ne*1yp4O|@M0@N%2w_QdC{&v{f5bNaexYX?XpE}2=QluP z6|!;U*VLIaz*2V3jyw&U4AZSbN?Pe$Yi;uVFmi)T2%QXzoPRH+r@&$Is*>>C+Z#d% z_=u}?^%2-VK!xh8m=m&zt?e-T^-6fQOs!?6*4bwlzW8S0+4BqUzS%@OD-=`$@ijBN ztzIzW zUnb>~aQoNF7w4XX;e6#=6y^8J7v_G0?V=yg9i2OdoulP*Saa+!Ega!Z)q8joOHN(Q zy1G}~94CmDZ=k1oX%>nep@CS zUsWtoCFV-xJp6vr0JG9y!^o1+K(F)q`AOk~qPk*jo(m7aejB94A_KDx%jq(fls4}* zmFXOx;M82d)W8E$Q{V(-nQ7nW62!{lk>mhP2{2-W?{;0c!nGYT;N%a z!x_@Q*#=IsDf%SV=NJyl&39*B2mzJ@E_xErhTDYTl2GhoCP}gw2*t|B-0X)YqgHF# z4P8uXTyO8<%*Y5i-hfN1Gtz~Wb(&*QIq0!ful(2{tXqw^xDq^vAMWGyE}VGd?#UH^ z`#0CdQ)4{u>m450JKW=22D@pF@kGSQ)D(1PpgqrjNQthgFS!#G7@I&G1Jl?J-X=^Q z{52&{S=b!ZauR{dA#Q1xLPm0SW0P;u^a~`P4@H6w(Zn63*l~|d;~*SxHeG3H&%){B zI0rW1LV~0akwGp|?_kWw4%6Ky_ZjZ(+c_{WEg60bX?ivaZg0KhHblWYK>9}iHkvWP zbce7);Y2l|Q+eZ2Tu~+JN`$W1sMR*Ygr&9BET?-Py!Q8R-zE2c`|iCzzpz3N@84b^_s+iX^PkV$`{eKnm|tzG+O}Lk_a|qm z6*3r7)I?Z`1(b*bki5OVA-$>@jWxqAriHCe34fB^p)(Uo@GF?)>1H7jg+ZuF zz(u$Dh&VWK71g>n16k8hv;dj9C;A1__H z^P5wxXVcw&)iBT@7ztb39eTl8<%$K>OSpR3<)KW zsBM9I_#gj0Lg%!ZRAFI66b{B}XMU>!YMB9b^x2;dAOG>fZ+^V+`kin8)Ih{|eTtyM zcHQhF2(Y(bXhP36vDiF}u&^PQ0qX~0v~?RdPh2U?s&3yg#40DaOU8lZYU}ej4R>jmRegC zK;pSe3%`3~;lh!H-yQyG<`0!5QWKS1D4?j}TIJ?B$BYsai#_@R*%CRzgk#4hZLzgB z0t-#}HwpSAothHMD*U4Ln*pP%bOVAIkP)voLhKo+^1y-eE9Li-BN6X*(&u zXEXgGeZ!gZMFR7D#vb71xfihTt$bnbIo}UW`8D|BxkYqMxaPYRc+9yIHdpd<_6P8j z&S$EAuSs;3UXw7E|HfU#+Yi6pGHe+|(M+ofm6^E#t zE6V$l3Wt=y7R=w8UAZiipgsgEaI8*DORzU{O+xwunMRa|+AwVW5QTYz9jbjOC)Tlw zbc1pcFo=w#?W6G3B9jdih<0XS8V(93vsEcXtLcZR$D((TLQILefW`cTH}*<1Cokz-P&`9F-Y|=j%w0dd`kU0(Ahu{1g)G6B1kF2rO)7PE}gu z%u0jNrd|+B1q1F9g5+*J#dV}rmmMQT&a_Ehse$%cD0=@YRS^Z_VMjTlcOeCpK(tm- zd0rx5@92+#L`abu#L|g9Q83$#{L^L^QjJeTr-W4y6Gj7E*qzAws|(UiO}a#va=*pO z(P8CTBw;1t0$uiu4gd4k&kzp=Y(FIlYdtDHABw(Z`#S01;&|K=Bezj(N% z3+v4c&9s2-A^-KZ6R1=YS%K=y7O_Obg(6j!fvr{KXok&F5(zsJR z{>d#tJaHxAG8{Z(hneh9tF>ms26d1YxK4&Cx6w}gb}pp3b1zb(j@o$pjThAyNd}P4 zq+wcm@fbdcVEJt#vPBxdHi`B~nije#3QQ9^ncya3r#6W+K7u8q;m}&v)(*B9((=qz z*e@p81e3%@A6a~}{h@*&8Ksg=GU-3gP#1&5H>zR;OHe~`7s};^!Nu+2d7)9uAOlur zl4otqsi(OP&Kh3z4&@#%E92t<2k7Y;|>89Bf)fkcM~mlsQvT(Y8O}P)diS)IAHAN6THRb}b^;I)yVY%S{Z0n+?NwXTKB*V!L zYT=MH`ajz?dn#*PfN459D?mJ=LrwVSd0hdeMvWz^@!rZCk1|at(QvKx;kf{@LTz;5 zK=;x{R{m(QM()boZ%O$pQvL>6vl07_%pIHi9lShJK41Q}e8HxhCUY}$N9-C!#DTlw zSh)R6D<*f)3`}l&zcAtY)0ID)e+if0wjut`SE#Vd7v^59J}&OhYG&-^3Mn^yjWncUCf z?=0t4SF4e2)pJ}sgnRNmEFbFZJGyhrIb#fv@@j8{Kp3UB(;JRs$0@Z(I zQGBI=t;qBYGYE$*4E`{a=-3giabvMYa(o=o*t2Zuj0uKFh7_R;4Lh)muO!1C0prE~ zl7aTLM#=u67Bi#=u-m2rzt*04Y%k1?6#g4;hPWZrm}AL0>}bz|4hNK2yw;5N+}ppm z0InI&+HL(N*I=XVLdHRvAwwl_6n21{GtLl{=>Z97Y00fW%A9);WZU@cg^Wqmr~=QO z!7A;y@o2)o4(>+!6{oe;NJS;c4t&#K!*DmljoLMsjg3x zyQmaXlLtG>$FWNV4xhDc8aT1T#)QVsnIm(rca%S|O_Zzf?-I_nPcWLw4^^cduX;9d z&wfo9qpG!Q&2v<>f_}o8v~0Fk4mW~9Rz`+#lK+jguPeja25n;8up?RsGpy%fD~DY> zotwR4PMJ(6ds7El-b4;s(#P!XS#t`ub{Mb$Q(~iU`oywfa*C57uIJZS_7Xun+>ncU zdEA%-Qn1nBf#Kw8AjAPrOuu^gMYmK#qHC7x^w#Y z-OoO1EqCzt=_~DOP=?RPKvp8XM8I7pW*9RI9S5iHqsdj)<_A!rh#-sKcgQCcQtJu`j;|`AsG_=(f1ab_ z@Ov08`$f357RO+EfFkq2;83_dsQRA#(Mc|~Qjs|-RD6TrM6j09O(p4TZE#&<+(ZlE zUpfeWO@qyjOj)FB_XwfZc$R6i-)#Q2fSSyBI;RQ4$a9&_F|w^oQVYGQ2I~HX!C=C6 zjU2V|PT$FY?RhjA2q=Mgt+4#y3jTERh1>7FG(Yp{{LCBkGw=U+_1%R(%&bt3xyuor zSS-ifMT<7lM)S)t=Z}0ifAqEaBcIP7{q6jbi}OdW&mTQ^=j(TVy!0EgO-;n&3m3m` zR}=9B{B59CQ&p9Kl87l$N2_PJ>TQKRdRri|pc0MLDmKr#q9RN_C&q-(Y&6SFy?&kq z^Vwuo3022&G95o%>P>6A3JNh;G9hjG%kbRvohc2_(Ku8CtAIaZPPMU+s2(Y3%1 z42U$z=-j#Ih9dfM}} zYdv8n?9+@c(n2?XIV~HUZ$YU+nA`;DDkT_jpi?@STD%gqL=m1f?&8nMc}*}IQ?pm# zS7uKcoWMUMN443LnwXkB2?z2R)R{de`RvJjo@(&Em^Oq)O<|CObYk|j38#q-LM~b_ zsAHz5rx@v)J*D|CV(m68snO+u9!I^ucbP>%o7Uoqt+=+@2CYJhR!lpmO9eWRu8i5Y zxf`Vhsm8LjUS6*%@#Rwv$*m*dB`}T0vda_ki*? z$pO3=v~s#y($9lBULO(&QW-nV{wZN! zjpKsdd++6+zr5mGu!8yn>+QeEyWtHtOK)Q53E@U8*MSibODEHrNCia(zFSO#~FTZ z)6Ns3C`!>@=vl%GS#+$j6z0xGL1-c8sjo6N|AC7fHzs*HZ*vkkQ|duSPTh0pA{ety#nXTM{d_tWHyhmxJx@Q7WG<3d&j1d{03 zW$BFUvXCyqf3*WGOi~T(^MrE|Ijr)^8*j3U1eB1gz{{84#H8H^qo*0^Dv^w})zsbV{9>&YG~QK^OLH&_#ku#08G+YikoGO~KuuGdWS&!obZG z6D&jQS+eVSm?*+2lQ}`KSo<*(PO(ds7QdvB3iyKn;mgX`T21bM`y)$nJx?Wt{fqjK z*Y_fFrp|f<2+iSfg%foZZ8)*2OruJ`RY|txarsP6Y77~s4YQ;*F)PkxLH)Pm=50Dv%V-GuZ%{zywc&uA_!CE!jZHYz!L%g2AoCT#!=Gl5%Bc~M<@E}3#1K3?=)yzyWeHYT>CxV+ zV89@qD;vjFcM8f5(+~gN3SRUm)$&#G;Jf6AefPeFbMw-zV>ds!_3X`yw~pOWq z>%`5ox1M*blW?^W+J5a#R4Nkg)^0_{ghx@$elx+OK~3c{L5qnr$y$tIpu6;^N;!(v#Za;4VTDZti$fVsUTbFnOgpMqz)iCMiD#% z@kR&tjS7P~!k({A@zzi{G)1l#*rwp5rs2?IDssFoFyn}r5SYIl@)A!n^E;TywJ^GMLe zEVN+|E4zfjiaD|av;pFHJ#D4a-~sBq#UNK!8isG7t_K!&EE!alaH6&a#B+-;XvN!a ze7x}dbFEF)B0EvT{04>cC6 zI$<%-PFOHnhaGGyqTz4J?f`$93(YcMm)FcjLR9M%(Fn$sK%`b_Zb`FIX%?*%la|(q zVY5E+3Ms8$#IAzSVTUqEo#bjT~;L8?)%r*kYNX{o(Sr16+<{fJYR23tgP+LwL{ z;bU0B8;Ld?rn1*mA3tQ55(osH;F z(`+#4jM>Cm#2BKpt1d^(Pt0()4aWu}u^S69BGp$xW4A%@S}5r8z~SYpv3bvtd906K zQ!ncha*7&2M3{q?VUo$kL-CYl5>1G_K{9;YE-W&)AYzt5#5jO6Lmr_KVs>G5<2{is z+8PYIlWwYm)k<|@!Yxd2#2FxP=bT}=BFNEj@YC!cO7?fJKPAn^h_O2-iZDsZ%830d z#t~3KodqnJ6G-4x;U^VUiAJhrzY&jBzcCx0U9D{+@}nbR8VJ?1tN`ksp_aUl8^I)K zO%1$Cf+d@Irmd$b|A5ryD&V!t8b_ zf{OiLNKpgon({1>fRb<(Wcb<4Y@*lQg4@-hY0!!#Vc||@U}ZoC6z9JaIw3gr zmFFrH+D+zfYeWx9gl^EN-LWw$kng4o7lgi@bnV|q!v13xOlBx(8K!5P6``TJaty61 zN9=lSSdN_)U}$y-BTkYn7Bsh*c>1Yjvi5y;jjewjH(AJWNtu~*#*e~|0Fvj5wMaw> zxk_{WM36?XQ5V-yXWHZw+?+i{s7)aR1LT+PxTs4D2a#?3xL84RmBsGRST!>*B-?aS0b_6)_l+65!U!-Hw9|SE$Ec63Sz>@ zH&Dfyr3hsLcArTltCS5ky4A#!0aXb%UVNtWx(cansNeOtF9#59{&wG88EymngQ^C> z`H3hA*yv-9N8dMoJs*RVh&DrA*X&iud(V|bx_70HatXXaicHof%~yOw8j?^$br8h!;v zdwuRO9GIE&A1oSSGRs1$10gyT2v*lk<&8(-!REHri?!C27H1O-u|GK_2bo@=O@pb6 zW5s90RR{m@tHOSf$s#rLYo@)CN}lP>KC`>C&CwtdOVJT%xH-Qd~HF_v7me$3FY% z&u{sj{eCd|OwbK^N1CqzUS62~m=ccFhAqZ@O#jUa4@nEwPt z4~L(DZ#x4N_2Uz(PI|D8@52l~7TkdYPZ}2DEhPISGtvSMcJ%6q3vEmrTsP?9H?d<%NsyYVI}ei0tokTs%fH;zP9zV%R*6@ z`bRH5&|6^A6gq1J9bz3PKcNJ@^Fq~2o>~Pk{7M)v%W$J;nKqr|uBjcky%&Vs{8{Z) zLF9TC#{_BynN_gnOcy3kvy`Y%2>GnopW5M6R1U7)pvQ!29P1$iu1c(pi;s4O$+iLO z5G$mFC9{vW*j`h;d%XE3MMT3PCF&|3@thL3s}V?H_EU+J5O91e9?^HjTNeRv-SIXoPE} zR1@in;)r7!3O8I$wVz78QuaGiWNENyY;xx%dydN7Q?c^8g z1KOvSM6@nhc498W(Yb#pSu-TxqU_cz)N5pqb9)K}s@3|HH{PTffek8H8$;O01YS-j zLtxi(xI~L>Ps{GTNxKuM(m>jZGS@7kG9tgcl{7;BZ4)X*pT=FgA@l>B3>a=A@8$|2fhXXu};T1Fsna z_M8f?5_3D&(9ts ziF@z7+RWB z$0C8C8j1$PiN}?L#a07*t{t)e7@{yHSRmqP4v1F$w5UffErDg00T=f6ma_F($^AmlFgXsF<>RdN`mKdNfkk_|N!Y|E9#YV^vz`@Ar5HV|6z-IKHYp zqHA%-x3uJ9Ll)ywAngY?V1VGS3(oXL3ZvIcT_%)(3-D@7s%wxchzHD>rU^8tD$zu3 z3}E9z&V93|B%5lnSaap3*CEL4$P#$AK7(N;dZln;NrI93Bvz4CSgb&X2_+V&b-p~edjE9z z?K|&({C}^!{nN?wKfU+ifdjWszqD}i+E35DuyF0$+pk?18(X;km!A&5zi{G%pWgcD z_KPnqy!5+GZ^N&etnQ2i)d=`Y{++B{_!Eg(AP{u09jFl}pk~A?q-Fvuq9egtP3U{= zZLcgmSU(-3*zqm48IL0&1F2=>0RRKrs(;qffeFc|i=?z87y(18QxSIX}ve>ncqH2gOeogwAx*kMSvmA_s7`Qu5N zGEK@K%pIMZfnPdP{+2BL?+LQ}LSa1dk9Ioe-11YOAj6iCBmKgXB>cnSKmYY|orkAw z_{p}Kho@~A<2pDW-b=vNt(K)GPqlogz8{`YOac2p7=qR8&6n&hd24Y&s-$16!(&EpEC&3k93^s2A%5v#fz|n9T4ZH*;VkIW43HFdjf6 zF(nqQm585~u-mU*`|0q^!r8;ECy@CgpUofre17Kr`I(dRGhfXgd2W8@`25W8=8yd0 z_8Twy7R$o#-dOlOTG9 z8IO&PJ)T5*c0wx@^u%WRQ^=FL#tegS^D%?qt-_DNO)?POByu4V1+QF4j!uHBrm^YK z{cxcc(^rjSZ;eV(3q6KkJ!iOm%vKaSd$nNzeo^GDdPtE28s~<6z>kLw55L8bR50B$3_- zY5jQdz1wHsU-;e3kC$Fpc<*`t_F84k?I~w`F#CP6h4hnECC|MJGw8)t7^1T_wSuHLwQ zH4*`6!`-y!LxNo^niis_$N!okKCOH_S!`?beM zCZWXDN3Y_1kV#JUOcNQZ6WFVvhSvh?-fQM`0UjbV7{-~&0E|edxk5`gzpJ3Q4~s*H z1J_|=`8LoWz4-k#`Bsr&9nK$adK6B#WP(hH2Zk&CfE<$^0Z(>Tkf*7Zu^PavL&P=5 zqEH>3fR#5M=RT}Ng0%@s&vOq8qvw9<6eC>cQ=FQVaE+)zVldNWg%nu)c0pAIz&(wP zC#~KtJw&s}O`oBT5mlBD!ZeU>o=)c)Yp{`s#BEB)f(pePaN#lUf{iMHkmsm>Tfdbt zgg(Jxg4&ixF2-?KId0T1cx!vo!9l%vttWMkUg?O5^>x020F8<-g7`42^gM|a**3aw zkw&FX9dX;>&bH-JX&dA)=&T*YxbOTUe^NwIL$T#exK;Uhmrh&>CTd-`EmPkqwA=50 z`Qz1d3!h(amXWUpu;(Tb+d?K><4(uG_!B8^Ards0fmtrGEY5!^!g{#bDAbSwJLHy7 z*Gs6m5=@OK=1Q8@q{?s>cdw z+B-blyJu)%=SasWg=_lnv0i=^N1W1c@ej%-ri5Kc*M6L@ksV}w_KL)9Oz7=MZ$ECs z?r_~QNas_qt~%RyZF-5_Hy`U_RMz`6IZ|X&7Uk@fam(Jl{mHQXU_RB}xMzlviz_i# z*2}Zxu;>gansmZscJ?EEsvdUBCNB8cC2-JW=RSpO8)WIc#`MPEbUKHIdWW6qB1f{w zt0)jIKG-me1LiBT%(f3FS;_$WoWLC*!$DoLpWSYSUF9Nkh$OETPz$eUx$w`m%qhIobpAio`K5@;tilRmfS%zS!T9Z z=yjEY#`+WOZVW!g2lnW$z*rH|VuqAoK~xL-?S;7+-;co4IdRBQx!|8>7oLD%KnaCv zp8!8(hC`C;XACAsVorm3-114Ys!I4v@Vc&rYw_{Qu*p(Ma?fAlIS1{iokX~DoX zY=DJu{q$2P2zFa-6-9BSnkNJ-HR$9r2lwd$w|U!ySxQg+=d)=_igJo&43okV0hMK# zlavez&B${yN%r@Q^bwI_X`M4-QZjDo`Sd{sruyHG=$!P8ju1F|bd(%qhDp*>e>)=d zJZG3^(AhyU&cNEG>4O};`?q88Tl;&u>-p6~11`TBZsEZI2m4=lHi;zZ!Q8UV;7t1L zVbw?PXd8UmmB^%qn1ER)`vjkWlZ+fMI-Kg5UDX>5M-&GUY$H+LP*{n(FtE1p>2PS+ z@-sIB2KOaWg{Yf*cG(cI9{@a(H+(Pe2ZmIKGDX+A;h-aCRa(ywU&4t5Tf&kO-G3N6 zs1Xy%Z@CWI$uJ8P;cNYO;s!ed!DM%=Cm9QNCgZ(8DGV7tTC)=e5u7eEP-0i_b57@%Ei-r|zEo)W7&ttN>9#TPo3rLrhgEc@dsocWLva&1$Oz|8Zs6gehEGwnw-GF&zDTQHf8!({&B=(~$B zneo`q&mL)0X1Y!dZcGm|+g)K&3b~#g zeh+4CbuSLr5s_Vjy;?p2q&sKHbj~pe{v%Z|OUr~FshSWG&o)E^YzSA$)5E{shQJul zYNcY1HIZ^0r$3~q;f5iNMkUn^4ZL_*a|L+NB&Qk~m!e?TX!J>@!z1RbLEqdi)aWt{ z99tFiQ_E!S`#7ir|7ncw$wCh7L}ogODgC47qm&{_#1*nyF^t=`-;z^oc}>O$HK$V4 z+D@1Ws~?0j%8SKzy9sC{os*2G##v4S1(+A$E@nBCrb0%zu8+fKjCAVbvsZG`(zC`C z(sW`qlx~hQS)a+Or`P3D?O=%ARoWB|RWF6d@=0!F>xi^%WKV7EX@njU_MI<9o5C2s zRn(xWB%C!jYJJX8b!E~~6fM{rE;glyU;pi4qR36q5=}KY5=tUkyR7__+m2A(q@)kq z=g7YiuoIA!V|@1X`l-4NiS6y@S_YOr)(kX|IqYKkK{`P}c{up2eMb}2WI$D-u59|F zuo5KPa%R0!A|XegmB+G76`Kj-E>OIOf9lZ)D@ri)*DGJ%*yKl*fZOd^Nei`g7VJl5 zd6u?9B~92Nu)%b(Lg7eTue-}_q*$4e=5NZoXc|0F4Xa0XqqUVOk-gx8)isaXx@-67 zPSP!;HG7t}IpUllB^rR;Rf#$Q2xAWX!I;Myauuup6Rz%FUx{?xG)$JOq|Z0KC0xVz z1@>Z>5P3`}j0j8fud4QlRH`(*w(TLIL=%smj_S2cOX}!ZHO){`k+CA|ntNXvG)rOy z2{C?U(M;g>QNHAO#a4cH2=$E6f9>Ao$F3M=<5-%SUa8x;ZPeWZIBLl}+Z=(&(b+39 zPvOy`rA83e)mT&shQKbyPOyYS;f3l)B15Z0B2_GyPlReqZv0^>r)EF0GDzGYiEh{S z!HkHh3gCYwYCOPlu%=R6%U~>-!V^ff6*>hs^K$AJ=^ScF>_?KkfyT{#re)bM$0Eq8 z+#G;x9`V>vY+!{%x5^RL(F%F%cYnWdrm40^Fs`Vse9yygdkhJ!WYVTqA*jhvK#97N z0#@F;8njWdugtw}lL;K3dwuQ=#G$U#121;^_-j)l_&3o< zb@9tzt|<@{3MzqM?LOtXL>mS4zAXU61pfwva;%~X&T-m?!>&=7K z%QdQmTp?D^-IPBZ{sU}E-PD{tl@+ihCx^haH6<7$7Ww7Aos1ZK_S3Y+#>U8hbLz5l zr^#Q>)<0RFJ;ji4Aw#(_HrD<>{@au4+{rZg*ZLR7nj=ZMiLGxnztmc(#a%QlQe;0O z@)e9J@VT^c|2=VoOmm_ow5#>IwbfPHwN_11P*@C_7gH%p%o$R0`Mz$G3ZvZlBfC)0lWZIGrr~$uBStx^$5^>SL`vDsG z;qae{!O}v{bYz%y2sWVg#e9B1dx>2-ID5*CUuz#r?v)hw(GCL?0kB~gyJrI53dYwq9_qiT@u3ptUy!0w3Fs71a?0vXV2*wOOn~sWXJ3U(*m;~ z&r|-*f?lpMrjMXU=h$sRiRuEWyz#h1qHAfJsZF9Fq9RENk#DihXBVS-dlxNMT;?7r zEM{&e-J$^RGLOO1sRuOh->sV**RpX(@mSDNy&v>izOQ19cgQ87Eu3_x_!f?TzHs_a z3(ubV>4Rted#URLm9An%csN!Y{Pk>TOEy8C_#XsM9)D20LJxBNkP&KSWCM75okqSi znWLK1c2mPEv_phf$wCIP6ge1Ki>otI(velfAgIiAVND>iDmp)lSR_ID7mvx91_v1d&9jhga&E4Z_r`o!!wGPMfgH zRKQSVW+Po7Pl;tVhj0EJt zWQ%L2&`F9KEJJ=c{>uGB?Uh3h2}JM!E5IoKnG8`Tg@1AO&ZTTtJ^PEhvN&P?HGUb$RcZnJ> zl{zH|1(=!Q^lwM%wTfVTHDC71d_X1;MJ;_lZBMqrG3~M0 zQzkXH6Ii-`{erGx9C8{~(>uM*HqMPW(I>l65bqz8qCk+2N@2>d)IF^o;W z;A^+m3>Ho>7=CY|h`li>$xWDdbWD6N-R>NP+mkkf=Y6Zl0kInL?2H5hN+MhvH|80y z#U#gH8iCn{EfF5tm5Aqqoq^?kcqEVv_4US*eVyIC$*w@4I~hpE1A)M_my>Eq z>OEaJ1*%cfv9ZTVhNmcg4{kNj>_?Lmc&unX$0ZRO%aO{(lRL9HF^&H;cUSjN7Pu;H z<#Slw)?IlJm z_K|~8CE#Mj_7i{~6-8pCg~VN@1>EH20>SC{;qV{SC>ey0GiVFxq>oF(Bz;1dDXM8? zfYEFN+rglbiiIKp2S{ZsQuUGT*B&JlQ{tgoAM4?-r|!;aLW;Dcg_QrvU$2VlAXc-7 zU0ptDuL(nNC)#PdhT*|MF@a6QU~Sk%ljvpGRUjp>q+t`Kbq?3(``ygzslSw!;5|(> zz@`oMCS4Yf@Lj=T3oGGLW)J^T8Ds`ru8?Y8=Rq(TQk9r1YT$XD*wX=iHQbd+s|bkT zrTqHW1hWK8*EBl&3AiPN6s(kV<@4kcLSphsJ+F&N9ab4!KqGr)ItNZW+L$dB>|1FV z4OC4?wp^%<7zd$novg@;!Gv0&<~KiRUQHpZ80VC9nkF+aUeQ2j+{jdVilyWfGp0(q zk>`%=dv#F-y)UY$5)RZlf_{v%UK>*fo+K59Mvk88+#%@%t9-V8Dc=pT zqzA!+JVKL4_h2<-W54z&qnHwL!A~9Zs`ys&&s8;{fN8FV z<4Pzzz3g4B$G!a2AFsT9_taT*t>36q(|KY;S z`S~N~=8rx#x9Z5Nf)CQQWI_`s$r6S z+_3NU{$c-{AlR6ePRY(f5y^}sKSTyCJ>#%h8c08dqe?8)a5$+F8tfgJ{^9sbV`JTt zQKVMJNvJ5f$9j5uhn^fDyE=F7dF%nn&{O@b2Nw-DXoq2sG3*}3ik5beHUL|)?5j&_ z$+4Kcucm|&57%lm&o$|J+F}IqU2xwTE0H)E;HLDi#QIAaCo-(>_NQPn5>x__hWGp1 z-(FsL{)1tCLEW~V+5lax2ecqId56d6iv z8fHO%;+bbgcJ}n{=^Wexp4T+!V7SRA4jdSV7s*kYw+?dhl+am0m_j*N%uWXbf#M-V zOBjdWHtSqseD*>n1IPN{ufa4ULrppN7R2m{x<+q5G#ACe6SP_LoFfj46up z*UOvS5US}zT0Td22x>3j_#ix z-T&ZXo$KjQ+9BWNZe=A3+=umkF&uFe!O&I;fuHDXTF#}@|XFeSLToYru6~i2B)$)b%`Ax6sYCwA5DPJJvPv?%7&y+9v zmcbx5O%{wTQ*SG@$HiA^j|*X6N~H02<3}KF6IVB1+$I=NB9YoW*%rosgDdgpGq3&p z@=O1C^BCE-IQo0XK7dg*lW!}2Y(bc!SLqW1VbSu?h06V?xfHuA~Eo^IVs2#C4)P6K23%? z_iTD8)kZI*&i!Zn28)iGw@5rzoqc@marz@lI8eJkZ53TRrE}-ickh0B`0l&!-u>*4 z{v#OcRB;``mKV2)?>T~DTM`EjJZZvPmh6+vNDDaD0ea3vI%bk#-0pXob{%Q4PO%{> zG-2;VzJ+yvS{znius%#ca21$<5VBGvT*bZ*e-yS`6kytIoW>@GVZ#9>mS`9$yk#eU zH_upEtC{ida8QZ3fIS&W(?5oc`{)8&_|9iw@0=OE=EmxQGW|yYbGA_iy~^pQ``c z?5@s@PrjSE@x^yXzdLs0+wYE4<+c2HvV!tSx{-CZXhZEo<^4&4~1LvN-@#T%{4P;5Gzv~R=#_Pv(cI3)!!f!3ki{4jWRT2)3S5qd|CR{nJK!;LcOf4du@ zg-MD?jSdo~1`;9Rm=cN9h9b7i_hm`>s~;>Je);x^m*L+ZFTW1|HqnR=Cy?5AEiHRs zqB2>*H-&XJVp7vGP)vyCih(4nh&3&BB-GaVWLu4d+MJ$$^AQMpgp@oh$cV{kfecfw zP12Gb85;yaSOwp5Qx!FsP!drk7ED}-TiQ07iBUPuD7T7AD z=Kk9{YD@H-JdK{`Kc>V~7k`;o?X{+6~x$4g9R#MlGjlQ^}bH?f-%QJs=};-Pne@q?Khd!;O3`14Qj7 z$#_6fqqu3sl}OOPNonI;+tjQEb3}%N4m6ISS2&IkHju>w4U^v4?F0SY)BWAp!_%vq z<4o#^@V0vq%8ey^n4T|Cjqn25W7{(}VszHFt8Fm%3dca23~3ZujJAPX$Wub*A8)m; zo&lS^COtYRBj z>4LY_#E+gJ`~LH&PSCF-+W$Udk%|Ahd^Syt*{?x}|8?mjo&4uZ|8@B-qRoClR17NN z3BNV#_u{neb5vR^{^~x?u$lh>1eARHz5{UQ>6~2w=NOt>a&6IY+j7aZjlu-y$WTsz zuS53O)3_8WZ`S9MZ-r-RKes-Ez_Oj&#t3WF3T{`84 zVRHiDH#COe3I-m|wgk<1l>m`gCI|6?&@D&0rT*v19!v7^254Wa-@6}1489&ww)$zV`XT{emCqwr&IvA|@WY5ZWG5jsA}rr@r~1hq$*q1jg0oL1#3>R}`= z1!;3wQ6=HxqTA}}@tK&k zN|eAAOxtGV&Fba}V~3Wzu6o?7G<$V}7YefrozqO$?u%T&SybJ%0C+oD8gv%%(x7`O0SuOpGFE1p`khfuTVrDNMAfkiBw((?fVz7bX$9Rw5}1 z-c1Hm=G4BDSt41BO870twv&`)5)h6cSVFI}QaherJ)s3srFH7*2dT^#gn;>Opq$0= z)ukQKK`(B<_W10Cl(?%P!tx3zRK8w5KX^CxqPPlwT)$SjT|_TWIFa*{>p;$ zxPc|W<4mGCgGEtT4re4gI-HNFB4oB>vV-dxc1S7aivMnMQ)5p!uJOO_U*n&Jmf@Kv zz-SJrFrLt6cRFf-b3Q4bnVXqAQoaWNT`iv{<*&*Y;J@>W(|2|5Mc)s^UX2~1WSCFs zrF`{~h({cJAeA>BABspFlDRE>md5c}?tSpuy|>S7B7BxGHqazoF!_GaEAKvk_Qwks z7QXnV6+D#X5}7}8dj9BfyFlgG!bhLndF$GpPhLhEytghdeDL*;moDA;%_-l)dFsjC zJNFKbIQMb~U~gC7Yf@y6-5N3p$(Q20+x5K+Vc`~7SpqRYb$t&$8a!Rz(ocU)a+ zYjgcq3PpEHvw|5jF68|(%8tF7|NOxqHFL{phSh*0KWe}6D7sLc(6r~x6xLMv)$+CS zrMaW!E4UTF{&1#zdF};x^*Wh5Hut*y=1TcOE3CNk+cv=61>EXjKwrV8_nc!Btn+Dd z7nNda@?b~#IBbJB3uoal?vsZ*$|rF$;M^l~uXn%>idF~gopA6{`HXLgFaA(f;<2jj z6!UC4MIuVb)!gB^H43Et%rn-sni!>0GQ-qGAsYGhuL-Q%7%5tM^C<#7n6>*j;y6`O z_>P+2Wrvf4OR39_VQ>@}v3nebD6U^#1OwiGf_C|&ISd{L;fHs9TMwV!PxNj-t@tmb zQ0c?}cTCZ)UmW3qm1-{w)#1D3eyU}J949PerR~JBe|HuP(Uf%7nmBqeRIfc45v;Qh zhH8bfRj0j-`s~!_x8M8t?#I`Ey!7_%GcVtM_4?iGFD*R(>B6&b`xo0f)og>{f=WVd zUccxLiya-+(C zFi1Boa2&QwK zc%|izAiFp_WZHNX7>hHk@p0IZ8PdsOz%89)X*-m+lNgLuf_(*Hn50W`lN&kGNu;0| zgXEAgo-Uf>+sPzekQ$_G@niHw4qEU%!zvW0i=9rLeck=j z_|MoF2`Gup-|^Ig=q8S;CVZRhimLI1{~{Tx=b1E8PIl2i)Y0YOF&_fYOF{_**8=)J z$kS3(Zj`U59l9Z5`9WqZ-a8|M__ZR^J)>UDFfuaxCqag&rTOnj?vr#1AJ(<4Wfe>t z7liwYYq@Qtl+jtKtwug*K9>ki!3qrun;!GRuG*s->0t$tmSAnB7YDWN{(aU7khSd@ z+26KYL~Ym>IwCAE@4$Yw750*wfOQ%%RSCnNhy!lM5ie6c8#ZcCQC+<64}XfWhssF~ z!)?G9Sg$N&1etc+4Apd{nwPeI%ExK#in;m8tz$RO-8ymelbfI3I)X0R&ChN<3;*0a zfAjRM=Wm`SH@~>~?ae=tThGFuPj6nl_5Ak5Q$D_V9=`hto&p!SbsWw=QTgoVXZYzi zaCqkCDRT2{d?j)mxc62V5(2ry;4GJ8b^4#QA_bfBx1C!D+d5;@0zZ zR$+fK+BeBE3GUN04bcavCrAVZ~r!qoc-{` z3pWm*zH#`|8%I98armp5E6?9JcnHog;hg!)jU%U=Gatd3?%L^`K=voj{bSYYjUyj* z;QAE|jSL&yjm+)DXOPp9qIfYF4`6GOo#NZ)%Jnm%wld90TuFqWIfOWvic(@7sC%cZ zT;QMZe9_1kHCnEjMA$h2{*U0)3h$S%2+%2x?M`Ib8yCBG;gwOl%nlH$RT$2L*=5lc z&)u&;Y^4)pMoCC1Uubh(H^Yf?58q$n@R;g&$?A3DjAKiTq>oRa5VU3pyZ4bw{nP>2 z_%a~I#~#~Ft2yHUnQHoWza~Q}XrN{24)sft^t+zB6s1|Qe)Di#bM<^{zR!5(FZ_CdX)enZt%=55b!W{t z!JeY1EG5$&)6&eII6M27w{E`i>1v80ybjNtet+h++T(xEKptWTnhnOTM9$3DAIyCcR??1RmSwKnCPv!Ve870| zoM&yF{FYk1P2#$c@#1(c7H!RXXcpzT1K*Bx{JdivQlr)+X9BDU)2wj#qpOWt=)lfr zwgrzpBmIo#i&gxdT8?U3?=W-7>EIj^nB(MySB&BWcg2g-EnHDLAgU;i5Z$;h`E&nA z9(OVeypRsi(JL2sjA5Tb@64~`U^~Xap1b{*WB+*d+0YVQIovDAG$qA-eV+nr6ZS&} zU#UoP^^+@_7`tnE?&9Ary<-r~{@bNb+zmI}hcg3zyY#+J%nr~7n9n70!Y!2cGFeU} zaoB}?sg)%|ygg$^sb+Y?``JB1LnHrl<@n8~zMB2?>DlMroc;6Vo6mppKUbbwB0;ue z#21F&K`T^W6H)jZhtD|?>#H{o|Fuo{U2fGL&(Fzs&y{N5J?9F*%PC2Yq@`#|QWRJ2 z%SD?Rlr=MsxYlI!!OMLO^O9Ia`j|LE6C+B;wcak)X<8^~MsJmBDjRp&kW1rYGra9p zS#R^eW{u4;Oj~%c%3;pnnAuwq+z{MiW|x5gFt%yy)VAtph`Pa<0F25v`Enki8e##O zE!jg0c$ZP;mdn}f!0?Wzq_l6z3oO~)hFxVE@}6OpUWPpOSpBISwZVp+$~(hG&Sf${ zO$)`?%dFmAtbSg9{BgK)yFPAUglS;@!VN<(fdzPlK_{b3JpcWgY_<7*1HZ6-2p$WH z*TJ+3hUI%@vvnPwPz9|pYA^l*Jdx9=Y3P!4w~7w#QjEeg_Rx7O@tqRZP zn;;KNA`_9NgwHz;Q?4^}`oygxA7GyI$a&aRFwHjm>btXVemwiy$(t`p0eEi0t^UVt+__iB|F5Wow>B4v1IQ-&I-+Vpu^vN}E1$lSZ3rNR% z+U*A<>}FPx361Kic^B-VPBvmj z9x)2->Z^Np_j`?@-G;D=q+ie+buy`Ql1jxShdWA=V@Ve%A#A(l1fnrxzMeS2qqSdN z29?>ncjnk9H(&m2=G?hke|rDcp_j(Gf%|aTimaJq=V#A&9l$HF#Ghop;al% zZIHXmW^$D@%Rx(;69|4xE8d`1;6P%tb{V<&cMbLJ9vYq6Gcuqtsxz@~AF0YxG(mbn z#||)cfy1&j1p|ngB@6H26B|d_T-CBk%3y6ErWn2;%O)LYWz{w;wsIiXd(zYhq0I$a z(0CIH88RSdS&RG3c7QcWG}(i_8WKyM#?^Y&2;g|1>8G|`0Xb zKvgmXpXTGW@GGJ)n7N}Rjbv*-O9tN0l!%l}bxfl)@6!LqrWz+DLx_7-+P_kgsGq<@ zn_wKCDJK=T#jgJub8uyE3NEr)mcz@)*l$?u4s8x4D#hYng}HaoyMD?YikBRPw{TsT zVeh=RA3oKv0?jZ%h_I8zyg-{!s~*XL^=MjDIK&e*!~R7#q=A;T2nU%CHis}e;1*u= z(FM27#i1Duh`j;PB&A5iiyzm68X52r(%bv*q-d`k4f;LVgmxGzXJi+hpv;;K?@m>@ z#`HUhMJEu#7)^G(`8Xu?utZbK7Tur;o+>mu53^qd%X=FYD${dJgeN!eQ}^4kfb7|e9PFnB*RH^gy$J0Is+G$TVNBe60vLo0ih`$ zNBX)nLjvqXtpamx5+kP>dV!A{owUMPDT$_}NHlQywO6A5I>%~6tQLR7noehK0sFvX z_$?gD4CRcbDLVj)CaAEaiXMLZ%TkS2smn4~S|~>)Aj%Qsl9m#_bu$!q+MnmRNH0&H zXj~$VuN!Blk2Eedt|GtY^f5Ah6b>9)cxZ`Ht7sRY(9mFBnxpIYl%oTtKumx|mh zhZe;SGo`rXkZv!B^zw3uO0th4be#}(?GwVv5f}6%5*T_Ck4p(3GHA(+cF%5}Z#obz ziEjfBNjT6gQlTcH`Mk}z<}j!^xi(U!+$4Egt~hTuIae{AU~h#o)m5TV-0W04-|P%& zJD^UL;qd};u{ocI4(X&|ah(~8;?=VFgyRw@vDwU8DNyJ8^U6xF5w5{GpmwgQ2D|{4Rl_i0ESq{#Z zI=Sdc7!k&IZ{%vkAQdsm_QQ`34+T?e`TaZtBAsAlvhdwu!*otydOfq#&`Y*3$p|-0 zrkA)gClh>fKS$#`(Kcb!GvWvTjTi|0TP0(4S%leglhQ`?y9T|Y|L#!zy!zcG@`%B! zaE)2ym6rIEDJhkbl(Zzft16@M*23?+4zdy?k-#?LYn8li%Z5|*G2;Nw!<7ht(P6X> z8w(W`JN$0GA~tOjA-ajUH%f8ulE7okA`gv^BjZJtby)`Zq2_HP38O|}AghuA1_{Qc zWc&eahuv%+tMU$XXQc|7lE{52$2$(bV!gT3G!;Hb(9|jM)n~~MuLw9RKOB5XPLSQK zJPAS}{dB)>l+5}mB-`CM0q&Q+0eN+G6)`a3YZ7)ERg!k?VZF)r&P#F`=Ou|C+oZ4h zd3_5D>3!zGX*Nts@`h%Yb6|9?QamP!ESgatL0{lJe?P+xxE3}v-@DgoVCMAn9?x>v zqNZH0@71<+w`@#UVd`J#8gDW}lwhW;Cj5-)n{@`8^bm(Z)K3?}gWt0y!7CM3a&KWS ztV}#0Rt7`~Zi?bXP76CkT;UDJSH9Z#23>9^qwzdxeAzfVci<96^{kx2$NxZVv4v8kQtB%2Q!1w z{wS)T=G)&Bqq4Zg0FjbNuX&7oVDW^O^7_^9pqvOCp$0 zNp%di?b>y1kcu)BtH+F&L^M^1215w5&-*%ZG-G+FEwGsN!7m$Yi=M?;`NA$u(^juwTkxAwaldJ5$$6^)U;^nZjm_2M~K zU{qP6UEZ=`QEaO2<75xa5|THBVEMUT#%p>F^Hjt?l{a9{N9rF4hFB%oSgm&9&VRN( zRaoBE3|EjXJV9#`VQpz2tSxZb)}kdw4BH|VVT?0}=cLP?{}FDDgB}y+wA2VM1X`eZ z4)wbXyy>~XcGw`}Iv6V3 zoU-J-d(kYLt5L?b&?}i=y<6JT&ve_GWP%KGvt-X}{5%tc|72z>SWy#ZW!aS-Z@vjAHAPCvUT%os%|GBMcmRTAEvl|uU_gAR3bKGB|By6kdey zMzM_wQF8^~S^(;R@aYs{vl2eNx&OmCBQ`G-sNhWNfMIr}VmdJUu(NcI_vzZ%&$6P< z%0^DqRuLMPLe|Pe+`IbR|Fi!aUbFQMFi|Tv@yI3VOCOJiXbADJRx5FgPgs-Z8>OUMB}REP}7a`1H>qkow}`(U9?I?I|$5L7ST?9KMhY1G{VFqu7GiC z{^J45a#KzR=JcH~1wLX-Fs%!Lx><17ij-(nFW9t$;9y_TM%WRrxx>2dxNg9(R0^SC zvX34pj0lG6=L=M)m_OdCF?Bb?_b1Q*2YvPMxTSunA&_Wu} zX#EqTTqIdTql=ngyIF06lG>&a4wmx4!2*lQA(+mQUSyRX$?O@}vu_`nrySY1U55eU z_c#k0nGoEYUDHq_b<3vVWPaM)-Uoh`u3Q0hT%>bhMzfk`%hm3Y|MqnY-g2^2 z)X!^R8to?|;4|#E$ppz-wmQkFUV_b;j9|UB&}&X&E2QNikd1K%D|4nTYE{%hS~-OT zk?F{T=bC0MRxVR(m7_9IMT#lj$q$_Va?s827+@NSm|))+tBxEDU%JRlGEm%Lag$U{ zg71c@Fd5h8?6r{gUJ>>?B^6l&6Lg^j1Ir~YC4J0*f#p&^Q>B{SY(`cwLB$R7yPk2% zVMqR5581eP?+6-wR%XOGrx^v#HrJn8_{`=3Y(Z8(GlebN7}~u=+u;6Ap6C1at^G7} ztMfri+^u7oYz-^Fowz#ESCpwnku0}jm{zYL3e+s~-`nB+vWG;aSU=<90}tH;DCKmk z_Te=GCksln%UXl)FibENVauYnH9ZNw8=LIuzwEOuqt|o6RNtff_GQQ* zJdRn$Ek-760Z*yUY;eX3aO(vt-EW4+r17OKN_XTI#hMXy}J%2)=<}Wo@DL04WhS z@nj?@^<)acta9SP^8vTfie^-bHOmM;|0`$T#A2|JcC5Y_e%JN-7Zxx4@0B!c?$V1P_3n}x&< zM`)8^^)NDZk;bcy&*6VpkWO^1%gj?g?7H(!$T@qlIi%2%YP^6Pw|scBAt;ppwL-ln?F@MoBPO!wo{UoLzQGhe;pa1j*5SsR^nR^SvLt zOjz5IZVL{nOeZS1V@-%v`1J5@CR7VP>ax+?CR%U{EuNI*^n(|H+C>GkRuvo(PfHOW zhc68geY$?``{UQ&zy87XFUj>!;l;V@AAkS!^)ubKfWbhXzW(9$3*R3lI3jrSS!lUj zKMx-~_WjcYejSCW_~iSizd!o@(=8dblp~;Oq_Hu0JQcVxc+YNHFlw~dEH;blT#}mo z48@6#!Kl}?aRhMUDH6|VbrmQ^fP&N1OMIe7eW(!5wL*9e|B zm%M5|I#h8gVab&4iVFRjkSSiq8)&`H(~1+_U3C6Gsb)!%w4uAAKwFt13aod8VkMqN z2AP?of?6HK^KGHk$e1lk%!RX2VlJ<@`6l3Fl$7ERAU)(D&)4!84Kh}-tUQhUkP#mm zqZl;kQ2*0@VrVMWHW7;oi)v+>AwL{^8`M!ytz@Y6^Gu0KD@qfR5RAuI* zRzFoG8}X$^tG{xepCa{BNcXk)L)#2dt)H^U7Pg<^Fw18BOaZPfvx3g5Y_fjFT)1|; zc@3!f6tU-S>Xga<6L02+gYW%t@U8Arb!bo8=XfS}1X({e%wWG+cMN|&F zhRQK1?yK0m|JTr=G8n`i6fdvf@(Ci0c_)6oHqb-9o0|FR!jG5!G<)H*pT2ov=IIZ9 zyz-|`;_NF@D~SJ>5|NIZ9(Ap58IAk!avf&^=khwqT|!mWOiF36^s# zF)*zqwvFzf)$F%**vqMH897?iiPEOadQWz2o7a_IJTG%nic84|c+(Ln;U=LD-AE&i z=v+DNEeH%uDR|?^lMb2S8HY*m_>IF~JFJ4oZyfsc)`^#H9eU-)p`$;3^Xcp-?{rx# z>eiEgn0@Pyv#&h=U&M4uvuV<*O+D<_vNz5Y4v*PH_JOT#S!^?o(87lbSz=5Z_Jv+pu3JPwB>C!%xhfisO#NB8aqJAD4*18E&{XOczdWrd>O0*2npE< zy_Q_NR=yd-C|MvBgIr;4|7-4=g z2ft11f~Hce5FwTC4%aL)L3YrJq4V&?bXj|nGZdXmS<^0UO$S$34m&YvfD=Z(%Nxs8XxjO6G>CsF7{hyQFV?Tx0oCXhA*73zNK{MBG$&N;=UVwUrn8^o88;yzS(U^}1*F)nk{#PEaewD)py4?i4yiTu{ENuKJ>0L0$C znSptkjll5DW`JV;qYK#N=I6V>$o62Ab=Bw{p(T!%5hr`7z>tfaPQ=5@mD-vTS6s+H zCFTMT1rn?hLDo+>`XJiZRD*klF|QtU2xniBVzO7Qte|n(Z_nxluJOq{Bh|SN=v*hX z29Fjud0FG6jJE`;gP{1*x93bsEGF{xE9JX|?!QZroe@K;83$_ciDFG-dib4ez{pX( zMoo(s7WJiyxS;*cn}C9gNl{<*WvHIO;0}S`<4{Z+I?)We6o6;DzuEnEQ&6U@i45#8 z&117zvW@a_O14M?QfSenu#y#fge+XbmP7u3t^g51jq96!md_ z1Xc;y3U%D!_wGgZcTTm;fB_TDb2fs9&CTfE#-vnAlI7Tx`^~<6q?i0sUhB&lwk9F4 z>6BV>TR4<)k0<=frrBuV5vPObLa@!)A!K&pvB&oA9m|041iLMmuC=Mw%YFMEe|+Y} zXPZ~p`ht_BPPyOURkJ6b{OQxPGsn-)zIE#68=tOu(Rx}W9Q$bI?A5XUJI+P8NHPO* zWTFt0QX=ZDSzCDwg=4E*bVZxL+@~OlRv;r-90P=?M5I)*<2>I30igKl?3XiFKV2>8 z1j;BXMSZ-%_rDpRwR5&c#p+c~bgv8NyS*Ui9H@TsZ z`^`0&6$sk8y2Au3X+dcwq-4q~&A@Cpuw(tMr|}|bTxeWvoJRmplw3^pAuLC|L;C1>)jF)VR>wQQnf|-uFVu)_z3mN6Ap>drP z7z5Z`q-+#4-T`0!B%NfMX8cQ;ib4^bS~qiTw6zFF124kZcU@iwc`hfr*-IbY{KL_k zPqtxBxMyYxxc3R)Ua1F7<74rr!3wMj|ENS(Hp7>aLB=$)qJ0fIfL{bKT$V+SjlomJ z3`5{7>c~dGb-Kaa>>4djWh5;t;q8CHRI-aAAAw7H9Yl(w=#cyH@FCjgEDV=u_0vU* zF};2oy*GTPvYcj=O6?kfudVfPSuqadzt|zDR;|^{%`k=G<@G4d7hR0Cikx;OTLSM; z(S#JcBVxEL1LJdLd%4^b+P*9;*@BX6g~P5A-bO%_%WnV=eYC>1-W@jPy*129@VHW$_o@51r& zFI@4dR@GrT7?}XbDHf5^K5lPj23M0|s@j50k}dU5v@&wtw$DcnWqtX`Y^C|!u6{al z2cvrL2xmetGR(9phc61Bkc>*vm?S6TlALleu7~aehC+!+315PL-IiNq%bj@r<_k~E z9y#{orAsrX|FVh}T`bXtTrm{&kzNUVL)Gt*8t=34HO){}^VA*@J=Z{jAl}NuZzPlP) z3XjmdMypaz7%ql-SHdc&6yk~$@v$g}F}PgmU+@76Rz;q+6^kw zCb^j9r6M=OXDeb-M|UmcoeK*uQ?Vf4TT&=qW(b__BI?69CU+zcd7#v zIcyU}xczt<{&U&Ht|n2=>wrQ4)4!a8yb> zUnFzYg?Kn$vy2JaY&d%_#MSY}KIc8wOv~?5;=#xg_PGcLV`(eq#s?M&m#rnBv z*=3TYI_YOR_&Jtk5SU9cCJfy&dPg~aJ=IM^XT8I4h-y}N;;x-SU1Y3SWP!m%yz+XSUnDNLnNAXEeIaY|07;#0CRr6d-v1^rNe zy?5`JQK}jH_RSr{R;+N)p{>P(6~pA_enbDYBuW2uc{LsC(D}`sw$97kGHoyvOT1t= z*=)nosKY#vEk;v3E{8eMiYz&=gFHhr(h#TR2eBr|v~#PNXO@w0I2Bn>aV!gVI{W<1 z3&&GeO~{BqT}OHpj#yGI^8vh-~%59^bWM~a74 zwEX0R6c66OsAn4$bPieM3SkXscFrWQQj@Ha=gi`4iC|XGqkDYGsz-S_M{^T*ga5jN zaIZZ{CpZQU3`6sarDt1T&tkE(W@Jt@No;wClPbwk7dBdP1w@pf5fK?d?sw3T0z}hS zt`vEW!;E5e^EljU_k|gEVQD(7zGjhEOniY$azE7z8W?b9k?f|`@NK2ChCoL%tAW8W!Y=MHWUdK>2^r z`}PsrY!(}D14BErm8u11^=>mP_@=4muX`0`Sh3{&pAcj2jPg?B^TuZgFyDmm`?E+M zI{kdJo$XuB1YC6x`gqaT9pw5)m}&AdY>`SNl4v~xi56xl5fHN^BTH&5c_-$^09}&7 zzHi89`pq%e_X(OKLuHP2q2`(}19wv7qbJQ@p(TP2O;U=bupNpk@({N9E6M{5$*9+* z4O|R>RkxZ3lM2fwDH|&8cVIcGe95Z*bWD;HNdFL(V)3a(??6d6?%mres9t=jx=>lj z_U-%CCL(yn5Eiv4p>PMJ%7k+cvHf|51(kERf!_xnpe)6~c~8Q;#ika$>o&C?nq(!; zG_tXeST-i>1Vdt*T6lW9Y>`)~4$ei5>Mo2z=mrS*6s4tj!n>;9|5oZY!_;hD0K7m$ zzg^YDN=%VbzR+P{1;(NTTX3^%z?y2}8c*?zK{Q%5gh5PDvb;eU8RrK`6%2~53Pv?6 z3Y~>udj}BR5w67(i{nqda{Jl?d3hPdq(tYN-ox;Osi5X_O}`Q#?szPfieZRwsbzMj zQ6=fnS8hhG<))6>Whrd);R%ROhTs)a#!qV)6f+{XeJRi?-RW=9y6!Xg(SgH4m~$wZZ&`q0QaO%zP2- zoUWkF^;n%6w&Ml7%5Z29kn5HI|HPyOI9mkLfkM!F@gqHhOL0OsYVg+!RT^pg&lDraJ z^f4*rYf2MX^tL{(8PwWDaEyEf^J4dCNwl$a+O$og+Ih{e4{+K+bEdv3&of=8CSOES z6q*NAMuJzxN4jx8_bc&-Pa2<3ABDSr0)AeGaU3R%KQ_)zzuZwl#1dUfZc3LoojP7Ee|fHl$4HjY_Ipak~Mv%X@Gg|SkvQkh;EX1kR2*F znQ76S#?2xq7S{>p8usn`H{$*RJhtdk{39k_-H)(+l=MQLXYT1(>D11>QR6KGWO(P! z-a(yJ1Q|67$f^*&cYVJG$3|_gyPttxd8rWAaxp9gA&OC7JBMNjxB1%3*IvJN>e_49 zUb*%Gyn4CooabwAUwif1o7djI_92et?Q3VQorcl8e(fy+FW)5BUVw4E2cvUN|Ir!b zhvDsefj)MvcqdsM5Egi`N$=9C1vOlQ;F7YnxFv76pV8t(bixV9Z`YNL(c470=SP%Rmu5 zKA-h62z$NYd21;o-1gl*yJh=avSRHRD(b=@U={Dv)uNzeL^!CcHnz#wZm(@JM$23y zyQylo3=~;TN=h^$rDRWjG<0T5L3mvH0qmBs%cf>=74upO^0q3zSkaQ!Vb3;jPO5Xh zu1iODu^d~f&BCDHuHVW;nd-GQp&>gQ-vkRxsjw!?v`xsiXZLomKI}3km;f*BfZ+^d zUL%lc9t^oA!%6av!gvZI)qe>GTa{r+#?i zg?;-51f%Ha;`o}i3L74K49mM6X6w8~c4M8Ez?xO_@y9pZ8%v>cY0Wzl9uEIShhd3| z#pa2hT*C{$6fPooa`i$D*a{UznUrEa7WsQuhxZ#;%q@-;U{yvchE_v>q{#@}49pKa z8ct%Q+p6;hMq%K@V&#hIU}wT-Bk#Pw8psyxlW5!d)i3^SMCnSJq-&~li4?cJH@zMTE@UuT|s^$s$U3i5i!lsn;38$;qt zFh4Z9lr6rdDb%ugOwU{7Az|^!&|*kOz^p*1URqM(E@L}1cbejWLw)h~FibZCRKTKC zKO?}a;IPD*gkq~fD_>kfF8zKRu%BrKZkO+ZWI|ivJHPMe{1-iuS~V zhkIhk8+~W&YI#<1!sh>LGudI7MeL{wFN$lwjK~vKK=A&?hp|b5UV_D0%h~1dE7~?j z)gs4EO?R~I$6``6btk-P^{1-!51m~`Wn@5O0?g+nY7_3AP{N0LoCfeJjG}&Kemr0d zP=PrI|M)^n3-)BSOed*uw=QrMeFd%gQV>;{PI*%hB_p7~e!lP|$$*>l^A|^de)qM^ zKVEtB@1K7{{_*nBpHIC>ZohKn@1K86{_*_Bw~xQ}!@)o7+jsk|qZ@yI_sXwYpV-;| z5c&DNkN^JV#kuSD?b~?!%`3lhKL6UMfB))WS2;qr-@5Sg`_IBKLQC}aD@Sj?cl`F@ zk6}vPiKXz+XeASmw5iPxyfzLPLa;nWD9~H4P+@^fr!^f`%Ww|nVS;D{=7xYJf(ORA zU}dTk_<@N~@wx&$?@n9RX5laFe{dsCwYkdS3M_CD7!shQXqXl7h_#jD7`;c}v!V_9 zOtx`NBV*3+LQ8!+Hc+N|5ta)>zza7~vK(t!`_AiN3LTG1%6gh=i8NK8uytyNZzZ`h zDXMrvibTAkSzl{AUu$n4J$L)n3uNQm=x-lA38M;c`>g2No04Kany+;_n?+%&H!sW_ zf9vK;&xdy(R*1ePiYUZL!W$I{<35Zbf9E*%9_QWJa=caZ<8MArOb5glT0Se(gFK4N zhDl#z_;8f6{Y%)(3A4@xIXuOQgC6UX5G`@kCUm%+Y z{Qj&-WwF*R`^MXF^M|I7zu8CrY*& zTCjuQLo7~Grd?Uebig!a)Ik!ayaE4QPzBq#cW*wkb#!X$C|1(pn}LUhGh0Tdwr~w= z>2MlVgn{hci$^kBg$)*@%}r2B{8log6_}9rS73`HKRof`{B@&tg_#%@wdkTtT7Th} z!Z8a>*Iz1bx6#Z5JuBf8pn@1lHLSw^O~*VSCqMl zEa93%M3c{Cn4Z}U&*3jegi&GQKXrI*{wJ*2lGJ65NE$CWgfeFv7f9nJr+evFPVd&Q zoTjav5XbD{*fxu~>P36wZ4IpWvX5X)iYe=?o6EW>U!Ayh`01G^znJ~>((I|@tLUuA zaVh2VDc9o$cN91H$7>hYP~0Gj@a1&!&ODy`vWW&bV@3xhy(`chBZ_d@bkrM1FNSFn zWK{WnkOHCKU9|-lRNucuKfpD-h3(mX1qlMN%TJ7)zoR2ulPGgGCi7-Z_{Y#51Hd z=~K*RO{W~`3QUow2A<|eicC1=7~o21*k~3zl9K@#m&Glc>G|;V&;tTmVNx zO-gcblA9&??z2d4xlVEKrMM5<*))oP$VklVt2_WknO!MbMXHtU@Z13_XPLz!Qry>| zJPaDOAF0*^<2g)<{jbgBQM*9(f>~vvSU;^YJo00}Y-VO877(>=M-KwOCQOy$@NA zw04H(n}8w#(<$ZMs?BO}k!+g-!*!o12ElyY534F8yTE@BEeTkd4L6)OY^6dJ!MrJ7 zv}A>d;MX+s!6+@0F(x?K3R@W^a8e837FCXcWt5f_F#hGV+d^uk(UrhS^ms&4;$A;I zuqb>z8J3~>@o@ae>166YqU_lUs}i@!FfUOUVU>5%@3t*xPom|9Bm(;(ajYf^EY+R? zqk^^T?I0{j#kZ6%GPu*HYJ*HKQ&CzI%9gf@WINLD>CW>=HG%{?+yXBby~Lpd*t)8H zE1RBF3wt|1N7Mm-lP0ft%Uww&r3B*GK_6YFStq$p@OTowHo3up6c-v-9R$VY#@F+} zf-X~|#tYNO8&}}QFKn7`Uh+A%b_uOWJM5CNWs7TNbOFJX;5e{&0<1Q#KtgR(MY^OX zzm^4o41)cJQNr|i*08nkp_EX*t9)k%$!Yt!DSXA3s1f zBhTRGwGK!{R?3lw*;k~5(lMWX_s%~2@yXRoZNa?bcF?h9Hp;5l;aDXx0@uPKvXHGbz^&`#`bhyrwQ;j zd3ap;T(1LF&=TC7i+e1RZoBTm@nxR?FfBTd?#I)K7=itV=i!!qMK(Ivi(zguQa?%d z8@9m8PK^?6p$2sRVj}8i@(#P^#(zcQ-Vhbjxd3j%G?^_-u%`IM)t^FLw%0}C(_A0>4Gae ziV*Z@OiD$h)SXZ*#1QkGRDz3Vo2{CLUd49ti{t~VYxe(X>+SYD>?e(xBc68h{P%_j zBaXfOmRTeRhe#qWw-L&)kDG27J>mj&IZh5IT^h4INpx0S9&gLx4J;32S@0$s)?uP< zVcq=m`I#rr&7Ao%eB^b70HQHO^CmkFHZ261hG{{km?Q<$*wKV;vMhWzA2uN|jzz+G zmn2#~RV-l9Mhsz`DcP$qfsJ~{lraBx`4g}n{&x9QP6o0v`CZS>X2U8=P*ULaQ}E(% zm!Bbep?a_GMj}z1A8S4Za8>|BpAS{y;->_t!6iAuz;sicIQdc-KMmD=$PhG%O^R zQDiC6`ML*I+^}u(d^aXM`btHvk&WYEg@(tiE8zb~X<2e7IXapBm=sHR^&(7{%uznh zutPZPgT%@C<@svI^a6x49=5O%Zc7x+_nEES*0_PGC(sI&Z_Z+`#10Ulp6_w_J1%lL zIViha43WHY%qyjP$9FnnXt1qL(j3Qzy4}WW>fF3FclR_rPm`b4)YaiVAj$zmn35vN zj?=nTS&Aldq_>$2+%vFe-#${AqZ-~gXje3j!Xi`@?1k1J)*J|$Wu27z;rBVBy0}Uya^G_;W-MxdlzPrS0rSBb%`o;%}Fd1m8F>CwJHMV3t?ZA z)2Z|i2Va)c>EvqfXPx*_Xqx7X@Zf8(1oK5+0_R?1WjB%~=IYnyybdUbsHFIqYXfUc zmRXMOb6sTzFs=&gU$YIKUO==&3r1m9V8=3?r)S?jiEQkDXzXr<)mN%K9+L+y3k2R} z1~ePD8nS)+_RVOg=9#(qpN==Yo80;44#2QebKm-PI%YDduNw?`z$j{^)sYj zut}q8beUn|x%(BcOI(aDfgR&$QcC$+um{eg>^AHwmi#vxA!Nvg4I4JfaXIpx1L*A;_v^zf{p(>d2A)=2$o$aR%&tgj4u2x%dr8x3LBj4dUljZ z%X@6T3HJQNB2qNHPC~G3E!*>FZytJQj?(|>Cufeo`qQZoRx#2Xi?*Q_4IB&G#q|=Q z_Bh!_?TV$%A;-JPI_jADdZ$t0+9Dsqwd5|M2}mqE|8Xgj@|NBNj=B76UKl!f3qgFmr`nTn96I3_}I|M&7CLiDho==HOVl_d-`@IVihl48hk4 zAG=wY<|8w2o&e?UK96Dc>_<2Mbadv0XMgR0BP}UD zM#l#LP6hW*Z2{Nns^}oHv2wVoxGW{3-rW6q>MJZZv0R~W5hZ5cs2~DtwX$Au1^lzX zP>eRq*>=5v#m`Pi5lO%WRrwrVyx*`nxMw+vwT9id;4QlS@Rk6!uD47(XOF|5cI`X8 zN_#Y$vMiV_9+#rA2MWYo+UU(!^>a=S3OLxcn#upZmv7v<~n^4hOvgYxnjua zo%B)auluQsY{0i)ojHEyr*j=N=UyS-G>+vAO2q3Mh3Y+XM#U^&(a^EQ*|2RQiWTv> z{fycNdYzgRm62UFxS*HF5p3xqCkb|`E9XR4Y-CQ4QOlL#blzvM%1(6#ws{vkgNB_i zc3?~HU{vo12h(MFxFSQ*pn;({mPW*49oru8_tv}|7HiLM?Hu39!e?BzS9rm&Tzb<% zscIYxA{1YJ-1_piC=7S{%*xZh7*(V@8` znPFNoYh{jIc1ZTS!?rnD;%tKF!}sLG#3)~6FowoV#j*Y%lJc61f#Y|mYKFj(>z1+R z*EpvfM017g<_=^Az;NH(&#`=ByaNY!G@gsOSS97C%WjVbG~DA@Y9PnFS^gy@(Sytg z3_VxPDCo@ekxLEOf*U5c>L-_i6OQ$GMB{XS!bN}8yATXhxOuV@TYV+kG)L+g^^r@> z6@hR{R8B@|mFW&iYpZr!Mz$%e-%h$Eg)fl2I4QEDUx&bT^S}!^oB8U(%(0Jty!`s?2Zv|gd~x=| z)tR%;-TdsSnNy$NJpRPZ!xw+N^vvul$5zo|kP}ig;*F3$&^UIDnoOr!p@Sfv2fQA> zGdIX1CB=N}+Ww#j7+BYSymp$~t0=h)4cFaP-Un=>yRLO!<}2M_-A-ZL{#Ub=DU z)tS>z&AxE~kDU5o=Ig)ATzUS+p^p|0&K@~7bMoAe7Z2V#^1)Bvd_D8@$u3(NXHLIA z`~Kxy@4Y_rZg_tFZ5(S%be&90xN3@(sEaVP`6i%trKMEDYfprUKJ-%|nr&e#nxPjk z_CTvtxBhoG>{zXAY`Y9?CTk10({ZxVguCi3W8d2FA-!P=Hdda+?%bkK6e+Gecm$RM zManFA=2wq>0j56QruKf_npkX2MAB$YJacIFlV@)|`N!EeKASoE;m!9C-a2&g=EooZ zcyD^XY@6@oMAq##c^r zt|s+V7x7;&zV}U}ak_C4270maP2(HVcw_nxaOyL7d!9@mZCq`99a?~m7hu9Jz-+;v zA&oOQVU2T*voMuc8=twiJJ)U(SgNv=2;NJxXS=4AxX2fKw}QVs&UDkFs-M4Zr%}|( z%;_oCOG-vT=T3icq=nNoUSRs<5@C=nqrNc67D1bI601>GxyLmN3k^@^4rTgmvt;Yz zZR);y5=n1Jq@S9-qeZ@`H6ylVOv4YNCS6Pna@sA)bO>2SK|{TKw_(Ut`vkUK@3s~+ zFtuDm21u=bx@58Nb$ESMCa~eWO9ZX?4?OvCDVg>LLhfC~!YD1-g>EZUXx?;ad96f! zLLBQ8Vm>P3zyS}3qNyl2*b-DHgS5(3vT!KNN_r~8ddpOfK?`H7G z&GkJFEuO7*0R*IJF8L^~23tzN4qI`o08o5w2*Y!+DgT^{O^(1i9D|=!T2kcXl;^#d zlg)ei0x#QWmDoCXu_DjZF6Uw!{FkkyzXMv0CE3^#?a5i;__EJ{BBg`75IrM;IrQL* zERwNWrCmO(=g~f&wUIS!@E7~W1<$VyDu;eUr{K~KQ}AJPZCvx7ts`DjV=J{S=6JPp zB8kb89G9{GvlMr?Cu&6{69L;8fIMPSB<)=_xf-JD9TWShMqyM)4w5r;WT)TBRD0EP zFQFqQyI_`S4aBnvMrCr73zB6TwP<*7OrN%&niGriG2u-R?B5dgnjq_F!_Lu$|2XpI z?KfUnLuf;gaSEJ;z`c&z+4$uOGH_$xik#ZA0j?~GZh#GjK zW|8cs)$nks?09b^61lUa4f6W?dUpp7^P>xpA@Df^JYEd-i~5N;{k8mci z*Oo=LBCnx&*X+!dszq;w`3cxs8aKgVv0c^a?ayb@iiEb6l9rc^WxOJR`Zz-1Y& zbzn|7x|5LPWLk>55@H>6I|Ml-B{=lJ=|C22Xv4IO)$Hd6lb@DSzOvEvvbk`g94iWb zz^pIJ$hn_i|$DYnA=DoD}N+-kJmP` z+O{Aif}cS+8dH%fZSUS;4hAUvzfyq}64_<3;SlATFsRxEf!xKy+yiL! zQE#sC0k8@yMV2=vn_MHSV-&*NM?s$+ytyAJYGApkwz+y?z!?SpHtUpsy6HFE9cYp-5=6JEY@?L)Hh+F!1{N#M*&*IwU5uDyNjJvjWzwU_S^ zsqIP$32yu^*UnsfnOythwU;*FI9_mu2LpQ(hj#5P0`Hq|oa1M%orIr1UVDYWvG?KU zm1}W#H$Oo}Z-nUu&V5m@Q&q+RfKlYr48z?mR;4KV~6Wo4$; z9KO{taGOY0W2{03482MIf?#$c$V9P5U@D7j<+i6)0f>#VaTtCxmT#6eXY-sjLyI9T zHaMdhc_UwP?OjP%|LtVJ_S+!Ss1!|hjHqx3`UI;`E;5Al!whI(2w8L!8L^!nb)!6Q zklugCVukkS9G>xs4AV1vMlz$jMyGyj*d$MN2c7+kdxc(FxvjCp$RG`$Y8o>tIn02& z8r+9Y_a+c5AM>Sq0|S^a6__J(DiZ0w{9m7DA@CF#>g!iblWD3U#=}P@cGwzKQtseH z)TLI@5Nef_(*EE?&>Gaa4ome`8POUlVs z5F;5lh!Ksnu|V@dWlT$!4ZA>j?bqddjF6m0O-rwTLIhQDABVwfBx?v(3N58UTFp;V zw^`C25f)np5SBo|dMe^A5xW1K-(9R?8F}!4SJU}*IQPBlT&c7a7_Uz&Q>)hFp8v*) z#@YXVr*W1{A8VY&B8KUMjmz-rO5<#XH2#Dw)Ta-_(Ie!)-+;q->hkn6|NRE}zppng zO`m}4F2KL%r%$wVq<22*bEJ3DNm?qJO*?uuoax3pPDgbt2Kl^kp>c8g`Ib!m=Q54c zjjPiKr;jwQZ2s>*HLGJ_60kyM`tbA-ESSOh@XX%y5&tvM)3?h<%g~eu`3(h~MzPF2 zA5JI8h=hC$?w(IESM_8pX!RsKBQYuEYu_HGKStvP7}b^O=ix~>YkufX+@oj|0Mo`*RrGd8Z2M=-iVLQnRT5pEpa3RMgzq!F0=-Ob0e6qX{XI z_C^)gYcJiB{PEJuvu_@sIeljK%F8n^pPPO2$SPB=gVfSeG~rdZFf~6}%g~4%>pr!@ ztR*G&FRDzCY?bL|iITn&1tSaGY?HvZ(sKQaW?4^2hubj9IqC{)#ar={c;F04Ol}j{ z3QXqYWWAWiVdFLy5|)IQE5fJ<{%@pgs2JA*#$lW?Uz`>R*`nc}vhj7r-(4nQmK?pfiIUNzfLPGPF6F$yyAH ztR=yUxM^TvzIxX-?GZuhzIXUALq%;91<8-CX+a-EIfwr;qH zGvs6}=E}Ro0+!8zxMEV;SIT<-N3BMwVRzD{V&y_sh+bE`%Tiz~Jg;FBQoFK=IDeSv zMsH3qT0&3P8S&z-tLW=u{|2Gcxn368?L^obGl^~DHGB7tG<)l2`u6ls!B5x03!||} zjBLZUnW6*FYDB<%6*=uqrd)D$^@j`W_r}vw!q+f2Fu^g{?wHKvWRgwj-@a~%N!>vU zl&IBh;N2Zmx6Mh?2i|Uel7^4U?=mJ-W5Q*hCt4d)BH(tE%tT}M#C|By;b ziV~4z_evJNGg%Ug5YoN?-u-VG3~1Cg4YR`Yb_WUHm9h_#SXU7B5q;}H_mPKSv(Fb{ zVOrgu5}{}R(FrOnMv$I2Jiu(2BEJjS#uj1BQf(jo)UtDrq^LeLQc&A$@7_n8>l~=n zSpAFo7lLC*SlztMum-3#wQXSEz6^0nift84ubmDG0og$^L&yP$TrT!X7I@qf(9tex zn9RBL@UEw6891+|QCD;3y1EjkW&%XFQaSdb)lL$>0fDiazjkOIpIo0r@z*?p%ngItsz z84B;E?l)lVQJt@GDPuI{f-$86!kChXZI7nBHg%`yz@n|H)Byv+3oMblzp#Hhsa>V2 z5uP2Z4fd}ktl*+*$JT!TTMhrgbfZ$NY1%3lWhar$Tk*mD)-&a9LEU`qnVavvee>w4 zTW`F+YT5B*Mv*1iS82T@BT`S6HTiWH`9AoiKOZ}~gy*xex`GAgcJC-AnT zfLNp6l`k;b1lx)^c#Uze)rs5;l{(ml`f0tik&0h!BBoI$x=rfu!l`kt>n5qx-z6po zA%gj!W`3hY(uqVOnb^0_IfKWhu;fCe7VR?n)2zjqF1XV`Eu{y|Q*RV_KW ztUMD^+E?QfrYrr>tykWhIep=Ot{m?h$@UKX^#0M=lUM%d%2OD{`{~pN@Mh-IXJ?PU z&|O-0^TJ%c#f~mAI>YeKeAGx4ldU-^@}{Jn8Fn71TU; zxJdhkvQs}i@xs1+1F$V2H!2ReX?<*X>@ifMx%>nfvIY2xk3YWQF8Nu3$_;`@ZmzbV z<}kDNcuNkNhrtdot=>FLgbhc)0S5PrFnpdX+94VJ!N?Fbt(GO9N`M6~IUcWT3v1!aqXF^dNyL|@5*>3Xu0y_>B<*G`UokVy6Kb?9tv}kS|`OA&NXKx(-=Ejlpi){P$elr|j zRzhhhmAsR*?{-~9-Y-ryqOvN}j0&FXhTyxWicy0_8}EG7py2_`gF+(xnlT4{zN1h^ zOpy{k_U6Ed^Q|lgJC_MoM0c5TkATesu0GYnOGrsy&2*Rr2rL-2$Vr766|IR3QXQ<_d_l}cv&)Ejngen?PFP$!Kt}E` z3Y@5hyHP+V%~g^}B%gc$X?@`@QL&1M)MO?uOEF)^>@ZbFBf=oNSk35$eR%l>YAVcV ziGm)e)UpNBH#@`HxQ8>9i3wWNoa9S*VvlWDvBcdB?Wj3v1=Feer~p@a1LiT?@7?h)>m3gA7w))juI)$ZWEb z9UAV1K-&%CWOyIb-=fX6Nd?}c;wdQ_?U;h>VO&^xmpM8b-uZ5Jwu_{1yh+wM3Q5H8?yB5 z!lo+Tc;N`%2grJyYh~=bQki@2aAd7a#h2z<(LTf2oA*4v?JnP8@x5;% zjd!Py!0W4PudOf$F(xG+#Kt*-nlL;=6losZyjAluKmZ9T7W3M!_s$j?-~xgDuqyI= zQ5f7TcbOeD$n(x^Gm@Vh$_m&(t>sV-2(?Z*vV&;*)?38%ScL2hL&B{V&f$ijmBM#{!)<^Y{PjSOfh9yQhafo zWirx+s4&N9TAYl=N|Y*0<0g{FwtvDh3HV=xPEesP(cWjxrl$W18!~JhCf+1?mK?M5 zo3SV|-^SqF-M_1?#p(2?(O`O7re``LDRNwjrKFU*sKwoRKgmT=hVpoe!DFjuB&gx~CT z7-p44;)`fo!}2$!uv{hKTPxQisf$1Z-@NwYr6*>eIyn30Q>zw3O*!ot5?;(e7^7qt z>_dWDw!H+d>YnLJc(M{KqRspd zP{EeOwDk=wWjOQ+nj=GH4x)1qW2=vB%ThQ;vPEWD;cFe@k{n5*RiG%Y`je;|cpX$D z6a~JkV~IGpwYjYIf*(!h{p6`pqYTd*_S-eEVo2pio|(+a4-eO^MLB%B7H4UZixzKrXz{?l zw-S?N#T$xTl02x1bD5+*6W@RlEPz{xB#P?5se<*YPo$rrbKL;<5z|ZmT zpCI2KgMY5S0~Z|s{wTgUcm1z;?e#NYEMEWM`uo?ZL{&fo}QAD5=pP8w+d0UWF&bXMAfjZlQ1!!MmCn52wQjgp5Ayx_@N{IuOjBG9 zd3u%h-lnpwyNPATaWCoLdZ|G;iH^_$oTISd7IZ!i;tg+wtw^Oo3R_AjzCPXee=>0! zOpP#1i2nHbI2f!LuVWk7#g$t>-_y2?c zRyAP}L^&QdZ7MjK4~^E(w*=)om6=QZZLZi@URve9x4A%qLnm?3Q6iC4gjR7CGs;m{ zGBFz1qlqS^sLwRM|5a(AR?Zo)HR-FWNLWPo;B~Hoxi?hs=T~#;W3m)WJcv`T8>`qY z8|FPG$-e68!0gKy&BURj9im&@>2xA=-UaFHhpoBL$ovlByw7aqw#E@!i$w@3-<$>G zKWBum(D8ha%e-m{DH>cz@iyHniy=C84YNZ1l&00s7nxRLt1h4Lrmh@`_EMANkVtuB zzpc^=JZj-x*qw;`td*%F`T6KXD>xBY9tkP!D|KJ@Uc!1W8i)H5;J_4_S~JYD!PR!P z@b|`r!D_2W$ECRKk?c`rIVAfIHa&9R^y8l?d#)#^X{V>6L-KRPWg~GHAR#eW>v@V%r;E zPoE%-Z=AA#C*Wn**(;5c(&BPi*FGwY$%*|gd!kUK7-fR_-tloJkb8q784)gi zf>pHIn%B4m1IZodTd#0obkGUDpo+k6vHVu}<^ z`k3^CN~}*+9Y%#p6Y_YML=%YdFMw}72)n4|kxc7YW(Tcdg}iBYUKGL)GdqrcgFal#3II5H|(NPf{(t6RUp5CS#Q>)^3M z`qe^R1Z-@X4*{)*G?qgpqTVFN(qR2TrWJXQXI0C>w5qTjW zbp2GDptKDDxM$E0wx92mGe9dGw2F)|Au#w^L!|BeZ+>fmD_ty z%StIFDZYjvVVYS#_88TxQ@fqgk$my-$8o#)S9#6vHOtmb4Aa0R8a%&oyb`v~p zpc;x4%V-Sjh6YWZGgDYEY5d|B9i`?rOCcs1+3J)oVtM)&MrCl92}V)mtarzlOEAIF zOOwdc67y6O`!0srF6>DLGf_neMtRJfN>dyB@o75Ar7)&Tuv-cPGzGV2P7zw1IVE`p zCWH}Kdrv>T)tG?U-A6{TOqEX%G7Z+b!#vdJzb8qLb5%SI#^3N9=d?j;<#omk(#xb( z5O_W>!YwMw59L(146V4HL}b}O0zW1LOcx*yGAJzO!b#8~$y(UC`o`zu0$SVy2hL32 zX|kMc2$N8(3l-)Q#5S-`m}WW*dAN^P8$>5KJn51KxA2E(mg;G2oxYDe07xUMB-Ay~ z`5TX_4?&IMF0Svz3DB*n))px?wsXR(8agT%I?;Ofpe>>eKLn!P&TJzOf@w`Jx4Ht> z>MFpVNo?CmK7t^Vo9HB?cKa#}l1L5{@&Np-cNW$|gfCpa@Xt^GaO=Xk+n=0Y_|+@l zynd(&o3u4OhDl_R(ZcoGF9=!i5S{tRl58VIpeXe5`h=97mQa~_Hm7rwkx}$}HV$({ z1w*rAGx$K+MkcP<pD!(*H$J}M?uo09JKsP4(RY_05OPzC7U12tUcGzh!1u>51(xUCR}bHP z@7Udgr(sInTU29_s;kQsY>s%-V0y6Xobc)}H!0b=jN`tCOQ#L71JY=#2K9ICDeMd0 zX*$9tFc{0^kSpyHjkwwkBfg`QT2hI((p_KjzlW;i<75K{j7*C?-p~<$l1w6EbVKj^ zNo>yKjP%)fg(Zi3l~a^Gr5v25&z#C36V0;o;J9hax*x&A9p}KiM;zRHm#i=ErU3&I$bQG^`@lz^x~Oiaf8D!DC|FN z>PSjrPmJ1pzI8cMavwDVSp=Q2?I%!YX=vPJzVU`Pl-GRJ*iz3hV=wdO^f`5 zo6fD?I~?4p^fD_qlY|NLk=HYkce@07I(H`qHijG9 z606RCOi3i0-hL30gzH5tM=j;c?Y8rR%c95FgYD(E6?}^rwbL+RwgP`!7DX4R>=K1q zGX+%Nra&YKTIiSjS8z>wrg5R9oU%6DdTDe%c=KQ%-xkzUB z&mO9LQpLnRSGf+yu2s&!IhP3hb*6G1ztVYYrgCET(Co3wr&i?Y4CgJCtUxwRar*f|VNR7lKWsL!)oioVJmYE=54#X@!hRugl1{TiI?|ef#Fi3n!ji_~p@WUw@{` z`%X5%JgN(qFSL!`C>+AF{8;lqgbtAj!c)NF>L#0+M8rK)=vkyP$xzFzmgbwKHNUgn zD~+s+auC`em!bim9T}Z)n*Bm4k<}d;>OQyyV#cKEy^_3 zU<>qZWbWR*YdAGJI6nRaDIiHf86M8^6C^7|+F+nTv$T*Y?=jdDWXTA90*{(`K4CpU zw(v}$io?CO7inFE)#g4D4TplaZVdAR%?cMy6Qe-^>2baw*RZ zyKPZfn0Ym%Bs>r<>k2TS0+igsXcHPId93V7OQbyw?%TCU1FB&q>{(8G{A0R-2JGn{ zo&NfnUyqOP-hJ!cU*0@;<>sODw@$n{K7RASrJDynx_Rh>n}_z_JowJdL#J;Z{P^af zW9`iB2VQtEq<>!f6vh)+LbuLb`1a@v-@JD1*1IpetdEGRAm6u?B1$q*--jEdPHsRL zS#YWyz7<;sL*Fs-e+16>gg_=k1k!C`b^ZftWOJ|-Kuf%jB`x4f^9(Ohy;?gbnIY0m z6wZNzP$;`_Mt zZUHC^%saMSB%5wL#YhTAR0`Y{cb2jgez4;NQO*VL8yq!N)wS7a$(jx6y!JWy+SnswYhq#{BL$&RaFz)AUf|I2EO4AD%6ra9R{ z`2-~cN}obZMw=lw9O60VGW}}H%m3<1Dq&B@{CesA4_11wxjH2L^+$k^!XYK@fhP$P zM?qx&YPlR|D`5W&jC9orr)xkWmPuEgfZ9RiwTY$EWinE31po6dB#}rWg`NUhQHjP~ zq2!Q@ejw!AUJq+Z!m}B*!(`O7hrndqMx{V-RExElb_pdLRiYlNalNc$8N=%{Z{5D~ ztAAd(vhdt%Z7d@R#~buF_~vtQrUX`=iSQ;hM#~JfKx7z@WrnWPnUj>zVj5Ogruunm ziSaV^AGVNo5q4|?&BzX0al=ezSSiD2&SSLs=l0Aw*+d$9ksZdROiW@7RRjyR7&mO3Jihs`@oPC~PK_|uutM9APHF$;L#lO6@!tS@T z$FuwGEJy@``T@4TCW3}|s@KNykgL{{}Z__0bQDWhzV*4!m{Pe z9!BiXua>QX*lX|tU6ExY5mv&g5=~%1=d}AxJvJQ!BPt z&YaH(gY6-EnKbusVhK{cMru%`B~8O;=5+1l_;~BG6%QnNHwX`h31?5hBQGqQEOF0s zse`Ft5;Y#29B&Lfgkgb!I+qL6m^5@{&QN%GqF_!X9zBY%O)P_baIA?{+qxRFsH(&x z^=jC+-E;HMv73jkVv*5-kG}bMKm2{`w`Xp@_u@aVow#-E*D11aygY<0qGuOxj?bmEBGQDq(#`Drk4AY!Lfo- zYEdPutpT{vMC74J=W>>1k{Xk(=0pz+H;ST-D_Zz|I)`>KR?y2;%&5g0!qF0}IWtfC zoMWA3hioOId3atixHy_;l21$oPaRRq%5!Rh69dMF%Ww_V1mwK-x%d)F)YE3-yCvvH z+l4G*`AVnt73|`=&S{Yha0{NRrVbXpnwHoJhGy9F?fb+)$|oPd48oy(>vs-tU<=d3!djB5SP)uh(bLd|KtWFi^~MIHjBw~HH@A3*0N zRzQnPtPB^4l1(ZhPm5Nwl5N%>OGFaU=tCgcAtudC9>huwO8}`rR=*f#-k!9xV7p{n zBjCsiK3QZHr%X+$suD>&dP;McN}4a1sFZnTv{iQ^+l{Ab!?)G%B!9MX)8Z3lwDldn z%Ox{8)rak!=#iOZLP%nk7|WK4b6;3}xj~JF3ByDJGg}AP@WZU6omD197hHC#ubgGw zICd<{$*elm?om7Kd^MAKnhOQ1_{+Dp?^9SIc^KB@ud34l7p_mAt%p!n3!%i;S_nl`L559-!A6kH*mS5IJ=>jw7|=Jm z>ECunr{N#6Gm7vj_l|B>Rp&7L>!if1=a2209@_=)*Zk6KutV;UX(Uda3@pasJlIDf zj~gq7r+4uwDzh$%=q3u_Rfi-=txw23+HAyNgpy%KgEy|$lMx4o7)N8MX37n8)@RgbuS&{c`f%-vb*klHwpi zPV3T`KiF1daZtiwN!ackr*<&atBXtlex!SJ`LKcsX6Y&}H5YBD=Ax_0C8;E$O*1L1 zFSNxbA0sP3<8w-2m1QNTaBezy&wq>7TDVb~El4lj76*aTp`&V{*Y!Zk1|A6VP*Ao$ zlu1~LCz~$s*bZ*}1kX5a!6}dFjzr{DrZ+T6ckkXL>4cTq%=Ik6%F7{POJjVzBTe=d ziQsT_V0;&GY=PyuT}TVeGloRyYA6(P87o?(=9N|7_>^xnBvFpC|D!#R*t~K8} z90wWJ>rLqw8El87|L-+MWnQU6I@U*VXI7N5FM9a<;`J7$1@chf`>s?;Hs$5lQgime0k>CfE2k^=rnmHskEck#VNEb;FnAsY zt5y(asR7BYDV@sk@gI|ZnVWgXm|A{9FSYkFSwNepc5Y`$69$u~k{iq@aG3$0e#6p8 zf>%e4FwnVFX7ki8xVbl6)EoYVt0_qj-OW0p%rcbn!>`{ubmZF)U%7ScH-Tl=!~{$I z@rNxYKO!;(2W2XtLBccnyf89(7tE&`PSlEMs&D+rRKqsGh^K7hVMci$=E9QX^821J*Msc$Q%45;|)V zzA0Iz!`KkE@LZk@3vO8GelTQv8HHzE=)uG6hLv;CbZyicyXM;JqKPQsaMOt@5X}e! zAB-6p2a&(S7Ry$!9jHOJ)5+lUl0$F_Fvl}wh#Gu?IVCB=aFzW94wPuv8W#0roYq03 zTMZgrEC2J^ZVr-(D+v!}d~54$JG`Q{&ZI+{MXW*q6Nr|;8lstMfh>a;I1i%&<9DF? zIIAKq2i%va%!!gtr`%$7r|NwRHrXJ_n=yR~R%8ows&$Oz4@Nd)yeO=EnQW%IzLy2p zd|TlD-I?83&DeHc`Acu<ki4> zC4_TEAmqwrQu(!mf4mM^AqB{8!JDf1H{>O7QjE>A910 zAH#b%{bT1GY>gxvzB>2SnXfKA-T}3`aO)@G=Bs^>i_)(hOZ61F>>2D{ng%%jLGYA` za}8Ivqy~@Z7adVU+O&9Nz5V2V8@N~|&tSCR3#C~G$pqR2A`JqKD&c6oK-UwMZgJiI z=+f5oHQNleb8bTO`XQ*#^UR zUa&kXf;4i>Ae&}BkrQypHZV_SkO7^unKyH6@60=tTRZ z!T^yqhM);DM23?`50Gkzm*Kw40>;t4N*)cHykd#_No#HvBBwJ#Ge5Vv$x)3yflpWl z;~R$Ep$VK(F=1t>4Bj@R#X=!)wL|@>URCXfRiMRFA?JTJ6uz&#S~#IZJ^AN{f0=Ay zy2zEAFfv-VL?R(DQ|e>Q>%0QHs>3gv4u{7*EX#WbQ$yVRxeX6p#x}dt(#{&9)P%R( zkZB+=Wv+%DTuD#2@`T^@1AFDnJC=oI`>AcVC=_UFfa}5(6P%8ZBc#yMWm3bO8G@T4 zzlZ5X+Py_sum?|9|Ji7jXFe%V9j^Ysuq<1p6Sp&ww~L6rUknbp@t-uCw_S!p&D7U? z=dY0pt@+xvN7C3^b2%SmKJ&=%;orqTSLAHQB{ka!Jqoj*RN@}Ofro$gM=d6q$mDg} zURyh=fgw@v=@k4nF(hZDNtE=J~@2n-Q%e{2an%*-h_XVXjHXs0gpA?oUA zYQp}PqjQE8JZLXznXoc*700O|*FzBU{}e=&NT|Md!uK}T2Sx{OT3MC`W&Z?`%(Qij z?4@S49&7FM3G8x>bGg+B##4{9ct&5u^sRyU=DA@8H#$bCku@#GU8TQjOm$VxId6PQ zEvCdh9WTDu?wt!qiEGt<|LTeFE`CV9f93FL1%JO1_l7ski$ zzHxZNop-N3Ui-#i?+$Y3z0=?Q`O@NfJA+@_Ndj>WeP5Jyxy_%o#&qWVD3{)X3w3SKQeb7|0Q$p&mRSu z&AmT&ZtmF}?abv;s~Z;%!O7uY zARAyZ-k&=Qug?Xq&#zB!$7-SAn9oRX7@H)b9)PeQajd5lvW?LkCN=nC@3hu=fvY8S zbSLA-Oa^*XSZS*3n4>q$2_{Wq+AwV7-B5$mp6<$Xvjo;uR&}>f*Hmi4r$P;5KSjt> z{bl%7s6Wz~QD6_q3T>5e`b`JktJF-71s%iMp|-A$z!E_X z4G%Pv+`(=gqB6q_+JrHn^@P+^e>j@b`g;dbiQc|YDiKmQfw`Q}Qt?#FiPSYhXJE=c7)7}>dMQwmJP?0%f3z;fP-#EwM}LnfbI26Evtox{Fu zXH8X-9uqBy?Yy1KN$`*n<_=|i&FO_XGy5U|`CSBa`plY_*ALcWRCxm)X_PoV&CkQf zmz)-%VB-D>&j~Z{BAkyWYl3!O`xJCU(W3Pg)xKR`5cD>ko8azd5eVM|D^NGV&m~|D zQhnyFcJFtA)$AC9FW(zVa32Y2ML8J|#xiB=>Sa#&td9v}pMvIz4L$rD-7u#Tv0mHi zBx!Lap*D@ou(RYiMk5pdiFF@(nQVy0;a!jU_!3}fs;5YGE6*s|08{t)zis`=^wys| zI6#J0a)2(4q#K6YJ%+<=BH!SijGE{I7iET#Y9{2_w|on(C!57ou3SQ&S@U~TT?I3h zTrV4BNElKyQB4F-#TG`TH455~xr$>|H{q&ZNrW|6LhGbI_Zb*3JA^-JTTctKO&E3{ z%4h?q<#ka86(u!5#bxbD(2-ooCZGmdCC<2_dLn@9;%rKsrd?t(-M*EXP!b-b&=t44 zuBYf;s?$QA>0Nc(9wI7bGPZ*oiJc;BpQR8q9^nI16u}}!E5E?ZM$m;_+r1sG+gZu6 zpWB~Y+*x8q79Pu=(kuO1;z|<+lZP>#Ky<(6<05!ojdjK9oO?X&_}kjB1I#|w1S(O zy*eBlvAN;yb&S;v&6saXGZI$f@pV}8)nv8s&L3~T_gphhbW2+OK}Jy}>1h-)Cj*pM zaFLUAf|Gymme_7WC4r?iIl=T4**NpAG|3)2&2?rM$=A>Prh4e!k)pF+J*^C6gQ`ZN zp$_AE7~vMGP?*Mz15^roDzI3e66PceMO`3o8zYTtG$N}OX++{m)YD=92*gE@_9-U} zPGO!(?G2A$+d{H_T?MMF_Qy3*)p^Rz73&AjAo8}t9U#PrP zxjuWSa_I?Dd1dxjm2)tZ3&AZ7C8;UVq@t-V*L$PZaD`~ANhcaol4{d!Ss+z|?rpVfS)>^XGXpBXDe$4~qQeMRJ3ZE4Cs0K^ytaTUA`V%G|5Qi43=PJzNbi$o4&SSzTh_LMtTQuc z8T|3SJl(5Pk|H|YTXr0!p!P;qvb{19S*9RY(2uJLLk%ej&;As|#U0@LNRAl{F*M;x zBd~U`PSCK$Bqb)Bg8N_l)fcc|WvcrXC$l4de_PB+FAfh9N0^=Pan zwS=qKFX6Y?FM_=vAx~uD;SZf|7DleD3p2xMTUjz9*u&v1Px9S-1M6n1F(FK+nQ#vy z1|ufrCJ{52NlAy1cSL_dZDllCEvcCo5WLXdM|RjUcoeCRmMoC{7Lbg{xLOTjVHd?* z*!QNR#+0N7(mO~fb%5)|6*$bgwr=;tv>ooSH4)Mt5_t})Yr_~Va-GnOC7Edk@B7f= ziWUXmG!a*luIAim%_xuhY#WSV=ZEGgLux_)!MV!0HNR-u(>&l>H2&9&7E)9Xj=;(_ zDO>EUDU3-DbJ69N7ueP`M;n7OuZ^zuD8{5$DJ9P{(vyQ*KsI3~=GC78d)h3M4HKXh z8my(Rp{-rZh14|!65%-qE8gAjo{TCgmT|dlg20&y5LH--s7+@oGCHFMu}e>oYHf-k zHm0$`tI9|RQ-UQ$4r5ymO>dk5j?bCl)T|&++g2iN#1?wc-qa7ZE>$%YkA%qcej=)u8MZa$?)6wYlIj7?&5A|Z-3taG)6EQSY;a`J{SCT|su!z? zrq&pr^edWhNYTQNUPtgIn`bF&Uqf)jA$#!e9d(TeVX{g1y?VvfKQRHjzKNSkMi>)( zRhr?h;Vm8oHk{`=FO|5lX7G$^qX;Y?6_Y;^B^FY`F6%B@6HwH@fWk_=d4T3!Rokt@ zY_L&FQY(+o0_lTs$lyu|+o;3@HDd1kdEh#%2)4~=o<0lTvyYwANKA?5G+5XY?7~U3 z$--cSft8&#pMaNG+FtEgbGV>Qcu|DeaCn}Xk=iw!8t(7u+w#-tVWdbd6HN(`4HPvV zNj%;G{f4VTla4G68lg$?Q&N(`LV%hkR$X5?%_QP?ZLJACaQ|Thduzgx`W5H9;@DGT zylwa~!>KIZHk>jZu7cuGi@_QrBjqeieDKl1fE0$lmp>~lz%`_Q9ExiM3qpSD z%2_9CVS5n9JlZu?OQ!U8gw`vo?vwCS z!jD%`QV6V8yBSEy7S&co65wNcxXA-#<*Ge)r98)06C!6$37JDuqiPI*ffppCtXlOX z`Q~npLe<tgfW%qhv?o=J}Cy4WZM26PM+IEmzM%f(OeZ_SEpnf zXmcDy6ZTCUcsI5eJ_7pY3H? zW)M*#d2j=svJEbz(}93=K+F_IZi1N>pKJ!GbL1!eBll4d5XBa)goh=4y~4&VKbSLZ z;qrxrcV1lh@V^$`J$dU7zgu|z^1_klzCG~V!gH_QI&o;>%;8%v|8n8o^9vvTdf}tr zFZ}wYZ~yD{Z{PUj*2k~hK7Xx=a)Q=WZ*k9tN$nupM6~?;i#MHGFrUUgwt7~nVFwYEw<`MGNY6wx$u%=pHR#5GP zOcWKl*tGjnWvT`jS~m$+ExNsY<<_C||Gf03qrKY>1D!mKV^dMV)R-?7|$TSuxpY5$)Vmzles>3O6!NcQ;v@mPPTz zlZqNwv~U=k6kI!0a1ysDc3^oSNPNfBdJF>@x17`gGf5G2+!9ljNmMlvjU|)OI7X&+ z?;b79oPzP_sXovplBaFwtnN5spoEkNG7QG2m%YQ%n<}1rb$3{PYPOa2RS(1r+)-jO zcgzQU;duE~*+vYxQDMWr+B1P5FF$||4lcP~2})Qe_d26$B*Ba}=vVIg+4g#U zJh+$s%+~FV+mU$1W!!*Oqb4Gtk5MOU_o<9EO^HVyJqo%l48AQ)OX$K7rMzfDNw&^q z(;^Ml%_AWLtL-FBP-xJMOMLciemd##8a2;TXJAotbH_+v1M@! ze3$BWrXQ|;L7)z*bR(YSO_fcfziF*jff|zwfaKAc%LnREBIc-5? zgcZdkJZo9Iq@z90(6nc|xzvh`HXK;-+^~_@0PMTWqM?uxNMxCTlONP-Yn6$x6`Zr+ zjq{jv?xPNR18SD!zqd_rAl5WW7aAQ>T@G7Y2QS8MFCz_7rK7w0QkDaq& ztgYLa)W*vSj;|`Wh@yFx)x&SjjUh&K8w<5y;=z_v>utFe8@j>&#BS%TkQ4Pj=w@bS z@;sBbu);}_oesnW!_0u0lXj`Jf>!I=)bH_2H#1qJ#t#t|Y+}DDTFgPxtoJ>)@2dAb zw+jjG^hll?Y~wCwTJTKj&ipMop$Y85Pb%Psx^XR37yYy96W%+Dlt-#3@BZ*Fn87kQ zo*Aq~Y2%uSgdRy{wVjpnl4hCFR)rPOlvvDvxBVz==d>i`@^poS^Cfy4f2^bNxy?bk-OvmUQX{UR*kox_4_!K$8Tf*=Zwq0(X5iuqf4d z6cLv&VWg-@S7IYtOWQ_$r)?vs4eux1J;JEd`#otYWNXRXi{cxa0bjr)LrPlQo50t5)MCl8m|wGv$jUsA_*lNZQ9#>1PfD=YHY0) zrXcR4hb784x}51SPU0IubUG=dASP(@j3A6m@O=a!`rEjqnkeo@ghkAjk5bE)&DatS z2=MZvhmBg}3VUsp3VV=DSc$1k2McnN7K^lX`s@KEM`_WdWrF^RFtEzLR0^)(M!|G4 zY>M$JxOI`aue}9KiG-)Lp&)!p;m(DlBnA5H&L@ZOyn8(L{i`RwyZ9mb{*}Xb-abL@ zzIye$i>Jx=FP^@8?D*IFUlEVdZ7s3hV|=SHjLJ7BtC zZH`?uuoQK)#B{y$0LX^$uE>aR2pjM!IZm5^MjXYtk@vI1#E``%sn#!tbt`%mD_RS! zEh{>P{Sj)UzMKHYf!0YQ3ra|vr#kF^ZYE=r(YkyKvVfSi2&a;k6Q!OR8Ey+PD zY3`I+1eR$_5NTpgO}__K&hWd!c9cpIYx;@Fv($lH{L7!7etN4@-CsbgNvAN=1lxp4 zG7XbqJo)s~qgyxiZ|@n}{)=CbX=H!5*pp;>0yGwap`TF9jgG~YT}je;#(U$aF0HXY2pKYwiQ z1K3yRkIbE)`(W;)xsT_Mr24_R=$<<{e;AHjgnys;>iPKt1P{Pzm*-BxNr&f7O}o#S z|7C+S$ObL;cnj^V2ca0|F2LQNoj-&_f>9hJ;S`zsulZy1N6GwA80BFCpB(#Nfd&7w z!ZN=lQd9jA|C&FpMDMSz2`mHH#fp66BK$1NI#gzCFC1M0qg~tk^t~3Ai5309(^W7fWfX5{oq*P#$4$ z_qc3Shnx{*fbCeUK{pnbk*Hc!!L~3pF}?h4eLq&+$#`sh{P7goX_&&68J1NR;d{n* zO^@w*aB$eIGN`vL7V9BwOfS!oUJ8#nyg`2!TbL;5q9!=hdXR~YZDToRWZm&lZH0iS zw)&$HQ6iqbbUg;KS_~q()?yG^tU+tudZwb;OvUex-Fflw_pcpqW2Pd8b@}0B{ieEd z{o|-*TUL(B<>VbBER(k!Og7NaNc&}yDVxl^*+3p^z232xf-mNkNLR0?+_4u+!8Y!i zxhgG-6%oCjj5wcn)k=jTsuB-5)I$*$?O|{hi5S-1hht60)vN6gyLT_?m{gBx3fNbx zJ)%oph8f*Xo8+Fxfp37i(L$jJ?5IoH7PiSPebZZn1()hLVzz-kGAwjqnN8%TbT~F* zb7NU@s@*Bm$qEWrWgs^>cAJHPA;0-N(Ba9~88&#}vmt|~~Cd(L>=EoB2q?4;j=$lVY z-2UAg-@NeL?f0I8zuV}tgM||I2rHn=j^ZjIXe{RKvD1)(`A}@=znk5f0wyQih5{bW zwEGPh8I~Y$J>>t!FRN-Cj^pQGJ|&c@@zK{iFg!iNP@#|koOjsKO>Xf?_J804V=#t- z`NYzx+c!`5={ACClkJuLUQxXW#G6mxQG1*$AIm{qAT;ZLhdAId# z!k$TtM>LW6>h0}=3%C*&;c%_`srts}qK+!bX#JvIH!*|Bau|{1O{)yL3}i~d{|Ub4 zT4Z4&S&FAvU|pWg%d==(Rm`9Eh%>1~nups2hhdOP$(if~_h8kW(&n)(JAwo$NO;<=gYaXFQU?S?je$)E z7k|7H)2WNb_XZCiemh}|N@h5{B<(X2VDe*1_`Z5cdTFMRlQ6w2fr9QF^&}&A8XUtBqZYNC zMw<4gB>RNWS($9%xdu2a|3VmQ2r*X>i8!oiuE=f^G`ex52?%+%k0A0Gw$nNXZ<5JL zo?#uB-q@jM$slw3dYXV?+awjzs|(alHAw{S=ZRzQKq6FMOVde}&EaV^G##$5Y#tMt zJkR~h!e*oMDx0@cQKn>27+Decp;@V_IT5WrInHZ;&yK2LtbN@CTt!;9KltF*@fU9W z_GlaJKQ*GnlTD{(9>sFGaB^*x%f*qiK=qJHtVeC{BMOgSGusrl1$icdkb!meatZsH zk+Z~dyn4wTT7spn0P_h-50xElKWN=yqWF%MAviklsHE>ahnLRX#D zdh6Jkg%{7QWe*||(^iuRlIdOl)$P~bynXD0Z%%!B>*HVDe)hcvqUr0YmrNhk`GiJbbv4$%w(`L_{EHIAj&A$7go|2{2rw9oS_lV_C3eA&q4 znzK|SNI~RN7#Ne(FtO;AY!db)3yvT%#B|M-JrCF1jIeJ%1Hs5+O#>`p zH3LJyD^qk{MEqxV8f=$gVKssGw)LqVmjlt5H(?dl1ekXum08!{t<&EkUEkWA5xOM2nI&m$z*&OMmsIIcNcbf^?hnHw{T8T643p>{qRiMy}Xu-Eg<+ zDOff(*Kh_#-DoDi7skh=({WI+kj9~-cAj3zj49-tHf^132Hg~Vg$>H=vSgd;Mtbrq zn5fzypB@bR$F8zTBjS%N5^M)|I<6gN4TWrHEQ_?$AT%)IsbQBo%*xxWqau^Rm(3uZ zC{VJEPq_@enA*_Ni>a&1D6Awyj~?I8HuR-SBG-gD*1F9n8imcLz7DIT2dKm1po2wD z(g}{3)}#Q=W{0hSF;S%E6zTozk>GyLHnBG=_8K)Q#A@3|%=h*YMf8<*XQy?}PFe^n z&uP2by5debcBnp5m7#unZ*gM4qJdkoD0%wnMM8Jjh<@=4@^hOJX6QPdIY|jEroqt5 zR6kE;mM}nuz)%9Kw!=UK7wem!d+a?X28^G zw_ECG;#b90-dUWGaLni)l+2zoEy;Lrh=+b1>{clzxan{VE?L*H+Aih1_9+`xNk;2y z=GPTu^9qut(~?bcYPCs2Cb0ZI684nb_-%vu(LKPUnGb1di(RA!k_nk4%SaDAyff#G z@$ny%<)@8F3f4Tb<1IfAD`jD`uo!|1+*<`zAf(pgAw3vC0h_~Di)1?jHB&pVkOz83 zT*0K^*enS|wGSm9J)(|5W*}xo`vOV{tmTiWP5Zo^$^A4$Ed-Y4pv!+M%se+}F1QF< z$o-T^sUaoeL3Q(8jSc3+?e{-i_~`eoG$tO1=2^IQ>h^1w7tXx+?R%&H=j!hlUVQG> zp(6`t&fYry@wXqoa_ju7!F7??S5`GO>54Nn4l}G0Wk_Ikd?-=BDuS4IJ?P%{bO2dR zFEeaw%1ytybR;!GvtW+tGw?B2y>0rpJGB3fXXxG&^#ervw3?`0BIwB{l6I(D1b z#`JPK_XstnBt3A>`2efo%a^`9`Q@oEU;Xj}^5q}C{N0zY!P{4Gg?#w|ygKpa%U`~- z*7X+LJl^$dU%mr(`Va}WiEaARvkxn=BmcK+_wK>P zw=}g`mPtVrOE$K^>}6zc8J>i!T@ZVz0q+pX*37W+@&DUJrr_ZaMUnaIk*c(}Q#tk5 z*RoTbc6YRgx&aHBdIx!SIM|r zFS^vCZ`s7bT5lS*p^sy%H4hGUnlXb&*yl3LFj*Gcl_Qv1=5NjO^+f@Ikw(?tot zNcO@P5C|H6fNa2)WMNyRRK**Bn*hmWm~&zpOoJ>loiY)ufT3wTlUqa>Wa~&@r;>k6 z!k}P&;5@>Lwobuf3vT&r7l;AR+{t=v1+4m2074Vk(x7>|mwmg@KOzJ6%$tUAO5!?6 z7DH6#sDue3lkhMN>hO4y4PWnni>MJ*4X!Qws7&*$lN)jS&*5+l@y>beZ&ijhCGH^* z4dl$71c&4``2T(I;lRB)`Q>}y>!6GCLFeshUmk~hf9uQF$d|tXKkY5Y%?b$Su}`|T zv9OEMD(siiD&cSecwq1XpfBs(v=(_8YJ{iK;u&y&$OPEO&TYT|x<}|9N_upi=Cd?Q z2J;B1vF2sAI*O${DVuCE2Q!|4Dh-$WSoR^@KFR4C%(gd6A91iy>U&xJ zRtSE5>^HDf=FZN2JbyGbciN#2KKk|k=jV^ko&4%~@{d#VN8rQv=bxGT_^ao~$LF4R zIzfG&Pv$iPL+g70fZE_()DPlpS|+_y`Iv;c{E_3FD*ZujmKGg8Qt zvaP>od;iGvuHimxPGAy*If-<0WFr~i`&hO{{&vs!?#r`Gf!uqK;$Ku{oDm5JE!$!b zNZ2i^eCyMxX)@B+8(f%{C6=xAW_Xhl*dG-4l-b`$f6y)}E!)VMy3_T&YsbjowymRM z$bl11kbZ0o$ISeL3V1awrB$84d-g%WB0Uq57ijm)i8PZ`rWtoezjx-U%<>#o@>#sl zh}-dBNOy=yeVKHd$iqf8CF<#33DQBajT;3{rr2KNudi8h%3vMZM$uzr$D5D=@aIWFkt~(}=td&b*ADDyqhImnRG!rC4Jh6No-L`M%Eyg;={Hrf;p_iRNwfNOhi#V_`VPSx)`y|3}OSowoYN{m+qogWeVs7d;Pd4G(63PPIJ~8+PBdM^ zd9L$Pi5qK#8EL8bEp3|w23w*AXC@v%Yhd}!da=%jyOs;EYjK7cUZ9Vb9L$<~^^sW7 zro^I3Y*~j{Ge*9eP{N)SW%!K+rum*6+skCZ>M%{$BSEW;*|Ld(oTqFlIi<#y+SJ#Y zjeYhSCsU3Ho9=ks?1U$nPX(E8Zus{7ysw#v0KMpy#4Wog%{7V?&5(| zAFQG7pTJj}SxY0)bi~bH6KUM&5Lu^)tU7AMcZlB5x25 zvKqI2?CGc7^>3TFNsDQovmLYj7r%IHjaw39%ZvzomjXu@;mXK{?6R>m(MblrzAj>6 zce7HswiktcZ$)8E(bQJ>S0fZnzz%k%2B}_VqfC|?lo>YXE~%jin2S-D2keSL_||Y? zxJbC^B6^rIG6YW@E~*VCcUq%hkx|_Hwy=@d^k^Mv< z`A|{7IQSw!x@k-Dmp>6lQ(6vr5#>5z7I8rAt?oW*ut{om55s(PYXlc4{IW^@a)nR= zkNV6hZ0EN1nN!ZH71-eo$FfL~lrvfw75JfyF0kS=wrS#A*cMoTCg@t*1;f?&^VDW! zH*!x-2pOrdTAZ#A53#6hU^*HO5K}GE69HylIye$qw9ufyJ05 zFmEeVs&w`Ad;Fe7dfXmK)1<%1h1cP+~8S5ncCqIx>^A99G!Z9GkL!gzv=f)fCuFvvDmwt_E|7NT?}kslc*H z(BP$M$$exa`QoL&9sb+lFD`%a=@%cZU{Y-!@t9PbC1V-3CXA>r-udFsUtIa(;@_V6 z+p}L>a|ZI+7ax-^K83dz?!AQ}{`qeQ$rrD}saL)@4WFF-+rhxH3&)k1qD3O$l(-T( zTB6}f(h@7mFA_rTh*#mH5x7_N2ChW%l2q6EWA zwE5i%B)FX$7DK$$&2UDaoo1vP+*RHQCHM~%5-zBAMA<}@ka}N0((Bk`%c8Z*u*bBT z>7e4Q7KtG z6TGaWk4{+NGr6n@*M6k=v%OI~nBS=1k4(RvV8fExW^oUb;QWH2N_Owwp=ZgcEmBW`(MLvjsk97AF>q8IM5C&T>gg0&4}z{31ceD) zTU5L#!?2R{q=0=_O=Ttt+6tgAboOI1K`l$HdCj*1$M^`FWYX#tR`&=)HBM*=d?7|q zXn_TTrXkNzBJBwmwt7&<7R3`)l9Bqv)59P2+0I0s8L)Y@FXV=kpHQNn1VIoEoPKUh zG2Ng@PYuf+9iSf-sMTrK0d@u#m|@alQO9I*ztrhshgne3m>FAlx`>u^mMi_H`Aolo zWHeRLJZAKIhCN`GCy#ZOoFZ_DB@ha{E$MX>XG95m0Z{xJ5>#znPQ4spHZ;+}J2wv@Li&R0Kq$M3*rxP%u56G9Vz@T3H@^u*4n*{!P z1qKEVI=uQFe)dPleP8r^PkniUeEDm5|6Y|Z>6I^Ea%SU$;KdC{AM3K|6Rv(De7b0(>dd14 z3e8YhNhKQc#M=D=NA=8{rCII-`gHV+UDU6`LZA5%yngY}Grx|)a5&8gve~A(P9+%@ zQ^9S{W&E#A1#QD%(L^7;?(!j68^xIq+i2v6BT6VRy&s)q88U!SHq0U>E;xLX7W>A>AAf>qM2DqllG4m62{UraBt^ayHRPo&}uct<>>?%pnXCX z0*fpWQq%;P5uuo(CSBfd6Pb}nQi*ujID7+%|19_UL19iYgEOp~vrXIJe|@c*Zdlt| zt}ZkWikS>Y!_nZ{i)Yy+b%l!}wUra`y>g-J3 zKR;OR#@|iBFREJV-iSMZA!^kfso_ZPUyRY3O&;}alSeg0i`FZ^b-3MSrGIMVn-d4V zJ@7Kwu<-ip3x9s28K{PqAnc;NqDm;fj$Cu!`7fFSe=L)Sec56XOv3gxnW9kR4Iosg z-$>Nl$+`1$AIzO~*orO_czJp5cD-ICF?weHAe?o9z!wfXU;F^B^6~tUl>2K-XN`|PPUb$G`(XZAGWSO~?<|>r z2EOvq+$l2m{`@iL{{9Sa;JV8%#lxD8J!aFjmz@?xo;yE(5Qg|?;?N@Xuy-rdpVuT{`Cmyo1$4eus}QnIxdD=p9HrkkBAbB z*Biq5vVRxyzt8@C|BcUXy#4pXH~w_v6LRCsjdM3H{r%vL3pYNw@$TREcV2P#*5SM` z7zyr~Y-VDIE7-3!4)$xS1p8I28rHP>y);NxXB#CG)V3HVbkJtcQj2zi3>^Pr=9Jgx zaE_(6G1ALT_IgQ0Ve(r-AmrLXl_><1K==_8|H z%I2ErJ?hor*oe&ycRhHl=9R>JdL=5>gU1r}l6?4sn437wNCzNYwlA%tVkNh?LEQH` za_l?om*Nu+^4NqFMRIRZ^&tXJ7V#;Z__QK8!?X>gNz(sfP8_Ri-!pUUAjSY^kD(0vX_JDW$F4+H(PFomvc|f|2 z+eJotrlxF!okNf&ZP=x|%eHM-7v3_vY}>YN+qP|V*|u%lHmAP%V`5elv&hI*?lLp( z=bUq2hf-)KtVV!U$wCULe3}dr6}ndd@8~^5=ij45++ltq-MRaFD!%SpSWT){9r2CYm8xoZYE>5} zF0h@5f#BaSC~*Zv`0-EoP(s~Mzl_v9QRplg`!zCqK@{V{-|!1wvGg(e=#gd&YUOn( z=NlA`iMsAZ2%w#omKQ~6Fa?rBmL}@{V%zN&Nrm80E1599b3J6W zlxP}|SOFH1?#QzR9%@UcZ)QGt?$~rFWtd#Qvm*8P)a3TV({1f+EES2P+VOW@%JoB8 z38>(K8fQdEHN-$HEJVspE{eb~3yg3j4`9`zUxfL=rbDevfY9ax4I6gq5Y2B{7hOJR zapaM*BUPzyk8x%6GGf&>3~6tYqpT$o*^)&*_s|0)^^nYNq1&YAA|+RlwD|2~@WHE7 zonFEqC9$%8q4I|t-K`kJL`7L{vNLrQnY?3Qp5m{w*WpGJ21rw^Rt0Bmu;!sb2Fd3k ztg4^IF$Jx0-qR7uEi-xaZmd|c7Rzb+5Tur z4&G*4nUBZxjI(QQ@jQ;r-RNdJ`Hs+8iF39cBN>6$MtX~9wYY7?*IM77%KF@Do?vNp_HTER*u1@g?- zgrz)%>SsBo05hBv8{)<$O1Mo>>PSW&w4g@<#^Q)EVQhHYqb?)<%-G*lyrJ%VDr$}Z zjRZ^oYG~`R__^r8V$t$%7E4=+3z8J^laSSnHCq%_pSjpuelxByN7wN8^}~~O!MC8z)i)887B@0q_DoZTnND8v_e`@?!(f? zM2P}#^DD|Feev;H6nOF)Jx%}NZ)WuSkZU$6L5289h1Kd|vvLelMdR|k^t5fliM-f& zEawb*6vYdIUn$O7#m~#In)!?(u-51dxBW#Uwes04$e-3piU#6y#Ebqq-Xomo(eWsy zQ~w{jpUd~l$0qBaEpBJe{}q8!;@;+tsmUsPO4AT4Zp!i7Q8?;}^tDjrE!2x98AFeQ%lmNenu#H&GNx{sG* z4DTjmX;7I^$yIxPD3y6TBDo3yd+m3P9=QXcau*Q%%qXTFl9kH_sQuVt55bT$@$C0* zpNO)uyDn2z$dC+SigNvNzk2_N6*A|)7hBxw`l5U}X9BUmvRT@ZO{K_DQCou-PK~b1 z4_BBsat=%O2x+>S5Qh63LIsuyVLs9>Z~C#Nic7FvGO8$#8>`uLIBqDFG*ih($l7;w z%N|Tx{a$8Yd!~EI*dekz`Cyx?tt-n_yb$um2Na-oIQwjC#wu+*xrE+~tdDRFpk@`5 zp)}OaD0elNNi_aCb2Lv%R|Me)xiJY!&&g*Syf`2=G+4eKjXUJ=LMhq#5<}JK^s?OUc0esHAx`C*f z{1(V`_OhxTBLvK=h3(m8c3IZ0NgP_Vs8E!7uJgG#8K)$br{D8>^Sn@3iAg01D}vB( zZDDx++F?6y!YtQoi{=^W-HW+;(0pvSCk$he>Y8+z{3b;#Pni}g^to>Qi^$$4-AwmW z6Pze8oBY37AZdzrs($9DnYD+cHN6lV(|>6$P=}-Ts~6RA(C0r^7wyna!Ol7usMjUq z?tY_Y^bZ!Va8G=0+!Tpo6uvleBh>e=DTr~b7Hou=O74iZ2ERpY<0pKEF;s??nalGz z4e&OzYp|hfwEj?i*jc^Dazz*wJ%Mkq7f76HXX+9yz(w8qwrez)&=mxFgmp!5Qdq*0 zVbTmuHlrH#VrA2WM2BU)>$wz>6tNvYI!CXU;T;4Hv#(pikm355lClZ-N&Gsk!)TJ=dt-5U9+fWICzBtp<2`!K;Z+@6`5!xT$!hE;dUqzQ5HKiZLcgAPu zCf-_7LJ4EoKPHkx5}=+g{y)T<$+OCoVzttrAEFz30+G+_yg>|iu^BdXiXLkk_kScs z_18*!$SynBF1y&Cb%_^PAd#AcF}0T~P%h7B~+`e9?8a!tY{ z>ZFLNVU^u{=^)iZu8wwko@jIq5vm$R9zSR=sP0<)DNcYE=gs5S^P0vQvS*pHGz){p zBs25ViDeGlw675w5#%yjO6X#C{Ta583>ML%hvFqiUARWuwl=%t`>I28wAa(6jn3E6 zb4WM4o9FJni9d9;&<+(rT`dx1bM1{6`iogp9ALK-^x8QR#(JHDLzqRd|CU5vP}Tk@ z3&%ohd*f`~WS(DR(W70(RiSpIRvOlr&DOMY!C3ft+LFk}hs#pfOv^)C8LSoDE#i~|zIKV! z)jASABj-;C&GP>Z>cT6QLjdgM+Q)cv`IWQYp(wPKo%*^U4xJ_0V>m~HW3bYM6V;bp zKc}*@x88$F+#ipM-tOTrZ=`Is-Ai!}U`0WTRwcVr7_ZPl0p! z7r!iHxMv!X3n4r`khi?JboQr$(_SfO*c01cT{FxrpZ(nJ?R{+C$LE6{CcsiQA#5Ov zIAlF7ZyCXy7t>0oYF(i(lU?YmKNwVtOrSPz!qc?23{K*xk3hAI140{n;M2r$QC(H$ z5qA-~;s-LPkN`}K7a|-r;q+6xb|o6UkaaS^O;1c(Ed5MZ<(2R-kvlJAkr~^tsc~78c(L7$wpU80<0B!6I4Z>kDh9zo<0XYO& zQH9C$Y?L$H4YmJzU@|1_ak);JHVs1kA!|D9&(1bi8#Se$kHREF#i&5bGm0LMa!-a) zq=Z~=OQ~TF{MUZw0u)`pZooCb7GxVA5A+L{e8 zTM+~Jkc@5N-C}zM6C;#Z`ip8sh?eLt;zFyaLP*T#jrF8#xE%K9wx1KQ$!#W#2p=`w7@4h<6XyA)`^xHk$3Xn60`|b0z_1GRowT}L&vh?)zcZKs$g*X+r=bw@gd@;+ryC$i* zkjzNVcwZ^1C;>kL{s`NbUP+@`?J3Xw?1d`8+VEE24=01k_BSKLo~(yG zi5Q-^;$kND_Z12!OR8J{3uFS^fZVbF!)Z1Gj9&da1^caT+$wu9327DAK zHdH%?N~tz+q!z5y49m776ka*Y$)G`v=bhmM0CiOfR#SOe;V>sT3fmhG@YRo^DyIwa z^vKg0=7*na%!;mhAdr)MFmomW$u4~_O>?kID=|*iwQyVhXA(1SV0ee%W=PZkLM9I8 z!W4tusA47cR#?sVh9dML$#12c%}<&^JVn_r@t6*$b|h; z3Nc<)|BX&9R6f{vl_bh)lE4+uazAf&RDhNU(ZdDT6i(=35YG&SHHzAtCd?=eg|69B zV?4Q>vJJir$8`>7B%fj`1nb4T_j z#66)l>${*P7dR!jzKqbegp}i}ldrY~hW#q6wmb6(Nt$ly>>orpis=_d*RqHj$u`ME z!D8ByOE1xqM0v~drGqXSNW}u_pTz(x6@YsCkd1kpyC*Y{-2V3wRT8?E=g>BZTT_JcY~}OLKT@@ z-Pvll7O7-FG9N3R0a>-FVq(QH|F~6-Ms1lwuJ*FNMbcjl1;_OA6P_$x?Cd?%<;x1X zP4Dw8&TbDn&l95n0!q2Sp9-|}$Q}v&BfMvcHKf9Yvig7|q2g-Hm1?1A95;;O*+4^+ zR-2;fH8hz3jK~%^RyIkHBX=CRD&A`uUvo48REz~CS~P9QlA$pmX>-Q-n+yH8p7A2I z0QO}|1YGEJWqnJ{*~s_$joyh?r=ytWqd%MJ&^oWqyp2=~^tSzjZ86Bid>K8vT8r%F6W-f?r|Yu)(F#M|6+^^mOc+4JiexTtUh!I_vwXliEdx-Q#^T<+|(bj%?dA zMPP!O7&?Mx#az&@H4;=a-)kn5^Vl0*JrMq0|KFU1O7&s366sAoQjkelWeB>lznW3D zUem)Le%wvb<46Xh`Bo-25edlLFI+yG@NVgZIXxpmf zw<6YDM?Zf?_j&B6M*re@b~nwl#;IlwCuNs6Uyn9U)%HlOwXc;=9L0fg>jj-+GV ztm&rQYC9k4b74)Tws@{hy57S``5yJ_1_DvXmgOSzHDp9$do==pXr2_7Jz(mynNsyJ zgKh8}W;@a|3597er!t&U-6WQ!n8p`f#BqjkviOUbi{l$F8251!V%tj?0it4DRa#gH z2eUSQOJ#dihDjQ+(LAz(kx9AoFFb%HUzN-5hje@p_RU)$e_gGh`yUyS9oUd&@N%1V z=`^Fbk#B_!%Nn0RU z=3@MQrXga5Z8cj(06tz$5{=UT7vQzQG!nUU_iGlIeEURmq7%G__?*@L2GZE z;$Q{JFe$nKw9=05&gY7sEA6iNIs0YZ1O5AH%ZRN<;rg5Zi~FWH;bvdyK2@eo0ii*_utWkHG>9M+j}8DVIxuN)o2X1D zmHj(4u|PicO+yA_Fy)Y_Izu(JvuSmoLSR2@P%5|D=_zHTXJ>D-A^2DAB?&4dXoF_3 z)8^)kNa*Y83rh9xq#`+B6ZWMQ7B& z($!}o(tR(6_lLlD?x3I;hV?OprM^?HjDw)1MbWGfYpSe_R?%KnoAT&7hQ6EpPgiya z+H2zZV-ocyd61qm8K)_ju`RY|o*V0U?*>T#iX-w_75eIqEu`o!l7M7rP7N1VNxHE3 z#6;vkR=hm^KPrVBrG;uynV#*RYf@*{U76los3MdT97dxJ%=1cxc~MnH7M$W}boW4&{kN zoDnr4J432B0@f~VHYqG{gt{-{? zgsMI8mrB(;5aw4ICFVffi?4uBl zK|0cWI3+KxxdxUTB7V8&xNk; z8)zzI<1uMSAF1S(1{?LiPc2qVb2{B<5+o!hRbc0_WZKMRL)c}p%gjQu9>cMUkqv_T zufCFm%Yq`w>@-lBDIILkNd3_8(05Vm=CKC9q+vVF@r#RzStZi7t_1Np?767(kM{%I zF+dz@vPA-4(1i*;Yz-{=Y~WyBv4mA!OmBU}L|dga_m2u;rSA7Q>QordEJUeew?ULA zoUR7f%&An(7`zkY`W|eOLT8h*O^Hywg@HZ%+AtEv!w z6c8T>X$>nE!2sQg3u}$m76tTO@&}VX#=*Ra!srFQ@P>l<9#3J|;6xC(FV|_}OQtv- zc?g*NH{W$dGwz!i#xQ?V?8#w$=sQJxm$g*H%{utFAv9!$1seKe_WRkgPUq_)v`$utLbu;@&3?znS+niO`48OiY%uGf0)6-HUgtCKaqz}p$N8(` z_%OtFPsMiK!S;se^$bWeJV&e4=Z>p`D}=nwlAo2Ci>n`T<4xn@*G1HEx4SoP6E*~I znV{$BxSsN^T!N&=gfoof-AP-P9UU{~s+`R)3*u8@>`>Fs_F&@AwZE1s-?eCAzRVre ztKFO|4edUv__FUV@G$PR?AePfFZvb2iGej;TDq9Dfcc9O_JriX!-KNkHYpbzJ>_&X z30oy>CRoR@JhW#v_V}oc2E)I``nVO+f{n%uslu{7gyYsYuu|B5`7=#71e1i&Qe@K- zc^;lE+KoTU_SF_{@(j4OZf;PSHU2dDePetfjzY?J>1Gt~wfhCe_-kNs>T1FG$@GCHWsD_6RV1^s1KMU$8kcYr zt%9hTSFqDys#5SWyORi zb~m)G&6k~{ghzN5xy{0vio?nJ87NBvCiwNY-6;64?PmcAT1+{8(W$^t#%aq1?s_?R z9&80#Ds|9k>K{DJd1Fj$fR=qEO{Q@CO^`z4c;mR}#g|0nKwJ=c>~Y2d4@tb{f&FDc zP|5E0MxTQ1KDi)LqE95#b%yuj;~??*1Z7E{luB-^1D^P!3|Hb=;ggB!>F`L$+|Zju zmA-nY|8PS7pa!`@&ue2~RT(XkruPTj`%e`D=@R=yqBb4R+=UVi=pq!pU2za|zvNUX zWig)&xtqC<_mMX0C1m0i&R>WKbFo3^k2a|6i=0BWqcwnG-|q;6bQ9yZx#Wo7)y`B( ztUf6!GPA^ZLPGGBD&{|u6ZIGhM!khyz#67BE}#21CuOUdTL0qe#|3yor7Gi@n1ICY z!64uhIdw|2P%tqy2-HBxlpS|3niH_^msN__G}oi;P79~3ysrI5`+KqTb1CS$n&QKoY4( zlVd8>iJ~mOl7n2QFDgJJ#jN0jzLHow{~M>)@pm-Z{!ue5 z4O6aKn;@7%xsUU6C}?tm(%A4@WoT&YzS33C!u}+5MC|3P5afx95VJy;T{)OnEx+9B zrpPdrAwaNwYo7rUnh9BzYfzWWFQ!!tzxJK$a&QXdp2@R!zH`AYRjpR@%FewMQbz4T zP!~XrrNpH3Sa~Blx_BGsoD;}5w2R+mT2Boh=X)`N2=8<>7IBL+XDs24xK4Jsz~fgyA6)wAsO}#Q|;kfs6c!> z5(6q8a!ul;-K%DO&8S;#6Dzf^=({9CAB#X~UpXpCx;EuuWLowf>!jZQSp(|)EwV38 zvcxsroRx8D!oG$h3GdvYP(ArW3o)H^t-Qo(cS0d{`(T$$=6ts{4YSsa&Wa}0C+gBi zU!QOIG*d6cgxnSQ`BN#(9dl&&mKjKXgsp;ySES*wvp2? zb{dV8U016ty|VMymz)>4j0>$WCrTEt97Lo-nnIo%UVYB_{( zUEl>;VRB293rJs;Vt{$_^AFATH#65W%B!OvJGsSbJegLH42$awm<-Sllty0xX2DF{ zcVC^&?e`U>5N$$=@`%hg72y0iH%YGq=9^imd~33VE7NxkIC2+>B)p}QFq(s?$)hr- z1^V2s78~SLEjYgJjmX}V_ZxgBgqV=SH{(%ej7MARBO)aaI`~VnMWT(e9fHy5)kU6p z$2wzIynI3R>Jb0*_u7gkCwJD)D*-863|Hu%&~CV&w2pE)MJym17YBHnZ~z?>g*ADh zB|FYozlkxl<~Kp$J;M8nvG>}$NwkFi2E9kBl%Q@Phy~q7BQq(UgL(|lb*idkf1!`IP zJ@`_MAS2>v$j#3F!nY__RVjssD-*UNSOP>(dzk8lf#!zF&Y3x;2BT|RvWOH`w$da_#+ zBwLK98ev=v26El*e1U=o_Uq2XI;@T|taS8n$#>L+$f$kg>B}QdNsC&5qx;zU(iHJ$ zeNwrFr3Q(94$XWrw0@PNUlZOfl6Qtq^VxtMW2lpQqT~z-l~4v#6`!cAu=n@+GrDQ#S0SN?Rjr`+FB!g9`xalB?5jf^@q+lKuT?n(oZVaGZ5nPTVy;GE{=gD7p&~gZeXRR}J7?{}D zH!Bsp^AY>18Dv)i{=s@;Tz2_S8+~PKq?PdSMUkM8HzsHw&zHxebIpp0O_GVR(2)F*yg!Q>6CRBj} z4QV(7Fl5X#=ej>~IK4&;IRQkG?Bm0x$$v}ifMUR~G@h%h3Je${$VtivDFVz@=q_7v zGA%l9EiA4+LXjhw>G=KZ^nuB;dsi}hMdb`%QKk=!CQKSXf)r5%aOFrABB$Kp(*fwp znyH7*LyZg){$ zNq>Q7$Ye4HgKo&%n0eMtmeHz{;1FQA+J?AmLSDZzrnQg|g+`AF3j&3vTc?)m2k4B! zd3~SeJ^ikEjo|$l&Up)-pFv%-{5(9FMU}YQGx@PIC$FbFA_{^yB{{H%aubEE3_3^nFyx#V_dzYE| zx;a`MYv`ms0w+{5-&uN{reQz|i)72sXt!ej^{7Xn;;! z0oChi8yWpQ#sG$^uH6!EuaL(c;YQUf#LeC|o6Fv22hvmM>DlsZlY>p~0Q2-_$C|P~ zfuz;c%gp@=c3}xi4%r1u!7mwZ-kym+S1{`zWa(=NUtezH#4EAZO_)39m)tV|tS}BF z)glvBnL%L%!nsVQK0#a)NDh|{fddgGl&ILTnsfo?M7lghn_1@7GSU}VE$(=`yW1yW z>ssXe58rsNlpWN|GZRO<_|UNSi_H-2y3`BiS%>7O%u@6c^;Iipsk#r{6GG<+#v8KC zOEx2g*D>zG7fld9(1TaInCq?n+AzxkNRzrnN)a>U2?I%Y_OA=oFURf3CFo!6ScUu_Z1}X1|+6)4TSn2YA~eT3D*7 zkw^fHOq@hGE$z^$iz#a-P8=Wm$}t64%V5J8bYJstLjwGAULD;P$3?tj6Ygbm#hQ)* zX=>)ZJ0=4^NpD@N3nj9r9bX%LcqnYhF@^-wKsCUOnvwNZgc{zKfQ=rzcEp7N6Zl*4 z1K(a9CA zd6h*V_a}R0>tyN+33PO6?%Erot5JffweIC3QqX;&mBi{X+BXFt4aPc*ss1o(Xa|rd zx{Z<=T61WO^7jU3$eMQy>vsPVBNUmdtE!=4;;-9(Em2nCM%_;w52L3+DQAcKhFmC4 zaFw{vSeDB%qi=SP2u9;$cHFa~_w=h8yN9I7<%9oX!>Phb{ek%umYKW%d8%=-mAl1w zz<^oavteX#2_nqVNG^D)z}`MYUOt9dN22NGY>7N@-x5V_?x8{ zg)P`D8++h1xQhzotx8-hLnhAbg`1>j-fC4lEsD|zf6$cleutSvYb?q7S$6}p{zjmc zG6R2Fq{+6+kYXVRC`i}H=T6ryt$$gPgbTJb7~U!~Q9qjv1OFaCokVZqLR;YNwjXZ4 zpBvOD`QBeTt#3ZkZe@14TuOWO-nh5UYW1|RzFJD-?Q6Z3p5AslGvgn*xIFf@doR=B zay*}~x$SQqw#jnyYF(ww~;jlF4hfdAf>Fv6<$6N9gLj zU(4CKdRwor+44Hh_l)4RJp#M}c)y=EUTvnkJ6zwI?=-xf&L>(m9J-TvpAPnpfujp3 z#6_OBX9UPaLxS*Rnvb?xm|gh12Y)JGCb&H13=ByNqVh<$!ErPoNp~x@(b#Nl4@MwZ zi&ei1s-P#Uc?ZMEogz9)@Bli!YiFC%tJp-;eZxj&flVjfAL_{>))84Vcz;nNlg zILv%2f#JePUg4-dOGJKFk*wX3C zdqS&4R z!boj_IcU!HUm@I0-Mzn=)?J~OjgnK1HIcUjQjRFRJ7b%#0-iW;Zbn)d2Q53gVlIw` z)tV63)ZohD57mnQCdV+Ky3O|XFciw_M?+)F(p{VaX3FS21I6qo^3kpcMSp%dBb|om zj9WV;LdypkxEw9^x?<`0d{3>pLxFhmxnto;5iY99FCfaw^PZ4HdPv+)`swVXz`+W+ z(%Kju2s)%~?X75M%=O%0b+rtCk_H@Tji0J4x1;-U3b^K(Y({QmUkE<+f-c6egoe^m z499@|mi_g2kVADPUXx$wb)(zrqtgvXXnP$und>rk!d3brTZ<@E(L)^%7uFPh)OuA^ zunijWb~5<*PJz6j29-oHscjDcKAG5kHo8#0BJ zo~drSmpnwKXekBfPqqo*0$8$<&0HX(-lp+mOjhooQ7ml1olCy?B+&+ToAcprpr5tw zn=H7?!fvGq>b5l+U)iJEaf#E^#@8T|b{p#I&vu&EurNZE5x6w*JIHLv8WvcL3%0a! zfH57V)IZ?~tuA3qM(dI~gA z(`2+p{zQJC?mCD&<=GJ_1P_F+oi0jqx}dB{dEDpc*Apv%71 zXe`=LxI=D6h~^Rgo+uuH_(v<^BQVrnDtdG}x3bXXnWYp#k~%E&@^cCjE&UOHSotyZ z4?^(wLhzBoL_DD6j0|^~JVWJ1A{xgJe4a)s&M{gopk>CPcc=R>TRfz{2CqvU3<;~D z#p=II;MG!zp=D^e;G4B-91Oe~ZXWu|f=aGYwJ-6=dWj2`4}-|jvs2POWCTh?4SziZ z2!iRB{7vyvf3MvVd5rW~JqfCphr68hgH;U$IC5v(GarUI6#`KUt6q%{H$U&Y1Vv&_ z1cgB)%wCjovY-Xv)?dyQ=bpmoD@#9CQHC7jkk$J?^;oI4KI*(d_0_Rhi*FLsI!0|| z@EQAUTBWG|7FO973jWF2a3%E{>u2m$#?G1N1&)pWtV9KaE}z}R#|2}3K*fp;}KhH2V zn`jz@hp5OrDY`OT-%2x6`~gDJBtg(A8qA<=+7Ta33Tjw3sE*pPNt_fs>3+5yuO+4YR!kWMdE+My zLEM!23bD=dBsR@CCJpPuw7nB$&w}hejj=nT2zmYI&&r#pVr4~EVX-#aRxSA0WYL#r zz@1>ChRz`tZ-zg?rXOY#@DQ(;Y<6Q)2_%QZJu^?zpDhPyAEf{#OWQrt^f&UfKX% zpsQj}koz~9$`+etb99uLIY*`|Bv)p{_7*i^OGSoIHcb$Lo2nU_1^Jt2SIGizUltjU zTE%6r5XVtbi^}S1#R_dDv&qI1;hhHWiJNNygz|^e1-K?4x;)L)&)8J`b-2El>wUQ2 z2g(228tzmoY%tj%FXBcsNSo|sd>GWw@%$KEr2F&zc-zhMIOwz*JKg1)<7o#9$E(g0 zNap>q3^<6;=`B*}enIxU>(!YK%r?U4P-b!*sj*2@U?xpob3eTthx#=xFzo33uHux+ zYi08d3{gx)%t&njkzq6{QxbgXe#ks}%JYOH5fU-h~KLKAwVaMiT zTQxb>5|Na8q}^nc>o!1iWTCc5TIXeVciflNu#j170-!EUq<#!%&66`&9NzRBwo-Ek zkQ@<)Mu9$HlOUcJy%U}K#RZT;gNaO`hQ3a;K@6WaRd&eLOP()^>taujgx~yZA%peb zX&=(>H<3%!s4S}Np3N_ZR+A!Pq0FW)PxTm$y~WH zsW=&oyQ1p`eY9$oBO|f4v?0D}Qx{oi;#Ex#YKj9^9!b;Rfv`l7#5=9Y;iTt zdx|rXoxpuUTfjd7^>dLw;kfktJKf-;{k3>Uy#yvK@iO!$@XLXZ=nGS4pc0DP(?qOP zWK;?yibk61pCs?T$3Pw|^^RSJc7T3~g;N2AA-;$q7gN zAsQ_-uWd`3)O=<7PZb^JqG`u2-Ot>GBan_HW5vC!xA}+0PJk`c{H%0eHiEM3sk3=K z5yHtl!Yo)zXfD~JDM`73Fc3lvL}k$b80ArNXD!uWlEwRxv~afx14ar4ROU|JEc_7G z7*UdXPc=Lb72=daV-USq(4QNJoNonk7>PIw~Qq5JUsUwNjmbX>Ow_& zl>P8>cbdVOv`n!|Z_wbkp&QHS{zo*^bqU3cXy2$a-vfstuvL-yKBE1JSaM57iqa%f zBgVc`(Dt>|3>MeouT^PAbK@(_?icR01-J`rd_k+4*=+DLSV(U0S=&)Y8HRpA;wnZ& z`U=J{`W(?h%q{Ge!Uh2cb|zk#OxuN3#{Hqu1gA4a@@dg*Oj1SI`}DNKpXV`?gLQQ6@24{OZtnMm%VA#2ewswDnf81v zSM0->1l4V-BLf4N%<+dB)lXOLrJEM@jGB1^i8iuIm=DX&@R7kF0Q58sm&cuw3aj{uCmK|< z42w34sQW`x;b?L5W|o#b)+vsdGVzULL+FG_dXk-LNv+54lrBo|%$3T}2V1agJBA@X zD}QQkPv6m70$Pu>EbG~r?f*&kIB8s(uVl^9e@N=fSxe4C{mW7s)BFJTgPUiZhLfA$ zKZeul<_HVT37zKH7Dpc2Q&7;IaC8MpP4fCNl2Jw*! z6|x6iVY0z$naHx~alQzHHJqVt!%N$-BJ)r+jMG9DkYMGT*Uo_)6pHC<{n=E3AfxAz z<{$%yf>SPUV5iE&xQl%}6mKa-tu;2ttB9=STOQM+E+xH?jPQ5=o_QOqrjP9}_()Kd zlem997g(68w>#VEai~=+4w9x^@x}RowV;gc_F$CSH(2g=DGYBaIKa7i^U&)%#?F zRSa{80`tHu0pPMm#tE04W>kxlMr!6wW5HK32_8n__vkoL1!zH%BPrji8HR|IWKA&u zOb~V+&=;YVISB45lCdYV)Sg^+mSf{t9tAIn2#@otf-b}TKy@&98!R^J{fE>|I0lv^ z9cP?`=9};?ym2feJYCg~qZB?K1S1l2dOVi#j2R=a22kM6$@f)3(V_JLGdUVkdM$_@gXj`+!cErYyXcGwA~I<6ICAmkL--Pj5@3ZNrs{v4KJa&oBt-n z7SpeqWQH9tLp*AzqvPvePgsh({H#~$SWhbjn=D`O!UR2|@a{8?@J=hE1iQ~7bUfI) z69^uhx<1z0T+~x-x}*?v zh+sQVX_O5lBmkUCK!9@*%98_jHSM}|>)W4IB{14L{xK4%OCX1z7f^Ke^J9K`+Xnug zgMSk9MEyE5+_hNo@3Oo2-YAiVO-gzt%KV0T0n{Uyl>+4eoscKtNK{47>x_U1*?sNH zn8HoET1R1MaGI7rCeI*Xfu0IUJ6i4-37Hflao&yZu9j-dA=sDIl@F4IdzeQ&+7>(- zFoVr<<)g;K49dcaA#_KbHekD4vl)gm;1-Wc02E?k$Z+zrlb_O64t*B4tK4F5aIr1m z0Hgpj3L>;UO5ha9RarZ4l?rd+IqyzIT!;_|3f)KWd@(U(VR?CWIeP^mnZCrO83 zEG|a2W~{}Bb-7JMx86rdBh)4N@deZ?yl`!#o>B+dIZLYWiQcp!IOTaAvY`{%DI0pd3LKXqVWr-(L-%I7F z4eXpJ3Qq=dA}ua$3ppb02tWowvC2y^4k%9fIFBRtK$l>7ybg$}QuV@iZl+Ylc2U`G zD!-p#T%9fWj&iNRMpl*W^2Ianu-#LA%>3P^_t-XgNhyE=RXsR)l<_p1ndlQ|1Pj(} z{puZ?GZu%=#|5vBp==&CLx<1&2kF zU}l1R<;nf)NV?QThKP~Xh5u*N7qvF9H`@x*O$Xnz(+5=5`&acC#zgX0{>5IWS382p z!HSo`0`mv~1>{E;WdW{lC*7m_OK1W9?N-G(ZHldyj)UtU>gF&EmU&nA_v)ny-PY$z z(pAQ_i}v@!|IUanR%~-Sp+UD~zLGTHkd+dPqss6rXmn}OmJN6jj`ZkJA@o7u%xDgu z#{@f9TnC)`q@qb!DeCxp%~P3Ww-t!RaHPRhOXusCTXC1(LNGhHsS0pl&V~9{=QT<- z{qFb9)jVoCc_@XiyMb|-S+O!)+8G$^__c%(^j-g@XMUK9$kUDV8=fnGQe1)B6GJWr zp7>>A26eNhDb?;4i-O^^?W=Dx$3+ZFdigQgA` zg2<6ofY7m~Akmw)2BCt6AJ~1XW}9fWPX{N}{k6(?EdO5s>_8L0snC+Ce}_jX9tgA= z6a$Ufk4nN<-ZZe^h!;ziC^HG}-t@wlT#=B9NBt9dQO`J{#Gz>;yo2T6TxdEsAKB{_ zL>rWXDFk1{qs==)U@&_1m19I0S!(QrhFA_YinO16ed;$SP8=TFeQ;>>F4 z`ST}FetqiodnBt3<7hdw0*8)A{S%H04I}$AL)0)r%W3$~--b?1VtOsNtHG@Ty8HS`e)o%S=!SYa>p#!{=S&Z*T`XB4ugM11TO z+ld1q69-VeaBWz9(%iJs_%_|nqz#}%!T4?;MAvmWZR0ePX9gJ(q7Z%>)l+WCkR@!7 znHQEU5<5Mv#C=81!h{VcsuwKI2u9&><>&?dZMEFt%_WUT9L%u?Wy|#fE*dBMhf35a z^$l9W<%ZD)v5UOlqi*PpNg>|{6i`Z0W9td7-+B4XyDwhtx-_+K=6`hugx)pM1X ztBBU9&DSn>*PN=ohTNKnx*@e!p$VTZhEt!x%EI;d>T~dNNrobySvb>Mdr<;-PQz_; z@b?OGjfNK9K7(a4!aEgPD}3sj`koqBl4|oh7-plIw5Vy9gyl#^&Q(HJn^<^b7Dd`5 z*k^YjGvNgW65K&yA4Bw6gj zI~XlDQz$dbuJgk!`GOms;Ghc_qLI|C7oS>Yc`LdW&rmnp$*Qy{22F{@n;i_HvRdB2 zWw>@tCbC?q&3A*o+_aVaN9_a2Lj6hY`oc>x2?vR(N$vH8mk`Xj@WjFsa!(0gv3|_v znYBv0!bXeZ!>YF=lqMd>{w`vxI%tM_~U}o@Pn!eU`^vxU?d1SW`NVa8=A;V3Fmc}k>R|sZD zcwHv3cx(41oL^Hcn9toTmM}Qy#rlO9fH%8OAVbu4(07ly1A8nix)ELpbF3LJ#956b zS4qH5t|^F=67}IAXwdm_<99b7zwyzH-;o>d+&p{Z^3BsX-o0^w+&KThi4$=2Z;hmdjZa#72omu$gPd6WXD5^(p zoWD7D@+A4cOu;+NycJ~Z+&ok7t;YCSI@90iWjMb1^P5lJe60TVG3cE?Ku^GDeGQ?9 z9%ayp?|{S}^G1t2JY0HUlPjhwYO*k@r!Y*|BS%I}r~Z}GENeR!KbD>4OQ$g3Da)~LnS=FRABEqy zS*J%?c)H-_=8$>|VD~7CI#r`Fh3Q^uDi!I4KjCwLhdE{rf~M+4mTGae;r3lT3OKT` zmgjo#r2}S$Gq}+vB?}bjpJDCW>;u&c1}kp@*l8O%Yha$6n5AZBDR0BD9L5qb%q)BJ zof>`TM#}bxswDS_eFN!5W&ggg%nrNu zV<{!!<8#_d5-8I)k~cT5|8*7me+zpm;nREt>pSKv9i0byOi3i0`jCJ4pYGLatb)uI*mjQvagW6t3tc>*^`au#w&p9v6$X5}j~3xN*QR8LoIMZ9qV8+d4jqDH zEX}r`WtE+zv;=jP$t=<{iZ|J?^td)&Zei`VZUds-%VMJ%n7xfSmOKb3YA}{kz9}3u za$P5$6(r9MvHA0*9KMb`ugya9LU$Q-0)Xs!AW>?(WZYGu5uc=Bb-b`~5 zJevIuhE6xVR8~$B8T@`99{jPJKfigF+&nGAzjN^S$?y{;m(7x?oJZ|z4aHakiJ@Y8 zvlee&3-1NUU5!M3EFkw~9_0e^D4T^MHA+(k%dx&;<~nvJkHNgY#hwVCD$)#XZ)avD zl!R}GD+P_;!w&Z-%V-v=8hrbZQc#u8v)WC{e!p<~6$@EU`#99UCu zep1+vJ>*uN(oz8{OQA&LO2U_|8)kyl3yWn|OM&0-cUJ7DhGcE){5G>68p#YKtAHVR zh}t$ajPRv2QyQ3}suGVXs<&XV+S+}$rZAF9EVYHhLN7t#v3qAL9|FB5?VTYn2~1IAhp&RHj2fR z_!g|!?OCBL$q>#3>SCQ-qY}t$xlKp2K2V6r3R9G9ZW9A3iZSP$w^2)bBPeN9%r}hq z5O1N<5#B!Y?c1k2z3YS*!>!Y2Z=HVQ*6BBIo&LqGxo2;kdF$4h`CD_pyES*|*4%%D z7RT+kUb*x91^livFWj1Y^48oRZk_og zHyd9^Zvy^M(8oGnNDt&);Pl5_Ja4(HLl3G})$?M}mgZk9y5^wLl|-~{T7bGEKPxk7 zPn;Nn^TM=AW-)XG$Aaa~+ zr0eD0?CeV+de17d6trJ#t4Y#}T@YTv&@C%;Hjj8?!HT({coRx0+BR-#kb-eWT~fV} zrpV>8lW=niI_)?q87?BJsuy4rV1&W;!ZAe*rf>%9`DSpKZ~oBd$}OP!RL;QG91xDh zdWYeixD&AGNiNz#+d7}GtH)q{&RrRd4^)SiQ7 zDQ7WRwrv#v+@{X^*#$Gqg`Oe<>2{bsQc-lD#W3RKSwq9tkk&P%g4UzWuA9I?v%{5) zn4(}~vuq`Ej1l+qYZa#;rcGWzm20BJGMpOyLjpv=)2^IXeHd>--(NJQXxi+GqsF#e z_u#L*G+VI?u6>M3H}?SS)fPEGTvX3-ujD z4+lSH7w|lis)V0+)S$;b>vOyzs(5fw!A2Qvo_T?FAZ)b5Oo{|VV-$YRVT6`hX32o5 zUXT_HtfhE3Xo+cf^bH1y-IrBTO+_Mvy2nZNDTg|C?;~a#Vffv{BJPbyX{{@IYpYlE z27#%nqWT>2VG?NTfN_~lx!Glao9xQSd;fmdq6Q~>EpF#wVhvg}UqqOHctGY_-_&lF zkcRoqnlCvr@O(NT+%gggBO?bR{Yam&LyJaI3H?E^5W!TMVp*iSh4Yx_N?a3W8nr?D zwnX*)EJNlEsg)l&GQ?nNx)!ME4|p!HXERIZOBOdXyaclr?^juOT&OpUN@nG}ZrP~NLurDW*)+|nsif)&BWSwrNs!kArT}PlI_Cb8^5hssQFvQZN$g{naThJH-^CH_wci&@CR%H8F zE(iORXBpO8yN2{NTtgZrL94O0>&kCytPQ`iD2W_aax~HcR3b1}J5_{0ljPC9M1y;f zxU0ouKbE}RMsceBa6H4!zUpgfW>Hz93fj)hz5y5h<{38%u9@A4vQ%1p6~!sOrsjlc z+3TvOvzr@5d&%EdyIS@l3YG z6aC}Dh9^NT4vSU#hIZ`_Et?hJur4)MODL&Sb8LKL;mWUE3Bo{`VjK@bqgx{OW9I@opS&83deC5_&TmNnpBPtfTN|ogSCeuGQviSFj+#!Y?HG z?=>UqN)=2mH7?4V-J{cMj~UuD7Hya$h-`B!X~Pk z)Dw}Y9cZ1Xda91@m+a%hxEub=2YK+8N= zAD#x27Q;cy!CI~cBV`!G`y@rDL$l$Nfk)^xXPLamIT1G1R6}HfmT1<-X7T}rb?2pv zcYgc&-B;i1F8|P-59jWD^y+t~U%dO|r{Dhmv)hk7O?KRU^_{yfzI^AU_u=UF$G-_* z?Q7Di6f0>ErMK%A1(WiiMp8jxBN`$VR9{`?ZFfXScLYWH2-q`7Nf?!kL9^k}jvM5O zh^CZSvN@g2E{A|Ta3hqmPT2XsGB)V*LZDeBw8TFKuyj?{0M_QWlVVNDa$k-7R-afvt2pWA)FC%REHyGg_b zy%Zx4gLPUixfXfEl;{HJ1X8ku!)&L8%zE|?^sJOw&t8h0?zDhyTCNzVDQZH6E>~2| zlUdk?ey*pKxDO8-1~_e^~MOeo!P4^Nnac+uxRTW-KKXvtpONR6M6*6d?$qKFMjaDnTs7Xrlatwl4u_bD*1~+mmV+-ascZqk{x>m zHRywg{*>>?4*XtQn3C7nT*~vd>uAtcD(LQUp9+fQ3Nq{-=%4}okPSRX(~4Qgc#Odr zn^fN{igPr z%%R`xvtWWwsxw{Z5{{2xp`F3`=MA^Crm2b= zjVRHm67^8SXsHHpPreT*+_;iRZN1EqAy}SSv9r@Esc1f3CLdInDYD!#maEKEFW9WG zlbCGtyR@2fcXAbq9B(t#b2$Q=P06YL*(z5rn3XnAan#CjNB7{3W1h^>SkTPTC>B(R zM4L^3FyzYngeavc?YO!hzNP{acv<9OYPM7OcQ13NsRQPyC&@6`h(0F+hMz$Inxd!f zJ#*nC%j8WY`Kff8x^OS-hqT}*T)=N_GaY(G(}uC>r#eiv);3dXkHVM|_0ft1&hCe0 z;Ja6^e*5-Iw}1Y`|GD<$-LvQJ{`rsfGymt>Q=MKJ+DQ9<{g=1jx^VZi7sCq#Yw3}4 zLmZ7@Uf@VxG_L5reKIh8Qc097I!$HNyn1Q6%X)r4W=_rs+kqC03BxXMGEE+Kv&gd^ zNre{2J~5pY(_SZ8_&o3^qAUDaC|dyfa`)56zWw8;-@Wm4c%^X-h)cp#I5ZKoa%dE* zKqP&YL$_?s@&3^RqdyHTmJLZQj$-nCJl-~L&(D)Fk;zY^1KBFoE5PB>)m73MtX{J$ z9v%v@*UF|>P+O?0kSDIeo*0$c_*%0HzjbE@&5(l=9h({nCAz&zt&{~xc1#+uai;>O zk2XugE-}NX)8#GEHs7D#->5D8VPQ)0Nk(>1Fg0u^d^BT7ASIJ`iE7Q3wFX&Mv|m@1 zln;s$hK+fEL-Sh;a8Dj%xTA1opG;i&&%3YIt-XW4>D zOQrB({X>&`$==a{;pz(>Q8i3?EpWD=t}9xsSsnQCJGl3A2d0t0VvJ@d(UDQCD-eq| zlOz531%RMOp2>?+zG4_16hN#$5>X-^Sw=3Y(jV?A#ZUz5>G`v(TIj26AS1Hk6{YC zpW8T8vde3d>gnPV2NY&|d|bP@@WjIN3unl}FEQI?;mO5Zkj1>07i8wg^@!o zpbuVx(NJggLb=yvevwF)uxm1HXLHNB_8SZLwZ8C zW@$i#xvhTylu?tK;UmXL?*`4o4c;d;*)0?86`usObyQJ-U36A8Tf=|M2kd=-J{Kk7GoAdA~Q1O zWH%xv>xtz{K)^^!(XhBUs50IPFlwr`dn&c20Fz2Q(RM8!hZRc(br{)A7|GJQEsj;A zI&1!gFIk$-B0PZR$>=Bn3GU&-%3>L;@T({{40F06m1tvHMFY<&RaGK-+b-Nif+1$Y zINa-+wzH8TzWd@DnW$dNIvnnABlCjyPS8xg$h;F{6)e#jP$G~)RMC7gXe;Kr!y9pZ zunXg@Pk!|6sZ)1e{^k1Pt)by{F(llN_>hxZ)@OTgqEFiZf31fQGO!$yN<>>v;e%}= z2Cqz3A!2C9pvLrbU(RG;$&9lysAFJ2s}iz<{9&sVZ_^Nwf;{aS=pWJ9m50p)tW6i= zIhlg=h-nMg%D@k1mw1?j)rZs%nCYGcWkSI(0xN&=@a*K_jpT2=Kf`5EISBvjCxfnq z6)0ig9CY%`niiIkH7ev_eccgl5UUJP%jCk4=&_}}ZI$fxwUXCkn7kg3we17jCMT*V zPvg7?rJfl!r=&UW(Y00Sy-R5{j-PQ3Ro?X_&Vdl zv|~0jiOSFZO-n=*u_SzAyZ5V(*al6i>QfoGB7Calt|E2qm4!&u)^$kKHOi30kXc{% zfnx&~udi3NU}hN>=@P}060!6E%{Z&nsol&Q*&$fg_8Hu5qsaFt*q$XYK&$5wu(mG! zKMJ<6+ANVUw(w}Zqy{``?=H)x&P*8@A@i0DS%Z;p5|(Brf$@`LY3g@aCD} z*d+^#Ad-Dcn0WhmS@bCqD z=hCGI_RKDk#T?ipIdrsWOpj@a2gwdN*<}R=CTWF?h`gAZ3a@}J_kw4gO=>MCby8b% zpu`m2$EUchfrgXh5Vvhm7)($Xgxw)FbP#1iiTg?qY;*3cgLCJTQ{TUT?T53klO0PP zaQ01TRVN8;W0+>2YCmj0&VJ@HTdwVPiLq>D8A7yZ><>*0|MMO-It#A^US*m0E^`C4 zD2Tfce{px84ZB!c91ktO1F$`TO(q$&)JP<C~90y#8X-~8Yx40 z3hD`SH^fRww4RHJH3b<}G@qd##ur;BoTzl4O)_faW%c>cT@OPMT1@td67#sv5)qFl zJQCO!9@CYG&wsXMV-gbv%@mMXe**#&Z6uO|IO0mu$F+VegwcL$?oYSQymxEv_qXOQ z+?u;~=fzLIJ9YZbug`pQ^>?cTEgl~81uY(?C2DSpfW__C&wMlg^V^^O;@0W+?mYS2 zcfb1h*4%Gy&0YGZe+n%jO@lo{Q}mePNgrz@nFr)`01<&@rmx3}Uo0S2FI|h4`{D`M zkX+J>ASDBAMzWV$&WzanxXLy0$b0q;>?fcZO$yf{ytv%BA=p`+eBZDrxY#l6r2`J2 z+j<%lG0!rndAK?Pi&WRZ%HAvNx6F6h7KuINSP5t z$+`vy-LmU^RlAu1Z72Jw-9~Wc&@r0HdLrj~Be`D>$W^2xZ+^@N#c302Hv}ukGR|(A zJ9+ZJi4&AtrjwmDr78N*ZUidiXPrukJyh>G0vBW~I_1nx;>oe5lWIzv9h@D1=){Q= zjCxOyrcNNx-pJRdUVxYIWJ!|~Q0$)7pOd}Md1`NgMh%Sg~TThXEloxwtyjaYZRWalBW{tJVC~miwA2#6jC%I&>scwIp1sVX zhPP6kswyc>QB`#Mu0W`@Ho4Z6R6a(U#);#SOQ{{&W)9LQR{`X+uf$n5HeNTf(PirAF0}A?_0n>S9u_dc2WbhphT0*|w-_vRMrp>g_Ik>O2BZqq5w4Q*0g3XJ z;9AK4bLq=Je|hf9i}3HWFX#XB%$FZ^gKhBTFTecbf1dsFk6&K-^7mi91+V8CqVj*% z|K*EcehLTY{`1tA@5@GB`0}!R{WJXU_y2h+wERYBrJzuYrr{^Op51t)TMIR`ap~AYdXi1N~VqM zy%IHo7bvSq%4q(JM53djy@Y^sk&(%Nr z3>?7K;mr*x7PyZ&W>AZFj$Fa{ZAG0{(#cjYX)ktlbFr(JpZ@ldcY7MIif5IK+L}fQ zQ*M0Y`Om(7``sJAedz1op6$!};*T#}KljV8e)r*xCmy{1+QrbqNP+>A$tW89O8V`W zcvBkQ>tLH@LQ_(0OpIYI7=oWI?0nGb7DI;HMN!F1UMPS$qf>*G!&@$6lJdf`P&L)B z@j5qkYTGn*f+RYHAa-qA#N_PY(H@b2B$4Sr-D92D5C!EWm+WsFMEJZ%7ofC~NPLiK zYlhe*VF*TvPA#WMrKl8nghsB8p@>K|OUam$ZtG~>MPfii@C8%Ip2|Q|(rvlFz|`5E zr;DItE|pwh)s3wSI5{+7nJ{$wj1A5$Jyp-@nc(uX`uP(lNEIH#a-PGvGCJ(6d$sMi zL!nJ!C{OCt)6WL8S$Ob|jmh%w`kgoZZ3UPBBkrcMd6)nC;a|V_{~rA7rN5s0>+i{5 z-}~$3zrOL;&*9(?duhFYEXFO;dfDc-lnuNuC6gU$a>F<_=^L*+t|bSx^o5XSUz5e9uU!Hy9%ZDES^6Z7L-Z}f#YkiTdz5e+5ubzGT>jxk8IX&yAhT)#oST-FS z`upW4Z@hf@%dK|;1Pe0wYMMnbm?BTz@aS9fyfEq8 zk9?$A$)K!RDWvj9WN!vGf8rFVa0H!4-euH(k`3yOvSqtI@CxYoQK(jdmjyL=)&LF%qJd?b!KdW^V^C!rNlCOj411j zC>3Vq1b8A5${M?n%}HU0QYzjWg$cas4saYe_FHKr5AGDG+y&+le(ksKs_W$A$A_6U zd~9}jZsNe9si`xP;fIiKW9MUHwzwp`ESzGMj8oHz#AM0z5~!vHddlaL_)^v0>wr?r zDrv3NDG6*weHV7%oo+yFp8EY7Xq9yGE;1(yOfE1392&L258Po0E(V5xoMmjH z!$6g8!V{!p-7sbp2{UbCQ1CURP2w?ltl-`KiBv|tOWEP#nl;f+GEJY0wQ*{Ajyq)* zj~P+et;WFP;?^d?g{FQ;c`Z%ZR2T+JU4oY`T+unE6FxbZ5Ifjp}ooic6MF# zu;wm>-7ndA0ezy8#?Gk}c;z!=E zpTZ-!06z0b?Q-qY)t}U^)ZRmn5&r<3`@D7m{?L2)?&?$h75b~a45NPt?)_)0KZdb= zOln`iP(H7H+Bp3no(%mU)ZT`fzT(aHIhevLIE_9NxoIE=ejt*&Qbn$BR27ptRh_zB zgJ+1gPo+UwwwlOW=9UJ=D3(Mx&dToxT2^OH6bj67IH4G6t|R<(5X-=!`GFR(3>yM#u1gK0+>2g15sd*$;CS?@&TDFmiBUvuFlprC8*a?uBwk3X;QC;b2m&A+2m& zQIQWNky122enXcA=#@fZm3Vxw=WfIQw+yjl6HQUW*L!rO^)5A>RFzn^wK^(r zgFh^twdb+3A3OcAsr|LJXB)lk*!KSNdUyZZYfrDe0xw@&Lm*bN_Qu+|wYTun7uJ3O zmv^t_|5h0$8%PFkzzx6T-SB$*a9l}b11rf65*{&e$*tEJc;IKWQ0QK^;Px1k6DC`RdFG^o$m0^MWK<%vm?3cl z8E0joqYBZ5cOGUjMfT^gaFHCfIQ%sBH4szZa&DTEe9R!qS61Ke7 za*E)99B;uAd)+Dmtak%YK zJNF;wC;sFYQQL6GWc?{3reMpBZBfyX&bcPzsC>l6STKZ=4d1+)- zNkvEh0;#CBgDmSQr%15A!If1qItELQIztq0bAG>q`dQK0#|l`BglLr{BtQQ)MEmqB@N^`X_DVEUz= zWVlrO1W(?gmC!XVr`oHv_u=**u6`D%uAN)`8K#>E99>*}c=aKI z9x5dKWxe{-BhFLz-m-YD2cW%pshprM-6VVP0{;t<~X!qa!_vz zFt4XN#ie$1+&Fc1^JY)wl4vcCs}$N6UPf0-jqP-tbQ8V zeo}4Ou)r3@@#D3h*M2<=`|st__XJfoq`vB~WPhd^uDh!mj zIXot?5jF36VZtE|^tOtzvbd5+o!NARJ#+{k$K?Ox1xz*I>B#DX zt7l=ezf9&U)x||#DphaK%IsPp$g^OXTqLzuyrP-UYafMvEDlqHE=p}vNO&;WIq;a= zHu9)}OW$T`?)87&QT<<&E}03xw|A@UD~3;2vM$vuK?5q7(8 zq&)_TToPnt0cE=*VsAOOLP-u(Vu^OBEgHCQrqvKWyhQJFO?PmHf?90az-lb1sF?&9 z!e|Wp!f~q4&kyY8Ybhn2XbtxS?!fh?2y8F8a=wCX(=M9(!p)o5L2YnZ#|t7@K@IEZ zpyasl$sAo4ZdokaCEjOi=^zZWAUN`)vGrjr*`@Gs-#@s269_@*w!AbpA;sGY^>%F- zbqt1eB-g8$X*{OHW3A@Z?v;h4RdS?$>#%?3syLV~h$b{DL>haIVr?W5fwkK#lB|1r zj2BkUvvoB#Sv6pCsy4f2q2;ax)LJZo#*MF>kCGYXeud?$KL}oZ5XU4^bX35=gi2~W zv5N?m=9pe|?4I%|foM}od>8gNjnN7>O<}T=3{0P1M$qDEC7FqA$BkHUritHH7RE*e zmrYbPlT62wsZ5>IIagUZ-@(Bju}mg~d+N;ki>6y&uY=mS!02^JR{${|b-0NREOY%J zRn3zdqZt~7pZE+!K-Oq{#Z8%kaZK0l#)T&h*pjRWG{0S2a*3Fdh#ovCGYQs(;Z{ss zXddEJ_t+0^k4Jl$mkE}g+GM|XgH@u#6I}`)X`ZH*Azj;7cQvRb;vE+QoZnU^ni5M# z2H)EYX!V1CF-sjX=}Oz_VNfdprfgevgdUVJWkxKnoFg_X=?vjNrajzka*T0%i+b9f zr3LC-(u~+-eYmzG+@^p!zdt;VofhF0_M9kFYnd)D3R&`#vujCzT}v`>#4C|hGMTQ% zy!$Kv;30u{&2;rLd>Dvp8B7kUX<`dqM|+J$pa(x%rm|2pT+81NGMMhD#namg2u#`7 z?lWn;#5^}Bp)hQdS#}Q@!qeE%7|TS42!T~{3@iZ**TWv=I<{V9#PeKj<}7=D>o6%g zPQ@M`8Zte6d}b{O%di!j>8@l7?4&wV;2Vb)nZ({t{ER^znx{6~`6_a=XzvlicF288 zaD+V&p~ZTTnQp1X!CqVvLK@7<3sz!QnU?&8e(_}1SF0Wn5GjoWZ0&f(yEb@v4_&0a z3nV8sC6jIKg4;S*HcG8RL&^D8-VX++w?f4jI5^stD8n{DW{|_M!VFQ~5vj6~iqNT9 zt$+;yo<*7KcBhu_nen!inb$H!#>JA=%$H@rLr%n%w312sq!SjxCyg9ysm#p~4*&h~ zyEk5V^TtaL{r%J5^n;nVSteh<_t94$pS}L*!+-xGd~@QL4?X$i+24G5_DP>*pe8%l z5hb@3LQ2WD;g#q?x24)Iv3#WVDVB@ASG(~4K3MyFnEdH?$B)As@2x&myHvYSM=}|% z{d)Cm?UUN)wGR+mgLnuf@UePrEuQ!c4jv_lTeA8ne%%#>@j&Pj_=@*dA3t&8PoMTO zIS|15XdQD?)a|qJ3>)b#(Io1d5?V~1wOC*~%m zrbfuYLvvF(GBR^$dSw34kr^^OcjTUlvH1=PSBmc?yy7>t){{dK?P@(u~w9D?^Ud`|DAn+Ffp z=Tpa-5Bb`NJWt=Kbn{`@!Hp#?DF^q><1^Z5EESwP>9I9i& z+UJm*BPwJ+4ljJRW{&H{qOkO$FV3emyrv^BW}x zEr%SeZI2=P@g^@*2D*Q&?{pT1bbbv8lo^w8NW0(zH|?WM^QN=84Dt&rjc_ z#Qe`sU*Kdcr;>jgIL!a^(}&B%k(3np$~k!PpPxQStisA`RpJ!E7AySQ0QoM_SI+61 zKCKs#5VQnW==~+qDs_qiA-4(jYLTr7FsuhBM+T3LkK}xoT6kE&3^J3Yv%Aky7{fTE zFnVgFsmQ(>xmtIFdunWIVq|7w?#!{-F_Te?N!XhfRVAJxgX91|1v}WrPSs3LBVQBQ zIL5M^F=S^2J;|}fQpYkLm|KB>U^+|-eAh-wn@ISS<2sv9dQf8+@HO|jrh~6-R`2y3 zBs0RH>z|@KorJE1#LQlzrU6qWonrWah2x0Lg>-6qi%z-p_K!+hiN_)n%jx)B5VvwS zsarIb7|csXk+6$+qb5|>g*5Qk5gxb5uM#`GmIg)6cvFS=NmwV9(v&xKGmIuVR)ET9V$t#C>aPL zJnK~_mr;$UlD>wuRKR$C#%o!NM`rNZ^GoXCGn-2ANs2JvQaJcUteGQ#oEhj&#nD>$FH&$(ATHyz|v2{y`T3mrC*DHrwku@eeG8S=HR)tXV#u+ ziXwD5c?LV#(rw7EpNd?cAT&)$Cien0H`&^2 z3PhZvX7v;?VGrlJYgZzq6uP7l;|Y*ym zZ4JN$Hb~_nqG@}TxjTE4o8M$b;?XdMpQacsg>Tf} zjcJrw1#m}7RmTaRI5+@aFL=ISuQP5fH)lQ3+?f)A!=6aE))sLNO!*Av#8Hck9vUO} zyNr~nrF-nbbyv)xWOQz7$FGHZ`TZ`h)MG4+j5m6NRxUK9wbd2F)53CbCc>|odnJcM zQSs&HlD-68GO!1oP?bzuZAV~*HR44e|7~Euq9}KHp~mCK4|7?J7Y6!HcXO5nQ-w>Vh;F8KW{UEGm62 zdfW$o9$42V;!3uyPBw7K7OZ)XyF_89k!!m+zx~uOJ z%{j`kKfs$#k7o_WZ|+<9@gf>;^F;yo2Lk3M7&Znuuuf?(X1e*%;+`-`1eiiW@(QSa8$PjQmq6Y<3o#|Zx=8Kf=IOkoHb*Vf%X086 z60OD6z>&T^(9`Ikr;((iDamwX6Qe97Ls~-}%LB+}aF`vxWv$n@`)W1TE8gde418~F zXK@_liau#X@R6i0bXs9>L%xlvX@dkM1kECaS}YscKKE7++(dzvJ=$xpEZp)qUJvk7 zJ6=jH;h=pCcT$rg>>48)86<~H#2G3uQ$z{yS_Qw|FqrM^#%8y1W3$-sl4t{Syz@7j zFO)v7c*z6q|7`V{)w4t;wJWR7*FLOWseQis zDx7?B^-;2V7Oujs=O?lI83Fh&)_bSl-?z>w1m?Rj{QXoS6<1R+WCQrjJLeTFZ`2^; zD+KR2v;LybAg5|i*FIf+3I==@hW#{hX7wE2%K zsWy9o1w~|QP#X&`XaT2P3oK}z0L(x$zpq^A$6SMu!g9hJ24K9Ege?~QFwdImizx)` z9Lr+2OESJk@wriI$*JYRyxw*M^elFhB(&B7^uWwNwVyW-Pd-4lA5y3MX##`PE))2F z^>H)^z(Dxi+xx+m@Y)d2j`;EFPgZ}{_kfeQ`+MfjS+B3)qDR~Ffd>$C8LR=%i24{u zPxjY-?HN^9Hr|9!d2fervz=#UF2&*|&6sUgOlJeuOJ|W>O-lz(;%^a)n4krdvFctd zVw+gRV7xO2Q+yi^rYv%EW!rcN!{AY3$%rUOS}FD+Kia5USgw+3YDUR=%y4A0=HAn~ zQRw2Ua|yaaw78;W19Kkl!LGF~l)$iSgW1lJhHZtDP~vS4??O<+dfHixcaq2sCu*N$ zvgCDP?q77_14xmbO%rB{lDkO5?2FYiAvKZ7l=V`U=g^}qX zjOfWby}0rT=I7qki@P+DH%k|hcA3~Z&C7_axLrx)Wx;4rVr_6XyVi#qXLiXEmC@XI zXnAzSjf#VXkZR56?ILN6S3UD=T^evSoQKt+_edJBoy8)v3449HBNMqZe|zld6DP(b zqiBfX(CVZN`%a(6Fm=OcB?p8oGfC=l;LJWVIX5xrP`l(oRDZ9^><1p$cZ;mh#PaKTsmss@!e?)uon>rSBB3%MGKexjXY3(kz( zcjCk_8RyuP!a?&bxzb?7VlJUp0dBkiit_Yn+s)g~@FX=&@xTMUbj1QJvQcKa&N4WZ zb(S={UNqY5+-`kyI7e-max!WPVb~5LD~4Z>v$R50EuQf0YO%C$U-J%vJqSrv$+ks8 z!qA|6`OxcMKK$gD51;$;>{|!LKf(Y1=gQ;%^`mEQy!^_I7vB6oSAN`oNvvrXoGCJK zsv?;kUO0!9c~KRHDEmrgn|YFeJWg5Ur-`)%g9B&!K`^y*wJ&OySI^cy$3nkPyc9kr z@;~xOKQNiycY>LswnJukq3B>&_ymmt>^UdOdXbl$B1oYu{HaEtWvB%=4U?%3`7Fx} z6TCW@J2S(XyTmYR#VtxGYQr+6CFjHm0`r40tR?!u1B86n0u^13M?;OUgt$yhjndzYdDiov`jsR zpAElUP3GDq-&$2;ejvM8P#`;O*HO}Hm@>EUfXz3-!A>#V%tU=Xs%Bs^_H83jzd z3fSg(Fn@3|L(6RvXW1s2-^$7Oa6>ZzT96x}i!1T$9wp+2bu-1gQ@T)S<}uE~PyGQU&n$zNZgRU(JAtW&0l%0;wtLADvaauDA7gM*Al=7ELM8hTND4bi z_;SO81?mUR4?{iF5CZJ^@m3#s*BW56W1a!l3tH-iIMRyR1_-m)L=vuv1{|otHjzgVPJ~__AoV; zjSrPAN$sI@Qb{OUGS+AMaK0$YVD!K*iPR1{M=$|Iec+Z2&jk(@aY@|;synV^G6I?2 z$t7-rb`0hglnXT%UFxg4wTCArED5Ip>8V2tC=^v>$RlQd=D)pI<$Kb)= zeKS96GuIF{?di4~*A%U-zr7gLHCg%PNg_;xns*b2N{5)!dpWXLS zLn?{~U$iMD+qP)$e1$nCs}%=Kg-Y+(h#T{tQDSX$C}Dh+1IVgl6SyzVw(wbYvRjW6 zLT2WsU;xyPB8hv{Y}V(oL~_~VStZ-%Fy8r#U>2TzSOPslXRS{?(wbGwWVfVYt!gdoW8Jzr$AVNB#AXm;U)L~Kj zY1teyi!{$7RMgzdb$Ck2l&IIPuGcumnsvzD>wv;bD5-Sdu%rjpQ4*zr776CCC~&KA z%lu?pxMjXb`Hn${I85zgfp!SE45+czdVuEm*P||J6_FrdUg4h~fcil0$QK*LBc8ab9 zog<{>-}E~pa;aNDay5jRRMhAU?0viGZ)6)6OsebfqK7-b$J-JakOp0NYBNF!GdWV^ zF2nsY0(+8IKi_u_{Q=j>3v-`UJxLCmZk~)+;aWisS6BYHh{fcI!AKS5BPKg#utn}G zW(tWf*br7$iS1@m7VwFYe}VUGy{JgY(hiQkrOh@l!_X{ibsAS5Bqk-!%6W6;{36Hh z%q<+q19NQ-WWAQ|7Pyb0#WD>atdeopvX?~1hPZ)brqzp_;VW-1dLTgSb>y0&VsCaD zJjsNwj@-AOr~zx2)y{k@ft491z$-!uVt_>kKgb|K0#$4S;-tc<#gG8xlmphNKvGcz z|5)%kg!oQ3t=HLr#U}Kv=E>pNZHI?utBKJkSbgxKq2$4o9fO&pt_+V}IN_FsS@DsF zs6OCAH3+*)L*fY~dvn$5V9Q|j707vE0WHAq@-j|Of9+dS?kQHz3s)A11Jdn9WYO)VrNG@+o~~d0fdPfi)nWvgvmUtNp8Y6LOm3d zb-07W;EEX7@a6|Mu{KVJ7I-_%Nv1Q(NoEU>lZLc2O1zD=dF!a^G}Vh7N$(tsmRq;U z5#1s2eyn7hmI@hV)FS&gUSXyJKHIVPYPNNk9I=d|q+}LWAG3?TkZUZis4+#un%InQ zvkcuDqGD@^)-c%^YT=!X!CtO|@2L8EyH<7FYt`jqc$p-7#NSPr*yhUsz zH3yRfZq@5uL)HY?tZW_C{NEn?`IaB4Fi|nRez#tCBXucE0}q~q!T>XdM$iZ&vYeN$ z?QGHtiDm_OB47o$>T%k%1v&#iY;&0Wvc0w{kP$Pg#QUpWKeGp8FqRwt=Ywke4E!eW z`^1U6l*T*r$Ii?jgX0~)ogcORNCS8@JT0&Rvv9;3vZ-a{Mfk>94^ExdA9C0oV?Kq^ zRPFJA-lvMpIBG0%sKDFEX#s9kT7o#305q}Q8qG{tj|JYn5NI!U>{$fRVCXgI=$|1u zFB=+zq0zcxvzdcE>0rO8YWTewZR}0UFcceeLwr~-)!GXcnAm6<9!7?lX_YVwb?<)r zE&|3DSPk%yu3TAp7gne);OyOm7Qt@BAS$R;FQaYXdC=(6BYE?Yx50OzCtO)M=fRJ7 z6mV)rRnxKLj@Pv#7CR+;K?*z^9>H9m%cN6hfEp>UoX6GYIZHWLdUN2^v-T03rgnn2 zROhg3YMZdDid}K3^d4SF_cq!wFk^(pDS<_Afn<9H76PJ8DcLqqpx{N3SONG+?egj) z-tYGiuF0$K+u56E?d@inW>YBsv%|GtA^jv)Ev`OO`wUqry(YQUhvDFp)rVG}B-@ZP z)e9e+^h{VU>gQ)_6h?XsO%ci=YcJLdEmwa+YOljBT!BgV{`gHs!_%bUYwdFgtHYJ;)P6n5CMp09(rBTHYsE=#U?pu3j?fly7 z|2DAptXH&mrP*M7V9`r0pGIBRdLJr9S^t-ZGP(%P@^#OrI%ti8SV623eS zXWxOB`y0cCLBCGm+Na^#SJr+R9#12kMjHaj^%O1X<7U;8z6f~zAfQ0AN-P~Y{D$mO zbsBMlH3D#$t7qVw^_gNlDDoU{8$*B0gh6nV3mnxw!d^K~4sz*XK82Ik7+M07+Pc5a zH+Cw^ebCbA-{)yH20hZ;jE411Cr$4LVH5 z_#!v4zM?~tI^C>#Y?+gFK2pt7ma3Xkl0K$h)dy)EIMWo5D~UEBgfPkJIk%9nk|{S>Af`{~1#=9!spJx01{cr2UgDkCOMF|2bgI;5K~2viQgUj7 zj?XiEHB9#H+XpIJB7ZNAFE6=KU}Rdz0=3{h%-)VlQ8rl>)!^~thv9qh)#=liH#(zc zvWYY58BKfO0rIy8e*qUv!B{2g(&jbUWaf-^CiTDrn5ROH4;%Iy=G{Son~V~e7>g{! zV@2Q#=0p|r^9)_Ao;l>$6ifWe)Sx8+>bb-XrYN^c?RE>fzGZQ5HN6p{nX+V3sl{Sx z|FZU_C+Y_Qmo}=0-A5QVTa~gX^0L|^w!X&XrvErI<=U^k zbkGMd%7?tH5{~)<^c=nH&^gbeya4Zf3W>Z)ZNGWZRH@!|Vpr-1h1sEC#dK4}Nop z8d^|9uB~RX;cGbjpIndQH@ARf@Ess~Gi-Xbi(n^X0_IaRxi^5kceWfhF9kK5NP0+e zwTq6jXFl$pGD^mkSX)KL zoj>nQJE&Ta!b1S*T0K+K2UG*+Hj)V?+2#W7S`?G%l@Ifvw#BT0Wz9^HQx;rSJi{=rMMsFYK+hs7$spS*}V%7GpW=}(m3oRD8 zJ^>cO6klK-1k{qK6NS_}cyEs=y0sSW5f*h|{A7~KMOG#ID<-WjNd)5zzd=kRoR`HG zsb#C056QAAp5W3+z%J9;>AGtztlZJu1o1e>$ON_uR$%EcGS2LhBPzYHu|Pm6B^i$# z4cyCuw8?nFq?Mv5n8LQ*GO;Vt)me{HAR3}i#A2-t8^K$in6=0(*AYYXpKedGKe7y( z*gc%M4%~i6T%Yy#h7Zs@7i1bSUCi)(r|kx=Z(XP4Z@FX zXyJfST85{aTRxPgY^G_rc&`HrCk`G~WYeXN(^M#M*u?~KL6J%?XS~DrvIE@oX-+&E zK7e>c6@f7b2~$gzlNA;oJf(xqY&L-=19pLG2-*;fw{B>GA)+&?np(%IweKSPJt0k_ z4Yu+cIVr#iHW=bB)S`2!hzxK%0Kay)_uRTGo{pjj@SKlJ=-Xm8p z!omCG>Tj;Td+kwt|6%wA=j(6&;p&I@Ccb?8>PPr}-T_?l?$zI4d;IF#_4huy`qtI= zuU_mF${lSdm>8u+S)c3PyJ0-hEU zhGG|e%*1J50i*Xim@G9FoV&=8FSk-aI6d$Z7IwVsT3A|mnIPpJ?8V*9+8vJ@rG zWf`D!eU^Nx*`1LJ*qs4#(3DKN)rZ@2vWE|H7gmcN0obXNgPrj^*qPvq1hl5Q3k$B` zFZDVyuceiETdiAQ{RG;nT&6pJUunq=PUlEYCHvK^nh39ua*`RAOKmd6;GTuv6hLUe z91R<6<88?7TV@Ccrl?yfGQJsf_W1EhZZ5Fl`&^sixdn`TGyr$M@50u|tbZN>-8I;9 z!I_c!PMjDf=kseSJlLa#8 zy+6>jB;yGsuEaB_5q)dhM*yS-WdPDVa^`Gn>?-i;uE*c7%Qx*aC2a)D@FV1I7s_jd z@U(Q|#0dp_`6Kf?-5HM>wG5xyHeCk#n;bfH=%!RiT)mm(P~UEzV^Y|*C{(Rp7I<{* z`P~I`Pt9t@!9ZhA+axq>%uFO(BaLA)GowP3`l3#}rAD(@rUp3-V-BCS{J|14R^Blv zIkH^kW|{h+%Ldb#j`}B^*;WXdUAYM6xX{52I@Qw(vqVfuW+I0db^}QHxTFh?&*#F! z;n|V7d2UmSlxcx3b1QtAc~@b|L5`Y|Y)dik+7g_jdAKJRyb`$xHaDtYYdhlcOa`%{;|`)6l(4=9vxh!5LTrKgi$(Mli8eV&M^# z+UQ>*n9qrnqG`do(eC9Y61dmHYMVJasRYr;t!TA(G^)WWIrtW5aGY72Ky|<$#@cmF$@(%OgQ?CCE~#xGTr}i(PPO$u1x_V{RB&W!nunWBOx}^8){~UZ)h8yIB^|rB5w_hrBjgC|Nij!eO_~XbBG*!jAb|sTk z{LEINS=BY9>Q1&{F4&798lPR8mx$1d#PJ{&`_HGHTzRvEweg^gZLonp3=e@ZAu;po zv&0iVgy;G}KxxE5O$9Ds8pu(Tp;@pQZ111vCdLoVO^nSSIx^!yD@ri;@+NZ`*WDN|)7P^+E9br3hwBJD9F<}T?*DeeXm_u)OH&DTPmz@$gZ*L;9-9wy4$kYCR#=iS|s-%DeIz7BEDM6!19PInKm4)T^m%boYtGbD##`YhF@#xQQ*Mp9EWX6+C>UCWs&`()I53Hf~H+7k~HD&$)XpW zl4P9eB^=7MS7N|q_|l{aGY=p1{=$0KlMewfApBDt+fB1=_5EQwp6->D7EZEW`=9}A zx$Vh^7%52+p^pZ!7)NpDq;HA%c<9xjMD%fN4$ZbThlbfu@Ltx5q!*+pdp(6Z7EknA zJVDk=2Y9q&tYCs3W04t#*T2AL!zDY%rjy&mrsGHut+oLZ2NumV@*)(lv|9l?dz!+6 z=hTGD$$si$#>*<)b>c)L+d6sh%;donC+;Hq;|l!M_|8{!C@KP;erdDxyi=fxU=UhN z(cqsNe*ZaQ7nf+UDZaEX8l4UrjE-YtXJ!|Y6^^@3k@pbgG47E%XIdwIuNHqecfZS| z8{SavP>G+>e3kg#K|s-@l=v=e?VF&svm}Jk!ypdKeOk#TBNK=mKkn_0<9Gw#cc}sQ zbH;xFd&xYtOXTRth}SxcML&b2@u^&q@ltWnH3z4u?Y$<`qKMfxgN)=2*Cf;QBx7(; zEW^)n%9p4~az!fnLVc6URhSO@*YKz*^wQAHeBT-GD|UX@?zW~Q&Z>7YSrC#?f7vjh z4lZ|inuhAnOvvVWpGMSwG0Lc3j1cQ8QQMUz z;iIPwoimPml%;Km^Si*{)#1{Lfa%w{sMtMhL5*V%q}rCeyz@nHqylC*45EjCB5`aU zQ-im~4;(e>werpBAIGxMg2@TD7GQQxbt0VG(csNFhk6jr2pBOs#RQPAghGuBh?!&5BD?v~=sPA9xe-OuE|BwGVI` zsa8yJg#ka>Yo8vGPX^>f26&*Nr|A8cOZiPvkFC%fM4?Jg3Q?ngc3Y_0oCTS(E zwe_zD~-GYZ}W1G}G| zI6=zzw?Lko`rhl5*Bi{&{Z*Zjb78Srl+4q`C@s7>aQaKdO}2Ma6$Vei#Is6b7h2yr zR_1vs^RC&;cymHkw6=mxZUo@hH=T>1c3$ou3prs=qz|j0jYJq|2CUWNs1uMp+==CTwuCb4SME$^u#9PLWuGuu~P2TJ>ft zZXx%HLu?m2cB*~XHQdr8f~ccMmMn6NR>_eF8M@YtwVL%Y%x433iNukZFq_;1>R<0e zfK@CC%$gSA!B@Ie!j{M=saUH&7FZzn3Y`;O$T}`~pRI{GZU`{CIpULp=}pQ4Z1C`X z^drl38POa61=K{L$mFIz!=$g1D)=JFD(SY!cqM4{njuBr!m1pDmh|dLi``8cr^-xL z!Op1a%U1Vfbz5w`DF%KOsP$PIAzv|K8FHDO%=${+cLOj1))(evBbVGR$Q z*K%0L$KSHmdg^dnxG4u(Z;D!_ozZisW@0fL%nznhBLX*=swEUK8?nrqUoWA5KW3X)Y!L zTZ_q*lFUSgR>w6YSI%Q-qHd=3TTj85No|{vDMscc7iDn&BJ>qZQ&)1GC49ly2%QDV zjH0Dl!+3$W8IZ(LDU4-qae14`JxETMUe`)C=wpcUaWOv3EW6<*ks%)foa@l2H0WH#9{&4_Dnc|zKMy@phJg+ zgel3r~f1bh9Xq_HIH{!$z!laFRo-X@N5H*y!y^~e$^6LS23A>g@_+<(Rb>*9*~IR&7{NjKHjm4%3lW0Q z#C-|u-if!lJS7oO7~fp{5gCRh^Ua5keDmgu!~gT#EB}4zUGhKAKJv}ESID=|UHR`z zzbD`R^1W|=b?Mv3Up#T*+h0Ah|C=|j+|~TV;n5@Ho8SKazdyR{zi$7xuUxsy`-&I; z@ZTRl*jMuW_E#5SB;Wk;N@#h0``jbn{`T>2ANoDasxL96cFy3c+lw`;#M-)imy5w| zzw1vu{Plwme|_OcH=ak+k9_^k7dM`}c;mGz*RMSF<%4H${Ospn{rK@dvq!F9e)#(1 zFWh+kk6-=x$DW>M<-AKBQluu*qLMr^0&%kM^ywM8m=~wW z8Jqz^zDK6veG~M=8HZ!2`FrGuG?0oT&n@auuqoh6tFG;G@S>~UF|~P-6uIG6@I6<$ z4qVQy1xZWHq&8f429X@`yH1}z?Y%rh!J*edCbVL+2OfBUoXHELs$<|4t}0^dAUQ+x zpl|S&zN0A0zFvCj0fIC03~7~p7~qW)!;Se2{pd_mi!YE7lT5lcJdnORH!YiOK120` zfKQR8#1gHJVqm3v_3dkqUwsEeyGcb&n3dv?r++=&d+o9R_{p`i>tu}|d&>BctLM-7 zuekO^hbzc_E!7#PaOc{pxcgi?3v;p`*)4HA;nQsQl{UrSeE1zf;X4x8rj^b_E)clW zqA}CC9}JEk$#o$X6QNoIgpq-ncGdbLI=e zj1qe8k~1EU-Hf8UbBNf-sI7xJMjR~bLLf~?GRwjO^hF`Lai)vwWrPNKas)o)0V?425tHpStcvz7Z-#yTyy0@i7bG3 zVXwUFU>OL3IdeSUX(vZjni#acbXyabcIC&u(UrKR+st-W&Ph77FLpQCmmG>|gIah- z?-56A>hx0M89#zs{V1RSI~DEhS|jjCZZNzwx?}+&t>!L-H`*vjsBp6YTWb&Pz<5?k zw^d9B-i5qB*X-n~4o_GGliNkY%DjMlgdRC5%p|#wQLK~N*q`b!dSl)1*1ApA)*{cO zLnaw@U`LuOiYU+{vs6f5s4>yB^AdsWyo9Qx+mbGOcKl70G_ndt{xJSukwTBsjMpU- zJ`q<^iPlKwo@j5P?!M|2vC;`NlV0}}dL!4QqQzTPt`ju2_(NuKV$6mAJMWabr(*M{f45Ezr5<^sYBFkO8r;z=l7P&+_{U8P3Qmi;7bN;ya%trIV!tF zxXH{QE={nHCy{qe`wfBmx? z=N`ZQ(~q{OKA%4}&>|N6(j||`toPODU;Xam>*qd(F$57wb2c`5Xf#EI$2x;>%Dx_& z(A-@KY&dJr00yqH%{kJ;eYe5G^I))cbd+RU(91ct$l20R{TD)%ka@Vo9vbI3JQ_ao zb&NAm`o5xu&~PgWq`k|;Tb=JctL7#G&n|4SR)L~fV;OwG6umQ9lW|3B%jg7NdY}$x znCmr0N~BG|G&@$J&xj?VFEiVsvs3+mZZL{)*vrTu!R=DLzNjg#4sQ-0=B#3uxEfk) zz7MYJOJ8G1GI(c55)2A0vK!gA2E~M1K?vX;L+-)!gDH{PgNE&%Qk-?cL0P2s{VrSP z;p1!FQis_N&|-UHy-Be;MS+D>)aubTmqb>vOdCvhU>Gb5jIk1{dPwBH(CNK`FA4+H z5rSRKiZX-koKtc{*SX=5bYcq^h!g(56X36!;9mo+mDC6bwlM*rWqec`XbwqQZAv+G zjdT*3%G#OE11E{3o0kH7)e0T_zZPI>cf8Hg;%LJZ+5#WH#?6Ps0=9M6q!m9JQ?}_U zWCQvNN$j0Vwm|^i`Bur4aMAwlC534cTjjJ&tBwm}%1?2NGcr-lV?*iAuQ6|}<|#01 zftd!`KLYb*gWIeZO{N!}2=HWjvCPb6f6qp|iN1(014<;Tq}t-fR?t-4kxB&%U?@RA zMchHipMH1x^dS!=xP-JAPM(^kU127bUWv2yz0;?2hsGynMy6-HvSW@n`8{%G0lwMk zaKXaLWy64T2jI62bLQ=3qClt#PTUzao5`F}wKK_$OKpc5-#dPMUVs}^%AY>%tEln3?UtxbNv_C+L#^;;s5b-TfT+vHr%Zh-IvadR ztLk3egHJkqnOVIld`MykmX?Z~b_eY`Cg4*Zn`{3u(T+2#Ol65oa?>P|a&w7RiCu@l z2`!Luy2y<&YWcNmQ@I9~M_NgxcAuqz!S0G9Dxwf!=DTNWs97bJY4wx$+VXF%E&r`Y zzJ2j*XrXqmEkB8^SjlXwk_*$8uf<}C>mOh2I|cXEBj>*Q@OS_9@~2<__WA2?Jpc8D zD>q(v^TsQ`zy9QNUp@D`>yN+i^||ofiF2ZAQm)41ezA1sNmaMuNln&!oAyvkTR6Bg zy)p%~gpz8j)W7q$o{e0-WV$OCqmVyzZv)a&N;iDgEWcz+ms`z=xC4IFH-0NegJ!a5Bfu^5@*4kJ0oD@^S>Ozdqbou^qL zp=#N5Xpznc>A=&>Buu9oi}?UDRW0t5talJhw%Ci*)@v4~(}gTaVHJqUtrFG~Bb0{M z%|Q;DLVDn3-z8;lvUj@7TH!PKf+MWEi8twPInQJROfuv$T;e)Iq*LmNpo!iXmVYgY zj7Dm7)|K!2@);wxEp%RAb_w2!gfHT0 zC6?TMHsq{HVOsOD8jbq3d)7-l3kzj8+mE~ezj`QugsNn<-3KLbfWbudbl&4&F6_Q} z_dWO(Y!&LI1QJTNEvXY6spg$)j}i}U=G|*gTz!{Z{lnFduKtExdlY`Xef8sOPuxxF zm%MrP(zQpfzJ-_6;l$p(_IUlhtMA~uAA0yM=gHOguRRWzy?yl~c=uzt@(F^Ug3JEc zLBIotzCX|+;DJMu-apVH;+_<=U;WM1-(7nGhWb`$@y3y0G^?aCN~$T|(8bb8?DI$` zTQM^O2gbnw`=94t{PvYc$hVLEv>ys*H_!h!7cPGDt1I8W`m_JO5FTFko8SKan_v9C zIr#}+1lxNZPyiVvk&TRL87?g0!8J@V3r@}no7(I2n|E4h!Z*b58S&{!TK1I{sjB9S zxq1h|1e;WqWLs%wz7SBbwO@O5@1ND)TRmI*eDz1Q4?IfVOQiP6>JzmqAWZLy_u%~H z)rXK6-s6h@p!PYz50Tn4wa;rGBgwD#(xZ?+i!{G|=dRX%>2dj9#?h}n^bbZCTHN=l zsygdi#9?VclGu@uink^^0+%t>ad1kGOHFQbCK<`;#U+M34iza*vXX-_ss3_ZaVp8Z zyc3i5Kaj>@X>4*-BZQZdNkz@>Vtdi1Ra#+IFS`wr9%)hZY{y<%dYgD>uwxJg>oCiW zAlWWer6C#a|tt7Zg#;I0Tag`7&6*q(5^FI?L-^(e{bZpl@HMH?Be z7)hVuk4S8I=qh_$6yq`Ut~gE6kQytp1#-wTgvD%5$+TOWp+(^mAdXAfL zx>MM7(B8FW(BLA;F(L}~%A0*xx0I`{yeHkl%H;^_VS<~CSvhc7>CMf4MoDM)6kQkl z!?9JY=-LF+sIY-c|64G zlLH{vKF5&vfBfXyV;JcExPjdM=zshqyn{z!A7{=@X-{N!X(Kq>wY8lGb3?sCH%}o6 zNG!WoX4??OB>ZJY_O}fpe8oh!#FJGrTB~>hNA@@Bl%{FHq7|=Z&h$zU`bl@qR>8A% z%}5!{F}mQu2lb{H(#=>@I$$g+HLR%}Na**>y_?W^)ru`DMQ$SIxmR*!_sqG~5{lN= zLx1~{EWfUO3h!RQ?;y2T@yk{phc9~&zV4IS; z(a^mGTzeIU^%*9JKwv2F%P8cFp0qz(eRB2D+a##SE3o!+xMi6B+81#97vX7q$|tq9 z_l9fd;Gmuo`-HT0_V4tOjc3b_)|kw(R0h-lIae%dvFhXMj_+EqNYJ|@Rf`>$XTd^ zZMO5_V_E`N-tEt?-+pK@%^^{Go_nSD>qcdB@dPawNnxi$A`!W*`}Aoe|G)z~eplKS zbMlxFRbho!T{*fWdh9T81!O;ac`k2&S zgcqNCQ4@@b{HR@M12gwF1LExyhO`bx@+P$%BoS^%G6^=VqNXxRLiH;;zorYU>Jplg z-J_knHQ~Oz;5HVi_6`{}<#nU9t<^ifzVo0kM74KaKq>4~NJZ~cn6FkkS!)9$?aFY( zwP9_UJ47Fj3n_U)ketJ4nanky#HeLp84-)XzN#v!mW4-H(bVis#nF;6C6?L6%yzaT z^pYI`db@jTBB?3Kc&jebgVrK=(?YlfcjZuW`t%sQBP?5|&I1pS`(1FREz_hcZ&5nM5dN7tO#*XCfR0* z`k0RbSC=H?N_sC@cx$YOva3sBd92kvvmTO3C7p>(i9BBa8)SMG1!!U7!MU3(QJd%# zCkTUKjV;3V_J~O;^TL(=mbE_smbyd6Xr5cli}2l}j4IO#cw5q07U4s$^F`{Ad%3}2 zA1#UsIq3S3(+A@|-Rs}|;QY3tOejeIwki|DVc|*98M6~qyqnI zT3k`HKFd&z`8cR!0h8uwB#=+Kf3s!1)~On=u8f(b^&sKZ{(@%p;S!#%lVdSrg03#q*(Wk*29jx-W3 zq*ARZ)=JP^cgZW?q;h$OuYvI@5*eQKMGn@_43AbFu?Qw3{9d`R6j}(!xNUKoi1{F| z)tGOyi3Q$m)VLCFlf*9C^y{7JtcMW4AcIr~-t#bwLgUPJkcJ0+_r5E~8I!{H@JXk@ zZ4YmFjpz=ws#>NggG9C|1n+e~DWsKbTU$zCyK9c!?=pC#U>r6p^TCP~PMwCN0j>NG z0`h%`W%Lgymc99~Ju*Lhmo#T2w2Y^#1phGMHnD_547%vI3496HWYC0bdf2lTBQLrI ze(F!JdEz=Y1;YB%Yu{DPg@66iYw16|c8ui_`^Ns$YX!FQ26z+F+3})mAplyrB1;i! zdPkS??y@f+jBR>vZN2U1E_tw9L(N6dJQbI4$$EP=IY?lhm?_ zbknHD*8%00P|}ILB9iVoN=INRIz^!~P%5TtlZsekrr9gr&wh3O!|SiTfBl&!5CY)uU;N?5yTAGB-N(Lu=T?OlZq;^m{iiSd*TZjr z_0}`j-@SbOqc?B7`R?^set!L@4~EYv6qyCOspvBEXDlWZGc%cD-oR5IzvwRgL>Y5_!-E4xkv z51{)63u$b^h^6-Svxc;OFgV2vMF%-(!qX9TNn0jjirSXUh(tv+?OI&t6>1V)NLXOB z&YR0E#GT``m3wGPz|tcH0lT<`--Z{?(UYKViXI{L9lJ?r4h3>a}9B->M#r8+CXRYFffvcE19;+*)V;yN5P+j zt?MwX6GN2yt{*K2?+n`P6e&|FIi3r>9-I-&W@gzZh_J1)X=Hy;&_4k%D-eXB*tT; zB&1{|$CbEw_-$-#R(gG5^^l~RpCX`GYt?pi^{3dV$s=^iyw+Vb{ zQF38}dsmJ|KiH*Uc`^mYTI5Q5o%Pw4G|I5ifsq-p1e)8-yXElP(ijum4>qG}ot2xm z?Yw7{WLvB2!dApqjR zphZ9f2PCyTY4*I>R!8~;k*FdTXSiIglP?am^E%<~z zctmongR4(nLxSlv*5NBz__e7z%x7f>HnWrPJ!)hf*|*!&EH{P6Y{3m4p;NY7dsGG~ z5ZXD`7J|-ysn}n>eQdrA$5wxkf@>zIgDq+%!oDQaa z)y@&%I5XIP5o?3vyz?!)W&na?5H%7WQ1{F`HLj!*fvYSBCWV=2R4;CFlV*fNs>~tq z2X*|59Xz6gqir71<}Ac5@QlJQk9)v%Emtwsmalf$*9N8z&9r(#Q{2*pY1u_CSs4S zlOwsUATo-k?rN1K3%v}PX1tCGtyN#`s-?gdu*k?US+!TIKvR`W8*B5Hv9%6%H*6SS zRKboRSI#f2oR`ef;kfVeZ=)2RATBA`jG;wwgqLaFSL5sJ9}3z&l<`_5)YioinBAfF z0a^X&>O;r|PFBxuU~gU`4T_H1M;^P!j~b*8mx+gLdC4QVco6v@8blsnc*h^W$30$% z5AfU-_~1kEQF!^m>RGtp|7Y)QyW6PFz45QIPtIC5<`j=LBaNQU>3uKTv6B!xacz^_ zZEnBV(rimR8qF4aX6(qM{}qz(&_WM|J{;P@;o;DRP-p{53M2v6LhzQ#uj-t>I;21; z>38^F*T{~>8rhv0E3p*^l6W-IjJ5YQ``Xvz@Anvi+r6{=Ec|>33U9-k$T)$NB5?RY^NL5PaT?!!0DtX=t6%@x@)_b?c5U^mUhT67 zX+7n{MUNQeoff%9mr0K?x6vb6M~7zzGijfuNM^hH=x~11M-)?Bv3JaSDXw5(7(6$} zpY*J0Us3r$lnA4aHWquOMkQg(^+KUd!-54-W(SS|~zSoRe=>HfCu&_dM|Ezsc@TBz{rh(WzxR7W_}^qSKP0-Y<{ zbFKPG{q1)p=q{x98JEl;12|XFC9}V7VynooE1BOU&b6~N-f>9jq?Amc_3fU;5nsmN zW9|tK5?6(%Z#0SEI9a5)v?G<-6EP0i-uVAk(XuK)ef1^)ed?}Ndz>|J-1{Ffly&{!;8HM zFyyUyW*XZU3e|Nb9Y~4Y{d%osYn2#$Zxlp{7T{;GM|v)vATmlW02DAE7W^1o#Fe*K zeslfFm5b#1V=M2joV$K@<^7d&D{l}ueC)r^!@rM{m5bLOUwQM-Pj7&=e;`=>e?WS5 zlG|SU-yX#0r=DILMr3KJS+J%OStSiW{Fs-j-jy)XP9~B@GUU0pRrXe+pg=I=Er(|*`{V!Vh z{B4lIbssdiT}0ANVKGGRH}9j8QDQJo)IX zFuufz0}@=|HecX4lQWe%IpEMZkIMuTWSq{=*ZpfZGR+n|EU7LK9HA}wny5*-NWEt2 z7U=iR)}5bLK<5l@wAkEj|vyZJ&;x;ctH%4+vMMp#wMN74X=VLMs zExeqP&F((P-x)5xW$RO*T-UbOmk~7)1)tEnhY7M%VBFkCfnJ0dpC!ShAl%b zvE$^R`{}PN&LoS@s~(IMwYQN>=zT;@cL4|uUF59wf2-kyDe4B8l#>+ZHL?%_WzOx# zKm5a=RdRt;^SyB07RXH9C3Jc{XnDzO@Itn!j39^g>&Y$kjSiCn1J?)(ck06Zr+%nh z-xNcBR`S|W9%H}pT^JWgE$3CS=Q{J$R@i+8fz(<}xU9m6ErbM*lQ;s~@T+XQMHAT^ zn_d0lkwu{mCp6Gns+QLiYCfAtB=YIhsG3h{T0S>C2B$~#F`rG)wg^JYNmp_K#Lp;l zQgb=2?*wuHJA#qa$R^)gO%?=m3e;rgJl5~Quo8ptPcE{|_F|mGa$q$6m_W`S zMa_lRY8ge>V%uDmz}Ob#!kEiy{Urwnr_nIvGHA_eF_FxzUq; zli09tu?1?`s^-J6$+V#gGa;Gga)`#S>7A}+44?vp?eGk{JG1lmu1It{Q3P^Wgso;e zYj8p*7-0I3G+qOXrSXRiA~NF#0uh;UZlTWD5fPagUX5OVlhu_>8q6e^Ie&)7*97Dp zhZI0kN#r^OPz;-t^QMEe05vFi$H zyP>N6Bg?wX!B47ZQd`q}8AVI&zAWEx*6G{$4_Jm6oT3%%%&|B*N+s?hYA&sBRQqu#Up{N1E!zk3)xAHLv8+=4avqf6bM~3c9v1|(4CI_jrWC%GQT_=I}4>pXh z`;=0uJMBYi6V`_u7P4=l#-e$t`62vm)gBx>*8C-6X*4f4FE&5KiiPE8^81+-%!7WM z^f;D0c5GVW#ko1se3y7P_{b|@_|UuR-|;rjHDBuAM(fr%|5t02WOe*U4jhOqe`#uH`19Te6Qt2rAOX~VMtBW>8ljMR3SX?wX{X;sv< zrP4l^$HhmBAS1U*49>_w$)h7eYid$a6Plu`@HguVsO8!j>RjjyHP*Xlwvw|M=Vn!u zB%S9y&C$eIFp>AANw~01rnyyGq-J#fweMCf59K7jSFdN3lo}g|$Qf1QSa-u*#C8x>T6}pKiXnjb;rnKVFr)iG6Vxg> zaOFDe`=})vD4HaAps`_$yg(X^*l^b>GrQ@o!;=Bsb+~$4+NNT}`LS(6n1@(&vaYPI zs9?{g6ixT(si?a?${?j=QoFCka6hbTZisTdW^Er23Qm7oNvC#sFswkU1=rr*b}6rC zunP+Fh5_O|lMLJ_i*{O4V;-CX-zbnD^bxV|=XMxUk!3Q?s+O zaY>6v_h?1$`0NtQk4}h&e?YbYAezwC{IFhTU1@Q4oROom!x+dku-~e};U^eamE!T< z*;RLDPak1eEXEOH#&K9>5Z$flCCRHj136!wUTf<#EwoNk!9WO5LGG$-@d+4%bXsjZ zF;>aekBfNX zJ5XD_EPlj8ODb9*=No$Rhv9@+^sI|TChg;#q>_3LHuwagF6bn>Atb{=%Bn&=+4|85 z!?60MQODHKaf;{*!mtE(D^^HN!f!N|JcA6YM3HjNt+=0_cKgf}7@l{KSkN@{m)*=SmghM|L(g@3vf-; zTa7OKgY@&{(09q0$zwF)jvu1qWM+}!M2s-H;7<~0K0JLbB%nS%68;lRT@mil5;yp$x9MrPi4e;!y;>s%7bY~u+5H>K{LhkmFj4%CfBn_M7LVU8PW4DNkIxeh`Z?cH}EHw zP*gpD2pYl28=Id;6sUPZi;m08jrw+jK&|7FTE*J|q!w9K9+XImLuSMRbHoCf0ev3> zqbr+M)Lc%{w6vn7e6`o;2e8HC!!l zQ1O%4blWB6dT^pM<$4f&LDQFIRaI5VX^N^P6*c3l3DMhCQ+nv4sTABKkPF{fCX*EG zACtobzyLc1GYNXG*pDKalnVNn-NZ3G)hUi)%P50$K7~%YUG|A3fd$f7P6(!%ujhTR zd+c`8&@D(Mo9>i~T@Fk8Mv`b;c%9zI*}syH-#&Ki;M!}NKPV(Sj+s|UK{N2Vxz8Ve zZrum=w|v3l2z?J9Ww3+QL{)t9giA`^=7r%$UwFi%4j*c5YkcT-?Q6H|NobGTr#rh( zR%GotW|XKT!={OmXyHdWQhgPUYWT(KY2n+3bcAo4^TKsVi50PIs>GOA(%DCFJ9i5I zC`JKZ3Mn=_i-ap8r1ZYI^aRPny8Gr2XTSON^ZCF3?1jHwevAD5r)R%;eDUnSH^07ifBTAqBZtX1zj^a-@4n~%+`!EjuHEn5;`!hG z?fp}Im0|RZh+@zXB#U7qy zwlnt@0VYk#7P$S#FB|ORm**w58s{jtKYsaJAHPhijq@g2blwxPnrxg$`WSo(9y%$) z%44L;OBlRqAt(2foTlJnUaXK|>BR9F%Amwp%@^|2+kMpDaYzwm6;1Edx}i&hzSY}) z-8(b!6OHr2DkHD=cU)SE(|3(=KvEG_rkeTe@9}lMY#l;|U=bz*oCx&TP?~ zslq!k9FQKFGio=M*j?m`Pq7NON`r@My=g(oba%run~ET*WCO@5wXhcDxZt?OP(u(w zBzznU2Kc(vV_@6y!q^n?b{LEw_^1tIldU4RDwW{U2?DMi83Q8^e?DY!_z_undxxhF z940ekN5-ZP_W&N%GL>0WxfZa-J2g?7z^~AzrJA!6V$_WZYvLh+fPyWj3x=eIy0RTs!~e9fuQ) zrop<1O{&wk>*7GI4AUxVJ*)eSwJJ2&SOUF`(daw0+;%_xk>`F~obU=x$f%IDRp&Ue z5IP{cR38X_XwwSWQxctJhj;(8zpm`INLw*jYg?;tRcsET3+^gMg7p zXxY3#mLG)^AGR1Tn^&3_FwNV1t$CsOe)DRc%!n$6od#RuvIj951@?=k)U47;r|eb> zCq0gmlWI0mJ8`o45?10h-|^@(A7B3Y@_&%#E31T=SC@Z2H-|T0f5N-q-R32%XY{_( zW#WA=HA@0_d*)>GquUqN3)PVlqg-+GIf?uv9Rb;r!v`mQkCEnM-D$lD_b4S3N+wWE zyz7>=3lDg#)JpHxbW&3?fzWDbh^k}9j!xx=$7Wg`VWX148Z`(GmMn6oOzwN+5!)@; zPJU+U0Qgm~lQ`sLvCN7U9LTpVLH_NZ1_udPz|=%DbEv z!~(p8MZK+kW?@!!Oz_4@ZWu7JRVI0n9>@Ddf6hduE=pXQnVkPQOKH9*Efo?j5T3OZ zEc)0jH1|SZ0d+Dess+g$w@M0<79A$+7yt-8ih(85N-BVQA0tQ8XI|oHS%QyZ!XjZ_ z10f|7FqBnV@rd3c><5ZnEQ6k$_qTk;sn1fe0VK$yVue-ghif%>upH`j_j=8}AOH4| zM-Eu?Fnbl7%nKy)Gwm8PsU#<1daQ3h^2p4Ay<>-lCl7u9`{ZPSPv`#hi$9(J)6f3&I{bW*{OLd7$8X@}%kUFVY%hRy^tJw{7rZb2 z!k>N$W4Z57KZc{9L>AMZegZS_69V({3!jX%_KxZuhm%lBQ*z19p!aUdI~*5|BuA-j zbMrVuno%$X_PPab^`agyDtQtLBneAqOKtX)w0fxshU{B=N(~`kxr77LwZS?j!)B>* zUU~@d#HrVIeuAP?(b--Oe|5hsx+cOf62m1$(0kDZH;I7X3*0n)od(JFmYED$sjgz5 zK`Id62<;24t|UvcKI0jkE`z64MPy5Kj;3jhwX+fJYpjTCDyOK4T@D$G|J1e@@<7EB ze=Sf|yCIm{6WP^K`I5U=bF@VEAE9(XF>9u+h#@M}MZfasrhj{i(i5)`< zdz8aA)3_)rlz_Ea6DIE#z1cGp=w_S&d)P*3t|Njk2yCI_SkSYE%I`H(C@Quk`uk1W zhFfT>O;0MRUATGPU33B`WJ=N*E@!`KM>?N_5pY zAr=j>XtifP-Jbb$NLyP2^Fc`k+89EIX&$N>2(66?=^`eb#R8KH;MI|VbczV@xxLf& z{cC+;`Qa)xoid|j0Os@le?D;Fz)AQ!H+TOX5@76Q6Ekf*io+i|ygDtBWxRg6{8_B- zNvusJom7%qr&lH2@J4#`+2(slIE&C_%V(SKEkA=XQ?Kag?E3TlWuY|BH7~<={{V}5 zmY;&7OEAt49w5!<;OCWASrzdLt}Zt}@@RWMY`z1-d?K`~tty`0IWQe;1D;uX3W-Hn*SE>>_x1~WeBrWi=I z8W!(o0>nidBWjIs3ARV69E8)_&r9V&yM`=_-i$L@bfrWvUT~NYka`USZD9Y*=;-ME z8^jU!jRfL|4_O?`lMce~HAHns#1lt18)28g;>dyN&dB1rnOut}rW7@oQB+M;RMjWh z$niw2)fLdW+lJ0D9%3$&WVm1#%P{#4K`K^+1ZSoV^IdWJqcPwg5lt^r*gq@LSKmQV z;jwvcW1?lto)bLK9hGbP+;Sz(rYDv^uQAf<5kcz-Jsa)FmOK6pR zgRBm=ZhQ0;Ym2GNyZK4*P^np{FGXKhRbJ#pUnW1@?z{1hL&__yTB;<&X^Se;(mxJm6YY+AfZHz6zEsZQA}H7jlD;Hx|?L#&A^gFDJ>XAo!F3mLS= z2w!=go7FA}ggqlOvjHFN)}*b9F9Xh50;@Nbb&j{{qg+!%lCEi9>3yJ1CyI*QCqbFo zs7g&QD=;M+ugx>?NSFFSvu+yae5g7v;hZ&eP%?K3oKD80XS|2G#W?lnyc0f*q(n%J zBn@lw_1splBBz-rmLhzaevnLn7_unHgS^@}Z?PqEh?c9=h`yJx6G+!&*=&MQUzxkF zCn;=4l7@^4nLvO!wA&0@Q!&49`sBW8q!h##qlYK+Dx?Z+eAo$&Pd6H2ZG=i#c(NIJt|YQwQRUyf!Smc zUwV;L1b`w2w`(TtWi#xKVgI6Jv~kH{gT{S_B^j6;{9bf8wWAzq3z)AIIY!JaJ-wG0Z|NY%Bg%rso96j^B4_9t6*y}ig8xTYo7sZy^D$QQd#wT)NrIGh}H zRmmk{fWVH*g-iS@yv=@3VONrN$qjE`c6 z6^yFJ9F7=pw@ibF#>R)E3#ZHmsa0p@mbMxl)RDUTfYCbE2nu9dmq%7fq;@-I30!S$ zBVb7qp~Dk_eDvKfg$Y`sc5xf=0b%AfB^Qidgsr>t(wCSprxqC^`!JGi5nGrBI82xd z^RmqbH^4oa_sY(8_V#hh!18e8yBJuM;Isdg;NI}RyLT~U*}U7NXJam{aQ|3wlji4y??rsH9gDX^?^U%v1pE}pNSy!7P@ zuYd8{xi4P0@a2VPzIgJ*zg+#V8>ipDapuJvXRh5ibM}iXkAD5Lcfa`E^Iu$e`!83o z-Z=f-UtfRzuh)LC;}_d{5#kJf@&2V7kN(a-xEO&%HfU4dLdqt59sh~YYEexn*#IZ? z(nh3(duAz|<(=u1FW>p->le>&kTsb-8pxW!T^7CXzWpU^U%dP4FMs{kmp?r9*H?b> z)yqHq>ii$R{>7_dq|u&=H7ZF>Q`A&i$*Q25d>wv10%R?`STsf3g?{A(tHB;4%PdCG zMej@nHL2(Udesf%>R1IS3v5B~K)?7X;zHRqVHvI1WgjvU@1rfKi`lYk4Vr8aoMdWr zYkS?3(>n8p`>E^Lg9aO%VJ$?dHGX1vxRWY8;P5l0wuLq5!;x7Bysd32EnUe5>e51I zLH4qF>Y9#?*nX=pUVXNqOfWJdxFiD;$QF%27SZGW`(4{v2ileEDu~>sa>yu|TqmVs z==#7(YErp0a(FgRYN96TB7!FqmB5JV75CHAjf=grs&7x9*U?+Wf5Av5p3L7G;*jCi zOkSZ*eF&L)Z6`YKz^!}v_8HYze5;16y-nt|grcUm5;of>-I^e&kFWF{BG$AwyVL~N zu9cZ~)=+&$Mb)*qK@D<*nqI~_)`ldxRm%BGmfD+9Xz^%DI?-vLg?3w~TY#di{t^by zA@P1IW?2V3^wZIqz5hI%ISD_=aAt1qer5Ii?9r36NBgfbDYXyYl8VKmS4kYU%Zt?4 z_Pnl`CA743upDAzg!Zx7w!mTiY-dUh=02n7T4!i23MR$~v;NO^$Noc1J@z)5tVBwz zAYESZ*JK9X${)qipNK4#TbFHVG30h7md^xTKofa8mjW2qi9mKa!lnU_5tazTCmov< z>KhVLCNp5jgV&qLfy=9(T>r9@oU-G?cn&t0JbB1sCu$~Q&Kg9rs#t&xSy!qP3i{W*+nDZ%r7=EA4$?{)c3v{eOu7)g`8=>aX})sV){BtO1~^>- z#^;Tl+Yz?3Ml9*CYX3o%+bw>>GAkION)5_oe|oqxl-pnr<n>_}N3d13hgntfZm^(nX>7v?JJgz(^{U&8j&)Ll&sD;ZoKT z2{oU~Ci5fXdM>Z#$8*{7{K(!hHJ=_EN#ysc$;8PJpXprmVQwp0yB4l)VU?Ic;cJ#y zmD0*K!%RfyodK_|!YndLsg0DG<>+Q8_$16)6lUePuLy52*}Zotxk>L3JUqIR4fGC$ z-UXaRMk|DSnAs-?wG3i1ui&+x{DGs~G&!x3NqE*v0%4xW0Q^GoJw_3fk$4b@9NoL< zF_S6U%Dmj_@9N&(wW)-%N;=&cGAV}jy)D?MNtGJ~+B^H`&WPn8wI!kE;0%u*`PfSr zic7wE*L-zbp-t0dPDuu`9-)`U0RoQ{uhyhkK+n2bCk5)1Nk8z*qGoQ(!9$3d+)ED8 zh5F#ZGK~S9yZ++%J6#GaL%lN$6}l;FKUnG3IcyVb!3KU$|(UWJccBL97UvotN=p>WgmH`1D3wG~*Cqy7 zIii3OJTl+7X3Z0ugK}STCH4Ycn|HmwrxL?nn%+hCZvvOUPpb0<1zWO0f4T#rkmsUTg*asfYA-$Kw6H+S63Bx4y`i zj#F%WU4jWLRwxvQ8=YuIEbWZhuWm-CN|!Is2A|3rlHj43Ml`T*AGExImZ+=oqzuJm*pJuX}aPBqS( zjq|M@yDGIFb}1Zni$v_~xYmP+H8;&HDh)C~T?a-vh@An#=6Kv?SK5;a#JAQaZ!NTy z@hUgKZ`E{viBqQS{CdYB1*t3PWa!$nE*kKQ7k>WNS6=Tw<@WVc7ry%C*)M+l^j|*s z(N~vVj=qpIJ)O&Dd{hgA8&uA0I{g_%4=|?pGD~0&&6b%^bGg*XwS&(1;H?j=rUS5# z*QB;X7BF@R3l#4AJr_f1iG;n#?H!NfwO-ILJlm$GuwY8bfpt)h!yKupV+05%XLMgl zfu3vcX5MkgrI%IGdZ+ObTJ|1}rCaN>RR?Mx3E9g6Tb$$c-Oht0;RohYeHoaL`Ii)S zfaHRf_`N{KZ3uMWptdp40kTekiQm~|9eR}&kV!?ha0~iusU2muQxLASkP${!h~pL* zK@M~t1I=rKRtrp)!ASF?o~xOzJD!@^R3K^C@&eFL8=8)jRN8FS*;{9L%(Q?gu-VdZ zH|iMO+2-NlDwSQOu}4jIT7e_NEy7MVXt;yZ<;JCTwY)hcsi?^_6Ob z?|bvb_x}F;dt{(B0q`n%Eb8XZ&ffgZlQ&Pl2{Y^$aCBDR6k^}!lAU{B=)kYX8!=pQ zTm%HbdNVRpEH5&wT&`i0TE$t7hW3w$M$5vmD~%6GErvnT3J3elk*L+BDeQF60%qXd zU!t|Hjc%TsFjmWOdsV#{qmM3y!14(#wj}P+aLf#jdd(>QQv4nL&E!Q{Sfe#w@GW%oYmyT8v}$OnyeV z4m^$&ZpST5gCSGR2;9LCC7TqeU11)%J1GhZ&#grl3L<)GGI;Tox>IK0##_vp_m>?E zYTetGv`zY!Qh7~P(gDpI1sGsdz*Sy286AOneuCJn3M&gX7z>#?K(5p#CL(g$tVSh; z-SSQ_c-)Z)*9bGK!g7eD++3vfH7Ab@%(#vtVw6?wr3>5`rIw!#yS*O6E=Mh?q_j@Y zJ4$Hzt3SN(*Z+92FI$?we&X4$F1`H4Z+;Yc?o~CVq`*Z^D4OaE@9S*==;0-hQj)pO z9WM$nz?aXz{MApd_FYrimv6uH#WVl)<)x?oa`mam0@x2ed6lO8o^<50!W4GGWHOyy zNTEZFF!k5ZVl1%#@+vED{Q2q6Pd$D8N$&?nKmYsu%0&-n`_FJTdba=Y{^3J=hkerM zUWdC$8Ey*OVDtdFauj^yox{}|r{xnNxa!fve-8?~uKBc4x*hmShb*>8 zVMmOvcjk;k3t`xuFHjeOe@qWR5KZPm#9wUT_m{zuEU_g~!f+tRnp}s>aK~-!gxn(g znXI_hyko=e2u6`bEUXQ;$&yPP*CJ)=v?ivX*4nt)xCVp3q6WJH{xnjRkui^%l;HG7 zmh%Cy6TxJVm{;^yrp2gtN@3$fQtRx1y2of9WHe8yJ!dr6y4Tlen-tij5CuLFD22Bd zLMNk5wm`w!wrRzpx2rI1O(a_u!mzc_c+E1XG`di8b$K)YpeS3g(p)>8i5{b3B|m_p zn)0z`q(b|**s8E4gZhiBSHAlF55Bta3fM*cXIcE^y|=!2;i)fw{=^qg{~R%@-g@ur z-~RNkr{DSdwbvsHWJHK+0sa=t%;&tSK9(mnEXz|0G2qjIjcsH8i%GXypaJ}*eZttl z(>ZV`;OQK&T6f)kN9VhuL<#tr65MMs51QJc)^TcfX|CKexxX`~HA&0hirZz!YvNgR zfE4Tu(O^n_g7LTDhV8*kVM9YE(Yavm<@w|1`MWnTkoIV|CB=_L@R`n(mJ`;{*vBNR zG|sU+`4)C(BGmO#mC!0}oHIXpn;7hqm-weI@rucNm)E*8YjeyfF)~nUT$Ak6mm24I z^gz%wnq8(3n%C5)rH+lH*C>eOzvViGumW@_55L3qG0W|aN#jJ4R3=7Gfj_E+ksy(H}|irEY1J;q6dXh zuDNnT;dYt4OV3uQuOlez<&U`HAM`S#~^THD$y>K1DnxGMbhdk*GrOr2(@HiV z#b}I_Hfy}S5gp~8v6+A}oT;^*=&i(>CM__DY5d;U=rK$<9Z~&*O-4~_nr0q$so|kN$-bmS z4stGhe(EP`n(Skipk!1TS4I_Ldqkv>#gt1Z-l9onm8|C9h`OsVeNz>tYDy-EUVbl3 zyrnSltAX~f?F|!ym{UqBupokIKOpr>lBWVw!*U> zfcHKZbTDm^#)`qlTUH!E)+w;;*2hbHyxF0hIW?`Q0hpZ7y7-WA_&hIqIyfHy@wNo) z7cKn^qCIwO3|ovH^wuPyHm7rQcL>nCXQ}a9t>JAUx4XqBslmZR@0#NKN3z6hpD3@b zfTMhgO2fyAp3G+3wOy?jA;p_jGMU&=a6cm}`hrP{ad?Cw_WKNM52=iEbwwtBxhw$p9Ox4BPBHN-avk&Eng- zNko=J>uX1zb!|AI<+bs|SU!_TtNH9m0;VuMItp5Sd^|rshIHx)ebncPsmZRMn3~*7 zB*}!LC3YWo$NJ>~_Oj7aPDvlTXK0HVQP1ON82eTd6QWk;!iZrk^=_+9>WZ#)TC;^x z$f^Gq_>XvigzsXSrg;fmlKtM0z=;tO2HtnI5Y1mL{}8ii&5yja*_9Z{rbCssP*UwC z{Zz;qrcq{M=SR%iXj**lIvpq4?ty8uc(y1l;_ zoPTEi%U<8cnCya=ST$F|=&^Q=L z7Pk#Q{<{y3Qw`Z2m&P*lOg6L)wr4_Et>9#kNx>?4S@kii1ZuAgEhrFwFXB$GonDv!9%NX`ZX$}^I+|jZn3f& zKWR%Pa+@g#MOQMrJ#|#((%C*c4rGv3lF8UTGsliKpKHF9|NQYEpM<}2bENrV^TXyl zEmEaNmrt*1@H$PlH3m>}HVqjW+eEF%9 z&6nWB)#hcm>I$5G|NGx3_uY42^Le~6QX6%=orgF7ZK!=`@-KmpJV4r4uO8s{{QT5! zKE8UJ)^*QHyE+mp8;&fx;fS57V7D+em+|%T=*f2E&pQtHAYtwFZZBdli(+LPYgN)% z7M%>(kK76Cl`(4pNfX5o{ii##Dq?$Dd%0a~f&6gdBH_IcSvB}x9`?IMR)`)w1&KKB z(vs_29hqdil&qjvj(Bc}L7cx-pI-qJP z!Z6Fi>^qdyh%MMIOuM6XgAL)>F{4-9bf`l!Zef)gE+RZG{El^~@PmH9g>}_1sZG*m zAZD;>c43!&hZ;16?bOX}jFhJlIy|=v%l0N{wT@tdwc0iYYSW18t)@CRqI;|)@$J5w zFTZf}#Y^OyA6!V5N#dtDFiOCN^HKwWD*@)d@VA%2UYWsBH(s2{9#F*=I6uQ5khk>JAZyJ#WIdS0dp|Ocfx-G)A51`jO z4k^T(qGe)3Dr2J?TGrU-)_2WqBg|Z&=AaE8VLmy>4 zrv-vxYGhy*snrJ$G0@P1%o&`Z1xXFjEsORKgfm55hf2dIm&h~>AeWSq3Di18QD2Wd zvX@~hmtiIs3M9$)j^NO zMu`P^&U3R$81g#_up+_owro;tHR9W(M(h2@T27M5$kI4$))%NB+cVwWA5U*CjI5Fk zB)Baz{5qYYW|c~|0jAT=S|sb|{h&#yO(sP2O!l;E2(wleu5U5q+S#vM=;T)_(S;y+ z zD2{*vm~UC=Yd}G|0=rHwsc5PX067jnMjE?V)c6FFeH*4)7mf!}NNk4K&N73PuB3B; za7Kh4up`{ymO($cEROe|hMHMo6?>`iij3}uj!I$Mt*38ow{BN=Y9Wg|Vg55+cu4LH zUe=q}A6+(XnP90XYl9yKodGtV7w<)_&al@6JujdRgUJVvQBnLUZo zQhxv=nQe=_^&*_mb+9)AEGe}p)}CBq1`D)70Yef=_ns%#9X0 zbE9iYI$(}OLa2VZc^PZhNb|Mkt0=ryAzQ29?lio2t$CrF`8fShz=X$%EqnrQP ze0%xP$iG z86}qqP*jKQrttElYa5-&kArVq6mW8A%rarf_DP#t4n4tJ*onf%?qobaH%IEENUH@7 z=ipufF6=8KXVQgDfY-PnlX0AeTNAaFz+(oxEVA*bM}rM!GsW^E*diOHGeK`{^v z?>5fYz(8W&$NS5{8-~dk1nUkAjZsmBQE)4|jW}kK8h5B<+{fOf`F5R9?0k$-qJK>$If4L7Ge7 zB+-?@rkCWFFvh1D#tCut9G&Pd%Y4e@3(@zJeG=bEM>nrcC{GP15g^^f{Ii6kl-SxBA zf4K5y>&0&|00}S8lY#56Tt5&0zj8lW`89mx2dxjRUig0d%=;d!|7mjl(d$2it6shS zGW`Ev_xGQOG2p!NHVpoE-oUP(zWyZ4gEz)EuRrNcihsa(BKl)ct};W_bTyY6 z*oKYRX0S6Mliihi5TQ$=8(%$Cg9xv%729ig9y4iZd-WY(-waHCg;W`JA`9Sc_GaptKTXU?pR z$Ad#UeI>GFCZ%Gq1x|hSRMCrpK{Vj52Y5$r1GrdLy&J*B#)=gtt@*|uq``VeHL3X% zo?$bd8L!x0?KHV|+lC{oQUybgJ+ggcZ8%9y(F0&REJDe^XFvPw!$1Duk7qx-{KrQ> zd;ha5pMCV%@5vv}{P9Pheel^OIC?hF>-*Dir?dufX)U>$sA}$Wccpn-*}Wx~&kZb`AH$=Uv@GLg_Z7=vuc7MvTFPA%z#OxP^@5 z14atHn$t=y7rHdOQ^2c8fgsc(EpUW3_e9`z>)K`T%b5l5O-&MqJ7&LyBxcK88f0{J zU}By;sEjC+3fT_|Y)M47S!T-=TZKSyWTq@?WS?92EzTG`vl(owNG3XUaXCW4men}F zK#j&l@@+yZFwP30#9|AL^Pn}k%}oA(`eDPISz@xn>?P;ZXBW8Rvp_rE$R80W=4tfW zgj<)z$K}i*-%C2zDXY*$k@gZTn8JF1z@jf$*j{DOO0j<2Vz*9JnXH2yS}!j7PC5*s z16rqQq9*AgC0JY1xVZQL`SglxTr@sCO^0f7sb1rDeaDNs(~foOpZ==I$r7>X@kRd0 zFDmxZCy&7hi%SLgX~Vc+qO5wcOlqH=t~+FrJVf~yq{T;0FFR$ zzmplt7g+SH`b-TxRVgfAfFTY8Z^mmXQag>UC{+zJ#WFKovk&WfJr#IhYLpB+Wki52 z+T^H^ra?x*g0I26+htyh9{(AaE_Wt{RbjpWf|^x*mT>4aqn=bU0T8cyrf`1B1KNvG zCf~&j&L(_ju=gUQY%)qh>r`BkC~to7{+FkIedEkqH_rU0mjZtO#_8YQICE_S5Qhh+ z0tPBL%bvr#{TL2kTzKZIpI(6BeDT^ZzkK%mSMUBZvS>6FEUa8g0gaUPAdZ=7tcbOli0JCCyLL@Z2_K$X zBBX}PVy9Lr)fwTwr{VJBEvj3K>}8dThiblcMVC(d6kR%GKrD-mTj}IIUVwmzM(dV? z8T!3+kwrHXx&>r1{(wv-_%q*YT(c2AkPM7V z?kt%!2GZTAuswG8q zEiC8^*rB$y78V4NQ&NH5DT?w_a|&@SgJ=nbU~y;}Uchqnz#k8B2FE4Z2f>9A++>&h z{3mN?^84v1n`C@wgt}nRA{gcj7;v?D#^<9a5>{{4mRimSIp`gSlN(t3!H95_8s-*C z$2(rS;DhFe4-hX+@KN(aFX;d7@|op-Up@_=+WEE3&DUTc?>Ddd0{p9I^3&r}6U3W4 zHBC%W5=t#bJ?2pm`rrX;UdSr!GKXs*6IjOn(2?FfatG%(m0_|AQAfXQdlT*n!cLG` zX0d|geYX91pz~yZ^yS+TuU!Wy`yV*i7vq{#Q&;NS%O8`sxgeuosvcVoMfVtPv9O$(iR|e zcoB9=O0c+(xiOkpdqRwq#m31@{H%cY>qwi+2zG~gL1;m^4%rWLQztVu2|Bai*uJNz zYm!M?7H0IazwhzV5-({9U(=qR^q0DataZ;~-(E6MRu;Mf*K1~&IXE}hr3Sa}!~q`q zq=-RwGi*|e%+FW-t3>xLlhDzvWI{=^94 zEU%{rH9e~(5`$LBt$XagR@o`@LD#a~g4YgW3|eJsQ%gBsE1h)8ZnbbS!c}u~2P`mD za@hwwk~iBKDuUXF$AQdBct$5&0s9!%Ahq5lhrAnt5>k{M-$Aq&%ofMz3gXNBp)w^i zFtSm+PZ3*)B0oeEFq;(+dbvn_D_G0rd?lBwuR=acNhOnsZN~KND*nF=E@hQCiS6Dk zV_ne$k+IzkKGZdy>6j^2&My*E=c?OC=J(fbo_#B_WP8Uzoy8_EH4ub}M65{*+@TvWF74pV z0T?K}AUSEgOiD=bN($gK6wA@gm2oiS8s`MJ$;?Az4^8_d;F{ijv16dGDw`^7Ye&_Uux+k_#XUgvNwjIk$3g<&BlMSKfr9_mPY1{gvNe|Di|C z^#-zT{SKMB^2p4!a_;&Q@Dt7tshP9_&!9(dqMcm#s`X%8fBgDs_~fM)lImmL@Aw_u z__6%@OV>V!kIKqhD{mmE?5ps}w~5Dw^*h8|ef|2A-nV=gUcjCGHyfUzRd^yKXt$HT z<9=frT6y#O4?R%X$6E-q7d?>P0Yucr*^91cVDc^_jT%n=+3Qb`l`9^u?_++wvq8Fl z(|ms_qhxZO7Vw4}^PW*^Azz@=cymw37BFnv0`$$2x4SViFDzsAMfZk>1G9TNWVh!4 zsF>S9S{d0z85eSi68t)#4+db)lHCX^hxhGSFR(pRQY>)T&k?X>xFmTAW(V<(*4Qht zpeE<>2Wg3{PjcD@z|or^M`yj(xmUv2~Sm)lfI8aO6B2tyR=hikj{8_@lJz{pHG)uYPd8|Jq!?_|Y%Fcn$uX`{Ktxh&=sA%djqOFy4Iz zt1t7ehEKigx{?kgBkpZR`4N*AD+N(^XjSxNGxAYgw_b#lPew_k`xQ8HuWME6lhlFj z#Bq9;@4q9Cz&;l%@F=G4!5U#~Otno5?>LO4 za`{A3(Q~nDHo&x)Q1$Pn2_HN%J2tw3AwRo6V93vwDMwPe9bwDwM{k}?5N4IQ6+M}N z*1DDeuQ!!Zv}AH?>75+%}YE%+dtpC^aWAvS2eC&pk>^LLPhL=n$n zQ)SRux|j9dLe~4Kmv8>?{hRMZCz0!2);otim#IKw(7kn+AE7ok8KIT~Cwte%_2e_l1JyGZ$iz`Q)&THLZ0ni+k-J zTL|njZi6SwDfTECad_Z+((eJ1?UpBv0uHd&Lr*5Dtcaz4i2iLFnmvL{GA6gHu3crO z5ho&hh!=(F@32a?;|tz#NYQ|ypYAMYxcdVP>i{&WR8sgh%!*S+&?^Xy95#_A)!kPo zeuxzvDn$+JbG8xiJck{jiR`^RZ4mI>;Vv&o%41ZGcLjR%rD?qvA;ptaQi<3kEEwJq zE^H*!!$=+WdI+oBFyN&(H#>eqS-)x~uw6&YlQC*LV~CVvgP8()tmt@k>_~DPy>4I}LvqN1?Hu;*X0^`PLTG6X8^>J_^s&Z^jw@LUCz%#{NRM-8$`+Bo=+4pO zXx&o`+9oLmkW@xVBswKk3#<0N#XUS5+Bny^%L4#{h%c?4Cma-sv?zSQ zYnNX=TSCGvhZiZg2_r%}q9hxy%)^O(TaL43QKdEr8ukE#$KXRf0uQ>7prdNc*TWkN z+Wn|9i(yWtF2{u-dfh@XQMO+(6<-DN#{?U{Bg#%aLHeuhcbvW&ogzbXE z9wVk+sN3rxC?j(z!4R(8si!B#4i2=2%`YdHDSB?dj;8rJO8LQ;>zm8|0H`IkSjU7PJd~|DJSe&w5Fr>1mb8 zR9TSylpONrYgKNlnpV_AMp1K`qNdxTYp>Ms%QUB`TWWBcl472D%7>a@fX??JA;zRA zQv`@SL`!ut&_ZU2?#u0!3K=nBGEBptuS)9O6IMxmGqI!+N;=h<$dAG`)_eu|!aS(Z zE6qPNFZZ8RnlCj!Tz&#Z@L}_v<)>f-&1=mI%cqdl3rC0lN6#uwa+ptMl47mnU*rgY60HZAa!w7TE2t2uD%$1_ncx`p9!@~Bs zZ~AO4kDHBa#WDirlgUmjWlN$_%+&N8*Te|fC0h2ll9&B}Gt+%`5L zltfa==$&5Iy;9IEfEmNgP$|pYa;wA^RoGIQ6$5|27X_gxUtLl4&~d6g`}q8z?aIOL zF5K!XUC*6RLXU~mWaE5|ws-~*nZG8iB3KXJOS*tiL6XgrL4srVLR|!ef*6qww~%^a zU_o%%O2|wcn49a?{l8~?IuPGS1hyiJ6y6V~VIT2N=Ai?A96`mSnz@FZrGk&n~Q#RpkuZzEU&9<4dNlIXi501>57)}18hYY7Xt7MX$E?lHU^$W{S!E=La57*&kOgDHw_pS8H z2FL&p4F?Rvhboe`y70FH7$E8@`X%_{aD3rb9_rtRG$ix<_vQznHe$u<5pYrkTVGeQ zKHOE!x01B*D@juoEj}^LDjSW%ie-xzmwL!Xb4P+Flfq*4eUe7+k2we)y+1C}-swVb zbO_-Vs1RQYTTTo*tb~Pbk38b2Cl?quzyJLfU_T-`l$lw>8q_ie>yn*d9uc7>sO!s`q7_6mecB}2iL`TeK64>L84JyDJ7?M+9gpy)n}+> zlcL!;kM5%*Hvk6RQLDwnx%08Xv|i_joiYr=0lS?Ip|Bigt@`Q_ql?_Gdz5&$5>wg+ zbyV7>F0PhVQh~TuXlr1mEF_G88DwgbcsxHlT`d2w8-`kBrs_H|%=jKR!k}=nN;1GP zvD^4cb4y+S%d*z@hdI0M_^iI+(LcV7-2%t3s*1!^zw+TZl zvJ|iulUrmOu5W##XtSRJlNXDa;YUjM?d0ux&Q!Q^PMm+eOSC8S{KKS>Jb~-PS|qq} zenAvl@o?M4NcZ}&-l^8E9(0`$*TJ&m5mKD6 z`v)M5qZGIA=a%7uF#`tKfq4Qh0WpQBpsglsX4bLGQg~*yWG08G?;R5)lbtP=j6G9r~AgHynMTt$Zg{BNwgOSfpIM^w<`m107owi?HbY-G1h(X^WJgD#}1zT zTx>{Lc`9 zbVXOvfbUG~?aY6D$7LYa)m>pN#F0=HHLWSCuBh46mfY-ALP-RYqW6;CTWAKQwK6sW z72%=9+J0UXQp5n0?NQP@g#Z@2Ja=Q!1zvPbD!0|HO{EllH=-&#ZUHu3A?N3}kChdt zKdqz#aA2Xkq>;kL#HdtFiV}02e&nsuQ@P0Vk65}nC8_8>GL5L9uQ>Twv(Z^)5v`SP z`QNdHZ}~LV*fgJCezc=SBv^fMnX4=*^g5dlZCW&~h%5&TKKftAm-4+ERGeq1aN&A9h(!c{A^j87IJV1!s1a zN^6iggZrplElP?ek|~OaG|VKv%v5xwBb0}#THqCbyO9L^Y>)4bxgeArI2BEPIN;L3@TwaH~T0 zNrmj^rLw7z5n8IjU3Yq>ZLQ$4Ey1-j`Tca1O)@?-LJ?CBZV76$SgFl3J|BJCT)mmE znxid2Eu;ih?6gS4Pqp0(bp$rXDamX^O{b*Dz&kam1>55G1Uf*vDmW>faqf+)`^Sa* zu#20YJ!rz;31+pF<+`qDj(jGFOmUpK7PB{^u-@7A`THB^&Bp66j)7|3DGO7SIO`UB zOlzH?$0=$L8f^Rt1(-m2?J_p_21kv28O(4H+a>vExPy$Sibq6&NPHi_qUD z4Y_H3_xGwuGes=Av{)zZr`PIEMJ!riV-79a&L_X94RJ4?Unw4szN+9EkXvX}N8#v7 zHqn_6kFw$IV|KC3i)9Cw4Vh&poE>kDbcC*oWmDKTIY<#VbuqFGhS7FeL=Xbs&eb*# zL$)-dDAJ0qZzY-HG5&0nLm3MwuT@6YVMz;QtjZ+<^z4Xq%9OASc%qn;^)~2CbhaGb zLk`2Bl;%SYYiqFZ3M{Lr0f4_aEi3sPDAly0A>Cmn-46kh+ugY`!`PxQ=lv_kXY?gP zTYYLmNoQkI=~;rC*YpgcNGY1yS#5Q%%sLPN zo_+AkAHDnaqfdSD*qeX7`orj^K#vNoRqXA_B|34PL)*+=C>Hd<(kH(#mN*?UxINBK zFnO{jYLYI3N6M{A^aKgYma%gX$F~ z>}dOI{{QT~>vI!Vwm1B*?BlBEg`9)i>Wi-Dne&uvbH~ABn{W~i_0*QMEp625*7U{3 z#CZ!aB!rO5B;-0|2njP_5)#ZM0Rp6IoK&4pGx8_flTT+H8}d$0oqyq7t7Thlwe0B@ z0$bqBVE09mwfE|^_qzUm3$Qecz~Z{f5szObWqk1=Tr_Y2>9nnMj`XmiohqYj1eQ@+ zPtzh>0o9(5Uk;Be@{Ds4Bdt&Cum$R>){Q1N4Ki&KI z=TGi5gXuRj^2W@_9r*5UJ7fM*!r|Ecd}id+nUU9mhi!GO6}O9uxWb`IvfnxODK1G6 zmFi*(-}%Y`QII2~6_&g1#J1gC!`skY8Y+M{w|_rLkPc*%a7el_Cn%Po9q`qNMa`+% zY|GF?XV53qlRyq3Atme6>M|SAGkTnDk4PUb@PCLta?~~ znTe$d>o&S`T;)a5P7hl3Ao|k5JR4%J+L=~KTqa#Fu8^=7k?trSt5%rxRV~rZa))RJ zrb}+bau##3QI=19sz_XjE7d;mvMSQj$5fIW-~-k9qHZuX;jdu@40uymZ%Wd(>M)5d zQ!Kd46M5Zk8q_I5SOc-v@^EToiu#h(nIhMeEF_7r_XYk%B1>uq#dt+T&apoU$s*{_ zI%&K|e|FhvPGMU#47(u;6mzpA#(Xf$OY_9UTJ5Y$S(W6`7DjQ?%n)RuNrO|GrdozO zEFo=G18$>6I>!f?7+8!QOy@3@0#R1>rKs;jh8z>*ryabth|RvH&NFQFx@$!2H;mWc zZ@6z?8AA%cz^`qZf$uZyKH0yY^ieZUHWw&Yv6S6zTxsQ_Nng}L$XXas+tOhK4AHr! zr6RMyMXw@uriEhFR`=7Hn0=13LDFyN1p?oN7C5d7t?dTSfIVa()(B70RwbRoM%c8W zn^;x9df8hgEXbaY@+W`M+iftqYDn%l?NLFg9~T3Rg=Z-@O(%j#m`0w}2UsBmzNr|G zNzGW-xMTo<1N4C|aO55ej3@%3LmIP%L! zV`WnLzaKvq+)!^Ay9)?791>#6!!Rr*VTw6Rl_}FUYv9^~^HF3@E!#56*XebswlbqY zwMWrR8eBv@9j#sPLKXen;M#|XnczQx5{6uOH7?nRzU_#xEW~9GmAv1A2&}>5DV6J4 zgZ6_pyY^A%<{hOW)AD*xcrnqNqpDtLOBr@X(~EWVyq7<~9kg>b%*1_H-i0GIc&V8g zNNHG_WiZm!(A>l0Nvi5<85Dmr$RwlV(E%HtGqTY^DB(2tiq@kKQf-iHnL#X+P~BaG zVkjDMmo@nIn~)zS?Axkn>3~N=rQTkNUfUSp19bT;(b~2pl3G^PTjB2&kUM)Lj4*50 zY?6poN`>YJ@f_1K^bS(f;`!jY@13v?s!J?zR}qlUFgFN962j8chv2a^+Jp0FtzN~d zgiS;7aCPfD_3H;seK!5_)b**)$kc`DV^cS#k4#;j8YNQ~H@x`b^hr1#n;LuRCGz4h zNvqT}fAt?NTyp#6lR5gB^EnqwNjO5wFFUFzxA)AU)QWw zlOJLU5)J*Ye!^$xnnvZz724VqzQuCk9tL;=%^5kU;QTm)B~-fx&*LXw|Yv|1*< zi`-{Vyg&QO?b)|Z{B-xs+?~_2@0`1T?A+|h(Yg2T{B--}2NynjFnaa=>9Y?GA16P3 z_vPH>&+cD-`N5_0Yksp`D!cn{ymtS0zni`HO<-->G34?U0tbR6?9d5`)!VA?r=nuA(GRnohi7`6&_>jGFwcNs$= zk=)xp4F8%AuOt?VM#(mPNUZ~(#(>F!ZN4??BG#(IE0>%WAMxpH@#)D8!A7mO=7x$| zt>y1AMT;fm2~s>|fBr*~fBfQ?yQlwqJ3n$3muU-G; z>(O66IYWN=`t<*IrE#If$D<#=ocZLJtCz{fUp_wd%WEG8cf+^Q%wQ}gx(YL6m7=DY z&zdHXP!u$z?u=y(2&dozblq@g<%utQEUiM_*Gi}b0cCgh8)U2n-vx^(p zth$rghAW;@5vYH8#sxW6H*PjqecQ{P2JA$8GYC4-#|lLie60+^CMU1}I;DP>DP(dS_pdyCA_3yA2WYY1A+ z%_3|Xfu-9JOdpVMSdcxq7whtEjC_rZt=6-WHcGY;Arh_j`hwIxb}{C#2B`Dw5*Xmd zUgiwx8!GB+eub@yv6|p?b<;HO=V`JTln;cICLOvC@}l$Gc)$WjYN)yiUtmabx|GQ%)7Fci zf?CGzcR`%AN^-PzdyyPnNkp;;&fa6d1C55F&5sA#o6ZelQh5y`fr2CzL5x1-bYMkz z1`P-8akWxRa#)CalHyOD6kz*U%{-qR6C{s;;M<=bKTK3gOq*MdbBv5DH01zIG|(k} zEQKVPRLLTlWbi$P+7>l36*&avhmo+ZJ|@)-P*RRUfjNWiYE8DmchEvnx4?u7PNcN= zK>nbM8B!4|yq(Umzo}P5A?hjk)%=EV`?#8-d351aPywc83^bPM0NJ;%-LUfv_ByPq zIG}5ZSOhkmuplB+*|7U$|9-M@-@e`=1M5^xbm&;bn6e#XyKTvvQaoeO?Ed}F5cqS% z>DH-0+)$vy_g5^J1I{&JS)Q&xN) zWFZ!Nm?G2#utCyAjl6!S847|PHOgRjj=Q%hS4=SC+d)F6KMx|;lW#3_(<0JaWw>5! z7;$S|sV*fZh)*GLYj;kimZe)cqha+}bxnUbB#552;wOLlyQm2RT-Aw}YM5s0pa_ZQ zbprNBI#;A>9Y%94m?Sp_S=28G83-sW$l}xP%k6=Id@Os@uxp+tbXU-~#BR2N$Pj!D14J^Sb66P z^8K|7y}ZY?$IlsD-{`a)E+T>~UQ9g4(`Y!SscF)-K4w~VAi20d!=*tC6F9r{T3-OW@Ip1or15CXBsz6*!fm< zp$uLwPBxnJ#FRyv&oe8yDYsLFMJew>%$DRzaM?Nbr*onx#Omi5+JgvP#9#y5sn(3d zLXmJh9*H66gOfo>dou_K&I%(e=;m>oC;vpD#@~v~Q38@8NUE{0Rg&|`XMp%COe50iiAX_^?Q%q}E zlAw+-kc{0Ha{b_!Ld__vk}ZC7sBHog?iOX3cu(rNwou?5PZ( zfuvEGh2b7n)D4R;!_bXJE@+P?`mh3XM~!g@7BNoD67edcD_QPaya*^$z0CFdt;?Sp zo4SZi#|Z*IZ(wWj)MYYtY3lZmr+z#&H8y>G>KgtgQ(sM=fXf%~GWH<90(;wEN1Ycq z&B*Zc6?hT8@(Vb>ICY)Ct?TfD6I0*dtqb@u_}a&1a5gCBG(A#tke^Q7hN-&h9%Mc3 z*q%qA4w6I}`)^|5>OimW){LKzyfOR6w=+k+y#MNnnIjjOP z2XzQzwiT zh-nj;7kIYG*6!A@(~W5B18|)k;>`d{?M^0J^Yc+V)x2;wvE4me!FN8G|BSBExQiko z;!3kQ$37o`m>_#l6nq=Fy*#U-aXWvd?6y0o!I4lA?AD>-`QPB5w}PI-0|W9e@N#n| zJR!)^1Gat+o|gaP2%ZftbfEr6%Nked#b|>&Jh~8o=%g8D=)vvUUaqmFZcC^lSLfKD zT;-4;dN|V8t2`2&x@ASpLWOeTQ%RMF#stsXMl^zMFb>)1Bpr0Y{SWRoh8r zS%`ZGqt~Tczf`vy%!I{|X2OYyYW-z2nR%=4rI(s& zk$R3B*Z@n41|L8#NMqKZZsdbo^w_!0+IROoWHd{Wg-E0yCcnVA3f2&Z*Jf-3gwN1AqTt`atmKXH^ZgdKp*!vA z@KFZuxBjVFI09Bw;iPTmZLPnCFWs_jbIVd0w`^mk=|a_xivrkgk|+vd+>K2rxq7zB zCqCsJ6BJL0(Yjlm=%Sj2Q5-6+<@f=%_jyEu<)e>cA*-X;3s!NR^cgT3 zUIU7!rTvBK<*Qe>tsLnag1=c=#9~naqj_;b6qm>(D%X}uR9;aoVId^d-FUZdj(EiU zEK^PJkAGv+jDZXgs~4sSM;@}SzunGca$qbsgV|r@CFdv@Fu|9*P)JO~V{$?YCzS*o ziV3ANxh2t_+#E}UqcNAwAW4-i$vO5Z6Hr^JZXB#8zV?Wb8OMdGYYy$h7jSe7 zE3R%$eT7^M4gOglu3=O+a%C*DOFFQ;V*Bx>P z=dG9EbH18Bo>=n6g_j{{$<*bkOVh7nJ=2#{V`Tbec+Zy(D~Ll;ac$}wIDrp#e-@nF zfNy(oc;TSgS_oj27jUx692kF?y5`L8tuhnFJTb=fNx1a2LyQsZ*@6$hfdyhHfn!)M zb{lKQrjJ*iHW!1r?`KV}!Q$Y10p7*6Jchn0Hxh*O!Q_*q3V%w6HLmsF7BA=pENURF zAn(Bh(-M2T6Fs|nhM(7Mq@gA`rWXIH*+jiPMS69008ARP4Z&=JgGu)3#T3=3=4$N; zTzv!r3=44&i((KgDbmZ*x|SixpO$a74YdQm6*CsUd%)taK3#GfF=EAr5q+94iqj1j zsuqChceO5^oj6>&F)=c6q;v;<Do%bOp3!*8w|Y<|+C9b+DV8O+!i-CPqr% zmaYUAOP5MByq~N3A-(@04V}0U5`CMOElK!P$}_Cm&ECS*0zncx3;ybSH%@JbJGsZ> za68O|R9z!%LfbVH21s_>wE~8{$=V*bdkFI~60tH)5M^1Al(<_+wc!k6E5I2j*v=+- z;3S^>++`QbQZo;`r&)(@)~NKQf~+!<#{_utXTLi~&GafZGQq?vg5n9s`YzSZ7vDo7rTvt=2~TG|a= z&#OaP_3kit^6-Om@64WfdG7abEwe4`>Gf<2do&JTk@Q-$+!_Apw}x;?Trom^_x-t# zul(to2iI@S-uq_uwYO_*5YIkey+Ob?BF~p|`U^>=Seal>i3?G8K(=-`c7VDEf$*{+ixPQE*Ndh)Hwv)Hfy!Q{nq-#^*t1d7fEkJ>%2(*g1?1|CYuP!|J!*{c1ZwI%_jG@9%3XBT(EX_+HCRgk;=h&wlqC!Ne z4#ck4GbFklGJ!NAQ`A8YSF51GMGRa%@hPCV5cYH^n*%-tl-CV|XBh&a(8jqgkH;|B zU3hN%yrE{8MmAbJW9G4nK3MTuGg2|#F$<_4?hV4J4RQhb< zaA~Y`rF3@_DP1UCfwONXPL;mFzNgaN(!J7Exb`NQI9j?H*F-NL1}OE*im;oo=g zg|C-xPQ2-gjCt-QN*@u7mQ9RIyfSeDuHJ>KZ?uULnK*)H6GtcBER9VZh8e)A4=m?o ziVx{SF0^zhB)bY|eJdQsdp!J_rr({Z+hDd?IKT|1=|rO#;G{p?QxEU~o^E7#^Ret# z_|-K&kPjZZ+1){Uc(LfV0;LK?5gz&!Mz{u6@sl68>9x38)b#9-)*uMnx`H=yR0w&3 z2g|||w)9e!aTu_5(X~{Ood#ph8lW6!5*9h;EP+w&F0;T=S+!4NTWwc95IF{n;%BLmp^5eS>Jnn9qF-KJd!Oi8!0Q|xo$8OUy; zW#1A|gm|cKIEyFOo$aCqrzvJwDzBpfs+K$JN=KAWdZ9)JOsX1JX~DJO zxEgL{za6)VL#Gpe`f3Y$(>YL}T%K#I*qE+hGfOyL9U$;uNQ{-bm#S-YH-k*toJ!V9 zQ#Y^x^~Y1wM;6M0Z#m_!$EHSy-LIHFS>qLCqZE0j$q*YIoW|n_YZ+LmFXWeqGhRsx z#eJuR!V$2bt6d!5^&+s`^{@iZn?uVKDRl4h1Qxn=71okR5LgI4_kDVS^bYEEkhY7_ zm;la)7!CY{8eJq%$)5qb~m*s2IFB8zF6Vvymt~QwpO&^~= zIrRnBz`MpkV2wC2{qoe+;QLfH%4ut)X6zucyiCK`4x_9?PBl=-eV9tuvnp7A+$1Un zCt)Em>eve#l28w0M~!#o^l#Ing!k>+Q?`S*7|cxTaIMYcnnl4#2DhFQ@E!BRo-0kJ zok}=DHfFd<3n`wpX>bp0adO<@yk#13S1A1oBMFliq0i0-$S1yBx;Akb^M(`e5v;2( zeOLG|3B`$#iKC@^6R(u+I;pn1PI7K-Tl`g}!#ta_$p1sSXqKn3r%-1md2ajm zzyd41J@MYes}m!oyANfYgq=Q=cy*oFy4#$1p}=e;lkH?(G|lZgAoyxI(2@&W(K+!c zq?jOj06Lz0cWen#o5|DlG@&;NsV$66DUwvZw59z9!n=R?kY~VTAjKa(WI31*a)9M@ z=SB@NlxG_sd*GdPZ3C%7rx#n%&6DhlEp`xA{A5+bqnf-i5u zmu{MNA-Gs-mU_9DF29^RVdUJA#pk) zH(!15=GgtS*X|#^{PU}0bAPxzd;Y@g`FCgU-1zy`V?Q5x=l(Yz1s2k7eaKFMbsmQ~ zcX`Lb8AHO@(GvI2MFfdZbPkz$+dM!M$*jhic_>fGR^!l6isc>rYkd%L&-3X6Yd^`c zFcLC9_94@9vAxu~n@DD8Avh*>HG_fVf;yM{LY;%?B7ze400x@G1c~q_?k_B!gceYL4m_9Of8QbhHO&^~c!V4-Jqi>6dYicBx6Sm8jN zZ$S*!kF}C%M*4Mw?Ay1E8?-=raxgxV^soV@nvbk%@=Dw*XAbDg=`1# z|6(yXO63bYdx&SHkc-Ywbk(Maerr?02`MBf9-fC~tCw54nNet|jyHTTal{EK9L9Ry zWkT#d-uCbwe5lt(Z+PjfgFtX+;)qk6e1jk)LFszwYPm|;E1#C`%{?tk1?0>P*=bM| z=e(R0>;{B?GFS)h>b;0dRuTWQ5`~Bd9mlt}?c2AvJCWSnJN&no-`c;w2VQTmG&W-5 z93bZgXxR|evYKvEt=USFyDWjsfBROG2G5lAfEkiwS$oK3O}KbZeS*q>h-5+XBuSpO zFavy2&Q__lnxO`4B=eN(Hv*YFnHgY))nstwecYnj7On;_8*dB%c^DhY!k!>$5KCe6 zLF}2Qs*_wm)K7!8&NF6mvs}7bWN0c>szK(7CBd6=tcc8_9o4o=&$i)LBs*-_l9m1Y zNf~T@-@a}`$9mXAa&P-E{A)Jk85nX7+qxLpq8|({Y@?7eY}2KCv5Kr?zGaOZL&|zU z++(%xUbyTKX~PKrjNZkyd_?@SnJhU7?f5Y~6U8kl*<98>R3>DzL%R_R8Q+2M|{ z_o`#?-E}(aoc!3;(!J7mV89-kI2C-`Dh>?{@O<87J+#Wvnz=tYx}t0pL5bE)7W9Oe z51=f1ni|E`G#_L%AD%v@?QT7TJdAAhO83IpUKaLLEch;U70F(fLKA^x+4y-a1GZX| zEl<6StIoUY*d)vxSZl6DC|(*vfAy7RLIc}&dO`!+bTGzt>Q4FY>PH8D{_xuD>+j5c7#x{FLZrATEG2>% zarLkShs_To)lxLPZa_b3S?^#}V@R^Hi!rjXsOu^zs&t4M!7G9qWKAoQ5SFXealJ12p@B&Pj z26y0C)8S#pSM@R@%Q)Fd4Qi|Uz~U|VRz(=G4lmiTVZ;2Zp5=L6rEGwI|N7!P;O_rU z8UF^qO6LSY_|vz|6=|#*f8YnFA{19dC8AUwp7M#$$0sZ(p3;EjQC@fHT1H=r;`cOMK4aeLCW z{O*>`&u*;_fr<`OV zELHcWu6rhp*2|=-aucSahq26R`66M&QK%NR@6donvso0yi!WN@@Brf~n5Bc{n^()e{i9W(y8*GTT|{xno^OL z;$K=3A?{%(Utd5Iy;L(A)6O4tqc=wVqBn+-{44B%{SSf&v(>PRI#K>~yZP*k9%p`G zweOxU^0Lq)B`PHydcoElw3z3p%Ct16b>PMH*;ro{cNyuD3-8aTgd*5C7Yo%LZZlXx z9{@LK-@f^cYAEr%Zui+p!MlI|zs?uAx$Y(l%s?Cl;(FQQ#)<9eA%L=2GGI}Gv@_ZeP}`{D zE+_T7VMVa@G3GHZ9xfWTsAJ=o!D`g|4Vr-?6IYyGR)7VDz;0kUVX@6HgX3eA6v3dR znE^db(|r7zHoS{w4PKz2k*gc4@{!G}jCDy#NbT{f@M{;$nKL-Tw1o)>`~-ATjhmTh zxbg%q_eKFQi(qpFCAhXjFGRk@{O+c5rs zMLQ(K8)vLOswH}l>A>18;IzS+(}F<^J)P&Q&4eX4RY(0$57qLpb){TryRIS?za=UW zY}Sl;Hs7Zy1zbdpN`qI_TQn&CrglF;#OtkwK7JN{qmuxi9y~bX$q~EY;r)t7L<+2@ zkF=`WWOhc!pqQ31R%}|fb7*^YL(LGFFvNbbH4rOi)P&{=H14)7hbggFw~ci0Dx}@L zq}*DfyBxMirI{k)vza1-y-BhMoyfmT=6!2)ws{j0U^%c~1q_dFH0*3@h}fDF33sYd zNJ=*}BF3X)9d#)%b*8J|*~Jj!mmmgYBMFM9O7F?vgO*gPS~;F#2%p)Q9jUH$O9~63 zSRLc_Z7MrEQZjGZ2%5>60ZO0F4LbCMMa-7u5g0h%R69zCu9;w@;kuh4-R;WrfrYcv z$SjGyyHK1&-|<5!BE;ot8+kq4l-O$Xt+q+FGzd*sab+w;{~JqOkfZBF2#r&^zu!6q z10w(Z;o-nyZ5$Iq#GzvHrd9kJ49|nT;|d-TYAs1QCkGqvW>!)0p>k?p$zQXvB*MnWq8>;`1V03Mu#CSSjN0jCfIlAqY5)MDj12wC)N^1 zhQ&+XX=y~TNsL5YUY&E|Z$Xe{A?l$We8|uv9Amiwp~WJjhKU{4DMly(F|=xnum=XF z^dGN$A88nI*@wCj@~wmWA%d+uArHJu+Aq9IM>&~*^(Ww@X6!WP1m^RV#qm#^gac+7 z5==|<78w$!Bo=z>+mI&EDM|^4qoJ5AN2TE|3O#IjlN4j8u zXNuJ29g0fCrMJ$BPuWHUB~tAnu3K4XWvME)3#^0Y8`NzYcc=8?zEgTmGg0-H?At$) z7DD-E6UXwg9>MXaV8svKXy6IhixJlr=T{7hD1_?|Ts|T0xy_sgg0h$OdBE{`2B`R zk{j446g=8Ez|&0AsL5SmRZ_TO$vMY9btfvsJ=?u+kj0{2H0a>k$4{*0{G=K5Mqfz2 zKnBSR6ojDvaL2A=AIZA05?gD-6Rdzv5J`kXHDX#pltc=JqHUoV`LB_;|7+y4{~9^- zUnB4Q*T@IU^aXF-+)^c!maR-TvTFo0Y)c2(N%J{oani?BuxSH}M2-rQED26`ksNj} zVxibF(T~`Q@{xqNQr$4&TRywW3ssSFBS8o@O@SXnw&@l)bFj3LZid7F&OkB0y(Ve> zw`FdSReU6ADm5jl ztSC=Kh=_H|6Wfr``Z1Pay;;kP-WJ_pJgbpi&J2Sb(uU0)NUp7h?(`xC%t`Lp-R0J= z+OP=X3a|(uji?azV3GQUVcxfIAv)k-ck+~tfkLxQ6v074l)#uEE5l39_V35QP$gux z^rb)D`RKvu*#7;`keMT&&m285bM!T`y)zMx#H3Ix5+;cC{3=|1d*8loiJs?%XWx8p z=IF_}uTRVzd4KGqwkQp_pZ#0 zd<%n~IdX0G{3SS;d;bq}XTErF>D~LM&jx;AdTF6BM0WR+cGY%)t4kH*Qu1AS9FEs0 zC4cgLuRh&`8G(m=6)Se4iI5~jLhE@rZ?|n0<^neFG%aALs39RDS#dYR_{_#du`=J= zrsNxt3;}4*O&pnct@JHow>IBiS2|m|SGqfK1b>#sN;fx=(pwXUCyq{xOa#xSl|Cw6 zDcvbunK%r;uanXR2c2y}fugQVcGPDoJBs`^;fF$=*`t|GDN$w~YoIBbqH9ozcZqS* zN5M+&!$^_?n-RQSBB^DDNGF&wDs#cb$*usn>^J@mqKVdkYVwcecM%?vG0+=$^2o$U z={lT#JMmkG5NT}-r%Hp~0{uDa=+C{<)zXzs2uXdobQ9tIK$#q%!rP!=SKz|+GBXz5 z3M{&14OMccrBGWXRznfc34M*kT4VeQPc$mWB( z#nS{S%d@)bIG`|hfepLd5G|NCW)j@aS#3ZrWd%g8D3-y8JlH+{Ic6_7)Usn!U%=5V zq{6&4_0{xAhZz&8XOZx7>PEt0puI5t3jBuaZDJH00B{2kVpW!gm!6g#hUu55kHTxO zFYwM*2yCm~yzoNwA*L=*U7CKCOnnM39V62(!@Itm8YK=9E`HH(;G~KUx1Hwmx_xMP z{&%za_fvnEy5>ykEu8e}lQ`Njn6i_|OZo*Kkd4zvoO{QE*I2>~-as}}6u~j1(zxx= z0Uvj*9+kDj@5&)!Ci`JF;?5zO0TtS8nslfsHsM-*N*XtF6qW%sr(1QjVR+xah~-n8 zKvi0SK^0^Y6GGAI9rz*YFZWWzvbC%U=N6pp?%BC(dv707M$2R~2ce9g&$7qmtk>;~ zbc1a~Rac6~IZyPmEK{|%EqdM#o|NulhFV2E~V z%>c9F1}S}4x<|lhx>p)QmQ8__&cN~4r5h6`@G`vSnqzvvAzryRag;c3{|3AihcV*{ z{6oeJxB@Q*%LHt&YjEYL^WlyOf3XsdtC#rkRw=SIKZO&(v> z&(pM_?FWx#d)z|pgQRzSG(}B|*U@L_1kf^ciMi~umVHluM2LB^<-WJPJ;h9!$RWcl z%8@aWgW-2@o>Qn2S>443xoPG(`~jmaX{2*}fc3H>!LPG_zd$y&@9uj{)s}THxiz-R z4%KF`Vv_CKx3_gJ%)oXHOaNxK`5S#;xx4;mMf8d*-K+Rb_ePO3BBJ<4H}vQpQ&mJ3 z(uDGVYu?zYf=RJcfJrphZ{vTRRPAX52yx4l)@2Ts&42~HGCk;9SdWQ1@2<7f4O@pBFdC?g=D zUfnJlbO0{iB?BtWH-l_CIg)=eVlJ?MVuIRUlBlS0KM@NHQYb25_(GKGNdE4qNG|50 z^@{oB^@<`Xfh<{;3 zW97kFyp-s3JY<6)QDiSOXfek%2Hysj)Zjknw(OytYatHHHCN)WyyBx%tJ%!X`;F0G z2d8ihyrnnL`vaHgE;@|g&1ZP-fYWpokJh~AyK@aLpU`zR0b5Q7Et0MHuE3Jo!c5cM za2}juA&N!PvL`bC4ZEn7&N<6oaCP$*8rG$3;gBH59tOfW3BEQx#0$1X zklmmR+PwA&(U%HBG%d1FdDuY*sA>!mL@RZM8Cc_NSXEscQbpZ->snX>%b{-kUPOsy zde)%*9N`UBV#3DM3{FDT?qr%UR2>XjkcnX-BT4XWsROq-PBlHExYj_|3g}-Z*uGO8 zlX~)Pgq|EtW%LYI^VLC?)3|cJa9D`SbwdJSTtn5Z=2HUP6SwK96t@!2{@6|}OJxX` zj?n2__y8kGqrmI1o4eQ{o^_YFx{y143k9)G>h7>krk(Vsb%-vsRu^>5#+xSLT3Xdj z-6ZgGLP>ka_FYXTxm~Kmv$SMXaMD3xS*wc)_SZHq6m6arT?p4PS4_kIN){KwO5MC- zO}mmU@pcATiWweMs~Khtj^EJ`EVuy`HDaW8t_iNMD7u&s^0<$_ z4FG43V5^5O1v}-7FLuBk9Y#d0mtG>ju^C~Ss?zZbl+Z#79RGCrej>e4@!F+>2q;(0 z$L0CgEzSknA}>5Ur^%@1LVX??+^!2u5i_alukV+cGmE-l^M9ze~ekib*Vo^a1W5PxfB+&)HDT)=N z+d1|pFHsbt9%z+E9g2IhLT@{}c0adDLAMyfct>OFM3COrG?g8U))5S?)shwoyIRsB zzL!EVDkzbMNj@Yk7)ypCnQrq%8`cN|B#0p{$nkY}FRzAxXq+;>M(9Y)Rm)O7@h2P! zwjVE9k6#_>VFO_9A?>hE7GUy7BoL!-GH2S6V$+>VG;;v&b`622L&-)B zzVI``JZ59x-r>Hz&DT9YKlbuu1TUz`8q72wO0y*y9-eAf+kagzJp7p@k4;3DWAciR zjR?A zEh^XQ;0y*g^EsEz8*>@GG5^m7npV|GP2Uu!W-1_(JYuq)$jQg-0FXHb>@!< zako55C$2I-$~x>mTg#Tg?-s!z6j#*2Iq|2!a!3e!XxS~l&krbY7gyEsFUc0$O6e-; zqxtX03g3@4pF(dz(sC^`{+UiTilCV!qkn(aE~qx?cZQa>gZB+@qj{1l8-l{h5*;nLlw>~umV1-8&-}a%Y60CXS8)K z(6ekW6m>X67M8yqeuwnvSsMno(5W9xLOXPm6>}<84NyghShF>HR ziYG#m#1;eFa(O!ce>-#)!y{|mZR#wK)0JIcxKbGlamEr@#9P2@F~PzwV2(TO_M=2s zQ?=-`tvcplX?w)$A3O?inJ!f$GHr$(u?-=I;K~ie8~X9o^pOP^s9RGP;6=x#Mu*+6 zm_AwK6^J(T%$l#IsDg9)3ZW!GY%=E)+R~<>19QW6;jNo4`t#x zj>#jCPu}eTL|ybyQU97K3yKG2A`tv20^1Fv@@L{UY2AsEoji8m_}_1;UAK3as0glh=b^G2IUCXV3A#L|3wTzIlFj?6sd?J@)gFcMxU!^tZDk zZ|>hebM);8-`$-(`N@20u*zQj>F#&;PhV_Y8tD<4turI<&Kx~AbM*Yo$mbQ=yRRl@ zPrX0;<*T!2-}w2$aSXQx52e_a-O=IBBm^cP28qT5Nva!|3DkBwOt+q~(@v|}#*h$t z1{d3Z5!bwub|W*dRS{14Jds4}oLPjX(>CL38?L}zyLWfAVIGue!5Krg(Tu^VX1j8M zV#sCeg#0hDaY2bz2fTc1AIR##xtAYYzVh>jpTf`Ccg{7#Qo7VUnfu_4xvSsKp1&}A z`H!$>{B-;F{Z~hSx_S7)A1?%!%iMc!%)b6zMJP%|eH5SS6GvS1uvEPu28rU2pD~d8 znv(%WdW%T9Y+$!_(@CbC{cx7&E&9V*vauYuC)RvQ;^D*hQmE3I;~e`Gjv^>>-D}&9 z4U~hYqG{u!MBShpJj*pkGBzpb7U|}P4pH(On-c>(8OmY3Fim!d^sBm!s|Pa-9p0EA zWEXSZP;}lfz%daApRtu1c7b60ifhFAobgfTV{G^vCbiQH(|6YRCCfP|>fmp~`dXmr z@lgZ5P=;zI0iWB%hE$#7=bKn0XJWaHNf<&xS@6+C8{93p2iK`J|1L`46C6Sij6?7= z6Yxteyl4}Alk{q1NCgfCI`RnooNd>$gG|$1tblSP>WY8+Oyv0&QeS;n6smm9OkgEtQ8sHE`Ba z+@Qf@3P_E%wEv-e3(dtr{s1UBdBL*ad)qM%F$Kg%j8hTO&#mYYe&x%DuFM>w|7vQrjV%miQ zHIVVs0an{e;lax~aIuf0G`L|w^ToQ?ic~A5urU$gj|DL%3z8f>h|>=HV}`k+dD34C5H-E;8lzJ@c5mfqS#N~cSAFbeE& z(%f=0(%f*~aC71nukc#BcJFOjD!t0Q%}dZ>4#jF_DGF-pUOj8GO^PefO~$b%U-np% z{eFFc=@UPmLIMr=d41U;{Hfc>Z-Tidq}(9WFHfJC8k;^jyv+6L!u)05esSvZ)aMS5 z$wegFa9B)!JT?6?(yHA4@f3cuA5W3Lj>63a1WXE2G(ZK07mwhtVM@#)|2R&jzH-uG zuar4Zz9LA3^4G6mgkL!?a_(H8`f9DmbC*7&e;tMSdWCG9eg%gxb#3~U;P!1#fttA- z&A6P;RykJNbAjtCK5=0o8n523pM3MSBgeR^tLy-?HMW{It`rd>9>B}GOTjm6*wB$P zhD=kpRZ4bJ%cKKfF17Y@9SkS9sN3)ovXP;+lr>ly6z;_rorWtm%+%pOU--+Bza0O| z$e+La%j@UaQ!q}g${^iJ@Z~gg;)<1vt=Uaby6;8hU%i;OgefO8+ zFTM24CZdgxV)%hX3Dw8Hwcu}v@i5}%j(>uGhBIv#|JJYIFZCP0N^0=sXw#K~|Dy>G z{4ni0V5hj2cbRB$rLuWA$3CB>Xbq-@Ao-YXgB3XPNEfLCk(6H8Ihbvbja)O+1~1|^ zvWaBIN7GJ&fD>fIZa#|{{V-qVCQ>xUM@{(9G6Z49c95jgdO0_O0jdq-qp)J&hYoPl zX6iHG>kCpj`LpI@*Ugsuf#)JF#Kr38!gs%qMYCCS#$J5U5{Cz1XfM440=05ZV0vMlY~~4BP*bn#Hs5hKd}qaHQF=Tf zi$W}1H`zwxFvB0wGP;n!ukv)4Tu$sZvQ*=yvvimwMNNk>hLm|01Iwl+)nFZE>TzUq z@S+|*`F2v$;Ff^`A*!xrd79dp)wGsm1)SWv8{U)n=jM}JIAmP7Y>hijuGUa{B{nus^DGBw)NYWu|i)w=fF|GsCl1 zV!>V9j76Us*bx@hhutU0zA9DT$q;s&A>#ni^mGoUrymXv*~Em4V7P!MJAPiZ7{-9% z>LR$%BBIb*-mz9J^LvhJ84Pw8Y_q`BjI9N)3;OrpEZxSWN=nl?_W2-0g|H_U<@-Sp z$zGOH5gRGX#?Ry0{iwFKN2Q#5bwkaN?WRf#JWV>Ns*+wk&6%^v1dR}yIlbU=^efAz z?^RO?2_dQO#slZC5m)^k(w9*U!BnY{&Y4dN%W-Fp8k}sSg%rodKn4G+L^^ufk4TRyIeFR~3Yvd`*+WFu8mb|JWQI+)H? zR{(Sw+{~3=GeJaQLGf@0uV=LoUJmPX$a=NZoueXsy9vWw<}cM@;22f=Yjfn}U^ zng6khn4DvuQi=;v4|a3i<>1Z6TmlXmeCwia8(1Zo86Pd^X(VA`q#qpU@e353tia;g zuJQl;oy+(WU5$*Q-$uqbcDlus$AIfBhLk6ul4cu26)YR@**B{!4{sVo!kNGV=rrtfejdNfr9eU1cjMT% zBziWV;&LVe+5}mfskFMZB8i*KR)ZGC&tdF>q;&(!!ZWm1nMHE9r_3U$Wp(vYlvxCh z3GP~;(}2(5hZxoulXioejx|bZ`VGhLQVGiVw z)7Iiaj9u0YXz9?^0yV0&r3bc4OaRobLfX>fQI9PKBK@uf_2`7He|%JfliH+ZEsBP~}6>Ky84z3-Zc(Zi9^tID}>TsiCZPm4j_mEPwbOX6jaqXKJ!6w{^SMXZ- zg-y0Cl|CumEZvznGI6S0RkmcDffcUwworNp+je1IPQVA=b46VP^3{_>G*Yx zwKA)9m(C65S|nvbL>6vXCtXxrF1&fa(h)~mYxUTTI&k3LeETh&8_7h$p#+8 zjBQzNQW?=zDLI(UdW5j+o6f%-u7Ym z*L11zs2B=`$TmBh!(3w>=V^vj|kMXN)lnqPp+a?}9+r#daW75|%? zt`m(jixe0$1VLy9IjvY>*U3oU&{^=hy50jiGe9~i!*)klMHhjG=##($;Nb zf3*iF2&@Y64WxxS$TYJ$^a(Dv#`}4?+)-vADI#a^MVm1Mx{DSOg>@c7tCw1}T&0#+ z$P)#{38_jDMPP0PF&eMPC}^1hNJtR{S+3iru0aGBlEb315)oYQu6Bg)7@q%K-DAFR zyE$p!_H5pgB%5tRFH*9L(X1VO8SJ5E4#~K=I~E~HaaU7@Kim$Kt}IC5>g1pA(DuLV zdTf#FhShO1V~{%?9myiXq6KGa>Ox`;rpi*(jp!h|ppE?%kKpWwg{TL1)3(2Iq*u0b_#qqvcS5IteTi@0DD{0ty~!~r_8j8<(p}h5t`C%tLpttrcb?=6Q9)k z$reQVL4LE~5~W63Ptzjy%De14*M{a^EyMDPB8>?lPg3G3vF^SfQ>n2D+&7>*Y`~V3>H8yg*>?AD8Yq7(O>h={n-L zm+p~`rPHM=rE8@d6GyP}q;&V0W^i7EW4YtFVTkjCbmpOg>n=qQ$^MPfxAg+L?}hcR z70Dot0_-Zgiy%U7y5al7E(FuCzji;AxFUo+y13p%k?4Vi-l7A+vxK!;CnB!+s&nE~ zHcO@FfZPfvOPyjI@68DYERxJQe^JVEHSQx%3-JSTYQ$zO}E|OQtzdyHq z`*vJBZoraq!JN=VF zV(Kdl+^+eJuZ#wV1Gq3XhVj^nAU(I1gOm&IHG5Iflu%{WbdG&WP7%VMv{WDy<{#iF zt}SU*Y1#)9M?kHvlx`DTQ{DnKyM|U+@LdeKzB0Put|(V(SClI&RFs1_B7&mSjry_f zL@#lo_gDr?=jsTG)OMkz+J$4%3WZ}L7Q@P+rw??nx2V!|PSaDykX=|clSIU7S5d6i zr*-1H`ibu@PM@6m3ihlM1b;Y(C#P=Lk%6otzLQ-bvd*zj`HF(1)E)8_VZB=b;W=a* z<9E{eEX(QY+HZ|sF?IiItj@N^&s*T5rD4UlwQO6wngy?I0d**_b+qVo9XE$#Bzj`liS>3ZK*Z99IW)-N*zFHA=~l z1ZyHptJon$6YZf;C=rdxk%S!IoJ@ojxYn^HuDG-PwR8O|ic1k<(dv~nNR~M|cjmwGn4%_SW^_C1smMS&4+BvFO98PgDrS zp3c~OK1F{xi=`Kxe8Aw}pAFu!(Le(u25Pa6s^SR&!^!=x8pk=b zqTuS@dS*|I-amcu!Qtb7JN(v9cRqSB>WWZ!?}0@X6A;52aUHG_r@YbepUBjwWgcyUQ9rD7B*^ga*U=wO&6hi_aW4sdyT5)leZu7_ z&0k5t7?H+d%*76YlL7AF)R%bDhLeB?HHI8a-diZx_k2k&Vx<+)XD6bFeFxFdW1ymn zM5QMHK*uiVM%(-lg9n0Q}Oh} z@4@0gdqZ8lT(XFb1(A4lQTx*(2J~%{LjyDoC#xxgs1OfVtHQc(o7l>A115hL#a7th z!G{_?Zq&tsQ$F!2p|~J=UtD_pV-hQ(s8{3he2Tx@|9n?Er(Shna{or=_vUgY@%*n3;ul(WXErsA3qW21kk{ zQU-&;w*SvuJReKJtbbLIB|}BN+m4BLr))+N@Zv7qlRe*M;2(@zOFhCv6o(2Y4v5gi>4UzLu9x^k;m}*T94JNm>ln>S%j9$P*@LZ;QX4}8uCFMmr<0f2F$l-M&Tk^^`XRlJ)|ti`gxERb zL6CUzCD3abtZ3}1wGh?GsuJzj(vKqG7L5uyW1Ic>lE4(d(a)-#+ z;U@l#4S#?XsF}7^SkDF0?XY?*-svA7g(1S7O=Ljl2HCf7{MGWE%!YDbsPj?%y6#{O zFTQ^NegW+M7hl8BEWE3N=<9k{6Yyg1vABvp^y*@KYiNVMKijUW9ijXWC{DAVh@E>+N~FDwTzeo zx&=y7(6a_D=6D+I4`K}!8)r5?mdeQ;dQJ}>ZtBSCS~kz!im4 zHVo4;in2-vbVDa57*lDTFpSZX0qVs5*K)w3%HYV*f=-;~mO8Q)-CPYvmNI`=ASHZ4 zBuUr~6Y|t^E!%F^@;u8^%&}-G+sL+5=}PHsf;rSud*JR_wvC>WK_$=LVZ1qTuLgSX+=Qg)2m03&g z05dYGJ^<$V5GA`)Dm5)E2w12U7iA$96(l8&q#F<6VM$JJP5dd%l#hQsv*j$g2yFt1(p$| zuo#cIfnwt|SGD+xGK&aGq(XX&L zdgRVBTh)$k&sMdA79A64^>?b7kux}!k;Bf<_vQz9^sB&vk|a@x;{KxuqT&MclPVe} z`PVNjP^tEtgUD*`k4GQ;@ksNj)b_rflOZ8$Kew4$r0IWYYa%sChivAPL1k?DCp zz>k(_UO&_flfPml{5cGRs@yEtW`U`hy6S+PnW%^*CRGBd&apqyfNF(3_$$7>&$UmT zS)*!nff+lXd%2WPTX7+0_+EP?S%`aLroPQC(1TsnsiZC=hqJ88!Q%((MbihUX3^DF zi{NxuTw?`FxCOdEHN6g*sH&~%JmXrPL>Gy+=#xVmMH3ccay2ORqu>CR#pdr|=wtBI znQjVT9O~fs9V5fDj3f<<=UH$S^Rza!O3`rb>`*kkqA+8EypFi$b#JzCqoB7&g9n;x z`8QJ3<=;p?J5MC+Xb5@Wlp78Bwf*0pbhNJ8sD{Gk8(Y>Z`})Bw@XMI9|PP`EC%hu-L?@$+N`a0mjvrULu3s$~iq-IZzjb zM9RVbDzJkLAS-Pz+~Ki$4Cd;FvOUwkq9`jOe=pU-`9 zZua;^xb@SwpFKEtarXR$pYD7VJd#kJm8FqxSNF2t-en1S8DbuU2H)m*kAvk!3|JzO zQd7W6@dfNIV%q84LPqydvA^zHS;ci__o*GPaT zxjRWV{(SbG*)y-zN;LHS`_}42gKH`la|@ynl?5>t727_4UuM zoC@v@bEbwuqANkyDoM*!*GVf>*GX9HAFn?s1WVf<&D3op4f_y&K$bN{A!OeYwp6F@ zpi7LCKC0!3(>aEH2X*wpB(=;C>2$afhg`C$-44D&H@JkT{vi)>YT9q?+6hu1@UubF zeEgbI&IOVIYk?aqo#xn*k`|(ly`Wv@0!winN z=U$uJ%CNV1KAx2UF=5j>-@L)WMU zq99uhef(@ix|M{i&u%6O%fqEewS`{vixb(YXQ)PrgJhhi=>xQuY2LldW14sC^XxmT zZ<>RHbcn$io*l~bIwG-A#PbqFQ4w4=Ihg7dH4}ZCcHyWHtskBm^sL9uriSLZ!idF? zmxfyKXf}mWQB7{AWl1+Rf+IBS)v<$OkRPH(rXrF^CD#%0nd*?S);Hv-9Dh_+#It&q z8sk^(CNm$LuN^!g(jZTXQm&1}pVrF=6G%>91$+zB_mM)ZFi`HlJ8#&%gQ8 zt=DF61!p>#yLof&#G7-YAI@HW%k7*7h0c(X>L3<<`q~t*d{qDbtZHN5*APwBLE6}; zj04CZUL#av=ssoO^niXpLNwj}egylDOgm*-3Eb90!#xgjEhXb$YNP{9unc|aCDPjZ z|H6Mw5s>@O-*HBq4xdA`{D`?ja#eTBLAh8ciij|=P(tbqg%X`x6mU%99f@Q#Dkl_4 z4Bk+_mm|awTav4F=k$Cg^<=Eoj)m7@iao*9F2?GJ0@Xq(j#pfp@`+Eu$b#ev&pj#w zuA2^kMwWvpO=iU5R}T*G@V z@z0Y#_1!A$tgFnOBRkdA9vj)W^RhIL(_t_Qf3e)6>{KOe*lrrsAgLVXnAE0ZkfmV# zGI>^G#wM~C&N}DMNR|)43O1zMPAx-GA4K|eour))B`HR2i}%|qVOo|tH4xkSx^{)j zNE@0(ghvI4+9-(j8ilRxI4@hp;NTgqH!cW2R52NxW1r%W3$jwZ?D>wVjvvDqhYhk2 zm2f-~3dJI1K-XXs>MgP~r)mPhLJXcV+hRyTCq%(jjt#qx_wOfVMxOEWT9#)?mKA6^ zXZ2^BSb=OV7{NEX&H@s- zKb%8`<>MY_z_-39ooc>LcdU|N($ zOb{b60ZSh%vXCpgrtEvyl){4K$?HD(Cd+1H$jTWyJXIE7RjWmc2vX$fn|}-|S@egq zn@F<%hqHY3%2+`Hu(El&@`5B|yBH=fjphrhZW$lVjE@=!Zxg&OzYBM8rl#SqbInu~ z7hjuHi7iOJBMY)vqwvS{8{*C!V5(l=-@mTuq-Zev5CenTq{J+DFR%3wUUjPOhkNP) zKEUe$huPghdU&z81o$=@aGVHOO|Fsg z_YPSsU&qR{!!`^~spf*}7G+msnCv@SDu)Hdla70O_67tW-kdQQxK69MJ;>PoT^-X2 zk`r|!i#OjM(aY5S)}o<1?9v(VCoJAdxsf(#zeNTaE!NY50uD*{_U_HD9i!s%`)%iv zkxfAMP@fe2CJFvDM&tz;XiiOHo8T+y;P5-G(y%jfn&?d%yl9ziP2I7T)If)ZNHUeN z2dK)2*g#-`ZJ{Y_k|llAGSfMi+a4IOLdHV*hzEve9p%1JDEAe2+_y81yaS%h_9M&b z=-T)Nsx@0W2S!0P(yOPqQAZ(2L-INbG8%gjf=&O5(=mQ6!?QZpv~Ub*;vGgP2CZpE zcL4i(7&1C*jwG?*l2A2kYjCef$1h_!Oi?$8^AfnzPgCQg7EeDeWavgMh}(O26IqNz zTE*Zy*Fg@}hzs|!RjFOgqpYlPQ6VN)cTNVPVSx`u0MuoIU%fj$!LMFbAE0>tNlDg#{68V=*oYF;~lp=r*!_FMcs0=3&GMLP=DE*r)ldJjYjnq3}Aa zxhhs7xu!nIE6E@n65>+z2DE%MxZU7pifYUo4^DwfSj*@{(hIGav`x!E{Es!E4ZSR_ zYZ2R_`Ca zI}#B<`S%k?5n2a+meF=5j+DMD-7G`Up!b3ui~m1+-~Qahm96`)?CVo?egH5 z=A812M{I+~cHm4H?$lOmTiU4At?7r19nUSm1ju9Z8j_F<36O+!@kARIqdU8K;o%a8!J{* zdb@O{+_p$cN8s=rwq8!{wTB49DqZz0z0xW8n9B&&Q#xL{07n-}R~&ISzbu6zY(z4jBIN! zH*DS^m1%Xb>2<@W4{ExjWJZcr_9gq+qxix?IOHm!f1LV)Un`x5uYL)Z#3US@A+RoB z&A}3b@wC9IyHNVZ9@rIFZx`keuFlws#GlA|6tHVGr_PiXL6no0K$HWCVX&Y&KkeC| z!)D>)u^cfo|G4-qQH#ek^8M@oxHL|Te_T3EZSOTr{s2jUOLXU}eRobA zPWK1mS#{nN|;^ToB3*Z2SQGLi7Y>1&^o=IeW} zeeu)F*Ps9C<)3%IeC?BKX9>LFOY+m%pU#tOXRjZ;_T`gs@AU&9=<5gJ6DI5%uV2M? zT|1e$_67Wn&*8R{*Unx01OdzUU;6|;^W^n?tyO>Xr?WeDEFUP>gLwpM46*V=_#A&7 z2Jo)reL3YJ?8yiQ>-IvtL%d4Cz#{Xz-!QfDX<_^ zDT+&q61L+|-P>^k$5(c0Dkb0xTR8-Nrs^^gR|P9uYS8`WMCoX2 z>2T@zzmyJvQ~4^mm1q`DlBxZ0x*M#%FKxGS)mua`mPs3h^;;Og`8lU|-r9A&mMDE> z`^n>_%dMrOHXzQIHXzO!-}2mmIVCykpskf64xEmN=OB$F2~wciK=Mo|D8|HSd>&rX zc#<^R`V#ytMvBMCPGktgXfq|s9LS7u?`EPGCuH~(gEape?4%nkrh|5sc>z%XOii(P zYW4Gc+iH_n>rt*GdJ(t-jC=Uw+-Pm9941 zoD3Xk=7u;H|37lPXB{+$oQ$NCkfA*;ip3hwspCbPXZ&}Z? zIx|TdCduG4n#FinY=IxrllIFS=?J`Ra+P#b1*w8zhSVK`ApD`}0c`-wRy_Q}`!uO& zjwfjDLSlZ(7S6Ko$7z7-1xE}X=FHLZ6ZgnK5yCqASm5#NOK#J%G)1#Uc7!x%tdW0Q zI-vTG!PibqIa;qgTTVd{w%Yn}0Vk*S*!+J7>=4;S8$)k(TSryVzrWgefLHBs-$fV< zqSzyAA!Zc5A@JhaGDaXNeFX#j8i(n>A|gZeTvF4r4lB2U^&IrbT@=Fl`*`^AYbWo@ z^SsIQk@QM-X2I+u;7{Ew*LFP$!hmWzXNf)-*&nSk|FUZS5lqlOrZiP8PT&FiBL*o` z)TDnb9{Z!I)0D;aWz9A|pxjlZQo1S@Tdtn&VPL2k@cvd&A_fDQ=8`ZjoM``I&2TWy zM_^8MM@U_aI#giPyAF&AVX--m>vUwUEotuJQI zT%Ni3>8%(12T%v~k=#g<8K!fh=0p(_E4f7b*rO=Gkq_4G3Ou?%KRxhW1Xar_LwwY? zoND3yM4Wg(5szJYp+ra$#7OlPpEIC|YVEyahpY+o>lD8|C!STGVKsV6Lz zkxhLGaMtF?=~#o=6$5Z)wqWfSX@-rL87(Q+vXO!`Tl#dZfN9`5tUfT~TVOwM1Xx<6 zKyB6txW1$uLp69K{ABGrV7p>+S#tl{ijK9xa-7JTf>+=RrwBFd2n(uusw2#pagY88 z=3pnhJJGhSeGL9JS_{oao(5COu;Eh)$U_;XBg8Q0AtQXI6KH{_I?DB{_q_AkRkPv} zWwRhttn{l6f_m1Snr_YoQ5s7YJxDXtLzBMez01mJO5SNb_^&1l#cT9nWW45arVmXY zpFTDH2`q@|lhcQ%-6 zhC`18iP-c<_Gk~|Fpt5Ak4?YfTb#ewXrbR3iL9*`gG;K#uqeb_WaMiUR@L&FJ-+wm zB1Zt%LzWbE)!nnwHmsWd9e2J)kiY-I1e`8yZ zbAzf~@F^}?@fi*aG0{~d;lqP2$NBT%`(bHd#x(PZ)>fS=Ls{%bR+(-HaDVHtrl;~c zGmI7Wn0?1N`e7iXdwwMT!?IbHqcop)Y-o=1B++|$64t7_3e5bpK(*!Bu9gg?y2_30 zicw*v7_p#mCpG#Of32-IqSQ`|g>$AMN%n z$>pmO1Cb)TTy@#cTB374(a8*|Gt6@iP!(LuUv4~;zfDy+>E7aH73r2*@ls3gGabHVGlvB>AZIkD@;StddthJ=Y?ru4a2F6$ zObEO9G}k;cmuGh3#ha7w-+Jxbig{*XEN_=0)ho(#W3W9Rgsbqa!2EjdxejmY zAhSPNj4)MEXz?T^G?xTrD3@PP$n!MpeF_}``xzzLrN{__x_;HS40p`q4mrzy<}M}H zTV_%#7qr-gYfp43yI@fMcSokPg=tVn;fBYu4Un!Rgk940969b%H7lL5DWMI-z&117 zD5Gj4K3_%~CaPxABv;@QR=pOj@){V-2B;Jygw9aKe@XK15b~L;vDdxq>72-Hss6w*}gF z_mlqwYjvlAlcmEmNk2&uz%t3kZl5Y6~#HRvfGH%67>InoF7&=AqF->B;uEPkac z#R&s^_novr4NZY(lIFE+>%yV>7Fxf;3UiBizHCmcC_wwz+cJ|RA>fK=dPauqD;@{u zfs!26WdxXqS2WCa+op=K&Vdx!msn2?v)wXHut>)o$BpKlW6immDSoK6m{bL$+qdXo z?3y)7hjhkajvNB?-`x{Nb^ytR$+ZT#y1bC9zO&!9T;hSc=|N?M>U?N~D2%3TUWy!M z^5OoudLXVMqFsZ@s@u|4n$-$6Ik0c3Y>nyMQ6%giRV8fq(!}aUulLbe8jUo5XK5SN z)6{BhrxpyZ{IK0;7pd=F;IQ3P6RX=gx5?j=I&XmT;40Kj6-)(~Rq*^R*jGu4lMbG* zgXp+h*v7cwFcKsu5Q=9h#0ANP3$=0{^Z$GSSN2~!S^oW)c+Lw4?ed^6{Nz8XS?xhb z9OaVsvA3rqC<^iauX+5+RWK#qo1i|yFu5Wz2grE(tT$g_y z908=D_xMo|VI@6V;7tSEnyU$LR_hE?J!RMyxFPF|sqnNJi-6{C+T2tn8ra8Es{}B2 zFhl7GCj&IYk~Nk|)3)}eg(7L%21Yt*N7_xa4d-Mdp}Q#|%ST)gBVmVK5vb%#>|>8I ziVI;^sr_2?HBsiF;3O7YjtFcl=!n4DC*GtJl7xT@ImAzFp=+1Pz~@uua#{@+U04sg zSgVI7xnBc$iUHBNKpgACvjDp8W-iuYihStL(XXM{8yW3 zJ(veE6iq|y?=Gei0%J8fhE5CvPBY9T@GH&iAER*A2|tNpB7p$Y3C@nfcV>yo%Xh-H z@{90Jd{-@7Z*z~!*4xa?4AYTSz<2-_pMRctgf$^uhO8xPv4%7~Ggxi#VechZWth2V zOr6OYvtstef47??hUi6}BgQ%cO_z z1e>}5dc2@9Gf81TFy|f_EbGPU82C1%g9ZkD>Mv8Kz{dQ zwRFS7nCB_5HrQgB_|?x7)274sUoUhBqU7xZ0f)67@TjsP*q#-1Ee+56$unw-r-;r99K&2B z5(`8kfp|QITmSa$19^rVi-`^m=AI`lyT+k)zASBQ{s55(w{L$2Mnz1jWE1ypz_T4Y zo+NsK;O2vH$joT8SjPh_=W|x`g}nBu#$$E=y;~Tziabg@0QJj8#lN0c7e%X_0sRV+WP!VYwi| zkzlg}wmo2Z(>lUU_&$n44Dig+bHKkxsAXz7K@gsJI3fSKeLtMR%F}{pi9x34xQcK; zajCUSO>RhYHr}~vdJ5_i3&ysG|bRBl_ zIp^3Rt%#%6FygV;Fp`iE7u{7^W%#M1r#gIDMZ=!$d$iUdx-zY`&1&6dfQ_PYq%kt# zl~v!SUKMzDeET|_#Q*wJ6(faT?2ZUFpoCosoX4ReG)KQbO4W6wO;52Oj@B{x%T<*d z!G@DiYuktK!9SnrMR!D2Exr zl%|)lLvrBvGTf3V0gJNup+*L@0xwR0GfWYZil(>>aH1t0g9OwJn$~jpjD>s! zIi9YgxncR#S2!R9q`FI{%1&#_nR@-bW(`_NJI#vjXlmiB_foQs+wNs!K^1Q7SbpE~ z;b_A_I-+R$e05ZTYhc%aN~@zZitCuJ=w_DgP=}RDQIDWm5$s+GxG)b4A8iZ+Y!t>z zRGQZ5PGr0)G@chTpdeWnvKZvGJS)I&F&>ZAq4h?YbEyR!<%yo<5flQCX|P+RMCI@l z;o`3&i?@vG4lD#$3*DN9;Oc2Td|>v4zux-l*b15l;V3MZ>YeG)UoL|>l`|voK*=k~ zmJ8D#7b1bW4FlLc8W5-tLy)+H^GwStj11{oj`U)MU>y~-8Oq59(b0(LKR+79ZUWIo zPv@E38-!3gR=QC7vUH|&W$GYlp4tm%mr56qEPHBqX%eZj*TIGui$B>2!YaK{rYJ9c z3&S{Hnw)yIB0BeLiPF0ShJJNwk4&YJD8W+nA!X zY_h>{z{=f{2Ptynez4>~;F1t@<&ZqDx_@(6+BU3Zsf4=O(`tyIX?=fF zl?a;FgEJvx{C>roy*sJOnL;|*kmf(15Q_*Bwl+qhLM(EBD-1p%>cR>1A%&;6c93qS z4r^M9^keSjyYUJlW)HtS^UBfLLsxI_-ZQ)V(#%=^ylKm2 z$ww2S2x9Zh^x4!y$)L_vZWbq63`=k6Wn}~2u?G8#6NcTnu^7(Ts@8-9nv!8WM>?#$ zTPjU9c*O$Hi(XAn@^b6VJj9tG3DKzF#JLcI&K#1*BvRC_-WS&(6n;x&H@|s#=Jb)7 zgP&sI*xRqJzVoW4^2DiADSM{ zx?C>n1*u-)u~V5<&7s#pj(RZg2Y=$8)CNV8D%X5~YGx(i_;ncxt z!R3uqPql>O6=^xJYdlLTE`%a$wVfVJnVD~o&3yfS*_2sXJ4Ol!QCIuN<6Q}($|Iz? z%hFj3Gh2Ezt}jC^Or=Cj^jISnMS_@UxH?+7(`#SU%=4#a-}&O!p-VI4uin`E#xLJq znR)S8gSl}x_U^y2ch8Nz-`?2!@s06wGY2l-{@a(nh0$Z>IN_SzPC8$_UWs+YJjXhu zkPwg5T@};M3QWgtf4CD&{3Y5Dr9e;!M+Hg5ZlAFQXVp!J3vZo=<7h62wl<7y++d#u z1yK+q4@pTbUx!>uWg}!@n5zCe6StW;ZenYr;*3=XrhB-W;%UBe^&3%an~BBi1{;5D7Rjww zUYhxKeCEK*E4fI*U~W~%B1|8NmTo$t(kv2%H(XAoThlFq*h&5i2L`BKsLZ@n0fU!3 zQX;_2#{@A{t)Beg7thDOSi+DqI^PM__)5#bgRM{&h#usc8cyGff4Bj6NFV+AIL z?*hfVJR3_L)JRcmy8%G!q{As(w+JPU|PZICI|$|XUm!FJDbNeV6jE6YyL zkrXp2SIWg-$d9Qt*xu95bfu1cr_D+^Kv!DAwb6;fOP~`)vHvCZQ^JGBrN6yL7em71qs7?VoxH^;+rb z)b6Q$aP=I#b_R~hwRjDr=t}R|DbEAY&07F+GdRus7XM9vy>&< zs@%7HHdFBK2Iebz@2#}BW2t#;uZd#kRLDhMuoh8EtY;|}jr~6ys-_w^8_N>+3_0Lp z?Gukuk_0hOooDr9*ZrPRy@i2k0Poiqt2bKB>*g7QkpbA9l3L0?&*d|!&ZrW-FWb%T zd#byW;9ga7ho)<)md|K9Ram0|r*~v1Gc^MY98kaYR3LZIA(QG{b_8WA213ZQfq?x_ zu2Q^5-$Lu;nBW@8t)y)hjtap*wQc5$CY};m=LRQ?q!kYC3g}!=lvFiWJVNGMARSy~ zIZD7KQ?UNY|8jsuWGRSU=#YwEg10Dg+(cXu8f#TJ)zZ3s>Gj)J-xIvVc;~}YcMl%kvE%N0`Lswl)P_X*oYu_Izto1LCA@1~F&~HOK~?=Zp*;_AP;p zqnZ2I#bEX$LO4=A`@W(+ZR99j@`t!a4n;5v3H@N>Y&t;taPz}Vp zHK)ndGB)|b?RSpOzI9;c<+oPcT@s^0C|12-{1|9k!KgN|35qmx)yOj$mJUG$iFm@7rvEj+-Nn>#5&T13dOmQg?t zBE2i&InqrPro!<$Vu0JOk{nfRlGi*n%3zJF+q>PW=?YCd60ma@5}Rp(r|mt+P?#O? z>SwBIm^zcQb2$3F8fY7xrdqc3`hEZgXp-qII_w_0=#2A4YRCB^OFjotL3H(o`9WaW zqz)+-7BQ@NHr9XM!vZ@>WeCdz8b$>r0}r!*K$RXVr4_hLLa{)=QCKHca^#Z740+6E znUSik4nGXa+cZlVVz8VTS+7w0_rJE1oMjj!t>Ahkie{`MDVkwec&FQI&RG3rb=w>h zFekoymlB|DXx1(#?8vYjuS0}v#6gQ7#vO6LxaY`UFi@l8?$I|xI%rbIiqw@PMu=fA zis(w{de&nKA}rZ;!muf)iW7N+mB80X$|C)l;(4{)`74ZdQj+Tg+XM-J? zUjo&6CC?QrvcjlJto*t;6nIr@0j*85Xr0?LL8vTidJ4}?&5ly=g-r*WYe4fO0-<-9;gH@THvNZp=GEZBEX~D-S&L+KhMG?k5>%mZ{2E+YYT-IGBHh%l) zrCYD~N1a)|xNImYgj_X5kG^W_<+84UI^g}j!V&qWJAyDbWsqXxF|>%R0;$IDHtAGcv=kuRCRM|1PLg2QUw1|IT;2IV!>Odf2AuLs!5uO9B&-8C?Nm2`yyh?O9AT0#1Z3SHkU!19A*g|XqJEmkZIco~5 zUTRgzMU%?eoSnSTng6D-K|^!W2a2J1?c78tzNC<%HE2)LKCm;^E4WoL5W_A81LDPV z%UTifklK})UO>=P|N3i_!4)2GBnm^7-6P~NbQH(x{!pk|clpUlv{^&2SQ@SrWdk+9 zdCM}zzqcchZh>qR$auc8n`M{ag1!8oo4ngFAy_n*K{?bDULv^@7nt~*51bg)itQz z+a{0?#W3w1rQswr%qiJGQ=0$fvVjkoqu^$wNz4&VYms7$*t)SLj9m~xSO~h>$UI}R zmq|3Og>dRRTxEt)6L!3v!-1|BFB%4?WtvSc!#Fj|3!=OG1Bhk7!Pa#ml3`%j?MRiD$uP(xs{KsRMI-rmrr-gx3E|Q0Ru%o!V16 zGqrbWKPjDn!JW4oj?YXz@0L+h@0M*%3x!mfs5wE2V6{mI`ve=87z{aNR#{S#_a!CO zXvXy%34tX|tb7WV6IkM%zwah0*2qz$w#aB&mf%WzSZ;US_W6=^u$??bMqqEws@iZX zj?f;L{mp$ZF?jn%7#PbkmWK(uKTv{rZ4MG&ZSlu#D8$`U&m$<})OhKljT4BVh;VRr z>P1+37vZ^`heX_mt>mT4^Ay9S3k|p)YYrI}( z{Qg865dx95PPyNb=FKZ_&R+Rw_Pw(!=tD~}A?P9=@w^JRZ?~~2e%}Au82kfkzVx2s zfrQ6!w+*&AIkmrZCPBXY^ttCsN8qXb8W!nAWH+CB*+wz_hRmIug?C>jWDGy&j16^q zro2?ig8!Z({j9(XRIzXU01+dI`@g5asVE(XFENf_5~Xiy{HYy3@Ba9^OAkoHFB|<~ zbzHSP^k(Nk{MTQ#sqiPLk;!V(f+iI)5{NheH_K<nBWF1$ z-I{7=b9`dTFjeQBw7?v!PEuw4N}ks*XzhqAz2GMm)TJ;P#;kIg3{z}Ot8J%tcDeS& z4K`f=$g0>FH&}ecBF&~YtomOHu_$&v!p;{H#E65lw?(X-qFz!SVAV>GALK|cW0@5W zgaspl7^@C*`0=j$SzZU<8P4bY>xOfpw{i_;%A}-+7yP%pJAEH>@S!VLU=}a85mZnA8U$l~E9>s@Gk5o$z5DJNa5Cbt z*q!J1`QK)i4~#L6?Fmw}dYkcF%Zq5W++;d6EgkCz2{`rgPDZlY5Bp?oxY63`Ix<}0 zI}Le+Sn?<{Ot57}$bgpD=3}DNe+9xI_?ms1!VO38y`ls?GJeOgH$i#~k+8aQAdwaA z|AncK2qCH38_9SryqP{UeSG@V^e3RwrcX{Eo_=HcO*nmL`cPu}b$I0+{70tWhJzyn z?tKkjeiKekk?9G%@*3RyFsggR?i`tsni?2Y(DFuMzxFZCKi9Dv6PsLhCiU+_&=h+oh!eL1Wg{q%>f7rKEFA)rSm(EYU zj;ObIa@7vzjg!*1bC9rCOBa0v`B#@MQ+mI2c52Vmc0>ihIV9 z1n~BTL_1Sc{#V|{E$v(Swsdq<1Rtr`JrU1_PdHp7F7+5Ky-t?nx?D$_#PTh@Xbsru zM_&S9zGw=oXU|!tsSgFPKOz{e-glB|k5fONL#dHL`FZ#I+qakAg@wKw7WyQ-fD64_ zQ*snR#AKFB>fA65VP`%)W@fBhax54K1#=iNtYg^=4fxI)P7 zmXfBr&*bWI;e1r3R;vbN4~EzpcHz8D;x6Pr#vHP%9*|5{az0&g2Oa9%na5{L6vDCU z#GN0$8f=?jGS5@h~bDfc{T^XL_HqO_euH`fS8MN9)WLDq6i96 z7nQcpN`|ZVlA#wCWGLrORWRTuN_8}ro+^sQ^jZ{-CaKM_V9?J5OGy~cn)n9 z-`zFJ4x# za+;1hKTWp3L-BZ$mhZ=RZZdM9QJML}>u~n-?vH=o{a!6|w||Sv-0j!qyG54S+O4B{ z#a|>A5kR-%|_d^5QPg7j8$zU5IZQrTW zA+sV3`!umx!+IL>w3bmxue_DZ**f4b2DBkXTwp6zqBoMa=ncvvB*a|d>NWD@L|Z}U z3hAJ#|L8&u(&T8skx{fyJW3}j1YJlaMq#m5g^iS@ZO{?5$yM=E&=lBIi6&>rK-v^Y zAIs;N{)9n#H2B;RvXR-W${mW8B<--1+sWd*UObV_TQy7Ln+Dv4Vgti0W&iP&uM@+x zZabrnB{(3@dFbQFvEg>e8mN&z8MKP_KB19QqS z%XpoPrACPTSJ4-4wxO{)20H)cneZ6=BF}_(?08a`yFR#WY;YT#uX-(ss9+V*;aNfq zfVC0Cel|gr!mYovxTj=(l$XcJS6*$fL zClIblF<;35mQOqi2}>}lSJ_%Kf{V82y;o+w9iP2&sE#PjvNR*0sG>r^#n-pI3b9|Z z6=J{ChzfC`#Blm>Q%+~KEO|!d*tWJ_=n#m(sG<4y2?sP;W_klJ3(=3bG$50@dDxrib610?Xt^Wu?L5Ek_vFT zPEHO9^5{APPL1rk0sB!!%YhXLJm)@eCBPj1(9v@@x$`G zmuerN`uAtEH23}4AN&`0Enm_%7zQ!a4afiJ8iODnN>rE;E_2oYLaF69MadQHopFI_EtTl%te#m;73v6-tIX>aP~>5NJKH|eEmE`w*UYs?^9H*NAQ zlF}PfZ%w^0HD0>nSTf*^E0RA>?M|zEPcGK^AI;(>ezq1%aic>Y!-2sZ$TF}5%%VB~S#H1L-bKf2N?pUFCTrss#6O+7#i2)&-!~(8zBQY?_^ei(*G1Z$b zel@DI+z)5rAcbwhyfqdO;&DL?3qf(ralB*4lj}&q{*&-AO7XN=prl=R*3c9aBu9+- zPlZn@PUtK_RF)%5m3datYy`ZlWx!p