From ab2524ba0af2fc27d3952b600ac394af4b974672 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 18 Mar 2014 13:44:29 -0700 Subject: [PATCH 01/10] add router node --- .../java/io/druid/db/DatabaseRuleManager.java | 6 +- .../druid/db/DatabaseRuleManagerConfig.java | 6 +- .../druid/db/DatabaseRuleManagerProvider.java | 2 +- .../coordinator/rules/ForeverDropRule.java | 7 + .../coordinator/rules/ForeverLoadRule.java | 7 + .../coordinator/rules/IntervalDropRule.java | 8 +- .../coordinator/rules/IntervalLoadRule.java | 8 +- .../coordinator/rules/PeriodDropRule.java | 8 +- .../coordinator/rules/PeriodLoadRule.java | 8 +- .../druid/server/coordinator/rules/Rule.java | 5 +- .../druid/server/router/BrokerSelector.java | 86 ++++++++ .../server/router/CoordinatorRuleManager.java | 193 ++++++++++++++++++ .../router/RouterQuerySegmentWalker.java | 86 ++++++++ .../server/router/TierAwareQueryRunner.java | 163 +++++++++++++++ .../io/druid/server/router/TierConfig.java | 92 +++++++++ .../coordinator/rules/LoadRuleTest.java | 12 ++ .../src/main/java/io/druid/cli/CliRouter.java | 105 ++++++++++ services/src/main/java/io/druid/cli/Main.java | 2 +- 18 files changed, 791 insertions(+), 13 deletions(-) create mode 100644 server/src/main/java/io/druid/server/router/BrokerSelector.java create mode 100644 server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java create mode 100644 server/src/main/java/io/druid/server/router/RouterQuerySegmentWalker.java create mode 100644 server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java create mode 100644 server/src/main/java/io/druid/server/router/TierConfig.java create mode 100644 services/src/main/java/io/druid/cli/CliRouter.java diff --git a/server/src/main/java/io/druid/db/DatabaseRuleManager.java b/server/src/main/java/io/druid/db/DatabaseRuleManager.java index 9a1c014b36f..5882c964346 100644 --- a/server/src/main/java/io/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/io/druid/db/DatabaseRuleManager.java @@ -159,7 +159,7 @@ public class DatabaseRuleManager this.exec = Execs.scheduledSingleThreaded("DatabaseRuleManager-Exec--%d"); - createDefaultRule(dbi, getRulesTable(), config.get().getDefaultTier(), jsonMapper); + createDefaultRule(dbi, getRulesTable(), config.get().getDefaultRule(), jsonMapper); ScheduledExecutors.scheduleWithFixedDelay( exec, new Duration(0), @@ -274,8 +274,8 @@ public class DatabaseRuleManager if (theRules.get(dataSource) != null) { retVal.addAll(theRules.get(dataSource)); } - if (theRules.get(config.get().getDefaultTier()) != null) { - retVal.addAll(theRules.get(config.get().getDefaultTier())); + if (theRules.get(config.get().getDefaultRule()) != null) { + retVal.addAll(theRules.get(config.get().getDefaultRule())); } return retVal; } diff --git a/server/src/main/java/io/druid/db/DatabaseRuleManagerConfig.java b/server/src/main/java/io/druid/db/DatabaseRuleManagerConfig.java index 8b770a57955..63f506649b5 100644 --- a/server/src/main/java/io/druid/db/DatabaseRuleManagerConfig.java +++ b/server/src/main/java/io/druid/db/DatabaseRuleManagerConfig.java @@ -27,14 +27,14 @@ import org.joda.time.Period; public class DatabaseRuleManagerConfig { @JsonProperty - private String defaultTier = "_default"; + private String defaultRule = "_default"; @JsonProperty private Period pollDuration = new Period("PT1M"); - public String getDefaultTier() + public String getDefaultRule() { - return defaultTier; + return defaultRule; } public Period getPollDuration() diff --git a/server/src/main/java/io/druid/db/DatabaseRuleManagerProvider.java b/server/src/main/java/io/druid/db/DatabaseRuleManagerProvider.java index dfc25bea283..0296f25867e 100644 --- a/server/src/main/java/io/druid/db/DatabaseRuleManagerProvider.java +++ b/server/src/main/java/io/druid/db/DatabaseRuleManagerProvider.java @@ -64,7 +64,7 @@ public class DatabaseRuleManagerProvider implements Provider +{ + private final CoordinatorRuleManager ruleManager; + private final TierConfig tierConfig; + + @Inject + public BrokerSelector(CoordinatorRuleManager ruleManager, TierConfig tierConfig) + { + this.ruleManager = ruleManager; + this.tierConfig = tierConfig; + } + + public String select(final Query query) + { + if (!ruleManager.isStarted()) { + return null; + } + + List rules = ruleManager.getRulesWithDefault((query.getDataSource()).getName()); + + // find the rule that can apply to the entire set of intervals + DateTime now = new DateTime(); + int lastRulePosition = -1; + LoadRule baseRule = null; + + for (Interval interval : query.getIntervals()) { + int currRulePosition = 0; + for (Rule rule : rules) { + if (rule instanceof LoadRule && currRulePosition > lastRulePosition && rule.appliesTo(interval, now)) { + lastRulePosition = currRulePosition; + baseRule = (LoadRule) rule; + break; + } + currRulePosition++; + } + } + + if (baseRule == null) { + return null; + } + + // in the baseRule, find the broker of highest priority + String brokerName = null; + for (Map.Entry entry : tierConfig.getTierToBrokerMap().entrySet()) { + if (baseRule.getTieredReplicants().containsKey(entry.getKey())) { + brokerName = entry.getValue(); + break; + } + } + + return brokerName; + } +} diff --git a/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java b/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java new file mode 100644 index 00000000000..9acf5f94858 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java @@ -0,0 +1,193 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Charsets; +import com.google.common.base.Supplier; +import com.google.common.collect.Lists; +import com.google.inject.Inject; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; +import io.druid.client.selector.Server; +import io.druid.concurrent.Execs; +import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Json; +import io.druid.server.coordinator.rules.Rule; +import org.joda.time.Duration; + +import java.net.URL; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +@ManageLifecycle +public class CoordinatorRuleManager +{ + private static final Logger log = new Logger(CoordinatorRuleManager.class); + + private final HttpClient httpClient; + private final ObjectMapper jsonMapper; + private final Supplier config; + private final ServerDiscoverySelector selector; + + private final StatusResponseHandler responseHandler; + private final AtomicReference>> rules; + + private volatile ScheduledExecutorService exec; + + private final Object lock = new Object(); + + private volatile boolean started = false; + + @Inject + public CoordinatorRuleManager( + @Global HttpClient httpClient, + @Json ObjectMapper jsonMapper, + Supplier config, + ServerDiscoverySelector selector + ) + { + this.httpClient = httpClient; + this.jsonMapper = jsonMapper; + this.config = config; + this.selector = selector; + + this.responseHandler = new StatusResponseHandler(Charsets.UTF_8); + this.rules = new AtomicReference<>( + new ConcurrentHashMap>() + ); + } + + @LifecycleStart + public void start() + { + synchronized (lock) { + if (started) { + return; + } + + this.exec = Execs.scheduledSingleThreaded("CoordinatorRuleManager-Exec--%d"); + + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(0), + config.get().getPollPeriod().toStandardDuration(), + new Runnable() + { + @Override + public void run() + { + poll(); + } + } + ); + + started = true; + } + } + + @LifecycleStop + public void stop() + { + synchronized (lock) { + if (!started) { + return; + } + + rules.set(new ConcurrentHashMap>()); + + started = false; + exec.shutdownNow(); + exec = null; + } + } + + public boolean isStarted() + { + return started; + } + + public void poll() + { + try { + String url = getRuleURL(); + if (url == null) { + return; + } + + StatusResponseHolder response = httpClient.get(new URL(url)) + .go(responseHandler) + .get(); + + ConcurrentHashMap> newRules = new ConcurrentHashMap>( + (Map>) jsonMapper.readValue( + response.getContent(), new TypeReference>>() + { + } + ) + ); + + log.info("Got [%,d] rules", newRules.keySet().size()); + + rules.set(newRules); + } + catch (Exception e) { + log.error(e, "Exception while polling for rules"); + } + } + + public List getRulesWithDefault(final String dataSource) + { + List retVal = Lists.newArrayList(); + Map> theRules = rules.get(); + if (theRules.get(dataSource) != null) { + retVal.addAll(theRules.get(dataSource)); + } + if (theRules.get(config.get().getDefaultRule()) != null) { + retVal.addAll(theRules.get(config.get().getDefaultRule())); + } + return retVal; + } + + private String getRuleURL() + { + Server server = selector.pick(); + + if (server == null) { + log.error("No instances found for [%s]!", config.get().getCoordinatorServiceName()); + return null; + } + + return String.format("http://%s%s", server.getHost(), config.get().getRulesEndpoint()); + } +} diff --git a/server/src/main/java/io/druid/server/router/RouterQuerySegmentWalker.java b/server/src/main/java/io/druid/server/router/RouterQuerySegmentWalker.java new file mode 100644 index 00000000000..705e12fb8d6 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/RouterQuerySegmentWalker.java @@ -0,0 +1,86 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.metamx.http.client.HttpClient; +import io.druid.curator.discovery.ServerDiscoveryFactory; +import io.druid.guice.annotations.Global; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; +import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.SegmentDescriptor; +import org.joda.time.Interval; + +/** + */ +public class RouterQuerySegmentWalker implements QuerySegmentWalker +{ + private final QueryToolChestWarehouse warehouse; + private final ObjectMapper objectMapper; + private final HttpClient httpClient; + private final BrokerSelector brokerSelector; + private final TierConfig tierConfig; + private final ServerDiscoveryFactory serverDiscoveryFactory; + + @Inject + public RouterQuerySegmentWalker( + QueryToolChestWarehouse warehouse, + ObjectMapper objectMapper, + @Global HttpClient httpClient, + BrokerSelector brokerSelector, + TierConfig tierConfig, + ServerDiscoveryFactory serverDiscoveryFactory + ) + { + this.warehouse = warehouse; + this.objectMapper = objectMapper; + this.httpClient = httpClient; + this.brokerSelector = brokerSelector; + this.tierConfig = tierConfig; + this.serverDiscoveryFactory = serverDiscoveryFactory; + } + + @Override + public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) + { + return makeRunner(); + } + + @Override + public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) + { + return makeRunner(); + } + + private QueryRunner makeRunner() + { + return new TierAwareQueryRunner( + warehouse, + objectMapper, + httpClient, + brokerSelector, + tierConfig, + serverDiscoveryFactory + ); + } +} diff --git a/server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java b/server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java new file mode 100644 index 00000000000..5acb1b6dd8c --- /dev/null +++ b/server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java @@ -0,0 +1,163 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.emitter.EmittingLogger; +import com.metamx.http.client.HttpClient; +import io.druid.client.DirectDruidClient; +import io.druid.client.selector.Server; +import io.druid.curator.discovery.ServerDiscoveryFactory; +import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChestWarehouse; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + */ +public class TierAwareQueryRunner implements QueryRunner +{ + private static EmittingLogger log = new EmittingLogger(TierAwareQueryRunner.class); + + private final QueryToolChestWarehouse warehouse; + private final ObjectMapper objectMapper; + private final HttpClient httpClient; + private final BrokerSelector brokerSelector; + private final TierConfig tierConfig; + + private final ConcurrentHashMap selectorMap = new ConcurrentHashMap(); + private final ConcurrentHashMap serverBackup = new ConcurrentHashMap(); + + public TierAwareQueryRunner( + QueryToolChestWarehouse warehouse, + ObjectMapper objectMapper, + HttpClient httpClient, + BrokerSelector brokerSelector, + TierConfig tierConfig, + ServerDiscoveryFactory serverDiscoveryFactory + ) + { + this.warehouse = warehouse; + this.objectMapper = objectMapper; + this.httpClient = httpClient; + this.brokerSelector = brokerSelector; + this.tierConfig = tierConfig; + + try { + for (Map.Entry entry : tierConfig.getTierToBrokerMap().entrySet()) { + ServerDiscoverySelector selector = serverDiscoveryFactory.createSelector(entry.getValue()); + selector.start(); + // TODO: stop? + selectorMap.put(entry.getValue(), selector); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public Server findServer(Query query) + { + String brokerServiceName = brokerSelector.select(query); + + if (brokerServiceName == null) { + log.error( + "WTF?! No brokerServiceName found for datasource[%s], intervals[%s]. Using default[%s].", + query.getDataSource(), + query.getIntervals(), + tierConfig.getDefaultBrokerServiceName() + ); + brokerServiceName = tierConfig.getDefaultBrokerServiceName(); + } + + ServerDiscoverySelector selector = selectorMap.get(brokerServiceName); + + Server server; + if (selector == null) { + log.error( + "WTF?! No selector found for brokerServiceName[%s]. Using default selector for[%s]", + brokerServiceName, + tierConfig.getDefaultBrokerServiceName() + ); + selector = selectorMap.get(tierConfig.getDefaultBrokerServiceName()); + + if (selector != null) { + server = selector.pick(); + } else { + return null; + } + } else { + server = selector.pick(); + } + + if (server == null) { + log.error( + "WTF?! No server found for brokerServiceName[%s]. Using backup", + brokerServiceName + ); + + server = serverBackup.get(brokerServiceName); + + if (server == null) { + log.error( + "WTF?! No backup found for brokerServiceName[%s]. Using default[%s]", + brokerServiceName, + tierConfig.getDefaultBrokerServiceName() + ); + + server = serverBackup.get(tierConfig.getDefaultBrokerServiceName()); + } + } else { + serverBackup.put(brokerServiceName, server); + } + + return server; + } + + @Override + public Sequence run(Query query) + { + Server server = findServer(query); + + if (server == null) { + log.makeAlert( + "Catastrophic failure! No servers found for default broker [%s]!", + tierConfig.getDefaultBrokerServiceName() + ).emit(); + + return Sequences.empty(); + } + + QueryRunner client = new DirectDruidClient( + warehouse, + objectMapper, + httpClient, + server.getHost() + ); + + return client.run(query); + } +} diff --git a/server/src/main/java/io/druid/server/router/TierConfig.java b/server/src/main/java/io/druid/server/router/TierConfig.java new file mode 100644 index 00000000000..8fba0c6255d --- /dev/null +++ b/server/src/main/java/io/druid/server/router/TierConfig.java @@ -0,0 +1,92 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; +import io.druid.client.DruidServer; +import org.joda.time.Period; + +import javax.validation.constraints.NotNull; +import java.util.LinkedHashMap; + +/** + */ +public class TierConfig +{ + @JsonProperty + @NotNull + private String defaultBrokerServiceName = ""; + + @JsonProperty + @NotNull + private LinkedHashMap tierToBrokerMap = new LinkedHashMap( + ImmutableMap.of( + DruidServer.DEFAULT_TIER, defaultBrokerServiceName + ) + ); + + @JsonProperty + @NotNull + private String defaultRule = "_default"; + + @JsonProperty + @NotNull + private String rulesEndpoint = "/druid/coordinator/v1/rules"; + + @JsonProperty + @NotNull + private String coordinatorServiceName = null; + + @JsonProperty + @NotNull + private Period pollPeriod = new Period("PT1M"); + + // tier, + public LinkedHashMap getTierToBrokerMap() + { + return tierToBrokerMap; + } + + public String getDefaultBrokerServiceName() + { + return defaultBrokerServiceName; + } + + public String getDefaultRule() + { + return defaultRule; + } + + public String getRulesEndpoint() + { + return rulesEndpoint; + } + + public String getCoordinatorServiceName() + { + return coordinatorServiceName; + } + + public Period getPollPeriod() + { + return pollPeriod; + } +} diff --git a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java index 3e75d1b79c3..24542e70df2 100644 --- a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java @@ -122,6 +122,12 @@ public class LoadRuleTest { return true; } + + @Override + public boolean appliesTo(Interval interval, DateTime referenceTimestamp) + { + return true; + } }; DruidCluster druidCluster = new DruidCluster( @@ -214,6 +220,12 @@ public class LoadRuleTest { return true; } + + @Override + public boolean appliesTo(Interval interval, DateTime referenceTimestamp) + { + return true; + } }; DruidServer server1 = new DruidServer( diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java new file mode 100644 index 00000000000..7a29594e306 --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -0,0 +1,105 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.cli; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.logger.Logger; +import io.airlift.command.Command; +import io.druid.curator.discovery.DiscoveryModule; +import io.druid.curator.discovery.ServerDiscoveryFactory; +import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.guice.Jerseys; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleModule; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.annotations.Self; +import io.druid.query.MapQueryToolChestWarehouse; +import io.druid.query.QuerySegmentWalker; +import io.druid.query.QueryToolChestWarehouse; +import io.druid.server.QueryResource; +import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.router.BrokerSelector; +import io.druid.server.router.CoordinatorRuleManager; +import io.druid.server.router.RouterQuerySegmentWalker; +import io.druid.server.router.TierConfig; +import org.eclipse.jetty.server.Server; + +import java.util.List; + +/** + */ +@Command( + name = "router", + description = "Experimental! Understands tiers and routes things to different brokers" +) +public class CliRouter extends ServerRunnable +{ + private static final Logger log = new Logger(CliRouter.class); + + public CliRouter() + { + super(log); + } + + @Override + protected List getModules() + { + return ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.broker", TierConfig.class); + + binder.bind(CoordinatorRuleManager.class); + LifecycleModule.register(binder, CoordinatorRuleManager.class); + + binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); + + binder.bind(BrokerSelector.class).in(LazySingleton.class); + binder.bind(QuerySegmentWalker.class).to(RouterQuerySegmentWalker.class).in(LazySingleton.class); + + binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); + Jerseys.addResource(binder, QueryResource.class); + LifecycleModule.register(binder, QueryResource.class); + + LifecycleModule.register(binder, Server.class); + DiscoveryModule.register(binder, Self.class); + } + + @Provides + @ManageLifecycle + public ServerDiscoverySelector getCoordinatorServerDiscoverySelector( + TierConfig config, + ServerDiscoveryFactory factory + + ) + { + return factory.createSelector(config.getCoordinatorServiceName()); + } + } + ); + } +} diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 7012e3a3f79..450aa36afe9 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -52,7 +52,7 @@ public class Main .withCommands( CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliOverlord.class, CliMiddleManager.class, - CliBridge.class + CliBridge.class, CliRouter.class ); builder.withGroup("example") From 76ef9d192c26483725b11d47aeb19e32da7e1b13 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 18 Mar 2014 13:47:53 -0700 Subject: [PATCH 02/10] add tests --- .../server/router/BrokerSelectorTest.java | 200 ++++++++++++++++++ .../router/TierAwareQueryRunnerTest.java | 142 +++++++++++++ 2 files changed, 342 insertions(+) create mode 100644 server/src/test/java/io/druid/server/router/BrokerSelectorTest.java create mode 100644 server/src/test/java/io/druid/server/router/TierAwareQueryRunnerTest.java diff --git a/server/src/test/java/io/druid/server/router/BrokerSelectorTest.java b/server/src/test/java/io/druid/server/router/BrokerSelectorTest.java new file mode 100644 index 00000000000..4a470d75d51 --- /dev/null +++ b/server/src/test/java/io/druid/server/router/BrokerSelectorTest.java @@ -0,0 +1,200 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableMap; +import com.metamx.http.client.HttpClient; +import io.druid.client.DruidServer; +import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Json; +import io.druid.query.Druids; +import io.druid.query.TableDataSource; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.server.coordinator.rules.IntervalLoadRule; +import io.druid.server.coordinator.rules.Rule; +import junit.framework.Assert; +import org.joda.time.Interval; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; + +/** + */ +public class BrokerSelectorTest +{ + private BrokerSelector brokerSelector; + + @Before + public void setUp() throws Exception + { + brokerSelector = new BrokerSelector( + new TestRuleManager(null, null, null, null), + new TierConfig() + { + @Override + public LinkedHashMap getTierToBrokerMap() + { + return new LinkedHashMap( + ImmutableMap.of( + "hot", "hotBroker", + "medium", "mediumBroker", + DruidServer.DEFAULT_TIER, "coldBroker" + ) + ); + } + + @Override + public String getDefaultBrokerServiceName() + { + return "hotBroker"; + } + } + ); + } + + @Test + public void testBasicSelect() throws Exception + { + String brokerName = brokerSelector.select( + new TimeBoundaryQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(Arrays.asList(new Interval("2011-08-31/2011-09-01"))), + null + ) + ); + + Assert.assertEquals("coldBroker", brokerName); + } + + + @Test + public void testBasicSelect2() throws Exception + { + String brokerName = brokerSelector.select( + new TimeBoundaryQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(Arrays.asList(new Interval("2013-08-31/2013-09-01"))), + null + ) + ); + + Assert.assertEquals("hotBroker", brokerName); + } + + @Test + public void testSelectMatchesNothing() throws Exception + { + String brokerName = brokerSelector.select( + new TimeBoundaryQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(Arrays.asList(new Interval("2010-08-31/2010-09-01"))), + null + ) + ); + + Assert.assertEquals(null, brokerName); + } + + + @Test + public void testSelectMultiInterval() throws Exception + { + String brokerName = brokerSelector.select( + Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) + .intervals( + new MultipleIntervalSegmentSpec( + Arrays.asList( + new Interval("2013-08-31/2013-09-01"), + new Interval("2012-08-31/2012-09-01"), + new Interval("2011-08-31/2011-09-01") + ) + ) + ).build() + ); + + Assert.assertEquals("coldBroker", brokerName); + } + + @Test + public void testSelectMultiInterval2() throws Exception + { + String brokerName = brokerSelector.select( + Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) + .intervals( + new MultipleIntervalSegmentSpec( + Arrays.asList( + new Interval("2011-08-31/2011-09-01"), + new Interval("2012-08-31/2012-09-01"), + new Interval("2013-08-31/2013-09-01") + ) + ) + ).build() + ); + + Assert.assertEquals("coldBroker", brokerName); + } + + private static class TestRuleManager extends CoordinatorRuleManager + { + public TestRuleManager( + @Global HttpClient httpClient, + @Json ObjectMapper jsonMapper, + Supplier config, + ServerDiscoverySelector selector + ) + { + super(httpClient, jsonMapper, config, selector); + } + + @Override + public boolean isStarted() + { + return true; + } + + @Override + public List getRulesWithDefault(String dataSource) + { + return Arrays.asList( + new IntervalLoadRule(new Interval("2013/2014"), ImmutableMap.of("hot", 1), null, null), + new IntervalLoadRule(new Interval("2012/2013"), ImmutableMap.of("medium", 1), null, null), + new IntervalLoadRule( + new Interval("2011/2012"), + ImmutableMap.of(DruidServer.DEFAULT_TIER, 1), + null, + null + ) + ); + } + } +} diff --git a/server/src/test/java/io/druid/server/router/TierAwareQueryRunnerTest.java b/server/src/test/java/io/druid/server/router/TierAwareQueryRunnerTest.java new file mode 100644 index 00000000000..f7301229c46 --- /dev/null +++ b/server/src/test/java/io/druid/server/router/TierAwareQueryRunnerTest.java @@ -0,0 +1,142 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package io.druid.server.router; + +import com.google.common.collect.ImmutableMap; +import io.druid.client.DruidServer; +import io.druid.client.selector.Server; +import io.druid.curator.discovery.ServerDiscoveryFactory; +import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.query.TableDataSource; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import junit.framework.Assert; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.LinkedHashMap; + +/** + */ +public class TierAwareQueryRunnerTest +{ + private ServerDiscoveryFactory factory; + private ServerDiscoverySelector selector; + private TierConfig config; + private Server server; + + @Before + public void setUp() throws Exception + { + factory = EasyMock.createMock(ServerDiscoveryFactory.class); + selector = EasyMock.createMock(ServerDiscoverySelector.class); + + config = new TierConfig() + { + @Override + public LinkedHashMap getTierToBrokerMap() + { + return new LinkedHashMap<>( + ImmutableMap.of( + "hot", "hotBroker", + "medium", "mediumBroker", + DruidServer.DEFAULT_TIER, "coldBroker" + ) + ); + } + + @Override + public String getDefaultBrokerServiceName() + { + return "hotBroker"; + } + }; + + server = new Server() + { + @Override + public String getScheme() + { + return null; + } + + @Override + public String getHost() + { + return "foo"; + } + + @Override + public String getAddress() + { + return null; + } + + @Override + public int getPort() + { + return 0; + } + }; + } + + @After + public void tearDown() throws Exception + { + EasyMock.verify(selector); + EasyMock.verify(factory); + } + + @Test + public void testFindServer() throws Exception + { + selector.start(); + EasyMock.expectLastCall().atLeastOnce(); + EasyMock.expect(selector.pick()).andReturn(server).once(); + EasyMock.replay(selector); + + EasyMock.expect(factory.createSelector(EasyMock.anyObject())).andReturn(selector).atLeastOnce(); + EasyMock.replay(factory); + + + TierAwareQueryRunner queryRunner = new TierAwareQueryRunner( + null, + null, + null, + new BrokerSelector(new CoordinatorRuleManager(null, null, null, null), config), + config, + factory + ); + + Server server = queryRunner.findServer( + new TimeBoundaryQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(Arrays.asList(new Interval("2011-08-31/2011-09-01"))), + null + ) + ); + + Assert.assertEquals("foo", server.getHost()); + } +} From a96dfc7742444e7b951106bdaa60e5c7a02ee0b0 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 19 Mar 2014 10:30:52 -0700 Subject: [PATCH 03/10] clean up failure scenarios --- .../server/router/TierAwareQueryRunner.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java b/server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java index 5acb1b6dd8c..381d3c2e82d 100644 --- a/server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java +++ b/server/src/main/java/io/druid/server/router/TierAwareQueryRunner.java @@ -84,12 +84,12 @@ public class TierAwareQueryRunner implements QueryRunner String brokerServiceName = brokerSelector.select(query); if (brokerServiceName == null) { - log.error( + log.makeAlert( "WTF?! No brokerServiceName found for datasource[%s], intervals[%s]. Using default[%s].", query.getDataSource(), query.getIntervals(), tierConfig.getDefaultBrokerServiceName() - ); + ).emit(); brokerServiceName = tierConfig.getDefaultBrokerServiceName(); } @@ -97,11 +97,11 @@ public class TierAwareQueryRunner implements QueryRunner Server server; if (selector == null) { - log.error( + log.makeAlert( "WTF?! No selector found for brokerServiceName[%s]. Using default selector for[%s]", brokerServiceName, tierConfig.getDefaultBrokerServiceName() - ); + ).emit(); selector = selectorMap.get(tierConfig.getDefaultBrokerServiceName()); if (selector != null) { @@ -122,11 +122,11 @@ public class TierAwareQueryRunner implements QueryRunner server = serverBackup.get(brokerServiceName); if (server == null) { - log.error( + log.makeAlert( "WTF?! No backup found for brokerServiceName[%s]. Using default[%s]", brokerServiceName, tierConfig.getDefaultBrokerServiceName() - ); + ).emit(); server = serverBackup.get(tierConfig.getDefaultBrokerServiceName()); } @@ -144,10 +144,8 @@ public class TierAwareQueryRunner implements QueryRunner if (server == null) { log.makeAlert( - "Catastrophic failure! No servers found for default broker [%s]!", - tierConfig.getDefaultBrokerServiceName() + "Catastrophic failure! No servers found at all! Failing request!" ).emit(); - return Sequences.empty(); } From 7f8211ecc7b516c1d76855b03fcd7631c2e38b6e Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 20 Mar 2014 13:41:45 -0700 Subject: [PATCH 04/10] extend context to use map string object instead of map string string --- .../main/java/io/druid/query/BaseQuery.java | 49 ++++++++++++------- .../io/druid/query/BySegmentQueryRunner.java | 2 +- .../query/BySegmentSkippingQueryRunner.java | 2 +- .../query/ChainedExecutionQueryRunner.java | 2 +- .../src/main/java/io/druid/query/Druids.java | 14 +++--- .../query/FinalizeResultsQueryRunner.java | 4 +- .../query/GroupByParallelQueryRunner.java | 2 +- .../src/main/java/io/druid/query/Query.java | 6 +-- .../io/druid/query/groupby/GroupByQuery.java | 10 ++-- .../groupby/GroupByQueryQueryToolChest.java | 4 +- .../metadata/SegmentMetadataQuery.java | 4 +- .../search/SearchQueryQueryToolChest.java | 2 +- .../query/search/search/SearchQuery.java | 4 +- .../io/druid/query/select/SelectQuery.java | 4 +- .../query/timeboundary/TimeBoundaryQuery.java | 4 +- .../query/timeseries/TimeseriesQuery.java | 4 +- .../java/io/druid/query/topn/TopNQuery.java | 4 +- .../io/druid/query/topn/TopNQueryBuilder.java | 6 +-- .../query/topn/TopNQueryQueryToolChest.java | 2 +- .../druid/client/CachingClusteredClient.java | 10 ++-- .../client/CachingClusteredClientTest.java | 8 +-- 21 files changed, 80 insertions(+), 67 deletions(-) diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index ed13f9ddf39..71beaa26652 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -37,14 +37,14 @@ public abstract class BaseQuery implements Query { public static String QUERYID = "queryId"; private final DataSource dataSource; - private final Map context; + private final Map context; private final QuerySegmentSpec querySegmentSpec; private volatile Duration duration; public BaseQuery( DataSource dataSource, QuerySegmentSpec querySegmentSpec, - Map context + Map context ) { Preconditions.checkNotNull(dataSource, "dataSource can't be null"); @@ -102,28 +102,28 @@ public abstract class BaseQuery implements Query } @JsonProperty - public Map getContext() + public Map getContext() { return context; } @Override - public String getContextValue(String key) + public ContextType getContextValue(String key) { - return context == null ? null : context.get(key); + return context == null ? null : (ContextType) context.get(key); } @Override - public String getContextValue(String key, String defaultValue) + public ContextType getContextValue(String key, ContextType defaultValue) { - String retVal = getContextValue(key); + ContextType retVal = getContextValue(key); return retVal == null ? defaultValue : retVal; } - protected Map computeOverridenContext(Map overrides) + protected Map computeOverridenContext(Map overrides) { - Map overridden = Maps.newTreeMap(); - final Map context = getContext(); + Map overridden = Maps.newTreeMap(); + final Map context = getContext(); if (context != null) { overridden.putAll(context); } @@ -135,28 +135,41 @@ public abstract class BaseQuery implements Query @Override public String getId() { - return getContextValue(QUERYID); + return (String) getContextValue(QUERYID); } @Override public Query withId(String id) { - return withOverriddenContext(ImmutableMap.of(QUERYID, id)); + return withOverriddenContext(ImmutableMap.of(QUERYID, id)); } @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } BaseQuery baseQuery = (BaseQuery) o; - if (context != null ? !context.equals(baseQuery.context) : baseQuery.context != null) return false; - if (dataSource != null ? !dataSource.equals(baseQuery.dataSource) : baseQuery.dataSource != null) return false; - if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) return false; - if (querySegmentSpec != null ? !querySegmentSpec.equals(baseQuery.querySegmentSpec) : baseQuery.querySegmentSpec != null) + if (context != null ? !context.equals(baseQuery.context) : baseQuery.context != null) { return false; + } + if (dataSource != null ? !dataSource.equals(baseQuery.dataSource) : baseQuery.dataSource != null) { + return false; + } + if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) { + return false; + } + if (querySegmentSpec != null + ? !querySegmentSpec.equals(baseQuery.querySegmentSpec) + : baseQuery.querySegmentSpec != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index 79c6a4e03a9..d6150f63456 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -53,7 +53,7 @@ public class BySegmentQueryRunner implements QueryRunner @SuppressWarnings("unchecked") public Sequence run(final Query query) { - if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { + if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { final Sequence baseSequence = base.run(query); return new Sequence() { diff --git a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java index 1711fb316f9..8e666c30b16 100644 --- a/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java @@ -37,7 +37,7 @@ public abstract class BySegmentSkippingQueryRunner implements QueryRunner @Override public Sequence run(Query query) { - if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { + if (Boolean.parseBoolean(query.getContextValue("bySegment"))) { return baseRunner.run(query); } diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 83d2ff48f98..d3600068a23 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -83,7 +83,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner @Override public Sequence run(final Query query) { - final int priority = Integer.parseInt(query.getContextValue("priority", "0")); + final int priority = Integer.parseInt((String) query.getContextValue("priority", "0")); return new BaseSequence>( new BaseSequence.IteratorMaker>() diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index fd2abd13ab8..3ab6b0a8ff7 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -304,7 +304,7 @@ public class Druids private QueryGranularity granularity; private List aggregatorSpecs; private List postAggregatorSpecs; - private Map context; + private Map context; private TimeseriesQueryBuilder() { @@ -384,7 +384,7 @@ public class Druids return postAggregatorSpecs; } - public Map getContext() + public Map getContext() { return context; } @@ -465,7 +465,7 @@ public class Druids return this; } - public TimeseriesQueryBuilder context(Map c) + public TimeseriesQueryBuilder context(Map c) { context = c; return this; @@ -505,7 +505,7 @@ public class Druids private QuerySegmentSpec querySegmentSpec; private List dimensions; private SearchQuerySpec querySpec; - private Map context; + private Map context; public SearchQueryBuilder() { @@ -660,7 +660,7 @@ public class Druids return this; } - public SearchQueryBuilder context(Map c) + public SearchQueryBuilder context(Map c) { context = c; return this; @@ -690,7 +690,7 @@ public class Druids { private DataSource dataSource; private QuerySegmentSpec querySegmentSpec; - private Map context; + private Map context; public TimeBoundaryQueryBuilder() { @@ -746,7 +746,7 @@ public class Druids return this; } - public TimeBoundaryQueryBuilder context(Map c) + public TimeBoundaryQueryBuilder context(Map c) { context = c; return this; diff --git a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java index 77823a60730..2880332e184 100644 --- a/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -48,7 +48,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner @Override public Sequence run(final Query query) { - final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment")); + final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment")); final boolean shouldFinalize = Boolean.parseBoolean(query.getContextValue("finalize", "true")); if (shouldFinalize) { Function finalizerFn; @@ -100,7 +100,7 @@ public class FinalizeResultsQueryRunner implements QueryRunner } return Sequences.map( - baseRunner.run(query.withOverriddenContext(ImmutableMap.of("finalize", "false"))), + baseRunner.run(query.withOverriddenContext(ImmutableMap.of("finalize", "false"))), finalizerFn ); } diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index fb98968fb43..10dde9b26ea 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -83,7 +83,7 @@ public class GroupByParallelQueryRunner implements QueryRunner query, configSupplier.get() ); - final int priority = Integer.parseInt(query.getContextValue("priority", "0")); + final int priority = Integer.parseInt((String) query.getContextValue("priority", "0")); if (Iterables.isEmpty(queryables)) { log.warn("No queryables found."); diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index d58798539ba..10a84328584 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -70,11 +70,11 @@ public interface Query public Duration getDuration(); - public String getContextValue(String key); + public ContextType getContextValue(String key); - public String getContextValue(String key, String defaultValue); + public ContextType getContextValue(String key, ContextType defaultValue); - public Query withOverriddenContext(Map contextOverride); + public Query withOverriddenContext(Map contextOverride); public Query withQuerySegmentSpec(QuerySegmentSpec spec); diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 80322a29531..3f04f30b8aa 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -86,7 +86,7 @@ public class GroupByQuery extends BaseQuery @JsonProperty("having") HavingSpec havingSpec, @JsonProperty("limitSpec") LimitSpec limitSpec, @JsonProperty("orderBy") LimitSpec orderBySpec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context ) { super(dataSource, querySegmentSpec, context); @@ -147,7 +147,7 @@ public class GroupByQuery extends BaseQuery HavingSpec havingSpec, LimitSpec orderBySpec, Function, Sequence> orderByLimitFn, - Map context + Map context ) { super(dataSource, querySegmentSpec, context); @@ -222,7 +222,7 @@ public class GroupByQuery extends BaseQuery } @Override - public GroupByQuery withOverriddenContext(Map contextOverride) + public GroupByQuery withOverriddenContext(Map contextOverride) { return new GroupByQuery( getDataSource(), @@ -268,7 +268,7 @@ public class GroupByQuery extends BaseQuery private List postAggregatorSpecs; private HavingSpec havingSpec; - private Map context; + private Map context; private LimitSpec limitSpec = null; private List orderByColumnSpecs = Lists.newArrayList(); @@ -443,7 +443,7 @@ public class GroupByQuery extends BaseQuery return this; } - public Builder setContext(Map context) + public Builder setContext(Map context) { this.context = context; return this; diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 0f032f50121..1b77f2299ba 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -58,7 +58,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); + private static final Map NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); private final Supplier configSupplier; private GroupByQueryEngine engine; // For running the outer query around a subquery @@ -80,7 +80,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest run(Query input) { - if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { + if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner); } else { return runner.run(input); diff --git a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java index 4f7d27d2d73..d01a4f4c6a4 100644 --- a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -40,7 +40,7 @@ public class SegmentMetadataQuery extends BaseQuery @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, @JsonProperty("toInclude") ColumnIncluderator toInclude, @JsonProperty("merge") Boolean merge, - @JsonProperty("context") Map context + @JsonProperty("context") Map context ) { super(new TableDataSource(dataSource), querySegmentSpec, context); @@ -74,7 +74,7 @@ public class SegmentMetadataQuery extends BaseQuery } @Override - public Query withOverriddenContext(Map contextOverride) + public Query withOverriddenContext(Map contextOverride) { return new SegmentMetadataQuery( ((TableDataSource)getDataSource()).getName(), diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 59fe269de51..6e14ef1c1f3 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -294,7 +294,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest> @JsonProperty("searchDimensions") List dimensions, @JsonProperty("query") SearchQuerySpec querySpec, @JsonProperty("sort") SearchSortSpec sortSpec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context ) { super(dataSource, querySegmentSpec, context); @@ -112,7 +112,7 @@ public class SearchQuery extends BaseQuery> } @Override - public SearchQuery withOverriddenContext(Map contextOverrides) + public SearchQuery withOverriddenContext(Map contextOverrides) { return new SearchQuery( getDataSource(), diff --git a/processing/src/main/java/io/druid/query/select/SelectQuery.java b/processing/src/main/java/io/druid/query/select/SelectQuery.java index bcd29cb7f96..7556006734f 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQuery.java +++ b/processing/src/main/java/io/druid/query/select/SelectQuery.java @@ -53,7 +53,7 @@ public class SelectQuery extends BaseQuery> @JsonProperty("dimensions") List dimensions, @JsonProperty("metrics") List metrics, @JsonProperty("pagingSpec") PagingSpec pagingSpec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context ) { super(dataSource, querySegmentSpec, context); @@ -120,7 +120,7 @@ public class SelectQuery extends BaseQuery> ); } - public SelectQuery withOverriddenContext(Map contextOverrides) + public SelectQuery withOverriddenContext(Map contextOverrides) { return new SelectQuery( getDataSource(), diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index bad00f2bb87..357854f1958 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -54,7 +54,7 @@ public class TimeBoundaryQuery extends BaseQuery public TimeBoundaryQuery( @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context ) { super( @@ -78,7 +78,7 @@ public class TimeBoundaryQuery extends BaseQuery } @Override - public TimeBoundaryQuery withOverriddenContext(Map contextOverrides) + public TimeBoundaryQuery withOverriddenContext(Map contextOverrides) { return new TimeBoundaryQuery( getDataSource(), diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java index a1de320f5ec..3a03018a63e 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java @@ -55,7 +55,7 @@ public class TimeseriesQuery extends BaseQuery> @JsonProperty("granularity") QueryGranularity granularity, @JsonProperty("aggregations") List aggregatorSpecs, @JsonProperty("postAggregations") List postAggregatorSpecs, - @JsonProperty("context") Map context + @JsonProperty("context") Map context ) { super(dataSource, querySegmentSpec, context); @@ -116,7 +116,7 @@ public class TimeseriesQuery extends BaseQuery> ); } - public TimeseriesQuery withOverriddenContext(Map contextOverrides) + public TimeseriesQuery withOverriddenContext(Map contextOverrides) { return new TimeseriesQuery( getDataSource(), diff --git a/processing/src/main/java/io/druid/query/topn/TopNQuery.java b/processing/src/main/java/io/druid/query/topn/TopNQuery.java index dc59a663b2c..0e7a796d045 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQuery.java @@ -62,7 +62,7 @@ public class TopNQuery extends BaseQuery> @JsonProperty("granularity") QueryGranularity granularity, @JsonProperty("aggregations") List aggregatorSpecs, @JsonProperty("postAggregations") List postAggregatorSpecs, - @JsonProperty("context") Map context + @JsonProperty("context") Map context ) { super(dataSource, querySegmentSpec, context); @@ -178,7 +178,7 @@ public class TopNQuery extends BaseQuery> ); } - public TopNQuery withOverriddenContext(Map contextOverrides) + public TopNQuery withOverriddenContext(Map contextOverrides) { return new TopNQuery( getDataSource(), diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index 21efd3b8351..8f78ffe6191 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -69,7 +69,7 @@ public class TopNQueryBuilder private QueryGranularity granularity; private List aggregatorSpecs; private List postAggregatorSpecs; - private Map context; + private Map context; public TopNQueryBuilder() { @@ -130,7 +130,7 @@ public class TopNQueryBuilder return postAggregatorSpecs; } - public Map getContext() + public Map getContext() { return context; } @@ -290,7 +290,7 @@ public class TopNQueryBuilder return this; } - public TopNQueryBuilder context(Map c) + public TopNQueryBuilder context(Map c) { context = c; return this; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 6f0400a1f1b..a7d77fde396 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -339,7 +339,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest implements QueryRunner final List> cachedResults = Lists.newArrayList(); final Map cachePopulatorMap = Maps.newHashMap(); - final boolean useCache = Boolean.parseBoolean(query.getContextValue("useCache", "true")) && strategy != null; - final boolean populateCache = Boolean.parseBoolean(query.getContextValue("populateCache", "true")) + final boolean useCache = Boolean.parseBoolean((String) query.getContextValue("useCache", "true")) && strategy != null; + final boolean populateCache = Boolean.parseBoolean((String) query.getContextValue("populateCache", "true")) && strategy != null; - final boolean isBySegment = Boolean.parseBoolean(query.getContextValue("bySegment", "false")); + final boolean isBySegment = Boolean.parseBoolean((String) query.getContextValue("bySegment", "false")); - ImmutableMap.Builder contextBuilder = new ImmutableMap.Builder<>(); + ImmutableMap.Builder contextBuilder = new ImmutableMap.Builder<>(); - final String priority = query.getContextValue("priority", "0"); + final String priority = (String) query.getContextValue("priority", "0"); contextBuilder.put("priority", priority); if (populateCache) { diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 40045ec8fa7..f1cd1451a78 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -121,7 +121,7 @@ public class CachingClusteredClientTest */ private static final int RANDOMNESS = 10; - public static final ImmutableMap CONTEXT = ImmutableMap.of(); + public static final ImmutableMap CONTEXT = ImmutableMap.of(); public static final MultipleIntervalSegmentSpec SEG_SPEC = new MultipleIntervalSegmentSpec(ImmutableList.of()); public static final String DATA_SOURCE = "test"; @@ -325,7 +325,7 @@ public class CachingClusteredClientTest testQueryCaching( 1, true, - builder.context(ImmutableMap.of("useCache", "false", + builder.context(ImmutableMap.of("useCache", "false", "populateCache", "true")).build(), new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) ); @@ -339,7 +339,7 @@ public class CachingClusteredClientTest testQueryCaching( 1, false, - builder.context(ImmutableMap.of("useCache", "false", + builder.context(ImmutableMap.of("useCache", "false", "populateCache", "false")).build(), new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) ); @@ -351,7 +351,7 @@ public class CachingClusteredClientTest testQueryCaching( 1, false, - builder.context(ImmutableMap.of("useCache", "true", + builder.context(ImmutableMap.of("useCache", "true", "populateCache", "false")).build(), new Interval("2011-01-01/2011-01-02"), makeTimeResults(new DateTime("2011-01-01"), 50, 5000) ); From 8f8402b5ba50ba05c5eb47e434b444cc4f8159ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 24 Mar 2014 11:18:50 -0700 Subject: [PATCH 05/10] replace png with pdf graphics --- publications/whitepaper/druid.tex | 3 ++- publications/whitepaper/figures/caching.pdf | Bin 0 -> 25198 bytes publications/whitepaper/figures/caching.png | Bin 36103 -> 0 bytes publications/whitepaper/figures/cluster.pdf | Bin 0 -> 27474 bytes publications/whitepaper/figures/cluster.png | Bin 53840 -> 0 bytes .../whitepaper/figures/historical_download.pdf | Bin 0 -> 23835 bytes .../whitepaper/figures/historical_download.png | Bin 28808 -> 0 bytes .../whitepaper/figures/realtime_flow.pdf | Bin 0 -> 27060 bytes .../whitepaper/figures/realtime_flow.png | Bin 52345 -> 0 bytes .../whitepaper/figures/realtime_pipeline.pdf | Bin 0 -> 25788 bytes .../whitepaper/figures/realtime_pipeline.png | Bin 36023 -> 0 bytes .../whitepaper/figures/realtime_timeline.pdf | Bin 0 -> 21797 bytes .../whitepaper/figures/realtime_timeline.png | Bin 37015 -> 0 bytes .../whitepaper/figures/tpch_scaling.pdf | Bin 0 -> 4946 bytes .../whitepaper/figures/tpch_scaling.png | Bin 44252 -> 0 bytes .../whitepaper/figures/tpch_scaling_factor.pdf | Bin 0 -> 5273 bytes 16 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 publications/whitepaper/figures/caching.pdf delete mode 100644 publications/whitepaper/figures/caching.png create mode 100644 publications/whitepaper/figures/cluster.pdf delete mode 100644 publications/whitepaper/figures/cluster.png create mode 100644 publications/whitepaper/figures/historical_download.pdf delete mode 100644 publications/whitepaper/figures/historical_download.png create mode 100644 publications/whitepaper/figures/realtime_flow.pdf delete mode 100644 publications/whitepaper/figures/realtime_flow.png create mode 100644 publications/whitepaper/figures/realtime_pipeline.pdf delete mode 100644 publications/whitepaper/figures/realtime_pipeline.png create mode 100644 publications/whitepaper/figures/realtime_timeline.pdf delete mode 100644 publications/whitepaper/figures/realtime_timeline.png create mode 100644 publications/whitepaper/figures/tpch_scaling.pdf delete mode 100644 publications/whitepaper/figures/tpch_scaling.png create mode 100644 publications/whitepaper/figures/tpch_scaling_factor.pdf diff --git a/publications/whitepaper/druid.tex b/publications/whitepaper/druid.tex index 3340e83a0ac..f212a713a4c 100644 --- a/publications/whitepaper/druid.tex +++ b/publications/whitepaper/druid.tex @@ -216,7 +216,7 @@ Figure~\ref{fig:cluster}. \begin{figure*} \centering -\includegraphics[width = 4.5in]{cluster} +\includegraphics[width = 4.51in]{cluster} \caption{An overview of a Druid cluster and the flow of data through the cluster.} \label{fig:cluster} \end{figure*} @@ -900,6 +900,7 @@ well. \begin{figure} \centering \includegraphics[width = 2.3in]{tpch_scaling} +\includegraphics[width = 2.3in]{tpch_scaling_factor} \caption{Druid scaling benchmarks -- 100GB TPC-H data.} \label{fig:tpch_scaling} \end{figure} diff --git a/publications/whitepaper/figures/caching.pdf b/publications/whitepaper/figures/caching.pdf new file mode 100644 index 0000000000000000000000000000000000000000..e42b317148d0c322dcf16f9d5858ff3bff5a075d GIT binary patch literal 25198 zcma&N1yCGa*RCBP*r37P-CcqOcXyY;-CYvg-5Emg;O_1Y!QI{6?U3hr-}8M{r|Pf2 zW~#e)?|ZL(uc_{9)%5HkR}c}SXJ%jnkP~ffPfP>Y089WoBTE1;FMv_S!`=kIC~W9# zXl-W(V3aeoF>(5*L(0(AP#NfGVebrJ{m?4``1t{h3WjDTGKNMb*8ghcTx^^G+8@1t zBPjzIMLxF90Or3^-GB7*F3#2#wjU8${t5BVc&87+f8v?g8vpgd{?Y!gA7MumLuWfj z0M*<3C-F~qA1|{{)}KtM|AzaIoa2AV|F!?u)?aguCboa4{~MiA*2LJtP{_{xAAd{$ zW+oO+05ccUhcQNBJ6q=ur4xYjU#XIblbwqr@WUq8zb6E3ZS6j~nVJ4=iaXl5*#9?2 z6-Pr`C;Pus0X;rK{mt6M)dFauBrf#VYd(#jFjTO+-w9cE%rm zWlU_%oXr8;|4i~R&dK>BcN>5^XiP)X4wV_XK9RFW81iNZ{|9pU`SzEY40|$JZL(pA z9Lo|*ZF9k`bJjbGOIwFrGFQm3>O0@d@kq0N)*7cM?Cb?U{aI8~>d1A5OZH$`6FF^B z6Ag!F0*uE^6x#8$62gkDC}6VoR`%VB{>0^%c5sY&_g9zO?HAfUGXX1thdpb=;{6A- z`wcHlQL+5sARP&YFiHkPOxZjf(&V7*?tbPWLyA&ZpiWLEQ+*JXWD@DPMQ3npx&pl( zJ=V29_9^un)aD&i4`!+)Nubgk)eEBEo6zNu!rj|lM~aUCmD)mFVXvlr!ekU7H!}*s z9HNz2UcUcjT*hafR_so$#;#-S^$K4HdybqQXe7Di=mTD6QLL2UZ9!Q>d;cubF#c?O z=kL)miqNMy{4zY|xHi8?A6)b{x**r^Mu!yAJ7X6y26PQ3#R-3mk$6V_{6$`i-2aq5 zq}0+&5c6F>e0loOgSkS6lx$iunY6o2vY`FhU2{Cjjg=y`+$WLNBH#L(YG5rI2+I6M zN2FR#N|M>s!LtAcL^E6&D?$zR#@a9ew^0$O+PsSly>S(?@j%$gj)?2)2lGJCxQBSu zC13&VllTpD-;FGrzA6|e4uk!ijYKrFr(JF+c&J) zpfcAWhav9_fvWwh{ppXUuh~S`gW#nHPzf9;?qW}B;aDYsEh#r+Nx7Tobgn#~XQw!_ zX@s_YLy&`mcyVwEwneM9nS;|`qnJmG5HjA)P+1cZ6qHbCn`kB*e+VZY_uov60HdD=fkwtI#I>vlI2Cj#6MDcwm9sU_}JRC~J4zexLsG=I-&fAHx)GUmS^`!7I*1wR^$K*Nvi z{}6`xFIE4Sf|=o8&ENh1sQxnTzap^@^8Ev{zXvP;=KoE0=MO^J{_Xr2%BUf4WN8BY zNBg%a>h3JA?EJy5e+CO$+nXCQ1OA!*gS7uNKM4DOvCgRAXa`g_asJ4L@uT#76d4nD zX8@z5%?GB1{<9bU&tCH1O7Y*E{~G$Q*?-yok7Y(>7bE9?D%IbUA6W_+I+^_2BdaQ= zs-;OQ=xAYREvxcj)%Js~7PcR?Ow#xR@)ph>^b&s=?F9Uo!_e0G@2I~;i}`OARQg*E zRP0o3E&f3=^WRGJPo?{>pZ{H&{=Xp3{BPy>_^kdf#eI}UYy+5(Td&di3qoL)zr{-p zDkT-8gz&2@A${_aq(=v0?{}`xmu9d@lG80RwXoLIyH5MuYG9vAKJ7ti22f^l40mzi zWqzLO7R(5A$f>>QfRSJ1l z-lg+)a>?$X%UuYSIMkm3i;wdz|L{j1m9{#~PF>cV;Pt>mDjjlkYTwA9V3#>AY&kQu zgcMFP(XO%v30SV}6J4G)LXs(_@$y^NGEtnQxHi;?f=ID+ zEe;br3p&s1vthOt_Fde(-L!9LqK}A*$)q$3CS51`UA$9Q58(*HHi(M2j5QX_8feUo zJrl`{9Sxzu%g-ZMPTG5i-1j;JnQ*85UwZf_@aLe$mCd1fO6=c1h0IRk|G&2m^Z(zX z%KZOU)sr?@cNNh&0q>*ow);~bNy#p=@6X7>(KA*LGC%u+pQ@;KfQihJ3>XL#8 zD?2>s4bbO5ZtIa2355iA9S`W8kj$miHc+PSBLR=81W||G*iPjdyj90YK;|~AU&k{e zdva-XYY!*)pGwwxZX&r8Fek3(cv8S#${CYmCbXGs^d^^u?}45)-(T;q^g@Ypu7JpK zD9BKQ-r*0C-8ZA7X3kkcn{cJ82A>Szc!x~86DkRR0tc$*To)j2FT`O4^}Q>|FRU~m z0g(fsj1BOYq11OPjG-A}@rA(cN?5&tYRqnzN=$z`&?8=Qchpvf#<7^a5dV$;7bZ+w zLtz&6?#~Yj&0x(v$c=b=RVHZ;Y1*uA#vDDQmEteJ0@xGeAnFG%zU#Ud{j~dEd~=O( zXF*7d_;ahV-D=5(L(m3brI=Ra$xUod0Iyi+j&6PlCNw>>C?S+Aegu9K!+MdE0C^8C zAqhWrk$P(BhA%dLO*$lS=p|?ey)n@dzkSuq~s0ifO^1h9l^W09Nimr2>pf) z-+f;c!l{v)YOD3Ly#GsrL*kMk`C1xWM@Y&B5v{=lz{wq&EN>o#v0~Ce61nU$8-3MvVg~79JM>E=c*$$oc z1}0ry_&LP>o43XiMW$NybOrY0h5DlFi575`u1lD9Np<@Qp3pB1gEh%l|h}9E;dfn!R^%qfG!XU;sxud8K z#VMgfD35&ank&tZ0FO6WQ}2V$L#2Un5=bLe_ikMQ52julpIQStkTyvvSb3Z_8ON^{ zuv`(|%7G1)AqyCHTqOf)h}AI*`zP9W_&oHV-q47p2T`vs)rshkeaT<68uhupcLa2+ zcMJ5zrZQ6Wl^Z2#kdF81m7&aU+BH0IQq$navT%*-?zHS8Tw!=*ynwuQ9DdJhcU~lDvd|F(J!Hq}`QH_g$pL#Kyy)?_PBXLWVLEIR2~P~p+TP*6R8+wFotI4Jz{j**`y^huC)O4; z5e|&-yq>|;I&lsenE=JCR;(eh z&C;%lJ7Ew4aACv0Q6Vq%0bU?Xh4XK5u+c4IZ*Y z*>8+~#a_S1Nu|({MCuQ)JjvjI~+;#Y-uD z&*;6!dgt26b_Uzx(LE3JK5&*>E(7k~R52CX4wU`ZSCvAo`tD-u?)VvK=j!u*lacJN z6t{f?rc{D)Cbah zzfmKeWPfUL8tpXt`qW%qSuUl_R`DwX7>WY?*IdrX7{PtqSzV;Ez9w;^W=YyXa1OX7 zvX^Tv|LP)7h>tFURHl^2Qm2$F&*tEjv~IA|JvbuWWR>)8Y?x?Nr9U?fvw8|L>!)rq z9V>QU?T_2mS{uu?l11Q$I0lHmQkEUY$X4_T;_=c%m%(z*MV0_LjS?I@Q9Ws2UklK0 zazT_j%$Zv7TfLNlVE#<9+1P90m zIlcpyGU(E@DTs(oeS{t@7i9-A(>cv&8LiygJnv$CN=ysLOZIgzrEgtT731S+);m<9 zOK#s3a}rFIdY(;N8PZB=>B&(PpEp28gcI9`<)@bj!*R@V8$MNB(r7dtkJcpFT$-P-Sx8d2qP&wW7cV^0?nE^r3psvc}@uL>lh?S(GR^4@qBXys-hCJP%&H}zw?dXS1z#vm;2*Oq$L*AgL zENZ{O&a-DjhIsT0PbTk`>aa}>EO4b(^JCCI7aQKAb?!Zaln+Y8B(3O!8>K z)!F`?fN!D0*OlKy(%q!FxMg^hN_@caLkNynVE2GP-D~i+K-W^kds`fjVTO_U#Zz*4 z(>M2;!O`FTnFnKNVnlbS1!D^fku2+ zuXAQ8;k}x~kSWL1sbaBty5Txy8OESP)S_S0i(WD^*(J+@>Z-ZNh(BPMOd|Fqss#oP z;WevRvM|H*NZ6RwkhrLz{Sl${@Dri&9hCG2MvfJHN^ziRtU|7~DKk4Msc-V+?Snmb zj#YYX)equucE9i0|4F(^C-m4$yOrY2>5DbC=U(!Q0cQ)nAX!Q4DKd<&meR1%KlNQC z*H-n?IM~NU;S!jlw5qieG7#GFFI_ycenwk2`LP%$si_Xzp@gwm;aZ_|rg%Q5huA^e zTGndwUc$4qqI*OiMJCo>R%qBWcuy^Wpk$uSo}{iYIW*i^tO@yMq~edH7%8POzw7zX z>aOG51I(705c(#T91fi2`gs0C-YG=>J>|O#H{+@dTQs9?25+!>_}uuuA`n^Q)SGSI z>kpyu%naF>fCPn4dV{t{iou@<>2K77jB*dP1|BK41oVor6*NH8jwXl6aazv_Pp4^Z z_0{(MOLA!fzn5n6{ds^LOIYS}Z?Ep*Xv?AV6O#PeHJcaFYldBPE+6mlaT;yduz<)E zPn3#TfSa<9jpxMUZ2vm)!)!qZWJfr4VWCldtxBV8<2(r;F*7YyHIbId9`XV$2R|AI zj;KK|GY(4#H(}=>CN&ae1r{YECM6^2S|u=aAP-rNUNIyR?w3%3S<`_gOT-AZW-RGl zTWR0f!IYr#^V_|Hb?=!= zz+w7z^3I!UCoO7KifW^tDqavCaVxZn4mKM&hLNs!)1+25%*18{KoW?lul{6N>B(lc zu&>RLI z&(#OnC~L%kT6~TZURD zyx2V1j@@Uf*57L72d9j*>XUP+J0|c36nWt>;LEsrCh;;%6%*Bs6#eKh>Uwhq1?VSJ z4@M_(0>@f{y;2K*1i4M)esb}^M8)H9%3c}`vdj${Zc4K;Jdh>R(<(i|HstOV1o9~% zPMld9<}j&igd2HPD8MwG)Z=?E&XS<=r;+K6&TFeGtKL=>ERhm0P6G7Qn!b|h4yH5z zNQBtKIRUFHb^TF=5&`ZJb|xXX+U!!+sap8EZ`E89>P15HAuI3gk1h;4U+Y^2{k@*k z>#KFl`LjTqhvNotf{%@Oe~8jcRo0^Q>@j!RY~AA>XZ?p;ySwAcoKJ?j{Fw}`z4aa{ zb!csh@vVHA;rI?NJRond9OGn}0JhNls+-xMmLncfrqUIXBN5{aFQZJCh>~KrHe%=$ z1Su!Z5B9u@tc;b^2(G2`=j!b^`zP znOps9zVA;szKH5xIR6xjniu5&^{*9I70%_C>xs9a(-Z@ua?Y+H)8s|)%O-4vdH8%i zgH{K|x|b&4O70bv*d*PNiRfO`K+a4(r|`BtG_MSgD!0wvhm}TTq-m5eM&+@bVoenr zhGGU8`}$0S*gq)X?T2B7MQTR6&-M_ z`IG!*!sdGK&~+>Qg-XY{(Ym*%Rz}~Jhor^nLcI=#*Tv-t(*=Z0FZEg74&#J(LBq>B z<#PQhlJ@6(X*an{cfSzO`sz~Z+ZpNB@Y@`7a3vSk9z~bxVP(k#9nGd73wy6DVgY}d#-8>ZkV;lJ&9jLG99rfn&);JAdHLb~1& zCk}ayy?T6b_I#WjPlHldIzi*<5m6!ze{6I;*i~n?reX8`au`Ilw>_MbqJL`0zpfH0 zf48V(j>Lp#Ly5o`(PYj2>>f9w>FX?;ApH<^C{vjD56dcMun18I6vT>;!2F zF~#)wNYJkgeQ5EDVZ8*OhI8h1LOeU3w{(-Lr|$Bp-{z-eZq?_aKS)RlC%x;x&+mKn z?cTL``=%_1olxxNM(63^PX4rSQU)`1KC)=&T_aNq+#=u2KJ%ckbgIi=vPsW-1y_c3 zDnKzsp6eD`Fs9jbaX9CnLti;z(DC4NZFYS}a~k5yXae==S;j3%JY6{az9ou+~Ak&|MIfxoJ`K_}e$21k|x}~!%MOXiQX9;dTnb)`E zLT-AxM}1{&kWPKYaVtjFI<0t$NH;ha|n&xC49{nM@j zMlAAHA-g&|6m`?8bC6TMO!aM0GgfpoUDqV}rDK8XEfBB?cVa1qQtLi1= z)t+o#@_GVuTv7n;L+Z1=-Ilvu(^qt)GD3Gg^MIaA3*^*N>tCyHMZfv&kiKA)X-0L! zLN!j6JYe_RIhRWhc_J5=;Aj`7@cZkikzM@R!Z(t!6IRk65G@g-#563_CBM_E!!ayD z)qiZ~V!Jls2>jBW^=)*q9qS3G^%3=Fxk1eP_81w{oixgiv)N(UP1*Ux)V*UJ$b@|g zu}8uFGt&wI3XQaj))ohjrR0bB(?K|xrd%>mnTm=i=JZWyOL#J(Eg7S5qU4f$r>(cjfXXq>vl){FLQ=)90JZ` zv)cKJ<(iqnu3yiO?+aw4-nS>LqICzLYRFG6%^G;`d!FmGt#Alh&7NKHCWok&{jY!< zfgaTJ*K!n_e*U7>ITu7?(~<`9vzt0p;ot~6_T|@MLNhj+=C3Hhq|CBty_WEM)%oV1 zM0NU&@Cq`2XX>-x^i~-zNmgSFGEnMPPAM1B`ZjJ1LjU^y`#0oj>)KgD->TL00F;~4 z`In0KXW6MD%K5dVy8?O6ms`m@NUuZRtA)sJRn#t3RDbXTvb%K%=4q7}M~>3qs5;=z z-nFy1uZ`W2$7#BU+m?)j?c)A*5}K8qxJPql)5X&DzEUY^N8E;b+t)90{QEmqd(Mf4 z%W0rY^?e){abMCe{5^Ze__0aTKR(kg(RZgJ;|yX?h&JYj~W{JG^DkeA{9 zxlFdf!dxM6-8n>b8l5aXzju$F9s>$bg|{;cYs2cReZPQx{q+yJL|{_V7=LJ?hd5dd zY4}EKjR@|O?4e?wEiqJCWa~$JS{{a60 zlq@d(b24q7DyvPcdXeUCmi6{dhy3fKKVy}_Wl3sU$;;|oYG3VO+x9#r_s7KVs#q~% zl#pZG@_za73i6UqF`Rt|F>zJm;__5tklIwo;Uz@vAmeVzu$gW8($<`zGX&)+QdwX~ zaa^bG2)WTjxAu>9mNh}TLA3CpAENBRQ;cHEq_U=7bbpNtL+H*g`bONLIcSG*l@SC7 zQ9w+9lm{+kpu3+Uv)B)b<7N&rPqrIds%fZBMk29LfvVVaEOYJhPSw9okcxX7KnW{H z4|Xx3nfT7l?+5pDIv%V_^P1t=*l!!NX1}zV+~+FOZHm&a!+7@un{$orb!wImx(v6@ zdn93=(MM*~(ruzN>fE={;mK|(iLb*`9}3H=^;DC?vp*b{LQb=8D`fPKOAn#0>+2kS zo}KK3Hw|O_x`@Za*co+qsr8S;-m=*Ft(qB;@V@K_C%|X`ZFNdl_=!uU-Wmn})(Yty zkG>;8AnBst#9d?S;?EQ7Di!L>#Qv=&2oUfU6RKa#bg-tT;&SaomvQY_i*UJ{15TSa zM2G)ESmoMEF-#|lN1CQ7ZN4H;$q6E%T7fN8lRI8jx^W^z1jLLnh5);g(YH@)yiG#< zrGZIhSMuTO@04VT@kChAd$tUg)N}wIGHEdHg?%oRT^uw3+7AqlT1n71`%hiBw#J$$ zpa>QRNW5`B6)Jr@k(u$6}wr)hB2>HZ2 z@jq{e9)a1^YC}vSTlXTltAVS~&FrtDrJ5UveEj$CzxT^&w2XC5gvvD=xI z<<}rxSz$#1+>|EOH!a@?*UrVBq817Y4zfUrMz3#`19>_{M6`Pox0qD6jNU#V}Gs8eYKl$G9e5m?}RbSneZl&G|{Bp0bJ(W9JbZ~TVE;Bc5{uOt%3hnxr zb{+>RVsuDpRP?Ul-d`8LG&)v%Ap=l>^xM2Z@NM9B&A^@>RtJ+`0WTjDkAv*i+(KDc zNAS9|$cbUF_f}b^X3{q`rNUAv>Jy}tVLlT{AFq3~7;*MX>ii+@$+qwNf@) z%!DsN)+;anvqxBC=HeqxRY+00ZZ^%GM9W^%RI7@mJR)DVeE!hlL;B|1cPYEyD7?E% zBTFN7{UrX7*H)URFOZ^wI$V8H5kY}8iQk)6dC`UZt&|kj^1X62o@{6RhoK4GAA(Y* zp6lnUd>K22pX^#%4-@E}>UD1)PtWrNj5N>3Qtc;ep)C_iXeye1$H}_i<8!#2*Bpwk zt!BPmBuCfX*9>Qlx|%&PTF`-fb00u3hu8aN{0T-=9tz;ALG(o|CDU3s4r0>F+|tst znXBNdHJ-_cG-WPx$>WaWbeS_2QBEc;w_VRUF+33D6uGo}UJz(rY}Iu;MT4y56XJ-8 zFWeWtVfl8kan2<9V%XcaQ>uF`b+q)In`k2lyTJ)2`RGYXCW}NC28$ZfvX0FrrjBvI zq-~P{fy{Rq&t~<;)I1luQW%$^>k!KNjAc1<%;SS%*VVnHjp4=}g!0pJJDr>eMxoUL zrn~rdCbty1ysN>Hqph5k{McV^e;>&>Ex6d#8uEVSv5juqueePnCC)x{8ljorJX5ttpWC_m@^1F^Mw4vQpMn8Ez+q zSal9H04}X2xeYMSbUIgf#R8QEuOdrx2)S1JW%Z#01x2*AV%kAPCywDrh$ieY_;^geB7|Di+}#J@hn<=Wjk)TDcLpr}PD z9&c_#7IVCuJ-vJsSMymo5cC zaCT=*09m}ahD2$4v)SC84+I9{$=ByqX-$ugS?P-5ayWh8i-K8gaq~Haw|0$#`2Bsk z0~WKr$Pko8&d9@#(|*H|kVoBdbHljPtGoTVQKX3u4|~h2M|=!I0iW$AzJ+IU<}zdR zfgcGBIKF2U=Imv?vxyOo(gD_Yjz>5YJS#zt>P=a6>icuMfTpI$sw<82h1M09CMkJ+ z)k>PLWL#ugRz)t+PV-QV(c{%Rd!WsBUU!dL=6n2smTWgt!pC-i5Bek5v%M_^CrS!W zGtU#x(Ck?vZ!e$FPlk?+*Gy+ACv&irJeXK?WffDnUY)zjCg?95Nj*$qog{KALC~i% zHH9Y$CnG0H4EN~EkKx2Ah8amDf#+$ zIj&I-ndLkq_2SGkFcFmQHBz7YUuN(kcHj?eSq@FBwZ22?CTz~|_$MwpbJQJKw=5|a zwLEZ#1V4H}QaLG3j(!2hIZExz5ttDwvVPZi^~X=K&dQSVdv5Wm2?}R!VD#LIqs8R= z;s4mO9K3!v+S}m^U$~s*sKsl!4;TtQJ7fTdZD}3v~9(%mf4cFyO8A9NK zO9IiVNGwQgC5QoTf8@3x$!wOohVwbl1jKsE(%>A zHA&x)ArYlN6{Wvm)hEh^Pn2z+_T7GTL(q(v>vMcGC5H0QH&0=s`TQ)uqy+Ib392A}Wo+apd4~^d=rB17dw;W#)SL$l8Lg>>p zQ#P87KC$zCwtXUmxB6w|UIu}J0M#FwdOe1=x z?n!!3cqDtoGJe@TS{d3VPu|$vN_|KaC;obGZ@ar$%r}-hAbNd3d>T_QmCIQmu9WZ~ z)ZATSCwAcLAcwFfq`F-*#)tPj3vm=;DV7w#yH z5^3ILVLp&X8#Fc=WTs{T`ko%ID()<%EXvhmxi?QzcqrwMK%OCoJ?N?-4Xhui-A^HC zcAV4Zlxc3+E-!dMRcG)t9$}B*Ec3QMJq+Pwau^%b^M|7cJGZLfeco!R;FCwOV3Nwi z+hGICi-uOt)(M3SWHu_(vBC#tQH&?=YZWfigT2lm5Ehdb!BHjSZ=cDX^ITEC5B;85 zTxOl(o*Us|TcYKd;vsevw61oOb9S_@s^+p+;=-OI8eADyFc+~`ve#39?_W8tjY|Jz zp`(&Y{Od&Y4(R+22iAYcItGkE$!Ho9yyL zv8FYh#&CHT0Rt6#4)c&B+lXqSt?qV<^IrV&emyVe_(lwOJ!bWpoN}7WH|0!IlfV;M zX#z8#Fw%UDPGUBqTKpT)t2}+X&pEv7{4hT&0eRZCu%iY=2Q2X%L(+Bwi(4KrPniro z4YO&&q2t=2ko<_wVNMi>R4Fi+9eu29u%1gagvS&JMtpcLnFC&r;JY{R@PPHC6u+sk z5rFM@KvVEJ#~yl{*U1?F^J{ zwLB#`-{elHQg`ermKGZcwyD>6)^z4J)@a0&OQ$fJN$SMF{h2w_TC2<9X{wKtN6kR| zBepV*fa^Vpi-C|9JDWQK>4SuaJYSJ1(30~Wv<7YcN~v3Y7joEmxn@CX1;`t`{dwypGFx%ikTFty}P$v{)lY{!mrw@*JbUUL_>NuIMg(jEuJxLCyoe>L{m8ME7O@ zaY2w$wq&*maS?o>2$phxlS8P}wgM|BqJynW1|07_TqdqE^^!GHqeEyZj?Xi()3DBQ zp%N?CuFj^ZXl&L<7L9bF<~q5v)7nWi*+}^y=^Y5q)iiCKusDN4OB!)ZlTE*!(&GFywCdGWL;%P|DC$my z27g%p8T-V4JYQIMgwK0|baLvP%crGHI*Ybk7vHr?)Wwg__c6i?vaHQ)5vitKpwc}<TNPvB3nE+o+u zrlqgzJORvBJRp$fK@ni*jodhEg?F%<>3C3GunxtpxUvz$7E981G0r4 z)1i*a%^ggqIaiCTsurV#bNqoTGr6NB{V*$!YDEq=*Q5KOlhGhn1WH^7e~6vzo3{ zA2@V2!7I9FlBnGogD@&oA|9tFn_d-j2v7!1oqj^*@5NXWc;t;TSTH2$O z2g!a4J};1J0HaA$-$!NW%g1O!6UT7UiWUBpiKUGw`ITF(JOirBpi_BcQ@P?ZZ^==G zmCns_*RA?EfrB*q64?5hV$0)OL0ETz>U?DL?NWhREKh0wP@lGQdJ121I1ESo`FI7O zqxeOtKI+i=p=%{_o)o1amWtv4_vxkhqGY&2qiIMbKVvjk$n!HTXMQy`g}t!sj(U*3 zs;|kDKf4x3{nIUsT@&T{&AC{Qz)JIY^Q4xUO|7pvpRHcr`uIZ|QF@C*iX}u2Z z(RndhsG4XVw(W#=Tl5bS+IU&T2;*1|3EcdzMT%ePsH3JFBtKCz;P)ny9TxojMvptq zczKv&J=5L`8DtCqgjq|Zd#=gj#`&|2PNle);(_+Z`bv#;|=MQX~GP?bO~b>$0c3ytNYL0Tn`v zHmqUNs_M-1&XA=dQWCJHG0AS?TPYf&F&1imGC~dP5ACRi)gwO}De2scKEJ|H*O1Kb z{zH2m$@?=->j!r@*--q{>ep}a5s7#84)ywB>Pu_r0%|kj=xV>nW=N#-r+Tzb@y<%) zEE63pqi(ehX&&{CBZjl`8fI6$^T-S#vlqWRN81_?N+JSIuxXYiv93dQ;$ZQx2op4fagi`l~lcR=OwzIyn# zjWqBHH1`69UaTYZSth&`G&!$gRx<<~b+X2RPh3H<^5B|aoR{C1(~%vx-jg{EJf*w7 zJ-4T4GsH@}y-8l3W3HA_jZ}VkKea(^G_ho{=hWJrzT2!qP_F{>5>x@`mt#b@e ztPIZl2{1;#wz^>zqRyzJYnief8ZTEezouwWB6f2JLGxTiCVVpo{1W^$L?)n`XN=>v zr#e-=h@H0)kg1iQ1I@d;(S_Kerzs#VBI~i9lG;buJNNQN@%@yxjaejo>m zM3CjyzN)HmPicQ&iC50-%e=8UPi{S+d&@nS9sw=?@mSYg^}2BCXrXC7GAtbTOx4fJ zf@Tn6K1_D{SgsdR&4$M~V8X2alaeukwK~wTk0P}BcQ&O%9O^edJBUg#uIRiOp)+qk zu`}a_Se~z%hd9N33i2-H0dbK{_rOX*EQ-uA-_ZA1)I=`ic?OJ^)a5^DD+-L~e6#C+ zAQC@YCO~$Jdv%&sqLuZ9N$_nklP>OE^o>d+V~3^3S+7o8CuaY1{3WDmbBm3Xfpv?6 zlwa9zW;Ap@(tR{^LPeO3b&H*pfOU(_&5X|UbK0C=e(+Kzy6DuG!_Zs1>|BS(SZ@}F zK)OqCD=u1F^ge%3Dhg4byor<`Th1aPMwN>wZbV>?0@V?X*1jP@SPRm_UTnzocGGjG zJnA7cBDtZ=hTLapHxhgKR&jOJ2}knEB^9}aDI+rws#T-8F2=|ff-yU1Y#;6*31@N$ z&fu1o6vjP=$(Nd~Bc{BD%e)x@Ht^uWk>89*gbyU15575Hv%^!gcwW zY{wT@20dP**HhVZESy`)#&~~g@nOn{aQTHCc5}jrtf%M*N}q5MM;1cKGn%N?Tzq&! zM6+(Cl`Xf&0KJ*tX(-N1Acl|;GOd)Rtp^5W27Tfn!*bD(CNZ1+=Gx5TsXJ5MrSZj3 zzO0mA={hO(PP#bL?=xXNwydpZl*&4;i%re8W)E3D8mS9v*oU*5+lGdE?!ZQd6|Cg$ z90dXx$7WbZ9Dw5ytfQ}6_1gAi)1rLIbiLY9ZDG(?dhC@l=3SezpGa^PlvdhQa?Soe z{v``$s}e`cxl}TeBjxuzf?IE3#B~Bm7OR##KC>})OkK`(;s9hzKf+u zNoL<^xFgE#p@FFEZaQ0_nZuM~hCqm7Q^oYfUdA#DuMr;q3=(5eGij7Kj&gZ-PPiNm zgsSp(@$aA9nv0LZJ8CLMh)qLGXInd4jt5@ZpmeydO_KS@6jqG(v`@@(_$yybFObIR ze`GNPqH%~vN`5*MV&%8rf)3AiO4KpwFO;RH7R<;h6=g5EvwmXga3#s2vS680yrMcL zzam!)n?a}9-G+wm;<8qvT@6^D%P&F^3t}inb=8~%E&@+Nw9!bYb`$KK5-)GZxRNah=8f|U84_eb^F?GH5 z&e!&JPA1||GWL*89yKn%A4$TcuXXdQoAIN%Z45f2#3>(@^XndnKGfy)!RqtHn>+kc zjX_I~180r56T2FP!N-=P-&M0{7w9Ow^|o5?u6mrlO3|>ZE2r|FX(&74p0q#^?a7u4 z7+FIc((J|({L{>&?e1^l@8WOsap3~}osvzGU6O5*eUgF6HT;VDCk%3E7?}0PYFuk- zz^wFCEl1HV^q>*IdF|?XW9|;8iJQxkwb#m}{pDV~ZWty?7mD)sASWrLY2Kemtn-r8 zieIUW`Kc8}u~N5!!x_rlYf3shm6DV0Bt(lm#^OWNDTLJ9+Bk7XgBOFngLi{JLbsxK z#P-DaBPE*RmPv$m+$6xj^2E>RlcGPnr}l=p*yIH^UiJ+Gy|lgKuf+uVLN`SWybo z!FBsp-XNtFK@}G&$j#mwcnnezm$bRKUG0)&wvltsW?1?z!&wCFUfmJfynU?c@#69BqZ0^mS z;{2O!YvFj6{a(Cgm%%_O`m-Oa>t4h>fqO_MZbqAd*18j1x2P@@Jt48@M zNZ2bawgNFjW6eM?CoUVAz+a}a%{Ed@6icgG{5!0LRV9Vz`-_cWN;?MFvr`jGcJW{3 zbqmUs$-fadDwZkZ&IT0c#OtRoR2CE3Am3E*Jlr*vFJgab2g@swZg!VlZ2F}NPhCQb zbfjm95vbXKKdr6lK^bj60B^S)hpOs5Ur{t1FkDsGs^g{kKA7>29(0o_uw;MXys!U! zkz3-Afv@B;WMV?AcCysAoI*FWSS!e5wg46}RUlxyUHDa$^f}{p!9bBq1YnkE zRYP1p5Zt=5xb9s{-5aF-TQdOR5uu@D69LiTnj(*Is)u+aPy-km6fA0C6d~ymMGynH z&cV%fnUqn<5r*C=*;78W$C5iT6_i@n87PA;3n`1AMa!Er6|{UPv{|8*rFVR^*j*|~ z;t440Au+?=3Ldy0uq#xSkU6&d3SS(j=)IgeuVyNyrEQ;Ux4jmtMThb&F^VRzzc4xb9_s>sce;ZfHwWqP_T6irjPYo{HQw zS+IFduH2EO-RQ~5Xe1>kqa^aTs!Y2j!zBBvKTBTKp>02wS(V&X-4Al3v%X{_L z%)bYnD!;ppQ=DdfEa4ikYFzf`_o9<_ymzUvk2sTXx_9vz)YcrS%Pv)MYgo?qXw}0F z{n9a^zEk(eyy@D=7fsTF&{sAl9)flX;QM5xC3AqaOo(cPAarZX87(@zP82D(Skgva z!8F;jPV0`VabCDWNBCs2cz{7zrm`|Q#JoZrZEuX{4e3X9OPgF&K_du!US=bZcjTXe;SWB*?jk(7be6^QF=HE zb^7_WuI*3>fe!JlLV`svU|xAb8YzGu2ejmSWQh>@vL|2`Wn?_r(P8>a?g37Rjy~{{ zQEWOwNBlrHS3RIVSFJ9P$W&O$6CA@}u`Km0Xj88^Vt0UuUqpqtpCm_YO`+Ggmr4q_ z$ftst#T@gqrzePMEgsHkwX_WzWEqb#9pv%(;+7W8c<< zb|v8Pe*2X=^`0nro2dQJ2}z z7>zKS$j(XuBc|j_!v)-`@V3t-i_&Q!7yQBY5cM75S6_Uh?1}jI#ai{FZUM|b5dFCe zNRYyuBuI)RykJTGgm;DuU)JA+UfaGfcENk@@40oq;rcu=-|!>a!dhRug@R|ZIH3t! zMYPxTBa|^o(1n;_jf)u*ivcQn@GM}M@G87^bBR!5ugpRG-$1moL58WO3loZte5HkK zMpOgcG6~KC)@btIkXqg9968$uRPfHnLlgceJ^+?zMJv$`OBnh6xVHqo_sl>91>NS> z9T360Q;KrJv|p^M%IroRa;my+96|GNm1TtHq6V*up3LsO2(wg;|7;4I3Aw|JtoUzxVhO{3`Nf~Q;cWWpye)JhjOgQU zKRFOvR9nQo)*|w~1|qQa2n*$dhv*i;<_Xi#|EII_j%sS#_Am-kLT}PTQ$VRn2wf0@ zARRf{A1@grW9{FZ zYtFf6)?OncBm2ux$Qex(=RxFf$pr=F&hZGchiC5Th-StPL62CBa(~{wf@p;AxT$RF zDBa*&CYO_pUd)B9msU>EA{8=Y?=rbv$|9s{qrATgxS<3yhlZ@iMmci4&%a3F zj3gaoW~VizReJJs_L=lHG%%Z3+hpK`+|oMgo~s1D*c^oSRuYHSI5B=cze$#dv>_K; zXp@=rGKYJMj1Rkj0k01U!xQFg)+HB-f=zpR6il14;;9RF(~=l#KWqyx3gd%`i4kTq z``jqFTSvBL;hE~h7LDbzBOfh9=f(`|p#0ur(E(f4jpUw-)jbS5+3JMoB?^7om823{ zE{Bk01Z@SG&I^^#-U5|RgTmW$t~y->Jyw<1lDsyRuauUFx8rF@nk=Vu!Lr8S7?7F= zOK~3qKjC2weD?y6ht!>0tF=0~W`^+U9rjjF!R2Lv_qG~pmexbIt1%u;^15}5w`(s@ zB2I1JRNCC{D+k5$aNl4FlXE@xmtn4W@B!IVWO?Gl1Z>{2jF&YS#H{JM))DJ=+dcUU1U77gDi=o zTgvKK*}6}y9#ywKI2P@b-nolHN47J!ZMVVu6M1Hm%R;{#0jZtoKPmSpt7m~`T(3~^ z)ZGQ^G?I3tJRcNv3Oaf`!cgcS=~3HQ7(!aHDZDF_P(=42!-rEQ&xhn$9;nln#q@SB z{a2#>&_$HXO?}!c^R%H^m3tD2B>v>x_Vea3fgKXhw5P@kAIv-{)V+J}JzI#yvS+GC zI-N)4_E=9xAX52Ruyu0#)yyJ->8qc#d`MX4K~9{XvNcw~V|j0zp$z_lM{kwOQH(33 zqY*Biz6^%JtG3Xcf}fqP{=?X#9vf$trhc>eRbpqRs6jE}?rtYCt6ld^evHg3`cigh z{u^OR2@Nrn@v_bgC9!b>S}Ei;CMZGhj*8u(WsgC?Y7OnDsnl}#r{%O8{ig9AvL#AV8y0n>9WzSS(Kj?y%_(v!DKeV+zUi~c zXbr;>d0#jQE|4y4S=lvH#BegzE9+B0@+$`Jt-ZBH;;-k+&FG)={^G4S&DjI|;)Q!H zbI0vE!m$n!Bi0C8+nep!6wkn}ZLt*y15IAQKnBwG3hFU!>4^YYdyN$P65XlVcI8Qg zul>e{@s7-8c0VmIq0wCMknf(br3DH`hcWfab0W2a+0mh1%g$=ohTH#G`|iAs2H+bM zv&rlNR@L&o-DBB^23me>CBe)#etmGnQgg%oJD>yDIaW38ot0mh0=vj8N zFg>%&kwlWe@~~vjApuJKimNhdpk5}*Rot#D^`(QQ@D-bFpTN&_XDeJ=5#N(DgZB0F z7X%Eo8nU&6d*t9nT3Emlvgo@+fqql2Bs(J^mb?+k^;XxqGX(nzebMmk_*r=2!rUp8 z;3e}_25;$IV2JtnWY(LC%4EP>WFV z7!41b=x{5W6HxwR`%c)X4Q`zP zCP!&-gbh~#1LuM1iJH1`2xI8B8zZ5J0PKj$sKie^U@w+^JR7?@qczEnbqJH{rdzWT zI&Ks(9GVC@s)4!6`f=LX+a~W~)l5k<<%_l1iK5H+JSG#b8H>4H@hp;9ET-V|4QSINse($-Szm$; zkK;LY^!c@v14R0Tmc_gutl$+$q=FtUgjS@)4+{ntG7B^$ z9b3GBFF6Z~dW;lozwRk}eG|^}Q|Uj~8ZO!{-S+SeTg5!p{#u7gg+!Ta`MtdKq;qz*QldzLkr zp#mS&k!Jguvtr$yAff*GlIk1;t>Va`kMTz^zIuHe#tT|+uVV46Fy7`sYc`wDM|TW_ zWRoACc0y#E)Z<;C!mhH1g2q?yWBZMT*tpP-7dqza ze;CKvh@82`CphX*7=`T&sD8$uWa~Z*wp5$%IKK`ToX9SlfgPkaJ>SuaZG0f*-+s<0 zc;3klv$DCmckiLviR@3t9YmP&u~F~sorZ{3lgtXlBdbXOrM__>k)Uoi!r~U%vEHDO z-Qio~=}&_h_{|@OkK2=7-aVhyI=bjgVzNK5Vcz_u&pUmdiU;dtAO8HTG9zEEYsj&V zUG>oXBG4fS3@11H*kl|lc|3g4*JB%kb`9a2V}N#_jlxezL>@F!tjiAbp7Mn_wtgQ{ z4F#53F@J?AuE}ghJWdS1@c|ehSd*2>jUCs0i@_obkVufIaw8xLAbrPiEjqnSI z==Ft8$&r7MG?IUI#dK|frhq!QRG@#1w*)jh?!~%$d*&9!E|bB`;Pr1VDbZ9PUK^WF zel`6#;BRl&)?^}`6S7)HU*30`a?2h4k*>J=Jm52k z<&N^vZ)c{ZrvYaq>tlNn)~T`qGk#1v7p`QXCSfBMsVP8e|DxM)N$ z(UpQCj9IeiBs6+G^O(4*7{cApaSjdvowJ6FYc6Uk-ZCpeUH_>ems`_oT9tyzM>=}Z zw8~Yh5>aO^uWwhsIo$YOj#q*5K>So{aPvYByJy_WjKXYouKrAyS5h?e-**MEx)EZg zq`-dPwrYrXBATA>Ma)XJdkjd+Y4l6AGE65_0X^Gq;3fQoyA=afk>9BLTq)CT7B4!( zk3e=aXF@@ZWq^7<%@G*TAC)IgMVa(rdq=!mS4Zs13a|Mw9A$?Q=^P@Zij8hj#JDW; z%f9y7#+$DhBSa@PThXF<(qWAxAzVjBH_@~?U*o@*cj@7NtxDc~l&@8RIBbDejq7#K z`J4LQTldGf@gc>V&5oICC%{Ap{2di#p(avY#fBDK54(`9cZaJVE&4xtm1SfSoak*5 zt{-x2z$xT=6|{PLWR;?Yn6+}la$C%!!#ffiT%qMg#7p&V%3NID@JUlU!Y_x@8XYufR!SZCSkcL##7BIY@QJy zec@4zh`sv)an!DLc?#!xN6H1>$ds50q_$~!;Z#3FE=y80Q~uPx+e9$^opmWP+3R7Y zrs)y!dtZvhQl+?5LJ-$cA z{udyq#9UQ?`bgFA9M3~bj((o$vHy|DCwqeFP+k;VbX ze)2NeWy<=-3H?Z(MQEw`jumGjhvmv4#>lb==y;;!CyS_26h;THiom>`6iAJGL#1lj zXdH_^Fk2PrUGT|Nx@+wXV;NfghC`KXllvZ?w*mcDw=~JDl5IwIW!s=$M5B@1bh^639EZCt08o75XnuR z7Fj_crM0E2tsO@-(%l!5y@VA=m@Egj#7#1%AZW|`yWO5|K(_J&LW#Lx2F#pOD4gti zws&0L)P+AL)G}lgdh(g#6BbK4&FCM_KQ_CNt?+rv`05w3z>~;;RE_6g0@q)?&Y~UH z(Cf55n@qjFVw2(-r*i2XSWM%Y*2rX)^+9Am>QS2kw2r2Nu}bfSQO+dd7ULpQQEQr<+dycSJScwDrt z^8vH>;ri-aTmSlUJyOSq@`H2xtNP|sKqYkLevy)pvEkuozLi_tp|My^qfjRNq6S;h)i;VJ9wlJ{R{5}^RmH3SpY zk_5Md)&&)1%$6tfI)nydFCO;G4GYH~hNI`a(?W>ljNi{5>=WDYOzc58x!`XOb&Rvl zd7Qy}Tq&HdU{m3JzXZhky5lxy@X6dHoDG%24D?GJrHo-qFR0GcDSC$^`0xwQA{v`) z&`D0%A-nlGzt@KF1ES4s$eklHsZ~X<*s$xbWmqw0Hxk)Wj^jNjL-5IcqH9@CrLS88 z__MxBu&vJo6>buU_{o|HwYM_XQ$xc}sUxx={c@2nT>#zOiX_8jrhK{f!saMB@{*tO zhjm}i>D2F~d)$`Q5_S(m4rYIQc);pt`*DmVGjxHVDnW6Cug#zO@zzD;6GwytBHlQl zYuDS7X*C7B16p=Wr8uW&pDpMys(WlHe9HPUZ+p5~(v7Qr)(~MjMwzF~c6Zv^0@$|4 zudwQRWl^?{x=X7?{n*f2SUelP1srNNEsWD@@pXYF@NsKL-MONuw#3m{ovOL?=J@F> zEJa)$T|;8QN>C#F4KEFrwaa43XO~Nvx9!{EF%v-D6cPnp92IoBDYI%jQRednzLYGBPfKc=sW+McPNL$6nc z^DJd;Lfw6$m`Z8DQ_$#;Yf7j$262KU-g$~bn_)2yE!fFZ1BGJW!puqGll)31#7n(7 zWenCyD}R4H(pai{I|8HUErh5t4yjU=a`8*xKYdr40!xN7J4^2%dkILuOdIS!_rIhn zN`G4CA0T8ScpajZaF!?8iEzpL^6LZQfgkWFMtpzNcQ=Lo?Ivrj+&drFyrgi`3pT}( zqfY@DeoZro1+<3fsSUpg*|z>{k)h?lxI@)&3|jprimQ=qp@mY4(kxWM--G&4?Vw%u zV39MLG@nolL#~(2ieAuU%aPs%FR2KnTfOn`x8;C`>^xq2JZ|=I$+*Qpijqe58<d)s3p!dmlG%TU-u`^2xDF= zrQ0i|vnPokIA|vx)vCiuCI-h*mJOs?X_=81!IS*>+p-uAt397#bl(!m(tY%#Rq6WG ziT><5sJ+*eVf9Y3T=>hk2Hn*9Fx7ng-8bYm;xwq%!74 zjh;W$*c_;Vi3;@|3=m4m#h|zmp>z9OPXu8cIcq=n>7AJbI1my6P5c~Njvv(RzPsJ? z@!>pnJTk`k`Ip{b;MSjUU<(YSAzb%KDIxH2G`Qx^ePvpoWKSzIkf`H^(;*^y`mvmn z#CCUB)OjW+ONZDY)ZZsBv&8}L4c%R5wvTDNSK=hJ);E!Z7@&XOQN}Z8iiTXW5>|J@ zJF!iD#YFeB?N;5+?DnRRA!1e8k;G9{XaCIVb>ZO1B5n=*1t>0mU1K7 zx1|^=*ecjLn-xJsyU<3Vdjc>!G-W!_vnUjLr4b zYq;CLXZYWH7#S|^?RgvLU(MRv~Dir1Q#IPU~l_S*6S0&jXdH@KQ_xZu` z$_HqavL#c?6l6HYN0>#y85SNJPxG|_K#Q%ZD2|BCt#3I*J#Cz0v_ejfjL&YPo|d^! zcwE&?8qk=XPM=zvP4F#$GiBZJq=L5Nfxjw8DVy3mW}XM!&=+rX`19>={|pNS1W;90 zzLTzwP3yWXC}dBi5kz-Ewr~abEjV;Y#vbAAZXfnuHGSK?$+iD@qd%?gboX<{$XPfI z$3s)VOBV zXh>`0QGMflO~V4Qi2*(q}Eihv2R_V_BWV0)atzjND8`jq9AcKl5^_TBl@9osefAAY8O z31_z@2#w6kJV2VC1v_c?NP=3YjpjE^{0d(wdk9Xp>uF0W=0P zG2v4l+uH#{-0W%>f}VOeM{|FHDNONKS{R8oLWv+{{VY9Tp$mY9>045{Al|pzxtT4r z?MKZ|Jz5yia^zZ!bMv`d!62oM84`1{sBb~rSWF!*8XSqvm9av9Fni(%faM>y6X%!QE>$vW5JVwvPingTr7B7|vAe~@! zcqOS-+V|4F^6O%)q3#Mj{r#v%p&7&2c z#Sr#;ic8mN`C*v4Wlr5rNNGq-kb~!KSt?ty4-(f;!wLW>HPM5s5(_rUgM#x+4aYQ7 zLSBMfsS}}R=Dk)~Uo zQ+Mz3v1jkoM5*KP;lk^~x}zJ@-%V~wFrI0O(Enn4OXOJ&8KeG{tu#_z|Lm*oRKdp= z@yyyOsv~T0`zi2uTA;h=528EanJ;x|V&KUn{8+hbMuQJ6=H+MM8n-^rIz?Td>W4G2Z*k}az zxz)zP^6x3wOxAsnBJf-NrCY6$v;F}xsGf(g$KoGPCcg$(P#wSC{Q8!8e#n)XRsk}ei`@mVaWVh;(TM8O1ANfs`>&q zU)|w)ipXOuP*-v|@29M{ONq!kKJ^;_-EYCC@xvc-KDjC&`QxkaFMl#qPfM1$d32Ou z%UiIs@tmvd@kzTwJdhUCq}Zg;4&cu3N`}5WJSk~XnDsz?q99w>K!cITrF91sQr>bPzZ_F{S)O8)9s`T&X*I?#upFStrkCoA~IZGpf@El_Y zRFlV^dIwEYsf@C|`s7zgtgFG5*m`MicfGu=_cTT8#G%YZ%PCDB9W=h5VgcO05<96~ zy$WPtIj@y>_7Px;-~ubf=G9RFY#DxCpeH{R$GUD#PIeadm*jMj-{{}VOqaOaOY}Pq z#16W|t>Y-axY!RJl~oiqOr8FX*2RH!9|NFJF@W&n-<)*=fH?FLnl1tWi;Lsfx}twD z)NyEF94r_Dfm{-Ye?xY0Q4M|ze|qA8tvEpYWhk%&0CI_Wg-GCB#KB-(7>Fc5^pbTf z3i(SC6$L;qfx}SnrSv-)6ao;toG2y%5R?2JN?ZhIgo@z2aB+Xf#yS2q#)ZaNa8@vG zB!MgLPs?Tfm&N|J{cFZEfP^GK@-n03WyVX>Gu-pvnC0Jn`X{>hKRPLV+1X3LJ`PU* zH)t`A3vcC$>(*}?6aW^%Wdn(bKye9$*e<5+Df1 z0i=aha14JNb9o0VI}2QyIHteTeG3~aYn(5xHXNAW!d?%61K|J8fnyotmeOyW?*HN) z{|Sq}O#3GT`A=Ro7$hbN`b)*7_}^!H3E_7I2&>sVzr-v5v1)Mj!W`XxL;3&RDGwJ5 zOMnOm7}w@MA7BnKC=>$au;lnd1B;4@iE#Y3aya};yF7yYOZ(IB505`y|3d?b{bMfZ zzr+KJi2uV6A};!O4FrY!QxgUMS<7GbfkZ{Y|ClQ(_Fpu~f7AyO7x}vehDeJ2qdt%X z^zVHEb`V9FG?WKz;Ze6(i-Z^w_yex1l4Af_zaZJJFHK!u{(d*XqT;v<_~y-fT1tTb0W-bV7ytkO literal 0 HcmV?d00001 diff --git a/publications/whitepaper/figures/caching.png b/publications/whitepaper/figures/caching.png deleted file mode 100644 index e3ee4dd94df22a91d0a6b49cf2c4da153efcce67..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 36103 zcmdqIby%Ct(u8xyA>$Ky*LCZ?(WjyPSM~{v}kaL;O1tN@aHhW#2F1qJ1qw3N6K3JSUw^7|3y z6Xf4x7x-@!6mJx1aS;`l**#qf6%QB**Z>Js8X6jyzMrWueFbSTeJTI{K1+{9o}>Nq z!NIvrIrW6YJuj@j9?V-6mu76H0W;;DH;3+pVFJz@WYoyX{_g^?fVoXsm-QO)ro-Og zgyO;nT$;uC=OK?CcG>b2!^WEVwuth%JtLY{8eecCR@gnz%d&TZ@U?S>4Rx_w(947`z2}j)7V`b zg+~y=hrZWn&Lw$b_xn1f=)bc7DMAhyTk0XcjA@BGJaJrpld#@RlS97goO>bBIChw852=lGq^5Gi7Wj(=*lmsubKCrc%Mo%Za0!BMheMSV&{QWTj^Og2!ilvb zsGUPtUvGTiJwO5CgUUeJ@ucc{uRzsBVnA=rP<~ubQDDh42LZx$oA#(Be<|0UI*;wl z^QacW_zXq7dab2zaz9(^F?&<92If1fi(tW+ntE#IKys>knIdoz0g;xZ>TL;{RJP20 z7(MojdBZl)n8@#4rbs~ zH`}(CLSv8Q1%@cja|= z8?v)7f%V#rUbwj^S;*z~r1;dv?K?buXPR<7*%>CccHQHg8TCjOubwu@3mEU};ur*(<(GWOyqY?N-y!5n;ZKgZr_G9mVJ zcJF|2X6}>J#nYP_ZWEc1x55r}WJ6;mfpz;QhcJdaj6NQ6)+y2 zels>-0QRh~O4K48YZ%~&lhL1+SX(4{agi73cSxT#5wUjRJLaA>diA=1wh$Ik2z$Ik zWS^?zRNe>wPTllc8?o0FS3#4bT60;aZstML+UC`Z_Lm9Xur5YeW4L!}wg==q!L>u+ z6bii42ATF^oT_gyo=`OpAbxasWOb3siI)KZn*jcvnISrWnf|j#x8=)3-T~DOnyCTj zq5+HQOtUU1RJO@gbk~oe#|+m-?>Nt{<2WO!16FnKAm7v<;sB~>mFap$5`9)ppx?4` z5)+P>T12i>7hCky5pjyxi@U9nd{+tGb+d|_>v^lCs+rXb6yL|1T4eV9Yn6zxp;n(?6RVVqlut+3njMK0^_ zsyuTstFcKnU=!n0lfO#Rk(6>7`gu?cmz15b!d^`fFj~;XNTtFUFucWj^9&4?OZKMw zihD=z#S)?0*Aml1*`9mb$MnT4k;AThXbqYj-%z#@rEu>NT?utB4e^W{&6k7bfoPN|0y{AbbnQ^GFGW>>eZDT zur%pBuydk#8zpntyz5ie-uK(M{FWirE2p{z%`_#CRa8Zq>VuR`2O7CC#6`rh`pS7& zr}7F%lT3l6S*8PxtY&0dvSaAN@5(+$OwNaMR^R&t0DGDr0j{Q7=#CFaRk+x`9XU>* zDiK9Z5U^U3jTWGKpYSxqd*;1g@dlQBOH_~T;2o2lHO3+tM7DbDYVOp}z#l~MHnW3# zrMzDbv%mJ37WF)gcJB?K7PFKpS{2Zg5;)Fpe+!E`O6~kZbw&E@`Eo*$yuK@@tLJjH z>=Vqg@ta~-m}I*}9?QjJvu|%lOjbU#EB$#OEie7B@Ncu&aBuz}b;Aq-8_D$@Qj{4_ znu)d6w_3{OFj>je?S$MAnvip?KE~RKO7l7Mx z$B4DP(D9Xwp!LUeJp!u1rAsC;3<-sN~f|8^^y1`5qZU= zSeu1P}QkCZ8Px|>06~z$bicJIFWLp%Pfz6Uu#c9j$8SC9KYT1 zVC>cK(MeOj`bd7>g0Q>w1R4Y-tpjY@zNg^ZUu%T`|? z!cnFo>z@IJSn~IOUE*MsN)`=t4yH7ii@R8EwU<1HPwX2)Ki`R3K=5JLGOZ+yq-u$` zIzDbq@=}eL3t1OVIy{(Jj7VFUC?a1#Sql%P+Nr&NB{~xoiVeW9vRsIsk>T9VUtCLO zsLmsOV?prtfH}nLe9|oRN{&!V3`{<5qZT!OD7+R`fG$=?PG4oFb&&&vtw$dz2r2tb zY9wzrL@xL?tV^2=Y&&u1&z8^x#G(^n{wqEDXJV#=Eqpz9;&Bd!!{7YiIhduV(Y##u z6Ia0g-m1CaOYs?YH`*wc69*FckYR?#m%2$kX+5v{oPvo6wL&q!sd!6DQ_R1;m$cs$ zCrwPRDq~4^*>PfQjFsvqAl7mwz#!%!)GR(zHjPiuE5^DC4Ois0*0!M+Ee0H$6ebN{ zKbOAneZ{%J*I}i@t7y?F={v)Wbb?uFYmk({tC&*`zf8`uzWHFk<{s%~fhMK0M>IO! zC0AQoeQ85W1nw-j4h?)RuQ31emh4>Jv^kiG0rYId?3M5){tIKQBlJXdv5CwDIjTR_ zr_bLkLp<^@OCt$dK7XQs#X2p$$+}@+u^^`;Vl?iC}~0|Ldbm(C>o zCu#gyGs8Wf=eNBUp(N(KI?#N%;!=#b$35+~rwoH}lMPF)H8$w}_P3~dbT>2b%0bCr zZ3ei@FHu)AN_k$le+>P)(vUL{mgvu!y(EMhS7*JKA^|sebnq1EfcWP)g;tUPL@ZnJ z7^AV&X+G6QJEYKR12e4HZD>5HgoNhAco@P|Am_gAA6IqV`x?Xgx?D{%RFmt{-4RoT z+CReZp(>D?eY0KaKcB9}M2nw$WIh}56uYxZCU+TSK`VN#AzMNl^;dG#D!y;gZ z;;UpkA)9OEw(G@RjL9cERQ}59!u3;Q`f1#)hacU;3BQ!{;r>Vw;>gjaT@*0IB|z!| zJhBc)BpT#mmR25rp1%Z)Nhyc$t3ICJ9$n+2sZJ0$#EdB5fy5_?X&={a8EcS`moY=f z5>d_4Rkj5}N|fF3wJx?D{3o9nlaN^6fgbO-^L(7Cvuc)$g%7Cvb8r%fq3_x;x3%i( zKR-2c^E5m)VAXqANQ?J)L$3{5=bt$$sA6BlL<*HK?HQQoOX^~%rX!a~e~xe`;5Z%2 zIbxB*SsGtUV$a@;!ClI95%iJjQ#?xam|sZ==I}REI~hVPV_IA3=sQN1F@xC51XnNd z=n*P0y>-_hQuX?BGXUxfSOq&vx~bnRog_)xBQO=oLb7rfr4&g$>ZGEtp|IQhDn5a@ zERc}p{@uieK|b@$svWL{@8 zD!)LTQ!Z<=-E7L@yZShPQ!8GK%f~P9B$>MUdgoLt+G=5=8wj%+x*$yK2oLVZx2u1@ z7ts-i^J|D6pife?ch62wxKj5@yWU(jzDM1j|6}{ppwHUec^PP6E?-meqKDovG3e*&fW#po z>{jwBFLNg@IW6Y$58YdJXi?WzxhD#nnc}(Ov4(Ufh$dG7a2#eB%zplJa#I+cr zU}bk!J{-QlX_uQOTJ(yA`Ksk&++;ynsO3DgT$jo91S)+j;}P87PYf+z{3MVkSmaXn zczvn3=bgNcsx=QQ&A==9+dE}&&Mo&}C*A|E5WRwow2n2tQMP61@;>%aE2vuF$H3=S zQWO0@Q3VDx!L+ghM9(TTFrSwbK@$U9g?^bi*imq>)6uS#59GK`ooWYIW2_cyt-V7T3}HnKu5Ol(mx75#}LN@Jxyanyo!(jJ<9t!1A$Na0gCxMC_XR@#BgSV%cm_VxSsnk*jgm%ii~oR~&i2Q$Xtv~y>&u6yR1VPMv zOb@Tf)wW5GQ2e`4@BfA?j_2P#-oH#0ypsP5>HUxIk+*LljC%tj~ zaWphc2Hq56``YMozlREQ&~~e}zDs6h8a1f&ys~O?*y|fOfF<*zBwc&51iv5+W?<&gTV*;&u zO2@Grw{6rt{Nt-P6Y^5AtN7Y!Wx5*{LSsV4pnIW4*FK&{D#opEujJ2+W)-hq4=(rcWY{mRUhq(_^URA~4JusvAx+8~JJyTg4F<&G zqd?BYBT#b>SaF>ztTSFKe>cvn5AR=v2OD0o9OXn0#{u*xsgS5L+ zavF0+EZuPA;}3r|D%x_=vRabu6`k0e%2IM$q?u?3RK?Bg>8RR87F5nDT$P{)4F*u>jz!mU=InU&tg)*F3fY4Jrbft_ zo&&16S>H0LZ8v|Q@m-)d=BFkWxNGI z?9(+9@4jV$@;fupvPjE_b*;`(&9N;rox)q3R!&^3e&+!=)4dxzJ@15uow=!vbTT~G zwceKV(`3d>sY^74VwO^mAG(2K|Ips<#Z{7ZB*+q5VW3DM%|Xb?-L1wtPyp{)pAeCr z!mQ*9c(skQHP0}(0lgmlId<%{)0q0JJBzH>7;RWt5>%@7>B1ugrQHxU?@%UibTIkVLmc6$WlWtfhJ2c_P2rx)(kPze<5iXYASCShD_omA>kRW2NE-P?z zGxTyKiWRl%cB)@c{wjQ|=$}u!$j`ZLF-rLr<4?idlXnt9{xmN{f7YVroc($p9FJ1r zs(44vJ9t5pX~NLTv(x}7Pm{l~f{~YIuGM|Qva@>2vXLiC zwufY2gJAA~?hY5tX-(V955OcGmMo5EcyL!~qZd6SRpA=jL)?_P9Ybk2YftD2b3&h_^X(&#o(>#8#!IyR zq7@#Y76lU)&s<(7>1Iwf%G;Zej&Qb2T)Q@|t}di-jG2eOi8`MYt~!Y264A7)!ikug zY)p%Lfq(9IyN!1$mX;Zz)uVdVY@rG!)H|)|ZI%9_RC>xny$Y-BUBwAZMXi@KY(MH5 zu2b9msqel;Tv=#1KL!#b@!SC#|6}oyOe^rvkN-TX0O)Bk;V3G?Dud9Z5pyn^8GaaV z94_h@y#j`Gx<+XTG2_weaJj~pN2+lz>3jUjYTf(Lx$>8=a5#2xL12k zkzluK-dT3LF?Qy_!vq^;h4WMM=eCv&8srMaFUr=MaNXonA~DlSG+TCyF-mG z1M6(xP8EX?8?Mb}G-=pgyr+1@E`Rl`(8*bB23I4LS}Gf>zDODKf8gyNT=N_P1icdi zoE?*uQvaMrNTr2KrMa$;DT!ACl74=P;pawk;(_SATTrVuXOpHY4!-hP#Bq&b3D$NR ziEtsJnC#A-UST4!XF%(e)Zl(fPbN*7I_ks}QRwC5WCHYp7I}5ITMYO3Pv%h@m66K7 z2v(FuAf7_b3t)E76g8N}-yS3_dL{k5Rm-Lj-NiK~4zIt9F?43AU9x;v8>{xq4oavg z!1Ok5o9_{21=7nrz>li1OLgrjUSV?}_p*pfe;tzzZ>hJ;sEizGulRa7Q@289IJr6R z2W%%@{v;@TiuqnSj!ZByR&4SPf!)JtxMJsFqJ_nOk*2on#^`f$o!_@JvR2r;9hV+q z&9cC=O}^>;wcy6oc~9&#hT($S;kOtpO&BzDAke#+ZidmWYb7KzZ+$S)*gkk-8Q;M7 z)Kj5Zb>7Gq;f%%`bXAznqH13hg$8LY!)E5W%&n=59>{2%A;0GefMj)3aVy{0gm6*ipdy;$Ytx0Ug z17O3lMyVM)-DN}fJ^77nPr}G)iB8UdN^Xbo~kyS;q5b zj7ztFL4sfjW5+!WuEdF{C2TE9aaD9V#P=883MqKgWD>FY1>gTXjg>a{z-5qa@bD8q zh4Q-HVQ)UBhW++%TG6SVe-3*`uoyMu&T8_gYPf+qz>eiqvX3~p%lgK^n|~S}Kw$2`ERI;EdaQEZ>1Jj3q%6Wce%Q&C$mmzM`4d*!o{Nv$r*)-(8affb^P z7KsDcWM0>gGQv`1%y0U39TM`*F{AF>I*c0-5yOy}C#p^Da=w|2oT^=!H!nJ?CBP3##8O6{mD3spYpDkUrNYcB23%oVa~~qlt4J5I-hX z=|x7JmXE)NWi*X3Lt}s%7D_Psun?Gn_EyRjKH|>FYacyS`J&9jLCe2E{%L%2=)&*~ zPSim!Yjs8v8w?fEPA=lL*2?&Zh26ZbJtO9YhzrnXDtADactB9{F}PGSuhEZt^Mw8n zr%b?w5D6|FqHPgE$}>Apu&*4+_IDfA*^JNY*H{-Edn8 z#7~Dj+7WZy>dV}odOx3f3%7}QG%7BE%lTgoOTcWXYHta&LdlIvIqYJ3cSkToTBNTV zBq0;s^)@TAgj%%rzk_TfEm*Cfz2XOY{#4K`ig!(3y=a-Q*6f6ct5cF7P3$ z6xl+!XJS;=E#{QDxN6V$twSsHwE z7?7IY!*ct(wX#4FfEEw-NjK4^MfwyjEI_??y@4HHITf?m6Ee)#Vnp*N#X0M>bzFgu$z_wukLAQ%@4#)oikT-b_5$wBbVm} z(@8{iJc%1?_a;^;D+Pg;=eCCWvbMUifqZOPwIZ%e>0rJgQ((I@5b#vFJ(~T&P5BY#ZL@K8rWLMJ;`+t#{#&MM-O9{ zioS3Lmc)cq83(m7_b7jnEFRiT>Fp5Ssf8+wo2IL+*!OZKhi=to?R*f=jQKQHN%a+K z9?hPGY0~PaR&l)AUh0k(mpq6TqoRi#k$X+*-Bp zKdKm#w4B?67PZGaY3?IwH?ZBHV5FkUywQigs)bjbEW?F+xg&vG)PU( z&ZSS!1~W^?Dyrx%fkRW`hZLhmwM2TbqNySzp6FG=S5Eeezz?v{d0D@sM3vxPn(1D|%?R8N1vFQ&e&6k!TJ%P~X{{ znw7qi>h6nZToc(VMSI@{XXl_y43R5dLvss+5Ex~KD+QtWuM zvY_ZB(#`DeT(vQ=9A++gPgj-cDtDDgv~|RR2v&0d!o=LyJliYP1Z_-+Ww|5$Fh6% z=VVm)n`+-!)TJCaoFY)b!6tMwnfMKF*po#H4(b&xR8~aL@asm!7Q@9`YzP-jeBwJc zRHBOqa4BAG6dUvT6l&#<&sp+F66{Z^6o1$(M&DDb7Jhe)hvS^sq&$bze43Fi#K(iiHq2!oZG@(O zxGzw+%frqb)>=QadUE}BFGiEYv(vP!YZ4abBAVmkw<%~f&5 zdXhO@dxrwFD9ibyk2=3@*!o=Cira^F%;)`F;vR}^y-W=X0&=EBuuB-SKBZ5QE7`>s zwsDi1Wym*4#8R69WE{TC(0QGEGf+^}q2>2 z2RGFb0W$>UZwk*9^4g{bgO_qpc(<3%dT_5IKk8QoQ8qDGSIG-`JUivxYs@Li>#ND# z_>Hta*!lb2%90izd9Sb`6U!m!KNbItTm(N#}!rE;+(3Kefy)p-uxKf-P3=J);AP^EUSu?l=tm-=$Z@ z%GP7l?`_a`a(>;FskmR%XyPlJ%KFuyx<<)8sffjdATLPQ>W>}cX{GvjR~pgEYmr@`S_NBDmfX0;jMr-%#e&AFQVh7K>B{U~ zwb`zuwXfGQ_wtEgzts=1E829@A|1&{9cK`l96p1m6&;?nAQktO?yJaR?p6atuW?3b zTAu5{V=`ob=?3&ESZX?WyeF(sxc<3~Y>oC*o*~767_KHD&U2?mbOcP1GA5b9v zDJ=?!kN0+DeGse2s!z7K?*1^vqw)Umr57m3HYuSY6WvmiGc2sZIr36<#O}+ zq!*7JDbzDnmV4O`RClHz$5an+gLL1|LOdJ{I61YP}_UWM5>i2beqdxWOUXJ`4P@Dcn!FNJl z6oiOjsL*-iGL2@1F`tC+{S!lEGDajB!0v@SYFlHo`;2)cGd7)d5*~6UM;066QCcXS zM4k`nGL#l~P$vS*JN8a5fWs`0UV!`1*lGwuD?~@;i}D}dmKrz32QrcS zJ6ta~cv;t|(lhhp8~5wYREcu6|Mw>G%_(7+T39X05%VUQX zNmOU?_8A5p6CRn?4n0Vt^!&ogmUT7BO~7RjTnDg!*YiR<>TgDiPKeg<4w(S@rTBnq zsRtI^jCK$Kt|OtI8~42V!zeuW<8I-_fp(XA3)Qm&x!{6OI$qFl55{m7!=ug;COE+VeJ_Fr631UWuXOVzl zdl7mCKqd?&A5v-F6vDJb&pgs^DPr`c9ATho*fk#09uDZn8Rl3v|1({p+w!Lm^?db# zzQV?L)-S!7 zw6me)zqv?Eb)J+7w|xx*vXZfQJ^z>amk0kGC&+~A14)I!Af^|W3^%%;jv5ER>Q{#< zkO`~|P-l~ygIFINO;}hF_T{1wpwKUCN}9G*;5^1xm!2sfhZReuG4_Zh3mAbBlu&6M32Nt@|0mZ{o0;eYHXEgKLt0s;-MaF7h7JT+KNBn8y>byonI( z>%2q%n~bd!-UCrqqdwQh5Vf=_y2Jb@(|a+su=oSBR06^Ba)edYl#t}7at7N)_)iy@ zVeUI-nLQgd!PH{`8GNn$t3$3xIPlJDX3dTp;=w(0nB=OrE?@?66nef9K(CxRyo0t7 z@4aM!uKRrM?!)YJ(tp5D4aIPmGP&0G`(Nn)mTH46^eebA^wr1b4?QrAXD#JVmy=FoX{&cQ zl2_bj46-g#{U;CK){f$W1gu`-KEMHMJ@s26QAQNdwX132f7$D!XG!6c3uvwGhZTd7y1H*j7mlmloULj($!F)-7sbIYOj&D0a;9gx zpkPMIb&NoVob&%jwFxr+IUSe&7K!^Js{ z<9>f_p;I&#OK51-kXMhNS|G-s!+ANwyOLoB-~EW5!w_i;WT;>C z$Js+F!g*JbMnL+vkiW$M#Pr%FnByh`w3Ygqdc)~QpS8*+OQH6BtZptw-2Tcp>ebL- zxc_M<=#bj6mH*MnC&7N?#r}#6`sN!KyI0GbTXoVQU#b?CeZ08+7G~<|PeNoPuT^2< z0)&sMbNd={c0S9+JBDWB6*`mX4E1zJGJP!Dw!tdZm$no!d^mi6T&7L;BQ!Y?%KXli z=2sYr3n{&@PU0T;4{Ly?1f`B6k?xWBrJPcvG${BT#T+qKlx&txt z^~0a~2G);58qkN0REdanoLCOz!x(bZ*(YNXJ9Dff7t4Ao~H5b z?i*q+p)yv@Xuzy!z-JAbPNC%l9RQG|m6vZ=v}e2HYiTcuKK}I|+d{I@8HuL1da;%? zDR2ud9t|!6p;?mH?_#bzz0u9JoH7nw#DN@KrD40zim~;~bS+`-nE42B6}n&KgBs@y z8p)+$n626@&oi~3GTA5LAgp84yW{eiJR}>6bgXf9>KW(Wfv1w0qR6T{5a9Jl$AbTA zE13Aa`U1D&+YYV4t%ky5!aw)sB`*&JT*oOfONG;8|Ft4K1HqZET*614W57>3TNomk zAN^PEXmlRZ5d^P61+&f^KEL1ZY`&2~FrQlna-6Dy*aUEn0@Mj3lX|uCjw0q{uv1%8 zR^Epcmf{St+9i=~up$$jWsBVRrSf9Bwkeq^DPK{K%P^%Y7Ev2vS-!QE(E}t0Uy!cxs)3w=24VI9yB*28`F3RC+EwI2q-tv4E$~d8w z?qi0DUSk`W+<-A_e>GQ|zE$T?uhjSJv4-va`=gGjfIYIu0HlrW6GSnJzJ6XU;JCf0 zebzMhxKzslT%30#SuIE1@htH+1T_=+%u3}Xw+-$`8RU#rI%kS&X#PD&yEdmDHN09F2>A zoLk#*RxFP+xP!HQ8;(s7)&=Ulc5m+#YU~80^QY0e+WJqTf}1CE)V7kN8@yrTa~kRQ zT6lI`Tdg?w8?BUOa9K+)zTot0-|EIcHQmNsk?i0l1w)a~R0aaWM&qW2mz+QYP(elA ztk`vb5t+Q=CF;v?_6TTV-Pdtm{T=0G>!c>U!mBSS2+2TC2hF}qRg>0=deF+jxo@xD zM&cFuW}*S1`kR&VNwGO}p+pm+bm2&3X7y8@LDH>qXQn@)*7_)2X1cQUZb6?>(QpIo5ZC?Ednj=nmSaNd z@#BFZziMdjB;34S5f4*0vM9FXKnu9nn#!5&l^6@9MjmrGSH1BqAl!LfjIJ# zCGzwheijL{j}Aq!cYE=Lj6(nf2Ll>QjgHzxw@&lC1Fy0-sBg3L#PJ1*SP6gcCTSgo zHV(dWx0r|HVJGbRy{0|m)jtiRZQ&4n8pg~>TK0KNdB_@^d>jN!#JbzlaU#e6k|1ze zP@LUO3u7%79AcfL$pBH7Im)LDZC`luBFx^_ZXtl;GtuW_2eAi_{Q)H9stX6n7fy6@ zBAOfE0$^fwqaxTzCV6@X(T}d)$S^RK#=a&#=vlZwi(RTp4%edtM99P?EAnzWxE2jF zTzxa<-g^bb9Z$ONThAZ+mer@#eV?#W`UZi>=mFdsbOhJRW368#`naP#DDRRxa3>#w zMZdWRqiOgHyO-#U1iF!I0$K97WJbj03@YH40S$-M$zfBWf|Ex-Z8bNrNv5VQ^Q)nU zj9Fvx8s;>9>~skAA1XWdUsY04znZ8!2%X#spq{m}Mmkknwu@OlGgN8w|HrH?3{n} zFuE5F-i;GzuBmI1)+uNGdfMYyh=^BB`QmJ)8oueC@e`(^m0-WY!K{&XQsv4Ouwb9^ z+NWL??L{UfC#1WzfgV=;C`~`7#0i+eV4O&~oqoz$Bx|8V%i)#rx$aUeW2PTLe{SjT zQ1OVtNll$@4offpfBc%Z>1ti zOJp|KZ3%;5u?r}5DG4UgFdA>hG0=#X4(9`iG-76`;LIm}&OpERE=BKfZ4K6K|iY1IhPH|4f!e1Ch+MLG`|k6;)>WxvR$yLic9`kSZU!p`_ccl z?38!xX;@W7`YA^OVTBkUc~F1Nt{Hit+HafLAJSbs65}o#i>Ud zDp#L?Om@HlMJTiGk?)1 z15ff`dT{oI#1>_Mg(f>GxjNGJN+tarE!lj%vR^YWy`l=fpQz) z&wcH}?FHv;b#{H~5?sla9hTx-zh{HZmoUaX!=#8@=41!H*8R$)Q&L-h;tJtA3Cph{ zJ!QFG2(LB@ZE}|L3UtMOV_@Fpl?2$7j1@%H_PW0CF=|3mOqLu=Br)WZ!*Dzu7rO3j$ zp){gG+zG!bn;om5Rn4c>g;vTv<>o6w%C1wf{t>4kUKTWKsu2zu$wx}vAz0vi2+eeo z95e*$H15WO{zKMKomb|^9UGt91&&}-w?2jLPJEVZZLrD5^;Hf0+6t5k;D%c(rtIFR zQ#Sz?@2Z3)Jp|X&jvP%YXj-(yj*oAR=Z4Z~Cr;WVwI?nrNqhN{1if1D_6w?Sk zo?YGXgpgQES5t^KtWddVFb^}iQdvVhShwsZ%Jhv{1X)e%x|0>WS=C|2^YdOR?m#&` z#j{p>a5PS&$5&MGm)3Nn0`VFYZBr)$u(wT)A#>b!*vHx5|Db!Y)839sth07gL=o|M7$haGWU=%ZiId+S@tN_ZjQ?Kpn;X1?GH8{azmYPiYrMAks<0%xE*WCj1m z4rAx`mi&}a@(IrbHTU?VFIlfSLQi8PqdrkxIt$dT;d|0YPDowXe9l9L`x|)Uj}lOQ zV+8Gi7d~}&*cHsP2KuVnF}-$v=$hsm0CD9=wXfy76ytCbukRw@c+XrGb7Gc+nU`MhMHM5`iEK zx;O%FJ`=r24PKq9rXIsVzm}8^9~r~CJKVmF`1lLpNU=@9X{4?QnFz=q39gn&hhpIN z#GLl~T@oWrb!zz@wa7*p75S8m4?E?P?cx^QqUHfs2hSA!jW_$BBEI)nPDI#KG*d6} zyKdMJ_$1V4wmi3td8`kUz89sq!Jelbisf(~1RH4yQ&$sl0b>H|z1euymv!46Qv;Np zPwe_k)k z+AEpsUD=G?#A4fD4AhOX%#M3*zp=Zs6qJ5>BEnvssy(R~OA{G6-b_VV9xkK|l@cbV zw{g!DjRxBd3N$Au%dboEql57t+#S!n$Kt%2YN5PJhQD*R{cgUFyumqjAB%#iNHP~+ zbZzJ;KD8sA)k|ONp#>kTefv!SQFV%~aYd1yxuXD3YXKiSk8~Bv%huR2U(lDqKU(&C z?kJb%;pX7|CWd2;d3`~(DFPct67gIvh@CsLiWXaA@CoOW=F?noP*KoE0GY?8JdL`1 zUNPnj3Kn5;pT)bUA!S`l`6WoS zXMDL>vgAdh3oFo31C4{Ij3(?^cCgX7=#a)f=EY;e#&7Ir=z_-4Dk`R)gTcFeo1SLQ zPS71IRQ5!&M=$R-@Va5blhSL$lU+ za$fPidDW}|@sEq7d<4>=k$`GCelS_;spL$#4ylG8p30RVnL7EG!Ijk=C_@>8zx=0k zj(ld42o{p2Ayi|bo@FR@SuWR*pxv(9WGYG?-5gt$WS74hkX{x0hy-3XsoKXK=Qq2M zQ}mtTiYG?u`W(YU6JzGtdwVUjxf1^JiR*RtwrCh{Cd4UbO8(t}F3e^sqqca;^u8cz zn1MFeBlXik!Q9&k4cvTm@L#tV`71lJm*(T1;&2c-S)a9}75eLwCs)ppGecS@DaW6# zW6SyueKfVagEE2ByLBqxC}SFR6W|xchadiSi6yLE#@JB9 z*^b8_e^I6g$I~sbm%6R0Vw4<3mRDMhwj8}avx7ML?e1$}xdVbdw;BB>zmu=N7`1f% zf7p8KuqeBBeHcLr5s?s-4k;<=?(XgqkP?QJj**fQ5b17)?oR0rX&7MWksP{)_y(V6 z@BQxm`xgJ)$2|x4TI*VUUFUf&4B3T9C{tmN@|EJc&op+nYjAfAwO0rRCcGle#H>Ek ziV9#KTRp{2O1nja+#X?dFP-hmEAs;R>-c6K`Hs0noNo`_+(qf!d-*>&mx?fvpUuLR z63c{hP1noD;vkVpp`;IVh429WrqSMbb=>@Lr@22K)zCV72%s~3pM1+ohOnALbx|I-!Inybf$PwbS#(%9({Mob_AH4+s zO_K2Y+8f$%QZ8M)H*m`S&>TE>N+Fb_Jnw57N+?Ak1%%N*~xb6<46?4{#WKW@j7i;Yb+V0Hm3|w5{?q6Hct_}KP*^SwGmipT$k4L+*`($sodDDc8E%2 zrS8sx!Agy$A<|v&-Dyp*wh8&l&9`fdUOb&ZFUarWDw+A1qh&$7{uIPfKJAR!SJCd% z*eUOxWG9|ihmDl-vqsmk4;G32y7mo4+ly)hG|I0t?ULDAMt4Xzu9hIqDdsW3JcH5q zx~u}nJ*f8|KoY>lIg8^F*mpHIwY)SEMwe!?g*qyi9bOBNmNtEhu7kAG4TGOa&^R5o zj`z0n!T`>0Q0@Vn$QzATo$>3b@ZreQQv!?BuLYX7a~VDvjA9N+8$VB45326Q)Gwt# z>QIgtKEsA4=Ky2TzK0JDsfj(pcrDbThdzAI^NPd}QRZWqfS2)QUhaq|uL^x2xaSLc zG>MYor6B4Fi{KSiwogAjT(-vW<1LYjuF2w;*RS;(27ygYQMMyzQbHLRa}KYL40sVo zF*00zR^*7n-lwKi?~or^)h#5f{+%s_k+ajm0Ce`E-OD-y)vYynF;BWid3LW2eR;C1 z$0@$>#oLHI)>n)AgYE<4GmOb>A#bI4Sy zpOGsRl3VPXNWLNa*g$U6%@~(5t|re-#>T!Eq$ zbm+mSd;o4K>-2)d^xq6_59YRXLu;5=yGqu}vYUl1(*O{sCMvG|%9!~zdUfLb_9^EN z{Ot23$Gtd>m?|Yl&t^%rY_4rcLPOdGKeWn8-bNAoiT%r4!Kj-Ze-HvEVCT+Q{ikgZ z=Luqq@$Ai2LkTQ5Ip|Ab;Ty8(wT1Esw4l-%8|QU zLmX=yZFLrUoT_?!3j8}<{fYr#sxQtX=CH1SKxprY$mn5*s(PMx^x{3aWS>@5gN zk2g?P!Z3;GzMA>B3i!URcoBQ#Z5;#Q^kC0Q^RDYei5!Kri{M{|v^;0u_H)9*&>%W~ zI+^Z#k9+cVy{I35;0{S9yBUq2KrV%&&)LYgosF}P&?FaJpZZ{r2TeRjy4!l3#zEoK zqo_RrMs-ou=BEiMiDrq^i+C8>%aLfGnXpGqIX;)y?ch`ars9H)sBi2>FW=YBQu%~6 zz9fI4qp)^gbu!Xo9@O|Ek-@|o%juU>uWfji+yZK0SP++pn%nQ+GC{Na{RdAghh-P8 zuda2_%ev!OYUKqol;{Jc!ZYJ0Np`xf;C9)r0q~Ska&i<~B#6h@h!#sLMZ zOwuU@YPv#3<<{PZ^A7`8j)a9ETdwO%#_|qPYdB?X`0}`cO51mrwo(SuL*7Z9J$wTE z-YXx(J~k#UmW+-KTtQRO&OOg=E`R69S$hRT5T!-qCVzG#e8@i(70rKnwkq?OlOF)Oh4pO~C= z6St!G;DCN_Ycx?Hv_}lFpzQZ(@kWl48?u_Mptx=N@gShRA#J z&c`UzGIlXkZ5UnRMvxak@mbtBt2P?XJj^qUUlDg~u>@CpLF!|OwlK@GBZ(qK+sJ=F z2+FS@SZ5M){jXeLxjpQm>2W`ociJ&M!kU#Df@hH^@#}u`+n&8K=>Qb^a8tLo#^;)H z37-FnV7%z0vh4b$;|@(~_j~rZtJVnCUS75M_#^oH3wvrls$Vie+6ystqigy+7O2@# z;T;7%-z>-0&$vFs3@~SL(Ht78dt8?Gw&Ete7re^%dA(!d)SQQU$u4khR=Wg6IAF35 zLJ*V}J~gLkbxQ{i%{miGOPZPSjPH#Uo1dkFX$}NwYom(3)HeFck{b?%am2qWE4d&e zW(-F!Q<}lv3)hrg%qQ876A1$;FC^aRVY3;0>bPLQUZ2X!3SBx;V_G4DHy8-qnc%kIrUMwacNqrurbsFj89l3A->m&tLbY#(erhb zwIrr7LGb=1173{p2G-GK&@6j2pDo`#k>;)Rj#ybDLCT*@96j#?b85u-DXx*09Hu<8 zSn%4BE8N6v3NdKJ%HqX$-d>AmYhBG*e!4%{EDbR~R-{2MQwRqg9xVwd`=#C_U~6i& z;I&lMkhtM!_%X)$a*~CWrYBOlz$MB1t5Im?nJ2vQjM&j?ciKErbB?6TTKUXI@#c^% zxJ`V!%JwT)s5;~rGx1)&_+KTc|h!v6fBiCI8wE9uZs9^q(cy~ z>+D8t>Vwajx2XuF0L-MkfQd!-?7E)xxfS&*Yuayqm}TY|C=x_p$;H6+~XT{U%O_{_vA)}iHe<=rabdku(b8KbNntd~zZno?`*xp>t^$)Pdr zt56aO3gHvC)r$}IX{%`=IUB7o7h*aTH}6Gp1t!sd=Kr{WXBo0WKCGW(Ph7t4FjF%> zac$3$+ySyr)hEZ%4;$PJ1o@~GdV{PHQ&E9EI3b(AS?-MhaRH0KzA-)y>m^04{Pjw* zlSW_8bc8UiuAQ%zjhF1Jaj%6z-lEBC+&}I*VHRFnHi;W??Hn?F0k*LjZXZokscLum z>s=N`(4dKdDoG~taR+18C0`t5wsq1g(B$!={UMCC^|;S8n|7MdF?XMd??G_%A}*@r zygJtMhEApWLy6m~Y6eX;ouBrn3qLQ@z+Z`D?{U%Dzzm*6oR7YSww(%UcvV#mC5{Wl zR#;^~n4(kUU+EK49}n1Xw{EOZ-QMR`mdP(%Lj`XQgSpjFlVO8n1%PS1R(F!ITP)34=CJ3P+lSD{uE#GtFOaei#c>e$-- z9S>$~Cu>zVb8}-NpS51YDUJmtb<38r3{`^Bv@eafBTh**&w`Ls=cLHc`|)lO{VgFt z?X}Kn?8eP#P`!2L=dDmtS9=*uz60yDW_PRC-Z$W!`2_(%#8P8NcXcx3cHjc+CgCS*Ya z8&9sbei!{!A+*0!SbYgiyGU6}V#$al>tpAOui07~0La($x9=C>vDKWDtkFl3ZR0ZR z+2AEEEvvHVqNbkR=iwK5Wh$%!gkw~NHT!vi)_567dDTmPEu!GWc3n0@flpqP4!5D_ zRcKKa>uVU`0#;$~zBKcZnvU??m)fgHM`#$#H`A~lBD{oeGShl18tc6{hH+Tp@N_$6s=1lNo&BYjpR=xR zEx+C+s91U+^vq@w*dJr?;rxw_aFnZm(QDqiC-1)qWE-p-!!!Y`RUO=oCit|l* z0fOFtJmAXL?deglpQmX@@6TAa}VQgvjviU|^JO z#3joZaQRMI(anZ1C_i8&{^pqa zX%$cG8c0S=Om7EoJh7ntcw$e&QY4i;FD7`(WF1Wp7W#9$PapGnfqK8z)3miHs6t9( zHacz0Xy}m1(V|cVIP!k&b80$l+wtQO?~V-LF1v-eMhLi3<)i*8q(^du=GuLPtFg?; z(I8X9CPh^|9KQVg-V8FC&>JOlmAvYAQaa?=jn0SZfAl$SVGMtu4b=?FNsD|vd3bWr zBg_W)(toE8i?cmx(Rj97Bemd*$zsQhD&PKI@TFi$V*`E53ptoBiQ4s&j%TyFd0?JX zz=H^>97HN$>W#*=)4`v#6O~S;r$dG(?Q+TVw<8;dIAg+}hG5*;4_Aw-pF2b}ot9No z7Q_rt&BC)SZbxaRLzdfG%7KAT7wMnlU4P7OdA3PkMclGy`Gf_N%bU-3g(VO5I#|1y!L<_N&RRoP>YxXkUuoTq#gViARG_ZSChL#hN9T(>A>9%cm*YGTXmrWfxK zi|g%B)xaA={pFXmjJo`4pJN_ea}G4W=Ki5>biN)_%NpsiuZuhPp>S|c)3x`!I8gq+ zJoMrP>Oa}1;XOPgyB#iG;6eX_pHSm-?~^bHk^C65{~NJjhH!n38|TzxHz=F3RVpSg zOOaYui0@_hSIh@TbYdty;0{lbn!2rn@Is;)1SSzGoRVf_;%->(>?nk{|Sr2;OprK<=KIf^cz_h1L;ebYa*MAGlOcmoG z*Hz}XCYd=HT7h01?sMAXD*i>w29B!%mx3^nIwB|j0n8#$3&$J?at+@u9!kh{gD>8; z4rbYy+ioPZ2~TJFx1W`STVyr$-D?#;DQkfGd|tqC1qZ2W{+Ddmh=;zY?##jeKr|U6 zKfK7j9GlHv5^NGbCPpP)FtFgsNLq0{fxMDYw3>u9X?EqPdKFu zCu6JdUvx`{#UARxvvMaUU0bX6@qQGcm4PGwQLG$)fiBk-C?_x5QSj6nxlYX@2{&V)zR&JLk*Sst_TYMoV7(L5b_s`36M}CuJ#zvx= zr`6Lb*X%PH>7=f-CH$lv(oxCG8B2 z<^5wetBKCjD%iJOKboS=baA+@f5On0%0qfsN~~zysB&&z94F$loyQEm$J3>gj4HHx z;8hw$-g)SU@i&s9v*On`&~$+IQzopP4WYCw4XV5x*M6(5k;y)HTN@_8-uu73{Z~*p+)19<3n#bUjUo6J5-*n+p7YDwL{(x3IB+2F+|}1$<=LKNKTTkAqn9UrzyCPMz+;W$ytl<0e9|*!*`)saHaJil?f%B2O1A*tRqa8N9X75c_ z$x%KzQ~q#os&W;gyz}07f2Uf$@l7-A!5g8DGGxEvmGwgA8s>EGJ$yyH0TrOAtpxrR z6+)3Vi@0(n-y+_Gxawn%oE~mmk%Wj4VYmnInRBgzdYtwk%Mb~&4v9GX=D}W}-ksE( zOUpv}G?q*F_wWi_dQ6>{HU78S*Xhm_4(+XtAGY~g**Fi#{}Yl&SE+;_chx)3BZ^_a;63j8SOpsnrt$K)@B!+Ial@BfL`K8(WQtAYhT zu(~NO>zu!Q*zOUc^I36hY+HOoRmA&-3slTXF}g~LI({x1grq0o3Lskoq( zj7DeAZ#tDsq0_|6Y{^EzYQt24+ha08*$>k&Rl=bhZ9r@&c<@EBlh$ z1fc7Ed&vhX*+2H~@e2{P{s-Oq@$2?qQp3Oa)!Qq+<6QAQ_6}H$4Zq?t@<(*#Pw@)% z1t@;+N&({z;9Ib)GMT}4lR}Mvy2){CO*d>a8N0E-4qDR3+tESbQ6bi=i1w=A<-k)| zZkaH;iQnsa5T(ToQuZC^z7x?!lS${_bf@Kxoc;Ri0RJm0Ta9y>ecgM#p9HDB+s4*bU!gw z$$+6ruH$0XjTgKSvxZ}`0iXOMMuzq`AN2WOyifnWxCFYh$s%SMO@V!rAVEG$n43R; zOa9^f?Q?j`AP(b}QB&<8^r;!uYN(t-WOeS*sSrZJ=A?N!bBakg%8^Wh(C2z!8WXRG zZEO!0diUdm{r(vCUkHV=xR74m|k)NfcZL>Fj zx3ii-vW5a`pI$3gezZL`-l>_`T^%76Ol=$!bW*CE_=cMOZ~cs-`hMge zX|vTrlQ)gjFPXg@pi@Hb0usyln%!ZhT)1gc=^IH~576wPwH~+GD5tlLq*?J~L%)L$ z?DF|3o0C^f+>*7uLrc20_ScCx9h<0GJ{M; zA3fn8?_me~%TYjE(*lo9_;mhh&FS_w^*two%ObhjP#g}fi>x9$;k(_QU0GU5HBcfE z*j`y$N88Xp?wQ#Xe(lokZU}j!KCbUze>ZyI;*%S;HK#Zr5xe&;$m!*9a8r*HelKVL z+LEHU0E1I<6#jQm&wys0DPrHxpm-$^-HqpUd^$4fr)+ zZn`FbXE8%4A2LH*IXeAbB9Mb5AcSc4!Tp2c1BQp2U}#eU==>Q=kW*k4MoK(2xgqp3kxp4<>E%= zKW&w^r!)t7!E5UVP)USn;eSle^!_l7`*Pw&$ObKf(A|AMbJ_L0tV|ec72KWLJAZE> zTP86#x3~n`-Myyj0G4JjAJs6QD+cxZs?00@Sd0@NYBmZXBcyFz)^WFB_5N_*a*nJX zf{TiF@=WGh`!k9xByx$*q~Yj?0`|UR8{a~0=bls@gx>9#!#0?Z+l%i3Q|!Itb3*0; z=&VIK8Khv#>n3ij)u*}(G2Rl0z&_4-sZF5g(8CYaZQe{lbHn&yhwz8?Ts!v?X@u~Z zm>ZS4A4{tQUdLVtYg|}?aQ1l|qzv^$I73xAdP$q<-f4+u3N_Eykbs2FFeBqyC_-T@ zmo%^s7mR)6>O#fK@b_<*B1bc8BuW$9e*dlZd4tVa(s+|FsNX4+bVym2@6bv59{bnE z2sS^$?*Iv!egg+QYwRSjphHl!nWYpdu$+?H3#%x`r!|A`0B0YzNs=44Q9Rqx8|#on z+Bx=+b?N~_h|ujM9D9_;PbU$=y#@RsyjiRIwIidbv7+#5JC&iX+?vCLbj;JLx!;e? ze4YPgklI>xOtsf`y>{o(bpy1AChF;-Lu14f8a?35{me7?vPI4+t@uI=cNa>tVxzmD ze}bUzl{B+%`3tpTDOw0=#4!HS9cq z&L;Uxnfe$j^~3UJ1JuTVi{K;!E2i)%?Qc4oiBYeiqtIb5>6zhJ+)e^5zNJ{8JLm@$ zWqv;yE+xRq_&nWe%$x>+IO_p$iHb(^eSaT2rbE3F6F&Yt5-n|9?;I}v@x>eVaiV=7 zYZYtT(}D4lqTFY+iUv!42p;d8AxKy}BheT^a=o*Nn$l7OSWr&XyQc7BqPl`&aw{ zUwkiXjXYRs-&#@qqRP?#;a*OB&&f0p^8U_TWw)Q0=BKl?|1v(`kj{U?-LMc!R-Axf=& z!!EsB;0#?_L7r1@wL>Z>jjd})O!D>&ql{Tb%SC9;5;PaFU$4C)PH%nzYa%n@ zE#;y~$#4{I;C(QQ{W4AcIUKO072jt%&vgMd6WgjWWke;TIth%9zC zq@NHmTuxKFu?L|&hmm=5o(tRcB_qT_Fca&r0ndShpMeH#y z!%F`|w`~YJWyD=u-VjgSAmXa%T+QWmt#6l9?>d7grF=-E*kZf>qZGyEGYJ)3l zN8hoUO&5n*P4!TJI%NVKpqtu2r%m6VSx6(m!wPW+wM+kqvO(G7a?Pb~CZ(by@^Zv& z_ZiQH8mi`z>1&D4{F@KLZYTth4__bG-_;M~Fj|<>2&ft>J;>=fj!EA@<;Pme0x;vW z%~FX$c7)$~dA48rQ+@40(2mapq-KqWT1E^l_IXtoX^Cr6o@<9=pW}fC>|UO7-kb2jSjSY) zj0(%0Hd)ZH=~8UME7~T%?W^U9G!rWk+q5|hl{Q1P6TO^~_FJNm*h!vu%J}MVHMDvNtc0fZX*HD>sJL2+*@L|`|0Rl77N44c82J6*d@bxL4WsmG} zjy=&;C@cXZ!N|AkKPV#cq9ZhG#o=39do^p*q39yw(w}npweiG+Udb@EA|PIIyr)B; zeA?_+P47AQuB-<7yu<~Sq`LI+2|k%necICN^I*`(Q{RKjXC5RA9e0DL;vSSC~tLhGLnqjEnBhbfhSHB8P ztF>KNUiy9prh58_q1cjwJb_Y5*+qusm-V z9T76h&R^)k=N6XZWC5JotW^OGKfR;8#hNqLlwQu;*9FfJv2+auE zkTUlUA5bUwI%m?oybF74Y6g}b4l29PbXK!rINH|>(_mL&MK~5;YBIkV4i2Mv?{^s( zlxP+)Ywt)=%H#$PTN_bb?UFMT5BR&BT4r(Q9c0hMrEu;o?%X?onrahj(FnwiW@ z2-+?I0Uhr_ZZWGNzOL=k6dX%Xcwn0V5_Jr~4iQ!Z)c#R+u%Qz^C9j6-C}!;3IBksc zu2Bj8mHrC401wYuS=Hvt&bRg3h!x2#D03_TwrNw@qOWEU>R({VpEuxwT2@%Xt2+Aj zngi*4hwP6FyVOi(&@UOy4n&C%pRFY|Cx}!Jy}>ui>`@)Ggzq~Bt7Ul3OPXs&@6c__ z$ZmYJm_yx=4_x>tp89E8)6{&#t(u8>TQya(97WZGP_#04+6rsdiK!>Yjhq|kNEHLg z*cmLO(_5W2iI-Ho+}aq30;Ltii9n892M2QG`%MXzWilF5;tt3V+7V3(C$KD&yfbXR zzv81e;9>Z8PiqYfs#ME+Rt-xXXK5jWQ*^bi!Oe32>gVHhwe3efdh48m8kXG}R|WH_ ztw*60i(?(*g5TqMsY5n0<{0o;EeP!bq1$Npz=2!!#2cw+$ZBf-9uF&1-YS+>R9^i&oYwhWh9rbLxAh8B^EtvVyZZw z1GlxKpw+~#(1AQQ_Re&WZ#QZgN9QxqVkd68oi(g3q_V?=BX0NvvC42KynAqXRlC^f zlX)~(WczeG1M^)RAoe@O>yhZ^V>t^{LTAFv^~Nv5f?@~8w$|+J(>($rBsw@?8UqXhb>=+ zEgV!H>RFZ}a(FA@lGpaq=~TP>T~yR`I)&BGm--vreO5cZ_oBt=e}?6reI))3Z{UE^ zi?{>0tmnA++&yK9GvRGIn{w0I0=AVT*iSZ2NMd#=OIJ{r;_i(HsCK}W8yCOw(pfbC z&8%F2h_u&WnFjS|9r=}Cw7y~M>*O_T^m-Rz>nqGD=W}}$16Z*V>z1=3o5>7YTI()Z zWY1B=ZodZs^918$JTCR8_E+ky4lC)y0$4 znX=xGMjg-H<$t3-c^!^=5E19F`|dC@bIAjQ$MxXHs1-eOoYE&wLcxutzhS{5Ib6y} zE`pPgp?KIpuq;XUyTC#KhbxaZQ1r7wG3a+D7d=!%4x9827WvNy*M*QTvPJ`Tvz%X?;f{Kf<~CuPLXx}Ru#eVAr8z8ai?*C}oh>c!d8RiE zTGu0TSd+(zV{cz!_>CCK`@Q(B-ULzL6C8O~rf-aveCcEB_?u6U{&Jex7(g^qR4Pe@!M-l#hb7<3ye=Csq1}*#a`7e zX9>Nkmw@OdePKKZoc-_#E4ohIp@-|Our+B7AQOGJv_a{XpCOHQbNp!;UYJFsLQ7Vi zIoyG&Dssh-9BXkL@_A@Wa?-uxZYRrCu-WpCQj20qPLZ0+)q5g~tD&@BY;U}$b;TT* zt|0_?rJ8L$aSz{0)IPHhj+U@>1W?6 z$4P6W&?3{D={{vF`Z!69XGZa>yWsqMuCF1M>^D|-5Yc@6)uo{rx+}ii&Twq}lQCI( zCRNeMFY{WkG0mwj6P=6^@kERL*`!0hvv8YLMkk&WG2U{0)%fM$6qffO zrtCDqPh8sH%q4#jaGp_PQLX?BEZH3brOs;u9pix+(K38$c$4{x;F~3Vga{Yjsl}I{ ze$+F%rNRZXKjW2n-DX2Iv-kbWy2~}OQ9a7DX1bL-0Kh(EijF(D`I)Cu!0dI8u;Kz{ z)OhX)Cr&&#{_a7sa%mKm5bao)1cRuZ-0mS5+&1+w6aGd+B+u)^SBNu^ZPiIvKLL9M zcD@~W_U)7s`=iT~3s6rL2a(}?+NX822E zilEetL#H4@yohjrD*g#8l}l^LcoDOeaj}6qAtr=})PN9D{x${Uv9;q)4*3Y%>bS?# z&sH!t8%W0Go2`JW+tBgM@)LI~imC9|b0c*>ci&vsk1F8>uvF>D?51|}1w@AyNHMS!GG#^K) zu*P*l`&XgqP@sQCbbAkW{zF)il^Wl;ESG&OWW5qF%<`-ZH|X_R_mh`IOf^b@*34C9 zP_?8fw?9Z&iXBp!*G4zqf-Le6Stpm1BJ>@XhaN9>PkxSiFJy_Wyt6)^8_PNWEq&N8 zo|`wF^d@Gd>W4EnbA-3Rrr#D8@po*s^kPg&1-K)7qcx_tl8?31t&+ldu6SzaA2b{E zU={s)y2c-GZGZoAaSyv|p+`VpJnn8u?(UR#$<=k5wHz6o42|U;$`WXZC&75zpw0eI z2`QKdw8J-VXELXm4Q#R64aBGkYULibPgWG460N<-T7%|yg-QtkRphS;ZpKf1#7ZDe zRhfKNz!zgC57HC6rgIjwLDx7>wU-IteccVq;+}2+NUW}CgVob$Wx`Cqr3(%wK8*~I zcealQSDUMobvJOf`yMuTg`mp@m)!@#BuAT}2}WKXI^{2Cn&Qq#O;abmbrLSARD2zb z6kMUH5C*OcYdSdgaDL02;Ht4~YX?3%{213H^E(>g#L$zJ$9e;ra96ke1|T54XJ<;j zBGo_5 z;CJ8cv^bz=X8pA#$5yI0ESJ;UL(BlLUikDx5T!_^&iW%K7tyr;vDNL^8@CqUO!xa> zbH9tX3Fn6Ws=9tIz*Pb32>?m#?oO#W&;$5=!Fgqxsbt5U(Yhyqdk3Ps0$~<>6VYx0 z%O|>2>k!}?X$o6JTERPE^XNcjB0>)CzABYJ&-I-%CZ@GMfA#KVjC9^ZwgDhuwnDvK zOaJ-a;U4Mb_Bop!1nuA}cYjx8yu+J!h<$*0uEU(l4!O3>rPgl!F(z0!b55fDN;&^% zO+DYuAMK}--@v`uUh}KE$XY)V?!qZ2-(&KU=v;uw&uwf{1G%{0NxtqrthH(yu7x(s zVcz8{GVpP4{XP->E8TuF@q)v+=OG)##8h@e7yX`^e!`qaMUlySzr8Z%b#4~Od|0{Y zK@pN|CIHnHK%llrA6d?6kJozAymIR{giocG%_i{cbE&`t6avpCgTz+K1__)eNKnZM zT3?#2gU)DFd11pyYykes+f9kHKR%fNuiCi;%q<|}%qz+9x3yzv!y@&U*G3B7Vb!4` z71VdxqfV*>V5}h~L-#wJ8Sf{V3pZEGA578b0iH}F3vQnaZ>5T}PFNOMu)kE;Enflz z1kaxLzZH5>?*h44$n`z+Z+p-zUK`xauqKBvC!{RJF<*k>e_IuKZ76dVPA&I9e83KN z-nX6gfG8m3Qs0=!*so@sJ~*I=U(@Q{Je^gi-=sy`=!m<;>2(K}E-B`a-w%QO`do8x z|I7@a!`@>w7E>P-2;i@GRUu7vJ--i`8AgGp(HS(TL%)j!%z7FmL@W8?#<{!^j+W=* zTy6p+0_OnEDWJc*Y5kYFn@Zq8ZDL)6df;UqkOZMT_t#WdTWueo^ad-*{5-_qAs+>0 zOTPa)B`WS6WMf@pe%gcGcx=(v;8URh(>=CVxRFSagWtLskYkyzo4!im$mB7t=%=hg z(@(`3vZbwB=KtQ6To#SWeLSDc5qvVbqpN2;Z&g8CZ`(7=SGP~PLuzT-b?-~5H7*Z$ zAVY<_jT@)dZN(w?=b~y@43f)S}(nny=UhAjfruv6+uK6gTor7fJ z#L=jkd}7A&2Ik=+b654boym5I-3yyTcy1JV;#~b0Ima+jm9L}a>8|@2706?e;&DK4 zeU%adHb#l~Uv?^=Y2afaIVwK|s1FIqbw?V>MF$?V(O-8GA&C;d)}$w@B|%)^^Uv=s z-&S_vT#IOnv7-(WXh>hV-xsQKTJwE;g0z~3zL-Z6&2`(ULsWW$Qs)CXs(gd=?|kq4 z3mNGbA5N_t9~trP0PL9M5@7V@c0|n$^Z(})fAf01tH+3sg=kd7y-sui+zkB)9ho#d zvEUrB;U@?+l*g9MM_i7B$hB4RKVNQnJVpBILiN8&^>`al8lHb*(f=;$KlywP{bz(B zA&rG)DGePX?mL1?e*9lAZy+3(w)|D>0=4W`W{>H{0EDtw=&>WsjJ0^Nr9hn!hDbXO znaY3Xss}AS4z|B3%y2@wSAqMp%|v}oXCeXKW6~I}l|4nOl{3-gX)F}1s{ZdUQU_?a zYUU=O9;})&niwnoO;jbEn4+m?B5&;vypQU%43w*ZyUjMg1<^NBC z{-FGyPdoPcDeZL>V1z#c2mZTqh-W0Ef2;NQ_)BVw@xzU+el7c9`>&HacMs?s=Lt}o zrP^{F|3*fuE8$*7npZXo)7aM~j!gt5CIt zJE{jPXeI`lCpIsoO3UvH=PIM!o-vgq3%qWBNJ+322KH_r(B2nxuIkn9W7-go?X>cA z!ZZDpGV@66cAd#l=@qbq90D!wYP}NmXSFr8v9I71nxUfp%zD|H47+XPhwZ2_#M-H~C-ZJ0LEv>; zx#_+{r7|Eu8`}|@JEu}sdh72`3cuY@OwOP&#%>Vv#k(cHT>W~lmdxYM}{KvLsL zX6HqBT}7vp_uSPj7l3Op?rV^l-UAKVsd15n#eNhHQ6io*yVb;{FkcIXoGH!W;KKYoGQ4=9WX! zJSq5eMigeSelE3xZ7WnbeW3sILTs-sUaQzRxWAOuikq#1OvOpH1O_*Si$?o1%3%lM zBo5QT-33{S$nc0b{lXH6U!G@y5eMnH1Ksbw%KPRFrRK#Ue{3q#4`*jxS#_mBYxhUK zSHj0U%k}+T-ZFvaHUU#(CH((b^23Cbg^nK@ArZZGF54i|l30drjMnqogQ4FSCGLNpsN_x4QD1sJr(`Hj9^=6J`gI8F?& z4_6FcD@;8r40n?KG}GxK?b_4hZP25O#g=HF|&1$-R! zX>^R274_+UEzbSh#POfMPa70@mECy6lGU$(;Ts2M;8Wl{xAC(r;XfYb8|upe<${We zk!-sS%_08H31Guo@)oQ_-w886@*xUt3!nhHu|Z9<0M$%mI+%H=J0wl^0P0v6fLCH= z)<0Lw+!A}e&pHW#b6IfQ*dKYNIMcOdSB5N_8Z}!t9>i)#I~bIj@Ak(V?O-r{23()bUSC~U zVj26%!vNc#@4>L7%Y9}V(xH9{VUr0KI)^39a2*;BOsr$+}wy5dw> z;eOT;*11pz^sc<1Oo7Qxu6;x6!eyf4@>~C0dtgjZ5_S+FQ7smzjn&Ng<0d6(4&^O& zUh_}b%$wABpAYT~ZBJrQ`-(u#<23agFZH=ztqkp(KpX&`Z+yr~H5eN8W7Yz-fk<9NX z;NJ1XiIc1K80}R)vH&pLW9~lPJ4=)03cXv*WK7P?_tp)f^82D!y z*@=*>D0=b|o-~3-e;;O;#i$%#5^2nR7T1h~-tZ~qBJjpD-{1t4N7T=G-zq{{+bmZk zzTW#Wvp?1zQoh+ej`fzwdX(ArN0w{PMJG=Yu#)iJ``>DPYc8BV>gqQ)+L47=N7fgFQ2raT;MRL9Z zKEKGmpb$(zxX4hq8alm!^FAi|o)g7;s}k%a3P1;SNJe~DefjkCj(x<&>e31aPOC^- zYkx^}a_VA+Iy!i8|1!6y`b$Pp7xdc7ll36= z$HN*A;W68oxKX8kct^_|GTB(vP7e8$5Cgv-U>*kI%VKF%re3Fth}y~=>5u=j;;nm)F>lnmI-FlKoo*9)yGx78e)6URgbhEw}H9ZcAgL# zaz!%2iwY46R=U@4Pt%q{hszcs7W9f})`J$v}*sH&lJ?Dg+z zUW;hGiEZEYbH*Ysc0YQbl>*4p$g zSN?KVVa*=#ZH_v_K4KCS5xmnGArA(vkli%>w>_N&Xi&11_&%w2%nL|ynfI-y5%>a} z;tLfsCz&1t?~!A|MxT|5(2yF!t;lBLV#ql{!n_KGnS{O;AG|5^`+h)|aQ^X}-&Il2 zF~zg)e+Jz2gtSE9(ZxK2_h_I5mT;{*_|m!^;gW1RCi}j!p@>3+Wz?5v4S-UVMoEi>B! zft!C6f{wdee0ak&lsPIc(*E8l`|B5BJQ0hvEL4a}=;QCBEhSEf`G?Ht9ios5B1sPi zg-o)5xn^!iC1NOwu7K48R}>6jSNqduX*4)yKE<)#A*0MAZx zduZL=L#(EdtZxf<;GGl6e!meM=5&zY{{W8{y7ZZDg9vVXz67ki#xs=r-+E^ov8J$gq=qouAaRI@p#=tKrQ)6u0)??A%-bN#3E0f4Gs(I1z|EPTH^T1L^R{P3^Uc-v zCZ}uh{<@YM>#Z1^cXw)^|4#8W=W`bE+*iG}`N(ze)3xzc$7N)@PvzNfe53l3cZXZk z+b`FTH`o3?X;*C$aX;w6k97xCI-geCMos>;eEWIPQ!!_6e3OZk)Zs0-)ADq?>eG93 zc3NGza(ltOR|2d%4c*mq_RHM-;~f}#>ALmIOevV6w(E%2Q zTMzC`gCq(d9byoZ0oum~F_DFLKsxLoVPqr!6^l=8FH1am1O&cp_fvNQp0fZ15*ONH zLyRE92!H2^Wr8+b0qgFCiLcGT!wABY*IWXhLJt-n8pOZsf9$iXzDHzKIv)hN*wfX| JWt~$(6990=t)2h? diff --git a/publications/whitepaper/figures/cluster.pdf b/publications/whitepaper/figures/cluster.pdf new file mode 100644 index 0000000000000000000000000000000000000000..649be9011079ae07a6e99e05ff86cb6d16a10b0e GIT binary patch literal 27474 zcma&M1#BEmyRB=88DmV$%*<@Z%*;$<_L!LxGc#k%%*@Qp%oH;-9q0T0|L(1Oq^qlG zbx(Dzs#@=qM(XvbN#%t_X&LEQU`Yvfb|+_HSzsApZ4517d3a#ymECLsu=GL(js{jX zrm*y~2G#(Fe>x-$oDGzW?9FW*VVOVmim-fqu=Mf;VRjHukU- z?;jvyAU2;LQxGc<0OjB5{v&7mU-EzL|F!kkoIL>eclp1w)5`#i%?$)?T>tUM0L#hB z3d_R5@ac6DRh8k|5vF9aIkT*H~MtR@$ZcvK%mWMHzUKpO)+~LC)@v)sBCWl zbg=!ql9AizRDX*GIGY;*6vYJpTKpfEe<%NMQKIHnjsSaDdQmF_M}RQE$j127t~3B> z>SzYb`OhMs;~X46OSgt~O+VAtswAoZibrYlhPKadE0M#}5r1|qgy%bS9%=>V@k_*) zN7xS}o09}C=KA3#VIA07_%g;h%Ue~u^RtihFbz+wxkURe;q1Vv&|`kM(`Td;nfi5Y z)3K8KHOKMx6fK`T@?${gdKYrjj_vvOPUx){S;Nw!JMumH!+Wv)*!ySn;yRQM=j&ms zPx1L`R!#i*pp9C@i_U}01N=wg*ZS~eviTPti|`dJ5KU~@c0YfkJ^8P&g!iGid`gkl z7rahfu9EmW=CsaKT^+ND%g-BZNSuJ=LZOH$0vkQTP+cWO1ON1S|L?WtqAt3{OC|C@ zy-LkMN3D*W z4wGIrTTRU#hm5v7%U1RKOBqMb4iLw>L5&;g;gf~*%{d{@Tl>2o-G2BkEiZTtr{zwO zekbo6UQcrKUc4NSPrFc1o#=?(F_ORb6*rGeA1qZ^uOJ!~C}zI<|riE&34f)4{5shX5cc~WLv z*l2Epw5rOyVM;X!f+R*q_YDB(^ieQ^jX`Ghu=L!R6QUb$cB2Mf+f)CZ-Vn=#SIs7J z*wpWaIElqWz=%LPDpdi&ww@P`@;%jZ8XO#ui11^bBDl%SfASR4q&{ItRb_@#n>7Z5 zNcmRtGh%*3zYsy=PW`%jO-Tw-=lVs^aqPgmM01k0tr@Glax%=unUiMONC?-*_HFg- znVE`5 zvg&1yIdg7R)y7oI)(b~#9jMYH%1KxsX$r&{&BzycLgg%rAVf5s=yo}EK-uu z6-7TsEP};y8VCe58{9JXClnT^`HIu?)~rnlvJ6XBQUo}?z%fB+3g2%JrGv0 zj;`P1PXm1TZkOa-qGmRpD@zwuc`Z3fVXxEBDD59=a(f9z^p{%*!B& z!`A|$XZ+kdscUfQ~E;tJ*i_hYtIyr4= zGmY_2(FQ`GafS(R9uU>P9G7NnPM`ZB3=iG^8vXlTT(T#mD%ubbF-Tv0jUh~Bb7|q5?M-#tCcmmSHPVGaDoJ3YW0UYxldVB0v`WA0 zRY_lq239maj178qW2wNN7z7+^0)}NQ7qDv<)Xq)Bu1-4yq3QawD1j)g>V2=(2@7JG zeU^49yIvBs^hy*Ioyy%xQVI#!M#i4mJCgH56H*GTW=TwSx{?o4VNATFaj%fS&;fZF z$F0VLP>Te+K}1>LE~E&zQ#+HYQ6hZiY3|4bkyQuih#tA2yt8SJ9Iq^a-bo z;H7RgpX{3jn%i8M<2$Q&g0EC9)Lwa`yw?4M_O;=$seH~=G5j~)9n5;n5soRRr?$1O zAGC^8_?W6@$s!IegWs<0rt%Go{HTH)qu)V0e<2d6d#IQ)es06+SMEC1@6*%ixj#eU zEvzIJknq}2kTRVR!rW~%p6Tv1^v$3p4uN0oFZx^tX;mI)kLv_X3SqEqBw`8GlLiKvp*o zw)bbTuR5i!1E>D3zKe=oml3r(2BP#6AYYr{Cq^@py4E*0RrH|fp8qQ(UKRYHNU@*2 z#u`Pth5mgTUlnEmM|>I2+A|z9@H#!*6b%cG)0vRrq$q&+_Zg#vCOpQ;Sh%6hLl`qv zK`t&SY0J1)zk;cIe`(v9k-E2ZVI2Xy43zI`FbjhCw8e)$#i0l8mJfbjckH$XVgLh0 zAbYlpLM(&u${0efvwi2zyDF0=WLAL-GV>eraxuCZJj7VvnhCHD6-)at5`lE~2gH?< zka_Ou8x3o#p~tK?S4I7aVF4Gd+(~v$nf5g#K(mlbComOWPPaj`yv%iXZ%ScA7u`Ln zI31BtM+5Q@M_Ftz=9`PawC-2sr&)jyt$1%^9gZs`F~Cg6h`0u%!w$Db8$= zr0ZNHTvc4h8V9KI%pRJ0s_^Q}G(?a>y@8=-abKbtY1v==F@(`{sv{Vok=Ml}wSU;* zyM~oUE7qYg(dsI2xrvm8*BB`iVW$t)T3=(YdKa$u;sL53>a15~Y!~h6N z0b6Eu?>bcKF^Bn=1B*7iE%iiu)Fhayh6E;zk4A??d1z?_@_rlsx!V&W(O228ju(~` ze>lroV?Lx||E+0RjRK=nsC>i&M&l4WFQr!inpQaM7T>l}I6TbV)3E4fZybb1i{xh` zwbP(03%un8b9bIEhpDVD{h$a`AS@qKSyA1dq8qF-twJQ1Ql$JA$|lT6eW=9h?@z%k zPv=tQyd(w1ihe#(90Z7rw+bS|PG{=Bf8NmGHh_TiibGK(AFq~<3|~Wy29AhhNRlYC zsRs)aFd=>`qp;&g$*SPBpah$ZRhmSh8{D~)NAr@H0eOuhT>)vy-nUht0DVGe#mVf# zGcH_*KD;Niw*QtwBK9+Mijibg0F$N-PcAk|q^dlyM~M@k;ufM>u9dwC#4_ms!rk|q zbhm!;9)hG>e6mBOyV}S3yltuSemi;SZz%-(Z(mUs5~8=I(2HzdH!Xd>$|77s)qJ&J zRf0BHe>dz$?o(|NagBIIEDtOw$ zZbS=H>YjWWDupNk_FKT(?JPoSoT)E!1e&UCJhrb_UyzjYue@2euPcQACdt;)K)a$2 zjgvG}7S>B74p=@}GW>i#OGr7)eggJSL|~X-WUL(~$jIf5g^7)}w#rq#W)Xn4wRnR4 zUdKF|c$&3Vn--WVBXqDk)Kva?`r9l8^3Z}|1g0T^fHsr>?X@_#{?rKv1}(}#NSMoZ z2(2@6lgAwApSNTA!8t`pQ-Rbo}f^@a%fe{0~m&*FsF8YH~bVg_3p4Pc2nQ;Jd zaBJ5E`*ba@glGji>{#@x)qGRJ#o|V$GJX1kcMMV4Z|cy?^Q#5Ja9fyDL&wI$6DKYx zCpf#MH$@t?(O^&*xIndJw z^i0Rgb#ywGt&Ow(@sQE=aA54z51{3_>7KQTE7VKRiH+gxL@?P;v_0o&m(Y@d;8Gq^ z_$K=@NU!?G%0!rAV|rQkwmxsXzj)f?w}{@)Z@e8-VMF$Xl$x>ZyxyRRj^{Se4~~A`P4r2C{b?d?=NQ5Vet$hBVW9}i#i+z82h(M;exucrteJt76RwwWQ70S zeEr^1U(*rnDSko7RFs^T2A0QzEC?sDmJ3sDzsBHAk1}kRNKgYq1DC!RjD0078>uiG zSYiy8Uw$gc&FL7^1(m0B&HE=PPPX_L@H{u@C5 z232M@cFzAs?0=%~-%U)^U>W~6X>k0E%)q~$pF`=@f)?6s?TqUfw8nNIJ5}M(W!z$x13u}9* zimbHV0_yy;tht>DVp%MYBq8E^zOI5p$SD4r5*S6>iyKmWsqDXB1_vA`tSwL%^-sd#gySsY)VDqggP|V7%4nFWswDb&yw>QkB2ka!bYnM#~GNTDpCU&;n^C088!_+VPBe> z;F_b~IsOV5UvcNYqT@n{h)fZdK|1O(MUFQ0j0Hz>Q#BzH`#|AB#Jy)JJ!~N7L^5Bz zcuzj?ZlW@EY84)6+yw&nXA!70!KV|;b@vM1#r0?-brtCJ4#Hz zXULJ0bnK5*m|bMNo+Lrka0NPtYfYpxy!$@ZHbF%B0<7I9dcgEF2#{{b8MF8;H+D#s z+&%g6^p&B158g`AXR)cIp7%aeCmMP$v=vJoyg0WWgWlu!KTkiwnU2xBdOd>vBnN;D z`cS?Qh6htb38+Vc^AX>{>?lcUnH2Knzr!p%6KuKJK#@x#C-S0Hno&)}YO|!*e>!wzB0n>(hA?y+?p49(d7X9v9 zuKE-1dW+{9L{jhg>FXRi!}pD(${<}^K5fVw4KEABALm*P$WQw_MIUk7 z`qM{3<__jypoL)IyDydzDr?A!^WEH}&?A$5bLU{FE+_Uc^)7jE3>oh|yqkBFB@^=O zosL;AgX$?D#D{G@% zefR=4d->zNY5A!>mYL8ypNW#8*n8O?$#1GRtXNv3#|@>Hshp`>)7pCwtL2{DJZZ8< zJL63x;PN(y-!5&!+r&F`HHz2ryjPg2_fr!2>tC5wQZ-?6WAGD=E-rq)T#nv%|8Pb9 z*a$*O>A@L`@KHz{9jLavyx?+X_V{US5LL7Om?(5AbpQP&)N5SUp^HDk^in%B$cM2b z`os4lb#oMHXOr-q{XGLM{yNWpPN*c_B5$->tu)W+u~F<I1u88pZPW5s;ymq%mUF|YtQW`;N6=TC0Gy(}4Y~eKPL9o!nB9h6 zy=2rux}MPG=2K6c6iO>Zt**o^@r^FFsvzM_pV7=71$RHM5HIH|>FGndHmHk1v7Jp3 zPe{G4#jDjv;5!#TsRl{VUd&9sp?S12^>{Rfl$yjzKI9SLNaslC2>Zz6RchFoQ*0t` z*Ip2hL@uT>`XY`$hF^hwZUM=&zdN4%)um$@6&Vl0mT02$mbyj<1oC#P4)IOTfJKlaancZKM<(I`$! zkJj~V%zH=?+4y2J>WpDL_(VKIvFE!vd`N{#sMdn%O--x6*n`K1+~vK}g7M_~Wi`~^ zfQA~dDM=aq2)&o1J0oVxJGSo=u-g==)|+9~;t8_`2C$;^1Vh>_`IDzzfKhsM3c;tP z+dI1i+`7(vbV8@ek_g2I>etUe-buZaX@f!mxZB8i1KatBe5etUTxO*$wqukl799L2>tT>lE120tUS^0_+2t#m}2&R=XfK^ev8MPBhf$FVM46qTM&vN z`1^W*y`U&!YJ{aHm}Iq@K_gcpdk<-7TJ{`hw4OyX2w6T}V5?G67od3cH}i;}N^$;U zAFVCCH-y$wXriS)pYiU9TU17+@{OOQQwg(P@8+J-<8$pHrdaFGfGvmD7h01El}vxu zYitI{5=Bd;pqSP$`JN2~%wBnpP`xJ?&tCmvm=R6UOWa}$b3X5j7Y437ajKwY1Jz^2 zZ!mJ*LKqMQnkXED2&a>e?TcOT-ajm=S#kkvk-e3+Xt5lcYQJBb8JB(5mTqEI4l|XO z8|6eUba_WcC0kuiR`MMra+;JB4B80z6O*vVhbKnHg8hZM1zPyW59fJ+NwWrx8nh)u zQt3a2yVRX@>Iiig2?mY)pj-+(t*9`ht(D{#k^6IwYfo5&PlW6LEVIU`JD9YlO0=8Q z)RWETB*-r}lW;L$Si1rQXjX>EA~J(-$H5I*rvrC=@#y|I`8pG3q=g~mfD6?^3S+D5 zIO+@CCDbm!mtp^mTHUUCh`P%%xX=dUK1C$K6=(fobmLOB)hvz7!%;Pfh^UI`TtbL~ z%}_;%gDt>o77Yt2o3X)QaohI`oXOt7k}HX&>V4J_lU<-p&ySq2M8M}N6|&yPPo#~E zE3E?(7^C?aHM^)WOx1i=cTgGo|P zP%qzSGki&asyGeRx~l)V-=sW6DM$%bV-^HcJE4^sMdDlv_t#L@pVVqpqh{U;RS(==;Pi2G zf-c1(z6fGvQGn!1g1U6uD-I}5f)yGTB0}?KCtDqL80r}{L8``ft!~wU(f=g~NpyQj zw)|)oJ&2z@iR`gpfJ8eyCGdF~_0AKGZej^$xFRGKQdi6+b&UIH_m6D5TVcJM4m_tD zdvtT^hJHlj`(!S}4CKV|Y!}$a&OQ9wdlh4~K`yyWIHIT&dmNADV(_N|$HKz<9N zt_As6Otv_dDH=8t--*5}gdOs6kBQ+eH4OZ*xW&ISF+n;{@n@l|I8vl^ajraNCAp-N z17qIM^8TvuY1Z-psf_u1A!GeVL3rSCJu8Q5tvbFo8qV`saP@qy3(4{^V;tG}?39{1 zuk()VgzHC(H!Jt`)kdF}!p}U4ePzjj_|Gx}8DEDo)~Jq6Po==-!dh!6ke|#W;DuA> zX^a@kV@?368UxkR3!||);qNiwNBqu)TIXM-Y6_U`iA{UDR6%H+;RGt!7qxz321-7> z6wTdwDgK&8PDqny>~SlLncTjqGLc~7$(l@hv}H?ozyQZnI4}yR;`um`H}rR|Q+@f? zg2=zoj!mOqZ_aB!JGq;ozMIK=i*_678gPy2Ig;!}kiW5dRJj#V{1B_))1QH|JM%W< z93rI#c4K6Hn)$`~NNFppM)aP7Phi8zi6UFYf;`G{FuSCt#6Dq$RKcVCUitx8#ug)P z3JB+IsQ?(NKdn;od(FFB-F$yG)E)9QY%s(hDvleLMGWGSO)NFqHDWdjvtnPGTaKZQ ze`*kkJV`jgwJ_)+3xEx)1=na8m@90}nDE~HvTgfEn5lPPa3t|yl#cFv4u+|=#;(Dk zrnX%lfwe$W7>=X^g*N(;?8x3QOkUf`0t>&8t$>%M3!8)e2$tV!}YFJ z+Z9(!pRc^jQxlYzHX>6M*~MgLHr?4hu=Z{gRc)Y$(kwQb7rtxaAvva(N-DBTM@#cJfr6PcK7>J1-z)i=od+k zj8P|d-Y6S4^XV}u)mIC_w~NJwwTTIh4Ua!n$9fx!*KBDn3T+>Gd`1`QtJh66ohF;% zVR&@Zo+ac(2ttFF=evhS$Kxa$g8_TaL$)2VP5FNFYoK;1F zHz0A$Gc_5Cq77EFJDJ(YQA`6woIef&49(G5<)nnOhzO{voa?^oNj1$#mJF_UTxZ| zO5xzce*y1s$pJu>Hr-{Pg9mN89R&|+ z>8jornr8Uejdq*jnVY*gS{&{2wj#ZrZbyA+q(^MNgN(3+GIQ#?FzKk&l&gk8kfcCv z0!N*H&I_(BsN5uI4M+;*-WP?f64Nb{ag33P&V8Qm@u8v8&dBrpYcB#I<*<6W*3i)E zd*$%nei{GSK!76UUF-t4e}8z82)RI#VyAe)w<&fn>%Zf;H)FO}!}B6^I0wh{mL{NW z*G{alzK^&Uat5k#xTxEiNR!p3N}G_9)rOFw)~m~Jz3X_QqUe$03LaPp9>Z$4<<>tM|m zJ_C!xoU}D6*Yzb9QOiE2vS@`(G^P-+J4I$}1A3=sxj!G7s4Q>3GGsZHq+VFyMe>ri ztA!I6cG1Vl4%m&MRfz(~*ssN6n=$A@~^4opgl+GPn4rVu9y z3$}m$)39SF$;N_S#ukKK#s+jLx`jZDa6jwPy~x)@L}-1v0`9JMuy&q#yse5$v=hz_Pz z;%rwhk7Tx0#^TQ@+>d?`zT&lyXQz;k!oW_aPhDaBZ0nv%&?l6+x7!Vcg`eg4%k^u~ zfq(EISgsA{@?BoEu}VV1^WfaE9`XCHCKHk*&$7|&#uuGl(6mpg~miyarK6WWtY0b0F*k zHEbi7P6aR&U83{`Y!ZTIn4?kC+8L^Xw^1wCt>hD{HreoRg3&E_6{6I$)tnFioHc1g zm0CIL%e*(6X&&pwdAagiANy7BN=$}FmK=Tj^@o+O@*K0CktQ+zhp=+7(lNS(`JR@u zoB6MR`nY;>OKPFtx^y1l}{7P`S_KjAT3oN#>?vRJjOT6I{8<(atBwTSqN9z7IgW{Kpk+-9 zSZAxbjZ*-tMoC;gdSYAErQUsD49&a*Rm72Q1hJ}t-OE?=c>ZQ~gkhSDF+rw`Asd%G z?3*loYmCAPFhF=8&&|e5Ti*l!hZmgeqrKaAS?9MS_qq(vV`?!wf1pu8pz)2Ueg;0<@q*O%vCT6%ly2Xqn#5UExm&7xo z7|G>{Xg@h~DM3yBJl|8*pxGo3kGTKEXFUOZ@xsV!T>Qw}9vTJWJUc}w*9a$$`4B)? zM{B3IuVJ8f);xPxR5*Q8{DK-4=+1I{_t>r*wYXdN#N-Mj(O2`*9p6_LWYZ$}24f0U zJQ$cX9WPy|0S`&Y9%W?+P1xI1zD%YBNRv)jx1cc@Tz@F(6AP>^g2OW)O@#AMK=*dD zX7Ta%_Ueew!bH(V`}cU(U>=D!IF=i!Qs1QIH<7?*Gk7!`x>OLC`i|j5G##E2S~27f z|74rcDkW$h5nA^omXnXsuQ3CZai`+Z#ILMGg28SO_(RD)QYPNi6Q(#GWJ}IWUB|;u z1T{@nV6nn+`DZ?_KaUOvHzgS}EI_5@m%cq~F zT83|T=tsLA|3>`bI^SMtioO@slmh@g@$OyOK$N;cSJbF1g&fzEz_PMGMty?2U1+^B zefW{hVD%gk#*QTn8lf6p6fKI1jIkKp)G{U?F-w9&7GsX`J@0nnYYC088eN67cA>=V zp2&=$1|7vIp^EW+QlR?8BvvvrL1&9?xUx}7I&1cy0T!3=8WSHMC-Uyt=I`?lSL=mIIqtIN8rfgT<__WN8qmD0kwdS_cgv zM?lpMOPPRUTi+o*zn%7gss8?g78}#Uqf*1YT2l)i3r$8MpE*wyo|hOyyL{L5o)mMc z9ev@<8L#d5#9W4y5kpe_TuDS@&jXSnaw((^i4|6(P!>6EdXhYX zr6Dx4o}w#JIe>>$e>5ZFzuLjH3pDg1Ko62FQ!+ZfSqHM?FNU;a2u+@%QI0z`#2l$6 zMIZ5DJjDDGOz=8jx}Bco5%i46pT|1qlX20t48kW(srdc_U1OV8?fSVe{-or`y@IjCGhf|Hf(xKpiyca)EP6`b4;-JSp za&4t|*Ry@C!SK|z#{HS;TKeYmQa%31bxAoWEV zfL-oSqghv%lwo&qh%VAZ)9b^mX=mG-*DUQQJ&uMdTVv5hCUqr8li8JNT@6>Z>C#WG zuHl_MfLFq8$Q8Pu38q?B8rjJ1bt009UzwkE?D}z)onP^qtZi#D@oI>-OT$PuEN|O^ zsrZ?SF$wCbF`3 zFcM;MY72gMrwOaX>=_#Nv)otoVhHc=DxBAEaI7oW~ULij$f9;2jDyCqnu&&m_= zlL173`Ws24tmw4tme48B_o(PZ7b)A-M6-4X`|B%Kb~r1R`<>PonpB-vkekS_i>TX& z;6*TE^Cb{fnFtxUn zu65dPUdT2lysWRT>$Q+JPAT=l1IZ~F*vr{b0Kj9xih^QSzq$miJJ(CB!u|t{%)r$Q zN(k(yg*<9yqPZR1_c3X>I%ix+j1?(nQnA9~mAMA*wMN74IHgR-*;xT}*jfViti(z} zi;sdA%qOODoiY6W1z!%zK%@|tSj1zUjz4OA&5yS;ucF=tGdjusO&C!Ft^JLI39wA) z;#9w(z1K-(&(YlrJOVDoYK->RsZyyP2Vhmd<{>3ZMIma6@YE8-FRBbBR|pZWX}XLKk$6D?(v{b>jzxgB9O^ zHlLGvy`!KfaWkgmNrJmV6PJ5{@~v*l(_F(DW`fnZkESZTNUMD-{t*WGko>YhT4uF% z3{V5G`qObc?jD!tvS=|}6pwB7PipfuJ8qn6AobN&;H$M;_XqD2zXH`rI=j3z4jn8o z{4v&8XgHnBl1RyAD>(|;JsK&LsIfMjY}W&bC!ZjOixrCH@^_n3iB9of2$H#C#!v}C z8Kc?k?t)9nDs=7U{ltdtADEDCIMh)@yFH@??g3mh+0x71YeyK=GW8;3zxpK{^ry`b zX$-q`-Of05X84t~B>!yC5_1k67;~n)%|1g7=0(6P!M8cHW?Ip<9##jJyjH zFA99(mJziTSi&!tOPR&{1T0mzar;h$=r*HG>mx^p2}iLRGh_h&%}Bxp%sPdn8dpk@ zc0oynqRo|arvK;dA$A)7>H1d+JVbo z8qtp^ZToyPvE`DnOL8S^F?ahmn-))pJ(EDs*NpVDZ9tr=&AMlA`X{>ILWR=6sO2Ac zygo}~I?#O%>TG+zrB`55Bc^Y!ya1~KAI^8CYlYR`uK=aX59$;<_`3nvUaBp_saA*l8Jd2`8Gb37+pdWi4j+o9f!&#sjJW~ z`0?CDnVUbXLLs`$#^S;?vNU=9eBB6qjGcVz1VfF`v#D)8-n)qEh?*OXc{EH{gW_Sv z!x$%GL(EbQ9Ti_Dp&<-Hpfh1ilcFqk=c2txXi5**oE2%QY`O);Lq>92)N~|}UxNAw z!gq|CdByd!`(`+MwGIHjv_ls|O8FbinHtWIp`4XEdW{_?6$gKe)U=|_aKA;fSLNKhXVVN4;kU}^AptbOt(xM*S)r}>5k!TIHtnq1Cas~ zI)$V>pWaRwb7XxyH3kkUdCX+|Xt0z_rxno%%hdE3^Tsakjs(?^DaRf~45j>S-wtAg ze!=J*tTXnR4)kNHWZ|qsT!W>STeD!vzC*)Mom&bm13Ndrux298U00lz3r)yU$Y+h;e_xc$gypz=DXU^FiaC{X{+0OR_WxCSD_I~oC( zPpVVbxp(oxXZaNSw8=TPcKRZB#;qo;wfc&Ie|F0IK9C<(bJY#yQj0^zNKS3xR=Lkj z+v4>><2*R>UW-R2`)83!^BISe5Ic0ac9<(j=qiZ=_Vth(@>xYom7}!p>qAl>6%Oy4 z5bpf@Rboh_Mx@ex)NL0E*P)T4qLxWC`wkGdOp|IPR)>13>e< zPS%O^j?@O-JN4yZDJft>UqHrYkJ}j&P@v&&F_kkVLlNk1<}36Q1GST_m@*{=@1?6v zx{xxDkZ@*%Ht(ql(_b*iR~nAY&#C+Qw|1=K)>u6B5i;5g;Sv?DuOQbK z@EyFGP&(YAL`gRtt}k~8C_mDf+%A;u6StcC@l-+FIDM7uzZC94Zww9dZ*l%{+l;#& z;>@I)6rc0VStB+M61xm17v$L;&TaHyIvU3#q~;M+b?pg>?QbTbLm{518VBqBI=dnX zPssFz+e<0r)gfN}4@G{RjOgknoNwJ3l@$hTtDTO;7ms+ifGl)5OJXh)piBI{p6A(d zbehic<$cKLyW@?L8b8nOS;WWP(>jt~tK;ad8vQa8BU8(7$LBm<1or9?mkUFyGo8E5 z$*23RSZl?rtW?Cd*Pz39fy7ZNkdS-8+i9d1au-Ar9!=7aI3}d-uQeeoiZhqt917lV zbV!=*vXy#@-&~o{!!`|EQn|X!Wg9s*6r^|m&{UC*x+fM|l?@Mo34(r!HN0sP*M4KE zW~+&BH^kOaA8CM|uMHVT!2+gbk!e28As(Yzw}rj3d-R*k2ZKmN$nNuwVuO)`d3x&` z{U(ft9RfYaR64^{I>wB3pLhl;0=Ax&1cPMEYZ{{bP-cuu8tZH3t4A;pazozr4bk%}=bCoB+M$>6L zS*ya@8zsNjB3-*p#58}2c$lH_B(1P#1XC@niAj?!P$?HRRAPw4qE@(1!b($OrzG?< zeOBAmny(!B8=@>Vai)l%rP9|KP*;>RY4#-?!8K$x*PFMcKwlm35qZzV1^VP!XIST! zcP|)wY&!RtfNDirca(29cPcpcCDtrESeeOb8q+N+ubhiJW7xDu{wgV|(WytHAlRh0 ziO1d}dkQXN8zwhKv{&Zy=hPs>D>zEhD%8W6rC&XItI<2nJO<_yX!;j@eXB#I9wII~ zOVKBEzm*qJ%0O=UB>?ATuxd8C{4wJmpD)F+S?Y4JLF;38xpm~o>HSQrna`0n)9Bdn zlfO}I6RNSEllL-#Q%{eyHLMsNr!utEa7wVdcz0fcZFu)N=sJRGdF)2V-D(Yp!WM_z zYE9I@C;J?b@iJ$oNf#g}dwPwYxGLUnbgU%&qk=+w<0oke4Ry`RD4Vz{?A*kC$i1Gp z+R|n$pCT<%DNVruEyMzBePj6u>x>HVVi77WYlG!}X?&~oWU)`8o?28%md1Orx2ep- z@>wX!u(k>_Z zDcSr>VH+3PW*$W&Xdu8&R}dOH=o^t3sI9_~2q9)b+s|sJt`dC`I?D8WV;4fKK5yP& zrSAkBUpxf`pp(G;FTf27PyE(?Ft1WUDueVo?|YI{z`Q8yKAShF6t;73)y- zfSi@@iSQSxRn?>>wPPjHsxV+nG-~~^(-0lz{4w?Y#{o*Wy_SS(z<6qcO7RS1MHy@Y zTYV1Ea!hGZzh&RT8JIYUFo})~>)9B3t3Nk*d|RVZsr;SrUOLh8HbbNL@L9In(?&wd zJbqW>(_6xh;3_h=(A=X9H`DELS%c`q_T(3=yLCbO44-gDxX+aOtTBq?yw#p4GaCQ4 z$={EqSz(nq;ZOl=Mt1vf*9K`9kFDO8P}AacPCS)#UzIAU%|i`vm`D^c9mb!bX;SsC zQy4luop067+vmNl({kY*r$#of0h!Me8&)`1+fdt!aplYw=N$3O*Oj1TI!5PCa#RtL zL!lyXFB5xu>q?ypJFvc+Xf(Z*RmE=4txZ7Y(I+hiFPQs51OWDa+kp}tG7ZCzJcd+R^ITf{q`JJfD$#4x9*LZCiY_DJXx*K0vooIuZ9rB z36pxD>krWrIHo>0CW^A1d`x3#YEmI#ekO5s7zfH;iOLZMzp(GOfkayX#;eCxQ0>48@4WG1qHQ7}FZq77Av94z=A(C!FXv8lBXvuA6Us+C6q|Z;~Ejc0U=6 z{Z-%F>H{0*^`W%+QlAs63zGe?S%t&gs^8LZ3)l@;*f=!b01QPCqvFw|Tm zG!{Ws`M5oa&UGs@h=igzZaI+kXbp8jOEbwIvZjcCJB=;j*7uPQTKu*@aEJv-8YG7* z5Hp~!p0Xpd1Lc*(5hdDacnKcK2wW)$n#kpUXE|QGM}WKXTga}irp#`z9n@KgHx}eo zw!85W6?x4%r%AF`$-$z&lwrfXMT1i*)zp+mW>r^%mn@nzuFyJ5%+m^p0?J4DpI4aVtocAe%tl7X*dx;$(&9pAr6G@GGin+OA#do%nIAf8ts{;2y%YYC*gB{P_IE z-5=7T*RvN4TqrJKxB;$CQ_p(KcZvs)Y2Cs$$xhF}dkdEcR6<`2nEAx>Z-R&UGw^`G zH^oBUWJ;|zilSGt!N>8U^sMwkmTmZbQ)ql{p{mG$HuG&GE#cGgV7fF@#V>^LVnk*I zH17Ih$TAWx-ap*f;lRU8LNrO>>{6EQ>KuLW^U>#%#SF0X)_rx>WW6RuY#sPG)*V*_ zUQ0lPZLygkT!Op83Hkl&{C{pmk#t0UL52D{1A4`RBIx5iEGNzMXaH{W+5KOts|GE6RVD00)W>ItWbg#(B~ z6Bdz=9Ut~v;2!<5*qc~wl(5pYfV8Yx$Ss~dez_)IiOftSXUCAmG>&c~+Rzw@7y6JR zcNZ?(cOok$;U$i-5aoA5?%7ibkBs&WIxz?>VTiD$qa=JpLqw0Ep)4lefCZ&Y^3T_} zLK;1SZ!*I)G{4g`_tdxxs3J{ZfUsmUj}$T94q47uxA!)_hU^B=r`;QqJ=*$p!&btN?K*2x zIVn0V1{7FlY}IRb`Qfcyjz!57UY2F9WwvF$WyWRBW!6EiLAF7@LB>JO!FukF)+r$| zqHFHEu`J_CJLxDndQBXybiFK});HZlYY~^kwQ1|xH-$sLfV42@{;UY7P-!XV$-iD~ zPzD}K4#g!Q~0IDz3(^lCB<) z`s#Q74n0Xhn@FRa+j;?Rk1j)?M?+CW7FoASP7#P)mZ!+i6VgVv;JF1l?*h%i2bMq^ zP9q&MQ=QcQTBcdrnhsOi9b)`6#^|Ux^FC2~6HGB#f066n;W~LYO4f@Xg2Xs8m31Y( zpgVoG@tT3Tb5t~*S5?8SvpVT7Z?A$?c^xa%MBkqJAAj?vpa+F&Ft*37V5SAFhu+P` zQGLw{Bcv%ZCS{)_ds_(jrpPpSMi=)TU zvxXr@f-waY#P)mOIAb-`bn^VQght*OF>xuMV-tqmx??s;0>^DSh~^`)R{B)OBBOYt zJq`bVI{WIVHoAUYC=`lAan}|o?$Y93XerPZcPQ>2AQTU-rFd|c0zryX+_gw3ZpGaR zxuM_po^$Tkv+f@^S(EH}_D{CVOxEnId7f{?+PujVo(Ak@Ac^^0K0_ zPnv|zXzP~RX4hjegly*#++QLH2fuQK3YsM5e5kd+EK=90{LD#)wHB=J8RL9o_pT{d z>gwyTTQ_+nbo={pYQ8dn=-2(P1;)``eV^TA8$Y4``DHzR@9lKSESnx2^aCo1+wV4b zt^FhY=}nG3pIuC{119JP6|fsiSeSXPQkA7iRa&z&+4i(&$2Y$Uh#K=?{AAke+N=_mD@~0zjV;3CGkW_qLkWM(79)n*tziAg z(kr%C&!U*8siG&H-Mk7Hf>TyAR6nIVt!*Z`W|5oUF<~ZRrpA#+sf4ZTy1)Xe>f_){G;@~y%(cf1-r>W{zs-SpGTLmJ|FC&mGvGqe0zU)u_%{7nwF zNdrrl#A>>tLrH+C*g$j3JN4dXV5(Gw+(>2f)kHj{{cNp;s!k4z_aqGZosVXDa+-HA zpF^n9uGQC5DJX}Fr8H*Dj}N5ZhQ|OAL)?OHjYE*AL&V~+Z|^q3_>&yoUrs?pB97ZWLfKaX6}yT?4TpV{voUz1n!BaPc)Z;%vmag(kI2rX%Jn2S;v2ofaN4U*xWGnXf>H;LjC7NC#jgQy33xs zCl{sIO;T&q;ewdwVjoS!mECZY{eigUS+ZgICUN1?5AyRYzZ*YCd4ZANy)=f~3WbpA zB4>5Hs<37!`DY8AR3$@pFEsVtzj+5)Ub;RrEh%|@7sR)Q5fkHe@`J%XnTog~d0m^U zvVL)YaoI{eb?W(`GEEvYc@ap_s?JqYcDLh3IH^XLYPzBo!#OIyhiBZExoHsibq!%(w85!rq|ko(BPe_`aqu`Win>Qjuh z$R(8zU)VpUMHQF4iF>9uOG9Nzdg`f{in5j7L=~@5Q%ZACWi#60B3MEEc^JWdxRLc%>p-yxwcw+z0~gOn0zg))>R%>)SizTA=qqvDe9Xp#y5X_;(Q-AEZ=0B%tNusQ5WlJnvvm-d zEvGt>!?cWDQR3IOT$NUFtk*0Khlk&h(bu8)0DT9@uK|0&Ww|&9_?Th(WFgBQHh@?Y zNi#H;vC*+s!rZEhOuCOwCt7h)a|t;c7KFJ*YhADI#6=nX>NnF|z#z&Nc0xQr;adMr zPyc4ZHC0MHgFM47g=g^KQW2MxZ|}GmSAb*tRF+!FV#;X1aYlB1LGkw&MXg*}tqN)L zPkSe36UOJUh7bvxj%V zYfH6nN5BI^g4#l{n|(vEVkobKJegD*YHu6O%fTO>hwN`03ky_^TA;J;sVcJ_&B2n7 zg~bdk{5zvV(!`>oHALS7kj$)8nr$}yRWj%LN-sPe+2il!#Ra#EPM-Ptgdcv6xuwu< zYoSR(N8#IZRgV0~yR)}RwVfz0qVe{ii=wG}+MASu!iZ3}?p2Q9CXwN|1>jOMJz8Fnega|bBV%t24%H%&*nwkIyFrKsSE;RKl zFZnL3w$#^a28&f{2B!f}z8I_s;2zAxjs-ogeGbmNMh}BX`^HegwX`XC9CXNK>QSDp z@%k;r4|GU5=s7`#yl)m{4c@NiE^vI!{JLbfeRYzgO!G7Rgl`j2y`Hn#RH@Z^_FDY% zjJE!=l$`Ni7cVPF$oBp8NXBaM9W>RL7#^IDF8+eO;4ELw_c9Z@=o5zavGA($j%QpN zQj5T(fYQyRDgRx7yjM)eJ<$!7V*FE6{TZb^OFez8FuX>Eb~_?W~D|?pLg4Xy+-uY3*pzN+zYC9RrkVOw}_QvPuVUO%IM^9G8wP3rWoid-{z!?Y~z#X+Ahn^ z<~}9lOZ5g`G@lp_WNwhJ8E11ow!a#3=y69k`)u$YeZEYy{)fI6kZTbfVt#2XN^qJM zn;E&^NDdix<_lt#E7DrT0J|g{w**fdW*0WIoFfC3al~C%OrEux5*7Re8EY})Oaeh4 z_Pn%)GgZYtU-AU+9>YqDsCe2H(brr)F2i2fevP@_jf*ytc&{P`(vv0d!+y~~oB0F6 z285%+ng~Wa*>5(d9e&(X`rgo=iR0Q}`D`8H^BBa5h7^~-S%R+JxRWt_krFF#As`2h z-w`J>gYmF?ckMUG6j9e-Q50U{Z}m+Yqi&L4xI|V{3cjP^(KAvj(QYLbe9fFKwAiGh zAG5?m-P`nXmGbuh94+JLBdw4-0VHr6l@qP_sYVChq%nJI8&-k&2~UR%8-dNSzG3Q` ztZCr&hHxe6a#L#|vHyNA9okCs6eDN;{qk+oMDk>F%`peD)+Nhyic0%Q783B~F;i{! z#Z}p(Ks;LEgdZqcTc40Q$`C);(_U z8=G*?9+k&_qr^UNkDIrv#nVJHnRkz|bHqUcpl0&|$icT!2+efvU?asOPG9c~;ix)% zD6uEl^Gzg11qNfe?P>W!7e&YWM>3Hf$!0^?O=($NEE2Z(bW->J8`XAjV z7-V#A0i3Ue*gFq98gcHh8rq?#=E7eYdHg(@&Nu@$xYKg^S3K-Hk6&T8b@1rZ(QbO3 z7#Fb2tAknF4c^rq@1nv2A!1qyz!-Jud2hJYwIBMOn>ppckX`ZUm(F4$;HVwsIlJLE zdr(n|$@N=Vv7FgdBM)?gS!vsmw?8$g ziKhb+e#(eRV6$;-bAPE+NC`Pyt?WEU1&9}%&L53yznaPXIyALDbHUp06IkfX{lN*= zV!d4C?|^T}NQ)zriJf3icg=o=GGN*DI>S(-Q0MyevFDS?G?jbX-a?o8VuIMA}pH?dQA ziBO%jR`eXy__9{+B)_gGUlC56ok!DgXfRQt!Cat&?F;Rn$_I!DtB92(cD$0wv?Dp$ zWV@;zL;M26ZFk(Y3etPL6v$a_n#Q#Pg)?UlzdKV$U!!S7ED6wlZIQGaY-c-@BUWi|ZKE4_SE&Bld;rx`687lzh7*fTnvv)* zu`l|0=;k8e@lF0V3p1{a$F*8n0my;`KyQyKyvDZ8y4rC?Lz6f~0cIRdTF%z7n= zQotx#%&-Z&tT#js6l%qOgjgigquUGPS_Ya^sk93~`qlB$rkol!CycSg*XV79qHhBB z8Ke!DkV#i>bfT)Q1O}+^?<82=bf6(iHK-e+(j~n-P&E2{nqJ3>s|)06PE*bCo47jZ zbWhn9lnw4VLXUO|5oDB~-yQjq7Rs{HyDk@l%!QV3_Cstg{$f*V=&`%Rt?zYJgj_as2LuTW{@d@W=_+u^fngsYsK|(DvlAFX~&; zi8BR}63!K@X}m7)B#=9N;2O;1N2(IfRt2+eP|180kw@L>aDGPx_zh~qb7D7y-c#C!vJaI!`zhNberkO6K?d=lOq(qDr{SKB zjl{6{y&}BJW5T71`e)ps|7)s35B$dfC~xW#_B=l2weFZ91+HclM=?ye;^#hhgPp-a zM-oG0pV%@+mvf#nH;k*xT`ZJb>`t>O`9D7g&FV$Rocr(NajQr>K-%euVN z^3r7?FNRrcTxL~BeM=~Y1znswrF*a50tUA7E(l{uL!VBWFW2i~0E_VlYpV2{n&bU< zt?<^BV{f)b&$kVc;chm3(fxHxmLuo|{(dNj5T)EiipG_;g?c!rBk(J|b@$ah^@JPE zBO!0GbKHWpx!o)!O*%XV?j`b5t4+_;)1Zy|h!gkI(B?<{oBM0d8S17XSol|S%R z){;LOiK0zre}cLX*BRFD52|RNn7TpBqwFQlhu{*(Gb0h8ErL;)1dJ1{uk;B{c59%! zA~7^BbX@#9i%XHCiXG?Pu=?zd(S&{V+F zheE0ycV9q--0;VnTyiS10P2G6@}`Cj;$v_%-V8 zC4258-`*~bzzaoB+Ojp2aa62A!7qfy=iBk6z4hf%iXU`hkkL=3;O7k*udQ)_!#jx>59yr z_Cl=hxx3rl^%(Ua~qo?26D zg=_H}l}N6iWdSk_Y>`jtxTo7hl-07V z2JzPXt1fOknnmc$S>lkmgZ9g4Qg3M&^YAs`;9ImxMHW=}NvC z@>MN-gudiOW*eb%dryOIgDR`XagKCZcwyWa+s1T3Ce~wMq-B6*DyMQflxd{M`P6Bb z!HfOT7OfNU=4p35WieSPQ{8Enp~8}4)pt%L&(Rm>0w#I#`h$e;?35VVM6-0}9$!yv zShn$JmkoA(rwl*c@!!qIqt`vanPlQpVg9x=Rl0|zGOo^iG+K4{M#Z7~)jq5hnKb3r z8sJ0m7ONRl_vpv$;IkpRqjL6{S>`gH-|QJZ)tIwvM&hPyuYRFvah=nz^1}sMn`l+G zqOPK&W=VZw@>Xta&~06+{2A870@pDbdPd~o7wvV4?sjTuc$gD!=hIW5jpoeINo7-l|7W+JZ*_BQ5dRX~$b> zS-@j4c3!x?lfB%q((FTt&NCmCxLch}DGFG`lnNtUGj}oL$(f&q2jiIL+jG9pyj*hP zM=w<1A(CyP0yo$?2JQSB5~0Se%x*7p$aoC=NNer9)cxu|ACDyI18z5ad0?hZK1jz^ z_u|L)@h!JJQS^djOZ8Ashrq?lr1{Y)hyJNHHHeLHU}8`M()^Z@=q9$##cxFz^5hZ& z3+GMqI9E5Co^%aG0a(zui z1Ff%FKImJ9-l5&ZElBd}rO@vz<*6tvpjv_>NpHPeHW;(|My0hx&@|QujXpz2_gAG@ z9FD0?4AH=EoYQP^x8=#A@x8y8sDduuCbhq~wKs0F)avE(3xKsy;%1H3>7+cto-a+} zOg0P!_!6?KRrr^dpDup}Jh}yteadMaP%D%SK2avKy{X%I^^Nr{1S@h1K&n!cXmtG% zG(am;U`Gsf3pFB&*5*20DEGc3IWoQO1o+4u(XQU2uKJ8y>~TSjd2gnHr@ua{N0{>J zpIHRWx**6M&&H*$pSoS*vsBNHUF=|%+R@Jz0Rxw^b21QT@d;m4)uH^YYLaqRQ#Iyl z!C?hWw-XJPJ6}Fbn2+Xj-TZT}q;I6YCWeC`bV$J_pJiJ)&GdwJ*oAwPl|n6)NIo)4 zVABYQSCA*6+qR@QKT8Qv$;Q~9bX6P^&jf`*RHMYhe4C%IKkLk_rTUyrf-xCtT+sNm zho#d=Z^9^Y18=zoI{<&qf+XVG?q*!{$5t6#SL`k)`y0w#80ixo<-kaRR>MY|$k#eA>O-Hn2{4E96%!Vfah8WX7S#EjdR_c=7qi(^lM-P}zBC zagTP%7YW7Dy#+$}d${e5L(iBc&SXjn?cACJcrkjLoiE0H@wP$Up|egc$QnAnuc0jg zcFxGeoR6IdjXaZD6{tRk7VeK$-c5NW`5E`_><+1nieH)(kdj+9pX-f`=Gl$*rz>f& zG9KQ0n0m+$ahMcHINwXg^k*pjq`gwL)8JvIBX?(?JCx}bm&h2tCbbmnJXc=#ui?p* z+`<52zp#E9urX0%JbNvHajrRwYdmZfGbQn4c56da{gUkJM>`5?STWr;3AH$d^|-(bPp>30zQW57?=^iLu+}0oQzqub&J}K>P8pBzs8`vCKPvKNtu}~< z&4zf6x9_9su@Q)tG7U=tQ|U+8>gukMX@5nwvUc=QZaKUs=6I>BXo&85p@2ShH+!Y^ z>Gz!l>r}5&`uw)i%0uH zN*XJ#Cdk!QA?3e^hM%rAvv<=hcD@hA>NQV<&BpwW-4kG&EL75%&;!3<5i z_l*N_4avDxw3(|Tn9f)diR-!4;d^6`FAFQX+RE6PrKpv@TP-Y?*H<7AZA1M&X^GI07MXAg`knd?VHa zR{`3UknVptQgby>hP1r9RMsG>ycY;CNxH0?^V3ni^7HnKsGkqLDTd{MU#1twFD82076>A1 zv&q`R6IiWZ^&l*Ff;y8aR4190jFG$6QNs8Wg(j8ij!3g2{VE6P+_jJtNx~Y;B=$@t z;xJQ5DS7j0XFbV*gZrXPsymPA6Mi)g#Up`d6i=|vD>*Bx^FW+!S2m53PGNGh2kObM$x(nMSHt(F>@z5dj!KSVeZ*IGXcy?qP; z%edIwoB@87ZPW^G6H!a6P6`79+*7E}53X0KS=PFMqd8Mbx!`qhP@JlR^*Kz%aO6w? z>1N=vjHC{vTPS8TP9PEO8O4t*rRI~ zkX~~A=4Ky1O1V_RRU(7da;~sBaP6FGwropAYoT(UO1V~WQ zHGI)N>>}N~Igmsb9rMR^%|fC6{r_J)g41)~_(1|Hvz`J`NS3ptG;bjvBGmVrHX-KMM!FLq2-#kqa6mxMYkTqh?he_(eWEKEjsxV^%QI_e@dh z>S>&`Vr<|{1t<>03NoZWP$-QZU~IFlslcX4GFBvZYq6tdm?nW#Mm+o*SI?aQ%6 zdneg?yF4p31-_noRD9n0{1;lT7Wii-l!S&ZBhmG|RHnMg4Gyu<5sf?LBPm??ONol5 zm0pkUE%}#Vp+vtY^91PlPvQ7*@nza}USLoURpkfJEdoO)Fl2ZSM1?Q*wq0I%FRvU* zYswOvV6FnYkh{JmfKpT6iR@pszIkv8=6`di!8N?xU}4kTb@BEpi8t%UKZuickD_Eh5g$Tc8!_%b@&ClL=AtNMgiR{ZCd?>r8O@3?X4)O&I3 zy2bC{`y~{(eLz}Goy~YdzyPcIJSlu8FZHI+Ndo2%d?RSm-B!5dcpq)BcEv_0K{BT#Xe1~5FpUokTIFHv6wbXFZJjP>c`Ix@-?*bS<5DfK3au0Y!i9M z-P+gH#bB0~YgN@W)_wm_N7CMiJ&mNb)iK~NfHRHp&E4M(i5-U>ql`{VR4ZaEEdQG% z{4Q?utr*frj51}T6r*Azxpr%0W&fgsv3D!q3q|sUiP!cDWEK|!55)~AeuRfOj)oqI ztzbeaLA(yzEb6IjtIPqBX%XdBuf26OZxO2(jX~63k9gVRmhkPHC6ybZk{n~7QME@1 zhLB0=$^DW~BoHXF?+G4t-j$mV0ny9jS2Ef&T3 zwsOHZm*T0y4`?TH?n+l1vgzQFaS<*jsuPhuEyDyn&f0EBR#GbyUd0-%?^h=XlZAC- z3(1H3E0tPK)WVpZzjPrJ>NaZ9ybIlE5jIe#NN^+UO;0eLK=#srIzcMLiG-fLLGOFl!aT0Rs<95yrOx!(g^D9bh9^VY@UmMy{W6y@F`bXrGiK31)j}of+<-By|sG9Fq_xF z2&+9B&N!w-)MuRjZ)8`WVhB3*-=oLZm6&~!JjUVgtmnih91Dem2zk2mk4~#DpJ0D7roIWF9YLDcWmxNRm4YEyK<$m z4Qz&2e+Qf!h?A~(H%latD*c#ED0hsTuIPq|Ldkfa0+-mtd`{g%A5BJKUD<6sD*lP> ztuJgmL$76R<71Paf0Subd`@=pQNw@x;N99Iy`lmivI;zAkRXrM+jz;He6x+wE)*HU zgtBGg(iFIx5vTbsdcr0uy;YMNw_WcAj>Sp9=W#Xn*X}dUWp#oZiIpD%=DBCP12|{G zW4qm}l6y6X?{9BZKc+~WPv=C)jRkT`sdSp7Ps$K2VTObp>(z9oJig=X7N=1`Q5{3Y z8QS0}8T=H}UXK5A%Ung}V~aJ%^Af%7F#Vy8_%f<8E$Hj+=Z=ymc(4j}l>dba^KYe@ z|DaEmQIdT?9#~X)|6eSs2s;fg7guvTCHqefe@I`|%`L1D+@@ZbEK&#&8H7L#m#VXw zx$^_lD$55M4aB~No1>$xx!r>%%pd+)u7^wy#H$aYHVBCso(CZr1Yay7_In2fCD}K| zj{i;4ilAuK#1s$^#N^WaqadS;DJ1Zq2g8lYD|NPg?SK#f92#y zxFed1fd7~v(*Dc+-{BB$`iM7zMD`#0*@xJF)8YP)G5s^f4kxQdHH$yg#XiypNIFa9Q-`|{|hGgH-?zczvA)p{3pgM$jkS4TLc97 z|HGG8P*CKre(;M3{cR^K@;4^J^FJ_GXJf>1;r!0su{G16??R*Mu#eDl%n=bg4i>t zEmUHQEwO3D_?fJo0M#n5ezK&Rv`gHQ8^k#j=I_K+Zy)%*Jg{|{9UOI!_{byn2d2U zF+?}u`1G|6TN@XdauxC{zS#GShE0e zC_=tLh~=eogua|+8R0NPPQV-W;!af`CM0Y`Zmpb5C@^CyLsf|t>W>~ASnOYS#^kmd zNN|2bP6-ULe=~^7?btzB9E@z7D^Id%lsPjPSOlx26!`XF`f2`dFe8sY4{Hf6wq;9J ztG=T|n&FVB-nITHk7))T&k3Kx(2 z@1skfswnu$dF;0y5_Bu@8|`uUV1->))SOdRLjyg&uzU1iX&g1b$G%cQ7!lm@63o-J ziA%w`_POeeAojPs?v!T~ zWO&wvzbj-ubu~=9$@M)B_tMA69ZMJak}{?8$TfX+jNH{evB{leePK#7^F*{GCh{?S zV`=C3xOa(!l=qs-sW?uiLvhycYI}C1T9{QC-(Pwy-&@j}uWTD?{LJe0>Hx#_RtJtr z%@`Z78=slbEI8p;2IhS4M_j`PQC4i33ryV};!Fb=_sizr;L?n?IRDb1V_S9)bMHyR zz7Pw06v3$n>3Mq>fwMQSd<`uXLG(R%fY`HlBb*vCOJ?3bF=~||y}#(WOM5i;nPmQM zj-JIvkbGzQj-$;Zm(6Ah!|yDy2md^NG6LuIS4wm;%Upsv zdhS_?vQ^H$dn6`z_&dKoJ~;YxvbCoKB(!wM%Q!AUyfb+RQ8y9p{^a!ZVLDagL{>$r-n{qPJz5QP#|o|Ly0UL9TznR$2ivMXaMAIf zY2c3ts)?+U`#VGfVr3ZYI@cxbKSl4rQ1R2$cd1Q(94Tv2M*O?Z=c=xvAa=eA(l`T3 z5OPexn9uZ%e=_8KdSu+mBNB>scn@npFl^HVg9j@@GDgD@!+pribhJu(rG3XAxPP+0BMa(->%uc&E({3%a zJ}0A5puDWoekDczK1hEyqxBy0qmzsOqfJxK0}lqk4^Y37splBZy;sk~6zr9V4T2NB zQ%p+=UdJpU7Y2(*cj$M6z{N$4e1Ie>5wGhmJqRJ{2G2EN`o{pEmiy zw?m=NWiqnx0CsD0y7gq+9aCq&{p3{Q>~2JK?yMDjQIx4``IkS->Gi*o`a<0vGv~0{ zZ2JPzk8QLZpHjtg@5K0rXDoK2T{Z5T!Nd*@8D%k{zH}0+@uN(+M9nM*Yy91D@3w>K#mK$^ z+s+TXuFBw}BLpm;*Trxa);Jrks;3lBko-9j-U+9=$A`S9vGp6#qg++h8)K$D_a`J> z)3>4y+p>Zj1 z8L{{R5z&kYzOFxhzG`>mdeEmrxd%QgYz1#~eRrt_^Na7FoFE--1i_a>4&4xx+}Zo! z*BTfZ0zcX#X!v$^jM#7=rHeDoB0K+v{HE^t3>^)T8ZP%tul=rz@Q?|o5G4Mbg5f8J zYF|>X$?Nn@(OO3(rBs`OQwEl~vpRD74Fo>z|Bn2%Bt|S>%PB(CBo}25RuI{?xIu%p z#_Cn9t^jiX*mLYP1nmG_?jTdAqkJeT%X%^*R;~Z4c7IjS!CuXtHKlU0vZK4!W39n;x*}gF(rI9S zK>1oU%4oIW*pshW```EPm_(oaSLb1$?_;El?V_YbE`48pgwmt=tcJ|59SvfaGV1a6 z6~AtVlM5)-buo+9#Hg<9B*en}9`;V|ZOMV;zpU$swzArGz z#=P4ns5IK`X*ZDCkhk_(S;pVbEVCAl=w0--aoyzkcX4Z>j+p4n|0u!FDUe}8E? zv;O2?p}C-rQ9tu{m6jjUdA^og_|^7L*BfzM`kL+A?v@fSHfy#+5TS1_UQqsTq#6(% zm0F$(br;Md6>S@`tuHiAXBXL9;Zb1{r}cf*Rtib|=De^M4E%hme~rL-Y;VyC5yq8| zm80~Sq~iX;ihkT(Z+~tPu5gAIbpJCf7JP|UWF2iEp+-)6pCoCF$r^;vh3SR&bB}r% zkIr!DE+KUh?_cAw8Y5p`#~VW)&9lGz z7i!Er3Qu(p&n-P?I5Qabdx0{#!uE~l;Wzc6;rMc1sK>3DbsrJ`&)=O?AchVj#BF>M zQqW?=wik2XfA?euUqzhb5Z2lb++fCmxY=4Lb>U9IBh$G6G*2L&zRv9uA^jf#OUEsP zuh=&w&rX>*Jel~S#*bq`?O)^O?VM8qhnh2X42+?1X-}N9Myx%I#1pjB2`p%$L93_< z({ttuV#0_!6qc3r7K#Ewg93+iw46U5+wY`TJ4#tAu&kB0NeUta*(IkeT3I?DlMz^} z6&}&5aGCrd0) z5%1Fyn1f}ENb)gw`-x9xHo^S9$3*?Mv#-z#7&<2_B0|vBiM!b5iGePu-_FVKs$-nn zz*~pFzncFiMHau<-rl&RTjcR8O#SDLD<-^nrynte@~`5-g8e^HE%$j`YK0~3d%Ar+ z2KrQLu7(qR&f>j%CA~2ocV^(=s(xtxt3!pDlF_pQgvQOw|1t1frbP#;gXQuOFee z9P#F~{`o4s;nfFnh)Jg++vHks2kS56k*#kqK7IB}LVvi6CduT45hOqxTe*LiK&nAu~6|5y@m zBQ2)50<*9iGukHFAkP2Kowo;{s22}I{7Rz{aTqB!ttwN@oGo&;@Sv&N)WH|eisoro zDN&HKYOphzfL%Sh-55y+dug^*!(O9f+hUMJSc3sVm zlpY(aiX9EMyZ?{fdhzB&ud+RZuu+Or69H|#wEV_5es{NJr4}ZQ2HC0oOS$o5h3o8| zl*fCH0Z7Fy&avwCneB-t*1_Y!>XOInA%~=|OTX&XgQRNxv{#QjIefJ}!V==ibJ%Y4 z1c<9_^p$JIh`sI{x#``8xBg6Rjz^vARvq$qIoH&6s1`@(?4Nw(Hv0=|ybr1lOuYD(E{S7ot%MS)ndLiTHC8`&H;#tf-@aB$2Dq%+i=a&v)b|G-a4YtK*ZAyCgXXo6zR41djz=O&i zFs1+*%<6WRPw)XBSZ2XO`}<^@0WY&T4zAts?Vx>%!@MU7F!O;u5j}c{nOvTO_7(cpyVyANn~rDBzRQm1uIojHOPUbF;4gfuNesE;)pqk94%c3h zTW=bNa@5`GOe)5~ewDH1q1N;5%!uO8SgP(?~(Rjq$Z#4`Cido}JU!S(VZr0G9wUza)$g}D# z9Z&o)U3ZF9>QhqLxX766m#~fhQUr^#L`!=3B&ZKNW#J2)LZI7~OGz3&qu}E&^-^-5 zD>V%lg*Dj6Pt4G(Vd$^8<5y9y8L{e&doLvjK2`ZH)@G3*8-d?~YW}>dVr*vXYi7MK z{!W`eMQxGX&z5&1;1O?;?Ig2V0YuZIgji|EpCYOq$6Y&mu=ZB$O4?y73QP0FZ%_~% z`pplm>wlUqXw6rN8hq%ydbwQgn-Fzu2Zu>NDFOQFz_o4C%@BWoM8-oF*BbN67JV$8 zz$UK?SHOUE&hDytDdoeo2UTv==Cb5st1+9-no}N;#heCKQ#*YVUAgwN-xcYk3Fh3K zXZ`Q>)7E}8HA7W^Wy;UPUpc0e#M88EbuZ1Dc8s3#2du?5}bmO zSX+hh-P{yxU!vqko!Z1Uq!~5d4*2*~lpkU>wTUYl40CVU_v>U^L2<-gfB&+IfBXR| zHip&Op)5w*TZ9XUBr0=?V1}OwtqOk}22!-Q@w!1YhbMk0u6VYuxEsr}wv~@D4{4^o zi@fMC{W@#^HRWmY;oQ69e&dLAS;BX*q0%g#% zJ^QH_qll=s|B;e%zuXX$g?;^r$n1wN4udmpeniPvm>FZ{@Si^IXkuTF{%(A5lqx9K zFo(j|x>Hn0rH}QfTNq_q>8~_oxt$zSI5cx^U)8Dq0Y7AmfpV$BZ2!DU4es_!o=E+B z_ZYvf$2h0>H0-+h=2E;+suA|{;Xlv#+l)li)XRta_IUi>uN|vCt%Zgfpf?upz$&3T zYccOvqA^$OM+Kcprld_X(t}Sc<_~1|GrIb|e$s~K*ta4+(@E_KO7ENG?J094-0d!u z%akUv(b_wK`Pev^$ZRcFcOnl*4OME&*gad%u1C%sG|#Ohl6L#nBPbT#_m0iaRQUqA z{OR?`yMzem{z*Uc>GhCUl55Qht)qLEvqr`HbPf^FBGDtOd0}0vIf{(sDYw4JpF&#| zsb2+^5+Rt(I;q1?_dCTwn{Eu;cNx0%WtyI2ax>PHRAx>(XT^Nd zf5u$aC$EWP?B2dI+b_F;Bz??Qmcs!!NxhbDp@cnop~{{dDc176?>Pq$K~*{`l=Q?o zUq~j!XuGZ%Y2>ar>Mc(=xqPG9gpGlL^qPTu?esdi*UjXt!cytpE6$(9qDdMByc)w~ z72zAa!z=*SZgyboF&sS2Oy0B3LJ+U1)WAHFvdnH}FQ?;^V0~fgL)db>#C%jXVzl5l zG{T0D+p*m0RX!e8D+^V3M9Y)Bkrk_MN$9AyyhxC8%^9eNcGL581@W5(>1!}FQAVAC z+&56!K|&R4mY;kvy#9pt{Qj9?+s5(pv4q7+u5cSCCvYu8sDh?QO&bv#l?!G6M()sG zg1(kj(CF#*k>YV(*Ayr8?+YQdc`HRK{Y%BKkH_8IowCm3=cA_rQM!d^m1|(!q{X}3 zEaWJ*VW-RdDNzpeH)}fw-#AR4BD+EQaIjDO;}1QO+9)LYJ5YB8=hq6(4}6xbhSej{ zRi29TH|BK%=CVBcU~Lko6Gr_EJ31JrCBDRTXLWXrbTe+rjGpW59^&NFED>FsdKd=x zp8P4lQ|?ntmioyrKu{s$Z5M>~7To(}66u+yPm=rebFgn^LamMMwgHv=pk;KFZ7Y>7= zeB`8AF1*@PjS&bZBksp#t-3_`&rwW+PjuAWHy`&oyA!p5B5==hy2{*+pO<4`HDL;Vr)4QQ1r z3uzFy3g^wGjtQ=Dridyx?a$g?_cV=f1y-{;11^sS` zdTU;ksSH)2k9b?RZ(U1si@H6TDf@40do||z1(j<-VWLts=fe(CU_Kx5t!wP_&GfTfQq*)V^6QUUvDT`U?d|mzZH&N z<6W01pSBoBCUUGpMPGX{lC`4w6&50_`An>g9u!ND7_@eoEN##Db*#Xar>sNQgto8b zzZNs~1r7W}zx43VJyke+o^$1cb{L#qckJHBY_}}gq!`wjf}Cfp96uegg7hACM09oW z&Z|yDf@JjgZ4aFWsW*V4T=gCSKjKdTZ=}4s5SU^E=*f7hW}cZxu%;X#Vtnw}7ru3g zXqO}_=n%GeC+5@}xL?sTgAdc2VZXZ>d6v%M-O87wI|0qW+BCXOHg}+W7RDvQuU9B^ z>+cYB(UUhkyw}5*=azLZSIcd`*b)9ESiPOwyIH|`I>6|jVQRY&q4YORn7@Lyi5{xH z+J+C=py#cP4o-?yGZ6z~%C?KBZe@%CH&ZL@$8npsiOz1P6{(0N44^z)pibj zZ=m$%h;slFaVyCqZGH2yTR(=fzGX?T1?q@3Xp&TaTal}mg-2H?j-JokIwdbMB)bn~ zJQ3g?At!nn2E;1>`@Tcipf|eixY_H4CCQu6TV>_Dy!xs_@~sSvy9y}fG7n1I6*HH` zxjv&_NGD4FQ{yD6FGxyZ!X|>U|B~A&p63R^gw&N4sKmuKFsodoln4i$EY%=scqFxp ziev2J(bbjXHjel9HhA9KnALn|iI0>Vp{bA}!(E0-`*aCcCbC zF)~iOF5j4twIBy)?+LbjZ?Yzpt!Zx^2TfQ6<^;qzm$+RjsppyhrYXJJKd<1sWYqYG z0|aszG1qSm{DeI}`0?Zk{Z7e^BtI@V&#CY|H9`8dSx>8qky6@idE%OX5wRzE zOu;61mb2GXsEivJ7KJ|vAlVZZ-v?B(fcy%jdl-f6j8D?_s8&x27u{IiVIXS+;5ft~ z5ghIs$Sr&DS_bG$AA@GYM>>1d`0c7s>s9y)`-oR`w41(AnTm$$B9GnVs z2Ly*2F(jEUmZf_8dE=dmV^$E|MnzNCn7BAiil}HY06s5p_RWp^tbVaY4=$x|WQ4*2 zMEnnsZT1X`R}q2K&SJHBuVZP{seT&DAwHyz&o@p;Qq^jopU@K6DZAAQi{MyfIGaHy zk2Ef;1J0t@%!i^v*>&{|?=LzE*6F6cya7c6K{>5_(%|bPy6%FvUFk7Jejn(v4lF!n zBboCOxRVZ=ebQXIoNbC@v}queQ@wVKEbd9HqZ~;F-&0t3%l^DkNQdYSPd z{d4P`X)Q&PVea?xlz@}(HsE*$*Nbwze1MZ@z(xwDhZn#s@F^{79D2v6C>0M3a|Xk_i#7#X^^o3RENG1cUoWGnH}&%Ul1maev8wP_}%H+ z17}u$C9p?S66+!KCF&R`h9{nxyjiL?c@@xKylqI5pO!->_JRC9iGZ~Ad32KW%)S@_ zgw-)xffjP&n=|;pEq?`^&Nwo8e{5B9?eN@s>^J?LgnLfFlC7gFU)GR7g!Zy=&uhCP zr=pSR{X)6!=w)YcykC{%LiHnpn&xcrq={#lsIBE1Vzuf;jNK5l88`T!I6(A|k|fqV6Z ziL<|<&uE}Hv>FgH#UfN1yul0fz&-6XZltpAI&(+ zh9eV!MRf_eJs0aC@;q^8U=3_1M^)BAJ`^i3^Cm zI_*y{_=@N88`offCfav+;6UYjM&c1pWqftT8R{$CFGt?DD!?~DF3oic(Oukw*0LIPJfA~BK*PvToqYu1s;atX=1~J zxpea4v_)8 zjHSJie;&`PpMD0cd6hUP!I9sNcP0m>=B@P6|ChgYi%|6MLNjq?iH-ff?;HSVg zcp@T5>4>JzEU=Yu(W={D&iKBcN|f%nX+J9eW&Peytbs^v{_?e>LG1t*E)kD)kD&AQ za~q%Y3h!i$Odb%d3FZ(PJmc@U>1nr_!m;zy@-I7dcma}aA@DnWZ^0ZPUw!SDhzt$} zhQ9j02Ys&O0j&ienUAR6c^4C%;E1{2ma0A33$3cs==)fN3~?9F;|Ew=CR3uX>co`$ z4!8!zhxW`42wAnnMPpul9`t!ofFpqNXI>_J^6Zi7Glgqf65S7p+|AnuR>AWP z)&SUtUPl2f$YRh=CMwu<_LsDcKzF!Yc7m)2sc_Whc;*%ebjb?XXf`AHz7ixQvMS99 zg&wJ{+S7QaMnb$n+|3(nqN!Ee+X_JpGuY68zMe|(6F`*@20h9NMz2&Wc$PWnt~PPj z+_B#3*)5nz*fHlmr1%GnvVatkK>0-X!g`zS`0kyLV%(d51po8Sj1Ce*d{p&(tRV7} zy~?I{rXH_AX`N{DmzRYi=C*2?XZ7?nTld9jD8 z9+sGLk~s#3?D$r?%24sE-^OA1RXPeC&iWIOo1{%~g`j&PLx&qup&oCRaxf`uMhE+9LDCSotr2rD2R7uH8+FdwGSR z`_;Cea~+|*RKURXAXfs3lD*F ziDZ7T9IX<+>u!O(o!7{bpL3CQ&#kT+pBB5DGYlAk$E2@GF=AL-{uG&0TrDor68crb z-&5s8vN$q~t;9jD2f4gjbzRgT)3k>=b*dloznox|zwA9z|tqVs%$iwb$1sh;Dwld!w#$hu+A0{EVk_bC5_I2a*e6l( zrJ~8fvHiBW4@^1Rm95H2;x;38YpU9?awuzM2ZV=dek$U{W>Z&fBN5F*+Wr<5`rf^m zD2et(bcAtwM!bQWJf~P9&n|Fjnz=?KImA#ZLTazFD2Rk!jq zVT*D>Ai<@WmG?}#eL~Xj>K=9!SWOXbc)9nq=WYH zH*EmsDG&KXYz2py8WE&zNgM62ti<_1)GC-*EK8rHkstSol58B$&lTjJ!!j*@M$0B`KL61 z2%cHjq}eadpSqg#Q06Uekwx?qw0f7r&!@d!3Iqy)#}==uJz6fmM67Cy>85|SuYd1p zkAFq^^;d~p5yzj|?I>JWU-;vi&-5{>d?}o-an%}sdkybR8KwEe^22V4 z4#0Nx)G$NrvK;fgDj3vN4lbzh)7&gULO;W;!>bulDp;o~Y^W@K<_$tFto4C!INsYO zgo2M}=vizzzwRFG5-$Qkp}@8wDmMUZJ{m$1q__zw*C4EZ7cQNRj(`M+@@|qQh)f`A zwNmQ90@EG;q_d;aao%K@#Uo%ikzvp6)=y_`?7RLZz}KB#kF~}O5oq(RB&$}HTLt;Y zxrE2;O+GPfUgKUT(}DYpt)sPcF{t|WW7^|ERrwDt{eICXFbWSC{8&PK#o5%heTG+< z$3yDM_MxG2&2wTUBHVXTWfY-`gkEd}L$&4n)~@H zx&YBS_I2S{7^(ZW7qwjtQgz)`fV`=WjE{3O>Yqk8^T5fmzu`%*=`D$Bcu7#|p7()7dLoa^6jbYgw6%e{|I)HsDUSXOg>j%1~(Zf z@j76#@GdyB?v~%si)9U7Ec@@!>d&JgDE1SnR;S3Do_rpZMifQp&ywcI3IKci)iizz z-}eR(&)KA6O3x$`zgVyWa)y9jJ}vdxI5`IYXc6&lvB{O=SCZA-C|}WQhKE{)NVTia z;cgRyvn&kSMu4Yy;-T2`KpO<&A76e0Pmd)z*w@Px&~)ScD=${S(NWhONDaf!zqYs)+q_HuqAC%6fzs2aGh(iqI(^!w0VKQhRf*= z;jRbeV{4rH*=UtmyjXDhh&kx%nF`BpDN`#OO08xc0F%G8*)(`@_xd$+{C?eb$PwqdL(>!?zmO$;_%}h zz3iTXsJD=f8!T;0$evx z3^=2!0QRK2EmyKu=)w%boTu}xU{4*$B#%^cFcYtB=6vq6%E!8yMHI09>8lahXk0Ss zn6D^=F-C6Vn(&IMJ+1~KLN%eX=JtBM^H>hpIZ!HFWBj-eY3c6Sb@{TIh?v6nDe6GQ z6vVFdy-V~k(8Dfxe-Y7rW;@oWJgefo!nzVg@8*1uYpVC>m>=K%JcJu;aOk`mfPE)> zi~DPr$?9U6y?X*MF}7of#)Du|RRwIa-sEP_H{bW94pL5q_K02mRXte! zdSkV_uFL5xmn#*0&jzz2eLdnwufLDfH`ZMu-KOSLPER_Vh0!NLw}KOLXS)^btW3Vc zN-}!Qe;F}@6s37}4B6aF9c|EieS)h#gj)263k4!aB&=IPwMiUWOQQrn*l~P|fSap@KHLd&H`}NkO)iB3n{Bn0lxw@Fj^7K+b^AD7CKeZk{hk=nnB#O0xTaDh{jtrxe)elv!BSN6bN=YHf}mXTrnwPcI({}#&9*dC6f`^Uv<|=d3XExt ztSsI=>AGV~ZBMvWIQj0(bya=Lk2Ck}>#^S~BCq~|_dLkr=k)&E-~NMWdCE{+n>*&& zOtOHii1Em^gG%g@(zyg@Ne=g;l={W$vBGP>c{u5ws?T;6`y-y{Z@+sJ0Km1lrzwX? zf($-erM()p3Bg(s{-2E>xg4U>@fk4dP!F!qtZnPj0|2ohb)Q$RdV$ZUy^0tU>M?mf zY)3Pok>Sv>>Xm=S=u}nJ?Gx1A7dwItpk@524RtK(`A047F;5sDvK`pfhiQ_6Nr7Zn z;(hGMh+wuwRYp(V!lLcuNl5M&weQ9yY~@#%9CDeo`!SVe8^ZAS5bf!yw2i4q?=#0&(SU{Fj`bi{+CY?eHqfEO9bAQl9l>#oW3s%P&SMehhT-)cJ59uUd=acB8 z2Q4~w1K?rXCjqmM(q9f>!G}vy^D>bZ4(pSy~#C${}^VWi|XSkXNta~p8$NuQD zt#6rUg9|eZ(W=?x73CX1)aE_nmG{ytW*j)-9`CC2!$I9VH5Fb-7g-0eK(HMo9EA>Y1wC@s|476?v( z?=!n{jiZjw)XN>`3OkhGyww%H%xU-jGx{i1rrM@swcm@PLJ)23&b*JQqZR2c6FS0( zq*pRJfjddfGPm>+8gXa!qgDAKa-02Aw`4^alXCyrY#u+M<3#86%!+1X;blJu#50i} zio#-<32IR=V{z8mfEjwfW28+jtB@O1%9jKf$T6XFbJt*WnIDee6~Jgv>lGsN-_Hbe zQd1US`D|c$aHP1<9J?3dwY>!E-Ip=5^SVWO(v%Y-3Y+~Se(0MCOVM*cAq)gceQ|o- zdfX}Z7uHT|3Rux1-{bo7%@uOV!`a`+%o+J z3CSu8+FJlVoXOE3Z3&=ri(geOeV71S4}7=!8OZG_SCOYA!?4iaDTX>nTZUZWyhbW~ z1vK0ug(H#X)Nyspwp5N19(v`V1mFG~ps@U6qw`|i1bUcPn~y2b1-xm%>C_jhQ^$gC zour5)NI6=EjyiV^d0flmwJH$*6=lug8<@s1_5w)6BaHn_hX2BR7UEn+1eF>MZuuV{ zvJpbfoo0`LB%Iw#?f8e%&=RyeiqCp|&+h;$6Lsxh#iHy13 zfFwqTAbQiJuOFE&kXnZ1U=CyT3YQ)tAq%8YcQ!(wAkQ@zGxNp|=+Cz7eOIIR_Yy!c zY64+wm!IKmr6PYUq6F_-YKeb7OSfVyfhJxgzCOLq|4<+>%7pFGv#Ak?JV~NnQ2oWsR!*oi(nP%VC16vTylpL` zul_xypmPZP!P9by^ZF@zBmku(DC$du0qhEOKaw;DZv(@Q{1Wvg>Qfv=qss)9;?&g0 zQJvJb>X(M2Y(}ZbF{~x=*oJ99}(UEn2$@`fJNd>B9YTh z9_0S!p|hP<<+xEbBvY$?a2gEE5*bHaQZH{q7hS9c1 zs;%2wCi{}Scv<&Oj4ULzOo>xjdJD=x4p=a7eiAXw0izQN0BnZ|+iprp19B{#gy{JN zWW4~*Y!%F#_z4wJwY48swW;6iqQITltiM)c$$jM%U^Jc-98o(-BY#U8d$Htw4` z-cj(ty&=u3lWUk=(A9nkGCjt$3*1-#VzqIz0Ys)a;1qQd=;{Ngm$4V>)y28&&rjq} zadaBNTF$qHwD-U4O_OJ=48G69M; zNbvkoWCgl9@s=_{GDrr@fNL@9rmQgh@S7XO@{~6Fvm$GbPClPsRGl{&SnsekJle_2 z4{B>kC6nRXhgR)x{uD5dN^v!p_oi((3ndFFM(HmB?TwL}@`;{>JtJSZ_Op5mCI0G0 z?2l1HGtLZ^^gmrDmKdoQy{4v^tchrHn@(HKs83@-(Js|-3WTzMmiik=envw{jXg3j zI*mF%Vdg(@yQZ$QuqVly9FeBR9R;A*o4ntFx){)dqqrx3KYvB`rg3lFhhpj;+gbSe zG*pY9z}wg*mxlE?d0Z%=q0#}$3ABp+H1i$7M+Gg)O-2kAYXQ11IT*+~LfbeD4Fvwt z7yOwuJ@Uf&BsG$|A<)lQ$gV}-b8hfuM2#brbiiXkYqNXk9XoR~+BVq)#aiZ{bcomVQ;O^)A$;G)fq?FOW_^4C&V!QgBHFW0+AobdsvM<$2FhhA_B(;@ zD7OgHrsGR{Qeq0fbb?O)MEKDD2_O7q?*)`vJB5&8*mFVw=5@)y+r>P9|a zRIXY7?&91}oouofydbesEor!2EE^QdD#s>=L)`G3VuF0vD3XzKWF}GEY+Q!c_3Ga^ z8*}~2&CzgpVt%K&n-e79!0eE&D)5&6TwxnHBJ;dq*AmsoDc-mQJkaFjQ^opkR&~FB z<{JnbtrRy%t6&FnFZl%*(*iYPP=FE8RC6MF63ElK;+b|TA%2lDdA}BuQVw-jfg*_o zsmhDf(ev5(3;pwhRsJfZ*PzFbb5)6(t@%aRgRc}y<@-^puv=sTzCBLOf~RLXYfuXT zKl?V;gXpy{7ToM^KuKLG(sW`o|0E3hG-S~$2o`bs1f(_4-uc>C$qE-dD2pqMzuwuEg982^$Wb@VIOu|@ zvu{~5kW1`JD*{%u52p$lYK8L5W%v3m>VqD^dr&XmrI;}ChMC?*oxkP9_zQ7)3%0PmUnH>#Lc!^FuO-{dzW*Qtbbf<$pJsAqB^?t(Xq&=n-x=zbxppp{br)3)zr{c)gBBaUa9?5ZxxzHPRBWEI5QUg$P$P3 zhx5!~tUOg0126clIHPQ@jNbql~)4FSV_9E00MT84fTb0)heF4xt+POc+AWDh_ZuKZ40&v9*Rd3utp zrb4-N%ayBPvBWlnm41prxg;V7F>15ke$2A^Rs{+V^ahahy5P!|%X4!uD1X$SQ0I6;9bVqmWu=Nyj|ChJm)c31jGcsK1_v9%YU-UeL84Vf z{%9U~C5y0N0RckCFX~`DpzB4tLF)}-=sf@N&qASIj9YA^t9WmAS|MgfDOzenkHzl) z89p?t9DJ^d4iuky%l!Y8NA>W|JVyup8umX2W&d4}`$W0JlL>T3U0{c8002+EN4+p9 zjQWnOwfeSxvN6^iIct2j8S zUxMn6|IdcbtegF~WKlQjp+-wHoQ52e_+O!4ssu2QMRQK&%H}mz z$%EpDxB$tFGVz;F&A zl6vdj<%^gjj-OeNj8`LXop;@)G0LR21~>t&0iF|CARPfO^i4+K%?}6^B+wAJET>O> z*){&vmJO1E7Yl(al6#hngx7gXfe`E*2VxIdoCpln3CEdvHockXJ9+{d8LbA&e<=mB z&h4sR;VH5$j`Exs;u|!N_^(Mo@ahrO0lU|f2ro*&da6#XK%SZ9mN6jQABrR^hJcDI z5Q7R)YhBKZ%P@-Lbu_KuqfSfc$t7Du6@a}-j1F&9N12cBQwP4hcvK*9ml>t^Roep1 z6b@F;?u1HMkORj9WUKq3K$95=)E{-!$szroMAN>579}U$oLl;*%=^wq7yleFapEA14 zn3`tGZl2$gtLH`e8%ffSP#ph?*j4U}XINg$6$>@TtZYCI?2DGIJP3=#lzppSBFjbc za5ItWXI??Nsc`Cu`zX8vT2J8`OUWItB3f1S`*|-y-X%MCFZo}{gHL>)sBBpZ#tC!I z9bmg<@q$~EthGI`#xV>-RK1|09;L%>ZuGw!hhqEl)bt|9XG`t{!NkNc*O zs{Yi^0gm6u4c+Z~@Q4#7%u&Rin_No~Q<`=jiIH{gf8+cs)Wu`m$6vIl-)=Vv8J;rd zeCRbX1vPPb=%Ul5DbGJFn2A{d8YU9CjSL)R_x;TEup!Dd^nAlHvg1gFJ-^$iSQ8kd8J(ZP(o>RvvffF|Hpb7ZCWcHJ5uJmy#Fmz2-xY7T00T6|1PJ0 zEM-(3PfF0iWPR|CD6Gt!Uy(?21{|i@o)NLGk{7XkO?|OheRs5c^UDaV(id3mSbn)z zPc<)%;+Kk@kZCFcFtT3|OAS&2E}V>FpfA}ljD3ySNg3N+N)rLrW^W=ZY5=c{`t4q% z;Qk+5UmeiY_x_IvBA_B5tzZz+NGdT@1VLI-8j3u_R0MA&_%T`veK7N_a&`e z(#uz{*jwQqh(mANf@dc{>PARAO{6Z|yHunKKH;A$1)K@}mC`}K8<-FG}1xoV;LcjVQo@W{90WWRQ%`gmdsvkOgf{_T}=_mY5vLd5nBlYOQ zrCt)X$+TO8LMLbNGOm;F8B+*az2U$w5b^#{?AZ4Qf;P&pz$ z#*$$Kc?~w>LY$4$W({m>Czo;hv)(C27-)q&EZHm;WSXrt@H$*@6Ac4ku}t>TTW(kV zRV}LQ&9$**-^*$xfwSmkLX%#AQRADL`{Hn4EJC9w7z~fsXIcPm2BG_;3v1v&IijCJvrxC<#Wy# zu;v&aGV0Rg6wTy)WyJJEML;Ck&}OW6D(xDP5i<@CJ_Jt0w@XO|`IB5l`~{wJ315G@ zI^kJb37oZY)2-gH8kzLqv;$7JLmk(+>&g8|mJYBqu#2wtSxeotWvxj?PRZJg<;|h~ zS?{H=seYaDN6+?@3oxi=>7LXoo7W-U6Xw{hFX)W7k zt|J_+$V*tMgyy^JnA27|lC5D9hKvk|b|-q{$lma-Y5~v}+pf_KkwF&)i7=jHPT;)0 zNu(5qSf;aHNj-7>guHHbEQF_xC~{J>pd?TG!nbO(JOC&4Kz1!-c-z7_{2n#;a1;<7 z1~BS!>CE9KjHCZ6d|klFQ_sEA;E3ZXulAQPoPnx;c5omDe82Hl*pAw9!7OL4nnOYN zpH*19mVS(wzHRI7{^`MRv)+w8!8a-^gXcrdyBc1|oa2Rk3^#WCUQt}leSl7Y)lyCS zHUf^)}_!Bg3xtSJTB3@Z*18b=S5EB9(HPG1hdPzD?-&K}d$mgHCF_yf& zfypl(7A_%oH<{4PxjK?uGU#Q}1g<_@7e5hU#J9@e7~kh^dIuQ)N|jzDkr7Gt=W2;k zB;eVK4)R(@zsFxX=q~vN(g>Y#Di%no6sTD{SY|BF`iop=FmbVK1UIQ0UgDR3MGP}T z)M4kkmQ{y1aTdtul7iX#9iL5!yC+{&4zd3AdII2k=$u$!ob}lt!HdcL@7Q6-LHl1> zRKDE1&tRQ{m-|yi`bUyYhhv}IK7k+Sde(S_Nk79=kZ^S>bRFv_rH5R`5S6$X!oNu4 z$QegG@c5giKHgAS-`TxqMuYF~?f!&FmNrrQtf~ABVp)T6eq1VAvulIYnz8}^=bg8@ z_OG=Yxaj3laTTr^_v8+TchI|ppFZIdI)5o#-^l~Z81D_TN=`h59J7#RqJOZ@g4RiL z&6KA*SI8i7#s@Rp__M3 z&YCM`FBtrA_;UWUrwjg^Vi8ohC?wcX+geqg7f*pG$YaxZv&!O?Ar1v?ZU#C%CDg07 zfo)aL`fDVB=<(rV8?6wOY$Ex2bNuubqiYu7AjL-xzW^*J=8!bU2dDrUtq1-m4rA&l z1R9D@j({8K!S|`(nYCTaj5yNAZQiZ$r$4*?iS3M2i^mD^%o24R#cUV-c?~3QmUN%p zlOJ^o(Fs82kq3rV6q_XgN(SWFTYp1LKIvQ9f?bf0`d5tZcx7mtsm|p$3+c8^nhri> zkTAYSTEbiE%i#T8WJGAyuFcUsL>2&GnChgRY6zcm96r#~eDkliTvCE0;=5A! zCJ1!~>*G;ZkJji2o+$lhAn6o@7(l{6wFWOVcoRjcRx1g&NA*PgECyRKD^|R^4(0R3 z-Jd}YioKYQ6Es0FLwSR^1!iEO@nkhjizp2(NC(OP4EOo2>gZkRKhVLwkd9g3r2T+GTX2mZ_UC#vXiF8%P5MMU(O%?8?c? zi_1Xfk0Q?1_<+omJ_N=EIKjtdf1Ti8d8Kpsv~nl%8GT;upX^b!-Y5|Z_7@}3D2RvF z$9S3gro+~6bHlLyMVXd3lSq-lx8IBW)m{|f_XF&2k&_`I)dbeeix1pWN9{s!egX*b zc3<%POBLw8PiGTvSWi$rm$R<1$W0Hx1gLB`h0vs{r4$Dg=kU%XFf_@05!t0b9z2;# z+b+EUJ36*)0r+yR;keGe9}Flv)0Ycto&omo2l;>RkQ8kgek^PC7b{mpO#JmkJB-S7 z{R(3fiwJ(%MQ{W~Pw(?I-yDIlJ#2e${I%X~h@$Q;I#b&DWMp?37)yZSzsKqp6qV+D z%VVwuSi4C=>EFl)(VB51>pysHVUK8p$)UZzhvySg^-(dK+}otfy=jz8YX$A%;X5+` zly)6++0~Y%e%GbHR&j&GXmTXDy6rvZoRz6|FL}YBN!4t;o26i$CX;6O0l;?zz`BO9 zrORsovV0vVH(sxK{1Vi|8uob7qDP2Ofvq>VsQw9W-w$^aB-M%`#>so4+)Z~Y@4043 zn&4Car3&q8y7nX^`&+k|BS@+z8)d2yjQ zDzz#>ZW!%RZ<4@DMyN1Qx#A-PQnlgM@=lpc_f2V|>Un4}#`+Qz`}S$p%7U>QA)g(6 zQ|&B~WC?>K{O2sf>Tz!ecs{?$2Nw&t5qEO4Tf| z2)bK3ZyNBfhQHK+!EdgoQ&L^Dx2a&Xck2C39ofL!rA{(O5&$#YuBy+4?y>z%USI?L z_wuRh<|w=UR%Wl)6nY5pV$i;Q`I-i-kI9k5o5`)~^HSU#iRsvS4d>YgkNJ9}-yVA_ zgrv0AeWpZO13Z9^-(A8TA$n&$^l`R&%x8|Pg?b4Eq$&67T;@%;p^=b}3B+#_bq1=k z7d5->Un5ZI&7lS_@+OPMesYTpr;S;a5*Wnk0<2Sa1@&B$m_}-&5&(|M=bnDgK_yq4 zna&rtiR%jW&UVkG&fA{z@{-5a32oJTJ#&G)&PAF1-Ag8@moSoO z-LLk8OVum4j!shn^0+tiTLFX#dVHF*Qf^cX?FDh!L~ZmBSEYbcWH|i-=oRm;7Vlt= zb&{!n&r&`E0S^~gKqg1&Rj&)@!3)A6cf9Q#316a+irV_GD8HtF}DIcymR07DYzsp|N@$2KX7~ zS?tq)=6T6<-FbhX{Q$G>Zt#$RbN}Qtp&5s$?-897c@Y21{^^$|&1s)eGl(!O`csa? z5rae(xS8GymHOk0uJD8;Tu#@tKSFxRSb7j5buFu-ZdW@#7x!g zHtS%&H}^F^=b+<#Ho`q_f1eH#E3_d&EB%tnwx02}-KwwL`xqSXx{Fpl4(2w}hq$PZNv1y7F=2 zx!+c`PWMpbg5?LTjC2|;9@u(gx`-dP|LUg?rkdNQrx^jkqwa({($k+`A66X85`F^1 z`ZlWO*23q%D_5@{p@}4B^`&Lw=e=-bp0*-mhG2!2-fB|iX)1}C-QO9w)0041neoV$ zxa9YqR$(r`I#20{MOoM13#?SZBcwsP=ouG~r1DqThcc6{(V5J!2rz)?>VIL^7er~^5y3p;$m zisRw=1TGYW)URx5x)FBi6-aZg+#1Z*t&_XSWxkCYZCX^Hyoy=2f6Pr#Q6JcZQ>)Q!Bj4ao#RBy~ zxnbxkVod3~m`aBrkuD@1rIQ)-y`+)`Iq?3?mJbB5?hgzrm70w{YFhZL*0ien!iGV` z8v4yZ;88Dqzsd^m`t1lip@(m;6fHj=e%v&fOjv}icB~+b01pQwFoiJ&!$I-?+O`kg z3wSB<`mrT9y~6zpJ>kzLUD(TDn=|`0`+x2)GL#8!BXpIeADwA(S7(;zAHP>lMkwc; zj2; zdz|U@@h5{p@1Gn`yPU zOKcV^cm4K*$;X97GAWblYSMNxszqTl$X_JDc@~ngx&=<{jdm7;!R&VGQKJxUmw_y5 zsKYhyU4zs?m!MK zANY(3_PDdkK=?L=HHD3@xZAetJIr$MqVcJO23fc=s$X}|$<#-f!LbUZBPK+V=I+}= zKvtV%)ZG--BUUp=a3;oT?{^<({P8m19-D5whBBl?@HD8{RjrMmkEf2J$@M;kSeK`u z%6;~qN=Jez$=Ab+yT03l=*tSA7sZ9YzUk)JJa^R}Npv|Bkl`ix%}LK2I?C?s%FM-7 zFF|2WFQ7b&+=1?{nT7l%XMCJ0O)R;*R!O?_jd80I`TardbsBdr?TvCDdEEGUv>f7D z3QslSU3ZU=gYE|=zt5t2=oMxE)UOH6WTrGj2a7rPKA6ouxLh)x%xZmlwz8!X^dC0_Ule4VU2_Vd{n8g#;(7df zk6iW=$oE(O(SQek{r4-7^LhDB?)Uc@J@>lBPak2fGvf+~&mYoBx$4Ujp*AXtS1Tob z(VpAvq7w@Ki;p6~mmhpB!$E`O9m#x}2=mh9Pi|hz$=gkK6}Cl*0Hbg_))5|YU6zh6 zF!dTU)5pcDxQ{@1g12tls)&?#;$1S67o|<-D$~d&Hwlg84##^e3}e3+9|EmuxLbw4 z27cf^ev-hXSjQTeO-$1|z8Fw-6!9qq4}wA$bn6+^Gy`QG$oHVkgM zHT*k2CKAEe@j@&je0=4QRm7$DF5<}l%-z7|9J{fbTi#+l?uv(p+sNm*{k;6>g#Y;Y zrg~7NGZD-qrGoEhPdj~dXl;i zSLLGad#88qzMtp3u;;~zi)vFV)hYHY8NBYIuU|D<1B|#svh)f_%zb>26kdS}K4v zCa1sR^aUq)|9*-ub`$T(K;oR%YY-N*5Fu6f83Tc*A&}ef+D@3J>y(%Bj;(ND15rlE zqOQqj>H|P(aF^f5i7)QMusz47DaXyPUFsdUMCP`GBb;qi*RgPo%)4?Y6K+ZnF0adf7l|%(ZJ!J8`C$aGy-0#XaF`DHP!bU zgH~hqwyl*oktO0Q`fy{!^~%3x7x*P!BunIZ@I&8I{D0pTu8@b9n-Ka!>FWRd-GANs z<#+!7|9^nFtqVLL+#vs@CcwWl7vQucK%=dC?jqutX!aK9FsY9`2W9Rj#?yTPxj^>+ z*qO6y~k0%(+_kFdH>XH0%OZ4gU?uDD>@E8~F00vorDU8^2+$EKx#(rrsN3*2r_2LxGoE-)Em0ZU{UglXEO@g$%yZcZmoeC~xOk`GQU#wN-qH z(DYWcc;n&cO-s}^Zj#N_lKjc~YYqG%r74wNH7e2m9XhB2(Wka=*Zm`66tP)1+tp*n zv;Wr+%6{K#J*`y#z7u&Pb$6yV=>J_%z}4#vLE611Z(ie zslR#dYD=2?Yz$ROazT7ajSiVe|A_L1Z%tWzfv^s2RBGFD=KT1xj(ht(Qfi{wT3&@a zpOIV`olPvJG8F?pl~j%B;$6C;X4EfPD z0n^grX9k#0hyp}%ITw^R+ zZ&e}4=`V*qf@D5Ke)SZ7yIXZ#r!fQ0e*}20>&q^z2_$T;eoOo%aLTH^XRF|$g53Xl z0T$c^TFsw?E@Hd!a~*Y%zpiKRcwA|9ytv-FXK@9!dy!O#ux+4Az9({5Z1W)KI>feS zGh6iy43pkivOnoMiLUKnHZ;2MS3#aVc?jcjPt&2H^lLEbc0*0wrS$W0OaH4L{`iUJ zklqXLhb$S;Dl5+uku_(kxq%fqlQB7)wI8;;&FQ@cLl<#Z5(JgfL2CIayoh1V00<`8 zI{3?r1i1u3yNnojo%~VG*%RwfaU>PFW4GeES!oXlE>;%YE$iYmK<1-`G^G%Q*FsPQ zG3*$A)-luW9ZW8GM#EoSg|+9x(n)_m68y^{3A`>-#8oUncvtt4>0X&&o4fs{;3M~_ zG;^6PGhMCs{m*ufe>>mt205||C$E&fWXaH#F?p$LN|YWd2b=liwrg6X=>D={!i~{; z1Euc?o4rfviUfd4W~QHs+Wf6#Y8+iJbX46_q>qRWiMJmt9>0IM{3|^d;efB0we^!Hu&-J}w$$?6QEqv0!XXPIiyuDB1Dj%guu^$b!>EnY%2Uer z^e`97(_HfTnZfXgt=h&F&*A9p9Z^b`vU~KeMH1%@`gWs~bn=%a=~_#$oP%hwYHmM< zvXsd?&h^O!Fn8iGctI$Ksn%iAslHo1&!QyL#k0`uP}L3027S;X!s;5$>|^!4d!#wD z%%r*rdE<+6u&a+`VKOlc{P!)>3&gKHuHS&3Gya&((nbK`8(_32nz?SFzc+nVBw&9j zSrw<6#q+dT2tJ(%dqN)F&s+6GuX^ewbH+ENryuhbYIAEibjTBIIke|HhC)JQni)zb z+1k;*2m4px1qzMe%_&+>yZU4vnD2V{-30K7$gH@bFb3T4IufkNiY5vM?~5%eBFy~! zuj zL@d!<@I-%7l6{D||6*Od{c>m5ybUE8Dearu^dwUk#o`O}DS~8S9mbPMEdMBcgh+eIxK#>b9=Ak|uS8@@f)^Fz-@<_trNoJiBUR)+OHdfvvY;wp-u zAnF+h>K3wTZyN0?NgDIcB1tdFMa}m1X+-_svk6n~vSCm4iW_vf{Z1 zR}AEj*@yUM!Y0CuS1YeZfsNjoGi8#VwGBO!hk^AK4ks4sm)J0|c~2Yq2-pmt@3G8v zY@!zLq^cYxFWP{@;tFh@neJ@W+G=`bFjZ~0{~$FKfSc^~%4;9-G@MSV%7PqQWxx}M zZcU?zF@&NW7?jbQzhl>RX<=yNC52o@f)29nM9aIo8$838uI{4tJ>)!j+JNN56-{i$^w@e9qLt^N z(L&i;P7kILBsz^(mZgtw#Ta~OG>NDdDrAR^fL?fRzYMP*o?auVKXvW^q6QtfEvtbXu=f{JzKFt1vfRt5Zl{x3r-WVO;mQ$tn( z_s}Vse!n}Tw>N+1VAc)M=^X^%Lub`7o$9f@1(T&iutf6KF)O7_otQX8(kx&_bT5K! zl{up(3O-~jlCkK$uSzNIAb(JwKLlmrUKtn~bQ)GdyhP%$%7GNIdx3*)PPQ=}_`~kr z>*cEr><1Sn48Jw@S{5_8B+x|Sa!<6r*06`R%{2Mg+lY$DWEjqCVozKYGOH)iP(`~9 zi3(w3SkN1oE31}K8*!_SvNjKzIorU!<5edAYwYF)AYVX;%ev<}kypZ#8zg;{=rhY4 z(I(=Z&785AlB@Y({+q>P0sFP9oIfV)?3&V4c)^9d%K8Pw&({OaMpwI$zuL4cpY3UpHsv%e z%E`QWpij^9rRL-0h++g3gpF1nTE$v1PAVlAAZXS(HqF(nyn2;UnJU!T@v0V%x0wys zU(J3LZ&qe$Q<}s!@OCqAc-f}J2p_!usc2>quUC*3@nKdqh9P8Ec0)ueD;m;4>EWX( zlU@m__1YBNHksLuX=5GJNI(MIeSk5#bCA&C7jfn<+GqAL*1&yMAdr)}hNU{?%Q|%B z&xn@(JLL5Hn3uN0!iFO=gP)f*I!^`7(kIVWX_^`L-iZgQpq}atMjgAW)-J#&k7Pm2 z8IUjV~$1lmI`~natFzQs)t-i{F3Vy)r7MRtS5Yz(?6v0KC8ZGRd6dV zd8^?#x}R88u7x>im1Eb+M@YFB0l8g@uIh~hn}(dvHI9-9YP>KoCCqGfZPQ0qYq6$6 zFv+?n=bzb}`waWG&2^J9p_A0ZCZK27?_>i-lRo>uLW*ZY*UBJ@_lYO

_{H&L=L7yDDeUB?y>Bzw%#=WHwHHQ}xR0h`E0-FI_(e7p3d+mKh20 zh)}2hjC=f*bK77R>PTQ<5FIP<9GaOrH?>wQttN!)?(ma9mQJt5`ey9w=&6Z(-ZHvZ zrQnO#?3L*MV?ZiGsj5Y!XruE!_<{-+g)eDLYzRfN?SV=mIIrbZqW81RL$QO{o23A# zaqEP_HYrs989yL96hA^9Wmd?-rjVrL^L1?&f{&*~12^T1S58M80SBg=5Ol9*!SzV@ z{FH{hHc|gJRKpO!E;I}+dnyk?YRRv<7}AY)L%$KaQR+G>+bQgFxSC+ht_9LmzLC8A zO`J`(YM2?zpC?eu-f*!@yL{I2(AV=oNK0!tBoe%i;T7}4LXCN+KA!pcs?BdR#S{6t z8=9bWtl6bO62Guf`DwEfCx@Rv!M4@cC^k?Pnj6|FSBeD!FZOvO-K1o6fM31htgKh{ zD>b@I8n!Dvr($<4c%{m)>0$#ub5bt~+SLI3m?MCqOEQkQI#-%4wht9yUb;HD+g&-4 zPPr0Z^GJTIeK|gXi*kx~(e>6@YBS-*lCOs&FQuHo7VDb6vs+xF>c_jOOfG@#g&JOMCnWY z5uFx(Qbf=O!;YHBYBMJDDiQUExMorx7#r0*zCl`e$K~se8r4kq<5#!% zf2X8z$JW{h2Bld#@A}A3K(W`)OvQYgpX$uR?=M@LeF}pJ#9Q&PkX%6cCmD1UEiN90x^$fLt3)}5nlZyr}1$k(HC~_q%2!! zqPHLBWj@GQ@;{ySfOSdepjLK!!PPCxZkDs$+LMKs)y_yNslnKax#5oQ&-4bxQTp4`O#N^6q4+^EZQ86oi_Q& zW;BgBSc;)(xZ#eh#rG-Y!5kelo8iJ|?4-Wv>zu9IC(T1wy5~Ths)g-YZBOlzYE0Rt&mN$KTvg{-|79$ zu*Zrp!fncy_J#c{sCEye@^ZN1qvn0v!9%8Fip$!8yL&M{(ZL_lddpi1@aD!g`PF#0 zOG*27VO9-YOSfYbxGd%LeAZ`0cw&+&M^9m2et&UQCC%(wB;7 z`;VPzhO<-jv{N{2@}@4dD{&S6Cuq(<)yPs7UinRgRubvA)g0K$tRNjS_>MWU$`DGf7V*ctua+=HQLX76YX$wh0AR! z+_WS+dXP|rH@F@!HzNWwrO70rV1y&{rHL;_>`~va>dWN4&e7~*=C&->z2&BQWMkC+ zOZ$Me0r(AnsB zjKA|4{r2hk6UE@^1IsFh>7G%=sxbu9T^Qy@?7R~TDU7qAcUYDowk#1U@pb(zZYCn6 zJIq+zwahyO{$|r+mR3*9LT>vRG4T4}q5sC?3$LdVbJ6}>6kVikBW>YNDzX};czx=Fsd8j-?5|WwJUx5DDijmZ>E(05vAz*LCoL$+d!Bgc#`FE}exXQ}&2EW@8_wq6Q z23&&g;_6!dy0R_TX*=ex^Vf~n5mX64LbAU`g+sqB378Bu!Hwjso}f08yL#1?Rr%`J zC=&H8fhzLCedm7Mo**tttSN&S>t|r)moJKwkb3(5Yq zFGUexqxwCl)-eMI@IoltJ$qbltpR!^=LEJMYp16({5)#*6xGx8%+9A*?i5uw{Fh^f zvk;9oKvP6da*VRJ@**~UwX1lGR=!p?;A5K%_`?*;BFJe@0*mh*SY` zq30^0h2fBzh)WVbgXzG@Fac>(nMeVH-=wB#d1RP3kU*1J#UH*0fuCfG)-7v6Umkrz zH3(1=YP|7k-2Lz&{Bz)Ip%-OB?1n0UkD?06e^wum{(N`c938tCvSguIV9%S5{*#@8v*0 z<}P{kH?^pGnsp^?Uuc2!3f(Wwp6?7U_$+oWyL+h8j3ocrPBN z`gn`aemWr6DBa&j`Q}K-Ddn%|Eyui(!jUL9SwWr(D0ux#S4GiFA_O;_Jfw)JxecfH zYAOeO`KtYEE|sDS9zJVk@eI7Aru zDPbS$+qH~pRxV71t-_Ul=OR;ph^2P8gzy2OI#2ZKgwtKCgrkl`wzXcG=>-!j&>Ppj zB@^q&bTb5+HK%q<$!Gl2p-+$0<5}JPnp=qc7|a)I;hX9J4YyXIy9pnPVXtTHEVS3f ziI5`2AR9J=FLM}qk_t`<+DC-*3ARysiq$4926Rq4XkU_;&75qJKiGP5m8m47L3`Pq zHJ47j+c1`PME;6`I@nTq zd)J}5Peda~DXkA0#e4}oI*P>Qij4`MwLF7QM=FRBz~25vLTeqQs6+nlft;4gz}#Z> z!(uSfg7hloKd@3>);5Gs%*#GhSLJQym9)>1Dfxq4XGCE%cp0^s4v}2ot`1|C>z4PB zZ$n784P?@RC4G^U$czS+h^c%#O%>(ygj>t?^sGC|REugfxVBY78bOjbsw|vY7_shj z5~g#dPztZoX97p%|HfQfamd2|U3;#j1+e>r=T$_MC(6dXLwNn_=rMF40Zy(rnB=vwEW6_&M%$%!dO|^?R zkx|1G`wK++pSLegqz2-`CyhvH0oBWb?GXVlIYL;DBWm4#%!Pse3^m z2qVMfUAFtwFKtsHaflf8)Z;k;9ctwf+F0c2nVdhrWmI)mdmPx*HyRcjHUHChf2pbG zlrZhXlv(IG`pY*g;ZnaoA~e|*w#x7r>IMQbXDyH3sTuL@^Uzk?9L@Pqy_DX+&08Ck zYIwm4>K#687QJAof~3>^*s&qs4h?dGW*d8~h*O&~pWh089+Un&)k05VN;|azxx~0K zGZ;kPYpHp*{d0~ozCMtwKWP2@cg}TcvS*V0yuYFEGE^oTGZv&~gG#+b_JX z>h^pE__?;AH?f-<`2sJm3U2+9fH&l(c2!% z82Wk>c2Gfv9cx#cQxeM91L`JnLO3`Y4%uocDFSBw4`7d2w2YgWK||aKoqiX~v4*?F zvZ6W~maV?ndilylobH7wSfo6F%WGe>r z;wLU>W=O$!=2bq!q5iL=?A44+|5ql|vd0YL9x!(Q_?%Nxt`sb()Rnk%0}L>89M7CQf4IQ zXMak5KK(JFGqxJpCUgc%gX*2Bo5#y4;8MM4L$6xqnOX{_RLF%Kl&DW=dq?$t0cqMv z&Y+ZBbX)=u?|M%3NNFKdz4;&5QMt(f*`KJ26O(y&tJfcTbI+WzAC- zO3Abf$+kf77B7mjHWA|cOrI|gTb}V1y-@tpVW#59zi{)9L1#gsgzTFd2GoIkNN`HxsGVBx8Y-sqOyW{b}- zI_oiP$#gfBLObP*)!t_% zmOa*O@nmxpNH04U)YOC`9ruq=o0+_`4Ir^34r{DGdjaG4l|*sV>noth;jEd9b0YA$8Jv9iS0axP=hchhcz+2>);&7^!_5S2zsX^&F73dV6OE`6F~C?>;~uL-hg~^v zrfhXb2Hj4*vB6o6y!X(Y|2f7E8rNspYNa}!K{XWV&dceiXNu`AQ(JsLQm~%Q@OXnh zgjJABT-y-|C^B)$iZ%s{qR4X6C$Q4J(e{-g)My~W`Zcw@h!0T}>*H-)9#Yvn1KVHt z%pT>kgT)!g<8>&B7xWpY51OKnQwgQ!pQy~UB!;xe!y*r{x?)6AQFP;wVvMs7zzInp z33Yzlh=#n1vzE!MgJ&2=T-QAGt&0%WsokSjwtNZ%F-H%+>*SUk^Dl(Cs83Mb)*UrP zaKQPW>tnt%9*Pk*L(u^?S6*P#*Bc{PMnA-5eWwHu-=*Z|PcX$x$@zH{JQZu7nGN`3 zHOhU8Izj-kcA~G2Itpz?WRaY-xit}^r#NH0jw^@YKWH=U_%~Ik|7TJ%K^HHqp1VcA zFX8z++_|gagFPzf%uD~l3gh)HPOU_9ye*K8@B?Z-fXjH2%aQ2hj(2+3ahHkh^TwY- z9sC=kzoGrT0>6EC6wlXAasIO_zuunCU!n3Y-+7(*p;4c1F4$A@m{VhH>76PCsS-Xl z>$N@ZMJCV+Q<`Y$7^Mt-KSaj+>*$Arg8fI$yW5uaq8H6XiGvMUP**O1U-X27W+G0h zC|TakIez-Z2$Em1Z!J>nHLgX{$s=D@!l+h~T*)=tb4;3nfAjTb!xI7#@1j4jQ(S8Q9@q9F35=^7-*dIP$+-d7 zJG2CFEMzea1p5%3%|Wvln{B!&PV*i&+lFS$znpLI7gQvwKIt;7QqPS(VZE4+H869s zN8ub*h_gE3Tgvh%HoqhJz(kNL3O<5lQU}Z#8!a@jHY!6mt{iif;qRch-XYBwAYkm{ zegF%BtXCp96VTM}0;;q84lC#AuWNb4BHqB_t%F8QsMjT*CWx?48g^SfvLF`lAcQW1Xg|JE;)s232FMALZic$tAXNb{ArJnly`|X~8z-R#}?!Cb9gkG`CNmi{j zRXd{EBA;~TYX5Q8Ohm1{%FLv%Sssa4yxmQdcuHP*O59;K*)LXo`KcG#k6=m;WWucQ zoj;_3^9jzLg3|uY7oumeOp7qOT{v8bhy5P=6VA}q<7CdL645j?%Xz9pm|Jhz12l(R z>>#)6F87YN-%Dh`olFO&m;Wny-1V<;gVo_5-3>viVqthUuE;LXr0WOW>JF{tpiw8u zI=5!B0E0(5XP&>82allL8WJX;*0(y8$hhC0O?X3TXndwAW^u(y+#13ExbzO%=PgsB z1TlZ26FzlG=U%rd2Unh9x;y)d!seN5kg)9KNA%dHa+K}ITKz8T1cdb9j#CGoI5629 zDN0m%kGGv+ISkuj!ZjoI$>8cXy^Y*k0woMULMFgM&W~ORK}Y`x1Z(oO;vA^@GHy!? zzQ_G!BSl|TI7O@L?167e@HV;pL-7m1Cw4$8G2otgforQ3mu9{;SapGvDEO<){Pot8 zbmOdon*3f~oG!=kF?^YMJQ+4jKXRIsuKbUGzuU09m@aFje&LiidWZXIFS+^{tLwJ( zug0?8Wx@a006dq2C6~XNsmYlj&_60`J!VyvDxHN@3y(PAEEbMU4ux;owekEP4qoL? z%8NY-ak&c8b?S7x8d-GF#$0&aByFPH#A8*l7U z(s5@B+Oeks9;BD)*8AYK53}{rJ-60EHvQ3qx*!98?PdG`;aiK_*CW;cDor*OhhM2(P#I|iPxzq4h4;V&mAx>MY4C1?FYv81uH;vRjStc2%h4QUB z;f}|jTUmBZD0&0O*BqTeOqGO6FbNNOLGAb90aiRZPoEFTf?B( zv^CAw+)plIQ+HS!UOc4g@{Q~;dD&vFG2m2bT;@FN;5xH%G!jl1%XLm*N%)o@1Pq*o zH;{8bSb1o+J%xJ0J$>mJjm7l2^M_63k2P`YN>q*WkLayh9;3zHn-e%73lK1#N5&2W z-{Xo4^QTILWzm3dpFET4$N-d~^<}@5c8=kn7o5mHn!3F=C1~`|9m@A^48Lg1B3IAs zy(hNknQU$}Mg}7(w>g@02)^(Yksj!oVj9=!@@+elnFthl&GRimW}xJyeQf_He9#U) z=y&MLCoH@*-0OyCxYIH`rJsIH&gh_i#plr`@dyn5B-jGFTK@_YaOc)$I)9l38+sKx zljIk9u_taf4P`ySDelHhP36BG`0NVa?4V%g=pa@z7U z(7J!u4+ z8a%$X8D{Z}DYrl6vjc4n-?L$*;hX<8(8d%X0sP}Av_HH6qKTxtsN{=F{$V${^V(o2T7r^;P#+wR%=wyi@Rx_6dy~C7Og%g;o&mC+gM*tNb3198gBoPGm2k}qD@e{FI z0VG(Jy2ndi7I5MDJjI2;r0f%Qqup8QvIV;?&HF1V(bslvuE&|5Gt``~a$c$x`&;|$ zpzzl&u093fq$(pE9FB1v`WzQg55QhI`c9(%anxWY##(eC{gZe=+bh9%;4*xM*X7)` zNzdQUJUp_U49rFEO+6lTJEmH0#UQtWZox;d9yM57A!weT?dV&%a9cIvD1VrN5qvw* z3Kh6yE=RGf)e8&7DfC@@-+Rt^tRK$`I>x=0$Kku|JNR8ck{8srkUiJ)I&d#+Iat*H zJn26!VFA)#w-E1}&klj{|Ht1cU93GBe^FZ4wU}!T$Q^A1fhW+{%^SdM{zyGVkU4FE z_uJ_8$J#DDR>2kRvDG%T-eYV5jR}K~gq$ zco@`I(jxFHX|fbKvR~MIwM3IhAR_6U_L2xtD8PEe!rSKOS-w#_eto{Y`HFHG2lNq( z^E(ujtI*E6Q0BFDIpnV71NyJ4e$^&mNK)=tH0{)7n%7;h3H#Z3$&2nP!uJwYl z@r_}j=dz$Z86W1qn(hDnPnPSJ0bc)o5NIjjEhhgJohw8aM1>2o<_2h??Fl%t{VX8u zYybHF-)~j?qe#b+IczJ^@yO%supwI5H9#6&U}jY|R~L+Q?AR9Su)#@t4EibsbnIB- z)V#J5xpfbX{e|4>;?h8YHGU`5A8ER8e>M|R3|o}@jK%p$7wVaR)UcHxrO(jm$x!FH z>|qix6I0Y{%6nf7y(z9~Ub}^KjMtL$0S$H>v^WuzPPspZYdBi(mwPIolXSN?#oXFj z8n0SBDo(q(s(Q0G%e#vlCUlg_q+6~2y={9!P)T}h^PHs7U{o-1JC!(wL1f7L)fKY< z0s5C9H$X;rF5~o%lkZ;vSxzlfzj$h_HN0pfIGgX`-Go0TK;6vXH2CIKT>^^-0IZFM! zH~bcQP*F*}nOYf;+6w=&HFfKVWJ1U+l|s?y5kLq&JzNVRev5wmmS;J|87OE5nfE(= zrdcDOP16bZ&X+0AOXAGi4G}bIJD*orrk9|ih4t@e@O)-F%<2$p>!O*8e$}5|>fW^RL4B=#`|g?% zuX+LR!t3cr{s7_j=M>m+mE-bm{;4bOJ!@q4qMnq@m$&$<^ecFs^NQUhK2H_<*6Q9L z`Q0p;MD-kb+5H|4zc@k&9)8G0k4JT%QM-WB9m>i_#q+h z14p~%P*Xci<10fZ(&hK#G?wcW2W8M{LG(h$jRoJX$Qt9hx<4W6jpiR!04ygV!gTdd zP^mH$mq)v0@Asxm=j1ovzI${JRkShy@u9gNKZhVy#>4gDO zyDIAXZFwrEDO1->A1Lih`@X3C45&7@k|76LB#1!V_1F6~Ah96e487200XIu+Ye8y1 zKts(bVPELbw*|)(T15h)+x0NnN|J|`dZgn05d^Od7btK58B2j49C$Irw~F1P4Osgy z5@O#^9be8w){Fcc(EGruIm;QQB@fIij)k&bae9(Np7Nld{WFyh^3;N#FZ}Cr8~?zyj3yt;rGvD;P&Y8FE?ihL=j=4_lNbjj4eWT>9Sm)GbLb zkvc8&Iey*|{$u$;`-MTF@jrTz$wB+iX} zijW{s-;u^&o_v;l4v2I+iRI*p3uQvYl(_Xm!^{0viy|rJ<$ciHDv%_aZ$_8+1|i3N zkMisx9e{U2v|1}zJn`w3bx*sDq!Za^yuVl71TwXN@J9{+)<*{*=KesG2aBhhX_ymC#qsMyB510>7!UX( z%K&H%h^^co36=+vJLy;gfNQNKKCUuYKDxw*{?57bxP~qHQg>OFF8LZ6SvWQke$w9wLrGV zRQZ&@`PXdFFc5sw&d2lD%w`q~#=gK6-~gLJW6+z-+;xG!(Ck5fJNz)N(}UmPqU>;)JVtyWR4(GsTL3_Tt5ctXT3@@T zMg?XH>m<>9Td95NMku1J5!CN`(JFm<@*j8D^1If(>YRZ)0n2ZGd?wL1=_s3;HGer0 z7XVIWGX$uz$fW#s|Lp<5!}vAO7AH7epzXB~LPq3$^wI70vOjwKi}mNJ_J;GHasQjR zUju$+4LdJvfA2cuow}gTzF>T5AoO`h0$H{IIOh)t^`a%YPi==zh7gvR28#I2@pzG!e@<4wdWes}{xgVcL2sWI`+%VH!(_I?rCbe<=zyZZf1eMuRQ3))86^M1B6v1*2|zAo z-C5X4W+(iQ{=t81MNGTHVCWW5PNaOT+Et-r;pd~6Z0-;tgf6~la0}3MNqwa7PTgg;m#7Iu5bUHFTU-TU%t$^!HH%8 z=1GJBNQ~bm*4E_ww!1}%Nmk-1tjEM5C2q`)=U>X+#AN0ELw6;}U2t$DuB-AXpbf7j zqgGL>aBz1%NrFcO`FnW&`MZckK$&SsD#niPY$de2%6XDVZ_jz8@{#GC64U#d!Hm}E zEn7By)`x5~SK;Yr2A8}tGc=}1m^aHe@Os!*V}+OQ&?NITTA0j@ z|J6Hf7Wj>p=5fi3YV>H!0I>ByUgB0`Kd_9wM@UMAzwq&2Jq6SiJ(CZU4r_6|3b~Me z(Y88#BT?ve$T}C98>i>yOBd6=8!zsIZ8iabEy4yXBg!?G-Wa1E@+h{}fJMtZwHoJ# z5?-xGrHTszRS4Ucwc4vPUs#BDe?nT6+w>^k%l*3oxQ9Tae)V{>51!YpwTqZ4kE!#l zU3uN<+)@a!L*1~lJ6^X=_saNKlq9g3@BvRDw>>MP@h--1|NAZ2(XR}S0jDgezvlbR zK#KSPc1u75u>Q0D;lKrlkm_}zQx3OW4SUE{`G7h>%%2@)42~)IvLzYn^ze?}sU~Rz z+APW8y)6g2{*0YL@Vm4p%=2PCRC*70n0R8}PHVx_WdNv&OQ3`WIb=r4Z%Lbpd?NXe z&inghdkx)jY6L3DWH5YMA5Y8o@n1N|+|kpjnW1!0u$(4Pe`_X>X=Hvs9uI|-S()Ry z`cS`b1e#2wgmD?t^y&|^9u9RD>V6 z)X^S)lIXZ6)DY?nEScC4F9ww+x5@

Jp*g8SsBmR`^#@Q?zKh0e0=-mZ)*g>}tC2 zVLgGmWEB?KU_ElQ2Emqx>{plX{f;p^5BIX@-^K5KdeUz5BiWI_Ezkht zUWGa6gv*(0OE`io%Fl|rI>)#foxk=LtGJrNXdTbTzJ^M4+w}~)TeP~_R{5rzSLr>s z>UMuCs&Qcb%Tnig0suZ+M^(p^Cwwo%Z55r@z7c%u$6cFwo_WG3dCk|Vkag{`Dcny} zQBX+rZRn_#FY*aF@n2FXm6mfKA`>(c4!3?P?FB0xk0}m}d}C^^kr2>Q_YOFE?lCef zwp{*d&!{Ehd%BiyiA-01ckXcf?bnPK`?j7zRV_WYH+j46!eQhiu@XpZu5v;jxm)bc&UUd)#_Htbxo1L)3|}GDFWI0@k1NDI zV@L-~RiGa%Eu~l|?$EG3&T6*obfl5OdTRsNG9Tb+?JrznY5elPn_Ugxp3{!MEc4># z1|xCHi}1JJQq_p*SZ_%eO0rv~1zpcgG8hM4oWmFS-|3?%hvSJwLr0SPhC(MrrxDAr zQzF={pmw^4ElWH*yj;P9g=6)X4s$IBK;5YLDFC=-Mv(0~NRP{O=Sf9Ib4iHT_f8eO zPOcUTAA5yar6}fFtWO!l0v$35%Tu{;^7y4wcoif7W} z=FLpwy8E{lEOTdVWgy2gr&v?>>D(q~Q~oEgCK*-HyWcbBOdiT!1IcVw+GlbEN`9`* zDf3*~dio*;dr$M48H(A=wcWzh{CG98P9B@J7shW(M!PmgJkzMnbM&A0a>{4U`jsz< zSalSY?VqW$$LwzK<733{q}4+7$aIgQ0)jtFrnq64J-46qHx`+MwFy!~ubMkdNk1Tb z+-9qz#N&%t?NHq?=9Ndj0!6YBf-gE^c5i;$52Ywvp2^q>DzoRk(SHbk8U3ul(;_w- zvZq^7KRMv(P`uP==i|p>lryWlaPHbzI%~l&*HpfJuM=ksz59Czs~>ae_y}Sr-a;ic z;_*b_$Y>&feEfJeZK^dix*7l+waR>ljo9ejnuDW*bC}WZ3Rlm}yQf$`{0QC;w8Bn( zF6u#7o_Z~@%mGd(opOnYrzBAtzR((*Q+Al zM;oAgFdj3%T5~Y3i7pmz{t?XP4b&YH)I&9(=0w5I4F!khI>T^Zh>?Y%Jp zYRxc%moZ`8zAQ}5I9tPz-@f}9_qOWnDU8?eJFAL1+d)-{;GSFI&6wyW%$f;d=xBCC zXX7Jfpg=g*v-H`NX<4ZKfiG#NMI$=;3FPM+$jgc`7K((;BA{w^epTOFM znW-=HQ(7=0!$!Mn~plX`59uhk%-IPD&EzTOgFRzl~ZycZh3do5pL^<-fV(fN!?Zr_%hnfm< zPFmUShKYAwSDPn+sLuSyrY&)^xPw1A+R)S0%Iu-L7COw@XuQ83E*Z>`8(ffc!DL+# z=N*$8TA6@;->1-T2H864zwxx!cB#qdQAA36x%CR+Gj_&c7g1jVEg}N#Yq(EagizG{h-^q{Jo zB5z($N675{mGZeK&2nXHnsXi9t&j15x*joqNV?aZ z(X@X)b;YC1e7G5Z2WBbr?JgL{#U1Cj&>#^2Uq=--Wk;cDlOaSc5fQj;R)(r={JpDg zR{FH<@jlAmcG+>k9Q%~7uIJ+|#;p_axS}Ku4lr`#J>mqSGxMG;6Wd!@wYuN0=Y1-O z%P+oAW{fsRFayBoxlB{aWCt6lPM_u8n%83&1mA7sQ4^)v zoxTWzJnegaAYuh`$z1wiNIV}uMSFUqwltRofgWS0YNqVvK{oocipm&BW7EsH=k(qPW; z5!#=$ww@s8Bn8^fCUJSf4&ayNj-o$oiLj>m$86Xx)Rrk#+ldYb><+F+-vPMJ!NfSDdXe1+PwMRe|Wj{9^UtRzCjwM6t}f1S2#|B`h<0Dw^HybvjezR*nK zpPn&5`yy7Vq{n^sJvZIj=8rNecpH(na}eZ*%heuh^(mof<(kmP_wBkP<*a#qv94X= zd)2ByLN>-ggtPXQW{K?C^LGnns2fG4B88m;DbX2hsm@9ms9nvjc{Ln>M6jIvlTpqb zSB(9Eg~u16D~DKX}7qutfn z+=*nOfoFF4eiDr%>IetMQzDo!IVVIXad)b3_c(R?WzD^~E+%-Abj|zT&LX%sFFoj^seXs&6qhjPK`SRW);7F@W#~ zydYz1hFhCWtEzLC?JgNPq5O@zxa8L_Z+Wj*{P%q=cX5Lafq#$Wi-4a&z#0APx47V5 zU-I|8ElxTfxm~=8QcEW}b2GLdfVF$8T z-IdlXpAY$&jJ^NPGF^*S(Ki|SQU7MD^S43O);;yW=O<(COu86TSI1R%CBNf97(tn% z#$mamhc5DUOoA!q(;bwKnYk=h(BJdPslD08uh1#5Upi8o)Or)S#=YMeksXkqZGAM` ziQj($Vvr_;Z3>0T)xoKnww&c8Ij=9IPz3-R3WYT>^q; zqA3C&R+RK1N&_3?4&RJMDp$i)i2+LJfUnDB{EOd8IzPqK^34QMiDnhmn9+q=t6oIXmMo&)EEcPt zBA0g39U7L+1~Iu?@Y4KCv!M#{Kh_qb$nVBFRTU~`;m_Qby^5yfz8kbr6^E9r*C|N= zhF4=F(y`72dz^Nk5T?gI+p$f0J86!05;R0+k+_m4s zSF9mQ6jSH-dc(VQ`_F{|{ zqvP!fPKCE9{oH;>xHzq_+`=&tz(No=BeaBJOddpDNTAl5Qgw__s|QXVzBN2!u8JBh zIZC{h*#yULvXA}L$-xA6Iu&5yO+z?qrW=YB4*;OHkv6c%J&6C)JLsyzp9zy*)!LH^>C-=x#>?8)b zA6!yhsw3i3K@8r?k^!|QQZ=fxjv zPk105VyUGHtSjnx+Uu>Y=u`8&0T)^e+&Ni zOEb4dT;?5~4G&D;u9*?kR5_*732cHwI+8wY%i~NWd7Sfn*DIj3VPm+!kHofX=~lQJdK7I&i^EJDJq;;m;<*8Q=_C+fDl z5?w|aSEd5%oRF2iDO|N=j9HRk`A6%dsngaB(xaTYeYH5VF^{aQ$_M-=cZw4n`5(p# zbs2jkakL0Uu0Zp=Y4S zUxcyH8t=4S1hGZjzElcF5cv3%nsEJ&^1KDJ2?$46r3ZCag})+tIfB`_c;+sV8s~AB z!m!7(7A?)Ie2JG4)MN#%I(yVpB}xH)3_TZue+cxu#T0j7eZ?$oA^8gKMIeFV*a)sa zj6gk3Mm{C&8SklxUKMA7#3BMtb@7+!`q}2cr}6sCaC?_VmZ*M7Dnh=I?M+HMO4bZg zx?Q(D-=Pk+Mlp7DVHK(~`#z^J^Ml(W7YVe^ zI%p9&=#BZE*iVzpJ%pVh+-p~?V;B7f!rJD_r>YTw6XIL?u?j?)dY5Cb$2B8dx0HTX zbascolvbN6zEgO%Bu7{^h0Evu-VY!1OZ9QUbL9%n;f1VT8L@)QppTnM)1C5UBxw1K z*B9y)CYFJ&$9WmVWkW3*(3EL_*GBiv|yX*Ti;i3Q5j9}OpLJ>*;ph^An%M; znoD;$;$ix1m>%2d zQTJPUjwr_Xg~<_ZY3=Q$#C~o_!)`aOi$Kgb1WDbpSs+h89Se)h&~*Bqn@ZYTT}HVi zdlCB)Nt%z!ly9Qh8}ptQw9B}mte{dzHxC4UYS&8VNx#E7THzzBAz-b%|Ho8eT8;CVxse?3U7vkGb15%eAR#Z}~`-%jwnfZ^DQ zyD5lxv4~T%;%K&*D8Ub4+;elS>A+dfh+1VKSMumKi=VMz2MbVy2_K-pX9$#Z(6P36;~!-XZ)y>N1cvK)nac2zgSmYn zG9!zQDK54KHANh_`bx3&D~bc0<#Jt^hf<=f6VLe3_=h+fERGeiZz@gL7iJr`U`v=1 zf8F9xs;-P?qU*)&f+1?-aeXm?=b5#~a9A*_v<}iulU^4;==u?VV)H} zRXp+t2YwGUpLNz|SfSlC+vQF(qQ$rY zAjl}yBW?!{oroXkcsKKjr2vb-bQAm$pxh3sGe&jMZ#;Z~w9=#uMacS`83KjE)4mX} ztkf17VgcWXl{$huT=YUXy)H!A@Q&kkDFs+b$&4g8y$5{vLL5?@wK^{A8f zPIonr(lzZTyxxR-dc7uCvgYeS1p6ToD7VMo8k;mFp{_PF;=?!LNVgb=dSaUdORO)2 z7o(Ka%&{tS7X`~Bye^a2h1z1g3kMEY@b;<UwoI>O^0hR_wisz}e<`$> zkF9>8la)LU7&mw*nMuC!O*^u)tav<-h)%8N^j zH_9y)D*OgFm1LS5f%5<3v!*5jC7Q)FK+qa0XZjyO#}9^*^RpMiO2jhMz2c{s4`rq8arYc4NS z-76H_?a`$KZ=D@)!#DJxQCLo*$A!#XTW3YeN7EC3_M_0b{u@pRe0Vl6%~%{&Jx5j6 zr>#2)TS^S@&z7rSGS*LEA3iI)(s)C=l15rKwCGHunKo~*;UirxYP!598wm>tcR9{$ zvz(q3r3j0ONZbA1P->yjL*EfCspAumWc1Kjv!Qv@mc?t~AJ!2Z{>NK--E7QDQb(s$ z*%R}oC#+HLD{UN7+Ms}sySiUxfja9MW37o+@~(9rTilG7d30ic>}^`d2VSV>8BGKF zB0CTT)S+v?j0k-~r*4u*v04MZLAIk+ap&QK+pw`G3{MWUd?qTInSdfZxN(FBjn9P8 zNf7?jijrv@@n2IAo*VwMxp!i4h=QXy9Iy%zrBq;DUK*K$Vwxx#3qf0~eCOFow0=ht zd&@Ep7&ZpXmj_N}4;a9hCF@QjSK6bjm%`-R-y4{aW?aXkY1T%c-V|H7LaV?=PkHS| zbdt-LPmcQM1t5)!;Q8*O&GU@;vJ1_57x|!<<54xEmR)@5O#I2)0j&*Zev4j&u(@lZ zvI+4SVVtK8T4naBqAwNZ2l?C%5OxvT38$?qzn`sc4a*WK29{|SQcqXpCnCN6PzBz< zUF>YHlP3;_vA&$(Nc23h|K`w%6;HYaL*Ta-WDs(AVq9%CZWp_G7H;ZBxY&*mC#^bd zU|rB-IY<{n%DW4loXyp48G3YmstqD_4p1o`a@S)I@6i9va`#S z>hZx0JF*=ix@qUh()L>wN#6#n3_NEnKVO?tGBKAebu4H8Szb8aOP#}R*sprc(Z$gMWZv9M~WHYQ@J64Vq#Ic(U;~QH_aQGTxhYA z(;qhlf2AC_d89In2Nu!V2nvo6=DL5tS?+n_l^Hc6+%Fts?RdWtUJ<8uriV&3U4 z?a>{u!EM~(>YKb%iEZ8{5xbj`uMg^Ed>%U+BZWqFdY`mnnS+#esUEVOHb`f%1y(6Q zqL`uQZ~6(wS8>V)$iOX6p;7#qsZm^OK%^?=k0qtz)A=%;ofEQ0rGS+N&z1Bn&yYm> z5z8M5>?vyQ*w*hp9jSZ+Oll7mS!BOfExQ|S(`cn)*q1Wd|Ivqc zQxP|J>K4?bu`agh;H?#hk#s(5DF52oqvAZ5u?xaCna4UW^oG-ggWY6)DwYz= zJ(Kq42S4}O&nUz`3;nEJJa_S+*r8Xsp5TU`obMfb73_8ur$>{~gR9Nj0XM&e%(~cOx3KeH|$Z9FhDu0B%Yz_;JEP9a1YwzonhFNV`otOzT zt(UvZNSrj6bsB>I@a5a+%e*fyNy)*8OHiU0nuH4B90$L#(mWnO-Iu1lA!+YDT#?2t z?&zmnadjTi1n19P*UT23R;h=P7dRYg`-sQQI*nU+QV9eNJmb*uG#K(ek`ldM{(^WR zH5?mbsD`6VbqE0fY8Lc{{l}wG-C1H1 zi{aRy8ubcwBLmE%SbooQdbf?`0sRCY)mEPsl5UHaDD*F$z*fM>+~MQRjtl1>tGK)ecbDeMZ?xJYhf+LEk|tL6*2 zUd=FurO)lRZya-mAu5ta4;Y9g#pmzGkA&%mt{h^>9Qt#oOFvO}^OBLQ-Ku9R<##Jr zD>s(NRJoXLrmVHiKX>QsbV^8~5_1Bqm8AP&sx#j*j*-iUmF3~?Cm8mP?SQcarR5J8 zAr=VS3tF^sDQLDp5NFBgY<#*AT$g{gA0OX>YpEP7snOL!&vMAqaF?P1tYBagwJyb} z3ZYrw+hdYz>1D$lr$4LSJmtN5{G(oww=rXs)I9%_OpVGwi40dIX52sg#-GnPqNi=$ zDsFFk&PLlT`*=qJG*6bKbA43JWq;P*&;2E~Xo7i7&9~SO_W#8NKgc&x=rtx`QVm52 zpp?3NIe97r#``W@!`760Tz`jcbCMv;bqYwYt+OCMAZ^^GpWtT05RBzTk^++*ONE6Y zqSWgLAn=I$@EZtU2BMZRr!{YTvlY6wL4HOrtt-ti1bKR5-<9$^?Y`!g9Lx5M=4Jj$ zV#|&UXY2hh{6>6ZMAHpKJbmqZeLWkqJ5L#94aH9c;*lh^VQL#bCEpmfT~r1YNT=GH z`4E~(x7R-Mp>$ef3EXs2U<09JeSfUGJiC$)9c+_C^O#&_dN`h4_cT-v1Va1IrHh`L zbs2~Ei%^cb*TomHg)oZE=ASj0#jRsV#JQ4DX*b`Ski1tq#Xi$}2pMrL;w%>kYAl^o ze5b5o?zQYVkKnUz8XO2eZ2IuT06?4LOSe-wNvwil@~u59O-(%O{6?Fl7V0U z$6AL#TJKwu8SMr&SgcB2iKxx;yh*^mOT9W-blEyKiYD&L|2soh!!xuz;Ac&7-{JL| z#{WxA_)r1Y)!AM}>Kh`Nqf-Y;UndduXScb;<>I6>zb6uE!e7`nCup!g`G66e9vijL zm*86C>qJN~=lZLq9*}{RJW@(UiaN~fW7ph^5*+0tO@5z8W0mj|R-9P1PLI>f_!_m# z0t{&#>d6=h9t{}lPHjs{_88TCz+tQs*C7K>$bjDZ3O<5Haa|peP5;?Xvx}>yJe{jD ze<4#DhYCCP(R)aY!!gR(;88xqMF<;P_W3TCyeI=Ev~HWvy@|FW$wxo+xAJd>xp<#P z%nSHqvzDDodtwP5XmkYyCKs2^F#B+xU?Yz98C?Otr2@#w0~}apGH`KRrq{`GUTUCP zx3o6246U!hF+0^=NULe5Y&!Dujmcfe=~tm2mcpjD{jyuP*xIsaj0u?Y>X3bt1fhuh zz@$LiBU&(47!uRO3MpZqW%!Kbg>j0Hh;u^jfC~YzuWL7%8^X676*O!EXPt9cdu|wZdmpbFJZO>cwI;5eg zrzhoUWYUabhjj}5-LHSdZsjETQTh3RS0XOG$zbpuGnodK=Cpme;IJzWP0A|=Xi?Do zG9CzsVFvL*PRVMPxlUQzP3>0cFl>;UN2`&EMA=teJ^F>;Z$=!?)+$^$Oc(Z5gGm@$ z(g}50a~YL`l1i4t5CVOEByx*R0Y#fNGj)Y%NX*)4THT?8-CODC6_4+wVrGNV5%3QK z;Oii2K;4w0RMp+N@S-+AHH&@ zwa@`d9Ng`Dule&P&QVMD;44y%+OO^f@0z!n1u%%^@SD7TiaF9kr8)D1OmLY~@ciJD zOljF&<0Pc9^pA+wo9y{B5#7he=C$`?>bv)&V^C{Z)3SFVHE!@T66}l-!IRFt(~Pdy zGZ{snzs6ij_eF%bDF4fXUPTLJWYGF-{UBF$_T%RdA8KmIPW9lW$?c?#zMgzHu@SLD zzH%Ygnd&uv*YuTmO0s!RGm~)!^+BcV^u`5`bMaSyc?;)y9}Pst0YrO&0l%sd`j0V`c+*^i+h|*Xiwyh=*R#V`cS#I(M-=XapuW@w`Vgy z&mkDMdGwE#28^zME=k+R5%ql=aPuFf`2X=)Tv>?qBDAW6!CJZ9HP8$g?hO`+?X zsLTL@1D{{BiOL49|Dfj%a0B>!8_D-m=K3z}A^dTh>oZ&pfb9-^K2o6hUp0dFZ8gx3 z>7Hs#Xo6w=)CLmZU68cpo@Tdb#n!=8_oUugM1IShxd05 zq&DNzJQscHh6cvTE;AxKen^ov4tZOq=uh0Zo-P0w6g->3n3!2O3J4KcIZX@B3sjMr zOOw{_H7aNypiIp8Q6Y6DBW9xj0+iI|n!S+2&A&`}8@b&{>%(;U8CC2i#?fX;=ukj^ z1O&YB8ND9(2Jrbdf6v-yC+EXtz^p{^{?hf40@kWZgf{#w)Etm5k)I;K_St**3P0XO z-XfUuiA6FNekq-^7@*bbWzbn%@!(pwUK0t!AKLWrBB)snnl}8lW~n0@k=49ya?-zq z<;eBf@F={x%F4V*q)RbUe`cs^o*Kc-@xpB@GM_|-sy#kJWsrwKA_1qh$5>dr|XnJtK zz}HTC`qDE(*px)`C%$? zzHd|@CL)E^*(W!3(1VV-|Qx!tx0^CQ}-7*x49Q$&d|NX)okUMd7)OBxDF z`zq-Ei}XQ?&?@BOtO}U?_XIzUZ9#;L(e1kU9$GY{g?u~F+Dha)B9yPKp3jd;I!UVZ1#N&O}_Y* z6l6mf(NfrRTWJ}~0Taa_(3Ya*{+3^&S3OdZX?}f|8AGqbUT4UV_C)x^uKH$dcql52OlEfPiG|RYT3+2}?=)y_xW72-T3nTWHRW_q8(lrrFq5%& zG#lB~&)!YUMiv}mEC3(#1PTd9`rEJN!l_I5UkGRBr)qJW`YYk(@oxwmC(eEk--gNi zxEcYt`Jf#I?glCnM%@T(pj{Z8DGYI9+q8TCtI_i*E%p8B;!bbPo!cGREPw2wQYz~E zhH{z3UdLBDFPgNOi0I!GSqqOop*_nd-E}gBZfB8vVmfe^|j;^Xr5!&#JeYc#r!J%Zf~wy3azGf>k#Lj_z|-os7%9uAz_tjOti}XV!jdOvudV z5_>x)z>0fDfK$(B z{>`8gBY$Wt1<6A{XJXhEA8Y<7hk&U)=gfZvZR|aGj5BWi72=&k#QZn?;|qa!Atzg$bz8#-6B;1@zy->^=C{S87q%?1 zeh1unMhvSC*?n@-w+5w&Lm4Bv8$*##unq9pu3Lr=H*)Wds;w_e0{c`X6$y8GW1)hgVq*TKW$C{jB502P z+o~~XvQnH!!yP`a>BP-DzavO*!-Ay+E!8T_b!>$7PCBg&2QpccqwJw6NY_&m7&VXA zOp~1b$~&SWtJ7LiA2vSW5}J5NGdT;xSM~|T4Fi*G(r}fr@*ZF=hAdc8ehYUuw)6>H zt>f)t(%ChiFUw!{-N1cexP7(cLGZ*HF<-^Kk-t>?lH%Itc#EU&UVQiJ{=`LOR$I%R zu;f@m7!{4gxyy3MQwV$J^d4U`E99mu$A)(&iq!B!NT0?Fy4t+6yrliOnSnLfF*z+S zjl1J2dw1E&7^3@YI^jF#Td>ynmyFxTmRjcKW5%KoeU6RZj;pzU7Jay(g4Yu~RoWuX zZdQ^I5}Lbhs9h^YhbG^C)UN;2*uHP}gpO-A`%jR`F_o*hrc(SZy}M1R5lA`inUGX3 zRl@>0F-$pAw`i4cKM>(&Z}A$j>)LhrvdH%>%yE(ZwpZaBr+Xnrv)E@tNXK*rjsi{z zC!z8{AqnEYqXicARV~}DX7~9Sf|Vw;!hb9Aq+7i1W0ekdrqvHxOb*(XGh-tL3IAG8 z+C#rAxZIN|P*=2V3&IMBzm^nN!!b7Wj2y9?D-s#)y37o z5t_l{<(^Fcb1xjd*qoOW-%Y@00_0IVmx-0etDwnh3i^%d)KaI8qS>Z1^<^4c^0E<< zb2cnk3t6s0gY(1Z<4_8Q@KN-9V{WG9>kf}*L>=m{VfynqcLQrO&^q&>n;V|tEGJ2R z^KDW!)UeALdE0@O$4SF2($tm*;w53;3_IFrL23XRN7*45-1^v#(kCzfDg==q^ z&`m)s{C?0wxyqg@Id7>ESB$%lB>f#L*%?o-(kt0vLC(L%FklKY*$dFUbWXeo2fX2$ zL!2JUw%G_G#hJ}RffCYXdwNPawYi|v$qEK%b-X`?XECo1Bqg_9)1Um9%j;*uRI)-d z>{B}AY8_u1Am&1dKEO`DI8&vMS|4*55v@kTYaaijH)y~FW)Ck_=gYE38i%yq4DRk! zyH9_Jn-}O^)bET9LzYaVQYHEnh0RSY_}%DS2%rC;zmM?807l(!2HeSSMEIzt--KK^P+ZW`zj}i6lLXdL2KINa2x}aCV$g_l&Wv4o&Z{(hwRi z!iX_0FdDc8bEBs&@qmsDfa|fOhn1YCC*(A50@rOe_x+x3!xc?3b|!9-=xCXB1Pzwe zd=}dN4!6N^e%g7KLhB`mVsNv3co?9$>TSIZt0HSIW!;wGg!Hd^?{nnWlLy$SolgA0 z2IX3_)1OP62DC@!|2uFXTy4*BVJ!DEJO&xTwcSa78^dn@$@q7Q#V`C~JQ9(DxEVS0 zyYTABbi234$dlv5!X_8!!q{uIf@gd$wteQe zR3)N2VpYMBK0p3qufF!$><+HPGx7JrFMkByQ@iz&)A?!K-j%lPQ3_qryXrtwC(bsW zEE~OR_4$PDQX<{eH(z`%uCn{Mf0fSsJq0_Kolb7l(3kN$ar&NJxXS+K>C>W@&k~6* ziY|Gdu>Gy_*PXF*)7+W=tmA%P=zTwA%b#uUz8)#jedE95{TuhBJ%*b1rMw_L2H=40 z>%do+_AS-ui7d#PGUNQdw!R;Ao2o<_oVp`+{3*6E?6s&_dgx(>lEkl{U!-h49eOxr z&%4fI_fvO^K^4vG`7(TKw_ce3cS4Ho46(bnBh_Tgmj18{jK1#+u3rp(6hwra?0;YU zy3BrFdFY0i@0|=Ls~DBGu(XNkyiVO5ZNa1UD~@B=LiT{&&)@Dnlo-0nTW@K9!H>Nu z7BVjtWp3Fv)Mb2;;QMT%vRmed?!$tZu4q1Kr_+j{X28=!iLW-Yme%I)?Joe<=y4zB z@jE2`lAZglH|4vA+*FSCe}%mKR_A3;M!DQqmbtZsZ?Qqy(?cm9&j{#Mj& zxOVcx|H)06Vyr6K|9AbXohN*1O|tmz_M#B`$L$N{fSz`ad{vb6`p>f!33K1fyL9FG z+l!)RYxmsq(v<*CxqvE_gYAcRHa$qvb+}yH;`~};ndIKQ#0!TO#x_~Z=Fr#<>a@vC z@BFFnS${tGO{#2pmBe*9GkNx>uQPtGbucnMHKF+e4-0~qKW?6LxBSO^z48|) zugLv;pl`nR^``a%Hwsd(OnfwLPu$_eg#N`Gd(Zb}+&ugG&wAC@xuILmJkGrFss8%; zw)byc;_RJ{L@oIKb@!I%@xQ#Mae@X74m2EI$huRxvX=k$ygfNrU#ePkwjG+CF>Cgx zPxB6UrSykXlsqiF&Uz)^>fMArZ&su^F+bk>ulDt_jVnG|N&|;R=ND&e6H1-K{aE&S1*7q|*iFB^=HDrgQ3PjL;Kqiw2E$pO z<958?u=HS-&hyP*<)-Vh?24SbZmU}B&*R_4{a#sc=iPI*+W#=4>hh$mGEp8gFP}f} zagFKb`A^?I&73%=Fjs12HN){;Q#MLon{f3~X{~pM)LS8)<0X|*lkTrPo8h?AXXpEM z*HSIT7Uw?n2M;MU>^G8Of{Z1I#xj5hB!J@(kO2vNRU-pfH?VI88IS-DW(++;{xs)p s*>w8Afq*=VryD>+GPDE+9RK-mO>|E#URerU3&+6V>FVdQ&MBb@06Qeu)&Kwi diff --git a/publications/whitepaper/figures/tpch_scaling.pdf b/publications/whitepaper/figures/tpch_scaling.pdf new file mode 100644 index 0000000000000000000000000000000000000000..428e2d60c0fb6dcc49d06e10c1c03b2475827745 GIT binary patch literal 4946 zcmZ{oc|4Tu_s2<;Np_VeH?l9c8Dn3vj3u%!BU{E8OpO_4u{QRQtt^SMX6Nxpln|0c z*_V`PLx@laq595vp8EEEp5O2O@A_Qly3V=I>vg|h=NvIhgrOW%9u5(Un2eZ;=#Ln7 zCPJWq0zhy-2hr4ofciuXii{=T5hyYSkVL4%6rgYgMHn0kg(@hjNg#@Vt_{fUdcobq+?{EV2v%d59_U$*z`7Y1ZRfGizP_>a);%O#b>YVVtKzFN<_kxz} z_##I@HSP^{Q|8;VOJ{qZFYCqIMNP(Na~$*}g()v*w&0fo8YL&QSz7To0-kw0F*>*YvT~+cxeMvzl|ce7S0YKkt;dMIpX-YE{w2 zre}%Szg7V>7(H&wIdY)uF;1ezJmWIBJ+{M4&bYg1UYjbD zqc-X#OP}se9{pfXxUrFVDo5g*m}rBW_0yts9KXrsTj}LF8%{ApqO!Ojk1V$mFN!+G z40*{44#Lv4(eL6i3fA}!7}c>A_d6V!{B57r?{VsnQcl~Kh1R)CTe5Mrxs7i`b+gy% zwzHZ1`fN5rc;hsbZid~SXm*!M68DVsU@*OwR&BQ+5X%QWK5Kp8Rzu3^k7dItiSBZP zZn~h^rdSUW-~`YyWevE{ za7LieumlA_GmHlorAG(^oD^t#a77pZQ&n~Ojcjx>{Ui|$plBNm8E^t=2&Z|#1d?4K zpdpQFfTkvB2%|A-*8vSkpg|zn$c9WOxTYrkI+{xyfoS84LhmXd$=XN{Q2J^9Uy{-# zx<4oIl)kw%%iWu-Q08UK+~EY4YF-gNQQ1))!%wQnDW= z`8K>zKcz+_{vxAQ&B9)O#E73SuRz8rk1GB*_k)&uZ@dxO2tA+I-2cjJnL66YaAhZo z`Qd2?)^N502i|wH9dStsUT*ss**-Ba;TYcf*4FAB+BDpgwdcZ%L|hin=Hcc>cHrx_ z^zg~VHijR0=?!N@pbVCoiL)!4W6u1CpQvgfv&HHyo@CyIoAoJ}^og8_gxRN&R%D$n ztY=|3)K+29n*80QzDdld=?RIcBsK1q#1uFF!?XK08(=Ns=qU237@fz2=^FkE=E@9` zsT1-uBpVN$96D7J7AZFs#PHVm$=OdsphHhTPd7(LUyq|a_leRtesO1;n4~nGgm?Y@ zX0r($Jqqk9Nura`7g_vg1m6)EJo|Uf@CNzt-xkq$p{k~;;3MbLA~_P~vU4MK!#~3{ zI&6s_A+*`PjttE@$Z)F!b7yQT)(6YS;IhDI+4L>8*9~NdzuiJ%r}`alyx?m6Xj0Zo zrgCD!*_1Kwyc9UMyp_iurhoQAzje#2<@cz-)il*qGF0DH1@9SLf_T)`H|hJ8Z6+^v zQ)u6J%JD;7J11J=z;pvdcgbnsbjBU0E}l5%EWH?=&AjWI9PQtW#u_p%z1-v)$Z*(G zEPPZa^f-g0G#9`c;vqAzOg-?}i0Nx@!eCZr&USy~cPjGU>F-^KPJ|_+jS#SV+*W-b zukSeSfddQ%I!yCA0&`3}5BA=Dz;%y-<1N!3UCFq;es0jWk-S$J5x`;I@H_cO(j%+f zB+6JQ`CRZQW#1#@2=rUH>OT31ZnwP?jMfV5x#3cuAU!&q(HFZo`K)8ffPRqs)Ax(%>lyFi(^<&3E)%SXnJ@61bDEPRA8lkfs9&FN<;}gt7rakYfwiQ~ z@fH`3xz^1>b$@d7;kGBAV5&m$k@XL*`Z8VMT#B}6zxI~P0dg{;lq;N*OZ{eo0@tC3 zhI%Q1Cs?_JE3a@D?ZN82ymGuqs3G1srL@b5m&L#UYI!>Sz*8Z}{!kJyb!o`ZbhGBZ4Xvv;s>j)5$z&YzPYZ-~slF>q|Ic8+UJb&HQBssNn`O+FHK zT2P#?c26yTt-x@y+C~1yiKvnqhdC(*dBT!BiL8G{8|z7< zb-c_`Mj1Qp+EbVL+SQ3YX+pgI<|B+ekP!Ndl zGyYu8?fS&{Qq?0X$70(;vh9bMp*on|n4y}jnto6%F=2)Iy%Dc$N8F_TJF9rfZAmXl zv~6^S1;(afgfe0{e`a3wpuS=W^MEzBxQ^f*RVT$4AD?4VXOe4@^wf_}>R5rmAk*B3 zHP{wvi?XF*Cue72*D^2uO!=8iMfJ0pvY@ih&rC*MJ~JG7U9LDn9T^^}FApexGpvKn z#73W=KKF4@wemBW-6y$Qa49=gEp~QdG+L<_E(%9_FAWP0WRDJwt_(jNsu)r){9=3K z&&7hAie~-o==CCYyG}Es{966lqCRvCq&+RAugqf&T0&N$e4~gperb$qR1AI`Y!!Sl zcrkcmjde@*+lg<-wwkwWSkzgBS(sU>Sn3acj*)HWZFhpc(yaJGriLoUFD8^>>58D$McBNiJmH) zx-nULcuRJo!B?|QyFm5bmdKPrP=dVds6%$k<|cQCShkoE_uwVyCAmw>Y6aOWeQQJE z#dl0g@8Oo@mw0mb5%v*Mm!E>O7AG3SuS#f{_?x}X)y$gAwN6>FeQ?P3KyK+)>1c6_ z`2n->ymcup@n$WiYblw2w#Hl~tlO|VimR~L+Me3@+M@34Ym3gxVi0a6jQ&`{?WPX+ zSpP`)BgAc2*!8Ilo3vGN74;3{Tw<;Q!L4Cz#jIOmg}U168sXUHGU$}=bnC7gxp?;J zmfb)_ab+-ap<3QitK~7FH{wq89qjoq$vSyi@Ir&Ur;A03{owVRHD5v(oD%rr4LVem z@8QbE<6cU>P`!s;Dyc9NG7S_vY+7Y1WAcVDgKycq+Kaq;mijWa4s)xg0M~EdWB+b+ zVSJ+gm?ToZ9~2rte!H2p4{h`$`I^SHkM}K6NUQpUTc)>79r>NdTrie*?#xpuCSQz` zkY*uOo{J?fpEm_HRe${aV%Waa4(YgouXbDR4Q}1)V=0V@r)E;swvQ_ts;r0gZZ$TX zDlD)2&>s3O^g~1(6SX(Fw^#1E9Hz>*vWT+c_OPw{O<>sYtH-!XLTUZA<9lLH#2yd3 zvAOuAT{kx5@~z}Ls@JBA&%Qf_MLhG)uhpJi9THniUO29OG_bC|v$%9BqEAn>=~uGkMwCS1I!!K1>gzn@xk4voC3ps=)h%Pq=}x)!7UEIUhDYo(y{Z zWvAvq-{E9e50{UP)4t`T2ch{>`?iJDS&p%2#0thXf*LMM{JAO@CgZoS4kMeZbDIwN zE)I3=q`X_OdQNz4d{4OfNz>Ok`>m?kdJ?{W;tc69wSGExs5yT7`i4L#{9BF7JfNjL zuciLUtu0CeJy9cZ67yp%^!}XiS5!xIA7WK>+^AEWDgmB!-3;70HJX-H+Hq3mhx2yN z^jJV^;0{cIPebXu^V-gE^^LP-tuKBIEn$|)FV?;x`Kvu$uidu_BfXVd@cGe{8Nj31 zuw1h&b22k@?(31Y&r79=7I~pe8_}!V|0=cBewCq+zefZHcv?4(#nWa4^s@E8D|8s1hNOl699?3L857uKMsfbP4}kYy--Ao4-Q2k10cbRfXAGtc|+~qOrlrrF2Bu4 zc5D5gh4^1XlHE=Kw4u0@cV{Mak=A0EqkL!~(Yk}(y!>TjjKTS1$XGOr)+qc;8$F}H z&r|}57*B`-0EI#netm$ll9Hkl;0gTFV6@2*{R8m7G^jGI5BgsXs;KfWJ1AU<*4+HV zPDz>8U;I;3qlNarngR?;n-Tp}Q~Q?&hpGL`2V6<%w^+zT6c&de(gt4uXoC&L(B2Dx pr|_NxT6adz2WU+okOBHYYBx6|GKxsv%?w-_t_l$oGq5m({0Ae@eVqUR literal 0 HcmV?d00001 diff --git a/publications/whitepaper/figures/tpch_scaling.png b/publications/whitepaper/figures/tpch_scaling.png deleted file mode 100644 index e929da0c5ddab385bdcb91ef20045ffd78785a0a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 44252 zcmdSBWmH_-x-A+sK!UrwySpT~6P(~4+$}gEI0Os92?+s$OYqw@AQmDB)S=rcIK6xU5`Ce<$X;u?QEYe9m4^NyzS2jGVyIe~{ zzk!&jv>D#^n-nZ>3CYm)N{>bmdPiMDli&TAs`4R76B&Ole{ zYlJfu$;iMg|al2YYoYtqeS2lHsbEhOnV|?(i#jv0JGqJq?^wNu}P>LhR{bt(5Ek*i?DDrd4 zioMR%7>=MXX<5sWxRJ^#p}I8D)OBQ;{ew##=Oa2>hR4~S;XB>i%x?|m_2`ic%omz1 zk>ZmzXCz086~^Ta@}_|ZS08Rh{b0SxU!8qV^mkv_lr=Uk#aTdHK=A$JT<}c!*4Bg5 zmxQ45&pGXGY9F=0q=z+Dbsm1uq4w2}Df6v_!afa~CDWyL1^=Qn<8&A8jnMgl>ep$5 zu+$$cS__8Of}RG6|0w-gh%huN#LxW&??h3@&av$Kiuq$*keHx>@bl zHrXvHDQXowRh%fIjK}fvFzu41V60yhQ?IkH?w|u z^-Z|9+`08vrLMTfG|U~jS@j+Vu{CyH1^-mW9+G0aP&*m|imQVVF4LGl!};k?RS!)( zAr-ZgI|+Y=h*!hYheS6+A0EQeD-@Z}s8xS+^62klQ#{3f04o0?SA0A`F9b=PT0 z@Ar)QOy$>aJJ-R#sp;Cq`@{xE8?c!)waqtzO2Xa1U%@ph*^k}J(}I2#qM}|vpGrm>U@1Y zQ)ko#o(7rgDN)`+*-`Xzk%!T$UA-&Zuyxr#uZ*tfD)QmZACj;+goKtHSd?h{xL2}- zT(qUnP~0|+U0zhqi0s#2&t*6v8;(1~gFCX$z~@!p+!6hi zrsW+~+}4YQ+ZJih`R5J?`cmPK9m2cXZ}bbX{r54>TZv}Ej#T_MxW7a{J#TDY+59TE zhM1rsRIzwVx~;e}?_T%EJLpntgkDORoS*2ijGsu@nCcPlkL9Vj-V+yZdtzp602|28 za(Zr0o;(9TKf{K3lAcKb1rbeMcU@&AA#*1OHd6~HGfOsa2WN2flP4nHLf}USOLtQW zZwGrvHz99PD(DeH;AhC&>{JxcL)`5|sdSaqC?uU+Eh%`}c-T0o#84tc291 zWd1cAyb`5)S(#Ln*J<;CX3&F184&CV$(D9Fyi#m>dW3XWiP^Ko=H^=5T+qyFcY z{MYZ1vUD?dwQ+X0adM=9{9aQtCl7Z~Dk{i{{?|YM%+uY*>c7wA==QJ60vE^*`GlR5 zjf4Gv8fNKj^S>De`Q(2)%)(0?JXTF9o^l)IL`m^bl~3puYv!k zNqU<)|8p|vP5kHDL?AO4Qnz$-viE@80xd@ycQH;8=mq`f+y8w>{uw3dE;H8 ziShnx$iKh$pU3O|_i4oVx&Afa-{1Vt0jjPx;7Uy)^ZH+N`uF$#^LV}g=>-4&=D!Vq z+Ma!bN7GyzQDHzefnRTTC@M#+am1$Ts57X?8Ts$;Q)C_ z!OaFP`#(Sb*H965$U6GBb@k7pg}w_`tr!Xz`CkYihT_;x2?ilQXM>mM(4Y>)!ZwWc|iIbRx@E^zfMYuHkf^DdgW+z%+Z<`WO%|GIC zWDs~C91I=z|NnzCuge4P6>+2}w2)W~8{LO91(_f2>}P&#|IU?^j>a2GVb#~Y(Kx<6 zY%SOPo>=@hj#RkidLx5LtEm1&P{i-*D29-mP9;-r-k`x1mr1j)PAOk-cd|T(-^r*u z3Rg8-B$9zgRx`iG?6t|>RHat2wg>Fp`J{1hP}pX+Z&~%W#C5Wu=iby@gPZvETGCJk z|K_ix0*PSQ{FUJ>;R2;JGeo=dorwaKEGFFwlZ!o9hAac~vqnd0h-f&G><6-%(w~ezeNRD6(<~Cz8?2vBLhb6e2sj(wY(~UP1Z`vs;W^283)z^(o#rn;PwW#5*p7^94UZnk35`B(U^)m z1%Zrbcr4zx`*jYBVvl!jA(SLsw0_6k_;lpL(GF(ATZ^sY6Aq@Fc8K z-2J-`=4d6vY5BKQ|FVbJN#vxDI+ViMLwVz1zR5W7{ybaM_hNg@dNfa7#U_tYM#N$y zTeSBBxvVMHls^hOi9mcyYR4dz5+Azw-O&Jht5W?tvx{WK*YM%|i+Ww1}MZ!KdF8>^TXz}4GtYcv|5nwMSYlO18S9^LRWH7 zjIYwMg7+XYMs3U4sN&{Gj;f+S?&DRj_)MLXs@(lzx|p?^)^O~jJRxKen%c;Db99`Iq$v7-h+lsGL1yo zx;fWpV^8Q>OgqC!)TbC}($Ba@MBq4+r6N%4xfKna+Yz+wYiuX_mO0eKTs+6^W@}4J z&9LBY$pY_Obm&5T);P?%koKB(lhMqjC}d;S6iSe~V%umx8H9}3Wi%Z$?#AqU`sCLJ zDVwB6X0P7z+x?9h^gV38e1|SpVeJ4*PBMGH63uSrhfHoBorYE6k5m8na2q*w#FF3+ z9#_ZKdrv>l{1ik~BCz~72DwLQT5ZPijs4GYH6n1g6h3pf*m6bgtrnq^i)H#EwO&f9 zDQV55Bzi<;OlbK6;JpR5{mK*QgKHgjQxf0#S6Z0tn_t+j? zFR32xKsjv>q#u2mnI+rw6rGHx9kpOBC-9*Gz^O>kx+yy%(d>@MZU&!IV!V?GP@T07 z#E5@L5{?mf+JP1g2pXx+Ug zed-g}52x!Z2Wy<6MAnwx5X=#AS|X3Fv~+B9eX?Hu^F$%(o$hlXZWL7{&i0?)<_7i) z@YZM)Z7;hBtL1(#>ROYnQFZCK#*<(l)<#{6C4OV8!+b#_41VNXbrMNj;bi}cTYLtn3Avtfj1EnMHow2(zlirQ6aRnLc#J}H%Ut+{&1v$ zY1VlnbDfiPnH|L}b@QIYw46g1vPuJ=*y$TPGMJ?HD=h;8yF!;Y&c%rNH=hd2aj>q0 za+RBGAW^N(m~Z!OT7_KdpR5mfE%^|Wj{26kEa>DuQcSUHk&&fwhdp?7Q_|;|SUxX5UN$k#jK)*fsK>yA&v$GPe22#B_`BBJf_4 zz#%^8z|Qa)nUBFo6NX8%D{BZtZi~{Q%7?F_oc}#(j47uGtE*08y=Nc4a4bp8lFnVb zR9A8RJvo+QmTOA(fZjkE<9Dy3huC(_I=Av?^wu?=!y~m++4@1YpW#V1-CNI0Fs=}= zlX8%Ub%TRX9AIo+2C(!3OEG9g05dzFREorcaY8a-h@8PbqNM34VgZ*)pZnZQZ9(o+ za>)y@=80|NJxB#yDHPrc7?J3nSh|^|@CK6NDUc(tg&V zG>_t^S?*H5-d2#49uas)9&;4WDz-dH!GxK8;ru;`0u@298%}6tCer^+@^g|1_fv!R z&Tixo4s`dZfkb`^Y+*c+cA4*DQZKzdvOQ@=qjpG$g!qj`T6k zzGQgWN>tC^`#IET!Pnz4IrX{F(I^iImYrH+FkBceQe-ntJqxxN9>L#Gx21t#BCCy9)zc@o%-}(2DgmaNE}e{tIYaHBj)q9G*ya~;ZT@jtcqxh@ zq&_#*Y8WjqDG7L~C3JwIjcAY`DNw=qg5}Reicn&_T4o2{lz<}Rnxoo#i01L$6$=*0A3jKi;~-_iH1(}ka;WMq0S_GgPVzmEcD z*0+}NI`@s7+g^GP)eC#M9x(^|{hqMQk`e1#hv9gonNA5kMk4WCZUVet!-Y9Ej_*kE(t1hiXsEmzMj}=QWQ> zLu=1KS?|d*!?a+AyHQ2JgRsMvvIgtH*C%m^}SoCL#^(Cb#S_EaM~s%Zq5> zo|gI4dMNKXgP*F`*ozx|dFy{s^sf7deN+D#E9~ifvi8fn30_wrh2__9ygh!TBB&?& zdrpT7%?%3KZ+949l1$0!s3L9Xe)3yv@lW;82x1K0k90u0CTZMp-AepW9=keH+qY-k zW6{PRdIlJU+%NKgD;U)2s>D`MvAT6{6(rsEe3R&uW#UL8G5HZvldj?_4Ml}}0mIBf z=#`Bjc+q6>dO6=>b2!U^6>CtdMfqw7W?&}k!({Ijx6cDaR125$X_-thFij zamb=bK0+jLFau3qX&T8mb>DW3eKsFv#;#RG^}I6CUeSH)a-3i6e_LGz5X>g-QC7 z#|JXe7-BLr@ib)TlL^#{%Blrhu2QMdJGp|^r}L^Ke~+aJ^yNN7DbB0h)8tIaf=|Ghv$nH#@c_@|Zt^cU zCB%_5$(hM^epr`HpI4)Lsh<|JPKFp~8>{o=iBbRb>t0}-=a#wUBU3HbNv{JpAtERo z9&5G1cPf|%gLXvp?>h$YpU`9TwFWrB?;@9DFtNIz@I^xo1aut$@SIvPCB&47AcrBm zysFfOVl)Onz+)H5(l96@2f$m*t@gfVh1^1kIV745t-g8>c2?qkK?{P`VP*Tt#O3<~ zQ1+dsDkv)twdp-^#v75~DJe!okw~(zAYZ*EEwDC{xG=DlS|oBXJTWlMTu zUi!6$-LQ0^p6;SC<~lfqx2i;T)1?yaD@cvNQUPUkS&{+)3YxxPRtwx8rJyE?0GMwm zdN2;eJApX_6~=c&6ASwLm%)6o#o*PURpuu>sNJF(@;H!rgkpmde1#zg8G?C~Euj>1 z3s%w;y%3ir00z-FzbQ06v8v0)5}nW3mHR(jZ_;gMIJKB!J6vSPx%Tz}tzC_4yhZc{ zF{+`TT=cDbdk8|A5*m;e0_Ht2Kx6tHwLfjK{zwfeXf6~O*sF2dbCu@df3jKa4Cj%m zeD^>p3PU5c9oicIt`UvTzEG@FzCT;XZrtJr@N|$6{E8B_+>Si?Bm=YkT*Ze<^WLc+ zGuhJQXs!1b`kv){Y3-ZH8V_y3aJAHY!P-C#RwkBa?tn5;Zk(fCh^v6D9fAcZvP-xy zho8vXdH~P*P+H8aT_OfvHM!r>6bdy2SutaK1Ma4Q`uXq@x=xhJm&Li`YKJp;ptnTH}BcV<0_( zf8HIN8=lyj{q>IsI#979bn|&`y4Bw!m1q>8jRMzYr7rTD;-?vTB!t3UTcO{%huKTP z3Uu-cO?ZtAI3~oGT}ADdX-CZ12tgQQyR2g`Ucd~J>PY9) zO0op~z)-xwJ6)(Zzp7XTnqeS`9=(Mxj<3uSlo!t50@dC%I))w-XWL`Q6+dhz`oCdV zmKYCy2SP3_cja)oGyJ#)+-rHUo@7brO<-@iS^)?H9D(-&$j8_*!hGqsLBN*zdHz?Q zf5CedXtFD9Aky0#r&QA3oNO25h@B2{ANj9D(hI!jZQZLHVA3hOoA+9D@frmMo<%X1 zh!n(v^mLbqm456Sv8xTc8-+}HKgPw3FydAPqF!^o~5;zzcEMfoZc4l2m%VH7U&76NSLyylNDslN3#5&~W{ zd}Bz+<6WYcbLJ>VpNL`MdN<(ZD~9|YC<;6j00=Z7hdx*qkO%rLp2}Xuevls|WsbRq zu(g}>-9PBzZ^!bLfqlnjgLOxnVM=);lF0@B=h$LS;sg@QcjELrK{#N}_z_L8!5 z;L;^#Na=ynXWIv{<}5)4P#*FypmnXW)^8C`nvn|^3wtC>+a*PLmam<1L#uw;G^&lE3B>~Cs1DCNfa&tu; z?WV!nCVkfeAEga=xUwQ)VE4oz83Lstc?#a0FB`{eCP0D6>kBQoHAA1+$!j6DkHCz^ICC4ynYwDZeaH#`B9`E}{*8eo$G z9$ewH*h1i?O|mZbWHoN}L)g@Prl6bW*6(2XZpKXjGmh8w@7I^1kJ{)EV;PVGpRF1t$X*9?ubc1;~v`2H#N1^W?S&a$(Qs$f+JNEO4Ea?<;>A zFbCKPuAA5MJI?hvz{8o8*_XMpG?mJk<|jew$#3AOTAZd~*}nyMKY$)fzb$QvjJDuZ z*9P+_?#X3S0;m@-31&ogTUkL)w2~ZGsO6|@hG?7F z5EXH<=K0(<>mP?%Yx8NrH9cm$MO{xaSKw*1r%=8sP$9Lhn9Z)5P>vJr+U9#fobhtesA-Tw0nQb?kQb9sSA`6t0su#b%sT-^ zMH>2eN2dWAK}Xf`0YQr?pm{|xaVs&5kd?30KLL=gEB2L{OJHE8gL@j#-`nVcOdiJ| zU(~n>zt-RzrwI$K7sn6YCfMKOX?8YY4{KH~UUkeT(8KbmB zwyy%}r(~!B^c$3q@E-YZ(CK~yFABz;<@F%*MS@7BH<_A%xk$jPM}Zgr&j(3}#M}a8 zoIYmbVgWS7s>xQv3s4Eku;)Ja=aYdq+w`tex$fhIugaS*W;SyI$bfOZQUZvi3Z~0$Z zLwOSbrv8sNF2G`fP#=J-X+j>h$1B~w=Q}lk;(~iR`+B$ww62FqOx!|#9H@7a%}AYY z3@LtO>Xxc8??t1=umSB+A?Kwa++Bm8$mbzH>{J;kA(fz#0^@Z3>6FP0XiO3cd)kBR zf#?9BG%N;EN`W*080UYz{%QKP5NLjao+V)aCx)UTo_o4Lb|XF~FgfA;YT2JmX;^6{ zodPF?Q!|AIx<4}jtH<1NG#rb%4vx=smk-OD4p5RG zf6?buv|jx5Ii0#7LO5Eq9?txgGJSh_DEke-ymF=hCy*wo4?yzd5?_N>AaA3K`3r2M z*6sX^E4v$Ti|~YLm{=_W$U1Nc zE9i?8RQW-fT}COo++r>HgWqho!t*t9KY69ZT*p_i*=>)01O1*JAdt%S>&igMBd7te zZFY_Z`i29nbtXQ@wI*4`+NC?=F+7$f2wf<%&lo`WKn|2%a^$Eqh_*OB%Xz!=9l*9q zI!|O|`-G4RKWo(^Z<6eEm8Db5wH2&bAr1pMbqMUB{9cnkaB8s7-fpB>6LiyOdxz8( zPW`urXfg1dwoIZ|?U~+;cLyXra&+n3*!35!%n0BB>fL$eO$S2xs7b?T?5t(O1jm8h zTp_sDmP;lDUxmYoB5R6d?oWq>*&73-^o0opj_%n(bKESPnoz(*i%{jR`^{6{s2k_c zwc+N$?}#gfIQb8>xvA)<)X*C6p5ZV^uprRmG^wIp3ws@$@gCH&A>03m-H&gRwZhpJ zd^wm~&0#|a=ZI7&eUoli*1qalqY_YQ)_s*huSzE3g*AAMX!+T%KNa1`CxJ?OFz

k18ADBk3XVhgQHR23R(8!^blh^os#>kVP@nX=9Z7dC0;uWeUAKXiHN)U zXu(xt9=F(r8Xs;<^Js{gp7ER{(NxrnP0>*mu(`RftIyJfw?Dtb>r9%0-yk3!h;5;5 zGux#~!v2ujlr{lqhg1Y2t4l1u!%XPuGuY?kf@|2K=nF7~e^g04*(k{D)w3hpk-BL4 z|4uW)*hPQmJ%w@Px_nypm(f1|tu<>BTC!8o=O=Mkc+*m+v8bfnR!U$IHF_P!nYqU& zTA>w8f{rfjwT`iWO?hIAGBa?{Ayf|oeT0;G(Ctq+c|bi-gFkOxhUQrgr0Ck(GbCV_ z@RgN~6{C(<5yp11K}SwT2C+pQtSuw|b1f`^2h9?lax#BYnfA*~cEPnk(0o?T;G@M82TJ6CM#$@c zL;;%#>SO%Dj?h@cp`RB)?&2{YyZ(Fw@(#CGs~=VkGM3~cvMRqQ`4n-h!hhdoIkXp* z`?yqq(GePIevlYOw{}`N;V{nc{9sQ2}%0lPLXA{T)qgTK^#jDm)F)X9vrG1B7MY?Wqt<9oFZ4A^7!FUE+pzOes z_w>{CZ%OoN*ig%tk%LnwEyyc0EouGGvIA5H8MQQ^3AJ z3U!zRY_7Xp2{hyVfU~sdvzb-mPH;T>^u_q-cM5o2OqTrJ9(bGA1&|8!USi6W&|XOK zysA+#N6a7ai&xEWJ(TVMFsOg)9EB2wD%jwP*rwnr>k%@%+V#$I9zEE-$NSdDN#Gil zv@sLj=x!7<{`wr|4iOYqu%_Xbx`n&l_{MAOb{_X2aA9%Ih8JZ@Div@rQWec|nK0P= z(GfD+cxlLFKe6b!g0fT{!Dd&bFEe%Nwdk*-IP;p~A1+{q97>u(iPMQEr!wFu)Fx!? znN`x|DPQ`Ju?U7wdq?cA>>YSfK?MUl8*bC%V|s( z^Po8>3q4pQGu8q*BlNyf zJqfNbKlpjxcj3)zJ6^w$ZkMl=HVTTRXC$@)^&;(mB4N z7L=c$WrRNAkOMY16+eiaL`+wI^di4@V5&7Dq2vg-^VYV+K!JX>h>*z{CAPQdPgK6K zZ*fFV5^-7Sii8y2h{R0V1F7JpZidyjgvlqsa;p`QgCHe76vry98u%4Q@b&K7qx^Huogfhv2*gJ+ ztG*}TtJXRGd|s4po5SlYq+snXgrO3Hx^f0mCr3L48nd7;T*W|_=KPhQ*!;xYkPD<< zbPlYDc-{p02X1D02}Cyl3vu@P_IfkNe0M}}lSKRXT6x07aW6SAGRN~kTmkpy=GPrT z{mM9DaR#5KE$$Vw zYwRW+#|tsF$zD}kkMy)uzl0kjJM9_~fAj+>1FbTH>!!n&)n4PlXc;1I7-PAljW2+0 zq!xda2>)%A#kunQ;o2{*`6Nu$sDvx^8TKvf7xNhW=R*`dMLB7|lo8RqpT|g>g~OFc zu)j3xg(ba$4SUe)x3{Mj9H9xfl-rZCJ4c(XzucX4>WX-*X6drP2<`R*X|Iy5Jmnuw zix1K4Ed;wmH|BE3%8Ih<7PDZt#-Y)tOzBKpS z`OROo+qEQBj^m%&Qp~B|`Z2%S^3xQ70DXi0j=ug^ANgmgt(UM!NYupdR`68_sJ^iE zqqA&Pk+u}BOH)x03UK@)6?CP6sLo$|iBpv(U%HEWPW6|3yG;ly4#`x01f4t3c4bz? z`U3c8Yd%xY6udoqV3cQU5Pyqhz^OcC_EswgEEIhNJU zS9NN&^c20%Tm&!GkV^a27JT&CCy)Hga3w>^bJMhr_s_6XOv5*! zA66m_q+Sk7YPRVO?t2T3Tq~~@fE3bP=88Z1OAL^UJJIJt=Jxq($|}5rP&q8)v*LwK zr{wIJr|^*p6fr(dFY6ZO{U3XoC}z`!xE@e|%*H^Ptv}VR0%5FthZt(pG~i2ke^J0f zp-hAr8;v-tsB$`SXi#x84S{DLYT>g_RY|3Fqfp*Zg%ljWfL_UhWK&r*xcK;R2?-wk zktJWZev!E|T}_{>st-&jC?1AlBXzoMO3#3pWmP!SVb4*~70|JISil@DU(*S(!dwdV zl0kvCl8^{sO$jc6oqA*iD}J~Z`Q`f^cuY@%5+)y{pe0C(;yY3(X3Fwr)=wsLR^fS* z37mjjZot9wb`p~zWfdr=3AfGe{Zjq9bn?IYw!o^O+5|P|2S~>SOAJ$X@;NP#gollX z0+8In`sY~Ni6U0R#_?&b0O=~fYu8OK#^GuxjDp%5B~jQzdav7-B8}}6XX{}pT7qxn z9sJSWAr4ySDPVs|P>feRZ@k!?eU{Vn9PQ*YOssGATe45u!)G8mM zYYTn%4mlLl?L(`pEwoQ1MZpD8w=Np2&}v}|QMaNZ>MBrmOAXX5A^!xl6P8B@B=LAB z^*X5X|HB}d@bHhjdL5Pzch~k#q!sZh&~vF{$ziVs?2I!8h!Tf#MB`SI@*yz%N}&!` z5L&NL5!J#N(5?*b7e4*IJ6AHz*pI`FB2di03gI1`R2UZ@QqA&YdCc^XmO;dmO}Bz# zHgXPBbNeZXv-PjyvZrJS_=R?+HUGF_lV_zmY=#Vglu!p5(5Kk5O+G2n9@&`GHU0bA zIm?605vWm+i?DwoyNb&gSPAc><#wNYA2Ycd2wegk-r_kwYiV7JTFj_k)m(3H7mL*`EJS1Ls z;kyYDT!IW*8iG?7zn&?LxGmoB#F-tBLdTPaW!ixY4+@xG>AnTG|#D?z1_hgKR57dI_@Upi2#tCH@ z>YXPgX$&N00NPhqcc&w(rKgqmU3uv|pTr)#vqgMjSWQGpl&V>+u&=~GUk$G*!KCV4 zsH4&~@E#ulhYod=D|uaG3!I{Nh^Q+#sHQdk(!fj`66)X zBbt{^Gt;jr!iKpjRjOCcdSZ$98G0K9TThaLB_pwDFAnB+FB30{-B`F9Sf)=XE6v;c z8T=ev8qEH*V_2S?Z#JPvVGLNmeqJ$rpTNa>$bwNa}fjf zfUb2iq-*(#H(B^E(6XvIV)qY#ivT&85Wj8N&sJqmKLBeR>d&NE7}P4^99HM?I?=REtrG^3(4|f&hVy$W(twbK}o^yI=D8{l_o2cekIaMKAH(B=D*u?ya7NK-ldh9b(TTK z@0fXND*avkTf{uD!$2<5LKUk3!;H%wZP+0g;*VL)C!aC*GfMp<21tVkV@>vER7%!> zh4H@dN%^aVx4Wff9bKjdf)d+WLktpr1mXdA;Bf>I>w&lI z_Wg|}?XWuXq|j@)L(co)A`X)L-!%$CFaZ4y6|}<+cqY6KS?63E<;6!J<~=O5Tk@35 z`!fjD+n4!o{iYne2W}-0GiHilJ&xwftF`$Q{v+z*B+^;LIsC4iW4N@hK+L{OtN1U7 z>OW4Ie|X{07f%V;NE(-Ae|hV}+qFK-L7^~62ny^17zRy~^$rD(wYC0_cA&d4UIdcs z#X+Y>fV9p6!VA3Q>)FpQ;M{gVm>mjWnDS1P!Z*ak`LF`pun_K@Mu`fqM)eK4K%y!eH=um~vdLby417E5ik z3H(kAU1(2*>J8Gaj=X9U0Tpj?0Va`~Rga+Zu3w06n(23(*g|)XgR?+jR9`Lz1|G~c zR1Q>BOAR9Qnt}!;F|Vx@4x}LrLhh#^i5^!zq|<>+w7LN13pQV{7}UpaDLAhNfE>Bm zVg`>5m+P9`tN}111HtMg;J%RUYGU?~Nv?Nu-34Dn@&7r$T^kHhfY^M)Wuc?-8KTBNf zO*afv?+5drV-3R~Z7oqg&6AFyBaIQ30O31+&;kO0WS(Vex;!HVJfOZ2(8$jgjs$m} zxOduePVnw#xBU3~aZ>^zr-mbDL@PQm-*aSw17Nc!aG>a>0s2w3oh9tW1vAyTgv>KS z{!=QbI+q_)qRqJK7<+&1>=U+&NSniAol_3!ia@Pj<9CatAT+SN$op7b$kOVWM54gb9>nkJ_AcnV#Uaa+EODI)jo0wn1Rqw6iY&MFzqndUcM5Y+PIo zmUN5myg25udaYM0(O~g|B`9H4s;CY9|`0op)@rE;L}u}<_pMq6LW$(xPFPtIHC^t@pfAcH58^RGeiCElpqnW z=$B9?Ek_Q3Z<{-1ClYcNUGM_$J(PsTi-FURS+h()yVVuo1uF`aYEt+^S_&h&Mf8wa zng9&c$N9%jxq)D9PNk+Q1Z&%{;$7+(Ab5-sdktbW%cv zB5w}()sL=}nIIzwf;WM`Zg$&&iT?mlCqWi$u2FqBU)!cG$OKC26+$I&PDKr0GD5C3 zf)=Kf)h+!1kPz{QtJM&MXAfZ6fR5B+;j6D@^(#>T#6Tl&A7n$zjGAx2Hm%e3fjU6u z09A3As#pda7r=tDJlq?jdQf^50CwV7XZz6cHG@oSS@p&dM1Hitd#Zc`cH)#6x3(;r z!EoVE`;Av`v!pU;XFC8pXhE{}=gF;a`$7 zJ&Scdao&+M&dIW|0pfe*g{sp0RiPT?9OEvx4_C`FO8d8gY*Ar`j#7d`KafxYIacf~ z`y)RvYrk}WkiZgxG;X}6wlQZFYvmHCo|Bb;*sgMo+pyqM@<;dk$8QAlV7nDFaQ*8q zf?x6*L_9JF^niV6l8;u1SZ}_)&xlYmIV#kw7m_j{Ti@T3VxdWo`qc&li>&i5ST3yc z34*N@_vb&8=h{!7t8_6jZUhj`gXK@e?+8(5D{(+LB@1Hj4sL_qeCkWx{jqOwd1&Nf zBK|jLfWbWu3m;0oG*ec6Li!Q10v^B$$RRT)IgN>lxw}5a4zQVTcoJ8=Ya z-h5)xikkA@N|~3<=7KIBIM_5|c61_nJ4{gZ6n7uKyMe{VC{J-pAIdn*$_Q0OdXUBb zmn`ty2xu^6NH~4FtwJnElqA zE*7frHNbIcRO(Ru{Qq_kqx~GPOKSlURZ0lg9qczn?*TcsS;-6}ow(9BSf^kw8n0an zvqm@si!i8RBl!>LRWh9~Vp$!t!dtdM5z80^;_Yy%va{}qV4sDP-J+RAV|sOWNhU~j z)Hp3Jkli>OfEZ($f13X~0kY@p3P|z_*1fJd`3SuvT@1OkqWGyHNHc;t(()&8A32*q z&SX!12yX&p|HeQYss|*nfP8Gc2ii&eV#*vO`I*%|KhegTeLEt0IS9gw9grmsfC766 zYy$&2ukAa?fz`eF(}7bAf|4L4{Q`Rr+ZdRMkd6q*QCN*!0d>gWdoh+ol6?#C41gn# z`<7aDoj(XxWboQqW#~7$Z-b`&?}iue5Wzq$SB^RNP2iPS-kyT}e-)rrSjYnwGH5As z!9gM#-zjZKe1siv7}Y}q8UO*~@%5zUQwzRY%klWDCzEu=gI|a%P;WIYL2I6z13c2V z*Znk}0`+^KO|nPu0__!xc`#Ary)9^n#xD1bOQKfFzQR+?bIB3)^#sZSvb7Jy7C>^t zA7J97k*~uyg^&5l8DaaNQAi{+=RQIMjKBz0LlC9`Q3-M(cObUNAciap1_kiRN%6!9miar3<0zgG@m^9yjJPc^Vx(}20Nkw?9u{WQMg6{Qg(K7#2 zK?td9Z+-Nyy1(3iq+PufU;NVb8hcEuhd;poVp5&Uc`322g3eRmcd#B`B{nNOR`C;j zq@Jt~yao?!X-6lp@8Mr2B+9L}fSLlncyd-C+yP**@O^?ex>);_RN0+3)Z4ZK3(N-+ zL7$T~WN3%AV<_3tU_af*h*Xc+&C%vw<*K29i^a^m$1{}}# z@&J6d2_w^B-$OiyYpS3cao|sok$dh8C~yPZXEqshz!kST8(gzY}SGwY)L4*4E zhXkhXP-;#)-I20acfeO0+=|@AzfD9p(cLat(dU#Y{=fnK5z>8kG3!*hP0VM%gfW6| z7okVP)k*nPnjs8+oFkoPFkG?~e+*LhfDH=-#M2Rf@Id5GCDJN#G#BzJ7e2btb6ob2pGne}rl>F?g)) z2J8X4(3fwBXJbie-in!a!1HuHrOZ>IcrUl2@6`N$EVgO?>op;*VhWAShgma{K>Z-y zp{4eB-=zl&fC$Mac;VNofvR# zp+g2Ofvn&G>=!6km3s=y_{d}-^!#P2f8Re5na+~D6>ykdbdbnp3Gc#ADex`vATnJV z@%21#9{OIxW3|>Q{ax8FuQM&3?0?H!;zoSziv2M2f#w4O8CFx_Gll93C7^c=qP=8B z(-?MW;KG4vTjlVicuwlihd0F_Zi6>vYKYMq{aj|Y*?Gq|I^jUvW*DD?!1Y{e$P2k= z;Kti<#_T|~)!oOSm|J6?EohE(G$d%jO|{t+@|OeXmD7K?t}E0|41;vuEj6HrT+6NV za!9>;K*Z;W=NJc98t#MktrzebKe6ibQKr+>{sDDLDMu{JxGop5)#v-=CVeFnZJx(P z`Xiv$dOq8Lpo89?$m88<15MSQhPJ4ha-TGT)>q(OfkFnmXdKYmDYgsTgjH*0AROZq z-k@mrRE=E%IOD7Y-J~e?3!t+>XRM`!Z{+yd1iIMOQthmx`P)@Xru;_WgZ{7@N(Zem zmG)P$29{;PfQCM#=sTK*Y!0P;mHiL^B)gI~h@nInykVNu)TKL@IRs=HWX*?@=s)H?@gF{ehL9046@J_hzGjF2+8 zM2_#uyAzrJ0UW^X7wxAC?{~*@;BKZM1o|6+5(w&3(zk#p97Zz{uLBzZc0{D}7lUW;7F0$KOpHCe>a?lLpg-AXIFKtD z?oHCI>;OnAHEznu`C3E48U~izW>R92Z^0SRO!cVrT<;8TH!_@3kLo}EvJJL+m()}T zFm&o^(Okd9i`9UWHTmp(Bd>W6@NC=xF|`4brI($_kUv~;$kwQtNVA%)@Rimu3!V=o zOmgDB_PYiFU!K3f`?JCtY4jmW%7qPyR(4?%S)nU)(!_?~OSi6djspX9aCdCM4l(eh zOD&ZFd0`WXdbMw&{qxi1vzKX%3PJQ%@L4XJ*Z&7^Zyi=uzvd4EqJVUZA|29Qf`CXV zE!_<&4N}qw!X^X>X;4sFDG^YlI|W2iN?N2P6!CYj?U^|<=bf2%uIr6|&U1N)z4v0R zJ3jT@j*8qd(|*JEr05k+F|RYmU9jh9Jjbmi_w~-|zZxO9X=qw~pvc>y{eZ{TTi(6_<6i)Cn9@bMLb@ z8=sTy`3QQl+?zFlkEZFbX%f;TpJ$`T+-&nb+yPzKycTC8?B%T)Pr3{2`b37$=tf`o zHSnWpO0fRYt)xd8b*0CzF=cJ0zw16nKbJ3vG2!Vzw&6;)5V)mx8{$j0s6m1YJo##m z>Q?JR9AdcfQeI2%ZU#YbHBB>I)a$rJ(z^xLz`_58^F?pZr=5`OW z%&)oFaWD*bc%{`?J6$bdrFhzXkNNd!J|pG~9u4TAXHO5rW`b2)H^WC*#osthiI#jI z`j!$O^WL1h2MyeQVgtXfw%^H%56HHE1he}lb+jOQTCYYHh=Ze>en^SgR*~)a!bfF` zMlA0kcBAh*i3jjUVRcKe4@!oK{N)wJqRU~hfRKn^O;*4wzbKwzydyAWnkL*!xNH0z zwmC)O@KG#-;`l7qTx!a9VbD=OY`5SC_BKLIj!X> zjKhYgQV{Tz3f?hl8MBj^ST#muckeyFb-eZ&ir$rP0sedxgdyNuVH((rW$#f@X29B= zKJ!GY0k;+pNQiZi(=pX-<!Jy)OiW|#-pLJWR`5zn} z0a5Zts!Zmt8~{FQ=Wc(AJQD&+zGU;75t4HkGiD$ug=!43*8Du&8l;m&loNnv4j7iM zz{(rf{s$IgnEl#!tJJ>)p6n>S9qM}WH7;Yhlg_Vinugzge`7*O)3j7nrnJ*ch&9y; zSd#kV6pxl?E?z0wDu7RSaikCjV*pXYu$hI zt9F$&JF4XMyLBj$ZC{pKvA5yFW+W_cfj?vt*z3dwn8}^%cN7osZKB1kqyxW^J+o&! z0uhHqFIguyb8G$s19~YpW^9>&reI7}M)5heh;h=AM>Ka;rP#=$q1_BxPa8U)edQAw zW#bXmnCkb`rXu3j3ry&WcLi^jTykTRRc?C|so|7eS^4P5rSZ2Sn*e!>sLd{8$ak;x zM|atptlZ6fmnh%T)WO{cb~S16rZjri=|95*UnaM}4%nb9KKCYZ-017XWh{TVqCOe5 zBvC$ueVCcN=_EjQ<>n2<+B^!{r)tpbc|3h*H^3k9GvgeBky((8hGUc9R?1Vqe^2_x z(m`7~Yz_FQwNDU0c_aUCmA`SLCzyJ0k=|SWL)qfcKJf-^ z$HvpmU*bvc&x@Aww;Rx;VmHX_{0iH| zb?=p&(wEt^|ZY=UicQ1%}Lk;Tgk4Goyl^! zDQKnIJ5WEF<)<>C2JRK^uZTw_gdbO@SZ56;6;YW`!aSqCD65GrATv`@DRYp3u#$aY z^PVmm+`R?TYe>hbMG7hlLJwKhmgu6fC1m7FT}XCC&uj$nk2Y8R5@Z-8L68w7ngu-> zUVwj~=3ncft8hNNN9!p47W6*^;2-3zrmxUdSQYAXCq-EpT8U+afHaTZ8=>{yserV` zahTzu#p?eO&j7fqc0CyyX+|M~^nZBU9oO=GSWS@6K!)&L0@I_#$Y=PsxS#*OKM*=! zi%Wn9E*@@96Tl-YeQ8w-gM$DffIv*=&7C3Oc*0pxQL{mR0$_G9sRj-W;ljbb*q%H% z53rCu(D`l)^6Sez$uNz|HP{OF8#V_11_%VqLOw9Z$Um3J6?{cGf(7#*jDyAGb3luo z#}<}4oc734!|5sLgWdx3%`cE+JI>TgKq}YS^A!A)fOATKN>|7LsT`i`I$)TsvQN3O zws^63e+z<0)=7L;pGSZJMr&gbL}{6=VqbHkd4IyH+`8Tj!!BSocV{_s0uE>d1NLA# zeAHg>1+<`+yZjmeNF=yasCu0Kn*EaC(4;Kl6Oj-~{Z;n6IGFfa*--!^7dmKfhfz#0wfuc2yRs0qdFX z$&ry+11Ox=tbSayKwcFulT>>R?uZ$(rdnVEr@Gog9mmU`Yvm~dPBvTZpubA^eYO!H z+ZvY6&>Vt4*4(X~(!i#8Znyz_q2opLK_^4{a@({H(?k9s$W|}8Sf9IM5Ajp3vjK-R z?32YikUeD)3I&2WojF#EPUK$F&0qa1pI{Sr>tI(+wGg2Nc67o%=bH1hxO9A&R3CSc>T&EctMOC7hN$UE>@rMGkboY=!PfZd{F!lc}y z8CU|RUkA`g$O-1-Ip#sP#(z9+=t%CN2$2*;&iIn#2&}}zIw#9^uEMy+8JSE+P#jeN zk0B1%CrWgT*dZkjf*{WOjj#t`iUFYjNFm&BF{=qIFapSg0Qct;n>Mw5U{cs`r?8{4 ztn`Im4{L*M%VmnH0+V4T@p_1Qt8?M6N_zG97TlccAD^F*U;&+`xZ^ngaL6v6uE33F z7ZP(Qhv9YxVmEj-t-k`VMdVGdQ+&S`WWW4<>aEp-%YQ)T+SQ;_f1?(>-<>k2Lk$6b za_flXf5?|RZWUC;S=0V~V9$qlVrU&8DW2 zWDdugUZLujKf>{Y!ol<Y3N#HW%^s1`P)Ca?Ky zKf`ktXGP{SD5ndU+{AtYAIJ4p3}@f?CWiTcUCY}8vnGG!a{LJy>#d{DKcUENW~_-t z#1kdP&T0?z0WfG46vHic#L~^ z1%v&gd|Nk_CX*uaxM1s9eU-;hiK>_jdEw$$|F-#tOva#x4qema4YH$2n_Z00eS~4x zGws#VEwJJ=02~flO4d)OZL|8ArsX3SiMQN->8sTqo45xCCKb~_szT^#$JIRsYQ~0y zhl{B`HM)I=Pq;nnVsV)mL$$G|edqcW+;+k)JIIg(pJDC3()X3)B&$0!E4i4^@;3M4TrJzgBspnaE46d0KoYx#Q$Ha{o}+a&HV`Zf3Zxj&`*4osBPhi9f{_ZBfR z63#3MdG6F&0q-gw>vQ-d$2$N(!jmgx)zHzLb(s#@EiiM$l5qWZ_CtTs`RSo9kj?!Z zQ33@&#uoP^-x)0mZjK>5)r6rOZ2Rw*R*!mt1)ryD5S^zmv#tC7cIpc1?pqlvqXvJ8 zA<7vY#(Qx*d%u|32*k<`Cu${R#;=FoC1p?YJu9R{LNi9Eq9iRwUOx-$e*=iIh=qQ` zYUAD+EO>Cu8-D0(tJSX8ykFy`Blg78W@&es`Fo+k(q_ebH_Lc(?qUTT zu@?A7M6^h?*m})h%T(o3G{<<@#Fv_SqUzdlo*>BuUo+bHY1dDO@GJ9-m^9wRmba2{ zeQsdfxcMnNn3$1N&$bReCY~+H!(GL9FKh_mp)<739PQO%mZiwZxuphhTO^JQ=8~!E zuU*w*Plcas|1w2>I8UQp&xptVv`0f15gOC7Fyo2C31p6KiRotL}m+#xE9`(nl~C)A0&y_3k+l z_jF%Ty~d$b`g7S%S#HiJRr#Kqu_>JqY<_9TTL zkCeR##aW~Ex^4=BUCVZujv>G_?hE(}9SYOxc-}+?U-)LU2H}78&1Bp7x(5?h`bS+K ztFH9}+l5G~imTFSrV=Vd>3$hkYVo0ffrg^44}=)(N>94Qs^x~_m&Zk7e`Jiu#{^7{ zPsS2TOrSa(r>NAv6>}jV`VhHt@Cq_hmee@(3q9TOSQ<}Y8UHU6yD}C2nb=j8tM;qM z`@4zbQ?2TYl>-Ic6=aT95ZJ}N-SLmWu8k$dzkyvHqbdJ$V3#rRe+uk6tNgElU8fZ> zc>k8zH6ZwJiCt7HZU0STSLx3$e-pdJKBEIeV;^lK*MkI(iosUq=35D`Tg<~7ko)=b2;bGsgt8l? zk;Isv6_+J11Q?s_l`7DFplxU=W%%B0)K6C&c15G%#e6XZ;jqso?~OSi;dZ?6%9FSa z6S5d5c*#=g0l6BV1x#intswhB6bTN63HAAA+HVP4Q#Hg@&gU_*{+oO*R)wFhFUou9Vr4_=bJ#bqbf+tEkivn?oob((&a zD?V2{PP;%&uE?$hcEgTjP(85XKER$03ahC399A+S-nht5FxNz{^!VG?gu%0I`MWVR zk7bM(KfWnJ{Ln!VQ42WxYcMQ8LpK=?rLBB5{ExgY8s0;=dQV*OwubT|lAGuh^JDt) zvc>t1*5otWuZE3=%U%_vjdTyM!lER~hA1Fk=dD~ZW0MQxxUVbb`orCyPKkF+&Qyfo zZg-*KeqT56JDZS;94C4!{WoX0?wIa)$wZP%4=8S0Co}q9qOR@VdWm!g`8&3(N}0I& ztacO!AyK6g#Lk^9PN^!1oRBIkDw-FwWfaRTTyA%TRhIh5LIoFYXZmO(pQXfHe;bn6Tv19Ekv{*=&Hb z#kq(1{1|c?1hdkE#jpEs{D9%f?cO-gOwBtL>Cv4YfwsQx6Wd-jNFY;24kX(o6E5eG(*#_)^|>rWaYs6d0tHdC;j-AS*kmuzk5ACq&t$#><~)XK{%)r%F(16th8}I>XmE?_WvCCwWRl@1Lp7(y6nTj&fm-I{6dHCnMIwa zFMPn7d1Nc2 zEkHPQmDC@g(GwDA!Pn+8oA#J5RU++@iHZ!Po=oL0t2+Ao+ z!g)><8C-xJSghZhh+!A?numGt%Sg%17BrbEfgm`Y;>!5wPX!Un?+e6P(Q?5Ah*qRo0`=_trb%>2Gr16@D82+_M@ zi0K>RBXUpXClG_byySgCtuyT=OCY&J3z*p=@Tqw-1*|(jpbWg-nc1qpo>Os>NR(UW zeK1L56t*XlyQl@q-U5#lI9-p;nKjo@#i@KyYc+m^TLM3}%RJfx%1=5U45V_us$Q z%nks`V&n!Q%r$36@eICqUyJ*aKs5a)*q#30Bosu@>=m4V0`X8sMaUN z?Snsu0cV}bXt~k;e-{n5f4=OXxa4)Zbih^2r|?Vb?!S!r5iJ4wC+RaujS$P zl`O{>w24eTCGK}4>RxcRYQ4NgxxmtTN09!i0-2A81nxP(cNE77$uqIo$%`h8z{!<&E z9rq$$fpF5YJrZ)kT-VbG)~2@}j{^|3pU(dyl()1*Ml(-!^x!~1Gfi{%SkxU;oUd2& z-Z4+>Pwl>+Lx)5G7z!VUUAQc5f$>ar7&MEr9QCAJt~``i2bj+7bbHTamsz|mXjh)~ zL}u}sF80;h;xH9dh1X5o-yi+HI#031-v-$YQ5{KzIktfE*MYs16G-8Hf_vH=Dbz97 zRQi>APekYj(VhRCC3Y6?(@QBa!vFg;F#*|sq>1tWTbP&vwW-bdZLw)x#xIAGdYW?N z2Z}5gc`R^^HMlwMZ9BcGa%?kl!*t#8v0~PI^lNf8R?ysM?`34I_FJZ>6vj z1@A5?#m|+*N@ts>n-#TD(~>8VB-dI;20t$%OH7qCKll18`1D?xH)h+p>cV|{A2Jpa z*qY6E@$wUKPk~c@^Q7srEX=8y5gp-&uQrq7C_8)`A$GY>?M!$qioJR4459CKbkJCV zMh;kXcX2h^ z(NqZ#avo-_Z5-1O>Ses_X|h}znLsJxbG3%x-+HCO@b*1O#L60ea$aU^YnuS%F?0MU zWbmi9qK34Caq4Mz2`u(khg7$GXit`;o1$8t5D$m(}>r$3*5;{Fu7~vAw&J;PLyKo|Ouf#{g=p~K~zH7vW zVU!T*Yp&3cy@xf?Q3S(bvL@!?s^oU4&5q}P_V;W^iD&v(a714JAY)pkZRAlNSaN1s zjV_Lq127K3= z#Ow8BM02lr27uybDd$@i{oW+i^SM&gw#X)qObq zVrd{N zja z4PtwXp0j`Rm!SN6FO*&N^wQ+HkHMcoWE*{tZ(A$Xr^BVC;wX zNH{w^`Umuea|zmgi_i-n1-vsHEB_m1^THpHcL~I_k2rpiHh~0a0#X}cJNw?-1-v*e zweydFkI?!6nsJEql8Xh@F9w1je_^~?2YeLvyI9pvZ^_AonxQo=O+9s&XDA_^i;f2k zX~Xqwf94~s2TbfCoJ059#$^ACY%zj);1DWOFZ4iEsJZ8;?nr?~=V}cEU54koFZB#$ zB0-mU!ALukhR_3W!*K+UE`Kx}e(_%gj2(gu+=D*Yt#e1|#zr8x1|{#WHoaI#f-$%p zyas8sNjAK0*@W3A za^1RHO14}IO2cf7!zeP&9Ue%4D|3)#7AMeudB^#u_HeCneuq_hcbaSuSfs>lv339? z$P#r*=b^b0LJF3!3uOG?2PKSfHLP-T-1$kCqx#>ptK!e~Y|MO^GTBR`A~M5B;ku||6Hd*gN1%^l zzbDGR5T?(W1^Q2>RN%pJ*)%PY^_<^x#8m-q}TvCr0e-Wkb{g~?KtGLO;R?tdFwcAlD7 z^^+esiXkrnL{J?0Z($8fV*+QzM5=(b-9RRa>k}+3Yy@;AyS#*l*sy{h8h_!)(Ss^-*!1@V1)ny}6515HxnTMc-F|cI$kgM*Sywb#W_v_^mswt%C2v|N1 z$M$*)4dH?w7Ta@N0b57U#t3zZz@@$Ir-ji~sW*)Y1%4LV>-^wRH0W^gpRIY(EwPmo zhjTT#@B~H}1}VybZ?IQwNNNpTS#~n816kbpUTk`zaNA7(@=FF_bz5CSBFz{dH64r? zcoE^<-Ek6rnss3U0x2D7$RL4m{kJb z(Vi$KHrGjMMQ)}dPMEG|)?3N7a+wbJL$Sp${h102Inia=0ITSytVqQth%|h}dTrGvNX9V_0-_1!z!V&cEgvvFP5l(K3AyDQkk;SJyRI%$W9;cJt@E5NtND?%Bw(r%zVs?#L$d zA#a&O752Gs9F1ahr$S=)L%Oa&k%r8fR4DA{iL-UkY&Zh(p~XS)*wOAzZqMe7+ ztSUO*TGk)8t+f`3bAj`F#D0T1{6&d1Vq91nZ-t^jc=_t5^Z^ixgGFEgxG;pxg2hJg z-$N-Kpm+ai{IK^pG1p*hJNyjwglVqIRcz9D3kK#$+zYtpL$91t$T#N&N8$@!30dDKu!CA7#?!vRNy~u z-rIA{g=XsN1F+>i+9h+mp5*{>Fmv>_jA{vQAUX9&ljNo19M`yoo{Ns>*r$1+KYB%XP$JhVug2a|#5i z!JP}eST*Zvazm!?avGxisvKlngOCc+mzJNwF~|JX9s&ZsPY{@2^a~`Jj-JcZCGR|E z&nCw1v0n=pqGRif(gX25)GP(g4=_2jxkNB_aQ(U#(`f+_e8+Shjlt5=&q%XtbxI8c z0&fjeO{Ge9m9>&fo;0L-o;_j05;eWrP2EkNauhBnDN2uJ68rh3Par-XwfB@56Z@Sj z8UB37w@6hiC!;qLZ%A`7pNoF~N(C$S?LFmC3Bjts{DTVu83QdZBrdqh$?^;XHfhyy z`Qz1NxUJ^-I=$>RSuV&CJnFVjIp*i$(=DdBY&oL(PNIo_Y2{bHf2Qdl^X(wLQiJPA z(1P|kJXFmwOfpq>e-60JI4i2!C~-eoIlF4AV~V@PfFZDia%9{zSwpSVSZ3NJk$P(c zj{AR4!*1gfqjl*UlcrESJU)3$;=NbxZO%28V8+wjwUtg!+Bj=sjN%;ON{JC{^Pm5? z=JAGi=uOoFowfSNx;@iHF}>=0x^lweoqe_-MpO+vZf5XbjMvzgs-HdSSYSSxmbfF1e(?CZ>^F*KrEjHHBhPo$C}}OuKVr%-Ujuh}caD^= zJ4ARsINXf2w!5cT)?F}$#48`FifWs#l#1#QOM+SfRBOY9a%BKpKEzQ$xk?JdTNi<}>IPSNOR_d|)hB6$c1xsA+`rTb9&@a9$agxYA~yfuGiIIpY~T=N$0PND3)lGbWzyE?={%L@?2_Qlqt zxt+fS6NT~(7Lj7iiF;+0L9}J_hVNTfLr}0&3p*-C%UU8UYL*5LBQV*RFMeOvBA6-d zJoyGyrk43kilCR@aNb$iJlq!dDH+C&t^u2Z`R6CpLf#EMiW4`tzg5pVF2D6xi>)jJ z={UeQUetJ-E{u-reFAqKdK`^xSB77H_!DN@CJWW+dW+%A#JeK8R&$3owiwcHsOq~J zL^HO%SJ0kX6oaeIJ#x!y_5CmbBiU@z@^+5LbJ%VPbAM}oQ^ie&L$0V!HvGTYY>*|; z_Ro}oJZzRtrGMrREc|l;(!wG;%=>3%!GB0IxDrz65^mGXt*19{sX})obx*NKiY0#U zh-785MWCkd%b}De`JbzLQZ+wvl-z$nn87xANTK~EbfmAx0-rxDG(Vs^W5LKG0t85lITPuL3|Oa(B%THxwF9>%N{t30 zl(r3LcMy?22l40bU5(D+@x*tte^z)Np8n)ES>OxWwsYI$Te+2=)UUF z9_obtuj0QI4^hS#O&f;qRCLmP<--@40Z%@6e;21ko0DO0xQOSIqt!`$bSMQ$2sTNi z5z>kM;bgf=xvMU8<)_cKK#V{9V2LX7uVdK+ORIAluYEgEvY;}9s5~;y6kDLq^!B&E z?^-Kb(k1z;>9g#5T($kbD=nEk!+JS^i}>QGTzz(u7;!4K_M5vS%i#OC6(1ys=I5A5 z&V&@UXKky1&gKq^x|SqUz`M8YV|nr4E5`Cz_LC(I+XfWWRXD!@B$@r+it9WcjlnQ3 zV~Fg^aKDGZRh5PlAi+xK2r_hGuK|e_*9Za$0pU{gbjE;%VkW7s^@xCZV0OUIa|bnL z?cj`C{L<6UZ;Y-hRWC>g?xp(dfz}t9I)HX4h`h$*BBum&SVmQ zCaJw?Hm-TFAqR6cLB#dyc*WORx3B4dr~*g?P+z4qlRXmKX1fKa0$dbzto8bC^jZEr z1W?yXcCB@T_ZR7^f-ot_2~cT%_2m^b%zsjq#-^73Yzx>i8$S10-~>+_;B>q@zd?F4 zR_9@VdeH$w3EuPyBy^?lBX8+3errc~pmxSpny4?t{RO1$$^=Fs_bq_z!c1 z>%1J>7{~z(-rOObVJcECSBoF$=~%wZa**Myh(ef7WXIxnf$4cj zdQsff0&vH)~xX$z}nLnfC+zh#$%WW^)4+jr!PwM6|#m`_LY|YYp;N($x8_8jp)h(x|W~l15 zLpa_doaf^M6?c8CY%9r;dB>;SExIu+Ee+0+V``o?Nl{x7;PsGVrfbK#m&9%SmOH55 z9!41M%$t+kReLb9Fz>*S!|{q?=a2b%5I_qyQ!mdeVZG`%e=M z$vvNIVW(#newJjOQ+NCg63aZR^2({lu1&7#FSXRmHfDmc83XBX@Vf}A(gOYj^ZO-M za#?>e2)AWQ&^KU%bFe`o@gmGQ_18dF8t+I5xkk#UV+A}E;rhjsYB}tYVFtFHqx|cR zSLKqvgPPOBs~8koVNFbCH@r9{M_aGFAE`w{XmgZE!SVeGTdnFI4iPOwgA`aD%j$PB zE;C6Wo5yyPB*Pi^HylVQ4n!A@LA;up3}^nmS1=92<9hO$>lhsg*(r{xe;8q)IGxww zAuE;Kb4>^kUfwYRtpf=3bstRQ$wLF7k+Q>ZuBOKd+h+tn9owjezHQtNIS2tm3e;Hv z2NP1=-LGbEZlP$AB@UCc96Oif5L~<-(YSHmlNswU4|!@k z>aX*SY&-q*)aLQBfDbu(&c2IIrhsq=S4oeR4yWu)QR2^2Js|C$hGu}@^1 zUe$Vh^@1r43GnRUC@^R=Y=T z(({BZ^CNaHZf|)oPc)nRXZ*Fq!; zJ`2ZolrOr>yy8K!Lv#$!32sGb@}Pk|4Y&>-hp0|>bpDY%0~037PhOrA-9?L(WET*# zd>qn?{$KeDoqHdE9hDF%0Q(e(K_qP$zoY+R=8(MRW`Fm_=MQ){QpaM-v6vuutrzP? z^8LwqLT#O9?k!AW-I+8EQ=fOX2|291hKM2wcUtjJtn8@$L@f3zGE3ob5TLBk%hqdVw5JyTOVDLs5LQs8wvbwJ3 zC%nM=m2G4q40%TG-oWq_4B*gwCad$!O+YDzM(%3lFVHv}6j#Eq()A5wuNwrDY?`Am zin$S7faZLLSgIV18)kc#xi|aX4>_EIraX!H`i1;E0yqf(6?tS{0lSdyQ(n_srbe`) zjzADvMD@eUG!4w?XSUJfRJD7MFg~~$m1-d3=GZ3aYr^2MFV4_8d*}4zk3=%I@<~2` zdL0<;E;uO{viR-&HZv?9RRG(0F0?uHsht$wT8wdD{9k2-M*O=Np`e|5K>KyI%wx2P6_E|qlVm?{LA4e9u zF!fpmWu7PA#XfweABALVBpyn}5L?hO^$wqsv+aE@3GQD1_&d)M%jP&CCW;0Pu%y0& z_!htYRZA8&|71{C@tM`9zKp~Bl?fY*GF~%HF7Jz?3ywXp#Pi0V%#bSNs=;vCZd0`n z_7G*CFb)rZL~fSfTnw9d>cK=$cYg0DV8VdGJig1k7HPk)x=ngt?rxzd&n{65SOq*f z`c-TvVtRCZI^XCcsE-u`$&cQvR(F_Pib<|6(vG}oPvcwpn)yy~O4!YHAj9PI#cL0Q zRieQaSiY3GVC1Nq2|2Vzc{(IAr5{< zaI2)8fG?CGcNPANsfxwWaK6{Z-kx26JJFl(TjU>{Oa^?tiHbNyc}LszPelWMv>;a1 z*pSw@T`B39!p=H}>aK^WDU-^;*1jy+Q(I79Tb=oDAnyXMJXLdBb(TplOsDQ^7-ACl zzfX$5p~9)bv`**(6of0^FuY1{zJ!kVi40c3BJ)mprh^t5+(y9ylB}rGfd0<^?N^#A z{J3m5V~5QH`hEQ)P491%`Q=VoO`NAILua^4<{d~J&ygQ6^s**{JG*x;|gZiYO* zcIBE!H995~k3C-6p_heLx8>qL$|-vvH4Rlr5)1#zCzI?8ab5vmG_Pq4Roe=C5Q{wP zK|^^IYz%H1^UaCqX`Uns-Z}-%U!v+f{O*G7xo#-^5Y(w4yFeEevx_dO;dsi^+CJNh zXwWl^iNTFn;oosR76kEgMV)u0XM!=3fC2dqNbF20 z%#<*}1G2XmI|V0v<6bFJLteGZ4{MEi6NXuW#wNsxLTH}RMOia40CIZ{>(g@-GZofK zfH)!i2pB3-Aoq%$9j9&Wa{rZm?9Ob016E^<9vsp_JaL--Do8 z$vYpP`EAzW+yk!j9`Npng#2^Ym&PEOHX}Yl3=9^koATFnDj$;osvWDq{u&@Wj8dIW zb%=I=J?OF+-@IkXBDfAP82|Y+63tSIHQ*yhNt82a z=?Sx{0lqo+f|*{)An6O*@>@-Ju;4qU$`&|Z^; zrA z2Lc!5wcLe6F62#CUl7V?6^DvZbcJSzX zQ`l?Y1+x75O6HWzS>0u-f&RkbHHOVA%ulG6+YvxV)X@NvI*$V5yRXOjKWvRPVE$H+ zy zGt8)YCL{TZm}R&x279kYM!CO>;=Lp+Z)z3!+Vx)dHJ(WKKL3iQ#V?tvZU=K6I)5P_ z*}hHILM8$4O`OCNc1C&bn1|B)j8~WNfBVEbhU4~KLvxsJMdDi>-;oW6CLDMn=?mC(=xNf7Tsdb6r=Bg*`ykge@EPi}$Kd;B zEfuCYO!S1RmupHIGR~nQ-?kiLw^{Lzgx_ruczb0LN(1P(=Lo;=|GTgFwM5weEuEQv z4<90*Nxg_nry(*>rT^?ISfJXrT|#Bt7bAWBuCXGyvW+@kj^f`J7Z)|pR>30VN}+Wd zY1MG3fyyaeIAryvgY=e?&n)w-=db-WAG&fKP2X96g8(E8KNxA2I+0EXpP_A&n9qcGY*ljSHtc2q6CV7nmWX zrQ9RYkd;7nC0?mGL#h8o+cs?M8-SBKj~0(YXNcJq{AHK@ctx}VO69-IoKf&|*wK>J zKM%h5&cZKuXX&cm`m{7|;RP^8AsF}SsCq>_M%yatme~b_V5XeP@|gxagnpFpz-L2K zOi0KB;R`N4zskEaFGz_>emn>sMFd3;Q~0J%`q$Pi)B!(Y6ZDVp;dcSxWEOw6st|8`BIa*N+|naS)s>S2eyYn zc=^&>&u@C#k@Uwy!;H84sK6BH3H{}8b*NQYH!`#btx~}z``GvUiV52?93=$lTUX{& zK;oQefvgmLlbPhld6e!d0+uWtf?!0<&v^lSP?j+Va(HW`tQFp>?d1@ER1{gHa9!qg z53@lISBvKm9Gb9n!gago&nEjbQr-LF$neLrVpstnG$=BtgFHac194A%3NDS|wjjG& ztSRq<*Q9)k_h(ei*??_1#q-WGop`C<#6-wv>`kJ{Y&fTTkFFCG9pvx5%QmhbB%#4% zI+93kGdkgf&uq}QlRp2}UV;1mc7p687ZU#_ftK-fiejRERa`q=hb#x#bTXKM{t^H7 zzx~3;SMfz+ z1D^%hZ{Effh+7L%eV-ImqvDGzx~vNZ3}wL(`iq}qIrDFA2rnb!o5rq1i7)cw%Gj{_ zG8OjZqxt7F7~i_jJX=OjX{1nhP|r9sps_H;+W^|lLL8iqp84cS;%nJMiye^xRW@v6 z=A5v7w4tOz6h+-)g35;7>;8&_1xSAbs=nE)&+J#>4)B)1(#@6T{@WBMm~x^Z2Ez*K zR&LCfcb+%A5YFV8YzPcCU@7(mAx9#+2F!?JS@hsvghNJ!Y`Ws%#($o0=bOlwi4f$% zoJY(gaFT|)EIgR)P=v;H;n{{;E7c0lN?L&YZ&`AfeM*6V2Kf-M$r=}Tz60HPdZI+^ z-Zu??+mCRV64XIj?`by*!16zXi`^ju!U(&O!ZoOtE?fksJXJh&9mxXDHTs7OVnu`k zF*gi_qO!7591GO2y=#nXV6C)+O2k$LD>jg!cvncAwHDLb*Y4p#}r?buK#(zu?Y?IeZ-Oq~~K;GFD*0S-9`Bu*D~x~SSI9HA0- z?cjnR{e#u#pi?rGJesI+G4COO$XvG?FTkJQfZiAcLnB#WF5H0gh*n`}1Xo19de+Yo z4H=0z`f*{R)O;A`Ye7F`Attv}nl3K->9x(0eauDt*GI-HAG`EtANxt2>rLjW0qEF= zJM|p_ZH}hBoz?^>Y5HI~Wy;P_e3J%K{W34+$}>`jB_jq&$&85@_I!hdp6uo2$)~-D z)lsmbs-T`1zS_r?5=G6YKDNy{rkC3@U1hJKlXSHC++9n=5}#dr`Zzb|{_xG(P_sR6 z=$OiMcy<#2VaP_`%!oU#9vLK87Z1rq#!DGdA`I2xS>W|xUuJITkb^q|){D-K_fY2# zx*#ro;*V(jR3m^i&qX%oBWw;Kk%Ej7NuBAo0bqk+F`iD7SXXh!kzUBIK3bML#fi zJGJb!%IuNXPITlgW;w$S)_2R@T9A}eX!7+S2KqRE~C$1>qg+pvSxQ*i^Y=Ixr^VUWIWy&U$VJ_9@f{Y!MJ6)-&I_X@0iLF(BxT2I=d%Gc!-@! zBxlhUR94X_PE15o`9r5g2h4p&j$vu|tvAtKkUDY6(Fwplpoo6EEXn+F(@IgfIzXH8 z){AMqx`^x}wSy8Kx(^74EJpifUR(4^G1>BQyZd%Bhdw3zZ%{Iv7gfs@RWyuqj4sSf zB(ku}W61w)3`Ec+*7&2=A0iC|8yW)+#t^!hyf_1$D=LEJl?xcXJUe`U*N8~NWgox zOp;>y&Jiv5HcG}vflD(~;&V&Qs$H4Bbtg5(^4fZWej|m@PrDs`u{`p8)i=Bq*|)a2 zxLU^l^zsHOLz09Pwz&8guPLV;gY9}=!3V8`V&&tBD^I@)_yDP2sca%eB~|tdLVfm( z1P1#%thd2Ox1@Ft`R+{+a>7KsR5u}iNylKBFXE~bdL4NHm#UysyqFqkuiMnnUK{5= zK0+?y3mi;SO|>{TRXe`gX*l(#A=UhA2E&-|2dCG3+qX15ocng);>`4}c6&;Q_1AH< zHtqbx*{TVyd|XonP)M5{Aw-bj0I zBE;m8nR}Gb6UN*W5CN`<-BEq>e!r&HwQ3yTzMOT4C_-Jc9|s z6oMaE>SdF=pC7k^#_LdN=y8vv?qL$r%~=Y1fM6E?>I zdwy&Lu8b|n=7O$d2PH6JGr`RQf{~^2=g94SmJWg7L?WS1cDj>M;Mqoua-j9S%x8QG zTwQ>By_U;Br^9vt1Za6B zIY0u9iT3Z_hD9fT_P*k{#Lv zB#yi=HRCq}(}KBYK6uHdr^=yu(gz+rhqo@h6Pq^&73r~$n|Bnw$z-K1e&m4Eo~n$v zwSQN4HwUPc$itC*Z2)t&A6nDKz{b*_H-P<9eEq1I^-x-Q`b*XA?j*)?m^mQo>R@f8 zQ)cA$X0MGx7-VmG!e$jXCk~qyV9F#ObDvw`D4?IstaG5Te^`zjIt+olTR17Tzbjx3 zg17)Z*dg$G|HE(o#83|XSVQ_4n-wfUwHi_oPrdNY^PB4h>aflwO5 zC$D?Rz`k7D{Y{>191iA%)fBpPyY$QH4u zMvHb%PXia{$6@TQpw8KODnf+p>gw8Ez7X#V^BtqV=r=C~N3FBBgJpAPn|OXP#k*st zUaXSs=P1Y=ZVU?2qqrWeMy)PBk4>j!1e5~omyZEk1(>no3R`b`%w?Hkrg_m5Xb0kmtu2;;Nm<~spjDA z50Qe=EqSPp2pCtHcls)Q`Lj^`oN@k;tcB@&nct!b*d7gmwybx3=}*RjPQKD30cDlZ z{@THHC!o)7^)3Jc2@JcoqgL6r?SZnEsuigr$0T-^d?mQgx=iRSSQV(Kmg8@*3HFmG z-w%6f1j#gy{QP~$BA9WwdNXuQQVWEPJ8>hf6q~>Vueu05JXcgZDcEY0WC*r_ za#%GU%6i)}`|)B$D(0|iB3x$?LUIhtt0(Z@syMlsoIlJ^_EJ%kF0&a#j=LB0<$cahXY!(W? zK>P{xW7|fq6uFnWtQip|?Z}+`6zB#r4`kK(MM? zu#T+)lKQ*^1{F5bxHE`V%(bWm0``ZL)QFZ)FG*yTWcwF(vG@B`(>zgkdkw6MxD)62 zl&~NeO-9EG!eqD$ZbUw!LanEMMQqd`y|u&lh#$+oz3LXE*Dsu*Gp{oTZa13|{j%=2 zGn}MCFoyK4>IhuJD=){>BF+EMmcPch9`m5`QSKU#7_DTf_i zt4LXTS=iN<*84*KjjssK)!wsx2tc&OEacoGLg5F`YPoEvrnr_%KEy&TJF#!U7BHvW zG;u~C4G{Y@uhtWVEYv*7-DWI&7n-l)ibvshhQWdS`on?IYhv=J&1_zg^&Y6yn$CI7 z5TBM?CKkBX=I?JOL2XjNQFjJoGw1(l@5;lW-1j(RvRpzqL=>_mS!$FeB82RZEeRtU zOGlQGiLx&xkuBR8L#jdIl#WbgiODun%sH1CRQ5!ZwQT8pU-v%e&->gz?w{9x^SqCF z$8X-p@ALb7KT8iu;oKKrvvce?(`xFOK$cGx{kX)NXlMfDZL0I!1cC0Euf)!t96Lix zr6sTP%N4pyQJ8!}@Oy~u`9ra#HE7Cuw9jiR+YyX{HGcBzyA_Or=a%yQ4LQRXD`NmA z4sF`=ou55NnXK9ED;2zusmR(`neA|PRNKE!W3SQsZ(9#if2F7H2#xL4ozEOJ=v%YT zyF>0rKMgr^T8Lb1C(Ef+Uu}?PJv7PTU{aGI-;_ip84HLNXY#+gNI90|HR#(k*E0+m zNk&>YhP2#VAkrGEOc5MLThdZoiZ8cO9$Tv@ef$z`I_l;(C`B=sZKs(P9DJZI4a0*5?h>v!t z=cQE|3ERL+4t`HcuWRdUyrv^x{=$%f`}0QCs#s620_jO@nO!Oet@li9QLWr@6A7DD6dRi$v~CB>uXRF zw+UE^GYAA?9jncNl)=K!J$IQ(_3g@bzeH7Rm_JziauEc4JEpG^mjG9<%rO~**8pci zXhSR9fX(`y&g_QMKGZuI^}^f^@k@h@z@F~bazW3@j;jheiRau%|}O^n9afO(%;n|CDKyvpW9vcGucK^Va0><@t}s)3~wzr=ZvV4uG|! z83K3n>&l=2g$%75py&7rs2_-)mr*N$)1E{_Xfe0R(E-*9zl8Y124GxAeDlB%39bd3-2<_uf?AV|XMo!T2Ykp*o7oUSl)Uw9k4l_NJ^ zF{g&@cUZ}NeAy6Bk7MFIIi|=&-;V=oU&>k_ zca1|5Th!(U_yL*1K3u7nQ*Za!nHW{4K9Dv=+4+GvPj13b!nv( zswG(9J0FuS>AOR=2DLD|+~w1Xl}2OIAe778DsP33Cd(@~w+@G%V%$dySTW( z5yQbc>MWp3c!C(A6t~)bABaA5qzEpB`ydWLX)ituOj}b;s)LfTV&-l~p>THqQn^b( zNd~44$s;%c6D%FQoXdCuW;0$L&>4*RzdDdnM=4gvUPqZe1uw`OtvD2{*D;<}I50>N zrC(P1N7q5TSP8Jlqig=bC_c@R_I23&toQKJK>k-CQC`+F3DcRVf4ccKg0x}I?1O{z zJs77L8gF#^ooa>^0%ndhe~b*h;BjjnsOTSzWzY;2)GZk|UxFtctb0Uphv&$MAea=X z-uTfG+HFpU`34R*Xr%&<=`GMoOuYsp&Owa1&eUQ^sn>GR?(aZ9GLu<#AfkAX5U?Q@H*lavhImTYQZv$=ndK^W{RpAETw8?~@2#|9O%`S0#U5Q* z7{e?)YyOd*Xo>fvJ#W^}2vPt(@)2Ho2s_ok_$H-Be&& zhYt6ah`5gn`>WraU^xonKz_AtB?gnGtwfB7gDkJq;C(&ha6(>P*v4U3RF$um~#u!qh-VA6>#|6kB zp-n~UNLHDpWY>fWaLBw8x8*H1F#FMk+0h^yFB3MHq#RVXQJ|=Lr?mX63YoG0#J-OA z%b!xCvYw+GI#ZfTSL*pqvD6gt)75+^+1Yx{z6jcg+X%aRCMMaYqZr%erEl56yJU}1 zDW!s4tK4hg9Nxz1k>Ld;{=QE%|?C5LERIc?irB=@6b>_{eE3rLt&Dd(doe z{oa^W@2fr{6tzpaq|clWP*FFl-7ioyM=;&CQM4PoV6H}HaNhTTaoLEX$}d)ysMG@L zbyWNlW%lkiG*gn^ODw!vQNpbwT2-LD7Rk(07ToB#fn#q;@<(Q~&?1z%kWkgD{aTav zk3=$E`d>xO#v$y>lkMYglPny6Uu$-5vw!n>|KSQdZy}FTy$y3Tc>Sl?Y2KNl+eqOM tV3udcFW4Yh<{Aus{(t4aT!Z+ZT*DRe-=6&%Bgz3kmS(o5H7BtNe*qG_w_E@K diff --git a/publications/whitepaper/figures/tpch_scaling_factor.pdf b/publications/whitepaper/figures/tpch_scaling_factor.pdf new file mode 100644 index 0000000000000000000000000000000000000000..7317fb4fac7652c54098974a4912486301c331ec GIT binary patch literal 5273 zcmb7ocT|&U(>_H&B1#hxL3ltwP#`4qrU(H;uTldNLx8Ly!4&DeN?R$?dlg+4ktQGr zA|TSMqNo(5iXb2$MZYKPy6?Wbzu$Mxm;dhTo_l7V$vM}|Tw*q34|(K7A7w%2bYJzU~m{r5&|_MI{$y~ zEO8V(_{to(OhS9O;V@*tgi7(CQov`XB%(7FgC)^!?SOVowbEr0vKmm_r zK>6S~)ErADQb`!lpZq`k(F}f;!w^ewrMLkwunYiH7l)^SUjWs`gT+B$F+^u9EpIXf zteHE+E33)IfXJ`J(Y#VIX{MU?akb{dD1p00^GMN&0d20sA)>tSjeCO9Yb$D^o~c!d zzeyz&u$T$1bn^A}Efof7$T<-|RV-TU$hL+fW31qQdUH=-qPxC3y;&@sSrHN%-?O3r zZXw$sIJZ1gn#%v{)trDMKWaYZF0NaPzbatZjLofs6x3h6uT~=TW}P~BXyMU%&nT+l z@NdDvUr{60mw66TVt=2yyT$X^#rn|JYqqAQC8bxGK^l`_Y>wYWYDXv8E)&9P^ zZ`0L^)#PH;gmfLH1nCyhjVw@@MH3jZrcGUGuwHx33SAtM-s(nblN*9zK#MGUI*+p~ zQ+o(tdaCw2!jwUwu7I;JenX&zv;@yfkkm@gQ$CYu$jR(rc__g-`fjiq5G866#w(zH zNi-cO@|YOm8+?9TSX!J5Jd0FM>2o0x&pg%poixdx!0f0`(3?y`(w{`K>98P0jy^U; zl?iN%ng>};yk0*bSW1-Ws2@r-(uq#f40&0} zaec|x{#rBg2&(>Fj+zS{V_|-eanjA9?QIClsi)gx%>$+{99hLvzG-Twi8~!RS}bzA z<3!%yz>}z{`EB<{aYj(M+@#tgSQ>*k>9(0`#RUe*7bmnOcdD;5R zx$VNQvyJPU@)|iVkqf{0+6i`iqSuD)_{Q3H=~jK+KA)*>{lu_fVYi`jMk3})v~(}a z$HEzJx$N?n7)5XX68hocGd!j(R3in$tJX_iwG28PPI^10Erb15!%7skefz#+OgW?C zmI5|x(LIZsQ`o72)8(_>3|M9CgW1+`SyV_;>DvH^mf>7uuM5?#$GCecJ|p8g{+9AS z?E4qz^4fhV0h7XtQcTx^rM?eONqZxtH0C3&)-F}T>KxmNGFap2>2{+c`zH5kZhf!x zuP>`-qq+2Z%Rjpr|N6D4RVdd}zSP85qM%sC|Ft%!bnn8jV6+>~ICs+&VL;xgjs#RQ zhD$Y$4-RErSk8}1nccvdXreg2|ByI*)HC{9&4Pubes;#3hPY0;#cmdtq<$875Y+Jo z4?`0AnXl)Pbp}(-2SW5VFWxSoo2yUwtwZ*i{>j#IpFr2DU*-%o15$9Qp+qzdHvTl7teguDb@G98tz(yj+l2r zHh)U(ZujoL5S!krsF0a^^QH8?78%&4W*?oP_n}(0wqlcri`k90_Lg5vF0c7*u`t|q z|G=9zs2uPZa$<y>+aRPkX|2>y(BvXGTJ7UznT5^0Dq2?6au2Znf|1KQe@c;m*$Z zDf2j{^30koGP6eOQE^X`uB9I{%;twUm?hsm z8)W?Q_5H~?2xcK91N)425~mx3(-V}Ydr_E4}1ff2A?6bA@Y7vyC? zT^*_m2VgWO<{-by0WexmIv{zdju+*!1%=j*)YWO8!RUA*$-)DTIY6Lf^UGR*!cX)6 zv;`VO3vCb8)7I8Pld;Z#BF)O-XQQE=etzyH8nY8GT9MQ=G*WJt+kVl)42{PwktQ`6+x9LO_5tbeZ zvxJhvaEZyFLhXb~k?3%Gv&yAIe29@79>@7p^qkB2-aqzU>CJpEv=wkYx2gZF>x%za z1KsVtP^PDbHmpHx`8LO==Gvp;VqBeA=-F2381ZzI9#vIZC$@DN(i!|-#p2Vsw|Scy zjsf3x&kbLUZKeB>n_O=z0;4lci=A8B9!K%(b6bBag;?wToQN8 zRDZwGTugh9>@lUdGt;nF>3p^VQzSZ<{yp2{-tK%4Mbus?t0>F5OS?Bqj08IDWhQRj zOmPeg{LF_C+HTvl4oE*r_n-t@Grkk)j^m+oSfV#=+>h*ag3?7lY^EObzaiM*=V<JmLM-2@f+sllM4O|a$vTMc5<|c+rN;g_cN9~)R)pquZpJoy+xMdJ`jlI*w>bw= zY#54#PiY1S(n*}-1Xz8Y&kwBlAL+Qv_^mf)Fg-1Mw?E{&zx5--@6S(&2F7D9BjAs? z%=$iM?AbBM19Uo?jEkE5^NidrhiY3mAJMT-GBR9}h&pt`2{svW{5CxT;5{BxlXo&X zq}=K3U1n+?r+lcQ$4N>sW>Q|6MJBl0>Cgndx$Lo=Ajwrok0wW0_;U^(^GFJy{aoN- zxXo?;4~$IOoTdWFhsCtDkPJMUOL^vZ1@1*G=DW<>&r48FHP9W^uFErX z8H;K5-3udEMkIcLeC0IKyxe~{RNDNeIwr6beYBnQ;2IuQ3-RUf}xi}w#bGfnZR zRlbT+H;y|0?8>B_$5{_+5AH{NkGbA)ti-anudjB0!EWpNu?HN!7@sRF!5z!`dl*vd z99lX*aX?qu;E{pXB~h-gQO3_a3W6R**yyPv)J+7$n#IhJE~1K(!J@+M!pmo~Mc#@u z0ZCoD&g>+0Tf=u+1o2$q_h+A&<(;!RcSvIOxYhX)3CcN?nO8}awY=qqCF-}d3Y&Z{ z#0}(^GA_q1zvk#}WsBmN`9BXwn-W$%YI zacQ`)>mPpkG^l+43+0%5e7C^otVET_xrwnbgyFgE=>F-QrX34rRc2vkCgyVHx}#qrq}qhB=1ckZU+8y$z3zw;PfUl|CZfD3xo=XVY%)UbR@af4-G90eV*To9vWvaH4)4`9X#WGLw6b3{n>LaCv}FEmbu=WyVURMIOZr)ajfDc-gQ%dQ02u*`B}_; z8$&_GH3p@R@SkNqbLX%SS%`@%FQ9WKqV?i;&T8o2GDXRPU%kR zSaGxQ5u-P`o01yhO&W};328Shk(?#0yYL#hb$Db|PgQhPQFm7AGD=Yl!li)K9*=q0 z*e*ZbKN9o|@z4>TF_U7Ev@Wisx`oUk<;W79>c`iNy3elpuRn1Nwrh15w9m7DQ0qh~ zo_n!#W#C@%eIM(k3K=_%<_=JY`Mq%BI=sc)7I|yUV+1JJMj03rFE%-43VOd zf`OUa%U|0rMf(2wAimn)b=$$6rKYfmdlB_cW%SOF*mC@m;Du9O)pec4r8BZ9}`#c1LNy+e*M`CTc0j^ zzx%pZd8Cgw-qG3NQ^N<3GIC2m-VDpGkSgCSDjiz)i4v4XD?!ZYy>=>7ygEB59>p$pLuh+Q{4Y7^rGXo*Ph;3QhI6o#q&Q< zyFDMqJ)d~(!DV^W6uzT2_J%7muiky~>c`M$>~Y<>4%Q@6@rV<_ZLy+elpN((pwD0tAu*8iNW zafC|*GVUh;fg*4&E?7|Hps5)4fV=|K9Y>&&0To5)4Jwg>#baG45SY9?)R_qCGGsCi zfVz^I)2);yPFraP%fjZ;B zP-Gk#fXWbv6lbgp03{tHi2;`YJRbcw+>J_bMU$xRcr=v)K#8tI0`@u>4SnF5Oq<*t z{wDYwjQu|c@jn!wgQWm!LA^#f5P)bPxNR7t-NBT=Md6?>f7l?g_?uV?4ugh3f7S({ zRrK!y(ZMqDe+U2${j|dWBY?CP5%2uJMDZeFT_CccR0NUz>jx+*D99-QF2G+H99(Q@ zKY;KT22%tVum8qi3X0(R@()Z=9#p3Ofhj5d%O6Zx7Tmi2+fEh^17)>;Vk-Z_ Date: Mon, 24 Mar 2014 13:14:54 -0700 Subject: [PATCH 06/10] include final license and DOI link --- publications/whitepaper/druid.tex | 13 ++++++++++++- publications/whitepaper/sig-license.tex | 12 ------------ 2 files changed, 12 insertions(+), 13 deletions(-) delete mode 100644 publications/whitepaper/sig-license.tex diff --git a/publications/whitepaper/druid.tex b/publications/whitepaper/druid.tex index f212a713a4c..64a98bd1da6 100644 --- a/publications/whitepaper/druid.tex +++ b/publications/whitepaper/druid.tex @@ -1,6 +1,17 @@ \documentclass{sig-alternate-2013} -\input{sig-license.tex} +\newfont{\mycrnotice}{ptmr8t at 7pt} +\newfont{\myconfname}{ptmri8t at 7pt} +\let\crnotice\mycrnotice% +\let\confname\myconfname% +\permission{Permission to make digital or hard copies of all or part of this work for personal or classroom use is granted without fee provided that copies are not made or distributed for profit or commercial advantage and that copies bear this notice and the full citation on the first page. Copyrights for components of this work owned by others than the author(s) must be honored. Abstracting with credit is permitted. To copy otherwise, or republish, to post on servers or to redistribute to lists, requires prior specific permission and/or a fee. Request permissions from permissions@acm.org.} +\conferenceinfo{SIGMOD/PODS'14,}{June 22--27, 2014, Salt Lake City, UT, USA. \\ +{\mycrnotice{Copyright is held by the owner/author(s). Publication rights licensed to ACM.}}} +\copyrightetc{ACM \the\acmcopyr} +\crdata{978-1-4503-2376-5/14/06\ ...\$15.00.\\ +\href{http://dx.doi.org/10.1145/2588555.2595631}{http://dx.doi.org/10.1145/2588555.2595631}} +\clubpenalty=10000 +\widowpenalty = 10000 \usepackage{graphicx} \usepackage{balance} diff --git a/publications/whitepaper/sig-license.tex b/publications/whitepaper/sig-license.tex deleted file mode 100644 index 4c639061afd..00000000000 --- a/publications/whitepaper/sig-license.tex +++ /dev/null @@ -1,12 +0,0 @@ -\newfont{\mycrnotice}{ptmr8t at 7pt} -\newfont{\myconfname}{ptmri8t at 7pt} -\let\crnotice\mycrnotice% -\let\confname\myconfname% -\permission{Permission to make digital or hard copies of all or part of this work for personal or classroom use is granted without fee provided that copies are not made or distributed for profit or commercial advantage and that copies bear this notice and the full citation on the first page. Copyrights for components of this work owned by others than the author(s) must be honored. Abstracting with credit is permitted. To copy otherwise, or republish, to post on servers or to redistribute to lists, requires prior specific permission and/or a fee. Request permissions from permissions@acm.org.} -\conferenceinfo{SIGMOD'14,}{June 22--27, 2014, Snowbird, UT, USA. \\ -{\mycrnotice{Copyright is held by the owner/author(s). Publication rights licensed to ACM.}}} -\copyrightetc{ACM \the\acmcopyr} -\crdata{978-1-4503-2376-5/14/06\ ...\$15.00.\\ -Include the http://DOI string/url which is specific for your submission and included in the ACM rightsreview confirmation email upon completing your ACM form} -\clubpenalty=10000 -\widowpenalty = 10000 From f1c4fe41800a9851a89d86fba189471ead8c1b5e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 24 Mar 2014 13:25:51 -0700 Subject: [PATCH 07/10] final sigmod version --- publications/whitepaper/druid.pdf | Bin 610357 -> 536765 bytes publications/whitepaper/sgmd0658-yang.pdf | Bin 0 -> 536765 bytes publications/whitepaper/sgmd0658-yang.zip | Bin 0 -> 530645 bytes 3 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 publications/whitepaper/sgmd0658-yang.pdf create mode 100644 publications/whitepaper/sgmd0658-yang.zip diff --git a/publications/whitepaper/druid.pdf b/publications/whitepaper/druid.pdf index 1a1f0972952f297e2f65c49440d6be0faa828bc2..6c2b519bc6fd2d4822ab85be3bda6ba253062455 100644 GIT binary patch delta 229701 zcmZVlQ^s zW<)gSk$3G8QIac(NzgMhu)~tiE)K22GIOvIF%j7tTf_44G0K_QS-4mdaj>&9CxX0_ z$IHP2ANT$y`pY5h#0DCPC0q@eKD1BLIfPF6&8}u(+uBUfPK-_W2ns})#!scN3nTUo zp4&BO(P2bB9o}zg*4Cz{Df4ShH)VBwFcq@ye;2ZvpArRSSQS87KVEKqUEVxQd_6or zUlQJQKYdL`-S@#(SK0+3ZcZj02jI;!E2Gw(Jmpiv;-V#{q;O#t9#s(pnQ*S@r*yK1r-2cm9U)2H4}T#^Cw1?o zA^HRUZ@T>wBYe6VNC}Uir?2C>ZiAZ{eP!mKr>OO}<6E@5?+L$`PmkfD&x_a3<2`dA zS4PuQdnJ1vNQX<#P(LSB z$!+R^b^AZDJ(TJJ@<+W@p1dC;QSex%5(&TMfY-Bn)WS0rj_rnABh&JOx@`x3d3?9kL3V9(J5VuPX()NaN4 zkp;s<5No(8mSOK2FNA9t^^B~*A zkDyz|7Gqdd1w~TFF<%3J$!gYFrn)G=QV%e_c%l`H8>+1_W!*amRn>3`vBQkd{3#e? z*zYcu9bn7?BI9hlv8Q_5;Hf*fRite@%mx zB(z;bahorOl~ks`LnfS@dUax6`LeO^1H7yFHJunxzDpz}T8jchz?#t4E4Cd7tT0BI z1(u<;&%-cG4w3ALhSeJV_bstYo5YNa9;7B3l4b>>lDkXA?+0Oi=KbStqkZRdRR7sy z=tl-nlk2hIu$Tg|vD%~3xImFmlL=lUQt0EPeGp7z=%H<*#D1;ymxZD41P$Fz?7;mmnwrNTQ>{X2wXpJL~DKrFnizvJAM6rW~Qe4 z_@OTzr%E%wp69gF6-!U`MC$sI0<-A(?XGi!dU6B;-UfhAuuLsgPzt!E#}CN=B82MH zNNiu>GuUv%NbveB&K#Fy%Y_@~UJDL zujTj?0LehNKgLkn(VA&w<5%4k;oc5Vscjn5?W9?nbreb5N|K7&;C8FMMG=ugP!`dScv&KjK7 zkWD8?Xo{3RTG*`RV*hd=tzeNA5}cBwfJG||6Z6}j}~G}>Yb zh;(_zNUdeTuXtc9ihjs%%a7?0qv;X#Tm~P>gc;N=nB_-_xW#!7Ck`y4r-8eW z12_ukaRS()?b_OvKo`-hY0yW|)?iuBV>Zc)n7}xLo${Ak?y7&saV|GDRnQO`(k$E58>GYfTR!j`|JvmTYczwk8*J$9LoKMqqaLFRoyprEUBiRASF6%aH(m%nf zrqORbsdu@d)3tc;J-iso7SSOQ;K7@;3=k=Vxf^htkKBob(3rsRIA1#x4TRk*0Z9dz z*>X^d?F=mt4%)ifU>l?wb>p%1pQIfUf7O_4A2JczaEMrj-@=DOrF9cRq2L^$+jwW) zu(3IEhE*@%bCWI?T{NT`uNE`IPf41|*xXdsw9KpN4z}v6&WBvOhJ6VsiCLT-Iz}dM zXu{A;7%-66nYzGRX>q6qKrc4_HG8)R#L1L1>!(hw6C zFpheADS08))Xj?x)E$uW{EIfj8ijQh%2{N-%oREKO$)KQG5*5O;ZT#l+Igb#x799< z7~W^i7%K{vg`8N%ZMp+ot1gFbhh?H73ay5lMo;R~NOyfNx76!MCq`R7~UG|b@k{&HUf>pmoW3J&B^VXXy zk}SohK{ZPs#%WgUcVA74RuG5iw3e31<#Dp>DV!m!kU*`)v$k6W`TVDA}Q?&Sm_#}L;K}u5FNpl!?bsf>N{4bBe{|b zI$0LYxGsmlyo5|?4a(|Llqd=<9T)^lEy#)lW2`NaoC+czbDK6cuH4*W19(SPqA)BL zDF02O4lEZCH;-ZN`Q)h{c_X(dCW^U93v(LFN-K)(95x7|{-k*l&Tll{B2Sh}1Zo0q zPogD6^F<3)d{PEVT88uwI&{ho!yjpO|BOFVlqB@(SVL;0`&_0zq%7Sn%AJ~2)7|6M zDE19iakmXu^)T~0O^frwr-dm3R6!McCHo-Lwk(Xm3Jwp0X6qQBO}!>VCxD(nJS;6N zvd^?wtra;3pxx#5veYBN=3p&Q@8;(Gv$XRgY~Fgk(aO5tQkecl0N9?PY7R`A{OZo^ z`a19F0d!|hsyos|wm3;NV`qW77*|BF5$8u*GebuP3-S1WMW|4&QAPfUQ94OmY4ZVW z0w94hN}fzbG-OereNEe+N%_!m5+4@a!N`+Z73of<`x0(G&x_g!XngVyxf~xKR|^Zv zC*6+b$Dw+k-aX$>lhd4cdu|_&-nX7v-$t!3z^l&%FCgyD<|Tos7=zE(@1B~QjN{mg zuY5bF*QQ{Fo{oIC4}-739N;KF$4BWF7!I(dHFeX#n)utnxz*2W{l@Ku8TIlFhDEtG zj^^31$B)a#?qJ|%W1w{Ff%v%CcU5R%~_zV^x8hZYA`UUVh<6|Aw>vWB{>y zyVf3L7*U1NzEv^w>$z)&)cbkhc@|S)&HVQ!JgSA?{>K)hT`c~-s+V_TN6*fseG8B{ zcUkrInUnC*NfiKmB;~~!af1s za(OVEO$AK&Q-=*gNH~$fGkqcnCVwpu3-IyihO)Y=HNIKDoEEy~zHG$i`mhj|;w^7C zJ16>X^4k#XLm_)o2yOLe_OsUPTmz%pIFbCOJ`X--W+#a2finCn(2d)4X-@NCT2^0| z|4e=&H*6KJfr|@5lQ+4!pz6hirali=FLfLCc3V+g`42Le#>7-t-s{FnAe83%hg4~@Q z+gyP+f}~)*Dd=3m>wfsn+PI7JZvorq9?uvg7h}Z8Ub}12^L^x4pUJS3gH;+|R};LP zexmN9o}AW?UK_u%FOC4?htiR$spZB#1RolfQcfge$ej6jeA#Y9L!UaqZ%6MX)XDnk z8N1#oRPZl2rBQ^}I;4Q!FQ7^bRhi@$~cjW`r7Vc8NKRwqPI$o-IPYd$RGgMg0BFAZ@r~)XeWA zpvpY9#Q!~Y2Rnmgk0}}(mjWLk%3$5RM)FBNd$w?lm_YT}atAzk24Zs9SAJcX3xdZq zKjiWyZi%)1_A2SR1a>fHhL$4qBElbhGfe->oO;m<@)ZX?jB0la_~%xixO9x}d!!)s z9pPB5KenRQNkQX>7qTZ0;Sz>qO?DF&dxY=`SMbI@oCNx-xd!#d9 zwTenoNRXyQFCCc61F~DpV}H5!GoDH^_e8j{_B6C(1Ie#+P6)VBqq8|-ZSS{LIMNrj z3k2p!d96-V)>nCdHIL&Dj|IZxezAx$4fTsF70|@41gN4{h!@R@!rVcLh><6daGELm za!2rIev-#a%`e{tF5E~tvX#dT538r^%4EVxt61|iOp3l*0TZGU3QmpP-1WhjBmSe9 z+Gp#ypd|*Wy1EGStI8qunj_qQ%wr3WlBLHQql+BG&?mvB>-@*xOyh`~`OQ1PnhxRi zJ59<%mEbkipY<+OyP&C~>Wc@pJ^7B1f|(Pj@rJ3x+k64-3)w8HZ$N(en%)I-hi^mZ z_+z#B%abbz;92|}I1>iB4zX9L>>zKZ0rfv?hcs|iwjeQ+cc^jYs4K@y-Y8#nUFF5o z7)GRFtAR4^uA)Tqtg~aWN)>Wxnv?Gg=^7C$6=HmLQ(wTL!~h6m_&=Bu0X2@f@M>6g_KXpu1mEQUf(JKIp@;4>M5u(0npf zG=jrwQu7|~LFEn^EQ*{*&1N5Oi7Tl@FBe`$y)KM$?`qTeHV zf#2;5Uh;^XNYA9a=GX(N5{hWjW4Z4Y63R%i%lSgQm97I>IZ4r^H!^EPj5o!se`dA| zZIX%$FdWfv*>u+Z%)%!az^xZsyj#+^rv-a6`ldQIiJYYTxm)~b7*OCq?Gnp6(i^F( z5?!e_p)l@?iYrf(aa&wbby#VN>;);JYv{w8cCo7qre!H6nJ4QGn-7v+X-6N51(`_->#uK&QJjc26|0;A zi)tiCyC|l`dHQ1eu^j{Uz`v-wm{V(@!xJIWrv1LplSs>6Awu`k&Tm14v|AivuY_cV zxxq+qElu?5YbguBGwX9!i_;nns;SrPL9dv~3XNra^S}GlGLHvj>M4$+=Ivwx0r_2J zr~){$ddUfSyI(;q!{;=^m`xa#TZ%`(@g@^CUH&*r3)vJa#JQ$>J)F_7hcTL@R3I7I z#s&p47iGC936|=k)_!1kuQE$>$76vD zk+$tYB%JMxsz4~eMnx(~X-eLvQtq$fb?`EpgB=cRJ8qPX;aR#qcr(dxm=rf4J_3=@ zs1!_tDa{?pKul((b_i2U+&{Y}MJW+T?S4(}iM@8hNkz|NzOa+j@hIa;)QKpS3u#7G zYvp37L4mx(_ikMe(L8JKqAGi1UME?|afFzZ{rw)@i$2%l<1qbuSj^-`(U?W1Ib@3T z5w*nF+*9i?W`&_wGt|zosV--1y&uvRg&G4z;z3lny`*2vod`4s_-l6G|m0o z)eQ0x-50#tE_XuE86t@-xdUMZN1=E*9hQV;P3GJb+3+w}#6M%>qaPs(${=~L>{`0j zC05ARG<5^Uc+b+V6kt|6mWt%-HB`?WncL&AAX-jxjjkkywU4AWG%yNGH2d*YNHm{S z*?T`fiv+plD&5-2Q~YN~EG@aNt~|~T>$2v>#sJF%R zw^%DTrk|2cs?(%CGLOqTo^7ZZSHh*>_b#4sH7P@L?FEGM2dM?fqilLQ9`C>Y%t8Nr z0!GvRiG)sc@LK(ZA9vqM+>f86`f}+9wX zcC(_KQdx`a(4BJ~y5U2kMAlI@nkh7g^Nnd8PS@5|kUS#Pv?hajD#*?LB}#*>I#QYH zo&L@raU|0h-m+|_HO(AJ0oM+oO2S?i86^R0M@ux$>u=J~(E{0kER}(V4XCaE+fW!; z(84YBYzzSetE9og`e}#`nr~KymXp`=U1=M#dZBc&qI}JZB3OD*JXs>_K#r9)Fr&m* zdX3P)y8EyX+mGx2iq3dWs1Eyr{|qe+!&-J|eW;08NLokY(tru6d3 zC`*f=o{l+SL*yJ1pjFY$F*;)B-)k>J-rhnt?*;{`?2T`fXHNz7`Jv_a4FoHmjQ5Z> zK%RSYYE;g>^Y}Zkgu1OgkWJSADGlOw(ihrcTp}^9mEa7D*U!tT{Ywp_eek|rl%Cqj z7OlI_Z5ElKjGNU0YLA1<_Y5%75kEpsNJ{D|LSk*_Qy}CGrbeV2+#XrRw~4n3ZDj-s zuT%pKbCo6sBxJUXk`46Y=6{7Zim9=pF8^^cS6q!e@-FzchjVAVfXSC%f9H zRp8eepF%A5r@Bm_=tNtXDrPDJYk8bu3~GTYP?nBZD!W3ezKZjy;2#y)XNu~M0wHpX2KAklC9vQQ==IANa5~PBTh)ftZm7wne=7}w>{lvxmZqlp^B?$MH55@>=u%VXT{ph2=SRopluCSpfT6r@lvjOsE1)vhB z*k-7R5a&s7##3B`)dR-mx31$|%nZQz>~ZB7@mkO$b)v>9*Ems6JEvYHyx}Qd^%3N) zAVcKm1MlHoTJ2@mNufZFLFayG8AZAd>v(BTe(}8;6Ex;GPl3=6vObL(|GEi zq`G$1%}Fs)Uv%^S=mR!?Tint7!_Hm*CDs;#apJ1rkiEce8SdD!PF|F#8U6^YAXF?K z6%A;%t*KD!_PfbZ#2d;~o@6pEuK3r~ue`82!O4e1U(SqK|xS=w|dDIJwP{ z4{K&;>g?iVW@P)ny8orDBue8jfpT&DFG&Sht-WElA%)_*qWNfYY?R^KLk9`AI%aNS z$a2x#xGK8~C1ax#M6E(pE_eUp9Z|$K4M@P(`3sRSbzY&GS|cDYOOa+a&O8!j9N*Sk zWT?zko}LsV9mqRWWHE09ZXakYv(Gq*c3ssQTTUb$it$gZqrN->H(nGu$2rm+`nvWykQbioYjfJ~af{kJZJTdpC8`z9S>QhXxIRDU8} zb<(7!adIHXc~a7ACV*uU4#RNh560~SAH(G12}q=shbQmD*8Z(``%HX&5ya@y!B-EKq_d zoSWacXaO)uXtLxaOL71-nEXU4#LC;CxI7!aV1!~*m{?I}?s3EdbAErx=2anS(WQ;D zGsymUb5FE*q3EA`S-0%3>bFIp>CI{;1O0nLdE4jNih+ea!+E!D#1zxxf=y2m4jJQy zxqdOacdFe)J0C?Y)LWxlJ)Jka?fRpX6YCC)qEL;B{?}$|<)EK^#Nt)v-Byv+`dVsd z)xD>7@)u{y3Pgwg;l@d!hGeNglHN`e`7z&2RmP+rjp_%y(G&E6vYZKUcY5Pkpm}DX zF?xFgudn7N2jfRr%zSa1%~pAtTjYsP%RF;Y{t0gndC^{8zs}jTR-bPGomKpTrkeK9 z-kx6|!#L;JFM)u(cF)l-b@ih}@8~ODDt*g|W?cMym7e6OyXt+hAN`r(=Akfg$$j}7 zu#@b{6Eum~@UvR&?C4;LSiq2!kJd> z(yFc2MP!j?!CnFE2E5ga2V$h<#X>*XF~m$;JEr;As{;GyDSD z%r0d-3!Pn2C`>Uij!Yt&|4@C9PprA_l=z54D}m}Q5G=8_e>ehFpN`sPL_JnF)C!GP zdD)6vvg~7?R+-R&$uH%GKi2CXq6vCb=qYP6oaFYSrK-E+IERo%i-W`31Y9yK*5s<|b%n=j1I6EldA5~?a6 zpG>XHfW{(rRKx<$AsdaVprC#S&T8v%7LmiZ7#^|4gR;nG8Fs#(;gtNrtCf(OF>hZ# zt0;MKYPZX!>@rW_Ozcd*OdE^&pOSsKDxI7CxF#K&{kZynmw(kcAR|)@Iik=F7gL?3 zr90#AIc6uPE9Lkkl=clsYvRw(tm&AX(7?w6$vF7foY69&_3$ldyI-?_G0}fx)2r^( zKMosg{8dFpYkZ?e&b9~7iW|d@kyU^4+oA{K`C1Jn0M^$9KvN$Ols;a%mSPl@6r{KD z%6cm5Hn5jX<39hts5kr(W;yp5N{gwB&#RvLvHd1sf3L!(@Ky}l9v%M{C2b=a1p~i~ zdh5z3HdxRp!-B&3@<@`P{(vZBYz9nK=5bAneh8yc*9!Lx&=|DBP)k#mMgPUZGQLcp z1qV4nJ_J+SmaZA_pV#sOd26%(jaWY!S)zT@=eXLGC07WGj-TGS>mS2a1}{H?MKs^P zJ=y(0#=BufNQ0dA?ddAq!#UAy@B<{DfL+J(STkKD1L7g1w+i3vQqhM#c0wya@wu-S za5#5cYOK*by`5i#-I|?Nfl_7Blf*Xt?IKJ#98jtg#4E4uw3OLya!(iI6fu}eLg)ib zBu6h>wILlMVWiY&98^_AAGy?VPCNMXEHDLhxAxR!G*u0&YfS$qhT9(CNdhakgoPG^ zyg)@-wOTxdiNwG;b=4@fOom71Ug+bfxHRq@iO^HD-bF&Md=R>QBVr2Y9Kn47efns8 zpeE&6v)%|-@=#tkH0rLNin3jn!ip8BK?6Jrgdi0F)qv40Y_S zmMiE1NT|Mi7V>2HyM@{&s;2U($-bi*-)F|N%);{D5@UWUZL_%I+~cNwqAMq#&c2pPhnz1mFHH?zB$pw-5}e(f&`ui$;wDP18# zl7eTD7?=BO6R-aQj>}A7w6SV~3TvhEpVbbS2rvXb)AiimKg4bMGe;b^e=Cu12`uyj z0$6W_7xu(2b!ct)%Sajf#3$<>#PFi}+^%G)j#7MLIVb$ozs97A-3=%jfIlZa@Yw<1 zWBdH0=?hicYD-i4-29{39`>KChr40IIzBSfLJAT+$sdGO+8l@f--&{SnX|E)1fKwd z`Tt|yU}fiG`=3c;NLMzJj1#5%rM9fl*HZfiiVZ=XUG?(4&d|+n7|P@(pfB*w%joBu zrZ`MVs(f<#nG9<|PF;rD7dE)tqx|zDv3>M2x7~xWb0O6*Kx$go@BQ=f3+eve+3w?+ zBv}6LzGwd7VF;4(@_xNG6)52Q^7vs5jUUv~A$qv|=C2848vJjQ&KX<8s7!Vv-|6%I z0x<#)kZSBr=ofQ+dwje9icF&Dtcd$*cBjss{2BhNodFU9Tb|NcYXyDYpU(~`LzFzp z10GI9)C8c-K4A;XwJu%zT89hNQ5f2t>n&{`L|gko@2Q=}`yDIBtRki;^NqQ%b?W-x zn2Z-Rdz*oVuwjnTW200B%qPYyHLNbhkxP3MzvHK^<<8a@?(dYSRYK!Jx74rNKc6&Ng3MTa$ zBMJXeaLjX8!=qK~Ba5h3wQ>@Tthkwq^7GQ`kv9;-On~*a>L&}miT~$UuKdrM&nodK zYpN*tTCJo7T@31r(=HlyL#5*QbB8T}2!|h)dG7S@sy|@c?Uu+jCjzRxam0iSuWTf1 zIGkl>P$Y@l@Xld$1F^*I5U!21+$--_muOWfiG(r!^nw$Iod&z|Y^yQ^M(Z-65I`$Mc2U%F$(u$wJi;k4A z`S*lsEIT|{%C6VR7vT(oD%z6?uEiMZnd(Z-N6IR3!8sIl&yDKpzgMilQkX`BlAi70 zRRt3IfH1{xrongLYBCSh(3Dr0${8QdNkF%Ar?~l$r_)&VJ2Rmbzsx#?p}-*aeGvO9 zyndt&Q&$EIqDqhn;&d8$aK~b`$1LoG0qYO_NF`!DR zv2X%T9Ugf(YyEF&1g-u`$=Hh5;g@qpQK~k1wNhiz0-Wg}3)ALANnUg~PUKy6+9Qd664RS6cX}H=QJ{U#o2UgZ(^wIfiemM5Ip}yV-qU z4d+OLltFZC3>DgoOJO!$Bo#>>hk&Mx#Uy%$;S(GT0@8RREG0u#kQD4&mG^95H8qn8 zdj^Plp58f_g#dCa#;Axlo!GXi-IYIGAg z=DJtGcyt)@ZJ&kDX$c z*OBkP+ibR@Zb8&UPfRK+t(#(TEAllfl%ag=NUu+WSxHRWmB#2;@l_&;=D@TUN&Ydz z-IaYq9VwD^Yqp?p1X@cH=m4)#F5n!3CR8KG<9Cr-1&N4v_e$pCv8<6McNf==lzjs^Q>}lz^;D2c zO(!n*kh-LsT5F2)n7}k*1|f!lM_nW`(KpBs@8dvm!*wVBr8T;O93U)%cT-7vY6Cog z>(W+EQY_;zM(SaLanjz)@JBR~@u7BTjEw{N?KV-_Q8wgbTtYrUY#k8NJOiHcb$dz; zd5%>JIK~BzQ>qQRKH0A^Uts9zz$kg073tOUxE6wKc7xFcIIDfS!%2?r5UVC-xVzq@Zs`)zTje*$ z+?idP0Q@PxAc}FUDJaXzGdt4J-e`4S3-hYQ`GOm?%9_IrQ7bd-B?h~k5o~&t{d=~m zteIZLT*7Pt&99s^J{NV$LU(z?O7e%EugnPErfAXg^ ztqPmE|Eb*UVasU7rUS%I}V^2cJPixVHd~<81ge{tMKXpt` zNxx<&?c&i(Si=Bj-9()^+#OSGm%JF0@*hKdyE59Lp>6XNyXo1T2Hoe0xPdR)S=2|`xwtl?tp=_eEZv*zlQi@(qRjKb+>5ZT%tB&SY;^lpkH>Mcw6-{QZB}ghJDhs@ zqP0$_$7cV0E;`_9XTjucr>d|Cc-t9Bs%(n{fS)klD{iz0Nq48_W|%UqF}1G!L4)|8 z4eQ>{fM!lr5$3WA&XU8(dymL#GMu{J2CuN$%)n`$C{ES)h$!3{BelGcmI!JqP!K3E zGk@fll*=e|=^+NLf~B*8%7q&(z0}xKIq>Mkbe4yCHm9_DR=tcrZ!tyC12VYB&YWF4 zKng)yal{Nhb}%m}iOkZu!Ss(c!@W@_au==q(GDz*5rh`oK&@@2rnDHMefN!Ot;1Y?zs&>Q0vP-Msi6 zo};3T_#(ZbE*{^_pqE^i_*-B#Ui`mBL%OR?O(e||7RI2l0rUQvsWR&;1rti$+ymE! zbs4IO;4@8c>COq8RV?Zs-Da-m9g-g0Z}j>#l{kT)^T5Q+3ts1~1!cI}P0K)DAWVMD zX-ps6U6{f`_Mh_XehczDyO1Wxn1+1bYW7F6RAbidFQwp0 zH{k>s*#xXYHm0u<7CTV{ng{z#fKpwtJT{utgR=_9W##Mb)jXzFYc(fEtA=d`HoO$| zb^vbo3r-Hy@8e9bIp_12=bO+JvqY?SrWngl~H={DR2zmAq^b}RksfFCd`f?ixLjzc%o+PTx@ zW1=6j!&-7x$WS~>Gj*>Aq`BRRD*zeViMt1bkAoG{;@q+h%(Zd0+23iL&oI;7=oIFAP@p$?t}4dAU<$75%!X; z&cJ93@69aVMj5s065Ps>lm!UOx!qP63-F#;E3oU6{d_~xHUD$~d4NT56zqy^6wl?8 zA}BnreOtriF_%1C*B-6*_Lk(2hxw7jf#Zto85Sp=eXfa|OhWy4B{edc8;)#GE`b+q z%94Nh-Q}9@C&&k9%{1wMzT`4a5tf8bHH){@i4}RzIBQ|wIBPbWGV#gdxF-9}Ha!~x z4dgGja@fr|zjHW%*A#6@k^Mm$(i#I_&?)*#+Yk6poounvgBp^DdsN=2k4kS$8n0#c zhVA-0gK_BF>(G?>t|(X{aQF5GKqZ@(#dU2kz7%HJNlrj|3HUW4|p^2r|SS??GdZKUZJQQ$!B6@l?T2OhD^N#h8vQ2_yYx<^MAien` zpp)mind()XM{&H<2%)#!cc5dnjI@lembK)A`KD{BN$c;Rr~X1KsnJacy=ShiNXOw$ zi)_S|lLZ2B1*2?>>@Nm?mxDwHX;+-E``$rUHZ}d$#O)6$OlR=|?3qMt@iR6*#wk=w zTL&jIV4vg1BRb{0Y~B&6)Q6SumNUvyR+|OYCRcbvv){C8_5N^zk7`&4UC!CuNnXLD z_@7pXP_N;1XbpE$=@zms`tQV4nX|pClZlx#5eqBp|J?MyV+E`noZN~2I0V3~D_xyh zvSwrgYWpv=BS8o00=Dj?t9wy`fYIv+8&Ge5@c=%tz+d^?WN`8KFAvFEb{(Z3liUmZ zb&dOf2f0r&2{hX(bf1#1j$KQ=m&SU4<2}D=KR0(=YI#2kTplmciYa4$hDGlWAa@)& z-XEVtzXpD5|MBjN`HuVfUqAD?-`}{EEhr%O=Sc^!{CXp|A?bR=UNibb??wIv{wD>w zIVz1}>4VQYY8~en02j6^Fv#Rk2^p67IRalmHOBUa-<8KhhH&4C-i@ZWdm&}*eVYS` z+bpeAGr6tIO&AAi=^*-=<%#Jk@;$$uJ z*T9oY&6&t-O{|wO_nm&_)2;iT)_bXRU;N{srHV&#=NaFI-G6P~C(I6fYc|bCtJ!C6 z&JgGNp{?6mQ8T5CZ3W@)yGI8d!m({K@ffwJ~zHf?4zWiLTZwF9NJ?Myj z@iP8LDm&-)<^Pfe4@-e}6)Tf-?pQlCB=$0nvb665sjqMMBiDhFKi_*jX;7m4Z>8RE zQ%yb@v$%zLkPZ1Kz4uZCCr2w=AH1#J4Ih>-W7n-6njNJbigzpaemh;C-5(bd=nc?3 zA6TRA3=E_59VZ2GSx%iC?8kT$@&oClT+qfkgvbo78= zGrd77X3$B<+6J_hQHBE8gP?#P6y-@+lV) zeQ505)jI-Q>tGRBZnRG{(@@|dde_R8=_Qs+vV}kZq8L^6Yg~&~yc+B&K6Sel;;{@| ze$)cyiGu2R9)i@ z3;1^;XFy-5;IFr9LaJ`6+f=H-_21TAM@{?>tVzZP(e>di;`vPJ*VA>Pk^b(lVmCp~ z;*KTC<2Y+c%t}h7SFZuR&N7m5Y(c}&fY}MD#6BF$xVGA~ddR{biC031X*z$H@x@ahDuuxsV(Ed`6w42L6xWOdE zKLO8Vx^=%Ac#EQ~h}ZB~`e0V@nG0c*mojIK$ahIK5YrlQ{EgxT@s#ZiBbwI4pat>s zLr3<9MkJt$&Ay4Peb8`=%6e*lzXEm(M#A1kYhEOZP^_vTNuxG{W zjz;Z;ew0yGzw?VT3!LFu}75NOznzQ4*800cU_Imcf8Sa)#WQn6c1;LU9VhOhW}DPv*eIR@6(6dIH@qsMkEV@c3M^|h&&5cW zZ+5@r0NF;U!|GNgo#$!Yd*gEprGmS1cr5;X%x27Su36W&&dpw7dKDT%ObyF4acB1t ztb51VV&k$vnoyUxZ;&2;L?SKk|H28t|Df1F=55lzo}0@m{2KvpZ6l+MM9_(Xl;esJ z>1D6|&R7IyQmB)f6$Hk*<|5V+m${AGFdw3ztWeV|4jV6FCmckWF(2zjc#~seXbCNS zVu$Q>Gw41-ul~Al-XvsR3WIAeo$!)(;Sh+p{4#4`3ZCNqTgxiUDTvLf?vfVR1V)Rk zb%2W7lpVV<`AhY$nNnkNpd{UR#^&JSY}t#7SFwM1k_OmunaU7ngDr|~JLC5rp$5z_ zp41wFtxpt4$bD9n1sV<Py(uMhs@63gOImS59COTc1QI zbaU$2|JGmuK<1XX|7QJST`Nb|gol_M+%&gqLdDTNi9sM=5Qex@6}2k7{Gwy;F!o;1 z<*8}DFfQStSG>qCsM5WMGy|0K=!Im!E9$oZs;fK>4ri6e_0hdj%d>!p#CqD0CwOX- zBk@@7LUa1aYHtf>qV!S&txb3ykfdgwvUa93Qr)KQhS{BSiOwc+%^TWslI!@ScZJwr zzi<$DARY;1q=vUrP#Wc)wd^piv-M^*v^KV7;fm%!3XmMU;=)3)=YF04osQi13KVD4 z%p>HHz779l_A)?B%w7j1;?Ans6DQ{n-<%mKrQ-9FWuC(;l$OO#{AG*$o>oesc?2Sn zvT>$eM{HCsFr}&M+FE+9$A6MN0awxzKdc>dl6#(Yo5T!Tsoaq6#sNvb#Y4tZ$8)ZI zjH<-y4KUDv*J7n3ffOAGiLgurk7K6i{0PDjL({8|W`_Q~B`E{c6?P=_h^&lLX+mS6 z*H`B86t4;|_$!Cn?s5ZGKbBC1upE52d{||Py3@0vbD*rU2JuguOTRs@wW8B-A!J-E zXY7ST!yDEY1Vz}6|9F73z96N93~DP2&hx{>!znWN+|mp-(aPxRRhJwG;llK$3Z=L) zV252%b1_V>p@0Dhfp!LLnq$01@ zHihLr&5-Lyry}CwYNJ%5LsrwM!P@+)$`i@}D=u2g6K6)~Dk1n1Rp(}P*I-o>C+)ll*~~5s6I(dM(X9NV3YNP zjx!JNryd1HuT<@f50uV1VM*4Y1SGR`%rmKTyL4=fx4&h@)5$axjUFk`UUt7lU%~E? zAJofjsT!uXx$=i9M47mHQhi-xZX3d_ZbB;HBSgJo2WQ={KdHwMUIAEgeTHd= z2r`~YY0kA?nm^Y|4wY(0U6c`dvItIC$SBLnaeK1pW%i#te}Kpe2)9rT$kyzt&_-L| z#@*_RlK3WvYu!5*XXG=A3vfh)_G_kS%v8+K)o0hmo$B{Wsp zF@qG+;I)fYtXwx#lC;6w#9dhgliRs zn3qkd5Ew(1z171Bns#54TCShR(&QWeUYIf92WV{19piqeCFb>ppXuGo(wK)4M|KZ@ z?zoqmMHM9L(2?g7$Tmy=l5bYFbJQ5KUi{)os`6+fZm;i_jiVf3E{$DUPcB?}pj_b` zSHBQ#o?);qOGSw{?#&a$vyZ|Q&G%b+4B^{T7YwirBKSiP>Cs@)0ox@ekL+Y$rADbW z1?y#MGMW%^t!Gs%8t(`1ah)n0_~jzN16p5*h66YAV68VFfS#nJfJ3)>{;v;-nY(ZXIzK6aA`@~X%tb&Ai?j8j5qZEg0t}baEc?ce$+Nd&? z!p8`s#;J|D7}fUNn!;mq(bQ1+oOhnM!Cx%??%Bvur&4M_0w=#8NQ#RU=)=-S2Vm}3f20}LHp)?pa#ot#39@k={N3X=tKCxj*F93wl2i0{{)$H@pbtZ3ihXD}c1Jmxr4g?Fz20 z)|qa055uxkPqrV~Z{^qsPkey9UCce60Z^ga-MQd_e#zUv@#K|uY(`-d-TVC;_}0v<1}W1n1U zsbG-(7A@wGul8zl%JjZwC@vMnn>xlYgulpCE0_-x6u z3~T9OLw_lupI1?^iAE^KhJf32@>7@I)Q~}#L$hX`$M|fp^V8&{U4D+sPSB{oOaiF3eZl4lVDM9(2@#%x`oc(BS+h9z4UC z3c3{de+t~Gu#2Fmi2*n${|6BNvyk{dfw=5~@%X<0WeSh)+5rXtHmC;)v}so^ zj&90);|nag7eF05bN(gsUcRbk1JWlDmtAt{RWo-@efdfNR~|@Hm?dkHZ0Ynw`K_+hHHapf;U?FUf zYGszXBy|dVkPE-SKFBZ}USUlHD)q;wJeX=oJ%hC}p%!Mr2H4Mff3QJ3 zSDBm|o_YlW_>Lv`)VpCP?8U!E_#M2(oXUyR=F|xYLkSGQ@7WEnvL>med3j3}j+la@ z)Cnr^&gJl5@H`x1G4^|Yx#Cnxm85cY>NW5~cNmAgeFR&6%>K!*!@p(Hb8>a6AGmQ` zZl%$NXW;`DVO`lsHlBClm-G9ie}!PhH+ILr=`b6|`!0O?do02Bawk71Jt%LJPperi zM^bK_kvQA~zk%N~7k+7!En>f9ud|PMe?EoZ!#|XMEk7#%*)kQs`XZPIH^VmgC-br1 zY&5%&&0P`|!*5!9jQv|K5iWnZg{*jek*=%f_*5@b5ZyGkbt-VUM!i z_?AQLL-rZ_iha#abHG29XYgE}hkr4?kpF`Jn%~P0;@=_uH~yUzl=7s6)I+M2YNX4t zSF5CT__s&;K#s@<sxlV z>`r|E={ScG9EBXH#NJQEzd1PH>yhKO!(W&a$1cK(*}3d|9GNL>4qL#M;7H!UHn4|8 z-+#-V!clsIeTjW`@l4U@F1!ct$4BDdMSKQdz}NBH_-_6>KP6cte~09e(xqZ)kW?$p zkQPf1F9%=|kxo>3jT3$#ywM&XePELLMYfk*|>NlRuF^Q6?)dDaTa1I!|4x zHmd)}($jLTWt3&KrPlH*%U;WC);i?QKfoS%0^0mzN2KM_Kxq%$!pr0^f0e(A{5%Dw zOT#&GD&N9Z^K02|f1a-_QTy_~Yy_N;<2csO@%#BVystEz4P#?q4)3l%P4&xLF{zaQ z0LSI0@Y`O+CofT*>^lCX>V%yPcm=-mS*e?xkY0kfr1zOc-UM&Ub{1sE`6JROlz<%1g7WsGt+lq2wEGuQ-N-2=|e+cBFUeZT!J;*nsUc=MaZs78x_;nR5pGk@zZeX?22K7t+ zCR_mrWjnklJ%+tM$bTyhmrp1cuvxeQu7#DbAhjH>QYOlOW|x4(#zTR81f|phh4w{W?d;&_bS7En0mN&vB zikqR-0gzv6xd0}m9)`P9m%wGI+n_VbgH@@e_{1%641NV$*ou~GU_NBy%6X5SuT=Ac zN_DC;U&G(zWBB^B=NCu2fQ8_5{QE6t&Q+d-HS!xUe+H^jYg31jyNgi5-VGPSP&kTT z`FDK7V5tepT1N04scLCHe((D*I`v2@huL9P>QWd9Pr*YL1*TdOIL8g_&-lI9zzlvt zYOyqoi2`M78MIHIrC#X1kh6y=JntCexyoP}#X8IxrS(cx6B=+Q=r2 zo{03#17bDNhU0=BF6ecFb|Iaciw&X!L$d}%e;Zg`bYMgEm9y3itQ&xj+u^YHkM*Bn z?_2^q><%OyNHqjw^LMb|xl9l|IIzzS4ptX-sv!~^Ft8yU8$dm1kO~G)o!&5N^u&P! zGIDckI+rxC{?lR?H$bdkgC`-XK!5R?2DN{KMZ6|Do4y8aitZ?BTC=uMgNy4D&grq~ ze^Vz=kb&^x>deD8pU;MGS%XR`QkU;l}#+TP8O)(i~Ij#75bnpM$;jiV>Fy_-wd zni_ltw&exYb!)2e)oXDOhmDEi+xUu_i4AN8zA{Q*LSLnS-;CHmDyf?jZLr1q#b&LU zQ-@O%SE-E@&NDMm>rRL566EPK#nsCdTladedb_Z&)*}H&(-c zH4IxdOvf3S-C*lqw?^wj<K*iNnK(-nt5H(rp6>(jG2b| z(G&H26fVx#3CXUBng(7+cbd#Q>Eq~5y?Lj#d0h-Sc{h-`n%-cIxBkzg1=0r2>eIji z|Kj!x{r<2qv0?D^q5o zvq3Jv|EYqhr#D)x$d#gmMXMXMy21Lj#-5w|Z#HU7ouF1CIn&JOW<#IE+4;U_<}TCWR?}ZN#7L0iVeG`*X#WNnhbyxH|JRsm>P@K0e`rYJkoBhu z$bWi~kv*$ohC$chKjO~LCDkbC)~u`MaFi@UNY5cz413`CjLEMqXOk?UoeS;nzU*2|Kj zjQ){~=|6U2Tka7{vZga~BWJ?ERsbz_3qY0a5kKC76Tqp=3|AhJpblUVuW_IR{I#b4nN+#e;X`dwP`(a)6e6b zSmT~O^c3`tc!ly8e9D-8Af?{h>9638srQmmf5n{=0WW8-5pV~cDz*wopPE0p=RMULy0>m6!gz$S$X zi?ah+w#L+_yPeK{;{@Sw=tMHZmZdX42D~c6x@h5x=XDp5Tm>V1KAlwwqdH+o^VTltpp)qtpWM8N2EAf}qw&oZ3yD zFbI{16G=Pu0%{X|C*z=hlMw%EE1U6JIYQ)zAIHl{{YHXloRj8Cg0G@$rPf?o(G?55 z6<%Kj^93ttf0FQVW!~J<09r4qg|2QMW4WchdiLy59*@V6`$~JA%XNA^k8gQt(acL$ z{Axn|@7K26&dyohd+4z0U*6a9Hk)@*y#J&=WA9ko^0>0UX5Wm99xf|4 zyn&g+hg{z2^hS%b_uT3WuIf&lH#7B#awT$J7JRv9e;S{Y#hE^*;#)sWP9Z7^rLJjc zEiTTgha0li!3JfU^pIku5x`bU=6?aT6JWl6Ik2cg^8k}G(z2aWMz(Cz z;+~-pe~&W8A~``ZbwM$8K{0i~c+|?QP9m*N>ZLWDJ#8|rG#vR_F$FjqNSwqaalGod zR}3`Xs$GDConQ-DP?ckO`8cS){IXIxXu+|~Q+c}I7bq+3Der#zoR&WvJKplfJ=y!)4@f6p%afAcXmZpV?AI&Zw~ftD|CeX`}VHBX`Z zK+hDENw~f|kj27DUo?mHx9aP`t7Ut@8Z%ZHE4*$?vhftg2P9 zf67)_4TmBjt~%_v>g z1HN=WM}9BJE$yi@d?9lAeeAn!ldh{-ylBMLx4yEXWe2Oc^`Y(qhu?YWh{s!AR`#c7 zoquu5!Dk+6X?b*N>Ek`S5B%)mkN;Vme~sV%0LnIsGdW;)GF?@&tyYT#B$)=^Zp(In z)k56XRr8fw#!5q@_Na^7BQDt{ILszS)FwF0_8$lbo9!p0gtPC322UEpI@}Z#wZl&y z{efiY-VFyOJ=eeka{l-CNr~?dOE)O{A8)C8ti|;>_GmM5R~~&KbM6BdeoP=Of4y9m zvTgQ__Ct1VS2%ZAab`>=vzkn1B_^|pB@?w+P~iSud=nC1B^^TMYC`5Vrj8_wh1_LA z?$*}3n2S5a2sy+EIm8G#^bw-*v||_k)lelJVyvu&WNE{F0b`(|E*5o-a@D!!yX3w# zp+xNhlNZl$W366-BSlxC>MClxe+rglC_r)S5HY-Newv?p`srp>+28yypLD94?`j^7 zJte(Yj3cMOFOyEjam*C3Mr9^9@<)=n7A}cTl*A`W;uEERn+d-8!GcnMY(afotIs*j zG|lz;r}>|i{ipu%I7#EX08|h5Pm_X4r?uFD9pK`+gOj3pp*RG)~+t zgqYClc5$^(4`|@5G=w=sf5jE%jM{zW9&u5e5->L|U@NX<`fz%hAjCtS0IgjBMzl=h;xWX z>LeGqO578r3*{?aOWdnmRtHzC6|SD{k$jjmz>>5MclC4I@8Wk$f9oylty`o=EUJ%t z-0p4)_bUpwI$f@AiWNz#^8(KWEXg>xT5Wa*u06L~qj{?H)%&<_Ki>i_)_tcEwKlTu z$$-;lw-Za+?Zlq;XwtdN!5sVXgWb%5wRj^Y9wwNO>JdR~f>hKqUt?M$AOA#DsZ;6| zNonL;c6n)mg|(BlepA|+|0q%Wn)k6l|qJO z{B#cyK8Gp9tRZKBkCVvw$s)?iB`IpL`YjeKw@8xJCUb7HSY@nnT1Z#Pl&(~Q(v^y$ zs){S}Z(>0yf34D0(U&GlzhqpjkJ_k%MIECYb&mOtdWYh$wyGbeP(MPly0B9}BZ3U! zD*sTp8j{FP^W7>=YZH}Xl4}>7Y%fiH6&1`|QL#!EGpsLg@<@(6;q;bUqj8UyZ*_$=S*nwz6_&iP@L%?Zku23oe=3dXUX3GGVR8ErX_P;?Hx0=w zoyWKmUCTPWW3X!jCZ=vt{;s@6 zkxBMo@{TF-`{O(u>XptNnR1RC%gFTS_+x6Z(m9xj_f;x`ed6aU=Ld(xYn5@aiSf&o zYox1{f3?zD`ps;&^$WB0eNE zBy(WSz}T?(ceoN*p@wY?o#KVpdOGhr>nNfjr z^jfl8Mui;mhl-Uxip-@zkz^@~hXM+yxl&q0f8mq?B|kgcBXMhfw#61P^?JnQyhy7v z6^W;Xh&|GTVsHuKQ=y@dI3Xk^5+{W~BqDhZu}a!d9*Gv$7xUs=p#XCQ8|DfHm>WkX zbqG7#5u)}EAyysXPOa8l2VEVMw%{$(UeI2sH4JfBNXvS` zf5L(Q8sCL#8)KX-LM-j+E$3*IgI9yH(q8dG`Ri2+EAIQvgU>$S^3?VQHt>a)SoOBc znm^t$Z`)O$-TG$Bhb-frS(7iE@tfMjs)}naY+{q&dXr7x|NEAQ-rCdh{>@!$?`0J` znf>;bH(IcA%d3Tb!(^>*Lb-tM5`-X+fA>k|`W$ZN>zO$zXQp*t4!R*iCyA>?T=S91 z#gMvaB#CgEgu^6!jj0cJ`6A_*p4gRFSnj2KR$;kjq#h&1`+wb)71!@$ea%SeesTyB z1@57lL!)CHlQZXKF0?IiU*%b0U+uZm^{A)O^Qrq2kA`|Q>h<`&UXRz~wD~f4e{LjT zSAAa1;4CR1gbOqJbgEM~{u_o~E4 zhOwv%9$_Vx3wu!&etB&)e|~;_zLcLA5?mA#Toe*q6#7{+%8+5|Ps*@Z-z`68NgFm{ z_-M%RQAkS}7Ih-gjG2nA$a)Ov^+l|2hl2~YT7vP=+&gfq%IsCnV1)gg;x{H-Ne^C^YYa% z{$+U4xbssdpB{JFgwDCcK46J#}3^wzuaXZJy&Fl_pbi#HlX0gYCWsE|ez5Ej>nxVp?xjngC? zCXs0p8Km^i5vnXiS1zthT&GFX|n?xZoVKq@ju6RDrmkJAdddg!13I=wa z6dfNMS8%Cgj%%)arhi80D#z8Xt3B6hSL81$SShV>+~`{4xmjD0e|^DiuJxYv>Dh+4 z+BrAw%ZNv8@lGrb&?(}ROS{Km2Fd|f=c_Vq%-|UX0axej!U9&He*_egk-|JxvOC+d zvjdU{CL~av)gp|fmXR&l^|=0*k?dTM?{+zqTy*&}tQJ+4xXKFh^RNv4(TvWKB(eIh zA}n$|0G&nfLs$kHi?UIyj?HK5n93SiL(<)uUP-UU9uKwY(nW;ZzB1!(VtpVLhfb_h zW9swWZZ{v-iMpLbf6wd`Db01Zk?B^Kkm)`a_mP1@m3`(i^wHBiLVS&-)f({*7Z$cV9l|hxp_tTvPsd?d4m?jGElHf92BAvoE>st6x9(-AZM@ z=kZ4yHdXXyZ%(YgdgbZgJm2z`^BBvD{wd$Sh}cv64Q(RK_%!My1$9$+oE6Vb6Ag)&3;ggT0m6o@54Cx>=J}ZXN2Z zLJb{pGf>pg5O(doTTI&UlPLQr&QLl0wo~uCT9228uj_aN)_w12dSr#MCAl?LWz>%@(Q`4WQaVtWW2Q|G1EFbaiw#W^9AR3 zuJ01wUgd5kYhClpgQdCt(3DP>cjBEgySl5~f4_3y?@qau``z2!U%Dl?&{M<=ZYL=) zH?21kWp3(1uBNGYgS3@f^?2|ar^V(Lo`&0)4Zw00c^7TQZQ zB}b>J+SImw+_z%ElT)5_)6+l|3bPn#VbWcggt!)u#=FJ0$CU~)6-X+e2!7opPVN;n5>ZDJ zRmYnXN4>%%Xl=Vd7vL2_c$26SG75P=0SgL9-Uw;bOZ@dHFC=+nDLj`8@f1i;_XmQp zxTIR#9LX{YUG|VFr|+Ay{i#8V2KSiz)+MZL;Ob>pWi^B@J9Ojft)nzsFz>0%f8fQ> zTt2yU-t1Wq#Ivp+SG{e;h~*>vZdW9~z>v9em^zWn6CnXHkT!Akh^r>9UUAifN6B*cG7s}O zSQ19Te2_r)WjZXOOxeNQ=@u(Zf3-!-fyGI4V9{s}EMmZ3dF?q}(mzvMO6cA#BRRTyIrPT%R;rB=zT${Dc5vSD^w$I_Sr8&wK&iSr-*Tc>|w!L=rv;)qS z&X2g{&YR-7+_TIhc^IvUcsHO<)nT{R!A3X&C%^_Cj|0y1F%!FPsy|Poe<`0m`EILN zZ+RIa4tdgCEq_lg7Yiugk;q{T7-LCy0&STju^mfNE!I<5H%VeLmLwKtgNeylgr3NT zWTp!~P8WQfF8DY-|Dc8CSgI`C;-+R6J2kTisY7GePmCR2yJwU?)9P^RY}zK6xzN~l z+y{)dTk-C~ljJci6oZURf9kF1s?{FF{|N<(YO}`hfmx9F61?R;(vq!8lGsnG+>!O= zZ{KS9=fcl!eEgl9?crsUR&U*W!<<{#ir^CmSr)TD#`yB>n=AvIEJbb}qXLT;;9-E~SURN9F`R)_#G1Oy(tgx-!Ey&0m+OI*j) zPXeddNw>z*CAUNKKn6~&7wqXM^+KizOT-hxc_^d#{OSoYiv?bfS>bFZC=^UkplOG^ zO!Fqab>4cfoRcKZf7N$Tygm|%^XN50iC!~gNJO$u zETqLg!3{n!zdqd{$hQd7n!Lh(;|^F3THd#$EHd%ONQ-327VAgIGfTFkhT}cr8hr}R^f8t}gJxE}o#L_v#Tjphg zEy(Z4Q>ETBo>_MIiaD=cU$?&NuIA`tS6unfmTQ)5T6y2v(+}Ryq&1_@+Wx!g9xVNEAxV+_^No^{$rjcb~mk!iQUe!i7fiO_VyQWjIC;ex&+4aGu`q8jZt(!LeyDwsr{ z=N)0#y7Xv>O2p@4ie2<5TVZi|gNwN$Ihfm35HF|ne+g1=IV>kYDt9tHC{U&gQ)9PH z@PU?ZN#+-qn{%zNdR+oU+su2Wn0H-PWQrLVyB##|z1ljniQ0tlW{+YS^1wI3P9AOq zQP8qZ92Ih*wz7Far7$Xe=B_bmtc9jnpfCw|{g&JSO+Cwv3#BDpw7=x<`#x*=lKI~` z%-rnze^2c@S4>;m{1zYW>^=U*rH``l!3TG<940X*D{6VK;uM9ouQPWm@jEPIV>qC2tjen)~yD294yfCGM?D?CD0c*2R2#knLd5+lyA%>u&_)7nZ5AulZg-USUwrNAkeglE@b-zz{&e$G zS3f-C?T!ES`P~m+yL8LrS1;KzF*3TKbo!)T4L7mMckg0s?OpZX&-wP?l5JA)pPCN5 z{DB|-cd~B=q#`$%X*flY^=$~0zv@9iu{N@ly;b5J4Q4NH3cdPbjK^iuDx4zTMFE&GlRLTo(u*QTwe|-e)R2A z;d`fhV_q@!YPxsT?(3Sa{Oz#aSIixCbEUGs`K#M%H{aVlg>PDQ&6r!Re{FserNe5R zT)ahRrPw3MFt_WQr^3n?SBtnZS~iALqZ5ir$RujBM8rvKT1n!nimOFjv2n9$+UbO1 z64 zTaInMt=@LOQFO$Xve|QNe`rryWG>lM+U`hp7H_?dfvTvoU9}V_Am1-TabVT7;+({j?J+$Rn)nnP4G$^iV}p zoe;W#Kns0g+i1(#|I&JtrAs*9t9S3-Eq`(F;OTTZe)=t1r8t|?f46kDfIqia-v_s~ zIX&=|IbUBTyNjt69IuM2V$edtspHj2Hp%1qhw_ao+064@dTiWo5;l_%yKN%2K2G|D zo%_^iS}uh`PVDj(mfJ|hW9m~xL9QrB-hd^lEGx3w%Qi?>3e?W_iS}PeSJ>Z@K2j|Y zt1PC*Ed|yJwYROxe>KuoBiE=CEj6}l<*SstZO^HHmS0zos-Ic@seWfo_u1`=B*|P= zEjF7KGd8QWz@qvs7FCkv0>$oE6uTX#TedQsN=3C;aj}A3Ze*UMO_4>=AHBH7_Ew|>d(p;!6dm6_Eaz_*1RC|-LNA?vj4=zcJKOqM%N%y#y_QYOl>Og7#Tr_! zY@W_@u5P&ydH;G`0KG_(vEfNqTRC=?>`HpuORBS^e-njs&Xz@I$sf`zxK@O;QONA< z)w^7zN$x-fC-dE(2csZd+Im&#cUP&o3>J>YebX7x75-KOX ztVj9%z?#sD=56)Pw`cTDRvW#evy~7$U?r&orqn?qWp2Y5;s-{^XNw<_KNM^s2_&qG zg+G)we>4Tx?^gDosy6HZHICL~jDNTf(hQNJnX<)a5?+(YGKVkIBr;4QViH+;M9Zi{ zM21O3Ou}jIaJfvvZ4w@nNHgt6%_Mv#;Wddg(+tr}!e`PtX7k-Q9yUN!&P1& zAC-^VJ_sI*DuqWtt;4Azzc%Gh`V5& zmw9F3&WbHwuZa5zch*l6=_Mf=q{;Mh;nX^XQ!65VUh`;L8`qZACbL77cpuEoZ+VLCbeX>q^Po56gpbklS) zeXyTWw8;ADVn-uL7u~WFF`KqF)~3>jf9G6zXU=sm-nVsEZ1TDDf4zI+^z)bZk>hua zm~!#N{oD687xLd+I;GDYo15?CJC`gOwc*z0Hw}yWV_ZoAwj-IQNNO72qBUwCNuQ*h zkiJP%Wzv&*$Z}U{>@Mw4=tw9PlA~6?+aK_uMa|TJ%kFYJ-6qqz%@yZ1c8`$Ce-8;O zIwUM=hp?p`!j^Wl+R_doMjUyf2DKEnv_sg^nEy^^4u`O%9b{_@-Q^JGx`W~W93w&` zwo1sZ4xI?``Js)WhEP*T4oSQ$Js|idAb2PsxF=AcpLFtijRSZ;YEIkRnbY1j=Co`` zx2B}8y?H(&sD0CRn@TsZPYM(Jf2FoUSDxZ$(d!_GnKVC#6bf5v)=I@@Y3 z*ZTPS1bT!9YQucP0s}*nmC3dXv|3+n;DXRRWu9%iHqSRNFg^4OmTprO*M-tpWvu-| z=cUpNWrqDyr#+Y{Tf8W!{3gfv&8VE8oToJ54CQACA3H;^j)lBw;bU7w?9GTSn`cr4 zAq+<%Xr?TBU`G&5Nq2rhe|a|x1B+&fS|kf`nb`HTbblYE71fwWVH>2~f?eEBea}U3 zb3QoTblO1tssKcA8!`oVh`@{?KZLY|fZ&fLzK#SF2NBW%@iY=fc)Rl{2@ZRVoMDch&{D2${5?l%EdOB%oVDYru9Nb0DX%Sf7-g<12*6I?At7G z%@;Sl-*SB4&Q&XS?pm>GCr@LAw_MrsLGvqL{E}re*UK-x{HJGMdJ#Ljs%5sEi(KJ@ zY_>kR*r|2a&e4Wxa#ge;%5$QfoUyFZ^wO+;S@WaoqSijaJ{d!ULo;fu7dj^gCuhvD z&UMb#<^|_wG)4d7e}6ahZsaf7NBu{$k3>_^Kuk_(iS!lMA$gYiB$d}JJak*d!4=BE=L7hL6af4Vh(%A$e%Jj zh1OG^Ck0#BiMM-*^PY=YS{W}h!m@fmRu9e=05qX7%{H6=5NjK?)0K+|wVXV9CIq7g{Lvs1 z=btK!C@t271!<-}2&DTtg<1-|QkxlHwYkr2vsNFPbH)4DO!`$9@55Iv+4jidMLSw% zE5BPadi2`Ve_ao@oWAM&KFz14&96N3(&3k0e1j~i6(|y(!+H0@d&%Kl)0if+m|QOR zm&eF6<;AjU^IC0In=8$0bAe=K4l!Y1w-v3kGHYHmjivECuNXnE7)`GjNAFKZFj_6i zZMT922Qkz(-pUBypl5z8h;dY_J-P5Ge-0WAS%p_@%T-i>_QEQ+IAdD7 zkbW?rk6zFsHserUu6W?w*;N-_bZ)jqP|4uqIM+&|dk#7>l_p0V6sOCqh=4Yy8e@efX^kr8}vKAIN?Q$dUxj)2cgO+h$d4jwpY&68jSf0%>=63S zOk?x|qo!T{K)HzKYi@5jV_ha^U7E$SGSe-xzGp4sBOjYFkdKpT#71&v7e`KnMe@VR zf3#3}DK+mwLL(f~ue7Se0pi4Plv;+P)G91|@vw-y(_NhV{rp0(q`ulKA{hC__@F(R z6^DvhF};)87n_IOiuKczDiNQHAuOidaC%B{MD!rHu(-Ugw5gO=mDZQ?QreivhmdZ< z3Vx63gUU;1LJqjYzO6yp_E7%|Jo8 zyRnZ@yI^?RADp4EP8?r2!rT^5Bo@$aLK}nkIPEoJT6KJZ*c2zfjn42;qHl{EyW(ii zBnvxd#}t1_+^hMtG)+?TT+s}$6=~@6&}LT?e{u%o zXjx?W(2o-<^@}i-ixbP2FNZc_l%iF&X9}oNTCadE7z^WtU3iaj`fX6Vqe1wW&2_~Y zuiELk@tUPedKBFL+}$J3>s@^7m}`GG$=l#uG<)fsK%i^J4F~QVKl{0B55CFH$(*}z z#(;BTp@PyO%SQ~lswgKh_?k;Xe-})?pjRw2E6tu?cHYv-lkT7J7|GWB)K|P%xf_Bk zXCF8XH__4Nmo++JF$vWq?8N4=I7O?Pl4Fpl4>Mp+mz_xv&}<2h9d)MU@Mw9E$6RNr zF}tqDoGfLr4zvxdv&^^DTh>`*K;vhlrNPoLTXfwhh_A$ne+}c86b2zS zqI5G^H>*h1Qi#s5u5_c(vY*d^5bL>PW;^d5729zm%6#->rPzIJt|X=AEi2PrXmgaZ zAgJ#^dSgAxyuDCT$Gm=;L9Rv4ue|uuk{fQ=wP#OSq9}XQ{o1)R9^li~GRvhcH?M8J zeRxTPqAw_`kI;WJh+V%Ae^hCPuC`~2mJG?s5o3piO?z<~j8n8rt-)qe>Tgo5N~M1&U;6#h%lCpLLnPt62{@(Q+kE zkfnmGDae8&A~dmuzV{-@+O7pnv**a7H!G=*J#k=|`uu)~Iq) z53z~7aYiNG!yKy2Z7Nq?cCdSdX!mp_W2&l3{Dd>(36GC?^1>o+m>drngi#YyBv~Jv#TENv1`x4b2hFOfXxqJlep_S7jqRj zCBM@t*=;sSf8r|Y4yP0BBYn&_#C4rB&+eJ3+OA958(GE^Np+N3uS#ko@1Jx>rR#X! zNF3}T-lf8>sy%r;5^k<-j)abD$F!5j#qZ7Zm@i$$kw4?dJM;^2pzeD-tE}SupMJ%} z;?^oFt(8Wocegu~RpAgn-N>qN<^?Mx1m$<;R%qgfe>3S7EHAghmdrFyLnms4pDUuW zlr3YdEJ#1x=|!X<$ zWkYLPYDo&`wv3kkj+_+Xb5V1K^vyQ?bdFg0;_8u&VBE=MSKSE|p*V4M>NRw?P+VR5 z(sY_Ze?FZVoo$&h%kK9`4kQm=Gf<-OW}b@~d|e;V&v-`cbr{dd{7IG)m;^zQe6ZvVvp zwX0eC&Y#N6Hd!~2=E%&DJp(*9cqC7_wfnkdt)EpUhVA9?I5n@&X9p?lPfIKC+5MRD zI6Yoxfy0i5n>~$=Q#w?70Ay;JylduxOrF`utM+(sERz04J~rv7@+EzIitm7r`x;rl zf4v@-2Lm(gbW4m@GV1K+9O;xsIa5yV#2UN0(m^g>wL2ra6jcolUh@L9UXWYq7iS^u zhKKL$?DZ z<1&BWlFGqBZ(MP-%zOIXL|#tfquni+o|oTk>G<-NOCHsV@-yaovgD%XyRTTj^h!SW z^dGnPs~IEyTMwXwH*8ea&Yj|^{Mwpf75{~$2RKv^Dx~{ZL$71#QTU!#CSx+C>xHLiWozAth zn~ z!n7-X{{;QJ5zoUc`sUVe!u}h5`XTf_4r>d1=kE|PUXeP6sjW@l+T;3m7$TKHJ*IOJ z8HmyR6__WV4-CiaZsizA1mrv#r}q%&$kSm2<{9>LjItXxP#!T{U{R`Bz85x1C!sgq zyINh3-!UD>yF20==*s^Ff1T9=Scd#H0H41caUVYOQ^60@VJyDC3#MiAF~LJC5o_^X zUz+1YV}tqSIIkDryS}Ftf$hg22I0KaBQB->;QPDMnA5zo@huft_bAqxOu+J>0De31 zB(VpZvX9n=zbNHMvLPcaDf4U%!As#@?Mf5=o zLloh=0pBYLenp-k-W2?SyrDdgV~BkdJga}hePTZKHM7Zh7QH7|eGKLr$fak}dLf?0 zzU(lcO>2sH#!Ll&&lPw)@%eorV4ampW z6VFor%#_B3cukB6f3685t!(qH-NYKfR04<@ynelznq$$L&VtSO%sTaAlqnlwu)G)s zOSi(s@(CCqb%HKRH!Q>N!MY9n3%J1A1Z6mvBQbw>yYw!CS?97w%Q(dJwFU9E$J|ouSc1N@8Yk+^#po8rt{nI(OEp#o|nwj}7f2e_P5MLu74MUy`lWrjSEc64)Xv9jLUsI=A`SS&M2-BNP{%psmbKCK! zvprw7Plawl`EIUL?8lAf_mg}i8AmdbMQkf6}T`C#G5qecH*?rFuCyOg*ZU z!8}8rZ5Hz8EBLk0qvJ*YZ0dGcrkuukM;R#kyU|#KIG)(=xpEzj`36{v-w~Ep;d;b! z#AF&1F&6+K(hH;$r8{t(Nf+G=*Gq4sy+Cctz$^4c6->aszaYv`r&3B~$^>{o{Q^qm zaVTq>e_%Sz6@3TwhvwdT1^T7wC?^g>cllAQlMZ&Qw^58i5*`tJL2c)vNrhu!nFbc* z*%4TWp0`P~NrKN9kIiB%M0>PFh=*w`@R@2lTp;X;FW`P<98AD9w8>Hro78c*=F(vc zKI0*5H;(#(?IJ>7-vJlm`dW>$X*J3m5d1$We|1{ghTpRUBwnChEM7;y$FxeA#z#2IL1J${3GnU<~pq#|_-h^d0;QH-`YxhQMpJT{v zz<1w><@}F?6eh2qR z9+)D(2=~a1u$J<28Wc$l`2D-lwxsfdFpA3gpD}+IOp+_H{c5;Oo&t-c9mspHfnA=7 zbB*nkTaZukvHe&0Tm$SQm?VwIwY3uIfA3O{Q9bdF-Khx#c`$Sq?b<-})kM2K^I`Z^ zLvc>AzeJzaU+imZKg~Y=NdLrl(sQvL)sgRkN*t$m5CwYLGMe8E+YlT1ThLz`4p*@) zsr@*v)$K62ZC>udmLj^yJ>Ut%a->Tz{V&9JJ&(3q4|p4~0-ybROm~rAK;ZqLeuC?zumY0 zE^vOAA!^0={vFbF9s)OeJ@pQzf8*Kfph;IC4kDu$L46kQJ^RDF{K(H>5)?}ts87g) zg+I)zBcYJM^RcfHYJ+PI(V9PqG6F$$MSB+vBGB3;@ZIRIAzJVEfPsWIf8*Md##P#& z-#6#SoR{|VgZ)d&ufh;a7FV|CiCjcz~n%>O?D*UpQG z=Mm5SKm9HS$g2nqp}vE5NEOFU0c0I#OKDe}IjKT^Yu*y+&KB@w_p5`_q2}ICT=SU2lJSD`F1P{|ABk z{Peq+{vOl2u>R-R?gmVs*6%k@!Tgnor!fBo<}XD|M0y>j>6n%v(hxpuzn(z$$PfI` z|Mm38{f~b)MVns7XtHI=eirtlkqZA!*s=It0Kb`%?M(J7+0JCYV!BDh z3>49yapdQVei?eBt!!l{;XZ_h$S~5mSm!%ln0gguj|chtYxIpa6EN>ajMtGmh`P&z zy6ynV-LEly1@l>$e|~N1G*hVXyW>C3x3f#k ze0%)hetX_C?U@KfTG4LZ{ih(`Lf&)6_BZ>`ejT^2HzSW@e>AWyP!1FsI`jdQ7ij;d zvJj}FQnz8*b=L2p)cQD-Vtx+-^=8X)OsCU5Oz&qmQ+zwsjQL+;UXx!Dbtf8_{&W1? zo^Q!M7xp2JnUI6)sQ=Iv(HG%E>_E&j=bC&-eD|9iwJ!O&@}$()@~en;HsH^uJzxRi zam;%#??GAYe^_l_5PLUUz`<9-Fd zmK-p^h#~vUSTe<@NMEU4KofC{wsGW-pu0R71|X_re|?{79L1cyRa729yQmowoZtb1 z2MO-(1b26LcMsaZ-Q6X)TX1)GcXx*m`;g(^bLPz6Yv$_Qbk}-&b>Eb9b@f}%qnB@F zSY?XQzg~cPl~B_;ufGZZu$&bPE}zX^X^}o-uSmWKJgqs?9i_FeN<#rt?@G5za$SoV z0^dd>>IYaN?QHGPL*8V9ZYc?7@7qnetyoWHs)X?*qmAAoZ2Y-=5Di;8^kSb626r<{ zk|UcAX@my#mP>C}2af^rUh`wFJMdV_S#L@Q64o#^>cIEL%kU>UyyS!I+Cb@5$YVI- zg`&#``L9$J<9yoxz8_BZc_HR3#2o?L5!-TWcS(>>JdTa`E2l?H#5R_ig>tg8Ym+)NPE6F*#(31F-s zg|%rxpB{?t4O(J4N$mnG5NZ*cS>5J2$Ih+!^(|kT8big>c?Qg^_`Cdla zaGRqiwkclIO!w*cBo&C&Fke(>6kXm%g1D0_M{^nKX=dNNlO?x&w67UA?DoX>v;dh% z>jH6>lwO*=`eY6$J42A3RwTcgta*`5&f3vRD18|D7!|pM33~@9xsgn%$%2^wFwu+fT~SHD2mR{gVtxLqsoTY( z#2*Y8bdbAn=Y40?1+f0vxGMsUf37EzbUeTZgN(06tzMt5Yu@;-ZR6U(jFCia-!~gq ze%twPJS447&e;vmbt=?barh?$G<{k<)aN?*s(F7T{t7X>NlZca+%`?0R~LDmW9Q?S z#Qs&kZL$)s(XJ2oZB0SkiR~`cJAMBZNYQHW(`|!TR5# zH0W>FD?A@CuZ*60J-kG&2zS7cJ6_XfXY?xuOwS8Gw`+eC!;P49?#u#~oIV(Zchdq- zXf6*Z6tzoJ_b2dWHbhIsKK`4zd6{Vd3iyUOvgT40ESDse(^?C^Q+QcP(CdmZVxv?$gtVN$_Bq%-qRN3eE$AKm=LQ&=UYsi#c0K$gU6 z$1rmEBZ=7b^x;_7GF6rxDB;&z(<)uG8*oX<+a4GIC4E7Rvn8vNOgnOX3wF*Cst zcLF=CwV;)v^U120Irk*MVDRL0OnQ>4_f>q_%`)Fm4e&#}B)7xX-2d@cpr;>kDLPE`4N-oy|uW(&xo%{S%1&Q8(aU z9nvTQSHSB#{U4)GnA+W{+2SfQf$wPMCMYy|I8DRCR3Z$6>v&>L+E=T51D9FzD`vN^ z15WQnw~zdp$?p}HZ)$+I9l~xzR_e?$FGvg=c}sGIjK4!Ot}4!)v(v;Ex)k(#y6YTL zn;Mm?jCA~$Pa4q#bwk4w1mRjl2W9sV?xCGB1}(RgxE_r2y%@~g%hIE|>CuZB<=mh| z#_F_9mC++U-EPFWw9x(js%SUaZiY3|Fy>rq{Y2|TmiJ4%pe1`Q}M1`n3T26O#6;U`~7fAfwFp&u9F zd}G@SspJ1GmvP)}Hl;yc8%}yhkq|qC^|$8YXf8Xyl>y&+ORLXhY4e&Pee+XhLFy4Q zp`a%@{wUTDIx((*5*P*lCM<`a&RF|Jv2wBo-T|aWZzg~=#@#sv4fsEwAkKP^tDXj2 zD}A%q?jG&Ed67sMg>^;!t%XvrSMKW3e~U%$(@^jp8QyvsU3mC|3Nx>%c? zPz>g-@S&H^v-CxlypFh+yq+-^Cy@JM9nJ+F`8vIpZu7_ByP){~O>9BFrN6m9{53G&{5N1+Fa#MrUNPoRg_^#v zp9qUL@*?=#Bl%9)rBH5)zK24+Y&ZxK9=`*##_cw@Yh48=3p>ti*$yN=0sjVhUCr_JsaznHQMzMgRiKQ#hUl2^00GcGwL+*kpJI-lctP zO0mvWt`wYgrRv+|AEcDNRlTwL874h6ECZTykANo--nbh4T=JTi&7FSjWX9?f^N#$W zypp~C*j$IlpKjN9c;f^%87ea-QTwaQnPV%1Kk$McqNUJbOEwfE8TwdkTeVKXX>UU> z3J^NO?%^^jgvFcweHlqw3`=>Fe07frsx8STNSE`rdzldEQ=()Ob1nIX*6APez0)e^ z5SewSamNk4!ePbnyI5_Kap@Gca|7-xO9K0>M8g$UP6SXj`omTIlQ?E*lQI995q~Uv z8Wt{70pZebY@g9{E>gJ@D5Chama(``*}$Us%~wzrT~%#<2wmr+Rm<0%O9Om#a@6cu zUBsu^o}$8AlxX;MXUXJvK}dD`Z|lI#6i)tT_e(~*@lw+2N`YM3@#J!#Xu6GP zI^hEQy{&w5K3wJtk8@sKdxa0yuT&1)*P1Ie=w0txfr+Ij(>9$k^oiJx#>o{h6i}Zn zvn$#3HD^-=qU(wWIrgRzJ) zd@b^=6ZJjpJvhZ~Yv`RIW*4G@MBW~6{-eX(EY=z5eBSDEaC}|F{%wEexvFC%n)40e z+ZI^AT)Cc_V1)B7VGe+c`q4CJVMa`CmaR--a(UCU@b;~8C;H9bN5V-?# z>~rmd2>7a$Z%wm!)$&y#dMS$I4@ZDJ#FeDm-;hz%2a>2Md`hX2HSF=}3F?HNCAJ9f zJ8^+Wv{*s$82`n}kk}B)5Iv#`vEn>}MC~Z`ppUzWE=UWuCVdWp-_E6)+3@IrQ z*V3#^ZR*-qvbm$@3Wrqmrte`YgrPvijq;Ki|5dzv>Sj z?j~NFb@hJV>k_5J$3gkfY~45}WnvJd3@#$e~e zqvzK4rJ$NHh9UfpPai~IGAsgsRfPitH;HJ6Rtnm`A?dQdvH2{l{svt&#(0Ex2I_!E zCv-6iz>u*=mMEN1(5Pb!uzET5vG4AOZkaKEcQ|H&Aq6=}cYC@733qOUmO~cF+|0x} zC!8m-*1XV+$U~RiDuFM4Msm8n?48gGeH6m@hJ1n|CocKA!5)2FeWJo&J~w}@!XX~f z6K;!m?2Od0vqm@h4rDi)xS2$_qDz@HZn)^kEklqvE!A{;&{hb64UE$B*>Ty}(tf1q zJK{V^M~H6h#}0^ zEt|KV8;4Krz(w=JiqG){RxN%m$ulbPYt5fojyq8SnIQ-MS6?3)HpBkTqD}j^DBax` z+SjmKm%Sg3p}N$NCTLatJl%YJ#)OWz2a8hxeK;+mBd4xhmHxSLy-mpXJY+_{6p^_p z>iTB;4@Z^&ri27(F-?qC20ziENFHIViwm;tuR!d7R_q?;6O@O_rDbM(ePqlh$8OU` zqTahUcPR>byuJj?C!>QO0bXZFUemd4)Q>1QXi8{zD!S9$T`}q&AL{2tfs%RwB~U)| zP1Pc->G+#xEZDZ8AsW7-LcON0rJ-4+0kf;+>%u4X!|oAZR>TFkYFA&EPc(=$aOFSL zxzxj_LJO*Yy}!gB$nt(tg)5iRPt`Bc7wNKukn%0Qn3CUaF5)cUz}P)26xvw#zvR=! z=?M$6pA5d@-G*Pm4Pml7rcg|y-T}z{BGSVvY9u|i)W9F>8!Z9&$yZD?v70EiZNH?N zB9rW6e&Th7vju$@&=y$}O+aNWwjBtZbUYPV4GJL=MX4gOC9!;Tv?Zb7RMpur^D?lR z7kQ|`)ag-U?hs_jXMZBiK3F|XOYFi8=+NgB|J!#-9>6je4nmq6UoQiLbpibAKHfW) z%%3v9i8e}ZthasQ&$?isToj`CKNJQzxNi7oq*TH&+@Zs(+##e)7eCP2L)LU-?v2>( z1lEb^yF3=GWIO0z={Do=!Y&AB@?OeznsR@QLU9K~Le(iYdY#AFh&UzX23N$sH5>Akej1)ClB*_uDyi6>;KS&<8W(jbQ*AX|5%D1^BiQ6kw1}wE~z0MUT+4P*(zJKZcn=vRs9wA8a9f4(&~{ z+SFl&kw=&%;(ds+Plc5=;+sGcq{KwEc2OZH&qwoAs8`D1&ZvdVeLZ<=?4+aPe z@Ck=RH08aXzX8m`1#ZJ)0J=VVzJv#bft#g;7zOf@0<8_gE4}H>ry|6Nl{*=jA^8Dc zCWK7udfa>C1GrzW!%C46*<9`;4B}k?4`vAK-Q1-YTXPt8#Y!+iF81#}yV}Wo80PyT znV|KF49ED`hD}`MS|HWE2>~dKyR{FFjZJj}b*;~R3UBp@3yO8s=U+sHy2wFt8^feakS8?wCykPcWHGK)!Vo-HL;oNSq-~98*!og z;3Tp#E~?g<9!RGiIS8w5_5kVfVQqW1@yBCH^en|!*vBqZ_P=tid}l)Q*`((6x^)fy zJR6DdXtV#hWd5tC4u`=ak-DXomsLi=RhD51==;5@lJD__{O)J%;zFXDN zalcLqjQ1z|z6hbmmv3Q|rM|o9)0v#Y#2-KsS+u$^*Z_m~h-No_f2I{&C0!RC%NyAC z6RK4GRvDK1{pc3MV)|WcQU;YU-oVdts;T^-g}iW)DSrk5>0T3+I79D zP%m`-d<4>8JyUBtxBqg5Xfm<<--eI>?FwOMW&6R#`hVgI zVPX02t`MeV1rkVN;9N_q3bzqe_lKRAq&&F;Zf6mevCPC3lUMong2gGOgBMPt__I8P zc!YlDf`#wftIys=+}!?9lIvs*;!sbGYIaiZ_8O;lyK>uq zZbrjX)cRx-WqZ3kO%loew$N)=`7o)W|H@O$8&H{5LdZA_NV&W>_I75!E0tZoGf(9C zJipPjOwPNV!09edvuMKX+FP+qFZ0R#T2YEXWKkZfX3t_$6=hrESmlV8xD9yee`i=E zOjO{fF!D4#B;^2^fk6EA&#s31qhe)sH!Sq&CJQF*BVif*`EEB`qk<`6<=rZCOw>$L z$K4nkRX&Qq1Mpr&_%1afetFC90$Oh<2{~_f==+fm9->n(suUGe+bq>{&2|F@H{&lf z9&*6dVAv||_25oLHU8pFolM3|--}F4q2MC!Y>u!qc;Du`e585Agyf46vCzrGj+~y4TV}6&|KI}gEpchE} zj(9Sc{VUjQ#~TR;IyXhy&KfEh)A&$ojiQ&ji6LkmQixoncBw9e!i;rkZo7>m=`#({ z7Ftc`g26>ZWoH*x{beQ`v>txinO9G!cgmf^{a}-e8^%4dOhbe>Joi<=4(kMfaLYn^Iyqv+aeeLv-T@)G`^EH(z&l z0;o9+WPV`Q9j(7G`)fUsbD(vm15{0_g4(gUPH1_J_l4e!>5dP>tta8BPZt~gSZX)@ zabl(Wi(X2IoraO?$u(9_+b3fgi70m6aKL{a?fIWacbe0CJM_E;z}AHPhOb+%bERcd zj;N1n?qr#(K~x1>6d4mD=m^VWMVYZB)gBw!*>;6kOT5osI6SC4Bww^&5J?}QZe;hQ zyus6KPugcz4?eX{wpLv~%zxIYIVPz$L1wO#*C-hsY*LWck5~B@{uK&N|8o!^L`T#6XME$9*WMGHnnhMDQ6DJ=y7n4|T0CF2NCQ@FBz%(>Fu=OG9% z%s^z6%?+C{d;&e63{GF+qXb5(`{qXj#Y}xH{vI;G!u#C;Neu!ub9*?o(E-K1nc?;kr*_^)V?8$4Z5jVdm^48e_yV_uB1Q`Mlfln$+Wv{!7F zON=0s5)!r%!byKbG|;5)pKK+#X@@dKK)g0TmE5xiUdQU{Jn9fec{xaDyQdBDlzo`0 z>MP^ag=rmaUiS$grXFsDFQL8Z z@jDxAP72_|Z>!I|YxcN0{kubyEa}gZvZmb%jqe?fh=@5<6EUDYpn4{3)u8#X&_ix- z;^{Ea`p=1@5gY79Eh9s_`(zqd7x5ECGG*9{r?tEj8ynsbMYqSicD5LrlQqfyQl`Ey zO#A_l_jYAw)23)$2(<&)lSj%`As(hS>`w*5NLZYETwKy^^kQ8+6lxB*PYrU)tp|IC zv+)JqwwEM15LhJN&b#t>=3{YLT>qMr=(fQyB0IP5nc;EN{|pz(sjSpB<1;u|cMB^n z;1(x~pj4Ji4vS1J4lY#gD}XcnBUjLc_WfUAV9LK)G8OD9S<*hhcwlGZuQcB65FLv$ z#UG9QQfEOdD5gG@5-uC#)uck)z%(XDtuCan%$mb+Y2}0WX_<|bDJzWB zg*Aa-KkE}>1U5WiAnZ51y0JQW#njkpkN2QRx?Lt)^tsp&SLhn<*{k{_{?HI#z(>Cv zL-Bj6m!b5>wNC}Dec~ps7jAdit$|$k61#Jh1&%{h=}^Mr#*YUsYUE}~KSs)8#w#XA zT0-sgMD!Q>F7MgcKAXqQ{gAgEATgZGjRDHYjY$roQ4v6v^gH(5F#QdsHWFaPCsD})!E*~mHGY?B}d@^e{!1sufIsgV|M0th!dzZ zjQ)bhjTSzq3u^V|{TA`yQP`-ycB$8WY&4;-7yw)ovTA-ZEX~OpEi)YlH{UEX5v~ae zxAVnUEQYew#t;?rEJr2A&roFgLjIxtRGza_MRTHcw75{qENhtdx{5?9f80Bo**#hk zcr4=OFmNb2rG&K<(_(2U$MrEJh+LH=p>UDl9{=DVSz76pfYUa$9vwK(ogQ~+x1VK7 zPXR<6P!?$r?YYB@E?rDvXLto+HK=YeXN_zRlSfcg>E+eh3Y9ye^BxTz%)1ZZ&lpGC0p>}|K`C4XjBk8lu|F6`U~uYNCY{NObe z{l>Z(LQoBZfraD1hMY0ay!vzdW(pD_eM%V?{3-aYx$ER!q9v|O(?7yEe<`79}@j& zp5hua4$Zk4C3%aG$NCU`25XByR=CDeebz5&?=QPmm-83jAzp&Z-v2wp+}7C6&=Q`B z^*;dl|CD%m{wG?{aYi#5 z0Dj?QKOgErUM%}8s@f>Z{r%Zrsf)^76WzYsxop5Fz->N}4VJe#B?z@;ZO%KU5TJRv z>o;&l*Wx|y@C*Lo<%Xkeh12l7fyGCMwD2YfV`{7rV6Rq%>5w2;Gb2uES>-Vmv+aQ>iUnkh`CU~~S|BK{r z^8ZxOe`c@$zZLXflhOZ9oL~Dd;(WuW|3#elmYI<-BN>v~?%eX#$`GiXY1y^5^tP{7E^81jw{6D05UZV{tC%yJOTl|A)Vr}Q{ zK((>o2g_~)Uz*6ieDyLqxTBoqLplR#zKR`WPrB#7L4MOGGknw06g9|qhJ3MZ-EJ&$Z@=%MXjbeEwN z0#a@$g!dIgKJD=dJ!Z_Dm9HH-XDhz_ytifgq>s)sY}y@PjhkgARZP;d|qhp zi0|olR!@vih@;=UPW8IAo8Sn70S3a;VbR*#CFzEg3ggCF&n2IGE^tQnxY#Y{3$8^W zi{5kQ8gA#`y}%on5%_&k(Cyjd*~VFR?K?2!GG6f~RvdZM{@$(2!>caS{?kVPrPvjF zAp?xlF8eV2@^@DZpDF|P8ki-r`|nnL3c6;1kzTdkvfCLX%u6ZH1cE!NkCGQeXU2bF z=C>YP-rU~Wza@p`*DL>=7k8QI=O{~@(R{H@exn-P`{Iany{6Kv^g)@==c(kW0`OEa zldL{4^PoNdag1?7Q=(QXPcv7m2F3?9*d3e!6a*-F|GI3y&E=*IH{yO?C_5(d9l8Ok zB@Js>LsPW$@q;>%kl}r6d{6bU*TyrAz8UT_+>E+#^1cl28>)?TlY!78m>YC5@WCGRH3ww$9(Jxnu-cFp~@y*EiNOp1qu`rJH-6?|t# z7O8V`D@g3OwvGv%1A0OpS&*~&xpk9|J z5Y7m6odQF;+jMpz^xk;N+z{}Gbt3=in@m7zY9MJKC zwDi$i`8+Yk{Jr)vke5*k`1y10CSA75o?5-Wb>7~AM68pmVZA~(Hx0LG_OaDzJHPzR zVQVCFLn&#+`+3G|&j)0D3BIE396*JZ^vuz6{k|D@NKroKMz&$pm1KYc; z@p~z+>zfqLVv}#X@1-Y(^x@$FU2LJ584-c40F5zj7dM~Mg(ZkRau>L_fz@-Ta4Zc$egI*2)`HfX>Ra@TRpY|S)T*S} ztiN-1d~E4&S(x8uuBN9y=681i@H`NS9j=N=qXw#~a5uXu2Z`eE$nI31!7_57ziGpD z@rVt|mvdwDb>zDyn(Aleg8ludSh=P9R;<7S3(gQ_TnI1DH z9&+lu|H#>TlcOE2D&}KLcB7J9v|5`=vju}T#FKLG;auS|Z@biLYIX4r-Vfc>vG9P~YOZG?%Z1Y!2a+!yY=KbJM}b(%I9yT1wIS!!1bSkr{Abv>@u2Xg>vR2-?Bvey3WGm#~-|j`&Re zgYr_KAJ(jnj4yq%Kk1x#+TAOwFSs^0qwd>P3KKG}@V+~{$V_j&&_~+Ll*LgjU8VtF z*@GGe^yI<0`YdpNewU#0TO;(%AM$4^Y!w7|Rcb)cD;f*RKH_h{zwP=$T$5W@4}i41 zMqMv$fpR?AydH*(A#)-^#kj~V2}P98jVz9-mq;KFMkES71WWY(pkMy1kXX&ES4 z5a4XbNQyQQq*J?OgiEX>AlISA(P^|7R2m={O-}sPF$!Qb7#xa52S?{hlbtvkOd!aX z`o_$Ls*%qN6ghLX6VihnI#_leF+qcJTtRY*#}e6WE!xDlE@UV6#dqS7mcN|1jwM}a zqj^AMqO92SXhW9bEK!?(Q;>-sQFxl8nR|_0e(LS3@4F9IEYU>4ULF#n@sO{O^R4hR zaxi+>dI3=;QAVZjJzE(^dyg0|jv%JN#H_@V^(5|p$;6k}jq{E8z;&Nyp;X;ZX~Tn6 zQ9)eOw^TTkxE=(X)E;FMGu$ufrn@`+#eu&R{4|#qFn{;A{9bqNwJMOj6#V5nN?K!u z79uNxP_2l7LaT4{7S2A?vMaS%6up&L3%V6q_yv3}%Z`>ck$YwcXlbc`@^xORdTMby z#eskF!%dUuTUh`bGzDJhZcx7p&_grKbo3=`S1(jn6*ANmNXZ|X!>-9|SaP>aevBa{2 z-2p5YHwCo|B1W}1#dLe@wjb2@$&F3FsBFi{7|2fOHXtZ3!G(9lJ-!sK4MwH~1fW{G4}$kvO>|WQ@#-w|2$BRfuO?hNJ`Hts3!I3)ehxyt-*2SJ&NGa8~ppI zpKEIFggCbl^s#`Db_EBrDBPrxc2}8Koc|zPMBBay?=IU#b)4E|(#2MBSvhpu!L;$f znQ8bpG(gqFOpDK0X3pj$&1dx77ETRkv&QtLZ_=aiM;r@M1-3UoMNO_ zF|*0^YWw^eCWnVt@_+GaNzb<5&GaH~xA8#-$aeSc?Kw1Z+&Seie|yhoZmh^Hz5-LB z;-orKi|sn0ufIqC$Yw`}Lol;NC#UavWjH*(cuD*GIXPO^x}JVUu+IZ^nn4}_9|^wV zIMJf${w6Kixf!{9f6r92Yue1cDGjjq%AKNICmm#+^p0C+0RKyk&F!V_$LSQO#$5~k zXe&5cxzWYB&BG{SOnzwULN@Jfb)ZA1uJmLyRQ*S6hz2@isDl*^QS;3|vbU**ZQTPK z!`SmX=wC`315LBGOr`6_%G`{lF{bx{+w8HwLp7F}y|76IjCh%4FYh(^LiNUya~zlZ z!*lRYJ9kG)G#{rJ%ovYdp#|WSn*P{zR?>o6hp8>3Dvij@}QmH+cB2 zrVj~IQr`FC%@(ECoyIqMnGe={E-l}E-8#J-Uk=O&HCYP~9JZF$iX34r=*Mx~#8HX% z(!avzk2R?7tfEF5n_m;d=>vVZ7UY=ftqJ)50?9XR?Xbq=hMvyRB)Qc__9KL`ALDQw zv}P2kvO>qDxMQ!@H$ywN$|@`7qs+zZucOaq@1IKlwI5eP)<5otQSY4`YHr7Y-3Q(2 z!=(;`=;lme3Ipj}bWqi|HSwE>-l|6{Nq-7BBNdI(YJHCyHJ6bH<_CHy8ukrU9+?C< zYs2{=75JEqdNrsxWRfSf4YQm3IxscPv^Xf;9pcDI`iO@MFzZVCtt~AqEH%`r`dlgf z6)O^U(+ND&JQ6F?6YMqCh-=Y3MPt|?tM`)1M((ELR*l^ z-+FKA-6K_6442}hy|y+S4`FuNReL5c69@Y*9jg!LR%5C)639~AvRSwGt|Vc#+-Zj~89%_)^6ro~%fk_y z^hvYil}JP^0kJzdYm`KBa7q*O{NNSdfq;fO6 zP*&*xL3*jC^QUh89xO^N>1sl+(gBow0nK%)pQ!G!Kz>>B}aQE~|-^vTo$iH>p zB)?|-!V(+d>7U-Zo+TZF9q+M1QpYawE!P7W{%zs0A+gfGf_9E6v*068EF2&o$`E3X zO28N>X>2p|KvVV^x%CTY)uHOceKr~$R1re*Y@ZabBCZ9YZ?FgbXUQ!g>I(c>_&7XR z4AOe!biL^O*$*=0DDF3=w;ui606%Cy2n5Zb0wW)+w!7x#iq`@OZVq#uMEM5J)@BF9d*_~hUOtP}!kynA_ zlviH$6a5VLJUwFzH#37orcSzU;$B8d*RY+%q1{Df$L3UvsOqN#f>z<^*F;_DdF*}o zenf(BY)ZT(UBkTMzWnky%n>S$Vz?1t=DRt? z))iDVpG-qlLj$j5Ll!pBv324_lyVywGHz!$pPILFm4DU0{nLPxX05Zj%aB9ZyO$`< zqA3*1PO9-YP78sRjE9(W0a*fe-f#}Hr6hdCzXZW-e+|je0MA@a@%X2q83_zuwNn9) zNGQ$NTxUbA{MQ^#qe%iMK7KH;jDV9)4|qT6)dgR)Zr_sgn_n=#92g)Fcx;tYvyc8= ztK&vK_gYxIl*`O`9mFk**6xL_>p!bGWa9;H{GZ2NWKJ)APAL11zRlSX?JL-Kw?%kX zg%3YQYnk9uA)~Qma>m=pE(%KQ1PpEW9K`<~Gtw~!!v~Qt%Iql8l3@XLKaW0%tV1&+ z%|^{nR$+Ub4KbW^xFg`?`CE5e+WoVAB+R9yG&AW+Ibx2hF`c6&HWsC>P| z-6PD>b`o1i=AFJvCSOiI>-VIrWPoob3)4mq`sSSUd>IgMTsj!=y+Qcv2BT%TaC3Z< zE5mu4D{XD-owC#RG8P;lD_Ngll(#lv+=jH%oN!8Nn-Es^u~ni-T>M5@ij!>QTzZ(U z1tGj8zDWUx^XCC>Cd%Zcco@bjCSReEn4UpLX~H2``^i`#q^r9IpV&6$J|^PF#$8Gj zv`L?goN!)x{_xMb#JQeXMTaSgasQoAf$rIErEA|=f8@Pfypl@5HdKIzmMkv~T>%t| zhgJg{UGfB_=^2@M5OeN$W0wO#j)$81o zh*|$ast%%BUH{F`g{LT}2`L@5AiIEh>&UjAma&?(<{lTNN0m#y&MMgc!3AL^Q31oR|#@pPpm6+c4;d1Ur#-y=#30xoSdOB?DR`utIb$LxZWp=zX~hX3AVAU z$AMYWXeN=tzGAoba_FtB=;t`7pUH?JrG;TrhI*hwC1hUwq^s4}XfWBs`^R?z1!hs@ zpqT{-v{&~C9E7 z^e%Sw0HOI$CryMOY@Qo|*%uqP`x(vDb|VD9!}{5^)}$Vz*#U@stVv&`O2A5tcJ*%W zpAw&S~WVrGekuxD8`Vd7YO}rF=68PvqKSTgauAGFz)LX6bEsgA@~A!_nXYLIGesCMA` zzdW0}boR(@QnqfNL>b8NB$5wQ1{rFj!-imr`d8C|%1f$xVYM0`@t3|x9E%lXU2{0$ z`m{dEaC#*oX}&7`lWc6zUCqkZ`4Z5t)DhhpqFm{dwyCDMug5)8YfDfi)JAHDRiW!C zopg8VXUj{;xpMJ?@o&sRr$TZ;`MT$YC2bMq{oig>^BGFWMsI^#$zv^2a)_4GNu2+! zi^o#{4DRJ!#@U~lN0h3=Rs4IK&VR7X+623=3aqSnwxl*S@M?TYWk~zvhbIY1n5gd3 ze`&K#VmC0_ZZ%tUjcRG>AN&=`k`YZt1`+h(XkCw%?!hd(qR~~+5Y@y{`gMQpF49z; z07nJ~f06p|G2=^6P>VjAXyIpe8<6o%E|dT1t|)d|!3k2&f?92F^Qi25Opl5HKOE9sX1v4<>SsSS_Cl#NS|YLyog zc(x%}n=^9^|3YH~T5k)~>CLr=jidlB4oc#BDn)<5(U$js&cK5oJ^;7j9s{+FmiOl~ zOjYmGUY%#e_R}5%Ia56qSE$>djBMWOhiU8k8cLn)vyFmHtZ1+2=rAv@hhUTkkq}Q9 zr#Sx>a`YP*Y7@#$(@lL~ec~Mc5Q%zXRr_;UGo-5KXIxreR*RbAJ$3YZhFgFZ4sQ$& zSL^4#vH+uJA}aj+bAfq|6AaeN4^>E1fs7ay!*)+5*NmP{&2Mp$+K_|uY*y$qivy|7 zHKdwL@jAEKH`p$yEV^2LZM0S@Rf}`xc_E`JTK+AfG~cEEik;I6i*JK+51)`H`8Ug{ zTGI%((S4348(ES2Q#V2}O`!loSzjw=*#T5Dc=dRu`1HrWse%Za@jbDaRTQCW32!!z z+;1d~N=2qt9Eqj6v|I0&spNf)GHYFj2oiFYVO#(LFZS@~a*uLKD zCK@ZpE>zdP{lOmYHSn4u2VUr{vF-r(8b41)Xr$V}>VSn&Tl@=pPdq?uWVE}(9(1{_ z$)XqM^@OA~l^>WRfgq3vbsg3{lOL!tCWDN6SJ?RP$N?*slS+3+uqO+1%Fq%Ip~9NB zz42!QOZ2`w@I=@k2jMH?PXatsx>}8CkL>&o zpAJ3$o=PUhgbXUG173jT+wZN4J&I#}Dy{yO&Yoxfjg<8o@GsZJLjKw|x3``hH7|!# zt7W&xQER%^%cry6OCVvr`Ze0R=d$j7h(JG)rX!Nmc9LMBAZ;z;ZrcwQZ+uVgVFK=| zdu6N3eZ?JOd0W`zAjm!-A5YzzOj0h~@B(E!L9yFHcN=CEE*Ma)ffD6&DE!ji%$gLz z_!1n{{3}%|XY8l^!LvvlbIPyY^}^W zgo%`=xLy}K9!Jlk#xc>kpjK4#?;6Bue(;vV@ejPO_(8qfy|L@gIu0cZ$Yzfg?Yq+$~7+|7cP?8NnBQrs#iqA_&zA zzol(RX+C3gS~jX*pwnztFX0yq-)Gkttn&0Ac2cg6+BC)=F`mc|yv^=48V)PKXv>qI z63opB6chju0t+NyoL~nWgo`_c7@|m4F23z%?4;)o-a)Z17v6`kYt!y5&Ue_e{%*CP zx1gVoFPC0i?DcyB4Jji?3L4x!su&iQO6obVXEs+iw05iL8rBdLnB2HgV<}g4UUp;Y zj`4w&znngxIjo3`XWOQMIhdAOmpf}7!OY}Y?5{w_^WDfv*~DruS;qridc$e+M+>u= z^l?@ydiK-+shW23f@Lvr&Ge|bN#MI2f$@nrHM35^Y_1@t=A`>|BB}>Ytus)n=LOpH zA=Hi0vc{sW6pO3ug%c-HHq`!yhwXlhhc-2 zD$u_-QKmz^Qi1SFcArA6Ua7TAd~w^{q46s2?DQ0M#JunM7iMG=)<@p<9M^EqH+Hu& z>iX%Z6t=S_?AxyW2u1(`@$o%I;=H2^Sm37X-l)Rm7<)nEcr=4Llb*BF%{%-4zW9Tt zcB-t6eWCe1MB38LfqR2)dRXVQqu;xe8#q+GaO2;9gGMy!p7dCYFdvi=Oj6&JQKHDt z6mDcVSf|1UUnrtc*>1b%=uiiyKORI-&*(lnyFsGCA=FFbKPN7?l97tFQm%Ys7*|)6 z_w(^xGi+SiQVGsD{zbaIkaZr$ye{B|{TqJ9KXCto85e-lv z`HWN=lUkaNQVM@5em--XkSPEj23lmIvS4xc4R_y$7;g(>{vP)Rhc+S=Ov}p=&OFFz zJx>fWJuwX=q0S*RCVg&U=nSB7cC?>{&Zl2F^PqX4k;N6=mPerl*_SFS(%M$yo-c`) zrZza;D@`J)b5vVScw$xCzN*qT6IT&APM@-uukZ-g1%_wI1WayvUiVcx0cPV~IV_hg zjIV^9K1;O*5RKHT+J>FQy{Rs@B)4{W8L5u4gCb&b+JLFty@0zQHQ|cI*4}MUwmYYT z()-%sj94+K&3yE7f~LV~S!Jix8RSB#;#=Cs!MEaLtFOHCnNKFwHXSD=V@c^M$#Lpk z;I8`Jek=V-ZMxT;5Ni(}5QKs{R+lLzW8+|}(zo#~Afb14`3l-msqAjcE(7%C4KuvU zPd_gjfa<1@&r-CE1X$`>ko(Y)!m-W-^5;U1jT8%Dwd0g}o zcH11M{>Q0f#;6#beMNRY$6y9OH1B@Q?VD3_TURY4b-vm?8b{xP1*SD(jT;M0OacES zF9SI^oO;;8hQ5I7Z{t;X(}pmiLWnJKwz7?dV>mLx+G(?_!Ol;*LR%FJZdIqTWBNV@ z7^v8asT>?=qUiC!KcuX?j1ITqZYi@)m-RhvZW}9EK8IryWV;s2=>d$>#f;xqK4cXH z?rd&<|DAb)_7CsdA5L`hqPi)pZWR`wW4RX0dZ1!pZx|KZ742|$Q|QJO=o)0))cr|+ zd{xFD$&hMp3xKTxZ>add;zbYZc(v#0pqYV#1}0=9?qni>tn`$fblmhf^?W*bg2O=) zoPd-;45VmS8KiLpiQE9QiUxt*G_pZ-SA?^jbc=ThadOla6sZEnfr~40W3i>R3Xb#N z(I|}D5eO!U@`H3v)YBq@Neq{HV#^~`RJc>Up&0rWS`rpN3*_R(sq~?U(BjcWSPIk0 zjKjC%;kl-OFxu3eFiEG1OpjmU)~=*l6`q3Sty<+D$EGU}^L5X9O}CX?JV`4b*wYy3 zs7X7MN_-1ZN<+y1i>-SM4kX$dMm@1@8xz}@m=oK!HF0NRYhq_&+qP|IV%xkq=l#C- z)~&i#ySlo%`d4>V@4a@fwH`>Tw8N#)l&~qjYhfib5IDSCPeU{)x*$@OL0GXL`iZ^` zW$#P8UBkY8pqE4@c5YuKkc+29m6pCSF)k1IUVS@5%ga{*Q9!w$)cD9~9g$hrQoQ_m z+Q@~*jQIgsJ=(E^lSXMfjNu}~AEIhz`FiDgXEwAv7c!tI&(EQAJ^E zN;QR4b5|)j`QAx?K7y;Uc-V(mFuK@VIQ$n@Ck>Nc&{V@Bh6~f!CtKO4#I@GDa`X>V z9z}%~Obs@X^)GR}Io6AWm~PL>kPr(+P{gZG_+zm;7A!bO7FX3o00V`!Hg#YVjt@Ib zV9ZU31mD~(G=FYLn-#ysddR9Cx5rgtR^nvDtP-V2vh=uEF`JeJWTI^Kt7WZgtl8~A zPnqXe{qiivSW*q!%`T{mL|Ln<>O)QidVL)=K3aa%7EtlHbs3}kYb7@9I!kxz+-uvf z-frTJJMn$!`J(PI+xsK!4=eH(EeQF`fU4g{4VrLhq(QS1ek<}xnP_GFi&UFVZ82y(lLT0ESS7-4fwPKXF_}7U4PQ?Vh4-4v6sz}HEK;T3n{iTLnVlC zan#wjDgb&6nztJ>&y9n{J*c}L0=kL=HOA;L5|sO|w|7YBHay(SYD3wX-U>1g?#`_+ zo-jhLjwIr>pT>Vj&hH;#gi_NEY7)mPj;MMeF+!M#(f{DyV__m(lO_y6_kS`{qoG=l z4fHzwgGPWu#GV;NK`k1Hlv zmCk^+dZO;DZLhtGkImT183YI9xc_G2&qRwL-cZFV!yKAY8+CdL*x|=9ygf*dn75kp zi@2Ck{Js1)o+Oi0ildj9A=-RYrr*>qA#lyV*&M3=`6v88=fgXjf7)3(R zp=3@t92XDc9`2wBxY(^a!Ax-IP-o}h7}Y1>av-Hn{t6!RNh=HQp2tX{+q(!s_3VpzCXg{d}K3PeT-lY=y%n9w*#3r0hr<9a7@r6S1 zms3O`RNUfUTn^Gm;U8Rge;6pb#VD{6byRLp2NlTXdmT)^>paL}k>IlBf{Nm!fd~1? zLqq=?oKaMR<>l4|=<%7r_TQ_G=|zR``dV|S%soFGZ^~i{Vt^dc=_{FBahF1Mnim)n zS+bp@9I?7Z4Qp~ocEq4o$Tf}Mwk`d(D9KHzVgqkH?`(>LcGlVIwsi;l3){24&FocL z1wTZU$hBdkSVb)QbH76!*K+~+2M;#%<&=#pxPSNhDzq9(ss8Svk6?yuY6?1viaZO% zrm`dV&Vc^Lbp(3!hML5SZ?~HI?+?m5Sv^{* zEbDV+*T?(EBj@0Io5v&9#@)vwvmr4a(o(NeajH_U(*al|R#pMlU3OLmRwH9})?qnd z65h3$MrKdshGb2@FXYdt(7t)KWbO;gnBI54lg@e4KlR^NVCpxDf%4^UFBPLweu~F5sVJz~t;Ro^^zjOVZu$ zrb@o?)KE>j1rK00Oh5PLSXaDry7`zqnEt}hxxIOrSW`lqN_#vVyq*iDJEch8WVV@o z-u=bjkk=Bq_3K~0-Y>p+Z>+6Wifzc_;#mwIs91D`;_KsF#@VFs(2E3@$9p2@Fi$rF-6zwN2q$ey+K_@sWo%j%RHS zHtzh=q60oACT|%%(Z(GwWn|Z!UH*OS_i&HePpg=W+7IFX1q!4)N_i*0fC3?=`X0{g z{x)KGvcS1;8Z`ljW2irz(bH#97|xIk-YN_orzO6(!EPO`9whpXGr)SCy;sAt=1tY> zUqCmQ_R^nvt%D4k*}pFvZn5x|E%i7dA{o!$OcDWIg>t&HIBfF8wg@N2p|vYfm14oVE0i!!|+$zn8h=zV)!b zTyV!4Yt*I*=id9?>9XM}8eu!J_}CH$T148%uhR&OZ$UwoFDHXBwGwW2x-CYhji>5% z`@U*!x27vy9G9ObT1itnRh>5{*?R8#rWt3WXLK#dCB0D3XOrU;auoq?g&Z1xtp6ZL z>BZU{%I>=Y2yPRnnVgI*9G2%87Z`$6-g51eo>|UD?nd5?uF=j(9GrYqom9_e+X2H+ zHJrCd5QzIg#yEw@ckYUU097Z(iZ1VH4k6yNCmO~w?smKH36im$uoUZv4iZ28L({-f zDEz63>BJ-Sn^-5ZYUCRk8^)xBFh?RCwO;2U148=%LgrDy6pphce}S`u%fU0zHPw2_ z2bq*Eyi2pd#k-s|LIYZmAH~BhG@=(0W7dluOQgA!=;o zA^K6EU8xL1|5>4G&8*VX=b?Z|m;GN-^ri%Z_T5unGTnN7X2F}?{cxPM(=Wjke#hNd zy$P!PkcUg3{cT#tE|Jatdur5&f305s`o^c_d<#eDDAY%N_7rd6pHVd%JSsTKCiWj! zb%-@nMd+}XZTGD!#1{wM{_EnktY%; zGY_AT9UVtcO%-Mp8Omgp3Cz|(r#>;|9a?O1x0$*7JJ6VJI~GB7Ex7sNF=7fgMEY+s zV6BTlZO0q_P)pq`Mz&tA{%5_si>!;B%S7X3<3!_>+qm1L8+#H*lEH zm9s2urkbzRN&HU1K-1c#>!zBo`YSKMBWMDQKO3xw?5|1dQzIm%Cz-{iN~NY6u^P8m zS#|nbbtYJK7D^Z1h2ZG>-h*3fhRrby#TN*z%rG;Gdx?w9JdpAcRzxAj$b}1F@FT18 z&-fn=%*6b461q#Gj`=|RP8@niOi40%kVYsU`$zR+OYVd*{WDCnhe5GN4dd7Pzfr%<9 zt2kO5DD4)C$pl3A5hR0}>CA8u#nJux#=0R*kJO3E!hJ4u3lEd${1Jjh#$J_5Nk;8P zc&#wS@Vxo#Qm*qMCDQ^^0uNQwN5a-^(Ss%V;u(_fjt@W1;uh@g*Busi#8fGoF5AqX zR`_Y{2wd+kDc$ih;LPcPECN{9aK#R#_Zi+;90OPN@ z3ac>{i+rLIzNx$!x>=Gq4pY(CBssKYWBpJxkFt|{Jt?(O<=i}wqN{E!msF?my7Ka~ zalDSMp}b+(Z*U4yCcrdi-!+0JdKKH;E*i!Nqf3^=(PoC;lgv1y_gQF~FjY0*>btv% zhI;Fa#ma!WNTBN$OJ0M+V4~1~TYo4($fHK)7Mo@*S>n;=I;Zy+I%TGwy!?_>@oIHV z^j>>7|8Vj67<_H0D-`T5sx(Z3tK+P&?DVmC#Ws53NmwA7 z+d?v4IuFFQtpV>#(orLN(1i!euJyh@rljS`**3vsr>n&M9 zid;5YT-Kf0@LyM;Phg(In4PNzb(KTJAu48;Of!W50j3>bt07qZn|ds0C5s6TDPpEK zDQ*sx7kBs82+s(iMn^W9dq9cjYh;Qqw~NKiUJm?eT(Tuv`!7#GA3h(NvL=?1A&&ks zN7;^<4lTbHEG8czi_3gO)JD~+gh9n)RT@P^Ps}>>e|!QWWl=0Kx4|CM`@!G%3E|;W{8C*ymS&Vj4ex%z>HrjQzwIzFb9AN0=Cn6q&bGP6<#2IT8uZq9V9V)> z!{jtiU*>)v_UIsHT#xme*6MKMm%r*3(>cF*%;;|h1qe8wl5Cv=5;E`go4MRPoE}&m zn?&3$5LjBZYHiiRRiPS9jKM@BQr`k);3eRF73wvPW+XVLDN0bcfQkA>ZB446rG0_p z&WpCfUSRFF~;&hr*>$vIW-j}7?O5_y_Fz5Ow(8zzHXBGrn6&b7xfKiiFcY>VU0bQ*b3(lI-i!l^kCx0HDYO{Y!B>f0$ z`SA5Vo@kPU(agci!HdS;aIU9MT}>+m?g>WjbfWsD?h3Q86hWHnpj3!*_F@-3jmgGa zuEQhYhh5}5hu6PLq(pSlSzs%*Lz2N(&$nkFk8!u`&b<}t`*~OFEZgJ};8G@JsU>5= zpJ$&OS}Pt()F1y|XdHUG*KwE<7=q?&YsMY&E#r_6#$=1m{0wl&|c{KL(?FHges zv24|AR_M!<0A>)^d<2ZYQkmthJor(tM)4g#wtjNQz^+5M@v0Aas0y2CZMlV-n-!Ou z)U4Xbs@jIr<-%SQ%e18o?aty;>o$*(aI?3a{3C2@M0!|hRs(JP=J#`_(ud)I#!(ui zNV#^+Cca;_UMWMa8tIXFi+G8BlC;e{+^UVzcqq|_Coftdxio$i=zld?mpaDPW5bW* zA8vE6x|Z>%S#DljH9TKNag4sN+-RGvIA5qr(OB2|8PzZd z{fo1e%wqwuw~4H3RTI533Awnvp;`;gJUC#XytsNbNlGPMMTv@(hLq)}Sbn0N;omeu z^q8d4lo0x<&D$j?fR}PL*~W{?i?;1L;L+2xYkQyW_1$l>qb+loCtct8aNhJZEFQLw zte`3`-pCtq)dIz*Pw_2OL`UR5kVN%mjVu-aI#=mVW$+(YBbP?X zK|Zf;GVXlQ`CVm#Qo-F@asnLlS>jQPBA2rO4HJN_RFM?$CO6C&X!1ZE7gVtt`1oVuXm|+uDh$f+ePBgF@^MYL0@m86Rd zC7Iuce~cf9r&?w4*$BB+UBUxj>)Um-(#u8?lV#g}Ony28p>xk5tg7obl)&n_C<1dv zOeFjUL*K(o;YQ{DH*Cr7k(hzyH>Cj2Zt=aTw|BgKQ_O3c#w7uv8uon zCSmylCPWzmO97hhJ!NJis)r?z>gxp8RU}IIUJ|E=8dNy)hs8iY^-ycmgSc zBg(g#>~1>m}3FBa#=*$x*}r0o=E%c_ei!gZUfKoH#( ze}eLYd;G(D^4`4*^A*GRnP+^+fsm$uHl_8VV>C6)K5;S&|84;*y9?fsU*OhR2x-pXt0n+t&hExgNQ4%7aa&_6#hrBBzyqi96lmdEl-hwl z6NEMVRyn2=2xlq4HY$- z90^z`AzYk7buYZ-Sf?eoo5ei0IqIQ71nscFp?Qs%uPPvave&(z8s-9MgDv8gv;562{OfvR~#9-7P=#4V^b zj;pF|om0Z2U~6vtvB!F;OUi0|le$Fyr>qz<0i<4^(7ERuAw6;;lkgk&3pj{nnbF-h z+_+E0Rm^3=cftbjip%$+`%koOd)k=1ac{oI0w<>AzQ>j*_udIz$ZwZOs032=HclWYC6{c^1+z zgy}{1r5a_!kLW^JLYcm0&RhzQmdk5ttT!p7>}dQcb%4jF>Q@r}_Lv3|0?!qFMKC*K zkUv6w3>iNqESM81u8})ueG!8B;tfMMcqjfxU~T$1gX8WPu#ri&RnFB`=uGzxzisaE z2I6DLcO}K+Gsw7&8|~8@PrB0V#d#>X+leT#y>(5xxw(eAaN1*>u$R1a7>XXI_)9qs zIHWq)DH~`HJ|lDX2nQbZ=Km@W9o7a5*qWaxbr9kg5~{ZL=!j(TlkZJ5U0LT!423_h z%oiDiIe{(#1g1GDmyiKaDA`<>+&3ou&pa9dmney9q=H8Ly`sHg8gPEl#}Kn&vjm~$ zvtpb*uCw~ndO4jN|6wHPBl}vq$WQ*mNVtx2eK8WSt)s>Yix(%ZE*@(2fP7Ahc zArGh^l$JvY`H1yxdd>MF-v_64s`X`$PIVBZt`W|&huB?Y;iW=^9Gr@gJo;Wkc#Pf!s%uxKB!MddB6TGL^1V&58*?kWZh zw{M}`v6=q6GWy-($M_mSF5Zpzi2Z%{(;!l5k%QiW3_PJX?2zR%Li3iMqp%bRm4%+0 z2%Fs~_}{o^b9whonPdpzbPE`U+w_izKLHqzd#|Ucz28zdV^?@BFpCPB4Cf^iIU>&Zq$U8?1G6RCw zS$8^T61IC=(_7Tz9We@ji&o;^DBbZ5!2qNeNs)iBjo+K#u2;Jd;B9o1%9f1+qn$AL zqN?F<&zax(dzC$LY@2AF;Ck<$&JLM2#C<_PE)h1q547Gx{~~KQ+O$Sso0MN@f~3e= zbJgMQt-M4H^!&Y5MN&M6+`O{Z z^2kguwh8&O72OgUWQ?HtjPu&E5^Zsd;bg6xXciWEa&57RzDajUpXSzNhBj-k5;Jz z@tx=g&hH<#VgMFAG4UqFcGQ<6QNLd0?+adWwVHWFQ!#g$erjAPvv%L}Y`V8(fLvIR zqGrqkJLf!J&py?-zt-qlNC!NZaM-%%-A?h}e#-l`-?H>ZK)-%$5k`Q_()b7GbjG+z z(yzur%?NNKRf1WFYdY1U^^#GCh#S8kF6KT=Hi|eHZbrCb8F&(Phg~vteLF=uiQRR= z=fHL|ML%`J_Nf@W6+eELe?!?0F@~1^IA znI;l{=w%-2s)?dhfG7Lj=mAF%P&4U9)|Dy~-aqf*$Koig3jN-LvDu6VrbkhWwTT1) z_t3b)>yn2C>6adw7wifAg!z}3)1026-^T0$Yk2k$b1rUjD%YZ%9(iRSjk%HLE@d;# zMi+@jmi4DCKbQ7DRU|%;T&&c=C>wrfmvQD6fdoLWp6q<01HMwfK4V;;r;d&mJzoHA zj{A0)#=0EZ-xPn=#HJS+-s1?a;P&-vzbFYO?Jr7#p{Yh@+)4Xs=Do4re3g|swBZL{ z=d1j0Z#a+F?`vBbBH99W|2Xp^xlWdF6%Z5f{*NSac!R7Uv2WMcYF$p1TXrvX6uy5B zUCdc00Wf8XKQeV>Z-`!>>#!zO4b)OzV%ZMiF`XwLwa{uTMo4TAcYxsjVYeSXVc0L`%rZSPY*)A4b;OPknx3?A|>%tM}U=L?Ylt|7P#{j zy2+kXZ%Jf@U+6NT;69Z8|=9e$U%io%96 zWI-VD0p2={mdhwyQGTf$mHQOoqH(rRp5NTY2A{?L(1d%IZrE`yreU86`CcK@?>mnqm&kTB#Rmv11DkC4@#TRKH=dI%5-wPZcK(c6!T#>pXO!zSUa_Bbt z@}Xnzfzo)0oY*BaZ5(3+L>u)%?>@CH`|{r$i4D_A2Hcz*A=$TpEOR`D9{tMgv@eVn zE*|ih==M#wyc`B^{y^VD95FU%_=ciB#@`Lrl9|G=T^0JZOHzY{ow(F(fim}1e@lfO zHzu~;szVs%cN*gWkrXk|hF8+93i;|WU6uDMe_2Q3Tf2GUB-58Ap*=P?y%jIy>G}gO z#Y30D%6qScGSvlLy1>^_N7A8sN4VL~Vp6^7WdmIle8hc)Rj!|hc}mv!WGHM3|J*PG zJ%>sEA4>w-9mf^|+?X->OMC*^8i$SY>E$8IE;_8K#C3ya%C5m7w1ik@c;pO3dt2gi zvridJcZfvb!oruBjs0Bx_Q1X6=VY7#a>KP(GnXr`-QAhsW(B7qZz-kCbYg4^a{B#z z1GBpGQGR^)aP@ti&$qI(p5DOc__K%4_oAhw(eJ(lVD9Sqx)BQagt~p==EG`&C};e@ zy2W0^toGGd^T_cQ7UXC{@|mMzdauj9?CN%bE|m0H=}3|IB<1X*{#>#6g$4T3)#8Ob ztv-3r@y%2(+Iu@O<%0m^{bYSuIo1beF3=aT51;B6uP{Wfh7a9Ix{o-KwjZ;ZTl!|# zvy|-Tav{F(lL`?aANjNP7gpi-K-a<73YJ0J>BFp!ca>KgVL)Q0>L2 zC(v%L(3Z*fbP-MU5nTSGNvuP4jMx$5RLWwV+p84j+5w>T@`K^;?c6s*Im3)URJh^) z>_y~M)q|byfKb~z!q+ecOi{M{404Vwtc z^uT$IFDv=JJc*%pnA^rF6R6{2AAXzZG-s%ocG3p>flM~`UY!R z-qL|WCL?fqW~f^qCW`l!n-B5T(amXNSg&aT1mU!YYf;&JTdQ%YuXG3@EtWIQF*~jy zQo|SMaRv2mYE$Z+0i--S;9IuoH_>5%I*mYG&SaPR8~)Y5c{%7yg)u#s$z;fk2mq7& z8)`WK^$#YKD6Fek?}~Dfy^8h3oQzI9rC_VbtA)a|^BM2`d1uq8!-w_8F#2KJy-Z_5 zf(KfVd}URReNw67Nzv2lGV7@5ld;Hc*CLici-sc!=gIwPf)E% z2i*uZ2cAfZoLg`~h%Ze7bS$VS!0_;suDHoS5Zqcwd-$CbWYQxQv1JPzxJm>;LY9R0 zu>^l!Y>a)i2NCq?tgP}RijFWi&k5M4Aovd`@y+oY=AJ>Bd_K)_F97oeN@#R4Z;PWj zaf6~=QWE}b-){2>zvQnoojiE@0wtPGpSpCDKg%9J)Sc}S266aV3+lX1lSX$Sx3?oqL?XArwH5W7`dMFA+OaHNKFH|0mWVhW5)_D59P4Rzf%R{}m-f55ag| z^T6ic(S7OWTwm%Ej@?lY-CWjAnL0zpxQu!GVtoc0QL1OSsI-67LfkW4dze3Sr#QAw zB;ysl&b2@($kQxumg#`LKhw5ACDJNV;!nIQYWg>R0|lZR*8P&2?N6P@8>xUvywPs&}0IL`X$UGns4_2=EJM1qr;k<1y0zZi7M0r?bd zhCbxQG`rrP7E1g5adm2O30~tX&^wS4V{jr=EYghikL8H;M4mYF&F>tA9A~V`s5HmE z%3$O1D@A;|j>z|5EsZV1fY4GY{rPwh~SOq&3I81oC?7{G2%T>yP_o z`=*O7Wu4!v5ng6^Yb6|`C`b0$An&Szy$CJ3qbn`CXK=-FRDM`X^Fazh*6(TmH6^}* z)Ff)h>?eDyIfEPsw8YWK_tNdl9_6s*`CJLi_6#ZH;fHKdHz-s_|KNWBN2bsG4gBcb zfaj5?n?xlV?=Tze{HZV`GW`Gw-^~ur(`8ur=J?`F_|$0}D1d5yjH78EbBdYDC0KEr zR_#ph^N(k5_X8R%%6W_R?wz?_DDKhZJ;SQyjcM=L4gXm|vtKRx$q;=sIBA)-z@;$C zVyqgX#`*&01M<0{g|J?|M`GH89+-LL+hCkfY=2pTMz3pcTKJgD1pm~(w7=$fRS;~_ zzmqX%^*t^hj2Zp>mnFSchu7tjw2L)n`=H}%*#&QzXtVcGtnOBp?hIA)#@1UwtxrT6 zOOlLst+=7rCgenHukSB#L`=W|!FpT3=9{;w%P@3<SKRsE#9>Ttl0r z7#@0t!wH)PyBk-!c1uDxdwQrWBNbzGFjpQw$)^-PPi|=B!;fzwx(ddKKS&yx|6%tg z*Q<+s0QFZT$9(<9!kq6pV^Sn=-x0$+{s)ynMw-1e@Ey@(u^Ag$%b!X)X8b}WvWKDs z1nun4HgXuQ;WFPy>+|Y5=)X`2PDhRx<$qQ1U%Tl68MU_)?rj=WI%Gcf*$ww_AK4F+ z930`hk-JmlxLYgHEp>2>J%?*`Qv%)QOaOPrvVqZ7s=}?VJ)r%Ez_6j^L9VH9t5@~41S!r7Va>Qb*(U60e?3KFT zNz#h11w>{`U56fS2VJanb10H{QTS*6Ug_ z_zf>P)-lvg+xV5F7g%f5gNI0#h#r)Bw|#fWcG|nZg5U?SFE9Cv>4&d{hBvVNrv`#^ zS|5vgs3k1PBxY@iHcdz$^D+XiAGm}CziCo-ZuehG*mc67 znjV^D+h^Blpd5AUIiZ)ny1_vbQH*#h_`>=(*fG5U+xn*%7dnn;9K9&sB-9FD*3bIu zpUJssAauYr0FEYK|97oEt5Y6vXRlD}T3^3za_>D56PYMa&enrP;jcb-QAG|;cLRo6 zwJgzGlBKCy9-*A&hsnR4SlU{p+uV9BOP}i3ZcQeb+G6!v(hUSJaO`cm6maz)94O_( z_rHA%u*XC3D`amq1U3jX3+a1WWv0cG)nx045!irPLu(=4BM1=#Y!jJG$rY$*KdGy@ zaTx1?#arv?2cuJ*<9M%!;JqD0ubRUMC+6qp=5xWem4_KBn-w1I2Zzke_R@ieF)JH= z4GfmRsR^Xa7<5aq~m6n zA3eZW%~yyzer(P=30CTm=mzNw|3k>hP{@Moh8&2Q^q0!Heq`zz+bSUQ7Qf@RVa100 zv(1tMvEmJR%bQ|#w=JZQs}WjKHjwkKxF={RUZ{-W4?A3AZ>< zMbqdcK7?87;lDMbZPk8)bF`&aBS0RyY1b% zZPaG*G9hf2Z&Y1IxoPNKHV|5Ok|>|-pAZ;%LtEQj2_L9k#P3y4r?dKjkG=A@a6poJ zktE>_%xUw#jjM)X{ca$woNj(D4pbnlbMEWauv$QM7(kocO6eJ%&%!AVga71(k&f6* z{s_7I`={6I&cOf-GDkk8yLAY;2mUn}j1QmdS(gqLjq93^)DTqvlkz`D?r_PJ?#+PE zK$x@D0LhdezZk!+o@a&{D7^teMCuH4r)$|}ouLs#!i)s}Gyc-~N{+iw$<&{FskyRy zo~dv(|JC9gOm&RL(N4K0Wra*}Ou>;zt;xb!OGPmW4#Ckg3ruD1!JWo{g!??|9BBB5 zY9{-YVrfR6f?^c> zG84iz5z+k}j3jmHUjZ?ZgM|-LN36|Zj_I(PY`LSkQoSmhw>E&9-K+DvB_laxY@P7e zbcZ6|4O4P*gV!9OrzbBaCaR*bO($9NHWPFE`Qa$AFY5FJBOq$Jj51~V7VBChD_?V| z@f(J}HpM5sT@%~Yh8{u#!yAlZ?}lv85msM<@Po#C_{OHsih))kFQzWYs6z2Q`cVGm zH49zh32Q3w!%LqgLi6P^_U5G6cfnbwOC#NhbYF*fV0#eoQQ|qUoYqfIn#~*LGCd9n z!dvJ%u$;?&7>)$tHTD9PF--2Hn69uF^-z@WB9zM+SL|4+)77^xBSblkJSm$A%pp+A ztD)tqRq9g5d}?_?Un8h=%y4nxUL}a(5st={Xc769z%5M>kr5=<e#!#(dY2<~z@_&aBR|O7#P9eFP?}Ke%~|JQ4r8!}65@ZJqoq zt+@^#My!h&le)Efa(aww+_WN5F~jU8)9k*Kve<9eWQTIBrOZYcQ?SfYLPpdELGWe1 zl{oM8)4A+9d)jV%*H#a@&n-hoY|X$gjc+Z@z%Lmy66TCAwuy>XQIq~z6N9Sshx%co zEVUBw|0<=k3WrZiN;@VMu^6ZPe(g?t?U^4)PZ5*gzO3|p#Z6r)Jq|xR89$lb@~m>u z*!sr&fbE%e8eH(4Z5kE6*2LJ`T>jLrt6^%3=iAG;IpPHabChqky@|Pw$&)Po^ir2| z(-kk|cr!q3b#W)*Ll(u4G<<=w67_KeeP0i_xkcRtLj6@UZmwDABEyw$be&(=QeV1$ zZav{4RL}^aE|;5MQ2X>d^Nib|JSoL;U5IF1?X|6Ch&OJ}v+kFozeeD1l)q`$iCpTO zy>XX`^8)_hw@2+$gvbmzHr%l6L+l=id|CwB+t_kjKFv&EBrD}wQ;Fmsn+W0pi`Eo&MaKbY9i zB}0T#dhMr7Jw2GcxzSQSzIEH7A+JOWOXFZfw>%Zol9LmEx_tfoF*tU(iR(=|(yPZI z8P;s(X`G24;y-C;V=ez{iL0`p|CmGvWeAc|BbMS|beK(>4}8hZDE$zdFE)(DhdJf? zAx(pVd5Plo&r$idcMQVcA1_puw?S?He?5@7{hLW_fIn`WisWwpyl-g9M9xuWbIslJ zbtr5#U_&^ZYT8kmGGntd`@>0Xl)PY_;**~ig88$Dir~fzOOPMO?@i|Q?eTf;k1K=V zN^Doh@MX?z_K1D&7009h%^*f9aQ@4qNKYn$r?=pqo~&)z%Z~~t`epaCgHhuQtU^C= zKRJ&JpWl}|YQo{VEm^3vjX~&YU zo<)AsPUf+7j^lt0gHLpUn6J$gR)*nTqR`3|4`=csZ-mxJc)UH*UGm`-Ft$b{IDSyr z3%U8Hg}ZyQ0L;$YXHl%3C!*rVehU|UM~e2!IFIFX5QkjZ!u-Ol3P!27MgrP9!ov-p zASXdQPfV@pd}DDt5mg3Oj^aS+&EH~1`|>ga4!=M5ltjh>ojtAVEziLpx80o0nf4Z_ z*@44rSv_Vz{Cf|7mQc}O0G42ywfF=_#ag{0V&2?xr69(4D7XP-5W#wFdT&~aKG>O< zJ2t^ce7~${LXA#;nNOOeWtUGyFixxDY+Wn!oG<=HQr6kdhS#t3&i$&KKD-bnS%8t# zM%8c)E;I1BW$jq36|(Wx++#8S59pRWrPk4qdUoYa7N?cG%CyjM>aKAqEsv78R^$I2Zk+!X9w-;cXQgyja9v6Jwe%{-(% z!lQ|kHLl1PzN!5epb+8qhx#{SBnJd#+WnyFW8dY{VXQFF+6%Ds_5&)Mf4wKMmf@i# z?rh3^$PGR*rPV{is+1NyAhwW8HmxzIB~%j{y%Rh! zm`|jLp&V@j4|E`tF5L`yK23|fg6cyRjV>g_gSNJUo8a*$fc`e7D__=#YBKq2WLanJ zcUlVnt;kbk^BW<0o=Jh?j*f$XaDrBFj8vHYJa&W6AYo|son~V%7h)FlP`9`GnD?Lj z=VgTEA=M#2$t%s8qSFR@FiERVf+>d4bNffhljpJ=h6}A=IC+-olhO>Q>!&I{8~T!k zmVEhSzda#~16rO6rMfBut&AOc$z}agHI2W`=c;ZW2k9R$O^Z#@~v= zi@Dcxpm*)R_tv+As%{K9U>AKyYNN}&P)_1E#iDjZxmmSool89&G3;$Jj*d4O6i@0H z7msw*zO&;hg(MNbH~aNkdGLyPRqr)$5|XCEC-4-74*WWH$Q^uTrOc?nSP7A399d#7 zVQH}SF_0~QvcO0$$K8xP2pIEGhvA(hNuQKXUA2{4F}<>Ywst#5KvJxyO!yhXhgdAt zyA)mPmRp=hmmA3-#511e+RPkWtyI}fX0n<9b-27Z zL;0aQ74S!oQ*Pba4$*oy<}@G1f|WFkw{`#It0>& zDB9N2ta{JJab1?Ljx53>Of6%Kftl)ZD6g^MOTccm%_goZdcG?O8usQ_!0IME4D;=F z73{}*m*i(VJ{!7ZjtIfn8`aOL-&=Aw+f>aIPDRCK@g9}N`L zB|sN3#2X|BQ1jiZ5mKJ;%#7pxN#!Xsq)Ear#9ki|u)4o`-DUUujh}I?Eq&2}C^wWh zT)rF470$x5J6Gp-8E=KKzQb~7)`@ytM4loFlDB&FL*au3br~Y#7@TDsdO3=$B@2n5 z(#bTc)h3F-|7T{Wsl>L!zkBVI1RLwVaR6SN+vai){CkLlArB8!IZreL$Xhs1Vvd-) zfp=5u`M7G;jT_`;U+Jo*Zi&+KPYAMWG6hUI44BCrJ=F=fTk3|$4Z6!B#u+QoQw?slkfyI$>;RKI z`#BU1MKJci3Ek@zONwuOS&aNW=S5KzIM3vBumYgprB3!)UdW)K2DgM#r0j|pLk2Lbr}Z4tg7WX6+)p*ItPxY#I7f6<+;Eb9Z4mrcV;ZJ=PYeJ08Ht!&!h+UT^MEN&S zSsSt@jj0bRS+;%hQS0|qdW*4S2Ts#M(4~{<%u{}5&v{{7mfETt=%tn`^#HS1S@_ax zeS4n_t2GU+P%8~_cHFU)j8Up4*aQ%#k}VUh)iOcF9H5U^@tQwiV44mVOiHpwK-B-O z^ek{VSA-HI7F)`AFg)bvI?Cs*lU#+p!bjjc6_EXLKE>|)A=8>Gi6EkAXViNX*|t`| zMmS?G4{oSV$!NzMrjVB1>j5~e)UY}~sUF}pG!+*t&`>8+Sig!yQI51Bt2p{XJ7tB+ zV7-NlSbGA=cEkqq(wX^$*}Sny<}k`uE38 zyEY|5F-(FukMT;p5UyYN*E5FPn)Lx=++x`)Y-6RAlDy*E2Sefa%ud!e?XpJDs#}?k zz+yB)I`WJ-wJzKHK$Kn^HRuQhy=(`o7-%4{pfByfRphN2fbKdR3KTs!3wMEcZw9F@C3H@D9YvV{7MqagfA7sU;_I4ok=J^Zcm%GeBZwgo$s$* z>MbzV25=-_%Zx&T_H3W1kTYxKnUveb-!lg%PCl)9#pL#4DZe)JQ#v;NO_2`uKT?B_->)Mm1m9cnMUL!O@eu zXX;7VOFc=$(6*F8?Qc7UcB>hDRwHY`%7$l(w)*Mh^$II)qA9MHa4Z*__)%X>`2)KY zIu23?H#hwE6qoftQPyRvG?UE1A@wlY{RLZ*a3?&aCWU?B`j zgYVEPMoo{L0d_Gfw35{u+I6hfNoPU}C2>SaY=6kF4Q93ysFgm?s45mxx%wYfm>om+ zDcUS+WfEJPHBI+2IprLWHwU|`(=5~ru|m@E&ZX2?Dwy`F?FHvmVBpaLqeOI7>|)yO z@XyaEcQ4E&>uFiDS4R|A9Nh0+@`4SH;!0Ml#gsc%5% z(zYhe98`1g5B&?rTIvniu=6_N?v}r2R>o2SQk%ci?Zm4$WtVO6uRd#YSasM5 z?d)zLw#e)O6bgFMXM<)-$P@l0XoHCzXqCKoDf4;ooIV=Ic^W%EJNiD{h@6A{!@OuD z!4J(+fd1BH7j-AdFzx#E(>JUo z?g=(tCLgRH>Ezv;{0&pkLy#Pg)O%kkF5kvFTl-l>iW3r8W5K3SCR7@f7yoF^{9eZu z>73irAS#``aa*v3v)jBEP)^zb<`HzCIh?p7DK75Tsgbsw6;Uv9xl_?~x;Z$diJ9~C|FgnpX8M1PIF$c2;{F50*;v|{{@01q zW#M4{ukQB0P8u~qd1Y?&XEcDK2Mpq!kaywCVNuJX@Qp2_+jjs4& zpS0&O=*66{Hn#9_zw&)myFPOl#>C%7{!w2Si5!4vk?DKb50it?gYsCw|2fAa1BQP% zSk$k@P7B?pa4Gf0#)(@8q;H(%ctqlt7!FGUQ&jH%iG0gz@l!mvJ`0m%JQkC4~636fD$qZoKz1vu}=zajO3= zDx(rNRrQc--?g3kXI~w~{8rz*>sf=lf4Zw@*R!`1pu9A0Q^pF$&G7zbr97rQ@wmpD zOnI^#XTI=Qp=#1Y6S8NT2e%wl(rJCb@0a-$TLe*y&^3JLG#gDluX?9YH8_*pVJ!z? zp!uwxlpUTSu1Yj$Tb(4zYV|7^krh5z?KyL#lxjCcoyGJUB3ek>eV$uD6#I%pTzsIy z2)IiIP%8;d6D-rhq8o3VI{XDDR~4EknT0no?uk+o*eur-&RzaH-GZMsplGWFxJ${rO*vvaLK{Yd-YcFPzdR9?mw07U4`sat5>2NrCaZ}l@xMzcHC?xsy) z)bt@sLF!a!)wbcZ_5alPy!+p!wIs*zlqS{gc;o^w#ZOeug}IffcV3iz+{{=z&14p> z;$Y}TvVvU{`m;EDRNEvGdvp(Cel!HMZyK3c(|>bhsYNEM8;7=AtPz>T-LCj$*AG{W z2>9UL_S0;8;jFyme#?)7x>mv*4N1|c$C0dkeJ?puW^2CD_^zyuN zjH;=UVjiBuTLztq{f)7K36;MWiQJw5+e(KscRr4BhbX%YEsHyzeNMt6Uoj8HFy&{i z8QjrS=Gbl9cTHYRy1zrSN>x&2xB^%+J%Gk_019ln-D_7jxKWy~E2mm-O}tWR3Mo$y zLqJ&N>hGpRJHo;(Kmidmlwa$ZZ&@TS{&ug3U#-_@rg0{{-NoL1h^!(&Mu5*s|7-70 zFWWp@tvqX44-N6A<(NyhEu$~VHQaSVQtx7iMuv6g)I)74vV-k2=c5DSM3#glQ$PtQ z+$8C>8 zf$?~mG!?Pkp_dVXGZV*xQfbRB2O|Ek!hHrTX_A8!`Jgo}2OG;?05chQ;^GY+uTrly z{FCV&iSIG?_WzE>=>KCh;^O{)>6Xv1hyncHILXGv`=0>(zl}!b|D&9Gq4!llLF0-m zf-$50zsf0GpZt`F+9;Q;3)^e-7@yb8T$F$2Z=7x1S`L|M*MCc%7Q*o{lr_?S*+|9p%q%26%OU}O9laxQo~=@K0iM!6{MQgpo=I3 znAD&y`bEBU>XF%p>O{*>shg}bcDbv*x~Q7;s*(PcSwZseh9+|0gyzPTu>JcwfZx@O z{X&JW*KFv2U5Ahg_owi|$1dkHiPOv^Pqm&k*(Hp>-_ zLS55-b-~P6D*_sup-M|T4lO7`CLTJu0g(4)lD4zYkE z*S!hEDCdIZ)_&DxLi$%KP(U@0Q>%e%V}xxN+)=?#FjlV!B+jE9i(Z&#-3oGG0h4~=X?jh13o?xDqAyC_7qtO?Y`3q8)*~@@8*1^L^hni)x0B~i zk|baUK>3^kzq)3Pxh5Vu{-c)4Iy*r2m+_X^V9Hfx%Z&ZQq!;hoLP8w8$wu92oy2It zUDLd5`1D06WOGtD#h+A|_G4NAYx5aB4W_xr{SijqxMK;|oi(#RrJnN?*`0)c2)TaJ zOIRppa-X_Ldqxbj=nT9PQ8TA#rfXz6s0uk&+4uvsPblLbDv|T^EYMHJlrirPp0JyC zD1?C?mk;ERY3XU%t(V**b<2?MjPC}M4(2HJ)JusKU_QKKLKdHDsio#qM3~p_o}Y>g z-baGn$j9%mIIm#-umH+U*VRSQ&m{8x;&ZiDb;j{Q)yjCTtqb-U!#&rhxQZPCoXm4p zdp1DOc*Ws48&tHzvS|c1dLk?^qrP-&%+675-&y*JW)OWtDB8v^*=3N(Ba{}#0uHyl zH-iY9AvVmAv;Mkj3_aIv^RRzgB5aCd>`iZvw>OK8sQri#jIF<=b_X9k-7{uAm}uv} zn3UeWp!9_{9OAy@_ksX20p@hdzk4Y&MSz%fjXL>wT#AgQ{DX`v$PEMmG6gldkf>OlE~IKVB4FM)^ZRvut#@rvU`1d+&$M&Bef2e^ z7`ycb$`d*#L__%WcZ1rWq?~|N&v3#W$4lgn)Jw z0yye@->j3=Co2Dphj^(PV>`mJ?e&IIS|Klj1m`Cx9wy<~i+gAxuk;Hl(gG;DbJ%g|StUX4zmb$Ua3SD)=M(5GhEO>9_Wi{$~ z^be2d#7TMp=`WO9KFlQSIOzX%Jy*wMK3&THD$LBCl1LP7X8wzPMjYt=DLkH4>%aX{Vpc_EdwU# zrU7k<6h)q6~6EPi#k0$R+5zEGSWGnq3C<2UUyQ3f(~;? z82MSG?TV4D=m^m);emZ-*xWe{k-9KIG;2Crs#GN>_pQ7P#StKUAxcBkb7Ao0ibPSy zQ)w)@TSv6kH~-P0mrwvAAP1N{cdiZX%qjvi7}J`{ZD5XQrrHzGtbt{$zF*!)LBA&HzuJyOS;@ zyJ9HpyD|~(1_%gpGp*jhl9LcW+yJRJ%-oAoy4d@ZwBqT?mfG;^Qf%X7~zRm%38dlK+snE@NRkgikak^y>Ta zG?;AK&xG{Q5B&MJ4%m6TVMwLrFMoc%mSTaHUFZOdBaqE2>G`QT2Q1cln_G~{wCIV= zqN2U}2~)JBV6B_at=c#9#ta9xNZg{BBcYag5v$SluCik_X zKZ3+-^c6&g-zV`%R93sZEtxjTS1_Pt!qk+C)n(FY%#Cn+Q!5U!RfSEJ!Sj^Xk|hI% zlx+dCuOkdnctO_Ddh|EA=7ghWl?2x zl7M(McAqfYqIfnc=^4y8%hu~>pRZynuF?U^8sW4&e<~lvmu`skwd@~lM99(~a1NpC z8$jU9Z~vT2wGGUo_GR%!M7qXra^)dU-{87tmjN-Y{PI;%&HB(Pe%=n7o|GKQOkZs! z%YPBk1ZxgL0qEhMYA9M}p5KtK$S?*(>d^W-SC69YguxNJSUi_e*(0&JuJE}w}IO?dKKm^ARq(3&oWKi20u~bqcBlw*Q zoeOV&M$I@^gWB2LJXfZsenwrDCC*ZX4p@wFPBtpbcL>-!{N~!V@lURs^N;q4cCLOR zo-FV7cv>AWu&nTGeWI_WOTnex3+l8|53{6yZrp5Y|h!w>2ILuO7wV~u7?5>BsiqcAG1UO5#{67v zW}YeE4%=JzE6XpIZuYMCm4&?CuJbux6{+eGax&`5wf&k~Tc1KoFZ@yjmtIFPsGSZu z4^NiM!##w>7WjkoEQ35SAnaF$`sb-OQ2XXfIzn3H;7XfatJ(uMCobxTl!wRQBf{l1 zFkF7P|D!7Dk*bGfh?sO#kL3b91%_mzk5l2QjjC&mm{X0I^=>4>(gwFsyBR^q81N!x z;W_&Smg!oU9t~@lbr$o~kIamKA>7WLhX})^Zc{_E^W_3>H37mq&$*-G%v6c3LnU{Umffqo$gS6rRS?+`fDt+Hmp7lp+(RBho&q@zOL9d~MciI8yxI!>XmL zOE=11Kid2L=<>rcgA3cC_iN2jpym* zr!5UyEBv36VE6#bx96s}@lAg@e10MiZQ@yjBqC}bcNCtv3k>DyS01cx9SwTKoS&MS1y`jB&-zIleE(EfttSxDW!d5WsfGyUH&#*^p<83v^W6Js=qO15Wy z95S2HhBzVcU=G{WEmN|OG|YO5B={vz$SZ9jm)fP|>;jVkP>kyb!ANF!<^RAk*M=;G zY}uC4)8o}LN!z&;>m&OX#rjwAwI_pTvxdTxGfwLMW^%Xi!Ue5^=%G0Nsxmp*DA=<& z*>N{+PqyGMd)ICNyIPlDW$oZQYoA4#27=Vv^qJvQFicTd`&9vu~aHA`jdG~7R!v1xfqOk z{bMZfHHNE$pZ+SPsJ}fS^~xv1VGm{F?s^nye0@1dx_Dg}(XYV^;hDgLdz{i|-|Rl_aM^ll(O(k9sKRml~WySc$-7+h@u-I$^PV<8DluHg%|VWS{U(EojZ15Gx`r z6rUNK^RkvRWx@%gj_~UX99EE=g)vf{5}?~;UFoPYTx3ydF8|x=>$s)VNtW>TqVzv5&}+f~flSl!;jw=PE$w;Z?T_Q%VCXH_i!_lMGqR3!(Yw@}UV z-@4haAhYlTMgT*Cp3%)u^1liy=J(-$m5M zYKduS1xw{7D;$M{Dnfl?U3-Cb2H@lN`ETa8j7g5p2acD7j_WVhU4lGY^Uj@@u|AXq zb#R#V$@Kem>JymD1EcZNKNC@Efd0CD13ef*h`Y!Vr6KP3lLgL3AK8rHC6~)B(X!{Q zZW3Z)Pt;sN^*24_QNMhM+dXB9hzccJoQZTiPio zK^LE9;vTBhHET281JT3jdy9jIy>l7UEY%|NB3$a8jYR#|cGuL)Dt(HWB6WR^V7pl9 zZi3HeAmsI{rjS#o{BM+0fSv&g|EVdMT}uK5cH;~obbN}<^*2KeRw$*z}&5SGxv4D zqbJ7^U`-_^LLFd=!x;mSjcMWJ%m0OSS)%mFb(YyiD;C?1P)<|GwRjEIcAM6tGl?yL z1Yu#qMCVq)Zxq9Q50)=;J(=I3fEQM!`O@yGf(IM=`bfKL;f0|u{1nWwSIc7))^aJR z)2JWjWn!?7B#MIp0Ko}AL%iVOfCn{K_I}5Q8E{46+x@*@$X9D?A>eCDEZf0Ongmi{ za?v%&=it=S|8j?P{jk#qkG@Rr@6Pd<*=0m8clA2giYd~b012|*^UmvV|Bc0@H;=H( ze6Z-MBYvxM(d53u=)?XD6L7qYZVi~oV3;Cg*eH1-oZpI- zNNbZ9sa$+Ovv*SoKf^?vJ#sGNIIBKLYpcRV=)iNJZS_8?ciW@F{WV#>DvS2~=^V>p zcwC0+`Y$-rAMDUZFs@!z9ypC3f6sonM;Q~g85^)%DUwOrrMT#0lREO^nE=isaVgap z^dVpNS9Z|`ezKLLje6y$?c@!_5 zbk0fHBh%=|)=mp7EC@K+5h_g=+>Z*HT6LvBk)o#Jg|#5f>UN|M2fik9(Qif9+3D2&cMdng*nUnovlr6t z!5V;q&R_eJXcq@lZm1l=KDP8T+5ye;r3BW8Yn%<=7Z2_{3*zARTeRnPgCVmQ^ig{~ z_XA$}Y>4cV^Um|_$Vbmrt<9E8bKf@WUv3%ITKdtA$p5!7pCQrSs9BpPZZ=SY(l;=%1#4W zF5}9WYjFv-_hIwcs_A|Q;s>!~f3&IIam`c4w8i{`h1e~vohdz~aA95Aw$fEJKvu>x zx?G)CXnhQdm4uiL_r%>IidWaBD?)#=wiq!Ea=4T@YgqpB(=yCy;ahsjw~u)&RL6Ud zdph;U(;v1s{<+U#`kPUptlxS&OK#oRIR!mR*EJjh#PI1r3MfUzTxi%qU)X)Vibp2y~ z>ODuw87Dr!8p1Qq#Jj(?r=aYa2^VbBqCI5s3RxSeWfpdZ2xNL3b zmSt9BeUN=iV=HQGJu{pILL2)Ohcij4M8O(l});J-?KB)pl)Mx=tU0Qy64>^?!+3qUjR^ze8c-LR}+?}#9f*m8dIWC&o zt&LmHe}Gic>a-#e8@Iq*^*V%HntoVoyLYna%uxD7c5d8A-oPFBe89TYk_a`=Z*tiC z&HnX`DqQT#Ocfrcf+n^$LZf1ZIhlGjzVXO8is`6dS}%tg^NzApqN@YMvq6fmH8J4~ zxyh~K`eORx;ZuQAZ!s`g%6yEV3FHYgD?kQ$x`Aqjyf_`Zbni!~Jwx=MUs+wzPy58! zr#W@j&@wpQujpW`>0qG;d_!Na`0<9{-u2hLlOry_2@uk*e_=@a3PPgHqUqSe5dDoz z_9a>T8d1{w%Q3-9ttt|LmNnPT9{;E=!O42{R^PMgO@Zs3a~5L9Gj4|(HK^Q-&9l;< zL-&w`P{vOzY#>hT&ilNCH8@H`$hmbKJh}V5;*${m9yOSe0A|pi-MzzL(Be!PSt!6n zcQw$K>dJg3D~d2TTU0V@WvbI<{mth2PjN4`CV7GF zoAc&&DiOfFr8xuOKTZ5X-d3+?>Cd5Y#roBPeBW!lhRkt&f=>L6=d|hjK`;>)pd)J# zK90&2kBSUyJpzu_H-rF<0+U7^2RjFS4DO7^1edEq(cmOvwJQ?Ig$!#?cp2$~sgAjl zW<#5_*x>ZD{d)qwg0JJy^Fz3Hpi4@NHu0VWB)jB)Hh^8y`Jol1RoJ=Qfp@3sN358# zcw{$%{uwsgu@bd`)1k`6Im=zeS;o2;_h0d=0?Os0oL%zC2v84N#wf_TGCw2&d&ei^ zzF_e7?5H-vSrD_G1yv+f^)U>J=C|JLOu#v%LGC5{H{G)1bpF$Rh!nkMt|Jrq#Eb0q z9h7Dv02nuA-&lRkWWMk@ZQEBC@EfnA{xdVUhozi5$L@td|FLB-u&}B7(3=Gi>on=p zz4pHQRd-!pBTzKjexZ048XY{}fbhpGYY-`eWrPpqQ$A<k!S8A%hZM~g5Q!S(*5u+`S9bxPcp zM8@XmpozH|DQy?3a3dHmkE?tQ86P1104>S zlHPGkO}_!!n-OOu_5g4$9W3>uW08x4W5|$*&y`AXLinbNsaQ_v_!;-Z6Q$_b`M?^h zYZ2XmJlrIsA@o6mLYwG_1r`>|rA=7-Ln1FwnL5J& zPfl^?n8HaLbZyb?1E>%oICu@@1N$K(yn>*p6l69D7`%qzTQwLwu~4(Hj;|?Pwj&I^ zqWb2z`6-CU`4E>0X`a3cG&N}|MzsR&xVZMdUXXM~m~vc#l;4i%VlV9kR%olHwEXo= z-;5r)O3{V#fi}plz^QZ8k#SqIQFn6`p+$eV^LYV9<#MsJv;(_NYqR~RfLx}&H<~uV zArSeI!YBbMn3Q-euW#9tP$+j$G0!YwzlCR!Ff}x0CQd02Seuo%a*I?Y=T^$j-~q#% z`Q9@Giv229=9))0Gyb0v6vJiRi7ubLB>D3D^QCol;2%l5uiVAoX0Eq{MP^+t?_DCr zi=+GVZ%GuwtLZYFvT}4l&+O4t`K|V`EXef_+3b}Wa^2cIv3vF5k4q~Rq!HD~$^Dzs zybjZdO>37nLZ%>V{a=0IO9m1$G7c(BEqOH_?#7_go|$D`hHQN+9G{rqqKqNM^$7Bz zPZt~qKn;StCABi}w zWsn(h69iqYS??rle`;qRmED|Nn+P?iQc7zdeK_><&{dJXu_V3gThEsq7YHB*VN^?r zt7hj>8{tT^T_dyKqDeM_nxuIXZ_HkP8n5IBK-nriloZH%`%j0yO6%q_nPkoZ@2r*+ zbAqVu(TVpXm7fkLil(#5n$^&+zun)XbK_RvOdV!Nsqu^E1{`UVqelkMj>5c>F6N9y zF13XU9fdx#2Za$>P#x#L2ny1n^)5y z;6~x5Vpi}*?ck{trrN|>qkpem3*Qmn@$ZsTD@QL!uX}4RHteVcGaX%SH>(i2)HeAd z)%O;;*RK5r<@9qwOX{+4((Wku`7gKMAW^r45ACLV%Mvc&44DN1=!tW2YgGAPkc zoaVNEZ6mb-5glc-a57N&pW z-wp1yQ0FY@pP%7Tpxh~3`^8H+?1E}2+8?uekB zDY0&khHZKMuxdB|byQy^thqXr(4a>JsWe(jWN4wD2G- z^+!w+BuVH8#JUAm#POD}=&3SOOKusR$3*1FV1p@B4J{n(68!srxd(-g#@7h{1;ejz ze8{d}AkJ~wmtItA>!^RIEgo$W?lilaI9M(*tR`T$=oQCuh8niF7H29xYNo0s-33)NyD!o z;?hc^q|O*o^P1qzg|sGtdwYG~$^A9*X4$hUw@)Ybd!lxB1zQ|V&3w8Emo~`owTy9q zRe8Kja`4BD)yPPcsq1BX+ko48Bu7A`R1=OCE1Y;T$b7CbNed zZbyz*tf^0W56|0~wXu-D5YUR@VzI~2ykXJ7`cOM}CfSvkwIJfnGUcW~| zY1(pNB4c7@qCJp!mPnMC?ZV(v;bP}<>O$y}h4h33bsqS-7PuB8=uEx*=WO<-)I$qh zH$gYmC-n?$Fq_C<+tp*SvWc1~7SrymhW?}27l7%V9b-jjC6BkIxPAB@rKu8?sus1V zK9u~<5})v$YgjGrux1!E{mQYt!ONa4M#-B`hGvEQJs6M*DG7EDMh?OBph3?dd8VO| z7HbamJX#zAvmS~i`;~sF>V_iEa&8=o2XkPYWs>pN_gxKb^|qzGILa+2%gc*u60tPh z{0vh0)If;SohFw`Ej)VfhouiicJbja^s??56PpeFzM;UCD#@)v%jpkN`XZq+zx*pNn}4=oy2$ z13RFbH&+sC56AUjUEb$*4%2cJ4^=V^Tw9Ml>j~@bJhNKAv%+UXpLvt}k70EW|A79( ziR?y^>DB4J%j-K69{2gne%~^)p~n%CyQt@R0UiW%7}QM!vmUSWp-xgFnoT$CxYyfV>)1$f#-ab=QjpBjW$gG+~l4GW^?1UQVD|4O65#17~W&lm_wijJ8wudK%tY`T6o% zo|(DrYCwNdAQAr}uF*WP^dc(sso`$S*XyS8B=a0!H7SO@1K#aU^*K_$$p)op)>u*} zFG>2_Td~7cws}M~N6v^dW0^mU{2j15{nN|oJ+II4)bRD!&9>um*W*nf*oswAq`rd4 zsUbKm8RY6UlBfu7waDlM<<@Lat#=gD*QQ3Wl~x|>k-0CcBt`-8poSW#25VX2^a-3I zM9R^ znS><~9v?Md;o!D^yWn&bU8Sa^SNeNCBD_o6sT1$IQdJsbM88=#W5nsum(u=Bum(|D z2EFhq_@+7KMfe|r^6fM>6cK_njNciAorlQ_1@58$K18sZI-Ja2iuA^L6}B-ScyOJy zHLh;4?m8Hbj!v{k(tD#_P$&aQRgl68U**5Pu%8Nbokk51`4O&l7ko9Tch{u#ELdnJ zyi8%HZJVIQu<3stXlSr$$i1ZPH;9D z`|J3R>8#Qb$M%vswuOrn;_&aLqlZMFR_q~uB0X&SaRt2P!^&&oZC*h@^mYChZNt^2 zc&eaazy5nAWiQ#g+Ua%|HYU2Cy=(6J=7m+;7zB3cqpdlWPMG^a=Qz3XPngclt2a*K zsZm}3toaB73E;4leE0)0btsY)$tj-~ql!&2n63$$D!*7yDUOYlR2V9<=A3KnLA)(> z#pDexEEaV?EIiD$!wwQKp)(=euRSNWiJa6D`Gg3wD%43Bby^p!1i!7DdP7-|5%=8o zUK?(ImQf!V`_nhAQbXJp@hu$?n-i=>?xSqIS!>;+&d`lHcHf%WEq+(l9d8y3qY>-A zj(N<|y?tPe^G6We>pwQU)cF|8fJ_&Vvr)-Mb&UWPBX^=vg8+TKBVm zygsM-Bicvx>Oj1_a*_><&MAN1VbPV08jal{YH)L;*^xscQQI)aIRb>Z6N`Q&$|7W6 z52pqE!dgUfkjVko$sbv}3P?5a*I{Vjk@HkfU`q_yfd-xz`1{1Zu^^t!b=1;)gbxB& zwQD1&>qQaYce%b#KJp}#utMn75@1+i4#myn1ip zCrD!yFDvemQNr)$%q$|`8IGxSz2Fayhe|9H+EFKWwA}G0snW@ESbuL_v<6;ul{`@7 z*yPjrTmzacX#-!)P`yU;C_H#95<-2Zy!0Yn$*-^IcC_(xMXe3O7W=aD6pk9&=l{B6 zu?4Dwhk@EVNAd>r8m6>suyVD_^&%@P#!Q{pObaHs|JEX8{rgZAyT&&V#9X7?hUJhF`}?AX@As~>Q6n!>$+eG4 zDPU~)YOf`kcSx>36%jkJ-P+V{rV?bhsy6e)rz{yqM*}z>@s#Xx?C9%C)&JogKolhq#gxAveK=@^ic^AxLJ% zrN}bs+1|SR{^XHs&vd32#yEj#k(cimu!AeSU`}$!QuQGD=MSP9i6;IWbrI$rTrut} zAeA;Zc-4{GxHL!CsNGyjgrxf?IcUA6umnA=^rXI0Y||H_3)S7V05|vhG8nyr<&Sk$ zn20Nf1#{Fxj6p%os(#*0*pn+ku^Id%X$EP95?(Z0FjR`aB3!C%1c&K%)0H)HfH%q; zl#MR9fiW}P78&77c>$t5ohlu>J|qggp=hlDnA@3unv$*Lh3Ia6W3yg3u?hZhIQf*K zKl({g0Mdy)woBa4K`29&)z%}mZN0S806Z^uiiJ-`G@3nnT*2g^1}jd1;4@iYF9*(} z0wvM_Jg`H*WR~#-Cm+$Cb+H>8Sd)Sn>Bo4ClCE8c#7Ah7$GLxYXu;jLWjrov{Ouxg z`9X0P)j+gqiFfQ4tRCvpKonTQm(|<4vko(o5u-1DO>Q6LyMwmxFxR`MyoPkL$D>Z? zvaP}JcRtt~{iOqH)Is9uETYbby(#v{o1o?*vo>o*BaT^I2z)q_7pzPU$i9RSi3)hd zJT-gM_uO#d%ffP#EqI0N7va6LoBNDG6x8optyfY zwmTyBq8v-17OR~@GG9c<-V@HN@P~PjcSs{6`T4RhrKRk}i^V4+>-zzH%UJe`8dc)_ zE+uln) z;UUORB1bB7@KV>9--I2&ftDfPvbhg#!#Q+QAwaawPW@ZIA2tCvdw62kqR&do06|On z`_mpL$QPszJ(eHXAf^=@R8QbY6myhVY%S7ryk!x8Zc;h9`4_GDx{uO3UIQnryWG0G zl`hF)E6n|(c&Es^yodpzoQlTJ$#zU%HLw&*M>B|N9q6bug)Y!#hT*(z`%hKxciwt#Lb9d0or}f}~ z-&klqP1!WN=9Nix!{RMY^XEakz1OPms+*u;&P5Ce=Cj2$d5n_c{~3p|hVmCpa;!_zxcXigJ&8ORVTgW;8BFoNq3#(!8m1t%;G{ zby#`cnw&r%I8Xmz1~WhWvV+w(`T;!jfAPZF7Nxl;BvA|S-$T1qX8QI0i{fv2|D!-p z#JefaA^_-NYuFekLQ_)ksXb*YLIKZu!k&$+c@2Y39>uHCexMiEYSMMIozCS}vIPzv zzHH`-oldRfF8@jk0(Fs3<>Wqn0XfERMBa7vxpR}fWxID-!ylxQYYHH$KtOn0_hN0M7($e|sF05hrJ6& zz{#;8`KK3-Ndfd7>IxJ0N}Jzrk1k1X)SGmfE>63Qd|jZ(bkcP8z4_*&Ec(@PY0iY` z+sfAA2-5A06KQ@W50%SVR0R+qN3lFRrV zddz1psK6mP26VW4*yJn!-g=4RG!aa^CbApK6ebjg*{9SeU|MO767JX4JAFI@2x;aN z`O|j=SnWdOq`HTOgq5;h473xt)hb^VY$*NP#f+7<5w{V46FRVqQnpvCLUa2Sv2=}& z8|eKG{{-9LIfo;uP>r`{5y~U29m2(7(g$M6m8vCmN!p>bDJ0G<|K2Ie?}@e|PQQ$N z>QTUhkwO2B@PXhl#4I$x8*$kZ=u4m#lDwA#Ark*L6`I~Pk-AqzBBUbwn_}K$|&{puF!3H@c9IUizO1?r*VgAB?)ItHJAz61h!ef>`(kiAt2im%~I zB>il&x+WRV{LOc8eUDVVKOzPj&KIpL@q(Lf3D81kcG3>7?575*#Q}xI3HrHNVWJCl z!^Pd4dRKz3hW>-*=vn1otTT5l7wjf=v7~+kO!1oJx4aJg*oDklsU^8LA&8rhvpC^e zx$Ya{tiR_p4u1rF))JF0oQUbLCt{c{DV|iH$2Z=zzF)IdOp;RMidfZ3I|)hG8uv-Q z5>jOK87g9u8ntlomfA#pezYUPm{m^5tUFYKCXZymP9pbH!^W_9m zLQl~z?PX3-2Ip$A53f*H7y)tk#JD}oI!6!#afC=>tu07^IN&6<1^?!AWpblCpbI!# zYx!>`wB;ugI{x|SQ+i(Yq>XE)^tEfnIL;bpPwr+KSdInSn4)aCF&F2N%8O!2<v;RYfM>Xe+HvOHja%9xVBzcyM<$e9 z^4!)RZMLOL#q#(`_~gz%{NLLrw-)?i#0CI}e&49CKvau9b3>k&&$C)iztYXr+!Otp zj`hYVKxI`uiCp0lF1=&SdsuFlx81zunAvO|enHE%c?<89eK?oI7q^?+bgbNPR3q@Y z(YHL|4sG1Dy;qHYE<|)Ut z6mxa2LtI$G5B}Kc<`a5iC%o|c)Lj8~wu;~X)pVGgM?E1~EjIsSn{E6NZBBgpO=(9f z_Mbw7CYm{XQo6%W(t!<)bQq09h&S+To%0au12Tq9Xw|20u)2$Rh90<5yIFS}&knqQ zxr0zfNoIHG13yrO`_Ov()12CmLWf(L@9{Q~(rkVxl|)`eVOlCF2S)k0vSf<@tA3=e zEV|PV$KbY=qDvK70}m=w%F&T7PwCHV9>q7*H|A604ba!o9e54{AI5akY?qy)bB zH8Wl_Y%`aHxiAV6Pl-iaPI>7PRg^kXt~_)8BAMU$cElQ(k`{C&Bs1*a)ZV3}`$V9l zU@ZJxYK>azB7lZrEq})v?$Z7S49T|%+it*!kE72F8kN-H52Q%3bKpyMAAx%^0VX{gi#`=2Hq`uIp+-gwR!%gx0TZx3!fZVW}(f;La zWYTZ)n)y{raRSl%+=CgqL!$JFadST_wCVlR6!-nR;Kj3tsfYI%<9eR()ms2|BENhx z$yu(>w_2*NAb*A@;6l8gSJ*G}hI#55X%T4%;?RUFpp*KMmc~jJ_F!}}LHB`ek#llWw%hR5@XWFzi?Q&Kk@D~Bd&YAZ2!9Ta zdVVugBaiv#zfOm_t3NU085gDJQ(ZyoeGmUuw}uz~b=YGE92)_VXtBrdTu-%;92`JN zaxPJ7zE3f*zuZ!qi$n^dZ;u7u!Lzw#zvYuGBbLEDYut{Hv%UFf zM+o|csRv;knm61^&_9)2*dG1G4}Sb)L4(X2z|=p?r#F^1Jy5TTTYnF328BJ|!(4^W#yvfq?$IrWcYCwsjKJ><~6G1W41H2x1~cNtSh*fk2fKyfJU?(Xiz-Q8{D?#>h~ zPH~6g?(W6i-6`%?TtA-kzB!Wb{5r|b*`&8**>QUC~g5?zyH+9kdVX#``_R9myF+$@ zCvmp+Lj}@mF~x|vie&l$4{CqyI5z9rj|#@8;0EMT!oFW(do0EMzIXuz$AYQrycVc3F{a&iV=4mI)4g zNWJJZQe~-M&D?e-Tl&;CF{#rSyup7RpeM3)dv`_p`*Hh(>EQD1&gD@2@P%{Ncgy;f z`hzIDwBgPD^NVvP*Ve)KzVD&gaQPk?l1O`<;z|)=5Aa$a#;4S6Pci(vj8w4f$%aeT zJ@K5P1-cX2`7V`uK)PNDFdI1)mjfE_`K81+<_#k!%7u=A|E8e3)!-t!gQS)|ssgfw z)MB?Hx-%@p=aPHB&NddPhD`i+Xm`p9eKzMk#(bwuydq*J=#PQhWlm4cgN%#NZt%JpPXb9bGUr{cinCt17c}iv`He3BQU~m~Nx*$nuuw zzBMeX&OIP2!mZtQ7Ey&T7I2dn4Q%CGzEE+E5Wf+OKfO|HG_P-l3R+D6dkXQ+IQPyV zT3VX2pH41-%Xo!zEpYJBz1ROjv9oprC=U3W!E90y?N#JB^sf!@{>8Pjc&Rw=`Z4KZ zX%{WfiOFT-UyhNLC%kNxJ~TDWRG+&+!1(-~fNIhQv%L7$Y4)8w*wuM)55??|=V^xWP7vs>)cJeC2zUz#Xz&t^2V-T@{(Xm-!HVbsDeR(@cwEvK;A zB1t#A-#vF*!|+8h-;jwzAO5`;0lDA7!gg0N7a&}_u3wMh*J9io@(Io@CHYoo{@;hY zAIR8mM3vik)IXtn5b*h@L*C1)Z&dHMe!xBwVl%ldClN*A`x~6VNCGH&Ct5^PQD<>* z1>?_xabo%CYf14MXBSJ-5j%5jVK(U1|4{glUyxbl;_7PFgO)JDc>BNg3-%iaga6QM z!xJBuv=zZGAeCkmmi0wUfM-;F$W}5c?*F}Eu z2y#29LC46-53&`T#|#%1yR<@JG{}Dq|Lv|4`1_`e>2{QEDZK+MXW!36rCv>6-{3#s zE4C%ACa-k@mV-ZkWH5Jt(xdY08w?<{M--R=e9=e72%`y;5AzE{gJJvlQcd_S=oHza z(gxo1^x}DU^>Ed=*;k&?9a-w})Xv?&b?TFhafuS{i6wwCGKUs`AP~31WWNVgd{mJ+ zs|i-IJI}kQd;@MnP-%(K2Iz%qB9V?F5gSN$AnK#Q_FD#r5LFG?<38m+zrM%$`@cg0 zkl@_A&`+0;ep%i0M>ek%!iM7MA-7+ERzW)Ow=++o8Xy|F6xs@_!(pd;BPjz_`Niu= zFtZ)Tpq-c&IUiXSxtR>TEB0_+F}7e^G&=|a^H2KSRmktyo<2;e@o@4RXkrM75&mqR-!-dqpy9qr*LidZjiD zG#>cN@y1RF&E43vHts6yDaz?A+)`@VxXa<|+vqh?jNGPb4Y)Df=w}=in&?qLZqQmA za~1Y1SVq{ZQ-8EvrWB_6JJF;9+Z(+lELHq1zw)ANJoM>KSA_F86 zwK!R$LdzHpoji}hnO3A{m52LkZD1jKi7^bWAH5&0A2ygY)CIi9xJ$5~z{E)ZRyZ~f9{>YffGe&H z3o2FtW5sngu{MG6b!}U<hhBjp?h6HIWZ10&;YOQq)}TgfuU&LmqyfFxmv46G`AW?8plXUK;>?WE zx{>cj|50l%(vfUv@9^&-DGzjOoDW;<>R-k$_L;&)FLObru~_=F{6r zsz`*auDKH(eHV8ceioyLMTnn3P^pW z3-rntvR2PxTn9+{8$l%BnoPA^amP@RGNd=dbLy`TG)j4CA$>|x zMKXPlHBN6UHGA(?%xANfH@7WXWcAT$iR=T<^`KE^Y|bvAPSaN_Uz;G~4Cm8Ni|uN$ zrZR0-H?@kdwgWMDmBj(2YObmhtY+UA2#v;l<|=W-_|{!*+&I3z$e%a@2ck-BdZvjHvr8bDe?q{|KC2ij|e$e>NcE zN-{0gJK?Ko^VTBEpgs{!d?Pp|8G6;9um5da|b45f1QP0}mDwAD)@#C#nK= z&iheXW_A_MPR25p|5|_WN;gH8~V|`Zh_$bG7+0t8!=8mJ~ zP;vc2x*1y4P@uFrdwyxV-)II4hLA|VaSr^g99nMih!L=FCh1-@2WdV$_+Z{*jIUAg zdHM|++-hWh@#)|_4!cw2MR6@x?N)d@!W>24a5k>h?Z)WnQgk0QxDWOp3mVwe0>YSD zov)LSD>4O?UYfCi8q1L+_P?tfU5|(UR|aORENYA0!n*mMEvX{o*Nv-7!IEvCj(xfESw1qOw*Xw7)(m614NVnHkefqe85_dwJI_ zr^Y7}rB`^4MKSb1_kKWoLmKh?FCGW}KSKu-YdBUGw*Q+H=V;~lze{mk|K+>=PbrRr zhn1L(^Z!MP<6`?C=G=d)(=0sfoc}Gw>8{0~b0OQ&IJ}{U+rSY0661&QWnZ6-yhU4m+%DX%9~Sc^q)pcp@BS--c3VJLeDnE4 zwIkmts$vQID9sQG*29fXI*E zc{(t-jSRol!C30|qwdI{=~tC3Eh2yq-LsA*k(V;aa^$RkXRDWMhW-oZ^>>Lx8>RNc zyJC>flihvAd+QgnzLLlWl@tJ3PU5xYi2J1cW{Vs7rR3#m#@74$0TWp!oy^B-{^9Ob zI43EcMYO~gp&5$g)GAYY`^41EzZ$RAb!$e>l=QKQq=S=`^O^GdjC=@muwOSloymNzVhW2<>^TeDAelAR7h&g$CaGP8zn}>zmr5wAgcAUB@{hAy9J&I< zep!d01EV^LnaoBnPVh;Og%Rl^r&KGY`o6bID-=z_3FhRNa_Afg*r3lBT zp+};a5nIHjL!(afPhY!{s0;$)-#y01c(G-S@LE}?eH~;|R;aEq@hM+bqDycVDURGxm#w&-VmlZ=!cTC)nFV$(lVzvk^;OI5^}2y5d&wlFm2d{3kpz>x-Nk7)$R<91N0 zeX&9fG~tH_z)l18XtkO~r~>W%8oihkeG91Rm=_;in?M5LL;9|RlnuDV&|&edCLvD5 zT`HEkKW}8Z(CD*1bu`mb!mu4N*y?vu2jXol6==8T80cZsRdsPRZz=N;FC(T6pD+69 z(_lj(S#RPSAUJ7(hDDDKuk0jgQ^{*``4-jtFZX#sP(L`$Q;<3)&ExFtH|mvBrjt@! z@^hnVX5+wnyDd3x$F(uE+!~(!@l975dz^=7`emQwj{CnR{(&Koh1ctIL`lUD%Z&XK zP~Hv$Z#q2M(MCPFt?Gn@Q`^dHy9G(D8?Vd0wCCe*2f{87R}zCh`-PRvbLp7bDRYU} z)BM)J#qd`_ZRaoI!U6C`MQ)$0nS%bqK**{cy}m#bElVH0wt6tW7M^Efwtx!dkfS!% zrkzpt<$n)D|^lZN5=qonksm1&l?ytr{zm$&^^1PCHIGJ{( z(S?#fx5Hum?{(P`28#-)@N`YbIaf1@bX5F67Xha3zw8Cb#PKZHjH&bwK@h5(lX5-@ zx5u;O!L6t_#BD1Vw*w-jZV;p-Tar%D@hCzMb_%bu-7huXEaBKzU|!I@ajfi50eE7V zoPLG5O;S-1l3X4&RW3tb)S?8BRZl)RC+g-Gd{tS-Bp75o9<9s7zMDUL=E}_qzDQ(R z5NgEbge0?xL~F+qh{KL8T;*5a5%8x?tm#{ItWC~+uw&CN_wK}IACBubv&|rXq2jo2 zwQw^BglNGZr%@xOW{0C*@(vZq+Nv7?Mgj*J5`D{HYjcQ<-YjiB$ZuHoT1t{z<#^!V zCj*fkY)|h>D<_b(*1P5Qx^Z_+ccKz}WYQkf3TBYaRCr)KdvMrwthmYVq;=UxEfwIX z`}4gI^Wct|=nF+H8{)hAD?#4AO_Sr)*y8=(1jK7hW9cI(r%w9F*0mg!vC4vgiWJ@>mSmn#(LUES6d=xV%i)xsc59kY#UIyG-u-PpoDMg?Koqombks9br*gmZ3RDOn^>3%kj!ml`F9lZX5oLgg)q3FpHYXPuE%?IOtyI!SF& z?V(2tYWRuwKUXP7U;=e`kw>+>64`ks*=Z_buv{QlEebY|@>Gi=L}7=~qZ%3W3k{Yj zIcKj<2DATKwQNTD@`^4rtT6Dv^{bP>xE$Oxeu2XHZ5nK~UYkv)N#I(Dvq59k39sP6 zA3%EN*RI)3YK#w4)zt1*Pr$%d@eXu(;wkIc4Mb_8Wd@Eh##fo%K97I1DO8NY_TuLq z0bydgu?$I|=J|*3C%5~bF9f6Adov#?K=`KGiv#xM;s!cjwX7w__w5R8(TzV~pLB9T z70d>*q~&p3>vOg;N3+?~qo~V^<2wP(A}DSX{^OfoI@qs~v-;_9$hs4h6{_KEM&VOs z$hym1SPAnJijoR{p&muld!a(MHRuUH=|G=4tLk`UR7bdFCb_W3XDpY@#NPiELHy4E z;uDQ!CjL(-!O#DH>W19^k@x-&-7rLD%({;mx#NY-RSu~q{uKtyl$r*T==rz3(D&rU z(%j^bGyK`{aWQ3_U9qZcH!p4=34gncw* zH3y1W31wEWP`%fxgmllUUS|oxf|;@gj8cvwMdT=T{btP}5~R7;GM<3foCx^rEEEH( zFpRw}InLeW_2`~mq#A0acqF@cV&od;aVVv7zXB6|e-b1^hQn0G=xLE7KK4q3i^LcR z)beDJ_L-W(Z!CkAa!$*6VeIymKA=&2-u4Sa9mHa5C@8LXlxe|R&+9$Jqfa`O|Nos3 zL^xLN|B#0N*E8}z!npt4Iz0Qd4%HT(pg*;&=9R-q5UXrc8mNvn;5#NU_CyUCUmcV* z*orVnU0FB~p4ddZ63-3HL)m7T{Wrrh=^|vUo7Kk54&&$+X{8A3)=RdJ1B|lG%g2LH z*Wpw4ulEb>i`IusA7HXQhYb_^%W^Il`E;><1A7zCDU+em1i_b+)HHaD3E()sEjm>w zqQz+ODP=tK_8~gy%LUCBKctCl7PsfzKkrOn5){{I7xlHhhA88GH>~uBQ`z0FD|=-L zyERdl<>~(KfmW)8WBfc{Y_8lF&Yo0L7;Y~exxVlV;L(}DV*yiIT33TL0$K*}+pk%5 zc%)$n`@&(vA;?fteu0mPv&lTAipPS5uP(lABVYfBUSVfI#P$$A3h+fJ3Vdrff__C2 znMUgVL=NHFHH}FZ^&k$na{}+WP2%RiQ3p+MBW3*k(nNri(q|7-=Xk=V1*<7ry`!ob zgm?3sD=HY$kpRF;Z$qZ+dAUiNh6wG2tog-JKw03Q*2dySw2stXAXJpz1B=6q`0=fv zOXWS?8=7|)jjQkcO0a#V3oU+G7NV&G4YN<8)WEz{yY$lOYpWwmjBu#23I!e_MdLbM~038 zz0sL@ztVi<02OY$ZVTv+@p~DDiEZXhYVj8$xo314hBGGR3i6fFie^?qyr0q6>4|JjEVm+hfwZ(e18m#b8a?$5pssQ zlg(u#3${l6t{fUV#JW_pEA`6#S6Oq-499YG-ht09kCmayQb^^Fw_JVtk7n)L#2w+l zW#3OZV}*mBZtO8L(-pHv51#gzjXpvXqk662lXLjg|OOGTtC-nxL-3M|Gtx&iFNU502F=SG+(-(p{mOUF z)}yA{3%CJ2i99hrp*^9Q#oy%J#GNP|mX|M3-T@64tJe-V8a_Yo?gSxe<}isp~th#F^Qu@CN@uSO+klrWAki}IJVsQKoY!D z)x*E`Cnx_<_9#T-d}GE-^A35$VO{3ZxQ{QAd&Spl27_Eos`_Wn?~Nycb_kvv_fHk` z>Iza!y}Gk6atU00Yjr^(%P8-^*-=Zkn{Tv7zYZr|{MeF;T!vd7d4b?oSY9L}K(f$P zVvI80Kg&^pBNxOkE%|xN)q;9(4RrUS?Cmx>(q%AC1saM#>VC`l12K$2F$!r-#gQd2 zbvS(LiMs}-=a1c8mA=m=4f{vE!eF(;e_|^b{_pRz9}fkRPPfoc4h6W<;8!tZPukf?ajkT;(&@K~WU57t}~wK`?4oDX=b57%hG<}7msEI_u8Al`-@ ziTddrZiG6ENWi5B@e2|bE|1lS67*#lCGQwhyM5Wi(wz(lFNNN|pQmpjb|Z6@@T#F~ z#NT>E_WiolCwg|0JZ40)M|{{Z6iN&6C~Wq>M ziPfq>uBi#+T>)jyn=<%3WEu_qJYXtsyHf(2+FfPX!MNxtw)Zpq6)MidRFDgcDC`bt z>H^CKq-yep2d&5M1L%Lyyj{8JX2o<}cd_AT$r_jUg=(I`#%5+;F++HXmeXl}fv%Sx3pu-!<7 zyr}>#x^+F^6#>T!w(YioE1*T zS#gzLZOOw1mm9ESKqC0vrBt8ZVeqGSXu|fYIdoMzc-%tjf_v^)TruaPEFiE|f_`+p z+~xpAL+rmdHfK|m&S4$!&Y+ZhRFE`()s^HRM~=$}xRIjHJny9MU{9g$mrWL8aa?=) z<`VXOEk=^qSy!k$RS%uZ$&iKjwNpi%>=QBVD}j~ZfjTs^4+3fDRmCah^7&Nqed??V zL2ku&*ECKAdj#JPtiAIqYwiF8fz(C7y;=hE^;0~=e)t&`gM`E3BvSFwMsu+K4aY@i zq-6`JzspXukvruR9^EsKv5?teMBOh~98^o)woZh<0?LR_j^)!4OI7A;mVZe2jS}zb z91c&saS>U|5EK@RIiIk&q(*n57N<;cSNg7A>HWt%EE^>;rL59DwKop_YN%@MpeUkmZi-r5!31!8e`YHi-?2huoA4!gCp$UQbtR!gU z7+H+T<~B}=>FQl&L2Ggvd{H=7ZRQ}llwZbxkc!YzXjGF9!NxFx>5E+-w9E55U<7V8 zaQpSO)xvHXD53Hu%@Qy?6f>ws6|U85OZb zj>9s*Qb1TjS~?{XU8y14XzL}_$N@BxGBz_(YKpS>YK5>fp2hBIe1w16UdXp~GNzPI zX;qIMpRsA!$@rz;Dr>{4YR?^~OE)Tv;#Ai7(&Mi@N)I{#+UxQlZ3gYG*lwz*_h|z293NdGeb+P743e7+8sbN+iV8dKvJ3B|mf} zPuqFCOJLtT?3)zDokiQJQj9yT&)BA-e@ds=>GBgd@<@A_vE^!VEB6ztc!hB|QF3mn zsQPP@Wz#w3gWi6C@bBG9bPihb**r!%BTb@+F55+C_38K@sVXbeU#xO09;Ul0 zWrsQYnH;39kCq5wi(>FK%J9exMxv80OQC4bPOnGL{U{wO zP5yDi=?Q;MuyRe86W&?G^!0n6p~J<}FD?y~2@#&>+^PouMZnG78i@m59ytcsQ*-mR z{|q) z)Iwz4Lt43gdPs_Th3f$r`p`=U8G_Z^QzXl_a4}=)6{V|)>F|Bufb%i%(77VfP=G*WIPKg|rOg7x`Jp4chJo_F(Iv5^ms1^|I z8N!+;!Hf|<=-#6V#kK6$#+R8~6yt_A@#stXPkpxPG<6J?nnz5S#QIl)sE11(zbDm@ zxM9Xze~wIB3mJNZxEs-7ZON3O{{1Rg8{rRD(l{{&{f8v9m+QrE`u;*#1eE6nM9t4U ziLTFckIR`~7G{9<=D+39NZVqkhLi*2A6gFmaq~oXl)SR;LZUpPeljchf`tMWK7z7* zYwCx%yJ?5%7p?Pv)hfq(@f(quZ=x4_U*pgd2zP3}{?=ANQWHqwji~p8u}xid3So1` zZiX|3zk@5=kQd10P0qFCcKI#tSZxi)*8n z@q9ZYM~KsSZ0?~lYc-nNQSZc*C#)gGh%ftBugEZ08oE^4xm|xV8>+(WPDuR^6bEuW zEh7piXlr;#I#L=X+dMavpS4D;2HE3ng)szJROJ^?=_^+-saFLb53w0I(U zOV;uzd4Kxyw(#5~#uI?qt>py0@wQRE;S6kfZi%PO^7Q<>YYjN=f!|2^s?oj#SvAtJ zb*^L*Pt039b{@wO`o^lGI@_nn5~0|tjo~lXhQB#+83dD))v!yAfnT)dz)-uTV*-@Z z1NTg6_`1luwam3r6~guD)*7mnjipTmN@3?BP$AwVLrenqw z`RnfWR2~6zcYZg@=|s)6Fp^=8>U!s)+*q38VJ`m}Sz>8UP=3vBu@-S!v6TP>SS%`l0O4f(f&bPTKIN;Tj6ZT_S=gt7|E;eKIjuU_>#wDWK)gLe8dmi@ zu5)<IVsh;LdG z*SLlp49;$M>5Q;A`vFU*1N#!D09K3YrQJWx{02XUQBQr*M>_~qnx6lD^RkGUUV%c2%KaNcs?49&uyTj6oQ#T24F5YH5RAcAH zzeAK);CEF(!TqHBA^_^o7C7xPFCrNnR8|WA+^?NGyyUy_XKpLa1p=w}&`Wq#&1`n} z4K>B8p`gn$SGNm^5YpU&nK}I9_l5>6{>>j$4BOXj%IoBFmc-Iilt>Sr9V4#h-%g~f z#|Rued_*b1!snuzBQN&x+rlPYg#xJ`RvM57?rhEmQ+C-| zLX>)W#qcR}{jvfocveI#NOyu?Ww!flZ1qv$;_+cH$QzJ!KkKXKvTavK@z7bVT8~>% z;c%spEG=;`W8UnZ&kqf2cWdu&*sgQt zaSneR&Ll@tHUjt`b1q!Ad}63(*t43vC%Lvq{e}9L{zMoqyEd$BEH+KH8b!mT`JQw* z`?&AnE#6;Q!WTuHe--X9>!B{uWMSUbQD=;kCu?UWxUzlLbfn5rFi~y0?=e$PAJhEJ zKH?$CvZN(v*O6+WBAH9!zFGx)(~fL9i9LEb32*v@Pk|bL9oA060z=z8K+ZOsT|dJ< z<1~j**Sekg)Ozitg3GoV(oZJ}vuXGj4y_+T8QbW71Rm}Nb;N*at81TRKK}6m_oUoC z?MjS%!E6N!n+kVL{h#za@W?ti9h!2Za7Jobi{JS_`jGga$yd6bLsb%O1rn@y0?S>3 zjV4h@If3GEb*g{Xq6wn#B6On8QY`=pOJpvw03O_|l&CyE;FGs7$9$`$JIiN6&5=9q zm@dbgUsN!?DnoP2ld7Rbz7DTWCVxOBOO*{R*a>~ifoUlkIvaleceeg8`c&YafAi&7 zn=hTof4>Y#=i8RoL~(5A+PY~!(@&K&FerxyPTf8~i9Q_0ZNp%L9_C*EPq z3gBKHFO-NC|7nj_stoh;;^85c_FQkv86@R#$#@*mZAb=uEI()HzA{jyZ~901O=h0S1dO=WvRZf!*FbMl>$CzT*DZZ&;uMSCWAU ze|Vy`!bl-5N%aVq*9-hQYO(F@Wi(RzUsv%_#GAJ|F;Hgx@(x;g83iL0weU0Bm3fuwl` zfi%Vjg|^38)h!FRP`_GMmoJ!76}KgDb@AIPJ@xQH?tY8vwH{`4Mv%IUOpOfAjPCkJ zUF@z56{e^A3r*^i<$C2COCbg~=ZgOhb03$JF+S^NG3oLPTq=HnWD3X+0yCLg9I-Rn zYB<^K!sGB|qxODbqlmCuj1cnG1D6N&#KxR5TG6PLb(FRJ8Bw_v*Q{431{ZcFT;yc? z#w13)B7cHIe!wzWC8o}Qvd?q>1}1!+*bLXl4dXv~vf5h^uj1D9Ige~JYzQSd?vT!% zZu#9=brn{E6lM~QO|U_^VFwK6!dr(EBVR2}vf6qIqKDcJU>{d{alI z&Eb5G#>n>aZmNc*zPio%rh!8n=v!s6rxlIm$v7(MHy^maqj(J8g<>0F@Q=$4yz=MhinOETbY=QxFb zQe`Q#E5P|$rsHGX5^%Ct4&%#Xq6Pf)3+(kHZP|f;kwKf>!LOz|!TB4Pj}Y!GYJ5+e8Ck;q~{Z=Y&{B3(t;NgPN4&dB8ibBs=Mj_z)| zR^8MWoAN7$49y2*X`FR?Q2;!jlGJg7ETM(tG`jjY=TJs_q+ z{*_9B)$Zd?lCk|3xbV7Vc;9)GO|kRwe2SQ`ZSX&a6x9UY-E!&p?FsAO?)-`zY@c+; zxEOL8Y(FLlOK+AG&)Q7|%B(X7b-yIKOe4A7G=O3fm-1K5FV*)-RNDEVcn4+63%jLO z#bdY%Kl^*veDsRp42*hvWz~tluNQHHG$^`BR4eY3W-q2H6HQ#LWICHS)X$4o&$XJ; zO69DYq#gkK2k6rG&b>60v-;oLav1`sJZRAv?S1wx)baE~=#pPLrQ0Vk=jMMDm41V! z)-%_0S^abHt0e)fEweLzvkGplfSuMSNi?DYp-DNbzI@O`Cup%mS_jQAYuw3M7}7)_ z>lD772Kkghb&u7kL}NqIt=?OQQE^YJL=-+}`RxgMywb zd5haw1SRmb1KIIziCV%DZm#A^h9kqkwA@$}w=e%EyLn7{7-3rGv&|H*D816(Blnb8 z%4=%gA6+u)50A8VZIaaus*}0xcQVXvRieGDG-3Q&IRZ#j_3SDPXg93oGtj=5;l$=a z3$znJiUqyGAGZcw6^0mv-}n+5uRgrzL)1NiCrvvg~uVN3kDGvRtm4(`-78 zzYa6~2bs1MABK$I_=w45+$7Cp$Rt$pP6G}LTDvIsItyjshCTlL6f!u?*Ut@JAPgO5 z`h*y~9W#}&?9s*)=Gj$2dH!)Yd6q?VvtkUh5aP|{OT+r8HX3L38u%MKR8F~eEFHiP zEp@%GIx_@Kdq1pSFFF+wN}4FfqHVnzL&wIgtjJOsxa!pKuvQA#3@W|A!`Nwwd`J=N zS0p&`>W>rO4i^{QEGij;YTsLq5S9cceLeW8t~aK#Y?a+}v?!`y{eP@G=7DjXn(G}7 zIcD|!`k9%0>fxQ*?X%c_B)9fjVv7ane&KXl=;t7UX3!e33$qqnG&9T87~D#Hc5D~Q zN1e-G`xim$r#8u0Ha{_MQ{?$ky;ct&{jzMprP`LmkH-d?-J>03nxB8;hmShJ3u)#0mG{%G_adgvq6YpCgu!?u$#Cz zSUIp}B@3Vju6N*1atHSrR1bxQ>Wv}EweL!0gH|!TFO(|Zcos;>lf*z2eidc24JCKA zuOUx}Pb&Tfsm&}$!M1Qi%iWR*CyKM6OwhcetQ5tGO(-nu{3&Fb#J8UlM!eGW&iVaN zB5astV`6$6Fy*RpL7b)va~gmcEh*@rhN5%9pfhD^D>CdS18OZN(yWn0!Rk-76G_i_ z?FC*L0s*E7ON#b3IKwIoLqteXE2mO;$)6k-eb!`BIqhlHYY(%9OdY%xo@dja8h31` zk?-L;2~o|2<`(PgqK|GHZM)>_bv)nZFyk4UDw`VxsnV${iKd1%oAIL+)VMO0nV| z>&ONT&Ffr*yN1ryoa^gcSnbkjW%bM;pS-k+X?fDJ*y6=bmMm(b$b^sR5e!I<-7rnb zR?wS%*%{HU+C@t%TAakU4Wb}5=@#W~>?nU;hGtB`s2U;~)kA$F`GUz&vwX6wqb`V& z-yKb>YF-_ow1C_k^lu#Zy0+Brd>tab(IocKt{LuCC`H)rsTJR|xNTdOX$co8{qwn& zsJ_kZV^4tJ;NVOk)KfnMaEj}0lr%y1xLOG~dxrE7h=)9h_fDX;qmxulGoy$K^N^>) zoe#eDN-MTeeK-vSGUQ7t@aFHoD!Z8LzfV+OyxBL_tpP8~ebE`^K6PAF?0Rj_{aI6u zuW#=6SZ{%+gAdxX)gKj}mezSgootx=Y<`on<@`(U&I_$%w6q5pIko@t;E@=$pK+Wu zPJ)toZrYNWzJu!#Xvk7g5R*QfGq}4~s-mi>+k30|x0p?0X#SGqK7=?R}aV;ohzL_D*z|o^m~``XMsLYCw(`oS~ebxBBQoauWa2Ye`EHk zfQOF?;lBOUZg%wyA}Z=~SbV(YJu(GKg1041YlFiXN>zp=#nPmz6lf2H6UCcFr6k-z zx~;EJWP1yLcy#u++tGCQuyQs9k8IJ2YLByyR1Lfe6kMQZzk3(0ny|3FAW9L9SQk5e zF5sJ1!kUvugPuLYr62;M{;Du5oj-=P+j}FO%YBpa*eb249Phqu5~2@J#H3ax4U5{D z@95Pxs`ML*!m`=gm}md@p;~8ixY7q+prb+Bb~bB|qFG?h=*>e3 zv^t@>jd~SSa63M8Q(}M3KFQHKmVxJR&pc>VtJ1L;V@OakFTFg_Ae3O8hnVT6^3y!MfSX z44P)nNM1FWwx{>_guc(k2 zMCD5ODGv|W`yKTQl8$bl@7re;*D+3?lSg9b6$7{GO$oCKWeJNfLOy$h96(6&s#q_p zlq7noq_Sdwbb)^b#2er_OV-$b@L9kZn*9Cfv9g!=}NbQ zPh|qD!smVrg(J$98h8PMrr)AP(AZ1%m4Ni$!t%_G$-)FyahaoWc5?Lbiz1U;RI4iH z(Mk|U5cBw)JJuXu4I7;_ask2-1k8O6lsJ;4a8nVIo3_=^QSZs+=9@eC!!GyzMBULQ zt|7wCtGpMYHrthh1&>N0r^^jF9j-F22VbEdFNWTK)7~698)8dLVMbT+6C01E2s|i6 zm4{^{tIG{an`7N3e;v@kpRr{&xuPN;s1Yb+i?*K8*QV_Ns!xKB{DA1i!6n9V^1_-B zeZI#c^+&-lkGAUDKei%s2Gx{8qQjv+-mhs(1aQ#^3b2d_GClN{P<{!MZu*ns&XzS9 z4!Ii=Bw4|mn6iazgZ2tz@mVA`4krzLdLsS7@sDq17)Xy-)?;w(zZ%xb!<{l-qCb*YdKG| zDHb=#76oK0T9%7)-7aSq8zg=zE-(J9{bse@GQHZT6ZGA)4+dg^ov923g|*wQOZst@ z9%MiMU7Y6;T&D^qs!p%gFJGp6S^s&VYD%ZeJix=ig3$d&bqHuGFRpa(kBoJ`!}plD z`MJ_E-N~nwC_O$9p}T4Gq!266<-^(T<+ing^b0XFbZ`0xr(;+WxeK`aL3E;cn?!tE zNStB&g`?0}lUYmTMs!>!3XjNK`goXz`PzE?SjN85F+Yk_0tE)X2ecM(66FCi1I1(i}Dm{KW|BeMUhhMsb zTn?qtGkbxu_<<=35k*tH|;TDm(2>No@ z)q^efhYc$nO#^mU74xYZmuCWgYCFAdON0F%*YA7;5Jj2I?|X849fAz`1ygbVoPVSF zaPkZstAQ6z2L4g9`1oNx6$47)g@-7-wzhcqV|4iA54T;QOVdA}q2}Xk#_QR<`W*`l zM21zRwo*0VG<&VHpvm=z%Xj;ac$QXEnFGr?=i($e8aryg@}s%+>K#L@ClO(e&Txkf z<;DY}XK)8f07}rVemR^(zc%{gKfd-seNJR{WmZh`@*@=(?`j7|S;{ZSt64 z^x(eg;8a4H(4t^m5oa1kSR!580w*b{U8QpNZ?IN?oNh|qy^AQWD&6WRd!bVpJ%Xu= zYX8jFNFXPZf!d-p1>?Jc8nWtteb|B6hu;r=4x@CEosF9)1|$6Pbf8Afs}imt%Yh6k z&eFOPi(Q03&0%u~a^*JmlztbJM;cCjQ2JkQU04RO2BWRrhI(K*YK^d}k_E|)5iS?- zu|B#ZCF7J;;Z;-ri>Bdjyrv(W7RxvG!l+-%d1j2o(dBm=D5U>zTZlC7=`(jJ*!>oO zF-GKlHfx3?Ka8!eBRc8MXA5hqwj`+{pWfhsnX~Lkz9)=}m$nuTEi8!cccF(Fp***|?!>Ron?%Ht>uP?OLEkaK2axK5ix*8J#zkNk2Mvo8+T(H8$$v$%GI5IL z)(BQ1BLOwF!fJH4R4TeZcQ=$pxWc+UtP=3K4;$k1dy?ui@?T?)80pC$=a7zeeZY>G zdgpCvS*Uc^E}mw9xJ(U!3tnsaU z6)^;K`e)nz9Lba-*OJXIk+kOp-Yj=?d8C?Zd6?|=f*_Tbdc5-LtlpmW#MZttgwy|g> zxuad>aH~;LUV_$v0Jb$aWM04D8P*~AFXBTZwL;02*SP)$wM3#>*ls8RPWw{2k9O92$iQV?VkI(0 z^on)NqMzOC=~nC(cW$9VuC(}I2ZP@sO`VEJTI(5d8pq9iHf>fqR95_MtQMD@mH)XC zb~fzm6}-{KfqDNTH|>kRXmUPdUop!Jw4^IlNKhF38`)HM#QtPzTTd&pI$iqjT);bG z?r@E+JgUJ4;3Au#V^%1{D)>p;DWle^>tz@xxxHUPDxMS~6BMrkg4*6z2{0bLWtLBK zZGgUjYQl;kr4dGGJSF+ngUU}zn!ykzYcHx1N@2=2fh~GsL}>K{-*!=1RBQ%qmEiVBQ+mmUONAxu26R-X14i zkjQEEh_Ql)S%9yWpzR0ot+Je#J`dq8i7vf8lEvUzJo?m#%G}8J=~UdQu%j4FxaH75 zdB_fc7=82#-S&zlhkkv$#*sxFLA)gmtoF^0FuD&PYqCg|JA>4Fr@jelttGQUFb%D# z5%v(?1QE-27?hS5G`{$aRUA6)RuqEB?HA2`3M;|9FeNOVQy9aXxW1WAK)T#XF1Dw* zUWXuKu)ZElyC;|rq|&+_I3XmeS3I)UiV!m3MOp+dJU1$6rM)r~4`ytRM}EvwVB{~1 zrrRXov3;DJI2{{0mET;-9%Gn8`~NX^kHMKlZNKP)NixyIwr$&**tREjChFLBCbn%) zCbn(c_Q~^}{noor?fv0==~}hw{%}`!S9h;f*Y*2zAY2OMZw^MF7jZjbj;WOEok?#i=bT=Pt3ubm#e{;}I%O{T+X zy$(L$fR~Hs7ng&ns~lz(Q<{Ae@ksdKI8JEDpSfmLCFe^EA$1Fzu$|S1CV(ma=QxIb z_09i-7Q&MD$hC!!K1QqoKVp~HEE3DK;_y+OQBYY?QRh%lZ!VsGWKXW$+M-$I5`?l@k=%JYeq$Vj=pQO_UQNH-rHr15 z5vF7Ms(&Dz5O4TEA!_|&%toxG4z!7hi3W}&Hc4B>Je8$MQ3}nQyDj1V5mCAq&ORA7 z(F@rmO_eJjQjo|)4KxhiUs34)<4N!3HgXWxW-L+sJ0O{t5@X}V{WGX{M4gpkyPP3) z^K~HQPd0gayZ(uC>v#1Hc%-JoFz4&pAlBWAOim!~!O{_OpLi!&-$_&ld|s%@;a8?j zGWVca`JCsoCq((b_j{rbqMna!sS(mwE|1P`es?+dF6K>s-R#S3f(-nw)WTKOe(x$) zajad=MVBsCk+f~j&HeAD!+#JURR89s`l8Xn#sbOWH0VQtTaPo?Arz)Nm^|ylqCQF%B0{@` zaH2CesXv&@dX1=mx#QZ!@WXAjzs(cs#HYp9 z;4yW$wcv^BeylZsgS8k$_+Rj zPWneP4!d_>htTvmN|3@qV%Pef@=$(5An93f_ZYv5Kk_}K?FW@NNrA~tZN};T7*du= zx=Hq4>R!g)pS>)rbgS&s)YFX9Kc^dtE%`&zI2O-j>$eo#CmFM_#TX1x4C8jdQ1`Gy z$mzh2wzS=aNA|AaVyD<6njM2z6QhQ;ChqV5`a6}8&z)uW&}H|EW%p^+X)2dwU%Qbg zHewL;AsI+CGAWt5bu5(b2^dspxtEeQCZZ;ed)Grfp(jzIB(TJn5_=YVFW+O{(5VRV zjuk~!6F`4e)syaWdqQ|q76B}BiVNp{28E$bUv}R>o6HEmK)Shkp;cWi0_odc44wYd zt_gNrT*iDq*0=%IbDW7*OI-$jHL_xRjRP5bOsGD(gwdS|skppYpgcN~gJN5RRd;$e ztky|7MIc4`CbWJ7OZiKZTm{O#)TCQow74_#gnW78vyO}*#fo8GGvGXeNHRosqQX_^ zWye)bXGeLskIFd_Qe!ys-6IAoVKPdeW2)s_qo}1dlTc8tS9NT-9+Rt`Okaeg3tBk% zBt8-yGqJ=@vHC{GBV7p7%y{JO0t2>dMbw?tQ7S#YtqmI)as=hd>cAD3KRNqd)m)~`jb%)LuS;oGkx&x>-i zW1CXh7dIRparT3a$Xr-uk{mvj@Nl-}dqSzQ(R`We)PP!kiP(E-_^i)^nu^9;kh-et z<#j-m$=m35FbG}6`a$PG-Tr79jO_~W8oI&9v(D(zZ%6Th4d_ZA&}QSnC7bFs9xgaI zJydf(k@K)vZR}vZPeg{mj!f#Yd2NyPPxs}Fngva!Im^YktGWIm>CSt+kwUq||D^m) z_gre{PO^L@OiHdg3xN7PW^?g@`O3d@mD2sUU+GU}cjqUnP3}UD&6J5CS&Nj_+{3SGi!$+R zY2@BN*Qf5bZv&xywPqTwX5!YUmdI@s57OKjl*aHDg>61Z>r(G^)JPemhhVP+< zaK|Ar^P%gx2iOGIgI(b;@6YT|b}+&NKae1by!ra?PjKI*XDvu;coAD;hvf>lOC>#>+-Zs2%%oho!ZUK z4t)}0Z^WW1SpS2t1e*#MWRKt=CwuhbGWn8uU7A<4O|&2l@?jK9SilK6dQAIrwUa}v)<2ciG+YX(9)92Uc z_bqXLA>8}#m&8*v70hCO<;ZW8^jz#mp7~Cq3rpz*J64kFAORajN%|G;p zKabABS`WQX^j9}g_v^x0@0vXdR2d)AlWzgH7D8+}uh#N?u-2OAGELY4oxVk-3T6Ll zP*wq4#LsnjiaFBvvVUo3CAG_}vdLs2`2ge~?sC!mmM@9?PqwSP({Ve?mhn6D9|{7< z)Rzm5d1b_Wi+zhwOP)|m?FqoKE?n*=?^pSQvtWkUux4p{+KAqC+wRrpa*sw zTHk2hqJag4vV@%)`p2D1Xe7AcjgoetNxNnZXLkLLwuvCZoj@@FGh~SIG?=J}SX>!v z^&-#w8V~lEk60HB#YRyhCsLZt7-*)P?2+lyP3@NWz~L=W>-D1D44FLtS{Eqbcpad- zz^(<~yZuLJcf%+`sp-JFO}GMDtIRg4bfJ)bB2h1-YuaD-mop&u#NS zk*FCHkTBu-V*1<6FrEi!^kxM#p0&>Lxf3Y2AiwI1+=u#@aev=*V%wghc(UztVodL9y-&A)J4p5UZX7AH0`>~L(GgFSD-#(0 zvSVp_XsN@SNT$vKo&6mu(?{t8axg05x9zoF<@F62lq%-nn6EGG#u+o(G8I=ln!$WzLcCY^7Rr2hNWdpV-pHje<>9V6a= z!KW@`Wvl=)YXf#I{_6{{@`vIN4sG>67e1(0ZYf;9$ry3CJ{L>D4Ef!jk5P*W%k|}! z1f4nHIcXB%zd78MRYNzXc%Azc4)JL9KIj_}6JYhvz%6ojzu$Yp@h`gZ`@N6k(>nA?XoJwS*djrdd*1ok@Bx5- z1mAog8Iw@Trl~Ggdg9|imbcKasR~U^GJ0nh1#?o>%DR)q=EkJ{%hU(P1>KoQ zdKde+^HPwXA))v8rFI0WMxyOURh+(j=P-7TU-yslM(5NKJE62-?`w{&pJ1H?vq3gT zN34IW|*K8;eIDN~h>3H|hBM9dwMn1%@T!9Y{%}CgL(`SDNKldApa4~AF{h6PA2|#;Dxh7MQ*Dt9}abQ-QDi8!9SfVZ)WE#paljLNCPjrgw=P4|mBnAvJ)M zm-5IT$jS^MN1F&@hi5HV-d+mcXx8LS_}0xu@EF#LIGIl;%qrvLcd0VFjcX|H1?om) zY{+{vWmXBwo-YPBgn_Wlb1`^0Fr#tR3YS}L7wi<*y^v?E6~PSk?Ok3bGZw$+QZv{Q zkBDsxmk-Ra5cJSK>*>L#j>vcs@l#SeC)Pp~=)~G*T>k3mpZ{%-x1d{`lhNl> z-QQLo3$zno>IFp?i}IVjKr zE+QNELgvr%L*<)2)np7Z$@;$fqp~$$VJO(B(os+EJX1{tF9D44C(qfnUN|y|R#(rB z_I|ur$&%V}4|?DsyeNy<*~&g)&hasmK)@wi&j%z*p~H}NIn$@^iQ+Gcm!&vt`B*3+ z^J^WP&{{1E4m195-M<+BqqN@?M_~YiE8KEckpj&8*O_;EUePN<-9?`Z>L+-jesN8a zJ73M6p|WpuM>OyO(n}(x>zvi#D~1ZIbvpgYByB^gSENR02hvRMcsmaHkxklY9Z@aV zo9}Q%tHTV0SNv0%N)_#}4mdkvr+V570s?BA=s(SVJzDN(mRKR4Y5d~I_4WiZ4lkDU z-JZwhY{6!B4E)=W!B)f&W5a9sp|Y*i)tKY5U4^^8l>%>TsUK)p`itzhsn)Mi3myj8 zZyZ;0>8#?Vr8pmp#;==SPCHqi-q@vjtnCz{`w~v68zbJ}28eDOq9vl={)C2H{lTmj zMMbyNT(h=70a^~1o&ILmnh&50RE;(F?lsjMw{GCno z^qu-h-h!oi-BPbGYQWpdyTZk?o1e3*;Eg#W%95yPJ~ev4xhAP;e;Ii2TJ4&MClgpG z!n1RJFLnF)gdXByAj}du`vlqI(lMloh?bMPvTGnxHF^7nBN zG3INuf4+vb-7fl*?;52>WX1Bx4ZsI(qQy4!J_swd2fQ=rKkQ?}b7agZ2 z6VD*}czvLQEt3dbiF}?O$Gj4J+J>`_vf`R28tJ9d+g*yht(G=s2_!TI) z<>>wz1Ambr$Xj1rT6la`=Z{X>@n~uY;$@$YWG{?3v#2v227J&T^3z~GWtVD~7zF?R z9CjstO|mwx(`QkKxhnz7r!@;ewIeR=^xEGq9m+t z7oX7Gu{yBl0-f|9^4CsnuR!G}AS}^5Kl`~@S9nObH0kf?pDc4u-q0)!e6^X&JRhE| zeKxFZTs*~$o}VF?ui*={Y-T;sdXj@PYfVRAekjM<5nb|{zG@p=Ob%AQnI$_!zcfRP z_!%jOYy{kZoqwvZgSn`jh*z|)xgw9)#0YUEyAdk@c1^fF&fM$0cebv0{mshb8eL$c zZC-*6nFdBLlhaO~ZDg-HU+|#0@DJOZYZ0s+xIlthG^R~0s~e))OAfAIx;IYyVWoEa zDM;tmf3s!F4Ul%2XusHgF-=?rZ_4I`E5iO=CKrPf!Ds#xl}Lv93O(u-IP}{6LdNg= zZxA=YDeCLba}=gd^s8U=wK0xlCsQS8#l4Q){lN{G{nARZW?r`8FYqTl*Chz&1w1vI z4o318Suzf5c3bN4C@*PDd%kGWMNg>5G>x!F#L?62(Qh0_h-}p7c=(mD{Gs-oUQ%Z&nA0xn~oX(HYuF}9P7mEpz22JcqF*6!9$S;7Gh=wKz@BAf07rzX%6>24$kN|NYjmL~XpNx?h@6*=9pP|f5n;oqeo4qWhh2(a37*-@l>ZZA6hkiPKI#-Q&`O==z~EZ zLL<9&^UG^*V1Km-@8bo?7f!9USo))Q&kFD5&*P)sTlsI8ttW|yfhb0!7$YFss5kWb zP1YK~xH*CtqLB=^ku^fNq()g{zxQ>&@G{<%!HX*CcNp(GY?|7LgJ^j{RSMa!t&tP# zn^$_g68wcx6!9Ek6f7+cZvp_#`jlFi3mw`2_2(d6|$= z3iLWt7D;;3y)wppi2W+k{3u_(Q2X}MUAl^NL{t2dDwvi+OyCY*q)$9mEf>WdlzhlE z&OBkL@m^pn>?|C~@{(t+oyGiu&iHD*Y|Fd(hFJtytN$S8XMX`uyc5pWqbzZ2r{B67 z&AOuomb7iT=ex*8pTw_qcBu+T`?#LGP-mx$r|M<}PY_Z+AcY*J562u7`3Ex)UJfXZ zb&L1Fycq06)0iMYTwDZKjS6l|8C8bX0*a5zf`suu;PH`f=!5F2AxBg>R*2k$3F>ep zgK1bqU#*gkOTGlr^8^PY0lO@(@2Jn3p_BwYCeL6;6nuWJ*Mf=7OCd`sJ#GnW(!Tcaj&bNHKj4P# z%R?Np27P;fS_({r zL*<+|X#!bvXJXw;i_E*Am8JJTR_v^N@1521I(3bg6)V|;ut+STxHL8csE#KWE;(k| zAA*sTmm(f4xk)VuI1X7-RNsOh%$TE z`2yQHpS9+Pp-)8i178=wYP`+jY35WdA~X!FeTN;keh9hY``=4FEl zq<^F{vF;dhE8mlNzMapeGiKT(O;30d{JsiJ<7xAz#yM1`=TI`lQS#_c+mLP$ zZvymQ8{VIjvbn#+_A`xu$%?x>CkC|_SQa{KzLZ-Q%T*`l;u)0@TF;O5rA#cR(V7_o zF2JB~PpOn_U4 z=aV8=FJ_X+dip`~J=51%&y?`q3PnG)1qdNzx+_Gwfj^IXY5yCOU42@-5%x}Ooe6I- ziG%p?*$pxQYBAKq-Yuy*6ZDB}VT#yLEvsWiltip?YdfCL#_(iAu!TZ~XtzbssGMFW~O2%R_Vho`kqXP=)Z-6Xd zf?A3tHp?vZHoS83&G$!@$Y!EJ^7t)BS$J|LI{TCF#0REt8$mjOO|TH46~`pMvleaW z=^ePa%|5v_!D_1C#U)pjuZmim)E~@WTss8kh67`WK160=+7Uvx|AQR& z0)sH^UoT8$hP)pRP&>p&lR)?fQUjqZiqfjqe4ZrDBlAmB3-5*;ifHlCIz}V5cajUw(jquD?uS# z|Dc8IWBbP4ZCLn*^rA@TbE%OV6Ztu5SGR%BM9b{pv$Dlzc=+pm>*wAsXsFOZ>t))& z2j)u7kY}BPT%$%8K^;6#v{wmwMkVTXuSFt!Wv%+pV;P+3WJ}nc{8EYM?BQzJ`aw;9xapd1LwN!`b@dC0aUdILJU`&Z9J-sgO#Co{*NTI;a|C_h6ysm~{E z-_|svv+0^wot|6YnG00w;-GT$mh)|;I-~a~kT`O@I%5b$sL6KngF9WL_Cz=I!ugMu zW9_(ZQ(OT+znUZ3t?cNh8g@%(mU#+>k&tm&^)z?bkGusl5Pc!ZA65&lSlM>+pX<*`evf5zi=l zmVNmT)?O~)%#lf8v0T^@aN9_5)|88==!WwUJ_iy(MK71LiH6u`S4?Xy)J4qI6}ZE( zqYo5+lAvdSNzf>~AXm06mv}2cakGQ8MDNV#z~o?n1qBnTJZ%b@TvM6I@#1lbvUfir zIOO}ii4-e@o~rx*YM5pAo!P`Xs@o9cn6un(kz4l!yA%C5V>?2f;6&(0{1OLZ{JKWD zuk?Fze>mrW;~fb?aZmaE&k*?E#{YkYTirjzKV#Ic;OG8k=72)sjU08qPz#*A;Zgo| z>%lB__cY{u`D#Ri#uc_NHPI&KX_KaouWQakl4X~=KL5CLw7Z;8g4+U>kNzexWJT8f;{3pfjBlW1h z>60jZht6}S6Nwa*5!Z=n!O0DGRjCK5MXrT%P+61unQOzLwzVX(T7@?J(0Z5>v7?gD zS*nNH98=9QW0)V=u(>Q=uP0t_nx3J)7eM``>|!QN(;zJ|{uz7!<6Ls*Y-1Iijny9IH>)d=M4&$g z)9oql9AE!=y%XvdVcVe(Kd?`ruN&bBswRH+<;1;GY%OXjfHU+7JNHK9>5>1V6ByhC zX8L)(!&5%oz}Kj`&bK7J^EG45UHJOPOe^Z~X* z)`v5N7AG1hy7WY|T9Z!hW3O$PZE;BmqzOO~Wn6>5#SnXkU1A(ZylKYq$5{R-Vz_!= zb@qrfZ9_AM&q&{j^t(ml1jy{s|8gNrAlb|L<_C}j zpg;_QnB%?FqW-|#1T<9u4k z>k%4;slo#8(F<-9rAy+SZRwQkotor%U0Urf5Y&TDGFS0a{@A~k~!xT>~;jib$!+ZaY3gR zp5-^BuCQv2?<3e}MxAPjP&alX1a&kyuk9UGZ+UG#CDa1OdoSTP2^}SGQ@cMgTT5Gg z&1CnF%GRpambQ+5%jlF+!;V!@UJmt+V5L#~jOv5N(H2+0nk`B+=G2qRQiQ`9)KQ$d zf!>!4;0F3$Dkvwnk?(?6qaxNRZl!h=`L}4KFywGv`$vt0aFO&8J^SB{WK{Y>mxwNs zR|;Yqq*tiUV@XS;ZlwK13)XGln{w+ZcrR>s5Ijr-jv;preIyP3d;%~X`9jD;qk;U# zED%n-u;iU=(qmXjA8hz;hJ)8sFQUA#4dbK-qCoP)X%WrBpoJ1gDd)?tTrm6Lf0Sq& zZhkVTIwkc}K|fa5RW!&zDlzC?8DGzWB9g`aogdC@JFgm?ZeuvDmRQOQ+~7`J8T$n4 zu@A!Dora2erYXm0kiVPg(4nhsjhC;wtrkkhR1_oB+mjy{y8oCWRzoQiwr(iG=MD*BPB?=5Fp>WkC(RP)q2 z67%b?Gqj}8nx!x_xzQ$7C7A>iOHiYfs{Sx5JwKIZH1hJUqTe%wwsfz3N)LKCft*oa%$%M#{A%&+8J^0l(vIZDvaT)ewQK!Ywv32r5v7d zAy0H?S-ma08dkZs7txk>PakTQl|xxg z@MhuCV1Ugw^h!=ZCs%N@weBN+r|c4!-Ot|>w;fuEe5*%cx@Y6@)_4^fFKYcGoP^I% zoUL=n{|EdD!_3C^Ka@L)g{70Touj3(p|!q=otv$-ouNs}_8U04E;Hxuw8JFj`uh+z8ulzcXcAl3V z6iM-36>e*9QR!!=ZRYT0S_A~i1h}kdDC~C}K1-YBAF7`$i5=$v!Mowp;QV9P=XG{| z^2Z{fVeJDfWXh%An&A>1Op z*sPuw1LPDFFXRTyCkiA>ee~+{pW=zx_j8}$(&z&d$E1LZ&j+{7Px}l;hAjTT>jaqH z(Ka}dv=-CWvOebbn#;dG^1Pw>q#e5~=uv7_$e(VIH&eQShnTFlo7?WO9~Afc!1F<3 zTZ7iVQ2`v3UNkX1aMXRoe@TD!d)a$I>aomo+IyAnBFhD%`2Bi{L33z8M2|kj zo=mY0sA)ANeoUZjXzoLPNS5S3r;oqAAl}8qT~;(HS{M7(qP=(PRynS{tW4+T(W9z% z-6~q(Loo;>!n7-eDWwRcD5QKgip8gDTU#&+sR$! zmqK(JI^Asph(EFdFlo6EMRB6xt>l)8w#G-c!o6U?wjDN$x_Dk?4;58(>8N8`l;K;H z4kfm703q6eU{5D|DCS_vjQ}h#&;DA zvth}+;S`8JG6J*;6J!MAsX6gV--nuzv{;yV7fh=Y^ZB^61wcbgLs3{+ZBuKORU@vj z{>Ur=4NLgM4{^k<2wg&d1j(c+b&00R7}gbha*;ce?068$o{Q9UtM#)dImBBGmRrh< zwVS>LoYOMhZXm~*Jye-Ux?*4rEcM}ug>UMbY`2T~q(+Jb7HY{s+Ye|#-a)YL$156K z9f8{iE!3*~phfa7h}!;+2_2m32o8U$FnpvA+z`X4q}68m=GfgADMS^-Q>66$%HFB; zks;w==2o&YVlzChWuKEoa^tpLcEnccjXG2098h*Nd7u&rQFM$f0ukQo*5VmAPL1nj ze!;@QCKxwJ&?AR{WnHe7m-7rgE=HgAH02YTKXHz#*@c(RC57S zuD2jRjt{m5`7{f&pKjK=fz;TayLg(VA2D^VfWPp@RPkyv-r2lkuYogN!+(PYe6HQ> z8m@1b$7w?E*e^D;@qvdVkul++W-o*AR@BP7MXl$9OS-z3T7VF*P^eK|zd?fRJ`j7g zO@5?bI6M}-8oNz(TwH0lz)WR;eFL1L2$21{|*mXJu*pinmYR)r*p~ z1Jkvse`g=dwY~2rQP)7)Y1c-SSPAOC^Wg;`lA-nC{yik%AA@)(e9+3w+(ffUGXa4If zs9V_f;tftoM;e^Aztaua&|{jaAm>1bNxW?xyS+ca9Ys#>{x9!;POkrQzrX#T6g$^{ zMDYI)#qRyTDE8C;qS!B6AV~gqi9P4PCH7rb*<3K~jpfX={x6Q&r=&8?{q6*6{%;?T zabt)Brw2qwP(UrMR4e;0Zm}n|PG?$}Ju`fYBdb;TiZXZL4n2N%&%CR3Q3RHQ*5ewx zqZX>`d5MF_HhRKa2*GuS3B}*)$bTxqd9n@)pU6i0G?rtw13n`={Bapno=L6*i_HQv zQ)R1=52Sp@{#!MEFA`iu*T`VsFCG|?o|m)=ta~cH6B8kd=vfT~tes=KkN8c+Wjejz zP+T%%Fy&f%=38Z&9@jkxq=KYH;O79cqCfrKvn!6^qxGzn!a+G z&&wLZ_4seF@eG&rJUSoub;yT8W%-YW!7Ut|9j4X;|9&^FZK0#{Z>mQAjsL%#{I5iu z|3=pt#4W9zO&u8$!mwe1|Nc5+=K8PCuKyihKmQNDF18@xbyC%7ckIk9MX@eb%tM3+ z7g+O+*fJRec^LzR6xk$B%mi&(5jx+0CzzW#JbEWOM+5~7(pitWY8Qru^fKVWx)ODQBTcodE4cs;km8Nfdx@iSgT*e_lCLn!3?8ehsoF0fiBLIjJd|%U- z@!oIx*@FB7DFz(|8+mBuW_Rzl)B^zMl-ompg!wq?2G=3IFHKj^Q->IfI^8wE#)AX; zRT&jZtI2LIbymsr${20=@EQLoOinEA1IJv*`3_7=w#z0=TOLr9}>EHs(@kB4Hc4TDM}&(%K<)VNGo5~WZoTj~8* z`d7%f28s{bhP7w0qk9EjK@a3lxg0V*3;e-k0;ZpcRRf#zI}tPAZ75spb!_vegE6)y zX(N*A9IVTxL^NpEDCoff*o7q=z`WNB_p<%!l%VF@DjjFw`;N#A`*C9YtdFGAc4Xf+ z{uA0;GFNReQd7ozMrS`idN4Fgpl4uApT1;_G@xK^RL$TkX3t_B05B5Z{3WlsUFLNq z(;l=ncB@}p27PE{6iub2MU4)>?JL}|E69D~kx?`8OR5_U=yx8SgQrU7I;A+v`{q=Z#R@X1tFfTuxWbid1;B!=0fq1K1^{CK!nut)?)(f2$u>4{*^l zt7nu=%T$5x>66GWFd{(?Po92^5*pOjh_)Dif|>&p@~D9ovF22I^azrE4~OU$;SQZO z%A&r-zQ_KUK9n7|r>{^GD%yQ%2P&edL^H{hpN~@6QrW|^`|A7hsOfc*n&l0YpZO2U zQ02Sx(7!r8%GrZo^j9oi*sfXcu1>|%KLTgBc>!qP&W`a1bwnV>-(9&9Ls8g zbQyH|{{#VGzmu3HG5ZGoF;PVuzvtO^+Y4ioMjNtOBhw;!oa!sT&8U$k%{bYMdO*HD zK>GYn+A&avl|X}HWkRH`nZW4NQxf(<+dvaADZ2xCld7wQa}{-$!Hv*M;Qr-(VsB*z%Bk2-tON1UbTC=onqbO zjI;*ky_9r?#T#Cp7vdGBR8df$67)RdK@`OnK*$%}S#AI!llX2=SB}9!B@yJdrL@Gb zyaj#@ZT;@Ey|{yUoXQ(F-iv0BRuNRVg59E1mGtqK9xP5aDAoCU;cAjmT4IPZ+s~w9 z%2`)n_hci&(IAu)QrUlq>yTJqMhLnV8n_hZA|iqsA7t~v|5?kCqnSqA@K>xZB6kin zR?jB;`)q^6w(U7_i)cHZM;@<6v2K|D8d&z<%CED!_`#w9*4pC8Z*Vw568L`ey|TDQ z&s`#w)SB1H$DzTP_B+tCsc&pj^Tt|{vJnhODHN_3;E7()3qyNy=zP!LW_-A@)99tF zMSE$8L36br=_6a(MEHtXN<}wgPc7Dk+ug!)7fmuO`tz4du2#SR03e+L!&YD%YNXot{Zq^Yx^PYxyON_%xvhhho6L zR<7K$iZKIm!u9NfTaPvR;$GW&Efuk3Q9mIm7iUosQZzeBsq3Q>MLsJ0%rIyc9x~Pf z;#Ah1G2T*1@JVzo7IS2$iSu}t(nA^7J(Da+GL;9~Fck@s(!~%75<2UZKrOgUUS2H> zDZK4%Cf*LekW9>p(O$5)ldX>JI8?wYRHazFBBQyh&5+Z_He{WhmfJeI(j5A%W|_%i zP=T7H%>-%`bIe_Es06=71SA3lCut=7N41=(c!)Brl2L`n!P(psbmM$){q4<(M{>bZ zjpj70G9um|M+$dz%z<(#s$M1PYI5dm4{#1~*TjaZ?$Um640cl7`uuuxxoSW*jODcC zhaK1{Ue-vL;<^e$@v?5OqPqw6FeFhPRn$RYSsjq?!V(hV`egXtH|l0aPLgKcb*x#}yE_l%8Zv704R>O~ zj|=b8J=DBdVZ1Lo&U7yCh-(P3bg1HcQvrLpH@0957r4QD1!yWfaDtiqb9?|RExYcA zmBuBOqwRzDPt5Ra>p;ZpywG>qOwkJ!*Pf|dszrT<8gfT#D~ccQwrgT-xF3_Qt~B&RXhU5T+EHT*>?{QVU}{bbMi0 zTjG?Uoz^(VvGS2@{&P{jy9g>*E7p*Xti0w9-e)VHU&G07aAn#ndA-T$%|)5;1({Se zoKn3_gGih4Jr<*vS5N@&GE#xsMx!#74wHyTd(SS8s@Tn>NV1ZItp7?Nh~@qaecoSZ zSXskFbYzzCC$O-5bMw(JOt->B-9LP6B+jA?j$%Flm5C&f|2z7gm>pBqbbQb9ba8dU zJ*1*Y-91_;cGu9+M$be8rf6#_m&IFt0HHjd z$&rFW%|7p;7AAc!!S^iM4Fj%lFb{I5Gp4(+?Y|nb;1>c+vB`?1&ga zAR=y}%l9a*q|5gSMe`TeTYB3s!;P|FD8X!Sm8*)rm%koGLE?E;b5wp7NCpxs^wuQE z`Uf3OZX|rO*aPgyvi|PYYhhloMx+fGs72%y{~lunoh9X?8=gH1F&V2FZp>^AVHj@9 z8j1qjg=~Ves4WNiy}@e-M*x>};f7Ru-(CP$Oi0|AXB61FVt(`UIVKid!fhNGeeZjP z*lWkKt$Gm%WIPKcsvq+q?OYR$Blh-627Haw208!E8VSJ8qwk3C+sg$^UblI1ZDioq zYEa#vhY)YaO_YAz&J>$XBw!r8jk)F9CAc?RDOoeg!ANbUmE%dX^FWujEk0bbX zU#BGer(TarH|wN7uAtNFcBjhuI5s~FUZ+>HY=KXscNWeMi{*f?bc=A?A;UB&ebTXD zO<}IqAwE6%DjZrYIxSC@OT~x>RhCPY2v_1XJLj>He^SNDsYX%myiX0S>rZeuiRkC< zbyM34MH==0xbXS$@zYTve&gQp$p?zrtNrH6_=_^Gk_c783MpsQ$a@uI_5uUhC1vy* zZXP)N@o)P!L-H#7WrW>J4SU)HTdt(cx}5axg6-_aM1V=VkrI8GSd)~DB^3=7wSC;s zX+@NbZwsLSLX=E1pEE?CMk)kRh8MaqC+u{%Oq?XFL zT^ejCp1fgw-%k9Vnf zDz|O=p{VoQP%NpC!lh_<#3!c^K|t6FK_hnYTqU=E{|-i30p_hT4U8EwUJTXo>8SC{ zHB`;OzyFt;UqAKI8O;6Pi(IwU+WB^>l=Xn|_lD)8RSvVwuGv?$@N@4>ft<3k#(AeL z!Sclv#W+cO%UIttJQy}=EHx5RG1L##XXPFFGy%F6g#<+yXq}Z%n5cxU9YA_oNFE0$ zn*+n)?$&Q+*Mh4Zy_0DxxOv1hZy+QmJoTPItivsK~HY6^`h4VSlX*XcCrVHoxA z-*$e8r^9UEByXdrBSrZZ(f_M{HB&BSROEJU%jyogtMebaecgtvW8cg_4+`)f<>TVC zmi0)UE5uTRzj*2|`+u~j;yFRnNv=h*5bfuK)ItXnUy$J+SRxhiqXBU^d*Ygj+JSUD zodGBQVKS4~FT3|iJ?uuKB${biB$%TosC+sIJ=D0A^2M99lW`=cd-cetRUSU;nnJ{# zoN`!B_D0&mp9?UDbQk~L71-(Bp*h3XeR!?{Acs@ z+zFQEy$nY#u+Hz|>G^eA&Z##?VFNKw>%&Mh<;5>QL*&FPTx1vd6S_pB)}`GI5l|hQ zz8-oTM&FG3T(MALJS>iwpIIUVGTVL;aQ7QRo|zGvK-Ni}0@>qa60uoe0P{x%*+hJD z1Pv*b+LC(3cX*UZ#mVu~9|#PXikwmv%EgBn>dqp6Xns;hp&R>Qg{JwnU{SJyE7Awt zjd>87LGGC4#6B5(3kuJ;2Fn$sjl6eCkq8+qlP(r35-;wfZ(^{gcOJi=gx0g|{`WiR zm3p!eV$|5GAGm+O$v9o-@T{U~A$M(e%J($(sH$D^Jm~jczNWS6fQkUW8@iD)?c8OA zVr#BH++XG}fnBcktv9zdjO^d@mElfdu;pt0e>gkKpFWys0Yia8akrwSxVyW%7I$}d z+fv-UxVyW%ySuymio3(*O>XWVaFfmCWGDM!cPE+5%sJ0f#LCW2=HBdODHptX%D8lxk~hfdg_$f#V@+^uu!RUB%g`NYibrl{Vlm~cu<@t$iQS( zJ;>$2qD&sIaL54{3aD?e7oo!bdSY*|Krie3T0MzES33KA1Z!>&^6biHAh^)l{M?JL z73i)Wo_)gsrzAm^c%X(|wYmbybDF)Lmkwx=M`rOAzwLgzp>KUI)8n~4)AXo%85vSW zQi0dMm}k?ixu37nwfuP5IJKD~WzS-Fe!Bb@o{Phb_C}am2Rwd$)UL)smpxjmNy4R~ zsoq?{HdWnL;3j{@XdAngch*_>uNXq@VNhlWn<;BNAH z7i#ip0HI@%d+B}|y_x*P!OSz(f(2{#dL(~3FGVZd)r?Q)*8A{1mp{Kt&lrb5zlDWV zclT#j^bOh(P})hmnzPRV8_3+-BO>YG5vZc*7{yX*Tv|ZKxWvR?Gg~uxRlBEk)b!2n z!x1ZOb=_l@quGH)jc?W_X3P^^PIdkYNFPJ=aKGimz~ajW{}a;Ekf?u1 z2BoZMr~-W}ap-olqIwX4CtK>e;EXB6$nUB3%9eg2ctKZv>Z#CCH#>dZUXt7mmA{+H ze$R9K(wzR0p5=@|MtmMBzT2_8w&w^{i8I+oL9xjMG34fCv7Dw$meDeEt&35C(R&E1 z8Y_qos7qu`R@;wpV~*1wQ0F96MJjc&Nx|JyzF-iSt5E*gEkYlV_dwSbbC^-5Lj3m! zHJ^?G(Xy+TwP80G{1?x>ZovcCobj&doRUw)PgH{a5481MHrZZC_7c=AZy`94eeORA ztzCx7%vm{;q<@Bmf5Ndf6}(AlMZZWJ)l)sD0-!12QKXbuCqpjwamf*+p@AzDhD?7&0X^fR zfLTt$A}2n%Bky$GYCRK!1}O{`EDU8E6`2F2;6YmoLT>N-NjjjjuYGCdmSt~|o*56` z>vyaX)D|6gW;QvYv9Xe!Xi3pK!~4(x59|Y1AN;@u>ouwSn9t8KZV}F!EcF(WqvNVA zn~$MddRzDFj?4S;g}KVD=gS5-z}8^Ngr_0vfME?yM!oLM?=%pL|6!;p^NmyL5M@}J za5t(q`~~DQLT&SBKZ0{Cqc6iz57?Ua=rvvw`N*FZ?X~uZa{1duJdf+E%$I!cdq{4P z^%!mHo$@3(nNy947|<75J}O|MD!UkW8C320S7DonnnB_Cq#UI*gXHxH2$-ysbe?6B zpWh#aGKrRzDTv;_5Ph~t-v)6?_LwamoA62=iuCDA<1 zBy&p3!5+7bv1GW@+t#9F+7f-m{-IwHt)fTrJ-N7j@Sn@L%O772j3JKhNc{p)rH|m( z`Isg#i$7vT`@x+u^?WX8q3m?$s%xChwDv=#)%a=;%s$<&UR$z25kYHK=m|MIiZ;18 zOy`pB(^>dM=RFVT40nF5g~DPsmit`VG=MmPP6p-V!1J?I3C|&?eVJ>>R+8Mr2!3H_ zUj+6b;8^3>OPmkHA4__7k1&3UkGuOlp7ke74?C}up?RU5;^O!;mC3K?Z#yE>5a5Td zrp6ISERcU8`vr!9j-I*|5_cXY*WtomT?6WhPd8jM?@-aGKiu;q3p!je5alytA>KM?PNpeKbLRFAeIYazKhm!m@6El%atL(<6>*YFSc0T9%;}(Mwo!3 zr=o?i)$15$6$tIqrkhu5q^&T{uQNZ?;8oMG1~DnqxM$1bxW8HI{w~Zs7-*SArC-*% zZsOIeCp39~KxD#tId{^UbE&hn^ioUS#%8z+q9!y`W+t#SH6UYxrndaZe$$dSe%bUm zoowhkZ*Ebx*Y$(}RnWhAnfi3>{_ zjkE=39lg*0z>%dnb*ZNtuN5VG@T}Y_C+Czqo5q@@AnuqF>F5 z&=QCFgF^!m(JFoxYJ^dd`r`$xn^KAlqBxht&NM|QU3}fO$PgJ?KfA9rpssA?J2s?N z2Dn^DF6?{{!^DqXEpC9fgLCmuUBOs9pjN)Bl*US|%I2l{F&|le{nm!v2C5vq!1S&I z7T9VYA1YpSz0L=Yh8;DaiQ=GVrq0(Pz_sl`I0rv)K8B8*g;NWdk+0O{vKvLg{2xIz- zlECFn$^8xG-Ob3ukOa^`#=Bf<$jN;{M=ChZQJ%7t^M zfj>nqfrSzNB}yXMA};jG)8)I9YIW1OS*V}}bmqK4y;tmNqh4#BtsU<>UTdZ>NaZgp z`C65hUbqFHR?L>iFgej>qY;^<-tc#HlPQfJu{Wuuu$X*i_($Vj9r*_G6uq^t!0` zv&V3UZLWR}q86vS*JcC{Hs?u5PUBtWQTaf=kbXI*@MX3*s|UA{u7*wXf;|pVgSCv-TGUG^5pt$9gr$$yqe6{Y!C17G(o*It$k(DS zAx^VEIBu2-KOQpX2QUjBA&Gj$Alo(1_=cM0J0f-`?R!qV(3HqCJt>M{R}`r(h-(EJ zg&d<=vrm~*A#b)NnikSqTp_nVl-Jzi1O1qMxSN=mm`!?l$dXFhE!r&j6;?;-E9~pL zGqBNG)z(L9$OwUoBiHdX?--@_sLa-@-9xFIXrVfD_S)tx1fWzP{;SE=vi&k*%B#-` z@#E}*SH(WeJw^7CGPb_mvHW~lPS2nvQH4h-YaH#`UFcd08&s9sC zq|l_*&jSe6d2tNPU%Qs$N;&Y=s+>cdC;40jsugv*q&jq4Ha?0^{^yxfI{RaskZ>%m{VbzsaSKzkZ>RQj)E`)>_x?I7?#&DN|CHLSa!a=VKnqX_|y` z<6qOFmqlG}fNxclYJ8t=uD%hhcLFy!5;l(Z>qdjhq_`9z9$fya|BC@J30>|J=Z%H1 zPL4!CN5m2QI~oqA^8l>#!A~amX3B%)$W;}-7;NxZEDtzW*Yx~@C*3?E5Gs(VdF|lE zOf{@Qt#jZ(V8O7{`}<#wiYArlJ4?FfBrY|aRuu-6)ij7!T(PDeF~w^6b~S%OGfvr{ zVSitR%6U6k*>8WM5Qd!l+8N0P-))L6P~?x73r0l)W!0a*vdX`Qp=C2+Mag*%w~d20 zoG)e@gG+(y4tmCL{@)ExZ;7vd1#&DDkEYb}Htn|h z>NpPUz@_+EGsl>Gb##vY8yoy(#gu3ao@0E@pa{o41-uDIg85858s{pv* zKF*kVvg{TNw&FuBs)l>aXx0E!eK&=ZC9>~-9U2m25Gqp9a~kpsJ+FnBGy|SLk#eS7 zCi+Yu!Y0Hf>RU}j1dW=tf074jCy;?~T|x8ka&=aoMK&uf#NtmjM|~+<;csWzGBjmH zkn6g-TKjBl8)IkvM;jN4v+k?W_gW3MDx$W$Ki0Z}Y%RaqRRr0ZlxkZZNYVl@*pq7U zRem*sw8sl^!vv1e4zi*h4X_S+>2Qspf5(=I!$5@c#D=%0Uxf0ULkIjeKm&Z1tgDZ% z^0U5+b4j;4Q>r6d8I*m3jd|vT4xbgbv^=;-9m`tcJLcIMJtr^aljww>)HOm|(F=-# z>DXh0SY^qB`DxGH-0M=NQfyP@4hc?&&}nOZxgOfgriGizo2LrvBJTuN^59h8nY{0) zA8a;oGWgs_=u@rCvf50u(Sa_f(+D`L2ivU;Rf^YtyEqUR-pyx8LUX0R_SiQT@1qP9 zfiP;yZr7i4#)^_?jqaK}gyC5Q_-OL^l&GK)Z{wCR-c!0MzwMlC%!D79U1NloIieE@ zzvm8}Ikm~=TE{Vtm?k+GRFL;oony4-U2Ub;T4-%9steAvBbIMG5CN(Y8w>8H0><}> zNxc7hM-~rT$0jFSxcfd%jr@N$N)1;c62QSq|4H*N(YI274HL?`1v^RuJIVn&$`p{{ zpSbmm6uu{9DYDTSx3D93(qzilvYK65;2})g;hcG55M{59c0zlJzP6*m>--4hAy{sn&;KXSG znXnH>ue&%#6SW3ftVP{g^Wn3qhO0D&zJ{p+p>z*&<657#Isf@A7B1SI7g1D5CyIm- zy+;+TLZf`9QKnxdb=vvkP3oh(5IztPKK=`txO%eLTpT(;rftS?$-@FP*;lNQ@bWs| zaPROhb^b{;&SgKir?AM!;%$z&=`fFmovg9At#ODY?{c%R)A07#I~;vic?zy36a?9} zjLe)C6)o5&9+s&dI&d~kn9iD=0=}%<=YvfolybYeXS0H+LJ}X8aEb|7bHk_{aQM@d zy?~I}Fh?5TM-xJjPP&dZ;@fzjlM^_HwzEgi5EPu7t#F7FwA(oLwZF79jQzWWq07%? z$Zb0OZ9Ni$fT>N5{nj1W>UVg#OB&hvyp&%68DO78MwN^{k1s+5#43T!I3{%~45(R)un zU^=p@>b!r!()EG*-d5?;nGmeK@5Pq>C<#H1y0sT^8l4?;3_7F< zwGL7LY6?G2kip2+HtE=DM2tc2d++w1i0-Y^yWW_ZV7%1c0v-Zmms1d z>JD=EBqdS8|1)eyv{r(QGDqHHagn06+uXv+g*g_ zH-0(%F1Wa2RYZ+-ZprEUSPv)W_LwK=954peCmrr#V3Asa^abSWr%fBETQe6?<>kJ8 zc2mh;y*-Nru$?W|wMSQKH=NaAjJefMB>{E5y#Z+{WERO(>WUcjvL(X?^$ZK$GBgpmx|?qhX$|K7 zfg0B&7BsTM57#KS*E6@*4!7^k(0P4vuVkRJpN<%v#go%>t!ToadM%uj?@zBtoQ#NG z|Dx%+BmBjLRd)`C95FDqJv=hPw@82zJu$Vl+nF(iCj-3Eu@eG3p00nda9@$|xsJa8 zmFR#zsMs$WBdXj|ees^Gu|?`dd)<#@PFe(^WXowjGDGH_hN-1~8oqx*U>qJj*{>L9 zm>t5ylt@TNXQBh;>HXDjab|G`O8Mq!U(cdrz|*ED&tgp~6C-0WqFi6BDiw%pV%oPh z2qSVIO{V48VA|l?VB6r^VBBcwuI+B^ZtU*tuIXOEv!ipK>+unc_Jx|WcDY}T)uDK%jpdccOBFVfew?AX5NAqnGo>9VR$Y})=?Ojqcmm5 zU@G93cy^>N)POZYOiREBEdr$P>l|x`aniChdZSgD%9w^sCWqHETlke7%z~J)%zTq6 z1DWk6(3t>#yqKuh;f{Mrev^l36EY<3-(F^i2F@6bE=BwblO0b4NlocBb%jUT=_LCB zv%e(y>ki*z{e!ZO8M^jtGhZ6*bXnJNIYrNAUSGP$W(v9W1@ zl*x66JK00qoL2fvNAFX%rQX>gLBt=!*Xc@y0Y5`k85gA|L#2XSMS|?>f&I7~dGt98 z4xa*!@Ft?F>gM*G@&=h=Dk~ba(PRBwus^Ne#-4a{kvh9m0vRC^d4YGzfpeg&BUwqM%{OwCIr@1=;X>Z>b`P?DqgED zWXWaASiC{B>eEDEWs=^~7HO61&{c2SRc2F9y8nuQ3ChxGDYZL|!nz<}Hx{_EiLZQ~ zV6hvm@qSN)`{FlyYW48u`0o{~YV`w=o4ekAC4I+C+`p#QX)yxd!$XmS@t+wM6uDg1 zGC%I9WF5vWjw5hzYw?iv06DVw<%~K{`mi)D}a`A5!u~A_!M$Nve z(~vEx_y_(=QIb6lTh(P_7((yr`}aW`Ns?})TxVtt_A&`zAufEfy3#pMaf6!XH_=Ix zk!{ijFI6K+P}0-om6Y%EvZ|!i#9C4Gk;&@ycMT()=WqG!t4+~O{8Imlm*hX0tF@CA z?U%c8R#h7Zd2L47;Fl?!c}J7Z|a8j{s3xe{{0hE^VujlOZ%fZm^A*Y&pL z`o+JbBfS~&zPg=@B*Xpy%`LBYxD3I^Rlgfwz6Ap)!H*mtM(>efLB=C1l%UdNG_-Bs8;p=}F3lg~=Wqs>Q;x~N>#)eZ%n6HZ6;Tu8430K0wAG1ahY zk$x@MV{8;%HU#Et;9RM_bh_d3o3m_i;CEP78z;49gmSe%>(O0qLY+?_iTh zsX80FrhSMi(T08%w{g+JVrh(waTKRU^H0Bcd{FR7LJ=|CQ34T=o&e+dtIrA7gUQq8ga!zY;6h(x8Bq zmBXvqgV59rrbmUV^!Riur?wwc9>1hT?w-BBY21y~{_FLK;?(ginltIq(d;$7GIEH4 zvZj8-$?Nh}^obidb7@X*_uwn)69gcLU}UaZHt4XHU&(po7Ps?X_;Tw4dH>dn4z4U4 zu34t(TcH&Gtk`R}u5tS`MktM+$7%o&4@O~3rkewDpELQGi^gH{-rb$xyy&Bzs_FM2 z{xf^6G_^>dR;$9^Az4xH%I|Yen^!52EshyuH|_tfRZuBpiX{Wd7udK-qCzesTw%@< zg|}xBS%m4`AeXL)*}5M~$|MuzcOOoWP8GZuGh^6G$QFY2c52LnU5in87tDZl=S^Y4 z)yP!5uE@A;f%w(%&Sgl5fDTVte8LNgq@FR424QX+6v{8Rz4rBUOPlLwWxr7&pBCqa zw{*;G1yG*@<$d9vWe}c?_&v(yHO0E#Egt)!!@~l9lvx0t7kj^yvW6bkw;oP6*4EOp zeA|cCl-kcS`!>}Aq*u>9?_2=}Jlq zauPq46z`un<99Q)_;K=9zI~#RLh!?zyeClkUd}CqF4`wX8@-g`S{`t*YYLS#o26|g z5$zb|y;ZMt;!|* zd_3y#gd0DrCbusBZ3D0khE~f*=$8RBHOI|vjQY>INwMnIrehilX6{tmgb07fkpD~w zx(^Ic2v=wri!_S8ZGwr9zo*ihB_F>$yZP3M$>!XYb|CE17^!mGcF zjF)U=MyXSdR+?HzqPBb8*JGs>wrH#xvvrN+8AUjGi#Ld1qY6wr1q2gGkbzYQa`DzW z^YMN;{YF=8{he!{^ft9G|R(*j^REZkB95nHI$u--O-w#G3WN*^r z$Tmo=JiB9 ztyhw{>h=qA`ati!K1tEf2Z9zbP(rg4Ag1l7gnn~Bd?_yup9`32ov~%Xr-WN zdLR4LZL=V!v##}1B1cu5;pQ#|9_kp*`0^XD0kFQg8f|(K>tA+3GPy8Nb6(qSwk*$D z6z`bNlE7F4zmeX*%d|WctD2}Q32`Dz26^1soDi2&(&o&LeBO|Y*j`rk4=yQjc=lt8 z*GZShPS`OF{a0LgGXrk)d|MHMTnYCtRFM2U{`KfDK<1`hzcfSi^ZE?Ab=OScg|MBG zteJjBm!L=>>=|1AqMXf1m{sdGUJVk<51o)QESO~X_^3thjJw~W0kc73g~p4S!Y9~HC`tj9HM?2&dK2!Cqgk!H_|1Vp<+SuJd# zkUIN5ToitgBX7nv<&3WvzKQY`TP$a-S;!Xa)#`?XNHTVOn+fSgfNdMAjcF)W?)KMO z+Y1>`Lj5`_c}(B1`})b39cuUu0+jdp`}nR*n+3UwJNj^Q6ggn57MznQtwmCTT)x@$ zutGlb&5~w39>Zszj7pQSJ~dF2rr2GoV3+(kxj=vjMo1U%j`UXX=Sh)l_1kA&LZbK_ z$S%GMa;}$0W0x?q-(}xVdyC-OE{pQ0^;`?Io3g#xmna+ezSe#GB$OPC1tNZOiaQRD zZzZ}?b?GjLdCNFHx+I&HpLj#$)kG5FSP>pDOg?y|7Dh}HD5o~i=E|?-CKx<+q14`E z-+TQEmaAA;Meteen=kO>T9)Oddhins;_*rzORL(((vFrpIIE*tGidOJJ;FC3vb!)f zLfCaTY{Qa5;5ba%COZ``1ey_-ND4^`=aMg5z+mEZi83fQL-%A_k4>A!UeQ+JR2q@o zTRMyX+*F^@->}S=h)~M%ArYu_1~6H`j;^XE%Q}g^W#3P)Ytca!k>Vfp3m((*MtRKk z1U<3cFiP|qrGJ_vgjh=KY34Pue@sOU@Z5GlyQlnojJR@7j)tO-1{z_m7$Tv3l_i5i z*E1hc&y$L}J(`0E6u$3(o4{7KI4}R%a)%s~&1uwcp{cC82)J?Z(TT1Fc_wgnKLM$( zQtLr)8iAjDS+-ql8~^O}5ToXp2cPErv1ONf-~I|@astJ35l)%={)}o}DS?JsWZS{| zG?RpQ^^3@4SK7)gK)2b|uKU_R*cM{Z2cl4Ug{szMSS0gt5^l=c*EVX^~{1_eZs6xRFi zt2X^qmC%E#VCj6E>d&Qi`vci>ehbHvx?CsJQ$E-j5V*zK%@<^`tD>!C#rRrr4|%ZX z!eVuVlCL-&pL7DbsfwnvN;9utCAODt=tb!*=2cUH|Cx{S2ywRemVaT$IZCntk;ox; zL2SrnlSQ-+z^@^v1-cm|b9|*U)2i>_s3d9CS2=2)PtpnHkp2fJPrRt#3<&~=8&4eS zH=cX5ke7CvfNRJC=Cy%JY}MS3jVR$f)!JS#gGkdGh3$0Kn)?zm#h#FR4d=US~_Gr^Nl z&~F26Se=AP1TWuY{0$w@8oqf{U%oh);Gv@ie6hB|OzdF|J`TAM$MMTI$|8xL|*m4>NyMv?>0qmG5To-lgAS70XxFWtZl6l^9~i@J>2NFE$St zx-f^yvl%i*3_EGapYueB7X1-GJ!~=AeLdKU7*kzLISVt z{za98-7N$79$mi*{;Kc1j_th5i^92YcmfU5U1rd) zBCkcK>j3-a!ueyu0wOvN*R5azmidPjC(OL!>RGaD!vQx%9nUut?5#k7jul@>khN#Yxk(q`*Z(^vkA#zycus@;4E0FpC|yS2BKy--D*|XaN`1|*u^4? zT6V7U@1}rr4LA^YA6O7V&Uil9{N1ER?<#j)uGeTkTHI3DQuv8)r#@QVO1W~|L(FOG z8Wuay@f{UJ^we2@)FNMnzh3H}m8b2`#bW&oV#t*1BxV(QN39sR1g2Oi^l}JzknVC4 zouL=G6SKb2CJaZvV7v;|_u2OapR)Um>}UP(DM*JRI3GGb6@SO}l%6x-$d#I%Dk+Tl z5Pc2ES|}&!YU9AnWSneLYOJu&x+86#V)`zou6lD*7`fYFk5IT#w)y3O>YHyT)4+UL zJ{r*dAl^A%j+j+A0~}Sdf!l34b0oboW}L8&X^3Uhy@kExIP)LMFS8ZzEs>a>9AT`6 z65hL6R^j+gc?3o2t>oOaJ%ft+Ag?UfA5!>M#A7?5uq3Qd2C?02!Zom>q}QTXgbHo` z?A$b0$@5?&C^mYgE-P7odo+l9w1nR|F`J;~!mhDmjoB2`k)YXcSeOSF zP*!L9zbw>4Lv=MmgP3-H`3@elO@3i6NYiI}u*l>?gO?Irx6}D4E4KYgLO0|Ne$giQ zq)u9s{{6va9uU8t$bkBNL9# zhF~IV6OTbogAA0(nZ9II@HWnIJL^DkZtxomLSngZT|jqhaIbX^)X8#$O^>GtQe_ui zHxm{v1=Bwq%8X#EQyKUZSQ_d2dTNFY^K$Bl+w#h8=Um~oFLS}~0^GhdThR@BL$iB2 zexR$5K-br0x4h4A2AJ(kt?a%{1@iFu*(RYrZ$W+J;~RZGHRNgoJxabl)Yui`HAf!v z-UQ@abP0IUDeZUOA2CL}z}ZxyeAtm}(W5kfES!sN>uP+r1T)DL^V)Wi_z3IvX<*x7 zS=AH6oxLzfzB6|t3PVx8Yb~~Zne*Jh6U#W`zPq-O|M8}itP z2P}QRE4*!N+H&9myd*A2=##h*WJP59uK|Pnc)2{EB*Zh${4TK=y<5^e9rQ`oLI3Ah zbf1%Y?cedUWC=J3V8vx`|5P)i`2C$)#y5bVO3mJbouTT$F>VACJ<G5#4!b*1MD60vwpGL-hq8}fJAZA5dT&CYnQBJ8eolh z^%dKH+br=_CS+1hwTDZK`Ez}kJVfskbk_0xekb$Hvjr3TuiAVaLnrv%Geou2JC3%2 z2INbE(ek1{1J>Oy^%QW7>nVeTmZo2re!pC^b@Ua+bQq**du}U`WBCGOT<}+2p!Ip8 zG=RGEs5&o<*W~x&bG@C{;72+?4j`!sXD!?{Il#-R%fn8fa)>cyVvYH47AG$v>iLWy zUs&r49)(hcxmRNE-6D~-s+>MQdc88*tF~_?<9)~Rt7Hd^=hz96?fFJxAID^TkEs$@ zH1iv{c^fa$#NK^wbe0v)`)@=`U;oK9+#Rx3>Y0T~#9eGm z3-yn2RtZ(p31)26-!4%tXViWN+aR2iHzvr6g4jV^JwK2gN#^ttNd>*skrmq9DjN<6 z4q5HT;mHLkqfry734ft$3fR&qIkM|>DW1VepB_=W$wBNy3UzK&j%-u0>JpdF+Cck= zc$9pmf1Wl+n9C?7)x6Mr^hYf$rASV-KN!x!!Q);uC*7dc-|&7Wlgvdl11EZS8#sJTH1Y4< zZx9>gZmCpTYy2*S-f{vgr|Pv^;x|sdt!X#%)1QW}qhSa$Ln^;>v{z*G`Ja$?yUYg5 zPyDb4sx-mX0c}Dr+~8~NV!^J_e)I@D6395`1t_yNIyr9{d%+h=2x`88cRy%&U5e>X zw(mdq!K1*7oBz=A0dg3hP+9a(sO)^+S6h}G3rMH+2?v%TsF!=ndoNw_w29})i}$R_ zy$?|PtKHp8RznAK1h*$yL7UeoQseid;Jpj3N1fD)7*l~B2Or!TgsU(=ZkT4(+T+SN z4(KNf#ICAk#X~P|d**Z5Z({4sKputpN{OHk{`3YRzQThru$MPE^CV9@KNPyPHe*vI zUri{t7j>pBe2<|%)CV!hyTUMnIp0=4=_T0}*kZQoz53IKzk)Z!Q)%#3*8?g7S$BaT zu1X-qUgjmE9oGgNa?z#2FA2eV(T3lq%>hYi=fSV3cQ3=wYfbyhLC^DF5(36kn_8nJ zkokv%CG8!3|0RPxl-U3V1YbkAs_J}X zn&_bS*BgnDXkn+{0f*ieRNo2mMc614yyWI&k^e-%X1~&3Umw1AhP)(U=Hh*z1w3sK zdJkRPZnGa(ellf$oKx&O3}c}&TtJ>Z?CMB}<2-%^9KJQWQH^6&24?PRuOkr(c#-4J zCH=rO;s}|KEEZl5SO3=G@l3fNAD>J}EQJjqltg%W;mmci+Q%b$PCcx(8L+MP*Ijve zcKWwRAVQhenR)2c4dguXj+1Z8Ns5FE$*}jrc6K3-oy|R5?5kzJzGkEhd2&U2Ly9dQ z_QH??2oC;@Y6Q7wCVBz|EC}r!MRFo~=_l+FZxAbrFDM+Et3KF(=&>h#HxD^`@$1j= zq^Y1x$M37qpVKPdAE_tEbrLnfSSEtIWi)|y78pUVwif5CKL3Af2_o)ojgx^SIT}H3OzzHiEN8?{?auo_7GR?HEMe#PegQV zKE>?`mxyMX7UsB5f}jBiex&H~Qpe(L1XOzgTn@ z6}})xJ8d{CCB=xfm$+?7~m*!ZavT}4xg*&|Hsvu18_%Adn@!3};U*cu<=e~1)=49(mu3Le@$DR8J z4}?!+{u6bJYwiRJ=icuPfG=Ib*GxRU7fd7r^-GLf6gqbH_}5={!FC@)x`;Pt-&AZa zrneAhnw?LB&x>9l&b;;??AvuA%MP@MRqp@Yhb_a54b#hIcTp2vc4aWtU@T#bTpjFp z?eGtLxcy=Mf#n@ceeO>f9MCG{a}jKC@sQQyQh|sx8HjY$^AQ;p0rY^SfTL3fyX@nA zfr$ni-U3soCLoHUu0E#2=S>CkB!H&q*k*ZNb-o44zHc0hvA51d+B)a`|HHu#QTUW&(IZcd=9OVB|l`d6cnenb4j{~1hdK!+e=bU%zvf^Le4^bV7{4#v?QPR99fEx7;+mD?t%${cawKS5@C{@ zi{s_88%AMUh6V5n!w$&Mq=nizhvV3z=Mam)?A4(&A_9yM4%EB80lhbhqlW#K{e1n5 z?_`vwEE}89Yne07B;?#!+S-G8%y-APo&*~nx*2jqGSi?-r(0`?JL`_IyZySLfA!TO zMV;yzB#LQuV527e5s~6~_~$=p}xALk9Sf`M{bk`4WDVA~PdDklYhT{ZQHQ@%H>Y_SSlg@BIZM z=#G>{up_@*!M`&{+a8|d7U))E0a*XjQ)~=|22cyci13)d;H-#sV^LV4yiecSAng}( zOtLu6&l&Q;2k{yDDwn%8i^8-0nnY@XA-stA*_#%8i@F(rF4Xs2!!D` zp`K|iY)?2`l<&xCD3p+nIDEVzA(ybbto>TkaG(5GarFQQS#{a)>Mlw?7M`C(?ZoYn zH~`0P6PD*?GQ5W?$TxeKkE#dDH1=>8)kkpY^&`J6VjD)blrxUtW|>0RQ{ZaUK1G;|0!+iWdIpj5 zwSx!cjFY%VQDA4YAocM+I zYJQrA?VqGDJB1!z&{WRunIX_<@fasu;$eCt7VU68A`|aG@&;Sq;`#1#zW)B6^(zD? zM73WrAV`71jgReB_1$@|4nS87O!iZc-&`h z*PG~B*{Ndc){MnQ4CFwE*VOO|wa=Q(!_~Y{~p-N#g`&G)^oq5ekJtG=LeY*T~&)`lYe}C9_7bdq3E;Z z&-UZyn9eGw##?4Uv0{lSk&}sPCEN7YC*a|aDs|!(r|___0z`^)xt3_kZ)mKPF0W4w z+}Ldeh|0>nceL?Pk^hU1P|M$cOdNNcexWPc~@m8hockfJI z7`>5KCT z`j*h0H$xXG3~>v5u8e%?JUKI+x!pp+QNhLe@Uk_7WGLx@KdmP0k-pI+9$-O-*H+6mUjcO3r2IaENa23Lehl?;oCaFK? z4dN)AZbk=IMIx=f{Col4Ob#XaAKg2=Stpo|URKd6?c`H7U8pL8grtrn8%A9ZhL!L5 zu%Q@9bY19w7H)!1K4^ysWi%Zq5t`p+Xg+EPy#cihsY$#}ZzuuTQg(4L+bY``&Aa|O1_vV2Fhg-Msj_^QvW4Ntx2LZ8?ANUh8}!W0k+>FXD4HrX zD0Ha!A_Ki5`Ld1hM)Z>41DEBV1Dc+{X~YUzz=EchJV^_x?^M<0UWDr2{o4=XpLApJvnh3Xew{9eoaeDIIXkZRgRBlmHoKb;PO5&8bC<|JJucq}< zT^f4|REOy&!`LZEh(1u^VT0Mx3|}!Oeo89$0t_WHHm-%pxNUW^py z_NesNOBp>E7n2$mga!04kO?7*=ayjloHZ8$;Hh z{*n1m@`F7e6h=o8g1?S;_n%=}pWDtn{RyIBx9SZ+%USSZg{bttI{izu2xN20X@Euj zH~?YjyjLHU)D^SwTRm?wpL+$jm)D8|T~2_$ULF7YYq-XQ&rj@}0OrW^puY3>_d#DS zT)E7q7QJmMFnU2J|AJ4!N5P)G;%|D#icI9<#QhPl)HV}8@`d8UFO!rz;w;-DDDFPW z;t4VKp(?>@l3Osmew;tGOG-$Hx`8yM4W$oP8aC}O$Hf;HW_o15F3QVqPRFh!U3Gn{ zw4f$0hD`1YnyKnYij|`NQC82vJZj37wMkZ`7b*zQW{tgPbCh=;<)hfdMwf=qphft% z;PN7gX`GXZdyGODOc z;xaaL=5Oz+dX9rkf*11jgv*;!kQ?9Sc1W*^N8F5Kj3GgFGaf19D~X&#rVq| z+lzxG5Sxf;?ifvWj_@yTX%5(Q(O5cw6i+#oKmYoMk1YUci@ch%r+iD^CsAmTOO_z7NE8us}VIea(^;JIndC5 zm5os3G^#$}lT8A(rOP*K6ej3z**}3ugUr(u@}g=^N7*fIEt>6*I0?|RCue_&*r;w@ z?HOgv=oVy5BPc>rZFjhLJz79q#D{9yW)c%?CXhoWZV@zxO4(H{y*oKi>M#u82su7@ zdC1#sT-8M}qKWP|H4RfA%JbHY2HkyxZQjX>+Pdr9mRYvH$^DM#K>_pojo;H!CK>H) z{FN?ps~*D>8nb(af)_y1=q4af>LI4BVU6pJM>vtSXQmudh81KVIFPM*yRIM-?;qI` zn*Q-s18*Q79ebtlAPrxj+p5yxl=zV|Hehe9@=7}G_N^AoUJ%zendo)&)>m?y zk1e>TQ6X(GURXI{SoB;*my~yEvxJ>BxfGfwt?M_@DLS0qQt0}4IAVfI5N3Jz&i>#z zRsZNcrscgb{Rnvd><5a^TQVQVl)JGP*i6z@X(c+Sd8@mkAdd}2+G~u8!6@qudiUQ| z{}wM7%*>py_{ay_?paG**E%DPk8zPE zve5Nq)<(w|3zR`aQi&_I-WC?s;>Eg^5$f8d*kW}H)8O^e)#L?LQ7!Q* zB{cg{8J*N31(O2GZCnx=dHWYXp<4A(eZFHSC#H@smNu{u)ZFDZ2jtZ<`Q6Nyv!{7k@Y@sJm4{3)9oe*_0g8x2%}*esxT1J6<(5Ch+s2%P*d2N$&>2Xv~)up>AL&qjw#Y`=M% zwOJGj6a&pJM<)hCBDPu-B#R&K@}U`dJjJ?AD+q(*{eR||pd+T}gTyTif zoIf7imc!t5>k}ZF{;In?vI~zMBC$jSgRo_I4DoTW?^MXT1Rc}pd_Q@r4!M6DE&#T=S!S=AsXplBNLgg|ZpHgzW;h5w z!jvaQKk6Iuth@$}v9!Lv58)Oei+dUDfRBTOa`q7PWiFmcA%)t@<_4wweXWr@3jo-K z;C4{{ig@b^MeITd$u`5LGeW7V{F^M7Mc_M9_*|0CNZ-$;dqY;~7mqO{uizQ)DNn{! z{RQG?sGB$D!dUcX(drg5v(@7ye-?+`TiFM$x&W3~S4fqB(M5bnE;9NFhEY-@(<2(+ ztZJ7!zHUdYt``9f!hm*XE75l}1&{C|Eas=c9*-tsGtWBDru@WgxME+*%3m$b-lT)M zU**Vl;lfNBZIyEh%Q=di4ZmEDaP}07@c^9?k2aLGfB95_0!rT}cK*)p<2;v>9VM6+ zsKQl8ES?{Y`$I~P)UA)!clmLI3YTuz_N54%3IcKt^qi+BD-Mv&-|F!d7~S*y8u(x* zGf^@>-^6>Djxmv!Sv9IOw|^>q@iU%|=DLH4%jK9nZtqqxy5IO(<9Fg{9nJMihYLZV z%XA*e*m;XO!m$X!)xzpXrgdq(5QVaT$hqj@zSs(WJVO}Ub+d=+UvdUCq5amS%DJ+v z(N~$`{UmY)J0$zB?>wyU|6L3GpH|@i@qNd^$-5qx7uCZVH`Of8aK2V z`VD>C;CBXvf{x_6!`$z6JU9AX5ykZh6u9I|g_WJIAYcaY>sS6E58utQeV~l_n$M`y z5yRT_R_mkvdMq{+c%OzcPOPtu6atc~)x4_tpHJMopPQ?Jx9jR*;G=w+;Lrm6{^p}! z@k+W|JPzpKyt#NY+Aph{SuP1C@+ZlcDh&* zaXgP)mMe!oZhX7^Gvyj1q(qa7S5Tr#GUP{U+sl5*q~L~Ar}-E;0!P{cpKDKww7%~j zM3*_4Z_kQb0tX%fU01?s|(z(6~0nk`f5=B%U$VQx&FRapuRT>4%V&UY0160GhygnR)9(pM$pi}@(D&` zs0=ljyc@9y(KokITjJOi|AtwA z6Nz%Rt5X=Ih!DOnj6<-Dr@{gzpmr0aYCQ6wg><)Srsi3p(X5Nw*nqb7k%uLROBkMN z&dhO^l(H*Z`W&2Nrt@sG!nORL)?Tt}kcK$p5-ZxIImIYA+Q1^KBE}uuNI)BLwXF-b zKJxfF+s8gYTnMK>+^>LYNOf!A?O#+vG-wK@7{o`Pui9lG!WxfKd26skv0xIk|;%Que#=aDStY2_x4l572ss70oeq=Xoz z442h|j{izC-c}BhuJKxOtdn|OQw5{*5%p(&b7lt}vEX757pQ42i^Q`}DC#!%ilHL@^C^r>YhH>2@o|)oi6ac8466z` zNT2B#$W~3T&^@aY*26bx^CxN^v{GEKbt+oLFc2t7q6TI79*Lx{Etv$qx8aTUb=uSaz=_N>339G zTaCoyB_i#97(FYBVLNP|6kUhowXHkEs{Hr%NoW7rrX~a#1aX60oSMC(tl=KE(H||bPt33I6ndtj3_#YWKOGXkb zKu|}J1UE_r60WB3#UfER75eSVk%?&+**Bek3c+sqnTPY`@SO*R{2nB3+8R%%vw-r6=kYBo0m{j#o26)vcK8#!fa+PA}&; z+PkP;sHrBxlF~!zDYVMHjgGMYL9OQcXFw;xbQ|a?(Y}&ZX6Pe&(vZ7hn&=2?wkkgS zFSbZUCfut79&pssTHgb>}t4h!!9*7gan*kL#rAl%M6s$Nt$1TeqHQ42gyf1F>BT3+9B^X@DyhHDn#rSR14dXh#Qr?B#jH1@tK*sjSKPDj{(~ z1}p>X`*H0%4*dN0nlZqAg-P-A zWt9q*h~7~sc60IUNhvT)j`NF{*F~5kJD$Qv4Qew*ijG>X z$mh?3Ig{x>4T}FM&Dd1nWpj&u6$kBvVuP>~MA{RLpOUvy4g198OE>Apln??&-Qk3m<=T&@n!pwovLT(^pq}G82usGs6Nn{*aY>ud;kt54#i@R?>`IK+3{ak!W%Fp$BLN#_v-_45Fsy zKhtbr9@`dVUIMTieRHwNJm)!jpdnQ^t$1laK;Q4P*L9I;@R3q=a1Qd70v2PUUaN<@ z0~o!NdtSzT_zT)cEoa*VScV1dPQN<&l#ZbLn6`zN37W=|^b^?8)HW%%h6_AZ3mp|OO7H|34BoGTE|X;YSR>eF zeZxsEUy6FBpWp~Gjg#+~Av_mB_iF5J{bC^q59r8~m2xlgUn4 z0p*?W1wcoC2y6@VUQhYjO`dOhPpQN|mS!rgY}qH87GkJatlU%z=kHQV%Ld7+>9@M+ z!TOzVWhhI1Ik(_*#rLpe;UP1Z^L-(tj0-!bB4Pd~2dxFiWU1u~T^_~3G7sQ%@wZVb z^^zpY>0|yM=|c#LB)saMslwAr;x&E66d)Hf7 zOnhSyZ?8aHost;RH&(5w_qo`iVH~AZy!ToF3BQbs5RzqdpAoC?r;nLlm2{lzqKt#$ z0`d?4va^q1l^Wl=9Jc5|-%P6V_T}>6|E<=2wWmupvuZrMID^%Nxq2+xu3MWrTtlNC zhtOeF-9ZX(i)=eIv0X5VPbYOcKCB$<`q#wCvZyek8~_3*lC)de`!_H%Qr3wGx(%vL zof>biB%jTRWb1Quoz{ih@lX(Uk5}L6Ja{Zm?8g1TqH`3^;a}9UJv-E2z z#`<4_H07G0{&BhFfoKqx8N{}a=~bcxep~v2c0_J^AfTpcMHu&0q$B=0gn#wM3wRtw zcgg2^(Xa6)B36D>uVve;ozfHvN89}j*4i{P01Pg)tPD2V=p0(9Nc63hvwrj71>p(P zBXs$3K{4ij9I|U}qdo??41T;M1JhYlmL&WZeCSHyewLq5;!n@x;6UXW7>i&b` zPQ@WwMr>C@`7Qp$a?rQBRU{~P0YAXb`}~#&c#M=oK6iBr|4P!Soz2{|cYhvD1Fv^q zzGLSndxip5Z#ND=HPI&Uc?G<`TnTxfAK1C&WFX|IGP zq7ZrE63d$LJGR`-IpRP3wdU4M?fr-B=;V9*CmfK?J=>ci)`*up0p>gSsNXWCEDcj@ zw<;$)>t-UD_5PJ-hpAKK<%!^<5TQ<3^q*SORA{1cSVKk5UWUP5vtE^{wo@u8ki61P z)`lN#8<*q%evhqKnL}(0+=DiDsd(g1>Ior4+SA!Io_Z9I2^P!!2Y07E@pm7EE@D8l4l{MMf=f4Y}q3j-yOCW63HvB=^{d zQh_^;W8ERz@x8yT{qX&S78UrB3ds3xV?tR;C;B^Vw|wGysPGcg5@8cm>*07FZVjq!hdschyD2&w zr~GF}E-w}{k#tFbyS_3lK3uP5Gngl%l}aty=El;D^fTjm7=n2PMQ_M9L7K?&w9QYb zpWCN8J#cFiiFKY}H45HucHoBl->~Nol}PsD1-OTI*#RCh^}LA^Ff4+FxbmF8ak8lt zrqa#=1g(5W2x_^ZNe)I$(*ljU8u?sD8T6VrjQ# zPEW{0&=EbnDb47p2$A=CD^4Pz;!PUlbA{02CUSpK4J8=q27#_q452Bjbu+}>#;N@N z(29d&&F^Fp-fMzuGrzQz|Jmok>*X&86M7X?+=ru@{{`YLS*KaqHOXD@aq`Dp_rY4V z3eT|k@1Y0gM%g|Tj@|LJ&Eigd$2iz(`0XYYVsi2OeR|8|hYhDQE`o7~(CR5f zR7Cmih*8xbO~|VH+ohP+@SnOWU(aWA6^vTgcNGknh+(~Di+hpZV&-0JTVpF) z+UTM`eQX#DBC0PmEWjVqEJ%8eWb;W1S{*LtReW)cR}kA@NKlM>Ker<6J6D3l>W=X; zV4$#FrJu!|+dRGI_C26xdr=VYd_C_-(FU<87E{WC><^begOYr4=7T3!h;1Fa8Nx#D z9$$;L0mvwyA)EOQ@u6nIz1NRd_Xmi*?w5Yn6{O6m_+ZQPp?s$g1b*k6Tav4VSN&qre7V(>W1I=(8Ek{emt0yo+w$E8qhyA^1s~PD4}ma2-*wvy&xUR6O0{3+V1#JeX3qUpc6We|i!I z!nfz^`jYgsr-&|`n0Da{rCb=CkSn)apk{CfGb^WQs2Xm^r@kL_=2VTEY1<~VyLMv6 z|9NCU<-Isg9LPJXry0$+`pgpR0CexQhlvw8Wye;6xC%%ccPz7r&G<{I9k+R|SQxG?Kyrk{9G>HX0k(oa^E`aVzcUKW<% zi0@Os+bZ22<#JrrroYjDmi=oeYV~{Bx*u487iJTDj}ABlxPu2bjORSi14ZpBcBo*x z;yJ=f-0S02NiVb9FzgVJm~KUYh@r&uDkINp8div73L3lQhBg#HkamZ`I~cI`xg}Ky)SAj5s+W+(lKsiXI1~SgN?6_ep6K<6_(4=GCwR)X|V&Aq~dn!ujEYD3uT^h#~G?$4O&>yAdpemC; zt;?s!h~sMHk`4lvD`=ZWn=_*7pHh?oDv?GM2rn})=UiC`qQBvjng%YvNuQDy(OK=x&#*iSM@M_yTA_;I2q z6OK|x$Mn~`$i^pS9`UFI3cQJ z_swZlx}{-dem>1PtG=Y{NqWD|&ptY4yK$$-h%an=qUU!@ETWI(p&wjDHSeB7v+A@# z2tWTa`JJBzsV*hZT!Fw`E4$9162EVNcF4sincf+c<59cc{X;rYo`a|t6HkKZFwVP;dS%1EdntW! z&_T}TmhY<^Ns21Y^4|GCTS{@9?(^}Z{|kgyg#GIO1}1V5GpCTEf)J$0je*fXnHrl~ zTG?Ct$Ec>#WBo5Hk?DUD)hx{4If+@gnE(G9)vT=lhf&SV$^PF&^`w@z0~!l*QzB=- zD8&6f!EfZMt38zY3`cT#UGg!oKi1{ex|YJbSFDeeH}+2XQVA?C{tD&16CIZ^d)lxec=Zr{b}0^OI)HbI7m;5F^r1Q7)!nYmn=CbuWyKD)R?jY z##HZ57IRY&wR95MlvPh~N4gS&0R#4(Kh6csC*;l}b3az9G-;s9BK12W@F{XLs`U8z z*q!1dNUgCPS3IEYm@pGX#La?AxQJ*YQBdfA6PE$b+kpf0aJBTF>26i~IyrLu>4!p6 zSc^X9V-d$r3EmTyH+J;TAqx}8BXIGHj!}lX_#+_4Yl&xi9a!_l-?7& zf;p^jG$Tp$dy>>MGCP-|0eR>(eN?4=fH20-Fnn$9#e=08p7sh_LHQ&KVI8@X=P}d<H z9jm|uQHoElb zL)bEGb3-^Oaa!z76WmKv&eO!d?em(AN)bK3EuQtZ;QZWri*@4lyC+%jH2`GGrZ(PG?l)Cc)SO&yeK|L?r$-HW21_n^oUlOo1ozl(CzIUXWl%u?kmMW#jC z4uk>d6LrZs>jEe0D&O8Xj)k6bAH)(Rjj1_id?Aep8+HEhK>?wr%rOgvsrhYf zA!Uv0dJ;+H(&|+h%4RYn=kUT=+s1`J5{KTgqv{~u zmPSp#5uYA@9u8WwYrGbTaYlnnpKn;3Q4Nr9^oOzF5`nsl$MNEKD^DKr-3VyKF=PS< z>Mw~`jd1Mp!1k2;$)x-p40<BAU<3?!aecYJ(l3~k0_RL6NHQ} z3pCb51SJ(zx>nkmmfxa@$NB3PM-nUOl2?}k;jW?#?d)X^tZmfuDDoK1Jd-R>lc4Gb zNT-y3`|0p+SX1GUYGx0TMS0zVL4L?IX|`zC!s6hWQb4>Pi9vHSGzG`Ppc)rJK&e<{ z2EHowpMeu>If_4+3LAfz{l`MN;(C3#B>CT=ptdaU{`0#V9rl0y?f&aNiBBxWdUH9ng|3f#hS>MnWSC#CL`4 zA8&|hAX!`PFiIO{ffi5xt`K-~?Rl74bvW+*C4xp0>d%PH&-qw*^1FaqSCe+XF=tWu zZumKs9yvO7aC}6#*Afq=iUmqa1~-}bKvjzrB;Wp)t!$OCPl)TIPF2E}9nq(5Ta>1l zbe4Il>VdUV95*Sh6D^`BQX*Z)ki!h$ir(|?a*VB={Q$4vAWaBe`~^`tnT&SXtoPio zmv8p=IUGUM4pAA8sosi33!SB$0|yE!y??QQ(lS@-aS`$>;53+`eFrC(s0U=eak z)e?%g-0|yM$ifW4e=RY9WBXtD?f>VZ|JRHBe@Ac6|2OpZ>B2{v>cloH7SW~;HL_?$ z*iD;U3j9Cdttuo21Ud)`hN}xYmXI=v3p6qtF{{|HnVzwPlK}yiL@&X?74e?m$!Cu0 z({CZa0TA^hV&cA}L?2#4Qi zv(VX~z090{D29x&`@XKcD8j}8A8I=j{IB~~q*X#O;bZqRMh^r_#he|a`LFSSmsG;2 zlRg~hDlNX+GbB@%PVC&X`SC-AH2Tfw^QUj+oBj8Z+zD9IcZ<9!An#QGQ*z9-F0-A% z%$n$vsVA-9$J4DrDDj_LQ{*^QWXKWk@aM?B`-usQf057~xQY#Hd+v{k;9o8+o12Gsb4mjqw}JY%Yl0}Fb2bQSbeTFSpM{x zFZjuQQM(yhXA+Jg0{8v^3Nx0yvFLZrKJaIyHjuV{H{>%x$2k7&&BWn^Q>YLQeBV=P2&Yzls=ZEj)euU8Q{t*H#bz2@cSy=MF`dyg ztg~6jy12rqro|?xpi3~z^4f6CA=NoipX;+OZ5Q>!^gSe2^!Nqa>NV(ll(@DeM`jep zDvWaw)F6aWFD5Ur=Ri9#aMJ~q^9dqbR}3EFDCDiRN|~imHy6Unm>CDY4u#ZHKN04D zYXP_NaBlZl`&TlRdY#If{8PtM@3o!>XY17@6|A|3;~DC%j?k98J#^AYcA3_Vw>_nE z3bAfF(4f!Kxam5IOEi<|Q{gnKTX|OG1k$6>yZ%=D-vNM^`<&UY5tq@5z&I(S@!BW% z-hgLwukCLg0o_PDWR$GD&O1y~x69aW2%lA^%{3v*|G?YwVGYE(nB}8$-A4jmhHszf zB(fuDw>O%^^vJ#x?>a4pTz=gFeVToOgR!Yhl!H|!Nm^u6g9eqTOFItD&zv;01hL<_ zru6sQ4-kM`OplEBOm97>k|o`q+f^ha&Lj?>*x>uq`;C*Vlh=tYVn1i>LFJJk+VhVe z?~{EmU3fJ1yd9(eyjx|Bu_;qg#~+k++O`6^kz}p@l@>AAU0=x;{mQR!Z;)^2=q4T0 z1c6l|4#r%cjF?3*cR{Xw1R|Z+O^z~Z;_b(nU?xB*mrw;pahPVPW|(_~>#oi2+pgrU z^bORz8F@w`-GM9&4GlsprHrNwl1z?F&KTOh=0SeAd|G~m8gAlOzMJZ_s$>b-g3bfS z1I+`&IC;U8V2}MTT%GsY{@tIoIW2hh7-NN@*TL72vv*>tw1^n7DnSgkjsNjtVGjn! zxPd-yTWtIkRIWuPSiHc5KJHy5j?^fBZ4rwPt8l~BaN(qTC@#v2es?dhoDMCgnzKL$ zbg1(m#A+rPOrco?1<$k{biN~MR%q|)b+yR6lzf!YLbT6WF4=sDw*zlG&AZ-i4Ck1y zAj=B6i=+fdrn+%Vk>>i#PbQFJ#@H&#?!b*{3y3+~7sRk5L3Qv}P3df*BVA%W#Wl@; zz}qbER+TdF^IjtmunFKGvkNqh>gtnKJx9GKw+{UaO6q51`U@F}F4X?}bcG>&0&^4X0$(k%=OjU{E(4P0DAC&R58p^{(j5pM=@S;>$Or315PKhQAN9^<{SlUA zduY%O-1`S6+R^BA5AF#QGi(_vV8a>`+a?>^zX_Mp^Wq4;b!%g5oVyHtaAMpSEeALe zc3p)qf4|eBOj$9KIQ)){__mFr9z1G|dejpAfwT35n@Xt{+bWWooE^63h!-saD|sff zFUVQ=8i}((r|z@as(y4|aX!;&BkqwhmBr&`X_!oJr>T-jO-nyA9dUFB{1~qL6X!^c zef`!!IC=4t=bWQlT>XbMI(Fg%-LW8zK}xC`6$LCLqOIbI|7a>ptN2hs8<+WmV7r{Y zH!2gRVEYePSp^66?}aIH>?uEVI$+CIQwHBK2F?twbdep5_9UbGpBa4Mtan|9e|b~K zRPQ-a4c*<=h;$hKlGyr10A!$FX)XaXk?bFo4}-%OHEXZv_i5%fhLF6rZjq}j8d$Wc z%>hAErhKG#eGCJ9cSXr6Q*A4M4V`I*x7yRM7=n-Hvv6X}7fX!@#9Sb9m# zthn+Qqxiv=-OuSWz#i=FX<0q2HRV&(#dUu=T-7Ru^dOnVmX%?X=-n|`d;b9kDWb*r zNW-BQ-%y0A?KQ+C*?5U0L_jR@>8|STa)J}YvjYF|_e!XWv{|r-9YchE9amK+3G+qm zH#wdBhXU_1Ln{D!vqrd5?d={K5ILMeTOd2Q6muhz!g>aECZSNdlrd&RjcBtmuO zU;EEaH@-HM0FEi|uX}Bp58s$WFNun&C5y~d3Tze<*?spoYAQCYq5&?+EVaNdn#_`7 zjuOlQne6`EkFy48um_tc2Bxd6iuqV>g~nfwMd>q>ZP}ysb4Qs;0_9%O?4)kk;s@I^ zdrK9yS*}RZ@(zx%D8z&P$4(wM0EyNhjV7B;iP21j zQ~?GXn7F2Jhs87yuu+6H!s({FfzQsK?eRO(p5TL2#RcbAWW58QNVxAnpUxNDW$}@S zv61HYmR@X-v_{9)tT7Xc(uWwi+xG~+?o&GcM8?7cq(&WacU8z>CA8uRY2@O!z!9e8 zSHd^o*{ZE)8Xz^g;5b_7v5WXlA(sY;txfa*^g${ ziew6KMe!&5jd2VN03@mw>B`ge_}OQ?npU8hcr^Yb?rr&L;^CoPz8Gn-qV+9ptI&Dw zrhYXi`DY?DMY>U=6?})r>2JG$5$1{zgxx{Lf)P)7v>~JYH_zq_$>@3BEWTUyF}r#I z#&o?(FJ;@a&*q*b45)lPq5GyN?gR=P2q!kC|82;mNO(EIH0<==dd`EIC0j9mZ1(ZG zG+_OlF3#Be;Q6py_3u0%#0}K$!^fYlQ~Qz4tcV_5m*?LDD-=3*S0f~r?k>y4Eyt@; z?gL5?LU^_gb7*R+e+Swb=vHC;WRDBjG>`2TnV}6AnX3wO0)_m(42R&)q+1Q@Bel8kF-ZUM0yvYH^#! zRLjOhj2LG1Q9(0P-j(jh;W^pXL%5HfxN+fk4!01vd0nx{h#uA_hRps<{a5-eu|z-C zs4B-4bb?lzJGbH`Czn^gCakHRe}?!Qn%OMkP^wOF_yk_N_InOicmXLJiv|)GHIzRh zlmS5^6oHe9!SMK*l1~LL6s=9j?H*ND4;9UQfuduu$NrhxfW78%9PWYNq2u4A+jJt2 z!?Xt(zCVMpKODJNff!J>&}&j)J*~ggIKECs%g*q^cZEV%-An8E2oIG@aF)uZ!9m1G zWZ%DH<<#~Kear0kN}RN|CQP>q=1R3&waTUP)uI7nH(f_%huv2>@9Mh#DMJ*wL|0|8 zasS8@jUa-GWgdHyrqaylSWlTY#HWdxKaz5!jMmbD=X1v|J(qsdJh^E=L`Whz+;oBK z-{gX#M})#J#m|+SX+w@Jnn^!{FIY2taq39f6j|%Sn{CPKFOlf{Jo%)c6s1pkv#v*q z(ccJJA)0@g|K|oHj}&`C2IbglT2u4xR;QUMI?riO=Q&-?jjp2`3Ryzny^Z2%3D)6z zSk~LXfd0uu`-#gdlHw+C$L2-+k>L=X&(C*umPQvgCMY(`8>MCu;I8Up=Q;hdFtmmI zyin8)(H%}xTx`puH5b>8cda0b@gy~yyZQA1ACHwm%RV!)W46E+ zdGLII|0fq2V`LP_Yv*Rc+8lNIyLQM1JzaqB&ww6&WKAd>dgR%WM6l(VRaGr2NgDTq zT|ehaWL(kG{_oBMY)5b6!Uf8N8)EUfK@KzJu%gHPdDgS#2L$+^hiVn?eV~k8CvKjw zaR^V=l~+gNt@xw0+u(&6Dly*L>~PXM^I% ztcgxjaz0J>H2$zMA3P>}C0G9peulYnqNa&5K#$ot@MlDjVJ7u>Vg@&GvOU-f91l}va0x6n68o9y5lCHX{`a^BeDj6Zu46K1hD-XH;NIJ{!M6g5LbCAXgx8Jp> z5uhGnmr}wTZLXC)>cu648&IR&48^`SBNJ3ccQo(!BnK5S#I-UK^6er}si z^Rtm0jZ%53%Ug9^zU0qYYkrqqnu+?mQn0q zRM4)Iu;Nk;6a6D%eIZ=lKF8Qsks6rPv&7-5jr_^TIWvwkGe; zEgQq-nxBgkji0X_Oup^UNh@>32HlJwD{Ir{%LqGOnhW~fKN7x#jc*!Ns|;&NyWR?A z0e6M0UqDFa*2Ze;=Ox-K=AV&THV>v-J1Q?vB(CqpDvE!fxiA#CGK1*etru3fyT>xK z{r1;MC+^wzHUlN%xUnhs#?d7YQtVV z2Th{mbLf|E$5pp_8rv9cs4B2087k#5j-QoPIm0zzpDO?mxuk?xV!vrs2_o@5!cU>w z^$~0ta;~bSAQRTPgvZ-t-o*AyM0u56>&8^26ZEuO%S2T~e7D`&o!Vx=*T4V;wQrqC zBX(OdE8}#7?~h4;60`KhV@iZgdcM`G~!J zd3N!9S(r+L)Kob~=j|6$AqfXwTHKBgG+AtE*}PFsf~XJo#{Q%jUKk5(sf8*&uIO1J zG2`1&AuvU>+H(IEnvf-(xJ)j{Or0APLddT<8l#ADj27<;;zF#X)l-q5CQBiqoSPaC z%FQr@lB^yxNbqUCV%Z|Xci{a@H>-W^tC}sbydZb4yAuCRN?JVQ-Q))>9eE8NJhprL zrmTgXQy%0;7wF;5WIML1f|$FUS~U-Bl4}I+QXJ%6dQe(BHx{nir5AjFs=_!Ip_(Hv z_K7V2pxtqGx)NB#SU+df^Wb-DbNfPf9_7zy%^Wnaj$4&_y>=>jAdUqi(l$GZO^n<3 ziy&t7S@v!~v(`zFm-Ph#Md9O=~(lJR;KRcK+q6^em`YfF&=L^FAlre8ne+Z$>?<`Rzc7BN29& zP^~-WuJ(ovJFbOW2a{FBHeOkn73bjO5vIG1i{)lE(>b=tWKJGH9mYvZJ~MB%B}LQL zkX9HQ%~Y5-soQ_Y@19HU&y~v8S?8PiDS|tr7Ao6-%&906%y9FnTW!OX+MwO_-6&h3Bg?8UrbEI zr{f-PGqL3lKRDOkXBJ={bHRRynzz&?dzRtr>NvPHD@oXLQctzza4PV=Xbi_8=rXyW zTc}*6ofYiHb9MGqBq#H=H*FKGKMYwbVicJ!#jK@s*HtM0-?=IyjCKTw0oSmekXOU7iE=X8M5CG)Eci!*I|w@Qt8*ss+Q9Ewrr0;<@%MBKx}kuUM37~*vt(>x;tN? zRDZq6&z4dxZ6-YyDRRC)NIyb&o%r4^NA{_s^{NAC{-DR?k6U0Yb80a^IVysq8cp{P z?_4B(?Ho=$F48^RcjcVySB~zI&}|eXJ=(HbuUGGmRQ}a`#%*hM@}MXP9PQU0x+E5_ zrDbMm9^tx5`jVjt^dFuP#3s%C2B%wP=u1T;==ST$!Yylf0(MrNQF>(|PDIb2KUWSj&nT zqkEREVTod&B|rHX{gt z11lmSL@EFkG15Ouky{ypz;(CGv`luGT&-`eOGYBKQiH7C`B~}K>z!(Nmmm}OIgA=s zg%Rv(Mmz1tEdan}&&7k(YTq%w{Q&l`>W?bgD12_S+-IpAx_=y;<8W@WeatTKLsjB6 zdXFb8G5N?%Si4VGL|yr|IK4WoCh-H^o`#UvHXRq*9G*krc$5m-jk_Tee_i{D^xZxd z844War@d*J;V^Gpij)Kl7BNBuh;t!eoBs$zjXP1>hr-1@m1y=0iHfLBT=+AaFD)!oGQb8~b#% zHpO8}VQ@`Jwx4E0Wgn)qGCqO+8Pk*Q_2#Sxk4t_Hx2ezDqf77iJs4*J?U!pw|GzI8 zbJKO>U*3sPD^AY~k9b$Vs6QqzrV9iO>tiHbS^Y9&+#<5LrAqDPAbftqN>9|Ta;ze* zf@jb}yG4jgRFc@y;%%jOr6kbM0i`Wvv_IgHnR-W>2iBN}9zcUI)o6gu#y-n1y2f#( zQsi^*#Q%45>K9x{tuw|Vw*OVCa65b(x|7EvUZK5>$S?5pRdQ5CtMfzeT%?lh@`uwr zEG735ccHU&AGKo!f$HdL8Ku3&{3*9OuHw%W<|?E_LAGa0h+ntJ5WhCwGaXM`kJ8aS z9Om--o>f1B^yL9jWkI}@R`pLE-w3y!mHyIpN=gp$K&cilp_<_Wy;5Sj!|4Ys>cXFI zO8+!*J&e;dY*Hi`1ey6UQ`jMj!?C4M&2gK-##{XwgLU||0Z~WlU+B@n(ZTsF+%Sc= z+;wVnTPR(;PShkA5VB|(y<-Ep*ST8V>%I^HXr>H%e5L^Cp6Oow@Sy>AH#1Pg$Ir~` zB)_}3Tp89Kyd^7kZXE2rTbZSuB&4BITp>eqj+8RSZzhc#K=DT>9vaF_=~m284U-z= zr?En`E;@Y>cNx3&)UWx>8|ulHghSf&>mVx)?>3kbGIDN@eJ;ZU0sYB&vQlTOVDBTi znO`S$DtQ1~%(O3I&Ig~sn@3nn*2)WRZAfXCeje?kRQqAlY=@e)A|ijDV&Uk@bNbGw zpNvBZD&N8C`09A$5UD@JosIUzJA}Bf9@n5uL{K1YqF?IVBJ8bmy zf6kRpckYMGtRyRICYd#BGRgaU9$-TtOW)*+Q&%@AmC3Eu-~oJb1t=9U)w!Co>}U_SdHK@aTiJ3nzNy3jdynP z+jHw^{qJP^0c!0R-g}O7B%J;iHP0k zrj@SPEnh79M?;c3x~e78LjZI*bF-%7~3z5Zh(W-j~daMYK|>=J~!ngEpAODA_g652eVSkr4sQ?8!URHW3Pk7PO6L@ z622NQ#?3lnM}W>dHU;|NT}9~Lo3VI#^WeX2QVBI%}?);gz&lSGiF8N7qwA8iCDl zSFMf$My=yNhe#bA;D%f6tJUsJj}BQSBaM1`{;#}Wb?quW65N1rED5uX`bW9Dy@K98 z&1_F3V_ijFRAk^@qyQW+@2j&T4G(69K&QZq0AOO}GEHz;NGzYZFXuhaUDnMSGNTkB z2}e!c5_wqvvAzQygZt+YYji)Q;$}Gfg?v-RdFuJpxhnG$&IUM!A_L7Zy5FYwf@Y&R zC&$aP(UlMR>m;G1n^Rwi4@~dC$i#>mnqe>R*IrWI88u>{a>sbI;xbQE3`|jSgNHp; z6~JsPD{bocnsf72bl*~^NppE)^dJGP(C8ue9VUoP(KpU8**XUgRrN_b^JVmHnK=a%$hTu2$Lg%oiB}_JRH8rm`@N0Y!M4IZ`aLELLg% zsr?>8l3`y^APacy3TO(CVQXXY-AiV`0|*6&fV(y#b{;1O?|DFNWX4`sWNvq7b)^0` z9*a&==E7Ks6u|h#zN=v6BX9ZwKO-=LE_h|8E?c<#&DL9tUSDo&+QEtrKbXD@A%(|M zYlv-PRv&Uj47&UlQmvyw!mr=^5c%w(&F~;Q_tD=04@!p)x*9E==xqNCi|~*LfRlZY zgWajruB3mpS?!R(&o~?9kj|=>OKk+oJk~$*UvAiVNpol$!f>y4_h*c%eT%;AxGZyA zn2GYD;fM6F#q_izu@4(^Ca(VLxv?uxX13~)O1>qLl&iTJ&WwCK1=i6A@|(D>)TvGJ zG2yJIcElBt&Z^d$U$HStVh^hV)MGqKFd0YaC`Tqt$Z12q(T0dNK+v{9(Dpzadj$_d z(@$wsX^48LaIHwKs59rO2k_)(XbMrzaTo zc-+P4v^5fmE0GzmO+Uh5r917UalJN*bb}w=fK+p!#O8a7TVnLv-1kaqZc$fUK8D{9QRV&^rvq_^b5Nz6zp!F3yRdPYGingTTe z(@RwVA$%5XAYV-D*tiH#(9{P|J{Q~2J}#RrnXDL4I}ejHgIutX4y8h__yaxeKHQW! z`=sPnuBLbgzF6|djTLlk>R*#ko{ld3v4rNEc_iaQ zrXTB8tT92pFQbzErm95#mQJ#Nsdy!DOZPePYk6&hW0CLAlmO>C1NWi;g{!E2qnDz) zt9?Twud^yI!5{MR%~@q@31?MjLuHiF&9mnC>_Qt=&FON$prHvv4Q6T+Px-UDO?%%QA3D|I@RKm0= z*U+y-H1nY$hx{BX&((IhXw70=8rr%hvn{=+9%x^j*sj=wulVbtZU~NXNoHQxFw435 znxm1^O$ZkO{S)9cp5-J4gj`~1OtCZkKKEVitPp+s zMQ+@oW;Haf%))7WX>Pmkz?=dI(^2|($v1+1=%1pVwd_5D?j08I?z^7<+ZTjW@FpG_ zdL$L#b=7b@G<6zo*ZkIe3RYU`KAOHJrmoX$!mQGUH{Y2#X8x+gT}{5Q2VWKg>($S=(sgA2gPGx;7B&f z1alMZn+5F8N5_x5i}!c46_-bXN43@~s9H2wi;-iTTrI&;Z-cc__7*+EHqRA{_lx(t zmlmQtp3W_K583TB&(z8EpFMhSI@>_l9c(`_QR;c}^It5{V0Gr07GC$g5N`s`LDBC3 zIjMH;vWbSwjXx|RNAXmsV!W^_1>dTDL0E}|OVJtaiUc*!KazhR?*T44yzLuSjw4W@+r z=cpDIuXqExdgKZiY7Iy{8zemdLV@4{$2G5qlBJK%nhfrKikJTD_Q?E$9caHW;#prw8`|gOy{BHwT@3$c+13onXglal>v3j*v;}*C zPbQ*!5Et~F8q2VW@ZmlK@dKU4G~ZxhMG9<13RFc3`~k_Ac1!hIV<2}6;>su1AI%3d z(54lt6thfUm$sx_Le+8?FCf0~`h96%V`F4BmS`{A;})@bSwEsx%2j{>L|m22wn5To zX&qxRkCfqdU`yk==_ZLoa;s1)XQ?FOT>I(71dqA6%Mq*jWhx+UP8EZxm#Zz z5dLKD59;{HKY#$Tzz&qx=)=pq9_$4oMu_7zl~|DA~lu*7l$%CKMV9fl1z zM?#9W|B>yxtwfX@!a2Q|jex6u15LuaBEuXTe-XJ3dzUKmbbO{u`o*PJ3kU?t8Ikfj z^OI96_Q_RYTdBR}zpV_$Dn&EzUrRLoGab40_9}QBscr0S#{0lUGGI??TXbu<3w$yc zs7q9ZZfh@aSp?PtBonn~qO8xS=*&i=`Wg`psV}Fh`i~N>ZgI7=Wy%KU8Sdf)^OJRh z`C`6Kq%3ane@}@`du(-SHHy|+-^LNqT$aYsEd07mDOa{QqL9q?$ZF6-fi=LK)^L;7ZCUW3Sk@7!TSuU|0TIe>8AZQew<=jM5#ra~&rJ zo#`zk%a+89QwcK#EPhg7qD;3b|2E3iGuw

43xy$ zlfITu?&-t7NRneTs#0D{|FP*&;;G5v7i!KEvqu5s(7(jC@YE^F zyLt3ovo@H{qi}c!T!bFJ0C68h(T8wRpcQ3N^n|4QmhK=3|0W4MM2bA@mb~2^49X2 z62mHHgF2}Z_b31m0Xm7wEo>@T8KT|j7)UoF5V;;tLobqc&hhQNXnJQ(sj6+99`g$w zJ(aO_mq(aTW7JA@mfSF7)>bfXdS1n}ni7HO+P?#}OrH1I}+-^%1H&le+$({%GQ z){gaF^XLwKRq&BakS?@pKR-D^Qd>y~v1Y}qb&iAoLWYm2=-U!kZ)}kDKl@Hl9gnNza&~^A|-tHsL`KV9w=V(V_#r&0gz}Olsm#TKxKbea6P+tZ2eq!V+*=__yu^| zo~aMy7+Tm}b}=0bFYE!a9ah~u(3$XuEe;75XO_4nCxDz0O4E)$S2*CsZe2F&V%39! z_Momr+;;Q9aHx#2OX1tqQ$EqnibpTp23|??DMe`3jiiYPSy(;wyPwS4d>@A@Kc=4b zGX1`QE@!+$#%o8nH`AH|q2(`=v~NEsQtL|(yt6@yw4n8F8>jCm=9^TErMoT*Q!`He zesKw`(rUpAc=D~bgFmD#CUtl2OHE6S z*&6w&%LRUiqA+nqWJu@yOKC2dNl;g^yw#(Mq8n{s)Pj4z52mFsZ84MoplT}jCEf0J z!tnIFeIVHzaj*7?`R09z#u@4P>g_$kr%*w+)$ome0v$$y2V*ZS!vj?bZR!v{1t*ekOT zfT1t`!q7lO^s%SP7oyFB8 z&ARSY{ABMW=9Px^erf5z+jxYoF7>r(Z%|5a0O#u{_wol7>@Dnr?DLqP`b@oM{LAEi zU7x1x`x`o2+ zVJv_(KlE53qao~<0INCl+bUw9XKzj{720k|jm*AXDV#Phxil)5NGdR1B$#ioFl6~e1Q1K;B2(v>%ClI-B9b!tA$#FqJNN_g zlF~8O|GhYZuIsw*G&ByiV6TePgjsZbj`FH&O-(8P*zO^ZGkChql!`lF_r|V!JR#HI zGWa>)b4Z7KIR9D>9$f34;HQduXz@BzD%V>Y)iCmcjVDl`FXnN|+mYZAQ|MM>5o||I zWboFlZQ`7c*goyAf2}fr`~$axCkrEaNq9wALrbFx^=lxN%&!lz&!#%q-|`7r zLP3&0)(97=pWG%Z*J;C*hW=_Kb999ypYFt+HILgW^MnL6^9vsvKwW{)PfQI!qt>&& z=exWQYrlNdcG>24?{jzh_OD71UheUW%%v_b+=#YgB0KjZLpuWcj#dD0Fc*L(FHJ*Ql`f*mdzx(HYM_c)n1m@~4gJv9gRr?KP)RLL#qEO}Jl z$?vOV_@>=2HfJGC$NbSD*EJ>2uv_5S_M(rt;t()DUgRFpM|0}rxj&=|nmfZ|ia98| zpmt8fE`p%~CHU4w{548(C+~L2>w-ypj428eUs`U8%zDwoH70ho{G2)Jve!7=VZ00R zU;Z_oBd%&wCTbf3K{`<*`==?CqB z={h#}&%)O$5hXs2n@eL+`c!w7C9z=}^#=NC`+g}0TH${SYk(QtJpqbsUk)U$r_NXc zS)&496KmJHn%mg7*#=R-!oJPbrfZ*1LldB)ja_peQ2J4G2e&%q7N*E6(?7H`u z?fcYW*3rIGXAGaT2s;1-U4sv`AP--V`d7h_+3=a<^#ePh7Rt?M69w8}RNRZPkdUYk z?BOEKZdwCwK5fsJ4HT=185jfi%zOZ+IF2 zmP@$y<)u-(6y-Qf;Ljr4!H8C})PxuxXAoHR`+N&Sz$3p?fSc;;?=>dzyLTsdi{=3H z7)a)~b1GoRG^!NVF)arBO7xMW;902J^_fKYg9|Lcu&A2h# z>orPP9ay|1-23e)=ZK9Ae`4!v*9Ys;9qZ@bv;|?ktAIFQ+y~PDiBFxwicp5VAigvp zz&!qoCBKM1N-3wz6*VXffZ;~-B0p-DHvkXX0x(8OJUvNcxn4`$GpeS!gmcAQjhJsA zIqG=;90+P)-Mo9B@5|YdnB6jeB$dxZD2zV*&jpckRJ*6dbYFBfz8#dMh zVo31%_ADW3E-8q0RfmB}ZT`i5a#vM7lfuXADr3_oTeMI|U&I8U6;3-jhNtv$@q0Z@ zVSYQCbIGXXldF)$QODmc?fA;A8Z>GuaQFmO~f?hd7%~W zG~&ps3@yo`W-Tg49gON7AyPm|^Q`Z6ffZpq#Uxfq1EpDAJ3e2vm64I=fom zd7mN^8Lo0QhXegR%!WuOLiw@4%b4bFlN24aH`Lhf4tcm&wT<3R~QiYJAmFJ8GQxrDuT%9#-WL~K%8>bHY^%0qr_K^?UyglnZ_||pF%o>Ek}!<;729#3>ybvR zL$9EsCUGiN{AsM0n!TDeHl#$%8n~^`5cZ9y8-5nB)oH*96}o<*>go%8 z0VWEqkZDFz)TU65-U6rbU5+2yx!UGvEnV8`yRCrO0F~f>mdMH#ZqMH$rYtk|_Yf8V=l8^~ z-}!jLr}U}f9-N3+)__ATYIn^a_Y857*YiC<17@gUh~_h>ojtde4R!BLSXF5Wkr6B=U6o z=Csbo|6bYio%?)ih3_pAvgCR0C^gWfer$+V#9wl=zgRY}&mgZKaCm%*Yg+^a5$AH_ z_`80tSD+AZ_M3#2oUEAe3c1YC$f~|@OqWEML)N_F#$kDy8t#03B#>g^!?lWgR=lp} z{M;@^{l*u*8B5H)RX6HDH8u7nhXD{D+kkbB{c__{9pD057CO!doryb_R!8N1T{+a; zBzhY!?%u zX8=x2X%_6L_xo^4W*_!V%F(Z+=LRhQ-JolEan~seS_O9<&RBC2Y9d@GJ1j{}?i*_9 z(=?2~QjsY{*g?WK_pN*Bpj(=ntF`)@!&h3+-8-|_lyI~L|6Df=^gaxXT>a2 zxoh0!x^GdKi50LqK=5fr);BBsy1r&b_zWD_*C7aA!+IpX4nY zes79W{=o^%G(bc%$*8@IHBrbpa$N0T{7k$50gm%T7IZ`fWL9|!usCC`g<%4`M3u=_ zAPIFeyg#BOR3JHRHRFwWL3wxj=b7WfABV9T^~iIZgmojY-rE)Kt*`VUh^H=qg*1Bk zE9%8wo|z#KhDyrK=@r@4GGPH@;rZH!5@|QC()Ovb@i6X z(<~~zr)>xQO;ZQcj$s(|g$c}aJ1HpY_Rw}zEn`UW?<24Tg;}11p*ZGc6-eJY$40uW zU@LCDru02+l68;wn4#T?q(i!=eciUL6kQZ{^XQM8_55voAa01Jd3_C@kn9t*_=q*D z!H%+k2@tx+*6L51reERg^HGrF@T+&$Efxz4`--y~G%0>fyHd-;+)TngwZ6X&W=Cz# zeI~?*3Oweh_sc&ZOx$0g`6QXiklPYY-pnhf$FF{PYVB)S&@m;Oo^LKdKYrWJAy#}e z=*btVZ(J%6)ssRg!yOD%IPrGo2#`=U0-XMj^FZzsG<$2;>K;W z|7n3M=vie!c(fBYDCx6O#EWeTeaJgw+`l<(14u-s-oGUxG;pzGm(dp(yCZ-^`7tG6 zn()0d_Drgq(6T?z_(GcnRH^4XjNj*poISKZUzb6A=!D^P_!oIJuOd#QfB|$rtYmS8 zdMOD3#?L%9t^xjP!fgvw6V?mt9i9q-M^Vn-)saR8%j~=>s|Aetq(UDkcpRan>Gr7uBkqbQKs|R`hiSws@z^8lGoi^^R{RRHIGkQwpS=-HuR=7e^2&y zJ9yAM`y~V3R0*EZ!^l-^*n<#Udy&x;oVN1O+rvm*1-Rh0)F7Fyt9x%klcv}R!A+VQ z)c$#_7}kexSuqDc574a7OcpZQES6pHw>0qhUdZm9H6NWLNkNszz(9DJ@7aqN)s9k@ z3wR3Ks@147;wPF-9B2#`3vN8`m8#sy>&qIl3T;>ti%#;m2 z^|zOxhKsq9kDeaDKw6)(Vxp4IF?muf8KcVk4nvJ+9sm&dMJFBlpy3c~GBU7xy%{sH zDnP7QZJkV2R@;4J_*7pr=X|6w1amKh3nO@I_C<`}#-lUu9+fYEjCEvqvX1Ga&CPb0 zGCLP=Lg)W2uP|Eqz5M=Ab54>{_M7N8Z&CHPC^zaPDYTOlNZjk(qV*`=TX9z89mb#j zv6d5Ls}l@Q&rfrH$9z+WSCI|F+Oi57a$>Gx-bmH!5n;{fh7N%RH$OKh?hNY>R_VY9 z%*8o(KI%SHE!rF$y^CuSGLG=p$w41L7z!NfwhJ9^A!k)>&$gc`C4qMxkN4UQl-L2G ze@&fe@#Xro^R2|y2>Gsc=j;eBpuGiL*LzLbsMfWjTZ;yA#W+&k@XM9MT?bQx=yKOx^RTt>h zw2_}R;S=rh1NC{kJY`$V3V2W1z27hPSi;)eg^x~N(TZ)hY27RU_&-ChLfO)4&}q{Z z*sxcI5r2gSk0UYr4ahV#$+!b4Pr?QP3MCgk!~zMuY~+ts;{};CcH(xTSDo;=W^Bi~ z<>|n?Yc&1b3VXn*6E4AY@X#L1bTqWr>RuR~!Cu&*81=h73A<<_VXeEjOqz_v1%Pev z8}&wKD0)uveZU1ycOE-Q!Fkz! zrg}JMRiG6ZiPwIO%0~0D%RjMvM{nW)bOmRsA7wDK_1}1e`L>G_5Z`$T^&;RAq9TuAftSM(EgE(KCx+H%yo0O{+#nHW5^JbjUHL)eC-Yfjwa zkhZHCZIQ+oJZB0gu4rCM9m?*-nA{<6dQRmDcM2bR(g7-LCxGLhiHvS*e}JROSjTSG z#t#wJX~(VjyN2FR1LQtD$S!wnS;v#W#$F05AMHu;Xg1cmof(XoGETwymvkTLOZwmyV0?unr(^@vLN&sa`Qj} zNd-=GIjFtRZhjO^mTAQQSpnq#HkUor4%p8pVbp*3bMfP%`~-|6A21Nf-P&qs^Li zjvU&+n@rS-_A1Un82tf)BO601wY_?L3gFO*X?l{jhNkPYL&A)1pp^$baOCc5V-cq# z$3x85td9%|Bz3>R_Opu1luop$AkyDC?qkRk)(i%*{vcSwtNe}!!`5<=Y_C@I;;q@{ zwL`aYH81}=f9$d=&@LCxEDGaEfjLzufWV^$@{JpFS_;xg(xA>*)YcnkfxoC$ds-jQ z8xDSERp5t2UaVE;Yq9FX)<(`WhOEzq&xdPTx?LdCjnhv(b{&sW9YT2JAFAyq3=CD- zc^8%~Z$e5c(aga4dPbD{E1!-XS!cOtA)ws-+?egdh+iCRI<2! zPQF3G&x5wxPuN;5{ux2|KO^AzJDMrcHXWTiafM#H6rAU4fjGgzMc05Gq9#pxy3Z{KW(|K8G{ ztN&a8I2P19Alqs~eYRKY*D3i^l30RS)E zLZl@e8~g?~B&PU%$$(POkith32qfg~X+`a3cNGV*CwiLzs|hlEFY9GOB5FYseQGcPPmyTvfi3XtZYmkp_i8&(ozhSr)AnC@~N z3o4tE6oa6Zw$`@9Q{jP~dZfV3or^K)HN!i7<>X$E7PzRlxgD&pAH-J8d zhaYYv0VVp}2Y|X?tlrECQp81SczlkzN_%V7P*k#Z?X(AjMu(7QF*~-e8imzlK8Y=J z-wRrmFg=l5J+$ks9=c#z3uVI?T!0U}+NJ}?AXrY-_&JHPL6nFlKvD;p1=>ftc@Awr zI9(2(5xS!(-dM20nL)BjodZtmMXc3_i`wwhD) zO-fzKLscgV{N2R$_P%Zw4I)hUxG;KJhVTZDx>8BgL(H@n)yc+3-RvG%ZrA0Ivg{ni zj*~M2>NYv?mqV%;xO}QW@Qpjs%a$zy>+EI2oR&j*^t22N_lgQ%zkPBGDF+H0DzJkk zyTcrid+h3SQgi|39N*vfnbzC2;+84<#9BdV$5ud91E3rLRxMDdomlA;rwK18eJUql zdQjg=UET}i0+0yXo{X%54GAB+_egAg^r{LnDOz@>q%+uJbFBO*qp})^hHXdsGde(Yu8C@CoFpjmQ*?!Zo18s@! zheM9NWKV<)DKsWV#MK?OHUaD9ckMwmnb99OZ7~Wzl-d;4WVTJ-wa^gpcLQcFjo%+G z%=@=iDhy$9Y*!wyqMx zV&%rX+6RQ!`B5}KRfd(qpnDw+Q*S2M1jdvmeAfG{#o+~!crO31w)YHcvT4GGK|rL5 ziU<%%I|?eIQWOCZLhrqW-V_0qDhNn#N+(E%0HI0mQbJ2Y@1a8op}be%zMtp$ zj^lfOzWI?KaAmLE+1;7hopa79?Muta#~$sz{T6=25IDI~!Gn9_L6pwcfjP_Vlx08k3@Y2m;v{r^Ehvpwm z^K_dehkB6|p~GQSTHhgc9Xh;rdHB9r!f2BMVI*>)zj#RT8dfv!<%I^0QP8F5M9a>o z)0d5(@7!1tyt)F;M)eGaZkpC%mN7?2Ep`G5Z1Xj1Vde`!_!H{yxrIeU8g zrSm!cCmi=qEJ<%L7kr_#x`0DyDgF=|<)i-LJe|Skdr?)p{+XJ~Mb3cRd562fX7g{;@5#Hhx3|?!y?v2=_GGV@Wfa!mK9 zID8gORVTEpt3NsLfJ^v<{-Z_Bj-o>Ul~{t0`f0B5 zhLN|5Hai0Q@9&8(CaNnYP7sRTubYIOzn7l&j?wzntZ>9d4x8zGN{vb-wO7I^^Urcm{%(<(J(E8WaCAdlPA1hCY;R$$~xqSe@<#pz^z!xSn zfHa33`M)N7}V4%@O<+2m!q|&?V!_lDQ+mu9`Ws*6|1+ zJ^vEZ^@?mxxM&mAz@70wqg-3x*C|Q3kS7hWA$=`n(h8T8^iBF;!(V{fAQJV$`PnwL z%Ti5B7=c`3{H3D$5tpt*ZM__CsDtRm40GIIGmsiQwDCs6C~ozKm)2f0THwVXeTIqg zH*zuER|f;x7>G5Koi z_zjJo)663hP^%M-nJD4g8w`pZEZ_NWznF`vf0jlLEVy%990T}BC%N8Hmk;HvF=ksI zc6-KiHiVkCRX&%NYxVlhyV1NVU|5b@n@2jgOrwFStWUB^6fJ@Cr~N-O_v68LT2`ll zo@_3Pi53Y%XJani^tlr0S6c^jriZcCxo>tD3tnwGCXvY6Z;Z!6U1O zoMHOjMRMku4_(d|Bf;0x>5?H2kSA@0hRSc?KG$crjAQwf+}TcOt|R{3!7+u1JyzMj zo3~W@xN`jD3eX!R4gfpdyCHP*@%d;1$R1@{rhI9mkg}6W81^uFTW->Vcx@DtT6`bX zcAxESY+erkaCBjrd16@`<#q4XjSB8CEEABq#op_SJlPX%5hTY?J-`e>v{hdeX$bDp z7VbmlCdZi0+fwJTeDF+j8Nt+cDyW)h%1SHJ`xP1osU9KjKjoyUD{UqQ+3@n>)NZJ? zzCwBm-SF%R@R^Q#zR1lhgsKFv`cD$RPp%)>&khjL)8ex+f9P4Si%x;y@QV#dkUl)v zI1`G`;gX}#1XDdDSAJwh@Zu6rsLDOyRwbEIrjGx{$DGR8xN2$OLQ+2@HDW$x)V=*= z@TkD>3rvmhh5!<+HK_t}VBcG$hdL96(egC4m}WLBd7~eCOz2OZPtJ>ot2AH+TN91O zE7THv9QnxFzfUgNsL_Yy7xbKoWg8NTU8ACV66+s(XIWv6=tX|3VlyA$9#ZW{+}510qqH-Td} z;K`VEL``A~iM%RC6GpTeJ!#?3 zl2^d5GlhR7iX87XsefI6c;CGW)n@+j-u)V%31$PH%BFYc)efR=Up(OYV}>N{$t#*9 zvacPth2P!1e||l7R}*i;({B#`NPu*kzF9Y;d3=3*@s6&|`1vA=0+JJe*7MoG<6sc? zQkiG^6)xRP(akaFg%d?!MS{2azrL=OuKh_3ZLz)0b%`f;&@7VMEZqFUKt-8fH1Vf) zJ+3X8n=v-Q%M;BKW|6-Xje5+_z3oRe`0TiVH*K4xI5nf;GUaKe*W#OgF$H~k z?H4a_K0itRidZYH4Obyt>adRCoi0TTEC2!rc*n1uw=T6>%KLcePj-NdVOb{_qu+jh z77cJdA0SN-6e%`tNp*j&)%QUE`%UU3%j5E&{X%39`*p8ArRAYBTdR4Xt~{%Hemo@j zE?jiLnwOvgj4ms^4=F9g5zV=JHBX)iohWD5l_YF!b2ZH$Iou%WaP%FyqLFhP%AhA> z(rjX_vhT3gSAfM)n7@)f{g5yeG2EYgP|rOwa@Gsk_3{?bGX}^@YO5YZ^ar|~*}Q6Z zyLcZkinsTo*^U&W{=`EiHYn#B%kxGW3KQO-$&JEznpZvos3=n7z7PR&p`v&yfSt#E zKgz?ysXGe8T)md{WvEI|N8VC$k*PJEc)@eX`t;uMnLSPCqjYC($1T1G}m+dw;Uah)#6Duk7qIqDW|rksm6PnD%Ti;V*^Gs zHP5twaJqj&?v39VASqU2=REL6P6M4*j-S)_(3x z0^E=b1*R6)J?u9n%&xOv5dZgbQG=PbwdoU(lBDh-t1iustdBp^;MZdyh zeF{Ai*PXU`L;0ezKk1) z!v&SbekZF$c`$ej+t~Ew2mEm~K~OJ zi$G8S*mcVc#Xv(=sC9M!$@guKMtgzqma5Me-a_o>7_XLAj`JpjP#z#>3y`|^0e+GF zVNAJFCe*p*y^dV_2;I#-!kBgz-nx21_l&Y|w;K?a<(VZ8$A3t?ZQC zOGKdlCf8<`)jdUP5eZ;}!YB)(Aa%Tco7QK{_3*d$3-8R<_b1SfV}kaq_}@K>!%>2O zQ_tVVfa%(;;}wgytaMmS&+V>wAwN3gi<`c;U5fkt&Q5}P{t&ceY*%SoLC=+jI>YkJ z^e2V~7^Ju>;1qI+imgo?mr#So`m4QT@3iad+!oOjX+e}}YYs^M{yU$qv?&yu)Okgn zi?JSsX%L*_JhoP0yIGZ#t3JT*ZI=kTH2ElqcoLrwogV$pW;vgNK)p*OXi&Z)`zVZD z=q~g#9`TpYeU_CMW-Df(H84bAxVIF}A}`)b<45s|-{{c|So?gb-Tr2xGV=2KNCXq$ ztpMCAA=?hY5GY>#3$cAKwVnU3lY zU-S8U-uZw1Ak7ZnWCheF7Dl|%Dw4RY3`^5^*`%Z;0PCCncg#E&-@$y-vWIw(BrzQT zC7jxS?cT4ipWW`8!pVK)p&l}*FZ|_o`(usj^_YRRiY0T{>gWr$sAcKiGQfv!{FBKD zWeknP_l^Cqal+Y57SBQfZ1yglojhYm0UO`9#vktmZwnnjbyfz=p55|3Gt9KrF++x% zCwIqY|9((^b<#c(gS40S{VOvEvSG)?TIGf*f=Pg@AJG!9UCB_z#6oTg@2l_Sg&eq3 zCxD^h3-nI=O8MmP_DY0FR4ibZ@4+0-1^t!M^q0D3ygJiKUv#?p3vkT~xs#ca$@$Z* zv1WDxLt9M5yJVpyjsqXPICS#$39D0Iw}c3vxp{PDnk3s{L;@GhmMG3wzs&Byb1k)G z^9~u{nZ4O~o@&;+J8DtuyCG>kS#&T_^RSzGY{6dib=5rV5=E^>6L6}J!(2Xe%?;yvqO7Msq zMXZm#L=I(+)EfUMGdjsj^NNs6KMc50qBs0 zr)BjCu8BL0=@^0TEom!*5tp$`j}l~Z%}CVx-8YpyG%YV1XtZFr3zMC{`K8QcHz4-YhEJmxV)oJ_#=m6ZNS|yxO)bN0FMU1oD zW%r$pfiy$)-Ac5zL?Hpe-^j~x_5H}Zs<$sMsMs)7if@cF{RPSwbz*uHXhig`freJs zvIoIc&9kurW|w(Z^}))$4lFhGg=}fC7ONP>+E7xffWM1BWDRU6EZ3kdqu~lWqGM-i zipO*wKy40_U-U_&6q{=9K}C=k?NeMR z__!Yqc=gG(oU+>qYk^2#>bp%{5)o%BLJiUOj+lLoBQ==eM~gF!@#iowaCK@+W$9n=UkoDh=u-YSOnoyfB?j7&8`adm_fKea)j_rOQuYz>QY5C z+muJ5{c&BCP88^|3_Kmlab~Si;h#m@CDlfSby4boII_TTi@I54iEE6M&0h%%<~ouD2LLTMj}A zI;JJQ0fF82?`Uq?_~UT$3(;!STiz=C!ol)T!k>}%510=6QIZGAag@3a&at*9_oRF| zv`Ff=(>MTt!xcY?2p57 zO**n>T{>XwldrK3_}uyr_I2Q;w6psy1SovSq|Y6 zyxS5dfgyp4(sHyifV(r`gC!wyb~qZ&FN-{G&!0Zbo^93@UD*8e z7i$(Z!5c2D&o~Bup7(mpU#2#QhogUsuqg|YI--U=DU5xzSAU|u)ut_xC)X^o}c`VAI{WGi8qvc+L z9tKflZrO^A<%pkIQV?>Cg`0~+t2OKcWzh;obNJkfq~ZoPJ~Jf80;g{FulbP6LpJ@= zy9Q_NY4;m>xDLQ4u8;OcUL{q=YXGysgFTkmDm4rs_PaPkd;D7VffC5(2a!kyjX z+tJ^mV*PHPseqow>bT8@tNzsc#xn~8vSrFfB9}le+t>1Pst-!@F=;xMA0ZX5AI1)D zz`rZY$v(l1@1un%zY(fi^_TA^e3d3d8f^1HbAj{Rr1yatOB|5>(MNy!hLpgJ z4G>XDD{=k+be)67XMVN`;RVXY<>o`>Fk33)_paqnyc9B_)~%gi}KfjHmt#OJq6 zjvN5l7&-uO3dBOSXyC0p;5Zm|7gd=kS-)zl>+<&GFkJIB`@HdH?f2)f3Z&b3R~{w4 z2Xo4mV{HI|1!3n?V)>2J-o`^84($>Of^=JUgExBsI!XgbZNVd7tOOscRCMx|-@KuJ z!=TGl10X+>(k)Y12W-Q~s>%mFT9IfVGfBrti+pwi@WOi_Y5?B6Z2ar{ehU`p!$~`x zHUFjXc#GQcwoqfXr@QRm?v4OpDhbtv1e6 z@SI;^d#?=TNUegCcuJ6dgWUio0qZ2*lwkm4Vh5l(Bg948TsD0E2L>=8Gfe)+?o8;% zS8HZOtR);xi|r;D{rFf1zvs+tD2@H;!}8NQT0ZSji?ZuVwU8m39-aR`z-vsE{EK%$ z;{S0yhc<}|Sq`39pCC+p90Dy9uv%6G@*kjn2UU;vzjIrl1IdU2iGb{w0Co4bn!&!i zgO;)*C)!Whfq;`4QU03s&;uIaN-8VPc8)fj?%CS+8mSv`kyFfHgu@eZHk~{WhpuIO zE}c26KSI{M@&=wU<_O?l_KNtZ+u2bU2Vm|WazS7RNA74Ma%xThtIimrK0{{KoL5>H zTE!9x%VLqBg^{>;}5ly}b-=A)$`1US3iT7LgAZ&P9{@H?~k!_cktLaX0!=BH~{?4)gLOOChe z#x!f_ew0wLKZJ~bosQtwV+;DV7E9(3 zPm5Rfhk=1D_+kE3j!dF&+KUKSBA-XQ~gNELA;;|ckzWp3ka*iYf7HO z8dZT<$3>g~=unr!PSG}TvzV$zKy;js-*MfQ=Kjtq*#lH;OGmjcJ_>iv0ES1qzD|sG zNaCa%b^NOEWzf1m)aOP)k%FB6V3?lOvBeVCu~6!iv{7{JI>2w8;>m_$;w6jJopHMx z=CI;2Oi&- zG&^f(^viy*ID+)?UKi446iV*?>cjb_Sk@!4P0-=a@SP<=P^2VK@Znsh`0->q1E@DG z%)~-|DAG4MRXrcxW7HRppxwd302~ktsS+zr2q|SN+zE2qCnW$c6}^n|VZe=d3RZeG ztD?2W#qMZL)ojbTPnr$umgKIuPn^^beHAZi_i>P1L+>fs)gmf;JAcr1j$yplk8JlK z;P+ZZU&txZ5t4VbU(Ji+PW`0O=DcJ?xZ7RNcWP2L+l@92^W@t-(}b4dHondiU5F9? z4I6x8!BUQp&|^?l;d%`m5fX}5ZIpt1U}Xd)7;2++q1Tu7dbz4fYER1(=J{)(`s>?M zpLgj*M}xSIUrc!!BsiWRaS5N+rlfHCbyD`VzWxQi2+LMP#+bWHLJ*8@r4(M#HYOuJ zS&4qwKj#j&tJt(tt2VbzZI@v0ez^mt!kF-dRhaw`2UgETc!seFo2o9X5(#HnJA&A&@;$8kQ8-|$>5T(*7n(Dy0k}8m5h3oZ`b6JU z?MDR}X}$n@ro!f;CPhwNC3IerM&&= z{3R7+(GKAZGJUBmN2R;=5ynf`1 z@gl{h{+J)KAT`;QxT8yVj`vnP?_aKMC)_;k<eWVA52{$h*gR9iLr5Pm5y#JaXB^i{D`6N zb-p@Zb0E6#?aS_s#oGQD2QaX*7|oy09yEOC~5K7lrY)o6*^aR}@se>F+g3?UesMk$Q;@wIRdb zN_z?<7$nSxTI_(wh~QLuwcP#XW$31CqJ2X`KnM92n|%UzX?vKMGz*>UKR}@YsyS}n z#s65laBhz%;1YXUfzaBMG=?^VYW*<4!}_hDKHdT(ZjZJTCntXvB8ug9ALDPKMQ!DT z3oNB~QmJU!v?g^E0|bZ$4Zi3npcJsEGF+z!MOJ(KtQcXgX4|)$+^>8!eka-o+crN$ z=r+adNR>J^)a2JC{lhigjqWZsSr!7Ujtg1fzfoL+Ei25Mto1P()CA91v0?cOVR#qW*%1c$Ux z@)H}svQL(aFJJhRwN~QUMU!HGC_;WqGhlwVHJQdn3Eh+)_>99VqJErT;L%PE%}+7_ zqrwoq2_zPMi+#O~&opX+Y@X_D0r`96MtCsKdCmi|>`y3C@r97(QaMq8QEtTsYpSV; zbYzUBP)%2@T(kjZeC=J5GV|^9Q9lmt*PEa+sdZO0hq!sJuAh(GuI(hrLD@LjY_CDQ zX9>zFHcU)=U!1}t&#H=pp-x6;-qm*~xkce=mwp&PY# zN<2415y`Q)^Y1aj*!4{>Q;Pxoid@2-n3IOXwa43>4kv=|DvdP-e`Rw3G;~@AvVZRU zV<-ha+=}7t$>FzGNwx=C$Fidwoth~(9Q=l3+>ahsO@5UY7o4{)Mn805wy&dvN7l8g z*7n0T4*swJz3yQ{IU+pwu>eV1VD*I}c(l{{8ZIOb8DWi?ipAri9@Nqp$Dd?m7uHR@ z^WGQGej3NmSc~qh8a+kD`|!ExllDwgHi%7X4HblLNC``nITSFwSC|;{To=2qy+i`Q zuAc00LoA^@NrM<5N6l-x9+C!>6|8XvI|3*Fe$`?zYT2Wpb4Ea+`pVu47l)3%xnl~s zYf~kL+fn?D=QMK-cq%gVz4gVKqZO1Ix;g-`;(*-&ND7Z%e8*AFJw3nDjxH`|2dHf( ze7yT%>o%QoUnG=ke^D3q__?1Hz#CkK+5^*=fseuf5OqM`tE6ILC8qPAZ!l%!vlH_O6fy*wR zKYw1x(e|s;O?d<(Ibs;4EM;@>s765Fy}5_|{$a^q$!tp0z{Sp3*VpZYN}Q@|c31dd zcKibl>L>M^8Q_s~OVmOS)6xUcBtTPgyxl>I)s!5t9a-05R9*bf&ldn5fDN_8a%NoF z8|Tmv-P=5LCR&NCi$saB4O3}nstq9$v8I(pKMm?qbRlb(3gK1dh=-=*wdXRSQ*!lN_9<@`w|o7_V>I=vA4k8=HQ4^K_UE(25f z>+z2A@4Rp*%I{3z)GM8<4Y>J95En@?O`W(3?*q4tc;((z`nw|<$wqkItH}AGQekas z0m2dU)}k6rqWRN@6K42?wRow}+SB&}MJNxuG?x>|92fSOzs92e?2IGKGg%eLJP<-w zuygN)1Q(e^De=63=3HrLrGEEqX>6e=D?b$DRXkGp_v^L1jsiB$7IFME1BgU`kkYJ@ zK6s+0M~Ze#Jx>3Pcf$<`qng^de@lhSS_XODwj2IhsB9P5qO`!aOmAjGl_q7<+db1$ z*`LPVyk9vbeS8$3FketX4SD==8ON(g3nJix%ukwU7Ru2bDJoddBRI6DEFSE#bUSRn z=Vjb1D~?and%FFsIOX#!7Ag#%YF?))5o*7(GpmcN|O1CcXmz_xzC#Aa{~auUX~ z5rdwkezmlGPrLECUpHDEDf9-7zHIT;$T5H5PT)Ypd7LnF-Ygv6# z!1nOH)--pV+;5>t)dauoeaoYck#SF})El9RqqIACF9|Sr-X@wiw;|PC`5LwObnX;R zmo%%M1_0W)Yh@DwE0DjMe%dg1x|Ql)dwyf#AS>o7TV97x-vxNMWCoe}UIG}%pLR>d z{VsO~*M4pu^!7jN*0j7cH%AlqV;<+dEeQ2ITFrG#tk!sQ zEfGBMLbUaDt$hV@NeKUU=xv$t3=|YBBp= z5`0}|NvCEduB=xVL*j-bKG@}Vm@M}m+8;Y}cYN`b@%~DiSC@U`Q4Lw#G^{4dy_l zzbIS8VOb|APTkq~^C6avVLx?|H*a}S;M?-V^Nkf*UojxJ~NqbIMn9XviEu+iqL{Rm}V>jLQ z;YD91?YFH(bH2}EmRTDStw5C9Lo`kfmD4l*n2qr6(FfgS72Xkmt;{S|lrB;sJN6hC zQ|0%gcU|KkMF_J~F(Naf`i%|EhMlVR_ohB&o8(7=@fg6U!|4$R|KWaaa3|*%_Pp;m zyAl~yclI|w{}}@?C_LNHuXT7LxtcH-n@%oZJ@bG|l+UM|NOv9o(`tX!X5~GwKYMiG zdsV$1ww5oAOqMB7epg&QAL>wbxft>BlihE= z%H)%h1(C6}&W$zUDFC{aVH?ge=0I9;#Se z=HN@J=y%cg-Q1*lCQ7tezcaMzxKCt+vmE|x3^3{@&*xYC<+EY{J!FVu!`?0m3mr=S z&0kO`QrLO_LluWD3rP56n<$b}IMTjOcvD=9uS_OF= zJ?g7b2#TJcv_k2c(6@l!M)!}U(2NCgswN?FC&p|{E<=&h%hY3M;(z1}jGgmhQ#d>w zl>?mTAEhQ~MQ;Lh(u84@?Z`$kcl}j>X`JsoTfmA;et%<0_qB2`=Y5=@Z(Ba=yx7`#q#F{>F z*h(k05+h#srJ`gvg)#LZE*m||pE3QI7#W6*`lV#=jTG-0BuhUImRl~6?$9nOK-=T9 z%8D9xi4o*05OE!$64cXHG_DB0B-Cq{<0!ynokX|8{Yrz*aMaW4%I`{)s}lA%-<_Rs zmKd4TCON+(xEdzMMf=ga53DHRJb_hV&!mh96$%)I|MxD&zeo+Uc7()d!_9vFm8n%{ z#iRn4->j(X1}p0J8PpS%)qkz}x`KRe-CW(Q`Xv|k#$uu*%w>F?G9dGDgALp zxkH<|)Qf*d7?U>-hHFAnyUDPnrCBKq@DSECME^$EZpqSb?|=u^t#=h*WKDBxyX^+6 zatS1Q*8KA>8-LuCPs*K8^?7v1E_rl5T3+(Jhe1q~r&CGf1XOk-Aa@P z!MArRYG&)aIGFfrQ6V;!zgr)%qCVYaS=rb5DmAw8BIRWa_0nQ?bDh0^n9S%xrQnsZ ztF<&a1{4K)j!PfU0vIBq94s6qi?gpPb}SuR0Cb`5aJu8ZRXDGOgss zQJy@vF<(vVJC$*aj2!nT(01vd8{PQTXF0}KTzc-0|3cs5%rH(QgwN2MC-zJCXUeC%f=HIK3gItX&6btSz<# zQsa4>6n)+64!J^2f@FW|6YcenjKI{UHq=R&)Y_eHS5PT-MzvaZs5PkG|1LzrZj6(mqzEVbGkBs2`NXlLTGeHr2- z)}u{f^9kewHMpZuaQHqT7t-R$$k#KYw{miAhcH+1Rx{d_^Hgb6jKzMRcV7)6%P73rZ<(?tp7(+Eq3==M__~NZ*gZ zHUuoy9LJfT7ocp+xwErC;tkPKfC)wc9CEAg=?wO@v>;o#(N#O`hlk%*`%NRZgN9POc%cW=Kq)|O61lCwsR3HGvs zG+k8@R%gXg{`?7au7LYWhpm3S+?hlAN$uE5)g2nsyK(WZ(7&RjwtaQXXD(G|NNsy) zI5W%lU;Q=xq+eHz{vb}rR_wI1D;ei7n-r;mv~mEK>?_tZ*Xr4~_C|Itzj$Y^ip^8> zcS7ltG#WSyh$6tumnpG{C!xZ6c5Sz2-z}vXQk*Sby*IzMC%_|%b;r^j9I_HeJgU&$ zL`fqcu9^zkcC4rZhk6>mnzQ!7SZxD3m74W}!MzQe%%GQk*Xs9hD)TWYxP5g`Riao?g6n=iWwR@qW0iyN-9>kjg%v}~Ubqca1F%BtUCMXC8? zla`BB@8}**Bb|?`mkTQeZu+h6bZ8`IX{yQUmYy6L97BIoor`PY6tg-+j?L|Q0Mz2j zy&@*!{{GoOb_A?f=4l0BF}GQ#VjyGsQgXCAcFO21r&T4P)9c!BWJFk0v2aC$O4RcX z(zhNRm|FRRXBK{^#aXYjl$6wDaLr^hsOtU{hHy*kuYUWho4DeJ14b8emL0MspyKa& zy>x0q^5?#aOZQBE&6_&8K$a zcK$_z@o7{iXo<6C%keHH3dG0CUX6Y`?)99x6*nr+h| zX!YH^mRH_^g&gGmSd*+tvhor@wp?!?(v4^YG z(q6M0`DHtQyW%yGC+{!?=^NRTi*$X8A5tSSx+m-hG0dMQ8a(ukyn}+=!a|r)>Ng0i zKU@oPzwKPOwoh;C6jiL}tTXe&%T_A8`H@}D9M4-e;3$2rqkv1F+@mP93bUo86Qji4 zP`ljwsxt`yiFz6{#O86vC73qySaGp(wT?njF34eYCMRvRd1&8Vz(3d2|Bf2mKn~4v z6ex{|?9&E$RU*0_j+OjLGlG1i{6YHLk!mTw9X~^~rnH(@pDN&N~i*RAYE)fU^9={^Nn@CmnTdBo%R)%CJB zq=xi6c|&R+3`gfi>|)Cm1c4U+9R*Ob>%ePezluW9zTaLR^9fSr+F{WILe9Mz$Tq5( zsl4^)SKzavE${YB@2mtk>g|{kD5$UhEXQtmCBDl4XiUr??LZ|Sd;*j{d;|{h3DK@y zo{rQTuCkxD3*PmkE_(45UsDd4#e&ao?q<`v?%Hcxr1ZLr0~P6Oc1j~MRAqYSwE5?U z6zIj^X$p@?oK|~=YJyUH#E9JmS<`kY?3R6Y@lE7Q&nLO*-;hDLj5JPCE}c|HzWi^? z&`g_WGeot)Ba3!f0eqDEe_7L@o34^Jw0c3Gpeh&*p#85^{1)_(OV22J6*?X16WjX-r&>P_!~Rtg&YaMw|BT73`weN!m>?~^^UKv2HA>D ziSf>BMCBM&tp0h)qNwOsQKZ$yN4KLB^#h4o#Fa&MH#x2+$w|71wuEEWv{%6`y?YRm2D$-3j&~?<)UW)c10ClRHeqB2{X! zJSw_NK;rn#`YL;d@a`BGCUjIr?fci|*$BHJ`9q#ov@hLATK3G;4 zZ5R}+OjNGYmrtv;c!FP%xJ~fORFZs`3TBS%TUin~eR($5%c%bf=)s7j9(;jvvX=sT z!0$d7D9aB;JJazsNYx~RSxB{RRq1+{g?ho~Lwt&T#3f1Xa2@NWa*cap6K5-u+YO%D z)ysV={1c8g&UemN=c5ix$qSI<`*(^Teo$zQXj)a<&~J^)_*L3TFljwb#pj| zdt{yGOtQ~v9|dN_QI`re(p4t~AqnCE71tbt%?&bHg4qow?Z0W;VIv*A~f5g)QG| zSBYn92Lk&~Q|@%iNu5&cz9dpaKqj-3=Q2Aj=r9Z$t&j0_*a-rpD6q|*tHUeV8#1v{ zGhHNsCj11P(0v(Vx9Ff5W~Cl8QHcGYXrl#gN*hlMw%^RiSkQu%ExAqN!F+qTt3 zQri#ahx|sb|B#a>iIfTsCR*ExrDL0XZ)v%|bB_+K`}p72ytaB*yoee~{*Y*+rs!ao zTK|GE7_jyMQJeF@J~}OKEq23ZIr0X=+&B-hc`cB~YGIjtPL)Qw1uH6blXJ4Sn#;5A zuUS=4Q-*>7+J=-I$(_)++zoclZr-E*QJ%c<}}n z;2pyj9r4VSjP_XrYW|~jUPRy>o$2_BouUXURuqK&m!dPN?(NcorLo0Jw$eosY51Q2 zdk$YCo$wJN#AoI=sJD#H-URS3LtVzy^-1jVon3aUr1?6GSko2{hE9NbfP0=y`Y8`8 zOI9L`535&?*83hU@pv!)J3xF(cju+^03X;Ees(h9UmxV|t}5ULY&v&n^ct=C_$9_l zfT&t((g0U8yb2xN$8@a!6&%o-t%y>iaIlnvIew!G`UITeHhAyI+&E9GyU_f@;10U0 zlT=l&53@I|Tj7MK5R8Pt*EQr=fv_{L%?Pu8wP^OJh1GO#qqT z8%D$%^ILs#B`0`z)sLli=8sX`>ex_zom-DKEJ^m0_EOe{Jk=0LtmUb9d>NGgzv4^@ z{>NZlh?f9g9{yswfXdB4u}Yx$Gw^Uae}f30E|#n5mgbwPzJ^c%)&KtM#e>lh6A~$6 z-M8| zaRP!7qk6f!0(v_{m-zG54H`TlzE;r4D$@U$&fKe@qkt*E;e=_Uyh5T1`qqBIQ4!A~ zWEQH={th=y68*Q+k)Wd}84IMsM~GDM3IaYxm%Zen`;Th_sG(J%hY(oX_4^TLd+`md z>-j_P_lL_4%<{w`TF6`fmCRA$DCt5;PKuvTcb_;}{c7ROjR=`vOaHJ z*U->Q+Bfo?TIvfE+w=(BDp0jP5%9Ycep_||##J;^^goa4mhIw8|9VcRrE(A#Y`@>D zC(XRBqH@4)CG0@>U*ALeF-rJU(Zz^0SY^`S{7rTN$Gp*JTFfgY2t z;*RMA@||t+&OqQc(cpN20G_AkY5`bAKstrwzmEeew(I&1eg&U@G$Z~MHsA(22Ssg` zFanjljS((Db@}rL9*HKH96#A%;8A*;03@=5*nBU z;`$FQcpb>j9}8F=W`^@GEiL0z<3q0&a}X3PNd5l%v`TM6UPim3l|3cppVo|3oDg*I zlIqNFDjO}6fY$({B{r^jTN@8u2CUg7#v@sAk!tt3v|Uf^c_@=$_!bphZ_Z~dClPDr z?z3cksni-$M4>l4{(61N#?q1Hk9?XdWj`M428hv$k6*{E0**Dbe3OK8#E$S!?D{H{ zy(or(`6nv=JjLiOWJU-BO9(TYwl=#u&5jRr18%YW%>zC+Kc}k&{gGf69~?QIRYrt~ z4cQ1!MuglL5Al;NRz@ccd9T|DI;e>mmG%t4M zv*MZ5_!7V}<3F+a_aXf8lo-wQ3U)|6|H}K&;5QcG;LQ}&Dzahr)gT5VeKMOSMXNVV+eF*@Zg-G)DmCr*)`ea`}a6aUGRX3l!i>|z)J z?=xm{-!oKlLTdOKP79v0r#dN%rw2orj2P#+xE1kZQHhO+qP}nwr$(CPS5ka@A)4BvWLC@wB28fhIP192T)1M`11k`5MjP-@@3eUjRY? zR^J~Z0CNBXvcKW}BWM0!@_+6Bwe{DWt%2p=>HkKjkucCV(c!gn{>L9J6cY<86eA1W zw=Wt#D@*%tp&b;FMS|DNEnw6yx}rlbA0DP(KqVEx||Tl8pjwX5rvO>InE&h+ozrp{Tl%R>Zy@4$hji9-Xy#c?0o|XQ$U2y|TBYR^g zwtpu19%pC&ow@~-bNYpH{&bSeX3Q2G#9q=)qyV8`Jv`+zYT zIcDfee-A@v(H3~yiWoGkHzBx#j1kOl<$lnU1`w*seUNbIYAl)OJBh|i>;!}GacO7$GQD3{Pcr1!MmAcJ%tZ`muI-qr3(0HA8wQ}&G*wp_(jXcAgN;3DW zZZW`riX*LW>PLJ`6z001j=j6Y-$Tb*k=t<95z6%hSYJwNHEGnhpK(CWiN^ z;h&y1Mm&h-{BzzK^Gi^SP5?fz@kv`wJrhs>E zRB>)f=}98oiRA29S%~3A1XTuljzxp2_&iQFHEzHlgJ2{^M$6>t6@{>Cj8yUE`ehva z#~6G^_)ebGKSW}r8hF#CG#k>MIfxwzR_ySlFNKO4Ra#k7EP~D2E6t^PYK9S21*(#e)`O~G_n?diu`=4%$H3MBi#76a zln5UA5jzm*kb%jzz_4f1Bgaa&_)zjGHJRSoRu6^J5&5wsN&Ti-yA__|1T6HwWzBS% zbWUoR=YMj}h4ND}9g^+=r=S-FBfaZHZY-du8nA6f)2d zrUq1vCSyTn+^DqWceN&SMOuzB+jSP*d)GyECT)h)V=Qq+3)Xw?8(fo9uw3idZDS!XsSi3AxCn=3Zvea)N80GR#j;1tE)VUR4 z*ECzue9-#5#Qs!`T#&gzmWQrUghsj{L$02)*9zKL&e4$LC3o)Dk0b6e-?45?`jx{) zOpMYH4Xw&#oUfgLIq0JnJ_p{$CnJl}#v)jb+_fPg!v+q_?Vl!FBOO#qL^RYsAQ2D~ z>?$gaLJX=6RbC;F65RS{K}=c=%Q!?=ahy2k=^Bwkm)(^!M;&HPk|WT;7Zx^A__=w4 zgS$<8>@4rSwwoA@%2X?6#rOuTw2TzWN4|;p8Y3CcJ zR}+?jnFX5c^RJy^{R!qe&Syh+q(m9e1ElW<;y|E&3cPpW;B z!hdvhe~IG1OF19UcY{Vx=ezu$wVv)T-uy2G9reGOzxDs9{?f&NMS|Z9@()t{J)no8 z``?IV|IG@Pe>=a2(kM&mni}Z+qy5_yaJCncv;U@ue+KiITN~@pLH#rRH}d?`q=ov& z?|*pB)=E##!2UZMnr{aDrac2^dng(ai*H`y{ZGyJpIYQ!WczQU9#CjY<@-Crd9Cx!p|`QMEAf5Iuce;MxkTK&H~ z5b@+micF;$lg5V!%1p`%{9`9-~AKklN--M#@0j zOs8jYMT+zNDfmKnKf7K-kPWLVXnTNP4W&p6v*~YvXT=VCN~nE99Fk+RHQ4e}=X=!H zy?vw3mL)!D7E0F}%=Q}a&X;8tJj=*Q4<2~OT{CU`zM^L$)1|&hi3_-0r899abmyzePwkM2gt)uwneGLGVoy0T5Tv zkO&d=V+8b3W@I4qe0KwX(}zaxMrQLN0fE_T(pT<5F%Z0-Hq(Z#vBI>&(L)hGu@_x- z;PGP{9e#OEJ$ZGKgmk%;o+@svN@m!d$_sb{dZS?SfJ~JqP~gM+Z+;@mU4lvzPiffB zz1sDd#DcZRBoLa>7J)r@{#?d` zHr%;7Tix3KTHN%@RNj?r{>i1^q~^qfXlBx%-sNjhEmEG8>=@&Ss1^;Lr1O&y9k{Aa z5zj{}3plIhSDP$Wpq7iUn8}o0<&XJ3j%5%x6)k4j;kY6*z4|3QZ-?^Us!dpl>YrX} z+bq}AcEdyUK@ED*kVMvSk3|@p+7-HPYpIXqSacXdCE)VgX#>|<*N9l!GLLFHHRsV| z`}ytxkHpV8EaE+jT!BQ~246oa2e%e>!)75YNSm$IE%Rmq(YB{3!xCz3%uA>FlquIq zsDYsumw$R0^|@nRwOya#R)efjvHE}P^35`z#K+Eg3ft|3_wV35qr4}w)f6E#rhlY& z3~-?aLNfTf`A79@2}g-}NgL@_(Rd43Gno5%(c=E>Ca%6);cz5U8?w}Ms#{tCd~BiR zPo|_qjtsr)FW9xp&v|ASQKi-(s7kPs=d!A@@ap_{y7-zNt#6|=j^!*lP~H#DMtkErHg4)}oasxoA$17AUXWHlR9g{zKQMmwi?WMU@*_$23Tj3(LN zvyxXPRwsVvZB|;2>H_!gg4znj(CtkA@fMn(o6(G})*!j=-PP95Ev7Q-Dbi;{9mg{I zG^Vz1aa(swH<0tW4wSgm=~i6PeBtM4+|}u!=O*V?_h=+M#5Ue51Hnhs=t=xZ-Tkr| zjMaqwIgG{Xs8XH`Yk0IfJgyDBEZYbrbgSAJ!r8m_Y4!*cIlFdF+PFgI@0>P;_zJ}r zVDs$eLo1|AZvMj(Ej6IYKQ5PiRiG)E8a0eyz||(QnYUeiowTrjssCvpsvl|B>A6gV znT+y4%!UjvBHmCqN%To1OC)P#?m+243OTJ-Sf#9<^egXC4!mq{0n)U?wX9XmV{X=a z{RQjXw z=hr0#yv(}dMKJmO;*9Jl(*_lf|FEViLt`Nm3UJz%U8Y&t2EKtb|M1*d+C@J}=7^c- zL$O9F4=7m0Y*w#K`0SaO^{q)VR+%0Mm6wp`jrqXhmRw-6OY)tb&i$%N({cAhr%dXx}eJzH9O z`ql4@xEb0(>JG76tR?2;(H7Kw3Q7Dea1pP$ITktx*kIATp z%*jO>Fnik=&in~Fg{j?ak~Q4gUS42!{voSSHj;1dSV3ww$g!-dNxAPWcx>jQ`*T6dzc#WC?_AfZi~20< zROJN{`F(gq9PIh|2vMwrMQ)CZWjM)jGD093*oheP@sk;M$2m%bfu~T}7>p3@CNE+c zN{+-G_w_S{iR2$CL*#`BN|u6z2&l|g{Z(PMI5u=s4Paf-=x2$NGR`PPf~4 zVvx;(<%VuD=k30VvF2-r*0|-O< znJ^j(S0!m67$gTRt6T1Rcs~CO*|5-8cXxa0nwY;_tug~G_X9iCmc$tqeXvY~tWTD_ zikLpj6_`cPF}}XCt7HHejhPUuF0al=vPvR^;j9?W3g`?wbF_Ka`r00u`|_t^TjkxF zxs8W|wWojX*-}+lKC@Ku#XECUrh~&XEG0$1AY(bMkHVI}pE3iPZv*7g!b0J)gR+6Y zD{sf_8w-i51Q{$eKAD-lvHj*NC@d?J%9$MP&24*OdHFuh#SeW2p+%|6h{J-?+B^q_ zpdiO*osa%8CqrU_6r;`)mD;|&1#ripF$)l^$w?nJ>?`MBqax{v{>T`E`P^f+;U6W# zDnF5kY;!AH99u}zE%(i?y-!X#O|u!qo0+Xfk15Yiey{l`jmg5^Z+SR(JGFwdC}nXLt5 zSY1L&Wf%Lah0~}0)CE|M@>)uBYG!LmJakb!S@CCy=9YH2#l${??&}*M_EoqvxrK5? zG8H-=p4z@u3|WzrexYy$0nxxUcL2k~IqHJ1`iPveKL6+(t*F=H%I)pv058=l9r?h> ziLM}n959m6Ab17>cb-e+13ojlg2BYT$=TA{q;pVtp^|eXPxPLSt%at(GF0LAX!wp7 zmFgJ)uZAOe2p@Ae{jaYR5yAWL@D;&kIIm!Kx5V&2JWZY-!N&UiuYnH?YW0KAc?eSW zA2>TIJ1Y-jr9Z_7O=cc9lQS@FM@381GvVWp?k?siI#qhNTQ@ zD~4qkxr{Rc&Jl7_jm#Yf>5o^BG-R{{(Tuca4o3j(fi=Py*OURcY;st^;KC$aIw90N zwB^GT{lag^)%9;#HM$jjiHZgma+*Ly-G?X_cxzv=R4N1ji)A21_MtzboNuJC#oSp< zhpiUbB<9*(CtzmRvc>W4WAmE2X?17YOvkEGCc8xq!rzIREcv{fEi#;pLpyvQcgnMh zb8a$|Glq6kUf$(A7Uk>6I=FGGCCm|c%PuJfIy0A0L{o5H@6JE3R}V_gpitqoc`^C8ax`lS0@>Pa?Ff@qF0CY^8W=`-E5t~6aFJ}SL;W88_K|K{PtW$=-IXf$Q% zbs0x+VYOA40qge2sYq|Em1T70{JwV*#-;H#E#y1>c3iSmEB1K}nO3_uUB=0>^{M?f zvzBT3rTf9a`f0r4#{v5kYB{WvBBDh&9;m|0)-uegDOLHC5`#+Bjp0fr>`{T?NR(Sz*^CT zExj9QLOve6m=R1;w}In2%J3BmtW!kS1!mz0u6O4yUHCfdb(Gm%5p%{BQ?j_!yo~Vv zlIi?bpPNptffRKGUxko}AsGc3xotwnZk3;ia~nP%l%Gg#d{~bQjmtMt8V7p0-Nlt$$l4L3{WQNL?*L5t({ggex(<_bRdL-@^} zosnb?g>6Y!M&Sn$rh6LG5*%tmbH)4~g%U|v9zkQUT(tok5ycTL1+L6@dDg{|NRn2u z7>UePJT%VZ0P}lmTSQNJ=-TF>+~xiHgziBhUeVRq*hYHx4u*pi#oZ|@`S~*2xuX4P zxSe9RYdryJOAq=W zXnA+ewc}u3)AVN0D1dv|P3rSQB$mu+hk7{T;w~6NB&c9H686WlUC|}Y!B7!#+q+^jJW@ZsO6P$vS)pNdl6h@X*zlRPTGF`ngCNEO%#C?G^CRb$ zoV~sFO?1egy9vZ4e)}kagtFni*L4@u42ebEJt!-vlsFY?Ucr6=evV64g7sM3K(C;h z6zEm#IkCn^+F>0v58%j-tL&DdOeHD)R%%&fqjKbFlxKZw8H8lowPrHG*uy%l-5^@0 zvXp00Z48bo36-*LQ}594p&9cX*s*_%r9!V~C2l3DB}4)V8!)X~%aBYO{#jn-HWlsY7sE#TOYeWv{y$fNM*}wu7V+UJqx$JIDj5feggI zB*Hl~K`7)xiTSxNs1mQ{PsQHhcj_A=K6UfD_mI%TtUE@al9EY)K6Z-CsSe*mjzua} zv_&};LvXfVhj>=$>bar9gWuuCBQeTVLYxrxde5>wEs2-Z3F`Z}=nV*bsw^?;a}|2M zfn#mDb#A>viXuT4+S$Th)vFMzv}4|*{RO3@;)S!mfF$RpcpN9&WNyMqSz+|x;K0h= ztgrklPs=OuD8f}G-sP&-;`yZmD8*w1hM0SU%fZd<`;MGfYKp-0ihAA{L6|KodKC?l z6fky>SmKK7%MfhWFgyBL8zdk!{RSxK4`uj+U6N4HP^nmvK%rn! zKXoIGHMRZ3!xW^ZWmm6Dz#I8g0qB^X*}%i&&-Akm7Pm@@W@5)SyF52N*UFk@x5ELC zl^aU4cJMIZ`{7#=gN{8~aHgiZqk|O|edv`M?>Zw(o$!HuZ}HzrG?r{lMNBL#zij>R zcODex31DQWjENF)X1wa+IQ2lOi!>(r8nPnVoY(s)C>5ilOp4=b-3*CD>^D*eY9m3i zNm#s_hwrK(enz$B5SatZDMO19X>gW^mSU=Lq{0ALvMEc9^_UKu!VKhqpUvcc6s#N| z7T#wY2A!mF){fpbJ46pD@MZ?k??F^IR|&jPOr0}0_(XLc7Rq*xKLpl2Ra;>lu2O~4%uLW);C!5lcR&3S?es&NZ7wuPa;s4&b}T&8C##Yc4gDz zT&Zn-zagpp(_KA0`+)&SM({)Ui2`cX?B+y@&FKBIbU=+HB8#i|V;Ay{ruDT{ZzalT5^^y_Wo++vD|C5y%W<@z)%7lQ%m12?tq`TL4?H4ZxL zQJGERE)`An=JK>DYPNd4gwhO~{;cqz4#@A32Kfk}8bK_ZQ%ye^C%@dGq!zqM-sE=E zzF`}9n0(y@oqQR9?O0@AdR#_sCONY)a*Z)zM4P=G$)Cf233G$s6!kgNYxduH!O4@bYW00iWfkKRHR>z~u6If2Q2NOF6?qUDPiH5?K2s-6e zQxUFzNjr5MA5tt^~1S#3SWjyg_rK#>z)6(QTnEDHTd_J)F2E>GsWTZB9y<$|m!XfvZs z4tMH{m`_a#XWG@v)UcZisK7C=S?~lfr@w1BC+kr`h={ZQg|wc_EYSGGUfG_h&Ed)0}T%CjsQA4_u&(6 ztS&E)W@%|`<(JYwy5fdv?q#@tE8~II>4tD7=kakmW+&St2Dfhgv3P@HCiuya6AqoF z#nE9t^X>vaXjhp+ue;`;w-o-+maYOyw?@P{RjG!l?y4tYdw5)@h|TV>u>!pOCV(e3 zlRPT}dlbd~Mn5aJ*Dy6e(41Pky!G@4Ott?-{o>B2{VLgxF_)+*1g zV^d)CHj!y%2Ua6Y?tm!ENIHW6PHiGeKungg%aKH6XlFwX*0IUqL?Hnyq`LHp0$Tb< z{_ytbxYdiB@%zw)OAdd*!T9TL0n<}p3xxRM_|4@pe2 zo}&!iQ(iU`ulxxk!XYaun6jbXnOGxW!crxr&zJ8(Tie#5fKX?Xu@!W~2O@CZA8j*YqdGmeRnv4@4zPS?25N@j6^7v8W`}<(NI4e zo)IJwr2o%wM^D`ffis7!<8WcGrVd2~V&zFQ=TOn8AN1=aBQkWcSER?(!5yrU5{5|q z_}1QGQk+#lje?}?GUKh}YkWg>Gsl7cWI64iR2BO9b;gG(oJuO@C-icZ&e>8J&L5_l-i4V511+hwg6sBLinqpumN|(K6MVbM);(R%xoVqDo^C9uZs?IEjxWF6p!~5?IQF_(b$Z+5)nU zKB!#lr_&ImXVv_%Cj@dB6UxA(KwNDGrXeb<0j6~kD@9%RrH3ubCgIu4eeStxxR~v3 zUA!syRa*iWaT}Fsoqzec(8prHVnkrbLRxwTkP)f>*0(;1p25SWZ?+$XxON&QZ?(@> zhZ=A=iOyl?ao)}!aU+SlW1Xt7u0!7sVk`3h`?d0ier$Pa zc^E2m^lozJ^>J`1RMl!b7N=CcDx1cHr^xK4`ZXU>e*4je-gZ(sc!la-=e59G^ZZot zrs;M$a5QYI0{$x&d}iu$9T-sE`UK+uALC@bJ50o5z|^KnZdc?4Bjo@CwW{w7%>e8& zy8QKmIj95}>(HLv_%PH23!BcYR;tT(VpO$)vK!^kuaS**f%lW5kcQZT^8+|7OER@O z(yRn_e&gl}&s5{iyE?hFVS({I5=XnB`DFl+2%LkDQ8^zyKphcnyI&~NK(c?X?@Au; z$R6%So`!_I8i+ZUOZDtiQFtynK@gvyFeMw}Mqo@)uLr#YFo9!jgstf@vPXHg(|~>0 zO9l(0uyD`Kg()S_Bl%#kx#Iv41FbNaK*^!j85OO+YQ=0y=#qAa4JE_a(*R#0mHmj!nz~^Y z^gXb)JcmjOEbEPk_Y|@Vw{qE4+VV z1FpsB?zI@fg35W}l~nnZf0jFtDx_J?DSVqPP628^^l?wxiVE~fckpwlV=}O4@HFN$ zROrEMq^@CBy<&-lQ(-EjG8b@@O#qoG4QA})bg58ckk%KdB{LNVJMlEBi;q<;;ESCl z$BqMu#-9ZY7eTzCmFSwM`#{X{91(mF^Ss1cXiDIio)mz!DvD6%!8AJ=1sNk>vrd_l zCuz3)HO;HFxI$umD5bi^b%H-_0GqBNG)z(L( zO9zaIA=&XVZyTxfEXUld-a{srV4^s4@!sani%9c|Y0udSU+Iy{e-HyE&YM>k$MJXs6#d1FCp`@xo2s`#I6>?eR^#E`M?o_ae#og+T|D4%X5&`i>6e!RhDFl_}|xFlprw^|l7*mV+k9O_@JLEcf& z5S<4AolidEL3dLw1V@gFFvS3a$AWp6%BrruSYpj1e?tCbs@~eTF^~_dP-q;u;F!?v z^m?Dx$g7eId@`oHPGV9(t5u>FRsK(0aH0y29@4+(9nf( zdpEp%B)t0+NHT(!Tx&V$pRK5PWON2`U%>52>M#bp(V`lV*Rrp@3cLX|Uim$OD~`E! zxdj%iDm6}^_A1q9MYVguQhR?Ajw~nV*VSRM^EncAXb>%6%!aJ98I9&L?8BR&s#B=1 z4(mHU;!01NxM*cOo>t7gqZf^!mx?rDC0-?Q#~0Womgdmd zB*bkNS-pI8tJpc6rsi(gZ3bpLz5|`&W6lt50A_0+ zl^7EQv|{*c42E@lPNxXNIt8!^Lzv-0C<@~`1+eP-aN@^*4y|ML879l_0iY_rBqM9s z$Mj}(PO2ZKFtYgf{cb~oqjf^~D|*g@6_E2<@Q5;CxD&``%EhBD{)AfuTSP+Egojfq znfoQV5Orq2-d0dPz1>`tXA#ee@iO`m&rw`URQOq$whT>~;p955uGYTlTSi-%`>JDt zvekj<{j61CuKLxM=WDLX!`!0KF3-c!je>rEw9jcqCQ@T8T#iK=^!i0 zRtN2%ml{(K+&iXB2m&~SBPOgp{VIgx65QWg2gzf}y!z-SKkKK^Z?RT;GUXp;I%STB z5gc>8hp#eQYA(NtY|C2WI_8-hT_>-llBoG!lvP4nkqe4~s9Bq<4Ryc(T~Q$l!7wp-D9}%4##r zM((mZ4~I5;vfSEGBz-^K#Q?r?Z@x(4ohwz?W7$}IjMS0w3Z)?Hb|jqBmk~i~bXMiS z4a+LPMv}@WLp&LA*KZl)Jg1)W+0MyEjmJmr8pFNL5txYgo;$Q>QzxEl9Y;B$pJb(# z|FN%VAFVd;XeqYVLS=DPU2vfuzI^8bM?PX9YV@9SL8 zkFZg6xDpNr8cNJJ&96k;Od2YbH|rkYC=K8!2jD34k2v?ly=w&DBQ9f+g~qsv6^WfH zeZHF6?9u`UZrTpp%nPjmOLdeT(pxy%GoQHsXngs3DM69Nx%Y0vDkbv*F?nXQl8pgF z(}WQ%jK;LGsAxPx;Ew2JKgtiR|C38kv?E$0bh-;KX2PBVpxroMG-^Ky`jeHTGrr>{DnYV{sNo z?9`}7!*=Fq?B*DFA`iLQw`o{=EFHFS_)V~Z8tEG9a3A9V zc6NZQ>h>-@L!i)3mVCi>;7;S{w|-(`5Z0fi`r)t@ku z*OU_Tc_|8B#9r1()Z~JL!_=!w}iH{X9n%PuD?K&O1_+ z*L5^j@6{i}nS~B!X^xsrl_J7=0ZrxRUv_I}TA%3$^hah@osVy5njWA(+bTUe}{dY67yd;~#1V4eIs0b{!ld3P^$6)LiR}WX zZHWks7jIU{CwdI7QalciULtUqk=k6lc-uR|#au+quVRK+fK`p3HEcU@doDjCxet|b zmt-P?cSs^~FP+M~f6e11BNBzOB5Hcbdh8TVWrOV}u(5z%s)Zq$W4MTp;=mwX%#8#w zB*Y*S%pPW-4VV5p!UiG5t7gN_3$Up;)+=j1=^)4o9A;qw4l>- z9*@2M7-*kZn2U}{Y6(2hiKmY`Re)yATzHk6^Y+DEC3p4qEW8)<#bRB1RHb^uMGeZB zQ~gwuSDk0Ce_DMTjw!dpUJLM=WC(oM=X)8rmdAlbZE+=E%r*XG8yc5zB$c{jJUYXES(S zU+g>a$pxV;N@wxp^js?v-$}h1#@Wx8cX&2BIJeU%8uoBMK|aNugCScKlx-K6jIb?& z5-mZcwfmVd=@%WW(XlfeERL?zcW5wp?BB;gUX{pR+9$<6Q7Dn+rpk+t#EmVYZ|dtl z1ao5iuq9h|^AQ>1pOo}1_0ur@6Mx2Gkdu9iF@{-yT?~KmYG_P!AiR9O`z+2Z?tseO z9qntGbPTv!^yHbWiDsf?EQXhB3s$AZHqq~!>xBMt9!;iV-Jsw2y}`V}wL!Ph(p}r# z+TGaQ*W2da~mMC#fmDrmpZvJ)L0Re^x<+yYBEaMoI#AxM+k?mw?PZku!au z7K%R3r015Nd^5>k&`{cIwsaEa2Z^CzvA&_dsKITAGx1Z~oLc%@NAF9vsn*3IPPnh` z`*fxBfRC=CxP$DAu57`*3{Lj#zgM*G@&@r@ax+S#(PQmg z0N+-~u@}x<_|EPW|EFL(vE)_M9d&dvYdf<2LklON=C362L{_Gr37S5POznUx#anJh zY|xaB2s{7R0=-D5+25q)NJ5DbOi=?!AqDoS(<9rQyhahiWs#HzLA&)mxs`Z`nV#pg63Zk$|FsY3F%88PP7s3j}nmmyK=+=6%NGv>olzp)c>8qF2ujT^`y z$$s``V%54M?%Pk}uFF$PmaEi({SJ}2Rb0rD%oaEK0B+T$3`ff(x~D2qE7zc|-nOdD zrkHeAhb7K)#`8vU9HCp5TnE(ysGkb3K^kIAYj#jn$1;@@_ zZ@rSfVt@YbPt(uXkh3sy158lvHy`pM2gp{f;3YQ7 z8%RTKs?PFTj<`#-yyns3rrdQ-oO8$n+qNlICG`?sTe5mNS7Ub&KQ8YjLz?C;Xxq_- zB(zFycpcEe<%T4p?rb)|_a|61-7SABa4%_yZ3cg?Zs)>_vpj*bOKBZ019SbZ-;FEZ zf{=w7JDD&HlAG=_EQ)5?oz|*nZnN*QGpmi~-6V`!YsUu$CzmU1w$QfwjjW3vlmr$DQ zI-3dX3W{lsA2(;H=YUw!uzQffvuK?Kb<;j@l|VzkywkWqVX+uW#yEmqqp=myTU^Qs zvTk|uTAJjcl$^Pv)q3K!WFqlup&QMk=848tgNsEhM0g=zR`ML5##BY_Z2yhyhh+8< z?Z+wN&#y7bR{<4NWQBwZ<}}cimBX9agOJn=`e*5z^tg00yEgnO7X>l?hgWw}}kJy0=hvxKl7p|f{9xvcu5aKs28`Nma z?<5?Oi`)5cT)A}toQd@UgDcCrYo=-1W(b9Z6?^UGHBJpm*Rtc6F)F}MdZF}&n*);H zEBBa#%3<=}!voK}z_XU3;m<(s3v0DBr3jB!v%=mX2?6)YpK~vpHz^=3wi#o0?Wec$ zh@|4hA^{`|%L}%L4f@`TqUq^uSk=6##|cs*ewvq zfbM&(>*tm>*DuPv5kX!S=Z3d5j4b~kz6MBnLcfZ`zUpziluM}!c72*W_k)Lp2H=;O zc)4EfDad9GJ*{s&oo}qIrDb`x53R|zUu5=eD*218Ub;Rw=vc%;g+SH=Vqn>`Mjp*|!q69BuR% zRN44Y4IMg#Inu?SJd!yQQJvTcor?(dPh4<28JggqeUxvXsiffGyOZ<;$URCr1ye_P zM5`m0lHN)=ST%)+7|l{O69{ySa^6d0D{6IP+@q{QV+TxnMtV+mReob;EJ7arPB~cKv@QfWht0uQD?qkC;2wW*2wqM*!Rdw9xPOtx>n+UCL zZ92NKVCF%wjThHDnuIVu@G-z&I!wA@EJ82lz6l~O?vY$;mSp_;;tsMCmD#>2?SRjr zF+%fZMpO{118Wa!*E#6$2UfD45t&9gQfX=(fzs}6Uyqqu=%T)2^wurBYb5UEJ=P$s zh2pfGf6y;sVt@*s-<-AfT%17XkksX)qR^F8Xt(x1TwYD;Q@Oy~9@);uXFMys^0Su`3tv6kERqQ#esn>?5}7*3IFs<4oT*>8E*?3-DOFfc4F0lo1@fGrj%M(t?rDt z6{AJ7!41TX=L)0}1*PnL?o+nR0-4Ub)lP{RRcwZyyBc_^qrKqDZ$S3~=$oriry(#` zunH8Kd{ra;M12(*Yv9(?I=xQILolm}yb%`qWlAfBIhzyga8A^m*^$o~ zd==Bnr2O?;R0xLUnDl+p;kgrP3`P4L6UIn~9Xa2UA17DX8HgN&o5Qaj`OPbH)2d&L zHtKbKhQz#Urtpf-N>9Q_JEMz-KLF|iDSuJY;w;pxbsMV&o)KRocvw*Uhh~t#Y)iLw z`(+|GV2pRV(B3`ZuDJP3mR#z984X)>%-6VXS4*jAR>MYkRNqW#CzGZJ~Wa=Kjp5F}ss-}+%yk6!Bi1^fYcPEmRECzYh_YQ0K=WP=x zc-_o28+CZRgdZpKU`XjxFxzctH|!}8pfP@|&^M5~JpH`=fB0c@A)(IBVBrbXew zUSBJx<#xm5D-RNk+3p!3GEsA!eYdUiZs=!=HyS@dE2;{Z*T1&Sdx8#w=GLuZOC#R* zsl6G0O(b4l&x@NrW`o|<2Hg>sjiV2}uXujMj5XzVO@TF6Fw29d^?O*~CG9Z)=G?#~ z&6)|`E6NenY+(z5$lmkms_={C$7XC(&iH!ahX7Zx$#T}3iA1qht!8kr2wex{OmH_W zRNGi>bVIRRx1ZYDUhse{BG{t`wUC)q~G_jgXe;zPl!}2qC;Jlg$TnsbZ1KPZQ{ZpYt-7SX0icWaCdo4shIefIFuUK8N2pCr5$OL^VR( z&_;lI%83MptY3Y zDf*|O_J`$QeVRdhobpvfvLjXHmS(e~Rrjq9pC$032XLX>3VE%;uweE>E-{{j$-X1N zHFI=^??aeYH}I};GtP^1@PQ;&&{;M~Eh9!mOYr7z=XXD8H#@QUVd?s40R44;oEA=+bW3msNrPC(ssN+D4Sqkf@c_0=Ubx}Wjx_}QW!L``g zr#@#@#Ru{k=%-tnpcB3zd4D{m_AUgd0bNXCz0{+~|HJF1Cw-NPWT zfzW#=K}0|iLPA3CRq3D;7@G*3A0mH*cG7-kGednLH_UGKW}ycAY10z>2wU)7ud0%R*v=n`bjHjrW97 zC(_P`=>uh8`R0j_12;2c)CG4p)~Bs7Pe_4MyiR#wf?>GCrBvgJ@Tk zLZ3Cw`7547n2a6!KJD?N)_s9|$*y2tij%5aJgU&{_%F)hVD>r_AyUO&wR{U(2?;S z1}ZR4kz!=Aa8~p9&g%f%Gn7r8TarQv$}1yc>uHP=_)MTvOk>3EKFJW`wLpwS6kV^^ zr6=;+O`%J8nN#d(7wX2KV|w%&bxJ?g@>9zLdOQTSxFhI1-t;wEZT} zWY~zM=ZN2l!FpKyu3y|D=yY<#rNI3(EAEssb05Wn5Aa(Ee`VZD{b}g|P+L;GrA0cb zwEhs&p})M2H%r6EB&@$J#Ou!&WJ%icP&<&qiV2A;SOR%-eZM!M`Cgxn4DhaS~@fXCsv=Xsa?vTDX!Rmy z2#T+F)^a&iSa0P{>!Ixz5;a{_T~&`1uvLlrZpNiw12udx+W_|HduNQxprhuh#X9{m zp6NG{*?AU`G7|2`%it`%W4Wbq=o_-CoX91o@ck(`?FXsojYuEI5^_ zHt#m%s$^)-LiWYb{Fh|B>s?E$nrG}m+QYU|xi^OTiwmCM)p1eTqi^|94N|OG;(awn z)$hEs_XKMDAzT_}CObO?@pDbyGzBvyvlkAoP=+-_Iz~!fGljPtXf}U(OPgIVxL67& z_QD{f(s1ILE3jU3jaQX}K|UvIlrPHfKxFo=W_pQsD6byHi`HueiYnym`COx0WH~ zCF<*rAMNM#)ppow!O{4>*74D_feXubHjuGQ>_BKDE+iqOEoLVw0E`c<)evj_@IX#z zrnP^w6!vG}2hG5`$9uj|2gY2$6pXFcBfpZL$$Q4d8JRC!o)vjAT18HVswQ8CU~h$W zF2VaQK=U(fAO|j43QW{D)Tg~HFX?K0#qo=F-6!5?P(CsfsCdbB;5VYV{W+8D&kd0| zP@AUf!maEF46ckvqqO`BnFb(Hjgx#92TO`YPl>(~u2KfY2No_wpBS6K87&&lUgcs* zx%qnZXAhP)Kut9Bi2W0|g-uq;@|lZN#};4`Krt_p21bJ19RfS0Z8I7A2HT#S5I0DF z@^X*XlqKGA0rKflqFU;aX!jhlm5Dfy)*v;yQj};5gpE@O@`FP-6X~|9L%jkJ(XOhx z3eI$Xdo^zNZD|W4SGD27F!D@EHcSMQ_+h?lXx=hnc?%!ykDCa{BMb+_hnioMw(PzL z&*M>q_mL4KGW=^EdxBS&zX$n8!RfAQR7~)&;FGhOoDGyxkf_l@^ZufE+zFLODFfb% zA0x(4iyvLr_-bX&a}6}<-G85uDycY>KcrXOn9)?2ESOE42`au zPW!g)>lD4b&@|dL>DMTYC)T?Hd6vw5u+GS19CPSOmBpVaLzh!IXo!kRPJftW-cXG6 zt@|_+{;>2m`Wv>ZJi1qw#=%-3Ramb|HH=KP#}mXM+K)Z*N_EQfyilcIu)Wh5J;e7U zpU_-qq=df{ws`wUPImAc&f4+dEdD_0-pg>O-TH$)@~hKRoK8%io=L0B9Vp=^;PWoA z(1&YHT>IFpwV*my_GhLemEdOLy~M`NravfJ2%LKKF{yxCT6>Foc&4b8W!0% z#fiOKe|2!QRD7+7HUquMfH>s-bOsVQ{W2IH>Hl65k*#udvT*syD_^;FrsW&)@Gaj) zlLdjctXl`I&bQQ#g+ARHLthG{PSEDmF(Ul4SBCMy-R$BkH7l%13C~dKcMy#2KAsOb zR!$+L@A8v6Ijtj>z4)^R#`)ig-B@&E8lHr6EK|{&Ui`kYv7A|bo34gu6xENhEMD|# z^D7$UcsbBxx|2iOO!o*;Z5-cl(+#DmpFP8bk2}16EOtELOf#JM1}HVZabrVCravw( z^MJ2(aH_(cH0QZL!t~1WfpO`Wa1G5P1-k)c%y{?M%8iCu>!XSj5qx{X=y>||{>B5l zAu4LljoP#wrm7uz!iyloakVSfdovsgH6kfzEvFEYs z!|F43^ap&9D~sJEPIgx!)|qhh%p;db_vWCs6Gw`GZzdwh@~8%t6ItS^Mh&EYVuTot zoEAODdOpLDhh@%b<~(v^oW?XPdheo$ykC*n%GZ6oB5?w7QSF0k3VJL<(RgNO2l|n` zKRhLJTu6PVKl?~@k)7-d?+06HW_&)5-#E+eo5wFV;=J}wvOhFI<79&_=Qjj@q^ssy zM9y!>99q73_ry`D)hCp*f@Ui=LYAV|6qWUh%_4aF%LzL z1|K>j>_J2gdOkBEc*^3!x7Ool{Ji3;9`EhbmYwlB`ie86#{01t8$~_k%P8v?3nwdE zLm|_{{YafH?U2O%3z?FSptcmt)D^%<6U3t9Qy=SRA#^~{D7M{)L(Frzs(Cq#W~M}W zQ&w6+Uo4mQ6;5n>`}SzWS(zUVsRdT76>5KZsb)f~HZZ;fXMk~vm71zJ{i68x%nUG1@o zjwBk;jH2yb8$Q@rFnxu?Wu|up$s~r5YD|1bQwY%bL{^*f=B_zdE9FbjG)fifsad~V zk&HVf`C@p&Aa#2(ls#O1DM+kEPEfK4cj;4F|HG_%T$AL4?@{gtH;zcB`AwCRAiR4M zOiGOs6w1L3b%^Ze7y*T%Cfnm?U7-(8JVS{!=D07+V8F%Y@n?l>6BuC&;4*#;Xzn$I zcpQhx9vkPZ@g2o_)0ECL#29uqZPXnfu}{9L-M@62<(jlhv9RRNy_PfSQzdbmF2w+j zIJ9~+lnI}^*2dbNJIRCwI$M|Q6;=XlO&3JV%&uU#o$@^w_wlLcoWkjc;{yPkXWVu;Gv6Gv~JOUc6^+oBkbIW391doo_Oc>;Cf!G~YhECtpMN z4rm_gr75q1$e)PJh4ti2bmqL=E;n%RPR+4lIkvH0AKl27&o%$OL$MD@g_xv=1bKGAmuVU(6)!I}(YY{P9B?w+P(jDb#h zKIwnae&|v6jw*EWwW0$Xr$aS|LvuDpszKQ@xzlf3dmwUs-+#4%=y8RQiELuk5z^%Ap zUdkgmdHp0sXJMF2mhj@VRo}E55q0uKa3gW{VVI=$(!F@T{RtVOIM*9%Wb)`Axfyc?}ZOTJlj-`xMP zA3H(O+byQ|3Uw8@0S{mo*+P(4*CT<*sCt#q4W#YHL3XR( zJ6gKFXu8E#e0)M&D^WTz^HrqZ0`~=qM51mCkwG~xDB)`Pk|>Y-3!)%ia-pWL633H> z-S85wnI#R$`oVZlMBe{@*ojxS*%iCpYHC1xa{-!5jo(hz4md}oLfj(Q6c1wMU)SW$m9$G>ca4pbR|RzFh-5tSKs@H~X3pVLqnNK` z7N->_@wqw|7KPq1{Akzi+OE(pes)>dQDSD6d@5@Y!7nJwW?|8p2i;rR4dR^%w93@$ z(ivFZ^xbtQ*>i8|-CL-9_@k;^t+3h5oZm2`@~xki619yW$;76>;a&93S4V1exw#IK zS%sd3UqJZSpS8Vr9V2+7MNRoGmKWu}A&bZ9gtivO4k+pU*YvwFst z7oBKBI2aXX1vEyHD*WLnnvc^r#Ds5Mb5(U!lUrQ~&4)Xs4_BmDAgb9^NWJ)m9!h_} zzt5L&@g_DjB#7|f`p#hr`WA)qeY$L3tp3D1MeOjk#>Y7U(E$}M;qE`I4XcsljQJWM zZs-M$FB&auLT(Ia1G^q{3q?|W68A=i?G&gVD%gb?zYVBWr-t9^qpP6+Zp2+eXCQYO zXQP-^TAnpNf}toG+V888*iovq4%}f6qT*!WqR(IuCcVd@ARiO637CVm-x;7vQj60x ziz3lA(|ufy5>}9qy98?FYb526nk%D1&F{jn>U!2JR$q4=Ue{?^Gg!HQq-mpvR&%1^ha3Dh&ISN4k}53b^j!%~&$96H`d*a^B`E zvSccB&ms733x)g~Nq;B3!g}H~kS%#DKEOVeck7O47rp%4_-qtkP>Gqj?e?@y^;PR9mUEsR7sw<>q8YN zl^fW-6DY*AkM&hlnXkCy7`w2N4)uNXeM6*lXx@oN+sV>dPXw_X0<)ZU>lScP(_WTiRS*%8LRrTY`eg$ii~A1LE2pgh z)KJu@(CW4S5ji`#>!DX0Ju}K2E+cm-^OW(Pz=Dhku@9@N4AHZl`_>fZG0#-BCLUJ3 z6_#LmT$eE?fNlDup6`!D(eTiQGq^H<4+EE$H>Z+VAHkyO>^@e{U?o*0qpU?eDr`PN zw^$$E^@?(uVdA6@+h)n~qiLQPVQzlWy}o%XCkY$PcV?GxHuvBgyLNpW)BG?xZ!@?} zmF;0l?^Z@b+pQcb^wXfy=9`WIwQO*{9P-B%74j;LL(Zc3uc>F(+^ycL%@mEjnI8#r zO~0M{z1S9}`Pt-@HM_&!$DErrqr=WnBw4-u&kjs3&yCAV*7tX>;($S~6_*`BTjiRs z1o3#nv-lJ@1{;ajL+zY7jzb0}DXy7hFg6V%!OQA*o_nCHQn`6P8Z~nkrOQdcK!(G) zu1vN8e;fY{B< zS>A5b?3=`~pOGZ~G5=*5I7pqv(0ASAZTiiO$rr|HSMvURW^9^LZ*O!rXpMKDMb<<< zgzf}*(eFcWd+Tb+k4LGZw7gcPro-rh?GSofeB{v+`!nQXJ{8I?Z=;6~0f&Tyy4TtWo!$(f7`o7i9Grx!$eI;qRqsyquKT$27DfMVB7<5((~C zJUZ~}$E8Chl5AwjM?b%?k{7%^L;o%-(phV!_MR1Z^uFz-_x2_{O);RG@qM#ClfNtNKryeB zOXkNFAf5!sr4PJXSg)~-~7B=fSHZ+O!>0RjPQg< zcNxu>KtENIEP+Vr9ommnR+&v~@19<>(I3g@9?TIId|*N#gode?Sv+5bTalIV zrlvKed5BvDg1>O_03NZViK3VvjP4*;@YlM4I=6jxfdRjGhu7{`E9$>& zV&NJUS7Byr&~%B_+>`ct7afgBu6?`7S8^<#xaImE_5Xzy_D|OEZyY(*>uTr9IZh7p zf8pc^cE*9e2nUbro{nBW>32pBPA&w|lvohbkYg9F4& z36wf8VUdKL6M?}1h@>O|S0@30o}#+LmvHz$O;mm% zmaoC#5`b$^hy(xvCgc|fgJFb(V#?>RKLnu+slUj72*5Q60kSA|odBG6u~+tVa(5t9 zNB~fKYdN?$ITLINCxt+fcJMF(5U`9ta}aocgx>fGp8F3b;9o#RC;|HS*Ma>Fb(EHX z!2ebWDgF~?cutJ=1&C?8+&xDS{%V7tXlTEm1nGaz)ct!7jsP$SLYU3JJ`fNb27|&t zj-X!}1PYOs1pV9#^8A-}z6Jf4_Lto+i(l*irHRA;;VTXY|3ia-|Cs|S`B#p=^Fkz` z(!cu>3jBvJ1S;`gH26O?39$G-;=#p#*B}r$>~{?ek%Ym1^MydcFsa`(FjQRP_nHYM z{}*3Ez<>K9?%5Lh;@;0&fw4=F1EKQ(Vn$wG2tsTAysCfRDE;2yKuJz7Z;;fnf*=O&4_StoQXM$CfWC`&p@i8zk2<7FZo?u{H z!otA7a)4Y0pQIJP@58{bQI?kyS9kffQcK{X@jI!aRvxRt2A>jBfEl7r+DvN~`6i5( z9!FeU$)9r6;h7(c?(%(neD%95ESK=pC8BBZmp)3C@LL5tJ%h3lc1fRF#1FEOPb50ecx z-?wnHCfiBK$(U51y59@Sx|78ZxKa_n|1cSMc7&=|uREUiSmDG_64EizjOszLpRE&Y zvW1v1XGvgSc#owrXbycxsi&vIv$H=4AiJnvZKOVbHREUTxW#e1ix4p>0qH1y-%tA=T`*lV|=Z;*ute=dA0@B=7rf; zWNoH$|MkQ{nXBu9xSH#R<#tig={usRPp-9Fo~1}!QEm&GpT@70Souu$7jMP*qJmI0 z4>fs(h=2%OAD5i2i`l&6U}NLC9T>sI8dLD#*h%Zjb*MyFGKC>;DEU1Cw<|AF2x*>V zG|WQ4a)`}y-c5ewtg4llqKjd+$>cW}WAe3@^a_)mWDo zW~80t5LaszR3#y1eZ_QyOb%fLI5;@)x2Vyi0d~NHTdE_<`KP8Jw{=f91cQztM+ZKS zH}J`xZ&Z*dpNl70jRQP1@PjFt&%xHp(8Ij==$bSE9Rrzw!<3@^NVe@6izKT6&Ls>C z9zxx1y;-O3D$gUFrw630ev#_3O;>AAAzzz6@&T^;vb&rM?0)Tp4tYc9!s6HyoUjwF zwdQJ)|GJS!nGOL@g!PdSJ;opYHA#BMFY>dlxjF%f_6?VJ$v^|%gHI?+n|$hQ2Z%$U z5^~WpHKc+!f)?&7^@DLaX=*|=i@t)6TWldj5^}n8RqLosw~s19dHfoHF$Rh-xNttf zYH?a@vG{d5hJvkHrsSdt54L1Gr4cZ^!zlJ%@p$j@As)#aJi4Xpt1qbY;yE^A58$W# zmT19lmKVmUji&5zD)b9xHvThlO8US+DcVrNl-RGF9`+&U51EDj!wQ64I zsraIc0#G)+uiz-Y3On3^Jc6aB`yX=w&>GY0T!r>9WsLXK7{T@ zbO>_3%D_i5@jH-rxp%fY(rY?IKIMrvkp!1zkV=PH7R>zRGvX|Jpp9z#clT0paJu7Y zA4578HT-6B-j*QmgM~^ONhetRCcvkr0!DHDTF=x&l!}ya-47jj2v`m7e3BcBXf(f(RpK>AC^k}4(q!JV}qb1NV9pIGwr5L8!eV!r157*(w-IQNsUYY(|2u)0-+(gaZyoMgV1&wcYkf!NE3VBJ{}MbnlHW=)9Z<1 zlBe+Lh<=5QwaF>?X+_tmTPVwlSMi_J&8afmgq|>PPqv6C&osYQ3CS zViW~)Bl=N}33U{^)a7GSx35_ehO25$9-9_(zJqn`C%a7Kdp=XMsmYXy_|!IJFIu|# zsl9D*-$4|gOCsJw+2>Z`JXo@Hym{Q4VgvsMC~@?AwD)XHl}9zqWK6u^D2Pc3@zyd_E>N0fbieiiMopVzyj??Cp0xjv-;^4_)@y<-P(~BV?A6vj`i|- zY}{YqtJmrIKplJMdI|P1Q{}Yq+QO3r2s4ON_>62dX$l=mk$TZca=P;1xe0b?qm zkb}JW%tzQ`mF@+Lvk^JhLcP-uF_kD!?nvLkt@b>PXQE`o(p8m`bSV?ciMg@!QnJmo zu_!b19{tpu%sP92GM-;#YXlWm5Zbfzq(LePn;mNI^^^yDqWyrd*cwExQ zB1#CwtjNsP-70_djzRJmTMlEV`~i_*|LkEW=_BSBE4RcTXpSP0^r-~J7FGMPsKM3g z?iL~cvi(Y07da*xh#0{eC5q}H{<{!jP9fe5sgA18RSpaVpI7Q^mfscs{8Z#8>D&U&J#A z@79{~{^5>^9n0`09}mCRet=&1@kCEKAMRw$$&8o49jqgrX0B(+I6$XBp}?jxrBh8R zu6~|1TP5~_O;wG*G4R<~h5f4N5%Qcw_%8IrVImtgbEl5S_IcJazTOMtp1!k^M=qRa1FOv!b3CqFhtoniTQeR!G zZN}vkiuM*AllSU0S*Sb{iKcfaM!icDcbS_5WA{YXI8QpdqHyuTU~dEV@(%-bcnu<~ z;I4bx=sjY7Q}4-GwVQ}HYvJj?r&PzNk#D5%DAZHY&mDToex}-%SBV(&p<76~)x}}6 zePySdx=rz(W@k12Ydl|z7KD9HCBoicx?cEmc)w>+$^#aDrA9jWzPYEnH~DRCf22+L zZ)`6W2M{;LrA%m}G;!t+oRRa%d5j%c>iCD2)~5-fx8AEe}tN7QZRv zMkL+1Yy|7(hP2oVD0S(xrhTZ2E&Q;jC|*GAdv2(`bjw#k8REVQlh{=4 ztFqEH0gDHXMCCL-Zq$wRsmB2k1NM2lB9Hjhv5%!a29dUQv-1RtC`xitFY1CAKeYLy zx85_8=S8%`kyOIT0VmllBIJ~!G-P~uBOH6D+1IayiVc>yv5_r-P{zR)JQJ}0kEV$* z1-oSYOWZJtZ)L16ZLS&eXSh{{>n@F7ZC33+<(V*SvPq{_n^cH_$ng&*sEi_QYl+dm z0!#_4nU=bIt4*m`*_#A&-({K3rU};A->k>SG!+nMG|qx#*E=h?<=PAv z-y-XbCt9&m>#UwkU$4uelVFi4KZA#O?UEj_Z&NCfmLhj*TrnwIGw38ch%sAMMN`ua zNfOF;vXdpTN>QT~LweZy%E8o<(eE-IFiBz`2uhxd?(;TpNI|W zf=Qb4yyBR32dXzY7+T8W>J9l4`u9s{cwi4C|>C4hCLYjl+H zfb#Y~1FukERy^t(w(cg>WcYF2qg%kHGnDwRLzRmO0ZaWSUjFxP6oL+4|EoS10=(MB zi0p;&fH-vS(}5-ZF9x=M1%>}13jSpUq#G9w^n&#NPJH+u1q;IXC#r+#{)3T{w|}=H zdiFni{anJoy9GS^$E6Zs{&?@YeOU8mWu&rwWx__m@vE1G)kcj5V+o4Y0*wjyS4U;heFCW4#W*Q}?u--nb)2Q+%gI zK-@9W{c38Eu_0QUDcSc6)SkmA_Re((RO$2B`f$=?LF>&YfzugLQPIgOH}5x( zIW!fBhl-}~p7s3Az1n?O^xaG2oDQc{HVvNz5Y{qi$1U{v@Ev+UF9ZT{kj@%qqfsnq z?a~F;J&T>J`wnX7%PDvNraNK}=FkZRh^r#n2vPF*mAvPoZ}*x0rf$a1k+8ObUqjk6 z_MrFY%X@OT^2c=p(yv$GjA!I?3t9wX>js`(w<-#Z$3*|y4ee;m={J)Go`F)mK*|D? z?n25i?4L_s9eW@X&p_;RSlrlKe5aSLy3!h+_c>6ypWbpGA5U4ycvSk1G|wlnAD zp90K%d*2RNQgHrEUG?Gn8|#FeXcN%hh-aVkXV?8WO|i=ty6~9u81AW8{Pxx7)q}j3 zqP#m z_N?_E^K+{7tPV9juJZ7!a&LGmc9rL>j+5``2qa|jgbLr!chS9DO2TWqIR}WotXOC2 z{LKj-2Ig0Rmp5w||76GJ!J7T=uB+?waWEH`e|z=*W|_snuPgjT6>*#XP{5>ogSiw( zp`oc)t!I_%e#lR`-N(b?o(^D z@zTaiSIhO}RK826Ys!c@V5&G{Ch~K&w|W4!{r#o@1L7+j=AccQgn#mpuS5%Zbd8#) z>v5uu;>Ka$togkri@*ut90Wx&Dl#RdHyM-R_YKdbpQ59^V}JSqai2oxKFns_{AAi5 z_93LqI4iXI`9kS&E?i0Dd12eo0viWMQNWe^W*;*jo*o(4Gp^cU$+a#)5bZBEcUDJfa`BWA3}l#Drvd`6-sG&Z^Dwj1Mh z$-G_UU23^68RRJK0+Q%Bp2tr}tUG=^QrZT}>c2VM@+4XY{r#*HNg+FWcsuB?Ii{3Q z{Au$^Q*^e3uBE%s%17)`_I*|=myQlT)OlxYGW?bU6-(wLC~)2m4t!GJW4ubT2cOs$x#dwDBybLTRG!jZ4|*` z7`I!jPngf=cIU_Y*^om8c7uRJh=ygm5u;LNK>8b^D5q_#%Y@$!j94*0l;nn?3w&C6 zQ?j%OhqR&Mt0C7Un#}Kow zZ%&I(Vf>L?u>)M7*qY^>{Ms1zu1OY$)k!6l2#A@}1LE1|byo@x%H!AkeWXflK>K0J zI_TpAk5?6IDQg~1H2(O{X)vR{$Wbc`md{21G@%OMWCD++;w@2@9$!hcd(@u=e9z}+ zbSt;TqUwYe(Q&&=@e6S=t7iGO0CNX<>>Z^6fC2w9C z!+iO(Poax@)F8c*&M)ykXe?VE?XvIUh=mdSPSgde;0lZZ$7BmZHK`!HjC3VxQtL04 zpU_BOqs!cy8Wl&vV=h@5dgO9om_N=g4kG{zMEaOYq_pnTEj*o4R8?i=cIx`ja4-q% zz?cI|!p`)XH)uM?>3ALIK3kdwxeENiSr~)hwGleQFvk0|$re7M5zwe=k=lxF!Te4a zw_IlgMA|FHKE_()?E0_f)yJNZqh_C;2z}xCwd<_Z64o&JMKkDOjjZ*+S4BaR&5EK^ zIWNlIMi)2Rcx()ecCWIP?CMj)poLp{SFTIQ4KHM$CQOp|<^xN=1?Kd|$Fd~K{2{Yt(owE2V$W~x{7#MRzy1GBK zZ=8?uzWMx#P(&NIFAe%!jt{Upu) z*~`kTZU0C$?+v^1-6Kga@rHiwp~o-@x!a#~N?*}Xd=>hgbTBztx>-cg%*~S=lb8+$ z(#B<%!`!yX*}syJ@}LydRi6yzepnhleQn?0a=ZJWgS)HTp5h}Me)vE_?h<$2i@G>| zN6kouHl_OuhWzb`PHt^-0_~l>sG%zLr#2sQ$L|VqJ-@)Dt(=9DUQPMkYYQ+dBQs2 zZ6(h^&2ICQO1^oR6^3`2rN`Tn2n@04A6e3&jY)+reH)j^P{*)++z>ViE*1Pbp>^cv zgTVn0x&yPf)M&HXSbQIwf|4>Ym|$Wd|5av5@mBFw`!%`|PCwJl@4VBlIOdM5^MfGn zM2=ozXEr^98#zHD85z$Gv`{jb(sD7k`_nhnSTC(NFZzN9+jYLL`blu-zrHey z!t3x^T-Rs4rgMXzJTE>KH?qJ~eteB{uM(VQ-Yuo@%$UlQ6@MKh`rcV9-cng208~1_Ab!b$EPR*eiu#|VvJ`^ujgfZfLM6!Od z8tHi%6g$f_Hf9zV_kn!9`^rV$5SCMEm0UG_q$fxDs&&u~ed0R*i^JsTu3#sqkiIj*5?+(|NLd_`DW5YUu4syJQ^? z^IVN!kW8mngDan2J#;mGb#+DHH9FS5I$X{3h0EhfXkVGxI)l%k3PDJq#SIg<$=s9t zah&L}0>Kk(sKonQ?y%pT_2f#ik&LE+BP}4?fETFB%~4>pzjGe@`eSAEsXS)P9dP^{ zocck>pFuQYCD>0Nt{2>(W3mU?CqEg!iZnV<$?fRuHC%AR(k2TcBJOc^UfT}DE11j2 zQyg)>)oTv_ZP;{`r)&Y7#wz}9sJL4U>F5)x+Le8^*M&b&cc2cBokIOglYGpt1SLBX zuV8+CLx1J28-z~P)p(TP4Eu!>eL$;Bfv>{$*4EjI?wQ7kH5v@C+E+uW>SwG9!mH@w z-M*Wblp$vW52Wa@W%Ou2TGeyf?iOpoU9&dYL#eHy&lEjUvVg|vN{s$tt%&*&MBGhBr!*B0;^V9mMY z*xr|aj9Zd_Zmg)o5nm`Y8Loaj_|mZQ`^steI`@kc`-(La*}A}R69>*oule7UZPt^H zwAv;~>5+5h_ihO`oQ9q7w9Y-3V~>vKwb-*>OBAxHk4$Aw<)>%Z-x}M85hq>}Nx1eN zMtf#0suay64;)tMk+=cZQAqSJCNcU{0UWB76VGVM7^$=D^QtE*7{nAl zoy{igC!E5MC2eZXh7i&Px^XjV%6WB9s}^V8HMnQcGE`BX3Mcmee#|Fjde1~n4)c>H z&ztQ6MyUBmj20p~DSMwsj~-$#To%lI$r<)!cY&UdW6C&iQx5yR)Fgt4O1N!u(nC*| zBkX!kUPD8KWc$X}y{ry3U@Xr+m3U_wi?9E0}v9ik+ zOMM`G%`J%g4h8V!Xxsb0Uawc4K{?Ir=*Y%Xke}tJNgoAEi4_Ext7;Ybz>a=gVa!J0=2x)O@xGuYI+l! z>0h|JH`3FD(eW$?iSzNHD|xgx-Z3OnH=_khf89Gt@P7QjV(<;BQU(8j&}HKgZ_$_o z;LFuJeem@OTT~nJu*L{)KY^}JYDrgEX_e%oB2t8IFE*&WTcN0VD1%c{mu0l`(e3K^ z8pX2eyZ2>gqxXnnia2y^8z-P-`wtBw*x$YDt4zib4sa)QXKOr2zlIslL8-z?vkT z0wtyAux=UiAW@o8NiOd=rV{z;Tf7%?6Qr1#Jyaj6I+~rb`y`Zd%2l*~ zZtdFm#IrzE#Pb&H*q1?og47L%NS@iQIUz#Ub(ilDpPk^8`{gZabXIU+mGtGxvTWQT zV#!oy5L|Y&X{n{}??d)m49@yRT)ReS2Rhx?<6TzZm&s1tYU7`+go38^j_0^mk+Lv- z^;0flUI+Q2HVO*i6ja@gy?xPSu(x|*3oByKM?XL9Ym~JG+1t8Z#xk#+r2mXgcMfm79m3a(i^LDfidA@Bqd#Xs?RlZ*@`kPX zG{V+gaeskAYSkv6+`jxam`OqR%}(`zB-WaH1Ddmh`X0Ob1xg6RQ56lAsx#98bW13u>**mbe2<%gy)}h?tP6M~dRZL_FS_9C{ODUO zwb&<5)xuB;NZa&CwMUEyyOlC9(`Ab59M^v~ztP^=IZACrZ0lex#-;lPNsq-{n4l+I zCXMqZnuzS{batf{Yu#EF>?!8W@DMN0-txL*e>)eQK5cdq9}B8`ndDBjPJBFV)*9ec zxOEq4vCAe^@F;WL>}?$fR$(CRO~SMv>oQF)JRWU?9Cqu<6~~eV>6r@jQ_YW>{L{Sx z+zgqH#-UFT{I(G$WF|VA=obS7)uUbz+G;wjZ$B7 zyLEPaCDfa>{?ztln%r}oNk2dEcT)7b9m) zNHj_pKu}09yXiI-o=A0<0x!Zjc9Q^K;Oq6B}tfc zW$yQH%a__J&pSO!7?g+$6tQ`(j+fgUv+kRYRqN(7`B&)9?`m5m)>2_*klCfrLrlLE zh{}pZ+?c5BDtM@_7Ya|{PcS`H>An>ixp+t%>`k>rs^NgFZTl8H5-e3;;6_>pXVf_s z#-}7E`C?3Zx^%_5`V{G6WhDNJKPJBjuP;LJ*H;j`Q zE6?6l@hK~{v~^9oEq5~g{V&O#wS2W#Yqj_uz`xdB9pm??Wh@}gL}d8t6%Bdz8vbq% z3mmqzO7?YQ&=0L6A@06mv))i!C`F&TaJ82^vc7U=BG?;sawLFsG?$AyK0J&lKx7s1 z3h1qO1c&K8;pv|s^h;cMl3vpL;QFPYZ&zFF`Azu`Pp$`Cf`Z~O?14$Wo+7hJEO`?3 zHe&&!0=E9Lj;@a3pbA~rIxMC@PWwhCDQ;vD6^8kHu3Sy4bQu8%v zwb(v&QjBp}Tp->RQ4b1qR~h>*T-ff!zGc;mjp8-ceh<$yrHA%;;L3kE9$9kT$YRX}Fp)rdTUOSNYky;4OA`*_!~er!hF(K@I}_RapJv+F`xO_!6?xgHo99h{Iw z+)Hn60I|&yp=tpKKrc;k=`!!y4G>mn@&?UQcwEoZ6=MpV$1M}Ob3J;gLdis-QPSj# zKorL=mtWv6I2F51T%LkO7e>Z+lRcRt+r7a^(`K`~6Q%xihm-+8UP5A6!f{yf$WDY} z6j<@hyu2q|(#fr)Xt~oTQvwL9=oE+rcZDhiE>BNu`0>0No4B?4O?s6eCUoXdE+v$# zQ&TMnbQySX_umeOBooi#Dqd#f zc4yJCr)_Ue_tCAyZPSHk0|iK(lwt`YI)GbTad)KO zXvS?YssnyjDusDpXef|NuB8*?E!itgD&Ku5r?zrC)mN`=-F0)Q%>CNZPQ0A}=o_rM z;ha(C{aU$Squg$VFpz-AYm9My{fdcYN7Ah@&}tW;;VY-MdfOWkS#3Pyx7?bS_2n@0 z8&AxfFo2yfu#~dQ-^MC_l!D1VAe$y0&%$&Yi}K5DSKm%y`NbbQWO@!rtU_F839?;3&IwXr^8 zvgqksukS#ju7pW*tcn)9bv%;tHUKA40MH`#HH~1gWWGBA7PTVL7IrP-JG#?D`f~a> zdslN3;fdVVcl!oFo2swbK#13xM)N6QG3dkJxW z*s`3QzO;9&VZefdxD2KBqs5906S{77LvfFx^A{ID7g6r3S`=6T!g^v(OJiX(%(zlw z{?!ciWSi5bE|DQezhY9=Lt)1=YI!tG1ybna9K29PDcV(coFjmsM00%V!l7Y*iJ1LX zj)W2p3dDTeNkrrom_$~c)?_*Xv_0Gtdzl{V$iPx^#G$5co$(sd&{FD?0k$>a5v>yT z4&|<+y7J)Gs1_ZT+09jND_9CE6jK5~dQ!cbe0g@Jwbkca16&7{(+_#0S) zqU6$a8~sR0gf}plUYgz7iP|IVS=8wSW;YpxCd-KDtsDx0B-mFvOt~#ZTRZDUF?%E8 z!Z;i=p1_^MBCJ@THqnAv023-8D<-Kh7@zl>Mx$%#xbVJOr!_s)X0}@GjzFdv6M+Sn z4cm+D8bzwFhyx)r{>ZwCikdI2wuQL$p05TwGm|fQt`d9>$mP`GKP@$#{*s@_0}dZz#oUq2JX_HSgxLA%08>Bd`rYfMylESi&nmbcYhx#3OO>xhbrE5f?;%ms z{3Qbmt?XpElcjFdJlE>k?=IuRc6pXI8KR=F<%>c4<48Jy$nq+Lgvb*2R zBlUy>=p^FGBQWR)I{>D%%3{Rf@90pk$@rkr>(Z$K+or@>>5T@4H5~~@a8aSV03+MXdr|jzOso_YIAeDL0Lzau0$IF)j&J`Q#t;>iXh=w6Kflb(9QSUN`U*uTuJju%ezrQ=}Kn}jHUU(hdP3L_66Dhz3PT0{?$D1M) z3)*0B8^HmK<&uqCTSf_+jsWDsQN4Csa`u$Tx+i;=5<)i;e<@Nzv*Rq)swSdP;ATs)DxLWZbs61`QzxqV+NggGzKDm*_s+i zA-7x#4*tzH+Dig%8KZHOaz8@K+G4y~PI}YYr=ff{JIv zmaW6|Sd=uPqRC;K`Qavygm5PIvqszSd-<3vB9LmYt=B1(IgVl7^2}i-pFG14_bN$( z#e6*1a4o7*jLJMu+E(3}D~dQDbGTc<#zYJh{=L{X)9vI?70lXhkqO+*n;{F@LZV?V zSIM?ySN5-sr2qVhs=+I(nK@)u*N)hwp3EAX5NSL{u^g|vZ+e)bP)ga^pc#>TTyIuche*nwV=l>#& zovt~xMD2D}iw*I$v9Ga@D3Q3Rp5tq;eKU3GL!Q_vW9esR%_^B2p8Ib0*whtGUT^ud zo;k~5o4XgeL;qN;7`EIK=_%#o;W$Gt=bVw+Y4Yv0(0zM58SMCK*;K7X0jH57g5|#3 zD(3Lymw|*D0pSDtlla2x9U=9;D!meHJOV#qFu`w!M7O_HU3!V~!EB&1(Q-vihd^2w;40OP|ZTDkZ<)H^ht41oXZypO0`K zj!%J^$oExdpwwR9K6e3P#vI=GXTB#w+by<|ZyF8v2jrLXxgYtqp<&Dd&QI=HsTmh0 z-Mj64sSKLPS7#ibpZ{VehL^+8uFaO>BME)Be5< z+I{NqUnq*7D7ew5`t?H>4hAeMe+Jl4DwgsywEc3ioXHZ36vz=Ror8_Vs~1m+Hg8p$!+pT$A((#jdEd!_e9ZzbM>_% z^twrT^8`Z{vC@nT-Iojfn3#62`01&k@lwwgyS9);SB+=T2q0G!oZ4ehsN$$~%67+X zQ2KO!+ZlJ8Xy^Jvnch`uLy9l%||t`9{jz{nQyr2y8jx@&{0O4<_^r$ zux;6OPWj*o3X1)vd3kdmgkJb50UAqQ|7e&0HLi+8sy&opFyDPQ{U(hvZzl^9I z*kgOow}k^rEh6YgA-88X-voOPZ5PU?Kh@fUyEMl#=RwCu*9{v00o#`wPozdlZQm{Z zNb|H2MHr=`Chf*6Dcf?U)<;VfHiQYwRr4Op+agozhb@ol!qqx!jAsFA(Zb;+(pF>4 z;>;ITlRPkv*ww!xRN=n6&0H9{@6I*ITNwK6VDp3;>fFDPF-qhk1I3h^wIINGbQJ0x z;JQ4PxOgntBPz(x=y7~pgX(yFIy^R&m<}W1^;&fDGz{Z1pHh{K$ZWE9=|??nin2%? z0f@gm8nu1bqq-uTh&0%)TNZ!S=jOz z^+>{@2DWKrN(6EPvCn&uL6dj5w~&*@YpKt>WyRhL6;33}}=ZeeQ=XL{awT|`G( ziOcHR$V@5`TjbnaMK01fe?3R#nrOJvMj`JbCi`0yBCbfkB41H9tfOBXb5DI}cwfs? z;E?TtMt!oZ?T_hX+5O<<2U?x#mA`3#gT;Wh4+>ye4e=!*H2RJU=BB6L7hZP`V?WX| zL<%8$J``fgMJ!Qz^EeEU?LLS0dEUWtV@ui|ca&^Qn|WF!iy&jVXWp;8BlNI`@VUnA z53q2jFyS>~ef&@r=OU_KBwoU6?n?I)A$~NAG*<6b3*;S=a9X+YiulA~J{bYlL zTOq3QzXmWOyyvc2*a=^rPLrPA7P7zC5YcaSE%j~JV~89J4$GyPCTuJ`2Jwo(T{6RG z=~DIM6<;e;E&;a!5>M7+NcMrILJg&q{(5eDz;(DYx<-;Am0r-%a>;wnoU1AmOi4-V zI5BLGbR$ldoe%Za-P(;$H;72ko5#NWDprb6G@^#_B5M=7rQ{J$lWK+dWh@r4>;LXF zFA$lH)MVO_>QBT(d?6XSJ*B%GO@~$3`BU*`Xb7?!ohFH4+c}y%C)~E(ZW!D~hHQ|u z@M1_Og=X{&ln09)BvO)NA`(>e#vHNE!@XD(E!J|b#`6@@D>>D1Mot=ZC-qis_#+5f@0kXM^@3o%0&+gizt1UtNIG$`fHeB^q14Ha+Z4c3f!cEd2%ES_dD? z)=l7H)EIBJdYq-SOxd)g+~e3A%LY~9E2++ROb}(%Kp`yM6xa5SLcCi?=t1fS;L@~M zR^PnTZC8k``1=L8gsdi|!3{1tj((GS36oSNpCkR=lE7{jr|7wly23| z!=V?S5b1bUZZy^1hB8GW^iwfK1NK$Fl|kl!&d=o*(!{Meag)@Vs5q8SK5wV@L|fSv z3kI5U*|^B7ju*a6OERt;avAf~h^%Q81~(+!qJ|Wp$?EwF&V-7fLwOb=d6BgfmB@co zI(i7&4~D2^WcDqV@nR~n@rO#Pp{j$;&j|GpVi&AlT>QGR!B8%D zNHTh&5jZyYlA=Jl;1FwVcX6|^h5$6in>=?W9BV|!9YVnd>4vIhB~W0iihNHr&;_|k?aC@ln_5F6`H(W0DEBPTW0*d{BWv>F;hafrIK2* zE%#m9?h!bDAgvAfSaFim(`0TMReylUs1EU_npk$bM@;xe`aF6To2w9KL>J-1${C~; z?hnt{ks6MrX#a7yBlGP3NM$nk^4&hHtJ4Np&fh347%*5? z9P@j|&(1i{h*X5bLoCN-+ZH#+t4|Bl7;`aY_wlS9KBwzf1iC-nEq9}0;8jv-Bvh<6 ztQ2pz*50xnAY$6(Vpc^kNO{8*&|XG9TrW_Uf(&-ghbmx}k%k16Dn2*a_;b1K4wy^L zn)cooQ4U6C<7&H?`~4PMa4IhgjK#SFl5kKs-lV`pHP6rERIVaVZ9ivDi%zm;bzM1I zqhbZe@~;_{l71^Cvd-SM$lM9dawowCcjM&aAeB37Lv8umUw;AE-#ECmQ6X3ae^cW| z%;Uim^K9tBMbXz`3$-!YAM(tj#&{~lBf@L$k8ig{aW(40A#<{GJS{xP#G1K!JaIgu z;)al^_7C($-B$pj7?!*>V(*KU%E@iq7tAO28bEq?g?t^Lm`SDMF9_nTr(#ILUe|ooSL{o;SLE~T zt0?Vw&J0`NJ)+Y%H~uLm!#t)Bw!Rf5gx;gpc`62u<9o)wtWa-%*Q4Q*r7c|=#BNG; zlfKg7rVmF7knw#4KcSGx0PuxSQn9cka5tZq-5s0foy&G>R`*F2fj}%eEj_Ptm4tL$ zW}`)O>9DhsCOBV8cth^XdNWvudmEm>4Gz72b^j^)K;>%Oo-0o5i}~iBQVtQi1~mH! zA!TAd8?YUs{aZxK1+ddpAC;^&Bw`vu99HtIY0w=CQ{OS$(eEVWD6pXQllWj!0W}6o z8lPe!2c(zN8~^iG!YmncVkh*x9tYJg;V%`oIKa(+Dxr=0dKy`d9KSKtQ((zs{Vn%_ z8vylM#}jC_B4kicMT_n~3f6NGxm2(zS+FlEx_=nG9O@k!Lq0^Dp{vtQfyFKz0%js* zMlTsU^`ymOP>@c2K-b6k^7(%y_eBnS`Ts?ZJjA;b4H-EQDmNP~3?S0yuEDxcFd%oq zpC;611XUw?lL`Alg&J@?|7r~w=Ro}YQ;Ul~aadKPlb3+aG){pmT)6+`%2 zTh(q5`yOtRssMZc%Qi4Q0@=L~Z=)x2kclt7!RMxYArX-jZOsl&P@|On%i{I1`YnqblipTodCQ-FFD#)sW&91QYaEJ8WT)msOD?)KeoTiCEJx$E zkFG)R-Ab!j#Y2SAHI|mhSVxd$W7BNmBTsXrXEEM(A!26_?IfsC9w6 zQMiPHyLY#0oYu_U_7=?o^mzsWEHVEf`i9dxaGJVZC40W8neL!wd%it$o~WywmEcg~elq2G-_euG z$-&DI-1HD{QnD1T0taWj;5RPfgGmbUytL2=!F+H{y!7S3)$OgGH}jyoe{l0+#lL)# zHtiPe;zD1{D!a#$Z_C-<+sn6*=v)_PJFZ{)qT6Wx@D$!1*Q*_$#9xl6_l+bnRrcjF zcSQ>vjE_QJ(wTpJm)q0bDVXhGl%j$U{Aj{kH1HCjpINrSuf=4wUkGaK?sYi+X?{2a zuR*(a!#~X$g*m@&brJ{=u^%c#x<#t*4uFq1kSJzOzD-AWUEDGPxT`9%?%C?+^VbJH zNgh)!?D?K}aNE>Bvj+$2@!;x7`!|C_krd$zD=jQIu>qHp8eC8|o3Pz3yY2R`6kWhO zPnUjh1>5TDf~+CLsafq&*g0LT6N|02!H!a2aum3Fw5>CH-Z2%sSGbyQn-(p5Ex(`l z@n*^gdbD>45(}Rz&syWE(C1sLoI&Mt*Ce2Cyr6$?#p5fwi|V6qFlk7E2MbnK+#A9d z-M9Q1A3b|&FNf_xr+uThKayxCV;*SGr)yJMddi7Lp%lYd}8eB%x;}dAmL8lZ_Q5;_v+B-WRcrJVwMuc{UI3Vp~>uhl7+R1m6BFgxCyps zhGz}y;Wr|w`lNh~6_sq2%%#%Q4>opYz8{^AKJCJ7?N-NdMwqd}4Ky(liEfq)Urr(d zzl9)`y1BJ?O)CyG+w(a$wF*#LZ{walu>9F$0uIvX$_$)}JD~oLtYl{=@|YBoREbwU zW|P^W`yh!QY3Ezd298py-zK?)9lI*`&R|^!lKn^*>-Z#d>uDCa>bPcGTOEIJ9F`FC z@sj)1?GfMxuy!u{lB?i<80!J?YW#7WNN1wKlP9xMxp*4C)5Z$L6T_dLJ>qkx(5W6ROwJq>5}dckOo2N z4gqO~PALHa0qI5%X$GWo&K`Z<=e*~~`SpGC3x<8~d+oi}+I!_im-Us>mr{J7U=>u+ z()N1adoAj;eOi7Y;W&^PXTK#>)_z7WY8>{{7~z`(pW1vtZ}ie6JJQf0i>T$nnUNW$ zqlH0qha(-^A2#yF`)ND;TeceOyCJp2(I{)7$EH4$DYJWLET-Oh2Z$<3_yy%OB_b=#oIQ?%+E}V9LKj*?JG4szPEwA$q|@D&^0GgcE%VD1wFP zcVBgUpR&*l9c54ARDSkGNYI6+{fydUjTupa{gt9;{b)?>>5($2Uu+J-HU|{8b3IW^ z%{tht3K`oB-B4ch#oC28$mcNRSG^ip*y{tg2(Ev|Q#VXUYAd27h;a7=^wZ(plv(A! z&Wt^X8O4QLESfGpnvDzz$_BRRUL3f$m6n@EA^Sn1ZE`1!1=<7W<3mw2 zC2Gw}%C^d`GZK{!(Rvh%3pyQYJ!;uIyK-wso#vX9Mb-H6CD5!ySyoYyFe&XG^gbJBB8(2JWjd8IEj9$tTb)sH>J z`*e--eE`CIy1IDnqE7Oa>*VSJ*icX;{ZjL^$9EO^%qObB&Nl_z@ht6G%2fjIS0kr2 z@tW{fj<#%dsi;zS8jBVUA_R3q`UMvh8Agx`LhJcQnqB6K8rD%R7WUE4ybn3G2+vZkOv6Uqp#l zet0Q~4FQND=r}V{iN`yO`%!gBNq+#n` zRlUgT1zlC+$Dl{uiJ|Zmm^~M(pM)Nde4;8Y35d=yJuAW94me?Qx>UN_s&|b8SbQJ> z*-fciWphzK7ajCUtrlQ+UE9*PQB5c;K&d`}&2>wIg9qd|rSZzOZ511eH2#iwY^<7) zU9h1V3iT*=>!t8-*ChX&rKJlWtG>rLuqyil)EKR=tApq<30KBP)q2t=A2H4a$h)${ za`9N5FuOmVpRJ3627x-J=Q8vKOR>B-sNC}Z4Cw})H6=oVFFJB5Zv)DC$foO+^=fnn zVe3aZ*}3vn9jjN;NYCM#Rg~O+lC{!JT)dz;grPa`P zjsLTX3PoEhP~6V_0>(;|L?3OWmJ$22+|^yZQ0Nn>X96C^l$8Z7&yCRnwsVWB?hb{_ z(<-1zOQb+w)Y)H5Nm9{k?zpL;npA2Lzm>Ov3+M04i~Git#qw-@2*1iiG7z0Uw;;qkA^jW_5IY!podsj+$G36n5_H9j z0~WFsC;?MJ9{R=Y;RC!d-KsmPTi$`tx1<_*`Vz>QREBw^$r!7b8Y7iy4|*pK+_0TP(sx2b{QC$$LB${i`Mwh=#Eugt>wx%E0ua4y_GpK1 zrc>I3$m}>rLWWU!0U>K8?4c! zo3c1}#Qx;|eb}2Y_Up=CC@cr}1~F6k&WOm_EPm5WeUQ8ywK6td}zI;YWd@XMDNOB#05i6kb7pEx)K*+cn+A*sg zkP=?AH&@d(3&skhW?d6GzweWMsy8}c51&S|E z=~H}q8AmC6O$5h7GAlyFgm+`Xep%PV&c4s*I5OyufCTSuMFuGJpz z*eJJ^+t)5rM~Mwv?6-tyZ3=QhjuU27mlkF=M}tJg!{2W&mn- z=N4mwx0j==o{QX!6Y+q)pJ>x6y-+!u(wc<3mI^t~QlU{3UqWaa7iXxJ9kzNrYnrUR z^LL!ZI2)EUr&cbynXNB zuT{?jzGJtVpi`CdId2-(|C?a7gskgUsB1Wg3g~bZE?=^(vp}il?9$7kt&m*^qLWGT z)46ssxrh;u--ZOqhA!>hZi!Ad$*M0dX z3j%pfMb&B3f4O$JLEtg{To1-cj5qQyp=Vz7D?0sjAJ$>jYFkuR%M{*zc)(MIGRZ$`45Lu&E{7?Zy(Y5* zV7C2B0n)`I7?+lk)rvMwFFA1Zj`&p&wSquiU!z+7kf|w9_1mRlzEl~9z&L&D%EGos zAN9&5fw+mu9BQ#&ZxS+xv`$+k2`au zVHDye)AY|D68H!akz)n;ia^5A>2HCf4}cl2)K6#C4lB*nCpGZ$*)1k`FUIL89Llm{~P?m z1naiW5TF^@Jj?FBj8Zpb^M(`X$jnQ zezgHC51cYpX^w3Q%v%5*!4TQcOjshds+c1a{LTX9<-0~CUh6X(N~`G4Y@ zfNU!&tazX!Bc%QSEcVrIRzn|v-H`v2X~e9`iptN58VcqC5oPd!tC6q@*QmBE1uPtV zJ^Cl=3TV`m|NIj8RTi*64XAT+9KdA%6W#>Rkaf!dqvW6Evp|XZfOmsSP6}oPu}%=4 z1497xT}D2QG^(8jdO)2&{^xD*&udIz=7;hu!@vr}oONKJFDyq9rpcqnSj6)_@(*AB z0XEX0-4e{WuHeNwn1-VpN{J!&@AwyPGSN2wJ(5lbhUvMVsx}|b`S0(OFkrXWDDj&~ z_3v4vxNV;S7#z&`3f{^Bg<+P1p%#4L(WA1r0Rr^Lva&Mu|A#bnOq(xgL@1#4KR<_p zeq%yR_zh69 zOb+0&$|sm7Dv;AZ%jk$+!~&uA>_)ud<%tEW>+5ls$C{K6CGkj@1O8oJ0}Q2r_489){K^mauU zOx#OScs=vpwfs_;*=kgz5%8h{y83w1sTq)m(z>NZTvzVdN5DdP~o5 zM~C(gh&#`mV;kWdUi|86wU3EGEM|%j4m37OxJtNtdqu!@YTo?if)3d&J`h?r_+g~B zlZXR+1)fa8k^4ALqzi>IE(*3RZ+#@W?7dw2g+JrCLDcFsYpF5fkl}{+qwY#O?={^e z;hz5APhCbxvHB?z4ET{>?5{IEh}_5>95+1>2EV0>x`c0nto48Jiz-P88;@#WlcD^k*9MAsI}g9Pfmobb(Vt+~JJ;*Y^yoSYu{7iD1(m5? z+Mb4H4(@{NOo}@hKn&hMdK}$OPMaYWPeaHYJsv`(X&rs=Tosn+%B=Bb5-)8Y+d7rR z+m%(Uq>pexIg$SG>88PZ75c$g!@gXKn;tHJn+M%yAX&YyDv+o}t|nTB#+T3F zmBRj4+%^BS%_B`U6Ut3W2njA!7{!v$8_xp*?SSz(W;x12_1nU{v{-wbK;p(2&8;(q z2r3K7TO5CL4Fx7FIk#@E5Ob zuy930#_xBr;C#_znNRkoqTZeB%lJ|DxJ{uN_;0=(Sw>20GtHzxIH5`O8Kv2p``0{{ zms@YhfZb>0v=4EQaJTuF^lvm=PFrMy4^05+vhSs4242@~4jI!gZg7mRcK(-XufhF}n1g@qVvoQkRw6IV4}sTs~R2qNovV0nCvFJ&ir>nbUIMCVaRu8c{Ajk706 ze^25pa_ukGUZ1b$yfEZ17fOn*7J!zM=^RQJ2WFVT7Cq!CC`z>&*P|7r1)z`&vO!p- z@uD5&sQO^nV7hkX-Ajf*&s|zH*MiCYMXqM#l4glc8ZRfE%kq*#U{_aJwyQsBDZLRz zdK4<&OY)E{m?|jL=i{+$FkYrz*P(KnD4*P)gBcM-8?lD*M^=ZT z?Fxj-QG8aEHkXCTDkt9jJlv96YZ0Ftg|nB51fw1;!@a1T@IHfgn2#s^9;|QYYgXc zC?C29;yO`Y?(S}yR6X1-IabIJoIG6Jn>+i57vD}txBR*KhrKdzQl+r}V{;(fFN6;mXl%J~&y#Pb82hhoymw>UY1=mazVxa;HL=;Ipo71& z@LK0iju}BkNO!h^Q7IYeA88SQe)tYUsHgX;Qu0dj{(^Rh2Uu|r6%U-VDDU!5K9F58 z@xh7k1%ix`J7GCCHBYT(!3b$~J-WMwH9NJV^y|Y>T{FJN9(_csUN2Fxg)k~_FG89d z<3IkDM;uGKRY*VbkWv3EnABrqmw91HL!nEin`e~G-yVAS z#CSLEoP8mXEU#CkJf37c7v|7AMINuzZbe?}m!uXW;?q8lsN9Jfr{#a`DvDq0sx)*w zS%IPmX}UTlX@};17#e57W>-(tZbkj^FB8WVqICe4wcpjXUux%=!(UCN?~6MWCUZJzX3*c-e#vI_k5 zsqplbDozl<^_O;FTY`&Zpjp77m9dEghtPMl(H`||qP~#D3ysl&PcQt8z|lZ`EHv-< zu)?7VR( zhV$Y@88$nay*bpJO4`NvmeOofRZkh2+(Y8!kW4XkfWo)ma{Vod=Ry;EUjKIdeD|Cf zN~|AS+MM}I=WA5qTS^i-zpHn?NpT!}OA@=gHd<(kY-`aP=#8DWX_C-sdF7xx;x@0S zM=sa;8c~e+iI4a*4vjUaovgl|sIbR#>pwJDo(7i1G8s+1%mfvxw;^gR=71t{gEQnca=~C9Ujk23?(Q2|_u@!!| zYVt7j?3t00&*hLw%Sh2NYOL6!z8d{F3Oy2FRL2tX;p1;q*GkG9R>sT1r7V%7NLwZ? zgyW-its={6itPz}lbEUNE}SG%yR(jF$|BU4Z>2o%8f=-iF50q>6Ef;o97i#FO6bH~ zP31RYiYL0Ny9)m2$u0UsOWm3a3VB2qd+4(KRf||MHiftu{+jV+iC85el}DUYUv@HS zj>OZc7a1kS(3!VLa-`%4PYTwbjBc84dUby2X%x9ih1SpviTK6auvsxcKcCI7Q~P#U z43aYWj%zz@_A=Bs>y3jQ+V=Z%ormjjoERbg-f;Bcdh#1Zpe6fjtr*eFMOU&sLF!7-w!5C-jvo3ug@!dmALzH;MC+&;yem~77mKWE-kt_wO|EZ`QY}3 zG}g{!d?dk6-1PAL#rF>SVTI?PePxPmC||Fsl^L$Zq}j(yB}GgOxh&NO?c4Z>=kB2O zZrFDl6#2&aM3Fe^Uhy4U8%db&!W|JseMrCXm{&e*cF%u7xb`kgaEN^+*|%L~4KjEy zYK4%5{2f8R(Y*Uv-igAqs$mgGW&iuyV2m=a`gx&l@*r}IE6$dM2(+%!Z95V}tZVKj z`E`b^OKRx{E9UHT9WEAm;|?NcW^ZLY>RTM)MK$<$KPk!H5lYO+mi;9oE&)W26@{1{pbl;Gh$hb# zBJ46B61I`hLW?T!4SxU9m>YOaXK34IvV5m0Jl{im`4aO+>-u{&S#pT4x5}{$oAHHX zET^vGLd-aiI^~^@X-Y4i46(wOtHbO{)8b8U;`Te&gXc!C`{doysdoIZ2%h6`nQLA^ zdQ$8#tBgYIydMn+S`WrrdHpoU4Gz}dDQfb0e2;yVB;wi@z)xHm2(4Y!)Y4*}KHw|8 zi9r1Hzy6D7*@ZMxVL>I&6e5iN-h_L4Sa-U5*@U{Gh{$P2f}M*gF=%)x=f4S1@fg`P ze+HM=uEk`PTcx)l5TZB0nHCK&e}$77&Ql)WfjAaTm~Cd)k`AOIcN{uHf*nV^v5Zn( z+w)F17VJ#ig&DD>yTU@;_+|KX3|eH~!Qm?sVKk#J9iYZX3w|o?5W628I8BaiX&(X* zKRsVvB=t`c$dtQJBJzwNwL8XHh8j*fqaoT~Ra@ckEeQsI*7myyKbXgP3fK3JCEn{3 z5o`|9Fhvh#!e;y5VLI;jVm125eFwA@ z=d;jZ9?1LB1nHC#j*(JZpXA7pZk?B1)u~DYOidMD&^hZq(mX8)mrr@6S^a=?{5o$} zji+rxn*u^NHDZH~z=d!w5sBb}Jb6FixY=_aD2`za^giO7Ft_VvV@p4sZ?{Xz1FVeK z#odC+I}b1Sr=-q2r?Urcx*~qQ7q2vbxJm6PJ`d^8&kj^chp3%=6#Av;9B+HB#gXc| z*43hR9;_%zvoJ+A&dBFVK-z%o(`ilt;bk(Qb1*WI{Kk8XeDzIP3s;>mMBhCm87r1^ z)-x*uD_9=S*vD5zB_3jDUk@J_*glHY_ok=-avy_zHmAi9q7R)sThDv}uGqljH=X@{ zR42)}kk8jyG^s;-I@Zqpwr4#UqEh7~SV`+#(;_v&W z+vivp*cB4G*YG5N)0&nnU-E(lD&I8s^E~t2`I>>zKsn#YWm_nr<0&?z?fn<=xrBL| zd$@u=uSY}N9H=?;nB7F(Ru*ijwBavP+OJ3|(lL4Rq@stj?+>)Sltzj`=7B7W!XS2` z30%S=;{q=h6rZyWRaw?h4BMW#W3F{ws}2h`{yFV8EZ=u=eF&Gr=3BK7k=~v4KZ9CQ zk{q{ZP7vz5%r_yODJ+E$EQmF=+*?hdU*qz)3dsF+H@7=htHCeMYn9dz9kjZYhnLDd z*z`=)?~soKW}T{zKDrz)xom)2EY@bWp7Yi@@0YT4qhCpyF#5%PTu5a+?PLAvgrG0t z@qEIPeal)bHGiRS+ez&;0)nQGDYp diff --git a/publications/whitepaper/figures/realtime_flow.pdf b/publications/whitepaper/figures/realtime_flow.pdf new file mode 100644 index 0000000000000000000000000000000000000000..182e64d2c4cd420294a03c254936092048a98abc GIT binary patch literal 27060 zcmaHw1yCH%+U{f6;t(KsfZ*;9!QI{6VR5$vcXx;2?(Vj@yW8UK&L#hI&i&4Kr5?fVkD`jZ)Eu|jkKecJwW5V^=~8v z0IlHr*#SWRSE}`oUdGYE(hT?>k>Q^Z|Ma(i7yLV(5zz3j59as!fBXp88RiLVEz~zm)Pu_STMe2JbG}{&j;72(*50rlTl9UPG$y1@*@0yE&h+ozrp{Ll(3nlgOMG8R@hR{!AQ`^z}oQLuA~vr z#K9E6{?8!q{p=myQ?~-Rq@8GJRA4tCYk#-)5SJyj#qKD;Fq9fQXYeT9oHIK@xAnkk z5Pg&-6AjZ%pELXY`r@^78Z!ePNN^gjf*eS#Oo4 zMWd+=)=sN`h*%wOByVezrAi7lyH1&=;_FOQz$um zV;sxzdVHm79-nnSgwdX#WKxIPwy|KET;!4Zxug&V&!jj|$(q5SEX=aNw#*hOcIEfn z_eM95A1}vCX5enT_k-=mtn^c9->3`2=0Txf<9Zufkinu9Z%o{`)i?T_CW4Gt)*ZFmGNB1X?9sO=q67wP|La z`v~9B&vEK>*Bm*g3hFvQM>1Z8y67Ui2%Nj%bEE!l&HX&(&LQaqvp&UnpW7({h6a;@ zJZ@C43CFWp;hvngJQ1)Uvy&vPEg=HY4fiBg$a*Ln=>k^22#^X@FVqB)nJ_QRY__r` ze4--QK&k4O(>o0-@95;HJxhmy)WJA3S^;UW0h7tF*s54YNjWndQ7BDC@>uZdthqdoq4@g(&-r2 zU$(aVDB1O-zN6RduRb&SYCMp#p>(8rQ#2~`Yer|=qvSN)=6lko+T9Jd9EK)8oUZj@ zs9g5Nh!lh8Jrv+O^ut#Zt1KQi4@c7Ck*vG^iR}J2a)&9cr)~F(A9Pia3!rAT#)+Cm zF|0PCse@^z3SRlitiTW-R!dM8Bf^9wq58nU8rT_RDfTvfYI~=2mw4KCN+7w9yq4LW z^a`Nb9JfiY1UHOfiq#wyi?!a>03ehLtvfE287 z*!*bY^dYs`rTGIkSrq>Ue>GOYFzlu=R8cqTL}BOCz{e-3 z{_qH^OBO*u94(I&R&V~j7+SLH`up5OPrfhw7Szwg{mm9Z z!fvt!stWOTLR8dLqatul9czAF0f_TyGOrR@RC4hnCpMO8A<|Ywy~08&r1Jhwp-DC& ztW;7^$XXCP?LI+&qpokF1^>D=(g+^m%Irj9_sY{MMrX%`Hq*C_Fs+Kvh`3X20^Wphdm<-CQ@RjghRhbppgLvl{3g%EIG+ z)mD^Xi)WU2HS5IXo{Pa@cJXsYtjh|`faJucdy30W_Y+JoyP`tplotq8a|JER=Nuyq zt5A|n3XMo9`joHOn+K!+OD3-quagWqe$E)Xh1eijT4(lk3W0zR}; z;ILOaVQD#RsMfW!S4HVkr?JBP&!rG; z-t(ZSZwqHg>pNE;imrC+TH4h-%B3pjx{QZwmU@dwkgaq3q%Dmi)~D{_rh$wHH@lom zo4GhJa4=$RY7IYzOhxa@f6!p& zbu_0^YuaZP`!oz4(c32R0>(rUfJ1|6L{!dAgr+)Np=P9E{}o1`Ugx zT|!VpJ|&u(d{h@rjKE1z90D5w=KdEJg1LoG(NoI0#=R~3iT(W+_2#2g@ew@MO~#J^ z>J!^3ZJMb3j?gmv%A1;T37WCny=U%nRBmqXWF$3IHn(R^=2G5_jicOm?r?bmD0GJNq&Y$ z)3-XlR$b+)g;ZB+yoFDSKFeADoM?4uMgn!KTCZy${7*RBg((DLbtw zXa389?#EuemHF>W93#nIt7p`=XI;vRx$|!yp94$Y{u0=~G3DPZ_LpA%QVs(H9n*jK z>>qCWdsE|`w*IZ7|4U{6!65;@_X@3n-uwA~c!~ZmBL0_xp5|YgzvX|a{+nI@#(AgO zeq$Ql>z@YzIUZnmC-jhGWbXPw<_e~Afn*# z&TId4=Ciah)uRXeWAYs?|Ebae{>kb;RYm~)UvvKkWjkvF1tW*|M?m`y-|v`h{>94wc~F1-{m1*i==PrsXcZjw9sXhUzc=3>4Zohf(Z5=xl!TSk z)hPMw%=9d!6yKc#-vQGM_)ha;hVPtf=HN;#`WI8}4c^Dl13LWe^%vXe|8lqdUm{kt zRsx#+L#6b8;rgEx{^RGrY5V`jvGo5NWIg4FfxdJ|tygF~`GL@uk}=}YMdC`q0{9ds zP`}z_bM>hN5S~3pfudJY`oW;nv16)Xzo%z*){}v5#QBm>!paT8oZlXO6*?(Q)lqG_ z97_5P{G4awS)jR_k^+#MS7tn;@~NJ$`}7=8HMkGjeFDFGIAdvAVAVgaVerr(%)RnM z85_z)8A%G9V7DdQmL31{7uDEXnqRh6cOT>3vZ(I-(w`O}*A~!~0MInUf}R@449lZ^ zHT?L^Qz1AAua-{!%X8!+Y*TL|wz0LbX|G(Rgl{Od`MWOl+9>u3q?sq4i zQd43kM5EH5zWgy;jMkd_j`kR3itOL zN1@AoFFgjr_?zGC57)5yXKZ)d-g*A+Wi?G>xU5;}Zf3EBrnhst5DkCnti_ZO@8%aLHc^K?olTY&?96xb(E{Gpb zdy**MH~WWFdJrUpnvAs>PUaTJ8dSBSXPGbgW5}cvax{m$F3AMJ@9q=Nmn9YR3hIR zEZf?^ANwGM?zI0RwrRU2xoyVNl>K7T1Ks!ys4rlb*e!T3+K%AXiPQ+aTb0*z(UfZ6 z0LJqP)GN3KX)>~Xhg7&|;k~Q7+*^5?_~^d!@vc+km}S{%V1rDe3hY_=VNUFZVOT5b z>7$7(bmBIS;Wxn3B3?#+flT1IsvmX22=mUZe#w`a9!XWG7F9ItPT2V1rtGE{ zq;7ijO@*`HWtE>hq|P~ABH82rGzHp;5CBf}klJO4TKjKB7k7;o)q9hCRL!AgUH0_v zxm%$tYQDNh=#;X>mx~KGG2Eel$$|~*TI31g<~4fTnBQ;?u9Z2>Jc0-I4wv~^xqN&b z(l15D9_zn!de(2(R{x-kQw8f)&_&gIw|ak#sFrfFA?@|kP1Z}3Cwte3cdW@)NPLsk z2A>P0d7m#NT*A)5e6^Nn-2eIP`LVPL9#xn z%&?)_eZlLN<0&n3ROE{N8QUzMN#`+r1-s+Vj{l|O5CANEb9H=wymp*f{Wjn@TJ|ei z6mi(**16NoqbA+v!&={&$T@319hChx>mXq9n-iK>g&u1a)B?%vH;Z05Z4++;ol5IP z=i@I>&&6D0u&&5n3LYOj(%w?mhY>gK9ABMZ+rB1*@J96qgn%PptrF4kzgTY zeQPp())yPsoC*85A*>llw`lq)7St@E^^H+dM|W$4f(F5sxbCVWFAc|Py;GdW*l9JP zq`hgJmlSKMh@i8Idk3D(3UI@P@GHw3U$rR0AQ40!!0eYItyCgSba~Wi99FJM9%62X z7$`6Hess{O=_gH$1h|&#XT}d5ii$+uSY8DZiVCtor4|kqt!$Q}H=_d}`!GW{!<4*r zQ(tw@Wr8Dv{2sn8Cl`Df&#{47@{G(90_zpfF0P|rN5C!P%XDrw4tcz_e#!l8X*339 z=nUgMrsX%BcnReBr~8*@HeLa&A;=mzx>V=M%zVh5(WIFH?_WfhHTS@Z%~*^gX(#wM zVaIM|$W|Z1+7kZ*X+_^Fv97#QIa3WEWDqUKZDIOnJ;8qv(@8R(H z%1sM2bCnDEZc#FcD|Wr+pf$bV*2`1S7{y>Z?UZTD6goD~y7O#)*7pQXp|U-Fer}kO zeJHe}Ne#xaVJ`jk&}3xe&!Fw)tMvf|8+`)nd1 zP(|Vs$N1$yA5J{T@xBck>=*Wx3D(sEN4#o2OAdE5b>matsO;f~feOQMf0!nbjf@ot z8@W@K{a=^5MnHNVTU49?4PDe09(S}6Uyq$M#6_e$Uf!(hamS64hZc{o9aq;kLY9e@ z&>q3->-w8iU`$o&j?aIxSQ<#2k&0Syh>y8#c+x%xo>O=9BSVV2XK6TnTaP)S9qqY$ zffnAy8%3kz92wDq?%7mxS;*=9DuFfM_`B`*nLC=~-rg=vbiRoR0UnT#$`DB-G3U97 zVA^D4zlO2SBwfv3xZ*H_yO^TI@not=0fcZejdn6LEG`kEAml&j^V}-Pg&yuF(DiG5 zZ|~O@MPGbSgq&1u2>x`^^G0xiuFcnmKZxC=vg*ehw_qU z1Or_^(6D1*qz2?x<5!fQ{K27B;g_ zFd*>hW~saDuv?6|SKjl9Sx~{#3=&>mtb=?SOE?T?y~4V+va(#@33ATF{O}o4^agmI zD}=gF6ksO%#H89wH>h^aVq;(@@x)y+&tzuvx>IYfZoU*3yp5ep?#L9t8oqsN1+C*s zW>*}9cz1)@QGK)8Slg^)s2megqEaErV)>1|{brTFd2V)-v67bdfY;U08^;Zv(DuCW zM?`-`1@?MpIfx+kn&evf@kvVN=1;Pms!2V0a$`r#-$v zsgB*v{zXxDz>-TbP<*{N;X^6UU5M4BB!k;yI>JeVsWxtv#tI(MwnN>Rq;(UiT@E`K zGIQBSO_)&}p5Mn*R9b3FvcKRw-EGRqO-7Zwj+CWG2De8We%eOJn0et*QoHOX$L(-E zE6%iN=kV+U`FF>ivx0d^dh;2S6V5EENio=>O;5)4C}~zjMWkTW$BmRNoN*LzF~SM0 z#a7V_7j-a94x;dPV8P7Iq2E3&RW3w(75V2qgKWKnfeSZwm_1Mff4)L<`EQwgPWKVc zE_w%(BG-bvr=73zc0^H4SeBIu_0LzP1l}FVZCI-NIx89F>-bz0%q}~A7lU3Uq}YwP zT#vOeUL7XG+(h#mXN=k*_C%oVn8r{+_NqL3w0=(P9hBz+Ku*S-VHKfL2v)-JdX^Mk zq0Z5%|A#setYM2?zwRomB&vxyl*|dFmf=zcJ1yp24rCZ%gt-GlC<1gQ^ zFXVW_k+hPxX20kMl8Nst$Ques`LtRIe94Xk;GLO_XV` z^2Q%`xLqESOEu-~6~w5U-$=i5d3)Wodc{h0n+=^b)2~fhWGSW`UA2ChutNwv;ZaO*RVSnVWN=h(#(ru0AduKx&gFJ@ft!;_hWFK z^0q4IXgOSU!M_V@QXJv(S!~;+8IRcJ2HmuTXC?y;B{L_tH54Ls2b&Q@!_#0sYkaI- zp#I>$#%~Ry{f%NZR?KXCFzgfQck(m7K4_B~5}wrYzJwFTNmq}I-oWbYw3@HyNeqbC zf?!uxp{br)ft#g*3A6oZnshy$k~o_eJwcuCZFT}~v6(2b(lKlc+ zP?(Uw@GjoItrw@F>YUm--e1fwkXMVFA)HUvJMi}a26mZ)+D3?`Q6O&2M9_i5E#9^* z2TmKPP^xL1QJaDinL>l9xxZ+RkG%~oA<~GSM&*niCcYexREru*tHFj}p&xHJG5%-! zFj~Fdo^WJfWVR&9p`G3staS13sOex8(pkO&2aYy;+9%s~rtN$58;}e~fQ z>tNXeRRr|e9zF^e=@Kc=5-&X)y&KRY!YIO^_^o>*ZGYz;?b+^z0TiDRf4G{!S(iw7 zcF{1~fa`tHtDY}WLoBJkyDZF)ZTy-HgA~&ZYn9xsXk>!@In{W3tFO@ir<{-a!W_Cw zU$e`qLytwC_!<9Cr{Nz}7AQf|La>$cut?OpR%Isl#3Sv1(RchUGJ;VqlD-la67MQTeWXBhOGwSgtNh{C zqZ!QMRE6cly&uM=1Nw8Xn{8}r4?i;(@cxP@VE zlaH`AIX4|9Uv|CCnYKi+B85?XjhZvF)3SN~94lv5wVHa8>tXJc+DABz+e+K(9JEY% z`<|(-Z!T=!PbS$nY@6{$TEI}t4A0N3??n)z^FmVQvuJLrwrkZC9}Wkrevb}PL!}S4 zwV)zsx~wC4ov7c`-nG(?KDmbcsj$}HI1OYdUNuzYq%V##zV+W^jdlrETV(XW{K03y z%_x0#tIiXwGm@BPx7Zh&{cE*as8Z{tib#}wnIW>g8t&!TGgGWBW4T|LBzjZ@-;8JnWu#P4Ly3^I&Z^ z7FG)Epv`DUv7AMb33gK90J$Ue%3I6G;fAIcgfP0j*k+{Ysx5K2b^fI5Kx>QUwbdMoD?uY=o(Z%|+z#^zB1&UE4wVm)d)9 z2<6VRQ43jhQysUF;I(qN{0A|g146+twZ^xIVN)rw zK;9m*da%CIJp&(mbtvx_IUYuX9yJO!sl;(j{miD`c67C44K{LD+Za-!Uc$jV^qQhR zOLH?bb2U|pUMF&2`Lej}R6O?-xA^iCBr|>P2H|zX7FC%Kf=Pdbge$Dpdx4CFM4o~Z zfdr9}c}E((c2@*jp+qU)(bJV>tG_#D+2~yziMHOm`-y&q&8fUL9Zb3A`=XysnHwSw zVnJV_HyKnw)LgX^?c7scsMWtMW&YjYP3zsV4`ye~*J@#TdRQI@FE~cKE3;^}xESbi z>p0xZSR~TQ0KG4K%BD$Hm+9Y}SEl#Ws$s$Pq1RtzoOrh zkTwcp>bk%0au7p3#rTX_nd4tiU;w4oPjX(Z z1>9oHQ2y#pQjs+Uq(Eo^Dg}+qcehruR^eeL5-Y?N*HTfJR+rw`=tBll%EFk^kTmrc zG2V2IWaG{RT065=G3&RzNpmD!_tWbiVVx*Q6~dpWj$U9{GgI3+-V8;UVS;~yw!6T> zB6_5acKE;>r9ZWvl-v8t)M$={2|KN=SZ;#2D4**GzocP`)Y%^dnVaO-E~&+J)(X@` z;W|>iY$N7QqL=Bfwyye+SL_%)kL`k=+n4W7EJl>8#1JJor8BNZ$CR>;cm41} z+rs^^fp_D!QFg`fnJGHV-8Z#mHA6BAGuCa1sD@SSYqr~O|&FCs?WvV}E6M?bglq^MUmF2&*2DXW)g=y4P)iovFzPlzK2= zy>d>?#A|<^;~sGvlmfjjj3k(sSg$;FHkb8S2t0RW3}~~^yFbg&t~K8n#E=O@xBC-vl*ReUbz%2-pYYtBz=bui~UB&nrK% z1dCB=XJp#aC&HE`HOj}~+bmvT#+lAYG?J=&sS>3CV8hYTPsJ9_iElgVblkgg%N~iA z*4%Q+5bSeGD~M5#u}@OdHn7vvh@@&HYsPM+B((M0m~7h|g|@7YGzcn)#b7n^hvDM2 zA!jkcfIfKGRWgbE7=MQzi5i^EEw*ts_gG61iM-iv8YVf;g0_B6VQ+3}4Ehj-S|QBP z)HhSIjdO@d9;y0@`g(5h`V36(_Kiaif}|_|piyi6+2ow1^W5{kO=3NEs+Eq)PJK2( z&mMvlv&LWyYl#LItY$n5DK`;^Jd!xfoWU$cb8$dfT^xU=ubTLJr7H;oL-^Wo$siQNmL-OAJuQOHzU^$zv z)-d;q(82pi4I^v{L=>iU_E;4I^aD)Pc(0ZPp-QBs%JZ%Lhgx0wFsp1-Emy` z%Nh0#euk#K$Xp`#kpWi_-$nRUdAlw?gd8lAh zY@-w1Nvjieb<4G){pKPDjI?-doI%dYm|-izR#V*34`5tK$yz#efvD)EpaeU~(23+A zO*3q0b8Mp=7CZ4Sb~@7dnP>>w3p!7}fryS?dvV;}ljeh=Tu^6M6)qt#>NYCu``UF< z1f)@~l#F0bYVII$P5eyvw7l(v*r@MTFkjbnm%_#GX8F04h*D4;EIa_7;$`MR z$fR#BSqomJrtgw??jZtVOhQW~z}j!tGQ73Bd8BHksoPQEUg?adqXN1QG$+U)EXJ@H z9D^_@A45LrBu2U;(iuFxFcDqXT}v)}DMcS6qb~?O`Rw0fHBuj{^TF(o+>&J+Fq-+m zKSL75C_M1D$d!!@Y6~;!2^R7v62dQ%g3w6=-H^dzQqMk;m8vUL=&YfAqg(!b)5tQA zjC^>S%U~ge1{i;mPgo%JYmLyGy zn3etbro{H@u-{qS*Zeb2*RH^6O_bI{c;0ub#}^voI`k&HH>@L#x=Iy%79!Nk*E@9* zc=1~&rx|UVZILV^jU`%7Z6z6_9}V1rt`XrGP0!6@3Mw?b$=J$DVK(6*leo<>!Y2VjN8wiJBPLCob1%Gp;=D8Xv*^ORuPNl7+CajL7@bmV8K~sU_9lL8c2axdfI@$k7o&Sn?=F{OusXCDU&aDN!CN zU1f=od;KYB8a#{{QU+n&i$Gyy6U(8OKXQKP@PolkkI#%G!s`5iH_W5mh%=!#1^j~0 zS{!#xgwt9f8l}ZbRYS^Z;lldJLK*tEzNIZ^({Yy8z& zQwM-&DubKVR)4LYY#ZosveVt7nC^1n?=aM*-8aWiAW}^Q=fQ40m9^%#$76J5&!Zl# zNBFj*Zao=e9VI7-tAMmj?#jDK%iAh)t<2-KymgF7kH^Rm50AS*q&uM?cPRT9-)2(O zODIYs@^#~NUGLSg6Tlt<<=C?3r;?^G%Icr6sedz@RTb{2qTbS7QN!Sjz+h_-gBAJc z-Q$q~vyXYES@uvE&)=25AoHa~G3mFtGdQJncc_1j3D^7rnq{#-otp1YcBuNHz7VT* zrFn_zh|Hv|;nPZOp;R$HW115*tfb-FEKK!H;*ZD)wV>$cP4?ac;yCYmDMd>P{wAu| z{&)j3VqfxFFuE~>zVvSkM(KWJ6M$+gLu~5(pJaYm_1Nxc^a`?ImAF?cJI+@kI|c8M z6`r#BqEZPH`t^^hO1 zdcmM>&XuPXc{ZHjD?{ym&J|v+wBT@+{^fo%gVxw*)b3b91B2}?)_}83btat{j|T*e ziCq6IF<8DFh>MV}sa$`x5h+CE>->hgeOruZb_(q&{_YI)34L=M*fLA%wg%#Qrbw{s zmshS&-gp-_GUVAZ>!`N!5QRQp6n~eR>K_W~Wx>M16XW3+(^RWXx@G3Jd$sHMc9%2I z$E8tF>~b@Gb=fG}Av@5e(CBOK=zio~OIodZ`sp;E&s*K<{Mx;x;$eGavFLn1Y)R8{ z_Hf*D<{h_M`4VZ_eOB`}fTtTz)gI0c9LJl>OIZoK-t>XS8Qqb&8-qFTTH5GzU2^@n zxG89~8(`y?i=*mEA}$lEe~PpjC*NhJy$Q7p6R2DTAay!3ZNH1j$GsKuRYQ* zK)!ZbpN+AmeCK$ZcTC|lTXS?2=%AL-#m z7wzFAGeI;yO;U98CXW<$`_e>2fH6?yLMV$QT7)JZXw!TRmq)Tv;t<5}gR8Js8#5M5 z$F0gP-XX7ASpCn+$D`c94cmk7IB>WDJ)1qzs{CRC{&oR`^OOxjr&Qdk6p4Nm;*<-0c;IA8uoJ21O_6ug6ra`R8-c)SK>=Avl(PP(I;!G zaFx$OHyof6u;_RB>}z>L8vh|;30JcpxxduO9t;2lpUC8UA;Ym;|EtU%gTatIwALKi3I6OXe||n#|2#1$ zd+2^!!NLv!x(K4B)34iUTdCQgYY5iG{M#T_P3o=r*>)RdmliWxGuqkMQpx%G9-oJs z0YzAGe!c5^CH?$jaUEOM^rp)CmM$f2{VKdXqf2K>48@9$vn~wn5uPW-&qsGCwo5{z znZT4MHim_k#g3}`Cq~i?)|d9j>!HJvvE?3;_B)!?`lF`z$z?U}<)BdT=&p97Y}~>9 zB#n-3qC?3^1n9K$9vzEOG3nsXDgaA+YS`XDWG+Cm_sg&lVm*nIu&2ldgt(EA@3bmEA>(^T-_w9_8Xi+Ye!M>2( zCQ+)EYb+9;UNyC=y@)#4KSb;^g5Cc>4Xs0a$pTNX^>==8QHK@!-es)9;kv^LIW>eQ#Nt;$2*D{G6&OS?@oJdq(;U2A`! zkSETJ1sx7##rM@dTQ=TJii*qAD)z^ZiL7_H$uMuvPEndx_B2M|=AdFB=_ZcOk?xK; z{Xx|!?ZeA~cBYJ2*!r_fsR;`SLGkB$A<}80#y1wi$}C|&IigPp#Zk${sYu0uGtrZ& ztGIN&r@>~ahzw{fu>SVz0R2^d)SrW%z~Baiyh&LZ{HZ$`jmI$%!vjNq0`d%OL&B$K zx(+`o2RoZd$Xwc`V>hZhDrs!tO<5%B8=GQ9d1_!e_Q`^1adN%=t-?5hDqE%bm^(%# z@I{%riLipeZt{q=bcu_<#y>Pe%5Qw#{i3(r-elAxi|MSH{sq6oYoS{2V*{nKrhZ3Z zPqO0`(Umn$TC$xqNJvCR)7x0)md}-+5`W2TW9KR$)0N#;;caDaN~G|n)pYo5jH=## zQE98#;l`0%>34B28_$v#P*-v56OU9fFcm8)Z9(BY!EWM>@4E8MW+U}nWwOT=A7cl= z4*_wYDpg3r!p2geYxO&ih}Ovw?q*A=ysI^{#9LQZKh3lB=+nI3P0a-2agv4sA5$$8 zVlOH}D8?~g?o7~uf!wYca@oG{Ex17jc8dDKZ3Gf;+Vd*vLY)LI0!7Mwb(rb8io`w9 z4eSF~)_7{-;wd<%-GwEHsU1HsLCJ+sLFXMz(+bStpcS*;WIOUbNFFgnM(gM;u=YB5 zqVqxV>_gwYJR$`;X&|WbRPRvP{mz?XSRquqG($({^G)#3lYzmih0d?<^}F~wT7fgE z4;3r`-y8#!Zpa_Ok`mNRnPv`jc+eKuD4TPxIEo(8_0mH_#*|NHV?z-}jjcn-xK3+# z9_*4Z>2O@#;pDWkkmj*HFhaC$HlOTAJDN{(Irkzd!*gYE{!@4CegodSZM!?v&WY$E zv$&F*gN){wGwFtig1)4eZI@f}VPcPLGwsnrj<;%QCE3mPYK zP|Kq&M+?Q|DWHE$I_z31EF(2(D-}C6Mm3iPfVT%C!tzT3p&>*wml-2WX zr;vb9onVi*QqA7PL`jjGktFiy`%lkF4Mi4K%Ggd^B9Z7f!(ff#WkEFdl#@dI33O*U zB8x*56xb6z!DzZ>8e(R|c`~u06uJ-uD6yzQO!=uKhM}9W0FH?e>g25uar?4#x1XYx zPCqot-1$peG)mvkjpuHrtM0Yx&P&-i;uc=eM^TUwXyXm1qQTY=p_5*}|>co51mAuSzb_{s+d^*E71Ug;XKg}H#l$&`)MmpT$ zxc6*<78cHZ!%KgOjSLRe;2U?$$H<(d44$e^n(PwQq8^CYsTQ}w=*`i@;#bnk)GE~4 zGohwqyo6tA&l}KKj3}L&2b~5F$?;o|D9a|BIEqQjbdULP;hqo20B)Z?Q$=0D;yg0i zsTy{FO4ci&IW>y9x01Y%Uv9oG{rYOeAt&2}uF52^f<4+8)F`8qP@l{_}iQ5*{=oH#;LKV~H_9q9Cl4d2P+pMy$wKZfo zsJY?mWq0dmky_4+QNb4(TH`0~8^3CSqtEQ|FQA9hn!8Mo*OXXBq;<-VKOXy(eyvua z@&|?KHp=5PBOR6qmd8AbwP;os0s{WT>y^wOz>5Mt%kWoynGj*k%bVT-kcxEz_-kr< z6AtFu7)%V@E%z-@ssb1YzbVj^f8;L=KmJewJq*p+iT=llg~7$Yvlarnf(0c~e=qD4 zdya=kV9*8}?Br5i$+7k#5(n1S6+bpVe72@2!o@Gue~zr*KZEhaCTx^N4is%sv;!mg z&=DdwVPB$PLLC!Ebw7cnXvt7fEQb5K?KYv|VG+Jf4k4oy^o5CbKVMfl9QCmvf}*E@gOwyW48|!~grW-}&GaB!79+_~rl2=4km`}7+U!-s+zBt)z=h+Ld zDE>J%f4!}L3_P{U=w&T$HJ=$(VdeEjaYSQ4UCWs+I-Zn_TfVXJt<-yeIa?kkNzr4- zX$D+A!c6v)#{)OE-_Vv9H>&rdak8cwYuC6IijM2<8FMp*hh*^MY!!!;2x;~WbTcq9 zh9n~s-JyU!H={lVF3&BoEMQr>TjDQ9EK%uFR*u$C*R5JBdRh)I9z(D|j`*y_OU0Y| za|X#(=w(q9TdGhK0|xGfv3DR{BcH2EPopD;aCUN^IT8$$O_LIIfNEZ9R2`ND7e-xM z)q`6Mt|qnD`Wa4y5Uxpe)W}qY$RfNpMbrEtSlDPcu)77`r`r{W=y5h}D$Fb_Lpr!@ zHiTq}?;FHi!t(rUnUMYqPtNOs=rX0&6Ak;e_ECpPgSP_d{iFg%KMGk;F{)(@=w^M$ zR;il(AY5s+A6Z2AI&5@Av>zdZ!#K>LDg9|Cx1$mzkh6JIMcItVD5yvV8O=Y3um;CM zQBB#fF3{>%XtVERP&@WFb{vg1UI=thtOvnMGrXbh!dZk)2W3}j6p>ybkw-dXVjd!s_pT?K+l*xKoH2BVJSS5~W|3 zXr|l7kX-Xt5`zGXDf^Qk4(b>GUoz0poBfk=$^cGwEpKWZy3gR3N&{*^KJ1?6EK(D< zSKG^y$h^ob!HIK;Y+;9d6^cg~0!gB+{VbuHIaP}vw#*3r&5+Bgzpa{jt&kHNP=xxP zIbN9L`hgaiDpobS;Mt8SuSVty^}L^g@;@{H;fw;Nz1ieY2eoY8xx2TP)TJc#i&!~3 zJ!R^3#iThqUkA|x*VK4z1qB{>qLP`Bx+g#V#WLsnUo zZRxr7TNqzT(#Oy-+A^XtGP*XCn$>1Yu8j1KgiXVBH;#lYkGPD4WkRCerX*h^V^$O&#U|1yIkNC>@kC?Wf&p&-Pj{Cy6`x1od{{~MWppnT$U2Ty3 zOLm!G*;iw4d@F{e-c*BN@I-;CSOU1pGpDX}7|1FUpBi|Rx^_4H0Q)P9Iqp*6Ah(3I zG_VUU?-cG{;x<5@}>&~0G+Rj`v9vJJ*rjtRA4J;iN3vQW& zr2Y#DVLH)@yz9BlW|^Xy50T=amtskjR%o$PK=^7|8{Pw3Oz41^MJF!w!^E6{`GApg zQ#vr#bd>}YOeX((BWf$C@{F*1&K{a_oS>9TBs6{xDP>mfhq|nc9+=%PJ`jB7xMc%P z8Ob8d$Y0FiM{uBfKTc1@kJZkt<`Mk|nTNKTU^-p83qOm0CNZf^XJUv z&%A|Y{&Q}&iM7GLs|Kc9b=EhmnfW;z9CUQf5^DU_Yfh4|j%i@-O%!>EYxSFXWLotr zfAPF=!xFWV*sOS4G3OYUQzjo>Z5L~1Z_9T$BERVn3S}Of1E^Tmh>7EHG<2zGas_$HllCzK0TSRbNN?H$p>+vA=o^j z+=n<>2(=MPN2M5rHKlXnWr-Ckg%ndV*vHs7hZxC)lC|k2vrmTtd&0XE_EyuiP3=wD zX3@iH6ZsV7SGkizrcbV8uz3eV4p08l?ra1atz?CtNcz5v42b_4aqFO)4&5dULBYs1 zFnbUS6zY}PR}t%{iLPRIm|4M@LfrUufka)se2w7#NQYh+ zxP26&yXAXVX$$e6do6hIzaFO#dRBsS*kBJ-m&uIcPCURKsgfazs=-@PINwBe)$uhB zo+jXFpZ)#I9}oK@t3{lET21;#^+zf|u$QvamC<4c%h}tZdcwG7Mf>GprnU>%DD8OY zn5qe>s2l3xcx;4Ny3E%ppG7s+A{JguJIZoT66^@?GHKFU#{)vZF2Q@Q7gH<2=5n zpez919(kbS2T4%KO39L=bbsi&cOKQ;!R>Xp7-H7-aYF-7F<@U6}^>6>cBoT_M5xTO2N~m{J7w2jJMp0wID}=QK`<7 zCMMa)`{R4k1f8VmHDF9GZFcJ^IBNqvs`r8Cix4MY71DKE{Arf~g zMP?Q&)kvQ;Zz-r&Pb2A!^@fk^8xn|ms8zlMP3J+jV%fOd&GR-7UyFG&A!1F0M&tVa zPa@SyOnTlk@Ma*|!p@mDiQ9HNN_&(vH}Lk%6TCr5+aa*ldqakDyV>l%*)uXB?Ug@B zMXWO9xg&Q8_kf~S=UT>6G77$1()_NLEI{>b!3u0qHliMP&B#m3aUx>=I6mjK*6>=4tLw{WlhI=CGZmAYQ_ud2PY-qn{STkrxz$;+oGyKK8*Ikvn#S{}gObNk z%tg5^bFn$)Y^kjsIn>7E>N&znOhQH_hep08C-XB3fYKQy{4zDak{udxg60}rEGMsX z`s!1yhC}f$c-LNC2aRAsgqvf&%N@ANTOM$G>MF*OlC{#cQnfM;k`B@iqxEC;qxIuX zBTi#Z%n2+Bu0!IX!y(I#v>Cb;_L7w8%3flJF;=@%-yROdYSAFBWRS(=?!%Jif(AdRk$3 zVWG)eLN2_raD+(d5FRvcBqi=ipZ&hc$Q(Pq>jbjMR|N9!LDyu270Dl%=T!r2!`mXX zai_l(h#uGfER+{CWNv7Psfs|oq8ZtC4bBppy9@E#do4Oh_fEM$#T*Qdi17IqiYCJN zDr%&1s7*}hj{G97iHp`D5>8TN!X(y~(=8DevOvDtSGV3b{#oL58MnKEro0o%N=C6m zyNkk9@=!n9XUV6MB$cEDQ4qCl=rt)^cqYBf%8-~47?Z=~2Totze ztz>?wfR{K-E$xjt&uE31c{C*^ltvTrSR8`OAfj&7M0$vT+z`07z6PY`nmk&b4`%yn z>T39qJBT|++pSnGN~=*1ryeX9@@FQqnC%24Z#AgQhu6FzxH~UXI7vWmd;9Y^ zdKP%IVl%rXq(t0s)?#wM$V~~vb-XzvaluZ5HKESH1vrM#u6xces6CbQoYQbJ(wlT$ zx0~b>5^F(GW3z08T>4yhme2&HabB$86$xgnYpMQk}>{joFI%zS1>&G1KO;<78JpuVCPaxNTy#t zJ(HxRVj!JRqx!V?C{;gFLseH=H}I=}98$v9C=%=#N)fU2%>)<$qmR}hNnmR^N$o~t zkk;{IJ@H%zh)Qm~@Bcji9;(em(|92uy$ZxB>}(C< z-A8A5NsKb{@Kb+Ilok6S86hm`La(>k;pgf1$7UGls7h906S{|jo*~gl%!7*#v{L1* z{7p9OznDP>ix@iDoe({Z!ilwgrH`!-U!^G-!QLm&@qVNWFSH6pPo4F}*3VnvFB_&7 ziCR&I%mgJZ3#FwAe|?-LX+=+kmRSxEk_nZ>qTk1FA#IjNBjqqJ4#%hd&ImFbJ-KXb zEXG$5#1L}o?=rgS|BVwD5@KXptSe(6QwcX;lZv4hu3E(j&Rgn(^1j)y8J*le*JiTY z88u;ATV}I4JumiuuHUs{b;P8zo1rdoxe30r5i+R7_)TfPx9aSpv`%-zEgU)Yn?@D_ z*0U&6bC-a~W94!>`wwOpK+`fFtK%bpQn^w?DSwHtN)4?)UZ2n_PvOz`WeMsjnqn{@ zt0A0ME02!sQh8Y{uc>vG<=UOH%wdylpG7h`dWoFFV3@Yl&#o1kXj)lO9)_S*_n>Wr z8H`Iw5R4I7gB!Ojelja-gb)4s*kDmr+XUA-wafTiZBDIT$cDWoO-E15uw|t^=KnNy z7Eo<;Yrn_cQlLQ5LXqMST#FPhP^7qfan}SW6!+p3D_&d*MFSKoUTD$c?hsr;Zs>d7 z?>qOrXWey^OlI<*XYXg{nOVt7)}G(La-FS^lD)j5-0`w&q0!(#trPWPscSNo<05St z{np=Q{NDSj2aFOoJHNI!!$4G*CR5fP4id857hqh&)n<+OsB~Jgb|rDP`{Kf-j{&@l zB@HCvELGM097_6nh_rVnZxVyH5Fm(kJ9#@fUe@?VT}Ev3n^Ud6vcpBth&5)Ud&3n< z%!=&3Ewz5-N8m5>CdJNVRn<-QI%{5vd5s$jO!f}(0(R}M@0w#cz5^=;PBtiIQ{~Nn zY~F0%=&lds`+@3fTNt10lID-6=o}f%@<=OEXL|N2hHK_dK{!|~7w+<1E@|$aVqgOP zD@TMWB(f8MOBwA7oQ}reuHFK!wemBc7R=`dkc0`o(QS`%DLZ{-OYs8dw1_$>c_~8o zQr@OAL(i2xzfi!+j;yulDl_|b|C&Y4r_zXQYdP6+W}n04qlU#b&>2G@dgi~v?sHaprT-~yZMtK%NWUcdH#B{;M^}B*KtIP5^{|Tx2T*JU0 z1oI#Xi&xaD$G+kKvOy}>`=wXH@i@QHy+n0-eYK@6^_RRNY^+N^Th`7wsAxMzvFGEQ z&?&d3_peP*u^4?FdMeD{y1PQt+DLdY`}GYN!1mtX!*&Ba?^~AHm)PMI%cQ$jRcviK z7*%-2NBRdwYX)nOr!GyP4{Le{6h~E^X%}NCOU;>Ij!>IrI}}@w{2*G} z+o(#@gJ10!!%SB2I%L2;C*F%!gUvc#w!tF&)0wwaTo3&mATb1sLXc@k-{5G5=%9)u zw8h&c#Q*1V*uVhxTCSA;vAfH%aP5gZ$m(bT7usebOX5-bE7=>c&OtyV$*XYU?P;hSsP3d8c})XAkl!)*uL9T#W5*w zU$evND^Aa=VRxc8%j68)2l3qG^H)3vu{BBUSmiU(&6vj$#V*uwkfdF}&6AxM0i)+W zkjJ;A7S|%f{Vp`D#WbPPp>FqRTBk^2(RR!MaMv4uhBhqVPPV}!_LlSTPJiB*tBZXn zPhfb$>aXT+>2vxcl^llC7#NBRrq7GriB_=~uEFVSgveo6E%L&6kB~KJR(51Ufg5(* zctdzw!b*0bo?3F`tfppFdL3i}3jKM^i>(^%#7h=WR$FnqUAKGS1k^tt>ApMrMV1S} zGZU9Mb(bQXG)2tW2TVFa&7@@5h^Cfxm(qRcf1CPJRP8kjO)0Ko?MvdAAyo}&gyPys z2!WZqg)2oVvnu-LDu>bBxqY6>5wX0vL34=yYY2s#XBXX1$Y05J)BigbN8u)!&F# zknNS74$H0S@u}wDpbhinCFCgHN%KX!&zehQ`9#_Fsi1?#03LAucIF>i%2931f#|5V z1n)2t8;6sx2)0!2T$pK5%ejrhJp!nIu#b$XU~|dZqc9H;%jr+<8j0f4bZMa|P&9zE zGcv#0D%y0*Tn&HUy=KuA=zMYgh@52sXKP44zxpSE2y1pN!4l@Yz;X4m;U3L)n7uI7 z&}D<-5mOD7WqpeHRZcvE1YtKQV)7a)W&DiRG79Q*h>Jd>Jh*oEBoY34?(qx_jHU>! z@(5OP4kuoAW{WQv4iLL6a^p_>cG(>3(+wrvi%H*egp7>!}_?=}3BWX-pGA~9> zH-FOpF#PdN05(nEncN47`SG2P0%to8%4v?8`3CA8SpiX}U%LVVlj5JPDaa*#WDvG! zbk1trg=9L*i7*Mb(K3+-7+H1-3-v=N+Tz{Fr~M5%OD@L5KMQOMD0cj2`ogvekI7r7 z>Af`m`AM^Xt1e8!(PmeporbEIM!TiUP$ozG+4@M+v3Kh+rC^umgvq#ZUdKWw z+u2Kwm~YLHXS?bp8cLpnLOw5GI4y$~>Qjfob`5XNX!vOL{r8HN>o8ucQiXCtEoPXo zyd0XOmg!LAaa!^^+{X*04Rwp(q1$Mx^2W_V146;H6|+Cd!ox?>($yUS#R=(k)`0oZp?-K zC!ojqhVlui^l>!d5p^W4Lh^0?CVFYLY>|P5BkD?gpB97N()S}pO}~5$1MgCO#olw$ zP*+|3Drmmow!Snv&?;yON0D(tjRT=hX)fv4Sq;e1fom;0t2s%%kRpT=vfr&Kn{>+U zQ0cz+2xvSf==pX&j;xp&(-#oRDG~wY$Ef%r_0!I85Qh=Js@QK4f3Xulg(N(YFS=pH ztxSV!RKz*ZYP?Mc$M!wCCngf4JP zRdwB$FVp)zVoz?QJ7`Du^2Mm)ztD7&6M5cuzqIO(Z<^bQX2%Yl!Wj)ipAF8ogVPO#Je^9C>)zu zuW-B1ulBdN7v#Xm$Vd1K&wE?WvAh}D%opwP`9{@#HDNI3%zNtlitau^^I~s`&U5td zardUGZVj+@B14{76-L{Agr&?b9cUib^?cJbbhLdqseI6g!R;syNo~GV_cfBsY?&djiXWC)}XNi%?41) zwcJhyHcX|3p{FB}PKaa@Pngce5AC|JFf8{9O_QHX<;d7Rr+q;Y{KD~NAQ>JV-6G?1 z>~{6jhTqliz-X1nbJ@qNm6JzVdlps7^XFaHR_oKIM8!qvZ!AP`Cf$b{_{SPI<{LeW z*{{n49DNE-$HY(JYC&zMyxq}QxA#jl(a005@51sr;=L3&<`OY~NC*>Fq1wvnxz!PO zGq8lqS=UQ9*=&=XB-@pa*S%5i; zrBvn_t^ffua8;B#dGLDpeQt^5LEs%?rU<*HYYsZeX}5=~0qyFN>d5)s`-aMDvnn`O zQKdPCqXsqMHV}q?&=EG(F$RIU4wFm+=bx-Y0*XxRm zVy)B&oUOJL{Q!&FkplqLdjk3ra=A9GwqvL2Qm8}Dv7Si(R@gGlI`Y*!lWqpBYm%HzwaUY823J2|jSV()K%R&%FDN^1)q(?XeQ%%7FMX6TkZ@WNEQk@> zoqnQDkW49ZhMY`h3*UkgsmpITeQU9=V9qZ;SKN-;IKV0ute5A`mb>I`&s~r`IWPpK zR`=F19g#WqQ*s}qZ8vi)B!M{hWqL9aLhSAC$FW!N&}h3Khc<8Oe;|ysfDFCfOUh+a z8vrpw!p8eB?s^+=HW|bstq4=aS)x%tlWllm&=mQI6?>^1uq?^vM4fD8{$hKX+Nv!f2qQSOB1cRUtkz6($E<*X(p^t`&2n;m9d$LK|)Xu_V|2R=x4aV z1hvqiZY?vPe6i3W;+x|;GANeZtuuyEGa%|LWe&gx=(nYozej5sAkOEKuBArc6hX z`CfKE-aMl=HNE&`)X@=z2NKC0y2?}8&ws0TTfhl>)02NDw8=FiP`akWpfY*M=GGr5 zvY6rjE5d)2&*RRDLj>aZ5lkPGTqOk=i}^0`ZfE|O%@_S$@LHogPrq~|nj5Gg+ zwRY7y0~UvFzZ_sw2!S4$6FN5|&j?&bYYIJWaWm%^- z#;xyob%A%`jG;VIn@!3nw|SwQ!O-ORQj(OP=ZioqN)lzV0$3fUl@ZGHA(f#Hp&>I2 zAG(pltqtwwLz$Aid~%DU+3*wlDoq#8S=lg4u70ZXh5gpBfE6r2{ zd1)9Y7St`mkXy1{5g979UOSnuX3*J@?DZB`SIth*fP2@%5qa=xAp$Ysn5S35Vq{I|6x!RmENQgdZjZ$3_T{EdA4puU}b{up>1Rv{u{IX zyAvHnl?vH`vsQ7w6H0?xUlgoO7}}pV!1Y2)81KV+bY67>>cJ{9i?b_UEmz1GQ}Zi< za1T_xmqh+4=4ufas*k&r{0nMK_#B86B)?rRqPG69F(`_`41;Ma%d=O%Z0`U(ey-D% zGSysTARgr|js;XVAuauor{DY-BeSDYgv=p|_yC3B%Obue$3ZO^_-sS%shL5f*TNn* zitE%js#dF`MHJ+UK-u_etQ#4j3^{W8&6v}dqAGGrBBnW%>bn5u$)13)Nb!>X7jm@J z$Akmj5yH!`}7&mut_3^e0OnPYjd#-f_AQKg12_9~*EdyBcv(KvW8PeI&_uvzH6 z+}z;exCM^?xCFj*=EaGuN}=Q1VMWAb1}S~seZ3(kp?-R^8?0e?{?4?bnl4$l)1_LlbF52`U zFJyy}Iur)yLUKiVylz^qR>->39f)mfMWoxovn)sKCX7OS#6{!OJi2b=;lt`zXYLMfLu>YNM3X@|}TubfXgv7!YLE)pLr_et@ySvKPvF z3VF76QZa-0E3z$8gPXeu0w?LE?vC&wAq&xO383CaK+O*!W`h@X=;xd#zBbpw>8>p!X_Q5`lg*gwYOFJh<%TG* zYqn<06t>q)R3ot0dhZ&MP~yJ)4obtD$(si^m&CXyd=?^XqoA?J-=5zjR;Q$Le%k#l zto@bOJ_>bR?#_A1rq!VFPVN;AGac(_nssQPxOC2Y2^f2XlF16Ffy@@7n2BQZVKkwo z!11Mx(_@M)tLGQTPm4vq5^*sUzN);+8l4_*euC!86h37~yNmP|qIM<-yLAJYWQMri zajDad-Y<*07Gvg0*<=rJCXE-UhcOt%^fWTK2PPf%dE_C(Wxrx=i9WVI*%+6~48A|! zvpqtq1xs}IrpKG^N|nldC0pU$rr%Yg%f1Ub06e1@sy6?e=Ui>NaEbWc5qyXn=#Qx) zljZY7Le1G*{p^R-d%JyKo+-XEK?%Zn*R$kj4J?Iui8?3u=QGYthY(G};JIjjYs&dD zfkD8wb1pJWTe2I!*=TUK*=QfPG?6)oZ2ISW_4oF$%oRz*m83|>wJ8Si8)?4t?#P=G z&55jP-Rv7G+j)Eipks-Nu733bs_-S2FX~lsFT^jX|x>aYlvcx1Cfq30aG_c%jHvhp;u#+?JSBC@`AkA)iR zz63mj@ia&!URuIFzHPqaUf=Pex>ncgc@uYKMlu+dI>T1vQ5{1-o?@M*O%Cp=tYw!5FD_Klv=@Xu&5udzXFm9D{qjNTS zC$UY(B4GZBgn2~%I)$qmzyl+_Iqi(ED{G?38q4MbZ_%xmkz_HnYDGmXN%h1m@sFR5 z@vJtjdJ}WF&MA@0)TLT}9;fhVq=fM$3r;FD9;f9|N|p`Ld8s2Mus?GBz9`hcNKfi( zY&MX*Xu7s=Y~(a81zk6O#QVK+h#mXb_tmp8b@(onRV``<8VxBLqPu_E$g@J;oz|cu zbKh`WwjI}H~Wshp`gB<15EZp&{g0@bEO&&@&9s-V3{>!<6(}wMS$+Z@CvaN+z zbp%vv>s#WHXC}Y{l1pY~Zq?ij$YoY*T2|{c-P>WmVB?97Q2YE}tD=g~fWX6?+xav! zsOU4Y9ZYEJ@UenFYD?^euS|}NF{VzZb7%N+CQRa&i z2^mX&b7ZPz{QNXqrj*I!qiEcoBOEknlcsyyyXh36IOd*PLATHMw6lbxGq$8cL+t(9 z)N@;Hs>%M5ED(%V>*Rrj_{jmkPpkK8YTrk0_4J`!g@>zlbgZ|miQr9$WnUj;ms2|{ zBGvJmQ-`T$?5W>~an{^|z|-hbf<4I_;HNOxtS{vDuU-m~P{<~7$Wo5Zisig&IC=G6 zI$xcI1n2*2brOh1tk&>pz9FYWjbL@XSpUStq<(ZA7W$Zowm{XNHOC8}#i!HMgC5+}L%cI;Zpt`2J|*%k1fBIxhqO z6$9tNAXD<;2&zhzQ(cJ#YQ18Pt`_BSxwJPb^>p}r6K3{OqTA>)rsS5&*$T6maY2g5 ziNo0Ty;G<5;}_<86gx@&7h&tG1pc*K(QZ7~*Iy4LTW2qRFgwhO7+knyXSaRsy%>7y zV4_RP8#*>Zm>o}I_i|={o-aU>eNB>?%%X78&XFf0$bL2>x#mrJ1zu`)mf9n7NgyiT;-kK^0M={mTP<-{X$Y#;PM?tMHb}!h#&4$ zZ#6^9Xmx=dhk&tQDWb z&y(MnR|YDnok%i)q+lMF{17FkW!vxCKLb(%@=(u$VRV!~_+_UDkj8>1JJHCkYMhqW zCYK+5l`EGb_dsHXnCEAbKpfEZ&lCV~emFX5LI2V+B>Dakgbr4HwKriBdfBaZikqra zqCf-1-ut?;aQtP!q%$zHg8k=YB8o(0$0Xvo0BtebO|dLb6^+dfCU9R5puG~sR*GIs zyAMD6^r74T%*7NHBTp^8v!$P?3-2}zRg6mL+K3&8Sm;EI-VDVA&h+ps9Ho%nxfl`= zioHJj@M!IV{g?vVB4#j*1z%neH!q_@AgQx4sI4K4$e7mFf%gQ5b z6~wej>u2czQ#n!~YsQMj`3ebFQ;sf!QMt$W2%jHal2f4t@zH4o@bl~znnf)l>&-}MlFMnbSmL06HO-RZ5+byg zdSWSK6dXpVxc9CSeI$>J3b~%nK)~j)%Uqh>s@`nPoJzrb@1xvq!{FwRY2ib^X}rhV zUsFQ4)6%~9PZ}Gmy?iOFrMt|oPyiebGOj}z)?IN-3Wy@KpG;OcH<^lsS^bQLRLHB?#)h74+)2B97lRLv+*22vn zK1OToWVon{t+>8!3>YuCQWPv;r)X`ifxHWrw zq%(RPpRqKzySMYiqG?vX@&}KX@>ief^r;@FCMUp-z9t_{e9??R83S!AI@yPxC2a|50C=lSI%1T7z_J}0$&{zz3eWG*e3RO%2Lr}akZRc-tujSnmr3*{MA z?+{nOy^X@lOwHV(m$4mYdVb@%bBeAY`xBP)?AH;pQd-)_IE&%z#R zv&nZ297mHp;2VF0S7&JvV57Dd1o<+yB!YA^Y(sF!$rB6Cb+)uIw7gQ`7@vl?Hyw*& z41W!zdwX~$4`PTFCmc8g%*NhtW1Tm6dBvWEVt#sK@wN6BgyD%=qvy|GOFGjh*Dbpu zH0rwHEETFhf4zYR?fWGzFztRO^ZXUvGS}@`$C+f&R$$+w$Z>)%-lzs`GmM$;n1K2y zC;A6|plXZVU<_C8cl`q8-9TU4l(~Nlq>ZRZNlXDGzMoDW(2uaxVJcsDvGCmy@)!QF zA?FqhdH*g1^ATog;dxmpEUh!NunYCmH%Cg9yAo{QN^2i^J{|=UV0lyD@aD*Z5d#6H z#%&K%#@^-x6soVebLzFq`mBmLGE;zy#BT4UzN)I+)zK~dMc>fY;*$XS@7=}%3Xxx} z{VcMn!rzTLIoPXtPYYEQ_3S(a^kXpTCNORe4)7U>p-p?4XKj&-B}DRqc+yWUGIbew zrWsE^Y*#D?#N&T@al>47`o8tQpl<)&HRQj7l4TTSA5g~+`*{Cfyktb2rmwq)m7}7Q zweug^vxb$89fFJNi^C#?FepKok8r8ESz5V0FqK)fWi%15H9cKi9IPB4{6_xZskt69 zJs_4J3{w#HB|QJ~7D4o_z5VwqRyPeWc|9N+`$7byJDn3wmV5jPIs145e*_`eb# z4&MX5nNJWQJ>+{xA&5vJARzE(ZG^)QM7W|LRD|$%$lrkv>xboGOx%BMArc|BxF5O^ zLX67@M~DZp@n@e9AEE=qfDrOSz(b=y9sCJ?P#zW|#QlFM-G6HIPt^6lRq;<%A1rAQ zb}s(`xkgy2*tjDq^(RF7FP|3<0RcW74j?Zd4lg$%8xJ=(KO!NQ)C2h115qZSzbV)r zI2^nPN;#Jz!u7_^Ldwa;!3r^UgbR*~f|Z?(Eg}|CdxVvZmE#*6A;gY{gNtBaBTl0~ zfaU*-kNr0R9mxMb0Q5g_<%c)M%gZCk|33(HM2i3W-9FgXxZ`js+gU!~xc?$=|M92s zL=5+ji_X6w+&*qr);QdByal{&Ot6{6akcsc{2&fd5%5UIE^JwB-i!3;Z*V?;myH72p&4w_16)nIX;< zw?CgGns$CxIEdrMrQz)CfjC_s;Obm|K4<=dum1sXo4I-XIl_n%3Gm=BGQL!k!}&ky C09b|q literal 0 HcmV?d00001 diff --git a/publications/whitepaper/figures/realtime_flow.png b/publications/whitepaper/figures/realtime_flow.png deleted file mode 100644 index 4f50a5c54b5ad3c25fe42989ffc52384fc32400c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 52345 zcmeFY_dnZz^f(%OkJ@{+XlvCdwRfpmv_MC}=S zlf(>y-1M!V&-dOx;65JrhwJh1kT}otJkPVw^PJZ!_Mwpu4J8L9005wQpsQsH01%<^ zzYgT2`2VQ6Jlp^Pz<>u@>W_k_ciO}Z%G8-KLBxdBgpq1|Px$D`i3uYoiSeK5--^)7 zs9XR2v_kWop1hmc`~zvE+R!T*{1SDu;o&zj1^hH#x_X?+0&W_n#9ls_)~o{+DD*(7 zg{r?7c?s%Uu?=*EB!)a2l*XOj(;>!_|Nnpf-;uyV7~8j^BX@<8q8WFE%A)o$nMk(W^<>Q>Yte+@Kd4%z)I&q#u|qdU<=2C)kc_twk1lz#%-`njQg)Cz9cj#T?{%{O zR)US3dtRNoM~5?}p=3^WjE156GsiYRU)<%^t2e+|B97}~96`gNV(H!VWU+OL)vQth zAM7E{vN^p0xDQNfoG&Yi;hPV*D~8^&Udis*{W9rOdN~z>ODL_lAhM@KvmORCE2(>t?dzAaZ>!k z$+148a256*Hk${YuWJ`Efw5!5l^M`O6CxXpaE8iY@Abwa5T2%{XX^7x`-0! zN=O?_NP+4LcEZbB{CV8Wi{?pST;@;Koc{C!AHn^2?fFd~|00lBqxaQna6%X61Kr&u z%&&dgTD}FcU7Uoh1-u!r;lo}IeyR8JJNS?=0d*@hQo-@O*;29d`_FWaOth`omAOV<$EXpgk{DiCmF^A ztib*b5}av(3Dsf8-VvY2j*5SousnX3bwH`T@>h4IxS3s=r*Tt}-5=C!ORIPWq#jji zU>!pUn$>C(x3iO4RPiCS);@gnj?nhh3sT~zM02ZnaQn%wQc$N_m+hT4HW%hs$ zI{WI+$kI(nAx`WB=XLiY@9jSfSfbut8&XFboT|(Gz@w4J=(*I@D_;hHB^(Pr(-Qes zKv5-@$0{X|?1|N*WhVYB%#QUh37hq$n?2o)$e;-zD*m`>8{@HGFJT*jd1@H8zlNs$ zM`HTbU;(b8sF#is-RVAEJ&%J%f(<*Jl>hA$lTlv8*_YM<>FZ7J*k(A7UihSNr^=Ct z0{D;W64M$L&$Us}^PEWkzumjPW?xbuRW{FLf{ zcUNNwkZBP2N!SdBv?!Eezg=u9s8u~Oq>KEZ`mM->hvwfII5`NB;tYe==l#VLKIG=? zY9<~v3jh+2uE0Z3gT9%U0jx;#eZRDdQhVI>I-c#vlgrgSSMbm`lT@wZR(9u|6QOK2 z+cgXS188Q;ZbKwDOzpgQ($Y>;VPj5}rC;8ACeL&03elfY!KxfOo)`&0d>ouLS8-jM zLoLsC|C5ZGj`a09{%INHnO4mFM>^eZo^74r*?&iR{{Ha@-m&E&sG2a*z8Mzqox^Me zSl9nZ^*s1d>t*Wiu$DJYR;kFqvm#6@=bT^h&hEcB&7;|~^?CG%@x;`IEoF=BZ6Ezj zJkR=3z5gV8!Y}%0#D>p0v9JLXZC)S?yNLf^u3$-hx?gQGf?vz|!y`7TVVqq5;*T1c zNktLRp(I$^Eu&1=i@c5KfAtV9&+i$x#CByWysPw z+ktL5XPf%`j`oPuRyD}jU3t8cif8*OxGh8*n2!YT&B9*;XLs!l5Lp)?O$hUmyegV2 zIN+(4W?nWip;>nIfSIUQv4GDYPD2MQ%|vxYK}G}Pj7yysM3jDKK*872N5jf^i!H8T zd`0uG&OSODjbBn4jl<5^%ZT9}K$`z$`0~1M>q{F2RlVwPPW-v_-PkE1hmct0YgL+@iR{*pvVWQo zFMMQ}Y1$%Vg?siq&^5R8x?|!|*{3V?4FUgWHm;t$1>^EhGx9cd4=oW7%Hf1c<5zC& ze~-B&yRPkP(Sx|cHB3L9namY>_Uku1ifPlp5h^DAi7zv}fdzq9-Dy=~ zg#UqauM^MlhGZrMz*iN_-mUK=?~{3I^u9WqH4jE!;j7Zkt;25D>3_pL8KKu6G25^` zfvWwSrxcO6f`h0Jghn-rpE)t<8%tXmPN-)$rpQH*UjflH-OzHmF&WKja8jH~F6bKD z?3JQU1pF<}ZTWujls=y#=2Ux+jJ8V;O+50fyuv%t;;-iq=8C~c-t^(wwu?T4s$X0o zx=HMrZV-hWDA5kxe(FOT-CS_A)le06C5r{kN8dJY(_68Q$!J%raehj{Z+Y-f_i7#+ zqMIgs707R!tUZK~j;vOBHEOch+y4WWY~~HIssa9j@2`Q&AbF9-TIq0!2fMN~+-i=Q ze=BEv?X01;sj@O;c2co$iwl;P+$vumM!U6Wx5fV-xPueAUyqw~RWV)_`=QtoBC0 z+Rvx{E{=YqVV&L}cKyFS0IDq6siVMM6_L%w272kCU3}1~9t|y2`NsmSUV+!qE z^lejsp2K{9&4m-7V$?JJXcAU*>q_2kuV3Ef)&&}P0Nhlk>%TLLZ*e8-UL~@Jy1%Bz z@83v>4a;`Ti{yaKeqdY{l5sn427bcK{E%)O;Q+44m4P=^-3q zrhYP#;5_3n>U2?5_3c0V&&|JSJ_6#Ky{j~vTm|RO_fCN)-bC}e{D#37C}cl}d&(RK%mHTl2z z4k8{gakas;8b1^TC>y#zMCrb;{jaJ)i@E8XTUdxw0Zza6p3#&XS+tA%6WIyF=)oax zpOUdhZ}@=1765STjX=wu<9D8ai5mi@JbYY|{Q&C`#RJG?$Mf@L^!`^1mVYKz8y^y4 zC4kCCGCD|z09qlpLJrced3R>yUwIY+t~J{C#5G_9FXh@;`4Vm}B{|=#ve7VaasK#~ zO9cRZdYA!>RUS`#colL_Q&tA5N8dRo6x9>!m;ulM$Lp3-s+?OVwEvDopg>fAvrgKc zZ0u2y8(`s`NheICE=l~+B;)3PZV{S;FULr}U*vrj!dv9Yvr&_^q*3oxONRRWzfxAC z%fsp?Mi1`U+!ezM3f=Yf=S)Z5%s0+|(d3Z48DfJ;eba9?@oKdMfQH6)S?2Uv6;S_0 z23KgNXIFf(4vF4E`^yxzI%a$ zfD^$Zo^5={_`B_i_w6x%cK=65I5WH=4(~M!+o<+&Z(N$$9cygEYF2Y^F9hw%0RTf>&8(L)^!q{kYnuOgZ|-xwvBP=Gn3uw5-cQ`QrLrk$ zGXMaU@3r_c0%Eixk~tt|;{2E3!g;#7j;ZTJZvm)i<43m={s?5*z;>#go!C*@4?I)r zIAwwV%EpxP+=Gox^%@|u+0!;1jxl)z0C?%(xeb!f z@*-S|T5H(!m#+W-ETzU7iVoT2y7pHwL!53K4LF;)^g$f{j$izF?Up#c^k3EGpsNsA z!pGZY>VSJsDvC77*v!!aVLS}u7+|7q|eH&S9t4Atd_fg^5?k|r)U-G1wuf$WV5G{ z=5HXy&yO4rcrwcvQ1gGT6d!Z0p@d8TzDV(mpml8p&93*N4rC*RcLNC2U2(Lpz-ep4sp0TznQl02W>R}8#o zQ>vrbh$Kg|&(H$66TCMM%O5qjf}#Wnz_vZEuA?`NOt=8}(rJB{pMLLmx2s!LU=Z<< znhq96T3OPcf5FGXhhmwNmw17L_~W8C=fBI6hCKa9A!BpoRLb6}m}#@$Y%S!+JTdI{xg zw>f;$Rr`1v(4k1%Bxm0X0HeqvZUTmmPGCXVk*+>B0RB|d&s(Ih^<$cta)-rmQuo*H zAFSj?r^f1>vCn{2I?D#d^&A`6J*l594yOrGa9XOt68Mrh|0Tgw2gKH&8`>ti8+5DS zB>>oaySvfV@wkmhtwGZJ*zh9k{b9!it>wBcDkQNNqUAcPoRyQ7S}|&o+@Q|GdN^UK zn%l)H}HzjrxOvE{j@Rb_(@--TSHNuwJ6OYk$8- z1#UGx=ekl(K9$mv1&MVA4c57Eewa$)(?KAR|aZCEF&xC5}RgR>)ad`l+_j+-JhNJ(LV#SvA?K&i)znxiNvYWR7Kh-S*vJ zInL+V6H^rCVa*s-&R~8gjq#p4E3;vPNse%c6$#st*P+s{Fi*#O$CmWzF>$jo-63lN z1iLEKIR)bpo$Xo2e$RS$J%&gjjBtU#yfsX4(pR(Vf7H4lH{qK0{q*SaYQ~fUKo0>R z`)z`*>OKPi%;O+K-#9lOLS@cY_4ykcD|kRlFjyvX0?B^j$~fV|STA>cs>L^3Av&L& z9dYO0MCgm(nvBQ#fySSMgqn}hvz3umbUio&M#9UnS4XcX40)+TYH^8A6%x-_Gy}!x zk_TkcqdUPwwBkofKD!XbmZ_Vf;_%y3x(VzqX?F?A25d zU_;Eh%s;csh}9IUwDe{+_i}H5n%dD^$!K% zt$QNL>Z7A0Wg+U#>zxdGVwjH)eW3G6OCI?-^gEhGkv`LyrA5x4(C zqe@^Z8Hv8Ie!%6SU&D@L(iF-y_9G5=4>_qG`vN0f4a`1Ky8D7@9-bOo^itrI`bh0X z>&I6cdNe90wrK5^WAhdilXxgLdbyCaHGDiw;Tu`H*tm2#8KZg=O4}ukQjTq0g7vb* z66Zo3ZiOJi4A1Iu0l>Unz|Q{GkDqE2v`xyK5;GLwPPoL4)=r6xA zmpk+=bHly|Y%8yH-4jHfgD5V{ZX9P`LyO<;@=yCHXN=By5E=pe6wcfSj&1cQnVY%% zbbX;e7CZa#M=3cn3j3Di+%_89*78YA;`|%wNq8IW1e+ZU%N)|0L6@sHf@dXM)`h?1Y`CwUX+TmcZRpE>)Dw8?t0YSd2#x(YOt|KTkWL8a1Uo9>{p%H4Z8%PJIWo(qdrv96`-AcN78z97mIf<8O< z_BgN)RYXDX7%8A5ap!Qrzq2j;QAgra*RLj`2!}q$ouZ|^Gnfu{^~AMH62)hw-=E*O zml()H>NhyzDiI7S&-=j&v+6OOb*Iq9gXAup;Ez=VrX3v z62%W|H_f9qo7kF|?34VlUS?pVID6dmGx+9Lvk%YAq>A0TC}!4Y7>=ok$6YCd?F55Z zlxO(+ve2nF`~n+tf-6JrLH9Ohi<2%+#<~>`^9d32oe_-3daezX99)%_1nnOWDllQQ zUZx#kDLnJue}21XyOz*N7voex#bDOHa7F_yRFWdz=FuD6l~Fi)zy=^3VlC%?h7eiM zH|2$oD|!~*oL3i^_ZWS?@8>!7?%c9w)zZpvHYTov@j>stjZxNm!|0F7qrx1ZiLiJ= zMrn3~{&OHj8w8U+ibVOvXaWsqmooxBVe;nZCqtGjeyp2tbEk?D8zB793D0iZ?xnUB zEKj!VQOD5o%*qARA}Fwot4Gbrl&f_P4){2}5Yn6Hs3Hp&6q-F_a9*F{P8``%I8rPO2b`lx!Kd%(rLU{$0rX+2D@BW5DVL)Kn| z-h1&9qZ2moIkg_KRt7KKq7fyP&V!Yc}|0h!a?0Bb#=r?%HMk=M^W)X7|>f+J~lxGCV<(qk>dSwvi{ zPy3Sx8jWW975y9Q*!&gD%dSfaSVmw|G)~#Jn(lLBuLI;-GmQ$> z!9|>c1LD-KHjKUAuQ!@du;n;fXzQiuLYq;-)2WmF>Gs&cwW4f~JB{mT`Ltby?^qH2 z-JF@K_*af|%9z0D5nf34&S)_Dqq-uJF<4@GyxzM^)W@K4ogzQ`)mxid_-D)1?%xS!i@qi3&mg6CP(95gI-K4p#TjC-Z_Lb~txgmumsGPU_m z?G-Z9#>2n$OV)lyHQZIes>OThbnwz95s43kC9?H%&Nf?6DOml1?Q{rsK!umY>ux|| z$Wi~iT|?SoXp!jrJ;Rf(leEv~n>(p4fT7j*da~P7hJ>(;TDG=NX4y%ToT!`vOm4aE zE_ox>Y;?E&9QO&6?~ToC_K?@??*kecA}m z#wY2Yd09DFa`b{?kR9fgW%?XEXCR6mu(>M(pZ zG1D)`>24=<|1#z5ms5dz3Pjv2XrNNyUFW4om$1J>w?>=tt_Shqzoha8nevW{!yx$! zuw~akwz*PxL16{dq;;uv{>@U5%spFo-a3>%jk#dfXL@8<40yip7PQ5UUZC!mlX7T+ zoX{_`-4%^3Cv_}nkChQUS7VH}P?(|nN#jUOkB0pCP**J-mnV$I%ra{e08~@8H!Chw zssSR+uB`X>cy=D`FTd{J`_9&ak^kPjKbT;~){l9-<@8fl!B{l--O?Enh@Vc2-0W9c z4=HI&-Mf%sWNk8ip_H_C?9M{t-2<1b}K(;UajEHasS17wM6VP}T2{cmSkc zX_BQlS&v$OoaXr)qeI+EJDjzi5?eX=$&GKJOQC3KPV@3TC>d;bGgwytk6(CA^OHvj zUv6gAB<#E^{3Bk?nJv0tPExFoh6Sy@p*O9$7O{KLr>28a#&}amPy%e;QbE0^whwp+ zP+>Es{VT!-ht&Ekzt(fre)oJ2l~2Nqb{iuQ4t9iXR%p}bk2_je_fi_r+r~_2?QpFQ z6!Y5!%dj+j4V9jZnBbvZqn7cVXH-`lak8y_m~O?4MoKIgcdA9H_P&e4Vg>6|NJ(!T z%+n(yQEx2gvuRn-$f}nK6YKyrD>P`5o-0hUk zxIX;mhLwgYYxeSe_w4k?pXnQ6xZ8;BGRLFMY(fNTPpX+Gk$dhrA)a&OC-)t` zLK4YOQnSuv#4Ky0pP{f%rCgu)LvAkgZ0(xKbVzcILSFhuR8L|uj;m&m^VRtvr*q$F zL7dUar#EimyP~zx$T-Zu?-3bguuC6x9i{4A80?m%+r2~)MdpF3 zm!h_XfrXtq%gk5|Uzeu%vzhmv+qyli**7$Q&9ympjIF3L!PQ&G2n_(Rl|&xQ0J|?z69sRZ8epaf;d< zv^DZeD>m_*kyk|6M@yw-9TZvg%cJT?V_N!KAS8WD0%2T>eL@N8LD?l|`5FP*HNidR zfxkOhP&3>D+_e_z#d%Vn`fovZyHRuPopm%?kN@%j(YfAO-C6W-1!d93`?Ki>&CWu@ zg4KgG{1AFN7ArEHcM||k@bIb9ZPMs2Tfy=Jb{4|WZvo6a{8}4oMYKQ5TMrs`X|I(` z47;IUT5(j8dZhY22MsatbvH=!m@wfL#Y{lg_Il8M`*)ppL?g2X(1C0w#^->2@tTH~ zHiwU}NNCXG>2PIjetB*?lVM~e(NaTK*o&AdIVKXokgs-!rq8t>Pa(zks-nGCrYbmv z^@ou4Xl}wwX|;~7u+4^0teyz~z&(MmpZyh>m6sdeH}ji8f4w?-x8O!bzQGCTSL6nF zb#uWNwiqgqH2Vr?Ur6Hn%nuJEcn^|ahmWQzNt3kmR&C$Xfh1ahF6L@S-+bUo_hbpz z-nxjTp1ix^vwlOB$^F((xv3~HZ#*gOcPVb#im+O>7X#Ys1B~ddCq?f96JT0XhAh_0 z_JdIClyhdu!iBT7Livb$=(gZj{7>Ww0NmyPyxd~gsy$$Dr zzmUqm8zfLx8Yq`DD|d6L)z`4r#C@(w*S(Mi#@-f6Qqr@ibw;|I!Oj_`(XLIb%Kc4R zr<-@n{`iArPS^ z$F*`Y4tWT%!3}O0Tc|L9jM&9c$rFNpTq06D*IFGX!r0&b$zHhE^~VQJl%(j6`O?ly z0m3oSP#Ta%)8MZP3J>r2VC01??~V24X7P$abavSqUcQ63ctf6t(2!y<_7hXQu}6xE zmG912`<{#+P1S9<_IE<96LVbJogM7k3}T2D;XC4l_{K2-9X_E4qSKwBnP|VQhf4Vh z_JpX>%bfncVy;j2yzjd;TT`%zZadw+SIb%n;2;zBn9E0BE){)wzu3#mNpi*{aZz-? z2u|3%X}20ps6zX3xjBDEYqeJ>naDi3s6RiWwSd)O8b*k9kTCKXWU2V?aWeqOpte1lLb<$AZw>m8Z zj71lUcTmENv(V%~5F1r+&`Ll?qphLB!LDty@lL2&k}QuZ-$F0P?97)HML%jxM;QB# zFw)7k#*5n568q%sy(|kTyvDd@^+G4D-Qc5o2N8g8fdCNNOI!4eco7THNg`(>zqCtX zAtHdpy;E_{yF1%kPUtI5UtRr^S<$=E_`HXBRVxxKA?Mo$iZ@T@?Y-C{B81JVbwHR? zE-YUldW4XIupdbaEqhtpUTc;KdTogWVjt&avUvw+fzE*3Wl8{;yE`z*hnqt;Jcx_j zfECCnWMEb@s@9 zdNi5_?sR)Edoa{)1~*XN~NDh3Q^{!ii(Q%gx0T-_Cy4g zA>#Q5DW16tLOLtn56b1vEnYQ~2YumC$UAW9-ZSvsSTQUJObiM3%G{R}4vZ&mRV-CZ z7;H(qhbm0PJsNC^AXz$lu+=uyPD(_w2<6hHE;@l`rHIyA>sr3hip?egblfArD|&Gz zr#G5Hfw>%My{n`%>Vfdmhsh})M&&fTLbSEnW!{-j7Il7V%htrNIRB2KkEZYVPt1fI ztM!cKoyHel5D0Pmh3h&^PbvJ3orTH(PP9&XahClWbn^1>4Gj z%W3KY)C8NIrX9fumpaUepdWp0$axrC>%oib?tQ$U1+plRFjpt?-a#U79S(N4pme66qZNa!YtrdaF@GeF?B?k1@~(~* zYQSPP)oR+**Z+WD#hr&K{)k;Y8WQUM)`M4}Jz4b}c`%@3P-B7;>OL^AcD zm=xoe3XDa!ZO zobRs>1zpyM54_HQ5tLw-Hh=2@&kg>X*Wa_@Q@|9Dw)Q~+#au+1x8OUklCS1^+59{P z=Lk)k#uJ_-&ytgNwX!$TBQAZ8PCUA@Gvzp>o-7o|!o7~n#;}D+*?jQ#G?MHD@{(^` zbg=fUjJlxGJHc<$HbYWeZ!+v$$mZU8%(pP{wTMj^pa1ykIq~wP60e_oK3(Qq&27^S z*9UsuXU(sS{3(|C7FgTLjKy>g+uvAte6lU&8$d3{N*)MdH+S{ZNgmLEN1$!7J4$qs zy2cb(J-%@#0@Os-l01hq8hB_W3zl?nDRq&Z>OSJpXR#&uYp|2*p7?oYV0}ym4e_(L zgOp+K4On4`rmC4c*e2j{@Wf@#+kHdb-TjxW=pej%S41h~)On{zGq#)a-utRGzs(Rl zSIMK$fN#Y~GcNd2>+!Ax$E(bzPd_HEH06z!ex1I{KUOOoKk}YCQ%fL8<#W+S!Jlek zyt)I)#2d}9h=#)P$mxU&Zt>Uclv=P)0N8K&^tF~);_zRL=h>r!jt%-9iJ#xv)p}68 z_J>d$aN@gyISlBL&V??CZ8ou)+>ZQEf(c`WJ-mtGiGn0u4O>q{(t-@!(;Ipt=wzmn$VtwBuu`7T+nk~FFCM#9T9Uac z{Yqy*J2LOzY81Y}C8rkKVI>|Y{u()Gy54HK)FRd$2Imx3Jx;oS;5kBM4#GN~@Z%E> zKoQ$R2y@$#aT&?vc!U9H~?I}b?Nb}pUKMVt8oFuB7LLx z^5HR6x9?Q9YopFZ!z!;_btuEd{T^h%V3QDGHSRH?ZA5ARlG&FX+8Z+}xicik41LP` z+c;oT8zRj86#I5{{yp)zn?PmyvH>4r&Ub-jEZIo%4IQBBvGSj4=l>zpY#0aHq3hwU zyogu9xF8fnxZfoB3Hzz|wSSQ2yTb@n@lM@dAtD4Fa%0vt2;zJje3##*dURXfo{O&x8=K-q|md z_NDY_wUz=E0z#hV81)A+Vfuw$aa8yd$5gw)FO>-q+>SZA<&__N8TZmVU5k&WEDo{e zj28(yBa?5nE}BMlZtbDQZyZD>GoX>zU2dPvRbM=da|i#QLjwQ`Jqurb5*{!%0KM)Z zs$!8Z8AHg#P%kn`&fPnG=ynxtdRG3((=wipF1oRiXpw0^3J~disP3jiMCHyw7Z<_2 z>v!A@U*vMO8 zJdE9kN8LtR-^bIlc<+ZK68W~W0)r-&+GsxIB8|h>p)=Y>+K;VOCy(SDdlOj2`zQS7 zf6bJ&F_E88h%|hPTZxe%@nk0CaZ4W+R>P;pp<75vqjJxN6|cS%XFl(A4)Q*AMikjG zriubVhApSOlxM7lcLKmvC&G-ipIsH`4-`A?JJNvl0(D>HRKI_vvB+X2$HeAxtx+Wu zVp?xcT^@Y^mDmh?Xn3VUCqb7sjV+PbKyP2aE8uHl(vfJ&#r&g4*g@FMby3u|O z0t@D%xNBofRBy&9(GI~iCgHV=_uDOlgmH9Fs}=VVn4SAm++80x$ zr@a`LM#^(`EVSDNio6V16Ywx95!rD)|?~LvSyNu=luA;2{ZbAywPMPMhDNT}$R} zqiqS=8L}F*?)Km#wc=+7=)2uUUYq!XWG5X`!FO*L7sY^gq*%^jHGTPh!K3f)4<^hB zA^8{DR=-l&fHOqg+u(0;w$dK~>{_%o))<%xjXxyiwqo3JslKIVwZJ!10Y~ zjwNmN{P5duDT6lZJox!*@5va)Hq2T7kxyR2*4u!$bJ2(RZ+^Wcg#K|~#JJ?#S{yRy ztY+pYrZ-ff!e7+9QZxwKGa8Ga#ti8EXYUSqKy_$E)Q0S5-I2-?{15xU>Qv5#b|?EJ3@oOQzmp4VuxT9oQ+&9!tDs z*jdeTeWqqln-J#6J@5M5yTtz2Go&p3Wfuiue1E8LA!K9gE9jdrK2FX{A#iRwykDGq zIot(&+E=;Hi3wDl>nC=5of5eJP^9-0avt|y+Da-iSF(Jr4lj*$+5jr+`V4enrM)Kh zk?M*FHGF_zSZd(ru9DrdjacDLW<7UIc%gFxRdC{O z29=SSuk#@Sb$`q?j0}g)TKpbKxmM>G<*S4XZp^$%vaF@Oq{$h>-w&nq5l~D}n7hZ+ zmuz9wRN{Ri)h>A=I@n#$Rx&S+i%AJ4q_0I`%+IgfQn)YgJJuf8=pYq&mW1`F20xD+=+!j=jSlqf{SHuBd z!R`;w1(63s#-8TuIhJl%)Q2l)MsjCLCV}|sh#b2}z8{S5WY0v2K+QRy1+7KHh4PQj z?T@z_vS(axB}QG&Ke6XTnEfC!vAb>DvA%fhQqgfaFw-`xd$9#s-9WV1bU>yqyk`y| zb)anzzNI!emrpqdtyV*ObWWLPdf~Y1FU?&3*7-0WR3Ay0En*$hxtgJNy-d&dRm!uC zh65!F#kXWPKvNfwD042thho|jNpF4MWkasiJHuC|xAv=D<^}|}r!1^E6Zh1g{x7IF`m}ASDD!u84#wtyFX;Irvu^gx*=YGfVJIg(`8rmA5 z;suvQ=HOnA;}rHEL#HFok-C-POZ`L}Oc((eX1816x_+dZox}#Uitr9t(PY1E|J}1< zM=~G$J54Aq>j!@QlbY6^VKH*4)|1$sk-~tdAd=%C2@WPS)>2xr!KHrhOQ)K;UI(`3 z`$Uw7*5u!?^1KzEI;|S!IUl0Bt>rFku2rLM08u^{+sx2~e7cu7?H3%rKCF_672CZ% z|AhFwl?l6M@nQNg)gI4=i3l`)1J?4m1HRsh)q8@Ji$ntrQntU1A_XxR52fv$i`m_; z3A4JpJ15{#MX~drw`V^zEljBj=d^97?~=knifq51HoUxaIufVm4P$F1;w|%}-MO?C zY*Qd5pfg;K4-(_w@ew>|oONYJFJ;FEX}G7$r229AwBH54Gj?l*nk>h2!hv~CyvQ!S zT}jU5L(wpc1gvRG+`?Usw>PIGO`#4E*Uiak_F9rdYVn+@ z7G+k#?Z8{_RCWUtK_3uzw%ddA;~iE(KePg?64tK;a)}om|N6yx={jvRrWzwb${4NHfbT<)x5SdN zCrZWZL?klFAc~`YgfXrJ;0N6!!6=oIJtM2-gH;E8_2s#gL+Y&dZWe}lFXmQ*t}z;` zWHG~ixJG9cPb=5DAl2p}7+h>djqW?tKQLM$y(x~baxP~ogzI{hj5yU>K*_(`6X1C~ z+85>bdpYU4e^1ddqdg4YVm|SUK(M*07=a_0X0v4no2M0O(rn^p(J4X!%brMVQ7WlO}2r@g>s` zF!>G1f<^>c{H$hG$Y=HLIj}Ti=G94l-Q#am#+$x>AnksU4&%MOfF&e=37>vBtQXuZ zGrDn5{m9v4P6V5?qWEL`5kb4eY3U1{ymg7>kS7zNg^U)yhXr|phm4&e_{qMHUTB!R zQZRJxJN}i|vklWh%@@1cCbZ$&2X5)pGH2Cx9o~^Yxg`17Ioek3M{kY=2i(UhtXqHV z4Ebd@8H^I#RmSnTwQfzWS`TAY=Z3nC*2ev`8V?$BBhU0FF^h101+HIlwLvMg8%#um zJb1C63^Jiz7-v~2IzvB<*frLlOPIE=p9C4%d3JRFbQ&T%$B(Duf+OI%sqI7r4syWn^>lue%XyL9p?MOk$tzN)&3(&y7Sc5 z;IK6KaOl}BpOv;7c+E+|e1MqK+(gLJi%kJ#u2~A~xNQYb8afC)b)rbzidi-m+%5k0 zHrYAJQ8wBYU#u@KJ;j&M$1`^xA&Q(;*z{sM^G9+$B?|BT<|pL&yms@>`O_v43A5-} zFouc)RrPp_ReJWRy1AIoWU$pWw0>j~;`tupJ6rM0@yn-ulCyjIzUnne^#k42ni3cB z)cF$ae`?x3oK=anN8ML(Wxo9OfQabIW!m@`1nK$3e?!U=KRy*waSJh~)EczgW<1^V zE(C4*-2V1zG6cnthafpVA?sgP@qgYUaVkXL%Q>wxSK?wnVV^Bo6#qQgQSI4IF=6(~ zL@H~Bi;4Sk&wO96rsMU6MOBIcnm(nRmOa~FeRkE9jj0E1aR*%sE<^4Xg^hb@o-Y=A zs15AiVT$uBTWBhJPr2KeP?YZ{y$1V5UD$Y|oow|(aZhJ|!UfyPc<>gA*J-GEN6gZ* z#`p=-v_$wPob%Z(kG_pHo$AZkMOhUb&`t3@LC^d){Q598E|(ryEaJZ+FvJv_(cp9 z5ljmnm4G8zwuUJKVm+cDnj*Ft4Y$N|&8WHGcxV18*GLVz1odwXraFJ|%@KP&^XA7c zRG4uuVi)&0j_b`9D%{6O^w{-=9};?fN@X&J&{yTgRDGdTBJbb(ox;=D0!dx`h5icx z{$jsOg5Nd!RZLz;BAEB%Z~o_vk8d)IZ;q;P#)B#$%uv47yFwGM>tO~ zUy(o^%$y$m|Ga2Z2T*?oj|$H3qOp?hy&gZ7_usB^dqKf=A}8fBm+W`=D;!mrIUeLY zuV_yU-7!l{em(S_qyZF|a*`c)c#nG2u8*rI=lVi5K78*eUo|rT;3q>L8`n%rWe|2_qL=?)2F&%t@Ew3v_jxFtYn-2-Q*{a z3P^99w`$W$VGVQbgkYI<^KjA$b}H|S^1<&hP@DoM?lb+d`4hN4NiPwx(*MyV69=au z&WX`6$H>?9rF)dAYiR(fb$UF?Dx_s3^8r>d!B5exeU9!`6Y}~5`&tM#?1$D4)LD8* zd;zk1E*(hAS^;adKJX!#keVtA-rT|2Wo2+OX zReQ00aUMGjFQvc=(Bp{`hX+kzL8lgvVtt{Nu4#ujr`T|!V~|Q+xZ?2a?Pm6V$n<`# zz-i9FHT1luVwZ?Hz5;$p1qaO{Y(mQzOXV*9gm8rXk_g(9F6O1em1v$P{b;ez9`eA- zRcahr5~)z^|A3kxrDWCnVN$dy6MT{dT!W&g6Okf!W_>VL*MAEtVc ztPP!%jM;NNlL)kX>Noh0apU@O`&uYh@;F4SR1KQUjyj1b(6|HapL2g3Dj zZI6;@K}19g5~4(n-bq5V=z<`KXwf5j86^?D*N`xxOLWo8Nc1{dbVl!tHU`7YcP77k z?|a|(kI#QNXP33tUVE))J$oHT*}^$XVZ@WrXFPT?yQa`h(wxxUm zx9c7frd?%B9H3QZRt^T2J=nFW&-_je@vHk&7$eroLIEtO$}ef&p~8l0wX6J2#W+UP&DzW@}Lx;%X9O#xXhZl8E7}1B}6%{q1Tw$7Wbg z+UyT<-aLlWkN!h%T?)1NRrXt?1qM|PkFfBgN_uZNQ3-5I9>B>YVn2F948uZmF5_&Z z#i%`dh^|L%TX8Llx2m+EvkT4qez09S3wz}^&=W#E&FABfN}%J}mLG$Z+t?OOpjiRb zUsp!6R2XKu(+BL@eSE@Zc^5bG0!BLyd+Fe$?)InRUEquxYg0LsZDyUGr)fXEYmPySD|o_Rd}A8+e7&fK7@*jih+fxlxSfK|MjpD%jN4Eh96-1Y zo*lK&)kTKs+uO+NP9LSt&Ux4TTHs``XCL`C;ZGZ3Iq;IGx~J$$?XHXIwsKe|_0FrX zv!V(Ly055;Agr^c-h&zR=4~$#xp!&CW|GdH-{dzw1#luY7oFMVS7+ScWdRU^+k|Sv z!wMrgX{gTd`9i{_tiRzCW=Qc&>Q8x4XXfl68(5TEhhGeBqALj zPl!3ZP@&jdl@2>$NOhGklj2VML|y>N?DR=Dc$23}@V>9CKt#Z9c*>Yka3<1Av6s-P zVesh=C#iQFA%Gtc?Z``apI*HllIA^O$%Sp`Zs(Rpb1Vpx=|Y9}?@1WQR30sq=36&) z;gB!_#g7IY)~YV^Eq(M7_MulFo&KI;xwG2Op-0Dd;Gl3s%@d-%BMwCkejm3&wKenb%TmXWoYSo_uNX376smCzk4R zWdy5Diy`9o*gHbKiimJFR4SFFH7oP+rn=ht!64C}Gq?7@lj)UV4E zn2x+wN~eEES_w}V2JanX2AvuEy&qQr_7bs5)oWuY{(g7()0I4IC~I0aw13yU#Q!;y zE~Sa2Gi2UvvnlV^Nk4e_4^g#cSYG?-IV4+PFTp5a8V>znD}K90fO+>-wE^PQ09xoY zebJ~y&0+S}vR||P!EW5%Ox%ZG(X>DGmZ}3QJw=%>yU@`euT3tEIGy@+P>jW0SiI<(pAWM{AL0gwFtw=X3yCj*V>2t(+F?>Jb-?b+gBiHPaF^+;99N^Y7K<;R)h^_<-sd-4o1PRnzA7O;R0cv)>3EAD+77Xg5^ z+XH3QFwE&e(S|2D3!!vWH|zU(B7wQW?!*!rbgdYU+KbU=oB$u7TE+z!jr#4H4@U<_ zR5sK8A_|dFVq#_yijB~a-)k08PuRecR`LS2H zI5QnWPFU-umik2kX1x3|VDCpFd64h)5cJp^mkpU*{@_=Z9k=qvU+%uISH7ocG+7+x zD_(HmlZC*<1sgh);mpHDG`q9O0y_9pp;$eBFQrWrX!bB_vvWA(`!#`E^m|E2hZf9mSQF0m;c4tE?ye*RD@X zmjSE^YljNA#66f%EW^oj_rTT@4i^sak45Y4kM~-W9Vdp|nA}ebadG+Itmf0Uy7)RS zC&FcvPF1=kkSP z?@b;~r|K2uT1VwUnj+k7BK52_nyW{EE&>yI^zmCq3vA6}v#x+mwbIys zzONa1B@Pf;=J8vnyZrd4t1b>Yfec3!2Lr;3?Cc<4k3#W6>B&kA)Gh4gV04df_ziQ4 zom^K1z{GXZ$@N$aJ3gr<0b-DO`~};zz_k-$j(pEq4Z6uouKTQWP2PP^W83fWHUIvLf%3 zg2}o=tcEa8v+_OloqO`E`Vr8lZE>{+yF8~peTMz&?xi!=b#*8@`c`v(fk61oGasa7 z)jlyYh_BEZjJ{i!_~Q*BWpH^`WxZ+6zF4=Hp__FdY~EjC#TG^cWGg5x#qzxGfIG}; zABNF1cC6P%{K5qyfG;Y8i@sUY;hE2iA%n+q8h4*8k0wUOdOvI=s~Qa1i)d-B<6cYy zP=Tunao^Og$|;uzdoN(^t2B~B>U_D(3B2~YFKfXZ`)2tq)T;7N+F)zp{fV++X>Z?k zXAU$Vk{j-72qxV`$Q2!#;lcSgT#9WGd)rcnyOtYn{gNLg<$9i^q6<^^lVJud;KyxoEh`xRw)+8gF&R{(sPA8A0ES_#+l@NE0T>zI zGWt`kTc1A@o>n1UI}r$n5t2==oC}o32@9mBosK^Ui8>j#iiy&rbv@?Lo};ps7=xDt zp^UI5y*3hGtO1z;=`0CwR9iH1(7E)k%}2S-hT8&$&bNs$;}?}=PtHr+Rb(BJ^_)+z z{=fQ!PN`{@hsgDw|L#9+pu97fo{j9`%>{Dh{lA3gofXbkWzmD#vFK{~=dWxfvX8sB zmm25c8c(UuTsi%E3P$3tSepg2RPJ1CG-e;}U%M{m+^jI+-SWooWqx7NVJ+)o+J`j# z9~6qe)5gq;DUU_yNu1+00BVqFo{N}k=EGN)F0+TV4;Wi^mD|Q1b{*j165;S=B=7Yr z5n4S)$sCVi#eLVIO0@}PpBu4W?8p(e{rA_2YpKECgsB zJY{3v)WSGp*ER>X!S4o4dMV8(fooB)fa9BU#~#0AwM?d+R{hJb&`EFJi=}h-K%4rRj}!g$ z&*gBpcu!R@cQ#FT+3Yxb8OhsbfETgE15}2%lhr33h`7(~!I*rpPmIb5^}}#HI6klB zn`VA_m*SX2i%k^=)USP?TI zZ$C`3>eJ(N9fi3ooxzZDan{UiU@y^E2+350%8R+0D;f#K4S!>U2uj<~6>JV4g%Vs- z;MgoL`afCv8;o!^FU+Uv%%;^XSTtl@cWXNc^Q~`;*{GKlg~}7lI;b?&iVpjY|G!i8 zp;)d&(*a*y*9PX*bDpiwz5H?z0>SV7qb|8CcEmYbZz~odB)d^uhaLVHS7f|0XqLv+At$ zZy)XHQ%`V@dJzQkacCnEgCS>uDUOn7;SoMGGs2$D4|`J!Dp`5a%-JT#~3y38#d9}PLyq^R~=-u!oX#!39W3?-#^eX-7A`^*=h*X z(ZE3`|D{LkC9fu_;l&k`)3%V7iL2#9$AidmejOD3ab?IogGxU-k~8UINW;Y1R^2Z+ z%QS(2B>wqCnXL=}72F&6!{{&Z9O?{#XVg#Z7>nVtL|)mcA^<*kvk&rINCFjcl1k0v zNW=6Tz-n&cQR1jpQl0T11&*`CbeIZCRlb(1T?HKVt2oL}!PgWUtMG!g+Ysz!TL1~G zuTHOR$HzteEsS<`jD|=G4Yl!c`4Alakor(z#Kqr0-d$>=|HLEZS7NhIKD6>zeDg&~ zH+^L@%%C9L`+qOyzN_W~`MWdFq5*J2vw#ar(#147P$N&f%_ZPl5JW_QU9}K$wWc!kZ_Oc>hD(!SQfVH@kskgyDN#iLny15wKAU zY?x9RAM<_jWr|9hi+@58MEvmbUMV6M6&yLwag=4W;N8{ypeE}Ato=Ig*Wq<`6E({J zjf~B;J3bHiENZG$Mlt(4{{4QNhHV9c`uEX<;(PODZ?{CB9IP5%`SN_)uQa!#=-Dz;sW!$;l~%KA8BLol$-@d<&7BALqz zHgkPQkz4cZy?0waRri89hQX9xyU)O5a^^8|5YnMzGd%gLSTtjeUwz7j?9TlwPjkwV zpl1*(^b|PWQhZ|}jrE)SSGHDLYE*#bBz<=SobtLl2*3-804$Q^CQvihJ<{(k*9KVy8b9J3TrDK-zwN&=lXe{g^4rG9*7IInjns&G9BEL@J>v zTCsu54JCo?-p)g~L6eQ+ko>18K6vb!bX=eFjNGKrxhPMau&Q4r(f?Uu3z54s&+h%y zf4%Nw$0F+5kagTbI#v3(NB%Q06FE?eQ&1P?#(Aq0%#rNk^x7i(6I!a`_;&s~n^smA z;eF%l0tOVt61(jjLo6SDeJWLHM%TDZp_s)4QnW_+$T&TnS#G>6uzv%19RFfC0mK>{ z{*TKd0h}HWlBVk*I`tbYz@NA%cVz*p7UNS{LY&?EaW;y!q5jMIBKF><24GrzfNpdH z-WK%d6YlTd(bt-1K2?ohq;gH$y1pQ}at!AP3KDqTv#m;(ZaplwAv(x1<&R^Og*g>82DOrCAJ02e8ln7Ac?BDNLM*QD*faIZ^PFg(&&LjX_ z#;0w1p#T@lkpX!V^yo94+}~OLkK*EHZSfzw9 zY*P1pRa(+*05&9xF~Ekz2W0hkNj6`uhnJD-!t$>H?82}~_KQl~0s7bDl1c@En!SJ$ zM1P_GigJMVYJ?lmK}@uPUVcXC2UzHC_4|YAAh4Kb0$>8r0HYsZm;)A@aHKhhVC7dg#(-hCHPjT<^0CZZQY{|beM?eW8 zprvT)f06uvIxPsGq|U?ZAe?6ZtDpb%Uw}#eFaP0xMuA&cU|-`%<|`pMA|+sJL7)Yo zu+~2rnVa&oV)C(0olsVB+S%VV*lNYeEKtCO+R2+H%DT})0vu{`T)%X z(g+Y5v-t{eK|sT~OvNVBr^IW(7=bl7>y1u#Z3O(p$8nx{S$k9LeJctBQ^;d40VP7``KtW;srp2i>mne+Ac(Wp9fEI?5DYlWgP(6H)mdV+B$ZkF!*m?=T=|FRxInr{}O$0c!n=DB&f zbbrnM?>7I2w^Bg=&PUGNTG!39Sz4;;pY3-O*naL+hcXUhhpBts z4{k+_mx+vBpB?9P(Ug{MunY|XMxmWn;Q$(5x$$Bw>8CiyWflG7TM>#EZ;jt3nI26c z>NwW7c1K69{d=?tC8}uDZvvn|4{(_mY`(dO1}d(|F&MwwC4kB&)X_M(N`)V7-5YiB zi3-i8vT*)xXR=UR`w6jLecg_$r#yfu_Q%$*L`>i zFqgq-`r0GyUv9Md-a-EGqwhUtP|s=D~`A3kx+Yfa4ZI{1grT;k9pT#Uq6+xx&mGU6EtyG?WeG(Y z1L_t~>BRJRyoY0xMG!EBh8J?)@@^Uv!mzsZn@)pg{s-@|H2$8B_zn*kw6%2tgM+XW zr#j9ng5b_o-Gk|GV6i*XEx>K2A#>F*ic{Z>q3C(@9;b1ZuSpBUve7|AL9tKBPgCei z6@aoojU8pCDq>^Pd>^vMt~EyvT%*or8}a(c;jJ4$s&;BB(<5Uy`G|9iz4e08yi3N} zNB=>Xa0G#;2ETS63s>CI^>cTh8ZO22b7snyw{- zUHY(*@ZI4;y(Lm5%5P=l0KKrftVP90`!GJb@B8)UqqtxYee!_O zYzbJNcRFj~lS+N_=OD zTW}4PS}Wh_Sg;*3us~tkf~VV~JJ~nz__Qw7v{~KXcr@vjrEU{y*0xUINS|y z%iE;hSB{f%EF?osPUkbS1W?}d>gAP0$qX{cRa8yPyTwhRL$t0)&mt<(p1|aEo(s6< zw`g0*pU##g1gI)gsPsuBiiB+uitz2?LpU!t$tQ40JQAeg56g-gTHDCdAO_R0iYEjG zrd}Aedq=pg5v)nyym7na%j2(!j$KD0MPK$^)~5+a-rUPLx*R?sBDnLA^Nlb3i-Ggf z;x8cW;JB;>pa<7{${(1J+&5q;a|sH6k{+WK1vEYm{wO6`_V9Ojhg5M zAgj1Eb-IvV!{?W^&0{0*zHcOXx({0WBUXj46oP}aKB>+-z$0Ij!dcRQ{K6F(%$jkeMKrtcdBaHc`YL3 z<3HS<8BG>(q^W!GO*H0h%j{z6gfwZSc1v#ds~1Vv&vWj;v%_moGHSnO{QMad z>e;k*GW^O}p}byZ-6&l&3fU^$4ohD;PGH^}l->cukt^wJK8HVQ_)I&$GOz5-M`HvF8?uOXqr~(w6tNNR%4YK@0bXr>PVn z<-*fczIALyN{V%FvQh-6M=ql79F%f+?FG)+y!i(EjuiKnTV|G)f-|3$2r9iZ_z)Dw zgS9Uw=x~0?xwb#FUeQzm+wfUShJczdQ2FI~&k!hQ!SU;g z--OLf?9SmTZiSpeO~q;OLg8D z>oGcy=sEjOQPzM5_u@tiN7IwY%=gzo?-G96TaG&Gde;0NGa14+Q$pA3FXlB>FaMw`O;>D8uRtNJU=BKVML+O}DH z_ol-)UmLqgT?^I!X%eWxq|oLijrbyGlhu)F>9RHA?fiC{QrbFjerIk<7xEJh)o%gr z!`n2OnUB2_KSAhIFOaYR8lnCB#SUf2JeL{+heF}2Zv-I4Kk}0ro?`LDrN5rPz$P~Z z(PromysWH#or5w=c%S0YyCMkNnXq*%@}~Jk&Ctt2HYZJY^HkPXgV1wD@F=f80;Z(s z?>H0s;;qvr6wJ}=)&4ocOc;@!i<0 z|9D|uqCy8U!Zh+tWX)szm7%$@giY3Mz~5=%)Sx!W zd$M)UIZ#C@Ov(d>NzXvq4@9|VtQGAhf0$coxU{nm(*6zT%yY~LKoT@o8r#~B2M}L& zL!Xjy?^=@vKn!B(LQTz*G@7jQDy8lh~c#mld~O3$lKbtk@0Xy>yo3OoXTG8PR8m+#az4G}=D zJntX@E#@SM9tu5$2!@TlA&PROvq-TeyC9HMFni^j#8x9!6QJ_tfWr*%xd~+!pN~5L z(Zl6q{DYPyKOh;9P4SBo2pi6V34vf7XcoAF1VmRXPM}ut4kEx07g!4W_wx;o@RgHb z(X4|2NvBTd#YTGyRDf%59ohM3YJ<~|crfe4iOEF>l$yaWYdPDU-6)Na{{?qRROyE|l6r ztO$|xNSbIOBnW!bF+DHN2V1ltU*0P69rLhtIm>srSgx`-!1ft58C;DTpa-v$$G}T4 zvbU(?1j%te8R&8N0RahsA7m5dA(Nklw>CL$GbBt;$QsYTONp=D!7ZcF{^lBc|Y(``6t4LMd9^!e{4N64o;S%*))Y1YTH+1 zU-xfZ#kTd{PA@QS@!~y5S9fALSlAFQM%m&sk+z#n{p2hv+lDIyZ%}O7YFICZb^AE~ zemKU@N0esc=6+jhXfiPaGJoMBbgxAC!X<^ec941|Ml;>BtK)uTDt>>fDC^hZjq`Dh zt0s8M*5W^Zn&s8sKN+0_VKkf4PuhEY1a`{pX|7;9h@uJ* z+e~rcf(d0oN}hGFE%{T@n1n89? zO-eHndg`@ZNBA_&=xG0F6<+uFo%hv&Z+&wh=;@~0Wv$6rMK0#eWScp1bO(+oa5a>^ z^VCra{^=$T004y$%g2M#RNBNd4!EjaLt<%Elek~KdgAaYk|~{K1ZvV^8=;-5)8(P% zzxz2icgc;zD^J$#!)1$e6)^;jo=d+j?Q5GAWB)m8Xv^BA516q&bz&yKPIYcc2pj0CVds?>r=YErvX#GLl986N3xn!5-jcu|7v(D ztr_-d90vZdh7!d|`G2d74@;hII+bHYnHQ=cB&e&Sj1dJ$45l{2*eU|%x| zxMH+(fu(A3seF2^dGO=u<8!So+f#|Q`$$)cwNaw=dv4c1DMN%y?R8pmcN)iwcO)|H&aqsUXT}iu$&vS~-(- z^Af;V@EP<8>9-?2v!IrZnUB5W zg`N296Mmok%4#*SyEIwOA3v?eAlp4ZIU)Zp;7dAp=f{UruOb6j=3-gBG+NI-4Lo}L zs9|T#3-AY0Jw@&>+Cl!=LqS2c#0U3uc zTPKy?R4=}#Pw1Y6vFMV{DR5s-qQ^lK6LZs_E1AWonvDU$b>TeCp%ay3}f*ZvQ&oY z?!WSD_OD^0H`H&|@u1Bk)wWD@p;2|MAG)X@{B{xjoRks zib6U^MfZX!dh(A89@2hqmI?7&4*@q%y4jhqFTeymh|Kj6U@`u=%Sqs~E~j_YL)Te0dfn60NX3*|c0n+GJ>pC6&LgWEol3$na{Rirvsl7R=#XE3~U_`Hd_O1h~ z|3ZQ5v?44TSBeF|bO2lNvW-mWX~3A(oO`}ag->aVsI zSv(z9yL7`}=7G^Nsy~K}^(-WMn5Q&2jr?y3+dXX+Y1{{qb{M z>I1Ps;fzU#Z4+|AR&l%EV}qqo^6#%}zi`ERGu=a8ME{^-BAslhVw|Y@BCQlKRR!_t z!-SG(^|}a8)AQ6S{E%8sXPrnB*?p=Wa0dUD^<2Af5V^vCJ1c;v4fs^JbnoAHfnV_`&u$@c-(3JC=kIdH3|y!v zQaDfmf$G)A@4hrH#k2_tiWSSpuN2hBsOvol|NnO&;=Uo*z;r4{e_ZrwByUt$Au5Qyzg1@-RVJZ4266Szjkd3w-JiM8WSF^=)8n=gGLXu~I8TAD9NQ`h0Old=S@if^1W1{k&H{-njx|$7lXWxcO1h(X+)HrbH7i zRk=&77pX6Z_cIl>4fOS|`ZK*29^zk=GkY%vyT~2&utl*ZLi~?eGFGkZi;?U96 z6^H;-ll)czVlB-aqyObpEAV1$MNCWXVk8bzwwf}fb=jeH2^sa#HZ-hzkV#-z_Uar_ zI4=156EqxSxA!?A^jUdt>}k|g`T7q9%d|6q8*{*Zc@qTEq@*(D;~ud7enC#N#zRzV zR8sFZ*|dPfh7;4`@#hY)U=G5t^))pfnuOL~UZ}KJOIO=J$^(=-io(ZqsrvG{7h%7} z#Kq9$O&qgI`40l7fPD7sX5HaVFvt#tMF;)rm=r zyxN(;dw?eFNe(DDHZ-uTiwork7TsKmx@-k-pi|<)*`{SmO&KS0iNri!M-_y7EDOEy zL(O}+w4z!NS3f8N4n97QlgFG*K3j~Kq8{PQ{guzrL3Lm&aQW=SyvcL#cvO%Lb6_cd zY6A}6&^D9e7xsGz%FO3_^bZl++ZR>g@1@zV zsv`6VFSlk)Tw4Bc2AqsO7igJ+!?B-iAz)AAQ}UbN%uP+Nyq>9Vz=gtc&qjk1WND;+ zx^b)0Xy0;okklCccjq6v9x5CH_==oRBLA20UJ)<)r9_>R6n|DzrN@+tqj2m2CHCM; zob2h8xD~mNxjCHHdi6vo5-6{{6}#F0p47?0+3Qz5GAhJw-&{4mz*(hzUi zObKCq9_HB0Id*If3nkR=Zb%rZdYOGlPVC*5Hbf}xCZvJun4)I=U1O8Fa!P^{y=$}e zve!#23XOmUnoeZM9}fsaIrTEkeb!PEkVvG~JqoIWls8dPQ6GgO1@O&nBFF#yxk~@a zrQMd{pPnJ$Cl#IEZp2-+pQVEyD>OUyi|GQmWw=G&jgaN#Apq5bzQ0g8DXNIq`Qbz@ zoEWgyml6{i8Y*t(uB$X+Juk`1gcr9LsBV+gv)RInjgLVw`yc$mdQO^pT`4eI2t3sj z`;)YFCiDhUU@S%TC!>5;q!nklbUP<2K_D6p<7>a;Nkc`cg);)~$zjwK!yRvM=*`2u zS7IE(MF?d)n~|uhcFZwM9xGjSiWyO6xk~aww4egJ)O*uH6C4jDoasvCDe^_6LKdI8 zFKKD3TASHde+7ZKLm6mxs%Fv65p?>rc&_&zwDPfSv+`PvNrXt@GfT51joq?Jj~K8z z?@^XJBHD7l5Xe6sP>h@CFNja#-exRbXMLEDOz#wHWgnt(QG!Kl2|nN{hJHMJxaDJz|##Eajq z4#eIiqq~FR@XI{)&NR&$ma7pK(?s%nY)U&`KR-GvO7BYaW8!sZ?l(+zd2uz*8wE~L zUx?&T9ee`nYgow9p9O^Y1NA`ftL^1cf`Y0Zi+xVY{jpjkmTuFuBfd)=(NU%zoaV7D ztEJ0lKAA6$>FhxGYxL7*H}yG}m&vM=A1&?!8IVqY>c z?pU`piX2hN0Jl%xM~^=z%G?b#H!*4A_@bV%spD(cBW{`~RjoB9_iTlczafaof9t2w z=-E%)M~;jVjRpJ44wS_*n)YgEF)K@9ESUa6!2UW24W=?J3X}&y($mi~&CgTJ&n@9{ zs0P?z%jHo&w63D!;0NtbMs(DzIOtgv^h~`Hg&C8-qQ?0D7DbLGaCS>B0JaTILVRJu z?Pz_Q-}IZu+tM)~*Ad3)`}H?X&!}IhrKjo6lY-uT>$yD{Z7@1o-C*0ib$|%OqOr>j z8(HYSALYPj+d@e37bvN1p_l#8%PtgP&D`d99xwAEFzxb}?dAgOt#IrWoCTW>IZqEP zywrl_eJqDUv1EUD{C?w3S)ka`LzMWy()jc?>3734QE=%$Nri@Dj8r#M<&% zXwYrw?x|G(^vTtbSL?An9O&sPCc2`Ze#cQHEC@#1MccSRA$1@O%p!a93dSPJ!i=YiWq{SL5?areEPHh8EO9lLw$jr}oZRMM|mMHu^lqbgfi!R6n znhy%)k$Un-Y#$PG*$cT`7#F zz;n)z-~xXRJ?!DfWB68pyZecomds(tj@N?EhDp|E+WWld_dk<$t{h+=p*qX^^=2(Tt%sfnh!`QM^nw+$J$^?RIO>On5X&1Xhj zsCM3KC^bQkQp)|GC^iz?P{GQ=(;msA+~n(!fVz&suLf-8s#&q{g$2wR40{Q~W`?9_ z94QvDJmOn6`#nv*c`15ORXg9c(r=DAqFrkdV?Me|!gfeUM|W2HD#^VPjp>{P4zH7( zfg6qJ{iZtBj^+{pfi=tRE|FHYf2mqTeevODp$aHlH-1Zzp!F?ur>qI1FBX@m!#7cd zTYzUb%S%hIK8Q|VnTAv`S<~Y2Ev;qH^z6G?y)Y%L54SgLi)vuoxuscl)#Y;8#Zr%o zil;qvy4EDHxHs8_Yt7d|rpfet0%#k30z z27tH>b-Ngy(gq>SKC21xX&Rev7cDpV7N4EmX)J)M6uig(YoR+!-C!CDJR)__nV zSXSLGw$BT|PyJ~}Du*3nndieynH}086}OQ2Pm4X3;KC{(5SMt~QF65aG*Hve!viTV zgWjzkb5smV+2-)Xbh)#Rn^q#esQ-$iK6rCWX`;11*^fRUt)hw_gPb%E6x_b+oh~FV zEPO02WNo&*l%*FJBoLT1Cj8g?bmmUEU;TiFkZ5j_`%{ga%T(G~tFTg~cpf#-0-KW#mU(^ti#st4Qo?8nSr+-$rEc0M- z&-v)+F4objfuD)+KAwQXmOSP#4*Mk#({$H%LvvQS&2XX8y``lkX~FnB>edn21;!#^ z1_UB~qq{SxT7?hl)2x5amTbCO2~?(I9eSqT&(%D~!v=p}PVC|nTVfr@7)<-kCGKmI zj@R^_*^qA?OKW*~Hi) zYDA0JH((L}WY$p!lrHxQd`sTkp-HyGI}ZdLb$JP@mW^3IWt9b=#)omqtJ>t5k zX97oeNhir!ettgH5y_7ZwoM=&iI~jipHN!)O&{qnEd$dePw_b0RigyGQ&E!^&ktyR z0*JKuUQ8WcZugrW$^E-TVJVTlx7`9W>uZ3(Ebx9@g_>A#6yP5&1d7tPgdVAR{ z`9Elq)UZxP*R>Z2P6N$hF^B|+5PjfReAGK zU$`&>)P87vvo2b4ZrGV+gpmEI?zP)(K0I<*a1{ZLyFD^%nc*%^57k>yf2NfxvBv(TI4?ovmFcfHDPbkemd z{Gpr0#~hhbF5$d*LAsA85}edJ$45tDwaO1bAaxN{#V0R>wAko)ChbntA>9>MajHbS zVe*$t@?z7Xhqjdp@5_Z5L@g!WPzPlTOcKX8X14}b zM7)=xulD3+@vxLuWM6f|RyE2kxt`R59zPu6q3w(cQySl4;#^+u>r(yr=k~R{rtRlPF7nfWG?enZ(7SQ(-PeikQw=xC`7B2Em@J2iL>^>A5F#FnxJ7I)hP%=lgK%vg;aW{T4C+m12o4;f4qke+- zs`i{Y>QT_OKk@^l_(4@Gds#b$kbsLHBF=wX$N&Nn4bPud<4SEEt33r@801Nkl*Sc; z9=g>py(Fg3WZ{hBuX$NMcA;=OAOG;M$xOEW#yt>MARcxzHx5G*V}cWJ?aa&!^484* zxtR5L1jW|n341fOVvQRfw>22x;0%i0_vztPur-onq4O7q98K+U_TY+awk>HYWEk&{ z_@RD08Wae$Ca{PLv_|W3!hV%x`4o;U^8%L>UTc0{N_zazDcOZk3HInldyb=W=jDe; zbk^r-0J>Y_S?pZ!KOWTjXOYzX)=~_WzA8C;rC%B7e`zlsp^rglV*)JbJbg5j#+!bc z#V2@(2@nSLvoFeDZE3}g{asu29RT8ZPfz9@R6mp{vJ8BToIf)t5RP6zX)rX<0k_K~ z!5F|uqs8!;M_JKC8A5> zEm>=et(|p7$8~mgK4ECaHD>_?N!zyW$5PV~J`6ZO5~Pj6Z=H=i3u6g56RqsG;3<%H7yiOq1%wk7cFHmitWfZJ zl&&$d7ASuRUF0TG9G7c5}f79sbSfeEX@{~}`O&l1Zm zt!+{)*{_!F0s!vgHXfuw$K=FB8T5Sp3nVWX*Lem9XSj+f6WapS!Mo`h;KB|lZ9hzl zA;?9X7{E495$`o+fQ4Zs12 z!AioR7E_%Wp>8t( z0HP}P(^$&g|BForJ?CEJgR{iYqT7h%Dp+IG7$yPv+j7q3m(?K4BwwMvNES-{{mOJ7 z4uoH3`_k<=eLerK`N?vW07fdt3D_vY)cYpJD1%J5|MbuHsoC*+NPIUSd-|o=pfI&h zZU7+!gcq3c6!VGJ>`X(fsU7&=K;H2f_5Q4}p<&v~OiFFLJvtzv_58fuOR- zjB`%tQ@S7RR?Px2!3F~Z!-6*sA(d6GFE8dQ=CA(GyqZLqrs^f{>i9M#_-ThT`nU?-0&f+Ee%=-FzH~QkDmCIq)nj^87$%#dxB7?N_pH+Rz;6~oXm_qxbwA3_h zqe z%4zyiGgj~E&DV4W+Z@&5R$>X^o-TQ&is*r0jE zXD6pjzpeZAaJG2^%1P_g^US#$VxEW7pnTHgdU5T;-J0|KIjSqa-K-t7KG*{9X@GFv1!$^%P7hq!S7KaVZ<-Q#PH2QzA~Ir(oy4?ick`Uj`N9X_M) z(yq%qz243r)?0jMetzCO(OMj};D?aU$kki!v{GnfTz_ZmEb1YLYianiG>g?w^$8f| z-Kn34Vjb0|`(9AAbk)O@ud@S{9&`qhxjPWdNn!WjG{oxLI$-ZJo?-fHp1IX`hn84s z>1^Os4;0fd`fcKFY)On=`;|(r8N)^Ki*1vCQ06*6$=28e-y1xpNo#^Qzo9wYBi+Ic z2g3e{U^Y$D*svn^OOpIrqwvqp6=h#w+m2_uq>-AiJ%#e5L(y^Z@wc2soXyM@B!au#wt6wH zb$6sRs%J2Dh?0qX|=J`=tehIp@6zWmrTFVz$woG@6*UOnHj~%yV;dAxZOa z2rewKrdPv7eH~xGi*Gsmthnmiyf1 zGRCQH{**=Ny#TAS(gTRkdO93%_|ZAh3&Nr-JfeO(rF9opw@nEf2ZRzR9!loyfq|d^ zJ_qb|6CcIK`4`UnZC!&)Z;fYz|LB0V|E&YMPsGL&kGGo!mNs+ly4Jm+5TBoc*;d`D z2Ymi31(u6pNZ*M+{dBTjPgU{47njy^)ev^ijNAifOlzRtPM z-upV&`E~LO*36nUGi%K=_j5mGGJk%u-G3=tvH}-K#>J43r)v#KU;6ZD ztCnN-7DJudzRbzqFS7{WYkmuF{TA}^G7@`L&|>XFd|R5Ukoxm>MlCmNw^rI!Gu6M{ zxscAWX%+1k6Tb10{I22q+HV&A%hzA69dafY3-z9&&RsqV3-*ud6<{KGxe`Ho-;jpy z)uX}n_o(6*Afg8Tu`m8nJJ0%@-jqe=utii|LBMHR(vgVrC(b@k&XP9lAc2!fniya|psYLGAH;9X>gb)lYogR?ARmiSFz5xZ2v@y`!O zKeHP2+sPfjg8+NkpyBLS$#G$GMUZ6*^hHTkQ_0bDGL`i3Yz^!D9|M`dYX7q^FY|K= zy^(UHR6P*pxz?C)exTMVis;D|@iY57!*I^{S60Qm!^G(?6wLzn!fr0(v(< z2%mj{@|KhY1b<~aBLM#XtOZG6?fqGSz&Zq;KzHfS{G1h_((z7!Kok@KAO-qhcwho+ z`oFsUzaEwMw8h3qanS3TXn;@efmhm3h??ncgs0zdv1quVB-p;{L%l8b;dq9tT618g zY})0x?P}mK54?P)ce6_9ygepU@B_AhkqdhE^zK`Rr7uEJLVP%qFv#~yOvB?h+&Nd~ zspl6h54zPb7Xe=MOnf}vU1WTyW0kvxs-$$#m4m2u>P8DR!XI%=cEL+6o4m~0W~)e` zn~}1BV;Ekb{gJy6vOaFPld%h>mUVrL)a$by%4B;*fPVlm5qp4@WxXD<*e97{?!ALm zq4iyp(VTPP+06?wkt@(ajBT-8S@-p>7SNVX-I@$SJE!CFqm%8I(gD8bH2Xqn(p1Vh z3-)e89c+W4Lla}(p+L&Gl&2 zY#C3bZ8I`@^qDf!OLaU=13Y?MdQ#g_Vb)1uDqA2U`cBuL)iGBK3K3!UB_UnZL%m(W zRBKf^n%=59G~TZEIY7S)#&Pzu48}=l*^d}kZco$*lq~K9u($PQoQ~`nnyn@qrFLGB zP(8!fHti<8?qar)YR4eUmqP+WS$B;df9lOuFsd65rCvX#e<8)%{WOcxK@;O@k|38c zG;;!+_8Zh>`IzLifc)_xv$~P;a4GoVcGl!5cft6r_Ozffrt`Zp99rc@v|gamM7Ay_ zh>~kvTb~DoKte~>v+b?j7pyWT^m_~kIEwUSb6pH?8%I`LA>g>rfE#&f*b80MC9F(e zbZA)>ewFIx8!Z$}}8_R&SO zuW7tf^mxxxhEA5_@V)$uy-Q?!OsfPgwmtI@uf-?po9d@v=~1Cb?LjS^UQFx6T#CZ zZcwa9_;=xszajJFefC1*DDAlc3X&la%`IPLOg%J7#?CtmC9F&&>a~7apSI&n?V=vw z!lVp1sYM?8FtJr%Cylr}V!P@T5BHkNA&sMO|G4Lc#Lm{k(>M&3q#Ud@d{`Y4`XnuU zEu@>$;T)5SUQ*bU+@_h%^R746(e|nsMIZ5D%Gf2;@q5lHMZc$pPQV#f9#WJ`84m!l z1}s2W)3{e!5t{lF8==`O>3t->=c7jJI+`dXC-7%!uxBesO8;t0@L|>#qZvw|Aw@UP zl1GJjkd|$-*>a7=O=$_<)Fuu(9`EQw!zA#Zh!T3urw)@BRi<0@w!KHV_xK1oRn`0f z%f|)R*%0MKen7brLo+lu}SxP+$4u$a0^-L)gyFya1o$MG!!>I^JuwSZbK;=fY3LRUMT z_HBnCd)6p2UF%J0;Ieys(PAJ*M0H5$Z9!igvTFIACAnwvD=DOGvLJysqyW;t@y`Few$_y{bYG17S+x$C8a zLG0EIWZ#eNb+!w#S=|4)dwRY(>)JpQ#-g25`W$o)#M%zSh;I;nQp+IU< z)*4`*^+<9FkfU!s%{;KY{RJ{5)RskkiXZbRpf2i?%xnd2)!3CV`|jqoOgzbXNQg;E zQlB2qBJ%2fo*qqCdGYu&xe>K1c&m_1Ab(dxU~K2u@c=F3=5Fx%6df*)ZmrFxAF6}_ z4P!G=`F(pM(k`kUw@l0QX2d)d_JT~^`sTnh!_-*GjxKLihlErZ)@8Gy2GafRVM$!5+$sAO z_Iv)nMs$A6&2rgNjS?o@xJBeLhV-AXmk1<=hLNY2XKr&NnBJ6?W+vGfDI0ivB0^_; z0}pXIgQl~o2x}b*K`CQ9vqAtet168d*X&rKU*46E4?>U8{2bH7`kA59HP+@JVtzE7 zkQwjp{i8|_C>P_sDlF+E|Nb!n9EIvi!dKYk*_Jd!cqoopjs+|VC(EbFOWnSOM+G0`wm0GQiXT z3ow@POlAKMRKnvcf`10{`vzbcg#81R{)KhU>YizGpYJpLyB!!n5CZVwH>Tox`+pCd zVCsXB_I>4+5zR*Hw{k8Wz56?hlJ0in-`lKwtXV4oC$@hs$0?3mZ=XJy9jm*s&s!5H z_gAr%oTjvTIJbA@L@AoE6;b?0VN)H)I||X*p8vBx}Ap(T6T4#ke?bQEWl|l!o;6Z8&Xqy1zaWu=8xc@nntk3h= z>fHL#c)5!$#n5N2Sby#Fgl4<_*A_TVWTLLuk{a^Vy^YyE*6e3Xxxw0nrTBw3r~UGR z<(cJXtJE&Y+RGMr9FLE#1yFcTDHUjXO_g5G5${4~Uv^Ne2ZBc6c|@%|k*h^4r7kTH(%pK<~sG`g+*`}|+>nW=J8yMjkLrlY=rpuuf&g^L` zpq+!1rZ!P=%&ZBBXIOa2C$6)E7(X?PXA?=a-cWsq1RBRtP{u)A)d6H7^nQdj^awnO zm+eZGnyyS$m%}KaB64A6Rx5`VUqcr5gAb)Nz$wytyF)wPn(;XG3*AmAi0byqr9bmc zwf>a_7f9Z2OUp$Qe?4QsI>>tiaS#RCjD4gdn4Z)GH>i9nQ{YD{_nIE8b(-K#jL>Wg zI;q{bzW+Uu$R4GnoroPMeybu#r{6Fp@s8D=EN~3IU~_-jI{CmW_2Ql#U$Y_K>GJ41 z3){i@pfum13#}FR625=yV!W$56o<>!9*{D(?$77jY!^hVQqd2VUqj?&6h^?K22dXF zy7+vt&y_~hnH}f6AYVgpG1QnJ#oSOnS&MKHJkQb{$=g@XrT^{0-Ywc2;JDpy9P7&8 za){Tkg9S_n5CnO+3GKr^9#e?)lyGL()Bbh#jKMh1VK@P=XRd^7ryhzBG10)~L;?L` zgDAc;Pd1d?#YFB7zhj5K#UB96e=KNc&7ontza|kUy>rmsU2~qsJAZY3vAfoJvFODP zRuq90>GfVnV#NFJ?ROdisWo=T8$X` zP6>+_gB9Tqe_*vQ$mWW!H&d@8yf$CfQNN*?TOd981|}GWX1f#CGA`s~4pN3Xsygsz zCY0~%r<%`gW2Mr!8fY{5ynRP$*1?XIjZTbZ_JzrPwr+)@>-@i#P7e=!GdE?iL!Ch? zT@g?1c1Giy+|+8h4iEB=nN37Tv_$=Bkj9+Syaa;kf7Yk|!*Lp((r}^#Lvz7Q|FO72 z)pO=e+gJkx=q0w-{-wBAkwuf-`D_j<+}@?*x2Y3O$< zK^z=6C&x5OIoeiJ3WTLr+Ao9plzSzTG1Y-{Bk+*D|4eHBc==dnJMY}W+ED%6Lcc3W z%H}(=Qrz1gb4j`_B!W%5eJ_bamx#zT@Y94Nbo6Q|vTP<*0~1dDJj(T>-aTwH@-s;l zZIp1pCX0AQ-w0yK%xV5-XiMkZT~6f2q|Kz47>h6*hPnN%3JCQ@ro>SOerfVot`&GE zO@ux$)}O453+tVkS@lM-9Bvnx;@|b;faZ;QNg(=4-ZIZ(+Np7~lmiAecCW*EC)p0n z$GjtY-PJ!gq@)sMZSb5z za`Ao98=>`~U!f=n?cPdg-;}xPr-@s%VB(sO69a&!*;t`7DRgj2y6j^i#---93AJr8 z6Z>i>WJe_w#*%!((LrDu*Zyblb<=+8jY{mSFCsd6Oa@l-rhoCw*%Q6G)6d4ruB@_U z3}>6fb>2gb*>-K_bw(bpM$dM`e`sYuK+TC)0#=;lIpNLd6Qy5Lj zq$lL6kPbie3Vsp?Il@X4Lu38NZqP9aTFI?8qz-OH!BN7i2j0l`=fcy-QQhawWdD{Q z#-H5#sL8=@rN^+zLXI#HwSuFea>1dKPjM{rq8DHYQ+l6s{Pk1Q4EL<2)bxR7nBvdS zkaST^|E3BIGo;jFI(_2t0sl6*AKKmW)*E5M!Q6^7adQ`Y+e1mYaU<$8X5}e%u*!53 zngZbW+*9qVQ7pXl7_odei{|Ga-oIy`+46|Kwu~8UUYYZ8QvGC7Cp1-+es9ZKxC=>L z1Z**Qt>-C8gG}P9XJ&6&f=+moEN)K6caXX|D@T|t`?QfQvv}x-!V-i=GMnquzlhg- ztg_nF_7RI`Em8G#j9PgC+gBOjH9@H#IL>#KBD6uA(yc4Z$p$ogRgK&X;`xs|dfX;2 zQm>P7X}#gc-8CJ_S8Yg_?W)$1#Y{ox#Y8X;)}Jvy?k>%wYre(&2#CDZ0dxtldXRrW zhR#q#uADb^wMbD;hznl4O64l|yfCIjUsampCc&(C98Q{AQYM{MkA7$WPHuSMyGHNUaNHj}%6jMd zz#ebSm$#$rN15dQ>>UJFf&IpR7a_ddjP(s;kB8CnEcaF`ohXCyp6+>kB|INPAO~%z zcDWv)ib~)NMfI(#uE?j>;lE!Pj0=x!j^3SZKLA#nro_jvvYkx{zJS(@s5S1--o&w% z8kQ4Ro8`qm@2VO%#UQ`Tp!3OLk^lpt>ww`!vj6monlM$R_0bj)S zAuS0h<7&dhZ6dk~uC0^SK+_^~Jk@MGwuTaO)xO@RUqZeGf=L~ihT$#oKoq}|PhE|2 z{e(~tXcWkaszYR_lOV`gAVPF9{#{{D1b$s3(^={3aqNLg#tK86i{inh#gr@^dwzvpOLe``U~}W#K3*X16K+j)@js9|52Daq$wYa?1RLV5Ef@M z5sGVqD=WvSqTS;wGR=n=U2kDEY=2pry=k=-Q}W)a@b%h!aj$P$yq&(J2OoP=Src58 zd{oVIcV1r}Y)}BpA9^`4P(1n)?FMZCJ7^HECG;0?Yu_HDjJ5UM4I+Qi7SzK3_=&#% z=%$oy10;NVG$?%4H7#7shS8x}BV%QZ>6G5JwWjT&HSYd>yP8Wj@n(GTUoibXF`1|L zg=6od9@|#4d32Ny()t={|123q;hdh<&Oa{U-?9!7U94tU(tQk=MjN|;&(fwn?&jJr zPHE+%prQ2}1GUFGy0!)5kG_BEZffO2AeZxlQ#@$(^x+Q?^Bvv_Ry^I4QY}{xsKBc) z`$u=Lg}tS46YU;0?rVw!*b=d>U6G?#E$}ssm_8;^K3c-sBwL4dDTVM7nx;8i(wyVh zhaYe+Gl9{hFX3O9P~yu(e|l(tAO6FqSi>h{fP60 zRzH<|TugGzFkj??Z54RNKj`3tNc}sKDieiS$ik>0zx@FuY>R0(YN+du+nmjOz6tnE z0-y>3_C^B$OhsNhmoVU2mtuNavnJ{^ZV&Dk6tNnI_m}w))wEn3q7FU;Vn)AkEG4OI zf%H*hnIG`sx8SD?9wlBt`U=gpZKc`J7P$j6I8lw{B!^$c?h*#PR6ZuzYxNSxcD<)o zv5hn5v|obAkE@&80`f{b;WMl@53A;LvKVn3817s4co$4wYcgnV?YWYXaI=`hvcCa) z#~D;ffA*@Lpahm6cioHUDX>UR^CP&~zEhVJk*s+Vu5Ip%Es{RTgk&>T5&K#xw5{%FWCu>g^*HJRUVVl=Ni^&OvJ?zxaL;jZcO}bY^#d zcX1lACim}sSIV`ic6!^QfDn{5NnIvG663~ zeMfEe?&0_Ru!>(z6Hc6Brju?6o^noom%TpJg>54?hK{&&820W+M<6|WU2)XH#I$35 zJO7QJ))S-M%b_h77fGMw-z9Xm=5+^<+BMbS-I!$Ey46Ro72s}CmkG`8kBrTOqsI}9 zPX|kAe$p;RT_?msNJpTVi2zD-;UO}8Q4|$^59~=4bEL(Ss^sOfowpBVj0?=T|L9sEdJM z+G-loJp;sq62DXPMhvw@=7X(ygZ65Jt6JwbcMVvbV>Dd*^4fq58Hq>Qx^n8Lf)V9Nm+R_6Uk}~A6iw|*4f*wZjW$dK{TV{tac`ArIHh~biszi zM^<#};N6G$5EH4E((S{Qpe=DR&8QX5uVCjd-JIYx=id|nU~0)JzbuiI-DDd&oIE?q zP7JVgai-3OkAAEq)K)vbg!l@m0Gm>Gws>Qhl+lLc5E^nJr}15~$zunGXICC0!9J5f zq^y&Y*i>+V_o~LQUv6XhTLSJ3 z{Uz@wNWYN|8GGZ`>h*T@Dq_1*_}(>*wIB!?VuIGE-^Xe7hWkl305|l!qk?v=qTFj# z2RM-xi{DNtTWpXfYrTs{TEn-5`HT|Wm#}^C$3r4OIw1;kKq5l` zE?PBC$gDZk@B8=jFEVg?XF*$^ca5S2M~ceD&(I$2`fjMn_fPDbi4!vkn=9mrrnHa6 zSa$n2tA!t+-cunLXWjrq6YJ^AZZkn~X@bKt66QYE_S-RIRopv8Kwaflg~?CbOs(Y; ziA~;@&fh2DR{OS(6DQGJg2H*A zsHb*8pl|FGn3yEP1oQel(A2sx@KMpbafSHAk?)#2!up#6bHzs}e3O3KB&)=8aoo_S z?pn_rT;|rkRATQPA}Pk3+7(iCfRd@E$I807A6N!w1zQ`^$5kx|O%=@p8C#GPkIRm- zw*nno5w(v_gtGXOEpw^6Y~Tw|TRcx5=Y}n;N&A`JcD2Gx&L~3Sp*9}9!;I8TesXqa zGLK9xSNxIJQi^*(?PZnPeA~in_<3cd(7CPuo9F_13v@T4ijAi)r;o%)6kxDs6H7_` zv+5G=c8;|-mP4I%ZYgv?q4IZ~Ksl>*t~0G409wHy*2w^=K#_B?oWG5&&)xx7 z2>*>hnSkt^ObXG2TIYT@dYtsqOh%<^O_$#P?4ebxVhi?xkFx$BG z4M1Y}|9)fQ|5Qz8BmWOXc8MZ^R!fejg1eFYLwPQRJpat$}5OTo}=fG*tQhwWb(q`dOabF8_2(xTK4>z5`ZV+fMy=% z5@idruEbQ^cX$KC*GcJD?ImQ%hUDoh?_I~sTv%Vg89{pf4xWCT)c!qne+b8a7{)VQ z;vt^gJA38-UA2b_NU8Wg5tl^FW^W(7`k!0=?H%)>{4CP@zA(xbb*2EHD{Y^aP26ba z!Mx~vg8HStNAs6&2A%U3T^pV)5`&WQn)@gHwIos3*u>Rlj~iGU4sQU{~L*14&>| zc<ge4nlEWzR5 z7_rXv*CL=go_ML1(d^%=|G$<9eG-1reF2@(*nbu;o-hZx1IQ5(rT^2lY*wikH;Fzt z*QWHOgig%$nd1eql))w6_zlT{y2NtUFIj6 z?L|n@*uea-cdA)}vBut~X+fWS4v1XK*Y>)@ zPyDA)pZ`pNJaEjDIw9{P`1{}_+K$VZM_T1xv|6hN5QRo`O^9G3|Fon~Wo2>_Pp)~) zHd-0M;UOZO5cn%vF|F2wc}M!d`&DhNA>zm>kS%oGq0s5-g0GRe40 zzr6jCTPOV0Voq@I?XWYd9++t0^eh1gp;4t5w0$Yu{n| z#HTtHc#VV!3EyGE%>DL`ro)8QeghYWXx2eDO3c7{e3=3VOmez&1;4n}{s>!Q=xlSf z3duqxIpK}mHSrOh8Y$ZDMDA+t;~Y!{XPsVJu5;#Oq`2zP=gV2^_p>F?F9To3;T%9# zo@}han>}eb$PAfy)|4Im6K0o5oITKc?lak*O1ehs_%#_vL{@@AmQ!@K!0#G@g4E;g`WtO~A*dCqWj??XLN`E%h9#plX?^L< zKAeB(F4jq2k+l+@Vx{sA*;EN)BPEz{QCe-;qG?+ndlw*sc}_$1KHn!#_B>xw}=K?|6Sp z_4X+DHba_FdUsitfSOmBjjVCc_`G}jN+uO9O0Xii1>AxAENicnP@T*~5>ByGWWrZw zK40AjA1}ndZV4rz@ovN$FgWm~qifS@7#Z8Cs`ZuSKDaA&Yo>kZ`H=y_oJQwD8Ep zfHjaJ3nS{?L4KGfuljkFVAvqxB?rfcX(kbg$OFtnS1M~rP`E&UcR{ZP%`#KH2&~Bd^hRcZqY+{QDu6pho#xmLqgf?Er7>!UyOY< zE{{0AWcuAD-pOosrAb?|z1u*9;8is3dZOsAJx^g0v3)<~B)EShq2!sh@R#~8^A_=A zzuaH#&DPF%1P*r)&c_|n0g3wFcE};$*Pcp4+(*MPDjG-H#;7swRT}DqE}IeSs^#MV z93;cPqPYjK_8JNd4k;Wy6k(nG4K9dkHc6mOGkF&%M?AD`hAF0$V2<8I?KKeoIj@sB zlet&KRXRPzQ^BfU+BP?RdzUi`hU4ZHIU2y{>m4SG0>ObfdXX@eJKv5bCI*{hp}7}> zA0h*$o`(*YeT3CIO^dUS1E)DJ-ZkFIYYFzvZ))?aU0DhEA}KZ2*qAT@`w#UQ>XS=` z{l@erG`uZiF_eS8GPY}V!GaBGp70iyKUlT!?*9A3ak671Kh-iHTCu4(M`n2}~C#o`4EGPp;Uywr+o563ku<#uQFBiaORY(DP>umz`A7 zZM4hM$oLTdBbx#Hi*j%v_+c4C@P1$YtTVD_xRgsehczZF7!m4O2IL6BGU}opFIIyg zUUZ;|F)d5y4^&_^TD$kuaV+AN=-N!8b~D%MnF#$7`%2keBFPJU-%@QD6k zqNN%lAbThxpuk?*+QNGyHD8BNnt4H*TMsoPb%M5qLqcO^4t;NOYDBbjuG{n$IuBxD zht*lV(2?%>2jvDd=nhN`M5j-8b*7POhvjeQ0XGf>x3L+gq+8Q$c05%RAM+&F4Iee` z$K$H35FnXqfVy9ip7;Lb_37cKU(MDnj3#~YkG`)4jfqZ~?5t8-#Cf6Ro)seBA-%>7 zI8TgL+>$qaZY6D{8f{rO2daLw2hc>7J&<{70h~yLmWUGqnD$_vL#S6T!$!5;Wp`2s z@2NozPvPC&eOOalWlIt>s*=6}X=<_C&5>Deqdh8<<&BuzRqr2oCzJ7*5wvd`z-vzaYM~PZX zfD;pazWr*HO%_Xf&C2w1OJxT)Be;3<##FM7xp7=!e#ribmB>?F>YAsj$Es&GzsXIq zGmI~f`n8M~5rFz~nAwxG$kRZ-@fgZ|nF%1z&lq-~Ox_Y+P6y8kz})UWI4gV)6bk>p zVEKpcf0Uj&E`04-JJ<0|()${e4LOE%{pi*R+V-kHIL+9au-YnOqZqOx2Z5Lg{Yzt0 zE~UigGbgVgHIF3MW$3l)=w7~h6-y|mhF7Za!yq(!5Ih+#^W&Jk08HtCLk(_Ys?ROE z)oIwpHyRq%4>Asin2zZoP6aL~)%dQ#YmCvnpy(mMH9(_W+olr!`U^HmcbDPdly>K0 z#4^AHW@8+qlaV9o+grXnpB#@*UE)3O*h}EuAb^L#fU_+lE`1JvV~IA-ue0r;ej>`_ zx67!*KP5z1f7Ef~nD1lHCFVa2H9Ws_hs?puLRZj&In?W3I z8MAYMi=@jlItDi((fQbs9?!p`_^A+~z%*E)F>Smouj{@7>yCf_w5<2$X!diJHdW9b6rG5WBc9?tpB7Iur&!d7@ zKJDB-%a@a>*=2x$`b%&IPKZCrFJIMN&9xP3$O9Ro__Mc*x<#`t%##K+cNlVpc>5nF zrwMH?z@?m*Mg4`vLF{I~-XV0bUIqmwJH%bCY29Ost>Pl4F$b|s)qXN_XxcDm9?lQA zesCO|=-0bez!8r8SU+7i=`%4@p$|#@Qydh-wz@Zw$Mr3@Ai=$1sRn#|pGnx*NX5zY z(9aU^vs;D$4g}oWrRSOj4h0b)9%u37Z1}?U(e;`OEs<6E?}IP=)6XmGYP4)c)CsFG zEA-=Exbi|n{2+??lr+_)NL{(*&A|7y3_d=rYt6LlHxLsGZM07Y;00~OYxb^z{=v+& zPT8a11yfEj9=-Em@1LYDuWM@c8|T3!>+T&h0rTamo zh#dIW(G`*NCOd)e0V3VGaa5yWK5^c_mP`3dc_~LWQ$W{B#!LTiMX0q4&l-pM+zN|U z)pbypuRc7HAq?QHE7v-eyGoUT5H#=y3O6_ z!@oZ+a zdW{Cct6#MWwa( z%(68tW?bi+j1uaty+Ol5EPb{w!_;q1k|0OIRk|!1i)YOGOq?|%=bz-k%ceow4}P7 zP%0FCnA5D3pIN3ZZP%%T8pAd`pA=pe2ft|gB3Y@HAgDUDGbSAqLUkZcwf>uL0bDVM z!{!b2iqrIAPJr*dcUTKtWKB7ESWR_CWMS*P$Wa!@N(lA(@~C0A+oc}ke0Fb>zOREK zWA-)W@j@GY;bDO0!aOiPBng;}>PEPCgtf{)MN-0mEElD(o59l}wtl$il_tlUZzaof z$0Lj?Z7=kchm1TrJi(1nX?=mPJ6{Id+!i(TvyVWL?ujOAs# z+wt}No*avbu&$R%*070Ses~JUwn+>nslvNMz;*Sf!#E>TV%}B4gokXE#?Sn?L|4Gy zgc~!u+M|+VeoU-y7Ca=xhOB2~F7LPc2~A>z=N9x%Yk-u#t^2Z15Liq5`0{@?3k5>w zH;)nI3u)3sMNLZBP{+(cYb*9T6`AA*`CR9w7&JFH()h)EDFn@x0tc?`ReeD;jY{uZ z)en4$s%6GiJo4jT*4O4*MGWA!n=t6Y zM;&(~Q6if6K1F*J7B@O>3{7g23Y~H73Q`t9|ZKO)OU!BtC{+CJLRHE>_U$Xpi zGrsW7wJGh6{}45JUiFNWJ;M)YMC$(*r}Gbf`b+i#?Er`===L8r7~d=r^KCaM8J-FL zUHF$m#Q)1WX&XEZ_&WUW@RR>*j+K90D7gLbkD+%u!8XOBkl|0Y=YR!yXRc|o47zHuI}MOPk(0|Uu|Qt9#_^90_NwiAImqya ztH0|tl}3(s)?!Utk+9qAt>k<@J#c*0Vo~h-3L4;wt(*SS`PkZL z5(ETkefsLoV)&3!zV>WLV~-;gD+hnhK)&61Ph_Vhkl6f4nXsj<^qU4ToBPW znk0{t+w;2-LFWj*ftdANKV7Wmt0%m>19sDr*f~ zdQ{ePj-dPAX$y(jougehHd2)PlZ)b9|;&AnQ$3|UP|Wq$U#A+`$Pw(ZzC@)<$^ zyJ5;$W5Y-JLA_d=8FonP=Bn#a+gUNHX+S`5f{7nl$qn^1Svn3UfSO&Dyny+FHoR3T zNl36R9+Qg8s3_tQ3kkW5!fmQ|R+mZAIn!SuAXqEZYxGyor}t<7bm4kTMXaT^(bn3% z&PD=)yPr#*Dv4ce%^;V_w?KCjgx+%fkgFlb0GLf`^x6I6-&M^wpMzHH$0{E3uGt5j zCxE%$+KkYz%9;M2NVuo^Uf<#C0cS=769Ehx{<94CQ~Z6m4atD%A8*?RjE8WP3(Uw#&?7Ln`B5Po5y3P1qbH4HkIKSIvvm@LTFeBMP*+13zPc(3Icbz4;n6W)mz7I^E_2gf z9P!NCFx_AF+Yip@$A%x=O_x+8(a0F?t1)TR!7yIa$7R&0zCTi|Cnk`OXtye4Ry#Dr zDD1H#j-HST(DO_(H~Q@^u!o54A!;gkDnm~0?1Z(DD3TDMa)QMUFzUXO6OLw3{+)Ln zJ@6$~>1VM?x;A|C$rA2sY0k1CeKQz6-~>rNAS3UrD{bx9N~WEc@(L_I^T{t+^z$zS zUEHH%C;QNB;`H?(YGr5^o4c`b`l|}AUr$0n05NW9^pK8Uvjc9>FN}!l&h9=3iRx04 zrcCgWd#)wfDtXIjzN87rw(;05%SwlWL67f!u*{B5^#|*&d_!2h#{1^7Mlbq&^@(aX z-Sf_U@BD`b042+^ADvH6dvK2rNBzA_b~jhW{^Y@T2yNwpIp>rh3-r1W%lLD4mU+7d zw=s`ZAQnKz`}4)Ej0|Pom2W)0#U0Gy-U7$@`qF$nJ9jn1g$zHeiXNu=U7h_F14-Y< z_6XH3xdGvn2uE3(?I);ajs&fsJKN4%n51J_9?#k@mB$7p%xl`k#rgW*4VSXKV!d2N zXSNt_!78?dj$Vu@XO?xW%;D}ZC=d`tGPy+&H*C%QB3{a9NI-v*e`;k>n_$M0yVsLs zug!N!yZ4vywSkTD8`o%eG9nX>>*rT)*NRozOz$uA04xDY{=2o9(fTifs*K&nMI0UE zCgX9-arP1VzO>%jr#Q_~I~o#6jVHfITaGYXbWO&8Knf*Mt7U zE$S<=FZ`YL)-l#eG(I~CR-bEcj+kp)6N-Xwv0149)G|ft-1}giY8K|Qik=y2Dnat9 z0Ra?gGHM~@FN#Tfv7(iu_AYOMtD_+v&&T` zXI_+UNX^}Ur7REGMxBq`s{~(AJIYyLeaFXc1RTx>i_)l!M}L~Z_?eH-;2O=J>1a;0 zB~*NEd8|8O){)}hVtm(iZp+sz7CUfxAMN=3N_7l&`5x!6wV7+2(wk$qWqMmDQe|b? z-4dT5DPWEsKWR^~<3p6srw(m0axJjD=#_miwq@cPE;oM{QD2tJKmhW3o#;#R?cMBP z2h30gIJbh9*Y{UZ53WX(%u|-XIH!D~hM9^#pX6t5`m4p3`DqtXOn$?j|6$T|Gcmd0y};h6ILFc2am|p= zXh@qN+^|pCwfA76!+x}dfsi3L21r&6yp6k-*M#@Y_^N&T8FCvu%f8-cz7+hc<|fWq z?`!(4QU9&<8duw<6@4=zbgbFBNb~V0bt<46YSo6_@RPx<;N24}*dd}Pd-b)2W#C@s sj+DWnX||X<(D48N`~P?bU~Bs+qONKOst7*+t$Rkt-WL0nb^t1PA0Z(+qN~atuJ%m&-1-+)mQb`Uwc<~ zuU_YMt@GN|UF&FUQh8xfT1Gk+Xi|cm-N{*K7H9@&8$)ww9v*0VWj9+OG`*04qk*N3 z2{gT|ffdl(5#$H z(9CQcpT6jYY^)tWg$~f{|0)%M4mM8qMxQP@{@o#9ZEf?}&B*X?Q_SAR$@ae`D%%@a zJJ|jm$;j<9)!(9l&SpkHMKQs@7XQcP-{k)-O4Q8K5oixhFKTJv2owey*#JK6N&~G; z98IA){~6>n&cX4sbSr4r^fRs1C=~YZR+KhxD4`aR_`igEnzWr0-Y|r}3Xp8qfQTsI z<%s;8`tdb1K6*s1z*5eo)x_Pzqd-PI9$3fCY_h7duY{hoKE8?;Z@qgW`u=lo|LxJ~ zrD0yu0&*{-;V5kU!d}7CV?AfVQU|L8tK+B1ToF;j73sV8GN09ENNnTTc%_<8+pA0? z{ri~aiVHn_>`RT;24B1yIby8vK1ednpj>7|$S=9>6#Z^t8T~p@m+}<*u3Jt@oE3j%I_x|lv(e;4av^rHlD7H&en3y0PA&Pdl6$y&8OlaPZ4xQE zi3da!n_Qt!Z5^MyzNvCsSu>|&OG+IbPdGSEI-4%NPs;^G0r_>^?aE(%Zrg1KPN$4c zxh58KFD$zd!ICw<1&~~2dJ(WMYY^X+-3E+{JCuWy#~q_hblE$_pvaQ$m$dQO(5Vmt zCD(f}1CF~*^obtX#GA=f_T3$t!72YLzm!5`*>OIuh309;Q8aI8o6@QNE4MA8MPf~t z9Jv#gOt?)^ZnOmlFAcA=Mx#ukj$z8T-i9r5zs$~p>_DVE*((+viEjZiS5*-|;+>nn z<7F;6GZPlhXSSfldroBssDk#vPLLL1vlBF9(ra#8cDJcE6<~VQb&D3#VF+5Zsn)1< z^|tB@Ny5Ty@6kQR3NN9CR!iCKYkoIk0@k37*Np@+9;h&IzdXV`nfSg1B@oK3wj`!Z zxX<4Dz@t;J#S-r}Ix<%E z8QCD=sh>*Ua(1u8%c2YgG%&%MXzR0n#FO@8N&qxQRefc!+Q^i@m?8NY^1y(c_-RwA zHw=^eS$oyHF~oc3QBX51JUTW2ctZzu90$nium~VRVw(*E?Fc#)O|^orB|DI5GKk82 zjf(L?c7&lS--&Dpw$v3NoSvf~hKyEJM3CLZjfdUz>DN8J=%h}83luVgUrySLhu=CEBI zMb56`+#lO;l(fctc&1tMNNl^`HuUxj1k1l#o5fEkd|0CE8wYo{>3`GY)QHsY&Tdx0 z%b(m*VBXD3XkLF^@}xQ&dpi(tc(@cD@Yv5UXP8Yz%S@V$znbDPKOg$Wr(yp^M8FqD zzrbniyh~Q+p+9iNibjXGfr_!0Mnfe4M;+TWK9g6GY|vH%eb2uPM8krnMhFYLC-U~! z0vzpNjK3@VMC44a*vKme-HFNEDAup~0k5QwWs;ntIcUjN`H}geuG^uIzW18UFx>@N zBpB+3qpZv6cxrN<4qP;?o6LEz_^}MAw8_*DJ^+%mokA`lhs(3r!L5)x=xsAQhYcK= zmOq#nbAqP-(FkleMiQ5T)h}hP41wropE*9~`q7fEysz;g(mG|v76}FZ2+}ze{^~7_(1+}<6qywh7WKP` zxJK2VP(QL8T7)<#2mC&^mpJU_67%HF&~PSE>RH6DCjzlLs4_t*9^p(p{_2q^+>mV( zd+Fs5Sez+CbDCyNbHg(ajOf&hy*uHVhohQ}Od!}VBuwYcCJqMQKy{d-6biWH%uu8Y zuE9JhOBH>6-h(vJ-X)OLSvWd(#+L5yZ|K(Q@?x8%*k6B4_|_AOEht)+xQ$ ziM4CA6A|Yjnev#DH~k%`$O-A%jme^E#zAr?p~W&{Dho}~m+O9*1ARnKlP_di7u(TS z4siEum>8qL5b1NrC0L~&O&vx&vD5i(Ud>t_t-z-!4!!HZVCxzh*%R>EMHEqAOZDhz ziO)blmE|NJiIARY0VtJVi~sE(2d~nKIr?il{+x}#^pxzk*Vb3f2FP*6OV_ASwyjHf zlf)?jC+W3f{lrj`z+Zz%Tt!i#+^Azr(^5(ugxUVb$xTY#G|2vS!Px(|OCb!c20Q#u zHJ4~+j$vksqA(OYz)_u)*`+kuqySF9X5>$mr17~bxMf*~=>m|B!6| z1N*ObT!YfDMzQmx25(a!E43QTnhm^H{A_iqD|Xm<58gfq?>rh+y9xENAxdf*ohost z7>e$G4o{pVZM%Mm4O9$AsADViZ=XMYqY-$N?AC(0&pD`}(fSe?uCi+vcQA)@*B9K8 z&b{f6B%ja?cB-^EOiVw#^b6<8aAlaE3U zFShTvlna2E4VVvj?NpFo!>6@VpvW|gtuk!+!t=%qs*pV1xC<9q({$A2d2|B%Svx%=-N&&bKb_Fp>rhZ_Fw z)cRzN|L7S1GRc2oKuF-TL2qR6x%?kmVEhY6|4YG0_pj#f`hQe^`Q*PM(NBu`2TJ~K zFhMi^Z%A|eqz&u8ou8rf>T-tWK%;-Of14t%j$%rVpB(Z}uz;nlsR1MOKf`}Q(?3lH z=zskFhxqJmjFfW8|L**(QqaHw_-~Jlil~aF2DN~_nSrH@@~2hnPf{|o{=`xVz$dVoIl9q` z|K%+Qqt7u6tR4SG{pC-_zd)<_7wwd7RIJVZ!7IkUQ29>@|Ml~~Df9me$Qb{nz|Y_6 z{}NfC;;4Bq!}qoqYDZ~=?$}pI5F-joF#PA=*8D#b7mBkJ15a^Y9}-;BSj-4em#(Xr zxm#U#bnfJrtqreA@g3oiEx0XsB6_tVs}AHc;!4b*zH z_|m0x>BMaX3rG-a`hbmrqD0wOlCEyEf?sE5z-f^LAg#4XFz+U=M)s`2m66It!dXP( z!&lLcg2)v5Wa(-862NHFY$nr2P6}*sFqZ2aL`MODrYDmWOQ|}X`VuH{`;@E~x=v4t z14_B)Ew2!y0d(fNyuw;r$tLWzoSs8$nuHUX|Myg4{Qt|#jQ=k%pSDA}D$hSbykkZFP?x;#luB6V2gdW~bxwUP;_;Rm4SUs`tB}nNt2J59b$)OXvC1C#59X+Y4Lv zFFd^1)P^wVlj`c116920x-eU>88z5MA+Y-bAwz-R!NtA&9^+>cIg90v`0`&JJX?mp zbqQT!q=80v<2~|nhsp86x9UT@A_`6+bXst}Lbs|J5G`nf9&Tm%-E|tq%)wLmk8>cT zZGUONMM&zkhODtYW>$w%ld9ZNQuD{Y{>>f{0A`Deo!au9to!9UVG1;;2dwHBYaUsi zcS;MR6aE@PYaV|=YBv-n1KbCEUWej)syhVNE;4)X*(G1=bO&j34*JwuJ!Nb zAd_v-#O{$LNVce?OR!fuGsdN=LF-3$+=DI8%^(|}XwoI*wSEN2hmq(Z^10JQAxeY_ zal%)u^8k}%>`s<}#473~>{FXI`m8aZasGI3Qhu5bjkGYmH<)CE#ctwDgaZ@aWeR7y zwDH7R)>Ck2ZtgB!gRJWfPuQ%%&O~FW@Vw0tuS=V{HsKCMjdJZ$?-izOpx$Vf=6ej4 zTzQa!I6|rIEqkxdBd~Rgpe@wrqBmIDK=z=!6Jr!;vzycXaLywR>$%ULVu$>Yg+)#WT{siU^k_c+>XAJ0hLwD3)=r){jWv{;UL{xV|+ z$83w1Bk!M^73Sk$*KQ*a=7FONq)TswT?ozS584@%Hg;GhYJfGlrfP^;ryqF7zGs=3 z(OX*NFlL`bGCJ=mb;alV`r+Meaw*~o+8J}PWpjB!Dq}CHOQ~~A0vCUrk2#}wgmi>f ztl%!;uJ$a?MVHan8GrgK*FIC5f_%^S8t^3eME8XJgba+m&bf{`mOm^lohQ5VsXJfM zSP{31SFO~k?r*6HN0t{Z%Jls!g!7-9 z;^9lsOT#aq9I{Ib#JC71TJm%e2K%R3iqIr{IK@T5x9m+w2UndvhQ<#Wx6Xx-s77AFx$URe?H`i z+1)}s+2k2qE9~R2VRPz(l?90utJsDz3A{6>fgxc-fL#;DTaV`3bE3~CP^ge)OM856 z_Fy0J-I!r0gCgFC6=}f;M=~USUk|bu6opRp=i$T4UmC3v!tG7dPuww}aQd=`t~KEs zS`4vuKS$F<;6&^w>Q+TokF)vs-SgXGuh8jn;;2619>HPTU=S7PqX6(n3A5hd=3ed- zZDA$-gfj?C@8RPEHcfgy57^0GXP0S_Ou1Zp)SONP(!DbW0(h^%>quo)u)vINktyax zV*$80%$3JxutG6}lET5y<>DKIZ#-}h7SPi3NKFHgr<0HEN!_sCvgXAsxj^`d-a8H$ ziS%+9g?H_^Q*LMJ3&5F%$wK;cX^Tm8$w=6>wa#dZBb|&R6(t=N%~15={y+}rvPKGK zj4eenp7_m+W98tPJ=ZClj0zx>nc@r|pMi8iog-g?E&>5-E@kAZSe>z6rayiFLlusr zUB<{V%oL5g9f+hgVY$d;Rwv!i0B|n@H|I(kcpfy0gm~^ZlCj(=f=ceJ&}*Yxa24JQ z4t)iWaWUd!N5>DjLzp~Aw*V*^zF|XXvv}xyP`+I{X=OxpTy-$xWJnp5_J(SZA7BYx z1!`&hp&?`9nd@9sg=b@jFJUC!sPr)+*@zaMqCmaP;%c|sD}s7^0UjJIIMG=FybAj# zEdpG#L~R6Fd^>W|MC)yG$y!3kvnzcqjI^X4DzrB9dfk+eDvqv>l*naF;|mrP@H@WU zuU0ZT9&KM16LqX@y7Z(rxG?8Cz4ba6Dr+}d%aqy87C~8E9L&^Lz;F6aUi*P4Ex>hE zEbfxPW8(I1JkC2IV=d1w0V6sBIz}ElFt6T~nSs`2(|8xiS?$*6DB7tJPLZ|~q|K{L zOtGzs!34QJmg~t-S8D?+W3KCS-Wpnuk)q zcbP+>2l3eUvL|CqSPHL}Bq0M!br?|qL7dKFavkH_0H;}ZL*`et!OP--qb4E;tTV5| zvRMxWUf#_jls{KXEjA#O1m1h2vla!ZtmeM&wDO66@)G7QI}&W9NwBz`h>)hAcT#sS zCXx0_Ci2l)uUtK|@p``%B8cv+$(Em}1kGlpNx^tpDIrbt3L5nmL5XsL9|EoY0U9|K zF>=}59_8FeMn|&#PK9?@ly-T0xIYfe-E+&U?tFB4lNWrPD@D;>(Hl)L9)csH5HMNo z1j{~JC=b@)F An>K6jGE*$1Pq+m}_Kc%UBzNeL_VX47l#{p2<6$m+N(hdRWK!XZ z73Zp!eu{dH5bSCm4voLDkyU6N za-8ACrcQ4!8~dRk7?JG_JhSzv?~Q$Z)l)LJkrUK0HbN>aqi6l`%v>_IV+O}O?uq+O ze0-|S^#yFb#R+t$8swy)ENvy;oP!>3cK5{jYV&5ZQftX4OoT(Gczw~B4PIR@*nu_e zPOxBW%EThQ9Jiizcp)m2!yqZDqi2N?xxsGWMfOOkkrn?!W(gHkJWNxzURfd#163cg zH+pTrD#z=9?p776Pe(&J_dFu7MD4j_6{?C4tD21U4^+&i4=+NE%GxTMT{846vgohj%DhCLo8$FfipYha3Z9aUhL^5KSC&5qz zqh1aZyDiKSLCfzk=(^~#u!{(bC-_3kRmAHpJw&QmstIWuX-QQD8Qj(U7-`SK_mmz2 z!ItMTE$wtkrIYHFqerLAs#cO-skaImP)ZuJN2wC^vLl$~Rh~4Mj;${d5Kc@D3O2dzT)5M75bd7=(@hXOc zPOg=_U&;(#%22f*?c!QD4tXX-aAZ)mD;8o+>Cm+(>YPx^wL1i3h5ykQqHDUG*vu}1 zCwhgnIhMC?Dy!JG$gpUi^gwAnfW3J)6Pkq(dp3@eNQ>vAw3j{LvC4GNTzNXWAyQ(Z z-$pM*=cK=@P_UV`p3Xw(?BA5%wCy+>H^rrkOf*Jf-aP$n);q=@2a=jMh2+7+uDd)x z(N(e9{K#NVSU?f3xx_B)WvJtc*Rox!kZ%J;FiK+dXbKy$APiHb0P~$zUueQ%F$nqD z?)A@eA7YzggLlkOYFyVbdbZJ0+&iPNj!w@rM5u7;`Gu|m9_;h1Q^mkd81(F|zKEMk zmM+HR>|FJY?z-c2L%!ERN6S;al=afx<@G)Szvt6+_uZ=sSuHbuMq4}SovM`7C_Y~6 znS`^+Xt^G~$fU)&nu99xSx0o9%h%!w5F64L&WM=sQXx;mkkZ~qXuE`1YrTCQH`30` zP3t)PgcMxmKq=?IW)6>T!h#-wT1c9p3mZw?mFyF7ahc-DI6eVg4Zk)Xzc$Cx zWIJZ|b0wAX5|0Nw)YrbKjmoIzyWX1oYoG~j^t^7x}C(T&PX#Yyrg6a}ZS0HOJ>I3X8@ z8Xi~jK@q*Q*}g2vmL_79FcByG!|I|*J)Qd%kQTh3Ohhrlv^oz7$S+q5@H*c7=(uEO zy7-OHobis&vyV&ZU*@M<8@Ed%5w?U+^vL?hKGbdcV&?Gg$hf4O`Gq)zyd;-%`SN*9 zJou!zS5*$Nc2f>h&zt9b%vM<63*Yd8p7Cxhz4e1n0IbQm+FMIrF*TnguCQ8HNXz6E zyFg}poE8}sy`d3E01&uT!$dM*g{cwCEoQUxV1>VJ)Io-}A))t$W$!Ga6$dSsi$>Jx zdoHxufjW{qYG_h&0&l;X0IcHz^U?KL*?!#9*A67qKI=>b1?M-_Zs)tfD1z*{KZCH| zrDheyyffN9ONZo|-J@7?NdYE-9SAVuocdS#az!S0;WDx`;*+$ez9@6;9x9~v86~Jr zp(<*8d!{wmkPD8RP#YX5j%&7Py9z`zyrP|`c zvDhtf)1%n^$;;dPbB8deFIuO%9mM+Edg;2o&!+39NXiUn_sw0i?@>3*deS%5)PiZ|N4CjqwYA$gGUIT?$m3f>j&Z>P<~U zpHiB8`V<_k@4M9u)#7CWwW;Q+O6B#%4SDh*XM*5??u3K%y!Rx$DoxoAO`&&dXl@B- z#=Ic5J?Z64?m}kBtXqh@5Hcs{}v&mf0ZrM_lSEfov z8`Vywf1a))Nb^$8cHG&@k@wVnBOf*)sKtj=${S!~TO3=!kscfL+;`?J5DOYhB4xcr z*H|VYcU9OGq`FX$2MWMU>wo{l!cOWPE=$XOq zK!s(PT}41ZAJM)bQg9QrMu{7~BqkCQsdx6b_8l&g-Plv7<;FbTxk?21NatNXgD zLZu+UMTw)+xo99!cHZlsc_^4{B;)T3y*!ps370|1A7-jxy3Wk@x|3F!8T@40 zIfXDuvwc#$ir8lOj0ksp-z2wsEi84Apkr|$QAz3$w3MUsW-~2UMzB$s%$kpyk)W}} z5ll@n(W2h0p3e^Ts&}jJb{En4U!~rSZ{mn6S`6rGY1_}2@Y6K>JI+_U9xT_`a@dAG z4yO~tKd+QNW*yiqxrLE{7&B^I$Jv&D`to&5z41{Ub}g8g87&$tRSSlQa$Tu0_p#nX zo7}%KhAs%&|H|KC&_-IM%s{)Vp-3AeNz_P>b7XGBE>D&wr6=EV-eaJcI-=ap{KH9@ zVM;~9qAA`)PB@#yak&EWrV-w79R26zIJALlGQ7%Llc`-VPtP*PN7^!zMJLTV?Ia6N z%eK)7&g{AMOeR4vtzGVF3!;b$At7f)pC?f z-b@)Ivm!@TZC7f}*YFx>P0CXJP&x`Jli#^Ndl7h^NtQdFgB0T}dE?AD{Yo7I^d}IB z*a|~c$ZyPr;)GxXsfFytn|ufv!?Ou^v7u+gg=BbqJaXn|8E)0JX1EP0ShL4$Q>D3b z3-YE`Bq?t>lU3D8)?ieKWe&)t$TK1M+98fu(M*McWH1yh;tt3bJ-`mcI_AMc`Ch)uu zvNy9l8@;7;W_22Q-#I4!D9>35>~+tE(}4j!|!cm4zK4pH59^ITT949*1s-eBM3HbHKV|ReKI!cIca&rq@@XSBeRO$E#aw!;p_!4 znX>$-_#C{CKk!Ckj*k~G`r-@;N?5w{_0(bv=3(>d$>>0oiBEi*dV zpKnl^DAg`qUkuc}K9jpS%zj)-L>>K2f<~R2=TP6&iyh z6|wgV15uF0WEhXT_Tr$Hz<^CsJrb$BhOD|TEh4+@is|xL_uR^morHMbfKb0juqz<& zCltL|eDYkM3F+)j|G1|ev)SUVN9V@d*`I*yw&~z!D`)ODLf@BJ5B1<9^ZNeOc#*AAtUC#)C3JjQDNsl+uyg} zP-8ET&c=~YOPNIHS}U%-8xR<3-f81M?z@Jh6jQNq&sE9mA9PB?2JL5o^6(7zG62@5dm> zhBwkH=+V^)H>>lLc<+d2u$;9{Hh>eY!0yENC!LZ=^JVEDXZ^Xm?O1VAS!l=w04pQ1 zy8*9jTaS;@N3UD25gJ!g%&L23lTB93}xKOesMcbEFp_&Srf2>vEWN#Yc3R0}O%U5M* z7CL6f`{42cAS6PjUm@)&zI!U`Goy4?CPaHm;6GT2rw@n~70ci;aTd*s*Rin9(Hq`5 z@iRbamG{)TssItq%%nk<2QQGfdj!dnNpx%{dj|bR3!`)yYq(F|=U&NWnXEqU#OPXY zea_u(Y2SC=q>`*WT%W?mE$h6Gz=YJ^owBKUtnq5!?z{?YET6PS*=e)tEI-Eaimw;t zPFsz5OU%;;w7$eUOd&X3*8!s97jsvPFI4u56kEB$xdx<4^E<^?M55U9*>Pj5K6->P z`$yc}GHOKL*9tfQs-&HSN@aKQGv|}#@rI6OlI@M_Drbc&XX*_p#nNUC;t$pj5XJB9 zdnt&gwZFHd(|D0Mkt0!Cd#xQPVrd3Z#J;pkw2q_A&iyPXhKHcgHr94n={oq;6bI6h z-X6PA0lk{XLgkSl6jlb?ppa2pI$)^jzfdHhiL93~W@j$|X2_cX&(xmI2yHts_UFJ( z;GM`wl&y1<@N0&sX#N;GNeiRe1=lX2_M?&UpV-9KX76c4lx(qEto8zM-fwN+ZSNK- zL~WsGt1cy3({znW4TP|IbAwroqf$fgQqrF-Cb@)Yh2kW_hiq_S!e ztE^L+$ZoxpWN4`n>S3Y`;Ze`xMIf(bQKUt_W-6VA@Vp2mF!rCP8n2(#vl@j_9v%L{ zI{r5Cb*wemxMP#wmJos^es+KS3$8BmeN(#0ViBn#EkT2-9&9df&DE5^2;>MGjbG^cSJ_I{ebp0~@v z7B!HLMq0Yn2aS%InGq+`vezhMqc7(%>z2EH4Pl|e_aK6+T^41-u04u>J6u?BGAXC? zuYPa(1G~sO;pxIdeziWCVX5GprA}J;>iu)gHV1_D#8~@q&^Dv@S8#gbiHm!3r^iCy zAL-TCB1?3wFKl-6eXRHpwCclFA?ADwKw!G+z-IijZ7Y8+(roVPO&FDz@&sMU-1wYD zf$K-*YAsCU%aSg;QcD&OHZxdew}!t_Zbn?qfeCE`{f+ic`%3kKN9l`!P27XhB)RVT zu6@7f#?Q&aJ2xF?AF|1DNG@gm{@sMCbv#(0u)QFQp@WT?4O2!UFN)t<8_onrK(B7) zU{H|uD1vnBu6QP31=amrzWj}Io`@tt7(nV(Rx(ppbXWZv_yqT)=&hUF$oMC~5_)i{ zQ!MUSZU&GJn6s4TyV1k)l_vmwmri{`BaD<6xw#%j!SN#gR-hnbSLwm|Dupz6Va z37Ody!-xg5p7)v)i)Xc_RIWbEj55-psS-0@~;-UKR&L`#8BptTysO@Y4(x}mmQo45$X6hFYwg8q2C zB{blN(V{1+z}z`*DC0;EU_BYL>}?h){0Vkw96+6!?j)BYpkX*pZRaz6q3dOd`SopcHDS zbYq84iV6ZQHo!JLfsdNdYP+jyutV;{mP0pVaN_`aJKf`sxmLkW1|NxTbF+*atOu?O zr|0=6HMF%q9sM3s?orL`3!IHpSsKX(^fU&6Uq~^%o7=Pq$9`~boqs-eW=-L#CHQlm zuB6-r9sgTU)bDaUK7HaoJ^eQcY~Qz)z&l9hV)U5D8sY&x;~EEnj=?i!``Q`@CaY8` zDQzIwlbd=W6=zBYbFA?3qDfT*vA`c1SY5&+Cp07CWt4_rR{B&cR*@2NCdaWY1BeI> zS_RqbI|`piAdo%)NmWp_a?EVnsWtW0fuvO*^dkk%cdnNyah7vT=qqB_}P7o@vqmInB%au6c4^^B3kQ3 z4Pf1lX1w;Uf!(+wflnem<4CP2gymBVh(ZFKB+1Zc1Ft<2axLT^cKv>|xniC6qyP1{Ka7ai?A+fQh_uoko z&5i8QfNEauR!~uJVY>4iRibbI|5-q*?f?Jz7LnAZ4o&HR94k=`ka!M5( zD|OpzidVrXfc3U{#3WP7F*F=gdYqGS%U%L+oOyvoi<HElFE;N1UxsNCL|YExo7 z@5n&R8Lf+pv?`zNXRDjCmC4jC2lUGtxqc{uP{C8NP^kb9XvJY@$PcJJL&Q=Fh?%RB zPatX&R#Q&K%VrCtFLORwHn)yp(ga+hKW7Xv?<5Z(N?k!2(ZoFdAvWyjg>oE43XWXA zUNAw&%o$!G_?CSp{D_B!`kfq#?}YGuN{E^0!NjiHwcGJ_RPahrVxDKscDgD?nb~V` zZ7>V32_Qe;)@WQE_HLo38@rCtv6W!4V2xS4D&!_-5;(EisIU0+%2|6sIo4r+`ctd| zufyE6Yj-_#*O!EeMrm< zAFKo33-O8KMV?%%qD8)cuaiIbmC;$B(Vkva2Y&&-Xr} z=j8l0YmM+S@}=+!&jZJ33J*ZVxD?Ho6pP~B6qA`y0a`vcc?9cUvlh*_>Pc#nBJPRo z?u&mc>RmD7@*Y!mfM?;z%!G;c^EFS(0nU)i1md@qLSemnpzfe*v;OoP5|-Rd-{(Na zVPs&Y?4eXc)XdpfI$^b1`ystOrOMw9JeBaS2%LG*<_{~Bt77}|8No*iBC{0j%loAM z7LZ}6PZYp4i%B1ev67~VT@ajLCtp!Cj+6)e13HJpwqwrvO|RZgH5+dj7HwY@F@`W9 z)JTx9^@%j$tu)}#jUTdU*eGsqxGUo-q#d7&z-ld%1?qXe9lf8uf!IS^z zi=O*-%9~AlU38HV6;YJVNIO7pO;x^Q-(cxh@M7A9+KZ8cJ_B%mv1@N=fGyJ;@Kz2YXEbSB2^O{>yuve!ikGXgfyr1F^utYt@IGYBngj_Z201^WVG zO?}&m=gL-BIGp2t<)^sJmY#}aP+_l+WgzC83JMsI=F7#3ht-(=>f0kh^eCMQw4ClQ zIq`$P^`uSNES}M{NJ4x^*3znGJIN$nSSMcKl`3mmD#&)am|mz84VGJ42(E@V+iIFx z>DBcA;o1ucI?qC1f{4h}>C_?dxIzQ49{VA}`S?|{0y3gTyV@&PvU5r2dA?#&v%}ce zMb`w+`C4hPp|r5v#ydRP{tm}w{5p8KX{w!DJziq0KTK=G;z>4|x5I<2)y-*h8Q~XP zdeGj~Pd3|-1QLg@&Igh4A}yk^F@Z69t>?D=K2sQhcaFS3D zVhMJ8`fmYpN;EjBJUb>7tX`=$(phBs&n$TgA_qo@_@tF$r*+-!+e5RK?0Y?S2S06z z?AmC1W1<~k*D<=C#E`^}&60{l(=kg5{DEHd65x(m=8MYM#{{@KKQ49RN~8c$w3Jmd0ESZQ~f>hAx%dgsOk zEl6*C-;>^JZn~b@!|BdXy#@TEADlPxQm@2QQluYO(hE z4Mg2x9vNytz1n3FyT56F8aZ2WO_9kite@JDu?>kr3rkX;gW)43n+_m#+g%HaR=7@o zbGNS6XDMP8ucDk0!HEt!+r#|N9T72`lp>e1$~Lm9v%#uPWJ)ym=nI`rHQT8HI%|j5 ztslS6!)-pon5)O^1z+czFX|}1`{@i2L1qX;MV`;gFnYj3W4yJIvEjWTD9+)-&32hs zv5^04{bmOL{=;|gK@+f!HFu6=dfE*L;0{QmtSD(R-x)D=mjS)Xb0X;*8Y1~x8>nZF zLyf4WqRnd8ADl*;TeHknWem97Ha1!X9f#&(rTRB0B9PUC4Iv!^bJ~G^8|YM+k7cl_ zzX`CZyQuH9;WuCG!mFpRV_&*pD}`THau%DL_-V$JbV>!*9Gf9hnZzuQs^VOv>HGD1 z{hF*L(E5jvmIBNb9Re!4zCf7)GK0 zkO-bNHb;%jHv}_7yGRbz07YU#KtnT>YG+fqobz*MT}hZDl+(isF1PcL9uAKykq#Zt z722@Aw#-o$(MZP!)Uc6z&ZfGFVrTWjNg9CNNbhLwi2j5mS)S*Z!zoASy5*w+M~$>K zyhFX7$P}L1sF5kQnY%0sms;m^%bPWkOz=vq@gO&4AJiB_;liVCb;wcwx6E8OCo zMbO~2Gb1J6jEahz$!L6NjX?4&{~hKLk|A=YCb5R+tTfi>t&m}-NPAS3WURJowYfa)r`gD&MAoBkqQUoVBiOskzD~{b1iLF{TR?=!(zEj zaoOwIY+=b>;l>fKy>4GS?~OVJ#QhhEQE%*dgZ(MXGBDlGl)6|pPIh2ByrJd~%af61 zEtTNPRPoIjuY1_+;VN}$M4bgY@i;YuY(9EkFjc#xa zYfI7BVC0rTCdIK1;vBpIJ&K4@0Ji>w@K-ky4-rusZHSb$kZKUA5%V~P(6K%l5X?g6 z3bc2BwjW^_-(FkdD->BZtZ1KZ&Oqi+z!2a%a)d3@5n=09O4YMQlZX**Pb)0c?NK3+% z)@PiZ7WN(+wZGs=<*@_(=W>VBqrQB0Wv6RyAax05gnkW zZ>schHioN+b{ijEfQ~j~t1E~HnEaa6S3EEzx7hD)%lzNnhzh<6%#QGxX)F)Mf*2TM zlN>ST>3a(xYc=q?Y#k-WPelh!<~A0yMCoNwu_IjY=WY&$qhYUR*OTzx5%bxZ>OKB2 z4fD|Ut?T(Ynl$`+ek%=Lhh0B%9qBDOJ(Hq^_y8L!MAp$%IRohvhGWdoXzcI=>OkMZ z>iz`dVzrEy<+#wJcb3fH^`}JFetW!^Jd54nVONd?{fc`sYwhIVCDmqsHIWv(`6}pu z9ZoiuPgEAVrhJ%1L~-^}$UW|z{V1+3cjk&kg^V{Pn8YP?!g^K@3Lm<#$9@d$@|)i~ z4VW3tALk}sx@eI)yzpHf;|NTLr)?4iZS?@I^!|7kaMP4@VH`e9s>1KY6Fk#?nJ5GG z%s(sw_b*^~Y`IMP|CkCX{-HCe$?*V=3}5<8Y#ShHndK$PYH8Hla(*%?1 z#^9kiBBQXNpv)noT%SL}XHBl%Sfg5GmMp`Tur;(rHGn|7Nlk5 zRmq7EzxBYwp|gbgWpYAK_Z zu^s#TH8nii$@rv^l68~X)zfp9taZ59NX}aE>7Tv2ZQTQwZq+P_12?iS&DmF=!o6+c z$7dS(ks?=GA%DM?HMplAMw*VXAK`(ucifHXkXUh-c0Ug}<^ zMVdv{N%BehN$SalLR0RL6t?LT>G};h*KztROd&dbB>lL}Q0K5+@X5fAmXyu8d)BVO zV!Ox#stvto1B1Gy29Dc5ZptH`+Dq=BO70X&?oy~ylrKm>Z!#iViGa}srz28JCuQi= zFjKh3p;Mw}Ux-@)gaP(@S3}()$C1LsFhmz(d!~EOkkPMblmxg(3c@OJAm$adB)eQ5 zU|tjj%(4m#XTJLRAq_Kjgaa+oBYga6CMNk7HPr|tuY1w7dXKvRteBYe`F_lCeaxpA zfM!!oI$kA`LTi;BDO+@i9-5fpEr3K+P9#7M4be`aCETJjEel5TIE_4jJZ%$7ua3F& zIYG7z`A%Zer6x+$k!eD%H2z6j+JJn;Ag9rBY?XfavzNZamC9Ms`;XVq-L~xbC z2&8*7X53_?9{W_&w|ZeSOGbgfYR}4FVY-aYHqw3J;!dbxpp$rrv`j=|JB4Z+Z4b1; zj5FgAHw*Mw&SjCe5`U8E@T{#^Ns+=SR#pctIsMQOSj%{8#4TB3-ekEu;xV*j4rGy! z>j4Uf=}kx!w6{mzyfR61JT5fIr~R5_N?g07>&kXtJP@{makYLO2@NY<)3}iUza5CnmzgRMO3o(E_3XYx+SUD=mPR-;TkHeJ={|jWYon0I4@6I1rz92-Kt(+@6aWD?+ss6`PzRkD8{PH3Eur< zkb^C1ahYt%q$b5P$|}kWQ+tf=AZaOanclg~z~a&T2+OM6D&tMIY24d?SD|U}O3PJo z_1G0J)2R)tlDi|IlT)^fQH;_%iWpWvj1h&)okR4;jDcG0yMDf3oRZla0H-sU50~OH z@iq*f&F>d63%c?ldIi-)SkZpij{Qu>O)FG7cAsyb@3;8*g)pzKF#tlaaV_c+E=cP- z2-2Y`{x^(XUyh$qUG@{o;|57TDS>ncBekS)5+0y$V({t*6L~YgZNVHQJR-<7GH-79 zkuz|ILs%Bs7fk}9aVppwGD@_FRX*=SYStQVrqUTm{y7b8KJ+@)Tirz2uL)zhZFJ95 zVYp9Ayzy~qBEXXMY%bmZ+FbQiq5(6Y-M6S%rsP)%!orX9{bL=De2(O;q$lOHsCt=2 zCXqBa7km(BxnO?lC$XF)>t)W#xQ$uU_^k=NJUZ8oQlwBIL4p(9ofa+b?$Y9=xECoDcPQ?K7AO#kwFGZ)EAA8x3Ea@< zC+~fpcilg3vXadC&YszmeI{q+oOR~2dFY95A;IqQ(M~k&$lY2tf5*;d>sd(f`c7^a zxp(1dU$U*(F>>ukqgP!UB-lPVnoV&d#7B47ZL6~PA~86);-O98F4#B6L3b>;=({ZQ zEgnMJ1YEt6LU!>nsb&kUpup?u>!4%iRV@68Qs-fNo__oLVOL~?B^a~2h3BXWPqQ(&>S zZ}X%+fMtWvjfSJtM3%cS1B^Q*hNfX@7_kIiymWzh0W&a&JP8yP|OE#qWnF|H9@ zOe2*#^yUt(d@bjn8fZt-Ir7mbUPY-w*n`kEhGjxm0v2il?vbEBP2AWPhh(03Q6^YN ztkG2Og~UIX$0yZ)=a7CFqBaiPioW1};o5zek??zgz@293^8Kg*S4f(oDe?$eN#wgC z3MM}V(0((!H3n2Cfl2tAsOssi;jk3uF2|OCZvJ)cmS}XTq0TSyD?Ca?OCqtT zChHQ2uaM1Zq|vC5aqPBy@fv5ejeUTw@^9%T){T3wdQ=@}1040bnKP8p+*77&o3tId z4JT&Y=sP`QUegVwssR*DaKnnJm&LBwPg9ihW5>yNkNoSLnQt*h>i5HOUk(%NcXk-R zrk3*0`SId>)mP%p?<NGgokIJ=nq*UrjzOyY(5{D-&0ZNpu2Pv1fyaW&$`~E= z6XDKVTl^m!HB@vR#vHFxtvZ!-v{YaG>Uyt619GQ1<{BR(DjvMQjr|TP4bG2~uL5le zoCR_QN)TRHzOI(b+u!ms5rum-mkpm+jB>FraY^*KzqAYl4=TKKf5NOZlAK*X+o~}q z)|KEYCBiBkfX4bEAjj4MkZ>%Ma-sEb?RBC?$s~FO&J=A$sILDvg zH}6b7sCcB4m#*JFS&mayqL7B0;X2fK{r1Zhb`ZR zbYD(ObFq_An`(4WksSU21>uV5LI38ras8t<{DNB72+!oM=|U&NSYDiaNDbATB)aZH zg9r~=MuBA-aSZbk^i8+i%k!WHM$4Z>w~9vjNp`XwkJkY+sP&a1!sy7US8lY^#%1~0 zkf7SW_R6H-&rF9VVNeCNHUij%fW8KF`W=@H2;(?5{Vjj96bzl=o1_o!IZq&VR!Mi= zqwPqoh_u_usBtNp<>wb{LMBA=ng5~+k zU6b2eX7NnQ_DW-*a;dwPg-12rG{kgRtuA-SgNjTg{t_uV+ENS;1H9Y0%Bv5cR!wz0 zx@vX{6?>enXq0?zf*C1B`jaY8Cn(zMT_Q9%>``P~EA9!)}#DaW71&E|wbuWr*F(UsG z|A}e|iC;Q5$ir|XF`A=Y^%>8ddR1=M#7Gki)?Ip}Y~yfYzn)X>NVKQ#AXpISpS^hi zGxfdtKIwunzH0WSodjc6jwmUnP68*-QBRjQInPJB^Ike zGb)7Ok$M)td_@KI%}>kgC5)fGEQKcD5A~8*v2-dU;hR@oTFd&(dB9xf$U7GB zbSbT?7qY_l=JvsWC=Wxf%-(JAXC7?D%KNkhonbzTT(ilwMS&jNIeZTC1|ALg$7n+Z zz>FKLT}}ViREGJC3-g6%s)zT4j=44{ERnm|+Al}ms+Af)?rvxMZtJ{pEl&;2yJY@) z(NC0!D#6}l#9rM>H z(wWAIf&y+HVazatk7RClw-c}FZ(U+Q!~8V1sTL*~+^&Z*h`Ea9TyRRug+SHWZCjJt z_k3fA@0f5^`oe{uu<45{J&A<0)z>Xe;bNevI8aIKo?V0WV9=xUqD1*;F+Votwld6YZi8v0{Tkz8>3 z3*x#xsW`qQk*N0J*mf7fp#@iY^p4&3#L9xOYf<>Y0(9wL1aO_Y5q+G;idE-`R|o$lju!u?{=x+QlD z4ZB=b_j%W4y?3PWOyI8&a+T6cvTe?0ye+1LF;D_RaEn#rO;D8K46KYw<(Oidxn8d^ zV)bC$&2M&o$-&Dh^z$CH+qh*L@U=yIPjG>00bS2xfp(T>1$Fv(uJ_IGb(+^|BIJfW zJ#&m|g1@V+Lyh5^;DZQlu~O%*Y0nn)a+NM(cXsB4E4~(jolEbi?QcPHJC}+5qB=a8 z?$DXigs$=zo2`ix8gI15*S9VVfHOFNK+GV-4UZAzxp-*j~w*#{Ro_WShhZuM?D!DL5Wrk!TT|V9^_Oj8Jn5 zra0Nc8sJn6J5jSBJmesSfzCqw4gxHfRB7YO``mv(Hd{s)ld(R*$+RQa>+94dAye8H zN0JT{rLkAB7Kvky2UzmI&H(ZdevSPH5$hJjt=#GT3~or`_h!PT0iQEt+LTv4O-^^W zsGov{ae;jL9@*Bpj(Wty7Q^HOa5#kKXhlTw{7oLGNhJ`^lREQ z^ldj9{kpQje*U>8}+&JO{Y>1Ke*?=+E zj|VKvid_Ad8&lJpai(<)N9k+=<^9dXO9E#xcdroxLO~xFG~(CFcJSsfckyoV)-Ee9 zzM+b1w-y01D3eG84oZ48;!0Qry`@;AT%^+ne?baqN)*t~HP8o6C;*X_6^GznFZNQ! zK(1>*;*Fd`sliR<%z^Gbcvr<7@fK$pJVP?`JrlhjL8&RzXsvp@NO<}t@CWaRxjwSk zMcPX`jt^dFYn00ie+0O8nsE_ZLQs|lJLx@Sk^sR;15@@E;_Yeb#}J*(HeK=GU`8$? zQ=Kfsm-5Fi*Pa$Xfe2p)kWGx0jW$e3?ht0&U`x4=ZGLyxlIYJSgm18I8<+jU2;gy* z%>-hhd3#IFSrngG1D+4ehm~!A440<6A)q5U;SO)8#U6SgILqiOP2WJR7{SRWd+nIE zU7m%upWZ~}B5L97FdJ+~Km zVn?q5k)18u$bd*JniR84mrsIqJpVLXBu-BEG6b}#C%LKg|`T4--x$CzJz!{Xx1baRI2$; zewb1pQe31{?`I_w>NMf>&p{j(1sM=Y`bMHtVk*sWob8GZ|H3?UaMn#0kbl)&MCXjz zAF)4?1;}wu8yoSbe|0J!0E*D$5^WFUpx)Hs7SuMU)_&cUxv0`4-(qszguF`26M>%a z?CFO=t4>cnd351;d|XcA?|<{z&w28ejXxiH=6|yQ%X#cMTsKZH9Bc;prCzc_h$w5R z&?ik&6;0o}=(fnq8`(hMHRZ0fW7scy#W9b*dP5%;^WklNPOylo=q+1;Ng|ZdY3y7P znG<4ZV$E>oNNSq#DUtzrT0(q6u%8V7{53JJcDHOX8p-68LtqD`COW_ELK{K18|Vgg zDZUsFza_D=CTglxd+ErS#-#t-Fg zJgkvfN;OOW@mug?OnS~E&}GQT4P{6>eHOiQgjBF$GEif6KEd2S@a@tH2nrkFw>Iqz zeBuOx4$3RF5cCt71ow{62Aww*Fw$L=i?FtOiXivWoiDho)=9!FARetZGj+ z5DBD2#G5K9JzaE9p;CE44`qvajwELBnwFlZg)bKK6(86cjb&z8Z@e;y5&sfdfi5^Q zthikOLa1#zZoMRs&K@lAo=p@9nLVU{8r=leNAdiS5xxP*_RY|LVH-JJ7Tu~MoslDX zO*VADI93bh^Bi^St8>Of-eV02me8U*Akq{orzCTWuwycliOt2+=yoo9>&uDF8HC&pNF6At-iJGgU!`8{6e{$oGA zDhN9X=iE zETV}tbZh@7yfGHs`XJ?*E@f^d?}Z)!pp!LYccm?gEQZ}yz#&jt8 z&bXXTXfCR<&Gh6&;s%W;N$q)~GZ;KseHZ8!7bwed@+IY|4m>1BZ@G2eO7f&Mmx6cD za^Ts6qd>JMP|aJa6Aq&xz%Pp6A4yUOaVN_a%2m(VO2%Nr%SkV~Rr7+ZPQhQrN#SIX zkS~|wF)QF#(Ky?_0I)4jX3IMB{&-o{>E@pF?@Wd7wn4DP`Oi|u?^wM~JT;|l{f&^^ zA-L`!xaCSMe$Ta4Sa!qjYQN9=Y%iHDUGthR@a({m)6*y0@t+Ka9i7JqvESyFzCzsF z=f&AsC@3UMb+_oURoclARzJ~TQY`z~0uM^(vAv=~vmNm54jg~h6y83b`J~`5&t+|C zQGv)-F*owM&%j2t{s66o+G>lSKqgo2%bQEmRZa-CeD=pHHTXW!50y%=mtYRP-Au`s zu;0z}do2Y-TE5ihF_UPqPr23em&l1Au4T`hq&k^|JKwGfu9}wJGC#`2P~g;rlT^3Q zlm}|!h=9;OC9ICW#LcI`2#;WRez_!Nb^O#yU5r*(7Swe~=w1|xkSWuQ+SPmG&@{mt zGQRY9t6@=6aMFJLllp=`#;NShp1>B_h!9~P5o*DIVrb{E$dPGx!9%ZnOxi6;9Fft` z|L@{**F*W+MBO8T$*(#2=s2`PTU~Gt@D75O{i|;`Wrd%=xey$+HtYe7v|ELh5tC@U z3dg&K)*NCrkkaw~&8dg$;fOZ=k}hiv@C&!3SOP38$-E&o>`cT&kBN#Xk@=j#M5-Lk zujDi8N*81yO2$iJD7~SjukiID)6+LldAB%|r}s>mweLSD^y89=lT?*|nfQsiAjao; zBU(L4J&ybHc_rlnXF=U%n9N)a`QUlm=KI*y=S4ylCS3zEheGiQ4VSP$VMf_6x$A{uyOSz`{ zK^JsN*sSi+-Cj|p=gLl5b9#$&-8u-zBbn6TzC=pj%jA7J)2oFJoZo~i?hv}S5HO^R z@Diu)-2{BcuUc+C9u_7PbxBolB6E2mansSigvk?3F+|iHH5m1o*q_Lhng52{#QWx< zR_rJ3&PUx>qJC3XbC(9LYg5BZJ@cq_fi$y~bFZJjKP1VN&CHB`l#!`fnt6;foOPMk z8mRY%>_rr3JlR=lDvqsGcxr2PzEP7Bu4`o`sq<>^9G!}+;Mu{DX_)-I{^Tza;$7H4 zh$4gxBKUI~HM(ERkscu$1QrIp5ZKP=NW>Xm;(c)mAMyjr$(ye^zwKrOej*t%{f zwPKr$=mwBo;=9Kl9- zrOu43A9xi7y5(OuZDeTnCJ21N7dYXJq36*gXw@RXsaQdinzT<%VQ6ubxhq^N9Q`mT*WeZG!5_@erKd&e|GV>^SkrIQQfBa(r8Z&i9oqL;%j2O3qTaESZDp&w`4-@U?> zV9^2}hzP0jE9URBfn4dC8X!*e$>FuR9{wb;C-)RKOE>CdYs9o z91nRwZ6vSxc%n4bQpiL-R|}uI|E1RW$%A#pFcAp6iXFN_avCugAG^TzTWL*8VwqEk zLY?}$Z`eYLhS-$IzYl4l`WzAkV?0QoElz4unSHvS^rS-RMCFZa@q!DorJ#{b?7(G} z@U<0q8+&y#NYU)WZ7}YaA}NxA;kOdod{~}hc&r@)D&HTc-3ZY6n)e?+h0BWJHT`(9 zA)9`_Tgo}zZ>KF-Aq00Qz;GMvVJBbo4OJz3S>8Ar{%{ubT%QH4nu6O0aJ?5En=aS0 zzd63*HD|WFLg6`AZm$78p&2={40>w<>c>0Xjgs@v)JfE5MOwY&dZfEThscCh_ZZt$ z^?Y%z!WLE!M*teVP-?4Rbm(Rd3>1!M&HJ5lZ1Cq@r`G+ zWQb>w*~`9zp-m2BD;xx>& ztama~Yig*fkA&mQeuA&DEK~eNq3)N|qQ>KFLnZQ7B?P1>NdE1>tm->g6aMZDs`6>^@dYG7U$rGazk50ytxF$ihn?pX0qZm z6$faqVe|{;#jcma_o0@^MXFy}g5-ESUR5itX^zDPty?30=`HznlcC>3@l{sD`?Nm; zXil|P!k<@J64bn^Ac`MLA!DHPBj1?%V0@UXJ>xI?$HzmU;EZR>cuxj9)oqbRN@ zEbASH_RiBC$@MP501EdHyo36q0JNySpSvn3$*P&V{g?WRV!rC(3J8F3dG!9!{Ecu0 z1@74V09;-{K@|3tANLRK71f6kE+5|=cNTzBQJ_#>p+C)dg>m`s=&vY9uMsW?bmt0y zE6B@>GD7tuaEIO%;6qj2Nq5yKSpXMw*8;U+^rs7y1YEGw z8?5Vv%f*Xg81pEhGD6r}O1aoNS)&Gu$`IkEXl-w6hq6V*8I|V2+F2J@2vu=6EEG-` zHD&%FT>mfR?j1?@4gt)^C;Xo;c871h(|Ccrp#Q4>691c#;x46wCoYeQz11Dz_pjOU zCwGH4YPf$gO#CZSeje5~xBv!TR5<_r;AH>_2=ECo*f9L1@e1$?02uz%GPwLpyW8UX zm-cVFzbyWG{~sC<^pCbc&_Da*1qlAbj!*F4KK|nuH57rr+wt-V|I>~i_`hg;|BD9t zhhJWh;NLY~egPriKl}>u{#`>25BPTtz{kt?_ZXmJ3K05Re<+Rje`sDF=BRn$@#hGk xWB=M3HS=(Jv|U}jP*d~IkNwZ_<3F5cPcL&1uRnW}SAY*7jLX6zry-B~e*j|e*$4mt literal 0 HcmV?d00001 diff --git a/publications/whitepaper/figures/realtime_pipeline.png b/publications/whitepaper/figures/realtime_pipeline.png deleted file mode 100644 index f338d239117d9a3a16b7b1843fceb7fa5e1721a1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 36023 zcmbSzbyQUC*ET9hio_5iokN2H5+V*TbPU~%fOJWh14v3pN_TfRiXhU`4N6EeBHiB^ z@Oggk`>yZ({`h9OT(i!ZbKm#5WAA)M3C6eJ&Dkzt{rpgfS45?4k+xd%c)xwD0F zAGo4EHS3Im;)fzF4pDWR*=~Ipr8b_5X!cY03`nG8pi^@8sF7`yVKcUeD_HRcN(&ix zVqj|8gy8n<#IDFlbQ|7hU4QWx^f;1n^i5I?pFJFPiVCs3FIg5=3RTeP-_?TKGOFrZ9Nv4zdYYvMAXMS z0fYO0e`3P57_3Sw+our;ybmg`R~JgswdP!q&-jHv40@`_jDHBX`;OwB?SiMJeOpRq z+m9S(?;!6p)@4K`@~<41QW-C{UY*1-BR#4A#GIGvjYFaFv*UN$R1u^iI9|rW2HG zYA_i-;ZwK??sM|EY5wJqCpK!Jc)%vxf>WIF&h%$@uOE@M+TrtF zrZ+vBD8d4av+5!AaI)Yw%5c{Yt5@miuV|i&U3uXFTMYP^z+j!*e`cQ%TThnA%IG9* z)Vtz{!pM(0Oy$2P46m?kXvlabAy}A|Sn+QkSQdj7NX?qrRjmiI%hihv`y5iELJg=X zk39D(KG`KBX04^eD|^NjW9}lG_v5ApcM`NC5=lr2Q1V-(Ft7{P&iMR^K*kXyp!hSM z?)qs$ipifWZ%9~Ar;rbzd~M6X6E5?(40?an8qlHMu19MU_Lx-5LLA1=DcUiXudkuy zZ(M9g6|sjw^%(iUrzykS;s}<`{@>@XYC%1heAOHXT7%qL+s){4!gOsKkH~@|KMVq+yJi$pW;Cr(<<}0R( zY&V>IYt@;3yM^w2WbvBrojf?%cX@!3P5n9`h?$6<@+$8XAe${KgiN_IA_ zs)iP-t1_MD4wfRUuiig17ji*7u28(3KW!|#ledF``S%r!gO8@(*_n?fB@H&Dr(f@i zyKL>azJeZ`&3`rD_a^Z@7%8zD-?W}?Ev^cC;pp(9F2JwWbL*4cpPjX|$rParODn`P zyDfHdgKqD2_`@}mgO)#L<89tSR~8C?P?}ATR{zse0b{=2q!XhzILlt8JO3>8;AElY zyjWxf{&M)fz;WH>&arPYc6dBCXK(i7nLG<|Hmos)tgOS{VyvN`F+14j-H~X4S=2Ke zdjS@=OA;;%*;`J(6m)f7o!fkU^-_qm}m|g49gnGoOJvP=nI`lqUR#ek7nYSY@ zQ0Ap&D!1Q1z0z}ENGF1Am~Hb9z_l@MA;-cOCo!idsjeKdSXh`L_Bgle%w>xjsO+#p z-Ap|op*Z`6Cp}WnSIp(=VB=D8L#(;vFtmtif6;qm{G!ljD*ep&a^B#mK3dEMx5bjE zqt5Y6{yTV{Y1bT#9}$@9YTD|JIF5Y0_!!DSZwV?4jx=!ab@h=e+NI&q3kr~xVUw17 ze%ATyJoTq04BRSX<2+BGCgQe}y3%;NTm85hz$;R+cv5WN8Z~$zfjtCxMHD9sM|$qf zhl?{QLJkhx)iajoVg~0D?Xd3-j%8|!8CPjD7s?zkdfa|#zHWVm8s%V~h#4zgn@KlI zrtTT#IA=NG{I+#66PnS2DdidnE#xjUQSGn~l!)yQo+Lm0r&x(sq6Jd0f(D&qFvz7T zvgz5+QX^EM?{+}n=eTSu#OlSCwC{zfUPO5=OJOVdZ~h%gV;W*g56((F zN!c2`4Y4m;V%#7I%^f3LpWzaikzPOxz<41WpMa9_cGCs+E+us83l(yqQaoQ>ezoaN1UZe&dj_ z5qu06z1!Q_NkVmIIdBEaReB$g^*q#c>XIA(aHX0P*Z!RtB(svfMtRLNJrWvvdsaUs z5`Y}ui}tE|;!O%qC1G*6d6YZncz92P3TnyEtJ%RI`HCjoabcdB`T+k3CH#8vz|O$D zLV1BN4!tVT%-TwdW4CtaM+p{IE?>`Q_Nodzot;{)6d(0775>yBOssK2&|z*;a3veq z`9WQtIAD#Pho7&h#E$t2JB)whojzrGgT;Mo5=fokq8l58^k8Rs8J!POBenSvle{vj(mY+~s*+i43^ij3yK2_)61i3* zp0e*p*t80rstP161$9u-ZcO2i5^OBb4?YIRNp}VvEM4eWlhpTWW#w{1rZQY2)sjFG5YlcV+b>A4d1A+v(kWg^i%Q$s`2tE;RlG?L1~kX z6y+@^YVfTMG}nU#JblEhow=c{X2`at-SJZ30{ZlE;UM(_)0(EJxI%>`rZcXGnt=>M z6@2wG2G;mPeyUT=%k@b~aoCi#2HzX2Ub*MfV+~D`MU^mvqPPlAO)I5OS!7r`&lNXJ z&u~;rIJkepnVq?BOxUMHLQwoUC({5{%_>`8C3}*V6s;Jp%P?@Tu>__Fh&L6>wH0Z+ zvN+dty@3*){s(zEk9!AspZTEPZ!Pobu4%wq^18~ ze8#_i0R?{d)nir2GJ-v1_#zEl0UzRr4+4E2 z)x}8;u1ov(f^d1u$1p75$9&gcljY?O;!FJhy|*%FGIryU{$Cyb$IJbfSNz{S;J+jO z|8JAj_WZLI0_Sw z(@|}%@Ae4)9|xQgf;9h)_XJ{1e6N}b8?(!o?X;YAcEf#+(&DCT^76K`B6U+*-~4!X zX|wvHI5i-E{QT8%S?{25xqwycpQ5C!rD3MFk5o+fcnF>PJuN36^hEK%T9s=u$+g`5 z(G$UkwJKEqae~2&-JS*TrOF43gOO?hv~ySU3Hk%|eeU(^pMp3}>~}l2%&qb<-p=%V zRy=HX8hWeiEQOJS+7)`Q} z27DyMS-5sE^Bb|tLNryf<2TN$_1wlUwv#>sZzh=#A0sr#R5!_D(++x$0^L27Wncx0 zA+a9JS_BM$*95`}xWsFX!UMLyCdN<>18eoPZ|@C6Fd{89NZ`sbaVY&o>v!&lslW31 z+GFzx1222_Cio}v*5aO>kGijn1=++U(MM;xIF5Yr7L5GNpDx3I)%`#vlZ{qs>x8Op zZ`4Z{O19+fcrYFW*rJT@!pQi?=Pg4RA~TDCOzfCFT!EVM*8G_0!@m8Dr;b`?4N_O^ zAX17_DSrVnf(aew9xJ{$Hy+<(#8H*R6YOU3i{CG~Qe9_zOH-|lDx^>-$`Yj>@N%%3 z_RdV1Dg*1+IvYVoq8-vutyXa*mKa)wCiM{1R9Eu1A<&mHu=S)d31j&VI*^r%w^3>! ztr!9G8QVeIYDd`p9Lw8K^Tuaa6xlpaj!MgtwE$NcD)z6ge=QVI|6$E6;Fz|G(GBhY zW-On2A`!e{{k{jm5fU zqojD&lNglQXie~`oubs9m-!s(n`kQ?-Fx^VJnHXmkcZGnYT6u%>nr{gt3JC&h+XLx zwH{?+oAq)wRZ%PS{6nF_C*^oKSb*Z@m!v?h+SmMOfgzXzr-iCuLd8%HFqyGQ(BT6D z0>=w6?hgbYL47zorQ{pxP-Q7M>4HT65+^Jx@fyWw%K3eMQTX2wIUEy$KMBt1CpQ*Y z=8&?hV)`9V{Oa(2ZGVa_AqWcnPDcagiBE9~c}vS3{Y3Wq!^?^>&RmDC4mLG0B~t?M zrep_H>9IzXk6NtdKT$W(WahzR;YSJn2Bt>%l{2iRJ>)hNRvRzXpHiDQ7I5lh*w&y0 zro36(dURh%C3akAU6(yerH_o6yx!^q##5tsZM%-$n2DvAa(_*CN=8LKqVTo!YX7FR z7Z_afOSI#q;<=hQc^=m5x>u{j@U}F;1DPI~QaPK+5~=V( z4O|=RPzCUNX^~mpLR@+^McPr}k=81$0^JHm2^LW!CF;M?V;*W?idXG;hoys26}hQS zKFu75SVVi?E3w6%JoqLd_C23wYzKSXO%i zg2{iVzV*-?EBzK?Db%6LmKwC{)LMB9Bkd=E_SJRx6%+^5=-Gqh-`pp_btv`g&vU39 z%Xho;JQuR@x?MbU5K_Kb42014!x?2Is>}4cMfV;UEdzA#!~#cl1L#@ zahM!#>sisOz0EOTkF7A!f*yT$gC5iHq}o~LT492Bsy%5ki5J7{-$L~AzmA_I{bY`V z2mZ6qm~Y^mpHhCKX(=+Z5%Del1mP;Ilfv&(hN%CjgM)&^SYohGC)uT@sSI03zG3OC z({&#X zmvgWYJYtfqt2cctGcl+Eq}6U#AaT23*C^HXLvhMl{G+vBihrou5og=AOd_cizL#wd ztYq(unHD{$BW<7W|A1R-brZRZ)6sB6G&>f>(h7k5@BJNG44jygw#br7b(XqYYXcH0 zD95<>y+B%(nI&6FLevlpwM-x(`3HE*@v9P6zU~DbUndeiKc^erPr93Xkup<_HOv*@ z%naqSz3j|&5dMmObsXDai-H93G+@IHjF9VGL?pSvIOewZm4f(Zu6(INT$907_R9|A z-dN^#_SQ0p=08)l9Cyk$eL>?*VfL~FP2!In*wy8kHU z>pt9x(4lvFfAHfGlSZ0;U0^x$KW(UM6(f2~^!Dvnx)ZfFReIw0dC^cv(c#A3yWXd) z@8m%`Q>TxdBx;@@p8L#aPAo#?4MGV;>MWUGJinY1*#o*VB*zT>@tWhEytvL#xzN~| z5ZH64M{&6y7|&g|&j?NcMi@TF2u;&i=!c>K4I*?+M?vs@E!dX~x-TI|@5mf&L=BE(hafC6^}E ze->xs_w`(6Ymp%E_O(+kD7-d&5?c3OCCH1Ev$2@zx%nKa9vhf5`FM#8USZ6lbF%lf z*>{sCk25Df?Ryn;)$>Zx>pl>A--fZ_bdYH8p;uHTuG4z^EE{|*I9I)EFXBanyHBa~ zJ%JPON3CZ^pOs}V4$2`^S$Ls^bQU*BAU_8ta6j$~eReieneNjy&bcUxnIm`9&v@px z9*2+9d$aLLu20jaaTKQeAHCW$6zxD-|G%B;K#W}6{QKIz?^ZopQws}Y=4RBX1Ng`w)DhF)SKpW<^i+ts>s!uw>gx};=kLh*<6yb7LIYzIfvA~D|c!4 zuO-WilxMd9Cj3P6FyCiQci{sz`Lp1-HHO64rh{A^-J`L24$mD(0)@Aw z8-fani*N1O7(6S!(C6hLDW`MXXx?sn;e}~?>%gm0lDK?Dig14s-8>BcuZV^suNQ)x zsp*6Jr9r)Do7rz7k)h`;1cEQ&$SYpfe$rW_TeS!pb7dOjqn`|zkW zU2nnzdbCcsNM3>X*`411YthYV#ASOyWBlk`A^Ljcr4;;?($=;f`Wi?#sB_*2PXMtL^#tcvr+!MN^Niio!dhvwL?k0kmEM|7j4 zf%Tw1@V z_Uy@s<4r4^s8ZJ(IpdsoK3kk7-Ct2a(S7Yft8 z7?wp(5rW@^+`+&?BL8q^2o1Uxvj9h;8BFo(7oKFrI?n$3K*bq$UMYq06qnh$E1l&1 znzRa$aoPZ5h(dE6YpQg2kPUCV_`8{&LfqrRG(-NB5u#()wac!%5`)cylK427iLn{5 z&<}ORWy^=Iz8?+3IyKpnVwH0}&1M-!$cV&?PQR0qVIv{zE#mrGk`WbS)zuLS{&uRS z`tmj3=5pZxr&=yNTy1#+lg$bqw!3 z{80W6S^K=)nW{w%?*js$XrZWqBbhaNv@FE(MwbAIQzMTyWu!A7raIOkcPx7Rs-cq* zeiyKi$fF}62TOD!7HS|M4#oX;S_@);QhUjmq~r(& zuYBsh+YLQLGglVFPuc|c7baN1Sk#LaI?Q5O2tev%?3vkHQ;lmc)2?!wus-JcZd^&j zERrLXRwVErVX|W0r$`o5gz}B~4*MYA9hqW5K`DUNUrX{^@t_8#!TTEH2+5P7*m7)a z&8}(->Pq+e_8*4D$m4u-S2)bC15Q3F)bfayB$w|&LPmx5L(C9lURMoOVwNEh)QYhs z0jzyAd5C|5?_usylcG_RIyq)EN;HNFD&4MFH2w~@$~PeVy*W$Y3kcokuHh7Wv+=Rp z1&aXf#ZJdnmNGHz|2(hI)KjMJU8K#*73SGcOTMX-q%y3N7mK-ij3MlaCU=$aaQ*tb z0e#gz9(^!ad5vEgP8$&*^#cqyROw@}%(UXYL;o|=p;^6jaIsq{VNA7}K-WgYAf|n? zm^D$lH;Y`Im0%*BKQ9EZk;u)6;-52cZ7luV`uCEk!gqr2Vfq&bOG zOGB>0RP(`Dq2FB+N}T9X1IaCO!9AZjtG;i<#gx=AvW~?Q+CsZM{_J{@sVEt!n{Xw; zvn=B2-~aTokr1XCTfbBljL8l8Xd#7(-<;RLxRMv4xSB&+4jELru$l-#FM6D{lxBh_{q!f_7{2yx%W$KDd2fi?X*-Dw!wo5il)J!R`u##oe z6cMEvf+?G-vrB^5o*w1-yXy5^+Q2fPLur?!{Ny6qkEHFtNxzqm#uOrH)o`dK_YS!) zaOL1DgXYv$N9*jhto?xSDVb24D3Tfn`)+XCa`YO0Httns4bb=YAr2|=~@!6#Ix zm9uW?m8iC5P4Y%}-^EGkn=+%yNV61<>AGrq@XRGudL3rl_4NI;cq-T0EjBV@5Fe{b z%33)O#p)kcIQ}G!6pux-Y|eh-%aDAbaW)u-Qmim3M2T~?V1i|KFX>@?nVMDYHju1@ zS~x!hMPqjPvB#S;8~(JG{~255va>fuz&4n#NkfhY@3dMD#!_O{o7{^hs~J8%S>n3|rkEXD7aU#hQAgxw)jmLXCRN^Or& zOVT1Is>-t6Wcp|9VVxZkh~S|IDPP&!2A*qh*w(o?PLQA~g_Zi~6c^`olpG;ub#v2< z^g6h`XCj5VugUXZ1AEJ_d@dpcCke3Ps~N!YC){iDzb|zNI(|CSie|!hpqSj;7Ld&R zHJ1DFgmx;ay{EY8pSrCS3v7n7tQ0cKV(J1~l?UyWR z=V(743k&EG=8&Yq>(<~Pa-US$%@>;s>y(8eA3Ni+C<}d@-^<*q_AJ)QP-Zr>%->UH zRt8KM(unx=bQ^2v-ZOfGVH537d1YM7nBsyNYl4Yh#RPy#kuoOkT^B^+Oo@VjX(se&M9cwRQq~bDM|G7W5ThUF7<~UHVjUn#*0{Ei5kC@D5Zz zby@YaR&}|$u_Al*d`Y;A)rBS_Bj5cU;>O-4l3Bgw(=()Vz5#xiq2+9j)6u?#bz;{m zRS2(zS5F6>v_{7qK}KvCp%KORL(x<7fuNmh?>lFo=L<{wUP(4V!r|~Xi*vSi_N!Mq zyXQhg!S8A@xhuxPf>ok19F)eE1USJF4NGbXdY@1K1pN&ix!|G^wo1wQyBziIX46wd zpqm`~?Q2S)bv-svs8`w?ZcTUAnl)&!9%M`K&TJ(g4~`T$adTg6>FX7JS$_Tdbpplp z(GRq)%V&+&6KJaiAwE8PZGT)2nXsWza?!*>lJCzigjX*Sm)U*0Ng6MxFk>ev$@wRA z9>%E@kD2W3|7TH+1j72j@+e&+#og(Efk!l7cV{J* zwRCfDR&QcKeOFQ8%pNne;hiWggjUpsH*LnUeSbr*2)l8)scP!`kf3BpI9H&I4MQwt zb156+n8IbM^d`xFLgy&BsN&Z0qqNv1)4}$%-Sh?JkB?+;PTt0B z(0)2Zetfw%TBAKwA4jrR1;`|eV3N+}t;4S`dtAIKt1friIqj-3y3x1#?k?9^u}-q7 z96+-~z3KizWGJZ6d)3c-q!Y3!u!2aMde4=gid^o0`12x6dHoF{yDOp=GdP+6w8v9p znGF_ekT;gJA?o_>38g=H5w4l&y%GPXEi>;0GVHzuc$m!K4y`nL6jh;&`~w2QpDJhH z3)NM{6x=L`hcUY2NtNEiX(RC?al? z$RM4T<2@D?GA6uT^w+4+qIaUa_Zj(?1n=7z*h8ls6hB%rl}w1f79A`%=f39bJH#Mh zt3?eQ7YZ;tA5?|qD94LJjJk>SlKYiw{|ltZdIQ ziwa@vZ!OtYoLL<+Y?3ZH>6qgrCg*_9kqzVwD z=tl#lx!lg|O%ggk6blW-|CxGVCH;X-vs{YXVP-wTHx6qk7=-W={2cX6=I)uu`d&OE z;Fa_|`ot2D@H`6EBFF_lgh0(@YuS$~FW{EMRul>s#HK7r8gDVXRZxpi%rMu)mh zK&cKshXn~$vSs#F39cBeIO1`s4lJGJ1UEnb!MS=nvqKhN;Y{RM?X-)KYp#Qm*c}wE z9@PKFvH@9*SZm7mUHd%v-N$-JmM;7$1poO!S4D`$*6ADc+vr;Y21BYII9Rx>v!*T{ z+wqy_)SFUVGy!o4a1K%Ez_)o1WG(|h%(p3xe{j)_;sn4*KU_)wV$i<=l^gSg0?8aN z884fFJ5ETHe_7SEV#hdq!~uVn;&MR%~KQWuu0bguP0a;=ezdBfxlVubr;M6owyZHGsiLo(y)?3(Kw<&jf)Yw;x@u>KS! zA8)hi6CzN#3m|l$P2iO;)qRLO>Ubaq&tHKFL0;R)kR$ZJpnIjKRc3wj=xdqFLYA%Bm6S8sN^KWh1Exh8GZLga{|N8>x zOJW8C*QYqEH$)>cs218_-9-!{dX}0sRxUSqtJG`AeAbqt;4OoR6>`@mCdv)Sz<9q>5@4Z*&}JI#ESyUw4Dgq?C@#I6aP*lr zhR_UW+U(f8>ORNbS-twz?>N0@H`yLPNm+6cezS+5E3pq(r;yqYs&gO}P%={nep`o_ zbIM@B%CbSKQkpK)08EY)^Nd^XBSkU@L_D-!%_c$fyGz)MZJ~p1-4v;u042gpHYJ)n ze7#XLs zu-}`kHhbo?ciuvE$b>HJMP~O}N>HNwiNL<)dFxJd(G0*_j6^vLTTN?*>kq`3Rh*n9yJCeZ1k9N)F}vPokt5Fl zX$)Lcp?UQrd84qw@%qcmU14a#J{5K}Qu~xvL@4;#^6`=Do+Rf0RC?Rm3TK=sz-BxF z)9<83HJgiEq0o|L%m54nNq={jxkzllq9Zjb^^b_di~6Q4R2vD65&yVrLY@gTr45HA zc4bMr03wVQn?#8~AlMo^Yn6nhG-gC~6)HL;znV-~%un$R17GYDB(z>0ucjNUvrtZ= z23j)G8`Gm2t8|LlFu;&5f(zi8Lj702%Yvp8?|6oOX_l42=xY(`S}~>ca$&#RfeZ431wC zW0Qvz+Uq*~c_VKmMDbWsvxHA{(vTNKphF;0i4r0|S>_b|O`0f*4wz~@5FHbEX)M1% zzLb|z4))EVjwf%-JqI;#la0qVnpas8Nwxba5rN`=5ejuU8WK@_50_G(D=}W>7tQGS z#pOuDq!aOGn{7+F4I3+59@ZH+Oz?Dua%fvmWrC zg7}pVR{Wrzv9ze9sBCwy$N56JOr)}nIQTg|!RGjDuy&@@`x)698v3t5gzLImlO@LB zS#GMBU~N|6Yt!Q)GZD)VusJ%i6Wfc$1DhXqIHvKhXDBFyMyawL5>RbiFS{f*oUB^I z6rPiq7#!A9S+EChIO=MujGU@jDxsKN$5VWX*GRV$lzs zwHzPKM0+{RP(DQPF&h4T?rx!DZ;1G3^6MyuVS(~P#H7TIwWljRSmC5HywT^p?BsYs zE&ut3u17X^6JyBJXKZj<=YubN1X!V9u*cq#_v&#`euc3Ok>>24;=JC+Txn)mOA`L^ zOStGTE|Gq*zUb(nj8s8uu0FF29Jx`snZRsNX~n@Xw*JNTvze_k3Nt8e?4sQ{{qt>1 zZzTibc~W_Z63=SU*Wh#caTd)~Da{;x)zJ)R#(9-yji&nAypC~UO!JYdnnHfG2SQtL zV}Gh@;mY!EL^V%V>Oyone$Dho_M^I;s7Qhaar)(e$R`1CW1XxWrA0+yhyy)%X55ZxzwMPL^0n<=z%^6jGm9Yg$R+B(RZURaE8&klD z8c6g!KgE_vla-yRBnXBTv2VrOU?HPgi#?iTEUBk>Qc7u91Ds2em5h9>3?hao6pVax z`P9g9{q>~M2~lGfTl3M++Lu8C;c;ZMyUpWiiHT7tm{wdB+;m`};Pgy}o zkNu7wtorQ7MTy)`71!K_Z;vKajv8=3s|QD9#d^+(V7U0tPWB+)BSyP_Fs&?qA^--dl+R~D*BMMFaxKQ2NjU@rZiI9M+5(8uxVXiP zu15Y*fJbOG2iWC|`>{OH%=)bC^XN=BN>kMl{+NKloHVGaILB~zrJ`6wU^EmIP$=SJ+lQ+i1q zo$2lW<|S|0S@D|}W-?Zx2r=@&%ITyUMVm48?wmY$TmdXcbgj&VfU+sxUJpArZllRk zYZHJ}66235r<0oq&trR?zlWilMA+GN80$HQ56tig&2W5n{HA8fI4BM)us`?zP%2IC z$wTA#H--F;1F5c7iXlb>6xSIQE>5hZ9+ptgR7^_JDli53k85qa#*NVp;EivMR6pmX z1$TVRCT{p+ttz;-eAQYxQa=m7Rbby_VY0i)sf53IIsc3;F2Rr11`k>?g(` z`&kod*+7F4>JIT$ADiDr3$&@eEE-OHd;2uk25R6$2kZ<+H&JWJi^~r;ZctJct)?$M zf@ic!=s+<9b$629eSERB7tC7za9A+-1+jQEIw5tRUL)SDnd1k0&82MS`4E%8FU1&{ z0o!?7R*cVDwD$>_GXwqu+=4XPKSBS01q1&90HjZf9t}G;+1&CAxBQpbnb7La4~$^! zTf_l$zz2o^68*>F8_e$AcycDSxSx|w_G3Yy>HqPg%EC_&3PfHlT?EeO1*g?R#`{(} zL54%8?Lz*QZTHJA3^Hf~=&Q%E8S7+OPg^`O6pj~Y1W%b53)UY!n0+c`^cILv zfYAcLwjlF&zl*dq@%yWb} zHSKMp{sv+g1E%@9*!0?+6dB?F?OO~cN1*;th5qghVdNJL0s(lT&wv|%R2_puiBD^Up`ORQ3ZXz#Us{@*48 zZB+#ZFz3BnJ{+Fn8BT_kn8Wt{*mRJM{G`&9iLcT6tNX6=Ms@VSk`=&M->5vpU*T>v z|FL`)aiLl1tH0ENLH7?SX+q{L36R+H>1j=m&9AIydJs+9h^w!c#nNUa*<>BCn{B<8-R zbOfBvjoW8^XW0Z*FC^zt|4A(`ozH&f$N`m^yeyVV-PjsgbTaWMZ z`EmYBDe{B%a(S-@cmC!%_7tjF%SXKV?cMgLGgdI#4`HVpf_F2y0R-i4sh=AIrG*!! z{_|tZ_i6E*9MkG+qnZ6lLk?QqY8>Z5ip>D;%PKUGUr$X5>T%!s{6|h+=5g|y&whc7 zYD|@r*AFe1S?~XyHaSLB^U6V?&U`va0g^}i*CVeCEI37?HRzYWi^#WFybBop`GCXo z9MQ*%ryZcMRJ3~0F}@iYTas@oD?8CKiSZW=`&F5gtar_P*m3p|4x>#L$Y$d^7}HmJ zzlLBg_s;aV3w?HAY57|^=8=s3vC}eugvKtDk{~a#281GuGUd&(M>VYF zj>19ZK#YHroeAKmhnUOq+LYf@+MJF+iY_+&Y!P6i0Sb7g z%{i><-D2&6ldMSXhYP* zovyEr2tN^xD!qZV2JbZmr}pb=r(!Ff!FSRp>(6Ce4lbPLuSVQ=j@B9{(p^1!v`@iz zPz*b;LIGI!e8ptZ=ek~NEw!)8X~=5uy0T8!P3eXj2C6|2cKiLGNK7Bfxe;nxzKnyR>ITvI#hr^hywu*`iPj#(hGz7qThbUE*sTDh+JA%7qc#t5MgR$-8yT-m} z)zZ~&OS=y7{KTT|e%%7Hz=Ua_^V4x1g6#YCvF}05)n#SI_bIC8t9M?DNLX!xlsUSe zM=si%#AUbx34wi=bZdjeu>5X5FCh{V%VwvcI|0CefKE{$+^Ws{PI_X*AjkeU*5p57 zV=rg7peV?4h6M%u_x9&QzOJ;0*(^$Ch>-~^Exv66*RV?nMwqb|k}%gW@YGukQmaPd>cU0`*FSg6~vMvWu`L^+!`Mq3-F2k_13R_GnEG$Hra(l2y+qdunC5K^nF z81m^3)~1~mi$L#$&LGv#r@T1n^b?`9_ z68Qba)U`7#GSQA6JD8p<1V)N`450qkZ9omp^CCPEcMYRT>`lymqw8&9R`eXv*Ue_H z$ZR1rO!Gon?(`B%Sp}t`*$ggAW96SQLIGy2xKE10WYR9rN-A>2(EqIfEpM=34O&TLz6FJ zYu@#pK-S;EUZwgu^OJ0E0%J?_(Mj!I6|eAs#E6)U1*tJGT8X;lO*o2zGMJPRYz0y_ zdvl-o)%TbyuGSu`Dz%q-<(%(HutLWKfI5NTu0UFjLNfGmKa1z`44#ZY0Y$<{H%~k! z<~)-uGj@JNDG@}O3}k%c`%t)k0bF}h>=m7P!?%w7Wu)LEZ6YWAaBSWOH+`G+ePID7 z6kS3ECdCgT($|Tl=DioQNkaA>Ym5K{B_^@nnsQ?-sgURrC%>(EBUJ&!#Qc_45k4)B z``L|R%#z%IBstQ8JZk4AwH)^R$q5+@ z3fqwT$+&#Pw7)L8Yw_bHOsk(gfWmNc`Lz_9BF0acC51T>>&}hB22(>aD$urTL`tG7 znB8pTAIz&j#ba@aLm$rT%$x7XLOE|k1r$FZ*hQ-Tk0+mfbJyUm689&c9usvW9*6C{ zX;D;YSAJ*Rex_m>yC2xYsC}Of+hX%1m6jO~CN^MK6^FFa*zqNIPP5-x2l2Ew~^`b+I&W`cA z+aebrb^Avv%31HdW;XWdt-Ks1k^a_^0n71k^~G(ph+LrCWyfGxu@4PbK23Q6)!HkkB*zcWWy#IM)Sg{PkQbAR*}jyA6oj>)ek3| zqxA95WNSEF)^J+4q%b4G;feh%Togilf%KqDs;0G;Wv#G$|0A7N6GY2s(N0k#zjhTN zgU(7&m19SHbU^&R^>-U`7vVWM19e$)UFyV=#od}sYA_( zdupTO?K^u}Ddc}{IqCU5vbt*)v%U2x>9M{2#mvi=)FV;z*7p4T3t@|2lj!es{s(ix zJv{sA8WY9OK~^``yW{$23}LzlL8^@VW>{JqPhbT6aiUPuh@HXmBIh5(Gw~fW&mb2d z6EnsauB(}|T2mYWaz5RO9K#1OWm(m#R?J8L{y-=5p!Gd`U{OUx0?z1B(|OAEUrqhpkZmqFfXj`#APjh`T(voo}rD)eAw zW`m!NvkqGXJ84!xnzcxAY6IaE=F5riGy=$akIv5x*PjY%{ubDr(i?UcNwx5#Z7Sar z82r7K>sjGvfPWj!p%`+lGhb_Kgxr5V>6gKuW^Y^l$>IG@fUA!f+oW%dYiMM%OsmBD zI+b*a(;No~=Mc5)>@LJ;*|~WWlo=HVl$^JIKJ79)o=Q1dEfYM$3(J}YA|&{g0@|9* zDfeyS`A1U!IcG2V*-9%76bmDpHiKVAX9TBuIe9%iHwo?kI=wt= z#YXhdZa?C8B~T$w>98^L4q%ySGzxdXRRVAEVxp>e3LZpfFV1}O_HO#&fzApJpp_IZ z6J+1Duy6ZoFh0?L#d5b22%7au*765|2E+VYAEb1@`&1d3?n6a&4{jXtSd-kg#0H=r z-*41dUG!WzwO#fHEH5sxbd4U8;zIC8es#%YFWms$gf^ga*&nR=@SxcQYPrT zNIvxIK57@`Gkc@wYZR|ouj-bcY829Ol}veDYY~H zJyUAYl>dW2dDYUaxnXTl{G_E+e)RWA>+Ph%rs>K) z{31|Dw^S^s^@UJChJT`UWq$gp&eLg$Rif|-D|r$Px0zf!x%c23(CMJsNE*ln#@@B` zV*x13WYRpO)J7drO8TYo%FTw4qRBqsQvt?Z$3*-6nz6l=fj@F01L5W?HO%i4p8L29I(nFhD0g_BYO>^1di;@H zp{8`ad%N>8R77JdTbSQmz32Wd3yESFr8&@j7SI4o8kV-j{|lf{VAw>8=|_W#L9CLf zE&`jFw?Gr{J@;WAuyPWjpB^|?4ZQz2r0xE(Pk@D zt28Tdc7arOQt*4VWY$?812om|8*w(0iZr)GYGSv6>N|lP6Cs3dyxx=ZKS{)OzSdvr ztHVS*USDU$+~{bJb2ZQpW&z@9ns-ufaZ#bWWcI}R&m6KCeEVW!bl(N2ey`H8YcDRt z&;shp?yUX$X8^=1kityd6%=zg!i^V6$Rjf46)>hF9z3Bhib`?3IQgAwJ2 z(G^fv3xG6^mwa|x1M&0CEFwo?550NacRenFO6_T6aW)4>S=&*Qu653!miId8Z88T5 zcyL_S)|%e2fkU^EbXT{}TH3wK*_BVMBY1fX%sWy`KLNSjpS?+qEYdrpqC=yWI(}5Q*$Eh48H3pN?vH-MGZrPqT zVFyqPe>8>NBfCFTU;bWJ{B^m}b> zam`J|;XwQ5{o#-X??kfPB+T08SHm{iK|2IgpiJtEBkj$PQ##C|b%tvcs3LFj6g&9vq* zLrKEYcdFqjyjO2i^yvGZW72qf#OM$Ky3tJ3e&I{UtlGs*oLDo{<#jg=t&=AkzaM@D zIeh0w;>vtyYixU#0#(ymhZjK~o0gnyvGu4AWo>rngKM$+F#BVpVBht~q@wBrjw(#UPj zX?z?Z*}5152x(D9)IC-vZG{$rY7e-r`-M6V`!IgnLplr&sl{OxUqU+K*HQjuVE+LB z_eOBhS&;l`b4U3ahR_iN~8l){!7gnp*FG)sD3cWX z`}zVtRXdtuwRC#X``v-M&g&F^I)1ZaDp&7&JM`8r&{=ag zC5PT?96vGiPsa}>lz}R5Nz2E{;~Z0??y|3{!2W3yeU*Gl87?!NK|kFRruOaO^g;@|Fu*RX623`LNh?hOIrMOGin@a5bg6O0|Hu%P=L76mZ{4# zz2>E%?+41G%YU4>`;_c8wFmS0rtHe+7mrCMRxkWyQM^9fzkBV`j~yCeA(tyw>bMI2 z3~2v!GTWN5zN!_CI*RP7i%7^qL?$@m9BV%#bUwA;`;>X_$2mG38+eH+9%zZxmhA|TN@UEP!3x3Di|h=b6%I89 z7a0u`jmLpf>&qTOxR(IQ*cLX!dHP{nLfH}F-r=&BZ$A#Aq6DN;&<(k&?| z9ZJWc8|jWi9k~0T@8|n|e&2ilxMSQgu75ZNaQ5D7uQk_Pb3XH#&&f}Stj;J~vXCUP z;Lpz=<>+G_>qb;&S&`^eV@g_dSkJ=u=N>_VRiWG-Sa7ERaq)5OHv66ZX60pPpI>A? zeTwz6vL9%1yBe|04#qd9caB8{PD`bAzq_c23>d2+C}j^cinjqFZaJ4hr;T9mNl3R17=qbv-f>>~n{h`?HOEf`9nuI{S*wu>YPyE_=aC7}v9B&)h5_CLWnpjO7EbL&q_vSx+F>=_BL~`fJlw(mQcs zmCR?)6~KJwa=OmM&V~1N+8JJ;u6=0kJilTAQ3)V74X%GHl?VU6;bM6)CN(){VWh<) z70V2Snm#f6E*V~=U47qO9WwB0gq(m;{&0dVO3jW|Cj!-11^}0CY4gHdXlh)qWNG z+g7Qzi|_H^eUuag(qLA1A`Z-F_eQ?2pJ`cxl?I*V(AS0Y$izjqYRf)oFG;|qri)YC zLpPrm8O**SCazDcBfm>tM<&U?Iad7;BK2l!g4Ui(6R7V(oXECCDBlN3p$9=@P8Juu zwZLEL?mnq1+QUZ=%A)JKGOhdMgYtJzYeGK-ltTQe-1v72X%OyD;PYyM3Cilnfx1~@ z`AQCzW9fRUMIrgh4!vX24YW1<5sTlq_#~gNd$TUW@W#xTTF{I=?&#+f4*#VH?2e8%Zdujs^Pw%M3{IMwDZIkPqk z+{KaDxZ1@63SYh%S<8uS9`*Z3kCt@RY#r{IG6u?rO$@4=>AqGwmt(J}lL{sIlY; zk99ZByh4RI8*^<`t}dc3focls7H)d*pjoUpm;*~AbxYkP?o{AAw z-`-vjNY~6elwQ3DzljC!Kzo%Nf&<&Tg7b813ia5g-5w2YVZ4mV(AXL%P3T+vf=N|( z@Kfg|#5?=u%b?1j!bmUM$qre4Ct2y|#cA?lZ~m^mJ%=jwZWR$Z&*|*zXdR(A{V7dj zpJbYMtB@Pwy#dlI zI`czCx}|cDyZ%y&+DqTNNegs!jb}7%7M$D{Ck7Zg=XRM3KO7>^{zjmW8zf+x!A%0$ zM+B8ousHj^xgovHRCt@8@%$$B8PrThh6=9Wq4d?LoF(n))%PFo(>J<4+Cd3SG$zJ& zr+4mjrtL7hrR1L%T9>C3i9(Cum+rdRoL1kPcT+uY{-#?1AJNOCZi#8}*J8EPqhwk3 zH25WPv)40vXiq{DlZ*-8jOV}j2>?ZR`Ct?u17cHgz!-g?u3!gOs?*!n=aA3 zl)m$*vU^XPI`nfcUdf^c{k@`jA-{<*jG(^h#{7k&-Yc(Lw)m~6i0Ss=WSn_!1Gejw z>T7c~EPshh50WlhhcU(JRo|`r3#Dd70tAkT6a;exo#u3U{>W$%O#T59h=PmB%2DVhMAK?b+%0eq) zL{A-c+oETC}0`(1TRuy-5j(1F?cA~HdcC6-q3#sd#33kQri4u<-}~B3v`I!#_~j`w3reT!xur$A+5aeY;K5 zvrFf?s8Cs)Ff!m6FYi^Q1-M*)6ZeG(Dgj4#kGq#-F9@1RY2IO3lS%uC_9-d!4OqoQ z@cU(asF+D{5I|-0uEmh7Cr9hjVJI*nxa$8gIk(vn*?+*Vo*yu{CDvS;ADn?LsmzyN zMd}W_3ts$QRO~1u+4|`DTxZ%|X+1KS|K~L!AvXDiC5qU6RKwPObj92``}x-{dnwx7 ztvNEYpVtRWShXnmrlrGD2-IJ0Q7DL&(-V={vq?qE!DNdZ<2AF|rRJqwJ!3M=j$nCS zj8}+QQB7q)gAadKl@6vuPoOgB1)&tgdol0wpGVr|5Tuemra4qF?`<_&j4XN0@rc-- z*&*y57J*HSa~gwKIdARwc>8s%o<;|^8%owi)OS6ihP8pNUx6y6J~XGL$6v;pB6sH- z7Lt})&EChgc}MJGmKLg2W=_KC%D-X{dER{IdEE@a`G2>gm=d|1riIf`Ex#X~)%UYS zE2vOOooGGT7bXZVM%25XL{vwO*E)hTKZ2HCk1MTt*3JCLH)A8+{94#}cw*2m*pa1l zksES*y9^NM6S{xvxa>sw1Y_=58970@V=?2o$JI3v2}Lyhg)+5&+F?!Pw~Vsxz~Qna zi%P+5>Wkm)LIN9@WOiL}=9B0bh#h7kWm$0_!6Q2~S{na@?LiLb28ve0J z?6A*%BQuL(bofkG%oDNKRBDfZl?;q0W2+~O7iQbDQTu1CA1rg+*^SPQ4C)wfEP1UQ zBhZyCK?JULO~Yx;5!JD#l9(6y6!>3|VXQoq`@3wx$6qE7he?h{HwV=OW3(nUA#A;2 zeu@~@^LSK9_DK0NzUJ(JTC6R~vo@MA-#dCBv&*3cN;Mcot_)NLpBjI6D?d;|q{7ef z`4e<8t~OPZ&jQ2|v&$CJ7674h>A%b3j%3f{HWy&jpvTD>oKp=FSE28wOh?TM?RNbI zXtUb{mkRN&YspTAHZi4WlNxI`n2ER~e;s&@ebXG`sz7*sII4JFE+Q!>rtiX`LXx>I zT66#2fw|2B75OOf)2i03@GH}T0mI#U*Virdcfo&!hF-lALjmIcKfn#>(>_6JTIV}% zGd9mV|D3HL5TDZ{HF8d>ks8!_k-s|^I6MFS_y4)W-?Q&lTIo;t@%PqL0-T^~7V+eLB^uD~#EBzRKjV9Gt=h&IiT9?JlzEJ#=1=-WdQqJE(s7^wn$k) zm{qR#Xi1FlEv)dV0bq0O=X|6-?%H_y&a_N_gHK!ce8c@$Lwl3Ud`Mc35A++G-o3!0 zlkwlF`)%G=cS=&2!Kbn3UX-T&L>TX*V9Ga5{vKZ(-2}=TduZm#`ETUzT9an;E&A9T zz@5yqbaVHNi+$#|<_AKj`u3-5u+rNS{5|$7ZKuNkf6=kdcLoHQ7ayW_fCgfhp6#EO zTI;`%EbQBQH^-D4AXsF2oAO@FHWWWY>hxW^u#xM9Jj;u2hgXpqHhB z6NH4WcIbUaZ-BD=rU*?VEwUKyn}JPj^*-o(OhAE=QTA!&x%D*S^0QL?a>?Ow(^yT6 zD0X`Fz~K&o)N-)mMsYL3`?4rwU@_;F1} z*@bUfdlN?Gp zF%oB?e8L(e?cbqG5y3<%!-I0*Dj?F>i&9hV?timedObM53v|8rBlS5y>qd79D2J$Q zC?6n(_$`t6~r&CEn-UN(O?!hp0cV6Q2Zpa1H z361O4_JT+eS^`RD2^Mc8JHsAOokSI8@){;Z@=ptrAU-=bHAH=(5?nQvfI{AGd?0^* zkVGmG33a;gAnjiPd#g#jhgxB`w@28Y0g zjVEGs9VI4%v!Ix*tR60G({bZ@!USV$G-BFWjxh$z6} z7LPc{0>=3&I#%YtG2Pd=KQO&?WL(eTPX1Ldugh8GS20GF|xH1>0LY>4e zopu-TUpZ`Ak2BV%CuP6fQ>~xxBu3m%%k!y@C^dRFyHml7_m(F33h0bd3J!Sh ztJWP6X?Yz4}Guo^ktvRMqR8cHEubO2K=xXdj>9+ zD1UxHXaRpH`OqJ=GZkoOYn|#eslp@s4jui0A=%~7uo{>QfeP-x?DMZzsI=`XaBx{( z5%fvs%LAR7c;gJu(2^DA>EXu;7xsV zeM&ko8*&R1fsK?G9!}#c;OQ{zyVlO#8|N~s`WELO%hFeYM@k5a;jb&ee+Q<&aj$;bw&@q&%JmU^Am zE;6|sLt_1FUtFT!(V`Ip=L7sM0%*Bm5{bmcwX{UAQlp3ssSPaJja6{=h4wQYwHm4p zn#iMh1uG?D=&1x=o(M-IY+GausG(y-yc5f0f}U_UD4DW*Zn8^jb%iiVmSh<4R}Yba zlV0WzIkGqyY{}1BnCq&&GqRNXsx~uzLB7}+TMSe)e5O>NGq53v4Cs;(R731HUC@!1 zaoM5IYhjl7;#iYuNgVfH&=lJk(2Hl8#)yJNDP_mZ^OD(E=IIu`qJV&_LmYmdl+9|L6i&fC%T}&5+ zFKmz^jRr`N3^7)udBJj0X{Xfk z?3^^oTg{LVo9Bbw%I+0pO_;#buW)b+gFf(mkSM13K+m31`c8)bfH$&$YS}fNN~9H zm<%z*Qmji_DE!WnBcEltA+yqohYZkT%F=IanOVZG@v6-|Tz5AWE5u%QYk0L9$iwIg zVG6LXQ&^-bTMgkRfv>+xtYvK}A4_Qi#_ksPVUyK8m{xkF%{RTD#=p(}Zyn>pAoYO{b}WJD65*SW`MEA@prrXvwc2jo zB(aF6ys^VJo!Hv;blTKD8ReF>a!Wz}0|s0Cw{8{*Sg0h04FB&0NXbIc6&<;(Xs6u0 zXAM(~C?Jly{Eb4D3?)>4g{I*oUHvBrUdM2IB#78j_J1g5J$~%oh=#$LVCJ}r<5Gv1 zUs{u`E`H;socrqhEui5ho%_oEMu5^^dpTvA`-E^U?db`5iB5R^So(mwkGH@8KZE6x zaocTGBFO&_m)X})S_vdc{D5gP;p0vkuXFDn=Kp)OI$6&Y823ptuYB@h6ie>i61GVP zbp@suR=@%xBfAA(BS0(RsX+oc;r-lKpATJL@?FjUf1z0y1bgbd8FvFu$;rpk<-b*~ z`};^FqL=~d>&$n`nEUv}i*@x43u*?WHF2+r_ub&U>t^-#VD&(B67(L9;FZ4G8(SeS zk5i+0Va8eiiJR0J5-mDnOQ8>RjM#L-^ z3ive+%)f`+`weCqGyL_vA^SFgL;ntuKqKo(ekiR7AZ`i#J( zM)JMQdw9;3ozZ`ECk7ho;N(8nuVXZ!XOsY<^)d+Cs1K5yxh@=EV)l_sP1W(HWV-y7{VZ8U$BF$IskS3YJe0S42ZD)7}O8|4BAKe zzZCy8*=N$8tolGK+F>Mr;0>eESEzt6Ru)8cBWbH-K#RQrq9{4c`yMOu5ejilyd8E> zNV_jBj!V%bQqz1cWH{qkZ%J(Lgt8jhR(-T{mo9;T0Z?cqc#yoS=}3hkwug2~v8P`oj{EO3oOqDmxfE(45$C3;QvZx&+V%NZJNS(`Mq}Y&>!okzD5Kw`}|58xwKyEfp6hJmf zmH(F2Uck6<+CG<7oeY8&i?T}Td}#WDShRmEK+ne=Wr5auO;2_lC|u+==t2sKe$hM9 z+Anz6wEcYPHVHYoQZ>pS9#gsW^n2tE6#$J1_UpZdK&l}zzk0gL3Kla4AQ<`h~7{!E#B=%HEQ%*r7m1R}AOt80Fq?pB& z%364o#G1CblA$0evt>Qi%_152uA=??;(0#1IP9L3VO;f_ryP@rQJ#9pTiL9aSw#Sl zShha5aV_)WCFfX@g<5(DEsup9GC7vG`7im$-5c9IeBYvienD?vRbG=Xmf5wG5!&^t z5$#s2QkD1X@sqN}0*GnL`HcLTqBuJP@4d1M!=(GV{sEifV+)z~a7%=Xoj;C3`CH1) zClpDKD26{0So5at#Pts*YoR|R{d9i2>9W5O-cVyY zf9bt;+2j~jICmftArW%VCy&@y{ONBprO>{#3I+wX3xkSFwO3XfA_<3Ij=q&JcFmYf z-+_F)xF@)Jf{Lrn!&Le}ObJb>)Eid-o#0}2II5_$;8{6?F$S@${jf2qlXjB&cYiyU zIo{eZdE)y4I4F0BP@ox$C>?If$=-;owR-#Yfvo(fw|z8Ca9D}cq|J4R{W{SyArC6h z`Kt%bNOv<~!Pksy0ksZi;d8EBZ5{~rH$Q}H6};!WFZxJYZwTciQ%8{03y%n0@**Gh z=4X2_#j>7zvChwG};m_9KDvJ0dN$^tN`k|WJHvy7t=C{R^q-5G5FmT-ZCj;X= z4-_yTUEa5WmbE>{9+pC#;U_%KM!C9!p6<(Ga7M;vM;fH=X3QEnWBS>ks%`2t_jjVt zpC*5Q{GwR9axjg8OC`{LCUFM?{JW@2viAx9JhKde=@3sq$!Wp^FgZC3(0-b{MEyne zR}nX@r3f&ljyboV{kXw#O<;J}X(slT_-J~O06rR_2M6h*9lwXndQ#Yn3FGMsabhZ$ zPD}{Y%a8UvST3r_v4=Xf!|6ULrpVP09BeXRf4yZ{YL${cBXF6TyVIzqvb5;N&i|bs zp$!7_IfWjCP~t;YuBK$g4Y|JgNi+;4FNibAKgr(SzF8c20=I%{#fb%ZCplVs6<|WF z;u-jVbLFX6xZKn>KjMSDrnAFVuOHsB8`@NKVJ+$y+(QktJ7Ju*6=%_*h}#f$X_{jR zplD1CyMh#CuD`Gn#T4}m#sZs!(6C_OOc69^5-+D@IVk)@OmyTiqOJ za%%a~?c1b>X0cozvydE-N=c*z!?7@=_wH^NaqdeTS@Z|hGbP2laVOps1~_W1aB*ic zOlh)7h-u`vEq+nIeqQJ&YVEy`m-S)!P?JRct5bdRXKM)k#^MDC8vgmlD{Y$V#;=!N zi4EpV@+KAR&OPe84li%Ux8V=|nwqbuVZ}->B5{Pn-~&l|O;NtdFXx@2Y?HQ0w`qiy z$nsd2Rql3?Q$`VWIVo2Roo3JV9nBp1jYF4@4nRI}+2X)AN-o#RgJ~+uXpBnXgmi_cnmzv$Y z8KNs(MTym{2dJp-&U?L`qB)U6D2wQOLD--)#c=F@?uJ|*P?%wo38JcH`g+W~Qo;*g zeu$8NR$_9z9xJ_YYoo62`as@zuL|wYmImSCqd|y7{Q#eZ+~NQJ5u5A!_7B)7VC1BU zM~I-`EnMv)uprJf*GZLF*h0PlT&T z$;xJ6=XZn^O_{>=EU_pG^8SEJq{9ZcSg{B5WEC&ZQ~zgB-2S5F2<>?`QH)@ou)d}P zwBHfi56W!*j|*=LY7c!;(U6vmTz}NtWk));KksP>!a#+9aQ_}lDBydT(0@L@O!!|v z-d+t%8uH^*0)h8@fZ6BtMg7l!{>)M*F_g-yl>t5*XG=JMJO`xahJ^QJ+(k}uKc03y zixx$jeLM}vOs28LF!iCQwmwmXEBORlT*lYyJgF!?{&Yf) zTfDJ`%r)0$%-fac{oH$I|_3<@c3iapVQTe=VwHs;{_0-2)cY-sqmMJD!Qp z-|a0NJ)7J3NI_0E-4plicq+FLD{P;VU8W;UISrlq$%&ViTsh*(QpaI)vQiO{SL#Z7 zT)%Q%GJKgAY4M~e@>NVyA`SWi zF_+PCdV{}8Njp{fC5_$;9I+zka})i2z-O=PMz2gg;mu+XtlgU07;^FS=) zr@!5f27LWed?ZPnUMXC|Tz1*t8#F9m;o$<`Ikh-Ff2nlX6C#`_Kn^eN4#JsLw*KbZ zD*Ahw&^u>PEoa_&J7-rL1;-aP5)~R)_*^sW!)w0>wQhD?)sob$aSRp3?{Wu%CO5sA5JPPF-#N(!FA9~4_YDQNil##vx3&+yj?hxb#BmqI1$}JUhNJ`;S zT`-$7i^T)|hO3&OF4WL_G9`7tr%<0ylzXUq77X%6*#Q6+jbXvZejg{qQ;dw5wlUtl0d=_B`jI{>fAxXo0C~xmN$0+)E@!+ja5vYeV&5T4_DlHqD<2Kx{qALbcE01^ zq3U=qH>I%&xXg+?SEBKv$9#nPWr1%x49LQCFLKH!;{4@g^u>o7n$FRJ17*P^r* zlZ&n=(3M7B&2d@PKSyCE4ldYXil?Fkobg?#Ce?mnbw%sCX+g9rZjNN_I$f~p`-d&t zr-%Nbt%+;CBgI>q#eTcSYe2%-Bqb`Mx;$t zx%kmKm9(v;>wFTCOR{a3(-O$Sai+T`<11`7lq^cIr9IgSNjOcXeQD7{ z2ARlxNAVzWCE4Cgud}W+Wu9P8D$<@ zxu>7=or(W4WJ~1{zrx1Ekv%mxIHS~7l6nWQE<3-NDb z-p2%MdPSrjE`ow2AD}B3%7>2e1{!3Tg)g+w^2)Xi@5vg5`O(@^XeY)JsNR9DHxln- zKW1gXLxT)!KluK^Ma?!k<;Z+v76o=#?!jVshB@EJD+P^PP(|6oon7Fs;Bl ze-6qd5OEl1u{mT!_LYV0rC_%qhkBfj4VgDL;2BN$ehy7}6QN?tZQ<&_!=|&y1s7`h zwo<*7l$v0;pCP)82LS?z5CX32!9Wb$!pu;($_J*qnX2)vlU~J6ZxkwiLW~kB*sz4> zxU>kqJ#0~Pr)0|*bdIwiwJ{)6u?tJ7@~d>AwbR@c|T*&;j3(Uj3f2#jX@Qby^t^9^lFDgd-=yuNzsV;yQ_O+ z)uIc?AmZyvy-Ga&xy5SPFvF<2hbjBgi@foZF8O8TOi zg8`=$!8Tn;{oNTc5vl4bz6sJ^uKAuAu!T}?buJ5pEx7-Wn=dYm*m)g38g`R?V&8z2b9_#4`T zFoIL%GbUE|>Sra|vrq(AfPT$E@^fC6EXUk)7eO>A-|i!WD!DBhHG&BoLi7>Tbs5ik zy6d{12rm+!k|V?C^d5DY^7T|**D#9U@5^BnXk4p5l>=gQ+nYypsF}xkDZ2Pw z&xZ9myxc`rjuIE5V52}s;B}wHaTr6aF5z{`(wd%Dj4=Ep`+w4%iAmJ32R;MxLkDDg z88moV_Za=L{mipPeDsKV2A50nW>eOMF~N-`+zbU{($uFNH}DefRi=Z7;Q3`yuqFyb zr#wfj`6mtyE`u=s;MWD=lBw+ZR(jsE>w1 zi6GT0o~LjE6fjUgD0fGFLBEL#mc@HL1x>k#!vNRlq&^_I*KoQ~X)qm?&p9>K69eNS zl`V`Ub-mAd4~h*%C#-7*tqhV6Cq-!{<_uKf`1kM4fW-TUHrsrLjh-HEsS6!m9GJ()J{E>qZVrWqzF@&scI4RorfvW&Yn5V{s zJA45UJM3dA9PTRoO%Mg{_EYck&t}QztTT2u7l<29cVa~ki+q<8O{x^CZgckywoHNZ z91si4Dk9zEqt5yLuT;OO0b@Y{tD5;NOsr|%YvfxAGl!Oc5$O{Ve$svdq%xSC1*sFO zbaZN%Mlgk(@Y?-Gi!)yR!eNwb7$+l7-ol67h?i+W-l8Ha0a%C zBUcfz^(+LTIJqfr&n6N3?w6u;x}xEbQ^N8a7Aa@xF4GIk9U4riMsZ-ao^}8F(fcQ& zXi!gMJESOSRNVtpNfPmlvQda63o}X>1$j=_krjk*@Q6&8SS3!|85bGSnz2SVPolYS zp0)152|O(qwhIT6pY*40q-#@CjXZhp)pDZnVnEP`ja0}2BmPT?Q>3f5>f5MhmnBmy z4_$lnjv#L`u3g8UeAFbSzT$#%c+8wdO0NVnE^kI=DG1JGdKV0mZ*-LU+Vy;_yreF5 zTh`>SG>3PiwOcCnx?bOWcbf9iQ^fvSE>aH zHyD#^Vxsy6mRIF%VwFo7SOI+Q<#RsWzYe2e7hM;$`b1t7!TJk=S3(LGleKTJf3`D}Okx9R>$bmpH}de&mQ-7Y4E#F}Vh+6*45u5r)rmK2UoqpgS-uF}>;3 z_?zlpfoVD;J~R-yFzEsZ=J85tmzU(|9yI4z*79C0(vC*W+(6}oprbTJ+3Iuo)zsGgVn7I6BNr>oY z72)cdVbc(V3=32KZi}ykC=th{1=NrjCxFN=tc1cwE0_A4>LrnsqyUH9V%CghnPq$h z-{Eaq&P~>(pL@T~_fTWdJ2wQkuy|sZMln1KFCP4c657650)n6m*6Wxf+Jy;umz#o7 zvA{^|i|a5aJ_8*szOvA1jA6%^Yh29?3GK9>-8b*E@9_WemcJTeCc+W=1oFbCSMZb_ zHO%>K2}~0!1KUpqKD^1CVCy+LW*g6Z3n`6eAh9&0$&E=@YmS7!rM*{7;ib;~V>rs) zf32wB=w*MSfKGX}5s)N6+Kbj3#vfXgqu&BZW4c;n7R4;=j(EVa&gayC5qhBh0ks8t!dc9n&qe`--hombA%?0< zvAl#g7c-}8M0SfwGk%oWqkvP{MTDLVBwXB_TNICws>>I2CY|;6qK6H7t1#Ar{y<=z z2WMLj9aV#|KhF1lnK%Dz4sPIx9xHJ&uJ@+N#Ky~cw%R~g{aII|zI=uh7V|7rb3yi5 zb%pQ%)qwse2!pr9*Qmns*X)l=UHgIe7TX`vBYK4c} z9EH9hVIKQLTQtivL96byIq{7+44Se}yy3?4>hXW0cSGyx$`q&iZNAvPsQ-r09zTSR z;?W{EWQYxZzT2m@f_3L)IlL9JIP*fgBDE_ZCc<>lN#FK7WK^AmhH=GgPfU7T%(=iHN|j~Yq-0#o`FMq z%?3_?Y^>)D>AQQ^Dt7Y{P4c-cyHD!oZ|JIFI1n<)m}rs*%*^(i(*dw83s9Kb&}DM& zgL#{$qJEv6(3DeNu+FVt`}?lW>rw8e0_=C^e$P+v4=q!O-5;Y&IaI&n2GAvRP23$>i(svi#hQy&bd#Xy?K}RdGkPX^Uj^; z*Mkc;@u!ql35C`bc^D-Zky$SFS9j9qdiin=<^T*Dgdq;38$+K6bWQhz($^wv_oLU& zx-6g4f)aKsh5bK3?IT;gX;RI4KO?qF(4aDx{xH5aX|nO}M((g2T)(Uu*$g|e_P6!v z`KiC@xEMn~VZ4OyNhi7!8Ox` z?=134=*XlNxvxZrfKnGm7voc^Ea7Lh8RvO5gPI}_L|YVm6!a;uEX_gS#<#;vn;2k+ zenP2!1$~gj7BhNS8xKfQ4#aK3sKZ5Vp-s?z;`T{n^vAO8PxN=SN87HEAjdK(gDF+C z6l(U7o14>EENi!$>DwLPi;DFi!;CdU<98U7nA_9V#Z9R~U3=#&in)zDU+#OLG714- z$w2{mb#%2AA6t_2fnB{W#fsl-aBD^z4f?6>su9DE`7mt*lY?=~^Xpn1RznFUjn6EP z6hy6dQ`sruJXGf09o~_#1pm>QAL(qGl9SSi`5d>rqgw+5a);-lLmHw(x}BP~4grZ= z)<^v(zd`IKZ+{64^nNZT97&c!gCwDbaE?#qm3lNhrO7pLt{omNweZbqn->{BT;~xW z6mYkVUQHPZ1#)$tjES}0z2}6t3mUU`5MOV%(3K?geD4>0>UFUB2|;S{oYSv6b3E>v z{j3)c)Mrlk^+xAo}9jimGQfI1E$;CVgs&t8!x z?-(xS{Zt@DfedO4Q;N(97Cr~!Utr4&spd(iP6nfiBy5jXu$v3*{VE8u#vhrY5(wuu zdpW_r_k#moapK^fyDBQtaOrf!B}6~EVZ3W}I&JZc&s zOmZXEIk|u}w%6!x1S`B_#sc3*y|=j!pNmCd578+PreiVnA`$hYWcY@HeAT3G0aJ&? zOD?x_F{<-&BSjA3W!SD^Ai+Q;p8A6)v9W?rO~-En)6Z@DT=gn$>UB2ijxW5l!5H~q zND2#?2kdUxx9j$xQcZJ8_eT4cjXC_h0x&S^893qJ+F*mfgV{%StoQ$M#xG(X8^ias3#DycIcn-R$7bRPL_K~_pcz+5DgHO{9Egy;lEbITfh8Q%L)K-RTxcoS9$!_}j7nN4rRU zrv^S&NQvx8ZEqrOc3JR0OPe6aoS#vg(Q5q5xoLsc=*pqrdYn+DdO%2^?4MbAYH z@paffO=lh8CV`&0 zK9<82J2USZYGx|)D3C}TJM%qj@u%;GKVDvE>d#fSyCpoG^~#o0>ywA83_c)FDyG>h zls4B?&-To}7K_0eOPp~jmdmMnBaJQOXVNn_eT4pE`{N25&IVff*dMGUvYdbS2 zI#~l76Q_Ryk_N5@N=A+r_RdgDUwB0*US23Vc>^;OX#+zO>wht_E;de3T3^w>nUtXD zgukxNPz-;eI{)D1T%4^fY`-!x{*&UL^-f=a|Ku~VHU7(kIBf9e0F^_QHZiS6I*|7NF?F)_9<5VUjuhaWu@D;ql$ zGXwhHvxIoi3{|F=YCM*~|Y z`@cIGd3>e%TeOL*g^`J(nBZTE|HJZc^8XekYGLhc;s`}2YHi?bB5Y!0XZ%H1+Qin( z*&K@FpH05jIXQooZUg0>d9FEOg~NuRcO_FID2)x(*&JfaOf(_R*;MGT+k~(-C6RZh zkBRgndybRhZtGHfRp-1u>%6PEJq+|Z@cpSa9Yf;wI$@s*tWov)(Z{!D`{3iKs7wL+xR zt21F8`d$)e>{5u4#_7!P3b<5uH>y6_bDC3P*oL6V1zY5k>O2Oylnu(-1}4s}&{w7V zGjCx`^lfB^{JKQw5LlB)&%!aan+q(oa>wQqYvPzv_}&A_X4=ViB$48A zndx`bHFnwJ_m=qAlPu4tr|Ra<+<4Z4 zRDRD8RnwnV4f-A*80;-;Og}e0T{D>Ue`Z2wFv-_2t`?n7GHq8#G3h5~20*xZL5Vx+ zVv1Z1wqk?+S`OlEx-i)PnbAMbN;5rhADLc>o}zHXdf?na_Ori^cwxJL!TG$w)i=%A zz5B(}L2h}%vWx+kjr!yFYI$^d!by!Mq1;qC+CssJbk&rb+PB^rcJy*UahJs*mrurX zOd)73boa=;!(0^Eg2KIY)zEBGr-dY_uG))oVpdqXkOIz-Rdu2OgZZx@7zU^y#g~lH z64Je7WqKne7{uVVhg=u`NTyZW*th`cQ4pu}S`l#aAn_J@tvG{}5g>>h1#tExdX9wH zXA%*=t#U2CoR#*O7R=OPA9kKI$%n}6R^b}Z3^h~qa>-+((}29j|C^I)}u+;Sg+E+ zhy%w%xFP(6wwQBYO@3yF4SaoBJwZ^DPc{k5<|qSCK~&6CN6&`bWAV+?sboiTpbrlQ87M%g|*CtD?2hR%&L+pBux5C1*Q z`mMVML_gKrnMJX$CQh~_nSi~Q+CQ{%t+Xk@8oOZIPZ|a}giHN|bA>-Q?ry)3OR-OX zwsAJC!^y^Gn6ScMoSV~J2fy#9hgGhXdai|po2pRLO7s=uwmwi&4O<GF3#8AiV#n_+_jv03GbTJ^zF!8Miffz>fQN7P(d=s;vW_zV~Kc8N*N1qb(CXcB)%%aM{}sc(qRGI>!tr0Z|3`-Z zPHKIL_kVBMA7gHJ~#x_`3d&{4)G?od1Pjp#2x~xBVZ~U%~$`Nc77V{!#hA z9Y!dI|LuCtU-E7HH~6)bPF>E>(!}T=?BAh?yR(>*^Ovyyvsl2|-rRuUtAMY}UoP~| zkRIwEzW=dMM>``W6X&mL=)P?E%YaSXouTL?Y`!c^@IS54e_DxuUG%>-|0VQavj6(t zKa}Z|TnwH6aq+*CUsVbkIGOw#kx>y<(bS+4aI`S6mQntqYWrn97PenLFJb)UfELak zG~#~^%*p6$4+C50zpMTlI>X-+g5qDVRJK#GwfM)c82-BSKPCK^&;PdV|C7Ugx$b{f z{=XJyqBv$7NRQC*O64sLkH{KZ2w_YisS+lHM|lo`?v)vB3`?+32j0&qr^#-4L4}IT@~J(6b$}!7d$$+TNupFCwpK(r(`)6 zXek3;z?cuaX8*q*8;1Ye!v@3u&uOC*(j7%)k|u_y!J zlVf3nKp4k#^YNW;>(N^d$=B!i@A9_a<&&xF~>hNo5*Oh7LW|Ahugj`squC0$;W`c z*~G1dzpVyc_A^?ACJ=4|ID8*@M))SrC*&>L;L-HLu|YQ4-M<*c36%Y_WX0MY^P%T* zS;D#z^noAr4HtwSVatl+4P@~rNJ7oNDSAXVDfK59bQ$b4SkpGQwSS=cBVh~fLqHDL z0-ZkCI&o)Mr7Z~AK0hBi?Cp*!t9B<6X`k2)N3)iQpnTOqQ`yS6RxyoPR?IsV@s6hm0>f<$T*e#fPGroHe)<6~9zG7Qs$ zz}3BTN6X~^=hWhB(Zc>I;m#*rK&w?nx70_q4(hDn!WyraNmxKU5x6qK8@nod6yP}< z+!B3a=@`QA$Cl1Q}PTT)z@sn-Yx2)B|#0hLN71FGXC~+t2`3lP&46Yvil_tl| zF*RHUa*pLhYdg@E9$4&^z3DCSJy0*Eo2oG7K)}+^ey3wypdOa4M^Byev{$q?tb-_L z3NIejAI$6QSNEI&>*QnS(;d7<=J_F6J@~T$N%V)U`}Zwt zc_Xm7y2-~HfkyRR<`eZeKFZ>g_u}$5!leqh(w$ z*f8=KtlGY00Sb5NxAcL>PWiZJm6z`fn3$_=x=LERS_R&r3wJP!_o#()vBmgM{EoIg zw*s-7K{t)zGTWV1hhOS;*qM};v72IP`qDb$Y>Z>VYd^zzV(Rb6TtSA;_6-;gCp)+< zr^I$HNxZ@JhS{#TJV5<2{OHn1+V_)Z3k5A|l!+!{lf~7f9>pa9E&xz~5uo7?^y7D( zO!P2^V;6P`#F)}P)j`Pq)c%xof=qQ9uY&a`w@Kq{>D$J7-}(~wD)*qaQP)Pt+FNoF zYU?eSJ9u`mD&JXYgW~VRZ2vX)Fzj8sE4J92l0VKC-^64O!MI{P zyCo?r$4=1=bSsKU)z}2N9AQP(A9~)KHQx`=8}pG-iKXEI>W)XPJKe_6h1BOV(uncq zj<*r&q|ZjF_miX)_a=5!kONfLKpo!j^*JQ&HC{=v>U6=_1|l_K^9BydQ*x$P$wMVE zKTzh-RPU8l0Jppmcj;h^nZ%Dku+?E&6gUQSgR1ostbpT5%IzG(U+YN_?I#r1TBAz- zajC&X{Y|xFsg(34=87S1O}IZIrqE845`JJ^NF`q`Et=JYRpdo?;_o~Aof?wx2u7?c z^(Uqw(Z^RyUVpqee_b#s--m;yR(9F1Ez;>kG-cd|n z(z63yZgTXol}k?-h=w_j!asy#un5WTg|3|VtPMPk=d%l~NQV9@V<@(l~` zLQ8xE=q#3-J?Eo&4R4dRpW9)0AI2i+rF&lPMTc%Fms3Ks)7%mbxS(y;I7;ZV^WJOwYIL^xQ zLvun{O6D*uS;(U#-`s>cotvE=)y6nsP}$&R(ZBY@{H8@b1_1y|c;4^q%E}t53T_4) zNvdJfz{p_^sBlSY1ig0PI&T851!l6&3fC{0h?d|gj`v><$fBN33Ht`G{IEIFv)eB~ zGG_!zm5{g0k13!yTAs-Xr)HZ(fXQ77k zk1QEG&}Lyb)N{S&t{Kpj#X+1oqq!2{pyr7;PtNch(4aISl#6;`JwwN+YAGVCDg@q>=>#?us+U5tJ=cq7wJl!geWxVywJ&1St1W;H-MmrS?d8YYX$Mv&3sMwS^{6uRuGk*+;ih#vpDx=hFty0ZdGuWFNv3T@qT%S`KnFm%; z1Yc6(7D951qabTmE|yLrqBz>&6QRGq6RihpRZOfo&Mho)4Rx(-GBNwih{9Y*n3s&L zpSUw1^%|O*c;q1So4mq4CB~*tA2rw`tJE^%I>N0>U)%}i6o?MFR#}+=mPq{>pBD0o zx$xfkeF0u#gWfBkS|jEXXF;^}^pgxa(9EIC?5m z7)5uNq(fd*=j!+f&jO0LaW!IBL{Dsj_~IB;U0khUQkn|=YarM}r$ikrxV z9V?W{t+=I78(WPABl%*mH{Po(d1!svXDS zQUb^H38+z8dF-#4v69ckl11(YK-+!7{J1lv)KAr&1R#q3(_T73UKX3Vk{*um)H zzB)bqGxEADVU9Fp&~%(-s!>j=liS9&ly%S2H>qykH_9u@vHBTzs=UYTd2Mv<;c*>6 z(^oVzD{Z)&0iMn({l>IYAOS_)L|SK5q*Dn~flGx(C(6=~99;IxMW~-@ao{u`OBWi& z6Ad(6nlN0ZfWL#(@@KH+&r)ElI^9+h^vBG>l&UbiAu19p)bK)H$O=;eTNtkycp+uA zp(wo2AtfFP9?9_ck3?p8HA4=t=Q3PB2?zLUGQMOZRnl=Er`y>VlJ}#I)`QCO%jMhs zn*%vskN2BAmNyyFdg$!*x^fMl=GNBd;F3$9WS-@>u^(g(M=VFDE9GHse1eNyf!bz) zZa+Y6R)+@`NH-Ds7m7QBTO=V%nw_gU0=6bE>xbn=#-YN)q*Q-6*}MKBE$$Vq|4tn~ zWv3j&>drkZ9(9reS*>4PqfeKjPp5q=9GW_`jnG2_fcpW(Ulx{qkZ+czj_lU3j$C(X za&SgE9b}Qm8nX+*bU>6onwL)~KiFZyoFo6kY<1!+RZU@h!|U!?qj!q=&dceIj_;yQ zxU1>%X1E$0FEyYbv(%UJ)d6+4!k`%$IY9KrfY`MWiNK3IH8&5U?~l2+1B9viP%13R zVwOq0i}V=OmTuAzSC>{o)%8$l%FRg|@ zZq&x$#s&~$}7)twe^*zZNnnCZ4w_`W1UaGJfpG`&x3 z`ATAP;kaqw&gmxNka@Ynv(I1tkehj9N9kb~j6`;5v?L^{A|+4^Z_#AAMpCN}zR>Jn zpiT@K$WNO4rq#bdT=f+hQ28;DNI673Bs!E3t%g&)Gi!fDXG{_70L_j(VqLdF!Zg}2 z=fNNE6HhE9yA4-jot(Wl{Rim+|QO2NXZ;@MW_tB|8`V*PDLaz3{Ya$4tT>li~hab(_$whUYTsY7XpK|S$ zZo3b^d4DqoJF&5nC|0z=2jkP=0sVsI#x_A>Bf-kKc86|Ui-%NJX2_w`zd)P#iyqlk zGE0Rp45&ykI9L`@jlvuqg+N%7M6w7fFpIG2Ay?h*H! z_BO5#DPvjUMlZS9E@NjhiXu_esaLaC9ZKA{JXGAxUdB@%iIVc=ZD;6`?b6)wbT#;*fa3e{SdyM1Ys>o{qIOZrC;7&L!V%7JkirW_#8YC4 znjFLK;$3*eydp8>fZTLbe8X6cz~a3W+>52cuQ*XS#=Rda7<%?IC_FE?!Vgad9;0!dOI1ff6b7+nz{pk%^-&JD)RL8Zmk&Bn2e zc8+ujZV57F-%_|9zr#7@s!E4Ax=>k%ImkLjtOx7s?hk&$SL1c)k}8Eyp`r%Ed~OP2 zQWu8*zIhHEGBM5QTuNPo5<=oNRF++)?x`6HPx?Xe4eAFUv39S6Qc!hLO4eHGhs>P} zr1GMAI#q&Qz_Zb^Cb4W7ps)WGk}HZs{Omu zGKpJ`qxcSDk>F0aT&i@A>06M7%ZxUqVNCuv017fhR8AGtW)Y-HkW`uT>B25C6rTb) zP=~ub#`lo7CyG5&j~_aG&q2)lwd|ImEmu66jXI$ohPoTD0%$+{g5rM$dq6?@Jt{gg z4cI--g2?dQ9qb2vd$X|M^}|n&VLaST6^0E+D!c)B9iF-SUhNWW9CdkNP?c%7@1Bes zUxoLvRd2Aa8o};!3P1Nhm00K^edxD!` zq@;0HJyu4T*mR~Ne5MJ`0QC7|v8xecI`$|QNi!!$vkr;H&JTKC_&;&oqOU~zg*68j zXsU1>?!(J3&Xe0zThi7hk?1i|b4$l44B9!+lh4PKOq-?^GX^kIaT>rI=myGDYXa>$ zsbQjy2GR{dFw%QeAtHhx3gNy-O5vcD%2dLxPhF<4ayEN56?0Kf#@fK;&`dDs0MxfD zoOlo22iw*jBKvmSDqP=^q^dG0E}kz?Y)2-GT+P9zEyG9yd zw;AZtU&|1RSSLH_pc2?|U>N#2j>aZe|DXy6I#}I99Q?_UY(MZSrw?h!m7J?t0kQDa zV@$?JH5CdSQRn1|(?OU0;hjnP#4dB?pm{;S6p>0bzJ8W(YKq0e1YK#g=z5&r)T$*4 zh!8LmDyRi$3|49shEN0@T&hI5p=A+jH4WDsn)h?F3|-+l90;z-{uSEFH09#@@OZe9)#qO1 z6noiwCDIf&IhL72p7Y#oR-%oYOxk*e%VxYi>)Dh1OOsAKJo+#OX=)Z3T*{oM+e5X& zFnh^HnLPy(>f4Wf5AKl8@jV%Lt4hnM|h%Ipp=&c&mrd1yqGJpF^R;nDO5lq#x)e$>SP= zzCpZ9W)_Yl?xN`5o!Yk3}&8)p)_0+tei|*3NK5C zv_`UPdJdfDpQ-gpe*K2-#qK|J&wUrL*%P=I*TWbF85j>YHpC>ovtc}K>Xh(McN=A; zsM8;rZw}wZQ_J~5+|Vwe?N%(TT{4YpEDp2o$gKnnEM`YC0f^)0qhoX=FVtzx-y@zz z_GOD9o~4*gP7up45WTlYSjZfqrJ@=a3w7s!W>imcQBSJ?;Wg?G-ReHtkJ*)rS+4R# z<`Xdn7&kzit`1p0g6tyLm@k_;EDTyN>;Vdhb?Ol?joZM^+MT>kP4*TVu3hw+vm{;- zU7NQew~&WkpWmHo@pzjTwwP^v=Gwg@3zqsbQuszl!Er705lI;!PN&}tZr!qvqdV)D zH_9Q#JtM7UDeHu?05aceqQj_jl3Imy1a*YMrURzmqaiZoIB5WraFfR7e&TR54WzR~ zMQNxd`}QF=)KNn|Wp#x=Z4zRh=am^k${;!Lk$@S}fI|*Bhw-nuFh<@#bT&Mb!mpIL zu_-oys1xx4uq2t}o!dVIO5q5Bl7wzxL_C2`uvTjoU=cItJDB30l!aLsuHWl>*F1^Q zJ+se)t=T865h91=no-$T2eK(26QRqvaQSqFa9ufGmQjYr$gx?rPlBfQR4Pmd0zM*# z(&K@32eNv0sdZZ%Ng@ikX(_J<+ftnA&Lsq(=jRHG=gf^XyDgL~Uw#+$k(u35;&NS0 z6IkrM1UpGV=p^3c+Q^?5o%*GQ#V>VxMBb4|>Y^gor64uBwvVV8Pu;>ctGyswnUdEJ zXJ1lymMDiAF)Wi@K*l*R9DjcZzEjbB&2c5eYmCtfS(J zp2FR7X!6i}XozkUaS^#h_0E2GITQA0+g6*!e4hLTx1(H7KafrCjPk1m?xD|M9gg|t z6bV;}{j5plFbId$Pg6oSYyyEb4gv1F#V813|1dN-JVYv4?Du)_6A(v4TF4xE;sytP z^F96uR=Dpr*jEu=$jZp8sg@LpOAQV`J5=H^rM>NjUmn9W0-TatG;sHYzcGpY`*wV6 zg>U6^EVt#@t+J01loN{Rfz~-kWjv83({(tKzr3KouQ*TN5M*l?y3QwADa_s@nhFQ> zBBqZ4EGqMY!%=s=(jW4N?#_>E!yS2$JLnPkQxu*;!N^Os=VtvcNOW_qn3S~2PSUu} z20)Xwn^}(yr4lZ)I(FfkdHp7gm^RnmGUzV7&e{&-xP2z-$bQcb?W4%$%rkjFQ+;mh z4lZtKJ@#e#;c7PNP`>qj_*8da-9QsJS^*Ki@Qw{#Y=ZhCmo@O0eW!s6;gq|eRgv_{ zsw0<#4w3{#t?oNbkcL?|NLVpz=77mKrB7ZHpi&dRJsz$nmGn&ft=hCT`qsPs@}r?; z5pMUSAh>aMvY#rNsDH9QUG~1_uAymK6_2JNNqvZR64Ux+rVp^u0J0e8I}3y(<&{YK zPWJBY?^n+UP%y-+jFa+7x%US0)3%Fov&tAsZuk5ShRG5aKrkqr(i` zQB+!mU;gmTBWRI_*AA)@D~83zu|991nRi7wXxLzPp#?qO@v9rxT@5cW_v|fd8TX(a zi0jEjok!qTI7>n(F?WSrHK)l3M0pI%Qdw-Wz%E_ty_6fks!ihfShZT%&9S*HTKnW$ zKyB3>t~xa9EZt#ShBUUcxJNvkoLvitt_fpHYNI%?L{Jgc&alPtq4!YG5F}WP6(TQz zXz%_GE!WVlbx7EeWW&gFqVbQ)KuSv#bc>rr=vP5ht8ZbH(ZA2WpH4|e;CVz9sKI4^ zFQ~w($f%S!^*t3^Oi*P~Ct3ts%<|#W0iA z5X_^nh~@`1B^t1-_DM_4fG*;@K1&4Zpx=d9kf?p<5-T(FupSPlGpWoZ-)$9bktFZQ z3;L%!eBp`Xp#@6!5|S=aoQ1T>vbZsNm@`pa+4D9(_$ZxLXOb|Q8Hr^iJDD{ygS$Rm5jwxfV zthrYys;1R_b&THipIA$fc=Ou6bq7peAdF5}8IQS|!1FHoLS8Iz%gB`ro~IsKcUc%8 zMEd8@_P>+2@eBiqj-|)25P(I6YB{{io_Ryq0*lz^Uvku%Ur*n!*3J>^zzA(F2B<_kpMXVCY-xAhO1j&z;&%YvcEEy zhaFXjm^!#E$?Y@>-?DIO!=??i(D~ILwyY~GE^aHo+>%@4=4t>q>z!TEqR!GWNArp< z6`%<&s)v>edA?*mtbvv?BazU{=w)c~Vlk6a# zpnDpmb04N_8KQ&P0)SVm)jROnoLSpMW;G|(CV&memr&S59S#3Ha+aZLEKciI>IKSn z0`-fA9#fZPt=WCjfIe31P|X-HZITEl!)xBc7`Ks{!6>#TX%!nz3}CqXqe)ezetVUK zH*bq^Udw_!iBtFF!10;FMTr(k-c@D6pyxBt;p^75dB=UO46!TQ2&A?NNzi2Hmd?H_ zy&$WFJZqXmW-3ofY+IOWLolRo5^9pKXwxd4W>cH!=~;63;0CYi`Kt)#Rt=migH&tC z(^Ap$%_#6z`nF<@=T`CXxdfux&_Z=!ze63<4%4oE*`bxWkGapawGZ|Cm?<44Wlj$R zFOldD(Gsaj3uP-tFX$vfg^?`>KU;)&5Uy20XT>t5qTtyN3IQsU#imw5f35BHj@c?x zaMKu$@C>JBkmrKj^z+aznvzt~G|z z1~uqdbS#rJWLUmS0RUxLU7JaUdFJ+5Jc%B?(6$Q}i*~h%%xc#H%xi zW7M)#lV>@t_-zE10G#7224;HZf>p;_BS(G5LF_T@A?vA*KkJe2Gi*M1-}uD+m`+nK z)fZ<C$(Fq0X z(MNf#jc~PkRjtbFM-+NcXM%ibUoCoN#|@xQJnB?m{AnuVCY~gRL_0RJP)C}AijuF8 zc*Ki>OB&==aR2xYveryckH|7|^inR9uLGdbVv)c7K>Y;?-S-WT5wXOuP+3?x@i){_ zYFHqO@)I&1tO)ocOx+>_%tT9Q)O4AV8JoD~QvzH>knXgRsyZ5KG3LXcoWlY;gB$1$ z#c~wA`o_ns`bFGqrvtGi*|yI5$J(N?CcZA?>&e5FV!diCCeu#OOQH!gTa3+iKxD)H zc0@If$N8U};es|umEN(I3*9GrhI}c?+hJ3hvAe2r8c zYV_p_99H#I60-DBMUP32oNpF*kne9FyE(r`-_3g0Bo8PBODAjRR#Alz6-{QE&?y4- z-pXhO8RRC)M275V%|}NgjhwGK+6G-VBAERn)T0b!8m|H{?hj<(AG35Eu1S1eA1|H4 z=%gmM>g>fYhoHgL=4mZGp@+2dd}+`7(*Trt!Sf*W>GeO5mf*e~LqrerCWvzKyjF@=$~o{*7LKomv)fW-6B$knvh{fl)n}2nA4K2WI0F&`QJfX=-J?pRntWu6 zV&szYaMA~TT)YbFh+^zf%?M!Tjd^90gDFdpgd>j-(Hu@CNHn-O$TbKq7}<>+DIM>H zoLEe-ImG>VX&BgIIEL_7+LeL}ycGR~K?nxKp+Tl$db{_BBI4RzOGizi&0jo)coWxxp&?03fwIX0>qD~uv(5(Z@ z8w!Y71SuprzqVb&mLpM15OWS&oOnS*b(uos*?4|y*4y`&sj5OGxx78)#|;t~aCgNT zmRDkL?m6ECH0(YJItYi3LXy--Ae_g_^AQRAkt$46P?_GK9C-0iWGlxM?z0zQ=%4FP z>KDtpkK7oruAz+m%uJm#0GN;I<3LrglF*v*_a{hX%dH3c#=9a`IU0B$j3!M>M$PKh z9a{HrPRd{Xc&~35()~oL28q|3_ zmDnsax;{H_di$WoV7qud=wG4Jb34Xy6>z`E#{i`Z1-lKW)8=rzTEHQ^CRox%#pd#` zo>FILT1eKKwR_%|$e{HMcebi}fu&>KfB;6#vny<{$=_I}(~JyS5hoYz6}844qiaM; ziR=^B`R;ZPR}v+${iwcYi?u2}a`CXw59gF^G-#cU;Y7b+UDlwwn2wfuI*v8Yn0s5l zlF59@hQ~@}S{GILYa-odf0d1VS6=D%_&(_Sn3Zkap=w;&Kw`d^TKb&%=GYN`?ENxF zeWT-Gl0cuUb__5MmkRBqVfyGusfUy#ceNrie|0+A^rR6v%-K0bm&DBsl%w3s)r}M# zUEB4b&J=e7=4EW7NkYkGWXN;F{kXTsZRKgk1*Sq`G*c&(%e}%&gjACyeBqpdh-Pl0 zn9_TZ?RAz(cr|nOs3T38FND4;`p_VTYALhc*Mx~mN##lM3yusb!;1PIR_(joheI=B_!!w}s zQAyHA_@Q{V(~XLZikoP&J5CZ!unkn`Kv+i8%CaBri8e?cYQZ{J6}(% z9~$^lHLa0>-W75#AL5!|tw$WdZ@*ipKOoFfj@$Lz8Cx%@$Z1V93x<*l_S{52Woq3$ z(!}~g^Xw0t=w0djzD^TDEU%535r-i7Wss#@_>O3`OjVQIna>;f7}raA<_DWZ1b*WD zo%e{eL(l)*k)a!E4!{lvR)!T9b^IqMfbk;mD9*L#v0L2BuAhoNVzc!i%l_>p)fdJl zvQHEA^^JvaaBN=g>*k88q)~0`308!hC&-Ez=8xQgFv#Y{MxR{rDVF0W#6Ozh_6cp_ z&xRvnn6l5@lZLH{y9q@EiC7?g241Gl3ea`G#5}TZlcOXpds=H!Nk){!5Gn8&$wgy~w7e5kbTjr5_-Kf#0 z4&oalc#P!|yRnkUXh%2`-CR@dYGCFFSm=f>^=IZvA2)O?w7a4(1}KAs0y?@z za|g8>rZs9%ax}`d!#PcuVB%8#AOPT6lzAaRi=wAj=26D3d}?gt{lJW9Uh}>!IU*lk zS_xf}q{h^66u&=ZuQ~9>{0U?)fj%JO7US)eEQ%2c3l8uMFRjZ&Pw3Mf=7Mmx4qH%DoHlej*-UH_cW)DuK zBg<{30OmP(Z{f8lJyJle@eTmc)yTD>*e1u6Ue<7y?uqF)a=;efc*&O-=)Kvfi{u^= z=}d>mOzyNcbr{PB>a8ixK6A>6#8Q&`*&VYN?=kOc?H3pfClJ)Z7Ul%|@Z&_OS6-LD z_(1!0LB$Yu4s1`cdUH0z`f;w_m}pU2L!Y<_$vR-OVO`3-#5rRF#Alrf%_3iHEXp5F zpIG;eX8Rxv;%S#SIDhS;^DUa--P2b+iv0c!qll-5IZsxId=FWKKIbQzIzM#XnbNpC zPuZx^T!I6u^*bqWqo$x3DYfLZzEW_@8?+n2)j1zMM`Z<=O4`hRLjfZE+IGGQLQsh5oRoHmFXq zKZn5d&Do}@`&2ci6Ssg{k? zfza|`Y#5e$P}fC4M+bg9MT*sKe8Yrpl0tiUv28&=u%bCBZY*`;zxpJ;k8Hr%GQ&7= z2~rMmYQPC7=FIGC-Q9o~O^?QkMw^-KGS;vX%ZJQpt*72 z8{1%qXA>YZws6F(N7!HveY@vev_de!jjFRA0bNqyp!4T5gm(o>vOdQ3AQ?|46Re$w zHCckr+ULuy@P&AkvP~t#`}ukxsxIfjfx;;+;r$7IM^pBO5LxW_AvV4Rov`m{-Hlom zTk^-yg6w8fOFzMESJQ}V&Ssdbb@>C!y9(|>EtObCcttY`a5-O6ntfa#&>4yk?=PFL zLls(!Y0hNI3$tZOUPTd|GJ6z>HnhqpYZS@Hz zGGy{Y0~aUSwtNU`LpySj$AYoQN@=e@2%U6vb4RU3niCQC3tY}Sm~lIWyCksfHTy&b z(5|8(xceOoAdle+u1B~}v@BuHPst@U|Dq7u@RI$&Xkhv7D!CzLu7!8h3h}Tc)WyFc z#jlGkDX;o-ssq_u5jfe*pqtoI6^%7*s4J==D{BkL__!^;QdF|v?oUE?7C-8OK5af`{fOjEP&M2$qjaFgP z^gUMX=V6D9$C~$=3y)s*WdKE`!Ic0s-#VlZq$Aj(Si^*tZgAJP1Cox%*_hj#y`vst z@*-vUm;A7s&`@mEN$z}Kq@G%wsjBn8Vo z&IN8MZRRhqPK_$0*6w?*N%q;c1q+|W$6}*}IOlUKO`0m#n`oGvN8}bPh_H10F48{f zfK85oc2WArKK&jCfILuk1jsK7@D%-h_Yv>pXn&~y$&^a@9tXI?+)uj~`t>q4Y);@H z%1V3Ho-r1}gJeFV&PCL`g@PxIVbo|mQVFRyY1vuLY#uQOUvgra!Z8~1R=Kyc0Y-PP z@s*FfpO!I3^4e2}i?L*-GRI1s!!e|K$!}&md?(6hTS0gc?;uYwl3h&C)%#otpW)6C zp%|(C?xfSadXZAiSd}hRTww47+@Ua8lLMSv*C6+>^E|fumv#x0(+4j0&{Y0G}3{HMiguK6$D7vZY>-Ht(}|&%j3`Ia5>ba~d|>A^`(~3bAA{(tqYe=r0FTUq(M5iqX@TZE_<}C}PrV7@ z+$hD%{<>2dKK|MR3-ZA?rtb|I6GLDBk@FH~tJR|Ds$@0z03Y|-56rhsiUu6*9W{AN zeY9S|J4^=QuJiAO(1!AcBKOPoa~oA!z=!#C_syKl@~UMQ`cifKo9}^Ur?`d$hn6s0 z4t8L<)XH7wZ_57JLyi%%6tWb0=RLHJl(SK+LUj2RzI=m(9^m-__56LHYaUHRx*B8M zG=yDDBbb%hupdC5BU($~l(jze6>Gu?Q&)j-B=aZ^|A&&=>yI}UcW(c{^XRv}s(6{Vuw}df#yKs+^cSw?d&vGL z(mwm}p@xekb2E&fraLUeklEeTqidV#!D=DtrAey!IX;|AWxb`n?0RRc?uLQG=BPQj zUktPNEtgD&buk1!ShR6!M0XsvT&M+fnJL9Nx4|%5-{#Q5)N@=ng&0cbRgdfg`?>uH z7Ec8=nG$}OEX$l$U&J-uwtn0&R!k8P=kS}?iaGF#)f)7Ryx~-QB8(oRkW&o@-stSY zT#DkTtNBPbv}<#*DGd@(+G{h_Zm@E4F#@q=1$#o)|T-}eVykKN4s9?$0*R^ zgN7M7HRuSn$maP0Gm0a;-WJG>>wg;4f_eL8MlO_xl>X=IE!72Xh0EGzKpkdrFUJgu zomI~o=*F`6-Kz$%7HIpDw=;e#F>SPwR&2;i3$W#dF$7WJ65;AXk#<;=SM0;D3HSZH`*&A!3qR{xxya)f01?u|9JI0adL?_8WbSjV_bqt9)G z#k~lnF87Y^@aq%g^#Xp*gXp0aV<_NV95=C>^$%;x$#L!@lX};8lQV4?b7*U+1}QCl zj+u_8>9M&rkg>1_J80_5&mVhFt)6^r(K%F8U*oAyJ@la|9=1hPb=+69J*=;m_l)mr zYQo7@OW)(78+~ZF>Ca7^&5-{KHl`_i9f(kjGI=Kb+EAnS@}xt zX>uP|I;5<&?wDt{nnzv{b1dJ(x+ESiL@OPT4{l-H^}Y&GIBe zYnOQgoYys&`3eMlfC1GA$Y?$ycxXlSch9+QzRB4xs&s!IZ$#%Y8iCBqeGY9fld6Mv z1dnhzhg4}0UaqRIf~MUlPr4k*RWpx6xWB)T*x}>YrW2fO8Ky@!K~D3fkx$+pj*9gV zcYh<8Bcm7T0s)t7pP|Ezd_jFhIAVOVo?Y8_5@+lC9>iXbY$oXJS_iYZj2ZN~+r!EG z%!GaEQv?5KZ6%}fr>4{JBJvs5e5v^l<6NVClnL&clI*T}%pd6nHAEAr#5CKV1cRHZ zX%MOjpzk1Anis(qhlJEy;0n)5-?f%<_1w@!_cHI9Uu-$r*#hCm@aA@@0zMJ=`VspE zQXM*uLq=MfA27CHQ!VXfizBWhk9hFPdO=@=!!#~Xn1g^1*6QS zJ+yEtO6*8hUs!(OE$k{CGXx~3243@ukN7rqbjxZzV`)ko@V%5+Ae1<9BmS_Ex@QP; z>UbZPYU8uogb}9`&jy1%E5xAKeH{Ds9Ai}bFf%BPhh>fvg z;!k-jMkK}&*O>8~q}`hPk(@2IAht`7s2U~GVZib4>KbW$h@ zT{?&YQey$+LMT$A6v0p;0*X{oic(bqUdly~rYIm71VjxTqzfp;AT9Ki?Bo|ElRu0G7GmBT z3XwFA_V=50*X5q2i{n=)S!+P#({%@R#SiA;3PQ(X>*Q*7PRoiXgbo`@<^=pPAgP0( zO;uX;)S5lF&uKs%-T2cKa>QQT;fb8rCjBirguDo&VNQM6h5r@++g_vhcwKJfbH>d| zhUphsYio_UjR2?GpBvUqNpj5bl&wb}|9XI+*vQXEie{k-&<8{&{SlsP7 zH$dJRy0KXu_BOl^*3e-bzHYs{fyvzeN`*t=LDk5OOg}qD)$pc6T#XVHN8NQ|^!Q}EBj(?kmny>lYEgLI_ zTdbH}w0o{@G*|hyFUG-DIqUY>qe19poTuwdHLO=vsB&tg#lWcQ!4OnsPMRs`lJ#{d zf4rPtz5(wy-q*95mhwuJk+u;vE!4F`qI)FW<^k;5t6J}_g@X9^X>*Uiwbg}Nkyc*~ zjJEW=s__yGo4G)U;Z2xfD6f~sC@K>2lCy zre$6xFt{>N#+DBuA9v-0Hf*Tx-y`WkXQDNNqpZUBO=8QbdNS*Ia~@MKgd*0C-Rzmh zmUCJ$vF*csBLO@MxzGP>8VQeDTLI18!r|u+QvcewuFv;P@?_3!*qaMOh#3emC~;5w zf*DBwJ$wtEKEjyr$nfsYc8RQba_5XNua}}EJdhsR7Bs_k`)xCc;R<#HCiFuRbZLedyEG&3Yg+Zt2T{V=QlBp75`+K`?O~rfAZ`2Y* zwPtv@Tunl_mq397b6l~$KACA*H$LX=d~i2^2R(c>ac{iN^J&FFT*0+HZ{5nB%s3Y; zAX{f=T{>1tW9P=iXAT)q2Ci9bbMHyL053k$wt8s$=Yi>j%!LpM?jRza;M*KDP<+*S zY}5TzPeitd2zQ}*Jn-Gl)JO=q>qAo;NdM#Aia9H_<^-L*FHP+t1qWSCpjQvvtbPbx zIgwQBExcM{u_I_eg*9CFD zdzO7C^(Qx#$Ad>M(j=LXto(BN6`mDeRD@n7^XP_D$pXGa!jmBl>1Z>M!{gZ@>6?G&1p)QA*_k~(hgh|n{ zeExLtIxBE=UAR#%vp6hE%PL_cx-Q-AR%c4X){Y9IW#sz@xEGhRZ)~`zP75-mljb3c z!lZa^j%hL8R7!#q1@@=Wl8)VDDo6%aAMo5He8R>-K(!@zT6V|$)i@%#jd5{9+VVs% zbw9zRQ=6t+on+4 z)!^_AP1m-i^+*f`wQ6q@*2rU%Ie5Bpyp4LQegOxoT5|-aME-djONU-oR^>PQ} zXhbmHq;;6w*LpB$_chH$-4g5ST_F#Psi*FgtdgtxN4%j?I3xQ=!^Wz2Mx_+do*8-Cm5*H!9SMxt@mf{RB){-%YgCx1+@4^h_vvoM|}-S5t!q>6}Vr6x21Dm*e;F4 zF_`xRp@>;5?qj7{tmxC_Yq5tK$tU$#^T|2lX>d;}IbLzG?pyR5@$^mq&p2(q`pZXg zj%1uRKSJnq4Cb0Sd|5ef_=YNJJIO_lFYNpc+r46M_lnvZzT9ocyMyEw5eX@7@)lZu zxItqihK4e>6Si@o$3g3J+d?wyWT!l6P;CeOv`8uk;kl*^7o8FaDTxE+gWUJO#EsPr zC8zV9-n_c5nel?#{)MPCPY#bIPn9@-ZNg-iSpsoXH!FOL;H$Ld`N#(e^{)ku-rYL7 z2ol|5Fnwoalk4Ostfil&rrE@ZFT`c~4q+hk-Xjx6+*gEtD$T>@k7)VzRN!f#-Qr_% zJy1%VVwHw*P5kyk>$>DGf_nJ)wAM%TL7H?%ZsDbr;)XNTVCPT)dP4v*ZkDmppp#J= zcPp@|!Dz_dW>vlLQ0t6{i@yS9Xt~H6S}5b{pITClF5FpGEo+l{!Jk}ej~4KkZNFq% zDa-5Z55f#}7I`BITV4IH7Rje}>z^U(iIa=#O7w1@(7v)P{!kaM?a-y^ofW5$7N?#a zr|Jl?-+RO=(d0s|TfB`*lD?v=tz%@$+~AAA(7GeEZJujf@~?N!^Xk0WhUfe^wJo@L z=NbpfUIxh4Ca(mKv`&4$3VME^u{*9;J_=0YCxJ;jBE)X^a@9N5YLbNixL`-sPIz>f zz)A3(N2R+;0Dprw8&uG%)_6B~#?vY#tug5O$X*U9`p}Jbus~6-%oyv8?Vwm zCQsU)Ic(3gb0H=Uoy%sPCprjiLY-Th{fl-jo)PJ=FTqT_(CyPH=WhsB>loXZ*Yo^R zOol~Mu<{ox)E4d5FcUGC#G582KTV1eqLtqkY5Ht6sho>{DNw7jcrn#rXNr3ENb_w< zTy#NDLyedjgvZ);EG}+2@#`&0{Q3PNhwmQ}M7g-37w3Q{kKf!;04q`DYuc)m z`J?-hug#6}PP|?#xlI39GQn_%Q##xhsn0OtWpVeD6dO0noZrF^DAy96+6H5C&IqSd z4nk5C6KbDblyIuDc5=V-#LF38TacJ<VyWMkK3a|e}+rJu`{sh&BxL#YgpSuDQX-eMiphoU%tex*T~ z(<>-kF$(#JN_NUWKciMsjmL8`y=J6tsQ6ToriMUr#`rM`-!I|EqcH*m1Cy54z|qn#fG)G zmxi@xh8$y(W?TGTn0bzx-aD^SdvS7t6dSf2!WSa#WC^mF6H>1(pHvAM8tky0p2`mF zbHih#mX)4Onhkz4t^!#)XyiZ1>gpXOU1;CWEdsHt=;Wdof=>2K$$M8Fsq1lZwq0Ef z5~K>e8<0>740&TuwaMZLdZ{`cwQQ!!>sM%5XG)(Ee9J%N<5+sGPT*y&*_Ai;D-ofs z;f=!>LKN(|&q6X!QHF;3X4L^giCasu;-`M4+O^5hd3!7p;XgVC<#>H3a{LBV=FQ21 zes^Ujcexsir&Xt2s%(`DHY*xFVEWR*(EMDnJE6j3YOFcID!Bp5N77%jP+pwjwG#hErGhF|D{WwST7b9 zkd1wUu{^POM=Y)(Kp8}^Sb_ki2^iZ7XK1W<%JI@qHWENavIQ$CDT86Q022ug94I9g z5)ckXqEG-FNfFFuA^~HZ11l)7=uB|7IjajCXh$N!NJX}SRALYCz2Qf91s0SLFjPhZ zjo+J;;6F!T`CzH60|^Xe2L=Qbi{FE0g@Ohg+JVss;Dl|0R%E+i#lafoM{mGH1+2n~ zM+L=l16bPu@1F!Lb_RZ5pMT_toii2}5@0I+!jl4chOWLq9@r*2-(ib#%F2phIR#+m zNH`D(0uEOK9Ky6&j43}b3W@#!obm(9Apy1*%ospVx;trmyLw`QFaZ?hB_pi6s~cbo z#0Kyrv0gT;DYJb5m?>bBuyIHKLQ(zTcqssYqaS#xzgc1^6_uYV;Nsu2Ww9fD!7vkd zXBPSD`+i_UBK_Gc$)A}D^ufA-;i5=jnSZ>HqRL823QD3bqTe+ni(4kjZWZc57?UQ%bR`2_?utm>*wg>$6ie!EC5ahh3cCbfd3C(VF{D~ literal 0 HcmV?d00001 diff --git a/publications/whitepaper/figures/realtime_timeline.png b/publications/whitepaper/figures/realtime_timeline.png deleted file mode 100644 index 76806bc9aa2a89d0a102e1a5299bb8e095494838..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 37015 zcmeGDfb>|{D4F)rrE9^ANRDuJlK za4}Wc2~|l#ND5@t$uL#F1TYfXq5eeo&%skwRU^av@+B;XLxW>4y{@fhejjq!luWSH zca)ql?0D!=QNys;&R`RcTKxb2`~Q6cEr@-WW;1j1y#c%IXT*;*8MNvEnR}ea#|}|U z#s>_54BC^LkLKnn6r)Uln1_GKb92X^0G4=uw_`WxfUE2i48LD={LfKONG!mK^cD`l zMf`&{>gk98u<1KU1o-RPk%M}IQvv!P5*Pr}2Y#|>{SPgCsL42_0mX}Gf#@+09;N`s zhu1y;?FI~NEXIe^HWU*@w}2dboG18z5c&V-C7|b#jsH6muwF$HY}()-*QZAdo_T70 zA_0&Ap~p0!CIwtQgu{`2dl$FAMv}p|_Ye-NcH z8EC{Xz^2`$X5bHVK_dh;-812kRJ7;X;``op2>XE^I0Ub=s4u(K9!VI|%_{E7yU=cq zESnMaosBAuO+fpl1p471e1?Vm(j#3O=LImu4oosDTJ@p}cHx|lDKdlvrI`&@{G{Kf zt%p+CLA^`>_|t4F$mm3ISmmNMUT1^7+*c{Y9Lb`{(|z%ukn22i_I(;IEM$6(`AgMa zfH5l&S#JBB`59lDSA|q_pM9u#D+$5lDwOM^x_@tVAntF7dE4v-;|P5X9FC;|T{AD2 zn7HP2(-|IMV+1n(3!68*?e;Yn4_9LA+P^a{R(Frb3`T$YM2{? z1C}xtqCLqu{k_(B#q*12(Hq57bn(K^4c#$erX$3&u@tSst-dj_t!SLJN$5T^5bg0$ zbIX^QJDWa@y<^{=1ZB0U3>S7)?X8FjxiCA{{|%LQTb;Qq1CSS+G6F5Pyk%0W*Q<2< zM4^LD#8d|@tZ{rY??SrqxNziFeyD&B7X>h|){ohkc4{*wgP{)noSeQ<7*0*$U`(&4 zPZs){&PTI_P|IXw&hH@Ka=BwOGnJe=^6o!;>dC#3fY0}WKqy@qql9rD*8bcZytuhA ztfoYIa72j}_t8<@x~9w>b)SZ!6OOm7J1zyyMP11_iSir7z`}cSM{RtPxU1e~ftM1l zPw|F^MHN8kDEZknfh7`bhkuP?WNpz_g!3344*oh`zbGq z3|BU|ON3&Hh!8jJ<6bYnxK!V2O4A>J_M$`3K5IL%O3VU`+XX9s5+c z12)Z{2j>??oa%d7X?w9ACBg(eEmpm%ZxZ2H~WI}9c7TWTJnnSuT-ga@2JVU|-BSu^L@Q^U7uwBNUYQ{>rXe&}@?zt_B? z1h*mN4(cp_adGLyUh1zF5XL*BE$D&j*G0{p+;zJ|#J(I$g~&5HbSH;)7lF{gELXCW z37*}xC!Rn_84Q0bz;kWs4e!?3O7$TxVZ_{ozpc_3-eyPV{uT$T3-2i8svUz8PJ{y{n&Ay|3h*|Gn zUi0qV+OWBaN@_H{ybmNaF^$fC?qt&5I%buQ@V#Y4g{l8di(D)jXxBbDCCn<4`}ha* zkMiN7dVyRkG7m3HFE?+eGpl4CVU10(^=r=EWbWbjPSc@jLn8KmEzd)e&&es@*PuA1 z$QFVqXcbgM{?Di>Un(%GaH$%HcaW<-`3KVpl>zP z9MfhC?!~IJS%L2kmJA3Es_bTs>fLCr(HBekB@%Ci*L>z3lrJjC*n&)6;fwedt+4s{ zrUF_L8zk$=*v3cqrQ%(+B;6|T%-r9eEb8s+4_1U0ffj3c4Ht}mwrk~YQ11VN&V=xr zkDWiyo(c~EV?Ku01;Gf+DAg4z_&XR41)<$vLFIz^H7$x}Wl1T|>`Srife)%hGqAJs z*awdlHvI=Br^b7{_2W{ijVz&`Cq|4>6&l0m%3J;AcY2k#(>4s7bKX32ZWei8_&J#? z4xSF>$$vcxbo+s{d)p`}gmQkQW4vs{ru%LUu?q*!I5(!*v?>=(A49r`fK`5m(C+ig zjKm{H?b?r~wm<1!ZXO(GR;DEjy1jF zdfrzHJ@gGGUqUq&Uw)jurYp!KfqZv6Xt=y(f@a04P~TSH92ndeDZ;e}6| z+f5+#GN7nAVK+O&@GQF*jmV=NxVP&;`Kq_?pD%MytB^?gmaCn#pFByMcM_@(-!b$) z+b=@41-Rz<`$|Ag1xIlWAFfKa)?%)9<`kO`Z8Z&AaVKYM5=A|>9;`67(l4ZV6Nq8$ z`)%!C;VbQGbNP3DQFNNq-fO}VcI9k`dvcAHBZJ}~ZNCQfX%6Ly7a`qzmX!kt-DxR0RPkpch%IMqCW8W!(q897;MoRu1EL~_Qjczy1TmOL%WR}~VJS4YPB0UfE zyL)vQ_mQL?ozRb?!?{Qgb9o)}q{C*y$x@2oQOqN6jE3cQu7;7}H~L3WPk4BaS+8Q(XUC)FRd`_Mjq$>j&(arvYU+9i zBsFgRluWS+79lU9^HhVaZP(|+gS1P~$=72-nx}fA)+-y}g$L3^K41L7nC{iN0d%JG zDW99S)e-5@lW(%XWod*W2M;Ak*p(`-*$}d$p=f*+Pf7A;3tK2k?Pn49*=if<4MG zeuhX@34~soM6*t8smhHOVfd-p;@eHYCQNj8(Cv5BYfBM&-5vSD;IfaV(VUlQZ%I86 zTI{4-F?7xLiCY%yKJZtPrd^^t4I4A8v_BHqf$V`uV?hKTIiqJL6hgBOeWy-cM%Cf~ zzcU#gNQvfL7^mclv;TGu=?ys2pRREu1~)E>Qd}0`)yI;K_Peh)RYEcd9$W=uAp)_) zP!IVk-6gE3e=B0^5Mlz6cld>+0vW=9NN-sKm^%VXP`x^ z9sw31J-k#B6x!|HxSTIKf}ul`1q{+K!mCvE1g{f$(9+eGfh8KMIS!i)BTn{1gyFaS zKMyqMoQf|NR5HX3CkH}?;Hm>hZ?ogM-8Cje>3* zZ@FN-`AYrEaUS~xtKO24wv+`USM+2s6U!%?mnzJ$3<4k9d9d!~Sc}@y@NSC}82IT6 zN}X;l_LA6dNTr5DmTd%%!%w^tLsvadLoGp}e+Ms3HtR|r1W>~t$)c+IOG zVq7Mn>ZEpUzn-?|Pj*3WGzBxi3&r=UJnq(BSA@K)M~Br;_>pC?eJ2BMdTKkiZsL|W z-Hc?aF}Nnugt>B@2g(KoxJmR~4MWnDm=QeZ#@4YI>1l$y>s_ac;C)U?RyZyy2XtBI z-*R_n>D2~QsJzS`Zdg|7$b1iv`j`8SDCzL^s4h3qJEeMz!Emf(b11AEkx?sbz{vPx z2Cxm`$$k~mT{V1bw(BDgA7!;w0P!jlD*YrQPh!277y-VuwdffLNtD)dIh?h20>`Mp zAcSG^T83N`4fYD4tVCj%;ulCsUyqrIF)8B4dR6dhC(-fpLM6e?0sUc1X;Q7D3@7qA zpFbZd(fI|`VZOBsDy<-A&sdzz5CH9yZ#|_!`w@z8emI@UI+8jTu!&`ahu&%-N7(Xu z^T&@)+&BdBZCE`Ulm{4i&Or&0hw(hDG}ZKhmJ=Hl!)~piCu1JkCUBe;pRQ34{44~g zwN=&=xS$szGsyH(xtasulkFJP>)G{CHi+~G-+p$YW-nk!BCZVZK9wyr!ZT3xqF;Fb z=GfzNNotKiyyN%MxNpeO()X1-0`CIkJ{rP_%sCAQjFqsod|CQgtkH}kK=Jtsom34! z^tb~}lz?ls3F+d=oC@r$=?_96iDvQK(ol!%zO0&98OcvQYcY5^fs6-1!)Y3p`N(;0 z_jeDj2p!Mod(Oh^Uf(mm7m86_2i(1+*@7jQi%=A1y5DQ|2iCf(yjZXDS*S=>(j^q~ zh?HR$Mb3V;9vQ4`aapqt721?F&C2VoGbIe`ekGfl!fCp_b{Jw=U12Z{W&zR^KlX#|>|oH#Gv+j|bbf)X6Uh!F9^cJ)Ovq2;RFw$AWBV3 zSkIErCLn{mjb{{E2PU>iFDiA#Hb)*dKl|OCwoTBp7BbKL=fvA&{iXn4htEribnH;_ zhd0WTv2hwbb)pW|Kw>4|yqc5z^ICw|yEz^Grq%!ieE&;|S0}em=miUR#p*_j=XvB+ zJJoM9>-SOO#%46n$>@*;7Ct6VK!M$)iB;h+o&kewWm%N>SCjDgpg)XB|a=>*1g(Qqm zx2==;o5=fu!K)(1%n{Z5rDLU&w|rsU`8~pitb0Gtga1uiCAxzh<{;EBP}gwU8^_(0 zy^fcGCqDO1OVq#S3&1U-|KjNb+gp=`gca_`Z?&m3%|(xNh5lMSW9N%9FsQ&h?=jB< zmb{k&_y5b5C%H~w2Ev}TkJ4hx_dcVw-&|tnwON`W!ZVf5VvYp&=wCGZcmbmaj1I9UDPMW5WUa$TkT3D6s9MPC!JMR2SB5^k}Q z>h=4yqd(M-)CiL|73ch|_vU%qCiq)9Da<=muPG1LUmCw>O*?+waXL$2=?BW;5WNt% zRb6-Fxun0r@;J{xKUe#`W-IqZ0^4Jb?(|!aFlG6gWupuTxF}E}8g53UZiL6TmGE0_ z!YOi8bYDy8Fz9Mx>ei}@_dGb!b7NK3U&R`A^u|6^LZjtAdEhe2>F^p|U*6R>QRauh z&6WOLNj_rp$Wi0~8LY17LXenMm?Mj*J>#{TrqZX(WQ4dV@LG{pEauZ#C7UgHLC&c$ z_SDvcnM`Sz!64SQ_(Pqhq{#>gtl5R8?eo{k+EeO~Ii=rgCYir{ii4t=;v(_Yu8J*P zXxOe<_ZY!7XA-ef#9uQZ(djY~bXbv*921n`-HRF3y9H5%(Sfz(4))!#0egiscEm8S z+Jiu685M25QBvB_n`@9? za?YQvvI1}heiY%#d4;Oedm$l#3S24Drn*$c3dvia!qFRBs@n_p%X=gWSYhHE*M$fU zg>>((rgja&VT2C}>uXvNtX&CU&%Zav2Vp+>GIt;+cM5RVno&|y$5W3OIW3h7xtT^ZV-Dial9oRf)bz(`u%ewHqGH zEBS!nZug1Pe>uY@R@$InRo-EPxj!r*{=rf@pldhpDBJydXavp6Sysu2Q%yJu_fSnJ zzrS6JX?WV~Ciux+V-Tl$fDF6Ds!+=7K;k(9rz{E@G1)}0IH1tIuZt7jJ?&MtOc~aF zPP(o=wVc~R=jy{gU(GiX)*UIf?${Rfp-~lNs0TWnA%Xpg`*Qs~DBIwTX2g4Ddx-*H zVa~n3b{^*eo3}}BwojejZ)e)D{x+j<3aKGy?|2igKDYRq$BHmP49gVXwCcXnBHp79 z(qp#6qOovl#D<^vBB0-}`nBeRtMf{J$4fz3R%t^y(DiMHP3akX0Dj#*e{CbbSIjN_ z_m$Yw%ctv(RWjUv_U!iJ8{~O)cR#WI();mS0_6p0!Jpdx&^SuX(pnv>2$*P_@hHD+ zNjMvy=#1mZBQGwefK1Zm0KzhrX1{zPwJ6$qIK@)I#@0h`BuJOL{8V!1cGX=8Kl$-PTde zqU`$P#6&%3+((Goo9jfWcKGb@Qu6k9vD$EO8SCGSdA6RlH>mKEtm>J-{q22Q=|_=nv}?{9f}26_X)` zjiZuOK1WB(fZ^;xcd6RrcbP%LTu1rVVHP$WJnZ=2n8>82LeZJ%v-FjSDF`FM@yKp! zU3tNGs7Ohb7SEd1Tn2WAmByX!uMroZ-0vYEijq7?adfz(p!;uMfVb>qEWY>Uf*#pX z!dI!%IC|^1^@u7h_yzj6edU$z%*{20tjh>FYNz)Z^>yFo@}2C}JX4yIF-l#RD(afpW|QA!a-h0JP5jPy9{{F=oNyA73Rj{e;CRc7VMjbXXU4!*w=Tc8a-)SNh^Sv zR3IcCszeX$r*;FXEtp=hj|4|o&U|@JP_o(L`vbvmOTLIdx$WLr_ii7|@NpSAa`jL; zG_(>IMOL+hiq~cxC56#IDSeb)&P@}K|LjLNI)O7cP^u49gaZvDhgR6h25&^q%Wj_9 zV6FUx6(VBA2%o=;qyNFD$HuGpu{EimN{fdacDNX^b$PtkS@R5-i?T)O{8v0#fPJ#8 z*{}E=bz*p2CIEvxb)n4XR$xcvT>+#gSp%{IA}bt~XsU<2mS)gWn(lrD}LX}o0@+N#{>a%Apwo!%FQ0d`b<5nMw< zg$j~~tv^OWi0XjJw?7oyy6oRnW_!zk?o}^Kq$a!e##1|# zceO5g`>R-$ehkA&vJyrxTY+VJv!*S`W?L$Vcpf^L|oTq@qN0Fy~z zA@b1@9=1I4L`hZb`4Kx$FD6N+Jz`d`_fRjM?DidD&x&(CCvWj<6-0=6wee@1>N_rd z%#>TzsXp&(ncXv(^mo+M06-mLa4PTuSwSe}P<+S0noisF=_SGJYY@t727l~XmbfCk8 zJa~9_;G^;p^PF3U1m14TXujZevFksw|b=jyiep+jBzUKHP?VMn0}>z zas)vKKP~pm`&v`IWit=&HC`Q(1Su zf_?sz5&wo>^o8Eh5>b^5-sI}$${76&;5oYWP*pZq{0WKDV0lB9`q>a(QTdLg?d5?d z^y2W}_;)EAQPJ9svjwSPM-8~v%t|fv=6FHC=0NPHP1$?2WSZ|38{M9|=yqbnpRN0_ zJfw{*jnk;;ga@`$zo+0$4)}*6?=x(sNp~$ds&`fVLx@%&+Zb0&4s!+*KNBiT?WH%2fvBHu?5)sS79 zlwtP`!JRWbcefH>q*p-;)I5`#Lt6TZsY`L%?iuvS$0 z?BCUdt*vcqL#B?29Br6luxdinp ztC_d?tUIX7!}l=*X9}8dV^%`zd!8IWR{GF>r5JvZdL^iB(uK@}pt->FMt7$?zL}>4 zwOlPzyb>W|zLI7Q@9n9gd09ULtU>oX>g%4~U|jPf`6zzU&^7bS z8!e~ZRUn&UdZB0k!I%*Ley(`l-uZ!YzFSaM59ED}qz)?FTUP-P@Nan(Wa^4Enfstu^Por)pa4`nXbhihoQ>^G$ZBv=X%Gp;AcuwxN0rt$wAk77FLab-U^R|?y z;?{o0^pozmyD0}Nj1K4t$^_fq1D+&Ag;xRyR11J<=g&X-Le}-k7}s%gKelTb7FLMj zG#n-##+244VFhiNayZ{TDu%=5&j=AXDWCEtOV3NunAhwx3Pike+|*23oPCET;$ zgASEkQ&mDf_CfJmlTQch$;SwBO?sGQ5+=MrEtF3nkJ_^kgEkP1Ap96r50QC2+8*Be zc3`_UGEKe`-G+r!GI~3?SmrVolI;VnxREJ1!&yhLIX&=?tFu~JP={mMgEznSPcAmK zWFkeF_47TM6G&iiwOz?5-~1K!gQv#)daS;Sx@b zA7=l-JDLKp8L3X_Z4m=-F1`bkI1#io^Y_(Bk`5ZBA1G1Q#O$N`(v*I|6a-9*(emcY zwrrrlD;gn1Iyu#65Jw^>1WTXaWi|#AX=U*c!#F_?n;}iG^|~_?Up*O@a9r5o>yO+93N>X?-3Qh7kE4CTHBVFOuqw zT44GPZUEpo@=R^UUH>DEQkMjz^Z+_BLEMM9N zT>Ojwox5@W9?t(a7hQ+;!Ik@}HLxu~$!f*X&_M>Jo^8^x9cSjgGe7;2oP3nG^%lM> zYWGXi;v88GVPH%v03CW2Df#y_06RYc0N_P-NN(~WEQ$nHgQeLfenF*yTcj~y`eMW3 zY#71j>1?%IYvGUFRtxF=vgpR4A2fa+RCQMj1|{6~ml?9|pICMAZNplO_Z*vLtV~p2 zN{{8kf*0K})*VeI(Zbewh)lmS3-`>;PG;b>9T~5KvmKh0;h0uD=5ciul7>p3j5z>+ zOKXX$ewyz)s7#s1aa@^MU?v27tIsG_8EAP?$aq1JRfmH8)55>SB0n~~0{ya56YgSA zbsEcMI88Yo)62WL%2>`u;X2)oQwrBAq1{X#(%f}yTTmcH=7Mt)^Zg%&as@)He=it$ z_o@2R0p-e25)l@|oin);!x)s2N(C;0l<81KaL&GM=^zMR-8+O3DR1OxUvrI+KnAh= z{V*JZ0{P=hH&R{>BsH*T7UH{kw|?U7pJI~;&UlhlBh3^b@04zU7>8OlF#B(-L4#bI z8@v-&z3+V(9fV2y*&IgAT1L&>_s4b`5yd~v)T@O_ZUW9+>{lA);Fy;ZY-KY4OnT}V znbA?wpgfc?H4lp-gUS0JIev_4t)yr)k^@r|hANW*II~8FMVdmg=9zzHr^Sd0k?aha zlwtOH#-u@^^$e~fIM735wZK`|{dl35X+L)8W-&UiM2N?dW39qwMV-9_E(LEM0_WKe zBQxsUe%0JudJV2-(>0S;p#tt}gvR&Dm$2`&=b*bFXJR~I~4UzZLk|U6H5IB{y zBCkPpG_ju;hB3d8gMxE}A=6xVE3z0Npp^VdAB82gt?+_(w2j#pd3I?#OLaEGN_m29 z$nSJTk&6rcbQkJx50H*@odQre8QJr2*O39XtOmIFk2}|W6P(Ug-B}g9VN+Ll=vHEE z0OFD?(R5}P&y+^we>d&*t*8$*=1Nzg) zJFPIMluPGajMF&OQS1<|5=OT3V*#I|>bAM36T+@uZ6Wr*^u__7cdbUtWKiv5d#!d4 z1Vc3CTv>MJK=J<(s*R6taOriH%CyA6-)AIr;i5XiMox$RMwkz4rV>4}C)I1(Kv25> z_W3>KOj#a3I^e`NO|q0PNkIIQ)ZNj0&xCxZ(39BgY)`- zqZx@JKU`aA27V5v`QPB72ZY|Jpl;BAhXVK(G1y*5+)AZ0tFqt6P4|}i3+BSVfR%Ho zW`BE=5=IJ5PEkgz*+7y|o^9WMswZQ7y}44AZwYx;p{_1-L#4!d!MnM_zeXM&HzM#y z0~+SLC{o$?-qEDx>dqRgFQ_{SZ55gk%Q_GcgZ7`G(OwDK%WeD)TK7~XL0frEUBWLB z*<;YJb#rj+a>MlC8Tv5SR!2IUU1J`tzWQzP2c2Nr<%o!>V|w9hK-<_;Q8tG4|Fk?u z-+6B0yG3gVCuJYM+9HM0>k~eqJBUBHCBb(BZzI>@oi49+Ze{q)a2)%T>mg+}oH%g$