From 393627d8cef1b4ee705cd66d0042d452c0888562 Mon Sep 17 00:00:00 2001 From: keith-ratcliffe Date: Tue, 11 Jun 2024 20:51:20 +0000 Subject: [PATCH] WIP: Bulk V2 --- .../bin/services/datawave/bootstrap-ingest.sh | 6 +- .../bin/services/datawave/install-ingest.sh | 19 +- properties/dev.properties | 2 +- .../main/resources/config/ingest-config.xml | 20 ++ .../resources/config/log4j-bulkloader.xml | 20 ++ warehouse/ingest-core/pom.xml | 5 + .../ingest/config/BaseHdfsFileCacheUtil.java | 2 +- .../job/BulkIngestMapFileLoader.java | 150 ++++++-- .../job/MultiRFileOutputFormatter.java | 336 +++++++++++++++++- .../job/BulkIngestMapFileLoaderTest.java | 291 ++++++++++++++- .../job/MultiRFileOutputFormatterTest.java | 138 +++++-- .../datawave/rfiles/metadata/I3abcdef01.rf | Bin 0 -> 2153 bytes .../datawave/rfiles/shard/I2abcdef01.rf | Bin 0 -> 235223 bytes .../bin/ingest/map-file-bulk-loader.sh | 4 +- 14 files changed, 927 insertions(+), 66 deletions(-) create mode 100644 warehouse/ingest-configuration/src/main/resources/config/log4j-bulkloader.xml create mode 100644 warehouse/ingest-core/src/test/resources/datawave/rfiles/metadata/I3abcdef01.rf create mode 100644 warehouse/ingest-core/src/test/resources/datawave/rfiles/shard/I2abcdef01.rf diff --git a/contrib/datawave-quickstart/bin/services/datawave/bootstrap-ingest.sh b/contrib/datawave-quickstart/bin/services/datawave/bootstrap-ingest.sh index bd689ec71d1..09af6bbab08 100644 --- a/contrib/datawave-quickstart/bin/services/datawave/bootstrap-ingest.sh +++ b/contrib/datawave-quickstart/bin/services/datawave/bootstrap-ingest.sh @@ -9,7 +9,7 @@ DW_DATAWAVE_INGEST_HOME="${DW_CLOUD_HOME}/${DW_DATAWAVE_INGEST_SYMLINK}" # ingest reducers. Set to 1 for standalone instance, but typically set to the first prime number that is less than the # number of available Accumulo tablet servers... -DW_DATAWAVE_INGEST_NUM_SHARDS=${DW_DATAWAVE_INGEST_NUM_SHARDS:-1} +DW_DATAWAVE_INGEST_NUM_SHARDS=${DW_DATAWAVE_INGEST_NUM_SHARDS:-10} # Ingest job logs, etc @@ -39,7 +39,7 @@ DW_DATAWAVE_INGEST_FLAGFILE_DIR="${DW_DATAWAVE_DATA_DIR}/flags" # Comma-delimited list of configs for the FlagMaker process(es) -DW_DATAWAVE_INGEST_FLAGMAKER_CONFIGS=${DW_DATAWAVE_INGEST_FLAGMAKER_CONFIGS:-"${DW_DATAWAVE_INGEST_CONFIG_HOME}/flag-maker-live.xml"} +DW_DATAWAVE_INGEST_FLAGMAKER_CONFIGS=${DW_DATAWAVE_INGEST_FLAGMAKER_CONFIGS:-"${DW_DATAWAVE_INGEST_CONFIG_HOME}/flag-maker-live.xml,${DW_DATAWAVE_INGEST_CONFIG_HOME}/flag-maker-bulk.xml"} # Dir for ingest-related 'pid' files @@ -72,7 +72,7 @@ DW_DATAWAVE_INGEST_LIVE_DATA_TYPES=${DW_DATAWAVE_INGEST_LIVE_DATA_TYPES:-"wikipe # Comma-delimited data type identifiers to be ingested via "bulk" ingest, ie via bulk import of RFiles into Accumulo tables -DW_DATAWAVE_INGEST_BULK_DATA_TYPES=${DW_DATAWAVE_INGEST_BULK_DATA_TYPES:-"shardStats"} +DW_DATAWAVE_INGEST_BULK_DATA_TYPES=${DW_DATAWAVE_INGEST_BULK_DATA_TYPES:-"shardStats,wikipedia,mycsv,myjson"} DW_DATAWAVE_MAPRED_INGEST_OPTS=${DW_DATAWAVE_MAPRED_INGEST_OPTS:-"-useInlineCombiner -ingestMetricsDisabled"} diff --git a/contrib/datawave-quickstart/bin/services/datawave/install-ingest.sh b/contrib/datawave-quickstart/bin/services/datawave/install-ingest.sh index a2cd713d960..6990644fc7b 100755 --- a/contrib/datawave-quickstart/bin/services/datawave/install-ingest.sh +++ b/contrib/datawave-quickstart/bin/services/datawave/install-ingest.sh @@ -30,12 +30,14 @@ tar xf "${DW_DATAWAVE_SERVICE_DIR}/${DW_DATAWAVE_INGEST_DIST}" -C "${TARBALL_BAS info "DataWave Ingest tarball extracted and symlinked" +source "${THIS_DIR}/fix-hadoop-classpath.sh" + if ! hadoopIsRunning ; then info "Starting Hadoop, so that we can initialize Accumulo" hadoopStart fi -# Create any Hadoop directories related to Datawave Ingest +# Create any Hadoop directories needed for live ingest input if [[ -n "${DW_DATAWAVE_INGEST_LIVE_DATA_TYPES}" ]] ; then OLD_IFS="${IFS}" @@ -44,10 +46,25 @@ if [[ -n "${DW_DATAWAVE_INGEST_LIVE_DATA_TYPES}" ]] ; then IFS="${OLD_IFS}" for dir in "${HDFS_RAW_INPUT_DIRS[@]}" ; do + # Dirs created here should be configured in your live flag maker config (e.g., in config/flag-maker-live.xml) hdfs dfs -mkdir -p "${DW_DATAWAVE_INGEST_HDFS_BASEDIR}/${dir}" || fatal "Failed to create HDFS directory: ${dir}" done fi +# Create any Hadoop directories needed for bulk ingest input +if [[ -n "${DW_DATAWAVE_INGEST_BULK_DATA_TYPES}" ]] ; then + + OLD_IFS="${IFS}" + IFS="," + HDFS_RAW_INPUT_DIRS=( ${DW_DATAWAVE_INGEST_BULK_DATA_TYPES} ) + IFS="${OLD_IFS}" + + for dir in "${HDFS_RAW_INPUT_DIRS[@]}" ; do + # Dirs created here should be configured in your bulk flag maker config (e.g., in config/flag-maker-bulk.xml) + hdfs dfs -mkdir -p "${DW_DATAWAVE_INGEST_HDFS_BASEDIR}/${dir}-bulk" || fatal "Failed to create HDFS directory: ${dir}-bulk" + done +fi + #---------------------------------------------------------- # Configure/update Accumulo classpath, set auths, etc #---------------------------------------------------------- diff --git a/properties/dev.properties b/properties/dev.properties index ea526d63026..55e84792edd 100644 --- a/properties/dev.properties +++ b/properties/dev.properties @@ -41,7 +41,7 @@ LIVE_CHILD_MAP_MAX_MEMORY_MB=1024 BULK_CHILD_REDUCE_MAX_MEMORY_MB=2048 LIVE_CHILD_REDUCE_MAX_MEMORY_MB=1024 -BULK_INGEST_DATA_TYPES=shardStats +BULK_INGEST_DATA_TYPES=shardStats,wikipedia,mycsv,myjson LIVE_INGEST_DATA_TYPES=wikipedia,mycsv,myjson # Clear out these values if you do not want standard shard ingest. diff --git a/warehouse/ingest-configuration/src/main/resources/config/ingest-config.xml b/warehouse/ingest-configuration/src/main/resources/config/ingest-config.xml index b23a72e2670..c51abb4d599 100644 --- a/warehouse/ingest-configuration/src/main/resources/config/ingest-config.xml +++ b/warehouse/ingest-configuration/src/main/resources/config/ingest-config.xml @@ -91,4 +91,24 @@ partitioner.default.delegate datawave.ingest.mapreduce.partition.MultiTableRRRangePartitioner + + + datawave.ingest.splits.cache.dir + ${WAREHOUSE_HDFS_NAME_NODE}/data/splitsCache + + + + accumulo.config.cache.path + ${WAREHOUSE_HDFS_NAME_NODE}/data/accumuloConfigCache/accConfCache.txt + + + + ingest.bulk.import.mode + V2_LOAD_PLANNING + + Must be one of [V1, V2_LOCAL_MAPPING, V2_LOAD_PLANNING] + (See BulkIngestMapFileLoader.ImportMode) + + + diff --git a/warehouse/ingest-configuration/src/main/resources/config/log4j-bulkloader.xml b/warehouse/ingest-configuration/src/main/resources/config/log4j-bulkloader.xml new file mode 100644 index 00000000000..2878273c05b --- /dev/null +++ b/warehouse/ingest-configuration/src/main/resources/config/log4j-bulkloader.xml @@ -0,0 +1,20 @@ + + + + + + + + + + + + + + + + + + + + diff --git a/warehouse/ingest-core/pom.xml b/warehouse/ingest-core/pom.xml index d46f4a23f1b..081b5d7d87c 100644 --- a/warehouse/ingest-core/pom.xml +++ b/warehouse/ingest-core/pom.xml @@ -14,6 +14,11 @@ com.clearspring.analytics stream + + com.google.code.gson + gson + 2.9.0 + com.sun.xml.bind jaxb-impl diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/config/BaseHdfsFileCacheUtil.java b/warehouse/ingest-core/src/main/java/datawave/ingest/config/BaseHdfsFileCacheUtil.java index 57f7d5f8991..8d96fc6c002 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/config/BaseHdfsFileCacheUtil.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/config/BaseHdfsFileCacheUtil.java @@ -130,7 +130,7 @@ public Path createTempFile(FileSystem fs) throws IOException { do { Path parentDirectory = this.cacheFilePath.getParent(); String fileName = this.cacheFilePath.getName() + "." + count; - log.info("Attempting to create " + fileName + "under " + parentDirectory); + log.info("Attempting to create " + fileName + " under " + parentDirectory); tmpCacheFile = new Path(parentDirectory, fileName); count++; } while (!fs.createNewFile(tmpCacheFile)); diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoader.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoader.java index cfd066ee75b..b95a7a6a177 100755 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoader.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoader.java @@ -6,14 +6,17 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Type; import java.net.ServerSocket; import java.net.Socket; import java.net.SocketAddress; import java.net.SocketTimeoutException; import java.net.URI; import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Base64; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -33,6 +36,7 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.clientImpl.ClientContext; +import org.apache.accumulo.core.data.LoadPlan; import org.apache.accumulo.core.manager.thrift.ManagerClientService; import org.apache.accumulo.core.manager.thrift.ManagerMonitorInfo; import org.apache.accumulo.core.master.thrift.TableInfo; @@ -40,6 +44,7 @@ import org.apache.accumulo.core.rpc.clients.ThriftClientTypes; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -62,6 +67,15 @@ import com.google.common.base.Objects; import com.google.common.collect.Lists; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.JsonDeserializationContext; +import com.google.gson.JsonDeserializer; +import com.google.gson.JsonElement; +import com.google.gson.JsonParseException; +import com.google.gson.JsonPrimitive; +import com.google.gson.JsonSerializationContext; +import com.google.gson.JsonSerializer; import datawave.ingest.data.TypeRegistry; import datawave.ingest.mapreduce.StandaloneStatusReporter; @@ -73,7 +87,8 @@ * various tablet servers. */ public final class BulkIngestMapFileLoader implements Runnable { - private static Logger log = Logger.getLogger(BulkIngestMapFileLoader.class); + private static final Logger log = Logger.getLogger(BulkIngestMapFileLoader.class); + private static final Gson gson = new GsonBuilder().registerTypeHierarchyAdapter(byte[].class, new ByteArrayToBase64TypeAdapter()).create(); private static int SLEEP_TIME = 30000; private static int FAILURE_SLEEP_TIME = 10 * 60 * 1000; // 10 minutes private static int MAX_DIRECTORIES = 1; @@ -83,6 +98,7 @@ public final class BulkIngestMapFileLoader implements Runnable { private static int SHUTDOWN_PORT = 24111; private static boolean FIFO = true; private static boolean INGEST_METRICS = true; + private static ImportMode BULK_IMPORT_MODE = ImportMode.V1; public static final String CLEANUP_FILE_MARKER = "job.cleanup"; public static final String COMPLETE_FILE_MARKER = "job.complete"; @@ -90,6 +106,7 @@ public final class BulkIngestMapFileLoader implements Runnable { public static final String FAILED_FILE_MARKER = "job.failed"; public static final String ATTEMPT_FILE_MARKER = "job.load.attempt.failed.do.not.delete"; public static final String INPUT_FILES_MARKER = "job.paths"; + public static final String BULK_IMPORT_MODE_CONFIG = "ingest.bulk.import.mode"; private Path workDir; private String jobDirPattern; @@ -106,6 +123,29 @@ public final class BulkIngestMapFileLoader implements Runnable { private volatile boolean running; private ExecutorService executor; private JobObservable jobObservable; + private int sleepTime; + private int failSleepTime; + private boolean writeStats; + private ImportMode importMode; + + public enum ImportMode { + /** + * Accumulo's 1.x bulk api will be used to import rfiles. + */ + @Deprecated + V1, + /** + * Accumulo's 2.x bulk api will be used to import rfiles. All rfile-to-tablet mappings are computed locally within the {@link BulkIngestMapFileLoader} + * JVM upon import. This will incur an import latency cost that is proportional to the size/number of rfiles to be imported and the number of tablets to + * be targeted + */ + V2_LOCAL_MAPPING, + /** + * Accumulo's 2.x bulk api will be used to import rfiles. All rfile-to-tablet mappings are determined from precomputed + * {@link org.apache.accumulo.core.data.LoadPlan} files created in {@link MultiRFileOutputFormatter} + */ + V2_LOAD_PLANNING + } public static void main(String[] args) throws AccumuloSecurityException, IOException, NoSuchMethodException { @@ -316,6 +356,8 @@ public static void main(String[] args) throws AccumuloSecurityException, IOExcep } } + BULK_IMPORT_MODE = conf.getEnum(BULK_IMPORT_MODE_CONFIG, ImportMode.V1); + log.info("Set sleep time to " + SLEEP_TIME + "ms"); log.info("Will wait to bring map files online if there are more than " + MAJC_THRESHOLD + " running or queued major compactions."); log.info("Will not bring map files online unless at least " + MAJC_WAIT_TIMEOUT + "ms have passed since last time."); @@ -330,6 +372,7 @@ public static void main(String[] args) throws AccumuloSecurityException, IOExcep log.info("Using " + jobtracker + " as the jobtracker"); log.info("Using " + SHUTDOWN_PORT + " as the shutdown port"); log.info("Using " + (FIFO ? "FIFO" : "LIFO") + " processing order"); + log.info("Using " + BULK_IMPORT_MODE + " bulk import mode"); for (String[] s : properties) { conf.set(s[0], s[1]); @@ -357,7 +400,8 @@ public static void main(String[] args) throws AccumuloSecurityException, IOExcep String passwordStr = PasswordConverter.parseArg(args[5]); BulkIngestMapFileLoader processor = new BulkIngestMapFileLoader(workDir, jobDirPattern, instanceName, zooKeepers, user, new PasswordToken(passwordStr), - seqFileHdfs, srcHdfs, destHdfs, jobtracker, tablePriorities, conf, SHUTDOWN_PORT, numHdfsThreads, jobObservers); + seqFileHdfs, srcHdfs, destHdfs, jobtracker, tablePriorities, conf, SHUTDOWN_PORT, numHdfsThreads, jobObservers, SLEEP_TIME, + FAILURE_SLEEP_TIME, INGEST_METRICS, BULK_IMPORT_MODE); Thread t = new Thread(processor, "map-file-watcher"); t.start(); } @@ -365,18 +409,18 @@ public static void main(String[] args) throws AccumuloSecurityException, IOExcep public BulkIngestMapFileLoader(String workDir, String jobDirPattern, String instanceName, String zooKeepers, String user, PasswordToken passToken, URI seqFileHdfs, URI srcHdfs, URI destHdfs, String jobtracker, Map tablePriorities, Configuration conf) { this(workDir, jobDirPattern, instanceName, zooKeepers, user, passToken, seqFileHdfs, srcHdfs, destHdfs, jobtracker, tablePriorities, conf, - SHUTDOWN_PORT, 1, Collections.emptyList()); + SHUTDOWN_PORT, 1, Collections.emptyList(), SLEEP_TIME, FAILURE_SLEEP_TIME, INGEST_METRICS, BULK_IMPORT_MODE); } public BulkIngestMapFileLoader(String workDir, String jobDirPattern, String instanceName, String zooKeepers, String user, PasswordToken passToken, URI seqFileHdfs, URI srcHdfs, URI destHdfs, String jobtracker, Map tablePriorities, Configuration conf, int shutdownPort) { this(workDir, jobDirPattern, instanceName, zooKeepers, user, passToken, seqFileHdfs, srcHdfs, destHdfs, jobtracker, tablePriorities, conf, shutdownPort, - 1, Collections.emptyList()); + 1, Collections.emptyList(), SLEEP_TIME, FAILURE_SLEEP_TIME, INGEST_METRICS, BULK_IMPORT_MODE); } public BulkIngestMapFileLoader(String workDir, String jobDirPattern, String instanceName, String zooKeepers, String user, PasswordToken passToken, URI seqFileHdfs, URI srcHdfs, URI destHdfs, String jobtracker, Map tablePriorities, Configuration conf, int shutdownPort, - int numHdfsThreads, List jobObservers) { + int numHdfsThreads, List jobObservers, int sleepTime, int failSleepTime, boolean writeStats, ImportMode importMode) { this.conf = conf; this.tablePriorities = tablePriorities; this.workDir = new Path(workDir); @@ -390,6 +434,10 @@ public BulkIngestMapFileLoader(String workDir, String jobDirPattern, String inst this.destHdfs = destHdfs; this.jobtracker = jobtracker; this.running = true; + this.sleepTime = sleepTime; + this.failSleepTime = failSleepTime; + this.writeStats = writeStats; + this.importMode = importMode; this.executor = Executors.newFixedThreadPool(numHdfsThreads > 0 ? numHdfsThreads : 1); try { this.jobObservable = new JobObservable(seqFileHdfs != null ? getFileSystem(seqFileHdfs) : null); @@ -468,7 +516,7 @@ public void run() { if (takeOwnershipJobDirectory(srcJobDirectory)) { processedDirectories.add(srcJobDirectory); Path mapFilesDir = new Path(srcJobDirectory, "mapFiles"); - if (INGEST_METRICS) { + if (writeStats) { reporter.getCounter("MapFileLoader.StartTimes", srcJobDirectory.getName()).increment(System.currentTimeMillis()); } Path dstJobDirectory = srcJobDirectory; @@ -508,7 +556,7 @@ public void run() { } else { log.warn("Failed to mark " + dstJobDirectory + " as failed. Sleeping in case this was a transient failure."); try { - Thread.sleep(FAILURE_SLEEP_TIME); + Thread.sleep(failSleepTime); } catch (InterruptedException ie) { log.warn("Interrupted while sleeping.", ie); } @@ -907,19 +955,41 @@ public void run() { // Ensure all of the files put just under tableDir.... collapseDirectory(); - // create the failures directory - String failuresDir = mapFilesDir + "/failures/" + tableName; - Path failuresPath = new Path(failuresDir); - FileSystem fileSystem = FileSystem.get(srcHdfs, new Configuration()); - if (fileSystem.exists(failuresPath)) { - log.fatal("Cannot bring map files online because a failures directory already exists: " + failuresDir); - throw new IOException("Cannot bring map files online because a failures directory already exists: " + failuresDir); - } - fileSystem.mkdirs(failuresPath); - // import the directory log.info("Bringing Map Files online for " + tableName); - accumuloClient.tableOperations().importDirectory(tableName, tableDir.toString(), failuresDir, false); + + // @formatter:off + switch (importMode) { + case V1: + // create the failures directory + String failuresDir = mapFilesDir + "/failures/" + tableName; + Path failuresPath = new Path(failuresDir); + FileSystem fileSystem = FileSystem.get(srcHdfs, new Configuration()); + if (fileSystem.exists(failuresPath)) { + log.fatal("Cannot bring map files online because a failures directory already exists: " + failuresDir); + throw new IOException("Cannot bring map files online because a failures directory already exists: " + failuresDir); + } + fileSystem.mkdirs(failuresPath); + accumuloClient.tableOperations() + .importDirectory(tableName, tableDir.toString(), failuresDir, false); + break; + case V2_LOCAL_MAPPING: + accumuloClient.tableOperations().importDirectory(tableDir.toString()) + .to(tableName) + .ignoreEmptyDir(true) + .tableTime(false).load(); + break; + case V2_LOAD_PLANNING: + accumuloClient.tableOperations().importDirectory(tableDir.toString()) + .to(tableName) + .plan(getLoadPlan()) + .ignoreEmptyDir(true) + .tableTime(false).load(); + break; + default: + throw new RuntimeException("Unsupported import mode " + importMode); + } + // @formatter:on log.info("Completed bringing map files online for " + tableName); validateComplete(); } catch (Exception e) { @@ -933,6 +1003,28 @@ public void run() { } } + private LoadPlan getLoadPlan() throws IOException { + FileSystem fs = FileSystem.get(srcHdfs, new Configuration()); + FileStatus[] loadPlans = fs.globStatus(new Path(tableDir, "loadplan*.json")); + var builder = LoadPlan.builder(); + log.debug("Deserializing load plan for " + tableDir); + for (FileStatus lp : loadPlans) { + try (FSDataInputStream in = fs.open(lp.getPath())) { + byte[] buffer = new byte[(int) lp.getLen()]; + in.readFully(0, buffer); + String s = new String(buffer, StandardCharsets.UTF_8); + // TODO: Use Gson streaming api instead to minimize impact on heap and cpu + builder.addPlan(gson.fromJson(s, LoadPlan.class)); + } + } + LoadPlan lp = builder.build(); + log.debug("Completed deserializing load plan for " + tableDir); + if (log.isTraceEnabled()) { + log.trace("Consolidated LoadPlan for " + tableDir + ": " + gson.toJson(lp)); + } + return lp; + } + private void collapseDirectory() throws IOException { collapseDirectory(tableDir); } @@ -997,7 +1089,8 @@ private String getNextName(String rfile) { private void validateComplete() throws IOException { FileSystem fileSystem = FileSystem.get(srcHdfs, new Configuration()); - if (fileSystem.listStatus(tableDir).length > 0) { + // Make sure all rfiles are processed, disregarding any loadplan*.json files + if (fileSystem.globStatus(new Path(tableDir, "*.rf")).length > 0) { log.fatal("Failed to completely import " + tableDir); throw new IOException("Failed to completely import " + tableDir); } @@ -1261,7 +1354,7 @@ public boolean markDirectoryForCleanup(Path jobDirectory, URI destFs) { } private void writeStats(Path[] jobDirectories) throws IOException { - if (!INGEST_METRICS) { + if (!writeStats) { log.info("ingest metrics disabled"); } else { long now = System.currentTimeMillis(); @@ -1316,10 +1409,25 @@ private Path getCrcFile(Path path) { private void sleep() { try { System.gc(); - Thread.sleep(SLEEP_TIME); + Thread.sleep(sleepTime); } catch (InterruptedException e) { log.warn("Interrupted while sleeping.", e); } } + public static class ByteArrayToBase64TypeAdapter implements JsonSerializer, JsonDeserializer { + Base64.Decoder decoder = Base64.getUrlDecoder(); + Base64.Encoder encoder = Base64.getUrlEncoder(); + + @Override + public byte[] deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context) throws JsonParseException { + return decoder.decode(json.getAsString()); + } + + @Override + public JsonElement serialize(byte[] src, Type typeOfSrc, JsonSerializationContext context) { + return new JsonPrimitive(encoder.encodeToString(src)); + } + } + } diff --git a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java index 1495bf78b82..41fc242d393 100644 --- a/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java +++ b/warehouse/ingest-core/src/main/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatter.java @@ -1,16 +1,28 @@ package datawave.ingest.mapreduce.job; +import static datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.BULK_IMPORT_MODE_CONFIG; import static org.apache.accumulo.core.conf.Property.TABLE_CRYPTO_PREFIX; import java.io.DataOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; +import java.util.RandomAccess; import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.conf.AccumuloConfiguration; @@ -20,6 +32,8 @@ import org.apache.accumulo.core.data.ArrayByteSequence; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.LoadPlan; +import org.apache.accumulo.core.data.LoadPlan.RangeType; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.file.FileOperations; import org.apache.accumulo.core.file.FileSKVIterator; @@ -29,7 +43,10 @@ import org.apache.accumulo.core.spi.crypto.CryptoService; import org.apache.accumulo.core.spi.file.rfile.compression.NoCompression; import org.apache.commons.lang.mutable.MutableInt; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -46,9 +63,13 @@ import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; import datawave.ingest.data.config.ingest.AccumuloHelper; import datawave.ingest.mapreduce.handler.shard.ShardedDataTypeHandler; +import datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.ImportMode; import datawave.marking.MarkingFunctions; import datawave.util.StringUtils; @@ -56,7 +77,21 @@ public class MultiRFileOutputFormatter extends FileOutputFormat writers = null; + /** + *

+ * TableName-LoadPlan map. It would be nice if the LoadPlan API supported adding LoadPlan.Destination elements directly to an existing LoadPlan, so that we + * could just employ {@code Map} here, and let the LoadPlan itself take on the work of adding and deduping destinations as needed. + *

+ */ + protected Map> loadPlans = new ConcurrentHashMap<>(); + protected List> loadPlanFutures = new LinkedList<>(); + protected boolean loadPlanningEnabled = false; protected Map unusedWriterPaths = null; protected Map usedWriterPaths = null; protected Map writerTableNames = null; @@ -176,6 +211,17 @@ public static void addTableToLocalityGroupConfiguration(Configuration conf, Stri conf.set(CONFIGURE_LOCALITY_GROUPS, Joiner.on(",").join(splits, tableName)); } + public static boolean loadPlanningEnabled(Configuration conf) { + return conf.getEnum(BULK_IMPORT_MODE_CONFIG, ImportMode.V1).equals(ImportMode.V2_LOAD_PLANNING); + } + + protected synchronized static TableSplitsCache getSplitsCache(Configuration conf) { + if (splitsCache == null) { + splitsCache = new TableSplitsCache(conf); + } + return splitsCache; + } + /** * Insert a count into the filename. The filename is expected to end with our extension. * @@ -250,9 +296,13 @@ protected void createAndRegisterWriter(String key, String table, Path filename, protected SizeTrackingWriter openWriter(String filename, AccumuloConfiguration tableConf) throws IOException { startWriteTime = System.currentTimeMillis(); - CryptoService cs = CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE, tableConf.getAllCryptoProperties()); + // @formatter:off + CryptoService cs = CryptoFactoryLoader.getServiceForClient( + CryptoEnvironment.Scope.TABLE, tableConf.getAllCryptoProperties()); return new SizeTrackingWriter( - FileOperations.getInstance().newWriterBuilder().forFile(filename, fs, conf, cs).withTableConfiguration(tableConf).build()); + FileOperations.getInstance().newWriterBuilder().forFile(filename, fs, conf, cs).withTableConfiguration(tableConf).build(), + this.loadPlanningEnabled); + // @formatter:on } /** @@ -270,19 +320,201 @@ protected void closeAndUpdateWriter(String key) throws IOException, AccumuloExce // don't bother if we have not created a writer for this key yet if (writer != null) { String table = writerTableNames.get(key); - Path filename = usedWriterPaths.get(key); + Path file = usedWriterPaths.get(key); // don't bother if this writer has not been used yet - if (filename != null) { + if (file != null) { + if (writer.isLoadPlanning()) { + // @formatter:off + loadPlanFutures.add(CompletableFuture + .supplyAsync(() -> createLoadPlan(usedWriterPaths.get(key), writer.rows, table)) + .thenAccept(pair -> addPlan(pair.getKey(), pair.getValue()))); + // @formatter:on + } writer.close(); // pull the index off the filename - filename = removeFileCount(filename); - createAndRegisterWriter(key, table, filename, tableConfigs.get(table)); + file = removeFileCount(file); + createAndRegisterWriter(key, table, file, tableConfigs.get(table)); + } + } + } + + /** + * Creates a {@link LoadPlan} list for the given RFile by mapping its row values to the relevant KeyExtents from the given table. A list is required here + * due to constraints on how Accumulo builds LoadPlan instances (see return value note below) + * + * @param rfile + * RFile path + * @param rfileRows + * Set of rows contained in the RFile + * @param tableName + * Table whose splits are to be examined to create the mapping + * @return Table name-LoadPlan pair for the RFile. + *

+ * Note that the LoadPlan API doesn't currently provide a way to add {@link LoadPlan.Destination} entries (i.e., key extents) to an existing + * LoadPlan instance. The only way to construct the entire plan is to create a list of LoadPlans, one for each extent, and then use + * {@link LoadPlan.Builder#addPlan(LoadPlan)} after the fact to construct a single consolidated plan from that list. This consolidation is performed + * later in {@link #writeLoadPlans(TaskAttemptContext)} + *

+ */ + protected Pair> createLoadPlan(Path rfile, SortedSet rfileRows, String tableName) { + List destinations = new LinkedList<>(); + if (rfileRows != null && !rfileRows.isEmpty()) { + List tableSplits; + try { + tableSplits = getSplitsCache(conf).getSplits(tableName); + } catch (Exception e) { + throw new RuntimeException("Failed to retrieve splits!", e); + } + // @formatter:off + if (tableSplits == null || tableSplits.isEmpty()) { + // RFile examination and mapping are handled later by bulk import client + log.debug("Calculating FILE load plan for " + rfile); + var ke = new KeyExtent(rfileRows.first(), rfileRows.last()); + destinations.add(LoadPlan.builder().loadFileTo( + rfile.getName(), RangeType.FILE, ke.prevEndRow, ke.endRow).build()); + } else { + sanityCheckSplitsList(tableSplits); + // Calculate extent mapping, so that import client can skip examination/mapping at load time + log.debug("Calculating TABLE load plan for " + rfile); + rfileRows.stream().map(row -> findKeyExtent(row, tableSplits)) + .collect(Collectors.toCollection(HashSet::new)) + .forEach(ke -> destinations.add(LoadPlan.builder().loadFileTo( + rfile.getName(), RangeType.TABLE, ke.prevEndRow, ke.endRow).build()) + ); + log.debug("Table load plan completed for file: " + rfile); + } + // @formatter:on + } + return new ImmutablePair<>(tableName, destinations); + } + + /** + * Finds the KeyExtent where the specified row should reside + * + * @param lookupRow + * row value to be mapped + * @param tableSplits + * splits for the table in question + * @return KeyExtent mapping for the given row + */ + static KeyExtent findKeyExtent(Text lookupRow, List tableSplits) { + var ke = new KeyExtent(); + var ceilingIdx = findCeiling(tableSplits, lookupRow); + if (ceilingIdx == -1) { + ke.endRow = null; + ke.prevEndRow = tableSplits.get(tableSplits.size() - 1); + } else { + ke.endRow = tableSplits.get(ceilingIdx); + ke.prevEndRow = (ceilingIdx > 0) ? tableSplits.get(ceilingIdx - 1) : null; + } + return ke; + } + + /** + * Performs binary search on splits to find the index of the first (least) split ≥ to the lookup value + * + * @param splits + * sorted splits list whose implementation is assumed to provide fast random access (i.e., {@link java.util.RandomAccess}) + * @param lookup + * row for which we want to find the ceiling + * @return index of the first split ≥ lookup, or -1 if lookup > than the last split + */ + static int findCeiling(List splits, Text lookup) { + int begin = 0; + int end = splits.size() - 1; + int ceiling = -1; + while (begin <= end) { + int middle = (begin + end) / 2; + if (splits.get(middle).compareTo(lookup) >= 0) { + end = middle - 1; + ceiling = middle; + } else { + begin = middle + 1; } } + return ceiling; + } + + /** + * Ensures that certain assumptions about our list of cached splits is true prior to load planning, since they are dependent on TableSplitsCache + * implementation details + * + * @param splits + * Cached splits list + */ + private static void sanityCheckSplitsList(List splits) { + // TODO: Perhaps modify TableSplitsCache client interface to be more transparent about these? + // @formatter:off + if (!(splits instanceof RandomAccess)) { + throw new IllegalArgumentException("Yikes! We expected splits list to implement fast random access. Found " + + splits.getClass() + " instead. Please investigate, and disable load planning temporarily"); + } + // Hmmm, maybe overkill to verify ordering here, since it's unlikely to ever be unordered + //if (!Ordering.natural().isOrdered(splits)) { + // throw new IllegalArgumentException("Splits list is not sorted!!"); + //} + // @formatter:on + } + + private void persistLoadPlans(TaskAttemptContext context) { + loadPlanFutures.stream().forEach(f -> { + try { + f.get(); + } catch (Exception e) { + log.error("Load planning failed", e); + throw new RuntimeException(e); + } + }); + writeLoadPlans(context); + } + + private void addPlan(String table, List lps) { + if (lps != null && !lps.isEmpty()) { + if (!loadPlans.containsKey(table)) { + loadPlans.put(table, new LinkedList<>()); + } + loadPlans.get(table).addAll(lps); + } + } + + private void writeLoadPlans(TaskAttemptContext context) { + log.debug("Writing bulk load plans to disk for all tables"); + // Consolidate all plans for a table into a single file + List> futures = new LinkedList<>(); + for (Map.Entry> entry : loadPlans.entrySet()) { + var builder = LoadPlan.builder(); + var table = entry.getKey(); + var path = new Path(String.format("%s/%s", workDir, table), getUniqueFile(context, "loadplan", ".json")); + entry.getValue().stream().forEach(plan -> builder.addPlan(plan)); + var loadPlan = builder.build(); + // TODO: Use Gson streaming api instead (JsonWriter) to minimize impact on heap + //@formatter:off + futures.add(CompletableFuture.runAsync(() -> { + try (FSDataOutputStream out = fs.create(path)) { + log.debug("Begin writing load plan for " + path); + out.write(gson.toJson(loadPlan).getBytes(StandardCharsets.UTF_8)); + log.debug("Completed writing load plan for " + path); + } catch (IOException ioe) { + log.error("Failed to write plan for " + path, ioe); + throw new RuntimeException(ioe); + } + })); + //@formatter:on + } + futures.stream().forEach(f -> { + try { + f.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + log.debug("Finished writing bulk load plans to disk"); } public static class SizeTrackingWriter implements FileSKVWriter { private FileSKVWriter delegate; + private boolean loadPlanning; + SortedSet rows; long size = 0; int entries = 0; @@ -310,6 +542,9 @@ public void append(Key key, Value value) throws IOException { entries++; size += key.getLength() + (value == null ? 0 : value.getSize()); delegate.append(key, value); + if (loadPlanning) { + rows.add(key.getRow()); + } } public DataOutputStream createMetaStore(String name) throws IOException { @@ -325,8 +560,16 @@ public long getLength() throws IOException { return getSize(); } - public SizeTrackingWriter(FileSKVWriter delegate) { + public boolean isLoadPlanning() { + return loadPlanning; + } + + public SizeTrackingWriter(FileSKVWriter delegate, boolean loadPlanning) { this.delegate = delegate; + this.loadPlanning = loadPlanning; + if (this.loadPlanning) { + rows = new TreeSet<>(); + } } } @@ -458,6 +701,7 @@ public RecordWriter getRecordWriter(final TaskAttemptContex columnFamilyToLocalityGroup = Maps.newHashMap(); localityGroupToColumnFamilies = Maps.newHashMap(); + loadPlanningEnabled = loadPlanningEnabled(conf); extension = conf.get(FILE_TYPE); if (extension == null || extension.isEmpty()) @@ -557,15 +801,29 @@ public void write(BulkIngestKey key, Value value) throws IOException { } } writer.append(key.getKey(), value); - } @Override public void close(TaskAttemptContext context) throws IOException, InterruptedException { - // Close all of the Map File Writers - for (SizeTrackingWriter writer : writers.values()) { + for (Map.Entry entry : writers.entrySet()) { + var writer = entry.getValue(); + if (writer.isLoadPlanning()) { + var table = writerTableNames.get(entry.getKey()); + var file = usedWriterPaths.get(entry.getKey()); + // @formatter:off + loadPlanFutures.add(CompletableFuture + .supplyAsync(() -> createLoadPlan(file, writer.rows, table)) + .thenAccept(pair -> addPlan(pair.getKey(), pair.getValue())) + ); + // @formatter:on + } writer.close(); } + + if (loadPlanningEnabled) { + persistLoadPlans(context); + } + // To verify the file was actually written successfully, we need to reopen it which will reread // the index at the end and verify its integrity. FileOperations fops = FileOperations.getInstance(); @@ -626,6 +884,9 @@ private SizeTrackingWriter getOrCreateWriter(TaskAttemptContext context, String if (shardLocation == null) { // in this case we have a shard id that has no split. Lets put this in one "extra" file shardLocation = "extra"; + } else { + // Ensure there's no colon + shardLocation = shardLocation.replace(":", "_"); } } // Combine table name with shard location so that we end up @@ -675,4 +936,59 @@ protected Map getShardLocations(String tableName) throws IOExceptio return tableShardLocations.get(tableName); } + + /** + * Simplified representation of an Accumulo KeyExtent, used here to track mapped tablets during LoadPlan creation (and to avoid yet another Accumulo + * "disallowed import") + */ + static class KeyExtent implements Comparable { + Text prevEndRow; + Text endRow; + + KeyExtent(Text prevEndRow, Text endRow) { + this.prevEndRow = prevEndRow; + this.endRow = endRow; + } + + KeyExtent() { + this.prevEndRow = null; + this.endRow = null; + } + + private static final Comparator COMPARATOR = Comparator.comparing(KeyExtent::endRow, Comparator.nullsLast(Text::compareTo)) + .thenComparing(KeyExtent::prevEndRow, Comparator.nullsFirst(Text::compareTo)); + + public Text endRow() { + return endRow; + } + + public Text prevEndRow() { + return prevEndRow; + } + + @Override + public int compareTo(KeyExtent other) { + return COMPARATOR.compare(this, other); + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + KeyExtent keyExtent = (KeyExtent) o; + return Objects.equals(endRow, keyExtent.endRow) && Objects.equals(prevEndRow, keyExtent.prevEndRow); + } + + @Override + public int hashCode() { + return Objects.hash(endRow, prevEndRow); + } + + @Override + public String toString() { + return (prevEndRow == null ? "null" : prevEndRow.toString()) + ";" + (endRow == null ? "null" : endRow.toString()); + } + } } diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoaderTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoaderTest.java index 916d87f7a5d..7498bd25d65 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoaderTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/BulkIngestMapFileLoaderTest.java @@ -1,5 +1,9 @@ package datawave.ingest.mapreduce.job; +import static datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.COMPLETE_FILE_MARKER; +import static datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.FAILED_FILE_MARKER; +import static datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.INPUT_FILES_MARKER; + import java.io.BufferedWriter; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -11,12 +15,27 @@ import java.net.URI; import java.net.URISyntaxException; import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.BatchScanner; +import org.apache.accumulo.core.client.TableExistsException; +import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.security.tokens.PasswordToken; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.minicluster.MiniAccumuloCluster; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; @@ -32,12 +51,16 @@ import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.contrib.java.lang.system.ExpectedSystemExit; import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; import org.powermock.api.easymock.PowerMock; import org.powermock.reflect.Whitebox; @@ -51,18 +74,37 @@ import datawave.ingest.data.config.ingest.BaseIngestHelper; import datawave.ingest.input.reader.EventRecordReader; import datawave.ingest.input.reader.LongLineEventRecordReader; +import datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.ImportMode; @Category(IntegrationTest.class) public class BulkIngestMapFileLoaderTest { + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + protected static final URI FILE_SYSTEM_URI = URI.create("file:///"); protected static final Logger logger = Logger.getLogger(BulkIngestMapFileLoaderTest.class); - protected Level testDriverLevel; - private List systemProperties; + private static final String PASSWORD = "secret"; - private Configuration conf = new Configuration(); + private static final String USER = "root"; + + private static final Authorizations USER_AUTHS = new Authorizations("BAR", "FOO", "PRIVATE", "PUBLIC"); + + private static final String METADATA_TABLE = "metadata"; + private static final String METADATA_RFILE_PATH = "/datawave/rfiles/metadata/I3abcdef01.rf"; + + private static final String SHARD_TABLE = "shard"; + private static final String SHARD_RFILE_PATH = "/datawave/rfiles/shard/I2abcdef01.rf"; + + private static MiniAccumuloCluster cluster; + private static File tmpDir; + private static java.nio.file.Path workPath; + private static java.nio.file.Path flaggedPath; + private static java.nio.file.Path loadedPath; + private static URI metadataRfile; + private static URI shardRfile; @Rule public final ExpectedSystemExit exit = ExpectedSystemExit.none(); @@ -70,6 +112,141 @@ public class BulkIngestMapFileLoaderTest { @Rule public TestLogCollector logCollector = new TestLogCollector.Builder().with(BulkIngestMapFileLoader.class, Level.ALL).build(); + protected Level testDriverLevel; + + private List systemProperties; + private Configuration conf = new Configuration(); + + @BeforeClass + public static void setupClass() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException, IOException, + InterruptedException, URISyntaxException { + tmpDir = temporaryFolder.newFolder(); + cluster = new MiniAccumuloCluster(tmpDir, PASSWORD); + cluster.start(); + + workPath = Paths.get(tmpDir.getAbsolutePath(), "datawave", "ingest", "work"); + Files.createDirectories(workPath); + + flaggedPath = Files.createDirectory(Paths.get(workPath.toString(), "flagged")); + loadedPath = Files.createDirectory(Paths.get(workPath.toString(), "loaded")); + + metadataRfile = BulkIngestMapFileLoaderTest.class.getResource(METADATA_RFILE_PATH).toURI(); + shardRfile = BulkIngestMapFileLoaderTest.class.getResource(SHARD_RFILE_PATH).toURI(); + + try (AccumuloClient client = cluster.createAccumuloClient(USER, new PasswordToken(PASSWORD))) { + if (!client.tableOperations().exists(METADATA_TABLE)) { + client.tableOperations().create(METADATA_TABLE); + } + if (!client.tableOperations().exists(SHARD_TABLE)) { + client.tableOperations().create(SHARD_TABLE); + } + client.securityOperations().changeUserAuthorizations(USER, USER_AUTHS); + } + } + + /** + * Sets up all inputs required to process a completed ingest job (job.complete) against the running MAC + * + * @param jobName + * should uniquely identify the bulk load job to be run + * @param loaderSleepTime + * desired sleep time (in ms) for the bulk loader + * + * @return BulkIngestMapFileLoader instance for running the job + * @throws IOException + */ + private BulkIngestMapFileLoader setupJobComplete(String jobName, int loaderSleepTime) throws IOException { + + Assert.assertFalse("jobName can't be null/empty", jobName == null || jobName.isEmpty()); + + java.nio.file.Path metaSrc, metaDest, shardSrc, shardDest, inputFilesPath, inputFile, jobPathsFile; + + String parentDir = workPath.toString(); + String mapFilesDir = "mapFiles"; + + Assert.assertFalse(jobName + " directory already exists", Files.exists(Paths.get(parentDir, jobName))); + Assert.assertFalse(jobName + " flagged directory already exists", Files.exists(Paths.get(flaggedPath.toString(), jobName))); + Assert.assertFalse(jobName + " loaded directory already exists", Files.exists(Paths.get(loadedPath.toString(), jobName))); + + // Copy metadata rfile into jobName/mapFiles/DW_METADATA_TABLE dir + metaSrc = Paths.get(metadataRfile); + metaDest = Files.createDirectories(Paths.get(parentDir, jobName, mapFilesDir, METADATA_TABLE)); + Files.copy(metaSrc, Paths.get(metaDest.toString(), metaSrc.getFileName().toString())); + + // Copy shard rfile into jobName/mapFiles/DW_SHARD_TABLE dir + shardSrc = Paths.get(shardRfile); + shardDest = Files.createDirectories(Paths.get(parentDir, jobName, mapFilesDir, SHARD_TABLE)); + Files.copy(shardSrc, Paths.get(shardDest.toString(), shardSrc.getFileName().toString())); + + // Create 'job.paths' marker and associated dummy input file... + inputFilesPath = Files.createDirectory(Paths.get(flaggedPath.toString(), jobName)); + inputFile = Files.createFile(Paths.get(inputFilesPath.toString(), "dummy")); + jobPathsFile = Files.createFile(Paths.get(parentDir, jobName, INPUT_FILES_MARKER)); + Files.write(jobPathsFile, inputFile.toString().getBytes(StandardCharsets.UTF_8)); + + // Create 'job.complete' marker + Files.createFile(Paths.get(parentDir, jobName, COMPLETE_FILE_MARKER)); + + // @formatter:off + return new BulkIngestMapFileLoader( + workPath.toString(), + "*", + cluster.getInstanceName(), + cluster.getZooKeepers(), + USER, + new PasswordToken(PASSWORD), + tmpDir.toURI(), + tmpDir.toURI(), + tmpDir.toURI(), + null, + new HashMap<>(), + conf, + 0, + 1, + new ArrayList<>(), + loaderSleepTime, + loaderSleepTime, + false, + ImportMode.V2_LOCAL_MAPPING); + // @formatter:on + } + + private void verifyImportedData() throws TableNotFoundException { + + long shardKeyCount = 0; + long metaKeyCount = 0; + + Collection ranges = Collections.singleton(new Range()); + try (AccumuloClient client = cluster.createAccumuloClient(USER, new PasswordToken(PASSWORD))) { + // Count shard keys + BatchScanner scanner = client.createBatchScanner(SHARD_TABLE, USER_AUTHS); + scanner.setRanges(ranges); + Iterator it = scanner.iterator(); + while (it.hasNext()) { + it.next(); + shardKeyCount++; + } + scanner.close(); + + // Count metadata keys + scanner = client.createBatchScanner(METADATA_TABLE, USER_AUTHS); + scanner.setRanges(ranges); + it = scanner.iterator(); + while (it.hasNext()) { + it.next(); + metaKeyCount++; + } + scanner.close(); + } + Assert.assertEquals("Unexpected number of shard entries", 16301, shardKeyCount); + Assert.assertEquals("Unexpected number of metadata entries", 380, metaKeyCount); + } + + @AfterClass + public static void teardownClass() throws IOException { + cluster.close(); + } + @Test public void testShutdownPortAlreadyInUse() throws IOException { exit.expectSystemExitWithStatus(-3); @@ -540,6 +717,108 @@ public void testMainWithAllOptionalArgs() throws IOException, InterruptedExcepti } } + /** + * Use MAC to verify that bulk loader actually loads rfiles into shard, metadata tables successfully + */ + @Test + public void testLoaderWithMiniAccumuloCluster() { + BulkIngestMapFileLoaderTest.logger.info("testLoaderWithMiniAccumuloCluster called..."); + + List log = logCollector.getMessages(); + Assert.assertTrue("Unexpected log messages", log.isEmpty()); + + BulkIngestMapFileLoader processor = null; + try { + processor = setupJobComplete("job1", 1000); + new Thread(processor, "map-file-watcher").start(); + + // Wait up to 30 secs for the bulk loader to log completion + for (int i = 1; i <= 15; i++) { + Thread.sleep(2000); + if (log.contains("Marking 1 sequence files from flagged to loaded")) { + break; + } + } + + Assert.assertTrue("Unexpected log output", log.contains("Bringing Map Files online for " + METADATA_TABLE)); + Assert.assertTrue("Unexpected log output", log.contains("Bringing Map Files online for " + SHARD_TABLE)); + Assert.assertTrue("Unexpected log output", log.contains("Completed bringing map files online for " + METADATA_TABLE)); + Assert.assertTrue("Unexpected log output", log.contains("Completed bringing map files online for " + SHARD_TABLE)); + Assert.assertTrue("Unexpected log output", log.contains("Marking 1 sequence files from flagged to loaded")); + + verifyImportedData(); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + BulkIngestMapFileLoaderTest.logger.info("testLoaderWithMiniAccumuloCluster completed."); + if (processor != null) { + processor.shutdown(); + } + } + } + + /** + * Use MAC to verify that bulk loader fails as expected in the face of invalid rfile(s) + */ + @Test + public void testLoadFailedWithMiniAccumuloCluster() { + BulkIngestMapFileLoaderTest.logger.info("testLoadFailedWithMiniAccumuloCluster called..."); + + List log = logCollector.getMessages(); + Assert.assertTrue("Unexpected log messages", log.isEmpty()); + + String jobName = "job2"; + + java.nio.file.Path metaRfile, failedMarker; + + // expected marker file + failedMarker = Paths.get(workPath.toString(), jobName, FAILED_FILE_MARKER); + + BulkIngestMapFileLoader processor = null; + try { + // Create/configure 'job2' + processor = setupJobComplete(jobName, 500); + + // rfile to corrupt... + metaRfile = Paths.get(workPath.toString(), jobName, "mapFiles", METADATA_TABLE, "I3abcdef01.rf"); + + Assert.assertTrue("metadata rfile is missing after setup", Files.exists(metaRfile)); + + // Write invalid content... + Files.delete(metaRfile); + Files.createFile(metaRfile); + Files.write(metaRfile, "Invalid rfile content here".getBytes(StandardCharsets.UTF_8)); + + String expectedMsg = "Error importing files into table " + METADATA_TABLE + " from directory file:" + + Paths.get(workPath.toString(), jobName, "mapFiles"); + + // Start the loader + new Thread(processor, "map-file-watcher").start(); + + // Wait up to 30 secs for the bulk loader to log the failure + for (int i = 1; i <= 10; i++) { + Thread.sleep(3000); + if (log.contains(expectedMsg)) { + break; + } + } + + Assert.assertTrue("Unexpected log output", log.contains("Bringing Map Files online for " + METADATA_TABLE)); + Assert.assertTrue("Unexpected log output", log.contains(expectedMsg)); + Assert.assertTrue("Bad metadata rfile should have remained in the job dir: " + metaRfile, Files.exists(metaRfile)); + Assert.assertTrue("Missing 'job.failed' marker after failed import", Files.exists(failedMarker)); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + BulkIngestMapFileLoaderTest.logger.info("testLoadFailedWithMiniAccumuloCluster completed."); + if (processor != null) { + processor.shutdown(); + } + } + } + @Test public void testMainWithAllOptionalArgsNoTablePriorites() throws IOException, InterruptedException { @@ -1909,7 +2188,7 @@ public void testTakeOwnershipJobDirectoryHappyPath() throws Exception { String filePath = String.format("%s%s", url.toString(), BulkIngestMapFileLoader.LOADING_FILE_MARKER); exists.put(filePath, Boolean.TRUE); - filePath = String.format("%s%s", url.toString(), BulkIngestMapFileLoader.COMPLETE_FILE_MARKER); + filePath = String.format("%s%s", url.toString(), COMPLETE_FILE_MARKER); exists.put(filePath, Boolean.FALSE); BulkIngestMapFileLoaderTest.WrappedLocalFileSystem fs = new BulkIngestMapFileLoaderTest.WrappedLocalFileSystem(createMockInputStream(), @@ -1957,7 +2236,7 @@ public void testTakeOwnershipJobDirectoryFailedOwnershipExchangeLoading() throws Map existsResults = new HashMap<>(); String filePath = String.format("%s%s", url, BulkIngestMapFileLoader.LOADING_FILE_MARKER); existsResults.put(filePath, Boolean.FALSE); - filePath = String.format("%s%s", url, BulkIngestMapFileLoader.COMPLETE_FILE_MARKER); + filePath = String.format("%s%s", url, COMPLETE_FILE_MARKER); existsResults.put(filePath, Boolean.TRUE); BulkIngestMapFileLoaderTest.WrappedLocalFileSystem fs = new BulkIngestMapFileLoaderTest.WrappedLocalFileSystem(createMockInputStream(), @@ -2008,7 +2287,7 @@ public void testTakeOwnershipJobDirectoryFailedOwnershipExchangeComplete() throw Map existsResults = new HashMap<>(); String filePath = String.format("%s%s", url, BulkIngestMapFileLoader.LOADING_FILE_MARKER); existsResults.put(filePath, Boolean.TRUE); - filePath = String.format("%s%s", url, BulkIngestMapFileLoader.COMPLETE_FILE_MARKER); + filePath = String.format("%s%s", url, COMPLETE_FILE_MARKER); existsResults.put(filePath, Boolean.TRUE); BulkIngestMapFileLoaderTest.WrappedLocalFileSystem fs = new BulkIngestMapFileLoaderTest.WrappedLocalFileSystem(createMockInputStream(), diff --git a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java index 09c31d5e690..be7ea9df7c4 100644 --- a/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java +++ b/warehouse/ingest-core/src/test/java/datawave/ingest/mapreduce/job/MultiRFileOutputFormatterTest.java @@ -1,5 +1,9 @@ package datawave.ingest.mapreduce.job; +import static datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.BULK_IMPORT_MODE_CONFIG; +import static datawave.ingest.mapreduce.job.MultiRFileOutputFormatter.findKeyExtent; +import static org.junit.Assert.assertEquals; + import java.io.DataOutputStream; import java.io.IOException; import java.net.URISyntaxException; @@ -10,12 +14,16 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.Collectors; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.LoadPlan; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.file.FileSKVWriter; import org.apache.commons.codec.binary.Base64; @@ -40,6 +48,8 @@ import org.powermock.api.easymock.PowerMock; import datawave.ingest.data.config.ingest.AccumuloHelper; +import datawave.ingest.mapreduce.job.BulkIngestMapFileLoader.ImportMode; +import datawave.ingest.mapreduce.job.MultiRFileOutputFormatter.KeyExtent; import datawave.util.TableName; public class MultiRFileOutputFormatterTest { @@ -141,7 +151,6 @@ public void teardown() { MultiRFileOutputFormatterTest.logger.setLevel(testDriverLevel); Logger.getLogger(MultiRFileOutputFormatter.class).setLevel(uutLevel); - } @Test(expected = IllegalArgumentException.class) @@ -162,6 +171,87 @@ public void testSetCompressionTypeWithBadType() { } } + private Map> tableLoadPlans = new HashMap<>(); + + private ArrayList getSplits() { + var arr = new ArrayList(); + arr.add(new Text("20170601_0")); // 0 + arr.add(new Text("20170601_1")); // 1 + arr.add(new Text("20170601_2")); // 2 + arr.add(new Text("20170601_3")); // 3 + arr.add(new Text("20170601_4")); // 4 + arr.add(new Text("20170601_5")); // 5 + arr.add(new Text("20170601_6")); // 6 + arr.add(new Text("20170601_7")); // 7 + arr.add(new Text("20170601_8")); // 8 + arr.add(new Text("20170601_9")); // 9 + arr.add(new Text("20170602_0")); // 10 + arr.add(new Text("20170602_1")); // 11 + arr.add(new Text("20170602_2")); // 12 + arr.add(new Text("20170602_3")); // 13 + arr.add(new Text("20170602_4")); // 14 + arr.add(new Text("20170602_5")); // 15 + arr.add(new Text("20170602_6")); // 16 + arr.add(new Text("20170602_7")); // 17 + arr.add(new Text("20170602_8")); // 18 + arr.add(new Text("20170602_9")); // 19 + arr.add(new Text("20170602_9a")); // 20 + arr.add(new Text("20170602_9b")); // 21 + arr.add(new Text("20170602_9c")); // 22 + arr.add(new Text("20170603_0")); // 23 + arr.add(new Text("20170603_0a")); // 24 + arr.add(new Text("20170603_0b")); // 25 + arr.add(new Text("20170603_0c")); // 26 + arr.add(new Text("20170603_1")); // 27 + arr.add(new Text("20170603_2")); // 28 + arr.add(new Text("20170603_3")); // 29 + arr.add(new Text("20170603_4")); // 30 + arr.add(new Text("20170603_5")); // 31 + arr.add(new Text("20170603_6")); // 32 + arr.add(new Text("20170603_7")); // 34 + arr.add(new Text("20170603_8")); // 35 + arr.add(new Text("20170603_9")); // 36 + return arr; + } + + @Test + public void testPlanning() { + SortedSet rfileRows = new TreeSet<>(); + rfileRows.add(new Text("20160602_0")); + rfileRows.add(new Text("20170601_0")); + rfileRows.add(new Text("20170601_1")); + rfileRows.add(new Text("20170602_1")); + rfileRows.add(new Text("20170602_0a1")); + rfileRows.add(new Text("20170602_0a11")); + rfileRows.add(new Text("20170602_0a111")); + rfileRows.add(new Text("20170602_0b1")); + rfileRows.add(new Text("20170602_0c1")); + rfileRows.add(new Text("20170603_0")); + rfileRows.add(new Text("20170603_0a11")); + rfileRows.add(new Text("20170603_0a12")); + rfileRows.add(new Text("20170603_0b")); + rfileRows.add(new Text("20170603_0c")); + rfileRows.add(new Text("20170603_0d")); + rfileRows.add(new Text("20170601_9")); + rfileRows.add(new Text("20200601_9")); + + Set expectedExtents = new HashSet<>(); + expectedExtents.add(new KeyExtent(new Text("20170601_0"), new Text("20170601_1"))); + expectedExtents.add(new KeyExtent(new Text("20170601_8"), new Text("20170601_9"))); + expectedExtents.add(new KeyExtent(new Text("20170602_0"), new Text("20170602_1"))); + expectedExtents.add(new KeyExtent(new Text("20170603_9"), null)); + expectedExtents.add(new KeyExtent(new Text("20170603_0c"), new Text("20170603_1"))); + expectedExtents.add(new KeyExtent(null, new Text("20170601_0"))); + expectedExtents.add(new KeyExtent(new Text("20170602_9c"), new Text("20170603_0"))); + expectedExtents.add(new KeyExtent(new Text("20170603_0a"), new Text("20170603_0b"))); + expectedExtents.add(new KeyExtent(new Text("20170603_0b"), new Text("20170603_0c"))); + + List tableSplits = getSplits(); + Set extents = rfileRows.stream().map(row -> findKeyExtent(row, tableSplits)).collect(Collectors.toCollection(HashSet::new)); + + assertEquals(expectedExtents, extents); + } + @Test public void testSetCompressionType() { @@ -179,25 +269,25 @@ public void testSetCompressionType() { String expected = "snappy"; MultiRFileOutputFormatter.setCompressionType(createMockConfiguration(), expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatterTest.mockedConfiguration.get(compressionKey)); expected = "lzo"; MultiRFileOutputFormatter.setCompressionType(createMockConfiguration(), expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatterTest.mockedConfiguration.get(compressionKey)); expected = "gz"; MultiRFileOutputFormatter.setCompressionType(createMockConfiguration(), expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatterTest.mockedConfiguration.get(compressionKey)); expected = "none"; MultiRFileOutputFormatter.setCompressionType(createMockConfiguration(), expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatterTest.mockedConfiguration.get(compressionKey)); } finally { @@ -216,7 +306,7 @@ public void testGetCompressionTypeWithoutSetting() { String expected = "gz"; - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatter.getCompressionType(createMockConfiguration())); } finally { @@ -237,25 +327,25 @@ public void testGetCompressionType() { String expected = "snappy"; MultiRFileOutputFormatter.setCompressionType(conf, expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatter.getCompressionType(conf)); expected = "lzo"; MultiRFileOutputFormatter.setCompressionType(conf, expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatter.getCompressionType(conf)); expected = "gz"; MultiRFileOutputFormatter.setCompressionType(conf, expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatter.getCompressionType(conf)); expected = "none"; MultiRFileOutputFormatter.setCompressionType(conf, expected); - Assert.assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, + assertEquals("MultiRFileOutputFormatter.setCompressionType failed to set compression type", expected, MultiRFileOutputFormatter.getCompressionType(conf)); } finally { @@ -287,7 +377,7 @@ public void testSetFileType() { Assert.assertTrue("MultiRFileOutputFormatter#setFileType failed to add a File Type property", MultiRFileOutputFormatterTest.mockedConfiguration.containsKey(typeKey)); - Assert.assertEquals("MultiRFileOutputFormatter#setFileType failed to retain the expected File Type property value", "fileType", + assertEquals("MultiRFileOutputFormatter#setFileType failed to retain the expected File Type property value", "fileType", MultiRFileOutputFormatterTest.mockedConfiguration.get(typeKey)); } finally { @@ -323,13 +413,11 @@ public void testSetAccumuloConfiguration() { String passwordKey = String.format("%s.password", MultiRFileOutputFormatter.class.getName()); String zooKeeperKey = String.format("%s.zookeepers", MultiRFileOutputFormatter.class.getName()); - Assert.assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected instance value.", instance, - conf.get(instanceKey)); - Assert.assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected username value.", username, - conf.get(usernameKey)); - Assert.assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected password value.", + assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected instance value.", instance, conf.get(instanceKey)); + assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected username value.", username, conf.get(usernameKey)); + assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected password value.", new String(Base64.encodeBase64(password)), conf.get(passwordKey)); - Assert.assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected zookeepers value.", zookeepers, + assertEquals("MultiRFileOutputFormatter#setAccumuloConfiguration failed to retain the expected zookeepers value.", zookeepers, conf.get(zooKeeperKey)); } finally { @@ -421,8 +509,7 @@ public long getLength() throws IOException { @Override public void append(Key key, Value value) throws IOException {} - }); - + }, false); } }; @@ -434,7 +521,7 @@ public void before() { conf = new Configuration(); conf.set("mapred.output.dir", "/tmp"); conf.set(SplitsFile.CONFIGURED_SHARDED_TABLE_NAMES, TableName.SHARD); - + conf.setEnum(BULK_IMPORT_MODE_CONFIG, ImportMode.V2_LOAD_PLANNING); } @Test @@ -490,8 +577,10 @@ public void testRFileFileSizeLimitWithFilePerShardLoc() throws IOException, Inte public void testRFileFileSizeLimit() throws IOException, InterruptedException { // each key we write is 16 characters total, so a limit of 32 should allow two keys per file MultiRFileOutputFormatter.setRFileLimits(conf, 0, 32); - RecordWriter writer = createWriter(formatter, conf); + TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID(new TaskID(new JobID(JOB_ID, 1), TaskType.MAP, 1), 1)); + RecordWriter writer = createWriter(formatter, context); writeShardPairs(writer, 3); + // writer.close(context); assertNumFileNames(4); assertFileNameForShardIndex(0); expectShardFiles(3); @@ -528,6 +617,11 @@ private RecordWriter createWriter(MultiRFileOutputFormatter return formatter.getRecordWriter(context); } + private RecordWriter createWriter(MultiRFileOutputFormatter formatter, TaskAttemptContext context) + throws IOException, InterruptedException { + return formatter.getRecordWriter(context); + } + private void assertFileNameForShard(int index, String prefix, int shardId) { Assert.assertTrue(filenames.get(index).endsWith("/shard/" + prefix + "-m-00001_" + shardId + ".rf")); } @@ -537,6 +631,6 @@ private void writeShardEntry(RecordWriter writer, int shard } private void assertNumFileNames(int expectedNumFiles) { - Assert.assertEquals(filenames.toString(), expectedNumFiles, filenames.size()); + assertEquals(filenames.toString(), expectedNumFiles, filenames.size()); } } diff --git a/warehouse/ingest-core/src/test/resources/datawave/rfiles/metadata/I3abcdef01.rf b/warehouse/ingest-core/src/test/resources/datawave/rfiles/metadata/I3abcdef01.rf new file mode 100644 index 0000000000000000000000000000000000000000..577df6b96ce6fcb439a29e1650d66e24babb2e5c GIT binary patch literal 2153 zcmV-v2$uKJ5z}aqwb!;e-$6i&S%=BqOqOJ zXcvfKSn_IiW_RY}>^HND+>#7KD`JUYO8|HEr z%U8MXvxHGaW*V`BXUC_Pm)W1`&Lh9F%wdd=fav}FXY2Q2gh{|yGiH5yJ$lr1q*}3$ zBM{rqYU{(Ew_OHl_Pm6$ahGYub` zT&OnN^XzrYmu2R2y{B`5Vby0^)CtV1F`p9I>-eAP12u52J zkqkSnBb}ffv!rXvg(L5GCLJZ3FXX~C))ucKWdq4FlFg!I5VKg8bg4*?6Ux@{($D z>Xu;|s8TY^Ix6{#TT57GN8qDhXaIUsxU9qF;wI6yka|`Pmeb=-gqLGAz0wQuh}~rQ zD$)&2wZm{eTe=i1Xwlcdbhv8)qGky$O&PYO^9|dTFvGSqA#80?Ky`X%Kn6y3;y9s? z&Z_{KW7>zf!^NW%8?gYo&zdOOVixTXns_UQi4}PxSVmwnvqVftkTN25%a~t!n8`qj ztLuRKG3rxiR}Tr*A}kct1yj`sz|Te?-U|eY5GVy~ad8`&G91?I)~p9@ z0QIM$v-d7Zr9(q|{83~M(=fA(hn@;1aVpL^G}EPkU1dxZKD4!U*VI$XWI=CpU`3%o z>ymx&sWfF)v7yPxJ^zA0jRnsxRw-OG;JW?u$3=q=aL_VYsHG~JB#%Cr-pP36Vv2n4 zg3?MZq3k-Y^~ENu@P={l~V^6t76_IUC--7H`gBo%d zUS;G*rYN+<3Cz%vNfNA}Jx!e;myC|%(U!zSYhgGv9il_iTB(GK7`PcG{*FwBQ59$0 zly4dVX-wEPG0Oo_s5skvClgj%bh|8FZqVCaSdA2przkh7QZZmBWeU|<9oU18io|Z@ zjnU75G&e~ojCRg^^b$~gU8R^r-(a|v9V){4k5ko>>pghUdno8!;jSqf~H*9&%aQW#e`g}kXG zvpzM8I4d@?x0a5hBHZb_IC#%`-=9mojcPvQ ze&${!Pz|GIjlNcR!$fb|HQ7Lq-7VFuuJ>)>lcBbGs$0zUx6b)_r*+<^;X~p5UfT+++Q0JYl!8>u>wa#L^dlN$q_e~pa_jj0N(yd4*Htd0ArA&f!#b$0{pBoFTmk^yb z)%W%3_;^Xv6^ZnbqJcf}!;Em|t?--zoOZA-9yFY+N;>%c9L%$DwiTw(Iu&*01*{gGfr%A1|}?ff73S0rwSd)AVtIxP{Faz;EPQ1;KH+YmBL4 zcnT9|HGVS5wkR5?Skp1tH5EzUf-~a0yVL74EzszkWdtIes`wUmoKjLf9l-h+25aV9r=dIiuMO9-Pw+E|LF^3?Ru57iy8n9 zcInu#U%<8P}1@c3Ate(O)8r#tvJnVflh;x3-wsMjHTH zRYjG+UoW7|g4Cqm9LPjjyjdQk8-xC8C9Y3AOxt3TrULU(<3-}QarCVD^vU%*`>3!+ zm969p@JsUfFRb6yd_KQ-!o2!v`_Dh<-2VV>#oKmxoKr|HDPdq>-~eJqMks^5EVZaO zGe3_30zkqnsVV8HP`)5YtOUxj1!AU>G$>CRBn49!3*x|JqCuQYD90M4sR+iCXH?bn z^K%Ri_ILJl4Ds~$(|7jw3kmXc3JviO(nVr>x`12}?&<9rAK>Wj8t>~G;^^WS;;4%$ z)Wv9Pe&Y|^`EUT^IC1dn=zvU%&rYp`dRIHq)llEi(%eATzz~Q+3=FKmNYB8)AgYV8 zr#Jk?a07g4G$#|S&O0Qx700K1tNhEz literal 0 HcmV?d00001 diff --git a/warehouse/ingest-core/src/test/resources/datawave/rfiles/shard/I2abcdef01.rf b/warehouse/ingest-core/src/test/resources/datawave/rfiles/shard/I2abcdef01.rf new file mode 100644 index 0000000000000000000000000000000000000000..886bdd374d5e5b3eaa9dbb6f403ceac4148f4ebb GIT binary patch literal 235223 zcmV(yK*q^h74#X=DT1*F>*`yyfiyP~LAvG;-vW=oKM%YA5(A`!qK1e)7#XTUz1QpzXiJm zzeUX7(9vP&ipQq~+XUAfZeEiU_W9MY8WLQPWWHNzSo1_eea4JCe$0s{jydt-6OKFU z?2FGBGvTb$$N#VIGyh{3Js_kQ#ewKDkpBzS9ZOUyC#JU@VI#<4*H+inGE(7EZ>E zbkbR^isp~qv1BabR2R_vvCr-Y9oy-9)737)s~txiq6I~Jy@UR;#NQoEY#Y+d>$7yD zO^xNaB=j|Z9%(4S_87{uW`VL878r;ORO1Z%mw&#d3O`{D=M&KjCF5TDz#+QeWhc|l zM1rDCpUY)>C>;Bz$yml8>QiE0DG8UEf12JYlJ^ZHa#SU(C${5#Nq#3TQOqMMj@iy6 zWSVyx63a1Zs6`N0IPj!{#ELHD+(Dd@+I)f^Rl2!oEAODK93ncON9cUABcT>WaJ)&V zGdDVik*}r%6~P5ZkUVMiM^q+L1dq6y&MI?KM4%*+5Hf!)YgfWy#SRZ-XkO7Kb<^1k zza`gvRJK1VxkMZFB5g*N8f*)T_PCQjxIOkW`=JM0ILV5{Y7!bvB`i3gBpgjV;urkX z!DYEe5&e3R>|j&zj04X*q-DRBM66jV5L|^QMI~ON?=Nk)CkbO&HwE?h2N%3ypYW@l{OC22 z%t_{gPDuY*1i@lAIvLjxJr=9lU!k6rj?9X9HNlgTF`Wxd-7k{kE4dOAsL&*wNMvY) zbP+)`(_aeEcM{H{&x?fs8F1ilaAUm#8ywip<&a{i!LW!&`kNbE7G~H^Qxe(@x+8SI z7LNXnK+7Z{m7q?SY6y9@l?2A*h++s~$9NKshSH&V1=t`7qd7bla&H>v?T|p)C*xW= zt=td;Q6VuSo=Z1S)qv;*n>qUjG{KM4{*+ZB!dv0!MX;@{jNB9E(}E$f#4`W|xC9-z zl;lmhUn=Gb(=C20wDB7G84UXh?v!JS;HkQU=1W|U#_m`uW;(En zHZ3!(7$Qkc2`^zejG_Tqg%X- zcXKo!33`Rb$(7t>4GB4ZBHcV3(1jv&3Ee!sON0Q1;Bk8Ia-}ndYoupLkUhgEV8kR$ zqCM37B6^(7=}`obujeNt_`@U|MZbkaH@%&2kJc*s%i+DIrBELE_=yXF+3|z|yH$Zb zos3&)W3zkMp$Zl3Nzva^k8WwHojj$ZN^NYc)?2I8+SZ!3P#?vwdH!E@d=f^=QCKm3 zc9+sE`Z80K2KtN~z&GfqH zo1A5n7`Z0s=_4K}(Ys?LmL>PM2lXxyMVn@cYowtBVjXD`qDnzuV%1p7DG7%Wd-=uU ze+Zp=fH6n(brKf}84-_&Uye2Jmk|^?&P>4Ms?d#CPmQ(jCDxnEO@F7r3`t>T$QksG zn!v@KgnIKYD-6Zo;j`tY7TJ+~Nsr~11-*Lg{VaD-6Yikt>viUYJU&(I?sbs@P`iX8 zk5Hh9tCDasNdP9u1pbaB6cAyupM}pILH61dn{G&FdPL+cZls4Xxg8Supib}IBGHSE zr@!s;iBMD$N=X8UF!BI?TOluy8$3%l+J!qilo+X-{Vw>mpNVGv-lj!-idI!w@Ar8E z^;NBkr>3U0kD?ffQ#`4(ku8SeQv$ZwBtOgR7e$!3OD;dNRSYRI$V%}m!C+#BIO(u* zb&3-78aDnUKPMmx0{#z&6_lucO}Akej*6=GSbb%^PlaVnE7?PrgtSXxhQgYSIEpCZ7Zpw^bnuGUHfk|{Ems-E`|Uu&!qQ;arzGaW zA~!;5IAKvslv3_j68G|vAKgaOkTKw{;2#+ z{{@sOg7HX%itin}msEreB} z(`7vUwO{C#HApTQ8e^UWLl3k=CA!Z}sG^|MuLctQ13Ki4fY5lL7*9^HmD#&OThPS1 zg;}O~CG*G><}gta5eppCMwdr^Pa*6Q@%puJj~z1iDt>z~prAOYXf}v2L+ms-hCqnX zZI1;CiaRwMERa7qYApWcm&0TbFqy2x!a^GqR8$(HYU6Nyy3js7hqPuOVk_EQu1&G2 zGR1J}Li4AJF4N+)^EIwg)r`F-ej;VTm;;O>^+ts{w4feJc$?`vDm^$=S=C) zLlJ?8ISFSInCnrQHEEAzT_al+Gc;vSZK{56$QMb>4k1TnmhSiKB6BU>G`)oJ&0Maz z5{1#R_G;qLf(Jn~rlMn@wFSSF6vcd~C3pGN7;cP5t2#4; zg1XJa&NiJZq8^w!}(I_O1stfH( z)BY_SBBJN>IkGmi1Z@`5b)$UIlqZ|q%fP5V(L^XB>NO`hQd;etUsPiIOj=+XUlKwp_|{1t%5 zLm&aAt<5F_F>F3L5=}fy+?i0EsHcN#{7_ir^0EAM4?`AS8+Lp_c{HO*vy;(Oz)mEr zC&;b60?n_5VgRZo%^5b$E=DIFpAw_Ithk_D584I56?)LnZNi6)upncvcP}$^A)4;s zkgqMq_O^b(+6)UhSXgGTQ?L~gWyYJw4gpG(dpn2hc0b2FRNZe!cb4i#q7Dn<{T#iu zRr3hJWj^V>wl+KYJK_NCspQQuA|jFhH*##oqPzic;1Vdj3ZzVl*mVYU#}K%+e6l5M zMyY-^tl4y+<&X45t&$F#{(jvrj*z&Bu>IY3UHV4Y?u#=cB%}(RQ)W_%tISYUZ-Rv~ zTrB!jd$J_T5BADKMA;$*L>p~wcHT8ZS7O7^Nf>M1E3-v4L>m(zjjy-c%ET)VX+Ha zlk|w)JEQ1VNyM~`Aitq7Twvp%i(*VYONiZO&-IpH?o;gQcV@TSjbBOa)u-a({{}+@qj!f7*BkU0UlZ2fhq)yU>p7{RcNsJhsTBp!lKY_e) zqD?g`uSytZ5+%J1LzBs@Sz)DJNTs#hmuHj1GF88zwu%SB^D}%xD~__xGc+M>oW==9 zm@{l9Z4?wH=C$MDJz)u`P6DViK>@@INDQS@mFE$T>}EXJ$>UFgL_}pfjVc)-CwaEQWSwD=oS>v4qky^Sn&Z zhM9_#e!f$fpTMPsOO~HdKpU&Jip@c)L}4IUT&i&}F|Re8S_TI8TW^H7?K1&8H`tZf<76izp|wUNGG&^P zuoY;k%h)=bRAix?syCgZ{Hh*Q5)W~vS$Udxp`JML@Kgdm;M@j7(dUdHa#e*g-i#`Wi=rDzAL}*??_Gn# zOy-CvBbI-8Wo}_st6vp#FGxWl6?_zCN}5Ze=ujA$3&Ca+ou+7Ks1{_D)}k6( z;&DVG9jS?7T0Z%M*y)RiQnUvtBr+nxULuq?FDZ%S8D;sB-1J&5XA{q#AXE_TTq5=G zI>i-aJl89=!l)1FZQWJ1?ue(RyGm*9G3wg-FhD9~HF}=7?9@axuF`CDI%Is2CxnI}q<+3wa-_|Bf?;k_EXqZ$ebUdf^!g$dZpGNXFUQJt;kDaWvy%c-9 zfa(<;(*W`pFt{XqHIt_E%k_@LomnIvN+P<)Mq+%lo=z7rRWlhD-J#Y<&=a*QgquVf zt3s5d7l)SeLLI7y(UZ_3GmQ@C$X3OO_5a2r2UU`#W`ZYIh2CW&XL(Ss@gq9mYC6Ga znVg8z%#>MYIm7j;r#-QZ*4{4&YQnnAqF$+Lh6@ru;&PuF{SzBuxPtT!(F+ty%?94= z$aT2h8VW038^IxR015N3Oh8jmW&>K2KQB`i2N6wk$(g4GJ^qN-s_r72y}#lSv8Czd zcNS0564PoVBFc1|$%^?UzF^Q!iEwGVlBfwp+*hJ@+Nv56HnHM0D6Z^iR|6r^r)tzH zT@U*VrK5vs4_jn0N}wmsi9rt9(3`FXy$HZkQ z>I6*KT6LAHvBKv6(kJO00-kz@5RH|S*7tRXG>;Y%e)=FTO%!2&k6t&$Fr67(En2j< zkMNi;q+^~O(S*K^zZ0}Dlu2shF%X`;6i~Vo&%oil5G{XBoSbs>sE=0o;--HmzcR!C zW4qoJih&eyJaZ)>#g+if^Dtl&mYfv%lbK~N`q3<9aE)Z3!<%?8SmFRrK{|N~OdpKB zyd){Q97~KM3>91D_hd>y!?6S!G(!{jTOeOSML43wz_vKjxr~|SiKb%_0gdcZ)!0m0 zYa(@KhE<^%Pu{N#pDOZOn=bK!!O`4)kz^^4ds4u+>*p8JyxZnuAr9dSFUNt*=kBC|*Ye?jxfSlZZV>tgM1KiS(zv zghvsTP?dS6DrPpa7s?DqJ$B?laY@3g$H82;VJG74%JE2Jbc8H&eF{~pN{Kc*+SONB z{>l5(c$s12*E@tlhT8pPt4dhBZzj9um$ooqVJCgO+_q(3BZ+ME7OSa7Fd8wpQAL`c z%TC+l6G!f7_bb{IqsL|iUyjzMFbyZ#Z?d#YefF%Xg6yETTO5Be^DGOpI+)Ipd*#H2 zbuJxTBp7Q{TcYYoF=e!I%;rK{na%BLI6-7%qRePvtqpb+$ps39XZm$vHU3D_RpRZZ z?3;2=yUm1A(Q8l{1~VuVtr;1_jl4vb888xjb^=W?HPWI<$_MO4DXa_D^GBUTj zwL6agJFJC;UuhcS`9+f=b~3v47JCGoUv8)YMYE%+iw5hl-Mkz!P<9w;W`@K4LSu(A zj}Gr-nu6&PoB6U8f(KInWrl7Qk@JAcJO$`B1(`h0rK~6M3LEn+`W@dexphh;P*qtM zX=x31R5jFgHcf8nqqIa-b=YUq3nrY!(Th(i{Ccn_VTsaPMAN4F?IZ!0hJCsn_o*v~ z>E*E4T|l`Eqm4kcRfx@2llkSrXv1Y&f07&4TBALSMZp$|8drpTv8cpaGl)Zz{Wz>F zWe07mC3lGtn<{4|;}b0}=Nb8iH$m4fbxbL@?L>8z`<29V3y5$_bbrW3Sgo*V+XFTt zG*+2}uF=tHBD(Eh?cshk@PCj=`K4NGqIFChY3hEzmY~Ox2xw?9?*Nsov?J$a~vp=L@?&y0x8M@nwbW#(1HFK$Oh z_DV^cVXJ!-qPeoFM?&&u`kb>=lq+i;PD`}=MbyyZa!LyXY&6|+*=ZjU4NiwA*$F+A zMLk&dSU4E*hU}Q*&e48F*p?w~J1U!hP_El9+Ai&~>47ee#B@NkiiKAM6t&YPC@8TT z_A(M|QbFk^B_UfqaHKuYsS>W3qH*3<)mF1G+MH3`QDQ3;!js^HuOoh zz@oXaIYFsFGRUZiLl6-bNI@kJHkbJmceR_8TrG*~@?dHt81{>I!wi@eiMtt4`~H!U zkS`WnSR3J%t(u&py-HAv)&<&uefoEBFeX6uQCq2i<=*uFhosp;4E2I!Ae@7^6rgB+0h(z`2|FPbdkzAMtRJ|)G{A?OV zl%a)0uV$J&1?lA?EKI)%VKN_OGbth%UdK(k!mMsZJgw|(`eVO1J&l*ciwC2m5>%3i z0U9Th5l}=2Y!LmZ;$fBGnp?8V=gU!b4fm)dJ|n?AYPXdn6qT$|TN?53-3YzCtQ?ri zxHI(DR;kq|Hi(ar8N~gVLCi;L~d zbjp;vm}97(ZjaDGkVO>a*F2)zH*@U5I8{stjlHU0NZF>7-I*2g=tB2sZ?>nfdTKJM z!03~OUw#Rlg_Ikkq%E`ry7*ItIi{y@nJE0&9+QuUh*;cX-gGFFx1ma?M&lK+O<<*OL~_m5MNCLPbKq=f}9yVijV*%5i@iNghcP?R#y%~Hj-3iY8Fwh z`&;eRqF91Ek)jE-#capeZ?xiV?y8wm<*V!T1*TM0`f8f> zx^N%Gqj*)J4kRCYhB(Li6td5}hOK!XF4?3bs*Fe%qI2F&#xI!(`ET>*k`BYp=w??A zla6g-YoSJN2o2=XX`vw$onu$YA&oGZgzED%J&Lf)mTJX&7tnomVZRkB(aJOwA!{AQ zp%U7kSK9WCNbIN{J)?om4IrP~)~L_7A#L{!{L;2b~uVDWZQ15$H@xO~EOaiq;o zsmLhZFf<#%1u~4vyo!-%;$m8_;1-@T<*b$mgNb)@=W$DWr&5p|jgSee)F_EPPm7MK zHe)#wXe2AAGQq2JRJ6>Potr%ue-_g#LeW7rkU1VWtqTbV}8_?vtk^z7D1``y;rWX&uAh>Zd z{PGi{W>X0Z3yNr@O%g&}SBcHwa%oIuXB^jjx*n^ylDOY6D-zU%y$8k;QML<1WTrLi zEcPc@#=wVKi(96g!{L&r-LKjy`*dbUy@$eg=L0FlQ%`9GXtv>H@=C-vcNm2 z6qY-ft|n)Ry|!Ggk>$~eIrdbAtK$FgHzRp7aC=!#V$W5`%DkJ+8)Ye)=ny<4QH`+3 z=MaUUrYTe+p0i-q896Cu7oF@{E2Gp=7 z9%)Q|)bg<6u{lsGQ<7uV$pwX~3YmuT`O?C|keI7N>Gmg{FPGM9iir-JLP{lntzL=K zlvn0xcV+5Pco6$*pw}`JiXt=JxF}tSYPoZ$yF*0e#U!4XI*)o=1Q$^MuAELa+E_*O zXljyGuEloBi5LC0k-{Z)W|%z}oS71XzJm)Uo6XFeS>1X>UN7z}=?Ns5&`W{j3U71G z-meao=qyB#tO*^y*<@pe9-FE~y~B@5)qy!a8c|K_+b@OtnCn zK2a~6WUS zrwU`|)Rny=5{Sju*7mC`9y>6eZ{%FMAevq;?saG8=H}-G0(!8+uL!OD4#F0}P{hI} z&o7Hji~YZaIi;gP2>dg3;?+bNo1(dS-l!N1Xabi+GUigjPNu{cK>pp#PH`5!=>{mu z@CqYL#q^`d6OxQ&L>0MHRF60VzZc*!6(H2iD!q2AA_$BnV*!D~OtY9gmiEUIz-7E6 z)E-p#Bx#=4`Ihi(397-~~p*W*uI)-L<#Jo7= z;Lr3S5s3N+=W&0Bn4u{DBS$mCM55|;(;HcwZ5`~j?vq9#z06Q_SX%6_)cPYV#>^R{I{d=c)yMI#S0rzi!H-(_BYB2110pKzMcf&phOj(NEsn90NhoZQ zmW*3yH?#Csk(HZ_8#w1h!l8D}5Ta1aIOkgKd}feN7#;4-3(FtWCC==Mjpn8?2=Bs3 z)bXj}k{G~Xt}P`EpD5`#Tk|NPkf<9AIbZXlIaXq}O~w)~3=LIaE+pexE>$|kKVMHf zGg4?#S_^*5?%bj@2s8XtP+~8O#&(dM2A_yWTky0&VTK50cvLVr5xXK8+M2S{oqmDR3^ zN7A&>9twp#wGSN{&Hc2ZR;y7j z%O%`$jR-XcxeBwH0V?(_QT|LuSd;cm7d4_t#+AIWywDZfLZ92I_=U->WYTAN1KLp<4iIA;(|WG>L~ zMxpk|f2d5NlCO8gV)chM$Dk>|A+os(!NPdCkTChyuK~s zkD2Lc*~P2fPy&gEc_`MH=@(5kiB3`}q3A?Gx)7EhLB9zj_($_^6PCRlM;sfmnV=xs zU|wq!?TKrD>}bNXm8ttvMo>P2PxiUg8Hg!CuOi$lvUB5(QOOs#w56a)wjP$aaj0XG z#LW30*PEn#krMs#RD;;?kEbN6pr))}+FG5!WjGN*o8r4f$Qf~{)e z&0b8psJ9}t^1o&bNGh#TaTqcvM=W2GyGaEfd0IFkb&ybIX{MwnCpB6TqRL*^eRkN; z6d|alIVyQA5x?~5rbnYNvr`-u8p$mik*p^hG$-0DR2<;>bA+NMQK&}}%?k$*x;P?c z6z8|{wPbW#xD42~i5#tFLyLapliIY&io0r3TZ5wet15exHg}7!Ppw0!-^<6 zn}SlF{hQ@6bfMRrfI)Ux>l8N_HVL^%nC(M}R>iSFK+6s!ZE8%yUK}_fu6i?%s34oc zGO;a5lBFaVv{P%z1c9B_r@YTV$S0^ywX%_g@bv^r1aaAHmg*|ebz412t!AIpaIGq$ zYEv3GOFhvlXd7|j3d7J7cQEkU?6BWf^C?#aL#c_^2h&jx(*pr!8+v4*dN}cUFc{4= z6kcJ%)WRZL#k^UOU|1+I$vv7Ii6&x-J)ovcEws~+GdhH^CGpX`oStlsV3XzW6NoH| zX%uVO)FlwGSLp1@WqLo_8(kW?b7r)*h@4S;ZkAa?CmWR;t5{QTEYIyo7e*zh$~1dQ zoSr(-E>iHiOEcL}V$>H4{Q+sb=~!`hOc}9d^ANvcyM2-E z=~-p5Eq~&$S6M)5YqO=l%ItO%&M!@Tt|~W-ZG=ln42=wjeQH;N2K}F5)cleuHuu+t z#m>Jy@dM#fKB$D1*m*rxR4lg4PbMmj?bK}fx~kuvhAB5&8rk|nN!0~;3Nj^;c50}b zA&%dq5~ddW5GX=ZrWjf%!Dct43(MN=PAe;oB&-a2p-Z>Nr;B^d^I4e@Lv2lP$ShYd zhTftD6ZJNdBcsg5{2CG)#UVR6Z_zeL_J-=1ZIA?Jf3&uZ+MkyAlrCGMJ}kkA>D)+B zqVrDeoOeNS)X|*y$wb@w8Ct7kiV<&JA})9(UuWX!{j}@ZhD^Pk!lhYjt75Msvfic& z88qt2qYA!q!$vz);ZJN;ys@FSy`ia(;*Ca>Y(d}RNK0Z5 z=FtjCW$*~@g-|uGJ^Qu5CI*qs6XuPzB;#=?o>rdiK@ zbVWH6lz>`hdkk=?Znv$xq3i87zS>#nvE!yvIwCR7vp8=w(x-ITXoF<_kti97oZ>d7 zfkc%p$%@f_C1PtZG5TtoN(IGSm#%+5IO~qaVCf7*4tu59;EBzyK zwPhF)X)_?_NY$_PL`M$Ts9$Bk)n=ZDGpAK!5?tss!kHg7vh5B4a2ySq~6Q;gX0fR>F-ZS4HuUbj@@?hM`#R0K3Ld9$UGh!L_g z?z(Ie(@wTL9mTeCHe93CXu|>9u8aTQ-Du8|b|pq~TXuqG^228K34>9##)uXr2oU+r zyE|>+N!_r^jiudsqOg&>X_!V1#x^9k?geao%>*g2KNdcFS|OCl#EM(i%#SNH@$rry~1Kl&@yl=bVy=D^$Od zc)B2+|KY)4NC`(g?FoNcUyh@ry&mlJG0!jSu-O7Z+R8FITt^8e=-K5__=wqC-kwx6 z)DBx6yfZa%Db#S%z(!1EYa9SDvL~pu>z(!-8x?xAv3tbMC{f1Inmq*UEQ|Gk#xbUg zH9IOYzl2G08zRP4;qgV&I_#(Lt&c}MqN2R&5%s8&*H*fGkW%Zz z4&9KXvrx;};z-OWJ_}czv!YXp!~|C2SxjA`c4O8CME+>ARM_?;Dsc`Uwc{$$5}oAR z+dx4)2`HU#cDF4Q)o)bLA2#%ks>V8>F{!z#vbv+Qr>mxqBFog{1JL~chfx=H1IeWU zo0@)Pi({ z-j>-&yW-P0^h5`D(Uxbc(R%L|-PMw3T@`O4XxwuT6T%_9>Fw z9cuVfstRV#Svls#Q?mcuSz9A&GX3Jlf>d!nMDJ9^#ePw%(A({|lY$bp3T>t>j`Dc* zNL$P>%y=Fi)MaSmz!=p5 zWQ7fJ{d^8NLn9uK8VZRUebC^eGKOt!&6JGh8meJC$t=!{E&*0ZL25KYJS}}djk~(m zUF)vD*j=p#yR>$#kJe&@{gJB4s^YEnRYiL08k#(P+xO+7AkH+G| z@!{cES&2t$@OBNh*W${0?5f9-CVYP^{x}|MPQ<&X;e*rg-8gI?k1xk#_ZhfO!FL|a z(C|V#z7F7z0KN#}x-f1Cm3d}yUu|f8yuLnnRG`%rQ2QE zP+RA&uCA;VeUG^@hc*%$<~P>YHB>fKH#bz)NH6}e_u{6yW_M+EZF6H~gY@F9o2cl> zRbN|IUs>H)Z}AqJ=vrs4Te$OmqQ$FEQ$<*g>@LYTU=Go7<<`i)AQ>CYo-_qTZg+EC zW3AiWnzZYC?5+4Hj?zIJN7~ZI(Z5lld@bPPOEU{#ZE$ zfACJ>sA1bL-Opz>@J=5!&GPNK<3xI%?21&p(;si1!N09ezNvGw=|-|o8-h17aa9)H zpN#u?Q;WPhi;eh}nwjM_R?D^`6?<#2gYv6#o8)&|6s9+;mtJvu2R~u?O%D3#CNfeV zj32IZVA1ssJiURp{HT%LEfrVS;{~cWmF^eYQJ!4ZBzy6JxOX&u8I7-~J|2xV%`y^A z#*Z`+p}4lWwz|&U=&r7>c2_sZY4tmWKDbGG#d~IdL#L~miO~*b88f&fd(Q{{iFYo* z4;RysR@cet@%mA8Q;)|pxj|986h(UJOVlr8L>FB+S#se3Ji5q%eT(Q+oL;wid5i4w zWXz#16Hc{`uI!R>*b?gQGD6C&S7Tuk9_xn>`s3<>xMm<`4Z>A}uss_O55wbS_?YR0 zYCKwvcWdx;EjBgc(c`e>B)o7M_MCxh&%})yHihs+2-CZ8=M>yC1$SPK@0h@!Io*M0 z7{%>m>wETnzp(YpQU~s1ir|4&4!q6QZ>t@+cP+d2Pi0VFU)@kwSyS(}5ZOyezL(`y zSITMg4JvZUZ*bQ&HB`EzHKfiqM^5>-nHjA4INWU^%738#`O@2@j!x|ghOhHR8d*l)>HRSL9tzxZXGV$p!%pZ&C zL98=yKY#mNw_N`{HXfghClpy-SzRsr&H-3+r30IJ+YPBe?vLF%?p_;bM^f?hDD0=% zT)B;M!FMI^Kqpg_>_^U|2JOxkEC35vY1h<87ZwQ+%$Gy|ze*87__$9KP?2ju4;SWyhxwUc* z-x+s{Hchg|8+lmlz)A<6ci;mD_7B9~dfXR^$NQfiZv6p_c|z8_oQ!2X)_5ZuSJ&a8 zCTty#55{AA4_@erhb^vlVBSDHJ_svEa9)&C^Xj5_pRj|RrnweqZVshkSuo8zMI1B9 zH*+t)!~$ikEuv4Cu`fHe{v-nrZpSJJtcx<)E;POO%7WXhM`xIMm zF6Ct0B-^Axm_7>An8cgMC0?!+h!4Z661;ab?$_~b2d1;Xna|dqRb)L$i98kmEy4Sy zO;likB&X6;A`4OaaS0yhAUN!k-sj;0YfLy8Yx40?6;{;ZqtoIN>U`?Qm0K;B-FqT& zgefymCgISecr1BOZalN`gK!+47=V2Ya=m$5+!L8ufzKz?MY4U$#<~i8TY*CrcxohO zx^Zj7n%z?H`*KHIew#r_g~RLJ4b9CKw#*xC)~}a}kFxPgIcHwEka#O%y%%Y?aTJ!G zhz(crO_MQiDrUD~L7cZ5f~Be0IvVRIVPh{gkWad@PhWkkaS<%rr$N|Lhgaq~@HM@=e7^=^T{dPhLu=m}hc#6Uz(yvu(f(L|6!!PtM{eKUv%+D$n|Zh)1M4PW^9207%z@w8dZ71jWX#zg8z$jE zuZ*%%Asn{J4ypv#XJCF7X4K-#T5M^=XJhd57%V&q&tHOlmtgw!aqYKlHSt*gv9frq zfACT~8g;2M3;Kt(aSoYn=`C?iF9iqUGsN~d3^5Qp#^dYpm>#h{zd?Al9zV`>V9l~P zr#1wK((rB@9_+;Hy*w>fLHFLmw_dhw$(TLFdh640GqAT2GnU6;u~e)+7T=s?eM~9% zXlF;DhTyFx z92$$)PR30O9o9RThnXWW_c-i62_G|bbCB5?_pggX?rFHY2}@4FJ=5bzwM=|G1lM=r z?rz-JgN@7MNTzJ;_y<1!2Uf71e}TQsf(3Ep6jNL6@qphRk_EYL*>3g6DflOLpKSfj z3^O&y)8`MKYE6oR@%kxv^Avn`Dy}%qdMj$NunC`^f;r=`c^qCGhfl|0{pncB=h2VP}5b>T(_Zsma^xrLUBjb~%;+3`ff`;EBr zZ0q9~h-D{Y!Kqjn=TMTdaiX5ndB!9zvCc~I1uz6!#Xg4D}9yYkQ%X)J% zaC<*&?uYyO;e~#<{X}b$%EPoF*qn>ijrjH=9J&|_FToD(H*S<|ZVsLqjO){}JROgw zV^ujeY>K0a2IB4u@yvzz`V#9s9)x|zWBUdNKHC&W_YA@93_@gLK^O;aiKBj!G4I@X zw)#tF;KwuK0p{4gwnuzz$!+!S9`IvPf?wx{z8{^2&9PDbxyHlBTyw-uI zZeTEbQ?E-&PT@a}#@9wXZS|TSPZ!-+gXQD#@OXSQ-kSD?;Lg$5c_BUv;l0&yRQNz_ ztH*QoxLLE_upwAsU|SD%^v8@gE#$y@OeJBpX*u!j}QO%ZTE+rE%@9 zFOI~WYvT}FI(B>Ut`}>kIk0V7JR_|H&lKXdVtm?y^*+p-jD?eN7kinP7=?bb*nw;L z%!}MmSjsccanI!lB@iu}^~epvA56#F2m17Z~u^#w74H3mtfPkpmyH^)|EUekU4sT1WTx zwUU)8yW$AEm4Vxiz*9$I$IU+=)@?3ufm23T;b1M6>h zUbM%huED;;>yI zb~Dgig(oK9ISn^nhW(58{Qxau&>Pr}wg*p`kbMq%MtEKu-{f=5)$nv5kq*l;-> zWUsN~ItPBc*?~0-kAKhB4U`&CFs!n=rcxTWO~z^uy|ufxy4qb`TVE>&d^5*l1DS4x zdG*z@jlis0 ztgOY?wU}FnHDP?i*!Cb>D;di^#`fh9$2Kl^b#=YF*4G>yrCTNxK_WdimpM%YW)dVy{2`)pmu zgxy9a?4DV}1TI@!*E{g+rvF7{7CZSuxc8fC>fKG%)z$J#pQawF&L(#wyV~uRyH;OZ z)5~#gcVl&fTb?qPjBm!$U2EJ8?2PK_<@-OM{E$T>7(+BOE|&YKUSAqV(B>f*3LZ|v zvXihaguScShHd`7NCB+bC-dJWlKu23jCPu84TC%bf$sjgkaT68t6snW|9P-0i1 zM-R5DUOrDw?p&gq=6O|l++<5D9c@Iu7j%~*E$G!cHE%@mYoV|_r|nuZ6`-IXtQtXu zCU3~UXEWV9*Dt@{eCpln(xU0<(QYEL?;VUMndtk1ty}rXG6a2AuCxoqlT)e#!%$zD|iBTqeXhh_EFZ9n}V%#_-&f2<*tmyqv_RTM3#w{`(b?= z7G>h5Bk)Ua9;G} zD#$=ngA~Ul;}0tA00Vn+{<)h5)k;q;0}nF=awZO)i%%8YLur<|7WwEu@I#Vs@X{uH zKMr3i@s4U2jUliIOCO>dgIp;n>W{B|cy$v`I+wk49rh2$@)1~eIOf%0PCZ_%$L_Is zV*+kH7oRKmex?I+7dr4Yvz;GW;=m(!IIO)L$#^2&nyZqrgj{~PQuR0p_W<8A!+SnA zxN|$ENO||aVHoVwSn;v8Ve zo|b|~*K=7ZlM=^cV;XME#9KLdwG3~S;i*wrG8#K-unsiZ`wRBub4{*w7@Uhg95?hOKGV_$?K$4aNHQcwlMoIP;3(jg9q9brz1w z$3pQW!{f(U%V;LZ({P9exDIb_ZfvNv0J&R85EW=atx@lZXlhNw0<7f|!CIup?K+d_ zRkmB@*qVd|X;?Q1*JR^aCvIuNkK?fa415>B6+vvg36U_hV0!VkwJ)3 zTib~#c?&R4QK zjmCz)_-Y7tXW*xDym|@_O~8*H%+@e{5`GBa!^?3Od(`6VIiiY|JxU)ohd0@myOiGf zmo>X2<5OPWC9U*J#e%+Aq{QQx7jmrWoGFBn@z`ZuUmP-1(VZjeYHF$*8{Bdtdx1-W zyn1O4!vOrq6y9%(;!@)~9=->1HMDabKl?^Gm(HQe^bt+&n)+(DH0qLq=U2rMQ7L$W z3x(Wf*>(-aV>NiE22a)E8``w|Kzq2S!I<3G74b-KJg3(#(52t;y(=!6&j69R3`A7m zV!T&|I~(!Q(fD~Ro*s{H#^dF7eA0m{8Nk`YVE(eT{1c@zAO&ynisHO#w~X`#;-1m? zc{IKZar7*&Ho2-E&)4&#wvce^;#yAodpA*j#!KeLk(0^TOpSIBYzWGCdO-$lKwXVo zfPK$>2Xfw9k{Qp4U7AT=1KEkDUG71%7T)Mx8fQwy{JyweiN~@(UK&pe z-Lxf6sv`U5!%4_CHAu^P83}UZHm_M4V@|>J-o24|B~q|)9ybwWuUw3US@?JuZm+}> zHF)AUtUm|K&&AC*IB?er2X?KC1GiFf@LVk77Mt5L_2W*S`XNmMPsWSw{GjA^!rBw# za@1+QEmj;DY=&*9d*oIF>bLa$QWB>$-QPKyV|J|DtATTY!z z$akLCR4X^-mo?#saq&!+M~bnrB%UAROL`&HR!d(g1^2Cu<8tkfRVU)WOoug|V!$^W zcT-!05goPK zsvB#homEn?$$_2R!mX>Wmy^^RXV5citD7wV{2%AWquy2L#-q8-N8&cR2o3H!`JKKX zvdgQkmofGL+{v))j-w`E_;avQj zi=BB`3p_CtdxzouVOT#LyN6@%aC|fzpA5%$BXDR07CCW`6E8aPwi91F@oGN4%*Su} zm|B3>3h;da<`rUDA=VY*wL;ucgvX1puo!bnu(SmClwfZu_LpK-IUXy=JLP!10((c{ zqfz)`6dpPpKOT;!E3u~%-&W%KD(tAjJ5|{IH@y8fY;t2sHQubo^cq}OgXe1SUJV|o z!*g|bs}8@^;ktTkt;d1}JkWq;jab`={f$`Hgx{NRMKhK(V|g=HHskqbe0v1e9EsPD z#QR6$;F0*{DC{~K3;vF+$KaP^vFA7(Iu6eqj|C^-h7+)147QHJQ)BS`N%-Mp>^%h! zor?LVVZ~`!Iu2{c;g9iHdODst1D~IPU(dwSv#|SY96B3UO~f@5amz%knTQ7`V&_DB zat>}d7cZZOPtL>c^YO{~xZ$68?gBi20e-#!D=);>3vv6u@Zi6&?jk&Y5guXK{}Mc+ z;CltrTJTy6c6jlP7t4LvbHtY_=~ zXxM6&15dK`99tjFcHs6o4s2uV6}En3YsOp$X0vr4TOYBt_!7yU$Yg++|E4-r~TQ%N^Kss{^02_2xl?P#tY+`Q){V?2o3qw|d2Bt&)^BXxdIyu=Y|UH8?#H(Q^w^(a$e4{eSktC+f|Akd%$I@{WHx7yZFIuN@x*Rr*Zk>xLp45xB?EVonkF{${t z6!%g)J{YC)p$#+5_kec?!~Ovaj0EIbAudhx0s`zPbTWteq2UjGkPGUc&0T7p5* zxzgeO?y_J@s6+a^Crz`vATJ^nG3+0t~v$VF2jQ>;}T7OT-%7xc(ihu7S=j7*={7`GgE1l zSY%-SM#H^^8j44n)R&Aax=6ZL>4yBh9+-l~^LoifO3|tKqZ)VfN?F;P55&tu@nJ0% zgvhZWa{~;cm%FJ3Sa#b^Z$kE z9r%P{lkeI3iLI#&m;An%vwDZ2%jt5SnYUkLL2@g+#OG2HabgYA1f%d=6JGc`?mrQW z$KsY#F|8Fpn^aho>zC=dftc|(T)}Xt8FMSN5W<6T?I~Dy0Uo{p&$JLh%1P(pFp-Yz zqjIq81WfxEzHY(K-FUkPS8n2aWscF1-S%!bJy`ZNz{gB+55VWZfm3kDzwo9PxAztf z$!^O`n`Zoc1fFBS{wSZHjm2+cv6ad4WlWZTz-0M5 zz4F^i`@CpYju%+rnf}=5$8-|VB~biQDon=bc_da1Rm%G*_<$;oishfk)C$cn95G4t zFzF_xr8GP>7H?{pwUDP_1uO*NPeD2W*>?`W9n9R=#&ehRbiXVIeh|Ls!P2Xlz9s!$ zkk{dpOVK=HJ1vi#An#wop-jXdmY>49TeWSTdN{sh za{4Ij?7;5^{`((1$Sk!FZ*f@5rVM=EiT4A!IYPuxJOqR9-I!T7WqUsU7iIvnKci_yIFI9}q7K87>1N0Cz$ z6I@Ni*g?5tYL4thQWnvnGj#b)-}rnk5Q?V)ockqH3Ev$sCM9{UAVL8mOs!(E=oxy3@Lg;Nf8{eKy z!o|X6kc?~k6LRv)6*+G(F+%e&p%*dag6bM-k3Te``4zcuVaD0yZr5ZlKNxdP$9w1C zzn5Y=52MNu7qb^fV%C4G(Mx~))PS#MaB&fA)nykB#&_2-T*^e}b-b!^sM}p5jgbw& zPwes=dmFLI_BaDKw_d80 zzwB|hF!JBLmZ`^$%!uqwWRjoV>(nncPtLRH`0@yR=)+F-ke7QKRLU++!EL=G7tA>? z;;V*Y-g)@_MhE_2YwtP-R`436RZD;$1ldVi?MhTUhKf!5U&0=_Wm1pUWQ$aGQaI@ z04Wy^x0xZE0x4-FV=*;=8md|NeKN5y4=cvu#q;r6D;5!cDv+by!Pqbo2S?-3=~&^X z{nBM~4AQV@C|+R3Yj3|VnViVPJJTF^pRIjteKw6>E_amP!K<-`2UJ7xTlfK%PU$u8 z<(B)qDY!ld2P-*}4$4*8mUevKPP;B6$W&Z820Je%MG?}x@|*2W$5Y30Uxuo9<#x)F z0mO4&)obBlevUV?4$1Po+Hl_xd^ZG755up+Fn>69{ z#oL$SkN@DxsSa#sfaZpo4&25-@cxAiB1EGR3@YqknDz^{HZsVtV5I{eGx>SV1_$2a z6H1)C%2>)t8Mt>W?qBS{uWaojqAQns43aVLFd}829%+;1PUYarBFwGFS4ZK1f=B;@ zr?`Yfm(ndKz1Pz4M*|jKgsCl9Igcvr43u7g zr7gI#1@krR_G6chr^EO)jM))98^KEvd>z3p>>QtD>v?vLFD++AI9uOK#<- zC$CHUgB!TRLyUF2iaaW@hpH@Ogy0y1xyhKBf~Qlkr$4^vkLT0yTN>sL!7J&Qoq-vd zcs~?p&M3jD#Z!V}1zlMDSGvH*{iUC*JMEcmKw!%WyN3J4$NS3=ytXi&SQv@$)hhB9J-I;w>j*17qYc#sRIYu+R1484UBzg7wF9p( z$n+~)dl+n5#pKjo4C}qY*3?Z7yvWv1Y^~qyz!tVPZQ&=O_`5@L;E{qQGvYYz)3C#Z zr*zyx`m88dYW*Z*(J@4HRt>HGn?YV%8FS`fZVRrv)`4woO^fz3Siv7`k+F5lP-~7K zh@U57{MtmjQT{9oWZHxwlN_(oLF(JPbuWwm9&n10S(-abRX&+|w7Y^u@lu z_@XZk^u>(Bu;4H}br_~6VNMcWO~TJfxGNbCCFApCJkSq!reb?4UQWg4{=}$N+5bt$dl~pB18Xv|HWNQ&VO2Kn$;K<$n9lY#2itQnCl?RoVs|d~85e3OeG zb1^dyH|JqP9^TEvLSPMWH}D{^4|sV5z8!%TPONg`DB!0$HNn_<18$m zh*cBu+(f)N5${gKClm4GL>!oi=g+~r=i;+-@zc4u;yi3R4-cP*XV1e+=i%M+@bP*0 z@jN_vK0ZDlKc0`R|HQi&VE+Y}aUtehh-)vz;}_zE3-RrRxcXn%^)I}A5%yk$=@(Fir~2j9`D5Kop`Ggdpq%ECtm5n>pl3P2m5>QTMrih8|(j#egDQc|He=M z#>1E5*-P>IrFipFymcwAVg}@8xbJdY%gBB26%Nd2GU}!1phUJdO?TiIw*Fvi8N*H= z&UWA%w$?NGwVkcCbC}#=@@pDn(79|aW9v<}-ezk*TT7T++sxK)Y%O9|&`oTuVrv~+ z&$6|Bfdk*MHG|2%4a|0Zo~?as{Sy6OW@T@=f!VZd?PKeAwyt9GaWPvjv-K%k``Nmh z$;+GBTFuryY|UlraS>a$vh@sGyV&}it?$@6$ksH5&}OqWm#ypBTFllhY~8`ucD9~j zYX+0O+u8b!t(i>ltYPar23r;~8T|h34m{3e@9S*6z+~^sY`w$Q7i|5?)>I~k=drbp zt$l2L6>aFtjRS4rMcyDcg4FW<3n<~ zms%K9pLNp}aw&b~BocxWU~t$)a3C3LlW>1B-sp$d`{S+t__9A99*E^>*ggnP55lv9 z@Zn$_7>p%D@XiqYGz2@-@me~5Nyk+g*p-2qnV6G_mCPo}!gpC%mW}<{xFZ(_^RNTh zG7Qg;z^fy$Zv@39_rFgmwi^{Qh zBwiei)s=Xx3eQyGCO5XbvAZ6h)MIu7Ry5*C_D?hB9D&V8;Q1r)*%A2d2+TMVPacVt zN8$CO@bWR(cg+7+*m*`(bv0plp>hQ)Rvxef8;Oa=8e@q)SUYX)Tk)f zK*a(Q3nCU&5Jj=~u3*6$6f1TVd+*;|&hh6y{&-kfE7yBZ*?Z=ly=P`Kw-tL@@vb#T z+Ys;90j`i(H>cI65#B^jyN2YWlrV|@F@u3q@or&(un$EocAE8~?(uFVbw;R{H zvA8>1y0fAOTY54>tWqyhdvT^0mwJ)Yi#fen-J3VP+4u+1eMsoT%08^?!`8k$?aSMK zr1U3d0LuoDI)H}*NcxN0Vu=P4H;5G?y$3O62s4MUW(ZT<$Z#W5B=J!03}wHWbTj{% zc{_|x!&v0ON)J|hkmNzCCwo0P@5N&;UU{*|n`PdldUM2^6+YbXWs@IQ#me{-H{}zjKORU3Pu?(+;Xt54QgiPUz za8azoHQ|X^i1}gxa)hli^C!yejuH}uEXxd+**i}-BU~0<2s33SFBZZ?Q7scr3o(`i zA7O>e?zNT~E-LHf3i%zG>9>VlqPSj5L^yqo{9Bpz?}ghk)9(sTgcm}*%^N=5Ejt!g=g}W*`jeP`#P%a@2=V!? zf}M>zQ`@m!vj9%RTt^04_VWDwyw&|%&v)Qe2hH>-^;H>mIhRPJoQ8aC#IeTQYeML6JZs6JR!nS7 zbZeHkW^-#Mw_(Tcoco=_?bzC$v+cRpo?RVy(2-{yx!j3|op{%kh#sW(;Brr%_av+r zdwMgm4`=&us}Gs|**%EdK^z#uJa=BXlM~4EK+Xk`I+pe0*gt_pnGn-u;6IQ7pDlxY zo4CifGRSAi08baPg^$8cOD>R*B7=CD4B|ZDjWAyZZ=_`)t`b)zgE3nMVWbSaliCVi zRh5vnvgctj&KDD={ z0+C_unBJb*T{zX3b^SQjpZxh1)`c7=g)_ z&H};#;k*zp@*q*TD8^vD49z{l5#ge6Mc6OT@U&$|NLaG^8+`G0s#hNEH`L4B z(uz*q8fB^WZM6+jIWF62??^RAl1R1Yo(9JJYS+|b=Wp7=#Blkb>Zi-_xINQ4u)8lYGKL0S`LBGtrM6Sqw~H}T0tSYc)r=2l^H3$xUQRW=;5;hhZ;w#3-7 z(Uxtt+!qsWOJ)(Wi}0uj(M8!ch`Ia-`|#aUW{H6^%Lg2a;SFG*%evP$x} zB-wU6up_P%Cra_46dy{lq%`-+@TM$t$}y)rtL@3O=ZZZGVJV!4Y`6*04(xW|gafA> zc<4Z^Bg-AR;Yh9{ZygD(z_bd?s6bo=_EjLcBCjhlqY?>~SW$_qmD&FtU%q3u6LC%? zJ8{d2$4+dk!l^2pslt;gOsUGYs+_3GgQ~o)%HirftIntDOsv7=8ceUjh8j$BW~no) zo!R8fCuhRHXZrWd`+?&>5K)ulnmnz^#2-2RBM*M$%a5%5i9M&)n#j4PSoXs2;aI~uFJK$ zOsU7LdTgx6j(RMuPig~BG+@v^+w!lMEI|~{FS|p z$!yH|#$+`nuQ3r#*w}=#O~`4&M~TauvcD-8n)0G4ADc3%8IjG%`i&dEF|9fCniJog z4b9otoVU%H--6{W*wBLQEjZDF%Pn}&f)_2A)RLr@oM_42mMm;VLMt*`alRFkT9eqC zYpuD{nrUr_Zo|bk+-O65TXwhQL|d-6PEcE=0Q!@4_+{(%SK`9Us~e z+MeYdnAVZ)9r>>#4?8lg6JedX)tSlvBk6y<`5y^g*xZHiuFUGnhOS)d%Gz#h?Z(M& z%8SxIQAE> z{~~4}2?JR%kko8-v*&Ol~kAf|(UUY6vGo$O_?J2#-Q|9>P2M zJDQ|1oFBvGF+3W>l(8HhOY}d?|A(#ru=^kO{X^6^a>j9CJToScG=Z%m&_uXg5Rvv$ zNSGu-Mk1DZ!UAEv5Go#ar!d>HeLYi<$I3wH?PKr>vFT{vAdLqP%m)|7p60QmN zgvBB*_X-)77WTqo5lxqb9AUDE&InxA<{uB8dOkR+-h zP1q}(7oG`WBDyAtl2|9)7h=V&tP@g&L&8&Grl^a>!ZZ=HGcAHv*lUT-gj>Qx;j=JN z#95dSBg6^u!a?DPa7maX;x|Xg6Fvx`62a{eo>~@#f9+@)yr5Mj8pw@ zYK_$$U44QBmHW}Qoajbag#Lx*fI#IISuvJ3=Y0!e^&?``c@!mhXvgY5Irt~x{WQJl ztyE5D+=z82#a;WHYTw?LDM4&Y)&nKATHCDUe&JzOHi|ZJCr;mTQ|84MqLqG+>X>-h zK<%fze3c2(HY~H{sGFXYWRysqqHXn;_gCc%S#w7_GSSg*w6DLiCQX#zP~*e~YYuBi z1rg+_%$Kocjz9mc(YHeb%|5;cowUAk|8*1I-8|H+&bhN@d60IOp8h`m z9;%0L%ZFl2{CaFbfd2@~hrJ9oP><)ogYT8mn=Qx0>wP_U$4l|4Ffs~|Q<&35I9;6C zm3Sc`cOy! zG_OW;RNP0hnBFbIBQdv=#WY5X>DwZvW}C1>h!@k6E7Sg=Htn4%408>1H@e#h?NqSJ zzLwHH*8ro6ve}KavEc9LrEbRavWGS~RZp)3VU3wAg0l;kx@e(`>ZA49_#Kbx5b-OI zyOT4VeSXXwOZWsfi^B~Qrx`EKWWTUUoJNj*?utW@r+V8{f|sJa);sTxa#c^*ZCW$0 za%zitPdhxq(LXTA$Dsb=p>{C6;|SLPbxdettF{|k{TpUFN;~4z-fUnV4)yt#;k)gp zy(9G;eB!9>yc*=HFVPRZ!be?xVj?Y7&uAYRY*y!`W$On-IRyB-DtC}gT-R)Rb+s)n z!gJ9>tCRGyf&gW~S!rbQ>@QA2Q$qW3M?YVza$wN#dLz9;LVVTbzLWJF8;1Z_4}%nu zUGqkDe7sCOVVdfv0Y)^d!xneF5y4z&m*u z;SxK{6}C%k@LY^}j_CSpqS2#7ho{KRJs0h~RLt2lG30M8yAPuIP9*D&MlDg6YvW9P z-e7scJ29s#kGc`noh|aJpXraVf`_F9LAktCl1Gh+X~NmQ%w4MIR;itPmZlZT2dmTO zjuv26K|(!A_2NaKW`$HHrZgwN=Y1XGeq-ZUmT0+oPR7~mu_4CkU3R>!#=ROu%9CEG z7mSqmQkS0OXwuF;(7?ja);dqw4>zbwx?fsb$Wx}8i%~AEr-Zut8i*bfDG^^61yv8O z6ggvvl8N%t(t1WOi>^27pbzV_H}CjM2VIe+P4jJ0CCYRT5|q)oK?ng*x(x-$IRk=^b* zHT7-4Lwf}3BXIJx=JAe!B3S%Gln?!&-c>;~ zc(B@wq_@zo$z9oep%D@%9v6LR&(rTo7ctt1<(+ssf?c8}L&fl<2^qo)QIKbZoK!s@ zN?Ds>VxQLev6`b_u&J{A{|d07AbSe3%bI=GgqjF9@vn&$Cia*( zYGQj~o)l)I4d-o0DZ+^&JSoD;609x3@e*7v!Jd-bE<cA; z7F8mx5;H2ZsWQ7NbMiYjJ8|BLk4|i;!klU(S0keu@zqJFPFi(#ICI3A)bBZ0lM}Ug zREs5bSXGCVI_$5<;fB0v$lPC8{R>HrxZQ|}zw*8@8BMs_gxn@fYRbH(JZZ|VX6$K3 zZfnl8yD-Cr1uiUa$HoqXb!2uY_IBcAXJWf?w;T7n@xB|;-8m*cqks~Rt#Y7K&}mB;~E5o@loVC8}@#VZP)BQN& z$0t9s{fQjGsQ|7AkQ2bsKn@47B$%baEE&VnF{~cLhp}A$hb!ZFGmh!wi5^dAs7M51 zgOD}Rir34hWSdXY?;%Q9K!N${~t_$+J}Rry9N?^_AWJ_!j4R=gKBEEf;I zLc+B~D-wlu!e{aDXOpa0re~cyczUQMS~;c{U|K<<3Nh1_u#r6bI-u3!W{0*lRMB!w zn7g_ntr&}X6aEKVG-2zkR@XM1{F|%VE*J(IZDQK;KV&IQ4^e+8UE3{F##knH>u1#( zRB2jstH8J7oo6<>p>+)mG^ka)lApv{-Cq3`teMzLTO3f^{q+2AaW?3SV9mKEme15Q zxa6GQjD?Iy*QWY1oUfZfGR8?QymtsN>?iX{J2=_VD=^64PyLhG`S~T^RmeYgPU-q! zOC9oBvZ)8phH%WA$Z^CaTNx&WSdN_ji7#gD1*k9e7oOO1zY=q6viL{x>awOWS#3z@ z%_%bnhmkjor`{a*VUA4x?K0UX&D2s>41Rb)5vB#|;jUWxT$!(X2h~6fW0ozE5`Y#b z(T?+Fc?p>nm{*lYHF)aGduOiIB&8OMerEU2WnB4n~QR_Cg+LGQIrzNIVFCd+muLp3-YPh;AD8`K9tSipc ziiCd8TT!_kI4gFc7f*s&wo;UbD2T)Btaz+oU)U`T1|CreUM8oB7tF1_t$V;-m}errjWrK2u%uc`9$#kEsRtT!83;tSrdn zLL4u|UTZGME3)QsVO|zyxebqOIAF^OTkhGix&*nUnOcukgV{frNLM0;^Kk@GV@VdZ zmb}V}dwM%me7yYBHP}P>Sd{H$c`L4`HsOuA(w>W5Sk;xxuIw7lt`S_Gz~tcwx*~?nTHxEsu?EDA{m^v>BzeJoUhOB2CQ$*=AOKe zoAPBzev_n1hWidV`PWdfoSSln=Gd~R7}HC$%7wlDY|Gyfs8qgz+A?M6E7ta|?y3=P z%kD1ZE&f*Zv;?{OI*p@;mtWv;RqSEn=GXdCFCT-0haUxLD{$VbF=#_rdlt6W;so_} zXN1;|onSwu+i%D9!h9~wd{K@y$@q!mwaBeSRBfVbGv#NN|IE`mysg8gHf(Ogl{U<4 zOYT3+8prAJ%oR&JPj4=wuGFyRVQp=(-Alb@Va+A2B^SP~0ji2h(O`N#x1T`=miI%n zB_{QnEh4i}qo zY!EvKvq&O`J7UQ9&5y#s;$Nafmk5f(;Tu|z*PwqlSdcORo37uh~c^VRHKkU79W@5;g7W=vm8P!CSCcdn0A_jP}ihq{?1Dm?IuT?1H1y=mCNon+;t%;{|lWB z_B#J&zu*a-G>-Qxm%CI@T%L zEHn_Srz&U$)icnbyn3&88id18Zx4e-jKlhw5vos2;s=hKI9ZI(B{@@uEPFyBtpevO zlUsv_-?QKccGTg10~R*pS~IdclQ@99p`0JVl@ZJjX7@M_jN|Dz-j8F!1d=U%sfE?k zt=KWcige*}tQ9Lng`E|)Ewkdd@F_|DU+UM^1OdwNTZC}YzyIoOG;nuS=A6{$>W|#| zk%P5Z(~yG=$!N^%?xgl3V*rVRc%8%NxG3zYw5DQYQpoDY?pbtLvAT@ zrIoQ2wkBR{WMc2Hyh?lCmnGei%zCW+h0{$r)`qzwz7_7?s=&_tqw|a!*_qfIs&90e zRYSZP=ZX?tf*U0`S(g3q+>wQqnf(hlTN2lniEga-;I21ke7We)@+d1(=34Ra>+)QP zGL$L7#Kuf%%!MvwYv~>g^H(N={mkZ?T>q80Hazdjte!j=$g6=I^k!y+6|aTqVwtn! zt#}o$3px*_JhW$BH8wOQp&Q5LZTIKVKz4X*cHFFv>7yoS=0F`Rub%SN8>p3OToKI^ z_^Dq^S+>>TNN;8@u_8$b(+>~F(@Pop7iV5a4q579zMh4v{(fhotvP1PqY~V7V4X8h zd|2X3_;_Z?gM1^rO|mk+S8HM&G^_37;T5blJ1_ONi(cko>e2eH46*r-zlyp=ksUD- zowVh#__FvJda$5w;`_jfb5&UOt%L;8|9TGtgU3~jgC?f8B`jPw0qRbeCg$ipek%Kz zJzN6=M8^94}$_Et+APdpV?J;jF}DR%dt8C{0t|CdMTI0ZCKWvciMiS zgSV@%a+t~pv?jVRiwYCtOq?HyA-bV5=*(lS+K(G#f!^$-z2rbYSA+GYqgtAsa-rFtFSewQBs7F+;Z|f> z63bWV^-qCjA0MUDG;uZI>sGQlA<0CVZr>PXR(g!)#QscdBQjc&+mWk%$#Ub%aI$>) z7|3?HthwSE7f-d~)*>r5EEUh6AX3_LGK1dK$RWfWU=U>|_0{x{dgyF$VVjFST$T3H zj;R&c;Yp_D3N&3&={qksA1{B80M`+oYLUK6@03>A-Ru`+_Ve>H$lIM=g(X$=Wmb0| zv$C43DDi)@^ly^?u50ywts^T_c$|#AXLOUt*7to2kRog_-Tt}KTM`Hm2 zNJqAXEs!N6Hy{ZC(@pQDS7S`?y@VbJgx(=^0))_eeaD*d{k$KZ89isQ&pKzV@ulmU z+2z0M{7+E5I(1*J^KzXAKh-Ie+P1#ali_Ob`X5)UFV&q^yMxma|8ipOw9fTkc{T2T zVqH$p|9tsjtHv!Ff7bk`uRd)3)lXl2_UVVsZJ)P%uW94=vn+2lJpA~7#DA>(-l>jd zY>^6K9+&t3_7`G}ot%$2{!#w27!nekKFvBSgGg#Ky^a{tt+W@WmxM-1LrmW%0(7_T0f;_R5FeK#BZC;Hb_ z33H`((k|LIS#EE5lEW9*Ui)(08p-2IP1T;MooCJY*Ok96x}5GwHU`M;h%Tq!i6=4s z9}9D6U#K2i?VJy0Uu{;jW@~F^#j`DetG_V1hlNLiZuPeCbeM%IKcg(@$)@8+;Yq+9k8& z{W>SSy(8WrV{@}4Q$=`EQoTMMw?9V;z2w4BZ>(19WDE9r*h9)#XuR%W!CWfBnII;7 zDJ8?1>T+tAX2rOmaDN+gFS~)7g{G$q5Bwu04Jm zf6GNe-^RQ3%3$F{r3~EO)c?Cx5X(w2p*hoRb&DqmjF+-v=-K7)P5dL_Fqwg^E zJ$AkCSD58=I}py(cbNPx7vE*xdn|vSM_CpQ4UwN1YGClqANyYm;_E_j1&o#iQ{ zI+K$1KmSk30xRWmdOgBA>e$c9v~vCjYPIrqo>>@Kh?#b71u-d@N5TA2jD^LyTAD{? zm|K=7pU&%%nC7M9Gm@ZVgEJdn6Vt4;aHa+qYH*-758q%>QtT2znssjxc8T1_;9@Xy8Ec z!h+5VdM2p*APdvPuH*{(Q_x?6Hi;eTDild~K{Li$m?h|>*s(K$7Kj~NENFxxbrRhzGG7?f9J?!3$NyhO-a_i`39-{%ZC~? z`tley#;`Z0;1l(VY0+=_Ybu3#eY&RwkvU2n)+o90VQC(n{k3xHC8=(G2~Mv|uNyAB zBG0gs{z*d;B@ENSxnK@|NRN-Woh09?cD!D{?EO#ywgf$>zEDkWl_ySjhBl`6Q(~nD zAXhrGT0+)}5l&BRnos|WXFip`*x9ASSavSH%B5Eo>~ZKfE2m2!Rt_ewHJe&#a|>FI@@ZWwA25jilJJgUX)X{SuO2ec@i9kQ=IyPe3q8| z?3|9|Or-yaLKAiJYGuId^1Zb#R#8?oqwiStQmGEDx-LqO@8~gDo_Dy%BYZK<%F5(h z0Sn88))d%1h(CiE4%u)vng5cxl`IXqD@DhCLF}4r;mAxWh2dU@9x2(mVP(Z<62qsv zoEh4!7#1qGqP=e0?To0$@6z=P)B6{8wiM%fF@*~fO|WE`R1uMBsd0(kw2s)6@Ge^Zpc(erY*2nogA2@ao2$9z`iwebT~ZSX<(IxSTcAi?c0oLG3#Ib#EWS^ybWH&R@;h-8_&2v+^udg%t#pOwqkz3FfxvdV9wH z!bWM}^_DxH6NQ*qnCn)?+vsO!c^NL3VOI>Ns`KnMW;9_=Gp;x1@@K4T$$)Rk{g%N$ zG0#Pgn>;UTd~8i;)-SB;Lhs*L(M^EAo`U)aAUNK_gaY3;)#+<%aBD6OQy~Y1w zdK44Ov%EYP%CoQndn%A!nJJaoU74ZPI9HA3HJSPnr(fdyON^>RZXMRwVN+d>*X3M2 zroGI-`iyG8`d1k7I=Qd&@9R8!ohfhdS0m0fqW?QQdxu-^a^^jze$4Psc=QRqn{vG= zQ=8GdIXgdR;TLrKisfIC*OCn_IogWc*39^Z-@oPjcP##a{y%c^N3wrn{!cvmiD7Mc z(1v@l^onC(98=?2=45CB{al=KvB$#!57#^l@G{!VsAMK~U}FdRrLZ)WeQ7*SXJiI5 zJF=t`S39w|Gs`=(|5t`|Vcc&_`i+TQE$k6=N6=_-czXr?E%5BWf*yCbklD+^2|+^z zzWr0s5OJ`31PvCTJV($2K}Y&om?v=VLVC-LLeSo67P{q#XHOS@D;$A-0vnGK zbZC}^F~Y03BWQ`h$fpEd6*OI7k%-dMm1&E{*}2VQh$^S4D#@SCl6PO$tSHOn8mw)~HnBI0#J4UTtC(G^w&8u%oW94e=_377z zu=;uaeTN<&Ggi9!+UQ{jW6Cn95`(I9%q5YEHk=j3gy-2(i)|mXzZENgWJU+Z2p8$( zv;f|n@csQRdQ5bZ!zUn>PwOZI(L;cWF)GL!5$kisC+OCeBBv1NLYYvD!(BM8AZNTG zo^e11Hp`dOJ+cJ%UgWPP{QWr_RBQ*A+pUYZi#Pa!yf2yiB?DS8vW0xO&QY>+>r?K1 zD&1v=D?yKCpMA@s?_>l=dl)tr$TzGSsx^-SAxsW%G{J0rk&~ZtLxTTMZK%!4;MV4Z zL2afL<@aEog|RS_KgzS5Mi<-&J^B%G(}}kT&WJXK0w%#amBeKwDS(3ZF`NdFjvq< zK?ej)8fhU{&@w?sgrhoNXxGhxb_zNo=!BpvqlMNLG)ZXO$%3xPC_+^&HYm%ukJ%H) z0-@rsiibXsTUagBrPHByuK&y!ze`ljAemPrNDgxwaOS3Y1u+gi#9UNJX%F4zTDj|# zyr>f=D`QLfeFW|6+E_G5GIUsq!L{s~N{>{=r^+u&GH8Tp%Bc$Xx?LU}FWOnwoRbm} z;u9K7`@`MZu(=J3GnDV76<#|BYO>?Cf(WeMU$`(=?R<(JxsMMCqxMk^;;$c^)9v#1r=S%@=*=xOIr2t!}tz^jaWm+cPvwxw_T{6DW8 z=}Iww+NR1<7wTh#S6?)1OTiN_O8cqR^;RBNmdZwVb^QJkvDc45kI2&Z7F(0|#p)H&)OywI;8DjJlCjde2> zhD_E=RYm68$qDAhdrVAaP8WI!hi+MZ`ev$yaZ(3YHo&*@5%i2C#~)>^9G0fF&_x-Q z%A~0ZSo<7t`d2DK5x;ohO406YFqdCsUlP4M`Ae&l>Wb5Ap+Cz=an+mkt}@e5EkLHB zmr6*48aN~Oee==TnOL7*YLWh(=SiR*=GF?n*!z-PDaWIlW|sirvOLbpV* zhMeu`bQOAc>MKuxPmG&ZkmA*>B&heM&USFP8djr-xQ;?=I zL^!8PiLI#rf)Gv!1#w-_L+LA24D)G!G>BExEIgf|AdDgU`l@_20|tE@#`*~Pl}-a1 zyzq`x>|R%#9u4jIoW8<TQj0Xj0rTDKj56dv8ERRFE6v^snE=6;Qs>hys z+^EOOmpSw@PwO+k0fQUzv>`oSXUUt~eVgO&F#BCDyhoo8*!LlaK4$wTWH)7RGbT4@ zTXXt-L65J=ba2qY(Kwd0CpVdcDRfI^Y!`NR;mB`v&$O^f&~(9YPXsLyu;PlKxdJq< zn{quSoNq zrnhlRqNJ_t?DQCHrAwZpPWFp4|1B;H4R=PVUcElIaZJJIuyk$W`z^LtVo+rsH)Qau z?0J`c@3Q`V{%XdBX3TBQ!RE~PjCynT*dS)wubvsxkxf!|bm?}K_X96(PPs~zQ-0(6F zta6FgCAV>0hKnnQYISpEx)&y^2!B*$dLr}0zjk!%z}x{Au8McLA|FsGx~(!=WZ0u9H z0g{Aa(*EM$s#0J&8f@3l+EO{`nY^pa9EW+eIIb9TBiYr2rCsGQX-f*Mocvry2-9>m znnLiZDCsl>cV<_Viwq^}HulRnb#$1=kaYNaWPn6>0~VFwuM!+<$k~QWd6j9+SkR0s zPO{rFp)FI|vbZhF+cHF`(4N9CAJtDK>2!MrS#LcS|uYSH(xueDO}7 zS1;rXYe>#W)VdT_rnQ$AGR39+Q7b!sm0Y3qb%dOK!=dk_+o6pb1+(}$<`rj?tcZ)BXXPX&k|LLaK;(zT~T@!Qm@C%23;pi{S{gowM*xrTZzw!HT^2PK9cm9j)_4nLVi!j295N5teW;1$z z&Xr%4?bX^K;Ty~;NQQV!G37Bb2OXZ_ir4nuh=!VOHHLFFSkjb5UoxZxms@bYHRr$K$d6o2W?Tw~Q~Bp-`U+sXRM2)oj|6QLK;VHe zyk`em=r%}z-Vqjd3d+nc>J@ZafZZ*^8($KTTLW9XA$b`FHG_Tmaa$k-H-Jdr~Td zdxTBUE2N$u$QRdx`U|fsrQk_*XahM#xgO0Nxm6V+T&YQ7wf2d-evmtAQbjB4%Xox7 zz}?HvnwI9+q?P`k%h;+3B=>13-3+ZLMzXdX=PQxjjtAX@e<>LD!f3&m)0O?uA}lLe ze$lNIk>PZ>wE`#bVI|2NX&x;IE5@ZN9Da#=Jp#foE4M4i&-TUZ1E8bcV%u9}zbylx zx_y3w0W}$3lXbN?R-5a!>Gu*jFR|_=_PxZgI!vj<^g1l9!{E1=`4;QmCi`=a|HLeZ zS+IiXSBJ5mGuNS{OZzoDIZ&K~#W@$kg%B>k&e;!m=w_&g{$0598~1+_7 z*!4C2+Og^vmj1@x-xwf1t*`i`L4s9x3mPRpVUnQb;uB7YSDYGq*taJYR6Ia$ z2-Zt)INY10$CWmY4p$Z|&7ES3`1Fziv&v!m)J7091(@k2kG*_UN~$-my;Hkd__`{I zUp_6r@;~2wF~GRF9i`Tc*Y8~zHL z+<_oW88TmCsGB(gVGEtOacaOX8T}JS#uil4Xyw7q(n6e*%_QOOG?N4_`B-W$gVb2L zTu()i^)ZT#JynuB-euH9&pgsF8Mdy!mu#=uHrbh0k)<-O9iA3v^2z^BmU>OEt;WbO zYD`;aywm3}#2?BtuLRpmu&*Jvnlii@r=47MGOV9~Ba*$UV)*!A4vRm0pLH^}8}4&@ z4B5i%wHVljEp0du&oS9wj!UmvY}^>CRGw&*MAjad|pqgF*s{yTs*ttrPG_>bEfDm{-O-C)Kp-! zo!cduTZ;*`nOvLAwaKo_;RZZwz`Rz{)EYK+$#4}%@)*M_HkL_!Tp`kHOpIAMRaweP zhcu@iRoJ+%oLr+5bXh`=P;DOB&clzGDhss*Z@3IxSB$|B_S9FH#|>uOGb(vF=ifDz9(Z$sRq}rXG8fcU%D_~T9swxe5ZnVm7(~< zuVbp|qh)r^)#S`8NAU zKb8`b>Pk&BMG`6NRye!*;2 zr;u#a1!KCaV3(mfP?(-dfdo8=tMh~#s9O0eI+K#UK8MFC1k-R{%e%-U)uPr$%nD{< z6k{5*xdr3CChL0!e9xtL?#1&={P!KL3d?&aW$V*UtgRXun!pgHxHb4l0^ZlWO7`=pqaMjHnH{+9- zrp86284`7K-emBX@-sYf+8UM8Tz{U;wK($ump@=ccjXuwN--bSQYzmd%I7|i9-cla zT!LG#v7rh3zo4u12P;R$rMXknv`P73=6=uq@97%nUmc$8FwOqj=JydQ$M~G-UU!2Wz6B+bW@}ZT7RW zy_wuOy{51+x^F=dZmd(!EZgg_{%tnA&F}=}ZX440{fbEk*^v3X(M9rHx}msnVi22E zu3wm!wt5EntDth#;P=W-zEEwGyp<q zy(W;u<7okE`Gc8}$f88X$iXuPeKWJC3c%{6zbM0_xj$89b+o)%oVAU(JI%t}*?|fO zY;^6VGIjAT{Wsd*jbY`e6rlwS^ZjgWo~GhDsd0&JZJS{*7vq`lWQA-tH1$XJc%OS|a`&}nw=^r7 zu(1W1acuT7Ww>%74V@8oPPSuk7YW{U?p!c8igPE8wLhDqOW|AzS;SLL+G>5<8z_)5<{otR2{aq(OnwQN)i4^s`@~G?}DKPI(<-z9Zgv= z!on#*BS)&*QsI*ru--Tsmk;yA>%sr>3<+&wb#b*44F8G=U(vfI`wQA0bPAv>%R-s> z6)V5u?pKU%$rL&BCCZ)T@)(PZ`o60)fh*lqtv4Z3ajyYou8UNjw!Wz`ggr%A@e|$R z*s8Rw?l)`kB%D9qruT;|6Ygs#HjKBhB~SRRf=&wRH&e*(d<*B)*tiH^e(Y86DDG81 zs5|HgvbtHAlxg9pkfhzCEaZ$6|9?(^@O?Okf*4YQ<0TnXg`DR(B4qKzAjQflsoo?5 zpA?m_pp(at(w!HjETti$HAk*7gw{7ku{laP76KZ$go@ECt1OudiH_{-btiaJI_vR> ze>iWH0T5iN!J``F_#;Ho5!D@@R0A$K*h&7Z!xfw8ir0_JT9a*ImkdXg*N3=@u`+`7 zA8|nS+>_)mG`nqnU+OxOKAa_8Ab}cn3=RuISQbLJa%7dGTNtCNlRHl^Wges-hx=0P!R+pW1c~F-v^*CIg zOZDl|fJqIQ+JF@e==ly?zhl!6?C2#lp>U%P4pw$duj<#y24w9fr!%b#BV$-ro#i#S zR)eQCIPxJqKVs6y^!$?XU(u^2TfSy?2CFhylfl6Zj%RQxgKHV|?#S?t%<0IEj_mHp z{*IjM$iE%A*OA8^>Dh@fomfB6!m2?Q)(Sc}(!%6|skqw2xt&qP8ULjO5Biqe;>02x zlr=@zR8qQuZc}Dr+`Iq`A>1e_>V~mulVu^VermdvdtXQ~i`ARfV1~WI@vj-&(d={- z<%)~GD#R6k)>ndzyF6~ek*2+T(wkO0 zTYu)n&l1Gwq5s(|6>rvtWQsDo6bB22iMt&7pA-kD=An>Z&ZK8<6)Q1BW%d>_vsw^Q zMX9o(i&`$T^ZUTYPDHVf*mC|(2%E% zxzm)9%{c4ivXiqewq|fUgV7xsAq?JdRpcG+OfWevf7LR}ZwMz};&=m=yvl-BQswJ! z1(_)vgnixq7fovWsRVEJv;>s(fBflCQz?51#@kuwWT}%?a|0ZWo!fRE1W9ks8)rz^ zmSItO{w&Yk@?>?luyc$`#H4DSkYX$;Pw$`EC6`vikdd=-uwTHklD*F>U7wO_)R{K= z%ZJr8JZxaWaC`lfC>!?*28bj%eEN|OD;#S7(uCsBpWNIZ<&K$j<8s+4W@tma^;}>L zJ6B$1k)Jh+H+r}Gkka7wnKbbw)o|i7)(al`g4HS(qwSU}%C!#}STNn$Wy*Zzm6ngv zbMBauZ1&erm5+4zd|v&SKpQItDQg>-==B;vUJ#>Z3$Qu6Ag}9+)z`;ts>a;v^q3VO zb|$Pdff)Pu@Iahl=S~^cG?6&YV0TW*=937Q&rnff<#vF)T`)6++4r%gb0E#0W0e~l zAmM2x>kY-X2DQ7fiwY|_;!KkXM|YA|&M@eptGfEh)a!S}pR6_30G#x{bj64?Q|ey4 zkeoFt-rJ?mHd$HuLm<^{WxF&d(N14}49;W7%Wab7lcq95D`!1&s|<3xvJ82%3({a- zZHB`}U*&g&r#szVUEWPR?_=)w3RD$aiD5x34`WzW)>mbJZwtQ*7kq^9^qvSWZ^p>L zMI$A-oyOfXt_WjvtA}!9b%S(Du{(nUT`e3H^pC2Qjuy%w)tjVGQU)v;Yt^-b)lQA-H*t}f!cDO zD8?6Jc_FR_v%EBy64~(!cZJj3Q;nI?>Zu5xJ;wk$StYnqhWTNvsLGmG*eC;BT1`=! z^B;0qDD{tc(u$$;)n4g^V5*AzYqfP{W`4}kk2&!%Tg2IR6Gt#i(AB{fx(>0>V~F|* zU3{?=BT8_p1lbuZ>%LK0vpm9IKaXL!TwLhXKo*BVYeC*7oLAJoxYo9UIqTBz^00vugIcG zoT?<1kvG9}WN7reng- z8WSK7X5*@w034Cxir4Rno!(z?xt-i;a~YD2{RM8cVJO7SuNd&Z^$ZT9-?l{}S6m)L zmFFD)l|rUer=70RJe1yhWI~$Dkgm4#M`50-F%v%hVP}77ddgdA z!&80w@eRS83u0L{j(E&`Qk1{TutLR_!_y3U(Z*VJuVR=U|CT2Y#?@tKOZK(qqWJd> z_rK%3(2$vP0<}?Cx%r*>ZEf5r$P34t8hm@bAkmc;fR*M%5X0KD(#N=;*IVE6TEG8 z7kq_q?mZ@c&c4sNC8st}_qB%gwXeHS&Q6N-8+t>{`v`x^psF^nR*d=OnNoo%0}BcS z4Qlvw5v~*|20gYy@5zT5?!(HG^;40lK5xAKUJfgJe~`*TUrc)_L9XbaS@9zGV_4OcfteO| z3F<05`OAX(2zO@o2n+iJ_06_0UlmT3_c`Jc+k4YpsoHUwR{G1iCZXEu*OKh}n2R6t zK=_7(h7_cy9QvsJ-!F6PWqQ`clH7Tp^=(j;aFb*mjxZn zvd~Yk&L!b*ADmoJZ=y%VtNh8$=*SF*FFwhkPkZ!vopGS@2{@z?U_9EMT73tr~9n}JD8?rtGh&>;2u6}@fK+QoHhSKsY?&7n67 zeab7rFkWBr?0bnp4Ru%d!r0nDsumq**5hCqCRJf~RmRug=&M})h(({$`zywFU}734 z(^!+vQlX9~AH^E8%TPU-&WyLO($ayGwmOIZJ-s`^rk<80(iRk=5Bl6Es{Gr`W^ z#hG7@e=4x6BAY95=Xv(M&VYFOIvJn9vUV)!Wnos9g8bT#2xeyy^1@i?XB2bma~-p9 z5W|+@oGs3v51EtBOkp193FNyjrD$60o~8-O6ws zwnmcmIul-JRY3!KoGCCkG`ApXqCLW&Sx}CwSNOxr6M>`l2}L+yia>#a<_TISXq%t| zf-Z^aVPR9Ag`I+q3OXaG_e={T z1pOZjK0#y0fIM-Ljm9#gW+c@9DVm2Pd z7+QmK9XPKB{1}=GqBvKKybjz+VY1K;smx60nLtr<)W`sBQmh#3I&dR}C*1>lQ!tq+ z^h_ZyMe0gj#3PE{<=NDf?)i7JigP?7Xy8b}AlXW3COAEbPD9#zf%;ng4dMc!)n$8K z2G{3ceg3J>)X(Xwt{+X+-(V5KA6E7VFFl0O@A^L?qMb7-*>DlWgADnt`Z4*ZOVBri z1sOcdU|>f^btI=F>pHT&BO5!its^%&(!CS?Ix(RWGdeM}Gs}f>-cbN&1j`yw;WCv1 zGF3@;?_VHzdXzUy-ra;(-}7T8}qu zj6g-=48z;Et8iVY_8wI$N9#yaD|1-dWrq`GWJ|y42a;vX)0FY(J3{ejiavaJuZA?J z+GcNQUNch-@R$%XB`&EDo*~56Ft+wP&dUf)`6P!Y&XsH$6=r3NyuT$P)*G)cHYv*H zMD~s@D8w<@mlbbFu`o<5Xw85hltlXyv=SY{>WbWaoduup$RUlteh=Y5OOCbXy7>1E zUB4&ydwS$49H&>GQ!)cq2ls9-dBLT1Ns4o-GXsV5dP~+IRg7}vM`h#d=wP0W-s6;v zY9s0u$+L2^0T&z5>plK`k2S(X%$r~#XQG8M3R6cUJAI}>LH*>lKM{$J6w?sC`w0wd zCna59(0D$9wf>nPhR$N4ggs`BQIxE2nJsgh`dGHF~fGBw}NGgRm9Rd;tq zy1ku@sq2Zd78Y-YA?$I%C+(O{WSViV1mjB5qdXg%^6Upzwqbq-H!|qnk$xQ+*^wC? zx!RE-ojBHs)14U6nO(w;EfB{2u%H3Lz#kTrQ&6(zFxs_CFH3u5{IXs$R}miP@M(L} z>~yO_cUi+5oqwlOvNK*=!XcnVLGC-hhQlyuDp$=}3HLel!d#RG<+-IMkj8Y%|8a-m z*GsS=h`UKl^|4O14PaPapTEVn2JxsI_h4@%&nnTg3d343wgqQ@;rF%pAsgE@!^iIp0u?*U0 zkxXWiW}51|H$-VuL%CovZd*C?8Yk2MTCKMq!u=3(LK)qVaSi2j6AT9{FiE`nPJPIJ z--~9uY-e%^lRL;yHC*aanspV}Re?*Dxbqf=rxv*NDLTYdhe<_PR)p=Pd03i0l^Ix> z!7p}LVprPG!lTzBe2qc-NMu?;3|Vdq9=ZdO)|6_=<-W#c6_z?-Ix$Jm+lHV1t2 zBRZtHw7Ku%%uZ&kaJn~4{ok}qf|gwcN~p}dD$>fOxE=c1{vf9I6rQ^BgGe(S-Heyz zFXxv+>t$}Jx~|GF^HL~Wc|UM|Rki6nZYvi#Ix#K9dOpH)E5S;!lP|i*wW_j6yi_0pmYp z=BJErDka)uDh6CoL-}iZ4KnFNZcCw$@TR)-ZF4rdsg!R13+m3WIQ4LHxdFo!unkKw zqyJI^7z=h3OXh&akj@hQjhrPlle|1a@sFw=vmaxCZuo2}b(k!iM>UcKT;9OGW|1 zopJgwvrr`d52<1*alY_1#}ipGQTb7NrF2-Segy=IGxaQ@|+N zLeXNJ3}tXS$JLy8L+pBqm9=VGg{gGj%0Hcy)^`~Pc%FYnGMrw-w#)r5r|-ttm%*iu z+>yH$8S6HcWY!?J0r%cz=LcN=i2W_N){;|wEc_?T?PkGe4+NjB%lm&p1}7`h`z0BQ zF+}6LsY?eUTuFu!XrU!O=eoMoU9ZftOR~B;Yih8e6PIO%Fe1(=JPtiBIhRooJ}@;M z_skInDW@P6YRDB9XQGqYV=OE$;3~0L`{R8(6)=jSe|kiW^zRLEnibVqDO-mlJNb>4 zVH&Zls?_FHY}`>3gUIiz(7%=cXe$~nb+Yk9!81JiJEHAmrLjLv>25Eq>l>d#yQ;vfM(;7R67;)w}@D|=aER>XIS&Ngya!(A;*C)WrV~~=4R)* zn~Ab+CL+IYMB9B(j0wT4`HdUOUYPcJgq#->ljh5ZL`JLryWfyiH%&~MB->(5!%2_V zQdlX@I2ihLU6$5ov%fRklrpySdwqt=2J#5OF#5`ED`Wj}?MmUUw)&oYJ2RuX-$u!G zoautexqc^8-k1MMiLO|e$uMs%!_^OzsWb4* z@V8jpMru8MU~k0ROzRc^Vp*C0aR8s97&%2*?c;)!F~jB5CD~h&K0on5?9S=&sxCl} z+Jf2SWQvOi9_baQnh@_YyRwi575TI(BjrTe{L8!am)6)=tG>lFkH^mC80Ne!&(}1E zdcL~40Jpx4+s=&IbeBC}VIGqp`8#JTMEFb{Tvi79(OWe`0&qu7xkB%WO@EKo-7IVz zV&NY(`ZmHC83^L$I18gB&+DHY!f2tj2j^MHD;R{~*56@ZBx9p#@kbZ$m?xkc5hYlua70!Sy-5Lg$tY>!;RL9{AogCI)C*Q-1xcl&p! zVIjH|VQ6^{G^OuA;jxaea7ECRkwOb(TbPrrLV)^U%8mIJZVS3s;0DFU`gAon#kkgh zMNOEkig;oip8OW-_&BdV3c0SkdTyMl-QsQtQ$rcmkTDIVcdu^^+42oNf8xR@@!Asu z7AFdb+{mnzLfBG=A@Qt9B-brbl(w|2D5u}1XWxQ)Iz#bq@6TmWN1rO%_Yp@wVr5I_ z{vsnDx=k<2+zf^m477La$NyV7TQd+0zgRG3L7&83B*Wx}cvP<_2}lgpHh15Uj*xc1 zvz6IuaE$3Lq4_>_z=q7k!*>)s(!FqP9u*?92*b*AuqpF}X~`0zvPFo=e}bk8QOVA~ z##&g!@#+SXXrX2E%RdaI)T32&gd}}!d=P_rTG%_Vpwv%$hgNC2hLaP+(l;3KJv&q< zqP8ikCWnf0yeR(_<IfvUKGo6qAXdB znJx}ynV{7})t{tmbxU!w-sGM@dxJ*=Lh-_*dP{05?bT;?a%=MVC23^hwIR9UJpP!? zU4;}3QC_z;24rKaoUaod>2%~@LZaWs>uM@stS?RP7KX9o9sYWU15G&Ggyo-d@l%fZ zYr|ur0$kEqmL&}t`WiP|@VEtke9eIj$qIU9bH0TFWu{5UFS4w*88{1#?+Bbk~}EMxms*|i4!l;w+?gbva%twUS-B7JZj3`794HCnHCKCip4Ft z(vlIaxYCN{U$gsbj(p9c)(roKk>4<`4HMchvkeQ{@Ov9Zw&iGBHnrz3HxI=b_8AZW z_SYh>2wNL+{1ZktW@2M{w;{U?8=P!+GDNWH*8EZDf@Z1f7{WW}_Z?_A>~2dYwf;ydN~rLn?s$r_*T{Cg}FsJtBDI48q z1dRGj2`Y$%>1!HnOi<(0!`f=Wn4LuxSy5jBDTCVH_>Fv&sSEY~a}2M?%r64H{YKe{ zZ!nQ|ER#L70c!RNFjHt=}x~zVUh3_!6 zF_)V%O(@^*>6Sp>4jk_;lte#)p$1qOIod)mp)f{HQRxoDND&#jjB<5I%b)I~x1v`v z6$010Orrt~B)2hEp}R0W%NAusZ5H%UU_#47cFujuL;ua)6~aBnoOU-E1UHOizS^t6 zBQrq9E8ojg(y~S=a_cesWv;xz`cD}t=e>u=`b;Bs9~LyKd9)?AK^zoT@?uv7$CC`v z`RNUmPoa?ni-WpMw2)bd(S>*@JbxSi3N-y3bA(w8VrnqmigBYDhf6S}BsWUatt=zU zvaBp!!=LoY^cVp>h!6} zqMAIc$)x95_&mK|V89DZdV&2faP9^Ay~x}bS^gsXUu1AC?$=>nJr*}$cth4TB=c2n zy~(L}nEx)9-(%nh4EcZuAM(dX{L`4_O&HgdgU!fk&d%m!e?kARS=X8~-!lFOdO0}i zU`iaz;y4w@t+xE_Vnus;dN}UoyqA^9%uC^D3O!TVn##pg?u-BND-*l0rwgZkV`Lu- zlLRdqY+=7O@P~>W5x)^*5PIoMz>;b8x{+*wL#EcbqSpTty`J*p+YtKXw}WPnDMSO z@J^FJtousBUavPke|ojvUM>)uvt(8Qqib)rE6$iyM&<}tF2_<}$n~15zf3Qf#$i~w z8WczY_VdTPV`9?cQqzprep}fWB7d=?QA^0f!y9L~dV0pN0$<6{9NVwJD(FWDPFGjC zxV<`bVQ06M!QlZTlz&uWgN(xH2lJI;WhG`j&&}uQTbn;#Q{g1-W`uE92+j+PZosiO z>H8M<-eFt1*(TaqFB~!dOzD_NPrP}C@0M4TEa}+26#MEjpdQ^CFs}iNq;|}Y`s$;X zkLIYHi_@dM&@O~~LL(0sPVZPXM?J!u>hT)dlm=8Y1~n{=7-HdYVKx=va1kC9p@)!E zD=Y0R2qH6t`DNH!mU-pq9?rb-oT$iy%1nHTK6SWNm!0+5U!U{!>HjJ{U*pMZ40@AE zZ?gC;hQH0#x7pu_r|+`hJubb^nPzPIjLa`N(29qxIQR{Vzhmcj9Q%R$aol!t+{KM{ z^m4Pv%|$mi+>A~!}J9kCynwZ42fkuhmf|W`tigPr`BiD%6o-B$&odcx3N=&*+-+xuYHzS4 znr@BBisf%7Tiem!!?X@;OJP$Q=YFC0uPhU%wy?j2+k!902-o_IIE4YK#6cUC7y33s zZiXJ*O)ASY<))j!-9NP%_M?23ek|(!NCr$Uh-w@3M4m$hiFC(Uf)%yNY{h^$elKuw z^d@v;GqX{NW?K=iJV$ORR)5T~ByI?7{!p;}ST(~nJk4#oOD89cxm8)>R~e=Y8%Om~ zZg{H6JzD>bopHtJt%?)%o#fA|v+orqwPk5r zY04c5rk3cUWG2y9b>kQcJ3eE4NoJSjSXt)O;9+f!yu^&RS@JgL-e*T+{%FFrCR}gA z(o;Z`F{DH$%h32sKs4kTzc3SgI@?lnmi! zF_wlhswFd9N*$GA_!^;m)M15o(wm{Jt(2>wTe6C-Rz?UYUzA;;90=uUbq3UB>$_}! zmkXcMs}h9h#uHMGqYY`5}IVsQgmT_WJ4UHU>^tF>F6XhCZg+lP4#BOf<8B%2r}6FhBIAoe=V^cgcsva}3?V#uw`&?+3SLhtHq zt9eQ!D#-7;h)M3tjZXnp=)ud z7H4XaSDUl7xmcUwbve<1-VK@3kd?1;;59bAPR<+r@dg*(X4<<9{)`)6FuNrSTQU24 zZhp_YSo%BpuN_a?aiu+vyj)J^Kq{-!ne#Kxeqq6H91{9=i4fkEf<_A0V3NQ?SH!{0 z6GD7j&;S9oFANjHJX{FzNWpOeD{T>c-%SYUV8Lm5LJ0Q>!RtTALXWu?rU>e%(!HVD zJVgknnlPamtDCV^LdozruYQPWN$wWnNnxIem+7O1v5@96+<5Yz9}PuEq`BL=3M_K_W~9r*NYzK>gBUCL>TZ8kd8ETl zrTD!l8eq*t{s*ehPH&_@-nsw$=MSOKqf6*E@u zB+6Cs&J4r2mrF~Dqxw3YEw3>}vS*dB6oWkc`IEdm*`?y@v5A}Q9HYN#9^AVWm@G0}DwzQJ}mx)@)aVd_DSnlT=c&#wkHJk~(| zq@nyUPX<>Fjk6ERvH3?%PZSDI8klHP7yXWJ<@wT%_@@r(Zi+X=4V~kq$QI`MeGCzSlf$_%X_pm9q%G8^7tY8kMKD@aa&d4mYu{YpM z8M^wNFHb;D_fu48ux(qPQ}?}*{1eROV0M?H|96c3jxE2i?H7)# zTRp?{e`#azC>8li@HvvT)4l~lDWsz5TJZ5IN1vll5NArUp)|Kj(>s)`P_9Pts1h>- zK&ZpG)@*h$sU7D91e_%d@%@6)hPs`CizPW2#a&_jhY8C+M$keP)Fyvij^^*mv*81_ zen8ic*wc#L9hj2JvQ%~oV9-ysk!c022EA)Ax)Bo^F})EZKO*;2cDLeuD{i$SOFTzF z{aFRI<34?;BZRyP?5)6=icG7dVjd3D4Tmxi9~qY@prWCX`NW6x{YajMz5vb6p%3Wi zS0$AqQsNRF+JWO%o=G1|>-q(=x(vr&V4>_84^7mmdBIMq=r&ZVEA9N1L0>r`B7el0 zwz9*{#u8kT$~VH366^How{C1}1$*^kIXr?%5doC1(Dz}iswyqI+Z$_W{T$nv9YVK_ zRyS@$x>C$(hW&AV%@NVrv@C?R-*Wal9{j}FxB%fz!7ok;;tcaYB$LMYTy67jQ8vBb zoM;I@bl!SV{{DjF3KZ&xR+goYhpkTVFQGp#EBdL@H8V`H6$;UJqomB?JbVzexU4tK}vwOUSwzg#J%w_s)q=CojVYeuy;pL;M5E0J4`?p@?_Gqer)!mgELb_W)8kjT(5>|kE7 zdXrS1^Z}rpWb?dNji4h)**V){IV8z%y10PlMSR%ZNDg;^grrI&>kkaF)Tb2>z{WMqBqwUe1lic5=3Ympr1-pEtpjFbxteHUhz;EBSy1MuMd_i8 z59eegU8|D$8qdTj^#9)mO?}p9W{`@N>%%cOdI+W!G@(~P4U8@v9n754Y%9%!(oz%Y zCH1gi>q{e5cvPQKJLaZu5-ZYVoIXPz&kN$X(8rk*3xq_gU2WtQRK)3WwWY{@jU(?f z?;|#T#O64uPK^D2kDJR6*4eE$d7URE*%ZN?3M_kr#qTitV=`Nk)0#;?&^w+vUUvS< z6oIwp3YsOtYKbY!ezc zbEbuXGgYYA)MVRDxY+N?gVeEjZE{L5uOjQ zLVWP}i3Ryg(@!pH!O|9HV`pQ!irqw{#X91&Vj9feShD}W!p;IZs_P5)Yk-8{Qb>Lg zT!Mw*Qmic$ZA+m=D`ApMl7VC<%w&R9aCZ+L2=4Cg?(Q1gUEiL$H~qi&);c$9@m23! zIkNY+_t_G`B2Ccw#;c>5r*|{{%50qN!ltf#)+C5;yjlz$)z8~i50jY^>+6=D0TJYK zr@bX8&ttMM(u>X7?o^lR%)!N9nbKdFOIp;Vx}suEl4PaK5bR0|rBDmQK|koC$-pw2 z{5w;B*Uecan<&VNoUF^q@Z2QlW?gQ2=VwTMCI&Mvt=g2jg`PJj7N;8x>diF0hyY_i zoWFfQY^>R(%$v-~98qWga`j*CY3-7Y8v3Nv*WEpF25~GaTck@zTk6Cm!JMcZ$Wvlq zYbK^ao7HV~8RIF{QLt23Yl6)YXN`-m9^1td85bW>F;XS>va(W`*M)JqGgc*Pv+*K3 z_iHiBQ>CN;=5T}VvSakUVBmhd9d2xlG+|UumWVZ6jDw{)TbBMItO((12=799Qib7F zNvX=lx*YnBA@#Y~fWs|$+m@@pGa!s5(X5xPbVjdj<`=Cjfab&0^2{i&UDnB};yXSp z`<}I$!;m<87q!lwtc_$rr5g0%sf`=mwRNHT{2c7{<#In0SM}abgxl@v(88x$p%0>s zf*przJ2o&r-fB~)pL=suAGIKd*=f}3)|>HqohgH`m(L5*BB2e!)fcA4RYxhSb!VcS zc8T4l9{UeESMjqOq(z)vqFpJ7}cUXGe3bvqlMSGj{(JYJ0(~qmhru0;g7;5Ht7mk?F@?Mcs7RQ z;%P3PAh?`BIIWwRv5eb@jMjk16$8|0iF}><;H-IM>4298I4X zR$IB(iRC@$^Do=PhDsL0@`=36InifB7nzK?sTcdq+V1EUXLfW_Mm-lGwIcT`va%9K zD{;0G+bZ*_GJC7=v5Mx`QGKPAp)!46vDA;P{@g3f)55GP!J%MISL0kWF8$2z7K~`g zzCRe(iAkMU8^?h-);PGDz-JeGMH<|Y?;7j&iVB8^efUt&XSRt0Lfu};U=LiVpIn(E z4NK&D&e948xzk7z)bR&iEGciyx3aLi9#7K?Y7&c13wR(woyY9Wwtnf54sRaok1~pj zTA+o{k*I!_592D3q$T1ToNhg7u2IQIckLv#6nv3(b{B1lRVU?mF<7f6hgE$(l$9If zP2AMoiT*Z&SO%fGDQPk1DnXE+6?ItCnh_mX>tOG{tRD2A>{FXH>(Yss^kS`EfYoMG z=hbIn*KeE~sn3_P+JhIjx*N|!ezxRgdtRRBWlI8QU2N6ITe;p!^1_SHGT`1!@MeZL zOTAg{&1P?Qdvi)OKrpFVZ#BX#2DRNrwbjTtC4c#J@PC{?) z&P_AcjO^{3no2Kbq&V+r*AysZwK&X<2%{2^*%+UZt3C|$6xuGKPQ~zImnSl(xItBn zD?aSW!rm<0%c5-{W%XhQBIX$cS)88v$!9rRd&p;oRf5iqrF8J}lk68uTQ;>57xn7Vhg;-RW^+mZ}oSh{&Sc22SAc9#_ znnPt7Se`9a*jtmqKd_?(Q(7{h9V24d6wAF>CdRQfjsZ4$+3AzWh(uNQ-9`IID=fga~a$4UM%(;9~ARbZ;*Vvt<36Z_KDf^bf7?U zh#T}I^ydDAbPi~T@+m9tb8;sSEAlbSm$|-d@MWAI6Z}{JD+72Q#F{cpt;nXTOsmeN zT1=?Nv4#w3#`B+<(~=D>nbL~ctvJ$(scksbmV0fP+>U$gc=sC{|KLyuE_UFZm0NMl z{+l}kL;(tad@;vF|G7eMgoSU_9RVa5B}PvjY+~AAJ^n{EPSB?iJAPo97`jWuv4bWgNZEvS81RF&;| z6pYg4krfxRTApjU_<{CxG{iYg5R~*&mr~Ec?LX-$JMv(1T40;-9U~qyYjT<#vUax{ z2DA;+-8g{`Yg9KQg1fU$np4Gq=iKzG#gcY!Rcq7N&(IsbenlvO5DC zYw`LU9{kO2-PADP&*!v`bG`{Gt=E7b`GZ|S^eM%Hid?VEKpDiZ*;JcxjTrX}>)UhX zH(rL3(v^q*a!7>RfnFvqx)p#3{Cz3u#9^K%ITPzLF)1_CGqWr+$1<}h3#+nlDhpS$ zuqi9Mva&r7J@Yd#KlAgm$e-o@j0j*-0DX%vx(LroF{=V2zh=$%?E0RY-!rTs@0;@W z7sj_@Mk}VbX2@?$_>J=&m>tE&D5fQFzX#joEl;{5-~~NJstpnB6MPW#l})#KkbI3` zlgO?8f+b?Gt`BOGf-c>Mx9L#l+TZY{QMVOl-%EcC2d80GSN2 zK7QxjADsD#!vt_@Wn!*i zv0%AO>=T*TAu_R}1hYi{91yITVB(8lwoLYV!7+EGFu~l(@*A1>(Eel0X<>Y&badN=4gvK+#YEuA7PIR zHAk5{hgwBEcP$^25T_Qq3$%L`YwPwwI?G4f?a{H?9X$V`o;9+LEh?dkrCKGMHNqKD zGCsOayt!({l98S3XpgpsM402PBH4`+V+LzhZ?RgD?yR;rb3}RhNq38*yd~20-zOx5 zdH$YgafT{yv|D@Krt+em7?036i~LEfWUK7?%wiR*sLsH=xu*FbnpW=Qu*cgII$3R@ z;SPCoo6Tz0-l3b(Sc%Q(=>S>m+m512<73K4S~{8|I=d|8qZ}6ZUqUSrayw^etR*4A zVKK*djCbEoslwL;v)1D^C{{Vv8ZIMV-tLGFm46g_savSi9$~h{TB8gjhEICpv5SSo zS)CTA=jV}TYizgBN)@VBDORz(vI)8uZ~ffA3@P8y?s6BfbcS{mR1K@_Mp%{d@sTRC z#ETD}L$hLy9hL}ttfx1*-KH%b&#H;FSRzBC&GLWbEA+qrw+@wab4r`nnl+u)xY+*& z!79IwcCV}iXQbT~ZH;qSErw${y%x(oE{EbH#@OBVf3(9KV~#7|#qNlVch_ar_D0&C z2{DI6+9O<^P_3G+79?$}6Yjx?;rMx-9UHQ!HSqv3JDpfXDt>LcB>vY9=TBjJ!Y>H=QOErpi z|G-_s&g@EvvCEq7=DD5C?Cx4@j&n0UdB6VzkKvx#=dpi+n}>%-$k>;6bqaNuBdzu- zq55EpjFKgoV3mFR-#>*_s!*X?#Yz>bmv_5a+$%OA(&7~J+WkEDyt>OSI1S(PnpSM1ZsB;l zQ>?wH1jDO8YflR<*U4SdB*yHBkF|8uC4*;@mA|~Dk7vC&v!knLICRAl>gZ;(I6^Dd ztWY&H&T6xUhz1dp$Z(b1_mnO#+uSXLJoj=~qAU*ge*N#R;pR>*n`je}GM${E6>C(l z6xzw`hz;>P-ss-PJ=Ma4tJxwVLMm3TUQL_&SWCs4HLHmlsVSSFVg=*<_0ppPf@{Uc zyV54Y5*F_YkJV#UouQGDAqleQsu}Zu*_ywgR4bRQlT)^Tgr_*U=Y=}C(^f<JLD`R+M$JH7>bv95!!xpx^wP5+szsoTB%a?npLVA2)ARN zZ0S;U?6FpRxWnSmL|c5MB}(4MWfuF>X7?zpVor;DLh2|#anA^SLCLrLxA4;zl;Ni7 zt(|k#tLGlVP_tX0hQ(WK(dM|Yu>bB9B3~5evZW&ty*Z-Aa5Zdb>FTav?+B9t>?8{@ zEUZh6IU&sI3`@IF!hbi93o~8?D?M?q4eEy5!^6Ve;xH`C=I&Q0E70N$vqyyqA90HG z_i&^HdqgEQOM0gz7&T}bX|`D{v0>3Jk@vQ+|Lax4-1;v}22sR!Sb{w)-d)n#8euX1 z>=~M$r$Km}IUyk|My4&yeUGsJ)Svq`<;&w;u}=4mokA^T&TY~1&w?f$Y`p_Ar}0(JcDLA`i8@IjHB^5i%@pohJX+)0xHij-NFCSmqm%LY6L; zNUKGc2TrH#iwM~l;a1sqc8zw2SVY;zs6Dj9wKWy|WA`|7wC6^mI}${yhKdlEHBxzytA>9(poyteVy29P$RJ6C}-woH%T@M2& z5af#JD-FdIu5AK?~fdIqLsqNW@3yN zw`D-`v) zi7AtG&jDifJEYl&?}vyNPNxJMba{?=$kOAkRy2 zuo^EKF|09be_+Q?>}bQ_-X;zTKDyJghnwg-!o;LeCRPdNjWcmn&{sxl(PR@hXP7uN z%f#yWCWbCG@ir|s!=n1otjuv2LC`z-D*bGES(%0PS-AN#J!5%4E8Xb9`GLk>P&@bX zup&PreOc?rVt5tAwh|mJ!NYQ_3T1mJ$3l4(N>T-8ROEOo&bH;D{AtI--}v|&eIgha z!}=bq|Ca~<^7vog$cNl>A1(|u@y6|anPy_kJQMx(@SOl#g24f#H;45Mq@Z+DeC`$T zhz6qyZik9+>yPwKeJ**@sEQg@49d!&6cZzKKWLDrx2pOhyg2A#s)iZir{nauS3NU7 z$HEvj$;9S`CN3_MWtWz3s=8@Oc4p^9c4l?tiJif+LSKv)(K5!w48e^g6JPY0vjDpg zKV?d?59NI6VQMB;WnxDr z&UE0aR*4n2QdYjk_)OgN^eZY57%e7*IZ}OoxGNX;wMKeMq3z49%B*a{>}Kp}&4`gE zb}vp>xz4GpZE&aREk%-b;QfqrPR+d7-%UUNaywg;SDpDk?mS9Za<^6fhmhjtkp}=`MV6eyj(Bj zd2)c*M@AJ$uIRc^XpBMCkY3-UCm1j4eGJsAYDjKYdrqVa`dJM!dA65lS9vDrrMu#d zI?4>wy4VF;93G~teDaM@PH8=o1FRm8oAQo}wDZJ5u@;Zt%&?Nzmr#~#K^8%2_Nel{ z>wPqdXiYE>V97rEVWuO&Ai*m;2di_l4I2jPL{9ZJH5izO=>@r2m2I`S)PR0%8QGq* zfAFpoS6sXpYGRHk-!UTAQl^-gI@QEsJrv03@UC|josJqD-o5k$S{3nkTGg!sbMPT6 z+eD6T)~u((1rmkhI1Rc;db2^}!X=Gckk7VgLQZTuby>eJ1=yrTdC68pA|mfO%%Ues3;N`4My!6F}0YOzMkZ7$d?E~Z9Yw{VxTLYOye z=IfGBv0lCNGchCcGO{uw+cL7;i}BvAq+3is`9)k&!_VA3uRI$A42I{fyWg{s>s5Me5gqO zN{p(+oXV`P%=s!@ufoZy^sL6LYP_k&#_AlZ&gJUdt<;b^u_?8jhG50$ze#evVm{*_m z^|@J}jSbk=fKT5uts&PN(z_8!jhNert&K=-%;m;>YRtSQY-z&5CLC$P@g|&X!lfoW zZo=y(^l8e_ri^JyQd9Oe7sTJoSJPg}C- z7f$~|zh7DYD0(4FfN2~EsQ&1d=8^$2S#;ZMF)0u z;BE&Vb|BfzIy3vt95*vCoQdJg3ujLRA0ikZ$%;rWMRFyQaTa!3xM1Ouh5k`YisDEV z$D&vf&C6(}#W2FkZ7Zo(K3eJ9k@+3j*pb5>Io^?TofsU;fn%*K?zKBvBkx87q?vuNn~0gD-zkB$oWM2 zbSI@dXStY!>Vn zqzcv~%k_dcf&pT~-W7}#+vl9%j^Kr0$!z&P!Sp#MRtVM$4hfD3#?KX#PcU2XM9@o2 zzZAh?!BxRc!E-_K0uu`aYXoNn&jdrn&>1B-E_ft(F6b>3a;4bNYjndQ$Q)_u>{4@6 zFOFroEH%m$}ER+?ExZIr+BlRp@H7_a5w2ZvX!<;hgt-!si z9Inkrdl(r%`)Lmk|*j7-RIJdK1M#3OPVP_%6Xes+au2?rQHsEsuGzUNlwWG({2T^TFt!Q{Jv|l+2P9eziYxq0iWgHeGb1yb zzovgJ4%A|$Fp_9?SQ!z=k~j{>vEIgA8y{^9wll*{F9%79EbPwV9*pQK!c%Zg@LZ5A z0)LfYtO)vK!HfZ7U$_OXJ6cGDruW2QTCswq{VD1jq)yBn;0qz=cp)`r=mD+YGYK|z_W8R7el%* zup0}yG1g-&lriXeFuMy^J+A%IMrjBex@ngvlQ^<-#FwRZ`lsJZ_en0zYo`oT_2!Wl zE*flH_sE+Eef90A_PF%%VylBsPR&lUM~EeK z)YamBc^Sa?3f!zmax@==2@W4)O3z!oMGN#Y>Yol%O0nb@mbc^h@677V>O}78G1bbd z1UZ=2mEFQj&-T$%ksWac^|~H)WwYq)E!rR(xRJd$(lzakl?yg6JB3^Bk*&O6xYieq z1QhqPD>F8{Ip5QiF5`I-ZhZXU#dW>sV2oYmX3oX&#@uMk^Txc^-^<7y?8pAb3~R#7 zCJYi9JzsEAZ`i4H_IWYkA5CY+m}AwY_b+|Vz=oQ8)@m^OUR?1UM;R1!%*KeSocV@{ zKWbdf=#X({s5X12jb-#3utKy5sn>XH!IxR#rR zUvuVHjXcF$9mXPYhqfJq4OYVi&smMS_<|Sb|2C$?x#=kud;&{`i#elrBlI`&Pl_lj zq|tJVGPAD;53(|$BxCEdv<2(`X1{QS^|Jf=$);JU=RC`ra2fTlD8&8_9J6a1OFd3m zStAx#igwdrbzwR$7JHgumrl1*-+?T~!=9Z_S?F1k{(8zppu-hmIFV0u+bK{FWK=Gs z0bQ8WO(*sai_uMUbE-JUo3eO}ShboAF6D0=$NWLNthBnaZVk45!{cvxBZ7U3yTp=U zmzeQ;1gT=OFO-KJ7TgkSkl8#a*fmKfTI#*_PJ3{{0tuFAhuLamKJL`wa}D;lH&c7- zwvQ)(*Qm1Cz_!M(mW`f4tgFkBM9nE~@M7mZiA_cQjoM`Tkn}T)JZUY33ppZU%raj`cqI%+poe@#~U z%UvI7cPkcP*dTNsHJOu%L8Tc{npLGqDnoJ^E|uY48BUj_e>v8dV@Elbhp@0bo657J zJohW`t^#i=vY-mrsxYl83#ziUD!WC{S7mMuUex2%cMNF2lqL-MmBp>;+kv4S*w=yE z5p1)t*TNwS$1HrZa6gKdQ7q{xW{E7;p2I}c2o8%fn<|U7mph?hhE8YI^RX*R+aLyK zIQ#3T$B8F*B+S z94zt#C6@3rs;{soKyz4l!nX|U`)k_rC=!s+#h{G17i+_``y~onHTsCBUG#G_OBaKN z>G_R`VG-VVaXK5l12`DK-9qdxN548;Yr*(m8N5Ij+OZa6U(D!Q+I3yshkpj2wyBUN z+a2~uSA@ZJ8-p||W+2o{(<>tbGqFEEV+-*mo=LKvrwP`{8s6_tB3^9bwVre89!o>5 z(|=}~$IcyQs=RE`2z@{j(s9^Lsg?e@;QbKok@na`1E18TvP_EANSeVpUrrvvtlI4C zzNGAoa(_Ylm)%K{5S90Jp`OH*qKf zM>B9I19vkpCnI|^a@&VZ*;$yMP5Jo@m;AX9$o?Sa7hrh-wie`EAzl>Xa$)WirdLsp z1@o{XBbxB^H+or^YGJ;G^A@(paG?`#JFzpCgK;F;7-DCHoe6eQ>^$g9ZwGrF+;*@d zf#EK`xERxexjop?gGW7>_ZJ)fqVL~)`G@KMGC)>SPj`=9{=ExF)zMNEJL4ybz3EAomp977SWw zV)i205or-(;i?Vrh~XKSmw|!V7@eJdIk=mP;a@Q*AKUZMry!%su%iqU%Ce#?uga2C zjvM7j4&iKhHdo?eC3aP2KvfP^WpQ5OVw2#hppP)CZ8Bv0WQd-MQhM#q_MTy4)jZ+H z@~Qjf6ZgpHO_Gg1Pq0I9SnxvdS+IDCTqigvxGcCY=&6^O^>^fQ~z#_g@HIdvhA)Ro*V1`uV&L2EZiQ34XuW=z|z-(pS*z3H_Tfpc&6w@VOOVTJfL_FWS(%1G77DG>X}y(y2~j z<6uTkWaMf_p8BvX3v1+0R@Ua{axsn;XKX1>m*H_4-j`!>d3M$1NnPGGU|U1_HeuzD zd}z(}_G}JkiV1M3N7#7zgu(_Wyqr!gFJ+;0NpJHg~lgqg3yo* zP4Qe4l)t0miV_^l#r9x!l;UG0dR1m>WzIBa_@C^KXNb(&DZOb>fH}e*u6&uCY%j*n zVr(eRCQmP^LUeX1U0+#vRg4RIv2kCknj>A-BB71Em8p7xp*ZE!g6N%(+2y%ci(!9q z>0hq@%WStunX2n7o6D?pM-*aXCi+z-xiXI`Go%VfJ8-81D@9+f)r`49BX$u z)#oQe^;UBRWjkLMV~f^OwV>4%*}bYWJh6&8lI&1#T_VfIRol65Yw6#hJks!CUl7-V zw0m{28dQ%w-b`D?vRAqpOxH}bEv~Fq<-@Z;CTa20el`P^GPInw&+TTLvh{&COZ7FV zbO?C!X^d_{dGgYf66$~`zMErBX&!N#I`89tO~!xC$iJDLlFqdH@}MNE|6p?hum57R ztg9`_`Z0$y!N7u^m_gs!s!Ou7u`!cHcZ@Qnv%#}*qAb(CrH|I1!`GlYUC(^lP^uUF zX1#2fzd0espyKLPy`q;oK-HTwy2a`nuUd1zGAsw9^N}iK-;XDL94yJ)U{(dQE0~YL ztSrUqQtU3p;ZmF`#l=$eD$T+&90+AsGx|j{Jeq?Y+1QCK!kxt8(o3@{s}Os$Nna{{ zR*#G=x}AN)PV3=}_&q&(G#K^EF+G2VoKW@k=zhU6e6 z2a9vCF$YicF|QV*8_}-~No|8hx$bt3O*Ju0%qURE8Sm=aJ+@nSYiD(K(>lEbc!DxSH#_UI^EQS) zTFIIM{;o)?vUI06H`8hgshy=S#iAQO_P;ft-k{U;z9?#)v?oY2OVs!1r_&O#vh zN&1>W_2~HnquQ`Bg2{tS^wnjQx*?44&&qt5mbM2~21_n3cH&knW8yfgooy8Mvno%F z3UIv$qe^nk&cb+>#xq!!{WQUDy~U2YSClu;^osAkQEFwCno$$I-J=c0eRmg)jj4-O z&#$hj7ITb2vE&@u&^S~p!H?0Ixn71@Wl1f|`R_RG7i{e(CqR!&KF1%bP1{Z3H7#r^+%s^)y~=F&p#~dRS8XRAmyBH%E1o z&`&*E*;r7PCuO!9hF7u*Npm{h^nb%b~QHLchc+pbdCjeA z@4}f9&nO3v2I`}!T4c33=|ieN>r1k$3j1ras{u0`GQK%)BA6e^fzI?wWNvr9{L3KG z+2;nD*ej~=OPa2-J1horu4ik_ICt3Hb#NlpE&YalOVW2t_LT7`6rdHAQe6BEbF;7{ z3tO_XCo7+`(K9Q z3Ua(4?+db^5Tolep+57Q(7z?me&NF}>}^B;E)4I&s4gt-!uBrQ=|aD*?C#3-uH5cQ z|87j^#_Vn^?#8liY#JdfPcU7G*Gxg5WD`m5sBs}*PuJ1FCx- zCKNGgdS+lk29{@Lp=ge*BxmDGHlAc>MsA+u<;z#B&c|n8dIhqr051#hwlvGi&`+$} zvfL=kpmLlj$M6sqSK?$PHdUr)RaR7Gbai^x;L0~F`IfuiF{A-w8gRZ5lYZdTkIeds zgFi8;1%unr*UULHS0Y&3iFNU;aB|K`p9H41MlXnSI4uAHLLKtCb-4_ja8rHd*o=Z&7U&81MqLQU7Uf7~7W~Xqm$p$7)pyLn#}>Tl zXQHx9H?6r2&9 z9xdEn&{G6#U%?>3Ou=V&Ac9DZ$umuyUy{cA4Z5bR7?X}G%$tX!|Fd8XDw^a-S7qMR zBY~~XSmW5P8(PwNfV;?qvayL5$70eUKwjJ`sP85NZ`Z0MeHp0x0iG09_GVr>E^IF@ z*VD#Z-4e=+i)L*D#jAT`d2yw)F<clbGO3gXhoL+-<>1EJw_-=cyYx=f` zCOMpy0k>XfG2}d+B_};xGu`-63{)Lru>a z)neQ3!@Ee1j8A7RdGVyQ=H4{$`wbsz(jREh6aGRbZfD|2CiZxWuoUr6FsoaG1Ts1| zn{x9!A1{LGDduuhhPUHmPyIpaZXwxt^b1pdrJtuHZ%O68KF5(9+|9$=d@Rn-2w%?o za^IJ}ehl(slphEDSOhPFSRTyEV2)Q{dPT;6&xYoV`-2Y7`bb{cR;BX%kxBH8@A7Emn;I3e) zsDX8Y&4Q)lL_G+GxRdWh1>6_jeSfY{JyCb}1xJK?4_aiRpH?O(*kIy|av&cEd^zRE zt4LOn}xYuIiekZsk@^z;(~}(Z>D7? zIWwQKvnoHEN;140r#p}u&HHGE+ex;wHjxd9EOm$b$q+6QoSJRotYF5RH2$ylL04r> zP6ig{L@D0?!Egt|Wj;2^P@ELp5-fCQr7xASU1nmSV7%`5^^c5HH)qYp*1T-X%hSBZ zRK%N+X(517s%!F)?8VGL76-C3kj+|OHGgM9H-oU>%UUh~?s~Y2i!zhIfHVdWqY`#G zm{y!Vb7ZONCl~?Y7PFeoc{6)zI^k&EJQ%NkawpXXl%QV$_7vewC=cuKx&hallk^8G zh3xfU_iz*E^z$0`k-d5pCTWe-{Nk)?qO?Y%2}J@u+{GZB?t&KGWfb`=21aMy7iCaA zGG3=Q>i3>r$=LVm#e$;g`MSqv)mT}yOK74sLVe-HuQdLnS|xclMroeja_VX;pRUxR07wF34Sl+p6vk=EInB+}B-a z>S**V>}kOmeN*^5;_Y@dKRBD8b)B_AHaf%^>=7Xad}Gx?IFGd;oC4U5Dm%`YoW9#_ zkwx3ihG`o6^colHG7K*2GB^V%8R(adce&_QkfCMRQidgEc~+Kj<+xIgjg{D2l?gR? z^bPC3JH&2YO{E%iPgGW23w@^{VTAx7;}rWv@Fldawvp}HF*3Z^V_ghbe@y5 z^2l@ZOiWyCVvK%hs`TsTVtNjamgeOjT-0;?{9MXB9WnJPXrE%x2zgA)bT|F;;_3NL zym^sUwpcZ2zSJYR5wp5*xC_s_Ft971x~9KmKm25Dt$T4vs~B6*6&(|9ci3G9tqDJr z)V+*qyJCJ_#%ncE4ElzT2<56ivdZFrE$LmDJBd6?&oS$yu@eKLz$wJ@+UuNrSy^Dd@Gy2<*fe1iIf`G5}_im{+L{W|l^Q;MKKfGffpWza){oB2s<%bnS2ksn4L`iYv@G1%8nHErtGN!hS33tJP| zriY;h#93ow4LmF-(hd&dV=QXwzCjO`3vin44yU@t^J%dnuIVDrSD9E}kh2+ilY^x> zNiD**3OuO7@cImI&*TC6+ZZ%J_vX}8ojExSl0~+T&}mMzxx0GAy?Lnns{@5RI*gok z5A$jdwHcf=%%86Natw&`utu66$K6;!DUilCVnQQb->O}f6Vq|%MKa!x;iXBfz^auJl%pvAma`p!>XcOnZ9ZCp)D+fYf7Z+Yy+tIV9q%IIv&$is=R7*~S+br@fV zm328#pPA7tOyH78rIR8o&x)`dD6DOo?9wNC`~;m{N|K$8xlLHsgjG$n4XFO;iB@eF zD|1@{*_N4;nYmej+vT}mp1E}x)tGgo^n-XyM2x{f*i|hl$v@icP})#g7}kdIY1+;p z9Qf=Ald;n!8`mRvV_{}|ItG_Fv&ZNTH?=S1m@j91dF#u2Uy{a|j2DMDsroU4U#vkd z=Du2wWWPA2r@@O`R(-813rKr0HSI9ls*XGIW|U5D{GBpKst10>tE_w}N$+5G$@1E( zD*<1Znm4BCu~DjQyJB3FEti?;Ik{1Wl(O6{%l>j)_>PJ7x#DEv-`e~r3k3S`t_%;# z={B)J-Tn>#@NukYu1O{)Owy-F_1-0zkdx`9c~y>0-?95U`nBN5Uu^k@J^wIS^w59_ z>7451ry;G`P?}Zc7}$c;f4Kdxu}SX3m#$3gmZr$n2GEo|+K{Su^x>SGF2sjI`qEZQ zsbyOD9R_`nWFeODytsctj8$3iJR2V>(X%pBDjPcpycwF72%vtXH%D{>8<)c%2z#%d zs;*jFby$~=4f$B?&kBD=1~4Unp`{qzfWe&^*O}v;Ip3L!ow?r6#2A^?(GStApP^C-SKqJM|bP1EbNKwR-y+f2%E4nP}+4 zxq4jiDBXfVv1V6vOssNUu(dUB^wK7+K#587F{a%QqIpgxBjTySW1hC#N(F&n3BOzV-J z15pG+UTrz3wrDwedo#(KZQeZh=T;~B^k72|M*YQ_zq$Q4pM;OC6jQD5Y*Tukg^5}L z9se#dR)h1lq`JC#aygXpEIV)Vu)Z)8bVqQYySA25KeEq0+WL$%s5$UNTXg{zx940f zB)@1U2>!9|md{GV?c)R!p9EKh@XuXnVyWJ?-Y?N0E$6oGlZp#h-Z>9va`CVX%YWlc zM~=i96Ohs*=VD$i_U2+o1)f&tXgxN5!-M*)Xvn69q%`4eOHw`M4va$iax$eR!&~v# z!h4N0``ZnQsCqHPqaBL-Te}-J4XI*m3I?#c2=l72stV6)@o<`n)afQZrn6WlUDtYOb?&$4Uf*=N zrwNddXCOmsqI=IZj%{y*r}EK&ISm$eo59`$eNeYS+@sU-L1m=IQe z$FRRS@eiZ)I$#KqaVgEJ(^jTV(btx`Fvjh=ny7Zy{ZxA_$Nr(M1@(d-TGQB2_hI9& zJosJn%Q(~$JR29u^Y>11~S|qf=3C{zYHg4)N%|6VrGDtJsF(7O?*;AK( z-!trc4tT0d6!$aIPp2aP(_%`ut{2ZkbVcJ(I*onU(3X8|wd1TF9POH;tjgMCkcB+>2Tg2t=)9h|K9BJg=+}1ap zuvae@|DdZ4cTl?Ws*~+C>T8cQ$izKUR8tsgIdSd}p8cUMVTWpyeaF*m-0A`RlZ^xV1Bcrx;u2}Z-OzagL6r>7G+qh5{iz?NYlT`WXV7Yvw z7U^8nzzyNedp)^6Ahw&$;!??4QPR5IXylj9#p6&m*Wq$EM(P%GfwU&l>WzB&TVrYI z#i}CO1~l5qkE)Wa7lKooZQk6~Jzf6xc)LM#%1*m>C#6o!&B!)fF>_sp`qRSn!dy9LUj(JkQ97VjPWOt(9{&rYCZGQM!jrd6?p0xGdvO?mAG5(uoZ8VzuU$ zRvMGSr%SM@9CyldqYg9bv#vRl+S4vm z7H4O3C}V3eWoVi|LM>=~Sk;WNdfJ%7AVB1zFR6d&Vm89SnL77deW|L6^Ma}A5a%CA zDag$dTqw<#5S~`#Xmu{tU`S0Kh#B-X6TW73E#5R>etTw`xg5_97o+~BUw;!HWS}k# zH8E(ai78Wc_F!PAeAbVVNr+JnXjbmaG(75;j!EFfS&w(LP=Gbjum$MsN$JLzi?y#e zx>bkT_>hZxV|BA%{j`I6$e@vYyGe2mj^^Y_PB#0ptSHN>aI6YEX-_XGb{GB3K#0=?KnRnAefjHr^&O zrw7Y=npif(#C4hA2ZH2bGRf|WdV*Vm^X?|WV)Z`}yma>+c843yHgR{3eimlX{U&u( zI)_?~d60+EzPt(ILK*he=SF>=*5_Fp-v7?KKN%9n=rC58d2Oa|6cb|D(UAqQT$PvH z=WZ{vNM35u|NNaY8!G!O8iW>bMe@$s>pV!On2t!`Hb zI({0aI~JW$AyuoY!eqRes7KQHxs<0+h1r>di-mYlf>X^I_a|TeWMeqb!Wk6HLM<Mi3}9doGYWD1Pqv5{Iw;sY*u(+BHFrZfVJ;KWibXq}_E`1V zgE!L$rz0eN*yF_~J(Jg<>V8h1cVuWB+y7$MU+J-?l3dQgwH!S2<7FA9L^C~_RUKK= zk@X$<7|YQ(HoLTKqI$1<4DHCpj`Zxr_E^rva?KOwUqapA$cr!9B?k2KO&RuA(B{gZ z4*7e zI!N`{c|l5_+E@%g-u5Hv5qR zTZ7n8fGtItQH&$SxKWIr#W`1;p(U7Ag3Tq^Q-UKUnN^BSr5PN;kn$W7hEstP6&YHI zQhn~>au-Ax()15_J9DZtFFG^M$tfpaoE%JGkBei8tnJFXZY=4}w(bo4i!pz5?{A*>5*p<$ zG$VR^y5Q;n6SD@&78N9oF!5UOPB2!K$OpkdA$juzC)_N0=#HQKgjy)7y7d#d85(PBOt%#CLED|M@By8llVEi-_X9TN+${n9+qTehNlLY;R zZLbxa5D~FYmPIeIhlVXkKh$4$#RdrHw-_V?&aTS>t-rg!Gs37I%ZxvDd1BOPXV zhRB3Tf>SKGI|L07G`PFF6nA%b*KcL^$^E|XdER~IeV_Zk_rAS5XU^Gs?X~_(_S*l+ zR;fb8$`vYC_`FqxObKmj*ke05q8$Hmw00##Czfkxw}n@ZESK2%vnpSP{U_Fz;Bdy; zq8**x|K&eT>iyQ>mj;b~YSpyfA5B{|ZPc*dPhO3Gt^GrTIs+}=d~xdCf8_tHtck8* znV6+;{`miCynDpDB|yk zPtZQ}vHPKg^6P#!JBtIkUxJ1GEbf8wuo%tIPe|^-^bU+0W?{)N_wb_H=X$fF6L&kQ zK~Bw}9Bl2#YIzCP_rVV$kywA$u;;b)E-wKSa%9!67)`~@~coWVY7b_E)+ksR7f?`=`vp&e;ub$#~`?e_HLxIec_ zkQC|uQU5kJ-85TC{mwn6K!oTA?Pm7oVi9gcxc74bXM)3)pd0DMsnP0F{TRAj8~soB`x)53|~#*gcE zXd{N_W^x7QROCuE)_%>luX$LX0pIcLdnW$I#@{*n7svi)VgiFYGj@c9#e#9E7Ul?6 z&bF{!a7=Jxk^8qo3P##p(YDxd-LiSnbGRBwLbNSWD<~`Tu`VAg78+waI9*X%HM6V* zLyb@MZ*SL4qUiHrx4H~Wh_pw`?$C}KkV{Ql2hE59%*w&~9Bj|Y@xn|g!RrzXDb3Py z?EajWUvd8%`qW@VEw z92;a|)nE&chFBOjRMgEB3m*i1r&_o#m_EnCCBe)27B;43On739Emm(%ytp-AjSP;q zw@Gl>+dJ&K53@3~hI^>LJW{(&28J>&)IChQIdk!{IuC#5%HO5a7RAx!5sA`ba1$IdoVAL>ND^=HvP^x5y>SZ4BgXFHxIeFoR@6{*inh&4H@-= z(zlK7%$fN zvEWcY3#nptXA9;oGL%7UyY3aN^sb@C#MtyUXmnkr2z1jI!kw&~_hx!-7UrgJJoDpO z)rF_Rjwa4ln@&r%tehyTbDO21HG2fNuoD`(3nx}p6J9|o3YmLftS{vB+3dbK9r+@$tlT4s89b0Hh! zeff}skvW-}llysD;?L6nrUr33gtf&uT!Pb|vaBo{tMIBSgR3*Y4)5!-yBQ-|a=A6j z9ZZkrSuCke*14FK$mA|O=*GJ4EEnTGNQ8cq*ox!hEUcet;i0gNsbUvS8QLCO>uCLo zEUKVZL`%T3aYT&Jd`0Q>EX<25qtuWBVxkiwbra{s+Qr6uMLJ?5v|{iUdJ4DRD!h8{ zc;#OZYi6t8woq$kCZorr3{vNE*c0?`cs#{Ws)_oaJKaeQX|HoZ!-YwANbB9Q;k%_pJMgt3RnPb=YHCYk%F!vX15i2PS1>b#WFo zWph*J|G~Gx0kA%*r(P<_Ic~7#6K}WPDlCjGoO|(OmJ~aJ~2H!{iPuR*qp| z6AHb(qjH$_vpA)L`Z1GP%*(-V=uw-eHdZ8yK}pS69+%TZAhs(rLTqP;o{Z1zFNC?j zlCD5kLZVB*1$@{N$kLf=Y(Rp`5w4fC)9#N73g{#srF)(?%i`JKUYCN}8T4j%M~-$> zi)V{U=kHok>+8#g1>)&0P~uRqwLRQnVsNZX2vLSmxADB$keRg(H92kVVJ4Qui=9ap zPf?o(0q#c?jCMqro37RF-z;wOs9ox2piDKt{0~}Jf{eZH)a#~lj1xz_CWmTrq$XEuv91m;>+|Y6 zE(ix++1tVtae@vAKF&8HN#QZN(3dwaDlx9IV&854=Rw)p@f8>4v6)ztiEEjdn3;K* zS(%w_nK_o3*;&|?g}qrgn}sV`xRHhXS$Lj>URL^9nJlKj$}!PU^8b1<(TkN{Z1Lih z&;>7Ud-1}Ho>}RemBCpVk(KqrqO)>8D=)H=?9EDVHhFW{o0s0a^CsDceLf7$#*A#7 z%Er}fe8|RFUy^;<=*z#pobu(NFa5HUlATrAIg_1B*}0yb``LMyon&#;b8s{V4|6aj zC#Q4rHYd|_F*g@$aP&79oa%+1r>ywA;$JZ#L%{=6K^%f-A* z&&RTSoXp3AeB8*-}<3kq?m5cdo5q!1qqF|{yD z3bUy&M+KqX=V)kXnQtML1f7`$c$Eg!e@lQk1Jjxm%P6MR{42w?+9_l-|YY zSB&IhOfJUqVq7Z5wPHLi#=BzlEzZE=+$_$6;yfwNs1h72!L<@hElEl#W|U%1DRz|N zY$mL#1zuF(eFgegWJE=#RAf;_R#xO z{EDQn>Gw5*zGlo#oZpRh`q-=~096 zHAtyJY7NqAu&D++YjCIrA8OFACc|qovnGpbvc4u;YjUS1=W21e7I$m$s1`kIGp;r> zYm-)+b+y@2n=y4*T8EW&IA4c*b$DEdw{=)hm!)-CQ|5S{OaJ;TsLzJ_Y_HFO`W&v$mHOPM&)n}w`;HCYvF$r1Hz2hEXBsf>dsYe^m49!@ zzYY1&klsHs@JD+6#NeM8_7fw2V#QCK|A_&ONNvQ@Mr>)szl}KFh$oGB*@%6Oxz?DZ zpE>n2Pk&}z6P`C=$}cSag|)x1{TI&u!qs0G-jq>INo~r!rtE6U;ijBx%FU)s{*_t3 zviev4{go5H(&sl0{Km!Kxc?gyf9LS;Jo%l0f3WTk_WZ%|KREXXcmCk=*){*wjGS=yWp%{kPZ$SQfmgaW^8K~wPrS0NFgAy=Ka3vX^bco4ID5l6 z5zdovUW7Bu&Tcyw?F?$e&^Byn!{s)-X~V}h^p0Rq1j!MkMQ|>Hs}T&2WMd@9BRL<* zok#{anBm~QgEtN)w`FNtPPXM)Ted~9D~c0Q42@<^G#jJY5yS2n&c$#&hL!i{)b2Z@bVu9bZ2&V z7IbG{caDjlF2?za;DO-1pqKdVeFeh>V+At=2L=pFuC;wcqKyX5EQ!qvRl9hrp zf_H*d1BBQMjtg!I-U&Vm1`f0^TChuSOmJK9STJOeh1r4&f)#@;oD$p++!nkP3>YH( zMsQ3pVW@?;g2RHRg8ReeKLsxZ1BAc~5o{M67d#id7xW$}eI-g&k2)dZf?KYTsP>)nx2}hWgl;`GA zC0QtfJ!93EH535P=*KFFiZ^^Knot z*I!%^{qN+^JR{_;i{RvD&kwAW4Rl^G$_V2JM2S16f1j~`DSso{)?T}KOX69tLPNSx zx}`K9vT)9u(b+hjjoCRFnu{KJn46bn`I+R$c0VrpalxO|0Qwf-aS(HZIUCH;5MG3G zrZ86u^SCfi3$wL62g-A{JVPt7pfV|+GrI~Gt8)4)=6ub7>Kv)f`Py9gmgI&kY{-p< zylu$s#_a!@yFW9n2`8Fxz6rOQ@V*K2n{l=oOaJ6ZbC&R)VLhg#mI4NpDwj{ZPDu?w3;1aAr^%a4s2Z!ym|`tUY@ zl!qw8^u)Y{)}WQJ%%|MpRTkBXq3|Gt?m^Gbvu|^x}vgyUOyg zEU&AQR)_IXTo)eGj;UR_^AE$jGqF35L>l&rX*eW!E_0Y>tZGn#Guj#92-E5ZZ#LIs ze?1j1=wNecv+P!OR#V$DPOBz-xd^)gxEkQ07Q>fWm6%_dF>d`EY8pzJS6)3#8zsul zATQGXV34QGf|a9PR506R9!$FPn_Kbvo0)_~<<*=yV)akT#v<8yn`and+Bj#dJvKof zp_)>JOSRO;YO{CVq~>OEHx(b$O>ZFUeYhCP;-XA0#jDcvjAQ*jydRWN+@+mvZ$@Qi zfIHMz7?U{)WP_E{UR=-0rXuvM#q9{DwP!?M!yM=g+-g4Mf$YA_Jj%?GEbO;(!b*y~ ztuL2zFgPa@a#)bRPSnQGb8@{Q z&l~cpAzOY>);CTqMjNo-7j}y4oo?ace(Va1}JSl>^ zf+1oyrU-Tmj)?jAD0m|#WW9J4>%@DQBL;Dv7{cYk=7-8st`T;=PL}eB;HqGSFzj7| zV}dh+gThef3FEvicxjXy`DyD^ti1b8WrrQ{W(q_UeHAv==UoE^wqd!-W|q{8Pi_{4 z(?>pIu;A8kqXtr+{gUqyu<`B(6)s?!kpG);nOT^X4Ouyu zl>t7y&&h#Lm?!M1H0vrbydo)I@cv7d)F8Pjr<-!4DT97xZ%dNGIT}viDE7s(tt)BW zdEA{IvQ{r;jW){qY#;4uZVIulK9}mV{5#e+;_UD2m!;n$eCV-oo_?xCr?g4wYc{ru z7rI+z%1UZYLor_DVpvsr*5#GFeHX^cTTU0HFjMeWq$zc@k=QV?Y1b>TxFQ>WQN}~B zWV3VNOZr!*Pnuz*MOP%)!ptnvm|^?f?7JTqOJR~u9XX&jckZMb-UWAr_d!Uyy!F3sOE_``K5aa&vZdWp`JGihde4&iL+xNUa61v)sy6E8Bh8>Brar#us2_ z0fq%}I*6r}nO>9LwYXS|OLZ7gkB5!f{WEu(u;mvHw&Hnf7KYQa4OgP*)18%~G%pL5 ziHPqMn|Dq&;~tgcDWu&Xh1i*i2f2Ct4TswDxHIR)GENf4K1Uem)P;tHG|6cmHwpyw z+UP^DnukQQqPKpkHdjk9iZCL8=K;J5;9USGi*vp>Z~x(1dWb{3pXG}@)UOs|aZXN@ zVr(T2R3)i8Z)>vUXDD1RMKK)X?lS}VnS(#?{oC`hb zr5+UT2-iI*7n}04pc+XnSY*_=o0Q#_V?Uh#o}&#p(u9W*+>K&c44Yy(ArA6Y!3+`W zQzF(ECa4b$jJCza>EF?dE6UL;R6u6L>|3U&YWaYKNS9Mz@;bAO3I>@Zw$IKtw5Un# ztQV(^k_i6}TAwQim)kN=#C!fsV^Q@2XXS2NrTEOgadla>MRdPdhWTDB&CAFDUI(x# zgfS&qSB_1eakCxsx^kY^mQ;jyD+pX%et|?8*jw^ zkM3pRxS)qoJxa$gr>@|~i$@bOoOq{JKL@ia6I(K|H$S@_JQe2uQm`>SU_0NapY0Um zbeUM_eMMA3jP5H|@Z6Waxj2}gGx@olpO^XRPwPk^A#~uc)2=l}43UV1){mzwB65UxB=Pm%*%6zMfwt{j+XNGq}lT z6)I^5Et=@@oo*W(Cm>7+jQ$+UzfMIOwFPsqMEsLx9BIa$KiSiq{Vlle>6}&o`*QQT z84LepRCA^^XS@)d!zmulZCSa}!8}&&&AC6>+FZ$8f}R5IR<(J4TFW(z%N9muVnHSj zdb8G-;o0e(gPEVOGar-v*y_hROrOs= z{5ea%VCt7_t;N&YOs~t)?|3G@?+={+k&Qob<0nS`%z?icXX9KLGvav|&$$G)hy!zc znuP^(jgw0ibz*8|qD`quTWhD*hWc=$3$w?2h=J8$s4ok%lllom^Km(dzQOb>#q{49 z)Qr0}mdCR|H0w*jR(Zj9vn*^`U}3-Dsk)8=chd*KH)xrG{P$2zNxp=*#tH z^fD@~Wx3)a_2V{9F!YUnE+#2v@JVP$M4~-9+G(By8vd6t{kkE=8C;4{ zrPx@OGv&Baj#uRvSe}s;NUgxIZPHjlXi;0trPt!BuWc4b+(GFWox~Hx^ z0%>J(w0i3py^b6Iy&9r@iZ@$2v$L~7nw_!wWm!Mecz2!L2Xi8TJGEH#D>MIMe@l9| zW?vYyx^rQip(ym5^lT4?1K+>-76`D;wAS=~;!DRT%UI zJHKRMEBB+#(hpCIu(|@h6&Wq$r%f7obE*zQ+|h|b#M`tB>&xM43~$JohUU85hl7py zFu+(y-TPJGh1ji1%&ftj8l0%Zt#5hUh!>6M(U{&pv*u?u{mjiKyllesfd*BMkJplR zZ{C%qXLY5|^lWa)WDm0G#rR%^LD3%K@#UzMQ^Mu)xStj1a+s-YFQyGO)Sfe1f85ZQ zHaGqjIwcB9_-ENZro9kf0Hb_m6a-5)m(KV!ZCi-S( zNEW8#CgoG+f6DbwdGslZE3u>&%bSq;3v+)V?H5M4<7_@gGV`KpQ{4<>P=LEV*Ui?@d*n!R9;pB1~% z9=^%R!8{z#!^b@I&8s|0lQZT!{7EaJ)=j%6`FRt-kgvGafj(WC)Rl+oG)_^Ie48jU z6^AT_Y448#e<*Gl=izkLKIJKvFgY%|x1SM8O|)s9V=F_w)jX%4gEET*KQYq4`E`vX zL{g2&czd(oJBd=S@d>tQXJ>5(8Zi*Y$vhzrI(_gxvwYZ*jhESY?@O=j41>AF_)wgl zCD{KJJ6kaAAFlScaG|e-k?Cc(g6rcgJP-_BXknXRr_uRBt9CwNfvA9N9FvE<>zBc# z#5L4QWoj*h^f>e>n`OS{1XUPbpwfrBDfi|`Cyu$D-XLAk!++{CFf${v@HQ*Yf|(UU zT2a>gz}6<*Xvs!9lR7iFmnd?WKS z0`0NR7`>YJVr{aa^-T8R}Y;CMO7b|>O@i{Z%I5638 z-`o{oCdYwyxXE=P;lXkG3}hk3=VJ(r4yJbr`j=o(Nd`wSx*cy4d6wc~lUg}lKq&>2 zQnbSK$V|U1Y>i~HgV#LHD3-@GJ>Q+00J}h=}euz>~T7h&C z$|Vc$Eo{xq3U8M9u*!#LJ`Bsok!(E9#sXjV=ism(o8UDZ3uIwIt_Jg>5Iu{qA(lB# zMzrH`I~FByw=0JxSa>1WB7)X;zHCv!0zUeNIvYLn7GQmbFoAh+<4E0}aX*AO_ilbKRQfsWQXL5_dx+ zvziPqwi;EO1=5>bx%9E+1J#xGPjH$3ohd?61$|`t0f{dS9?tEaYiH-+mT`2%ZeL z&|fyw#S#DIHM&P-XVrIH{+)BGJgc}#V}SAQ)?h_RuxmR9`Ea5>?}i(Z8=b8A@Bp69 z@K7miW%dth?`b717cakNO%0wjVC)c!r_Joc^DlTd#X};&iwVY~0z0@I5s})uR&gLz zd9lCUgh>eEMCUIyA_0k7W#h%oDTW&puFU~?F@2oj)H%#bPWJz1 z1a0(jy$#=TQpM!;St&0@FEXlQ_01`I`?5Pj?S(~GtXNh1Rxjh!UN_Mk<4P$5;B2jL z5M^b0eZv6gX26??^>~zMFb|!R=gro795hziq4mkEJpDqMJH1D{!tiJd*jx^M9Ma16 zGKQzBjUVM9wG#6xv$rx+!kIYN&`!Fol#@}lx%@rLf8?lBiND@)8{)5|uwoT`zIv#? zBGCyZwC4^Xyt8mH3rSg7?oFC6)3Y-tJ45p@G7sbO@jgGZ{W;*zHMk8of_NCjs9>%H zb2*d)g*aV^#l`qojN!$3Uz+)!vFB?hf5YM$yspXomMm|@bsIOr7#+z72Rj|ykD_NZ zuVT39jW4WVEPE@w%sulmBgog~@^yf(?T0g7v~? z&Iv9Fqq!uG#%sYTVK!$37X()Y*B2XMFfD!aVR!|$s}?CmOj0&h29_`n&edjZCVFLM zqc^kjGcJT-p-c*8YY`q6=XG)JeZkabq&8zlD^5o-C7N;K6eJ67rxVSQ9;C#NV(gFP za6H$=rW}$98#2Za*iN<>eG0;hrHW#N1|`_TB4ZuxOu~J~#u~A4Z3jmyABrl%66cJz zX;oo1?#;3AXpZqwdS3Iffilb{`4K+`RAKUutoxA-e{puOh2gU^)=S$3$(!Rj*%hV? zY-jCu7JHKBA^qvYs}9_oWhnLplbF~!^{B!)%~;OuU(^n6@9eA{^S*;&F10?K7b%N9 zMvFH$Qe?$t(>luD^on3$gwp--4zpO$G1bk?Yq0OWRDprZyts7Zx{8UrVU}o#;JISX(zf5A)dUcYnc_J)Wr_KhnytmA=$yK{uUMr z&Q3Pw#8y!kJn-U#QPSr3`4`$T>y6WqCULv9xjC9!>4S1kS46o)eaqeDxp`?E0SIa5 zY#-J7i$rIK2z^fB=1`+P(WPa{UaVG>FF4Q^X2!vCu{|?a|0F4z{WCog6)Wd6D{E#Z z?i)LL(!H-bDkY=SL?zjflWjTam!IMJnOuU$)fg)__o!%zi-J@0ETk^Tt`Zul|11RF|oz6{gKvQ|9kFSu8e)wM{e z%efy|{S%jdVsjIcBiZYqM_cAZu_ubVF+6p$wjD2B3`t;rXXbZf&Oe;_hb3Zn#;5xW z;#zJK-8FT*u$=TZDU&TM7XM_s=(6<-EbJ1T5X=`Ba+P3@;Jsjs(ehi^gUM->=PEH* zpuH_xKUrREF$&xK&1UpzDPsvuN+`B`Z-hHc@_?sQU-JSnHkU(dQoMPPlRa)9p|DAy zZ+wIy3#NAt49dyVP*3s9oE#7WdP$aJYqH0P=EVZ)8)C!k30k_Io7I(AUx^P6ZpSb& zR*@h_oYv_2@VqILrx?+DEg>((%-lRK#K;CL`;)CvOzkZ)l-{~$riFWgDXA8Y%r|m6 z`aSvJPc=ffMT#&r3p2B@G7CdLW$SlrXhdpD&WnHFOMY!&nh_AyO;b*mi!g5xY{?*s z@phfP{etk)LVI4a)7r6doPLRR*mthl}ly_L5= zD|OgjU;3GcwBo$%!jP^k?Z#~tkuMzRsloUB;)WPU+qIk7n`@!;QN2sGK?~W+MlD7_ zYg=oR-F2Zl7h9=4A8prd313ErGBuQxC{Nz}nqSn9ik)-nM?LMJ_Oot8Hr|R>Jd|v3 zh-jN5#>Au?_E*aho~SkF-YkjYsqyxD*3y4~$8g0xxBe*XZ`LY!Q3thgO>9?xtC|75 zW;t5I(6U-_@uN={Ms;O%H*R+0V^6UoMvr#Cw)$Y!styMEF>|j*x%2o%0^Ch#wWID= zQQxWEo!;Cp&VrW80XGTnyo_d6j9P1FVr;mHfALi5TOwTAaynlIdUL!ud&{Y}i_vrT zrK1gXq@N34`g<|Si>+RU-O!qsZ0w$8VXqN{D;Vi$?J(Wz)70%BwAB}<<>2by3>FeF zeUTAN)H7r+9?kF|y6|A~s-8dON$R;+v-$Jjbc$|GAh_oK?}RNZ`m6kCSmx z(agf85zGTdzSTe0NqZ2 z`IwVY`AG9)Kwu7EGuL%7K)jI*#qa9-tf3I~|+`W&om&*~}S)2Di<2J~Y{b~fhbWo>5v&P@4% zwd0H*F!OGNid8kVp1!E66hlig)WNNG+)ZFrXBLSvp54pB9l>UCcgIb$kT%VTn(FMq z4ZdWhh5ngH$wX=vc6!sxm*qmbva=#PJ#ug)H+}N4F(1SIxgNxaAo`VNO=(7a%A8NR z_bDUFaHb5C%d)Bx{VMaKGTT3A_7{x)l8M#W@-;~{xK)#twRm2eDRmk5EpO{{`8!Vk zz?~mi_7i7+V#?2?{?1?CT$=eu`+A+t);jYY+x4fTm+ACIn((>8gN~Gf)`rPhaBc!h{HMH_7R2h7e zs@KhiqKPnxRlaJh43r}w{W?4?#kO)y1sMIKw5L_QSf}a~Lx^)~BcHkGYvsvj9BRtO zRO2(_^?~-4Wt4xc`^xN`4dhHC4vaQfX|y9=U*c?c3m3W_p^f{9qiUp*ap_Cr#lxs-1dwz@ z;}hm)XH*_8^|0{5a9qr;9NM8S+|Xu0{Mpul5gpmkk%t}W*@*+4IAv6SnJ~;GgBAtE zn>tL%LCTbx*sP&x8Rd#*U3XND%FHt``l6&42%E%C_I$~Ns_N@Z5+-xpsmkDj&IAv! z_$T=lyU{(WC|50P%EF%P?EHiQ!3-_Wf(rDi!_g?N{ll)|hUM3vPhDM^8(%3I)h$O( zk~5K6v8VWIqeZvP*UeZp$v2C%a^msPG|EGiqzJ)YzLr zEm_;j!v+xlZoY?7Q(ul2=S*?l|7}$-Uik%d<2=J<1klb|z(~OVMl-t(KE5rRY_fe;u4^uS|@7_fE~l z$Ui*#>n|7lFY1{8ZOKRD6lFk^$@_`QD8ihg>do}E9Rq|lxaAXx7Z&L>3lU*^^`_482o+S0#J{;^Opu zAgvs5-*{Un*yW5%kHnc-zJ6b+jiXOWYF z#CV>MUMk$_rwwT2;P^M39%*63yo^#g6T4y*vDP2xsLXvaX2{05$AG-nOrYt<-ym6LN%#g-5Jdu)zzh7v@OP5gws@uMzf;= zBg=DMm4gI2BVtYK+c*E=iRO7Rc)pQ8?W{N1Z=;l}XL2iXOLJ#XzAVHuAuTph+Ctl1hJ=9nt>}Srq}Y$JkvFW=#`+Rd1we z$zd7FqD+j+!t|_M@@AheSABWu%l7On&CP|}toLUIY%Iu{Af^`KKoKq%;bRd_7G+Q| zrWa#JF%A@GdP&xmBJEQKlx1K!_Lbvu1rAqaP$iC5X2IvotIDBz9IMZ+?-OcrZw74t};HCg_s|vLMdwYHE)tf8+Jo7j9afE(7{JAP@D2UV`hV&7Ooh;Tm z+2GAus-A~KS?C?a$RO?tNiyzkr;};1Zmg2B7_-Z^U$ik73Nhm~nWBz96>vqF6t-Cz zOi-9Uj;n!JzZvIARdiOPynlsGo=$IE7?CuITkeNcxsVqX)kG-3Gw4&fE-b&cx=77?)At zrmZ&1&A}g7C(hs%!FVHvRL~inXqM0N=51LeS*yZTm_P2X6;hi6&1V03sW;0QB`>3Ff!xEI7k{ccDy zBqlLh_mgFLT8`JBkz9v;KXBj&mi4f(TBK*K;G*ENV9Zzx+Y`Ag#uh6m3lorIfsfR>8-~MmCq_*+BeUKUR}9owB!kl_7LfRT-7jI ziTdq*NoCXvxEywUEm&?2{LQt$NeyF77&qMIiJ>9Rc%!SJ32{1N^uA4W=}G*KCiHK{ z*1nz<*6TkQ=01}j9NgaS5}GA6-lREHkfDCHYzTJcB&7uXYB2wM`a2lqVBSI_<0USd z_Lb8*(ytR+jWd(}?M?3eJZNKKYY&vmhbuPLWi;z|YIj0$_U2_rD9=kVr2k@e&&pTR?UEMs&(7{5EGf#&((L|}8D-c|hTY{@QiXS4^871? ze9fks+^BFNe^pzJKXS}FsBKx{K zC{)~}Ky7;|7S`mbtWI5Cxl4qDMDg2PCdHM5>ML`j=SJ^}s;#4sYUO06xFT)Y+L>MI zHae5&#D=`aUN;fyneHR8!T$OtR7FV3%swj@O0c;q3mUNhd*=SgrQQ}c4Y6=@tP#*N zQFwDb^b#n_t3R32n%%AW(2fNz_I9D)WO@D+*}zlfKaB`VKul{}qFxj3t~D7G`FYv_xw!xAfxFu#5sm-576Xeam@ObX6cBvbA23`I1(E0R>4> zwR-`H5s@YeWM@XeHaya-k>BJZN*d$E2i3PwexXJc5EY?kX(upppzG#Z{Nf(OVscGV!dh z2b!A-Z8XB`mUCefrge1ei`akI60JSV5axERLH zaIST$9W#~SnEXD;?jaHr^ zRUI8VN3{R!v$%MDr=kTaQxl|(^ZC;!gk3-Ip#{swSU4*fVbrA-Xq|q;fUa0Q8#l&_ zx*Kn32d#?C&a?n#{Z2}T9W=SZZDOzyDKpV@Q&lKjTOBCWr#4d>o14Jw4EJGW4N{CN zOSBsy3+^(lkkD|4J;E6&I@s)zn=M_ob-#<#JBnMO*ky3fZKGria<#n{6pUC9Q7GB(uQ zB*M9>9IV5Nx?FF-l9u#|W7~9NW%PyC4?=h!PLk7;G4NrulQb1m3ra6{)UAy#CqlT_ znA^V@@gIAb*;TuPRgH2%d$fn3%TXmm>6MCMW{JoBnU%Vaw8xk{ZLO*Z#HnwVY2}7; zar~URg{ei4tQ;@El>%&aGPVm(ME%SiXQB5D(e9#d`p*%KE_f@LB5G%vV1?kRASKOX zekdoy>O*Uy)rM2~kwGIZevIB=YG1};y=4V1AR}>+& zs3Fr~k!G&(_AUl>NiYeVpAT26rnN0T!J)l!EFb6cF`xmdBQ2~HtQ%=)sIc_<7!v^> ztHQ6e2{WlXpJ#NgA;PTT^X)E1_)uE|mxtkAyz=7QXN+#m@ZpAFn_aZ{SRL)~cWK*J zi`O^zfAOHxxeQMvy;Fo<5npSq6tS7=IP6gVn_i(8$Xz}@ytla1Dy@#$Ur(~CVtNe)ccNYgEx z5m#>fA`6EzZtgKDL&)zb=(N0@XSZfo5nd~q)?OOw#eJjdLLW)V#?w?0gIOLAvRLU^ z(Qs5v5=f^q8pXv%n0WF1RqjE{wS2kn#q8$XSIZd?k*II=AOSMW!*PQ z>llUM+Hw2-P;aMmi)%AKH;4Qg9ZE_lYZ@~qns+V+cITwH{dWZG#9LmJY~kZ13)h6e zj}Z>{Nbp=RCoN;DP0rM&d8t=0xu`6YlJpoIE6+kUHF>e&W{Y~^H$&7+Hu8(&>TUE9 z?eQIrc#z(Vj?d3Qe`cuae2+=njjBPTNk;Hs6Jr`p0_}r4s*SFz7R$}dl1z@{csowE z<5@dH3ENF-l)h)8H(Nj9TyCZptu$ zqY1r?ZnU(vnS`|We`e4rGwEo^a~F4TDUYlJ`q7N?o+d#*5b~$R89KDJvLl!bhKEQLh|s;tPIMH#7N1 znJmqf)@n?g$!|Ths*P_FKiI5xfLWKlng0vB{z}IgAmsxvaX|7jT;AGH?8B2#ij1g>o+wOY*Th zKkGi{)Zd)>n+-x=M~Z8HVNu3u3_T4v_upD1-ACo;eJ<7r`PnGgE!b=HruH{$Bt58! zvD_vd=GCDz(Viy#PxiQ}OpxAGdb6x4Z<;F08>TPJ8gA6x<=1g5Z{9@mHqwKyVdbL1 zQCe%=n{2?`Sl)IwcL}^X-;86;)g;)%Og!?9hHs;nlOFCwTS&oJNBRj0t;?K~Q`xy# zn#UE{?J-L}Qr(?NbbD<&(wh@?8K|yiBF48p(TGkxzJS(t2`YRYUUNdpWW9&6SgW7;o9j7J-zvof!sLA>>?ccx_| z*ToUgv+m~|lnm&*L!>M+7F8b+_F~)OjA*6ll>sL*PSu+oIjrg_I^;z!m3t|nrMKCb zG|EB`BSjO?(d5)Z5V!y0b_?#cpl?eiw&Y1m-nC>*E0(oVf8E*E+C-EF{;IUBzVv;b znw+4ZD7&YK`FW$6bb_|x+?(;gGo_soC~#>53b}cjlV>@Z=0}Pjql}B<^gHaJ`RQ4`Jx6Lq?xntHz&eRqc- zva_f#y^7J}Q+oVKk3@F1XJ0oqt6Wl$f2@}AieAspD)+PoX(I%|O!p!!8%u*Z6T-qe z%x=j%JM&_gVqD+i-`a%eo>9$q2#eH`YA@boT(+pc+GjqCGV^a1-ezH-m5Dw)^`UPz zPWy7!m&d*w%*pwj?8!^7f~*eZL?I5B;&5r!RAEXZ`u)bfznLAu+6c}^a6f_<5!{Pp zS`^Eocpk->_B`p#^Ul2P%%Uzl>%zw_JnGJXBn!s{xBFVyGswb~VHO^x-~22)ZJFTp zSRr6(^3Mh57F)Qf#J+HFxIM-h<}$f4#mbzjo_wGp+_Ew}EC1%;ZVr}~;B9G!mtkyU zUUlYjS6;a@4Eoa&!SpJ_#xnH&hV8YvT!(qzv#t@dqB*4wEfq6q;Pap)?@KB_)Z`=H zyWRg{1zaNbCWo`8eXo8_e^~B*DgG_ZxOzPNjVs3Gp#gT&OS{f^v-1~om!uPS#M|^{ zh7bSxFfto`^YGS>8wI&ijOWF8U71l|@un8H8nfva`bIJ|mff)&bh5_9{(l%N3x7p$ zp}z=%2*7Hg#zQ7r7%g}%UhPLgvUs)shqQv+}(qF2n2U`*Wm8%?*8V<-syjJSFN4C_q{jn?ZMaGBiT7;@3q&Q^P6)` z8NNvVcd3arvJtlF(Uttl0}CrY=|j!cmUvi7(SoysW%{vsyv@a-{3I1-auG%s=e{qKsxqQJ z!<#WSm>r>v4(E0h2l}$X(!6<~iN_O6+!F0@e!6TD!DQJiAN5lZJBuTvZqt0kky#~K z6UDfGCLYezPfmJtQZ^yCX0u*?ZPZCWsiC%@k><$2xM=ltx3mgv1ApP`pW2rhu$k=@ zHMU!|(EO6j!0nQiVTQ5y> zs7tFqnAi{ZN2lJXik)QQj1TAZ0hzXDWkHA)4^zT51ssG&n8TE{?U|V{Srkj^95Lng z&%@)~42WWA6qDr-;!{U!g(-uiww$cV!(Vx{#+1HFcE!Y^q6=4dU*(!T^=x|Y< zf<1y0g54ubBnvi);yk}7)kjdDti&)a2QOXOwcq6r(Dvb_Z#MUOv{uYV?ZC73I>uwF1m)W-PHCgup1 z2o^8a$&EQcr8cv3($YpoFl(Ov@{NK#ueGQkL28m?!{IU<@z%pRYR}`#0{#0!3@r1u zPTF=bD`~YgYi6e|e2_u(=#8{<6npjNLiV+Qik+<~z`0B)=)d6e?&`GyJ($kc7 z%xy<}dsYQ#zLDy_ae@hZf{kM6C*8E@R!Ee=Mei}*^;x)( zfsYwjVWoYe!H3!K0_N1zC|`7^fFA0zyyJDX0Ht~#S#y1Hn%X=YhkEnYVnPg_f@+<23CGAAE<^0C>C{_cFK z%)=U-?aYig9>y_joQYxLnk^EnUTEUTViPA*57E`)mxZJ0m|ljneKcWE6YpbsfZD+I zir0=-okSsr1jNJ|*t2!1XI)*@krFG`#_3j884$`%Qf9X2<6cv41Ts};uXfP}Ek`o3 z#gF}ZZ&zD&N@GLrXJz!yEc}^~-H7kT>293w#^r8Y?Z&%qeC)=uXg)`?KZaql)%FYS z2quXuw0WM;yaaJ+1PKfEgI(pgk1X8o#)?>yv{r0H8xgA~TD_oslyR7Gv!ia~%{`2o zrS9sW3A8fO2T=?#@wyJ9>azP+rj0kTcKpArvq9sYOIg!WugO|efLL>=dQV-+tQ&bH z&zI$E%KT~rC`R2##hO90Qe9lbhR=6gbm!hOMcm-J z%UWNc{7T|x!(eyDXua_q0;7eg8<5zk8pp7U3^khC`Mt7657bqendv7QbNv+EJM(l? zF30!Dsa5m_4>>^_WNaIy+~8T6Q-!n9oX}^S*qW6^&o&IG&-s)Hz6+hAl!v{Uh_BCZ zt$U2RK-!9%T2Be1L%Pocb?Tfl6JAX%E~LZ+iaJ^hgyGEc4QaHJ&BDE!?5)F4?WmDZ z^(j`H8Da|(seDH%V_@f!+LrYBEX<*?8*`o3zmxI zT&nBaU_-n1rh{&U45FJ4oR|@#@nZF4itn);k7JYA_N96oYM1VD<|wa-$dE3b|L7T^ z7S{24(U0vP%Js{{ypG&it`{Gvi!r}yG5&mk$}jt&Cwe=|8ioa_^K`5^t{#l#q$_YKmrw;ZFo zn~eyNH3#+OaEdW^i<2u2j*EAd)>vz_LDQck>wo#LvFb2{HG8yta%VeBb7S?#t<)`% zQJq1uc8w+G6=T$oXn<2%+URsjY_fn|P>@=G*l@B2BkOCsLVboUmw9u&lEy*|(q~_W zXarAX7^6u}&!zNi&%vEMOwZ4EM;;bnl`~5UlUR<2z8tQ|)LM+K&*)Z+Z_k(C*cHtF zNbdj1yMZRA4Ks0Ngo*y5ls1nMGaG%-;Z*hBw8auTsOHMqStoisS#U5lt2a{J>GXXxU-fD}XUk;5$XIh^utEKi zH9Hoha(Z=pz3dEi=i)D{Y0sD#Z9^J#FnpPdDRmjrSW{`rgbEv;m0@9;y}NAKn}sF0 zjMbXB>M6;Ah3qZeCk$T8*OYqAA_j>LHjJ!He06OTM;i=c->UagFlbyc(VjW>nv$}V z)vH@#ENI1!R?KV7qShL-Q19b}R=Qe_OUb-9D0|kXkl1xM|6!m#KE`PjB~0Cb%8LHK zrWKH~Vu6AB_xieaVJ>eJrR+sfd6j;1hRC%T)Jo{SBvV%Qp^1^;@OgO&cO z4MRg184;^b&HN&17bUw)o1=TTz2D3io06aF07|fLe#O!npCRo{WHz&b%+d# zHu%cSJREJ}u^t9+Oqn>PeiUD>WaFyvtL!{-Wx5wDz1Zc&CoiT};)j^20V9Ry3T_G} ziplyen6p@SBvgV`n`JrJZubFJE(7 zFiyT=g?#)?!O+oq?vrZ4^3uq`#UrW<0hM8qM^fTWe_03uHNN)kHZgXzU&FN`WEazreC7xDdRC`th zb5|c`r;aw{XHjOpm*!M69z}C>u&ksOvMXfJoMc@Y9(U0$Qp@@??PSo<1nkpN=Iz6~ zbPZ4%PHT4kD;J=r@}pL1sp~|k^*}B@*)kGd!yDM4*;#`go1)=^nTG+~1l{`WR0of3@PZ7A)nDa04TqsI_~@r(Ul&T5C85bAVCVb!Mh7 zHgP|-fvs9%Mmq6H>!sij)G5+1wdYD1O;W}6Q){QG`f-IrlqFNhU|DwtYx+g);_lFM znGMQP=Icp+c4otQcYCy!P!8eduAQPH4SF^<(=DtpWObEwc7D`kdo2#vVwtvq3`B02 zJ%>wZ%Gkor4XW>U^e_%JS@HRIjc6-Z)s~@|*;$wG0sIg>e%{jHK<^B%TxYnLo%1<) zIM&3CaVE}9G;vaIJZsQZ>2e!QvXy(;nl&?0YZFz1U6h5nnVW~hc{yy$z`{%_%$mYX zE5gYV?CEDBUe8@~2#d7rCv}?qoZKwPZLHbyla!u!d6}A@jYU`>pPsDuSa6IEi;Xg9 zGHK1bVY(oxV_%2-IO@k`p$soa=m~eqwXxw^Wv=VNT_e4>sU&AB@U|*@YjL8MariAW zZ-ra!)I06is_niNADZhbEyN%vYMh?Fo?khTI}4}mnf)`*yBV*1EONR$c~H7oUF{9$~2=kM5UNc-bc*6#$GG&25zA)9T^oMb@GcWyGBGh5Q*!VwKWl9{50~v(?Z8$CE;;hhkOSCOTa7*K^B)frKP4RzR5m%0ATYr^vuY;VPY)(mgMk@h5YV0uT^1@JwX#b)|< zVsIx8cVcxI_qy=43mYOC9L3})R!1?eD<`^gsVkeJxf{*f7{~ zDitJW0n$+z%LptJSH!*vPc`vQHdQ*-r{h98rln_odX}bVU3xaBXLowerDseq>;<6;rL4ZN(8QlC3yx#RXvoR;;sTqcxAMd11{PYd%=> z*_!Xx49&=hjEv3502`**uwAx_4WDiJX2aM_Ov%LIOq|QasLV{s%)HF3$;|G|9L~(= z%$(1{(=5!+%Jr<=$;!j5e8|f9Y)s9@!ff2l#+z)+%+7Mr$l3Xjov}HXnu8@d*qDRE zIk=L84>{;B$|fhtIk}aS*ExBglaaYNn2RI1c$JIqx#*vp5xE(cn}pmf&CT`P#OGm7 z9+u=`eI5?w;ancB=HX@@UgY6h9wz5yPF@o7l9ZQ2c{!Jti+OpPm$!MDo{w4i*p-iS z`FNI(Dfvmr&*uE>&(E#=%(o@cmJPP-wdIg44{dobY8-~aAvh_f1+Kz%xC8It2h6l1 z(T;m|JhNk^W)AS$p2w^TnR;_6&4jkpmkX*zCX#2TnTh z&4Fc(Y;fe5BPSiX?8qHQ9y>C`iD^zOb7G|v7oE7_#A_!;6ktpN78T%n0pbfXy&%gA zvaTST3o^bCvkS4H5Gx9?wGev?aj*~{3Ngx=>CUWiW`i@Eo!Q~cUS|$EbKRM{&OCAE zr8D208R^1I7v{OJ!i6<1tao9%3;SF+>_V~&r(8Je!VMSRy70k;?=B1|%+SJ&F3f_$ zBo<~_VOAIBRAJ5*=0af}7Up?jJ{M+u5vCSlMG-a^;Ybm#6ya_W9v0zg5ndGGT@gMO zVM{Fxg+GqMD;OOQ~4l_gkHg3Tq^TY^U=_*R0cC7Dx_^(DDdl7}UEQ<4#-NG!#! zQk*Hp^-?@8MZeMvF3tSXB$g(rG%HK9wlo_{v%54uN;9Ag%gV5-3|q>uy$pNHaHk9( z%P_7iQ_C{5EKAC=p)5Pfa;YpY%JRA_-^((j9K*{oz8q7^F{2!F%aKry#pT#hjsxX5 zU5@YN=C3!x$gt`H<+t zK_4#paNUQuK78&xN_tf;`c3T&#t!3w;uz^4lItH{8L%&o|} zitMe(^NPH!$bd>rs6>1v7FHsu5^E}vT#0j)xLS$(m3Uo=@0I9ZnGuy4U77KfNvh0> z%51C5*~)y9pI2eJSgtCZufmNgJgCCAD$J?M{Hm<4%J!=4s>-RVJg>@^stl{f=xU6w z#*At#tj6|g9IVErYTT>Fw`%nBW2he^{TS!R96#p!vBZzHejM=QgdgYqxb4RSKc4#W z)sM;5nOdDC)mc%U&DGgiojuh#Q=L&Y7+ZsBHAt+%x*BY(!Im2AsKJ#Q+^@m28oaK- z*BbPz$)K9V*JNo;Hq>NyO%Bu~xhAJ;aSet{jxmcUqwYgWDXSMlMn;~_GuR~HD_SE4>9g^#Crw+5~l2Di3!p-Y4 zt{#)>F|8hn^*CFPi}ko&kNfrbAV06qiu!D<&#w9$sn41EOliQ}1}tvCng;A_z<~yw zZNRk#Ja53)1`KP+goeCo$j65KXvok;jA_KGMr>)so<%m9Ce`ZL;}@%~KmXR1F7{7Lj@r9XT8IqlCKf4=y$yb0@@u&oKZny|MC$xXQ3 zgnLc+)`WRY+0c~jO}W*ShfNvSjCsvC(u_0BxZI3e&A8i)N6q-sj1kS5)|~ClIo_NT z&AHf|tIfIDoV(2#(t_bF7}J6&Etube#1<@V!I~DVZ^8Z+9BRRd7My9p;&@ zw&q4_UbNyu(w_J28PI{j9T?hyDMA!GFsB1+JFv3@ zdphu}10Or^r2`{6GP)z8#Fj0<9B5OafA9mM(|b_B6Ih~q(g52AlCBZJu*%${J9gLxCo1T$02 zOgA&%%mFh;&73iF&CE?R56lb;VN?j?Lzoi6jt~xpa4LigAzTUJb_kC`crCvQyHJ*NVtpsJcH%%Mj&$NoCmwWSUKm@$I1|R>FkXc*q%)H`v#>L#J9D`+&pY$I zGjqDIrVG2faJCC?yKp|78{v$MU_u1TBiIwcfe6M%vNn=Ukz9!6K_nj|`4Y*HD27Kd zD~erF9Esve6nCST+La4kx!IKs-8j&V6WzGejhEf{)QxZ5=oih1Xy!$;HkuvLyox3< zh7&Q|jNw2mmt(mT%Y#@x#4;_8WpUh>|L)Ga?tK4)l|5P0lS4h3)Qh>jNa)3`UL5Yl zv0j|(#pPby=*8_`jQEpre=_e+mi@`w9yqHJw>NKkGqn%%`>?$ayZf-O z5BK}5(Ik%CoNr!1y==k1P=vIEdc^SKXIvs3C0WJ1=9tIf>nZb zf<1y0f(wFsg6D$Qg3p3wmT4J+E0*IAalsY~)(Q>?k_DFp*98v*j|J}pKLq1O%l!$G z1gpelJ0_SVZrd|KKTDfn!6?B>!7af9!4tt-!3c5dwhQ(Mjtb6L;!BoE9Ts;_a8K}9 z&|jRhDT29zC4$|8{emNcmx6&4>71jEFe8!4D27#MG2u;8TNtzd$r<(XCSSc>$e!&C5YeDjK`Muz|;I&}v47pcJW#mlxF2NkZ7Qu{J@_q!n1P28V z1WyE?1Vd)aa|%WaCJ1H<77CIC%PfN<1kVNEEZ2!EJ5exKa8Phua87VV@JKLVt~`fe zsvuGDNHBb!ylcUH!6w0eL9*b2;Hu!J;I81gV70i&`vu1YS1s!stSEUeCf zI*hK*rKX&1!?iZtY{#>9tZvWl_8e``;SPKXVrwJ=qj}zkt)ib_$g^*iXMZg6vqSdw ziPYAVs=#G&K4fI94FfaLzbtF2bE^fHeqn^@;!W~7r$rad&`+!K8ywePuEmhrS|D1T zfBD^(Q}EeSOA3iGhpY2jOYq)`RhgNeo%eaTSAg&0(0XyOB3nB!)^Zpr7^9CyRmSx5 zaV`TJD|6qE7hPB#!E-&&&dw-Aote3!MU%(r@$8gCa^+b|9_DxAU?;|fvrcTx!_*1a zVd|r{FZ0sK2h7YPxq*jzM}Yjw{He^`789J1kjBZ8h)sRgjnvNxR8QTStr@wIjjh?) zot?q?7?Y3q;>;+{`;y$L!0b?7gt4hJS7n<`vNXFCoEDSta*~LcsQas;!ruxOh{)d( zd=~7~x1Ty6r#j=Zur3Fua`HJRd-HH24`UqI>df#WjH|*vKkhVTPBXrBVBBvcbmEwN z*Dev`LxP)nKODP&5Ov>YE3Rr2UM;0S>ds-K((^1m!}UF-ZfjYabJk4G%-3w}&cic% z5*%3I%+(T%@#Sa*23O)yCB9VRZWUIG*tTP0Fmr<0)}2c|c_#vTb#WTyADtPUo;_CV zw_>3+@3S!^J5Ta(+Mcha7+i+yWf)bCE51yw$T2@=S0}zPCz~><882G!qCH~*85YgG zK5P$Gk<9s zrHWa&Z^sCG=KpDYPGCi%wI*N&-EEV^`0h$fL&&o7+@7RRhOGE6N^l<))VHyE9^htc zhGk?%6&_Y$WTckAWzdAeiZLCvO`y7BFt^h2B?nXTao3iUcD%A$aGXfRhhGc=sbUD+DPj9#qk#m?T0?!)&!tP%mfDckU~ z?2Bn)8{dl{Z4_SiNS5u6@P+}x4;F}>KB?~yq9Ov+xe#+oaikEgqpNeW0YjRzxdqe3uMXr*D2qC= zM%+b<-50y}UbxKJ)RfS;P*nn1es<*MBQK|C+p$H=IX;@CH+hH& zFz6RD+&-!#t+?Z^ zX}`D#Rrx_h=GZc|1Q*@d?aPL0EUU-!hAeK*gmw%MWlT42SQ`F{Z?{q2-86YO!-aRv z)Z0loMFv|&Ll~v=FVMPc*hR+#2dGoTZWJ@t6ven}#dEOjO)d?;lanM zjB3S>)~pNWbr|1!^HR3)`jsZ*GwHl6&c=>x+>$k3A->s~6=|hhTk%R8&Suc5ZL41P zqOL(O6{kGnZ;R`4Q6w-OGtx0%_+&=5+3+zli?T2llJ}YChu{s;^IoY3!54ri8 zhn0DGmY0FHY!}MwzzHXwIPtXv>q;`b6!S}QzZ9cObG|gw%COp-iyL(~SC{z>xzUKFjXBepbxoMmlshe% z@GHXtxEa9hAa->jsVfU(xEjOISmyQNOdp>0W%2-dFu`I=Ji!t(5Hk0{QgJDJc)zUk z55aB`y&3Y7?uwteRcylo(W^T|O}-TVutj{nS1a_C*>JO)udkQVByW(jJINnh?W&QtT-$i z+KNfmthMHdHJ7cqpOK+9jI$x$h9@@k&&0$`Y!~k*GqW?ZA~Q>}a3~AMvT{2s53=$! z8-ud5EjxR%(=P}8axySCiFw$XhbwuQmX}9)8Iq5&`Bau zz~q8_aAu4PPhD79m|cZAUWB(rSS*I7C_9RhT$Ia2SyPO?#TZkZnZ;RNoEyctRh$by zbNy#tm*8wkE|lbQDb|ld^0s$K!IWc4f0G1KgP5PP_-(J-Fk+ zaBmX4>F3LIUoQBvrUF|ku&n~qDl)4gn<{dz5>qR)qcV>x^QJQMs&KXn6RUEwDg&#L z4)q)i*x$q0Oe_>!NKDXjnYbLbeXd6zo;YB-6wr4;` z5<9XefMtQ)4dzt{k3;zq%CJsc=)~qQ9)&TXGt)Zrursr}@VN`4!dVf{xp1~cuso9c z;#^0uG>U%R_|T2P(JYPTd^8WEnHZuI6wZ(jH2TW^N; z;aeY0_T@rfrirL65Oa|zND`bFWa7DCxajB8f(wG@LSJ49&I&afC01~eV5MM};J9F? zSjA(4n}Y9Rc7|A*iV2Pi-U=>@ihM3OE#iDj@YxdX6FTu+Fw_#l6IyXs=)_y0d|w5d z#boUhyb}x)T5(8lUhrM;OhopT;Iz=GnU>I((8H&KuYyqtCcX%ES$f+G&IrbedYEjX zU4rGJB7O)4ikcWCYGRmR%yJXs1y3v!C&f4XAUG45)?G~gX<@zX3gC25Mw8~7o zFoWuUD^_ZqD;-ltU>RLFxv!QEX;9#E3$BG}d_CM?gwwoQyo=U|WlU_8>f!V8#hJ5~ z6esaz5-c_=DXq={){IF_3|7}`44RTQ^_Zabh&EW<6WRnzrIQQS^K!ogODi*?F`t@o z`&Tv()19%dVL_@#$j5d^u8HVA54Ct&{kAkep!SNiN_vU>=rZnQNnI~rAMK4d*%VunY47wgXnyKGN zuv!u9%4O_xxv^aCp(HcCxa7rtZ&vzn!-t8LSW=q_jTjir>0myZ883IWV`Lgp7%Nt% zbl@pu7iQqLxH2~@$+;Mnn{$qgab~77Q)?36mIZCO8Oc+TyJ=!%W(g)P(c2v;1E|^f z7|026d+$$3qZ<)S&cZ||hSq08C@Uj59LZ@tFPKnsusR8RojVuZ*=7Eqj4n3?`taY}x zGaIhLS5G~@sEl;ku%!SWifW4xp)S3)=BYkF#lg~UE?RlUV8xPp+D7jl);SJ6*Xyvfzpzxn0L%Py!HaP^A5u@Uy5h)@LY&kLx4NTeHj--a zTFCHBy~UlqWm>yZgsqvTaSkWDxFB^8n!Icl3YC$W8Hq2(oJbaR*B&)Q*#N?Z6S?W9 z)!^q>HWtmrK`TxdW_&Y-1+g<_+PLxDmi#3n+NTO}on6k=`IbT(~ zCNxM*r4twEAs!VAc41pC&gEjHEeFNu7h!&VE{1SEltV%+9_xJ<>@4(AU3s~;8cY2+ z?8hrV9*fibW`r(pVpt8%W(Ks-d2^US^Uu4fld@yOBGhvLWIu#!&W}M6@D**IfPF+j zWLT_0-I+#9mFL~rdG<43e`a$JZs-lQ)pK!{^x%{>!p}Z3EW&6u+=FZyhf_X)4d2VK zErmK3C=eJO7946etku1#=`Y8i=rE(Rb+NEo16hN?nkyUXMn|PQo!C{EEB-8O!-xnj z^`hSdQ3wepuIZI?_My?1whhWho9V}$l%qOl$C%gvqsfYynAnL|3HrzOkcTtaVkc5w zc_D|GuxNv|%79jp=ol3jC?vyR3;poXmP)-}`{_fqokE0X#YGxz*(9x4(kW$JkRcNs z(p1wsMvW@8nICHBzX>aHFw2X}UOe~WfH#vWv9vZ@YICv<_v$dQ5flCS=FcZHM?>fr z!=f0z#4y29=r7nQcyCFF6B@8!if(TVhWXUyP)I5#QyYZq+SD7Prh6rg-Kj#XbK{#E zt9-cWLy|88W7uu!plM0$OZ7sPZ^4=?@%po<*0vV2^RYW0^GmYHjehQYuFQiP+^Em> z<~;t5Go6_p$AdTq$m$HWz%oWpuu3S(;l(CSEKbuST$jx_&ScF;t-qC!O>=aN!BoMd z5{!#xev0d0)G7+E6ECI8%*+#c$`^}u&r;@&Q@)} ztT<9!lk!Njr7@N|0WSk3(+X*Q(Wb6C#RPSViZbx6h4C)YPec=X&&|24O5RF)ag&Gcq_Q8*}kB zH$!ZB;=mr^eFb>q&ZY7UsldicjH=4HT0E&ka$Qz8WLIMzHfCxJxBD`9fQbWw{i5e4 zh$VPFQS{tQvGa3Hd=N|&owr85X=Rd$`%6s>(eKSZE;KIS4>jpJ@~gIB2J<0*)i#zo zxoMLfr-i!48gnu$zP2$o;)Th@>s&m~!=`*3&d)(xRuyJ#1-1o~@Fz$0XqeJq6kuf@ zlJjyqFB9`Ir7#c5aIgul+weHGC9(Q-$LH6?(Wouhqx?)MqHW(8_1XKP5juG{>eQcw zBUxD*#uRPNlx=K)I+naCtIk)##TG;x4I2CrrG0Z$SM^oaywIEz$9$@(HpTdmj_>Ii zRglvS88Svl$Yj}8f|-`K*i%g`Ua5N;B0@%^N>ai;h3FpBMJ-1A^j@j?!_-l-(;=_GNKuW|BN94c?H8-JvWJ`*%yQN1v5rA1RV%5XWECMR)1K z%t~@#!>A&Buc$>T)J}^N`(SM$_PO%Olb!yI5Jh=Ua90-*gNe{q>@1=!l0hT0_;Q?a z)#8nR7zIqW)zqayeMUK^8c&+&v8!-(sOWG%-C!kU|vQPglPOk_H#8nV$NtqYQjXYaUEYJ(>zurkq*xJW=1&5mD-yKvo=g z)uW;TVc|vwYfEK4!ewv{nQYH6?e#0^jtyJPY|}@Q82HU!!Znqu+RJ%4n2%ewys>3v zCHD3rS#x*oV$IP;{rI#lrTL=_rY-c)_JOj8jqnFuDD8q`!y?q5^fD#atB`$EbeK_y zV6LBWm?{VBYp|>l?;3Mf57ZhRZZFWH7k1{@NM%3mEPQOo!a$O=!P@y$Gvpk2VB&FR z##JR@kcnYKOl-G|`Wls5)u7sb*;wPmTVW6j6Vr(KWoC|8fVpG!Egc#ar_MzmAE-TA zc(}S9S7xq>lQwXUerByarLbn6&QI+`=PS<*F=&hG!pdBnbK+(jPW9%AI9P85V=VDq z!GdXNB<|YsF&m$RJoz%FA`d$WoJz4pVPJJICS zGhYK)n$|PrW1P&vDZP#iyonJ#wGC(3GWuM(6kR}6m~v$^bHoyon45-C&&|eWybt46 z1oL}ybyOPMJ`a;}GtZ4lVZ0b@B6(;ki7^;wThBOnW6gpI`UzNvW^sw~OM3K(08(=D0(D;HbwO#ZdzTU#CkusnvxL&eERH1SCs zixJC(1qwC`_UVbp_K_*PR4FOLda&pZ&CgJYXDOa%;%g?RW@bqNW)uSqxE3C&oGWg{=A;p&#TRt(lE zDs0uCX3aajyvf$6Tu!>!1n!>CcoXNqLoD9ju>^y81jh&YX`B-GnbO&xY^1zXK&RlV3gbNE? zNG!_XVk8ykL>cav;awT7mgQAhzLaH>D?44e?amPo9(%CLi>qF2_vVy0cfDEQ!*?IX z`LebmW2>;Y3fHPIq8g8C6JMA9^;uV+Wes@JfPoE};m;+1?)h_|3Co*uq$zisGQ2rw zn)9>;yIS(RC7)Un{|oDX;r=hYl$ZM}8(Q(BHTT-^stxnnak3pVJMg^&8#;2XBg1~< zAh=7pK~P|k-jHJshuSRKp#SSH1>FpjHnJdI;lZ=UsLY#&}(=BSE7oos0*U@4yy zERY?t+~VwyH}PP)iTNU&{e-H`5F8a85T3Pku8A#z*Mep9Ostx3V(mit8^Hs?dqIMj ziZP2!+!qX5Y~q?=*b);{m!%dZ7>qPu^e4;pS*psh2sW&)$>cy|?VOM6R!p?xc{?UV zalRL$hoq*)sn5XQcH-o$G-E}RQoJLlf`QQ?F#!PqK>-1smDj#WNziq240dzxYL1Rk zg=b|)>+Y?R9u(wqI>y*>uprk8a;G4V3-aEV6(T$1Q&S-W!VRVsjno_Jb~WHrN1A9n z)8O>Dt*Yf-*&)^DNuE4blY^cQ9TJ*2Wpt>Bb%agi1 zugm4WES4?3RPbz~iBXeGj1{c3q<;&}3EsvM1*2mA6cq6kZhF`z7q%CfR7pItdso|zu3 z_uz{s^SyZQ#UXE=`w(A|`<3Zeh2$!nuEL?JJgLgLYE1QGWOcsQO3w9qzMYsunmhnD?BWMOhi2o4xHw zNIA1Az1CzQitqU6>m>S9P zNQOqUH=0wiEbPhVKY40t!Z5+ank9NAs=2$t*FP$*%cSbLeR*iZYa0e-;$1dg=4YuB zQ`|{#=Tif=H05zqwudpk3kNJ+gM^Jg6TH=1I;gixQ5F_sepwFJXL4iqHDy>crnX^B z7nXP7xurRaK7=yALCoWeHuNTcpjunpF38|6X=?uL9LvB5J7zgC$gBxin9|s<#F=~P zd6J&lnV6oL8Ce*XgVA>EvtvgAJ{4tTapHgGPzlbLV!s==JxTCotRKf}vbqla>hrZB zSDMi8SC+IQ-psa8_I2WL1eYVZ6~(?7F7#%`5TQDPw}PeePEUv(xGZ+yjHOgaFnyZ* z{~0Ehhz(dTbmoSoyNzu5(L!53%R87a)McOE)g`|&m!$w&K&8Jthtu&a7bgpFs3@1p zu+@uk9e5qewqErAlh1v4(%(d~aID+Ht!B(fD?`GH3x38C3o9=9Y5H3g%k9WlD<)Vo z+nTM`?9ajJ91PCIfjnHw!;*aLcjmM+%UqaMjJc(G@5lOj4EASHQ?3Q_B9OsBY>DE5 z5XB8c#X^duJS9YMsI1PcWofNQ7M40Qr4846YwA-yaAI(FmgUhtO_@fVm$BKI>cDJ~ zj>}_BY}W@}EBRkm?%QxD6E8>Uyh?onbuQHd4K^$7Wgced;7uF$#xh*q-{Bd$7p(S1 zZ|gl)>|?qbRGw$yYgJbHv$%sE6b@C_yIXTwztjB6^E4|icQ7_rHYDcerq=nwJ}fFy zof=`yosp@&d7wem_fuz$G#ZTG(ofwCazStO)Gkq;-C46+?|DEcbGUk<>BBt6)WM4@ z=@=^(+nOmqGcJNv5p0QITO>OonHa-+`PYX#qAf;U@sO9J4egN}U!P zrVQ*?;$9x++A_W<6N_=7Dz`duDV(7ZY>HxTSGIO#dsm*uvN4V~@#2t)HrgsUs1NnC z4-hvZNFC+A*g?B2G%C12 zR!(JQupdhr@wqjF+c2{YE8B28lnCMYU0M!9I#T-04dTlE~*EF3FBd>3YDr@hK#dQ(0gXXjQvp5XJ#2m_0;ycmy*v7!XmO7Wuv3FvMQge zvZWedt1+ww!)r0P4!i4dOZ0es7B}R3Lk>2gzdz6YS>KdxO*z<(b_YIm;BZGyhp{G-1-)1%_wi!5iKAj^-;FVG zbDW9uV(zaAPFcDu$@ibLH0KkOpC|_H#avxx)x9^W@WjNc^vue{>fGGU&1E~T+p)S5 z`|EP56Bol6)`j=sEQ}yNno+$NEWB~BWr&{MLBP^wMoox+SOP3UwioFqF=`T7lzAOU zkf$0vOUS8Ut6;xiyL`kWON=JQOd)JpShBe z;hDLUi(#^t^Yg-yuP!8&VTmiVTCr@oiBH-=K@p<@)ho+&o~nEeVp8=)GE^4Z8Whmv%dms<+a#y#g@bZymnz< zQI5IssyzK`GpY@b!g7w2hN@Qogn3vXpZB|y7=0|UCjL;9V)CDzHiu1^e ztNx5^!n$y-rJU5es2AUgrJlxNpTS%z$rD%RyK}NU?>v~}$u3Ved$ZVw%|4v+;b;v$ z*I;&C1~+DwKbx8`peZ}sbEiG;I&j>~=n&3?a5ID_AtZ%!r8ggYv&_pIkJ2PpKoajdrS_Q}m-8-~N4<~;7pr+KLyTJ_R>*ksKkagDO^y$X}7 z@vR1{>(Q?P3mUNN7X}0~DwtEj+zDZ{JpOpWW<9Z*9_rE?YaVMQ*F~Ko(ww%tW1t=@ ziBsOECyALknTM~XNp@qc7YC~|qXw56a66pwUAfYgi@o{On~g$cRwkJEsyWq8mg$*2 zjiwRi)4fOoKWSD0V}Z!RXcv|?XL75wVw-ba)AG0$R!(ahX5vy&uB6ok(V5{{nP$&I zdzQ83TnE;PQcsqJ8Mwm4$^VH1s>#Ufe)?O9QZ9`P=N-7{$gU!ssL1^4eENkwt+*b= z1^K4ii}ZXdgPQok!CVg3+$V!U*IH90+a6}MQ*zNx`>q~}b4XJSg<1@(&Dc6TtHaQ`oUF^My3DE1!{$tE$+llv--cmr zx!jgZ?YPi^mmS&g8?!=KD;g}6r(NmSgW)~7+n0SpHuj1NJ0~jaq^R?6g59FVJ_@!7 z3E6DvrYt0A(s&aq1Zynaheh2_5K^>J$jBNYAfM(6VbX_lI{zO2ufDNvsy-YUiDs~# z!l@ebR)F}d3@J$e8XRiK`pG7)3MPm{v1xH?>Sp&aB`?m!q#WGH!IU0s_=9mhjXh{9 z24&Ouges%ditWX;&rscPCK}L|x7CdU($=g<(w|)A1^mo=@#;$QvN@ldGp+|md+?|S zpMGclAFTX?mqN5&B&Hs88$_z(H5VCq6>5EO*Iskmdqf&es?`pRY|YAp`m?MSnkqIg z9f|2!pOIrW^vlAgER4xYLRQ{pWqvj`XJcSqw&i7_Enl4&T#{WSnOurhrFc_{DW$nt znki*i@6Lf0$KIx5u1 zw~KmU=3PBE!!{~f#pb+Ol#T7#*q@y_#ra&EF(p}BmcyQWuFk#=BzEDlB_bx9>bc<7 zViP-4hexS2rW5<^IAX_HI|jP2;7=}#%k*lhiDT1JPbk%gPLJC$PfMXvR|Q(}OivqB zo5cI&N&J(fKlLc2`apK87|sxFnPLOf2Tm`w^p%3YM~40$Q$Ds!=Ref5cauaFot&)8 zu_|>&zPT0$_2H_O5I7F*jc5?Fe4UKb)?%}519Y0d2f|El)U_f|4Pj!mstqPjw z6BJ~a*fb~?6LV>X&7ghsN;}r+K@fF~(OO3>&=%d(z)@T;KJ5;{{?U3&Q}x=cd9f;u zU1-G=EyKl5lwePl!nJ1(T!I;PY_KEIot8vdLWJG81_QJ)5No%b}$VjF3Y zubvR33#szl-iDpEI9f~FPk$=6-AvI}vo(=-Bzbl6cI8*z)J8ZKNM3!S|B0c=J5LNy zUb;d%i?vU=aHI0F2e~zG#OR~$ytMd+~0KV5Y_R-z~pU)cQ-NR!i?o& zIz@;69#{5v&**NRL1n|r_cZ?=@{?Qnzw`{N;oqoHt;T;{)68TiuSOMoyo@f)Bkt;{%fRsYIus}=ki@Jm_o%X4JJ-_dY25nbcd!NUw_BIXhq8yyxHXGs>0QGb61Qj&lj9W9x!hS#On zcBq&yRS-D?8MMy|jf?*4@!H=%D}|moI&?CJcQv}~C+)KEfWHQf{(hOQCSvLZNw2M& zJYuvtNF~zx*A;N;ZjS!9-ZFpx!j$*wV*hX4>hD)+B3&S+t2yZ35{mzRrIq%au>s*- zjCM*+^66naqYulkeOO$~Uw6pgFUz6bouz*G-!hs1ex+#45xLT5^m6rZ_v+|cEGVY? zPi4F#yyLp~mW}G^+pX&#Wy?f{1ayk-{*&uZ{>|#u=~%m7!&)P4$uRDxl-o|Ht-nKV zcUL#hGOlj&@25tsJJ$4TQR{!!sc8OKSJw(D|H`_$y7v706;8R`%a?I;FXQg^Q$v5h zn*aIt#JYJ`aP_I+=2O{LkMO@pki+|M+-#y8HMB2YUquxQ2KH1qS*AxVoD?T|M1@YE;wnFV~j|^!D&} z4>G%YxOtTicJp!d@eOhH4D44cupA;bZ`r7kU$fT#qn{kI#+rYR z{kNZ6X@8>psuF*_HAh>Eg81F-r&=}Z)av+4t!AP|I@WIH-{^lXan4!(-+tS||9^gG zpS`TxUnS))8Y+w1fBZN_(GF(j{vRS>qELC22S0m9V{$Gej zfn07rJ}Sv5n9J8yDHu+EUS1(0u^yhmA|WB30U-fyz9At&Zb9xoo^BqlLBXzW=71nK z%SZS%^Q+U$uSwmGHR{%?(Li26N7?%wwRP4S_!JIu_w;si^RT2Zy7~s2ea*f><-I*U z0s}(aM0=S3UC5mLyxn}vAwJ&a-Ga>?zUClvK#03XklD?{*W1lC*ej%bh%8&M2%7PC z>w7;iD)^885SDb>7FP*{pXh<$fFMt|pz2In9xw;bAeI{y}7%%7l1_1O|Kfy88HpxcdZ{1AW~CL%hA*JiX2Gcy6wq zULj`p5Z@rz|Ij8**+Y9Oc8yXuxa5$VwOnuc-M{~;z(1>vf=abfP^C5ss?>&4)|e={ zufIyC{`0>{ttK7LzsI^qh-r~^_jYx6^Y(O;{h$z)@;SxiR*u*DAWugLm@XG zPtTs=WxYJ&V&jBe_v{%N>hsUK!_LC0y~?I&QX!ZY7-WjAI3sZN;5eai#YD2M|%Hr1R5{ zgtX`(-h1=2iI_^`MnYCM@2T$1nY3(mEm1vz)x`riWOc<;U16(h-R%ymi&ctGp`74-QPZ6O;NqNs$Rti~-=kcX+CC@PQ92>eJzyC68;5>#)P2M` zp(cDJ>E-(Ts-)AvpO8fr`+P_SRmBvjVun=#nB!D|fF)G1TkL@<9$Ny&Yh2B^#rUuPB=s_b}vL%Ay-h0k@u>H>q zfnXFuA^bm~f2IGP^+(lI|5Wk3w%dP9S(KvXe{47QCX;_m&Xp`lQ~#6E6opgm|Dikd zPh$S}zy25TU;q8jvtS~&NT2_=sJid$pZ#iX{bp7VQf1!}+25>w*bl$nC3x6>tNovU z`=9@2bH4r0zdri^zx_o;*cwN1&;cp}1|^wZn>aM3Wf{F-l91sgv2ynxB7O&@;=+zW z^Skv$+CJ8h^@YlbbiF<9J-+F%V}R5LlEMoDx|bqhoLQE&l7pnN;D@_l5a2svdkYm3 z2>xT<(ebA7Ntk+5LdG0@Eh`S^X&B{T&3d<|D77>2Ln?P4sB>IwsfOlju0m0%Qd3Jh zEh{%;MBJ4^{t?QwHb4RZWSU-s@8g9R4W8dXtojmMnEA0FzffX3?!cHIn; zH&#Y(Di?KH1bRj)(D%10$3^U%GKT{=YuTX)rkhk3zW}Y@_XBqW@qKN2i+1L~TXyNv zx09KY$X`_|R5Z9F#RdCrUu?YHX0OZjRh zVm2G4Kq8K1JJ<#17hKPlqnxy7p7z3;A$>n(YNNX=!eUqsCdAvo{z^$dc2ApN)u8!b z6T@F_Ev(5sN%WD&dj@+eu~FZjjs?8rEblb-z#OGWcnxtfy_ej0q#h<4FZQ6lCi6F1cwOr*VeE+D7eDyfriU9#CcJ7dir@ zX$q;<7E4;%N@hp`3zxRnF@J-RZxT69-<_cmKywyw28Uj8Rdp{URO>;jjOn4xuxrzG#5D*xM|?L;D6*5EX% zZy4M1ZQLd*Pv057gU+UZ-_;WNo6i@6xWs<{f=5kDB39;qDSz2T09gVyaw=qZX8+Co ziD^mdJU^^Z>6DrNK^S`1_!X<_;p}h^aF<9IIQ0Q2+3+3gXZ05XQIGlJzv>3-PjX?>G z*kj)OVL;FbU>^vpWc25+ha|6G!4=I_dJ>O|J1-C;HMS!GZ5V#$r2eQZVPf?&L2JL3 zTl~r$p|ADAR8kF%oX%&HWKup`yKn^8tvKe3#mSI)fGCb0x+BIO&P~G_P*Da2nw)nk zt=w?d)O{Wa;zG1tHa72^q&M?=iQ7lba_2hAhP0{5(5c>E970Nv-du$ z2B_o9E|rli1!o=Rsn~Ov^qKdx4i25s9kKxV_ylb595>dARpNWTfFu-@yGYeRyBj@T zbDgn}Z@;h!QZipn+3QUYx^H6VS`wei+lD(Wje317tP*Vg-DSVb5?RE(zwBHC-jNZ1tMoxfF^GGjsdULt1g9*sKT8tUvX8 zTj)2Tx_EtAwbDR*o{8Ul>^@kyL|P{-CAwpsKoc2?0u+ak`IO~AjFCAhugG8ZIE{sKW7OWDsipa#q)wYeXkgb|6ggdR`T!7tQ_vW8uySZW~-Ucd3O zmQA9nH@Z4TZthUe8Xr1RT4O$cg(O+^^UD%iKDka+l$wHjEP+; ztu-xIHmP~2>B0b%-F3Y}uy<*v{Vg$>xqiKGp22pt~ z#X?ptm+;(GDNCuxOGXLp%puY@vSb#r4FlIT(vidL_4_I(0u74@P>)NQlR^Q$nJIFE zweR_s08L=<<-=P)0ZVj7xSWmZ6$=$1>AK2px1PDXhj1`SFN0rpiY=X^bi)-`!WctY zKfEVHzo~$_L^;aocw^x!wF3u#Q$lc0pGuNa4*WTs?AO|&fo~-vKHjP}=1UA|au8xj zR8J@RLhm2c!~Hu5b$HpMM$N%SmA#6miU744QWxa{=u(yrAwZ0q$5gJx#GnFIwhR+q zgn@kQZ-Kd!2?t;Vv4nM~j$(k473s{+SC{T1v*Y^KvqpxjT64mW&sF-zSY{V-I|WbNTx;WV!#Q3F%}`SMVmELMi)fEktgLTiUMS#3iotDZ~j>6vm0m zPFt~!i*Y?Sg>9NiP}sNj8wfn}QFSQ2RFmHFM_P;VmET!oSpHk65eb)89bGBC1a+7q z$;;`wJJ{cR{st&z1Prw>n8RoaIS{#@3}vX6IBp1}S*plvy0kAp(7mMuyYxrfK`7Mr zw=Gyk&6g{S{NC_cuYc0TtX$n8$d$%E0f3JPe!&;(topwQbx+)`onZOh11>QL+tF`Q9 zanzZdpVl||l>b`K6pO1c~h-vaRx&?pwB5DERrUB>iI;Mk@P4OH) zBz#Fwke52Ww`saZ)z#-He2Y);^Ttp6ZpE}SG5?MLw8NzG5jQEM<-En@cw1MjLlNvT zMlk5=r3fNIt?O%e`Odctd5(_)f}ac!=kwE6dw@HM1r^-%{L2LV6>VG_7o zME@&>oty-Q%sFvzg&E)S+Pv&8-z#%EyvS_OA?0q3Vw$z8J~3Hp<*KQgcdF~s% zV^4PzP5U!5B?;ezc}ICcEm}5^=PAFu)D@PNpMWxx1a+CFJmFd#7178E2@*9qxnUE# zut8PuFBc09?>JO^?)bQCyolXjqMKO~vuuTyg`sc0h|J*Lw-KD3Y1 z`{$mz`;ohamqdI49+lxo`{5w8W~budx)Rlc;-i{FWCv1j1+;y|&T+j)rRi6~3e1(< z0rbOsBM2S!M27XitPF4I3~3*_M0afoO-M5$>6zEQwT#gC>I<7j`bcy-`f*j&{IWmm z*ZV{jn3>9>)>w~Y*G+2cPLu(36XPB9n6wyAe_?o^c-V|z?Wym(r78;Z4m4ICpzb=7 zH`0AxYU^z(hbr=7Aem{qkJQOM;q{X>9DU7X<8Ew$ALA@Dxi&kjNfu&hSn$TI5fw^r z{gjNGJ}F_5wlDFX5;&4uTr2?3ean!C=FqxOS;ENHcWtl3Wvu-w{oc*LluZ2|NR~Z$vK@-9 z2lhnZ8llZuIhU9y^WVV+h}%0r(6Q{GqA{8y&Fid_+wW{|%_{HNhZF%akWT<6>%2k) zBstEma?+bn;Sm-15?9f(P1;|Q8u2Lc&kFDgdq_d*Q&Yujqx^_?*oRv8HSNIphqoGR zq5+Z%0vUaFh}ORLB=R{{wH(l9zK!%qD8nR%Yp3-|(MN|YB&qMPL|;CSq$ZKr%aE}H z$ao%O$bOu%kb4xGi3LStHl7`UIOsh;X?%J8TW3w0pF#Q@4}R z1tebn+6wLQ>QRfQBwt@z<&cId+}%KW^R}If=~TpuI|#EA13mZO>0oi`2;_S-me^65 z33`2>)fr$FZ`x)jpGGIyhQ6v)llRXd2gs#25tAacR+V!*Zv4ES28}u;VBt7=9BcuG z60-UbQolmKlXaSzTEZ)>g>8jxxib z+roz|e>TUlhyiyPHq7c?@Iz*uF@KJ-P_^_=R5~mqGozW@;0W}ig|9 z_^}g~@{vR8(37gPaKFeq6Mg)C++Fc>f`2M$o}XzNlJE3u zVFaQkAj-g$Zu}{Cz@^qP;iY%N)a0pe8u8zIUM^fJB2+*c| zp57LqSr+?cdBOT}SbD|d{h1FU7kdg$dp>)mHzvgQj=S9Bk_I^~Ykzw=Q?Q!fCr?%9 zIFXlqr97I(Ek`cD64465(o20{wSsLe&s?*Z6X>6Icp>Ew0M0U%CYVhb?-bJlihK>d zb5TLiU#0umm!Lim>s(on87Vsy$o$M2hC6BiNcM4eDco-nT$j}gO4iKLf^*o1dEmO` zps;}DY(BKl20GLFo7Se$fNbQ(dz4~$CHu8%Xc*Hd z39^dSGBY&)I!qYxYQEc@C2pgQ3)<%`>t;(dIkAT#InBL~u{ zmVBwCkV!?F+y{$E(I_}ssa`39-?S9F$OLW+(C(MV$?gcnVyIS)8m}}jaj|E^nZ`t9 ziBfW(lIpb8hT@xZzp`0HCZ$LtHmn&KKftfmP`*i9P0XlZaD43SNERebA`A4vWUm_K zB!)sM&j1wyT9qm4%8!TX)4TeRs5M|}<}fb$r39M3&3XNZg*OPo%tKV{z!d^etzIRyp-BJYKdVCq1z~+~Aixr;@owZmZL}w|t zA_WM}BB*h{N=6#`3OJZ^d_9pR2;BEnyot`~_ZR>{=tO~s6xEPbHR=AIT}W*3O8G0X zz0sT{hQSU8MI}AVvrpZvhr&I_8yTM9!dzmZBcw^^2iYzJs109zF9d;wE2`0XBWK>j zAzg7&T=O#sraA$4;K0i7YtGQk4s%d>nJQOj^*4eEcxn8Uc zImr^qRbd&1`tq}{_=G1pHMU(@>6+?*C~ca8ZHIc`KOt9swNsV8>swe^PI@M%3*c}) zve^pkha3taxaC=ST!CU^e9{Ucq8t>B%38o&Vi38HUAF0Ce1H;xsr<@N0JKfpBxWB` zLAW+lE5N@(BAh^diZ-a1KPJ9YByG3VoiqydI_OKhcbX(V%q{QXs26XPvEBklytwiU zlVh=5o%k7di%`{Oz}PCJudGr1n=W>U;j2QF()Stk{L*&#H@r77Sd=KsKbM_EDo1{_ zNCfp);d{K%;d&x0a;CRm)W|^TRH%)ySv3X#czGZeQVj9F9YPZtoCNIXBgsEmW4cSl z==yNP$Yk&o_g<>d*%2+7Nr=638|s(YVv$*UfD0GdcSL=`QqRCS*gp(eY~HsbhOmd- zEevMj_lyEhVZG|Ug>2RA*I|LMG)){;Ywe?H|C1V5{}Do zC;(6RNcuD`E|~3;I{mxG5834}O}Foc@)i$Y2Ej3`x=gY69?b9lQ18C4TR(*#t8+(e zHU-AFMe?LRH}YK#O(8ONe`JJ7bAt(B_Dsav>HGpIm-sAj^7_P1j*u#l#6l-fizmse z!4}Y~RE@|?XJcZ5RxD1PM;k$kzO@(|bqt#DlhoqFRBucst>47~w!g|r!JSxEQQXG3ehf!lc@aS(XS6|#1ahn;A76YoEP7Vn z?i=?#&GRpY&o>_f5=Fg*7vR}q_OhR7Q5SPY|m0@=R1s#z~&MhKcqKMa3W3T7+bQNuW z4Xsst*{IDOivCuYlZI?Y<1i&GHP?+)YmXjPsdsNn+K6H zl|KVhqsJq0XD-YZa=D(mGjJx6t9|pO>h)Y8i7W)$*txPj!wS76=D536Ip8%f9BsJ} z$1r*q$C8h)m^*YPNJ<>?T5^kA%O?<8FVY`A)8ANB6dLu5Wt+!R5U_R{B00PbtA!Isu3U#3!Da#VDnH&SH-zx&mJCYa zJ2xfR_P8PcA|-K2&y7Oe2t}wmkAU;dG=1yDP3HsGuOT`n7@eT_DyU9H#U3-#GDt0y za1Au)Lnh-l&D~2uTl(wM@y7PKiJuyCdP#Hi&{SB*Oo&&Iyoz_FpBtjuwM&0d4bNUk z-rsZD9;@g*{Aon$CBgAnn2mB#FCh$2h}bK$+YvyDmO~9xMD*;qR=$|Ixj0RM!_YWI z8jU4zSjq%`m&{6vDLSxvS^Zt0+B#tRG09&!IVFp~lcbd>X0_La9j_LXJf)&pV>e~= z23eHJau%nSC$znhH|%ZHI(uN<>|lKXw1^O4;FmCBzf8+VFm{vePi^|-VZS-~vAw4e z#Do{$IE?eYzurJyMwZJ*(?vrmjn{PKDY9JiDba9nq?o-Bioegx{J>uJm3KmHljq9q z;r{s)41wAMR203W68Y@BWhAuFb1)V;#|`8RPKFL$Dz6_tN14x)Ce&~A#@N@U=*uId zbPz_a>m8$K!M)65n78&@>gUtPLliD#b@r=Y0Wd|sgu^oYvv$7Y<#mEEVWX4V7*Nt! zzuH8lfqgv`iS-D#=qjkO5=5+GT%?JU1>=3xKzM1?^!L66305CV8ir}zGq`z*z3iGR zBS_S&c}l!84E2jkZZD4BR|1s}VL3d;4h-$?AMTgq(>#dJ`o1G1>Oj`k@|!{DYnw|N zXP1o3)KaTpHM6+$crlK-{7A#fT3m+;+0}(XsdA;=?4QSu*h(+X2fg7_8cV0@YEE0N z4#%$X_JY>#vwPPe=(m*FI=R~`9N&E`ELhAj838SWO%cchy?^bX0}&QhouIG3@QJ+8 zwU^(|F0-vT$OUQbWMCRH_CRdN1{-)TY4{?YJYKSayIh25AzN+j`)7}izPj6NxxQuv zb4nmZ9-Y1qfi?{~do&|E#d>O+OlB}_1Ld9%bCVu(wJMSr zlofuh=5l_(rf8WBG^reDd~8R`Y76^Z|K=yUzuA+GC0|qmuhL#O%#2b}s?nQKYRpqR zw2Xu4$KULagW*O;bUp+wU$e)}p^C=G$xzVqVynaT5%=1h2qR48o#t?De_cjshq&s* z>i49G(FpYDeLisaRz>>onzP4-yVjKJTPbT}les7|QNEl}wP5-k!w+LnR&P#E)r;+w zn_$W}Kq~5z!mFbm$wAWDR%X&C<5gz_0$J!R3=R4qNU9+CHbIb*K@j{tPvLnjf)tQ~ zAC+~k*9TXgEMv!Rt$nKR)!+#{r#8QUk_ojY-`eI1g)>j~8uVUe;>lz%sGNoSVYKbu zos|F#FOe|raIyS_i!e-u?za;0+?>qz%z5VNcpZO&)aT0>Cw^PkE|e9-N4&z>RLz>* z%6;RCz_e4PkKKWnWo&qJ>R-)hm^)+3+VR2R{gOZV)?V`&KF(xy-5A6LDmamn%kB#K zD(UZM*L~nG49H_1n_%??ryk9AxSQbZwNU;uD4PC2R=-mN0Jv`79aWtosPZG0Gt@I> z*fU5#K@z%2q^RI)bEg zaXxqxU|roeWDXcsV6(v3SV4O9*E`{A4CrI_Ab`_7KFrBx{6@PtLirkADkBkJ)mDb3BlwGd=c0ft6w%am-LeM8}l}3}|vO8{cK5 zza{m0BB7Jgna5Q$!RS}J(9;=`VWWA5392|u_|5O=nu!HCNAsv}X={=ROj&E`a9;jk z=~A{;--OL4%2zeUl2=T~EYLS}ck3GRE5Te^grwPzfv-*=c(e*D*EVQmhhz=6zp;X` zfkCCFs6yd5*j_XaB*47^tP#y@UysTzg>eKwSA#ZJyt0`$TskXFqXfR$x z^2ep4s`+S2_{*HtFNntkAS^T>vwI-}D@;WGJGh1OO}Q;rC&#_LBDMq-B)=PPJW zo8;l)Dl`2UXD4^MK`@Geo8tzleO8!Q-UAV%z}9 zo<@X+P{gK!?z@gXX4+1KfW}CMf5#!>in)K4%#i2@c1ja}TjBlAtV@WyLF4-uHtF_5 zkbJsq9!oFx7c!%01Ulx3;U9CESTMFyu}r)&cRa}_2nB{Tw6e%3bq_Yc58q>hYcO|w zy04EUd@I9rxIEU)-#p1QqHN8KXeh_9%{xM}$R#o5escgQFXJR$x4mi^0C_9y;z!|)9xMJKub8ZQGnpw5K4}mRQ5Rk^s@90(gsIu!Tt!% zrF2TZ)k#^VM%Xxw!zM3G+a)`!*hefQ8ppvS-jtjUy@A7=Zxv$`#5p?leiTqB_Pgyi z#JAiyEUK~MYBcA3M$z)$+=2(JzCMv~hIiz4PQSvEE!sog9`szPG0(o~Ar;<)kNqw| zo{>AijmFUGk8bTvg*UhT0RgtFq0kd3UUtJ5gRdm@1q}5_IoF!QD}JjKjb7DeeE_i6 zSUj=|85kHSf5$_XT|m*)deiLECfFee`xWy*ja8M(Lnzx1u&Ey|CvRW8!;g4MV9jf7 zN-Zb)_WK^oKAhh!5ur5HcpoIY5@u?WhQF%RYRWl0{g3En9p!V zEH9{9Jwju6AW}Cru-acvkqc3TA|swq0Q`b{mx+Dxh@KFe_mqSi0nvTwaT30+cZe@C z>9hLSvxSGh44M@S!wA+GOh=ld3}_^k*2v$>-Ng7#CuxuFLTcMazv0iI^tB(`7`4NQ zFX+C;`RLQhiWh&ew0Y!Uoez7f~Q~CoJ$T*@n%`_N`$II2n z`QpQ!EVps~4stkJq^JGVw7xT8_e%WQP2A>%D#P z@fgHG?;;DWgTGAN{+WYn;L^i@Do>gc(hXsWxtb9RN{y>_%r=6+tO?sW?rVPUoA5>H zbgVTCd#+4CY^m;5rK}dG!pNPh;Hrpnk~?e6cg8#AftNc9&!1bo$$fr9SrYsT%e>j? zg^*cpiogt<1L?%cA#{2^f2%rHe^?lgT!3&NI=_YGx?8AfjRZP8K1@ndY?(eeE;Tt+ zW+3Z${%Efbp?^BFu@_ew^7SNKg8+)87G3Pd?$wl5W$eZ#b^HA zZDi|#INrm!@O&xI0coZuG8@vD{g#@Po*K#oEI#O`$UR!4O(5vew7E-W@kIST5cY;l zFtoyUZ~uUd$l`E0rAN(D94$~Y{Bv7q{k)KQbX6>siVzVBAXvUNX2h76pMc+r-1Paa zz+3gxt+U7XkaTru`p$8{x$>U7_aapJJ2~$HRH@cLUa8I(ccGBuSpv%`7x0eBp0#4) zTIM9U%9bJJ5Rm`sMS?PpPcrVE+zccZ!b2(%BgFA`3i&RcZG*j-C{+cHRPv7H5r8L; zB{;;;Qq%?PScea9dLJiENb4iWsUiNv=h`Gj^R~XEgK^@PrP~TO9~ha@^@X;fE8XKc zgI$v@hL-#$EBwt?IGi5pGfU6=WU1m%u8ccU+yDVm!!C(>77ww%q7Ejb&s;UZ&Lr5& zn$Ve!u}CfMv8SuAI&GAbwATWC3c^lIpBmEI>@dxJWK%a8^dyS73H{KWX;J-ciop`g`0u{`fjr4?zvhy6`~t zNHa$cJZ!lDauY2de_&9iCa9MO@hZZxD$bu5XC_F);-;a>I>nCiQ!V4xq@;>=LYYe+ zioH>0o3D_i*wNfV#R-V$Kv6e+f_D)=Cj1i_*Bd$GEyZ9+K0Ipdyx&(_)HB9vfAm(k z5XQP>3b=x0l736)+fEOl-;oGrSlv+VQt`tJA60qp%j`&JVE8+=-;S&mDTPjKLC#b4PeHvEVStL=I5csO<7`9|v50tL0ZGxYRM{OR15Z`sVN3zh5rN zl7*czHAHHPMKJr{{@QOh#?+7qeYTe5nc z^v30`Z0N48`20n#H{i5Gz4qn(30aAa#vOgUKk-#9S}RH5_Aip8NxStz_>-_5&QN1a zEPHcBJrQbkaf&4%zrN)3FY;Y}_wci$Nk@dRF~;(@i*!?lzaI`YQIeOvy@*;GDpFyABV1uu=3v2^!_0Q zRqqiKgcW>zQ*2B`fU3S}yKAYGfN`9rT}TdF=D(*|38ST}Ugr>CVe2z%7!~C%8z#!h z(I@)F&h_J+nf>ZNCl}BUN;#L_8zxNdE{bCeXgg$Bl#md$V^fv1Q2 zI<%OM#%?Lpw2A_XRQ`qa`|YZeX)J@)dNp%pJ{$ThjFP{$sI1S%x?K{sm19vF`+hlv zXNmgfE9$?YZjh8_Tv;kH#F|Lh6u5IjYQNj6w#IMY+O4m^<~@aNPu8|5FS;?s5amak z%j%Nqmq&ow-vRiNn;Ne^O<^~F)$dn!USybSDl;PCzrWwDJ0A6rz8BUzHS5VNI3&%1 zmvZ{G1Mod3c}KtCZY5SQ)pPJh`xFBZi|IS~Nf6>Ax_gP!kv3H;RgXyz=#>q4rI{4c z3MFPq%DKi6u%Pt5nKhXcy-bet=;_llAhb)wd0HZe2)a{5;Q4KV9LlZVoN$xYWL06# z1gz%>m{K|oA^?IVaFBegSrf~fY;NLZOE^#Xoa@7iTnCJrNWm9A1%+$3z1R`;IlV~? zEGbkS(V4u;mfK~pU2-WT5}BU3r-Gnt`#ME!>5J;#J({6wEPxa=HG6BL`5X{c2o33x zLh0f_ZR>XuNO# zDWN$gg5BjYM!?<&O>vDQxwvfaXAp6s#zZ3GLpv(`U=7p6`m&M0XQCf0ImBsF>6! zKTuxq@v(@J6!2qsYcTpX1AStDc8v2ZOU=Yd!`I=%q_3VKAz`|3-LAmLER~zjaFVm@ zLiE5&htvoTB4KfjUV8 zG&-;L41zyop6g2vmpXnizHi#+luog>R4^XEwp;UlhC5R>!t)@(D4_Hb-K!fNbeG`3Lyn?ov&O*+}P--W1C&TeqJI0*Z#--Kw0sF}h&s{V6QF!(degql+ zevFW>kCON%;OA$B9l%2t!qZpM5DrCSLzu3PL-o$t(|o<#$O2RRxqju|M0PcjeqRuO z|BVte?mFFmXPNj+_|7UNp?6_7jazu1ib{1uQf=>N>6EUr6zfa8fX4*G>q+j~v=Ndb z4nCXD^KKsr$+(EmvS*gqaRa3N2VP^`{89l2q^!w zErT*>{`7ij9-ElXf4?(dt%}L9NI<%v``scWpTHOy%I(5Q4t|2_NAm&p_I9pk3BcsR z;T5y%Lr2OH)QF0@h90D{NkM}u+k%{@o$_57jYp#@z!gMoh(czH`v^-aMD{KDkzUWJDa9^8ys{F3%d|!$nsgJl>I^t=^A=R4UCk2Aj zCFcW<&W|Je<8B9>5SEPBImihm1#LFc?iG0H^pfvvyAU^^-6qJ22eA-4^k#c7wP9f; zW+9M2IR%J(gnyEp>SujN;GUpnJ|MmvJC7!}A?*&5$jcewojTXuYqFcf^AY%EUg`XWOJwGcBG4MbbMAP*9x#}JuMOWBQNDB3 zR8d0C>MZh6H%gouge>(Mv&;WdcKuA9OYP{V(cwUtGqh}_r-^R6NX_$ zivn|ix{_XWCq;@^e{s(!OT*r?8~YKe7F+8b*8n>}#J@9trE|Y=yYlwB(qFFyWk~4M znx?*oeuzv|-{enem+i@1*!hxpUvEQ@myeCzdZX2#d3u6Z=f-ohr*iCf+b|t3Z zGBR0XeLC=}2=O+eg z>MoBgU6d+^hmN6FdSUs^wlPf=SIruu-Q%f8)^cwMO zrwX+KO4UUWrfvDusH!Jzs!lg|5J@pho z>>^5a7ft!mZ(erN!$FDw0vEs~we?*6{&<&Q2OldORV@@aq;w&gQ9g^6Midl=)k%|Q z`zWPeWPwXHqL?Kuv^tX3_(n&f)dI|DFq7jkg)Mve=b*rlO@X75<7 z8HJh|cldUUTg~v>z-f&3wi022w&G>pWVF#EuF5cz+2LsC=VD;&%#8?U3Ui^~sA0cJ zp;|BwJb|zqwjtKu-*Xljdh|N+s~ch+OVY=|UhMr9HbQJZTbgN>mNp($By?RZNXa>O ztFMt>_-eO!+#dI3-v9D3snjhZk#GMkv{(+lNy#d?Nxs*Y0?=6+iks19$|@yNWw0Je z6Mg`ttj$W*;SbaNkLr)KW6E={a=dCW4A#dilTe} z3JcRQON!~e!;WTp?|glT6h%rA8PB|T?>(5A%r8ib|F?A3$HxWnjm}MkQf7=8uf{}X zUZEg2E#%!(SEgNs1XQ6r><`!W9Wxw=OZ^r1Fn12%f*vBis$d0-Omd@c&n_u)&ta+?BV1Tz9KV>}tcW~`J^B0) zsx90oLTh`Yk0+nWAwAWi&d$cz(@_1b80T50R4N>t2G)_Wp>jok6`d@^oX$3;*S3ci z;qfh$xe~Rqii?j|vN>4fFLbox&W37bqhhFT*~uVTd{Fr_;GeQw3s|7w%d5L&#Sz=D zDiO9dHp|@m=b5UMJ3S5r*O^L(iYc#xrY;w~KRiEEi|&M7BC5T_oG<{npDm}`NKpt5 zKV4`u2mkGoN2xTem3^p{=6~RYL6+h)eZlDG`LemAbl1L0oGF0IwLoC}7D;H(=*}j; zmPiK!+d=^n2nPA?*!cozk$!j86v)z>*`jziML>DaT-;Qzjd|OL^9If@%!CH49+2k_ zy}6(wPN6d%%B)NG`^C7|cS7QMmh}*JB)yNh_8HdgK+zZZ3T%P3aQrf7#pQHi!W$R0 zjU9YjdY$o~jpXt{W`?#VP8nuS3!iBCETX4aUVh7=cZ3Zs8W-eMzauHDJJbD3{t3fd!rBFbsNY7 zfE!8V;TuH05lWosOda!-Ti~+76wsCWS``F+hseO1r{H?wBZ1R}YCBEiz+Wbqyz)Er0uQoD`^DTu zO~psijoK*Z2g+otlf>q&_90gSt5P8nL2#>svy#o8>WhBp<0d%@hqEs)>W%|DI5f9_ z6WqcaI4q%SM*J{+?+;>G z8D859bqvcbD(-3&cTT;rQH+0q=Jm%y7xd?6?}4TnGt0sxz$TM(Y(cE&`ZPX$u}6Ru zysq|IjRX$8x}CxRA4(+`&6@uPDugbqSWFV8^uOD0Mdv-=nGu@s6FXJsCazrUeqRT> z1>|D`K*HHC*|i>!qwrppblskAANJ*Bn|hzp%T(;E_@lTaD%lO{FUCNuf>~@F8^(Dr zDBd}J@{YITkt#iQHC}>mj)bZT_}6hY=^z>vbG47;B{Fu9ju(B%yv5%U(6;Q$1?gX4Jvp?tBhU z9T>_V=qW$J5E`0V3o&Wb+v8Gb%c6gk7AUV}c#L}+u^1Y`^E<0QYQRvU6Q+a#oq@Z^VA?ro-H`!+`JDtipC4&~{jtnc<6xOe~N18NF9&Wlv zY=3=M*_3X~l!FYz@tg z;V@QnUC`v)yNxR&-?cu&bwa)v1DbH>>^RJCRx*em^}mx0w0w6svdCX{pVXGA^HVwe zm>Z%%p-H4L4qy-vn+P<1&ro^$nIhWF`a&*+q~{YLcTn9swZa%S2aM_sLaByQu=rf$ z_r~&&0c9Pcoei33Iukvj7~XF&sGXn2fovUA#N(l_ANixd@8pWD;UET|`zbh}O#!<% zNCKj{#3+6z@J47o2@)QjT{7ISXs6G04&E1+pzAv(f{-VpOi4@2zjwY=A+9{fRGnS{ zWfngT6e#^PO_+WX`}bQTg4PcxR|+IA?h5ovrOT^+`k=(js9bAEqaY!5BsEDX6)oGe zT`U=i*=sUW!OWld{Ds&WqanTee9onnwW&Na*LM({#0%w|B#y8y*RX$^I`S1$~fnK`1^LlTPIR^&?s@P1D;gVbUkbiB1 zt#Raj#87_GrdLt&&l2i&4H53wwX!X1U2Au0>A7*gb(Uz`tW;7mz&?@~dnawXN1F8h z@O7r|OD`zIB~M+y8Rw*NFJjNbc+*|V7R-d9_;QLPSt@kMdY0IBs-TG&bt?BWVlf;g z60E?Ck9YM4L+PtJmQU%b&Ae^hcazs9#GHI^#!G??#0cmk15T2^waCv&G|!+b982M= z9&)YgtvIt7&UYc0N8%(#FBn%`cbYnPZ`E@v9?sEbr5*6;Yn!p zQ0TaaA5Pjz_db0iT`8>onr^K9m`r*h?<ZH=`3xZgw87{a&2{$N_&GrA9DA zIFG6!B2t@FnR%GsIwaNaeN=$I8q2`XLz*k-C-2>QREeT4_!OSv51PnIfRzjUy5!wr zKI@X0EHq`~|7|bNiMw@gSNFF1(?ggV6abAt`=WoWG7;4z-V2U` z(B&&mGfHRio)6Jw#=bSLhx~eSC8PLIvzB1%4TTrIUC0Z<_9mPu`>{{c)Z8?HcCMD{ z#}}%f+X$(IzEZ%#ufiSMrG_j$lhv!2w_vR)TESdM6^2Cn9$~fww(5DnH~4B0yg_;i zx;SU`3t`DRnGM-M@`CuX^6ni?RiSNGAaz zA|}YZUy%-j-hSMqxbfvooJ_im$Z{9dO=64>Y`S(j+!3pFolpBXSn^5`E`KZUH*Gg% z)_W>?w<@42@YRlw3TBg`tj z#gSh>LoD<_|8%NqLNSl`wo2GrcU4x_8qn_`YXg0|U(D)`XWeG)rvO5aQ20f-JLJi? zF&8mmakQ1&`ir!fR9q=a*&BBQeqB&vq#$~c@j00qRP-!@h+Y3QmQZTsdi7tuVjE=5 z1ObPP2{QydZ-|ayuy@gc#vzrSkK{ zFnbjhwW%_FflMqRATm7zQ+fcCGk+n98t1AUGqEb95CnCWA!DQ%JS>5o;~KZ$$RF10 z9GK@P;png^ihvuONq&^Y)_4A5yR9nc0@LpL3UKrq1Dh+n{7F?2_7oC$CBpF8R{|#D zPttA)omP)M(~OJnVnPsjs5HPpCVQq$8j&G6GRHFypz-Sg=$yb8NU*xeNNBJjfF_dt z37+#NZ%XyiiscuJLBH)2F}sZ7ytyB!Ec=&)@d?5lH8^H$pS-|&^b8?$ck9$76s9Vp z7Tl+~?}~W8@I02IVx?vPhA4H5FN6QgvHC%BAc(EZKC{uWocNuiJnly@J@er4O(HEj zGiI;YuBISoajMtiOYx3k=rMukP;&q~oi?#RS7FjQE7q)&h*yT_Y{57rBKR9S9t27} z_S(b`6>147-}07ta~G`=__1!!a$fkorqE<*f3&buydP9J-m~S> zAo%}otM~Cqus>o?- zMPJ{#4KCs{BqLp)a#ALIlXTa;hu4SiXl~{=b~x_^`?L!ED~_qyfpAK_L7=9vPHlbO z6$SAf?U9XeV}u&w$bN6ZAs~?U!_1qzGg;)l^t|swK{}pEmiex?;k~ur)0FsKle7Uf z!tL5eY%H0lxfXBQg!@V7TBr17(-q$Z7Z|4#ho^RbF7I`ACJhqe=w!j;i*QDE|4-Dn zqee{LKv&VhqBZ#A_eByvm18J%b%1!a*4bj*L101zd=kNx<_0WeFVmdh;xvx=L`eLK z;HG}7o%@mUc6!5NlEosV*dbb4-SP>}L&E}|Cn(0*M0SfD+8)g8z{a=zw4VBckAD9@ zk!n_YTW*LYZM7~x!2E{>4!(*XtzyseCXaGw7?e<{0H;4oIBIU#Axzmg>r?BwRGR<6 z?{%^iKM?%*hzxa_EWMwlTI`_=G}JoIv`A#IQFT7=pqQ9A5Voj zM*J+QK_qZ5r@|$5FSW1w^=Ry^(WAvm=^$xK7ut=gqoXg1C|pY#Z%_CKjSubrPLGzl z*q7Sn34nEwFW%{*#M;ZlSeUrvvBp;E12d~VZ?Pf0dvKBJ)p_cF6L=iD`4DlJ3=`-i zbaenvq`Og=<1>mD6b;}I-j%4qFEN^8%ygLW^MCoZUBJwe<}RZrzuVrV9Q&p|u|{0m zR&X_zq}F~L-Bo!a2-Pbj&+WVf#Nh@zEdKh`lM-0)Z~oN-ocCfDx@b!RyQVQlCTAia zkQyo8+rF+Q>qU`YK#^8>apVZ;cpHV8?$s8422}mbuMvd|-MAaj9k|JMO`m<@L^!ic zLfdpo{WiHkdk&Fq8jhcuw`g$dtu;V}T(+GshRv270T;q!jY1xt?S=ubb7I!=he zg=(>~rjx9JyqBl5vV9<(<^+M1`>cXtl|5LY0;_B$A6w{G&aAbH7t-yJ?lnLlA-M8R z>R5#+7KD8T;&MwQ)QcSq5x@RF{~{>KM7fywqbZzqej)dTV?PL^MHgyz>AKQ&k4w*(-ScE zt8aa`OZjDev%KSDRB7XKL$-Cw^NXO%D8g-}Sju9MgyR?QlYTVocp+lF~@xl@=puI636kHeg zsnuqm@E_#T1$i)Qg`r4w?$j*IzGzz1->18a?NvY6?qc_%L& z-fevqpO=xwG>aITT1CFV*BWR6X&*%ymxtcsE;c^mLy+JBc*P3{cB(|o{O24-JB{V? z6IvP^!H(l*tLlP`NIiZ7BeN|DVZ4uR@&dzENiFC)KYq!v}z zZ6IH_oGv)u5anZS5RYexLosU7kIgAvl`S@ux+ANI#TrEApgX1Gh&~cBwhlHYryMW3 z4<*XL9q%+EM5KGreBw@ic(-Rrfc_Uh!K>UiJ z46!tVUkstqGY7f{Sx03y0`aP9bw_PE33t=T1Xp!I&#YxEIpj3zT<{Ibt?B#0f#X9s zF2-2190d&VN@Q3*f?RKaYu!q4_$_nsbpU?KX76V5JHPYJ%Ch1S zN4PxK78Lgv7zk1=Fu!8*kNb3+k9CV8aiam3-mJW$Pg4M2A7a{lC3|e%i9I2FgybwO zo!bo!u%5axcQrJ_kEAC~X+_i5z%buy^)7hl7B{mJ+x3iN{*~hTCY*X3J!#c$#h;Gs zGy_+nSdZZa+uC~qO7M%%B;3e*&ZO+E8ot|RcxpC&2&k?iZ-T_spDi$7rrtbs!8lxL$DZCBh2z8m*t|@=$j+np`)}YffZ~St|kh;DZb3otA4B zt&ZSt2#|zfsszxC3ousje73KOqtzEZT(o)M1+p|k-@R8m>aK|^Q?3j)nnsKLbidlQ z+(|X)utIiNc4UwvKDjx!(XC5=sT-g9r|f=<`r=0w1GvcmuWNycZ<@#exsv2S1TkLw zka3Qa{A~p~B(vy#bLq9M=UOg7yD9&>ZrGTUx`54cEzH(e$F1cpT7wft5Wsw3E(!R~ zGhLTDzY_p9TpWV@Q;HCyNo^e&(5SdA)nv5AQce9Bas8mh2d0@TvR;A;GYlbnv}qLw zZi3pg+Ac;pcOX~%tvfuI z`So>P#fe{mBgh|UBzI(snE#_X50g=mMXNpx>jI9&X0q%#wvmKT7AQ@3#VGfv>IdzS z^BN@sLHlC@UkkH5RP|QK=K~MSm|osau9nblJAEwG`~c9aT(sa>l50Fr=3v3SA4?S{ z)t%e1zc|V)F@?-bp!t`f<6cwQxtzZtpOhichXOkP-rVn3ZG(?(73r(xG{QW=dHXI|) zuW}&+u~tpQ%IQon*qh+08GE9x6BW0VuE4T_?7RaY`Vkr2Cdh^rSrU=t`0DIYK0F0~ z0hgVOl#cus&dV5pUGErRfxL^+fOovYtXCPEl!M{7i*>&jkF(@$#o!g2X)Ts+zPKMR zG3THMCF`Xv0WiQtMrgbEV4aQI`a>IBA!EQck4X|$_(#DU5HwG6%KgwW#^~pmG&Oz1 zdD`X!PL9meKHvSGU;cRZ8_CUC5aBRT(mS5EAcF3b6v|pb@aE4s3EZF5Mz;@uS?Y^m z3X;KVCCI;QC5%et+A0ILr|7*#KtR^jyHAr_J4QIwYHDheiBQ1Vo}9J*2NitJ z%T)oV17#6Zc_t(4U89;QB=^ow4NMeq_I4GWSJ)ZeS#1KB&b|sx`Tk~2SSuQKCr5g7 zJS4Eunsu@1G=;q>&u1S!+J|fLSlBpyP2j&N6KG##5$d{4Tna{WKd$&Ou&Gek8F5Ye=*Z zMH?Y3!L{`Rf=H{_9`_8bCtI#q2*v#>yp*r4{n9t53#K5pC<@xBqa1?r`+qLmm46rU z-lDgKe7IO!Ew5^~0Ed^r5)s=K*Hv&oMJ>!+XWp~-Qr0IM9J1D)66BQnaiy)$E{2FJ z`GCs>y>nO=XZ`7*hN^!h85}uX8zkv&x4O4OD*|Zde6!WKZTl<0?&$INo;bMLnxrDr zQ~g}cRw2;kN{A9shXW62x9SZlG1vT{Zi^)im-Jcgwq3C5;f6rTg6a?N<^7~^7I!{S zHOsBv1l!@RqyB1uye!-KHKN+l%_KkJB`rw|uX}H1v%3}S3Jxg^*JUviKGBUrLj`L+0NnzjDeT-$U4aEgrHzd1D=KBf~?af-e4> zkFen)c&$MXpq0cl`JlWl@8*v7w9~-Ge(7WeIwQSzkVFh5G!q%4t3afRZF~Eq?@sXWf$Q{Xe;AS9iO;jFiH4&TIISL_vhQSV6g?WON4y7TJ%Z%N&t_olI6ftHn6lF)iM*nL2J<$uVg<_Pr_O4MlBN4EKw-RM3$+rqcE zj-Q7lD}^=0%{2oism;BRctd#yEMY?}*r8L(L66}%7) z_cvgaQr?tkr`nInvBKFf0hQO-yh*KxVM9a&_5gb-hvJ3~5uu;fGryyU3|ASU8b<=;=Dd9POBN)ynYgOJInJbNg6 z>JdLDxiOriT}g+C?5iN(S4pQfLr(LIa8|soE&J1GEx}DO$nz1MTESeZ8o{Xbnfx;U z4bE(5sf%#`k9kI3lK=KY(%YprPW(E5s>v z`MoLzoavgcM)&B#eF_^}2DFrhtOAWJ%}+n0SK{_oOdzvYmC+wL*wmc17L@|xURP26 zM<_qWMJ5$fzfuvWN&2xR_5ZxV6&Zf1v&!yi{jxPzO7bg>=);##F88nM_T$|u)Ej0i zDz+z^QfJVYb<%PbvqN?giJpG$Ml-wj3;Ys+mCeiP+453)m;0zZ%NYL~9RU9$ya>w8 z#^T$PB;nvNf}qm$RqY$%OGPp!xO~$GjzD#D>-|8|q*7yS>Rq_*jd=O!j^X~_*Zzq! zcDKh1_=<5cH~T!da813ZnpQO3uet3AH8!+pRw7$tuTl4NMH)KIE01qgMO&Oa5R_h8 zQq~1GM)Z6zhOZwM#+7~|t3rny1m!k>BUnXGX_FyViCE$p{u+~VB!LH3x`fOd$EPwq z8X-XP7CBVo@m#xVIypaImt~0MFP6k`q_Hj)VGa8!&D&O7GW(P`HjIX}xgBqxz0K0f zT@Ai`%ykefh%pH(rC+*mIaA>8#OL+huA!VR2U-%YpDP=4kG3~NZv>kLGV#!j%hHvR z4pWsmC^!e1FM6JS7v^QTydDW-|boP7wHuMzpEeihi?L&vpx{x!4X3X==9ZE zx|*rMSTQtUN6-vomXu=_Rbb@`EPJlxgn{^*=_U9@{5yOaom1U(p&c_D6>;BPA>_mI zyTQ`Z7mKQGvm37tX*(*0SlutH-A z$3JJI0D}i$_5$U}xorCk@PSQY=C53$;DH&viZh|(&^Gg_8y&B{ zMf*y$Nk@_U%BXpjr6j3a^Yb9yh**v{W6NSnY2h8nV_ju)SLAEs*(x$$bO3MOubGq|pRd~2Jd6A|63V{vTj$Nbv^qC<|{W7@q-S`gf@` zIW^$Z?OK3@Z~Fe$?~&pXTpNw+5|}r<&+${m187$@O4R-0a z;c}Q(jY(TLPQICLLtQJiPPOfvqlOl>1#V+)$ifGJ*1w$w5nnC)+sb3E@pj2P~f_=;Uirvg-?D!Xi~z zCPoo{uy}zW#YL#YEvB}1S}4o5*J@qvAO5|;#v*nPWEnjmB^peEd%gg!fxo>iYX>5$ zuMfk>+rHpm3KXO3u6444-vt8bTfMBp@_x@n^wwbi20q3+QQYkpb!T9}QNk0jX&@aj z3E=){SiWyBMX5icQ&sN;5e;%$8RyFgU`o;GcO;CC9};D2!maIbD)|M-b*{BA`ox`7 zOF2N?uVdZA0Y9lX-^vT5kpd&+*fO!+n8Af|U2@`r0h@1N(ST|AneW_=COwxp{{arq z#ihF0rA7)aAAh(}>5i?m&7U=quS4SKTb#%y3&8&E!oo(MP%#ZO`FY4^%aSZ)cPzBU6ky`}IOY3IqwXK4WP0DTc#x z&^3I-j-!)FDgv81Uu_zXPa>UHs;nE2@BG;Mjw0hUkXMPS=-L;4ffXzQ!epU&Bu$ZG zIm`}18v-V9m^LM2cAwlg@zlYL8l@c8pSd;AqzNoiyHY0PGJD6bCQyTJWwsWp?z6#z zotk5Z!=}l8d!K4!<>hvC3*Vi{+ zyjIOL+~nm4Mv4#cL5dx{)!ZcZ8Ri~t1?PO5A&{>0APpl2Tu?00`G+H>5dm#*5yt@^ z@0|fQsDIU}1tkz368J6pB&?LbUYfaK6~?hpez^Ps3*gV z%oyGbDY0UWSSRlCYn?^F48XE%h#!r)s> zs!gh%T5u3Z_a$6Lg?l&1iBJ+fq&KYyMmvo^c`CKbbP%TZ#*CYG=|}mN={Ihx2=@OKe6Fnzh06F@Kx7zRgt`Vgf?Cq; zXjYqr^KPXx!}xDc+2QrQ*V=K0C{B5f%rEW{M*CG;c-}=~xxyP|;fkaWO4pY|ZM~Wg z&0}P0&u%-uM0oj&Xmq4;e(F0$Tf|CGwR^3hZ)Q$)lZ}3tEl<}zX89Wpn zP8;P@4C_{ZOy#1|RsiOQD@+VtMJU#_&4|6X#TXQ5#8SX=7k<@1G7 zVEvBKEUzt|TO0P8Y`4`z(r>tWKtOs)+W(#25f+NP1nAaRDa-T@W;Wi#d`<|wi-$7! zoLR31@Xd)lDjQ_j42yK_-pj=OcAsv{*7Q;SHA;t?>_FF7JD^OmHOY!H)s_5;UsmC4 zqN7FjT=lo&LDH+`H6iK%qvCi#j9-PXsjkM*A0LAoPu1F2TeOj+EAX!em_-jVzn}rp zA>@cWF`zJTM;!=O)s{XoZ()e$uB@FcC^83XA-)bZpe5<>dO;qop@^AK`@9icA~B3I zsFwNR47oZ03r4xQNk)kLkj(0HVj;8X%|_wEw&+)5i&CQ3m*_JNUt8W-bMh?kPclL; z-j3r7m~eW(H1|G$cag;+w>g(!bD}r7!UxCLu{#71MZcy5dPB#O=B~b^w~Dh2yUTS^ zhX-y5Qf38Oaj~oflU-3-NH^Yv>jK96SbBT@XF8w zLZGuHqe!`;>jWbKH zyiWa1WQ(*(jyCLMki~?F?>@ZW&!Pto*(3};y}*nkK2-3|4%4V~Td4w#_o_McCLPXr zWye_|y}xJ)UQd)XK9wM-B#KAUmrnLpUavJWpXl<2>-kV$aZ9Q|i9Tv1`-Hvz;IhyN z{Nb;{TUUEpVn~rSD1zi&=!Uf(Rp!)ETvlWSSKM+e=asrI+RDN>0A6oheLI4px4P)+ z#nOXcw=FyqrG*sZUfAS@tIy4G9b8h%SMz;?KOhm%OZ6d9mye}*D-cFbw`A>-e2j9XmPaMl zbs0ZXk-46-A+H;fyhP=YJSC~^z6OFF-h^YbcA!f6_uCmmhphB7lBR7jZtR#JXGD`F zI05zKwYQenlum%wg_EH2N)kTK-+tuP`)zIOs@>V{E~cAw z2bV)?Se2g<5N6_1DNDGIm71eOtGb&& z6QANTLT@N0$lIZ$-_@WxBD_hL#O-d93$-}A7Ca(0saI~m)z&Qh_viJa0<9-Q@|`I> z9_46@K!Btp)|VOcPC`){8#8-L0LXc>#RUDvh|GkUe3uF{Z)3Y7X^6@lAByhoKb5I$2api`!KCg|Ivj!cYEWO zwvMUSJZV3g)2p)?o`hf!irI-UdPUI}wu~Ndoh~S_BQx~X6>g3Kzo`LmMc`F6jkX!i zxqHC~LJ*}dC^^^4)am!>E`wtX{9(gFlz0>aI(+D`>|HtHXb_VQ=Gur}Va4~dg&EpB=Rm>x4tG@HQF(8*lxX)@1On@ec8#-uN zSiK&bN)Xf`5u&t;*6R7>4lh*4@)ASGl^IY}6Z8>BH#A}F41ney}_Q=t@l9AdvKkGRXpnT{jzzvC`F?518(V1p72PGQIufPQzsd4`D zE3{rVTj&gN#|y1FgV83v^Pc;6(Mw=e!kyirxjpBmh{G{FDm?FN6>Js2*rGu@CyUbr zHvk{ZI@G-vzLbGylcZ`-cZg!DC=LO3)06-ctXvXYDa~k8pkGZLF#&O5nIKH40=3e^ zKNba_0kB$+Nx3(X&$VAf5YKx8-+O{U9D1UghiN`+Jnm=zCMaazohSdOPn7;2(?1?e<{LU4b3?Q8eSa_-U$1 zolmo$B;Fz`>d9PMi&O70&47&>E`GdrX~U zjMta0DeQ_8p~f)KePO|MDOHCX#gbDEdo5vF!%r z6%!jVaD7612`#eT0%fmJ9QUz994)EqL28=8xa=S;j0)r{=$VR^Hat_V6yqp`bS^-sb1xLosc$+VhNc|2xuz7;B#>`C5|lwIc6@cCHU>dNNjnhANL;$GY|OTa@nN_4)o zzgAgMzB2f!Mo5@YdK#SeD=Z35Lt@EPNLN^Ejz@V$LTS+#npwhy58KVq$>1w}qr1~f42kZr#n^=Do5 z_U_Ne<+INlgi>5+Q{jtT9Z#p;(^)!DUF?^H}`yO9YM@!+< z7F|(q5i5xKV`&2eo?{on*LAAz11hO$p`2hf+uf+;nccLuYkn0=JJ*@fJ1xigKuPn( zzjFR}e>ud3ZLvM)9dS1E2@Xb2=IG#bTb|pV^x+?IBkY{Is4sDl4~_u@cbkA)lQ-GJ zQUL%N%{AU@+SZ$f@RoIVu^fj|tc$#|zd!sM?`O<_@c}WU_%$-fHFdW+t23OOMQE_e zmUK-+IgS(4kM8d-_FptbrvS~G1w&K5=5xQiB7M57N#-aJ?emRMx%AU46AMhBDMp0J~u_H52AMWGv%FAf^XmI|4c>XFjmjF){8Yq9aNx#&F?{{YodF`Sz znBK_YQbn>5u`G>jr|X@49uzu#&f7ldq%0roPbxPH^0x%V{ zGhV+RZL72z5&oOnpE}@z#v86~IPWLlejKjJJ{<7rq!-%|dNg=URf;oC#q(P+oN11R zS?B8kF2m{>vnm(L=6&f;H#})@OKc^?c?&73j_}jmSqq%Lm1N0ZIca@r!%uYl7Kky6 zEJ2{jcp?Sdw;j9xnnB{;3V&FHC6HUsgLzPQqwmM-_Y?cun221tDGrq1I-;cq{iJG)F9w;&sIL$?vI|#2KW3i> zdeQXWn8Bq|$wJ+ER2x4B^v-rh@8dIm!&55{l>(T&#uTUhUTt>5hn6@f3dFc);ai33 zj+;?F6@AVZ^+umY=mzPU;|AJjxbPnI8$olXX#m8zM-genEu|JI+v+YPvP7lhq>OS6 zXUeF-GQu|kDy4z-=@*Tx`+4FfRvEr2r1p_ZN&^((cjZYby?>+)A~9_OHNCm%Nsd@1 z>=IGhjE^Jo*$5|QHkL^XAkHAKzjKe=&+@T~9V<-kLvGRx7E+DUSpH}rXGsfn0H%FJ z732?oE*i)e#;;*)nD1I2z4EB}7gnL2M8FqJxAQt?sg@hZWBr2TQ-;6)ZdSpPJ&nog zjwV9|Ldll2wKU6_y;=yhr*RkYGZixo5Pu7)eh>QCUu=*ryHgpqJqgGt3`8D4DBn9U z@Ixy!U#P3h7Ql8G2UbJ?Ia z!{4JjnA7NQc4_y{sOpF7-r}_f!uxizPi`dw^`UwPGb4?`?&3XZsw2W?vc1$>9@0GM zA0U(j)LzQDXwKE<&y?LClVQJvIXq-OD30mSZ$>+0R`)Ll={1c>Nn{$6l^cMdD7X-O zu^{u7X`#+7gSG~nLG#}`Qg+ZALS<&Py+k148V~eJ)zqlm1kPmK@;Ggq2`C#yHTXHj zlL}%=((IRgNHNZhU!bau@Vq{GQEa$!grA9jL{1fS1**;C61xjIWpq(M2rRbiLc73^ zr`^Qf4ba(-axpx^yvp~QyeoZn29;MiL9oBomAGocqs&{X+>9?UMs=DvZqfCKQJ6*T zPr3q7a)%RzZL)0Pj z)A}sYCsT~0Ownq>)8xu_Fn0me2ZnFNvtHl&zRB?K#49Y-7rA6O!u&QcBs%Zzw;(V* zG4BP0u1*+op>Pxz5>RrBB}x(w`wfI4?m~}GsC#SKFR7s#)WZ6^`2x#lWx6jRWs%hN z8-+z829BaM~LHc&);0qt?7Z-o)#a|G+6mz+9wbv1&#il^dk}M zW|kmIm4^c*X6t}$)?O;o1SI!{=5rjteX3{diI~TbM%IQ%n1Z(I;9X%>3@@o=?6DD< zsJa|5*uTGVAPf~c*1RQf=i6k(lEt3>tKN;WQFQjbDk?uB7<{{#ScyH*X>mFErTo3m zS39z#5^cNBIQ++1B8V;F2rhSyT=FL>UMA<2P37h(wZ5~MLVIM9MWNNqglMvFZs|y^zIe14tjp8{S+KJ0z?qV4H;N zq>Q(DKP%EU6l6#7pDOo;ZCv>&vh2hP5L0~c%c)I|)-ITf%>E^^=jZEdvY$@&=tYt^ zta|$nEOKSwT|C`tQ!%u&A0NrJomT)Brq-1E)N^iRZK7Y1EzN0+1)$}9r11>d^5DO>lgRBdEZPx&OkO4o>7Fb>+ zZ<1t!BA+Qx1`I`flFV+F4>);4bULT}C9;YtcQ=uf`NxlXL=@3NxX!93XVQQFLV3!; zB;eG(d6gA5Bm=|8Btt`V72CXzJxY#XS-GcQ#i>N-fc_G#4csIXYJ5j zH{xXt~SZp8*KRxHm~Z9tw)(v~;79&)6`(c4EEyCCJ z(eahDHV6#ViBW9B91DxAvxWuVl72oUQWfvy#yxUpL^Ilk5?ISswhS;Ig`;B;_SxAr zmj@9+y<-db#WXrP-VDqXY47Lw=m1~;3Y{szsW}yH#kynht3`0JD3-WEbhAL`-3uR^ z;0wovjs8@;Xt%~sPMj}XzoGqLJkYlLEAQnE8_qiX#*jEdI(C9hI?FVZo*B>d-YAGd zxO2GEuBu2}#yH(Wb<33&qgAs9uMdZwpC|p+LHs?(WguqFV@+cu;fA6qF6GEC0okXm zZ@9R;@gc=7Az?f=Wj0s#Ci$}dc9@YO-Q&^MA4ANY46kN47(aN7o+c(hE~8mFx%@?W@}>YgK6I_AP)$ z(Yry0bmmSOv*FPjw!QnqjhwM4ejm|kLUPmy;rE#g4Ab#)1tJ-KEhthbrTy*SF5M=^ z??dFD2e6IKf@n{tTHwP89ilisi1ZZeTALUWNj&T^=n{Z|6tnZTR7by50OVeZ8O(ek z9HOo(Ne92FL&3YEoZbg=sN{Un@;-H0oi8*XjmLyb;eK6OvJqN`oLIl{_FEFLz|zbE zdOg(&l?a8vTpn9W#9UqRL&!etU!tCE%lSsa0sGKJh-(3kB;xy&#io72 zPy3iK_6^n7{$s*N$4=diw(v^2!!-$K`iKife6)j7SoD~{Jufi|A86!I;RoPVu2(8GK4t|!zq7C`Xxq%1ZczKXqVf{@nJ>Z{2f z?Rt*tp4n^bY#_pBBs`G~M#su*kElosU^l=4&|VV{k!CJOD2FIiGSR(k%KS16O)J0(LMZ zh@Bf)%ezW{yw=V`=$YO_wU+SC!FCW!VF{(=>fM1tXQ;A;qhZmS@vk18 zo8WTTI&)}t8|lTHr47@th!xoS>K zAMdtT9A?r@lml?l>x8-E*9BlBbXU3Ia_>uK{WoeV0H?w|nY=q7qMh=4cY1R~s_3QN zcgsw3K&D6BB4$Hi4Py21U51|Ly@&LbF=y`YwR~+%v3^>?>}QVu2Kd(l<-G@Cpi{IA zh)Cqj!WM^fW&3VeJB7I*`lCKz!8o?dM`tFm2wy@+Bnkfk9N#>PZMM-(TNEdu?3g^} z-@Skc3oYy1147t+ztZY2X0D88l8j#EX%7Y+PhWrHA;DJgHFIG*B4ZWB5~fpSU zP=jZJR!l7K{HjWvRm})u#VUi8*zBw zSRudE0}XpLWz&K{kX~4~GZaP;C&OX-QnF-S8!`CnK(9-|ZJe?%lHFmPjDxAyR7roD zpI}Re0(9Q9_EM!Y3j$1w{oaR2U*y(2CzS}1Ee~FJ?=O=6ZOz` zV0i)2Y9DPJO5iXZ*>5;M?%=z#~*Pwv;-B$qy-=X+?@5p9OrigD+jhsLVDh3H>K17Qf>;C=@ zB&3B|@I#8JI-g?09<<5$4GxyZ%)`ZtZ;q;=O>Nzn<%}1kW)<#hs7n_q5q10q4SW!r0L!0t*ndZ-+f6uK;buTWqr?)^2?)$o${ zMcOdCH1u7`hW`C|-y9B|@XH;d``np{mZkkJH&7%7ND@YF*drvcuXTM&<>lSW17>#G z*AhyhK>)NeP*`^DGD+A*sG7~|_e-F}F|!kwmKgPrvR7Sk`(8!M44Q}T8)EVBj|YSs z?Or(-wXq1kCprFTicGqSoJRP4Fv=t;>q$p4$0cQh%hi1UaF$L=p&GboWt$>u#JAfC z1&qcDn%rRb5;TpJoiY!<83g)7lk9#aAGLZeG3|*+B7VL8o-(sR;gUyscX&hQG(rj;-IeRkd{l}@ zl|7{(bu-i*DRlfgM_H#}$!R|X?zXMie1vm7aGk>Y@x>=RUeTuLww{&2%;Ar%pJr`i&7Lu^>1>aF3yk+}FUE3Va^R-_gh04OiW8%eB|VbIrDA);rb zQw_~2#7>&(-y&hQmcbmWJ34$=4(FQb&Q(z7tgZfEi+(({m2ZGoY=MqCrsI+ zdb!II^1q{nWb%*)Zw9iw<2cO~Z5!%f{mw30{Ix7Z{rD(aa=1OIcijt_1C#x(RWZ2N z!B;1|DCH;lw8Nf!O`}{Gk|z8`8KW8A;~{1X8DxTlce|9V zmqz~8ODFEKI~dxuEnwUX|}g^6;OJAKO>c29MCL_8#?qvFXcv4$E#g{9oa>AF54@lC5O^h>C6GL zGqIk^ao@{e>xeE$&^#IeLEeEu9XS#fF~Bop`Dsy6`eEZtqX_CHmc`pe11xphiT8J! zp{X(NzU8j8cB5VR;Gs>{O9HbO`!$|Ra*tpvqb!<&MpPy2!}MUnC~weTlNTKTjqsEz zyqDI@h!vz#nK2pmt*{Pb)&@T$#n1jE4MH$QwJTWsIdmoW#TcOYlKy!)4iA=>eS`*YNpMnZpys`Av<|cm zy;$AEFb(-?V_a^I`~v!MkNf(<_zI|dIZ{g`cs;Tt9>?e4*+oNEJA46Xf&H( zH7uwD701|OyZzXA7ZB;U%PSFkAI>3{ocPK!?3>Y5Li*E8b;<$`F=uMQJd$6YN!-oL zor53t$2!@$&#n7o{H*84Wq{7GvXR0q=|n%>e2;vI7v`2vloC8rI1U!QoV1Oq0%Eqq zdK5yCDj$gNPC&6g5;O?m+2z2E{z;u+(Xq`$3nb1QHuz;yDnp}(IyQu z1=Nr3)47^?e3-H7&;)hfSYew zlPh0L^R!n!y>u9$h*X(tuFLyHnS2q88+Q5VmTb@r7tqGfT$9JT^8&5G3rLq8wF8hy z2mJX-F~7mX{gTNoiM#rWl(R1{J9{#^NNdiD&N3B+{qzRgd>($G=b_)aWaLgxWPHQW zMUoL(t);a5f!l78;xgX2aSo5NP$;0c=;e?N82Zd^A;GEo)7~|w$ z9+o}SKN^Fz$s132fsS4lAc35Q6<8;5Lx;4nj$Y%6&)UAzq5`Pk-{jiEWoUrcrHr!)<#h0!a&6M9Ab*n7&Njzt%(>cPJJ1ht?2eO7 zi$IjOCLQb=2wSr3?^>fRWqhc}z{*7^R*^RM^ia7)bMZ4e>KF1^o__0@asaB$Ecb)T ztjFa-3a^Di8|5)c`$-u-u0YH+vtHao8|{;+H!>h0LOCs}PgO87tclT~fi+JIW zQBIe&$XzWUzp#oN8oTr|$1(KD_mCRet8O*}ZKt-?xjS&@JAMt7@2A>e<8un;hoXhr zV^^L3B1$EPoemiS4;9`;BiCE<#v@jIvKaPjohK$VkgkTk1!x;*A1yhxS5!`d{P1=Q zER<9_#pyqy0exPDbp*aieCw~YET;Ir1qx^7_xHWKQ#-dm1E&L4jS-6&LbO2EF^de> zA|UR))Yjj!wSEv|gYHe76kjwm0Ew1;?~q_NHC5-yt*;`Zz{nywz%*t{A+~~oz65(L z9V2!jT%aHd-~q}EokxAI7##%ahkQEtb;;N0;8ulQQ~ty&3q1 zgH5VoN~^SQIJxlqtra$lRF+TIN8jwp+&TtQT1gAQI@RvIg%S&{&PX({Pda^;7b6(XlKZxZH#>J{M&E6vDrGLeB!P1*G_`~tuK5GpC4Ut(d~A#g7dbyUh2 zC3lycSD&LX_PGY;{NB1Q1l1;1BX8>OQP8KOtJ45%7DJG@Da6)hyPh#ekhi+19z`xbn@LcTwdX136% zKJVC%et<}11dxZe_v^JQQD0FeMc!4xe2aZQwUYd-#tjW;=B#z-YgqP4)w&jFil!-T zD8C?%MPfv~D!*cRf5}GrtxV&|c&seZ%dMUJ?PLd+k$s28eQZl%ntxc}yNS}!Dk48g z+Dv&(BwXSLnAm0@!p5de?AjBx{+eBvaPJ0OwdVIC#rekH#1mHE#Q_EpnE#~kb2mqY z3X$`P^+lzWxA7S{Cgxlo^M8%O;R-qPvUD-bH0jGgrbsAfrSwXeMQ6be2{@o$-DU$P zH@bV?$Q5hFz`B^!5iA76fZVys{}@+_YC{Bxk(-?N8tjk(b8o7fEwG@I*Ib@X-35A; zT`aPieuroRtJYGVES>$#MkF1_dRc#4C!{0}+klyDCrvGOIi9~OPssoW?yS$O-hno{#4s9iOt} zq72zOe7?f+V(FfHbKxbZ)+>8@m`AmmQnp zslZ(TOjaIZ)w7VjE1UYafWoGJjuHT`=GSwgOLYPS1Go^68!g2sJ^x7F*47x9uM1O+ z*ciI?)9B$l4nFdXs-TDqj?h0ih0Z%0JxEB$*THRR1*%DhqiX%h(ZVwL zn6F4;i7N5^>u$?yy?0`Vv2eSw-VjL;#a&FnR{;)i5)fupwb8#QMD1vT@1^%YkG zu~%t|KVQ7byrj>%%H5-iZN~*jMncO;Jde}gN+u`Zo56v80%))HU=h6Pcy6#5b8>Uu zu=Mcjdr`W+>W%5zgIXvLWIFhSEIIK6kbl;=m=2=S~c4a|w zH@2A*Mq@d7Aymw6891vYOf^nmaQfmhO-pX`dWVdeuSg8B{FLb9?`hl=z2(C%?Q(6JcD;Wl&SKlTnwgfoIQw#T6S&K!B@|O1hkw@S%kaf zeH@7KH-FsEmG%uKSO@zCS-^j%P>-0I)+I$E681M}`R}yXqb|BcbCB^g`LX|d(X>=I zpb1A3h%?ht0F)WxXl3W38(H7`)A4h4_A*U=0?2n5Yq;|6c+!8a*YK1s7`udb^^P#( z?Z+?0$ zdUWANMmMYwOq(ipgER40o|Le*^ly_d!Rrfv_HzAg6gT;+J}ziTQ6dc5JB@0a=F=)A zf({TJ6hNu&*vVXSUuq1Eu^pih6L0 zw{7p@%vM1D`Rpg1eAhlmO7$cgx1!bI8Dvz0`X|FaV#t1lo{YLkKk|hfBB8Zsqj#rN z!~^#vDL-ln;1@C!kGT74SCb>=NsqvatsJL|l+%~;$ehif1%e~H$crG^)Sh?kmz2N- z$9qtI@Dl#>DKNx;C_RW+qGILGp})UFbDNU2RbS4XXg_DTm~s3%th7Sk{1I4Tlr1EI zvh4jb%0-P)^_m{qDtr&?Xs}MKe+*D zz3fhH?wl2=c$p3?T>+A$zUy^&UrlO9E}oadu3{pRbv9pcV9Rl_sL;-sP^-?GdN7#w zi*AY=Tl(QShY}Tt1nvGx0|3qU6kW+2^BqIj<$8hi#S18wdSC)X*YBdTwSQ9hWBup& zZJ|*9yxH&lvv}{ldYs65xcOetvw1b!p0yb)JPo6#M&n0k)LV#NQfE2z_k-lhX5-Yr zS`)m}82jd7YTOymc#qT1XM9iST!SLUM&Y{l6-4ay#Ca{4{Yg|ZD2g>mh?lD2#LtP1 z?<&|X+_nW#Rpk^i=)gLxy9Injpg*5vW^g}i;vKRnOx|B*%$WcfKJKsRx}KkveSW0# zRsq)H4s5|`JeAdq_wz$D2jd4i$K;Z_bpXhp`=iiV{4k<|=0eizSJf}o&R1R)+ubA( z9@5Ir;N8C)25Qz{i^_Zh7|%-ia(UVoh-tvE7PAlTDep_E^$?zH4)-wt3|3$MyqChv zDzWx6yK&fnqd;*%`w^|K3dLve8&}{Ye^^HuJfiP-JL9gR-@#~8=Z)NlrJwouOH7fS z->-Pqz}XnZ-w?~-s<0E{SjUR~?i9}Ah_Vemm5e$yDq$D&cK!PC++S8hDg^Us?O zJ(Q`4BzQ%9l_LNb#V4k*INRDiiMD1@YBTK;}m zx156I8wti>AcEFS1TBIX3yDNZvbE|7Pr)j zJO0Sog;6Sh9tav=c6#f;VH_O8$Q^w3ZQ3!egc2L$Gn-eV!cqqwMHoC zKEzc^z*Dnv{?Rl}P4DuzkTxFa=g;nc+wohftN9Mdsus(Nx8kB`n-qG}tj7RE!04%F zFS`gHVDd5p#xvJ6Y?l1USvzhxZp?ISEdp#6W1y3So@MLbF~HxvxwT&j^(R??TLuWX z#i+wtJ=j^22LSF6`Pm@q})BPUnWZi?k*oa)U+59 zOL;GJ`>oxc$1^k(+xP6|tR|)HNp}d%|d`~NbbrN02m`o0(R-6H^ef@p9BvUytZC&v3RPFadgqyZ6)=k;Jf&JN~x2fx{ZwC5R`B1n)ehS z7s|yvy_{EeNdtN%+Z>(1HH&LMrjk-lU!V7Yuitndm7`m#YEcer-36f9t*NkP$|s>5 zm^L3R4tfXH7 zPT;Z@3cowSpkD>O^QH*`-R%3+fPd|-&&hOMP!OF+Q7ZS%co)>QT@o%IVl}cMx@fiG z_+rir;FS{ixJ{n7aNjhz(3vPW(kz)(U1h;^oAvC9t-m}dIOzs;%v6BbVEjONL!%~A;C^E2AP8|(dqqCPUqd%4(Bf)sk169%A-#^u{o@s zy}wY)kmD0lt1EKi@!brmsJ^bRT(ZGNE8mFurk0uTMtr_i`&8@3ScNCicoz_Q;_peo zt_xLB0z}-Wj)Pw7swjG)=PC!|-0MXk559~XN z$b+AW&ZuTVhPim_g)8YBRfxkFQAj(UlMh4yNACJEbA8 zV>3OU3S5IfqJsTShHlR28Y}Q~JA{x(@Y6fL{@Pf##>dj8qcLs7n;&l}Tk2(+bpO zC<9Qap*xOtT?JaUY$ToYEofh1&r2fvKK^9ykGeC{+CslsN{QZXl(6W&;3&YbnWL{0 zJ&&#?41jB^1|v$DJka!dk3~4U0mI1MGKFSZyc5?bPCrs~vBS_FyO5@Q27`p2<5X$dC zfarCuA+ziXDKNC-{z`kLyx7nN0dL39xK&J*Y^nvvF}D4&iTXYwt+JD#;rn>-x7_q+ z{YtERn@MYbq*ETtcTyJ46)moh+4}N&XN)@wwFCSJmxJ%j`bW`uBq<04P;~EASdh~a z$vMXzQE~=_(+{&7tEP&!>Hhyc6`*>MJv6A*I*v1;&Q^PZ5>zg&-y;WJjZf?elWrmP z{n@ayfL|0VrOaxZ>Sw%;_bs6)2!G$X$|3p3rZMjhF+^TiNdA(a^dpR#PlqH6UC<rHHM}+F<0`vObn~wOrB9>3S=~_-G4b%m>hzfo=eOj; z(a4LkFdwjBBjTutlUrpmM+ilZpaz~7;d%pv@BzxkUa=q-fh_>a*On-uHa|9ISzn-x zcsP?pHc+`Fb;*eh#(};Y$4}txM`90~USN-;Lz2av8qh=>DAaU{;*3LRSl=&habnrH zFv#h~S!FUVqhanOavL^`Dd*F>2nnedDBKf~>|j~R{oHyI4`i6Q_$ls1#Za`ojMEs= zFQe%^j+IBW-3`Q{^IONsyrWZg^CIZ&?np`yPdGPzIivIMk#~U^mNC(l@(n1a}< zl~8t;#f@vPtT>92DcKYCsfdnuL zntcr7vwLyn-DeI}FY*i&HniIVW>N$G_(M2flxb;UdKV?)`oge%7b*bzdTbCjXr>Se4EUGDZW4+F>>! zh?cY6&lK6st@?h~mD#79%Is<^P|HWr6H?SQuQD&r)OU69&G!|4@hr)-)@}SzOYJoZ zRHcetvu4kE6c;K7W}~uauEA`wQ@|_6vaeQ|cBtNGpD|y|lZmMOs81K{uN7UpDN7J$ z<-`;{BhyC0|e+F#DzgD0xzG+@n$$p4N%A-B`1gK3k7ru*P|% zYYj@%1#+#7(!<+4c_b$X5F*}dBXBTNuo**mW`o`VDWjs4KFt)$fR4h*4k;N{{FriZR@M(yeMhk%Axp|1+tzo4|6p_EYtK{% z4EPYPpFo;g>2)giFM9x|Sg-4^WG~oI0R{7Eh?M%3NPfGKrKCAe%A>Xk%4z#-9j+ls z3sJYk@4UlCbNZ|e2;MDT^XES$+ahkty5ci8LBC8occ|$%4FTa27=NCR8 zL(#jnTKHJ1kz5`Wx&R+MXH}fD)nU&SY}RPb#dI03m2lH zClJ$)v&e^2c&f%Qww@Z07XNP@9U-6{?8QV9@|$MBd@tIPV|U`!U{uO8HlsE~RHR=f zt9158Ft4ol(|M}QoQ4zsy&YXu8Ka+Kq~G7)k~Ua-%wcqRhe|_zm!7C|QF)PXZ{r7Y zCkOJ(N}-Igq@{sjhsu!ldvkV zDwfkrO>}!@-cD0h8cHPF5-VYDb19`*H_GjJ68M*MECorR51IRR*jq*iG{3Q@9>4y7 zP(6CnaA=^aK#3o!G+vjKuWsP8W~vLKgAmRxuT;CO{~s?EiJ~Vr(gB@iOgx4K>m$aQKgMh{IHPM69XM^4dY~^VaMe58LL}lWjD}@->)9rD;;vkOb zYVtV0@;=nFOX+{A@}7|KYDY%DKE0CM%Kmr zVpnk~9IM_}`mvzS>)Bi4MLVRp>%11b%eI$GYmZUA+$cj^aK@06w$B z8N3-rI2WZNk$C3rM9LD64^6+f7(o~2PTs=?k4|i=MbpWmWKckIl~Bna^$Q7;LP`S9 zx0)8EEfnI6@M1PZ16+OQpTEy)$m0wkf1!UTin)N9AAsKX;g-9yTdJapGf|DgTle-A z25l@Bq<02MITY`~S^z-&(M`o!#}A0UCMtWK|GSTpx{=8nBFS>>!%k98b0{vBQs_Cp zH>jK_1+Q4OJpChSwm|J7(^`Zi@b#2#v^`?gHi)AYiaMEShArs7x0MEAZva6O8-u5} zYV%!jruC$DzL*1l>2|nd|9GD0yW%%y=mUQG@3spSGv7XCZbPV(v!V)0K9Ruy zF9#k*)OqvaeTb}Yh(pR4i2O1GSBE(@jPv634QD?F^}uuHsa_ylCxVwN2L1&OvnNIp zSA06(6Wd~$Mw7u8Z^l6=iBqvZ-vyb2mzi;aMbFqBFmg$u<%(46k;OxFftJxE8jcS_ z7cD`=QyEEeBFfNF@V3i=B}3(#?`rbd8hf9CD>+|flNr7V7ipX=E4ydumlHd$vr_@u zVc2|H%n9#i`yg{q%}5vR*oiy&n|tGRr7sYt-tyXOc;imO{`wYoy~%bYP}E0M1gkYz z?Hgz9lN9cZzLTjaqgxWno%EKt(0ED0YmY>*nE81vG(_{fYDuKEyXk#i*22nP?$@%> zfJ`OW@Ls8Y>y&A$D5am!=ILy}s!0xq`KXg$fpLuPO%m|Tn^#Y>4uwu8@8vuH9@Yz5 zP3TrZnXpu?RNeNF2oO}G($QnHU0KWqBHN*$tKU_v^K;so1nb#`xVTupdcjhmE6HzT zCBL)VS2T>)u$-*>>B2>X2E{VS4K+`L+uKIy5J{97K-~ zH}0H)~R`yL1PXA+fK!_0krqjJ*=mxWS@S+c_8pgtj-lf!a3_xPAn9P$D;4qrDvdp@h;cP-1Gw@ynE9{7u(SdM;Rs zt9LYg>u-S-QiJ}M|7-D`Z3Mja^s;3*G}Dxx={R#qmSmuGUjDj<={AD#o1kTKXR;nV zQ^D&focSK4oY2NQlpZ-iFw{)ei~f<`w4tIp&DmEW0aXS!nX5jc$)D}(M~hCW3U*!J zBV2UJgg{C@%k3dRwj-B=A}wlsev2svd}fQ^Vq(VVS#=Nya_-ls8wW8xFh^-ROgfDD3)2sq}p!98G+zE7_is`e2^ zWUyVmYU(Z;;8}OTj?y@FS)yUpkPqwRDK(8D?BB}1D`x?B9x6|3>Ks+opEE~~v#%aE zBs>D41DHRAQU2N}1;Rx9JKv&~*XuiKuD!W$RS-G@!>FIy)NhCtayDho$qsV{T}eLW z7km0adZ%cS&E|i z9u)sH3zVHLEV(EKQpk~xbk^+M1L+rif7Hb}iN&xSsWUh;zenFov%I5p(#|F+Z)dE0 zqUZa5|BQiKzv6@3V!vZK)F!{!Kr}qLwY{pQ>~(i8cz(0%aVV>31R6N1_zE72|2;M? zM?dYB_zf|K{rwK_v^g_wK}BiK^+ftH?B$`us9@BGv<{|tC8Ett*NBOy9dMhWU!V2TcJxq%(Tv z)CgFkKtj&}^-Mo}<{f`qY|J#O-z?paPdrrDNcOJcgjI@8x;nYXdM%+w9LqEOL8?{k z%-l?Kwf_G=ZRk2=`Y=E?X+(CID)>HARxN%M3sMG6PrFy82c+56+b#7-b9|Z9*rR)| z-~qpOTPK~{AWG1?siMaXbVh_X0+`h})0I%uAcx)M(QZH+P`TtFpmiXCb%;J%SeGb4 zW_p}P0da2qn6M^CZX6Yf{nlcOUOeQ0=FsG(k3`Zu@L|9ey2&}hm%xnP5b_Dzdqrxf z`v7zp2p7gR;m1qwX#{>RK3{76t-g~_%u}A~)OtK(+wfL8Aa<*Mw%B!ot14hmE`6Jj za(vk)gE&yJa-B5abm-W(-W5>M{qR@Reeh@BPTU8AkhOD%7SSOpAyh2@rnEGH)KcM$ zfg1etBT0krVQNmlqp{YWO5hJw^gjHdNj(R3_*La5_t>+AIVSz_9hCK^Yw?INYpQEj zKA|X;;!6Eovp4)9G2eaZ2wx&EzSG^QjOjJDNNXB`cp%8XNCMIEopAg0**7WFSNY!R zBduHS48>Yf7hJ+Jaya_+jqa$_`^F~3hkw7gs>r7be_Heq{zW9XA*?SQmK;79lM@g{ zYMlGk+pnlXVtY>jb-@Z#BiwPt(qr^`YcTQgCudh&gG*2i^>-fC0Hrj!ssVfyU-M@w z0kB$l*;&ge&dZ3xT%=b{Xkv}H7vA_Iq=E!OG#INm;XDI^D$ut9^?IL-U&3|3hr1|8 z5nH_8mL$@KRb4W9?Ko3FNnldH%1}Fnv2Sb7pdX=W9hi^d?^G*Cb5N=6H}F|tvppp6 zbaE530J^d1{!2}hgf<6|8UdF8j<{GA0u}(RD@sFI%w%BshZfZB6EKd?SGj644>{6m zO;K^j{Cp=@oLHvjjE(PC-AG@{0~fbMSYxsiTBKA*Qf0%)=vApL4BvBFsp1)}vvOFeOhw2(9Zzqo?PXbGBw{5qolQs3RBrAMfAj6peY2* z=<8ZlJE3azoN5Oj(poBLieSi~AoyQ%nLOCQz(7$Q>wnj3hn+-{8~1m>k|zH6S*jC} zR`b!b%VLgRpKWRNyiSg#mKH%J-`{5CrC&H3Vs&|Ce0_`aN-G_UQNb`HNMoyRE#&*e z7~11-@%J@}kY3rqTIkn^0mp=y?22$gTe{(Q9IqD0XX~`OYY05qGDep80&XoGYD$(pNxkXa1zbeYd3=E!=wU~D}zzjl+WY| zEv}*W#fiOY$ITwI)qdQYuKzMLD*9E*|1wB-E{+RDBe?XN}GD82(7ZSg!AJH-M-w+ z{E}7Q+RpaYQwk7nG|=mnFO` zRDpcDPUZ71ldMQG$&9lIX&D(bn@ey;2x@ssAL@Z~8z;IY#0 zGj*{j4x^}-(TsO*8%?awslno7DBkzbV5r5_;C?rBKTMsh*HlP^(iRJFem*?<)_pZw z9?QVtD*@#MEy3y}0GdM0lB zcc=>>#PoRg;qn%m&nY&|5N2NJwps7KHLhB`gs{9$e~I9C8Q}7Rljow4j<;4MyE$~J z5I^yvI@(uCftLsrEBprCi#(yP&A=U<;QKN+IPMqyh}CLDbi%{e*wJ+LU03(=702@k z!Voe@+I}{HH_av{*9IQ3Tu`4Z|2g)|s50xw>HqyClFU4WTEC~(M6te42KQx9EmAF; zfsMzP`bbOzKs>VczuI-=-?&O2J5N|)Dwy%?c=Ogq?wz{<*{OH0WLWY6cjuQ&m}2>C z{$f(jg0o!hEgKp)^%1$C-Bw62U=CdQ^2|pq8~z{0XE3eH!!^&5Hpj29?kGTGB71JI z%bJkPbz<7nPPj9hj|A!1sIv#92K$Tr=BbwFdRwm8S6 z_j$i!BU~QyslB(GNEoxz!OQq4&hD+y{|%dZu-;wfeH%DNf5HPA^yjd3crXT+Pyi@L zb=&}3ado>wwLMS+#M7apg6b(Zt5?p_p<>W7)vDBmy{!H!7?r@9D_31n97bBzG{N^T3pAGAen# z+M_^HB&gYguq!I0!_RV8Lo>mdm)goHz^N#Hfs=?XN94X0p}%nri%L8zAnU9EM=#E8 z|6TVTrQ-`N+B_|Rx4APnLURgnv6Q*90FhJjoj9h%HS}hp!UK_j02k%1tF zzB^WdL`mJB@Pxi;^@_Zpb>uK}=;VET^Uh?I47`^!lC8%#MIS1A{-Z2fc75Dzx=!U& zEZxMuzua)(nyJc+=KWtjdG(V1`o=ghp)%MNQn@)6C?%$Avx5C|TpgJWi!u2R;7!cP z7%LaC3*A7z(mrjxV(j8Hv?A<=&`j1R*NE4E3}rjlt2lE%s5d@Q8F^~KavhLf9X~Us zR_Qy-zLk^%HiaA_Lu-{cO-#Q5;Q&HFy}wAd{}CypG`73kh4$i*S(4_FwKtGbJoqfi z?!ZHgkKqbF`S%?LlD-P-BLXdr{rK}+8dLXC8aWk(=vIk74zM2n1;-~5UO9LQ8WY%= z@|!k7qSJWyt2$&KND?F)v}D^Rr~Ls2-y*&`aeYdpMNQ7ZN4HqVu>R^$L91q?FN*dm z5E}i##B%If@yn1F>-_Vv6lGEaiy>2<+*L%MuyPB*J)dC>fqk0Y*aB>ue(Cq0-8RFC zMry9g#4Ea^sLAyL{m}WoXlXVv`E3t)t4n@FOiqG9P{zLVGJgWpkA4<$&^?M`u%q$6 zUNJr_!5oO!M}88#Ca(p7#Azb89IUZQEuKe+M#3W%3>6dyO=rVDw44zhk)*E5hXl=- z0|B)L-=eMY|9DSS`5iPKyWwko+6A-5aJeP8$f(Gmt@(Wt*|X|YqCABOgEML;f6P(q zTm;=h88u6Mh`;N*Jn+g@Io=C)(&E-*!ph*&45ECTF9Vo*2l%!3rdJM0J4jo$mc_|A(qG2Nu@O1kxpnPXm0dBMZrNieJy3#sQpFtVDD|mGi*?n2~6h}uwCoby8 zU)Li>7Bzy0=ad?xeH0t0NDqwxaCf;BZnsWGPxA)53K3uo0Sv&&Nm<)LJEB*e37?JObP`Wm+4F`#9M6d@ z@w_<1JhO;(-HNCXE?EhGtS(B5k)<1~Qa1I{N_U_Xq&D4j{n*EQcX`!$wgl_eR)wY& zsNCCV0C`#LBzT}F3;{aaQKkzaG`_tm+5+jgCtDgIpeMI$okiWM--i)hE3NH99B_RF z;Nxr-23f?Ac-bkM<7Z7lxz`DQ@tC4?Lqv?^Wic#W(t7GiE2o#Ha0(BQ}GOO z2P*{WA!IxZuSFUvJnuK?q1ozXP*uTimy%}Wf{-4>+D&^PdilI@p3a3LtdHC95)8Hb z==YNENpya;>zSf;VJo~V%5TwkkLVY010W{0jMCg>)UVxL=!yp@7efKR=&oVK{H+A$ zeW{*&-E=FcFNXX^x@y!)@JY6YEz^U26Vlmhw`N{QH{fVbIVlgENs*Xc%Sjwr!Ha}; ze8`G1%*(?1llD)w?_Bpn&4KCm4R=8MA{euPK!iNZ#=>C2yA4vB#m@jHJ-Rd$jIf=9 zVKusAuN{45@{+ z#68v2rPB<=HM$J|q(;qOdVcT$ZuVT0_;gQR`r|le@&-{l%UFm|90I+X^`nPZWxI#w z5+u3<SOMba2- zlHPNC2na=bb@q}-VQ%FoqOazM4H)4pFLx`@^wXe7eE1BkP#>=~MhGTb)$BvnZ#B!7 zRTxqNc3~G31$hHF3bN_h6t@qIG=m=gLz z3&CNtq?XECreC8=VfPc?-+4A^(vc5k$isjn0@Fpp{I%@v@_W_dgc#_Bc zw%)onclZ`+_k<6eGCi+r?bk9Pwqc!Bqq-2z6Q4zn{N~DiQftTK^viE<`gqa2baH_$ zDs%FSGLB&?h2M~V?sFd4h}}0?l2|!hyD1Q18Kg`CwpG7KC%2;e8QRZ#M41tKxQ8R9 zmwRPmOLt7>kggk}Ah3^g?UTCv2A!2ujgBylJbGZ;OZs3fQL~$b!eK0g4eGCK1LwpD zt$NN$KIbnl3*_<=teJCuf@;;|zh-VuCnOpSfbW}6g?dc`v%@#SsQH4iZMztC%5rYj zav1#Qlme})0Mj4zYnc0cumGRrpCo7N0-yH98F$kCIh&E-|6lxl-bBb7Io9G0&lrX_MqnqiVwrUIMkOrO~*uqsJeZ@NHC&LA|96d!dk5lGMR7* zT#Sg?``IJRtnSgji_Hh^pBbPc*C8c*wV$B%6aU#@+zj^Go?l}~u9+Zu%qIZ1Q4(Gv zr)m+3{&Y$H6nTfA`L20U=EkNfL@-{~+q3G+*0q~?qx!b6{kG>LvG?7==RlGz9~&Tc zRq{zLco{6s^x0fPx>`Z?H6k7%)~6qpDhDrHHO1_-;qiw_+Bj}Gd(I>Zuf5=K@+cFJ zK&*z4*A(CS6om0Q(x>^Vw-aV?G}ivdDlCD+^7oe9(XZY#8ehD&ukPN{5Z`BNLf-3; z@X@e}n&x8@;+AnmWZYr^3a>L5@H{%;yeQ5BXPB|9RkP~7LRumExFN#gD8MXR9mZPB zk{KVm1eE6xW%l2scA=lcxuQTAQ&$mQQD4_w#pumZEG!X;Zg0)Nlu}JcQ#E*bj(6$5 zFY>v>+3Fueu$Db)j4owZbU2e?-o9?UN!;NUBK-Qe1^cNA!`XOVxlu#BO6!Y*5EQ0s z5jFfxy7#PMJO~&NV#QqKvEW6dj)+P)v!C)D-_q<>>k;1Pi#%XX7C8?n#;&7n^X+3^ zmPG~mwS7I zxdueZ{q~`yi6^{hpD8f$_((wNeezT3A0#;QjDz?+6rFPsu1sj_c~=>9<7Ij;zm!5HvHS2gOA4Zt%X4;L%kOKjIBDPrCYF* z^jSw4r#d69@0U8uhgj;euJdHn+t>9g{o^`I7(%551_V6ef;SX;&4YV(!t=mNNGnBU zvPffz%y*!?s~LvXd`h-UAwPJSP7}8Lz=20zcHdk_W|%fc4iaxgxl0>;lFDI_KCFjh zAK=Td_kF9M(Z9jr2e}4W`v}JN+f~iiOc|D|&dg;5zx{=20R3%Q(F-n2?uBw#?)So4 zdCPHrY`?)4G;n*^uCWJMRDBq;;VmSLoogMt4I6*txyapoyBj$^=+6H*7a~o8$!z3P zb*Lg1BF6V(2~pc=X=ps-G}4oIpJ5bMJi?6GIGFD$@K`l_rTC`e-XL}2wR4-Jd@_V17{79)6_NaqQ`jfjzXVC zNEx>=yU|EeA)H=6d5uOExDOtwa7LoqK<_NS5I~^@U3yYrqpZ4;4TaB&rwIPFGO;*C zx)_QqgpKc9fk&_ci4nYIBWRwb;fi9uOVx{bCoNXEz$a7)>cnU%qZce|;2p8CLXRdk zw3|*v;|On|_T8P}+a@o*gxG=f23QU=wJIW>f%^exUICz|9{>b_zOoP(h~*)gCmuS? zu9gCnFMDH&e?M9|x$fdQ2E3AwRy6pwq2p2}7WFIN*HrxKNcW?cfEBnfYrrWcEJqPQ zdo}KyaBB}>yG;Vqd#t>y5iVL3W^A-rjjqSFY`3b}-rk8M zA^2w{IoS&fm#M2FJCU5KZP0@DTy3bKFCOYCiV1p(`47YRx?3%!2F6lSL<@o~?GQ3w)J+kn{8rT^Xv_Nk4m*yQf0T`G5RJdsj9_ir0l2dY!4#H&Jpj{h>wnW14;NB>ZHZR*MkorViWfhG^tDb0h=;jIz^BuVp;u$~e z2N(x^9E$^WrHg)h4-YSElUXOwz|B!`eU-kw1R~mFb?LNQJf`?i6nz?^!Aw~_)m=kt zWe&Rt%CGrYdc(&DAG_$X&4&-|OIW#i>)R~l6?D?xFbVuH733AQu2m-JBns@gooP{Y zWAGp$pHV|K9(+I>ny4~#f}wd;^G~je%8@F{7Fur;@%#RUic)(gTf~UyYa1!_a?9Hf ze*8{<`&2vQW#_MnDyV6FSEOPI;lM^dkmVnEBvk5C7b!2ej(9Rdb65pG;m~r;s#?8n zGeU+!+#5ykoDQK6%LiyXx@$MhjkgIU-8f&Nn9xKfFl!^9ZpX) z{bCQ~d}Zdgid-ci)<{#I6R~TCZ2=XZyHQosc(kTB-P0e33tD77ug4nVG5%>|JY0ep zGH;#**zPcQCV6IK9hscPMCONaL0)JztDSq!(c z!neOQEdGG@-n7us4}@SMrhhBb`4PUn`T`ykOS`Gdq`WU?YWRMFsw-np9SY&INAI|S z(khRu5CaJ(zqVDs#gu-^@~!$gUX_@0FR6Ny^SAY104)RG&QL@(p!=VvVF8ZK_^e;l z+o}IA5mUknvYKBh_nV;R3NlFxXBAR5%TY$P$R zN@#X~36CJ(rvB+@OuK0oL=kE~(Vzx8!m+XqD5c~!z*Fko0!v2VG7=0zUwGM7*$mi@ zZCup6S0nnnpL$bSB^sVsG`0GwN?br)XaMIX2UIah%A`WFqqQWmt-pna2&*Kv<;K5& zG1PyjI=h6pvR2T@sXwOG{rPBY6#AI)p8kDYyZe8?;yZw+DlfUI=V5$oTAh#I|XiS#rV7g{Jv>Ms-4p1TjA z80A9R_(b&{zFp(Ym!)Z@S$So>J%0vq2I#9nOZyXtv5FWON zPUtY&8!RW`si(e=jcF+>JPblV!#W&a)idFb#(x3W01Rwj(ZDxkq4RM`dBt$ zMI%U9@T2m+2fnO_*k}c;q?&$^mMpU007`aj68mq0A-vVP$i-xDjy_^D#U)AfD+UdR zW104{n30{om#*O4J_sI1uLQl9f5dtq9W@bimMMa_Xr|nqN7XNvWKSO-fpsuwEnu*tQ8j0bp>XM`JkPC^{vtq)^-s zm_6V?24__2KmVtXoWRO#eR@5iCiAU*#a3<5WW>FjWA~VSrC#SgTwt$|yyrsLW zA=j{R@~@Bn-}dV~lwL^GUDp{(0bF<{)Dvc}+528+od@f8V?zYCf+x&c#8^TPP-uDj*xDkDZfE-!9sWL%V=dZD5T=5=D3 zoymWg7LQ4CXv)JOo9m=CsS9x3O6uK{5FHozH2-}q|C{YFV}B?l%YF~*JWKS=nF4k- z4h5)#E-%|5057f>e;3allQGE%ltQ}a=J6b!MY4hyLVbnzTXcb`NxzH;#3|}WY~v3# zYmJLzuoZVzxgKg=ml0O+O4o1!!#Rs1%WRLrUm_QUmI2^@2pD#(1M z3@`v4F(u@JSz~a&#rqWk@uz(hYfY*aTf=sna_raf#Ft!)VOe)f`m5?ZavO%WGyDbt z0%#v^MBq{E;vfU|-a97&JH_7HRej;;9Xj;yw5L-)Bqrq}EVD3$MB0hc@=< zgp85zW|lEE@z;dEP7ybbgQ0Gsyw*B5xq43%Fyf~iScvE1_YQD7qx{TM+)do*B74c( zi4UVevTnJzHVR!tp+I(xzOz?GCT6(n2#dUSv8y$Mv9Co-%tOC;G5UPiOw<~ViRTfAwZna`r;G6ydvL>g!LauE{9=swT-Pm#@s6C;gz5E`V}Hj7K*GxaCC!OJ z+CRoFn1%2?%<(+ETj2<{OqR+_0{Q%`9hVFc)aNy%W}_{WI&V!o&BVWCB)pHDmZ0zp zy$(2f+P+kON<)06(aQ$R;062CIPJMr4?YrvDehnW4TQ`Au*F9GoA01hpF%xc9vj{U zGpQ=hw!=E_OPKy{CzEOnKpic%bD3;W`001vWj$m2 z)bRZ2J9VWZu+^O-!t!If09k{%xYswCb07EhY3L&&RP5k`w4MSSIrZvePy}YI6O;|2} z9fH&jX6VaVh{s4N0zazJ%M@edQ@pckg|hVRw8?$0OXEZlnvX92bJ5ueRQ2`iBAW`b zSf^a>U=hqRtHdakQt6UDc6PC`6BxIkU+1@CS71UTR$hh*G=~>^j&~J$2U@*d6N0aX!^!CvmJN_L9iruewjx&{{XCrY(QK0 zUMD6348s#bUMm5I^L~*zJA9$zdBL_LR|5AzdK5GZ3BW!ziLrZ&eeu+cmZN3~8b(kW z7!;#IBQ301ecwI>uwVE-g=@|#ikVvdwjsdF^h_{60FgAG_9Kx8c-VK0*a%oCdltqY z!~EW34!WZYCg{w8ySWiDIl%zXTa}bnGu)LuN^8o)XvrzO*qzB{gkq`Mj*J6I z4Fl<1@Q2W(m=>%Re+I-zXsoM*;)5!XE9_8<>p!~rwgApWAYO@mhRr6n8SP8|oR&+b z7=WrOE`ZZ{TBq1ogln;>_`}xXk|FuhE$ZC$MI-NR$6qm(4RQ!uA-6g*e z_B=y*7G5CIHz>9hroLQYeakYs-)-yN_T{*66RzRUQGe}~TDnbD>lyeo31#FH25WQ0cR$yV5nK#s2oRb-`f^d}8v)+JrX1SX1I!3I8@j*}QZ#zz^EN)T=*mnLUyo)`=rN zd+8i`BOw4|5HCzDV3Hp>x^&2Te0x_Sk4=CYS<;B&MjL zr`-l#V57BcM;P9_ARSM>6juFZsqUihM{2}*2N4SS!jeD{mz24~-VKb!fA|;c*P~8o zpHSo9`W)C4W6K#m1VTiiQZ40N?V@_!^@yv)-HuB;HkpZPf$|!V;8X^b_}K~P``d-@ z@&z{3=P;oY_#tmp=Cmr0NRgAUAI$)0##lI9ste|uf3qSiPvya#X-t|a>jPPEBjr$KGNAD0!S!?$nb$WNt`{H7V3jGrmLJ+}O(MPT6_^p7umy$TM< z{or9|Ceb$obp6qqQhdd}>n6UZVpqWXrY`HnfSWco4xa(_0iqxM8(Rw#MnQY-&EJ-f z5BG$IgI<$6RVcJ$U1y!eR)L7+r2uc}lp}x*fOk9+vhy5tM1V!dg*@`G$rdugZLX%Z zC-VlHh{;U~fq$-M*t&tvJA|h-htUq`7 z(x&fr&Bv{)nA@zo@$r1Ai>W51P~S32tBoa$!goS*hU(w<4yi@m2qX59R;LS@5xeB6MyXK zHI-7zAdTy%8VBBxL%5B7CV47ww@A$J#Kh1u&E#Wr^Ay{=(TJ{{#XBKU z7K1l*(DiBv`P8gY-eZLOcH=^C`A#F@*H8~S{Pl1#VRqhPTTMF+C^od)8Ws9n@`~|S z^xJbJxKN%|`u-b#ztj}@>z&+ap;{hYaF%e4Ed(j=f3%nxkBP?O!;`*Fi*+4#PqB(6 zbC{;jUc5y+8Y(ptNm!ia>TA3{75Q!}R)CfB8m`m;_mtJGMqmfQcDWMO=_)l;E#$#@ zv9qevYs}-euZs+>6GSVL=e?%1_D=>8F1h6T3vj-&#FOy`^4n zy5mo?oa*)f>_=>Yq^Cb~?lIj!`eCJ(^PBh^7l<)0!_0=r!S<_ohzdVgq$=0ZoHfKD zvH0`0XiEDn@%H|kqS^B+X36d6A`zZBSkH{qPBl4ZG*vZbrt?)MoOXD?%{j_Nb8j3` z{FZ?(po~`yqy~&d+8}Su-gDCiVpkZHgGOv@gazZyz4!QuEPM06J8kMtn(p`w#N?a8k6(2+v0{>@r_|GW>v7_*E0rSv+U9v(PbaUdN| zUSea$ZU;l0??csI5@cS$12~U^a$ePEQd6pu-~6FfPCK73 ztHi;3Q%85{8v2FI)EvVQ4rYkS7r+zFpp9%u&XqyyMT#(>ki9OTODuNU6$`n$3@qo| zV(O&aGc-0|!>zQ#a=I7(vadri*<}(VSUD7VB%c{w={sq99#ym_znGQ)&bw6B(!V?gjI+MiLs#o)1SKj&!YE!`0d z*Ssmas*I|88z{48iIkCoHCZjYUt2;5KRc3>OIk2`R!tdw!MkeM57=Y!;#8z>TZ&(m zF)L-cJUj9+`qr3)`Ol6aGol-M4R}!W0KK3r3f)lbJkZ$V+(_l;T7H=yF18}~O14;C zu3)&Q0aUILuT(IAdWt2RG^Q6DhHf*-8f(cGOY#kkfD zJ{$UK>*~_5FOW(@R%fsBe!jTi-8C|6$H?#c%;*=3WeW7>01G$HTSI6vJtj|Kn@=~q z*=&Ob>^PvnJehGU;d?h9H_ZgBcqx7PZOCpSm$JKcSdyZG(@LK$4K3)^o&ILKBDH2myp=M2ILX|lSTy{Sqiye za8qFgNkm9bgu@mEDI{>;2VVOtb=#zFbS}R1sjkJ`oL|G(;uZ4YC@hjXh(H#+7Irm z?oqx0NE#Vq1|L|Nr9GH|5AfrH08cF8yIAy@i!_fb2ejAu){e)MnZa*^aMw$@A?ohZ zcsa9-M|Abto{RtppmGO-cY6zA4nY3UxmRpib+5J5K4%yLLT&hzgyC$-HE=zfGjJc| zODEwbfdivq$+Ym>2>vWt2Tp*)e*$jq@O|*9=RT82o8|DEt#|BoNE8qXv4Zv za#>eoIB21ySW=vYji2s(Re*4Mu6(bWLE80?57uPJhgu_zVj46L!Q=_ffiU=T(8K4_ z8ZDFH`A~qgUSQCK)lvDA#ZUXZ9@^5{Z^5?|#Uo6oOYBByBz(}gESS+r30WDR; z=K^%MJI*&Jh0jr`(&| z2|snvi{H@Km?X3+DU;g*Jz1>Uj7~JmofEfeiU*X2*sg9&o5Npv)y6ep2~SzjJ(xw8=8RV ze|H1w)@Q=@+6Ez4d&V{UQdUIMNKVG2bRP4h;G3U+eCRl>*yc(0n_DM*A?NCT^iil@ z=h$@LGXBBtme#i`xQX&{lS>Kf)|^@%LDxX#Ycx&!p*6IRlO3*;c=|a?f$??^a>-1Rx``N z^6J=laab{x$wl58rV}VYjF-l7$Hn)wWkd!`g1=NYyDusH))5D&TE9P@s7LhJw+`g> z9-ukS+aVGqF?d$eA8-fZzvVdLPLZOY0!gM)`Z|o zAT@s{J2SfAR+$Zc5q#wuSs!~#mvK~Ye>tA1Z~FaBIO_MU&wXzRNsc zxTO$sfDbI_z?$(YFSzn+^RSos6i>BieK8uzZ?&8xV1p!F6PYr0dj7J;&UAbrUjVh{ zfcCwpybtxBz9!{K-le2RnsZX5_lrITcy+)V|FwJD^FIvW`r=O;cQewg8-rAGS*pZ^ z)D-m$np{!gTe)1ld_5$Qu8}HRr>Y|v>*0Y4^PZeuz4j?sU|JtK99lC9lM}pLlfG`5A1Xrhdqo#XyUG@vLcZ%y|Fr9wz1)UeJ?dA8aCYUyUe<1Qe zlIraNJrB)ETR&-`CeiJ3_6RTqTicInPQ4WWx{{g_pFX6oSm}rS@8b0YzdSv$FY{NY z74z8WN+EokRiyWn`sqoU)gdBxS68NT6l_m*H!x9d38k!|KN#|{JCL6B9Nz{AQh$>{ zr_l_wN*-5>@5^xk{q~V&!pU5<9vUd#RzdQIZy|6>h~DvASZ5hh;5EqfB=&wa0e*#d zxsb(27+%Q0I-ygeRX)f;_pNOh%`}6PdXuM)q`zE*9`FpD;-F|?tTOl{#>C!?ohiGL zM7T&H&2gpbO=i?zQ-6TpYpr~YIJ@MDJR1Eflab!iPwN4OUi@Bcz{2`=Q(Dc=Vw+As zP<&M?f+*hJ_l`njPRoZsefniewJ^p}T0h|)(`ifUT-D4!I6b17@}rib6RRR4(EQF6 z$Aj8-?P#ar(1|uRbM4JGe2sUFsap&9FDxM-1V&|q-E@WA9qhK038a<1DweY~)#<5S z1S0E>V*oYnq3~ooI4$68j5_~7HPv_;g|pr6N4&2QU%%WppeB?WKSVGs_vS^om(i>A z^YuKjCrObRE%qYOn;WT;>kVBuQPR9`-Iv6-Uo?ctQ#X`F;1!A;Q%PMwVHq|YB-q6~ zZo~>w1>4bZkIspG32kj0FlKvC<%BLHCVb#Xzmhy%Yj~R%InmP@3;W|GCp(ds9?^Yn zCr|0N-pDb2NIbuKa(o9rtn0~zggK0l?+Y0x6dK zr26xL;tZM39$T?ZLJsO$g2$hvI=RXkw3R7nkya4-6;@pB_oF7+oO1%ZrMV>(idQLBbTC17h+LLl#GI^?YEUP z^#b>wY)yL0dq2^v#Vd~<&g%rjN2ueE&@!yFxm1xgUW@D}EeN;4VcVxkd)RPh+T5r& zI%TF{?Fl~psxfZkl1k>=4IcWJ&V(YpcFpvldPKG^z|do^EK0GNH-R{v%!l)?x!JWLIQNUtFcw^#__ z_nYh=Chk9NM)w(PKN4uHfrn*YS;DrIK-e%dkXwR8&X)cD0w^-{mMO>ejRbMuP0aXVCZ?t(`fWwf#E*|J!3Q|YA($@ElQp?Lc8 zbM43$HaUDw>}gj{CoB0d`02)|@x9y=ZuklMOgYFRgPmOG<$E{b72t23?X4iiS7jV+ zw+S(F7_4#|3u*-x=x-+muN=_FiP*GEeD}Au(MP!oxFBiKc>~ydES?mwmLSlTxE)vY zRovH4se)$opJ(qQJfH8+w@MmkH~3UtoPBb}Aj-^#Io|BJEW{*oBU}U5@!7*WVECP& zNhfKTXXGSP_2eQkq9u3XTt;Pvw3iF@0TGZRYl zQ}^>*GUw>OmrS+JIr}KD<@U<#!-Ok5JlYqO7=OKCSqS1$Rewg$9oeDhW?epV$+-2K z``)I<7T~ghQ2I2Hs5OzRNC+j8l=k9g(_Jr{i)}J_D0S5D&JngYAc~<&e_g6$ud&|h zpw@&X8AP5HV;OfC(znCz;p!zKIoi#A3@9;M;w}sop-5sLt_9b){%+MI@V+^lSO@qIo9&lvaUEkCdmf^XA ze}x_2fVnqi!Qf1b1HfvVs5p@Y+Q04wp}3lo4;rhin{Sl%P8e!RUOmVXwiOD9Yjyr< zmf#>hGuJYTmj?o0Y+-CX<&rgkKB0msw6W=I^rMzm9uqKLkSLJt1oVE?Tx?(AcZ}T4 zMsLLVCiole+o-xoaZlfbqn&>z5&%jqu`K^NY`-HP68lBoK#Yisp!@i%UTNy-yC~GE z#$Sz>c8?pXZ+NaxzuhLCm80b(!-d5MK>=k%BhPOQEuCRnF(y4-Fs3y|mgTcatwM-# zkAw!G0obIzL=nm9&NnxpX$D&WqD!sy5fXhw^kyQTnWeQ znzjB)amuJ!7nL$f)Zx`%sS~1UD5|*Kk$q%Kr~2LIS(kfxX_YtcC2!R5Ee%DG`hZ6J z_Y4Ob&my1KgTC}rDJ8>3Dmz?LUb@9Pu{@yJ*E3j>cWG5H%Gj<|nsva(&neyuri7R! z!ImXxncfuo<|2J5C?o_ewB%=ck9%qar}3 z4z94*ACw#8dDYno@#2P-KP#U9Sb#+VzrFEV9lVpi>*~qFPpi7GpT7LS%gd_={K>I5 z)~+DgBi*Q(s>son73udqA10zq9V;AI$d_9;{EDW30H{0g_e%`D+Dy&wgWYaE_d<%A|GuHLHT; zUoiEKRVTtR(#>fs_hoNeRo>0UG9UeuT2WQ<= zmFx^u$WQ$;u4a}J4k=h(U)nLOP~87c5{7?Vnhx!w&{ql2CqT@37CPPHt+AzS>ZZq{7vW(a6mK432T2kT zM{tTmbMf9`!3LIvTnb3w7___cglayEMET=ah(~6$UX_j4p6WVO_jmgK^z@w7$`rFy zpp5tREJ$M~bVVWe< zGObu?86wd1rwS;>6xD&d9!))~MElf(OIHhoYc)mW{k%$Bg&r3#4VkUF4Kulh!cH|b;vW9N^42w(}1k2t{c0z}O~L9``h zb%%Z)FTpNOM~e6tzpsu*+D%6qxZGB92V?eQ#z?7`n}D>~2Q3C0uBDMR(oaDiK9S)_ zf+J}a7M3pA&{n3?OfAdTwc-(e*%SJdBkcIV)cn2U_4~_|mWv==0|fn3d1r3b3?n(3 zPzWMN;<}xqs1DSX+@`h)6(UfKq4yCdg9Be_Z*@dO^lbj73D2LGtIum7Co47fLV<7v zOSZqC>Uaa#iOygA9^_`h)CRlGDh$3BD}!Ics@qvFwHNkzxJ;B*{;kWuT!b-dhE7(o zSIStdAr=b%s(vLlRNW7`w=W%ePMPfbvVe1UO^3 zTn;eL(VQ{=j^9s&j+>ddhS%1+E7EPZFvE%u6D{yLWBb;#-N=XM_O9>y?Km$?+T++B zd<`3{O_p<-kXsGonP*lq!i(VRy#><@2)5finxvT)BE6u+vNvLNYA!>irr-X{8SHsTp15t z+~gkNg#@oxPA8GZgBsi2S{U%2r{h~I~-=g+a`baz6EGe3}8J3OrH2UJa zj;pLf8mf0+CI`WpSq5vQ9TP*Pylgs8G=h=GdmzoX*S4sB<9dKbg8UBehn^fA7GMI$ zx;SX05Kx`C6`=W783v<&+4(v)m_OmXayRpJ@Pe3Z{>QZDad8bNLAg4kmG0CuwngvL z=9E~Iw7a0rw1iFAWJ)VS6Tb2pujD+n$K&hl3MEFQ?nQ|R_ceAyCa4JgU03quNJ7~Z z&xDb3t2rGfmZS^krK4KnJ`p%eR-Ctr3LQ1Q7_bbf~2AADZ;gU;J_ zq)3kSy~c)e-R5EW1To^i!1eoOS-)rHzq>^#&+EH#OoUX>MF)E470Z>yYcdl*6qAPX z6M|E_v30Qp!n@qngp#;~!xm~G3K8Ga%6<)DLRtH5B&;=VY>J&Rv`<79(pWIlXDR8o zBn26v+-zyn^~Q!_41(mlDbQ<_TY3&buGEFBDbF9%Mv&8J(M%y9u1okhxK<6d{Y@dh zhvrw)_kw=4rOC!y-vIyZYZ3ScD0zzNfu0iNQ1)vWKQ!~8EOfk}kqUns%JIy4=(eKKcR<#2{KCBVuFnYmf%8omrEUaluado27@&=?Drtk|W|ly1Pi8LFPOL=U`q+ zHgmq~W$Qx&d$=d~f`}(earC)U%aDRb8WB|P8`Ms*M1?NZ=3!U#;}uLVSF+utQem`0&)B4 z9q((^HWGliyxrY{bN0jyuJpCP`B`}FhxgyC#%~%%J^qU=>IG!oyeR;LR%8NmnW3^9 zcT`pq@M@>a`hjYtLz1w@uE)eAvW@y0TC%Y{Cu#l zciSV&cQy8N&f!Yih!4l{rd@qjct?_0CJY!}qp;6!`;2ui4tCI4r~WePv{Ku?@LvRD z6D<(j?1qv3D)zsPLquw9~d0D+#9sb{mU*Q^D_NNr)ie%lLLftuKHk08~J$zdcd< z_)>mbM5TL=9y4I?GezRpg?_B^tW>E|(4Sx>AhB~mt4IMYc-hx&T)T5YTJQLYCu~tN z0h+N~NV~%~Pdh)}{)o`G*|r_D9^!S_2u@NdxzjmNEaB||sMdDgba@^)Wk2ca9Q3j} z@~e5D-E@irv<%1UZ|O`gwF7b6UEH}w4enW$7dxD7<;F2^Vyl#1I=_2f5cfkns%fwV zNc1p63cst27qX8|Zp7aoKc$}sNJ5r1ExJ3B^EBN>Ci~}gQ{YgT(XxH;+<7|&3zevz z=55GWVIn}>%{{|8o?|A%aO1I8elWz5RZ%*XPxfTk>t#bYy2Aqh!oDuqF zA{PeoG#udbPO(M#D$$a|QAYa5vP|x>SGRW(o&v!CgBKZs^fh{5-@^ZBu1~_%C z8?diD+@-f^fj?N0`II%~81>8hO@F58t+m#%B4;3`%$9x9b%0|e%q8Pu00)r+Mcm?T z2K00(N2b9@<|Xz~rlis;8lTDrjy>Y>6+0SJFClT!prf*No+*qiB7{u=k8uMnW_m2Z z(U@S79Z{u}qC`|5=>qkIvk-Ynf?w+XPNK(Z!7} z7-dyTb+A8qXH;utxI=>tV=a8V6?Aj`PDW~avY-U_)}vk|?>Lx7UT^sabbKgMVb7LW zP4D0vY-CWUGuP=gf@sZ=F*g zrxX#hK7*er_`;NdQA>uRlN8R>-rE)#{D{Ud{=%Z7!J6S&D^oOUHe5}a9y8{}tm4Ug zj9@&Vs+9-Yv6C3L8p!8WiZ!fqZK;NimhhMb-xPBD>{&M#zpRe6l?)#5JHllby;{*A zp_FGy=_i|~I($bWs;2rnD5##T?F&S$V|)(>OMW!CwVsPRp)!-B$>NG|2x~D+-;pC4 zt`+&eKIX{*SJuQc8Om{BR*m)@;fEi*gC@m#9UMn1qj9(MDA#2=JQudPUrk>(e~RA_ zvjhml$OJ+{90!&mbW+2YmOtQtQEZf$EZ^g|bC(otvdT+#R9iHjNYJ`w2Vo0u%kh>| zh4hc|4nwYcw-~p5GsKh~V>J1vjh32+rhUzEbf5LFh9tIBGB9 zo1v)x_>cekw}1cVS})bL|Lek*8O#3lMV{g+|6y~!M(wtF^nr$I{FvN>iPi(>|J;;dk}{b>;PK z6xV6j`vV#ZnISqUohwP5+hbZ<6G4k7))Zkc_B#>z+4B@RYBAVPMyKVOH5C%(8)8h; z%a`LRFN@jL%1IbPln&Kf42oh`r?I52`n%P?m-)|sbGh*EReb;7tKc;w8^VhfWr*zr zsToq^x!JPH`0 zOk~^tz3Tq}o^+>)4llT8AGMdf(sRcmJ_HSpXW@Byj3$iY-StJYh>B zE0I;1dGcJpbI!H5%k5^VR4$dDmNGr(+VO+Wj_=(&zW4O*oySj}-u?2<$(_4Tj!&LG zyz}5VCuQJ!*P^qLd~RPW^qt8-I>kcW9ay7!u_vucrT5`CC-=X*^W^xi`=8x8+5P;{ zql0Vef5wsQ+g>EPb|;WxEQ6sPEGIhcJKxJv@wDHNzFRCjm;F*_pnmk@){Wbzw&yuL zI}jrouHMc4WPjugZ85dO2pDhdOV@U~Qgj{pd^vjf{bbe*iY?zht#^urYNI-AwC1y? z^UuVYoP;(oHns0qwipI-)U{W;*6CEge>N`K&1Tu}742%T(hnEocioxjs4YI9-E*VB z9~YaoL2%kG7RuwZ=hI63-9aMo45Y$7zKPy&C6RR>S%J%c1)}Oy& z>u_Xs{YhWC1@t|6>2(8J@`oOe+?Bz|2_stba&Y7O-m}T5Sg1`pJ#SoWRL@%9cNS~i z^L)2Ur_@<}I5+$t8cYH~wBvQ7<@oBP;q?Z^g7>T&w7p`%A3Yma=cjs0h6C50i7=XU zI$`1q=-Z#UVqjat^L195R_}RG91ns)pMG*GpOq%%c_&e3d8ODZ2i<|Y+||IPU3KWU$l}un zzAMJIYe$Z@1DlWGwl$D}>nj==5QjmI`PO9Q+C)i{FtlBF`HOD5@_pzPC*QZmEt}p{ zsy%m}#jj{sj@O-uy^bqsuj4=L(aUD51#Le2L(7#;py)S>2KF=r_-d+RkBEaiMC@Za zsaemK^wZ?aOIzUwbQNho0!L%v8yBV+XE^d>#<`lm*Xl=OwTNiYIE{*hpc+-1-_I|5 zmv&eT{NRkf2szAD*>h-%ZDILtXI8K#;b62PxMui#TCC+D#JNb?o#jfwt z@r6RhTrV2Tk`Mf-8x(~MVvZd~0ZB`QzP5hj+CJ@8SDmDc9AB6nT2^R^k!ATa;7V`p z2R?meMDj4`CVytWJ9YYl2)N91KIz$zY2PG3_v!gz0E@LK;4F zyngbBAJ`*@G=-5hw9ij{vn(WyC{GkjAF|rR&4MYFA4xHAR{UqP=49HQ(?wXJJt5VP zxCZSY#5;HF=k|~;C@{3ziQYDDed`%DKB+bqczh*bgUf@K} z0e*6QhYlT;lBBgc#2xkmySpa>`?==~{WC-jH3J*ri^gH!_Ispi!%8TL#Zu6>Ns*b5 z;I>EwlHPL(aP0lQOCm%B0~=#=2R^A;g2YePU(LUnk=FV?=_zz$#aNNDGZrKxhluuM z_5+FOuQ|K#2Y}vWWcyPautRPW8ASBOc4$eB(Xa|bK)RBPtA^!)KUMto)TdufiLXyJ zD&5EvjQvw-k)}8$i3#Zm*{%brRH=-U%)N}mQ#&Av3~k5j*=`r&rME-HOvzNRgAkF_ z#vW~nAT4wfCF{-Veg}aMa7elydox_cGWK1mmfCTBe}vrY!)ai9WMLi&>H6MtKapmP z1EpYw&U1T2G94kY+FT;LHeDZ5;rt$5@oog!nxw?JbP7|)8?N-<%~ZDQDw#&tZ z^CR0mbG&2*GPm?>RB1PEj~p`Hh(Y4ZHGK~~pCKylNjETwMfc9<<*bt1>exfNmR{fY zyU?DLLK4`noXp)AWZ24nvP;XjMVEta9!U@n(z13(WC?_0-%7T~jhts|mztUMmgiW* znV^3Td>0|B^o~yoVzAO27;!_|e9y5+qdav+q|uPc+J{^bZypdeTFMVZmZYEf1EVyS z6IYn0b~qsmC-Bc4z{N&Hks}=;umNFW#zuNF-@P?<#Am13?zOB$HSSeLormay%xL77``ycyJjeFvV}xV125aqIqe=tPnf$g$)3A>y&!3)EqN9CtUABVu0Cogtij zFZN4MTo-clr*4GT-lImZPvA6$Kf6azJ0D+xzVH??=VcB5L&+2lP>7w@#%S zVs51ZDX6|Dl#*rn{uoe`j9lo9Ah~oHj+L?(z}_L_HtA~C>@xn;)ZRoYiqjQPMq>*n zxHL;Op2vRjJu8sUXTqM@;U4WQ@iKsY+>VqJjqc{C<02jerMN1B+Kc3j#G<_?PHkn_ zh6u0PkL)gq)1Iq|(#;xmfpmLP@jc;qB6h?AdR;p23(xPYv|`SJ*e7mJYgacZS>rW+ z6FI$}^k!mDvHws;6B3nsqU-njBjM`&zxbNuea$}if%pHA%G`MR{ZEZh81ccDygq$ePtmF;*b(jGx~8QG5CK?M8FkJt~Wz0w4&Y=JT;}S;%i`>|RFIN@pR%kp|}G z0da_N#j)^YBgu@z-pRC+o5#Bmg)29?z(}~`*;z)!lX;^Vhnf**P8Kp8V&YgaWzIr1 zQ6EMs8;*R0-KKMBpvR2WedK&F1y_*<7OXeSu7QBrHEbudJR=Lj=2BVA`HAvXtOBAnaQfm56n7l%+FMFDbFn5d)dCU+T9SqN=vWEI2$(VDi0 z0PAI$+I6wI2&@-MAh--VO3ZO->@0965cke1_y)w-FuaD+`=FS{&Jx!F7}BI7_5tQ& z~p=LADyh!0lZQ5|IVrX2b#%0unJLO&Ii=Rp6gwwr$36 zALJ^$R-lhM>=EEBFm`cYeyrQa*(1<$g4K0p)h;vC^e&!1PmpV^V$I2_wt{T|%Xo)% zAZ@zaF=;$VL043N=7tDOv! zud-pra52awqHtmmC*DSOFP(wZ0=3D=?p+r;Jc>z9%Rp8yYY1$KaODbR8Q~CGA(nEO z%HdfwtLPYa@?ge8DmTyO0~GsI77L+X23Tiec&FAA)rU+FOex^VyISS0b($`%qZEdhJ*U@mw@wShKOKIXMBL% zO%#YhBn%-PwgK>89!|f?`QX6(0=EcE2`68t#02teFn5FAhCxeJPyx)XREg5(*cM>o zt!+eSx}HZ98rAJw#fLU;^v)vr1aVTtAWY!Gqd8P@2u;B>ljqFMv6+ZkDctcO@dl_@ z$0$$86B+|G6WP5Xf|c)_Ri?qLD8s~ZkBs~R6ACy#h%aHcWpg+)H^+PeB={a5@K$+Z7PINZV2yZi25}|}LY=q{}XJtb;fKWbo7L$<; z`#?-dS#b=jj{&S0w}_#uh;zWi(4;W3HNLf2LXxqw0KsBvACz`r#wc0juIyYUzrw`w zP}P2-^%s;=5KE;ddsVKu2zQtQs*aHjCs=sJj6uGD977Y54KvcB==^bOcaSwjAZ)18 zgB5fbkR&l)7}!KoE_~qd6l8D}>}2l3vf>3|Cz>%TIC(Ld=rs9egCOC#OBKmg$<5Kk z$;$Q40!9q*+P!o1F!ZGYDd}UFF}@gK7D99BF+38_EyzMpdBvviv>dpgON>3iRAYf= z31j7%V2JjQTW@5rXharENQKkyh=XH^)F_Dj!?(UUK6(7;;nVx~a^+H~*=j;gz;d

qZzr<`+T&`e1@T{^lu9G znMxJ9(6u^rY}t10e7%f2IajWiSL@(4PVLR?LYj4iA3`@MCF(8YrF3sgWoTThoqQT} zy3J~<0nJxyB!3^x*DBYOsVXM=%K17NHM~+PwaU<`Wy>W<*wIdxlXsBbObQ-CSZ1YC zt%Fyny|`L}Xd#`cmXkS$S*f9nqYb>4=4+HnC2U^+{zvmQOOkna%`=^;m4IPe+sU`#!ltsXM3r4&9PMhYSxpvtG_KyRC2M)K z`;GESE#`PwwT+Q@jW`Z8jhzt0On&UCxzGYD@##-CDI>sU$KG8dqz#>&d%o z)*6j+Gx6(bT(hx~vD`vh?UhZ$A-rqt(#r1SRmtbon)Aeas_l|H4x{Bk!@7!$@T@kw^~PTdA8A5X=6-vmu96@Lyp&M(xFwMJ5);? zWn`Pp8X{hs?K+^$8fL5Bt~3FX+S*14Ph$g{J^k5MquEZ}Uu>YL#ortjox?ON|n6>kYGAYa+Hw zw%r8005q)CNM>GqMB+F{%aVk%CHaeEEqU2p8Tj%WEU z_F=dBwkt&_1KFElK<|Ceu~j<}Ck!DJ->qoid#aM@c<6XQ==(s91IHDX>reWFWK!Q# z^|LG)=&|jMJgy+4$gw7><$>(=pvDTf<)o{M>5pvL^;dvnPRt*Kk>d$DvRy5qh&{0v zpqN~cwPSm#K%Ix=VIOn?*;R!Y11-UuQv{2is-|`-RjG4VYT-|YhZL*k-u6D>oVUb8Wlr2a_ytI@}IXO#W+Gc2xR1A zkhKJ%bC?+4|ZOix6wNqsiLLebg5#gmeK&Xxmo^(Wi5;z$8 zBYiM$=+Fs#e~5sH-@0|_-VX;M(Uutd#BnXCdduxIr>n{d$IJ;S6@J*Yd#a{EB!+4= z$r%8&QNALpIczXYyi~@4t$He`x@{|vJ#(8>h zBR`Ts*o7+7+{lk70bO7ycj`8MuAl>c7FUx)yev1&jxL`z-;%nRqkiqVL9k!R$7y={&rSQUUbbJIXClfw5kc$I?x7V&0B)nl@e1h6-AA@Rt`IQbs# zt4h#B@^oM;Da=@5dm~4L&S)G0#Z8Rcr~bereY8)cU}4beeNWvZFB(X!?8L2bu+UVT z2vXTo2QyX3nh4WJ@cu^mbmBU4-UOtewA0=SlgRY|8OHq0n)oYe%B_1(wOgCG#6zdR z`EKpGGf#NZ0_!yHol0WN0qrcJ$qwx^Rqs`wopZ_kf{Qxb$cLn_jevX!HsVojE*b1Z0ZNdsPsDkHV=Rtm;KJsT_h(#x#@4^E|fjz?^A3{4JJVdRbo?ZEikk!oult5 zQLTEsB@>mg@5x9iTl=|>{lb;JYP?m|^V_Q$H0eP`v)lZTG$lcBE4#M6mC2QGU#bh@Ex*wi6KP7J+CH^geI z?h|(=2B8kV4;=iBz3)|`nH`J(XT4FOgR|)eYR4kSE5v=R5Jne9_8F;ze!q_p)*I%` zo*?rU!U-L=^7-II7?J+1j+~S=FS+cuQbSa>nFy^o6&(19aXYe=-VzOb)Py8U(CaF9 ztUW~vYSJPWJt8{3R)(E%L`A+bfM;MP>8&%>`GqX^a4Mazk->;kFJaVD9A)#c3Eh#D0q&mo7GpT{a8BHFoxW9$uq#0cp1Uwyosj+qsOo zEI9K@{901#8@zVN)Gm^ zrUcfnB5lGj3snFc3M8u*1sESdT3{GiD2_LF(m?Ax!HQoSafI%kV>goTRPJK$3FZ*F<`c9bXNZSk?-FPRV;8qkf1EcM1HCUf z)T3<1=n>$8f^7@(mxv;p7z&T(gx!E?6LX#YxOGDyZ72*F)=x~f<9Uyz7Smr1x3?HFdj%J zui#^e>oc;$8Bjvo$RIvSGX|9*d&+UeErE&HEdlmT$X{&I!7M1 z?$<#q28f3nc-|HbMM393qOCM8_Q>FRqJUVR`~uGgxPwz!y`2WD8p2^9)!%-_>1FS8+AE`q#gSvp^T=wi!(<1gk(*-%k*h0^tl8M8 z12Y`bc{qD3kD}9!9E?j(nKAqkU`AVb{bL9WZYppjbiuM1>;z9Mz0jVAZQ=t(q)a$V zQ9xK~6HA_psSFB@7+DC~WaRNY=eT7fq{F7OU}8%uKL?N^H1Pr7odARqk>rA)erAkC zP4<86!1+Oj7@sC@o@&Q|CWEH)bNnmf-vBcz3^?D+xD-w=H&1~?{7yMAM<~V!0nmD- z7bkbAHS#b>`o;DgjAr`&=Dr*kYD;q>Q{!7)tECPBNt6TNQn^JAT-lmZ}`EM1J!)=?2^$efGmmt&p=l1Ru(Lbv6Vra!p;!Z zxr-xMV=w}u#9yWzN*o1HaBk%F`Ad{c5J>Y}V+XF#p^jX^!ZE`PgQULC`Iyt0DW=5$`*PX7|x-??iB`x-dSMG5$1trBhttifR51<0`!fY<52SP&Jv>z zu~#sSG!Nt5%^2^FSruAW6}DMQh%`6JFF@^(5`rS-^9$g6^0sHOstd&2p?G&{2hIbF z;V}}z&0nIl#V(#JWQfW0WQEqqFOc}j^W~;k>VRQlHKk}An8u_4XBr^kRT*p+OE`l2 zg`_e-X%Qvrp?o3(Ev#VVFGA`$dvqodBMD8kfs!|kEL0=S*jeUCB+mFlYU|y9cyfI5 z@XptdpWc6P@3Y+~DwS%L@~RjeSEbNwlu8w(Z0wfVs@8zgt28p#Znam1k6BA1S1#Af zjq{3robI2iHS0~N=yy8TYPTv4q?%xAr(CO5Y7MMygxRi?R>g<8CW-larBo?nEndua zqq*8x?n>n=rDnNNLn`H`bJb?0+(<4~`a|m7H=cZX{N%fPpV48IORFl`T=R)qxx6YR z&tSF6^=hdCY=e=j)LU9PPlnM}Xaj&U`!taDwSg2^ZdMzsP8zJ=wY@5|xH)WXHk+j; za#2#EzUmsmbQf;6%MGCWg<-ZzRiK=xk!#oWL)l32KcwEf{@ux)FOGASlGe9^!}?XK zl@>riW~J6_CzHIQiV{}5wMtwBXz5{AEA4u6A$=oTtu>Ncy3lmZdNsN7vSHTCt6R&( z1MAglHJKu^Ev@as#&iB5mA>)C@xznj$9ZW*4(5t~k4X&Nm0dCrqRA@wu=x}7mQR9i zhOpK5H->rWbJ*Aoi+p0xc7|yM&Ik!jv)CpkIB(>-fgC|mt}6mwaUYlO$qRtxSHmnC z`GFIX#*gGrFm6W$77PL((bx_{%Td;Bq32vAXfDx}>n%A59M=W9gsa|-@@%CoG7?Xz z_jZtMr_t?9-M^PDSDWQz{vpaufN6wBYgq#lL1Yw3{hW+1Wv8mP8R+~22dvr(L{>0i)J(V<`H?sb(jlsWCxBVJu!By zp$5_2OoBE0%noK9zdIAIe`bdwkYNAunkrY*w}lgmXb`8`i;gR;y)x5-9 z!1`d|dp7Mq{WCy-6kGX`)9q@lJ2`{o)>OJf`Vdvo5Xm*zQQj1P^&OluMKQ9eAGpcJ z`46c#7TNhbOKMG~(yZEb;7ZW}C66*eGBv?{+C9HQ$(QC5|*Pk<0pT7bn?|xCE>ENV*$jLl_LxI4q%P_-P4@kfGA@%-wjh6p9(f=GLn!fqy^Uv?!y?^KH z$M>Hc|9%if&W@IH>nK@pkwp6MA)TJue9w-0uJdd@ixau_QL|L3RjNQUPa=vpPQHEk zsn{2YnP0put^CN~T zZgQ;ZCgTW+ktJ!_}D@y)lN zeSQD#f(Psrtk{Q(R1_&USIud;T$@b77hL|;X1QH20s3>cQU%h13{fp5cQZ9atqQoP z1kys0l$#EaEf!mNSVB`;OrT7V_ZVtTa5Xd3%H3)sFD==%N9V`NCPoGR3iQbA3!f`5 zBB;?=+H}rjgdD^5Z2;qumt9vPH62u3c@ZfIlL;y(dl3d0O^60CrnT%tsu`mw8>Vy} zAZu6oQ(a&budg$t_zovR(m6+ZmW@!D0_l>|n< zu?=-}+8!tK(Y*A0z1ju4AN1b=!0%ILdyj+MQ|@$_rt|LCOpeDo|$K5V;m95qOA!p4L&$Y^GOeLrMW> zQ*D{btO1NcYY%V&6hql4u&2hegIzu(u~eB6e+T^FiDBK#q8 zdvq(41v2Y!bv9*w66+@jD@bLgK>C&031kSBp2|8PgUqSI?jBN??n72B))^?{i(@M} znHMxf#%bP}K&;1fJWY>bD|d4DH5aL}^W9|5srq&xgu6SE0ok(}?4Nu6HiANWhb}mx|xpUr~s46KDhQ3Pec$V#)*?<|U?sCs{ ztO&ZK>AYuK;76qQLd)?dp)T#Xj3_hGB|g_}FQX|~45BPM89@#kQk`r_*VxKC-URZn zC>7?6Hg$CZeEle((0Q!6d6A3h4%bBh`Ha{IP(mPwNo=)9JPgNva&MBj6mQpwkYn1O z0+%W!G=%$Rg??A{7@|{$Dsgv}2*yeqWP%|;>&ivAA|qC=2`eF77*+v_^2`hi{nk=wAHa4aKhU?Xx+H3I-V+=l1^ott8R z81#u5KRNIJ;&oxN+AbutuoZD_IYJCTvVzaF3(XA7On{bhd)+wwd!XOyjjOrUorS%L z7iW<`#XoJiD(u549GL)#$)q)3#w&s_F0xJz$ZcE7Knpb0ZsWFO0Bi{Dh1Z4rtUJW7 z$AJ?9>0*0!IJQ;%!-brT`N$c`2nh43o(gs{btiyw(x4KN5MO8Z>bmGkMD@9z)qK*t zKEuuOLV%3*lKJjL&$ZetQ1|U;l1?#@5ri;G|)+x&EjA)#QjiDV`Pd1 z*fSdlcjbE0al!D}5E5J_o}tHpMSC45`Htjzw#s0@L`Cu^8MhbddD3;BgTmkJMOq^Y z)<)2(K(LkAJjm!vWzGQhl<)c0Bm}@5aRn!RFWLSenEiMI^Bb{7iYf(aLqUfv(~q>O zIUHBq9sz{RVivaqW=^m^Jq~xVr<%`L15E~TUZ1UPB=~tBsb@Ybxx!fO#%j%W3&@J0iA(K^rb7GK>;W;kbNKwWMupH2#OZv zJ(7Z&D#L904fBGhlr#H-NaELipO{y-ccBdttJRHA4A*rLVtQB3mLN8f zq!c8VO$Tg9v$c(+%BYr*V`MrWm!8f5BAZBYf?0Q%CYTK)fV7RQ-Wl}Oz?%uENn%*t z*$AxPo*j>&I@2tISJP?uV(|glj~J;!p@x(=17x?686ZH{w|R$}>|!A0YyhGk1qK=* z4RnkIZZBelIDrCrU2ihl$vYezCY|*W&J4R z6vi;P9lpusDsfP2HX{EM*2JV$D(pw(YYr)fL~meH;PvV}=D;68QFbzxrXGfx&a6;Q zoM$@&bsR+0huCj=J{jk*-9}7*7Q&KNZZ@Rd!eD|FXOFR$=hLw_Cl_`kVm%qVj%T@( zt^*-SKjaZdDmV{y2uPa~w-3fjdhX;x^Cez$(7Mg=;3!)GRDdO4-yO$oaU|4J<$6A} zEyyiMSC~G+b%B}*hHyemzuH3(7wPTfQHJS6_6VUgq@2S=PR>-TJC!3SGw9TYx(gcO z)bAh+K{yboWJ_rAV8#|dop_Fq@T)ZBSlKdXGU!4UHiV#6&G59=b*!Sfova8L=FsVa znmtri?1I;Z%~ZntLtHAg9+2mtI9aThOMa-T@C^>s`% z>&wO{S;O-4Ek#%So8MwAQ3A5+c?OxzV!3+6Ah7_R)-sp1>9`s$Y)dOs_R(G)T`dcm zZwae;``A7%P-v3L<27<4YiI*e8N(de63W*~Wo!gKw}mE~g2;Y0RhjkYM}Y2xmt%WP zypiY@k(O*}7R;}d7AqY%v2!+5?3D?D5*hg(*iRuEKo07JzUw}MFSrCNG@P$zHwvSA}>>^f+E5qVVx(;+9@_u)ww|iv@Py4+d;%YG+3j~G>4P3$|&@%Fc ztwxjeT`*k6^R*{Z}7wL!3!~D23YB1^@unQlFo%g zM{E7aRTbaZ>f#nsYgo{-v5aSj3F~F;_ ztEHQ>$b=aZE{hvP>!ras<#{JkWsW?|jlNHYU&nEs$bk%+RBoyp4tyT4Ui4&hdh$51 zLmSJ+Ol5VX3MaJp{GJ_~v=VT=?vBHmHSje69+RM_8en6o3R_t)Ydw{>LlmO;A@5fu zC2ambe6lpH;~~ScylKJ5Ef(MQpBgPN7a9=U-Y(TnJ)e zMw0-FFy6pqekyyWkp(BE$$&&^yJb~xct{v&5hG5tWuR(3Jkf4LTYDa9s{bM|vCx{yD^%<~~zqct}j*Z~$*#()e= ztlQWN$I`N`K=v?*j|~;oxDI7}q)iV5{ibqG;OM#Q%+NhQh{CBF4irSpg^>n9V@=lW zWR$GJN%maZMz#_J6(y7=(5u>Tayf!z!h3u|uomfO0i{r(sEiGX0pYk%T{e@-l?n%W7*G7R-KF}E!jI3XCA$elT77}Wko2GX8yWVN1t9QHgU%O=^-*E$n2 zD`vtc^Jb*7 z5#oO`G7-=mb|>=tHWc`e{az5`N(m2=f+;NAo)VKjR9st3A)G@LU{SE^SV~@c*9&{T zs!NAZ`t|&nG)SxpU^){*3^1!f`CrxUR|)@mWh9p2zRq5=ad=3&fC;Axs&*zWYVoQ4 zgZysFtFe;EydP)+q(`Y0%`jzXCz^zz7UN}ZvxugC;^7$CXf{sv6&ty@Vjgn&uA3KZ z7`~j1x(1M1sTo_ZYB12L*+B?-yG7&xMf@{bISCHm!`zIwIYIDtLX0%DBoKNt#ALN- zHd=#K!2(QaBR`2oK7=paBh^;(gadsRCV8W;PSXRS8qGZLNG2Qx^QwCO1)=u9Qa;sfbnUFDW-*+uBGMd^A$Rg zi^J-@sXW>&*f6I935DlhT?M+nS8jK{IvE?ShPhZ#rQVX7`SP`^HN>bpHiIW_&M zvnp7}2EoB7GwX399TQ1V;tq%GOf0)xQ@lA(Y6hyUjHYV2P>S-a7r;$pJ%p9&-A!D* zu{RCuFmkcH33I09*6=ipdD{6VR=^i!XMTVHNE7myw@dGY6_yJ2+A8@+J2ON|9V6%MtA(c*l zd-C-cgD4vR=ICfTofhXiUL>{dC>;3H@TgpEmXD@3s(Ssn_3LJ{bTrSUDL7#v8rTKT zo))lHcl7f@yK$s`)bmwIU#wW~r?)HZX1#V41vcvV__R`ORqFAZ1}aB&)VH=kb%LcfY)I za_8=o|h{k3wXR9=jf*4AHE zz2a@G=r5~X@yp8E%l=ZonwOQ1>W${=J=_2CzaO34|Kk3`J70q&x*UFqZ}5Y8(&y8O z7x@#_fo3h8Hc!{n>Fa%?POQz-^~Cy`-`LjQxSnEP^R8C(H?Ak!*Zf9}cANHk+I>B5 ze1U*JKK=Us!>=AceSG}&=TE;pIsQC>jOU|E#dv(FsD82f9yxaOkFf@F-Yk<;xs~8) z{h|E2W~?q}th}xnYs(p{uWQEAa>m;0m{Aj;tv}RX*EcRi?#GpfcOD!k5P%v&^WJ&p ziZ4|s(fF5zUzgVndOWf9L7B?BL7$NlIMjz_tLuh+PHO$oxiH^$ZQZ~J&hX+}>ucZY zcf=oUsbHz|wRgxGNmx3KSncQw7R!sYt|@&5`OGa0&OhkzVfj^Ot?ILe6I-A4o64)s zTGMCksMFBrJ*mFxyd{0!-u(FVnIF|&W#;$>=rfP#yf17-{Z&7>eqkg0PGPDM$a_cM z9G^UX^l-hVy?C}%W6PELg4^wXR9^G2x_nq!^RTvjSY7k5w0u}w^RT>pSYPvSamj!6 zoL8$?KYU#(Q{|{xYiJU+(kWlTTgs)@QKea~mmAuf{;_gZ>#6DMsb0l;h=){KrCP13 zucvl3>#5fD_55S~s&=iudW-Q_Msa@W^{+?wd65!p{WX;Mx)i&Y$+2dwA;;IavYNiK zH8lB}Ra}_VGWEwo44~@zeVc?tNBV{ZaAAd!Ny?QB_a~|${_lx6)C&!Pk-StWI z&*ndUdZp1=$dVhsTgkr5aN2qNZgJRu`}Lzc_nzLn^W^wixml`~o2^>2 zUYg4o<9)@Pu00tLYn?|()L&K}9zXfxqm!?m-hK4#!zU+yqJQ5zE-wF)*wo^$)qkm_ zUHkU&{Chtr#Qc2S$ORMp)>9dO;&Bvr6Q2L5_2mA8<3B%oxc-OBhl+2{BUVQb?dj7$ zsfd02`!|0;Iq5&LXRGO{`vj8_vudEr`=iZWydP^eQh7L z_>ub2uN4tL{rvvd$BS@MLGAu&=zD)H|1}o3#xLpgAKX7aIlf01wA3!p|5fKtc24d* zx&QEsr+2 zm!O>!3MwE}`j;Jwg6PmhQ}w{{hC=r#(X5j&5*?bC7?>D3G_T(i;bg2@j0b*E+!3y? zy1I+0)3f9E&fD*)VbLJ4?L$FAYcCY@vl5mo!%#S0{1uD8i-*!9pO{H7izgwO9LTuR zkdWhX;7d!D|6KlnSWxk|N8{hA%~*V``jL9=U=p{&ITXs{btne%%oeV5M$}F3e56{t z9E$s1+`8saz10cCbl|)5L7`gDNP6mvdiGSP9I;UB#{il@Wxv5y;d}HJF^6v)%;#61 zDb){D4RSh)co~sD^~AgcRs2=UFQl>Y1lDF#k|P#F7h^)EH8P2V*2Y1+g20S`z8b0VP- zqVy?#yj{Im63x%Zbw%G-4^GvYFUZTA3D;H~7ieo}F;gcR2-N_D_G=iEQaC?h+UYKR zhN_%i6i-xEHJ$mLXYOJAn$R9Og?V9Cp;{c%eyP%Q^evuc$EPzLICKax5%C4OB2az2gsfc?~tL=-AFD@TjEEeZm zp;nSzb_{~c)! z$MpY8+I&nq9Ve=*Z4YgGwAi)ygA_R_k_c3GgxYm=r|DmF5*i;0pS~dq7P+MHDXTN1 znL4CN`lG}3w|03C<_AkFrBm&xYb%L_;*~0Zz+pUY^;c@&h=r(29%sa>%ktaK?^Y7% zw@01d#mlH?<#E4wZ}^ck@~B@`Su$n`R-jS z*UQyLvs!DcCqk@os@7MpCq%6Aw$@j#Cq}Gs!;hPn2*9cetm3OhQ>^&s1#@i__C(dY zd|~AF+EDCqS3LBCa~DU!TpNjf5;)ID{OTh=_FiA~G$w?Xz>x*1)>|0rt2q4gt3^^{ zx_J2wab3_C-tgV4MO9-mdHD@?x8P~JcKGL4i><~K^YZLbd@dK>aG9WGy;?6VV3scj zEnD?=rMZ0gV$f1OJpbj3K}+>;;j#GD|NOZ_kL~(VrP^#XG;~cxUPbh=U7;718dW`v zCa)s+*lZDxu2)+c3aj#$tB5~V8%OO@Rd=Tu|Mc@m?A9D$t#@TXyEUg+>s?vQZp~5F z>?;e~y?mZm9p2Iy%1;e%Y3%2xhPO0k^;5%J8f*Kg4hE9$OuB(R6Ax_JRb4XUAKwx$ zN0I(O#x)7qFI-!OtH95iP*M8>97XzrKONqKXk4pLbA~B^&p9LMM07mU$r4lBIg(H zN@5w6TD{d;RzCavPYGq5p9>qz_~<;C@ui;EbL~GxNMo^&n?f3QKZ`$@i%wKuj!uvO ztf=TjW&R|W9G!?6?n|+Wn~PAwvDckD-hT0a|J(o50tpvvpt&!bRxH$tMQ0Lqr3e^m^DE_Eisml3jTbAvqkg(|ces|#+Jrw8GQ|#Cb?P=_k4u*E1 zJj2QXtV*gx?o33gs-Um})QqJ~Bs4DJ7amRJz*3PZCVHyZ+M#modZHVFbBoo9pvAe`-Efso6Nsw_4|HrRNJda`^F@-`891YGwI+&F|~gYO}h0zRp)$YA)Rw zhimh>O0C-R_n)lE=PH*-Uq1i9bqQVcg2wWBW9{?y`3suso|7ysEhlKLf5GzkC+)Qp zwA)K>^oA{HG?^_{FQ_qF+^mwUPuSw~6|ULhO1;utCZMj}<4UR8YE}~Vc=-jNtli^E zqrQB@$7?oOQPKGXYxcO(XttJF2OBrJ)mm=yS~?;dZaKl4O|H@*SI-Bmy$E!S=ij(y zn=AElvtHx&nHEt!zYFVVxm+rrf5Mt=uC{8Wb6Q?MwUOr!YSK6)e^7-1U(q1u}HK$0-VmbdO|MI{71L88Z+EKY&ZnyLd#qJdaW)|;i zYnpJQdzI^|lu9N2U01oTN~x?lOd5Z=qKH%Js9ve+@42FI6TPS0)c5X+5=ocmq`&J5 z^GGXoeNBz4rH7XOuI81#tD@8SUs_j36BpLhzEVEX)FZgvE4oiho^)aVu81co`qSK; z4S7ZUsHB$E&|h?A%!poeflFKwH)@uT$m(zC*%nvEky_=WN~zYURev6ibm5?`m?Kqm z$t$mjBUR|ME|7WmirCTeplUxCOH$dOI!n4DVqdbP+RvaSwJNlS&04eZGpb1!MDP`H zrdIu^Qm&P&wV#1GE#6hrMTop2<`iG93;OdFai(_rs8VZJFD~kexRY9xu2|HsRN17P zr57!^stsJetD(Q^D%m@|>%tyht-@K-k$hLh;g|2yE81K!hcBHQ!v7zj+;5C{oYj5X zd)r2qr>AEp>FL|_Ovh=v)k!Mu+$0kOaR+*OlIUhzvSqs^Iqsw{4oDVBNFcxfpk#V` zW@mTbyZ2#V=G&Kj-9KQzZ{O!X+5JbhP5}iLiIOamDCO&!{*s6Q>T>GTsZ;0jJ2$?k zc}}aPwX_$lce1mg{|etw`oVbM2TjEa2c4nS?D-vC@BR6I?jCII?(FRDyx876I)1VJ zVDoTu`*`>8#ooc@{oNN&4)^ZwJ>GoOEMIFi>5JrF<*Th`{&K6yUVqWLG03vfFI6?o z{4D4+^FcH_@lYL)!r1SoYB2QsI91~$=m(J>z8FVYJnjr|S2ayl(=^ZZnkS?F-#-=Xvw>Mc#frf z4qK|F8+>Pm+|D$IC#87}YsYtpEy<*`_}SUlz$h-?p(DRTo!<)BmCP(Nv-9Lj7Khsk zP9SsNF;O-{PMM+%xYOi1>&X;jEts6m_a;|{k=|zHicBHqXsYJ8wryUQAagZUZ<)4f zT%#bHS-k5IWIBJQ_hbq(ovh%@P6%X5F}8T-wMsH_Fw96YnfrlBGV{8WGFK-gbxg;; zZl!GIaF;32Tuaq8Q!~u#l4(VUCVv|BWXd$U31`%JnF7u8RNeH9+3Cp?YA`)sd2!@x zx?~4xe1|gW$vU+84y7Lwq{A7KUW2yVviJ^Vo)fG?o$v4(&WV;c?eNygqrJxuk6s+@ zKDzhf!Qt+`*BBV`w_8p6eya&zOvmJp0)9Z9fuH!D3@53owH(_hcHzEOSBqxeqPJhm z7%kT`bk<@mW3&vzF<6UwO|y0^)}m2Qi>}GW_|UAUMcrtXOJmhFZJV`d*VLj{PTQ%c z#g?hJ*dX1yrp;P>=+!hx&$HM@tcxXWr`C-K(>80}h>oFaTJ0OL2px-Q#_HOX?vW@7STEnCqb zBF!k9tm)t;BB)%QS~jC@X^v&qyBo=Z8|B$u&$+0zw33vyY(~R1TjeQM)48a%OouUL zP3nMSmnTkLGRQ7BPF>o7?XebX(gr+NV=dOC40ux&x)upU)KfO=Th`;vgWcEY{_qNE z_b}+3qWeiO#A*Jqf;7>4JAN_=(N6F(j`H`KzYw-}TD3%PKO*<987ej3w?*$A#7WTVQt+2RT5w(Te3?C_Cz@dkc`(Q`jc2gQ4Lg)4VIKEp{AC+O(_7J7uXLjRPF@qOVC zcKi^d2XS&n7QJ}!W8nfm^~WK)vyanpdIufGX8~L9HQ_q$oCp0ucIo0B;Dq!&!@9!7 zI~w7rLm2WPj*^(o=0oA~9Z$%rqV!x{-`4Oh1+!3Z6>Pu3}v8o;U z+{vdrHlU+oXcq&bl-u!)iH7u4S5BsaxZdViV z<@Ac794;pkz`Q1)Hg_{=e6gf5<=BF=whtE?B`Gg7jpJn{rAW{ZXYN6bua;GzY)^Qa z$DuNsxjr?1yQC83xq@Ak&|8;+XKr1MFPGG&Jkf0^BQGeOnaf(^%cW!`HP#8F487wi zBkqp(BlYR-;n9=FFZOl>H@)IntGP#hyg`mAt1$O|@|HID&}uO=>-t;s9~$L%>y_`w zwShgmsaHE>E{gS=M&+Slcw~&fG(jjEBwgrYmSxXVdiN z9@++TTYVy$rd7^FKy+piU%y!;p3dSL*58_M*72+hiD$Jujg9r~D)BUqv?A&;0=+d) zJzLYgi-_m7>=t|Y);#eH!*wsBoo>21W4rl@7$sc=q%&IV!z$_YmewlCQ&|z+HJpoR zrxQ)KT#r(?>R zj?MRC$?9c>&1^}%@Dbm}3r9jPuBz*rJHuzLERu1dy2jU@gkH#1*>Pu5Vsu%%#_=3} z*T|4FmZfTj<9OE~YRKYTkEqdkewHC>Ji^bmJF_-21dY{FHQn|Mk7sEavc_nsj;&cO z`O?NV^_khokTyDD0-I>6vc*jcas&8%7d8Ei3-BxYOnpHk!X*hhY zf-Xf=n`Cix&6(*)mRdM-SnINg=6L*EqzGzFC(4w=XE^F#OVJBWm0Z91JDdIo8KRnQ z9FC#a{&)%CMy30@63Axnz-C7vTWjo!BdXPrHmmvDuqAU=vsYrPBRDc@BjC)v*y;(+ zjhb@LtCwS|BM^i&PR!=J5{SYYyP9D_fjq3QEB^Ggx&onCQ(Eo(h1u!|1Y?a9(&UriVr^k*BC2}3 zmMWxXuh858{<3ZLq)92kjoa)<luOp9_zSZsPv8r#=cZJhbVe62rP6TbqHf9)_to3FiRCG! zl`Fg1ktXox@9j4`uZOcJhq6f8+z7{`5UX6gjJ2h$)YLS(#@Wv@Cx7GPp;J1aHCMKjsk?#dI?8C!WP zH8qWtvQ}p+tK@~1w=#>uvbR^=Hf79~y|vl|rx9MOZW}Hh%-&gb+i2VtTz%g#X5U(E z-%KOZR^2vS9Gkth+P0ZSy{)=yrZIBX)@rOUji_6B(@f*&wen$FMqXOm ztBD0JPV?qPs10TNJPN`&dK5&RI4n{xHr~@J?>&x_p&y|`jJx9^(c$h#)#i@k2!Av7 z(Se_2L8Q>c^5X4xtBoE}e1$v{sd(!>z4F#0f1Cu6?;m!jjTx2Kc85VYK@Wo{VmTgn z-#067L4LN6L$p2ahC%kKc=1E4IyOJ?2k5y!i7?vs$q-rU;N5ra>JV`hks=_|wu zIt=>#at76f+m9o(g_9^7_#vZSb-{N0v!II(;v@;uv?Q~j+kM>YC3u2%@WD7@vskMx z+HpKYPl3j4qSY1K9H)ep(B=?_LCkurt`}R_LrC&T*n8EbDw=x?`6(eI$otjBB82Or zM@_UHhj_@c+3tQ=U9Ra=$^OVOJBpV-sjk{)h+jhbwU4$E+&RTj5_J2;Dd*q)dkkeRf^Q4RRlVecF1q%c z=s8Y{bkB`@Z3JY+qg#1yk{<*eLOJKTCBYIXS33pa(++S=&V7O4z{4i05`C-zZ9wr-~EjBE$ zUZ^PrC*V(;jZJ+~I@z`w((_0eaOcs+=f;8!kz13B;lxr-xf5%H*SoINj%hmTFc*DF zhd?uEveMyCy^YW9MQ7MD)R3Nw%E|Ox+}Ly$YKNLF)P$Uzzy{pO zx$(KX@ceW&g|iispQromb8n$`z+$1E=7+BmO%LDot%Z5X@*s_m9?wcLJ)So{UwHT0 zmUz^&g3Sg?BvKL2Wo~1st z@NNM+g?b7a6Yx~2F>P!uwp*x?f^a6$sg&M~PPMVUpdVE5=Eh-jt!chgWCZ%?`{`ulEp+&?d(I1sQr$4T5FFG&X*3@CHVG7U>i+}R=xXPIW{trD&IdmkJ>*ggJY?%m=y^15aj|Jy@c!=O!`-7Bo1H9RGko)Jb`pPBEa{tXJRs*{Ot-<>-|X$& z$bp{Mx_f+9Y>mkdwrZG!WSo2U4P1sF*B#g~-`U!aA8kIq|BQ5UV>jv(wxk<&?W^1} z^%#ixbnocN&Tj5bWdjjU z*akx^e&f@rA9xWb6m3TS~=YC+D*8#-8ddl z2uwg_00RNlUDx!-yT{L;96o%p{p8u>zjt<<>_>8gD}ILmf(hPuc9f6$#|pLT ziQX)3h|i)R!(DVl7=UTqKX#Az4tBqK@>u*Kdk7cNAx?shuRg}-FTRYE)BOAIzO}c* zeOctw7te0b4TxQ754Pz{?I-u{?QQRo9k)k_#EAxf*x~3ok6l5(eFymZ#l5{pyEpO$ zP+%=ii0t{J`A6FS+FOJd4)%5rcXw!pze4_(qkFG?c({4I_xS#c%`bN8&M*FKHH#m} z7~Lkd;I4h|kdT59;YMy|Ui;{1`@!zcvq!sRWxqUXmVdRH@Hf~K#!=TV4(PxAIef~$ z1b+jCv|qOJ_I`A9ym?Gk@Q^6MWP~4)1Hu*62b_GoYz4IlUI5{huRNIY~Kbt zAH;~rs52a*AVMh_MP~qw`6%^!m<}z)9ipnCK|CZ~Z|2iDkCU*A0L?5JN6;1g2n&uf z{}iVv$jEzSh$xQoX-~&S{tOb*8JXRP9zcGG$la?)#z;OIB{;=VM(2fnpa40K`{Z9) zTzpC1C93uq=$p4nz9)3c`vKm=jF3s}59zO+m`KND(nL?rpd~W%M*e#Vy@LFN{z6 zKPAl}46v02pKn`P~M`q|@kerh1Km#4e(F7gDaYCj5 zqu3@>2mvIv8xqBicJ#*6XIqc<=(hdIkH|53kV_}Bv21aD*R>xWfBAIx#!)8;MnvY& zpZ?+5QxJzc>x`S~0!VhNy&3 z4Y`V*Y4dT5zPJEdJst5;i@vz>02ptJWp6hYMsn^>N-;EWX_ZLn(94q zd#$E=Pu%XTsc;RqpKlpghrd5an&^lexP{`Rw1gG|l5pnVNnv0+?yrTH*13`!%OUB{p{ToMu`(&*8<497H7|UTl}o0`_HO>3gl* z=EcPnuU&Rj$O=WTQdMG(?mTJDp0xZ60exi$Qga*2S}~A}+lH%#mvWo-rg_EK8YZnD z2SX-dHUvQxMcoHBJ^`hqSb zN^rHn76w@C>4er7w2D)Lt6kiqQ&vvPDPPT2*lO9WmR>CW_kMA?tY+L1{x=dt-0shUw+TfQ}qXq0>@9`dD1j z$QxZrhQdl;7uX5E&UcvwH;wOLl&OdNwIIzJnen-D*e_Ad$HO3XzmR{T$r(ku-%`v(` zw8fC=)MG@wSAIX%BgHGFD)fe)WG$()*Kmm0K^AjRmD$J^TqVLolWfqXQI&vDWl zhq=is1ydjyO*-;BxRdz340QwYM+Q3*3_iPlk`0tLwG_d)dm+Xd+Kz`q+F=aVs4xa& z6zRL~hyKe+F`!QV4%h@c5Kcu#ZW=8##1S3qc})MS9m7F32vP(Zbvg(}U<3?01JnT< zT5byTGcrJoNn7bS8Tn~S?O9-(f?uKK5P6uCQLL8$jfo`fB*HrL!ZzM zjEV3+>Lu}TIwSHzkWv#F#-}JAXDA{=4uitx2!C{fvmgv!fr1~UxwVVD-;Tp>I6?iG zj2aU%L^1S$27XF@i!f@_bEaYl`ypotbG7B{V7dcQL%CU6d7T}M$fN0?LYtLZ^5vq8MCCjq{hi!6cxLoHS*C4QH#qc!cmMofzZKiINVYg9GVR+U0ppy=v~i21jB_HEGBx*T z@jZ$~P0c+ze2-#zQ*)0l-=kRQ)ZC-T_b8S;_4de%A14rtpPFolRT<>;#4@NZGvcfu zx)ofH1E|zB-MxDJKc#l+?$zV|DfL-*ua=VGQw`YNtEF=GRA+YgYAKvO)w12aTIyy` z^>TNwma^GXO}?h8*_`h0?$uH?dwN^A_372Uh7aO%nidt48}#!yjL*vyjtd-!hyDpp z2k6ir`V(@8eHh2tAn`{dKMacRnnDL6GeSND-G%J{sV#8%N|T zjJD%=RKET*VVT^gIEq+_ElXG-7qSffF3!m1{=iQHKO)!f!#K@wbV_c(r)&-%2`lGr z4kzfIpFnEnz4&}u=JtJI@!X?q>?ax83gfuQ__?5RZk{`ObPgro*yzQTb06Up0%x8F z(P>y%eQ%xB=W%t5!kg`&ueANBuaH0DWTFfxlPD*INtE*$CTRrAxnbR+zTtdN3FR&& z$+=58t#DVPznnkTEednacgWlhA!SQ4(^qw-zYMV@z;^1Hq9+oejcfyBRnsX~8*p3R3AkUi$HXY-Y;$ak!6)LZivr^vIiY}9>KaRsNU_myKDkDy{ZdVoW6 zDP`&A0;I2pV+g8w>`x{>3vd*nM#TeiX-4<`5O>%$cH^BY@X&q$s8Nsw#y}GUus;t; zD-*OmjynwO;XMIrQ~~I7=o4AMhA+VV@h7Cqt-ue1Q`T4&^yeWy3nFwhPSYs{!+TY{ zAHa0-(a{J31sO5kt%Cgk02!brK_*wRD95XZdV=s#Q&==*z%SbRf`~dGdel=`RVCt9 zb$vmg5Rg0S30-!j&oBG>g2)FTcGQyqa6&k~SpRe}NB|Hy>M1ae(wQ>OqKE(>ZpuXd zWXf6VpDl(60K!Hc0_!Plj`h|*UknWZB#n9sys4x-Z>}$hpaXJ-0&w_B$}{i!XN!RV zfRv#?Lx3#7xciDxy{_k$=+Uo>>UCQNJJc?X>UFIgQ=rWM-;T+xBabzPB>%iWS)seM z?0{wSwO^!#?pFG`))ar+)Y#!EEydq%8D61&td`U?lCTY+K(kyM1`F{}A zS~%wNWPHB}?L7@58fbf`js0;JKmhTO{FL6wbKA#&F$C%9T}0M|eld=MGx8fv&Oc3? zXbXaBAtaXOm6LD!ID-5{(%Xn;@elKmWWST4^B^0bT&gz^CWk_Y98=*W4;pTw%@A_z z$=uqQNFo}ynZ$IqJBg)jL!fTAVLv-nW1hRLxv@r zcR`GgQ9qv(@T7Mi))LagFP8hTmY^PfdEJM#g!S+Y z;XbS-u!mh&)Dw6S^|7#w!%et=obcq9CRMy!gH(K z5ys(cCK*u@&{jeueQ`rqJ`hIUd_iuM06&9gTD>KVx7qhu1k5dAfX!Di`~E#)c+KOW z6Qlcn@~XfrQ`UrmGzT~&7eY2_Vf4%aQ5+^j$;1gcCF25iRk8Uk%$V;l70!~2 znzHE=#w3mVOGT~ZVy0~F72~PK@O9Gl1rJg)gi~RjiaT4 zPI4hLZ4nq;NoN_n@nETFlU&%8O=snPSO`<1)hgpKDwq;FQK48+NCi{EFsvyW#eC$4 z0fkE0!}-t;({sJ@;e70eWtjHG!5=PzUHRnZeDH^6dS*G;`REU#UOL5KzXbpc>Yei} z-XY$W5#^z^EL8`@K~Hw1hoQPNITgAuBhteo(=j}cuOq4ZG6FqlT}NHBxIlRQYZ-AK zT1(X})AeMBd1$KP@%BCag^Vx{6VO#Pho6j$I1dfbBKQD|Zv$?nx`$1PC2s{ASz;W< z{BAbomAn;fY1#E!XZN)!uj*}eb4v`unBU=MS6UE8Wt*Gw>T}QUb?MMQQ~hUE=pRm7 zS{3?-)2*x(Yshh$*s3MgIK6Gv5^J26w{m<$sS{c&7GC5u$dya1P4&$y*G0>X0LMg~ z_k*(TgrKO>gE1JxAsQhGN<{m2s-X>eY`_**tV>W0>j4c}I6fH{ue@80YS?YZ(ZByM zj2_3BmHD_MC|))Q!We2jB@vVrU?pLGUX5&c+T3rVtw~bO@tta1!vXF?o)ON<5D7sU zENTXc0qz8BCy)Gu)yAo#KoJ!TMZKJCW=+-?E!zNcK^@RlgIIy-s;z&vSm**33x!|3 z&DDWfu)+xy3F?5x9As?e7cFH#g@K9z&=4z8$r|fBiv=!FL7=9Td6eEP^Q`~lg24+! z5~!mSr9I`X_0JXyT%aO=D4>1FLz;5m`uzn97{K;d+c|$KCqCGaLdKtK-`(4}p?gl@ z(iQu|@6R85i~EzOU0mu?{z2|i&aDXam%lJKvH&ceL8*TCXJS9{(WICAir&508La7r zH+UW<>Sp0~-HXr;KEq)=8sdnW@qY5Bzf$MCuQ^$#9@m8Im^cF;8)1kv4LZnc0WJ|? zmChhYaRaq6xY@dS%Q@5%C&_c)E5O~6rm}Gofr}FyjaiU|7$xA%BaP3Qe{&S^$N0?e z`UvVjO#q6BLaL9`G{kA{iR=f2?}^F=Kt&2B${RqJlef>~WSBO=d(6g92M_`zxKI87 zpY6aOjd1R9ok1)obN&X>tH^&*w04GE6-SpsAKA-6fVW&vcig;u^6R0G=ni29H2MjH zKFWP8|Ms0XZ`$AVUXF)N#T>VL(Wq$~XYM!c6~VtfV~TDh0{>jA0&C%x!0`w~Xt4$a zen-QybgiU<$nR(xrM@P_NQXHvl&Fj5$lSBH1apMv%(r?=FhzJ?dQJs1q$DUE#G)x0 zx@I!>t_Um&dSDIQ5`aa)^MV4m|NO7t0$P+~sG4iqCJ#K+l>`>W`r=G5c30NCTorHy zO@r@C*1VuEr)9PHz9eB_(Y`El#pe5x1cAk9=LcnqjqOf}!k7QzZB>`Zu_0svi zB;j8bG~{{1lFA#TuNm|!S-er~WghoU7H=><p(_hea*~E5;7X&d2TPi`{)1it$?Emj#4L(iH#;WPu)Q346BE#D|d#~lhqAmg1-rl zS2vJ}{U$hL-9RSvo8X{z1DVKgg45OwWCFhlj$B7R)6AZ`j$Edx9=?uLrV*ULj(nz> zJ%%0mOf!2HJAq8FK*5157b{S3Dm#HptU$rhd`Is16jbGk=EdkJ+xy-94{+Tzhc{Rm9b0)c( z3ZrpGJhwg$VuAkqwQvw;D0KI0VIa=5SW^H!Z;Y%dfSxyB)`XDB89d$nS`ach6X`8` zfx>z|KEvoB2t$^Tr+gp`$%h(wMeWmjaZ*I|Hh&>(^cV_3q(0gnhavM-H-9E<_EFqt zNz=`Dg%SRn$qm-G@rA+=t5X zQe)1QE3WM<-?&PL+qlZZrG}krSzO!Ubf+s%mP!K2wJfele^xXoM@uDuP)&<#bwEXl zf_YJ*950m&lB--?+Z`r*ltC$blxIt2hUB^zPvAmWDLyBvlwU8EAVRe;zHP`Ro)K-z z;Zk`bxiU7hh9C=-^QkOU9xj(ALUl5(<^EK7Hh#Sn!%>}#Yq6aRTf)jr*fzdcn&o)5 z=;p#%@n?l|?#sv!Gc}d)kLPg4ZR2Yh(}7$F+i^IHukk|0ge+Avj27RGl%mg;NzsIKE;GUS zgN&3n!UhfxiE8{oM!p*){Iqnw7a7TJhU$5?!E=C&R5wr6O^+{wZTwyiq_JntaI=g| zH!>G}raG4XdpS7WGjq{jNJ-u#>tgY#Jo*b6tegt~J{9$a3|`K;Y$l^p1~bEKW|&#` zWiWH@r#1}f5Z!b>#thkHKy*7~I)={8=G(yNo!7*-OX%nlS`LSIiV9$MrLlWO4>0BNdqo*Ad(!w_pb<#K+OMbuW>*^4 zenmHsol?$OlvYhC=j>1R_G+WztX1}kqvC8^_ReaF?z|<;UNO_1vzyr~X1a4$HhaZP zcg`kfZ?Bf=&ROv6Ra4!kRy%veRCmrgxGL-cH>y=@LQE}>tD+uocFNUJ4@#@1wIVNY zHc)G4RoH{6#dKxdgQ3UweC^z1(LMUuT*&vRfAnE8d zV95P9cX=08K^l%R)RZ_L#KTd_W`6ghDon$681x8f z&~_Y#WabUDpEQfN?^K~0@=_bW&hvLm6#u)Z3fQoZd%Xlt(DMWUkBj#Nl_(zi*@=%H z;G~TcR!KutmP63o#+@pCcF+6>9s1;;hf220kGJt&l|cLc5T{W7HSOXO52H$--7b1S zAsSdqRif+%!(o63X{MCk5K!dN_!N_g4%72E$ry^oCsm3Z5uS|*m7y{%rjvlWtBiN6 zMLIU4J`Q~ZwK{qezedChs<)_FzJ-*C+d}Kxiy>Z6wMF&wO{NU^O}6o1u|ZL_MNMHF zDs8$Al~aBjZhX4f(5UXBwlM%`OgHMrgT;V2aR=JJ;`N+a?Z<$&BcH^sPdwof>2Thyii)-T96|y9MbxNya1YuiiuoP z5-Qh}^HQ#@Z!X3QRCQ5LK=MJ^-l^nU|EEP!T8QGJ;;u5_WaIkgVjMtK7ATlA43w0Y zp6lC-g0z6bqJ~7kl0};luuWv6_YKE#Us!xTql5#@& zisfcZ$8wlv_q`7;FE^umw!?y3zmG1dH)A-KL2p@WKPU>$>_D~c{L450?g(lLbg*(S zpk)V&pWl3rDNrr_?#;cO8=8@)MaZg4<9Ot%?zVUfm9)AvmA)>(;qi0LRZ8s`&oo*r zw*8IISAqH|SLRZrpC3`ApXV%whW_>!hV>&T8B+wHUsN*Y!PXNx(e;Z;#thmqrR4bx zT`s8-H6mpMfgdwpcg0{ofigjZF@PQcOi>m<;g%>yLq7vxA}CA~#chC2 zA|$}0#6QE`2I`R4+c2i?IPBx@YpR~;O9B=-j%DYtifhf+0T#hNpJP;pSmc$6MNaza z+{8`Wk5AlwQ;EIw9J?#lI)NkCD+`eb5Qpds3~Wb)I7DA4Y?;?oC4?VzO;tkpY1ULF zWQH&o)xZ>Gs062QT?~6H@E8dbN$w?nbc*gJK~|vZDR+el6FYts;E-&A?Ko^tlaRx9#*eSb0g z45hSsv6#Nv?Oe0SjhhwGe%*Gbe~CA70qU+*NX75IbHP=7%GGPXyzn}nzr+!CUfFux{oEDa&&lOn zQ#ALg_j9LdFLysbh*!U#|Mkyj?&sv%9i-968JfTqy$`6;!x*xiK89O$7JM@%Up5&t zp%4y;=_x_)%-n%M3%ABl3w0QzDcrNOnB1p3lMXowip6fgl{rE|5AM}N>_;i;24?^V zOfKj#x!99E9Kxl36aaWIxv2M(U^qc1W6}$`L-%n;!GnqBoMNaB3&_9yaip9R@tfC1 zjfXfskLVmiGQuW$5ai#q{mv;$Kysj75)VmdfBlG*18cBLF6lyK%*823{6O?vf=_oKV&@P$4D9|A2cF6$yct~rq z0`(?-0@YsIgl>@EM*g!;E+2iI+fTq}MSdD$Oi4u+J~^FCv6l0f281f`G@q?cHV4IV z2D;$1iS~NHM^Gh~tQCwXXCaK*<20a_8Zh&OVPV@DkJ64$I0OEG*NTCRvy2d{tBgme zi^=mOZRCY_Vb7wWKVfxuDbGULGo<9!8|-%mbOb&fLm&7B)-&&}BxxZB5G3u=xuA1@ za=8hnbrELt52y*|kc>0PCi4~;O*1sx%|kra%-309h?Ex#q|y#!Gdqmm_0IK|&4GC~ z7`|*O1Ko?wY*jmqW0i$7#C8~Xp{(~Rvcr%Url7 zVypldnZCe~n?=CL^aX<%AFjt(#8I{`V-dF&bs3Ae)vKwbEWh40m6YW_ zH-sQ4+y02Wy@QDklFm23E(Ah(((45sa*FN;Lvnvil3)sAu`WbJVIO=E0B+1-OzvF_ zib0GT^92#E0Xp_48#FG~UmY>|}P}zg-HcA(t9_+a1C~N|&%QboB^9E3CE_5qowd0aw)@iaR!X7 zWE^8RzE}#yA(u9M(_Ne?BeD~exINtni`btv*V8X92kW$Sk3F2XKbx9vU04nl$`qF% z3zc%Pj?GLeRUiP@@h%1efQRL1=Yas=VbSOHmwe4(a&41K(9{1Q!y;~|uI^n!xmVA2 zdF#HEDy*)p>Y6oMy|yi-3af6YM$2vKT(;)7QB+jGC1yvOg@2w^q!tvN73#v=zarrxbQdV!=c&8eczu8R!az*-Th|8v`f%%)GBnZ)V7?1lb-v0e+T>b;{ z6y1Y3)~H`bxYyzDqWe?@M7TCb?Dd@mt3|>VRR{TUDB;Xd z`sa&9CDT2sCi#*=s5A?Y{d~c)k+4CFkmN46=%-InTre6RFoSnG;Wzb)4=OTXeTF83=sPRq%i6F)56g`0fX?%!-D z!4R`Q$~a#d!E2g%yy;q7j_Xiwsg~yQ6|mpvNb()dcl9mA@9JJl^NK}VEq=FLmrI?8 zf1g+CY@!*X&R?42P!Pgf9B3DX@NRF>0bg3IQ>!}tG|$zQmZ9VciC1y@S+#BMxh@>h{zct z+sTEs9}eM}CU26e9FB|*s- zMHS9A_NnU(G7jPIG)pkfQWja8O-8g@XxbSM$q!JMw zvOyDyk)DFjY8(YHqDC=x+5prw?XyuNFlFjy>iAte1Q%T=83*9mf)T^}MdRcHT7|R? zezyin;~~xhP!0xhMot{^br<|bQJ+jcJ`M1UDY1#3!?*b#g!G*O9rpIS_Y&Y`gT zm^9{x;iRbG3UfZi69lFiM(|>$=^z+2iXVam+<4@ahwnP%Ttxji?xG&pZ@Rg&&Ch?z zus>ux0-i`Xtssh7iO6bU@;~s9r6~2!$@GS#OK^Tdm09X#qok#R7P3G6Q3mV6OuY?o z=pmR>NZaVgc{uh-+(un+in65|1pPrs-K-n@h_J0hU8o_(|Ar-CE@AXc)pZBBxjg8vgCKAnZYk) zjMp6InEmAKd1Jh$w{*Ki!L6Y3!c&HV`}Vv!o<1$O`|W?Z{$O!aH4NKvdE18>nhA6&>Ird#UjB7Gar) zkM5h#YOqYgN~?l%dQo7He4SO%sl2|kAYcMrW>t6wukS8qoXnS56`iiwC@zgpA;u&K?lnOy2w}_$#OA<}x zAA{YolMoq`p~9v&!#IeDgCL~8Yo66A41CjGXyQRiUJ20vha+g7Cgu~Opj?onp?``I zwR=H`JB_iQ?gDBT&Bz-m4IdApH0a_Yrap|5-Z(7s{!3#U`H}^-H_>Kp(z*{7=05U= zqklw4K}h;Xc@JcSVE%(w;&GOq21#xweMBho1@8Nlb_r&7J}{$|c2ss^qnuM}$;c@p*{Adj>x}B~m3#W3Z7%8JjC<$Kl(_2=kDk;g`5WK_!}KFC+9B zkBT7;;on`bzYn3*-TAJzS&;Qgz8#$i)Aj{ z5?D~-IkT|d5*Se7xxtN< zHI;bhq*G0muQ_>EQ{`(;0^V}3?$p@F2|2rta1ixy$SSM8e}Pvc9meQk0x5FEiyK0p z#(f+oeT*K(;~*s}5aFHZRe?vRd@S^A?1J+g7ci+?xWw+PG0%f?g_;%a(kW%d~??yWOjiIZq z=IVSuvWnf=d<~*TudM28zuPza*E{GSvISs_SBZ90q=V zm@7L8p7h2=t99U#)5TUrhw-AZ)pV`ZVLpH6pl4{t3&vK{#a2&%LW)02q4oO*Lg3#mKaKGy1L0CiC6S5!zJT|FPf)T&LxQ2~q>(0Rf#7!?`?TPeAw0O$^pgfFS(cY?FR z?wTdR83f&yX4WQpLLZPV(@BDM9)Ehr&p~H$02x5)foATMQvho$&M7h&590)e!ukd< zWpb%ZgKS)w^eKAMIgvj(+7bCX!VQ#SKj{n#j|dB;nM&5J`yLsv8^kjPN z=0&62OE@*dU$ZOL7ryxIoIu3TO{-fr%a=GEiv=B{<#el56`+(po<3D%OkI(qxH zz*=&bO>e&zR7>u9T2p8PccHB*w1K+<4+3e#eD}d1fCY&YRz9)=iIh+6_tDkg>i0$TWVWaoY8G=poV_`da zuRFM%_~@yhl(?JC_k?jV2Y%F@phtexXRQi@VRn)x+Dx*+I6+6`0x?bt!+P^wAzJGr zh$OzBu;}yVdqRlT;{lG*W2l2xye0-_h04PNbZ;DnxM%#Xp#ATB6yW&uw;m}79g6UhrNgwzx`2|#dA)GXnX;u%Qrrnf2$_K)UZ-;)6 zKm<~j4e)u&rXrmCzTb)4gwMjTe9slV_tYPU=r~Ri<{DA%co&dnGS~C1sJ6f-s>P_J zZ6!yq52C)}!|D}vxw!4pI9e(IifT1;Jpf7;Bx{u`^msXRUaH>AwcRiGrSjWl2dvKz zI44OL_OdeK_Of!cRPYv6dFHwp!v0sna{nt|Efuy!b)UJWyNTjck)fo#$WRWKLg%Hr z&|D885i?4Ex{Z#P1LdVU(OlEyF913zWs~x3DKuWH9L+TiMS2HHkCSK0@0Y^krFxQ( zWljniNS1dBW3U{kGgY1DIys54(&6Q1<8Vo?AxhL-+nC5&We`kTUYc9x=|1#d^1HwB z#Zuftm8y9@Br<(S_9w@Sjo&QGFS;|=dWRkZ%5X*)H;$KO8DpN}CvbESnOhzmjbnG& zw&y7`BFBxAmB-Eh2j@f4k9eHjeOq_iR+grkq$Jx(s_T~O)MA+cNy(06g9Jc=Ak)>U z=w@3f-20S*9IGT< zkdEB!{`J4?z1ZH{-QC+g-q|`lI^KDE}xy9m4<{$hSMt>UE&hMr>1vGw)}w z5B9&=I@19KL#ayuUj$6uC6|CYScvUoNSQMx{3V z+2@}smp)c1TBSbxxbe%&zf|OEWAtt2hl*C!weX`l+iO)0d#zmhSW~r-;-}TiW39*) zCH$yeCRwfN<6qWhKI&>E{HV`Pq@s?N;Fp&NtEi3e+ozYu%9p1R{_yf(8)J%}|8i!m zidr3g`^C&q6+NVPd7ymZl<;GH&d2dc8*}z!)a|b>UTi(xJASdZyZ`3J2iYTNCzpmT zCmief#5av7(+>KULt0+h?2!)j%01dL2fgFIk%qMOMptXQD%}` zQKuHImekg$5h~y5Nlkzy-2JM5Sx(Icjh*q~;i1&6uMO7r0#!C8E(e3UNWCX3TFjNj99e!PVs0 z@UkJUCd`J7D5=EQ@Uls+wiDyQId-{fB*uo9jdRtQW1JUAYHC(r+H={%8Q>SmyawLwT?I^eEDiU?6ZQpKG zlNe3ymV=5!bZR^5ZRZ!pe`?#6TXi*w7S(pDx7)u-Y^k=R-EOHSk*L~E?Ur4Qqe!(K z{kE(SM!B@@+il<^CZ_G&ZvP??G;RBK8#sx#Y1_Blzj1U<+q&iUg|R&CmYWxe{AoM& zTX!|yy~eF|!f1k@R$p$t*!z%9<=^k+TOUSFx%gjZJShlOw;jh0o87O> zG<)jOPDgx3ZTEcCNUY8@+HnG!FHLHBG#pw^n`!sLG`fUZ;vTWW-g33sR;1Hmy*+h_ z5e}@N&$RlQ5}VjIYv(xLnCj-7nSM=r)DvG1T-tWd!)CKH{f1b~xKeRv;D&SEoHN(2 zsppvn5f6RW8T7;MX6JfHdg2?qMcp&UX@$E#JK2}SCl;gnM@mMG^0RY&X&R1r#E#{R zkt{20_?!Cd-nWP??hxi0jN0THv;1~$u}Ko&qP{=k=r{9jaw1Ilp=+F|Q&9Fe%S$B6 z@*BTXuGLZC5|mH4P)N;*P;BmAZ#>)`C|``EMsla5W^?O$Lkb*%GT}Tf^`mmNxusrr zZdJML4<%1(P9$&h%gVJn;?6+1!$ni-PDFDvYFCgwUWPtJ*dDVWzFz3lxCVnJ9qGUH!VdUs20%M=C<w0?@H~{6-p_Y-1iI&;ey56>N^UvlMs3FOVsG*JP?i#cHWsmfv!L***_=jum znxdEOp&pYiLOr&zbG(}l|Efb~-C1ausH@<3ITk7@l3DcIW zm$A0YKHB@oqrC%mXgfUKf3dq=n*5QU>vp$Ux4y5+x>Bo-4iWd(j-KrueRHYp=&x_;)4?tLfM}_-+KA<4I{9$XLl^q z>jr;IhA!UggS~I|U%ffpd%b`7YIkpV7!JL^%O7JLLE1Mxr^UvkXqws>oqtz;H#uZq z$E)mdO5`o9(5eE)}1Ixe>wNt1tJC zzI}D@`{SKgZ(bf9{DJ*@cdr!wC9~n9zvlnNXSej`a5(0lN`e16ujR-?cw?JBZ7~mr z_|zNysD8BnV(*WyUe5nv_>sFHJ?b*WRDMa%j{o4eC&S<0`NjTjK}uN<~ZLRKiPl2w=`Tqi7)0`X3f7>D&GgE>+CPtX1>_pJJ{P@Qe~wg$%>=~ zhluR@!Pe3K%csX%-|Pi~HTt7m8vO+$q?d+5b?<;phfQRuHth7l;m)(Y-8awo*w*~v zur&UYT;hM`j;dp~$ViFb{SE(Vv=RR^mu}??dD!2_heumSY!~*Jwhc7xFaG=A{IA0| zFJ5dN{9)<0{V#((-rg6f=QH+aOE3Ce>Qa%g|8a9cZQnW%Sk6CMjP*WLtfk+U z`(LiSezX02e<#??pD>O6VknJEilWU=S;~HL^oQ4bONWMQ_WdCjWPkqVHJ5#+qBTWT z)mnWR$jU0rNL^u~#`Q6yx`x7AQ}od>{n2O2-0zK%fIt3h{_ktG@caDV*VOR4I`?~h{QJt> z?^+}LUY++{mB+u==6=`4-)r;dKki?jKYls_y zbLX#wM(ol2`72>$@6i?WuW8};`STCsm5+WsXZ}h>*5xsQxeK6%3;65*_~AURsoyTl zp^?iflgH@IDN&Lk#VA$P$hAa`Bs8ahP%TO*!5LMeGcwdfcc~toQzf}MDbd9M(0#dkR_%TuaO|3Xbu8NNS=aSisn@$DNWPL zH6dvV2vwtm5|X8WP&9ogQAwI&hJ++3;5sdui;|EWDYbG{R;tlC83{?!$Y4bDc}u2#pIg-5sCVb<$A#`(JvzJjYI1S2 zi+UdQ?tERj$k|2BTD?18WiE)`j1N+8&)1v_t`0 zxTu8ZDH zQsQR)m(SkDr~5_6bDevNNR4JB;8*`zSx}E*h_LbXqQ)2o6&nj1V;F{Pe6231$NG4T zl?C+}hB_NxR~OUcI5^t)T3b+$+PKHsqWEz-F+D1HnB z$@PWtV;Fj_FNz<>!Rq>=_;DQGt}l!q!+>~wVf+||(Cfcn2tP)H?)616= zQ8Hl`ox^j``hzezLnSj@bzfUn;o7vIcg(R=C#?f=zH-SG6-|U zNRn0ud5+6RXh(R8TPaXS$SmEt_NvvHl1hYlNU6X{J=k$5) zc&g&FNzztHtj}xpoVf0}xNSL3FI>*X+@YGDC+=8e&|(Wu#n;qgzRYNsOLLd=h>!o+Qs*;KRX1kpKB`$0Zl%;@&y+9*W-*eosF7U19EuKX+WV z+b@CyqmcUwa~FM=oD-XIdSnHq!rYa9X)P?)RP zpC}cn71b$H`=-;YsaH=g&`6R?TkL;PeI)%+y&<*w)pKl`9iEFiqVVU+4Jl}iD=9Ed z^b)A;J;~r^Pom5qNM{kN_xQUT z$`;Qix^nNh&6IjkoB8;=8;TcmysxCw8d=;3zd!!&rczbYuB2mgd(HnjvDc5kyQy4h zSI?)t(T#8nc z$0$!Tz*>4WJW$Oda%q}83d_tQa;X(r9%E-F&C44Rix z)y4o?m*>ephEQK~=Bd?3oAUWDFV9mSr?Jlg^kThU4d|t4^7xl?0KK#k$zOf{Lj!sR zms5K5YEk_z5g07Ftkx>EDIlzbK)ouJNN~JDl9&xU9x$I~s{Xz5T_SKx!@EN{x@-vn zE>)s8PnF{&0GFza4Kz6l?3M_$m!>jn!V3+eyORjC7iVg9YIl+W_X>nGdRJFzCV{%7 z$U2))y{1lK?SlM?;$j(SnM*-fxO9ZZL$ZJ!uXi3OU zuxe9Ck_n-eRiZbu>UW9A&2Y`8=p-REfxf9lbDXmO^ur^T&AJR#njE^LYH0kCA3yzm{BthMS|F6bG! zqvkmoTB6`lQ=UeGxNMi*jbw4zrhJYnO-#mS9giwa4w-Gn`=};|4|v%HNfwxG$`7g1 zmS@I9ntDcUy3_Nx1H|;B1lL-fh9zBB8+AEy2Aafx}y) zvrWQ`w?=211SM~a&K3z_-Wr{467a0u65uNm90 z4$9jhwWw`+;&Xa_I!b_eB+T}8MBH;*e8I7+?2!d}l$(P=$AJ-E46KezM>HSZp9NBT zPKmfnpYwb6u&Y@ZwdZ8u67jW5`(u#0!qq5hugxB{eQJe7XK~cNCdLUzWpf9k3&BV0 zvnXnNmU%&%)bEN1PK%l00Sz0AvuJ7|3f(i}+BC$WyFUx2_BHVwTiiA~9WU%`7EO(J zMlFkq&jzkH$_d%b&#v7r;Q(8#-L~sEC!-EGv$N~>=E4!TT7%{ooW;(Yu$o_+QFoCAD}+RcI6J0j zlqY8Gl*AX#1p06e6x|`|nT%P!HLY+TK0UlW6kXr~aU?08*D#Tyjq9SmardX}4dqO- zCUUm<)%C{1ou9JL1ySln1#$DM>kTMyeaa_HRHa^2R5yQjO+Yj5_mnM4q!zU%!o2y_ z_3+FBpQmhbahLiLao^m%CJG&Qc*b~U{V9d8`MYZ(nsJY({5DV-k}*~pn@_I?Xcl-o zWg03K$(>dzo7>k!qvIaW7#*)0Q9GNzyB@Mx;P9~dF~!8wwnD|UxqVG&Gw$pJ1L7)4 z>P}SG#?JM4De!g5t&wt*ni1u;@ekL;$K$?Ed2r!KWTO5yzPuhY1zt}1oUgMtQIH$k z*JUH-;ILV76)N?kDs*G(dYr`ln*bY+X9-ym9lLQ&M8m+nVOoT1Vac1;!W+BS#K+_Q zO__}Jv}8!ViJHCvN}i^15J9HMrL$0xgD80wT^*O##G>TYs#@rwe;6fCZD>QltxrOf zJf}xnBZ<3{nG2aWgvir}ziTA~k*CzkYD3lu`8hRnn^Wiq z|3(nze%t)=ipAkzXQDa$OQty800nL;SDizlojL43IK&(WRr9;1>zmXQ&$gXq-#-Lc zR-aTk&IK&1M*-930G8D!wRm#@%j%O#&qwnB+v=0*z+eBHog^_kW<*@?BnjFvBjsz}vFi*v#FC=zKl2t=za1!`GCgIZ zZ#u1W$#PDqC%MDbXD4d5uTnEY`tk9NN}L0&bR`*HNK2>5bZ=Ci8K|l&$%OwV0Z&cS z`|d`?nSmm^nw~RD=lDhinSpw{l1`iR^E8>i+^8Ni(0NyqAxs*ksWfj?jTz{{E2(rw zvNTPJ+^8Ni(2rMh-RKTxictGT1(|``yn5k!=TgsT4dpmW?ONrLimHt?^PKWXrCJ}C z>CIF*vgIC^2F_IfX^rammzS&ms?|m#EbyDD{;M{|(cI5xsvOxzbzI?hx%`iR98cwP z`CnC0!zcfK6%$%XqBub^$f{OEm{(FJ;S|;9N4lS451{J&^s%@ z)mCDaDKk=BZ6%g5KO@T3R$?jhGxA(bt}bOpsH?5y>QXLCcD0pQ#{7(US6hjt%+JVp zwUt=T{EVO{p2a@#d~Th^KJk-oo7fxis&1XwJMqD8o7fxi*lwHH8}aY7TPBQ0yusUM z_C|ci+h+DgJk3w9igjF0QhZH(*ECGRA@as3C-YKrcbVjP;xHx=CT9I+HWT!KoU=wf zVvpbGolOGWc1cTo%29nnoUzUOvw5JeDZfkKcFwIZ$1*?X&1KR$w@h0+Jfp06IQ%)8 zpoiq_oOL(dZcZlX;lQ%YQ*yb-+-xT33x_`;CqCnM;iE?hW}z@}j%Fv3ocVxIh_{y%7&y>g%iP z<-rBopxg_%RO*IYisi1!ilU8SZtoR8oV%u1YGD}oBjH1-YtXb&KCdmy+87?xAR;QJMi?K;sUFE8` zSpWK8BZ+d{Mh~O!JhP36-oP-Z=d}lx7-XY)XRgU@FHe2rDQMpCQPeJL+;pAso1=Zg zA8qv>2kk}=ulkN?@poGzFQ0n8%e2@`;@T&X#IbQ7eizn(Kf2n}5*%&;MH*^2^6RWx{(9$j6rDlmGochYwiipKTo;bJ;%JXUcu& z)r%K3X*|p1)Tqf`wmZ^eHQ_VIe_fRP@yuT^DDADw#ct{pzxQkE=F zcHGvx?%P4ztf+0H^3G{Erb8;aN3GWAEOPf3!B!0C+;^!*-P14z{?q8d)mnHGx_6&R zC=<*VuSW^A?Elpx{(!Rbg(m-zH~Qz)>My;n+j&1Iz1Q4#no%+<7xaDmSC@E9=Q%Fn zso~?FkD2O~6*VdUM%YT8VXKgb=CIW|Q*zr$)dPc4O7YAHV@OIx1|BJT%{AZCl5I$3Y=i;CKbl9iN zIk&#pact(NH(8gC?UnUfNv)TZdaa~XD&J52iT(WZKmF;A#}kW*b@Ih@Q~pf0yrBJpu#@nVHkYPZ|G}fq zo?Z&rf5c%+^8_Z?-_IX)_mt8|9WQ-Ag%3sVZ|{V1R3J<_n{M1svg?jTNAeakmVZvA zS4ZqEK5FhMaqa&llgX9T_%YwPI~Eb~(bk?E_Y0;i<|>#`Z_9zel*w3NjIqGPE%rJY zYx>wxr?WF!X~G}kBeEAe^K5~wqFU19*ZtGXtKaXv6quOBPC?*L*Tgl-srC3aI?zw^ zJhsh-5!ElT&GUhVil1iNd>^~M`A(pC;`$Q1OkKve;#w`XRdINB@?yuf=8lwdycDen z2=5lXp+1kF2-no{TkAXVJq?(8j33y@62!Cl$Ep3@jYsBQr8HLfv5S*l*GeOgDOP4z zKE4y$i1dRYk-$pC6M4XijNGhP&G2!C+nc|Svo~?;_V;lL=(HZW$>YAIf-*B?=) zmI9wImc-8k&2&FjXJ#;-6=4D-NAbk3$(x@m^S(kZugr~m7Pd8TkXCKN9S)qN;r>2c;t5S{U z$%nFG81HI?o)H45h_$m}+jju{^_CnoZbFS|b=deUcF6Z*nKiqiNSemXouq0qqD}7! ziiGgU!;j z&4bt-SUII5Bm4aTA2rms#N8=RzrgKki=#RlA=n;-9zSqTu~HR2~c2TTQ3V5axbu2 ztjUe4Rt2WByx!pO|J4Q(xQB$0%}7x+xgs}IKpIzzb-7*z(soAsRAXD!&>$v)x@4=b z%BrfVkkLkNVuyQ=%$-wLHMy=RNX%h*y)NtQ2QrdU!q!Tbb)_Z)sVux}Rp$6M{3V~* z#w%-#VH-?X7Nrw7!^oQ4&>B@tS`}GWxx_9m4+M%h=W58kGGu}TVz&T_4?NE?OyWlp z5kJsrq3KZHq(Doa-rA-u>Y`l>Eed&}1E8->({7okW@|t!)ANB0l+5Qob?soN#6qT* zvyi-&2PQAfGure_Kej3JaUfBHbauQWmxU4>Q$KdZmCo#`7I&`55MIuDKikQCCBcjzUV%pxIXL^7(%wqY6f{x`l zAS^62;xl>Juppj;k_jNo4owe8`owO>7o!gtHHg80L>I_mgjP~ut~#LL#d-}&CKj28 z7&}Qeek}A0rB6X?3Z1RukH3N%?>c3XY^KV>vMO_k@k{6WvBf`dvKHjj(pZ0mEdnH0 z(3oPym{CJI5|nsroE0kZ$K`CkQd8;(NwHo>(wQKOozCe9^Fd+1_;Q6TkZ{Nr3rf}! zIo2IAEFIY&f~md+WJWOe$M)TGWVeI*+xQtYfskFrs30M`O1#PvDm0#+Sg|ai_NU?k zDiScwRVa}fvLuj=5E>d_d4XD)e6^y+U$C&Q5Jf81>c}3)8*$bqUP-Q1bzQ4h<70=c zH^>D+NH!XJtrG9BW;d#)ZR*a&z@+K`P&`#2)Q&h?P-ZUV+ZNdoPOAg;T{#UhjeynD z7M6w&l)>cUW20Fp+?B#oxv?Z|pvpziuuTShd<_{mQXrYlYaH+cpa`izhUZk zXoy_J_mdB5n~WU@A)du(CiEXBMb7%dXe^74uzNH{)#$P!d{b$Z`c* zdQsJ&uu#6jyhJE8Tdl`?ifn+)JVBrec|}*__cS1^#5=`|a7Ae#1?dRXMgxFWXfXd1 zx=c~x4`W%Oftc&`rX)*pO;Rc~S(6b^m&_Phpn{>mtZ-PDiY6(lDgm3b@~|OuMw1#4 zZbABiqBYndK~)eH1BKTLXQu5>K#{KO?NNgoR-;au{a*9pq!o+tkd2SqWpf0Oe?oev zofAinm;VB35@UemDirLNV^bgtV>OCo^?_z$kJ>=aVCsJCEI7w^o!I^03SbDj2v`v) zAED@QNIfLgV_C#9N7A_gl1G_4MTRH34goeSQ|O?1Eo+|7HVtZfPXG}1cu0E?AAIE- zJn(=t2!ilRfwX8Z-Zt_DG-w~#junwW`5Y^lWMpz4z)>q=i^OM!uy(}|YIM}}0|;n> zgX{%3jk`tb{DIx{`oN5Z0ip_4q4a)nK#`1{6p{dx%6CnQ$lB`t#VuT_VEX2%2`tDR zYz{6lkc603vE;bG_!yfY44S%+Xfh}XT7Wdpir9A@fZJ9OHp)OMb|q&4)?^t|g{#Pd z-9ZMw!mJHql~(TqqfbLpCk#le&@ugtsCwk~GM(!|EQn-{<`AzY4eD;t&fgLFlb(n+ zb15QhtI#A;Yj3H+bPK{8DWQj`pwxXRAJ(ye%*~9@?;@EYV&6w*y9}yZwq*u#VJ;3V zA6dAzIsXjWuC4}sVu zBih)R;B&{Ny`lRH37hC*d?f&9NyaSsGvc=5@0IeTLlEr4WgyX)@xf!Jr^vu($1Ao- zn129yR%yY8glVnA6(DvxDVqGz8_dOW!9;e(ZmdE*D3C#!$rHPUD4^9G#ExqwwLvLNLn>%lzzhQx6S@R`oX7VlCg5rodFZPO_ zwOSw5VIhq95pp-uV-jI2N_vI1=;w6+h zg(QhMNCoMw;8G8|Axw)*%|Mey@SF-BGr`U&vMhdgfJDAU{!ld% zbU`2U0eNK^J^nJ`*LN3>_Q(M6!o`+nb-4URT_z2NA_pns4Pq1(p!nk?bYa;QcOx^`k zfD7!X;l#f#%#$1fv^zvKVevm~@jk+0xbc1~lT_r&NTfDJI6U8_(6K93@R_d+-Rj>5 zR1)U#gSnqzZXzO9AjPyudq|QqmSKk31ju25Q|1e%Kfv%73obDraRF->G6nwRd3=YA z(Gkx;>kFO_VmcQ6Ed$Qbz1Zmp|p2717jCG2uCNP0v7LX7`ZL`*J*0rU#n<084xuw0=zN zdO(&QCe^ilGt?&Sl*GNP@4BCl=vsM9ViP{ z=s?NkIf^8}LMjG26E2H3SQFDVp%^N*2ha>sd8=a%plef;wh;z_GJ>JXO)PBtED7X5AS`;zS zLJzC-u%LP5kOb!CoC|oqt3{@QP0IpYG%#%?@7V-7^{0^pvrNIGmWA08AG)$li)R#3 z$IR29E_9B9vK|Bnz`*{zX*Mm>>A0ldMJPWosO?kRHjxB_bb;T|AlYk}y~+qs=15BH zbg=!ingrp(B9oDh4_)I!^$|?VK=$ata9?-=B@o=fk{k-I(Irqh7WAS>ib>0RC>iXI z02Lgj^X#~Y9LO>0W8NcB9M!~%qmW@Oixeu9TwZq_k2W3W1TpIT`qV{E+sjPXK)YM8 zgZxt@ngx|CVikkw0>2%DY&PWDrE(4un9ZLMAXc6&Hf_&$;!O@aP&u*DCq6i&{w5`Y z?=cA-Bwm;*bZHL?8sEW!#!O2>!M$|ea(w92!I+tznt&`}_jgcLL0W*umdf>jjm-!Y zdG#nGf~+puHd5NiqaVFK(jhY(CUtCPwib1U&4@-95y*_l3_h|pSfY{(RhoKGrV?WT z%!ES-F34#D&ezN#3V@Fv;Pe#%Jgh_cLTPp~p^gb{);Vw`*hKs?p#)Vp3Zw%i_1Z{| zC=^9qE>OEUz~F)|pkz{kBIqz6>jnXS#IY1G6o4EB@&uQ$8cwl8kWF2NZpkybE>y&V zr2_aEbVQz=dXPfT4700<=bU#4WLTHQQH>zvB9A?;*Z2qz=fCRMOH*d4?j=njJEbc;Zn zi?NhwQyC(KkA*fFskFKEm*0hi7|%D+*vOMWSyoB_PSfk8C2Th{V3+Ke@3`J0E;BhU^s1(6)DU$A*5sBGBj)%}*+|5BJA?z{-I^RMOHi5Jt_zvvcmBZ9lpBPw(izjwr zCDC^g4R;T^+lLB-(}D#h#pRG{ASTUFE&)^wh-KYFUb4`GL^$tQP%RM8=0MU&m|uut zE@E|HU~UL>J;vA(+4f}-H4C)kxKIuu$9A!x+c@qOYO|99U<9b~aDu{qMFeA7~+<&!bdf;sCA~ew}UPuC1 z(ekO?MPSF6Ghv@25}h{<(t$vipk&?$#3ruM1yg#9ydiK=#`o?7a<+0vuw)sU%d+4? z$plylH1s2bv?}Z>)lNs@XLPe#J?SRxzN;pMsewK^o1+=!=g7LO%oW{o28S`1h z1lD933haP>6qcqdV($g9bD4oar$LF*P&(8y6gpXj(34lz+f3*1EnQSu*Rtq3$dxa@ zMMO#kL`)}wH+p#&>kiB_Nf%D3MW8!r;T*c*E|~2$^tMl0>_Nr&%ffpM5d$#+7_0{b zzm&=ifUFfJg^kn<^e zP$g~(@{6%Di1fNenvM&;4^bFGlp>)PWFio3 zb9-UP)))vi#u(r$Z4kHwq5a?=us8N#@+*3b18F^TF^N(ha{ga=bawpJ7fw23wu~;Re%DV%hAiS52SFc zdA?(uP^8QB!vW7UEL6ulgenr8c7&_o`(e^=?6GSmv&OyLUB6<$T z?su?rGvWh)kW%1#(%Ah8CK`Eb0Ey&^h)GZ;2LU-2S|)fsjAbN2OOhWL&;e`(N-=!U zrtEwOLLuG-A-n+UT)3Q*_|idEwmvgEaxQ!V2~X4`T-tlxN5r7Huu2zJ&KTm;JiQ{|vG0D(JUaRX&& zIp+XXK&rnWd4hrkIiJN->UtRb#QN#*;bPEvM=EjFojygAGw))UH>tHgPY&up^~vk5 zO)b;&pro2Lt{Fgg1$zqwss?f)JniLz1-=r=&g^~gg=0usQo;bbRS|6ixyc7JtOV>W z9W3{DwFp0zyqfP52s&c*flE&vYXF{3#06%>VudT1f?)jDL;%KkIzE!+4CUFUn5f5J zndS|bw6VffC^rqNkrODABbLcwVZcS@bb{$6E8zVAbc-vr5G*bXh!{hI6-?}bF%OAx zW`sWEIIek2-cb}7-{@j+V$co13YI7XW;fr#$^j77mb-v1wifuN0Cc>?5K`|Tt&_@A*zI9{ee9 zoZApaG8PuX&_CB*W}Qu@1>(oIW?VQ{$MG&=TgI!4Y_au_*gk@b&?08DYqo$}^)wVi zN{KxT`&xh$S(( z8(NXwreOXmW$lhMZ?FHw zT7CDozx#UG{rv~|*YOkXui$9zz_oxakupP_1U&v~TUa{4m=Oqxcuq8-3TpTz7fkI`5AvC(P(-<)h^ zt-C`F_EFr(cJi@{uu$OD&hknRx=|90o_`8HHb_J<8n8@f*`Q5dDKIWUYG+Ltuw1=U zFo53aAKiUlKbEY3u-4>OkKm=2YiaDfqKSy?wO|LuvdzJ#9+P?XlCqQryquTnQnutX zywHHR3&I?{-W+WmYg5SfpNBPAYkpL#4fx9QUX(^kvT5zto^_oRMxcocGSdilUp7Ae zd9J$JT2rII4h~{HvS*WZb4+&SEZUNl&+PZ{DVXKp%qVxh4CIbMI`%zUV*bx$yLQuD zsh-WC|60}c=h+pLMmL{6z5_SR_9wam3-06;jPH!uD?e^T36`aoSho(H#IYVK5*39b z8D{2NdEpLxYmf~_7xvTy`+hP4wAXx|yykUbL|$$5RtFY`+>Ddpr< zi%`A+&l6^S>g(0$usV-GcZ1Ef|HR9p~ zv2M^k*DBH_TwC-r(o*%1g-x$njvR?E!S&-a&b#%(6*B7Jv67* zk{Xi#oxay1k|puqkP(ek-+-WvR6WS6hmsW9vQoB{B;`UsoOb@);uxxFpJb8AJK1#8 zRVw=x#J5lE3PJ^8=8xNbK*H--8?vidGB)`h3MG7>iVY+)LDEPt zGZFTr-V;jl1tEF+8Jnv|QInLW7u6_18jfU8Ur!ZU>A-Y+_!X?V3&<_Vlbf|ZvfO0{R+bmaDf>{LIdaGPM;o!M9pBRH1Z#+%gapO5S^^9+V9a$aC%ck;HJ+PX|m3R>4Q; zh)8tA1}6$4+2T>w$Vs-BfF~BZ!+0d9~~#a#<7q$yvpYTooB|nNO?l$1wx`_^AWC6O8l;8|CxSIERKE%FjcR z@(ED-*E&8q!#%?qqo3LZgRrLaM6DGH*AkPXwB#{L^F%1MgnwMHHx?rhL=;=q2SiK;wQG;g?wWxZM8rW;{jLiNs1S&~s7>{n} zGI%8LIbt)*U?5Aa2Elq!W)aH%xJZR0B3X-$58FmyQ!F!`$EfQjIMba`aPVG|*?11S zea4z?MK2y0LOhaW##nbCwG#+g5yUL$+1Dy+8 z`MGFa&u5Wq09^6noOQ;dY52!So<%6*H<8Fu z`kXBs%@_al;VEkseW^bFv@AU4&XP8a5{Ov?4QsfQi1{!33fI8Imp<8?uDS^>zTES7622K$|;D7P{rqn=n6(m?3W{FK1i zB1K{l=Fmdo6Lz!Y(E+7dZjMwyAoj3vnGF)lsupc8ggj!6m+V==e=Y!s8TJ5tdty(3)fk z;~;5SMH6utL8lOs^&~3;!AdQGF8Wb}+(uTrkM2QrFJ9MzIE2gt@?b7RvPoEKU6cZm zMA;E0bxLRSk;{6>*5>AI5!t1wHy6JbIC=bxuyX8G}-UHQ;vgqgsO|Y?fX*LNrN` z7LS?LtSw-c%MV$D`+$%eK`gC{M!au5w3-y}S+JL`6yYRrpNrrT2P-!Ov0AZOq3i`% zaejz!mh5~ZP#hA_Fj_U%m&MXn0n2N$oDP}BWULR@ykmD0Ywrk0Fpoa!CfqlRv+7-B z|AO3ZM9!xl61QQN(2@qL%#oQ#2^LtCW9dWK)V~Aq%OYH(N~|bpTO2E9vLJCD_5iUs zi&anZ2s086&sldU;=~M2fsYU*J$rpWd_Z6<(u5a81t}#2aji*VYKE8+mkzVDK_sE` zL^HE6i!`Ad@>z4GUAv@1B6IzJ44058Suh-O5Vr+Ofg>D^7m^8n@h5#{%KP#1{=ND9 zKEG6NALae}o5dv)kmQ>2LdBz+HXw^^J=SYQQt24QPkavs;m@AvZ%`Vc5ELWu9Qi$7 zVn|2#9tg}>ga(kQS)5KsR%v-zUfuzYp7KX<4MN1OtgAx0>XSn5Q1f-n+E%=@uBB)v zxrUrq0N(F7>dBcp3rT?i8DTGNR&pTVHIH%c#U@AXf2mu%XowLM!UMD}QHkmX*6y;{ zRYH~lgynMJ8xr%Qo&$tmN6!{94#)V~&84P(xP?vM- zPXQ4MFO{p6MBTKo8M;8`!~;ys`xT6;F6fag*A?whKUzrUz>i7>BQgbpNA?}^4VGKi zdj&HZD5C^}mU#sDLKc}l=ZcZRB;^WuIZ-7D@`$BxqKoVI#2I>XO|J9frq2nWj}aCY zk`=bP3iYUGeQ&fzDKg}!N0K;*GgNnB#G`&FNxlj(UL&!Nc!x40b}ta)_qAgqLWWZ30<$eL|Bu0)P@O4y-fEkYD`z#f3S@8YZ+9m0Ab z$x}py6hw#?hY_PNkK6_`sC}NvKu|DnTGwKnhSu-P7R2twjw0hwPE_0vVoO3Y;#^$c ztD!ZGYPyg0P8Lm<-xSlxw9`oow@50nLV`*mIbN8?SyUpDifE%y%R!@e2#=3u7KzHN zv?d6QxP)9rU8h>qaO5qG=&2|Zi$j>#9jysHVG!w-A?XQh5h@oz%x7X05+w0M#8h<( zwssJqJW3F@gtVN25m65r*$R#`L72@nPHPcXNL4`;?8IL>4CH`HbqrOB>e}&unm}NK zE-8}x@fJOjm50*8^^YU+k5bkMV(P_;Wv7!C(jA3#+aoU-Wir|pwIBH*X@+A{!Fqda zNQ{g)a+esD89D(YjU<#_+a0-LO^DmwHtsIzx)3>_(Q0jf%>#C9VpY5CW?fzr4n|B~NdikdtqdPrp zRC~2D_Wj#diKq8?pkwx{J8MU}>avh`BL6(f%e9g2KUXrhckHifISfXM?ykE;tK;ZT zYqhdNf2mD*Wq)fuoQJDk=c29E8bXQQH>#KY{tkt(@ygYk>-X=nn@ggf_0K=t>CsVX z^jCNG-}kp~-M_c*F1vzw7Jp-8uIs(hm2RJw(|#S*QWjUGuzzDe=gY6(*5>qmPycPN zn)=&#DEsrJTWv#M=&@9dQIg&0_`m;7J1@R8q7u#JPMe~0yfIHZ=Ic(n5Eq5c_d2ci zaTjh?YWu5RU)VRWKfUkwyRea#=r6XqwijOLR$i=)wmgF=rmBbQZ()qv~T&%F0rlj{O?z* z+|I-0xVbW-Q2O1~mw~rd%&y5n***U4ceih@uRq+px&CH6Eg!yEj{U5IpZ$wQnEU;} zd^wD+%tI^9)y=2t_g9T-)rCECtnD_x`319qD*GjL9G*5Q0E9Ba~)hMj@-s=59 zm-W(3&rf8P*>C!`C%&#SwxhDYaKm{eoCUt^WLt{n_V(MoF6@UjW6SO5&wefM9YdeF zwS&2_>+|+@SBc?P^|xPdHGI2$a|`eK`p;K-b-Qx5^Y*rNItq4S%KlcbZ>?L?+cRVD zZw)$s16-5We_M!F$@;b&?IF?L{=xg)y3_O4j;1yEw}Zaz98Z_ZC4JAkMhwGkW#{nL z-R1V%+38-@4nQ~7y}Na9-Br5)+R{F|`2BF(uj%bC4!c92_*Zr#KkD07Z-qUjqSrII^YDEY-)^u66Kx!o=+WjCaLa_BFlJ*q1ZAUyxO z$3~Rii>i8sMzyL5P z36r)1)B|2pM6aXDL=^m#eF|ammU|5o^eL>V>KGLfj#%|iTmNOcQOEA6X*jW6ZDiLG zQ{>0)xR`w4QkB;K6$RN+#C#(gANJEPh6vofWS>AFDSu>WPY}{(mZ@Q{2?1syLWyKQ z86k&iY8`lT5G3{mNs9{uqD^-L?(t=;AGAZnt$rrW-t{7bqV`E7aMM?{+-HNBtqYc4 zLnmU))W8G~wHX++oHd&>R!)c?O?riB_=B^pJ+6Bu#_;SPpu5F-k8Qky_>oJLzG#3mCI!G!iH( z(w&9GM}DM9IdPVC!Z0O@qhx@4WobH%thU3(8$_n>pSh^remOEZ%dvKLjEvI>Yk6~B z6bL8;+?in6NgEAAE`M}%s8Y6w+cJRTeVp#w`iaqIRn&B=SO?ED1hIU^{F@hK_+AqC z@_XVpyhLvHOBAd@K7CRSe~$YmS-*3tb9)dHpFA7;!61%C?;oI`=%s7XFe+C@JD90f z4TzA`$?>qAdpNaeDv06}-k-pAZzB~z6fr_R<>N>rwp4bSG9s!M4K8d+pKq2ALUiNSv8{snsL@(MWjP7YR%$zeL=f9 z3Plq~Y;y+7J@^E9Na!U4#A-E9AWL}g%H(@xI~FT^k3y=Vy#q`Y5He=r4)s zc!}&Y^5l-P$NihJQ<5YN&N?FpD8%qAf${E1gi2qtY+s&3DVTKFF;c2owzaMo`7t>^*Nubv3PvnueR?9TqCwy5Dj&Y z#Ny(enr=6@%hay-2eM=DrW*hk@m)K82rbe(Em}nHg&5nCJ?xL=Yg)wAfQXi;!aag} zu}zCq5g8jrB8H0bu%T9`utUrS?kRnmK9jdYH;M|*L-|`2K1L{D_2ZH$F|cmi>Eh!e+z=1o{O-Mao6B-FZJg}@*Sgu5aRX|w2{Cpoo$SEopJ?1AEZyMsMA-TOf&y2rVp&eS^Y|E zQTlxm)`+uxCE3pLk+L`Asw8T^Zg3wQMhrzX`341Ws_zkNG!h~TVlg7#9#TII;t^pu zA$hA8=||Lo(Tyu)=@5rH)2^H`rN)DClfZKy(FrjoV~1@-bIDY;;;}f8hVGu9jbS(- zgY}~@(*gJbX}5amQ}ZLrsv*8NL~9qT6gw4N}(^ zK{(&oX=>Loi|UDt8cJF*)<{3@5Q`=|ui_QTH^4j%()Jw)`pk<)*47DX9)2tkde#p~ zlsL2Y8NFI^uX2#rj_YzY?K~$o_`e2(-hPym&<+a=(I!>|&nJ@r#einXAP~XZ5hHke zMt10PHW~=7h0whxNIep3`yj=)qCmvo?4%0UNT9tR9uT4pqqsWocnW^Iy(K5KM-;hn$0R)gTZW)A5bAMcwS|a@STcv?^N9R*R?}mwJuxQRBuybe3jz{C zAzej>MH!&TGDvonA`Ymv(4#>uH<|0aMn=&2PSaiCAAZ}AZu?+(7#^XAbIXX z0!{oxGYXpHL_NYBA^KFq7-gaoK@SsEpt89GULkZG%ac?@_l!AA=z+;-V1*Uk*p-*Xzs~a zz}|vH*Qk(bASwY7UoeL*s);YbAbtVEIQX$P1Xt)SzY^e+jn9CJ2P=)nazNM{l7kN- zM3?2d|Hw2PM~uoV*{jo_Q^p`3&$ySQbj!H4AZeDr>{Z{Zj)>R$Ka5TT!YLUr?Ot?# z*;B$nC4pnA3k1Ys@t+|SCFES1vtZdv>xWj3z;t>rqv@S}6__{_Z+H-~O&OA~$nK9$ zBku~yBeSaeQEKE?Ns#nm%fvc}h#4_LusK*UV4~7U2{_hBfzSrxkg?V<973O?#2d@P zFv@ANJiK_Q4worT)t0fcK=2|}Yezjl%!VAJd?0ea_9-O$P>}og(uY=oPWHv?rt9iW zy{qm&PAkk>@4sJPynjDk_l}zp5m3Zc&zhL-Sbt2*6kKD2bk-RYr`@th**5EYCtZXR ztcZe#5#_ z97BiG*z!Ws#Ttau-?xa{RX@$F5ViUAc^J1+986=|%W~QG3|s8DlTq3n^}eyJ!{Krh zUbGg31|}_`7L1?jK{1t#NU=#*x3>flGH|yLDFLpw$qe_VLQ#n5u)84tr#0lgObmTWQrAG zWsCHr60=2RHGjEP))R#i5oBq+a)J$t`f0Nmo*OJNmdQOR3bOsPo z;HOG9D|GfupUc)q?x&S!vv$mDMsbJ6b;UQ7*h><&38vq&-6GiuvqSh?63 zHqO$q!fSamguDqjW)ntnl`VCxVwD+jtmHZw=KKt>$gANe?q3|M{v=r zRoB=6;{IXR0jt=Fo40@*IXES}m8WD-&_{%Cz^%OK-Ss2^K71`N$?Z`H(k63&3ppfO zg~2&Pgyam7>_JG_vV%mdG$GVsC=Mh{=%saO1S-Zw4a>!Mtc>NQO;^>3VJ+R`{1K9# zEH^JgFaYLEA?6CCpqLb@Y*5k8&RAC=;5ZnKAd=G>X--_Z9A#azeAn+N2OLv%Lfjhu zd98#4eKRQ_ME&I>L2m0;A?;a)<40``naWj%5LM5KKl+ThEYDGb@>!%2NG=|05>GvY zS`{QmHH&*eW(v02GhlOnFwPgMqCseKGNYhT4H8{56js{n(Gcr!5?^%4+hZiHUN|Dq zAnJ5SbgEr34OUM&DM+e*7Olza`$#*@cPsp0tTb66N|!eWGs=>GNOu{v>UqmMZ6N;C zTiSzbWV#hz#H6>Qk|3>uXhf_iscQR?=s@{VPbeI{Sa-+?i#y~CIr0=aEoeslp;Wdl z>oJbkGDp~{N=%1lPp~7x6@mQ7W0v2K7%@(2Z7k~f2og1*t~)<%ozepF?aw+njDit1 zKV!(ate5$S-gAbaTMUo~8(mlu-?0Lsnt9eySPbfTaT;TYST%E)6G-!nVCGx&TyNPA zqSlW{xr}LN`NVEmPozZ557uuWA^}jeesbg%ArRS3yGrzoOG&o6LDu$|AqLJ>ZRfS_ zDzunGq>PpAE9$8C$-0FdDqpa6u$a~HU!u6-8HyWD*g_#hPp6Xt@ph%Ce8Z^cA*%|* z79iYdVhN#j_Gj*L;hVTQ&gC^+=FB$TRftc8-=tqEq%mxm3)7_~_SBsovjF{Lp zg7~OjAuEl*&TDN*AL9WVNsr?}PB-JYAw#NUjM3tNt_8oq+U{QGS!TTCZ{&hPhN4a-|>( z;}88{#(Z!D7_7T3+94>OwNq`bgh9Ad>vkiwb~-kU01jmskdhQm+0US4^$Bd61irj= zb`X?N+)T!yd$7&NgCt1Y=5=(8m($7}i`U&}z1^%nj(S%DijU4rJ8dqfBJzX!1IH0s z#>hxQNHhZVmyH;ZvxxdbB0>1N=8VNI&1Jz7+%fBYn`NCE7y;cZ>-X3Iq`F#<1&Zy` z(e#>N(EYPQI`SBFageB-k?PrDLa>8)jp~p=rb=t`SBQ!lNh;ipg+k4fe%v7O*I76c zdV^Ow&%d-Vl7}aqK+S@F+@tk-(A3wS)PX@~72ICTs~i++`bxK=W{-{7;6<#Y8v!Wu zQ&r6_jo4 z1m=xLOu;bTB2d;m!(1tL$<-1i)=(s#ti%c-=c8I4O5om4>>;2@w8%J1YM~ltuZxi! ze5hpued#;`R1j|ep;knsF2#cNS$K!?G;0)x93_xE^9XI$AVkI5RF7?e5(RN-2sq~m zQG3oJG$zOF-13)|S}C^m{rC=nUydT=ShBsktmR;jOhIO3>9mVkJRiimB>73FW(#33 zv1V>E+_xQ(mHl4jIH?qIZBGu^eaf;y^N^%ugG94gL;iw;>5?KY z<5Gd$akIpUJCI`~>pRJmFZ|^oO3ie!tbdF)JCl$M$uc>L7*>?UGt3i2PRdwROpO(h zx5FlbXv=`w$l58ilo$#(9s3@9!XoOBW_K*dV!CJ=cf)(ov*bmf?!HL7aaE&Eumrlr z2r4wEVrkakATY==$sr>XM}aS4i)ddvq;G`~J^?ZvP7w~d6KdG&0-|i=hV%^iai>YH z$P1@c9BJ2iTB{x**Z>F4;`H<6CFh?zjj4fQ`TCjEdgWXSZe`IZLuF)Z=a>OcmCkC-1Z6034 znEJww*tn#HGS26}|8QcjnMtI{0tm|YlC1z|&jP|(Mb1#sP-uhd2p$f%>n9ufYD6QD z^F6WXYlh;-8Q1tYEK?dOCna5nsEcOjMt3fC0&@u{r3}B95t=fFa|i&M#Wmx5i|t{6 zdc7oLmZd9z#I{7|Xy4#7s4WZm<4Qb|959O`r9tGqXR0$gXY{Ve3}xppYthgm1q175 zx@;M$6nb$-;p0DQAq&1%@Izg@hA`c(wBC@@<-BQRThP(Y+FaEvu+1Fh6RK{{mxCHc z*6hIKo1pNs0nH*AKkLhB@MvyU^g{Uz&9($y%%97!Xq_+gDga^*Qv!Okctu&P6(uVH{ zB@&=QAXJw-T$i$+89g$FJwJ5_dnRM1@~R&MlEz-!I39YlVTqdD+qFg>4LY;^cz8sT zJ+P39*Ym-|2)(3+07kfp(DFmcuagK+3c^z3Jl-N3>?d-IN>X-fm#l??d8Y9L^Lyfk zi99eMi`g(Rj&(>1#CFN#QoOTPsJ<7-F$p4uwF!~tIMWRXA5k_<=dgFf8q!jHP#& zG!Y@?2}uTcX)hgR7g;69f1bz81TfDSf&>WUgzkkFp~08B9w`(&?hYI6pe2`R20Vc^ z<2mZ;KFXe&-@v(Yg5bieVJ$HNbt%tMm9ty~Q@e=9`j8GFda=}$Ecl-##x6U=Hj1j% zloTk)YM!muBVzV4N+KLh@JjbzSd*9wG!oTDsIGOsmBzYC*7atbVZ^ryA{$ka%eIG$ z=IyMYWlgG{hYZta$q4ns5@C=F_ky7GpRscfh?XrfV?$pvjgxYA51hzBgc2$&uL_|< zP$-zutFmCWS&0#;uIU5Xb`PC*9@%$`)HwY%vZWjeb45_*FMf8p)28UGv)x}J`u!m5 zJ0%TMAXP1sjSh(lH7iVe$RA*Hmo%b9lyh+_YO=JDXr3RU(>Cd?bwVZp1=2N^S|iewCk zIcMA8b)jUbD{EvBm6I2n0b#V`9a_C!)K_H6vOyUJ*$kHbTI9^RKVd=ul)W7o{sU6H zC%qjH)Uak_cO+TmEoF^kP6s&3QO(3lD%miS1&vs^f>0AitV33Hld(q-b}xZy=3b;b zgis%l{)|y8VsKd?f^;|SvdSx1h8PA!^`5OnKPaT9kmatz&KA`=XT7W?%KQ0|CfJ*K z#8h4)L--Zx-O_z$EV6vCWiff_3};4Dp~~z zSo~;1rofLn^ib501zH+8tGa#v$#MHX_tAo8c%0o`+ix3JdS_-x>Jn#@baVBHb0H<$ z*+yE<@a`a+Di&i4Ye|7EXX6xwa5!^Fjy0T7=0ZwD9(s9b(S7MtfncBe2kb*z6ll@^ zzaSOwSxh0zM|5N9E7Zg ztSYK5^#SP<74HO54cSLjTMI%u0L)3Q07$67tRb=kEh3QF%qun`%qt2zFFx7p1;79Z z+G%Jbn-H`y=|Jbo_tw(2;Fz9-wh8S+R6?_lSRdQB5J$n+aY+jtc4CMO52=sX0F@4H z1nH<=k8eTQ>kWv*)kH}RTyMmsk^{uFD;*zELe=b3$Q;`Quq63w79Dt|Pnf7c+JlJG zGNKkM2;^Lmk=I%&`t0y2_aNd$0X@uW6jN9TfM;bFL45^4L~%|@iIReDd6^;=F z45&Xuyn~Gx!vf;oRAM8#AeL;4_;!eFRf|H%@(VH_FUKMh<7z~f)KQ-xe37AWq~ub! z6ZsGfM?4OM>;#43CIlvi5Yml$eSjgN;v<+Yk8G2@8Q|S=3iT0gl$b-Jb`L=jG9;Rc zO}PwV4(Ws-{*7TiJ3xH7LJ6S#IR%T8RTFYIEK&x}E2nf3vo9(XK`o3UCK8_uahd_~ zzzSrXvXDK@GvMlAF^&V-SH(L5XOyCHGv`|U_avxdMg7TC( zwd+B2*rM3OM9uG$5ukTqUJ*6;G7`My3J}{I8X#Ul8&DU47#M%CP$rK^q=eUBs_xRD-cz_f)l`}qT7;cM@983pb%R1wp zZ90(5tBf|1p!k*b`fj&(FtWxGNdju)ks*! z;IoI={v{dN^si5b+$wHWW9 zGVhR{AxfTAnVG`M!BB*PWtpMosNxCpmHc;;v|vd2v58a+M$|$=#W>(avZWe5!Fa-+ z$K{Gt3eKD%6l3KIQ1X4y@i5^N*H1{;r%+vUNwddLr^FHj?nsCzT8Oo92&##_mJC(# zv0@(*hi4HrgnHVbe0rSyagI+2;++^+7$cnZAa%-Ny*s{dLoF|{I9_7;Q-IK-dgdVB zP$+QuCas5*n0+ihgQTrw_Fb%3A+NF^3CZi0ytw}XAzgv&G?JYbG#Wy#K`1%&r0=GB{wQ$HMV!iKZ4BicqOw6Px4Tb&z)OidU9Gi9#iG+I7p~g5&ZV^L*&GJB!1DDdV zAtNcEs?#=$kRQEYxs zi3UhfXT!B<-0KYu^k{xy5vYxr@9VFHHgR2o6&3{<%N;|Vx>#rorJ=!r6`)R6VnlkY zRp$~a;eabkk{lxQ1*Ivjfm{$&f>2Dof=GvfoKh8ett#*+vCu>g90}?ncr7BG`+{a; z4LHKOGf_~iPKwRtInZ>f8iXEFPlZ~41O$?zT8GTdA+B9ePizx&2LojOI!>b^nkQ

+GBSIW zg@zY4!VwcYn?gfhSfPL`Zw#pO)1s0Zo*{$wSj@8Svxo|u234S5A!wd82aNeKZj8BU z1F66#QKPM33!v(mT)ZjC5rPr57+diJxMIY-9A*)G8>zbr#n2u=6YQS(uzHa(9m>q@+ep_`Sa5_y?Ff+g5 z(zxjvEdC=ncOx+So(D#T8_}fa8tSJOln@C*1T_L$Rhgqny$3Jxf)<$k5{i9FhKQ_+ z3N%6k-i1(P@c6R2WEhI@2{d6LK$of^cWVg6s27nlK;M@6#ani$(91)220`UV#4SMR z8Ej6I10=@5hWHsy4GA)QuwgsQBKpuxZ-VBX0EAM3dxeWsWa>nrNhoy~90LL=$~H8? z>=3A{KPy97Svk6_Y@kqdnXX`0Px>t0M0yyomfARJ4;2ToD?!dsVTZ~-k;9L4rdSWv&pWSj!L=&R&8AlA&P*w`7=xxx@+&vp@+7u7Jj zyi1it$N=Ovs5NPLh4LsWGBYb_L(hipXN4phx&(IT%!~#L_@P!*ww7SWuV}M!q6u+Qc#jU9tKt?h)(+h&ln7v1Ad;l zIE5cj4G1A7A&rilh+J(A)pbN;9F?L>9_brkJ z$YQspW=jLx4KQg^Lx4^KFVtn2eF(}h;xngVha&iZ1E(FbpVndgp%20_AtV%(! z59tEL-U?|Z#YPa2o9X2dl51Bj=(8lU2vErugA;55LC|OqntKrBW5k|9h^)whJgi=& zIvph2kcY;?ppS?Vv7E5lrw){!K`cCW8&OuVk&Mt%kubnxB4yHI&$C8IV2j;(r3Bc0 z=!m$9(5L2<;tmMtqpjfJBQQNTfRf+}bg!WRT@{Yy-LuGqkED&f0d#Q;=BU&@)X9KF z_(jh$JtXz93{3^(WMtw=z~UVvw*K8egFLz>F z@O-2tncpW3kI7>nR0$HbLPpX~aWbcjR7NHMr*nn=+`QpOOXgPH(Ue+)DM$zvps49T0uw0;bjD+C#`1G9k- zdX&_$3w%+_YZ%INXLZs-x0(HJW#s! z_&*;1+v9&_{{>!ESpVVg9{(ruqTdT|Ebs2M?;mdOeOWfjdU>9%q_bzrl;MRh-_9i%k0Yt4gF?Wzga7DHTNIh7MJSfs!=~GUp0ea zX0tJ9M8~ZyZ`}I!}Hn_3&W!lgd;D+h4W!507@X58D~G%lc-y%656k z-nPPCxbn>miYdE0nb7y$EjK38^#&n>v#s*|e85Gny^Ss&`1a zTQxhKCMg@VR<0Qn%C)_RbJo;0%R2iyESGOiz9y`Z{_No1?(NRngC2_N9G^|a_ zs#@%S^^U=I!w$AJ)#t0imj0~fOdj;hTTK39EnrN_9DaRh?9GI{Z0HTPgGQyc)f<=e zn!$vqR;|6Dq8X`*R(@L1HaLp5Ny2b*D#rD5O0<6vf8P4^BiCcGnlO`{?A^O6`<426 zTTOnI=+@ikb!+qUgPn(q73{6Kg0=T=-`%}`2h_2;Y1A6JUX6>N<(5TTv}INrwQ7fS zb*51CA`LS1X7fS~(q|g9{pI$(?c4XZzr1sF>rVUDubHeIv7H|2nR>m;m{hG>5o%J1wKrZPfKzgR;Ho&5{ zX{fF~ue`07QeNpb<0W;seqML$m+B;qm(X3~)zIDU0^Mz0sJq6G&|UM#=x*y7-KCeV zyZTR{yZR;S?o`6Hr*${(k^)hLV@q8*RjUh_iT z#?`czs-}8;2v1|#Ua!}gtLfm3ke9=<#{W|+J7taWl32F-N?_R}NV^OyTYE-{F2b@G zL)r7*P=}T1gO`W0`gwP#r|MbjIX%-;^^8GT;~96Drf07blMNwaP<;(xG`CN<7pixu7$zcmsRGgT`F2zhYsxmx2{Z4yr zmM&bqFwUL+PM(u13AyrGWok&o*Uwpu^(R0*euaykm^^8D%~(uhBs7)9G&2;nEZ0)D zJ6}z%r%}yP-n}vKC^=!l(tBz9TT95+rks-W@H8|A&xJQ!M!hF*&E6}G>cfo^&ii)( z-gA~Uo?Djx$ajI`&t&dUaH z9{HIJa`v8V`C=8H{WgEKw(zZ{2asKv;~f2o7Ymaj8X|9Rs>Yl$jYU_qS=w+h>6Gwi z8h7oPuwA_Q8N07+7{DeHK9;KIi5D~VdtpV`-KnLfJJYwfrp@v+Jcen25!ARAOKPOH z8Sr?#Ryy~#>4Ko8%XgykR~G}}X4tm_HRuz^QC%-oJ5&w5W9r&tY5_0MNDP6R>iB92 zSuKGGxuqGr);gYb2CSqek61}vY3_;TnTu5h_JDGEt_aVslVaKm!lL!!Nt;b@OB>1G z{w#G%b7Nadn3r8nVi1Pyx}NtZ#dga{tg}wto)do@Eq|v^{6*?c=P#!vhv&v!V7%Qe zKB)<-W--`?Q8p@@y0K~SJ>ABf0oRgOdNYMzr8)U}LcX4e!|F$edgErfc~ft0vDoh$ z?%I|4!>Y-{3Mp&rb8;ggH=e8=o?n}9-Y;!!CMO~ba6jdCPw-_&k-LC5QP%og>#nTX zTpsSPq?Kju6U&hZ2n;)u1UKV_jydlk}6p2X!tV51mpZ4@C8FuXXQ4Ir(L~% z!JDL!U+^A_c_Y5@R4PQ9q(b--^kN<(?ut%x(n(-ADJ}jO)3lVEBtK>_HWd-f^{kM* zY(7OTug=M(gj|{#RGE(|Bmh%LSF5QKt8^)*3ubl;hv?n0J9QTTWr# z1-vfIm{%C2dG#6X^T=Q+Wi@F|{%md_9*5aN_NUDsQA{53ddi9yARToco+*_{XqqgBDJwA`F{<#VN z$?WwvQ_b1u$*F!mliFKwO@_y}>rga(JuN<()1+vBC0Pxy!fm*p5fIm3@e|jVeLCL@ z`77tM;2I}d`H|5OgBNc+m0Z^hOiW+gg{C^1>EX$7vklMKEVk9(3#-dXJz(pidI0ay zTXR$R%VYx|q*%_6=j59S`Q{w$uFN}tw60AZix1is8)>QF^Zz`}s|YuarY~_cBu>N+ zpx#;hDk}Db#nI&V!t1XYQ=agF@t=Sv#{UVPh;Kbxr_xTMj$@mDI(N@(fp}+IzXgv? z@ZRkD)owZ{Y{PBcfR_Ob4C;#(W7#LqNx`#;As3BW{jYv>7dn6IpUG))NAk*_rMcMT zNof#!{+uQchX3V{o}%H2)@S^zdVX44t*w%UwN=hGnyZ7fCJP>;@*kmpFzXjw4HmU2 z^G`Ql;*(C7sjW^jyzDFNKUwIL_|Lg)f449vTWky`{O0j?a<4Y9uNO@2YB39OxFi*4 z`+t#gPkM!GVdfHI@TH}lnMcXSEdrlGEdpG1o3Ha1$vPjU9?Y*2vR}=%|0&c5oF-U* z{kE`dRCL48kK)db-+uPb|MG|U=b8TnA!Ur#c$|$_-EZ4e6nEBb-O_gJ=mzV@U;;Ka zO}TRHd?Gf@P2AQriG%HQD-w#F#E#SE)3MXkLP!%lu!)DFNeFo2G1(-fiFd>wfOz3$ zFHA^CJn@8tgai`Yn_M?HY18dW=fU@!dp>{X-0RDE%c3ai6r>4?nz1yqqMKU7)~fkk zxaWa%%+jgent$T)PTzjd9}UyApY~^IOSf7T)zK&)6VB_73F?Mv6f3~DrM79?PpP>) zmEx5FyoxX8{q5Uk) z(m`ZbtL~JxOH?UmT4ftFG_}AQz;*(mNS?}-3h305EABt!?85K@wFwkUrj7<9@Y6tFED_^OUmEk*!KwKUVx z+J2hiI+_e1th1>Qbrp=p@K zwhh`rvk}aL666~7oVq!{>gyoDPN?9R}jUF3u z&?;}DFPF-f>F2GjjjA_a>j+fYlu7c3k3N)YI3o?&pyc%Hd_ zNMekaWj>yjh4)EwXCYHuSrK9amym^hPf$a?3R$D|gjtu8@FSB_h+tmoMJgfwijd%y z&9tlM1R}`Y&-xiiuZ)x=Bl8l<5utG!+4e(id?L;eK9T!=Vt)yxGr7#i<1oY+Co9=B zFUinvLhn~Z?R;jiSu5@WmbUZEE>NS687N}8FL8N#wNnHoK{|FCGGv8QGBOd+zSg>- z<(vvAShc9b0NrmEj53ys;YvW%qG%IxR!GK|F>?meYX?Gzt#J|;gW_ZqUz7H7IkwTR z11%i(*L0wR z_HNh&k(#y@L`@W8w?h#OMx^e_bE52fhYwss_Kz9{{dIvt6RbofxYTL}77cGBc%OvZ z$#{_1K?o8hVYP=;ypOJcB{Wif;Y!%vj)IyKZPa(+P}Nn#YylT(3y{1B0GVWxlQ!}0 zBBb*}l~vYwWn-wgZh8t)rqQedyAi2JG}y9qz$vw{+!{g?ZVl8*8A-sLdE`6fWHS4G zG>Nx}UOV%gDBM0;Ap8k6^tKJBOTfv+u#tO0d#GlC*MR1TJLJdf{Fi zcRyj7i&VdpR{4~~%Y${UoG0)(OiAE2)Sz6GHZ1Z z4jATE6!`r0mb2rAejn27M~#Xh_(d^X;6tH5aYkU25V<%xz7Y-=kGdWE>pazmYk&L? zRTQN=xk>6giPJr3L0AenfRbq}fST{+gFQM3W9}@4Cv^V~5K^n7caEDnwGiE5+Q103 zj7l5$3?^C&SXd5C$63-2Q6_g{qn64Nh&+eO3`Ln>h%mS?t_J(rK*M@&xT!XzL2pkX zCF8M9zziXZ7-?r*y~N{aMG}*QbBHbTG1z{}W)}AOI?Ncsp2Y)1fL@K>rVC@=Wn}Y> zmtqO2*}fLHXkDxQ;(IL)wN6CrUtGp;4XptG5 z`5`yZfyMrFfm@Zh^ctN1d~6*CiU5^$^&-rk8!3|_63X4z4{L|2x&i%a7tClCj)h9Q z&bqI#3e}&?`_c6uDT5e2-s_1E20s+fxtmngtW~KapIwLlT%4eY<$j>+;|EB8!%0I! zlD%)OtPnppN0}MtrI75sH<9|&^rQPfcb32Je({s{s-hfFeCX?SoL-1Eu+O7Gj~n@~ z7ftKYkDqtqm%4j%-pg063?A`^kEDXutoN?%Q{~6A-*&=(cK2pq(|cwjm~#ZsQ&>wS|@~foMTF_VgLXF zH2~xS5oBR>VLC!XMrmwiE@^ILWq1N-daIUZ3