From d16945214a1b23c70dd02b7bcfeddf69d0aca7a5 Mon Sep 17 00:00:00 2001 From: housezhang Date: Mon, 11 Jan 2021 14:57:46 +0800 Subject: [PATCH 01/14] [BAHIR-254] Update Redis to remove usage of deprecated methods (#102) RedisDescriptorTest that uses SQL or TAB API will run exception(flink1.11) because use Deprecated method and field Co-authored-by: house.zhang <> --- .../streaming/connectors/redis/RedisTableSinkFactory.java | 2 +- .../flink/streaming/connectors/redis/RedisDescriptorTest.java | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java index b454feec..0ddbcea2 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java @@ -60,7 +60,7 @@ public List supportedProperties() { properties.add(REDIS_SENTINEL); properties.add(REDIS_KEY_TTL); // schema - properties.add(SCHEMA + ".#." + SCHEMA_TYPE); + properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); properties.add(SCHEMA + ".#." + SCHEMA_NAME); properties.add(SCHEMA + ".#." + SCHEMA_FROM); // format wildcard diff --git a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java index 9b52d6ad..0e4ef172 100644 --- a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java +++ b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java @@ -68,9 +68,7 @@ public void testRedisDescriptor() throws Exception { .field("v", TypeInformation.of(Long.class))) .createTemporaryTable("redis"); - - tableEnvironment.sqlUpdate("insert into redis select k, v from t1"); - env.execute("Test Redis Table"); + tableEnvironment.executeSql("insert into redis select k, v from t1"); } From 567a887fbfb6a95f870be50584d3fce820a76af1 Mon Sep 17 00:00:00 2001 From: housezhang Date: Fri, 22 Jan 2021 22:21:27 +0800 Subject: [PATCH 02/14] [BAHIR-257] RedisDescriptorTest add TableFactory sql operate test (#108) Co-authored-by: house.zhang --- .../connectors/redis/RedisDescriptorTest.java | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java index 0e4ef172..3b48ea2a 100644 --- a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java +++ b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/RedisDescriptorTest.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.connectors.redis.descriptor.Redis; import org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Table; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.descriptors.Schema; import org.apache.flink.types.Row; @@ -71,6 +72,26 @@ public void testRedisDescriptor() throws Exception { tableEnvironment.executeSql("insert into redis select k, v from t1"); } + @Test + public void testRedisTableFactory() throws Exception { + DataStreamSource source = (DataStreamSource) env.addSource(new TestSourceFunctionString()) + .returns(new RowTypeInfo(TypeInformation.of(String.class), TypeInformation.of(Long.class))); + + EnvironmentSettings settings = EnvironmentSettings + .newInstance() + .useOldPlanner() + .inStreamingMode() + .build(); + StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings); + Table table = tableEnv.fromDataStream(source); + tableEnv.createTemporaryView("t1", table); + + tableEnv.executeSql("CREATE TABLE redis (key STRING, number BIGINT) WITH ('connector.type'='redis'," + + "'redis-mode'='cluster', 'key.ttl' = '70000','command'='INCRBY_EX','cluster-nodes'='" + REDIS_HOST + ":" + REDIS_PORT + "')"); + + tableEnv.executeSql("insert into redis select * from t1"); + + } private static class TestSourceFunctionString implements SourceFunction { private static final long serialVersionUID = 1L; From f968c8183fff067569a0924a0a407b0f8c5956ff Mon Sep 17 00:00:00 2001 From: ahern88 Date: Thu, 11 Mar 2021 20:59:12 +0800 Subject: [PATCH 03/14] [BAHIR-262] Add support to redis cluster password (#101) * support set redis cluster password --- .../connectors/redis/RedisTableSinkFactory.java | 1 + .../handler/FlinkJedisClusterConfigHandler.java | 12 ++++++++++-- .../connectors/redis/descriptor/RedisValidator.java | 1 + .../connectors/redis/common/RedisHandlerTest.java | 12 ++++++++++++ .../redis/common/config/JedisClusterConfigTest.java | 1 + 5 files changed, 25 insertions(+), 2 deletions(-) diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java index 0ddbcea2..16dfdbc2 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/RedisTableSinkFactory.java @@ -57,6 +57,7 @@ public List supportedProperties() { properties.add(REDIS_COMMAND); properties.add(REDIS_NODES); properties.add(REDIS_MASTER_NAME); + properties.add(REDIS_CLUSTER_PASSWORD); properties.add(REDIS_SENTINEL); properties.add(REDIS_KEY_TTL); // schema diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java index 6ac0c0a2..9e851827 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java @@ -19,6 +19,7 @@ import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_CLUSTER; import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_MODE; import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_NODES; +import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_CLUSTER_PASSWORD; import java.net.InetSocketAddress; import java.util.Arrays; @@ -26,6 +27,8 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; + +import org.apache.commons.lang3.StringUtils; import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisClusterConfig; import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisConfigBase; import org.apache.flink.streaming.connectors.redis.common.hanlder.FlinkJedisConfigHandler; @@ -44,8 +47,13 @@ public FlinkJedisConfigBase createFlinkJedisConfig(Map propertie String[] arr = r.split(":"); return new InetSocketAddress(arr[0].trim(), Integer.parseInt(arr[1].trim())); }).collect(Collectors.toSet()); - return new FlinkJedisClusterConfig.Builder() - .setNodes(nodes).build(); + String clusterPassword = properties.getOrDefault(REDIS_CLUSTER_PASSWORD, null); + FlinkJedisClusterConfig.Builder builder = new FlinkJedisClusterConfig.Builder(); + builder.setNodes(nodes); + if (StringUtils.isNotBlank(clusterPassword)) { + builder.setPassword(clusterPassword); + } + return builder.build(); } @Override diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/descriptor/RedisValidator.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/descriptor/RedisValidator.java index ac97722c..2b4aae20 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/descriptor/RedisValidator.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/descriptor/RedisValidator.java @@ -30,6 +30,7 @@ public class RedisValidator { public static final String REDIS_MASTER_NAME = "master.name"; public static final String SENTINELS_INFO = "sentinels.info"; public static final String SENTINELS_PASSWORD = "sentinels.password"; + public static final String REDIS_CLUSTER_PASSWORD = "cluster.password"; public static final String REDIS_KEY_TTL = "key.ttl"; } diff --git a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/RedisHandlerTest.java b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/RedisHandlerTest.java index ee1e789f..9fe3b0c7 100644 --- a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/RedisHandlerTest.java +++ b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/RedisHandlerTest.java @@ -22,8 +22,10 @@ import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_KEY_TTL; import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_MODE; import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_NODES; +import static org.apache.flink.streaming.connectors.redis.descriptor.RedisValidator.REDIS_CLUSTER_PASSWORD; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotNull; import java.util.HashMap; import java.util.Map; @@ -48,6 +50,7 @@ public static void setUp() { properties.put(REDIS_COMMAND, RedisCommand.SETEX.name()); properties.put(REDIS_NODES, "localhost:8080"); properties.put(REDIS_KEY_TTL, "1000"); + properties.put(REDIS_CLUSTER_PASSWORD, "test-pwd"); } @Test @@ -65,4 +68,13 @@ public void testFlinkJedisConfigHandler() { .createFlinkJedisConfig(properties); assertTrue(flinkJedisConfigBase instanceof FlinkJedisClusterConfig); } + + @Test + public void testFlinkJedisConfigHasPassword() { + FlinkJedisConfigBase flinkJedisConfigBase = RedisHandlerServices + .findRedisHandler(FlinkJedisConfigHandler.class, properties) + .createFlinkJedisConfig(properties); + assertNotNull(flinkJedisConfigBase.getPassword()); + assertEquals("test-pwd", flinkJedisConfigBase.getPassword()); + } } diff --git a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java index addb469d..d64be847 100644 --- a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java +++ b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/JedisClusterConfigTest.java @@ -80,4 +80,5 @@ public void shouldPasswordNotBeenSet() { .build(); assertNull(clusterConfig.getPassword()); } + } From 9dcb7d623cb39b192d8004295d6dc4c9dcbbff96 Mon Sep 17 00:00:00 2001 From: yiksanchan Date: Fri, 12 Mar 2021 01:35:24 +0800 Subject: [PATCH 04/14] [BAHIR-265] Fix Redis log messages (#118) --- .../redis/common/container/RedisClusterContainer.java | 2 +- .../connectors/redis/common/container/RedisContainer.java | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java index 307d93ed..d61716bd 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisClusterContainer.java @@ -257,7 +257,7 @@ public void incrBy(String key, Long value) { jedisCluster.incrBy(key, value); } catch (Exception e) { if (LOG.isErrorEnabled()) { - LOG.error("Cannot send Redis message with command incrby to key {} with increment {} and tll {} error message {}", + LOG.error("Cannot send Redis message with command incrby to key {} with increment {} error message {}", key, value, e.getMessage()); } throw e; diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java index 46309ab7..539192be 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisContainer.java @@ -335,7 +335,7 @@ public void incrByEx(String key, Long value, Integer ttl) { } } catch (Exception e) { if (LOG.isErrorEnabled()) { - LOG.error("Cannot send Redis with incrby command with increment {} with ttl {} error message {}", + LOG.error("Cannot send Redis with incrby command to key {} with increment {} with ttl {} error message {}", key, value, ttl, e.getMessage()); } throw e; @@ -355,7 +355,7 @@ public void decrByEx(String key, Long value, Integer ttl) { } } catch (Exception e) { if (LOG.isErrorEnabled()) { - LOG.error("Cannot send Redis with decrBy command with decrement {} with ttl {} error message {}", + LOG.error("Cannot send Redis with decrBy command to key {} with decrement {} with ttl {} error message {}", key, value, ttl, e.getMessage()); } throw e; @@ -372,7 +372,7 @@ public void incrBy(String key, Long value) { jedis.incrBy(key, value); } catch (Exception e) { if (LOG.isErrorEnabled()) { - LOG.error("Cannot send Redis with incrby command with increment {} error message {}", + LOG.error("Cannot send Redis with incrby command to key {} with increment {} error message {}", key, value, e.getMessage()); } throw e; @@ -389,7 +389,7 @@ public void decrBy(String key, Long value) { jedis.decrBy(key, value); } catch (Exception e) { if (LOG.isErrorEnabled()) { - LOG.error("Cannot send Redis with decrBy command with increment {} error message {}", + LOG.error("Cannot send Redis with decrBy command to key {} with increment {} error message {}", key, value, e.getMessage()); } throw e; From b4efa3053bba2b2d44ac1827af1e94cf3473b6c5 Mon Sep 17 00:00:00 2001 From: yiksanchan Date: Fri, 12 Mar 2021 01:37:41 +0800 Subject: [PATCH 05/14] [BAHIR-264] Fix dependencies in flink-connector-redis (#116) * [BAHIR-264] Dedup dependencies * [BAHIR-264] Change hardcode scala binary version 2.11 --- flink-connector-redis/pom.xml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/flink-connector-redis/pom.xml b/flink-connector-redis/pom.xml index 2f9232a3..43767ed7 100644 --- a/flink-connector-redis/pom.xml +++ b/flink-connector-redis/pom.xml @@ -28,7 +28,7 @@ under the License. .. - flink-connector-redis_2.11 + flink-connector-redis_${scala.binary.version} flink-connector-redis jar @@ -45,7 +45,7 @@ under the License. org.apache.flink - flink-table-api-java-bridge_2.11 + flink-table-api-java-bridge_${scala.binary.version} ${flink.version} @@ -79,15 +79,10 @@ under the License. org.apache.flink - flink-table-planner_2.11 + flink-table-planner_${scala.binary.version} ${flink.version} - - org.apache.flink - flink-table-api-java-bridge_2.11 - ${flink.version} - org.apache.flink flink-table-common From 464a08f84c7df9c93c58f6456fbd709100fd2620 Mon Sep 17 00:00:00 2001 From: yiksanchan Date: Fri, 12 Mar 2021 01:42:17 +0800 Subject: [PATCH 06/14] [BAHIR-266] Redis - Fix java8 stream use (#117) --- .../config/handler/FlinkJedisClusterConfigHandler.java | 2 +- .../config/handler/FlinkJedisSentinelConfigHandler.java | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java index 9e851827..09d8911c 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisClusterConfigHandler.java @@ -43,7 +43,7 @@ public class FlinkJedisClusterConfigHandler implements FlinkJedisConfigHandler { public FlinkJedisConfigBase createFlinkJedisConfig(Map properties) { Preconditions.checkArgument(properties.containsKey(REDIS_NODES), "nodes should not be null in cluster mode"); String nodesInfo = properties.get(REDIS_NODES); - Set nodes = Arrays.asList(nodesInfo.split(",")).stream().map(r -> { + Set nodes = Arrays.stream(nodesInfo.split(",")).map(r -> { String[] arr = r.split(":"); return new InetSocketAddress(arr[0].trim(), Integer.parseInt(arr[1].trim())); }).collect(Collectors.toSet()); diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisSentinelConfigHandler.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisSentinelConfigHandler.java index 72065d57..b621f97c 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisSentinelConfigHandler.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/handler/FlinkJedisSentinelConfigHandler.java @@ -25,10 +25,10 @@ import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisConfigBase; import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisSentinelConfig; import org.apache.flink.streaming.connectors.redis.common.hanlder.FlinkJedisConfigHandler; @@ -41,8 +41,7 @@ public FlinkJedisConfigBase createFlinkJedisConfig(Map propertie String sentinelsInfo = properties.computeIfAbsent(SENTINELS_INFO, null); Objects.requireNonNull(masterName, "master should not be null in sentinel mode"); Objects.requireNonNull(sentinelsInfo, "sentinels should not be null in sentinel mode"); - Set sentinels = Arrays.asList(sentinelsInfo.split(",")) - .stream().collect(Collectors.toSet()); + Set sentinels = new HashSet<>(Arrays.asList(sentinelsInfo.split(","))); String sentinelsPassword = properties.computeIfAbsent(SENTINELS_PASSWORD, null); if (sentinelsPassword != null && sentinelsPassword.trim().isEmpty()) { sentinelsPassword = null; From abd99411b031c0a5ca377698cf5c7812f57e9346 Mon Sep 17 00:00:00 2001 From: cheegoday <972684638@qq.com> Date: Fri, 12 Mar 2021 01:46:10 +0800 Subject: [PATCH 07/14] [BAHIR-263] Update flink version to 1.11.3 (#112) BAHIR-263 - update flink version to 1.11.3 BAHIR-263 - fix netty connector unit test --- .travis.yml | 4 ++-- flink-connector-netty/pom.xml | 12 ++++++++++++ .../connectors/netty/example/StreamSqlExample.scala | 6 ++++-- pom.xml | 2 +- 4 files changed, 19 insertions(+), 5 deletions(-) diff --git a/.travis.yml b/.travis.yml index 4a473f50..f0c3c34a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -34,8 +34,8 @@ jdk: - openjdk8 env: - - FLINK_VERSION="1.11.2" SCALA_VERSION="2.11" - - FLINK_VERSION="1.11.2" SCALA_VERSION="2.12" + - FLINK_VERSION="1.11.3" SCALA_VERSION="2.11" + - FLINK_VERSION="1.11.3" SCALA_VERSION="2.12" before_install: - ./dev/change-scala-version.sh $SCALA_VERSION diff --git a/flink-connector-netty/pom.xml b/flink-connector-netty/pom.xml index 51469851..a93a0671 100644 --- a/flink-connector-netty/pom.xml +++ b/flink-connector-netty/pom.xml @@ -71,6 +71,18 @@ flink-table-planner_${scala.binary.version} ${flink.version} + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${flink.version} + test + + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + test + diff --git a/flink-connector-netty/src/test/scala/org/apache/flink/streaming/connectors/netty/example/StreamSqlExample.scala b/flink-connector-netty/src/test/scala/org/apache/flink/streaming/connectors/netty/example/StreamSqlExample.scala index 08b50683..5bca265e 100644 --- a/flink-connector-netty/src/test/scala/org/apache/flink/streaming/connectors/netty/example/StreamSqlExample.scala +++ b/flink-connector-netty/src/test/scala/org/apache/flink/streaming/connectors/netty/example/StreamSqlExample.scala @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.netty.example import org.apache.flink.api.java.utils.ParameterTool import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} +import org.apache.flink.table.api.EnvironmentSettings import org.apache.flink.table.api.bridge.scala._ /** @@ -39,8 +40,9 @@ object StreamSqlExample { val param = ParameterTool.fromArgs(args) // set up execution environment + val envSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build() val env = StreamExecutionEnvironment.getExecutionEnvironment - val tEnv = StreamTableEnvironment.create(env) + val tEnv = StreamTableEnvironment.create(env, envSettings) val spec = if (param.get("tcp") == "true") { new TcpReceiverSource(7070, Some("http://localhost:9090/cb")) @@ -59,7 +61,7 @@ object StreamSqlExample { tEnv.createTemporaryView("OrderA", orderA) // union the two tables - val result = tEnv.sqlQuery("SELECT STREAM * FROM OrderA WHERE amount > 2") + val result = tEnv.sqlQuery("SELECT * FROM OrderA WHERE amount > 2") result.toAppendStream[Order].print() diff --git a/pom.xml b/pom.xml index 109b5282..d45cd2b5 100644 --- a/pom.xml +++ b/pom.xml @@ -96,7 +96,7 @@ 2.13.3 - 1.11.2 + 1.11.3 5.4.1 From bfc09d38b4f1dfd0686732584175d7322dc34f11 Mon Sep 17 00:00:00 2001 From: Joao Boto Date: Fri, 12 Mar 2021 08:53:06 +0100 Subject: [PATCH 08/14] [BAHIR-263] Update flink to 1.12.2 (#115) --- .travis.yml | 4 +-- .../activemq/ActiveMQConnectorITCase.java | 2 -- .../kudu/table/KuduCatalogTest.java | 16 +++++------ .../kudu/table/KuduTableFactoryTest.java | 12 ++++----- .../kudu/table/KuduTableSourceTest.java | 2 +- .../src/test/resources/log4j.properties | 27 +++++++++++++++++++ pom.xml | 2 +- 7 files changed, 45 insertions(+), 20 deletions(-) create mode 100644 flink-connector-kudu/src/test/resources/log4j.properties diff --git a/.travis.yml b/.travis.yml index f0c3c34a..b6793dc4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -34,8 +34,8 @@ jdk: - openjdk8 env: - - FLINK_VERSION="1.11.3" SCALA_VERSION="2.11" - - FLINK_VERSION="1.11.3" SCALA_VERSION="2.12" + - FLINK_VERSION="1.12.2" SCALA_VERSION="2.11" + - FLINK_VERSION="1.12.2" SCALA_VERSION="2.12" before_install: - ./dev/change-scala-version.sh $SCALA_VERSION diff --git a/flink-connector-activemq/src/test/java/org/apache/flink/streaming/connectors/activemq/ActiveMQConnectorITCase.java b/flink-connector-activemq/src/test/java/org/apache/flink/streaming/connectors/activemq/ActiveMQConnectorITCase.java index 6e3a2479..c784d9ab 100644 --- a/flink-connector-activemq/src/test/java/org/apache/flink/streaming/connectors/activemq/ActiveMQConnectorITCase.java +++ b/flink-connector-activemq/src/test/java/org/apache/flink/streaming/connectors/activemq/ActiveMQConnectorITCase.java @@ -60,7 +60,6 @@ public static void beforeClass() { flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS.key(), 8); flinkConfig.setInteger(TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), 16); - flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); } @@ -111,7 +110,6 @@ public void amqTopologyWithTopic() throws Exception { private StreamExecutionEnvironment createExecutionEnvironment() { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironmentWithWebUI(flinkConfig); env.setRestartStrategy(RestartStrategies.noRestart()); - env.getConfig().disableSysoutLogging(); return env; } diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduCatalogTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduCatalogTest.java index 4bb1871e..2bc8b127 100644 --- a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduCatalogTest.java +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduCatalogTest.java @@ -77,7 +77,7 @@ public void testCreateAlterDrop() throws Exception { tableEnv.executeSql("INSERT INTO TestTable1 VALUES ('f', 's')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); // Add this once Primary key support has been enabled @@ -101,7 +101,7 @@ public void testCreateAndInsertMultiKey() throws Exception { tableEnv.executeSql("INSERT INTO TestTable3 VALUES ('f', 2, 't')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); validateMultiKey("TestTable3"); @@ -113,14 +113,14 @@ public void testSourceProjection() throws Exception { tableEnv.executeSql("INSERT INTO TestTable5 VALUES ('s', 'f', 't')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); tableEnv.executeSql("CREATE TABLE TestTable6 (`first` STRING, `second` String) WITH ('kudu.hash-columns' = 'first', 'kudu.primary-key-columns' = 'first')"); tableEnv.executeSql("INSERT INTO TestTable6 (SELECT `first`, `second` FROM TestTable5)") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); validateSingleKey("TestTable6"); @@ -133,12 +133,12 @@ public void testEmptyProjection() throws Exception { tableEnv.executeSql("INSERT INTO TestTableEP VALUES ('f','s')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); tableEnv.executeSql("INSERT INTO TestTableEP VALUES ('f2','s2')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); Table result = tableEnv.sqlQuery("SELECT COUNT(*) FROM TestTableEP"); @@ -225,7 +225,7 @@ public void testTimestamp() throws Exception { tableEnv.executeSql("INSERT INTO TestTableTsC values ('f', TIMESTAMP '2020-01-01 12:12:12.123456')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); KuduTable kuduTable = harness.getClient().openTable("TestTableTsC"); @@ -252,7 +252,7 @@ public void testDatatypes() throws Exception { "TIMESTAMP '2020-04-15 12:34:56.123') ") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); validateManyTypes("TestTable8"); diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java index d852f8e9..d4de7f68 100644 --- a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java @@ -64,7 +64,7 @@ public void testNonExistingTable() throws Exception { "WITH ('connector.type'='kudu', 'kudu.table'='TestTable11', 'kudu.masters'='" + kuduMasters + "')"); JobClient jobClient = tableEnv.executeSql("INSERT INTO TestTable11 values ('f', 1)").getJobClient().get(); try { - jobClient.getJobExecutionResult(getClass().getClassLoader()).get(); + jobClient.getJobExecutionResult().get(); fail(); } catch (ExecutionException ee) { assertTrue(ee.getCause() instanceof JobExecutionException); @@ -80,7 +80,7 @@ public void testCreateTable() throws Exception { tableEnv.executeSql("INSERT INTO TestTable11 values ('f', 's')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); validateSingleKey("TestTable11"); @@ -96,13 +96,13 @@ public void testTimestamp() throws Exception { tableEnv.executeSql("INSERT INTO TestTableTs values ('f', TIMESTAMP '2020-01-01 12:12:12.123456')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); tableEnv.executeSql("INSERT INTO TestTableTs values ('s', TIMESTAMP '2020-02-02 23:23:23')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); KuduTable kuduTable = harness.getClient().openTable("TestTableTs"); @@ -129,7 +129,7 @@ public void testExistingTable() throws Exception { tableEnv.executeSql("INSERT INTO TestTable12 values ('f', 's')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); // Then another one in SQL that refers to the previously created one @@ -138,7 +138,7 @@ public void testExistingTable() throws Exception { tableEnv.executeSql("INSERT INTO TestTable12b values ('f2','s2')") .getJobClient() .get() - .getJobExecutionResult(getClass().getClassLoader()) + .getJobExecutionResult() .get(1, TimeUnit.MINUTES); // Validate that both insertions were into the same table diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableSourceTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableSourceTest.java index 2dfb71b0..e02a2972 100644 --- a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableSourceTest.java +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableSourceTest.java @@ -95,7 +95,7 @@ void testGetTableSchema() throws Exception { void testGetProducedDataType() throws Exception { DataType producedDataType = kuduTableSource.getProducedDataType(); assertNotNull(producedDataType); - assertEquals(getReturnDataType(getFieldNames(), getFieldDataTypes()), producedDataType); + assertEquals(getReturnDataType(getFieldNames(), getFieldDataTypes()).notNull(), producedDataType); } @Test diff --git a/flink-connector-kudu/src/test/resources/log4j.properties b/flink-connector-kudu/src/test/resources/log4j.properties new file mode 100644 index 00000000..15efe08c --- /dev/null +++ b/flink-connector-kudu/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF 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. +################################################################################ + +# This file ensures that tests executed from the IDE show log output + +log4j.rootLogger=WARN, console + +# Log all infos in the given file +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/pom.xml b/pom.xml index d45cd2b5..7e83d852 100644 --- a/pom.xml +++ b/pom.xml @@ -96,7 +96,7 @@ 2.13.3 - 1.11.3 + 1.12.2 5.4.1 From cabee21712283dddd2f7c6f4e60e42a848a728f1 Mon Sep 17 00:00:00 2001 From: yiksanchan Date: Fri, 12 Mar 2021 16:14:34 +0800 Subject: [PATCH 09/14] [BAHIR-267] Unify Redis commands builder (#119) --- .../RedisCommandsContainerBuilder.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java index 3e80d572..bdb9fed1 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java @@ -62,10 +62,7 @@ public static RedisCommandsContainer build(FlinkJedisConfigBase flinkJedisConfig public static RedisCommandsContainer build(FlinkJedisPoolConfig jedisPoolConfig) { Objects.requireNonNull(jedisPoolConfig, "Redis pool config should not be Null"); - GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig(); - genericObjectPoolConfig.setMaxIdle(jedisPoolConfig.getMaxIdle()); - genericObjectPoolConfig.setMaxTotal(jedisPoolConfig.getMaxTotal()); - genericObjectPoolConfig.setMinIdle(jedisPoolConfig.getMinIdle()); + GenericObjectPoolConfig genericObjectPoolConfig = getGenericObjectPoolConfig(jedisPoolConfig); JedisPool jedisPool = new JedisPool(genericObjectPoolConfig, jedisPoolConfig.getHost(), jedisPoolConfig.getPort(), jedisPoolConfig.getConnectionTimeout(), jedisPoolConfig.getPassword(), @@ -83,10 +80,7 @@ public static RedisCommandsContainer build(FlinkJedisPoolConfig jedisPoolConfig) public static RedisCommandsContainer build(FlinkJedisClusterConfig jedisClusterConfig) { Objects.requireNonNull(jedisClusterConfig, "Redis cluster config should not be Null"); - GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig(); - genericObjectPoolConfig.setMaxIdle(jedisClusterConfig.getMaxIdle()); - genericObjectPoolConfig.setMaxTotal(jedisClusterConfig.getMaxTotal()); - genericObjectPoolConfig.setMinIdle(jedisClusterConfig.getMinIdle()); + GenericObjectPoolConfig genericObjectPoolConfig = getGenericObjectPoolConfig(jedisClusterConfig); JedisCluster jedisCluster = new JedisCluster(jedisClusterConfig.getNodes(), jedisClusterConfig.getConnectionTimeout(), @@ -107,10 +101,7 @@ public static RedisCommandsContainer build(FlinkJedisClusterConfig jedisClusterC public static RedisCommandsContainer build(FlinkJedisSentinelConfig jedisSentinelConfig) { Objects.requireNonNull(jedisSentinelConfig, "Redis sentinel config should not be Null"); - GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig(); - genericObjectPoolConfig.setMaxIdle(jedisSentinelConfig.getMaxIdle()); - genericObjectPoolConfig.setMaxTotal(jedisSentinelConfig.getMaxTotal()); - genericObjectPoolConfig.setMinIdle(jedisSentinelConfig.getMinIdle()); + GenericObjectPoolConfig genericObjectPoolConfig = getGenericObjectPoolConfig(jedisSentinelConfig); JedisSentinelPool jedisSentinelPool = new JedisSentinelPool(jedisSentinelConfig.getMasterName(), jedisSentinelConfig.getSentinels(), genericObjectPoolConfig, @@ -118,4 +109,12 @@ public static RedisCommandsContainer build(FlinkJedisSentinelConfig jedisSentine jedisSentinelConfig.getPassword(), jedisSentinelConfig.getDatabase()); return new RedisContainer(jedisSentinelPool); } + + private static GenericObjectPoolConfig getGenericObjectPoolConfig(FlinkJedisConfigBase jedisConfig) { + GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig(); + genericObjectPoolConfig.setMaxIdle(jedisConfig.getMaxIdle()); + genericObjectPoolConfig.setMaxTotal(jedisConfig.getMaxTotal()); + genericObjectPoolConfig.setMinIdle(jedisConfig.getMinIdle()); + return genericObjectPoolConfig; + } } From b618a77cca2e44be86f18607e21a6b2d2bf8dd39 Mon Sep 17 00:00:00 2001 From: yiksanchan Date: Fri, 12 Mar 2021 16:16:37 +0800 Subject: [PATCH 10/14] [BAHIR-267] Fix FlinkJedisConfig names in toString methods (#120) --- .../connectors/redis/common/config/FlinkJedisClusterConfig.java | 2 +- .../connectors/redis/common/config/FlinkJedisPoolConfig.java | 2 +- .../redis/common/config/FlinkJedisSentinelConfig.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java index f05dfd8b..c56ac14e 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java @@ -191,7 +191,7 @@ public FlinkJedisClusterConfig build() { @Override public String toString() { - return "JedisClusterConfig{" + + return "FlinkJedisClusterConfig{" + "nodes=" + nodes + ", timeout=" + connectionTimeout + ", maxRedirections=" + maxRedirections + diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java index 7c37ecbb..3f8fc2ff 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java @@ -201,7 +201,7 @@ public FlinkJedisPoolConfig build() { @Override public String toString() { - return "JedisPoolConfig{" + + return "FlinkJedisPoolConfig{" + "host='" + host + '\'' + ", port=" + port + ", timeout=" + connectionTimeout + diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java index 2fb87b9a..928f5e8c 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java @@ -236,7 +236,7 @@ public FlinkJedisSentinelConfig build(){ @Override public String toString() { - return "JedisSentinelConfig{" + + return "FlinkJedisSentinelConfig{" + "masterName='" + masterName + '\'' + ", connectionTimeout=" + connectionTimeout + ", soTimeout=" + soTimeout + From bd63cb1576a0fef3b973cccf4896f123ea1dc3e1 Mon Sep 17 00:00:00 2001 From: mschroederi Date: Wed, 7 Apr 2021 13:19:06 +0200 Subject: [PATCH 11/14] [BAHIR-269] Apache Pinot Connector Sink (#113) --- distribution/pom.xml | 5 + flink-connector-pinot/README.md | 122 +++++ .../PinotSinkGlobalCommitter_combine.png | Bin 0 -> 306316 bytes .../docs/images/PinotSinkWriter.png | Bin 0 -> 197924 bytes .../images/PinotSinkWriter_prepareCommit.png | Bin 0 -> 253074 bytes flink-connector-pinot/pom.xml | 196 ++++++++ .../pinot/PinotControllerClient.java | 142 ++++++ .../pinot/PinotControllerHttpClient.java | 134 +++++ .../streaming/connectors/pinot/PinotSink.java | 376 ++++++++++++++ .../pinot/committer/PinotSinkCommittable.java | 59 +++ .../committer/PinotSinkGlobalCommittable.java | 59 +++ .../committer/PinotSinkGlobalCommitter.java | 470 +++++++++++++++++ .../PinotControllerApiException.java | 34 ++ .../pinot/external/EventTimeExtractor.java | 51 ++ .../pinot/external/JsonSerializer.java | 32 ++ .../pinot/filesystem/FileSystemAdapter.java | 55 ++ .../pinot/filesystem/FileSystemUtils.java | 64 +++ .../name/PinotSinkSegmentNameGenerator.java | 30 ++ .../name/SimpleSegmentNameGenerator.java | 62 +++ .../PinotSinkCommittableSerializer.java | 71 +++ .../PinotSinkGlobalCommittableSerializer.java | 83 +++ .../PinotSinkWriterStateSerializer.java | 83 +++ .../pinot/writer/PinotSinkWriter.java | 176 +++++++ .../pinot/writer/PinotSinkWriterState.java | 47 ++ .../pinot/writer/PinotWriterSegment.java | 153 ++++++ .../pinot/LocalFileSystemAdapter.java | 79 +++ .../connectors/pinot/PinotSinkTest.java | 475 ++++++++++++++++++ .../connectors/pinot/PinotTestBase.java | 251 +++++++++ .../connectors/pinot/PinotTestHelper.java | 168 +++++++ .../src/test/resources/log4j.properties | 27 + pom.xml | 1 + 31 files changed, 3505 insertions(+) create mode 100644 flink-connector-pinot/README.md create mode 100644 flink-connector-pinot/docs/images/PinotSinkGlobalCommitter_combine.png create mode 100644 flink-connector-pinot/docs/images/PinotSinkWriter.png create mode 100644 flink-connector-pinot/docs/images/PinotSinkWriter_prepareCommit.png create mode 100644 flink-connector-pinot/pom.xml create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerClient.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotSink.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkCommittable.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommittable.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/exceptions/PinotControllerApiException.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/EventTimeExtractor.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/JsonSerializer.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemAdapter.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemUtils.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/SimpleSegmentNameGenerator.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkGlobalCommittableSerializer.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkWriterStateSerializer.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriterState.java create mode 100644 flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java create mode 100644 flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/LocalFileSystemAdapter.java create mode 100644 flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java create mode 100644 flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java create mode 100644 flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestHelper.java create mode 100644 flink-connector-pinot/src/test/resources/log4j.properties diff --git a/distribution/pom.xml b/distribution/pom.xml index b4834b12..89d21d50 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -52,6 +52,11 @@ flink-connector-netty_${scala.binary.version} ${project.version} + + org.apache.bahir + flink-connector-pinot_${scala.binary.version} + ${project.version} + org.apache.bahir flink-connector-redis_${scala.binary.version} diff --git a/flink-connector-pinot/README.md b/flink-connector-pinot/README.md new file mode 100644 index 00000000..2044e001 --- /dev/null +++ b/flink-connector-pinot/README.md @@ -0,0 +1,122 @@ +# Flink Pinot Connector + +This connector provides a sink to [Apache Pinot](http://pinot.apache.org/)™. +To use this connector, add the following dependency to your project: + + + org.apache.bahir + flink-connector-pinot_2.11 + 1.1-SNAPSHOT + + +*Version Compatibility*: This module is compatible with Pinot 0.6.0. + +Note that the streaming connectors are not part of the binary distribution of Flink. You need to link them into your job jar for cluster execution. +See how to link with them for cluster execution [here](https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/linking.html). + +The sink class is called `PinotSink`. + +## Architecture +The Pinot sink stores elements from upstream Flink tasks in an Apache Pinot table. +We support two execution modes +* `RuntimeExecutionMode.BATCH` +* `RuntimeExecutionMode.STREAMING` which requires checkpointing to be enabled. + +### PinotSinkWriter +Whenever the sink receives elements from upstream tasks, they are received by an instance of the PinotSinkWriter. +The `PinotSinkWriter` holds a list of `PinotWriterSegment`s where each `PinotWriterSegment` is capable of storing `maxRowsPerSegment` elements. +Whenever the maximum number of elements to hold is not yet reached the `PinotWriterSegment` is considered to be active. +Once the maximum number of elements to hold was reached, an active `PinotWriterSegment` gets inactivated and a new empty `PinotWriterSegment` is created. + +PinotSinkWriter + +Thus, there is always one active `PinotWriterSegment` that new incoming elements will go to. +Over time, the list of `PinotWriterSegment` per `PinotSinkWriter` increases up to the point where a checkpoint is created. + +**Checkpointing** +On checkpoint creation `PinotSinkWriter.prepareCommit` gets called by the Flink environment. +This triggers the creation of `PinotSinkCommittable`s where each inactive `PinotWriterSegment` creates exactly one `PinotSinkCommittable`. + +PinotSinkWriter prepareCommit + +In order to create a `PinotSinkCommittable`, a file containing a `PinotWriterSegment`'s elements is on the shared filesystem defined via `FileSystemAdapter`. +The file contains a list of elements in JSON format. The serialization is done via `JSONSerializer`. +A `PinotSinkCommittables` then holds the path to the data file on the shared filesystem as well as the minimum and maximum timestamp of all contained elements (extracted via `EventTimeExtractor`). + + +### PinotSinkGlobalCommitter +In order to be able to follow the guidelines for Pinot segment naming, we need to include the minimum and maximum timestamp of an element in the metadata of a segment and in its name. +The minimum and maximum timestamp of all elements between two checkpoints is determined at a parallelism of 1 in the `PinotSinkGlobalCommitter`. +This procedure allows recovering from failure by deleting previously uploaded segments which prevents having duplicate segments in the Pinot table. + +PinotSinkGlobalCommitter combine + +After all `PinotSinkWriter` subtasks emitted their `PinotSinkCommittable`s, they are sent to the `PinotSinkGlobalCommitter` which first combines all collected `PinotSinkCommittable`s into a single `PinotSinkGlobalCommittable`. +Therefore, the minimum and maximum timestamps of all collected `PinotSinkCommittable`s is determined. +Moreover, the `PinotSinkGlobalCommittable` holds references to all data files from the `PinotSinkCommittable`s. + +When finally committing a `PinotSinkGlobalCommittable` the following procedure is executed: +* Read all data files from the shared filesystem (using `FileSystemAdapter`). +* Generate Pinot segment names using `PinotSinkSegmentNameGenerator`. +* Create Pinot segments with minimum and maximum timestamps (stored in `PinotSinkGlobalCommittable`) and previously generated segment assigned. +* Upload Pinot segments to the Pinot controller + + +## Delivery Guarantees +Resulting from the above described architecture the `PinotSink` provides an at-least-once delivery guarantee. +While the failure recovery mechanism ensures that duplicate segments are prevented, there might be temporary inconsistencies in the Pinot table which can result in downstream tasks receiving an element multiple times. + +## Options +| Option | Description | +| ---------------------- | -------------------------------------------------------------------------------- | +| `pinotControllerHost` | Host of the Pinot controller | +| `pinotControllerPort` | Port of the Pinot controller | +| `tableName` | Target Pinot table's name | +| `maxRowsPerSegment` | Maximum number of rows to be stored within a Pinot segment | +| `tempDirPrefix` | Prefix for temp directories used | +| `jsonSerializer` | Serializer used to convert elements to JSON | +| `eventTimeExtractor` | Defines the way event times are extracted from received objects | +| `segmentNameGenerator` | Pinot segment name generator | +| `fsAdapter` | Filesystem adapter used to save files for sharing files across nodes | +| `numCommitThreads` | Number of threads used in the `PinotSinkGlobalCommitter` for committing segments | + +## Usage +```java +StreamExecutionEnvironment env = ... +// Checkpointing needs to be enabled when executing in STREAMING mode + env.enableCheckpointing(long interval); + + DataStream dataStream = ... + PinotSink pinotSink = new PinotSink.Builder(String pinotControllerHost, String pinotControllerPort, String tableName) + + // Serializes a PinotRow to JSON format + .withJsonSerializer(JsonSerializer jsonSerializer) + + // Extracts the timestamp from a PinotRow + .withEventTimeExtractor(EventTimeExtractor eventTimeExtractor) + + // Defines the segment name generation via the predefined SimpleSegmentNameGenerator + // Exemplary segment name: tableName_minTimestamp_maxTimestamp_segmentNamePostfix_0 + .withSimpleSegmentNameGenerator(String tableName, String segmentNamePostfix) + + // Use a custom segment name generator if the SimpleSegmentNameGenerator does not work for your use case + .withSegmentNameGenerator(SegmentNameGenerator segmentNameGenerator) + + // Use a custom filesystem adapter. + // CAUTION: Make sure all nodes your Flink app runs on can access the shared filesystem via the provided FileSystemAdapter + .withFileSystemAdapter(FileSystemAdapter fsAdapter) + + // Defines the size of the Pinot segments + .withMaxRowsPerSegment(int maxRowsPerSegment) + + // Prefix within the local filesystem's temp directory used for storing intermediate files + .withTempDirectoryPrefix(String tempDirPrefix) + + // Number of threads used in the `PinotSinkGlobalCommitter` to commit a batch of segments + // Optional - Default is 4 + .withNumCommitThreads(int numCommitThreads) + + // Builds the PinotSink + .build() + dataStream.addSink(pinotSink); +``` diff --git a/flink-connector-pinot/docs/images/PinotSinkGlobalCommitter_combine.png b/flink-connector-pinot/docs/images/PinotSinkGlobalCommitter_combine.png new file mode 100644 index 0000000000000000000000000000000000000000..e9ea878bfc9954045d33400f78dbfe6b3b91834b GIT binary patch literal 306316 zcmeFZby!sE+ct~}f=G&h^ia~EC?K7Z(%sS}jC8lc2oi#TbazP&-G~Cx-QC>{-@v0Rfc)0Ra&W2G={ zlyhEEQhC3*r!OQ|6!jcwn3>|GuuA=(>Eq z)aBV7cPd-?8{cs7#pj!SqYV`MKczW%AutKtQhF5E5iX%-Z7q+` z{_@(<*%L=YxN<3L=4|=I6W6j%p!F640jBkB-PR7@lV=D{xUv*X2wqt|nlI==X?KVr z_GymJSOSM~aVPRCJs2 zlnL+A*O3oE4okMe6y)qa&tAk4;iUJO7zQ*w!)hM%k-^1%eUE&|-|2$@UQAN)C{<^O z>Z#EBlk``SlAgf~Bs;k~hjIzwEegB!xTw$QJ>Kh3y?EjM>1L5pGA-wAeXbUn;c?(}rpQP*U(w~R%jJ~WJd6*2rNo_huee({x{p@9? z(}eel9`gvU_os-b-zY4ycBGY(jsw(J|QH`XU1t!D~_TqNO5$2$!FsA z+bLpwM|C|PbLIps1pLe}vhI)fTd3*hTLOzKQ|CYK&U6Ugf9Z?WfM_$qD!}*cc1mP{ zAG1>?@ts>*AS87G1Y+fwMgqE9l=@E62+wD5MQ(zKV(@t$(IdQ-yEYQft>NBm>whrVNALc9Ty?5BHzAuoai zrSxeZzj+{tMmaX?=Y+83FV<**^{SwPW9!MMuN zj+};>Dac2D_GEJc_lFSsrzwyrN+ggaK4d|~i z3lbc5?%DYsC*ccX-4hai0J)(bphk`XVVDa-1SN~#zZa55u_}&7A-eDl&5%zaAWwuo zv{M{?kHH3o(>qQ~;{*3M$}}A3KsV9H9|P8sTC>==?j4G%n^MIY=V&Xt77R%ZQE7f~ zZ!0q7b0%~qaAv4{=N3I7vz2Zt|BCkNqlhHC5p#N+f*7ZCrL1#iL4j4lRw|#Ub{d~p zt(JS?8RwzcT4L6ik}G{IR|0(|bGlU~Y35XNU23oCP$@`*Fy9OMQAL5%L-YG)AI&y~EAo2NC5B3e3@vQfOw3fwN&9NjM^nG_ z*z^Vrw)bnKlqs&)sQ9x>jB9W!#TM2IxCS0cY|@~;*K2(F#OeL%`~3I)?;S`|f`c%` zXf!jGOv+^fWy%Yn1=Uu?TS}9Xlay7KE}s(0NX1xG!O>HEwXrUV{sfo?=s0MaJb6|l zgL-5K{_DuAcMqfvj#s98u3t4gMPd8)SWU0zu6w!0s{UQem~;Pr$@uqgj7=}Bprd(T@`fi`dD`3(PD%HX@$TOR2fhd##OuVH zjLeAajeNpr!L+UMA-^X7jfzEnmxlXiXpu~jj_KsfNxf8Xa}ix}c3CA@ssz0>{*|Vs z>twL$vSpM(`P5bbulcqfOKD+g{e~Bp6BmSdh`Tfx^nQncoK&87E|PNa!=l=X+WN^i ziEg@fvjj!o{`C)$l>MRw51btqR826KwU+a{j)#ohA)hBBiZIKQ>w?wmkG4ZMWard0|oJmWf^h>H2k|WVOe2<@5 ztIyR^jZ--**cOZKoh-yGVA7G(F({6;1`~7JZhTQ{R+2Y2Af#jBwA9q-uIRD0Xe|pl zexS>9wS*XZN0BBj(e&A4$GHy@gy9w8N0~T%ihYh5$JPdwgA)m8GNQ_&93h>W9jn*y38bnuXy!qwo&KW-#$lk>8YuF`dH zV6Sf<<6YSo&N;=?ho_4Bn$#F!`V!-Y(`^4ouUuA13eo~H#nAW8l2CPmk{A=s z?>djF-DOVVNNnkBZI`b2x}B%mojp5cqo+$7oEqbhWIa%?v!S1K7@6wIdcd=-&rc|N zQzE}u1r9S^X*97^%B^W>sEn1rDPL&nX6BE&o~VCnYu3TqZA27_6BQUWt%Y5o|5Y+S z;yFWNUVx5reN1JQ;i0a@!+cKNB^{_WlTp5#rKYoz^E_9FeaG;jG75v0(rwkPY}OnN ztwTLt&&xq{L83KwLNMdxiF3GV|7PcG?0Rl~ZU+>r{Jxbll+Ag4RJX=6d(^lh-)=l= zb8gd;d(JT~XE!H%)U4E2ok@eCaC3sqV>n`xyqaj8c%6N_cU-Xwe&drFhTbEN&z2I#g11`Sw(+)?b3vbo z@;Sn-rs~+PRK_i@HiNk<9j7LkYQ|l`UG7bnVaPF`$NIc$JUeNJ_ui;I+)&{%leGwE))QsXc=C^_z(0 zFE@4(lQ0|trk#$g5ssRSNf0ZS5wPx_w}8B=D$&^@zOhbfMQ9ypAcz}ZUV>+@(Fve+ zJ6Bgz6<1d*%8y3Y5r{YC5(IID-kA7FccI!1+VP*re_Zb0wz()m4nUb`47sdFSd$CM zvQZEG29zEFMrsnq($Wa@z%>d2;tfIsB;e`>@XL3D=-0LA4O)a-f8D={fZ%V2fcW=! zWPnfDPblyUYxCptR+t|GGVs?u;MXbX=Fe}VQY78_d5wq$JVOvv6qb+xJ{1jYjEt;o zO~H0sh(sE|jXST!)ol?F@Tg(GHzX9Oc7Xmz%#_sZ)TCc<8GtRB^$fxKM$Ar@uVMWl z@HlY+mzG9$dLSpuS5~%MPP`Og&b$DEv;ERR@OSXr5XZ!p<9 zTiNM3FK)-%|%+tup?Dt4kwtp`R zSRf1R3Cm;VM=Zbk2AcA~?sCbQIT^iD7csL0WCo1E_xRBhcAmc){I^HHhlFdY_PZ%7 z8|PE_rf`q`Yg1)gBO76`B`~NR-*1Ke-5CDi-wk{oI6A$Fx5_f$N*38w&gVFA)#~5F|tdm7H#DOghHle7XL* zi&-l!=Ca-AikywAkPf+j0|`5n;yGe^RuXW1Bi%8L^J}!=} z)nbmSrgAKF=Q%Mpc5rYw=4?B?oWK0^?kI5+3j+~L0O7{J_^08Pw<#hvuiX5-f6)jw z!sokKA`5|D2scqc|KcA~pw+>r;JeuWZkApkis^r|D%b>f)Vx3`U+uIY|8k0m*ryo( zW=X*My}sTAWSB88^e<%yf_!uQSK0!;%!3DltUM;cr}!6Ahp9+Vr0&+gQWXklrU+;i zvRZ-lFGY_+u}1=@Ur+#Q+^CD%6|VD1@*6A6mM``r8=okhS41xo}i^xfR!-}A!S$pHHLw0L+k4D?(dJmpg) zME?5q?891)|8;-Bbiw9+?uN& zGCuvE%@7fSAlQNsO-qUcCqytR&}oM8$X5!;0|#MII7|6G%CD=PcwoNMqqZh+6Loti zfYobq5XtcnV%jD|`M(kB{}TdWS^rN6|Gx;~>b{8E#X`|VrCF+0rNeU@!%pT!sR(PE z(Hh&>xM!#3X%Z0{Q(0=y^;;pc12m=RnvR>Hmer1HQMJ<*(NwZAFABvBy+QgS=C&+w zM*atH1jCXily)9I5+d9LhJQ7c9CT4+R;#^P60DuxAx6f1huw7W1t$8@`jpgI*{jU~ zny2U`_?=M+x6a!4@mUT!hM087D~lI<*0I$=kOv>eCho#H=zpI@!#LpdwM^|!a2W*A z%c~v{yIZ%4mO58;;EVCqQ+fKR+@xXU+4}3Na4jR$j!R_Bjm>zzS2T%^TUIOcR?mYD zFSBO%=35funaLq5B0{(RN7qHTX$6EIyTErcaHEPa-|)OcaRoJwa2u!%4|aWd>EO(! zZc|%;dL75hv}j(RT`qm0_kJLYP-$59jhxfjLAG08W+tQdVXTqt|3$!GI+Yo#WzhVT&1vDvBLfw z>b4`{I$8;K{5*Mw;zf8YS?-c73E}?;6r~k4??EY2q%!|=Oa@`)0V{lI z_RRZtmK0dP1y-n^u=i!*D4yG4g_NX9d>Bd+*FyyLkcDoI>46K+T%Jesx%en1&Ja3m zj#F-X_Q?zcK*q3nUF=qeYopLed9_IST+-O?{^!Jma5MQHV0HA&T$I11jKzs~K@M@+ znnR*HRG}uZYyEUbA>HjcOtF{8i8U%v$V9d0^wdj+&{rl>A=zVJ-h z#5L`?FF`a0x6#AbBBnzR-rBTvM6n#&B4XR4qQY6vpK#q9NiUFoR>cY&Xb@Py+VKa? z6;G>5j?4NAy*(ec48Px|%Z8A<`LScjMHsA@OP-^QEzBh!+pf2q7Yli`yhx6w5f#B?Uf=eL~ybmax}t;viO_&oMOtbxo2G9CPY!r);@d1{7C zwB%92;*^XWBn48SWm|LjYar&CTn>8Q)DsP>L@k77C6)*W_qx77MZaY_G}QK6-a~^H7OS zp7bj3#uud9SFv*ftRc^5(O)y!hRTI)OCjC)>1k{l+oIRy36?aHhXw! zcK?dKvK#{~2H-c~T&wY_O7Sb7BbS2*kEy_2!wZLfM${UmrY>ZTcLC#wNFnF|Gu9WU>LAf^XNe!mS_;3L+KJyy~7@h!u?yj6$w?*y#vLyMlzD!a9-#H)@r z&A_U;q@>DvUl8xapE>sd$PI_BK=2>&@4uzG1`VKeE4!NyzZ&x!y2#k;&{vGrN1H`N z!`Ujrtr!HW3zO&7SFA=yXs)i2@1u=6z!q_5PGi6zikt<)|6veyV33kKOz?xa+|a#^ zty||t$0>y$<{v^j8h>tg`GW5%DJa}|b1BHM`^x?3sA|xnt=re1`*63kG0_(iy+0Ys zh-Bq-X0GM`w^*!DM5MH|^r><|z53LSsT|t>ZH0eL4MYW)I{6)I(%-q)Tby>L12p$e zktSWgw%l&e87D)cGR)|pioV9_RLCZho-u^nYnZ}r-4-%28mwn#_y ztpSa!_~IGcA@mf-M11sK`hklYV3W=C-wuCB4zW()U_qH$bdTQqnxf{$9}U$51gHe_ zp|=@P z8nlfPznXC7C5#27h`rK!^Or#hG{C(3gl+)|ZVkUAd7%W@aiR=b$Xht4bV3zT!mcYy#pjTx!9viNv2Uv810gN4#{QwF8!SNFn#l~s|J5Db5Ic&1eR^;`=D~wg z>MR|Wyr9S-#`RQ%)wuC#(WQMbF+Y8Nm0Mo@SPNOJc-@3EV68mPdm{9Mh2$){b*1IU zD?QH%2XZV{Qt9d6JYdpzn(5$3h(b}7Dnj-n$WtT&h~yhbX3gKTf^$iLT=4|T;)4+9 z89KPBLF8b;jjL`X@2ze6*J1Y!>E2YaD4g-Du#??6PKOWf{utt&8R!R4-OTO?{kH)t z_Tc*{6fVLa9{=bBg$G!gH-`z&W4NXL99G~ZtPp?iJVytYAa-)R3Vk5F$OjPw`^zKK z)F#l4Vvk+P%;Oj?$4Ix`P1U2L2fYU0-ff;!>1Y|K^{%GKu>-wOKbZW{34{tr^OS8O zdxSrS`K>O5n;tZPgKTHKK^FQQv_Ka~Znz?OQxyrs&uqS3`Jh|WGx2+Hx2zfgA&|~C zin+Laxjj-Cfbu|Dk%#^dp9E^VmWDWv1VT6u|Cvx5t^toYyC&X!12+>V?FM|o7L)D- zi6aH1=k7VFp51(VdL@xE+bWeF(wq)^Z)z{&#|{k_%6;R{-jo6LiW*YokHF>K03?M6 zpY~f7;o#ga4?L*{%FOFT$^>{8sqpRz1td#FNQr^M{A-_?ZJN_;ipJp8=}s?Yn{w4> zAl5bMkoA-lmRb!p^?W75^?^*u8KV7Iz)e{Iug!E4$ie}(U!uf%07&_kL4uRy&(#9* ztf8L?1kHU%vRx>-a^KvBK~~P9O_q0LED5u<4qM|);PO`~2;!Z9UkkD8OP$J#EL4q$d-tqKV;W{cIQVP!7}Yh`Zy zE$kYYGJ$;b!+U?t?T1LbQUQ60p|``yUIhDwE+V$8+oYH=Q=A2US2R;~o^DN_gzLq7 zy1l_r@1vucrT`gyr2f|z1JerGIzcbHs;0FvoFvpAnPr^gJN1NvQjkrxe&9?mWIzJE z0{;g-cLEDw4{zUtPe_ngH|tO+R9ufOIh_tZ#}_O2jBZZMVO!cD#Y5k|o(n!Rf+{39I*SI}1Zv_|g%j z@J#nc^wT*SiBoe_8b9gr@^=48u5Y#H$LWIkRjlSSa+!QvfFs$d26+4snFtBsy7DFk z&Wdo|{tOM?5L{qFMN<>#aE4!x1z7>5=+=~y@gr(JIp1B_G5xEn4>07P79*o`8}7?L zJ8cL7K;l`i;SMdlral6G1qpN5<^xcfZ3_O6;GqW_Q$?^q?AOZSqUoQpj1T0Btrijc z(t^*45H_OsHyD82>tX`KL91VuGouMCx$baz3vMbWB0$)e9r3NDgDd8|Q~}p^i3o4} zDUW|FtOt0wcK<%yGYA)8ESDAqcPP$=-FBY-kcuiGm4$fCsXq~)4dp-s_x0|-gW6x~ z+<6D6Iq@^HtD+C2vG4WYkfRXB^3l`y!jTofVge`$NNE2o1+eU4;vUw*5jiM4oiY1s zjueHkw7^|nL-2>J*Z*n1*qk$yNonmH*0)}zWv@FyK3+|ULd$_V}pXMire zvnYX?*Du9@Qan46Wqo1YV>tr~5S^-A?1VF@UvhKr1_D1gFas2#m6SuEPz|2?yy5yuiVH_tzSe0BPT{ zn{#>v_uXI1tp{vpRxvE8;rhVSJ0QWUL3pi7c~%HYF&JyQRl2(;n9 z7aqS9h~gv+%#Gtd!L8qN^G3M;tmaj&)`Tq&LL$!|pG}opp=8FrY$}hwXCIEKf7+xQ zEg*6KlZij2e2@a*aPJ5DJ38Rp5vvj@A^t?mvvK9lo++p zM+i5zVQLJfO#MA+0Hk5UtPCMu^M@#~8enpB+HOJ!}hWxWBm>ZZck841kLj z*ggAML&3nZG4|S@tQ^Hj3=ondF#*zVA>BZ9gVh+wD--wry6KGo4Dz5M{A(&)eE4Y{ zC?{orUDYR$l48L3par8F)0;W}p6Me%W={j<;Jg~ngt&nP*3N%}PY$4&Td?pWupIeg z(ukjd@P&}TL;5E%HWb3b*Wtm=Yxot;02wKTfu!qi!Fa(eMg0xeA6;V#792>#ME_As zlfiUzgRA^UItFJb2H=43 zFBiw!2?nIN{freAetstcz??CADye_mya8C!V=tNzu|J7%#{v+T2QM!VF?2Le@oL>TpG{sAX3u$n{%@Hs=m#-h>fB7V!y9VT&SHvP(<$9TMT<+vb^p6c%L!Vje zOT~>8Q919Wg^|0{x5Yiq7YQNdRhqwG1;6ESJ5j4&+8m2m99)n9yx(GCq^$Q!Ai{e!;~pI8>_abmV>uPi_Vkccsxw@mX{vp^U54 znnUl8y0ti2rQ!*3sjX_BVehIlwXqHRHK74zyF(f%w z`U+${*XpoN$aPxom0ADJ^_cWqM!svhXxW}seG<)Z1+Nn=HwGE6FyE#ACy&8u^xO`X zaO72cFz%WA6VW;n9zn7K_gGO@ml6E`osa~SP$;ei>9w|`_@8QD+@UK13- zZQAH{{q#MdULklxhoaKP*nzd!{F%)LnOX1BU3ms_7ng;OaP8KhHJbDQd=zUGSz%Rj zj14_x0P836IX_H1B6w)f+PS-v{agv5|iRE){&(=rI z>zy`@F=C6LxfSb@bntbr%`j9bhHC~ z&N~)Q#zfjL0SiT3`Q^^{s`^+k{Dnl2u_I2aJNYBDT``f*+08Zg4xmZ1o={SJ7Hym| z{q#X>pxBRobS8D0bULn*X?7ROyXWGeOkypqC6@=2Ik$;@MM(glLlHz009MOkR3Qtt zfbL5EDOjO6DFShYgN@)H8&woapim!(`5?3vecEO8`P_F>-(>tzF{Vait_Pq0&^cTd%_C+zH>|a%a?bhkun_ z{9Y+l>~<(hz^b3qkJm+CnguO!&%BD2G3W!z*yK~`_XPZ6T%Lrb?nT^!BC-YVP&j=c z*vZ6pm_86eypO+>{tuu2v8l@n&)A@dWC?(fusb5@sfaui3F#B@yLH=BbcZFENqysl zRKep-)^|-VUvf_6u*EJGF_os1;CA3gQAyl09~8e_mB6yzXmRdK9vASHO1(X3c+kTF zM9~&Mhui=SqkMz%ZIMcDQfZ>WPy(Ewd9H-%vgOZ~5%sw)igh)dRIWp4F1#ssv2jIe z+|Lv4)9(kz)xZ9-695cQ0_N38G367kR$iQK{T)#xr6IsBP~LEPvkwyVv;MssT3#uxw7T}vYm@!coDp0Ktf0qu2}NTJZ=jgm2!H= zKKCV#1sr|LY}aKprP!rS?HvhCSafOzx(BjUbRs*+ z(ue6p_-@Dwy-m=}Ce4!D zbajiyaJ2m}#YacAKqu-wL7zCbW?~bPtyJB_C@;o(c9Ame=63k1loqQxz@t3`1F9=8CMh+!M!56#v1>-JxLP!}ev z>{Y>%QS9kyFFw2|w0H}I!*BAj;SOmshdF5ympLjC=f`C2eT(ac-F1{UyL)V|phCGT zErm2Wt8*>Rqd}HpS|>u0i{rDu?5o=Ov@}+9mg+m5iAh?Ar35ev>IQV{{^R1#rt8ntuE&-Y8-9>ks;pD|Uw-p#qx zUSiU)Jjt(BPk)bgkyCP&P}mYe!)a@HPqv^JxrjEaUK_|GOhoM4k>QzL9YL|>iy51a z#M};y5@FAhYhrlbePoQ(9hf$8toJPNfvkC~mG6(YqaW3OIo{&MI^+OpMjj|+ic+`5 z(!_pI)ZjQgH-7KN?d0Q8{0o$%H37-MXnu&d*rVU7((qi0f{LSB}kTyL~=A0PdjveFuKmEH(WXit2`Uh0VT(6S<05hEy9I3!YYV za2$OOcj&sv7Shn>dK#rOsX=g4WywS3aMCqIJgWP48#Ep{PiEbcejpW^u+=xq>%17@ zi@J8a z`QX&-0sty2MIKXHv?|X+&@Q(WT_3f(bro$%V9CEtC9E6PPT;+ZT4&t$QD%ra5fn{w zGii;Fqze6G5zg{z`HN38xSWm~juIbfX&>Yyq8aIJ>xvnTg!FIjt3d6@+oEWyVWmQg zywW$@k*rl#(-EpDKl7vjXnsQdX>p0>xI?V!lxDe2Sd9)ei}!veG)#rT`nKajXT0>o zYaW3B+_qFl=iQ5Rv6*T2Ah0hskgYCXw=t-LiucXz znMna6zKD`%AphD)88MelpW8bf)oYSZY(}9>pFRnK?-l@znmp1=|Ep)6i^sCn%R~`U z9mki|ekWbP>xXXlL=N6^9sh~Jg`fiEUuqNMelQ#+6H)Na!V`HBTN|KN>mZ!ii6?E* z@v-REL6r)M6qfbD$!K56_GBEu>}r5O5r$;W3p+?`(^ z@eYSNG<&|T2${^+6=U!(n{`{S8y|(zuM9Ns&ofaAsik7RL5;vYkqyBl zO3zky3ceg|nv;x|*_ir3a)+%xxJXyEhttTGbx=DNFcuq+A~r*N$0TwiXzIy&l=&`O zPS10-aS{@nU*%T251CXfGWBIjQJ?NFg_c0`OG4tG84w`mYNw_!Ch%$z*J}p*MmKwe zdz@+9Mk6#q;cX2f5@Q|NO24UtNy=waGohn04!JI~_a5*l56>I_-2Ym~vix#q)}Wbv zLBn6INWcBc@-9`BWwmi`b7CS>+VmTZ4|1B=kJF}Ivgb~X_QzI_l+lD;gHp%;Dp+en z;;*j0gy5?dk3q#$802#*c}&Ket`4FDBNxB$PepT-nJi2dab#Apq9zL+zQ48v>aBq^Q#;UkHcQxQXhMeDU%qbzew0I>Dr|^ zmGR=k_qukq&2st?C_6Jf0DO9w@9vw3Co^o`UIL}e^JG8zIF?hn%MBj&6{H8*&CV&S zms@9Mz3#vAr76|hNQ@t2i@^5fe^P3O>*xzH*PN}|_RvQm&92m{wacSr$u(F7uW#hi za3zdGtuj8+#&j3a3AOLz6+~v_yS$rsd?xySWexd(abSS?cCPowg=ZgOqaQnn(?mx~)&X(T%S;x*Gv#*Kc-!z0*G= zp}0KVi7gwg#Ih(Kil=H+roG=)>u)%bcFJmS#M*3M_G)@K-ByfntTce1KeoF5&LQ*C zvD4qEdXtTbA22zMHbu)okBjXmXHF8~RAGfK*7ssy2b-oG>ddxJb5J(IX0Vs z8lnWuRyat)-zg~a)i#U>KE6xrs#EF`AIGs-JpI%5A&`MfjU(C2OT-cm_Bgy{@@z;TN`ZtQS zp3HQ*PzcjUHIeqwt89T!nf}$u47OPE)rr8NM6P48wovl2A;Xgzo5iY+ITs0Wj%wuc z`QIJ3Mz$lzWvX#Rl%~Io1#z`eYKlv62aNfoD_{bMmnVr~3SaJiyQlfNXYD)#%BkAegPY5-I{c;N zt^Kz4sMRUa(Xxr{F-Nk_zy;|nw#d`#he}Cyhhaqujd&03?Rh+pgaTAeDUdOckh}-+ z_DzEGifb4vKy&Lxf$cQ}!c5EB=yn??X6Mj0nrqI&kldzK`ZO#1HxD5vn=^Ng^5e>^ z2Lj&LGZ8Z_O5zX2#PJQP95eB)G10%zCRy8VK018DRKKs02)UwyJm{U9z=S8@|1R7? zIRK6Ip8iRvD;&DdlhgXEpDut+ZzF0&eMwdv!L;#{BaUh zXC6^`<6J<8p-g0zJ)@qF#1G`J+ z>0LrbgzX;;u2(wOI8rQKRpb< zlGSk}W%ldo9oOw&I))cgm{9h*qwns`eUZwv$WfxFt7jou9w=Y~>Q1Fx_swZ2s3rgX zpnJx3@%OrjJO|mboQbzM7G*hCoc2o!W|u{ev&zgmOR;X*jUgPr8mC^SRg=1^^^meC zMt-D>)xQQHR{5aJt>ey_T~_C_d%{kAYdT9%#^^qYVVTZ!?OahSq>Tx5ZFMBDZ^S z%rlIvZ6AlMLBzs%^+ILJjVDqGYc^eaZ@=Y4NRF0&bU428J=ZlyAZzT3+a&mfANhB3 zPJ5fJQu4mfQ$&6qO99GR8@@~6j+V!|(bZgows?6==CRhBBS7)AxMmO34O>?a2e@OV zo5ux3%5wD53G)FW;mO(bO?o&>$+>s8ifIsj-_UL$g?D{*g154 zsZQmPo{j(GAvEc;17bOSN;S}jbj2A_AxZLOd{$@`YOaO* zJil2^YQLNytWeeG>Y`g_xx6W*2?*2lY8L+pm!zypue(R3uSTY z3p-n1TqcXKhCnIokz)@XdyDxEXZy-hppY~y)bLVboFBknWh#{SUYVr7kQN7~2-R`oAPsp#WO z_%2RICoZM499irA%Y7liR^{O7@FylL63IT5#If3ugKHmmb(pM1wk}7ebg9LT-pdR} z`$Fim&byOwBQ>+QwO?7tX?Ele5|vR(%)POhck5A^)TjoOY3syU%0LJlRmaI>upDW+ zTxeWOX&SqF4LGR3`V7Gz9TYg9)bj0FZVe8+gNlH102Ys4wZGMqlD=heZRaWEK}_fj zj9>)sCgrmaMz&#IPp%%Q5$L)qGV4@jlwUu}@^{;_ep0(7GrRLaQ6WjG2i3!TvGrWl zK^)i@ieo-j!RobsS;jgB12>(C;~UAqj*8pfLbd^ZX7QMen_@Oj`|KAnWFW z^i6NtN%O7H0l;g>is4rM8Y6FY89-z-(nC&8GxO73CXZ~kuWseXW<_ABGM}l#$hh*i zN-KllH)sTpmA5n85+){aa`N*I19=%=#6ZRQ)&_LzxvWzVZT5xa1QhwPg_7k1NsC{% z5!Mc?>Xt^;)%pm%TzkVwm3dQ*9)b(he+w_d3i6W2&y>SAm;6Kn1DuIw32=}l`$gH} zT!7-Q@;p|K#mFOM2RNBX?$?uaWxR2<{@J}_VYyp(r51Bbi^^H2BL^S|fujZ!0e%{u z`^&z_Ayo1)eSXHdGr7Qi1Wibyq|ZS5!eU3#pp&nwRWrIYuI(d-!LY>5@sZf7(=RgZ(uk_8X4ghSByi1a~rbP_};mxRr>>c5r(9{kB9TV^J~_4}<$z zNaVJk%m6r9rC_4kC_%@uya(VH2)JxwrfS=`!S9@NU)zcoi1fM@TTZ^XjF~|_-sb<< zV%G6MCXxH7I!!9Jug6%~xdUpxxb69BP_~jtKnNvTCi% zVC}9H^@kOT$bHkgv{b#Zx6+$J)QXy&q9QN}Z1zrB4)k;{&WB75uGcXFj+f4N4EH+a z&%B5Qf$g+Y?r-c_!39?l8*c$r3ggZ=ohm!DvGx&jbKoU1@;f+n6z^~zhEhr3NB|r% zwe=jLincb?nNWmypg!-Ut)C?I0SS!44QtUO?;P_{K2e(AXSIC+lIaYu7S&hmKk#Iem> zhGZPr!t`4L7qNSMC=N2bZp6`0et3CMR&%YR4ZQNOg@_%Muf_uA*wU})B8uqRjBVT5 zc6BnydK4mv_4xZ6v|mK(uZ#vL!Yds(;vj`b2*0Tg!a}@B2b^BbP+(Ac-ZE>3WxW6p zc}12u<)ppwpcH2*BetC&+gKRiY&F?jq?jGhX*fL9?9vZuOq9rUkbaFsMg8;t0ywNv z*?A5W6vyxCyCpkslPn}Y!7qJyzFe~^oYpuFegN<^lRXi@8^DZa`lrVmw=;KrEOtbJ zn%u|fstLE5NrjP+()_NvW3V(7caB~KY_wAKa_LdFYGwe;cb1qUc+o}kVU(Eph`i@O zn#8@fLtCQ5D*8{p5b$sTBW&Mq#Xx!Y`ELO`-Np_fKL3ixp1@rHU32z6J2XaYFhfOz z4LrMLg|!>i<||SwYZ}ZI#MQBu2irS} zPg5sW3AfyV1NhvJIeu+#!KEEY3XEV}Y1toxxBA-vku_AZ!U7~PHF@66MMY1^_WNyl zWE0OV>gEm?R(AVcFv|!NH6ps*037HBkgGiI=HrrF9>n{h$vL@9x%>BX8A4u-b6EHf z4%M#30r>-Xd4y=lRXF2ao|kavI`BDPC@B~gfB@d7ukhIL zT>Vl%u<0{dro{J8kJI%l=M+Gm81kC2+@wna#?_caZJ)>{#sem-<4yxTqFT2-GdSq{ zOT<*y?Uf%PV=K=nAh8H@xnchuX!MMTbp%`9N)3%HRcF??o%Ssre0fhGJb;=-9f%*Q zF{m5GK_3;lK+w@qV$yxM4p^}a0k?q0)BJqhWtrK|D$B(XdqIZuV3JYQF}I$E*V#a# z@D_e3IejDbaD8Ct`|e!p;CcdHdjuUt2mbA#_0?xn6~X`Y5E)#?Faf07^{^D)%>R1z zyflFp_G;zvv}f2OdpcoS@OV|U057QO@~2}F*aDO=uH@9N#D-6}pA=07MHEjA3rIgeUB{Cr=@OV)Ba-(74V=L7x9 ziFAEZ#NGO9y<)fB!H_e{oO=1L)RzN2ANtcwrf1KdZN2K;kk#Y&#E_^0Qt`>sF-iz% z&acyTA>U;)UWNTV8B4)OdUZL}g-&v$y~Wb&j&O$DmUl#&*E(7Co<-q!Y={pg(J3%a zp|N)B143jS@l6(Q*S}3?cCBg;GZV2y(0}Y%&zX_IvTmGX+q3FOV)~e0H|-~Q>$4En zV+5KYu|LIi6cj+*Hy%s#P{H9T6gA+eS*Zf{ZWJiRrOGFrusxZQ<`6mliT&pq#cOF8 zW}03vr?MZC^efQ3`ox_6iXkLAq$dwzit*8?J4P`08u#esL`!yHn&tAf!X9{(5 zE+HD@$^IUIjlAJWCc6zjIgf-# zvfEe25v;{FyRt6YxIn?;10m~xW!E63nR2sKLKkg@3P1suovl+C6gu_Rnl^ZO)Q($I z!=$nUYhKfsEA*QhY^c_U1J#aX;(Z6L`5l* z6V7F#^45hFr;$H$KHH&h`$S$gs{zECbIiCT=3Al<1U<$+oVw#`fXq(rnz=8vm%TW!zhsChc`}l2?Sr;MUaO=9#nO-LmObD&Uw} zUv(~%-JnWUusyt-?t^4wKIjF! z&-!3T@!MVa(D8Hk;UpW#l>G4zNdIv_M+kO!&vHRlutn!m^jj#l;vuqfa?R$t34zLJ zzwB+rm8(-Zg2%=g9z%nXb;)-qbZYFRix={(P+GH<$C|2mZ3EeBjfN%`&-QQQ)1$Tk z-jdBfydBE1RysH>i<{TvR@ltr7ERBtltPn+b!G{=toO zWla0B&8|cRO62V#W|Bl_FzPP*bZFhrqkx`%C!!fDnE*v9UE_9dD*yl3d+VsGwl56y zs$d|gfmf=G9R(%q6uH`1U|Qqm359nu|wba!`m$AJTH9TmjiyKjv5-hc0n`_CQM zbJ%C^z1CcFezU&Wvo;>#)_R-Z38TR%<}Fp!ZmjqYf2stm(Zh`GAIg1wlmF}&*KFUW z?$SjjAQ27Xq>er9&F#;hWn9?t=Z>c#$Q|`>vddXR2ZT4K)D-=y z8aaY>SgZV7l8M(&!hvq^oSGq~~}z zRm9zYQ-35qv`r;fIAcT(HImuLSt|F{CMx%I{iH6jQ!`}e^!RS&A_zD%-)f&60I6c} zj~*+>GEHw=yopCLhB!hmqFmt=gJ+dwIq2D%LmRTLL9#U1q@Zx{g*D2zY*{Z#r9#&+ zr>(9o>(1m-$y9Y}p2+l|?7f(^F>^s*@xe3Q4Q2gbA{*H(yGTG9VPs zo3mt?4$fo^8Z}q_9VraqaW;*UCy;LbB#7b47xHv2*5dGC$l%M8Lm zC*%<`6%Mplak6#3c{_*gZuz5w?Ey1$yI--=tP~rnA1lcIM*D|)XgSB3~HThBQw4?P}wE+C}kn3aOX!P zPJcH^1$9Wbq}~24Y06mx_(}Y8{_2E;13mY=t7&xZ+g65ARD1~YRx6pZ{z?TFSrhDL zf?~@7y)AopV z<|L-4$dV@|>vj7e%RpyTIgyNhk$z^!TXNeqv`KlDRaK*Nh?jtX7qMc!Dr7N zA3hljcsE;ZYS7a6@vfGYttqEGWvkVE#CB^9dbL2*ZtNKP%S_OKYd?;vFNYAU)(XFIY5YU^59qft#sq<%Cj_hOFO2_>br0{2!ajBeL z<8~-}Rx*!v>mK%O@Mv~Ptb)A)y_k9;b8efKgQ|^Z%j9sWmiA|Bu0h$SF>B*a()xK$ z>HY8^n=HN2RrDlcUfVw_uf`*GVB+ zhHw1PeUlx@Dn`1IYVo6Ad8@!w9p0lk`M7wOC;DjobRXjJ&w(%`9Y)nOhk#z9oj*uf`J3 z=cO|mQ<{c&nwMYi^H4zlrG3@=Gs!1TTjBUq+03Vm1)&y4so&yqXedAzPFiQ>)g8O= zxC!!MfiBka3d}1S)n6G@neM}#D4|NQY3Dc+YBhPJx!rQ4x4CybfT2gGl%6}+65d<| z`HEn>K4B+cb?jInCH~sA_rM^A@Xq}w#kH1L1&+Phd3(i{wt)zpsd#UySceVqYflcS z0z@!cb7JM-(Y@`;n@L4SQ}v=}hPfGnRcY{t^L2ux$pV`xa_R$Tonrk71;m=)j8?BQ z80o7Z#Tv4|#nTixpf6yOz0T&5I`N|-Jy}l)crL8h@1+Y3Jj#KOE$N2OHD;}xHs@h(rd^`d^~Y;Mp{g)}#vN1xxXm;AUlB_usq zL}Z5d4w2QGJndFShuf_??eu!1n;JO`A)tqJRWr3(;UmP_ALI3kpm$jJzLm^Su8rU6 zi#NWM@)ZqT8}btPdg#e7ackkxWU5(nv2jQ$93+0+WisjJx*PWAm?XXM<19shiyt3a2eq&UX;&vsyb>t=3*TG^$$(AnDJb)o6QFsgiYYJo`lz1{gnj;5zMP zzly1kHqI#@gAuts`A51mGK1CUh3LxqoE2R!*Bp? zCMK0(5h;7%3EAZ~*mp#iA>2U1-3w0zP)k7THbJ^|6tj6+!P${W-KtxGYteuLe4X1F zV}DV=m@*8v4XhpLewGUq-s~I^97UP9v;SFt)C%WIa?yH7r%K7p0W^R_2-IL6gpSaY zmsPmo#ru=87U{@Z%Y*)tK7*ld+OU+4Sk@BdJT2jr&k(c-eN{j&(ay0(wX*r4+#P|; zk>jhE1MpBmwa_tx;!0>bRY)hU65(^rA6=u;wOG7S&rQG9=Nm%Dzhi_r01rGaM~2vG z0xJ(#-xO>5LnK2~2^FWE*xjB~8|z`GXK!2Wc1vU}3&~gJVj&_sbDmkECj~yBQ}no$ z>587akatcZSeuk5X{g+lCLp{#(D_sl^Tr8@Ku;{ANMQXODz8v!<;7|{S3wZHk|H}S zd5yPF#t39e3Pi>s(iH23%X`6;kbRUC=K#ex{n4_o=q;HyE37wN+EhvuWlyV`i;%XE zOV6j`WEd)`K)Hluw#Az?wc_*_#>%5vHHl@6c_6G;UZGH&H7*=n$dje6kfOtvwX!2+ z=vQfR97}6EhBaKG?blkhl89jMxW~n_Jdnm0CaWYh+IhzMF6PwWE_Zs3jEO+UT6kjN zdihYE0=C3-;}_%)9{h1zz_yfAs+CHVIk4B8orTa~QT0zg)wO?wlC4l}8ftN_k$P%5 zRnjY;jt@{$d=;S&W)(54B!t)Vtk|KVAHiF;QwyEfU?kNIi^G(kdH+cCOu4$(>_;tP zQC4cX#d|Sk$G&>9vA6qQW58Q5l7C4%BSFTc=Q-Hm&^x9aflB}LO)aM=zR2ic@FQM) zbq8YSUv=HTw*qwu=%O1AH*FlRK#%xbSc^`=RzV4L#m^AGzJ2ttnM~=QSBCnK?-*9i zLHIsUH5dER?-c<9Fsr^PwKxayi41u5=6Z7&bs^VP^C8WZjI4j_Ne^LJiC~@I4WXY)VQ=rrR0-QwKU0H8*=dDf zO644mr}BV}i1@Lg`NN(=q8EdyGRiRj_l(J*_KZQqa6Fn&iN86N#T|4M<%38J3)aMn z6A!4TR(Ls7R4A9>#%_0}+>tjd#lUS(^pk@ zO)&6qzpj9F6SMew|HX{O4x=WxVPW*M@z$C1A8-@Ispi@4 zf5UF4FoGa8F%*=GZqc8IC>z*`vYzRSWe*e~bI5h~4E7aEZJ>%71Fd+sn7L2_3`&IvIqXl8-|JrDQ$MOHoHroGG z!vE(~!V$;8#trt*Gn=zVeiL?^m4uZDk~eO92uq+nKt)A`yDJ##`+!UQv-4MJJ~Uob z@q4xsWe=7sA1+rOJ6R1Z4wSd{&jpOWWl*%nDLmpiWrK7!?j^V6Mw%d3hLHx9csx?0-|& zVK8-{=I(yc7ymQn>`V%NAjab9#Z&!EYY|}m+x2*;kHFu$m?olMRo1sZfB*pAB|Qx# zjPv08M}fhAGdQncz4Co^JQte%XAq$VM*S?t9CXpG4>NhXXukh;J?d?5aI+f5p1*qK z;kbZ>LA~si$AAUPr)v{duddUD%}R!SxiTGh(H8&n^C=;)Fr?Tm7xv~Sv&8Mjf4lxR z96z|3I@Y_tI)^_1(uC)?FGv%v-Gu@Ft*P#RHQQ3)%NP4wT^IZaKRxO#BCs%l$lAYe zIR#rpg-@DS*ZI=(z7EX(^Jl&Vf0=zQ7=sYph1qv@1x8!({npaI0*^G9MYnywbYbCt z<{{q0=mLb?X@B3+1gz5%<@bN%_4?LTIpHB?_z)vzN)g5o9@Y zs>5#9;4k|Jj-hhm-3IUV@e=;!0_F7_vZYc=ofRzAZZ;YMPSq0Du=(w9>{X(@< zuYp&>fZy`3j*v$faD+Vmr;a5IMjCeKCjRI0t>I6@I}+8M-^aCA%Z!r@mgA>8%zt;R z{*TM~mzTp?4=^wPm@mcEo8mRWU@hxER>jp0C)tSD^sku<3`syYa4v=C*hbwgKP~72 zw%Pi*E-x1ShW>EnVJ^r%MN!DXe8_KrAyLBs+a?C{UvNC=h5&afe71ho+vE%aet>T3SMdYjv3>sjVC?a~{fZ9kg5lZL z-m4A62`~pu_x&C4AIhaWuzZuV9q?B+44+>0xGsMVj~2WSVM1f!UoR;QuHc%tE@9&0 zy@ibYb5UNwh%*y!@DyKNq6@c*5^z{|@9;F;xZ+cx+QDaH@!IbkrGX*{-j3L6%Fy#F z5|FtLj$+CaJgZhTVtBVASeTM_OQ~5@2_TE4ZONymzA25hOZO?|J`=T{yl+ ziCyO#CL2i9Seb*s=Z92yKfLBb;qJ`dnmc|A2YU}Z9azTl@obaSmAhZ~?6<=JS8@Dt z!zY|8i{$L=cU|W$K@6j%=uPBE%1~-G%l9AN+$a-1&ik=vuG^^+@J+2PSI2?`|(#?@9<#X+UDrWO-! zcA9g%QUnJmAA9SwuBSf6$VW%?2f2RR^o2r7$J6&__LB(BA@m7@Wkm=FT=jivQcV=b z7K7LQ-S@w|2^LOD{Gt=iEQ7%0xRbfYVe$$Yn<-5#{q&&|!$eM3!g@@_&)juA77*j- zx|BBmI#|yfsEQYZ@4GOs21>SgY7Dq|`}ccwsh`jhAY;?Zgh0zwQ3y7MEIDr?MkaEo z)GcN-FUB4Ou#Vho?idBkgxXymG}=1_n|$r8@}iOZ#`6L10?1{|7SpNyWWXcwVF_VxG<(nQfxXa`5(XT7z8X7~V&L zSN?L51A!=$hVwg)9i+R9?fCC{5-r%*w}&tQd6Z@9-s7WE<&vY~zU~lg>t zd|d~TV1z?;)cR&qrj1CV(Yo$_Qb7MJ2Xf~l{@sUb8Kq-t#is5Bo2=ro!#Ueth4YQ2 znkob4>}lV<^t|qq`kMQ<&cX=ZKI*8z3g~EsBOSgq+GVK>>{^*^oT?>Sx}U}Cm|}61 zr4+BAQ5S;(E`!fvF|7S{uJ;@F_Dc%({1Z}=4EhJ>HY!i@^mkDpltE}4|NI)fCl8BA zCb|1{TvehMrLbYJSf=MK_w2t;z3#OPI5b1CZ}#0pX%8iTSS{`Mvva2C>`2i;_MrDi|SVzr~9D|$Fq2(LY6ua-8I=`i7VN0NY_VS_@}8MdNdK5A;GjNCn79rcH^eM zz`q%fXRxLfDdfA+S6b=+0TknzU{KZKx}5gRP*oLc2v2kx9fsf{iknCkE7M$8tcZQg za#tK0%`H9I`ugQoA-BDEb2HoYs(R(rz8iQNt%FZKE5!jUG@%P9Re@MZ=qCxwT5 z_*4#l^fxONZwN=M$q?BUy)s*=G0sl^#bAnu0aMTLAKM786qH;(7aSk3@!~_JWLF;+ zj$6I%Ly=^f7D;7VNwb$o)Po;4x(#YZ-kx;)6HagJ`9e>e2cr`gciQR%DF%cj`;XD3Qbh67B#A;ldER?u_+t-mrqT|Dh;KnSfJ&dA$-yLP zdu;9Js?<{h+5e_rVWar#Fl171E%m7FY#s4uaNLCciNEEBah`g^6FaUb2jHowR@;iA zcGFOkUzI**1vtQ8LaN#5>rYK3BIsw8R1f%7xEv<^{p_s0pN)V25x1k4;g{Gff}f|} z@hEm=xbWnwJ-ZEt zKvDw#I-w8TCT)ycMWP5;~CYctLsm9bh zJ5TP+_`d#uLYaM*gqSNbU{@tAHPiCv^}GkTHj4SEWbC(hCe6;zcoJ~9ct3u?hCQRp zBY+*AyCGBlo{zza_~5<$e#zLGrXHd_EzX@wFj9kX{Z7w9@PlAzq!y^tK<5u(zbZ4% zA8UXi&~EyI-%yIPE-X?kniKibgVA1HZh5>ix?9c3+VM===f%5imja{Xczfu)Q|J0v z$EMPuVAEaW){nPPh!uK>dcKWQ^c=J|1dz(f3Pln5n_JpI=5U8|+QUlB53fnI3f|{M z6sZ3>FU;i;s3LyuhV<0$C)+kS@)#z_1PvlFWH&je{4UZixAD|5TvDv5msP;vqY$(> z+^eS!b^f~HNP)klzFh0jgiNAD)>yJ?#BeNeuu3fw7Q2F*PXx)Vz+q5!v|>p*0jXGQ zXbb&hSE%~N*fjm)zLfk#kK+IAe%{X^;WmE~;x1DNtc(@HAlz(?Si1sRHw`?CyblRlGQ*@5qyRJfcxm zR;V^hmL^yGuIsz|Fk7%$xP5M9KQeFbgeLI~RF67`&1|pW0rhS?tJ1ebgWJ*bdA=!u zuo}=8!x_0R@=1w)<$^RmIn!{}dp)FS`R;m`n23lM0iFB-VN9g^g}qpXH6Trdbzf#` zYE}Ri&tm_u_m|H=K&c*#rx6~&rv;!>e8}4vChgR|Ox4;|x=oJv%8Hfaa#ri?PYspm zLvl!vE$rHTFpG&Dvh@OJ#H^SS+as1bt!Da|Y?k|tb_QKE{SgSVM{`y5=nZISUS{HR zI514aQXk0_zv;D}AMEnUvm7s$EV7V@&8Y;HV~I`5LV4QlPg78`_l}v#f38(MLm@HS01&X0p;p zwmgs(`6`xOg;uL+3cZ$`t3K&2Nz;c|*_JiqP~j*>(?p(89a_DnlK=v~Jd@6&xoxS( zyFb7v5VLeZ8!adtN=d07(rKbFcsBDrNiB%5QM9#ae^Mx1l5fpxY$HdxP~k4iK_G>b z+Td!UXfStn%gM;O&mb-T3DA<;ZScUf}_c)*Yk(`)tLrt zs)Iy4^3}r^rmEhX{JJMwg{9Qtq$r3=Nfv`jk!X9i@o>l(K$K9bsHc{C>=Zl#V!>#d z$*q8Gi&!H5lVqj|JK6RtE;8k3eo^dpTvnTa-t_?6f7E(I_ygQ_l9=B?PqCey(3h_K zkK7To&|Awp)1(0m)>aCo7U^@%!R=Y&-|p+e+>rSc;*33J7^GRId&sB9&)YAaAXnD1 zGx<`m@(cM6G_xD0E)eFkXfx42IDXLp4&%Jqp4 z9)}y_fr^EC&Ea&qtVJQLHfDrDsX8LB{0UqeArCIk*cog{gPb?ykAn37$vE^sV>_g7 zOQ(u+0p+B-mn7havr~Hu_fccE6%0C6iON9H&c+-B*&nm0z>xXfNjAhodGfFF00cee zNDl?Qg-?}BQ|`Bq?ghbn$qIk}bZ2iqDnpu_xU`T{1(xY!fnZr9$o#mFYG>!UiFF`6RSIJ6*&&aWh&Kah*nC)P{P? zJ>N=$p2~8XX>}4<7nsa<&+78J;`OhF7~0K64UiTRi$)3F+}n6s$!>tTA?t_B=FgvF z6%6_YmbDp-CPKg~toOhT+5VtfQLmAj%zFv=d;M}n z1{+R~YYcBZxI}-VKm?;X884MEs)rfC5E7zoNPlTzX1Ig3VM3oL^gTLv?FrB+WP={l z>AcOKg<^g-w|1hqiDuYowmP6Pd~_npWVaDe^};M>Y&cPCn_DuK)5tRxFD5HOEe8}* zhj{i@1x_ba-$NfVFV=1e6P;KclLW3Rb#owTe?}T!JT!zzh~Rh$TCAlWD|(Ppxe?25 zE^!k%mT0$1jYKT=Y__OMF}_UNlY0kv8FTwO`@~Yo7SyZkCFkPl1BO)PR=5@lF_KCX zwqvO=Jqtde--C02Qd4!x2vsh#r&kVV_Gi(RRVcJCJ6p-SPWlTKM>*$k@t)k`uv!a4 z6F!9=POf_?Z-i3Fb=)S#Ad{@u#{M(2a{`viy&sQtiA(UhzyjW&61MvzxuAODh0Ach zZtHZNyY|*p?LC^9!)GzUA3Y>IPY?Ss^DK@P9TsX68pZVU(I!IM6uZRWH&^cd-^A$55{&^{2KYM{o@uFS43+*=q#zzh2;>9C=temLrkima1@%hnpKH$u;U#lV4LQuEA-9f@dD0pJ1i>sglQu)ygio z0Kf<8u+Dd#NAQqK7X@~VaNu7Wju6SDOK;1Q61akhS$qTiFJY)xby-3n7`YqEX!`v& zar!ywPEjzKR0EJID#bn}%Zmsso*RtaFYZRG=A}nU0xKt(sg2XWB31QPE+7gb*yxL` z-JdQ)Ydpc42J9@S5CCvNfepX*!DIV+?QMdQ1n@=wZDKc9*Jj{j!}e^kGj{Sev240D z*`GbbTY%lSRI@ek^1eDi>AoHCk5jvPUo#h_Vx#6(nrcdSWb7Vd$>djLK-_|`n9Sf_ zHF*E*Mr+x}EZ)~$>o(4$RK_BoeD#=|K89j*YfOPQ2^NoIiJHCGws={*A#m7Ir10yz zy2K@1c=waIKgSfk3a4EbVaS+|?3WuWF=dlUxhR&P1ao(~XKn}{{*rgF!FRz^7fgNa zzB(XA!;ACeU|sviXY_{Ij&M2wz5aAgM@%v_N<@85UE2 zN7%A^X-~jQICsKeO>#WJZ&>s8x-1A4p^O#k4|VT&kglWyHbbhRFZ)IRMx*jefY<$QRnGh7~+vI~~T}&mb}E=RByuCz`E-yHQsp z9}g9%Lk-4z7jt8sZ@Gj$0NSdpJ(2IeVOWpWW=AWWUSC#Rw->mH9~f?^{Bi?oxUWBF@_5UE z2t=EX+q2wiJ&(q$^nA>`^KotcSgCpWA5r+17#;6zf*tA$f8wXB{T5yPT0I&ee-p>~ zDWP#SvHd4#!hrsHMcW^%Mg$UYeokMkO1XkqiC4?0q+V3$30@@VWAO_t( z!>}8dSuBGm5ftf>x?n{;6 ziJ;fVsF8&lf5wYH*HQlM`WnEB;2ihPk7iK_gWm5gNgSdV2Eg692L!h_MQrmM`)^f% zH}!KmV|&RTPFOrM@UDc5jr~`o2=9MN(tZ}KLENgG&K(JZO3W8Ua~{g1N&dJ^vd|tm zsP91j`>Z-!gYBE}N&Pcu9915C;AE@i*GxTin``uUHS%@Z=RkB3lK+50_LXe5@;lE(0*|+zZ{h=gT^j}~F5N@? z0hYo?eCA+%v~r@rY`(?znM8X~7>ycx&VJf2NW*CWi(F&!hwEPGe7H7P(}eOYm+=(| z&yA{yX0c8LD8p=LPN;Gd_yR$(ju0WU#STM=E>L0+{P>T*Zw;H`xQEhsyt@c)4a``j z6(>~E8-d8}(>u*i@S$`k?fSFDW>F$#RPVC zf2E&ICh7lC{6`(>TA_hzso8vH_we~9F&FQoyFLi9Yt@RKQ8!U~f#$vTYJ zSnK?M&RER-1fJa-sW)9F1E{bFrtaPlavAbpzHb1LXUfX&x@53{A3Q>Exp6KnMQ#+h zvK8s^l~^r^lX3z}yQczpEn|~Z7F{6lngV-Y4$CX1Pk!_vk~-`DHc}yA``%8x87VT1 zL=y&D^lPIpkPA7rG6k6Tm;5T6bs**Uq}TsjSmqiW0Y2D%Yt|3uRZ7Q__r=tLQegql z-6z zrPOylafWowu?zxh@88A%_^u3~nMJe4O5j6kL-}z}Z;@yfn1} zrTNarcsX)hJwVhpYryvy263YH2zs`Gq~F_;4@*6cdURc~#N#c%(lW=IZTGLJnW^HM z1OdOf%EO63Vv!I*5D41?qhMQ(#(jrrz&f7X#%4DYmi{Vch~COZ*!iHiBhR%6Pot1r zTbpy!*9h0P<5W#+pn|w34VuxjmV{co4?#RtE?PeoA-K<5ke8wq_DK1eoSkvOMweQd zl^#BbLwnbXwItv>J2QaynD~?_Q>D}XgfQ&=!J2|!v{*Q85}GjLojd+o%>g!x$#KrBGxLt~vmse2*DTz5-;>rLB??L*((c-+wn}5?ICEc&qndd8S`8e&jTO zg|Arn&j9r1f4c3*akd5NLMN97qUnl&{>M)8x;vwP^Z;X4zPwhlI{tB=!3awz5n3S( zZ6lXynG2v2A!!bzh$IpXYRa*yQYuj1m~>KC-j#$Z27qYOY#8<=+}W75xa=pxFH|g6 ztza75Tu*R@*+~bsP$WsAmSC(bZL$c(d@C#AK2Jm}ObCShF!HzWrqC{KmN#=*} zg+>!)gm*);Unq*Aa#U3uk9akhTC6LVofe-K7zAUn*eG_Wlu+u@CR2$lf%J>DE1VGcR6C` z#{xSGE!4+5bA4)e7dokYR{FP2PLe0FXFj||moGQXz~b1SY&%CU*sT2FC9fNt;Gwb3 zV!~Ubw`nt)Jh4ijE}M&ebB(?$cF@_f4GsxVY1iU)UCz)1yV~~D=s1;K@K5^<7MI2qpBHw4&CU-gx`+!W^UbpDMRAT2q?DAe&L!jOBN>cs?ZYw^@&ObN zgLtvwTzmQxx7|+XG(aX8saqV>Ib9jfipD5`&`|2A8IAg_g8?qz*Vwc&+s@y)t_|nP zgQ+2g3EM}uQ(LAstK(JiI^5n%vY94ea%Ar{P(E-c=QS$tY3SR!O_sU#j!LZW`KsH> z(ocgp{4gw5XLk+WfN`XFH@Bp+2&`GBhC5#ef8noelil_*v*F# zJT;Se@dyll;YU99%+vah>E;OEo$Ju$pxpDS83CNSJCB@O!93=F{7J4@Vong# z2K89Jk&FVe*$|aJ$4yYYUlEC>lv%YMkMF^Fe{?jbQ7iR41yW1fD zq1ClTvrlYm@DOS3#21#rb7b5+(WZ4+AQ=9pUp-c7Jt@oQB-2V^`F8XbyM1wRXKj)C zO|*obUc}9ZC$-_beLSIZkH~vd#4@3yJ<_^h^TuB#XAOsN2?0fcOj5Rw zhf)Fy^!w{mYP2n?)kLG1NpHBnwvx{i+IJdq&X%k)cn+25h$$!1tPZ2vbyA+*f^yRI zodvQgNWMF3+1;^<)RD~gHlD0A7~J^`QlUT*k}K#Jl^cyS1OIiQ?Xe$<|IftEqkH$_ zp$cK&|2Q!)Vl9E%vvC$rh*|NE1-zNLoa#WX1dukS*33JM&W@hz{xI;TsEw9Y;U<| ziEcFQ@pkrP#e8~%qbT%LYakH~LK$yF0d=~@#cRy%PvQ3pRl)&B$zlnA^!&8dkaPy* zP^i>mB~F+knvT6fRn97eR5Bg!*tZP1jAZ7~lpA_o-@SVT21K+nS`X+Du{-WeRu76| z^-t=fJ|`IDb$S*>mDq5`ByYhjhxKY8uCg@E2h)f_THBXE-vAXGU^oRn$mNtOsbn<$oWYqD?6gPbu{M^KtWD+_RsP@Y&;vZ@K#sW5iWQuQN#8b)wi zie>UcuY8cMo#eZzgg>08vnRP*JyZ3u>Psg{8;nHn5$Wzi2enP%V;pl(S-66GuOq7b zX^LnxLritpOP-a>szT*;-oumm##rw8!U0^@)z3R}vtO?_s+9XZ0=zK_YOOIHnR{=gVD4UhbwR8CO6A9^sNHsprO18VynqkDF+cuHK)?^Mh>N0u&6WK z{e10K9hP>AUl{n>eGf9g+V7nRv5PK$4WEz0W=p9jo2)-*+!B{Ss&yWxNu6=dXkJkv zUhLMIHv|~C6*Jv*tm{m+i9L>{rg_NJPGlrR?vEOiTMRAiH|zl9H!|r?=NX~$I$Emw zZTcBt+*z|x*b@l@gn>k3lHbW0i>E`GKn07{2?_RrCK!hjUfumI^fb>b>!T;LU?6$Y z3%j(Kz6=?@bQ$WLEXshkxDJQ)#m}oR4SRE-2aRgq;d%OZ+9pLY6Z|6UTt#oe(u3FS zA9dr+;Er001j>%jjreTQ6%A%8N-!7OseHz0A!W+fMnh+niG;-1thCuE;DD;gVrH7@ z%&N_f7*@%MN_mDFF(LDA;wdrI?fqiACb8D<-r2`sVrUb615&&RpjCV@0~vb3#fux4 zdKvZwWpuC5N=&9d9AM^)5!+cEa`btTb*`*4wPEnuo1DiLJ=lKEltLvoa1h_`1Rsdh@x+Ke3b zY#r<6&AHQ5_%F@Nl91nSEngpkva^>k;^?L^9*F9ZY_gl^>kP)DpR)Nrh00(xWVQy#*b{HrfMKxNBr%HlJ=qg z50Kf_@GOid*oT#I1=sD9iO`)d?%cw(;&$v;3$dapZ4^YNVKTmw7>eHlnRB0!b;;9)JgiMsEl zdS;+xz2?7hgTJ*%HyNuEZo+`G)|R5H;#4UHOmj?$r-h) z5iJ(Hmy_aAM3_g5(V864>+O?QoLU`I0F2MZIy($?zhQ&hFT05!+k8Zpj~($k49~@4 zb<1sUX=GsWFeBnl&SsY>gm&z}N4tmeFjFn_{6iYl=EMOOW_OwN6GYu^$QqT*9-uHg zIOTIkma!{}DyrW(k*u;Xb>GcK?(SJ(#YcilZu4vWV9rsRDhI*&@k*rY$eK6L4UjSzS1U{zk?Bke36h{2 zbU4|6Diq$c%}{K(-=DEVrgDt#(EQbBgibD7{Cm~Vd=@I z-aq?j85(_j9_X@zIB)`k=8|a(m^a;7x-%9E4=}nVL z>bl`!{KBbLKX%Y?41b*krSC?MUg z3kByN7Q}W&(IuwKl2f!+#}be~T3k)m5wg#o#zdx*Pme``7QRcH>v((?+HY^jz`V*Q zhq`@e=p?+}V2CaIOoD58Q`eq^`~{}JVU5QfTH6n^H2PNW2DuxMEzivXazXPyfJS)0 zp-ry!iC|D3w@;Jk@%NNU@nh!WGC#0oe8gxL?+wu?w2tWn-i7TzCIu13p8EvX0}{Z_7x_Tky?+0%#-Y5pWAxS zSS-uMhfr7*4qx|LmFoy14;C&>pXn{-gkM}U29Q79To0pUou`cBnElHyH~7Yy?;$&9_@wn1TXO3z zx%|OtYc!E*{HHwY)p(O@ilGnRkwzAsipO%9{}2n@=Zq}3?O0s2FP0v$IF&K%G| z-m~_~25ewZ$$bXnVXDslcU1e_=}kL)=N+_5Y0>HxYSK=i89-@LT1_U_VZ4uaQ9Gk5 znPzc+{^$%Am{g-l0)zZuAt3P%`N1I`LngLgNd~nXwd##-JoAM%{-@tWqnUA5C4c1? zoPkiF5RjwLo2w?3wPU}1(w9c!rE^@Q@u|-B`4RA*X23AFM0NPaXNzS?`Gha%;uM9; zrT0lRPJXcd7^}7;g|;Hhb))f0=l2KaKtE%#9B=BOwD3_ymM<)ZqtO;y8}^h%R=vns zSDxYZoNJrtkJquXC`9A6NV->Ieg?SvXh#jp_pO&|T9Nct@p+we0}APTl0{-o^0l*1 zP0K(V#;`^pwGMu#m@kk;T>x~z_ znzO#cA3b1QEDoa>rP0{yn`$Fy>77c@Lhhs+xhddO5s@;POT2lixV-+O1>=5J4(^?ZSaMm3Q z5~$!%cg>VQ5!S19*F3Yd*$oL30Kqh=6jpZY%hHuS`B?5>skGY!XD2w~`Avo+JCmK& zeLMLnXNa>%8l+RksHXi8j>I>+qWpJ-?vVzFxW9zZW5%0)E0O|nmpuK=YqQd`ZumRn za+=YX64YDk;tF3DV1R77#C4F~3Njim69U;3ZP1Q2(v$a2vv2qK zO=7WqSsn1d#nARnqnUhR=yX?TxF~C zad0sY9xY)Ns=NKQ#qVzY4jb(9tr4Z9>;goOI0>tC6bo|PJ6HKXG zCMnIie(KsfLZeI)+SB=KpdwyvfQlQUEO$L1D8QWDI^*n!KGsEPLymW{)${ zQ;})7sO@o&$ik~-Lw7R(TNNOyPlii%tD~ylE1;+ZKb2iNZ%p^#QJE!p$byDMaOSXI z7FB7|9LF~=pQx_tZocwW{Krk?H~Cn@w5{2NUSM!y;`9ptT)qA}dr+)g=1b}wlU4t8>pbW zq2w|MLa{FkgJ_`Mta54IsgB2a$1vaDa?n1_hbP}9;&nkkOXa+KtP}l+DDa<7OO5f$iPIRqCt0qj4Tgi4c2^Z+FK(6}@6K7gKgVUa zlEi@COQw0~w7r9F7$vnjtDLU~pBCp(I@))5Z69iTSjD4=OS5E%(N}f;pe0sGPbPy% z@7UZpGW;Z&JAKf2f))o~X~7$!+io>8w`^`UJ75US?fhbzr!$M8X z>D+Dgdqa2BZ=v;@XW))Bc-|L4QA>AvD-`zBrQ4DM9u1Q|72j*oj{&l=DOz27Uu$mY zaZjbv>g{%;#$=?bjTRn0 zQ@|)w?V}MZ8|kZze3?#IcT%~Dx@Rskcs%AiellNoMyTED%hth%+(^#2tC}L)lU0W~ z9n)Q<5kk<9b)Eb`yW>QpI-1f_k5xZv;%re(vXDz;R(?HG_0)$|DXyT7Ne!h#ueJk} zfD1EIl?034MNP{M+dUc@iFj@uXR*xGXY~|mRXWr)|9ei}*AGeB`t)=w`;Ix4h%5Wq zsZ;I?A~R%12v5D9ICU?RVAbZLFm3qJcT93b%P83VLsRDJqhABnHp9I*_Ts zl2o0(ojs8nTvy5@#otUpGhoj6F#IF2B3}OD*XOp3Vm{_RBafZ&X526pmMJ#ysS0~32PS*!v#f8Wa|h14?FW$1 z^qDel+N&>A6>f|PRH;>bU)o7^8K1B8HGb0Afv=VDe~|SSP*HE+`>?2lg2>QH$p8XF ziFDT>-Q5DxB`uAC3?LxgATe~ObVw-ONK2Q1bi;eld%xG;`(JC8%Y`_fnRE8pXYc2E z_TEixy%O7%QK??FQt(3ifkgPy%2z!c`^nz=!JLlgNS04ig8#T}$Ocj-N)KrRA;M=E& zq-@BDO?qeNn4-`r(}gbf#i;CBpomvE*@cO^A)NwRaD_zpCy@~EU{JkRIE}uv_ib;n zso8kKZbd(`OM}#>xWF3$TEaf2&lgB{w^ie+I80blkA>KSKzk7c+*JI8m_+`ac?E)C zz5Kqu;ThG;WoWtI<(CP#ee+JkfPthM_k-L&$x(EV$Il(Ny_G(@ceo5{xr-&te?(8L zI=+wQ8yC(@v-C@!f3s_UzS5m6j%|0r5^KnSki}v3u6s3jzDa)i!MyU2zdm=oE&A%o zh)Msz2`_=u`qek0rz`$6wzooh$Fhw%g2L~vMh%;uI97;1Mr7Zl=8+l3C15VIu_XK= z$FR)Pt!&DGxoGv`4T4O&5yn#a6KcsRX~22ly3Nvo^^voP%SC4zuom&JW&Lo}jt47q z`j=YbQfd>cH@{q_@AoGZ*j|Uy=a`C65Z11Ar|9?j1$$+A3OX)ymoX8%%3qQR%@TeP8v|Izszp~UYkD>^0Z7C5 zSqv`iM^rri2z4}Ne%qy5%`-eN3&qLR93-|+yg;*ydEGO1#Bm&_ar9lP?<_X?57oM8 z^5mG%H!M}ke96v@t2;i9EHP*xPlF1UT64 zT`HX`?9(oo!zcU@(qy3=y%P_I+*;PGZm zZiTaK4?M>-!>NL_wxBnYi4Un1)H`SDq$-n>tEg*ggC;b2+h>WYHefx}!=BGt0Ri&6)h&mOsb3eBF?)ULhO+Bc4~#cqr> z>|)roSKXhnpLJa+o-Mb3`#U#kDd-gR7?pKz+5pwzhdmo-mH{W5c|MWt1vQ-$!8&A} zBR^x0Djp~&HHPL%uV0jj)fPT)n8F_Wj@>=QWhc_Qyje@qRqAo(D2)2le}4@Isctt# z>ZdO_E`nh-J~PQ6)*bd3sm$4~Uy>mjyLk}$B8zjO*sEB7M9By3(Cy0R!LP70gT$kQ zJunWsMs6}gYQuK^{8II(Gm0v&fpiayDcDE0B%P&CDEfqW@g7s@IA(g&{J$L}aiXqI z(hv<|H6^BM;`Qza;F*05BVHV@7A;nrbx8u6yG|feVd7^@0RHcSP*fz`AwC)tz73S0ia>V$l4{+0|5XT_X!QE|{1jSooUUzasHub|Q z?KnrhW0TZyX{GRccK}EJqLI^hzIPfuW0UU|BOBE~x#%yf{lGBuTN3ggL9%0$wXac8 zx7f=ew2ngWf3x;&ew!tCH+hxW7o)0Fl1URww>jy9_>C^BiQaMjk+M9bImu`Nvl>IR zb8UA~L4!~c$YLf!BkqPA3`SdeZ!Vp>AB*`31653he}t;9N#ErcY`yitLcb4W6GL-} zn@wX6AYgh=wm*(S5|3T2>hRME*rHa$hYf>7)iT57e$(Xg+4g(eOxtxMQK!G2ngIqWaxV=le~1-AAEa%zXYA zs$P?QXT!=fuqT<(_!=oD9POQT7akNF#x}E`v@^#!$zhC_IhE%%+9A4|c^@YZg^s?J zk7K@=d4oKRsn)xxulCe&whk4tNwMo}qduDw?e`$-amYH$=Ipthn@4(+ZO5!t+Q#)s zEZ->2)(UI9G>TPaHkddTzj0u8%+gp8B|nYscgAM3u>;D~ri3ewJ!t)bvjQn8BNBNB^-G ziYdK7*$PWJ%P~c-0B;V7yBH)lx@KmrhuI&1wir&E!z~R6uiI(YZ4%0#I7VP#{2VXq zz@XWhsh_mM9Ldw*GL`O*W6K&lRsCHDP@H@XG3GYORDaYUGmTK8jB=JSmGUd4X}M<( zu93?3!-HuNf@IHcCtI)B6m4+X#k?oRfPv4bV=z^K4dr8n9U8-B-2v_5MpYh}CM1 z`P6H4T_%8Z`z}xJQtYN#rK(7Q$*jW8Xa1_ZkJI695%8WFqWqp~B7u-F_j{A!+H;PL zsl;#Ty1o4t9^?eytp|3f#w#6*HX{Wi<{E;9tcvD*c{#%)y`+i_PFxZe+c8^E4Kbsp%))T3WwR^^%uh1y4?*2q zBK%#uipYhC1KypHs=KuO4AZ;}qsnbQNdKghbQWZ4}|ZoJ1K0>7%O z8^Mk=b3%uJ85S%AR^>Pcwubbo;Nk)?RqZAI~-2Y{OKSPy%4TE zbBZ@2(mJN?LoIZP4@;V8ghdE$Y!vX#br#^myOWtBNGHqqMcGtx(eGOfi;U(^!mNBQ zv{RzkNYQ9<5JuPJ3zqC23WVEjHv=d=i{~1BgdN9T0f9M?<^gs6wAupy^-1<^yKQ9k zn}2({ljjBk@T%Df%d%p*(Uv!bm0!Vt5G>y_PUaony;-86W$8WB|4Di4VaQ97~$ucB)S&r(OFt3Yv_QZl*?2xyf{gL^-|OJ z`-p(`iYjqAeBjNgavzIjnTUG7=`Gb2&*{XsjDNCp(pw3+OdNQ~_~u@3l_RAxDK(hJ z;HA@NuL@5`WKBpJcJ5N$Na;*(LKV=r$!Jh#*z5Q|4{~p091}&r@YgI<*w#6$_8P>Z zgw%V-Vjj3~KL|!|K!Zql3+kZ-$FKI6%@TUjjO0bSa<8llGbQO}7>Si!o#*0gMV8$9 zqUK5j$I@XJ!)%Pc2l9eS zGBdweC)2w4Nk_{*_?a+x^vC3rW?7UDcpM18wHZr3{H7hIxn(g~=Nyy0?Rc0_tA*hO zto@05z>2<$X88D!Y|-J7I-7HJ-ZYTE2-+s9&ti@o!P)FUdd<(f|G)ZJ;2>FUQuv&g zcwQWgJu3j^1wrv-VLjJs=LVi}OH>%QnUzBQQ>wHB5J$@-E;8zB^csAITB&z`v{d82 zdOqe|M^)B%^~0X$6O;H9NX8+Srhw`nEa;H5n9MElhm)R)pPheq5u6Y z^(lJsKmu2P7kgK&DKC3hS64#UtXDy`wS4%lObslwbr}&(kEzIPu8uBFp&AZ*(=RkT zxmLbw!)4c%0%z*4B<;w_GtCFyW*LsglZ;h|Bfjo!&gK!^TN?{;?SWZw{miV9bH|lX z`s_BNfwP?+H$o9C()955BdGRUXnQd&%H@1`V`Jm_o*V)zO^u(Q{{?-CVydXNoD%xY zM>lRh`muOhGzjrMU5rhxdiiMkYSr&!OY~Lh)wePtqv2|n*9LD3u)7Qr9b!#xd(lus z=clB^vXzoxr|;DgJ&1orKE;@iu&t(syUPGd-~%(q&Nd7{i4*g74d{Z1dPYo_JXSd z3^R00LS9lrASA=NaLmlu=<(O-Cu^BX$*e9nIP5gV;@{Ve;6hRae8dGg;u(~!c4y68 z3**e5pce2K7qhwUJe)7Ds%iU(g`9lmnnFW(DS&Z{@rrR&Px;)!bPs&RAhkAPlsq$VR z(8Wa};dXX*fyAt7rXH$qZeaM{z~Cl&-46w4!*ZBT$$UJKRduHb7tJ3t0CBw7iBfN2 ze4xjuKXHzK%&{7T(&18o6`fpV;ong1@WrK?Phx%n;kg`XA*%Y7{B8R;y{DOuS(xZt zogFoNlmrlD2{*xdyN!qW5U0`T zXB3Iz1xXj>2W;61bh6SNgQFi2UvrwhEX#3JyrgY-^nMjO@6jcBz}6y{T^2K{*`20j z#S^kL?knB+ zN!&M}dbSxE;#@VAjH)KjWov0T$L11W7l^+c6KXwhMj8v_QM@v(W zXg>x{#mOzPC7DuE=Zj;`K*;nyOR)ZDhqt?D!(kmilKhDd71Q4B5UlKf^Ulmj8>N*g z{m`vkTGB$F#dA=u8r}U+t|Bio3VW%5t*<8z4u23^Y`k$K$pkIU_Snm~EZ?Zv3qL#} z=M^EXSAvnQ?TsJ&h(nY0G8i$ct%lswqEq{!n@Y*q^s4mZ<#(2aRp_{M)P zS?Z>oF>Zl9Q1OGHiv1KTe*>Dfmj4-9(aJNO2Zg3>h{G2T&M3^$hyU#b@bEbJ=}Hdl zeD@$n9D?SYg0e<(+^)<@06XAH;`R!79K0ryG{Mzq3xg)5y%>=w=wQSiwj!r3nLH$f+n zsJ2!>znk|81U`CM{DJP`b9J>vr3B5jt^4{e+!2PKm!Xl0*z9yGMRv*PyeU_2cRHl_cAS^|KRZ;~T$*G;C9oZ@iW^>VPzCXk#vlBk594@l@ zNZNWa0yQenR$ASh#%q7CK1{@{kQRUU~C+rh3#(6*@827kam-=YIb!F%an<}&03hYdoRMlVi_B=m5 z@Su#6xv|ufByLdgs+Y~oq{exLGKu$dxWm%d7sW}oZCe5os60g!^_~Z`BkEJ!C94PN zKW2MJHou&Oxa-GPkVyu`*ip__44H4XPnW;&D>}y9RqwGJj7`;}Q_m+}&l&8aXf!_` z7H47V4(G)ugJLIMGvpQ@O6bnZ5@0U`YLRn|*E*R8~X6MQCZ z)o9$I3@PJsfrr=U;0DI`6k$z52p+ApFzW5y)l5~xh|5CD9K_eB5E;}=Lisq) zZ4uuo-K4F(vrr0rK4<;vWY^JiJ}H%6752-k*Mea!iujy52*f&%eR=Cb7Bh}S{Ji8_ z(mQen0k}L*ZS*;lr11x?kQKn#PvN*;jz1jNhpXPYS7r(7X`@*c5<+-RGyOfJYaA{r z)0l#c&^=?6lS$wEaD7^RMEdiW5PamJvuff@g$p?WOTMQd{ykOn5F7}4aI(B{W^asH z%K$S^=cj7q_Ae=RdsD?_46n7e5ontFqJr31(PSEl%|wGlAt$Y0-jvk$IgC$S+f!MS zt`vyLGNPh^Fg!DY0Yc>AdpNa9>tk}`^(m8K$6v23xb3ZM%VovQFtLaG)cpzXnVESAgP z2t#FNY@=#RDP(?R(fiyC#i#m_C-^;5q+9w72{OT(27dDc4T!RUjC#k=>CZGz=t}S^{ zdWECBwm&@Q7;e~*lumezx_Ou+wv`h>lGu^t5t=>b`>ou*E5#tZ%3L{VnPuX~QwXH0 zAVpq~26Lde#L;~C_{0eAOQ!@?Ayl}(+Zp|upY1cI)j=e)Zk-4b%PjQrkXK@Bb*LWo zU~}{hR6NA|<2{&g6FCU@C*D3SoHM@(O3fpYt+&uG&$S9TvH(`prJNQM6Qiy&{QA#P z2opAkKxnZxa_Dh)g_b)$K;(}3v}FUS&NGfAZNguq>ED$RZ+(gkM9e3;^NV$~!&|`= zN~LwunZPaDbM=I%yXxzSgL#p^mMXXgDGLnv{b0xVF(CTzKp+MyGB))ny|x#5WGr?% z<(i>14)`+iR0Z2zA<=<|65e|X7%IAdOF4g9PHFlSjW2HOH#R%MqTpgp+kpL{eqV5waiTqKgmE1_}a%IrHuwF zZLezP4Cws293O&)ZQ5b&m+8D~hkje0x!TSQazhKZI_iwmdMIMlW*qeB&Jrt73bmcCPqE}2xJ;@tQpCH@NZ=z!D%jfd2>PCw672>G z&fl+>YBzfyLpg0nQqGUavVc1dq^IZB3(=|Lo@bGGzi%gTV8t*bl&@R+_95rC*g%?i zswS(k;yirEBK6~XWcvHH=u8_l#6Sl(K_K1v3wepjQTZ%7rMKQY4~~~e)bdA?xlwad zap^J^Oz7#{%d_#>GUp7wXUNAzjxwdtW$KS`slPS#GM{aZm}lV@!U$W|G>TYmZzSFt zK+;#(8-xu2hu(^?yo1+@x1A@p;XYjNXq=PN%!C=0PRocxb+X2J zdd~Ba+vvvCQ<(dHc8%J6wTRcG;+Gl%f|Yw@-`{3Yr4UV@2|8}UM$G%8$nT_45Xo00 z40WkQqO#pp9#@!iYj9a$)@yV#IIgDsYQh!Z)7(STpriYc@0`MNQ1K*)v_$#9N4$!LKsCMc}R@XhP{!B%XXSb0(PVU znJ9O-EB*nW|6bI(vip6vDbKczpn-dCGn+bNG)0VS^d^BB`%*P#O5 zyP~ADxxZxZ|D4wTK%C?H<0>V8GK02Yx;OzU5kj3w!L0kp|CP`=&o4M}5Y=Pb-D!r* ztYsfkNgN^Op2Xo$OVxu^j1tGM+c)nMso|XQR@q7Fy`#wCK6ML&LQ4^vqge>GSBp&c6NIu8JRACqNE0N?vR43hwnPLrN@92FookZ%1Sq%feN& zRF_|;=n28?jXR#6JX98Q&R~t^o#ZNFQR~tHaoAuRPulY(v`qpfXtizIl)Y+wTIjIs zj5Z8f(-thIQ26lbjh_>x6$0@s;xVC$^~PQA=}9i++y$>p%)nHG+gWnBiBe#CVtlL$ zzE17Shsu{pv8tj_Xs!xO2iE;_$}u+ga#|!Fw=D=SAz7b~j$$uF8*aVhDi<%9*_akw z^7dODRhZVq zK!p5{_rB_jCn=AIe=XqGa9-?!ElCfoql#=a+YCd~8aSuq^++k9yhBroYVh&{j|@Uc z%%5~bO^7_c1H`~-o8BTRTo<(!l$QfcnIHt{BbvUX9A{b4i{xaEBMhgso*;sz$?X`u z{68lbw0O8mHvo;Um?&-t1mnKsgp`N|W4ErU`&?ZX#PS8%8^3o>lgVl=06(IwFE#!2#XE@pJr1idEYZCZpfs{c;C)A@7PJZmMODKdIeM!0UObv%%#P6jShau-s!kQ%P|+AqkIjej&f5)T!3NlyfL; zk2a&^4@jHpkMXC-`-glS6{_{thnl%sCoI#HvJ)%WF{BsDumelgPkuwsB>m^rnV|77 zUp`T|Q)V?!I>J2kIz@kon@4azZvB&I(rEcn|DBVcA4+azyM1}M%eoy4&bE-jai6cb zQaY@?WbxC>OI=fOq+BI1~#93K7Ildn!mog_?<*=HT$67fdoBmLstF20yN;JtR%7JtD_cA|c0Ao@be( z@S9f}q`uRJ;SFT$_$!9rz8C2?QpvmZgb|!13wqtnpnmi9>zg8lwKwy}r-Le54on`0 z1Gtb)p8L9(w?(CIK#e5tr1_XbU!EV^E6Ki~-#N6|WGRCO;!9D|JeK_Uh&{Eb{&!xk zat&I|lyV0lz7Fh7@!zG(s1uuvj$IAl@BI2{``$D( zH9w`yn&hMrFVl81#Lha*PqmJFDk+&?T9Jw1wmofRf4->drXQUlv)qek#!Q&LBz0@4 zJ38P4($sXv?#uiqD;ckf%1w7Hq)&|9$AtZhEP{JK2jm5plf*iD72Dix-imWB=ChtS znq^tX4Fy?&P8MiJchp>MPLe{xo>qxI#{&74WA0>H!bfuV``dw_CM(kEAiKzkIE@L@ zGw0p#*5C@d1is0Dz2x>MjHvbDw&;w;B`JqtnGkRKsNRCB$HqBvNm7T7rc#6BaJWZ3 z$e``_n+k`K!ERUW(G@REYP_~n_caH5IUIA~VV=O^vV~G~xN_uFp23hC2MylCpPcVk zr!@4jRzJnx6ozAo2L3%j1CbjeO|pQ;=qmqRB4);*0E?kajtTO6e?X^h z+G|H*3LMF9zGF&*$=Rcm?S)fB8d?lao?|Npw|6>qPfu`A5>|a6 zkk?BpkR%W<^O@kxRJwJkl!&?ZnnS!bc=9|)ziVqZk}-Jxd-b+td>I{`aQ=`hGiD}MI$8znTSw7sU3#}4Z1C)U%A(^rax(1ViZbNT zoDm98?trI+(Pz88<6ax|j?eD5@d!QQ>JWm2ve|UTp8SXUbSBN!DtoU!D_!OaBsbgXFPjTiBj#}eAi$0ct~REkcmlDKfo5#GA#jh^FfjO;h0 ztI;XOv`CDtf63vzMihz7<$Bod^r9p1xj>>G`~v|ST78`x_4m3fLgmxKY1~NlSn*1) z1j01}v5}-8_45SzqsD9Rk&Qkabq@^{RZ&K6>gCYjnrfvJq&jr8 zw;mm()#8J5*P4J8ecs0{O+8siLWUrt3$jyR`s?714zGChvy@2mALI$+dZ|?n_6Wu@| zy9t`SW`a&aN_px9*>yx_U)Ssi2(D$XN zZJXZ{oBZ!nyr3VLolOBeL7H0+7%0%CY*b>@#y**b`};ByzK$Dg1lP6Aa3N>+Qe^Q= znZvWbwv5khcF}{-WtYd8PXIkME5|YBTTl65WjqzsA#iTrtqZo8|tl^E$uk9k;un%TSUk)G2Sex zqeY@Edt+t=e{$!5(Y;p;Lg;>@uji?#qF-u>B2oQOR1;V&SXba_iJyd)XpA z>ZsUi+c0?>%&f6cJC$NK2l|N`51|ojT>J--LJPJ@6C(8C^i+L3e{a7w`~x1} zV}ZKQ8evcMHpUYLeR!NHxaDnIQbvfdZ6mLli&>4(DU;Qmp%rDWheGA(&gKKcf}AE< z&STB^be@&FF?W=alG1*=OXbr0Dk_CGQX~jOw4xZr|BbhAy#~!Xm73q6ZoWhod)xtM z*>%%?1=ztM4RFQ{b@m*S*O(z+B{7dR&hrNYeSHd%cqsxNnQlWsJP?6|Q&3O@8ZA<+ zaiEL+M$q4h0Ds@3;{*(0bom8NW8?hd2!*lT@%k1K0FJ{WAx3zirv#r{R=7hZ(By6RKWx628 zB6>s=655dyNF+xgwKF}xn=0-X7n=Mu^qFj8I!+(RjdtOI>gJMJy?W6rkxVLRylagU zFXGgbK(Cx|ftvDAqmDqz&~JPIN(TbSp?8J?1e;i7!16v zJJP|ct#LuSJRzeUN@uc6YFAsrd`G0}>}Dn03L%Ww*HG?vYUdZt99ChSf#m~?^fm?p z{+hAuRWGGqHOwlcL>Z9t&Q-;intjVA4jF3nAk6+`K3Qp@7tbzA-vi@aBQ0Ky?W@JM zzb%(TEM2hKU7EtyCwEcJYO2V!K6Pq$r^+bLmd_1NVL4PDk`{d0lqh)U89EGw_IVt< z2~b7q(jPG0=EFWemGdmrdLJ!4>!Ur;M826J=4Q2iHt{bS-^Zj6I()xkt8NRTN*S8q z@jS=>!8%oedn#AHdx#N((%Hs-xbQc|8$0 zR#0AuM5)Ak;ezJ+wxiGW4KvM9O@GX#L`#H zv!BdOSG_+gMT4rL0EQey3rc7Vz3*UvsjL2n-=d!1-{RH ziyoS+7^ls&cKM~lL9UcKVD&V>2A@Zn`XsS@p%V(7ee15LHNU9ZbmHhCRh8{}Hn(8b zxuT-e96<6|Dp6Qgu*`JzI6WtTDGSoU9!M-0izp`T z*8D?_x?zRD+qVno{m$%toLTp05@4KbAzU^y4(qIRbnYZ$C9CD$124n_Ev5xe4^JF( zJf}GW`)F-78coGdUDscF>6xE@@Ei7l_t*U5>k3<#QpxI{)IXvQ(qc4UBNR(ZM5fd- zsyb-vPVcUdDn&p8)N^M;&JQV~Y9G3LC%M^LD)r+SG<>U^_8)8^5b?3a+L#Q`d_D$S z-Y)@>gaUjqyNU6EYOck$@Or z_h@;v*8d|~2;J-U9VLzTrr(akA3cr7qZ$8!YORU_8F==bmGL*b_8U$89nSB8@No6q z>fHmtF@I)$9oixkS?6`6RQG!8;&K0}oOk)l!zVB$_M-XB@Vkw}LHOgxLYefubAZ5s z0|Mu5(?V%#auaB~ZbHAazc%!2JBF&m!@~nlLVEw#uP6URl#dvH=M1@g35c&sHyva0 zb3T^bUHm5BwekVED22CCUu`$rs8WSuY;ig09OxdR`qjebUQ`{@iXkohU~B3P+R
KV06RrxP@A%B_}{x;wvQ72jwrAeUSpt64h4q#|*ER1c0yOIO`qhe9#w zNiUSR=5^6R*+m1FM_O+tsT)=oIUShpFLWq~vqdX$6R;FHE|tMa;Qg^;HB@TRs)RMt7CGFloI`}snFyIBC-6X_MI9ed88T?n(oh|TdO7x5SM&sPCRnpV1!k$dpiqS*6~;3= zP2oFupL4br`zU4V!RP&C(CFQDz=&DOo?R5X*ANoVuYli(v_E#N6$?6TZK3sPUt{18 zeQ)DF*lu$fXc@xWxn03V1VYq0-dK|VpSj5aR=1p99}D~DBb@hltf%dLE!kIdB@0Y2 z(6iKQob5?y5O!|7^kT*f{QqGfk8l#3fR}qge_hzxOp@ITnmCF_nFU^ z`+eDYnBhU(4lmv-WZq#TqPKlF^b31qNGj6v>8lLF^!ugdZ3ENK&hY`pc zT=Cs7iFnmWx>S-j1bMLSOtpfB-4w*f3H3?NeY*s%d=a*G?0x~{cUNafnB6lb<{cjS zn9hMtkn6afyH4siL#duFAat z1pgX$iSb=sk}%hjn~{7H>T`N`Bof_YRaiq~xpnNxy!BZq#?v%*mjqEeht)7Xos)%yXiR5g!L#MUdjS)Whq!Mp*`tSz^hmFxtc zC`RgxH@(spev*Fl;WVjcECvHTlSgeS0{$Tk?__g?Bn+on;1V4BjFOxC$nX`698$TW zI-GBV3^Hg+q9$`YKie3}&D$Pp*VEIbU+Pt39?Jp%(ZSx$+Jd;N3*Jc>IeX=;ev(t{>sd7GBG^Q4ucqf1JSZaN@2LZm+evHG(;@)L^BPu((+)hg(eu(ZeYs=S!jSzIY@ ztb``zubl;n-J_!zA3?9v47Uw1Qv%Rn^KC&^mEwP5IhWt&ZD@uM;R)BSct#eGUbV)+ zNc4>8cH_#M7nwyb-G`kqX*YtYvnKVE0+p*mj_x5>7f(hkW}?pTld8CutbR6KYBb;Z zFwoq}Jl!n%7V%a2KBiZPg26|-U~wXO6745;*e8jpWQ1qD`=rmTIJen{TYv6>0d&ZL zTeZpufr9Z~JrIc7qmadsaxtM(*tx#0-tiF%*nGm|*|ET{DK(ol6;`5Dx99p4m4XnY zkloTp*fS$i9KBvz(pa?T+tdC+S|m;#3r;JT?}onsz08O}!FP?UN71n9`F6_tN0E_q zBA)I$>m1W~${!ltFVu4htN59-T$6ZR<2;ko}oeI>b1hw=vNy4D)EMudkoXlrIHw*~2-j?!FT zw|`bfm2q61vmbdaWml@98Xosdev-@|Jb2ahhE1lr@14-l)!DYtG|XPet~bA2>=la^ zx9j0e$iTP;#@?;Jfg6I`U<{%N=y{U?^e>3DFJRgRK5VON#O@|EEwAX&^8J5I?SHX9 zjw0T7Fl~X6#$gd&04i2UwWJr>djVlK1l;X)B7xxySk(Bcj9xnEF4@b6GljR>5jjo= z0e9YSg;TxniQ|Uj7h_d~_3m+4a%>nqsWE)39cA7eX{fySTqneK)avVP_e0%AoCo=D z*KcD|EcowJ6GDQc`X51e1@s%8LxB=X3FFDyP-bStrPYT_^JwCNsp6!}KAp#%!h<}H zRiuy)RAVCqP|DILrunf9(A`(Gw@=;=1vzMg9yWE{B^EQ0sn>F~c)ek$(GZZ{q*MkY z#0$_v%dd38&`o#z)A1ZEtqNLp=`f?#CB3!1G?httJ_W^&Mj?lAqe!}XcsPF6mwC5_ z-Et19Hj1z{m|0OR#4}?`8^_cbTFB}f8F?0Z4IX;clxG0mK90;jc(y0`ZO4Y3xXo${ zsk1ZSdyG1D#cYoG;cyxI)S-XPfa-F4WgJ&e zV*2zRO{oL-$DNr*SjV_?7-5MPlamj*(~?7YNX zVrZHQGv;H|zkS672AGi*&8$~LJv-`)aw}UdfpeGDg;Wg0PsX1Ya{qw$kACEW9jsaJ zG@MBdYXWUL00av(zVTIbnX&sjkT!TUE*`CX{y2Hm_3z_61Tj6k9YK;BJaX#|(hq`X zI<(~_fuKrRV<43ZKy5X>mm6dec2L3U6w+ZmX;X~TY#S;XvT#W1x7J_z$=vp|{rGr| ztzf!N=Hr9=12bZ@2$eL;C=5tT5WP^I+OJGB4u&{t+1*#EKw0MTfWwWv6;` z5LopKm^h^j>9Oe4`l;v7nucR_uGM%+<7YS^g97M1?jgxWTn%O5aB}~l@LsL*XUAG{ zjPt}WM&nZ1?~;-zUCN0iX$tT}4(~_y4}X9s59Z#R+;3nhF>d>G%E2)BiF}hT`4LvU zHZ~~_O{WV{;Z*;mUz=!y|AbnX(%JGsK+zXJ)aDmjfdPBrytagcdC~N0Z~(zYBjf^6 zE`6GEhN^x}$*^7NiTj7942;3;Lri`U-m_AU;S3hVw+SAy4~?twRkxWchIgZh|KJP8O zzWJiIdQWQy<&+%sG;8&Vz@0{;n(O|gXn+0`C3t;yv^`|tb0MB0(-pgG+&|gIog*h3O{gYklcCPtfYVAk#?iy6rqYtJmrNv&nRn z>+t9^6a~my^uo&OSGsd(LKC@ZGn2jOGki@_Uk|NkN+(CwmA=5TFQ+E_Y&+{gml5Md zC8^39kp7%TE>V(AuQa|HGC$rBYDWk`5H77#I{eN;gxP8NA|(P5!kEYzmrgB6lu=P7 zs;wF#{k882X1IPFo2IknIltrjFMkw;R#Zz1d4lbkcho`c-Aq z$vLF)-`rP2@Rz^A_kChb(nHXjyK;9u)cf{=Z zK{ya%aI5=YnVfC_K=L%SnYuahabKP8=P=TgCYtBJ=uOlDkuR8Hkk7MzEvaiVxyA?L zkS*~41a^;xaR8cZ8I8+z^DjRH+{SZe@Q`I(F^_++CyYtpbKY+rbWGQhql8GZh)XYS z0*H~_4hm3dK);v1km{-YL%y3`a{~YOB75C00cfLD9hE7_wUPbz@(sAYj4Z&4Z=27^{`_8ILO(D$nPAk4Nd3pN{jdiAmv=Jox#0iS>JiQ< z$QrONtUmDl(EHJnYWcJA-^d&T-m9xifi^((cBUI~dw_*Czu);8 z?VoS^jvRb3u|Dm2d12w0CLlWp4?Nyks8Mr2+FRjbKH)c;un_=_s|-nqU%aQ+Y2B}>x} zrr~JUIOR;$0iD5crH#%BF<-9&*yOz;h$+TDpR)HFd0;C*TfTYokoF}QoT4PVC>an+ zE-;uV+#SpMk4d!!BL6X|=QiMr7s%l+{1K7;u~Xt}vYeQ5`0sW1^+mZU?gE<1^(@?n|Q8&(CgfsFWdyV4aPl!z1AqyW##o#MoND(GTL4I z7F;Run{mOQ2ZxT5266fK8ZTcXBFwDi=zoQU*USJ43JO3c?mM`;j`0NPX4d|8M`ShVS%sn$G8-i*s8yq^T>01*|C`9lS42k#6@N zJbeM!rSp@$f3uU=>qv&Dyr1x|&HP^(&;QQqpTE460(+t7F#g2ipF{AH7_1JqSSaRO zUm4uv!_5RyX^>eJEPGK)g@c&K%hU5;ZsWtX?SH`e?e+e3lKIyv;)(zx{{N2D@8`r^ zFPusAT|Cr();H^V;pn4LB5t6>(7asjr`+A!!&_##6(tN86Aq+ zG94)G8LpdtWZ#MR%sGpBlcZ)4^uxT~JB%{0ZtusD9l>vXr!nf>52T}UwPeqC7W|hh zhLAGgbRE~`J%*{i5sn21m$jRT$M&z?A-yh$_!4si?{8~+`S_-{xA$;{6rR1bzP>)Y z>z1xG7z4s42XQQcf^Vh%E+DlV3e190*ZlY8EPrjsLyV}8Z{9R~(*!UE0_FLQu_BIb zJ}_pfG;(!wQx43_9J3CU#<}sjA6TBzs=mS0Zy)!^r*Kn&xBZ8h{ogkFirvAY5KM$V z`ydG>JQ7u5Ct)$$s(=-=2^_OPYxG_qzemQ_vA*|!=QAEsDylopo)W74uE#J*(ED8W zOM5pG=*pFA%0CW~RpNYpg29afo*`e15qlHko#9PzSx2|EdK_(8gd-ku@W{KF33p=58)3OZ4bCA)SU74rT>spZG9-%o zm2PeSX4YTKT!0JRlF(A$RRH!cY^X4LzAkOy8qHmrK11n!A6yrE09f{~ zx&#(vFnC2P^6ni1&Xn$-GKyduSM@H;1i&O&C7SNX?!P47UH=GUHfqRN?@M(R@>*+EG!d=Z$43kC zF?{(_ft?(>*9yA#_mJ7Wp`|_lCJbaqJJIHR2OfZX?eC`p{xaIz+uQ`wSMW88(M4>? z!vrwbqT0&q;a_SQ&>g=a5Oz!~IV9@{|B4MVL`YnKnU3%KumT&dgV$;6#(~swpl#`# zrXakSTr&U-F;5r}f=Mp6J}RPiTzyvpAK(EX|5-!8Fqx;3cRAF6sRp|gll2u4F|F@@ zUj0ihA zAIV;V>2s=*)z%9^MRK}RU+bZ?A?djohB@5U!W8kW})vFyW z)e3c9OQWv7(L>CmAmMf0^9#b~68*sJ_!QKYT#s6B>Nti9n#4CkqN|Hi(wk*T?wgv3 zf332Aur$))yffQkeFTDrJt@-COFd~xWWF=S;uzTpC{ zP2nRy0x-n1qdY$O`h=x1rj9uWbpvAhD!kP`DJ|c$o78KNFP9zUwbYOr^HayMJUs=# zIAuSH^v_!N>gRvFZ>`jw%XMeYn$CkhjxW?|p`9*)xpe#Nlv0~e)M(>#n9a_CW2Mt+ z9yjG!E$A`&QNVTN>9$CJ$KBd95Kvp@d0FJZSDmj8x_AonaIz&P;@J_PlIWM~6 zH6!oVR96SEit+%1&I@u1(H%7YG07SpLfq!CWx4WKlcOskb35+^h)2?iG3hp?0B7YX z2v#?==Fad2q65q?lUbyd&I>l9kwYtkRJ+{er!k#hmR1HWK=LDoNDh-cNTamv+-JzN zr5dJCCc<%4rKqJ=W1jB7P$NnMim>?NxLtig8yaa#oo^}rlmqg9v%aZ`Xy(Va$8dGchO?&R*|Z(WO@y(o41gw4Z+eZH4l_nL}FvLyZxHCfnQ z*$?mOeOv2#oL4qnfIyI85sdYDIju{Ty7k3p8gC3RIrJk}ROw31HzCn3Ip3m=3=zc# zYM)-j)~-)iOzF<|{IC2~wqX3p=W2)vYO{N znmYonkH{a8c00RE>BlG@L>)?*=zVkAcN$Mr^o8X+QnEL99>lkvDtec$aX-;>^YIEllb^Jp!}8fSCATz z5Tscnz;Hui6y=$H#A3Z}ob|L6$4r&E zpXxmdE9dtQ*2)?1!s;F6%vZCTMIYahu=EinH0SSys~C-=;=`dP8XF=80VN4=7) zSg5~gJgn_QMOJF2s$N}JE(mYjS(00-Axx-t=JQZ_@uT2tLX)%_)hF%3mUqZ4(`qOJ zbSs0&2F zMAHrPj!e6fM^3%G)-0`h*=0TDE&hw0HKNn_ewdw7$98nzx%kGypn`I|G5~tBTjhld ziFBlmu(m!Oyd5~Zt;PkEPKv{pjtT2cC4;v8E?5_8`eGyr$!$Dl(QpA(IJfmWE*d_s zP__G+>2!1cx?rol?bKqrmuSabzAx-@dW6J9E9>;u)Y7raTgWh+-P zk?t>|)u6G{!LJ80*bY`j$o+VOiY;wXb2J;g!I$aMu~%$=eiN4zPP;N*QCR6aUwB?S zqaalt%uhT$6X5HbX}hf!y;c!Qn#pR$%gQ4N}+~jva8B<8*Beu>KahR^^>pJ zs*jtkj9nx@yZ`xk!>(X)n>LoePtOpj@?=-E^9)534hg^3=st3utBf2zPi^MdvGRDG zT7-6%)uK14R6);%%lQn~c*2$pvweYFWQt{vd)tGKF_LODJa4RA6j0H!FP(Zx1a8@~ z41R>g&!PBb<7|S_nN#b&?Vg-9eqhM@Q%PGd+eZee@|&uZCCR$oxSf!QwRYWCkP+na z%1mE%^BP`3EQS0Or58*L;Cg7$+1*ac>Bz@xUDd5yfl87Cc%Qpu-!J?U1jxWPbWOv7 z{bTz50tIIL$G%4XiliX`$7jkQwd`Ya)%zKs*O%MnprhK|We*BDL)EpIL%qLxRbk6c zA#&`L-fU?}ret_3o6rRG-9xj9P0H2}95V#mR&Qtqm*EfiTpveceVD6(3O6!jNqW;1 zXlyjvn$>J@f3SWU;@g8umY67zu%%OBzMylg?y`txl3ChV|Cy2Q{=17{G@0XXveI<`=ZEPUD(Y91BMGy39Ud!*{qoF|2M-~5Px>i0f5biVAl`JF?{yf%tqi9)+;h%CJUKURt%7Dzl7R!cMzCXjmR(P+`sx_ zhLMQ+lHGY8giLPV^T~Sci05bKkI;BtXW=nUR;=W~LXTz=9jFd-Fm0IPris&=$4)3D z(+}WHjs0BvgtBi|Bd9F$=xkiht&%lLaBmT?#=Jr1!&eMRrp2W_!H=Yy2mokrK}bjK z<*vxhp^#JCwx&D|3|cURbw_U`Bnvl+&q*K~^;6=h8++}ZzL8As#=vS$xN{vZ2@w?1 zfCUHE>!<)<`l*D69%QD-O1tc>ewcS{^2e6Bhze4P-)pOdCpp4ENC*c=N2Xi!kIgav z(pN0xL{q9FOM;{RLw5Bxh~I(;lEKLHR)V_}l25AaH=rzA%R<3b#5Cw?-uG0X4*JRy zYgGaBQ>4RdDaI7t#P1x5XR zypBs#5o*+98a1_&cVkX6?2-u;YW&!8H94PYVZ>hj(^&=|&{;SJ)N%~yi)u&tZF5q% z&&PLn4>wM!0Oia>TAz@y_}@StG6(Pzhz)$eFonM-htK~U1jAVjI`OwalW0ZIo3qIW z_633YGr1OH>mUTMvQLpd z%~y-)BIqq_A?G&VzDJrJ zFHTM|+nmnxOjVrRGnHma5+&V`yOseumNaJi(9`b$QMPr)0TTBmhIIW|XHlsV0#OVT zN%*ggA>Fk&xh{+q>=i`x9E!w8h6?dPuI|&j<#ub)IOHLu6reszO*L~NViK zr%>L)$~QQ!`pLK;SpyiW0z=ivjTubRcULCe?`xqlE}k^%b~`zmyO zEdh!1GRI?qVw==_?$1sot=&n(Uvlq`NfRoRT!n4jLDwajuAgEyY#ee_D1OS&8r#Bo zjvPxPe0O;)x58MsoeMbZJY%){@A95i34!*8X#gbFok(i12&T_X^9L>~Cpq?BSKqlc zECiJCG2lnb#*ab7YAYS;`?9-7=TB#@M=VXK3UxR?`HqW}1LfB+iFA-J4h-gSa!^P| z4#q<(E8?9z1QUvj!rwC+bWWnZ1fXSpP>2G%^FIpn|B7A%qN+E7!qjRgkzpv2rc`Oq z%?Gn?qpkM_O@9l<=&Nsj<~_ub1Im0y#@^5$f3FO5oBi-PF0%817+q7Lic*Ojwc6nZ zjvqdGXvnaN5*j!XHgF{E71^1(`?5RsHR8pFoe$fdk7i=0-_t5;P!5OnCJQf{_ttw7OQ<>$Vv|4k_J%M@#$g;i0%m9+70Hf$~ zav%!@zNXVvWIA#z#%CI3D91$T^N>8=4!&~bG6W3${3mVbzCwseG{lbA9* zXwp?teVd4$Ujv#v_X|VWpfj*0e(2IVK$66B zAyXwQ-1C)KQna|f)T8y~fI9f)$4g>RG%9Anm0ILh^R??wr;5=a6Ovft)`19gJX(wc74e8_Ze~+5<8}R zj!I#}Y|TvCn#2{LX2;9S`N+C7A~K(zt=cqO6b}1uPJJ=(i%Kf^n)#u^JynfhakJ2R z8ox9zMgY9;*5lx=gsYk2W1TuI#jg&l4M&eYIE`LgF`NcvB<}q8F<=$_Lt?0zAWk06 zri{f|U`;ISi#rT&jbzK#sKPLtu=O$58WAy)BWHFdq7F^AZ$Z>4ma@KHkm_T&vEK#g z4B&!#Kacrm@IPY=i}L($FSFTNLuAAO!2aFQEKFFCv7#b9*%XLZ6z%{3Dalvox)KsD z^8mjgDc;v1dig>#P575E>PsLqF?YAY^sitgQoxvGkpOA+G%8R;I8tKbON#T+!-F1N z!imuSSXFR1kuB5Dmofru-c-8t;m&sm+%=EWAezAs~Kl(4UwT_=p^U6C88ATABz37`YZ0#+~ z(8Rlq8gUdO9a{?zYufoJZHBjaTf671c$hXrqN}hk+k_dHqdZK^HAUpc?YJ~KOtq)L z;4z$OZRs(}HeEY~6w~bf-s)?UM(Swn4nh%f{{`mOkYw%0#=}t{Iz~9X(6RXS`xT@B z$O1t^&w}<}kAMWU67WyX&dx*(-cid_sEW78jq-Na1WKcz@z-C6WYiU!H@_3Z5J7zh zS0IaVhvx730cI11#$EMgDbh|G`e;*_8~=S;jqtYJR^rIM3%umaTKA$lp+w z%ftWhiiTy+Zi9oSS>SYWE*q4+09&VfMf5F;6IJ%JY8N5Btl~t_DMRl5MnkdNFg&bB z@Ep`NFSBh(;XOW<2Tdx9owxDDL&-d?UZMs;+^G+al%AvEd5cxYqQs;!E^mPs)xqmK z560O_H=R zVpME5W3C3&K)8%LYqpr~8#@>Y@i*kAg@s@(7Pn;*Tu!IkuE%?ER{HxXuA|tCW_UTw z(jFdxfEJ5kDz{LfEQS)v=Ha%%>+l26jH)990MQvFe90vu`%L}C(O04ahjZ{D)3+kzbTk)g)L(#&o^eG8~SC{}mp0n*@>PNi$+nfsJ-hh_9G*)ijllOX0%*B zF7jG1sI=T{L11~v>>kf82NC+1fse!T*9|qBo@)Z}^tJChT}R&kb{T>LvcWoVNsrpU zgL4mO?GR#%AEj=bhv9VaFHN0^vNH?4T?uvGPntuh{nKwa>b3F5-~aKt#7P{RVH}-o zPJL&q*TC-{HOcPM_^Ts36M+yCQI>L%j4KHL1IhtOb`T$a!Nf#F#BV(bMtAzxMQws! z31v(=Naso3^H%_zi4saUwqPOk{;j6=grGz^I3O%Y^4HrcNAZlHRh9!YwLl-#d&OWb zWh+oT*^&g?HnRU|Y}2#Cwm}6dpk}ccZ+{Jd#a%YHiBf*|>{m1(N!VYoP^al*qwjRD zAaOsnC|WOls0tRPT{~Zc%)Cvp*_w^tUDhIVW&x@dF{m{vfRXtw$)I943m730hb(N( zwxBZUOjAtM=s0O$zu%|@JUcTVi@T0$m zumwQnC&D??&a);jr+lo#zU`&diDTalwmsiy-rDXf-G0b{I0LddOZ_YMIvD!lyqYv4 zB}PHr3LV>6dFj9}^9ebz`F0L4alGy~?>U0LeTrq`+-w;Zc*Y=LHC>(PRnIt)p_9AU zNUet!!B1SnB%#!uESw6C3eQ+1Q=TH#wvEy+)6^D^12l)3x0?TM_?HfSQRY1yI5J%{|hORE}dw8`FILig>tkQA;GEV$$>|I(!u&re&!xIP^2ftC~k^R5SE}Uy-%bavL+BQG{*QvCKj<3d-w+Q-}6YY7=mP&N!8qopG?n?=#3IizIs1sQbf6cLQnT zKJ6YaEH0#~j(WZduXwEjr(2PSSA(ujznk0WDcNcZFlnvlusBLGwuTU#OMRU8()|X+ z`6AAGIn9Me^!f*m_BPxbK29MO-FJ~(Hh`#cSP5zSA8Py#dTlIXm~CGhDb`1CwgU8M+*IR{FMtM3<6sTTTBMUeSod>_%n z$N*+U549z&uTPH{$P+9UOz8N*NTY4Uf4H6Rd8Kt}ih*`0$@UxPDPqbj2j>W=!EvLk zANO*XTM-HTeJWqiH4|z^_k83vvynkv8swAL7NCMu>~ytFJ!q}!b!L?QRnou+3P*~Sf-7`(T8iT9y^~+JdQk>i;GuMn zBU1Yk$rYk6$X9;ENes|?bPrSl9BiFZ0%JIS51*YFPUb=(t_KCw;Wx(`6Sz5;<$aSJ zTRP~2Dzk;Zv03n-zvxm2-shDqRo8o)8UcMD#C=FwLdcjw+>1rAm>he4Wqrijdwtwl za-Wbp4ODu;M6XIklO-Ow-D7w!AN!VE_wB&Gj0C*c8{^2sBN?*M?m22)bUrZ^ojlxL z0Kk}yjR&j#+RL=gPEM_IIv~!_-w#~obO?iG6p4uO4_!Y55F4HMs>VO4{z_Shx;-E~ zCMygd!k>AVLCP0RIl$N*9M)PFzVd~U<9rwS)-8o6z94heTkQgd1cPqSHcLIxX296q zgRW$}(URw{U3~={zQgsu=H*2&qk#@Q)wWx*oB?Xt4*F1My(LEs6a`@}#<>6##fH{< zUcCWlf?$+j6nYBBxA#oCt#V0aWo#fXEb)nlMxg%xf`UB-arX?0SO0?@&BjovbK|iL z_jq*;Rn67(7BV6+IFmn}Vi6l$ZWeZ5a{CSO0C`C-3UW+@JB@K zgkrg$a**Z`fO9(n`L1iU_;%}eNOXo&O@2^Aw_*q@2Ve_2azZXU$?Qy(C`uVZV0(sB zhjs6POvC^Lq{F?(^bb*ccyNFXL<^ys{Y%vCBoB>x$Z+}Gs((-d*RI%dMg(98h$aoe zr4rO2mh%FjiVAcbIXO0qbuyr?**(zm?ZpW&c!GE+IP8s`(;*QRciEiuoWGSM8l3m;E@&^13jt!EL%=6p@2CGD zpNalYge(#%RF>w4Nvl+_&5i(|CE((_X+2h;<5EH#wD>_9^y=W}9A1a|Eo=sSy`Pd2 z*6Fte72r~G88%p8L2-KI5AR$ohzAh_$D&dj0o?t5jpto*aUW=Lk|89oFG$K5{PBMP z^8en=-{S+~mL8I+RV?puejVvQuqp6&<@lZ;qzkIyLCQIxlMeuLL4=CKT0eQM`w9~1 zrvc~w$0P???*B_{fEEA0Bv%wf^yHB@!!9II6v^gtKP{-hy4T6)cJe*ez0zh02qS(^ zL?DdrmqJ%Ox)NOV&mD_GrU3GCImK^607r<3_@oRU1q)Wj51{ere>VBQVA^^OM9^7R z`H3$GF%007{0E2FS|AL>>5!Vm5Bk4f%zhOdQD*_l`>TJSmE(8G{3+pd58MFcqOK=r|%z~(wa&!g8?s8B-M=@SyFJnEK0oq`tG?rV= z7?+CbEkf?fPCnE6KM3wv2v};M4r>RAqTjLO*w}giP|XA-PmM6xm)EGG08EyCUTFW| zxwL`_&Nn_4Csn6>i5p9b?`~&*e05C#27ek&MV0Y9(d#2o(i6pdPx(`BS}hUPur1Z> zk2`O5ejpbLl`0Z@+>&;(;JByX3XrO980Dd^+i}MIS%-%i_X!b8!PvABJ8i~aV{wMC zh>Ra{EVzis@>F9ihjr6X{PoY2V36)XJhi>| zm0zDesJbEmcXt;Ixbu6`$R@L=_Lnzeb#A8tvX~4SD(6H$hCZVpB9MWPdJtj>T}~XW zzXRrT9(@*ij(qW6tJ!84X?VHLS3ewKfVMI6zUz-gkE`p9i}V&u|Bw-BdI)fd9z;nd&x%E^K+7ZQ z16zM{#N}};3VuJ4nS_AGRFRWAF1u;h=lZ)uJ~%2}{@ot~b+Ao@gW9mB@1b9i|4T!);6sbFMtNS)Ay3kJLWV?Q zqp&|#3n*$Jc*aRW(tu13X5$QNtKV+Q1=m@=yRiKGfcUAv;%|LT@8luK zG&c1PP@{qOT~bm~a$t|#9<}~n25!f7rKA7%b`g-^$Rs}*E=ECJ{CJlp71J6@E^*7Z z)NE87tRlcn2_0DpdgalJ3dUcrBn7YJ%G>d~*lq%%$zUGqD_>AZXV3lwQ!q!fs`M;b z(3d8q`7ZRnnSq4tKRZjL0_1lufpZ_EQ!fud<)-OqsT8Ogq67r?4$2MQ;X5G${VmFj zzfvInM?QG40e*9mkE#W^GQe-D&j<4~ov18Dh-syqkPE00Wv9v-fhO{6>tAt@&U6D^QnLeorObQUDOkR2oe zzkj1zLcU;kk89Gg?AqRUqZl-CXW{_dx*eH366D>Q_Eiy8d>J0|!iAZhRkT==qxzh$bPy6Th3n_WOVFVWPpd~>|H*X;P z{nb|xHo&@f@)&FW5T_+gfz6Vf(P(LI04}R|mSQ$my6)68@%4!YBiuP6MHhF;k!5v`h^KM`tNy;*9nZiF*){Y#{x8r50GkP6_*< z&~I_(Lbf>*%-0HvlxMq>MV*8jL74N&eESDi8(Ri1{^wx7w*nR#sV+};6C3RJ>?aw{ z4QRjo1;`q8Zez&g3~I$sz@Dcz2Y}K8#0e%03DFSyEDpU*{!1P4*kI9)@=oyIZ$1(H zsQO922q4;Y={9PQ^LEz9Mu+sT{YN4wLnM$StbVx#4eX~0dYKTx*e-{ExeW@h^t|r^ z?xz&zm@-@B|M0hW;zuweif8|{^boCpe4}t^EM20YcPOaxJh63 z5?{U}(xKwkTZjEt0h^4_S?zM;p#A2)Nm+zl;XMD!P_D!TU;+m zc@lbCt>rw;<*})tu8vg=B9rH7)r}v$EM5bqz>^m*QV6BnL9KA{jAh9TC_GF`i`x4t zh~ACes_}O&bG-%-8Wq=9{vJON&2o@nS+=zPbosPV4Jfjln@gAX;i-Ze%j-IVOrAv@ ztE&b&M4EcvZ40Nykhjh!e88+9@B-R37%0QgsP4auA9G)?37HZyi^EIpH3$_8M=zd}&E~_!S=}MrNR5VKn zsILVp7Ra6MeTG=zTbViHgp&PmsMu1hkj z4+DGV$rRJN+_J;7=b%OFSi-rJ<<~goKj_HCKk_K@nx@&$6WHbCC9t6M=Jr&jlGPYuT$lf_Xu)K5K z`N4j9_y23%_+WSf;}~oh0HV7HtDc)kcg!E5|7Pj%eZ0 z2crZ?FinhDZtHl{;Zk+JOxJ9RP*!V^LC%k?zbVVZ106mq^;>x(BfaxDqS%kIsHFxj zK=@;`ktZOYebsR17AC?u2H-;XE^pX{=pdTx0_Ssjxb*PxC4a4=u1?fV0E*MB8gTpa zO|&(N5sbn{#iGN}bvD4Q1_r^;{R#I`Q$Pvwhx0KOri9k^lmhU<+40kY7|HjKB>0Th zUWv&S84_PC4}qw_t@rVVpZhKq*8A~HBOp52QvWOgcQogcc+eqwFYKPt_${F7-y+&? zj7JpdwY35Y2*?g1pJ=jgIX~woLBMF;ZBDg9+xHH z6^tkPQa~4K7vL5n=GTr1%+qO@Te<@fcNprivihp-FV?aLI=zzio8#pYfC$7#1}2#Z zD9p>!;?Qjhc%F)2;3 z)_%WMzAT(fOvf=>)-_YEFhG87%;=t}K;zi)?UsWUcZo8Yi7;^HRYDCMmQ<6u3-(Jt z6;%uE!hY0XE5Zu(-kOI}$qEAbc92uS0%!^SzqT{57eFR9vNv6(4RS#=D{Tba>wXX+ zj^6`RB&B)u%Lh4tbo>PKT+k(#Jj7NhHT^&+uJHNu%(vi}2Y#jTB*G}9;W+p2QyY(t z1Dg>0K8O+3eCggRGwv66+FQMER!7Tq!r^o9Q~g16^;cTW3Id~(Xm28ZHC(3T5MZI^Kk0dHL%pVoKfE=H+%lxWxYgI+$skPAn; z*>Lh9w(x?^9iD(IeoM6UmuDj&o+VQ)dQn@6KPW(y0q*ts?#$f$foKU1>-ptX zxX~8qwP69;vR>%K`3CMuisyZXpl_B{(xF_XjghRpU{#SqmV(&z+xS!!)(duXEXlci zH((YM6}0E(6ZG~QB8qyg9S_cv$q<>r=p z9%=XSD_0?p?A1EC)gqIaPt?d{)eK2y8>%VQ9{m)eR%3Vn&dX+g3ldlS-oP3_{LO5T z$owBq3q1G%ITBcSl$YWGeG?H#5EemSlYt!HEw|Q4dJ`bB0f;ug_Q*J3VaWI$jUtU} z);zYBOZR~G+hcHrh8B}zMUmlsQ`B?q@tAwj1ruTjTGxhS@uiFPndj*u8IRoh!pJ=$nfAoTWjQkuP;9v=0RhPbx+fpO)?SZ*L2u%koeuUbAhMscS0}`qsRQ$42g0l+u%_X}xqXJ{l^S z?a&Iu7T_yx!1!JZs#cW0G&7X#ymoAGIOgAx-l<-DDv~da8OI@gv81 z?n3~NA=83OFJ(*hW~Y7D_hp#OURcN`N?N+zrBzN=Q%68tC;grR;vhQEVtb=&L?Y}B z<8uF9skKXY09ga9!Si?(hvV0YB8Ai!Xn%PDLaC6+)&jyO=wtWGUU|4f7Ujmmm&ISK z15Kwq+k@i(To@$w&d3W~4y}^qrh#?I!^^%~F$IF{sX((a?Mb0Ci6pU`yj*VQ;BfmH zaAtk}`D#lTZ54dBodT5GY!AJ6!F=IwRIi*r_6|Wvr0sh*#@u?1bLe9!${tryhIt{q7Yf7*DYFgYDvZSN_2@8;~%C#%g(OO8yQ zZ$L5>FGLCbob6-)jF#8zksqH>Lo{7;ysy_=$2I3|mv2n1S?x315%=XjVfx7Gu)9j_ zB9{tK33+9VTM7b)m702QBV$8OHX+1%zsU{Ekimx-Pc+g%ufIT ztJd6Yg1D44Bc)zeE;SJ78W?t-EZBK)5*Ax$Sm&fyXHFn|Et$JQqry&;Lh=JS-}z%< z1?0AYP#W&ngx}?R`#~&EtJ0MR&U;=Efn{Zle}yae^+x6H1CsQvShmWslKqbvr(gZ= zy}Py4i+jA53ryjb?UHQqQLOw?lj)u_AOTtzC(^|e1W0G6Q)xrlWl0?C^mLbgB&Md^ zmRVBJl2FPEgVjlg#*~?>7bl#JqSx@1Sbv#c`#IBvF}vX?F&m_m*yq#|Ngl>xaZ*(5 zu#MRg-(Q1xg@%RGzS~N*!tPVf_-1}AyKU3aqa(YPq}M6#`#}tEm!^Lns@{MxYSz;5 z)ZV9&*Kfzzjc3(T_+Dh=OLAn>T9U2E!Z*8j)Dy8X_i22hrH*TF(X}sE6Ex{uA$$E( zw0w+`^1=^j{7=9GeHxL?Vm!RL`3ckcn;npJPkmP7M9lDIjr2eEHh`OcioOhbezQ)KCc{~1KNSaqXJ_m>2y7Wq zBXfF=?T4Cuy{>HwHxJCYb9DDuhMm)^&}i)(KHDSUCV|@jzS&I(Lg$=kiW zD5|v=6{34qJVI-A_?@$+ReJ;B%Ul&~26F3;Y^_Wb$rU#rnal?`4Ox%o)i@!`*}kJQ zx_ekVvCy-+8(_Q$h4he^67qGfYCOstCgjhW|R>sojE&*`XU>|<&v#(2$F6nqAjP`l9~xm`~Q zc*M1#UHFgjeT)cV{gDd^?$7%8%Ii+|)E(DK23OhUkwKV50jPLUj-?Adcg6D~WP0rc zzsI}u1BeDL@)1Bei@&*bKLv#P3fH@2Z@^$MCmhgAd$GR-@KqF-(@A&-r~)eyf9W5Y z3zNAH8#EiCV0`Q*lQ&UL-PNB%m*DnYkcLmQX@0UZzKSThNopXCDW1qUOS{BnI5RNg zyRxKz^!j1AsxVw}*K)5xuP5I`G*d0pR5E?#b1liWZQO|Ky4;^dyy!>mB_Q2$xEy%Y z4RG--Y&^V?R;hj?h6LLTyN(qo2LP~BJ1{N8gKV$gg<^Dew(!VeOWgIxgA$`OWi;bDe6kVyeLtncl=Tkk*9SUG&9e=ZQsj7U4!%lc&E};qDNb09XdBI0%xKE0 zmO^O1og35Y5uY%$Iylf-?&Ow6H0>k6GaI+3)p~k-XVxoHYWC9|b@3w`Bf2+PX#;24 zeH%Y__xaj#N9|wy*gX4r)MiUR(x^YP7gyzU?)4_H`@GsaU6LWeWiyDX4H%l=(o8xp zf=Qc+P?$%@n)(zpls!vca~#%nkff zBp>;KQzU%tu4~B-$i1j7h7cvK99fhdYn;V04JHdTrkQcI_0?emL_cnuy=xUaYX+pT z=#u$-Rjb+4U1YHDCS-5HdE}7Rs3|UJ7Rj;ws%NkwH$%Z<2v>V zGaeh0BrxZh7#4$OuFS;o_P(pmEzYcWlZ-RzOT=QB#tK~mJa-JR6wPE6k!G_j+WLyPY^Z(s_Qzab@=iOSf?Rsk=yDLPYzA zp0;sYYaPsICF|;D30&^7PVQWSO_H$;kPgNlek8Hc#np_L-`AhbG{X7T^-_64aZdO6 zYor2~?(Ky&gYW>-xJ>L(moXfodg$B^uD&?9)IJAKJ_0C9k#RrG15NtnM`(DD`?5(X zKq>!(0D8Bro6HTEq7S`uVYHfU8J?B#!XwvKJp%qyQP?UBCB7rmjGq2%&!aJraT zA6{F$YveAX)D=S|jJm)^@N;p6O-MxI4*mP#{kA)EVf*Uk{bh{t3Apc3NU{vl8zSp& zEzt&QxsGRU)t(I8a1w4U{BpE{AdoI?RoK1m`gYd8)=O_a3G?WK9++21YdVrmwXCzx z0wp`pH$wrDF@El$|7D?Ecv%N^;NiKuNWg^%xq{#h8MHDoUWJ!F*km0d@M1!MZF%hC z0T5o@UwpDiNP!1?$U^{Eh&bq(%ck0Y!2J$mG|DeG(`zkIPRc@Ke|O-~;o#?=f-hK| zJ66~C_1{S;X3F<3JE7q-EHG;H&`#G(6ai$y_tsSBVlD@+8|_{`7$Rul*LO+Qii_)x zb4xW}v`%cN4zDn_$apyqTY&Le{!#f6*yD!T!7_opabIJ)x?H#cHx?j_&5os^wcp{; zI5$6F=OH_=teSV)m3SMd>-e&z)%K@q{>F=y&Kf6~-Icy~ZeyH&S=B8UHs+)|5SFJK zA0CqYX1R+a`ZwGPau;x5X1wEmV*bmWWkhv3+4}YfxFU>NU)}bvUcdED#FnOYwDEk<(Kh9Lv3NOIgr+a&3_TMTO#*jvb5uTwk>FL+(mYImm9kDZ?&LP0D=W}gQ1Gy# zz`bOb`0ePqdV#_-ld}*O^UYTQhg-C7&3G)IDq^1=mSuu|G^e{ub&6j*7zorV73uph zW)kmQ9Lq5{q_xqPNFa0%p9GOOtF;l<3b)hBGpFM{vkLd~+OY)CS!I|TkL{ID)gECuXP{bJ#Wr@T_}520S*k*rnk7k(11)K)W;PoUEUpjF zpMR3$-C$PftI15&K3z!Z!X_ZI?=+3asT|{#6lg@?;R^whf5Z&C*VA}6x}{QTbT)s& zExfWMyRq-b{1)wCUfUA$0=ZFWqZ4NH%QbrsZSAey(nNH4Y?(u9#?ps=g-+c!d!LME zGvjEy+1*;{Tq{?44Z2o(-9X)ntt;*(%gDVkx=WEo4n$U6{EcJ*qD7<}A#WNLfvoTFH%-| z;YeXO_Fs9_p}fBVJGBi6K)n=zNGI|@*}L;%i@d&Y0YO-PwktuaZ$ot2iTWK+;EtKB ztQ@%nnImnjq;Mk+p$|C4$p`_EOq_kC5`la5~5W&Ok#oo60mN?Cyv#rGBepulr zj#eL{8<+cI1mLU>S$VdH_|AIL4V3nyRA@K5*EP+Y=I^LGrM7;oOr!QGH#}mLJ6w8X z9Hs7`mBnV&p77`(yK6^0g2^^1Z781AJ@e$TQ5rsnn_N&FJNaPVL@_(Pz^jKXFpM=9nMWe z1oDqG_u@IMMLOeoOty4uWEAmUYz6THfGnAOnj{tbDSZQI!R0uF@3e@>Lzqwp z*}kmZlt+G9uPUE^M)dXZDR91h1LS4QkqG1CseP8wzQ>c3 zO?RO6$#7EEF0IpN3>9u0XdVevpZgwyVYRS6W~kfoo;a@hNXP7O_7p3W$7J5{Tin>x z8c~uFJ^+}63w&v6zLV@0HEO?Z96>0*60xktW|E?u$6T`Am`&~D;4;b=O6;muR)v4I z@}MVkh3%-Xwubk(UYn13;fz!Z4VQY zu(cC^Zy&J$Jj{C~AIy{Zo^8&cQY6l&Dur++6&#!@cPN)<-*o?ilzE{vi@Z6*8;^5T zFc$9%iAu(+Hv?0GB{EXM_{jc8qNUMF9G-h&PqPILoiz!9In&b2-9Dzdx^40&jH_z7 zSDWQzKOV8%4Q}FBku$=?W*$bSQhSTlztaO)Ws<`a$e%WV2;aR8Kr-~YC0Ra8J*nG2 zt}TKr4>!r?;*rv!3@XXlKGnms(I!$hsXxI36HfrN|MI5G z+2O)b`%{n*0n=$@zAbf-ON2fhKy%c5AoTDC%%EMB&Z784h1YmgvUYvE#mw5`IxP1_ zs@pdk$OV77_e;sO=ebBdK!`WD9Q4nYQlrw()?FGX_m3vu1b}= z=dx$-AFH3Ae#lbU2c8RDv0%GT2Ze+oW;aHGD*3dmJf@9u>_#ha*-R_`7=O-Qy4KHn_-=-4kd?nF+uT)LL)QPxPgq z+SxFCyRhI-Bp@Npb4SF>I33EP)_evB}Y}I^?wfixN@OfW@f3R0T+UE3ERP4EvLu)Hg zOwN(-)|oIUUdy+0ux8K~pXtcnU7WWERI`V!Cx9LKA347R3B^DD<3Il;D8%1EldAv3 zcZFQ{$fl@VOH&sIWO`7lXwT5$ENIeFGdWFRWMU-4HTtC~5-}q-&6$JcC?wI>_v@}n z%=z<9wAFfv?ugr=t?;EQm$pTaW@q}g<);xPEwAi{Oq9FSatd4#mKDOrm1eIpv>4ysLP8;n`F3ACsJ@P(4T4I%f0ld_!?{r8Vcb{$V0i(OV&m-e=t z8R{0|>S*XMqJn>579Mx5wP)jHeM~eu9IzED7tj_0Ocwr2+RQNI?5qmsoR(T!_ zr(*eN{}sHlBY48ReS5H$*vxgtA%e2ptCqmb)^*yPOguR`zPH3>n4(c_Bb;x#@7C7j zlv)MC#%5@vQ@rx%vK^(57rH{^lC7@2T{9zqT-`&XA1@q_EfRBBNf48P>-P%X~xNH(E!+^Nf$$B8uHl@8Ii)1IY2$H0RD;W&}l6 zSw)!u6sjz7zXiBdzw6?pz}mvQy)#Z&HY|JTQ-2u`NSW$Zzrm0TjG(6clh(g@=|4Xj zYdo^-G7#U+CkmCg?JoD1KZqShF4FIy$zC|69vlpMvK)d{I20e)FyN3#u3M|M*rK~xxt&gsblFiw$^$Z z%Sl-RwF4)lY5JvC%dEQ)4s-VHlU`L`#6?}kYQhw96$U!*U#4D>$)HH_PBD(j6O@ea z?;R(fUPJGS<(P>SeuZ>%ry!A#vwfS)0S5+~QgV;I7Yil$T=B)ZscziKe5s?rzco~t zY(DW4G@+t<vdMxXb6zVAEs-akAJe|U^D z_kCSwtaGh(LTJCZDd18HTUjg}TMmDF&GJ43_TB&tMN);+gRt7-{E~M81ig>|GE!H8 zMmYq%LF4}|jDpC(j;=x{q2({3dOkQO={>H4NQJS_J~Hby&Cc1(3bTQ+=*36W79bk05A{NPEcu-|I)jTPDVMe{&l^h_Uw78EjA6s ze3^o-#bJ}xOr^Z*#*rkZdvQA{Ppefzv38uCPV+~RY$??&r>EL(7u)V3&j|A{S9sWx zp8jI^loIsr6#lDKO0&K5&f$vs!j`g9eo*Nhnv2czb4DoNa?Zi9Wea;P7*c(c`KT>? z{w2`K>^lC5Y|La411PFN$BhCy6N7icyf4WiC}3z-ibbJYWA{C)9t;A?AF?Tmr+L8E zf9J&=y{fX}@IC6x3L7Lr#zeWjK22Z$)z*pJZmsF2c8w@DziSNP<#m6~W=LgH$)yU! zCJdd23SqTn_$B{nK+t9^Cs@57+BU-Dy{_RC_Ixy~f9QArx3WM&>BpLFa33*Q+y@!8 z|FccGteaa61)oZW1YEh~o!GVN^?XM~?m#Ud8{14D|J(wTybRx@i<@={KstDMs)nz4 z@2kxYqldG&D*E7%18DFcaoN{-&3l&2PGCcFk-ozC*ETa5592^6{><+v*R(#VnSxzg z*h;mGn5X;?)~PS1qRIi=<*QL241WBrv{$>yY215?JHU%h zz(%P_U(KC!DscJ{bPmG+ansklJI&*fONO**PdfOZ|i5P3Ymjo}eMDX=4?K`wl5sL(7jm%qc>18hQ zR%Ya)bi9?{1mcDfPxy>lhAhQ!7(UGz$e~7QddTOr9!>FTwuyx?vBvk-CxJB`Y~#!W zghZgnm1a~ccm^;{#|^K?w2S8cz#DdJ@AZ+-M6q6RU~_rWo&ov)d#$qcYO#oiyr>X6 z7%1_aJt3Rr=E{?Uf_8~Q_1BsBzdIs6E%@H*jGBfvX@i?g3z#+I`5b^JaF9a%btT7M z`PeV9cHkT*;TYz^_u#K@fKy!g8rw8~!CrlK;OVYK z`BJWe)(^-X>ndgCF16uow42{+w$^HkT&drF*#4^Rem(s>kIpykbD7Nuk9ANgJA+0= zG*Azzhdvzish9vui$jvjPQc79lab&?*gYOB!IRv}PT;Y&PQYc6)v3Fx)9=B}yY3da z5F+fRq%KV6j%}!Qe6uRE0k@6H z$gLwJ_Chv^LmqoMEE=HZ{DO0WP&3%KocB-(Df!XPVrjF!Vi;Ay4yU^3dB>+Fm8RVrGyPXlylD$**TE}0F6kkUn>7n=dyRWO zEBcTgR6guV(Gsz_5or!je~RjJC8_dOt&bw>M@&d%ck!>}(hEmLjeM&Y2RZ($IbbE@ z-fY~$)4tfps@*f&ywi7pw<fI2P9@3c+@y3J1jXxUU1Y;n<~;&sl~~9bi9*06Z$l{1*qI_|fvxGidI6 zzliyrC#D0nJX}fev(Y`KQI#J!**DuIKfdtq1@caBinVfRsU zpmv?l;N84F`tpsS>flwh)|Mr&%~(Eek#_yhU+2ImKYDW@HsR^Lt)J4)mKQ6rwN;=) zP2mT_WUJ{qWjrU?hA)n#n7k@Ehk9f|_EXkoT~K->P+P?8+hWt$GYtEg%kF+rgW8_P zwNR-+-%}y5=vp`Mk(BR(RB^a@lSsC+yhyXJ>-IlX zrYgRp#i6K*o30|LB1TMv_FhBO8^^gmxmyE|-sY2#=jTh8b!B-WI$*vtED-|J%uGfJ zKu6ZQbC;AyM%QzP`)6!OXhjp}8vp;OEdN_3WiWj)nJHCW8B~LFnZ}N8D}xtyez@Ye z>Xb9{S;SX2{a|fL@{;iE^KNfQ5CYT$jI`Il!5%(-Lk4kLa(Q1C;xS-z<(fk_$Vdr$ ztV??E`5(df~?%0Ev4=7lmj{e!P8mC3wa|WqwpQ`KM4|GFZuNSJ?E(OLKD0d&cO8hmP~$Sog;V;zXAN-{JPhE3(NH7;w?|9N9adgI!4 zds0#OeEGp|O+*i=_8vl2^d2hqqtp8D3t4;sF0-)yD$RGWv_#}~{ksIAc!S+{`xmDT z(;{vF{`ne+m;Uk1fC`c7YC33Gw{f)VoL=)-&wdHKKhZTkX1P4d(5X)h_m9D*8_!dg zLg-1F4I}@2c$?VvOm!z$ zBg;yaufhAWYNq<5R*LYH+9Erv4_>+T3=e{sC6mbhedp31-uC2mOcwxT>42AaN8Ud_ z?0*gr{-sSd(0#cfFo${7cvfG7c~Fi#TUJl(M^dgoT`!5bTTnUx4K! zG?q1TU17VRD}1@l6?TF~z`{3)g#uoR*gk$OIwI!%J25_v^}`mFP32cL8AxMcQY$E# zvA(%N|E6YS-FT>9d$=Mg`@I*8T+)aU{CJ8t+Vv)d%(@k?@?Uw(uS$;cimAr6f%P3* zardox3{1Ola$zZ(d*{G^ET{n;sM(kRJg&j8P)!S6_jD}!j~?aksryjY0@StE7`SYD zP1`)ufQ1eI-rCwaw>v^gkMQ!sql9e+W8#vPxZhEYI{@83Pn5fXDQL5dMzUoyLk$zz z&A-24)xi#B3xhUvz0n5@%&FLxKnMoA4P2`u#CpOAT>#QE!Dw z!0Sc*0<)$rYn?A6g}-gW7vewV40Kl!ay`l8A&ob`gA>N4!G$@#K$l&5 z^hX#22>J+KSrvWvb(`ipAU^UyIw)?wDVs{bCW9?zfMiZZ1CZ3SVn2rHTwNftCbj;u zsZ|N%&%Ah-3$!Rhk>#{!=%dwfU@*f3#5EJPRR9X%v|oY%LoDO{1wmJ+C^Hy7cibOr z&3OPhW`8#Kzwu3KD#X4LS|c+GK0)ALI^$YwLeB8Ei2Q!l+S&bP}4Je-nO<5{%c0P3NYZWL9fERhRFbW4JF zbAniFCqMU6PEUMvmT#0Jj3wfqCbh*AO!^v4r3a9oe!0=Vo^xIJ2^t{lNI@E!sORHN z-%ku0JuLfU z(!Bv&I;>74@rRz`2%x@tH-44Hxtz1GGF~`?D@7hp4{*!;=|X zX`rjzc9F&(V!ijbhyL^V4G5aObcquw2}&f43V1*cU{Qg9EW~aDRiyFrQ$|VQwH`(d z2_s@HnM&J)@@Pzk0C4CnP#IJ#LXYkN{^Si7$^V9n7M1-2rW1-(oh6R2gDtMXi?NCN z3M9Nk*(z$6;>O@ihw38R1AzyR?F-}XSdTLRWToCQVAX5V#-pa-vunw01~)d;!>3dS zNa+{=dB~-n8fumhg!=FG*G13`=-$@BeUYV9@%7veB!xblsSIV__|hU63A03JrUwSz zb*uMC$ECIcHlahWCK2kR|Nrkz=>JtJYWgl22>25_TY31Y?(2gp2kibljpn&5AXso^ z>j9N}Mc)jYb`8)#$pG}Fk7rz<`FxcWZ0>==Z4ing3OTP|fyB$8Q;cw#og!zv`{G>- zQgGiL8vf;9RR1jz{D=XM)&n_?^;)lGUIeKJ6k06v?&@iI+Ecf-?+L(@??lU%PkH*) z_u{o&D*vNGdAPnJ5Nc%Zm?8={{7nQ4VT=4fGvI#?;@Hg#^S&XeI2S=RL~!zYa{JcP?@G5YG_WQPeP>4_-W z&wPr)*T*AfJD=w=E=UF4?83pz>Uag5PX2oV@emgsgx&LMQ6LC3x;i@`3Q$qSS4C26 zEJGBBz5}20Fz*-U)57qg0c#jt0SU`$zCiAXp^KBLDY1<{-}Fn;4%f-7mnj3!y^|vsXeNdLM8P4d0FVi!k4X{=O^c z!e#(1p~>!G2CAG7EBIn6=liXA5sEO|54N*4Lw{6df(R}ZS7h2|AXR40lMT%O25ZC_ zwUb>0R&$`r2k*hTkhXjrERHLE#3z$3+^E5%W_W+H(X*OD^fSBud!U0FFgXIdGPaNr zW!{V+#YjIms2kLftUwdMzE?x599yYq1cj1+@TR}-biWE@6NQ|aciLr^U9d1IjRHSe zv_<=idKAUs#IgBmy%AWBG9-Bk;yuW#o#Idr618q4MJg?R=${Lj z2hwx#+IO@6o*eK=pf?2|u`!n&!m_X0mIs79@Bjdbz@=VZ?x7+fs_L0Smdsl_ru@m+|JCGSxhaCPMr^+^ErM2MHATbLcpQV^I=$?Mp5=^?;1^#VTN)J-&Q|mb#T!A*3iSL3cPBxg1*QlnCx|9oJm-BUp^N!5^EA#)`nZ0B9!nn2r>`eh#JLfiW z9H1Q*Q-gtp-(G)u19oa=2IBVRuhb5b%UUDM+!k_uYC!u!$^uboDF>3nVIYJ02~4m0 zc>1?hMxKF!SO=G!-)KlX@bQ>RFP*gJP0CgMUehPB01S-^)pRW%KpG~xblII{7!bZJ zh-K58H=!y80fWvu46n;V!Aos& z!xTcqOML48kv6F3RUs_R98oDjWKa7Me6SB?eiRTgUU{5mbA~pzgD079I1oCYpg|@Y z3&3WK5*!1uIbU< zZpm`*@$+aP>gvX%^m;e63iJu_oW|X6w;d}0FYzPp-)Q=F)lfsTScu{<>;R)>` zqcrK>au}N4TfMdzJ)+K1_vwHg;Zect(i&Gl?=+4I1+Y%pMBB!- zRy9DhsDONSXz4RB9oSOE2Xh1SQax*+ht&es9mb{2AodPtNMMF}pYYsV9m#Ku=6&+1 z0e3f3JhX@z;PgY8|570R=n(-^{G;%dnyBngtDC4FdYzGn9A zj(x+4jj~2P1LO0_LS7CGP&Y`zOK| zOM~e%ol9-azCkYQ>F4ej8##s+Huj&XGSM+f|2^}<2nsT)5i0>N3JlE`xMr;U`mKlC zJiW-hF<)*vdp9Vb#!s{bC%0KzMYV0N-eu!dsmaS-QB_^Ti>U<6j2TpNqo2s9%~lhy zT7df_+wu_doV-g1G{CPY!8~PZD`X_++tQId4EKuXm_F}7Mm20JADdVb78^&(YrA>5 z;G0wFH7*Rx^pljJ=TXr^4f0`=@U1K``y!_!xZ`V21W_R*ki`Li?l=4!Kf${y!3NC3 zA7HEl;+D-q8FpcwUk1#{>_TxSL7#!Ea}ThoVlH|_K~aQ>uLm&2uS0RN8Lu|oy+ zBwdiTZ|&2>6waFTjJ(M6pxTvm)fi!G{xEwoVHXIjNu|^5Xrx-?&79^()~LNSHLGHl(h& zz#&6Y1zlq6G`TJfn-F&Tv2f*Ul<|H{ToxJQ7|tC8u)i zkSX)I;k3TCe|&fVH|+ZV1g4dDk zBR;#uPZ$M%0%YcU=uzd?%fK54brwbj&Tl^Vcew(u8m^|78(odsr*rjW1hpFD!*6WN zRL+LB>%5yOsYI@MUh=*H>B8z<%F@me-`NH8USHe^Xnnh?O7*H#y_(=Z&5ukTktRS#k!9NrVp!UPKDIN(1tg>F4nmerdo zFtKz7Bz%zTBE}*jPz}%xIRDif0`zkba2-kN287r5w69zDk2gjxykWMTg_`wpKYsjp zf74Ve1|DWwOI4v(md41_4)o25Vm8{B;p|l!2SqJbEDARVos^&=20fPhsljn15()>l zHowP9o(NU3wp_YQ3y#-m-&ItX>?OrV#FPWY*{q0*b;XLb8)2tc2cG21t>Zft&OKYr z^RT3XsTy5d<@`6?kGPD!RTYVQX1A9$o}ZK6&K0PNThS*}>Om9ZK~R&Ph)i9X`6bd- zI8!ZKPp2sf`3=3k`I_8za~5fyAD%C%!uE5tHwXif+WtADAVL$P@Y>9lJmP!qZL-&& z2#5>Zfo0BtfV&V)XNT^xJ*kg|4^}Xl9lZqR8`&Txy8S{==@cqG$}et;0f2$j77RSp z!zt5r$r;<}jE>n%NgPG$mq#lA&_n2w=DiXICIDaH^Jkp@zD{a{1;z3L%6A+EU^OrW zItlxL-?xcRfIAQOpR0?eO_<2z4CPDZ_+$C!F#s}Z z#LmPPo7S?i)RR-8ZX;wGWky#FyEt3|UW}0INX8=K_s}9Ce?Z9j8E>uFtZuac3o+{Z zJXDEmW=-vv^WUug0l46_MZ1q?xPFBxO#w5ZaMHc$#^8qWl8n+mY!G`FVWPzpb1TIg zBM~Di&P@mq@=la=zV;QaJ-`ll~Llbc~ig9wTSkRAFc`uwtePt$G;b)*-i6HG(j-bkH577VD z%mXA@1D5^8@2G!~RN&7x<4rI3c@nhXAX?j;{tUTc`SLzLdm~{icD9O$kdGT39}Goo z^FumM@}_J?V|&KK>U`3Ly&;WKJ1m!Boc*p8WO!>gQ=L zHz4EV)bD1#{#J?e^feG;YH#_C6QQ_i+LipZ$r0^Bc$Oqdk$do7Oj{QJ1e1+$(&D#y zG2tZePyamP(3aA#QlR%u?gonGh!HHB#}5H$E{cVn8wySFO|kv3>tHU{7CSxKEB_-X zTOs{a_fMkYFsabpU{?+bFY}onHk-Khysb3`7rlyR9K__W)BV)h&iq@kAh4AE_o5B^W%uK&LffPh`*9ja5cX4xr2ar;9|C)0iv2hl z1>v{r2I%%SMLsd+`M$HSx2atq$MNc$!0EIZSPF+Rm0Hqo*=s0;Hc4>$94-v+jGR6R39u;n&eW3n2RL*Crb!5h zsB;g^3Z1~wLYY7O(z^kCM8{r7j~^(@e9^}XbXoQ)&B?9(&Ol)=TQBf9^S_-LHPlz?b&(6XYo`_JDuYca#ew+gG6L7@%~CNr+8m&f z$V)o11nE^FV{LaK=_Zqr=4It*08|pZtFH@38ad>J3vb9eqfVpX>7n#9SLb#+1HqNV zGx0udXcbad$84|OCWp%3_`JJ^cxlT1U?ws3S?wIWew!#hU>FC&3ova%gqQEM?~;Ao z0Rq8e%LvqS9mhSOsH&do7!g3M!NA1NdFwr;t&;=BYs0{~g#=J<>h~IsA`w@kv?Kxd*wI{VYB$?hfj@G`ec<{6DSqJSUvs|%C}#{qyiQ&`;xkjwz8AIyS~b5r zx#kl5zO%1K^WlJQsM%Jz1GoSooGJ^#YLtAe+BFJ4T3dCtJ^&9Hd{dN{l5L%;EGqaz zDg;B%pUE3DJ(Wpk);rlU-W>m0EOJav#Wv$y{b4zH?bqZ` z4R3~NFU1kD(~S|SG>>kF4%8!_Y`Bo;$$A{VYRN=Ju4oZ9HO##8+8z8vwM~>Onpus)v%x6#`2hveU1JpM)3^ zLhc&L*RzXxefd8QBmrk~wcLNQzbyiKu42FLO!9Nb=KzF$2Qcb0Wd4AlKY%OZ@Anly zlOJRD~PWb>hCd{^1(h}gmtxFL96 z=Q4tE?&_&Rh_2k6)1MZM)6hc;y}QWF9pTitnpQ~M~E{1G46SPz?#P#774v8#T6j}BL_Eb>vxmf3Jg|4i~2A!Zv9`JDp%oaRVkc{$PyL1DsVeor_J(71QPU%Mt#Ac+^1I)nQh2Q;>^qusd5D;eAsDPVhn+ z^t8heo5tsMY;CxzeanSSB}dUxqt2nRK!^SH8zL5?Y8}|@3Z4DZhf-{s>7CE0J_tKT zHcFfTgkvQN!dF66m1l3&m%~AH>n};wV?`Np(uk#eQ{};1iDFi<}M4mh0BoV(`o-0@4TY8?)Hx#CuW_b|2iMsqckr<*3^u!+iU3n{}MOoE^vjsNy= zK!nq&3y)}oBY(r>!k0@_cyn-`9hlN8)HG&hOC)g^236SJwvFxd^8vp&FGfV+>7}YH z2^pc1iMwp5oU>H$eI#K1e%`xi`-Ra@Jx6$G(CS3WNpsSnc`+MY=b!?ncMB^qIDB~F zV%3?$NV8grOCvW9>L0~s?GWqRqBL6pXs+s+LtV$$jnIaRs(8A2x1+V#=A1K6s=*tI z5-b|9&h|K67l2A&gb(@+6D~V5UGKgzY2d?~2g^W^_z5)d?bH`RKMCSJC^t){B0Lrl z{EOR=e1?Xx0;cQB(Uj@o#LNn;BZa#3jz^B=t8Y=afoV2LFwjDL4gOP!ZXZp6 z(0_AbBq+iP{1ticRZ`rc37USe=7VTJ_W(=bIE_W9v((QS-~PDR&}g_V>t13D5qgo6TOQ5$tWH0l>Ek}%b!&&UF1lNp z%k-Us0MdzDd7QHq7QI=g>>GaDVLc_PVj@UIle{w1)^)Zu9@zqicU&HE2n?DiIaB3| ztgIX<)kwV$y}x+faw&F=ZlmcYvhTBa{qADq-Suq_-hj+FuH<>!M7A-mwASy}$_RG@ zf0BPDFJ^3mgc7b7`tb9r!j_C=izIYp`8&CJA)Qf_#liSHa1CXqt1l=}P+68vv~7Xa zdcpk{Kbr`K97_*eJs7W%OC>#78b z{YfCR<+mlXK^;v_;|lT<1nr_=p+QIPGd~m1=_iPv)6V`n2y?UnmQ`9+J24Vc za%QI~a+lf+)yv(#E(*jl9z(+OPkUN}ql|7dzFv5OgV2W1jQvcS^So{L{QtD@x#VLF zfYQsS$AOg$J+F6!A3VVt*<}k}2D01XZ-?6pmG*ESMZAq^cuJY+g4he^lLqaH@p9w7 zZ_TDO4kih}-HXCA(g`uNIST-p-`5R0{5um0pm}bJb5(4hc$}?Es5g}?lPizfa? zR8N9oD#7&=}x-%`9T-IVVE)FC+izQ(28@I@h!$J3UETZ}!PIF^~} zO>5oo0w#%zd;61TH=f_n37FoZ6I6nNOlo;d$iUIJdt~fL6<;wmXII3vojE2){Au9> zO!BFEq)YqX-iEh$RKCJzN8i)&ZoJHV@dO3A#_XV=ylbssM=)}z-hqBj+5+1cUJlSw z!)stsGcKDtuNJprg^ugE-1W@Kfti=qUxsoZR)$R6WNHE1TF zRCSo5XO}kY|G)u34?yF|F70Os1uzg?&X<5$um=_*X|L8_d(17ohq~_0$Mp+gn)`n> z4AN>%Nc*%Jq4)9Nlj-ZzV&8{^HA-R0Q<+vjbBq-B`jwhwM%UPQZRg^P^scEsnf8AQ zODH1wW4!b7-sXnWGs`{y)?wd$d54=NYmrrw0wci2r4FHwycV+JpGciRsMWJ9qB zr!)~V#qqw!ucdab9e@OB;{=X>e$}d`2^~1_CH*}9!*1>Z@8gJRKk3}=w5P)8&5|Vc ztDs}^i0czIfPpQaPpf8GhfVBom)7eAGOn>|vNBEB#E+5QjET3KT2-+C0{rpbyrP9l z(YvFX^ZpvQW@e3AxtCY`uSXYTS9^{Zf?h2_LXLGvD+Z6FMC7BJD@FD!!pEfnpSYqK zRWF{%wuxq{+@wml7vZyw9}Un=)WcE^GgGu}bb_&3$4Xy8-?=@eT2OUJs&CGGl}+LP z^fE#>;khRIb1+-M@^!f^vU$=!g65 z&oy%4cr{z@YO@C^_r^?{u<4FjN9!*9yox{^q_?%IbH1ES^)2M-?^C5NUv^K%E4qsh z%D_ME;1|F4(cTRU3qpc?&l1-Op_sUUFi{RAfMlB7Y&LA3xp8v=3fp!xP1VM}T_^n{aB z36C1KXiUX&Lj)0YTCRCmcomb|EFjwxY&}05;MVGsl)I-yh@7v+$i&!C?!2b^`lDb{ zL9&3@aJC%X0Jjw#BXgE;fP0_wmH1oOFj*(^+K?pZL2OIxPrKb*|_oygcY7Y^U*tTEre$(Jk|8s zE1fJ(tB$v(hvw;+PfkbP$2DT&ChRp|%<8ij0As*1z>Lk8*F>T&I|eD*6~%8r^ESFupMA0BmLcr;9`Kh5Vk_WGFvB)g9F?Dq6t%B z{u*p(h>f-xVSP2w;p=*n-gx8}wq(|NM#{v#6M{cH)o<%j7sMr~vkbAHsqs;M=5xIBG;$L!z{5phLsiRZfAZ0M zb5&b|(CFgkPwSVw9i=>0Bc%9?5AMt%9j$ibQY3LX1he6m0i$Ii)V^u-$uCBKJI9Be@&{2s8~knaOYN5QtU@*1m7hT}w0 zG$6mWQ=NWXcAmsX*0dZ zvQI2+u5YV2q#!;9t8zc7AiL16rp4itPNJa0Z^2H!!*J14R3Xas^DZBcgY=YRL3X(N z7kkL))iJ#k)!9d2-90xEa*Gk@P`BHZAbzElC;fi*x$jJ3&SWq8Md zzVC4Yhp$i)_={FM-j$uLveq&ieRb*Z#U?ee7iT*_L&}7=!&NpF-U=Z&JIicXEN&i^1{P2nW?okjO z7vDQ5e1B@%_3X*pG7qj;Hl5FboYyVi+bjsBTdy_@zwNhU`EVPxj0g5iyz5T%A8hs% zu%a;bxLVGG(Js~G9U;x#{fa}gB%QyW92^NPtixTRcIIVHy`ys(wF>syj^(2L4u%JL z)s_d4+1;~(uqt8k!PJcchn}K4USe#z-!tP#_2=*F`RM4Uenj;R7z*&rRaKA8xLj9G zqNS<(u>1Wo=6f*~s#F|4Roj+2->sx7Wz_G_hjE2?c_e%qadSOxBI>Mq_wl>NWOI2V zO+TDk)m$zT9Pf-1Wk-+}Xvuzz+So$xgtmOMgIr}9invi6W8?Gh4qEap!u|N+w}s?) zcV^HBzbo+`)Rvl(qx?wZuQdOWZeIs~CjaLu2r+=GAXG>cG4=#K6NNz16pMx?8u~8N z<`CFYejrHU2yO$I*Cs`_AUM>M87Roe@_b$h-ab+G?@#l0yW@&mklVm`X@k5K1qBQz zKVnOWi@yU68iL_Z!fQk3WXw)ahqyo~*z5c>CWQ(}N;Mtfw3(;9<|PK#k68;mvwQ8V zW^{! zr4E**sWNDB(#{JT3^*QoMs++m7bE-)kMWDA4pdNw(#g9mV2kz&>@nY!Z~4Lbi8iz5 zz+4);jU1ibtL(8I^jCZzu!}TJ_#%N!M&kw5zsK+qrL+*KNg*7GM%+6{sobQm1~oGw zAt8g2LpO3k=%)Rq>T3$Hcf|r!$6=O8PC!}pXmkNxWDo#~C8x&Km?mkU&nh$<_353+ z*@H}nba3bH4zb9j33KJt3TAB~|4>WBM`?}D!D8zkfor*b&)0O^XVgTJ+ZpoGdm>`f z?Q@yds^5CzJ%mB9vc}w{O|^F}v?{9PcIHaL@r4w#egBeC!o0KT>tN=Q=q{T3>9Fwb zUGvlPuq@gy$p@EFS4|9Anw-{+Ct z4S$2DZHPwfODPgid|r#Xb@zRkg?HtS@I#(GK`bGG%J)ga&H1WBO!ZDHBDsBJ683Z9 z=mgFJVhIXP)sS~p`s7>7(rfjxyTta^om&Pe~GQue5?B~fc8w50d! zGDX>fEQQDTX#WFP&Ma9Gr|i)YaoHKQ7hUc|aKlq^c5Sy%R~Y>%*o3FCd41C*-~|o# zBs)_8zL18S$;TWPE!W@iaho6IoEF-yeuwP})a`BWHsRn+_ZDeY*=?VOq4nQ|$S7-E z;$ZB21c^$A0Usa#3pSa-TTRCYO_QwJxGOExjmtJgnWO!h*i=|Fvv>&b)#EWQFE7KD z18_A|*=l?F3(Sb!o|(+q{}u{nMTiQx+~4M_6}|_fOQz|{cfNSZV62|mJI{H7MDz4b zEIzK)1x|OI5--MCmW1!5_dXw9;N|Hr*~R3SLJw=Sb{+QNmd_g8?b2HHR*T}YT3(!= zqeG#_5|>UvcUt3$QV#15uhp18990Pg{~9p=n&GHrJ&NZ}u6!DSP9VE8q2MjO!O&8c z*r1C~!J_&V-F17m$%6t@|6ZSRq1W_m7Q4xz_OpKRr|Gke9l2Zg)BBQe)QtQ6DHk^) z9tbX%X)ah8&tZfIs*<|2wQVBOY7Nnw* zFyr8@46?E)#C|x8xIbX;u^ylORpK)xtE1H(Om55Z_ZDOM0RopwA&AUA*@U8gdrcfk zCfwUv((IAt?x-Ye;dRtt|LN&qk%W|;t7`(K(i}a$qAF3I9ne!?6Sska^?;2z3%f!B z#9aHA$h#nj89=%*w1}U{z^auAjL=9HIb906 z4tD?%6C03-6=SCiJ19ikSetK9j^mHzME1bHmu9&_xP1 zgYcC<&(8YNgGC!+uLGQ%HNXd%N_4FA2I#8qOwMa1W%aM|f> zf43ToRW)B16*iIEU;R*Aq6l~A6G~}uOTIeW3op0jLH!!HgZG<$WMpI{ev=O$^IHFC z;_ENelq!rCg>b9J8j}#MPi9P@#8E3piJ>)Udm=~zUGEZu^b9jH z@fHMzl;#oCSW)s_LxX3+|mA`|ebSLQdcq#{eT4XoB|=3~>o4w;kU+sS|ro zBRit#05LKh=oW=IK*HmUp}U+)z=_olW7m7l6?`@uKg!wIgDcz6D z@L66t!KLXKg*(7I$ajMhIT}f4Z#|n|XM9-kz!dbxebKn-$K}}R>G8YfD*F5N)1Av7 z^AhwKQl0Vat)98B*KUr_!=`>!SU-XYI`49hnoU1?JX-WaOgdmmk_`Xvfe=K26li^= zvC?@V^)RKv`%;*sjBreFi{i#}@|$VUYx>OYZ%E(`y;TI~qVO53GkHBW1QYU*w%KWe z#%iW4P9QIYbB=|dHdBJ;eF!@C!$Bb#cSIE=hAx-R{}Z{bwHKMAtwn1Gta)6*glB1JE74$rk}zj;%^xm8mj452?Hl@TgkurMr?n;!pj^rLja>WZeW0nK>4-1Ztkgw z7~48I^qGRl&1Rn^*vN+?b-X)Uk1FItN#(r1wwx5r=1P%9T-fY$iS4Y#yRuJ1vA$49 z{JcO%Ui=F3Tjmj`L$2a-rBg^1q?W$F#e$QHQ$2|$wAFLMS|3qkJ$^HNvdP^mQUtn) ziUCbVa<{VmUXKUkJ86eYhhxl`8)6=26sAr>Ur?Mx20^?DSk?IAAo3~_y8RY~m?(KF ztE`F@k;O#ApY89?3E+}#iU+glHn3k_0bcz_KZ(1A%s&_nU(hMFq!CV5m_Kk8ZL{mJ zU;qqsGGQp?O;Dp^g_Qdnm zNbxhja&$Z#;*0M(GRfOrg?y{}5pNbtF$UW!N(TzVRg z&{D?UjOgpNeo=@eAB}}M$vwhy{Z1khq4tyjzx6lQ1HZVoWfzQOT*UgW{49BGMMvbU zpRBb-_!fT1M8_%Tf2C@+B}pGeWw2ZJMyjoGYmk5=MRLG!+s)QWq;&#$g4S~5?xF^# zj#IYjws~HfF)RJ>N2hO>V-8WKb?NIqWf#{EDJlYWOl8bgM?d?CLD6W+^t_*aL{S4F z#Q`#@K+NZLahx}03Za)kyVov-6uj>BD&!%Q37aZ*Yip}V1cQN+ycR1MiP&1r-QNxl@lz23Pb|l*}5erIvIm^x{3iP|^@1A8D zMOuQDOz&f|o>R&XtY61itP85hVYjJ+sSPMF*t8e*6YNPf7#n zuD0DgXgvLvwKw?j8Oj-D849r#&<#}B{$1RfLVNg3rK_pt= zZM7%K@D~*!ZAeUDirWX>M>Jo9aQ*e&Xdm211^80C>HBMUA>pskXt3$bq_H9L$W9{a zsJPeH*L}NuV8?Ate#Y!Cc2Gi?oW8#I8d*zy5ol|G8mrnS^?Sy6u3`Wxw5D>U0^s1d z)WV>E;plZ(>GqF`!V3-|#FXiJcZrTe^h^_Mi$%LQ-79_;hVux8Kvqvu+TZbzi+fOjW`JzTlXlccV65Lmj4Wfy?W@)Qpo!&Tk4I} zwOO*QKjPKbY(6G&m0>ZMWGA8!xNZwzKQLuBVvAXbdl@=N9rV3Fh5k;&N2r#mPBl;Ej+F-8v>7$vH4LMnWY z@dQ7NcP!`y$TZ!Z9*1rAd>wv}t9N|ST}mG_CAk%70v<@axl5|WMIt(aP?R%_tE{lQ zf{?RlZyHqp(icJ6FV9NejKLIg+4##EH3<(Q zp$583P~#(6{;D9bl4LoR2X@3IL;CxHJ<85`&}Fzki>J$>0~#s;{%8(u1)Q5aG5+@8 zw}2yrnaJLz@8unuV0TG~PidomTJ z-em{GxIEExnruKvy zxD<3SH8+c-fmlj`bD@0Q)xunN5eV!OXNC3IQh%)}bF!s{_kAEOOCo z4GU@Hxl3$06!5a7Uf&DZ0pR zD;5yT=8686nhdF<-1N`gKo%p|^fEm;hF;fqr;<}XbS!C?=G7goDHH=IE{;guMX-~{ zMGKe_Xa{xd{IckYpPltm7tw6h#10J&b-ZvKNaH^^SdtlkE%vK#8b$I;#QyPMw#1MT zp~L)`B)_>PnH+@?E_m7Ok^R&7lmfUq<;I__MSz+5=4fG59vjCC#P!b4jy~8sq#Ogk zGF|AFRWI~qu4j9^+rtN+YM3FMYPcd~%i^b*{%UPKs%Oz~^tsW$!t&HgufkW%<3Wa) zQ=L9n-dCwku?SVk>KIfgu=N2|A=4L!ad?EGfE?!%z6~cduVgQ|fkNAM_THf31ttge zFD#(Z9Q{|rfo{MU(GB>&u?}2kA=HBA`TA~-hMT1W;VZ^uxzT&j{`C;jVKI=fYET`m z4`BW|+K3=x)-0YVl-7inu^w!#DYbH@a-j(~(A>KGaZb?cuAS#b6Gnk=25!E4tdd}# zXF&HY2)?PqWzmhB^2CAWPTsCiG4UcfL!$Rio!m&H1fX`u{!b@p!oX5rJCJf{Wj+U9G$76ANTH6e)aVKNoY|#_8?C&1L~Bx{nt9PTa>V0u z27E5CfT~}eFH_Wu=Zo#6ss_m<4;p>iw>!rc*YGf9w3IbQCoD>6yq8&uAT}fF$hCBF$_I!6vl3 zqqk}o^DNw4ySJ?k`KW3b>mr;MQScL^sGUvFKhlJ9usB|wvDbOYpwi4YIHA6-h~@5` zo1<1tX6|dxNDf?n)C5(*ZH+i`>Uu&5*)RrcrMfPa)A;U9*E(<}hn+n|ByJV}qdiq(MMt9aftH;qlHB#^IGV|U~N%DioUY zu1S!O1IyY*94}8^byWaKLyr+55KfgcJz@OW=;44^+?>b!%`@7&>l6l77qQwgJF5Cy zIm5n4$anhq$0AR%3ti1~A9UYSSdY{GR-iK@;d;#`Sg49_EQ^!k7NcW?dCb+_x1k!U zqq3bk7QrtCbG2uCUVjhx(c{M08CZ?hRaAi^aB4{$+eHyG|)q*2zf>X2Y0tPGcK*?S^ujup^l7sjw3%v7V|KAR5T?*OBnlyZ|9kwwDZ|WszzLH7>z{AW zi8JpKS#(fz28})i(V<&fFR^>T-a6K43eragDdAgx3KS8jLN@h_->j1(cx9wO+aSJq zOBjp!IiH0!cLVdvOL{2c;?{Q<^cZ`D&ai|mqAw33h&(}U&*FHo)PY(3GBvgN69ZJH zzK`xsY;MQ6gpQf;oVc}xH3wvN*AC*SUqx!!G}ATqeJ#~4E-nwc+$)W~+7L$&x>0L` zVV_PTLL)=NJh51yt8U8@^C7wcqbsCoe6RADqDuc(cl!U8@10cfrcZ)HdIlwjR6m=3 z7}}3l`AOgNE#MCS_puJr%c3#IjdtVeOCr@61S_;~l}6$|p9Wp#S@=8*AS&_4ROuzJ z&f8WFlp=0TIdfB;{8M=fLi&0HLbN_6$f=#Tj{F?1uy>fpm@%gX012YHl1D&?J{2@xOk9PM)Hd((y(=MTQErx=BQJJ{$Ok z78P_T5f_KYKBl7wnj%6{vKTZv#d`M?6FKle2_#U`%+J~*PY5{`{n0)caC1q#HCAGw z;#J~+_{9w9bqsmsjxe+XrXESyI{Z61u>ipz9c3l+rhd^Fvs8!q=7C1rD#%dZ*N|g*?3bIMyk9M zidGNfmcdqD>I&wZ1}K#*e`ynV9V{e)49M z(|^?SoqrX7WR1D%*TcF<75B$F`8_S?#MY2%gc&1K9dnawvHiGWF~CB3YwvsS@o>Y8 zsVD7q+iS_rJoMxC#O>Ogx4IDjT~F^#`QZ^_v!UDv!}hIuo;T~%DJ#|`Gqh`e9~WE& zoRPOGr@v=)p0y@P!RFqWG6QY?rEoGI5T7I?jU`I1RJ2Wr`umFZU{Rby8- zfRETd`h}VrFmk1>wr7R$+1a$f6$t72pd(NQjR2y@`)KWZ{QG)8lI56eU<$fEST z(J%F8&D8H7N%#{_c@Uw%vcP|ed+ph#Tcyb>hg;%L2aM6*W>}8=hqYdTpgYnCg zgP}_HIF=V<0WEU7>+FYN@ef|g=;;a4#x@Jmi#IuAQSitQ*gn}mayy_IA@z2(>-|RA z9pK9P6D)=>TP!ZAcwO3!{*y{i>lee?-)ji&hd5$4{5cT4r@+bw;`Ye1_ZGr|`2eDg z`!c}w3ICG8cPiDlNDt5pQox!@br*H( zbQT`Yna9ATUwSyhh?JBJ{_kC}jBPCmQ>jIxe{N{J#YKUgW1AL;=foP=u`!gLK3Do6 zN-uZa5iGoYUVT*%4oa<@@>XD|AryFlaLA*zeB{Xqs99P90)lbj8e=$}JQz*G19oh~ zlv}`~D;K~%-fQbbh}Ht@5X?7g9s{>JBxyMC>`eNx+=m|o|4{w~!C0mCLPo@?Azc(P znLGr5HEOSp-qHyH>_L!x(%a>Vt+S3EwTc&$QWq}0&++rYXd0makP@8QXls6s4FlZ3 zvD-)8#l27=QHI#a^WVp$df(hxakIH_0N)_kI`#3X(N23oXiSJK>yscvUkOK^Gabv> ziw}-lm{MBlQ8fw-V-;uxx`FzTl`t7f+^lN*^V|xr4XKNIs~39W4)g`Ce7Fo*qE}CF zX?UY;H{$r1`ASSsCtlCnKZ$`m+;FtMtl?-5_wJ|>OM^?3<<%p()WeGU-jdHQo{34+ z`LA%yHf&=S7ur(1ys#L>vi`BI_#aMH7s08TNKv#%q3@8xwA06;DeqoUJs?l;%&cd` zKtn5j#`_Vsg>4L-sG3)AhIznoo!af*g zu+^!)stOAS<#0R8Fc~<6tt3Q}0A`q^Nus(p*SC%dgee$B7&rG6VrnKvFsUJ7D+QQ1 zeLKI2m*7ksBy^%{{Md0cX*ls^m<=@Q$PKic{&DF%!>5F6r-U@2G4M1{x_yAIqA>qHGG?= z8pU0yy}H#Sst`o3kE7XwAN#nUY@ylfOMZ zop~Z@&E)j1^GIe?ydf>|1)c8X&8ft*h-%TtZiD?&5>Bs%H=W~1RR z_uCV0QM=NySDpC6E&Q(*k|W&kZb-2AgJq!1A+NLwozXVNN=$(kMg6n+t2rxguQB_f z-1mWm6{5{Pw79vsW=-WVuyvW>kY7)DvS%}{bz*|Irku21@LRe&cX5A2M=(}5-NTe8 zx&q$^%6h1>Xaa3xOn+rw#ZGO|8Q$y*_+Fo+dIX)43{epncVYi%9Gj6RV^54AtrHZH z8noL(K;4pol?p+kaZfzlzb8leO--=dQy|1BDB0*vjP2u;9>~TPDcuCycmM;-o2;!w zPcFO01m9jhN1C@an&A)G(VwI96V~+~#EiXe+t)yv4cH_A0M61m*I6dua7moK*nmOD z{9;M4jJR~_bM~dSe8#|Y6k)XGo4qU{)QGbzp2<$zySX6tMuJKg(WM)r{&bwAf#V>L zed!#zpMwZMRv}aPkwj9r5I^QQOXBgemqFcMjY4d;C^{PCz0kw86=^~z8dObF zJ(A4T<>hwdt}n0VS_jzK3*1~#2gGO0vl3S%PMCfbp><_s(y&arHr7M>y@Ky|IPT`$Nb~THsNGUsVzH7ZRfImz9U%H7{xi&^~;b zoBHO(urW1vuQLEyvNH8tt^F=!k?&zO1wJviDurjyNdQC9zXJv;vo~0pw1gZ=tXQ`T=%) zs7|TQfGM?)&|h2vXb9#VfyEHYN>A)0>yuifmW6J|euOZ;p|!;W8mgBEyH~>YZFh|k6b*0vvu4^e2a4a)iX;YD>@sWoyJ9Ws5SG+JfMzF{Hh^s_*?9>LP*q<+Y2fSHZF zb%5;H%zHr>1@qn$!q{{_l2OfGCx5h{68`bbf{PR-+2&s6kj|};J~0yvtIDPW-|^3u zO7jN})e^I%ey3St)%_e*sXmSnB$?B#V7Tx&k^VOfrI)hz`rpX?JNw*fm45dxGQ#wX zacu>5SGkIqbZ6*4=@m*p#h)23Hq;^3duFR!{=x&Fg5#xG>N!uG>+G1shTOlzfG_Og#{jFy#15sozNf{|W;FrnEv-@-% z90Q5y#+z{2WIZ~IXZdA)5c(Cs7pNglpjxS{1^?5aI(+(%Zw_zgL?oe*%pJ!oLYd!H zH=CiRBVq*M+_3;USoS9d7pVsPj60Y&X|H z97)PqCSb(E0GeaX)*KWH`A&u%Pqgc9NtpcP7~6SB8(d;UAl$94{+P`G2akA3gGbDg z7@7x~oW?v?!-_8rKdg@m2!1mZmAaMR2OZ{R&bG{X+3@i zicdixId%?jB!#2r-K`(5_fNj3of$c3mQ0xVV!zIX5Iq^xoe1nF$MlfsOb#BuAKXrV z%@Yo*JrCWBAdA%Z_(slawaNhpUWzqSM29}Ap@fI}ESi@mzlB2eiq+{fh&XG3Ua`AR zHs@)N5IR@rVdYQKx(=h^sL6w^0RuWV_3vbJzv!Atl5scO3sAehTA*}X z*yqOd98ozqP4|dnmJ$B`j#q#4sjyo0&Z)PdgiUq?-Z|etxp1V@IDWrvpoYlV^ z{7x-3vZPUW`@1HZCE#jaM$rF~KK;E!wr)GxJ!ob+Yqq@oIzHJm!L8!U*v~8RLGVic z$HJU{!@4pTV91rZ!kN5sL~vk)NoOYEcUge^0HXdj@BuB4`pCk^uXQq+V zFdx-ao=1YhRceaz3gnetqMXS;D)2=Hx0Yu_4%x%P!lV=wuzXtyTsT&*Xr-slQ2Q)| zSXT_}$-~nmBRbCa7Yt-Mu1C{C!f|;{5~L!?CPc)`;t7Q~0^wMnl9;u^!&)xdi^9|o z0r#J}{pvZl%D+=eSco_e9hzdX&VT4+DR85MBzn;OBxh1aWme$D`uG4e;82jb1wc;CrMVi-`djYU+qFQ>Rc+gvB2ql`ayUnO)0T{$wF36)_ z5UGQ61=CFEYeJ^qB)}33@aaA%#7lpUoJ-8bp_7I;&@rX+y+RQe7iarqWfL{<8%*6% zN&8+LE^+~}7Rb&5*jn~7?C4&4 z5OZW--(s~mox?B~=~LiW$jHe3vipyk%(JHyFT$cwj8~!g45;MfP2Z8^A}sOa1VO^d zVB&sgKCw`j4c|THNz$EB>)e(f>+-YfnX#RwbB);L-cixhW2$|&am7YU5l_%@1NYUl zPO(}r^_z$-?wAg^ym*Rok64{91Ro+!B&nxt8{;B(f!p!A(Z{o5c$VTNXa^5*C>1ZU zJJAUtFT--xNH~rA8R4y5{!*K3YlRYKEPo$xYaq&Vd6)|N`9JXRCG3Z-sj}_1lgozC zt3uHf%Dh!LlOp;9mZJs(Vb#Ic@y$4}NlXNZw<*s5)vG~bdG4l7suS*}wrChn1^McM zy4y?CdeDm^kPz|gw!57@EuD)8y`i9&8sA9*j0>;3Kix#VO(=L-Q)B!my(c(wLN z?Kd3}VE$1{>~(C|#St$G94zwH!KKaLga%GExY^2jx$**IjzvOqQ`4NWXTTcj;ymM` zzW_}VN^gYzH>Y)G035bEW5V?@q9L(IKrp+fxg)YN2r8V8#x~M{l|GJ1`qrlQM-^D_ z4BLl|O|xJ>G!Jblw9!z({Koi^N0+!d`l<5`UKZMYPJ z%G~un5cQF&~U1mz*32I$N|Jg=N7R~E$c zdWCpizt?ixWaSahD?jVd{xDhRl;*kM!}dR%!VdnF5+CdId4U11 zsP9+Da2TnGQJs2g1dwjLNvEG`{j`h?hijVaIVgooR|c)+(t&D25|^b_s?&{)&jrnP z1qcv&_)7H7FmVRoy5Oz*7G zWA!U4bzyoP&`E+%y;z=8>HaY%2TEv`u!bMe*4Abf%MA9&d<2_*?hV+~Z~@6$1ur_+ zXtbk5=SJvrL!gt-pR^uI(ljYzNdVI={GXGV&6X32?pPHCUOt@~JsgaMBb)~Vh} zrF>1;ex`42UW4~$%73vnPc^mJqKC`k9atBJ_hdI^G(I9z?)&ga{O*cQ5pIXPR-_;L z@_jhjmCYy?A~v#!38D1T;EuBaavn(OeWg!7J{xWP5nlAJ^Qo5=q^D?TM^l#S87GGN zB@AXZu*3spv-l?NBSe;jUrk0J1Y4-tR1iEdoSoY1)9p#{SX}9Yp+pRZcxz^fzq=%_ znpLe=3y@e4rXl#yGW}`{asTUL7DXUA5)1@_8?Et`Gh=|*z5*CZ4!arvl9o9Tt}Jn+ z29|eVM4d|d`ub1xm2&?f7_ zgalSunFvdpgZ__oRC!$tdTfj}%$YxLZsN0!C?0qH@Xle#Lkp{_=+`HIOXg%gEw?<( z+$i~!arBK5>L_XyqWj$R&bew^bb*RmH@f;gbSwkfC1pGx<;}cD9)d(15uAEs1TOER zqM}$F6*O4;pCwGkqtukNzIniiL``@O*iFYkU}X~omS3O%BPD*7Y;kH}PW<^v^Md~s z9&q{2*M0p41=nj1SRl4Csn92Knq6R8oQx@q2V+-^40P5hyf7)Fner zzJV#UEo+Ey_i*1uFIa74lUb+kdAzQ&7wmZppp0U+r3^q{QvGN3ewp6L;%Src2*DnORq$+aE-nc=uc^BFBVryX2R|9DC7Ic@$4_J-8v7H z5+n1vaorB;;0WR=%3vZM1_kz%FdV*Q8bd9D0Xlp$Im`Gk*{E==v4p?>;s>N~t8Wdr zSXlWI=RCMDZ9)1f#kAtEFVV={t6eVz)ziXcIL$q)2WLZh$mS7CYM+2{tfH4Do|A0E#|F z?Ab#FxibP^7~X-@k8MqvkL@TAt0Y;4ljTqU>;=yj@pH?`D4TGywp$sLWW~7Qdg<%? z(4?19OM>H6_&3+r{WSXEvI9In@CQhnW8_C@@~?Cj+rx0EBxkE$M76yc`)rWh5dqSb zfF5HRmFMlXkcSko&Mh}r#InoBF$XLl@mf;9-Sn5I`yzv2M`kO+h|nudaE3Gp?&SXJ zmHGc3P5^0t0;z0zXLPtVuN2(GBhTWdf$%Lb4e`Gf_c{uV5Wk0ypA;TVr*!by?jLKb zCxZ+yr^_TNh-^@q2FY4tw?4^42#|R%jU@d&Ac)6Rh63I&P^!@4%30oVV{zvy$!=$H z(VY9A?->o>KtruxE5<>QG4b7u5L35iuan1O5RWDW1$cI@A?3ZEGUHZ3+Yh}m^YwBE zvc+f_I6$3)c@13 zgCL|v$rU&e*>3#DRdP~8tIHCmTYBSJEM)LfCm4UMvzbzyGmjL>kC=rpwRt~XWahMH5o zOZ$KbZ-Fu-511vC-unmiNXB4h#O&v<8*D-iLq#xrAO3v2uRTz&kRQix zW0sYvTAfNo+sPoi?7hh$buevD2&=H)fC#$5+kCP#I((W2EGNQ^wkMEo`!ya&brR4@ zJXML#j2df&gNFGl{XKcu^$jC37|jcd-vt&#%0K{euoUts=~W|eot;3%he&pEmHea2 zxnzUD{xD~xL*SA1ae(V>qoV|ydQKoH84STb0PPgsmof81J|~;G8|uZny(s7|8-rnB zBNdbP*2HJZx6~p!;Kp15>25?YvzX++U%(pT+s@B5)dW5uUO)|$k4h6718z@qDt(1C z#M{z?1qDpfhIpt2V@X@uOk=G{n!GM~7hl{5Tst@V#hVK6^E)0AMQ)@scR@YvFc8u3-b zpx-kEs?2$dVaP}%bopJPLH(7bwRMU#tCRoi2TQv0IXykA&>nIB%l^|iTGGpnp=4@- z^3;1kecoMc<;v5?3?#k8w(fCdEkwQVWw-7(*qsWb2)hV{R&Z^2QfG)(h$@n{y(q2i z7$a9AKRHj$?xASS7K&Wu7?nhU9A4IR*8#WjAuX*@(xC?d)DVw~?5qAIk~^0&!g6W| z*i2G_P4&&^d@w^Pwe~5n!Dn$qkcj`G>Ljs2X>xD)&+z0VDX@_&XMqDq1&nn_ySlsU ze14FO14h#ZjR?UOlY*4TTgxjes9=qPR)PToFGAKPq_sU5yRkXf_yja(vlK;9q<)|~ z;!H|?{Ph1xh=D4ILRp0!6R-OnB40k`FX05YH6i#rQ$h~=--AGbK1@EMe%~>Ro%Zf_ zOjkq{(?cOnnTABy1A{w#f7*wt$&ZvRnp4-E;QM<6N@*QQ!aS7qg|KfCoD1LDH-=2; zpj=qDjZOr?Rj{YJ?036yE;L4X2hXmQI^4wX#BFuGL;I}>90of<+`rD4|vG9%Zh*3uzPF#em}PaNf&ZH`Ql&<|TtmdN$}JFf1Us#C{~o9Xehomp zKj1H+n(NU&iVjSa5tFmb4b*~;P+ST=1rXyu5y8yY{1=1y-?~Wh1abFXQfLwgd_=@} z0~a5gX6qm^9FOo27%~E@S}l0&lIhO1bM6wND5Pd0yfnQZagWeWF^-<#dNn0IdRhQQ zg_LzNxW2A=eUU;~?R5JWMmYYa_2`8-7~5tgGkjuy=6o>-y4N1*%m()Kz zH}E{WP8`F}oFW;Z#!F=NB)4O{l&O0uf`b!vumN&Z;c0iAW6wGJ3w(R5o~`V=l^#yV zt^S2ilr`!*Tn+3x<RvubwUChj_KUtsJ%soXI z`0ZRaWqlN%Wel9+rF;|ZC+;!;=N)`EP#VP<$_J7Yk6Uv;bDOIhbQhRqGOxIQfODil z44jUR?qe%m_;Z0LLPA2Ic?9za|A7U;fzd?lvT?p}1Y098wgqe_-lD>djW0xeu9bs= zg1`V-YY-|nG8Hv7P=_{oxyaUkSV)XyY2a+d< zuE_gR!S6ET7?*T`}N~j zX2Gji^y_9JL=|tz9ClWtk{t#+dLuPXc1O~J;Fk=Toy-*3PTRPe{3=AW<3k<_3e>1b#w7mgX^K*pyvZyRNOm-_48P< z*``yZt(#FwX&D*W2r6mAdQbWU=4UC10z)ee^C=D8(TW@oX=ziYkpn$HPE4Qk%DuZ=YEeCF zE3XFc*Ssz;*LTGFd%kI^_SG#1q%-KH1osSCc;q{ec<^>CNR8UYsPMBzz0A+h2%k%N z$QVU*Al!xi(*$a^5}|*q&VJT@L)d{^`Y1Gq`-I0I?DWdfc{HB@T&MwNxxZjbo#%AK zZ!-qa-Xyl81*cDpeZb+!tSgc@@&Wj_F#N9uDny#{cwW$hvG7iXwcL>Zskr|2N;(KJ zCM#UgXgVCx7j_E|GbseseUMZRh6Ax_IB>*Qx}U`I47=v_9Lw6Ph=1)K^b-3!H3TB9 z9M|qn1&K6z?YdmIrnGz(D^>0!|@C{xU z;nT;w)y@v;xW&YRL~57#-a9Z7NPVGV!e|u7`Se|<<3@I7oZwUX>Py}YVeXf+%c2b{ zlY#J-ib~3WOgZ=4@_ufxC)e09Rk&@EYOv-b>`IiMLoQKaJ#rO?Gf&v3 z^g!O9`eOA#|3+58-=$mrCe@#oyp*?r@`nuXc8Aa9q*ktG3u3~ej}zshufJ-7vWWY* zSBr=mBJsvd|Apbw5-zO26zIWYPXF|LxjkEj!-s1qz{-v#ncv>rZar;nz1>Z?|KkN9 zMJ(v*A&+VDuTt)8Lb<)Yy&nkAT^-j1ix7l-r#?Po{n~N(5!dVlcm{TWU3(|%(Q6ozA+gJo|q)K>b?absf9r>$oEZAlxey_+Xl? z_{8w1gpjpYW5p0)@%M%`5ja5KKB5b&sOrFp2vm2E>OKz4yOBUp55R7rE=p zy5->d4=Wb_A1tBjMwzy2pASz-ab1pCt-RO@$Nr801b=NzY6 z2Q|gLura?U5zh_5E|JH#+`<~|aOv^3W@s+YZ7AcPZ4v+Stw0_uF)evaNJ7HV5b~#5 zP8vWF3`f(q-~4X62e&!Ll|VneINh1bcx#=n&*C@QRn^GRWilqTis8X#uDK3 z$4e*Qb<$vqH3&p3g*fhh-5@=6yOKfLTw0N=8@hx{t~8$0EiS*K;Q642^FoOPqkI|N zaxf_mvttDaBdF$EuAnScJ{XhA26xw8rH;$}5`<)AXb8y>km-ES;HDlcHIl;k`D)u% z__D3+RJY>&=XlyildKj`yx()9k}-DrP+fciCX*UL0(Oo}ZFlhzn$x<0y{ig1Q0u>N z_UDXRS@;&-e%tYKaqF~^@L&hMF{z^tei0=!QHGcQ88o&aTbL{tzv7|j-Yt#012j>9?qMO7`ezdrB zZM1Y=&X@{aWq(}lR1~YW3t=PIpUN%sY-^tF6R@tbe{hA31;Qzcx&g=7F&|R2cp2R| z`(lz#U-SFo`j4w}&H9G~_GLnEmTnIP#kZ^1f1*RU`FO$XWMq>VO^=(H)0RV>^QnN( zMAMpA)6h?VfjJ2*{2*3z9Ejdk`*z!%v7mW+N)73=4X2X13piK|VsRy-DZBY)wxvEM z&0;~0Fw_Ats1XE0+X)kPxLmeJ2Bs(Pft@#NlIcZ^xBklah*QAC3|P6QruG(St=r%b zhLazk^#jYIz@(jKhB;UxHT+4RSC;%m9^%FxnT+%Pzi8E8_?JrsV9F{#qikm45eat0 zd_*1K{+iGaQm}+|04K)@f4xA>rhX$U^N+;92P#0=y-ZWkI23%h;LYdI$SoX&RQj-D z(7W^b_%7k%_9wN(pc$CS&xt$g-Y4R?iCSgS z$sJgkLs{uXV3_MpPx<-td4jC+ISd!6kHC|? zNHuY^__plEgS7kd`d;!~Gx4h%ZDuV1>kOnMsaP&1FFiiu@3>kyWLLS^4=7&*x&1n_as+I`C1j=hIBYIWiVDX)4BKE<~~4yV2q#tF#?r|<{xvl-Xu2Xnd& zz1bj}YNWpI_9uazQg!=q+J2jkL1%p|RQ29t)g^&cf_ zfe$8aiEVFc(gKns3@?$snvhpNK)gI1BM<6_{t zhk?^a;9xHUe3E68YDly}CdG+-YV;Sy{VTykRI6l#Ai@x1*gFWjh+U^~>;8hkT~#P$5M;br`t{UTN#RfgOBDJWBi9 z(z-FY4GkrzMsHkhCp=J4A!<=<7e))1tP^{IPk-K;54v{?S<_{N2Nu46PxAV`@Y^YF zVpxW_j}T_{>dn|Eo$z|e81n#85{|cN`AWL>A&2i1S~ts7TylYr%lmUe-mCJ|4kqaY z&?0HJ#%pAXSztWT6AtiAs912gArtkMM2*EJgwTzvigoiW?zm_dEtSD3byP5Gzi?3T z?T)&zq47kt7vB#yRZ8hI8UIm;s{=%Yg;A~H#6L|(s#Lh*jnuje(g3`=TyFtwQ@dw+ zwU-Y_?W0FFTtlfwKA7%ot9!9F8{F$zOyAqNTLNhFCJf7!CBr@hg3_GFvob>*UT3ol zT+Cj5lY*-6uU4VBx*IiH@^~a960^D=6FZ>MirdKW?pvOB7azJTP6u>{t$psAuv}lI zi+hJ$9e!V!-g|u#8FBTy+`eVApZAB~DE~qI<`?B1Ei1vC`d}?sLXMX`DvCaR8jRWq z&|lvyF){+@j7Q!vaL@>UY+dS!eSWT@g$_)GbkXqv8=<@#xdi|YBxb@=Y_J-X zRrZU66oQP%Nf{&luMRp;0|B)qC~No9AUbRs-RN#*gzXW?0=E<+SbvL0fah|xFi&sI zLbz)MX#0E+Rrwzs3RwqN*v_QXkMxERRckR!<%Bdd!C!L*x1rK(%2lFtaKQ{A`ei4y zoR7H-^r%H+n(;gXUt-|0$5Le(`a<8~Zn8dT={z4iLIR71h*j;D1t&7M<|Gsy38adi z{!BJ?%Ua##E$|2vtryj|UV=(4-jD5#wLnE|z0F=%St2SwHe!3Y7wY`^FDx$i^BU%B2tw4k zcQ_w^q6NdapS()V&a!O_#4B!x+*sN(UUj7Mnn{-!?zmpqC$xKwf@-AM->Hxw^cU4^1d019SUjzxr3tQ`wMg#~K)JZ*Uqv zHO=t#rP1eUjh_qXm3C&8k2n`IW}G4rrq2sm&G_>z^4Owy?qUBgt=8D1+sfKI6$&(R zJ;C25jWLTu=uT9pE-r3n6^Aae0n486F~&_N#&Ulk%dW1TqfO|BxIpqc4_ZA9vtW19 z4E+q+dO=~O-2!Abl(BEz^Tf~SemFIH_rm7o=Fd&~p6~{GSI+wyW@yPgCc&VdGi&j~ z36or>fZM*g0>7AZ%u5Va9PbetWG&DpN%=QXtH4=uX4vB3UMe~lk%%x+WW9N|=j{8i zJmX4F`WY)tF+31mJi102v@>@uRu%=&<@Q2Kv!lDDtpUscj@9q$_%wUui*z^-VwOgn zaOs!CCsLnSppAWgCPg&UF942VWcDKoq2!3r9SddYC*=S$Knd45J3f9d4rD7&?pY^g zv;Nf@<>u#O>2sE=T>XU~reaaZYDnh@)G|k#QgTI(I(N=-UBx@qz}87uf}X2u0BP3Y zEQ9R^M9;zI>_}9&O1j+M^D=A8Al&v-EQGti+!o%PXEM7124bf$z-^BZvF4>xBLBCf z`w#K15cC}Zcsl!S7E>dHHPoCmc$q{MW{7ZI>dMXiM-qkufh7zhxuQnm7ZrR;5i(sz zfC|A(zek_<{#kVmD)ExV2=&4G!VEc4qIPZu5=52`#TQGetsXY4z^8EMU&Xb;y&=KSEt?BwVB~1eD@-R!vuHe)pJX~&&xBo< z4?zK|G7i+_d|T(QKjx-Fo~LtrBsMWeP>tk1n0&@Q{%RKW_59g;{AXp0=&Yny3V%|j z{~Z0Hyl&qs+f(u~Mm0%_y)jgX#0)%WLtpj(aoYD^qa6{evJ90GA!Tk?>)RM^ z&W@$esL|tX`tY3ZGoF#ia&uzXleVv-ZfCS)eAfX~ghV%{b30k7LK#sn{v26_aJP6w zw1+t#9}H)U2L$0HmvzwMVl_MO)>wDf=Kq>DQ+WNgplSc>G;Ov>W62tJHkO_Kh;jwE?%&7XTKbk-FoPijec6}xOWndpECh7k~B$(rNqZ)^t5i$a#~Y&@q$ov%G| z>)9BC(;4M>tvttdRe9L3KG`lMkQBeN&axhqvwL{cN9Jm4pgwm_D)MU|4|k&HPFu1Q z5D4BuP|E z>=6c8Egej|;nV39oBD84 zDTKUb5CQVT_^^Uk|I0uCPF@}nsq#^>GW`G386l_!Cu}_wSZ9SL*==W67do(?ahQEm z@*jpW@EK9X3otI3NC@*~6?%Gl`?>HJbeRoP;A2wSpb8U0TU5{}0w5{~EM zk7cfacTlnYh=jB*h7@eQAi~UlZ98_iJ9-D1dYW%_&1NIDM_N@ayIFeVHE3prlf&-m zDv>4mV7kDFHbA}3v765H@wVPmY+vgwWi8f(%Hdv*%qa^?A5YhhuZxp`*5|gR*=;_q z4*Jn4zl+M!$kr6CT5c5H6QI8s`)$A zSC5W!WM)s^gdiCqkB{z(5ajCQ?`hsGi`Loy%%h>SP(y#neA&wN6EE$2Z7*!uH$01A z>VpFL&DA`A^aGzP<>iX}P2F!@KDKNjzOr{myLRi2Q&X)O9LBc<)2)lAt{*Tng62=)Nb$F+LP0&W_4_7x<>q~O$Uv_iN)jR*FXJQ#lQDPbFov)@k<{YK08Vt4F*uq*FIS$I9p1j}^p zHfux6)?)x&o^gtZ*A?Az#LF0lw+u#<33LHt`3#lcAKwc~}Q)8bCmZI83l)PmQ zEDJ#hC%|g*&EMRwX9YLG=3V3ZuQ-G?@>qvk3C?!M!K;*=O&cU_!LA_@yw8)rBcV zpdfdc51LdurQ-~PdFJ2-x1(IxIZWNc@?D6_JJONc9Fy!I-gxLw6N87Mu_eB57r2pK z{QO>7oqtV|=Czfz=2~*(J6?tM!YYTH<pkJf6$U|E}oDVIaLy6}Ll zTJlXZbl;hTf6k+m5+?zEVkG%A3oZqX&2bw{Vng`Fl(*cHe=mG&=y%pv z6O0BYs}gHB*O~c%e>JXHJDTEnXBFWpsW}Ykm_2=aJ&aHOoPBEIo>f0NgJKo2%ijS%%7DA9JJ8cb`GQU+!0}X|FDq@kQv$iCd(v&uqWy7#jRe z3(T|_ePAX`WL8D;O;-2>N)MXjUF7YzGwewZoj`n3x+_?EM66zzlwO}4_kL)$)r-<2 znBeaDoG?1pYFw?Oo#c~?A2A=5o773SM@iSwp!>r2By?<-x;bHUja}aVC_Bgc!KxIZ z>-k$ikU%0|75+bPZZ-0Mq9;?>R4iEmfa}=oh$0&JF)pNC^;G^_2ilHy^8A$XAljL-;68?(p4uje;-jBC_ zO?5^OZ`>klgn>@L|0^3C9ua&e6ACLLCUB>UpcsUpR1~ctgtNgL0v`PTEi_hh;kdRm zIIbXsS>%cT-}=pu7{cH64pc};NQck!>6vHZSm6}X* zWWvI}kDg>F;}M*uHGXL|&0doGbxpGRoPy}rMAN&Q_m{s#@+82W3m3zyMD~CYBvv<> zG|YBK#C=&!*>^wed;Z;Po~F1#Y>{bb{m_Gv-F-marZ-~_*5*5F{2ky-pQ4?c9npiA z%&h~Nki79Ik59*NW$QraPLy_~SUh?S)Cv}P@szp5vn}3W3i%!#?Bw*zkDOIJaMR?{-1DJ4TsgmRm!D1dOjEM<=&QH*&2*2DAp!Q+PqSU+ zGc>zRiG{2-7Gh3>7rb5+l|Wg|qDZSG7&_(HnN{+GYVx$g2NdReM@_A+#uS*5doP+) z7at)r^C9hG-ZwVec$C*V%0=;F3=Q(?WdgLJI_)P13L7fL$SIQXxn_wMe&yu*v4Q0{K|Pvi)}Xyr&Rp!_%+Jql~D%XB^?_}<7^shn_eJM z&SXfND$z{$zF3xg$L5T8MC4Mm`*5QMAn~4b?Dnd$zT%Y*4#TgRvA-W;zyy2=j}GR)^Ni-F}*OJ*cE2+uZ*m>oNZ=|U!3+9+DSIexf z&z-miX!v+}MFC{Z=LlcU`8^sZMkN|k#rf?&giRETP`J3%oJCr^P71c8@CLw2_#3;q zCqXs|P|YV_%UQI(i&i|l*S)$}Itv(`DzW^C;0qz6fh|NN@VS`d&}vkztU;Rkj-kMBv%mvDbfsa;N8p^|i^F|Efu05~luqg#efbC^J zMY!|E5W3CGH$;^tKWqwqf*2~G)Ozo%h;37#X%PeWB7vtwLPSc>anHlo?TQD=X#RDR z;ZUPXzm&6m#CuDK-i36@;unvof;~Rrm8%zc%963UW;%`*TpZ=@Wjmb@v&|TX4CQRz zIWm~t`2?1Ms{|f1s|2hX3kXPPfy<;U4FW$GSsR+0n~5W8#?Hfx^B;s9PRuRLEqIz` zp8ad~?H}@SIfy?o58d5ObrHjJL2d7!OrH2*q*{BWede;Zp`UY2Ki^Aa?W?rz;^7C} ztz{(#xVZ8fLkEmLI!luG@Y(#aKBju5tlAJ1s|53nJXsm2ov*Q7H=nEAM9XM*eD|fW z>(?w1lh->kjjwN5t??)H1ilo$(C>HWnn>*CuO*0p`_y{n%Vx+kL^W0`&?_3|zUB6- zIcnbSE$n=cp;IrhQ|MD@E)H?(*-b<;{^U=g`=;dQc4~CSh@!}WZvBBWgznAO+RcTn zgquyW2lJl{uk9O`je~h7xQq~MuX3p)TbgijMW@$a7Rs~9%~wQ-^j4k)*ZaD2RiWr< z1#UL8G5uegT<5G*>Fdi<`$x&?`#&GI{BR#lnoW4YQP}C`L@CkA{;rOy)`@0+Y{V;p zT1+pD_r%TcU=w31S)EnCHrA=`;B}Nn>7Q?9v~aiPH=|e__A{AZ)2~~?h)@ysrriQ@ zoMy@ t2Z{JM=Rza4^Tpu4?xI1#KXMbArEa!NC!zb{H=?0DHm>ctV_=5=^jNsZy3 z9E3i`HaAnT`N?Fn$Xj2;P8i*b73Vljt6L75ORh<;QE(Re+74~)bMN@*iit(-*}-BN zGWR##S-gkQ9i9Xx7cV=(1{^MEZh4mBzHoMs2-mC2Ku0D5Pd9&!lejfL_f-V-yID;# z#m`;nR_s~5$DUdaOM+;*?S?G{&#*PDPNE1HKTEwvT2Z?H;F=>(lZ~f@#()I3Uud8a zUZc5tZ6I7-1x$Z1j*X&8xl3_M`0){d6&f)U{P#9?-UTQ@l|MlxPk-gq;UdH0@N(b` z*bnUD=p5?b)T-XvNuWQKgkinRR_94VV}RzqWcjA@zYicl$Pz?|fM52r9|d>>frll1 zA8wbAKf++ket3{9As;EQn)nrnZz0S|rHYHv93DXr!FIg&?InlUub+$c*TF$S|3D#Z zJVA{jzV(yBW}$(@ZlOWBLHx=A2+{6`D1$%ItSaz{z?2C zlZeH!q#ao7Mcgk(v6G3mxc8nqfmI;-ymF7;CXa*i{Ntd{t3dl~5gp$!2pFh*GKRH? zE&1QQ{KYof^+rP?>hk4Ma;o0XUEm)kkuh>lD18dB1LK<@gz23eaM8}ET@RDsY*foCxjTq&4 z;-g8)zv@va>0BTbI`LAj#2}vl`}vr`3@%mW4~Ly|#HpIZb}ZVO3Pv*0o-L9remP|C zyK5>|_lNVw1O_9)&W2ZtaYu2PI&*L_B|R4VV;9?iA912VuM4>HUJ?(4hC1<;FMV2j zBc2%*e7t>rHc7F8>s;Q+?PQs1rFVWld{I$mgJ{Q;V=Nf6*2`~!t02wZ3pG~p{i;js zbEC0iBBEG2pWK@zbn_+Y^(XBBkqBZQB8u7JheDkw_IO!{Q+J1Q0yk$#G;ezH^qPdu z)sN@r4Rw1w(2I3i`IO$DXS_CW#9AB0d*c*-h`7t^JR#E?4w#c|wdb z^hX~Y6x^~HJ=Cu^vv_hT@K{Evv}KiRZ*q1#OK#%n@4^TkHtwxC%C`nbR0muek=Y)X z+1$31!EJfG#3Ya(a@N#Tka?hoC> zS(ASu_YR8%P-q*stI;#h`w8x}W>4<3-QS;uV@bK?d$Re}K_nqSzz9@C1YzKoYg&1@ z0|7kMJdspq1b=Z5_2Y&b+IB*%2Ka>^Kd2&!V)-8bMQ`;y63^Rm+b$CLzz)M9=cQgX z?un9pba|n0Yuj00m$aEFC&U4(uQ?!n3wTUZSdHf*2S%ZtyeJ2|8*|A#N|4`;E`{al zo6Ti9wYhFP(*T2|hqZ#aJNhauL2TRA2ANJ>R6n_WN_)EqjS%N*HbRYWV}|Q06L~}= zF`FKB>CuCJip0U&YFq&*DbLs1wNlw)3f>&$om46PYcv{Fn z3)Cn2mgBR??p+z$Tn~noM6G|Ey?UPBhWm?Rv)@IjTr`fKrv0yL7%lgl=`gls*hl7O zW+c2g9D81uWl(8vIdZA6eVI!YTFQfK)u9zex}S3Pdg-(cGs<01@X9uyQ{)Tz#ExeL z2>o)5YcRod-l#$MBbTxMr}()%uN^4L)943I)j3t`Ln`gdj&WYl3K?8qEvzTBBEOxQ z>_P2VNL2Mbm@JDwb-;MLDkXk9RnWwd60=+EYLT17)JPGnLyRWEV?J4Y-sa8GSjosFSX zWOT?@UE)V_;->|*0c|586OX3>W{#<9y9i?=fqylrCC!bA37*Cg2-NdI$Z6qxOz9(d-%##N&}594 zPc}y%SH$gy{Q*_3#$icV3@eIKh#%C)Q)Rpk4i3QcGV$=^vnAkfz8GP%*u-6?RVjW+ zrd?^TC%zZYVUB#p__`YoiJoC0jn^<%om1;)z#ssurV)#k)w8LxQ4(FOhr0Nlh9~KH zPCk4Xq>1C%9f>dTOWt;f|HEa}xog6yzB+a&`Z}EPMv0l0G9vs$J4ackf7Q$C`1hYW zW3;`Uo2+_A?6X}SgEtb-Ckvl3W)7dEw*HWUCB(=K3!;Qbrl8(=tv*DkuG%j&#rC{2 z5KS&$Ajr8kRQghaRm-RP@Y(gH-&dFGv&k?wkQ&%*&>o5L_%$5fblxq-R=;N|H(bis{Cj-kr60@OWNaZ)iAYfCOKj`JuQ?iw ztRbEyN89tbTbG?-?W&o%r9XBJa62rNuXkhL7YG*B$lcPc>Jy{=!Cceb^V*dVf-} zYfW<%hHg1JJ*)HluGPy#ZrHZs1%DN01iTV!ZA-)x<6V~BP4nA>oAR6Z-qYOAFOO%_ zIEI&^dc)W-C1J~hYf7*#4}ERI3UC2@-=f`K?^Fc&@_Ov+c>|`{`qZJ`lfPgWMV!aaaLiYa**|5*Sa6vd4u`%sK#LqM(pVus306DN~F6va7*q&DzbH{uBb< zd#qQm)H?wNJLst`HcmBG2pex4b>#Othenev1gqpbmaAQxP&kZ?;hkl`;UMhiFwZ!I zSE{jT24$%pS)iJ&5Cv}R-C0d{B-U8Yp?yLf zzBHue7XSF87ybbX^^Ynk=wX30IIm*}`C7`MUSDlysn*%f{-sHt2OA{gfWZ(lI%62{ z&ZKuvwYUkxmkK7Nh~@zp$deOSbh4jAa47iyT6@lM0M86ozgHU`fBPVNDJ2=;|9zHs z!3xUa%tv)01Dd0Bz8r`rFYEz3)^&S|5kgEWg%t$-TIC@v$^fl0PEfMPaEO3+?;I0* zwpV{ovhyuVn}N2!kwv~BVzjqT_q_q%rnO!pBbz&jB`GXfZHU;7P?xgyy8RL z#WJ^Q_+1_Xh2lfjZ-Kfhq&U^hsca4z4Ix8qItasHw0u~aqlx#->;ZUUWTa15Vwi@&r35QD z!_o7Y!>I@$iWGX*KNMpy^Xj5?cO&4fB%LTx`GWh_KHv>OqPhd;^0=Qn>8@svH)XnX zsegF(aMA%kAR|syMPzI)oRAsC&h%q-2>Ml|EYD%tWDVJ#h@*I9uUB2nC{MW~^S%BZ zn_+7=W1_C*9b)X-4N-N(PBP_%VZOO8zuS3LD87myPpOR_ZXUk0wRyPDK{>1EFjFY+ z(w-8`cD^d_X9w`+@g6f+Mzuf+1W2fg+KUAj%b8Xm9j``}k)rJWP`;ZgJWjBpc zc1MT(?G>a_8}396y;qx2j0}dZ)}56eTk<=Ij!2R^dHsm^}94|;Bfgq8=ew6>9Nzx3O+-!tDv;6G2@`oTf~d9eXuUl z(8RjOYH{7O!}(@oh1AX;VoaFdi<@c8STOebvb`VLd z*@aR_&Bu2LPA)n^t}lo3!F0WYp*2uaX=9qY_(YKlHl?7dfxF~Zbz%z?Ma1im9|X|! zTD{d((cz@rOlG5*iy5!dc%5>SLg|5)4;OH_sb4(}5m1DCdw1wt;-9^02g^9k-0f`o zO=F->4{-on*js=5Gxg>&Wrr zkDVQf6p$Z{+3+RhBp~gk+#=dZwAf#rg%yTNHNk;VkDt5Yx?EnybJ20UqCzdAF z&ofUd=%lp}%VWG#rG$j=;ce`EhQ6$8vfd~|ge5#08Cp!}^1gzSXvM0r=@v0aX zmg^cfI(V@ji#>}|1;P4ukS%kus%HgXM$@mv{!+bHG*b&y<&355_~yealDBm$sqT0+ zTiuS{Mb9U)wQtqv@o>cDLBou*jaBUP$yig%F<{2wwj+<3&Iy@pVl4An9&`Dt=z4V@ z2IzibIW*ttE-Q5k#pN~Cyx140F{s5&KZz*B*CSeZi;15kz#y0aHJ^3yY(-2B8NQ6#eAS-* zk)`~GtKZIuZH|gJfe>QBnmnvJiK1R?gR1cg%jhm9t4UO3ivSYuHwc3D8 zypLR2LdWmQxtrg7e&`oGH6#z4B2y6snNhocZ{}iK3Usz{+ys2iFCvDlTM;y8A9QV7 zTdkD_3%FS~E8OVj6A7t`qNavA8t%~2leF{AlRlJ`J>orv5>K9^AAZiL-hJZvROK60 z!MiW#?I@!@sR!T*3H@j!Lm>407zl=`;c=Swy)VMu>i~AZ#UL?-+5^rVlMGxr7m4|V zEascI<$n!qwCC@$LO9$g?5=R48e}&n=HiW<|G=&e%Et?S!>vSnDGHL{Dk_2ocM0BQ zEbVd>oY!uS>*&cV=@{}=Y)4<&irqDdClP@%k~d&l1P}*qZrk5uWwO8l>f@m!36T2| zOcV5~(slASYpH*^;Pq^#OqVB(+a`(R3ur(og}mWmeQ2<^8Iy&I&D2!bKmN&2%hk_X1g)LMs^`G_|fcjhHYb%Y?02sk}13->bpU2E!8ow6c~D}Ih;gMMb^PlXf$ zB>{vz<2^7b&Uhhg4idvCm>RyTElDaT`eTqx1d#%9nayZ+64 zN5ZP>2acL-UYR?U;aWo@A4hC})oF+qm-(vixbS_9T-F=pLjQenrR2_A$9{_;b;;6d z@~&fKTD#&t{bl<3nNfC=5hfIIy7G!=-MrkK{OCQkgai<`tr{N>6R5Er9hB5D`TqRM zVactJhWlhd@~e0k@fs&SRVm2}n!nyqY|J_^L-$3}^8BP6&g&gM4BpVdqS3lJ(PQ+npLRUU(BQ+qTxy zad5T$eJa-3_zq+PJZFSr53MA=Y6pBPp}|`jN)R;)VDwF_fO(w|5(>%$(4~Rt<;i^dcev6yqKAFvolBX(I-1UB!ne%E+1@vrjEk?ZzJ)KGF9IbTrR(TwU zLZK?0ayO!n=@IelpMb`>cz5LA97b?3#tlcrYx%eVz~Z#`8V?dbZ_Ij&j84~JmS?^{ zHe3!*VOJ#j0?pk`UHha!62f9t+;$WCrZmlzIdrVezoivbKHG~w@u&Dz^<Adc3!^xhVV06?IIpz~{osAJMyP91z{8zh(~~_O;-CjJi2kNrcphyhc^+`G5v^1wp<`^NM5)(q3}ycQvD8+p)Yhcw zZqV=f+k82q>?7Kshm9(dSW0%#SjG~w&nMU~HWt=mBm60k=Cm!gQXbq3M)TT=xWl~Q zkXK{TEQn+68NLsA*ux#rKeZ^M+GR-lO>c>qDH1Y*7tdq3ndW6S~^ z8F&@2TfBeIVct{FyUQb7)hPw;zM04ow1%|1^**)U@4R5t-(6V*J#Ol4^AG}_~4ItX`8w3PX*%am>IOk7MC(Yk8hz}Y<&^%D-LzD|8CB5ywsCXX;I-WAjW=c z!Z!nzj6g%~0y7X@O3lQ5nh$C-eG87Dfszmf{uJGEV=F7O17P>RueZ1NRT~B&lh!kO zY=v)l+i1Q~urAExipkWI{8k!_iuz;cexfDeXJF%^y`3EN*Xz6aSH7+{z8 zttdB6e6$?;&ZcmeQEO5@+Kr^>H?UK@c2{hWTM%&4t3Ht@>raQl6;5*SY?(S?zj7 zw>{xgLqx@_fyLisk<-}%N#@Sdhd`d{P9v!}7gaY^?!yme9mrX=stk58^E?jBxv`Hf zb6Lv>A)hw$4=gtKX{WlH>!*qOdZaIoEod z2Ul1UDbAaClp3tVZ@YTGzH?4u(N4cELIk&%IOC|JdUkuJLecX)B-yTW?r_y142J@% zvq-0wDVjoHTB#6Y1u^hY3g&yiitKhwH4@UUODg({Ik6zLwq$IfTWrc^ObcNJBecEN z+59du^kNUz=weFb=6O}n`(eDs^B2me3{kb?iYgI;-2ToxA*fm4RIUFY+nmcX2{2m< zl8tU3*ua`EeLgwnGUKjl;M|v3;@OGWv$9HDIDEoFO}my*rXj*O2vGvwZ%o z3sD)}`+Od~hjlq3x(XET(_eG`;C{0kG%6rUE98Wwvv#B+pV~@mw{4E@Q)+qg8@{rQ zvgk{(sQUjZ&LRW_MA4VkU&GF6%G0~VbG4A5F(i#rN+^H*ZGa;2##Tt6m7f@2TwX3xlDwhzxQ~o zVBmDgEZX8S(Gy^q^Jx0DaLC6{F(({{x1G6%s9&bb_Dr!g?{?5VGV#t%t&9X|7Pi8@ zGx7FMVM0R(*KRwwaNQN!Cr>D~WMN`z3mc8w>P$y*J=#swE(^eG(P}7#L+6aW{Wf>6 zc}}~*F$V-As?)G40e3K{`s}y7q63Ut7tvLPAHSpzZ29ag=X^Q@YE0_G7jC5V9V4(ee=rm0_HSAntz`4~o_fXTs~*?5H~PE#78m?#K2vj0iYIbL`kR zP;3aqg~R!M?k_M05(?AGL(!~kBac!RxQpHrvCpXa{6?MFa?Cp|QyR(%=fj--4qb)q z897~5f#DZLpWs7SOWke))0!wZmcFQR&fLrv*_uP>z_-zAaHZV?xGsAp+Qh(h zhI>Ys%2*QxslUDeS58lQ(l6VuJ)`+Y@zD>OIQyG&>dbfV-MJGS{xnGgI#V84AhbEt z<}A$bb_uoevql9{{a3eycMA=i8GvU=-e1bO%=TfFrA6dw*v$7Ue zJp2ar5W*W5LtEoD`R1xdUNIeC#%MwH>U^X-k z>ibS>nQu#YcxVo=E*l(!eL@{D;2uJjw_dEeFF5O1}|D% z>pg@X7M}|cC6@n*5>4t@%JXtD&pEYf`x-re{%mT>*=!UhpfQZ0ti8KlqN!G_V1p1S zLn@3|pcw!8^UHTE1Oo=u$IS3>>Hz(AFX;yLGNsSviX<+cn&LtjLE-LXJdmrCPMy=yZv3H}1Wrl*YCa*0#c@6)jpHm7-vEE*z5yGZuA`zw$ zMMksQvoYm+TDmqd`+~oVUpb>W+scH9uGk*!^o3WeRVpxF-yPvQalORH94CTXUzSF? zP;UO~=Ojkobd6&Fs(Q}y+;a0zC)Wqra{oWrtU9%tc5`i-H4YVC%|e=+dFqi=$JdyV zC*AcoqG?>ghAo`4Yc{CetwjfM=v>CDEKZ}He$0F&T0#F(d6mVT0{GnabEUq;oOf?> zF=aFVs(3DZKg#n?mJmch(A`Nt7H86qjGM8K#Gl&~4Y_D_K;5ARKnnc%?ozv9ULg?P zj7z~ov)bT#Ibu$VhbIc)OX4sg8sKs8Us(JP+LQ>z${GKxY=VD($F#4Cr{(_PAZ95vr<<)8=dlb&MyvLc0 zGnxz(>cXQ1;lP$$&N|E;Q(Hz=7%fp89@Vopvwt(fc!1oNSZxi~0 z1W(MOEOCy^pP8qAMz-2@+{6e20APW$dJFQEbY0fgjaun0iKxb2Gp9iuXATh~_6zd3 z^22Vp@clI_9rV749t!wxm5BaYsD*iNc&5=EdUwL%B)xzP!+P@+qMVhcO$$xC_BvZ7 z1k+vEGClT-61PG6-*5-IfV-A3r|aMSlTjgZ(Rn-2k=d)?jhiw$ZF83|6~m3aEGI05 zH{b2;%^~LV1n`+wQt-4eC*J3LcuBc&d2%<#)X%~|3Ev$Yc}i{u z=$k_C+685p;1H}!xXNpw&$OSgv}4ou{@NSwQIRgc+ZqPL$1#7M**qfzhfkW%w{uPY zSB*6QC`zgoXRaYGO&bsL&Pk!akxpb*Sj|FAEh+Sb-R+wL#kBsliA@`OWJ&^g=S=D! zTwedIL75JUj^daSH727&&fflGPK7IjTwhi1(=U5+^ihzA?}fOm)ls8q6)5T3VNl~HTf$o5?=E=7scT)Xhs+vv z;bPjf4(0M((ghaoRg~VhkK@*_^h1-^*Xfh;2Y+}YjeA!(8X*-~OJRp$0FljyGkj9g z1<=K^s^}AK(rkZtDJGaO)}}vu%`Ez~5(KKGUiJJqS67fW(86b&ZN67jLn63KZ`T9R6hOqBmfSo9?1)zC|@|GZz-a0#}piY_Z&S z6nhmyHNk%4ik)5TlBDvph3_(T?*WOU61LL9Wn{_z_B?veObD-?iNk1Td0dsNNYm%o zPMK;&VXxvrC?D%8ATLFJfoa4pr!^PZMRhmjw!WbMFn2_U(xTq~>%WO#WPjJxlS41( zMZTmgP()~^dh+@0za|wnDxiK9gY!>HJLUEQJ7B~k(Vf(EuswBQL&THZZqZFN?vHsi znT+=9{AzM~{?Iv{=$<{x$%*2izO7d&%NQQv@Tm^}WoI5zdJVR~yo(Guoltp73a(P7Qp;>nutf8*av% z%sgc;%d06z%q~Nr%kymJgL=RPm0)gvHPIv>B*3-}FI^p$(!Jd+t=8>}g0KJSXF#E9 ztWIsHr=6(0(;j_*Bg-y(d{XB89wi)%l6Lmo>$`I7veZ3uw3 z{z3Exv_V_yi2sxm0&%#bQ?T{<7fAOfi8qWVOgf4rC%QvFh|2iXfufm=$9jr^lz*xm zVZ)w$3PH>`0VQ7y@3{nM%#+4}w;FOIyFxmzJw75w`J)msEdi79dl{LMwZ~{$2m)6w z9Md8K4mdeH3~IHu1H~G1uOLT=^SSwC9|5M=6Lg&o5W&+g&4;7VkFWl0e|$(YD}|@< zH-0QM0Wc;|&njm0Tn7VE0y=2gZdY++P{@nkU{(Tb<|KCOV^@}Y=Ziok?2w?$C=c$e zUp5|bhMHfRC~q;>N5v-TXJz$cgz&s>aI#tFaV_^~*?f{*xwwABwwZM#(7$nC^(E~T zd8TNaW1v;2;T2{s+Qg(^{BfF{(vb=kGTxzcGb>|PwP>K9Iu***ZvVU0kV+W>7#Npg zkfRPKI;f7;%}6JMrf^Y%81NLDg7xoq%<@-VY8v_^e*tlrC(?QjzmwI-CUL8PN9Vbd%YS<{ehtTu-~<{ zlM+m?ZR}npOhURTBC7G5bp-H1iiQ`R{M?^6xCjnaEVTL=0k>?I4gNOOT~S%=)(hQ# z`fKOf3WPhSlQc%07$DkV=3JhH!s#>JN21?UpeK?4_Y~)3^zYD;yz^P`OG3g#>?So0Um0A5o^*x z8*f5Yd0j5Z&W%uMrRH#dl{{fy%?KTEqWAzc)&sv_+s*=^zmrbiL-B7HTOF3xeH6O0 zbRyZuO+|i{B;-E5x07!T(N|`KhEkk=vv>?>oz&l5HWkD~^_tzwCUHask)A?eFtY>H zK)?e8C5RyBB-yjND#~xdj}^y!*v{h%*{(vJ!PUc2wE%*7EUsO%VRSmN07sY z$}Lg!aSPkD#no#Nzfa>;+>f|pg1?fj8NvbK&WS~0ja6oOi#D6vxZ3+`C-1x9w^dnR zP|g~I*0&>SYJH}M@fp7^ViDR0ODPiOiN{d{Zz$qzN0+Z2c{tA7BU)kL54FjDhQDD! zf5EW&(X{>UzU%d3weW~O7(lBxP_1wtH)|mtl-F3UU)R$?2iiKfeN%g26B-%U5QEHg zPN(nGC<7dP^Q$k}@Q4L2p0f$}?kzQZNQV%Y%@xh(;nb$fg{e~X0<9y|G5?us(iVO zAOz_%{0lnGC#z|_jiD|mHgLDzYv%tWTu?DT@}nxt(@8zxk8BG#9wX}GuYncs6+T3> zR7)(D%4MK;wHtlQ5{FVy=8^KKrnbDDWL@ATyRQJ1N#|=5#YwPeo^v=U_qUNw z?+Z{Sht!Nr+}_^)EuUlhZ`k}mfKUPEen|Hh><^KBu&zTu68vuJ2WItZOLMixP-7&6 zT-+kGTFcaEID;Zmo;lIBBhg(@T%MvT26B0g)#qmJHg%hN0Wcym%YnnM+|4^yG3eOd zcIsX&rvBAvmf|b)Bo}hZ^cBF~U}y1UtZPQVlnhG4nv-}b1C(}Q=n{goFfFT}O>Z`FetV7Hf zm35xrU&{cx5_tvKB`Q`LRZ#@t?P|^w2lrruU>R=Zy0Yr##~T1443w>JO~p&?w0ii_ zDqgD*dOAj(8ACepQJra3yNBk^<-tf5E#iweVyZg{zD8NkY(XbAYsMSnxU@!tGK{3X zoi&f$P^htC8JQUD@9DV4PJV2s6EGEyfH(6X7Z$!*xvcxIgmMEz`*}cj0EpI_xb+6< z8Nanl5Hyz>nzs6aPm|yJuU>qX?L+Uj-M`g*vWbi?A);^93b)FJm7%!$4X%qMYk%9K z+#Yl+p^l(me>T;w10-O8*Gmu;>6jN$-w7Eocv3w~tBy14zJ-t(4Q zx_900;ATZC?wF~5ZTh!J{vY?shsyj>1KE@A81Uf)5Hmf$3QE`IW!YgW!}qIy*0!(j z_bxh?&16c|-K^Ywkg#78)EgXjA+^1Y%*Yasj5id~t9>L$0qBbq zDZ2~xtTUc|@4^PSD-w|aden<|X=(v7d+oFceXCCiyP7?sA2!!Vvkc?#Ci7h~+0H1M z&w0sH-yW{k{%L7#UG(qQqAEO>pbCr)gp)8%Sk!z9cSk-ptw;J-7}{22K8cfq;b}Vf z?B0hzCf~552Klo4o95}wFT}bigd_t8Lq5+TwQV6w^0R|KN&Gcc0aEVu-5jbLD)bLV z(2x5Ad?pzq!~q^W7d!79H_6#?;IF>TMw6RCP+l^Zs*``usYsKLdm*pk!ro+wzC&88CE?kj z4UPXobi99#B%lL%Rh^w)$N%@Xuc%}}ojuV{wb+)>cF*)Q!v48TbA!2P-Jg0RVKe!x zxia4M$DW1(se|ls%=SkE7QNX@11MYv*l&NuJ!0h&qKdGtV6D8;Pobyn)VSM|33_0f z1>BqI^waHQDTUq&xbKd|FdJv1eEbiJBtW1d2%MR5Gc2-P&IsrUE>vS{g;VaM>8z;OEgpv`E4Trk2Vrr$bs}DGkUrRj9I8;G%GvR1w zJb-vKT^yZ-TwL}heop@6I}}dPC|g8*bSpmz2|^(Pqtjz5WOB zyiyxWA-2~2%9E^kRF-^orgUC5!za3?3n%2q=k62LDu?z0hj_YYd1GLcVuD*C(ry2i zo8Nmi9c*Fl&3`5Q_01r3r?i)OCoCpSHjp!FfLlg!zU>`i<;c=YMW{;HQu~8IhOkd~ zkD@jVC^&GOBG(S7M@p%l{xj4%pFv?t(ml842vVX`W1*`nHU6jrfHeJ7f=xldJi4)33FaDIuO^v@*x6){17ZmW z&fi*}d_V&OeLA)vEq^P&a;?S^(W`~9&?bd_4EqrHZHzlS zJ=psz1cC=Kepc5l>s(fz4%S-D5vuK6je`?q0vG@K&w@`I$=J6A?E3nj=E<-bmLRL& zRK6JgVmr$9w`;MA)T&_K^|`2_FJ)rV2>H8i%;)OnN4D9@U+l%PaTy(ESkqb1Qj(%u z&DMJrU|3}*$RT4LgxeG2@tc;r+51za{%7^#tvew^Fo*yrReRPyI&!Onn)1+zBK zhNFJ^Gi?3YVW)}8*Y`CyCmPGEyw{mr5$y!uf6yvIzCfWps1Lv7dK|4c2<%r-C1H8y zp)|q)e@eMroJH8Q*V?UigpYULh~wp(20(c)HI3;u1#kDlsS8qLzb=$2bAS3arG1k^ zMai|vsG^&Plz1Y2QYFOjDPy4>HuNj1*PIUGw$Jyr=(B79pRmuYyYTGb8(i>8ZrUPi z;;k~W`~3G$Qd~TePifCXJrTDe2Ct6-r%-Nx#|J=_aPzKyf!J}skWIv{$~k}detis{ zRS8{vyT>~hf4|p^1$MjiNez^mZPa4h8768AFM5H}wSWQ;y{$4L6%>y9S@7=P*XSAT z+dp@`MF8VQaN!fKap;ZzBFenbUb4nJoz|~Vrc1pS?I%ojVIl)Kl|qMS}|xGTSrbw*ASxWg(ju)7$KoNlO&` zJ=2@$)V#S=b!IKl4HV0tXS>Ld!L-EjP&bPhz@xZ0vD%9${gtea&ZcV7KFOAoBKQ33pC}Sp<6S%zPcEgDjp`?ZLZGw2}qLft^LWxF2Dv?3el9 zv%a|H)qe3%nU31B-0P5~DbV)+!r{9M>Ii;Ah0IolDLj1ow=$YBwucOf_5S+%3wMbs zn@n;}S`-KSvlQ173Ku1B4s+bTcqk7_)mnPCD^Q-@?H+78WGDy-;zNX5?_WSTYDS0~ zQ$EAB%1w=Qga>8*CJWQpQ9T^d`y~&%S2X(@W76a|1IFqjU$6493Za^hC_ZY624SNK zJJDGU)G30ZpdKkz{))nC*p{v?Y+%E>J6|9B`K4UJ|1V-VJVuNt6;)V*2@YLjU+#<6 z5S8b}sZ_$grT-sU-vLhb`^IgRgW?=JS?AcYsf>i`CN&-2{ReShxHy_VlsIqw(5ef^xZE%n8! z0pV#M1?|BkYMXT6#&`q^oK_?!#VrTmyZbwwjfEIi+?EDjt7D89F_@c>U?YfIVTga# zBE7v@64T$bnu|&ehug0CR0Zh=}sqkncsP*%IAnaZkyg zeSCY;OvnD{;DaySS1pI6Z%@TA*PgJSC>h&axoDEj@;yvD)ROZ3R@=*Ri)+<8h6YbM zXmfev>n`#p?pfV1jAieE!l|{vLdx4(PeP8l$SV1nzBf;RV0!0Wei3~%l8?4$`^e{D z5tGZpxW-#KH5VbIZqPbg7&)}@#;U(K!lnMl?aGwJwR=0y`OeEja?-F(mKBshZVg2f zAM>5)Y!Gp(QV%vHBu!ZRYDVG4Z6L*z^JA1lDHNfoG1siI2cO=ql;8hjK%%ynHKA!^ zB&Kq0r)6SBQ{2B75(r}B>V*sR=J)R(y~=;EX!Cm2w^sM7az6n$_0(cUYF#rie6?OTf$$89_<)S(e)RgMgN(q54^1lD z>fHO>WBONkb$!k{T!)ZAu?roveIUz9UE`h8iITzO&|fBOE{P4klM zG{%0~D>5Z>_ah8zeJ)K)*a$r63y*3t$_Z46;P}E2xj3(DS7}Y%_@$>p5ZBlZ97Hrx zeN*XX0-KFsu+pIg$7_ndOUw^bl~3^#n#Y~dT#KQjbnKadt7m4Zp&r8{z2UbModah59|g8$m$dy=^BBCA-q+0pa! z@<;ccETCBPxJ$aycSl)nbXv6_8(M(9%^k{f$o%~A38p4+846w-A}@cUHR?)HXtT=B zuqbiPOY1Lwo|1GVE^2CZ|NFx(nZM_hbQ}e{`douQ#g3UyR`>EeI}Yuk2ueHhCjx%+ z6dfA4d??VDYjS?}N?*|;?(yn)7ksYqo%O|QlunmIZ4Iw~b51@}5H7Mh&@xvw&-`bZ9_?$hteUkBN0Ls%t-?$vY;?6u`v|B|xz za`SZX1qtJrE5T5ym)M!^eL2g~?Gw!No;}o5rt=K?74)?=oj%Zin>Oj&@6K;NON+%N zHH+QqOH-Xzkq2#$X0x_wJ=YcRUTDaDTyFa;2b)*eLqE$sF#T%a^nC7G!Bs)g?~8$- zniBH84*K*>9MBbBnm0KYxUk&%aiZ;(QRrq}@uQj+uZzJ3f!Xd&JJz*2`j*Q^Ey;4T zLY{U54hz3d6mXoLZ5N+ei%H4N>JXBVG5!UO)|Him90nzS&#zrLS8QAUIO*ug_?eZZ z)nBWVYm3KjB+Z+|?Yzy_)n5<1w)XX~xXt6cQ2Qe?a{qVo8mU3$wt%ka@h3`;HqvCy zpU7*xoVTh=X*{3&y2$mR+p&e`taWT2i=~9%Z{n+#Q%bsb(kioW35#!87|>rE`>9x5usXa38klPfn5z?2{H zGfzzqCjq%@1Q|xA=UV0WhWg|T%nXg$CO;SKwhbRPIcK`?c-W%Kg~@A9Hg?_TiWa%b zj-99Q$llo8J#XIkEGRG)yDQt~Eia^O+_*eKlOr?`f1=4&=&dsET2(>&a(7i!N`^R{e@-+ zANM>SwJounT+s2e_IYsa*%HUG>r-O%h1nb4ce1Xok-Tb4*FWYzvZ=g$=n_Mgx~-{1 zk>Yu=#Zi0VN8i?Dw<>>D7zW|cg=9sVq1j;{ zDsh{-&{LfZ`PXgDPM$ES%(^(c(fwf{=GYj;ER! zU(k<3Ro z*?u>&>cqy>)YjCpmEwHUqUSenqWMME(D_HHhp%di!ZhoKViAr=7arr=z=r~O7PyKQ(`7=Oflj-0xll?nfpGInphaa(tdK|v6!?rK_o%oxD^WH)$hPm7G%>T z&ZV3qdymL);o;SBQBg;lmUZ!Z=ii^yJy%zqpL&0xe-^aZa?g2NsmA>o2fjaDJm$%B zcN4k?0)(2DtqT{iuX#wyn^5qpH_K}!EXT^3dK>+-I>82hlD(13PA<{ zqrSu(B}ZYt#L)|dk)Z-3RQ0e9pBt~eqErkreBJi)W%&CC7U~)|n0s6&-#@f6t#nvo zGn*@IWe`YIC8L6utGX&HF6d@MoOwiR84)MarEByBf4n_<|1q~~C#c+1MuX5@k*mj= zr$a9WaT8!Lo!7iB&z@T-=*Uh>KL<}TNNk1=jwEyM&Qea}v#V56;;Eu^0MG33w|j2` zskl{w#Ilwz4K_=R_lOwf5?<`71YfKt(z|BhS%A+Cg{kIdRhLfd}W_CHtfG z)9YjR?#t$-uW-75<;j(jpocE_2;q!2$BY+0K3(WK!x&Z}(j+0B&WCH_H+`bPrYQcX zGkWAjIK^J06z@CFrr|)v-CXgcuR-5uAUxk+qb9y^%O*=ktJJ3881(Jo`=nnfHQyA# zkl%Xn$;@!Up%)s9{deg+7drfU1nU9(^r(CqxOq5&`2OjRt2kk}e(k8J?RuYYxz~<8 zlhb;`Ik_uUDvz8`seJlAO4z;SEb!cyG`P1cGsq%{tXi@3s?7p4PBa)uiqXG+y8M)H z#oL>&h?dcxuTP)-Qu?t`xW$K@Q- zEh*nyI|)mdbX71O7H4#op6tA(S-8~uTntYNxOFx7`+2U}@uq=DsW|oWBbPo~$xFo_ z=+z?fJ<>}2Weifk&yflD_^VvV`>n>9DXO^q$DIcSIUDv^)w->5bXrwiKQC^rJvXJ` zw>0V-jS&|o)J!2#?qEG4>1e}kd&K2rFpq-v>ur>vx2^^SqBynq;Q8_%vbS86fvwfo zjJyFG<<5IgfH2WF*_{FPXT6|?RX&WvJRWBlk%d$(haZho@yF4_EdZ%Z&1_Z@k*k(k zxkVz*U7rz*yr8-J!N_rukvmf1crF{o+K(J7c-!He%k?uZGu#9DlN1|yP$6ktecNyO zYx4POXyL#}L4e}v&M(3>`^Z^&CCn?D@6D4_?nO6|Toa=A9D-8oJQ)6T|N7AjU7V#> zPs&R79e53!>#+2+GTl0U+?G)ZBayX6cle^Y()vA`0MH+w8Uqqs4u2Pp9D1fNesCbi z&xC@Q$v*V1%;wGIrB!YA{j&oao2gO{)8*cqKQea>C=yX4omo&DGz^ z?lx3(na%bYP2Z2C-V)e|kl_4)pY8R3RU)KIi+Ol$XULM+YLKRTJ|TuFxK5_NVk+tQ z{ZEb8v~o_P@UzI?UeN&iWp5PVgh)6PWMre0FLmJs{f(kf1^f2`;rrnis zG75v5gH4_(@!2?i@JWM)(s1#P@?As-Ce-A&$=P>6CeKB+l z@(w?9TceC;blSA<1_!$>K0s*^teN4sSoxYSdsS_W4OL$_ywR2_oyQoYaCNF9&CAe9 zfdUYYoclN-oE{i?Ndj>%^;~|4ZSmBFII-L$liqyoNK_?EuJ^z^Hc%zQlV&umD_ps} z4ocnFyRx38N&gjdt(o-fv(FF6{`toB^KFM1X}#1>XuP6>^M-o=s-8}z{z36U6BuKvK5iD-NS&8LcXq6ev`s@;#ncB#+Eh91l_89bkB$1*$*R$9m*G{^gPGQ4^K=cewa8nI6qD9 zGFTWTAJsj1(c22GCbk7R1UK7oHYCdqP*`fyY=r+J`)mP1~t=$ zfKC$8iz+84$F%CKw*Sudv*L?7tJCHOHTL}3TQyaAT)P;*7QrXt{2gbzKPnyW!aX%r z6eh2Iq=6rsyA;0bFii2gL7uoq$m&EJQ)l6tf)NZ=DTW!KX|Ii(EGRf}>FUxv@@}f{ z5}!PIg64f+Vxnbs`Q%-#1a2D4I1~jY+ZPyQS`yKRPLZ0NX_uTcH$20hxgqM0AiVc1nDh|&_!Ivw( zw0K=Epw}(=YA%iX5z9Sz0vCr#1aZfDHKl#PRpOc?lQv!9Uc>w4-wsT2d?ddduI?5r z2FT@lL!c+Uzm^YmUs2>@@PnsQl~Y{ny)Bo5d#5I4+kVIBq-Aj)vK|w*xXf!V_r}f;*n)PHSLLVRs}F%L;@;Io=UTU-CQN!$s?N<9-VS~!kkBuZ$G|b({H$`4jK;#Cff3i|#{rO} z_t7MR=@aYb{jcV(2fP>bGK4V)n1(HOmD~J!$(epbkN#swXr9kgLE0V zkKKcdkXT-RC1a3F*+1!s>!2$1EsE~*E8h19BxQs&^5v#a5K-7%_fzr||5z1XG)tNS zZAAKMfi_GfMFoP4Rl`sZY@S)7kUmX?L-h6UDmSonXc)*S2Y_^eYEb*;8T8RhAM(G> z-9C97BSCD?T**LsEfR=xQ^G>@)15ey!>b!@8iR@lKsmpo`%w5M0Wum4N>4A+S;}9N zD=;``9b5Zz)cvc9YXh25;5I(xXxUHQHv^m8LRzG)CHE+6nKaWVlS=%W^YbM#TDkzuMt#M% zLfW@(7F}?07dyjo63bb-Zefv_vg0$?ud;OJ_jAv0mr+3PVPA+|4fbdkS$jTo{mGSR4Rh_|+PM}9P1VKx zkrij8eR98U&{^3^u7{s_>|f0m(q=a z=W10l@$Z>_ckoGmz!&;6bwM%o+{yw!?0@iHE!=0^x;SRK@uppKqk3uU0>glA=zDx!4xR+v5lx_`hNv!UrMUcKBz^Uvzdhv#gkGmeVu;A(ZO)TOiC#GCBE)F z*E&aLk+tMwaXO63A=?|?Wp@vmzghl1=AZcb#mA36Kee%Urr-9yOOHr3*)i|92K@-` z8ygVMTx%-MXEbkla6h~JV{QzZLm~<9ekC~~pRYxqE&j&3m%|TP75W3)CZD@Z8&14< zH5s@?e?T^wp7S`S`jFZ2X#TTX(Ql8CJ z)J;P!5@SlI{leH=E4K5D$#EVl1a!C|LZaB~V*kJ$2X119TaFAIT{IOVj=(AHK-HP4 zlyZYv1b^}UgUVv|#Lr0+6{3wL zVdB2cSUF46$PMz}ls~A48F)@i6N@*_(BKIJYnd3?J)sedc3R5bD#J~|P3KLt`J=>H zc|3e=z43%9W%78r7dVn`ibTm?lNu+Zu`u(;4?d9Vo3$CpKpHYLjzE7%TqsDQZ7j9= z{9$I*Hm?dVZbV9IgsRwfo*4wtb0!lKpF~K>B-SgPYfrV2mw10?xXpns>f*_(&&?m` zhN!dec#RU>YU_7+lem?G@Fo1GyZ6oZ(OzioZcomqJr~0igJDV=RQtKQ)78V$wste~ z*qYFiJNvoLjA&p11-%Ev&g2%?US|4mx?|CT=sPiCF=prg&44JEeKNukbf-l`xPa2J z_Of8rv2mq_ z-f$W)CHY=|iOK6zW}MT8AX8_hOC&@Q%Ndf#^p*zWMSogRivSm0DZ5RUU^5-Dms^nn z(LKMWuV=U^kNP(+^M2mn6Cdpny*R>?$nZq>c3m`u2J zl-o9)x^CVW=m#okY00#`;>#w2E4B}}?1Rsqi-JY3RI{?vzyn_bFV4woduAjIL72SO za{DPH^mrTb1kDx)5_J8c6(V>OHqD`-H21q=qj0eGc(DIPKwX$X<02LB8y4|rptm&A zfI!r`wqj=IXp3206Q_wBZ8J9wVK|jj`2n+En{t5|Ay%S=8@YXdUaeG1>ax`b zah#ZQGK0W7&3$^7KDc$$7IpZJWsMVdm_avL^xOm5_Dh72mn>I>&(nvOWUsz*V8trl z2uUbcXT@r2YKrz18yDO_V=cc7<@Snr&xOJ2yes#yQ^D$D&~WG)tQ>t1Odux1A1ZP9 zJz4??HCD#ivY%o_pBIZKWDUU{iv+C9RT@-5gI$CQE8uR58>zc`LEuVBCd8I{%A2-J z<$~u1rEB>_w3{x1H^4V_~ZsKb!9FO%$_`!K%8xD@`sP&XToya#?_$ z=VA70q$8_^AWZLWV&9O!gPCNraMELfZIwX5xv{+oQG<;fBnNT+ZikVgdV{Q2#=1uo zRjU{pL5tBJdwoUhWTs*=+|gNxs{qyH{Gr00vaY^9-NV1(o0~0+Zp?rC*04u&e*9H5 zj~#ATie(XI{O)PCtCat^d5cVM*>Fz-%#t`UZ_=BXYJlIe`%Q&l{$U1%O%Gq++9}^R zNg{lts7j4jZ;u^_JcX@r|Drh-CagktTj}AchiPc+`symMPfe2-qF_%>$5n%=m4gnG zW(Q=-G+T#PujL!enf#C2m^-(bLFUS+K)7f>wU&V)*mlz8;ni*bpq`+V> z_>fhQ@&mDFe~mux&(mTE_Xv#rB!M#-`yBudn@;{ry~@s?O0}+g_U%95`BHWV%aA9zqjq6*S&6 zq6NBje~4hxhG++S_WZM4c`&Up%VTk*bOq5Suun!)_1#k&?D-T*;+wra7Tg+XlN za-V~Yg;Js2Sb)w(M@B@TqYjSuSJ@&i{d*F~0tRDMN&B^l>;i&rx``>n zAQ?lJ_Z$)Pgn$Anio^tc;e+-d^k|h4VLT& z|CQ6M5HxNi&$Wt7K z-(7kJ0~sBV`^Ro6w~sso zNICQPwGW(~x%+n)L`$B`mpgGC0%JEca$S>&4@vy(>oiu@JheO!8M_M3sKmAcP)M_^ zsubix0*KOZ`28}Rn!%+@3he5Q4f{}vJqYcvjq1o&vF-a_;oMhrC0quc3ej2WPY;6w5CcLPc)rBlyEr)yyr5O!s-WnAQ zW*_x;TiC!Z^c-=H^ZNGvhlP!Y2Epn4vlyOw-cdKI7cX8!Y+$u~YGLTi)aoL%1>9Ax z;3Ok~z-?4SLOyy!Xw2QAreKvvPCa;$ z?b)L2Vu--$e6sO@fRW(T)b#X)a&=NI7MyX9Bg3uwj5vTeJ$r&2Nz9pvWKC4V-29?|D@#M40gA{!z)`aK6dJ?e z4~KI@_tYpA-Nz%0e*%@p7gXWWJqvukX@#AdG$HHTO{{uEeTZ_x0Y@;0ZU)E|<-B9T zmNuj?eZ%6PC?gq)KO+Ry$@y*{Cx)quNjxOv5e*des}mV^R=+UhovB{L?l4Q*3_Edg zR-B}`@~HtsL10a7(c7FWfk^<;FHZ@m)Ed|va8RwG0v+d^E~(vU_OnZ= z9uTbOgUNU1ZI09kSb3nKiosP?Cb|(i4!(Od7s#JL<(J9F;X8vu;Au0xpT`RjUGjnO zz@r2up>}~TpI{UOSZAMOjBGBrsx>##Gz4u~(liCm_6l@9hl^uI)081`MzK>-xaJB4 z2n`F41cBSqy_P`wkA<~bY3akQ77Ushx@nbjpc7zhfbIEOXf;Dw!8(Qxi6}| zm#PLZS+T~RbN_u4yt#wKi3MhJFQp@dUg2tw0!m(-Jb)Fa$l>YDKL`d~+4om-j`|mQ z4xd;yDpgf|)oEg@Ll^@-0(jaq29N(f%!LA+ip;*e^a~0)6J|DN$Q9s*r!(1IIL6Am zAWQ8SohMV{+;`oaNSvspT7C5%(iVFw9bcQ?ZoMm8M;_+IVN=j;!;Zb z96U-azSm5T8r%~K0cg74FQJ?0#rP<7i6@swEH}@-xuaiAPsE|T3dU+MM{f4_=it-({h(1C)o%k9#v$)7`EF4_Z;-?RhKydN&HMX@9%ut=wwS6MaO7RN zt*zPheJGM9gaZdrWAgvp2;^Rb)FE4cIoGNCGV=IGbwfi#LQF7jd$Hr-kd?p~J<^r< z>VqM9`3PqQxo}Bi>l+&;8!O+Fk9i|1g!eerz6hd-0MA{hQ$6Wj)y|`5)}D3w3m$Ywka~; z8Vg!|#b(R?Fq|@8!w!gOd<59jGLBk68*XZzo<(mU2a7QV{d;aGFYPlRs^U`*ElXw2 zsx~u&xTWD~y#7{g9f-=l+DowVM%;mVOZt6#sWPsIJVRmDLchV9qPZx(+`{qkMe~qg zw9*}!^BuyRdxj1)V|t;LpgPD;5$r?ESeBAcJm6SP`x$mV*fp+Qhy2vZ%OayhCsQ$2 z6V9+>oV}wr&s!h6zCrn$`UgW%qY93vQa$;HrVqFPZiJdIwo6iSMf??)M-;KD#{yR? zrZ(j{uYg|>K_(5F8$!D?zEt2R3Q!RdqL6*F!u}=kCA*Tp|Mq4b8q0mm*YK_mv zdl6~+(DHD>J|$i>8?3k`lJ)DKkq4l0Rb=Im zYK14ZUI}}x5zEfbe&V*EzVZ7U^GXMjL3n(Ud4%86jM4bpjza#jpJQY8I%fSqYq+UF z8cfll>Zyv3%V4AyFpjx6&H)S=&!rxgG6riNJu+my;zqdRJe=!`*adsfVm%)bF(AMu z={T&=6`Y~q2?}g3K#W07sXbez38$)eN9^u-E@nl+P}pP>BKpd3ZBCjhgmok7a`PJO z?sEjjc0ltR@DWz$zqBVNiI~u0#xG=FD@?MdrW)4)F|K7)zxEdt0TK}o)gbpI(BlBLn-l;hNCNurG%6mL-ZHUy*ph0(xL1$5iYI{VV<8Uqcq6N�jmCRTET_bXN zKYt7F2@T0(tbpW(Jw*L9xf~cSLKgwEKJ&oaAhoebi;3yUi^6yUVCif{{ z05^hwD|@n#T3JBjI`Y&EwR;hVlI!d1$Aip)r372;=}G~<2&)89I(uw&b@gUbQ&V4r zXrtG+J7_G}RU?ZE_;wnM1F!?hGIJrSfv^sCTMv1LzjQ2bBdFF`Nrj9HzY!g~A@G7s ztr<%*2q`pg&0}Z2eLG~09w<9=vFc#1I>pJ+dkkEPgw3C!8S5+Gbs;VEb~VEMgpko_ z5stQ1=`u<%Jsu7DMusTbUkV}J{%L-xfE8ElkywR#&^6H>7igY!4){TqDa;eT*lr3CL4rVxq-8N~ z2NH({F&OxnG3&qOa}~Tc>Er!w@#={ChP^`*PDLKYXUBjiTyMdqDJv^u@v)P%{u?Xe zmxih$;Y&2A={b4n^*e2?d3VVZz0s{NO*1cL+s@s28P%@s-K%b+MFzQmquV!C_UC3E znPeR1sa0_cZuJS*{lxR*?$^YCs*Ij}71R{G=nE7nlD4cAB&4wlpeMQRa?_GqXx{a; zHD_+EoU7>33ulmA=ri@+C4!{xrolald=c`x<0pi~@C{d|<7Ent`7NL@xGRaO&DMzF zW7GcQr2GRS>_ov0(ie#Ir$>-gl$8vyWSn7$ishoXlevEZc+@1U`rg}`EF2s<3eQ)~V~Q=5x+-`U-L^N^}Kr%p&}YMu;0lSH1Cs740;sI{d+7f*>m|rJB{P3j;Doc`y4)sZ zhjv3zlnD|hDq1oXy>gMF65yIev~Tg#R~pc8orT8IGmtNV>GbvWMF*?hf>}usM`9L% zb@_jn&^(EPncKP;-s64?gsa;?^)XYB0JAxPEG=RNp9MTYgCfhxZcjcWWQ~TNv!%9k zzcxUe(Kr&wq3Bf+tH%$}*h1~wnj$ahQK`2PbCn+@V~NB*WKO&U=+B>64Mf69j4FMesjffD_j2K zmi)g<;Ps{eD8ooWg6XFRA6hpBT^kFUaY z?BTr80M~;VgUQjGgr1p9I09kR@TT2s3Pd-Nnaa`p;{_nH^UDaVau72Ey&{q%$>p~! z<&Cber6}Vk3^(}6dd>D~Sj0j@{yxCfNe!|;BOsR#!T>~d%1V@xZR$+NV$1OV`@jg$ z%Aq<{ZX-@bd>zl#=Zc^`oz-(ZJUocZ7*HVLAPM5m#^)VVrjFY}43+Zpi`^=xlaXi+43Ws zkPkwDeK+g84=S<7?l}eUy1aHGu)mf!#6Il#mxux1LJGhw!SSr&k6*||T@@e!N|&IK zzk!yPmrq?^n&o3ZvKvbZ8}GcW&eG&NALcRL#r5-u^%%k1aBir&C{?r?(0~MRgKt?1 z;}!l!RDTvyD;#Q4WI!`Nc7x?mEaR``Wfls;CtMwO=)t3xQEUYIlMk}NPvo;kZ*6Wa z#bllj2I#uSazV-1UcuGxr~Z@b|V`!Xx+RhRGBmjGl=YpCCDfw^lPIg2;KQ z7Qo5U+>mVKXCj8L*zt9|4!6>j(A!e8Ly%%s?j#Sh{ybOaERa%wH6?t^-=F%=d(e{U zN3VePhs$v>K7xw@NJ??MR5cUs)cyU(j~}hfeu&+LYt{!1 zo?)5`#I5u*70nO_!aNPG>K(z@mA|!u1A@E+F%y~B;g1H@NvSB*APcz`8YcrS9(G*@ zs(K_ZxY#JK_2tW#W#)<>?I4)3pF{2*l*%i7WR~Xn%TWGvqHKcAlE);h9_wvF|0Vs~ z5eW$iXx=ImrW-4VsUu*U_?0|o3>{84r$)b&0&n2>`O3ZtHgQV=z{zTt6jG*0^u{g>vzdDe@JILz$xL@T0 z&xsQ!h@)NcNst!qxiaez^#AX#gz6}y!QLR-4XDXNSy7dn%Ea616J`2#;7sBpd_l@s zh>#|B`(`D9$nMBL~ z|9N@s>?!Jfeijz!ru4#?*VvW*Tn)Oh5ObJPfaKJ9p(;=buY9FVVD8p_Jj?3`UuM>z zOrsC!t(&F^s>hi`#Oyj-2r*wHcxh?RUz)EOen;{etwFBfhk{om)#gHYD;J;5^8mcj z)25?tZ#RD(26f$XvKjNC`a;6$3r$VwbM4II!A&<+n~hM%g3+4xKHQ4!Zf6`aY@mxt zX{?Qe^l3fDzpPO6CKV*o7uE8>BxajxN{9-Ab3B&vPxz}$V-#>a2Cd6oZa|y}TQ z!k>+jaxpC4`N z#F^o5F+-st4?*UO^|RFXK^1oftAw?%s2K3l(9Yx0p*&M;1#r=p4ZhMqx2Hi3Fdo3L zX*@VxbD65Xt&UGCCeexMpH8Tp*l5+Fti&WdC|EFgd}}TDb*S?Hb%~|$e?ndEFlt!S z1--VmMnmoZ88@--pOX*pxqbg|7|6RG?j;k@%#iB^?Yux^dISSyvh=OIsdVZHjJtqH zO$ub~qbvTc;`pCR4*7<7d0qpbW-riVbNeKv1yK}705XifuB4`>?kh1VBA8Dl43Lbj zQ9!^h7d$W{2+J(R#Qjk5{Y5rb9^3nz{2~U$KvO_x#lUoc>0{gE!YS_5jD)&Z?p&d% z))Rr8t7pacL&^xGa$0o)KAALd zT8bCr?m|$KMm`Tc4}5x(a)9?m2ZAjPzhR|=#`?tVlf{0EQ#k%3_4|24fffcD^@$)#n%^m3VAglKW4ty>Bp#yt`zD1nKMVLBvFZk%H%YSA zxd3`0(ULnrsbuEy{Id6(Ml0?V_V)JR$O36$rs3n`17YP6pU)-jnTHp@B&gg7&<+j) zW8L3)EdWAU;UWK{^!|g5fEGc=dg%U8WV#MV_MSs_5BWt#H>jQfwj7+*=&-P`h{EBl*^U#Cb@_87BFl&04g+J2RzY3BU)^k0?-%M^EjUlrXRZ zP&B@#4%+<&Rn#0kTq}w_w8gNjzIvZ4me2ejx%hu+iDUYmP>KctMB*GDp5yFLf1MQ) z#PY1%<5Mt;C(h{HkE;s{Sw_jqmLg*k(u{|z>B`o(I%FO$Qt zaXypjYlGY2gSII2D5W5n9;#N4_!*2hjJ@i`>1 z4g5n8OQvP(ii_uoKXg*>uGnFnya`k)r;0e9LO_^JrGrgJM# zQ7`!4UI6@%xnp(O-|bRY%n}6YlB69qZ>Jq|AE)y3^TPy>LDlgTFE1zN$yWv39S9OX zgX6?$C{T!UVI@t&SSQ@Q60-PmA!&9mHWz>Je-Q)TQCcc<;T8y1;RB(5VJZngt|%1| zkbTdwcs%;8kQ7)(4wayWpv1u|17(?L>{FVff#EL=PB;H%>RwAw**5IF1QpH^0osvN z2b7SI0G9;q^*;$wyEec835pBZ+MUkghP5}Wrz)Y$<3p-xFaX3fh=t%gS<$Cyz5xSP z1c?u#T6ZLE4K^cDw-;%5_*`4 zW=KSznYHv^Un&7NSvS)|2e?o)HVE=9pkjyDZUC_h-zJ}r{tuDthDTD&=t3zXcg! z-0KD1mcN=|RVUCLKv2BpJhb!h^^YjnwLij&`vqg)u&Toq_TgXM#!1nUWJo<}smFzG z;JRVWkS4B(m!NWhXjm=0@sIqtH)Wha5b`;tn2Vr6>B%b_nA*rSKu; zhv;9y<-BGVCu&xT!Z?7dWIEZERRAg{MR`b_3sYXy0Jc$EPVvpEr zDc#7>zo1qh;y^jyVPuMiND3*Ha1(N$q^RL&I@K_jI3FNjZ)g zr+Ay=i7Ajdqkx1-V_yhbc@vVDARmwM4=cOtjD>u`*d2Wgq-1n9kXz0MqkSh;SjHj! z4i~B#h~iZAJWeLLP9c7N_(hqY^S00LvU+iY%J04x>K_>3Rsw!CtIJJ zf+hD_SD)9ngdrl1`?&LJXvrbta%1y=BF^$0`wah$5HZ5UatI$zg6^;HIYxo7b3dXW z=zdsn6(9_X+yP`W4$v-|7^iAzh}O@L@406Qq>~e1+`W@!hHl`sWAXcFXp9Q2priqd z??^tBVtkzo?Iwpj^oJYv4gb?HqpJwo*gQt|F^J5P-m-ZHm>V(o=_N`MBZl;Ys73dqu-T zlHp!D3A~-zOJyC496Cs@eq&=JLJa9-DjwF)I(FAQ*6AU1c{x~Wm_q@vf7MyWu0UGa z`r)yt)jX|)Lf~gn}MUYl9o>5g3V-1xzzlX65nS;FQ9G}=j6^aT%GZO5h_@AGp@ z%ctPx87#vOqZ<<2V4d#)~GNC`0o%L})}o zJRTBQ!Bundv;&OJHm-hg`}-wNDf!V~RNTh{g{g?(Q=RIZ{8Jbv=IQmW=0Z0&CjRbD zR+0&nnMa~}UcOY9c)?1=nFfs(a@9}n4c|@C{6})#hp#z@;4?RfEoBxn zC7#-=rR0BY?ilB8Myex~yx#nDwwW>g)znd7!$kZpj3~g6L{xVJAa_jG(1*pdTF2=z z9S$Twi@l&i9w<{Su~Ab1vsMHQ5m8n?b_IOT*1kSjdhvUud1#_Hp+C+Xg&D#avA(u& zSMvVK597&e*iT$-670{nQ4e>IWmmc0W!rdSliTd$`G>_p`XY0wU8+*=zXM0(^o>W! zY3u#x*;jAaCImC6y%widmKHT=S#!NRKN`ln4mn;L=-<{aDPqLkNy5vrX$>CNs!NeF z`0u(-3src1JB3v}J9hh&hmX(cVFrYaH-l6X2nmTDpD^qY)9gP9(1=BX2&7Dpq<6^{ zL}_U;x!X{fq_Po4S#q3ZPDwG8;ipY+-!nDlHNoMYqOf^TK7S}t_$kjDnm_c$sQqL* z$$7$!5X&TC`dBa&@|L&;8ub^Sl2SUDv7Z>%R2; ze7>L0`~6z)uNA4TrZKiuc72UQ{HS1=yrd`jLGGsc8wQ@%h^4}vOL|EKnqx;)7Ql-EpJdAm{PBxxz&cy{hfRYZs z5O9D$3Oyqt`yESa21k*pm$ra@bOF=$vX(a%cNo@^0;^5u#BUX<);tf9AqRyOjZ=cf&Ycs}CG_`s>NS!YIRbOXdC!;uJOT6H-ii|6U zDroI}#_i>HGW0Qf5=r0JkKV<|w?+*e=7U=OUuO2!N-`E>FZI~$?6l)Zk|I8F21li- zu-U6^(I441HVjI>I^^DLEf)UtV7M`D{Y9O(9&z)}`Rer+ZGlv9SCuWXwz$=q-T3Ru z3zr>*@r{5ys6Fvo0j{Un>}HOxG9eCI7mP@UN>6nZtcuhqjF(kr>Q1H%3=-i4TFEQF zX)*6sarl$1h3tgKp!>50QUM27{?(BD`@?-XD9Q%x74ulhN6B;O8Usd7Om>wt^a$Rs zAgM4raBGb6Ka+<#)}j=%KY{)}0I zz~eY=DX04knHn$DMR}2J$Yn}peers4T1mHytjIUa%2ArheD^wTn(k>E<_R|L@-Y`D zqRAH0KUF<^_2P-CaN!k4&Adw8f|;zebB9}{)k(Ye_Kza_qrYKPk1gW{$r&-aap(Tl zZ{9xtZ^HCN8V9W^+-B$b?&Z5nP!PUMtjVJyM&K-9uPV)_L#cJLJ2L-N2vzIiHN|niCK|8f z=oWXL!vIEca}63faxB^xCbYiDxtXU9XHX_s#C5>x892&y#51OLkGxG3MXyuya5##b zKk#ObjdeAStjl8qRTpHCbl=*b0BT1-UP_IPfk6-~JGHoZE`e0Q^BDtQ1? zs+iLycumt$GpsK!DPqohC!qb&>PS0A_hn-R?Vxg_iUuI{c{Xsu^RJ_?M8vngUa9tQ1?Zy$ z!JR-Z(l0=a-fK(}E&Ss^kp07LQ^MdFz;GBNbngb(x8B|0IW&JUcnmQHHX!rQDP3PD z;Q9R~heF_gAM6Rd8@NVS&mq0AgyME_0Zhnd&)WUB@YzW z+CpOl=cGa-T$Dx$GR9np5|_n`i+zPb5EAh)lfM1BEKwM}WP(*oQnp~BWfL{XTLPl$ z!aXTmT}&M@gI;PEjHbp)IZCaGIr#AS6(3A~FW5Hy`QTf`W&zRuG(u%9^M;JJ@Up+6%T4F2H2EiC#&+~Qu;`TbYCDrXQG0OkRY5<`l zfvLj6d2(Ui zE%yeagv8(!S&6AKqgK`49G7{}7qIXou&Si)(y5$p@FMysZUp!SxYKmc3cz4=s?OT4 zUW&{8OqAhRagfNx2)C5$$1tVoGKk|FKG|PsXPVp9@wQ|$k65^S&WLoy4e|sq> zK<%ik5-IdyMCoGYs)OBd{)5aB~YQ-cf8D;kr(Ap^>i&&tZEZ!@&^mVF3&FPD z(o^)-jNC%VTh;zUvdi-t$0k)dMOPiGHFOysda)qCqa#r1KggZ`r#O&^00IV^26x%+>1(sO$}xaEOcwWv$;KJI=f^cO3CC#rw-Qn>(EdL^%*z;@?XQtbdZUR z+9Z-FXA88l0&#YXs0wz6KkMs`$f4)^FX!ib+TJXRP0tIO*xLLmVK|lb-M@~sv&*e; z(kzPmhTyLyT;=-hJ;$T@mIn@!H#kg#`#&B0cx7xc?X>&j!7ZD~r$Q>JZjQT})uU!{ z&HyxN{eJd`P36yBO5h9Xv<_(V15fZnOh93SY#=ew_IKe2{iA{M*JarS*>Kx%cZdq48&YrQ%7 z=#?D!C+ozzL>j#)d$jll$IwHmKriJgP01FPb$Oup-R-k0pjB&4&!fO*IL~@-`soYB&i&{SRDM^E>ct8)4QtD%<~9W zmt!L1eh1w-O&T(NYLww!{9}O0VVyF2qV)CDvoa6-70Kj(HFbtI#Lb-fc5cE5bFE?Vb_WtfqFo9$YFvMS85rT73B=ZUInSl zn7p~?nNw%Zz5-764;u1+ORZ{BoJcV!)Q|B3TQ!Udi#9-e9=Ykwu}B8gC-S9w@#EA4 zMlsbXRYz&r+B0$YdeY<^1&Cex=6mzgL$!lF`$43d2J9^uL9cu;F8^hP!nkEDw9NeW z<-@n6Zz)L@bz;S7ec~^Q%L0CE++a}$X(EW#wU+Uy`bB^Hz4fCm#q$XUye|}^o*x_r zZD)_Hq|SKW$rw!P`+UAJCLLM1obyx8PK0s%@-<29(RS;FQjWVDdY*UtKSqu+)39cE z+A&S-(j^$UByF@DhE=dIrm3SiRps+fN7!g(a36i6-afrVGDx#Sr_?)r!YwunlO0lj zTwyUkiZA?dm>VC&0YQ`9Y5TCemcj?zEv++*C%Jr?>#XhBX3sBd%pRYko7a7QSI^#)zs+s$r_#L0De@UDVt zS+q(aXgS8ZwdDz&718WC{!~k{DkzzQPOOZ-wC_^Ghty_e03InntX7R%w_rjme|DWt z2oCgt3t0AQle)riZI^6~zIsIGZ}isk>)j^rbTTA!dwSwvIx{`LT%}@A%Z5Q8vBSmI zI?h7dn28X5*-CJ+OXGnrigHZOMY_UeF=6VghlfYN2^^Tc z<9;)moR~*`bgX_VvCPvr%BpF75QtxHYe*fPHPK=O7j%c3VT zH~3$gxgV{`mpLXU-!%|UzjtNBfqk)m`-On*Nb5T-F^-=F9!bT^Oz5_`>+>;UmKo#? zZce*OZVlpXL&=VoP3rO_)NxyxF3nfe>4%TL6WIPrb5ltLBYJvvI-6z=6-DCA@5B#i zw?WvLf_%+PPF|1KTBdWuPX4Fhw_;wId$ZA+*MdlpP}NN9VrlLbNhqN91v-TtuLg88 zcU^9kI$Ex3n}c9h2-j9|E8hlgQV>i-{=8t*X2oE$wKD}orW#Mp#(K$Ti%{hJ2r{JX7=-Pm~j-G_Cct$;ivXM1G4imdy#V2cv#;l-HBi z7qG4dj+mI}G3;Bu&(UNTDR)^$caE6uejTZ(o{d4d5y$o}{~8h(toMF7NOEp=o#{NU z<`qMHb|io2&dx}b%+dGRE$_z@aSa;8O@r@cn7T{utiIB2ID1&gl$KOMl9kEb5s`0Y zIgO#~F`-_GEK%C#$En=+kkgHu42w7)!)qh54zxthe2si=M19+rHFW93lmP|gZl0^_ z?fc?>4K!{s08Lsxs)kCHoD>VX2LVF@0_kdPd|6r90FtrK4+eCNGt}a>3CTq%PDdq3 zgR#Q~mLTUFxPuTQO*pgg|9pqzID|SwzA)i4C#f0FoH;`gN@4Nlev}+32}PpNLxqN2 z)d#rVG>g$5uyI44U@(a6037J=mVY90BtyZ`3-4f32Wlsvy5x1i!#|z+F|_sh_uTW` z?7?jrI2QM!z#YPGxoo2ekjXX4$~}vHdqY%7QY9y7#P3CCc-0a4kagp8Up&R|XRn_! z^grHtxVKq+dX3zcH}rfo%eA*z-IWgK<8r+}jNfv3m^C?nBlJi=<|WF_F@f$%ing-p z#k<|tZdMJpI5$9<&yD$M#Qs5@n>keW<7u*4F)d6SF3A8)VuwDhzOLS+9x<|1QZnE!CL!2Yu`uPTcH|75mncvrhI*Q3z$*ENr$Y8D35m|8Ub@0Lz|td` zf^z6CqydQ64rQz^`x|8~4M0KO|9Qdx*Bu4mKd>IO^8s2vWUF6_qf)1$#jgoP4|N7ghJfOdK#_rcMr0_aw&UO`%>XK4BlYA;TSYxCy|_Z^ArrK{ zx>md!jY_%`CX6D!{&Eva@DG1z$Hdy=rP%6IAfpB~_?klw)hJjMUVD32yZ|OCLPG?_ zSsZM54X6C|MCow;!tgRj+TxSG5v5FylOU`hKm2V5WsWf;4N|f7H?>+0*HPOk^G0>#)32~+k4pcR0(cnx~y4z+Mw}+%+77k z@=l0DVo`iY)>Yu3lQVtmR*#>_J`=OqXjMiBB%lt*|Eq{dogobKCS-vp7b8B1sc2C!_DdVaGO?*_qK0=^$BE(gIXno70EQUYqwF#+g92upBMtOQx|8+Kv3B23H9*?s*mE0|Xja(gaAN)_ zQU7}0xDVB@aXVPEDe!B8tPk=wMA9Vu$}`O~kc_6v3H)y6VetKhN-XaO9A^PXw9IJ4 zQ28wU%19kbV0;M*Emc@hA)yETF~l^y_mTcCYg$R-Y@#m8*-DWp;Z$1;CcwV#R-x57 zutTejC8 zt0OP|diY9js3;KB282|K7!E$$>tVJuTBK6P*2@y3>2J%}y@+^T2WV1eY!6dug@mvx z#st{<&$Ve@qUJ1?;`aOyr7Jp3?5Vt&Byl-XwnK|)e;**rBE9=7* zd#Tr=UnWV??4N2S7(V4nHY!OTx?Szxbgwk_gIS0gw6wRG!@O%x8V{1eZMozk%gD&6 z;?bL-Oeu@KRgBf7t;2_;Re1`9UQuxUhC`;zv{9G@C1e$rIm!+e*+ze50)m_wRReF! zB});g&i<_=`}2cR5|krqN!#ZNxBRM?hRS0ZwD37WuI4--8-PVAQJJL?QsKa!vDE63 z{cWz_uL;SM(bsdWZ_47yiOVQrjfSzIt`@d!K1xKx8N5hq|tS)^jff5L(LSo{g-)M8KM108^~ zR}?#I65=^1OAqd{P_2J{5KiXOume&D9bT-^-7aj?EOSCWi0v{kSns2TmV+%YNMHg8 zAM0GlSNuP#Fo;`)!ELg_Z9kuYN9VtWl`zwbI@UZOA?oJ#tC`k+u7B8PJ@iY45gP7e zLZd2|>!p3c>%aG$T^YYMhi#C$j;36)c<*$t)XSnKBMc67Saq>dz`|6Q=* zd`r0FC*Kq#xf-b*+n_Q@Hj?LOLA^Sgo*iuiO!XxT>(+#4XC5ue$M-#8%k6C6bU$ZZ z_%X^rsRkQ)U`iY+kv>&s~W!AM&7MR)GT!oKZenc?OO8s5@SdzQ@k==MP10omP`RuB&6Gi~aG>oQXPE{SAJ7oL;H?gTL%C z<@0?#@N59GnR-!esUd)fk^!@UeiF9w96yfNSOu&s0#72*wDIdd0`It@WNS`wea@$R z)r?>-dUUd!Ded#pdb2~r!zP=C5X-0IuSDF2!m$iQdAuAf;<}B<`J~I0AgXFIi)RWG zk7o3I5=v>*(spI;*Jmbijt~(oFOzkzN|fw@YyX}8{Z_B^V{XRyY;-7jp>^}c@gsAw z%JRK2EIZnz@vy!@w&vqlJkPxIl)pr%`a*buh*Z*5+8tlyFq>;CbMdxMAEdru#uZ|+%^TK-;O;!}XAG35n-PL1}ruupd z9+0G-oEsfQn>mK4fg1(9h|*d7ng?e6MYhb|D>2OIQ+M=a3u~^pAGc_cAU4^}xG`sc zGA~``r`(jkd0d!Rm7zE9)$@I4LnPy-cVj9Wa{Izn#8}B}sy+tG7%!kz)gs9*=uVoN zhZd9C+iO4PeookI;VC;f?A;76=ME71L=4OSV%j3u3L>vwH}m8bB$Gfk?C-kT_x=9% zkP;Qwc?(YwjVN)P3~@wL5nvF^rK&i8-k2PM@y#^ZD+XU_SrkLN3o}*Wfk{mg#3g4M zk*~UQT|Ga)>wAhIhOnJ3n;l%aDgpWxt`7GsC;scc{iEoikOYMSE5rFnN28C~KlDybcmfR@0 zj@@Bf|JWcPt2(_M$~2aD??%Uq5swdaWMBM{{wm{7!_w~4wKRSCiD|k{R|n-*d&z}` zfSuxTfA)BRGdiZ!j8kXNZI&S))`cp6Nx!s3J042jZjb58ydV3P?%nfZWy`Op_GRe! zS0zT>MkQ~Hh?%2rqg_jpsP8=&mdS{D6*;bs9ZOzFHthi5Cw3n$rsa{UWtTAP6gLGT zazNGQx5S8L7^H~6^Q1#WPQz<7Bk+t2eem3{*ltlf9FO3n8-XAMyr?HT9!Pz>a_nWC zL{37@J@XcO^?#4&6TZwq*3~dOapgY`V!^4)p zKv@rdj|@WtWr@=?>0*Cr^mm?zINZ2nSQ3?k6|*!u8H^>HY`W#E*(Es!aaL?sz(nCOH^YVr~(0MW^EV ze_~4G?G{K4YBDYNdiI@rL@PN-#MzIvCE<;D2I+@>n2qB0tRhKiW}=em%!RSqp7}3o zdi35M+KK{wh4!?T!PfVgTi3tTd4?x8sAE|X=07$ryw#Xqis4#7xRa#}3~=r&mvtlf z3mq{VBQ{4S=k{puO1y%&I}61 zZ!#Uyew6>a07^Jvx|(A#70KG^lMV+E_2s|n3`c-XIXHuaVkb>ga`Fm3Gs&VALeHJ2 z5IsqhU6dFYRPM6c(_*|)tUQS5?qDv;!&Y-T)jRE~%;j2WBg$wG^)oP>pb@b_As?A9QyJP?72Z@fzt3ApK(~SrR781nb+{=&J&_FVq ztrQvkK-URf>ggCfy?(yZ$LB*A!KMp?7M#^%>ILo-B)NjfYCP`rDTVc1!MMK|<0bhhhyy{9?-uqr|3Xdh0}M zqrQi?y+c+cMm>GDnJUpU%VYemNQ|S6)gisSjn%kfdXQ8w7JFro46r+7@gY>nO@vip zBk`s30pIXtZ-ZfG?o?Py=kBOK*;`PDx75~)Bc9vfj|H$J4|}_{+e2$|TKu!d%*=_f7AgrmwP zU2G^XhwRwX7=LlBeP_+b?8{*JPHQeS&4IX-610quHD#4ko;$XuK?pGUvysl|3KmBBQVtJ@smC{N*Q_@=; zlu&*hTFc?a><@D*yiD^b&WU|!9jb`18@zX)ho~tj`uo-Sn&wz`gxjT)P`r>g=c@OT z;lB24?XgA9L@8w3twGHQEf;lW8SLA~stXaoOKC3!f#*Hz_-McNfue@QYJekJ@HlMo zGYU?i6ruwdx7K*t0npC4cqLl1-~t%*!2HA_pZgdg;yg$70dlN8gGJg8RAXlp&aZ}2v-S^CHmzkz%cb9YI?q7Phfj)> zdLIx}9y{<`T@1V|VN;BN-S@3z&-}gcZZ4L_4lB&7i8hyNC(2EZUl41 z(T|P69hXh?8mP^SOZ9H+hH?aKJMymlGT#yHCO7F2{<0m37~g|Bw;S!E7^lhxGPp#2FWJhyOS>@fa7UF{nmAB3S=q{AQzV^M|>a`07C zi1~u}jVu54x&-*$9$W-~B=b5K?~;-h;<^38Sb!_YYttbXzB`5xQAH#8d=wwY!TkL@ zN)WEv=#9nNu`$1D@Dk3nqK}z(Gv3pI5ScdJjAbw#O97FN*H4q2AItNL;kvm*yXo7= zuyO?-waX_ocH3URk`!4`O7B=Q30}>bw*!Dlv_b8uLeZjoRDYu9JMC{dOPn-RWfU-Y zlc4Z2k6nRae`xK5C}!@y@9YT2_A7ICO<>>Nn?q0SbMi4p-P)ePIL36993|?$EAxLr zpErVKgh4eNdRc~0o9rl(S}OziaOGuqo*$=UWtc>GHm z7coM^-%Vb${WQcN8tt&6pvYUlTOc6iJ-eUzLylhJ4pT4X7C{PG1#+X)XPaf1K&y(3 z-pGXzq4SZPABP8iAFu7ZY3i9Ndu4fI59;85o6|Dfv0t`L%@xXqx7Nf}c%{9Q z&ni73#pBxzOmHCPrM_lO@aMdYnIBZ=35?cxP9i+9y_&(K2KH|Qf|AgV6G~wOkbkD_ zC&wcZV-d$4P9DX-M1C%c>Lii71mr2~!Fw6qRrZmV92@R);F}5aC8;1G0Ot(FSq`3_ zy^wZ~`e9EkVs*4}k=xFKP4kimP}=+)(+D#d7Pjbbc$XNI!e(13FWKoYlp zgeq56PMIw&6Ote%rRom}Ef~`rF?a9`)%?=0>*A4)GMJ11yHO=ur^iBEvK<*?+MWYf zzc;fOwglPuB8zMkuCg1!Wby$JNDDN|!IA6{vPJa0D=1oGNz9>VqHNM0b56k#lK>U8 z2oRWwfDKgjpFnhFoEt~^nBF3ARZUt}>t$f@SQ5-h?$d8pk&>KKe+3cfzWSR%3d+&a z_#`4l^()UAMI_YfB9Nh*egN&?N6^QyYoUI5zBcP%Z3B)e%5*~YZtq7%NVJo(<)eLf z>LjHvXq4Lk*V9!7Qgd_L%=G71N+gbO7bJJU?y|Tq{ zCMjDBR!u`{!i3JMAhHfD5%Pk(r}*Qnw9_P)c4yR=R$EP?t*@siv$A1qb>3h&aMIM| za{KVy!ue(6a<24&qav|(FaF0ZMqQr`R2wOx`W`r&Qqd2Uw9c@`43XC&wAgs!q=yww zNZdyWE)XmbF#1fNZ1u+qIDhk6u|w-$Ml=;NnHxMKS7gKUK3ENML?28gK1|9LDF9+f z53dGF&mUS)J}QmkGZWW?dpT^x^ktSfB&^U_qt~62ZV??UdmrOY+FQh?Ub~hM7?F?( z2E+f^_9uKNWZ<};a7fK>#8;4(QqGkVoge?M#(#QXmVFyj?vxhbPD;|GY$p1kPMTB> z8&gCGCS^$9hqm|w_`DhWbc9!bn@OQ}HA9%I>&%64 ztVeFr8~x^2bo(AJj}PYglxwhSg-uMa%>*l!rR6Z`pS+uH=4Awq@L3VmC~2G5wsYT5 zOZ@R0lnWkSd(Ir5*@XDX%I7ff^hhm>o?+ZmHf<5J-=N zX)a9zNON_NY|=keiQ)N6LM1{g2bq`=vzNHQbzOGcAdbXP2YRR@l|h}9mHFe+!wMPb zdWvHHL0$IT1Axt!9!o+u5~_&+9QN7hfg0#u(nzV(^lNK+3W{%O7;08AA}YOMlcP5O zSEo`D&)s>)z7<-xT$Gg?C^dGi$E$%KOKQUv^^w!Ci2Xc=cFkHWg#H9AUAoL{iU$uF zwVPR(QNt|@=u8$w48<|WJTGC)fSzG@Soi(lJv+OdnLf8)XkXbc;k# zYxuINyCVpPw%*7`3lk2X2B%ZK;x6_)KF`@|dLZ<{s92%9p~1Ws!F?l4`C^M*2tfp0 zh%#l~T&3YO%acqO5SSr%AvrEh%Wa(R11HT?VtTCqp$Fi333vJv7v91&fm zy0X53)xeU*zUnmE*&2A*2`=Jhd6ECC-BGf_9rIS#=$fv3;Dh+j3QN%}D^J8Y)^vxO zC1HrF@J_Srg{yZm&Mri~fEX7z(bpzFHstV?T3@p>vMv^pDzF%YVnxkQE~sA9Yj?RR z2C{QP+cZhYGU)!qK{=zw=)Sxz` zCj1u+1CWgL8Vpf0Un`&1cc5Uje`>s1bfw*+0<2@EH(zqKB z^RKDONoPORYa(kuoOo?{sC?OzO&$*=O$EvY3cGCphkqH+$y8_IR~{d^--XoHIq;_X z;ZjdHO)5$>>Q=h?o~_@Uscu9+Yck7R*ST#k#uO1NR`aQ4|ilf+x-HJ$u8hL9gbRJ=W!qw&?CL&M=9al|0?m4aO0wts${ zxUp_)70de06s!y=I2l?DDqB({?=>_zBYxH1REb2LKAoYa%(45Te=T(JWpdWU-e(t% zQ%0@GL2v18NVjk^~JE$^rMVp;tK?c7N`^g2{Kpu3S9mD@QT;b>o7|W7H^gpkAU?3 zeh7+Ef4djS*B+Woda7+26rK(C(Xpz>jsrJtCIP2@hho#y?i);>k}Jz){NV zqrTkzIlY zWPt!NG#hZ}17ELF&mb%mz98xCZLE^DKN9kb23Q?|CouNJY7<}8#?IF+XzAEbqG}AD zJRW`dl89UXqjWHkCs<>!n6k*mj1)80hE+pm%`BKo@&)JmKXE}7(frRl48m{C*D8C>gH8q) z28Zjjb3oy!2j=1yA1u=tBUnSc-?x61tzm$HAyp2OX}WUgr7rq;KmGa!anJbYdd4vwZ3VGhdssz zqE0c@b1Ob`EePqQ1i7W3|M@xK5;Yc1{kuT)b!CK{Xx1FC5LbjflaXQIr|d0$h=j5_ z1@rtRN}hOf4v}cB48_{8naz@8%pJuVwOP?t0jedMx{^uzN8?g~#)1(jOaN2b^|KlB zS|WY~%pE`G#Yi*Sdmr{bRCceDfBJ9~dHTVKf%YBl=sU7kk!Y_$V3r8vhB6^RhW(^D z*suqU6)q4m8{#d|?cd=yE_?JEXhXJs=i2QW_l+4DQ_sy<%ZsSF`!KropK9Bajq{)%Z_s&t~k;^fpmb`V8c@MyWwxL~8ZYD^%co}mi zk0mFK{r<1&^8{WBjw25~+xV9^{Xq5=G{HrWmrc4K+uMKKt91db(}BPZqepvC&uW1i zw@VYsz|7Zj9BMRu|IPvk(gqXVn7K_@q#Duv!IDVa7++9&GrGAz$~zqw#kH28&ad6x z4!05|7>n+h5C<@!#&6TFjuP5ZqWUFDYRL+^wY^nj z_rtNDk*O4gOk>AK+i2if7~{%uSp!T5ju2Nqs3!yKeisjn`ftE%So;1q*eUgnO+oOg z^5P!*O1SjE=LfTJ6&2~GTrtBFvT3XSr^5^!TYrpUAYA*4vh!kLNaNT|k7wQ;?*%bE z#Rsl2FfQU|h~6RfWVK@kN8Zi(?`Vcm4P2_@8#wYcAqniu#VKI5;qGQl&uzxxg9AD? z*&Yknm+ICIVHS9G!i(1w&hk*pf=(-%I&r3A9ntf^oXVP4j11o9F-Z-M(q%FYBTZ5~ z<#=gx2>G!8jn@oW>_p_PNuuLh!tk!jipB}qb_8g{75JGHkUV}$kw-Jn9c2zc+{e>Y zAIFpEC__b{nbAvW(^rrga@%YPM?Y-Hk3C`UOe{xob>krHuTuehl%(-tBHDS^i63vPRnK_4qB z==381ncxO_N-7>DU@kn2j&I6`xNCrJ6oE?1!JqK=#))dNSi&X>c@Oq>B1loU<52wJ zj<>xulnz!I-1t}SXK>xf0i5vvIllhymRA@B4wZT7KDhl>2WTDoRyELvnDwdV^t{SF zVzc;h3W*q3&ODp+O}P+muppW`nh z7CD34(&A&8%$aF2+OA=;ML|2c-kQj}wTFpT6U$6$oJ zwSl>`v=0RX1iAf$a{DJCj#>&P@5y_yoz4T@We8|o-=#^<2+;kt!2dxJ8E_F=3W|h1 z|Mhnak(3ZwiU1{Nhkh+oPaJa643|v`{psy(Pk37fk++XhBH2)wnEe$WA@kR@S9NT( z959ctfkifF6ca@#A%hx(BV>OFM+UZ~@9ZcXh2NvJWWz9yK|>R}qlu3NF}usYg)+Rr zF(X& zlvg-g4Wd1zfW|@*<06mwYXI|#q%XnmOJVAiQGdi+K0UDtSji8%W;PTt&R?TUYY2Oj z6dy7Xm4KabF}Mq{+HXO%#5>83y?~n!!NFDo{?Uu)AaNsxFds4zaD2{xW4XBBR^NiP zc%n+Y`Cos>KtxWiWez0V%nvteRPJu*%{xx`jV1zor_W@m;&G`ZseTkisl2RL3Q9}WS56WMweo}h(tYHn^U{H3nakoC%Mt0@8&t}cF^p-z$$7|@hGtB8^=MT5OD*Q+H27QZCdZZr&gI!wItdpgJxTv zANE5lkAsqt{qDt87EZ^<<<5!Al4!~;8e;&4vc=Gp;oC+XU#jl)94o`?-$Rnos1cL? z>QzzY&`4xt(cMMl8Pnefy4{K#&U3n34jU_bZjMr{$@9XBn1NOjA{lInc`b1Q+FR-n zZ~%I5EHqV-j?x|*?TV7WzMdvmQo;Xxzc0;*9RA<;4B`S9kbouVA=w)d@-{EbK+)2Jsdc5gmODwO8tkwh6D{aPGQd2aC9i0BjMVGo;Z1z*)s;O7S; zFuzo3Wo<^MG3KOht{VT{11rhK*sDQz$%kCCT1{T%eyO{dOmOB+X6qX=p|$o;+jW}_ zomFYi66IgrS6!~yM0LCWwvAn_b6wkue6&3**}peUZ@9USEP8y*7;@*U?jz-XDU*D` zZtqOi$tEUq<$-y%VmYbRRS~D~)L@fvYZNv$xOB@JRkmNcyezj-&m_vuERx=j*WUeN ze*gM|itN1YcV|cE_nQ09z8WZI>KdF(bx1GkWHS5wIC#R7-(r+ zZ+A^MmNld*Q&O9uJG4k7&oBDQoU0CW;Jp$g%-wEX{0L!Pjfa}w_(DYpiSVOHd?5s} z<0Zd^QQQCoT0;f=QdN}LTpc%1S0#X5F&PptKVYyB@zVtwrT^~taG-^B7~Ge%7I$p^ zrbfY!8pGWd2FAIAG4*eFmdoH`lAvT(PHJ_Lz+da#QcN61hHs~Uyp|bnz)=}Cn ziT9haT4L=w;fL;DT53m*70}!U*C9a1>G$$=Uy+jTY`td}zi1_Due#}av=+?dgvviU zuav5eq{!Pgiy~k&*PVCm^>kWUaYzuctaL|%0lsUep3d=J;nyv~Eu==L+zZFg5qbTlz_pnH&sq7f5Kl zttRA@$Gmoc)hp;+5-OI{d#|rm3OeE5wnb+M2-0S;AFVz)>4ythYod^olLNls%acv+ z$@-db9}$L-w>l6m&Z+4ZfN2AkFOE`hMRnOTADd^zJt1zE%zylE-!`Kg=J?q)H5;_VCH|5cuD&e-z za^;f0S|TM$HWydv1J~M(DHLn^QoZjPJt^`anuES|;(3c{h1ylL@q44qJ6|YLbS$oQ*jsL8_r#;@21DuRG-3A1OnGx?_gNr8u^WhmjX(Y7q`gMT`nkQtGT4 zrqZafu54u{2aJb`wWpWS-oO)=R3q8#Wp|epDt<%4APzFJDurUECD$bX5)|o7r(t0I zmOc>;g`n@%Y2LDu0EJC8?pvk4)I26Fcv6YcqjgNR@>kqr?iqdd8(1l%j~3Q)nMn_A z9`Lepz-(CY@L%3LDUK4hvptV$dAaf05ff%>8vNzx6apgNH_eTpeKT!c7N_bdD0vvQVbf&BM6xv;Nc1;NL^fS)&E82Uw_OON@51n_&u4@X!iZiByN;imq<~I? z1DC~knt)*cK}E@o2nyvl*OR9q?y{iy{q&hLIFnp9Xv7@&C4QVZS&XlKw998K<eu@Lg{xABsqN~iO8N_8pK`Qw6=gY8r_>I< zV3WV=2fI!ZkEAyJesr*N!LY#Na`NvhO)c3bi&=Vatsajzc(jNG=fWtuZ%MZP~T1HY|erZGrmjXi-=2T*T` zQBqK()!;&{7_mNr=joy;eSs_Gps7K6f#!jA9zv`>Im&ndOc_N!%w!()>tS~66T=Znm+|>QCqZM2LDv`)ElJimv(;@QC{BSnqG+b3ww>TgAJ>OT&-bX3&;UtdCqct=X35m2{>xiDLma>bCm!ZKE*5Zt$vxCIbmka`gI6P~X!tVTkqOqp7rYhp%j7TbI39NoN>2ELFT%8)G%y z9*zarzN{e1M^x*emChk1cfX1UJLAjNRP^%aR(R(w;nke* zI%e{fc4Ynt9@I&{nS&$=z_y@L&rph-z}E8J8wQCK1>@0ns$4an*(k`aNs5)L$p0SA z`ma;qId1a7OOIvA8~^TCiQ;?$EOT|UDS#qO2EThY2CK{>Q7HWaji>uzj+~68_mXe0 zuQWFywtg$G>i%ov$%qBum_z0HhOd=+Cj??CxL%y5u0hpM`5x3Ze|n3Oia4` zO}ih|T)22;gHqLXzR0#iGlIAIEhgWOzrU4+*Z8%}_bmOCtEG`+CphthOn_|PECL2&TspOnlDqd0f_9zK)UUjqIWp_4?d z5+LkhkyTqTi}!3ka|nu37MuVp3Qa#;!330@{2feAM#4JhwQI8jk0#&i3vK;!rhuCpvls3D#BFWoa*Ws}2ZZvW8 z`&|hGWje$q`33+^=Y>UNt>Sp~0`?x=L%iE(4)?p%r==N@%BiQ4aMkikrb@EFjlLd5*99BP)TmkZJ2AVl$DjGmpa4U&5)!nkEVTpKzO(JdluHm*x8Jgn0ifW4$b z&8KweUWt5nWL}U`Qi-OGc$hzX*qtCBc{ z&1Wm`_nWzKH_6dTAn~hGCw$|DUsn-}KAgIUoR64aMFtd-x2Je8Vh&*R8&3R=qXFma zcV2dgnHVC;Faagr$6iexpyvTeZmR1T5h%6D6mf8FSK8@gqkAei zI0{uGcpyXmU%_O5qO`Cwgb2^;RDVl?#g(!GEI8^Nr{^@Tupx=(q)E7R>!lb`>c(1@$a%GWx3}jgnFB=My)vn!5PuQ-XJutQ(WL1aYe&b#zB8 z$m+-Dj zd_g?a4hZ5xf!D70Q{W29W7468!zz^l4jQKpD(T5z!5!K}|5q|`lH8RL-VrH$3BkW* z65^J`p2dEqjj_MXnM>$yjrt=m^{3=oKL9p%+o`dN&rHsIG_0vp+wM%%>@hiaj{et_ z{ifrWxvrH>8gt32cRhBR`@G+u9c7I4n0STD8?`#6k~)t&Q}-E-J^%D%2$TkybbWQ_D?%ipRoY?f7?e>;P z7IPY|@9~~=tJO1&XxL*ex;q~(H*+#bx_c3QTTualarC|JY&p&Jdlh_rx8L(IFr1!j zj_$0V-x4_5CHW0N1<$@1b{B2F!PgUU8Qo+TzdtiJ$pwND=f(sUOn2mdOpyF@K&>4w zj0{!{qjjvj9*BrFI`d$RqFurCKo!o%CyS`aib2OzfKboM^hGVh&wH zw$riQe*T?0aX;hVQi*d@^CYDv^TekoZkCTb9euN`v@U+Ox4XXfMM#ow|5!?`eEah4 z-!3BPTgB{7O0113JXa;5VFu0UkfV3=p=qUuUa)*?Qu(WZ7DmReIh7KLmc7k7TajHo zbQ>Pc9vVrHB({BhZ8^BRR=rA^a$^TRFK2#NRL?nNCM_;VWv23oWovYxH7(6r+DAJc z2tF9ek7#L1I4aK^MntTv$DzILIActcZe3m`dAwfkzAS~N?B}_v9jM%z$fouzYufu@ z;=+?^KQV)zmHa{~3%6D!Z(}LfpaRS0pFPa-IjjDD9Z_yyzSp&6+$u($l}aACmS3E| z*x5#Bz|PLzR8J4do`)UC0hj2NFOHKJtJrkj@!!!cCg6o|c+jUjY?m+phbWfG9F3 zC`t|~Ac)k^3WJn@NVkBLG(*=YAwx?`OGrpb_n@S7cegO~2m?c%hwnfC_n*Db-fPWr z4YF9k%M;=Kh%4{*FJg}8ZO)rBPknbnOJr5$B8maZ z%KEjmpGGM~L>3sYXBcT=Is(||RY!0NrD08BqQ+(OL zmXGaTvyju$F<&GXkteoWV5rzwT0Rc+{ZPPcYWkn&3x^je5oqS1v;UxbKcugEpP#xU z@R*TYL@SoOZC}XH@4rRJD02BCZDAXI90o#BUxtew+z-C58NFmuLIl}dprC+eU1!sVShvzh-=BJKil_Vfu0b>JG~@Ul@E2!LX%tP%rF!osD;G&r zlMR(O3ENbcar^T>)`rpE`aP{Zd}p;rYL>IIYxZ}aH`FLmKtE4bNyF=9jBDp94RLZT zhk>;CS;a>C@IuYTbUi_M8w5C}a2E=uqN+-ryWjnBNM2_&QdxETj8%b^ zm5oDot!H$c+`{8=anpM<)}G&nQ#!Z zoYI|YtaCh{!D4~mkdkx7*wC$MH0xSx^Dmu7_1u?k4ss|mp+)r#>yZ)sQt#H+@EOTj zG4tM%|Ng7Q7N`C$iSK0V5T9tmXk_a0h;O!cHP!1ik%)i!3j2KVswdC_8ttZMtH!rf zDa^P>JT30itVD>V2zq=-4YYr852PIoY~=cK?3-*gHYN_0$|IOw*i0MBgiN5m3{s0j z;?Anx-m)I6^M!w>)Fad9OL6mo`_T;7_&0gb-prchcspXzwNn@!dhN^XX<4NAJnpN) zlfojQuszR)vpuY8(ZS%izUewXv$e%(T*0hl7U1x^X-Ew#P8b+PVAE}5qfqsto8s~B z>*7+}zCnl0BI-~b85jX{p|t*ZS7~{mlFD0-7K!G8AOqF&jfrykz}p~%aIpmVd)@`p zXf}g{TXp=xv?^!w9V~wIa`0oNRqmn|KQ2;(NMHWOt3+r$Bk~P$)jY`QFO}2 zjG5wgrK@jB;a-l(S)TVfumvA0FjkS~Nj#8Po^*cDnx1h!R80su-uFz+>d}_bsn`Ox zUMnwwI@KE}NWTyB?iby7=_v(g@+l_X_)HP?rxB|*o?KLDeKY$ku4wN2XZ;(HV1SKZ zIW8-oLPADStOH4JMTb+}r_|pjG_ct{5U)6vrlRWxJO~I>k#foTT+xxN+)E4r3rL%h zA->18&-&_Vk#rBXn8ARSBFRLiEzFtb;5<4W0E=*#)Xpn0Ir6Ma zBcNOFp&Q+CIEaqI?bF1T3}{7yfX7u?*M0k_m11nNq}_@|RnkV#SW%nD-N;8*sEyNR z58JWPl4;ra8r`ct3pX{*+m?`{0W)cUMQHBJ_Qu%bRg2RaJzYH{ z=@uaB)ib2SM%Q;b&3dZtd3FZ)1DDI^w)OiIwTGsBu|OFlvvm~&rhU1JBBL2g8YubY z35=NJ$}BnCs%;59frt4}rLBRy*~05>PR5F_|C6g7-5kS3GeyW)^Z%@T1F}!wWdW@9 zq$c{i9Kb%G?!hj7$dq{2=-n}c_ByU`xZe@Sl^X0Tt!n8+0cB>CZVsE4rVR60VBPNs z?sd4ooWQ9lHzOorJzWvNtZ?2jpx?}PO>iUExh^uJDxupDB!O$Z_IGHBdD1=&gTm{6 zrtyDG_9G)t3xTn%pJmn)CNhc?z>ESjVykJZIdi23HP*(n*(o<2J+Ss`MVJ`Ee!b(a zcwa#(#OE6-9tqdnp`ZJ)ac5ssr90!6oOp{vxRx6VCf)*1?8W#=dD_Tki2a+{idZ)} z^Wnn!)^AUz^(RJh^aH7PzU6)&shNtCwuf(w&#s&?pUT0;;-kbJwNksJ;MQcE8sS`* zhvHK?D;@`(4=+Pq$qX&72DtKp0N0|fG^fdWT$Zvy943PhB5+>ZS;ZNKTKFrcC@~zI zc0uT4`>zSUoJGxs{*x8%Dw~v)263Ici?XSM2DNA0BXMD)!d?r1-FPC1iN^?6;!MtN z1QgK-Ib07QMO(!;grouokn(%Ju*{=~jBGU-t5r9E0pS$8b6opRTHTXY1>EqLgBgrQ zNfa!ef(Xx*fzv=uc9^ky&9Ahf#JwRb<}r^KlahytdZ{oLY;m-GV+s@E&@ z!Zx(b|MYR%Z2OiC_ttX&c6o5gGh6_ZCC8%tuT|9z8Eg~VJw4?bO+XCV^-)qo;hrcuZvx}})w`X{ zKS!xDHrCro+`%(d@1d=ImAh!+l;q>Fk$ndT*A3w1SbAmM0?no2glnyns8+!BbgRm| zn+`u>^~l%4oW1Z`d(&~~tqgz(k4*{v+G^T(B{pZT7}5nnhqru<2^d^ON|7It?YTF@ z#q@!5Gx<0eQwIh(AN!ISE)kTW9ZmxC1R#|*RqwZ1Y??XT^&ZKgXdhFvIO=0HED z3T*~Rvi2L9u|41urTo+9+T-P4haZ3BYn7V!eR&?+1DNa=T=D;rL;cs~vx{U1@wIf@ ze{PzFJHXiF>T-J%iMcEXNa<3lh9`+5Q^2rM@*+X2NsleCfV@+7=IYYzlZG=qyY)<6 ze4#;uqAmJ}8JGZdTe9{@z?V@uI>Lg&#m+u0qis^)sFV#wyQ=@(0+|n z{O;GOVUAPgT#dKv`{X;*S!Lb5@>&2{q-s4=k_UsS(H)tsAO?~?Lrp17sL=!2flAaD zxF5689U?wYFs8*H&oY$REIKm<@!;p9>Z%a~-#<``?2+3~4rv*%Ukb5dfszwF(mq{l z+ccE7#z{{tIB((O#Eb==Kx7kCI$aES1j+Zv2T!B-3pDEkn>K`!TvGB?)rkxK0sAlA4#5b??#^yz)t|4|=twn-&!M?Vj&3%YJg(J)FQ}cr-?cw=^m5?fR@> zSJ9C^`J<$`2X)%Pj2yOf#FI-jG(w&}&Kr0ge@F8E5XAwT#^3tUf&Rcq{hSvq-IEq? za$DwU@3{vBy*IY$#&m&@x?71zAWUH^PcG|79z8oJ>v+Q`12Du4mH#g>*89Z0x8e9y9vK@h=uX-SR3V{06VLgU>7O%zJJ ziNf_i;qtt}S=+U};pWZ!5IcXFZ-*{DE6!&2jCLp_-!Mzy`~$SzBPN^al@ zD(-Sdwtr^lvuhJ=xkr<8-hq{)D!|p+a7hftG}xxmjz1a_Y%Yhrr|9a%#4hoIzh_s4 zTm^EO(lfV*CptnZPe{*{KE1B8;=J78Uas*`NlT7Sn~NSk?x?-?2Ux9y5UHDatG14< z`${Qd(x(A0)g@H)IB}WEzCwQ-hov8j$IYyGm3Xas@6Vu3Zg4;LlK;Hs_?#?VAJ-_I zy!ZeRlk~QSj2VSv^(w>75b(RZL4@|Yt1WqRCg#~p ztC-mmY!WJzYLr3imzrvI$@qleM3Gjdf`5Nm=$&>z{sWqn!qdu4k)-Js0PELyznS3W zAhlGK3SL6+^vb9)Z#mcEXufx4Ba3jdjl#l3Tnmal((Z@igTXC)a&9_VL`vt8iNv(i4DARuLoPC0{C^yAe?I}fGlV# zwsKP#QcypdFQ(>J;xkxdy+|eQ7K^`~VpVNWbBY*F{lXloud3+SgNc1I=o_GvLA>!R z8{^?F{%ha)WXh67RZmaBSk=UoQV813yKLHUlU@%gz`hb@|OZK1q0f4H_Sy?`#`Lc^j5l;M47E00 zh#aaCEVDf~%6fU(&VV;KPLM<2ctV~qeIyI7_qU|>H+lyA_WA4m%9*$s&A z9KA}eqEpY;><9GVZNgvM5*O-!`2a9j$pB$NTK{94_Mr-7y%_1fYQ6u@#uQ@#U^bLV zVu-qvYxW_(RZi&zuY%G3GTFK{D3n#po-wYrLO0HFuS-gGs6GT!^1~%Ed$UH+3##Idaz1{PflZpnT;r zGON?nEldTyKLQzzo+qylxdS}pm5y2HVmQ^F(=M)o(oHae^eY(|e+?tOX6^o|Of~m< z%>pw^4{B?`mLoP{6=ck$q3UJHdR*rhOv-|&nWUaAD#*ogegd-q_=1FbniY{V2UZcz z?v*Oomd;KhnHL3v%9)$GSdaxAO}D1=BO*VQio$MMI$AVFA5o0*+ABtXn%No{(h7$0 zVigx7tJVV%sfyao*;t2saMieIHM3l1Rg%?xpi@!mKQ`>7Zw#5W zLqk7qPG-bt3?6PtlSrRNJ*Fz>C7oYDJ*QS<)mn<{NwH$g;n9(~?LXpwsK zy-wkiXC{BAB_ZJ<2X&9O>l5;~|D|i6j)TDCY%q>jx0wSl_^~`W zQSD?}w@&QS)vkNQ-X*Xwb>z2nvN6HY;C3E*>*P`T`5eSY6U5d?5&y+t~sdZm147Ku9@Ep-5Moy+5mn|0~*DZ=;5Oj=a z%OEsbp3b6nZgE1ugjd8G4Zq%zb0AHlotooge>3!q4~e8=L-&rZbU2WKkJ!k zjsu>rTmAI0e%)BM@fTpt@xf32Vit2@zH980oDlo<#9yw2!N}&eqQZpoqph%H_po$! zW@)%RF^7Ksdu}rq78lsi+A_PwBaxE_dPWhcezUy7dKLkX?A7lYW;Drm6c_SgtR&#m zic}EcO526lU_v+eM)q+xV4j3hO$F+my{;*+rLjFQo6ULB?Vi*T)1jQGeoJq{2_qWI zsgE|w&v0yS?6B{z&xkdq#^bp=K>@}BFOpPR`+IzGDyvG z!+48Q^ zK*w{qHT32HXjt?iKiqhFJ}L;CuS~VH!a!ePvG94urVT>BKoP}Y#6S@{lB=cAZFUrN zs{`_?S0k)ECGTbE?A>f0jAt_T9+zI4jhEQd6)p@ZLF4K-@ik|fo=vZq3n(H1d*l-9 zc9RrW<>OYL5I{)j@#k36|G@|qoA}RT^j8e}{B7&yd(zFa#vbefeK{l8MJ~(V4=J|R z&z29pv3ONAK@#^Bs-Er89?`b@n{XJvN5?brnQ=D}$YS7^SKBNQR6Al@w^xwMM*t#? zaW2tl3;X?IyyR_IFY=n9-Bge01EV+gYY;v>PTwG4x^^<9%jWYW6n-ihVeBq0Q z0LB21$4&f9auPSum?Df6=hP@nBBKLJn5iSO3&uP1WwM?+z=*vNGN?RK26?^F#74j{ z3b3sXW8ZtSS^}L`tRfJU$O0K<$^WbhvA+nys>%#4{-;p$lzRho0KDdXbnp7UTmlgH z&up)Zzc-7*8z5kF1fg7f?fLW{h8i9Rco;*BBpl!yL%#kf4ckMiw?q^?&f7yT{mq?5 zj~;*pckXMS>#1cAw=!DlW+BT`l_*8E?y)Kw1f%tAhU^7;s%Tz?QBz>-J?qf+1;GL zawX4ws-9tnFpik+rVvUEzLQGdNEV9xev#pPYFNmu82E3A_)$ z-cGcG0I zN(N}OdHgqX-h+cOKx}!{|fV6K&J<~g9q8VUx5)hgqOF%*`u+HaQ z%1P{B!0G+>joodr*X9N0u;q4Ry0%?AoHo?1-A-Mr1SH8{3^UB`OO+r^Xj!5e!%zg_ z1{seGgl1eUj;J|$0`|xsJVZsO_iwEDJr@y*2P*M&5(PT;6u>QCk+1tr ze3gvj15jV~0W|G5P*Y~r^e$y2sO_=R|GgHcEX&A>4sWCrPNkR;RDawzc*SX!l9b_X zP#Nlvntj-$&Z?0lRF2q2{pRR!!-h_%R>pQ;>Qy4isO-$~5 zuyZ+Ij+O$hjLaAZLle}n@elbM|k%GUK@b(XV?{ZiBm2oDn9|Peq zXhKVdlA2evHouqX#QxUQzCh@z-Gq?rxkJO{%da)@qqtE|tNsJY8szy08@WYkTqZ}n zuik{9IPL;i|0bq2^EHKM^;&T3z*Fbeq4ucKG=#AIJ!&!2Sew?g{L((q_3E`EAgdGp zsGifcI;>C`sj>k$+`df8kVv1Wa^JI9Y=`vgdGKuK6iQE3OJW3UDhV@qOkXOce&6ZU ztF1_GhHy829u4qn?(g#mFt88R!{;&F{DYu+dP+?Zm$2#qXaVKs7&f@0GkmIcn=~~3 z$>?YKC%rmXgSS?!w`M(0d!}oLvKC3nb~705)ua2cWcpLOy(%8aIB9(o9DmBh(oyZ& zNOH95*NIZO_n(EzS9eyglO_RD1&bYsV_o(+?BpU*M$-`iuMc1=J>$+Ee`oJ}&trdG z3cD)ot^NseVR8dP+5eo+-M{o-oL&BPf~JG$W+UGBc9j459NCo*{Tb50r6Vc_xV^52 zlP;_SRag6m9|eoVYc#~v1J|;ka*6EE1)S@&rx>pkVyMV#Hh zu8ZT$``T&UAD0Xc+pQ-T6+`|VDiT=OcZ%uWlmq05LaQKwV-ZU z(V&0ud|9|8Ecl6i?~E_Sr3g0;h?$(GRdsmJ8W{bt-7Me)_tIJv9+SV!D)F*3r{KlD zoW9_gEK3}$uF_9xME0t6s^Y`+(%8ka@pF~5cqhXTp5c8F#{8?wddzKm95$O%g~ipZ z6WEI6k6N(6W7DV?`{~xJc{ji;$;JNW#@K;1pydYMzXgHV2WTVi>mkTo8opC$bxSRI z_O7E}b+Z$+H#nb4-gzkjCyEmT<7fHVl&1v~x3d5tm@=-d^opJ-1yqP-byZ)NzB86{ zUALR5!EX1d_CcZ~>JX?Mh8B`Fbk_y%QHgMDxssUuc%?sRE?^l1DrI9>kz}L1{I7$Y z*H#no;Z)ryTwRr5%5vWT^hC57BkK9*ZtxCO`m0*Kz@UjnkR+Lo7 zYpl+ml|`N%Z^tUT9R46-?D;}OFIrV1rLef&G_RVj%XdS@;3@q}Vg&H?QJMqvNexa^ z9G5{wS2vj@uKV_FGU7bAE~?+a=vx*rkvzY4vF>{ z9)dvMWNCs&0@!meQDR841doeR|L59$;b4dBe{2-|Z!9^7tY#snd$S%R`(J|K>9>Ir zLm4VOov_qm<8;5Ek@l6Iv9Y|fVc5a0MuDMWNP7&IL5j4w-Fb!cyNXG!eNiHaaidaE zuZXwm=62~11-OrR@!a0f8rD6xRe5TKO@qyB9(oF<0Nb}&?(Z|c-76zHE!f|{q;^R( ziz}qXKSb*XucLROJMnre?VDV+5NJ=lP&0p6J#xB73uXSYr#n!4^iCZoI%~3)O5qeo z2~!R%chpdF7~2>rc;Grc<6!VPwGt=YyFF3V*5`Rw?WeF_h|~Vm)S?=GyuJRTsSnkj}~lNm=aU$^=o-P zoQqs_GN(`9>a9k$y8u3@z)@FhX6QWXm( zNuka&d{8nM6IE?d6vbJMPDAafseWov5jbt>)4kr`rIZ6Q*w<)%PW+V*j8b!)wMlZU zbAY&2ou3DCS}wxY;~efDro6YK;LZ^QOuDccPprcKQXc4Elm|tJ+_3*FYY~xuJG+L4 z0YQO`lIVVPL>mzyqse7lZwaY#c3Jf}ggdUM<+zxK7*YhNa0dBUAaYrDYv{GS6(AY= zsXZe9;FI0wWzCb{?e&F{mC7LFb%+)}PY}(bk)wf_0ydqgUY0|Ep`DVP2mpxqr&!CT zb|rk#zAm|Yts%8Mwe3^r38lahaXC$HDyJTu0_EuF*R>!%otqj@D7T3yYRgTEb#xhN z!qDdyegz+hO*>GsFl8^2@Zvw0tgdUKwh z_ChkewxlK88d{R=<{5+G1UHYG?;DH=IC+YflVtRDM$t2veP5iGH5QjcttRm0<0Ppd z=TpVKzx(y57t`0^M38K6i>aa$H(CHrs*s0glbvZN(K5B;Tf^4@LQ1K(-ibgaH{HRU z^7>ticXNm1NBtY4+j+Z)>Fr!e1L`REzJLr2I)6yK$S z+-X$!Nd<~aph5+z2f!J8jNmXp*8!G1ke6MIsRDj@Y}ZTo%zqa@TDs%DSw4EgL+AeA zNP-u<_z#*FZilgReK-G~DuC>R_}QVPpmN>*h|UoB7Jj^ZUDvLj$o&BOvgztdg)lVh zO$Gx+{0>r^W8NXe7o+D9S`KJa%{Jym^(RSWE{BMVLLH>?T}b*C;7sXczP?_k9n@WS z6ULcki~?X>hgK6MgO7Mi!B<&o&1M^TN(1imsEYFZ*F^Mbq{eCD_ADT14VI0EJ#i$n zai01`6ShAlCB@DtvNjsFlNrM{A;I$yQ!$FDM5BaVSgjLyV~=Xfg463)gaPQln5NJz zd-b)UV2R*L=hYrT;)^U$Kl^XAEN7`*nr|TG<9?_Z>eDqhVCw7z3nr*7CRJu3>K5@> ze^c4p=4e{yuO*ya`Cv$S{xiYNGrz6x?2(Gg9b=8&wRTTKi>goR93OifFW#6|rEUuc zg-~lqA;xY{9|vIQ5aii{iFSKhwftT7XHvwg89>sng?Cz^yYm^A73V;+Z+f<}drXob zV+Olfq}VMevOB28F4Vesq{J6!_f;QksD6Mn+j$_n7q=&{MNAjd67dZNfc5tlRVV8r zw|$>qP&rRNQ}4o`t?TwKSFhTwxj6*#WGW=BZXJC(15{vQ$k+3GDWl*(cBYx274b~^ z7#`20pbFSswXoLGO5vvY&|~f9>J2iW;4s!#JQXf7Zl6U6K%`S2)tPe#e`cHxRf-z3V`Pb4>7!590S)4)mwxSV%gbUVk)TC-2 z_4%x}!OYSIZI)?uiyQ$FVfT?$>A z5j9wur<`0e8&mJ()_uxO2C^!oGdIpw<4$IGOBkw}+RgZ_ z(jJlCPw`PEUaT4kL0$_KP|NY1Kz84T?L7qVTQmS1M-dE0Ky6iFyTT~ewyh9mELDdO zBV=G~iCMYVmLb42r8^Ae?!^>Do=MXI>F+;}T(-2>b!nh9`dsog_;uXv1lQpP8oibJ zKclzGeorpH5a*ApsyJYCualu5K79_oI+RWU5R1XRTgR?p?j>RnhR{?ICI7K{=hN!| zm56&$EU&{_TkclsjXmkwqpDtY*yg(@=)Bn%-4V@Z72l=>(F2Xw&>iXKTtGsxPyIs} z@v$^hKMplP6^0sd60B@+Ji4*}TL2Ky@SmF`;L;9dsoZAV)-yGob1Y7Y=zTIZB;KB{ zXqla#jXNPFfJz5_{6NjC#HYq)Wp3)cUBzsz$xtQKjW72Of9cX#Y!;6EvU z`1Q|2QqnI}XA@;szqTB-)XrbeWDUM@O5HzUy6g7yq}oWtJ29+D#xJji)(vfCI7*HA z)Yk1MT`>A}a|W9pBr1Y)9GkaQCgh%~&d zE;E&6T3UK{3Uf4OYDR+tW1+#4S^oQ&*uE~n9+rk%p7-Fk`y!%v!O%BD&pa%j_uWN+ z@S>E%Z<|EioUGbpyvT%>JO5x|efg!%tyQQZU0s;@0ylB@B~oJZvxb=do$ z?A`QN+_CVN-P^DAdYNHP=-!dt?#+F?H6PXDWQUWO_oR3gwDI*M+MgjEeGlgI-uFj} zs7G|)E3pkzMNrhJaiV&~w~+z1*sAvZ6A&g3l&OW$LhZF~fQNYztD@(}@Sob{pW2*} z6|jhyb0$Bu|MOpmzCY5Xf>{B{6$qtFkdEPE054gHc&~u^b%(V0wlI_|z?5Hjts2H& z%*s$$=z;-fnZR9YNL-yN7%l$d=`*50cG+&$h_97$A-@wFp_0nnzlXPtr+J$_g!ZSkuyH2=KB)` zL0?Z)F4&`iXS^g}(pO=-~Brx@Qi+Nd--n$DlgE*m`Y$4M0F za_^9QHtA31b(`r^gR^hx*eqi-0p75s^RXHa;TxP0tOUS~}W zCb=ODC6M?Cwa_Ox>H7u6u*LH4Deo7JA{Oxq>tn{zKUJ=8OfHJD={^ZJIW5_CGB>3x zvS$F^`)fyb0yBlvK$%Xd6p6BtdXn1~9@KVBZIu?kj~AoqPnP7tdd-6sRDNIS){pc1 zz4e3Mb}6%?v*1^HkC@3$y0b^8 zU#0w0*5!{CIU0EbzS8t?1{rdM@M8Nnaa-atv`Y9!&D0$-+z<6_w9Op~tZ2}>_r_Rt zDo~=nDT}LEc$RZ4P?YTz2jPH+ysCs3_9mUs14#{A$65C#QVXp}#QgP&#kp#XbgtW= z#^cT9BkN)BPu%Dsa*bEOqSyNzsZ$<7abg>%R__ z@LY*EI}sJZ|Ns7dpmG9^F5Dx(`*zMZk_kM?r6yZa;qKeikL04*q~Kz3sjK979|i>l z-CfgjFQQ10iCM`*IbMAzrOZKh!-tTSu3lzaK^rZI;OX3&@rX+NA^S)PI%*)B-L*IUw;jM1(LtyL^f4yB{jYRuFpK z)j>o95!`Ph1Z}ki`P4ke>*PM)u&Z3!0i+4zizsvQqiJrib0l&jJ0QhPmdtsT$1%kE z$Yme1*Xyc})N+Q=%nx^*yZquWe~6VG-Y)o(beO`gWohY$%9=}I196mi1T&_bn%6UO zW8CEzThZ&>kg|6peI0Gi*}eo?Qln@rXZFNyWW=Ti-ju$l2ir64N`hjN`#+t)Wg_)5S1vfzA* zcgm~N9bXtI=1bztP)zMP)>hBgHCFR+&2_0<0m8#Tlr4R=zm55NdmC*u3r6&}IXQhH za`rGz?Ro@Lb1b(kU_eCvv{i6O+6yBSfhHA-x&=xPCy7%1NTQU655^L8TkR=Z&zf)B z!+D)Z6DTh$y98}|-Ij*Cn45+(q}LLdd9QArhV^yXkx+^C!Jsqw9*3+FJmKXx(rZ{= zwR=&MlCN|pHwn#-D!%NY6t<7Gn)*{Y{4y(qGE)ghp^vauv)iQOI1LFGFO}oTv70xR z{_}nBfZD^{H2Sv2R+wGbP2Qi^?%7l^i=M(CQ(qrnDz3L@kRXC2FF*+)^IPUhNSkra zjJEEwAUb|;P!iHGe??T3^Lk%nvm4iu-?7#Z^34=BB-(Jt<-UA|V@_mxvs~`dK;x#uy^n<7v zu$|Q5ZufL&(EVtZMLyq~J(~T|rgr^rES|ojIN%X=Uu{eqg!?bC zhWu$p6*1j6`&XplRd?S7eE)@opevl((qgaNOR6DlrD){y5WSO+UGbbw$}VLavJXca zFLexCstb4PdrU0qnuiYZG>}4j5{_fkZZ}G7qJEp5iN971&WmmHGG^}w6Ho9KS{`789sXpJ>D8MJ`!Xz4^^5AAy2C=w--MC%P2a>$ z=FDek4>_0CC!PwoQPXEDCdU-kztBFNa@MKxd{WquN85Wu3O%dYQVN!qBzL|i0mq$i z;61;3?(VbY(H!!5EkCL*(!QoqYf2-!*>tUB_S0_I&QCkCjq&n$GtUvv1Z>yH`99#6 z>&?&z>N#(Vnuii}%~DTjmE?SHJ~)c6G_NLkY_(Fmv*{M|edE?J?A;@l>c8-*Jji&C zbLa9F9+#etr&g~Bim)t-@$7b2TQvJ<8!g2D{18>%dwxzj1?4RjEV4^Gp^tCfHr z>iQY;kh?6n4boqJHjO*|F;h_fmtYMhfD|%nGl7H`+#-ZbW16N({7_9Z$?uZm*v-@* z!|Swq!Sh``pgBaVaq%_Agl}zDdf}XIu7yA86jb#pwdp!o!LJ>z2*)x&3ynXMajmGY z9wwYS>wWBOk&i3b9m>&H*7sOzXPvB^Me>+(#->K!@86M2WF5n>z(npd z*d+`vaiYzosn)j%o1H;d_W*Z@b+fS3)mu|yd8(Y7FxoIw_asSKtFJx1s^iC-Yt*C2 zKldfx&fa_&?(QCX_ZRhe%KW>EnZ;ViHSwRK@B8M)V`K!RD>zscS!7gBEX?2lI%{&j3$B0~{h2HkTkuvl8pnhP7N zbHi4d^Jh*1FZ<}ay$<%LK*^e`S!^U@Kun-?*_l2a%!R6eN`OO_7}A?m`tgc%bN2dZ z$=+@`co%N!Wy5DH!J~7nBALV%{dkp2w@faCuAL2KWY(=so^XUDE_h%E-H8TC3-hTeE)ElYwjn_p_gIfl%zk+LhZ zgIr#E&Cd1r4)>aXYaPYPXT;pI(eg!0u}J}^^a&rLvCmhtMXDO;8B0W|)zj-{t|V>Q zE=up0XcQl88hR_E`nt+#a^6vgzat1k*&a}usBx#o7hIm~A+Uq1O2DJ&p8Ae;CDS=S zgjuD(PnBI8$SRiLO?+xFc`djB;ule#ty3+K9KnHQt+%=<>eMp62`j~ z3U>wI=2W-Po}N3@3h$P8lk@u#)-4sWKfz})?)_cuwy?HU-jwDQ*|_uC^wArBL(E@& z7&bnOF=NIm*77BI3=aHETR-?5VF~@_KV*`#BO}JK4He(A5ba#ZZox{7kZRnK>#Wu6KQ)eJJUp+!Qb%`F%dn%!MgeHUHwP{Jq z=`+7uRQ4mng+NV&G{8JGN*C}5FTsCcSXjaxA1@KmFBD+}bWV9a@sP*xKf`A|ddJI~ z%7adNJcUnPrXLYXTDQ)x`0lZHFqPuol#KDg*3sTd^qHuRAbwxq>^%2)y)D&N zXezD2TU6&8dQW>Zarfsfo70bpVJOmYvCQU-6<7uHcw40f)|t}M1zsoGKtiygFOKd3 zUTa$@LJb+WSmUFJ8jf&n^`Y0a3@H=EbZxx zzXEzVf7HqeP=(5*K39~6qY&NpgN5Ub%71?Es;JkB*7C;)2i*;EIGsyrsyAC{7)?y+ zn?oNq%HHj@D^;W|BIZ7lRIadH!7(h$9XIu23cUMh&t59qWSX8F9NCUu3qzd?u1n4Z z_069Z(yl)Ixmb<3mv7&w0D&q;bsZg2f0wF#t}d!OdY?Rjl0IX4^t@KP_Ri1){-p2L z_;r;C#WLk{Pgq)W@09?hA#t6qBb-JNV^PlWT8&AR^vDmqbj}m)OYv#s{JzY8IY5mX}Fs8;KxB@F-vm znkj{1l8b)wzgz}jJ8}QcMqOe2_hmr$p^D*&Nyps}Jejhfe2)dz;)xuUjHC`%3u*Q3 z5M8-gf;h1DA#Bal3Ih3Um=PILY4~HfDt&;04}*~~(-byF$l0T99z{q{c=^cT_}jW# z#R~v|pV~&K>6csu^Mc~NgXA^4g3{`%jl(d<>?dS2tL^SRc9Uaq59;9(vrZ1*aqUfK~ zy|@*8g|J&U&-8s5N)D*odd>4hVNh>iIf@=2;Gy{4*R^tAf=7Js6<^f;gtrjR^2m8u zTl&b7OZ?Hf#-@#Qys)ojJg;>M1$;c5ob9s)QO-B{uR$*D*7NfRl3}L}6`N$Jg#%jH z0&81&x#dxMv>N@JQzkpwCyqW88}UMfx2G5OR~dCiUS|C&_7|SnaFf4glW}mB`!T#6 zO$OH`%(jly5qZ}!t@9XOCJ3c?O_JZc}g`sS8Se{>m6|BYeDqy>1`UIquTZyS092JeIiu_Ci?CA}yP|l%SL2#(%G0xEF?3{?O;Dnn z?I^uXlfY_Z6HS8PqMjrCWmo%~*+7Xmz)bR~%@nK%Jy~YK0Q6mZY-Z0-$e!IdMUg^c z$1_2OpasMTgaW>VRQ|Vy;NnA4va77{n}HS!bh7{I3BbQ}HiV$fkQFxY#II$q(+=>o zgIuf3XP1_tZSc$Vaz8lXH&tv z0w~ohLLEsRam3Ffrzr~>99-BxiMu>oJYF7EkVBcfz&Y5GC+iom_@11dOgkapvEC43 z9X~~MgS*rt)hDTRNvu;-2keIZS>2P4hx#b@jeUdHg$|@#HAPKJI9eEo?@Ypm-HFeA zQkzgh>YD=3szT!NrnT0`G?YAsStOR0+r6-~=Pm$Dkfni04n;Wnf+yu(N0-#$ zv=a^T%kFsTREunVgq~?46XHd#W?D2w{!h`{>E9Tn6+0TF!ybW1iw`oxy5U~OB@U_i zS{Ws?y9v8p=T`nC+*bG|3Zwk&lpLm(Emp>Agvn@H8NJVRDejYlzAn?TYQq6u^NyEC zU7Dn8qVwh?xL=fc&@b%Z`MF8l8P(_Kzz~`2M?97^p}G&Sy$K1=LP&D7b8j6rUUC^e26na3-+w(x&&vtQ0 z)G(~Jaxo@xtyN(%AHV+et7EQk%qzic5`?ZEkx?)O=Nt@qnd;T~;p75wkDX$7(bcgy zRn)+Fp1N|--2)g#!c_=y5T}0p`P`?&0_$(KE*H4qPrWJe*NxEuTh-4?(;l~lCMt^( zT>>0qIplWMSB46M$)Gh(HRo(Z405}@$Pvg3%VRr8&=ro!pjXPup-rn4hK_iZTpy1uYDTFoxR~y5AQ`D02ruV(RdUh0l!UnCay&~20~}gYUgU46b-#V zWJNjgg5+({Xb#xbL8&vqm(cceJvp?u)I8L*YYm}{6mP)S-n#Q&yNAE4Fu(hkZa^fh z8ezKsF2erqAuimd0VU_fVYq_*^C)XZX3&Cz;9@%z#Fx=&Q*>Y+b*M=VgSMWF?=(Vo z7sm}W?{>sI+8TuvtKeLy!%(M?bUp)4%v6r^2rE4_i9~@-D;?zsW~NAsda|`=Rj&K% z3T^x<_l)TLKKZ}DW&}4NQgrL(cO4t0&$}s+7{)fmE0Khkvx7d&mZ<-jcQ!=?O{b*g zI#`oiDSW6$V^qh_M&-OKY>U$wMisui21a_~B?0Y16?d|GunwcXIgdYG9WC4PYSUDk ze*kl{5@)xajR)Mc8{H?_YF8sDn@yF!K1Ocd^%Jhh9w6+W!ZUe9^qv+Vw}<8p&Q*L` zb!0Wg_tgMq5H3pkisxLfnr)lCc(#Jt!*irfH&ZeBHQkr}7`ia4!G&uLJxMSE&R9(q z=AW{hp9Foc%pOOH@nu^?*raZC7kH`)7YLC!fuAn}Zi@ysGaj?ODvbhw6HWwGL#buR z*B;P`9P{QqfG0^FSC;OXN(UoiNc%H$wupwb4mRI0wdxpt^RP};Z+z{?Uy=Pe$@fz$ zsXxfSXqxex>5b(pUHX+GRS=~*QNI8A2E^K+jr3%}E&RyOZX`3dJ0(g^j`g?VW=)-m zsK<9S9bv$mffTx5{yV+1HO=@&A#7X-X}`DUkd7_h2@bDU6EBR?!lDPDFOJExM^*_&pc*5hnq*1wDqCgSL+H5orrpCM2hDoo=?Nux1aej((AHH~@53XPqcQS?>h6bJ=56de z_Xis)>3}mko7oe@l-_KL0kJe37-&Y{=$@+9GuCUFJ+ga|nBMrfsh_>?>in(7Q^0s^ z6lyUz&e_2oOz+M3X^}jS1-&Pw*mvKz8keE9cl*m+wEYJ@j@_{FDptU$a^~qR%q5ve zTFj=1-;jpa{+1I2i?5LZ?l|faJfZ)}C_@{d)3s<(}enTVngv?1=t`Qm05A>r(4-?e68z0>ISM1F;t z`q0irNH((c1zBwIT}hsYRa4BemkCIy`Z!UYaa=CCv{SplWQ3zv>^MIoDc+BMr|Gu1 zIb^|Pxz+MD5Zjg$awAr{vQRqzb+P?jEE5V|z6b0Hzm+8)f9y9AcNgC1OyUAA zQL_M5s6`ALJDv5CL1(BOr?#J?V1vu3F?Aw{*H=Z+7df92?iEO$(#hQD18Fz$*~pW4 zC(aAK!#*LTK!R4-PUPgQZFcFH);lVrL#oT5qo5gfu2CO3mF@SOF!5njnFW?4zV=OS z@4;lHyK)C>8#-s8P?Vtj%KS>f`L8r&xRjb2ZD%Gtz$K3T5s0FV5e}vZsSsm(<~VG- zBxCwA-IPK&={KiaegZp`1Sj_{;E`UWodgv6Jd{DIO{kg&3L z;`X#Zn!Ud}Bt=H%Y!(<;*W4 zxsCPLJ0=Z02(N|pb$+%qut|n(S4`TdTyBnhy@-~3UE*qpR1oZ7VWWpOS|r~6on*Kf zmUxmYs2~Vk8J2+O(FG-sB+sG#4_#Lt5B1u$)uURcO@{m*`N)cbVEsPrsEO(};0CwYU$bn%o2A{tdutu8M< z;OxIsCfnRfoHO?aeG0P_YMG2$xUA959XPAglzF81=!X%{%vxQitX{BFD zn-mx0vl@KxTz={{Bz+96yP{^D@R0 zVgXAh2Ekw+T%C=8uF<|rhe>?u*X53gP4)WKO-SEqUVc=qS2x*BI~=mT)^ldyA~auS z4AtCK3zTm-KttDc-7;h%55FMivEUE(7izE4u*siaSw?2pxry>mBpfG}sBUt8BBTRF znM%GNlkLbwIus{v8O|>As%Ap_;0?Pbu}d-It&0U@!|g|t!A0Os)V`+C7r9tzZq$`Y z*n>oMKt^zr8kO#JLe5 znN%C{oue>{PS5*g@y3$$)VsLeA5%M*)(P@kIdG~Yxr(8Z`qZomX9=^C)46i~m8<52Q0l1KbQ*CO6Kk&PHNJ*nG64`x@P%%06@0MQn7r99?f;5sVn z;LW+=vtKoYhbl%`9-Q^0H&&W^zCokzQ&rYne{=Ias8pY#PG=emc&+@1DE3&C?S$#~ zuXeDBlfVdke7_ZcKiYHQvsEidSTurQB2%nxoWlp1cKdL)KLtn?BtZ71*w_5}T=p7N z+*L6D<}~<8+f?4zdGW_+i+#HaB^Q}HdYST@EBfKM*njNrg@)SfLsaXDli zpS*aO1K02Sc=nJUf0WC$+xu%xY-i;saYAgIlS0ek3?!3DC$m{&F6667seZ1k_(=Uo zwL_18{lBVEswzsD{4C`HuyB1; zh6UHZbx1{5HOjjGNG7xhH%fiSpXIsI4ymyY*fvof*Y;>OQd^{cq|MMZ9h`Xj1NcUJaq8qgPI zd$SB?uRaE!GG;=-ep3iSve!rPFf;}$3Kn01&$qqi4bMYCFM2-=qztk;+C_Nj@^i=f z;rZdo@QLj7Lg%Sr((eah3Rux*48|WcdS$(3;Desh1hVlly7wftngWYQz}v0+dA)c1 z>aSh>`y1^4KdqYhEodNY)Wu=SJ(a>sIz|fH79C1z{OyVlLk2J?<5p^<{hD}&*pxjL z4+F!NOxjFZ>590j`7-zlFPfyCpzL0zI&W<`$2)W=8V&wT8in6Y^6bjRzH4xKoeb`p z{o3%?!Ni1c$hEzne{u2sUDHVlMoD}@^XK;b2a`IZ1DC+k@cXu3(d0jCI1o{m!}Gu> zBS=7l#F6;6P=26qy- zJFoHmn2_CKMD;MviP1cGldZ)bE*fkSznACNgM>3<;0!Ta-DxNN>mh5&zQD8|*9H77 znH?8#V%yEG$gS;LrnV~5F0mu~X-n1!{Ee2m`_zFH8$Z!NlpnY*Lc#_Uw>bu(RWHq1 z<)BGHHso?u1ko!UqHGWtAI$+ZtPPYXJB+>jLq&0%PnlqTJi*A6_4jAyrQUMG?kH-s zwn(Rt$2^ab`(AT>EjyjagP;4cjKOc*s?A7aKPg+^ZXX4+1E@IuBlm`+D(E?6HNpD= z2`JL2Z98`3Pdw4JzuxoQK;+VgC$}Gk{3%pw)9ch-F58Uj^E4h zYq@xxka#iA_H{nu=L%M_`GL_#9dhnq@Qi(%<#8LdEj!q0-7HBdoGj3p8mYS@C}6^! z#fQci%f0H@bINe6h!H}V#IJuM+*gU3Q-c(?*2+UV*q?Ray%xR&jV}_ZSSvUIqi-(O zgcBmcI%|>emFcE(jIhZCFuD!x5pw}UnzaA1ko3qx(!Z|1{0q^}yGRX$QUq8-*TF79 zZX9Nw0t!S&1F-H?)cAI|*OLbfVjQByRH^NclnqkQVk%_v(?*d<{;VYjXYtT;mY3g| z%}xr1d7$(1W91<{qhsg$dK|p$;A?urPKi}lad#p^dsDhW<#fn?c~uPDG_5;=&oXLW zr-rHBZ4gkr785KELJ|NkK~Hfx1=*i`;IS)blxLS@f#p#T35VtrnYviQFC6~YH-1Az zII~0IABv9t3RxiPVKRsMgywA2U6FIb0Ms8cvfbqelUQrP;O^kQqw&~{LAn?c5%)rl z`^MA8eP*4wmKu({c@#Szt;00Msn(A@Wa-vfMb@afIrzw<%SQr3DVX`adsi{!GO(;X zfFuA=A<3nI#v1e)v2rf98Zr_$b^2}}?g@t(j}>B+mloA^Fd=gAe?(~}Buam{GI!=z zs4+kC5V~u{!H2|4v4TE~P2vWT#3c!rZ(8N?oNK*3)yG&Ol)DfID$QVdVpUOEl<2S{ zbJE7OP*FWjV{p1&k{2CNRb*I_*PLt$Ko3)_Q&P>YlR{R{HFlVXV2~(Vh!-||QD$#z zXc)_aKXj?J>R6-h08?3f-yq53p5&#-k6Gt>O{=_qF66I8-omKqjO-le6ODBIKYvOQ z@&n7$?L^5zE{ZsCB3~?BIRSxSXUxkaQ`B|wbvQ9&IqrM(={ylM^&admYN}Oz$Nk2) z43MJoIFYaV?(?MQT2dg@;W5ty9|5I2l! zWgo8R74{P}$V8JUy*``87WfZO^a_GEbz>8BTbia z128i?!!JcQ?l=f}2n>o#4^O@0`1xc%7Zt!7%|>LY8(%kF-uI86;{CAu?AAva`S|r2 z87^LBzN>u`3$P%;+n!zVXGBttXF`&o!NqNB8`wku>AyjLYslyGA5q?`6Ne}Km%zPL z_5e6s6ASTRu-X0B+n08mO<^Hqf`il^UFhJ>K}XD8B>eM&`xWKTffQpD;Yc6M(|-j$ z;Ge-kkXh)=N#LV+WWcKO+vS12fB>rY({}Y+HB_j1!?Q7-i`^~`GXrXFYcthR3xTs^ z19#ve)v>*W0k2YB8E)<#Zg$50>lX$NN&MWx`e7D|T+4HQN?d$IIc|0DD#$)u*A;Hx z_F&pA=bT;yK0KzhrXCYgLWu3~qU<~WDn7fVX)_^W7vQ&2;V{_C28TFKq>Cd`r~m&a z)x~yJJt5 zJr~#5K@@Wo#L4OgI~ig%EV@j(yP2TlXJXpk|q>FOif< z4hM&zPAN7io9G?8MU{n33YeSoF1Ke2fdz7_?agBezA@TRQnD0jR&YHTQo}2@pB9Z8 zDq#5fxxYgb|BWftAS>}o$BXiUW<{?R&F7~x(&b$=@%nHrRf$g6?FxNSUx&k~Q`DKs zO_{O#7&2*dwq-M!+VwDeRg^lr=o#nmdf0x%bC0kkd~l1>-i89vvzZJLdm-EoGPB<; z$s-FIkoXxXdei`Ee&nu#vn^fVHgI!aHaeokgx%1ef38Vs(;COEXng1yq_1blhG6V* zqDtrw?={@_HdP?y(C)6Cw$wZK9*PEm^fLQ}Tg(C|bMbSSNo_F7W|rn%(cXL5HX7U! z&vK3RmX^YF`Lya+DLVL)w$q|agTFs`p7(mejy;l^PmU&}iGan^tJ$}#=g{7X5)sO} z7-OZC{FalWFY3)MowT+bB)!XEc@PnF=F}nk18(88?Ez=DdMeK5hle}}KHRvL>C;_y z6`RR{3m%u>)POs*4^sLwzIib1M&kpb!eGI#co=+3t=CP)A`Xo#1WswYva!ujxyfd3 z(v`ShS^Pi0@dReN zP;F;_b39_WFvB`91Q^~&k-PdaM3U;k$aenYhje=&)<*XVWUM!6tbX}$9Fl!&df<1dh=1s6T)xx% zseB{DNYMC+OWQup-EPbrGxfCzX%fr~k_Ox2KFX-kQKANk#$Uh`?YTbRPTjs(8N?vh znPpmBfYi~qSf5h0ZQ-YgFS+JI^YV(xYc>{ktQEw|HMtT(#h>2$62CGb`I_*-o!r!5 zCW$NhUT#pOX*PE5JT&O-2jA>zxjIP1wBa5_`B$w@7V3Gg&1jT5tEhVbcY{#ve zYe&)!c)&79{=E#*nEC`cgJ7)vPK4u!#nb_7HR5vQfY*2MqlgM&lX&N$^d0|=5o$Lr zEWt*DbmIKUZS@}L@E{hkxZ(59^mVsD9(1yS-(JBEzG4f@8U4=C5WIbi4MHiF?vm!0L1EEws`MUQHH9RkeFm%S}+z;6r|A+YL@vKrC+yF?~GG?Mh&Y1>d;V!xQzmEA~ z$RoTQ%`Rbau`-TNByGDo_8dcUL2!jWt(e9^ZmPK_^*lxILC-a*?Awar`$<271++LR zQ#rfpVGMVGo3r#AeSZBy=c_8{eNl-XlQL`)mLavEq2R?bpyWeAd3Az?&~K+o-i;gc z*Ni|VR812Y-0T!mr5?6znF0qtPtIw#nvtZM+pqUB-$2hBp4@yaBHTa5z=WxyL0a2v zSV^nBm=065WU797UGEq@F*zLogu|+YfsEE28FPcktK9kPRhC68 z!pFkptsNupu9K*J8N}!}u2Cv_Z#&0bnS)0-r}_&M?srMKc75|yS3F9M03`2Z8}h~F zW&VROoO4mv@8vq&c%^x;a$;|y3I_M#7-iIl(T!K1dWmWRj6~3A$8rt*bF!Q5E1Q84QiFrU1!+L;$}|SKq5A9 zr3(fHMFBoykP9$pyxGp1H~!k8s9y&aLXsLb*{!e4FUea#(L9b@K+;Q!!romU*Ys-k%w2A{WPUpi4;!|6G#B1F`U6My_D z{!b+v-Y*qu*xgcGoI9zh5hx5f?QNQUride>ppT#&?9tY9IRjizB5Y4l5Z{XB!7!yC zTogWl(w=wk)qyA`=4KzLFKqV-5{#&mnZ`!8g%gLy#dELq3ZL^{ethN2nl%N1MaV954I@`Bth8@dNeu4`+2j zLNp6(h7Jq_-Fmg|IcJINMt>035WI&Nm$;SKdmt@qFEYysm(wzBsE{1SXW?QlU@ zuuNq^)jeW``_cP{GJ2{vMcdSZS-y=|DI7~T6jK@fGVs<*?zC~9or>b-nq#>V>Y>TU z>XmZ$>h&r(64(M1ls|kAyjPxua4DH8S*R{*OV8%IOrLwM-@Rvb7s6xRx_tyKc((g(0Hh_6AXr~cOQyLFqa9nF@EyUzVElc`dwt%(WFmk;F&x1D_uvpZ+;F%z z8eUiLIL!$B=rN$tr(f^>04%DL=57kl`eX1u)}oW8c8Xc~djDDq%xXBZfAskgH2=)w zMd0;^?D4No`RDCu?-x|*qKs2(`0TJ(EaBCVzE~*w!N9}s?%Uzc6-WTtsg1~%fvKYW zwZb7}DxBj8iv*DzATxb93tZstg43whR8KL@D*xO0g0S~80nNMHLr*?C&S#VhlUO!~ zqV!T!gXkT=J|*GsWUX;;ACs z!K*qwV&U6|>T#~%^nflcXB6~9OKk534=2pj>^q)bH=m@Y z{uin^7ty4#*hCYhr`;5^*I9!ERc7~8sJ(r;tDawJlya4&p6fY37X#>hf5Ymiq=~5q zZ$1;sSEO|9mGe_auDLrweGLAEqo$fLpapAwn@vLfp44T2V7BKzrL)+?Wu^D~ zE_FFJ-84D>Z7!vTh~_C>850Z&Z!uVhLt6v>$etYAQx5_%=-fUSIjBXAD1c|V0XGme%C{f9v60Ma>*L2O zz0JJ1%nU0a-0o%E6lX%AyiAmLJD!yTW8G4H=4SKmBzGm$J5mW#4CkCjPP&*xUF$B$ zVd=dmk0`5>RH(R6yGKrjfVG`cTk>M_ZUqgtUXcB*w~kSp#X%|7hWkgs;Tu$TTY^)= zdkC)3x0D+wYJT6a`~Ci--qlVJB%l~faOq*QJDVQsiUM@-vC`6;u)a8}#0WqQI)n4{ z{ssH}ya!bQWO|3){d}MN7SD@2a+R3U+C!+QMJPFaDNjuH*85q%!9IYKp+{FJe6Hjm z(6}!eePf%@E9u0Xty`)_TOLiQb)K5}Z1rFlxmF63TmsGI289I|Ya`X@m~oaJ!berR zN2~Om3EakXxY;~Nzn<)^W?A3OS`I45_b2oe_<*d*9vf;*__F;Xok1QvA`dzjHS}lQ1ILodEdE64pamH5|48)!!`?&nPxqR@VrO?WPhPqC=8bLl z0S5VLUv@d?dPyO`+M1JJf#dDB7gatR%TL!}g~cu2?a+xnoo8qE`PIeX6rGqxS&>6> zGcVtKPc|ago4z=&qO5uu@Iu{KQI+~0O!Y7zr1OM46~U{USrP` zDpP=99y`WIT&L9_k*(CrkBsm;g}cqZh!@T0bXA&OE*p7dUotd@&sSvNZnQmFDm8kJ z0VXfRlAj7@I_UCZrMAe+5tOS|1zd<1wtH&UGw(r7`$=%$>|kPFn&3ix3A6tNvi`H& z!`s2;*w%PUiYcP()y4NiEgD=oWb1{bF8gW@Sz`+zlQ=>h7GyUTd{(~g6Y`X(#&z!M z_j?2ueI~77EiW5~O5#Kv1CT{rt&Jr~v2Vo+byAwjKEeL^>jUbO9f1_P&jB4{)BTmy z_B{_(0ubGyEcpjqwJ2^$N^3Zt5lXjs`V;rT)7*sh=#5khz3MP7R?p#J{TP0sGufJQ zlW|3#i7(LhF*su)Glo`;eBb^l@Eh8J+jpTRcvSk(5pjh+j+_;zDy6WPu6JQvA^_LT z3mjYr81jewUX=r;vn5dbKY3<&ZP4K=+{8y-iw~Se55t@0YY7~@VJ8ZfI1VsV$%_Ml z5D^}(O-dy-4m9%$S^LQp`s5QHqZd!G+4|dclOM7W)y@gL8RxwEwmLpPkTOb3cs2qg z@|4HWkrk+~A0JJSJ#&E~o<0cq>A2mn;W&237n(HmC3-8h)%QbS9x?jjvlveF{zg6@ z05-@l%sD7L(p3!ea!)Iyt3u$s$duQcA7uhM z7KPaw9{(3bCFIa2^lwsMd#}55)a5JtQ6uVH*w>eyYN2z&@ju9A%?W*@v?Ch4=v;id zM1_Wn$q+CGX(Lpt*h-b?QJv|F*I1o;-N%K#^%ToFuk)in;HA|NZppczQm6Oq5&Qf% z_)zk61Mm031blx2aRK+Mls002B@+kC3KkR$zpvz5)vC*(EZ0Aja%9D0$Td+WNG z7fudd{_DtknhO!h<-T-g1k7lCg2Bz!Y;>wy3%dV^ExZc#77rau7`7mI=> zhe4NweP8oB4r9oBwoFh`IT+L`K7e1@NjG^Gjfm44bw%py!pWI+hFtk?K3c2DM`_ohzG%s@g5Rbes4;udz9B}8DIcbnOJ<$s9tu73l+KsG(4c2Ll`Xxd?4E*59& ziq^TW;Nk4@>1-X(LGg-@+yvAlBwodx0n*cO!L$rx@f8VTY1tOl^g(5Ggy^Y)ap7tK zY!ngo#%0RX#I(fIO(>BZaQ-?bDXTrx>~zju9io<0Knz6>vwh;hZGClNzs#cHc8yQ@c#)1-Q1$ISK!i_((k z23a;SM|hIoicxrR+rONvP}*~SqVtn%NOKc<8H&A|YGrE{ELETEW-oAqNK(OXyxP`V z6L(iO3|#x5{qZ+7))!Q^a~N=VZI!F7JdzTD6mNaLadSWvm?gTf6BwKj%m6VijmIizY`V5qYwbH@d_K{!s=s)2=di zyL#M45){X))WfB&c1lhhvfe}flF+@SDSEY&cz#<;g4X=lrw{Zu_^-XOB@6(XG>$zM z=bl>-_?%suTLt9`kf%uXTOo)6(2G1gsvRVJZ!9zjD%XAEE~%y1%`b#>Z9K?vv!q z&1T~y4OP!uIMxj!1x1s=_|u@655iMn&G|~U@!hFD*OY&31EJ*Qdxb213`%SIr5@!| zF$-h{{EXtZ2=-V|&L~cMLC(XmbrOKDqN2-A*DL^p)6buf|u zWiFK00?i3u@reOC$#QSi!!NQ{Bo#yUDq9seso3R4)=>nKK=_8;cIpEzujp_DY?agJ zw_cMgFB?>wZ@g&bCwzB6)bXdU5nhq<%u*1BSYVCBLB5}P{PA{YCr!(c?sfl#C_TlN z{?|@?^Ged}+*h3NmjDSz3SE<3W^uQp&f`bkr8H)w3wcU0uxid#6hN|aFF0#sf~ZG+ zrD)gZ+kPgHj*ne%0>gdWGwz&W{>gWeZ~r@M~XUfxwJNwGaX-o$P?plse;;)1-yaPR6z@hy4F4e>H3g1_QZU+07RTgPPj$oo^akS4)(5@vV=LXZIhB?(UTye!X)ul*E_m_ z2{jr94B_>{qc1AbJ)0Y=dIsQDdB@p)2XBhw^$B$+`9qh-A$QqDxvfjw&sATn&`)!? zLYn8h7$YfD{l}%Opmn(LYxC>}+23M&;CXbKs^Q%UfW5B+`(n(^h60V$xuturJJ2`? zKDL1x6>H}Tof~?G#~gAWS%7^2s5iupB~}LTVjrNU4+d`CS|QFOFL9y^S2jl#T8T{? z%tsTXs`_=cV#;zJox4O!8YEq0TYAPJ?P4^{pQUdda2Vt@ZtlC?EkNCu+>P2vUqq!cm`AEhZJacH8DK?`dEJzms_{-bOW-3=CO)i z?lK@ZDEz#C4jW%q+>!#iCsz#bz;TKzH2xVu%6g@z$Oy@}VKQ$^<_+|v+cD3bBOZxB zn!ny>2ROI8!VqwSV>9jPMjwp*dy+XTaJiR0ymO!??Y=C%n9oEU;yyWH*q;UB8in4QCf`q=4ZXK{EGGYcHo z>@IM63~-10`j@c$MT^&ygN}{+N!OdObG3m~iJJhi9uA@ zUcO?bfzj)Gx2(iB(7HjMidFd4uS=Hqn?87o_u$3G@KSnH&8xp6*-efICFUWTNi%|PXuH{KV;OW_;l*o~S>N_6;{Ev*wWV{rS zoa+?y@;0n<|4qWADlv9=ebr)tp?7JFC5ih8(fS+YYt0wt7fuIKTyJjA%>X=LdnOD6 zQ;vkCZ496=CCl=tlh}+I!J^N$7BHvpUHK=xm-=ZDpvzA4DG1z7DXvXd*2Ft*)p!da z^EeSbE!x;Y!RSQqC!kyIhYK|4(3tTPHB{GbA~tuQbFV!xjj8F)vsVo|tG=Y4VwLFj z&ODiJZH*zhn4o$?n*3%mjX~i@xaV{^twS}3OCG~F&nsNur8mUn!d7gJp8PK{$*vO-H}^m>^P0 zPuyzu%t71m*1=o26Apa7i#%9@ZEs1p-$PI}cGA3b7_JL-up|a3SBG*GD-NllB0;z= zBPs0r=$F>u?@<`A@P1f8?ak48oI#vYe%p5LsR1B3Lz|9Om^rWD^NA}cBhaldlikz+ zp$YXzN9?sGg0j&=3_~9X^@6xjaq%t!cP55g*~N8C@=~j{89VCU61eo*E<;|Jqg;7? zQjbCz#GX@J=&+at3hDlsV(#cu(Avwj17e1t&btd+x%S=dy~Sy&*W9It5sb0Sn^|FZ++f0Tj`<3}|t5;n& zc5m^Fw)Ei&=?ALf3^wV;%|Fq4_zfh1+zk1w%8lDCdp#85ApAvU=j9Lm;Ne}ZFdDQn8)S$lknTz*|9W-u6CZJL-!YqvTPf8d!QmR zmiZ1A7kV7BJe;ve#fi#tEHx42-NSW!cKb+6?}4CD7d$|+W?nU@0XFCZeXu2adCATT zSNrz>smKf7eAyNLl?sgY`166OOw{tcb`k@pO|X@l~^8Lc7Pp3iJBE(dg%hKu`lgg*Uj?opsI%sDDzYSH%@4+w`Et9#xrJB2VQJkxXmpUk z`E|Q+1j2h3pDCKE9_bVh2nx6WDcTSMqr+%nF4p=ix0wVU4f@ZeKBZ;F;=+u?2Egp~ z4$~xJ!wOV`s{hKYj!|~AN;Dn%;|KJ_Nqa86a8_9xd!EG}gnNdfCaroXYS38vCsHLx zD3cKg?~KJ;j~Bpo)xhz*@;}Hmycv>giN=4{IfzkSL$$C}opkj(3E-^nnS1hXukZ&m zw6L3qzr%%$jI-gNUR&w09l8?h)YVSk^%e1-e`_I~2;OiYYzhNhy4t`=gY8|I;*k}= zE@o-|2v^87(TMFc@v_{h-?LzhsfXJt(ez zmq6xf>`JIIt_A3v7wWqRV1whhZB;|c;0d$Q_Tcu?Rx{JHb1Hm!QH~Dd@`u-gS3yLL z5A!}7>0*?9#VqT0d+q6h)L@(+l+>Z(%+ry4I;e^)xC2J#rm4q(a|(w`!NZ9`EVg(J zVenSp3uZBK)AdQyj>trfwP$4oS{@7B?>$w#$lf2cSIQ=(^2fbH-@icWN3g66J{9G8 zb9;l>6@zp`0y3ucE#~4z<1>(JgLe(*sp~|;ubZo-n;-mT$pJSC2^H-Z8t$F4EnRmO zFwT3yyk8%$&H6rSpPtPEgs$eR$qA5K6PpcAVjAxnGQkCK2kJVtQ&M#nGsSPdCu{=$ zcSirRNToN(4G^B=-ksH0xv+dcU$FxtBx!kIEQ|d&!V`Tq+~O2G-V_QbG`%!hoRP9( z=5{}VqtjGwOjHm=nrxgG8i4CUAbD_+IU+uzOch!I=vi~I2P6zw|)2*R%i!x+=m_T!AmjIC<>b92%20|G=$7PlgZ-O$5(hEszb5v-@Q z{b|Lp`hKz4!FZ6>8qNExPUNC@xyuZ|rF1&(vm2x#93yvx&K@FSIC-KSA3jUY)wX7x zfyI&=J5X+31tu%(^WWL{DA{9zhl#6%(pBv$tAv1YZxG*jwH13VJCGR3|!5#Y*&i1`QIKe;sBR>*>e3=Y}rlqI0cJ>T4{0K8hJvnn>4=OWob@Q17w;2F5O zm*T|Yli7{OD%i8!{g`>HIvv6WOI9aezq=|eAojfQUQ?07T!?;BEze2H3fkkFRl_E!1mcFe8iA`gr5;@ zNeyHJr3Rcf=QU(z)c5BGo#%AW424AWD$b?LKIrW-6g9W?lZuJ6XQZwFoe6AV@B?E& zwbWGJwyEW>Q>TF~HAM9=hz1_oQgi}1$2S(<@9zk*CMhP&boKangq;p*CM;&RDA^aWDGr&?v6Vwem@XEA-z7sR}LlU|<2ksHh(N+5}U z-Jgb1hcOM|Ko3->-FTsbj=1f#o%JYt!2KZ*0$%;(25B4V=C+I?k9}{D_Z1RKnfz>) zh>}P3QSvMs_Do3A83cij5O;)~$v4;m(J`GaXQzMn-zOqTs}JHPi*!OEllN4v2BSTf zm7*jX|4I{h$?Mq+5(Cgs@*Z~gV<5i&-1dg|BXL7!%k5{CE5`G%OEHU8dB!Kz@cRNr|HITu6vy$k|>+}dWi?S3tifimFLy+m5b0~ zgP>(YjSMu+3|2dF4(O0IUhW$nOAWKC2`@LjG9g&4Rr=IIMLz<9p#0$4O5bRssSPmf`P`yJ}|$e`{kv7?|MI1{0k5f3Q3-9 zIl`U+Y0HGT>-2fIrT!p};d(OXmSJlc-?0!@63i?B+7N)@8H(u!D1Y4i$jBzlQJdq% zyp--fG!3h~bgfCLcenjByBmU_c29lrZsqNx?fu`SIZs)FNT4;{Na7elp8eCb_;Zag z&Xw5hN4`s2Hmv~?i^y0fr1PMa!%rE9Nq$igNEf`Ffg7&gPv&ybYBA=5BvK*BrCX9! zq*V%-$U$~epu>2~m-fB8Fqh>xk@Ag&n@CQrXfveFgN%85$-%H&G%MuPt-_HYy34uo zuwrVv@zNR}n(dUKWjURN^kJf}g&uaM{1on-g6b8v7y-rhyKGOl3kkUj*t7|;^Aj>} zYn^#<-miqV%nE9f&=Z8&s!m$f^askV8JVkj$c?@=PN)sUC7_`LI3>%t#M5Zn@xg!9 z4)|+WX_u*Gxd44qJb|+wF7cC}yf_2qI7*rW^L7$c zu=k|J7*}AlQK4nDpn2mniqka_8hiN*DH7O)jKKdvGAtf^rnnhe{}yS&0>u* zViH+faMJ@{#jLorboEIYI!?mUTid~GxwSZ9H2OS96Fe?2G^>vlH1 z$MWsIUq_!)Y2E97qsxfz(Td~QI%ch@0avx(Fuh$c3d=@AT?OTiTMl^^gaPi*_acc& zKa-;knkeIZa}#Bdb!4N(UUD}ZZvCUmINVnc9)G2(f(XAHy9C>H_~yRE7w5BWyU##5 z&3YBx2Oq~Y(Q{Ty#$zE}F&EM?aktqE&lA9~5rFrJ8`c%ok~SY}M&8d3Rgv4Z5iUJR zyRS3n@+`^gKhFF{A&eoQf1#<{i+cx;xKb_{)UBB6PNK<-9zyvOwTFZuA|4L#> zkTuAG#uzlukhRQWZ{h~OPDWiUM@8{Ohf10F#jVFfs%BRKyTUi*(kix*1*Ja0#G zZJ)j-z60GSIH$qsMncqxm#qWoSpT{ykjvF%9iWuQm3@fNcNP>b@4?3SAmS~2ii==h z0tyMO2yVB673I9o&><0A4OkgBY69TQH=Meno9gdx6_6>M+Vw;dedWf!@KTXzas&S+KS6 zYG-zYX|a2La2$gfHKM)hdt~WRvHBLuC(^2Y_B#9IA4U;SVR+?{Qan+{gV!&d8?n17 ze_sNEz<1VNW@cuyA5Z9ADpvlTPOvN zdO>5jr8MRlHN51V3%8E$?+Ptkk=c*vQQiLh8tO z=zsM|Fp$Es{kJTG*O3}&@_{*B&a&G=>|YlGzu+j*_eaCQ-fX9cpe2-+*om` zZ|k4ivy`dJwV<=0FP=5@7Rvut8Q7Uzgn=RA&fg7q;x5kF=H}-LAsE#AyEA@fpHn4j1l45R+r1{f9p7st{iUCCA2M z>E<`OO%ST(=qp7F7|ULDB%M4K_j*??&$HI2^us#azrEz@dDXth=g{yCO$`zq?e9$+ zDd1NdweLNU^+lPSTxbHbwUsycuD*3-|ivNT?yd)I9yZ6jh z9g9?M#v&Q^1;}Jyfl65nv5I8QsYvvZ*lS=i^-$GKCU+FgcII&C z^V~OJ-5~{f`QtoPgaPP-;jzn9t8m(D(pGz9)O1FT~@c)gI+h^9h}K4 zQo3@U$K}!BxYstFUqRbkgK8=sk+un*PPdcvrR>Gx2c}eXwSvo_F_2fmW8u2-M?eNe zpql@yx9N8f`*mcoUC4QzT<#zJ^{;A{iEQCfCokE7Zg5A#_ZVh|C2lH>s({jKx%3x> zuCU`K%r$bCZn-_}5Snyl|NN?2Q#l9Sau(#i{|qPiTE^JJ*AgL-)M$L5G?n)b6xY@` z^NqD-B==GD11MP=ZhH>a);xO)xABbJKey&lVNgoQ&Fm8@jl{~o|4i#3DoqFIsX9g; zjlnW3@z}xi4$iLWFve1~{SV%X&Pa1VIriyxUg+7dQBgB#_R~aC{noEM9uc0hC*PFj zTgdzl0=!bxFjNVqL7K<-c065KKvXl#|KhW3zm+3Kc7U| z9J>U9fE={_lh2 zU$FL*bX}9xxwrkOH@e}VSsKTw_zY`Pzuo@{S^wE7k+OaO$k)JmC>DPGcdIAT3AB@X zs%S2>XtBp5H27O{Qo!}`wuV1<6oxBXBa>XRzXm`0*RyV+4r1WC8^q;pc>Ia303Z~1 zJtp(_jjCsG;0}Ydp$5F+Kj2#WG^aD1Ng>~N*;0U0U1ENr`t##L+a!$2-ZtS+$x!G9 zZ4dzW0A(zgPwpd7msCB4aS2bZfA0{(nDB`N{viX3+y@&jYNTi`?Vp+WK!h^TKUEuH zX@K(G0UWQ|B24UBrw(Nln$S8W8@>3adS>&g*;=5XYU`jXA-VtUFD1Y7@qMCZ6(}~q zvfD3~tj)fMZn^`f4rEyz)ZBvtta=~pFGpitP+2}k5HRsSLz=`W$jp*tuCYVN`IWgM zow!g~7EM>sy}H~ktWW_JU@cT|0k+wu#Hxw|k`l1Zt|mgWmE@1_u|Gd8-VY7gyg?xz zN|M}A&exU=X*3iBK>}(uMeQL}46^`I$_d+m^cpE8{?{Fmg!B?OC&-1y|NNJIQPkYf zfXre9L!Lpd%J*<3{rWfx$OWZu(2DZ4#Qg2{+Bk`kjKh1Xg_J3-)`LonI2-})%E-t1 z!deN6x9bFbN=5q*`N^>NyfSFo`ZXXaZqW7RYnfj!m$XmRs*xQ5>x_$BTg2`nJWBJgg=`(eHO>85M@4Mn&)=P+*Eh zdI<0ndk$UMeaeY5*n%3`hXB`oRb~I&{zJSoG9~u0%twLe+Z*D=O>BgSk_x+zanEs5 z&(sdqw0W8POIr!WYrJ{}?Q*~4>=+6j?5PPBBv>O(0s-50MuK(&$s8}?tiQa;fP1;d z{us!zKMK$OFHD=52#;*D-5}!(AA+(EH9b%o+853G^LHfa_CecbB+xY`o0#Eo^xckOjZsO`1| z(G4KjRr~QAn^P112#6t57pLWpI+JZpK+>bZiXNX$FE<#k$MMIWYDkTCQrDVND0X-# zY}mPzq!RZAOFjR-yU1Fa_dTL4K^~l_ls;aKG>{AVUtzvlpMMqS+zyygZ!GBtZ@JN5Nm(JS{nw$!O>{ zI+~}!& zBL$zg{%QC?pzS=pI7Xre_+P?CD$Dr6p&K##7 zB60}E;sR?Dq3tuNxAe>VKg)Z=v(dbl2?PC=K{O)ekQQWFf$aVnh)_PHNL6C4OCgeL zSkdaSouQ{;p>k)tZnF7-Zrl{6a$HWh4zuZM6BP>YEan7=+}?NO9xuqs|L&+ugZ^!8 zq%92XBLVzpEufTx(h|HdWuPz7`cv8+r%QIlIlX7w{HRF|d>0L)$n_6OTt- ztk@#}tJVA7wZn`9Km11ncaURVgq&%m89FpXCVjG}o#@|d;-6uWo=$cJ_mfDZcYVW4 zdPSy0$eK-1VjBl3vDo!5t7RI@g^N>BbF7Uz3kN?BEl|^Z51QX737{RSdKMKa=Cw`W zOe%#Oq(R^H{K=;O^Rx#(pL(|(O;qID*A##EL|XCmv|a5zzULQ$*P=wOnPS4!?6xJESE8+9KuM!+O5V`a$WS zmm`%4RngH9l0{B}&YP8-lct~KpH=DozN>vI*PM!W4TFO^(GeM>@V&QG$@etLhsxJp z;YYKo>HOx>AVWU1zX=frL06Tc60ps5-Vh%1HkGl(i?)9BnPd`X{n(Z_GYQ$)nXm-y zwI?61X}iYEFBrpIt6|sFZB)vbwk`_rvE#VZN97b4h~7Q-_g zeR9T*zcdc?tq!`}^=d-I==G!lpL5Z}FVl;q-;G{lIz>+E=jW$sW9-*QHRy0W9`$FW zvo4%Sdi9SMM?5pSFCZJ#a5Yf)PVE*}ObetNfUt**ZMnp#oU)aIomt!)HGLFDC&Gtm zVz&Qv4*#+Cyy0XF?^(hCkRsVH#t_2W9ULpt1C6y@GY98eK~!_LNDRkq+wnpLxdQCv z@a81+wOdNqLrdDOm?eQ33(Dgp6 zc;O!C&&#tOJ-lDKy3$>35xK}msxrhAf84>o`@9x1@5Fxej&yGVqQn!F^aAP_H2{GH zrdjl*xi={+jy}l!Dht~D*%3wEU zyEd*T?WxN1yCs@Mr)`df=2Etka8eb**z3ig_^qx zHq(cKGC(+?{~ikMjQ`P5&3l!)_B2KMLJf93a>s|@@EDjZqxnMARm)xrSlZmzdw@J2 zped^VQsnkp$`@+G@vP>{A5)VOs`1+Zm5E`QQ~oiBP_j;fr_y+VL4FN0kfB1;g!=qC zE|B6Z-o?O2ZA&mUr^8+Tl=iq=b6u>f&?W0iXSUX+9~Hx5l3cymH{20i1^@kERS3V+WS5b8 zDEp;XZ7c&eKQ1yR9&-n?znqwwr`h$y2zT~6Y-oKcYE+0%LIke{4FsO zopb7eWR-_^b*9hI`)hZNV!D#;nY3aKC5)|GZazJ^dw8=U?&7Cs2CK*NRTZp~_x9$` zjSPJVoEwT9=X%$oU&k!KtMm1s!*c0+P_nNVsU^q-Gv9H~8BZ$~t#%#Y@fpf0UU64e zI@>SrumAa(!cw_}O%o`w(@g0&FpRs!ky(U5*l|eMJwgdV%#@PMK`E>50i6W~y zP*FE?(~=vE-du_Mm}&CG>Jz9nx06n20&GXBs=(s=SO)msV8x!S^TZz;e7D)u<5s41 zEM;o*;;s&pib%&sO~%@X6m3SVntl`6Ty|dmjzogY+NTp!JFE&GB%dcTZ`rl0mW@}Z zk?n4LOiGH6g4EgKPYPyMmR4Kq=T^)L3kwPgR=#{vuxR?w3fXG%3a=iRGx?E=-OdWCba$G|y}b<~{agk<-v z_1O@a*B5${d*|u5^RY%2+LknY*FEC3dN#-Qc1WW1(csrJg9{vodk&~=XSHv$x)R<} z8uj#OT{D+P&1A7h{3on%o57{5{?ypxl4?SP*;+tsE;QVj7k8XIXfS3;VUo)(Bne+R z@2%Wm-P>;c{jN(!*OgZn1aaJ;=qg$J75l!Kyzotw!%raFlkYQ~! zp>_pp6Oz1@q+iSifjA^BN|OGS&Hk!^VL&JW8$+yu(t)mPKo?*Y=Rpy(T*X9BGtCgE78g$V3-kI|pYCaK_K%dbQ2N$?TI}oKJ zY$xZ_D$?k7Rp}R&EOrgPC(MB`3pnisP{>e(24~AS(jK~GPnXX2S+B1-NtN*S`{&-R zlZ1U37^mddoHWb{o007<263IMlEVY zH^p7`@#D#nrRj7WH>pUu_V)|;2OvRFqP7^T9ftR>4T{r4C`ug_CsMm^b6S>vB|FuMcj zQWI_&#rp9iV~`Rh{+HHsXcx6zh7Y8V^Exk_AN0A?vXb%$;sZftKmvf13k(o)reE== z6K;lC%BGRdhX($L{FKS4;e{dDGV6S45u_M_WOdH>WS=(L2(#}9q`VB|48^t_O$aFl zqt&QP$p)i|mLrF|5ur;<1Bg~m-Rn}!MOU4n#M~#6M8Sgj{u+oF=o_K182h22rKNQ= zPJwBB@bFcHEnf-Ze;pz@1=5~F!D4Y0`Mc@gQRF7`D&1g`+U?gkC2AyDhQ_He2MU&; z4z+FSIR%#ZvnbC{%J?Uh8se!NwlP!7(n&k(lB+{m$8xw>*q@9>o#UY_cZ`piM{AZhyC-?(MO(>hoj)_)GkXmum$vZhC|h=Aq6Gq;A3xI6y11GE^9E@yi$Gv` zP~Ziik+lnPH5ItBEOHss2+VVUHoHctX5u{?3PL)7noOsK>p!5Fe`7L?R9Gq2O!1Tj zP^6Ouc%p}CK3R!@GZsKlPy#$D8zThzil=06ZG_gD2=cHjHrqKX`ML#GU_5~cUfCyV z{{YPCHQH8}?GeV2?kH>($VAyNBO&Spx`I0(n~;FA`nssY4Zx@L^lvwA+WYlQ?FR%!9I&lc88op{ zMY)1!WHdl5?3OAg+#+cN%%tJ34U#!sn!U*whq;RGohp#3n7mgxi&_&&E-RlB>vnUV zeA;y`cssjacdiPjA!i0V>h?NdSTPXmt>A^Iv|nkzGm)l#>tiBojCk(hmFCB+z877pj8fic z2}F^u`eBPtYyP_!$Q{D&%un`dN`2<;wqD7tdyYe&-k5W~5dHYwRD7D_`}glXLL#q; zD41de!u&U4_tyK>^SSiVO!E7({v9_A>wU6I)pPF(#2tIC+AxM2Qd^x|bGP|^s`xd@ zNc)o))5^r`^Usx!#jf=Gr|;G3Ip1roC)e&^d}ERQN}^NaY;}Iz>cq(i+P(sv*nPAXRc{^huMAcF!G9|+6 z0F-}O?HHg<3EM5TsVo=?RSUFhJnRz$EtP(VErSk1@B^jMd81nBZo*Z4&ySWDfzpno zY)tp^EH!}^F;a|zdEH&>7?7}pnb)wo8{WXkv*;3Vl4P3oij1<)(lu4B`KlCd6KY^> z$a*KT)>Q_`%YKm25ZKfS%(w)!Zx)umwC^&_nP5)Ac?)J{NglxEIBY~iyHa|~h`{}E z!pf~zcoV0pIN--R&gNVa%wq!iYJCWuTHc8x1F~&5V<6^kGf?KXq6WZ6T($dhqT;Z~ zP`k_Z@R1tepRu_|_RO|Bk-gZX@pC)LW;4_*lx@LFvNZuqEcaq0PlxkNB6xdI9iRE0 zWnuD1g5TAY2ddcV?UbG8JVn^!v4JS;-B>n@nr6M?4$!F>?Q_)RI-wmNJiT7(`ebRs zP)L*H{I_3GG!=ZxRmc46*+;WVr8c$1^smp}Po?BX(eYeYzY}==GEEY=EHvL*axc0s zlG0{B%`IBh8f*-CB7IlCC^}*w_m}@{Xu8rK!#C@VvG6Zn#dduk zXl$QMz@N?VPFu<#5^4@RSmxP_Ex>XQA<)B}Xxm=a;@rPQCj>d5k5MfE5*@o^{RLaHx@O@$Yca^k=yXvM6-MniLD!1+!ZL|=%Fv}~9tF+BPg$Zfb% zrWyxsShON?Q*faW`;VT{dR=%&6f7wRwUMX$v@8 zGyl_nS3tr-$%Sc4D8wFWKep*TC0PM{kk4@nGn_;-O&Lp2CkQ0h)%#>ggVZ=QXfmwn!_%V5V$AgDtU0Osa4J@p2 zyCDeQKj{nz*@h9CUuhrG;GX~VAy~}&kmh`ni4#m(`|S>fVj2GdnU=gMM_O4W)|97( z{=N-@@^wy-BC^AvXEA;RaJlyK5kPvckCPbp2;pJ z{_N~ms;UXzH=;YyYJ%RB=ei(6%evW zvSCeoKs?2NOsW5T9dyVJjuJZS6O*`evdlb&6ldR8NcAIzvQi1*iE|*I0SD!McvX-V zB+p>`2O-y8bpmR-guPicz=N|@11c;fe}sjZ0HMg-H?ieVSl*|Y>mzj4tY3&sI9cJLKPVv5pTpUv``%+-Y!eOv`J#>#L88zW&5F7} z?YfZu3C82uH^RIc*53+72H36{Ne_?`H%Xw~`@r~(13{Alu~=8U(1|>^C2=U=g|cVRKr6`Yqm0kn1bIJ>qt!{ra*v4c?TW$Fr_P;6CC!Be8P!QX5ccCD-hy~ z1haquF-)-Nx1m*}i)&iVX3g5EcgykN22;cBUa2A=N&Cpeu-$y0mmmT|6*|C!;F)jO z6konG^N2HWt*0&|@Px1aS;7{34094dW9uJ=v1M`tG`u*8SxxGuIr)j+bS>thY1yMN zr#yc+wkYoLi@bYdl7;8|6lE0Iu)nKMRT`)qf?54}R z!V~{7PWb4L&!Qj4t7$-6jW0|Pv!b%tG9@3b1jQMr`RyB?sddFZe75{ahKw+6IZ}zF@(s3@WrCD(CkOm7;LtpxVO#(;T z=q#wcVwA`eUFPN*ngt?h|npZg_lC`Etb; zMSS-Q-htGM6r3rfpaE)cjvoqsNN$I$_bm!;SOIfj=im!$k!UHuw0#r(ilby+diH!9 zpdKrxavv_a5~JgT=JIk=-6DcrUn0u+Dt45dr=F^r(+QXrwuaeU2^43P|KjrXy-Y}I z)l_uq7y&vT%6XZp-5V@c9~lKLLgc=YdQIBjicZQHeQ`)40fW1 zCO0Si#9f)SxNrfIki4bRb~F9VA=-C$5Z znEm-5l6~KA32l&N{DE@*u#v3g9-)u>KT6xUnT32gm}xW-M;vS{SBtItwJd}ufGW7 z|5OgbPyX*L-kdKXx_2$)VGv}Fp{kOVkOLBo%Hj=~;WBu@fb9hQX=Z_3Al|m22;M9m znvt0JExd$mPhp2M%@=|3M$27-JXC=Py(x+&53m5pfKEE!QgA#4qEQZ^=6(}6Jl+5| z7qaLPhU}TKR<>$KbsIIN?^$3s0YMJ5RFP*QBz_aDZE=8CaL^YYa86(0tDMLrd}+g! zyzn_iyonZJA(hqFf-jfFML~z*cxHUPjToIWixvUb`as+GHj;(7DgoGmIMV|MckzbF z(ZLuGVZ3|S8UNHhB!hKJBH1ihswxx|!PAzQi8VxVcWqc;2LI^*XBE*et8MKAM{>I3jUzTG)SxQOIjZ(*AXp9hW-qQ&ta0_y{pIT zzW?KcKR%cChtH+VwIHaotSeD=I8W!jgC|!(%jSp~)t@^nRP5Q>q0cwBzTDgGU4Jlr z?k8k1s0pbj_PuO2SzbF8o;Ux2Fa^@97JCxThS(>{{Wg~og1uLk_XXM3Dt;4uBCPghONb16SH#9i_IQBc;UJ3j@38A`)ZCiNyHA|D$%Cou9|9Ty75 z-LCt*Hv;2Iz%Ayg-@9EW`J|&D)2tWg6Ph^m zIndL%nLrRCg2ABV(jmEM;ztN_V+=^+5>>9FDP+nLDIa(*ClWyvCP7Y-OxpG`k5?-{UzBD@L9XM{!i*bCQB!In12hq*Vy z-8s<5;OP`J0t;lD+h@NP)vwZ>Y#N0P^7-Ph;c!jXv!rx&)7W`d!qaOk+$hXbOjb9u;)x>q1muL++8Ej2-_Q-Deu6VClYnK}B-7@YlDxWN< zkR+!mFu+D(Dad>N`H{*!BC%t&@OdMn+>&2B*=RU821E~h7e~DTr#>&}znl52uzz)< zYoQDR4B&TMKDFcnX~#35Gu+^whx}_c{0px6$BPkyIb2sB_qc=&Xg=$&a57E`7k+jB zD^8KoiFxzv2PE%~Fc~o=km}E!21bbdfEp-XhdN$uqzbY0mXf%im7RU3FHq zlC~{J=@R#pO`A1y^c?W(W1-Z+=~|YTA;my9q49OwxnfV_Jq*!=2g~z-0iq%n;6b4& zRM%1FNY+HTF$b*D#&1&CEC}=Ll^xf$5Mt?h^5=N~Ytb4YpuU~Sg4_b5kWf_&{gqg4 zsmiU}zZlR?m=~h32m|`O{EgUyLwWpmr*M5NE`Bz}1{pijJtgz{$bfhowINU$f(2yd zR~~Y0uw^WQ|Np_718Hz5Na}D8) zyd?w}ISb_mBqEWH3FL@IcMsT?+H}k<%Gl1*?A_jfCljzeKHni)U=5IIjVJKp%u=D? z1izC5vi&qKq`H)FB0bp;f{+1SZoSv<=tbj%vYV(ils~Vf>&7Hr>ro@-Ll>rN3!l*W zjmxcdTeFazvQtJ8nNij;U%{KA^p?f`_QxC(Gs1pe(jIRuV8caJ?+IrB79fe|jwWy& z<)-NnK6M?}O(0w5#jCrqMw^Kkz%xTw|8;--*9*Cu5Lb3Idi5IsMkS?D)t85gOpU?6 z1#3;S(7dP3FLA+19@QUMir!>aZr~O}$w`5Z*_yDjk_<@ZM}VIF;y&D_Y5EQ*snD?- zIcpZ|XaZYor&`F`2iw(gd+B?Z?=fl@Ck9s0paiI6DqJcey-1f)Qmr1hwzbMYk(4`iR0L$TwdjSX&^eUb{1z=v@Bwg`bW+eY>fGM8*d??>T z-EcPM1@Adx#=MH5YUsRf7Wc3$W;_bZ=6@}3K=i!;7{8-?n2H(-dEj&+D@i)fbG~(d z5oUa9O3P+4Qp2aK6h(4+R&PD9)L-~L)F}Gu!&1ZiO)oTWVJp_qA)4VY^)|r`=vEZg z`^go7Hbg-XlXuiJ`bMDfb*6Wh=^>sW92hox+U+?AK<-!47pFT#LVy2EiI#)o&Ou0g zuYY{lZXLaucLoc6mNWnT57BSjXoCyoWT%du;>`j#UKWr}F^9g_$r1#5i}h`)L^XTZb1H-WYoib$-urRZaV<)78F$<=5@n%8+31CXJ)sA!pu7J>4% zlDdlUT*u}23_E`g22*ntBO!jZBG{&4kb8vVF6ncsBd3kvnmF59wdx4O#(*LX!1hn`&C6m6UX(uB7FwK4hx4omq=qU{1RUe%RppyNE2hWkB*3wmzuMLnfZ zy5yV<<)3d&{Il3uo6n?^m+*K&&%wqs;P9}ZmtZacowW2Z%l>^^(fPwhKE1-cmgjLN zKO=G;tW`LHp7o~^@E`l%?|gTYPpRoYZHm0X%F3$sHa-q;`R-YdqW>vRgHR#;yt9F3 z?j3ej(JYP4;%DGqPk9rd?SI+Ea^Sy^kU0(DEAgCJG*j}WZs-j37ISRJr<8_!Pj&na ziU4qhAPjK}h#;6l5Hr7klF4=;e!drnkCEQc+8rXZm54{)D0S*nMlxPM$OXCjj<&TR z^LntHYgOLvPvzIMZiymI*|Ym(geJ75PgsF?ukN8^Z7!Hu+g*LGEJH!gh63opJDYh% zNx-G1J^@Y_gp-rHvE_q}UzrYZ<&Ohg#33l;umk7 z#w#;T)Xpe55`ic!Cm;oRI;q@usXx(@*Qc2WeZ|A{N#*w_Yu|HJ@{6ZsOjmbaGMtdT zQe&#pKR=ni7>ZQ!iz;sZb{hg9RM!aki|tndRc6fJzNEVLHWD*Rdus&*v@FXoma?XY z6OV*=p-mD~RZY+6%nMboeNVgJP}noH;=h;jD~Tmc0uh*MT63rM&oz|GI^(pv?9r;b z#FN#mRG0D0w;@2{Q^c{nJan8CBKoA`C0G5{wQ(3+upUGD<$TOH$huI23(sNVh=c0s!X;3OPmPpTazJ;(#A|a+@i|SOMlBR8^oKtPP#P=r2sKI|w_yjYnowjW6D-bI zt5Y@f=24bGwzf9B`oZyQuI|*8RZqu%j3(xrb$Oq>;JwScN>*FvXMZ+cHK#~IdGxd` z6wO^Mrv-?q0?ca+-hDKhRn;}%Wmm+l;a*B*C-iIn>qgo2vfSHjQJ{)HNE zzBpg5`W5e$ulJD~ScBpKA#w-!3irO|ruuXg4p9V4#3YW6W+dXuZRko9g{opkE%bDl z^Pn(@w(;l4G8vUmjLzPK7$~Rwf~~S%2zC^(EYl$T(m18*TUMON;uIHwKeXU7!@!f- zDg2BIKD*kv=R5>lJ!}y)>>~|d$bz2oGqP>~_kbg+1()%Pa|{#N1A*-`f29RcR)luT zyv!qrtj}yBF&zfOD5fTnlh@!nFVp$W6aqXQwAfSNF<{oxN~!$Lv`qn4iio$YpW$>%!nHWhSj^FtS)e5W9*bcqdZgU$N(oEY);x(%utei$DE-p=p z|9Vn;_I7RPTZ_Gphr|TQR-l1k!1?aEmD7b%TYRzX7Y09&G0w&LJx2Ch#G7jr{ToIV zF&y+FFGjbC#$*{`>H*r^G}=*EjKMxr?x)9xj_^ z^sQM!(*GLSe?bWaBJ3(5q)I3fb%@^k`affb!_ice@5 z@D{w*`)DLF{$xCSfPry3lN|Yl^~-_F*Rv&&zPcywlN`pijg;LOXZC>0p@pQbOlz}E-~S0y+u1qnG? zrd}U+NUz~%4`f3HeOM7PUx65MiR!jOvhB3^2KDRV5G?fPILZXc3qRBO9Qdt&!kl%^ zfAOt;u&(tJ3aY>#NTbSS#&wRz)-Zz-tBCVd;*2UL40h;4tcrX)tP@&^ zD3v?|%!T2HHej=M9@@ARwgYz5&4TS4UqE-=ura~ygJTvbs9y~>+qXO_!)uR zOmRE&8z3VaBg9X*|C=`vl-vC0G5Oc)J}8hjAVSQS@W{}bD(@L?`t1H<#U($Q5kCHW z#w}{1K{RnZqjb4(L|5N5+cYD3^O~$b>h%5xsDE-71B?fJGq=5d{b-c9DSg_MR6TpS z-wtQ})9_3dAD<*Il7mttHZ%|8qQZHVq+1ylyzh<>pha(%PKI?!s_g_rVW=U%WJpZb zfR)%})8QXON);Yr<5mbEan^&-Xt9Yvo-wfItlw{cQpQAAj4)!g@Jiv_{XW6d;0V|W zx9{w6=?1wLZF>%6ntRXH&g1vB_$oJVS1DmW!r?~hckhx(=6JHNQBH`mwuuC}JJcp; zCZk$@gt@ZJxy~JLr2Kd9urr! z&j}}kzeK1M1RRD89pRHPP_P`76>K#Au*IuqN;)x4Wk~bPDNN(>dKqnVf{eZ?TlU>| ze+Cffe%VQ;-tAebaZYnTj@-e)7V_cUcT`-hmObt3EKyhvZEkMm7E%oQ3SCaK2}}H? zqeHY&(2AAGE!=O~Dcd#c8x$1c+}d2L+pTMaad@tNrZfK4FL zW^Lx93i7IOncmEQWi9P13Uc!@DqpdW_?lH6Ogx_UgWx4as3hNEd9nbDTZ4>k=j>!a1tnQC_3vYMaQ*>YD?(S^ zpCq43`FzW)p_hti_luyXzi=9bb${@nb894|M|SRC8Dh1LkB~bcD2>tq)BP`EAkDL8 zU{u#u&0cp!o0Rw_bj3^Y*ib|&Sz%ZB{LT@fKRYFgJvc-*D6eg&Fsex{?AG25=gIQ< z>CjV>M_;f~q-*@s^@Dq!2)_&~f(hnlLC$fzQU$e~t_p8Sey{~vN{ z{zLQO13uL{Js_KqbyoRE>e>RVC%clwLwbb<89E=2b$h}fRh_J7$#)EKZikJ_*0m?r z&Cc(gg}@r(af$Lv7$wW*t~Kamf#f!K&EewLkv9_?6ozo|eY}zbAZq$M0Ju}Er9YH2 z)g;pH{lxT#HBC}2|3vF)3$u0a0+>}N7~x2?ZtlW0yFN| z$DS-EWku{objbt*P~A@KE&yp<{V5=>A19(Ne#3=J<8M6-_Ric08p!GGXiGl#6N*euu=vGK{!eA0YDcc`Z%%Vsa#;J)uN4ybXRE&3-QYL9>#CY<%`E8_vU9tjHMBd+t2;5uu7CHV zN=6eNs2IK?h540!qhL$_B|CTj>e&o~q9{-9vI;z8hq}n$Zou+V&5xqa^@} z^ite|+obH-{dy(4*^k3u07#GxsMmRM3>J-Iy5m?rW4&A_KJ6^5+=5QFDs75$B@p4_ z1-uY_32$`(e&O;5q$^Kigvx<~%EohWzHbDse_g~bz5RZz*M{1FzfKumUuH^8 z%|GYtuQfM6fYcfp=Pt|j*+wrL7jPQTFM*D?(MD2N7@eIlA??ZwEb#6XK{rWIRdW3H zeZ;WR9qCzC*|roW+#DUrU!x-BhQfxQkQWUAWW%vtqp-b-gm%<(i=TuV4~8+y!4lNdHW0?N%*0>!njj=S1#8k zjLpI)D~k;FOzM>DD{D4U&+0L4fquuJp|GbE^v}UbC*NzJH7`%@I|q7is}=#ay#u55 ztwf%uDx>|@&0`nqgznZytGdCxIjHa7-8_exweS1Ujc*WnX?2ar4h0-dLiy4~+Mg<( zP-qtzz2pV*-CikjezcmtZ6fue_@a;J(&+~)nrO-+jIWgm<&=aOOP$Qz&hh6^?9uB2 zRn**QQsT9XfUpbwX<(RGgS40OV4(#Cw$Lk7m+dR=9!1NfuYpQjN1^x5KE|n6DWo=B#)tG{(+Ny@;9bDX|dV5YD8!s+WWoCR<;3>GCQ3*Z%8H@?goXqT%n?m$#C)sl<-e!1TAs_9 zfUB`>A6yf|S6G{ZiMiuDS+yk;d@Wm`m!%nd;EB(oDf8|Re>7g^yc(o+1tC`Jgzc-N z@W{|UZW829ENjF(R8Y0dsw&4X3I^4|m628e)PX#_tJlPXev!jRjL#a2mctUpFh?~~ zU}xaF%?y*!k=Zmp=b&FJAP4T}3Tk7cZU<_}8d|i0jF41uxdJl>S=M_`LL0j1wt`r+ zFY6cRsf&})e+G=FO7A>nTdVoSvHk`&d=Y1n9Yik4vml$diJ}s)V2m4~3s%7%!Xc3& z@VZlwf`F>uei2(>)<~nB*W^Tn;KGW`c3cjt@TT^A0VJ8%JzFgs3_7t?aEk5Iwd9_nM_3c?s4_+jlu3k@E5c83n{$#_8)*{(15a@1j`_lmE%SV;B>7D16c!?6QQhg*7vBr6RsoBR#OlEizxexUzOt}X~_m!B~+H07(@(t z&1axsESlBm*|Vu=R)zJs^c@v*GCX>;4hB=)ytW(e8Pp#)7j6*yo}BphOgwQTfaoJhF@yo$MiRE+t5LY zP=_S8tr!T!bBtujE!sL^vMl;G1xch|LW9>;sjsK6w=F9TZ8cd3p9K>41VUSJJt01x z0E@D_iz=A;>oDL4EZQ-4NZ_gEcJ3~d-dBUC85a^#$`H_yi88a z9^*wj&r=$xgXqEUPYZwke|yKwJJMM$#%Z`k=p|~D`h+FsZZ&U=KU90}&ZWJ&7SPp}?C1^nT8kzHUBxAK%c6 z1lFfhB|djuoWCLHY1%!~Z7VPp5vD!MXd|llJo$8+?YfESHCv(}Fd*GYlkwXkMu&@jc^s?zQ2bdahI55OzXn;Cs+EY>;lo!W$l!Tg7uk+|au zcjR7Es%T!6w_vt3aGWf4?t>8m_fG@vc(1%#gQKLAKvm4eI4&YYg2wi_Ej93nCYN?y zqq-CXmckYKiS3CI0<(_xo$hyhr3bYiM_K+}+`Ih`uM4sjOj85S=JDa~vmk8v4t6&e$hT64?Q|`UDB~zT!D_HG9piq~T zX9Y~qhj2v^0uLoDUhd5-+g6bRz`+RAk;{T?z$0*9drIEdiQ?)JW@KjN4ym2QE!LkX zBwi7{!}y+pnm3cZ2EwMB#LpjP6coHR3Y?z>WGWZtpxM(>DiY;OtbZ>{BK#J$b3}q7 zV2jHb)^-SN)sl^|@~q!NN^fdchm%%u)&q6$7#|f%lYsejZ1_?7h9&Mn?0myo<$$11 zY}2&rA|ZAgI*K5dE>d_Aftk?dLSNBfK6nA51Ahfz7+cR6(x@|vN@qS=YdY9G4Y*3A zLVX+PkL}yF$*{-F!`ww&Y(>hFn7S(sNGak5@&jOYc9o2z z8Bds;Pwq(m?h1h-HbJJw{Dl4>&|oq#jg`850znQfKTT`y8hTEb&WEUq9v+E5Tii_; zsHt*_)a(Cj_N&+6>C-Nk$qIG<6y(sJ-xsV~NPXR8w*QInIO>K(!1h*&-3ryP7sjeC zXI$6KKi;`4=Zi{iEg(7)0gVBwa$2x|A?iUR)EKZSv05}4MkihqIEB-KvnOg+*0%Ej zb8arZe`D0bj3aP4uX5VI7Cs(HBJC1L4C|e{`u7`kG!wxb@CtkU(yE>fV-V3HwvV2o zTR}9x1U0a0_Y0wFGprQMr`Aiqt;X8KpakGU%jNv!h|n%a-3kl5NjUcrGp@w_0EvG+ z=0=Fclz1lVf`X(e(lM0nT1h!HZ-b_AS4mgD-S=4 zUK*3ye{V0`?_}Ybig9NBrXbfvX4{Nu*65?ipd(0{%^BZv6RM$HDn6w-5`rnRU4Th?4 zXF!OtDUj+CGeT8vlWr@RVF3d_?n)i}lxZTTiJVb?d#<(oGs(Cwz_M^VSxBwK8raVP zwXUPcr^Il++ncTeUv{Y!u}@CRm}EaN!s1*!d<^er+wezs=OyHJr*iQO7Y?29<}kK= zs?Cop4G=E1Y86Tl4Ja}T!Tc6u*X!DX@g&w{DbG;MKbtc#4`z2X-17ldbul+^! zXS%0FbrTYisZI*Byz|cKpMXMOQo{<{YOP8QyGMOi$CDwLD+p%((UZ^qiX z4r*`qS!@w_j)-RzEs(5pit^OnmnmR#qlc&(iOU*)8_?530deI)x2znkm-c3X1^Za7 zv6w`|OXXqG_%`6UNwx1GqW=tPWW*RwM<)&wQ_gl&0&gA2*E6t5as?rOas&i3>%^6J z=B1wx{+vC(6nMV^IKT9@(cpQx2qGw+KiDc%g)H+2ftVXoMg>?zM8q*sY~Al_-%YCJ zm{zHv`S7Ux!7c{0A&nyq7q_UpmlVQZc9nCqHXv+TBtoGJBL9Q!Hc(zg{%@AdhSln{||k>SUhvZ~JsH2ksdMIldz^69hgUtnZr z!s4}MMqiPW{qs2)NKm37M+O^6x}%ksu#?G#FymF?wyEkKcWtHBx1&d&2fS7f-3e;| zh!gN48fvS}j0&N#-(EETHM#xQi=cH5=rADhgQKsHR>jC)Is~+xcx{61*f|gq{39Q3 zbrKZ0;<_Jd{Y5t(1jwcVtl^6#$Tq2|2C}D)kz&hi zuuZWh$@-sG?QV?g<9xWL0*VsyKz@3cEK0%o+uIvZB7)xuSH#h8Bv*kUFB-^d_$Z>1 z!&M;F$^cU^tLIWU)ho`TLeOSN>&nQv&>aocKjSRgH#FlV@0{P~ z&%#{ry%~}j_v=ss*6P==fi+N+KrE#UcHycS zz2t#{;Im}qW}ws@&`>P%FHn(WQO8R?F7lt9qWd%_L5s$Mpa?e3zZd3zgF#&R&;+v) z)SRSK97-NrQ>OmsN4B@qk^m%T!+R1JZonw%7v>z6r60C}gw z-%^3??gN&Estf<&-~|{OvO%1>{6Qn_4X7r;jEGSNicsEX3BWy_Kn<8MB%XTt(koGY z09d1WMBO!jw!9 ze}cAKm}P6&ZrHxmS3(j+g!|<#YK1*gkJvg!`dPi@imqG0B5^^(!FFanLMb4{A`j>t zV&x%0#4`!EO3p#DYm4C|K>zqqyr4-_rwzQn|KE3mNA=Aa;wpe9cu+-22f~u3{7;)o5@N-Imt|GI_|A9%WY@wrfbXX2 zlNq?tVNUHGH!wxi@xr!I6M!{#1F2Rd6M@&jW84g4=6YvnND7Rp-B}GVK|>(9pTs1- z4|9Av^kJ}T$po2~z{)Wj@vVGS&as9*9!G+H}jl{{|nb<*pwG2;wYI#DdLI?0@(% zqvfJNn%}*LN@H&zgrp0^zg4_~iBv!gdo+OW^lf^Q6IOEcO+DY@oRm%CS;C5~q&J|# z5DfPrxFO39>G`uWAY92Y&UC1PtV?+}6`VRfZ$JeIUVIj>O!+<+D*j5io&*Wra@(CT znBYA>PCbDNB!clR$4F&zPlut#Mc<_1l8<`BEKGtWB~ik!G}bNjzJtv;OfW@&^RL;* zkRx43IJfe3I&fOK35xE3cx2G5-M>0^?;p`&b|qCKYe-E5oVB@cvSw1T9V1Lre!ZaOFQA_LfkBsQkbuJb z_HcIzBt=?I$S~0-N;8(K>yKD1mcZwJ9C3zIOyV+74@G^O$;H2cY8PH$682vM5kz4P z;Kt1|_Ym!j2k*f!61pMO^Oqv=zt(0s4P4YWr@g!RUw^^ig%EIU;R5vNaLC;Yix!jX z)EIOKw2$64%TAFUttJ%w4;gk%63EoGKq@ zgm|Dmv_O8!|2?sReuopiE4t>6?(*NB2BaA{`2Y#=2HL3wpouJ~jPJZLqdWry({VUu zjP7~d*HgtQz`pBWqyOuf|6ecQmAF0@2&4lY;Y4QMR53mP95flVMnhqO9i z#0XoZoo%Dh+hjr$g*C#)n?2CO}1Rf+ufNLwqIbXwNWh)33&B; z8;)ojn=Q#sSaGsyw}}}E3}5$!)}u!1{4k`1c2>(;*f1V0q`?3uVgQOF`q2&SAt=1* z0FJi@Udw8@aIz-=9!&u)X+W5LRpIJguL;j#Q(Or4m}2tZR-wUzOgKS%K~WmS>pxvg zfdp=;fj9@CBN9QU6@|TK^io*0`Q59|=dUztqrXV+8=zWf*wMU3QBhc$d-;uVuJVaN z-}2Zw5G>1=TU{Gn8&4Do4mt;vhq=uhp82jOX;>)UQF(FJDHut|2Tm&)9k0qo&SKL2 zPiCgIa5Jw_YrwWe%iV$iB}~QXvh0QHN8DPz2m*t>-ns_;`*i;2>G%Ty(WE;~h7VZ& z^S`RV!%0pv7-5ecl+ho=(c0YaVt1$kKb*s}B<=?S1gtn;-CnCGNIBe(&O~Dsos`k3 zf$Oq(BMM%r4=Ob48O>#iv$h{oKYJ|2DFNdcsBB>x18ReS7hJ_u_o(0+d>h zj{s3mRwp$z^_PMwH3%L_9k83=5@~COZi9-qKO>f}{`Yt(Nky|UG07~k_F*c~ZfGzE(c zH@G^7hZXuVxt*T$A+*cv9_AU;m&M*peO*wu?r*zSRqZEpwDU@KuUfn^&-aKb%WQI0 zLh{^gO}fyi^d8{JH`eR*y1BLC7e|6j)3!7tZwrQM!}vBOst%A+Jz*!$9mAhwoe6iq z`d$d%Da>MoS+b!gKz+FM$$F^}P?!&k*Tlxs@X6uI8VZuhZa|+mT$}`4Xu=)J=K;19 zhqD*)&;4yD*(H(%F5KmqH_-Smqs{+exN<#E;PByRM~mR@?6+(Pw#UTlrj*KPAgG;$ zJ65=;nNx9PhdkI@y&pHG8n3NIG9xrdy`sw~8mh@(i3hIcA=w(zF^Tq zB2P#5_kKcoKt-V}kQ_hK4^|^4EdR#QH6Q*9GnCv~@=zyV@_uVqGnw@fJV!8SF@zvb zJ1@D+{wMl*HqD)BbGv%z3;&(TPZ}pjy|O@+d1hogDw0-wu`U}h&)M@%bWKd6J$&B2 z|FZ`AROpxixPj-TJdevBjl^^F#Hw(6j`Q2TO$9`~PxzH&!>K2#xyJH(B{-(FlI_y`=C9=iW{?tu-<{eNEz*Kyav z7^s#0?`wewr$^DfhH619-UX5&h*C#$o+Ob{Ry4IvmW|k0%Fl7Jj#VdyQzbl;0yuYF zmAcLyJ!fAZe763Q93DNVXs&@}iE$wH20PIjk>m0s{_uerms0vWM{e0!^NUVGEz)$k za_!WI*AQiw z>Yi<+Ev{0;DSJ!InJg!QNQrS?SLZ-~=G$H-Hu;WB!*yxi0&;%ITWk)6-HD|t5zYgZ zf^)O*J@??=T=P>nR5R&EJJq~@iiDq%%=N>R(a_W>|DB(&v2oDfc(IY~X#*#(ve^(l z;{dPaH22*OjpU=lRm0)(B+ohwLA?Sot`+VNa3@sf*MW{0H(#RR1(oU&vKgaVZsaMpc%!bAue@kH8N`7+J zseOQi*K-cF)a+Kn z95N?k)0;0DxRlXov0dBme|`!%?pp-0S$qaHVPEN{m?eDFNXj)X7v!WUJoVaw6Ov|0 z)!HZo;G4Q=gjanHJb15i$S57VM73N~6miKnj^CWkeMx{e>z?xIVhxX9RF(m;7R$2B zicqhZ-=J9Q;Ieu2v7M^2*e905qk})W`dDIRG_~M}>;xS?$ljhQEAXP3*=bonH!`nDchzeXY%vt zO=3o*tBM!PcmbNc-Ksa$Pc?Z?pq|?_ziPK;mDCBb==nOBEmK_7eK^X=FSE*hVw)Ka75c^$tZH z$jmNSOjj*kf7!;J?V9xNPZl&;b279=j{*XeT3k(jKHA`;)xan-8igcQfUyOac zei1>-&RYCB@-C8WXSp{HI8oNTaOvF2Slp&c%D8bAr$hUCw&C&zh&QXmp5jvQ!Wf(} z|NG7p!rgg)MjuZ9`_6+?kbvML%`}-JHWpA)SZ4pezd*=BX=Z_i)9Pf-)0I=^n|-l@ z$O(^LMl=n7N>;C^whX^%y9XH6Wkt!2^3n)oJ8YM` zge_J}*#vi0^B<&*iy6)8v1zlR<8P-1)#9&3Gh(0n@G8ZBJClH!uL&#;B}g zK&$W#Ammy}Xa)>sq`Lypx)wLC%@0^AD_OrDyE|OB6lH1n?fTz9D(m$(v*Ke~`j2m> zgn;NF&x`AT+J8g+QluLu5rjzv^2N-BW;-KIxuRs(jR7@1i|!-dW$gkm{Qk4gkI|Ix zvQzEEBn($3JyIFBIM&K20EE0oaEyyA$~F7`lTE*~k3c<1*bkL#IE{E;?LT?oIIga1niQ zKk1&qGw7u)oWZd|6q>YnX{(w)$uWYRKp9$HpwFk52~`;ZX~qhmf!9VltWZHSe+k7_ z0n22Ia?#|W4+sIOX~nw{`D2Ou3mO{d>IIIp<&ObS`?}5{`h~;sJ;ud%gJI&0kjnt) z`W{#CAzh7bOh1<2JK3{e06it;=8iz7_tBDSUL8h|T&t8`_b)jc%mD`WntHd$3G*sl zAtS(Ld6&p|Ej+!{9qToA226nE0y{zfb*u5m#EWa%Wr3qS_V2t!5{?VVt(XGa6`V-E z?%}#jUh^;@;I@h`zTv6^11liw9QXh`WTO1iC1~*zH#8!8*&yR$JvDQAb$CgcJ4LAQ z31^C+`xoZ)NBn_#4wH@jRTEVbrEzL(O8o8eCKHh!0J{z}Xqdr7H*I<OnsFdhAl}Qrt1Y(65{W;1aGRZB`Bqp zjy(0+@Q-jMXSa-GN@0>XK2eBCiYzoc66tZNm-n{t?Ojh+& zJFaE6t}Si?!Bzn%?{WaqP{tax0@)q#JTT^}7b@bhu3;_M@$*{dAHejYmT7={*?%B7 zgDN5bm|%^ZFf;zQ9>_cm3j~nHegw!3DLmVJ_{VJ4-k5Yhni5VfyQd@{Uh)Sr+@}>^5VpP| z!5W~7EADgWb-8fGc0+)p3EzT^7s+M2%XiFQw%MYh2gek4MJ><$DKozrQI20}jInm) zbUnIp zO#h$&0KJp@Oc@^e(X_1p{IY`FKHnHYUYofIIZZ#^nyN_zHoPr-ze2zQDdIsSywF(} ze1M7DfQ?@=v4ErbVrVa9=fB(FgAMvqj7%t>eeBP4{qs{bp930nsaE-l(Rw0dB7fwM zOj@vr)YX|noccI0UpP?h*znln!ilp0JUO(;wq;vL)|qQjX!c1BJoet3rANeNF~_w_ z{3#&W`w4?T@@GD)J@|f8#OvF+7JKGPolv&F~TZjeyWin=PI^lMCuIrP5wOyF&Y zadJ}j91LoACw+2>E)dzgaIw33L>w?rP%?}gvqqS}bzEs?y%_mM|Mdr|gg!XGgvWZN zq^J4~Q`-o#XI8_4N^ivB$Z9wVXCj+!UsYhK>+R`p;5gg$M)N+3XKCLy;x%L%$rNxO zl`u1pKuh&oIsxg`WNN{m)1;&&p%1z5Ty~Z{HK>id&yhqgryNbiucS7X#Hae=LqBzC zgIX@X2a12AJ)Q3D61rihyS_-+Rzq7)=pHgi!-HyR=SQo6-~%33q)fuTGZ# zqi5QA7UML-fP9c?T@E_UQ5&!|>s%~T;0Ey;X8n2G$)Q`MU0i@HF;9IFR8CyJ+8Po* zRwv2&%T64&!{CX~$%_8$Dl$}^*Vnv#P`uGx;4x!v;4~f2)^M_!*luTB%T1|8q^7}k znli@z#?0&{GJ4K^a298>ilQ4EUT-|#9CCjj;>bxG=Ei>f!G3sJ>y{z;u}%mN^$81k zI+t_yqm(Td0};s!ZzxvNF&h`kS_UeiBnM&)jZI|W_GFU^VbaG3G;3MI_oj{S7L})^ zr9EBl`SR(zU7yL-0&Xa*7MfIxog7FJ{o-%lCbb@R2 zK5F$z<_Cyo-BB0adRo{H3}!x)Qm;;b#p|(yS&Ah;7gJGlj914L7Hrquz8mV*R}%AQ zw=J1@rocdAsA3>Rc6 zve%V8_byO@-Qfk{GC(HW03Db^{B5PoKx~sFQ)Rd(BUy-jdNx+gUb|$H0cphrp}`hu z6*8nJjCZM?e0_h2fu0==#$-PX<=4*5uP|(~`rKz&rV%x%c-5qSCm-_M!%lN7;O=O% zvs@Rpj0l*|U@5lDA|q-u?U^cRGB7$KSo_u+uk2MX0MfgRCWJSoFD#{hd8R4jd%QYN zBb7qU?-b@RTi?Eh-(m2If@XVfl}i^B^n=4#CisB^3!4`+Y_D?Ti>kaSjwXFuO%`*P z$OUJdRLivI8x6c(mYCu2%ea)uO6cLZ z%bD-$(>b&A(@w#x_mzy3!c*3sl2-kO@Z?im6PEj4Z+Ix-7zsb(rlae^oM&iMoP9XRRu(aYO`O-ETT=7oLM4Fa%R$4U~4j9<4PucL^DGuNd$ zB9UWtVXt;8$?i=ZDT$L57>$%9KiCgjpG!1uf4v(jX$0eFV5X>}$}HA$ju$;htf#@* z!aOc;36O3|8Brj3eo(k)eQWW>v>U@f#YSqNsb29*il8#V*90G$mZmsl4G5$QCy3jC zAqXWM^bX;nVZ=i4RyWtqWMR4$0U8Xur__*@r|sFS!xMFYi&vB$G4?nIlD7A7L#gPpiZlfdhm`*r&8?!w;|$!&IVfB%w4~G>zev` znTb;wyA-r~iR5nfa;ZJ)Dt=6gQs`Q9-xI=!e*+KH{zf@Zleu4JK_4=E*lQzERc?LK z=jM=cG-{K(0&C&XG)T&eDZ>$A0)z-O8yM(-p5?;(<^;g>BSqMmt1}9no=8U%lP1c3 zhoQ-p`++2N3e$Z~pGU|I-5kR7%o;v36zC z?S6mE*3+QV9^6n9K(_h*OP=@o)YBK6wTFvVYEyS^BDG(@?zDc2U`yzIW6g4x@V+sz zstlNAUHV5;u{`RS?{!4ZZ}t-e1V;u8%Z%S z$T!8UU#=XezLwNZaEA)0?B|Q_x_sDz!x9oJ8(u}(8c>)wnt-+QbZgT@Wx(o5lC)B` z)epWVFNyFnNkhX}NOQ>2axDRWm8|L4dQaWpdJ@K4 z@3U>n%1STxmlcXJ2Jj9idy9camR2E`1lj57z3*CjM0fD}3;7>B4cdFAo&JLGp&!a< zMzv0g6;BV*nLPL5C!{(wb>2}pKgtbR!^70`4&;3oSR_hue6I9sKVf6+)bRfCk(F~S zFR}{RIjQ=as|Mih?Q-l53}!HV$)fgIo3jmbB@f)`P&2udHFwrt7Ptp1^hvgjb#X!g z)o+$b55y5CaD}>g>{)BimSY|{%};wYeyj3aBl1d3`{o6VLZ!rh^QV&s*N!3p$^TOE z4^)L3M}YW2Ld84WHkKRR&M*?7(Y@txJv9LIkWbD2t#|mZk=q}2*D&5~fn)#Szd;=a zWr9;0(uAe2=iKz6Ff@B2Q#RL)&j#*h!566jZNa~6K;d{L9&|}$8EX9~8Hk)8C zWH$^eM3XW3dXEP`&dK1+8)*OxI$9ly@*%asad>MP3xKjNmu+#VpGVRAKTTkQbSr#P zu`CQ38|}eDyS#?)bo0hFqVJwd0~kE9nOqpDPUTVxCj!J|9Wg~f*qdb+ z7LNBwi?0Q`+20L*H_LUg4ep%+12nXx{SUM-w+XfHU?+kq+Y+ms3)2Za>NKH!r^(;t zL{(DW;XEE`n;h^J^=Og_Jw+D$a(yK_?X&cJ*;5DZt(wl|8|%s)>h@K5eiW5!t{-G> zUAFhmo&xnPf=i+CP9j*p3?C2jA%utWSzME9gB8Wro%41TaznUP+D6;^PaY|=FQ!cU zj@QOslkGo~@42yb#Ln`fOGH}Zsj*&E98`P2&-3vU7CSHToLgbYbYUJm#!oc-fngiX zHv)gHb^n>c>lMj+!)?2*Z~5Wg{nH4yCK<5i)tiKjSx=zZ7{7;Vo^9}S$XC0aNIH#< z6B8YBN#gAH1ZW`kqL@Q0r*7W1VUG)vnnde?>a(9NjkUsgg%^|~x>M+J2b_VdG%YFZ zk`J$LJHTOL;SiHboVD_g>%OI~FW$>$2}5QXy#c-UIOl}$QWwCO+l_!!;>bmUORJmi z)z`?u+lRL=+I&LZ&h}E4AZqF7o4+!@xCrXXN47Y8gRcDa_3&Qz!x4+qkS4pU6%V*| z_X3PB#az<>nxIP#s9vfF&FWlRJDZdvJzksM6%z!3)dA1B*? z1}3j%7B5LyNWyze4XC?h18XpT)t%0pvZJvTnaK@XrJ~;Xy_}HJv`F&!fO=n20klZH^tdYT}N4mfAU@R!cy_ zmyIG)irCa5nLb#5B9lnoV(6Lx@EC2oOnQtK0h6~}^A;HrK24km0dE%1%w3)-B|Yy3 zoCY)5YoE8-R%rZM6g%3tT~@4lL$F%yFjHcm|4iIU}h#Rn-<} z{U$^;_HJ1#ca*JK!sAWcXLouH;4%*QSvtyS$%H;!% zD`n#LCy3|jiV=~Q-JdV&2UjT#HmQUQn2+M`pwOQZRic)(isJj-G<&S{ZSY8$j$XZD}Euj+HCY$yy^$rPRTf&32 zo^}bRO*CObifEELYnQ_3mmXuHs6iVYjESRC8njo*?xwJKE_C-23oik_m>{f|#;(?wj=jx1A~ou|5kDhE-9Q@VtH zA?e-IPo=VHNJ&=%f?dG|&7LhBdVhAl=>sc`K#;Hc$q-~izyRV0!(R0|+%mTZw2IXL z58nUcK5&B|jU4F3KYyakeD5!xJs=4z#{K{d8FH@eZ`@WC9|82JfX$5{O6H6^IeB?K zDHI1e9-mzm-X(mcA!P(>0LAubRy?`gt%Y|gk3UlP%7$UqUBO_x6)PoRA?iLxzYJ1G zMqNkS%cjXuY&g-lg;0YtI@_%%v&mU8&>5)`cijtRH59ATg zp7s^9eu=|t%|9#m%E@T(9aV{^f6vEjmZU)gDkReGK4H5#yDf2%HuEEBv)(8@*Cgoj zW-R_T3@g*|B{>HF4o`Q{Xiedir|8990UT8&(dA%mE!rn}EK!sPqUUnFv&4Bma@uDJ zt3~1TQoFm6lZtbCH4Yr0b(Er%5_XB!tl+GLTX^i~eo5NHsGu&K4)_Kp6sFK;MLP!v ziiHdXTL=6RH{)*xytJfn&AGI}R)r+L%TxLRtD}vvxN1a7f#r$!uWid5hOmMG;=>ll zV#4Ld1I8Cb)0xY~zwNc3>wlZ7d>z1&huB#(ug>J@1Qw0L!RuZS<-BLd5&$a7aMO2N(w$zsKlK-Hf7vA$aBA4NNt|JP%hQ%-IsV&}N z1F{ULf$@Od_dkyog_Op;3(0DsMd`N{Lf0`4FRpsxyPC!VB8w*4VlLz?@g&vAWyc(z7>S=~_G!{?rIclubvfs@>cfUv?4OnWWOKl?WK4Nzuy}t%EkHK37I1;ce{?BD2!>Ix=QX?+v;VI!`q~+I z5bO4n`EP11sFwK#sTr{NVlly&`4BG#224SYtsN{9zf7as{m3kD0kGI&w;jz|4|0y) zFXuRjE-UBKl7iyhNyBsn;L$XCbo(<;Sx9;-D1g< z3Isk$<|aNUI9&I6uRu+7!cE?Mtkp{D@>?cflIA-DoeP*guYQsDbV@ab8E)HBNA{*3 zETa{`IilK2()WX9!bMEfqG8W}KwJxWdMgYi7pDRn z$!@m__V0#}8xC$K7P39sNMpO?%=a~^j*6N3N^K>pSvp+f zmxsUtdf4e?algmvEFI91K))Src@e=q^|aNz02@muEF00{DT*fa_*ma~GL}9vqkx`$a{pLzC@5MJyR`^Xf}N2ex&v2s2sMjlnVH71^nn%P2ZYp zvDBRU#Bnm)01_HI9P%SD;fYgFVg$Xygg=hIuVzDF8;YOSNPrs+3bpg)##7|qZd!7B z^MzA)OX4J1qJm#PCjs$Y52Gx3yxFHLdXUZ}5zD=8eyg+O7W{ykueK!(@{QmE@7P)R zUSA7lyPMA$t9IqBN2*q=(q%M9o4dS{>^zQ{9vc8%gNzzRR@G-! zpB^1~`)-R>R%~4TaZ`zOAa4m&QbI_CWi;p;4{8pIPGGmPwv0Vkcg3{bb*vrho3uE` z?Ba|F>36RD<|BsU*6W)=pj7nZ;q%rdse?=EyphS5bK2oSl3RbW`>Ynw!<%Y(*bCkU zaW6^ZB-fXUa^M{s-<@@d%@i$yvF53wvRS&mxdlvv52!;Kx+&Unt3V4!;$)NDu}ZO~ z#pJ5V(lmAj+x%Mt*b&RqkW6cMZKwExfBev^N0wN$BaAep_bmE_`E6j^k(OF!^s1q= z3Q)Tn3L^n{qqPUTayM?`lDzo0AAAb$7|dQ8Dl6mt^_Gp)EueM#y)PB5dCtbh_UT$| zKv{gN@)&1mVDJzSjz~c0XH!IXeb{8ZNM$h4Dk)sCY$xR!9r8uu(`P)QN~!bWM!+cx zZ>$QL4mhG2Uxpp;_MjD5E=JiPH>Boej*4u(yZXUR?v^(`DIyVH0z9_8H{$7h99 zn=U<=k?T2!_!Ja7YQ7_BJEB4YRG(M@v%SgFdUo3UA#F$&$>jIqC}{JwBL2&dQ=pbS{>$yYGV(!y5^q)p_~XDMyw zo!Uta^HPyITdiH{40Edz+emZ-bUUD^!SREktnVIAeNF=4wmgjmwEMFx5t+aqvR6`I z3%8}^a`pEtEsHVxq@D{^i7#BnoTbg!X1XpRM4pJ{e&LGS<=N$w7WiAtw`XR>E$eYz zb$RFZ!ABvpR63~j_pN!Fq<+VD=1_X1F6+=n2DmQ?e-syFOit`pV<dRWUC#2ApO@fF&Xhr zi!%AS&v(G{NDpwzZ5`^{`cvP493_EcnM<)a&aYiE()|Ct;1_VK`8m|*g!r8+D| zZOh)|yJL2qJB=q1Ee0)f(92h%9Yp1+3dk48_Sn!J91xS8VCNSHDmMLk4&T6GyR5?j?~UQNep~$$$H|Zrc~w*Ex}nfzDm(jrp6$Vuh3T39|C?$Dii*M% zq%*!$Z5N+F=rEL4*J)e!|c5k>zh#qPGjSyhwhh3xUo)*b6k8-!s3ca+3NAbrxL)Bisx6^`Re+S$GsH zwJ;YWoq9*yflwnB_6C>f6Ry$aW(i2!L(w1f-!&3_FOXTuf;{K}>(w}HoosC$%vTxGZ(}oz7u|a#cE=Sb^onla)Uw1CO@5f+|duAmuj?Ozi~-tj(vPs-7OtA z$quv+&$}{O^!(Z#r~3Pi_b$Aqc6fpmBw{v7XXR%}hK4Z>TY|6xS$^@B*Z4b6X#38c z2hYdnjA9GgA@p|5QPqFCQ65D~Nr!V+0d2;HC(xmr=F2zSXq&XIj!U3e;&suv^W%I9 z9KoUW%uGB^M;-Q~C1>HRQF%J(@X65pS|YcoIO{_L>#)~>gY+r7v6QrG27XDG-o+?? z*1?uTPn#>_v*2QfRa`RX(stvG^64`}i=rJ5zMEy^OLaUMQ&YV1w2zs?Y&+#pWIM(< zNmL!~7pR3X6+%rR#l|MQ{fs~%y>b;CM=FOXp&jq?1p@?Z^)@N_$&+J@N3#|INE7TX z4|6J=!VS;;O=6p91{&{_y63c^)1t9*lwsUxp}88M@s)z2|LxC+>Z`#Jm1M@9;w{Fp zlTH4ZC=G#8xnb-S1zFz)&rs%o{-QHIaiZ_kM&OLZ;w-)q$*?B)@VG=a#2zwQ6?_YU zTZr*QEMPTKO%Ye-89r{Wt^$WoVu-CXdd>unj$kvHTHw)UPMu~YyNbJ(S2!_{C*Ajh z0?4jK43~?FOYUwhb&#u6GJxxytBp2UOYF)O9R&qhSTvSZ99Kh_M%ST__}j{ z1m?Id4z}CB1_dm4ED21f_%>X%KCx7JY?G`1fnW2q(KKCsuDkhLVL~C+<7z3(a8-1* zP9U!{448ZU#j26to`OHsD<wzE#56B_>vW)ntT6ClH z#_(hM1OHkEcp$XAFl~I8C_BT>jnSbdQ0hSPcTZjGN-(U|GJ=f>X!A^m#m@NZ9jS+W z&<|3%(?l-4cv;A zI8O54(4V{Ou?I*2C0<>$ToH1b{|lM^ja+|P+zf^#`+JBA|09m_8GN<6xTtnb0?Fm* zlklnqD6y;9-T;S4;^5ni%P-w4fopgdjg7gBsDW}lHj}u;LT%02SAmNXcM8bj1ubi& zkfa~0prj5e2X7Za#-InQB+aEwyi*6lQzsi!y|~5(a$skDS|;D&#RM!?@czX3jnwVe zA-q&P?wHNls;IPp6_;7*TTpuvm*0NFrf%WH+uYrB1p`Lh-x21Sk@u8mdbd*|Z1QGtnoNMQGAp(lwzh%ZWLji)J( z3BIANYMLLqpa9yFu-hiD$spFAeM`c7iWS>QBd4OVI&1OWVuQC2=aU%`WsFrj=6>%Y z5^S>E@FaiEk;un?d6`kfcI8{Qk#3VxN3d5DC$@FHZ`91aJ10B zPZ#e5r5aiXf4uUYcbU^puNukHdQMdBHFu@4!U@xAc1M{T=RctlFC1>2GigcLuh1qP zzNkyo97<#`Q(vq`1^_K{rIM!O$)SmB<^RNM<<%_9zW*DnfY_DH^@4;KS;~VB|2vMo zz#|;phH3rzJpJj%P#*$1g0upMTm7C@9EZl!2)H;@cqxWJscyICK1j0rE66M3t72cNZ-TA#AnUp> znfbD5UD^-y%Z6|t^bhrwfN0?fB(PD<5dTymX%IqrXmt>v=JWA=itHQ{NqKC+|VFTOcT}^T& zlD!`fh?wdl4UTS;R(;nmgO7{m^?b$aP_up04}_A`tmGLN_oqYZ%0&32bN*;%-S6%R z#zFR#k9v-FUz4j%1lZ6bI8KTPe$_=9r1!<;Ql{8Bmjh2^#cZq=m^tWK~;j;gN%hfVyboa8R%^8Dj)s@JJYC zMm_~8IZ#WBxS*g2qGUS~m)T33+(s8YQr=vKMJ=E63E|5n;LfXg6^QQRVAx};Q$<`9{;p3^XQSZ_}vI9@rYC&)j36{YmP>lU)q z#BW;;1adw`Ap_RyIw|%&sNWM_yhKHv1-6~4e!mK2o*B_fpNxd%;S1&2vN3s@*sN$Z z(K(*YU;CO~0;^;9(%TZ74_VG;b^vfz8WeN4YAk zvgwlSQSe*17BDR`Ui2p2mvO9j2F)u$4}H8IQfrJT*hWm+1105tQY1`XG39CYOJ7ED zP?oJ`qlx(WPaI3{;H>X!&c~h#isZJ| zo5xCmBWI2TObf4)iriZ7dWhZkiQWDHwO-|^A6p4@Q368oTdOi#64h{|r{kwOD+lVLLgn7;XRzA-us_qDzDi`8m+XS3 zYP2a6f(WuAP4>@K;|H(qpMIFBE}!aXZlA^q{Ftne*Iu*Q<_FQQXTsq!2mW<}U?GQT zCMvMIQuvEz8&}{yQh~qmnc6rWH4#oXo8|Xve@j3q@=ObYWHUVXRh@n-D-jRllG5Mf zZ1|~R6AfY-7~wgBQ(%1yjjW*|Lc2kcFvoeOsvs^bpq`p&jNMPn$}kE%TXyM{Vdn|? zI~)D=%A)p$bO%w(k2Jc!0Qb=BZG|^)mVxnn6Yol*CQ=}F$cg>&pm?1f2@|7nDq~s< z#Z)paCRw$RMKMW(wI3B{F_@L{O-@+Tq`Jy=SOSGFP=&dW-6j=;9j#OETqIOQMv3>< zS*H)#;2`)4WR)8fjC?sjC!IndrsfN5KGs(j@8k^?c+}BDWYpw+OLnYC+3R2z-z~w= z_48s)2Ki2nKDR7(bRuv#$%%v!m{A5Z;JCD22#Ud0W3nkoTl`;tXIOL)kFVaS!HXl7 zgrs@43K^CpY`N>0)kq=s^C$rLKkGJCw-$ZXWD@S>c|k8wR!Za5$GE)_kYe{vNy)IK zvS`|Z|FU$TAy4`zK#g2(z!^20rE$y>nlq^N}qytC}1)RSdgQQ(JfZ5ztGI5dNFzezG`k_ zT0RWcmblan7yfOR=j4YNDQs;4Zw1*7zS7i8fX=kI@97#*w1L^5=&aoOyxshW*FTQN z0(MY!A`biO2r&$Oy5)u*NMJ9imC-6S;sjhF z7^BBOopQQ=n;fx`ljI#ELE73_QNDD$e-Vp{gmIFC{n0Zj)25EO)fPa0b~Ogh)CB9> z-&Hw3=c=3U0U%*%>mEk)Jc-doMU>HiRJC|sV(+jRD<(R@s-rRZwip}9OK;@6Y^Yn| z6w;!>zDw%?7O}`SwxcNZx&}M|ftD%0&LB)J9ALMy@f|>lP#@V=IcO)IWJkSSXCoG) z2zfRnAdoM6S)Zpl#PymC?61krl#-Qf2t3m21XXM209fua=dM*!8hZT+0AIkTwtDD* zs+>#@>kn2cT~p^bt_TvY@qbt7?$hQHQS4Oj;eI~5S!Z?n$Ffz*+&>@@|Er6yVcy zH=UQX6mQSe!?c_fbL&hH`og*-yvJC@qdMqSyANEG_xgk0Bf!<1*&nyvJQ^;_rn(XB z_yHh3rT_vqt|Bab!yb5`>NP8CA1=@iqkwlS-ot6VJfwno9>as zDs=n;2a_JuEIJANgSbyWgPze&>Qbi^PBpddIBEqN4@(K)s!`Me8(K?P=g6JFlZO2z z`YVw!!?J(HnSbP}^6TpUc*NE0@8cFu>zVXTx1uY&cE5*xIWN^|AAL4-eA|FTm0eg4 z_^xsik~-d3Zfk_M13rXp^1p%G!LtyzvW)`9LjP=e;mGoz>@auJ``NTDNeVDO7pS0o z#U`{Nv{}r6^q^J9C}-8 zwA>ctf;nVk>cS(v7o$H&J6K#<+vdL`h+sWF;}=ZIG0hu2r))5q7%R&eRPH%V(_q*_R&tB~FI|GzgZf94 zf|9TT>84#5$#aV!|NS*46s@?#I)op5z(Z>#TBn~(U@VC}?9EyT@$jBPDoajk1xHCJ zvmI1bRe7FH_3Oif)S6;G4@$fi^WeC1gbs;ax$Y>OVt$*(4nE&X`su$Ph<~w%#b=GK zVW8@Y3R`cRYB}%P9D+=W2^=J;%E|x@5L6vPw*d&<)-0>$-d9eT`9o}Yn=HQ{0RLZw zl^S2Vl_tDx=s-|5q!Y)|(SvE`<~>t|oI$tcBr3?4s@XaDDM&*W3@#K`frbgT;cWb1 zSLQucj&fr&mkOOx^mP*DuXfFu$Z}*>(p0A{Y6NCfO2flQ5;+VhCiFT857+qV=31^K zzfq7jWhHxrgebo9l%oFhVMlxOSa3z+Vs(l8ec&v(IN)Us>IAaSVCLS$=SCac-enIR zker!jwtBcef7W@-EYi}Q)@z?fRU#RpmtI-sX1#_g&3ES*vgX5Hs?Rv9BPfZOA_*8h z8DIs8$zOthI0qh3pY*j+A zH9g&@AMY&i`1jc66$i9H7t~Vw<a<9ZpPZ(Q}rs;N(=T z`(V~cHmTJOl?9EJm;_;{TdruWp6bw+2#;6Zn)&sFCK+;7-u*`4DzA{NkWmR$={s3x zooT~PiibiCja>nUxnvoj$Em^YZ{V1*1w@E+hb)l!!Y3C|Cttj$c7T4F)hi2>Ku&&p zGiyCiu9&Pvn#h6Gx12T~YV-0;(u1xf&iabsfhEhyL*o;y9c%Be+-=A0UJZ=4kv5Tls9h z5)Y=;wm8{i0uvuOU6emDBD($Tgj%>wLyVTXCScKCNT9yqaksub8#D|+)mu!~4X z$qqI=LnwjG9%HTugbc@r>XdRdnRy9saWro>p^b3-k1htWkie&*q65>^!&wd47bjvQ z>8Py65DNCFyZt0v*7WpV&p~LsOacU%Z~2uX_ym0{HG}KmamO7c@XBzcQ`7x}vGJhu zx%q6_+TK(ss6Bc$CLjZr=JuwN_|4yj8(* z;hK$}s$FS%66;anvVmWNV#^kDsJ8d9O7a3pZIrDI=gE3Tt?tgWdM=#h1AzFK^?X)r;Y>x+@G1RZL|6hqwNj>8wV-WDxt4LSh!9b>#X+&r58ED7-=ZEh#TewQJUg zyi6`3idwU9D@m*yS3HHJkpsgT2g27f1h#MlXZ*OjZ^`~UGHm-AW~)eIcSWEeMm*V9 z_03uWG-4L-&hF0`!#Aoc{1ZW}x7#(1%J}xqI8baKf+UYNkm|~n0SbXMv@%{tDT9^z zfk?Ih`DDol6rOy&YAvyn$>wdsya~6e$YxhRhAbVEQ`YlIOFEw-PnTHkRk28MqnttH zo%p1K(HpTPSLgclp#U?$hb+^DZ{)OZNnW(+XW1oU&X37x!^DQqOQgnF=Z{*R7!hka z^HV3iL)}n!bP6U&vHMZEa4@1N3Di-c=?~Ud0+G9ug->X0i)vU(cJqO#`ndjg?whPM zkI8bO`N<)dIo^bqM`w41Xoc$ZgB7mJ*N1dtY+tf5yEgj=NK+SCX|Q_LUbK3wP`dERZc_T}Ok6Nx1fty&5KsZYskP))-r&k@QV*MzGFb-^ zVSs)sMOP!=7HBbeY*l`a{gfe5-?@cClp?zoPqo?I>OIe&$;*e~#h#2Qz*~!|hk1M! zvAD6>JM-gyERwi1GlL;WYv6IRyAqf$l19x3n(u_Hucy@hF#nF$!U-#v?s&!u^dKIz zqf1XpX4_lyb~;R-HP(2Dc!St>Ce)XmC0$P$M5mw6m1CR{K0v$TkbvntG2-6Q9h>>x zYkg=N5Qx>tV@&hEoIbo+lY58NvftRuzK-B$`#izL{ z@RqCuHvZhuu+crY&zD>5xyw}`K!Jz&?QJc{E6Q~!sH^rJs-$YMajbpPGtriXRx_FX zxddvP*FuMsP#RjeyY5%JbS=T>f4;i#*>%L7CNRHo7eS%sE`Mk&Pi~#46x-QCp6b#I zzAea-Kvkaq21+3I+qd=h*j9hvRgm zV5OQh`7LpD-x8QT3n+yQC+wn0rT(XdV~)D-(@&I4LsUiUyX#HvL_^hL7R<7K0+2N4 zbE~=7MLFPbG&f~QcW7>0atjOzcvceMOs{PczED?3xYj`f+&Mwy#PmW>_wYXE`f~Xy zSIZ3;OL_4xNP}uXbfesS8WlAPn{%0t;~Z3xQ`Jp<&#|)yxf`o>v|fFW*XC5hvC7k< zhO%l^Rp+xj{SI>#*21DdSO+|=b;Qr`MKOZGg9_;q^0T3KbWz&zF3To!(^ zy-?ADeI5m=F~Ai=6YZ{X@ae%(z|h^mR))nhD)1pY)61I5!KPxE&AHbXpO>E`)Q1sZ ztJEw68b26jE(o&;BXKGKxEcsa`DcGD2Z$*w6^OA9P$!D3Y$^l(O4ZI&3vm>W~g2 zjOPNf2akKL#*ZDR6lT`dY}%0Mq@K8QY^39)FsK-B>NYx^rV#efU_BskVcF|X#h?)2 zxAyrn=Va!2hBM;E*>se11RlaMD(y;r z=fB(fw%Y@?QW|(v{GX5Woun%-_!$)Z65Fx^`6r3|;{5Jv>DD;_;`A;qXYoiuQP@mD zBcMMEjy|$E^Fhc^(gGzY((?d6&ArxG0iXqKJe_1gW4)Ro&CuTg6xekSCTO2(J3lTOLzAKqe@q zy8jIPxNGKpMa4RNfOO2{TgWj%@_f~@A6-xocmJs%61^SvA=QzhJ|$(fthzPeK842@ zBi3(u(vLCfwrl}gPdO`sSP?%T5JBs<4*it{Ws^P2Wjni0YAPKPd3`1jTzZ%-IH7V> zUpNW{u`IJ&Vq_%}V7+fgSi1Av-Fi>DgM%DAx?sAr6aGiKqvHZKKT6Izs%_-d2LG5R zD`0P_R7#eqPt}3}gfqZ6MS%=M2)jI5C2!^MvEPyL=kl_edY#I$*}fmlaWb8d?TSag zJ~HE_PMj{?9y|$&ZIdP1IM$=s{y|{w@U(o~z!_nbKeh`DS#!Bb8gl3h?mn>58{3IS z#+0Kthxd&587KcN4FADIftc3cE>Jatj-`h84ljTYw!23c2oU%&TE$36x;DXBx`}sy zSK|bI3IqHR-gp&o%lFO~+IMgNr3K(5QR-_|#$@I|GCU*BB4ll5F6 z?O~kzK?Gr18ruXVY?6}&p}pkU9Q`MJSN+qc^%fJ;W;j~1W%C{O3} z_;3tRVP#7+aD8p_i2v`p?(bI_H>APaxc`XcBsLYdsw{`If>`pw-}^G%ZGm)!$5*+p zB>)@uZ!&;OgUK_7ha$oXE5)W=x*r$9PFLNqyUF#_1CfYxjvfy99ZTfBbLY$4qa1riMky8ERSgH9PL zGK!r)puRMcg>|PuoC5|UjVd7C?gk2)8~|=?f8VfV`wEzc`t7H1+rI7G`DNObh0D~-WOoF_3K_sl(PmTXd6ObREgf?Ap{x*WZJgbrspR>>hrBK5R|NUP>ZBm#u z6+WN+KmV~^a}yU1&=1)U{AK z`W|kG-4k!1SRI?odrV;189wQ5v%+|E$aJ0nw0BvkSkK$;BVZ5b1Yos%JOUMAee2XI z6I;^k)OPRtOvEqW(!zaAAgXPHdJDTtqEQYeZ`$8{aP)=Rdvhc(7((W%0PCnH51mA7 zmEnUp_=n}~D)z5Rnvd&=^AWQA@+m<7sm=;eO%njZ-B8~L0w%nji-XE@qu{LMe1O{I z&hSJ0$fvk6pceNnAYjT95;y>)?w5rsG`WC|j8_U|yBOZ(U&Et-sn7xQVs5+>@(bPKq;*UnFvioTy%O%8QdJ*GaeRV@!7%n}00}4A_73iC zu7*uMA3)zRIO@+1^r|5(;CSM{WN*NdkSRN5?pGK_i)pf#5pa}qBuWe$WHI^)^2awew14~0%_{Dvmg2+iih7cI+Gzsc1|q+x{E z-k$W-FYco8I+Sdu+LR71H_*=T!+WJRe)$$ItMp1xd`XN@epEpR%T_>!8kI$z*VFu@uG7-CuuQ4$vxOBF_21=042&wk`CG5Hj%>(L z#RzsQ!EH^JSgM^Y>Iy4d*S;PkDA9l%hrVE{C&TB`wK=BY*+G&q+vwL<~m=&lYB7aq$b_V)Nd5LMrXbBSo{*nV7sc_4h5S_g;GY(suxoyC+UP* zsmmJ|9dsYnH!O0^_mQ8jD)L*;Jnv-5^S$`h?7pZ|yUQA$J7Glxz5B_{*LZm{Y&Cji ziOU}a?etTpXYILHCh}+PlSNyR%iWS!S;DBwu91}fc%0%)2JfL*Vp+i@&dg~~5&Y8l zK`@$D3hm|5z$Es1#f#yu+tzq6rwfnec?P^89D$ESg)i+!#tV^(xB%8r5o%Q0F2>1t zD!wz3DC$>j=FD zYO3mAjOL9o)6)wp88m~kXkZZXPcOk#LSs}S+>HX|1(cG4~ z4Ic9Pd{FXRO@pntHPbIgB?-z$l)L7yeY-&no54x8v?F3Hz)3x`Ffu;A$bnD)3o*0< zqP5s&J*XQem-#8RKTEg%$SGcOG`Yp}W51f0XKX7=4e5xKy@e4%wdSo;u;F!Czo&i9 zA0;D1)Mh(z3ptJ#`}+E#nh?=HD%)H3XQLb<Z@NFieItj0y|LrV|I+w_1lH?bR~8?TrHXfbCDvV%}XCAOGV|iAL?Q<-I`E@ z2a+?lGQbLB(8d3;4xS2^Od{mb7SC0FEK>{b0)a;jO_E8v`;C{-xBt zrt}jDm)}K4zdvDNL*Q4e&ZOcx#xOtxg&ve1XSxre_)-F?gvR3MMV)Ya-n)Ie1T|}J zx|3VOmf{e+Nbyh9r6Ab@_VC<3O)nqmFy5XIk# z_<7-X3Bs>W3EE&2@W^s6{~N^d0JSOVHmw|+fUEV;z_Au8-mtycef(1JQ;~2E4ECUQ zyHaWU>RrA1B5)|0--VQLTnJoh z)SmpWN$5()|wD4iB)N|w^Uux?*rLlzh}3kJoh%aG(MLXYY0xl9qWtoz7qY+w{q z6;rXt%#b24g2&65G-*d>v%$*J8=kXntKxY7b#jEDSG%3O(ZXfu97TuG;5eq_vPYqvxb2<~rtB!0NF8 zxR2XzuNzs3p!unY$jeooU)4!a%-CciBgIx9-JfK7pe7S_`Xl~OA&ll(K|+lvM%tUc zJ}VWOcOhWYnP_xH>yj%BOu60je9JLk;%`!>&I>##a{hce9PsxIqe5S=Jj}wQF(0oS z6+(@_AA;ztkBwjo;{2&NzTz7At_Vxh-#J3C&`jouOVE6&<0{_85M2=j0H?nAQaE(Y zE*mwsTc*DqS8qKZQz@rvaSlb&tk>)w3ub(IW~O<0j2TPiWko7t;GnAQ>mz&=KMz>& zSCa_$a#4JN9$yF`Yg}TUzWD2A{a#=77&z|{XTC!C@4ZL!R3c7gI=ne_kx-hwXO+Inaj$U835k%?qv zW$nuTO#K*?M^oDt%XiqkcjhF2y+DoyK$ghX;LP9m4I-RCbmMocBZum$~EjD1s#G53NzvqYgHMw)Y^l3bP%V4i*~oh1l5Gh&KPp zvOos7fRG)|)%X3r@T-rW1J-wIltfp1UL5XJNkOL7feIs-PF~1sKB7ula*=QKgcBzd zCy%Zv&FoL?Ke;dl$ZOp|WY{@T?_?V_R2F1Nq7w-Pc6#Jz!JN`R7gY);7=bIz`G(f{ zx5;a;^dz$=mjhsm60k&ko~%F(r!EdTiUgyif-1#;j2OKHJRA1bzp*mr)C#tFpU?}(7ygXKCdq7zb>vDQH-d@dS=A5FSrvC~NilfQtZTKBw z9+y@UcaQ)3`$O1f%!WF6G+B^n?V@ET`%jchhJ-+P-1vHEI1lojt^tJf09dr@u zu%a<1lXF~fX=YHUwkSPF`%89&FXObt@+lX*Z35h*cR?nIm^_5Hu)u|96StZ7B~E6(8>o&lV1wQpF`*M0mnlUIDJJ$LjBBxNAvD(=KiMD=W>>^MP8ndM1Uer2lCvW>i%1%}-+!+D+ zh4O+S3PBuAUHy&e<&xS?qCHfPpvNM<9`C0Y*^l2 zC?{IuTGkO5jFReoEj_XU8HXkaa`7)M-7VcyJ<6zbQbwpykJ;va4iS{Lo%sPA2{Uo- zjz3;8U zvVXib&lV`a5yMsCLI<6}GXTQj$2Ks#?@N{(Z8nQGNB-7jzxT{g1t?jj<<`#Mke>A! z3PYe6g`IsJMy$jEm^p~~pbYecZE71XO(|&b3iu$mT^S_R`s%I*#Kw;j^Zj^o=AnY8Hbx?N~!TE3%76xwHxkySEB`f_j5Pn6;W6ae1RI_QcJ zIp@V*qF&6R4f@;Ius#tMMUbdldbY%VAv%DwqexrcU*YfG=B3MqCH1G=m50i0pX+|5 z5+L%!!2siZy&Vr`$cCZDU3&8#pqLff1dt==oxChK`Q<^I7r##4^9o@kr`ZQoKL4qL z!c8{)mXM#*^#%hslE)Ws;=z)&loz4@i2o-~G$LsC1dKfimxF`DdQdN-F@U-9`p{CG z^I9ld8#ir1;EiAZ?{7Hg*+)c;7$`jCmG#~Ey`9D z`0^bkUE$`lm9~9B^{6ZHIcP{sqGI*7FOPc014>#oPQ_Wz7HLMXv;bWihk;GTWq0bC#gTSuAY z)Hj>f2qj zE(x5K5YLA=1Jp^%pnC#(|1l)A$w)z_d8Wx|WE}N0x;t&SY`E0&Q`vIkuIPlu+HU#Cw ztuKqN|K1Q*DdCFI}s-X_b&-vje-+jH1V2fH7uLv!^GtN+#339DG3{GcMMcV#f& z49g1euDI3jj=Q__k=p zw|>w|xnb?k%7^JZnAw1qcOr;5E#qc{?S~oB*^KR3E`lI~9ikE8|Lt zje^+7VSt)i<@0qe{eOSXupgaN1Qp+8V%|W}tDsoOK~w)vd^g}13_x_`pUg!bjR2V}zYu@{E>}wWs}oD?7r_<>o>SKjjF1C&}gZ zi+?>{gcGh17b-(6P+Pb*W>~Ta_;M^n5#MMjomX-J(ut3!Qg1Q%wWeR+1cLB40Zzly zxLbM#72ZJqmKeo09jLl~cWKx3eQA1TX8(ntz<)N$>?!pZQO_5%NaDqgT!=2Uqb&H_EcH<-N8Y|zuxnz zCN-nlMZ`Db#MQV@gBK`Wx8?SCi>fGi+oBBO?x2A0d1g8q$kFhe-N*j3s_U2W1aW|9 zh~q+{-wa@G!rj>^-)egY`cB%<@hA#3P<~hC3v%}+vd9`RLrFd zuo8e#&%-vusCV1l`G`F4-BfW<%xaqfT1T*+B7l^dH;2bhIM1<kZ{c~7R}JWt?zr#OP>H1M0rx}RZ8tyO=g-hJga@piZC=%y{`cx{z*x+N z*h_YYOjZZ~ZTh*w!zOd__d9f7asv)qGZi@V4kI_)E(Y(n zDj_2QS~TqOJNm~eDpBtAyq`vafG)LfoeAA}+Fsl1&32=6KO7zx(KwvMcfZ}QZXp#-2>yN%dfcNalBy7J zz+ATe0C7~PyYBR!w~%DiLdvsZh<^WjGLnwTsBx!drjo5TzTv&74%c6o(o>3LuTj~W;sG zCv;_TVqQ`dRC`#P?L;>cXlenaahq1%FR*+@hk5SaL2b9DLmTO@?WVqnf#NY_RL*)2Fx2Dqu?jM~hlQ{y zCot{fQ;2Lm!7IcYbknIYGe{>R;x+~}`g+9ZKexMD5Esy#{P2Ys0Cw{AFF842=1wsG z=A%E(9l3xplno?r*Ljuax61eAyqja#2^-AO_Hw^WNG}y~Pos2i3(qw*{SiPr5uot3 zgrgtH$AxNf(~e3*45)Xlko@!Nq>MRk$4+D-F@QC*%BaYE6d?Z)(W^iI47?Lu0r)Z< z?~l(H|Ft28AOGU7vs>B;=1VtzpnBh(<9%Opd(SmBw8-9&l8I>4ilizyFV(i{_4Ps24{ZoUB{2}nKDj;vu-4p@kn2YB6RBl&)EvJ3&8zra6N0e$%qF4Sr;IM zt>`pJAc8jcS#ZdS%FA^gWha;zG&Z*G%#T_p$I068y$3xPP88eRv8BYr&ka8^Gm!nV zS<*LI9M;r{M6|-i#yD{lUH3XRgiKrvmU^1>YUkK)v z>l1$l;ArR%0O)vtC-~dtr|yu!01j4r;emnL6A>}Ea%c`mvL0EHq|yro)4$@9zcK*@ z1=XOh19+PCK@=y6_}mj{V5~{+mxyx^C{``71%X7`l)oem&C{M5q6W%1Ehq|LIte#* zV(+|O_--XJ)h;eUcd!Lzc8^;(yCXJEdnrB^RHKrfy(p?rCjfVD5lt-^UV7*k%Wm`* zfbaM3_x|imdkDk8LSHL9`}Ohwo%!O{UPcwv#*~v2a%Qki*4!1cZE$IJ1Qu%pGpPIT zz!G)qB~6!~VjxeDmiE)xX}eQ=7^0IR6MNn%Qbz%qe!+V8w@?XFdtCw(j&6JyR1(g} z7rkH*O`T*G>hJqlHhx{8=!LNJWWLS28z(GJMdZH7v$ET;s)zq`9K!Or{97$-)*u`0 zYw!FW`xZd>GBPvBnvSABwW?!LLr_L)%$x3CYYI%&i}%(3M2w$+Lh1q;&C_zw9?C`t z5A+_#DYY#Gu|oZ4dR*l>(VG=C?32>-P`*($DsUq8-#GbR80w#L&iVb8Ay7GJ33 zVCjpfIL~T9_sp2{I^Lh-lW3m4s1(A@+Oy!ZK+bb%$rz;)-iL>S3tDV7U90#tEo*@G zGbuc zbq|!))EGbWHGfJ<`o1E0y2AMbFaHymW%x8ii#oQYAT_6B+(8#v#k_0+=?r>nX8*mT zh2inPG0VAY#V9ICaiIgb1qdp-cl0yZ9R%3^FzW&Ycojg@H<{sw)0xvPxPKuN!EMk+ zo|cjEer3hqTy2GXOoJLg%?KAQcrSJ?&uYL4oUoGcx ztA8cKI!-u(n>SV+3 z90&(73bzNki`WETiqm6FRgQ(YnkmJO-?p*F56vL$2`9$qh&{a?P^g`jQgkctYSFWn= znA5zlX1)HkQS;vS$smKJ*OLR~tWF~fQK&BZUZBfb7<0;8pq2PABtmXG-N5lh6;RBMO6u5In ztv8@uKq);$=teljhY}(NFt)x>nwUv#Du9yzr*LCyWCvCGHzzi<{~NIi^AQ!V77ows4qM>I9v4*|o9(-w zuAiQ#6spI7^%+s)Df~zFPPmf_ZocW>d1w%9-!qM04$H*)McFs0$>g6N`O^d1Sn7tT*=E=V{jDk4N3BSK}-_ z77BQxK}8{-$b;8uh)0lIo|3_T`pi}(Cc@RsL;nf_*i^2SF3%i}pSqpnaA2(j0LgIe z)KxpH#!@{p5yhajui_Qr>5~y&&=}&r8QRm+gHYln@5rv&=IX`K9X|RI@Tnl9=Nf0r z+qYBq@UBLvo9uENPUEi%+EtZ~HfYy;Hi>vUpi$qto|PI>f;%TEZT4nVrihBmNvh&P zU2*Kn4|Fm(7&h<_v|Eh@RODd)E1MAF;>G0X8Q>s`8%}~a1mS+0L*+SnKs5A224RSm z)}_I-2A}ixb4Hjkg1s zfa07ax3%%2cx=Yq?%-s)weekwVZF{gx-o{ibrJbXRwJqRANM1b16M3MyME+VDELY2 z4pCFYm85{4vOP!h3fKKg7s+i$RXn*f8ft*Vb2+<8^JKr>-Pisnh=7Pxg4o5mA}o(e zXySN;?H_wQey4=1o_x7l;3xbdr#e= z0L)>u#M|Su;_Qo&&WLPi!vjqP%{}{H@`?8HY^1TCL^B?d@abZTze&kkY<;5(JbBYEOVK)7+dy<>67`)=D zlK?OA>?60t8}w@J^{1ut^3rNj55 z-;fG^S5lI)cRTUFvo@|JwDAp?LDTs0GzlDDW$}Fb(4C_diY+Cy;77!vhn|Puur~NN z_y_mGsVGrb^Q2f!Mv|3R^Q}vU2>BnrQ8<54m zuWGN{K8;mPyae+tX(Yp$*-}2E!D4ys-&%dFXCFNIs5yDEI%MjrUl<45r}s|Pu7x<< zW!~Dupb}rh1drdsa@f4*d?K{nYsNM-BZlCRt%~VXQA2|%Lc=}^#T}d2T?|<2XA8Pz z&MB5b^PD39%Tdq!<%I)_>&Z?G(%xL=Th7mbegJKglOEBDVNpx3X7v$zRZ$@gap?ki zM$P)}a&_>zB9ImYuG=}qT^_=cC>4*b7j#y|f53(VQ)QOSYt-hC zcrD}FTFQ~9-A&Kt@*;dDS^_a$9~hu!*4&qwXWh!+zF+&3RA~g6{)Uua-7H9}#2{E9 zN=Q}vakMYNSD_zbVqMv?T6}l&kS-r*09Q0X_9Ipe+f#y6Qh??yTa zq#_>CJtASyUGixLDO8-?a+2sxfe>sWXn%$ZZG@&%T1&hT7{i$i=yAZIRA0b6*JYOk zh|Tn|S&+{eyZZraz#kD^ct}(BYDxwKbcf?D);31#V%|V298rAvSGWT+7ruvD(Dd5UUNx ze!w~Zk^|{ldlD^lv-$X&O@mFHO}N$i1$k(vd*1MKFchab93*+WK6jp`>QsAN;PmD; zS9kF-I&3C({}li`v+dL!=)n4_p+$+FvA!YZ=&I1%5e2yI~^FJS_T&Tv2DF(dR~ z(7|GIP?Bnsc!h!$Zk-=p_4GvvHmo!!7yxH&cs6=ufJCvsVT-ejqoq5B=yz1D21x3~ zd-3Dr3E)5d+IxB|ZtYLk>O4_||)5qEVtKY^+3qj3Pz=LEQq z#NzYad);8NxsE7;jMH?(nMc0`OyebIHw-yfnox)*Yu+%Ly=T`g4dKx zvrZHhl$yTN1+?aLaum9&-mns`I_b)H#;?*z{GvYaUY-oj&exL^cuDLf$hDr#zTPs* z+A^rhn!4GYTwYa@T^wH=?--xb?JO(>>D)<~^hadwHLFf^plE+VY431hs}`^JcMK4a z>rM$icV(!|PhZdcz}c0(^`L?W)-J0QNCim__`DUGEsr!DkPJ0F86d%&qH~p1K=yZz@<^}bc zn-?qa6yPq)nMI~px#i^}6{RyEq@7Ds3#i`CDR1x)HbF(UR~f5+o^iSjFL^FC2|TSV z=^-Bsn4clBd%rSbm@3P-bt%uQ@M(NonSyNkc(!JlQlaUvv6mkyZx4=_GzW!%&L`Wu z3)hznbPq|KDN8H-UHBb>Dj2e=#&SVo;^g&iws!l#>{v$&rDY}9iyFiQa|czaR>f~W zs_;rck+~yfi@?P3HC=zvB_XO$O-GtKClQ}zpUC9Ch!R>H0##8&AhMXAzFFot6Dkf0 zv9+}m&pWM8Uf{HBi*dHkW)q2tLG8ubMzFCLLf=q7ddVjDYDD<3-#$P}h^s$F$jyL* z^fn^Gz@Fl@7W)JD(fxL}CNY4<}u94}6z3^L^qoFe~hp-?bqoZS8z%b0U>j=8AgT=fMAbKt$ z25NDy;p3@TQCWg#gPi?`GpGo;R$PdM%QV>-^X1SgS)SQzWFl9XUi>*i z#}2Sy+2yn9EU+w&q~!}yz0ocPGn_NWH=-zTN|Tf8FrYRqg*DQ> zwd(mgwx>I0>#Mmkb4=>gioLySXsnLa)MUQ4<6QRcc$rluYM?A~Ui}V>HoQN{pkH0w zBGtiVytoXLxqPHMx@p+=&<8v}unT(H%Szd^5!||USN7K&s+OOvY-i-Ry)9d>%)jHkCtoQ=rqfX z7&pO^t3eFmY~sGyMnLD}l?u8N22W9Xelq5(|CF0G#ItNvHh;oQ+R ztOM{h($F~lE^EdSU%^X0rEo|Cw zy!qngnx6DY?KNyvwec7xKV~`R?y3NF?{Lf2mUo|o9P1;#coBk)U02w<+}BXJsu`I>Eu{4jkLpI5MA!bNM%jx+f4vi=lYNb}*bf;B zzWrv>gB%cS8>GZyn|oo`JjyEq*D95><=KPt^%J5=6^fguwV8B&|T%00uOmoGdZy^rXM)gbr2fE?QB% zHx=+#)$_S)1(~wQEM;G}GhnA95M5KO_Nh?Jd*x@4EQDdD70r+255}DdvLr3}z1TZF z2Rn<50;P5Vd~HdeB|j(Si71C=(PyL*;gQK78ir!?DJeJ(FAHvNZLUCC!SuL+4mbPv zVNFfFUu@Z`c2>qj4|dng<|;b$ZKND581oPNEu|i4$E6`6bwv^P?$-3+>P>lg`P!7= z7lX&~>h7!n$k-i=_=2(N+wNqBWBAAso8S@0={Jd9F0+hUp=#zObmv}a)L0x|EsBG) zi6S;^;8jQ-OKG^t)$3;?QkB_5xc;355C+CxY0crk<9FG3ve(`?dE(Uz8sza~OH?6FwyGu<)=A;)zpx>}NlgWAxE7}MuokiuN<{<3SYC`P^~WIxTThhD zKqNf;G7iH3O@kmapQUQETVlS85^{djzYML5^m-W!eatF7dB ziX*im)nvus5fsu&&^PM>4_ye915sXUM-gE=X1jQjzyIp|F_73+ujg)A22orh_KBdZ zmV0}vip~Ix@4Z?A=F^G#m4ocmU;Ps2UEe~~JYObtx`VS-9E7+%D`{^xndlMMjrKt# zW%ATuoqu8_z3FRJ@~!T7jW?sD7{s2`Y6?RkS(=x80<3>*VlzBL6RUlt{6Ork2RZ{a zg8KHrliZ6iP6c5FQ)=j=N3jKq@1){+#t$=G$6Wiy@{xCJ+IqI`x@)iIExBiYHCStX zH;;z}-I1QOMnL$=%1pg1N%<|SF4D7)jSXtsuM8Q*M<9eJj4&zqjkvx`6X=v!FpE&m z$5D%@mn8MF_eckGf8&MG`=tdT|*sosQ9`B{4P{Yqz z!3hV?anU$0m#T{oy%D9sfe45sJdk^IaVmhB6SUT7Bn;!bE{M)mth-wL-&Nd$f>qqh zB(DE2VdW(l%V-b+d!OvoU8ry@h%zx$Ia^(?sk!M^%jLwVB7gzw>guI+MZHu8yLc0& zvhcxgrJTjf7Ke-4ui)cHTNw>YTWV(9&d!sVP{grU=ZaX%y&a(nM2CkWB#?(sy^L;_q5ibW7RjxPZzA#@E5 z>qfdcP$Ovbe_uEY^}<!X`z zndMoq(U%aZimu{tR|tW{p3G~Pb?9xD_#8-R5W@C0Bc~s0TR-%1BhrVzp@yW@O6b}V zzf=V6)ZSk4OTg}6jOr#lUTvI1l_&C?>`h;6cRYuN8+_}QCz=mpOPh!X^FRC65bCjx zz)GQh9L8Y_-+t86;*RND3%!5GJR`%GR>y~%iw^D*nrA&F0JO>m|AODE@r!Qwdjbv9 z&MaMYLM!j8(844Q=k8d-$;rtJV6~uLvj6l4x`{ogYK)4gSkXIS(>zZcL#kb1!ZcGZ zs?u#f|Jk{`o6jkrWxEIA@|NJXGZQk!Aoyn7TT&uwgo86K!J#lgxS@VKRIZQa&chdL zuY(#q$*E^11t(AsPs2Rl^M}qe2IfzXnPIgAP+DTvo!wnk=oBDuo8VGYRMZngh*Q|T zKMf@L&sqHET*k3pT!RH&xe~QoUlipU>$+vO7K4w{D=oa}&G;%N#?fR3qs83Re{H4b zzIZh+%>J`meNkeR7-C@`Hw~+O3s>Sb!Al`Um&?uv$I^h$l^iO6=T7(|0W6K573-!% zu<6ma`1?0w+lz%@Sg3B0YvcQm0&P>;L_8^=!^6XwSU(f}C^rvNzkJDu)z;t(jkwq0 z{^#V*Ui!#j3|1Uy5r6SmM%DDmKDR^LYt)YFQ+R%$!<)--h%;Y~0BIdq34p-UAlm<`kaIW(JZu1*g5OO00_!4iTH1hYN#+_3QXVXF6I+SqrLm&XY0 z9?m$=Lbp4c{Mw(d3&Qn?e4Pev!2F-xz7r=oyR*_1zNB*W)O+W-3m%4?jUV#lQW1vpnzIWZ{H~)Mg1*VS?=r58FqUM{1&5 z!h0~dYMYjWzHUXZF?ht~^Wx}}Nj(tmVsAMg)^O576dUtiT z$6%#v*$pZxu3NFTZ;4=nhlA~fu-dt~IrV7Wm*>7&BnfKINSYfSqg0h13hm7HP7k2a z&hs^ZBEmab1YmHfbV7z7(AZFtS zwZTX{X`hV8Vqy~SQ;%gEDkcYi;spmro}658h3zm(NwBx>1-*&t?7XjXN+7SPWZa*k-j|$}JhaL5&OlIhHu)t+)z8}E#Z{l{ z18R;1!%wv4c@B*PTED&@^V>qND6$F5SuOoWaxaw?msnxRzTL>J@0)ymRL$PQhvG~T z&>NV$o2fIj-z(JHL_V*~Hc1@a;p*+hl~&Z#DJeO0~hYn$k#a zeay`@S4ECXWxC7C#>X%ZS~jjfaCObR!*qv<`_z0~T&9FoVrFBO*4BX9R;fN_yeg>G zp)-K5LxS*RSsu&Pz)fEW^p)_agJd*`8cmX$W>>3ta z)!H^iJ>MMAUkP!W)vdNKP9X;fqpy(D>^MKKr~BUYYL^+)@mN>(z{RRy8 zM6r!>+38hSrXmd9gFFf7_~GIhY+1=?UwBJYp3G|_Ig_c?J~?Wx$Qd7w7^PwOrbZMm zT>b7|;y6*4g+%&Wf<+?%B8^1>av)xoZ^mRA*xpn@)`#s^9{UFDmip07O^NO7CPP-s z_&W>@l(UHflc%xoS(cE6g|tpjUl+4YPfuqzQ7U>Sr>O8c?RwQP{xA~We2);xOD0Hb ziA_Y4M30n$_tu|yI$qcA2~|L(?u@E0*Y|cJ%_)#h>EoyKKU+`uJ1iXU`7BTgVNk7TBB=f`d~$_#r!%@!lNp>v6~(MnW!!3WRoR#J7j3;TWJ zIs#oKOAdyl(r@2r*zP42<%p8Ql}(hhbEK*-XRxn_ZWKbM)AEVQp=xRo2kT!)Tw3_b zoLscY<}Bu~io^Y6d&;2%*QZ)9{n=E+UY4xGvBf#CrY~dqNk?pB%O!61k<+}1 z{vpSbE)-}07ZJ8_>;ev6g;t94&UJ; z6DPH|W|CJ=N_Aw(Zw9$qxllQOCtP0?d{$JbNDgJxZT=X0D{esN;jBU68W+}yW#!fl zhmk$&mnX$oP^gTXL_OX7nozLC=Kd=(Z-k|8?Bc$x!>N7&^A4`8%Gcb7(0A;MNU*8Zf z21*Xv_4NR`CZ^*kyHr>pf$>PuI(tgc^)qtP#dusOR}Oaa=;)M-=3weL7S2iu%M7h$)4trUwBFuH5m?>ywvXvt)i)S~kvL z(|+JB4y?D=K)uY#8H;Gm)b4moasQ8BXq#o{I36 zlM0um42YZwOz8P(@2&wnUBgUPauKX~c-<=qu{j#H(PD)SLp3U3YWDZ`Mu8H7|Lbl) z5KEEHrK#u{cy-Q<<&VWw3afo4 z&X!WSiC^TkOL`M-)4VzNdKKQ&Z%>~uL?3ALcC~1Az>GS*X9gQehQoV8W2HS>{<$?y zZZ$b}_RwN|xZ^!Y1hS?akh{=By=JJ-M6+}~TxJqdKzvXY@1 z&k^%R=$5`&L3Z%Ml2zhwahl@wcur)C)P84kJ)tB?dAOVq9V0zcyg6F+{ZbIZ{1wFN z6|D}bTS<)UY577iNBpJL=={G$6L%c6*4 zXIU_+pO+iv4`E?gJfhz4?vR(a+Y!l6w~KnDZ#_R~wf;5*p}|6-n``mWMQd1CWRqy666(-f zPunYw72ZrJ7Ea^#NsQ*Yt<5uvwQsEtQ)lI8(N2|eWOBB!HvRbNBX6c2YkAmR;!Y_Z zX~aYIp|43j9gOsen3_wO{WvCieJ|46!@m3EFjImTRw_HaZN2(!jdlEJN0H?q7lxRMZu(-yt#%n3~u^Vy!LUR1%Q{Pqm^tqn$6*yH@x!{ zJL_%RERy#h?oi!=OVt}J7se9cxHQb;^h+QfejIAAby%--%dcR%{@rHv0}(9fmHhnn zT2*79kPK)WkMEbYpZ_9`5s#(4O7DFa+4f^*KS8=+l^$XHCH?Kyv`w9R8|?*EdxCo! z_yIVIZTyBgB?)Z^v8_40pB)p*V>u3VMMbh70&l>PG({H+?2A-Y9o}UcDWFM&KGOS%!sK zAzZ7Wh|7TYT~Cysbr7QyM3)hG-vs#ijWsqUUR=?G`KwSLEA-!GCz|b?wm+;LV8^f@ zA9GFql3zbBuTdc!nWBChgar7aIRiw;8xZ54hyr~h51851pUYT&x0P?sqr2P|IO)An z6In%{<8675fd=DeWI-mj$pn2dx(EgjE3batjBH5;@>Mzx*${b_v0PA~xogE(ZoURN zg;+;~t)(~ZR8pOjD@F82cLdR>0*&r%+>$8s;Y!KfZjW)nUHb0-=F7qe~#yPj+sB^unhBlzh2kty3Xr7&nwK4 z<|2*$+3jq>=8@KG(a}+($HiRKj5;GSkv-pr?Z&7eC2o-4`}Z2g8%!_t1G$+{le zi@S4PigK~{+DO94o}a#4SYKC)pPu&>jAm@x--f(&ZfanSv=+n^0A25uNo0|(3B zUZ@NZY^in4YQ8^&`{1D1*{{sc43?F=g$n4iHWyCh8=seytUWj9M*1w;Duybv$u7FI zlM*b45lb?x&&-@;0;TPv0Xs+?n@**!cMRGB~mPy%l>FH00v|3Xh`g0Uk zC?ZT%r$*Vy-!acWz~2un*Icu%x82^`-eor#V?_FxPDOqLq5@=bgChT9SHN!;mAd8b=`7|j4RmMAKx9<2f;MWuj zq}mSgaDf^tUkEG{NZ5R`%2`4DWAphN?>7``a?GT6ONaLLu?{IiDRu`)C8WR|w54U% z(iL%}SM43XC)fRL59m{(yu2tIp#PTW`k4ZY(6l7~&LVJ73)-BboHj8*=Kz%R6x=#T zPRAL>X?ov7srA7lKSU$;7gd<1P6agW0xq^+0%ozy4om8O>ZW)0HN)B9YviUkaI$a9 zWm}Qptei-C3>>(#nHv-4fi`n^I2FSAEQB6|llsiiOE2aAP(#wkP9Ra?8Tl6dBbAz& zy;;Nb6K`1BD5t*=-or{NZnT|ruhTvv@6wi3F#>6&HAcN?PXa$Jd?!&uIeQJ`?7Cti z3)ZW-6=E#AG4rX)y_3e*_n8`mOOz+)uEy}uT}XvXXRMH`n-6iadsk-syH3dT?Vf^* z+X}cjS%|)sHv`P{f#$XYV@a;029MloRrI{7E%Um;C#JdYPG5A3tV>Azh2B*@+9wBf z*hns96!`tfvAMLQgSf7bq&=u(I*rk_=wdjnNg-j4-Y>5_^o{4I;{R`-gxhwr@V!4 z_?piwvL0^M`{8=#uZYTm28O=G2bjQ3yud`e(Ygr-idl6TddcHU>WJc)ICdiT15=nE zFd)_-9=pi+na1o5sRy%SQyV`gV{i?60V`!?;19p-Qa;I}Pn!5`7=U6RDWzc1<>3Uq zjrq=a(+NluEv2ceThiXs+DGI8A0rmW@@5K3LdNEZv~o)9aBi?G=MvE>VaBTls2W;- z(&oQ=)C(y|PjA`TAy@p4lE*N^=>H|Ooff{XKuXdTMz8t@m>YH4Qts$0HTeUa@Plyx z{T?XSs}P_gpXo-XRnp@8GeOL*su^k+^m#U;Qjdk)1z){DYBjvXZv#$x;h#-IhH#Sw zG6g|pdKSpzJ3pfpcy(~U;4mGW#)aJpa(T*D4Y6_bqn+7?_TB!4&sUQm1*jfi#86w& z+6Qk??KB;nW$rSgcsLszmS&g{yX7wM#{9emKLrj3^(pia&Hj5Ow)S}{QEDG zcgrc8xDc0W`CV!n1s8N6(R^Tt@i_c(U!E5SD&WJQAv63~K5;R;WyCj|B0>OB2x5ab5Sktb}+$ymqjT8B1F76dA(a z*p`)sAy-$~ieX5S%sY1&7F3d&+SC(p@~=_rEZvtdAg3E~G`;mloDv3#jfYpCi2d0{ z*B~bkBD#fYYqG%*80j!oAXH(- zk`wa#6qE$G=M|LjoA@Ch z;$Y$Ct=ox$_i*Lbxp@U;x{^v|sEG#zut5+&gowQMzL`wktzh~$4m_fF4m8H$9Po<9 z-YYhe*GoZ$`S*9KEUvwBv@UiDjSvxt(5MN$UTzDAmWv_WYVRir1=I>XHtnoT@&orT z561Bb$>`2CTT1o36Lg~A=QAwTgPS&HS@^j$8yNa=Ez4uhyUA7Mj~Ug728>yxqXMTP zl*Gbgzq61S)7~tM?+fN5y@;5o7A8?`i5{5+vKYmDEXnR}f};)A&2f>fSrVi!B#Wbr zK(UEx*Ve|^W}7Tc)ikI`;IGdzXH8M#WP~bZ)+YQUQq>qV=0?@36DbGLi+&{0JYBsR z!NR$B8`=rC-{#P?mETxG=Go0kozawND9A~gtv}DX*&L6z39fS7ZmV@Rz0XW%Fr+w4 zKEG9Veu@u2Tytxpy16%k8<86K9S}Gx6qRhC>?d5>*_zle^hLFV5_Ryb(S%UNr4|2M zX+PlHI(*_2?xqjU6DSA1$?#0cI10Et($b1Nw9R!(-hB6uKJ@=md4L8A2Q#p$EEIyx zPrv$%>hx#~M2+0}BjkOmGiUivcx!9><{KNO!II2NFRA|!`pD3i~(exylt;v@#Xr3iC)p{{u2 z%gpr)eoqJY1<5hNfnSs@{fQvX+PljbQd6y3bF)iyz=sPxUYhe2vKTS2K=YAmI|bxo z;^bQ!32`jAq@I%@a|Hd%CL|I2lyf2Ft8{ncn*d^jKRdV7{qaXQN<~34 zk!+1RXH#Z;pwiZ^%%M*b6Q*%Y*2b*;bw{(i7a=fYnzH6v+GPXo$K^p=&&I)>pWmnCE>28%yM|-h z!vzOlUKti>HD9~Gs5Va-rq(PHBDYuQ@+#^|h4lWXOW?HnY@Z%z@pWX459z}P7pXC_ZD^Lr(_jt>#$4N?B+gmE8YtRRwxfFgs z_r%&oHAk)AZ9Iz_$mAuK*+9_%mm$~CjZHH7dtUTY=yL;iX}Gb9-bVPDX{WOjEL$!;Y6Os`R~aEkGu4Nb zi^~s|yOpSvOAEs^ryQ)#q!v_om*ef#>Nsaz9cx*bm;m!sIJKK$my9$`eY@Kwn!@DH zEOArNQ^$t1wxZokDRD5YAby;Xw4=czQ=%F^2D$3Sw9eHennzv{+zo> z=~7Vq0P|=DQ=G|~3T^<|P|F)ocS*%CP7>(;g!gmfVjr-!^A&_k3|RW?KLLh9q_ugg2QR(xz~# zuvfGBWTj~wM4;c3Y%AnXe!|bg?ud+i>9Eq#oAa?e=_oBejO9CC=M zot@qC*`j|0A0XuX?db&2W=tRA^STpl`fMGoHorwV98_-zO#=UrEvnSaOeyY9jkdC^b=9a0UUhRERJhGNh~A=#N#A( zC!8=8+-bq0^NN>r)lz#AQnk^E`;&2Yzhr_a?e$$s|uuo`7;MqoJ8ux5$-|9Y>m5dj~Q(z|SIxPY@ z4535&!Nm<8ZnR7l;WzxFm1z9n2rgdk*K+GDO{q>*m`=840VOh4i?iWGk{@%Y?MDaL-f%K&#o_J{IhoFUsI-%@ssc!DEjj>PqCKwH zvgR;Sqt!6RdHj+?!{M@@2i{;@reS2?b;XN=3d{W5;9$u|R&w0ZwTjR=8b)m>ASy1d zieI}{Z~k9UTY!=V1QW7rd4II4pujPwryOtl2SG>qLoDbM!)eMts0{F$6=#1weq^Eo zPL@cw<&4Zv7uiBqx8SW`!m3a6EXB{jM4dg{ymT-1a{AHt1Xt!G`skA?gZ*+ zJ^90ciMAZiUG&v*6+JEd@4&IpT6Xo<2?eka$B7%z7hB_F{=eU|QL4V_>x$3c{~UCY zCc4wYrv=swNXdJ52(ARnQ$&f?(yfU5_2VnaIQ7=rRF}zWDW->GPs`@%I|5hSO}|E^-y-TqUM77^ z5N|aH$5m7M4>&>M0$0@b#y2Jh%zH6=ni0;{7pvGCHW(cRns==q&6wd685S?}U~uT+ z{8&Ms0>)%46=c_2+g<@9RTyTJM76|8Ok75L`O-b2H4C@a?W~q)?XuUi<|T`+c%6Il zb3yxs#jWW;4aSn{F(Yjm?{L8maIm%iU3@BAPi2n#jP+b(lT4}39tat3U!A>eeLkUd z(!Sbxj?&kyuG)-GYwo_TZ*J#}HX?E}MX*+qIKSxJd>5k!$hGIp{!8=g3l_+(5Bkgu zPfwehXMm;&^_d8@KNg4(_!Vm;DA80aqG4*dYD`-I>`1;~%mqW@AoISCnA~M%o-U7dwtWzZCHZA{ zqKzKxp3{6@JEwsYYVzYv*qhiw44#+&An7juvkG5gf{>B{2L~r4Mg!rqbRm;+-f7L( z*>*6fo$c-0>VSZjl85g{1)M)TSz?q2Z z6oz-08*$@mIbII>tX|Vw{5&XWB~`me&I*CZUY&=oJFl3rBQV*!!t>zB=HQ;cpSz0-2)|N0A0_Vahmaj z@5uxGY7UivZEHMBYxP*=#_vv(o}~uHnGHsRGrYV!T)?Eb0F8}8PSXzk@?sJg{aL*K zh%F~QO^VG>DUo&<0`XtV&0Y0PcIj~Ib>S$t)NaL%HY}iw23_ukInXr75ZNRU0cNz; zz%p@Nsos8A7}&dcW{#40-c zA2jN}9Z<^8WKWsFvV}a0NB~F50s+Z#R&;2TD7TBmjm_{>5-n`4aF=D zcKmt7X7>H0YrDW_U{PR6rz}MADn6r6@?Wbpm&{0F%Sr-zq5-IirYJ!f@G)*vQ>)m( zi)3`LI>q^X?-jo8k}?kUlV4$t`Jg!vZ0P~(i)x9{MJVb%F;dj^*jNB`7rRzX47-JxDB&ko<~jNguIC>a`;yuFhc zhx9i@zV{MjS~WoPD_n3JME1(Zow;E^x4Kkm_SqIzh;9^^5w&?0kbx7JiyZbn`Sc3+ zN%GJX#fl`2HlCO^pI=xEmf3xODPh%DDiYEw{r#QYGW;~`mPdS&=Qh&_JTew3o z$@Ol$`d_GmxzcB;(_8<4fBKCB3nZ3VAesyK2|xc+_QbUE2*)3#)m;b#1cE>jiHc!3 zRB8ZcTBbGRqkQZIA*>3&c3rAOM{^RDFUJ$V>`ZMQ{6nocmNX)fsDMB^o;Rq0Prv-Q zW_NG@g%S>*W}b5?A%A0Fp0sxfhvK_I?xN`S#33BM)Wq;0lIq8fccr@?lhzIYd+?yZ z73^ntpcC>UVQ$h)2ug>U7Usc-kVi5py__f(2gdy=W?WDE02D`9MY|z24UFA7JZ*Uf zpl<-w1GisvdY1L@Ma6t>gTvDMj@u#36p8B`gZ<?i!5lea8CiZR7`5tK8DI`sA9zF(E+qT!tJ#TJ+77Dnyb`HFg>wt>t^nLLzc4QU@G%wz^rI4tOg z2%#s3N^jgp_>!atCfsKmUbdj(#whXqXpq9uN8MdMQ}A3^eqIk zoy4*756(RR{r&C9LIf z{)+;fe_B63q+(Z!r3p-9K5`vjQSGlZD32e@F0YljnT+m?3NB+qj9NEed)AoYwlvt^ zwhDHf_<;HK9pfED9C*)hZEP~yX*Tt$PK%&wFK99zyrN1A-Fr=s3FZvygcGNQSS$o+j{lCoTN56$jGxGIUlu z+@T2bz}~Hyz9kU6?vfm!jXE~b45NaS&77|i*Tvy9K6D~+zj|oHaGr!rmOXHX%l%LC zd~mj@TaShYSRM3sK$_#$5dFZ;c1lQDT==3&uu(~Z|Iwn;)U6p!=mnQZs*Lfi=KI|Y zFa(%FQ$zd2F-@#WCMHJ82uEQEO?#2v>h7rcswg^~9)`G!@g|3+@a8{?$Qx1@XT#w+ z2Aw0R=1X(f#pUMDCj&5Et4n#uCq?iqkDD~A2`%xA<%hx#)oQ>lo$uE=J|pMb;P)YN z;uxn%;RI?_t#qU0#HG%0`zk2de&)xIet-0N%fX$+&kSWIK8{}8iJlNi#Aa;H5gcmD zAMt}#c0lr; zICc}v!uaJl@7@=OgY3Jn@+iY_d7nQDTYG_03e!-pxkJ=zcCTK$Mv0RtDJnT`I*JBE zO%rI^7xou2SRnPgpwOVL?whwDC8qr~j8kX2M29^=x?5;L>*J+;bwziPpcK{nX&+~` zUvGB(JAw5snL7}Jg&Z)pf1K&RIe~K1=TI-)I1nVg|H{?$5MvFaK%Ay|OD_kF^Z9Uq%Nswh3UGQ*>RMY@ zM|EeO;0qoQUQobp^7=$HgGu>%lhUczS`jy~m}A@hsT-kOQ7z~4yX_s1{qJ5Ho<5`b zC=hd}n>&>+zy7z1ECnC z`dB(^erAs%g7r~2{GqFhyD7q(7QzO%R6$*LW*yUL)F0=u`CL&7&dJLfmK?Hr3eFJi zY#83f`~;b}e-of{!bk`82@GeD!3}9a-X&EN}3ge4_p zo*D<@h(_o{Pms3D;q==#^wwGVGOZ2DOvVb{NpP5jtNY_B*)>{3Ej&g;hW>?I&%t04 zmUPsyaJVe4elVMm<%t|~K_3NKxqD+v_Sfl7_$5LZo$R` zizk)&+fTRFpH2@s77;=O>kKhd@f>ZJ0K*FxOxeBugJ;=&xt08tK!;0v8(u2G zB~g0gAz;PVUx1YC?_!s~zkVwRB=_5vU#^@6X$}D(04xNCzyEXpha|q@Mc#%ewIYLY zkU&auMv)|i!}*|6yH}iPhjGh}O@4wb@mKxxvwa>%rH?6}zdy1G9M`;F@-Hkv>*GEg z)@Va|h|^^gi!!SzJBnr|&}~tDg5nbIco;7dIdaYMhRqeIDL!$sc9uO#S2L+0#kRpk z3S%_)qMvIeNcSl#Q~L{L97wnH@oe-$&r=eBZ#K#?@H|$uBgz_v$^4QOFuj}1Iopb! zrGjV{8nu>%e*P@--KTkV9MF8LUX;JGcL{MOt3xqG4nNUUtr6+6a#_eU(_^F#`|8pI zQ4{Hc0@Xmvj{B%ujCRmUGS}**)d<_sVHdwmzSVrZLE>Gg3VQ6Us(7N<(Cv8~Q#Zx# z3=RZPOF`pLkj{BWS)GPZ*z(h=Mx0CBYBl!Du*(H)zj&#Q?5%2l>DQ`(xET*U=KKgQ zhM$3*OGmBl9Ocq1khufXHSIc_4+m2FtO=g2S+Ev5qsf+5&jp!6yu8NG-k_^_YZ{0TkZsM3Vsk<1v{wZlu=s@3LQo2Wvu0a+xa59 zD{4Beq`)H_t!Q`-b4GL?q^f=)^Mw*}aKc2uSZ>g<%cd6|whQ*SwqNdy*Qdj>kJr zKOTY)+fb@6f;;_gQ%E~h7ln@#9%$rKcHLji!1#K!1~K6-a{Jq^}6JV@r*jIlMA&Q1RG+d4?u)mH}tW;q}O9sg0sa{8IO8;x7u* zJ5?!;?rbz#uZ2lCxiilZ_Q$+HB+Gm3^*7>1F|l1zc-$EfiDu;vtY zyZ9vqtDak<2JSmbrhfSBl8h7lxZ>-}_l@ms*OJbhk=ehGl-I7LEE&K!-6x1|2K=!6 zi1Rk^8I>h>Ms*C3Z^Zz!yzRsdj1@!_v@@)Bd3xP7H-NSWZs@bXS9Q`iLcAMcM_)eP z*tkOg(gSu1A}L)hph%5l5bB8MVaMDv;xa{6B}Ws+VnXqRo75)L#US#D!5Y&zP@Bpc zryVUqa^+V{Vu}m%WpM>>_MQqEg1h)ASSUG+eccEAJFuOiO2eh3l?ZEfDZnXBIBBlY zhETI3f^0}koGgqJ9+Jr@Z)T5cw2|9o_Pzk)lc#NfID=*%_K9WR)V}hq7+1i@5IKj4XX3o9G$%rhuZU>bOq+=*H}wC!AAq zV~K}K?SV3Bq;ne_$cs`i;>s(scY;2vjv(a<)}QgsbImf#cQryqKxerPxS=5ek)C`h zg+7^k+ujasL?M*}PQLdYz3HXS(AgHwp$MDgrTS>Ie}@gXQvif$+xLxx0#Lx|KLikI z{f4_NY=1V#^10qJF*GG{Va$`&04*Pn7&u;kJcr|6;xhRRHMwB?vNXZyO9Vl$H5*SFD$mq7d{8po(#fQLi{G0tb{@3LhU1MXO+hsK`(k}qC zEV_H5{Q4#=29!x;k70*TczVJ#*Q7q13XFZeL1&UDT8$9(4wAmTSCSg{G~KY>%kNp~ zS>^>@18CWOKz?ifc>E*5K5VzO#`+O7ZPJFuNXm7c5+e~5YZbW44tmpn+LbV$85bKV zOEnd!>cz{@W{Hch3N26De((~bv^<|004?pmEWRfb28Nh3h8E}ZQ!{pMGL^7HkzHIl zdCAd+w@kaTcVkzpHx`ln$C){y8AAa#8Oc7OuKD4_Z}ohZTFrS3 ziwad8Lrm#0z}nuoZ`-asd($+a2Ks8iI~8S1hYSQiu8}*ZjbnOjtQbz@XXJih{3+RL&Ij(AlK1$M$w$%O_&EEMLMY8mYOb0}g5%Sx?2pi^Jm_4Y8EKG(o0;AZ z^nsQ>p_{{jh#v^Y4+%#1Z{) zH2&hRHA(ly*>S(}L0ZJ7Luj(!IkOrcj)oS4a_^sDRes1fNMfC^BWlVmbib`;{syVt z3wjNC)2fQ4%`AytaW(x2Ax*BSfoEBVS~D{-m97N;27`v~CoC;|T+`c1%rDyuU@5c( z7(sm0Z~?A%T1O6pjWlBLli~B!m@0UQ;`?ErPvkZsj!QmwX{sb}yQ}cBAHFYWO}P&p z$IdEYn#0E0QC7^YEP>~io58 z6S8%G=2J-7Y?xfkhM0khV_QYT1TTE5-&O1|vnKIsyKHpRhxB*nXK~8Vw71Xx6)yIx z14{c?afH|(dF8>WZbHr{|4)Mejd5}=Ntg~5_%UObZG*Q`pItFqp~EhUHeu9pq^}DW z+h2}K1(7FQQQD(+69nnqslb6DX*}vl5k85@zXax$DV318zO;o!YWzD^`9!j_T6St` z&)s)u85Kcu21-JM+eizBh+QH;O?Zos=TK4jF;d!J`sMd8idk-4h5=1>cmu~?9>kA@ zWjt*mkVttj)z^mu%@VJxOUR{N0u?@q7WJG>>#Olh%agfj0Z@x{3tw^0BG()43hL_w z;(p2H){$5ce8c86YWYSZbo#Wgo?=cI6Mn7ws0m7I$}6tllq87m1OTMr`zs1(sG8^y z)@Bwn9vs|LCH(k!kOfWP0HDuEZ~u5ibJ2rLaIFnMc&L3&AM4Uwm{hVQM9DX4 z)f+dYupT^~-Z)m1JP^m-rZvR}P~Uk!+x!bA2ylABPrD9V3Fr#;u}#duaC-RuoKdB7 zj>PPFEC~=Mvz&EGVi6zu4dgvvlK!WNiR1=5BQHZtgF$V`pJ#=_7(DOV-ccY(y|XuD zqG$^M7w(IEpBtf80!##~8>we-H`jeMr7J^Glu+dR7>0u;&D+#IM@Ip(vOm@cbCi%9 z)8n@B88wu||i~j0s zz1`4Je^N%}+#c4+V=BpcZvfCd3%4A!N>Y$8U!059#`tI*mxX~ll-pb3Jw9hU{rcDb z73dui@_6B}?B9~I`DH6e3Z>SMs*Gc<<9ii#W62?1<=%KvQ+|)4-I)fgCDEX_zF~NA zh9oGC(`F8IH*hZq-$f|f@=*+l-6?zB#40zWtC$wce*CP+TEdhgo&h&;5k8hQ-N>|}#VcczsFYEe@=43Eg;1AXkKfyt>hclkoJ?F#yTR~-I zRYg-F*%NycL@;(%%Aa8GjTZ=N41Ypv)?7trAO6Du?f{kpQo|=(Y0(Pyfq0lKZB^0S zw~Ap;jA~nzP_A@?1ANM!bQ4=`w$^kJ-9*CRZW8tg39@l=4TMoyC$GFuT}ckwz5@rz zVYtYX`cTn-ZF+F90>$&dw>a!ZgBKv0($Y0p?p=-L3=^*MIk<2lI~GcDvd6f+nSRn! zxu4O=ik~{Wa06TaqJK9l&omW(#i`Eih!~S;w@B0Y^+sqy7iAHAvN5~{G#tz`CtK#5 zqw$p6cjVjizT;emg5NQ2C~}jIKwyVd_nGSdwTD1}0zWhHgadK=7K2dQ3x#}E?MK&z zwFqEe=$(&Tcr0{qK|7oRIBKYKDWQ=7*r>>6e=-&cm8rN*{+wZc_Q{QFaoNAq__x{} z6z)p(go3y4zFe%_h{oQeCcBT*Nuz7M51wtUR}a?DaP8Xf6UHk^gr=$=9@o&(*toJK z2Fy5fplhg++c{b1Fu_^`b$B$tG!y*R_dwP zD)^l+7DEJkUsNGEg__Rh&E{VC z^PwZ`l59H^Rkg(dANMqQwwpaTZaZlE<7ro@BXT;`$COTDV`?ql%$~7kg^0YSh2jl$ z)p7M4FJC_aG_iHqfz(bqQ@Isa#@UjI`9td4~06Xsji=YrYuOM{=`+^+c zg@JVZIzaA&GQJefJ`2QcQ4>~t*}7yNFjA(R+VaWjd2thU#GPNq298q-m|A5P z>i6og0+nn;#{nvr?p8PTQQD%$5TOalby4`U-kCAh_h2Vh*!ZTGB~e!Bf)di!;J_gV zr=kiB#A%-P{W~+{AFnyoPw^W{RN-TRsAubuF~+8aB;p$_*(0roUT>;@&qkq~I;0Yx zQv}XReqHCxOZpu)(6zyi@AV~gDbR?z`H);sWS3zE+_?{GEcdC0E$_<-$jmBlhr~2= zu{dpN4KOU$UI$iuHN#(j_1W)J^Mj1o56Hwtj(eAd&SM}VNiAplT!Q>Hs7zaP*mQm6 zSRkH9KGPY2nad^WTrD|jRSQnZ*)}zh0XcEp!@N*hG4I%tS}=u_ zK4WuStLdEy0cG8}PN?2H7X!yLEY~N5f5q2)2F}EZMa{L=`l2h0m2E*!R*GW^lWUOH zQnB0^GoXhDu8{A`SQAwyf&G@eYgpFb*oAGwMit>YRo(@(O2}b%NV6>H?yk4N{GM&gYnX?=-I`u&hsvlKiEMVuQdioqJ}-d5%PM0 zyDGZsaW7KdIklx9`Nku_FQkGMp)WWz(hc| zcV+q8tJ~uS32 zC%NTe1+FI0nc}fN=ag~Y1Hp(gQe!&;NDx# znMG;?D&PX`IZ(FCkMdh*7gAU~xk8+z;9u%?FIe_qn#mU(;qF zV_?YgnK-`t$Jl@Hj{dp2*iToNf&_$7%n;kA)KGu=64xOiM=HH|u+`PxE}hGrX^>F#Q_2 zp?rH~&Q;n(D#%0aNGkGl_}vc+lKN02FQUhyMeRJmI2u2RFW5FEy3613-~2{G_L0k* z8u-7WO5*^LIJ$zb`HsUWutE7N`r0J0s{j3(J@7FjqO+;VgtAHcnmF>rJr37@MHxQ0 z97so{gE4cQb4uZ?<7rN|_6JjtHDx=xI5!yz%!gfIl*D#~BbYK@nlc?5y#_Dg=D7G8T|VfTY6sd`Ub!CS6kRvh zMw`lL@LL;{!ean9lO{_cQ9zBa zGX@T*U+)~qBm$-5Iv`{Rt=wtyD=INEFcR^ybW8^7X!@SL-yc!PDDIwyYpkJgdRP>7 z40;2Qa!*My^jWiiDZrtF@#R58KS7$pk_eYpKf2W72}cv${v;GxRogz%P^?s}^MjLm zUg{P97W=Az+xnxf5J6Sdp9u@ERnPlpFKkDwG}M#wV}%NhS5*%UzAn_hakq7#mW1UB zjwKaI@q34fP(pZo8tx)lV?LdoTC1tC-dL4dVg6oNDF^UuZ&!pDfFOmq{}0Uube;87euh3mYs`OzSg`hM{NcK5q%bpx4Iw2yIWV=IB2biwo7 zN>EwY3Vx$;UFfM&2SUR(u&lIClSIAtOnuZlScZoHJ~QfYhJ0LEtChl2L|P7KsyctW zwc8}uk&+hwNMtzP)Y;iJwtK8Hern=`BB0){5k#4_D6=B5 z(8gON)U7t_)pVHyXc(QL_#bCP4sNpGZ&BMR@_H6dIQIc2dvdn7y-t!nnKP|wMs+#}(AoLsZTtUg4ElX4R+8OWcs3CS?n>HAc8fJW zUKf`wu#{d~S}5E&>}yP+WfW-=rLRJOAZ3(`ENtyzhEm-JjK(k4s|i&zx)$9Xj~Bum zChV_>iamU(+eHibwk&*K9RD-L`+Z|t0)}{Sh9ZwD@~zwu?mm*UYerg$W}IRiaKJsJ z*#u0nUXfTgSs;zFMf@vdfN!)~e#d zjrXj|iW-gCM-IQV7~3|uuf3t$>!2RF{8B$d9QdWBX7uP^NG=q-go_=#((3w&@Y|QN zt=}&*a^owvPXNmI)IgjO!`~x(8#W-U;p`_YJWVkDZ85s9x{(kp+bfS@F&MScm_-Kva+!9B#7B?S2;K>F0Udtc_Q%%>Da#e?mBz{KkK0s`<}0$Ke1fB47iznV|*tS6_J zh!tXK5!!VD3bQ!=5yai)n{>z{A}O{Ike!<=Ogp~CT$W!w;;Yle2wa}+W%T-=?{olU zz*Jw;1&(Q^esMjb1K&UTAah;S~HiFy)+E zy?;o3t~X>i2+?Y4=-%^J-xLgFM@R;GZJZ9G73Y)}=G&#HiUfv(!*b{T^6EV2Ls7T! z>MDDbvHUyhxqg)crHhXP_upge{?4KJ$LqV_PRTN7o=$JQVeVR!I|+6_{L0-$uWo zuXr^?Yw{%9cAD-{%+m7u9yhseYvaVW(!h_tX*u2@YQQ|mWnRMJ&)fn52JUNpv5d zqx9@lEu*7ZV4j@P{_K1!CgfOOoimV~Tr>xR>$aU`H>n@epY_ zw{FI+p~^K#0R?-~eu!?IszBXM-je6gVSz9klr#NC^Fu$?m^%sV6Nh$XuDkKUfoLz` z1PqRe*gbWWdGIAm4ifv>C54nGG1C}MEzR4;3|u0|?-G`$mnfF}Y(;5E-jj@ta|j@< zx-67-7(jFQ(B{NM=^gdP*YukNh5jvG*OTpfFU#_r-G3oenhgwsV=mp50d2N~|9YyyfuZ5Zuuz@yDwgY1xP z(^ErwDgcP8jpH|N&_(OF{?~-&0Ql^V?=DWcuIn!p8lUtRU2>cpn|zsjid-c;F5tb9 z;aDUeI|m$A!wZXzhk;jFem6HHqTAvYS{4%*hX+FA=wNIxll1m>`U{02${;|2V`gNr zC@3v$d%XLLbD;@)Y<*lduI#mL&qdt9LeQ28x4EANO(<$r`F07hq zJ%TpM;g}-+aIXAEsquf9r5p~x5&Dp9#^Ly7k^4Yg7D+TAAv!k2cbjw}EjPP7D}BOi zenmXqFBOWOac38%&b{*BrkEB2$@-=FJauj0&rS*POD}}&pg*tZj_w31>Et3K z3R$<+VMErNX3xb5L@!oR0huK@TkN_ie*3H+eR${}9J^Ey-?r{G7bxeQ5Zl|~&Bv0-OnL*b*=&U>Y ziQIM)yU$b#>yXv9LU{b|QsEPQAtQie|2nTn;?G!gIpnUXZPAqWeadFxPIj*UJ(F%446=J3gAglj44gSWo`*{xtUgIxE`eTe-J|4{KHm(lW&8GFn52rYn z`pU(H#$ARD4G+aVEj-9|KQ2p2aQ!tr=@n4DAEJ6B433!CK1#8;ctiRs6ej8Z@~E-f zvhiiLdyh!6`J~I}(uG35rGBdo_bitWgTh4;$K>c;OM&61aw#FZojSOei$&|vsDZ^; z0Qi++18v#^VIbc)d}`c*V}Wl02{jcel3n~WjeRTN z*P0DjIH}QHF#mW9nkIS2id4tgt_+vEL4WE2eQL8z!FJY%Hgv=D1+d`qo1 z$-GirSm8t8d)%x9YjpPw-F<>4Vhd@7};v+-ziw@i>)uO2Hp zh#>fUsw3^V5ebpiz(<=EQerW0A+wZP=qx&VSA95|cQ~7O{rT^dbz*@pyNi$^;P<(1 zkNy(TOTcOXE_!UWCmYB0Mh`v{1JApf6w)58urIh+){X4YUi>B4WhsFFlDJnRP9y|2 z#J)kaCN!Y)4t`nkQ$5(Vq#HVvz{-kj=Krt<(!@%JMXx{a2i5&gXe8z%ge9qCz7fos z>7o4TRs=y7{5zJJBwtxzxpTG5_aND8RX3c(`J2?U-LxICa)KxC)qX9*yN@rK3 zuXig)E6O0A9||MNZ5CU`dmkMgYaAc1kh=PBbRWy`1NX9@5;82e73<#pbm~Qor^lh= zWklVw_!YDeb0E$bf84oHnpbr3^?pOjgxOG`5&cUY+tVz4ECk)~XBv9`hF~SFW?aFG z&ZsXN8L)eQa~AW1lTkuoPfF4*-@H}u_C&jlVgBDO`T&sRlo@^QHXm`5=$kM%xp@u9 zfiuHxeFj6r1Wz`84kyI0yS&cHetV*1L1R@2f>hZ~SQg_WWdJi1}-8FXN&p-=xrCPZl zbPs&MYd>-Okd#w%daF;1lSW#kq${XyWv`@_MUJ1J_-Ca<6K`2Oj~8 z5->E4f&R-5kbh%9LTT4UB_HY4G2S`z{sG{8I!^0xZNGk+@M`@W+|T>oqa#a3`hi@ZFbD@MBibpWhZRC$c>(+waoV6x}U!qpM*^ z7B)69n-n+YXOmO4Ggr1}8CkBpT$p_D5e!KU^Yrk1pl)2W^j>cImn7{yN{CP;qR1F9 zN9viX4>|)uv@V_Q*8NI!uka}tf4l?zF>yHTOCo&skOzD*3Z)nSYGi`R0D)Q*SiJRJ z23RxY?+&~%uYele^5bmKO;hCGqW6C+S`NR%(>5tZi;Y7=lY!4#%*9reqt(}0u5DuE zL$M8sBg4$XO0mLNhQ5iTG2?>`0r-X&dgy7-=Ei`bH1UTAdi873UZ}Q?PQ040duyYc zK(l|fSs1)yoH)-s4!c^E;?cUDzpqYhv>Vr(Fx;?=1iX70RdAr5M&If11zOrkfrkO= zvF+lr_O5I9tJxp^(qn6-glJa2U140fCVEBM^Q(z~aFr2Sp(iZPaqXk0>=*_|Suz6O z$!3i2S?Blt(d)fX)~)VP-Q6hg|IWQxAsBNNzfjCx(i_+h(>VNM3FG%2gdw>zH zn<}x%swUbDC$LX$MZTgp#*YHKc%d}h(DwAGpV8g(DTD(>`~{`fhzN);+f@DWiq5k| zIgsH`UgY75+_6RB5%JPOOe0C{!<2@2MAZ@om#vX~x-Al}@QL+$5Jha27kp)iILw9C zlh85Mtorpg3uTN38C6=*y<*T=4@xcXuK%aAE02eAjsB6cmKlX^mSNP*axiUu-vK|D>yBFyrfHXZscd{-eNh%$ty-9i+Is#f|x z7W1`fgwvT@c}u-hfw$9>8KzO=x|{t%+};uMwLh7mB!+p<~)NRY^;pMpV5T zZ?#=$GHy}gOm|h6=#yIj`<*bP8Q(M18k2RNc<()b%z?H10c}NV)-vOAFs`o)IT$54 zhCfk{|2tbp=45vTLt4J6R)O0P3o?}647@!LFB7@`Ib%2IQqh{J05{MWW5oTqPV9RF zp!u-j`ZwqAj30z<>YFU|N+vK_Iq18k@bqUi>Rm|7qA&1-{Sypw9y4xj_w}TNlo+wq z8-s0AFM~7jCAAf!rd!gsc9$;@;g4oH6;qa!cS+Kx?U|9arl$U-oP&r=|CZ>Mi8+hM z6mFGs5^+mOIDo~8r5aQEAT1hgpBueZyCZGP z^oi3bn%f-~6xt8o2Mi%=wG32T_{ZzRi!_jz2v)f4wW2==L6|e9TpBA{yFO$%6O}w< zoCLmZXH^8IWs#?aIh|1?Z4x)cU}K&}8^Qkciq%_a4~J?YJjHw*Et)qaM}sWHv#}1s zcCTHGKRB;P-zHFSyRd^~BRiA5_@J)PKBd-cgt@e{jM}-sZ}B{i++^% z)_I17xTHM+2J*+RZq>>&UjDN2&>>zWBg3o7cmZ|u!^#7^FNAepo9g@a#V>FD3{P-l zu@_nV%wR{X>?`PlY(n${^>3ZSXJ2~=7>)8prItA4NCM@-8nb!5uyJ9BoyA%d7qHMH zgBLbao)(jbQK8eEsy^G0{(7!-z+#!{&t7VYn_#A((r?(GIkzz9uektrfKP;=j9Ern zo($at=3A9g0<&*u`x?j(NZTl`jbFAN#{f{V zd*8+A0tI+)q8nA;$KqeVVJrYXi)p$P4}6BP#QECK?0z#y7UiDixM0Zy`?GDD7^)Q^ z4|ZT*pCpnR47jff?=ElQU`7YoB?_Y02Mev%o^xPvVUG7>QwQsP4z^i<q6QD^)IBXt+ z54e_IOKzk?7oJ3sCEkak z!a3+&8H}Bt#_Df$g@35$mp{1!i~l1yo{XEcQ6X%>$j!D|&K#>aoGn3L?uJGk;- z+`t6^>o0dCAfAL68FI)0*YN^S&K2C}sN-|E{dw?j+}%JlOI0EG#%PaDDv0zf%Bfy9 z<}ewxly(jc%TlZ5FX$%qEDbuCDE5^H7HfZaoRkVe3yw^UCU%cBzlw4$$7RlpR@Igs z`h3CF@fGW(wX-i2o~xnN);m-0P8?|16HcC3l|GK7KOrR-W5%Q=oa1Ap;L7inxl1+2 zGv%b4z~E9htoNi`>t*9W_NeFd1ZSz5Pr_Wk?(R{7?K!Yj4zkA@E8p~=tvOF|BQvYC z=Ogm!Ox3B>wyn!f$eYTn_nJ_5sGN1x*uH(cQ;XZhrNLE4{$Xq7+|Zr_D~ooSqqOR$ z?^fZUEY(nl59$6Llk}M8f?D$)xptn5v>Sp~;;2`|P5E1c!eAUM>zXLH#Q@2ur^7E1 z0~4!lB1@?iiNxaRM+Sb}qecz~0eN==^tRsEn79A?FXbNIlPv&nUc!LI{v<;+q>$qAiyOp_;-^Rz zT%Lo{vUHPg`-L!TsJf&}`{cb{E`ds?$G1x&KV}@{a2^6dvG~mc9@npsa}MXESf<`N zJ&7E)*em?UzWE%8hHlC&3H7?|M^6)*>&tBu z&`NZc1h3N3$IblSnfiC@W&ZYrIy{k5g)Fwx)5BO|{DUINJ31WJbpQEhC;iRDVw%@E zGs3*Cw2aL8l1ThQ%%}(9;qJOef55_-=0Rf_<&FL(-SezKvM@rAbKL&T;cnAa(7>|5 zhnC8)zTP!U){Giq5;Ez$XdvwAp?f_B75gAQ_`;%AJ^_f_p(M0#5X4$v*XFtM(lN@0 zz3KBQc1@=S4$b{dA4w>hHxV9&EjDTuZ13IC0yD3kS3~OTLOv9SIxpQ4hZyQQ2TV)w z@Nyr7j#%+|fb->z#1Hk{E;?XcvNRDz8?@fWfll;3lPt2Ul*sJC!asPNHp{SfrMC7y z>mQRMyULfUNFvTx2GOG9bdOQ2jP2qtFg#gPe#~PdQ&~Q`L$#|Kl0|F*R_@KRz z2X!)b`X9-hv5{L1ZA3-lorMtJ5YOep0`OL#JRHIut_MSbVP7y5*<0Op#H~;-p-yf< zk!^f!)8&B!ee%+vyKTM;_E;Q$TvI~J3_oi(R*R+Pq$}d6yNOdTt~W@%Ff(nyRJicE zr`R=f9GZt2ns3h?ETML9`BG*dh4;#VXLlXDtS}WS_X-MzvD>d77Xz|I?^+eh&Q!D( zk$269Q*t80(z346NS-n%iDdsK#=_+ORa0!WXYHsNQC97Ia)#s-E3sAdhzh&`bv#+h zuw~5gqa?m&wHgj)*rOl2P7duz1R8T+9T3XZ@0rdfW6Ior!y0OVQ#c3OYV`M z(vXH#H(LigQ(1NFqi9YCr|_JX?ApOQr`fUe*Msz{(wSv}i~J&y_~7#L&EKxg;-aQHV%2RcL+Sfx?*THn%E=eLDPQNYW2hbBzv?rhz8JItO50@T z04w)>0R1uXp>o{+&g}aqXiuHFZKvWz6@@O}SE83!PDcb+>pIFFb&|#BE5#My-`y~~ zaPJcUz5wj2n*b~Ei~E6Se@+^xqGy;A;CTU-t9;AK_tR%%0Y@zNBRhaiIfJ2$yIMW4DYP^1{;;VNt!*;Y8`$-tkilB8>|Tz8 z^Gu&zaLSlAFegkSzkfK64NPXP*t8wL0v8L00R_+3C;ji5{QV7|{^EE5i04x6TJY;m z1kh}-pV_Yn_>LJsJkmH24(L?^LOblC3Wh(O@5Dub7~_rl>H?4rMGb}lj#2q@lLRcia>XS?;!hXhA3}i_LjVpiC-=|(!mT0(K>r&+>3TkaGy8y$ z3}SBG`4z&QoFMda1XhF`Hff{9q&nDgZ@TY_6OMBbBvr7dQnLo;e&soQqw9`tFf@Vr zdoP7Q=1|8MGe-|{@AJN0J;5H*ATxgdxuSuh%7p$9H}r7tPl*0020shsgcab|25a^U nzWx(t{tw{$gu?&N9P_Z5XnhOBDQATh;Aj7ygN8-=j*DIojo3@3RbUV&!4THc;Hzy2(;coc!K@$u2yT8_X!7r6Q0yFIt0(`el-cI z&{uoJpY79M3w&0hv5tDUD?NXof{#Z#{BfV}2|9wQJiSsJ3c@AdFf)d;N~qr~YDrlP z!^1l%j7Yx`l0v)ulpmabuB&rJq4yidpA;wa{e!R@xXV#?)tb!st8dv5Q^!E_PAdMh zFfp6zp{u^Z3#*Q-yrG;;a@cSeynwg;iNfD@}FyeJz*^IR~5XF`K2`a6d~eS zgvbl3ZJFmuftZ`9hhz>bwt~;d*}OO;5{MpU3>xYC|Kz}F9`X8shiCneeALhBs{npn zO6kOlo)D$ex0}ov=27oGf@z-aF?p1zPsM*^$9{DLfG*W`C*1*Uih7ThdHK=-Q_3qTbc#Ng}dK^tc}8q zlMek}rM)3q!j>C(ws@{OKN>o82fOu>oLj#YW-U1XxmJ*$mInuN!*6{} z=e?Wc+ZntiA>7g2{ziX_)M#*fokWn!p zTd(fR;)8LWe8kr!ddklo?s2C!3Fd9CH*F9e;`8H3VJ$tlY$jD&mXi^SwJosCvQ4(h z7kFJEQ1WN|G|WU+CaT!8!%GVlG5q-oXOMOcS8WYK%E{;}Ev=bo)CTg;&{6i8X;%bc z1=#U1=~-rgJ3u5%HVwR~FHE zprmL>-?UjiqkJL5llY{7|I?lxMXu@oYXV$v*<#{+L_F>1<46vkGuoua2!22AS>IJh zR%*hu#x6{9*n4Q_eVjt@2It{h!N;Ewb^Vpeu|CuM4nijTCW`woB>maCDE>3yrS|*! ze6s%e@2Eq2L^1biY|uEN2_mXrx!a$oKXMLm6=sa^-$-fAX61Sa6IL;P5%>9nKv{o> z@q-!!4gaXLku~~q5?-k0JF66nHSu!PJ270Dez7{iG3>3ax+N^MkHU4!iXZ0R&uiT% zraVa~(y>q`IJ)=p_x*$~+&}S%p)VJBT$%VVw7it&x%VI!xbB!9pBxvp?y-11_yL_4 zpT}u^Sc)cx*7AwzC)>|AKTWf~u2$QX`@A(56Vx#4w`ert)Mkb1; zB!hJs6KO5|HUs`6okOar6>^((ihgWjQ>xtZ@kR9lE&)ek+b{12>ip1Vb_zTVEC?J5 zbaB}GtX-*ATdc5Bq9t-R*q#qd@QN(ki}$_dIm$>2D< zI3|fZ5<;1Ik|D7T+8azN-Rg#%2e@TZ-R-nLB`hl^@>}x9rdxSBT$4^o4p8xNF{}b4 z0!HwA@MoejqXwdwY0c<%RlgS06?{@OE9g~qE3Pd0P@-u(qdlXOX4PCmRhm;#Z6#iY zS)ORFX5lguY`kg_tyeX>qfLnRt}DJeVwS?+H1H4Dau#=ObU2 zmDiLvPuj)$s5(uO1 zSHB}{i(wnTJ-T+(I`czgzTwAZC2I5YNxg3(Wo-mf3{p8piAGt`8A>%ukz;gLPj@+X zDP~t)(fEGwmAPS^yY9md>Goz;&$E)#w1Ve8GzQDCr6H$P219;Z?exLsw&w7g?u%lN zA60(WznFUAENfdTynnJ3w}eeaPDLX(*&0mDZM)SX-z+ao~sq)_;VlKYv9-jQFz zc)~X4Wi?f`%l`2G-~rZ`iWbh_a;KE1atCS@SYf(iQ~GlqW|WTN&oidb$W#HQ{R7;PkX^Z;oNzZqbk=T#~@jjTTYv(XVqVV>D^$ z8Y=p&7FvZ)o(-f%wO9s>(Os$jWT=+jP*+u)tol^7^s|qiKl*yQk;T@ei>c3mDC$vk zK=hpYgYUX+?+YS@Xp-~&H5D4;s%!LNT4t05oLVcIl^^L03S2GJoaLPtxw`DT#$XC) zG?wyrm3DHNa#ht~I=mj2BbaZ9HrNQQXlG8G!;Ocwdluq1^9u61DsifCEuAY_oi``6 z>O68L48IrHO+|12-nQWW?U<1JCpTxpq}*18PL-x;dz#gKEOLgtmS~fBlWlikO0I@r zD{|NEkJ(uf`?1iD+3aq8}ou;dsaaIRSNG2jWRJ&pj@Sp;FRh~Mn4PLF8IBv{Tz*4H8v0!3&c7|C{b+ldT*sey znXrHAyc@wMt943H%OI)sym5B7Z*w==I%68Dg?OKhh53WV?(_4p_~(d|a;_)laaR)=Ni$-V#|B958JE88hcN^t2}Sj26j4RP8Q~twp>oU&;B}t3mn5gW_U*S*C}>pyw8*+rO5=X zYz)ZQ=wH*ne#VDRMn=YCqi@J1`%d_umxI4}pBdZPS#vQkI66AgJ2KN-*%&b}a&mGq zyk=rxVxj|Q(Ahd$+UYpaS=v7T#~}YR&N~BJJsT5iI}ojfyw9G&Z}hL9 zf6UXs$>iU6vb6o@vcLiv;NLJX(!XZ-*Vy1v9{8tR(k4y@<|^+@EP%|wJ@`1-IC%a# z|3AL@_Z@Fssr>IN895kPZeDuhtG{2WU~6C_Xk`KJX~*~P{rcy{H^2Png**)KrQZ<6 zKji$^r$Eqr=sXPn+B80Nh~6tUu#dzh@1*3xJJ2%t4`M0!_39t*;22ToAw@tfBLac| zg4nw^@=l0bGxsi@4qaaVc@&n-`r`A8x9JYLb1ka6;^rkIvhv$gA(LlSxCN$pxu#YX z^5S$AD@5{i57K2HazA*3e2@Gq*|VtL>pGFuIBwgO+3MroddHp1hTg?LZqu>V%tp?= z(=Oqs-JDqgM&ab-o(M>2WdGYQ3gj9bW!i7eYXAFbXk_ykILb?J1rQLCasHQI&{8C1 z{H}*3cm98Fiv|WG{9nfT>n{Iqu-iNLe=oC8Y$!v+$+1+=L|TF;nY!tmSP#R@p+JHBDQ$_Y-faw{Qd$_Hc=47JK)KLe zuTEE21c~0=zcTEO(J|~x(rA0wL3m49j5VOUuB+UcN=2GiT75~pJrBicb#Se>Mhnz7 z*{J_?eIN^Ycp#1;7=>6^;h9^RC|Y1jqnucC&tTyu8}y7EvJn+}SeL2(9+lOf{gFK*6EAf*ib(`x+nR#~6F zut$zQ`WGPrjdC3At+_@IqvibU8DxB#-?~P(FJXcfHD1-cipkD04-Z4~jeGPC`mUZj zCY$q(=-+I49r;jTRYH=L(Ka*ow2@zu``dMVz z6bCA%IT;sZi>7%6hAjT~*VVvX=2VM6-Ev67v?-jAUP{EwqvC}#4Gk5cQ_|ONmAb!X zG~us&errBx`V4!sF1x4CyI!?t&jyN#y%9noxcAzf=AHH*HH@g6N`nB+Md?Jr9eK`( zMtQU~QMO^jfpbd{1<1IeIdWNNZ(6D3GGkQA3^|!8n@;F3pew7;7L9*+tUnJ8Z~xb>gR%q&}b_cRWc-RUJ+W|M4;uW(;7r% zr0KNaqdZHvA3@^Jh|hN06@Dik6Ri= z9NoM^UNH*f1HpoZak2^;f4&^x^LGd-!y&19u3XsI=acdoSEl0IH+?b8~Q&a=yoo=r{&{n4bmXNg*sWy!Is^Bri5BA?6f)0<;55u`-xuB7XKzBN#Jg(p(L zVrx(UOj8|h3-iDCyBOx!wqF*Hvt@<(-jZ7Z(xGg=Lg?ayn=ujEOC_CqoSHc>?z9-x z<9g6Dw$Fi#uXcqyh;UN|?QgYze!6FPxHcf9!WjJ@B1b?nKn{a2zzxf=i>`_#fVNaL zmU#b?4cg+GOlsifNauv6S9gNaCBO1D>IQ0i8_Ac*WsqazWq6n9%pPO&>>qj0UJ{aOOS|aJ9DtWgwQyD^ zFOF#|dz*OE<&w!krT8z0r|pI$g`9!U9m1x@x6uEOIo?I%0am)ge|;(?BW6}uojGA% z*5hBfY~jg36!hzx$C&iqQ+TQ*ocyk>-pijo-ZbmqO#E~#egqf7(*h7 zXM$(1>5fnf775RIe#6;@(A0kUTWv-uBk!79+qHreCnsY|b63okY6mFG59*>@9tuc` zMY0J`eOkxwX*b+p%ef+{W14iSp3dIdn<)IXwZ}S{9}*Rfo4qjTjj_1aQhst_{Kr^o z^T30$h45cF$*{6?1`AJhs68u=sbYTXc1wVdpnjz8;}UT!IiX5bmNVUmsKnYej=P%U ztw?|07-t-(v-PZ=lRt|AxlFDn){&+ZQ}x_ID`>M!aFGu^ZMI-f0_t(Sr>h63{a2s;(`ebWc+xyyRS%Z zsvHxZ)1MZCo)|_w|9xU6{m)NcYYPmpDc)QYT$9`5n=X&%tTH8HWwRCM8r*`eq~fCo z9MNd;KVaUHRsRN3q1Y~fRN`eDxp|e4p=|!!>lS#4b8>Oe*HeEyYsPo6*ZkHDmXhxk z;4F%ae_)NDeJkM@rhRQS`U|fJI=}2TrT_*xi*k59dA?KI=Xx|@ z}Jp=-LfSrg1@c$_m`L5;tcij?Y7OB>@1sr!q!Pdx#^DK9$ZyO zp!u7r@-SHs`<$!cc(G2VN_%*?un%I4_YJ~yG3KopWy-5+0PfSE8<3hVYe4L1v_ z=>N-1^X>%_vG=Yqh+du_vvY5k%N0*XC?N$%>SWzUx$m`s7c%_y`GMhRzKVNP0DVpN z6E3^#0h)U^R|2;h#eb{tPs(_6-BDGuf}fBy0yJ7S)r9?bBiHjeIQT6yP~08tFOxtoe^v_9<0zY_vl-lq>zPR5l#)!?{1?{b zQoaIo(#3z6j)w=$|pSFoDu{>=?B#RD*A+Icl`r5T(5uuKBj}|>GVrEDXCrow^JZ6GFL#;;rdXIhU1Ka@lF%;PLIQs zg_3g7DiX8>vJu99(;w(l!hz{oJ)Tid-14fJRZF*V2io6+)q*S>L}&m&oN_j-|C^vB za6cHEFjcZ*Wd@b~n`tBnHAyZzoXiee<8PA=6>0xd9XpUg?!gP(=rOKp{cnGoI1u71 zi#3Z-kOE*~6k0dUzgVDqP5?R&@L-06>0&;Uy8r_l z#u`LX<&WtBkHah9@#y9{fS+B7HR_g=;#Ft|gZ-y!apg1(=g3Klh=_cH!LyVJfJ4AC zmBBKx74~oDq|k2UmEJ@%pgvg4M!m%RQV<$LVR7M`Mg`1}j(%_68#|7D)+w#uf5vfc zhPM82&e-20lHG zu4=lpI0*)JGhe6!`}QKgAM|qfppv@I!}n6cq~n%p06oJ*CUKfaWOm+PdMkc39o{ammnx=V1UI)zpL%atPAb+Y$+@*pPD*VfNBlof5*jYXv`rVCApkvBtqpGH zao+^}@OMK82%#;3WF2>JuCkQiJ8)(_flrt(0>{*CCCh*-FwjEdycrGPIIaIT()HDG zV-E&HowRYw-a@lvk!C&mrV1U7a`*c%*IU*u2Q?YWjD7H27Y0l8AiuC&VKL?js~sA?q1E0DmSq9{@G^@ZR^9n)oyUb1~#QTYJu*H=?99bDxBV zBfz}lZy7KGv=SGMKPB((PUVy(JYL3DMmFhYCg|S=uW0Z5taTm_)5`z-`jkYJR5}0r zvR*F3v>>+5@(yHt=H(a6w{+tz8PB>a9A16%BZ(8LT!Q{3oqzuV>FM;xwx@r&;HmL4 zBdM}6O&5bnO{9QWo3JOh#6)fllBt-9GNW^qg7U%lhVd4a9naME|E_E)1t>|74XT4~ z=04oX?{HA%etCvBYNi3~Gmot8Cdna?iL_gH5L|sc>w@8o^@Qoy{l~9UihKVSkicZW zqs)3-yV)MkI1bfSW5S7uf^%QQ$=!^6%v!AjP8dTwfflbi7QaU2^{QKm<1m zG|}8tN}x^9K_tf+`{-Ccx0*(|(dKUtm1!;dYH#jfzcv(}1bun*@+{+MW2EaoDPP?W z))bDLvi%G8D9!(ABV}i3ys-I(h)Uv=q5|S3`ep?!;0%RA^lpU%Tj-Qk<4ku9V{@Nl zq=F)&nni`eia{v(O(Dbec$@mQ&QD}!$l==C1BdC4B4xnbfc#W|`~pbYZrTozA0QH{ zs~3EU(49?5(0Wkc{5_*XfZ8FEy6!=%fxE%rw)(!{YdB*+MwdTUDxh70&l~)=r~uHt zDHj(Py7SJzob-PnNCc#1l*9f)l34{+9Ur{zqbVz`s}x)p9AydUFZx?Td<+GY%K$L{ zOzCl;f2K9|oBt)3K}7Dv2qfkltOKC*DmZ|yGG@Z6@v@UNLKhn;fGTwSR#HXy{I%)u zE9Q^C-!&GK@IqqTj%TVEQRLA7E<}qL|6|m6s&?~G)Hok})$C%;ZQzSdmpQO*Dg+b3 z_wCB5Zh(rGR=KU`1W}c2UUMU!_U~zA0M}gG#C!_-S zOgI}f`fk!2gtb@m&PlPHfC1%osa&5Z%TjUu z)1p?4C;tS#cmX8#8N2Y)!yzfR`!GfMq88Oh&=ysc+xUy=!N16ncAC6XQuHUKo&tD( zq4#RwU2{41rvz&pwoUPswGsJ^4-M6EmR}+~z&Da# z2jbQW>Gg@oR=*$?=Px)fU$X5yWepH+Je&~DRt0uduj0HM8Bl3ZvfsvkHIR+!#<6be zGOpw2!>&-DUmcv}erB@zXX8zA2W33RT%LZK*IOkG`^W(^4%qh87~o?_79?ZotIq7B zsDv>L*g!6o##sovJUm$pi3i@P&lMyqD|tl?)a9vw6}xgA0<8A-vE6`UIG;D1UeZZc z)a*YFXfR3@+a?Byp`Go@CP;+JVj1;>S56h<7E<1LcSosNs;ui+lz2S%Fq%XP6fTcw z^2X*fKHQb2=qbv+HX4fqAdnTwv|VoID#86(mijQh0<~!`wjV_EUDzR75}jjBCt`rjwD&mKFtSYoR05HWEwL7 znc`sH_ehFo;9|$IF~FkF)Z4wStfu#H!WpRi=0l#|-1>r~zF%J+^{5yn8tl{`Yo>kM zr>LVUU4M7V%>T-U+}%}p#bQ;K+p~wz zhnb5X3n^&-0W?Q;b`=7&C0loU6TLFgZcuCmAVgNZ*1#ODW;lQGD)iuY0E1$p$cX?9 zw`T7rM$cc4&Tjpod)}5!6k>9z&xnqn@a!HhkY;!FY7&uI3mh990o5Nrdd~(g_8fu$ zvo;brT&7YVWx6c{0p=6?=mQ72;wMP@cHKd5$+n6``v#XB!v5$>el<*kC+dDk!idc*Uz&6YGwjaS?w(UbtLygiU%NIQ2SYR!h?M0S$WXp)e#P}v#7+Z69Txi| z6%ZQyv$eoSK(dog!*92sByi_+EXQb%|YWO)P8KYs|$K^4{?sg?n?GfK3dIe1=t zg2@OKLJF|IE|w9HLuNs062#_}f2DT)q5rzdObi*{Y+zCOW(aSWNJ&d;CN=%=gubks z$#6`=v4_@*>1~QC)DRh8K?zePy?VA2Ch@#jc{`Sl??bcTQ{V`Mnj;m7l!v9mH~z%1 zP7=2yTUNGrphS0~!eP za_9Bs+;uDxr{#2Rg8KbP(cOHGtTA)9&ZnS2)(4(zgEF#m!i;%aX!q@-(5i z%IbPStg_grE6z!$K&`q#>0;;Fq_7eMrBhy;_amFRd4)_OPwnM%G%k{Yx=WP9UX8@Z zHA*)J`pU6Fg)pOYPI2seHN*~{khpA>7#38FG677O&)@jXOBtPvCWXnM6Zc?L%_{cZ zBg&H>6a_Cc+>R#NE-rUH*x{AQL3*F<3S?LGSeB39zCwjqZ2`3;j9vRlZmdwFCtsz^ zbmLDINVV{NFBE6Fb_dK102O>S?X^%!qcU2t7)KgoLP&SJMaO;v9KRES|G<@f{ro*PADnx2$8MWH7h}(|0?gPi?KDVBu2DA# z)LXPPAw=;{m4|n|8qR4X&+77C%CfAhD-98uz#|qqy+iBp+FWd}u|$0QOAsI*HKE*Xg5~Q{b>r<>NxYaK@OM6|4w^S z(-&RXOjHzwC)|=_4wO}Le4h^Ytb&3E=iPk(Wkz-r30$5V5fQpIE7hZqxX{Pp(#lv3 zn1=bLr~`QCk2O-94d-(6t2Tg>4su7$69cJS&;b+~C1Mm17pA2WZW453PoOQs-3r^; zBzq#nAKz^$qu_d+G?T`GEPf=vw3E8zIGhd6rCsp^fe;_S+>}jr2Gl=dXS4oE(d?mLLLYN(cOxJuU8Ze;e7K%>3; zuNNE|eQEnMz#%gVBQ~jqK?+Tk-;D7DJ{JyrE$D|;Kyf}zIu%IN0Lr3&_+4~}F`qhI zUmp9hXK0ST4u=yE16Lag=?j8swl+KELmz}}$?=}b8wxcmHvnuezOFwpxc4fburChe>B+%-}&KPZ^N)3M00}Qveew2#q!u5@##Z{9>aF zM>3Jz8rQ-$ZOjalwiiU*WETc5tyygMJ@3@rO4Z2>geDNV3A^6%dJQvYC(1Efo&W*{ zTB1Fz8$h?x=M!tF*|m3i0_r1|?ztXuV``F zl6SNx`Q%s{?1yEqm>HZ|7JaQ`$>8pWbt6$#jF8n--h6gmr`b2wWv9BWuXUtLn)msL zU};#orrriGzNB2%2i6T}>?)X+yO9y77c+svl8X%Dwz-+ZW4TbB%iWkwsPPI)MF~uQe=}LbuO6y!iy{A+UWH52Jx%!;4;h8LNE( zvI5U5skAv8ic9|5LBHtKWgO)||Ki}AUI&vP#?VDBaBb~X1b25Y!DyD%9K1frBA)UU z+H!rd%74WJ8K|_Z7*Q1FO;Q(YTl2N#pxVn65X!Lwl{JF;g7{RzGsXnNo;U-`+HE#l zP{euyYsat@HYS}x+co1)tD!0Zuz+CxXiU!0_8R#n-@c0m>TpsMk*2VR13mzF`yCd-M=K*9nA>G7|BHnp#5POm|`6yAu7GN?4WT!(FnIhR(Q#5r7jVZ3NGm;S_vkr=A ziSEWpcJ((?SC{AUpwCD?-e8NPLNa-~NwE=Cb!>GxOq~A9HtDSOcEuQ>+p1r91GC+S zTrr5WWOJSG&_-U-&h>{+RX))}ckyAxpBuzSrj+NXe(Z+1hY!;@0bw$4kkrdD-kzA2&PN`7CHyXrcn&ru5QOuIBkKo*i72A%flqG@ zF8SgA>(ZM&)nZcH8`G<*4)n%#0!VnkGsVmp<#_mAdj>0KX+qP*hbLE~n0KZQj+e?1 z%Z3f|u#S8)ja;5W*dE8_8c#jSPrdMV*ch1|4~+a~vEb%qK;-mgTdnKvRC7n!8_mAE z);!!&A2#yMckzraw}SqJIgiITmn@pkr&tH_kyaj?nkT1R4k!_Y!puXv%Gm#4IH#*v zYXq=`#;!;2dFF1Y!UyLQeeNr}xqW$h3xeLcoMX1 zv|5X~`%tZtcTcx%ZF-zD_|k3L`e%|G$Ioi16XLsFY#h52GHANX`t@%bUEahlD7oRA zNy6zTBX7Iji)I4~x+#QuM_`sNY-{QY3Q3?Xt?%v%4gK6*XI}2Cd=ENBS?e-)BkG*z z=g*`Anm#``#=R;kSXK>B$|t*79z{Ts!~@-3^6uyi4`qvj2W57A>{0whQ;KjoA!w zf}*svG;_C&dO-@rWNVEuqyUE#?mn#ACuUaGPa|9^c(Zw?7GJ2#cPy(~o=}q&1F+4# zY3lOM8GV$JV(Os7IbnM}(oWR;JQ;XmWfH)4+L`n73gFn+f;08bDJ-N@SRr+VVu^X_ z5xB|`Ur6vjudJ$8HRt1(7At-TS7M!nAd2)9}i zs1P@*+60-?_n_q{i@p~BhTRe4DZzoEU8Cwc@p>y4CWynZXQ}TGte&&Pd!4IzPzY$r z&aZlXdaH=gi~`E{pvoX`?VSL0%5m~j)0(eY@vf^0wjXre!e>v` zr94z^tAAh$!xXvPbv8a3sc)9wT zCtwxBt0K=Ir(38^ULHD2#e*`n#=z%$85=3{OA1OFk-2NY-KFOebCZWc1xBGlE6U|_ ze77pn#)SyBv}|L^gW)KIoNcKqDMNLV1Nts>Po|owY9_QodaJX7;;JMMsv_SX8LACT znH5w-@hc^-$e_RM(a?8siG`g7fKILkudbUaO z<-qMk+^$)yW91krhNH}x6i!;!HUo|c?TD`y9P1E5?bVE-X&0vcnw74yW_?1JG_E?c zfqj4g&EXvjsTPHirYjAy)tzdq6mjO#Lu-VuSR3`N1Gj&ewFW;W%0DZr(PVw8U7cP{ zX~CebcPGL}0~R|t?Gs1yI)Q~hV+9T4<%raIW^7&$D{1S2meIzb5HLQ zY}6N;Z3}g&$tCX#na`y^4P3n$R4>taX;D>A8MM-P|7Fjc>}qkqnw*mNNN>67~ZJWpZ@-_^=^-b-;>w``%#tNeotxM=m^JqH)~JCc!d2@g^>0@nDHOD;gm+?+)yGFirj5vN|Yltrg~svPxxwtM$PL z9;aPeKjZzQ!dH}OC)Yh_;`*fKR?q4-&dTc@E>8_$Pq0>=1(9U!KHx6TEeTq$^E-Hc0^7Ty?P#iC$b(zWth!522lm#}`^QOJan1ma>aI zr*JRJ-G;Nu1B?qigIKFZR=>DcqvSLR@nKL@66+{Z^WoyGqm|4ul8s9}tI)f(QLdLA zQtl7C1e=i+=BAedh&s4PGx62E8~I zka#OU4TG4Pk}xUtW&4ppvdv{SVxRCbsA6NwZFK$`(PQLFw@T)7s{{Pa#CbF6)qLM| z)@$3F%W``Zy!E&^qcD2{!F_?cb*>V0lKnd3UyhJ_c#J*UFL_LM zJ+FZH!=?jI`F`+N-gfNz#l@@Fn{i#`lbaH-eAeOBphUutXWG9NWkuXC2#~vwpks|V0;%U(DJJEPBCMcwJLb(=l)fmoSDhOCMycDgNvGd zIkMMX5enKet3wKNBhQD`cvMNEDjzPzTYUx7BIDo8IrZY zoXtDp8v0X#jL-VXSR2n>;XGHeP_uudN5hn(RTkRv9-^V3wp!k{4w(dPEs2IYjW%6p z?q#;@5W>NlX-D&%nq>e-lQJVIHC`5)i8lKiPB63PHT?!%9Hg}*TRg?RBwZ%7a9&s0 zy(Xy;G>!jiW|cQTjLU1)#HRG|6CJb9Mg5NZoi;M=l3OmmW_f6jU|{YcaEJCk}wbo z$hgQ?r5}##g|t|bZ9GO%GZSm}#RTY~^Fng=)WG8eF;Snj?U#NHY#qtTfGfEK62G$^#U{c0W-m*YM72^)gskfVBcgJc7As#zC{ z3l)uM8d0Z50hcnfaAc+;7(L1)k^*5DIv9rJXiAAe;g6ZfFO6p>vRWl=GOwFDR$CC- zC}MZZhwqap=Vy-P5wraE?2?3z=gTSr-hCiH>4>nE=P~{9UNd%4P25KHu4%0>wdnhe z2gJO>jV?o(lHy!1KAwZR^rc_Kp{K~?@YY;xIZU4)ITs{Y$_2!2H`6WwOOOhz91Yu@ z`BR|ar?EGds*g?EFC*#v_|m~t{2f%vktY-{HsYR_%7+w!hL}F*J2HaJqhXq^e}s%e z4*+9L6Keq~25-J?6l9U;qbH0K>dWs2%v~K-;lXfn+2$9~K}M#q9U2UJ8jkf=H4aRo z#4|v;Rb2n4m8flh#AqMTel!!!`Rdg_&KKK4LY9mCkl(`wL8rX8ZksVllTF8nt?|j; z!WX}4FO`kNa7r*waTF%;WwmYlOrAVUT2m*r9uI;`tPhWV(T(84=pA7kqmD|lHi;M` z;@HSboF-VqLsI-Rpm%6QH&|xOW^>B2wt!#BoMOAjR=Xr_cH+$xYuyHTT;&1tC1_S7 z-jY2qQ-roy3k^@n@-DGzSJ61)0 z{G>`c$U9i0s*t2IWI_31q13)i1IY#kpl%&D6ZRDO|frc?#D5bhe;4KPAGK=YDcqc|96y zwAu*=U7q>!`ums3t~TZ#C|?HC(}9v6j-;CwF-I$Xa}Ou6t1 zPf3PcS2O6!NuVv|2GLVeg8U*yL0Xp^N?-hX5;%v3ycGuY%4b0%-7a`?M`Gj=cZnyH z70bC}rUcB12pR{3I4U7}_ur)&@PrXNcu>%p`}y$8zi9mCb`9$vC`l_{@?HSp4qJze zP{JGnXYb25J*TSn{WJr5fWjP#jBEG5j0&;Wgd;I`)&Y9eSZ{dI5;WEL2&9=QPZ+y+ z*<)%%8#WZ0-r*~3O6|;NOSI{vDzQy-e$YiRhB*gXStf~ioUCd2CyZY;+8jKLqG3L3 zK%#@X?13s(w6NpBSX&Sg!4$hBw|z0B>aO5}yUS~=>(D!6(oxvFhoNPf13~a{>0v5g zRvu^;9dXH#jNHZd0QBUq{N)VI%nHhAsSpz0hh>uJw5C^MiV`(gnjm8Er^uP4$Rtmc zkLe_V1pROJ9L!>h9AKCSRn7R1!D5VRExPl(e`Q_^L{Vi!$%hR_s$pL0JK%1eOE*gxXwDNZaT zYADp9Zf2`M7SO<~S&1{#9UB21)!UOJ9-?IG1aw*JZUp+3pw9oRvt}tw+^Y(s2>rc? zOI>bT2%nxw7>#viQq)K$TzQo&vn9r-%DAi~q7i?zDEdp|)!9Jq@QHIrbz`T@(hbY5Ndp^5yFB;`v#G zez*&I43Zj>&D>&MU^g$Q*b~WXs;p&IxoiAF)J_fVPf2tQW%2vir2Tyd##t7xJPIpwOBN|5<8W&&?e9!S;tuU-HWNC6f1 z?cq+?7qqYptNY7%LyDvlxmX?O%gL#VPr%l39$UeJz(Ws*0Mp|{N3DB}HcYh!D{Vk4 zgF0^m0bt>?{ttU^8CTWT#f>VW2nZ+&C~eRs-6bN@-KB(d!$#PIh=_oIBHhxt=`JNi zLAv|Uus0>J>3HWl=RD_t&wKBu`{n+A&o^CrGS`@MjycAE2WW?8=6ZNTc1UQ%@qRk901&R9&q@qFes0u_`goyMQA^>^= zy9i7b>+BSn`o(JOMD=Gq18|u-ZOhdbLfa2a7dAl##2eRNAU;EcTe?K+7JJvq_kBMT zv-^EGh~MmT4H@%UQ1&f(H5^W?H`avf!7dY9#hm5?qb4Gcrw|~x{^Us%DP?~Iz!Mhm z57nF2el|E^RSV?mf?*EOk$`Ad9kBS#`XenltFC;R?dkZH*E{V`MaeW<;C*bR3}6k9 zdkC=cD^fh$^35y7@uW#R7MC-VSOxmgRdA516|}=pHUscPmnCGhx8?E_0?1A3((Zl? z;CSdKdWJ$RX{C`E2VCoT5NECcQNLz{_6SOVDPogUYXzj`Wm{&tE8ZunL^Ld!@vFZG zNr3#tSM&UZ^;{H6Bq@$C zeXib}VBlRvHZfsn8QYbA+jIk3 z;yoTtOSzRV-U4YET`W|+Zfut*#Y-}{dgT0Vr;;Jh>g$dc7pUlW>nSf~f^ui}f%UFj zT2mKb^DUIhu#}~G6UuJ|g|BSD7pm!XaBcXHU~P~e^nO*Ic&ux3jt5i>X<5V5$!zP; z8YLh-mYCwj;xB3bJJu3CE*UO1&;^>t2_Ka%i^%nTG8^0@VVL&et@KHOGv5;@y$il4HGyN(46e zAEb(3l*7s=FaeOyE?S;m4&GeMeoN zDG4f3(SxM@=+{ogw;H7ztxVBhbZ5@^q`^i9cTHL9y228VHFiO-$Lp&D((R*UF>k8zzSSNF z9$jp>^@aPe_wGvR(xfRI<`Nqa?8x9R&GEFpPNR|7R>s?Lb1S!;PCF6KjEsNcf2=0R zy=`Pq`Ff~@Sodei#fryCrFKiB!T5d^PzDi&?InexP1%lhZD)IPN#VP`2UrE9Gwe^H zO2Cz4s*?@>uVdK;W1m}RK>h1=r>C)p97HmBg7f*UGa#{&u%xHV!>lu2+h!{;q&){b;e&(h+wu}b zP66IMcV8@m9R;CZj}?U%xl1eYbVvqhs#DhWDacN8OIwY&6!TFP9xKB|lfZ%ghYg6f z8dN2|72lutiqhZbnsxm)@S^bvz4wleZ3xD+u5$`;rFOb@ zm;ZqV;slYE2fNBl3F73KO+?xzMNX3#K@=1e3S+3USt>G~9pj>IK?r%zn0l-;>e-?u#uVR9p z6cY?KaaV3=$+egsd8g zw(*g%yCoEtf?(0z^uY}$K!vzpx^m5)bYvQq(5hD4GjLy{&NaN>imNh!qIUG|C#E0u z=2Pa^Ro$*QS=d%gz$8txGhb{NDAUqtjNUCY_t#~0ygE)~ZtmQs`%GEb^it>s3S_71 z@siF{u*$a2Q?4j&2)nZ@>)awyz!Ia5wkm^z-@is%acEw8@lh_a> z+-p&&FG3adB~6Es`My^EX&=0|I|1b;l znRKUu+bjwL8=b{R za)RDv;)@zvsQZJGHe|(|@@s=uAVHHs_&<<89~Ku;UrpgTa(=8B#-&6Yj-VII$3HkN>1(_Kw6t=herAlbpX z{y5l)eit`avHQr22#}YV2qH8^P8nTFEaGB#>c|7V(S2!EZv9i@Xf6wt98Z103Hv6z z&P`e2C1vSn%BF)`;KtFG-&M*5ZlvFBVF{ifyGpy%g`VZPxzo=ANTcFMvlQtezluORh45YJka3cO*PxG<;hy zV*5+5`w&tbM;J5h#uZRXv@o=s;5wO|gM*V(L&-tPW36T61sEqnxz5Xskm(0Vj{}xT z(G?;5b2UyL05<GpveX0@fQ=)N( zA;Cq^qbP(zQ<|VKzhY7VR%=U*%J6!4YM^YItEX&7j^+uR%rh{JP?x!)EMWXh+37WU zZ9?z`O$}6{(8jsZ@w{hC*u0zCdkwMV2IU}If{U-fWex*rI(rhwlI`x>JyJ}pOu)2( zBAhS;*NSy)De&sFghVK^5HKF03{M>wvc(>$$lzG2_oCPt)-5YSNx9ob@GmR(e`EfH zf~ze~Lq$Z|KR?`+FmmZ1FB?dMVtkQNNF|$R%B>3Vd_^-k^o3=kmbop+lrE>LsZ<%9 z_Irg(2gif!a%IvshCnMD*+iuyQ8KZI-Do=u;WuhFj(9Zs$Sqw^EuGlFIZz%qd;KXP zbz$^_CfHd=r@1ze{VZ_gX#EV>P}e|xURp(g<+Ly?$WrVi3mw20-pw^UZi|@$1iZR%CzfW*3o%@*VGkz9g9!Q+F-}VN?NbNf>Gd z7zrt#C{&fEoy|Ki#>Cg)4tbpGfc=Irk1I#T=5a3{;6;d?hY#9#B+oi1+z0}|M=>g;1)t9DgvUuj+B>za+2K*sy%Ma z<9)$1w5Dn3IWLe_xGE)j@Ug@z_OKvq8?b}`D!F`!5X9W`d(#X8 zShqlFMn-R*%vzGEBLiR z==7G7<-0jcj3CR$fN71yti_U_fc@mgaO68D9-)hV(uvJ|0NK^mOrK-F%z%=b8#Mu0 zWUp|pEY->M0uO0|#0CTCS31rReW!ju%k;7p;~kkE?s&rwi>?ia`ikQImVF>Oa{bTP zhXCU#sLG63B^uGb6tp?m?f%v6Y6GM#h!UnYK)==>bjEJg?ydCQ%2w6D@fFzC+p!MP z(u&N-fJ1~@?DrHepk(91M5ss?ih2#2SV;z?e2!HZ3Vd3C%A|Wirj;8t zd_#(ST0TjnPOXn{>eH&7*A|&*CFR^5FQ}}XFcfd{DJh4VL|2A94t0%)ueiP?P>lq_ zd>?q^qaS|N1+8F=7K#yES~`fI%osrpC+GrW7xg$714j8!;T{=+zzE;+G#{F>HCOI+!{FfS~RUW@lf%TUR_7HV)lN2%D!b~ zEr4ph+M&q`IC6SGD;mVQr8+kmG~K4O|8lYO?o7B@6svwMSPG3!#IU;=J~NaJ-8pSM z?>eBa*^<|5twg{)^r7^t+hFe7`TPeD6VwohT>E;&JgCtUTHXj7)4#~&8JQ~&`QE#@ zHDlhgdo6wTS5aRCK?}trN-g+mSsmy!@Yz~~wDDKGQ>XOy;JSEr_|TUWqzABkXv<}X z!VzYxsb7K?@|mMRXObE3D5%Flz`mtdBbS73ZbiP;ZL0w$i>06T$Dl#hVnmMrW;SHGfiejPCH8?sFDt!i?kRPAqqHo_KsrCiG(2~;|- z2}&TG3Lmt88Oy0$rySM`+Q8I-qECAEf@E8qt2wS60D{g!Nt=H{+-(0AP;WL*;nJ@> z3GCL}ak^}LJV&vxtPaRy`gyq*UvZhkaRM_+qDrkD&u~8fmDn(gmVwDf$y%tcPw!Xf z?kD#wOt&ap6FyG)VZLWyE|?wr7U#rVQjnYn4x_%Llm3w>mz_a&kFlcDB7}ntvL|Dt zo9?zt4}ExBIG~i#L-`)(mKC1_o;P{)6GJpVOB8T3t9{U&;_ZP|4Mu4hVV~Bl`HLQX z-u;wx)O>2&dp!(zw~Jl7#j)H#`l#iI2Nz}zhNOjR@6r35q=lLYo0sMa=)#qyr44x5 zlf{KTS6He3yhtc1LcsmKmA>$^0b~C7TkkxP*_lVw=B2FSlsLs2oO%wGN_REL@%=v1bXU)-FfT4IJh=|ZAGfhL=JaLTR#{F$;i(GaafN zF(eun1`P;}*R#!X+^^83b6vDYKkMI3tglhWx?J5Z-0k%3j@bhu?O_9}XGN3^oPKSf z)Zq${X`EsYaL;Py?@{0;c_a{6Zb8D$jzxxLfKW@V9_m&%2x{nu<-czPx0SY6_h@x^ zJWqekJ6$|blA-4gJX0vxB||4A10YtiF(|}^1;N(hgYGq(P?5tTcjODo$}b!*2lOna zQTJ-|$hKp*!2Y&UxIT!CfK?i5>OpaLgXmd6&Q zmKw!CBotFSAuqNVHCz_KX3(gIu+UW_F6h!U1!y26L703#(QP3{y@GSx_A&*x4%8PUVPAOf4_}(NeNfBbhMd0B zSrR!<1BXFfg49Ce%DGy`_T_zrQ z3nJq@4v4x9vj-jVVWd0upiO{7;<~+obUV@%2C8<|xI9dmFo}JwB%K-obO+Jz#}oa& z1(8IsMx@MFRz_yciiyHWmiuB2G^pvQK% zss~+WH-Y`M#T9RnFTEw1EoG$g<>`Du*`N*)5vS{#-a4ov-d$Abw|7MFgpXzNXBGG zebSS?%-j3ZHLKRSzMy7Jzz$SA{4$vbRf0C8M!yKW_ar1QN=Bsj%YfTqxw0?ej677` zYNbUl1(RMzK{X?yi!;E$SYC#+=BD`zaa9<{`g3eobzdT%4tIIR{$1B+LB$E9Jss+$ zdc6h&7qRhkQC5+_)0z_7q$Rx9=kcQ701rF&RUztd3*@~xtT=%fPEdNi!L4WgcEyOY zKdPC~Ij;~4bD*Lf_jocxJg&DEq`Le(aHW->7{LOixY}66HCalBJ`c`Dxj}OA&1fE% zoI5=|%JkmD-JdfQr|SJ&KnJGeE$k&&3uu$p0pMV8f@;UG!ToPL;~cOMT4VUlJ7gCg z!bDh%7iHSXTPLZxyF^cGi^TDb;a5rFO%ucXl*gj*lCj~>thuaPphF%fXynMb^EUf5 z6J{~8Gt336XG}dBcv5FuzrWz6o6GwC(hYOvi)2|GAYLtY zd*Zpj5;rW`=(Of>lAz#FoHo=e0z{2*y$=kEE=I_d*aaP%o&fPpCOGwcuy+m zTs;UKKy7n+hxZv!gRtW(LLjiR&?7~KZU{mPKJyjL@IE~`NlI`U8usT#YIv!9+Y!g( z1RWua{rS+|aVxc<-#SrN*4;hBpPhm!5CY5Hk)w#Ncb>^D!~1t}y?M-EScj?@Qyu;C zhOb__^X&=_YLZnyE~ga)!2(ZKn#7+F4iP)-ZDCqcqmXC0G1JRqhse1+~K z?O7NEs%hpFfaFxv3w6(KI?MzYTsRCIF`Xy=kisq=0Y}m~#Tp z9u}CI1Dbk4OTpAoq*tM~<&!8>Hg0>nBBM&%xUi=_QDZ1~b zMA_N*fzdiTV6beG2;(_6V+DM`N>u*)Au7dz%^N-9JJ(y>5g==Je>y+L-$s|#LOU($ z$93jdE@GyD6{g(4B zO{C1f&-@`647Mi!ajtid;{!0W6rFF*?=wS#%lRX?Fwf460d~U&^S@e^nn2!9L0QLh z;xQh=az6r+Nq=Jw8ydhI`3iKwXZx&P1UxV&aF)5C*HXj5`$poz?w@4>E^Tm5pA;{& z{IUX1llb_ zBqzG+1fS(vK(Zl3GMdKqtdKE(4UwFz`rvnx!OgTjRF%7Q=1MGiKqOD`XSV!KG6^8r z+gRr8_WN9cW_ZZg@*6KZ;{YlWX1MyEo*_d3GHW_r>MdA)Bf}VY5O}`DsGQk2v1A`W z2H87#<=?1Cfc$8g`~zIK&YmC0S}xj(x6=7ta*UEeVWLyzf{Y?dlWdi5M zaJB+(@ENCbvJ+s>74pRxjo;3MO9PN$T*rX-FBKOtNg=QG#qYO%>(ti04Auv$R#i9) zIn2EQX{+8L7QZzS6PpFPjb}2R@XV2jC3k|&KwXpD`F%5>;7g{^D%TkSc0@-5a`i zRxFez020at)46`@luC^S7UE=2n02lxn+b%XbQK-X;&kX{{y&Fv)5U+Ifgb>%Xx~p_ zL}yQw8iL0o?qOtSNrR3S$cpbS5as`zvklO0MXL%7cxTlcpy9g@SO|GMX3n|MBTy`B z@YDF&mqCmGWRoP~(%CaI7X&2OxXM^uKl3v1S}}ac@A}6BrE?9Pze94VJaO(^A4)L# z6&UPD@ei@k-~W$=|GIhqW8wdB=KomuZ*BhnjTGKE9TCLkW6I&dZ=ly#>1D{i7yZg7 ziyj8PcIN8J1ap0kMNAuL^vZ|+F3W`e!G4GL#x*Ci#kI*`!Id@7&h?t90Dh+zYfdks z&?39FQkd6+gs`r^x`0W2;o?=X3mDg5iB;WyrN!*3+D7*J%v0!}J}P1tinMTOeRR)` zzkn$ddBLd2#EpaW-%0-dOR@h=|G&-o|0Cx(zWH)Q<2@!Qkumc9N08$j#TL)qzb^>A zG={EAo>`4eZ{Y)flf0`Jn~gB9s{eI&{$~5+m78MBk;nXV6yQcxN&it1fV(Bji`68<$&xC6$!77h`gaS=@F4<(o) z$2O6kYjscmxvt4sm?2Z2)5|Qh>asGO41kykx_n{g;kj$51kgQk`((UBEA(fNsTqkY zwi{zU{(IfUG3zP#>6Tt=T~jhif2H*kN8^g$-|P5`)WHWAn`dIvy2_v(xRoHL#%xfi zIhI23E#^Oj%whUSmqd;-P03+TG8wUpeT={`VrSC2bL+3A{=IQpp8$PUVIo6sq2)V+ zrKOb2%Ka^L5QJjPw1QjjT1?!WxX*1SK%MVeFsYrep{Lr^fItf6=E;33y z`fK~z_`z?*H{WZ?Me=I;p4(YMuo}4w98nNIe7Fpj!YpR`^zRMfrvtyiUf&4%vAJOR zpN;-YKx2T_V2dpfLQg^Idr|b|tZUeREuMZ6{PsiPp%J_Gq#oY6?SJ|Ttmfi(Jq++v zj2JAl;#Pdox~{6F#d|Ekk0t{xq(H&}$^?JwXEG3M>Z{egc* z72`+(Q-fa5`A@813AB}LhnoHUzx6lx5(m)VYIpzC-}f+ON|fG@=iI%W`Z()9=n8z% zK^9ElzW7!UQn??ffa3CU=KBvs5F-cwCy!VuyGvZs%CjiN8py@m#y^6gf9s=GxKpvq z>3iINz=UHA(2Y@MNxS$LuZEifC0#589~b;f13v43v|~F3z#r@Jl{&PJg=q)jT-|cb z&!o`|%WyNHf02yO0MN|2+Ded7r4)H}E=7x6*fPzNa|!iGz7KqL%2Yi^t*9*`S8LV9 zQn%zqN4^V+Mq7Rj;O$_V^v%;dBNmJL`EEhP%UH8l z0r5s)AvA9}Qhlxm-Fcp(DuB^K^kzRHX{yM1S9N^nC=DdXRKeTUJ%;?r{{r-Meq& z>`MB&1n9CBWu6~ZI`F`En8&QQST<|tn+PVSCcW1ul08#d^=Dy=o_c0p*^~B+hy^?KiqloY8nLZ zy59cD(K4KX8-M&%7=x{@jN`KI&=`B&CswDP=Y63qQ0}qsShI^5TU`^hEs3+O=?f7) zX%?zD9d05Rm0T={%-3NN@M^o3tDXx>tl#I>dpS9>vv;3hemR9G#kZa+HnT4IE#7Da z&vD&suc@Yi4T*tcRW30T+ltdi>{g?RZc>q|)0*{RH?xGb0tIGd{z~!K8q9SHE5E2B z_9Lc`_04V>R6AQ;cll^Wrtq)rt&w`Y?Auge`CI@C&+9KBA_JO(P&aLHqbOdP>y{$C z9Bh_5T~}k~_N8kpjblnpd!=@Xhq@-5yju0B9M4>bpMW1YXCJdckJ5u72ZGt$Ogo}f zmbHOnHQA5osRb6JFV?NKt$0oayhmRr2X=k5in5f-CU{2Z=PBw>6uiei?H=ml43ay}(k9!i=H%L^kG3(d&imkr9qhg8JmeH~5VV7hEd5{OE)n4;OD$i&B z6r~Nv;3G*E>yML`8Ls^*PM7W5>G26k7_1-=*&jy^=119y;y(fg7VUc~JF^5>@$#aI zMZ+b6N6f@Tw2SjVxJ0uhjtfR7{k$1mIonfiFRX{3l!TbL!M0S1(j;_KTO!J!Tq9#e z)#K=f)<-Sv_ssVH$ASc~y%!+o46~*if36+~7>9usdh%r*qCpu!^@DMfmNugDMqLM9 zvrcO=ZpXCXz1iL26Sf_A6KujWblBMM#?zup;nUwz2}Fk{-B2BWP^8@qLSEOW$cpgM zxf^XkD){2P&7kRVGF)9c%e4Kq`xF*c(fv>no-7izm(x{dEALe`3J`g!nTX&Ru7T3n z;s+|v6N4J$PcZ0 zAM|;MqW4mF=1G$L6t~(Z+^sFvy&irUqbhSuH0BaZuEb1X%DWLX5EK-?)d!?x#>G!T z9m+9uS7J*24l9e>Q;jQOQJI2?hBYA(COEjYDUrDF#5GY51wCQ?xKZgd7SBG|tA#HA zROpcm6R<^dVq*Bc1Jv`hij?}cXLqMYXh;&xB)sG%P75qbUhK?vHRbD|YpDwHDK=B( zV}$rCLmC|sL*+I~Y+ZhhSBv@-1SSpKr>icsme-Y6R7{=HL))_b8Ea(*w-c{gT1NQj zi>J8fy60|5W|B8Q`0?Fj=+2-AvMLHkfMs^p&y5wQiG7xgo`xb+5cv?kg&tB_dvf?Z z(Q!29PQcf%Q`d_76#2iB1W${vdH(TS5E(t&C?9?o82ms1G)Yi3NXT0CJzQ^q(_qAk zUn`1EB@owIX}SB3l3+-GWL`iNu`)M&$4b4z3m{4e#9;-cT!OdFzOe%eVd z&QUW&kP=+Ero}g7>LIc{Ck~$G|5}z+kadevys%D%ah*OFdg%)Uy^*pRt4?Y6&WLA} zkmIkS%*3)w_yTEpBE;HN>tIhb4cxyf@IOZfHX^w#hu<)8az-{_tyWYpq-FF%t-u>j z`6sKLEzL&bl(J!~E=NTEF>Q0fyerRe+?IM{Z>jrDFRfKD&&{?5Z)wJyfJ9+j+uisr zw&L7^n!8zMD&0A)lXd0gG(jX-6Ey4I8z$b;`491~WPI4d+}HjhX!RItz9mOZEXXT zo_=F!f)ax;djxss077(b+#$lHRwjZ`BVUJG^(oG7Yy3?&2e%9xcRL5q0mNL0WV?e) z8G~P!_CVw58M(}d@lI0eHKe%2M((}R(&ILiBnlEZ95VDOH_%OAE34Q-)Qq(k(@?}~n zhU(hhxVv?k%lLH$n@^_)?E*` z78$$`ezdNRb1LHUj=-tBjC6O$dD4S%Fav=QdZ)#_#u>i6|E@(&I%3+3&~{m6J33d& zDS^rccSt>CaDv}qb#y~(rn|1R>X(WlZMB_cdd<{>F(NYk6y>mjUX|VZUl$Fg ztC?zqMqA66G#yhO#uf-vA5`Z0cVSm3LkR3|56(jg=FMA>0;i9I*lHo4&q(?H%Agt6 zA*_4ot3-E}vN0Ju%t$HonTFQgLE{nxk7F~{#p*b_n!YJBHuI;ddq*XT)(yh0xFLz2 zYnNdic)Z$E@U_xhwgxV9(e*~zl{PB`E^-;Cpnfwf!Fjyr>-5g5;P9^KDRKWN)?1Or zy}O$ID1Gk^b=c5q1!m^%s4W&e_tam!?xElQ)}rQZ>#CBaM_sdZ<0-NaxUHan`XOY2stSeoaqH`eHfYZFlM!{H^(Oj`s~ zYK2Ayk}yO;0DL~QZxECPP?uNVd^fFU$ErU;(f&Y_Dk}b!vrAN_m22yde}uXiE0A&v zjm%q=cQ5#CKD|&>X{iy!MBF91C|Tvm}o zzMYcva!EV`UZ883h1bC?kL3^**n zPxG40iba7$6sU|d36A4(Q1?DKLU8bWf z0S(c#e6OH%TG2xl#{_)CNpROk;$4yseAQNuorL|9@NPc?Uo5QyU(TIa=twYgK@R>} zuhHFQ$=_8FBtr7{-galz4}0Zu_#XejGP&*q$;1Ob31T&p6X61o*=oh9PlkE`tG@l3 z_D*vuK}`;N93}`~^bPfOe~v3?QxSZ;*R=c9qBYSzY{F$7xyZoxAtSj<&wGEL?r`N( z=<&l!@ae}5QU8J3a&0@|v7#>4LAO*^i_Xqk?Yqw=xIt|{s=XWF2AMt zmOTBMo$^z0EZyS+Je6-an$5GvyOZdb@3*`{JGS_pH`YF*Xyalvu8?pqCtmRs?yjfg z>xRG082%ZcRXY)%k(@F|_Me$XU)SHyD=((ECPZxKFmRGq~8dl1i1~gyB zCEhELoMd}4#Oom%6N&aOz*Qyf?2FL%CJ|A0%b1@T&zuZBxtOnCQ+}GGo*QY=lKRQx zdG>7nrP}gW=7o2dAzNf#NVGTjyo;dz*f~Y zEBkD|EEDP0~NY!mF@UKdrA!q)hTL+&rU?#^#Zk=7GH!B z9HL1KEl~n!)KEpW0qfdjnt&QFwgz)veed7^buVe7btKBRbYgd$slm3W^#S&J{j3K3 zu!Fa^)3)5nqt@2?~a=i zOu7~{7~1Syf$03FsDjB;6cu;22XrpT@a<`$su?@l;RaPJJK5?uZq z=Bx1e^&{4)m+hmdrj2(EEcvFjWTO(3i^^)va-RM8kf6gj?Z16nn@>S(<{ciD z5I8@+UsaUOB=1X0w-jY2QW{b;0pjH8d$`58J-m`qQzI^{(}6(`X4BSJ=6|U4uUedk zte4sj`gJVeAtwS30(DM(y`q;#J^t{Ng@HecEP9I5j}Kat#3zVw2%jPh<3WumxfB)b zDfKnIDQC9?0heQ?Fr64pC7mK8KjD|Bb+Fca-Z+t!E-vZE+nyUA#qcG&m_n}WR@(dI z57jyZh9A1^(OV5vyKHlaAG6t+&Ob>Qz2cJZ>^_gKczuvfM`^~^u;-!Vy;MZE`k^W0 zDzc~Xn&Knawn>^Sp(CMf&O3IIT>L^C9x{3 z6`lXFlyqmA#DF6p;YcN3>&WCwD-0z+lqtNLy=bsWQz_uFvwUI9yFl$ZliTsetYW%w zLmI);N(zVXK0JGI;(ny=p>;dut%)9kIqpH!9MX36C$R~RjqDjoHFCYglgO5#DX%Vp zG8=W1qL)*3uKT%fIja&MFPE_GaI(-V)BjPTAlg!zuOTi?xQkkUo0j)Y-Zex+l+y8w z2t%%3AKGQLnhLwoiLG*&yJf+_=-1v&#qvu#BTWyaIi(Lgb|sj?#%HrX(j^65Pzit5dh>?t*aEh&%eCYSO{wF{rsqh=Oy^0T?D2IWLUPYkz+ zS>!la8`#27xF|wRs%>vJ_i8rcQMbD+x@ESrp46huG(uLDTQ0m4MrMf#u9v%%{V8tc zY4w}w_uM);ddzf$sOC|Bv^riBASK-)%`RMs!J+S2B_WQgqJJI z2xLLA`%(7dYxmNPL9&9eA}NH9a;EXv_KNAc7a#Q~OQ=@uMp%2Dx*glzU9K(XlY;3_ zY*?UN-EMja$DK8>U74h6cTI35mWlOn9huANsgM=niW9ZJxYWbbhI@Cp(WcrUY&2B# z*h6MX$MxsDa?#6OPl8cDgO{o#VSQ+)s^$uf_m-&pG{RI9Jj*LnzNl2aM_a%FP-rV$ zevLQkrS@|@*C{tFGQk&eiA7~Nc1@jR`=SIA{zuV-ox?7JZKb;+M-N_#B7|c_kLPA# z^%Y3IHmxLO*h*! zq^R7fwTiU5*SUpBsJwZ)@w5?+Kw|X;KoTp!faM_Q9#FVZs7A&b0K&&96zOg*b|u+4 zEq0Di-Kyb+7CEw-tcnj{d+>WbYKKls8Xva8`g z@j=R6Xty^vgGhZ@#o7BQI#;(MabkH@s-mw`Fmq)!%X2#sJvYI^Ymju0>_KL!O14C+ z!ZNF|6qQ$D-PT~-*IzXUBF|bWy^WMYnI230%xUP9jY?rU^xAWK8F=f%tJUsmw+*)h z4XxnBha;zQ^tM0EZz~3LRSu~15CzTf0#}6h(|~Aw*Qq9ek7rle`9|uGjyUbunyM2E z_qZ!fGP?W}=y6(acpt5;v)zgG7={(aMKCMwjvS1b)z`< zM?9YjGp4t28L7N~-q`t;d#qAz)6f(GtEtE-wW)zItQwyj^|99MVH^TrsybRwyCBdv z2?~nM?Wj8Q>#ipR&MbsY`ahbXblkha0M|j1;#BPaXr-z~Ft}wHDT~IfxyNpmbO}y{ z{PGGJk0K;y(J>E8!1+k(B~p5d3Tu&lEv!Ohpxvk^I!oAbpuf>#mNYs~Vi>*Ia>bMQ zVEK!k-1OtwupQBrdhMuN%ZM%$MWK1Fn(e?`l|;6Q-RpKqKRFcz8A9z(Ha@Y*HFB{^ zwixa0X5zBdocj3h5vuMAnr@nq?aeQqOy=PzlPS<8!QM_d9_mAUzwL_l!lE=$d`T!m zLI?uL+E4(wW?0k7=%J8OWkE6#C z+-{L;RCW`WX5wv8V2fz!sEgNgMw=sw^y6wmkO<+?1!T|cG_36(XHYUI&SZ}C*Pasw zR%q3qHr8MeBQbi~y=yJ*kgB+5Q7Y-CxnH10&STXK42FZ}Wn@x^JrnR#i420)+1?>T zPm9CVDqU*AezP%yv9Ymr?^{E{T{40$y`G}zm7x8#ge+9N{r1PNUVZ{>6P6+rs*Hix z4;N4cwdy8yns|R4dwX$6syzOzO7uuw`mSo34rgrnSal7z|8FRX> zV_Y=OGmC}<<-xooTL;c z(4k*Jt+}KYfI13$6axBrHBDWzYtOIPXWaILDh}BsNuxg=`w4ciS)}W0(ME3bI&Id# zwD~hcJINL2!$S6HTGjN*a+JM~GCu{aQ8CgHr#@oW8FB&q9?z>-X!rl8_6o*r(BRGl?}T;l1l0=@QF* z@%fq4JJbX3HJ8ei$ZtIn|Nht$!C||(9j-2KdzAG?Xy?tR&AfN2Qz%4z?;1emuCXL9 zwq$DNHklyrKn&;DXqh$X)r12y*-BqG^9O4qvfIdyR9flc?W#Bq0akJC(ZiUy0<;jE&;($AMY z#1u^fRNixUYYy7lU%R=kE<4s`+D=rIQq&QZ*Q^)0CKA0}?o_Q2NuBVDqhxr_L9;>S zxfye$fE!}bC6mABB;D2Eg`XnLG_x~lvlzZWa#%NA@Iv0=efHyBd-bTsz?KcKNcO10 z^vQcsARO^aL{t8l+2ubj*0u2D)s(F+_Tx@RhKY5t{A=TlxUMzLV>|6MtHv+;*;}hD zNA>Q;(HuyH?A>2VJ=UMy5qp0#=%8wVvGw&MB%_{XLB2|Zd9tSt(WC&Yq_^gM;u}E@ zUDDE}cZp?R8xgnKJ3jOXu#aQ%cZ!%rS~_9$pp_^5w!-&R=oMXU_hK1kO}H)lzZ2Nx zLchHUOCny!4`Q$b&r zD8mxS=Hpw#8MU$r)rUE&dzVFPusrEIT(3jr508nOPo5_GvP^ircQ9gaIQg1qlN9&fZhWV+MySR$a9lGD@y~d_`HF z#G(9am_zAG`;a&2tyRJH?vnHKSDX}Lhc_$@zsaSo1$Bv9WhT`gP~slk37Kr#wotX1 z?M$d8ik3Z=IH*;8?o+1tj89Sm-zvgr+=)b_HKJ^HJ~GZ#9$Q`Kc2QOx85Y5Q3tkk@ zl-aKSN;{hfZV1YCru+H*qYt0OjpaU1wX+>C;Y@sn0-$#2F0w{+>8Fs2CjrZU*_2!`=vQ4}ZFBAVE=W+bv>B@Z%_Pc4^%b_zl@p z{qwFSTo00b71__=@Vcw4W!@6EBA1+wP+ycJNA(dqd5OTm334FU43CUwJKc!e&O$WX z~8a3#{2m^D!pccGwQh-tLD6-a(No!mTclQp7mRc%?u|e-qW2r=k;@JfTcxH4dP~iX_U}LaB$TuP4ZXUA)2#NUYrYPe`P=w5 zjm|m&2RE>OA&JBLJ;$pYcYZs_&P?QrRMceUp0RN9+;7W4PQyOXW9^{a@|&ulLB zX(@9lvdbFe)w2$6T+J5SvKJ1lUpC>}$a%BXge!cgne7pBPx8AFL>KRoGvPIaG~*X2 z&rNclaS*Z6R3uRtrf}T{0cLBgmAZpngToz)S~jht1$- zyeb&Ri`F>f0dXdkX)kFR4ot7C`nVlvFE%HV9_t5dJ!bwULQm5TJna|X(eS^y={wC1 znSPz>3cEN#k`Xl5X^Nl^6ym^RS=D!qX02RO40O1q+E_2Hnq|9vH2=w_XPK{4R@?EO zY>!R4sZj2Hq)V#;bM<(c-h7LKjy@D=JzO-C1F$bsI!~@j{vF z*xa4$yEsj=PueNQN}rDGIsh-x{K(d?X5-M}p@V#UhsrTZ_$Y2cEhRotnqF%ci29f9LNS?!lt39FHzrJ85)kYfzdWY z7s48V{$Qx-&#YckUSNszoEA@ts!3B1*OAwbF74Y9J{Fdr;4f(Cn}RJRkcROqFnjF` zRFwb7$n_4b+?w4qUm5`+t*@my!vC#iu!C>x5`wGR;Q2DUv$_pr$Z9nM$k z4mvs-(z@r~By)dsqv)0t!=Vk16so{v`@!sMu0uVUqrn2hDU$aGgK-OO0g_RWWQfyjMm=HL5c?(c<%Nx$a2! zUC}EOWesUNT0t3rM>eVs;^kAEGe`n8vU5l#tY-SPPjVg)DKd?gj<%)0epD^;YpDtL zwaTn&4K~A&Ib!3AAZElRVJ+n8Aic~Xe|Q6+i!Sj~^ScVnGI?pAw0pKVJz1R%p7V+9 z58uzY9?vTMD#-(R2NCPw)rTH(H~g7&bDJb+i85HZh#s5iPFzQt=feiz7O&r1xp}yk zN?^Td_^Y>6vORLAC-ahVi@pC~hp(CJuKQ?=nQk0UOuy{45v$g7 zU;Qxa1P-Uf+!x_eG!D<3XL9Z8jSnc)m8v&;gH})KU9l!gcR%llG1E&Xo$?j&S>dSXE*@koS{a~uy?8V)gC78mJ*9pibikSxNFQpj0;)$rSt<)XeyptJk z&L&jh9@^1fQ2mXya<6#!q-Sd2on*7oimwVr@VgemS6YM;OQB)*j_67~8wWPJTTT0f z>$*ZZ3G|<%S;w<6BXC`E9^Ogol@Qy?5LL@mNFZiEe`O0WieQq5W>Zpi0Qul=iXswhxn2`y)t36B;nG;37A1G$wQxco5OvNwZHS}g# zO8?-4LvdT)?}ky2Q|fpM)5FwO(NlN(`dq2jkUO*M!b^yoME*%0N1qwUQ)I*Bc_$rf zauK@AcNy%g`CsC=gJ?du zcJkuGgxg_01KhFtYpBqHmg-W6y$m05?sl9ulJ?m6{)-?hH=t@ZuoIvyFGdG>wpeeG*sd+*PPKcwW0 zMvAgedw5m-YgXJWENw|?A@`^mbV0uPe5f5~{d>9~hpyY9a;GjoJ1$I;LQ^lDJ6Tyx z_}ejt#nG?+VlWBx_SOfeJmiT{iurDOuVSP09cXM$ib-qz%L9qOe18ZQA2Sjs^Er3U z2?Bh*Rg(Nac~rDCxYi?ltaAj&YwZto=9(|Q@%zdCz|S)**z_`G`I=!2H3G{fULs8v z8u&acN-9BjZ>yK1y7{a~f3vPr-)q*1qv_lF7`V3^-Irets7@Q`r)f6Rvj&%DYOUUB zH|1oh%VNHKa+8D*CAn;2%6xCpV_n)f_Zx0qE#tkrdscMz#Yfyc7XoaTe|E9B4_#3a zTvP1@i3MK3l6x1*)y!+Em<=BaUxfcb)$m9|Htz@QP|1#aW%6pVEB zMvPH*JBe}33zcgzJ#>U?*9YGBbpT5jz=+fI38 z{+x3TunCbW{&Vb{A^CErBC+Z4Qir)i8eu$Ne`2#^FsgF7s`-lT*K6~W7vGnyAhOXw zB%rM5ge@x;x7?#hTM*95>P9;pqo)UNw~%8R0Q4Aka^!ZREPjaR_;K^JG>ZOz z2hzpTVe{H8N2l(MxuEtIJ;ZtoJ;#|N!b6lVS1#>NrTVXmLRO85;)?#36UL=HV9#_x<$Kji|y7 zlKU960UJI+i+-sTHPuwjWIL(QY&jw_QI9|>e{N1m<**RMuV>m*Wsgn#SI~(n7$DUx z8R8c8^l*)CFCyM#za>PD4;nI>;_~l2@B%D#9cRM@#=F7gLr)st3Cz-;*J!AWT`n zt7B>b3E{odg$@xw|Naby{|Zdp*>U1{0A!GW%d{u^xof|!!Xut;wbNf;WRHoFPq@&s zcdBE@wTrSb(h)9dhC$_7SD4ek=(ndQc>VL{(@JNmbI81dn}JS^bAGkWNUKJP?k3f& z-POYBFR#KygZJWj%G!vG(}*$ONcQYRGPg!P4+nr~iEo#SDUvCEw`|5 zgxnCb%GZt9eb@Y=U_{==ZvDa2Q&4=Ln`Is*FW1W88^sCKQh+^W#Q?o5ZQt(F6sZg1 zKsHe(^C+%^7HZZ+2yxmP%fF=zanLOOe&B0!ksx+zt-*R)!mf8b6>{JHDGNr?=<4lduvqc?YEF>=JW$*i(j7a~pM=L2(i6rq8 z&W4N*#-LNG1T`LJd=P=LS1Zr1+oZJc#mBXqK;?N;~*eWJ{L|U(Yrq$l>y&h(t+jiwAA3*B0q@E%Yxom6Z z9k$=N9$`J9j~c6{62`M!zr&_^rj;bzY@3d>v}&^J~_d&+BrVh*Ix@37A5(l^xv zAYo#dC%i=`de%^XxX;Q$;G$sb$vl^zWju#jJyUIs!Ai&glYvW#4F(g0VnNutj($~_ z=AfC(*Sp+)g0X8hcv+^spB|q_cMe7SPwFj40ED=f->`q9BlTaukQyhgwG6FL zxm>FS4G~dXs)S)H(X&5&hVFy~chYWj$M4uAys)-<&xX}>%lSYt!g{hI|L$$_k7iIh zvwf|^_Is{D^F4EM^jJu@f~xakEtTKJJo~AJV?ADZSR+w{Uuc5z&$veULM?J$*CN|` zXf7dnKn%ZjzuR0y(A6+pyT;1wnRZPD#nn#IIzRIlE$@|T&E4}%)s9IBFL^=67fUMW z?jR_44I}3~c3sqy zj=MazGv}pg5#j^*LoZ+2&GFr>iCg?>GzKyrQeHb7%ox@P5?@*TAKg-nkPiG8di>tc z{zv#5z5#@P$-^yO0;EL_{sKmQQssF!Ydi~OCOs?#1WNqFD&FWKL=K+w^T4eGSe+j9^LHs+IysAJCkR4sTUZu z@GjIOyDo6AMbjgeN0mf=-wnLyEjw$znjRf2#cA1|={5NRS`0NF!``k^5CzvqTeF5L zN7JixjUtK95F*yYx$^d8IhKiiar&>+cu&*Ri}6FFBo><;Izi{ahEHN)ItR%lmF&?3()(ML``#RaxZfSaoMQH}%oh=Be=`f$x8%{!y(e;LClq z-3o{#-yg>CZ3-mjwwf~-#j2Ky5V^NAR1fb@X|jTT%o5{s{za!GWQ#%J7%%GUdl#3b z#m4r!i7EgMJ6B6`J-Y0NCk7$hKV8+$kM)BJu^~Ionr1=~#Xx9zVIEY}m07 z_2%WR%*|; zsM_73^<9(lntMv_NihJ|CzQ-nd)N^Oq!QfM_G0G$d*()2Xmj3sV z$6ouxj0&Q-n<`0<=B-nGb`yv-O8c-=ujXqPr`QGUi0wFU=7g?PM7OJM?=$ZfFiCF1 z&686qg7UZ3Ac&0LdS3Ah?(^KgZO=u`XGRaVtnJHK)C=$LF8TjD1iAFf-AYnLZM|ak ziQ(uhE$_YL+TAV*+(&k=3ZJ)796Cw2@cKvfSfGHmitnH!sa5ChJriP0k+y#B;g3A! zF)D1;M!s=;@nOlSQKrXR3Hb5Pjr?2uUtD7|yV*5)o)jh2K5>a*QNI?=V8EM~_iNlD ziV0Pum3HFIv37I$U(>}`LSR^xEI1C*u#pVFun44K7kr#crK*{?B2QVPi$JI+zN^Fv z@K)OxT2Jv{VaUqAB&N3;V^nN+o_8lOFx+yg2Sv>>96?$a^sE6a#$E2o`nMg?F&4ki z|0d}Kfc|N~!29;C;LR$*;qm0vgzujTS@Z~FDBl-S{&s?{L~ZA*l=;)CQF<5pIe=KV zTgJgeip2QirtJZt!quwMHPEO+0n4g0hJI268dukuX7irr&r2|Fi?R~ihMUugB1mV<=$~rpc{LF`*2Y!a zPA=RfDL(Q$^~L#Jw+p%2_;kDUdt&S=X)jX=LmNnMRAxG2RF#AV^nDL5-bdXXjb?BE zxW&{hcD~-7G_c(W7oD^@?<`2rX#2b?(ojc&3d~Tdj4-ixBg_yk84D^c*5HIe|@lMAcKjmOS zk(CN~9b+1rQ?S76a2#5^awDNSw1p@$9pghDo-m5y2G{??{BHosU^8N7gS_d}V919q zf+r4<@j%yQn!yzaun-~^+K9XjG47#nlJQcYRYklW(4+{*zxj z9e$_ZpfeXDn_LuA7h;_2L*3y&h+Pz8Mdf`6XAZUUm0hRwId40?_1gP2`D`s-L+N+p zwjD?h)-iVRBeZGA<7R3smZhMH6y$QXJzN|621`Bi95q*03lvwo9mV(hgj^#}(Fvi3P-_rwSd3q8}4Arxs&AVz&#m|T6x!w&4X2AQy4 zm^wzl>av7o83*GNU9)ZH_|3Ah!Sz)D1;-C=$KfQJnw0FoFV}2fN{yd<5_S5e;Fq1z zvjlGNbL<|5S7Js`29lFLDOFXDB5{+waOdt0f3(c+M(5e(8JsV)!}Y@jz$N_~m@-)c zyG1_jaYyF9ku*rV@lPT$n1qnJrwaz+A0?M^XIxg}V=p-xZFU8n6($+7Z%StE%ePD6 zETH0C)ng9oFD2i2!ZF!gqLqQN&1%G|c_SpR_R`OysYV_JHkMw7@8C z{TKm#wJ$T_hgDIEWp67FeR;ASRw8L_lbw4uk)r)I*4G(Fj$KBtrzFCKpYWP~wAD^1 zxDh3q;9|$r`gqTsqUvz>_Z1~qaY#RxC1T}CZi)pE|7qvsp5O57G)ByW8s}}ntb*d$ z`Vi3GPTDmu)uZ|?R5_j_#h)D+DXt?2ZTpFC+jjRv?pX4Iu(I{ow)ByFBbG+-Ijh15 z%N7Qu#~i)mXf{3Yy+$CXoPBlmI|iRrz}p_6bhBdZS{&XO7$=aVSW2=%qG}0(NS|tJ z2w#Q}gebCD#Tcs{%4}5~@%Y+CwLKk2c(W#@WTG9`CS`_I|8 z5hbECRi4m6V_(M?Cmk`g`lnBGo3YT3hj6ueYj(`<)Ub?g3piPMi2SB z-U;=_K`EI80w(s(7k;;v628|9?*`(R00K*uWGg)1VP7~Kb{0r%VMN=pwVt_nGwef7y^E4q8(df+t6*UWm} zQ`mY<7^WSOw1|lqIi9#XhD0$n9#)viH#N~FUR~Kv6sR|F%^L^LV;Pr!SANHZXZ(Ur z!9$S$V;)_}j%kK97EF>6lB^J|HD>*L4&JObCfc}^6=Mt}%ZI$VIJK&QVmuE+!Rnza zX_bZegEx)#{>)qjAiZ`IfAMXb@+2tq%H+pdiE!fH$MXL&f)9Yo^Z2XPI~o~~D){Z^ zxIu@u4jMarU&w2NX z4N~l}3;gQ?|M=In2v9*08erLb2;|$y@|teE*#ao9mE}CNs9su6^)*J8U~6^-rqY9` zA`TX^>d#Vyd2084{|pyBN!s-rYO}tyb+{paik#PS_!t(jx+|HX=(Lun;Cxc1rZz1r zg=n8>c0F1jRDyegO7OLMi1sff|JHBxWnD1)(xJRzVbPZl!@em#-@2hDo?I<*wEY{3 zi`gNy#1q~}c1SHTXnEK2{T1m3CIm(6qbdnqmLWS}+OSjHdn{dNlh4pjt}*{6&Z?D$ zq*BkY$zJ^U=7EJTgQukmh>Y&Y*K82~T7qB!L`<^eqjfu%3<4N_h$7oSBJUus59z#x zX8Kd2OSQr_89b(EXwqH|b)BJ4qSjs8)1fOG>>xYGUjA;(HRy!$KVC}g1#o?^hhImL zwk@IouZ|lnr5B97;g|Y3`^sR}&d2q^LgneB>+9_{^xv8S(>iRG+lsT(S-}LKoV@HB zlADg&I$tHDqD`OyU|KQu820242PT<}Um}=KLKI?D&IISPo?uG^8_`l&T{ni>CXRS~ z7+$4u;m3nb9W9;?`L(Z{`M52HT5-cz$~RUsdKqvsm!K{sGnp#G;~gfd|I_Fr0NQ0^ ze>H$CoRz%;CvftSTL~r*=s-S7!5}m5$ZQQ?zEi+mI0xPIGXTlid+jx(-of?Xx3YrO z?IR2B#i{e&9o67Ip8x!sk?8YcLvVY_@+K|p|9tp;9C=9>+{q4=7c3CH4b`>LTqWLJ zJz_9ocHy%!X#Zd(*JqjIsOS6HfOr z;pHo8WS{C=M56io@5Yq$g=8bLQjElUlrh0Z`R8%Om(Hh2{Dl}Na^U(@oJ^W~xMLEs zm<6`FbIi4UeoUwGThj=aa6r>Tiw&r5Jn~NS0A(kosq`cX^M4=;IlAa^ngjB0{;xT9 zjv=ji5p+Aq_WH@oCL6!I8@J}Yrh;0Tcj3ZkH875bk(p}yRC@y)acWucedsL{X&|Hf z>;-Dhg@h;a?sIhxH5S7e(X7rpQ^PY=mQf?p=%{94A5JkTM7-U5Y&DCthG#s>rvN3C zQsFrqAmPIA_yV1b=-+I+0`v&PGn73-&YPg_MZSdlefvn9F@hHlnuL5ni$x-67JT!K zukEQP$1_r8i!rdj8kzal08BAyHBl7v4(%JwUk4+~EL!1uLhDEJt&ep7!7|RiagyZItKncN4R!(T z)qjzWX%rZKk?8z{3`xHsCy7;d#zy?(%GZFk}U@g7)jZ{o|MZ?wGlP%Kl5wIynt}m2 zEu=!|$v-2rzG5L=<^ZShf#||!Rt8RxCQD9K2=YD$1U||IQuDW9mA!X4zFqDk(CTS8 zl=iCT{Ze~$?L?h}$*BC2vYw1uT*FWvW@l`liY)cICVAuoH<)+-Lp_|x!D)Zs>mh$lv^eM#+j#`&IZ z22|FIeFC5`ViN3D9uauPV& zJ^0{%R#e7}8kg6pyr|F3Xit$#LH=l^S#Rt#v4QvFG>gUNCiPPNSPZF5ykd!nyOUfbO zT0|v>CK19EX)&uo9h!rYcwRq*-R+8nrEdj5(;FYiyz_X8-fYc`6{{w)`B4b9BM28) z934xBf#Z=Ch7vmqdBLpHaJN*PJb55`CP6U!B;QiS*8k279*AF|_~~E-jaZruQUf36 zvCRI@-N_C{9+~Mb{_+jUjOPs%1R*`h5JU(H$C%q-r^|7NPc#{Jgq>)h9fqF9pjTK^6UMbgx-N;ddT!wIS|?U zZRWI4XZ`nq1~>))pIBeTU@1oYfT@@c72rc2N-+|}$Uhax=L5u}PL&1pIbdTL^0o3+ zvXLQ^`m3B_*H;=Iuw*n1>J`CbzaXSlotH{Sv2!o--&p{`-^+AEL%?V1m9(|58NUDT z6o|C~>SFc9U$34A0Fd5ajSiE4ACZsTpfViKb^}b2AcKskpaOcmUD{%^(CESZa;(YA z4Ya4&$&tly%eVHI{zg5hdbdvNeX`ebw*Yn^5pa!zn#4CQOMEhjy2@}9SI|=2{r5*( zhla}k`{h4H!GHitbmVt81HQ4`W`N8M2z`-n=CApN-6Mm*4p~u$`@O{~&hS|!aH7N)R@WVqd`1@h5E@o_5#a(t^W#jut`O-F||fAqF6 zE}N+#kQxvc>4pYMK9d2P!BOP?dlX4hYo*Ri%qEvu&-cJSP`8~P&BeIq%%zCfk zT8-}IRMJkv8B#fuKskORd`0g!kGz9bkLje(opO#C^1wyhl4tFfKvH+$%T>Hd;6&KR zInOzdzIkKBzde?3Qf)PUc(p0iEuE#_E&8F-e6R+rGQXrX$b>loJa!F_&;kzX4gnDaGqqGga_DVt}Lm>zvzgan@`_{Ky zirI43I=kBAH{J)4Ybo)%OB6cX>r1mZuukBCw60EfZ}4itXK^-%eGw z6-UrlK+^KTANZ)hKhhH1ve6ISDo$`B7+CHNuZRjRDBRJHgf7Ktu*!K_*h|G(jpZ37 z+Bd_2e7c`IBJK|qMrQ`XYH>AI-0IXduz}ZRzjkp30D~s=kRzMv5rzft8(SjbVesu{ zQY^H42^1Ev-AAv(dE;2`PW4&u4(L}zYv!Ut#|zYJKAkPj5UWV|QCh3!_E#h1V?Aze z4}KibSDrZ@?M*hBFH@Diku6-OhgZEeK{^*7(rP*N$Z`PDZ!u+RGX_LI-m!9i1Oo%6 zr^-z}9U-)6tSJm0N}Yx1ROj(zmnlDr`}C0+d0ddk|Em+Yxct0pjYa^9Vte9DKPN5@qrxm zBk}Uz_lFMD3Xsovl7i9l$nYeSzq5re3(ep7uHl}&JO_8OJzVW(PSeZli_EO8CZ?Sg6%SUa8zB=%j-{NnNdYK3wWz84aC6k_R+Y{IvPv!F< z2ttp3aU7bi;V1Iy(5Mm5W*}IvkBHgd8r>cJmhCS7Q6Q*mwVIxBjA`=nkP9(551WMYr#}sZN_Nommh&;_ZvBPzpO{HWR;aP-G6?>*vUz> zGe#H-h*kaDvGf(p*naWsTM;>v8wk^q!;WTao>GVmQ;@}LXXnLMLX z87?T&Ga8qZ8f274L}*al?gj|x)!Sr;QHpjfkU#H(JiD63ne!vt9{*lc>MORL;24fQwvuL%@yt21Q?X|;DcFwaTB8wDpkegP-X*=I=b$C3%HL&L<;r}3 zO-ID98H`a`T)25x)pRs&p;JGHmAlp5x9J3#^_S|k4ImaDXDH4ZtreVg_u{lGM?fYT z-0Zif$}_nEv>*;=7s}mM(tnN;@GC;sRW{v%fw-d(q7KnvBmqo{;|n%|4l`c%|N4jF z$gzupPW^%bVkLO)V6p3Nw5sC6h2KF`z7}M*RZwtvl})LA_%&qjl267|mEHm&^N5SV z$}S*=M9}B{0h4KcAksV)G+Wa^zE4)rFX0sc06=8O%&bymFGDxoZlPAZ{RG^pTk|oV zm#mIMjL;=m0J)1GHmG`G@?$EJVTSj>52fI~e*9>%+uYPljh#(zY-t(46J)xAh}DN! zt5RCsdZw%d3`;0+v6*ir9m}_ixeml|fNqxPW;rRT=?$5p66YU_ck2rbTx)mc-cXvYWpLJ;nu1% zWMSdUOed5^m7%0ShhoG8;v;uT$2iBB-Y$56b zO9{cEaS_1rR|j;lH|*da`-vaHsUb}`3k<}XT zzBM3EU@())lb)b$uVyJS9kw<@Y-@4FX}1eUtm&evC;Em-}6Gmxr01moe_tIh(&GwTT#?4jZ>6xp}T zx!FiuN`TDNZ6)GA$0`V34gBR~qw0fbZd9ME(`o>Yx$AZn&V+QUEFzM@+Vb02a%~SJ zgA*zqb)T4p4#+S5U-CakItakC57gZ_hu84+?6LtH*?imy7f%b+9Aswu{3jmr)$01o z+WB}GL#iVn0WI|)me*QobDf?+^E@e-b!akRL!$v|lJP6C1x^D^SB}$tWaBR(1vev^F(sc_dr@ zUMLuxRXdO@h~a`6Ku2AJX_;S^3)X=9;hn1O=xNUqd~&hYtb7-?t<}kcZ5#AVP<#C( zaaN$Z@XG6%MnOglY#~IlyAE?f-VsWx%rndzvLJI8oOu_Td2=Odf4#OK#C}Gf+2i03 zUAG`tV!up>-W+H=WGeP7dfuy3U2g#{ZJ;A zRpB>S&iV;Qdv`iR3d?}4o(R^1&=y+xZmjj(ikk7sr&s?WH09sOeD{X%=sr!^kB1h5 zG1evUAGnqU%9*?i?wJ%$eJ{`#;s$0NIw@{qf{Jx&Yoe4yrg`Sxcc2dT#^8X;MuO!6 z<3E|$Dgd_fU(BA;#juudpk*GGDXYEZKxXZ}h~2_Gdt!hmII1~mWKutTsDo_+_;_ zYgu;BLc%6t{JU|HV9)&D-_tHFE;4ur?>uR^ZKv@o{Tjz%`oi`j?kT!O`W-B4r>FYI z^to=2TXX^tSjFu-o6SkOE`upUMXLdQ#pOc~uXN?Y8@*nZTRj98>@qwog{yPn+%XIL z+&Ae>LiNxru1etxO}+l$XKP5V80=u4sF`=w@l%p6ZiIO?I|~cP^~QP_kI8T43X?m% zz@`uw15Wct>{n(S+lc{5Ja9@bSpGZu~@@dHd0&#i788h zg(OgyOVm>V*~a-0L9FlI2C}H%gF%7H8zZFStU4W+)kRKMKU}Bu6K4bU^$Vrd2e*qO zJ)h|JR7#~hh$b0J3HJ1eEcfT2qKm3kjG zP?4VO`Q287X?UKFL%VI~rAPwHb=Hq47BUrAr!B$Sm;YnFUI5@ z6~t9IgyZ0`-cwWq;;pneXNX!7iiiB3^Yc9FCy=r+I1hK8SL5eJ}U$QLVqV zp3d6a0~z$8y~N$kX+Q5`(WVZ|(d28k*bC-W?0+&8eB{hrh(`hhhR?~)pT6N(D>n>E z7IIAW;~|kD^0Dr$v$yg{LJtHqkfqW*u!u=+Ph9{HeQPBptHUcgLkDHW=iM6njx+1z zG@I@vSg290QueI~FD`hGY%##`AZlB`tL zV~495ZY7MFW2Nnl(*Px~8E{_uOkSoI>zSt)tM`+w``wxE3H`CcQgU)vTvMviB+p(k z*HHfA2M;1Wfbs#}0~!w?%*LO&hnmii6-4~=PFz+7)dTI(pQ}(4{}&|pc%}?j^w($$ z&+4152NTANCdJteEvp*@2n_n0-1d?W`{Sv7uaZr~(zs!|5$`;CBF~1Y9xPw*)-H#0 z6id`{{0Xn@IXEq#*w{L4rFy_D)AhbQEluoW`>b69HsBF7&fq?~NqQBg4W3DlpWH z4@}gUpHVyBprF7yY!PP78Lj2atsoqMfUH&T%=~U3DHyq3{q;DtXlG09{+|2GcYUqd zar&NWd)*R28e*->>D3WRk|^W{}8iIA#Gh`O7r)m8htYl?G4Qhmb3A zsaS;$M6+SJXkHhM;B;}@)!6Py2L_j8+>`X#ZzvagahfC= zMf>mD{EV8P`{4s(Oh+{#g6VlYrqJ<_#5sb)_wlZAZ|t+Ko}Li>UfFA49}>~xY{q|& zG_^YNZx&+|9)bf+J4wpMsl0lmRn|qr_~hxt_;`9|CS3tnLkG!EnTumgE*_p`zyh@0 z!xj1f&}aUA&SaiYw(cLyjG-M;<_aC>!{}x4VHmB79Aa5yl3t!i3@Ya;RBhtn9HEp- zc)Rl=TwEt<@USnjlvQrdnVFS3{Fg%dxZe8u@^X$c0W=XYX~y;Q;{0UKy86p9pHBv7 zTtC7T{TAF;^vHDT&HJw(hoxIb74QxT41+uJ1u6~GlET0a^pM#5BqY)WIPGYV2E#t; zb-GUo>NQQ%vW@Khd_o*Z%=SgUalTegBE;b#~spb|Pj zrcy%N+m|dzd^YEV+tb^-GE(d8SO#ta?u10izdwiq{n>^2Kuf4X`rd`4$|tG&!Ma>p zre8_CzKcuU4|xQ6!Za~4Avyf_rEkRN4>uC?E=n z%Kr9xJXBcb;WR_U4><~`Gh7)WD?^vP+QmE`cdQCwsdkM}!;kCr2s3IBVP_LNuhvp{ z2#LZa?etL`D=NP(RgKKDqX}W75ho5$%mhCbIMu98D{02o(_`XS4NVf`(YQmxT~xC1Hvd*X>@3Qt0M}gxe`IVAbl@P z5Z0c?spWlB%|H_K(EF|^f&^;OSqyW_U`_hCIOofR0o8N3Mv+>9qO5_0ix; zH)E<={!^Sg>DAHs2Di+NjAaYT81ws3;K@EQm2f@#_wHqZU7Wy^mC0g2xD`wjxRl3S z{FEb>G6F*$wV&=S;^G7p^nMQ}c=4`}#E1C=%F?RM!_B?)E7AT3nSCP)F8n$P?J;rk zbI9UdPj@_5$%fdJ>iytV88kS`#q;N_E7qtEeS!A~uPRH*F{=A5O-ac=J*Ml7q${rD zt)+%5-=o+++Y#@2Ef#O^e7~Uy16|<6)5pSkTg*q|<*7Ot9Wl`~RcA)!x@VhqHNW8? z<;dX3Gy%=#j~8f0%N?KydK)GGkSnJ{dP&uiY3ylO)ZCx1P@@84rQ;iaFI=r^DeTdg z$*^l;!sMiwrKpfBytQ`gHJ z+_0~!IG*^f2|hbtdSc@6D@ka~4deX@mB^22V*QZqk{0NxS)aaX+`;9^cBUKWgN4}G zSm};*tI6dE@eHf7E^I`~YqD5^f2|}&DDwn_-<&>`9!bfMO4+TTd&;2=omg^liKAWL z=VW}?nqSr%{EMfzq}>nxjY%IRRlyDP)}*&=%Loi+BTl8lVF91LsJ$u?my|@lhB-Q^ zLY0hK#JjHIy%a#af0TMh{_NSa9&|ph8JRiPlLV5Nv~iA0kHyqVBpvA^SqWDaxt`aV z_~$ZFo=CkPGAyI{%;tRM7!ji*{d{;Pk%I$YfSdfVbuT}|J5cYKuXdCE=_a+-YaN+)go&Uo+LHvD3KIv~H9w(? z^YVC+d_saAtglK-s(MCr8h*Oye)2CB6SN-%^(&ZQs`^T~vYq-%d8Vct=dmW{F@=BB zSq*8WdwDsQ2*(d9W5WJ^k@y`^|D`_VK+u16O{>XUo_d_+*`Br8GrFm zcKJ?_b?%R@*DS@gTs|8R-V?%H(Z;jX^kqROVdFF$%dV>;q7@`|DUm9vHi1q) zddw}eqqBYbe>%G{74WL9igWv<3Hh|q&*jUKwz3);becGUFsku#tMK(zy-?gAz^y5K zkorpV4+!{-Dw_ks);YVDioeu-9bGQn9b7g}g-{pPdR_K4ja9FDFl<_9&FXJE>6a2W zxbC*&`khPI%+)=XmUy7bbiG-;D0t7wJeokFiaPAe^9F)2v;{8ZP#&ePjQt^8s!u~} zB97TmxbC6wV+I$bhx>Ao<0vaiIL5O)m;4YpL+hPea*^I6E(NYg zIXT2?n{iR#$ukN&M{aGX5bcLhan0}LGIS4I%&Er`Bn%^d5KezNei9ZcmeH2*MH%ar z3;{|>-y=u%Zx%R4&Ke#EJvwCj8}jxhnznRnK3pp;1LU7PqBxD@d6)baZCgj3uP-?K zeq3K*eNB-4BEt+t=ObhZl3Z(<)h*Isl@6!$`DzUnB))2w!iKo?@14zCdn<+fg@OOV zTUlR#mr^xSlF3F*6gwyXdX@QR;q8 zHsbC1W%kiQ7^`AnGKD7^5jNVxC-*6Jr;241B^}8~p9MVU|AD95m(j^o#5~H6zQL6Z z^NCtqv7S#U=|F=#ixs^c!92!1jiE()0K90k9uM%_;(3UPq$VO(Vp#h465DM zFM8F{qvcq3jxR{V?ne`vF8`U3d2qU=CsRvuaJ0wP-t%U>YoO{UmifBTcarwl=+3BA z>@&WHY`^aLf(6k`d$f-=vL35(O%*@jar-Gwst<~&6 zCjD#Xea9x<^X&HE6lD_JMvYG~g7fan!Hn;(4?5)+20vF|+!PfR?ewodfhWhs<-Y1g8Wh;n0LcZ7p@OML**4RYVs!$2U>FMbwaVG5t-hh)*YJHR1|Gp|t zU0vOMHRARDkOXeNX3iVRk5r4tQf%?O52bPLWCw(??Qp7Of4+Gog`u!%Vp#EozEuVe ziMfhCf{X}M%zk~SyV!0g<;u*dWqM{ZR^D|6-&gYYAyfkn@Uij^baJk_P0UBRW$7#N zD|`c^OFHH_J^^9v$0lEUGc8T8V@!LWRqHK@oBd8NchakJs?9DC-^i_7DM zrUOe-{;1%z_>#i*p^a7$GA_?`A63)x?Is#Y*s9h}h5+^tT~rt$BsO==5-ZKpjWNaJ zNJ0o~!r^Nc!ZP3&!B~+NRF=3P+Pi77+Ofl7F6*+>ZkVUUJ}8v%bQDg)y3;eCq%EZ) zp7_*4|I=AOT}JIb?4}I?u29 z3PVTzK-Knz-CW%g6Qb;!Q)64kBJuZ$aI2wVHbVz3+bTW<*RemnLsMk#b}jH=JBzOf z-QsyE!fyoo7-BbZk+2U#Ylx03j?gf|$e-Pt(#xW(>v_bW%LBDr*NpoxZf@?M_Iz9I zOUge&pJZ}m-$r9T*Ye(P!#G$My?cxDO{nLMdv^e1L1rBj*%Ly=F`F-|GNC+gF7QSb zrdL(Su`>DN#1dhLIFX6Zhu%9xCk;c&wFLI1?o=GN z*4!F?RRngm;o=*COd*z}m=zREJ+iNJJHt}%CY>?QzJ0m7X}Rycn?Y+zc4X*C)J~G) z%e|nMuGC#FYozpvLYV8RBik=Cd@cG;T7FWWWR#uRxzMZIX2QZ68-%738^Rw`u;(?u z^7IbEc5p{ZRp}cl3}|tn{P+0%%*@SmF=5{l6AS;)RvR_)92pw=fRyW*r$$7492pt0 zC_!k|b^VM;hTVKbXr+;OQQB%L{tseP=Oc07STExzV)SU#y~ZP)lDg%PJW|jn`pga# zi8rc~vT{%l?!&HXcr&eDJ6VM=SCTrSD>fUBqT*L&>}kdxUQiu9b_#fM$G;3fPls?SN3`Qe|r}P0Y=W7fewK2XiF~h1{O1 zlEVL(?;(m$p!^X(Je^0993Elj(Z$f2(%toeFTvj&dHDJty)fQzOd*HkQteXUkT{XJ~-TW0|{{j8mS~4c8G7B_+Ce z7fQa=da2Ff+vv`yUEguuYjEoi%KAn?-79B;v*b0vzy4G7?nqR9N-5T?E)K{j((CKB zyIUI!={I9iQhf1rP1Eu^(oZpnFU+tBU5ZJgss3Sdz#m`nk@z9lEL!$6YP_JE-Lade zD{F-{RL+W(+VDm7q+4CTU0GY`2gFiM=Ca^o+d|7ha@cX~7Wk>qJxoRltuf_@?Plc{ z5Vi`Fi17M)f#(lK&3dAlF=sZc^F=`t)iAlO&d>T}YG1o<-j0i#JEOpgW5V|sMO&u; zsm7AXjQ)zYlTYHr2$*3?vTjQ~zcS1WoEm74ph@OEYk zJF|cq--q&DS@)7=J{syPXIFUn)=lwwCyce+cT|L{>T6}h9E^OX+F0%BuYQ)+917cW zM3`#1w;9H}xUkN4W}>pM_jUPJ^V<}Slp2IhTeE7O*f=n$kp+2YiwMzNZ^xXR&x^)J zs#ap~roQ#}S0L!=A=j#0gf$TFiAglziryZCFCAU%5BlAz?i$>#U6_krJ>e5#3A&C( zb#`_(1j-F4^EuW%1Ol-d&3#6O&&?CfR_eFY76x9J95ds;Z(gj23A|skSe2)oeMEJD zNp3Y@YrH<}9VccFf|GX`UrJy@UJu1(XMaXbRY&2?9<#I*!sE6rYf~^NW4b!4EJ=s-O#JDM%NNZ=f=w#|V=ypstI|jxj;O)ZkFdS`MPj3=%auY;5=P|VyEI?;aFtOkw3^5LszKy4u5r&xU2&&m= z(T0RuGSI5B{6^n6>nQKsdh#O-*@HFoR(Q@${*IBps?oNRP|VkjaDPc8<)z&hIO{Un zUHG=Wf;7h&ZCz?HF%y^+4WV<3o>0sQ2X-DJGC>F(8J1j!KOSKnc*&@zT&`W!ovgTf%J%|hY`pQ&iWWG!h3r3g@` z)$cU|T@dKl+#raHur-)wv@3Z(X*>yu`OedcL@#FzeN%J&Uqlz@p>F)r=WTE`A4=61 zIltHE<#q2lKrhO1nx6}Cj;LxF4jEBN4L`~Bh3k?SUKwN;C=LH5ZNczth-ZB{;$33l zgz2x-G6aG)>gB6}xznHSv!;-0)HMOFw}#A;+ofa4f>WeW^BGXRm7e|LHF6p}phP?a*&c zRoKUs`6zVDUBi+$8|xC_r;Gtc&kCRYFbcfq26iEee|W5)UNf) zVUvl9R>A5YR=&UiJeu@2j=>+#k80{+5;N3lM(oMMT?K1(OGnHNH1NnwL6tBEV~!1V zb@;KOwSEVVh^6=K~hiR0`vK^jW>!h(bU}fCcOY&pN+!5F(i57B-6dcMch$$w! zvL~dmBxqbL4Kra!jWdf zTo7;n)zhDk9=@YUKPc^b@M~&J3v?qQj`R$+28@1c8NHQ@d9q5_Be`&+Gl9X*>_S1wW@ zD{|HV#Y2D$Ib4z*lD@LCGIRR3FV6SZXGFqoW$lSVSZJrIXI`Awp4;(ZXP4*sDK7;n z#ahkz&ZnA9Hw_~Z{?e$(ghtkmDG}j+wHO?H7Qg&Si>DznP!gXc=@VJlYfFD3Z%#c> zrDIp3t%R%_EdKtHHz*!7Y-6p|ed{O3hn|>+ z6-RGD4qBdG-t?E&aL145Ux~C+a5r+})h8qW!i1>2v|5ap6(rR5(Apo{#ah2a&KzjL zv%M%|a!co*8H%Ou=!_YSPBd^jGv;-@jyvM)QC8!D!Hp4n^bon6w*b$zWUCK%p;fVX zf>ALOX$uPrjEq#etiA#3Y85g_af`)JC8W2hC@Jp$?d1M{JN%U|fRqzOk|6A*i`74! zR{Z0)9>7B$WoW~9QpMt}&`UJO@M!u#c=e9|0xyvmGarWn2g3}(*HQtNHUw^su-4dOm@yUI@${xSn=a$IF$(WCO)R-tRF_u%B zQ#C|l#_`N_1yASgZjO)uFLRjaj9PqGPB{CXJngCMvAAn_KyG|`^!_IpS;S}*r4nC! zt0H;RsgoS)>u@Keb?o3=IsfG_Y7!G&MRJ|@_I;Wo2h}aJGWN2$!My2C^f!$OotwGD zTa3uGUCg(XTjNx1b4M4nrF6(vE!q-gYK%$~#=eagkGrQ6ml*;M`a)wV^6ZMtW31>p z2^MFM$yYs+o#%&N1@*qSV`64dBN9Aa4~XR!70K@yXmJe22I1+odDTJs&y}&QU_KQo_BWj*5k6uo0*Z`#I5lJn~hMtv=W zOMGbxy$d;KlIoES0z86<0k7DW+f(`gdb*ALiXU!O(4KA{jdIyf!(Rpyzv&p86@2NC z_Rvc=NY=%dYJ7Ymj*coD!O(BN(j$S`(ye;jj>~q@a<;aX?q5tVJ3oqnyRbPRf6PO$ zV@g*xFeVLQCHb+f!|Q~i!PAX2?EQ%sLJ7fsemc;0YHTk4n*n8zb3zKMYm9qVIEjA`_+J7q^xpOaz=r?3+EFLo%@7bY&qnq2seh z{ShC$9mjU9Xn7L4ziF6N{oX%c|8kerSJ2ktv=QZ!&drCj$g6awX0kjVtvgr08Aa?V z;2Z(p;a&dRz*Z!M+cE+zL|;dz!1SxFk{{VO@ed!^_}_&33>T9EtiyA&7s~Zi&&w$qM!47l`TkA=W z4p4XwbqSi7&@hVO$jQmqmw#(W2lOyenVvcB4_b#;{I}_)*Pai(YzHf_tsy6y_WM8N zNKdDyWzRiHT|cYF1s5F1j7%YnbLY%5J3U?WES;4;2+@s2hzyTHb2yHr)&4W4eW0I9(o0(eV>Y8rOBxHQFLWUBEG{mgj{>)=T zogb1j0&1+3C132E9YVa%R%chPm%Q!H*7yy>C-2E-5Jr7YYJZ zMP@W3ox8r+ZmhQ1MLD@{tdOx#Abi;9N!#bOwvqui!lovZR2fVQfgt7TOWlvyd@dWA zEX<6%n*u4Q2-NF2dE}umQnF~VB)<&927Z%KpANu`D=HtAIa^pu!OWEVY4Lfettr4v zkOy;!JL)eG_u!4;4MM)D$4JpiNYZkn(2<}ryQv%ZE%H%bzPy>3m)Ou4+8aEARHrJZ z=;?e~^V_0{Vi}Wjw#I(^+kaQ3;n!!^?jdwu!#(Xx4Uu)@b=kNac#JW6nRLC$iX1~} zsVh2)pG496y%;xP-)-B7@TTw$a~0ic;4+a^u>?`KFnfElz$ z$J`6vZ2K7I@_o{C7#s_(cEwaJQz(EjjSt8bC1SpP@H<@VFqvVcm4XKwK4a{uBhOeMFyvO66(I%_|>337Z+b?Acl`;?`@Jb9_IGdF3HQ|JQ{BL0V2-uWL$ zddlw*M6})H5OIhl?MomV`LT=DW?6C)ugyc`X$ATm&%y{*u7#^{`_gS2VScR>kYc<*XIWr%N&|3_Au7?8+(?u5obei2q zH1DdbhszBkbd5df?VI;rB!2nAoc#6#FF!)vf9|^NYE?>AHMV&~MaGRypOwJ1DpH-2 zgw?jF$POvInv?5Nq<}>bA5l60Pm79xb#aQxq!Lekoe1AiQ^1uSjv!Wj(89;4bCU~= zu`&`aOx<*?-4~5+p93k&sh{}zz~sp*g@2n?ef>A|D*mfKln`F{UsdW|f*%se1y$+o ztU`aL$G@tSO2?`<-I2}7IbIJVXSMtuN*Ejh`vVG+kBdh!8@8qC-#DHL_=NadP6RJ( zH${rw>53@nVAc z4>|&jbcNyB+EX#I%MXVm^Gmg)$z;|8Q5vN6{77fL$RD=f*RJ2zoO25O40h{)<$u*g zd2rQ?hmj`Y$xY$=S6wMxKwa=utMGga7Y!<}FEu#CuKuK*Ge3TA7+I|r|Fg*^yXe3s z2w=-ka7?A*ta{r%9-cQiqOU9og|Tp#Ix67-wxRl~?m`U;m-J^jXyIBfHF;oO_4%Vb z11E>7U;loAtM)}@1ROFbPafjvf{LR5L#JyLueYB=(dl;r?)vS2deal?*w}m$`l8nl-u_fHzrH?Zm?5p4%gK%!+35y?veSZPa?UteFQ&C>C)3m? z+Ehk_k^u1xYRnB@^zMEPxF}04&1+TorpHM}%A5&Vn9u2S@7{9prW9c#NgV2YOYO`M z+I?;_UeuIbLClMjh~fNOb5&IoTQZX(*w;QVnqt2_vO8yXN^f!KXulxmy2uqSN}7e3 zHS~8_Sa;&%UbL6UjS6xqBSS9xi-b#jJrAiiwHl@0 zwOP8$(0R6H*!du#qfs?*%O3_ez}(|ofmB#Q1OzQ;j%!|5Px>^gYxx39P`Gc`3K@eO7; zRJ)LN+jH^9`^2B}yh=bEone#_5tK9;H+54gD3*$|rD!S-Mu@~HXtV*22Xkg*(k#tbg}x*n&(F1VL!gag>Q+b9s-} z%;_*>VHGF9^Z62Wo9viWXc!Xd%$|#yieylbPw*QpTn_|65>(F5qg)!KQCfu)lqFLr zBA?MGJL_!o2t{P;zmR-hw89_>k_e0v4My<_HC#a;Hp1JiC4b>e{sV~nFDOG+TyeFw zRCcTCtU|55;Zpxq(afzQ+h|_OV&j4$Eq*$Ga1{+;GM%pP#|$EiqbY^svWb7Zk_4|QC=K}GDrJ<`2>hal2AT7XLF(R$ z#qdT$D47QbsdZld(Cm$;qLGrtq+p6-F3PBiw+)OS;SMFo)8LKFq&<~CmU9#Dek+$s zpRriD+vq^9leBJ=Vi-sb_bs3TE4U30XJ-7h^09=JD!H+-L1eP%zPPVtGo_TBfP#fxNSKc%YJvUy^bJO;ce3%%V8-)w(V}s7kF0a8PMn?fe_O`zoW%_jr zug*g_d|=ZfB!J|n1^zx6^G^O+fEpm3WIH#-%TjA|@IPHWf?~>sh7kja@xr{kvG zO*ygK_~GY8W_rSciplL~NRI~{kC7Mw?1sF~NEL$bU?)cru8WH9!?Fz{QPdbUu|sI{ zeZs!jE0oG_ydE`1B$5iH3XmEbkk?ZucOlt`Dv5>{yM7U(X?Z;u7f%+IqlQHOrX-Xa z93mB>GUB&>wNsvzk?|1TP(ftcCKh-E1t|=|37P(ZSBXzmNI%<5VxJ7ALQ{3nx(?>D zc?xt%qN4q0jdXpV1Bx=l1)^MkR2DS1)Aw`L2cB~FP=<=^o4b^NvlV-LI~6YNxl+}V za<+VF`t;k1uk9{k!}Uu3q5be`9GXZ&;rcZfL(uA?z<>fkBFX<1kl&AD$ zvzYzfPgr)2CCl^X<_0EcF=@LSQe(U4~(XU z&rm6u3Pz^zfB~aHgJx{CkEa$F^_QK0zVZP7n@{9O zr#C|hjy<`W@iJMZMOnRjG>-A1qd!zu=os<>3?pSLA>6k=GEQ+_cu9P><sA6t@V3nTnX9G$XA@g z^VM+hl=Dhega`yNiEt*$~q(SB*Lt;8xm^(gByS36Dyp;J4OTDwHNJY+^`Kb4W@xQFP2kzRg43F+moVAHugU9hGm@hp zIJ;SkWn$=3eMD`bXOQ<@{)WJRwUP@fQYfk!96@r~$ziRnB%(R7|1%398!V{6C$cFS zNkEg~H_fWZRJ_#MPVIs&y)(H7Ia_KVTj ze{FAfN;w`&-TpdoM~MMDl;v5Y&9KL^+br>5!FWCzBXf#8yp=xTD{Exh7F|!8ZMtp0 zdkS99CH12UnZe0`>U`P6Uqo@1sGSd89^%}Jc6Y36lzs$i6VBr@Aal1y;+V;G@|+W-MWsp&d3>cYEZWn6;JP;?g?zu z49=COG2@C?9&foAI z91glpj7ww<8nSL*2L53MVVd&cwno1PFHR+&-3)gv(GehmqU(uNAzZHqVNVlcT@%1d zyK(U>&pL9=))fKMRz!2mgSGsjrxHH|>Ktau1z#}Un?Z!Hq5gdcZ%|0UNADYS;@1$y zQ-Igebd6k-Zrf-CWr4Rs8(eV(`@>kVo_+t09>pEAaZS|KWAt3>x2%mE@DE*AI7SS)oGynXv;!%VT#dqOwLzcM|uw&(Yte>@DWfC$SK zDH_fEN|3k#OY~3WJ>t@5-*qfTGhf`F_mLcy{UMtU3Ndt?l8f!;A2SS|n=O-Sro6r^ z6Fd!*YII&Wm|vare}1F0?4yF(Pxv)1L*iKiQgfZC|4|)9`b#!)w@BF1D-Wl;HRg%Q zy*x9bDw_!!pIza#a3TEC(o%f2-Kq{Kvk&?7om3UA%K9h8`?dC0%rf@&?B8opO$=C> zPt42g;=5=mG4L#M05u2nTJjHSP>r1`AGAp+8Y6<+2>^(1D?TF!Uw0ASzXmK zes|JA3H&a$Y8V8BVJGQv6_S*t)Lf!?S)9w?kzk{i0v$j!5vZ7!lTiNwWR9c&b!DA8%kJ z5(EumVwP&)FL{jAB2+|9_~Yx6+T2udm~mx+Pvo|h7AJ3RT|A7oeQ3hE6(8dPbw!E@ zxwGO(Rb?KzJBNuvj+W8&BNYD?oD8Vd(Tkn~Am zzxt*Dx9mD+)MLHLJ(c*Mb7`w8L;9aaVAe8&WVs2ryC81VcP{f#5ZbPo#c3K3`<$@i zMj+8ZU6|mdURi0WW{&!eksE!6V>hY~vn2}Nx?`=d`*CkW`vu$Kn(pftBJi!S>&t(} zbPvvd@cc9d3Mm@SpHo&m?!Vu@;yRvOwSVef#(;}Jh%*p5<@Xw-mggK@J1~B`n+q3N z%YQB7y5e0?LF?VW^T&d}3GJ!cm)6adKc}oWs>bTAnOhV54e<*-Wu)Q0L5{zg-G_#} z)aw+gp7_rAhP*r5UGklp)UV;fsIjBVi{lL=6W1o%Deb5HT5p}${2sUkk%!(9@2~VU zUy!AdC*x?f2FJKShTB>#(2W;rp4ap*IxU7s6pU?UaKZm7||l|e5rw|8N0&7qC%rJpc<)qEu@#F>ckbZPBT zCqHk!7$W3fXE!&T>=4$~TsVn`$XYOIi`CKeANKI`e$Q!5cJd&c;v8QemIZ|-Ex)9& z2rL3o3ExKrWdi4f!$I1t9?&qlsN4~vIP%}2J;K+}ufjh*x10^pUq*n>Cq_C}` zM|w!@4oqL5SW$7#E-Vm-bc*n%^+jn~D|g4Xdx;K5eD5R*`@&8Jd zxE?3AP5aC64eB06kSzN>JGqfDb|!7(_5SM7DT$AOy$@f7XgN_H%C;aY-tqbPtW03U z8c=!=?YD<`K`T=;gWyaQ+?z))h!AP^I>tO zGHqO)jSFJC`?5&X@UU#!K!~?)djRp)4}k>9j2QM>s*LoAcWpr%H;6WeuMl*TKkW5u zo=oR_uOJl-EM3-V?u*F%t3;-&7kU#y%Q^WaK=*+8%uJP~-S24T`m&*etwJdOK7jHt zY6VQE{EH+kbTfkN+huMt2h4s>Y_gUKxay{Qz%JpW$LhTNg2|N2UW2ll7B*5YQJz8` zcsAD_>gJ3|Ggbh=hyoLTM7`Jmw9JJ<%L!fies_WYXeA2!m)kX}rQ)quCxzr$A{=OB zDxf0^)3ja}Bh`S0HB=Ra@j*)ln8%d}pf_tB?(M0UaH0r+Sy5fo*!jICP@o@s&QVTT z=0(U5GtwzDK)CK+dE9e+%YF?XKVDyLmuX+Kd5CMZnToC|XIu~OhW|aWh_XjYOhOXE zc1w!FS+5==ffZZQMXo7e<pachF_YuzanoKH3@zrLyth+Grz1Jf+U4+rTdKP2NyX4zHhIJK?}TI$6CnTY=h}h zg_}zw<2J`i1Mz5zzFHKI!hpGs{d35uPD-)wElS5)GZH39l6ow~Is(;!0;14!0go-H zwC1^+0gv?t68$AH(Fcwd(%#qp&mMOBZXmZJth9jJL7hq)P(&#uEW*5haH<4&o3G|d z{reuZ;=OxhU}Ls5H8q1GBG4{`7efe*1q=(=|0uyi0Y)bc))v!$=c_;M!?OYkaK2D9 z<6eYS|2tpTsK^8s>KKYoSt!|PwW zvgV=^KQ~!xNR&VR>Qn0skF(?5(f$$2B&-W0&L)Cl?3m{&jP`aXIz zuRjw?kXjlhpEm($sGe|5Ij!};NvFY@#dv$}9%=uCo_xLuAHO;$dhQp|eHmBF(Z!mD zvPN#U&(5WoPR|A?^D-eC``w-d87d4!FZP4Tp}^imZ5&yucRpz8SyfR|qCTEH8*~+B zy|6Cc14iQljqJBc=O!%M_WNt5Q-I@3J`TJB-oDVaZOi`Gl>mf%GwC*a^&jf!@oK@m z-rj2ai0lJ5xn2blf95{PV$_|vZq*mAEZlDoNs*GAS-ADKtI~jNU`{^A##PtJA|=}q%e%M>a6t0<$|xB=3Gs5LmzAyP?8K_ z;JX-R*iVLHupsL7xl3#Q{FjvjmBx$G)QUl9*VP72!92KE>3;>In7O;s7hc!*P@o}` z1`a6*GQ9g?#GBmgCbpw~*nR7pF|WQ9Jh z?Ips+6p^S+<$#6ooZ0OVy5r-`?iL7(YuFownrmP&cAAK;w`Mjw3QVV`rx)49_v8ZH zacKP3e^4JyZ+iGO8``&e=zZqxyY)&KHu6m8e*6JdPo$RbJp7+$J-iGomb5WB@lyPM zbgsuY{P>?@(Qp83WLbgPjEs(SVC5-Z=L9^6n03;?<6{zleyi`XJKL+xxVAyFQ65gD zIUO?06ZSk#jPUPq2}~DTbZrw2b+kNKMFyH5uD^H=iB2o|xJh(E4mp$>Iy0nGi+|98RbtIagPr zvyt&p9gc|4Ypa+J!74g21;{`I_FwJOBGZTXemJ)6tXWze$ok@-*(Bjvw6`5B?Aa^7 z>@n%YxzhkcAU3JQYY&%yQ~~40?_CuknNhJa6I!5xZHS?SiU-*+j3CHE-TMCLZ0C%J zE7@g;Tb*b&MH1F>0{?W2-(Yos-5t4rTnzcJyJagzxN|&;`fDOQ8d$2Gkz_fC`S*Ny z1O&jw>davh8^0fz&2D8wqiY1~jO!Ed)+0>&KMu<*SxWRi5lS}t4Q~u><#w7u^)5Y6 zr28If_(%-wrrud_rPc1E1H@m19r|W;bkwnBr34d+8ERMAJ-0Z;U%xOj?Xjz}Ul9@k zNjg0V6&YQ3!X0dv_0b~6(rU|(q^;?^RSnfIFs}zS#7M26fo-Mf6ZQFP#BMJyh0)BUP!c#Pmd%h z^^$)1`iAb|B6{AyBsy%aHF}OuOV)&gs_`L-7}xQ!mF(LB5C^HDZsy@nQi zS2p$?C8fN$0=~dF_AK218H#}D^_}T%4MUzYk0@uWo-eXoW_NfFjGvT^*sk|dEq}jg zvGNMV1>-2Wj@$3-omej?6igtWj34L`+-UbjB1u>w@>-aLH_qf}b-=X-Z)0?D2{>Nq zfNfn+^3E&GXhy;3X4(IG=^<-ff9FsWr#}1%B)|_+{ObsfN5UE7BA~58pEkLubHqP^ z#phB-#ERtPfiza|HD!++04Qfh2f=WW7-iZ?>F#xdSX6Z8LzB2sU z*qF+6yru(qc?^xw510<;TiiJSWxP%+{;M|CaVQYY@a0E@?^C0A{d`fOI+XS%t*}cZ z`lzQevys%PQ)XSdyGlLzeVXg1LeHp7f8{#nN(B166VwTe#7f~7RHBC!fN*kS@J?Ds z$o;{#jaZ3D{|IIAdkVYrzL%=w%fReKW$vU0lZ#^e$qAyam9AmCk7JWYUwB(9_j>Z6 ztHn80v8c7FZm4YxBj3wZ;Suzh<~txJ8FBv@U9!jk$MWYyxngMO2%_q{AF4V_Y*E`Zo-yqGm`0xX`al)J1+9BzK+Y`H@AWzZ^ z5jHe3sG1WtXPqrZj#CAGp$M^P>?>ZVa`unk5*(n})qUFlSW0sDbntp%Wj+NFBGEdx z(%W)k#`dSjJLqIP{OJ#NcJRA8+a@l=^_I^V)Ahat8L+2>s(ZSj?R}N4=%->X^*T); z|8AByg^9Jq zXh)jvvmN%3OL(@GP?%U2l)8yG3# zTWwpL?^5)XJn`;Q;d@+q5)gwL63HU-d6E7_>Vyt*?=vg?PGKL{B6e(Co3t3LQvdu_ zN4y?m!k?cy?nbw}%&8bftQ3=;;>zQFgBM4xF6r3Lf06F`?H1i`c=nM z+pzV|jw|rcRh_`yVU2dmqs0f^RYF7@ZLosj&D`7-dFuaF;EA#b5$9TH1*SM^+!j<{ zw}Hyqyv&%(JgA5SfU?vNleBcqhmtR%M7Gt6D#^eMYt&v{+$Rv43d>=4M5ER($ZBV6 zo5&O2li>$qP&PI;YFSJvL{q9lO;7y<)?0RwFE+oO-5~6tL_J?R$mT4v6Mf$obmBns zY*xAWnbba^Y2?tSNENcMN)9sk^BC&d*gwRnc&3*)hJUY_PN*7w=BWQU;v}4BEMp9j zrk~3lm2Dy%>XSF*)R^y=%Tez^IGJzzT{vDuy|24ty#3RB*lM`?x+S`_^JUP0OG}7h zq^4t6na10NyX_a>L?Tb~d)C>5@{3sE2wc6jYWSWW&Lcf64!#uOPQ9yHcb<|pXWenU zze7&4IXdrt&q=d(+>0th>!{!)C_CxdR*0;sW8b;hPqLk$d3Jw3Bv5X{8-dEn&3h5A z5-hi8G~6_wX2~tTRetym;GtP_?IUA|M`-6*3kcRBtzGx?$EBr_@@NW1Yx~3AfT>OR zZGD38^n~45YzNFufRslA-wVz%>2m;Sfnie~6AEejgA)tdl);zqKw%&@y+9^NW5l5Efn&$eQW4k)miWxjls0oqff`t~ih(C+ZKUB7$^>TPw-_z8MmeSeVq9Pd_any74S zmShq)E3KSf%v1oS2{)m$tR{s7(R2InG+K6rrv*s(^@&oU%|40F@t5$7_r+X27>ZwU z-|fBF;oTlDA{$PmCi^3^WB5(33CnwEL;pkrOLnYn&|fo|0D>&qxaZsYrbVua;Z3r$ zb@G;Xp=$3ZvVF1oVAL2S6q{y6Gx{nUPB{qcIqili3JWUZ>eVLtyuJahFN+utxwR5y zIHvY!QJI}wXkTctXJ6ynA9zYbgPMZ4q(1B-ZSLr+3<852svSaR1Xh;#R8#NGW>Lp- z=E?r%gpJ!aEx-x~yv~%A>x}&PdF%Y7g^xKoFNrL1H{BV_zG;U*DO2Q4SLJW9b*G!$ejfN={ed~vRB@Q&_soiFCEdqh?|)QU2c2hQ^Lo6@&L#ZVYUj~bFH zU`I`R^9>F}EZD)gAJ~`zY8hai#)3{G0WzUBgz2B2&ee1&mmA9mtho~xacA_&|5{CB zK;;nY?FGRmheinDl>ix}eDGx$-*mR!e}?GyKH7?bt9o*?%mknR^1t_s>7wrG%~wn| z2jRo{@0O8h{3CYRa|Ipl#D~hvBYktK8Bv~F5gS!xjt>_u2uy?)gkS8-yFQncZ|~>D z6>D)Hx}^)4iByPwKvHm)&oiqj;L>*USo+USY`0T z=KYa$4lS;(meN3LE6(;a)9uMa=dG{42Gx1xr~{g5)`?N-H+XD#lf2?bZsXUh@5*IF zcl6SK{gA0=`g|9kky#0BtefNq)s;2)vVkUGmU?WcxT^qJJlDuiDdtY^=8_WTTR~xh zHw}7o=4&F6_wHe^=4Aj^$r#CzCPEYI^0wg6y|_51UtULKeKGbKdVuTV;-&G{*XRK> z4h*C4l9vg-8B+BNboh9;L=y`-p8rwa+uX(fwL2+VOSBuvq$dw)r-6eo2~fA*nNdYB zY$QhGH6a^0oOX~5IZ~3H8xzD>%xz{5L@`m}i)TJfrxQZM9B4Q?T3YLXNR`&nnU%oy z6rsr90>Q;9V4~PjS?>7QF`4Z8>q0=wESU%8YlvPY~qe#WG{i~ZQ zPiy4N#cEVC`D^X!Y}66>W0%`r$pdp@1!%LiwIN1Jw{recJ6~Ra?NX?H)aAtn*R6w{eQ` z_Zc<}gH5G{mHBuLDh}@{)g+n zfs^csmAqTVF{jP`KQRNjL$ufQuBs1I1LpeUXr-PQN@MRyI@FI8`S-4Vw4AID?KtG* z%LE5aEC1Awn!v!w82lG z-~2oKZH+)uHup&N8AFEW5QWNNfUuW>J<{90kMJ1{=^+MB$;)Mo#D2l)_B1BJ%#5uG_sfm2gsNds#?YG_OJVQ-Jt&cUxIVxg1rWj^M&g928 zHi9H5znz!{UDiXLRXg{lMrK?Q;58^Q2;g<*dHOXe@d}cD4IcOI7g*h2?=e(oKmd!k z-5uwMC(&2h|FgF{@cAG$KaJgtiJ9I$^_4kt*Q{~|n@1S$cN{X2@^5fcHRnUcHab+_ z&lEy7qRKAuHU$~*o%-noFEql{he5^Buo)vo@Q2(_O2;R_l0Ar72Y26F=u2;hblcMH$yVj?_)n4D>G5?ITN?m^j6e|cvw5^ozyaj z>t@J)#{9fiR&{ zs~UV3g$;&1bp-Ey2T`je3l$UW-P9a&kdN_mIVjg_@uF|S#@r@x*-F$@)y{Dh*xm#F zF4SAk#Qw6RnfKpB!uWnrq{V77{IXE^ircg&3+G48x-=Of)>^G7h)qnEswDe^Ogo}7 zgPU|pmH^j-SH{Rw6z${U1fKYjJ*5Z&f;XI~&v+xL!)%!xBU|uILjUH{ZbnS_7PC04 z%!YkYj%#gp-crmDm*E$LEHBpLc4F$4FKd zEJSf83k<5;NDDO)ly4hrsYn2V%F>ob=?`k0-Nf8ZPst+leS~BVpQmBTo_a{4#n~Zl zwW&y$rClvR8Pkx%4Xyyg_-q$8cW~=;Q>yNc!jp~$u z^T*(a{~I)5o8$N$!(I^2HmML@@$#g1U^}%|{kRWZR<*Zv8RzX+##)xP0hqe5?!0y# zgK*ue4+E<)Ivzt!gHvW|g2EaHUHR-2~$WyP>gCFvlM@l7LNNrg*qj zI-n6Mn^!;-t>Mf5v-`sucbcsl0tSXN*`PqI6VPsCrV_E^hTQ=&U=KLP+a2jmn^w2A zJ!GqerkkiMNW)^C4R%aF{ygyuiZH~riuuu>SLr}qWv$a0v6AWI_`2b%dU`@kxaESt zJ4tG*C>NR~GmRXz2ZRmVMUm?rMT6>hCg>qlm|YSvJzNOu?N0k5N|f4?g7|D`>s;c0 znx$+Qisq}0u}}t7X1b8Q(9)u+sY; ziNqoekV#Vwk=l9otpFw@>wWfkVIpb#W>yIoi=Lcg^}qr0u7Q?2@uFp_*>GKt0=41P zk?b(`YnynnZ$U=sXsqAu00XNkg>Af4LyqlW9ORZIS0@}|x>sjCHsF*OF;n*#|HX<% z?JIm5ZPs5NQxgmL$Hu>|t(Rx|lfkBxb))uutTph2Uq(9v1A*1!1Lwr6<4iHdLXjjt zsb_U~ZWnp8Uadwgax0Ws8L>4Z{pL}$nmd}`?1j4RiouWT=~M|SBeNHjnI z2IRO~cQd^c@1{2#f14>hFyXC#%yc9&HPRaZ0!PoUH*XtgdSpUB3DGfkROaMXxTZm8 zP>MsKODj0XrY^^3%H^a3Al_z~Oi9dJ@S5=YntD_E$H$xN9?HkK63ryfa5yJzK*SpE zd>KR1`%j{RZ}snKZ34a+Gxi3hkN!{(duJd;P7^h3B--_OXJgE0`QR?#$`RI`G4Y$M z8fZSUBr3|vB7u}o(OB3W_H6iyp8JX`VM&8u@Q5l=h)X1@n5Q4+zJ3czZwN{-4B0R* ze@|#}9y;g8_pr=pQ(Qu8F|25`!y+-TI#Pwm9F>&=2@W5^Mz$Cjau9bAJ+Bvs{x~kD z#3N7+70XA5FlOZ@OJ?%TA&gJ1hy9eg;VRBL_D4S9`yl6<4XfE7(@S4dXpRs<>WZUP zYq>k=?bo<(yS`5oW}!*Ygzt-h(B{agN9Rt92`>xh$ww*3Q0nyz;`B*N);0cN6BS{j zAg%y|^TrIm304{Sc{h1k=5j;MCBIw7J*p{xycT?v@~>^HfLHzSNEGS*Avm(XD`X{c zP56UoKmLyFa?Cz^beBy)k1+ynK+okd#r;7Z_Sv<;h}zWRejh=cW z*22t1ib`KRehMvp`Pyw=uTCdkWnk!=6_#@Psuq!3A-4)^u&= z=1`}y2UQ>{ZF+Ko>VJ;yX9P|}{KP`X1j2#H>3uKAu9$sSIDk5||PbCs&$f(>J zXA$rwXlf=vxXxPu69I-25M*K9byJ(WAI}Grc3c^enf#)v(hK-vkA3W8Pj8zloK&v^ z7cH}(lFpY$hTgwJ8z~uPg{$@>CspYUT8XPSyk1KseuvkF1+Uv*H4hb9i9R1gNHp}lZHR#!Hiu)6HoqZ*)rl}(6;WfX*c$2-f}M6l$GGOf-oMLEC|&02#- zs8At8Dae8^LI=L{@7?JiWQ4k8(n7LHgQOqh);4TPFi3`O^h}l_kmp+EsP)}h__O0* z_&uKSpr&T}=SKa7zx&i#yr6W0e}sr|;jg{auyLhm+;^F3i(r!lF_SY7cnNP!sM_W9 zh8_(Ga{VUlCFTcej(YbxQ&)?Z`h(>umUR|bf+Rm#i>b107<6evWHAvZj<3w^tkG9X zmK2-t4zy7g48zE4Mk3Tx_c{$)N~`PI)s?rp(K&jdQ&)!)7)l>^2COwr!8{b3cj298 zNVR~_SN}%?O66WATc``;yFM()(mI5L>X8c_yP>W_^Rk~Bx=FwV^KX*`OV9}8PfblZ z?d2zL-X2sU@1A{u@<{_wxM?w^x=Nw{AD54HVw?A7?@kn z4n<8xE@1_BcQAgNotIiZX6ktV;ZzM}X1_pbc;7xs<;h|U;d@De%)z+YnUguFW%eHf&a*`PUk>E(9Nu77{~fImNO zSSenzV%5X1DvY>0nj&eY$X3Gk7%wY6WA&5nDhEFL19{`_$xqXxV(O`SkU8r{g}p4rEGq-i>_U zrK*~`=M)fCp}|-Dd9`@xas)W1b=jF@F&y?4(v}vbXk9~QYQSd-NlZ-qO)dXGj0YGw zbP)f{bs)gUXBxbgP!NU=Dd|RPK%|ilMY>Zuq+7~w z&vn1g_jk{%#airh&O9R3(Oh@g*6@?9@wzS9%Hj|iiWOQ%5!Or*4F$E@^sgkTv?4M^ z8Q%BhX+2!%5$H;2s^o~42XF~AjISQILNJIa_at?{?ZbW=}1 z>>B_AT@xV)nJ+%$6~l9q81_Isu5F`$uv&^=O@zmxfcaPB8bO)G>?peY#hOt_x?5oW zWxz%`rJ%sAlcLcnHhQf)M3nNcxnv~UbmCd?u9&Ulle;;xsV~_9!ih|1IclaF)z84V zfcQ7ICQM!$GZgx*NPSba%=6uF(rd#E0{15!D^rt4Z>!f5zvqU)*X8pdl02c*^+;4z zEP{U?jF-e$?*5^?qD&ndJy2JXz7!%WpUC<6YTo7*g>a<0TP^~(Z@WrujV)6lS07l5 zDlbf0zCvHH72UJgod;ts6|Zb%5)QJK+@SGJo2k8vXTi!KJf%`xExfqk84H(NS;fk` zl4T7gcer>9_uIX1!c-k`9u#cB>#3 z(iKEj^+NzdI@I?8bJMdep7Y?wfmu^aD+18_y~$_M($jXjkoBlUAV}Dit&`iPCFLYl zK4{jr_|00_yVaA|A$#100*U2gBL$`&6Z$}novXe?3ej_bcC`6?w=pb!w;Ba_-H1$e z)IL@a5`P8WzmR=}ui`6qVEGj_NOJ)mbV5mC`kq{01?*hIpG--(6yHvRqNAb8fBiNdG`ky-8D*zhe|=+(NfGiPmd_IMwJ z5w!gWlS>}tWLV6GE=^UU@rBObCB8VVPA*%}0ThHG_Lq`?oj9R}$FPPk>s8n;ZeKs? zP?G$i{?}epq15xKF_D;fZkMY*109G3b8Mmi$kcGnRIpDpK&(nH;eN4?pMh+}e&8o3?Q$#!^GnK}k*iD_%hzWHF9 zt#u%ByocgN1s-0LwB|LqW(~ictaSYX=a@Pg?>ki6{VkNjY@;Djlcxu5mh$6hKl|*k z?PeFTQ_hd(yc=CHkSHS#v-*V%&6=~`aAoU^CQe3#Z%w^qyi6oZjGf<_UY(t=zO!M2Ek>UJ@`?UyQ;(@b-v^j5v=zWW=^|28 zQ`5UK(m(=X4Hrud;D@~9ui1Jt;xGB#gDk6i zDp^x1YVcm&Yqhsns;C>%BWI0U$Q(Z9&h}&Zqx4N+SF%wgfT{9_#&PnmW0y7C(^G{G zR^7v(#cRvW4Y&i(7KgUPlI)0&jzZ6QnT&p z6C8jxZ}d;a-V;eKCEiLQl{9*K-0dzDbF-n2nz9_6`xc!lB2l<#jcc?-x-%khLo$3& z6xw(k*TJPkyS=je#oz1TYZ=Po#jt3z8^%TzbyWgMH!1gpiDeq9kRu*Y?ju=_3-DhX zBdK$i6ZE|S*2g>zSp)k6OBCT%fHN+IH^)45-ild&Lvyb^OKIoS%6B#~x9_hZVcCfM9?2B)5{Q^Ai{2 z^SvP~`@%oky(*L`s{2JzQttA;5*Ru&hFnt--ub!e2F2`lrd6wycxgHI*=({Hz zl~;b$J*_kJj%QF|?i|jxDF@1(iuMC{n;pg$h^2O+l^qD4efR>>(qd&GCw}~z35G*i z4ShUmH&dBl&l^?lE<6X)%B`8q(I%QnI7 z_Z|}8X~0G%EYQUI>LD9EkVTsJj3Sz)UXG1Ypu zG@Xmnq~>8N{m@~`{!bjn;4tpEskZFe1mxV5!ksRLWZHe1wy!QDMgU^(h5qe`M1!r(Kl9OQmxAi%X#s zl)@zxGiRkHo-+Z5*%y!9x7U?$*veVI%EPXXE_^xP!V!N>q)uR(&ItqyyJD+-YOtW1wk#4_o`!^YL0$gkOi? zII*$Z8lUEOP)Cd9?p{KrllWe8hU*fEm(l0E7*VD%nK!^Mf==$c{=AMe+Fz|br~dqT zl=No)K9veUgC*@S!W+-XR@js{E#<>SHi=@BVMbFLeB-nxj`&j3 zm(AC+-yXEU;Ew>z%=&l{%G%h(l}=R1(4*4ARna4Q)>*P=JdRhGHJ)V_5gYP{ozy~W zKH6GqzBph*t`1<-R~F{#TUfBcN=^QE**{&dQbYG&sd@E2A5E8_6(+B*$9DlN?RCO5 z$k9R;LS`{uy@90>XFidyL-_xtk;%aB@f-W?=coKX7YBc^IbiW~7R-bM<}WO+NlKCy zaAXJMxmlP>M(A)W@ji$U~B_Zzv9N7ZzOnfh<#NqFz7liGAx|YK;D7Z67 z5=5xcV^E<=J&+prfw~jFh4#-qblmDybc4PnbG*ju^liVh8d?Op1`Y|Tf34@9GJN;y@Xpuszx$vin(3dE3M;aT5fmj7FJiHr&b0zqmMfED zlFuG6L7UOpSKxQk<>Q%^GxOm|xQ%DlCogOEH$K;WOu|ydItqcj=M#fo$u1thxoNf} z+~{5l2o}cfH@`=SHzqt+Vqz-MMwS7U+FG)k7SOG7ijbk>>maGdZZ{2QjFzOO3A-Be zMq{_o(5=yuH2B}&U@dw#=6SDZwAvZOwP#gQ8rDU_&M1-cWhvOT1{IQpvFbWMKEiVsU@a;Dkw+-;LD*)(AW3d zfByW@i{A#|JrRI!2C~s$?(qih5g)x0Ksk;*Ukb_1`BXTstkXOY*tiK~; zlt{R$xMwTBe2?3}Je60=Mh-$tw= zLJBP}!&CWd5EW_`@jYqC{ikEupnq8nJ;9zcbRy^Kw`Z(6y?)_mSuu znvQm?S!_hamG} z(?*z_HFZuk%f9^HroFRepeinZ4dZh8Wc1u0d^T-uw^OB?aV!&{BO3a2WTg8hFh#N$ zE=lYXt*-M@c`hR-hqT7H$n-C%GDrUkvTDU4S(uzWGnl<6O`0L_ol2Ctlz3q% zEVS-_6yR7u>#d<~uc|lo)3HBbVPSRsfaxqrXRKpIJ$-dL~nV`I%lCuzRx!P&l4S0EgPqZM$MzCAwLI{iun=(bEyeK@LW`%CKo zA-jGd6P)NEFB&3=l^@thZk#|h^tp2-+Lm_)CU%gl^(cM_%W3{Pl&wf3P+v7D?C zQYUTh!YATr<+U@S78BJ>X-nn6O#w0ec#m+t@P_8#CTdSkcYdb#-C^_!ZW&uVv|^9* zR_nKMQf`vP)YKe?@mGp9?VuSV>2!}u3}Vl2WU#W!h;KX9Ta$W+nx5r7y8F{7Or9TU zoU6JKgN#G@)*M>x=4sjU+7un8x7>zbN(k@I5~w!aO|eS!`n%wa zIcd}fy@Ocu@+s%7yc;Is$4cp@l+JgoLgWM^(jnVlaWst%Lz_mNB4zoMtenN5Jfp3$ zf}iqiN*#DtIq{vn!*dKXr=1h$YOg&d%$B^->T8yw>>JSmnlCaXpiI(g!U@zNM8F|M z$7^GG5eXIE0poFNbi8kFfulE-K}FZOx|O&&O5N^s=LB$=Wo9dB`c^S2xcCnQ=Ko!6 zze2?MgW>CXbZTHNA#p2gnsMLf#0{#0<(oKIbt1Yrjt&0C&dDhs6x7jv0yz4X98C{5 z0iMRNRwtVUC}r7)6%(iCuyqYT_;}+lrL$ks-IeI#_00C6Ijl6*zjz=+ws_O*W@Gut zA<@QK^9Z29=0zgb2lN;~DI+K&rJV>M%l!$F%rgJTcv<8RXu9H56idlf39xX#_2$R7 zQ{!YFxcjskPM%3d_L|BjVbHWDMTm0S(<_}{Ybf}~#Zr}PDIO>5)=*XD%{QyTZzuj< z{fs7J7YOX)|1cJ59WYqs``*XZRAl8_%d8WNT*#M|q@@&FMoKsJN6tn%XMw+H8NE}$ zaUt~icpuf!>a0kSjhXhVW=pxFWDO}Q{>=HN*Ly!K ziQCOWJ>tZO%PvxKg9M>Ko7P&CWhz)Hctw^MRpVLlIWo_iJuu307Vq`T#pIj!+Qs)% z#e*Yd)|!dvGl%|#Rk%sDk4UJ7cibPO8vJ*Jg0G4hh)+lRwB)nvSeFu;O&pC#(ZsBw z)x3;TDzs{RYm_GZdt&iF$3CpM+S=Ih{?Wxl(I(@oEx_Yqd#wo{17+A#j*pOa-CjiE zP8HRjSfo9b^WccX=YNyxx|_`I;@ltBDc|CNzxHvP@%Q7#KM9zJ6yVeY_w(Pq0y!%O zG6IdN;;(_4n$5+P`HzSvzYgragUkbvPuu@+5)cqn8pZRXtf|*X`S>(fYdIcg*xRB~ zl;g!XvIA?5V$B2*2ryU=hQ~I(F(X@;ajFdm25VS|^yD_loqB>G>sGtj~0w;sR* z(ka6W~t+yij;2KohU)hvHMo%E@VnB-s3;yji_L@4#8eSGrc`ECVWqO!SrH9h(Dl7(sP{sO98}NvIXPb+2z4k|x zKZp;%<(6-u-9SBB{!PcPx+Ppi*F>Cjgfm^9h}>(H4w4HCA&cao@HX--sg@vFTP-+^ z8!TXCYhacd&r(8YJ7w0U)8keMskA=c|MZqH>x4X7S3a?Pq&c8c2xR<3(`ld+Cs=)J zE#v#k|7dT7xc{>P^lr@=c7@A)!Vn*@P3t7H8#k(X!^Y*5;41C-BCW}M=XTmxyYiU@gUb8R(Rd3=U-^U>I4!i`viS|~Odo*YiUaz169{mM zFg9jGh_E>0=IS`+MP3LLd^JrZQGgwEcRKi=tH)Zq-e7_9lbyN|<7HEf<@O6j{KnVc zTU*)pne`@Y?AQh5tAnKB@Tw$Qi-^D5rxrK$;2iCGnVZ$Z^+l8ZT5PC4ZRizDuL+T< zFq*#=QuIGyz}fcEo#wgdW$3NGBn$iz! zI9QJjZH)*8r(A=A!{hg_z^~4QS?K5=2G<4bcRUu7xQ1$V-u3+;)~cdvOPDJEZnZX$ zx+lNh|6WdK&uXZ0bW#5srtweT61v|8;dA~H3Bx1mJ^WP`^xfSSxQy68buP*Q&4-V~ zh>>IdyPMPd#NhNh6#5;^Z^X{WYj)%++yTd6%C|YJyVTg+%C|&rwZr56%|FWg0fPdV zMB?}76E~XI9l^Pn=2W7DS6&Ldfpd|h?fjV(i_wrv43F83hz`urKTg(hVW zb;Y=j<^%L?CeBE_^$My9?>_@y2~|(^YA@R8E$)%E=2?v!j*{~rR?3rexP*lK-8-~! z<28#F4*@8i=YWe+<-HDSo`2fq@bBtuBCC^L3NQfW-4@Pk^&ARb<-OxY5~TybwXhS+ z)=K_Vrm4@3Cu^u8j`272$9W3Ro?HWXZnY;c^b54424SQ9AXgEKuxCMri6$U>F8_Tb zRKL{D*mA<1dMP!MIWL07O;cJv=#$EvhN@NhyGjALjj1<2T<;ReZdEI`T-HnxK!?f< z;8))saGj5+$rFq7W4}?rQ%6PXi0T+J(9ZUK)Z`;7VW3m4C??83w@eq9-~7Ol8=*;b z%8`$XI{ZoXBYx3#V_eVmzjWvR_GslDk+u6w5-)eHUVEJ9!f#Q(UGDTdjYHuiJOWb9 z)QNdO2`*|#^`Y$qhy_(;%y%H6^l_o3rFfby*FER>A7eHr3OZv-N(NT1=x^Z$DBoF~ zz1fvG7A}8hJ~n9Iif2o8X4wiFEhw zHd3Z?mAn1ULf($jNz`a^on#0;aeH^#^nN*ug*Z1yo#LC7fN2(ySv$B51c{4F$J8jy zW=HtJ|D>%`_yY6a4%#>W9CwN3KXu^0Pz+B95E`%w?Q7;b0fL#T$un?Zg(IA_@PJb{ zLhv($HQeN|SmIP~VbT!4A)pL8=J9ENTKiI3ilojY>2^>KbltBUVSW1SjOYO>(SLJ& zVOZ&wNDpj1Nb$LQsn7t{bgN$oqe4Q~ESNmm;+!t-iD!X%&4XJx&A$)8#tj>k32mtH zv)GDv+5Y^r2|zPs0K|KVFbDD5mgR)2x`E1gF|n)7pNm@E1sD18e?A8Z*D>cq5*w(< ze%{e8wzls-r?C`Y;8sK zj62}KHO?_{vCyP$rGQqu&P>|n$mJ=fgu$x(EX{VtGW7*Il=I%3ZMjeH;{$7h)r5Ly z&~LbLjfwuewEdp3zV}=5HJIKOwzlYIw&eC%(5fpmwX+-^BDysPw5jI@^#GPnXG?KM ziEs4&XoY)YOz2IJ`5dkw>Be<0#UaP_@rkw;L05k>f37D&+o6J5VgSY z*{U0D;t>_4;=G+A!zvNS8_HQ$u8~k?7TXL~{Wa2ix)&(c?InPqkKgF&m6DcLa+Hi= z@fpx~l#jRg>?DseJu{;xE)8$@woOUw)5Qh>e4IJ3YHnqoNQM$?YR$|4d0bvBU-&O6 z!Ir}jJQ|2)nXryY6}ATOvb-)WE|!^VmjL*{x-Rb4kE8Q2Hz`5`lOkW0;Aa5e_im)E z-2wS4LXrFWTg1jLCXZTWs~6krf4@qzZ}TT2eLCPC~9(K>~a6`EiLBsG%UTArH^%$A=JI z2sb@vquSbtJ5QsAlDA>COP7;yw1w7qrWX>caw#K?exsDQ>)B3|>B{8$jm|Dor@M@z zG=r2ZbB|M3JC{KzCo<4}!cYs-FYSv8iSc)$`*l$k9!oDSow9KC3)slMP_8la9+7LY&6HcL~zs z5*T`g#qglPLFQkZ?iN$Uas?yw?MxcXYSI*_fKFmU6e-CN5rZfzjdWyEps&&N;om`8r} z@22hdJ3#RTeM$y5q9yA>@tb{4S_2-w_iX&uQXM-~?LF73Ny}n^V<%zkh~ssrlO%bxh@3ZifpoXIr2CIZA(s%R3i*n3|@veC1G)Q_Y?f zJKj(4Sx-MNQK~XMro{?Y0L{yVAF>UmEW6w1d~2iTEyxqiFDihA$ygA>FvBH9Y0kuk zGcx|Oux;dChd72+eauFN5x(|PA{zo&f4%or5EY(BTPv{a)0L0TVAFL84%l9)Sq51O*2Ac_Sc)EW(&v`hT^X|}RbJ_9&g^W#4>HjQqQX`Oh#x$lRowT6-k`iT8K0m}qh4MN`mIAoF46 za5W(@JFK$@+wt-Y74=$KjYVC{zzQkm-^UZcw0(}~pws|ZTG8bLl8pn0g{QGVASMIe zV}f8+W*+Z%D4j5;BSbT9@$mQBzfB~+qYHr7vf=+XE%pJ5snOO@E-=9Xv)ZZw;3*zo zul(+>>rgAwC0ZqSj+9(PEVkG=96(X6IARSvkTBtPkm!CB#4=4@bwHn0?JN>m@Un{@ z*)Fe2W*Fa5*NPZ$>O4>Jssx}T%KgMxbi{53W-PbuKJ=zJD?Y>6%yw|gIoiwlrjsHc z;a1=0+(|-S{Cs*EwuyL4WKhWWRboQpHY+(-3q{J@nqi2b)UO^wv z)wZx~KysZVPW<=hj0j&xlIZxqlq&~g3+Z2bibIn%Gc|1~`@x;V5= zOWqMaD1tD8H*dAN9<19dzr;kxhf5E0Yq019z~F?HL=B8^E|p&s$OMbq+6=J_LQymT z6vl9|Jsr1J)L_x&`vZ^H0A;bFBY4QK z_n4eyLF#TL1sh(hXUX0EV)+|191WCDhOo5S#Cb?$)?nUS+)B8+@&y5Fx49MXaozw} zQy3c?&xMl2!leY%ycJ9~4ha!=bad1W!RbGt-_BgriGBDB=meq`OepvjQ^@tlxj)La zSbOg&iX5vVX~L%qe%hWA3`3#1QrVBs`yl%LlA{z~3d6gG37n2Li$=#s29SV$L=yt0 zm}ObnlE`-X8@~eIJH2-4GtSkwtZ{;mTZ4@Drv6q|lYX&h`)s0e{4OrmUF(P$gPG3x z&nsC%&9AL!;DJF!IL4L!TBLV1CHB5K2S)rq8UGqQ(+H46anO8ylo47c|7|kk@#DL0 znQ#Dkt`I0GLe7j$Q>F4A_r+;NVbRpQ-$Ry%jt^TfkyHP)Ly+W8;NpsmJ*Ws%7we{C zs3nFPI^mU{=`@>5n^Sxk#T8qYs}zV3W~(!#JS1D(%KLq&7H;BGv9#D?l_(?ty(d6j z-2}iW9U1Q+bi~N4wEM8_>}%%0y+vfFq>5QyfC9)9AQsQP4|rb7!D2D4DfD!x28hL( z+n;8@n85XAv5G=F)!(MT&wu=$Q9#JdKRkm8s(WM+IZ4MQss?F>m41$l^n4cf8gJ&@KYrld1N2sp z*SpA6GC5DWG<0Y?L&r z5UZ&ji2F*Uu_4jq8X}0SHLv+q!0KkXXU*!9!D_(Y79L^0&PaEyzF|V+jlTKRw^?~U zh?g~Kb3Pe>unV%?yeL6mSs`hW)fjZdTVp6sIZvu-n;1-h11e9`f-9DNkKwKBt;$T) zYw$I#nS`Whc@-q<6(p4)Fy|UE144wbu%hVoFg%jk%Q@^m`n#-%3eD=irJk?)z1)nv zO{lx7;a4sjjKMA{ETY?TKSssk{bGntVCq{D_dZY3Rb*%yJxhg}NlN=d%M6-3sXiyG zYmn3iBD5fE79}IWSz4lZUkkz7!_=%F8fN|EZN`W0f|Q}H;Nj;* zVJ7|+^&h__iXHT{98T7uSV@IByoC+h>P?Bxa8TlNv3y~fNqu(s72Mgg`V9?vX&RJ; zI4XlAo22!q`incDPhh>;>QQ}L+fbiG&MO{=%_&i0)^T=j;@;k zN*aIwa^`Xf0N1nvnAA+7x^4@=CgdwZMH#T9-E#2JE+_>?TG$gHP^DymrLR?R1zl0k z)&*dikvXLeIx(+e&=C5%IX2G3y%8Ikq&t*-p$wPncy3yq^>#-tSV{SC|s zmyK{j!0bRrFDqL-T;?T*tWsj(zK$vOK8D)SIPtPJFrxyP1-V0+W9rFEWniy;6qYmd znvABDzs*3~lYkV85kV{nU{{yqCC;NrKkH)hTG1I#N*O|BF3g6D|Qq;^7wd& z-s_=7qa!ZHSmTsoWicl|ro1!+fUKd_*I9@2M>AR^nZQ+n`d}~zG3H{r`j?RRSFKS7 z>oLD}VJbQgk9B4-KMS_oZ~PV}tWGNG)-I|D!@ljB$VlCTXtThu{A4jiyVH1E{y|DR z$DZznDU)h3S|vKES9N|)ObcJ(9U=vUN{i=&;qKRPD55gAY@ijtx90g*JB09d;-T) zT=R_at+j#ddw@e&eX$PcKpy>QbO9*i_jC23$n^_9?N#v0<3C6I*#W78C3=hHmdY^MKp5LNrXb>CL@emsSgH#Qx zTZ+-OZ#<*mou zuj$M@BnZMgRbSU+Rju)Q_BA93u6*UU$EA9G`8aoj_Mg4b0Sn@HRO=QL66=o}{_lpB zk-#wxD#EBiq8DXmic|wd2Zf*|N)h8>_2JIqD{e9I-0iQla*m9*aB?0@7>W3;#6cWW z)x35A8%>Jlx?OUvDVFH=X#vy+D%zZbl!c1-^ixotB7e?aSHE{?Xx}_8w5etEMC3Y{ z=cD`e{BNHHgmhO5^Bu^Fwjx&%R(X9%hf2%YGWW;`A1@HL)frB2^0zR#(*)>O(DVmw z@9qAE?ks(7h455zMjH642pmFZb)SB9t9gqg_J972D;}iYiE*)CS6re;@x7_)b!eYl z$66vVD{KQtV+CpHh)8E9iJ;$Z;_ko0t1)fjcVx9HQ2@AbOm%N@QI?q#GD;?F_geS& z_-x9)VR3`c{rKfAI{T;e2AiJE{-dUWjepM@RQQ{iW+8wIY%icrSeJ;?vISVJfB$`8 z-2oUe72@f`rss?8VeHti5bKhKdKh@ZpBYG4V!HEp|9=ka`K=Axzm3299w|qMG8Yap z-vJX4v#2@t#g;fyzae?6r$-oimAg@SFbL>ZOGV6SNInw)UUDKd^y9<02^|V*_gJf`-^@7_oXmUoHp>F4Bn+afsRt(+ z2zH~@*Aw2EK8HjOLRndxvmXmW_ZOJ?P2-dfGbhFE-vU<<2Se`*CAGl`fg{e`jHeoL z=HujH5_NguPB!+*lo%d)y(7lzAuKE2j0Z?wiu!D`V%`|%ZMH|+9s$BQ=(v6@(g&3! zB+#2G*!w!fu#(t|F$WBpj^;NY#b{EnSZ&ZEcapf|^Zfm7sS1DyEs{ZBsM@s+tyUdP0xhI=+=Wv0Pw7 zH||7d3+dyRL_7pP+L-Y3p{X>U0t+oOV?zV22g3=GxQWw^$Tg0eAHDZC9{asNbFz&| z9UD>;2T4f!saHM+GsZJc%+Hq_J$EolQr+e;pWX4VY6ABMCUH?Me`xm6voGSC zJ@l*!gj_l&5=&RGn+8WgGcq!ye-}@c`hJluB89;unBrsvOdtr*M@~Iii*Jo!vu1V?0%-Tlg{%9%7|-koq>T5Y>MM*j3vj%)I2nKcHEv?q%c7xF+43)K}z}MsgAZCQVOoW83(s*f{4}<&DsxhR&q%emZdAfF&umFHPa{RSOCHe-1G>E2j zT9tYSdR2Hzvp)(ot??s~575EP|LczRTAzx+E8ePfCMt|MCyvCh|2NI50Ik|CFK(Rv z`g$ZsLNoEMdjH>ePpnDgHfRwk+v_tO+$x}bDvwjtyW17k4!SIRD?0Zu{X`q@9k8Qq zz9@UaJn_X=ZJ6k_>+r(^EB}-LE41>`S$Rz^KpoYZzWQlPbxSkhQ%=q@u%h4u>PI7J zxLC_tD@yJyPuD!gD^6JVZgRD>Q6fyyaQ}2Q@kLSju%mx7yi+m*ban+pVu%5HMb%E6 zp(IKso)yy%e>wrr;LUksx092t<^_`|u}~!*t(Bi#-{Uv_z1$sLNdv2`m*+=~sxSZV zKmc901{fN#~H^H=Of=&wJR{gp8zcU+7r=<>kc7B_8ui8S^I zxWH<_X_W&@LZ{3%{E1}MRD{ipo<+-6#0{QtwwoY?{_IYyy;K{fiO9l!*0M+RLTfA- z?9=QOIq=mI|MsqE2YQU1J0g$?Kktyif8aHP&$X0~_;#P;-dcpz!Md>NzOByxb&i?< z{kokXfp~_Xub)_oADf5%K zI8lah0lu7B8Pu{KIi^1{%M^^=8d}Lba`Dj!By$$l=Ugb*ZEC%8ufrEUCTJu=C`uHo z5K=-mtqK-A>smnps?LXzlH@XmZ|Tg;T1bpvXVAr7G;E7!-^cLi5O9BOI@{6~>qcij zrY}q%UYBoRa2;>F)qf&d`vCv)c~#VhlDFnfqG0x8`xNN)R+TuJJ4w-&u7?vZe^z$VJOk?wLkOy8+HmK^{t0x7kqy~@SUED zKmwK3LteQ*mT_nm=q|d{abKb`KFe)vb>B@65qQaa#!23uH8JdoA2Y}BcwNZ z7bE!uFZr~I9mHjBWK{hUHq)_5&t(>{OSlT?GIs?rOzTSU9m0-dz?Bt-fqXriJ@Y*O zf9X}~-+yyOv*e1!#Xm$ixPSllL{r57&$-|@57mNvxA~j7!(*L(m%-0Q=bztJJ9>3z z5YUrx0>gEwkrdH3u{X<8cJ}txaS)V#mnWMj%BWGNYpU`sw%)OhHgb#}87X}^k(&-1 zZ^^l#b*77QIf>)5;cx0_;@h|zQ^pYwoKPYyMdpFrCbuk$aB~;lK0ZDc)fQ=@ZMqZp zT5z|cAi$p=d5e_1WB;FGU~v+@L(Kj&K%V<8`28;yqiR<8iraSz;GS3zIKF*BOUHm( zq27i@q-%~{GXBb-CnBO3-UjSt!3$1KN4iIFJw98G{TF%xB0k8P80(y zLESFqg7&^k=wJ~tZ1AkU+4i^d3?SS%-@grYb~A+#ZG<30JYD52A7pLzNs^77i@jz> z=XlYYE5ykIZ~FyP9!0qqHLOn+{HfndjX=!YP;-Kg8fqN~eW_O8oB0tv95$Dh+R)Eb ztISu;RSMG1rAF&IkxthfjzITmhCd~#=5)xt`yeq2Cqk-(s9BKB9pannpc!b2mzs*o z!!5PI(EaRvB5frf1t+fJMe;u1Y+=fS)^0DJ|%@Trq3-F$} z_Rde?T6aeyYd1O%ev9RrEkcF-i69H`NX!WeF&A{qd(A$^A)UCSY%TVUY*Ay(_ttwW z?9RF8Q{5Z-!&Hw!!0hCcv1{+nXj;(;h6ad8P-8Ih38B(_-)wlDonBeC>=1G*WWjp~9 zkpdN!!@GYk&wv%Iz)%tu_RBwIKn_MhL^q!ZK}HIV9w|c5*L#!0sCjp#pl;IFmJR8B zI1Uc%BeaNqJo}@|&Cxrf;`)%oG41qyqmm@s{d+wBi$NlwyQx+;0)xXbxjGN5O{rsd zHa`x~s!%gTD&jyjcQSTq>)DA6nJ?$h?n>(GxkQ*jo}G+aeODnsi**iIXcRXz;0Ug| zBf0{E~t@QW(l>EYu?i)Whq{3RM z3UY3cKMOvSGEhD~M!EYNZDO`fE$T%vtv@EDIxpm1xQwxhM!he=b~?(mzL{6cD!3vf zu+=SW?Tb&JwZ=Dzv7W%=Qni{PJ9t}PPa#nI2m1Uw$o{k@W@hDsg;3>s`2vYOCkPx5 z+9}^JVXuhXGmVIxwOUMgvpDt!oWQtMRI-igFurpGUZ46wRT~k}koxf{;NtNxa2*ak zPkQl0K2_429=%a2JpT@P)~QHy=N|sy^F^v^YDyAc3V2LWkVgC$T3SjvL##q*!iQpC zaZp^aY3SIc)~LJy2$T|IrbC;$yOW$wUi!?v_6G}<^nAfjzbJ9XD(h6QiUVt-+U$K5 zNn67KmiH5sBUtVeBVi%A|8se!Gmu4Y~y8tVUrDpP#; z$3gt)+Xg0eA?Qfhhuo3aL0H-S{&G{FQIjR)K?`6*>hS!9tK`7^e}mbx-FbLjkm+Z= z53YPts!7AwKBMRIs4*1FJcoY0p50!kU&m;fh_#)mACPP6mT4T9cj&@tU#mSw$rGAv zBh#pR5hz3Kzh8~abzcVTLKrVv8&4UW*P6+$Jl&)`)PYSE&>_Yp`{l!mv?HWi>s0;@L+xO}xsBVXU8NPX2h($+Xs zO|`s+P~|Cer~bCi9@$Dvi%~v;YD3T@K{3H0p*5+o^ki@8NemeqpZIdG(|Nc3R1Gbdj_Q)&TCZM;GI--A*Hl8bsVS-KT-EAxgrKTE5wT{Wn|j}{kwSI312>Ub z*PE(QuU#P4F5OPWZpqR$`X}b|wj$S6Tb{J!%QrtVGv}HQNyMK+ZkO&alEP|U5c1Ok zt!;&kthCbNI&G^)JX6AWMjunbVh>EzTC(6{u35<=uJhroNBtTwy!yqtCwrd-d#Bs} zp8oL{@oR$%g@2_#zsOkFAjR8>GpdeD50)OMbV(-(^jAc&a-1g)d|I)ZULd6sn?Q2;m?m88E)Ue@KvMVRuPPoF$HwXY zHcJbCTq^LY#f2uJ_!_k@8OkhXLT4hM1#eIo5I1W768UQX9sC<0Xkj8P9QWhbq*mma zB7_n5?X&KuPs+K0iuCP>UGDza(*nL)*)xHyUFsr!AJus(_pEr9k_!wYi?2PH6}d{F zODsk9(MlQ&j4V~d4zhHyr1?q8h>f3hVkJKX;1&^O_|tdLGAIYA)L$@JX3grzX*itP zkj{|{M-VGfdH&#`rKUqSROZqa)&`|WD2qM>e~!nJODy7{Pm|>*O;zGewg1|nSll7C zmBK(L+H6BfS5my~Ur=9^5UUdx${Mn~c|7<<)K`1y6NP)L$x&b%h^(%va>!7QP?4sp zQin+x?_w&%jP1)q69?PWjz@7}4Z*()s(;d2dzcv(WNcM~pa(ga3Sdtpw4y?x3|nZ5 z+iP#ko@I8^DrL7=|8;X!6?$!Lt@l5r^HlfGl40G;{~9_LCMTtEFBItuCI-*QMlZEC z1XI0!@&HLg7VY}x0T)Al`lL@5jT{@{mjB_5@*#@JP(CS~5!6fc4YK8aK9|9V0I>6(P&lwr&;Pi4wJ zBMCr990t6dI7itj9Ga!C{a4Npeo$Gz+a?{$c4w?J@^8UQT3wk0{^MzhSc#_oR&!iR zNFr5k+wac*{@u`Tyj&CiM-Od9#l&=#vOPrF$?yyte2c%6pU95hreKwyD{a*xo3dk>B5Li>d%r!6aHiqU1LPa#N(?^oj3_ho}8H*G2$m2 z{s}g7c1K<_+rC~3iaW`=PpXIXCc1CXvl$^d{WQxLEyDTdkIY^f0B#p_zP zFjhkZLxv8xt_`CTK9oVBxBxyXPW{6prH2FGX}wVAaPcQ_JR#(>iSD81!?{DF-io{i zt0=61{4Whu;!O?|zxfC`z%Ik6q=;bu-zPX5NV4ohch4zP=>L5!9Ot6~0xd}6)Km$m zEM4do8U>-AEEE-AJj|s)kg%ucw{YkKQ77Wc%DRNV`u9QWiG9}?coAKS%{E`~idU^u zkPB#*KGvku-jHJt?w<8Jb-ctIypf1TN9MaI;|q;+Hcm;cKa~=U9roq-J!6F4OVYQt zb9X#>n{-@Lg0URHU3bVe8&FUMC=5$)n~A2Qr;?tOiB9-}xR5e&6@O~ep|^Xyc9zPW z&!~g!H)i|cO(CUxTX}m+tEado0YB~LO zIvjm3kzw>sap2g5LtpakA((;6nx0OD0&`&g(n{ zwklFJ@z)u$w&FJ^4hQry&v0_exJ{~Y&CwAA&PX>c*>$YiP916C*TmGzpmW1fJ`bYfv-6Q9V@nLt;U zm)w96Y03n5VxkRnRtq5TvA3&rp+;CpQ4vE8pL9{)7uvAcUaB!rQdkjDub#2-z zIea^wLmcF4Cul1O8K9L~skQ$%Jq*xNTk$|oGvohzjfJ>4(8`qL|N9i4hFcqWjS9*~ z_vD%GRy?^6#bbG*clTp2^aUSOieMyG|2?;`^7loQY%K!RGiKrSdCvvR6Zu8-Rt5F= z`qrea1C_a{vAqoi1KEk%kpbL)T$HyyphnKBKoPOKLQy;61;-0~C_bq_b>?8*N$(6; z=}fpjh3s-o9y-f==reHsf0VstSk(X4J#2!Ag3_G>2n^lbHS{oacXy{E(jYJ}lr#(} zB1lU~hk(FP0@B?L5xZ$!%N56*|iHU)fR-efnGF%xSlcZy*OO)tQa9&o)oKRxO2JJ@- zylaf~WkHA-)>ROEI3LfA_Ira?+`81(M&$s?W76O{Y&a@OO+&*J75Wj&?gY$`0lW6bQUJKlQ(pbNyvkW@fTgJ~4YmVW{7oQ}9Uy`Dz4nmbrGc?e zl>lgV*&nZ1zTiLjpTg)O48SH~3Ou=>_wQarJ8fz6wILa80m(w=_2sdgXa~O_yld&s z$kSQ@Oi=XUU?Q1biODE}4SX5Al|b~wAHlwo1swXwls0C&hNUOb8I=lP@@k~RV`l%b zP3`wE&rp8bN>5>Lc3@=HKVZSpp&!p5Q5dZ(DGKXcdYxOdeveb*2=GWJRWo#Bs^xBp zB%J(1$;3%OMK>7Hc=mkb)C2lkO-^zQYww&NFo1-7p)a%7i(yQ<d!=a?uVipS z`HO+0$1EW`8uY|gk&~^l!pOR>w@+L&S(*Z~i8_+-<=ceYaD@emm}@=hwdY6X>0F+# z3#>1ReuG^apg-N*;;?du0dwOwCdTFzSZo=*-X~nz3|YzddBNC|HTLW!yM0 z2}BU#tC+D%?OfPDE<`}0B8x91xvf=P^=ut@A^Cds=xZ;#>$^t6=Aji`dd_SrVmBVZ zVxd)BS9khl9A%d+#WAwZdH(&{n$@XjmRO2p>~6)Y$S!kPCg=Z| z|7mTqcc3?dy-(ubEm8a#5OfZ9m3F6g0LOOK+l=HTzFE64dh37wI79-W6L6ZXYGRf} zzVG1tEyk5_p*@Z7o0U@0Z5W6^WH*V38`?1P%%@noglz*_HDm|T%u`KqPD+YEi~g z$rDzcSP(S_fA89$nHtZBOQ(yw`BT^KSvr3~2#N^Sr}1;Slcu82<9)A|oc^9Wk;0>@ zq9vX*BxUF++-5HUMH8EDk(n$BGvm-BK+Jc*+2P_bBCKx0&4nPdm74?)ixnS&axgB! zqrkqzC=yJ%X#kOwH6Ek*@_8qmc_H_xqQ<4(pOKgZhBR)7vse-QNQ18VmP`D+e0V+A ze>pD;5eT6Vlb;wF09YpZA!0Q+JnRc*kH+R06i;s#@@wRttS2!QK_E!VTfMcDyqVml zASklM<(?Kr1ZPaeMlA>~)nF?=&O(g+~m(E4U_)fcU<@2PdO>^q0SBiUa22 z^F{teY#bG^TUCUAGu_Yj8+?C%e^=n5so8w(X)2#6HtE*a66^&ZgSwR-c?QPb8$5q` zaMcgU7ppR(sH``Mh3StHf5mv%XF;&-+`SXU1KiK&D{bfX>pd0+Q`1{vbl4A3A+E0{ zv6^bP|9q}oV}YPT6Qq~o2vz1QfQzzCU zh?vD36*pfnujgi_Eb^cwm9F`GU@p)r$wq^N@FitpAW?V@bi9|*>Iw40jElY6vAvRe zMLrwC8E#~Ll#2bP6}C7~MS9t|RacuHEJvP_zL|T7G)Gr26Z>Tf2m7BPh3~cKh;>1- z@ObWb9bmg|Zf9DKUS55K(zVC51zv*MN|Ko{7>qBFdr{Xs^SQ9%;eFazzcx(V0GL@y zL88qBD7nB6#Ek;2A2>C+*qJwO z*ENb%RHcIBy4z7Az618>VxJn;NI+Kx{N{R(BRegc8ZW6@{LY5e6GRk|&l|*O&efDS zEOTDRLv)}5FoM_&m{Q$Zn}fW%WSpFUc*)n~tY#hEbJc1A(gWmJP$@hPjHBLapT0QVJMh zNI@86JE_HbDvkws(gZ-X{FBA4h#=RAO^k;s*)n>*c_-dQN1IuGO z!5%ht9R5n*Ac1&_4kbDn`fq&&WQelUbKm7<5xFvss~g)qyHC<>|H(_Z#*tj*3Uyh3 zv84J9_4Q>~;p_a#)!gAsTsFtN6C^Y7q;f{^msgHyGH}2JPy*u2Dy9<`e&!(FI{i*e z2R`hQl9GJUt~jALfEu)8)52DD9v3v{$g5wC$CM(y3N1!!9R69F?@>{Ebl7xs zo>Y6}TfNWmE9GC4ArM!hz&j-Khobk%KS8Dp1@bOVQu!C{zfy4_Q5mSVea(L`s0lp6`Z9(0fV zg&n7+PDrwR)NeDz^qrGdLw9Gze%6R{>HOI@A>BSBd5nim-=V- z>f#mlukd3wer)ZaGn%QZTH5+^T|3q3@HHp4y+7Whzh?LHs){%3zxZL4a=Nv-j}xef zeA;&Ce(k6q>co`$yEpsKoy~ZeXwgrO$VTAauJ@+RWv&F<(lH7BslZ4hK@^sb1>5S6u(~9{yc{$#81dRTprxy4Qc4zJp1_ zhcj#MO?Oi~YxEvlFPA$7Ts{MtgHEBO0!HLc*^N(%@MENyPW599YrX51#p+t3AA$@i zeGVkR`nAt4FE1;@K`Xbw#TsN#%fUh%h(MsG7`PND=3ZS*<4jkX8eeCC_OB>sivwX5 z3+6K{1OtpKRgZs}P*S|!Ze$F`x0H$}T zzOQ(2kG_|#wx88GM+@IA$-yJRH*kdmI_!9Hkgxm$S1QH=NDNxLQ++^E3#f9vDA2eq z--2?OfqOx_$|Jr%{iuCZt%5dK$}DlUFl~h(C7)ZJSq*iE zj#=mPE6QCrns9q0>g=?OO^z>~J9EGMAYouEfSa9b6fHbCg`C)IW2yo}$*+stnI~=UM>2uiZ)c2GhrP z!1xtn{cR*P;<2tJzk5G^67)N*s})UTVbN{uCWt|o9iAhErrsS6_YWBo=NytQyYDf0 z4?~IYcK{Z**SN|4w_W}pwdmw?)B zb&>YDj_UGi57RQq?4dDfD4C#&Mi@B5Y9sWG!pttC*7z$m?46ytdNOLzUdI|SV z$5+0E@`;;b8seb&!1EH(Xz&HDr|HPBnY#XadIEv+Vp0E-%V=fhqw7n5IsHq6o}=ah zL;c!+4C4%c80Hp%DtG4h@8T$Vwg8m8QugUrs(bv&IDv6;TrkZF7V6R|@ zRBJY>@H3S9V$p{)wM(hAYg^GXGqLGm23V%lAN4b9{)VbTJ^XYZxA@*7KNEewXw`H! zdHt1=aIoR!*wd8K(LxntLd#A0AOP{qR>qsVep^-N`gjR>VRvj1OpsHREu^otv^m}i zkou9MfxJqLAHutEStSc=#OZKZx?wsBueM&{7oV!4hFl9+qIsM)i>=da9t~(1-g}6m8-_YlYbirX-)Q`* zSwS^p+?&T#DRQfUqSz_It1OE0L0C-9)7zWvE$^FcXnji*q>_^tgP1}gGr_615!V!= zH5KIH$)FgIrqb=w-~&b+t3!BOpxqzVCY0;F z6azoTLT%*^eU$3w{7Q7(FATZ-BKTP6K_AQJ-!%?-O?`1g?1=KUl4)zh%*iSv4Goe` z0&;T+;!sXu->kak&I7Ph484dsIXTU+6^jE?20}HCB>7G6Lf{|#F_#yW(7${jCai&8}@DUzphkKyv|xjPLhir86vwRD+PE7 zP1R)uvhUQg=7}r? zVRYAhh)h_L!LCP5Zc8AxUAp(T%|+4RBqd=bl7s<-Tb#5CIe@%ZO$F!F`2m z6fZo(isVDW`(>lZL!+f6+ZYRsksF<{k$2(tyx**O7nU$tr8QTjJ-j^4UkB2XVWH~@ zrI$E-LYJ|mOXczBy*()sTb$j22DAh`g+!?Cw6D)hPR9E4wT5W<4L{Dl6Km*hV8SWJ8FoL4y(Yp>dj-bI}Fvfz zw3SGH?FUJq_g}y`T{7`zshwz*3&h<1F*aEOM`nnBS7eh4`pHkCNC7R^O;S?QY$m#r5&1HqSD`BcwvrWjZScn=6&laVFT6W7Q6F4j_FOQN9AeW_p-n?R z=oYUL3C7hZVKAptklu^aF;kSv9aLFWW=Vp?M08ftyu|j#PBS1ygalSQG5>N9xR*E= z){lWG{*?YU@9l%FR^qmA$pi?zOfhfK)0~}ef!Nn1*~1lWkLeJ-HB-O5^+YLCv;$*? z&LuTisuZJL_kmMgfHkQ;0!_K|DEsvE6wsPP4{4n4bAQN>yxZ}YA|zJJlO$tl63PEg zp%@ZC!<3w*CoZ^4!G(uF_66OG6sG=nAOoHNfuww}!QV<&XU3t$ghdz)n3ffP`XoB^ zN@aG_Yxid;pz9)JYd*oDTZXGAXa_$=9RDEPY1dZ$n(-U^%-nL0Ubxluap}NNTaooI zB`7QLCviRnZaxgYQW?nGgg+RkSBscm8L3m>!ef}mwO#eco(eZEF(fUyQ}H!FT8G0v zvI=OPnpu<(ZHDu`t;zyldfdo!hAm(TeJ=8)otSJ2ZY(4R35I}PxT`0LP^=1!#b=9V z{g(KNkY9nVCZyF}Vhb6WQ@H7iX{L54Xtu3r~QbC_7`(*JpSxs&qDp*?wzw=so! zD_r3QlQ;9OtKNRZ^HW|uGSN9^~^)H%z!ySsbDTfu>LrqJ~d%n_*Rg{pU&;ic{TU02IL0g?=N ziMNj`^e)RNsn`v`K@0bMCI8hT|M^gILD4!nPk8b-wOro-C8Di3N`|GT*H*^SGPk!x zCONXwEde)LZ+hV5!~<+le~>?PSXB-c#=rm|-n&HN*zJB!?A*Qpcf?nHYz@&2{r@8=Wa ztlEsi{Lm0eDe#In4 zS(`&|g$vTYz=q|>Pvv1EBnmsR1s|DG_+n2KSFTAhq&2Y+T~y^s+$%=WXb@8p8%69A zC2KaX>W>|Yq}h2#Wh@}PK&Hk1NM*-GqX;huy^7L6yKmBMhV*79n90At8OKv{?|X)D+Lg4W#JJYuR3PPWlgPmJ!sjZW7A zc#z)?hwJt>tBG`Geq2u_y#7|l7zrSfJqyYRMu3t%hy=!vLtNJnmhiu5NafWX`oUvc zk_5X6pcav8=15?xiRz?bh!_&w{)nr%#-EwFUSl`Tbf=%;_c@@df3_9+g-zCT)1#?r z)wtf{TNhD^09FuOTu9Grxq>;nE#q)~X?BYgMiWm*TEVm_XDraAGuA@j><~n`bC(T*jliH72voez|#DvM7QEvEUC2QoNA z5?H|x8)80%g94p@RP^Vc{ooa!xBnigkwp)W{`k?c+Bhv;##V+k#s9*8AUp9C_2IDY z>m+S_*4ymslOmm3hcw9VSP!K`w^>r>rrcYf;GT}-@limJlFKh#S&Wd<%+@#S)(^*l|UQgS-B?!3c-55@EIBs-?3yOds8cehtczAW=b8|ZB3TUMA^Yb4P*D4v*T=>E8 zmAkeYz2jxf92B}rPN@0vYAT)&I|ZOU&tyT_ANs!<$bbl1zW|B{$`9=Kws~r%fmpTW zm|61-MOk1J)8Yh?)N}a_2uzGl2{%bDK~wLI9wZG9SZW&HI#t)(SGxY$cHy5%ur%NzsmsA3+0r)~@{(@ZG8A&sZ7z}=(6yx2 z%o=!6$V1ad?MD28nZ>pj8zCTI{T|NM?#Z!6!{vG5?4q-t!bI zK3zH!YK;Y?^JH4~W?g*(&j*3I;o^&LK_|_hfRTcw1B&QMXu`SgNbS#t|D{umVFmIe zf57+?>s`OP;D2PNFJj|%Av(mk()^zwF(iq5Mv^z!7OHOSs18-Dmg=l9x^Bfkrgjzp zFo_-`?6ylxQO0q#8pyE~^BQ0W>8614F`jkey9K!3X|_r{N<^pmLD;$=C{w~aGlxjnu1G>Po#h5x2XC~wXgmCO z3CeLw%J%ymXVg>MJ&qc40F<5__VFn&zR$mp=PrZnrr{6?p1A6#ypCYFjbNj}u8-ZE@xf)Nv2088 zyUyne1@J=EPgf5~gh*V@PRc5amuS-UIq5LTH%@Ev2WB3{)nWh9xuR+`BtOx^IT}^` z?9i>r|7FV#BAWg_bwR^Y75$3w`9|B&gyz)G`UhHx^&k1w=4!dggR{6O3 z(-wG9fFU#T<+=so(voW;2c1Yb#B+2u47v5*!_oJ48P_JiC1#jsMU z;e5BP79`ek9J0Th;Xb5L4=o-X9Gq$i^vA=(?3n7zo6t|F0~7kLn(MIJNvhA-Wp^#k zZ1h>m{k!=SX#!0}4!_mF;NKtp=bP_-6|9aQ4aUb#(Lq6@?+|#E11dQx>L~!h?XZ`< zX2!-s?tFmmUN0@2`lu%x2D4?Um{t0OZBbubcemz)oN?CTgh;J2W8nc3DB!&Oyw>g0 zG{(?+zdmn9WgUnlYs5oCq0bGj0>FUJmNf4{C9qI$jSMy!#N$ zmTw->#|gYRqkeIe(=qnu_gVSWWF;G5q4%mIH1CYx&u#kUgTM7f76EYn z>RoJv|A?ypVzvMI=7TvP3k8ufYduK38U!-0w#wl8GqgC}0JR6~UY?>mw!oi{+ivtPl?Njng zQx)@m<&C~AZ$9-2JU5@3tIq1@PDkW?5L$Ey@LQXVm)aMB!&Y!g0l$?%>fPaKA}?q z6}nDX=*uawr;%!-=#oPza-Y)3wzT6zIr@vLbe9fgzNNJj54zwjq3&P|I0T{#o7fFaxxpM4qub)fpNZ zUThoxiORlB@}q$+x!H}78TfBG0t;qcj9L{yyE=a;T$r2@#uys?ilvz0Qt-`U7gGEWk=*jy5cook6l1OB~TWN937I|vCkjAOdHPxeJJ?gEZsqn@j z&zH8T4xzfvvBEcj)y6rbF1(c^$A0!NSJ>Yn8HfHcDlIRT1 z?;xS?O#3qv{m01UGs@G*XYuUM)%!FE(nCT_d9uIVO3+Ovbr!%ZRC5R=aNtd!J?*sYu&ASuFOUN3-wK6w{&SF=qw9KSi#!&v>`b83waeIy{0DxD`P;oe z9l(-N(~oUrtqK5-^#-3dKAR(H_U-dr>vd40Q8Wbk8y0^niuf$GJYN;S#rXaP$J2gmUJ|%oH%4zPv8tpo?jbC}z{}V4o{;zU{$t{0m+_^qCBh}d zW!qUj8fE+1JqE&mWHSU-nIM!_6lfCCPZ}GtbL;LC$rrliSpmRvx;Wphi2k91Y=|<3 z9BU|q527H=YTm!9K-<@TFW^2Q*&cx&W3W6;(pZE@lJpMJgQf_tVWou0Iw}aB=vSZz zwW*=!Zg|C}vt5g|<%&C437mkbYMEI`-cH7oi(<%Lz5_kl+Tm8*jH4E>bi!C&k@+YY>A%yzE(ltx;TrhMu>ku;$Ge(-h1Gld z&&c<;=KRkbVFJb>@~kZGlVJ_*cRTFpq@k=}i6&%|wX>7%v>oo6N$y%Qj}r;YzS*(1 za~e}5)R^xCR!Sm~Vg2o%FZ}iRfTWFWK&eQdf{wCwz5~D$H z6ouBqdTrSiXYzyC_3Ck}Owg7O+y(SFoaJE>-U_r##TZ?o^>(8y0}GHJ$#Yl&3y0P`goFj}knZlw)pE{>9$)&ol#Bp|dEmmBNw! z85CG&!xQ)XQ$A-Qn8Ywpbm!AdxQaEQU88+vJtt?cDg*RT0?_p>MZcOw;_#Q}cyb#=06^#AQ=ue4AQA+wfG zhwp2AI>Fo?GGfl^mI()w#Dfm7^n$%Z6MFm$uQtYpUiM$pkGnV zKa_=Mi0olG{A_&-jc%^DnNS8TKLeo2wJCNwng795zrVn2gd*W=-n)h984Ahda-x*) zE%;vUVS>C~M;R0+;j=E6Up3+JHPo0gtgr33kO@Rexq%``^s7Z|KNxmrV{PozOf8~6 zjrZH?cHz}Hc8HN`Ne4Hc72c@cn`fk_!l1NX$q`6dD2OW$OJvF|=w0Gr?lD zB^W$Vuf#b2uq*6SS<$w8o|>vVzYzv^ixq8T*E*?vyu=OdFk6Xq@^s!OV*lYK&+MN4 zXTJ~IhCH97tS;-H&&C4xkxJpYhk|Rgv1N;G_ji7MVffAY+DS|aT&1=m8r>Aj64HOw zWb>B85Bk4a?ot@ga_bxLIbZz6oc}8^h@>9L%7E=G)c|?*%v2}MzB@6j9FNJlp%ox8 ze@zx(7Zt93Ps5$83-Jew-d-%%1DDQU9-zzrKxidiFT1bsa;8vDw> z;Z*P|+Eiy#4~$I1z-q-VHzG`*RRI8L+uj@OPwau8HokPz@{Jmd z##*gt5Klh%K;8vw=b2BJJQ|a3PZ(d8+m7v>bn^cCx$?{7KY}{ZY)ToCX}Xv4$m8y- z_6N!S%&g8LZ^--Z2iyqFch5e?j891+yrGCW7wCNNd|aq(v%e7`+SML^#Z+a0@?-8! zmROzSDS2ha5vBZZ+5>baG{bN&v&i{h@%oEVbY@fvAXF2i3u&1Q)ICV z5a8(yB;Xk-7NJy~+1qmxzFbMX7>NsdR57X8jLyY~xWEU0aOF{uIR2rAKCk#EH zNwO`TSB*o%R**4gqQgViB&n=*e8`Jp(-y&%x)Igs=>F^X6$0YtT;ew%Au?7Ef!{xNx7yOX>pqvcGdIDq#^v4I0BhP7 zLErZfCV}J-I)aSF$}7J$Hr1H%z}tRZZgcs-b6mKLmPZ;rbnPM z$R<`khGlFMl4P4TGNE_VwvY&=y2z%XLcHjV&kcb&SJVkOf zn+HsId3)S)Ca_Rmn;oHK@%?Y4x;cT^nP2Yp&;BK`{(}W1O{#%QtvaawTsLTfwlZsX z;oZWo9Q?T2^7i_&_xpDgMfY57T^%9dvmoX=EOOqq>!qUJp;SCX*kbStubP%{>xtR8iL8gcs-)ri!&f|5QHGzgJ3H>H7~2oaqiuC4V%g99xTC@Y-prqDvuMjRnk!IuLxiE zMB%ZL+t=yfm<-)MDIO(dCfN%)Sru)q^N_kEVLE|zFaMG8UD@mzIiyx#@$S@HZ>*-b zMohu|NDa)7UIF$$C6#C!Y)BtBbU%pgB2JNc9xs3VP71<4aq4OA?M%33yG^y(+kZ#&sAs6^+g%kAfT>{`@W+DQ>p0QTnBeFT^^f!nNBhwJ&au5Xj{C&_qEB; ziZR4yAkd|^W5xuQGT8!5Pw33{ve-~K-bRZPRG6E z0`Otd;{cE-Vgal*ii?YX0D+rWbz20^i+2Jp1&@0EG1JH*lQ%2I(#v$Y zGh<=i_P39j9!0ws8mivl(zYidsn3-RH2yp=n!S*lV$rQ%ZVxx*vM83O$##F!8P70P zh7)Ok1&ls6fU^j-fY*A5@fIuoL^RYphiE@!kg6g^Vi?^)dQvXG1uL8Oh_|@v;~+U`335o; zRt`&10{F*ex8j!qeUto`iqdhTJ2uw|)6d!!-C~hTjYC#Vv#@Ykrp$gf_PG8$BLiOD zQ~4FnLfYNOefys1*C?^EBI&gcj_3zk`VU8ytY#(2n2PH_3?RAahr=_f?xOOM{ge9V z=4Hc?hwfGjz(^x!Yg+<8WAyUpF84m#A3bbXI=GDjKl%q}nOi9lmpR?rxt`!CQ#l1kD9)hPiT${(26xJQR%Q17O{#OX!clC zhF;8M*ZT1yaGR(Dc#5tJcQS-LTARYG^jHeLn9X2IKyg)dYxK#mor*f-!;6QV10N|C z1ra136RJ=DkZ*t`hYpH(`uGng@{8X*-M>IdeI^>+$OO?xnu{SxzKWCsmJH(S4^AZt zZe9II&!E`AN(r6Zv$T4%DQfJq&k`p@gh0JPqiie9CnA#`^D?btpsTm?;Z{p%$IXGy zwTpki!R3R4$CBLdgHH9t6l79bF`v(Ut;}1T2kp%Eg3`R44j%>Nx}oiW);+?N)^e$1 zjwqtVRXBgY>gz>9y-z)sYPvbvul{-gtof0-oI>An6Cb-hb=8#NdCKV7D|ygC(84E* z<2`2-2T>3#d9*Dn$E;P&H`=V)4Q$PH;KotlL`|fC*iJytZ$aj<7w7Cqghu)`_P#qb z(O<PWopVcd8`3FeyV;;06P?N6A6g+(!Et^rOGu76~U!*tbSM@PpI+s{-MX+HJ0 zQI%$Q*xjohh6+Nn{l9cf1SSC8mw{t|RR67D#=MLmt?Yf9fTY2cWvTQ5RrX?ZPZ_Ue z15z^Ane*871HeIR2{^MXX>ZSrpsO*3{^ADK(+fup#u9>bPZQztv+)+ow?UuYkUCEa z$60C94rO0j;vuz175lc#dn{EZcl|N8OT?BO*F}nPqOVC5*ADyhl3vQ(VJVu~Y-i+lw| za013*ILQO$?^OrI8%HD`Bu3|#emG7j9A1h~OxYFX6O=?L!Y4!TY`-F?=dAu zgQAIRzXQ)-p`p4Ue8uiqDI+5a;7&k3&Mxh9Dy;cxv#zJ`c~KtDTMAQCk`Fonv4FK- zQLG^IGTPP0+*1B;n|m(_keBKUy^4Q7_&>oDr3p@CCGgF-jt(@^)v^WPrI~}BI&(;= zW6g@$PJs1bI`W>nNvRZi z=n|nfo-eJ=4d?J%KOHJ=jDm-j*a_~5WAmz2v?KhpdIkrA6rh}O1v&36jMNZhr=h}I z;0%%-7m7~z?bp#>)(_prqFhHZNZFe$UP8g3SQ{lfVtPD;2~1S+D<23?feB6iz27Uu z#*q&q<50tP_+DQr6-HGMiL(t=PXA}YhgiUy!-a#|(!=^hf=_1_}?Oh|DQnj|3=kh=OvM5CGaC)d0vDh z2#@YsgaT|dW!a77YcjfWdetnlI&bTlTF3hJEdRT+jvY>rE?wN=u9)`esp;B>+1Z}Y z5wDPLMWtXH95fpO3Aj|*S!?L&;C3-+%Q96xppYqdD;KoT{jg$bkX&_<+4?%BKP@?N z9;G$jEfd1`U!$Et-L!IDZwnltS8Fy0--7q3@2wmyG=oi7{7(qST z88M=;dCGSKt1`yf=uUNn5|FKihbnZ>GJ{o4;Nm-{-fHcm_nnKeB~+M!!pz$BQxyXO zO@zFvDncdlHy2mqOll3m=%mBZ64XfPyXlnA;g7C02I;wdGx+H$p50A%WYNkPbF|74 zV(SJ*;}WZCy3iwYBa{H+;20nCy` zE`U>OvPA+$r*6&-r78Z?IrHy?o5f2h(f@7fKt$&<+DUU9{S+UEx@ov9Sr3H+jF$M( za!lbQYogI713^aomR*t{BSIA>(UTui7t54?o^UC2axo3JKi14{zx{Ocst+t@+s(h_ zX+uU51jkVsOyV~%x?)P7Ew6Xl#E!RQmZaM03cIObw7m#L&P&M*fw#u2--t#~1n&e0pUR(r41eqj*bMh+r*5nMb*$q{}PO9Cluubb~gOz$L$59y}hIUX=Du zA^mm_A&i9OrC691eZBAhbt1o^;{A#qB@*u;sIfQ$l=q3q^SVnH9j!%k+*QOtIU8DsHhBU!4q&Jo0Eukkw&qZ zNf|piIAvggnHviJ4Gyt=BzM6#3+dC7&3fRt0KlHdd2UZEE_@3~5=I)ud9d8JEYs=E z@yHJZEkMX@7B(^jK~n?=HyTbw9H)xWGufQI-_ZGM^? z@RwD4#lX(*1A`Q=v1#9LXsd(6wrSnaV1 z)hVB-m!6-UjUhA+csbv2ECjIwzsii;UJD4eNTNXZOJ{F(K~rL?>CH>x3Q_>&229OW zlCtdu-EYvvbveuc#MOBi`Xs;FzrdDGnsg`4T)XJlx4y%|xZxlR+p0Q8=G*4WVbJ4d ze*dj?*EQ6DDNhmn4T_m}_LI6Cr@AUlL z77-Akkcko&ZMNSLzOF{RwW1VL=9to|f#=!-(1+3cA+e|eES0*46kE6rUp%=c_Vgw0 z_XZgl@DgjQSFuex1kQ^@4}3VK4l#R72)bWiREMd$Hxujx58zWxi&n6mhCHCy-37l8 zVY=+2vk(~fwOI^g6&e^#ShY+&^Boy^ZXOKm<`wh&Dl7L5|bvC0r!@&A7FxWmI)L*7I?zIcV(f_Dy^7O4;d8>&rwWZ7I64zi>? zHGf9kn)BlPx<`b6J;Oc81{R|5gKc&>DU^xrTBO}l#k?zz9v>*TQ=G*eNVA>ctf`wv zGN2BEf9+J*=`jGOhQvBe`u_%l^8p4fN+yf>k6E_15qD*8Tn!%nfb_5R0<(`gsCRi2 z<_UF`tv@P0eUepIPY`r&3IrbYPu089erow`_wFsV(!hteq-zGz@E97%gFPwgQ*oub zf<%6jtr~gGNDR+Lp-N!EY*gy|xJdQuGxwfsImVvdi>t4#;+lO12BLq2lBBB>d)@OU zCox|C2ub0+!MOTam(S@;@=4bNy$X6%f=fLnz>PhR zF@LFpgclx{LL(D(xUnMkxs*FnrKb0m0horzG??Z%5Y`2MoQZt#u$J=~31@sk(NSqq zdcj38j=73AI$0o#kL#7j)X&c(jDF;MS-wnGR;y+3tV1tn(Jde0B+=RNnaG~4TAJub zbFrH4h+^}XKGmTOSB#z;^xt|JWlnEgs`)yb3j5>;;tbFb1NrF+@adSOdfJBG;q!Qd zkFH*ey9AK5#>T}w(Nr`+*2Hf9(3N%I6mqI_%~&AMH_0lf+$bEkFbJL$nB+ApOQ&lr zpc6-*jXA)|Pgw_bOBl#=^96_nU~8`R$$*$#@OVOcX>nyqejvwA%)i?4(n|t zmdomU#DIQ>ZAqq(_tI}*>H>gr{}{ye@K~poisdvE zB@ZP=QVt7f^zpHN5KJau75OcwP;X6owlHCfSBtxMn!y3C)G|b0< zuPK_4MM>j3#mDT|8J`%9h7+mPm>k(jbN(6Cdqn)e!-+MaaeO4~rzkp|jAN?!dG2gh z@UyKiwxG1)!1+$2h*kzbV^F(H6;$W{u+JQm^>WtQ6}W{%BPM2ybTp4seRuN^9m`}m zOJpSXh|E0H%z+rjQrR0RYa~$Ft041>lMN7-F^pXRbI;>{FO7hgGo*KO&ze@R`2T^< z|MQ2*vzNfiqPKNpvVg`}puBf`v2)3#p?5Ul_4F|2pQAr04b{+ThxcD+I+j;Nz3XP( zFabv(rM}u{0d`Ai^&VSAu-77dr(@;`y$|AI9>I2lp|C+5&_Aj zcZa&ZC3D?SUm1fX}-qcdORTN*;{qb*yv6q}y zCh^-vqi|^v0&JG1VSBO6u~1Kc{3>mpa`=8T$=WdCPg&% zqz;v|gkI6Tjy-yLS+8?>@r@Q+q+@?EY59-CYiz(AEkQjsZvR&xA8X5n8RA69!@_LG z)wvI#N_li{bvwk^%(?)l1LZqTRS}HE7E-DpLlBv+gsP2{kd$To7tyegCdzi}GBz2_VxC?+n$r zZQ2+uj}OMDQ#!;4u{uCdUkX9EOhG(qS|J#i=kt1SPZm#`nGfu4gr3knS4)S4Cl8@z zh@O6_w>gU~9X3|*W zfR@|D@6BaBPqo^Yl!;1!08g592QA=H_s#DcFuK3s#;3|~AcjXNgeOg2We}v)lZ#VL zg=^Odhph@_OY6Bsj7C`(7i0v<-=m~$tQXDLFH7tX|K#M@DMo7V;<)Mo?q#|CVH91N zxb%)XQTb$DqU%1c;M=wCyU!EdsvErXr%Jz%mvrv zlINl-&xg-lbFp34nZb81vj7(gQxM&)_nALCJ2M+h=NU0QqGbKdPsU16Gg=G?#_pcC za9Af-ALTjb6UP5rrYlMH2@uN1l+93w{KZ56(ON$V=8!Z2#DT{eNAE13-n3_H2az$~ z#R*e7Il7~ZMr;kJnG3`H{q0S@$4f^OF4g#=`x zHSyw5CR=%f;&t0>84X2apLma8u`{R-ixBAe zX|AE{OW#Zi5}UI6rz=^iGM31PS|bXglwqZZqs6`-ABNnx_)apER1*KrkbhUiv$T$ggNLO3F%o>@CD0uz-v5 z>VssGhr#qjL@Ua7JhtQE)J3SPkDjZK*`W(m z{mM80|Lx0PD2SF>>wX|JbJRkQ?>hjHd|-T@%3?Z#@FIL&@R7Yit(raon^6o31=}^9 zb(nMFyOT|CfBK3=?zoGxu6#0Y;KI^vjh9F?{?4r6qrEh@9S$4KZvDJjthrd$&qEyE zn!OyHbkl;=vClR~m;B~O_%_}YZCv{|IDFr~-sg(8YFBa_NV1Zw;z)aEj3bVnp{_)D8T33^=3)0;*-oT!Yv0@so4@a`_8-sMHF zp!o8-5D9(X#Pm!=&*W@)*n4IEuQ^0~Drs`7LfEJG3Oc$`?MOxcKv7|%R`NknzPrVP zZJKYCQl3guE5kz&!S+?-;XQ<}qsm^7*zF;qV<69xzY@X8896- zjR!3bpXURHsQ_>7>TU0nk(rrUe{(i-2#oEt1MyLc4B5e$%qdV`MV$PV4Vo%e8a5Q} z$covDfcB4&LC3F$y@f?1Ynyoj^tr;`Pm{F_pWT>n0`CH1NC^XpeUr3M`Z4Tq9S}cE zw3c!Z-T%kb0##uELZV=^IW=kYEm@*}aXu{_EdsVuumplIx@~H-e4@uwpeQ|*wlG@( zH>r+>O$6bnQ?!zze|;s;ut*J&;a!e8)Vv_0yXZb@x0DLZrJAVaTTBZR;ji;~!1^k) z*pHfisY&<6mBN;@S9Q+dcK5Mz%?8i*;mMe)dHXvI3|fLqf5+wS*b)6UF;!VTG{CvC zESnKbaA}Q+DeXwgjB9?LPYC~C6~yVTf_%`Ye)wN2h%}W`CqOILpz`Pm0L_z`ihelo z1p3_0ZN#b0ybn(2>KzSx8iSL3cSGsU6x}4gKS5=nr0qP}06j!yyPW7HpnB;MkihW= z@zAs<)V=ALj6P)%IyeYpSllmy7IwpuEVRwsemyH_4?kvLkZJoCc!nCSJr$`D0O z*5v!YDz_=9Hba)<(@lutn0$P!fJINd$d#f2b)`{|Q9&N+g!Oe&SMrN{!URfsI5~*0 zsKl95tWF=xg9Wz^P_=TVlaF28$A!TKSs~`Wi*O`0T+36$@M7Rsc`v)J%Q5P*^xA&K ziGE_|lB>JwBxfd4ZoAejGavsFxBu?;=VRbX5QfH6&z|?>fS}ssk?3If0EWG-j0!{d zo_#a09g_gI`4$PqebSB}biB1?{~D+)Bm(abml(E#1|0rd=UjPl2YAL%0N&AkW#{Hv zu&Q=hKjl{JzCI-5<#pCa?ek1~v7!0;Xy9%UUD^7HqU+3Hw)sL2Z~6-@AlUbXfF8^) zv!(%$BL<5>#?t>ETUQwt_1?5)30b6L0qL$KMY>@D0THB@Udo`mn*9&22ON=mU6$;*3_%_KTf2kG$Sy{(shnJ0Eiztg{;t@8#H zw&ufIdoDY_Z+I;F0T+fqaY=1I&j<+2tr&j>z$vx;pV&SH`c=(~UX+=C_p43;+0mIl z?VzzwO?`#-7%@@UKzW9VsU^Aat}klJvA_h;?PD)}uAb#N&w@jVBkeM-BEc^J zTjZgqW0%&}aT8*KN;5t&{O3DwlBn@}h${U8+#*<#EM6$bW4+n1kyt6*Uj*?5)H0!^ zBrHM%(XVyaob$~RBX2TWjsnZ88gFjce10NvJHz;_BIs9aYhtG~l?=%YOM9jaY>a89 ze$VN`c1)eLON(R0fTi6Ab?iRa5sW^}$oQ7ivjFKQhbg(@i0W)i=;Mr@2;_3j=)Xx;-}8-P2QpekLCGt8^S#%C|qMXX?e-lwFS_h2j<|NpYos@&Y*! zI4PhOT{Lu5OYKexu)+MV5B~dyR7v2IAXu=lFdX=g88K^5gf45e$Y(oeY!=xh*uT$> zle(YI^=bO)%aQx7(^#tV;QpG&%YwW8<*hEmvtN#Lz@-ujKExeHMhy6+=f&X= zZ>2>aetxno(;~yR?nLtg;J`Q~4w<+!)cT80F_q4aL&M(>`#(kvJF^Jsse8Zx- zcF#py)pPuPKgN~)HYP=;5RT>nKX#%h#LxhwVemAxW^p4YFR#Ois~1bhT+W%YGq{_B zGh3G`#6pvj|2}t+bMqZm_j~d_x>MR&x@AKR->I=#9%`MfVJO+XR8W=Y*e-G&Gt!F= zeDS^Pv1e6O07kyRlhrFG)oyeWBb&&oGzD0C*wVKB<^2l9Ia9+G~Pup?BgBB4dzy15+ zTZ;pOV2eMfic|l#&#goJZ55fHI_>%Gj2+=}Fc6wCDPW2*Jji56za0kpMQ8wkZCTaR zw_ZtBdmTNKxVbv2KXo`|)J7~cFy$?mBq%X(3O?Od^_|8Ezz)2R!I?azBmUOHil^su zJKJD+3BSNvY!aQ0in$~t?Jm0xW0Etg;Y}kb}{WcmUj-V{Ci@kEd6`FVS$Uk zA%7Enpt!HcYvtsO88$Q*g|0!D_zj25tF0I7wa%NGP{|}bb-xspo+iTFQI)3CaFA%h zC@Afl^Xn;Xv7-Yg>E%hqREzyjb8hM)pXmr_Q2mI-%vW)xvy0kKwPFS@#B3Axvu_p} z4X9D!j9ffKL{guBm#ZEgT;xq+CsZa=(}q@E7+C5h<<^(=2E}aRQiX7ETxa;DKd7#+ z#VWV^ecww~huyBI=bIfUw#UmGSXQ)hMxrjGqB8hgF7Qj@A zK^o%j0XEV8cn-8ph90A7wUhE;ss$|+1_t#ke09*Z`OgRc|M}lP7Il}hf?$Iy?%Y)+ zhy=hrHjwD{^#zDAJhFxujYP-5$iReU4JIb=jxU#vwzf!Fkxf|xmUyj0-)sL6r-7MO zKEan2+S%2HT%p8rx%EVTDE%B&> zDpIh~P%`%F;96S_89(D(Fond13J^iT#vLI`m`s@ArcAFz1^RYA~3o_B`c(|r1LMLY5*daLT3%-C${R#;RMI;uIV&Svb*H#;|eOtrsl zV@$?ZQD4kj+jDEFzb*2|Of#mKzF5#I1{}y_O@F$K<}xD<>i&uLGNTK(OIt9^#GyNa zh(gGg0wt9+^X_^9&ag;64Qi8uol%Q!DgMbz2W*h|)@Uh|-XxpToPMcNDm;T*;^}@x?Eh2y;0l1@XCCa)^Iw7kHy@EXFmMJUI!dW@^3eamizFh&vO{R-gL+5Mhu z_!x5XEtjS?r@F4wHU4+Hn1qm(_%v!J7m_f1SSmEu48n;3sHHKBbM1hgKDl2%aIhW4E__r&PqU75WiZR67# zkL4{if{nDwc$cc=I@5%NghhCIDmDZZauqj}apN!hm+jK3q|)D%M-x5{WZaoy*Qiu6 zLLseTD|0@N~=)4aZVUU}aG)=0i7(!Jr6S0$Eoz3C5>Y zgrC`$HBfb{@5o!bn~sDn)>0zt*OLwTtOiL(U3&SK?|lcR;JRaJ#3ghX;LL*B6@VwX z@^ix1;)vWgbzQ~D@SxsnHUAiC=?I4c{_C8l3McKKbn`DG{gT7<#fNfgYfl4akpuDI zkUzvN@+jkzy)<9zPoNO6j3~QM8*vBDWvuj4IdlV)+~L2L9KkeVkmtCRanHb--=LpA zgHhjfE#$hm*F)jyb!d+gDk(^(pCEMNAG9rhTe0)J*m*TR?Ud(cZ0LaB6h$BbC$ zp<(;V)GkO+NSQ(lY~`(N{?PIQ_7Ot~3?Gi)Pazi2G0#iJ&a(8>!iF+xT$L-( zNje4 z!j^&4avgvSZWl>$O!iq(qwU2QV_fs&asQVocR8&^v@;9=F90&hwyt)5&PH=Vt(MDZ zOv%4cfYdES$oz4?IOUIn{R<-S0y!xdwyhYyH3BAWS!OiL7N;%}gS2Glh*3}uY8az5 zH8o>x5@vllDq{eS<7@@R>o}>Smt9mqK|=gxKm0(6i9rDWlkC`vCzFwFsHQom{D$Tl*sN0(dC0-MSlz^MZ))i}&ftpeUgx zbrj3W=PzsRFmQqa$1Ek5tEI2N))Xr7bEzK#5i3MTFGsz%D0qt3-O?ia*R;oju%$J$ zt`FsM3Zt>4KTJY7En|n{>g+m)cJ#8F3n6_p$w-&j#~t;{E4sM(xq8+et;gD-n8I*e zIpu9qB*6_oD40pJTT9JiU2&;V!#LWp zx6mTdw(c%Ar!$pVPN=L-4^;T?ymR#gC|G_^=T9;I$v=ORQcy^FyX;-m>%jO=coZ$S z#;pS&v?e)b)P3S4$HaD-FTl-e{uiI=jgM2)_j7@x7Pp(fOgw*kbqk?w5>UlyVa2{C zOfmzJGxap|yMH96Z-j`Bu0&mxdir%eMMDx%q^1NhteDs)?!Uh=sUzG`Jt8Eb4SgBX zw=58Zs#$+L)68$WkzGRGXHL?~I?2EaC5*{Fc!8~+d`^X~sc#sH6Z9(RQ%!_Tp-k}Z z^DQj`kHe?Y*ILue_$}LWgCgs)ci(RS@G zcg;#Z-z%o+cLpju2o%25z_Li9hC})gqiDldm>sU&m-CwfWz=idrv%hs!B=V`0h^6V4L@)Trf8WXd>p$@?E5hD@B-`OsaeUz{7CS_-JKUgr zY95bdm|s-*^p!km<>?&zsi((|Sjn6ChT|Z&6bVMqbDGpZP5sD-C)i3^NU3=jMj4V6 z2yy6sg0zzIpxFyf1TsS>c>P z;midL8WXS%Mp^3R93CY0LM+UF^&ELOtv7}$e0W&R__)yb^5nBSl`1tq}_C4F){WCFUSZLN)G!#Fm!$8AGhbOT5z7yJHseMldzkv8; z;XOHFDBG^xP+IY0$o;q<)-NT($=vUob$q6xkbY~-5xSWk0IRNIyLg5etqW;Fc zhDfqM@1Lc|jzCoExxUgF|6hkY?}79!yrQQB6HMnXd{Yy*cZwOC#gEmw@ZA&yXX zS;p75d9~uO=eOA6*Q z+u0m*R)>TcYb9>Bv)a+k(m+UqnXsa}5}a0vLa~upH3) znZmM(n0dGkJ9}~YYz^D>F*vR}p%9&)HJfgn$pT}mR!))N@Q8Gc({5bA#`dLLg9y_- z8=LM_IFbc(_Tcf$R&?IW9!!hv2xV$u2cEr|*my=(dr@iIB{Ab_mHpcmq>tp_1!v0rb?x8rHj zvKhs+7pP4*c#_l8FJv)MIyySBh6z70SzF2BQw^j5 z`oH0*5qo`Oask$tkZ7)gBqyxSBIa4+ZVLm~>vs@0bKuzA17i9TP zFT#l*>z*f$Yr9rZdlB!w+LbSMHkGxm;kGt-cU9|Cs3GBxz)XCSUav*ggZXMr`GUK0 z#t7fX_B>Un0W#0$JK&*5pvmL+44OkWhVm7M!}q~GjV2qH_aKgL(S3clC{PO-0=i&V4z9KBlRuoOm#JE*m1smKD9a zRH32YNyuvUcn3phBtx1m)a#t$HFYf4lSm~+xe8=ES-!RM9b!yQMHsfsuRV49&oH?P z952_P9^C@vG|EIV{N->X&MPt88m>$^zY&S%hW8W%t<~r?9oPHQZNJGDFBeYnt=T6Y z+DsvU%|x7~N2?&Y&XM+V|PuJM`znS_Sf!mY5#y{gSH-9La0EmOC>J?Du>oX*Jq z4s@JL)wdi^i!lm`vai_s0iUQS7zW?NCJ=m_CI|+W0XJ@IdAYgOpsU>OXY&1hz}Al& z+0Lt^?!K|QO6@EDMZLYBukaIyA~KhP(M7dEgziYl>>X11)0+lz8w?pyM@_j*byeEP zmyhIuCEpnQnqx0;qhKKw?^lYRc4JM@>vm>g)ijNci0_ptr1YL+g1_AV;BX)F9oZx0 z&%qbg=^}`3DPtFp)1!62?$yWX=+$|>WAB;j`!9|}?hfK8Gh^_)YBgl@JAF1{kS}o@ z@v?ru^IpQcclezGv1TBCDf0AJM}=&eokdBe*@_z!)ESgP#cgzPNOsDdPB0S&EgKt> zUi-o&t}yp^@*t)nD}!tQ>F06wy&*f)XQQOJK53p#mrOE7c2<&VO#dgT0v~AV7gt*v z4+8Ixy>a0>{QCT9T~3(v@yNVJg4N{2T-5vwhdakVK&=npqqi(*%qRcidJw#%@VXWp zmsV~$lSlI==~m%?>Qfn0J%siRMf6}q=L{OZpK%neUE63Ntaki|fp$f|Q3s=o8|FRq91R%z zU5bka`dp+E3pXa)rjQ;v*kbgFrZ=ymfE5!fL(0h2!=TT7NXk1}~g5wvYHL z9XTAup8fO(+Oz3(Gx$_vm|_s>;fvS)oTcTkNaSvMVj4%bR0xJoW5@>#05we%>A zk`Dp|<{8mrBTG3=b{H{MIbTt4TRW~gsngw0^l$H-Bh9+Pyzp1gMqlwYR6qOFY3R@z z+E|6d=K7-`h^T($J-i!+H7JoLla7>)iOVK&kQOA*w~^tSZxv|0Z?pc@w)dvxX+)gv zYkIrS6cUI1OzQL4-|-a<>g;KN3)f2xZlxm6z@{xhjMHvr#$M4p$SC`YnFPbJ^eEcM z(toPXUt&OBIcemn#`%k>Uea7@S;vGoR!)HHnM{-5Jaj)F-=(^HWmg}APIXrZe^<2P zwEf*8dagi@)5%BI(M2-dEq_d39B7pn4(Ys=1HeU~a3Bv)u~0^O9*Kj(W}O&);JlsI z*VU!z;(N!_rhSWh4R=J#rLRc-6eOuB}V!8Jlw>>D`80@=fe-&(NM16TYinJ z*2{Os>yjp!8#&)6?qY#jzhHSrOBelqO9uIfePRu%0S@8h^N_C3F?5K{Z&X}5WUHz) zw~>sMIANZ1>Z6H0dNVo|r0y#P;W65}N(nrjqNA*r&uTF`%Fx=MRSYUJ6&~SPM)m*y8w%4}`lx4pd zNQkIX1EapFbYn~~NGKawUw#!ld9!Llnr~8rRKg?LCndkXhxYK&>)gamtR~B+T2U@T z7iZ~NKU}VHg!t0hPM?StUX78Tc}imB^P`4B#4&4ge|OimZ{Q*`pe=TION9E}+~Syr zjj@!vSTm-@OOs8|I^^jlc~GKK8BVbQGYR4jvA!&K>2R|)RrQaGHGcd#Pos4nAf*f zwO=K5ljGI)mo!^TV_m;0514Q~dqX7YPJFRnA4T8wk)FBic(Oo{SGFPByH3`NO*#30q1T&Q?Z6E57=*rnf%pc(ELm!Y7Ck^0aBX63#^8(OOeNeA?DR zlN3W#f3aQc1av1D)i=INe2#^M$#8RUI0-Ia298*U3G1X3G*EF?4O?JC$T&ISme&og z7^KI>%X7oO|J5)t$p@NQCev2BW`E^c2^@)=9wA%q0{7OdC&YCt{JF%ErolsftN6<~ z_S19cSwAigjp}UdFJbmyp1*OjTJ3sQ9_!znwJPW06EJhxwU8;k{85TLjjm+VVx+Ez zop_q@r!*f6@q^gypXW`TIP>K)fROQ6Q)oN%E1zT23Vz7X4>{t#ur~Jr zlfU6LjlI2c{I=C323$FiB6_HS8?TXaDd50>rt9U$tqb|t-t={cd9*JuMIoIZA(Hpu znb?sC7AE!Cesb%#yONr`AXp=WlC>r~6~A|CzYJyMEB_HidjeX^g6rBlP}@?cY-XGkk+zV^rxz_r%2 zBC+=?CA`;GD)aL3gRnS?pteDYI3;Q)g{I9ywvW!HE&!yAMd{UzO%6?v)#Z$o_BTH$ z01{hsSrDuU@lOojYPqE}TKt|r{EIFOL^$PJ@RKUV*}6f~1peI)aed#e-e-Pi3oZyG zV3qm0Pn+$GZxypPGPtxY_png+i6?3BldOb|T;EK#tj)>%23UQ>uStb?109{vQK?H@ zkHv@g8g_&cuC~NwQft`ELq?IZBcm@)%1$s0U3vEiPgmp`$?M9T=3KNAzZ`URN-ugpP?uPL?Y^2w_~R#mWORVUFYGj~VyC%w zGq`>|jTte{ar}Gj?rV`buruK`5|SNVS+RS!T4g`WuB}6Aa}?(-LW8E z#-WjEE^k@>!G)w-8MiN)as8}D=$0G`#Q7KubAm>(AYGthhy=QK^O+8I^S>$%+WS5{ z{Cb2K4Ur_c5d0Vx_CWF?owgK_ut-yf4HxWN8A1E}fF%&I^WWzepbPP?-zc7m}L|F=9Yr+#b}*yfnJ$Lr@r1QVVJ(={Evl~C?Q zgpHBNv-wM5ODn4ys(c&>UBDR;Qg?cFC4oef+tIVNy+7yTe)J=Cd2a<$1CW)}W9?kV zdQ!@P7!hA>iZJH8OhSx=%M56s=%y)7uQd&W`RLQOqK>(8!_RFjGzmdw@Q+MXTvqA0 zG^wF%?>s@Nsi}9nYwCeDO>1jwo}8vyTC4SaPRHBP)(;pH~cmvQFnq8UR7!1p3VB=!m9#|27Wd6vwwi6heuk^Yj^@ z+VCi!`u+0<86x4+91Z-JJ2$a_cew0&BhWcg@KB!kD+Thhb@l)|Bmna1$Si+-wW#a5Lql)*#I~A150tj$|^sKKCG_??huF&8Do1-7E}Jf^&B)2=HSM!^C`> z{T^&gKvsjf{6k);i-98}Bj52AG(YV7C7tPgzxu|VcUzTnzsX4}f&TjG0z3|r#cM`G zKX^FB6E`AC-cy4w4LwbRBe0Suix?cqt*mORXx)4@{y;H3TvIjz{~lpoZlEGVOc2_Y zK2>)Or+PbUR!QC?xe-CLgA4DqHB1ID#}+v;)835C@aXJzdyjwW^#O20)_i}d zmu&mrZ2!rPM@C7b{F&z?fsAz|>;k9wLEG+J#Pq~G0+jGy{WgE-8;9GZ@MH9O1BSZrC(}Ds~hPOQT zAvF)CbEw#XR$rwnH>WneZI(^K9?VqT)&&@4PKN3SH|;f(d=PD*pvj217e*7zPzlDK zSE}E70lk?J6rwln%(aKKpXoT!9=?|l5u@$PBY-@&{D@u>jLDOlTkN9x~z6BpCskfVf3nNJCXV@JbHDA(7 zjeMoA;3Ji{+Uvmuzf!Gx*^;iudowxk^qWy+`)6;;Q@04nU5luCCJRP%LfX0AxuPPT zJLezp*vm?k-0U!N?pqL25ptO6G7&)u%c=^|emdx*xXC_Hu*UZE(FITgb0!L&p7l*d z^Cr)*ClV8hHybCi8yml&4*ZE!Lr2XYp3dy?tW+Zsl3100jWj&YW^}eJA%2pT zpRBB`&aRF|`F&1oZS`9em&`%WaOvXs4>+)iQ@Y=5|9-F7aM##%XDABb8KrP+iW3+Z z^w?NGn7y#gLFc0hpZ-1fJ}z@;KOZmWh&U*!eABMGo+P#_^v!U0e*%*vVu;F-+oYL7 zc;1mI`IBKpCCAHpL_ZuiDpp=j;r$g0QACXn%Nd^sQ!@F_J2WLsE0}yLZDU{iHOFQc z<27ZTxv7#Ab$^F*tD-1QK+L{Hj`ID_Fa+;1D#|VOVudq2O^-vq(s3SvfZsI9ZKoNU zM6sf!vJFwk*Dd;s19S5o-b}+w za?-%XzS{DS(5c1nRsh&^Q*r_%$j^0Tp0YHkL;Z)5u{|XU=cL@3#_Irf2MW#;Y5;-X zK7Yk&;Hl5jQB4o~Gj?rR6<6rH=dnGs*B0aX9gK#1VW@5RE-ugB2*CJg8l#NZqUi%a zD^w`AZ|y`9CY6gjxFFC+>@3iu_s}0nAY5rx$IGuLpFEkK)5J<2;a=^tN%v-Tgrqk} zfm^VLMWiSPwY~toc*k<2oZsF+J`p7$GE*hs$zydTZ&zG743LnZ>Y3K}Vj8!}6vNl3 z^VvFV30oPYQE?kmnTC+d55m4I`(YG23yhcX+h#{DOwKM*$|3#$hOG~ETbqkZa(fto zitap|D`hV$le3kRxIJ%cdJ+cO=j9;J9TakKasA{Z)n5f$-t4MIot)=AG1i<~#oW799kY0CB%yT({#? z+oppy*(EOQr$)!QB+27C-y0r7Kn1|_t;=@t(~Q$DavN2v*dDQbIdscd7&_{ku;i;# z&I)BPDNmy@3QEGcH&9dHsOdsWM@F`;nH;}X(S>g#e3SD$KaQ*by-U8#q~-pN|2kRo z<%VGdN#9rgmm!W%Hyqy&Qb)?p-?_%_z^0(AsxE6Fa(zH7csS~5A}uOGtV!(uCEdiB zihjb$!VA1GU=j`soyAy~w3uT?`$kELue~rEJzU14B>VWogSZ?DQX=Sb zfV?FGd<~vsmC$zh!j_o4VvbFHPg+m_f{v+tau20YIz#!>ke^_-XpqmyC<%}+3j+%Q z=5m2Cnk23aFPM&guwove8>>&mDs|Y&WVdu`|uzT9s51X5a87y)Gxx~Ju zbk25Kxo=7~7MHcU(txDH=!{>t;W^kDhs2Pv5S#1z`*Fl`F$#1BMUJE-J5kfC%LVgw z_8^fM(KOK<2Zy&)BJknsPaVa?e-O zgKe)|js+%=7%MBO51a;A-iWx!R=tx<5VD_(sIi&$=J~XlJX%A=Y<;FRp9he`f5|35 z0BSMzBngS4E&M}>oy7p?=sJ(9l;Ji)-6ea0iKcrb_ z+HKb1V&<%3Q-1O8LPHDP?lt1)-2h+dVdFkg$agqU>=Z1EOt0wM>39OBW`%d8&ySJ5*f_}zMDiIG z?`2CFRIFMa#d^pk;{389q^G2yMnavyBPsmo`5Er_2UUg~*U8$tdLP;lh0~{t`a;!* zDS`{K;^!N+&%IF7&kDX^1x3XmkE&isST~YWvL*%cBh8+!v#7;4mo<`B>v4B0rnGF5b{8(YU=vm-fjaxrc+3&e!H`CpA=4Qxi*~$zZMo^b13yw&BcbuDmp8K&U9H zUH_*zx&g?7v4D$l19Kon_&o_C-&|*GsxXie`clb`;8$(W@%6X67MOGX5~R@%s6hb= z^8XWq_#(Q?bh3yGG?4~RIhM2{W$g5KpjaUHo(>|RkzIQtT_(Xw4wejNfCEW747H%4 zIqf9@Q~}GK+2OQN{?`{Y?PW7*u|) zA1xqgRV6Y!jg(UW2rcP>8swo-X-)Du-r@2r2xecz)}zN5G3gHcjEr)^=f3KAoLp8? zExWeEnZg7C0b2~OH^~`w=1_fkc-eoTj?!=sIn1+g=&d0)PwP;^5L51jpKT!*tR z8}0;y_WPu-!E70>#8?EhyX{Ta4gSwJc}!d9oEEqJgB7K|ENCQ{Y0S$cT#|7XX=ws* zFIZk`6=1vR$Bd^g4eb8~ZcJ~%4X{@c^EPyNkaDFPvsm> zhOazZWb@JWl{&lK)?uDna@o%mdJ&dJ0s=YaguF)-icvQF@E7JvF4{tmOo@4tC#P5D zcilW6_;7>FbDho|0#GQPM(mo0HSnqz{J+m9wBURJXrLajC_5`{_~}sh>or`+XWxgC`m-6^UuO%aFQ^AU+S0KmJ>wd}bx7B~A}dpnMG)7^1A z2#Z?S@wK1V0A>!n(P6)rJ3=41wB-MZ4;)LlwE_|R`SraoJ9g%L3+ZdsQPeDw$QK7K2-d@HWx;%$rOwwY!sDMORbU3K!kAjFMR zb)}9@*BZT}I_g#m>g@bxeV18L93M6|ckk4xwM#MevSPAr z38Y#a<+uy(vzRK;kBEy?g26C_?B$LTbUR?+*p5Z;nm^(I)~`d^#jL+^Uc+>)i|YR&z{AJdWtHs z&qt++_yY(${xcQxcH?Yi|ElW&e#l_u4}i3~TlY0ZuZH`Nq)qEq()P7{iW9&eTU0pB zf~M`}w{=jLjT{yHb>^G8OKQ8FyzomOGAaTgT3d;&I=|?!XR95(@{3di7$}yc<3r=R zz#dCnAm65 z7`vq$+zO5L_bGXjW@~km&0w0et#A1;P&l>AsobVaF1s%zF1B*Ba+CRDR%;6iurvKm zC`}-N&4>=r1omrHBE2;_R_ajsm8XO}%QAKSzL6N=3?N#Tq9PLWkD7Z=7sipzgU+@j z5e0-#0l1BGswE1bl{L~4;~*~wEM5;Vq59cJw5?)X+U9HJKBd!*gR zShF7%WuH~R2gsT?yr29l_O`d(7X7BzFH3KW{?-reLl`JK7bpPf%q}9%T3G^X{)5B+ zLV+hLL0&s5nns$N>0cIs(*MJJEA^aB2P4%^SQL~C?kY}B1{Gin)GR1yXe4{M>E4Mw zQdtnWtSP$47-Zp~HWBtCzsYWAs?>A!c=}-ZE*)Yg?-RD2eFwsckM@-iHlTVBqrQa{X}mhhF1gzd3&=euWCR=@_Qo**AzR&P#j zkHoD7;90RyCFMN!S${R}7iRwjlK1IuB`{A+4O9V+FzYAB5*o@}Uo9G^<5vl?l`Z(c zLP1HpgcB5^_mJ?uXtqXxZMH^J3GkvbJMD~Ak2E$iNJ@6WJjA+Na@bu}AlN52LpU7X zbPfTQK-1K3&DFp8Er0j9ogKG|iVCn8I%J)g@woJ+Ir4bXAC(oR2ymlj2<`M$^YoP0 z8{k*>`)8Pvy^YYqI7`Xu&*mD*ws-?iQ+{=fNThICwRU|X`@mjU+(G`qi^6415LkR5 z*;J1$`g_smr{rz_5PtK?+DPa?*;lx*ZFrr6XQ_g6^0B~gmNkw|->agJcWw;YK!$6L zN;H0gu_;nC7QN9<2o^JQBO_`?OhMw6W9GozSMI=!FRk4F{kQFBQ}lNS zvf>nHINw53E?WTDM-(2^(=zSoFtRm?NWf!JbK=FJ0FO;HDA&o4P!hZ{kk znVi_%7Q;=*U8_=xy^~}^`;KigE`Ez)%k)Gcj7k9|rgGI$hRgVM-_)N6Umq9m?@iUz zZ=AOEMG~L9c42oO{*vuO-)?Ac&jW0v;i!4}nEDNF(4b=;4k6HED&tGo!H6>2JlNJ{ zR#Iqp{54Dm9*6JKg#|>wXWfya_DD`f|XmcpC<9!!+&H! zZMq0HIM5sT{lXy3wC)rkJi^<~2-60JiV{T0K`gcz}YGy^K#i+kT{15+9{oUMjLFCYpG;xo!umtS5<6rU^`m*C^o}==(7wt>okq z@?dfKN8MuVSgw+IwppOA#djj`W?j8pV~8Lm=i~tE@&I6bUaIp?sf#@lG)y4W=1UIAmbb z$*tut%K_Y7}(r8xC4hwMSW@iP$fIv`*j=Ka|t$s{jTcL zx3ogEyIvKiK`jJ7ZtkAkbkzd)VC!j6kCuU9$Lav_6O{_(N7y}r9NKG*!th;e9!~8E zqh-eFmLVkB-Z{i84w#=bZx2L+Ug$7-u1iFVyYD(0MGw8r|(@suel9Mq{ z7ugbEvf$$rfV@l?Q%*X+ynrb%@k8b292sJ{)U)w!7J2HO@)gTgMq*4pWXx~ZSn%pK zIrFxx?d^RXaQo(9-!|o~?=)UTe^bG%r9U{nixgYm;0yAT1zCOnP0GW?(N6g#T`Qp? zPUb+IQvNgZ>eaa$7k?W&`S-4{r9rOpZUX=P5Ow2F;OtF`xChU((RVRq;{meH>oSyG zk7FM9$NM+F>>N`DqZD8klJ62V3VyEJwvPrl@$-!+tQR$zDm;C?D)upuQ%4O>goPIjI>G}PL2n6|ZeO3M; zWVYGl9)#mI88oeNd0kTHrLe^Hsx% z@L3NVgFHR=d?w5BL%2Z&P3DH3bJ^^ZEW4QKFP(Yb(W!!HZKP+yZrj?9txts6UF;W6 z2rlDCq{TqrO`tqHiYd{H0oOa2!yZ<{5BLu`qC{(PfH^w$UYd{&wUvY8Xs1SqeDPCoZEp)*~YGO_17+lJ4F#E|b2&qZv0 z2z#CU@W?JT-@9;gm2PQS?S4(RQ5q{>D!W7Ui8`c85R729w9QIhQMAorb;j@NBW4|M zpiu0~xHE_)TBaLYn9Kr&l5#{oO5t;2%u|yFt>TMzI}NZhGmG#&{yC*N;=ek>N};Pf zT#+mMdl8TQ6=5Al`{c1(f%VJI_!6S}pRJdC=H1wAyADdzJ7snyU*v+mxb|4DJW!kg z^d(pk0B7Nle>>R$aI7By+Q=-YY^UW9npQ)*xs?FPz#L{3@W_AO5&w!R>40=9C0q6ha7qoLFQM%$$xXsk(*&YK=mRVcrl ziwCM9=N6`VWyd^JumVu5*hz#jXo4jKiCn3@))IaM=I$N!rC)GHTWzU;7ascyzH;YIeZ2@XOXMp}>Pn2r>x*M~K+N6VTj6f7^$^IByI zC@F!a`|LG{9CVXpToO(kTQX`xiOxzQM@joslr$1Xlc@yVF*$*DYc5+&66{q|$?TtL z8sM>XfXBKyDXkg4eJoS7yHLmwHR_N?E=L5CpK397BR+5N&$Rk(IOyIMyo4E!D|6HB zh>p6_A^7)Qqq)oAb6^%n@VPS!RM0$R-ikPs7lbotv(7RV8 zy39~xu3lOwdHQnXaBqB}J+q9uDgW|E8P9x!!xOj3YovvJ9qqj5e8LR*`ucj7i_`jG zQtTJ^nF=!sN$-#7+)X=D8Ix|`$3L*Mv+wUtUF^3m5ROr?J7?@C^zmoZ1G)Y@qOFH~^WcFQ8R#g&Evs9>oL=jv@NjE&U#4snkBt!L^P{^gmsP8ejb8HlKS zUnN^M$%xjQ(hluOgVrd+An~1~`AA|EqCWM>Z9w^IKt~wFB4J4YF#k zu7a^*+XW+nwLkI#N7>sq11(arVzTyR-ji|ZB7jkecZLTImdeJZ@=ElYb%k1$OtU+Q>bb}_ zYmrzs-YK-v)0V4$QF1=`7l%HG<%=qzz8s_+Szk^NY;&}L_Oj2RosrC09%ig}i*U~O zmU4n&4h35aBR0&+FH{)F-^H303{i+Se-HHvKX=vtnWDv^lXotyfls z9iR|Qq2GA-zx}6`G;{4ELxUn79j8X!UHU2C20~4sJ}Y!YCO_1xSFR`s)TBYBacRsM zkN9CgTzMM5@b^)N^!F>DvU0rz`RSrbY0>$gnKhlb#KNS+ad18Ek~fUffRgxtgeI@) zVxlLT(==BGs=Z}0|DWIVQv`6fUtV!(_U)G-9H+T9B3omY8l_np3zN3Zd-#u%gb%av z5n5j*yhjug6pi5mb$?ZoVbaSBbhu`TjF{Ed+yNj1W#(L723|4jm1`GF4D#LR<850_ z2B=b`u77F8ks_niE&+T+2_|ZpkHR4W6u75jZpqxKfB$1{*%QFNi#skhmACed1K~iv ze~u)t5U~?!v*9oA-D|m@7x&%bosiFyzj|+er;;*#=7oVUsoT!N)P(EJc~}2k!<9Nj zf`LyL3=?I)nG*)n>0@*WjoO>@DTB=`^%GH1?YMGxm;*WsbSZ3EXe!806HQ9qx9seS z0}kco_1JLNE-1);GFiE0a_Y|ce{_9$G}Qm!_BSoIqR2>DTCIarvTv0rAtgq(5F-1& z&P0|NO9*9+5|wT2`xM65Crh?r#Kd6CjBREx%se08`+n~Gcb;>eznt?q$1&scdcT(I zx?b0N!(t;q`kR4?Z}gk2$AN*3^0e2rlmE9eEfEhm5#psEQ=Y&VZM8TQiJHENwr3+i zgvC#NQWi-J8?2f3tMq<`iD+g2Z`ocD&>CEZrm@Z4x!5{KkB2NU1OQ=Sd)-@S{Hdi-VE$kWpBL4>I3v!9ViC1qvZ z1KFbi8w%?~K#TohqzE{0{|t$+J<$QyOux3@8Nfd2H2dEM?<3}b2*mg|DjtAmd-@UM zf`nSYkmK$7XWrlVxt^RYu@w)QlAaZfw40quC3!~ke2bL`D_zD*KEro(u@}6Pm^{#k zm9Iae#M-c+kUMDmC$BeweM|t{&&#w0ME(FGxN%zop2s4_7I;4C%$GULK8_T+%0T1 zOmuaplK#3?EKrk|#){QkXR)h-EPvW_Ut5CY*e9X#_WCdtKQBi60;wvn1ErxNC~!95 zUFhw*Z1=u&cHWT~hqZTfh%}rL5Gcw9D%swxh^8i0@Y5O&fOjQL-R|yozH3(yh&u%_ z{aDhddRo!^V~L9A#3?H`^HY}pA<;Y7?L}!~sbgNi%)JJ{TA=#O(cVnee+ZDAUKP;3 zvU}fTMS9Cc=|UU1r99O}z3nFZ)r2+C=#GiN8yK_p!8Q zLg8Cy7aG2N`4DH^y=!-QP@knn-sYrE?V#L}fw$n7&6ebl;tTF&4Gu3KOug~J?fXRjeZr-`=Lvokpmx3;Dyfp%r)hy?YiDIpEkZ9%)71?Yig8X1`0PbNGU|wk z=RXCdPaWeGNm6sx!d(fcUbgX^EZlkT-{;@dla>#q1uL@%$SERG!;4-<+VvQnSZ~-V zXU`!v5iuu1n{ks3zwQSHX#DnujFq%{@QQS2sWvlzPTR_O?bHh9|71{F4rMfF#T-8u z0o+i`_ds>k3nSGwp=molyHZcb=ouQ;?!9n~KHR{Fgu-vusDKuf&I`rOZ5Kr7V$_Ny zxWc)>kF@<|LI^8Xey?>i4#14H%1b3Be^SsJR=4i7^95!BSCBn=Ov;q@Ff5&BoCa=I zZ3pJxoQK8asm$(*nr*O@sp*T7^CsgnjQ~IfRFkW~T60|1W*e}T8#s>Y{qGV7`~XxO zz~o)en0@+hp0+#`sf$KP4!j`pF@{M}_?Esr&I<>aqJlT_|PZbbjy{y|}U-UznSe zxd)QS37cY0;|PP+Eq{E~nZH`+AAlzbO@?`@BFOE5-+aq*!TA+-5?O54PoCj&lGmsC zc^yU@)*I&ON`BNDn{Z(fc+r!2$h$j0hl9!^4Qg_Nq+FZZqRPr5Wr@3LKej7%JJ~|J`#bnIO{qne5kLC*kDb6bRwr=zrVB_n&NwT8_ z03CCk{vh*UG>7iraVOA%`20Uw$u0MT4u{079|L4!+7z=tS_T_$Ex`p3ub}ExKx&>T zF(*i?9J8~mbrut=0(pm-+&I)w-L=kLK10@GOmlk=?g;vaXddE2z8drUi9*uY;B6K8 z93OvE&Qn^LwiSEB-tmiY%M_PIM|1qP^(zRsk8hRtfTLhtGtFNK)4qN0@pDVUxTl64 zfhhx1IqbJpYv8_5|Mgh@l7dvxj2vZiMlDc(drAi8uhYy#QpvFjVb8zgeFz9O3eq>b zd2}6$X?XE{wj6(}Vkol0Tn??(>HXdPWrYKJe7u@|7nw_FFwlVQFBk~Uld;T5Cxw*N zR4a__o2UyOB^57WZ(eEzZi!vxP?}LZH#CHBM^nT0>C~MUWgmZ6)GwP?I;e?;|M3Wx zL;B%(N0K!}k*ze=R9AN3qV2u`+TP{K>k)lb5JtIw5wb;ICtFc~)w{@3K`(|k^8Ntj z2@t7>z(d*KA;K7VT$~QjxZrR-vAkkDav6Y0&}VG`8f*g;+5s4Tv~ow`H7QBc_kjHZ zNI-nWd*#sKH@^blNneR_CfB+Bb(@(cpKbCPpw`=O zi~>-qHS^D`2LQ%w;KDD=1QHS&Yk)$dU@JKY#DEmMV*o zN7WGEV5?}_D$eUr))KddWbPmluFt3QqtU16z}4q^VlKzTKgD;lrtrfXysH5TVyBY*D)dRCMvajT;}uyMM}f3~op zK|2ZW?Dp)(j{CG2=xo)NK)rLV#OwaM3fnVc9;-{g!l&wV zx^&|bxD}r}^)~1epIfaaCNWGuybj_cWLYWGuN!Nv2dlCQZRegx#9JSF{n88;t>=un zWkbG+QO5hKJib*3C4_1zzmcuBWPE4KIlQ&3=N#IF-EwGy_f_n`kvHqDn>j$1B5>Jb zAgt8Lb(w?8a*N5UotE)|cL=rG`Jdr8R&q56D*uVrPry1LPa**Mt&bXedTfCXN-I1ZF~$D{E8 zG`!aVhl>YV)LZ-qzy9Tw^s{Fs>6O#}&ds=DK)`8pFSp|c-{sRQ?*Pz604lY*R$8w( zR6SS${fqzPJcFiRKmD!VVSN&Ylz*n6ugOkX)iMqZ7-^fDIy>#%Q42xRu(=ZVmp5LP z3le$*Ei3A?H**=(lzsY4@MHD^_FklhwP~#RW0ul9^2sGANSV35m(9V&v*CudX7x8M zV4mq4iqnX1}%)QZNA;Z%`1Nkz8J_MyS&v!6PV4$ z|BwoUaf6u0fn)V$spMUH48d0q7httX%vYJ^|2|lpA1*q$zuey%f_x4qu6NC{x^_ej zE7CHC#OUf}zL^R#NGe}cy1$iaO z0dOn;f4LI%!NJ5M7!AOM3o9g*XivMCo*44^y5 zMN~&a0i2zU{YgYii+ZU=^)W?7Mbe6iwYBV=h_pt5As|N}JQ{W9IxV0s4O!(f{cAX* zZDLorNRoE^k8|~mtKonE9ff1AO2 zIc|Vy+0|mdMluMkcYB<Z1v5p^A&#JTcB(xQWT^WB6F#l7Xpu(n_&LvoW&ohYW;_C@ zBS$2PSMV=`c_n{I{JiK75aZI%0WH$mv+0#@9Rb&~bLRv8+WUza3xWvNmtS}nCnOw4 zx9VSY=sB%C!BuLD6t$X;`;jXLYy>^gF8T56#L3##7?gQ4e=R3$aJ*y78U_|rmk%8b-d<0DYY=k<7Q<<$L<1TnMy?=fYJyq(BOQr=ZQ38qDVPSUMQ_XVGYtWMfc09|$!V1PgXc3p+6 z0BbCgmEYmJz-o=B`44^Vj>?kb$ty;F*L`rP&z~>Tod9Hq{lTFo{^duXEr)W??7oxJ z)fD79-s%hFpk{h?KkCG3i<>1&W|RNf?^++=5^`5as{h8daL`d4AKJeBmG0d`iRP6P zptga>=d_+mK*Bas80v-&bIX~t?t)H_JA6ZUrfxG-&|`Y6P^_;q>@#DlkFpw-?oJR~ zq-Z-nDMI_Cy?-Ay(GY}QpYzkk3iS5%K9}pdh4KFCvi#=F#7?eR+^U0ef6KJ6N_qFB zQqYo@m(XQ{I}O5pJilvVcSGyz@08ywsh4$@@wp-o?$$H)J3i0Hch&yynS^sN$umZy zX{wFTpGoSbPs~*S6O|9J8sFpB`WH@rAU-V9@LuQCKdF&(c|4TX6Z`Y(>n!DpInOJd zAIX%`a8}5!2_V!(Qg2uVj)1+Ay2>nX*7;^Ul4&|m_iL3v&FuEDQ$orE=jpIF`CNbf zW|8G}JlJ|d(5}pd3#Z(8AGbwq|DBk~%a-km4SLQa{KiF3G&bt8}_XwutQ_pp8vFRo2opfyJ4jiUm#)9b+N*~(*%7% z{N|u9atz*wbRRDkH4q#)jCQC^5LirWZsM|DR#SiaC zPah>z4f7R;kB0pCc;UU*(wx4J@yohc;>gF8W?8I@)~vkuMiRBf@-tuUhBX+A2<}iN zXke#DR}*^10+%?}VlPZ-8RfQ;(TE>(IYxwUWZVAkpjb*mzTX*Wrx>`#!a%4?MOMSN zejJ;o+18Z*44CsDBN!ew>&yy=gUN&w|Cef|O4>Z_5# zd)=-O8;!a~uok}Y;-T?YqxHz?CDTKLpSpKWt5g)K4Hb2}bFt6-cz6LY2V_hW&Y1dr zcXj+%5A!ctwRG=b^7-5p;QsHk5HkP*BozTOm?t~_(=8ODf%xv4hUM$Sp;Tk7^N<$g zj3HL%M?h#(-NqCg^C!_fk1pI#$vWt4gMQq@<{@VT*%x8VuY`t8`MoPfn)_~;CXi+r z-B5jV`*u^j7A{C)>Cb#7IR``RLv3ozB7%8bW!7*<0&=qD8fS6QU+`30D4DZgATtjd zO9&4y_rBOHNg+(_YF)l;1W~ z!g-6uI3{*oX9aPHwb8$L(Ebi7AR7M6+dAwj%=*^>VTWwb0tNOd#)Phc4u~M|rOj zdt#00Zba9Jw`*nNHhxwuxzxiySC^ghlOF@NG=vY$&%1<5_%E?NW<|XAMzca|I`%UR z9$onryXgdurZkKS<>C!4rJyIf>3Bq2`C#>I^N+WE*Z)rTpY?r1mvhe53JN0kAOjmP zp_lRxX>v2)XSjhgAC^#!MuzdgAmb-S53baYpz`tu;sgK!1c11sxj?{NQ&5-z3|>0N z6Q#X|)Q+sYI&@f=evw7+=sCUQa`eB|>UGU0(9Vr}m|XnW2@cLvjpRW!HaOW=^>zH- zt05B(Q?n?>bYrzW>wn>A0EIaUoRt8b%W9qf;^$°OUk;9k9UW#FXF-Y$Sjldr3S zc7)!~>M6ihidbYnLGo=ajWCKW!V4#ip{AxX=5Ts>lMDA@nQycFJeI>TY4W{lhv68_ z^-4m6*0fhFf67usvuFC!U4cAONN$pk>T%X(?NpvFjVpr&+sOLaA26G6+#x{`YfM=t znGnd2Y*>{R_V$wz-e}|P2pAUnjHbwG8EWU?w_KX_4Gk6Ln%9ISa*FZ-F}IB`T~Qoh z+{~k1Jfvf8yL%n8FD|SqV9weBVVG#q2JB(O$vmKg2I+|eMmi${BTjC_e;|aG>rK$; zu(l@dtKyKB1&n!c5@RSKUu{zd7mlU!w2bUFZ}K1ca{wm|%Kq1J2+Rhit;4^p4PqgB z`^)9|hTwcqwTA*+%oJ?rPq6!~KvRn4#CFMyLyA$&r8LdQ9`kNS>>I z!&t|ud3Zs=LzC2&`f4)3htir2j=*#1W*9Bfs;_s8vKgh3B>zWqrmZ<=HkY|KJUcgm zH^wcP9f4!CsQQ`Z*R6KR$B;+ij*@^Afupi&!NUPWL*K*Yu@k`j_xJDXzQ2PO0rJOf zzj51L?CS?C57~4nRt1HN(-rWTyNN!r?r(QH(P#(2ffsc4_p%71SmN|Ye(#Q$6Ws#? z?twciE3Oh-Qc6vSfcZS&h#(ars`tLze_%&TCqr>hipB2s5Rtq3pA_26X8<<$;2V~6 zH~@Bk=sf&W=BW18{1*K#91qD+@0=BZQ{su9b$TvP!CK_A6!Mvv;{x~pgw-p0Z_G8U zG-Yehf3;tt&t9<|h0Xss;&BP0zFbr7S1-X@YOU9M01W0ef8%rcqe+wctcSYwJOMDNtYVhB(swm^b z5k;1}Z_n;vWaiC6t7f+6Xe4jz|rEjygSjkK@QjD@%=mQ}Lw_uDX3#9b8n|F*K5=H#!DV zU^qIssk_iFsYWEnBX~& zMsn3V1bsZ`QYJ*zdva$myCU|(2~ez!y!SfHDSBtQ*f))u_|)fvS6y6C@Z2Abz^ic7 zhkL)J0|v?~H@?-uiYyp6t@5)BJno`w?8Er35u7Q9lOIe|ZMy4TLkcxW&S6NYQtsj2z_KJ!hIs~`We$Bfmy+flRiXk3U~;XL_5hRw5;YW7R7 ztqpG1wo{OA2NUkUcl0L&+#eqo9PCMnEQH62Hosp&K(FrV5g$(7&Y@ki=g|I#?c}R} zj-3B>`cM?=3x^5sAr9@2c)smKPB=bI14MGHra^AlnRvQwW!gI@G03;7`b z{nk;7+r1Y(e*Yf7YUlmTC-w5~vg7_?`m|H%!obE{+V(F8BhD&q=|If|A8OnSn7)5% zEuPE} zXQ%em9LI~i$w-D-bs@M(YbI!uB2Mhx*w`yVymi^fwEhj@zpGb7%!FL{E)Nrw3e3rz z@iLziiL0f1tG7{iWjY*qS*q66QJ6Brf2bkJEE&5YFi7LgEZRxo%XxaLPp)b|4K&*J zK_0oC{0rTg;49ZAE>b*2ytP>yg%1@E$q(t*i9_72l4KZ%Qb-L%8@rCgdi|)|TAn_& zJCSkyTFvB*JKZ$ezFWTP`n!}Eu|VOz^kt0{8Pi5le{X~K>9chb1q~0j*TS*&Yo0eL zi+1(lJaQ^5n8F1*S2c5h+eb~h??**>tr3J~AQR&eVBERM*tI-s8y;60moN9>DLK~h ziE3D9cdGyWSFM{FFFUlDw9Hgm)t9sd_eTH3dJPj})s27DcsGZkP4%$7k??kuIfM)v~SI7ZJOeSvmz*bA3}BY-CJ> zFTSdT#PRqa97ENchdAM4Ml!2PiodK10z%-r{2&LoZf@>5@}56%;CwNoexU`>l_B;? z!5BD%sjB)g_>Ra?S6>E%AUoPctf5`t@Bg~0y zy^|LmYhrMER>>^X`bENB%)8$;|NNu<&%HZ3rox8*WIQyL>b^oWVC{xZJ^xyi$sH>a z_!PCTfQGG=TU6xv%|MtaDuF^)csciC_y>&%*auo!7DNvnSK~Y+!s@sVZ(Pv-}Q* zYci_3Z;7YxQUcHy6i<+~~vmvE(J zBbpw7d)1Az2yfcgC&I#pc(h+g4m`tW?@SB;vx>HS-_%a=^l9-6R9s#Hu$@ z{un7xlZ+Lb@3bGXDv$Z!8M6?ii)pI*}2r z+xRdUzzF2*)~ix)H!+iJ7r5XHNuIRT!B_>x97b%;Yu0g{IS_meoW)vreDp0l$zjxuqWzn#HPN|H}hV z=2qpMGN!cjdDa9z4^Jy>S#K@tMEUB8Q6?{sy`>kVdW;0)j2x{(gvI>x;+)qEz>R*Nh}Ydm-@aFOiL}ThWyasFf~svs%CvL% zCVxL!L8)JKr2=DxQSZK%g=BB4|DPXW1+rvH_-QQ

s)bX?UY+BzV{i;*#Fk9#G~# zW<1^W&9-qp{K0SH{-zLL_N?5~Z#}wXXsqz#C$L7vbb%Unl2Q}pXY|VpdA1sPV`TabBALY#spUW~|+go0jNf$7s3pYzHwHKlhAJ4$!Y{-a}mSlU*>wIUZ?s0BEG+Z+{cJ3e9n8}OjT=;rK*E*Ds z!}?4DsvV;3XENmAN!Yh~ur+a1H<|kSI%BBm2`JW0v;5KtAkP1*!^#5E{KV-CD;I#! z_p@T-13iZ8`bIa6U8@7E(w%Aq$+-&9T?LYNKyVM|4w<)3i4KLodsigXaTUfCwCmbh zw)@stmE#(UFwEMvtI~o@eYtR9{If%sb#6mgKRz(gV&-`?O17U1+-K3=OH#77zP6Xg z5EH$5m7Wre@7kBZ;+)Zo)8+^eo&SN z(#}$0zb0m163+8f-B0g+r{p1uZpbZF1tdNhYwfmq2Dkzg{$_f{@3JHKiqiGkyTtxM z#_HqbtnUNxkqiF3`Pa!=zpyx7G_i-axmWX5EE>-ECw><0ws#7%=|5l$HkjVUk(cBN3GAegU zzP3Jtc9G6N*@IJwR*B8K&T%itg2JmH?>L0mN&5|S9jDyf4jDCyLhR}X@`bMo($;97 z=(Aei$M?IGI-pgMQv1Bs7HS4;&YgsS)Ijb)5-J+%j4y%Bo;@vVC<*W=JA+2(m-x~6 z;c?!o9unRrVvRcK)WSPC%6~)^zv!qYYpc{3Hw`{7+Kvmy_UdW_7M0(@Al|;B--C=D z<#=oT1bpeorFpxXgK9jz|EBGOaMt$aI1j!V3v&NAn!#+yeMaY60SO^FBG`D#`sL)I z+})xjN`8A(%rceIhVm4WM4t8eTCOgmOoMq3p)O%WZb zB1*PtHj=KeWolqdo6{k)fj$C&&W_b_?%p85R^G!qhJEIQ1~wTIVx09iARew^jrY~* z!^GFqDqSS$0lHQ-S;&3N2CAOrHLX~G%);D>VXZ>WS*u5$j?%4VeI->$5C`3w-E1-n z0%t;61+Yh5?PT~huOvBQ6E){5KT;*ZRo#Vi!{^8kXtDiuj6FDp-{4Mt%t%9~RcPpv z14Fh89ZoM(iqaDeJ1=BRoFTrt)Xs9RarqRi*ailnbA0@U%~)Sstz0~s7U(`$^kxat zR!~#Bk$7vr(L7rImz{Qxa_0M|~My1GJaYT77jVLukCov9Pwa@wHiVTO%B{3(b$od>}f2 z98;aur^!FCvYiU4nodWquNbsn7nj>zF9R;yJ$mTd14awg=EmYXKJ)6(_X4uofoci(K&ZYZN_njS1~2cPMmH zd3`rYW8z425C6!z<78_=slato@4}T*(FQaRttiRZZK8cs8v`!8o=4bcwTc*N(4N^E zU7OX>%X_~%Bvdb>l-$<8MuCBlS520oVe=`o;vjfiUWEbV#>;WnMJXj%>pSGy*w$6U z@`t1w@|q>i6RrbxQoeld#{bwMIKPB=Np&KICymEkhmPJAJm!G7nk^?G9io<4P{T$x z;Xl4-el)^$I?wft;4zJWlwWusU{X#N4TTBvSzC?rUQ3r6gx)~1K5jxBkNn?ce*V`} zHV$~%6@A#j6Ox{G`+Ss(IQZ$AU(L;&aPi8^I+Pb&@}6psne;_2s`I$5iSp-d7_3nw z6Sd4m?a)iOL61eB?rg3xc7Tps2k2Y`hop%FPF{CwOQ#?pZu0aHMCHT`t+;u^$pZf* zP&4U9X3TXrmL9J}7Pq#AK5DT0G>+&u!|tXCRyLX|rxZ4JMm1(&*cv6G}e0h-ExlH*Z*RreCp5O2%q! z8SsM!2Hb?pTXt|CU)GD(SZ(^+z30u5r zay*^DcpZM>VqJ}BW9RT(!lohQ!EFR?T<_nG^`^gnZrSDOz={C3t>`FglBs>U|9`zBDibE{D{S4 z(5;EI6w=5OxzVkE{pVc}$67pxd*Yh*1~ zY_EF*g_zdfO z$^3bjVHb`UhwVrO+rUjC;>>RR;WYnwKwGkE8JS;X7B9!q^!9zCgPlV8CNv(O%nb*y zrIMCNf`$nCKL7s~3d@t)5I64lghK}ne>jJ_BphW5ajS%EWfn54G|!ED?*nC`GYZw0 zhRmee+=D6AUlx7F=#urcpNUz_+mf-}f4WqN?8tpAq&aR4ZHo(r=@YPa1P!kCLi)=$ z)T*a)lHt~u?#g~>iSl#}tIz@zxvnsEiaf$II^A5>Vs>IA$>4~97IOf(xnKE-v;#TG ze%8jvd29h+EI&|YGv?;my8g7+TdDfa-lDiuSyASi3G2JOPdtc)x}k;4z0&w8 z#jM%8sy?ile(uq&zd_H1LM9UfC!15sU62GQE6O6zI%o9do*22(V7uI^$qDV=t>t2N zm(L>?)8#ZoP1B$mU@;%As|h>JCxAcSz;&A#L=W+Khqd(Yq*n$damr}s%_}TI118q4 zWkh^nCRT8B4vGU;N0?X%pkvnafpMqP@KeQ0AfP*(W^q%_6{^rbdDyg8dRWd7!3K0U zOJ*=AoOge>4VdOIl=J*{7!1PA%@3Nb$`V*{9>_UmQ`@R&%N(EWfIHdUblmv-H*7~$ zL)6VILn1*_sr!j%0P~XJ+YAm05jWqAjVcJhUB1vbI%2xlMXbnzkD4A91 zV62!DN}3OYJ|FTg3Wmzr4VC(}&c@76sjq(7x4%v*{rHzTXv2aFX<4(pk?UeDnX92o zMB7fBC;2%D8{%;jrBu$UZnPHg6q7g=>8|yJ@EOuR?}8SP{pznSnEerFZr0P37uL1v zt@zioX)$$8&v$Y~7+SeMI2WoljR;aVVM&PXm(4hge=ExPOMb&R;7?M2Cw_k``2B%cT%4#gfWIDfNVzHlUW|*!iIWKq zF1~J!dg&!;CobtFJvh?!^Yy0ErNKH0Th@{11(G@@V%7%UzZ5Uk7gsxDa~y-akB=nwKdvT8f%ahBYPc&sc3odh8C5@lM$98_Eh6w1!Z5T0`iZ9flu; zZcNJloLR88zDw;hi@A{Xt( z@)17wMb3q_iTyiB&9wRLgqXfx-bf4S32JML{mVry&oA-?K7Pf7W~jscu}}9sDJQVk zSWOm#F8Ko!iA9)8DXYcesi@6?5BuKAozpRO{t$qF=x`xer5^vjlXpF3>D{VvWa(|R6)-f-ulL;A-r%0uRru)n}e;XJ}s{{(k-;{R$^HnKhP`N(BHbx ze%P3D@ZLo`Azo;S#qusiR4z%h-c@L~zy8BXSJ&U)L?H=C@(zNB@e{foi=8VKG|ePv z$~$VIX0RP_0~7D3yNO(I**b`gqGRr>AICRIOeTXO5m{IccY7OMsM&Pan&VwiiJ#&{ z`{^#mwsQQH^C7~PQeEwZW1n9t%7}+(Hal(n|HAgiZPuCwsQJB&X*IwX47Clr%cl*b z%JVZ`{&s655DRQT@Oqq=Bj?)pfm#RqL(b@zFcT$Ja-C`$vAp3I7<9?b1Oyj9?}#4n zbF~16KwjojI}TO7`sNcZA&l^RukIZgs8s*a3Gto-j>4-+BiM3M>QdBxDUREWZw#n0 z8RR{Splv03F}}31u8jHyf7V)S+{(uT;3_nDkDv2s(!JO0gQoY7Bu_!#>eeS?DJ{*M zo>~tYmiOUFC~GYmQR}=_(1L7_+9|?{w>tBl-m2}zHXFa+vO3}Zdp^rM&9;Kc_X^48 zZA}@!1$Iz?wwINTn)jAE95+a?4sZyu*<4I!7nL%jP7@RN6bTRTHT25M;vV0#=4`;lzYW)80=qJr! zo%TYj&pcw9>J?d4s+X_2Pt}axQt;fVG8)wzgUc~dK(ii&@Op2Ec2_Thh>4A$?j&CU zNt13B!ebZaTWTL4`yT}6sR8g;YmY4dOL6UU=tFX@x_fN&`hDbtS3y?$=Xi}5IpFov zf`79r#^!cqu7ESR^s`|E_QWsgp{;73jdHS?l?^OA-8tzp$+WkQBcJsl>Rq&RR(7aJ zRAFEW$_A`S4((R3Bl)v1-@hk$B6n8Swj7WK*a|xAsE(BE3w`~Qp>q|OB2@Egu=Y;@ z*Qbv(*HaoB)aTa1hmTcd$Z(YXq*;c_CTQ$W)OgHzx}B~mwt9S_nY_J?PSTWXaUp{Q zF6&$SklYKhEi}@(r#9VJmrM74_kBFwkr3H1L$s( z%AxxP5Orev_xn?Qsau)!`_nQ<8nsCta(0&IwxZ*1C#%#xKSOe{kOtEXXfl=8v@c0;G1@T?4xPmhKJ65di`ff4qn3#xIdz{FF!dhH_5m@`;SR{D!u@-{HEa z^z6q%0Onv(IgCxr6S_^9W%8BZpqGla)lLRRTFpag%f)k7v7d}6hXl-adROMw@OPDN z$k38i?m<)nYryxp?J6x$xoI459PXvM?OEpv?sY8qR8Ffkf~A<5m*3&t7jq2Pc5>Fc zPw`5j=iKzFox1^cXDP9-HMf2(>?TiKG(jF}BMm0Ei_G3u? z${Jzz49i|a^0CFHnufgI{nomPwyWjScF-50jNpllP_5%waznNwH}$&n171C`nr3r} zb=3HbD8MQ^E)Nsye@VjjQHM5&wy=dFub-0=#RHdZY(jF&Ej%etfOo|A%b-Gm6V7*h zbK7-RpM+SN8&$xcjY6$9j`|f@h=)6Op`VS;wi|d)IEOp z7UfYNSkOpylkMfm7IhA7B8)*(CoE5I@PYL5y&43kni*|#h2Q<%Ll|zWhTIE$Al5bm z-Fh_))|?*|S^1u%q@_+>x;;0TFMzyLIkE7_qL2XeueBJ$lvnpp4b7$?> zY1!p+5n&64bnrRkC*b7?RW!Zu7d7%S|5oUAdS51FR?Dtuz|iRVLPTC(QH&a);5 zUuw~iQk6MptU-J0+;KPC!=-%+>e4pV`NCI4nfW?!WKpYTzq@V^SLcG`AHNR?V;bk3 zP@}ZLCcM&Qhr=Fok{rJ5D`?~dPM#=Wz0$Fpo_O5xU7n>THO6@B`veF3s3da^w$YPw z{?H=)*dg!91#=E`Zh%J+Rg1-gAC!8keb0-!&(n8;Hg>KeoeO?DCJQ@ID1-gqjJqDE z2jz!C0UXf~(qrW%>fJ8n(k9FA6U%_-&t~vQnU^$ELSJN6yldENuYQmNz2IK;^KN#u z^#dz6_imqb#Yr3XcTcND5iB{oq}JI@uhBT?7y&bz-_`DXnD?5B!lB(ZR~!r z1}sex6J)DKaK}s2Yv6D4=uOH==pkW`pAe&Grj65|R<1w$@yhJi$Ct7k`rsU8Z~Ki8 zs{XFz)2?_>yhrXCcm(3(lD(Z(hJZk;p?XYlwSD0R;w^wHWCp8Bxk=J*DEv>Dm0ST7 zD^G_V-3}nQ+AC|#Fo1Dwj0;D-GKKi{9m{Xt{F@*A?HQs3Swt1RhoIe-@sEC*jc*0w ziSvkNh0T=bt~)2ZXmi<-I#Z8(TW5)XNsIxKY)yC_G76O}Zuo#9=0E&F9Nbm@kPs^& zCfh$JToe1}r(WZeCkyNN9Ud%Q!j3GFe_7bys+r=e5Ap2qoK#4EdeJ&`@@QO9E=B^5 zI-wL`hgjZWGWYWNzw1$8e^Ml(o|s!+7e>sM(~3Nsp(J%zvBtE)mdaC@pjNXFV}0?f zCwybS`U5}bUwUx!?yun1zB9=8lL>#T<+%(l7d}{C??lsqILtB_;D3ypasb{~X;>KF zkG$k^_yI_i@W*OZ{=2*dR5g}O&w{RKGi$&KxnKn?AT5l0!Tg#cj z?xWM+$k%x@d6^u7OJqU-8$Z4**iaMOjpQG0k%K6S@}UD8~Tm|zC$`Et^_R_ zXW4gL27|1T@$_O}N6x*p&v4shiCq7J8OuJ1#lm{Fr92#U-LP{jI0Vbs=B`MHM)hvq zgM^WJjX&?#Glz_X><|;Swu8SiZ|p(eq3m^9Jg~&#vI=zF8RPbOKYoy|!@7<}dp%MW ztb*J#KX(A>{{6-t8SD0RXBB-ErJQl5Bn-^eb}jxf^e9eaM#6x%*MB82%A$V zyi}RWd@@v9szRCh!o3&meKXf|L+pQJ`T`KsXWJcL9K^JuI1tlDT>M|4sr{)O@B^N( zcU0zmjFwWGDuyBR1jE=+pB*ADa6K9G42k^Dq@C3nD5C5>!eQMN|T)NRbYTC>>D<(m{Ifgeua+AlT>- zngY^8???%e8j;?c5T%FS^UdKd-aPO3$8VjrT#}Q_oY}MczV@8mSz0tM^TQ=~VVt5^ zR(NN3%JhevRFMOYqGL`aMEoWIEqMbU-P0YY5gonI_q$}iw%@O*@#~A0G}UvHl?#pE z2^l+MsB#G>?FsVATGdsz-7jyJgtrpSYjpcBeCv1fqSB|O6xm)++2SjfG00q3 zuRxt>Pq*oT-PSe~xh_MFf8hDj0Eu03`(}*mOmzcodBuRD%!?JrE1|1##5|J4*2@ab zg4ejW3AHnM$eO`yx_NS$4mt5TO@(6%^X%N09HgiF3cy+ruG+7Y(&b+MK04ed*Kp-| z_r!+XijMoPlmKqd=Q>>8>X(?MGug4Qna;gsBN1ll#;lZ|wnWOs3c9cM zrc54RAGzt#_c5RRN?A4vmq?BN><&(kU3(rPMMULNxd3Ej>Ca*h7P!*J*8HYA>q45G z@XzE8X|b^Bo*yIw&$K3OGA+!J`%5h48?Ui5TmGgh)cAOka}SD;(-1~$%Mk+aCeLhg zv}lnAZWep5fV@U;u;rI!cT)^$zDb{SEV?S^#Skc9yyUR5T3p{LwkjKKT*xSqT-l|C zRpqTXmF%z~>UAjEL<{=HpH6>csS1JP8>;!o*F7{t`tz!&(a&mae5J(bdC_NJ{Rc8x zgl!kf$I3%|MMF7O6*0xXB$LpuxGuPA+s3q-XOlqTEbx&$RS@mF=hXrhJvmoa1619E zKJ{Z>XvWGs{Fr*pIe9{oO`;7a&TukxEq_cPHgl=c7BQ-{VRl%mN+A~;T-(&hOgHf- zMUe>sOn8;+_W&j%;5804V*ytYQ`QZ;6=hf55Sy}1t>?tHmJKdVA75EyoI8bf(z@eu zUNe2XQkmktdNS%7GTCy_MB7V!cD8H1&(#R);*jJt(S>SkqYRzFB6EZQ72;77aa|{E%8)Ba`k=Iak6NOMW#j4X{b-yw1BWOmK26 z|L`$`7X22V)dwGb3A?T0AGLw9xNY+))$;0uJ{1Qs7L`b2P-7;6M0AU4XwNz(c(#mQ zr15>Q=SEt*HbOe5M-L-3Y+?Lws*fchyO7}|mmsURr?r8lv zMH8P7q6j{bj5ZjCwk|$4cob+n+_i zoXvA}RyVs@(8dut5wY|2>0DUiF#L`KtwXS)*xF+^=OOMjf!rP7%1_uR|4c^uFgg4T zyy}qs^oLHCk|Fk&gd0NJW}@Xy*9Yd|rJ^X84e2sZVK8n{(`;OYXV)|1ot!lZ>Cg;3 ze%1mD5Q4YdFt(A_FtkW*n2uKSyEEM{4%Fx~N^~A!q$J?6KQlDs{}dM1)rLFATBqP-77rW ze|U5}OKGdqsm6(ACla)x)g`%{4+E2AfsG?bR=cA*eCWd!0^5fFY^izk{ zhA@+c*!6ag<6ZAf_5A8S7_EfU6Y>KlS+?LttU61p!XI(JL7W+m+}!vmSVZfdD-rgF z7~^RXC|->^~%=%+?e44_m%L{!R+@EnA z*`=j}-O>RV5x38Vy>;tr!#om;%Qx-om*1y>$lg`Eq8dv%Z;I(Qa^;J$nDb`)&PP92 zHQm*ruWZSNkhy4i<9W8%EY@YSB%F6DdD?WDDj*W4*Yvmwc^!n{&xe zSC;o(n0#ESD-{cK`cWq~6lt2Jh#ld#z_{A`mT_;wwcJwE58H2E!GE%yT9|2NS9Aw24g>?TZ%=+Je{h zf&kG_rp-`1ETlD|?!LzrL_KR9wKvxRpvf^N5o)lar+xU>25~B4r&pAR+F8s8W$?D| z0~6FOk6FLV5Q-|$|;Q7vQofF|0 zFC#(!)se03CDbF&0^TkKP2>%gtSaZA>2dqSHD1@0lz3-ufYYMeq&e5~b92aboY%f# zYD>jGUIZrVi)-F^L+sY1g~gGV@?@^+^)oxmYbLmgPI8kn5qSSoy(I3t^Feh_Q$o6v zq(gElN)j}^YTr;x-JOo+wRhEL<|EaThByN!WE|^KKL;s3vDks577p6`xk^8~re|AJ zL387$eLZ2dyR4YnlF+m^!rX^Lp@t!f?lUKCB6Pb+Ni@5+w>Ls2 z`#Mg@3;z;)$Cj!2^~Uop@>(a^*VaH$c51sSw5t4BkK_EI+OHD#8Zlmh32Q^BJm*Ja zP##`W4~FRa@!EgfYkWoRrDkHi#9GS`C7k|2l(`B`Wo1>5?SSdlnulP~0*)qI4WXw@ zWTiSe&T#t@2yZb)Smd2l0aDPCEZ8toMTL$IE%Y+JZc|dShLzT77|*!zmT|UMiDW37 z{)NO{Q0?4Zh~Ao)Nq30zXhMAU0?9dKC*yZ+^7k$GziAEe zl@jR|{zgBw5zY{o(o5P|uc+|F5izJAd82Oqb0%3z=oNnJf-0 z)cjh1dUj1Qs>5_gY(T&5z$y4ETw>hZGu$yWFDoNDP}Gn5b)#&@QaM!p9wF+-qk|x@ z6@taJ&%ipm1#e(TY}>ZNU6Yjxd^quP;L9ndn_sshX-@dev;z>Rojk*)9aUhi+_EBD z)BV^Fy?MqdU4ff=Vf35JvWAGmu0yz53sJ6+_MujMW9v9;BK{dVTbjLd$@67s$UQ!I z_q#v2#)7K=L~dRaw3pjKaLYrq!?h~Ve|DrSTJf(9aOyor1FvL*mi zy}t#+I$zR=5CTPt^=Ib_I)1Evp0ZkyM~?%>VN73$PgH6wl+`IBYcg z1hS$>*GJdEX~qi?7rRq!K^4QXrihQMfYfg=l(cqROT8#DrAQDBJ?3E5@Y|!(a;$O9 zv)IR2R$skpcd9Zm=b9|x2@+wdF3RK4*KuDnNGq{E)9`cYkMOGZuRL~UMZOGW29rm2 zZVvfgVK%@VNLe$?jp@Mnht{-oYlqEwP0hNl&Cykoli-wrBLMtyLJRi${mt!RA#4BP zNbyyGV9GG8*!f!RQ>)C52|d=N9rnh$V)ym^^`It4GNNSpcHoh$)l(ncMvI2Gtpi80 z-ENC|^%>lvhmBVwJO&cn!iyzCbFN*={jz{AV?*p-wJiJs;$;(7XO&BPIdn^lFT8mn z3EY=s46S6wi65pO1Zm#3u%s9Q*++PtvV{H;*hUfI^(Hr;x*CCu=QZ=*0dQ-s3`WM; zATRRZgPJi^_~2>h?1-?ZHi0?owam3eMMHc=A+4QvQdiza5U~&oEnH1py%^GT#}2PaXjLz}4s?JUzSAA`p&1i${f2XZkw-^I$~5;xKwEYY;^_o4erhx2VTmDM0mOXqVjyre-bFYiPJz3#VWii}d zMX2QBpkc_Nny5*$+{bUPp+LdpqsJ8yC7T*> zpX@BQX;HXou-Rer4n%9uKn&XV@LWIWj|z(=a9l~$_8jdEOeHS4 zW{}DvLyfC;553X&uI|#t3g0w>4k_MUAu(whNmacs~X*$-S{;VAEqaRCPmU z{dM5>T2yEbfkSz>xV?T-{!DOq&N?Z>!JX=7CotoF>?|Adebyg1uOQ~$3rTB6RO~v! z8>LK3;#bvlT(e3pVj`IoSJw?!Qi#F6lUJU%&i`oQf}gr&PGk{r`>Aj#0TmSJ(W;oZ zQ4-E~4|%7k#VePa$%ULLCr4ggTlA9}RK&BCs;$=vkD#^$=B(V_YrA}oEjoTws)SOR z(l0Pa0PlEY2Ro}{Wr!^lgNPBvs>j`Oq$@u`PIbSESOt2HxeB2Gg z;PrXD=SZ~x7yHF}OOPrjF1cjr9Q&xMIX8|^a;35DYkCEtX7~72-0cj2M{zGUQ$z5m zmBUvM0NTjc@nUGOW=hu43N1y222fQGP$r=(bpfqGMVij)tccB@v@rf5eGWo03660Q zu;q<9+p}3|21C0syWD>^{6umq;&YZ#Ccx{2oNCI+VKZ*&L~qYg}F9LFM3Z8axCt`v{(n?lY$6wOkbTx zg&5iBKl%>v~Ha0gI{k5K5M40uVj3PIG-K#ntBC_06-F0pJycaG-bJn0|P zCSlxXs_SUyRoryc=XQUbj*k3VQJ~(ugIm2*A(ES+WmLj_WHvU6K|#PG?mkj(%3=pp zUY_Uy?8P}@(l_h&OZ-^lcD~ghGvJ<+aiDB&C^xZX`=gGg_RZCt#)F8t2KJRcp1m75UbaoeUxJ z<;;9$&vy!jKOvNbG}gPx79qIaBhEdwM=v6}Hn&+3T~Rr2;^@zM$S}}@v<4o$=fcUG zTfrD7RbVQkoAG3RXS|2F(nAct$yP<&eE7VBX36a3W>jxR!?s44EQQV9>6%2%|<2sd8?t3y_ea;ht1Y$xmTk4(^yW zgQr9P(+G-z^yddr`+Jc7WcH@j!&d7n>-2Z&B*fm@O)6O|Z(8MS9|pz1M3;fiqN`ry z?oAhKK0T6Om2h}#*&Fq`-xD5AL&MQI*PO#EMN2XU`h9U0Wp*3w^58H%)Z;$#cLT+f zI=91hqg3h;FW|!+kt#8PU(Y6x_^-aAR;tSC zO#&4hcjt2%KRD}#)}y)+aDv$Mr}s<1i{PUc>cM!2Zayu=z>A5i_mxQF9c6jgaP7F_ z7nvOdqFa+M&XyPxw%8>aY8>%!YDFH`sm(wV7`rx|+k2k6Hd#PMxSizvj=!dUlBu@;>2%ktn}$ROnAJ4B6g=6h{bXpkw#_-JZhsviEkA`*lRc zc6iNk4$odqAFlAAtpPYjPE%lu1l~t`k*;doacdh7ue6R}R!21E+I|F0fI0vnL9hnp z<<0(O7ycpl^o3exCo3p@0x#`Zu$hXmm0$fp{gYvL6TeVbTXk}NXMWaU2w^-i5vHEy zGI1NI_KNL8bTB-|Q&Uiz+uqb|j;))wm#_s~G2T(A?P`?_HLIX;aMc;|k1FkBeMxqU zpQud5#DaHdMEJhF@4?R_NMdJT8f#5zJZ_f}d~`_J=;``5KE!e4@A8m3ka&xo&_S&E8qRd$f ztM4wk4?n)Tc1zjdSAeBz-I6<#@e%Qf@!JG`dh_OD9pQi!5BUae{>>xo#3TbX6cc-7 z>cCRnpZwru08X7bX6(%iDPxoK-qJdE&^Xh{aJq{y_G!hx-<_ib5(cR`>~3n zEzd@erH#I`6eFxIx`XhG?w~3#Shrs{DQ&%KJ3#lgLQ%GUt+F#i&F$lE4dS>+M#UUC zbVg5odc$0#a=nr?tXAt_Un1Ppn!V~sZ-5Om?;P^jOwLFjeLB4o#^^QXOXD-ZF+I3^ zu}t)6UtX>nLEF-xi0RV zC{gWz9}}X2qMf+^DXaX_1v*Wrhf41*wD(aG@z)SexAd^)HxOp83SBmO+O++1F=M#u z4K0^C{W;>M=N?FA^f%rse5f3%Ddema3U&JROrj@I?Ijrg+`XJ&?9hf@&sP?`rB1=P zGPNovA<3Ye9tC3w){m5mpsloeJ{*VcTb)La;X|JIt|ULNeCWm~G- z(sOx3RhpmfSefIz(U|H|=7rd&(t(cKw5L`g`4zboh*!e`Y3DRQEjehL3@*U z1ZUGEZ&=8vt*~cIYgXIRShT!bWS*MH!GGI{9UMS{ka_0>RupnECa3_+>CSBXCke8s zR_&B52=X>nI)*9eYJH}?x4ens>6k+_1NH$WsvfsaFjl%CE!JN_0E4ruS2N#FZy3j% z8}iC7rumiS20vT5y0TKjmHrb?;Ms~Cswa@VD$Hxf@U*adNDEzOU!3p#+1-1{4Cel; z(qRXPm$+vsV!N+{^(znqJ+;H=sh1XQBU3uX6pZmIL*1uSDiFb;!D6(@&%LiPr1m0T zBBiMWvl3kdxaEZy7)HjBW3h`UTg_w#%sSB!Wvi_&9Ne9bn5WC;P(VAopTMjw#S8wA zGbMcmwAh^?95;UnFgQxg-gxa8i!3FvHh6j z2*Nuq#q+Jv9|CfR`mKsduu*BDS=G+s?5a(>;>E;UY&6U5ttD4XxD_$&+c9R@aqVPR z;|CG9@V-)In1GWr=lL`&O&WM*k}aWH11G^KB$?8gX{;P=o3`el%|z`c6TVe^G1*ly z9KF={kx^~pnOcc}c3zRP`fBmVaGMEzgCZ*}PMy;YPqbOX)GK9uUZ`gFgcj{?jCJNX zb)u!KkK1b730Gci8C_a`R*ncRs4o_64bQ03n^nJcv(>FsGlNvlWo^NY?xxG;RlwuF zdO(_!A&1qw@kFT3CSY!<{S*hxM~3% zu1xHhAY8+|A=$$Ng*}%h%&l-d$(29{c{h8F$lk6OLF$CYIFJ_#7{9SE^i%;yg^&MB z2O!RbI-<1r7dJ=Vt(AJhE_eYnN!Wd)b_J_647az`NV%25!fezM^R7IkbF2Ci`Xsf+ zj_y{@X4LeRqs*kGaWuCm4fpnn;^^q`09-Um_&O>sbuDwS)R9ISBOrJJj=lemoxp(M zo?UQ9fiq385i{6pYN=Ndo=4EQpA26v>;!<5@^d)PtI~{HEaa9W4N-B(Cn^*?C3c*e!R# z<8l1ME}WXYRv8sSyTbGgE80$9ze*IYjlH^Qv+#Cdqa3Bzmw#PvEMUAApJx=7-|kim zpDQ-+(M@W(Asn|;?$*39e~mb7LFfL@SpYg>mtKo8Op&+C0ln7M7k#WFsL;`@2$&~T zY8x43<|z7q=&dZY`WH91zs@t|Uh0u5FE4&C&=V!tGP?t4)LU>pdt}0PW&$wV*>bN- z-OFUG(&X=Q0WKB6-C`G_Atq!TAOH$?O?F){0`ul*gEmrmN^C@9-lc3rzrXOm)%stl zkXf0E)YIBCD_2Kq5_g}>4yTVCFSr&=uPyT-z(>2)p4kRm`E#P{OL>X`Zu7+@z4J)7 z<(cz^TckNXPz@o9-l5P8$Sgb6|8f{jjy82ecp7!v<%qz}z@m38(;YcMcK|*+xgsx7 zJqA=gE&7_q1UR_TT914!=Hg2K*t$w!tnCL4ZYUJDTE@tRMc>kE5f?ath*N)uXyusQ z5LjI-?>QpUi2?wf)~0ZdrBxJ`FzQ%|aY1l5p>efMBC|nHSpKJ5c`6W;xI|((yN*MF z=||9%@Tgs^pk^bXY89!Ot7Ce|Zclo?8y&vtzAoXi_-@#r0Yg?5CNDlm14chh~RLY|41?V?m9S zaxeFC3r6m>0+$NnlD;j;PEm*>)cBsZdf`dbzo{W5Zm?g|6LMGu=%Y7O!lcL5^l6B5 zo2DVS`^b)A@}@@Exl@{y1C+v)W+bKvK|?$68v5{>k0Dq6zNjSoeu2v?45_ceZ(dyH zUrW9GAn15{?=D)zb$JBUOh`Jlwgmc8@)^Gw=GE-7l>9B z-3pYdD9;bS9^8I`T_Y@fZ6nts4`4@;_&CV;D|;~IZWIUqM}UnX0d`+S3V~x4a_HUy zz$x<1o?bDeLckbS&0mfO`~^~d!n!cO)=UqwW5;tr>u|>X^c6>dfiW%?mwsy!b`qHD zz4_Zr$!TNW#?ONqoqKPb`EmjoLZ`%y;>6a49xtn zlf)^GGhC-^ML~LVR(Kv;znDKJtoMC-X=*H2VAdn*wTEX&=Mcmc>~C&tGoa0S^NIAv z?b@o>gQe~jG)+dUBo`h>E#W3Gsu?3YxxdTH8;|{E*dW2>p#uR9;uoD1Rz6Y;$GK$PK}VL~tPv2&p0br188*TS|c^LPiOOZPS zy8f}-if$POkNcJY;f(uHm3;yjj=2wfkI^z#in83vN1y0|!1ziR;ws9OUyhQx()pjY32#cqjx!p`n5Nug3U=+hW&A z5!#d3Zj>l@>mH4XIQdZAO6U*U*e)a&yxXuH1tjeTJ2d)V&Xr&2fts-z*U7t95I3H_ zb_@*=i)eL((@7xyqZxK5{{CLynTLP%h+IYJW068X{N4fKkT3Rm6USa-*fi#@EvPZM zQ7(`z_i(YXB16*yW;h_Kp~f|@udN9WzaHC?o*9l?v5hQh;ArYXi3W3g&U0+!Sl9Gp zBLC~?9XwVC)W5-nJ;%6%!8^qeD{v2nw)5-bbw7{BZ3=p4!2WvdoEKE(;8-@meEzHR zEPOTeX7U8UoAM;5f$F9}yWT|^RQ3{5diuB7xQxe`;AeXG0m-ZtvN7&1!PubXBWF5a zTsJ?L%Cl-b$IlDI=o@RUfAAXzyicEol^lnv-W}ZDDdh1ir8BVUHAJr6UuNp-%gaiv zfS)?vsGiRqxvF!_SQ4puOwo1qZBw8lYN9t+ATHc0P0n07x;Qp`Lom-#KGe-aJBEI~ zTwaqU<>@>sMB_1RP5Id+mTaZ>I;vTjYnT3r*l80v(2{VAPG)akm8xPg`zyxZIr&VX zyou#eq1I=2O@|__uA&J{QOs? z9yojf=qHP2Us`DGE0QB?xyT@`gs7Zf+>ZjO$EcV_E}jXTegR!jv^IKo!48~oML)$4 zcw^HX4(&)6c?A0_TqLTz)T}Ye?1S38y;|B5UuS*j;j&^bg{_|6Tnp{lG)@lwIeTkw7sbR1h%BS_co z@(TjcGf&pm`84{a#)2je0ta2H-NY&E7guXF`55+X+)iUBF_ai;nqfCV5^gaKpOt3$ zTNZ%OU4*6;Wsrd=&~eCnxau~W97dFXBf8XD5o8s8snF$eoLjefNi@{(^^1a}n320* z?R77Co+{rS!s~UhFVTAF{d&PN8qGnI6`+Hfa(Z(N?dUeu=RDeSX8lHirKwhCaOL~c zJwHlko^ibSU@7~bx#JB1_iNO@(S{^WkiuK+AyN+50-7CHN9y!%`tSZJNgO!bczRt? zA1lE91vI$%A43~HERY91hW$3r&t6t~bj3luGvD-)#(aBU@!donUdwdT_CDTr_@70_ z-vSD#Uhdjnu*rQG^r@&L`l>1`N~3amy2FNaU-Jx)alLzV6y;(gV&k>oo@4$MgBGSi zu}q6<8}HZXGmx>)3r4h3MX;0UT4XMR1D)c@?5cB(Dsx@5<{I>bBp9uT1M7;04t z$h?*u#-N@8e?98lRkILKRa~Tv4FA>}{_W?Pa2KuxO=$PN#vKa8>F470kKLnmpkd6Y zV?npXHVbRqBM@9&O0IE;&mbvpV-PI*DVj^xQd0l{VBC&3zkuMzx_1Ii$%pO#iAqCDi(sXP#V!wJT3HTFA*SgKr8w@`*a#+E)ja` zKk^0wIw#1D$y$$Of>=NOSaHs^Kyf!jR?IQA(AYDu^utm-|4@35=<%N5FqkeviT-DZwQ>(*0N*q#ei*A1DT~bMVej|- z6EQuc`lfH&6+?JGf8oUMZjo(xmXZoz!9(!#mp2Tc_}}+%44||YiRUg9g{ng4 z;rvyE({I|%Rn0pF*kAX`{Elu=9CSI0=mwNvx&Hzvs)I$vyLU8^bHM#3eIU%*5J96E zpoE$@=VO4Gi*Jnh&+uUJWwb!~sBO!gf~frTFvi2uEP_?9?WJZy)=wDPsBeZ3G|^-ifC_E-0VH~N4vy$zB3>9GUoQ3!W4f;JQ!6ykM9kCX&j^L=qg z(Na9(Zx1G&ICGoi-sIeHea)$(Hefo5<$lfjRw-od@DgO~$jefN5 z^V9oRdKr{2XC4~+6hI@Iuy{LPZJ{FX4xd2rP+MG5eNIV}ML7p9_ODAXhzALWsu*eK zQ`oAw5mhd6Ig6>4xTBx?pvj{z8;b8AxB|+it)18Pgg)kEU+5+3_y)F%oy;S8T8$;n zdK-~njsC@;FNg;qzg1p@^62XC3rAOevy^gOkj>^K$NFM60BM#=l%N>)4{voq`bkg} zg<$?KyX|R31h{j`%?x?&AKL9P%=gED1e|Qre=r9Bs2Xt;xHHkKbZGws*P#hSY!QtA zXM#J>1RkzPru`GpLK7rBfdBU~_nuhX5ZnpfP=dE1GoQ<^4A3IfJD(o_o?^YiEo7@x|hnJWS_)98kGU!wy`_dhx>nU7a9iW zv;Y160q-e@Y4L%o`|gM{1IEcn`ha`9PdR~A##=#Cw|vo0*#86Qkc4zLDWQGSLL?Di zf$kVf$?m%ouK*3x5U3xyZy0C=YS0R%9I*c*v=;!0I+YSn?jszu(eWr~ZFqFozLCV4 zLG38R=R+Os|5?EtH1*(hgX;Z1P=SVFy)W{ACxF;Anh?aY|NE5eKv7pKo^FEB>;G^Z z^k45%kmp8*r@WE+*Dmd`M<_(;#Cy#9zYVS6zo!4+p~NWxW~|3s@s#ZUodDPfWKl}6 z-~I_Ezyx8J??D=QZ^D0{H+}-FK%+Gn^Y15nJ3|SX8VSV{O;A4mzd!LVh2U!4DIvK1 z6F`(+N>%#Pvi;AmOqc;*caf6#_l@)YAkY#{Uq95m|L1%En2-25aHnhM`p*OV?s!82 zjySV-h(TZUS9cBZ@Z;$CO8ccRHd9~zdC2%IX#YBY>tpw?h8H|sdcu9ieSEgJ&L!{z zz7U4YeVpJe2<4S2*#p`3+cm(kgAntVsnoyR|AS;`7^zpT|34Fe)$qlg7umm=feK6Hvj&_5$c`J zQiP-TPp}A{b5c3s#Q*BOi{Ax-_-2kvTkl(2{2^fCXI`-Tu}S>Vo`31RK@fRlY^i)Z z_#eFr{;Lj#QD^;ci1If{G9hVn^$+?l8-qO1P4Io@8Q&bHee;6Jew_vEB7`&jd2v5Y zBY?S+^ZApk`1w4>bT6;tFehPtOEWbJKrvmf^ z6F2ZUmV}RI_V2J0q+?pyU;N(pF{G1Jpso3s66*7h+(A$JwFd0F?Wic3>Hnz74u3#7 z%i`NN_7UQHFnIp{GomW{6$ucs6iE4)Dsoc(mwCPY!l8VG+uU8iqw%{`a;e8cz=-#Fyjn9@7^=qmE$S)Tv4Eh6Ohc-8#4(yQ= z1tA|IVI%EAQT4=yv(v*E0U>t*ORLd|FZ|r93^LzprMbgJK*$5G^+Bbx@Cx58pDlw+ zWnx!uWZg^*89rFY3u%l)sVAuR$#Xt87=CqvFar($?B0mjJmENxkylsLrC7HWLv#qu0{-K)fqSKSIg_%w5;=xOdznE ze(~dPFYkK;mAftsZQrb~oB#*q4Y7VuijrBKAh`*Ok6kN>f2-JlWq{K(5J`1HlgyOfXiQ8Ax?K#G~!v0fpSYe zp0!4w?&~vk&>kZ5!ePJ06=VV27woKnJOe$k85M{Zse`ob`~)Au&`T|kvrRtiWs3%A ztkS>z<>X^1;c@^H7{?CV0Up(EednSR@NSKz758ytybmP*@$CMFkZMkS`PJOcZ!>WA z!yPZr8wglwNv#;%*S-4SMd4!6z1U7->^&t^4e*GDH@k(GREB$*!wiYB%yYtdgm*2+3Svap8BRwq5~yrSaU8?)Aq0% zqHuK$E391 zs_4wRQ$#@d9C40g<)XU=0HBnii@gRNn=}dIJ=64R^rCX)zsWM7xqjWf31B%_5ikU- z%#Z#7ke@L%?A0<>1Yh8*qx-m0yaI9zig@Z>A?`m5K}I$j%Ij(p)z+lsku&d}l#aeD zx?(;5k=Y-&@Pe}Y9?2i%w!wugo(rHUz>gUpv8LQ{b&^6NFuVV6g0!1J1vNBFazm6x z=Zb&F0B(}}j8?w}<<1YS9U>gU1Z)eB5EICv7eTQHd>4w#a|B>$s-V9M<0t-6e}8!m z`EW>28W2F<8|X>dk*ceRR7uXo4M zmN#$S`^VaTzYgs~Kd2=uLTsKvQU{~m()Hgjo-iojJ3@##5A9q&y*EGLC%Rc|56%}S zW#iYUs(YUDFHZXYi0Z5#ZX>~(kfn#xOz2vc11PPt+&=s5=Bxxzs3NI6><~v6g5m*b zyyS?pkw_^wtv`)!2(Iu_)3v`?$Gi2BZjHk%c3tY1zV=H)Z1MJjXzY@>&^P5v2&cDn z)wlP%J^%GSfD&A&KTHQBMu6NivmEr6kY(miuc(EZst=y((+l~RiJ^=y;FCa(j*X>z zaoR>u6I;8^Fi_O5!>j*CQ3KQ2!_pB+2MI|3X&3t%i+%%JZPut|aDrX{&5ZWBo?*=9 zouAa!Dx=jamna5k2526i-!sOD=iRlwzvH5EQgI77>C{PSdR#T|c8{2AD_6|)gr06_HO9I|l4 z&CF{3HWo?`0R5ftUn@MDpbFT^Tq?|V|0GZ~xCN8Mv!7I_@{bY2`E8Li* z!_?g;e_I|j$U3IBLc~bS=}8>@)o~aw@zn#$`<;dkNS-%^j58o=yr2W6BXI6rG~RK( zPZ#?u^pA&o@g9(&lqH~f`Jw@4=NS6DxMd8;a~Z^+*dG|>i$MxPj4}fvkhm-)ULtBR z^d(!rQY3lK#V<(LD7Nq&p8BHpZ8ZO}HK2snYHLtpaxuy_erZYb?FMZ*a#P0(zWF@q z2l~=NC@kpkK1a>rG@v%~&-*zipmlOyTv+||lH*AQ(_9vCu#gCLm`e#>G?oakL`Nvp zxiA4V?)SfD2Us+(R72O!I}a|G%8ED|T9}ypDiTpFsbzF)*q}}A!9W|s|^q<$K1=XS&xG{)2%;srWcq)A3!T{B=%-#i8IiazY{8sSJ|xF=gwrqPi{AMs$f9y zdy9eE@4pLsAua+1l*BxI-=zcIez9N4R{)|R{g7LCcSS!dNL*WsCo=foZU9h6$g3_X zyIhh`?A18_NPei1cXXp>$&V;oQkHaCSm!EX9pQdMEfwb4pbubIi|FKJdaxXggCSY_ zC&WX(=`8|v`X1Wxr1ukKe*GYLWm8!{2YKUapu?@&v*?!Fms-Gqf>4qfNOT-!Zv^mt zDKRv$s~a~>{YxY*Ps1jTJ6`w2nR!l_*)sa#U`a_3?CTF-e^-WbEFCB$LzV^}C7arWH$@-Ubzpf|=!d)Dgqf{1`035%Sv5y;tBYvv_Em**aM3tpH&4EpIPHop--A9! z*m`jY8H)8IgLE%fElSXw5uBy4R&Xf7FQ~+fBy_ z;`aC@CdD+jnCh|wE z8{wSlsyA`(*&-8PUYGq85iG3T8qjoigyQy`=X^UFGt1meL9NSBhIH2uAGcK{$YG`Lsyv|&1Z(a0aH`Ko*S~GM_oZ?9OH;(eQ|)|*u}s85$8Q} z43_?dP-3V3rH;D9gc)j-N1Y^~ta-f@To6u6I%R6|AdHvg9xhE&eHS9>XqR3X@ z^O6p@)ll)=sadkkx2g2*4CiTn%kzud4;4RU$7XGeyJ473^WJMKu4zqq+=?-HB$PE8 zdi3b#V99Fx+z&r{E8UA~%36)1tx+QQ%>JC-fS}JA1i@n;WyRMl@Ui_)^PF62;k<(c z`KJi~yD$wZ7>5V;7>t8cj{MJJp=IA3r8tbHQ)oQ}EjX9TS67)iO&>etgX1_33t{Gp zjuU*mGB#B(J@}*gc%gnQhj^*ytjyx#QR-q0OeTNA7N8E-?+s_7CPXWj6S7i>Q?YF$ftx*#gSEFT#i85<)+|x!! z_MJ?vs)kJUqD$&51R80Yeh=G%<*l*N+JH$HBP5>0q*ze&G{=2$T~8-gpvyqL*aX{O zY*-e}XSsO`sK2ZxU zWEX!?{8^Q9^;loxbay*7v$NHPcsI=vbN0dp$v6;bH2J3uZVmMxNvWJBr3{pBP*Y`Dl2Kd0$M|3kn}WeAl~#(i`}2$xG#+2>F|G|v z3WyyFH$#Ub95qs8a$KqiE303kimUx4EgjO2!FEKHzIdHRsLF(Sf~`Jy`1D`@#G8Tr zf0|^B2b2*n6p!G!xz+hRSggVw{BCyU{6eT+X3I$!!Yo+I9>3#Gs#yzTceblB%CfJtmY^Ilu{;;IJF@9&$x=~0o!98ktySb{&>*VC*O6A! zkmGUhUeeC?P4(4;tC>%N_{{!fNUx zOxhuE>G5;cYQf_f4qh}Y1J0mH9s}RJUz{}95Z*_jzJ1vlrs)X4~=M+Lg}yrJzsPEc$N0e4Oaz zJo3o7xuOw)7-?6pg3^q}mYY>Z0#yZ!d?oZ8YSFmwYbt%@p+TWXPc3@1yS>J{X03kb zhDKs?mC`Az@+q52%rzRO-PGfFSLeNLsyLrzyvAv%jCq!RevNw`EOi66BxM~XHT#K{WoBg_b?0RWISr@Hf?WLc(zcaFDs?S zt^aJwN>Cove#K`BYZNgY&(G(yu zDH>6*xj^jeZ1Qxe&aqNa`QSE(Q``zY8+N#8A!4;tDGCVcax{mW`{qXYz>xE>jDM?z znUxcv*Cw^^3ysd>Et(yFe8qg_>>VaE6Twx(G#93FogZ5?gaEu z1wq`ECDCl14Ea2xQwgsmv-5P~zVV)GBvxoR;w9xpKqWZbWC68%{?DP8F()NheFZyk&%DAz{ zBhAK91IOssy>+n^sSbhr*pwcyX_lCqG+@*E%kjFmvXGieM0mk}rmZIt8KXw`R3b7;ZsHd! zn_WZZb6HJReJr^kQzon;`B|@kOR3M#>k9mX&r2j@ls4*m7sAxEkbou zpAMn=EMoV+no+j~uoinv3^T-UhI9+niE#$Nfq2tOiT0t_Y*3goQNL$l7SUWRLLM`A z$LTK)TqCTF;NhjkX4uc4JJWt@%au~o!yD+5W~x{U#XO%>>w8}7I2nER)ybpi^DI&3 ziuUUM$tcS3Y|T)OW~7deRl4Iq1Xjex6 zTMd?Ev6ev8SkLJ5vhzHB19WbfC7on;#Z!3|Iw?WfaIyEJ#m!9Pn&m-T9BU+o>G^@) z_PMc*JgVOVsHNCR4$i_|8)CZ57-V}SR>n8qXbj}i`y8CPvV4BdeR_0BFBcQ%-z93F zb`v>Zv4mIXS~}mdC10%{chWDdvHAQH#ML;h&`rQLk8_vN|7Dw}bbxIxhzLD^*oM>Q z`ctexRVxQ9TdPF#rbFxj-sVZX}YJaaey89AXKPW1PkW?d6%7^l+(eWy>!3rb5Eze0oVan&z;!;YfG*7u%! zp2j%w7`NJ$Wxlull&p8kk>HzbBXh}k;0fDx4*1pokFfU+Ycgy4#}#GKAjK5|Qe|BW zArx17Ne~uM0jYsdLs99y*C?2vpkSj22nM7jl+b%OqSAZsLg>9Fg#2#yecpZa_s93W zxUN`$bD#U1IWwP``OKXCrD2DGlD_+T=%jJS6s-z@v#$Vcf@>T#q!or%9u z(ln^Bm$H>si=)g}?jO^2YK5j%W~7D%0LhLo~>| z&RzovUC{0O>I=>4mwNX)A&)Q~y|t{=`Dm}xFEMY0zlwEOy^ID`9^=Iqn_I?o6zGz} zm@0_9I^nD2d!l5};3vCOc0*@YL;XTsRJkF06}{Hpoe3S>V-kRniep`?flMW_B3%N(Pccsl^pJqO#OkMYdG zX|b_VoK7xKm>pVb!Y;uXO1hbFp#=-;3vvN*G|T_6E(dY7mrxxDohixaxdZL14fl}R zs!Eh8p3GqKEFR7ONYI-f_2Y^NRZw%*@*nrBi3yLotj zFL0sJ-#3r;ZcTe+tr+)+GYU8tQzptbOfIwj z3$6iygorB>k^VNj$L^)UJ>>>#-~3=Xy9ri@>%9jzSD5|k9!#wCPo>qG!EL6@Mw5f5 z-X29R7bLHJn^DgRm4LTqLe5K>J*uqJZ;qY|B{h6lirjoCV#s2w-4{)nmX~CVq^Qt8 zcd1r@>em}3q5(h*zOFb|_e@(*J-`_Cf*`-1iWq-daW;HG-v4exI5yrO{cUiW;t$39 zd`d}AKYqKd7NX-<_@(xcjkaw%PaFtUreC2=N^39s_lKu1%&@h9+IClhWro5UpNWq6 zUTIDja&tB7z+{-%H_he320@IeDBqgE>2^>7i(7sePj3o zV~ATRw$fZ1Op4uD&zLG%&k8}rWQG*y%|*XRR8coi^Mh|3yrTmBf?>Xmjc$x~Y4%%O zP>;k?r>$F-!p)f-AjbA|3ns>5Q_;?&c^Z5+R867$qT#o_!3%+J_B5(djPoFH-yD*d zZmH{eGT9NnlES+nh-V*l>TB4nv6|YqAG-;XRJX(XjX5`eifGyQE6nRT#T*hnM6AVo z@oVXK)_C+ST2TkY8TbavWEKn?rn?`utkC>A>2H)b8_&RV%0L4ojaZfnI)~<$CJ5WW zOS~K>H0sBVi>Nx>#sDT0WFCwfc0U?lw_2L=_+#{#R_C@EjY{0_Mi4xv5g1Jtj_#Ypf7B7m$voju!5F!)lZPZ`E#?!GA%kMcu6tIuf<`HSpa!%~A`gQG z@2gtF@(xbWOlpY_zQ+9s(0Y=NwK+F}QLo9su_8p{d4Dgt<_RaYt9o7-Fa6%siK28_ z5Su#8nSu}M9NH>FXB2`gQv?fdIA!`@jKsuO-XB;V#;b_sh|3@>87Fw9gK`>4>y%JT zWbh>p3vKv#@zn3y#&&%DMmt?T%>F$~!q4PA)@L3lVLstC`y}Fw2h?c7PMyR^`PH{z z*@;Ty!`d@Zl>!b^e%;XlTGaari+-GZBFKA!(T!81uVQZ=DvkE_$$diaRN1%qpxzrT z(>{K`$ry=4GqzWP9TsMK3-M=Cf+!V#zUT?y!5gZEepyePIE*%U`B`Y`$>IctdS>H%X5$N+e;yE&CdijOSo1barjNrf6ER^iB zwWUo_LFnRqZfHMN#r!@g)#ZJJ6ks{Kx^Mg&_d3>28jmdtOg5Lf06FBYdmKtKBc-+p z&WV6I4f|ZHLI4bCMY28vZuaQM7t6b#YMZsD+83pkrZW#aGNHYjsyNo#X%VpVomQ_Z z2Tux#Gm-+NkDl0xKz``=MKOd)z-=?#$m(a^Oz~5(HROwN~XgFoHO(j6C z&qS16(TJnYKuX)r@;VrL)?+eslySy(prsZ*iH(-WUtN;k!0kU?rDZ*rI~#yjLS$e1zoV1Y?WB|icDxOqt&^Ci{3kj zD@?S!e7LVT!TPmjL1A1R@KL)R^^5Dk4rn_D`KrGG75M$nSEnNl^}4#>5s*-QpohxK z&>L+i?y}63KzGesm|C5I(EvR6i^SS(!~NGTnNG=teqRj`6#Kjyub#L*kXW}C%0%Q< z57(^m%IX-apvRY!PZ;hXv>!mmn zS)8-yCW_dh1p6$lHdLGoh%9YVVx#ZXv1>d}nV(8)rJ?lVozPgvN;*O)3tqYA)d3FF zJ~yE4r^T76&c?}VO6WwzxHL)eXGY0L@;A9K7nzFQX>$M|6je3mqj3u7u*Kf?>>mOk zurKHPs5&x}2oDZej{0EIdk;r2(U<7b}G&!_wU-7Tf0k=5(Aqe*x(jxhkfNg#mjUxM! zAYkz%*FD84LA*{cMHKc)Y+K7!u>A*1e)0eI0?_VoyfeaVNaIHBb=vl)F(|Q!5ZLwm za6P$rm-3?-sy^B0$H&IGpvO5UP^uMQmPpN9aCCpj-CK1z>JGfWDnJT-)lzGa^2-QOG2Jdbrr1HSfJ3dFSlsBk%6im9~X`5BtGu>E`M4H3=GKi@NsSM?i1&RZq`{!G}5f7UE=Op zJU1lkauKx@+TY}S?_!rq`#clD=`VZ1Zrr05bzba7f&?~1Fd<;XNAmMp0!`u4QI6q5 z@^3)0Bc#d3D~M6-?f#C!2S8oDHI>fa7f<_?&)Aa2bqUE=bJ6ra$)107kn&4rc4+@T zGN6UBz__t_+SUtl| zXzux#sIrVuc43d|iePBh;Cy0)K?x=lflnWtIK}GM@VV-iJC!Ttsfl?uN9k zo56Bzr_Z;_uss4p(i9&M4aT#DrPO`FU)A{FTJR7d zAO{bNSoB;$GyY3&3ci;^0!s^DTG~sg8t!*qrOA}ftKMeuegCwwX&I$@!7;0?vyLIC z26d=|YjWahMTN-dP+)@(2b zShm0Nwj+KqY8-%3`vX?>PdrTcqh&#+YdzX_$f04{j4`s7<>kQg$gG{~+pQhmfxJho zH+en`#)tHBwSdednauK7cDlHh+-F}i{MjZNYeBC>K)*DkS_~ZefRr|!?dVgKtncEi zurky1C^q@jr8u86xU9KLEhW>m@ByhGmI2ohEbw8clWD=suISCZ7JKH?(+AU%y$|yA zga=FmXvCaXQUcbVLlH3{d1dPXgC@=Av7&t?I`!FB=e9(B6%$-nZ3y!bged{^qA?+~ z*Pk2>ghDn(-{M2e?6xLnO^*%*2458}ENZcME{Xj0mSuB^ocvw9j9`6(6Dnr>s4uB& z@YW$k`wjkv58&#s9!EDBXfGYZ60PB)Z%k0o<%3KYYUJx1q3V6dA5p@bq+=C0? z$Fn9r^f2*Z)!ijLUyb)x!k9uCt>DiolaQuVcy@sM&agLz^X<3c3fe;4xoLlGE+lD1 zo_QmZU%SxXtNNPr*TM2l8Q*;5tcwRtz<98(%yQn6<%xVV1iB?kJ#1RCb#8C*W4(iT zNwK_tytG^k?6>NcmMc&|9uzF{@}hqvkMWoE6J4r$sS>Vy=L8+E~D!D9TSklylzBL8H} z^g%2NzGoI97p&x&(lfNyyluO4$Owj-%=*N-4%-xp``Sg3o|qA3b1{}T(5{HNYoVm? z<=rB`pmJRRl}GF^^Bkk{%z6X@YC_Bsmy>Evg@6&b0ZTe_jPg!#4!e*+{GVjom}{W4 zFO@k5hX%S?kUu80&)3}qU&?k6U$&iHEC~@PE6&HNAmYLOjzpY5%IKC^VUh*Brg{S6 zLZLbfR5fhr?YzRCIM-rQN9(=I?38znoVy4MK7fsGG>?Q+XXm zeKspYA0L2KNK1kWUiC^utMg4EzNYd0)(v^6`{t;&f-Hyp2Z5eO4`=xl`g3-A1_<-P z=T}?o&{kt^X5;S9Mx#C9@e8w~xY2`%A6v40c9ipE^4^-J|sF0@1on zPV7|q_WUVRW_HAH4&K8YF574msA_wvVdKZnuHNS%|c=zd!WO(|rb-o{sn0)wyJ2v=|{Yf8W|89c* z&0s}=yg-XBMl_@1+}X~gsCaeC2E9PTY;UCTP0(p#zl~K z5?xCAaPD+u6LlT(vsh`W0AbR*eqFK;fZ(a6&LuAB9G=xAK@{l}Ntj-YzN?DE^GF=L z8LahUMS-H{bU8zA+O};ry={Ll(}8}i_{Ojb!Jqw?=(25}0|JWwJ%ecPr^IiA)JD}V zY3%Eb;7$ovB(JVMpaMZ76QXRUrDyQ1{7I;67XEIGT$(e!cd1x>Qx%Ty-BR_;kk=z- zO{uAyh?l%)<7XXKLFGwh$~u`u&R z>2KX}nRrio&mK~6X}QBl-C=+7yvdn6O+EpFB6j@Q>1*@_!}C=(dv6&N7r0JG@qG^` z0PROpWCHRXVXc!NVcSETW$Zt&Tb*fhnW8!SQDxT;51gpNnU`do>a6=5sEP$sVK){8z8C09l!cZQfvA+sWqOYbezMHnvF()BU7?GAtjq~<7Y$J(Ou+`{j-s9?De zhp!z8GP+M5V+Ra$rpWw7TLNtdBFgvXE*6U-fc94ln?fN|EsJjK2_h~tbIw*bOyGfZ z5O_w23DK9fnAYhP93?QiMjmJGz}R$c+U2D%)a)m5yn_t3jIkZe^hgjlbh0W==&43V z1<^l^8K|Y>Kf9qbP(~%zBdx~YmJYTGGIMqYlfuGtL!rf9cRb5xN)b{5%;HqXKT|q4 zX|wIxO&Gkjh-6Ki$5dwWMcP5GWZfuxg@4`pksir76ghgcw)Gk(b?L8 zvJV82Gp%f9a5}alv{L(}-5)zSGt2|xe6F(XhaRF&^8Bm0flkHUOLVAye#5R`>nGBhrZN3x10 zp=E1{Nwui$p-)s;wS^OPo0e}WEu-mpbjWe&5Yp)(!#@+zH??(jpemG~;&l7|**&TYQ0daG? zGvsn_5oGbb#ZJl7>vnis#B3rfI8Ll3@9}-KA=R>L&gKIsH|=J!O&0|G*3uo4MEjg_ zxncTqVv+jKiL5N*i3V%TlLZ05@MnEgJ_APNXGV}JfE2QhF1tX}$)~g1Ge7)+gV~Jx z5M?IxI5b?R;pTRMB02MfFzbL}j@btg*fWn(Xws*OYx^LBmOyG&h0hWnxfpL%?Cj2k zsbRR?F)8iZp07v^!zR?fg2oRI`gLPft>^_Ej0*P=Pp}!{{hl#a?(hm_%`F~mhR``K zF%zdq6Y*j<7$#bt>w~^sK{!@T(S6y??;Ga`+65C-w=fgcfuadRbsg>Oigl)i6ZMH* zv6~~!_f2GAzhR8rNwI5R>6Zhk(|c-2__n2|jftgn?~T)uRl>tWH4DU(38f zjC$;u5k2n5`0!Xry5R2#0s(FO5t%C8Z)E&ydWQk;gxLk$c3HdL(U2_NnJzm$BF&**~rnm1T;@Ci?v*Z8R39h*O_ zl2^!GEbd?^I0@ey55<5!G!I1a%T1DbSZcE5k$e7jr=-hLu`E%hVt<#q6}-?onV~Y? z$)OjW5qf__xH?i*hb6O;#1x4eI81B5O9m&*AwR&(8>ZK_8r6~TLCd8W%O8OA3bP=o`RPDB^X+^Z#Dwzx2#wCse9f**Z zcoXm~1385I=`OO^=K<9t4>==g6?C8Y+eD-K2~tMVZYr}H-A z>PYlvsl18d&MOEw7&ATTzo2o1y{*utW2uMRQgYqO5k?x=rPD-yAd9_**e4RDUE{Db z=T>=EwYPj?0)i4yIr(KWLm@yO4oE`IA4*E5)p87A1~#CDmd+SP$F!R9=4b)0F>~|h zA>8Rr@xoJHglV>d`?KG93GpE_xjdtMO~^!!@2la#$`mM`Ie*= zqV^AasL^p9#*@>k0}U%%djRbVbw;RZBX>Ir{3)ZlB&cciGDdAGWUJ5INBk)j-#a7n zFql!#NX+6+UIzmKY4)&bEY<^FGV&_75^DtqVU2RljS5kNV!^P5y+oCiz9r}~pI1L6 zT=eNu8M~OD%eqi?TgUsU_p5civq$4F0()u1NCRy#z^>c^#cw=K%crXDJ&)2Dxz=Fz zWjB1K)=(J;PvKr=mKhW6$O@x7u<6NBdYx}4I7jFI?I;hp`c zp+jpxwK?TSR7o?pPp?QlMJalvL$;P%^%6Y4i}o96k!i8&;0Et7noE zvB*{yOoFL^JsO5AaQjP=MJCIijcsMZmim1Kjm2C8>Gs}U)Xq;S)zOQwbl~~JVCxTu zJKBg_*sY-lv^JvEweThFrUHBd)Q^y=2VC?y-sFtVqrG5kmwCGa#14vet~pXCxue8o zAMK#&7V3I2**l!&b67jkwrh@bftI7!;}azWqtU)BkKS-Uz!r{z7j|Z5Tflc9@vYPn zq|~?zZ-_J<`(nT-Ivtu)cJFIcL*RXK3SkfkCj<9ij~R$wR7Bu1Unx~5D^R+3Ws;2i zQl1`3y4sGkzPED!Of#=}QzhQ?8_#`j4TB9X!F;TyR-@jC^4fEI!ztO(4f{YA1m03N zH$D#CRhLY!zBd8jT7=fcHP?EE>zHy}HW~q4tVst8(*5FgB3To*BuBMD;Ig%V)JZ?= zNJ_}Y-#{Q9cv1YY6b$f5VT2uxOH)U>-WAwT)_m$)k*$N51DR$c6Z)i7b0aAPR4n;s z8@&|{xeQ3cMWcXY=IH;*#}fz$VDp|&F~1BT`5BHg4XHqNLX;Ztn&N6p8yma1yo{dK z&$ath@k!a+2N2|oaFv}HdoVO(kGD35HLl+XYVs=9B7bnRpoxij4rxG?_3Y0HU6I0$I>Bx9Y(Vj&-c1?g%A9ji4-68sY~FraMNdwJmX5utlr0@w+96{8jAr zjJKb6Rkw-yy~%V$DZ8V@zQ+S+KX|{vT9KTMRT(RwZU{AB=+}l@$olpJyjD*rFdVh| zZhq!NM7^WS_s-j%=0%jq{J`A0WFY15ue*ZoG08%n+fz;#066?tH0pllu)3~fqa~R7BR?t!Rr;z>LGL+$VwAtAw99@hExzS z!1-w~@~~33-|Dl!fMKrzLolUQJz!J;ckE8q{f|Pxh4l}2IGI8dQgrvQ zg15V+6J*dh4BRo&IazUH=-}g}Ndsb*WqPml=PyQ3)*A8NiG!k*yM2UM@0tjClfT@d zoOixK^E45HO(kZGYVsd0<0m?4wLEgOwQ8_TIWEeb2K|>0`Ct zMUz}-wvPCu~uriiaCP8PN0!Uj(87`9Ds#l%vVG99OY zYLmGR9v6KaYV=3yN>KtC*AkoLw$_(LjhC3**SCt2-isDJPsZrUu%Mt z>K*Oy?m1mI0KD~d@w>!$wA{cgeo+y(J4|PJ1PXyjbLYWg-4 zRYSC6q-~>Wb1VeG;W;VeF#hhGqQ|#A&s=qOLnnlZ=MA=C_`$WvD>@A;Z{=Yt`Drty zDTKckdP1)Iz)a}fYQe@S`RDXGb+~_%4n8Es=3CUhJ1q#hoY7(DiLW-$pDbx>JcZ`J z86~q66!h^SPUnmj$K;o$5`3c;=N(d1^V;AM&T{re+Xe0f{DA@OeewCIYr!aJ300-Z z{i-OCq+DfJ=arUPIxMbfzeeEcOfed~)mw*la5Lug09nj*YQ{VhG0eMI2uDyP2%}AA>vA0CefYVlX{PXr)?eGTXj!lM{(RBiG!9-b0?%l|fJx)<1wW1~^ z!IO;fq0O%?#F6hYw@!wtxiJ1BJ_>+BsI2;w1XNjS3Y}z(tO>3VpX7wLC+i}g8@O_S z9!^&`2q6emMUxB z2(iKvR@V7CR4VwSpplHU>Gcf%-)9ac8Yxq{Sb3PUU_$MW2|Lw1hrGs|j82uN->?~S zf&;k|2K$Ubeg$9CAh7_2`F%B>{Tg2wgu7W!m;?2;9KQ1Rl%;T=0Y2!3NVcSt-7Vr9 zKKEU=jT?3^t@kh%ypd^GLZ8kDobXe`l?2FNky4mk9*9h>OmUtHq-_mIn}el_o$Nfx z!pPzig`YEscN<>g+q$*(=nRAm{?+7OX?a8bW`5>C4xgtPOL)lVP&d(PyJ(*8HwC~* zap7%K`>n@|G@YCUt`pM{#<`g%^sVd+^{;1mzDOLGAvoXZh3aDrDLD~g<;2|nfs(Hd zJRtviEGuj7pcHd{xH{wBf~EVyCg;>_YBZoA8mtQ(?$Q1V!&}Z|bSPtdNiYQtK0w2N z8LJsN@A_zO9Re?QZg^YzQ9_y@>d@Y!H@{U**)B!KRkoJrMt!du1RL591wowb9=R!!01N>v4c0%Ppvg+$zdr_<+9< zdlI-l$ckD5p5$bH$RTUb3eYJI9q_ff+?Ox6m1Lf{wKawDJOhNG5o)Y5qp6MEep#9; z+Dl92?@qQ6X{`wFNr{!$813eqMja}GnL5w`x@bU?9PYOwUtm+?zs;(6+^o$&iJ^>k(urWTkOCQ$uqH#bSR(o^6*hefh z$LkNwXI=x&qwFDy=vIIkO=pbUpylHp%Q$}%$UroEE$(-Pc3k6+!#?e+k!r%fQjI4p zWp`vQ8Rg54HM#eUMi2&EzEb}d|Gu*Q&E^|9Y&9_yG;KGQBghH$8LfzM%C9huWQ?>z zxx?j;jD1o7s`Y3MNNcI!z;Xi@3?89p8WYPUvXME>MY%_E=b?sl9=FrKV1!42O86_R zoB!CNJOMBQfH|ZqjeAyFbVlk@69a;(op0%xv0PttR&a_}R2aIKlCOec|88HW5i432 zkw{YAB}CNAKGi2?SzXakf+3dd1{*ph*@n&~(t#!%c*)397^3-VK||=9N6ZWqP>O(= z-a78Y-DYGCT4~~8(}gcns+Bk4&`&G~x$a=!ewEI4JK1lY(fC3W(N1oA>h@AFEtqku z0CU8#ELx&`3AkF?qYR@A`mCZ)Eba=8nNpk%0jOACX66Z+qB&yy9N6n$pOZ&l3?S?X zIF}h9;8jeqq`ouZaC*gpUX;+zjY_GacmkJI{Tk+ z>_rqr%-hC>*OadHxA+&4`6+ChP9sz8uH_-xDe|i#$KD1f=>&yt82(&v=O z=G?l2;$nv!VXSuF>bEM>cgwkFj`#yR z1D$o>O_HmXuNcDeiTaq085U%rtk2LE1jelayRg&KaUAu1#WF(BGxAK%7+`KmkkQ|Q zE*_m7xE5mYZ~+;zAheAlMDJAsaiOzd8rkLw{T2~<*PL?99r)MZ!=C~CJ2TcMZrxnQZDz2i>L0P`TRh7@q7bQe$(7~Vsehw7jVgt?x%CR{P~({+&b$$N zO}`&IGU_*SK4DM3AS2CePX3+9D0;ahBr$GfZf6M`GA?W)`Uxf(w4j)%+F>MYqO6dR zAtWu8WUI)OR64-5frmKo;ovM6E*dd8m?C=Y_d5$p9S!ZdjGQ3P28+F$p8>b>WjeI_ ztv0jgJRMLEfbP^jqh2JWHL_t$c)ClGJ0Es0pK5!Xh11h`FMhu{?IZN~%&r&D*yG>y z)nq9fG_^tIFXk=8BM#)7@wT=shMlZsAq4}H=d!UreM!$YA^jZvqQ#wew*sTm7id0f ztLCVhLED%FP!PgAQ^x?|ACMxf^RfNhNT1J3Dl8QbXN!jy{BE@Nv&mhQ^mqF6*T<-nrIoW9@BMiKnJAq^M0xoh4{ z*p17#31&=zr0PTuzMBXt1qgQN=6k1{1VtNyed4U$(OLsD9N^ zHA*%2Z{_BNVU?YWbu@6V^3)=gD6f3^1}c({K>Hl~3$mPF7j!$MjJ>n)175BU$tsx0 z+2Q<&OvVI!9xrk4aW&#RL{QFt0>+f`n5u8(rWih(RIxSc!B{5c5&R{PkCLSOir|I- z$R<|}l=JVwm87M#qtN1*4k^qe=!f_6QF6}PB@Prh&>Re<$bkhb`eUz_SsHFssI}}& zi0D@chLW;_cI1BccNPr*e#3onXeQDps$pYjG;15N1UxZp#7v{{u039spj^^Fl!@$Y(*zVCsKnLXm!gt7PzOVn1#flET? zkWNI)0x2hw4I_}>5_xg)}5<>3ZUY)7P9Q3g<>p;QBFv3P9x zn)l>?Px{Hzi}!&=Srrq$_7qr@JCnPoctIr-bLWeoK>guOPADaoP0%>@gg|WQ4;Mzd zw&MC^$k#d{EE?_wR7@<$UFQ-vPo4;h?}!uQ{bL(I_g$6k#|;A5=m}_#ecILmM!N!@ zFEGd#N#g-8E#gZI5L=z+DY|UyOeMkv& zE$=XDlk?jMs!9oz2omUs@0v5rvuGjT+%xh(AD~g%>(mY)EJe(+D<}a9N8h&=$i~ut z6Y=H+YG2clY(KBi)t4s#JX~SGcl{XD{0#(0Qy}16Dbv0N&9(w$XLpPMqxdjGK9^zW zr2h4KKy1z%NrT7v^=G$cfSv-YW>jII83wAKsFPDt(l>X{Z23`p@(TL(=OBuy)?*72L10l(d|OAi57=N**xOHFbg zAZ~=)9Z~<(Br}g*<3U{Xqn2Jsq8uWhJ+F1tw4tVR+p286#gUcrOs{KNse973vK%)e79l+nisk&KblD;i_Kf&4A(&MJJhZn1_gu>DjH z?8zCqm4(!-GjFNXBq9y(%d7spedT)OyBxs<{;?SVL?(!ie;vrD< zSN0O%R0qJL$=;!)jrju6pM9a;Pe9y6%L2>_0_Ju7E+7%8y4vgBdEfA?!zMkR(AZuU z5usz)^g>11^O}QSZwT*Z@dQu`m?rpU82h)ggTv<5_0qP4SsqcdPeMA53?FI?3+B}o z-15>ZfO6MHu4lkUy;t5eJM9VX5X!LQ{t9=Ifakgr~<=c~6miK}Mys$W(mCCaH2(`q^ z?+kAM%IW_s&^S8D49>j*^c&zSRHHMr_?>&CKFTHs=*L*x-*r`wnFL1Jrh42R4yi>N zoRMJw=pV6TI)_Q9JBm5ZHvW#^{-y34aa{Kefh+t|_kG2!%#mo_6>zq}`pj9sRt%%; zf^nWDW@tro_wKV&t0}-a z{ZaLZ@3_>2{WJ&k3!uT`ZEkd~Kl{%@USz1X)&-!9xVfavdFUhdm>uCzQlUrp#a}k} zI4CA+#u#<1Cb<*qYjg_i(q(tYK5jKw$BT$~eJiAIO1k~&`5bd?z2O=U_#L%roZIiI zHj~IkQy5hUtA){J@J1r;E_RDbT1<6Ubjvb5D=`{{(naWb*?@M z9IJ>>^9IUvCo{@)3XoS#Dvuw+uKZDqv63oK`J~+1J;2jE*)}(cqrp1Y>}=GjpEu?E zabijOp8T<%ziKvF%NQ99oYsuL9g8j9v4E#}PeQ;`pEUJ2Y+P^@xaDhAV2d0^?z_o2{U7ZCVV$?E_5HNfBAB7pElp@m!&FED$z z)mwXy5m1;Hi(y2M9X)>B0Vpc~b0`+7D+$K-L{~?#?J=!IZJfX0*@n4450)}_kUgidd=Ps)cOFjvRA}}#DAi#-_JfUFPX4zm|BXE!3!y97d4=^uA zxsJaM10TIXvu^LpfxRyZRy+dSx-@I|QTi=uO(uZQH@Ob9!~d@zsCgfFuW$ZivJQY4 zC^G=$4@7|-T|1Q2`i6A(Nc6gWwz+fnx$bl0&BvQL>1lbGatektkm;8kKG|U1o~lO& znj!ta4cWe>`{e5s4c#p(El1tHTd&3W-%Gar_1voVT{9D4`(g0?!{We8GoEicnky+m z%L}~gTOx~v(lS!z(hzVnZNjF>8&H=?toi?_;r~p%{u98HPI?#jGVIvdsRqiN3BICwl#8@_+U>RGD@})%S$LUd-8EMV=d|{wo?d zH}2_1|C%LWxwBFM#lg8TbiCcKgxPCbTZ6iiTkFP$9FP5Eyf8AUf7C92uJQG@&7c@9 z2syWtsw| zQC$Cu*PH>~NtGc8wtk>9k-}C(iyl>d8Rnqn32nah zvHtu&hMn^(E4V3cT5+zHulXbs=JbCLZYNn8VEtY}@-3kQUIr+Hl62n|ocvjIdMkm^ zb5TN3#|Z1X*hiqNlWNV;S2oOWHwOe9>DnanpSP0mW~BVq$QQ}7JHZ+iK>|&H{!cdt z&_=_2k+INWi7e@a_l{jUS$GxVY|s;MiyzSUF@&hik0LgcPo^*h325$gguze+b}46f zCz!Q%neXrxwf=tv$5@UJn1Wo*K5N-oJ)lKalW;3jz_5y^;h7II~Q07QAn(cI+H9 z-pl;&*g!idE#NdcEEc%}vSb;=AcQm!r%aatXZ9`oC622b@gh2nCz&`M+y&i$uXR}b zbjm3pnJx&G225csorBli5io_%k-vcYtBHUqd{AZ0cdQ#;am@p3LGkH4ny|YPDXN7x z`t-XY#}j(l>?`bCb(qw9WE8-PUPI=Vj{|9O6l zZD;e~)j5pAURj8+@L6zplI7<8uQZA7rR+ZA*0bzTk8A&~Iui z?DnABBN#GCh1{C`9}KMQf0`#731#~sQNs>p4QtZ)d0{2wj`^o#@0qM)R_a;1X&Dtf{;GyIl5YW>e1c$E*1B6a zdw>hnh)6lpfM22{5MXwOXOqYNL89#3M1VnUq*psz`@>f5%%rYF6z=Lw@hj*A@=F-? z$(|}ZfW06g0+Oc0!PdjpDJ`AXfclOgb_@T!%m(^^`c9xMA6p^rl*hHd8In>3-nL$E zeZoh^$MT;AyCa<_i^*^FdBYVUSVfY2BN$z4IsE705)BN|0&)YL zIzgHL0XWuxnK0Ph!o@3euz?2a>P98#M;_G{6R_&@g{QZDooOFTwa^{?6J^akJOakN z+DuNrbm=+s!f3MngTkshoTA&^)*@cFEbTn}4!U*f*VO_R?}f!=IQ$T|G(W%2>oQv2 zRkQhBByH`_VSshqIQc76HA)$1&_mrr)At^b#cTIdM=CsR3tQQK+1lgnkO+8!rk0^S z$HvA&n)nrb%Hu2o?6*JsZORV4&3*X44#7Xa)_2}-e;r4SLm$LFAZx>E#es|2nQtD3 z{3KZavt3WRm=Cw$^djc4IM~6Dx2x}MlP?Be>jIySJpUW~7q5KhbS-sktrF<>g+5W` zyXGTay{@vuid8yB`ae%q;u~W?VBlEY(Nf)2+l^7rj*4bMw4$*<3)|^P8(kaTpQZbH zRsCKaJ$P@j;nc)htf=XzEo|$3;ab>nH^pC$K|BO-u*W>%`weVwdT70wu-Q`QkyvQ- zEK?|{AgkaPwf@e0@*4c$d6oB4?3(Y&aLKA?3e{|A?^gKFvpk-(Cg&4g8JBV+RIrrX zSVzcikqeEo^KBCe#3cA4c}9Mby&*>6`zaM7MtS=oaiXSkwTH9falwLKzpOg&fO>=T z<$fEfRFZhe>!ek#+I7YcB^@@-*tWd8rL=Ry!UfIzx#Kep67XK6aBSBrl`-bx>#`!bdX#6 zi}>_q31(q*ai1a0(uOyLn<+f{{M6xGbQ}uYCvdOaKzd?!K@>w~Xrmg2#Nf3xH}@5b zM+pBQsF8oqW#Ma-}45T$g@s~?5A=VU01VfhbWUdpr*D=HO#zHn3t2ajX$tv(^ z$(&xXEB{`fvnoN|Sv6}zn>2>8F5n%uChfdpUeSZB{-lr^QwhJj@_URl9A*)ZWy>(d z9HJYUOk?8PUYy|^Nfy|%*QvO~;B%JA+e(jF@&+NQAd4cXxZ^04sn6?5j5ZR)owe{vg zy1b$@#-WZDUd2dSv(jizGp{isa~BE8I@>7M)JIxZB}g0d);&jbzQ z4t2!9XtTiN#chdu1`QW~f0a(0`86|XLrBq)xs5wKq4Vpb-+FhKqF2Dp1Wffu6~GWA z7N?!sZ&=5C!sQM5@48s#uh{_JHvd(qSZfw$W&+dRP9=tsO^BR8z`7nE1&`F{Yi@(s ziYf@nlaSn^!A@m`0|IGc0vEoD1TC~6ubPB$vF_0q@4}|5ae7CYfuQaz^*9i06d@j@ zuC4!H0=-t`6)Kvw4helupFYc+t|3-68>-E9mUg2uD6%Lggx5|KpEfWwq|$^`TEH(l z7V6GA5u3uk&JaBE;}UvXTl+e~t80@^HO!}n_Vg~DH$8h+JWceF>eqP@Lw#r!Uzioz ztL{GLLacd8YHEdezyjm%=H^yR1)8I(xCh@=b!&ua7LwqcuAgw~46hqC zV`N;BnoOdbaz7lY{1i+Ev^v+;0IkjlnbZ>%(bFWa+N6`|J!!e1KecCh zVT8A+{=Jdp<7JUZP83emRd=I&T)wo+XslJ~%b@Ch=EdK9qjc4B|6Y3lGA;G(+qbhK z%!`PlNlePGXX?jvcS$|}FG=1izYH3=9Ws*Fa$O~vEIH3>J#lqQbuV;dq-x|OMH(QU zD#1e;`HZ{f5ur4MvW_2Ahepd z{IWI4m{CbTS)##?@3dPjH_!LuFPA*YDjo#ENoO+d*-yUarayV|m@{IgWxssHZM(5~5!r7gie$cdOMQK3yy4HSEQzRg27Nvw0r=&( zBQeEa;R8bMVHu2=ZQITrY6+vYq8^O}{c^6Hs!7m0v4lsnRhZeCJbk4t*V8KZ zSPk&Y)sHQ4(i{N%a#09xk9hgWXaDWo1J%91z!rR>Y||ZkmMnhfR^VT@?s;Bb0U@+| zov&b$joh(u{#DxrvwjL%!pXEBnmh_lvX{cg!{;Ir_MEE{Pvz6BZBEoCnc9%Gnn2Ps zcMq(=qIBizA1p1T#f9v6^H?=b(%|;euQd5uI&yc_)%Du{&(AmcaTvxt!BCcl*Ace& zfsxDj9i4qJ4odR`PxhG}^!s_vqgU&W0WHxJjRVR@gEplj^VC7_x43E;*25~_jBV<= z+1d)~L(*DIQTU2OFN0N0k@U5t9{Xxw;k4b5jJaS5gS9kji{2DRlTu)v|@RUh1RNqz$NKzubc-%1|7tq-S7YVAFFv zTOXzbXe!58b$87<8TlPQPMJXCYy)bjg{fh?%aR6%C|mUSrI4wQUK-V%0SR*!V%Zf} zmVi6N>Br0z3=IuUwACb;wuSX-O#k??B#hlZg)!x8N&G=5pS$V0c(!+$LwUY@#NLLg17BLZvfsl%P` zMdqLFE`bZDtOGoyk=25G@c~?p zKST})=zDi7W<1d7r@71g-Dk9*T}YoQHQIO9qGNPyCqc5cTDL|b{d7QU0;@PlE|T7y zakM=ZP+ay~Sn~!#Sd7gV)I^6sJ`sWAUj(NS4<_9ifRyN(AeIhFum_faC|^>mX8Ox6 zw(z4@@;QK}{R^!3A!SOGihf}Wj@-UNUY>963;Kn3yK+6N(|-X~P={Wxere>{=}|H> z^)Q7Z$^))qnZ( z{CuimNAUd=YCMkg@eKX>W0tY9w1R-qtV}yVvADW^+HPDlJyF@2funVVfxOQ;?#cAx zQjgu-kA|?Q$PGUu5pxoW9hQP7za2P1Lp%ctt^4870MDSfD)B+4#fnxnzOG9D)LY%& zZ^kojq0?`po!$ez{{6~?FHDxQN#^g23*W%IvUVqfDSFTR@DBIp`r>XFg+g=uR`zq! z_2xwLQx}qZSwTSm)>;Q4NRGX6i>6>bSL7;wQx>Tg*zujo4BF*wpjitG!a*6$M<&%K z(VqqqM})$VHf}odXkp;#(tsv+(BPax;$iPUu-id_ydwud%5ny*a1c&NZ&{$wkV5}b zrYSvsir)x`QW@=pKrD&M11}2rRvOi5m$3vwmJm4AQGq~kU1I3V7;rz|dt;&HkHdO- zr3yQE#ZK8m+pxhSz90{it!=83S)gsFwaIUy&zgsu;o@Y&77k*-o98BZJnATkN0p^A z^L*lkZ#2vfj~&z11(m0UCYG}_T)-NQjEr!8H|)?H2m#dzPPmG@W^goYJ2|vGz^9b| zzK+x6gwt(pf|fqMc=F-Je(y6N6l~61RsrjJB3Bvtu`UQ3lO^_byw4Spi~Y{hScXuN za$QElHYtz>N-}am^$iVO#S@96N@%AjNTb0HMqA+;ExD#{!9YpIA}$uUWE}06bm>d( zErk9{CBjkKsF`5k{>98TI-QXZPyj?d2SjzulCJ=pjJ}#Fa@9+cxKPNpCA-$~B9XI@ zKA*~5`EVYfmM;?DdqV2m?Q0dMa~4WKyHmXdUKh)57o$~GrAAz4zYp9CAtdQVaX6d@ z!f=F2;cQu^Cc(He;IE3F6jrguOGv^tX5o=k8)RVK+3p{BRoYoV1w)VuNa03ia1~%d zo*vTuyCDM%dtCx08?$aFkloN@3}atNz4)gxrlzJgcRfTzTQ2v_YTK7y$zefH+a1q< z2GHNx#dy%#RHC6?ulq{#ufO~pPjCB!;Zg2E>krNZ55ASCF=|TqVM&s zJrctBgw&Qx@`Cv_h`e+lHb!Z#H}-$P>b;@uB(OUR5)av0+ZVE+X&O{!9UA zHDpH2YX_4;Ozq`*ELw_~Bkx)nC~3ayAsCIN295aFn(`8SyuGVhz5PuOWE&M^s#w74 zu1MuNLcG>7^e>aAN|Vyuby)a0LgMXGXf)>1NFi@je|vphYAt#Kyiw)siPDLf^A!Bq z?{TKluoi;WRzDH{ENZ(MsA=I*_WH_x*;3E~Fh6vff>6=p+l|xnd}I~w7!s!Lc0u_^ zQt~jsSBzNBUnD&xz=P4z3MjoKv#`R2$Frm;n(<4*_?;25{bpf;K;C4dEfrL8mC!hS zFe2A~i?JeHjU|-DOGSi~DJ~Z|d_p)iVnbHeb+sQu&($SY+pp*Cs#FXhN=>6mz^nqT zGFb%Z?c;2rfdh9NtH1}o@M~T@;lB6HJAvx>Ls8k#0oku;`jCvx!RF-T)H`i68-KH! zOKf<&bCdHSN;bUIDX)=D9Ce?$^Efdv(cDoTue$Hg)l#E?O6UU>O=Zo&H}((tAoVTi zt&X=9Oz8B2ZlTp-4>f6lup7R!OIsuHuq;_qItGtO>?~wDN5j|~ZO=u@DdnGO>q89< z#9E-iMdWkagDc-EYi_|a3eP=((*2;NXc&HgS33e(G-s0Ccq>425S!|qvPKUil7Fz^ zN!XhEB?lLE-#ZY+NOyR`@D@G`EsNa234bq5{J`+FHl}QcLIhyT0g}VjY=fr7Wvso~ zYc>Z=TO%8Sfpl^WzNNMD&o@(3L8%x;l-axC zNNmm@;rN`o|7?M237k*di2p#Ky3>cx*o0h>z_#Fqo>>~PWiY zm0s)u+gr5~@ODy%J4!)ogDSiB)K%igTu;lJFY%=p>wV20wrDsfG{h&zz(n^(4Cud& z*K_;ydIBJ=8WZRG-g)GN7c3&WfXzS*#Otc)x(_RD@ZP^h>;mwZfeL!R9#!h`-u+(+ z@}K<%!7_zJAB&X^?$}HvXdb< z_mN zn}fEWLFQr$SndWJZI)-S*3gBKlf9j+IW{%--?SB=$w_~ypehTSBb=ro-rB7T+QE(S z5#TMO(yE<19yY}QOwp%%kDt&+cr~w`Ud=29XEUDfPjpsfB;y=#>&vd+yDwn8byo;ERw zw=cRbzf5VIAlw)&k+y)6Ach7q*zB_=*5_GlWTYV4P2C48k|Mlg?klcq7pr}q{mEz2 zwTvgT(7a2;N<`g)Owe6Vutk>I-X!SB55;FVctO$dK2Ug(+#4mT{Q9*lRFC$JqqFA| zRe{>oocZ&$Q%_8bCf<$*P&jC>A8K{A4*qJi?t`KBg85&nE1FVH^Y1#h4d zm31uop3M9sQf^M|sK5ICPm!QZ6=;Chj%r-xaDN)7J~ zP=0(7*wEkxM;kDXH7O4E{|>eHZjZJ6x+9Igy?@0JMaMw%!AC5jNKu>Gq3iy^ORMhQ z7_{>e{}9S(p?0c+0H%hND8k$pg~0lD9Qx8ARe?*{cBN5#sA!^SVG@cRS>n`pC|mG{ zV24#(0itKjKCBYSxKA<%jr4G4jJL_)$l9;Vdb!*jNWy6o*zWy!uGF<*0to^cp{ z6QT@LYPUjs-N1`SP_W#I#F(k`o$?}*b3(gmoa{s3ll0aLB;U7>d(O49ou<7{b!f*i zz2jqEb`gh%ggpJ?M~8q2Sq-&EXctAKObv_lnRi-IPbTUPk)K#v^_jNzo5j<0RMGJ< zrA<|b!p@E)?2pE1C#76!4=$@sUyA-d-^+)g{RY$mYctNk%iZty4B0;sbRs zoC+@#9whM-4#fjvT_;H>!FkMAFksw7Y|a;yP<}Eu1|s^@(-2U@*x*yeHCox$kK zt*&P+)ZfFa_4hGMU)}8Y`GvqQef5*1zur4~Gimh{Yxq^}%LBeJi4Z<<-RJbafAKvc zo?P&Eg4zub{$vI+FREeLWsL7+acyPfr&^$2c=#d2+9=XE^)TRhIv?&bn*0dS(uL~$ zX!y_rw2rH19Qqwc)&lc^^#VN9oFJEDFV!kaOF>9HkM#}Bo4r*3p3rRxq3*UWKK&^| zqrsgG0s-nERZlHKg903ZBSW;1%k{J?jia-Yu*(5-HQ$}iK3(#XALBX&&@W2$l!+Y> zIX<=sF9GUt(iFZre}fPqbxJiZcsvyqmvWD^Q>SGs1|P}>x;Ro zta1wP;-nh(&9iO?^gv0AWJmykHU2@zmEXuk}Edke;t@P%Nt z+S9_xw>)}*jqygV1g`)ZQ=51k#=&bwo6s04;1iiC78Gl=qEv@{bh|ev@3Hwn z===hI1w!ivDL+FEq*@Gt1%D91Ko83x+7Q7a5?bzhq|YrERPc&2{HqAc0hJ8`n|HFX zY6M3YamEwpU^gN9=%CH_9XU+w=tn{-#w3Y>QoQBgx{V*#L1_fVrjM-CRuFJ`Yre92 zZ?P_30skPLJu4tVCp4a=qeHj!CH0di-SQIjf=BrsnOj1ODr9C%TmMNvtU~zBQ>e0Zz%|Vs8A~&mu$~lE6puP{g*AVulBaZi&>3 zbc!g6?qr0SZklG97MbdpnwaWVI-44pa*TxLX+T_yM#V=-)FRxwOZB;(pE~0>Q`Nq8 zOPH40%`}!Ze|{bPN{rcnHZxgHm`$=)+Bv(Z$g*fRom)sFgIl;!-M#pn?MQetEoV}} zl`@Gvl`@+)(=wYNd-g+9`hf9h1z7cIp;uM3q5y}etH_0j(YKU!JoYs1Bu&xcY^}`Z zQLve^Ny$qxQw)=6lkKsZf`LrY(TY)hGaDu&6Gc;k!G_F<^v~aH27*RD}48!goXO9PmGehM)?Qt7R(~RZ9g+RToti z)mxVBD$Iz@5Z76_#HUsg2-7K9CCqX+Cb^^q;GpTDJb9$XSztLnqCu2jX3npzv3+!!I{y+TV9?h{_7CkoAx5BV` zIb$3=K$=dV_hiXh@9lT?rKk=KG8ZyFvI(+$$J5?Us5*@B%%jP{I9x>Z{#MOkiTt(AB=N=1sfnuY63sPURboL=?pZV;F0 zo(^3_aYggC7rPUCAl@iPMJV{)KF%`&S+2!c;*p3I<#pw)(=O3|iXM|x`8Pvb5wXNW zLPdW$J1i?1p{=Q}751H|S7->;f3Y>Oao#fB>#;4gYupvGMYc`a8C^f_nE9kJ-}LFK z3cihaQtyLsc_+3sopi2Iicxl4rc#|!%ovRo{vPWd>Fk;t0{18Ga(C1Vw}YcYn*Euz zi|n*?t3sC2XH)mkhnIc=TFgI=+X#e^F!6s69l5r5Z>Kz9pk06kw;7qEIC18bexSKeK`Af%^{`gtRF7vQ59Ooh1=N?Uq|QCpxvKJ? z<>@5L)2f^%e|#+YsFk%e z%%-}k;$*dN^>S-JEl=Fdbn{DFlU{~?1KijralvtO>W^!5JH-lP1gX*rf;1t`iM4h5 zM_OiNg=|`@npLl93<}*W)SMNZm)Lvld&iC-2vn8|50!Rv8S+%sk94>^uSQV#a5tHs zT2arOI!74~?R;HG+R87?@2vt=V^}&@F*$EdXf=4|P8ik{+D*mnEbdrvEIKCVeb39C zFsZOrp;4tO-kE0d7>k*CR*$=dx5d0SFeP7yy&bdX{@v`nnB_!p*KBsLv3b{P#5?cJ zyEmjqB3BjXyAO2))bvdCwmu95Bqm)ZqF%2aP93)y22h2q6R0>u+t_^?xYiTwbO{v? zbxi4E|DNG2_oFdNPg;|QX;qMp_ol@3n)&?0HA)BQ5#lMSkiWwkft(iy`s2g)C8)vrSCbic=QX zJE0u4j&Ru+1PXE-2pVq*Wa+Q2==DZipEvI#cT~8;eMch2Q*T1zF zXPr6itp)vv`6c5Muj(o2_n9y)UTe%ftSc(kyr7jcdnoT4-<2zyu2R*E3SU@%Aqq_}sSnd3+G* z4+RhWLI*xh?_vIa8maaDgTK!oH~{yc_~ZpdMS-Kdo{fQlrLD1*orXZXA#efFT13Sb z3JQz#_5&>{N3sv}KW3t!Y^N+K!LDazL93&0rE5UzWMO^V4-}^pJ8)`YV5b9ivM{%_ zWq0Btx_g2hIKRD2M+Clm#LkS1NLf+_EMR400A{9rLHmM;8wm^sbK2+|vdg^^`qdox z#YJRnXJ^e$N9XA1NbAT*Yh`0ZN6*H_M)!h&j)8#&c!I{(+0stOiN?~F_-7}7_w&lY zR?o)7+Rntv5`5dQj;@ux9TySN?Lhzh{hX(PlgWQZvb6oREMS3jw|D61XKtR z^Y$vcjER$hxymaO3xH<87~HI^%$#@6|8eI(BmP%Q<^Qx~U}1Xs-%bDP*8QdsTLT*b zD+^#uJMRDR>sRCdzWJ*mC*AGR|BDnq>3nw;AT&1;C*40x<3`$Nrw|1Ah-dOjS^+o$ zQg-_XT@QRc|9J$?p$DA;pi?iPpm?D~U-2n8L2u8%J7EYCc7EsaCaL(w;X5m+y54v~ zD@ON`P8r1a>J=OY2Byrb67Qhr5a^9Y-`U_pvc?mr4{;}kuj8hAHEa-Tdo_0J`=d?_ z4ULS9cw8MD=IP223DlT%1fZZ{LA?L!tECqf9prmd#?fK-_}{ezhIt9?Q_}GGUyca0 zYmoB-7bwj#lZ8ORAb`F8%@;2y_T=9@|Fd&1B_v>xJyRVS|9+7Otp^1Enlhk$F@QFj z&QtL(|LqFBp!k$$AN)(Q0L#>*;029RS>Z+r{VM^ueHe_;>-w)7@wQOGA+)-zSm+J@ z`^m!UIHUbL#@{m1DHEVgaf5}#(|;}JUSR6If87WQx)=kDuoY7sB1-(Pr1bV-knalD zzitGckAZ>JX*{T+H~F`k40WrF@c&;Zqt_TBXG5JTLBQTgoin4(uayL_ivQ~(Lguzibnsy7Y%_( z`1g%}=jI% zA7RyecB^KVPoiG^4~_W-Xa1%H5r$#C-bwK9+uKN-O^%<8$q~$CWbWG$uqGfTA=uRs z!T0X~t=)uMm=U((viwO9NFHr*>_HV_+%3X;9g(xim5)F}#S-roX!QO2e^&czlDrLo zM&r_15@Y{4eT36cfFg6XW9k$?iB$@$hnX1z=fQmf^lxHX$_oXio8<6Iu&lbx>_wNAfodf#V;y6j9`bjD41m)rLld&hG7 z9~N$xr={w3cA}PZO*D>svm+k6GxgXeXV+I3mZ_xKzw1D%M$6os7R-YOHy4QEUQmsf zhfH8F?-U^h21%}5rps{@0}KqpsV^{KTFqB=l3$D^0~kB{90%b(W5aH3TIlb25(lVTQM3N;9$f%K0LXWyT9%?kquW{8_jH)zyrR|;^Lq}19$O@VAu~fXN=S{K zxsvp+q-O#TMI(p=zrZhxz=HN&Ze6J-axK(PeoRvW!d5t){D(?c1Cw%pcmR(^(Dn8S z`MWd@$6Zz~RUcTyS6pN-Wk0@}G}AYzq(uM6eB8L-Vqw%BCX>O8S)n$!$Gr16fNXISg9Cn=u6ey%soD0ot-ZGzZgPO|538ry zFI|4Detjw4=+;1}Y6%JdZRvi=54Z@RlaLKOtwtS5XDI#%Wuk?&qr!8 z{F*o-%QATn2daZXSXhRzzyz4f3S~acN_lpPm42F1uX?~p)rlwm zxNoFDEWpePr_+&I@@|at0<-|?o7w2gFT$zJBLQYvL*vMAAMs5mKSId z1(ZgLe*oo|_Wy6F9ykZXjqhtu29=f7XeW}yn=KAWR=8grd?oRGE ze=kSCWgUfHv{U~i;XVd1P^t}i{2wN_=<$9xczdNc&H&J^GOM{JguHX45?F2GZ{O|> zlj+6iJ-+xkSX@RXTuGBKN~i@JpvC#3##-XfK>}+5iyS}o`j=_@kHXD@=WY4MHR>I@ zM+?Q84lGnT^nqsN( z0d}Al%dN())^|?-(%5D|V^5N7h3+v9cA8@T37@iXehU_b+^gXwI>=wKfAj5cZ&D4b z0CpHFO0ik}qB7){-%Ol0qJxY{sc)L?0|k>x3`o1}E*vA@ohr8A0z&H2rlA}9A42^+ zYgY$r!t3A{!X!~bC@CO$(aU`)9IG~XZ+~Mu7(@prsx3lF*v%K)-&VfZw(aM_4l7O-7ihV-UBVofUdOMA1IT~8W_WG-{)%ce!Lx8W z#p<<0YE|YDx7<05Bo)ts(?-U4u6;+G-&%Ol&Kuj0C$rg_dS**Rk%i30D8YgP1ppC# z9oV1yOQQL2{f*O5Ji~AE1!Q&r6sW{EZoAeeYHXaN-s~~+Dx6%JV24LJOCn)#Sn7A7 z1025^v~YggLpV6mm7W+45ct(Ikd;pf zxm0pEsd#KxTL4;!IO?NE!!RYBt%KkXzpD^4`H12Ek2up+-5de?` zYvW+u>4LWt;5M04XO(G@f71jRX4d3IWi_UG^aT4X(pCcf`XgLAq0T+@u5j$jZ*1QI4?{AAAWDvSW$FMtWg zri;CmBLD6L#_Zu-`OaY>V5mEHa@3Iw{d1^O5@0Cq8Ofsi9QO`{(ZuzwJRt=4#?nxI z4#Es}imA9i3J4kI!-o$x`}2H5B7irJTm71pbUPO>Ge8pRM&ky4**q)2qbyC~uejX9 zWuOeZh2hH`-VkVP5bKnsLX7PcBrH~k~GR;XENb# zsI6N!))yP`YpAf>p_VjO{syoAcyuSqd3(Nbdc6+Llq=6np^&GdvD7L=6+?e;lnVBE zcZSMrwBYRvyRjk-;)(&_ky5}5GapOhJpM&C>RYnSX%yWvZcxyVi;-HTl38V2+u9^g zb~58MBNcMx?q>SCVVXDfE;s7$0rXcQTh@~Y;wfzaAmuS2aI`O^OaF<>Ky$#_I;uH( zj_wm)8*4uiy!|DCai!G{IcuW%@}PHdZ_nyZbbnJmz{~GW*Ah41T-R^dvDvOhv)gap zi3{)cZH%fRE%XzggPQwR- zE3)*y#jC~5*QV?iZ_!CJ>^26om?;s!AXk7Vdi{|RKe@jT2N$uwc&O-}p{}jAE zS!rg_pURcxI$ZNwuREEoJU0aCzV&)dYQ5BflECR)ws9Ck35f)Va5o{|?b{#}Us&ju z_fP`pcixk-%KlS0{s~~*AT8-{N;n+0!zGiL6LwKu(awmU;zER{F()b zOC*bx=AB$RIRNwTx9mdrN$Ar^z!hrMO{r7;K1(k;Bqwvw@wX2GWxAh{hPa%s&(<>C z&JONK^1IME1>J#@fscY&8^ifZ(q%^dmUpb|2LhktiA&aBNg}OVqh#ioy!B5+{)?ha z&|!~SQ`u~VuxQnvqY-ev*cHCMJj+zAG!;q#;;G-IC-r%W=Z%L-rD+U(r7z+Wu{M7c zQu!c5U@b>U6(%;#G|3qz*dn2*G9WymmuGMrPa<>`P|L zluBX$UjtH?xL-I-09K3sGX{m6v1hR);I9UR028E33+U0R!!NH5kPPs`pX>XW?xj>t zBfUD|bOZzhR^~sw>EP;$BrShwIy{~u{|`t11{b)n0D2;?cidynN(5q_T(#;V*=&H? z^w@v_x8(@^nd-O?%KqubO|&K5r=%(@Od^T7uu4}7Kyh?4Pl_=A%LD_437m>RtyU#* zeRZO>WZDx$MJ^bO#k7e5+%SYmCGbr-Tyv|n-;%1Jr3JTW!O-x$eF87m@y>+QwdzYG zsqMLO$YU%A`RjAn(%0{rJ&87ki{8p+&nLGu)}I(0)mboD4Ny)r?v<$5SPS-P)li@8 z@q{04$cg(Onvj&Q%qkSCt8R|vQct{LWMe6T&`q6)ly@+ZXiO?7a!spEd(P(+s5H;8 znuKmAWIBW#2+iR8nT=T*@6B;tT*oYt_jh&6uPwKC+oDHv<}UZ)Q#;!jL^afwi@*H{ zmuYxZwi}PWW}WVp?M^mrzabu59JfT?_cS|Sp~)A6I=nZhQ&DHvC&p&?(m;>(2&X58 zIA8X}4+apDnqGC{dqP5o>?8qVrlyuy?MtCrSXh|6+W+w*${z(+^;(v#oNkV=2moUL z?Gajn;fl1HX>C`(&|epA0rt#!QdRG95(-9L*Aukk!O=mew@w&b2m zliBEB@@S~2vFAryxb{eZDRzg}LTue+Fs&RXL&@)E(K?R#k}$aO^|2;mYmoUwl~P^v zkA!4ScTFim)u*4jWZMdAsg{pY(@d$F**Bj+>XE|oKuL^LXB{sOu}AQ9>v(8=N+_vk zi^R2LfvCnZQw$?^K+E%ara)@&1_*EM8O*?y#B`&rw7&fFq_YFMSPN}H{B05QU``ZW0$2(n0)HLcZsteRvX?rl;(s9piSxc2C7k0ts z>v>-Xt9maY9)>-5@PLHXObD^wL!MT%ZgXEuK1+nQclCPdONcVG(%PJF1dTCft6@Ygxc8_-YlQ+;*85VZLa1qZNN3xvu4R;Ql1-Tf!*0g zBX(x@x?%LXc_j*Ea9nOm&G98Y{hy6Jbn5hHA&u8^<&H-P^S zQsGrcXmE2-IX=*qzExniM>iD(R^NO9fOhz8$yh?;|F40{T3LDN*FxcJ%olYDf% zWQra$9`{wBhO<6U0wd4xu##UKi@($$Ohp#HLIXdTJ?(A0ud1HSJC1n(Hw&C~vjKb{y?0J(@BZH|Pn11RBlLM=vlxe)avFdDrRrv&2gXlj+g zJIA~R>-G#GUtA)kN@lWgP?~_pqku)UiF3l>=6FIV#YaTKr$jh3B@*UyQdhdLv=Z6Q zg%mcr{qSD z1WKpN-KmtrJo#crnrDj-Y38E8L6Rh|=A84>A6;zY%j3qJl7pgT%p^iz2jBr&#^*e2 z@IdN9#rE%n|38aaKi`$^rll>0Yrlboh^1>aP z$(rsz^l(l(C5GbL0uEy}XE@>~lMB_G35wvtPYB)4D@rl=x|dIzK1UJ>G8hGXNW(^I zCmDVkr8mtd02;;bk?!LIjk=E}BmVAjp}cEAf#u-sJV?GikQ?h@8Y@tdKUnV8`_rWW zCYjlC`uWY}kFeJ!Y&Of`b@p3N?|e$7S23K9Qo%Gy?@65+74 z*oJS%rg zkryw?C*PKFAUjud&>dhEKRLX#ctXIcTR|$Z4nVZP$!28$Tl?F{us3JYGPPaB&5_l&Ocy-RR`F!rfxyt z-ZgRA-usnKhBp};kqDf>jAU_i$dym`!JsnroE=?ju)QcUct@V^eeJrOQL#DP zIiA8SNvZO3OE|p<>2omFi?8*rd(RHq#(6@xNAjX$4QxA~9<5_h4zGi8tG5|Yw~l*) z53T~yLh9`K;2ihVid5#1R(nZBBOh);mW}}2VskLemY&MuSOTk}$Td>UXhhdjkr2Ex zzpymfDE2s7T?|X#Atc5atnac54#h%f*Zb7#A0-%4b|o6 zx@{ghC$PJJa3~&e?~i0_#Q2gXeqlbEr>KcecEEU4t@M4lR`4jHB=y4eO1SIbTSy1# zvFi!stF%>D{IkNuIRN=4l<`D3I2n3v0Z6lrH7 zFFGODFTR3A3dBuz$Nqueuv9>b?wwz?8dUjd*5v0thY@fM&eHW`_fbjWj)Z@aN)lIZ4yV!Ok6OgY^j@5NnLxH#ScHfszAV-M5-yZVmn{8sIM@8_ zrfjv}T|uwTmFNGqMFJw{8~^rAhRtdJy+fvG6j=-wjjHnX07QiGv05xgaEX_DKaOE9;xx!CZ|e@+HHv zo?FK}*>vcby5*>js(WL-l>29;r0Y|423!GaqjUBnJw&35t7*+6TiT(8niq&W z#kG3X4`~d7mTh!zbVXA+4R54sm(m`fJ6!o^h^EK^n1{xMrWgPg=lfY>=4WOJ?^ucx z3*eL`Q>bJ1IwT?p_B?c6(xo0n4HsxCavXxFk*5M{&<2AER ztodY}+AvZ%CJhgp4U;c=4u9})!* zV~N(9%Hb%bb|%z38XgXk<+;BriU0)gE^qFZ~i!KY$rvtr}&1EL3m?RD?Lich&R=CJk{VXJz}fnRB8V~(~gb`^s#P+p4y%Q^vtKLx#$zuX5E?X#|Azn1g~-9 z|0o-vePZnjayUC!u{qt>+3`_E88tYG@xGTb<5cJ6q7dYw7n}p&0WlC*4&S*%l3SOU za(serGLjo6s;{?BLIEi&$=y-$De*&Ic?)VdIFTm#q{7(!J<15L=oOpcFqCtqJn|5y zIaU6c#Sl!k3WX23(MNng=R~3fQx(MHqKve~exY=lMT4ZzRL)!78^59kEgPz;mTv#Wx<& zcHZOlX--8kwK3OmKyye<@jYbfv$#^F($r0NDoqsLESnbK8p>xfN7A48YE;-2k|s}x z-^Lp6GFzR*SKn1$ipHT{>9n5RnBj=l`Al;Pyj?&+yh7T$Hg?`vs7R=IvVGcJo`q?DBx-LyrOb`L z9~)gbI$B{mrWCvRjv6}iHBCYNPD8u#J?>jkjvRp)}3s4v+X>nMAm(FB&Z|} z#Z6qTl(nvNJ$4h~?#N_(?SC7Z)u}|1iBi|v8bnFp4827XNh_J|Oz)JM(x8XTZ0WgT ziw0qIOX#g*PJs(lA!qB~ht4f`ASu}j-D}IIZyC#s=9(WXdP}8J=So~2*_tRc=WVcV ziP^m=p#;w?XA+=$Y*ih-nC$t?R&EqQ)V~rX9#^EyUw_x&*60j$?1m6|Cm49WypZKwW=~$8e;SZ+-p!2S`52-rz1Ub`wt*h3f?JBc{Xn znA?8uj+!>llbDL09c?K@GLQzdKEmF-O8@Z`vMx>7%Zy;=UaSB<_`zo-9+GE33c$FJ zt=>I4pEo5L)_ACZi>IDu;(pU?lHWo&6TCOJK`Ac zcG32#k&?2)5r`E6KB%XsBO_TgF~0J+oVfImjAG9`b~TMs0j86M&E zr?CWV^EfHUKi91Aa${Q75O=BxBoU8&)k^c}b!FYbJe@}*5OOAD_@iymG}bx3sNBGm zPE9ILL$s%<8vEYHLMS>$E6sCrex%wgbn=(I)v#law?OARo#D+MLeJAWJJYH5L(Kt> z9IvB*w0DE8c`ivV_{S3*2YqVzg-P$qEI^~xR)TQdm+F9IbI~5u+Wj(*mP!~)lvbd~Lm?E8?W4xy{7-eq9SrQh z$9rCD$T{R$Ipo>vYE5ouw>woXw>z%Z;CXSracS_Je!vHOBE{^nxExgbs0dp*%a46c8ABCaVad>gqS|1xS?rTc`Rzn#Oi_q% zs&z-raix$ut+$59Fwub~i9<0DLSbP#L`9F%nzs~FY!hk@6fep2#86~u7_A(TRseKk zD_gQSVZ%KyM`_($%F*7Jo2u0`8wid<1i~WKPBb_~&fpLOR2JOK4Ohq)c_D z#>o#Gh%!D*QPK5u(OsB+u-kHgGyzdmbpEK6nu_Aw+WH0G&dG8u+eJc&BYqJ=>!b&n z!h7d|FNB$SE3?8?Ts1c`VoDao9FBx9>)_P*w>pn$UjoAk^Nq9ASH>0x97eplMl7ty zU77g7pLs=fB|%l?aos(2pjjLj!aeG@4y(h1%dYs)?fkGm{{3>d{_f#`T=&7unsL1c zmpZ${)whbtvU87d5Idhj;OG-(pa{eoJO07%?1zFWdSVTveU`hU%CtYhEQtYb@{fAO z`8Jr%Zi-Dr%Jb0*6ZzhyX2wm3=40NLP%k=Ioe{lUc+ayaUsZmm(Il5gCw(D9uee}l z99&1i+u>oZfi4+N`come3zK5r*-~jNb-A7G`JJDqEX`Q9!zu0AlM*kQ=Ooo+zYVXy zmrMUdRc7;5QP1iFOl?IB>j-6K)h%cmQJxD+pxi?{?ZBWc=L)drdhvp{Cys{f$V@3R z3ngD12t*4Vg;gAw4~^)6DiDw9W{au}i3HBUA8tQpU^iy0YI4I=MOmE>GP0NgL~up2k4#@Hq6 zyUQ?fLw;n6;bauPPhhu4O8+X$r{NjmU!3T0!+p$XGM4utR}!-AC#iKaC(dx5k4AJ@ zX_+&&ZLb1gHv6a?i06rA!fha$I9W)=Xh<{Cms`Ek8>MkLeH`%Y$RvJt{lB2gs9UEFjNF))%K}nmJ;sxbi7uo+^F1$zh&qF`aywTXA9+ zgdwfOJlbi^jF8_ng&gpb(H%P!hly8nW0;??XCeRG;O!!&YR~MIG?suYerG5_PNDDi zKA2?}NG1xIOZojLk9c>+i-%AH09dE*x@->H@eXeb9^%3~Cd&V;g8FBV&nd$TaBMC< zK~0#?Wq9)x3-7=$GzZKqmMIC9A+y;O;mmIXm?J7dtDE8 zdE+9D6d|9WvC(Yl5hB9jSb*(m9>Ef~$Kud?jjh-tgT>-B;q|#%HP17>1H5#uvhdh} z?mTTPIO$5H&HNPNhh?cu2>DVMdjnAi1MdeIZ2N3#9rv0>BtFu{0fakGKiK)Rr-%gT{b1JY~B(}S@nRLE0#V&rZ)~jPGAJgSi0syxt|JWG!Hme!t zPg|q$483|xM7&dq7+?6=>!A*_rrb+*&knHbz-1ap=1>Aw> zMP0HvMVgqEn>MF@Ar49lhj->e2HVsXzZBmWl3%B2^-b`YkdsWQ$ls(&O;I`m{WH?j zz6i{|1yAmsAIsH(TUH=+O=AK3 zD*y2M?yOxLx*AqT1%+9&$+717@A*=1Wpo*78Cg%q(Q2StLj3%2)kppI5IqG2^L~mrXbYMcYVu!SR zwde(^Y}nfykM;l|4Z3zAkRsvMVrIUom*Z?JWKt{(*gyQ{D0nv6AS#LT-I)*(wGYEY z>nEqNLyLf_Sa~7#rY1(`jxw8zV~s}2Bqhn%$j`_)_Kow;0V82!@$C|&7RN(;k8&hG zHbO;SNUTg>^j%dKQZ3ThQ0t6KTI^S|-r?_Uhk0t6o;W-LbxllNtZ_(d__J*N8)h=_ zyQgbjz6zEG?>-;4T z9_mS|x;%)EydjO%`7B$5S?Y36#K>QaMMyv;iCF57d)f@1C%EME60WMMxdO76Xrp8VA4hjslX&2!(A>;nY>a65u>j`76`fW!FrMev1F=RmhuV&o^H<{cuETVS^vAMr>@%O z4KoEQDsjUb{b7hsE-9;O^@dqiRWG@sc%A}ZR#grqGFEYf5KTSq3?5hLEQC*asr?3% zRtq&M#*EqQlQL6vRCeEl!@%a|McSopqR4nx>m4hX52Uu*S<(+!WH1s~Xt z?Fz^>v<3?`x3oajI(rECLRB*Ijk{lj0dc|CkWzHA!>Hosbz%?1%XiF1NSpnEK=^{A zA&jy|ZerNx=n0BQ$bq+c_vJGTostY2n*=z>qrJc8yf9+ZjGlDa?gul$xs8g z!x1i0NfGKjKEqNxkt1o1KYp5w*eZ3qTjtk+tun`HGb=PK%!&~f%uQ~4>s(FyUbBU8Lsc1Bo&=spIeiuMf0=y zuJAbA8Pyw3Q{k)G59-959krN_N|j1cTKzHTsFM^W%f^`aZwE+yc?ZZAEkb4<`*Jz% zMqY9m)tXN<38Sj=7n_c3A!8T6s=*F@nvBvav{tX2vNc(W8_DcyFl2(Zmq-qzTbU|v zRn^-bQ$1{lChOCW@l<%=r4=j#&ih))Dc-WeH$IZD)Z65KQIW3*t&Hh=TQzs@xpIId zJwU-@Nap~+Kn0*c$7uHzpt&z^nd0$n4T3X$vyqooR2Kqx*qPE@I_D-4?(D2p3DEHc zz5Gg{BM(al)v=q$MAfl$`mXuHXdwr>zOzF0QI-2lnp}m5#}?V!d~g9bA|}}E;^4w& z>lnXL6b9yUWxvnaLCMb^K-ajw7PA>y5=r|puS0W?N*@CVrDd|njl;9UQLb9?bBfQnSlS4{**^%Cc{ zO(r86t|u~9^7&<|#ozKLQ&3t{t9;uT*-Bw*JZO&+8EoN(C@Xr`ZipaMe#(xIjeJTr z*)QCTJ$RBHzSYU(o9SU#E0(3_WIskFM!VzfaBim|F0$3gAy~Fdeu1utoc91$e$5C=uM5bCpJFo%AvTnLN`Qiz*w;LdVv(3Qx{A00-feZoGNo@%ztE7&lCZ2`qb!&y z5y%_fFrUPq>1<9yt;*$eITV@o*b}FKEVAV)W>zdC^qCI{v@?S^SE@f8=mxVtJ{KI> zU*!;3?o3O2ST|K^`n);p>Wf{x1;}B!q-n5G4OgDfHKqnAgdUtuzLI#(p{Az^J?k!iTOApF>1-=9F({ZR+ej%_FXU_yGkS4C_*Vd65xL5Yyk|~I=))x! zfOi&vgh3Mk*GrA&MpYttv`~TVy*^l-fV;glkGe&>R$q@ulGnOJtn|4Gc`^*DR4Sdz zczt-lt}hPh`8{9?o^OPqMQJ7Aa!~+^Uv<}WMzlxr{&TBx$_H-VJsT!o2=JH`SAAJ%P%&b4?N6;))tBQ*pObyKF)@+D zp=LZ>R!^18WB+ zRPy;FxyaaEG~RC?ygTtm%6^!EN)KlY=$54_n<&2+26IPHp7rXsAR?{I)OuEH{=yo| zTZ@vp1aN^xe6xNjf0^@Li%+YmuFWU|ZRBy@T8wcKI>+ya7Ux8B+DY?wz z(|AA7$RW>@NMVN%a5|OwD1Tr4oGFpO&;pcShX7@pxZyW>sTjVEZfp_^%Xy}W>l2=F zw>5+k7P1iYH!8O^i9jgz7&X`uG|K!fhrfjK(M5sUh0=NVax>OnZ57U?O%kUow&_LK z0G{{^0aYa%81yoUcZvh04u=%I^@MJ*CG}W;|1SViVxb!y0Kx2549GQ0k;*4Eip7n= zcb&d157AIz$c}5f`NHT2ACfXa-Eh9z$*>!ikd=O{tVl>}tm3CU=W=+)-*jOTq!}}& z=`PxIasJ@e?w&Ew=mT5fLMOMIT)<`*03_$Tb>x2nL)Q)VY>AQDR|RFco5HQEA+6C@=86p|TJ=uDQH2}>*qYAXRPhC{ z2|H>vnd>)v?m#hXebR!nQ7Uf213$VTYq2!%q~4W0@^fofrJD z-H^TV1zQk-+S}s@K)dn{<7r(nYh8Bg=qHY-$V3vtiG*enT9xKP<4(Ar>exZT8kNFO zt7ksq`tJN#7&|!1nG4Y=Gx+*Ez~M`b7q4eJ4VhmY`|;x=R(toYne?bC_FT;|t(2c^ zl)cJhFxwPyubbQ|RCCNwu8p+;cJ9-;%wivXgz?CggTt{nMgoG$a4ZYg1c%Rd5XCLN z-RUN(gdd~fTP&?1tEMsH-DS=^xb~30j+RI^&SquO3pIDd=v@?CgCtf+FGr(1_Y~h; zowtE+D_iM{D4lG^hojb3w~bm;Uz@}0K<3rpnhRNs1m-sXI>PBzs)#sRDASwKI&h0{ zyGuK@Uf*HK@*nx~(f}M9MjAU;EO${+;oKts$sH85W*Ule1yJS3!X^u~AK4gGiCV=j zM^wNY^-b;)1o!ZO_j5RPEZ?h>IZuO$GJ_7-L?$yb90vUte^z|)flm)r80%*=+ zQh~wnOU@md3kKBb!s>xEa%=3b07&9ErJB8`r2qer_T_<4=-=Bnr8E^Plge5uj8HVGql0T}1f?eVBt!)V+Z-VYOIZ!E*KPj)MF3C#`M(iI6dAKWV3d{P~q3{BDNl$$<{*Vg)zBm&3!#^3i{ui^;vUkgqWx`irvG z7tJFnBukc+G{n)==CF{R&2Jq|8VMbubO!&>+wD28(Argov`s4+R9bh9>+3jX6W}Y} zTO83Du9ezCNGb^al^eS3ULI@gGKJ1)^8%DaRcoY~7^mrG1~5Go$kM7U$%SDtohc#F z|GRhwDWo9vK|MqAje>r@4+#n^!@dGLmNlKmgMX;c80YWrS}_lY!LEjK&$Ya_kk>R7 z!%<`3Kiu`8E22(0V;$h&DrAOPPOLUL)R-UTrhnrfF`TS_qvNez4KYPO?ORD!a_wW1 zv1dtM`sh}x`t&sw3*^_YI%Th7uDCScQ-cl*F=|KY#cqRQysd3+Lt$#?WG z#t^bSx_d3e{tS~>l&j9l_H3?^8=>7W3PV$UW7IK0ly=&%X=JnWzfM-!7Y4H$zkhm? zU!S5DZ~X4E!`$1W78ttEuc!}lzQFVwu=%5XZ}VLxE$j4D1WbzW&Nwai*eCm=O{?D& zx1~=j$UjFIrO$`Yx>7ufn9lcJ2SL52AUfk15p$@Z@DSTY)r~`} zRx=2?`A8{)?TCKy4=3zG5UAs6G=5#A`BUX7I&1V@D*C-)pK!l9azyd8f&Xaa!li?o z|IMJQh*ojjCjv*^dzo>TPueIGIgG$b%MLESw2_X5f@lfjV6##n4?4HyR|pMxG8MU- zMwVEwut<-(vOvNM+~XCzuC&q+NPzr+C_T#Ea_hUM%Gl~`>X1bU=0^d$NjQ0+Tu|g> z#T~?9LZ@@5;GXNNMz)e6geg#apKUDq6TrAXex&4=J>C)@srTXT37wuI4X*l_v(NvQ z+5Yk0hR{+oW*BBU1LWYKavc4Bx?B*1OMcY5;MUs?ymiDXW_D|+QLyf^Nf~!g*q**} zMx?b}`?{1~sp%6dAM;bSAg)#_VMQxb*fL(Exq*T)L_4$nmg~%wgv%$gnxZ>8#mp;X$AD_7Pp zAlqSYwN?e8sz5^>+`Ky6^r zIK}%~*y>Js^upAC0TJ!MQhB1#RPC+D4?7i4MO@g{1x5F^rASGwm(-NoB;bUaU;B@4 z=+$s=q4^XlDnxb8_q{G69;3dM;IAz|679! zwx7;cqp18pfo;i?hB)w(+CQYH&zyZZ13T0iF3NBYd8XJOlTsq;|MVICFaO@v!AvzCT1}l{oV=WYbv}e+TO<~G>MqbmEUB=v?goy z?Z=^}!2-41_qX2PXx9f?)7~xxGv))RO#J>tRDndc{{Gc%%D#dpR$UEO@Mvyk0X?om zCKL7&#)p_XW!NU0yOJXM6>I?_M6t)yj&;vhh$LmmEP?#poHMCSSpah*N;iY5`_@27<(+sHo4FUe5H{22 zR98|;)%pG07T2etp{UG0K!XrPXEBEP4v(C-yo-40jQn?KBIL;GlbY3=)KAqz*65zb?TdFb`}^SEut3aiwVT-RQ*CiK)v zckR$RE;E%1YVtkI1AFrwN3&Zz8c7a$&zIGy*pM|q%5wi`eqP-)DMNpi?J1Wl9LZFb{3AzhD?y-Zwgr|b91Z3cUSio z#A>Urb#v^!)%Y{t&h3PY8hy0K;@=#_1V3q2vDez}kP}HR_8rpvwH_}M&SS|f=|FH70Y~T!eKyALRQx$W@9YyoZ#w95Y_52n7u(?&<(8a6g=7! ztp!z1u6grukb9;-P&QiWgt`v=C+AJZ-dXkgu*OJqh_%+K9B95jN9gsDyw|snPWD%%mm4Lqi%2g=KUhpTyX89G zY*dO(vOaI2cXr{)D@uy_f}Dm6^7ql9nb%1HdW#J#yH&agm~w4`iF|%K+GC&hYPoXM z6&?G>L(lM@0hAV*NVUga{nt)*WU!i=Xdg1GRf!y=JSG{ncDLr-K#Yf{k36~ck*1i{ zrL|8=+;-~kW?15gkMx%1`n_-W%B?+`s9Lsx(r+C&RIB*U)?|gY@0|=>L^@Veb;4fc zYl`E2waK=eVj}5RsCMQ+eNXf$>`C!rpP(~~YWMQep(B9F_$j%OTA$=s)NG4-Lm@H5 zD?%0h$~DxU5p&2DjC&uZUvpTu?EZMOwqPI~qqDuTDf?Q~PLX8vK%bkAC3P!?_0QK9 zdEm1L<0_!HMUq2hkO$fT41V65*S{91WK{jX`=y+Zu6iBm$u{+$>0ck!_(9b$ug;Qc zQyWM08QsrOtK#)5;&g--cYsPtRnt1XMHYo=32)2!?cQnkja2P)uLts6ds0wKKpUCN zb(wV8_n4MKXLW`s)pw1J^nk~YPwvaJ?rd1p zz2Bm**BHf0yQlyB@T9EnWheKsNop4hZBtk``GI+~`U~G{;ahj=-*P=SXJ|fsD&kS( zCErhV0$X2upZ7Ino0?uegDnw`cyK95|J6kaQohY)b5clopIM8b9y>#3_3J|Wfv%c* z80|A}iji%IK%E>@e9gea{^QwaPQ525wXog*c@1YSJDCD+>&0 z(xysxN1-mT|BHZn(EDfZ)%{Caj1CJ0eB=LsH!;xzI#k@)_I7V}KQ-t+NIw?hqf zbsh8NBxcaL{8>u6_1FIM(pH8Om8yK3mTyL5-|9?zh(_QDkKh}VJD{A+yHRqo> z_P;UHG0?O$y2ac4rl&YJfZR=Xs5$)Sk-LXc|0I|T;)03N&V#SdsGet&a9uvf@blRp zYM4j0bT3#gtIiEoW;LbWYfaM3G!zeg%W_nfQ-JCRlp;O;+m7Z>N0k4`yw?@L^9QEW zDTRh*?i$h83T%7d%lhwaeanFTlOVl1EaM9^XALmBm!}_b{%wjza!n6VpEtlB(P8&L zU)NJ`i0hfS*ZMCcKtB4klXC`i17qXyKF3z@1$VwYy9Mk}xmw@}>g>5H;)?tH)dM?2 z<(@iiscM4YIN`K-vxEvgvo#|okRQf3O}`d{xfA^V z+2N#ML`!|w4dgfP2$jGm;G7m54|BiI^cTd$#B>iFmul>xmQPmgxEsgiDEsW?!Vl~I z$omdWuiX3J<*16=nVy_$9pzef9=}Xf1i%99HCU3>zu^r7Uy2(iMQwSIDD9UJS)`Ww zeYO@99{!q7w2FcIqy!EEDt2+Y@Fj=Dh&`wl&IgwTB^UuzvCGkTE8|naD&-+ut)5{} z7y)iv^7lgRKLH724mK#R3|ecHKxX6YQo)j7>%bMTP>7<}8oDu1;^JhL&>B5tdG0(1 zK8>6JKT92TwSX@%X)t+ea}q3k9{YKa)7C0CyfueTSby>7Q|z53G+XUpX}*RCe<=Ots}& zccRQ{GW7G0tgIat-Q#5hq)=An?xKIX{z+7mGY{kCe|aU5_S^yCl$BO00&25<=X>HzamIWPtUj&oOSV>8*UzgHF4Qr>qH99fY+$fT5k6o3riEk|Fe-P>a;}a= znlx;ZD;*@uFgo8?8Y2a55+9bb#n@_pl+X7|-Pq0@>`C#^AfDu~odRLZ-$!6e{ zWM3K@X{zm=g2R=_-=?`&4^qPeW5}m}5kN!y3Ko23B>>!^QHd75Exy!WE&5T>TmC&@j=O%$285D4TJI z<>bk!`E0N)y7$rj30q804mi;8>gU}Hh{J9yze>FJ1a5xQ;~w|;bk0hTtaxdx{(+aZ zKevu6jRFeOe`<)0OzZq*W~Oy&8g2>IJ;}_hNbCemfZzM*1f%am-kBSL&r>ZE0v>tp z$uurU`A}9OALO1CvAzioT-0e>FEsVK4nE9e`5R#QMp1nT-?AwDtBB^O$@x97%RAqe zb8X+Pm@`3zx))l&_okVXsH*$nuF6|+C!Ch4f?sxt{!AHaEl$d_0UC_?yE+kg7#?Yk zzN`ix?9|%a!1LH3z2GFw2AW{W%&YksFEVwA6+ec>8Opyne#(FEq1V)ED&Hpm zwU7arV#Jw%M+)#6hiW>7vfIK>iT8R=|AlEltz9PuAocXQA@_` zbQJ=xd+U33AXPB;iwIFQzI??{>^F6M@d!j!E>U{9KhywKHq=L4YDW@<=>Gh{f;I9S zgzMi?_33j?Vo9@RilSAhpBhZqzmHU56xC!;bu%uyFKg_*Zb6N5lTlH7OwsqVfNh{Q zcG`7AP*V?n$jZtlv1`rE;HtC1xG0=e+)>Q)=@q4`t&?FzX~~knjU4IV@<;ddcxd^s zrz1Nx-`_MHV9m%y=wEgiywYZ-eSzw6^J+E7dR$4c++2TMWkv1I`j;ajJ5^OxY4kog z%0+Cv-C&s>*!7%&D}Jdt%P(k;>7fuy0hpSd-joytb zDR1-|UcTmWddv#_4W4T^PoJ`AxPPQxSCMJo38k>R&nks)rEFunomMHt^2btSEcv@r zxv|umare8-m5Q!G(nF~siUH1ZB){8O>6dON9_{2u+BmPAm)zv1Dn6mILV-ccJl1Gdqe6S5 zuap+)O6?t=tGa!{W3?q@-)k|ffo z1RW{ML}t1*`3X6f>9xed=c8CkkgM}-UuT^AkD#?FPIYe1;G*qwrLh~=4+U}BGAZ;Oz#k!6e>7DYvN9@h&M|WBO_^D` z^^pvTLu@UqqR`nv^__5iai|Va_pmFE=jba18p4q$b1(PtHC&>Tygz)EMZ{XW4WSfq zKBqp6dvmpY?HVdnD$As@;?VI6Rc>XP6Ll+V%DhpI!Ru$|m)q4MDGs5#>BXw(Wwz$G zPI4y8Ca}VJMC_JG{cVMl@CbtMvH8)9&VNG8BaC(T7+nE9thTGkh^L}ybHUT@vhOKy z1@1-C9jK1;-k21<^ZDr;wM@pDS`HuEwL#A;-hdNbOfSUON5ej0NTBhUeP@%okijA0 zoVG;gI)sp*p%p>~*bP;bAL5YicME|yC$`i9Vf+2h8y|u7s2o_n=9}OOM@veO_4mW9 zZ}jC^jcB_%XBEwrOcNo&&t;Z*#ZEtJ0}M2i;7`A(2l2cSGScj{T3#|>MO8ZK-152S z0Camj$>q$Gj@#C%B8BMyQE z4iowYyir-GO!zon%c>V6ymu&WTDJ{-yA!V3rYMHn*1R}z3!M@2pd^NfYT>lSGpDtL zB;2o%8@oK{ZiautUr%wQNNK&UGjuLS9Apvwipeybd?9PPJ`y6CX;}NI{&3wsC}-C% z9C-1M|5`OT2OEeveYyE3zP)(^&PtcI)q8v0(vlT_KP}YMgbr=}qH@f!t-~)UB zek!hTaz`U0n~%Sg<}F=y^E;8o)gCj3_7Tg042YGgJ>s!dvbt56(Bkfg5T<6OEn3T) zU9R3De_?a&`KWEzIsm=UyBuEgzNLKMyFEC_nS=+`MnauLO+LGDWJuyq>Z3vox z(Hp4=j=hlVR%w$}-nFqhJ3T9aA6um30aVm%?6vTPr$EgG5%bTgBsX9}Wl1M^b z8g2HWOU0!U%>j?ZWjuy`Y@E0`8TkRX_W7UvPpw1!WdVcDpBIV8+4sd>({xR(iNb3S zLS%Ggb8lUyj7VliL_A)V-8f(}4i z^vK$YoH9M11qEjwZM&C(f)02OqB6&;iRUuq`2kM%}R3Q<3K zdlDax;aU-=m+ShDy9YI`aMx5 z?%!!8$77sci)%L*o~4VHT2$b`b_i5%IS%b-${eyQ9Ppx~JA3{5{=^lo@}_I~=xZh0 z;VR#XZLb+ly6DgkuS+k6*RCngWrkcjbsMZX#G;|ZzowC)FCw|t?_?^QFba`9VvGN; zf6DY*@RU@ZJrDlZ1reiO5Ojus=H&dN-SpX?7-Y(ZuO7d@uzysDj&Q+jwl5$1bjl$6 zwr`e9HyhReWAZS`BOc{kJc*0`X-kKX+JqunLbko>@=!mMgvwaXg6VG6%XFI%M8#wk z$P46(P5bHI52=NcV`GwYMW5`8$BqqclPh<;Z__munubVdC%OVUD;RJkrBHfY4i%Tt zC?fWQpWG$I*%z0ZvvrIgp2Z6eR!fr0%pk@*?VEwTb{Q?F!vbj~)@OTNW18X0{MF5k z>8#|9AN*;jQ&CPOq;t`>KeVGeJf5H(W@kF{3`K^ocD(GBeMnPYL zIBwG&PRseW`%t+`MVTs06OqrN9U~yj&qcM9KR?xbhzuXw)r zO%wR`gFRzYxwz$@R{fljG$#&K)1MShvz-A5Axud=r9Mbge6V^-AISqAf9MVi!kzbW z@*C53h!*_2+`@E3JYFg=KhoHv{%1=(NB+h$zRahHNK~4L$MU_@q)G00uO9)bOpk+w z1J$$>^S~dmSq5G9RlEJ+pvN@7R^51VsI((Wm)HHM6~bY1EBK^{Bh6#@q1@S$>C9K1 z4|YKK-tW{BZkLMClMR)r7?1NL0>ri;O@6t>9c#WhTWF#k|5M=RA!BS7zp?6Y}mho%H+>#I5X8Zr8|PmUzYM0dq4)KZ?jF9FaM<7eTaH^clAz6cu=JTUZmj`7IFbg=-xN1n&=oU>RJy(`N zJ-CdG^afZmmzmpr>F!tHs5XZz8a*C#XefkJ7;nVyk}$6Hrp%tI1!r&_%0jP*W!1! zYWL34)sG~!zkVbe~07?w+SUtre; z+%%$R;1F;v&$ZvpxuK<<3n5||L<$)78&z~e!C`?T_eYv|_pTead7f!o)+2H};60y3 zzG383U;tSb?=0;)qdOjNIR5qR$kw-yo^}YGX(_^vz&96RQ=r~L2Uvn@w!FKJT6v~>HuQ{NGE)AMxzqnekXPLdN^5*lk zx5s-!D(RR$04BH-_aC!W->=lSP{yn9ueT5n$o6Nu?S%*;9YA-&%h>9SLEL>?^?VVB z3qrZ^)lXffI)ct`Y>Qg8eP(9T49vf2@wwaBvq_+QSu?dS)?xe+ePA2pU}JBsHAl5r|>*F?yMHTrnXDo?JAl?%_LnQVb3<2DInQ@0N5*K zT}~5-5KHGR!ztoSuq)^eQPTqm3a@TQshdOU5z8Le!ENW=iU$yvLWCdi z5w1}Vv8aNQYcF0MVj^m*3%25h@141Mvey!P32~n1+MH#kDx5C|stYb_ySrqqRnX&a z6P|s>E3Q>^e~8PwXq!@1`y~Md_LE3ycwoMBJCxPg;A6dV|-rLaFQ;yzWN^ zqgv6xtH`aj$1lucbxuhKHu}#6>29V=;85Bnq=i}z2Ye6F6&`6{@a4e?W40CjcrziL zI-xMT`QeYvo6VleD|}aC(?y~5O_jjQ1tG4U(@(j?Q8#(3ybP)O)3<4sV4*_Qatn7ja*`}qS* z@jv!oxm5Dcvw-!)AKfcos^fDs!-uNlxu}S@M76W(y1$$3h(Os8Q<>HnNm&>zj?TX(g%?k=;G&ZR7IFGpqCeRg(UC3`E0>IziqKIKPMBaf(PhMh zBaugxQ+W>(6mtR7nDu{pe^OnOt^iVtudiyXNw~xsJnN`&ndNWSlG)D^g9VF6k8m3z zooaV7y#UX}-Y@nvK+9pFdKfTLy`XC<>86El0-Es*(`PSG1jld|d+u0w;>Bfivr$dd zyHR?mByym(YZ|T~37B{l-yb)XmZclvwK!#7@DU2V6#7DHA`8zked(emzvR8>o7BIh zY#9KYAxxGrTd(kP*3Z#5TB`ev;u{d?PXkcsc%g$_&P&O8|C;|^03Z$LIPqd8tOfoo z^X=9UmAngK6+hx(xB-*^n`^_HjTP}M#l8)%r`_I^ z6&)>?!R(#6pu$SGa_`&8Q{x)~A)O6Np69xXbe~+sL2$mS_-1WshEC-Mhb<=>IN=pQ@#x_v07zcd`fuv zhQ?nXL#Mf``pa|X(#ydf6aa1d&myHt;?YTt!->`fU{NprY0xunihq&zD# z;6t6sbJ{A>Ky4*$P#xwGUmp@=d=~4uaf*!#`Y?n#8^j&=v`=}B45xLXbKIB?Q9XrC zR9}O<^K(#m-UWg@Hr(}Sc?$pyBoae`Xl(~W#~R}rjA9e5KTDe7QLlZ!xPr8r_xf05!T>dAH^-pU)KA^VzQ4pnMqg3vaT+gg%4$1ki7AENtyt}|DJL{UAY-)7>Y?2aSwlvMH*y?VoopOKb4-P}p0qOkjGZ*%g zD6G)(10s^fT}UroArB2P8PeOx%IXHUu$^nqMWLpoGoa!= z(txU9X&9k>zzQ4>A9}>NKXv0gha!?7ez}*f#*Y93K(@u>8%N5;} ztnBjYG+<~wv&t5q4R|dzRj8M)YCJ;6Yvfdrh3(7tmCtq=IJg)hYYy2wKgOTSVqf&} zF+JN9>H2pg3J(_sy$UyM;ZPL)o?K2yQ3ujkbBC!c>lTyau77TrHrz1PV(}Ay3g5*+ z$r$bAs2XMCYq-Z0$*w`C<$KIQ>#3rt23q=nbuV2tz_p*N9a?JQp>G%@QcKl3hqO%} zK>0jp&wU{k^&m&ivQIX#HQ`xopBlyEienk>QDn9E$q$9RRms+&3+%UI(>aOF9LGi% zLilruw_hDZ2WiNPpeuu!4+hUfe;v}MdA0ywnLcmAd$U?=d|xW6#N%^OAuq8n(@4_6 zt;11h*V(|^g8je5mJ%+!!7t*4##JR}nOwGX4^UfIM?JK7`UaTx+V$q9ls2yEpBlb- z<6B+BQMD_rz?~k|d$Be(zhAZm-{p#L z*!AiIg1WI1F8u?l-we95TKLK3?;5Z;|5TO$3eXg%6$D3k45GYE*w+dF8}2pMh$rWpj&h z5}fc>Kl@F&P!7fytUZP*u(9Mq-MuAGw6VeyPx$egnbu=(t>I6Sc)z-F}gpf7h>Mi&ONTy9>ETUk|zXoN&4 zw0HaTOBJZ>dppK@0-+09pl%uR;J4Ymx-_7h#@g5qfBFr)x*twUpoJ72$ zD|v8Wxo0o|=Ms8l@#Z&FsvryGY!P&L=k;jRun~5pv!ZDPt=^G5hKyLisG(D&XzywB zUwrFIS?S*Q*n;U9h!wv8Vx98KUSx8Sch-%mA zuBPY5ueG;g6l8O?huL`#`ke^`kYANx()=2G{HZ2o5*%l{N1zKf7*4B|FoB)Mte z(F+)t^*|A1!yS{Z-U(x6yR&YD%K=TY8MW)B@;+0fEdsC1B38BIin^M7;mzgwWW)>T z5$qp8C*~gJlK<5;1g)Fqkzg#}4`-gOBoP(kjq9Wq_p|rw_qb5>R^pEA`CS@YhS`F5|@3B!Qr-Ar=6tp1Y^N- zqX%D*6;)n9=+4%2#~&O};JZU>st|nA0_Y^iuM~d_4$fa}62c%8X1?1XUk7$+Q|No_PHj02pjkK2{1XaMi>N-Q)!Bw^wk^Z-EI`o}IjL}P@V1B0c)Qt8ST^=! z>%LrPd11d}qq(afNpuJZK!GmHH{S1GDIY*o+w6ePLzFa&P%Ct&&?$X@ZU-*Er{MR<;3{rH9sKGaHX}bIl|- zOOg*tDL?dS37~(0q%()@MYfMQ#g#RHf#t_fj^6nVaqCI|2@{Vx(frwYYk-!&ZSkVh zq9I)Xt!W_;Z&9JFYauEhlaYKFu~5a;y&~l#`3UrwMyIk_KAitb&zx>3KRwHH8p{oz zY&hEJZTKq`l{Oe>>3yi#-{QenFxmRuNjqre_du7dn`BfF z6U<4Z&z#n}VIrmP@zIagVq9tE3&7|(ZPS&0C zhs4*@ssee(wGfz=FWB>xvyp~BN<%>KNdhQMme%f=24hZvh(G}KnlRNL!{ow!(Z?;knp`m`)f2o+_g{_eb$WD6#2emVKH4L?n-aE^%i_s z9P(+W2rpAsi(a72N}MWg%oE53|95NV(KLNP2IasV=eE{Ct9QGh&*|g*vZ+92=|pnE z8fbTHLC1qb^`&FzyYgu%ZoblC%e_n;-T-5(p4Hc5(z|oe>$)-$nzUw3z9^eon=y*i zQ3o%g0;dtg>q$he&qdf~!e68JpH)P4ad6x-y^iL89`~Xw9JlmW8nZh5Rb56=0{4LX z{B{4!kaAEwNt`;lp>mO5pg&|IST--mq(pr7mcwT&2O9DZG)Ok4RgZSLPNqa|Y#g0c zzi~6r4R&p?h3hjGMJN$i8iW0Nd zFE!tp>jr^i=tO?E=6gJ$>iWJPJM&~LvD zXxnLjA5B<=3dtZrc%0aqLb%hD>$ne_|pXUj|mHevpP4e zEbaB{zD<8oj$7mlfylKrWqHBJ{-+qJ81ApY%+~N>pDD=!(_-$SR(LZHl3Byq6 z9_xQ|aoC*J5v-}uQnX{sQIps8&-rIVi5aPNga6f%N8x__o%BZua%fqV&J}9=+8`x+ zXqDtZo>s$MlkEcrM1D3&!JY4dE{KxSw(I18oq4qc^?f92Ps(j#wP(E7Vi5eRhVol- zSHElSMBn4$=P{w?Sr^y**W-|H3%MR%vJ3)B42db zNk5&#ZW9-Tt9{J(e##!Zxq66rm1x&2F@4@-?9+7=3;syDU&i2tJGw4f=q2r`LPJRs>2KfNhE5Rn1o-ALQ^-vyjhPtuJ8CKr0JnOq@ar{_N?efLItlA)0}S&7bV zUHtBOj-~s{#lugjtDtVqGyCqq)}&3JV`cwHgM5EM1|DI_q5Ttj2f|LTf|r(|IoxNc z9YLCR(ABD#d{tjMTVNn()-F$lfPO;;4&~wV#RDwHP^aP46H3_}>rUZzQa#lV;PcWR zy}kVPSe`=}RIjBi53dxumBQZH9t{Pyv02v$rxO7y^tT~Gu2QLoq6Z_kpsk=b5>tiZ z+)5KHq^ym)Q=Ww_=)-GF;Ix%kdJn$F1WZMufI-3eyJvqSJbH~M;5`$%c-uO>B9cX<(Xo7LQLqwdmhUS^M0`Na9J30-Saq(JsKMI3(#hlX(~7vnn@3=u(6JpjcK$xi+J>!W^noOf|9An2xdPPR&tQNih-5A3 zO9MQqSoSXbQp&!z0ffsCb&PY)(RfSXw&y z!En0z9qMQy=skk}=F&eUG1(`GM^sRxYx4S&ZH zrE)5(lw2Wc-3_d5PX(-4ORK1pQ|~MUl>~{J2I|i7?qJ8lby(HNyOw*%1$HEl(%7*@ z-~jf$AAad~-6Lj|#i8u*W%qNJ16jcKZ%RtL!}0IKC89p!PursS&*!E5WlG$X;YoNu zi%d9V1FcjjAS65GJ{8=8OBL^(E49{t$(k80h(Ve;v~NAU)?G4Q{vAX1er17H!iZ?P z*|g(=on(qOmrFoisQR*RAnU>kkSw!v2rsr{%Rn@8Xl0f}LDKw~0&7jpU)lVo-|o|WgqMPtEWq8 z?ZX&`F8S_GKI(grssdp=^Mn@#hkZjN*6`r($E!!JmR;`rdz_@0dKq-QRpSo$LI$a} z)G=~F%U$a+4^xSGnZ(|jIUl!;de1Ca6jk@}(usiCwv^{mAnpo^=_*vomF=ODNNaVo z?x8T|FZ`a1C^rgg$5ea?2@z_ty5wKW?vAuv=vNnsXBT-k0UJZb*)#wGqw2>;KFFNU z3s;1dW%(g{55XKbh}q zV+#2i_o%Ft%DYizM+Dg2tG5Q~NIODCfg|dhyu`|h+!%wsV0AJYsC5-eB+GOP#s`ZHTjIx?;%sTI?(-@*=Y z{&7BJPT&)webK~EnO{g*sB!$SIlGO_PB;4!{+UZ6TF~jBAVqaL7vU7U)(5+=an{e7 z13DkL2=dlt!DO6DeCtMQ)mzKBN!DvSl<<21rqAU%;3_STh;x%TUp|9JsNK<%+C!y3 zHV-<%;$!OQk2wb3;9V*4lYhqGb?TjttiE-7s1j{u?FOVv2Jwx`JV(92!S`#uodjUt9bB&RlJRgKc>}ExOlSHsNe>c zJ}1*NYp1M0MX(ZRknHNnPc*3b5S& zxfUwZqCZJ%wM$ia&v4cPks@FpuCwOy#WVIMBc@gtHvIvw*2wNLaVp-Z%;|a;8Ct%s zv-#Z;olzqD7W7Zc_85A#d}iUEK75l}dEgyqbiPu{)Q7Ic|5!K)QOndN9P97(m;b9w z^`h3bY9Bu8=ALFehCf5c(iCe81nzxgfzxHF8olqa&e80aP|mvynS$!d0SL_EEEBds z#(++-%8e;Lo4&ksTJ)&rNbuztMfW~|w3H+2=ifk?@f`aFsj~-T-~XVNEw;fAAM?&$ z>+Clu5)|0vcgfIFPTRpqhN=kKjmK>|qp$#Ye}mJe#JUGm_|iVoK9B)RnpLy;+L)Is zw4w|X7q9E(=T5F1wl}to6h% z&O`7DG}|j0B_lwboH?P6LoOGYj&!ZlUt99{(fA|B~94pF4WM%rF zGx__Ue9J&s7=2mmPH4pD%QtcS&9+-trWxnaB$Hk?Z9vQW~ z`_S{xSKJrD^&L^gJCRmyeSz5Q zUa0+9k&3~Fo=@o6Y$v`YzWVZJXDY6MGz55y1=ECLRY4GG!1nh!slOxAhXsY=yBHUg2@O z_2U81FUjDIVOzIt8`lE%HRjo;!G+Y)GTUCE@j+;a4kgj2aHi2jm@d7VP+ zJBJ;rhK@xAPSIh`6{Np?R5i|=;^YdHGMLwKFqrGuUjPYu9X9DbQO-5AfAic|T1c^mdiqG&^@ zl@6`)fm?!>LUb&`40F3CJq-5ReB2ageCp88D5xtObh90GgZ$dW4R#w+2OYSz%J1(_*6$5;xMMeSMFG5@z0ZREzp+ zmR*?Atv{c3A8~EYq2zeEHSP>wMo1o36@fC(z8s}S^d4ms*=U!^W zfq%9Y^HGndx0qimq0%M+~J5}Hy+><&48@ie}rT-{c-2gcJ# zVQSKo6`ymv5mzgqoU&+q#q9B(l5P6w^mAUyA-ovzWt5ybz_9#S*FN5ZVc`1~|Io5Zid@$|i> zyPloz?d*y_Wl?t6_$QA3LG?(fT~q=UQ(WGKr*`6jOzYp zJNhcMwcirK^}CTl1$e(lI$tutAuO35JAMrV`0rdzG(dyUXF64%NmMRda=vOeJZOGk zGU{>JNu}wC3D|cp#}O2x%IFG8Kis#7LJ?4E!LSyWgWGSqlYKzNlmNpNMWso`k(!LF z-6lRFQhOyQ9`x2fzLCs$S>U`$MSlT+Lj!24_u`>DNzz`wn0E3fjI>M>jvE%3#W3Sz z((Xr;@Wnuil8Yams6ldr;>82V&C@+F}Cf`$b?0U%jJX?B{xZkA+t~k zy1CI-48mOe(cwW*FAzZw;!FBGwFO~qX8seyTC7YL!_n7=s_#-Y7Wp~!zS{6L<50EU zU#L_@lx3vNKhI?e&f$_n!{L8D7jAei>%uHA&Qc%LQ8Rch1O;X?d|4I#sRlVt;AV$o zd?P13hePBch92iTLsoX@HBVtoDojr@RlnI)Y9!R?@qQIR+BoS(xE1WnoR|)EegnG? zySIP{+T)Rinuq644$$Kgt>0deUNi_j7!OluE|YCvbW$*D5i}O*!swfrC|f|N7Nvxw z#OksslM-huJs%WO`1vV1zDEgv8Gy8($`(OH%n(X8H(8$PZBIuLE*Qf^Sq%Q|24SjF zwlihvA|2rw4j%$SSLwa9=H;ZbrTw5$7-IPAN1!_Ln9s29z9iMNCg}e|+IPoO-M{~r z5L#{$Mbae7EPGd08Q~x^qs(K5>=CJyQQ5Ps9LMI^BNZW(y(xQVZ+_Q1>HglQ$M^To z_n*&4=YGFm@7K6q*S?l7ADx6ltwYVi!qpgT}=$`joDo$%2)V?SV{l}EeHp@jiApmmLMz3-x8#FL(@Rah% zE2cjhUWXPUwaEAugk5}#Sf(ey&tozP zerw6*&?5qq>=A+v*2+3xE5M;;041%t7AXGrjxI~Jof_X1V>HOx$7h#%gW<|~ZyhCQ zX)l)Th2oNIB#Atu;TxoLv_rzcQAbTwRw?A6zipi;64h6YG`v*Gvpu9LZ%&p;oD zUSOJxb#Pd0QJuSn$vwSgiB=`Nh_FRSHYXgxDfZ`^V6%uTJuo2 zepai#@A}NxyC}U&24_f8#&gJ~95pa8fosl!P#xuhaDof03<+ciF3ov;>> zqB1bNQ{G*>m@|0@D}fFu_pHo^K(Zt{*)m;#0h5s@-i4^4ovgjc(NeK55_2xn*m&Qx z^{+B@bR#Tjpt^oazz&PV7kW?Ew3a~!9B8h{GoI@A*jf_Fh%E}Wkae$a3;;7L8Lt1$osnqkj zK$%RI;)2RW`D2clfWc#13>8ugkohowG1O&kEGdGy3@oA~D5IRUODc|h%785$fr7$8 z+D8?cYpimK7fDk0j6)!gz+UP61lZcAEu+?`jxOP#l(*w^;91Hwx+jwWh4eNaL$S6_~bZ{z-?m3p!#p$aw%NZNm?6XeBW}761?Cr zZZjQ;P|9<@AO@@-_L&UwPYkF7cuayL zPGD+u==%#D%dhXbgAr?obYKCGcTfQErS5yNFqq&|pkOa7GB&>HUwn5vPS1R8z+WW1 z!g&k_rN!-#NP6vvP33Vzq%Mx4P}uASa^`m-FcD|*4mh<>GQVjtq8xRJ?LRq4h$`t~ zp2SvlA>|jnsM33=pDlJ^5YDC+fp{;Du)Svg;I#wRwBJXbsg^Ag)2 zBJi}HwJa%8TF$5(MPoSEA+74u#z9q*jYyk%D`tQb z@V^B4J}p+tQ7S=S^ik9?AdJT5wPsu_1m-@Um2IhdbqMR^KcHQIo!kyIhA=gq3?VJ{5fi7g)(lKkqG4Q`X#6`)Q}9EWipe8ySb(e=E9wRNlcL zpp4@1!^E!+!4prD6M>w*0a^|92?^yzUX-Q4WX1q&r&>{K?A&dH~;3F z5G0FnL2crp>|O@+%pn3}5pQx@9`i}%!>IBRaMaim+!L!(rjbXDCdx(}%i+h!Q;}13 zD<#OF7*R&IzH<^eqEHqaE3+K?Zt#azgo3@}DE=M;!JL{gIKI4VhxyL2cfZ?{EdH<> zmqF$pI4}u(zy55=+1jxUl;{;~V}f5wKoUmxLf0Gs5R*qSzk(zKA5>TY07TR~iO)Eo z92dETh)V#R$zw(zoq~ym+^np9M+CnrCv@~6Nw$LFE(@dIrl-L+YE7;fK~fD`M_d9S zpZEpJV-?UooqVP=RN58VQl9m!nLSf(YB7AZb@3e&o5d&1We;$eDk(`J?jS3BpVBQ* zZ=+685Tav$4nj?V5xO(SO4y(3t7fTl0hb>HMmnS2wj4k)M0KPghPR8Pk?@(zlwxWp zZ!YUXWqnj;{=U?vBzb$!e06H&AUo&kjE79Uor%%{2hBp#6)Z^%Z=sF((UL zqqL4>pgJE>0i=-hhQ?&(WM(z06m-dAYhqOc%X5s0)2z4}Qur&5%k&%5PRdC2%$^O$ z5MUJ1YD%_uymj?aKM!{Cgt>a#P|+Va!06zQsu5GWGiue22)U3?xIR}WL^NN*%EVgu zY}+v{ny)le9ecqB zC4D%>8RK%a0tq-q$|N|(V4fv;zS2c+Z!YISVjZ9EVg|Z7W(H22V>DZRGt7+a(nK`M zyTc&db%h=BEkcV%ZEm3|7ULd4@*3HF<|_hjTN_!jK&9Fbzk7R}ez0jNCQ{#iKLR}o zNqbq~)eVWNT^HA<=g{+t0yD7}Wz1&C3@PY^7{x8n-U$7qWBk`RJeZSa@(29tmou2t z$89W^COb-9Oj@yH&b`c{U=ApnrhVvi-@6YQ*_-R8M7errUp-d$!IS8~Eqzil2Fb&3 z-8YcZLNX(tTyTE!n?v-XqEbeLAQ@reH(t0r1Hc%w%m(&0pB-_3AFQnVt*_$#sxP%b z#64PqM!pg;cO_07O;a>Np)@FsF};Z0$K8w;}1=p=f}W zh5Xntx+eYJ#n9DTS3pH>hx&kp_SNgmN6T#{b&u)ZIvqq#8(Ht^}X0oC8|hHv*}DL z_bZiu)sSY*da<5PAJ`4KiW7jaHJ@*@T&S!q0wt$QU#}hE^DjEwspFwRq`N3Jc|Dw< zl&qo@qTSq3+RFHaM!AD)Qq6I&gxME?u8@18G$NnykJqM7E`kOTrj!1LelzuJ#MXt* zXounBBfazHZcyFHhq_*}39cTpanNPQ1iUaY7=ruEm-_%jhZv>YY7{i=hQveice(a=D^84y1Zj#raYYw&12P zXeo(be#1;yg$c))jGplKoJ)4qmP|X_+Xwl8w!$U*snXik6_y_qCz59|7p6R$0C`NG zy9z2G79t?GUDQ`V;6=`G^^wU}fzc$}R}FYZnE>|X*hZHg!8HP(0$IuoYOsugQ!hhXXxUafw5MkWB3^ zaONclZFNByWztKYsu2T~@fYbZZjNRJ%+3;+hI@00RE6JGA(31asVT?0Zm!jr0ZOwv zXvB)bLE<@Nd&lw%={J>P#M{6C;UaQEya^*Q7~n4L>_ax0X9>KSW7*>|k61PHzXBM6 z8D^Ts;mG*rZK(W?XO;afTrz{GDv+@u;61(Q+B))N3#nVK*NI9w8u0~Xt0p39>Bc>l zi>aIj|8V`tOX0YqSJ2RFc-|pWHf5|SsSA-5Hp5>sZ}=Vy^9&5<+Ens3^RBn)S?Gy% zPpZ7)w&gsso@6)GS#PfhmCI4qB^&>K}*t&EJ8=9WXypQY3NWz4XXS zJXfI%VhTyi@=*K!@$UXodKHX-9uA-@X)IWgn=2k>v*Y?z5QN`-4Ot5)@Uy9k_cQDatkhXQNp5CQ#KmwNOiI3j@b|Nn$Qq}bt4a2Tmc~{#u1|}lw>mP+TRr1ei zz-%VhtQRVf9O}^d%6B_YFtNcV4V`ar`0EAem(t$NYx6rNdq-VE>d_Ptha3g10964z za3u-pOgGkL9JChH7?CiN(fHx@<^{)MXvze+oGznrv zAIBid>K)yo*d6*fnhL+ajZ$Yl_=kjkk0}YbiaZ4PN8kXCW^R8*pEp5(a@cMiH}diK zIPvgL&q!|K9fY_4am;x0{!{g5jbs=s4@Yur!v+m+({YIJtZYsic0#z0SRbFX8U9zN zb|`%N`EBdiGjMtu*I!j9{3fJ*uv9+L;iGt0f#fwJoUA{NG~@q=kB~V?EB|CM0X7$R z)G*3)=QJ2>VW8x$Oh)|O@P0?p%&47?#1KY#GEKVO%eu zigyeO3_v{x4Bpb*mqT{@b;PS3T~^^I^1CZp!5(~q)C0coqWXgj{_izNUG?gk&C&)^ z8W%HP@$R>=qY8Vp%Fp%uz<-yp$Kwlf&!3-_{Ode-V|oWPVMq^oh7Z4iNvmK^HV5m+ z{=JsJKamN$MEA&sAAe3Sk<^Fh$GU=%>?6f6<21@&bNJuCtb=UA#20o}{E@g})kEB( z{zG}XhwzX86LS1C_XAD;5cSSPnbct~ZV2RRs-1wPG?nW^+i>I& z83kAKf{rqgXg5Yh5tLwUc$NS@a5=RgdaX5wM7nBPafQ%G>>8=p>A2};~qRhB9}r+ zCDolL@uuo7Y6p9Fzbzz=IuSOzB9ceqH>RTl;xc1%sK=8R?xN-hi>wU$3vWH(B@FPo zy*>FejCgF7(8gisS{SIommiDq(cy2m1ClG&aKY<4{;K;CWZTyMs|4E+H4YZoLE73& zxJwpzmy`b^BBW2ovVRB6D%J@3eX7}77k?yPm{7q#f6==>a#n(T2U9`(R|L}v zG48OVSO#RA!5j`F{9~s0@4WwzMSv5GRKPRxd=+83+api@*;!eDJbZZGn41Q#ne85e zFoZ4~l>DCj8DmEXi&nx^?mvIIj|y6{w4VvT#ky0B??HzwbuVWU-c=BdrGv=^pA}BR z6KtXkNOin!Y4>3LU_)b{pNGLl(bj_RKHJk1^&4vHTu_t-ofw3S{_();GM!JATM0|{nr!KUNEy;Ox6c~-&&A_W@BHj zs^Y;PD}*Dr)~2_>oak+$9? zi}h8v_0w^mhL=X(KE3U7Y3HVpBg%!~=~lc!7!=x(i{P|HBZPMm0ue{n9O>}fjy|N( zYPY64e#yyvMXD%uBmsA)MV8Ah8baGmmGT?@o11;lGN|pQ?xC1fFOWoV=pIJ01YRRZ z%#C5DX?Tfhu~(4e8ZNeJ!P|Z!NNmH2UU;TEyi+PW5Is3gJDs=XerEyKs}KlTi|&(%PWd_m*@|dqRy=&=*mH2s!5^>K1mb<@PZ_2mA}qOWUy0a_ znCEo_ys>Cc`9Pn6ON}`1=1#Z_^v`fK;+L5Gzf#iuX&MN}8&(JYy5U&}iiJQL9o2ui z#vTu6M2xX9~|Me5f(iBMnqPs4pCe|7EGLJa^++e`&g9{x)Ydk7qmIa?%pdi=$Dflo~EyxH#| zexw4*78+ecxtqX+s#hDFo>Q1>pRu-Rm!ox}Irmu*v%v$;5w?|<`}N_52aa9F$xyL$ zvI?|iYuTrnLITFU%6g?A4MMYrV&ytS|T4;od5M2-B`i^WXPten*iwOhwGK zw7>LTHK1huu?##;3;}b*k6%w&J~pq&aOzRUDZeNK|HF>oRMIwBYRm?2Zlas&FuX3W zN>^Ix>Dj+*35}oCC|R3%HAQ=Mg1C{@VT1Wj^Sh zuVqv`#MT;{4ViX#Lb9*rk*eQD`&6oAir-kb`F7oK5g^ixOQxNhwNF5?_NI+PGpj~R zcjF23gCCdj=Lk%DC!f5t-MQ1R5$@R-VUtGlUn+{tL8!jf*(Mi{>WL8UR(|36_u=cv zMuaSn>hB{i87$hOD!Y{hfu;6T`y&~pM?_2~+S2o6xjR1E^(OnB%;4ZvPg+`98p!Eb zs>^Gd%?4;eEGMZ@J z93a$BG8gQ_#*7F=MD^kd4xMuLM`1KMvNXPw>Gy7bRF7`GJk0$R+imI}HMP#BEpO12 zh3*z>x9^R(ORe2l8@#dPg93z`t9-*5FIvne)V*by+ZV!Y&^38V=x=Oge64vs50ZPF zuZ3g{{3JlOB6b1UfZj-(Pk0-U09Gb0XE-7S4_T6l9XKcvx9&k6MgmAS?Lo3{j9w^v z3FY<#$&t@57Yw9c8;Ry#>mik6SPHLg$ct{X!=1Q_S;Z!9E@aylEt#(ybsHFJjU15A zW?bw0(Z%{se|0VhS0$E;t{vBCpOrn5H%z;IRR|Xdnf^U_1(p0J3%o^d+OBb<*Bb0l zd52rm<&X5)=|XJ{s~?8MX~gAGWQ&pP}$_ znfsrst{aWMA}4gCdHJ>ZKv%L#}`5??FvN`rHtqnofLF?lgnfzm-=<| zhhMhb|H*Qg;~<$JYPT&5UYhUrw)-(F*;#bzR!+%scSkh+L!YZA`7!oQ@9udk$I#7T z7W0#1QZ*Mtgq*hzwQjz&yWQn=`CX@(HyKrh%K93@2>gs%-;?ILE`4Jczv$Wc*J!wYC&Spk3wP8xNb5Yg)*8R3D zG?aU70vK>q#!U93=^UE|lRMmx)!5XdzV#%3Uf<{!Zk@h`si@4-I|X{X$YhmT4&#%YktNGsUC6u&HQ;Ik~U_C?O5 zspWV~WtdE@ExBF)hmy%3eo2gi&Bzt!S#bQCRx~bNl(dW@ovC{K>!y4H7LR*4{F3D_ zI}NF34j}to$?@;D4L$`HBp62WI8c?q`>2=Ds+(VKAltHh^^Ctr@ST#9Egn$;x6XBy z?E8za{U}Y7tSGni>(uSCxKqPBO+TD^o@+c>s7b8LqVmqjjWwKxKRA}kqEx80GukF7 zS7hD5X}Y?|ll#pt2TUxUiSTC<=Ru#v4O;BogC{cQ?!cUd8xHHUuz84+RV+<3eMlQ` zJ$(-oC(qj_9JTSHWVPmO>s-9PB7c2jLM{3`+f7KelTDRaG*Qv79aqUzBpOjmAon`C zIZbG4D`79Nj1A^qNG?+1G+;!i%A*32EA8`Dqxfp-69iu@ntTHPcP2@ov7sjo$@40) z08w)41-)4neGB`&+FGZD%)O7av$k9;a?G{rpE^ZCGHUzhyvojik!`d{!=1Y}!>l>? zYE>nsRH1=G7dm7O*A^|H5yTucHKbZ~)b<7dbl+}jKuAX6C8CmAcNXAT1|AfMXGHu} zT>_phbQBRI6%kw#kuJzSv4*d{L6`NkUW^l5`*^@^$aY!1fj(-gUZb!|u%6b!)u?3V zxRFxnX0n-pxAw$jeo0GuUR)~F$82fhDpvC>zv+P&cky0Vu5nAVHtp8k4r@*}e*2=j z#|K?LXrEki&qeoK3e$SkiR(Q4B2~}(8KR|1Pf7kMt9_;j$97weh1wfmp9zC`a*}EP zIr!hqJpk!YJ#QgLb9?o`QAKjQiuHz$e_kAxIzF|lJD%P*lQG|bKIs_gv;SbWsd;hV z9i4RZ%M(^ZM0PT@iE<=3y#%t-O-K$nNjX|FG@FZIV=OD371eEZAVjtxcFNt2oN zWjuj`=aKy*hZ!v3h2c77_5eW&rg#Rk?NT1~Jl2tTY}=q4Yt}4fSx)+eQnUS&d6NBB z5-;ctlW;|y%A1RCoYM3BPIjsX$+GM>pO~WK`7Y9my~pU8p-mI~i`0Q>DFKSUJ_loyiXdRiiG#u-rv65tmbN*dRaNRU;>Xt3W#`&i0qi zi1;kn5GE_KP&zI5?MuD$s#2Q8u4vetcBQ$t74ykzX(ojJaQC)Pa<0^yMj;RYrVYCe7KhN4U376xa=N>o6E7X zdP^m%OKPT0^{CcFl#&yAH~!QBUCc&m$LFd)L0Kh4bkOXz_-x?_9GO2mx4yT zmlP;`DPZ3+jxA}!Nb;m>fDqTHTvvivPkwZqgnzbNNRLU`W@&S@(;Ta*+fwzL_20>qb2>sGXeMcx)Oo%s%`5*s+*E8@s*~Ir9-&Cky?njpp1#T`^|9bt#=a zc{;j1g^~IZW?4^c@%;bJ&E1SaJJl9=l`Hdna&(J1CH80p@z9R((MUyLj z6H_aOJ&qG1mZ3clYp>zVdqnETA23&x3f(Wx6`r>?ZH>$4nTkeR z%5+BC)@G^Ax2vYgdHa^SKCG(lD93k}>g+{2K-upAe06jYIxkCdy~n^XvQzMMybWt{ zueQUKmT6Dc(?dOKVmQaepX`(QvpVf9JiQ}?HauNsN&<0c8>g*Lw~bL=oAV7+IJbc& zv-cIB?B)+?O}0jBf$l^0Mqg)sS$+NIfb>#(Y>3%H5#?n0=5ogNt%nyMtbZo+%=m|N z>lbrB2QCBdI~+LtCgiah$OUS3=1~7HpYWhSoQ!&u^+OlVDHw^hV0 zrxf0-fk0b-QR@i}mF!@~lln(`ok}PlFQUmCGtD|=1SZ>U%qp=CxGC$2s)?G z?8wjCVH+_s*y?Rf1Gnvx%B5Z#!=m`6FkBbwVruK*u1>bHEu58^4SE>6%Iz|&xo$A9 z$flFo$Y(ZnGA~=qzuV-poM5MY_^G6N+~FvTE2t8bM)BY_U(f?UlTmu|u)W{OtU>4m zZ|(H<{9u6URJ7}(sEM{vL~?QWLZ9x;J(srl(NX$!z?8a6qO!} z3btSQ9xABySe<`nk9|r^U6VkS3b|e5NM9$|h+UBdz5u0+;iWRS$EbGs@ozAgHd}6g z{b|~|zl!wNr(_TfL0V3L8b81xj%YqcSp~}fot#GnvY3Z9l~1w}*XJMLB@aDBBf6Dc zor|wh^#Ar~{E_E7O#0vY$~i0Ar} zXy7{?$A(wDkuVdv;R}u(rmx@%Yp89A=K4zp9Do-J6o%<`32j$9iiZKXZ}8f&m>`@? zs6_L(&-^&zy|q$2{FnDu8%e`R8N17lKhkxe+!srCN%z77L&Cq0*TVpob|LN)4*%&t zY9n5BKwG&D=z{U|Q;J>1=)wal5@wU;0380exm7g0wUMlpiZMHB}qz zuoKhTL(qZD+C{(quWRLi@MC$Gdb~4t7o~$`Te+)gy+kbclG$LN$t|B~X(M4jGw0YB zZZjQTxO-=RpNf<{g`zx+OU83Yrux{m;kcTk_)Zcsyi z>P^+-&YB=MqUS5yg^;pkfL6bnJCQ!U;uiG>YXr$+sbgl_{}=b|M%rH=l*AV9%ZB%WwH@5iSGULF!CqEPeK~6L&bapR7$^x*nL3s9K~HotC8il zXfcMtq7wBaci+54aIj*I1Nq+W!#jTx=D}y)3zZcM>^`~s0Es0GvCo;nVU_+LnBiG0 z-RqqRljp!##8b_4mAjArJFJYmn3^!juXX+BJ0izm2#B$V40FM#E)ortzvoV%=q}do z9?7kJ1%GA=<@js(^EvlrA{wv7Qoe@KZJ5g1Hr*`Q#huS>ohxU+iqsLOd+*Hke@p@z zbQA0LKmpnBNB_qp%_(7a=+|^iPmrB`TRoWi`xJ^9U^wak2QvK4iFH4MOyRV!yzGAu z2Tp|KCM+q#D_?|@jGct`xM9@tt(|8`%3!a*=M4n@k5BLX=^+Y4zztFNHYxn!0##r% zDS^detNpN6{_^jvyK}+9WOL8x?)z)w7~Ubv0F1V}U1HkZ2GI;sn35S+&AOEj%w<|( zkmmOUKA$aTFdGar?Y)RU+;utFx3eR~^J(}S#~_M)C*bLidyXhL_jK!dgWsdMQ^Rul zlLB}|_og2JEC7-Ata}F_oyi>%R?kTK(P@_vN3;JVg8H&&&}Zd@xdBv!KZNw;gXr#` zY2|<}kZCm_0i9RhLaSY=FwO*fHh06g-?UqP7WTo>E9j9T9`)}b>V`#No#^;vERkhu z&Q}+`*?H-qI?PZ0fQ>U5VMsY3!7CJ4-!p5^5bJ5*-t60c1GQi@SBw-adzfY_2M7S4 zkFwBi%7nL}E`lfUim~D~TApG?%kk!m2368P1P<7i|L=ydUm)iVE z*hwb-Q8_Gwe~d>h{gDl~8f1EZ9(LNYj&bJk&P)9WnG0S1GJBC+lc2`hZbXz0^Nxm^m}kVh^o+0j^c76QlyLs}(b4;w>yJfI)xx0rWiK+`!o z!G{W0RZ~eDCKAq<98?BA`TDe_ z>u4O}hD@@b$8mbQG|CW*5X`%=AD{5gPSgOdAa_v66kqJ0%dtmzdPzx`Mub;|1bjxgU5EYhmy~+P~FQtANs*|s5S&iVr)r; zhBbBhvw&?H=ObbUWus(}*UPC1&3XA4wzo9^%jnQl_b$g}$Ut@!yZWQ$uN^&y?C48b zUW>Mi$d2kY3ba@>p5z+7q}sKfS9haV@Xn%(rj?d^bXvqL39`sigh#h=mCq}IgaK&l zCy)^PW4JEja~Kz+gg^ZYit{pF4rc+4RJ_D-al!RHombuKi&T)JaTNq;paJT(D&Id--vC;C+~+5WKl43Af~-LC zRZkr}_Z)>Op{P~QDoQXg_M)-!5)RU$Vq|e=h znO?1#Q&_Qf=uyZ39XB7Y&LtG8LLJvd9xOYFY5&iZLk^F1N*ujuIP_LpY-ja$h06cV zocH=4fJ<$q7tOE-Ie2s<0bhQ>!w%w0YFYKUw1?yDCj-<%uOzC{luXVXHjrPprz&>x zi{kYe%MN*?8Ztx9;+4*Ss5fjl>UMc1@BlxT{1AepmNBA$*MmxsMt{5ul>>zjNCCks z%`W0Dxk?iIubK7}|KY?x-@$wILqq+$*Y*!z_jrO_dJYl#laa6`h1}^LblsEd^Mng) z#*wBxbP_Wwvp@QKTTbdOWR(d72u<~&eoh-V8Iw!+MR?oRUWi@r@qV-{;a~_1;2KcL z$vI0#*j@<_C?|*hh;?gE-a|W1;N8N|jA2t4QmZc>{B_FtyIcnj3vLWn(t(fn&vgHs zupZ1w;)}oJ{v?=xTHQo2dfjSm^1Q9W3+8(PyjBEl#6NW{E|YYza}ro|BiQ%{14g!Xf^ov!D@s=PWU-J0j;SJ15r%IS*yPi9`{ zN$e%#1;wW|@bL0&cY^7o0nEvYmr)v?S4jl!)f|;t)IjT!tGr@&m(=s)&nnm^-V zR?z@r{P(VsAd*{5l%D=ItblkZr?dFtWfB1sOX@48nr-(_8DG*!;NV7g_-=L$pRtI> z+O2<*QzYTgzwWO<-a6gi{XGPs&!L>3Z<+dSVOvY&>M0e=W%}WY2Z%sgx~dzN|6%fX z=nlqfJxC%KE%+2tn$Kw9%$PLWg-`=SThPhb8whyRexROV5{eQZLEi0m=xUSiFnj0y$M#h-#EwUn*wxv z{Stol!0or!t=7aJA3W{}B|W2J?I5S#0!XvmDK?L6s72LuY9@3A$ z86xLReZF5EXLbjxS)kar4Y278GkvGv(kXddb@4Gyfc)lA0Els**Q9~rO7)6^^teFlw$xZOcbyy4x;_Ir+Z6Gsp==2l8y|nX64?B;->-( zhZmhe?7HHMbO=C-V;fQo1odhwfW?7oSL>TCqax))nr;FniPBz^+D#m;=N~VZm7k{p z+5H(*A`Jk5gGy51YOrXYSD@44xHsrmcNR%g#4t3zT>vG}!iDb8@oa^f2yej#1YnFT zP-BU-!47t27Ao*kt_>N=KDPyDc+k8OWGCpgo_OHzgn(yF-$?!1;41 z#q*(1J|tA)CM88;-dFm9*XqY8+E2x9y89^tU-9)u=eZTb(qgDBgUYO2=@wN|jhX^T zAxOj|o2-8k5m_Ciyad_!#nMC_=h%}jQ+R1X<278eu-obo^%ZE-`gOQQ0%-8`dm%|* zLOCdHaRTj?e>`#K71r~Hk+p0R2phP+rT5LX^rdYG%er>}0HRpDgfwWmqA<6=WN>=mirU>WJImf`($0g~j z>^9Y${EEKYwpq1{u|;=?dcXDH=QmepU+t<%M}TEch-n`F+a7v?NUwjW24~+hkv{?F z9Ao{!=gcN1?JK8VGP-pQ=X_>Swx1C&!w5g1u` zh1MUGJ0^*>!?mZvS<%e#Zp$Z?)GlOZU-Rl{2!S4&i8FH@HX&AvYnQKcp%QP#uwLbi zzv-RVYdw>q=(1Q$vh~5|4X@qYr8J$g7e4;tfz0BaaG%Aa>1|I%1sV|b8;U37-| z%`MVPGDA1!KXH<{B((Q&&PMZyM6Y9Rhk(uS;ZDW zshF76GUp5r)NbBHsn_0M{vku8P3m1}8fe%Lvf(>@JtV;(N<5$|WjHRHNPSXIJ~3Hy z=KJmBTa!kl7UU|-r)oE(7w@E!WeqvR1GYI&d};y|U4?QP9{y4J!qhac7eO*YDqPlZ z*(`IeOJ!xnDeKY>S2i%ji4EF*<}MOq1a4WMbyTRogzWo3Y?*;j&ZKrsCv7CS4*B1Vy)#f~x8B}mH8yt$tC>xuVZZe> zs_4j%tc(4orqvOQEW}IbM~$bB;c~ma4*QPW8yFU%5wc24p6gcnkb5bDhk`@@lLFdl z=~U&D%YbCI&G7S9l#2|JMzX<#Ujo;_S~t@X?f&Imb6XheV%0~Cg-cBYh!+79^($Gd9Qb2o-GKF_9xh= z-Uer2S2t^1KD%{E2ef8!X4?7B9q(npDko_bSSPG1NQJV$!ALRhH3{1WNiGfzhK|sG z1Pvh5yodjq8)>pkf|64$nB=p7t9Qb#cAgm}fBxL1y(rTCH?OLb2Hm`&HE(DJIyJL_ zISm6HEydDQZ%NVj&tRv%kqSKX7?;6)?zP#LR>*ihMQ<bDr$zA`r*ms1G@*n38r2ecw|_P6cIO-n?vn&ugL^0(y70|Q}#Tjuc5>jYcJyo8*# zIJ1UR@4u^TO8gYUOu}!nQPyIc70UcJ_>%S>C)w|rk9_3Lm`Ap^XFD7NgU>xHi1z#F zZ1y;tqVKVKeGFj)eM;MuyYj0`nheyq)Tm+Qd!H_H1&d;I_|;o4NAxSN)c4ZJYAkN>SLZbjDD6Die%Dl-UA)`$TY6ip6uhMrr^xU`zWnW!|_(~LrIFjr0usZp`kIh2H);c zZl$}~R=&L$GOSq=JBp?LCEv-c@GYtQ+ZE-Ea#2;A`?po9nD1N?aXsF|@$3XP^NX$oHYPNsa%;^RjLQY2HO@C-` zDV6k@ndmy9Vxf`kx}Uk#EX~5l{>Jq7=+-B>Q(He)Zz^MY2q+j@fU$zwu3Oje7WRbP15sjCuG`cPXPH{+j4xqYC}h-}eDt|S=cq>Q24Gc}1tMLcM+ic`XB#Vx@Dwd-_ z0BKR_yOGflA<8jei_)Gr@6NcFUDDUlb0!qU-hT1 zW*IkKK-%&S%bN6n2zs9%cvkX9!Wgxw!79`N-v+b^sx!~2*`kx*uG)4NJ5b)M4bEM= zcZyW}G3p~th5DQ&8JCPJuN$9B^*0U$I* z0ZgVYVsTN(X(<-)M#sax(aN$YT0Ml?(Mt2PfG3uKImRe&zE1r- zo->26Fu^fPC0E(?y(y@dpF`MURuxP(q3H1;00GIC^{-s-V+?vV} z$-}U~kJ}|0a;$yVNP|i_OHPYxoZvPiAsy#0`Yd!aVYJA8X51$S-F0j(yVlz3Dv2rI zDQo)VR*iIRAu5SHI-N8z%vW{duzQiH&EEJ}PsKOveM~ilJWa0oCBm=t=aYP(DHm%b*+DS{=nx@}Vme;;&a2 z%EnMX{88*Uv*a_O?3n$H;o>$~QY~LMo8dauze=xgMljh&<7#tjBeQL}1s5s^*b znHuZ0zY-sAdi_zxTu4lnU(APBNev1kwRYpVOJ#K@o)OCyrL?_OO;e!QEX))2n>>Ed z?fAyrCzj_lJO^CX{3S9e)Aa1-BDB1Yo5`3xAv-}&LyV`2c!LOwS}&hFb9km z+`H8`WSuCIAaj%xHTBG}1LaV=_+VU^j!|Bxe7SBySISL4nB5IpkZoJdGv&iXM@3biup<-z0wM!nOFR0<*Mt2kHLTQnUX=34V* z_ogG9_NqxVlN6B|QB|bFG*1Ys6gMuN*QVYDw84t!3Utb*9h#+;Q@(Lo0r>JaZeqAH9G~Gihc$tlu%&})@<|gZO;wA zW62bkMXtrmU~y7GhRMCPf1)uB$R6=&`2CKE?0Sm8lm8?Gu@Wu}?h#|^2fVZw4xJ$8 zR(i_Zq%XHzJ#Trjg`zGbNk00lezZ)!)x4icF!|eJYr)4CV~tDc_^rttd|SodZ1828 zHr<-nVoi7ZmT_R1Rg>yF19tUk_E1V9^HUjy>BR}2(bnWlGG_j@1D)qpb&K zgVb_sl(|WVyNdf%<8YyJu(N=bc*J^+gdrTu*>d)W(ueF{d3@0>J8ruF}fdP<#2^+ zxl@}1BsD4~@fA(=#JZRF2QpVH_NWqcwWMrNf6Sbu{)Q$k&>%iY_u?W|Odp$;ytkdb zMYy-;stxaW+S2N{Lv=v*nR7?CZil^9E7E1mx+m?UQ?-l|iJ;TUAv77f^U6?y;3ww{ z=e8zO{{dhxE^q2>h$Gy+xS&z5{3b5`ObVga$;;4tyd02A7g*;^7Qv}8+67*qN-t8% z1+)fBf=rNSv8R^CAZl7Mu(4GS+#}ZJ{DI`7Y;R$k{+oI{UsOBFDO-jGnXhEsZ21o zq$g0Mr;?Yo`o!#NM%QnFo8ZantkEB4yc-;ImqbE(Ol0US z90qKTt@-jH*U64X=e(Z9P90p;+k^=Y*X#CIOnyG?aZQ`F`o!{mfQC@5s3OLH*p&g&J& zK5`LRMroBAw^Aw@vfl7#mHpPVP}TBDLrhII{pQ;8DM8YnwdWGsy{X?a>)hTjIO(tc za0oR0=9!VjdU~nMj_rEQQE7vw|$ zL}L_;w`vxEV|@o002O`ptq;)YRr|ZSwsGSejZ;@CNkvTw0{?C)?Kj=_^_$r>j0&q- zhNwjtH#*!ARn*)6IOr~E#dK%x0dTrmsbAN2is834p)zVdEE6qQuoY>F?o1k5mgt1O zJI2{_7{M6|HE^Yc+3^A_m>2Z^y3=QO{~q9g8TA$y^r9+8%C^>0N-whnduSw=J5ID- zZelz5c+>aUV~_5f(;{qH2Gsz$b zo3Aq*=RFoQR(GQ9)4TUtkwRZ|%S4Y8(d|O&X&3fPCv)y#>8j(8TKrTiaEv7~_|1^r7*# zl*iTjPDPHq>q-n22-$=N>{HkScC~W48BbEEfIaJYYpN;Sd1!L85AtN1@&MqIG&#U% zm23f!td6gZ?GIIfh7K;)*Q;}8gy z0k}U4FE6iAbUk)qx<})VsSBFubgs+y&4a-VlhE1ocE(+?5SsYHpTJEP^ao#1%0~-k z-a)knF--pWRJmB#0+D}J$GHJG(Ff1oq#6h0n-iUvc=#WTx`QD*QFl=>UZEv&YBTbu3RrFp zUjR1sEo3>ag0A? z3t}trg_{at(6Y!`|A@HWx*_?}bOJh2=I-od4+gcKc-iUndz9e^H~p%GJc@Tbf;+MCoLUnHd#2uM#&7VPi3mdJ8i;G8%d%TKH3cu8*f0^$sc> zJoWWkf{lp7+!y7jnu4|#_2|BsM;RKpZI4E(GO^s61m&-7bDfv-a*+>EPi?4^;~J~Z zXP9ldeWyd0am6f83tb<_Zf{|WtgTCKif~09k8xol4~A9gl3IxnMN$S}(j@)qTLjut zRF8m)|23~p70|LDL8M|r7E%Jlkoac0@iugZEdPDFu+67H25`6_}J4)GWvZHQ>!(>{5bcf>Pr}x zMqmMj7kKumz>G-nKaEm9Pjd3C#KDrfFsfu;7L-Zu4T3jp+QnbN4+~K7fy^l8YY}Er zS9J^S>nL9iGAXi_9j3A>Tb~w?^44(j?G90nF0~o)>rDVS)=_{IEkzgnNM)G(;B!Q_ zY)@#p%wo{pUaJS?^@6he3YX>DP^ z9^2A~J`@;?vg)Er#%$Y|@3(h1EOIP1uM#hy2i6`L*j*2k>C?+jR-b55nviw-wuS4U z?@E#Mk3t1$S@ahAzWg*7C;g$5Nn_mQoSfN4t!|g3PgzTwSo`|&sqdXHx)No~IRY89 zHP1acb>w#3joUo&F4P3)Nd)5EoSgfLweuO{)H>2LOlMAC(qSi~s--@%%qti`@Ex*% zxCKoF9}~0^D7NH+w`I)_KoC>filN;h%A!f-%e7>P4z=ye@joRZ0m|({MntMf5NBm= z2pg&loFzur^q}9xJC)RM*fl*@xeit6+_M2tI>9}Wol7@BGThvTl!DDQ@H-GDn~#lC zoH#*7x^7$xC{I4%@G*)oUwsaTOUfl9+T?tv$nJ7YQ1%v)q^f7eTydIuuFNTJXo34V zlq0Tms^*!)T`j*Nx9w6CQ#N!J3cX_dvL!?DJMO_)V?eAE#4}DeyvYkjnnIDMg)odk z)!D7Gqtso)=cQ2N06clCkFjjjerv-4Ts;yN*_yM!hg@4f`5JC`e}mvcEEvAQAs^AZ zx40WgJBnc7JGg85{gDFYl)wxf#e?*T3*joHH88;cMDX!fXhrv?{Y1<<8WCgRLVI&h zv#=(h!b42)l{RSH+vhK=<>1e!P+4 zTA^~HNOB;f>M?`=N85YHbJ@TD!x2J4GLju4LMnSxk-cYD64`rXk0`Q|5i+v(-ZDdW zWp9$b$qv8cY+Rqu_i^3#KN z+o`@pGpVLO7mOm`nAwU>3(2MBh54*S~)k&)fd|A{BsCU;VEjj%7u7Ho2 zQP!t~&o}#^ZF#@r;1?cvm^9Hs@c9D2+YJ4~**yHF%WD%u;8Gbt%wlrc0lM*qZjWlx z6;uP#;ifj*9#IHIE4R@Br2e@zc?P!kMp&yV#%iLi1XEE7YBMcCw;!VV6ky=~UAbcj z3t{vD5B0xasvEr2s-D-h=m@DNuO}5brj5EI^RLLhtHf0GJ!bo^zc$lZRSxRhHwW0& zdlWWvU*pzhOfnQs-aCEyx}#og4eIiP^F}?o>?*!47WTD|A6%LVUo!ksXn!_E_INT| zJD1n*%M@%auV=}=%+GJ`FK=8w4xsySC>xL>5zdP~QR58FN~yAi2JxO*vH z+&vr)l8B-ENz`y(F4|{+0wYoC$ns0R`1#|LN>LO`gqw81FrKyu^KttdETitYFP7y{ zSl$Qo;T@=XmO;b5C3Al~s{4>g?yN#}N|;H=L<$;YU5I#+`T;6-mw9Yb2JU?*a)WO_ zEX$1$#%e{_r|O6ck`#B^S0(&j7#^^u{6V3Uj#qqVeuaAtOofHPW&=^(CR?toGtS~ra%lw-tyAttU5050@{{t(a~<7z5~T{$>81Gy8HbBsmhp~kUgjl9)EAU zP0sJCi0dAD9{QJ_Wn04TAbv3yX&WUrfw;Z9svVqb8A+rpN>WYt0RUD zKryv^?!l4oL<{!IyeE5J@H$7bFcB6F5z_!plc@6J)L4CHDd^+BHy zv^sh#v&sqr-_jg#Zhy!F1@^Bt5>e%;S*Ge8FE24D?|g;*aT*paZnaTo^hJ2>Uaorr zg%U*lPr71A{RE%=U@S7~u`mvoUrQ7L8^DSciC9jS?s$PfMunWy8yNbtd;nU| z-*Mgic5j>JosJRyz*8oNl;700|I~W_p0fWWagIo_BKUR>fjY8A1$Ec3En*Vh66vey zj~8JvnqzzWIc_@OmvyUAY>d9PkLEAqg4mUKqGPw+x>Yu=>7y>mqt1_&SuslxUpEr@ z?bzWvG-t77Y-*56>U|HMXE(bPHF{T?hA1%Fo1l&$`}bk3n$ zd@b%Ri=%BUDkTw|@X0mg{I9WA#;Zx5pC>y#p8t8U&E$KH5RSyVu#oH{fLxIg5dN&` zRFz3J=LMuoi(QIXN?Bd)SBZ#iXWLWP1BsX;pLIEnRaozEYB76}`$imvF~uTbi=|A~ zoNO$o%;au*#3%TDfiZBiaC!^DSX&tyqkO(bmIwU%C^o#zn^`Qt-@|g8QY35YmvcUW zno6>F9t6Hk@Aw?eaA`#BGXr8674|2I^=$Q&W|GB2l;-eoGL zDhiR5v_Nx!i$Xs)e1HUTn_pxIFobX5Wcm?K(VUiusa#*X+4-mngqH_hk(4Kmcv@Wb z@dqCIEdMzc_YgvryHfDK$gYG|vB-o@{w^$noT$s;(8qB3NpP@LR66XXV)tdmS}mI7 zOTYT7^BYwpYyF6I3yjQbC7HAn$%$sSI{duu1kg=XUKfz(GM~iWG9VqRwZ$vGwkJNW zNmXrU?nUeyA=zk~=DS|nu86LdqZ7?^Do|5){>%I7x?At0_E*RkVfF;sDRqt`ZQH7~ilQo<{t_3;=WOLy`QTKy`_LUn=J_tSs zq=TK>T~$;8Oz3K40wt|`bplwiYs1Cjf8Nt7Kp`qqijAXXOevaz9k={}?4wK+#xk!Y z8V*f$`eIXK@_j{TKE6yYceb5{z*HSPgR=;$A=1rO0%yU&Xy}%tQq6Y-P{05f;{`E# zXpjkYEtP4-r*Xh)NfNaKUPymW3J^RNTZgm&wdQj>w7X;cS@_r{p7RNiQ{*A5b%mxL zj5;;k-&)JvNBz*8fw`GZ@g8_y9KT772-nU5314N~QU2DN{u92gA+%|?Nb>7Wzrt76 zaiXz1LbsV4F?8+?DW$Vr)jBbGK|yf7yxUSlVkLS`X{m6hl*}O}>uR%j($4{!%SN)A zM;*JN!Ye;?-9M7pGxONKm}o^_on~ydybK{*3|_3-X;wn-aD?#rM#;+tt6w*#v%JUt z1Win8j4xBS?{iCignjVuw1IsPJX{3szuI4)uT(C2_~BXUNEGaXpG*e~+hRDFgQ`Qy{U$? zZlejRSFiZmbG$NMo|q+-d`4FvbAi5C6Kkf`Lr6TuwA9If(yTNs zpPTTL?<;&4bwnTw2}G&{jF$ab5&D=1r@nyV8X0dlw}3w>cI7(MGnUh%rgi~C`(f^w zN6u|kzf?Ql`PK)T9x1;|0}|yB*f_YiLyaNZDraI!uQ|eK9WGTQD(G70gS-`LpE}V! z7kmnEuX$Df$u%euTAWw`dE~z<5V*)|P5Q*{jnK9Rjo$p{$W!x;aLH0H<{8MD^dgoK zY z=E(A;?bR1>%=GWl(9rlX#X@-6nkeEESIY%R!)#?|yg)VOZe*ZL__d&CyXM*z%rG(I z&Sn51pjC?>q_9f{X_67xipN|c-e}KaKM-N)ueKGu`~AA6c;Fog z7s*;n# zezLnk#Ld&}J@JC#agU#?BnXxFh{un;rPwVm-3BoHQQKCKda2V5K_;a>Rg!A+4@V*5 zRJ{pFLXN^favoYGiKpmrpL}&C!7aqHHt*%ja4PtZLadNTi5`(NVN--F!E}75uKiRp@ z#|(>Az<%jX)_YURjowBN2a92&u8UwA|2$nLb^xC`BA^23eRhZOt``y778a5)RNT+J z#pk^BSS+?|8X8*6X^WN;U=Bt?K05=Tfh>AMsR94a^?rp(mvw=r4Op9M+s4Lry?)Bx zKnJQ7jPd;r2Dw3y=qDMp217g9_TFa<#YC{aVKcs?Y-llF#o>oXWx*M9tO9Rz)C%u+ z4pEQr@1IFV&itG6JcdY3$nbgYenv_phi`6SE?2!xrX7Ql19rlU``ZxWRrgye(*ZA= zZT*rTF78o`=k*I>)dwFK;vQbuM}IfUad$LwS69UhcCH);bRxf31kKh?13!JG)XYTR z<&V&6mKayqyWjry5sQ72V(ze=V^luXLF&y(!A&UK<3I7(#!Z*iy<4)Nu!sCgmxJ%T%6a|d9u$JlxeLWM zl-5;^C-#*j(1ki?*XyVf?ORcz1x_cK>Zh^JYX!HNW8b}t!0!v2>exfXp=YwuG=Nm3 z4;`56fwZKo6nR4fN9A^@`&L6a8Ym8)CJm4i;21zeNS+do1<50C@zB?&8Her3O z+tCkmBUQTZ%n|(X&Hak2N=!KZr4q&^fK4HQPLe5X>0h8}Jj@JsggtEB?>by%&m2I) zW-;;IYP?7?Q~5^ZnPT5(<}46e&F<5P`3W=4#IO+@lHb(xbPksrKGepCLMaXgK*LXaQ~Df6f4ctM$0EL=6R0p_lzsDYiLt`wXiieb!=#k1ldr}A z_pS!a1{xTj;(mI=9%MG69nbIV*VaptWKgaT6#EG5&+a;{mv0oh6buJLTuZAq9aHga zmAJ~@H>(R zT%qgX;XCljAR-uY-C2q-3BUQ?m!L^g@(m+O<~8O(wB;hh4nK&Gv&=KkT+i2$i@B3I zg1byri`yuPKF067b(vL3^aG1l*_{%ekx*)DxKkW$giZ`#1`Xibehw~V(8!?)>T7+_ zo#DsgOqkl>V2;KmAeaoWi7-V;K37$Xgzwjf0Mk#5PM$@Di-JY*={7>Ei_=|RH^Juw zXmsfy+=wmbUSkH_38!7Se6lauDhrf#sRW*(IR|$&B@p1m3wa2B=bKx(YFi8;%g)M! z9UZL>IO%9abUF&sb8EkHT1^StE%iN^)Xq}AE*7dWIo(2pRS~167lB;}f+)^om$~!y zk)fgZ5HhJ8uUPGtK#kInA8VsU>yQh2=>t3a&ibbE|Pb#;|0N&G3G{#pC`MZ)e2L?el4F&9j3mD$8Y!D7~ir^yH4G z4YW$B>6az&g;*yfFH<2H*|6fh8+gSd#N|ZUWlJheCAfQQLdyCNNTuRBN#6Qo&K%Lm zt#!_IISdM{E-ueE|76YM)Z}sHFaLef;)5%MCfGS;J=lb9@KWhdz)2#|K7xyMwoyW z#`pcngp>yaGc8?w6*fE90ET)2@)NOmc8$XwV=va#I+f+=gVqV5;Hjn%vaEGa*TEQ9 zB{I_9bcIPdp82=7QC7QSL~(*HSbPo?@{{^Ja!?n%X{{`$y#ML)efBy(e>^dTf{*EG z?5x@A-pUePz;67SBAX|kh8mN>HOKTj{#9Z+Glcm?__6C6!qn-lm_@K|t`#nyD{PUf(V{J=Y~=-9qv zCM>?QEoC~-fbI%eiQt?}@{(1qR&{oiRNnqWNVcTMI!OAcR z?b7MRMk32{ePY=2xlk8MlVQo1vPnoohv3drotCp@4_19iU~ zL>lJq*lx$5N>qpT!C9}fRh~q87Af1vgtzQV8GpSNVKsYn&GtHq99Hv))<>8kHS^(2j0i(E6q)n4>vX9n3_dGRh83%(wVevmauQ^Kmw~I2RdrO`nWa&a>IAau8@a{+zbL!BUK3Qoz)8 z(?RcNju`Ms2*#~)o2Pts!aYD}aVqn?X>}*U(=5tiGSQ(cq|ec;V^>HSf1sugt$mh{ zl}1tH=d^PnbS=o!h))p0C2x)}<+^y~u#YgM+jA_9sY+jkWFQz;M-8$%%2S-LT-m84(FGYvVz<(E?f+z8=JGz_FL3Lx>7 z;Fo|(WjQgmR@jDR#o5S3NZEB931_U3?iycDARn3$pCUe;PbaT#ptR}|@0 z8w*CR+R~wzVY#QtB~1K~vME#+)`({T!U!3sv_P>YFn2A-D!$BlaGAJ`?Px&TPP37c zwmB=v??l6{>Ys|H9yC09!>Y}@ea+qju!RpvGMw(934%gW^Ji8sz0J`1ds=jz0K1tG zeT!V`DgySXoQi=UBqRee0?Uf`qR;;g+2Ro${p~#Rh<|6}#7H*Y_({ZD8sP(qv~-NV zOhLt? z)XC+nM~k4G$TDepzBO;d^wVTnGdO`e1X#RT*u^$-kWDM77^PyHKeuTPYg4|+g-;?b zG9+9>rDOXY2#%aK-F)jK+o%N6If|(P{Vzc|ot34ixmwUm?NtQrmdSS8*_tKp-NQQJ zsoodEcL2n5A8Nessvc*$GocI+URPA#i4ryi>X`DZdiFb6P6-ZqtX6ku6D>Y^wfkk> zLx+jP%+}vNDxhp=mYQg+6i(oP1|lyV5P{H{rs}&d3D60=OL250o%W%}ZWCmbX1FYq zALt&u&M@h@)e{+rRt*I1X${L8x9XtyQOeu@rx2-L>fb*liN6bkIi?L0$}0mca-q~H zAel^UT!6gg=Fg%hfSDrork}f`mJ9vaQ4W|s1yyK&$s+$-^8nH$>R9;+TtMIkA!$6^ z`LXdW4k462V01|K3`A9|R7SN8p%l19emx(z2Pw&QeQANc`%9A8IWdyVEyjLIZH=GM ze0Gp|S`Q10Pm?Z%F~2G*(V)>6%iJLX|Gjd`n=_e)scNz3e&#FaP_gwtqYqtaD|oLg zc?|vI5plP8l+P>D+m9g5Vg%x1mpkB#&(Smisx-5$b2b1WGP}hXx4p;LiNCTIpsYIp zF^uWc{%n-)H>ejJa__39Xf8p2&rndtxoG38PG4v=@sS8Eg2OCpDSu$DGhJQkOs_&| zANqOXM44Dld=H~2=zo|o&kk+Ex+w{!-4)GM;tjB5V%fkJKy8v{o6~4O8>_M1NpZRo zLVVTva(h;+iy-xq7YHY~t6OS91R#2bBgO3}uT*YoOtumIt)g$2$rJ_Dvb@X8hQH#AMo7McjQ6ux&NNmd@eOr+z&6#AG&2*A0J?0d zG^7RfVyMvvK&O$zz2DvC0)miK5TL=wXJxTg8Fmo!yXFtUW2g$1 z)&D5FV;r%}6mm9gN?{M%ILbLmyczp_DaD@KATmRpe1vo%3 zQKQ_)3W(>n7JW_%EzEOKBNQ)i)&%-K>PZmF_f8=A6dq8s#Q<<=XlQPJBR-Z3G70*E>}*6E1!yL;YiU=ui<0Uigw6{*RXg6;dUc!P;n9#M5?tK^sLi-6l0-+3h?t z#((bbk>bT1dRxWyyE7N1=%c$A#$u-;5MLEcR5(t2^az~K|7Dnj!ja>t4ZmIeY9)RK zaxyaE@MbB*C}N5R#mKvWl9$6D`M*D9v<+`q2s*;MJJ;t{D*Te8s16TrIl2E`NvQIP z4q$-}(cxix3m+}KfoQv+djp?!0$jOP{9OV`1sf4N8qn^JQPobAykR==%0CXsxS&n14? z10icpR!&ENCKSZmWXUFYqL{=Ugnnui(HpEObBbjU+1^?i&T93 zXDdaX)af3?s`)p!>HbMu>y41D_SJ+711DbNwFmgUE(~*T{|*W!s44K`mj}D3b7UKS zZ7YIE0mSGBf8noJ)f(O;1tuk^I*GO+)0gkKW-09{lvq)kX>oOB)~?(|(M9l1DYy7o zsNMkL(eR zuJg+^W)it0Esyv=cr!92zl~Ck8unov8t66%{yaPil;h0K$PlwZ?DB#T2I+pl(>a+c zM!1DQy%pKt74;Y2k-ykF=UFZXzo>^xZcv^5J82m4$KY=<_I_qn8-PcV^exLU7zbnm z-bZ4e7i~$%S5+YQ$3<=PA`x+~SV()rb@%sY{|^fUR8cq`J)^Jb(ET%GF%ci5>9q54 zXGwi8#8rNa_=B+ij|cWTg+kSFkGCn|XZWF?ZvQV8!1FfT+h3}Hx0W6>wp-{;51GCO zT!UZ;EXhXxLLJB*<3}S;GFMY+*AFG;kImh`=Z}{J8w^s42O1_k$o^ZnA9qfg8uB{C z-w8ZJN&d`f%*)7Pc#`Fm^v|^Wi!FL4@x&uT+2j@6~VMY7`Fb8ufox2~3?|5A4|ibF505*y$W>tl;&>jNBbEufQ)$ zU3Y1F44FJR@Ty8V_0Pi}v$y}bk{P-3eRJ)E5Gqq}oouc1po?qK`+6#AcsMFroqA(UmofHYjO!sArXL+?c5d- zxG3Br#rt!~T@$#1cIU_T91NNevWW2*o=1JRMaD%CJ?s#5j^$yH5H6(}MsBQ|m!smf zpgv0EHF%ti>}6)2qTlNWYv5Qbtl%Y^%YM?|Jyyw*uWa+qQfaBC&^!rV)cJ^&WNPc!nFPK z*j{IVvBN3M5FGVyBLgdB?J*>wR+7P*=Xa@b_&u=yxX~eWmijPwkYKwCwj$V~$ef?J zZx&rpFL?Y;%aR*_Zs{w%%wq;H4i>x#!0p!-nuUAC4m^ZrMxwn$T$zbt@0$A z(mp!jNBQ7>WqzRg1r;0an)dj`HFh{9x!%d)n*C2Np#T5wp_JMTlu${}W)&UQ)7(JZ z@3cS%G6+9u(yv3SYDrIS`+f`{O`=};YY`NDMWvnE`Wu=)W`LRjn*czPN2%JR!dl2^};QdiG zxNMg|x~CL7|Ni$PMXQBZ<|X7`1F_0~jA$yv;UDjDV>B`X3BPZO)p(#HZ(NG?|Bdkv z11b=sS^&!JuLR&M81mxkV_rZr{6YIY4JepHD=Tnt0A|y36mmp%1kC(;sFZAg2e{Ub0z&J3VC4$@AkeosnP#_B-OWefgRH{%8&0K5J5N;*_*y!>Ksk3KSkrG zfF2y*u?Vh7zldVf<-J42ZAA^zhN%KPM`X9d4DtMFM#}G0(4T}DkU*6+XJq*A`GJ^A z-uE7U@_*)Twqt~ijTr$*2!fid^Y=J{TgT8zy-A=f9T0JS@W*HbwT79hFOo#(J*@QB z!)v7EbHIU`S_^pX^d|tNl?K=p2y}@k(*#p*8A4C_{)dt~V~Ft@VD&G6VO}xMb&Jwz ze?vb7TG4NUB;-~3`lgHf&Jq#Om;(VL;<1Mhw}Au}M|NjNIh|HKSnuPDixt5B=WPb! zhSkDn8rZ~R_Q8>F1H5mp0xIuukeal?I{`;hwfsrZJH5JVDRNO)NWZ2wYFQp1MSabCH5*tP#(}wX!7d)?kGQ-dkOmQ9ypI1{6BlEHsWMQrRY@4dwWqy^*cEkn`qDtmUvwTHz55ACDsLxfIV0KbAq;32U13qdZM ze&muo1DP{oq5}us$9Sl^0EFZVMV=)1I9AH8wO`xhR6b18!{yG3%_@kLX|MVJS^syD zY?-!!Zkrgk&wqlPWqK5k^NoTohvchOxD9WMqwKt@65fb6<(udouFNhB7!lHKFm(K~ zkJPIrzsy-c8$j5ZJ0dEU@LnqaVsyC`$J*G3{E_ceAfRYX`SRf*Ho8ug>sjVOv*v+! z9*^eNd@9VaFOO($&9w!1xs4)DZ04(FqYhn!4!7&5wu3X3=v%>fpmp@+r5LX4o#qZs zy`>yOr`?rGeO<&DD^(N=%tL~cv`04#@pnJAZ7*#Vv z7)Z4ih3sy*>wX|YIIi3CY&cO=LOSH%+rzLD2C_WYuk9#6%@Ot_v2$_XoAdKN0Bq_#oMYl{Ut`H`mZkYodo%zw$br zm=9a_Oe~uzwf)lfxhAW3sDeo7&9K7%6dxPZ(+^dyy6$fzRaFC}6$txe5;1qKPq^)k zTTO1lhDn`N2ZSyV`I+9(D7vfN3Pki8N(*eDF2gPgF!qHH*oKuhv(z59o^)n3LV8sr zZ|ijU<7sCc50HxD{oLw}t-!3U)3&y&w$%O3vvs=Nnq?|Uq z9Tmo5$?-||%~PpZV%?Z5x(_WGo5dXVU^3?y*=qw$f*K4p-MG4azp3yEjp6lCP3aL5Bn8Gci0A z!E3r^rX+_KqF=-d1tU^La*RtIs!w0ZXze~aEvpO-&Hq3z`?c}KMS&|2kLc$IP6!&d z5J+$+h<8*OUV4ao>8NS@{dh0fdsO|l_b5q4>^&aE;Jv+s#nMr1^R3T$%wuDHIYf%M zBSg352lnmItRTyS%Jq=sj2{nd$BvI`MH-3>Aqi1Ruu;vGh^#(GlOju;_ib zlcf>Vih=E<1fhhb$@asr2Z@0Bh1O>()4q}jHHJIMfNjwiu&*+_0T4@b>WH=jZYvJ@ z3wEasJ;XYSbq|`CsmxAMx$ie@v^zWN9PX`K?s2c(gz(bz$HwjetO5lf8}^cEzs}ys z_N{`lNM^hTmcn62BemK7llLI5yv%8)?t8k&>}oWx@Ovjj96<%>m&Ip*WhNfpAoa&4 z7tIr9?sl7IG&&N)AW<=VnJ}d1Z8Nul5i#3zE*?GDN{nhn=dB4!-#w{CP>$x5~ z*c5uQ+EvZ{P{?-e)-{>q3qF2w94a4Q27D27uXm{|zFL}z`q)M(^y|RYJ+~i28vJf2 zh0p}e7Gk|C42Ft;$ZYna+Bi6DXYN$0{)nMKTp zB%DX0ma;j$OY|&9BIn3EJ(q^z2aawh2vKGyXN17Y1#FQ~K*Zeu z655MG?yfUIqro6-!t4OjGW?bZ){oj7MKZ-ns z!1CAoX>nxIwBxDYhm@)`G}ctj_Xk@)+H4hHU(S+X3m89qlFl!>}fMN}uu^X_x%J@a=4 zT->InujG~;KYHIY{Q|b9m)SGIb6Hi*T~4{Ziz2Sj{Ag2xC^ynaLPa-o?FeDu>7U|KiSs%I}LC20U8zd-o}lp2D4CQtjBx290sj*?Hjm+c2It&QS;RqA0ou zQgPls{`M{S!8C4G52tbMxAIyHTt=w#!+IiFVmH&8hBONwp=T*)zJJD!Nrn;GYyr*` z!Zj5K`sCapib~&EJ`(D-QOn_#y#YR|smev7&5*qH`+dgU{twM@pClKaPCZz7lJ$9{ z&e7D)thfF?UaL$~Yv_-gh=tsBYDM40xWb!lCZZWm>pBPZR#7bdQnnQ!E7^SEt=l z<(p9Btski>=-)?4%BbTrZ3>G!fX@7AsWWxqOX~G!F((yOPRy9gOZE>u)apDm@dsji z0w1P>++^pjC4IeaHOW=O0J@^ZzN<;r2i+2>D<5KwW=@Cs&NF6)m7Slo)F9Rv8`maL zAH#6mYaSmTx3_X69UC|IY#1Ncw=yb>IPJ+Deq80E`Ff4g=UETwYvnAa2(5_EO5l6!#j6#cr7X;Z z&qNqXZI-ccZTJ(lD)k-D0+tx%)>ki&&6s+Nn(#ZdD{yVoaPNj=sF=-3g%#qg3AuU; zB~Xa(V@{jR**dhwAGgY9DKdlCFDf&MGp|b5%7N+Uv~8=qm{hoAs?wr`S1Y@g4wgqJ zGbZ1MSe@;P7NmEuh@(HFS1iqH;p>3UnbC6I>$_|7(sbW2bF^R*nScd3|DA~x5 z4nGR8ASj+%U1f=T0lst0dL|wGWA2;Z&~Yhv=~{%w1B!v%DiQX>W3WJ#p%*fG zKQQvp2ay@|%}k6@=zVTr-4?QGjZzRhk0dfvMvO08lOS( ze3p|!ih=IQmtrrj!X1Oal-Q{o%CzV5OVtY>Va@CF;wqg^;g-ELU@kh zVfb{`V!S}oRagN}r$pI#*2hZPe%oKy=J3U8>_GHlD6N_fo!r}&8!|l@N6~KPq_f$5 zclS3g-Uz3#pc<(=4|<6&Tg_@M!Y7T*CFW|X-zy4oC;a!;t@i76_Q{8Q#PQaq`vd2l zj+dTq-Me<<(Z_&X*NouQhk*|i>XW&*h?oK-!jgybq}Re=fsCbj!iu?;nj`V+Q7Z1C zKs%&+_chua(}a!B=r86;MyY5ec^72Y@GImNnP*6&>kA1A1Y=$tKMh^=?0HJaGD;ZH zPI56ZDR-AwQn$wS_QDt`gm^hR{KSS*M1=Qh-4Fa?E>s(I?4JD;$wA>@7X4D4J((w~ zd*kFYE;4s{98Y*{2y>qMVu3k$1cjf+ks{yRbW^%SeC8{wb8hZES~6wTIqJn{%htGG zl(Co^d|ROO3kDZ8lDn@iPgoq6Iru%sA6VE)2dWZ>j1=6Jg7gKWz=-uAhJ?Ho-!-(1{+a9o@ceNx~Q=45QwyW5Z$*>^nO&%TtdG(Y+(m&@#BThVnvg}45Kf7Y zK!~{sRBB)>4{Ba4E&)xC9s~rCMP^2J+nbQ^*gSS}axz{FR>k~kzc!{3HL$yconu`F zT%a7Sa$I0-^y5yJTk3xAen)0HkaMxS0d2i+%27I4Kr~bTjnN@aqci!r6_Q?4&w^`5 znU9(SOst!vyt~0S;gVZ>Z3sz^$F)nfQ{vRypnp=Yw4pZ{$az}fu7wV|He4NA$Xi>= zEhkYQ8}KbiNEO7LDZ7&AS%9XCMX(OyQ@|cuK%Kj>*vn9AQcS;o9rzqX-Giq`lc$}# z!z23}(`Pa8C~<-I5Oek~ESayU>f ztWjcm&zu@loAuzz6q+ZWo?|eg4<-{rVN%WZk?dc-`gH3}Ro3;G+blZ#d#Bz-Cd(oz zJ#SRhhS#t6wiyUYgfU%s)DmFepc#jeKV{jrc`~lW^1;2AbAeqir7YLcaHYUq{f?2O z=SnY0-u;HF^k-A}BnwumBUh`FwYf&S1%j7uk6UV#Pgv_qiT75CE%PP~ZqgnaOs<*l ziyypBWoTryWxI@XqkI8skIibDEr)Uf=F*F|jq=w^tvCz-hS9 zShan4-U8oY?2`qU@x}L!D9FCrfia`Y7fJnn*hfGgd^j9#; zp00TDgQvhnPmR8eT_Lg>mpxhTUJV~QnOK>51>R5$Lo`lIsi;qW7s$Q=SU2)ip%>Iv zv@$VwoVMqJtcdSC7g{9fa0ez-LJ~<>-&y=;SAX803YpTCM{7{1q8&kGu89EcmN^94 z^?&JAO@qYO02XJNgXyAO<0@3^F|;o^(T9q6!7^s8SNDDbKKKLegsczME1JMcUCZ+^ z2sQe4hc!vrAE9CTR54=;1AT5Ja|zWY?n_}bYjj#6*fnqLu`Sgnb^}aAAe(QZ6MX#q zJPCFe*!Z#N8O+f~OGs>kdi$zSFOaUh!8+Nx-wKCH@Zm!?!}d1~;97(FX5Q_&XL0T^ zv?P$AzR|uZaY5Qgw&`c}vz;s}ah1J$uP$G#!2l^1&FeUHeXvhw(KDY=lJQ|pu}hq# zTBj$Uv@@c=(*YLy)fFW2j^ZL}+B$)e7X@9Nx*nQ=l;qn_84;O@asq+ia|4p@*?=$(>XI1&0p!o%$+z24i$4~(bAr_rBp>YEV>&az{! z+O8;FN@jQ2H5;lrw^@~RRP=PCnNY{n_kF+K5oTVtfoH)fyeYJl{JAhGn5+=SDI#H< z#Nta~Gk-LFZ4xe*?wW-$mz(Y#`hukihF#sLux0uVn2XkP1nL{T6&nn7nU-M8P$b8(?qLa4n@_u z2<*!$Q`xsv43brR+ke8|5ix#^j0M6|*H(zdcKOyGI-AHMLGtL= zP5rjC+zBYr(XNeqygI^j-8}KBJpZ*v$X_Ij@Iff6xy>w!xjMUcWp;3ZNnn|U(2FY< zs$QOJUi?ben#x>0rNGF*nHQZRtw5^KHMg6JrhiOlkJebWdJFaTe4gm34ojHMB9+>s zz;A0+X8oKi!~A6Tu;;(3^cNfDe@?uoiz1$l{-`w|7-W*vD(!RuIqz!BD+>>D?TZe~GL5yW(D zTMGTh^+Xb1`VxtA61>M3t!y};NzrrQTY6#Kh!jx|w!vxPZhw12SQByn26xwSP zs#t-U4L!7I?F(}c9KS+ig_qiPpB`4(Wllex!Q$Igw^K;%xK4p`mJjp9*~KMr_tFc) zQI+W6RyM7&oM9dZlYy>Z|HA|V0UUG9kp624B7T761hG~foApY5R zev5s^q~OGLF*wpr9**wRvRHZgWotqR?g-t|c1gCCEUP#OYGzB*xE3&_F#9elTjbUz zUN%BAUlf)cRH~{WIHXmVo_eiE+iDImZRtr;8US2GgGWFyy_@j4E8GzC;9`K`XVOTE zOgq#$?n=AG*Pao%@6tb%K!}Zft`|J-U!0V6x}2W4O+oB&k#e@G z47Vo%Cx<=xY{k^N54l`T9XrT3E-Z=6PXmG+D9CUoharP#nPE&KF;wZE7+MAG!5V_y=G4m?Oj_hpxTze8sYj?nJvzgSymXl_5(6`4f z`MpaoCMCD`gJa+xskv59dsT$a<)P#ck>LMJ+c4iKP%aXAzva0OMQMv)QAAt3*iEA) z?&{oY3+-Id;KCg{?rO-~U!U>7wzKx~5DT@VBsvb0)z42NIBWeVcU5>rnD2DXaQ^RGmMd~Xk!kNIdH1%F%E+^%rj3)@O$nd&IAp80sUw7vOD zvVp-YP5G%^%6&2_Wok3|haPaG3RZr=vwqZia|B)UB(pd&*!gwyJ-v#6J)DEId zVR=SKrxWEhC}+f#h1oEr)Wp2G&?>g(y2vsVFZzPF@_-|HBPF+$u&@c*Iot1_XwIae zqGNOSg~^6PMU+6vs`v&<7Vv8q9TdK!VBVeWNuaV@nf);hVoW=1R=Xi}W>Ri@Conod zdUK#WU$plq$q6XX2d9{i+jOY@c69cHzPZqo>Uxp2i6igK5?13QH2cg;Vr-bExfZ)b zUfPY_cFi~|n8DWf2L~a<4p5@it1%nCh2o~pvs=)^^fmkdO8QH}25{`!iM>P9*oOWyqve*#?#G84TU$kD z@5jv}4JI%8(xb`Lzin{2v_1;qLWx%VAc=@{)zp@YAy9OMokrtFD{`^1roL)K;ctRf zySbsQ6z@@xu#lH>uq`j=_B{-7YIeMCmk01=%>vv2g(i07#UIsu@Y_CpG%=`xA8C*Y zV~kp3#4{lGDdvd9XAcW?6cSz%*3!tM*)ljLsfjQxU_z3O3E-o(o_-M-;@*72b(VR= zBXEB5^oYdcx+VW6(8!JCtY${&`CYz|B|2hCSd6(4SkGvAjDEq~xSY5pA1}XA{@v-Q z#bX3di#ms6ufMW`i`KDLw2DZ*`q4#@@d=PXaQSVlDFZ2mbO_q3$P4p4(hwBirqL_C9ip!I)KT;RIb||e##4%D< z*_*uxFbu@RGZSiH>?Z`(B zok_+U;^^tiVG!GP!(vo7QVusvK@BGQe7=!DHF=a4B}Vl54g;EWxX{QYv0sM zS4B8PZS7vt5jH=~3PO9wa=Jf*hZ4+_e125;%pUCC-)0|(K8$OBcT-CbLz@wn4Q^HA zSFEr}wRc!mH-YqV8PZ%LtEhP1`>!t3r0Fp9NjhrNiVW&hRk)x0J@_tk~?_nzFBp8AvM<6tb-e;|shcDv^n0{{H?}-1O>*3u+T}PHq)NYIZ0$g&2t6 z&o{n(CWfxBVihroj`jA^3I{=!5=RI3`Pg%S3sMHo%NVVJz3OEi?JqqL8x!Z%+r0*x z_MD2=%UK`kimzjKRB;ny4Sk)B9K_YmS>>4^SSc-^{4Q^!@;jBQL{d4MHlGtuAzsYb z)jj?V&FbFJjZ>ym=kT&^{au_$jVw}?f*Gb;3t zPwsrhtk+=Dy|&NfnS5onYWH3Tu0WrRe@8dVvO!O<%+*P`_H#v=_OW`EQKwO1f?}=j zWqsywmlbZSm!+?Nw$2z&d|Nd-n;K@Ve^)$;jM$gx=kcB{qc(LQsh(_s>&lONj8oJ5 z`lR`7xnye0YjG4^dF-{v)z<4+bW+p%tK-!C+Rhp4cLC2hw0|5XT_`Vo5Ki+4`fNyt zD9lIhh-&o{n*JhY62`j+3+fF*YH5c`%eM21-UV|e7Q^mmRF>>ipKxONyiAVp|GL98 z6J>Qsd7ol~C+NLlY!x~XaemNCDjNr953Xb1EP8iq$2o%co$uMzg`{`rai$YCd*%=D z2jry0sm;1D#tR*qEIMNaF9$8D*yx@^6O~cR8*c!R^rrbP_Vg2eLo>^Cy`g&018n`8 z`58y~vjrQB<7ocDaSg+?*}~<5gN2%aXAS;jvQlH!ToXk)VOuxeKAc1aKHx1OWv zzq?EhClFz;ob44S*$cAuU*F0%$W~n>h%s=x?;V_irsuf!!)300@zY{J+j^+o)b~B{ zD$OcBWi@r)Do}=e35{hLZM3D&k(dfr3hLEXm9vt*RAxJ{Sq!e6>c#gDZt$7MSZVgz zN_X&!EJEkb8Nj@-FbG&niO?DhK5oRm)dLe1>J1BK%qs)gYI4GRXwo|^YRz{72T4Ni zP1Wc%cyk64*gVk@?dj&$2CQ50EZD+Ku;kIlo+~c&!7Z}$R`JQpf(B~}Txab`8Dz<= z<>m!mx@B-FsASEH)-H3on zcXxM(bPS+KC;}oSA>B1}3n<;)NT+o3t?_=|@27Go%Iv+?b)EUq%AN?1P!Zc+R(rAb zk_&ld#H2GEqs>q>kfsl0+CumZ)0un3hZbg(C8xnM8|wdNY7e;L%>Z=EV9-!qHPP$( zA4>iTey!tB~Lmo){;ia^k$!v(mdEv-8 z-Y+_c9jV^qROvW#M)@xx_6zT0Bqtg8pINUZTWn2D1YYAI^{Z``NB~#^d!ohVO!ahk zvb$J*J^74d+$~urdHxe-ofQ0mE&#)oR}{TY#J{V|Spg2(<$h-=F+vn-`_uPz$Tk=t zdbY~;bz#l0ZKyScMB}7is{yIqZL3birwAz^%apjBwf>F?_)7sUxSi&K3g%t&0gztqh64NeUh%;qdG4izsk$1l;J&G!I z_D$R6JCIz+ZN?LuXRmu_!fe_VWw}KXjM0I1c1Ao;#gYd7%1+|&I*^y!Apc$2b*_Bl z)#3$101lbV0^L{DU@8^(L;bu;03cQX5hc2s?XvdiJ`cTc35` z!!1~zd((&whKk?C7PMC$Tq!{sx%<4ik*kmc@8vBI*zP-O5(jHQa4`W}W#LJ7^3c}k zIAP)-tj6|ficJ=f8G$Z2F^CVo*f?2gq~Ge#4;hxJgY(wIykEvj+(><`j99RlaQOs! zzF~IY3ze$>KL6+gVEjD3Xc6)dxUsuRPCyYdiEmCfT*$9kX2{z8Ch`d*ip0M4Ug5zQ zJhUB<0H+i(e`pYVfM$)yY4I66wLx8Tp#Si>JH|TiZ^&ERM4S;g08{P-N<+u#1@)6d z%2C(>pw6!Uot&sTF8Bg%krH@aSDqM?g1HnPi3vPdh{6Ouo^&96B^=}#hp_V~_W@(1 zphg;IM`75NDN0(I)N#MEt=_uozvEj#75qqJo?PRc%i&JG!%bW`*Md(rSaW)jO<_p6(5+GlOl&{8`JT#MGRjmEtqWPEw2N%4j{$S96a3DpYc+ekzMC zW>Br%pGWo1lq&%)x6Ym?4TM``^*NeG9Y5bxbZ&;g$j6yw+#l`0>Rm-TilEvPzqIlh zELjc997%Ou_naRU;PLIT3wWM!2B;)*{XjZ^pJM^GZ*eEnvfob9{X3OY)xhr!Zqv&> z)oP=F6is$h&TZrNp{qm9LYcwpP$pu)&AnL|8FE$G>D%M?KrS>jkJz!;aDq$s1ajV< zoo79ofoZ95;B-#@FuFbj`%&qh zphpjMxYJj9oI;<$^M_F142+8CH?Hh59`R|VZ zNP>hD$=X3k?}?45%X%-NGs~Fd707;O6J!K!tc31zmQX%i74Y7lKEp{?kk%wfk>wLRUZUL4M;7A3Ba`O^{SybzqbWrkw1pO?no zv&Y(YPVEUqRd16ABBt#LqPXS8sC_i)&f;ZQRzY$gooLIvYpgzjIsYJWZp0#GGI zg^l+krDqRFJ;==3-Y>ho;>^mP+)^%AZMA)qD(tUsXVd6-H(crOZJ=9fEsEC8Y=h1;~^Q`a`P@C zZ!k^F!iuBcnEB}XXDEW~qPWA6H7iq}o8Rfp#TzGOGtEP0uDRiiC8#1Z3 z(?S0t;!8>}`(Yj=t+Sibv%M!7$zDCDTXWPE`LSc($2nNxNQ*=O#omE~Q`$S}3?}-i z;tev0DKxtW;pC0}%B8x(uRa#miIMBk>wMO z|8+{HnyYXacS5(M@8^606Zk&IU-ogmvDOJ!)RJYr&k5;IzZPW1bz{UMDrJgo?J)2j z&%r6H&c*Nc$WS=;L)BL*$*dohC$*?pz1c#<*_- zdzoYPg3lrrq!z`0kkjJQqAv~?SC4|5>%0%*vrgQch2M@_zkkY%6VUgW?CJ^Y7G?>* zcg#?_$YiD7qF%bVFKB|{XP7!;6;Ugdo80$$=#Q-KZJr#dkUtz+3%}_6+lgLn#_Kdi zNY|@52{Y@C1f{BLc5_Q(y^%Bl;>yRVw2&LovtkiqxLTwdh*PGAXWcT(7W{_W87EUQ zgm&|lS@P7e_P|Bf+K~eD$$p!rXgyobWU2IYxmBuX7B2iybD8zKxX)IDD_^JJ$Uu|d zndaBA-DdQbNH=?|`T7e8SbJ0r-O2Ok27Zrh10+I;T9j(9OOF3@wlQnh(_JIb>iFo) z_Ny0d*}xqbrrA4gZ@&!O&Hr6L(Yht&ul?oLZADmV`NoXDwI*$1$Fn_r;HKRF&OM$b zaY&aa#U@mC@F(uN({292)^u*^fZyOA>-#1@i+ZU`xL zLNe?r!vVpdz_rjAdu5b*TQWGvYjPvyzhKstf){y!mv+1ch}OZ?DuWQe;T!FQx8rkn z%?l6Ng<=>4TDBA6PSe!YI{bC6 zKN^c@yflSC8AJjmSl!()u5-p5Knb!7$W*Dal>JDgFg|p$LTTU&V$)-RMkHz7%|a?{ zYYXAG8%{A~(1l1eN$-Xq8(|>APErK`*V?fu!WBr?-Ti$Hp4DQBBTIs)BFZ~J9*-<+h zptk_;$7o2|i3ae?_UtUkr+`~7ut|A_=Xe^gw!}n$Wq=mg;@kXxJ4${@V}k5PeJOpz z`#-<{`Wk+bKX2@6j*bIiz*P3x$rdixmnJ1<@NH;4<=k#==j}+U&EM5}agKUG`XU+^ z+)U);^exxVmBmM(iLZ6csFBR(5tJI#=UZZzKgtWL!A`DG36MC5gvZCX1^8loWl!z; z6)gIM;}F_x|Na>r6DiC(_F7>s88Iy}?3kJ3c_yDI=a^ku(8G9BdgnFM*o=Y(zY0v% zBPv})X$}T6-KI^+3qs#uSzxGKZ1zVc{J1e5ne#eo z#%U{3`TddTV<>*)ur8FN>r}4)e1E@CK5rM3SX%n(BL#9ZTC_}kZ-CPeAyVSft$YeK zgLF}kM{BZ=INXdhF21p4%Y2NnbZE>_%ua;lA#}xLX@nLcS(=L>{mOF!4paL+AB}hy zUoHt3r1B2~K0Pd83N=q@Nk(7N7gAe0Tv5{gvY?(o2I&VGXHw|^|%8!Kcqv1i#w!fD35$&Hdw_+XOQVmBw_ zjkAv(QZ*TJw~*z}&*Ql8oBp+3N$TNRM3RUl1jcO4HfO~QG1n?F0Qr9xM9#$}ezw=g z(p8+ML9HnqhlowYM?Ms@wOBX|G3_Y6C8&WU%sQo94W|^(+dS;v3Dx9BGz;_V%@ISB}<&mRJM@a}eQ$ z|1r{su|$d0SAU&XrgbfJrF&WlaSc;N+!%eZlk(+Nb}!y7&s4)yW@B9t>_e4m{Cie5;u)H#4PD!ed@QPOL(Qa1WW zKQy(FtryjHU47&~vik`k;jBZRh2=XCzIu0eO)hj!5^Yg*s~?mWe75fyGtWrKkq?e4 zY)twpIaGHhKqhJ|AY&NglRWCT22Cm;!=3IKam1si_(>6%%C*9h+Cpl_g0GFcK%@uC zC9(Ed(OHYn8Rl&a2!^x(x*2r@-yJa;IW?_mkUo+L5FgKvZoNlfWey%&AYkyITWk%C z1qtG&-~bZ{xD^2HuQYAKb{_O^;~^z_wM3t25Jk0~Gl2Di!V1X5a*7xhq_6hU4$^!= zGkALh6rd2fvAbCnAiIIQmPcl_sAvq%Shq-QB!$4oIexI~g<^@$Z=1&2+G|5EL7f zV2;`bMSKAvuckf5m#8)8Oe!niEnuE*&#TI9&w4SC)@)@seC#t3tF$+{)pn-Lmtx|#w=8bPH!#Xs{FY4 zD7lD!TNj@sy~Vjksz*r<_k29s%DPaXJ5~LeO9^V`YI9GStz*uY?nmA8HfsZwI27Ed z8g?@Q%9e>U*ktbbj#EDpR}XGpc{$v!&bH_UB4Y0{{^U>X!9UiN&}&x7lXO#8KVbA= zA{vP#K{QEW;IJI>RH(3uXrXl7ikI&(hb6G}$`SXE$JjZAP4#5)3cHII&(~Q#4L?0F zC+My5d?r*Ald^(}^y|EX6w*hG;ceu9vr*HWH7h1GivKfLt=Iy7;p1U+=*B#%htvD7 zQA4!AM}t;&FFrc5fZ9=*9H!Djd_;|sXUD9P1x?;d1O)wnMe#yDx+e3I6?6Pqn<*++ z)Uz4BN50F$jP3`#GC}rWE0sFyrGu|dQZeP|eu~*yNJIg*Cj>G$2k$A;?ppOyYWuGz zPl(G@fG}0R?nl$O?jSNVQIv{h^<|fjWm*B8t9Z)}>6^a`m+$&j5ztDbUZDNROx9hx zk``l+>HhE9i(&P6&IJ#@5$_Ha6!zqMal z4=w&Swp=~*R&NF`e#*!hRFQnA=*b&08ZI1DE}pBCwZ><)1;l*3%q^WfE;{fM;!T|= z*m=b-q29mT(;ngU(5F-{TU~rWxA%NMPfb{!362B@2+fxqB({G?0>DU`eH*=B^>noo z<0e}5KE=aGPhdAP{I=g8I4Rx3BrqsR+_N)r_kB!|ZN)DJ*jPfB zbmC*HwZKna-+@1!&COiHy@OwnNAA~hyfs2I-{2vK&;lG-NDo^SauILQ^5Tqu*llZa z+a3iK(g#D`6u`ouVnNgnDKAB9_YH}S2`SCFEpY;qPrhzVVfG%*1wr*6m%aq%2%z~9 z=pTAyXlR&B!;?7a)4gEAuzQmDn`9;Zz0l){)>;xoiL`WJmtK1t_nCH5?PmET~PjF9It( zvWZcv&xekgfxPW<6po2f7jP6@u}Ds@tXX9_5xZ?Z39zRAd3AjzXMtHiWUEp|M=mR? zfVuVBHu(cv;aNOovLJ7x#qc(#L%95<2I6q3zKGStLV{Yol2JKn>n5YI&NI|X`mx?4 zx$hQ90Y_Oh#f;0{{QQE=cT0gc{?z6(3$kU?X)!HU5th?1mm1F`58+rUkA|q(^JPQw zav7JO>SoKo-sIoQ(s*#UxdPFw0EAdlLDGbmJuly=+DAAp_jL|EsO8+N`ma3tCr1pyPd`rPr>xfHV3~8ep{+~hC7&f{n4Fxyj;H;W$j_me0nHkbtA|9K* zH=+Ulbd%qTe2|@x)?$b;*#uzg_*C?DXEtciFAAi?xE-4=G5%hiyl7Gx{LM8T9__1W z+;^RjBxqAT=!`>m;p{>{@L!5sSIuNZV-szJxLA#{MIUtlzgSXo0qsiny7iK2_t9t#ro4qRAqqYnDw zyqS2$E2}Mg!7J3^xrcY8KLHEh0OyaC+u?7R9N6ubvcYa}N8!tBo=%V#<9_k@WapbJ zDUh_^W|PGi;f+d6Yn3f#nhu=TxgCB!*12UC^SdR$qk(EsXv)57s;gM$^%dZ8EbTSYz2h5BivbcNl?n+n z;_*8CPcowMxMkE4OW!@|WrR%xDyFvl@DtyOzIpr|!tr@&*IR@)j;Tkl(H;rXse?xT zTJ!~5+a|kFS1Eh%@m+*?m(cEVg(ll^H4Bk(N`u2p07u)!2Y#oek9^XxMABR4x}W^Z zoi{kl2jaSUW;U+tPo_rlHN-`H?|UuNfog+kiLw=AAU~~U*=x|R_sMY)aV+>D5YDf4 zRhN8B_*$&z@hn#Y$*4-s=Wp!B`@CvZ|a zuTA=Uv(80nmEBoBC*{i^_wlmJ)wJ>cU9qdu@L`MF(2g}_QUYklexCg{128f|?V(7= zz}#c!%Pk!j>=P=C)t>G>Mr4&-^R}g~aT%2JGh+O1xn6k-m&0IoP&@j5i@yq2avG4* z?Xn|%vlpBBD$g;t-gRtIAcbq3#mN;&I5UD}dRf(w;=)%8Xhq24h5VGL?72|WLC&pM zBD+xxXd^(+iE4>DJkTnqNhTBT9oLd8Ou)YWsyDPUUydJn`Q zS*N3h*9pCc?@obkOBu9{H%pgnaEB#fvA!2<(3a48EWF9CTN7i)tN$)09HW zgGFSj3Ew;R=ds+BF zS}vjI$Bf*>074%V-SP5dbdl~g{(7+S>HQk6(J_5KDH>L$5JR1xf9OW|=5lJfOY9)ltnE2uj6d7+BM9^}zsK4IYQtHs*#C zXUx_Tc3(46$ef?^_1xh*t_weMl|p7CYK3eUH?$;oX-PR*wM%B>#LEHm=J<$m zR7_wFoi>eH0vKyg?6W60gNIAayd{`Go*-GFO4vUt&(Cr=(d0ClKR3=l9Y;|w?`4JG@`gMV7+1E{*6C(Pqfy`6z7zYq*3}_o zv2&#_$n|YsRZ^SxGe^ddBfABuj2pieA=E;wBh>w;K8IVruDo80_Z1p)gp$_sY|Z5+ z<-^cl-S7uZ$glH**(bb(iBGu2h8Gg3ifwS^fz($zx%>0lrb@P<79w+UZqcqg!9tY> zPTUKHe0rNcm9-6%%OpXh{t5gCJq&`47b^@6iow?8(jM!VVB36*&@s#0BSw|63z=5m zrOMZ7b|g;OEZA_X{&VpM5N2j&Ss1Mmg?RYgA7_c!C`J4tP0TErm$&H=2~rNK^j}mP zR6DZ)E`f~h!SW_P=h|dln+c{<(gw z55+;_jGv`lMm+wo6(@cL$D}g+n9NYsrP!2(Gk_bSh5(aa8Roa_CviVqT=2^y&;yB? zxHz#;V~kET55PR2qvWDJB3VAp1X2B=ph7MS7(AT$joFZgq?OhBvCHhZhxqLTyU9*v zrIYPV@ut8CB+O4YhvEX9_)U$G71Ks8bDg6&MJMPZFml5U-TqjDKVSmr?-o{TPB+seGl z)j-jXrJ$j&-0Y9ot3Q4&Tk7}vJLkuV01V|}nT}${^rS+8@WOT%P6L;sf%LKSSW2j0 z!DA0q8mWz?a=)>FA|uGQjBcs3XP3r#Sh9gc)tZ$LwdP)(Zt^x}AK2ta_egdE^rnz9 z2bt)QVJQd~)u=@%pGqc88mdu*Cj3XTdfJ|?OIkHN`ZhsZmBZ^bl|y#^T#m1m-NEu( zLi9%O-EmTEZDJ~)KfV07R@J&P+Ydy{P)A|DGAy=<^366ojE+K!XVEubdi>&V-dSKj zv#g)L!Zaye4KQbB@c?cS)*SN&bt-q}i~yo#oh_ZI6eT5t(m$W~IS$JX4IbYo4T$p! z&}B8d!D@#RT&${#?Pa&f@<8cL7iebhy2LCDgvWwQR(z&GzfE^2BB#cN)=Ug8zi#!} zXFI19p?=1;jVt#nJ|^G2okcAxqjP&JG<)DUiCQYy>rl*eDR`}8 zWvf*m^mIF}(Uqjg0;7XM{GEz`z*6ARtE=ce;uY;=9v7rPTF?C4)_)aa1=MI>UHIJ1 zAe)qIQ;q#VqEf(PZwtY0TfAvIA)u4 zUZ>0@Q3^W41k8Sc%Fr%#5MX2FwN1rzJJsUvE1xbhv)v$XU{X@Aa=tZ!42W|iYuf28 z`y+u?rYbfeGWR*!R9TKG9oZ$|WeasnMdgQpmt6D6i?5^H-H6XWc;xkx*WR>BlwCt0 z{$MpR)M#5QMdY0;*pqter7ik19d!=RD|N>xK^L z^8^j7l5MLvO5<?UvuqzLJeWusa(&9XYg|GR8Bb8QCyEE=L5`1lC z__wy^SEV`>0^>kvxw}jnw+9Q%UfW{B{+Aeu=sUJv%|eqJ;ZtDja3*B#GpZx8^eL&* zEweF=*a$R!C@s;&H!#q$Kc9q65Qg~WoLEh!%6+?UT%D~q&I#1vEvs2S#h?ib&1yLO zYSdxUyoxE8r($RFWZ9g2Q;|~EzbjYR?(K2PQFd*vqays&J&Xy$$EHNQp^t#7gIyvo*NmvD&R+Ad;hTo;&7AODRtu4#XK&S>NUBMeu z8x$yZo&c}YxtIenUk);B8Ft?mw>!%YkG11Bdzb>YqhVul&P)UP!CXBmx!(B1@!#|c z*ImRPs?P$*!w$3284f>7g&uCJHabq>(JDQ4o>jG@vVYWd=fP~G{Qg8 zfca+MSkL?)-3%U#qkpBU!ym%Ow1hA*I)qeTAAJ3%^*w-JqX%QElTZ9Gr;$xSToYUF zAicF_vyaE4r(Cp`q)f3agb)k39=NG<9=iiSiT2wxQ!+!@M)}-n8jMGGi=cee=?2V zi2@Wov7Aon4RRR)Y01wkkkds*BXdcHtg}&7Lk<*8&M{TXXeM?Og<$|_4!+OQ>*DQA2C;^SK|08$&m2f#}<*XCd#cqQWxKvFNfk1(KqxZiaW2|Py` z`hLCo?fBxTU35TWyxKS&i6W`WJ=-R3ronJ|1i$q8X-&X|3a#Ad*cbU;A1uRuNvtj|42)WGu9veTwGdUsOq#2pW z^6`A(X1Br6yW}WYg$C$rlA6s!8@8R_Y^5I6lcKO)Fi(;76xzVG5AGcr*>9LF=Fe~h z4#w|tH6D5kSTCreI8sTZchK|Vkt0;`!CJ+T*_E41 z>bL9M-(w~h1z=5D5AUh=?xnB0D`@uDfz$t=UKkAFeWeucHk;y&=S68nI+vLKr>we+0{5x}$UK#%?Rm4PO@gBh?G0F&p?cdrIjq(VU5q8O-R6z3@JU%tz44#jh)JL>hB zD$$DoIOqF4VCN@nD~O0dKR>J$7tUJyt^iddE(89zmw1!< z-}Wco-JCmcrYoEr-OGqZ{RD7A*WOE(G0;7XjmD=MF^dkqPbbKKi+aKv|4&JuD!2%| zp^w?wAFN8lnOw~*fs#l?UMB%Q(yqe&OYe8_FZj1)(M=X|81fpvx0*6bvY3Gi4VP+-Y~>ME(;asYy`i`ah7}&( zt-(5*r8gd9_BJTXJgn0#HV)p0Go^HRs#(X9 zDz&aI4k>C?Wiu|?fQ7M5WVJ>MIuP;0!@$=GRO=^&G&*kL{ZRa>QZjg;dRS*nC*xiX z&BE|M6!e^M)MN1)Oyi>fgv(?cGq5!S(AaivClUaSvijdoF2)xW*)d{|MSviFty?2- zN=2419Qy_fu+@1%iK2Tgf?97~q;u!;Nny_nChijD8ag(5Kj9j+`b~Q7AJqpzKYX#B zbN|G4xFSx6o2HDBr$!^%Zl=y~Eezo+GC_fo^LJv=^=YbpoYhb4PZ{Zdm$=M;s4C0p za!PeXs+=ve&`pJFOFD@#=72sl(`=NSxY>|n=WvVip?uf6jnf1hD(Zf~mL(l)qLVYR zzcdEQ!;?-o3ro;H1DR3LJJ!PqpoC!hA|Lb_f)uhp&g*}La z)xhEDU`RQ@(TH(wnXC%wRRiK2^8z?sPOXOu-PS94o7oxTN%YP0VR8WYh#K~3oJ!gk6Kp#F2T4^BH) zG#3syjcIrI9kLyFbQc?32sPgR895$}J{dpDgPfcU=_WUrqAaWTIoG?l%SVy^c_1<` ziV+=weY_f97p^Uqt%XHbn@%3*3u6%CC)0`aT`oP`znC{upstUlT*P{z+n~)AN3ZmO zbrB}M4Csa8lgo)0o`~`)e*+)3$cg(MU-bYW-t;`aVm9+_Vo{XK7_rTqk0uV~G}g;f zTl0Z*EV;=;P5zo3I!FrL+@W@ZNf~U3oEhlgI!B~#x|V>p%yKxlSKa=epzosZI>Y>$EXw!|>`8Y*mMEyY7^VN5t{r&y@U>>7tPFHvH zi}AgN#Kh$WJY3I<5oW6VLk8Q?yNkF7vbq2E=9ne(4U#o&wQSw}a1tJ$$_B>79>1oM zrc&`Jxy^ZGE5~o^F4gVqW$(D?$Ai2KtzN|6C|o+P3`BzeG2U~TpBU>t<75&tlPl$Iu_gs+Rp#wNYoJ`1uFY;O4#(1E-(p6V?Z1og5js2cr<~`cd)W14>qQovpi^pY7xY^A-2kgN(XZPr+c>pu84nhnF~@(rNy8{c*srE_%~n6Us$Fthk9U z-1ldBIW<_R--A>R6lbO?k5PA%H8~L!YsmNE3Z-!M<^Mr46eLtEu@cteAUu`o(QjXz zOeRGIM;;)6j|QgY;H5hOY>}HG!=#Y%eyb@0^Bsrp<2j%>QT^ z@iMEsEf|^gF)&W83us9R{;vgp#(&sL=QTL*5u%(t5 z9p@P?``VG9CBV5QIHnf+1#AsK;vV85Z&P|4$l(FbD98(QQ7V?3r0)g-6khwY7Ux`_ zbX#lI$MTd$nti>%ko(iVmYJ#;vjztQ2aNxAqvSM&)_Hmx=C-bZcr6t7pSe|)*JO+f zaqn({r6i^1&0#mFQIe)~q1SC-PiY~^$;0W$2Z012u8H*%Z;2rtI?Ibx~xb$e7Yb(X_75kAjhSv7CS$&5^f+g9Es%+BKth?K^DWJ>@xY;Fh zWf;py=?2iX3P{u$qm9`8F6RI8;PmgnF&+t>fbCNm1jVt+S|>xP_0{^CAM;mOW(SoD zvwT|DUQSqwlx5wk9*swOI1gO54Gnv*D@c_$RcZHAxu zhhHP-%*H$bP!_l;@r@7v7A4NuJMNMb7q`8SfresW6iH?@NPNcM$t_)P+pUI(lF2~7 z9tA4;Mw`bEC8DQ5qlE+TwR!7i4;7T;Ve6N*XdU3!k2|kDMhf31KPTMbbIbwh2jBP{ z1%ON}+rp^zhD0@o^Lu1BK9p`+zD>s|o-x14Yx-19;(nl+aq*MVBu<0UIyrdV*NOY4OSkRCZe_#CCafk$Q8+I z=x6??jnIehZ>8HWLETZ*Od17`h(n~9l_k*-S2bHl>Xy`a@g8Zt-2V#j` zYU5uO>2z{LjT57EU)a9%_nqXru?KTEbAQ(^N?m@|gWtpaP?j0WC@G{TDUlRJGbj1_ zlLHqLI{Jiw@6|CK=mQQ19j}-d3&28|31$!^6l@Tx5j3{;PCD|*n;+lB11T!ZdSLI2 zbSg~4f}{i;w*&gG)!WpYV=yoX*fwi^5}Rm$Cv{($xbY41p|I6KDg;4U#KR1kCk!fC z*87?#No>F~9}5DDS03&UA82-Bz&ZChN&^|S7;`7=woH`C$Z$Bgybz)8H|OqXpN(q~7fS**P(rUSxm z`&caJZXY6Ws&B<*P!G-;EZp{h&rg2q_gjpc7&87r>K}s@?YnVPkh$K6KtacMlm`oQ zftE&9;mP(#4+_A)i6-9T}Ph0ogtfa*FNXJ98Or`{~v@XO?u zpmV&Y23X=NTn4xt8a3~7xA^UwU?o;`ZAlK(NdDp8Xbe8P*M#bf+m9VabFTLGzak#% z$d#j*XmW$98kZFUDdRjvoq}gdW?<*s{YEJ3R}+N~ zo%r#I`QMmLH4R^o%HKS6k_(heXp6ln)~?7xFpd`6tC_$6LwQb={QnRhm1Uq3ygc3h zm|txCJKX#9@@Z-6Y(`yu*c1*feZSJ{B&rw~s}?{tbP&Bl%K_H8T-{`gi%TO*8vr;o)Da^TKhTeO~B{I zn+y`kfuu0f*1Lt*c3c94OSJv%TO(lmaiBl?_XKPtv(y;pc)9D-!(_ptgAgB8X1n&? z;3K9??J>YFOt^=nHibhtlfyAPl5_v^W4z3BSEuK2V9%0?X6pV-6ZCENV5X|aPy;D{ z{Y++hLeqYsK|`Raa6hY4rSLe<_xj9`>W*f7&2?&=^CAIsO<$jUz*_X=` zA#kl&pNVyxJ(rGrZ})+NIXE!JhLVzUxJI*i1Mv9_#`1<%_GE~rV^n@}xV4rFbMzvU z655mPc1fGuW zft#3NoPhtXf2RtKKiQ(ZLM6`>PhaTpK@v8VF`}790sLBt5j$h`bbQREDYnJ)t_V-L z9_abXjGCdqVp7y7VKw!an;3lVnk>AP?oq{h^+_8?Z$2FjhxxugEn3opPVQ8pAY$m_ zH;t14%4nB6%k1D0#?kMt$EA~lCS8_J=KXO4_Fw5B!ruK))JrDXSd^6QkNQx_?X46j z%slJVF`HVz24BVBOFZx`1aMv7*4=v0(!BCmwR1Ek8caMcy$>g0X~9tcri%;@sX%m$ z=9qGni^oQFW#n^GLHTK_#4Z%Sjr)-=slatlfz^HexMLIw+xynd+%y3!$;c?LdIZ&e3 zW>t#)J9n4MB+3v8Rs>x#14Ghk{OYS#hYzGZU4I#<27k&7YLwJg^crC7zSy68pu4E zHU6)=(~sSZW-9&O-NfNy;Sk3FK#BH%=Vl+|lnN73n>TRckn%u4A7*(XodhYlF&U3F zRqhcCiiT{?_ht+9iPuem@@Bn!zqGtK zPJ)_T#sC}iUdFNu;y$89kV#TOb=;pAx^*ww8_o=MMy~m)btu*#IQWfQ7Qu@t-mZ+L zHFj(}&Oe>kPSaP%;Q*`OMTXxDDV7OhJ^LPnj)hV&ue=&7N9(ZcOH`QG+G?-`XE_U^ z0>u{zsxq^Xob#7uX0Bkf5Br#3bE3Sv=}jW+$&xf>hzp|>bp0_L=A2uSp~sr(Dil2W zGy%XKbf<%6aO1G81`0Ce={x4|5UT+ns^0CQ==5T3C`jAmzxLq0B3q> z&m&XjXeFm9A46+UBn5vo4b7|O`oDMZc6YITr^s0r&WZry6bdGD*tT=uXesc@o9HPD z4i#)pfCZIn>-9DR%iO;IH9d&VqY-P{=u0JeU{DL@aQ!nF8&(9HDH+6vmS~sYgvWlj zsg2qmY(p@)LglfV_3r)p>Y-MsGb}cv$$#Sl_2=oO%#;<2E}3zmw;Zl#{w9xlsm_q~ z#A-BmfVLNJ_{Cu)p`qJA;moQ(+M6#ots_Fdz1s3mh1L6{h)rbGT&=(D7n=`#d(2H# zGU9rAFpi`*SvC$!bAc{2m=OASoCbbW1VSiYUH8k>*1Z3{X9K&M9h{U-`$s07_aWu2 z&V@c1SpGnf0B5$^9zq~3 z{n;qsN;=f&Oh8S*_gNw!PMLRpSN^+I+ggx-ZOLr4x~un4AYd$Ews!!11h_9HV+YSE z_vV<`$8wM)3*c|t7>r)M=&Hmmq}O^!lApPGKPQ?hEaktlGDECjcNq z1MwSAj28>k@={BCRNyLv_uIcUE;NLsj)i%wL|5K{DPH{+? z`&AdWa^o&Oh;PMSsQcK*P3tHJzXi9QxM=HV|IOtRW@nq(QJs5GuAUPJnts&Evr@{} z>+|2$VGUndGD~_=t;^?-We;AXB*Zz z=oX%+Huwqz*mg7*|Jg#9;pvg>nb)!J#Pnh@p z2gItky$3>7-YiXq0p@=P$+~ihDi^z>L2Fj@Y{eK+miiUbEG#_%7)-OAJp&^Y5_K9( z9#@+8Jr<)$0wA`IxUvbfm1T8)(7&Tp6BtVRDU~P9T zyrM4=&U7nlN7AOP%kMr`xcSUz=jk!F!}TMi98jWNEopkye6{Uh5q)@?F>Ola&R_%whwvW(b@HcT)v@DE9^U4&WqA3Ca1Rz z2kM0yg~j$u)kLDR{3CIrxkFH1VH`UE7>1v2LxGCUT1N_GgtTO?LV2-!|cn9OPXE>hH|78JC$Mrw|&w!u|4iU7>Y4OWUkctW5+wo;BAu|*Y za1`ege(m{>4T4WaTbfiqg37^k*$vPvXc7o|##f8lRY@!jbSI1dzdV_HO!*`ZL=fI6 zIa#dZa=xc?dl>j|vcxzAKAB)#c-V}T{?}hSFy^tDcr!Oy<$BejGhVuElIXFy^6~X2hGlj7(vWfxyMvC%kJg$`QGbcR zUuq!Jxsy|~Hsv&nCW;m{4%;vBQ%RLQu~Li#3Xn*9cg!3!>#^B&m(?xm&qS;~kDdv* zk%`}(erYazSxmDB{fx{RxbdoF|9{fP8V9dZ0j!Qn?3!NU?u3t<)f>3K=#%zspY$do zIT^N&7Jh>n6Jipw<~W=vEQ?@5}yWwLE z$kG1EvkBG&N$@ebCw%#jp=i4(Yy-#Rr{~g`N4|T>ymrdqtA}Um>+jy6wvfR+W()2) zB6$D(V@%FAB^_2O0LSZ%-WlPY}N{5|IjOJjfK#1U2H%6VHN!E||-#Iwn zCYC@s#2@T6#|d#gt(OM1Ys_0TDcjA2Zbq-BWW zo$N&R_PY*xKF`zlpYQkg{Bb|$b)3$5zpwYT=aml2Hch}RwnR5RXz6&{>;=l;Gze?_ z^fr^0r>Bcq7u}fLH<#pDo7&=_l*xGf9ge%2Qmmz{&tS{Z8qHbafDE3Y3RviHnlPNklr79^!}3~W2ChIHsLEOh&0L{3*rD{8#%-2mbx44o)h<8 zTv1Z!UU1p%P{TfvH#*yWO>_61HE*&~vq)dz^g(s%h7zm!&M!l$$jcRlb7te%E%Zh#NM}Ue<1_;U~-#vU3Xs zJ~=RDbYx~*_iG*lhi+g9bOMl!40qvd>e=mzb4`++94eHf7cNZ~yZX@N>uVjX_9${4 zY=ho!Uq3s<_{8Jp9ph^k=zIlDR_M91wUS6v?)-MbYuja%InNr*EWpR{u`n22z)^~2 zJLGjid*(^|7vf}vrd$(^)N9(ozT%D8-sy&q4Z6CJCPDkbb5NazSC zJ`=tl_3cINQy3!c{LS!0*$K)ZgB=DJsJ1`c*<7iMWRBi8Y~$ zd#6bQ9lIfC#Xlb-s#bXU0fOt~1I^4YWE;w@)Bv`6rzJf&u0ZF{~*oOII<2j?}4wS#0t zsQD6~QG{&n?LwK2#)?&wc+>b-F7a!56i@ZyBgKShu4kBdl*qiU&FA_$L^Ly);ZzUJ~3YfywXPS}l~&z&MuvhzXEzncmb3m;R(RC%{AvzZ`tx?X>Lkt{OzkC502Lwo$j0_UK8O)$Xg5%w0>-4u4`OSs$wb zra$_63}JzB(aLnI>gwE(d90^Z&#+O?a8`D$MQq(DYAoaR=cfTuwVpy-+s@?NKk6y7 zR^6G#2+BALQCvn>=RgU2QAu*9zX#+RVcG)E77TW?m=P^IR&&@<(Y$_&07k%aR0X2m6_ z>AQD<+&>hLpl0*@i+);fEF8cW2W)11gC#yr8utaC^6kZjT4?)uty|1c0uP@6{@HCj zWpx|KytM!QeZ>~j*wYEY_6EpuOHs?#b?woMBRY2MIfxypWC@MEGSON~9tkX;>SNj! zt}2yVzxaBqysCCvU{p&3b9VWgoua8XY%tA(h3d)Er{R3_CIMWG} zbU{Xa5iP;&04mdUR?gEpsBwG5y6EBkUi)$5;zC8^XI({8qhc%9=+AfwF!2@E*5l7t zUM+iQ^}sUQH!`nEhmQVZ-X3pbBeQ2l&M1beOlr{eM1hJ<^OQ*OyBnB+Uoj;)(~mjX z)`#L>p;s288@Qz6D3S%@8`2jyOm{C5P3|(>IYa7C$!Q`Qg zRmEFJ1-N8N2Y=(IQw-iEcGoR)k`190yprI_p)z5P-g#%1WZL~sv_{Fq{py^}>`aDz z|2YlN?;OW8j0d~+sI=vP1Mv8~m*4+*D(NE~V{ zr6aG>C4AG09di>4D4UUdQmo?mySI+I1GM!_4q z+PNsFqWTW6rspKAZ|BVUymrgCr+>+X=XizoPlumMd3$>)s;yXqh&7YqGGF26`c>AL zapyK$iL#4*H0#RTT?No-9gWj$K{^p+s9-qLQISe57-G|Nt0OjjuA}};pQ%GsBlHjB zNeVDvxACGDRE&2wajaz;;|>$)X*Q|6cIJzKDZds~n#y~vd>gtM;`!j$9MM9{Wy5pj zI)uh`4VM@R^0J67NJ`?PMr|U{x** zVAzu9pu2^ItwoS;)vNgZ`}aluC~&YNZH1QHsm=jrdg0zPG+a@cUb0`47ur##c3=2V zt}Uvy|4!WFzy*&6cn}lR@bVF$Yh2_72jq!qU0tIeTaI?MT%=idNh^Hwjs@)c10Yh#IO(y&iygNI#ScRwOJX4t#W<1W+xyzg@tN?^okFH+L;-^$Zbv&(z#tfPH|sXW#*t_7vjqkcl;9gTAXVq%Z#VF1Q>LvI54 z$`Kk5(RhzhmE@ni>-XocdPJJ4l{mi2%{7^A&bk+}<>vO_g--uiFxPz3wkWUDw84u! zzPjz*dG--na=zb#KVLWeY&QA2es1rwg^G@R%qD+=&_g+5UN>GL{gGt0&GDN8ac#4( z>A@1~XUB_Y3$3=^)a=r@^9uDZsS1jabbZZ}96lvn9Rku|jvF16o(LC2Om!;C5ieXF?Xn!xD@* zRnh&&{%4~(`%g-nnVE%jm&%!(#Bt>#>t2b|>h3-gm9_X&9KEbUuQMLB<;yJx;=qHx zy&+nOZHVp5nEg1jnNP#-w%O3NiJq%V<>lM8$|b3LM|gB225*y+#Vvm|Q=lE+wt2Xl zlpW;$o6}=l-FE)|)Z~%H?k_YiD#{)@yFD5DRhaeAEWyuQ@tqY^A*Q6VC)piCSN}PXw+Mzcl&8!sb#G!CT$}j;5|8- zpW03H)Jl_ig4A04nzPlsY>H;90_NvbUq`Xv=6rY7y{s)^5W}W~-<^L>U0WP2%Wep_ zB{M1e{?1imgW{s`!WiA#;Ul5Wqy($RB&q^>5Z1Tq8fIqV*chuEG>_!j-aA3cBppp% z@pe*RhyGnBD^Gso?h8$UwX*EVdWQDlh(W%>iSx=T*gaMJE3q6_ZTwpl6Lu5Tp>D?Y zZ>||PJvCnJ&z8);J2kIZA;D_iRvyU_Y))^@=NQG*`DU4q|K`0wmQDM@`$Y_UtL57+ zJ$Vwn5A>ljZYFz^XjOxKINAjUH%of){J0<*(7F)fn{F;sLSQmtGm;~gP@cr-GKe-TLM; zKR+k83R87Vj&0~}JWo32yW7l<`8Tz9LGeq&vg(}STo#65837XolMMNt(qmkh=udi~ z7R@_49Z%n0r6E;cal-2<7wlHrG6+2$M%qxxDHj~Kp0!NA5T;PL>9gsQBjA}A%N>6S z^5a{q8ty#kcDs=gMeqC8qG004)>{4imoqD*mdxhp52~4pD$-^mx_mYV4DlE?ZYB!= z6ycLQ2xRqg2hJTp)(v`%uh{6`wC=mKHsIi)@HAbi!1idv-PK=L^F{vt0Jj5U^aThPOq)KY_7P+O~Z`w*|jBqM?!1S!iAD1y8WL*$&G2EtKokfI1?D+XX71pU>ka$dnuIdGAV&Y^;vhexBY$o3jCWd>SW zu7szh<=cMzx^nNrzSj1ypOGW*6+vN+Emne>*B+0iXmAMBbw)l64Hl4EWlIHXK1h+6 zD_yJ}NjgnU{i`#lgD`o}#{5$Bwbb)-hBxOr)jo1EaSNlWkzF1CHU_)&(FUb;0_T&t zB!K0?k9>Bs18%U$DonyP?5L|#45R#z;q9$X;6}9Xw=N<#KKVbUad5RzgatzbiDE`} zVIJW1V~&8=R;&wPco_ky1>dM#`B0V8PxX*uJoeSc#Z?9Y^_z|P>TjJFb zTwhxF*!da$GjKw6y5118+)H! zdaM4;t9R?;YtqF}A9v{K$HVW>o$=av%rAKFg~9}uk3Ip3|A|Mu&B*I>&jGisIAI#g z3>a17&|G(FpL^DYVu|!@r@44FLDR=eTwJPLwnHoi1_q1ZaP3Dws{$=uqsS&M-)2w~ zvI4F`HT~KQ@4nnU5Z4Fi8>RspH#tQSb*!e(PLcdH4#7)leais}@}5bj5|{zhnjK?1 zcjaS(hX`V{&uHVD$|Zzf+Z7OdCm(m|+_!{>Dt9ovD@Lm~(Qj`(xtkpF{{+@O%SXgC zS79tlJIz^Y>Z^#(P!mA3wfBJ^&hp<-t#$&aTHj33z`sSpg+9g!JMhlNGx1y~l<~Bd ziR#a=vn)t^eOlvvZY+<7qNjk&yDeOJ0g(L7!8Bmd-jOOkNq2dKwpI*4J{kYAJCkIR@B7O$l9uKcXvAs5tZ@B z#;O9RWy4#LNPZHsOYW@aP$R@ZO}lr;+9>=pla{7v~-&-2Ndp*vl#`d-Q3xjOh|#gR=Nc#rw*w6&!W89fwXw_;~_ApA5MM| z&Qn5vFhhOAY$hfbETodvbTCis)#kAmCr{C(7}W=A_qSvD2p0Rk zdm2fft7vTAS{j^hJ*@HcEbE0F!QUv{a)gs%&d$M`eX0=(31XLj9|e%IY&usVzE6p$ z9>5eofg_b}yImNq@o#+d95&oA&c2lXqvvwObqTTp>(_se1)F9;6Ze@ejO7cSe{`Dz z0NA~Q1>!eC*D-tTGys=PN}^ zOaf#bAT73`<>2%6b>}C+!7DQx8G}9V4e2jmmhf7cZkN1x@nU&tl*IpbwtqDNoWpXL zhslokSPj|BFp7hHLyZKq#C~ma^cfZo83{b$bNkIRYuBo9=BJ zP_k18{WF%`NMSG;sOog1qoaZ)H*X@C@~;yns08d7Y4#a*1EiV%#DDl#Ex@aSPDi%$ zxFk6~ussTyi((5UaFGyUk9d~gF)QHtU0v<7og=?WCaHV#%_D{ip`IWYBtUjn^559a z3D~V^7*E)7$2SM>``EpMcTh6GJMJre^9JkW2Y^jhP{i@t6$BG%zKI>6Z%!tBwLvbf znOD%F<4SLif#1*JDm@q{a}j!ktjr9(fB*jU>C+c)+_;g6nvFdFU+nc6L9i{vO*lcC zge!4?|MT)s$nLJ&pE+C^;`_Lv6gMjjn(;Vrx6|b0 zuPn7Xk)}?$P^wok=dm`(5OG;{@&9DbxT?UL^(*`qIS?IdGU?!S)!}ZYacW2iowv6v z98;g;J#tka01H`Q=(B^p+Qq^ezQ~ryAB}zS;!;!0&6~eC?|mX5hhz{CF8?_<;c9*W zQxp3+&)H)l>Q$}whKHwu*z!^BUlYOmR|8M5^1C<=+QP+-L__c)L{s1^T5{-N`~sNO zF%1bazhhwrx;E$(wl5rbxt)H#RL;`oNxd9a1&4+UN5Zin*VFh`w?~cV)LzK%HrWZhl+i7wk&vO4qk0xA4A)6-! z2G+MhFVN|>lxGm;3%hHTWx?@Zk|ew=Oj4-t8CComl}Z2ZiQIuwtA7E=;@vAkUSA-A zDfo^H9V!gy+)DL~#UY}j+~ASke!JQ=oCNp9C);B*d)IBs%~~&-50#hvpO9UHmw4D( zl=49=dhkUH?Ez$E5Pa`w7n=mg`oI%Z+|$Q-)ZoP>r(6Gkhz-!K$@@dnFy5n+R7}uY zrXxYA)X^Mq&z@ycuA6Tj$Nmqgdq*HDpZ@AUOdp|iL4>)9$=C$Ie%6H%rbbc{Fb9az zXCqBca=-;W5ss>+YZHH49|5NQxU_;-6pTa5kcs2vR3d8r4B_6Hq+V5w<`Y+yO!Za= zF^hx4{|BPE$Wle`RT28;ti~uw_#f~OAz06*QnVW#tiuJ-Ummh&SzQ!>Gc)c^w;&Pu{0`N3p}^d(ynM!c~(?WfA;wAclr@VGWf!T3OWsCUhX*L;KHl$ zj=ZALIgzK%6zwtT%(aSrq^^ys{AtO>LFl#7m%M}zA6_+D+w$`G- z=V@v!b>aXPe<13OxD%)~eyYIC65XYGvxCDw)HWTwVCg7?5SAvyElpwb4{G_@r4Nqw zk2Up059i5~CyO0Q^7Y`_T9Hg6$t3*fn#Zc1BMfbG=^~j_7T@t$|BHJ9G_?Tx3&!(H ziJzh}%kb$BUPw}hsAt~-}##f;p>>|ca~2bQ1TKig0oq>B_bB@s&+97 zt%?bM`SSdeprFQ*_fD(J)9r0J24WR)eW8$v3hw{cvk(495AKX%v9T=35QgsEwv;>e z&;YY2T606tK}XJ5eh%g)Gl5fPX@>Ol>F+9h-8<4zER3g4ol?wDX7tc2cfPN8`u;Us z5(01F{a7gctyBA=m!IZfU6#m^b-A~K=kA}QZ!CjH%VX~29e`X8JPNZvdW+V2pm!b< z6iKrkALokmLo@ly3yj-J=TyWAXA3a!c z9uo7|I^J5fO^A5R7eCr8x>UD2i|pW?_gZHcz(vJoz<8xYWMEF zQOH)+HHM^LP2)lsf@_m49m!M@*}2iua(Bes8x)r&R z>C}WM=twaetb*kPJyaEyi4t=DfYk-5i%Q$OfboojHmgM*aAackgFJL^i~Hzgdj`$m`o!-bs0OZl`sQa$WNKBgg+wEv})7^-4>*ivyy@yA5UO@NIm5c8!Ib5OJ#k z-|-f}1?3|s(XzJMQd;_HGyeGt7nn#Plpq>F*mtL2QZ)=}KBWDrjZ_S}m%Y(s&JO}K zKYsk^F+|Kaa2L-?CrQ2FCQd#%PQmRbxBKQxPJb>C^4u=gdqkbyWIo}Y$%J<0!i`0I zKG=D(@DtHUa=={=C9yq-T;XbbRJJU#pg)};XL_A zOja1WQY}f1bs-}}r6rQp8OI{B@9izt^XMJFx1Ww zX*CLRStxLv5JT&WMia+n-<(&%R}Otq!iYQC;m7DDL>iam4Pf~>!7jr{^>{+GS z7rfq@nwqyxalDFUPE_d%d5_+|MiUlEScr5hYK}Z_62wVQPj^-CB7Lob{tD&qGAGT@ z_%u@7D9Mm*#l#O1w{A&6?b}-z^iW%=oUVvh%5aWO|0pcgVK^F7QNDg6Po^e7Nuw9*;WMOpiKjCYT{Q2~S2J5-~TWbF%PZoVM=K6Bm9k zjQ z;VW}Zdrj zQ?HQ^{_0piI{l`iXaBY@AP|@HDt~V|&MUv10MUz0{Ti;1_7NQm=N(CLRJhr4veXK# zB6oh_KAX2cw+=XD1kLXt8Yy1!8-%7f-sSS%jtpl71%)AwltT&q?p#lvX=uMic1CPB z1{wm|_;mMajueCx_E`oS47~vLGYS(!~uXR$iO*#g8isLQ#YUM0mH?@F+sfC6TL~cVE!TAO}SU*wXe?k zG=kjHzH=CbA^~-ZWgfh8*K%MkZExuVH2*^N5#2!}Xvon9EddzyD$5bai>~u1Ub;r! zJGJgMf_)L=qfeCPg_Ud3$#fsHeBs_rM3pVP&l zRBLi`382b2J<(SE7p}osHz=KhIU)2+B>lO@cS<%UV(58!dF8v?&trRm=vpbtIreLB zxIof)a+Lk@)G?huN*emua#_k#4z$Ksi@|qQ2*1G~YM=!#c~B^BmJSa?x{y3s>#EfO zet}ma9+&W3^c&7Srhuo5Yg8`w$v1E|&c3{E;0r+iTC;);>lDVna@HtC3QOZRvMQ8U|>IWT4Y20n5mSRmt zb5%M~QL(4%D?9^~2y9WNa9BKBc3mYnYI0domXnLSoetkf4SyW&DiN13+J-)W?p`%` z53r6c|Lj%zJ~pH%8m*zjX!n0?z4% zC3x*>5Gy;*cF95ujPIQfI1U{K6h2~)PfjPPk-Dp74}0wibT^2KisrXiIv5tH70fWB zV<*2#9C=3cbFhp-;Cz{io`9-{Q+Y4!>Jo%GI>p{w6!a*jtV^7<9WYZu^cB3NF^hjK zX@6bYOduSx%MU>Q$6Nv7;g3Vb>lo(mk6U31CiG&}-cGZB#oIgkuvb0J?Xj8fzBc#} zG3s|BwT3$hqcNIAQCj^45n49pbgjT?oQ_SN%BiRZjokQ-cIQ}tZl6sEJ)S45clxN z$Ope%moW)}v*;RV3=-c|`QGUrxG5yA6v9YXp2G1!?>72RMC1yW_Fo(2a7)}=E}k}OCAg{jFLBF_Jg5ycG<^c(x8^k3r(*Y}zo3~@`> ztteHa;;|d?KJ}6nx|5eLedB9Gt6j^^I&eaIuk0HjV1P$L22VW{)8$dM1mEn6xH?!m z!(3HZIenQ?IqmMjA~())+y4lp4%8_#!F;N{8r{IiNNub7{+_r1Nq^ZxTk~|h_Fj^@ zY%mwLzuu?%4?qES*q|;`U>Q-vZvPEceHS0Eomo9L*JU$9%3;qf4EQKJt~l%;h`u~F zX(43d@il#Ao%Ogf2;yM4v(N3`DZb*rC?1TlX zPD(J{ngJMlL*WSqk~%ps$eh?Z{yt=V96035FL6u%I(&UsA1KHD3Rc*P=cC7u8=!u- zd}aw6+Fs(|Q1)_pfPLU-lA%QeiNJP-Xb$8JuJ)vo_05DcVfCvJRjXzy*BeqWof1an z;4nct$9s!Nb<(hv`~vfK3F1!>)lq}Nd*9QKJs>bT1qc}6Pxv>@HX=SKeg_fxeG2~H zPb2F-k12P45B)*3<2_b2vU#__V_F9jk*`wmId%9c`r#9={&7sODYEO zS9+=E|2RZZKC-tCjN9)tMt+x zH4Q2*qGP1f@M#z3zfJ?Ce-8mlJZ0%04ek&qBT_yvq-U179?6!X*$bl;yM<@iyN_M& z3}J&zIp8^H{|Jm#D_Dir1OHnb$@L^*psPha>wq2@N^LSth5l$ld*2;OM88zwVf|57 z4#9Cf53q*4`J~5R64UNY^}}HghEV>+z#Gd@tsoUjE%3*&k9X!J_x(5JTffRQZ0x%S zA2u90LK68!{oQhx#n#6(Um@9O;HXBb9Mvlh^Y8)3c3GR1v$2TB^VQO~ zw7+;C3zFV+@r?M*zwT4gxT>d+?Z%&VlL~kZns%?!1&JR zMMJK4Y^rvgid+f5fxw$B=P~{C7YUJ_230kw{tf!C`yrTx|<8H$qYdmKH6lM`Tq5HqZ6K`Q^Tx9~)0E134@UMf7*tzCXj3Yj1HN zXHBCwQZ*u`im;{J>Y&!;_3m`B`$jH^DanpupVGo^2HEgmMQe7X zy$FviA>`+Df0?y|v|&@904@zDwPuaw+kphU1wyizzc#Q%IyT^a6+Glbau&0a^2 z_y9AX?#)ZRpQ@(oB2MkOZjjuFP=~&;&&?rcu_4bbYeNE${K#Cr^Xy#*a@(0M$9h)P zWj)CUnsbm=xV&OBURE)f?ZV79jpx5df-NyJmp=aMNDzjalM46UzA+~sXvY8JvDFkl z0P*_N1WzW&hK+`L-kGhbsVVJi6?UV(L68|y`f&a#qRE0SqMe zsTvscB2;Jtsw}uYdAAB*aG}^iO?2Wt11&5q>J?o2|vck@(YULlh5Q)g9omGe}IvL zRA-L;hgL0Z!_B!Tr?)Lx}@d=>Pr*VSR}Z zN=C37+ibOfL2DFPM5={K$;i9_SLzeU9n26wG$tSL(607bE`p*k9UaZ0q1ctVM_X0& zofkkMeD389k5U6H2IPB7& z76YP!e|^Lc4$KV+9IkD%A`kkD#kFZ>knnd$28)KHUa+b`Y6gWLoG+4y~&_K@rqIP;=pI2sg;qKy#C@zc#K4)|_SMtK{Lo?t|n& z(b9_WI!@^yUoDx8kATs1hsjCuI1dLOL{pbPj4Rvo!5{BcNY)ldkO15%tG{~bh=n-r zeoGLt)s`gbO0@fgEut<<9@17?R@vQG`R2_X~bH`DV{6c{3IE zvnWPH5~0R1*K^0Uh&^K1MUB^-Stbv0r@N3kyMD6U5ajBbV}lw{%ED`aTsvpkh5te> zW(4ULf3`mS>pt$!Yl*(u?55vdz*IA7==y0KT6i($W5Vw zQ(>}v7(4pkA14X6!7NtHGm*_c%I~r1NY3Y!IaCOK@!Wf#2^p2%1s(X-GOvv=5}+kI zFpf?y5z+rB))>|=I2a?9au4i-5`M2$VEm*##*UkA1|DxgwsBW zGgFno&-wHAZ8^Es9th`2lJ%=X)0(*sw(I}f*ZvhBgzgGHvalWf>pplKSXzqj8}hg; zzSYIx+@CLBlAE|p-GFEgAbVW-CFwz7kVF}ha4}^(qU3dK!7TY3402zA&tLYba9gdj zW5wO-&#p4zb*PXcSuG7X69NHZgD7&yNBg%{;B5@?B~7Q9DenK?N0=H4vH70>{|9CL zc7LIDiexA?PkyE6ObFjEk_Xn%>ESJmp=tvfn`PhuLEAuJbiAg}Vo=kj0y-5htr`{W zKK#`t9#1K#ZPA@I2FX|0xUsAY9#)w$E|b=X8v$FP&`QI_tmWh4>W`uzUL>=h?j)@9 z-fMEin~%uX!yj)IWStKtrf&Xo>(3w8HTNT9haAdQz^ny@Bm`vBeB1%Ch=sVIX_~gx z1^1f0jA|^iO9gfw5t16cBzv^;QNYU)oeE$1c1B80E{gb_Y=lO_PQUd66U$!i>Xr}0 z-RqM5Xt=(e^5?ulc^}MxQ|*nYzs!J4#37c!E?M>0eNe5taX^EtKs>{>)~1awM^~hC zYsCRWycrQ-vsAZ~2lo14O6-gc?m(hMN8OIml803E+McXPPTU+qf+ z^rRl-Rmk@AD0v72r`n4Wv;^=#24z=6=7sUMb9!|HJb8J?b1jQ(*j7`bA-ybolgBql zBal)B$_EtMoAm#JQYNHgbXl-Vc>Q%Bpl5}{#J{yZ7D_SrIvd07vea~QUkQTbSRMeL zEyT-=YL}-Mo8IghPiuolni5(l%Hq?*!l8`Pr|?NQNg39gSJfhx1ECq4u?+2!Tz<%$ zTb}D4#>AsCDISkOHZ;}E@#E+JVnou2LeHVKO#0vZz)TviEC45J01tNNLyYt9?@0O% zqh@{@d6{$qiCvJejfna=f5Ms!;Y`9>lOcjnCS*`hzk&z(gE`cmaXX)_O6*46qp;H# z(1af{u3Jq6BZ-%$k&v@tSxL_N-^23$j2xm>&D7QZ-iJecDL3-|3v~o|*rs<5jhZiL z!QIt<+s$s)$B!Sob;ctCu)@0t%2B>R!@XI99(kdS=r8yrfh%>Q^f*zc?^aEGd-@aa z2^Q9=h1<78ls81RuvNA2R6QoxZDn6`v>wSRdN32a@j?MAhVODOf;unuPT==4Jx_xD zXWC@rmxok2LgZsHE|y|8Dq2}gy0)*#TtQL+a7ON@&*5e1a_ev1VEZF&7BIr2Eq#?0 zv=8^y&72@%d;46_W~Z4y47tu?bXQ;i$tWV^&07+E9A`RiON3I-?fbTU2JjZ4T7J-a z>=BJomua5S_|thE6&p9wz`3p+r|Gr=^sUacQ76Q1e5s`WY}L{M@iNTaeD(tp^E7y& z51#mjVS?adU9#9)FrGBNnP!5YOyQ%;7r=n)lh2AS*As8a1)gp5W$Dg(|HWqwJK>1u z$pr`Rgd^LO3p10#DY>6h)MUidHM9JWcz*e`A3I$_l9lEMTJlB+-|n*MWW_NMPz-2{ zOdo(oT|V)yx;3Mwb>IrGT<6?s4pf&A-MH>dkB5G^$Z7xeYeYUu3pyr*Z}6-yP*K)|{)LoHfp^;dF* z>m3ko{G9cQk|bP2Pm-CCOrTP|*JaJv$wr4cjgWKdbMiz(gLU7t69`_ToB_-IOq(N` zm=jffE9{+1UHGJ$4+U~{2ab3K@hR0uoGWKrdEn{lvpCQrFuLEStZAwNU}_A*v-Yiy zD=&pbpg|*pZC}0uS!2w#VV2DVD1_3*97n4Ls(k3F4W=;Vkq6fcuEdDFw6yCwr|K{| zHx&b6B6DnYYtkMVrsZl8ZcQc9V`WZx-xrm6*Q!fFp7E1q)4ypKm&h42>&L9WoM63gOZ`7)_N_7jdYRYjY? zx?y4>{JS@99SZjrT-EuFK|8ZyZ&HI3$B}e)9~6>ydzwF#vp{$PWi@IaCp)Oe`$sSN zaIV<(V%HR|h`OIIV&Y+83lXGMH6R{|U_SXcUKJ!?a^HS_du*MJV3uvLz%rFZuTVzy zQ?0b9s4tqgLGX<-W6AWpnDKWp?a+j|^K089@DA4{2tO@sn*|lvXiQ=*Wp2%EFYQt) zMhSH3k~?bC54v>})9xn4zMgOmL0M(pDzpI}-PIn$#Z5uorT9V;&poGZAD@J8RxUnCY8+ zJhEwxk)HR-no|!9HL+BQsxH&hPdJPCf!KIP<~cWz@~mTZG-DdVM@B_EFbttE;d?%* zfsV_@RK}^cyIQ1ODykp5FY>=&FDG6Y*{Pl!+r;;6K9YbhNc?S^zp>x0aeYU`jTN_F zf#_rIeku#DJj4kWe!Aqkfa3D{q0JQe0V6j-au{kglH>t-y*==t85UiF$LcG@Ph`=R z$p~iqibW0*PlfcfJCMrt7P5(Sb;RhPuw-N&mDwN>ps(YjP8!l=jfQB=V4dd{8B^}i%Ve%kGJ_*E#C~e8Dk*_730JZ_AZuoeYM{>h?{2-7)>^en)S(}1YHkG>XqjQJJnAEi zf+mXLcQGL+uo(zJ*_Cpd8%XkU;<7+jW_Xo9?+3r)8~g?ptioqx-mE(BWNf0(PVs{n z`mM2h56PgY8Ngroy1g^45oOsXj3Lk}F#2__73WaCQ!-!o8q2N~iZbs`q-3t)U5%@* z2FsQOHrT{=X8EHGa=}5deCUt=cUe3Z_H(4xxzJ`XLn*C#5ZnX?BGp|u9{?k9RcI@N zg|<-?TxQucTy@|;V2@vQ&h&jh!#vn|$3qTG;me)*q(;3YIWfW(78yS~M6=7Bes!$! z8Fz&`3Dz!vNb%nl&xc7Xk+G{-xZ&J6}Rw#VeDM}(yrY(ged)*A8(mnG_2Q72;HD4 zAT?;;BbVJeI-cUUdT(DU6U4*$jr1-V{Sj~M2fVBC3LJA#o@m0ph+rcE1@V?#EO(Nc z(B@P!o6SIRqbz^aQtnLV7mB-2gu`%sp=%=d&!x$4^{+e8#?V?@^SYDYwyVre5!-y> zj3!B;$VZ8`wEKog-YiGEt`&j0EQ;}l0&zJsO3MaQBhtE0sIeGaa@`-zJlbX3D#h>x z)-zP)PIgI z8W-KL(-nov95!RU5ABemUcU2vu-J*$_cUjksHrns6w;PoYa{M!U>-YoWND~O&1$u+ z9Wg!+@gDnCn*G9RTp6SU?NfwUw=Zms%WRqdP{qB2WSIWQZ#CcF_%+=!$%eSQu~qo&ea-?4E5Dio1*d>z+mqkMMaw9w%@dtZb57)6KM1TLn+%Jq_Lc2c^S3_N^fafJ zM**?SaUpbVmH5^rBofh(%?qc%*M%5~ACLluZJxw~5|7N)y8{2MlK;8seB=eYc3pW!Js(lpwJ?^}0)91%B+h z%gby$1D&xhHUG|wRGkNTlO8Wrq&Y7&NNZsamftY#h?rzT$chTjg zzS*1>U5#rMz8=u#NWWRwEJgq@+r2UoOOPDEW^_e(^0n`&^U zYS3V%bquvG|NZa&sqb{VRnj$8pl*qll+B(P;q{#n zh!lmtcyR?ur_`;0Q8C{Mt96U*l7!>3GAbU=45>F0RB%~$$v~S6-9)$7(;#L{UI$mN z1)!`(nvU##I;OXKr8`R}0^C%$E0{*xaxCg+4YHQQo;hjqWj|;lo%xzS$uC$g1j^Eq zh#Rr=ogRpS=0<~MWpGbiXqHEic z&ei9F!JHAx2S@UYA9<$ZR`$8?h1^v6g7}bm#yB&#-gJiG}L*A zr&ADSi8-Fs-S=RI+dkyQDS+(!*~6{ND>K(vj4AI$iD3w4$^0M0+40?x#6@>yV9b`4y6pM8Xf}`yp?#egbhn50xaD z9vC?ZLIA2o$47v{>V?=QABe783iJJo`7!p+#M9I^JgeJTZ z^C@0=V75d0jm77lAlhETD(Z-_zOQefwAtR4vO1jbQ(1uN3d;I6s}wrGv;fQcNy}-) z%>7jHmuFSB?uoD1D_}CaEsZM#E3EQw^r0==`=`1s*IK_qxB2G~LYvL*=;)aJl5tg~ zGgaDR(?vzD^J{m1YMh#Jg>3C=-}_XQWo-+(^y>MT#WuAS3<;wGr-jtqa-@#R0w)s_ z(|H}X74~B_uU%cYhrm(f$#8P_$mOez#TTt^Kk53z2>}GcPB=t2P1R2odh(d3VhE9x zlmA;_smlrne48;tAJ3@hgJG#C+sci1osUGkWFa5AP(LEc_tkAp6c;0-LdVx+CdwF9 zx$O3@A5J!sS^2M4mO|#dId*rQ6y202tsOwGwa^@DAV@*b zz>?VfK2>MZf<>;vTvJ*I&TT1(U+F*$;RjL%5?{agK*YwUmp!#vYO&YIL%bu;v{@Sd ztBn6BelaO2X&U|g9VP^H7Y(|O8N#9&beY9y_u_bi*aSY{fv0dm3+o;Y7S`V0xb(G0 zfBHcFjKk9=l|A=3^d9X|vb4a9Og*elUrZl9Jm>WA<~`=Zp6tQFL8@%b`JTZIPlr`D zhlguir`L49t$%zXptSc&r`Rn)fMQOF3>8!Z^$Fs4To!*|Zzqv#5X$s8N*=FaHs-H; z(RTamSdHWvNyjsevCBz2Np89er+I~@wiB295>Axfh@&a=^0X4%(q^$r3VMspH+-&7|NTHUKOU57UK~v2_1fh>4h^5RwPn1>y!Lhl z&VugtT{4UoCM^*bmzBkI(`xh<;1pPPKRUx8C*=VC4Vp&1;<~I!OXi5J-5oyn&E->q zp4+$yf}Tdx@+`qOwhWAEQeIM-jX7K>4ThmeH8a>w!(U63;#fq$@bQ@Z3Iz*R?)qM+hWEwok+ zfjm4sOK8v8tX>wqWDe zhC*J`@|H6>u8nf>yGxTdKLwpS1GAWp-7u`Z9#%W{+VI9Js{Oy;xaZ%Y_AXhReS)w- zI^dnb^c7)LX2>Ol3~%9{D>(xMHH!_LuKrDR`syj!yeDoYNBSx%wTiRt$ly)YL)`p}$h%EXrl4*1k5z z;Vo1~0zPbKDUsxe0Z|?e;o&Ntr@|_prwXiCRUijaVJSvOcYwSoyi5gb90SaK6*HN` zhZV$qpy2u=q%j!)Hwe2ddqJ4^{RDK1bE<~!D#!?sFm_Wphm5e`U`BvrFzfjf(Pwk8 zk*)iP6MmsRO+XRyUGq*mLCqWd`MvY~6OrLQIgIBC?gS|*o5q^$esXPl;GN~?NR>G2 z{b9B`3UW}ZC>G`)`|3V{S40WTm{Gz?NQ?nJxV~N$kR1pF47hOW@yUOH`qmptFluI` zMDY%n{8sqwm4zSmXY3pt*wMCndU`#@_mhocm{gd!AI!YaSgEUnQVysw>3S+5P1y4k z{&C+|>40YLWBw;-bTAp6PqbR-GA|pAP#ZjdI_}8w`GWLIM-ybVqzJ=B)GOUfZaDCo_#qGaJ(Cc-jF+j6oQAtkjf*p&eO`4}OF5`V+R5gq5FW84ke&Hkx6ATKzr_ju9Y>?}O<*#W z_3W|(W3qjXKN=c_uRPv z1$f8K8=mJdBXNXM4M%a4-yp&y{9jZ!z2tCfUS~%( zX>QK`CfK4`ay}RA;<^>O^Q?APA$}QXY4V5;@sz7n74g&>?Vg+Pv6{<|M0KB7ts~puAegEL()1TNPbJEtI^rt0Aq^Zua!dIdRH(5+E)C3OnqfQRD1WebTf1_ zlr%_7cZ-CebVx|2gbW=6C@3HxAkrxz3?ZF^ba!`mcfE&uAMgAB0w0(e&N;JxJJw!n z?-JP26+XJ1Ax)jocGFpLVwg_2=VD(yT=Rc!{hwdW(TE29Jz{WPCm3LZn&I95e(Pk9 z-#S@JKIx1m@U742b5U^q{nqiO9{*W< z;pne7k2n8@3xpH-H-5oH9xtVqF$3Yh1Gbde3`p>wEa`wFse}=MB_?7jrQaS8`!`N> zoG>C7(&IaVWc@z8Hbm#1F5K{gepiVF;ct>TYsYe%*#|1~Ti*cc2ny`{EW+(tD1 z)X8AK%7JwD;_!xWdQ4Im65k?oqzNu^V)IivVSiS3Jgzxhn6&siu^SMAe;edF@fbkDrc?> zpfXa5=lI^KMehT=^|_B*j<#~(&m!Q$UO~>j|Ik1cO0O&7#sJtQ#Crf=674WdW|bX~ z{D3y`$0J%`ME{-2$NR}{uEg5J*Csr+qK$$XVuk-}lN^sW$+4DiVh0=>T{~lH3d;X3 z#rN@f`r64b{O5^^lmh|c3XyQCGpwqe(mbc$vf5RpF69 zI1hh^z99WlIto^Vx6Ln+M<`(FJgeaEfEOm_KBU=Tl5fKr33>^9uEVFfa5LqXH(Ps*1J}k!=7`_13GOYHPJp9kejaE)G z0IN*K*#vUo@oSFQnO)!GXZ97!n*Zn7 zi`0R$^~*`sTJCR15BS9n1I8X%4`+W$YzIb%r)y-$&^vJ)Ew^EnH}q53b&N*=?t6z_ zX@wc(@4M;W%K>gKd;05W!YjPze;A6$DAtHeYSzJ&%4v8g*8YR{Z^6;>Koq(OJP#J4 z-rc(hp@Q}eNKGQ{h9Um@WTaLBKWD#cqb_dnc%=0YrUyI!JAHPqACHzHF$Lj&kMRZ* znCOiHY%0VC(6AsN^$f_$%IZ=!YVxB7csp2nWfp-zbdm+ARJ1^;!n8A$g#d)N>H>ad zP1^U5DbD+|sNH!C=$BpMj{*02ds0y_3Vl}JV3cYq9HR+Mp&x|?WVNpVK%F<&kr+>cJ&3`lR*HV}P0>Q9@1^*xc%D6ON zRZcyAj~hCYIxc)sZ4T$~9duT$TB&J%$HiT$h1YM|j*!CCTIX(}re0 zAwu^$C8|6SL%ac{g)T0V9xks@dY)Z#{)sw3e%3`GTbBKaP9@h(qtpirbJcLs}Iy@{Acw_71DtZGf*n)cQoD~t=uR;E8s9se|vU^+2xj(5sU1}+ou zyxb(O_kO@1&Q?R%5}Ttl(*K||u>9hI9ksYv8Lp3L%SAv8Yj>s3T-B}j;k&)PQgPGC zQxH~4109=%x2HiVXwNG^_i?h}b`V`|-5w078M=D{g@5 z$TJ8qgk^y7`*IR&jKJA_QK;(5h(fRMm}n_j+3kfq=9u5PsU|9@FZApv5*gV7V6h#z zfvq>3H*n>&S>>1-sjf2O^w%}r(eOT_v`=b!Jzn3I_I~JO&iap=9riAA9=~FAIU9yJ ztM9c)wf_lceYiS7OBw^FZeCx$iqpy0qtW-+WIA4FLfm*6rfQe4lL!KfW4ymfyy(V& ztZCct&3BF$J7$p#rb9$)lz)z7%#nt=VV_AA{ailDYk_(0y9xO&bKl5I7LBCl=nAcuM^2}|v{L*5r_Z%|VuM&?IETs;|A`@gf+*Anh6i*Vjpv2*Cw zgNqlM!>&}(6gwsd8{Zgh&ixifD9v3f!-yD6CG;$DnQL#*$g>nK-ylyCu@nSj%5KDk zEUn}XWUil;Q?|_X$gZ~1mp&}AvKYMaPUCY`P_KUP<*Uo@pydGgM6DkAU`D_q{)sQU z`hmc5*k*u@og>WjG7%3TzM&2sVob4+0hSRIp{B)z0@r&?N%Y>bJEa2t-6S1GHt&W3 z9ZQF8eHYZ_S|=t6M*TAO1(H_j!@SjEdaY!$00TFKcqCU0z2D&N=iH z?=K-hybrTQb)jC7$~(u&K7rFIoND}*{#GO#IG9XmHDD5^Ca%z$#XPXowbx*&`jp?j z*Y=X@GdlDR>xiT11#MAC&{yHVK}8A^`U4SIq#N*9hlgO1Huv@6SoSB|Xq(uL7X?xZ zI!Xa+fU;?HbacwM^~&*q460s)|G8%K^T$*3e1fw`L6z2Cm(`|i@8tr-L_=rGaD^9d zyo>fvR%p0AkL0^q4Fvc-fZK-W<=r#Uuxs0IihJ^}n$9cWY|$e`IhjPval5S3AHF5o0CL1nPGG`zNPu?PYN@)lRBjb5HbuEs^ ztVUJt?sayIQsL11mS21)!j~RexIdxmchJwtZFj6Yl$w>rz+qgaf>wsTS6g8l5O8WY z)L80EH-Wzq<~o#@ReA;WNXCEVB)>?dTlKzF*Z*qdi;pH(aU584 zH@FGtZwkH?#$TFdxS51WrV|gzI4Q*~xk(e8qQo=gk=- zgT*~cz&!V_*~-mv(w2cR^a1UnJuEa(+AHh)eqdu7rQhu(#-{49E_dF-pp<|6HFktK zKCo-%?wH!gI}+80vA%RMhJ)wn#{*;#jbv@r|!`JEA4~w8%GZV{Oi`0ktdOr;5i-YDJ z9c)BR6vd8eZufairI^tW>LG`G_tt`lFI8kXfIMIrNaAm&vVtO zP|z-Hjj2DK`C^iU__enGFj^SBUT&1aBSET^bg@it@g1ti9XVP->mqsh(eCCplEpAF z7r%U0SJi_3FtmKT7%NxO_3e9@&INyJw2+Oa(q^)58Ki^X`)Zk7l;!}Q>(0KowW4U| ze^mR%_IUshuF8w{udHp2Ha&G0W6pooEp#F{A5Tg*&ZQe|NN7YtpkFX=96A{H4y4jf zk>TO0QW*cZYkK9!D6QG5EV8DH_f2|IMAfV1T6OwHdi_cMPrVb5VAHK`&O{;vn(iX~eB=sy zO=>aCx$q)Krt2)DOKSYSzCT;MZ9T08n%M)i7@_C)urJndS=zXwZ=CX z^0n3*oGN>{Kp2-vLrB@1Y4|t0D);r}loli}4;##@6U(p)u;b$DD$5kD&+TOwmJsBc z;PaWulZ2A_;ry9=Bc#Ob37x!TOnX@bmHtnK6Z=w;LHbSu>jAKge8N}V_~?|cUX<4f z0|uHrIr?8(40x~_M9Im!AZ&78*bQ=1b|tV16m)-oZ#u6oC#XbgFUPKS_Z9KH1gHqk zo7&qgm1v82+;>+eu}=Yh`)~~}UVIDe9Q)`cx#Y`Z^avAuTDL@26%VpDV^=4P)ANl1 zCn3lwZEuGEHY+^p7oH;QJ(`T*@6}P8*QH?q8l766PkH_&< z75z=wegiUhrrHQ8H{LksZ{w6{!Cx39#uB8p<95Ro+x+&Xm1^jDDI$JK1vGOl2=_Xr zBD=G-#uXC<=AfS%Iq_3_`5hxkV_Z(szFIprU5zj~MXfvik)wqsf9bIG-%EXkI%X-$ ze_WucNQ%Go~5Oc#aXMI-$xz&>38I+{S3A?E+eeEvsw#lMwr&VUKX$0z`{ z77C!lM{#m;Vu)XI?HreyocpmZwH0$Y)p3h=y2*+S=0S=tna=&Wx`;=uMs0{t^WNfh z9IzCz#_sUxWev+f5+hRfg&QfvTMdc|ZSY$GTMU?}Tqk$`?n)9CFVf#dBX1ZvrVl;} z1Erbv{(&qCGHRH{wu__8)>adn{c+QXJL%0+H+g#y7%PUwVWp@A(x+_XhaBv1tqB>u~)=x$OxGEhO zx?p)$?l>pH0+ig<)#>G2JD5NPH{msCt%TtH2se4_whayhN=**hS@8)l;ys5_zvvK$ z^D4#7tyvY}g3&R=N+KP0FGun4;z6PekF%|pA VWtT&(H{97DnAf*X#j9cheom>3 z^OeB5b~PnSxvFt})`7RlMlv+BPS{M}64QZAWw9;JELFj1Hy@nVIAzsPWpF9FQgvew zR2&C>$sGGnYB~1pya@u--W4j+xZTzScm1J_u007 z4k_dZzhhXs<^IVqAxnK?TG#`be-TMDTW)lW+7lN1#MOV@T~MbnD2ZOe*u9q>8b>2n zQWo2OsLA^5O)-Jn?)*a09pUudFRiGk*eK}67!UOXOZs@o#xFyfhsXOQ6o<$E`Xi-F zNiIoKMaD%V7*&HX(6gM9@Lf^2kB>T<+v;e*Yz&cW9 zx|%H90VEX50~GQ%XN_pfVq+snWilqExmU*e36EF z?wG<1nH!5wQrnyKB`TOWC^$@_Vpg;FboOO_s*=>{hGbz{jtNDz#ZgW9Nbpf=@}p)F z@Ix!f>py@)De&<1BG5x7z<%7O4uI>xzE&xq*vh7R*A;^TG(6C9UVOtGpwl0H9tb1p zl5YoU*0-Sm3Xnt>L;V9t&pRS|@}9c_1NcV=IZ}&V1i+^6U=hZbgP~FCDcn?%;L%1* zF7x$W?)S?t@1m2*PiAXf1UOPfQ+in2RA(*`8)N2Z4J{Pv&U3wcuD=n}?v6sRq3-6! zR;SIs6hXTPhRopQcK4uW0ntQOr1G=L9`{CUyjzMHU}M z1@Xad9boFGfzGXMcw$%fs|fCflHu~uJIujfFxx7gM~8m$#lyc@@IR$}+_tq)IaulH z#A1+P8#ies%GLOG=IbxCn;SYHX)pt61pV9l0s!#IPR-P7f!qKp=(m|SKQ*L-%nDy) zm^|TT3xq8+`k@Wj6E9swOvr4@^+%>EnZ=pIA_UB!?vj_#u?@f1L-63B_Y~5&=+| zk;r0VV?zP%b-0fOGvH))iPP_u5pIBbLHEW$OXU$@WVD_7xz}rSG1ZB0iJ;UWHF?^# z5?=qh)2PvgclQC0)cJW*S}vk#1D};SZBe_5s`dI!aLYGz=H;D92#aAozuTTD$5UW) zV#t_Rm1Zz(*ZquDGslu*Yy0GW&cJ%+r^w3WVItFIpCPi%aIO`(k$0p25Rk{#z)Z(9 zRT!eC~+5GdXAXLD2@3sC72MN7uR2 zRi~bX*WbuAZl&EML$(q%hrg1_?|O1jH__d+h>JG&`Ij6kD;>W7b={{se|(iJ>Q2=T zI$)lLLm!IT7WbFP3(i+>*!fp7u9FtHl2A&0#DD2EDcj;zYH_h>eT6`YTN>Bd7ar=K z@kGJsHmn-m`WIv8^?H*C{)gM)KZbPt65%szeFRxrjuK#AflQYyhgUc-jFOfMw_FIY z3YO`?q4Kw*mo=Do#c6Zs^Ty!tvY*MNM9{nYJ+wb6#!mn=guz0z!VaH^HBn`iQ~8u) z5fp1>eW3GHc^8`tOjVKnN$EF^6P|Y3#H{#_4XJ#BD_L=;`jR+db@ys_W~GX+{Ga&5 zsw=UmR@zD4#lg5xWRQM`=Bml%D*&knYv2BRAAbb@`CU?Kk?N|z%}7tlum6Mp{_WZV zZ6%m#>)ZKy$T0E{LN_qbwgniAZOinMwrGq41qVt5qXfe6?;knXV{+(FoNxA(Z)`|T zfPU8v>6g0|u<4Nk&wWN~&mOC!16)bSLum(}M0ygR(DHW7r>@EgQ5|gKRAAK`V`QuK zYoQhGViDLfE2C5OdLNJ7g9_s^Zs%@UkK5<|R&NsGnXB{ppyf`~d4HA16G&L=y>C!s zm|O!Zwi;+1M4XiNzLHQSZm2-KmNb>3qH5}1 zlSGPv3|`@YY$5JkhT+=wa&^Yl+0HQP{z3!e8(H=A2yLe@K;Y66*=I+mv0hX4{)^XO zw!32Np1a-5$E~A85Rw-z6^aB(1lwtwfalHn??EK-x%SA`@oxaxyqi&|`+R0*q9pq& zS2Ya*lbDsU@7n<=_$FwquWjx8m#Q=>{ER|FqPVN*5*^OIn*31-6J$F+4P`v()50b!{_c={uuG| zh9NjjhG3y{=VEsf8!YKV>sCBd%{Xd1*t>hbc5(;5w$Upg*tw~)E}=vk!6IR|@$i0} zZ@HuM3@&gUY!~M2VZW|CDePkAtR9Q|2zx3gYWv(C-{Yi!d>Ym|oEEB4tL__6xl+YL zNOyfAThCR^JW|&zz%J#M12{ud6zO+SG?Lk_uePJMlDJLjm6G}M)gC0j&zXC0E_(nY zdIF*SpW6H`FYm>vB=K)U%p7fiHKxHe$qbD&}YB3B)BzcY1P8tQODvc47cZneUT4PmIZG!oUyf zMbQjZpmZH;-|MZNgVvi5hrs?<(}Sd8oMn-A;w!-t6L?n^8eO8%#ysq|yG4iChF_=yxgK&p%C4klCrH%~!r{Iw3eg@3~*9 zT~$d#_aVbp3H-OyNPGO)Q#!85OT}S-iwXTUae8_hSN{hX^-Ek|U%whAlM-o5yPI@Y zm*ICJf!T2G_#KNe;)bPrux@LnYC4R4PO~;ZrR)zTWp8@Il)uv)=~|ajEL;71zw1+! z=MJJzHl7)VF{_)vys7GK+v)Gm?(H1WU>2NEwld{Nayfl}?0$es7^Q2nv)kM)Hr1WMoO zzjrqIoRP$M#}h$bQcadV2`BE)V&M}kw>#kZsjGE^N?Ane<4sdt%QR$zNH@DDPLYL;hB3vpco}=DZ*m7Xv|Mp=?3-&N*<4JIfIN$Jq{+-WD#<5bN zT5W5jMkP&04u#>xsxQuX=@;3aW;PcJjik*1WyrUV*&3_~nJ~vJ z&xze~Uk=@dKOi<8mS^3c8U1{$F3=`{PAAEYson8G-wjZ7a&6TsAAG0i{>FwZT()CE zDZAG_!Qtn_taQ*dpGZXDyeH^52Ld3{;IOE)mVq?M7V$Ulr4Fjk>~x+WLvTV)NFPG_ zWas{Ll)=_u||d9fTR>e<@ zWsRF49s3EsvBf%@`7e|cU0>AFHom>zTk7TMsr$$$@pz!;uK(2iZ{8M6c4Vh z>PWGO@gM6Ok>S7gevA`^Wy(k4V(B;{BsYoK>z7k$bE{W~eLhY@A+IlQh6k|1>}F7n zIhHSv2&GOujM9}b`s^5A?nN{Wy>g?7sdl-;Q_DI z0fQwuOS}15Fqb~etPTC;?5=c_8D=2%wEz$Dt<;A-Qi6vLBz!|m7V|)lht;AdY-3f=0}eOP z_BSOWd}BQ{0lWR)s!L3$y?OS*0vtbjhQG`8rkH-w5N}(6*u%jHgk_~pZ0U*9Y=@9K zaAKtjbonAb*+IE6l^03%f!)CY3Vk!Oyc4V2t(t!HCy3tbfAAYRMnlH7`JthG6+5{S zjy+dK3BQ4K|FIiB9-k2V4^hwKk)GoL`{*Hrq9Ubxb?WQmgG}@zQT!vxG~SrhWN_Gk z)K<3x(R&9YQCY-{)sQdx=VAWi`*`w%9jDO!l%$Vh>OUJ8fcYI13b^}E1I=X>2B3Pb zI;_T~13O1TPQ|_4F7{k3-L!%#h={PeFtw0(5B36=A4Drj8A6fxEib4-&kvsD&z(K& zMSA?EI1Bhi$yirllQ|zaf8pbhy zFzA%&t(+~LOB$X3=_qCS&d33+pu=aEi4t1QM=W$L9-c7gz+`9759t!J+LOvAf zxuJ+bP1VJH6mJAD{^(=E>1Xw89GRONSV66*ZUHTKacQxqfwMQ2{lNrn$6bY?Y+qKa zryXfmO`|XX{GKyBILIt1{JlUdRU~}LSjYCM2`hpg_jL^ zI1J0ArytM-x3OH7|AgZH+JRr<&$ZAgxS0mMb(raIT{;zRz3{@zOc86WGCoWZ%$7K~ z(x=myYj{9Tq^V97SGo)S!ET1U-4V5mt4gW;Dvlw0Di$L9Jew}43yCalKF%VK5X4s5Uh521mG&l(<}IzKppi^`qE>tmFEeN6f_J*OzNJBWv1Xi&^Mwut>9e=H z11`0}1fPggQ3(jDx{`{h@Hq`BHka#bvzhN5?a$8>oyJB}!Rt&<=9>^qUH0!LA@6TT z8GT#v-hCoug!+ja$mCVxOx{`CiD0fMy#Ux0%5bf!{}l|6{w@|1U=^YOiYMkT)F~Kq zHLhrx>U{y)~ z%A^izf3WP0hmym#ow|uX`t}()FNqNPPe&SregM=Vr)GBg(2OqGnJSvLYKqtK;b@i> zDBq$It$_YIx9<>}3zfs@9rb(@7Jw14U)zmlioXW>uBthUGnvnVuZ%yg@R1ao&AS<0 znWmga(#(U<)m+7SEqW0d?ku+_8PMxu5PWAHSO0vszs)n<3zy>YlTZhLI1o&REOI>S z%kv%4w>ROADx=%!zCcMcVMh+<9K;v5Hd{R7xdVVxRAKdC@fpESexCmA%dhvrgC8b^ zoLy8itkvi4Co>vygja6yLJhm?@xM?F;uQ0NGOwoBKw+8YG8 z`OUGAD)y+JH$}K@J#h(Ay2i$h?^Uhm8zcI!4uv*>Snyu-v?wBg%?mE!ejBZKVmn-h z^x6y@>oOm)(HC9MdUF1`&C?H?M$*WqQ=n2J?0gRj!QNIJsB1s{b`h4UUw1)+i%v^v z;E!zTQgY&f7I~Tge%idl=9RX#*1qg{w4#F7a4U@>LGv&aywuypVEJ3CL5;zBu;|y& z(F?RG|9W48E(Y`nf5F8C5HkGuP1gSyzJDo9&x@q2rF9_@#O&H}>66{xey3rrc6`an zqCgk5YN7C;F{_Y?sne+nR@{yYF9XiPOjQBaH`}X6(4E+UqI+@S#5E5T7_Vwa#*4qU z>F(NH^p4vLE_Qi(6~Q9-O^Vxj{&U74%u&7}*B=W`!a72vmfXlOtpM9yy61F5GeCexlCe;5R@dB!qz|f} zT%TMr%25BX*ZQMAjRL2(W!(V{LPEgJm0uDR$?{g+g&UC|;n{*2=pZr-=q2clQhJpd zHT+3yc*UAAF+w+J>%A&ap*y{=t6FeQ9kJ6bbke(z&ipx#qy7jYjr`K!3o;f-syb*2JE%F9QJBWuX+jS$e4XlNYX#e=T3#6p5q z4=56>_IOhzT%LLhT>2sKK;Ct;wHRcia5RS284>O!h;ND}wfi@}rV+R1>6+^}P@JRoCAs3rF@& zy@#K%lgc-||5m`~RBGgYEgJ>%6{#Zdcw;*jM*DodhqV7ir4TAKI+pt*fLAyjF3pLS zHVr+A66IE}CA30)v0@0crZvpZ_?P(~%yHkIJFC9GT9dpDH>zkXO zQ%$sT--oTTYPO6lDju%BiblTgnduN8`VI+wrU7ucc*GyFNY~|i88{XcH{X01D+@V1 z>#@qf>gNkAFPC%iyFE0n2wJNL8=K_ynY1)QwY&)0cTTqOqF}SBg6aJXP{Ssh&3gHx z=f4Eze;uR$h!TVA?`RHH12hua@JC)Gk)Ac>4@=zum`^vbU5m)r42y)LGvZ78D;((F zsOx7$h42j`onue_bKSU|zP7W2 zsv^kwLQgSo(iK|{Z!Tzzi$3D(JB}6QX1+}2A%(V0^yyf=-P!l;SQ|4PuAUFLQ8Ll+ zo^R8s%~D+HL|2SV3p*YYh20?4zRh=qUq=6ch~6Jxu^ZOE*DAsB3KOqq$1^k%0Nro@ z2=?g%IMeR3=!W2!I;N1xQnMEu@r@02+;5@C71~8puS#<_O$AU2BTDioNC#t&mus?4 zsSyRxBL;a>PrT35@Xm2@hjJQDuX=PmueN`&Y+#&@`<(H6C;Fc~?~-NX>L44SW@c6l zHV(&ccO1&Dtq)iFA;LV!?pO?WXY}^<>Kbt8zrMXCnyn?YaG|bY*4oYUx^slNchK>u zr8=vBqtfJAoNQyn@D`m!^Tazv2BS3%?r_F?&#$?Ul~O?E15PFE)WP+9o&i+}Xoj^u zgh61N{iE)vmB!gx4<-wxG<{=^;rJlf=2@je@V?hZWD{wbrP{!HD%rELyAzEl#dkWn zJuTE&U+x|8f+L!d0RWffxx4&dZgVu^^O9zOa{VX=ao!xH+nlN5N)OX@4N6O+xv5{p z1ct{3#<8mUKd8_a2^YlR05lfeOrT1L2WVN90OxukX}g!@?Eumj6~XqWUc6DTEplwT z0n<=Pc#J+5>U=N5F3e>_MtqGp(DiN4wLj6>4K6iE_~eqMn|lObdJ=CD|HPbW;FTw| zjTc+19p{^rt}3p{6{!}Y3{TXeV*}H#X1)f&Kc-R?L75*M@7Wu)!5wr>QxRu)VAaj! zkAgWL0TvH#>y3M$H}?B)VWZe!hrJouZKNfBa4M=~=vWU)Gk|wNz7o~tDq;BXkzl~I zT)3Z(RJyoFBUu48Dj5oU>9q6ut5ehymTuV%_*8djjnAVAjf8LJy#&HlQ*+F8Ww1K6?QsOnleL`gC~O#qcWflj%xc;D=oMa9+sz;`|=B zRgz|_+^1M>&k8O8o{fNTjwRboc2JL_O)sYok>2a%qw-q?UqR*Roc1T^WVOgmR7QV(iHet&>T8VCmT9JvWYEwxF%YIfo?1$lZ zjMtwk)cs@xC7jE}xk){xips1blEFzWyR@&-$Uvo;kIT)?4GJ_?m5iN^lG>2qhZ|}% z!6@ZSJOj^7AUuE6H?g%Wx!x2-2z3$wQ(pk16zeQWL$9JwOyab1REp;Qx*BqJ6nx|S zEpQP1&hI{3*a`Nz2n`0ZcOt}3qsq|}#r=+{aH%A4Vy!qrRCMAl$V94~ z(6yLOmt+S&`F^)*ojUmjMmZv$E@uKkkKbX*`k)RR9o}BqB{q4AW7ANk2Rz`N);rtd zSac`G|JeTHo^%>?_TxPKIDj=c-JxfizJx1q9(AQYU)ohK+hp;Pty>p8&K~CT6sIz-UFmEqHz|sZbRd zl6NwW#L`>;q-TI}w*p|%aN9atDW&xr%|=~bt&k=+cmnfssqK-^TgnQF1n9rSWC_Ygev=m5u>!E@$df8j)|ljSrmDawWxSd%%_AMgO&CIo#e*(rNeW9kB4)jZ-`9GDjwC}iV7@HFt#~;0I zdms+sw9Ca3g}^oOJoe$g2Qy|^AcO(1z1{p^geAvWE3rl6WS$R{dBhs1z6ue0*${_KtsnuF7cv3=F5VuK)+a6xK3= zf!Z7o6KICvgjjpL2I$G6`Y&a*zBgosk zb&{@uSkM7qB;^k+=V|g!+(yoAZuRgIc^+U9UTUq$tSYtSO9c2a=H|O{Dx}3#LBcDZ z&-li28gIbc{I&=JWVg9ov_f_7j||%~V&1XKoK&Vvkv_e4YRo)xL`~u}KcteW78@;C zcv({Q+OM8E2)6rsRqS#=-<|+!)MSfMWQ5;YsN~vFx`p+@X)d6pK{QLj*C^gF9ZIjCl{p{r z4z^b@rwf|MF3oPO;BiB+gY}xw38A4Ut6acPuRoVdz9{JITqSx5?Xp*=w z^+oNU^LJ^)HzR3=K*fe)fXio2p_P_7>`$hyFSiK(jBTOb(@*ESlc_Z%)e(#$7_!3*~IJgrP!DUj59DN7r<-`#Qx^+~z7 ze@vrAz$RPbf`n?*wAMG;b)q+YkDh0bm2f3%El$wkvY8P4qA0Q{;92oXeE@wk?0u3ztH<3ob9zh`f%%8^ zceUY>DdHr%HOJwq5wBpo2Vr8V7ywM(A)hM@$ch3~A8R;OPMtV%d(S}Fdjw!*09X2? z+%6~SG2R;~-P2tCMoToQkIG=|h9HRwS`H^l)2_b4Tym%c{%3ZLf>OAv9US>+ap}w4 z?jB3+g|b>|4R(l%;98%GOQ6q`>AOW_B?n{F+w=oF+DuJB3S!&4=cw(!-!%#io8_~*eIy4JyrgK1O+t9Qa}?oetI zTa9gL8gd#NCIhWjX>N%OA#%iB$7xWcgZskupRkj&o)72iF{Lo;{Nt8+UxN*0DcY*o zj_Yx3y1~62OX+^9`S)61Q-v>RHizw!J9`K>mx#q1wy`#sDhA1`3~>PAw%#(z8Y(Zag`B|I(%Y! zG@q%sejL67f9NI(<$TE;MpkhD2DZ$J0gwSofi_!9`ByNYH`!8=Li#_10hW`=0u2{=W0N2$ zUEcx?av)p1eUxL}fAOj`0U)Uv6WLh4#0KV%Q`T=u)JYYAt_xF#${u^FuuTOnx~K(* z_m@hImQ$Z0l@+FSl+(?O2F!s_mP)4q1dX58HVu337PPxSlYj6fW)^c_Jgvm$`prp3nBGaMTpm6J#jfP_S(ONU7%^xP_kNd71x*oR*FY44cwaC75G>}6`aM0- zl&Aayw_Po@u<8z{FdjbrEznGC)1^Y{D}l%fc)Y57btsk+7vPWZzU#^`|8PF~DpAaX zMKIV0B|@gEvDIzYd6odZ?)B9>O0NKFNe2W#8X; z5jUZSqaN3^S}O0iY*TyGmhOvSAF=vdGNkq+pQRQ?VXu2576^;>Oca1#J~J8ggsz<% zlb6GzOJff#Ggp6|`F(OS%>W{}anLnd)u>jTxC5C>gi#|ex!*53B* z=m2dJa-WBDRsN{?Xq9ijEHu*={QYDfiX9e@vS?pbsph3>WWJP7xyRp}hw1Np!Ju31 za*da!M{RlEaMp(YkYN>;3$Nq!%BL1|09V)KjBi!~_0W*m;Wgb>M8%$(WRCulX4t>!BAnKF`7VBobe}{u@ zc|gnI#b#7#%YFhAq}iYK&K?v71X0fWooro5KbN2M+~S}|fY!kC)aW6{8ybRzohyZ- z0Gq0WMV1S?`9*`e`;8si4X!nvvCo;97xZNuOW^ct<|uCa{1@E=k}B(8LKV)~wkFL2 zDaQFa7&x5JP52hd?%oQyolYR0iRiF9Z0l$J&FderJJgJ!W@}08wN$eu)QfgIeHb}xm72RzFXwF-x!N{A=}@2k z`p$ZJ(Uag5En+TA?ezP1K|;uw%I&BT%dwGHRPcA#d;E95eoVxr%fG^l(Ll=9eIyiK zUd%FOy}brhU>WEq*@d_x%;$ob_*uUFLGDCg_XJKdES$tD8+X`hvc>>|n3V3B-zWT5 zgrMSsAV9fW5LPT`0<$m|^4%{G3QaX)j@)8BK88x*kz6qDt7~O&1gEcoUc^27ZW0tb zcE*8VT6b|4Y=h7vPe?wT6ME92O&RjG=19`+gHltNCs-WEMEJLwC;9r?S7KMZ^S(N# zL)uFQp=#~m&C)qqKn#M%9>2c#qP_k6`pga;6O(}HwK@aJ7JVg@Ub>cL6_0#aUMWK< z`WdFMT&NqqCJRA!gbeOiIwL4ixSZV1DDou*fa$ORi4h1sVffS1%{N{GF^ zHj|l3;CTuP;NR@i%sHsQXeD+gmN8i&Km7*pn_Kmwns53s*PBWkx|Yns(`#6vzj}2| zJzYn*-k-?Q&BFs=i&6t$w~sb^R@iZC`P8#$7VOgQx&9c}%$MFgX*in^$j3+nRi~cxLy|;waCh=_^X+ zn!l*z@{FF5^rpaW5dE#uJT2R^>XNTVwW6V~N-`oG^fHDng-iIx|*Ln9~ zWYdkl#W^LH8MTHd4RR+(hr!#oJ){E}1_phTH8}e#M2`8{Mbm!Uo@>8r-`zNB)YhZc zH)Gm*dv|9kP6EEKdHOq2&1+x=&;RKZ5@QO9imDi!6~+yqL1NQL%)?WwiR2>o=#RgC z*FZ87B0+8%_NpeQCTT3bJBlU6>lKTg2YDen%{NYzD(ap!N1~uMc-;ejFBJ`GbM6bQI5Kd9-z_uR2h|KF<*QMhSb9v)bZe%!poKcfJ%_Izx#vMh@hsyCoa-5^2 zSVhOtG))gJqBgPG&5YBc&+N8GLq3GNbCJxMLo8iT3tC3_)OlH6!Q6NL%aAOSOx||7 zad|Jf_)@pco@}qJcdCx;7j(*n+6rPZ&Ae46ua&jJO663cs}@bhg`5MqKY4btD~iC+;ok->P{_Fy}7RS5ALuO~@t{Atp#5 z{r74=H}Ju<9I1j?%)P+?M7x_Yx1f<@hB0|LK*R&m`s87x(0nVkmn3NSnwSLj^Pqt$ z4qU%X5Px1#$pCx`nG*mYTr{q-)nCv0zE?qjj=S{1$ZTjwwaziI-IL2^WY7rHInqL9 zcNp_!>BmT6^g7E8qfK#R?>=$J@zWD%al?bHwIMKDFm!+JH;7O}{EKFG#67iaZS=FB zPmGxPZvy(!L})gB2KNy}X!!M!(a>72PBwDVfH?v!DWXXS1t;r0XeXr4hA|ab|0#N9 z9_LVG;Ppio08DSi2pelQ*K<@1edb>(hU@sxivvCgRhX;uH1^*Low$86!CmUBf`!LE zbqx$;U<$Gek3&5hc@pI$*K|qYjtm~VrZ)_k2gDzQq`6q5bGRN(IaE?PZoEMu& zDwB1hZ6E)Zr^dHA&c*pNq+;!4;9J#`Q!1es9k?0t5OIiK$GSVrXULvM4H-0K{!`)x zx6ihv19Twoj}B*2-*PN#>e^h2Z=gtxB9u0V60faX-fl5eIB6RoN(}Y$1q*&o)UT$?q7FUX=uWHYK-Mo z;)6Ymt~YyooGkpS-R?gRvK!wbb9;ArQH#pnk+FdQur6|U0VN@^Y6_74emcMM#BKwG z@=(k%TRz%yp8rSLcZRbazHN6IT}De4HCio7(NcS)sz%XL#2!r*wQIxE!(!M7V5g}IbK7ei0A!7^i)*3Uw_yTE@q zq}na0$46A&Ab*duD0AhIc4i@Y_qesDz2lH@eh4>nKmIO&eeNzxndpDiZg1ZsuU{t< z5veb-4&OYO8EVs?phT3f&HzZvcw8f)ezHBpnpSg#x6Kyb6G-WXUY+ki ztcyGzefe+QO%tS$J2xZOm3Sl*#g~)N$mM0ww}$Y@U9-Bo{cMkT-#1%};v9M*QZ@9^ z!1X#H-59xtKhlMg++!+|E12fpua1_W+zI3WYQ^rcs`=-qqxl{s4LrB2dD*tzY-b#5-zWy?!0 za+@rf_55t$={or78ev?hqfv!LPd)KOTXDuY7jt=c9{ENehs;u!%HRdB62oubDhWe> zPGV5*9DzCipjrI9;|LB}`bo|wFhEq9D?`4V@x0?jdGJNX3z>{dA`%>a?p70w`x8hO zV1Ec37|F!k-$SBc1ek-aUGSRB3}gE#FaL2jlQlxyMd0tR=C1>R=gS3hfv+$5P>UjDlBs)Y0T|xK$}_9$m}uYYHijopS<3!DWvABixA7_#Nc}x+kfZV${`WKr7@;JMp6TjC z7u*N%Rir5(^wU6O!Nvje54DMr^*gIKnEOxJ2YpX^Te)$S8JLnxELXjLxXNxikUXZ7 z{~-CUw@Pq%>u|fC-TU0TU*U<;GfiOFD*Hj4=(P`m$l~(oSm(=HI)ZoP2hv4A5hjC# z=1a%R?E83R8+Xj`Tt2#4{#Q$_$M4qJ_P;g{zFon84dJwr;_P~0@OSqI({KmyG_w~a zoW{8>f~?Y*9y9T|o>2h7jz#-K^z8%#H#ry>FST49uXpRbqfP-kJ-07o?8B60EBP@y z>R-EM&1F{6?}in34ZhT}>>0eR{vu zMbr(#_Qz$PIdwm;{O4uok?PR&$5>ZudI(pgj|F01j}XZN=C zH6_w0Rl0NXs+66=`B0(swS-U9T!3U+Ry4P9`nQ=5EC4^t`lo=D3(WxR7fwMI5nQMr zXuY+??mE3owYR4)a_9bzzS){anrYYQ#@d_@!D|8Q60*2!Mgk`t#Of)eSkS;!RLd6eOzLad~6sQIn73mJDsMJtjs57Uj#0OT-OSyc3*xHULS15@N z^?r&lX8Y?nTvB?s#7vvH*Chd;WS$h6NBLil?nhqH=D@YiY9a{SAj zyGiGHH9mxB@_s+kqsRRC!Q>aI4%67;jxm~xLsQzkvu73&^_g`WZYeat{NRf%esC??5c`7m%SeavZ*3xlc>jc8 z18&0qDzMi0awI5>esFPGFzt%8z(=ZMUa^_{7oB_a@fXynhl>%zuv3I46`$A;?`f-@;GcU~W^tLPDT8f=oLYN8tiHDUrbds;D3G1N!@y%DeT_AdAjU|;eEq+)_J z|15G_nBm5gw}1t77a;W%Lehcd<;N%PJgj{B^y$tTPW6zgb3SiMh?y6Zb9uQp`Roq# zAZ{}A;@|E#d!ubYOqV;Se4;l|SbB^CNLgg$X(dv?6&Tln`>u2oU>mX-R|~21rCohxj@e%jTEBRut!wZ3tET|mnK;$u?OD~}0eWNQAO43N z7rdGAoqwwxnrmS8v-^#aEVJuxT-_`Eu z zwRQ@BjE&x2i@aER_)l9QRj_xr4|KxFC(kVO`=e%Kz5SdLV(Vxd{q^tUhBr?o7FMBu zgicQe0B&8qir;(BU;ODk_g6uwYx5V5!`5k~#|w7qY6*AT;EbU{tv}Np=Ha09+g9yj z_nm*_b$zPLcpGcQ-S}IhA@ql!e|xwJ;xZuU<1v-866}+4=w+d5AAV>pjSM+{<@(C* zPgT6W_L-A`R}*re7uOXEQl(TH!=f|aS$wQWPv~zqPl^8UjZs(djugV8;%uJN{QGl2 z?G3LHN$6ay6Tf-YWUhi`y!Qg23_S@*XN)sxP+C~Bs}6rkpa#KFKT?RP)-L31Wj1db zJ8pC896IkYE?!A?L}>HP7XDFFgvNz7{;yGQ@7$|35NQlEJ3qXC_3o)*-n!DfdHOng zy+gD|jEnfGweLd`I8F!1=Z$G6(dAy$iBOBlSpI-D6vwS8pOpyPh_s$pG8%voU=a{D zb_ETWJpPk>I~KT9SXaPySHhw^fA*)x7^S@ZlH2R@+M!RFu|{p*7S3>Ct$zxnH_4*^ z+mcxp|2A&MNdH`9>M6iG_nk$}esIbKgc=YTd8e}ipuHo2G#>GcI$5?Wc1{Tu;_@6j zUaEjk(ta#k!a0m|$Sy5#rKbRZ{ zg#Q3`oEp#S^I&+J{oHN>URdMLO@V>?^S^01N@$@|kzjuyU6~0GRkSU~?XHO9Gew3a zwJtFAiWdXAU#!dXv-`#G#72w}2K)zibGa8ifo^Yb$yPDUS2n;w$Nu-DXS6SYJ&%5y z@;e&vT>wa@M6M`Tp0;T@w<&}D&v(^T0tkmCzcVu*Y=5_axrh!ds46n(G-mT9+D&T zeAkWI_?;m0=;1eO5LHIEgow&kcP5e?+uXgMCmE#*{!1DCziqI;yD_BPu{Z_jB)JZN zpvFXg+bck?(AnAfno5R>S~uPW5?syz2`<+y61B}whd;XqTyFCMfN;6e>)_uFz|^@! z?9*5Lry+sw&k1gBmNIwoFB>yfe6%>BO4EF_KFz2;DW8u3R%q85AWnCsBAl*{_nZzY zeZ=}{HB!F9R-JTUd4gTFOVYCL$ulA_8oDYPv9k00M1&st^|$=Qw*!E^!XH3!^ta?| z6}i3yaYkVZu?SSYwWov#`)JY!wRZ`*Ec_eNI!D0J2V!Wj`b*~KvbfFd-=1W5_o=co+1U5w8xTa~F^Pl4sX8=T?ixx>$ zO_Wo-Km+3!hxj4CS|^YE0eIa2JG)YvHOyVMO(juMme5U>UjSx@T)xh|8s#$wzjcXC zd|L|a!unv6SJi*7(x+d^yeWVm1zh$VfA$+IoniaW8PDb}T9kC2faqOc6d#z``Z+QU zvw0VoT51wqhNVxlY`4M9mmVPOaF(_57 zl)ih^Eyd78U3gQ;l~D24 z%|e!>9}xWRZ{-Qdl2SLYfpe|5Uuc`@>ySpvl`8p~tD!pSGx->O5i^0Pwb`e5p6@20 zIFf(|SI#cbEadrAROQEBn`-@YT!sE;MF32W*g%0NqmH7M5Z8lES5!G!C$|?uzsJbV+?u2=WH#s6LL<-Jet%g+uEmLDb4~6a*C||{ zJErzhkmqiqg8!n@A%~~vot@TW+A8wz@qMnyCu#X?A>?DtT29js#!O}p=4aD?z?%#k zGz8sB3ctnt9}fJ%xpnY!fJYccTF7|09qp|c0#?$WCRvlG;r|2BtKv6Cz1KtM{K5Q02p486Ybt-5WlW| z$|)*akxJa2U^r@E10qK39_$817Dh1YYU}J4n|J(L9jlIxB6c9YNzwEy>b-Uokvu8J_ zm&IIfShBJAUiqv#MMwE`E7r_5k%k)kv(k^UY%^-q))cqb1{$}h^9!F@Z621mqrdfT zo2jdK;u~!foH~lzn+(C|CM!vV=R<$DF>4R&j&h5FfYNH|*N2Fz`eK3Lj>EQwI4Y6u zGF~o)x7);z1#E_lS-d4#ezLLuC_ zK(&3GE?rm`(jxOqH$@gm-XRX=$l1!3$5m$_W=g752*;OIIaejn8%z~#`uF=-dawA&pJl3O{<^R zQ%NJ%Ky}`H6&3gk%4JXUi>PcKUHTQ9def%4c4T_r=~o>BQ`ko>TQ0WC>d?r(f|g1} zlWS4MxiFy_6V86hHsTIgJUQ8;K#-cmvGLM^rWAq}Fz$=SPt_ZQRXVL*b zeQ|f~fPW%X(mQJNQ+DbaK^TNC!Jd<}PynPck*Ki|YY0orA3613_)$41h?7AU&$3C~-baBrt zJkF2S1btGZjn04;*f*%X>p5gU{Sp(O`;lVlnY-WJ;_aV;^r~-pHSc!h{mTdevI%1O zqKkIvE4K^%vMW|wEHYC{{;$kjav^je?i8SyOzRP<>N1#Vgokap|Yv^FyCy{9DO^TbtY)2O;7JP>@RS z#jDpzk4&qlsi9*^>59D;yPDCX&^n0c7A>f8i{6_URlWa%Vd4-QQ>BWg{LCdF$HCEP zRY%j>vf}W{W^y$y&^K7yq|T}4VRbubv=AjYwm=q~4$bM=SV;v#*CHq+XEA&1$hN~$ z+ch(g>Nb5E3T3O$oFPgWa$0tno&?v>{kl?^M1D*xc@`SOA}6YjAQfeO1?=HUX?Ua%nJUv%0~OeHk}xs z6u)bN;j_>IM}i^d!v9X^+wcDpMzd6%;w=QPhgXv3{Bt?qwBaO4+Oa($jy+d^G7trn z0ocyS&>k8aXR6&3ZwO+;jdWWcKvx%gY`>s7yiG0fM}NnpA->zMqA8anZ3uYM-e?}? z5fd+6Ezq*sssB#c7ER1DlU9DX^G&J{Y~@sFO!X`v^t314cQ$U4<-DM7ZX>Z@Qz%oY ztJ!n;eumn-jcEw249ce@Ncz#tS`bimYN9#Ku?S^(bB^-SXC`C^BX!F&=b8u3!~g40 z1gdxXVtn!Bx^7mTt7)PEC`;I1xZ}qwBPMRwai4d7(Bv6HPX~eGYZW?q8;ny~@taK- z;=KtABB6kM*QfbjmlqP>{wT5(&k3$iAIOm#x0G`_4m8-DY%=Hr`>3f3JH;ctUseR~ zjJaV#h@`80Zejbq{CiWWxB+1#CO?Xx=Rj;1wQ4R|$96gBab4yM^3T?a&%X6TtHYyg zPnJ1ztGw3xAdTQN<|3Gqt{DqToj@sy8!VW$g@+J+S$Sw5UW!CBeJLB&_ruS7@=}j8SD_s)sbzZtiMxqF z%15brRsJI_Ex1Tb!S05Fk4SiB(U}S$C(jTFsl2Q!tuS+)nKx@`)vO{;aI;M4Bapv# zfwymrSB$wOFoM8K(2PyJ#OJFUQ@VEEfcLF7!CHwK@xwo7?Bj=Tae~MuTP6sULB~#* zj|V?@WZ{Z=3HKhdidCyi8BFh#g33MgS@kaFrxxngQ3f<~ZK0KQRuqxB13ne3>}~7X z$Qh-!VPYWPAK&A5X}0__7EBPrxB)H^l~VcT zz9a)JgODG^2foD75B6CL=3ylp_M?A$>$RSH9cjlR<=&ONdHuCZv&_L$rdbT1sJU&P z>C1L-B-w^sX><%}lk!#8Vmk;Q^Q!g!U8vIfWNh4ironycr90WKe`2&YP~27Vx|6N3 z`Xr#9tH}WB1Z!zv*lAbAB5pqZ&3!i?@RP11g+{=bRHrV3)Z_aeg$woz(E>mBGc6mx z{%e}ZW6?to+VlE6$bPCMU(V|(+<_x>CaPcM!C%I?5-TQXK&A8vl zOi0bgH==G+Y00w$EP&g-r=j&Ub!2J5SG5dq95Grgbr~|dtYi?i0xqCNr{f|6vgfrM54m&ZxM zwFmfIImJO3WhX9OY zu||+*Zux_ke+1gICYc$?j4#czmI^LSLa0cBd_YNQZBS8LYL1p&p?HFyGpW(K#Sg7u zYjzG_%FjTD!|zgXL%DH;^`1V1^cushM(EIbnZg9}_+?c{?W-Jg0=5p@N64lEU@5?E zz2<1C5kUVd1OretnbV>P?u%DdDC&QY3E~r*SbRYU1!^zCK*^fvF6*%evBn#nksmR& zo9(M(WrjkN3b#^O{8D0(ZnB6U4N8%&9+(QDZiCd2qw|)On?DBx;CHU-xbv~o~!X^7r^-0qBT9jfrrwhOJ1Tlt-%dEjQpUzP!{i%5{k zx&8oVKJMfL2Ah*m8OS5o-VK>Gq^W$F#dB~=ooV+y!t{9HsDC9Px|99LyO)+dWalEw zD#Gll`~Hl*=A?LPg>L($i_6QUG80kj6z5-w+=+7TF0M9iGAO^RZf8aYVf^k!1}S#} z2CaRAi5I$p&ZAoR{!|T@_K2oaCoZ|-e_xmC`>uwv>JRG12)fa~COI50`y=f$)vUGE zY}Z@Y$}Xxa0oP84Dc<9Mu3aK??W>Wl5=MkUQJ9y7*}Eq(9|YZuM15G!bm_ejbPI@U zSVLcmByybXbl{7W9lOI|!d!3YM;-L%^nCX9Zn9+z&86Eh!k6L(+%V;Yn99?>n&yO{ zOS|MC=qRTcy=p5zwFfTfb`s1!fwssFTm5-&M67GVb-&WSLxD^}^>(ZjO1dL`)&(~+ zm4H(O(=*Z8t63TH48LLsHi&MO?p=?mSfY2fc=n}=r&_G_<;bLd%$C-c(C+Z`ePbMb zY0!DUz#>tm0shkEpg$yTkz9}HD{Ag4RRnA=EDyHlgaq0^DC=8ojPakOSHwzI*4G+_ zmPTojW{kk-EA)PRhqBL=^t^|y+0*r(#2esA*mF&MEhL}Dp3x4Jn#HBE8=F4a!Oa3( z{kv*&Tn+IFVRGBW3i~()Zp|O zf*kiI%4V7vsiT!m3LzMZU%s~Hq#qwxb5aR@!|fU#hu=&DE4%&(zqv9&INQm}*dd>u~rO{Cfc_UO;U>8g4xU#>0i8;~$$v!gS3d zi3+sexnaLQso4z7-ON6h^%VWq*awubTQ`BgDCHFX zc%iUzQgp-a)pGdX)qcD0?jUrfQd=;^QJGE)9w|X)Zfp%SZ2j6alB>>t`8%MY282Bn z{dNOcBVAk0JX4q&bSAGo-h5iVfrL9~J%$}iC8+~PMzv_w4?`Q=FPSeB@0yt4ggo&? znF$PSPlwj}x%q~SO9IDh;ep&z#8BZ`X2##zp|$) zfqe8Bgknd-W#A!qEEe44LQLo_tx8$)p%cC>kO;cHpM_%UB+JtcKol(+=@F}|Jj{-7 za<^%@KXuH~Y3T5BB)aEI`F_bJ4gV&vk>hu+6@v^YdHHq-y-*AH$4bU?&`u0*cg>@0 zB0&#S>y%vQHgv|7&6M6!x21vVpS}o6OTE!7hGoUgr(XJX-V)b3*ow$;9AAyP?vY78 za1*zG=kIT|hU(g8VtI_FSoMR7QI^rkqNav^9yTr#nKEc>ZOV4oBt%lTq8#=6&@9NJ z-)nH)wZaJZc>D99DVbGR%mD%jCAj{{4l*exm|w`v#Yx*m-^tiOd#y8_tz&f*^ATK>m9z zp#H#OKMxNaS&|BI{z8mi}93c&}+ z_3c|8GxdXqfu0q%?bH@0?0D%v`F5${*YI6!Ci|opH<&Xy`m#>H-MIvOV75#6z`8RK zRn3R5kIH0+Vk+6SG$&hLusEg5RR@TMy|z*f0BmRdInOKpDwm5JO^l9F2SMhZ{`k_x;l9 za&tDuecqTfn}Jn!N6XiC$)5?-7XMEOGroie!#$wM{d<8(FmFW<$@HxThHdBluFOpX zTjDw>A@o;m$i`^LC^*X5bH4Gb!L0<+L<+ZQJbe^o%be_{z3KJIgyV%FHL^S@`{=_rd{ACTGlR!#KA1)jN_|@DfPHX z8-N28a0huVz25wtE4^GOiQV&s58c~%exMhj^*E!pffRV(fA!{6qhz1k5c5;4ATkux z7~j9SQBjJD@I+Go_2?D)AsYJ#TO!D%eJjP8m?kxYL)}#raK5oelkc6 zeV(n&1gJA}ZutuvEY{~X5Z1ASO3YhI8_!#5);qKG--{8_wFv~K-sU}cs9nI;ntR~S zH~etv3K`~^GL+$Mvnt&zDxsAQurE#k08hnXwrASmqj1;tMu6NL^W+(8LdeSv9Ux71m=WfC+s233!8nnWQW%X`xwA>We8w{DC%Y`@{~m~ z_K5U>MFitMJSpOd@w27~+`)bG%kuTLiepwc$JM`N1kNgR8UP;OjQIRDKl9>T*96jH z3{K0Cv}rcCe7_(J^!-$z@266hoo^e}S6Me6R-mUOrBtpP_q5P?`={6MeWPEr_}aac zui?CEQn#mC4Izg3jNs9P>Eb6>1+$e^w(1jb;pKTH;MI&J1;X~~oF|3Q8VZtJvBxus82=-e*V0=7AP zl5fR<*n=sZ|AAl0tg}MSiJOT^EWnnILlf$*pO85NIsInnHan0E3=|f@UIyEMMFxwi z7g1jGAGu}vfOMI}R#sZpCqX30ax?bCsdgYfn5cQ%mjv|q&Gta-5YQAFzjbpi%(Gyh zuy#%fIsbV@ajd(4HO{pRaoec1U|(Q0gd$<=Y$9fhFkWhrx= z?BCRo3IiS-dn8^E3TDvh;Rp`%)tqd%nMu!1eF<+~%_6B)rPx$>_BBGfh585Mn)a5H zAWKDX?RpV;vr@C|wM>vx1~QJ^=Kvzcp{|x|+bsYr-6ON8`^9hJ=RCjW-+JM;FpDA{ ze~WItNg?6y$!VIFi?#S4-%%1o6pxY~>7q^wjc1W)Jj&d39}f&##?Dmzna}9;KLDYb z7AtKFe&41XTmEQCO)aNCH*D{-m4hW;q&wy@I8s)4?dXMNXsI!;6iE|O(bKaT;Od`% zvn%0e7a*c_eZJi^>+EM=D%Ly5YHF+B#*)TZ7Z75tLS^YTz{<`r1QR|%>Wc+r)H7gl`A=B?5?bv z{Z1@zeMEp7%01qaWZmXHHBC|D44vS#;xistYYD}}Ta{kq9k=^8K;2kY;1RW24~NZh z3bfH*v7;3fpS_A8SrNW#o(G3AE||3Kin4_=d3at%Pg(_(7XXe{oRP%FoNv*zPha6d zNW~4oO%A@ux}vlSr{%mH@8QB5K%cI$oO4cW7`H~Ph^R#Q-u=-yM4;x)TU8kFyAmIv z)_2P+YvTKr)@bp`@hf3ZsS<$QMiO9rX--iZ8q<_$p_CnY7q*gy?zn>ZBGK2wdY0Q@ zu1+h84;YSZA}Xox;(!XI7pxjOzv*1<5l8T;Ji3hH>0tXao#RA*=nyzKlr-UxQhWpH z(0Sos4!0-}Fa0)gt+ncGBO^o)ND7X!JI^7?HUl|Z5AaI>#0eQ(3>B87F1kwy; zXiN6>?@j+Uv~6at3{RiA2*%1?Y3+DOrTAB(t>-A)$>9EMdQP=mNqXxqYzsJfw%YHr ziKyKul*@JA$C~)AN(i9+`Q4GQ)`HHd$}07x^Is9$63ZZuTGOlx*H#c$b^<@7C)RpokCXZUK=B82;XIupu#4lU~rS|mVvaX~2a9?sie3=T>@ zRjhI3lG@7OQGO&03FJx9pTiAKuKomu`iemh_ficzCPQ6EzpkDKXw*YmY!iAHi@K*Y zVB2~u8jrLM`fn&G{ImipU#51~{=Ea{F#p$i*(~ zQgi!#Eyp20T6F^plHPhGG(rMfCxsfw!CG5pJhs~vEpyzCKFu^gkN|k;FDh%A|AAEn z9BIn_DYDGfr*h$C-#2uWia%&(gv2Iu@cDYT3_WUAvl%38rjH4? zT?lkYgfm+yWNZ&Swttza$MD+}RnJ8g^>w}WVq*cIO(uYsN*7} zO@8(yF8it<2- zXr~@Pcf1WN;=LD-75mk!4zjdPR2h!^y=R-t;*Z5P0wJAM@%?EV9)ZwlYhv0*-G zeQsrazvI@q=?3>`4r@Cr|fk1=z)C&vpVv7eQ6=A43C$4%EZ?loL z_H3y*Y`x*-h;lVn2E*1u7;Y9DpV7_kC(u-a)(sk7H?VX)&1w4%1Eo@a5Cepmga7#B zZra%K+-VM*FJB;%RkrDrEsXkKu5SXx6YIfPGtTGo#U`c3{g+X-sT7k}(@F{2^r39! zZ}2UaPfI=VY0=yjKf0eLw;r8WNHF#x2j#b0Uq8P#SbtDv*p%UGsy>-E-7vAB)E<^_ z;Jx?L3m0HhSDyxFcEzuac97OOZX-Hg({?J&d!_dh;djc!S*<7wE*MKc#d$PnvR)hD zYxy3w-(ETI_~o3~h5pEY1R$Hjzp`1gHgNAe0OD8rQ{zs!0f84U=I?#32cCKs*9O7O zglx&W^XKlFdR0HWs(Tt|>!Tj@gx58Oc1bJi~G^H=C*gWiMOtPTbE4{s2X@<2C5 z^p$N92D1h0?%lVD&3C6P2tnh9{oF{uh_3Q4w~iB5R?HmxK6KE%Ts1&lF#GYADpPzG zY2KR{J2)i3jAQiW{L^bs={{Wd=gV_L7ge|&ZfQ3V2GeQPwZW^;fM}#s>gL7|Zvyl6 zVuRcyBzs+!$TJ@=1~64oD{Mi{x;J-uy zU@nO*)A;&-nBDNRn^QH~_3ksmKz|OV^FQTvYEkyv{G=7f!%flD(&{UD_N@R2D)r*$ zD&YgTUELhQ4Yp=rh@;`}25XWIuWT1(I8yTr`xa3dfBPX51Pq=u%vl{s6I}4LcCvZF z&KC)rYSaB%ckiu}Eq2tD0g}D&USO~9QLkOz>)M*uuW(`7ANGIEA}oFXTMwmF`%`8T zFh{1$6|TOh#UxxAOp-EH5YJ!d+qYkYGFO(sQ!<5|_?I#pi|LBD!ZsJg#Z6xApg;#L zbx9K1PrvcFVaJ_kCt4&23>W7G8k)ntHRv~;OH}rZJoewnA1|@71L6XE%cX5o~BCvBrgl%k+(%LoN8;#X)yRWI?`@a)qFoKFjF{zS6JC5ywcr76FkQiCal_?1kRfv zwwCv-{Z6)u;Sj&YE#r`Qfm?_0W}EqS9;hc``V6Yzj)e7~P z(WLwVBcNT7uC!(XCQ4;MkD14(q?Gd>IJ|93(-O>x;8H|^oChu6DmqD|W2|I? zyc1DZXua_wQ4Ff97<>?ZTnQ^arzDv~N)6en?`VT_BlbF2gIaC>Yp-y5syL(39dq8^E>k?>ok489ZN+d$h;^N}MN_exSMS$TRTkqy zlXMkdeLotxWz#5(rgEIR>vOUO9IzjA7l>cN^!i z-|}{?oc0V>J|OCLV+Jvo(?%T)6$>-);vU^*C@tV$%#k->vDYUZA4H$)@Q~k%>o8EF zQ_B-#C6X+m9%pgpMDC9jyL7=GcCCv-502O)&~ajWiySM*_S^_JYtFq87yJOZBV+uQ~{$h z|A|lo6+-K}AhHH@j{@8uC1zIu5)BV#(4T$86i2|ux)Z)E_-)q?IUN{uU_aURnUtD{ zZhyq!mCno*u07~YlDaWLgAu2sEsp1vO*;l?=@o;y0a`q&qXQ-DSCi;ygjvDb`La0p z!&NUbIGO`+_UFYN+4C$&h;gVob-Rv`(Pi6)Li?rSazXJ4PIO;3Lo@RIIbWzXUnDf% zY?EY=%LRQf9cTv-UK#A44*ZVA1g;4L6f})!5(20i1r1J5K5tKiUB)Zb<J1(J^VD(6E8p^wvas*nl0u7?W>Q0 zbkPwq?p8SS`F{j@p{5@&2%+eBc5uD?A#Z2Zev#irOd`jH zchD?4+-?gqI4zY8XI6HeXu*cQrx2b1(t_V}o7!9sYs_8UYN0u#uqWU-V)|Ok&j5N@ z3;F!F7YLSvpq5Z-=)_lu>XrHW1v|Pr=c&-^-3J%|9UUu9Kp&aH#ejjyHP#;G$)?Rv z*y-SQK_xa1rG0Mc>Bi_RigE&|k?zmwv4yY8B?s_b+b&nr3)v-2vDoZ=aq8v+Hn|5oSrZfg3K(kwA0QAbhA|l=Vv2?c)6RS;eTDEq zex<*MN^?FUeBL2ABD)sU6y;vtZ`FWL3`Ps}+Gp@gAS838vNxbLz>HpL!+mLk&(?p( z-l6nT)H2RsO(*_Zyj|nI-z(8G5F+&AH92(2;YLv0w`i!$!=ZA3D^Ez@DFSi<=VpXE zCN@qi=lgf}0^}3fcXmsaY5A?Ta_@D{7YA}1F5gp4H1#kpdF2#zg)vwy3#2aX=zE-4 zjFa=p0gRGAD{w-fxP;nuv0PZwP~eS(ri+L*1RF)w!bdQ>~&k-!UTpxc6 zF@SX_e&?)mWje>uwIM*m=48W6wEg09Q z-Jy(AQ^`cB(=U6op2brO>1F(|(PVol!&I7ZSVSc`IL_Vn>Wyt!<)~}Y^!2cL>pjeF zk_YGW!E7Sw#zLmDmZhA($qg9{6FFJXEl>Lm;eTof=#v1{ z(F73Az3bEQ=3}1Jej|v4b>Co91?Vqk;!U+RWqwY``F8=Dlk!gO!d7x+4PVZ4Nqh~j zv`4P|%-Wc#%g$1yCa`CWu8jclBh_J_3|VLvfQfL>Gu? zFv^*D+|H{RuI8SA)+R_F;)er93X~Dsz3aF$WVz8Z05}{h7Y?B;wbyCu*b*g`5`+&~ zdLt%zRcM>Ii{3&kjn$>O<0ucK4T%K_0xG&@=i)={*6~G4EZ}4lgQDE3JQ`Q#(prCN ziEfd=RrfSXF#mQ_mLyEex%5;jX`;lP|1S}Jjf-&TG}93_9YyuGwssaJ?eoAy@?YdQxWr{p2v%SrRrd^lVO zm(*|VZB#080U1Li8x=k+J5?y9TNv#6Te+sVR~XzhxrsV0cAxgDzxZ4B+B(34hrFrb zsVaZ3ym65)GA?)&?ZV1G>C^^l+2;`@9U20RE3KpAsYW2VX{aj4aG}E!mEmF%goc>; zNLQd{s>^GwiWfEVH3+?lbC9h|c<3M;HNG}=&A!c};^B(RXjSk4Xmi)?Indf_v6)t2 zdT@4MhLhjhXY5m4re0{HZBR7e*nqqBz4mVvEk!`7@{*j8end9*?T~@)nkp>SnQhFdeEdPB(&d ziA?V&LwOs1)#}v}l_p7hZ{oX;_Cwq*tD6@;ePI%bF2nU9>UiougJ>#PmGu5uK!{%W z1Xm>KdJNK95RkAWT|vcs_lOI+vyLtH9J(spWHr4J`bw98*6M1wHoG|fC3Tt=ss$6B#zFDqn)e{;cWUzkU3H!dPX1OmJu?knP zt3hC=E^Coeiup$4|yLWvyN9S-Tnd z!Cv*ingQD^lFr4Q~rv0#ut5LOKhxX$k$V< z(U#D_q8nDfsj)=f4+w?g>9Q`RP)Iv}3DPA0oI$x6V?m(R|dwe*yKsD0>fhD&PNq z+$sm*NcN@(m4;-mj8MuZge2Ky?{!3kB1Ez!`yeCPM5UGA2-7~KvPefqmkJJ>$U$BB9IzML88v-4S7 zVi9qB>%upwSZ=DEsG9uAK0;qZ`_#1;FOH~F-ByY^$){5|k~E` z-I0se+NevTpZ!28AF9Ls_w%RMl=`w8U=J6d3MlyC*I zDwIthzdfjC>G#Ke4*%uxv#D@u$wZDycUtm%s`t8(>#BPMkA`_n$LH`h6OVY(?opRD zgxODPdt|>sH5Qj3LJwLtA|mbUSfMgL?fGzq5z`?i?zQ#!fvZ*CiR)XjlTG)V_&c&x zR3OiF>SG5-nOb`1uiGo0Uutvjw+%*lDije;ucjUCh2oksSblaU%D*LB+Zr0CA2MRJBtzr{*DOOxE`A9Px99axO| z`a-eB=H0hkk()UOMx|Vm;f2q1+pNQWBWvh!1wPBt5y|3hqs?{&VWduAXwgTLi zK6tvld{n>@Uwz3kZ*A>KktyGc)|?9_Q`00JZB}_tK9*HZ5o6b~riwh>OQPaB*mJGQ ziA(bCSt)DqL_fM>LH#78GussH)Eb^hzm~eg*k%>KzR6-?U@xb2%QH#p3f;L1Xs&3t zyHd)4>G-kZI?3m1vSs=6PEXg}jSsJAucl{nTddLa7C(GxreA&a@F<%?6&&UcY>?h> zW-U90by>Lib8D&9q(aPr^sVUn(yLIz8?FiCFxLhO1#ZGW+!UYn~7?`X^P(rGPafx+-kQ!cUZ)F z(AWwxtA&|vJGJ{>WpZoB1r_?W?W_2Hp|gieqaVkTc{@F-+Fx4cO?D{ZZ)LdMY)zDB7(Z@E?}emJ-;;=G>wICSJP45)=Az>)9vxeEI% zbA>7mQ?j)B6^5>{d=B}s{+@F@I8QInY9?AZo3`9ke?oSv`tWrB*K&2tNocBRxcOWk z$X#O!BokLFFs3Ey7D6$|)!X*PWL1+d)UO(h$2_a1>!9Y-+tOMbdw+_8r(0V!pnQbE zw!&>ZZfm2@i$l^W&`nWRd<8Y<;d=Llw=SHk5N_Z!qd~9N2VM|Wq$&AUYH1J5wsrUueKSYjiYOJrW;RhZoCZ3ErsjvS2F;A8_NnxW?mW$PE zNFK@L4p_!-uHRR?>-_xCRMCly_Se5|yo(nyuIOD|Gu(ca%X@pSzqob1U)*~*W#N_n zrAi4Bxv}%&rD|y}Ti56-^;w{=l_%%o?`j0vWqml%WYbB$HW^mMzl%F>5i#;o%8twr zD|P0w!(rbfq^(1}K5FP0EOr@6TebLfsBS7@yfs>m__h**5|8_}z7~2|7RQu;_-K2;O}T2y^W8{L?l3u(fAZ7bY{Hdi*(d3zRGCCT*vKnn|{H ziJ&@_qG%aQ8OvqdLGmh3<_9c7mpq;NAI=Efh0PycrCZlGW(i+1PZYljUX#P$&G9if z{MebZPVX7tz2F%hnR{AI6sT3_>iK%<@srQfcMm{7@5h?snz#@Q`ECMfu+*&WLKy>q z!+Je2lPpLREDqi(kkTT5A?d>I|4;;}HApJ_eO`I_^z6WEILv@sT)(lm*urF}#`iEC z9XE2;4>ZXuZ7eUiOJ`r@BJ+C&$fB0@vn+w%Wq8CxQf)lv7*|K5UEKff|-2GBBn zZ`tZmWuP+ph(+gzeL|B7PR{eK;jY#G7$meeReo%xD?sOcpkMuhD&p)t-nfSvoDG_; z+E^YXA6pt%%BdDFyxq(O{i73J6)Q?ac0UWm>;_9AmoQjEV3b?Rbq?v~tw8|Jg9OSx zZHQDt2?O)?lEH=-K|KW+kCgA`{Cy-J5`Bm}uO#KQX-{5&tt>{4p3vdm=_q$giQ_;!%ifv}4CT7VPx-=|-w8GS0kK3fZwD$Y zj=jru9K7u`w)P<*KRa9vNrc7r0~=f@6W}e^t(T+f2j|DywzwY zWIz6Br@m1k_%Zj-ExU!YI67eqJ4%NVZJeK}dNT=&<2a&M6M_!4wBr2_;DCX^UxsF{ z+XDLtR$(!a9@O7!>Z*~$gB2HY8eM+knocKG(}MKM9TbRZPMr{QUwtgk;@K{zMd_gr zai@o|TsQ?57^M9={MyKBIIHEzMsJEERG$J}+#P#d!sLFbYU{w3cr-rx(ji=E zAXqVUWV_*qz0Y>}DNNqS8@c7u&AEA`S&@14g9iKVaQ9cKL^$o6EK*<9bj5c)MW&%4 znOgY?9}*=b_)nd9n+J=gACl4z$g-jkjjClmc9RA|2KGGtxK3F0@P*2V1NtHF&LcME zXj#qdv+B25tplvg6j-U{-KyV)9_lu8ws1)ib?4 zah(E@=9GHu<=x@=QUW;z-CHaT5N%~*aJ#pstxSkLwNszml{_501)A;(*CH8N+50;? zI$7$OW0h-zOPQbfs@A0itox6{b}!K|Bc&>jpB}FkDO=!S@irlJN#EDCTRCjk{$$|Kg4s%oEswN?qA|FA*yQ(j5G$>^L0T?zy&r^ zOcGE8&8U8gXs=sR2Kv~a29@!L?tLHUz(F#`<4fN@o^yK$D_?tFJLjiejy>xmHq!nd zNv@%R`Bfbcm=)ixhvo9R+YC^gpF*zw|1x3|{N* zrxFtJr@gl2J%0?9J%avPKWt}=#%|LrF0vLD?WAmO4>uNIVJ+V`a~}~iXDK;JwrA#u zj$$o)&cd-EAsxMNyiU5)kKWF+=T;FBnOjJ92G0a@Gh{5rOrMkL#uUA@QX+xwmQ?qj z>L>wrCG&HABQEFNnbq#x`$2~5(Lcw~=D+E`d6DGFn*Rjv8X>&ouT@EOf)`>cwb9|J zi%Yy!fhYzp!q(Gn%9h5+=$IMH_t{E{u+{e3NlErwVkO{eGXd}4iWmqv=TbuKR-$S(+%g9|==7nk|a1!x7$Xq<2gJY#VcP#*#3OBuT z<;f`|S(jl{_jYLQ19pJpD_?<<@LieBsV*XB}EC(xe05#oqbOf#WQf6mtAyg6z zn}jIB1_Z`^0agN3K-M59WE-nh7aEFJ{Z`?Qq8)lckK9vp>&iH}A3=b+i)hn08^} z!72?rr0QG(*=U;`u4zL3HLjWsO%WRE4=ewErmE=C4KhLufmP5o1b74Cl zuXhNq1T2_Eh!IMg3-RsZ-<4UU3+RG4!SvD-@uJaw*nWb&f&E?<+)cBm zH5iIdU)##GfnBWbs^9`MI+>7wXC4wQdx9-Xi*HQ-Rt%07BBig479Aw+75h zU!K!!Ll!My7LV^1y>_8L3zbbQ^6g`^9D8Bf{bY%RC>Aki%G0;jpRl=!#ruS5YeE$4Jk$*JZ_n~c$LUVG+8$ymo=Y3cM<(ZqPQGT#(51q@>y4KG>0k; zCRQCSU}64_1=g#iO;JqjH-h7ekXZKH_Y&8~wB!Sb&iemjq$OVve~3LNe~*;mw*DZ< zV7l#a9he%_6VTWzL^nXqlEE}_KE7pJ0&7@Ww8@;F-+e={*J#R~ILxEgv9K z^o1qgMUYu3445c64mv(dmK)42JEV-)WT}%$A5)m zU)0flcnsbmR{BwS2bIvBBZj-J%0Zy8K3IFzr(jfyf8u-=+Y53mL039&<4Zg5j>}Kl z@Lj~QFO}}j+4;@UCnMiS@#=!cqY-zwKsDwW^ll0>G=g3*WHde6tXf9C&^y1524sL`m@wN#f|~R zH0z~IcPWO)iuL~G)l*{ibYbF=Jo6*-j;x>UA)tI}+?L*#x1&D* zy>Y`G9-~^d{IEu+XG1P971t@jYK0SKIVR`Y8?rfO0`bY@d~X$@kLQQ^B?%5Ck)+;b z!E|^Us$(k8&zh*COO8l&pI#4sq$Gm_Is&6xn*3BMi3rJ`c0X1@(AR?#WuGmOQTt*j z${fB&Alr_a=Rlt5?5($G#9MI4hh0h#b-fAR0@Ysv^X+>+DoKD2k^&(9{OLrXo zcQ%swrL-qJYZwt{#0sOr^_zHEg5Ztm;bi;0nRF|IZU1o(L$(V^8`KQfW(V%Vp?_wd zl6wx(G+3ve0d&4_=5BSn>E&a+a@gz|do3pRa+JC$JbE#G0nEUEXq35PkV)?>! ze)W4_e8MyU{Ec?vObyDYqlNHAnS;+=3F-iJThfx1Hy@+KIM$*%dX9Z%Qj7U^+$2Um zz;G$^^of033p9$Xady9&ePtu&jx{|S0K78_1e#rVg+fbUyy{=N$ zPoQ@AVtDZ^$P7WopDRD(fx~8s;`++$QXv#V00a6m{E@%e%Sm3mS|lnGY~(I*Q$Pz8 z+_Gw)m6twZIIt%&n?#ndUd!eRtPN8DW6ZdR(o%46Ur7c$kJi8$|u z3~z#|LM3*&_n$xsJ%*6a1820y6Ra@7p?mcbol@PpYSy5V^0L~I{#wZGfo${au$g`l zd(N)eCpnyrDE?6-I>2++8N{c8k#eG3XrM*)Q7!|z;#%8q}oQ*l0gjoBe zL61^JLj*&B-Cw&{=FPmv!hMJPpN%p~fLz*}VWeVvxxkt*gd>7H%ew(h5}b>E_uGTf z%>~4Gx-Nb5g;R?V)0RE_?Iy*g!^k-5Ec+tCqWt)DRIe$zToa?g)?INMR*1gp`j)Bf zBzlmZJP>CW(KJ7$gnt3s{IteOFLef^J-j{UR4GabsEl%xZ}0lW$G|$tclI}a+tCYq z^qB#-)ahj`9gYz3+F_&s_vC9exB82RqCdwX#q|^Na9&#v2xn?!NAp|ts?%Svo@uk2 z!AQP9NEY>IOc;&b!*8;zZqjH0^pt`}OZwMe>Cz%R>8Xm0-&*3|Q((?t6sZudvl1K{ zQf2GOA$XU)fvtN(=Qd{Vx5@Tkko@a(Ft(h(aAQMC3v==jm;PbzmAdbH{5tS~CGE+y zu*0Nb%VMx(H-c09k|*l+kYm|NAX4d@MyXu`B2LV2-hRnxrrT&nKV6YZr`#d)sywrV zm)_m6x5I5!Fy`aP*Waj&$DvUs!T#RS_FF)a1PorG~n zssdrI5Sz46SmF&rg4Bt0c@K#L0M;C8lu+p;n)}doWA>8T+v~fG0kW^74l#>r4|;7{ zrQU~wW}4@!f1TQh0x7u(K?lQYa$O3^8gH2D&eMP|6cEUo@f(*#o0PNXP??@_UcAgv zFq5EMhfm(c&$TZErG(yzn6DF3_eIq)j)T8GAO`>UE>4i3Yh}%;bXS_&r7)y}tP!yk zU|%K#)xyTqeD(fya=J4&5}d)$_5z6UgMShvBFl)l5uL#S82YdcA^#v{giY0~}& z>S!3yvcZltS`#uC&6nP!CN`HqmsH$`nXrj$W8OY7{SPJ{jCjujCwRrz@8Gs>*6DrQ z8G;vTV@>{kzdGi;#CsRS%Hjdw5V*D&-_?Q)NK|-v&zs$N|3&4lmh66?j@BdS%B2VZUG z9pNN}4N`!VxJUql-_ zR##%)@qVfmzvp&SaL@7HXJJFJo7+5X-^#Fj)TU)LA<(yi11E|RC@w0n^ zF3HM9a5h7U&AE)Tfigvh6d$h-`g@`kdW=x ztu)AbwPcKTgH&&UR8Lpa$pc$v;I77NL@T)<)t;DF;{U$)?_`7g#6B7R^Ke!iyf+~< zF_5C9HMRbdcE4h0V;XnzK!yyCZ#VF~@p-)f0lJI{u@9-#!cCy}&kDvA{~Eb80vAwZ zJTZIu^xeGD{;&Hy^UG~lmDJTA+lB2fp*y_d!5;=`Qjny*WCg1e+`eEFvjIDxC5zYT z?H3{T&^VV;A z;jt#w=LolFWxOA zAWDKj(48S#F1P!Vrx6GK{IlYE)nUvP7q#mUxBN?_S!-cPoV3SC{~n5uqz^$$ybwpa zQcOw9V7d|wouF+}@dsEEhe?FH&okKP(fT$3o`2e1w`!dT2D~Y7@UNR-Ucq0UuEs6% z_}4RG2+}e`kn@SDbYG*_qJ+8^uVxPF5%1_(wB(Qz3}`Tnz6scB3DhudBignCW%PdV z>UaHpU(tP8kqk(KzkJu<%7~Kk&9B-}L^8q~?$jpD(NUjMYrIr{@b4+1 zQ9hE{nz^^2C7>}D%z))b!X)1K!?*-~P&2bclQZkf*E~lQLin^o`$>>&f{$i@vw7b> zG@kjFc<)MvHeMv>l?WnAPVJ9#{jb!{n!LXSKoOBccEC$}Yxuo^bQ`HuZktf6sys6u z{MYTJFCo^2lrzQ;x#NGN&d28?K}({9jk-f9xT`KW^(bQyl1T~^ z0;#SzoB(3s;5`R{R`Ub!yRS!d@jrlkEq1A0|GF&>o)L^q;PSz^=Km7A`v;==3{E{0 zb}VcuH}Bx#hPJa}#=Z}5Fx*jtDvyF6bzm(eSv3)Rwuq&`C9uBe!#ZP?6!3`+WZ}-~ zS4aL$e&0_>qS?*(J=I?#$ZAOFtN+?b!T%s>PTm7XAyvponeGwpC`_Zfp*`J-piT`4 z;}Z7Twc|Hp{+E*MXY+{lcXOP7+zbGf=evEK3>T~~eftpWIb{@k)?^II)Qwf}$f%v| z&COlug0cmb%ePq5TPsr?a_3CmM@Ke#tWUbnmVFn z*qR`gB4O~P4(M*_^5C4-{eR00It?+54nvay{}v@2JcZ-eD%~7~T;`MP8X+eGSuLfA z`I&fP81TG-q_?u;+~9z)?a$z-2nqJOJ;XV$WjHxhTeuZz(r836wR8)@*d2k|9jOx_ zJwb%wdHZd&zej+n0>*t}G0sE(n+SkgiI-=3)S#MsF*O#J)duJJJU7_Q`@JTffyBypdRy z9@F1b&%^60wTVGYkAvh4)S(J3V^gzYr^N@qIij3u{SKsDWPzk{6zHx?U#KJKijNh6 za-#T;Rh_7l?oO`mgC6gmUdXrFxFip5Rx+pUGd{h4Z9bU8c?X9FOz>7J*v z$UE}ynX{r02RNMc+2mhR0rRs~Gwd(3JL@>5K7B{O=e&No!!FSCkvvZHfJNd4=hXyD zebuEV=14X^@}fPGcu=hW%t%c8QPlbt0>RJ$kl3S%9*g&p99B$y50?yBnR{;~@6!FH z?8!)GhQBE)`@huWX$@3M(qQVH&!K5psnTr)soL*`cV%uXS^PGGddq8)AHR1HQKlKC z5`K_{EHRF+-~l&>X`Zhy@{>PgdjHz_Qf#4R|ETLUl8NBN|7fyTcl~3<2r_^)LyQ;P z{_@~HKGf^qp4!L+>LNh%015R2%JGpmA*wmrXHtnR1Bj^r2A)zay8Us_Sxd5NA4o|_ z8T+1*=rq04TrdAN^<&(tTw`2Bd^ta5INRu~bi{N%zdQb(uES6*?UGThF{da(ktRX= zsJ|#8#RC-i$L>sj1h}{cj6=GV|G`CwZRp5A-tDuO=^0v(I4gjJCd`Va1hHdm$4+~! z+s2Bym-L;lY%k4!>9s*u>UaV%-FbKkY}%DSR=NygIIqN&%{F>GLE_&?-*3MW`fm&2 zhZr~8FosGbN&XL;fwJ+1z(pZbHC;x_r9emA?jB@$rY0B zSTb6_O|is)S(6{BlDHXdO0usl-AI)1KY@jCpS)SOjb*cUQM@r%X>wUsa#?=nW&r+t zmm(X2lyeb@v89^q%1=C@jjj2y$iQ!TywYHbL&$S$#!=)6j1DE_YtxzYC^M3)d1t%& zwSFH4BnoOG*3;T3K4gUF2lY<=TWg_b1?X0YoBH}M=CN^|Hv4k)=ST?Ot)?)hFG*7P zt`pCkJ<1#Bsy3B(w$}`h{VH#Lc=9>J7>Oa!C}-c{`aq;MG(byC&2tN5WHIQR*ylzK zx?DkUTRHJmk!j_y(`WDz5%(#%6jS9w`#X!!USLP-Z9crZ`A^=!L|Kt0EE#`Ut-rvH zv}zIvY#|hLM|)?>r%yqPLHJQ6>OM7`ixIvdyJfA*hVgD=KjC%O|1{+Q2Ta6f~-|zb-QE3;OM?sX+IQ0v}I88J6`kn!X-moJ<{ydQ?C>NesOc&%ipWG;}u~{+~^#xAgy*l)ExE|c^)d%RA@7_E2 z4@y#X-#MByk047DVkDdSujyh#0RO#yK(PJy`%h~?uw{O|7Gh_wLrMPpQVpj@zk@!v zbSN}5ayY^|x*yF(GYDv9HKg>lsovJ4=DTqM#CfGRl~KrE5o-MDH?|L|N_lpkWApJ5 zJCJwd73{>GeazhP@j_ON7}qw*<&)Q6K5Sxkzg%e2{Owh&2?3Ou>&or0fBe8cF?=fF z{ay>vNz#9-yN?(F)Ur-NEX6O}R$`?~ci#N*Ri?J4$)UZ3g`_RX)?zcC9IF!mm%FPo zI{7rPPFWb8?O!0wiY#sSrRlD+dP-cKg_8r-Tt)2Qa{xIyxZNUi6>WW|axSvQfgb|m z{Y4we8gWqcALpo-3I0V(kg(@!3dQ z|JZ7cFaC|zrvg^jIncUS58Rx~S8JLU&w=zp1LY@QE%}wMjue0FnJb|T7XX&1O)^-N;nc2;@lA-$Z zKp6X?Mg{+EBOwnh2_EJ!t36W6{D-N5svTh0izS(cn7qji;9m3maD6)16uXk_@a#-! zx%1}j4-vrbxi-KBDa_C%!99}>NrVGC>cf=)Uv<6V=~Sj_<-(S)7RSdE?vAJF2{lF=RvvteClqBB7#lMMXUWF9$3WM)?wiOqI72&s{9s{BX|n4kS%q z-gW`3;V&&O+nYbw#62i-Kgq zLPsZ7t^O9FiC5D1c>5L=Og^Lw?X#GPeI@k|`=CosnYrE3eZ3g+JqcY8E*pybyFWhJ z1mz`%R2r}{AG0#uG~=^pJ!Y^w0$pPJLIDp7l1bK-Y$+juX1Sr3!?s#Zf@o5 z4@~T4f~k1*=+zAkuSK2i9HDO>>3;a>T1>B`5&F(?yx7>-;O6Pu!|U&@^Lu)FPB{)_ zY4FN!3E%!eE4xK3?;SIrY%7L_DKRIxU5{*bX6jT>d!R~{w(^B(9)~>lG07$_ylOY{ z>GJ}gyNbOGB}x+*ff~l{lspMoF^(+m%QLSCc$A#mV*6tgL?j>l9-(ASD$)AOSecLb zA}-6cX9Usn(4L_jpG3+aaQvc?O7^+oLHD_I2oDmv4_LOpP+5jtu!tI|$v#bLIEb(e z-(Y>I8H5t=tM6~kjlc#x`t%loVN6t|-PQxIDOsE+KRrrGP1QHY50FX{AZSStQ_|7V zdHK}a$@I~Ys_4gMx6gj6@^n{JR(}5Y@ez7a7wqEZ4Xeh0a@K2IF!o%F6z1;1;Ft9> z8dK?#5-jA5Z1)K8SaEiLh7<@Cn_IQjt4G!XOA4e>45Vz3-6U_s!(;uljFLq4Eq5}W zbGlq}9Aov3W}@t{;vYHSH^Vc%N|m`3h}M+##Ybb64524}8A|Fr6Ps&uSK+L;5?xLA z6fE|kpT9r%(W5d?LygfmxX8~uJs#xtRb>iP8Jo?tF3QMYc!D(=3YbK$$+JBS4XroT z%PW*|<@F5?CYGJa3c3+6q(MI4!6)zKFG8!po+cP7T*L(E^X*ql(jX7}GERA&w=pQ(n^irra-y~vS>Xqigm~2|? zjA|TAgLYlqF8oR*5wGsYbAcx?R-8MdUs-w^YA7@ZX(;?bPkmkk^QKQNPIg5S z!Nog3LB9LyJ3-+C+l+P+aNH&L{qW)P%}4&{j38gk56-LjxJX^$xf6eg=VOfLzBb!I zWUO;k_bK|wku>64k-c0h($Y{dVF{x=eE3}xp-o^OslnA8_nGc9Jjc8Fyog_1+Zr1m z*SK%s95U)zOn;UnuUI*so16OxHMM58fC?Rfs4oyd0ZY6hdN0Z>@Xd5+z!TX!M#dOu zKn%mKTxR@;OaOi)M>!vsRLg9^QPg(mYKfQk$=_>;mi6X%RzBY43+4(pI_9$nZEWP7!RM)`$$3-Z&Yfpk!Od)rPUE7n2G_4&pMHc13{;CbrT@ZyqNO|$68>G~R{YRubqPmW`>I}) zFmqehGKX~YO}o_)GGBB_*g}fXZ>12%Gn+y2%3!xPFX|4X`sk&#^L33cqX;CE?0{n> zcDXcieWyX6t!P^&59|-N0-5??VZ;;pD<~}fz_A#;6iv^$%C*VD=2L2If_v%)zis(^ zb8=2-ux3-ePY`y@YB|_h;cBmOgjfGoda9+m!;ye{7}X$KB2prv)H-3;$>N?S8$}$p0qsj`Yt`*WO>?y5_){F_zd+t9d{p`z(ldh zmDS1-o%xRh;h?%X9=3#GHGG+j*f3H2=w0D-M}(7kcCw=g5zd}z|IfdLGXoLMHcf(2 zvrBkXR8+x~+ODpzzWivoPYDN@7+Ak$OXAz77mhDI`t7@S1$N;kTs`Hs#JSdjRaI5N ztkD4l)V`Le03LQ7>gHf$S$dykok2`zmZW>;li!zjPbk(+GhVr&h$ z?3yiFOw)^54f!*R*T4Ec(+l;}a`G^lfIK-4?n0gJ@ba-icijua_2d%j42lX0Y&%jf zEbuDyK4zJGyXWdsV6ujMn6kQceay5ZT+oIHegYE zhIbaUJZG~z)wJ~(TukJV${pM{$@`*0^&pdz0D8S;p&*e4(Ry0|&5cV?NVuC7%V1sR ztEi}Wr^W}@VfNL;mxpW@Hi%clA>n>^+aVeu5v39XJUX8YGsaK4f2^N1Gpgfl|o9E&F>c;C1-7o^9} zYPU};^>@+@kG(Gtx#*%zTX50F!=qw0aKOFTHB`lLMmcZt^>;)6W{3&3C|F*zs9&j;%Y@tr>XsZYhwi}e#LqCw~nvp;TyqU#h34u=?~ zUfd4y^m*fdIH(qP6Q>Jz7Qc*qNZe=B=c8xMB&v^CT6=BhY~Ua~B2E-!{pl7p4h|*Q zo;XP<>V3nHxCV4xudF|1=z|()yD$TL`u>HE_M_~jZLw{kH_823k}TE-+aC8+D@WH$5~ z$C~ox<;~~}Z*+mJSzK69@fs^gebOw#Q(vv7a3)*ewvj~$wT5u^hSYscAnsHUm@-F# zFOg4mB3;#sy+X-W(~l3MvV+;vX^9Kki8*fD9a+U+#a+eC!|x}ITlQo%_P+n5x{M-7 zf>T$XP#Tr(!tlYbrhQTgn@`U4jlxsD905pUv_~Dx2?lQKFHh1Cn^XFUX0fm z68I*1@5IhlEGb95f!J8SK9F4NAC~(+COi~omA7F2IQ^uwc1SS`1{->p=**q{Vcxlh zr^z;>!{!&jVd-BdZ;CtvdWEI6DUZK3UqnW}sEfqSyW<5E4#UEb1Xx^30(>SdEqz>2 zu-k~_z<~pgBO+3w-N&P}CO4+@|Ce!e6>wN~ShVP%7`VM4{QFIc8fM=U{v4J{!^E&X z@a|m2k;jFLxvlCURG95GyoJ1tP~kp8g$9j6m0}REOoH=4+dz)Kr-Yok zYkn7sKPXb#(>{%1OiG|3${kW{fAacVEM2twqirVHJ&(YxK-rG!>}H#(!6YbR&+#cIROWE$dLifvVq}oFH6;G<%A2< zd~&%z7Dx_p;JOrZbXQ$ng&eNDBv>bK#&6t^X=5S&e!hLtQc{I8Gk-HOI{`c-BjCafjM=DSG1^6MjA)< zFhnWhmaQf~7Z6gyg+C49Mf`oeAU_IYBRmyMNS68HMbNj_#H+B5Diw=8?Y8-&^cnp) zZ+?ZeQuK-Bv9VhanVi$Ho8Vpl>;Xdj|@ZyQ5W|S+XN45;SD6k7}!K=3J); zXDk;pm5DD!H*CC1>*}`YOu!Lb3HSg_eWxot? z?1Ha+Df1P$2*1I6h-_a%MZ-Xom`jf`3nf9S>U(Gs ze*2sR18f!RqTk{M7>1pZx>oMq`u#L?_Ib*i+6_ZKo<7qcRQ-gz%(P`Ol-K^g)pz!+ z&*3-wafr_*Vqe8qCwuG>i}cFE|Iu&2!-Zf~O073bjmywi(MG~okwj+MQ)tqje39^c zDE{9vg@d~f7_K}E{lJbX=BJ_h;>YfX0k4~I>){AZ+DU>MAWBzY1%vh*-?M~Wgn8t8 zEQ{vzQ`5%I!wQZGCj&T6KsZCjh#+hqU1VI;Gn4K0$hd|{FvGvc6^L-Wo;uCQvbfUT|=53UGASF+g&-MO~Kezejojk(G}mpU?v~Z^fU}SIZqF zVar)pIhXc=^b8~0FzWs^-fyZUO^Me?YrkRweF^&7`H2u=f%;Vt@~NLgmTe$na15vO z2l{;mnk&L5aS;~rD{z{yG=QR%h={``co1?vhw7Zjd8oUsUQ<<#P%}{h`25E(?HfOP zWX?hsDmENOY&nNcV8&U?Sf7U!hV*D(H*QV%IN3XK>(R1w{VW3e#7togVYtDc_wJ*_x}C+ z;{pPmnZYwSx&Rm-S3-GevO~+j5y=Jp8SC#0er2$=0xgdAX6*>#mxgEWr%HASJVVi) zYKOT3(_-cvL(smYKLQSGucfBSfl7AEz0{`V*(ZSu4k(tGA%(dh2C##wxj9dIMn)Ud z|0frkSe&!c!5_vz_P}hWyPzFX{uxtA(5v%%9QzlNqky0-NA*QAFt)Yd?D&`nrBqcCqnrw*>NP%&v=TTp1?nA|x^DIg{|riw#pe ztIUW@_kWL_8+s`89X`vxbX~fg?sE6;*=y#yvAVoh?|uu{m1A zGclR`U$%>%31OCRaS6)kB$+#Md7$aYK3YzSCb&x$f8-?AzQ7g}iejbyo#iVF2PvL4 z4o?t^LB#pB04?2bhOq;~2C8||)6=0c#tCL2y`na+aZ%t4_ zY=l0;pNygCBZRsJQLss48Kbp-#TxktYs1wvU|%zxBAkVRETyodcy zPL7E_)O==NE03=6s_{Bq6xO(1Cyj$xd%kM%)j|$SjUa!tW{Tu)U}cTqhrUm##^eaB zA><>w_dD2RhP*^~xxr&FRyVb}AbV&&P+qLYS9Nr zTlqhty?-r_5M1UhNsG+@FQZYmjINO{H6UYw#xE2tPx#sdZS2Axi#|BB?n5qOLBAGo z+1T3-cXHdG15=AJZ?S>Jvj_O8(hYzM~M_tqQWYioiZFlddi~2T5 zm|U87;@j__V+Hx7eDu1u_9f_lO$XG>8w#FHeUxb3y=TCOo(dz z{>g(}a3|IE(3dAjjM5#BowziL>bo@IZ?vCy!a=q3`RWa$c++$3hIbky{!cM`ce#EP z>=W|^iR|R~z(+#ojOKuoPv?KjK5Ux+T3L?DIYa%mWDP@GBHMD9dQ55fcglI+jvBJ% zyu<{69X&R-Ye(x5J36eLn$d{Ny|GA>mixC_;(?4XK8Hv`lPJU%S z!DA8`5pf4vX#Yphg^PZ{2P5gE(IGdF3Bb;vnpL%_5s&0MmbLh~lFb7XY7Bzs0b%M&p3my}2PGJy8e3 zBqafDX-4d6#jdSAgbbE@z*N+B z8Xw~M`8kY8==9PyYYKUL?}!WAe|sQ!vxikh&jYV`ekog#TSwqmTW6^d_t}c?9s!1G zth5T^qF>*>I8IDPHhi(o&}OHiv7%E`Iy&P1N!>@`;o-f}l^caVqTch86p+Hn{BeNb zpz7tzme1xVgml;5=oIiBes^_-gsjl}n9-tyjf6M!=?FW_`>b=Pe2F}!e&dF6#?>gb zb9PrYes1!{)f?>U-7P1r+$)}2o z?!3GH{{6S=*C&nUeFocKYnjH%%pCQN@84doe${+u_)GEQL<1RFLbA5ni}J_$`Jas3 zOg%2Z8SLgXcZueigr6Ll5=G+nq=bcid#GLb%R9qL7j9e_R4H~7`?ccyi!w1o=NDPo za>UYcPT`aroBRS*=0B)6?01OY_dLDo{)T?$jnbRg}6J%;}bns7@mpiCp)c$H@9TW+ioUCWgXBLeEZegXJ>8VY_VM0 zdg*wuQd8#l=XXK6ekvB>&xoz{rzYHT?uJE$S!TNk`){yfj5ro!oy_YG>;r8vQ8$x6 z@v!AwZoXZ~U1_y+kf%G5D?@X%TY}Z+ux?3mbZ*}@!)sXO59Z-rUu3;MbF;xwH=nR7 zkV(&5jQP1b<=*@g44=xSOF`?0TSr)rPq=F`p3o7D;lZq{+$&LFdSPFhpu7;}SXTK# zU*x#y)#wkZqh8$p+}DFRPpaNhDw#2$KC8rMz<)L9bgser%en3VVW-(F^)tM4ikqve zvW}y}tH{D7>0FhKAVQ~BmU9nw9JdL|$LcX%E1!CsMM_~|gx8jd`1a`H(zAqHSEd99Tu6!KUNS0k1uM13 ziKu!f=jlhZJLof>(5txD7+QVBDoehU?4{$6tLEjIjQj`KW}aVpdI!3k8oVVcplxLY zm#m4r{m79cpO7>!{=q{@5M7nShO^vf&J<)?l6;{2?>Hs|0W`89W;Q$`tIx3Uhs~*- zFr;dXMmnqP;}U~ZHhqN&#A_3M^(v`bL2r*wwWo^&$jfLnTlI{%y0OiCQaM83XyE6< zXY)h+=7rV=FPZ$jnMs-KP95M@&fnNBc2wh`qMFs}VpB&25E3c(I>iqcyS=%|!=2aL zXKo>r+sCQWHCw4~9zI~AbyD|*S-^3VPp5-}ZaGtIuUkYoIT}m~h8>;SI%<2* z;^K1TvnXYEC)2~;3)doR{JI5x^zwEcWpRIBahhzdqE#ayH;u8tz_r!sduPb?VzUX2 zuGC7;*K^pFS{!P~sFiR!^S0w6o@YN#Afq&RF^L;xRGdx4L?!hrPSE#;%VM7Sfc(>P zbVt|Ki>jkGHT@~y=_6E`(%XzEUBV|}d4h7U%)&q(xAg!=A3 zD>DD^FE&V?Cso&o`(Dun+rA}{0JP?}d&Cr$+7E5fMuc+I4_%EaJg4P&-nLSA`klMa<}QSaFnw}l|d9Sq48kGfZJ-aW^P}(gST1Eruo-o z)fEad>e|M8_rEeZ+$d}OdU4E#zR_bJ08SyfTt_JiD;Eae=bV_RY3JH9isgEmwl_~L zwoIZ=S-+R=mR0KfV&yDxRj20_apc-pw>oaN6CtL2<%?w3D;%DMrJtcd5AlgF3H_K) zRyN(py+7G1m{6{KAq`!!d`oCxp@x4%t4@msaWg0QpRCUEUhD4Tbsu#cHS2s}8r6!P ziIS=^rL-N76Lc+n!0h26{Bdhce%$SHP`KT3Os>MSd+Z0B z;#66xAu}&8U^zlVqt=zD-(hw+j{kmw>o>ap8RXA90St57Bo6smfYuDw>ZTCjuW4c* zd1N0ma*ZfP+>=~bQSm?#S`7*dRi74CN#_W;tP72? zpznxf442+q@JO!`6}mHdiI@lg)QK;VQffEeEk-OCf*<*=mN9n1EC`*-6NbI_Gb;M= zr>yai_PZXRyob=i4}uHE=-y|upVhe--`d)&(O`BqBp~}lO_Z-}nXi;Irn&b1s?-0Y z>%F7l{MxqvZz5upD2Wo$f+&eW^iCoX5uNCrjNZ%WL}GMO^xk_Jj6O<;lIVRHWt6Bh zh~9ZG_x+UL?|GkhEo(9VU}jx=@AEv)&v9(W8Kv|6W4+$U>z-~lwVf+Gunotigg%B! zk$P&Icrm#2&Z~2%nEpEnAtw1{7u4yc`CL*hUnSRbiRQbZ!Qb23WKK`@T8AO(Vv!k< zu>Bn%CTrt^LRHnXTIJ0b=}kwKt8`xCh^y_TY?4RIhwJCdb;?)|(#biWgMx$zKy^;o zUpnlQ=!~hPcCm|E0%5IM2A%(1=npRdC5eOcpY>tF+cFn_@syLE2kF-=5LiBv78GX2 z#V1DRLLKzAi8l{6h@Q~|>HsLc(zNd$r^hn1q^R<$oQn(N+jA&*nNI-Gpb>#E<7C*S z-^yItdcfS@eC_?~*tDf^wv>wTpg+Aeup7vvETY;R_Z?lAqs_g{$Eg%E7n7Fkaa20sz^1sgDVe~Uy&^ONE;Nu`=1EpEsJEKx zFSJ{G;`m5eD(s7oIN+4e5c_Tvgz~ zdFV~c1o-DQ{QM+uBy2&z^M4L=x%+<|GXLCZjsg~-ab1?>*mS%OxW<$^RD-%b#HW!J z7vqrXE<5!%jPo6}tfCx_T~3j739*EC^jQuT*J_vQoK_hjk<1(U#P?8H*kR}I%iZ^l zMO@zUuiwP-20mTSl5rAk^ZV`BLaq%8q2enE*IX;fC-#tCK4L(sY4i-LxS6}4gK};| zko$?6+^|;`#*Q&fmq>T4LOGgUY%aq^vSQM+CScOvY+86?i&#fgd=nKrm`U}iNvn@q z-8=cv11l}v+FMIW#bDthwb9M1U2hu5n7=by@IY$+jO>x{JKiS8+o)8oV*v*FSS-aF zY-)^@CZIB>XxJ6LNT;Mj4Y+JA>YH1jk1PThB|3$D7m1;vVE}KZS$T9x&r#jF81c3tVv+U$z`$De;l|%$7HEO?8QS z(BJ)(c=8%an~_v~ZD{%tn0xVeU$W`+sBry5@u9%oihbO&=6ZN2 zhmH=znq=HZF4Sw~*X2URV`@df(&8Ll%3XT%2jqKox;6$#=2W6LI#0{#YN`sb(5%0) zqydRMg_i}VUSP2s@PG+&bfUG0$7rIp2lv6h{f1!_bNrAk^T_xaKG-RF0RlG?h6 zDF>!FUwK327Ad~Tf+XIj!?~7ao~Ch%6jWr76&{*TnXwMB$}9@@(Lk*aYH?<_}?-aRcKwA{1qdjlK*i&owzIX~DM zZlrV2Z+k+`da@RP2B*lj*D)`PdefA4jl@tbsuIx#=?ez;sP zXf?BXuVg(#&wXoNkz!s^Wxk)|y7G{CB}q*Nn1GwJ{!0=Rxlx0a7e*Xv@92UgEV@Z* zl>k(iDpZ%c`)G4QaR3hgT<|zJ9=F|aEq68~kNKT`qu%#ZYmvK`=DdX~*MI%-~7w2x7(&AX*Eb9xIP2BNPjv zf`La$T^>4~9WR%@v?Yw?>tJ%-lu1E<3#BAhHSCr0KBGnFk_g9NkL$;Z-|GSa-8su# zLvZB-4wXE>_wvjcaOcT$=~UQ@s;v|LFWbnYr$D1kx6#qaZ{I>b8QO8%05p>997c7m zu%_YkZfCYNIfCe@G)xyXHQOnzMt0R!yscaxRv=nG1MACGWf;uWcZ zf88ej0n+{M-Mf19`1iJ|)V$Z~25j}9cNzG6%DzL#uXiovDP5||W($l6Y%x_2xM?nR zGOC*Szt6pT6UA6~uvH|*7^$5_OaAaJIM@SqSmjzw*z>r(>YEANMxstVi2P^o51z`F z^Lx+dm}0i&ac5;M-Kj!$Dy%~ujx7vZ1ZgzaJP#{S1!=1UTzTh$J99j9KI%E-UV=y1 zk>uSq1|BMbI!V$Wj+NjLz*&8xBRDewZ-1_bQEG<6l(NHCaS!VYjP9~TyV##Ru~wa! zcG0}wkr1Dt89Gm+ZZ^UkZ}Q>w!>u91x7-(g6U_wyz;axHoUX`4bQqg?TevfpAprj0Vp-0HS}&&v;NvrFUu6{I#n*5a7md@ z2qYkL`t8bpXul~%RqD*iNB`on7jIBXn7*AxVKzt>G^3U`#~VB_Fpw|okmhC+c!Lt< zCmL%k+RrwqyZC4cIRB@F_Fq!X%@3wG0};9>^NHUqfy3AzQ8O5ZpCAX*X13S$x^u|% zyi@;2SFV7p;qRr+-rje$mWF;T9y=2{%F7=q-{>>req^&gecMoy0@Ez)buiS`yh~R2 z^{X0Usi_%P=8i^M)u;4S$h||Ww7d{dZ$4^P6gh4I zuF0CRqk8nx#e;t0xA3RW(Z!`3CsnzdZ)3&X3!qyPB{n*wqzqsd=;nCHE%WO;5>Tjfsm95%!hz5}w%HyyC@|u{+5kM}7vGvdcQlw-Wux~N8zEF1enjkc* zeGAN@OF`y(53yJS&zymRC&7>mRn^p&yv=*r)ZANdFJVnkh0ztEmEcm@sl94V%J0u# zAA1jdmyyY)cy*2fEr0%^TquG$mo29Qnin}=>QWn3nIb3VvShpMW{h;!{u{CizDEHs zTI754<1&(S>nI*Dy!5TRemZ}!3}0RRfdu`>LX^9Avkh9vUsPF9KO8J zgcpx?yrIV*5>>mV)%_koZ*!_f7t@lQP?HS zkFt-4WexcIz1A2)lpW-Gc^JUJZe|J2678%{(Sq`f}&zEcG9w>5!s^+k5JS&m^6 z`8kZ!Z7e~gZimxfqd&hUFZkWNi1065Dw+3r_^13`Ry5T=|;wP z7QNSl8hU!P4EKGD-k0%x*@_}}t+!s}n0)fFgEf~^R+J49Tf&wEhu5`eRM8CmlHzJ2 zB_u=~JgKP4>F^VGJ*jlfSfi$_!lI(ps-F$`i#@?B@U=#2ZQk6)(t1D3==_dlVnnJZSYgZJs{3mGz@ zQo0v?G7w09?_vLW_mIqsuzHKmTIzh94v&d+>?WwKajGe#jv%F|qjP zVwDYczY!=QiBydZm`s>lu1R#Z5$#Kio27=xBRJF6rMIVL%ENQ!!`6-^hgD8I#F|i% z<5l;rNJ0%uFht2*4CtKzjMR1d-x%zF4Y+UJvVv7sC%@P|y0>xPcTjBY7w%DyvNGuW zQZ;a7LPRQWS?KzhY-85#?#ABK+`+GJ%eKa)2`-d0bom7@4~)}G`dpL zWagQyfpZS&Jk{o}Sj1VcEH_X{F1BN=3Q*CZ;NegCowBOzM=K+pzgf`TqLn`cbL}-6b0*4c4~F z4@(PfIs`(7L9H>r%JnaLg0@}}JVVo+nfp4!a*3F1u8hp8kAQON;hknmu}#Y7N$c0S z_Q>BSIWn!zu`es4L}VfO4uSfbiUUMw*VBuQ+psBwnnzF3`&7>TV2W^aq#TbeQfLW? z@2FCqz@*z!Aqj*iI;tORSCC+(zegW$&MT~G_pjWSNS6nstq&9>Ts^a7GFL&4ZO8bv zMuCAcXWRRz^(EqTm?3{H6L|^C`)YAUvn4Wq`ck# zth@Y6r+r|4@+Uyb+B9N0v?&_ppVP4YK+v(7fQ%vEDKM?n`PZBd=lsDRRR?`GPShpU za!*8Cpg|I8v`lG|gcCluUdB7I#`CVUXPpgbE(W~(t(_s+d8=M zIp$)a=ttjVpZfcLbt3Mw3Z1C!Su1(av6pgj(kF?o^yXOEk$EhwNs^@UiN`OVO%oZ*(3D6viK~{ZBEokfr%}Hu_#NuKk$k@Vv?LC7Y;| zdh5>N4|5(nN~!mZR_(nhv>1(xpx80X`$A;6tOD%v0%Axy_r_u8-~;f75e-@c|AyWF zuZem3%^+z|&ByW81~|B!h*|bdpSi9RDH3X*jR(UQAF;)JxTJYiF__sE4o-<>3aNfP z(GP(RM6gAyf1RoL{N%{MUjSkx%A-m|{Qmm}juz``U zAyNnPA^jB>asQsH+jxk@^8F%%A!%DQ?jyc)sSa3M#U=2Y`p7sE^AFp=r};xW?V<_Y zvRYJEY3wDa69!#d1VX2XISrVF2?QcWss&0(-l2(92;)3L|T(>@CaP6wQ zNbS}BC^HCSGQm(coucg`88VKy2DE`#`@YAgwqsyz2TWC@IDz+Ia*~mqH|9AVspS@8 zxGr{zV?4RzZ|E= z$Zx9x7{38wsOeQsoLtL9=!GP+k18T;zVQiC7V$9?zNX}I{oz&v^uq8>YD{>f&ZHLC z_!>VyZmLPaB`AJ*+0Vh>p~++BNj|q8Ds|~nzRcW$x*cFKl}jTVf#*)}%?*Cgt+ju} zhkO=1aeonYTT$J9C?lrJ0Nu3LP)uk>!)qjO!ISZ%5Z;Lh^irjewGn_LPu0(M$O4P> z;B6=W%z3W87UCw0+7A1o+!CJ=xThi~dNf{holDCUQ0fb7DdSkab?G~d<&}EN3i~w> z|Ks+Qd}csu>&1@!%CWt#hkqBI1Iom|%B$;gYrHPnZ+)chfC@FI+IHbcB{ZE-xhm;* zJ#erj;AtSxscwBiJ|~&!yF}`9!7OEE1hc&l9ye(AV~`_MH+Fw7DkuLYEL3d(;}@Ip z?bVzndr>(oFKxj-$A?nxik z2M`q*t_E-=48P824a;tpvfKCwtI>6Jyjm?g_x|OE#k!uS za$z;0#3GoWOW%l4vC|;Ao<$_n^%H8fHKLcc9zn3OAxUXDkQT4=uWvIW6|_U&)_ND0 zhXS>aWFqMXWM6!gnG3*b9rU^#hYuC}`PugxT$b*KLexMng%P&z7$dDQevp#lzed{p z#>cMZx@!KIT==Mb=G7DFHgUE{tU2}Fw!I5~_n)w9{6ryreP)%ARZ{r*Cze)j74 zs;}Z}0-d|Nvi)>|%4MigJ{H;3`8%gJE}^k!Mf~d4YNleDZcBJZ;3=N|<2(82`jBb5 zS2^SlWxR%z1y?2mwI!S?oY;dMQ0(6_d{^*%mgk13%&PQc!0%Um|9n24U3WH_d;{up z3w&+J!G*FHrVB}-p`um!BIztJCwBU1*SS$lzP2)zbe1`Ipuhjg7Tyb0nM*G)zZ}*vfaptm3ke$;~-Qgjd)U_Wv#vy_=hu+i6>&c-) z0#g~v{z5;9tG-XRnrF`UH_!N#?jWZZrRy{GI@+z-I=$m22kMUM1}+JK7Bugt@tDqA z$X`d>5-%uj>+cV&^^ivSR`XL90#`SQ(dibU5T1e2pOK>Sp z)?&I195LQHpCM<17aF5Blq#hEBrRYJsxNb3^^Jvi+tZmrO!I$0-pjd z2qK8W7=j_Dp6>)k1c^Bi_%gJ*pgx{&V$*v3eeB!wrSMJqM$3Hiw|4Z9=yUT4*LHP~ zSvnzgOr0aY`LY^2F>`5f<6un}WNLS8bG~#f*TG_jn)!p&hf~*R{Gy*jw2B70k9p`{ z58RFf5!|E8592+($e5#zc>dm?&aIwN_Md6}SV&u-Vm3P6idi(0zSc1(Wl)*Sn-M(x zfug86K$-Yia>6dH-273W!C(z#4$6!9&RkNHK6meJ2(>2mr{3aSt~(f65Z7mjy@END z)5M(SA8$m%XXkg8d{RSJo+0FUJ2Q6L{p>@o{(oD>=i6RR9xmF&%Uuroi9L}p?^$PD zY6>U6?YVDilerVGw}BjI0S7~>cw|g}d~&tA9a(s)KT0&Za^@;QKt?Gx7FEqVH8*v? zY+nRE+b}o-GJB;TmS10gv?}n1=iigaxtk+#TQ?`aiB*k`xeQ7PFvx{BdS1uWiC5`) zYh4u3qd({fc#ic=am5HSM3gyP=jPR&5zN`L245a?>x^LF4WH{A>#r_LvBO^%+7hMx zj-NK2hV+qG*~j9VPA8*yZP29x#SMS*9BxU7fl1O%=f~W)v}lqOipt5Mol|ty6Z+V- z+hDLk%U`a|2}G<-phx`1+k?U_d9IQV%(xCQ#QK@SmuE{F4vm{JdZ6=>YbnjG&y^J* zGra?et4*X$I|~*sefC;$X#N%1+%a!zwjp$&Gk2myQCLga z3X}lRoW1f4fb|BU%(tW|msVmbZ6vx%i-TDKQgnWE%!J4cQnAk$`8MCKsI2R@Qsh^q zRo$i9(gZY!F`u=d?nxx?G)(<2^+bW)os6_0yKeZKHJv&g+cYi+L(H_xxYE~B?w`w; zX0!4=eW-&p6C&$dPtWa|rKRNpbESY<@kq0uEG*Fz_=#O5vBk+h64oB4Q|j6Ou88LU zo2Jce0v^=q1)DEEYZIO~Hl+X~6?=cgAi;m)m<9X^s6g^cwBnvy*&q@K`XPe2KwGqT zI|cXMmIN3Cst2d>XU+^zI#yb`t@7hE#C5}7=%w|vkF=V`0Ob8UJ%sSeVIO`Jef&Gw z;a^L(&$EG_e7Do80~{=J3x+My8O|TT4{s?Vl(jIrYGH z?DK!H>2P29Aow!N0Au`0stC=oF|+ZSm>W~KOXVvb5}xQ|9PO$9Ql7 zs9iBawaD%#M*~@_r`^6d$A4x4q!mlP!sXbL_(=m}Xh@pxF>W&bi0b&$YKON1*X$&< z>8`7vESE#VfQ#`qW4f~;^#YcelQZmgr|R_Y*d;0-wx-J;D_-`%fJxiX6(VKPj{=(@ z`72FDIV3b&LO!&*Ckk2ofatEvI9HNmBBg7Cc7|~4eeENMzhwQWS$!=yMfEHCGf_|8 zg_$mbu+gPaVgN}h$bM7(6inEPTar^Dkt8tIg5I^#1B_JtdE> z!{f1?rTFdcABV_@CWWPmjTz_%v`4((A?$oT2Z$OD`%fUY!aQr4NC zhDI3(z$law^RD4-`*k3|w$aRmePoeKo`9b>og?M2jus0CG{2kygV9b8$4mS735)(> z=b_iRgX!pAu+dG)CqswCrJO6R4PNscY>p`OOGOBWUm{SQ>SD&udKW!C6IU}?bf$Q0 zz=0y=b5Nx6!{>O;a}2=dW?!(%i~Rre*8db?2X8n=+IP{-O5IoY`S2niFc6!)!t;kU zbmd&`b|=BCaR*QF00SZw?qW_&c{P$R}Wp`Upbtuel8JO zy_{7wJLsUDO_jJpX15aBAZZ*h;ayaOGW6T+kyQ_9Is5i1v{UrxS3!Id41&u+=fE;I zE_Chd0a;egtFZi6$J>A{apL_GMGmo64Q3UKM9uQIiP$Q&uX2U6nNl%!Fl_N;^fwq3 zTvnr#v0rbml)v0l2LFtGcBj2+mvGbTPlM>H6rdVZlgR`IJ`(+qSK$wJ85eJU)7ANm zG!_l5SvGsgEMedK!(46E37naVkf%WZc5Q9te$Qbjv-|A$R}tX$*Vp&?9z#@RZ77@5 zp;NbvYhBnAT5OOB%p7En?V;tI^2Ciscd$;uP(Ce)=i(-!AFCU5=yb!T5vtWOh4eB# zV3v4UU7kN&M+KBSdjUl7`|l8Hv2uk*up~ZqVPg~{(rZt5KsL$QF(xKlefGuYPoLzZ zTG`8%Co57*PyHC|4eRy&g0xDY16v$K6pjYW0hY`S_L=Zkt@FC?2T-4d52(*sXFmk~ z+NHz|<@O7AJhxvT|KkDdL721a6R{%2Pk8$t$g=Z{C*zwd8x z(PWLX?HtFo|HKz)6`8Iei!=}+v@&pD;OS#AZ{?{uPK}5Va6cKoRpsfq8QhMWwB7jG z#N&EJzqZrI2Azv%rh+GPkW_0^u-nlJs*kk~WroN$@H zwGc`T&e5&-d8`cGiC=3Gs}t=YT3DK=quP`oVW!Ub&S3tp^EN2thSvYN_r__EqKJ&S z442H5mGa!|6H5dGkmu7s*`=%iO6BO(@((pskdI;1p(SA)PSxhQkX7#coI+%F8T@!> zym;)Lz{{Xp2UFt)8=HZKMWTayJG19|L^dO%wFj%dCc~@c8`TnQtym0d)BrA>*Ehz# zO+FZMefoBX^A~(LV|R7gy}e;^X`ifA)7)&4`D`7LxjvmZj|#umU2k;hIv#H^@O2n# zS`Jk5`xtt8i;%TC~(AD+-dx=?HGHz zZeF*Ie2$F6Pkr@#Xu6ltiO}Kk9N)Jl4;xirl(R8aWzKA^lCRo!rc4Fut2;Z`%xQr2 z*6=}rE@!Yc(a4u+AidtOd~n&z=`8nP$AFtf?(p?Q`$#SaXVcbOqocl|IYrH#7KKC< zd3eo%l%Qe5KX!1o?c?6Q?h#_udAW4)I*P!ZUChfTjNtODab=11Bj1dA7X{1EC0=PC z^`}-xGFvIT^HToky}xeOZ!7to<`b*<5(oxW(>Sw}Gq2u$k-n@uNu4`dgLdgSn&d5!;_G#b+1dRSh9J(vJaN`52dvCYR1O#)*wI%9ewq}Nx z$&_1K_YdpI z8T<3{6^wis6@mfi1`pY(*5BaZd$k*>CKfyl1hpJ9o|`P4-WS=(UqZA@#Qb_m1@|0c z#W`7o9Wq?On~0Bz>Gx}yrL*%{DlJ~7v>aOXbm7XB_?${3LBuN!yh0DV>)(XE=Msdo zSuNUUo$QS|k{`MqTXHQ4>V?3<8N@wnpWNHZB=2bFrbGSCi71M}4?+LvP+|;y*zkSe zc60mB!EDO`2az+IU3?~q+TH~$R`D|>4XUI%`QnG41FtYt3d`@3bEGJ9KulakA)cHHezfkrE$I9T-V;(nKck zTAUajXUI}=cK&=f+@Co%Q*VDiCO;IsCV0K(-_Fydg?aGQN|)p}DZH0)EDqxGC#HNh zX5#d{n?CS)KOz3ULWZQalQGpHdemL0oj{PX-;E7vtV~Lusi#S@3t9Lnt4w>MZ|P|R z0jC_DHbWS+!#yg+D_zfkTV)?)iqkaNC=TC+Obc%^4; zmVRK-`9^<)idTP3gP)8b@%jGIMsQ_ylq1Ftow%l#HLyGu;jZ^%uGFa-Y#v>@N!M6) zZZE627-l!xT3ao;_h~jun!1Gg*nRIJCsTwV<#jn`%$F;dI= zdS1#gGuLI_Dl@K!g?wpzDGcv?b@wRSN!mXit<4HHss_hbw472rb|C(S};- zh%wJI=kf@FqTDT>7Tpxs$?4k}<})+fJ)mLAdm4LfQ$I5ryW)--Ph5}X4H${nd+uAn z0vd;Cv-M9#$z@%iyWBm*8XPtzW-nd)pO`eAwB*XBs-Q--9W5V2nn} zJnYJeCNhCe1}>J@xt%qJzFo~Q9a3&j8w@Jy#pl^NA10<>@jx%B=eg zt0bU&nLv3`#cFkIeCf`jt7nXqHq?pO_ma#uGaE0&yhz=xBwjtM<#Hzt>zAeQr|r&l zFo|%^f2$SqrES*K49?;xRNMwV@KI)cQ4;gUhkg zo)^vFF=xoJ0IS5H_huO-d|XYXTRPE*$U>JEQo+wn@4F^D>=MqZP(eqk+*-+Gkt**zst~=f|A%%$H5nUbp+}$r{?K9 zy-OrA(fS+DBPJcahu&o%M=5kyY`rtOVnW*NH3`x1{?Pv0U&+gRCo7#lmL{1#{ z=TJ+}JH$1(^81`78<}I|=N3VUx;L-iJaeqNp_ODSMPi1G&kl-R&DTt>ZojG?C%B5J&U>YJOv=}ihNIw%BN^OdiC-JqI|Ds@enCq9d{ZZ$Hx_Jhk3 zT8{qrq>keszh1=U*wULkm8htw_q)Zj_S656N%S?y5zoH2?~Fr*UXwnl7Ta^E=arE( zg|?K|&zNPB0D@S>J+awg<8QsSmv<#L83eCIN6HJgj2OzivBtSCmd72p@M^7X1rIMwA_E@CxQ<;??2Pdvvop!d8#=OUDTS za>g^p2s9l%{BSVSnT}ic1J&+c>Dn{}Kfi{g;o9TMheyqAm=dnT(|#NGIhpHZ+S=70EZTV#pX!Ik*zP= zMl)PIUARs=5=*<)w->&9A82d0psmpcTU3cLxjWK<15+>_|Kst*jce%>c#0=hIgu+z zPAGTt2j}Az+{b?c(T!IqTi8bFq+>QkhXU&1y#M8htz4&JD`q%wo!+A@?eM-4>uf=E zd%RtT=$A9{jz`Cl@RQlcRy~<p-6Y*FD6|A(;u2aEV3%j{;0 zt8AVd1$6lGO|d5c^>W*9mOmzBGY;mD*3sK9{cW*a8tBhJoTEb6P9HgPQm7PGO{o=U zHjHEZrSknqb~A-4VnID)ht)WL&@rL=Yn>Zi6>A}h#z7Yx;Yk@mL>#9R-scMbhwk}@ z2Bi`g_4a_?-l_b8(~okMU}Jao%C8CdJ{obqJ#_!+XAwpGU|EQW7~3ocg3e=ePoE!q za3zk=GRM6CbTaAyi2vfDTGI=Y%GlzQ;6tIWc%f00@K1=#>Y%PLK;io~Q3S!(Eop5q zOw>qDv34eDyy*45c;rJZp8{*1Z4p*!o`@HzTLk)HAFMVIdEv|&pKTJcVBMovUse+* zT+R8N%2%sBUTF21fpFOaIGQVA0(Mf3VCi@-P(lju-eqoQe{fgk(RgW7i0(Skf$g+r zZH*dx?QZZFczn_NLtEC_aE2}3ip=D?SW-%=B4x3rkcm%P1b`=QJX~V>rN@G)OUqLt zdAkkej{Mr1gS-?oCrmNEAb}<788W7teDb00*WM*VhqrGjArjLG6A6pU6MZyXORC93 zc0J~fmik7&03Ohn^P|y6gL>srtwrtzJ`bw+#Bg6LEz`nVmAQG#%A{OZ_!UZFNFSP_4Z1usxW74OE4Z zLE7B=m!?5@V4%rC4;WdI0kj3x`wN9klJKp;Z5IICqllD;ETAB&XDBNE;!ep>m6pSt z37U*1P4DQY3R5A%+|O2how~issaE>AfZpl&RVH@<71VY*PZRg87F9#$n+*O7(HZAp zOaU^2>N4?Yv5hQe0uj%q!=c!e%+Ak=+))GQJg3&nr)RiQ%=kqqy!mVE*$JCaxl|>P z9H)LXaqSZpaODuB-wZpETZ@Coidaa90p|I-|zVy zZqavzYZeUANhPgbT5^R561{D_*ov*lIWCZ4FHPz8bGXw@LP>UQAOqhVEoN{qI7htl z99Qs^R%8miLZG+ZRA*m)kr7l-?!)`9@(DFiqhqC^;@EnUwoPK>I%0jc(&qGgQ9Q~C zc`kOgu8B>-c+Gj8om{RdVQFEN6*y(;9mHD$x_(2?bgGWM&ooj5h9&)ab-&5pk}#QH z?*nI##XJ|DywR(RAHgI$%~zl~zEj&|cRMhw(#gLmdH&2EA)Wx-GZ*RW8T`KmHUNIU z7t1$!6w8JpiEq-`=*?hn!tZTx>V}!>kzfaySsg-`Kz$~pyP-}VR0so0vc;)Ku<|N?6(KTb`U*5uC)QUpA~_&#Z(htd_<&b(aKayENTi2nAvqi1A4E-w9|M#r=M4M zPW!j}BRh8giOhjb>BW@7GMVy89$5;#!f<7V|JMs!QZ?WqAQs$|W(|EVmvv>~?S-n7gDv{AaAb1lX^@JuYjGQZNbEGG#rB?vUCJZ*9=kilq6nS2^i2R%i z52>xW!p8E0^ck41=QnX)=y0@Z(G~GqtM1{4y_7`EIP;sNI}Nj_|0tJ-KM)>ljivK2 zGAbperM@PtddbKeTwV&hQ5vk#W1iCv6~eMpJ8fR+JekiY9cR!F0^62UQY4%jQ@T__ zab89D5Yp%2>=+1DKv&nBryV~ROwTh*t+D&N1p_v5{2JHjvx_MFC1r+@?wt1q%w_gp zue)`d-5LBZFPQ|L7N7ObS@}0;BvjGt)0+fr8=H?9c(hZEWU9aS?s3o=lYmQU80~M0 zEthWaw*I9*JDjPX*hk|DFxzKl7_B!MFfj*AAV~rOxWnd@)BWd8IZQ5O>*sf)$|^d+ zL=YG2S0T{3>jJkPh?ODqmj|0+z?T037AI#LKjU~xFyOWCasy@92}AZ|!sj#@@N;$R z{gK*njUF+%wg;$B@$QQd&2d*F2@U}EYAdqwHl_UPDvcbd#+Ky{xAx%I=(usux?kQ& z5n9%hpIEQ5n$OmmKhfnDIbG!npG*1gI-++sy|^eZ&Ht40@jR$Y*3ar^B;5bCs`Pb+ zei~DVsfZy0PH`^DNQ1Cn4LT)T?*; zv`|T$s!h_LTIvdiwAIsLtXAWi7z?88sF+fFfs*>0i0&)XL#%%y=283KLb!mDp>^5~ zDw~E~9W4SBT08*A0oHvGeikrjp4^~hWyvjyB_|NSudk4ujBECk0X&)C6of2M`T@}m zBsc6*ZKxZ|1V3}a%#+t{{FNYQz|YHH+`Z?4OBu$MO8Xc$7J(6n}C}j zRjGVsRw9N#@=#DZd%6oT_pc#M6RkmyX`9h?;osYh8=sO%EFpiyvYm6^GrvvPu94Kv z<@`41m|vNItwLCv@EQ9u2Qo=?Y-_8!rS@lw-zw18r58o=HpFnzPpb>@ipXz06K5Z+ zuz>6Ta(2f3GDA1|N2c_}v`d{I=#1@|CUUyHkBes(6pZpA3SDd>DO63@NHd@2J_6Y4 z&ZT$q%wS1b;D*(timvJ2cluoP&{IVvTHPP^LuNV&eBCo5PFW8gi7aimoNacZVdwj> zS2h3#@SIZ@ln$*g+d)B3euI( z(fQ@>fVUYg&*TKHm|i0Gfo-ze07 zDQ~kanLP}+S4vVY0=QrctX8!Ma_vSVz_~8BeagRkUfkR$-q~ zZ_UQ7+nkU9pM4?!Men+TKOuhOe5WbMI2GEagzl^eyNb$rr*kKbssAjaU{*wV>-vK4 z=lYjStDACL9fY(UNj#Xm_E*4MJ=Zaa3b?*9YFc-yBm00QvbhgVvRteYo-%mlklZ9` znb=oxd4L)`gJ3k;TF*iy84kske>!bc5R{vDkL2DR{WLv!{5P%N-f-2h1%)QZ$ z7Yi0+RPyF}3jjtta)FVFQ$X@XKGq|7iif;g=x}DlJ~0LK6Aygrh5&!T{c-*y(z`Lu zA;R|a#TfxdyH5`~f5uMKwBY4FK1iVBEhNME_ZBp8p@Ll)51$6t(iT0S#m6VusItP0 zH1e5vPm9XEaMYgT3>I>87b@l02apBjl&e82ew)obY#D%9A)77v--|fvbw8woO}`>I z%wE}-5utWd)F(1i?0aUtX~$)=@RxAu<@J6sVAOJ?B@@U0A4I^vUKSbLL}YLeojFD0 z8?m#zuC<8WM-q3@6lk5cUkw0`b|ixtVpI=gkjGNIaZ7?W1SY4}OrD1?wsG-$Nweig z1cTYYO|}NtD?f9^cbleMKCx@HyTyC1yF_~vw$zbbQL?&F2}3SJaBa5YaYkoFWkv8i z=7&ZcX}YyR__?=k)opWwc}j-?&ck^b17FxOWK3@F4e66;kG)$6Ir(D*Cr2TG&$?daK>w+90|IbUSgW zMfUaVd~MxVfQsY(J`Aj9?foWd0{pib3EhFXISFoaw|cVI?r^4%Kga!5+lJpYCWs^o zPY;1C*u@`5ku=IaN#~u7g{XBe=T#a_E(#4tr09WWRKifvz`8UDPOTCaYLf#hn|iHKwAzPrwIbmB1fWVS3fm1;RIeZH-3h} z!5Z;1%sb_PhNH*HyX~-K(Y7roJ_VJqzw)gVyihGm&#X}=wfA&>gedWN^zfMLp zp+`Iq;Id>BD`M|1c2TZPC%qCA<~Z_{RAE)1IA2Z~INgax8P1U)Pm$Oq@Q2w8eLEUjM7^v)Xk$QI@n0de(1T zRxs!N7*^>@@*mc0{{L@s=YI$EWPJl7>*~JoW(oYn80oS^RBc95icFaXW|qW#&$s;! zep+f-TvZ@2=J^y}ov^&7lf+SH&mCJrt!Ak!s6X(l2Nmk)=NFGkCLlDULXxk5jvs!$tL3YrQ|(#;S_KoyACH1Hi$ zGIzeeqnksHLZu%c=Env3&5wT4I|o2PP6+`cXm)dN8iR|Nvo};@4o<;iyY^v{F>ytl zzmnMI_*qqj-f2np)2`EI=tbIZ(zteW6X)lXRs?T0Ec@2jAIIvb(F|jk09v3&T-Y;tT?=nH`-(;#grZY3`RzVa@Q(6EhehH09Pp=T zk`Qw33Ws~Qp8;no%9i z3pC0eC_!03Yg>+1<;SHve#)-@ZC$ZNPvb!ewuRRDjdpXrxj5e&)}w*e9B(sb_g8NGC#ceWY{>^u8db6R3>l4NJtbdq@D1ey6 z$=z-*&%4BC<)i@@M8}RxTC3orlATjWC8OLhwH5O{u2sFi*0{tjEg%3PyPSN24YZq9 zm8O@%U{wn&yN-Jg`DI090vw(pWdP5wU-hqXEmDa($ITXX?yfmy1^O3P z9b%KZgWztsRuS%0-AoRzF_CgXI4qc^qY&!#~dLB9V0YteS*K(d;hBCLdQDW>V`y$McK9=cXX#1b;X6#D zGDolj@0}n%7s82(&);4dbDNsr^eHK4+f1M>ATnWU>X?4NF!zT&8_IHm?1y|w*dwBd08El^gM@Ntp z;)yAyF=hzJPMsX(ZY*Ty+NR3*U$QGVoEQpi;o5(ywkp8eE|c{yMimEhnc93_0cVpZ z$P9wvP>CA`99>FoYa3nhn(WuHfLN`rrxXn+{c|;YzMFHU2(-2b!S&F*_djpcYs^k^ zg9RTSkn4lUeF!#2-{VYoWXyAQpMK(`rj%2oq`XBZjMG)4BsPCVerqFG<>T;o^2f~X zUjyeG`{!}<#sh4PPcVgMGvl6?B8kYgdBVFOIx1S`$6KI{y`s(AF%80^X2GUNo!Qmr~!e>B>sWl|KnS5&TW#8kfowKjmp1m z#GH%?ZQ1odRyqfB_ueYn%evczM(hVFU@uQ9zJXxF&;5LOKk)&62RL)i{_nlk+H1YjG}j=Z zRF#bLtAL0J+09-)%0_L;tzpf=W*l z?qj%A`Q3sKaj-Y<8_i4b6KN3xzm23Aa!Jz8uFXsLwvN7lI2kt@CTDK!K@}H&AaNQ# z;bzeN!2zMZq2C>d-)&VIs|fa&mzy)?pGbA`c)cDk7XVqDrAdBI<4;8=Cq_7Md+ywz zQ$zYGxHxFH)r)sUG?t&1Hxoj7P7m6*B^h8P8vA_=(-S1gx6_#o<|k!+GkLa#zm)Gb z`Wl4bz?e~EMtLuz>s(jS?=$PueW-2%!HJLDyg`y9-7vl4Z)1FF?|@9D%<5Q0DcjLN zyQ)vcxTqBSmig~%W>H(YiGe-L%fjwb;k=aqIa@ApD{-K7&nKFE8b%C^C`!XfvU(FV zO|W{h!4m^$rGewRmJkz^8CvJA!8c;$dK!G`J6JukWgFi86_e7 zSlDYHt@bWfcx!Nd*Q>2_WZPCCrCr;$K3>>$n=kst_fxT}&*H|=$})C}3aiPL!*cIt z*=WjoU%7k1@ogxX9sH%qh-b7B7B+@f;w+`63`sJ5)oku)gD}=%4NGhK-E}FO^7&AqBRY9ET-(wTgp&#(kh5-1A>Y}LvFE)YO099CO5teD@rT2$ViywC37M*T^}1D!;p~U>Li4Pki%7ZKFKN1W5x0z zlb)JU?C3YhVl(RDu`rUwEKRJr=(Cqy@KgMmGms(6?X^a8;(dY;Q;(F8f_6E8JdHcf zhk-*A$GT;#;!y_PZ;3A`1vln7KZWJfCFpu(`TK3hM)u$cF~SRFPyJUIPn3}fj8*U% zJhjY_3(+>Y8U$pn$u@k53TclB>+SANSCFv`J+1((%XpV~H**A;aq*Mig_2Mh5|Uv+ z$2%{}`K8Dso6L{LpU_cg>>gdGt<=IHXF$0yBe~K9_jFxPvic0agsO*%e**dDVZbJT z+A!SrPH~i*w7K(pt#P*Wk&*R4xxk=QUQz&xI^ zumoYQKH*mpAi(0SCiI$!kziK+_N9JHX0FKaV|CDyx)%{f-f$!_!i5~{radx6N#F8V zotHV!`HEjZ|F|g#l)Q~4rP<&EtXILDk>rfJ3L z*vcVp{&x`EQBowJq2nTQdr&rF?lgIOpC+>+RsG={-T!tsJmcgbCK|)Q_{vvYt-ykA zy5i6@n*CPj@G@FtsLw^v=tJ8Afz%mGN>?^wg~AyN1p~Wk&`n<{3vV~Jr}Dh+@Y&T~ zph$=}=-ai+lIdu+q3%k2UBbltg+CP;-;kr*`sbu?;+ri00ju2Jgj5JA0V{k}1 zt}%Z&M1M|+3|hMgS)}te$!TBRcDjTZ?_5AypX&zWO(+j{>`q1=_L&w*7i&0CYiWX`dHk zc{P2Z>Oh@s`_eu87sd5?c3{a~Bc;6#^SF0>f)JGm0+$ zeVYsJ;D(kz@D{yOGS1YX$UI&{HuxuWArg%=Fj_L6ZtcfOc1)A&)VA6kzggC1eW+7rCVQKisJaWujO(v>8h7z~zxNJ8Bj;;r3VAqpPgQASgh6MT@F?ka}mPd#e36Y51um?7#BifmXAF)V+@ z-@HCDEG(Gzn#oCxzWnNmazRIyq(ZeR76}3MiAsBlS7Z9Za|*+yi<}M;Q3Dg+)~j#S ztW}6eQJBb@Q3fL)HT2@!g6{SB7rH&A1cIKb^%D}-d?Y9QY6Wuj+bg-yct)~1Yg!<3 zje$Va1CL5j^iM01`YAr^ph5t-V?Sn^P-2+N3)>77^PJC(aKLme5nJ7BD2sM!CpT3s zXLxw`J*D=@b|?9GL2BC6)MqM?A$XUuVYXs`|9%OBoUKD&37?nfpeB|Yp=phgwF){p z8BZ)9y-j893}nwP0{%d?CPQOY4+`{U{kW?o;?Q*l$mQWRxPOU{Ufz9?O2Bg?c_sFu zDh#n%kSowS9kE|<8?fr`o+(b_NCjo~EAHT2q*O-|8K7ak1wS&xe(PK~8VvD87B_gD zNA(vWkdgX`lXrnxJwUn1dnh-4H1vbzJi=UL{1q3%)6T{{&6fN_W9oFN&kLJB z=BT0LcxI>=+GU?FT1{Dvdd77}tF84_G08PN<-X!>#q?jI+@fK@Q^oQKE%p;B&F`vg zW2RH_EV$~wo$b0bCY{|z7Y)&o@Xc8{@5@Af(;5Gs#)$2-H~8c3=2B>cV5dXE+SIgL zICo;_-7d~h-cQL9FBDt$1nj>clL=ezUpLo^t4)WsavLiyFS5m#zWtOLBXB3}oSdrK z)xZ>uQ*`IMSl+<-l`wRYTf@XWkl964=V_mpRV za78cekgc&mL(W>8E9E?Z+-am^`Ie0B_41Gmr^$4nWUggV?HA$uMTr&7g&gHHCdxm~ z^BPjB_mB5;Vwc>68k~Gur^yP=rCpR5}_!?r*@ax zk}_*^KR(h&xU;BA);|r_x6VsC@}{YGSi)M zrKj$+#F!fNA{T1W1nagIG9Do@7x5RV;}yaI6)kF1WMEr9j-1cQ(b@i-HL_S5TS_yJ zCNpt_Gl#Fi z(Keb*TJSd+vXQCkyZm3I3ah0&^QAs@@EsGN!3kkNJs7`1&P-UstTu-YjqsAz|G7=$ zcEww(g-PR=_>-ESil=Y!rO6g+uvX@eO75L^ohA}bZm-QW@Gyd}fb4-cngv3M-F`oh zfRsr!)f| zhd6WIc~809C459DX&`I)%4K;fFPuS}knui+e5Gkg76DyL#^;s`3(E?p(jzskpjOuF zV?v7=t@_U7c5)8Wk%qR!k$$KG1y2Z-_lENbU!7m2u5I#|C8hrJ_R~p=Nb`|yTisc; z-6MB?(*V%-vk5nj046H|)k;RE+L4sD_Z>A&KG%bYIxa=`DYwnBbMw&^MX98VX}fZ- zTYT0UQhz7ecchetN(q7r73BJF(b@6HDO)_R_JYk@4@B!HHpOR!59}~g^U^iW8<33` z2Xszm(^_vd1zhJNTI^qa*eE+$6VN9$?G&;T5#m0wSs@hwI)=2Qcri9bw}Gyi{UE)Q zvCe)zTOEdU0i0IiVc(UHW&h3t5}LjM;wxt`R2Y0w|2VI*Hu#zNXXw#(3oO4-@@{XP z2m^b40>}{rtoHgN$xjm9V(BX^Co=gQ)|6)|txuUd?tq{UmZBB^JGa{JNSB~;2;jn& z4n*h5z(%K4UP)i5d^8>3)~$6+aXVg9kq@~DY53Ou5XytFl594qF&#*>Pr84}|Ij=- zb_`o9IYa#WzGfc`KOxeSEp*1=WDd0ZGnGg6Z0(|zuQ%ulF4cP zqlQKyP0nNa4392xedGdC{@Z??v6bezzqt8K{9VlYfW^{h;8khO{T5c-+A_vB5P!=v zARxtu(EZK+m$A6~Mg_cV^v8_|my9dp=rxhmcGVg6sSS|<6-el9OtF>iP;LS$%@~^% zugnWhvl!LH>$5jfs~EwhjMzLQ7w%u9oT~@$(e=-d#Wpv(_VuJDjyQ%@%}tNEIb|d@ zHJ@ipP(Pdw4KLP=h%COm{bNGW3ZALEWs{ewWd-j z)FDW-Ls-bVDee2Re(!rw4&vgJuIcoyG*aySy{L%5}4qtBMz}r!Xrw@Rz*py+je95I~7q8{&v81Ii^36e#95z zCitl!jxTD=hz@LTDKvzhblu z!HI1duo8?ps_Wf>r5>N`Luud#O2ZxRJzMR~^mXWGBfhHw%f_rp%|o87@Ylc)?>MFq z#*_6O%}41)(Y3F|MB(1MPa-g8b{fXYAyCN&ZxS|b~|4s}!%+@3x9_ls5Yd5hwF&L+s#2Gdg zCUWlIv{#tjmLzC7Zk@@k+pByN$dQy4SJb4&HL+c3n8$>9tQtU{JPJT_iOmg5a`C+0 zKgh~!XaBC<{HY_{{A|g_ZE_BveZp_!^t`YmB^z-g|33-(tkJ=XOC=2;S212s*XrH&6+;#poAC zwQNx!v^N!)8ipSfGC(*xPfTcbHUs8hM@JA5+&a}}uC^SAigF8hqe<=?bnvno44e>& z;{^_lN}t@0E>^-vF4vQk5Q&dI;DS7W7_h^4fl7~ zrGJ1jnfd*=p#Z1#st}%{P{To4_!MvU1!Fy}1UBh7Ceh1_7p6?}qwRrM0hjzyq#`kP z&7?)8COs-fKR@Diy$rvm_m%1+swmMUqX6Zm09o0{ypz?+rEx5D>H6k_Oc4sx3CwSD zXv1wna4~hQJDOj=P)U@82)1}4zNHOO4Y8xg0|q9hiV2xcM(^Et^K!q0=Dp#vY!|pr zTh*_!2nZ&(c}tu8)>_znmm`5>74$-fg~ACwnhy^Q!;n^OiAD-&SDLd$%`lm0H&xv8 zD$d~};V9;47g?5&Y+U7}l)$(%LvvTkt^&0h#+JupUG!%=TN)E|umV1`x?XG+k&f{# zMym}o^98#lyjm6Bvn@9H6^qf-EiTJnP7~c#{cO#xQl;@>3^7HBuJW*f%0BYYl4jDD zr)57tpAe+ZC2ga*Q` zm)*X-6h#Puw8w*rwo~`N5_T$RrHZip_F6K(%ei{35r(;;j;7xS}6+LC|008SPP zy~n~nF^K4XDyH+7Ur(oDHqgZTPnZ1Zp_BPEl4$8%ge3B3p=lgUUj!JL%h~8BlSGdU zcb%~5pxn33&nyO)?Yo7yZj|>M-|!6Ov4vII)~_<_(Nyia(|~1Fhc@e@%FSBPw+72A z&b+qLr8eh%d18k*&=Jc1KU1mBXYE)T?t-3jHkGBG3U!{+(VD4*C(fe>>fJ* zN1D!hc{U}dep`LRIl7*4-%=_)0-|5{(n z@T$d>dha1+#e^|a>L#uQ5FPrtsT!F+Z`-7r%+d>>s*1ha``Pv;WlZ5ku$|ALe@1hU zE0%la`_KK7JRM&N(Rq7@Vi#5idf$&+Qti80#|`~h5IN%VNX#^bax1N_w*EUBfz_Lp z&}yA`EwsXJmIA3Wfnzynpmq1Y@RRuf;N48iKu%07`4Bv5G1lud0L(D!Fx1y>ddOmt z`x_2j&yr&~8&X@*AzU&}#LQ8Fc2#%A9{2*B>wWfq;+*!|Wx0XYbF`k6ki-KX0jWb3 zkc08f#i3-N2M|;XATunHZ2I%OrgOhUe1vhS1Y-ar&P^Z>0LZqi?TSrTrf}JxPk1ug z1G7pYWp5(?1Iu6dLDYy-M~ks-#xrYhs2XA1dB*ahYRnAW3UTZXFBS_41+`*Hy|rwv z7{_XbcplR&V$|t@a^1_#7i2^@&z6dFMc>8yqtNRZ%3<&hXd7%EhMaJ!vb)^ANu@2( zsoN`^RBrAZ8d{aVY@oCPsJt+$zNkM&sL9b@v?H#E@Ip_VmX_fp<>c&c6a&Qx%Du{q z;&JU}q6y%ap`LJgCXg@;=u@gX(~bC+tEIgJ-7fxq$~vptAhEV2j>@2o&)cA$t0N&o zcV1;Z1zY@RIBhZmtY=itdNlNs38d0?Dl!*o_5 zA?0Q~)B7wSU)!me54|$!Qrh5Qt(BcAj-y63A7&uD^RPO$f{xZ)p(qTWvOAe_rZQXH z7?^2>=q)wZ9bbY8P@v~@+;o^yn%IC=Np;9lef!2GYr$|R`h%Q4?JJgMJ6m77PsS{c zYDfmz` zNa_U)7VEZEMob<)G+%r{agK%iZfOzk9lw*V1A$QEZM^0&cfbJnHm9FWl?IoBiC&0D ztp0NU$~&wA#|cNH!_t=g4Wb3^+ccoTlBm!ZSK}>e5^&TV8JgMVL6Vc7X$_PftUuUL z;#I9qqV*So91)Uult|sG5~Hz4Ta(A?BMTeA5SspykopjVMH^et{x1Wd2QxmI!~giD z%zVYY+_i<7QKR2!r?fQe-Abj`OD>i+27s5y`3^H(f8cmBw@)L=?dt3>p_}t8*uQgM zWQ00c&=0;ghE^v1_$D|<=y8^*lF;E5QjUx*Q;qIolOm^Q5xDyc?w!v*jm#tfeM6x= zCw47#+;9uVPTs{70~oFBHWCfZ=0#iyfm1%*Xk*am_yLi>L?b$tcVcGKBxZcUb?p;9b0 zZ?LepuU8;~LW3P{Q;GeG=SO?$w%x$30A4c3>98iBHUw-KWiS<1xALBqqtGv1oo?5m zG8$&Wk!f-zpY60k~XwhpgfpKU22E77PHknc27; z*L!5>HIk#8HSHwI+qz6nv5b3>%}$4g5zJ$D7>XA-qOCGxl;a9~3k4ljbn@+z;4kW1 z)C1SGUh}s1#VMNhGvL-UI^00!;$FQ~Z_b+FiHGTY;Ce zG(1=rxKP;e3K|Zw=)xWM@9>KADox*>u7JV;Nq-9(moWGs`L=#4qFhnTIX7{O%YtB5 zW-g;j0AEU4)Fm5#!MTmt*P6wQHY&KP$8-We5^wMT$ZE7du`j1wl?ck%1*7FTCRUme zPJi~?&~KN8r_*F+xt|_A)Cr}D$6nE|GSDf(dM_{skMmuJQl`;)hA-St(Z1-6Ti=5y zY>8Nl)_>0`UoQTb&ZpCXtCYFL19gKo#e%C5ulaFvK3_uJ3g9t5xQ4skt|MqUN^ki% z5`2}c0&hnH;T>XrW#{D5i_Yy+7G}I?|iu-3nhsUX`VX)87D0|hUJJ)fzSA5 zxT!A+t#e->pA#+9h#B=s`*9ywp)OOZ?jW?SAnqrIm+CRLHuAh$$!YEz6>KLlB4N3n zX0d-8WVqW?wT5nmcc6|I|FtKoy=n&d7gWgb%5!x=M@twxg9}^@346|~#Gf4r{3Azh zDB9l!0zLZ+9NT}6IR+RSsXq&BsYWXtOFl_hx3!4NlDL;7fYiT&I#C(r0o-7jMJr0{ z(@rQ_T57&5(X6f0UgthbfSN9i^PhYt`AuSzpg(94p&1>;cB585V07O;kdmgCoKxgu z6)}~jEwJQ$6(tq@PGFK4?YDZKe0VsS^n|Y0V|bp{ABK@y6zFc;zE4`oKwt8%0EKO8 z9_NJqNZ5iVe9mCvk=`4>uew(>UHI$L*FX z%XWL$UPYy3dK7z$S!-3DFfkHRt^Mevh7xOm$)tSLl6QdknJ|7AvmC`5E0X)NSk^s( zFJY7-hC0D{dI3KsZq2Ym$iE)h1-_Z|d=i&u^gy~l;OUy+a4aD3T8h=WhybuJ{Ph`+ zG8YeJo*jd{*s-QRh6H8>!~m?8o|x$bU=eK$KBL$FG4rTxu1tw~odoC51uAu@3_8w! zN4M^wtGR?dkiHD1Z_h13h^ZeZyqQep+XAMkK0B^cMl@JH#?97`uws*XpP$kn8Pe}} zY!PFyrn?04W%c zmr4HkK*mu8N$6VS8$KL+J3A9w6T%X7K|y|L1x;Y7PI2_UJyr&kp16s8qJ336bC6*L z9Z==2mKk!f&YkpN4ce3v3*hh5n=y7U-R7P@x%|f3EZkv6{@ASEznDl2m@)CcL0O}> zNa|_CF-z(_#rZx;f~WD~jlsOR%>;*L!6LqgbF4@$hlYNF#Y(K`$zeki2KinU(Z&lu zTu~lTtxnb6UsN6!A8Ge&l^js|3lZdFOHzL&sAkbshK8QJ*m?^h6Zzq!i7fMYn-FH2 zRS&y+sa;q_TGyoV~x(6ca**)1;;T zjsau6z_g|Am5v^u_wu3T2#!TkLxN=+9n5mbt$K5Q4? zw5lvV@E1|n^RLj5opl9Yy|kFg#dIQS-t}4DZ*?w+8{Vxtx;$Mar`w_j{X<`w0l3SQG$%#}VCI_!d3utgz?;2i3W zwk77rc;KeO2nU*5B)}~zedECT2pxZE3(Nzgowg+Ibf=fENGjiv}1w$YJ+Uoq&cJ7}P9%rqNF2g+;@ zCP6P~w{5(90gOIf4ERO6d_jSbrn@U9T|^6-q$#!6i3*vBcfFlXgbTOrynHAb--7lB zXr);gw_mY@gN{POI~cQ~Bo+qot9*_FY}_bYMy&f16wlsQ6gMDVu%rFk`7<_5SpHh? zF9whh2gY``&-z)xG744J$cIoZr!H`-yDjxB;gVT41y+gm)T6LsjES53Tl)ELPw>hS zBfqgpg{Z^8!%KdY4Rh}N&0n1Y{H*-{{2Ee;(@TPMGGarad)18<=qnil zXaeg2mUk!RYMAryXY#@*`ExT^)$Sb1Eu~6Pz2W>`*w#w~*ppAaCOSzrp-x-lSy)6& zK|&rvKhQAZB3Dkn-Z;IzXHqZR;TGyHAE7@!FqECD(S>AsIBXOa3BBK3NzX!2)-QTPgISR~e0oH=-j4~r3qkPA!QRLTH3?IV zA0tb_w^UC=%Vg1|Cdrl)OTy5Riwdo)?=w(F=krZw;$PJ4ix7Z@(WLXXI=ys-9~l?6 z3O703Y2@kz07(ftb$4kE$o!?r+^rHDz?R=n3DBO1oUqLLQbLQpJV31wMfhqe{zh=! zS@wL+Hu-qb3(pkM)xfqqW4xMibadT~%qpeTw#s_sE#dEagukWt_61R8TyEsh<;X3^(?)6Vsrd0 zl49Z4VThw?h2BVbbdhvwn30UV%h38K)|0l05oGfg;RG!Q#ZDEHoF;z(2H3ElSRDM;;8;%O1pd!1zZw0CwKej=lAb2*0p;3kV?q1(w8C`Ovtb>Q)6+XhfA)bpD@cj z{yP;3&@o4*pyGX}MV*HxO|P@WcYsr6$-WA3i7p zG%jXg+C|IJ#+~HHk1uX-jt3$lBHo&tn;+S30eErOkyeETU1KlNZbg^R-KCW0qXar( z2PP$8Q963(Q#Y#F?)83C79d8G(M#5@vT^L=n&|S1?5Nc(`-poBpuXj{3&P#~{ZWJD z)1RDE!bBlS<`3bBgX4gYhjNYmYPR)EIb0RvQbSw_ufI_;aS?Y4e)+Ln(vh;nID0SV*B2As;ZlSBFCQ9ps>n( zA(TD#LAskqi4+y(^c>#JWG*ss#A`wK`re2IBJvgknpTr%(fXD1u{jyCyFHp zh%vSgyM!kP{ssb8GJt81>?LRkr30Tc`KvushoI1Y{jvOc3EOHWkZqp}$Kw$7WW`q| z(Y8!aPlxVRJOJvzGu>?oxRFJ% zbX76yh^@Iw>$p)u&^}y4SAvLc#8{RQ|7>rLDJhx(UtR`?c3i#j6`q>GOdhRa!NOhhGFUC zf11ITYXy+&!)XaI)V>=S29LuU-Sy$*v(|Facxhx67T6f$x9E;tmXstX)W&4eAnpLw z!xX=#z8|99Kr|>=lGk3i)k|X9c`GGVHwX?1pU9s%RVM(k`TclRxdR+-491{5=Fb$P z+uZO_PS*hQ=vR+vp@ikdIZ0_Ei;Y;X3AB0WhDwaXM54Y4r+B9#KxcaxmI?QfsM%V1 z&bs07Wh%wTtnIac^EuA)^Sz#Kcp;s^$H?x|}kDSX8^I@gYzG3Y#^e}$jhEZ5d@gFT1TGj(?mL)VnQ47$omLl9K zSgolQwhh;Gp{|YultRw4jfdQ(ao7_r1M@!L$#`UoYqs3G33c9QsXNIkvI_`q(Sgz% zY$wagJxzXiPp{t|9I&5ke5u>`Zs;=2s!&LxrY1T`*r|l2dGP3`9FwEwSttEyzPVRJ zPvy0~PaQHbiSy^3?rPH-2nBt?3FU#h&ef8*|B+=4&-{6fCht0eR4GRsVvV>dpKVK&eX^ zsHQ}rR3|kGSu>A)$&)2HCzo%!sr)VddNn}hq}f12Qy<-sd^AsiiDV$Nkw;j%>9%CF zYug50pO*A`5=zFBD3sW$4^@v0Fx}!HoQLXPI5*{fm(8mj^Y=JQKcv|hCuh^h68i|C zr(HN`JpTWTQouyTaE1kPt-CN1OOa)?DD+mIPoj+rdV;>x`lEfpkkYi|uQNs#lTQdlBih(}m_zsE?iX!hDt(d=6a7%jXPp6D0QE)JIH%X&g!p&+n`kwNFENze+fPN_xagST}g;`x$k{U z1HxKWIG~iTMT#SY;x)h@YO#5f)rl0-LHMuNoNIIJ+MOR51JS3zl&)!eDX1&r&KXzw zRsgI3IXEFk>VbufSlv7c9c103@Ebb^J6If8nkN%5RDRqQ=YLgB%x?x*FFyCcIrh^4?qM7N1f_e3=FEdwD(^M`ovY5 zu`~skL@P}+73*q*6`Q%pmxByxp8CTbK3DxyU!V8`>nX(kTRNuMOb;z5MgTiTFmYDq z$H8N^zI3IJ12oL^d#trC)>02-oi0<-P>cAl%;m$SjwPYszemfN z*wf(oi5P)>{HwiAkQ$)dlC@@_q5*=gtOW@A@P~E^= z(yBH4+Q!xKo@uBn1;q@IcAu)+FDwZD`Zcxu>y*2Jf5rGPL+kpSdl59F0)P@^&gf3t zJ2$25I>k$$3sS}F)=tiCsdL4R0l0^!AeL#JCefIIhGp(jxT+gfYFv(WOL9Qu7TOXp zt&IwtP_w!(uQzK*7Sh?)Z^Q!eOlu0TY`O=%1`{R#*MGnRjHwG7xbf*%Iz0bXBy3=T zeh<&F+7;L?G@ksSPm-<_RoWhCq{O%$*S3{0lSOBoi^M$G1}ZRdk^ok5>x+Xa>iD<% z-j^m-KH~4oz(**aTda=Ivy?!pAEJA6E({4harE=4+6}Kq%|W>LuI*Vre>FT#SKe(WgL(|6!A|H@ppifz2jQURj;*&21JN zVC5e+qu@hfQbaLzgE$5mY`*sEeR1nUHKcp+gsr_vJdNhmni>3F~ zpRNcVf+aNc+}hc`&FI!B#BzRD&aMqR^d&|lo$>&ZU}>aN`04>~Ocuz}%gNx;q}%=N zIr$jSAV!~DABT(FeS;P=E=Ze|`JLarVgs#9zU94RwVtgmah$G@**kmEY>Ye|$2g1G zgfWY{i!M~n+6Vh<6*%P?=aUNh+<9&R(X5W=fC6JeQwe-Q{_5PiYHsc2`|mX!FstuR zWzQ+W7)-A;4;{jux5+)$F|hcS z2qV5d5TowTqDKq)`uc9asqDG!Qx(Z-e>Rn<&2UmVZ<5gC3;sK4zL(6?OT^fd%3j)N z2bM8NXRx%Sq@>*9NQO1yCbmnI%_foE9_vebrNnQq_!}7zk|DO6?=o-`Qe0%fZJ)39sRilBC?t$u z)4%3fz{~%LNAr=Pf(s7yKRa;W_JK|6ImLhC+9^#q29zvKJlK@E7#XMIefgz-&Cc zCvwt?33t0w)ym4`91Yc#j)q1R)X-x-(8$SWn);h`BDCm@ zlf^``b3yC{t}*%~PCs!n1T?;c6ZgyQZ8eGp!HRmqqU_eon?4kn^YM5>xo&gS%nz4f z{LyaZ;E!KN>&;VLiOu#GTyr^*mZpkH8T$7(QsW7+QE!NtB1i^Gw;bOLge|w-$ADyU z1hErm5WRXy<#6Gu@DJ3H1H10ack*9~=#pj(O|0fs@#?#i*FQkHH-6ar{<{kSV)}-S zw&;G*!`2idi*%rERsq=}nQn2JFn}Rax5l&X6H!Ts$Nqc+X8%`BbqeIuD6WZ>5hW3> z8r)n?tm#0LMpN&S3-Nl?{D|cD#f2+oP=RKNSSSoWYU8?~!6TeW@0O;az>`Zc}22}%ku?guNs zZ8ju4mO$$mU|rOU|MtzGmh)pTDTFp3eR{aj%OeL@_&jj!{)Xl*YN~!p`q5mdrkPJRRgAhPDS5jYGUbGsFCH3f#>I1lJ^<&@1&AKY%-DbT720$?|BY+~kDrL}$*p{T-p&0Hj)8Y&_t^>mN@294U_D{KYr6l+j7=$+SB zU^RtMC-ZSkU>vtKd%XL|$jGYTLX~Y&eb~S%I0zCEQ0cnX{E`$}{wcBmZw6oF=9SlR z9}l}ovhGbb^0+ub^$^Iu(4jCqHtKbR5`*l3%V3-ft%(ETukBh_hB0wC~>-!z= zdtkc-mG#S`h~Q5&%#CmTj9r#o6PETEll}vib|nD)!cuoY<*#&jO^?|jk4 z1c2v#OeQrt|DkVyFBorqfjtP>X1O=H?oUBcpnk(kOQaoW%M+6cBkzdQa$LT)57?s= zOZcJPTmV{r*w58RqcAh4kZp9iLpDf(Wb zX4^Ydq%mdj=*(UYMRPhSA)1$1+=QM-=;E>^u9@GNvAmjEffl^A#Q9Ysk$=NltYtdK zi?w?{d|`%bED*Fm8RQP0&*eK^i?b!eOGLhW{Q@}cGElMLbCVF(hMS|FRnOMfNQdTH z(DMS{jJ2YMGp&_!db2Y?6Z4$cfsvBhW9PN|)n=CUK27s%^#I83IVjubVkvOgZ9H3k z1_(&cMNNuaL)b{@)?h}Ub-4o=4r_ZeFAmupq7C<*c-{4%SjU3srZ-e|(9*B-aKN^YJDy1HR5cKL~z3y6<3iyd2CHJ51D$0z8z<*;-fS z>o8GGskc~A?%@QQeA(aH|9Lo?o=DHvzF+**`o2cq9}$F^1t17d+3FjJ{7IPNc=9>) zNS}M9z39uhE~={cOdtXtF1(KdjlN%wdL&9|tn@Z}SO3`_UCaX>$gA8ZSxkz+A*3BI z8F$_NubAPl9)Pbp6g`XnKTFyz@WlyzpUAx!0LHVy{>(uG>aE+W1wwvGKbE2xUO(tn z1gddsv8!_-Sa5LGG&XCsC?%P8b#m)msdymXaL!1zK^clmPq^?^S2e^%`ES{0yw+Sx zud9=1qWYktSqWKLEWS3y;UtSe zSO*iTAgo8Dna?$0%)(N43-@=X_H=NkY!*>oWC7?wMnSJpt~N1ije=$fW`XFv1=;(1 z(5;1Mff(}Y{2&%ytC1QyrJ8RvX_3Lq6rXq3()2(edC`a%j%tk^9bx!Y)q?j7Dz1@Y zI4}+Ba#xyhOottye#yhRiVytBBC1t4*wa1HT36(Id{=rIs^X>&IBlSvLfv47% zF#oqG#y6TbEl^_h&jjQ=eG_L6CizG>7+HMa)4m;e1o8j>?SZD`0q|0a0QW5WzeNi6 z1gK=(Pox57{?AH3V%JY@0Vw#U?0qGSegn{@(yOB$HbTH-p%qYkvEKr~L_;k*O<@_rW4s7z%Yh+nE|R zd>8352G{r%WvQ`Vpj{o_nn9#Y1wm{BYmGVPs>ka~054Awyi`cm3wnaltjcFO@3x-s zfY?Mcp44EH-U0)HlT2EYDjge?&1`N^kKAP?JaJtK>DD zoYxo`JMvL-ev9~;qUB>oE7`JK*S43aMeRBSOpEV*#fcxDB%$qgjen#dU6v20lVw27 zO=~vr2n5)TR`>nea*4J)2-jN72e|%s10^5nf(F8D1Z3B!297{ku;Bo?LO8LzddbwG z#G+YygJyu-P9tCQ&F*xCY9J>70M(S&+dW58Ru^@2c9Qc^``p-(@z}%z>$Z-Q(Dh;~ zo@N8v$ApC9@87>?e>y9+WV9`3@ZPPWoL`3oys`E3A0PrDs+^kVvWCb)bo=U2%P z(iFT^Km)0s^Df|3xc#=LMJ}*N450B5@$t;zi)Ppw>Xslz^}SypVtV-)0|g`gNBKB` za8v_*P$07W6luw_$3~Pr5tmjdi!HTk(i8PZ+{LDYv7l+Dw^yD||G?^f>4&c~I~x-} zm4s!K1P}e?$Pjoy2bP9Q&fR5+sG9@WlQ))igDGkV;H|WY*voL&ia>`FspPz)d(RD9 z5fv{YGyyTA?d-igdU!DMt30rp$hRhnY&l6Bp0L&?LX;c#nsG361J8t3S@I_UK0cpj zU8?S!k|3;j-9SO{2~|f}vA1-m%s(INfsF4;XV_5KuC}~+T3XM$u`RphcDIO4LA*P{ z1y2Fj1CU92mJEhE9R_}Qvj5drHL?0ve}P@icR-zRokLFGBMqMVvvdtD*8X!jSj|ni z76ijR+cv6OXgzvviwRp7?);(nyMcf(S@=2}pN05=vTtN{SNF-Hm|KDcvH{(hbkrdOYXnIsY-%Q#-Cug0fP}<7%%SG^7Ai%`r&J}vrriPHPJ)fd!y zKLQu^)zzob{#%>5-V9Kq3Yvk^$h(C&??&wQvPV;wq@jX34P_QdV%rh zN~e_%>o*CGeM!Ev#G^{nndhM-P03rH;cJ2fevoe&KRt-f>gG+qDI2^&iW%!OEU$Fl zNuD|FSRXd1I<|TJm!UYs)R&9p(?@GSbiy_h+eG`7w*&+^ma1=0eF&|03KJWw=P^Jn zB&Ep=#s=}(1(jSjN$jS8h%?+2t)d0O_rzShejL^9JhI*qZ}N4JmKmG78(V-?%AI-l z;+_E>$3n;BS(1D4g8FEB&kAond9GBV~gdvY;uP?Wwo@4U_B)C2+DB(wjRRFU_FF`+m-Wv*RN%Y$`koC&`w_U ziHF1}^+OwkoCH4Vl&5fs{4qV!IkJ`)pYNocQpZ0&SQSAVOh@aAUAdCp0a(f?C#suk z$lk`BVuG+|=z~wG14ddl1@Aae&uVvyoOsaDk6uC2j64pj5N1QKN4e=b7CXx$Qa0Fe z*Nt!}`R#7iT-Fx0u3VGs<)^?vt4AAEnl(JmQZHyq)^cXZL`jIl%^>#oJl>l#-TdCY z65Q6mw=z~i&Yru{5uDX7WLyW5PMxC#sg(uhuNYok; z-%Dkl#12kw@qFTkp|5-ZIWxU}J)mJ3iVj=-sxkv=Gq*azYiO2$JUCsO77RQ_C^#i| z$M3x=L7`B-ByB_cg>Dea%R-U5q1YJcp5iZ@R`$dd2c5Zrq5Wa{GSCZ7$CF>qXIoSE}tXyaO*jd-}XpN$aJ5c9sDp42`Ni$LIXA zw)2>g9aWR(E`&DaTvK_?RCJIs)B>-?iR0*I zvB}G`W}|-YAo)^aK0xR3ahwMYo5S6fZ!_h+l)QTvGhHkBP3MHtx&|YI9;RPn?UbQO zs5REhFvdn23j(fL+MhE5Engr*l@#jGR6;V;v~bp)kiRojawG;I*Dh1~ZO(m&KZ#&w z{+O;f8Z_Dhs?Y1L!OYS95BPuk(*W5YP(*wbmP}IGOU1R2c(X188Q^S8S;{ z(*y9Lqz~QE6VMy=rpU%TAmMX~7j*6U`j7&TfCM!3t$=6_nlx)T)3B1^3>cN3HF@yh zlT$-o5j14&dw=IV+7Fyv^5s(xGV0xQ(>DguP*i;QT2Qbr-mdNDxpe8$(@fO4yqoWR zz+S6@+jgnT-@2V7TbtUgW>0?X!;KpqjI|brb}8o*BoB zmqzvITCg=AMuwJH2wlKi6pfb>jBueU6I6wUQ}=fHJ@;Q8IuT_q$V4(T>2w8sCf$1W zOo2$bnBwy)Po5Y1{@Ns``kj zop8@kNa#Xh7jk^T{_J1BQS;S9Z1C0oulfIM%S0NY+HfLpZpP;##C;nVpf}V%nRSL4yv9iE8bX4UBr)I>_{9J$mDuWQo60Bx?CY z+1zs^Me;=dNL$LC5VJ=Wn{OSc!W-ZK-)nFB@{MHtSc$KNJUVs z6Lda})^WSDg06$fr5W)$X8AoIMmIuoo45 zdMLDQ(>?L)8ZtbngX&98WO%C=Sh5j|@jL zgiocJZ8%c>0l1_so8zK`(Yju*A(0nHsT(OWXQ0$ZwXW|jgsSqMF`}n);%9|!b$545 zk}o-}l+97O9W0m3(uc}AIq}pP6<@{JCnqEnOT8EOynPvB_IcOD9et3K_T?E$&GZ!w z=*U1)9neSMMlI+fuk*+l0S`xWsF)h} z^|~<-Lu(0%=X0IMW}(y%$cXWaaar_~H+`kI01vqsFJbz@m|JrdLe=)Wut`#Y%l4w5vB@1d9_f9nVIFwqoI*rU)-;a$CM zxVIByT|LCtAwxhS6E3p%^@XIikBu#U*K6&s^hi?$n$wR7e3KXE+FsM`*dLtLVkcHo zO}Tp!4P{Bj{~UG0FGNh2!px$g4I*&?o&L^v4E&55J_awz| zxuf|^Ti080*-2)0pxMwKVnVR=V-ad?vG3)s)T@HMoJ~`B_u!gO7OFN~M=U>`cj6O31y?ctp^ z=j*6x(;1X$XreSrM5!ZPT0dO#O#kxquTC;*zJkgSf(S-N<8r)5LXg{7;W6#MHxtoQ z*i0l(e`x)xGygebgUmMDOX@ES(=BORKOd}AM5M?iSj9FzUk3RZclgFRe?)JE6cp#P z^9ix5=YLEauwux8Vs;O9&EmT%XD$$mQjYT5lLIPA4^=(RlGnTs*Md&xbFr&A0g`g& zLHPUBR7C&`LcgG%!XSDW24)0WGIRm{Z$-4#Ek)hURx)Zp;XZuvngK?gx-w?VMXrZW z1zwJbiVZ%alRBp-fiXE-Sjgp9kdyiY(_aYmYq5%@IVg5i!XB?$qSzKcsO1a*oGt}h ztn3;7n}wZ1+e;D7Xs*Pq3KjzKj&o`MR2QF*gUkEFGbfo zP7@XT7cR12q9E9N^~=&NE$zA1c8VW+*4CW;aX_>2YKV~JDl676+V&r$ASfF#X)A*^ zt-np0uRPLHPM(7JrOWLT<<{q_=}J#PWhWaR z>xx;AhH}eum-XV>pkYL}eW%b|-q=GE7njm!-+&2D2I8QLPvwS>Szf3B+WgeYcNw)y zHy+}Uc8xZD{rtidiq$2rncZ=o@o&c;6?e7+ z$h<-}L)SeE$L`}L?Xx1MLk?1T_NM+#O zqiq(f31;Q=E|{JV293cfmC(Qtq>1!=8MKo(%O7^rDwa86m1~*;2w(~%F#Bh(gJs@@ zv5FE~DjWK5UTSKuY&#|>9}c9*k~cVB2ru_Z16*LqS^+gRC}a^3^4p-hIg5G^pq)*P zFWv8?lI@ovkq#?-^$sMqwVnz&IzrNS?mWALl5RCx+Wku8XZCNNFNrGAhgPaP8%fmEzd?@|9#0;KkDZa&I|gMde@kUEvfo4*5i`gbidJ%|586QT^hx@UH2mI( z{>8`=*xt2S{Cf#ZC)FbT%T%=;BfH1Ej`q7Va;g`)+a18m^*phKSVXdVXTm#Se`nR2 z!jK_HRLw`vre5i3`i(SY?o;!V6sHGO11bbzfEh`j(!!=`iywy|j#T!m#47$ZPzKE6{5JW#0{_6mMr~sP{G+;|>f@(%_)lOzarG zo(tc6eE_K?_GKhtVPw2$!`$@Fd>|W60_0>*9D?317feySYBoAXK)WTLkAL26?u7wi+`X{HQN;^WHxT;sac1v>z=S z_}jClF`Gtc*FDU2_m|FF)ab-FQT%dX%$LB z`-wEHj`sk$L@P^5NZcV+^<=W^CwX56m=bnZJQdVpX6x4Ilq)SH(o!fe$iBG+ZBM&0 zQUS4>ziLr z1rd+*F5!<%yZ3EpuyF_4L27Xx<(AMJUh8{Zg!1f@vV5fm;d<&WZA?oUj36a^>&e*6 zK54bT*(*mLon5&ueVPYt`lBG43&L5S)+T%Mu><;KeSjS56l174+@q?m8^^AGYs>rO zAYp1Zq|oMN0Q%Koldxa@qUDWve+jae=-)>Z9>OUjkd-<* zfZndWbUfWIv!e>d3iUnOw?csq@NE9z`R>{9V_w#NdlS=@13HT863=aHhj3KW^kq$O ze!>fyjIk=eBT#wsEMI1Q74Z(IRsd9MJ`M?aVUNH3gCq!_^0`$lipG|xcOc%yqy)R6 zX3n!uuYSXgkFd#+B}mNU?z&go)zV)AeVR7`k{RsCLzqMXwv6jno2PI>_`}>u@AbMk z4e{&DHNJvcL`m9GO=uaMVRAbvD&FyaVTQkjri&UVEXt4XpZ*+Z|L*r%fdXn#qE7T?<|)w9;fXM zcL<8Uv60ap8;a=FS}o|LYWq1Rnub-&oEj4UclS#5@X5Fy=duc+;b%;E?!@_X*y3G1 zBbtEbw1MU?A;f9imI!loXHJ141Lfrqq#UOkEGMetoR$W~Y-E*D3C{Z@j59c%`9Y;8 z@U<7+6@|G0lW4Z?!#UKsJAC)+lkMTy+;uH5Ktw1HVZ*Lou6=uaN{a`z%+1aWQ_D2q z0qPgDX^dK&AB0^TG@5*I3%)$4Hpr`vKg*ik;Wz#&C!OyZ@S8U zQzDF`>2%jLCaD%pvl*bI7-|>s8y}Ldysh>;AQLTTARt-($cK`>#XDKK51Ogr@r(<) zrDEvoeds%Q>vgD(09KtFE?x+A4LiVD#=jiJrYTu6<`hF=;=R#u5fa@YhtKrUyRrQX zw0J%wU1kmrh0WK6LGVz1v^^5WWn1qNVQwdRB3-3Uogurhw;2bJ zDgny$=BC3@VQdLsP?;YJ0dB2kMCAp@wV(v21h|bKg?K~H@v)@_?Wb$bP1ju4w+X;( zoA%L+4O40Uk|nJ#loP&JE{>WU%bY8 zBLYgDH;@8Q{MDn_2U#IL{&QyH8psN9b1Aj`dxd;L$c;~DPV+wgdxd<#SsE=FB_?t8O$jK>`JD8FQdz3mTVF=zZ_=x>l5d0MpzRBS6 z(Lrx8F^eE~d+8|x{P5K85fwfg^JjQRMZpW%MBj>E9joB=+?$Q=Nj^`rlq3=Ibq})O z^!q;55Xs~^d{>$iv_w@f*mIVX8Ou2A!jmtqcpMsfx%ROQZ_%m5TEJpP7CoT)IUGky z&e@)H9t~6f8L5%sYjgK*P#3N@4vsn)>DD5Ur|FyvB}=dK)E{gJ{_wGwWHDrwYe9nBwF0}`xdGZ!QiQ8%Ct=^(IRbeKUm?U_r z2#h7&o#hMmK)E;d7`IlD%_zcXZc6X;r>j6-FMxlEakw1EX%g=5WqU6Iw(mJDm4nTB zx=RoED)!I$Z~)aLI@27%%fEo8BB5^67T}%^EH_U4$&`UVGY|&1(2SLy@P5`d$sucjCIYCq)mQ+_y;NX$ECWn=f9Nh8B=SwYKA=_0eo!><$|3Ms!<+;!Ie41x1i^m}Dc zrbmdt(LDib9ORA-XD)0tpkB~(p;-e+zSjp8{eep97Bp|_U##p|M16QTYTwyC-54@E zIL)7mX6b)po6>poQo^%%QS(SF&C*W@`Wu+R*`c2a{#%@*@(0@1($M+TQOVpU$ka>D z%`Kok$}M}IWAmVc4-Lw|aB-3Hzuo$97d_l2iKWS@+U88GPONI`-M@2>{zYF%4=Hla ze}sU;1{KJTsQ-WdKo9dJn$HA;fVDRqzCXoiB;z)FAR5K2&1iv2h(<69d>yI<`t8Tb zpBfq(#NWE+pRrlOAh-x<1;lbMM5!sb(Zc{2>48ao;N;_MYisLE5}D!gos>qV?L113 z4=`DyA7NvCRWmr&r&xPryTyxQ*Kl-xBN4FEe9z-3!qCkwV}M*Rq*O!CPb&Cy9b=4I znRLW56`&-=iFj9c4o&6bmS_`0#)$<+TAR{s6s>r?)b^c2EaoI$$*^=JvuoS=(2&Vn zDCpu9eSJ#&saFHzQFL?{B+8(p*UJsGdDK?)@AzClmQZhpBp~W|4xHLDzNezm3ee)sH-J)GrOFm>V9GdnlN@ zUXusD-nwb2`fKIX;S{6@Z~`r8p8Nf!p?-ygd#^UNY_UN@ZGn9jpEvBcWyKZ)NaakO z*wmGOxS(H$1YAlJH84o>XX`G6&W62DFNu3t(QBy?@8Tq0_+jT&n<>nDmHK=~3~vN` z&7s}9CQMusd2m=^hQL6r-vpb!akHO*Wh3z$Hu<+{?2K4>*F~FKMwDt;3`;3A^5g5C zbvu|7q>an`CT~`XJ&+)uc2)M+?3Qw9Wh+T*Ferkqk4%nBgWcaZ$gL~I(6R8j!d-(9 zeGxn_i_u1=mP#wV9xGuB^^u643V+zNw-Hz?vSLw*R2w$x5G zr~4Ykk@solv*-BzKE8U$6y?7BYWI)P|HVfoWz`m2j-){-T?h%nyLXfV4vFA{6+wc{ z2#guhgl>L*zR-D92by4CtOEGQ--Eg7FAzk4)oBEwo~sYnWrkK^FDZaRNkT$`_y%QZ z!(I-I^Inc?^?=K5zua)MhDu+UDS`i;shhIC?-)ABWP=F^SZ~ym+5N!(iG*M#!6z|k zZ>W+@!+Ij0M*rAu_`9C^=3RI{@*dkU*}|qVzWGj3QE9lU*=|rw1IDzBC|67y=>#IbzBC28{neV22Q`qjNq|t3^}grV4*$Qv7K|*TAczXo^c!U9rRCL_ zIKp$qG9qyee&1Qmw?5qmL!UA*7tN3D4(K3>zhv7~{2XD%a|n=jIq=yH!UxD6%}>;# z_$|ek|Adlt0JheyD}P%j<*@~Q>?2rVhU zN(TMLm%Q(si?sqjj(i)tT#+o(RX%q0ks$-O)Zo=q`u&TH<3A;C^76=&%eUB9z0HI{ ze=8_3%JVyppxH!FiOnW|&HoWE{Ty|oQ<#$$q_)>5Dk0sgb8o#^9B!^hmhmK%gj{UnI;@xZxN2a0YX4(a2jvt z>S&VIJ-`$8v{X&k2^iVBeK)y&43IEMKM5 zlX+4PO`dXl{LJ)O1Upq+GS4nHM7=$Xot#4v!3tippB#Kr(6k#=K$kg3s^Jlde5bF6 zJRpA_1g|p~9gw;D9vDjA_8TQ4Lbwl-v_mca-^=ZP~igJuuJt(_^(m zF%{+ns}<@$f9;=!)_m`nLE~Fu9bLUNj_Tm^OT+iANPhmoKX&Bkw`uvk=|pXTFB_Z| zMlU`2ed&+$Y9zE_%y7rBSwe0)pK-2=cU%g+y~L+mu5{X6N=-So#cFKbF$hgXyYJd! z!@ejgH+7~al8Yb;3w=-av>5_VVTv2BDjoq%YxHk^TtbO~nu90CVLvZd<=1+USt>Ew0FTWOrZx=$vcTpuI7P2q$ zfsWMwG=NW;dE>VDU#Iz0J`lwj{(n4Vm%A{92&NQ#GTK;5EgAE#zCetyI@uU!%f{uA z8^}{pT9N-AZvr=D8M+zPvl;wtR{nS@KLpaLQ{ueUDnKi$7BGBUYVbaN;~M|`W|QU@ z)Nt?LkdaRU`XNPE0($ekI>Il$>_I`N2~_^kwEbDjqTS)18e!Okljek8r?a5HZS(iKM*R5-@ z0`VGr@c8Fw@Jww^*@y;mUA{w$ijE`oQ>m&bUTJafS6xm1pnf@_edSjf|>tf`DU(;+P}x@eUgr1~vgr*EpJA?y~36 z8?vL3RTEM^R^;2J1uoBQG(05RO{`iiMVkOZoQ83r@Y}J%;(Te)i>Yc`}$B)YN z8c|Q=!5-923yl-B#g-)qt8$=QB=x&b-O$Q8pEDZ^s3-+h9TFLzaFVgz`)^+<8C?4&sRe6#V2$Df~S`` zBlG#V+hUdj$U;)boJO+5jji7NdZh2z9*1kq^H0U`@0=V>oWyMdE8zg*^Rv5?@OZIG zLBB@l_b}>BV!-8&`*MF?Mn*S|-exJUIc3#z)qU@M$*@_50|-VPNNQ2qx1qe_F8Fz` z?h)v#knjyO?+!mPi3 zdS*J9t3Ss5dBPo(FFiomzNGTnog!-fc*MY5*>PQFNA9)fa?wB!Z3ydgS^{!Q>x$*# z!n#xBn=aMkTm=5hVPE7QYPKwtJMGl12~#ITG)*W?oc;L|A&Itf_K~~y!>?8%$W(FB z(_oJfy^#FxkYJ=^T#<{9Rnc*s-P|6ro;#2en15dgS_|c(yzbaqk)vLbWX?{(f`=5u zNBDe`K0imr2{1Z`FX|@5Sa4u;hUwb?66WH?te5#G@p~~7hgXgUypL3={EmJ!oXvP6 zp=VvxuOegE6nY2faR&{9C-mv#C1QHUL}^vLM|0lCCOVFVIrc&`p`1N1>gp8Zz0~p_ zo!3F(!#>PgaDBpKW7f9|DCrxrudZG?c|v2V8&w@!)9p(8w5A@1rO(rSq3^?d+PiRR zz~J@WzP&cNKN`Ut4S`%L_kb3sW!2V~`A>8ackSD`RcA*U_veIn${c_Z|27h1k$A$!xloYs9f~Q_rAr%2-qsc39gjR^oU_m_E|MhAOFVEQ?K0y>J-N>>Z#)+XLi~j(=17lYQMx6 z+U(*o?Rg(KljcEpj6&bU?RB`@GPbu5L}~OFz@lkyy91??8s{|pL8hX51r|z1?98v- z_S@3b;t>$|$Dw|@_pdkp`D&-{#`0Y|yDgg_ER--{?R_vJc$-gA*6!536dsE!*dro` z;~4s@WpnXd0KY0}qVh-*M36=#xb(SH8NCbRkp|U=DcLoHThS#m6?uaOA<1cy74s=E zDdss<#WLhKu<;yyhusw`PU$!BSGPiPV7TP!RurJU$?L0w zZmUwsqkPp)Lm(aR1*G-qDiybxQn1aED4Sm{I{gRge=H}mw69`=>3QUnF7xMQuspu6 zH73Dq$W9!L@2vAT(WNH1Og|xt2rt=`P5k**yv^$h?WgpiqU;fu+vfL8SbkAwWugD; z!~GTKni!zCqLbDQ{JZ15_U8rflDsV-w<>-6umYm&)Eldn1%ugIuG>Qf%+|3j`6f1t%M2y1QZKRfVyK%ebXrv|?*$Fk?AKklGQev#6Yjzts zdKBe^lnal(@z%T!7dhTFsHi+QbuUeKx_~@_Gq_Rg#8|g zZ+{IKNK;sD9NYcbhcJ<{Sdl%D|_2;{6QRTA!!JWMaLwyC8NB% zwV5M<6TsavCUK4`z0cPsytjc_+=Pv)L|Fja`s<}_nC@IN?`26#g5r|7oaG!lcVOA? z!D!m&Ek33Tzb&%&S*|CuphIIRL+eRP-xT#u){sepP(m-5%lkv`)u=g*?;=?F#&V9= zR>pjE&7sknj<*Vi{=vIBwLcfg)lbN}!OL|2^YYJiBRvV-3feM^q8PToo~6Q@R8`Iw zokc6s=M)ma<1NpbC`a@prRZUcUAvpIaE87osydwti!1n2J>J# zpBx>o_*Di&_1%KGQsRoKE5L*s%=thexiqAR91L~#lvHnJ9UcXcXfRfR3KIQ(kv2i` zr83?>_q^yLCWgMI_nFZDSqFdFgG5gZ+ELO=HENLLMv7do_LLOilG6A&-xH2zSFnMz zd%5{U&LfGhJN(AVD-rgp!S2q;7PA7s+@p^UF0B}5tFp@w!Op~aZV%0GB%FA-a(ai^ zEJ0O(PEn4XAxOMQwQR(OVHbxe#@W=Lo?p~GamyQ`cQwHs$Yxtf*`8cU|m5!=bx8h_13mA5Ga2~e3;yll*qnEOz03%Hxt9egvPmE>ZB$7TZ6%E*p&!mkgU zDjuoPN(?gtiv?$XI0@XdjR;miV!v25omz`=6hI6OhG9Nh!T_c#^6nhUpw%~OlM zXt1<{7(sTYcO)=*sb+Q@lpFIPk(B{D-Now|dsDKplC5|!X@Ri6YT2D;cp$L-6dm>f z6w%N1yRQPxu?eVhva*UgeO5t-K-rPUM*BdH>D@R1Zlv8w*)j;&Vv}mpNGJxJpweq) zyL+aHA)d?hez5!?KqfIsIlEf7MAiyGEX}j^{fDt%F4TLu2RA>9j>dl5=!#`_@rIK{ z`Ev2d`pQ1JzbEjeJ)pLDrCS3ENItf_kLLnD!_Mv5e2SRnQAe<*a^x1dVsC(92{C+k z(@Lq$LZ`Chm>JvRas}>4XKP3p-97v!2@}A9$R0`Vf|A1hfSca9ejgc zB+MxX1qny2o-^bVjKwUfr&zELdJ~ShC|$okYy3d#PpyCGs}(M^J-Au5>dCQs{Ts!E z?=d?pTVe;nKd11*Eo5dt_H+Mv8CkY>>E-wM=3o`kNuXTUao0%N|E5~K9=ZATh1?fQ z0U4F23o^R<_XUo3XL`|aS>;P)b)9v9RdAjB4#jcVBaONG4LwdHmeHU$Oz4-*V2>qH zMmq}uT6y!CuN}j=L!zhjPY&lq=9Jy(4mGTw@JrQlp;;-2xB4mvCR=MXQt>2+pv97tZ*DrdWPE&PE zI8y%X!&Y;&Y%MdJu+$`?ceL>MA5V)+yhFO^;mP9q$w0U&?tc~n{OJ~Eey+y$tFf0S zh~wWPS6)I&_oPn_J5RJ)>Z2NMXJ;0xjNexGK=`NBEqKwz#om&F@37>!AM!0U4* zqQfsnd8W-TSC(q1bO=N&8p9y!PT#lW>KqhV%q)mHp4B~0CK||mO`nmg*}}ZfGv-ZF z%x2_U%dXwbS74SYyYm9mTn3s98Ap3DJ3sLKoPIn_u@%*?qM4Ph-Nuk1t&w9Iae&y$ z^y=~g_L>ri>-v{@GgQ>~r_Oq-y|FAFdb7fbTN#-z-RgM2vKqRmDeSCL#61xgphxZ{ zQlMwH*L9y{fsNAkBjMiM`%x*m9QDauaE;QjHM|>{&K;}{qsa#<6Yd70;%fpUm%T-2bVu3?ZL%}$Kk<|3Pl@Wd*#+{ zFQmOxt$1*$u@FjCQ@l@hymLIwYfH>C*tbh)mw?Oa1yz|MNhoU*S)>#1@WXd6F<yyGr2hlJZm zq#2A;2n|PMf~9k2(=*og%BF)^f(@_8zIn+r=FnRiZ02gBbQlX^NXyN1)7M5(pC>{V zo0K}Mw zptzMqL&s}xcD7_=z&yL`isv`Zgb%~2sYGBO)kmyS(-WL-8J#l#$?$_{6B+3DmtK>~P3LAIi%73A7+s^#HcNjNJ5HOE`W zjza?&mQ^#%EPZL%nsT3n|BN%cZlDtDyU!*6*Gq`b;XFXCNU9(0^giCOx|Eb;6jjRP zRlVPz6EzO9#`0!Cu-?gXo(qZ+Bdnt;Eq)IiM|cW0K%~uTR9Bi*gSh)mh{C=IZ9-2o zjxY-T{D(0C2h!@V@~lSchSfD4^J<9o%~|S0m zo&?9tUi)aSbf*KY(eZ4;bt7pe_ewYA16D2763eB`OlTG1NnkMthYygRKy0jg*d_2D|PdvR|M*pb+W3B z=K7h0egIF9|5?5&e0GidK*QZS*mcX~A`Y>>;i3Ac>RrLcIp%{$LFK(X|s+E`g+Otd2X66E~N1!y@ITy_|MBc8oUVzfAmvbTsCA{C*Zg zt^a~A;<7Zjv&71hayOO>Ut-cM;$LI-l$7YSb8Jhy;yPp$-EXNmeo^`E%&W97q%tI; z>FM7!tOnZ5w{P!kmUVivTa!hy73k^e=;wFmF~m9C55_6UyA|2JbZv+!JVrsrvcK_I z$v+GyRzab#(gKpLd4RU$AXIaARgE4%(1)D1J!oP%w<=NeG0lcdS;stj6{096W(U}$ z*WE^KnsVRDTZ=B<6f24@=T8Ad$~wLR=nRpEV0x&M`+R)kv8n;8&X#3u>Z zzN7ERgX}mZ?V-93$g?V62Dr4wG@2Vp46b*i;gQchUUhPA1$y+E*ZPZu_pG@(C95rl z?=*gZzE)@HXGl2{OgSX&(VRqU<;p+%Qt`f0okiiTfU&eFb)BS}HdX1}S=BoqcLUcQ zSr2|qH%=`a5MfXdUTB`BHR8Ei z>@MB1{mm2h_Pb=ZKOO&fb#OcJVTJ!mzX@mv(1r{bLl*3Y=vK*-9!z~m(pHKOP0wg& zRhQ4)6^SF*7?K8h7ZBbHYNGZG5u-p1o!?Ak%xMiK0}?sh6iUG8Ys z$)P^Af~z+Dw(6z%t+uw51@aHOtpKrGP%P?Wr{pc8!%ZZf3txO0kh(}UWC0^yr(ny2 z>#EYuwx6>D!n;@ZIa548V7q_16r>cl@$tMI>)E`Kf!#wD6@6{6w=xP6MNS_Mbv};f z=Q__rs+?*0Wp$Y~RGT|OEu_|=!+v{0@ zCN0PvjZS>s+R;APz6osAQ+^PJGaG3?2;B}A!{2TnMmBR*j4KALvGQr}_e58}^ro&@ z{Tz**E;WK;EE?-_d8Xc2nKR>C9t4cMLLJ!mj3co^W`L?4pB>B;Fj({AYoz;b z^7Xm$nJfYCcZ+=^a1bEAdVlZzHd|;Aq&)ZG(Bh+;%5qV2*$zE+jPfK~$H>T2TmMn! z=8Gre6;2=E>-+u8WAHw0Lr$m49kuGUQ=z>d*6A|qQXt)h46GW6v4ag4wk|GfdagdX z*2X$;?c05K>-0)v^t|zOnx#4{YAuf_11fdy=~q_KMv@r%tqh(3(?}zrtZ4wTN*bAF zmi;I>d-?aIb^WpGrk(d59^S{L?>(NY8_+M>h(gardwh2TcQE#shL|0U-?~c#S zY;^BxE*?z%{OP`3d-V3;K%9>E;55Wfi5Gn#qi1Q9bL9s)Qfr0|a`E5b!)}nMU^zc- zDhZW(e_vq}nK=GZ2la8}%VsHR;Hss^^%xy-}K0 z>apK;YEC>`n3)j&&srJXA+VUaZoJSs$B1j60?Ysnf_^aJdXjo7XU94Xo6h6v?%tfu zHW0#XP074zT}>GwSp#1gva^s?3Ahv!E;Xt3a$)a1#4ICE@|k&G#nX5pj4h)y)@#2v zfkfM=Zle$qW|9%~qU%s?#|4{+bj;{d!^?nIAJGY70)qy z6-hMMoK^w*P@Y(x4%5kg;GOs^8?ck#c=GJl{!SZrl7)WP~gbJ%&pvqG@k16rA| z32pT$I-Xs$lo-rZ6Yn3b-Y&}NUoPqHU{%v_^r#B{P(-Yc#D|h&Tjt&*DA{B(BqS?6 zx*kr`@$$!lly;vkX%8!{>?WM-B_vr=AZ?0Ns^fA7yjC@BS6{!*Hcb379&@}Qa#9fD zZtZlq*Xf;I9u6hjF_?!LO=~X_4&SeEnZbKZE#CKVa^FxanYrTZ1r94Y?Pc}nD=|u8 zyw&LDSw$-Ikk>C1r&1q%muj?Cm@lu9S+K^6Y{souF;ipsb2lE50##7+mg-N~5J`)S zy?n7tjQa0~$jhq8=B&IgV16$VWA`a5CkLVm%`yAULe6^eA;qW=Th_`S^~ssECeaN^ zV>#YOi|MX$fF)}WpV^&_j7pWFwi(C&H``7s7B66;6Xl4yK=KhBxgk-<#(DTgc{L(WbH9So}5{3j`SKx-VF_f zXxNN(%P;p@X=c4mH~R|Pns(^wgXMD*W4rj#A~E2BoGo8?xWfh-UK9Azc-Y(lh{3YZ z+X}LS3+~xGXW0@RO-V?tVSmiFIjL# zD%FSvE){CK%3X45Sa?Tm#360*ip+0GY2wch=N*D_uBng%J7CY{CezSxiQesfsG|MK0c9++S|1*WNJd zcL{-JS8r`rO`{jpcoyf2w#SK@)M*M*8_24?n=YEX7up#ua1mMg{FiNAM_5rxz`hz+ zSRZR_jj@k^Z)^clM?11CbmE$A#N!F;qE@}wnr(`zp>cx{w)-jqk5Y;OFVJ)$d_O12F zRc|^wy|xR|XA0zP`qva*7mc0(y{!<;9B`z4#-g zJC$|`_MfT%WR)_u+ARZVhh#eeogUvflUX0C3P5rxBXYbJkul5#c{q9~POk33@Id-- z^g)&OCNLpu07OK5g>=RgqsU5n+97i^dO%`Nl;)ikSw=aM7{BuwtWfvP9cQuv$*WLyB zLm676*eB}aA3=_n@|mAW{J2std9t5#qERvyI`~@oIV)S)*XMgK`RRqpa4Tnmh_UQc7GI@`576e|qDWJ0c$xn*Wt z0Zi8lFoyX9fn{_{!w)Z-i4DSX%Uh_}TDJDC=l42ulmpz-L{_ZNN`HE4)i1 zqK$D8L=DSzf{r-{*=J(5xUu2!2S`>3MFl}m;?5mani7fH#H5&_MCsk9Af;SIJ-66- z9hmi`8f>BC&XYsAlCd)ZG)s^^YliY?CP`aXC6|Z|!f>Yj49h%CB1CfrEE_xtrL!uX z2t%{9_sRUjy4;N>2&+TZcr@Oesh1oq(=u{{Ius6xr$v>(s}xlZXAA)72}89+{8fZ( zU_5462)hDt-=0xmc;=^_#f1{ylK6)~=jDk;k(0g7t-!cdXv?W#3OzEIh7%ylJTt;` z5J+QDaDVt;)_?>XyS^A@jOo8o_wfD1^}IGh8nZ%++`3Z|yH}WVi<73!lrmYd0uk0r zQk?SKf%(s@2AUIED=gkD1?^1UYwY{<>>TtA6JDtmb#5919`mZx|;XS zn=7p<$O6PAV^ji9#rlS;r`>zf(j8poi<*}ph@CCjKLUf$c`1nEzK4my;aQY_kZT8rA=?jnTh*-DyY4uSY-K`;6P+mRK}y9Ah}@-_&3bk3TbOqA3djIF9rI=5YG zcHn2ngC-p6g(10f=j8;uleIZ-u!A{FO2BqN2sa@rCr~FTQT8%^+=lS4I(-a){o&-* ziNkNau8or*7b@HYD#*#TW;{IYawSN)l(&#t`2ll{biDeena?rmUO+{vYMvfiHQ5xx+5Kj$Q z4#dPE(1}{mwBu5kzR?IoKsQe5&u~&C)UD6ufl+=E9|c5plgqaR@8kQM6H9iW4I)`^_FBfo-2Q3O$+KiT^Tbd2yiE$uTaj)wW7M* z(hi}3PzZm%^dK2GB7_JQHW%*?yO}kMZlH55di}yoQTH_=OnoSeReA%*b_G`p!;?oD z2#&GAS4;ndSHqudc+bn|nz`KekZdsw>(o)IT{eA!^9%{-ZM6iyRHm9WU5jopOnPqd zj`^AA>H4NuN$K{k?8sfp8t06ID4iIGn{C6w_r8==EBCVxH|;n@2yHX|Bs3*193UtT z!eVzm>Or{S`G&J1tO@B;i+W@3WbwJ*!rr=2ngOw zxiq20s_BQSi8Hq(1`%zXnc$6;Qp1S;1Oz|sfwLiW9JS+f$lAVuI)a_|Mf0BU-q-Z>+{l@Sg;dH+KyLo%p^Ff_pVKPXrtWH?B(=M}O0c zkT%2NG4qGDRdw^|w2x#Fl^ovq)1`J>2t2N$Or~uZ#w|@)m|gO5rjLcCJF0VRfMyA_ zDB7VPMpY}V@)}8!U)&S;;4q-=9Ek`OR2jo)#vr>A)1ER_3O#BeKj+Kyy8JsfY64$ky?*y|B1+ZKEQ5dpzjqMl4CT^6|WZ*&PHiQ>A>z;Nztz4 zy&yKYOR2~ouNIN(>LJ$wk?-LK`A)|!Me{c3+?2Mk7`wed8OhY7W^MHU+Pm_2D!1;R zW1i_|h>Fs!2vI0wq$n~KGG(e{Ov)6YQ&d+;@6AvlDpTfp$e2pE5E+lT5|N=yhlBX7 z?Oyku=ji9Y@Bj7Jy*|(LoVE8}d+jxR*LTlM48#?vht`feFA$w4?@xL~3p%F1UWW}s zX>1|^(|R-8eGAs*LYX=u&-8$fpNYT2rgenGa}L!LP+f6bL`t=VtIBeA-H@mUhKi_T zq!dQllEHZdpB-RX=w+YXge>J zeSEF%jig980yV=kD|4?4jsr^8Bv2#~m=cohe;MHn8?yUNqu_>U*1@Js$x2kQDvOVv z?f9$#;5K)IU*CE90Yo!7Nva6CjE_xb0{STwk}w7JvY#W5&O8^G*A4t;c#4a_&?KHm zuzo2C9#`eu83cnlm5}AtJURkMGzKzp~Y@N1i`@MA@kGtmkB_4k@QrXsD0%pF~ zC;pve=WuYXX0LP?U|gO$m?yD5GThMKJ~`dx0|^rG1J{dN$y(R|2Ob6TItFSt5(U~IJ93ypM1@$LvTAvr>zYxjqUIwY37gL zOmYu3IO)wax8H!E=F@B_7gXvCa=6R-DXf;q9pnC`2`(SNgw$>? zQ&ZlNbP1iU1|L%PY}xl%FE{5Lsrs~1LtH1MM@(8{bp}BJ9^Cv=VzDEJ+X~%V3U1#dhWwo;Mz{j0joH*rnRk-wZRhxpzeWq7oGK7qNe1 zBE{V^&G{O7ZgOvr8;d>da4uFe=uoRt?(*h~dYtfGbwG|N)$H{ZT+1(=r@yRkv9;I^ z7X6aq`%BbbEWyqE;{bH%F$3mgN4q;(GQy=>L!$@121lHtZ-dpD*H_V=0ioYfvl+_K60B_#l*2C6GHLBc0ceaG0JX|lAyn|{c=c-$RXRCVv zyODI?VHbaIxr_cLu_FCjyG8m@wImMcL>^grphI^6!i4QjhP1bQH?3Ry?2YGP&kKcc z+T6Hw-tS6AuM$w`&D0}sPLh3g?}Zm`w{D-Mu@p*djrg^qVqrpOLH{Cmm5*9)hqLc^ z#PQ3`xjtnOKF6K7B^y67wWT6`bb89u(aWPW)Q1SG>vIu!(&G|YOux#hzrP45sJ8;W z=1La+0aY76djSG8j4#h>b$8G7nW>!p41Q2ielm7b)-9io&CZicPv;Q`G_*-jgCU4&am}8-AU3{IA$J2i>5~WI5QPaFYNoK z3Qxbm$b3>`S@a7{3hVJH_9x!pj*#Syx&D}dG;;#Lv)oLYyF&v6juH<}NVS-qk2}C^ zyjcwFtZ=gXu`-vME>S`lIi!X?EG$Iftk_J(UB#%gM0 z+C`81Og8oAdtDr^ElGIwHo{S2P1=b&s!fh`00L+Z`Tk*_@51ZLA4p#tA#osm%8Fnf zv}u+auRs6QY}9)v^X&xDi`~geeA3+hPMc3G!v6n>V0dwWa8-T3i5G{hLqY;c*_077 zQ}n{g#C!CP*6)Gg*)Q;0jjmCax@7{mEy4nCAQejrF&K69GLlRORIUVgCEg@S;A-$~ z19$bllDR65#*2Fopv4-@y@x}0-xR#)f6F`Dv6R$zU>c!jbzrMmJF~Tfl^@>D;kRw| z#_{PZ*Ni@TyTj1?2B}BcRS0<9MtZDhyqghr?s5}FhQ+;~YJ@R8qq=H(iE zwC|+8yVK-32z26x^y)|whNe!r2uv)c7&|1{cHo8y?~K$|+b!j|VW15Fj~r zB7pv0F9dEuko(|=3kPp?VSSZ5C)__Om!yBN=P;G`Am6uI=rUZ2h z(u-RPV}H0pu7tIJGp!YU$kEeS`)z57%zE=tXY8NRevcA53skTPo`)}pfAL4_!L<=JeJcZ?|i=o(TgX?P~p#lsN$^}3Ktf{sTI^I;Qs7!`0$ zgh#HKdp0b!(fuh35!!g1f(K8AU_`;1pHxnGCa5SII~VwP2*4w(>nWhB>tmTbif!BG9S!kf$pLqYE*LAZ>e( z+}d9;qL52_N!bAwrl_8R28-1`G|J7ILemK{VP7HF#-!v4@X=$WpA|oUb*emz=-naq4w8g{@kKJ zx9E~${oJBIx9E~__<4%{JVpOHseYcKKTpvm59_C+`~UJS{5(Y$-PWIuE?Jn8qh_gm zB?F4e=it&qFQq%EuL*j3qU~!^A}KdiRRR{IoEaO15k{4$~_F}3S;_2DZ^d+!H< zgYp6`ofLyNWb5Jpz`9$Y32L+tfYP94f%6L<`<`n?IpBY8BjB8wC4fv00Zir?y~ABO zY_PF({{I<<+T*EMD;_|;$L>vtFlnu!kySv>OKwyB=o)T{*Xp+8lhdAxC_=7~B#GEm zP@|%kTX7}XZ2S|L{q-x5K7sF>LzZp;r5Q@X%+9^e0gjlOYWF@0lrY`#_rGh3A3}{~ zz7L>XQfe5o%2#}~w_*gQ6iOUN&F7EciVaYz87~Zm!0Ydc0epqF;C02CyU6 z?MxKIt}9l4qOXB4E`ii24Th*RmOn)xcsg^}xE6LwfQ)!mj^UvZLmA;23{XfpTjas5tz?uS08<_fu$D52OYFRB{evNdGI@ zExblwy1skK!xN7>QcMu?!D&0iP?g7VaKVKBOndjd;ku0H#P)=j;YN=Ld6dXv@|_ja zfNa7n+?5fxckTmwzyS0Q+)<_;QX`6UZu%jZLOX=1$x9{w9HrIicSU=a zt`ERpq0oF9)Zqq%1={V7*;)zw5fsIV{yXKT!jFfcCR71ncD|ki<+G_L0`x27!#M65 z)6yw0cmr2`EEE?uP2GoutB2}?bavZ49;{aK>f;-<<;4&YB}sP4HnDsmxoT4iO=cuw zI%(khv#9qC=DqMN>PKn-euU$WrW75-Q2OvhWCjR*DK*!apcD$sNKq?m4i!$A)sur5 z7jCY>bKF(W3Qz;>@d$Lae9b0AzX47;L32k*j&1*f1<#rl^?ePFdV@de9F0HUnWcgzlO_N~YvZ9`ZShk<6?;b4z!(nt)J2I9ES0k?zq()Lb0RC=KGTJX$BV00|d(=sM9ncIwj_r zV7I7Im7Q2JA6_Fo2rCbSufKk66u8HhlG?Sx{3$Lu6G5(Xo6VlS{k`zre*CA~?jnqxiNsfO;!DQB45;@9gsTY(>Al z*`6K?RmC#fjQ@YJF*~Y!fr&+fg9!B^&4dSD_ErjDOml!E?FSpQ5~@aS#{#3rvcNA$ z0C-PP@c@P6fcf+)ydML6TV@7mSpPiq8lW3u@z7h?6URew*u=>HJ?G9@e;som?!DC$$6ghe9 z0xJiU$Ix1JL%)$LpzGoh-$X(?Tr1uHG;9Vg_Q@1dDW{eLCsydktzmGKioFyFLzTbW z8b|EGh_x7kw;5SXIAmN=iywkPey&4gJ8f3i){nU@U{nC!`@$Rv4TA$v^?2QFW);9y z&u!|cSN5=kLJq9zvz*}|{0abEw?Q5I+oy24FKp^+}#n%WOl&2 z@`J66`6XR3iA1hN9;?OaL3O#kgKGRvA8#|f(j(C{8J&2h|wgS=o zDUIK&b(Oi^y3z{tt60`u$w0(AGe8w&!XlK^dh>_c6FUJtr?F6KPmUqL6?BMz%VPON zVl*fwxjcs6DnTdu0qR#r;CpC_*94v-d!-)f zW+NdV@Nq_BS|dcTQ;AVMd?{n~ZNZ_~LwD-o^fbJAwxhts4l1)dD2+H?o7)pH=O7vk z4o6rc#3=QzYGh$~mBy;45p}S5L7Uu|ULup}f;9olg%>5PYs?CrCE;4ArRem&yUQsI z{?U8rz^%>z9vMp^7E^Qm05MHZOw|O$OiTs#UIN2&sN`aXik*diW6P@mV%wHfLAqBBIO}V4GxDG50U7DEm8A*z z?8aX(@?z!!7$K^^2|DT~Gqo2A?Ae8B;e+!?m$i8-MMQ^veSy{l=;>0KfQp_V1y@4P z*Q&8&)i{QgW~6Lk-W4#>+(MlIjc$NY$n@X#lq^1s#_EJ^sDafrR{k*XDvEM|BSw0+ zeJc6bp^hEFbohaS?D2E2f3*WRD2Zj$Dt;9s3eE(~q*CEZcx(Z1ktedKI{XhQpT4`6 zcGDam(wi`zCd;bTKkPGJ`s9Rp%qbND?-vL=!vI}SYTXD|9cdtZ>#}Wg@6daQVYI-4 z`bjR*?gg#g6~Ej;!_Mzl8WTSM)MJdN0g|bMCnFb00q{)Zp0b}pw(?0kAOwsYA$AYz@w-lDDq4VXacf;h%^>jt@8iP z(@a%4whL55va#Y{-t`$Tx45yJfM&EBAt6#7Y;~AgcV%&)Y3}+C72Hn?@kp0 zAZEn;4u7H-xOltP2COsb#3mk7$IQ~&w-nuzF#$NZJwiL&?A)QnTuUH8b9bS+L@cKE z9}*+)VjKKpTMlxGgmvcCr{aZ$2*% z@)%j-a{#A?Nb-4RQ?QCbVX^A`@EdvjgM<;k>A={~=zkFWN%A@3sBbO+_kz&+#rf9exS|JUWO)ABblrylrhM5|MKvoM8<28xGeTxyO!7#U$4l5O} z2o4mmdLbkGoB{yE3{Xq4h=(!8hGO3U5~xcwR~dZM>95QM3a9!vFAPrmg?!OcP=S>0^d zC8yZ9bUHD^V1YQfJl-nJ4k73D*y|;qF0Jh}MHZMPq&%su z(()<;y2xJiJT#!i0^AO@QlH6sy~Pdye#y&zWa8NSrQD#wZn$^sFzTOZQsT!RO=K<< z+GEvw@Of+%bPXsrThRYK2&ne}sO`0EX7NhQ9ne)e0Nqq6a=oPyOrHjqrKw z6?83#Qyb9_iWEY|RxyZwciQqTWr~MTUq%s#77eZi@kxsWt(VVR_7ue`*c6u^=JaRH zgAF4GX+i^o%bKE@`vbpZpvK51Uy7l}Y{kF6vla0K??O?Kffgch%K@HD!+)_20BN$h z4;Qt)^ouM8reOpJHLyjrBNuqxq0d?vp(Zi&+R4}f(cA?F;zGWMv!a$174Ic_+!`+S zkYDlsIU829zk9L0cyyurkI>&LRjvIsaKF)}= zmsDxBdr_x(eqmwZw>i?dZhDKilgNr|I#-5^W2#Vl)7jaXLHu}tU@J9q(jnJ`>4lAP zbIXW7c1M1~)!6K!wd60Gi%$_)h&S^gis~mf6(Kh|Uw!WQ z!gZ-k&$P9*iMjF=E00~6_vaWxlo%q;r>CbUEOc~qcGg7HXPT#913|BPPGDZek8%(Q z1P15sQr1zUIaq=T^*i~_-QqIIE1I~ohiEIe0O=Py8(aNI229RcBQ`vNYBejlAlstK z?WhQQ-obtFamOuBpFXX*+xZvD1O)k9=^ElQnJW5jOv8Ob-@Y^@*E!0J$9L%o=BYm< zH_Z5YhDfeUHCdtx7sfVsl=+oOjZ3dsA?ev95e7G+xb(cu$YglYx9Ju}%JA~9xC|E_ zg3(2CS+F|`Uf|WjXR=lFX>Q?&(A+~FU&gG%;WB!S)UzqhUGg6h{Dv3^+({&!valgnoj&mIV2|bnVY0GmkXUT4s-X*r5;)pM7PqJAhcbO!Mmb9itBn% zXS!M8@3_oKFU5`$?E^LU;Gx(e1RBIw#*9#OyGP(8@7u z0I_;n^aRTbXoZ(_Jagtjn}fZ*NTYr3n(JJFilMJ*$t5$ptISeT*=;{c7;x2Izq_$QwUoC%x!8@&MfEs4p6VGXsH*g89Z zy{sm~g0U>3qN1_8onXInRK0n_w1#{cmwWa!Dul9$;P>y}3ra=Qn8N|Vm})qHYbU0rrWyv7 zZ=}=K4374zk}V%TixK9_Un6ey*ojQv$uCW7e1qi6DhHKTTmcSVfO)&~#`$%8?(g6B z=^+7B$JbSEVVd#cdh+R*H?y?&lB2a4FCTpI;zcyOJH-Nn2@SERLQDHXtpF!OG%L%;`H={q<3~W3m5}e6(Poa93AKIg-!VqNyzJ z2k7vvw>-1~4s^D%SF?XnG+IbA=gKKb@)|yi5pky{8zkvI+p6FFjnboxin{qfI|GFtyjlhS4js_mpSI8P^8W$8I2+Rd literal 0 HcmV?d00001 diff --git a/flink-connector-pinot/pom.xml b/flink-connector-pinot/pom.xml new file mode 100644 index 00000000..74eacd52 --- /dev/null +++ b/flink-connector-pinot/pom.xml @@ -0,0 +1,196 @@ + + + + + 4.0.0 + + + org.apache.bahir + bahir-flink-parent_2.11 + 1.1-SNAPSHOT + .. + + + flink-connector-pinot_2.11 + flink-connector-pinot + + + jar + + + + 0.6.0 + + + + + org.apache.flink + flink-scala_${scala.binary.version} + ${flink.version} + + + + + org.apache.pinot + pinot-tools + ${pinot.version} + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + + + + org.apache.httpcomponents + httpmime + 4.5.13 + + + + + org.apache.pinot + pinot-java-client + ${pinot.version} + test + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${flink.version} + test + + + + org.apache.flink + flink-tests + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${flink.version} + test-jar + test + + + + org.testcontainers + testcontainers + 1.15.2 + test + + + org.testcontainers + junit-jupiter + 1.15.2 + test + + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + + + enforce-versions + + enforce + + + + + ${java.version} + + + + + + true + + + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + 3.1.1 + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.0 + + + package + + shade + + + + + reference.conf + + + + + + + + + + diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerClient.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerClient.java new file mode 100644 index 00000000..43aafab4 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerClient.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; + +/** + * Helpers to interact with the Pinot controller via its public API. + */ +@Internal +public class PinotControllerClient implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(PinotControllerClient.class); + private final PinotControllerHttpClient httpClient; + + /** + * @param controllerHost Pinot controller's host + * @param controllerPort Pinot controller's port + */ + public PinotControllerClient(String controllerHost, String controllerPort) { + this.httpClient = new PinotControllerHttpClient(controllerHost, controllerPort); + } + + /** + * Checks whether the provided segment name is registered with the given table. + * + * @param tableName Target table's name + * @param segmentName Segment name to check + * @return True if segment with the provided name exists + * @throws IOException + */ + public boolean tableHasSegment(String tableName, String segmentName) throws IOException { + PinotControllerHttpClient.ApiResponse res = httpClient.get(String.format("/tables/%s/%s/metadata", tableName, segmentName)); + + if (res.statusLine.getStatusCode() == 200) { + // A segment named `segmentName` exists within the table named `tableName` + return true; + } + if (res.statusLine.getStatusCode() == 404) { + // There is no such segment named `segmentName` within the table named `tableName` + // (or the table named `tableName` does not exist) + return false; + } + + // Received an unexpected status code + throw new PinotControllerApiException(res.responseBody); + } + + /** + * Deletes a segment from a table. + * + * @param tableName Target table's name + * @param segmentName Identifies the segment to delete + * @throws IOException + */ + public void deleteSegment(String tableName, String segmentName) throws IOException { + PinotControllerHttpClient.ApiResponse res = httpClient.delete(String.format("/tables/%s/%s", tableName, segmentName)); + + if (res.statusLine.getStatusCode() != 200) { + LOG.error("Could not delete segment {} from table {}. Pinot controller returned: {}", tableName, segmentName, res.responseBody); + throw new PinotControllerApiException(res.responseBody); + } + } + + /** + * Fetches a Pinot table's schema via the Pinot controller API. + * + * @param tableName Target table's name + * @return Pinot table schema + * @throws IOException + */ + public Schema getSchema(String tableName) throws IOException { + Schema schema; + PinotControllerHttpClient.ApiResponse res = httpClient.get(String.format("/tables/%s/schema", tableName)); + LOG.debug("Get schema request for table {} returned {}", tableName, res.responseBody); + + if (res.statusLine.getStatusCode() != 200) { + throw new PinotControllerApiException(res.responseBody); + } + + try { + schema = JsonUtils.stringToObject(res.responseBody, Schema.class); + } catch (IOException e) { + throw new PinotControllerApiException("Caught exception while reading schema from Pinot Controller's response: " + res.responseBody); + } + LOG.debug("Retrieved schema: {}", schema.toSingleLineJsonString()); + return schema; + } + + /** + * Fetches a Pinot table's configuration via the Pinot controller API. + * + * @param tableName Target table's name + * @return Pinot table configuration + * @throws IOException + */ + public TableConfig getTableConfig(String tableName) throws IOException { + TableConfig tableConfig; + PinotControllerHttpClient.ApiResponse res = httpClient.get(String.format("/tables/%s", tableName)); + LOG.debug("Get table config request for table {} returned {}", tableName, res.responseBody); + + try { + String tableConfigAsJson = JsonUtils.stringToJsonNode(res.responseBody).get("OFFLINE").toString(); + tableConfig = JsonUtils.stringToObject(tableConfigAsJson, TableConfig.class); + } catch (IOException e) { + throw new PinotControllerApiException("Caught exception while reading table config from Pinot Controller's response: " + res.responseBody); + } + LOG.debug("Retrieved table config: {}", tableConfig.toJsonString()); + return tableConfig; + } + + + @Override + public void close() throws IOException { + httpClient.close(); + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java new file mode 100644 index 00000000..6ac05d62 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotControllerHttpClient.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot; + +import org.apache.flink.annotation.Internal; +import org.apache.http.StatusLine; +import org.apache.http.client.methods.*; +import org.apache.http.entity.ContentType; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Helpers to interact with the Pinot controller via its public API. + */ +@Internal +public class PinotControllerHttpClient implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(PinotControllerHttpClient.class); + private final String controllerHostPort; + private final CloseableHttpClient httpClient; + + /** + * @param controllerHost Pinot controller's host + * @param controllerPort Pinot controller's port + */ + public PinotControllerHttpClient(String controllerHost, String controllerPort) { + checkNotNull(controllerHost); + checkNotNull(controllerPort); + this.controllerHostPort = String.format("http://%s:%s", controllerHost, controllerPort); + this.httpClient = HttpClients.createDefault(); + } + + /** + * Issues a request to the Pinot controller API. + * + * @param request Request to issue + * @return Api response + * @throws IOException + */ + private ApiResponse execute(HttpRequestBase request) throws IOException { + ApiResponse result; + + try (CloseableHttpResponse response = httpClient.execute(request)) { + String body = EntityUtils.toString(response.getEntity()); + result = new ApiResponse(response.getStatusLine(), body); + } + + return result; + } + + /** + * Issues a POST request to the Pinot controller API. + * + * @param path Path to POST to + * @param body Request's body + * @return API response + * @throws IOException + */ + ApiResponse post(String path, String body) throws IOException { + HttpPost httppost = new HttpPost(this.controllerHostPort + path); + httppost.setEntity(new StringEntity(body, ContentType.APPLICATION_JSON)); + LOG.debug("Posting string entity {} to {}", body, path); + return this.execute(httppost); + } + + /** + * Issues a GET request to the Pinot controller API. + * + * @param path Path to GET from + * @return API response + * @throws IOException + */ + ApiResponse get(String path) throws IOException { + HttpGet httpget = new HttpGet(this.controllerHostPort + path); + LOG.debug("Sending GET request to {}", path); + return this.execute(httpget); + } + + /** + * Issues a DELETE request to the Pinot controller API. + * + * @param path Path to issue DELETE request to + * @return API response + * @throws IOException + */ + ApiResponse delete(String path) throws IOException { + HttpDelete httpdelete = new HttpDelete(this.controllerHostPort + path); + LOG.debug("Sending DELETE request to {}", path); + return this.execute(httpdelete); + } + + @Override + public void close() throws IOException { + httpClient.close(); + } + + /** + * Helper class for wrapping Pinot controller API responses. + */ + static class ApiResponse { + public final StatusLine statusLine; + public final String responseBody; + + ApiResponse(StatusLine statusLine, String responseBody) { + this.statusLine = statusLine; + this.responseBody = responseBody; + } + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotSink.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotSink.java new file mode 100644 index 00000000..7d2fe94d --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/PinotSink.java @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot; + +import org.apache.flink.api.connector.sink.Committer; +import org.apache.flink.api.connector.sink.GlobalCommitter; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable; +import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable; +import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter; +import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor; +import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer; +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter; +import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator; +import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator; +import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkCommittableSerializer; +import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkGlobalCommittableSerializer; +import org.apache.flink.streaming.connectors.pinot.serializer.PinotSinkWriterStateSerializer; +import org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter; +import org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriterState; +import org.apache.flink.streaming.connectors.pinot.writer.PinotWriterSegment; +import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver; +import org.apache.pinot.core.segment.name.SegmentNameGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.tools.admin.command.UploadSegmentCommand; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Apache Pinot sink that stores objects from upstream Flink tasks in a Apache Pinot table. The sink + * can be operated in {@code RuntimeExecutionMode.STREAMING} or {@code RuntimeExecutionMode.BATCH} + * mode. But ensure to enable checkpointing when using in streaming mode. + * + *

We advise you to use the provided {@link PinotSink.Builder} to build and configure the + * PinotSink. All the communication with the Pinot cluster's table is managed via the Pinot + * controller. Thus you need to provide its host and port as well as the target Pinot table. + * The {@link TableConfig} and {@link Schema} is automatically retrieved via the Pinot controller API + * and therefore does not need to be provided. + * + *

Whenever an element is received by the sink it gets stored in a {@link PinotWriterSegment}. A + * {@link PinotWriterSegment} represents exactly one segment that will be pushed to the Pinot + * cluster later on. Its size is determined by the customizable {@code maxRowsPerSegment} parameter. + * Please note that the maximum segment size that can be handled by this sink is limited by the + * lower bound of memory available at each subTask. + * Each subTask holds a list of {@link PinotWriterSegment}s of which at most one is active. An + * active {@link PinotWriterSegment} is capable of accepting at least one more element. If a + * {@link PinotWriterSegment} switches from active to inactive it flushes its + * {@code maxRowsPerSegment} elements to disk. The data file is stored in the local filesystem's + * temporary directory and contains serialized elements. We use the {@link JsonSerializer} to + * serialize elements to JSON. + * + *

On checkpointing all not in-progress {@link PinotWriterSegment}s are transformed into + * committables. As the data files need to be shared across nodes, the sink requires access to a + * shared filesystem. We use the {@link FileSystemAdapter} for that purpose. + * A {@link FileSystemAdapter} is capable of copying a file from the local to the shared filesystem + * and vice-versa. A {@link PinotSinkCommittable} contains a reference to a data file on the shared + * filesystem as well as the minimum and maximum timestamp contained in the data file. A timestamp - + * usually the event time - is extracted from each received element via {@link EventTimeExtractor}. + * The timestamps are later on required to follow the guideline for naming Pinot segments. + * An eventually existent in-progress {@link PinotWriterSegment}'s state is saved in the snapshot + * taken when checkpointing. This ensures that the at-most-once delivery guarantee can be fulfilled + * when recovering from failures. + * + *

We use the {@link PinotSinkGlobalCommitter} to collect all created + * {@link PinotSinkCommittable}s, create segments from the referenced data files and finally push them + * to the Pinot table. Therefore, the minimum and maximum timestamp of all + * {@link PinotSinkCommittable} is determined. The segment names are then generated using the + * {@link PinotSinkSegmentNameGenerator} which gets the minimum and maximum timestamp as input. + * The segment generation starts with downloading the referenced data file from the shared file system + * using the provided {@link FileSystemAdapter}. Once this is was completed, we use Pinot's + * {@link SegmentIndexCreationDriver} to generate the final segment. Each segment is thereby stored + * in a temporary directory on the local filesystem. Next, the segment is uploaded to the Pinot + * controller using Pinot's {@link UploadSegmentCommand}. + * + *

To ensure that possible failures are handled accordingly each segment name is checked for + * existence within the Pinot cluster before uploading a segment. In case a segment name already + * exists, i.e. if the last commit failed partially with some segments already been uploaded, the + * existing segment is deleted first. When the elements since the last checkpoint are replayed the + * minimum and maximum timestamp of all received elements will be the same. Thus the same set of + * segment names is generated and we can delete previous segments by checking for segment name + * presence. Note: The {@link PinotSinkSegmentNameGenerator} must be deterministic. We also provide + * a {@link SimpleSegmentNameGenerator} which is a simple but for most users suitable segment name + * generator. + * + *

Please note that we use the {@link GlobalCommitter} to ensure consistent segment naming. This + * comes with performance limitations as a {@link GlobalCommitter} always runs at a parallelism of 1 + * which results in a clear bottleneck at the {@link PinotSinkGlobalCommitter} that does all the + * computational intensive work (i.e. generating and uploading segments). In order to overcome this + * issue we introduce a custom multithreading approach within the {@link PinotSinkGlobalCommitter} + * to parallelize the segment creation and upload process. + * + * @param Type of incoming elements + */ +public class PinotSink implements Sink { + + private final String pinotControllerHost; + private final String pinotControllerPort; + private final String tableName; + private final int maxRowsPerSegment; + private final String tempDirPrefix; + private final JsonSerializer jsonSerializer; + private final SegmentNameGenerator segmentNameGenerator; + private final FileSystemAdapter fsAdapter; + private final EventTimeExtractor eventTimeExtractor; + private final int numCommitThreads; + + /** + * @param pinotControllerHost Host of the Pinot controller + * @param pinotControllerPort Port of the Pinot controller + * @param tableName Target table's name + * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment + * @param tempDirPrefix Prefix for temp directories used + * @param jsonSerializer Serializer used to convert elements to JSON + * @param eventTimeExtractor Defines the way event times are extracted from received objects + * @param segmentNameGenerator Pinot segment name generator + * @param fsAdapter Filesystem adapter used to save files for sharing files across nodes + * @param numCommitThreads Number of threads used in the {@link PinotSinkGlobalCommitter} for committing segments + */ + private PinotSink(String pinotControllerHost, String pinotControllerPort, String tableName, + int maxRowsPerSegment, String tempDirPrefix, JsonSerializer jsonSerializer, + EventTimeExtractor eventTimeExtractor, + SegmentNameGenerator segmentNameGenerator, FileSystemAdapter fsAdapter, + int numCommitThreads) { + this.pinotControllerHost = checkNotNull(pinotControllerHost); + this.pinotControllerPort = checkNotNull(pinotControllerPort); + this.tableName = checkNotNull(tableName); + + checkArgument(maxRowsPerSegment > 0); + this.maxRowsPerSegment = maxRowsPerSegment; + this.tempDirPrefix = checkNotNull(tempDirPrefix); + this.jsonSerializer = checkNotNull(jsonSerializer); + this.eventTimeExtractor = checkNotNull(eventTimeExtractor); + this.segmentNameGenerator = checkNotNull(segmentNameGenerator); + this.fsAdapter = checkNotNull(fsAdapter); + checkArgument(numCommitThreads > 0); + this.numCommitThreads = numCommitThreads; + } + + /** + * Creates a Pinot sink writer. + * + * @param context InitContext + * @param states State extracted from snapshot. This list must not have a size larger than 1 + */ + @Override + public PinotSinkWriter createWriter(InitContext context, List states) { + PinotSinkWriter writer = new PinotSinkWriter<>( + context.getSubtaskId(), maxRowsPerSegment, eventTimeExtractor, + jsonSerializer, fsAdapter + ); + + if (states.size() == 1) { + writer.initializeState(states.get(0)); + } else if (states.size() > 1) { + throw new IllegalStateException("Did not expected more than one element in states."); + } + return writer; + } + + /** + * The PinotSink does not use a committer. Instead a global committer is used + * + * @return Empty Optional + */ + @Override + public Optional> createCommitter() { + return Optional.empty(); + } + + /** + * Creates the global committer. + */ + @Override + public Optional> createGlobalCommitter() throws IOException { + String timeColumnName = eventTimeExtractor.getTimeColumn(); + TimeUnit segmentTimeUnit = eventTimeExtractor.getSegmentTimeUnit(); + PinotSinkGlobalCommitter committer = new PinotSinkGlobalCommitter( + pinotControllerHost, pinotControllerPort, tableName, segmentNameGenerator, + tempDirPrefix, fsAdapter, timeColumnName, segmentTimeUnit, numCommitThreads + ); + return Optional.of(committer); + } + + /** + * Creates the committables' serializer. + */ + @Override + public Optional> getCommittableSerializer() { + return Optional.of(new PinotSinkCommittableSerializer()); + } + + /** + * Creates the global committables' serializer. + */ + @Override + public Optional> getGlobalCommittableSerializer() { + return Optional.of(new PinotSinkGlobalCommittableSerializer()); + } + + /** + * The PinotSink does not use writer states. + * + * @return Empty Optional + */ + @Override + public Optional> getWriterStateSerializer() { + return Optional.of(new PinotSinkWriterStateSerializer()); + } + + /** + * Builder for configuring a {@link PinotSink}. This is the recommended public API. + * + * @param Type of incoming elements + */ + public static class Builder { + + static final int DEFAULT_COMMIT_THREADS = 4; + + String pinotControllerHost; + String pinotControllerPort; + String tableName; + int maxRowsPerSegment; + String tempDirPrefix = "flink-connector-pinot"; + JsonSerializer jsonSerializer; + EventTimeExtractor eventTimeExtractor; + SegmentNameGenerator segmentNameGenerator; + FileSystemAdapter fsAdapter; + int numCommitThreads = DEFAULT_COMMIT_THREADS; + + /** + * Defines the basic connection parameters. + * + * @param pinotControllerHost Host of the Pinot controller + * @param pinotControllerPort Port of the Pinot controller + * @param tableName Target table's name + */ + public Builder(String pinotControllerHost, String pinotControllerPort, String tableName) { + this.pinotControllerHost = pinotControllerHost; + this.pinotControllerPort = pinotControllerPort; + this.tableName = tableName; + } + + /** + * Defines the serializer used to serialize elements to JSON format. + * + * @param jsonSerializer JsonSerializer + * @return Builder + */ + public Builder withJsonSerializer(JsonSerializer jsonSerializer) { + this.jsonSerializer = jsonSerializer; + return this; + } + + /** + * Defines the EventTimeExtractor used to extract event times from received objects. + * + * @param eventTimeExtractor EventTimeExtractor + * @return Builder + */ + public Builder withEventTimeExtractor(EventTimeExtractor eventTimeExtractor) { + this.eventTimeExtractor = eventTimeExtractor; + return this; + } + + /** + * Defines the SegmentNameGenerator used to generate names for the segments pushed to Pinot. + * + * @param segmentNameGenerator SegmentNameGenerator + * @return Builder + */ + public Builder withSegmentNameGenerator(SegmentNameGenerator segmentNameGenerator) { + this.segmentNameGenerator = segmentNameGenerator; + return this; + } + + /** + * Defines a basic segment name generator which will be used to generate names for the + * segments pushed to Pinot. + * + * @param segmentNamePostfix Postfix which will be appended to the segment name to identify + * segments coming from this Flink sink + * @return Builder + */ + public Builder withSimpleSegmentNameGenerator(String segmentNamePostfix) { + return withSegmentNameGenerator(new SimpleSegmentNameGenerator(tableName, segmentNamePostfix)); + } + + /** + * Defines the FileSystemAdapter used share data files between the {@link PinotSinkWriter} and + * the {@link PinotSinkGlobalCommitter}. + * + * @param fsAdapter Adapter for interacting with the shared file system + * @return Builder + */ + public Builder withFileSystemAdapter(FileSystemAdapter fsAdapter) { + this.fsAdapter = fsAdapter; + return this; + } + + /** + * Defines the segment size via the maximum number of elements per segment. + * + * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment + * @return Builder + */ + public Builder withMaxRowsPerSegment(int maxRowsPerSegment) { + this.maxRowsPerSegment = maxRowsPerSegment; + return this; + } + + /** + * Defines the path prefix for the files created in a node's local filesystem. + * + * @param tempDirPrefix Prefix for temp directories used + * @return Builder + */ + public Builder withTempDirectoryPrefix(String tempDirPrefix) { + this.tempDirPrefix = tempDirPrefix; + return this; + } + + /** + * Defines the number of threads that shall be used to commit segments in the {@link PinotSinkGlobalCommitter}. + * + * @param numCommitThreads Number of threads + * @return Builder + */ + public Builder withNumCommitThreads(int numCommitThreads) { + this.numCommitThreads = numCommitThreads; + return this; + } + + /** + * Finally builds the {@link PinotSink} according to the configuration. + * + * @return PinotSink + */ + public PinotSink build() { + return new PinotSink<>( + pinotControllerHost, + pinotControllerPort, + tableName, + maxRowsPerSegment, + tempDirPrefix, + jsonSerializer, + eventTimeExtractor, + segmentNameGenerator, + fsAdapter, + numCommitThreads + ); + } + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkCommittable.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkCommittable.java new file mode 100644 index 00000000..5a8c6553 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkCommittable.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.committer; + +import org.apache.flink.annotation.Internal; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The PinotSinkCommittable is required for sharing committables with the + * {@link PinotSinkGlobalCommitter} instance + */ +@Internal +public class PinotSinkCommittable implements Serializable { + private final String dataFilePath; + private final long minTimestamp; + private final long maxTimestamp; + + /** + * @param dataFilePath Path referencing a file on the shared filesystem defined via {@link org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter} + * @param minTimestamp The minimum timestamp of all the elements contained in {@link #dataFilePath} + * @param maxTimestamp The maximum timestamp of all the elements contained in {@link #dataFilePath} + */ + public PinotSinkCommittable(String dataFilePath, long minTimestamp, long maxTimestamp) { + this.dataFilePath = checkNotNull(dataFilePath); + this.minTimestamp = minTimestamp; + this.maxTimestamp = maxTimestamp; + } + + public String getDataFilePath() { + return dataFilePath; + } + + public long getMinTimestamp() { + return minTimestamp; + } + + public long getMaxTimestamp() { + return maxTimestamp; + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommittable.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommittable.java new file mode 100644 index 00000000..766b831d --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommittable.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.committer; + +import org.apache.flink.annotation.Internal; + +import java.io.Serializable; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Global committable references all data files that will be committed during checkpointing. + */ +@Internal +public class PinotSinkGlobalCommittable implements Serializable { + private final List dataFilePaths; + private final long minTimestamp; + private final long maxTimestamp; + + /** + * @param dataFilePaths List of paths to data files on shared file system + * @param minTimestamp Minimum timestamp of all objects in all data files + * @param maxTimestamp Maximum timestamp of all objects in all data files + */ + public PinotSinkGlobalCommittable(List dataFilePaths, long minTimestamp, long maxTimestamp) { + this.dataFilePaths = checkNotNull(dataFilePaths); + this.minTimestamp = minTimestamp; + this.maxTimestamp = maxTimestamp; + } + + public List getDataFilePaths() { + return dataFilePaths; + } + + public long getMinTimestamp() { + return minTimestamp; + } + + public long getMaxTimestamp() { + return maxTimestamp; + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java new file mode 100644 index 00000000..46e03e43 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/committer/PinotSinkGlobalCommitter.java @@ -0,0 +1,470 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.committer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink.GlobalCommitter; +import org.apache.flink.streaming.connectors.pinot.PinotControllerClient; +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter; +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemUtils; +import org.apache.pinot.common.segment.ReadMode; +import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig; +import org.apache.pinot.core.indexsegment.immutable.ImmutableSegment; +import org.apache.pinot.core.indexsegment.immutable.ImmutableSegmentLoader; +import org.apache.pinot.core.segment.creator.SegmentIndexCreationDriver; +import org.apache.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl; +import org.apache.pinot.core.segment.name.SegmentNameGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.FileFormat; +import org.apache.pinot.tools.admin.command.UploadSegmentCommand; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.*; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Global committer takes committables from {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter}, + * generates segments and pushed them to the Pinot controller. + * Note: We use a custom multithreading approach to parallelize the segment creation and upload to + * overcome the performance limitations resulting from using a {@link GlobalCommitter} always + * running at a parallelism of 1. + */ +@Internal +public class PinotSinkGlobalCommitter implements GlobalCommitter { + + private static final Logger LOG = LoggerFactory.getLogger(PinotSinkGlobalCommitter.class); + + private final String pinotControllerHost; + private final String pinotControllerPort; + private final String tableName; + private final SegmentNameGenerator segmentNameGenerator; + private final FileSystemAdapter fsAdapter; + private final String timeColumnName; + private final TimeUnit segmentTimeUnit; + private final PinotControllerClient pinotControllerClient; + private final File tempDirectory; + private final Schema tableSchema; + private final TableConfig tableConfig; + private final ExecutorService pool; + + /** + * @param pinotControllerHost Host of the Pinot controller + * @param pinotControllerPort Port of the Pinot controller + * @param tableName Target table's name + * @param segmentNameGenerator Pinot segment name generator + * @param tempDirPrefix Prefix for directory to store temporary files in + * @param fsAdapter Adapter for interacting with the shared file system + * @param timeColumnName Name of the column containing the timestamp + * @param segmentTimeUnit Unit of the time column + * @param numCommitThreads Number of threads used to commit the committables + */ + public PinotSinkGlobalCommitter(String pinotControllerHost, String pinotControllerPort, + String tableName, SegmentNameGenerator segmentNameGenerator, + String tempDirPrefix, FileSystemAdapter fsAdapter, + String timeColumnName, TimeUnit segmentTimeUnit, + int numCommitThreads) throws IOException { + this.pinotControllerHost = checkNotNull(pinotControllerHost); + this.pinotControllerPort = checkNotNull(pinotControllerPort); + this.tableName = checkNotNull(tableName); + this.segmentNameGenerator = checkNotNull(segmentNameGenerator); + this.fsAdapter = checkNotNull(fsAdapter); + this.timeColumnName = checkNotNull(timeColumnName); + this.segmentTimeUnit = checkNotNull(segmentTimeUnit); + this.pinotControllerClient = new PinotControllerClient(pinotControllerHost, pinotControllerPort); + + // Create directory that temporary files will be stored in + this.tempDirectory = Files.createTempDirectory(tempDirPrefix).toFile(); + + // Retrieve the Pinot table schema and the Pinot table config from the Pinot controller + this.tableSchema = pinotControllerClient.getSchema(tableName); + this.tableConfig = pinotControllerClient.getTableConfig(tableName); + + // We use a thread pool in order to parallelize the segment creation and segment upload + checkArgument(numCommitThreads > 0); + this.pool = Executors.newFixedThreadPool(numCommitThreads); + } + + /** + * Identifies global committables that need to be re-committed from a list of recovered committables. + * + * @param globalCommittables List of global committables that are checked for required re-commit + * @return List of global committable that need to be re-committed + * @throws IOException + */ + @Override + public List filterRecoveredCommittables(List globalCommittables) throws IOException { + // Holds identified global committables whose commit needs to be retried + List committablesToRetry = new ArrayList<>(); + + for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) { + CommitStatus commitStatus = getCommitStatus(globalCommittable); + + if (commitStatus.getMissingSegmentNames().isEmpty()) { + // All segments were already committed. Thus, we do not need to retry the commit. + continue; + } + + for (String existingSegment : commitStatus.getExistingSegmentNames()) { + // Some but not all segments were already committed. As we cannot assure the data + // files containing the same data as originally when recovering from failure, + // we delete the already committed segments in order to recommit them later on. + pinotControllerClient.deleteSegment(tableName, existingSegment); + } + committablesToRetry.add(globalCommittable); + } + return committablesToRetry; + } + + /** + * Combines multiple {@link PinotSinkCommittable}s into one {@link PinotSinkGlobalCommittable} + * by finding the minimum and maximum timestamps from the provided {@link PinotSinkCommittable}s. + * + * @param committables Committables created by {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter} + * @return Global committer committable + */ + @Override + public PinotSinkGlobalCommittable combine(List committables) { + List dataFilePaths = new ArrayList<>(); + long minTimestamp = Long.MAX_VALUE; + long maxTimestamp = Long.MIN_VALUE; + + // Extract all data file paths and the overall minimum and maximum timestamps + // from all committables + for (PinotSinkCommittable committable : committables) { + dataFilePaths.add(committable.getDataFilePath()); + minTimestamp = Long.min(minTimestamp, committable.getMinTimestamp()); + maxTimestamp = Long.max(maxTimestamp, committable.getMaxTimestamp()); + } + + LOG.debug("Combined {} committables into one global committable", committables.size()); + return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp); + } + + /** + * Copies data files from shared filesystem to the local filesystem, generates segments with names + * according to the segment naming schema and finally pushes the segments to the Pinot cluster. + * Before pushing a segment it is checked whether there already exists a segment with that name + * in the Pinot cluster by calling the Pinot controller. In case there is one, it gets deleted. + * + * @param globalCommittables List of global committables + * @return Global committables whose commit failed + * @throws IOException + */ + @Override + public List commit(List globalCommittables) throws IOException { + // List of failed global committables that can be retried later on + List failedCommits = new ArrayList<>(); + + for (PinotSinkGlobalCommittable globalCommittable : globalCommittables) { + Set> resultFutures = new HashSet<>(); + // Commit all segments in globalCommittable + for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) { + String dataFilePath = globalCommittable.getDataFilePaths().get(sequenceId); + // Get segment names with increasing sequenceIds + String segmentName = getSegmentName(globalCommittable, sequenceId); + // Segment committer handling the whole commit process for a single segment + Callable segmentCommitter = new SegmentCommitter( + pinotControllerHost, pinotControllerPort, tempDirectory, fsAdapter, + dataFilePath, segmentName, tableSchema, tableConfig, timeColumnName, + segmentTimeUnit + ); + // Submits the segment committer to the thread pool + resultFutures.add(pool.submit(segmentCommitter)); + } + + boolean commitSucceeded = true; + try { + for (Future wasSuccessful : resultFutures) { + // In case any of the segment commits wasn't successful we mark the whole + // globalCommittable as failed + if (!wasSuccessful.get()) { + commitSucceeded = false; + failedCommits.add(globalCommittable); + // Once any of the commits failed, we do not need to check the remaining + // ones, as we try to commit the globalCommittable next time + break; + } + } + } catch (InterruptedException | ExecutionException e) { + // In case of an exception thrown while accessing commit status, mark the whole + // globalCommittable as failed + failedCommits.add(globalCommittable); + LOG.error("Accessing a SegmentCommitter thread errored with {}", e.getMessage(), e); + } + + if (commitSucceeded) { + // If commit succeeded, cleanup the data files stored on the shared file system. In + // case the commit of at least one of the segments failed, nothing will be cleaned + // up here to enable retrying failed commits (data files must therefore stay + // available on the shared filesystem). + for (String path : globalCommittable.getDataFilePaths()) { + fsAdapter.deleteFromSharedFileSystem(path); + } + } + } + + // Return failed commits so that they can be retried later on + return failedCommits; + } + + /** + * Empty method. + */ + @Override + public void endOfInput() { + } + + /** + * Closes the Pinot controller http client, clears the created temporary directory and + * shuts the thread pool down. + */ + @Override + public void close() throws IOException { + pinotControllerClient.close(); + tempDirectory.delete(); + pool.shutdown(); + } + + /** + * Helper method for generating segment names using the segment name generator. + * + * @param globalCommittable Global committable the segment name shall be generated from + * @param sequenceId Incrementing counter + * @return generated segment name + */ + private String getSegmentName(PinotSinkGlobalCommittable globalCommittable, int sequenceId) { + return segmentNameGenerator.generateSegmentName(sequenceId, + globalCommittable.getMinTimestamp(), globalCommittable.getMaxTimestamp()); + } + + /** + * Evaluates the status of already uploaded segments by requesting segment metadata from the + * Pinot controller. + * + * @param globalCommittable Global committable whose commit status gets evaluated + * @return Commit status + * @throws IOException + */ + private CommitStatus getCommitStatus(PinotSinkGlobalCommittable globalCommittable) throws IOException { + List existingSegmentNames = new ArrayList<>(); + List missingSegmentNames = new ArrayList<>(); + + // For all segment names that will be used to submit new segments, check whether the segment + // name already exists for the target table + for (int sequenceId = 0; sequenceId < globalCommittable.getDataFilePaths().size(); sequenceId++) { + String segmentName = getSegmentName(globalCommittable, sequenceId); + if (pinotControllerClient.tableHasSegment(tableName, segmentName)) { + // Segment name already exists + existingSegmentNames.add(segmentName); + } else { + // Segment name does not exist yet + missingSegmentNames.add(segmentName); + } + } + return new CommitStatus(existingSegmentNames, missingSegmentNames); + } + + /** + * Wrapper for existing and missing segments in the Pinot cluster. + */ + private static class CommitStatus { + private final List existingSegmentNames; + private final List missingSegmentNames; + + CommitStatus(List existingSegmentNames, List missingSegmentNames) { + this.existingSegmentNames = existingSegmentNames; + this.missingSegmentNames = missingSegmentNames; + } + + public List getExistingSegmentNames() { + return existingSegmentNames; + } + + public List getMissingSegmentNames() { + return missingSegmentNames; + } + } + + /** + * Helper class for committing a single segment. Downloads a data file from the shared filesystem, + * generates a segment from the data file and uploads segment to the Pinot controller. + */ + private static class SegmentCommitter implements Callable { + + private static final Logger LOG = LoggerFactory.getLogger(SegmentCommitter.class); + + private final String pinotControllerHost; + private final String pinotControllerPort; + private final File tempDirectory; + private final FileSystemAdapter fsAdapter; + private final String dataFilePath; + private final String segmentName; + private final Schema tableSchema; + private final TableConfig tableConfig; + private final String timeColumnName; + private final TimeUnit segmentTimeUnit; + + /** + * @param pinotControllerHost Host of the Pinot controller + * @param pinotControllerPort Port of the Pinot controller + * @param tempDirectory Directory to store temporary files in + * @param fsAdapter Filesystem adapter used to load data files from the shared file system + * @param dataFilePath Data file to load from the shared file system + * @param segmentName Name of the segment to create and commit + * @param tableSchema Pinot table schema + * @param tableConfig Pinot table config + * @param timeColumnName Name of the column containing the timestamp + * @param segmentTimeUnit Unit of the time column + */ + SegmentCommitter(String pinotControllerHost, String pinotControllerPort, + File tempDirectory, FileSystemAdapter fsAdapter, + String dataFilePath, String segmentName, Schema tableSchema, + TableConfig tableConfig, String timeColumnName, + TimeUnit segmentTimeUnit) { + this.pinotControllerHost = pinotControllerHost; + this.pinotControllerPort = pinotControllerPort; + this.tempDirectory = tempDirectory; + this.fsAdapter = fsAdapter; + this.dataFilePath = dataFilePath; + this.segmentName = segmentName; + this.tableSchema = tableSchema; + this.tableConfig = tableConfig; + this.timeColumnName = timeColumnName; + this.segmentTimeUnit = segmentTimeUnit; + } + + /** + * Downloads a segment from the shared file system via {@code fsAdapter}, generates a segment + * and finally uploads the segment to the Pinot controller + * + * @return True if the commit succeeded + */ + @Override + public Boolean call() { + // Local copy of data file stored on the shared filesystem + File segmentData = null; + // File containing the final Pinot segment + File segmentFile = null; + try { + // Download data file from the shared filesystem + LOG.debug("Downloading data file {} from shared file system...", dataFilePath); + List serializedElements = fsAdapter.readFromSharedFileSystem(dataFilePath); + segmentData = FileSystemUtils.writeToLocalFile(serializedElements, tempDirectory); + LOG.debug("Successfully downloaded data file {} from shared file system", dataFilePath); + + segmentFile = FileSystemUtils.createFileInDir(tempDirectory); + LOG.debug("Creating segment in " + segmentFile.getAbsolutePath()); + + // Creates a segment with name `segmentName` in `segmentFile` + generateSegment(segmentData, segmentFile, true); + + // Uploads the recently created segment to the Pinot controller + uploadSegment(segmentFile); + + // Commit successful + return true; + } catch (IOException e) { + LOG.error("Error while committing segment data stored on shared filesystem.", e); + + // Commit failed + return false; + } finally { + // Finally cleanup all files created on the local filesystem + if (segmentData != null) { + segmentData.delete(); + } + if (segmentFile != null) { + segmentFile.delete(); + } + } + } + + /** + * Creates a segment from the given parameters. + * This method was adapted from {@link org.apache.pinot.tools.admin.command.CreateSegmentCommand}. + * + * @param dataFile File containing the JSON data + * @param outDir Segment target path + * @param _postCreationVerification Verify segment after generation + */ + private void generateSegment(File dataFile, File outDir, Boolean _postCreationVerification) { + SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(tableConfig, tableSchema); + segmentGeneratorConfig.setSegmentName(segmentName); + segmentGeneratorConfig.setSegmentTimeUnit(segmentTimeUnit); + segmentGeneratorConfig.setTimeColumnName(timeColumnName); + segmentGeneratorConfig.setInputFilePath(dataFile.getPath()); + segmentGeneratorConfig.setFormat(FileFormat.JSON); + segmentGeneratorConfig.setOutDir(outDir.getPath()); + segmentGeneratorConfig.setTableName(tableConfig.getTableName()); + + try { + SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl(); + driver.init(segmentGeneratorConfig); + driver.build(); + File indexDir = new File(outDir, segmentName); + LOG.debug("Successfully created segment: {} in directory: {}", segmentName, indexDir); + if (_postCreationVerification) { + LOG.debug("Verifying the segment by loading it"); + ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, ReadMode.mmap); + LOG.debug("Successfully loaded segment: {} of size: {} bytes", segmentName, + segment.getSegmentSizeBytes()); + segment.destroy(); + } + } + // SegmentIndexCreationDriverImpl throws generic Exceptions during init and build + // ImmutableSegmentLoader throws generic Exception during load + catch (Exception e) { + String message = String.format("Error while generating segment from file %s", dataFile.getAbsolutePath()); + LOG.error(message, e); + throw new RuntimeException(message); + } + LOG.debug("Successfully created 1 segment from data file: {}", dataFile); + } + + /** + * Uploads a segment using the Pinot admin tool. + * + * @param segmentFile File containing the segment to upload + * @throws IOException + */ + private void uploadSegment(File segmentFile) throws IOException { + try { + UploadSegmentCommand cmd = new UploadSegmentCommand(); + cmd.setControllerHost(pinotControllerHost); + cmd.setControllerPort(pinotControllerPort); + cmd.setSegmentDir(segmentFile.getAbsolutePath()); + cmd.execute(); + } catch (Exception e) { + // UploadSegmentCommand.execute() throws generic Exception + LOG.error("Could not upload segment {}", segmentFile.getAbsolutePath(), e); + throw new IOException(e.getMessage()); + } + } + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/exceptions/PinotControllerApiException.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/exceptions/PinotControllerApiException.java new file mode 100644 index 00000000..c5283d6c --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/exceptions/PinotControllerApiException.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.exceptions; + +import org.apache.flink.annotation.Internal; + +import java.io.IOException; + +/** + * Pinot controller API exception wrapper + */ +@Internal +public class PinotControllerApiException extends IOException { + + public PinotControllerApiException(String reason) { + super(reason); + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/EventTimeExtractor.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/EventTimeExtractor.java new file mode 100644 index 00000000..fe2647d0 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/EventTimeExtractor.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.external; + +import org.apache.flink.api.connector.sink.SinkWriter; + +import java.io.Serializable; +import java.util.concurrent.TimeUnit; + +/** + * Defines the interface for event time extractors + * + * @param Type of incoming elements + */ +public interface EventTimeExtractor extends Serializable { + + /** + * Extracts event time from incoming elements. + * + * @param element Incoming element + * @param context Context of SinkWriter + * @return timestamp + */ + long getEventTime(IN element, SinkWriter.Context context); + + /** + * @return Name of column in Pinot target table that contains the timestamp. + */ + String getTimeColumn(); + + /** + * @return Unit of the time column in the Pinot target table. + */ + TimeUnit getSegmentTimeUnit(); +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/JsonSerializer.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/JsonSerializer.java new file mode 100644 index 00000000..8774ac19 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/external/JsonSerializer.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.external; + +import java.io.Serializable; + +/** + * Defines the interface for serializing incoming elements to JSON format. + * The JSON format is expected during Pinot segment creation. + * + * @param Type of incoming elements + */ +public abstract class JsonSerializer implements Serializable { + + public abstract String toJson(IN element); +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemAdapter.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemAdapter.java new file mode 100644 index 00000000..42610a84 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemAdapter.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.filesystem; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +/** + * Defines the interaction with a shared filesystem. The shared filesystem must be accessible from all + * nodes within the cluster than run a partition of the {@link org.apache.flink.streaming.connectors.pinot.PinotSink}. + */ +public interface FileSystemAdapter extends Serializable { + + /** + * Writes a list of serialized elements to the shared filesystem. + * + * @param elements List of serialized elements + * @return Path identifying the remote file + * @throws IOException + */ + String writeToSharedFileSystem(List elements) throws IOException; + + /** + * Reads a previously written list of serialized elements from the shared filesystem. + * + * @param path Path returned by {@link #writeToSharedFileSystem} + * @return List of serialized elements read from the shared filesystem + * @throws IOException + */ + List readFromSharedFileSystem(String path) throws IOException; + + /** + * Deletes a file from the shared filesystem + * + * @param path Path returned by {@link #writeToSharedFileSystem} + * @throws IOException + */ + void deleteFromSharedFileSystem(String path) throws IOException; +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemUtils.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemUtils.java new file mode 100644 index 00000000..b61d9cf1 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/filesystem/FileSystemUtils.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.filesystem; + +import org.apache.flink.annotation.Internal; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.util.List; +import java.util.UUID; + +@Internal +public class FileSystemUtils { + + private static final Logger LOG = LoggerFactory.getLogger(FileSystemUtils.class); + + /** + * Writes a list of serialized elements to the temp directory of local filesystem + * with prefix tempDirPrefix + * + * @param elements List of serialized elements + * @param targetDir Directory to create file in + * @return File containing the written data + * @throws IOException + */ + public static File writeToLocalFile(List elements, File targetDir) throws IOException { + File dataFile = createFileInDir(targetDir); + + Files.write(dataFile.toPath(), elements, Charset.defaultCharset()); + LOG.debug("Successfully written data to file {}", dataFile.getAbsolutePath()); + + return dataFile; + } + + /** + * Creates file with random name in targetDir. + * + * @param targetDir Directory to create file in + * @return New File + */ + public static File createFileInDir(File targetDir) { + String fileName = String.format("%s.json", UUID.randomUUID().toString()); + return new File(targetDir.toString(), fileName); + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java new file mode 100644 index 00000000..ee3a9081 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/PinotSinkSegmentNameGenerator.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.segment.name; + +import org.apache.pinot.core.segment.name.SegmentNameGenerator; + +import java.io.Serializable; + +/** + * Defines the segment name generator interface that is used to generate segment names. The segment + * name generator is required to be serializable. We expect users to inherit from + * {@link PinotSinkSegmentNameGenerator} in case they want to define their custom name generator. + */ +public interface PinotSinkSegmentNameGenerator extends SegmentNameGenerator, Serializable { +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/SimpleSegmentNameGenerator.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/SimpleSegmentNameGenerator.java new file mode 100644 index 00000000..666673ca --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/segment/name/SimpleSegmentNameGenerator.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.segment.name; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Adapted from {@link org.apache.pinot.core.segment.name.SimpleSegmentNameGenerator}. + *

+ * Simple segment name generator which does not perform time conversion. + *

+ * The segment name is simply joining the following fields with '_' but ignoring all the {@code null}s. + *

+ */ +public class SimpleSegmentNameGenerator implements PinotSinkSegmentNameGenerator { + + private final String tableName; + private final String segmentNamePostfix; + + public SimpleSegmentNameGenerator(String tableName, String segmentNamePostfix) { + this.tableName = checkNotNull(tableName); + this.segmentNamePostfix = checkNotNull(segmentNamePostfix); + } + + @Override + public String generateSegmentName(int sequenceId, @Nullable Object minTimeValue, @Nullable Object maxTimeValue) { + return JOINER + .join(tableName, minTimeValue, maxTimeValue, segmentNamePostfix, sequenceId >= 0 ? sequenceId : null); + } + + @Override + public String toString() { + StringBuilder stringBuilder = new StringBuilder("SimpleSegmentNameGenerator: tableName=").append(tableName); + if (segmentNamePostfix != null) { + stringBuilder.append(", segmentNamePostfix=").append(segmentNamePostfix); + } + return stringBuilder.toString(); + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java new file mode 100644 index 00000000..ed61de26 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkCommittableSerializer.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.serializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable; + +import java.io.*; + +/** + * Serializer for {@link PinotSinkCommittable} + */ +@Internal +public class PinotSinkCommittableSerializer implements SimpleVersionedSerializer { + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(PinotSinkCommittable pinotSinkCommittable) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeLong(pinotSinkCommittable.getMinTimestamp()); + out.writeLong(pinotSinkCommittable.getMaxTimestamp()); + out.writeUTF(pinotSinkCommittable.getDataFilePath()); + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public PinotSinkCommittable deserialize(int version, byte[] serialized) throws IllegalStateException, IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IllegalStateException("Unrecognized version or corrupt state: " + version); + } + } + + private PinotSinkCommittable deserializeV1(byte[] serialized) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + long minTimestamp = in.readLong(); + long maxTimestamp = in.readLong(); + String dataFilePath = in.readUTF(); + return new PinotSinkCommittable(dataFilePath, minTimestamp, maxTimestamp); + } + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkGlobalCommittableSerializer.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkGlobalCommittableSerializer.java new file mode 100644 index 00000000..8e456206 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkGlobalCommittableSerializer.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.serializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommittable; + +import java.io.*; +import java.util.ArrayList; +import java.util.List; + +/** + * Serializer for {@link PinotSinkGlobalCommittable} + */ +@Internal +public class PinotSinkGlobalCommittableSerializer implements SimpleVersionedSerializer { + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(PinotSinkGlobalCommittable pinotSinkGlobalCommittable) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeLong(pinotSinkGlobalCommittable.getMinTimestamp()); + out.writeLong(pinotSinkGlobalCommittable.getMaxTimestamp()); + + int size = pinotSinkGlobalCommittable.getDataFilePaths().size(); + out.writeInt(size); + for (String dataFilePath : pinotSinkGlobalCommittable.getDataFilePaths()) { + out.writeUTF(dataFilePath); + } + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public PinotSinkGlobalCommittable deserialize(int version, byte[] serialized) throws IllegalStateException, IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IllegalStateException("Unrecognized version or corrupt state: " + version); + } + } + + private PinotSinkGlobalCommittable deserializeV1(byte[] serialized) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + long minTimestamp = in.readLong(); + long maxTimestamp = in.readLong(); + + long size = in.readInt(); + List dataFilePaths = new ArrayList<>(); + for (int i = 0; i < size; i++) { + dataFilePaths.add(in.readUTF()); + } + return new PinotSinkGlobalCommittable(dataFilePaths, minTimestamp, maxTimestamp); + } + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkWriterStateSerializer.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkWriterStateSerializer.java new file mode 100644 index 00000000..6dc7efaa --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/serializer/PinotSinkWriterStateSerializer.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.serializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriterState; + +import java.io.*; +import java.util.ArrayList; +import java.util.List; + +/** + * Serializer for {@link PinotSinkWriterState} + */ +@Internal +public class PinotSinkWriterStateSerializer implements SimpleVersionedSerializer { + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(PinotSinkWriterState writerState) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeLong(writerState.getMinTimestamp()); + out.writeLong(writerState.getMaxTimestamp()); + + out.writeInt(writerState.getSerializedElements().size()); + for (String serialized : writerState.getSerializedElements()) { + out.writeUTF(serialized); + } + + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public PinotSinkWriterState deserialize(int version, byte[] serialized) throws IllegalStateException, IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IllegalStateException("Unrecognized version or corrupt state: " + version); + } + } + + private PinotSinkWriterState deserializeV1(byte[] serialized) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + long minTimestamp = in.readLong(); + long maxTimestamp = in.readLong(); + + long size = in.readInt(); + List serializedElements = new ArrayList<>(); + for (int i = 0; i < size; i++) { + serializedElements.add(in.readUTF()); + } + return new PinotSinkWriterState(serializedElements, minTimestamp, maxTimestamp); + } + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java new file mode 100644 index 00000000..1a84e022 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriter.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.writer; + +import com.google.common.collect.Iterables; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable; +import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor; +import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer; +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Accepts incoming elements and creates {@link PinotSinkCommittable}s out of them on request. + * + * @param Type of incoming elements + */ +@Internal +public class PinotSinkWriter implements SinkWriter { + + private static final Logger LOG = LoggerFactory.getLogger(PinotSinkWriter.class); + + private final int maxRowsPerSegment; + private final EventTimeExtractor eventTimeExtractor; + private final JsonSerializer jsonSerializer; + + private final List> activeSegments; + private final FileSystemAdapter fsAdapter; + + private final int subtaskId; + + /** + * @param subtaskId Subtask id provided by Flink + * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment + * @param eventTimeExtractor Defines the way event times are extracted from received objects + * @param jsonSerializer Serializer used to convert elements to JSON + * @param fsAdapter Filesystem adapter used to save files for sharing files across nodes + */ + public PinotSinkWriter(int subtaskId, int maxRowsPerSegment, + EventTimeExtractor eventTimeExtractor, + JsonSerializer jsonSerializer, FileSystemAdapter fsAdapter) { + this.subtaskId = subtaskId; + this.maxRowsPerSegment = maxRowsPerSegment; + this.eventTimeExtractor = checkNotNull(eventTimeExtractor); + this.jsonSerializer = checkNotNull(jsonSerializer); + this.fsAdapter = checkNotNull(fsAdapter); + this.activeSegments = new ArrayList<>(); + } + + /** + * Takes elements from an upstream tasks and writes them into {@link PinotWriterSegment} + * + * @param element Object from upstream task + * @param context SinkWriter context + * @throws IOException + */ + @Override + public void write(IN element, Context context) throws IOException { + final PinotWriterSegment inProgressSegment = getOrCreateInProgressSegment(); + inProgressSegment.write(element, eventTimeExtractor.getEventTime(element, context)); + } + + /** + * Creates {@link PinotSinkCommittable}s from elements previously received via {@link #write}. + * If flush is set, all {@link PinotWriterSegment}s are transformed into + * {@link PinotSinkCommittable}s. If flush is not set, only currently non-active + * {@link PinotSinkCommittable}s are transformed into {@link PinotSinkCommittable}s. + * To convert a {@link PinotWriterSegment} into a {@link PinotSinkCommittable} the data gets + * written to the shared filesystem. Moreover, minimum and maximum timestamps are identified. + * Finally, all {@link PinotWriterSegment}s transformed into {@link PinotSinkCommittable}s are + * removed from {@link #activeSegments}. + * + * @param flush Flush all currently known elements into the {@link PinotSinkCommittable}s + * @return List of {@link PinotSinkCommittable} to process in {@link org.apache.flink.streaming.connectors.pinot.committer.PinotSinkGlobalCommitter} + * @throws IOException + */ + @Override + public List prepareCommit(boolean flush) throws IOException { + // Identify segments to commit. If the flush argument is set all segments shall be committed. + // Otherwise, take only those PinotWriterSegments that do not accept any more elements. + List> segmentsToCommit = activeSegments.stream() + .filter(s -> flush || !s.acceptsElements()) + .collect(Collectors.toList()); + LOG.debug("Identified {} segments to commit [subtaskId={}]", segmentsToCommit.size(), subtaskId); + + LOG.debug("Creating committables... [subtaskId={}]", subtaskId); + List committables = new ArrayList<>(); + for (final PinotWriterSegment segment : segmentsToCommit) { + committables.add(segment.prepareCommit()); + } + LOG.debug("Created {} committables [subtaskId={}]", committables.size(), subtaskId); + + // Remove all PinotWriterSegments that will be emitted within the committables. + activeSegments.removeAll(segmentsToCommit); + return committables; + } + + /** + * Gets the {@link PinotWriterSegment} still accepting elements or creates a new one. + * + * @return {@link PinotWriterSegment} accepting at least one more element + */ + private PinotWriterSegment getOrCreateInProgressSegment() { + final PinotWriterSegment latestSegment = Iterables.getLast(activeSegments, null); + if (latestSegment == null || !latestSegment.acceptsElements()) { + final PinotWriterSegment inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter); + activeSegments.add(inProgressSegment); + return inProgressSegment; + } + return latestSegment; + } + + /** + * Snapshots the latest PinotWriterSegment (if existent), so that the contained (and not yet + * committed) elements can be recovered later on in case of a failure. + * + * @return A list containing at most one PinotSinkWriterState + */ + @Override + public List snapshotState() { + final PinotWriterSegment latestSegment = Iterables.getLast(activeSegments, null); + if (latestSegment == null || !latestSegment.acceptsElements()) { + return new ArrayList<>(); + } + + return Collections.singletonList(latestSegment.snapshotState()); + } + + /** + * Initializes the writer according to a previously taken snapshot. + * + * @param state PinotSinkWriterState extracted from snapshot + */ + public void initializeState(PinotSinkWriterState state) { + if (activeSegments.size() != 0) { + throw new IllegalStateException("Please call the initialization before creating the first PinotWriterSegment."); + } + // Create a new PinotWriterSegment and recover its state from the given PinotSinkWriterState + final PinotWriterSegment inProgressSegment = new PinotWriterSegment<>(maxRowsPerSegment, jsonSerializer, fsAdapter); + inProgressSegment.initializeState(state.getSerializedElements(), state.getMinTimestamp(), state.getMaxTimestamp()); + activeSegments.add(inProgressSegment); + } + + /** + * Empty method, as we do not open any connections. + */ + @Override + public void close() { + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriterState.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriterState.java new file mode 100644 index 00000000..0e23e2fa --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotSinkWriterState.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.writer; + +import java.io.Serializable; +import java.util.List; + +public class PinotSinkWriterState implements Serializable { + + private final List serializedElements; + private final long minTimestamp; + private final long maxTimestamp; + + public PinotSinkWriterState(List serializedElements, long minTimestamp, long maxTimestamp) { + this.serializedElements = serializedElements; + this.minTimestamp = minTimestamp; + this.maxTimestamp = maxTimestamp; + } + + public List getSerializedElements() { + return serializedElements; + } + + public long getMinTimestamp() { + return minTimestamp; + } + + public long getMaxTimestamp() { + return maxTimestamp; + } +} diff --git a/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java new file mode 100644 index 00000000..50be1459 --- /dev/null +++ b/flink-connector-pinot/src/main/java/org/apache/flink/streaming/connectors/pinot/writer/PinotWriterSegment.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot.writer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.pinot.committer.PinotSinkCommittable; +import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer; +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link PinotWriterSegment} represents exactly one segment that can be found in the Pinot + * cluster once the commit has been completed. + * + * @param Type of incoming elements + */ +@Internal +public class PinotWriterSegment implements Serializable { + + private final int maxRowsPerSegment; + private final JsonSerializer jsonSerializer; + private final FileSystemAdapter fsAdapter; + + private boolean acceptsElements = true; + + private final List serializedElements; + private String dataPathOnSharedFS; + private long minTimestamp = Long.MAX_VALUE; + private long maxTimestamp = Long.MIN_VALUE; + + /** + * @param maxRowsPerSegment Maximum number of rows to be stored within a Pinot segment + * @param jsonSerializer Serializer used to convert elements to JSON + * @param fsAdapter Filesystem adapter used to save files for sharing files across nodes + */ + PinotWriterSegment(int maxRowsPerSegment, JsonSerializer jsonSerializer, FileSystemAdapter fsAdapter) { + checkArgument(maxRowsPerSegment > 0L); + this.maxRowsPerSegment = maxRowsPerSegment; + this.jsonSerializer = checkNotNull(jsonSerializer); + this.fsAdapter = checkNotNull(fsAdapter); + this.serializedElements = new ArrayList<>(); + } + + /** + * Takes elements and stores them in memory until either {@link #maxRowsPerSegment} is reached + * or {@link #prepareCommit} is called. + * + * @param element Object from upstream task + * @param timestamp Timestamp assigned to element + * @throws IOException + */ + public void write(IN element, long timestamp) throws IOException { + if (!acceptsElements()) { + throw new IllegalStateException("This PinotSegmentWriter does not accept any elements anymore."); + } + // Store serialized element in serializedElements + serializedElements.add(jsonSerializer.toJson(element)); + minTimestamp = Long.min(minTimestamp, timestamp); + maxTimestamp = Long.max(maxTimestamp, timestamp); + + // Writes elements to local filesystem once the maximum number of items is reached + if (serializedElements.size() == maxRowsPerSegment) { + acceptsElements = false; + dataPathOnSharedFS = writeToSharedFilesystem(); + serializedElements.clear(); + } + } + + /** + * Writes elements to local file (if not already done). Copies just created file to the shared + * filesystem defined via {@link FileSystemAdapter} and creates a {@link PinotSinkCommittable}. + * + * @return {@link PinotSinkCommittable} pointing to file on shared filesystem + * @throws IOException + */ + public PinotSinkCommittable prepareCommit() throws IOException { + if (dataPathOnSharedFS == null) { + dataPathOnSharedFS = writeToSharedFilesystem(); + } + return new PinotSinkCommittable(dataPathOnSharedFS, minTimestamp, maxTimestamp); + } + + /** + * Takes elements from {@link #serializedElements} and writes them to the shared filesystem. + * + * @return Path pointing to just written data on shared filesystem + * @throws IOException + */ + private String writeToSharedFilesystem() throws IOException { + return fsAdapter.writeToSharedFileSystem(serializedElements); + } + + /** + * Determines whether this segment can accept at least one more elements + * + * @return True if at least one more element will be accepted + */ + public boolean acceptsElements() { + return acceptsElements; + } + + /** + * Recovers a previously written state. + * + * @param _serializedElements List containing received, but not yet committed list of serialized elements. + * @param _minTimestamp Minimum event timestamp of all elements + * @param _maxTimestamp Maximum event timestamp of all elements + */ + public void initializeState(List _serializedElements, long _minTimestamp, long _maxTimestamp) { + if (!serializedElements.isEmpty()) { + throw new IllegalStateException("Cannot initialize a PinotWriterSegment that has already received elements."); + } + + serializedElements.addAll(_serializedElements); + minTimestamp = _minTimestamp; + maxTimestamp = _maxTimestamp; + } + + /** + * Snapshots the current state of an active {@link PinotWriterSegment}. + * + * @return List of elements currently stored within the {@link PinotWriterSegment} + */ + public PinotSinkWriterState snapshotState() { + if (!acceptsElements()) { + throw new IllegalStateException("Snapshots can only be created of in-progress segments."); + } + + return new PinotSinkWriterState(serializedElements, minTimestamp, maxTimestamp); + } +} diff --git a/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/LocalFileSystemAdapter.java b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/LocalFileSystemAdapter.java new file mode 100644 index 00000000..069daa39 --- /dev/null +++ b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/LocalFileSystemAdapter.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot; + +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter; +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemUtils; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The LocalFileSystemAdapter is used when sharing files via the local filesystem. + * Keep in mind that using this FileSystemAdapter requires running the Flink app on a single node. + */ +public class LocalFileSystemAdapter implements FileSystemAdapter { + + private final String tempDirPrefix; + + public LocalFileSystemAdapter(String tempDirPrefix) { + this.tempDirPrefix = checkNotNull(tempDirPrefix); + } + + /** + * Writes a list of serialized elements to the local filesystem. + * + * @param elements List of serialized elements + * @return Path identifying the written file + * @throws IOException + */ + @Override + public String writeToSharedFileSystem(List elements) throws IOException { + File tempDir = Files.createTempDirectory(tempDirPrefix).toFile(); + return FileSystemUtils.writeToLocalFile(elements, tempDir).getAbsolutePath(); + } + + /** + * Reads a previously written list of serialized elements from the local filesystem. + * + * @param path Path returned by {@link #writeToSharedFileSystem} + * @return List of serialized elements read from the local filesystem + * @throws IOException + */ + @Override + public List readFromSharedFileSystem(String path) throws IOException { + File dataFile = new File(path); + return Files.readAllLines(dataFile.toPath(), Charset.defaultCharset()); + } + + /** + * Deletes a file from the local filesystem + * + * @param path Path returned by {@link #writeToSharedFileSystem} + * @throws IOException + */ + @Override + public void deleteFromSharedFileSystem(String path) { + new File(path).delete(); + } +} diff --git a/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java new file mode 100644 index 00000000..8649eebe --- /dev/null +++ b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotSinkTest.java @@ -0,0 +1,475 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ + + +package org.apache.flink.streaming.connectors.pinot; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException; +import org.apache.flink.streaming.connectors.pinot.external.EventTimeExtractor; +import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer; +import org.apache.flink.streaming.connectors.pinot.filesystem.FileSystemAdapter; +import org.apache.flink.streaming.connectors.pinot.segment.name.PinotSinkSegmentNameGenerator; +import org.apache.flink.streaming.connectors.pinot.segment.name.SimpleSegmentNameGenerator; +import org.apache.pinot.client.PinotClientException; +import org.apache.pinot.client.ResultSet; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opentest4j.AssertionFailedError; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * E2e tests for Pinot Sink using BATCH and STREAMING execution mode + */ +public class PinotSinkTest extends PinotTestBase { + + private static final int MAX_ROWS_PER_SEGMENT = 5; + private static final long STREAMING_CHECKPOINTING_INTERVAL = 50; + private static final int DATA_CHECKING_TIMEOUT_SECONDS = 60; + private static final AtomicBoolean hasFailedOnce = new AtomicBoolean(false); + private static CountDownLatch latch; + + @BeforeEach + public void setUp() throws IOException { + super.setUp(); + // Reset hasFailedOnce flag used during failure recovery testing before each test. + hasFailedOnce.set(false); + // Reset latch used to keep the generator streaming source up until the test is completed. + latch = new CountDownLatch(1); + } + + /** + * Tests the BATCH execution of the {@link PinotSink}. + * + * @throws Exception + */ + @Test + public void testBatchSink() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(2); + + List rawData = getRawTestData(12); + DataStream dataStream = setupBatchDataSource(env, rawData); + setupSink(dataStream); + + // Run + env.execute(); + + // Check for data in Pinot + checkForDataInPinotWithRetry(rawData); + } + + /** + * Tests failure recovery of the {@link PinotSink} using BATCH execution mode. + * + * @throws Exception + */ + @Test + public void testFailureRecoveryInBatchingSink() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10)); + env.setParallelism(2); + + List rawData = getRawTestData(12); + DataStream dataStream = setupBatchDataSource(env, rawData); + dataStream = setupFailingMapper(dataStream, 8); + setupSink(dataStream); + + // Run + env.execute(); + + // Check for data in Pinot + checkForDataInPinotWithRetry(rawData); + } + + /** + * Tests the STREAMING execution of the {@link PinotSink}. + * + * @throws Exception + */ + @Test + public void testStreamingSink() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(2); + env.enableCheckpointing(STREAMING_CHECKPOINTING_INTERVAL); + + List rawData = getRawTestData(20); + DataStream dataStream = setupStreamingDataSource(env, rawData); + setupSink(dataStream); + + // Start execution of job + env.executeAsync(); + + // Check for data in Pinot + checkForDataInPinotWithRetry(rawData); + + // Generator source can now shut down + latch.countDown(); + } + + /** + * Tests failure recovery of the {@link PinotSink} using STREAMING execution mode. + * + * @throws Exception + */ + @Test + public void testFailureRecoveryInStreamingSink() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + env.setParallelism(1); + env.enableCheckpointing(STREAMING_CHECKPOINTING_INTERVAL); + + List rawData = getRawTestData(20); + DataStream dataStream = setupFailingStreamingDataSource(env, rawData, 12); + setupSink(dataStream); + + // Start execution of job + env.executeAsync(); + + // Check for data in Pinot + checkForDataInPinotWithRetry(rawData); + + // Generator source can now shut down + latch.countDown(); + } + + /** + * Generates a small test dataset consisting of {@link SingleColumnTableRow}s. + * + * @return List of SingleColumnTableRow + */ + private List getRawTestData(int numItems) { + return IntStream.range(1, numItems + 1) + .mapToObj(num -> "ColValue" + num) + .collect(Collectors.toList()); + } + + /** + * Setup the data source for STREAMING tests. + * + * @param env Stream execution environment + * @param rawDataValues Data values to send + * @return resulting data stream + */ + private DataStream setupStreamingDataSource(StreamExecutionEnvironment env, List rawDataValues) { + StreamingSource source = new StreamingSource.Builder(rawDataValues, 10).build(); + return env.addSource(source) + .name("Test input"); + } + + /** + * Setup the data source for STREAMING tests. + * + * @param env Stream execution environment + * @param rawDataValues Data values to send + * @param failOnceAtNthElement Number of elements to process before raising the exception + * @return resulting data stream + */ + private DataStream setupFailingStreamingDataSource(StreamExecutionEnvironment env, List rawDataValues, int failOnceAtNthElement) { + StreamingSource source = new StreamingSource.Builder(rawDataValues, 10) + .raiseFailureOnce(failOnceAtNthElement) + .build(); + return env.addSource(source) + .name("Test input"); + } + + /** + * Setup the data source for BATCH tests. + * + * @param env Stream execution environment + * @param rawDataValues Data values to send + * @return resulting data stream + */ + private DataStream setupBatchDataSource(StreamExecutionEnvironment env, List rawDataValues) { + return env.fromCollection(rawDataValues) + .map(value -> new SingleColumnTableRow(value, System.currentTimeMillis())) + .name("Test input"); + } + + /** + * Setup a mapper that fails when processing the nth element with n = failOnceAtNthElement. + * + * @param dataStream Input data stream + * @param failOnceAtNthElement Number of elements to process before raising the exception + * @return resulting data stream + */ + private DataStream setupFailingMapper(DataStream dataStream, int failOnceAtNthElement) { + AtomicInteger messageCounter = new AtomicInteger(0); + + return dataStream.map(element -> { + if (!hasFailedOnce.get() && messageCounter.incrementAndGet() == failOnceAtNthElement) { + hasFailedOnce.set(true); + throw new Exception(String.format("Mapper was expected to fail after %d elements", failOnceAtNthElement)); + } + return element; + }); + } + + /** + * Sets up a DataStream using the provided execution environment and the provided input data. + * + * @param dataStream data stream + */ + private void setupSink(DataStream dataStream) { + String tempDirPrefix = "flink-pinot-connector-test"; + PinotSinkSegmentNameGenerator segmentNameGenerator = new SimpleSegmentNameGenerator(getTableName(), "flink-connector"); + FileSystemAdapter fsAdapter = new LocalFileSystemAdapter(tempDirPrefix); + JsonSerializer jsonSerializer = new SingleColumnTableRowSerializer(); + + EventTimeExtractor eventTimeExtractor = new SingleColumnTableRowEventTimeExtractor(); + + PinotSink sink = new PinotSink.Builder(getPinotHost(), getPinotControllerPort(), getTableName()) + .withMaxRowsPerSegment(MAX_ROWS_PER_SEGMENT) + .withTempDirectoryPrefix(tempDirPrefix) + .withJsonSerializer(jsonSerializer) + .withEventTimeExtractor(eventTimeExtractor) + .withSegmentNameGenerator(segmentNameGenerator) + .withFileSystemAdapter(fsAdapter) + .withNumCommitThreads(2) + .build(); + + // Sink into Pinot + dataStream.sinkTo(sink).name("Pinot sink"); + } + + /** + * As Pinot might take some time to index the recently pushed segments we might need to retry + * the {@link #checkForDataInPinot} method multiple times. This method provides a simple wrapper + * using linear retry backoff delay. + * + * @param rawData Data to expect in the Pinot table + * @throws InterruptedException + */ + private void checkForDataInPinotWithRetry(List rawData) throws InterruptedException, PinotControllerApiException { + long endTime = System.currentTimeMillis() + 1000L * DATA_CHECKING_TIMEOUT_SECONDS; + // Use max 10 retries with linear retry backoff delay + long retryDelay = 1000L / 10 * DATA_CHECKING_TIMEOUT_SECONDS; + while (System.currentTimeMillis() < endTime) { + try { + checkForDataInPinot(rawData); + // In case of no error, we can skip further retries + return; + } catch (AssertionFailedError | PinotControllerApiException | PinotClientException e) { + // In case of an error retry after delay + Thread.sleep(retryDelay); + } + } + + // Finally check for data in Pinot if retryTimeoutInSeconds was exceeded + checkForDataInPinot(rawData); + } + + /** + * Checks whether data is present in the Pinot target table. numElementsToCheck defines the + * number of elements (from the head of data) to check for existence in the pinot table. + * + * @param rawData Data to expect in the Pinot table + * @throws AssertionFailedError in case the assertion fails + * @throws PinotControllerApiException in case there aren't any rows in the Pinot table + */ + private void checkForDataInPinot(List rawData) throws AssertionFailedError, PinotControllerApiException, PinotClientException { + // Now get the result from Pinot and verify if everything is there + ResultSet resultSet = pinotHelper.getTableEntries(getTableName(), rawData.size() + 5); + + Assertions.assertEquals(rawData.size(), resultSet.getRowCount(), + String.format("Expected %d elements in Pinot but saw %d", rawData.size(), resultSet.getRowCount())); + + // Check output strings + List output = IntStream.range(0, resultSet.getRowCount()) + .mapToObj(i -> resultSet.getString(i, 0)) + .collect(Collectors.toList()); + + for (String test : rawData) { + Assertions.assertTrue(output.contains(test), "Missing " + test); + } + } + + /** + * EventTimeExtractor for {@link SingleColumnTableRow} used in e2e tests. + * Extracts the timestamp column from {@link SingleColumnTableRow}. + */ + private static class SingleColumnTableRowEventTimeExtractor implements EventTimeExtractor { + + @Override + public long getEventTime(SingleColumnTableRow element, SinkWriter.Context context) { + return element.getTimestamp(); + } + + @Override + public String getTimeColumn() { + return "timestamp"; + } + + @Override + public TimeUnit getSegmentTimeUnit() { + return TimeUnit.MILLISECONDS; + } + } + + /** + * Simple source that publishes data and finally waits for {@link #latch}. + * By setting {@link #failOnceAtNthElement} > -1, one can define the number of elements to + * process before raising an exception. If configured, the exception will only be raised once. + */ + private static class StreamingSource implements SourceFunction, CheckpointedFunction { + + private static final int serialVersionUID = 1; + + private final List rawDataValues; + private final int sleepDurationMs; + private final int failOnceAtNthElement; + + private int numElementsEmitted = 0; + + private final AtomicBoolean waitingForNextSnapshot; + private final AtomicBoolean awaitedSnapshotCreated; + + private ListState state = null; + + private StreamingSource(final List rawDataValues, final int sleepDurationMs, int failOnceAtNthElement) { + this.rawDataValues = rawDataValues; + checkArgument(sleepDurationMs > 0); + this.sleepDurationMs = sleepDurationMs; + checkArgument(failOnceAtNthElement == -1 || failOnceAtNthElement > MAX_ROWS_PER_SEGMENT); + this.failOnceAtNthElement = failOnceAtNthElement; + + // Initializes exception raising logic + this.waitingForNextSnapshot = new AtomicBoolean(false); + this.awaitedSnapshotCreated = new AtomicBoolean(false); + } + + @Override + public void run(final SourceContext ctx) throws Exception { + while (numElementsEmitted < rawDataValues.size()) { + if (!hasFailedOnce.get() && failOnceAtNthElement == numElementsEmitted) { + failAfterNextSnapshot(); + } + + synchronized (ctx.getCheckpointLock()) { + SingleColumnTableRow element = new SingleColumnTableRow( + rawDataValues.get(numElementsEmitted), System.currentTimeMillis()); + ctx.collect(element); + numElementsEmitted++; + } + Thread.sleep(sleepDurationMs); + } + + // Keep generator source up until the test was completed. + latch.await(); + } + + /** + * When {@link #failOnceAtNthElement} elements were received, we raise an exception after + * the next checkpoint was created. We ensure that at least one segment has been committed + * to Pinot by then, as we require {@link #failOnceAtNthElement} to be greater than + * {@link #MAX_ROWS_PER_SEGMENT} (at a parallelism of 1). This allows to check whether the + * snapshot creation and failure recovery in + * {@link org.apache.flink.streaming.connectors.pinot.writer.PinotSinkWriter} works properly, + * respecting the already committed elements and those that are stored in an active + * {@link org.apache.flink.streaming.connectors.pinot.writer.PinotWriterSegment}. Committed + * elements must not be saved to the snapshot while those in an active segment must be saved + * to the snapshot in order to enable later-on recovery. + * + * @throws Exception + */ + private void failAfterNextSnapshot() throws Exception { + hasFailedOnce.set(true); + waitingForNextSnapshot.set(true); + + // Waiting for the next snapshot ensures that + // at least one segment has been committed to Pinot + while (!awaitedSnapshotCreated.get()) { + Thread.sleep(50); + } + throw new Exception(String.format("Source was expected to fail after %d elements", failOnceAtNthElement)); + } + + @Override + public void cancel() { + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore() + .getListState(new ListStateDescriptor<>("state", IntSerializer.INSTANCE)); + + for (Integer i : state.get()) { + numElementsEmitted += i; + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(numElementsEmitted); + + // Notify that the awaited snapshot was been created + if (waitingForNextSnapshot.get()) { + awaitedSnapshotCreated.set(true); + } + } + + static class Builder { + final List rawDataValues; + final int sleepDurationMs; + int failOnceAtNthElement = -1; + + Builder(List rawDataValues, int sleepDurationMs) { + this.rawDataValues = rawDataValues; + this.sleepDurationMs = sleepDurationMs; + } + + public Builder raiseFailureOnce(int failOnceAtNthElement) { + checkArgument(failOnceAtNthElement > MAX_ROWS_PER_SEGMENT, + "failOnceAtNthElement (if set) is required to be larger than the number of elements per segment (MAX_ROWS_PER_SEGMENT)."); + this.failOnceAtNthElement = failOnceAtNthElement; + return this; + } + + public StreamingSource build() { + return new StreamingSource(rawDataValues, sleepDurationMs, failOnceAtNthElement); + } + } + } +} diff --git a/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java new file mode 100644 index 00000000..a5f50215 --- /dev/null +++ b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestBase.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.streaming.connectors.pinot.external.JsonSerializer; +import org.apache.flink.util.TestLogger; +import org.apache.pinot.spi.config.table.*; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.JsonUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.time.Duration; + +/** + * Base class for PinotSink e2e tests + */ +@Testcontainers +public class PinotTestBase extends TestLogger { + + protected static final Logger LOG = LoggerFactory.getLogger(PinotTestBase.class); + + private static final String DOCKER_IMAGE_NAME = "apachepinot/pinot:0.6.0"; + private static final Integer PINOT_INTERNAL_BROKER_PORT = 8000; + private static final Integer PINOT_INTERNAL_CONTROLLER_PORT = 9000; + + protected static TableConfig TABLE_CONFIG; + protected static final Schema TABLE_SCHEMA = PinotTableConfig.getTableSchema(); + protected static PinotTestHelper pinotHelper; + + /** + * Creates the Pinot testcontainer. We delay the start of tests until Pinot has started all + * internal components. This is identified through a log statement. + */ + @Container + public static GenericContainer pinot = new GenericContainer<>(DockerImageName.parse(DOCKER_IMAGE_NAME)) + .withCommand("QuickStart", "-type", "batch") + .withExposedPorts(PINOT_INTERNAL_BROKER_PORT, PINOT_INTERNAL_CONTROLLER_PORT) + .waitingFor( + // Wait for controller, server and broker instances to be available + new HttpWaitStrategy() + .forPort(PINOT_INTERNAL_CONTROLLER_PORT) + .forPath("/instances") + .forStatusCode(200) + .forResponsePredicate(res -> { + try { + JsonNode instances = JsonUtils.stringToJsonNode(res).get("instances"); + // Expect 3 instances to be up and running (controller, broker and server) + return instances.size() == 3; + } catch (IOException e) { + LOG.error("Error while reading json response in HttpWaitStrategy.", e); + } + return false; + }) + // Allow Pinot to take up to 180s for starting up + .withStartupTimeout(Duration.ofSeconds(180)) + ); + + /** + * Creates a new instance of the {@link PinotTestHelper} using the testcontainer port mappings + * and creates the test table. + * + * @throws IOException + */ + @BeforeEach + public void setUp() throws IOException { + TABLE_CONFIG = PinotTableConfig.getTableConfig(); + pinotHelper = new PinotTestHelper(getPinotHost(), getPinotControllerPort(), getPinotBrokerPort()); + pinotHelper.createTable(TABLE_CONFIG, TABLE_SCHEMA); + } + + /** + * Delete the test table after each test. + * + * @throws Exception + */ + @AfterEach + public void tearDown() throws Exception { + pinotHelper.deleteTable(TABLE_CONFIG, TABLE_SCHEMA); + } + + /** + * Returns the current Pinot table name + * + * @return Pinot table name + */ + protected String getTableName() { + return TABLE_CONFIG.getTableName(); + } + + /** + * Returns the host the Pinot container is available at + * + * @return Pinot container host + */ + protected String getPinotHost() { + return pinot.getHost(); + } + + + /** + * Returns the Pinot controller port from the container ports. + * + * @return Pinot controller port + */ + protected String getPinotControllerPort() { + return pinot.getMappedPort(PINOT_INTERNAL_CONTROLLER_PORT).toString(); + } + + /** + * Returns the Pinot broker port from the container ports. + * + * @return Pinot broker port + */ + private String getPinotBrokerPort() { + return pinot.getMappedPort(PINOT_INTERNAL_BROKER_PORT).toString(); + } + + /** + * Class defining the elements passed to the {@link PinotSink} during the tests. + */ + protected static class SingleColumnTableRow { + + private String _col1; + private Long _timestamp; + + SingleColumnTableRow(@JsonProperty(value = "col1", required = true) String col1, + @JsonProperty(value = "timestamp", required = true) Long timestamp) { + this._col1 = col1; + this._timestamp = timestamp; + } + + @JsonProperty("col1") + public String getCol1() { + return this._col1; + } + + public void setCol1(String _col1) { + this._col1 = _col1; + } + + @JsonProperty("timestamp") + public Long getTimestamp() { + return this._timestamp; + } + + public void setTimestamp(Long timestamp) { + this._timestamp = timestamp; + } + } + + /** + * Serializes {@link SingleColumnTableRow} to JSON. + */ + protected static class SingleColumnTableRowSerializer extends JsonSerializer { + + @Override + public String toJson(SingleColumnTableRow element) { + return JsonUtils.objectToJsonNode(element).toString(); + } + } + + /** + * Pinot table configuration helpers. + */ + private static class PinotTableConfig { + + static final String TABLE_NAME_PREFIX = "FLTable"; + static final String SCHEMA_NAME = "FLTableSchema"; + + private static SegmentsValidationAndRetentionConfig getValidationConfig() { + SegmentsValidationAndRetentionConfig validationConfig = new SegmentsValidationAndRetentionConfig(); + validationConfig.setSegmentAssignmentStrategy("BalanceNumSegmentAssignmentStrategy"); + validationConfig.setSegmentPushType("APPEND"); + validationConfig.setSchemaName(SCHEMA_NAME); + validationConfig.setReplication("1"); + return validationConfig; + } + + private static TenantConfig getTenantConfig() { + TenantConfig tenantConfig = new TenantConfig("DefaultTenant", "DefaultTenant", null); + return tenantConfig; + } + + private static IndexingConfig getIndexingConfig() { + IndexingConfig indexingConfig = new IndexingConfig(); + return indexingConfig; + } + + private static TableCustomConfig getCustomConfig() { + TableCustomConfig customConfig = new TableCustomConfig(null); + return customConfig; + } + + private static String generateTableName() { + // We want to use a new table name for each test in order to prevent interference + // with segments that were pushed in the previous test, + // but whose indexing by Pinot was delayed (thus, the previous test must have failed). + return String.format("%s_%d", TABLE_NAME_PREFIX, System.currentTimeMillis()); + } + + static TableConfig getTableConfig() { + return new TableConfig( + generateTableName(), + TableType.OFFLINE.name(), + getValidationConfig(), + getTenantConfig(), + getIndexingConfig(), + getCustomConfig(), + null, null, null, null, null, + null, null, null, null + ); + } + + static Schema getTableSchema() { + Schema schema = new Schema(); + schema.setSchemaName(SCHEMA_NAME); + schema.addField(new DimensionFieldSpec("col1", FieldSpec.DataType.STRING, true)); + schema.addField(new DimensionFieldSpec("timestamp", FieldSpec.DataType.STRING, true)); + return schema; + } + } +} diff --git a/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestHelper.java b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestHelper.java new file mode 100644 index 00000000..73a4403b --- /dev/null +++ b/flink-connector-pinot/src/test/java/org/apache/flink/streaming/connectors/pinot/PinotTestHelper.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.flink.streaming.connectors.pinot; + +import org.apache.flink.streaming.connectors.pinot.exceptions.PinotControllerApiException; +import org.apache.pinot.client.*; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; + +/** + * Helper class ot interact with the Pinot controller and broker in the e2e tests + */ +public class PinotTestHelper implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(PinotTestHelper.class); + private final String host; + private final String brokerPort; + private final PinotControllerHttpClient httpClient; + + /** + * @param host Host the Pinot controller and broker are accessible at + * @param controllerPort The Pinot controller's external port at {@code host} + * @param brokerPort A Pinot broker's external port at {@code host} + */ + public PinotTestHelper(String host, String controllerPort, String brokerPort) { + this.host = host; + this.brokerPort = brokerPort; + this.httpClient = new PinotControllerHttpClient(host, controllerPort); + } + + /** + * Adds a Pinot table schema. + * + * @param tableSchema Pinot table schema to add + * @throws IOException + */ + private void addSchema(Schema tableSchema) throws IOException { + PinotControllerHttpClient.ApiResponse res = httpClient.post("/schemas", JsonUtils.objectToString(tableSchema)); + LOG.debug("Schema add request for schema {} returned {}", tableSchema.getSchemaName(), res.responseBody); + if (res.statusLine.getStatusCode() != 200) { + throw new PinotControllerApiException(res.responseBody); + } + } + + /** + * Deletes a Pinot table schema. + * + * @param tableSchema Pinot table schema to delete + * @throws IOException + */ + private void deleteSchema(Schema tableSchema) throws IOException { + PinotControllerHttpClient.ApiResponse res = httpClient.delete(String.format("/schemas/%s", tableSchema.getSchemaName())); + LOG.debug("Schema delete request for schema {} returned {}", tableSchema.getSchemaName(), res.responseBody); + if (res.statusLine.getStatusCode() != 200) { + throw new PinotControllerApiException(res.responseBody); + } + } + + /** + * Creates a Pinot table. + * + * @param tableConfig Pinot table configuration of table to create + * @throws IOException + */ + private void addTable(TableConfig tableConfig) throws IOException { + PinotControllerHttpClient.ApiResponse res = httpClient.post("/tables", JsonUtils.objectToString(tableConfig)); + LOG.debug("Table creation request for table {} returned {}", tableConfig.getTableName(), res.responseBody); + if (res.statusLine.getStatusCode() != 200) { + throw new PinotControllerApiException(res.responseBody); + } + } + + /** + * Deletes a Pinot table with all its segments. + * + * @param tableConfig Pinot table configuration of table to delete + * @throws IOException + */ + private void removeTable(TableConfig tableConfig) throws IOException { + PinotControllerHttpClient.ApiResponse res = httpClient.delete(String.format("/tables/%s", tableConfig.getTableName())); + LOG.debug("Table deletion request for table {} returned {}", tableConfig.getTableName(), res.responseBody); + if (res.statusLine.getStatusCode() != 200) { + throw new PinotControllerApiException(res.responseBody); + } + } + + /** + * Creates a Pinot table by first adding a schema and then creating the actual table using the + * Pinot table configuration + * + * @param tableConfig Pinot table configuration + * @param tableSchema Pinot table schema + * @throws IOException + */ + public void createTable(TableConfig tableConfig, Schema tableSchema) throws IOException { + this.addSchema(tableSchema); + this.addTable(tableConfig); + } + + /** + * Deletes a Pinot table by first deleting the table and its segments and then deleting the + * table's schema. + * + * @param tableConfig Pinot table configuration + * @param tableSchema Pinot table schema + * @throws IOException + */ + public void deleteTable(TableConfig tableConfig, Schema tableSchema) throws IOException { + this.removeTable(tableConfig); + this.deleteSchema(tableSchema); + } + + /** + * Fetch table entries via the Pinot broker. + * + * @param tableName Target table's name + * @param maxNumberOfEntries Max number of entries to fetch + * @return ResultSet + * @throws PinotControllerApiException + */ + public ResultSet getTableEntries(String tableName, Integer maxNumberOfEntries) throws PinotControllerApiException { + Connection brokerConnection = null; + try { + String brokerHostPort = String.format("%s:%s", this.host, this.brokerPort); + brokerConnection = ConnectionFactory.fromHostList(brokerHostPort); + String query = String.format("SELECT * FROM %s LIMIT %d", tableName, maxNumberOfEntries); + + Request pinotClientRequest = new Request("sql", query); + ResultSetGroup pinotResultSetGroup = brokerConnection.execute(pinotClientRequest); + + if (pinotResultSetGroup.getResultSetCount() != 1) { + throw new PinotControllerApiException("Could not find any data in Pinot cluster."); + } + return pinotResultSetGroup.getResultSet(0); + } finally { + if (brokerConnection != null) { + brokerConnection.close(); + } + } + } + + @Override + public void close() throws IOException { + httpClient.close(); + } +} diff --git a/flink-connector-pinot/src/test/resources/log4j.properties b/flink-connector-pinot/src/test/resources/log4j.properties new file mode 100644 index 00000000..b15f2bef --- /dev/null +++ b/flink-connector-pinot/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF 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. +################################################################################ + +# This file ensures that tests executed from the IDE show log output + +log4j.rootLogger=DEBUG, console + +# Log all infos in the given file +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.out +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/pom.xml b/pom.xml index 7e83d852..57eb579c 100644 --- a/pom.xml +++ b/pom.xml @@ -77,6 +77,7 @@ flink-connector-influxdb flink-connector-kudu flink-connector-netty + flink-connector-pinot flink-connector-redis flink-library-siddhi From b5c7e5d5061726df7667f29764ed01dfeda99861 Mon Sep 17 00:00:00 2001 From: yiksanchan Date: Wed, 7 Apr 2021 04:24:01 -0700 Subject: [PATCH 12/14] [BAHIR-247] Provide connection validation/idle testing for Flink-Redis Connector (#121) --- .../config/FlinkJedisClusterConfig.java | 71 ++++++++++++++--- .../common/config/FlinkJedisConfigBase.java | 46 ++++++++++- .../common/config/FlinkJedisPoolConfig.java | 72 ++++++++++++++--- .../config/FlinkJedisSentinelConfig.java | 79 +++++++++++++++---- .../RedisCommandsContainerBuilder.java | 28 ++++--- .../config/FlinkJedisConfigBaseTest.java | 14 ++-- .../RedisCommandsContainerBuilderTest.java | 60 ++++++++++++++ 7 files changed, 314 insertions(+), 56 deletions(-) create mode 100644 flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilderTest.java diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java index c56ac14e..0840deb5 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisClusterConfig.java @@ -48,12 +48,15 @@ public class FlinkJedisClusterConfig extends FlinkJedisConfigBase { * @param maxIdle the cap on the number of "idle" instances in the pool * @param minIdle the minimum number of idle objects to maintain in the pool * @param password the password of redis cluster + * @param testOnBorrow Whether objects borrowed from the pool will be validated before being returned, default value is false + * @param testOnReturn Whether objects borrowed from the pool will be validated when they are returned to the pool, default value is false + * @param testWhileIdle Whether objects sitting idle in the pool will be validated by the idle object evictor, default value is false * @throws NullPointerException if parameter {@code nodes} is {@code null} */ private FlinkJedisClusterConfig(Set nodes, int connectionTimeout, int maxRedirections, - int maxTotal, int maxIdle, int minIdle, - String password) { - super(connectionTimeout, maxTotal, maxIdle, minIdle, password); + int maxTotal, int maxIdle, int minIdle, String password, + boolean testOnBorrow, boolean testOnReturn, boolean testWhileIdle) { + super(connectionTimeout, maxTotal, maxIdle, minIdle, password, testOnBorrow, testOnReturn, testWhileIdle); Objects.requireNonNull(nodes, "Node information should be presented"); Util.checkArgument(!nodes.isEmpty(), "Redis cluster hosts should not be empty"); @@ -96,6 +99,9 @@ public static class Builder { private int maxTotal = GenericObjectPoolConfig.DEFAULT_MAX_TOTAL; private int maxIdle = GenericObjectPoolConfig.DEFAULT_MAX_IDLE; private int minIdle = GenericObjectPoolConfig.DEFAULT_MIN_IDLE; + private boolean testOnBorrow = GenericObjectPoolConfig.DEFAULT_TEST_ON_BORROW; + private boolean testOnReturn = GenericObjectPoolConfig.DEFAULT_TEST_ON_RETURN; + private boolean testWhileIdle = GenericObjectPoolConfig.DEFAULT_TEST_WHILE_IDLE; private String password; /** @@ -179,25 +185,68 @@ public Builder setPassword(String password) { return this; } + /** + * Sets value for the {@code testOnBorrow} configuration attribute + * for pools to be created with this configuration instance. + * + * @param testOnBorrow Whether objects borrowed from the pool will be validated before being returned + * @return Builder itself + */ + public Builder setTestOnBorrow(boolean testOnBorrow) { + this.testOnBorrow = testOnBorrow; + return this; + } + + /** + * Sets value for the {@code testOnReturn} configuration attribute + * for pools to be created with this configuration instance. + * + * @param testOnReturn Whether objects borrowed from the pool will be validated when they are returned to the pool + * @return Builder itself + */ + public Builder setTestOnReturn(boolean testOnReturn) { + this.testOnReturn = testOnReturn; + return this; + } + + /** + * Sets value for the {@code testWhileIdle} configuration attribute + * for pools to be created with this configuration instance. + * + * Setting this to true will also set default idle-testing parameters provided in Jedis + * @see redis.clients.jedis.JedisPoolConfig + * + * @param testWhileIdle Whether objects sitting idle in the pool will be validated by the idle object evictor + * @return Builder itself + */ + public Builder setTestWhileIdle(boolean testWhileIdle) { + this.testWhileIdle = testWhileIdle; + return this; + } + /** * Builds JedisClusterConfig. * * @return JedisClusterConfig */ public FlinkJedisClusterConfig build() { - return new FlinkJedisClusterConfig(nodes, timeout, maxRedirections, maxTotal, maxIdle, minIdle, password); + return new FlinkJedisClusterConfig(nodes, timeout, maxRedirections, maxTotal, maxIdle, minIdle, password, testOnBorrow, testOnReturn, testWhileIdle); } } @Override public String toString() { return "FlinkJedisClusterConfig{" + - "nodes=" + nodes + - ", timeout=" + connectionTimeout + - ", maxRedirections=" + maxRedirections + - ", maxTotal=" + maxTotal + - ", maxIdle=" + maxIdle + - ", minIdle=" + minIdle + - '}'; + "nodes=" + nodes + + ", maxRedirections=" + maxRedirections + + ", maxTotal=" + maxTotal + + ", maxIdle=" + maxIdle + + ", minIdle=" + minIdle + + ", connectionTimeout=" + connectionTimeout + + ", password=" + password + + ", testOnBorrow=" + testOnBorrow + + ", testOnReturn=" + testOnReturn + + ", testWhileIdle=" + testWhileIdle + + '}'; } } diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.java index 84b1bf23..a41b0e03 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBase.java @@ -33,7 +33,12 @@ public abstract class FlinkJedisConfigBase implements Serializable { protected final int connectionTimeout; protected final String password; - protected FlinkJedisConfigBase(int connectionTimeout, int maxTotal, int maxIdle, int minIdle, String password) { + protected final boolean testOnBorrow; + protected final boolean testOnReturn; + protected final boolean testWhileIdle; + + protected FlinkJedisConfigBase(int connectionTimeout, int maxTotal, int maxIdle, int minIdle, String password, boolean testOnBorrow, boolean testOnReturn, boolean testWhileIdle) { + Util.checkArgument(connectionTimeout >= 0, "connection timeout can not be negative"); Util.checkArgument(maxTotal >= 0, "maxTotal value can not be negative"); Util.checkArgument(maxIdle >= 0, "maxIdle value can not be negative"); @@ -43,6 +48,9 @@ protected FlinkJedisConfigBase(int connectionTimeout, int maxTotal, int maxIdle, this.maxTotal = maxTotal; this.maxIdle = maxIdle; this.minIdle = minIdle; + this.testOnBorrow = testOnBorrow; + this.testOnReturn = testOnReturn; + this.testWhileIdle = testWhileIdle; this.password = password; } @@ -99,4 +107,40 @@ public int getMinIdle() { public String getPassword() { return password; } + + /** + * Get the value for the {@code testOnBorrow} configuration attribute + * for pools to be created with this configuration instance. + * + * @return The current setting of {@code testOnBorrow} for this + * configuration instance + * @see GenericObjectPoolConfig#getTestOnBorrow() + */ + public boolean getTestOnBorrow() { + return testOnBorrow; + } + + /** + * Get the value for the {@code testOnReturn} configuration attribute + * for pools to be created with this configuration instance. + * + * @return The current setting of {@code testOnReturn} for this + * configuration instance + * @see GenericObjectPoolConfig#getTestOnReturn() + */ + public boolean getTestOnReturn() { + return testOnReturn; + } + + /** + * Get the value for the {@code testWhileIdle} configuration attribute + * for pools to be created with this configuration instance. + * + * @return The current setting of {@code testWhileIdle} for this + * configuration instance + * @see GenericObjectPoolConfig#getTestWhileIdle() + */ + public boolean getTestWhileIdle() { + return testWhileIdle; + } } diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java index 3f8fc2ff..5012da1f 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisPoolConfig.java @@ -45,11 +45,16 @@ public class FlinkJedisPoolConfig extends FlinkJedisConfigBase { * @param maxTotal the maximum number of objects that can be allocated by the pool, default value is 8 * @param maxIdle the cap on the number of "idle" instances in the pool, default value is 8 * @param minIdle the minimum number of idle objects to maintain in the pool, default value is 0 + * @param testOnBorrow Whether objects borrowed from the pool will be validated before being returned, default value is false + * @param testOnReturn Whether objects borrowed from the pool will be validated when they are returned to the pool, default value is false + * @param testWhileIdle Whether objects sitting idle in the pool will be validated by the idle object evictor, default value is false * @throws NullPointerException if parameter {@code host} is {@code null} */ private FlinkJedisPoolConfig(String host, int port, int connectionTimeout, String password, int database, - int maxTotal, int maxIdle, int minIdle) { - super(connectionTimeout, maxTotal, maxIdle, minIdle, password); + int maxTotal, int maxIdle, int minIdle, + boolean testOnBorrow, boolean testOnReturn, boolean testWhileIdle) { + super(connectionTimeout, maxTotal, maxIdle, minIdle, password, testOnBorrow, testOnReturn, testWhileIdle); + Objects.requireNonNull(host, "Host information should be presented"); this.host = host; this.port = port; @@ -96,6 +101,9 @@ public static class Builder { private int maxTotal = GenericObjectPoolConfig.DEFAULT_MAX_TOTAL; private int maxIdle = GenericObjectPoolConfig.DEFAULT_MAX_IDLE; private int minIdle = GenericObjectPoolConfig.DEFAULT_MIN_IDLE; + private boolean testOnBorrow = GenericObjectPoolConfig.DEFAULT_TEST_ON_BORROW; + private boolean testOnReturn = GenericObjectPoolConfig.DEFAULT_TEST_ON_RETURN; + private boolean testWhileIdle = GenericObjectPoolConfig.DEFAULT_TEST_WHILE_IDLE; /** * Sets value for the {@code maxTotal} configuration attribute @@ -188,6 +196,44 @@ public Builder setPassword(String password) { return this; } + /** + * Sets value for the {@code testOnBorrow} configuration attribute + * for pools to be created with this configuration instance. + * + * @param testOnBorrow Whether objects borrowed from the pool will be validated before being returned + * @return Builder itself + */ + public Builder setTestOnBorrow(boolean testOnBorrow) { + this.testOnBorrow = testOnBorrow; + return this; + } + + /** + * Sets value for the {@code testOnReturn} configuration attribute + * for pools to be created with this configuration instance. + * + * @param testOnReturn Whether objects borrowed from the pool will be validated when they are returned to the pool + * @return Builder itself + */ + public Builder setTestOnReturn(boolean testOnReturn) { + this.testOnReturn = testOnReturn; + return this; + } + + /** + * Sets value for the {@code testWhileIdle} configuration attribute + * for pools to be created with this configuration instance. + * + * Setting this to true will also set default idle-testing parameters provided in Jedis + * @see redis.clients.jedis.JedisPoolConfig + * + * @param testWhileIdle Whether objects sitting idle in the pool will be validated by the idle object evictor + * @return Builder itself + */ + public Builder setTestWhileIdle(boolean testWhileIdle) { + this.testWhileIdle = testWhileIdle; + return this; + } /** * Builds JedisPoolConfig. @@ -195,20 +241,24 @@ public Builder setPassword(String password) { * @return JedisPoolConfig */ public FlinkJedisPoolConfig build() { - return new FlinkJedisPoolConfig(host, port, timeout, password, database, maxTotal, maxIdle, minIdle); + return new FlinkJedisPoolConfig(host, port, timeout, password, database, maxTotal, maxIdle, minIdle, testOnBorrow, testOnReturn, testWhileIdle); } } @Override public String toString() { return "FlinkJedisPoolConfig{" + - "host='" + host + '\'' + - ", port=" + port + - ", timeout=" + connectionTimeout + - ", database=" + database + - ", maxTotal=" + maxTotal + - ", maxIdle=" + maxIdle + - ", minIdle=" + minIdle + - '}'; + "host=" + host + + ", port=" + port + + ", database=" + database + + ", maxTotal=" + maxTotal + + ", maxIdle=" + maxIdle + + ", minIdle=" + minIdle + + ", connectionTimeout=" + + ", password=" + password + + ", testOnBorrow=" + testOnBorrow + + ", testOnReturn=" + testOnReturn + + ", testWhileIdle=" + testWhileIdle + + '}'; } } diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java index 928f5e8c..340eb4e4 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisSentinelConfig.java @@ -52,15 +52,19 @@ public class FlinkJedisSentinelConfig extends FlinkJedisConfigBase { * @param maxTotal maxTotal the maximum number of objects that can be allocated by the pool * @param maxIdle the cap on the number of "idle" instances in the pool * @param minIdle the minimum number of idle objects to maintain in the pool - * + * @param testOnBorrow Whether objects borrowed from the pool will be validated before being returned, default value is false + * @param testOnReturn Whether objects borrowed from the pool will be validated when they are returned to the pool, default value is false + * @param testWhileIdle Whether objects sitting idle in the pool will be validated by the idle object evictor, default value is false * @throws NullPointerException if {@code masterName} or {@code sentinels} is {@code null} * @throws IllegalArgumentException if {@code sentinels} are empty */ private FlinkJedisSentinelConfig(String masterName, Set sentinels, - int connectionTimeout, int soTimeout, - String password, int database, - int maxTotal, int maxIdle, int minIdle) { - super(connectionTimeout, maxTotal, maxIdle, minIdle, password); + int connectionTimeout, int soTimeout, + String password, int database, + int maxTotal, int maxIdle, int minIdle, + boolean testOnBorrow, boolean testOnReturn, boolean testWhileIdle) { + super(connectionTimeout, maxTotal, maxIdle, minIdle, password, testOnBorrow, testOnReturn, testWhileIdle); + Objects.requireNonNull(masterName, "Master name should be presented"); Objects.requireNonNull(sentinels, "Sentinels information should be presented"); Util.checkArgument(!sentinels.isEmpty(), "Sentinel hosts should not be empty"); @@ -120,6 +124,9 @@ public static class Builder { private int maxTotal = GenericObjectPoolConfig.DEFAULT_MAX_TOTAL; private int maxIdle = GenericObjectPoolConfig.DEFAULT_MAX_IDLE; private int minIdle = GenericObjectPoolConfig.DEFAULT_MIN_IDLE; + private boolean testOnBorrow = GenericObjectPoolConfig.DEFAULT_TEST_ON_BORROW; + private boolean testOnReturn = GenericObjectPoolConfig.DEFAULT_TEST_ON_RETURN; + private boolean testWhileIdle = GenericObjectPoolConfig.DEFAULT_TEST_WHILE_IDLE; /** * Sets master name of the replica set. @@ -223,6 +230,45 @@ public Builder setMinIdle(int minIdle) { return this; } + /** + * Sets value for the {@code testOnBorrow} configuration attribute + * for pools to be created with this configuration instance. + * + * @param testOnBorrow Whether objects borrowed from the pool will be validated before being returned + * @return Builder itself + */ + public Builder setTestOnBorrow(boolean testOnBorrow) { + this.testOnBorrow = testOnBorrow; + return this; + } + + /** + * Sets value for the {@code testOnReturn} configuration attribute + * for pools to be created with this configuration instance. + * + * @param testOnReturn Whether objects borrowed from the pool will be validated when they are returned to the pool + * @return Builder itself + */ + public Builder setTestOnReturn(boolean testOnReturn) { + this.testOnReturn = testOnReturn; + return this; + } + + /** + * Sets value for the {@code testWhileIdle} configuration attribute + * for pools to be created with this configuration instance. + * + * Setting this to true will also set default idle-testing parameters provided in Jedis + * @see redis.clients.jedis.JedisPoolConfig + * + * @param testWhileIdle Whether objects sitting idle in the pool will be validated by the idle object evictor + * @return Builder itself + */ + public Builder setTestWhileIdle(boolean testWhileIdle) { + this.testWhileIdle = testWhileIdle; + return this; + } + /** * Builds JedisSentinelConfig. * @@ -230,20 +276,25 @@ public Builder setMinIdle(int minIdle) { */ public FlinkJedisSentinelConfig build(){ return new FlinkJedisSentinelConfig(masterName, sentinels, connectionTimeout, soTimeout, - password, database, maxTotal, maxIdle, minIdle); + password, database, maxTotal, maxIdle, minIdle, testOnBorrow, testOnReturn, testWhileIdle); } } @Override public String toString() { return "FlinkJedisSentinelConfig{" + - "masterName='" + masterName + '\'' + - ", connectionTimeout=" + connectionTimeout + - ", soTimeout=" + soTimeout + - ", database=" + database + - ", maxTotal=" + maxTotal + - ", maxIdle=" + maxIdle + - ", minIdle=" + minIdle + - '}'; + "masterName=" + masterName + + ", sentinels=" + sentinels + + ", soTimeout=" + soTimeout + + ", database=" + database + + ", maxTotal=" + maxTotal + + ", maxIdle=" + maxIdle + + ", minIdle=" + minIdle + + ", connectionTimeout=" + connectionTimeout + + ", password=" + password + + ", testOnBorrow=" + testOnBorrow + + ", testOnReturn=" + testOnReturn + + ", testWhileIdle=" + testWhileIdle + + '}'; } } diff --git a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java index bdb9fed1..b06a6e97 100644 --- a/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java +++ b/flink-connector-redis/src/main/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilder.java @@ -23,6 +23,7 @@ import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisSentinelConfig; import redis.clients.jedis.JedisCluster; import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisPoolConfig; import redis.clients.jedis.JedisSentinelPool; import java.util.Objects; @@ -65,8 +66,8 @@ public static RedisCommandsContainer build(FlinkJedisPoolConfig jedisPoolConfig) GenericObjectPoolConfig genericObjectPoolConfig = getGenericObjectPoolConfig(jedisPoolConfig); JedisPool jedisPool = new JedisPool(genericObjectPoolConfig, jedisPoolConfig.getHost(), - jedisPoolConfig.getPort(), jedisPoolConfig.getConnectionTimeout(), jedisPoolConfig.getPassword(), - jedisPoolConfig.getDatabase()); + jedisPoolConfig.getPort(), jedisPoolConfig.getConnectionTimeout(), jedisPoolConfig.getPassword(), + jedisPoolConfig.getDatabase()); return new RedisContainer(jedisPool); } @@ -83,11 +84,11 @@ public static RedisCommandsContainer build(FlinkJedisClusterConfig jedisClusterC GenericObjectPoolConfig genericObjectPoolConfig = getGenericObjectPoolConfig(jedisClusterConfig); JedisCluster jedisCluster = new JedisCluster(jedisClusterConfig.getNodes(), - jedisClusterConfig.getConnectionTimeout(), - jedisClusterConfig.getConnectionTimeout(), - jedisClusterConfig.getMaxRedirections(), - jedisClusterConfig.getPassword(), - genericObjectPoolConfig); + jedisClusterConfig.getConnectionTimeout(), + jedisClusterConfig.getConnectionTimeout(), + jedisClusterConfig.getMaxRedirections(), + jedisClusterConfig.getPassword(), + genericObjectPoolConfig); return new RedisClusterContainer(jedisCluster); } @@ -104,17 +105,20 @@ public static RedisCommandsContainer build(FlinkJedisSentinelConfig jedisSentine GenericObjectPoolConfig genericObjectPoolConfig = getGenericObjectPoolConfig(jedisSentinelConfig); JedisSentinelPool jedisSentinelPool = new JedisSentinelPool(jedisSentinelConfig.getMasterName(), - jedisSentinelConfig.getSentinels(), genericObjectPoolConfig, - jedisSentinelConfig.getConnectionTimeout(), jedisSentinelConfig.getSoTimeout(), - jedisSentinelConfig.getPassword(), jedisSentinelConfig.getDatabase()); + jedisSentinelConfig.getSentinels(), genericObjectPoolConfig, + jedisSentinelConfig.getConnectionTimeout(), jedisSentinelConfig.getSoTimeout(), + jedisSentinelConfig.getPassword(), jedisSentinelConfig.getDatabase()); return new RedisContainer(jedisSentinelPool); } - private static GenericObjectPoolConfig getGenericObjectPoolConfig(FlinkJedisConfigBase jedisConfig) { - GenericObjectPoolConfig genericObjectPoolConfig = new GenericObjectPoolConfig(); + public static GenericObjectPoolConfig getGenericObjectPoolConfig(FlinkJedisConfigBase jedisConfig) { + GenericObjectPoolConfig genericObjectPoolConfig = jedisConfig.getTestWhileIdle() ? new JedisPoolConfig(): new GenericObjectPoolConfig(); genericObjectPoolConfig.setMaxIdle(jedisConfig.getMaxIdle()); genericObjectPoolConfig.setMaxTotal(jedisConfig.getMaxTotal()); genericObjectPoolConfig.setMinIdle(jedisConfig.getMinIdle()); + genericObjectPoolConfig.setTestOnBorrow(jedisConfig.getTestOnBorrow()); + genericObjectPoolConfig.setTestOnReturn(jedisConfig.getTestOnReturn()); + return genericObjectPoolConfig; } } diff --git a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java index 6f519ed7..80189dff 100644 --- a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java +++ b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/config/FlinkJedisConfigBaseTest.java @@ -23,28 +23,28 @@ public class FlinkJedisConfigBaseTest extends TestLogger { @Test(expected = IllegalArgumentException.class) public void shouldThrowIllegalArgumentExceptionIfTimeOutIsNegative(){ - new TestConfig(-1, 0, 0, 0); + new TestConfig(-1, 0, 0, 0, false, false, false); } @Test(expected = IllegalArgumentException.class) public void shouldThrowIllegalArgumentExceptionIfMaxTotalIsNegative(){ - new TestConfig(1, -1, 0, 0); + new TestConfig(1, -1, 0, 0, false, false, false); } @Test(expected = IllegalArgumentException.class) public void shouldThrowIllegalArgumentExceptionIfMaxIdleIsNegative(){ - new TestConfig(0, 0, -1, 0); + new TestConfig(0, 0, -1, 0, false, false, false); } @Test(expected = IllegalArgumentException.class) public void shouldThrowIllegalArgumentExceptionIfMinIdleIsNegative(){ - new TestConfig(0, 0, 0, -1); + new TestConfig(0, 0, 0, -1, false, false, false); } private class TestConfig extends FlinkJedisConfigBase { - - protected TestConfig(int connectionTimeout, int maxTotal, int maxIdle, int minIdle) { - super(connectionTimeout, maxTotal, maxIdle, minIdle, "dummy"); + protected TestConfig(int connectionTimeout, int maxTotal, int maxIdle, int minIdle, + boolean testOnBorrow, boolean testOnReturn, boolean testWhileIdle) { + super(connectionTimeout, maxTotal, maxIdle, minIdle, "dummy", testOnBorrow, testOnReturn, testWhileIdle); } } } diff --git a/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilderTest.java b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilderTest.java new file mode 100644 index 00000000..eac5ca04 --- /dev/null +++ b/flink-connector-redis/src/test/java/org/apache/flink/streaming/connectors/redis/common/container/RedisCommandsContainerBuilderTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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. + */ +package org.apache.flink.streaming.connectors.redis.common.container; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; + +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; +import org.apache.flink.streaming.connectors.redis.common.config.FlinkJedisPoolConfig; +import org.apache.flink.test.util.AbstractTestBase; +import org.junit.Test; +import redis.clients.jedis.JedisPoolConfig; + +public class RedisCommandsContainerBuilderTest extends AbstractTestBase { + + @Test + public void testNotTestWhileIdle() { + FlinkJedisPoolConfig flinkJedisPoolConfig = new FlinkJedisPoolConfig.Builder().setHost("host").setPort(0).setDatabase(0).build(); + GenericObjectPoolConfig genericObjectPoolConfig = RedisCommandsContainerBuilder.getGenericObjectPoolConfig(flinkJedisPoolConfig); + assertFalse(genericObjectPoolConfig.getTestWhileIdle()); + assertEqualConfig(flinkJedisPoolConfig, genericObjectPoolConfig); + } + + @Test + public void testTestWhileIdle() { + FlinkJedisPoolConfig flinkJedisPoolConfig = new FlinkJedisPoolConfig.Builder().setHost("host").setPort(0).setDatabase(0).setTestWhileIdle(true).build(); + GenericObjectPoolConfig genericObjectPoolConfig = RedisCommandsContainerBuilder.getGenericObjectPoolConfig(flinkJedisPoolConfig); + assertTrue(genericObjectPoolConfig.getTestWhileIdle()); + assertEqualConfig(flinkJedisPoolConfig, genericObjectPoolConfig); + + JedisPoolConfig jedisPoolConfig = new JedisPoolConfig(); + assertEquals(genericObjectPoolConfig.getMinEvictableIdleTimeMillis(), jedisPoolConfig.getMinEvictableIdleTimeMillis()); + assertEquals(genericObjectPoolConfig.getTimeBetweenEvictionRunsMillis(), jedisPoolConfig.getTimeBetweenEvictionRunsMillis()); + assertEquals(genericObjectPoolConfig.getNumTestsPerEvictionRun(), jedisPoolConfig.getNumTestsPerEvictionRun()); + } + + private void assertEqualConfig(FlinkJedisPoolConfig flinkJedisPoolConfig, GenericObjectPoolConfig genericObjectPoolConfig) { + assertEquals(genericObjectPoolConfig.getMaxIdle(), flinkJedisPoolConfig.getMaxIdle()); + assertEquals(genericObjectPoolConfig.getMinIdle(), flinkJedisPoolConfig.getMinIdle()); + assertEquals(genericObjectPoolConfig.getMaxTotal(), flinkJedisPoolConfig.getMaxTotal()); + assertEquals(genericObjectPoolConfig.getTestWhileIdle(), flinkJedisPoolConfig.getTestWhileIdle()); + assertEquals(genericObjectPoolConfig.getTestOnBorrow(), flinkJedisPoolConfig.getTestOnBorrow()); + assertEquals(genericObjectPoolConfig.getTestOnReturn(), flinkJedisPoolConfig.getTestOnReturn()); + } +} From 47d6dc7b14bf2e9fd861555d014882220f340e3c Mon Sep 17 00:00:00 2001 From: hackergin Date: Wed, 7 Apr 2021 06:35:07 -0500 Subject: [PATCH 13/14] [BAHIR-260] Add kudu table writer config (#109) --- .../kudu/connector/KuduTableInfo.java | 18 +++ .../kudu/connector/writer/KuduWriter.java | 5 + .../connector/writer/KuduWriterConfig.java | 113 +++++++++++++++++- .../kudu/table/KuduTableFactory.java | 49 +++++++- .../connectors/kudu/table/KuduTableSink.java | 21 ++++ .../kudu/table/KuduTableFactoryTest.java | 44 ++++++- 6 files changed, 243 insertions(+), 7 deletions(-) diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/KuduTableInfo.java b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/KuduTableInfo.java index 83c7dde4..baae8a05 100644 --- a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/KuduTableInfo.java +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/KuduTableInfo.java @@ -23,6 +23,7 @@ import org.apache.kudu.client.CreateTableOptions; import java.io.Serializable; +import java.util.Objects; /** * Describes which table should be used in sources and sinks along with specifications @@ -103,4 +104,21 @@ public CreateTableOptions getCreateTableOptions() { } return createTableOptionsFactory.getCreateTableOptions(); } + + @Override + public int hashCode() { + return Objects.hash(name); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KuduTableInfo that = (KuduTableInfo) o; + return Objects.equals(this.name, that.name); + } } diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriter.java b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriter.java index 03c37ea1..59ad196f 100644 --- a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriter.java +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriter.java @@ -72,6 +72,11 @@ private KuduClient obtainClient() { private KuduSession obtainSession() { KuduSession session = client.newSession(); session.setFlushMode(writerConfig.getFlushMode()); + session.setTimeoutMillis(writerConfig.getOperationTimeout()); + session.setMutationBufferSpace(writerConfig.getMaxBufferSize()); + session.setFlushInterval(writerConfig.getFlushInterval()); + session.setIgnoreAllDuplicateRows(writerConfig.isIgnoreDuplicate()); + session.setIgnoreAllNotFoundRows(writerConfig.isIgnoreNotFound()); return session; } diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriterConfig.java b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriterConfig.java index 598f8d0e..ff93921a 100644 --- a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriterConfig.java +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/connector/writer/KuduWriterConfig.java @@ -19,8 +19,10 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.kudu.client.AsyncKuduClient; import java.io.Serializable; +import java.util.Objects; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.kudu.client.SessionConfiguration.FlushMode; @@ -34,13 +36,28 @@ public class KuduWriterConfig implements Serializable { private final String masters; private final FlushMode flushMode; + private final long operationTimeout; + private int maxBufferSize; + private int flushInterval; + private boolean ignoreNotFound; + private boolean ignoreDuplicate; private KuduWriterConfig( String masters, - FlushMode flushMode) { + FlushMode flushMode, + long operationTimeout, + int maxBufferSize, + int flushInterval, + boolean ignoreNotFound, + boolean ignoreDuplicate) { this.masters = checkNotNull(masters, "Kudu masters cannot be null"); this.flushMode = checkNotNull(flushMode, "Kudu flush mode cannot be null"); + this.operationTimeout = operationTimeout; + this.maxBufferSize = maxBufferSize; + this.flushInterval = flushInterval; + this.ignoreNotFound = ignoreNotFound; + this.ignoreDuplicate = ignoreDuplicate; } public String getMasters() { @@ -51,6 +68,26 @@ public FlushMode getFlushMode() { return flushMode; } + public long getOperationTimeout() { + return operationTimeout; + } + + public int getMaxBufferSize() { + return maxBufferSize; + } + + public int getFlushInterval() { + return flushInterval; + } + + public boolean isIgnoreNotFound() { + return ignoreNotFound; + } + + public boolean isIgnoreDuplicate() { + return ignoreDuplicate; + } + @Override public String toString() { return new ToStringBuilder(this) @@ -65,6 +102,16 @@ public String toString() { public static class Builder { private String masters; private FlushMode flushMode = FlushMode.AUTO_FLUSH_BACKGROUND; + // Reference from AsyncKuduClientBuilder defaultOperationTimeoutMs. + private long timeout = AsyncKuduClient.DEFAULT_OPERATION_TIMEOUT_MS; + // Reference from AsyncKuduSession mutationBufferMaxOps 1000. + private int maxBufferSize = 1000; + // Reference from AsyncKuduSession flushIntervalMillis 1000. + private int flushInterval = 1000; + // Reference from AsyncKuduSession ignoreAllNotFoundRows false. + private boolean ignoreNotFound = false; + // Reference from AsyncKuduSession ignoreAllDuplicateRows false. + private boolean ignoreDuplicate = false; private Builder(String masters) { this.masters = masters; @@ -87,10 +134,72 @@ public Builder setStrongConsistency() { return setConsistency(FlushMode.AUTO_FLUSH_SYNC); } + public Builder setMaxBufferSize(int maxBufferSize) { + this.maxBufferSize = maxBufferSize; + return this; + } + + public Builder setFlushInterval(int flushInterval) { + this.flushInterval = flushInterval; + return this; + } + + public Builder setOperationTimeout(long timeout) { + this.timeout = timeout; + return this; + } + + public Builder setIgnoreNotFound(boolean ignoreNotFound) { + this.ignoreNotFound = ignoreNotFound; + return this; + } + + public Builder setIgnoreDuplicate(boolean ignoreDuplicate) { + this.ignoreDuplicate = ignoreDuplicate; + return this; + } + public KuduWriterConfig build() { return new KuduWriterConfig( masters, - flushMode); + flushMode, + timeout, + maxBufferSize, + flushInterval, + ignoreNotFound, + ignoreDuplicate); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + masters, + flushMode, + timeout, + maxBufferSize, + flushInterval, + ignoreNotFound, + ignoreDuplicate); + return result; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder that = (Builder) o; + return Objects.equals(masters, that.masters) + && Objects.equals(flushMode, that.flushMode) + && Objects.equals(timeout, that.timeout) + && Objects.equals(maxBufferSize, that.maxBufferSize) + && Objects.equals(flushInterval, that.flushInterval) + && Objects.equals(ignoreNotFound, that.ignoreNotFound) + && Objects.equals(ignoreDuplicate, that.ignoreDuplicate); } } } diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableFactory.java b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableFactory.java index 1961aad1..524f5218 100644 --- a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableFactory.java +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableFactory.java @@ -35,11 +35,28 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.*; -import static org.apache.flink.table.descriptors.Rowtime.*; -import static org.apache.flink.table.descriptors.Schema.*; +import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_CLASS; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_FROM; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_SERIALIZED; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_TYPE; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_CLASS; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_DELAY; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_SERIALIZED; +import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_TYPE; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_DATA_TYPE; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_FROM; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_PROCTIME; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE; import static org.apache.flink.util.Preconditions.checkNotNull; public class KuduTableFactory implements TableSourceFactory, TableSinkFactory> { @@ -49,6 +66,11 @@ public class KuduTableFactory implements TableSourceFactory, TableSinkFacto public static final String KUDU_HASH_COLS = "kudu.hash-columns"; public static final String KUDU_PRIMARY_KEY_COLS = "kudu.primary-key-columns"; public static final String KUDU_REPLICAS = "kudu.replicas"; + public static final String KUDU_MAX_BUFFER_SIZE = "kudu.max-buffer-size"; + public static final String KUDU_FLUSH_INTERVAL = "kudu.flush-interval"; + public static final String KUDU_OPERATION_TIMEOUT = "kudu.operation-timeout"; + public static final String KUDU_IGNORE_NOT_FOUND = "kudu.ignore-not-found"; + public static final String KUDU_IGNORE_DUPLICATE = "kudu.ignore-duplicate"; public static final String KUDU = "kudu"; @Override @@ -65,6 +87,11 @@ public List supportedProperties() { properties.add(KUDU_MASTERS); properties.add(KUDU_HASH_COLS); properties.add(KUDU_PRIMARY_KEY_COLS); + properties.add(KUDU_MAX_BUFFER_SIZE); + properties.add(KUDU_FLUSH_INTERVAL); + properties.add(KUDU_OPERATION_TIMEOUT); + properties.add(KUDU_IGNORE_NOT_FOUND); + properties.add(KUDU_IGNORE_DUPLICATE); // schema properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); properties.add(SCHEMA + ".#." + SCHEMA_TYPE); @@ -123,10 +150,12 @@ private KuduTableSource createTableSource(String tableName, TableSchema schema, public KuduTableSink createTableSink(ObjectPath tablePath, CatalogTable table) { validateTable(table); String tableName = table.toProperties().getOrDefault(KUDU_TABLE, tablePath.getObjectName()); - return createTableSink(tableName, table.getSchema(), table.getProperties()); + return createTableSink(tableName, table.getSchema(), table.toProperties()); } private KuduTableSink createTableSink(String tableName, TableSchema schema, Map props) { + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(props); String masterAddresses = props.get(KUDU_MASTERS); TableSchema physicalSchema = KuduTableUtils.getSchemaWithSqlTimestamp(schema); KuduTableInfo tableInfo = KuduTableUtils.createTableInfo(tableName, schema, props); @@ -134,6 +163,18 @@ private KuduTableSink createTableSink(String tableName, TableSchema schema, Map< KuduWriterConfig.Builder configBuilder = KuduWriterConfig.Builder .setMasters(masterAddresses); + Optional operationTimeout = properties.getOptionalLong(KUDU_OPERATION_TIMEOUT); + Optional flushInterval = properties.getOptionalInt(KUDU_FLUSH_INTERVAL); + Optional bufferSize = properties.getOptionalInt(KUDU_MAX_BUFFER_SIZE); + Optional ignoreNotFound = properties.getOptionalBoolean(KUDU_IGNORE_NOT_FOUND); + Optional ignoreDuplicate = properties.getOptionalBoolean(KUDU_IGNORE_DUPLICATE); + + operationTimeout.ifPresent(time -> configBuilder.setOperationTimeout(time)); + flushInterval.ifPresent(interval -> configBuilder.setFlushInterval(interval)); + bufferSize.ifPresent(size -> configBuilder.setMaxBufferSize(size)); + ignoreNotFound.ifPresent(i -> configBuilder.setIgnoreNotFound(i)); + ignoreDuplicate.ifPresent(i -> configBuilder.setIgnoreDuplicate(i)); + return new KuduTableSink(configBuilder, tableInfo, physicalSchema); } } diff --git a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableSink.java b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableSink.java index 99325c07..5ada84e2 100644 --- a/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableSink.java +++ b/flink-connector-kudu/src/main/java/org/apache/flink/connectors/kudu/table/KuduTableSink.java @@ -30,6 +30,8 @@ import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; +import java.util.Objects; + public class KuduTableSink implements UpsertStreamTableSink { private final KuduWriterConfig.Builder writerConfigBuilder; @@ -68,4 +70,23 @@ public TableSink> configure(String[] fieldNames, TypeInform @Override public TableSchema getTableSchema() { return flinkSchema; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || o.getClass() != this.getClass()) { + return false; + } + KuduTableSink that = (KuduTableSink) o; + return this.writerConfigBuilder.equals(that.writerConfigBuilder) && + this.flinkSchema.equals(that.flinkSchema) && + this.tableInfo.equals(that.tableInfo); + } + + @Override + public int hashCode() { + return Objects.hash(writerConfigBuilder, flinkSchema, tableInfo); + } } diff --git a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java index d4de7f68..a8ec6867 100644 --- a/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java +++ b/flink-connector-kudu/src/test/java/org/apache/flink/connectors/kudu/table/KuduTableFactoryTest.java @@ -16,12 +16,21 @@ */ package org.apache.flink.connectors.kudu.table; +import org.apache.flink.connectors.kudu.connector.KuduTableInfo; import org.apache.flink.connectors.kudu.connector.KuduTestBase; +import org.apache.flink.connectors.kudu.connector.writer.KuduWriterConfig; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.table.sinks.TableSink; import org.apache.kudu.Type; import org.apache.kudu.client.KuduScanner; import org.apache.kudu.client.KuduTable; @@ -32,12 +41,17 @@ import java.sql.Timestamp; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; public class KuduTableFactoryTest extends KuduTestBase { private StreamTableEnvironment tableEnv; @@ -153,4 +167,32 @@ public void testExistingTable() throws Exception { assertEquals("f2", rows.get(1).getString("first")); assertEquals("s2", rows.get(1).getString("second")); } + + @Test + public void testTableSink() { + final TableSchema schema = TableSchema.builder() + .field("first", DataTypes.STRING()) + .field("second", DataTypes.STRING()) + .build(); + final Map properties = new HashMap<>(); + properties.put("connector.type", "kudu"); + properties.put("kudu.masters", kuduMasters); + properties.put("kudu.table", "TestTable12"); + properties.put("kudu.ignore-not-found", "true"); + properties.put("kudu.ignore-duplicate", "true"); + properties.put("kudu.flush-interval", "10000"); + properties.put("kudu.max-buffer-size", "10000"); + + KuduWriterConfig.Builder builder = KuduWriterConfig.Builder.setMasters(kuduMasters) + .setFlushInterval(10000) + .setMaxBufferSize(10000) + .setIgnoreDuplicate(true) + .setIgnoreNotFound(true); + KuduTableInfo kuduTableInfo = KuduTableInfo.forTable("TestTable12"); + KuduTableSink expected = new KuduTableSink(builder, kuduTableInfo, schema); + final TableSink actualSink = TableFactoryService.find(TableSinkFactory.class, properties) + .createTableSink(ObjectPath.fromString("default.TestTable12"), new CatalogTableImpl(schema, properties, null)); + + assertEquals(expected, actualSink); + } } From 12ee9ca736e06189475939628572174322fbc49c Mon Sep 17 00:00:00 2001 From: yiksanchan Date: Wed, 7 Apr 2021 04:38:43 -0700 Subject: [PATCH 14/14] [BAHIR-270] Fix malformed artifactId (#122) --- flink-connector-redis/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-redis/pom.xml b/flink-connector-redis/pom.xml index 43767ed7..77d4393d 100644 --- a/flink-connector-redis/pom.xml +++ b/flink-connector-redis/pom.xml @@ -28,7 +28,7 @@ under the License. .. - flink-connector-redis_${scala.binary.version} + flink-connector-redis_2.11 flink-connector-redis jar